Open codelipenghui opened 5 years ago
@codelipenghui thank you for reporting this.
I think there are two problems behind this issue.
1) why the disk usage keeps growing? 2) why the four entries are not available?
I think the disk usage kept growing because replication worker keeps trying to replicate fragments but fail at the last 4 entries. when replication worker keeps doing that, it is writing entries again and again. so the disk usage is going up. this can be resolved when we fix #1066
for the second problem, it is an interesting problem. my guess is it is related to ledger metadata update during closing. but I have to take a deeper look before coming to any conclusions.
would like to put it into 4.11
Faced this issue in our 3-node setup, but it is always the same 1 node that faced this problem.
Disk grows uncontrollably and the node died out of storage. Had to bookieformat
every time.
But after some time it will happen again.
Pulsar 2.10.1
How can I fix the issue?
This is what found in logs
Bookie
2022-10-20T09:53:23,089+0000 [db-storage-cleanup-10-1] INFO org.apache.bookkeeper.bookie.storage.ldb.EntryLocationIndex - Deleting indexes for ledgers: [501062, 501893, 501923, 501378, 501922, 501921, 500143, 500142, 501901, 500141, 501898, 501896, 501911, 501047, 501366, 501909, 501908, 501055, 501918, 501054, 501916, 501913, 501912]
2022-10-20T09:53:23,754+0000 [db-storage-cleanup-10-1] WARN org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage - Failed to cleanup db indexes
java.io.IOException: Error in RocksDB compact
at org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB.compact(KeyValueStorageRocksDB.java:295) ~[org.apache.bookkeeper-bookkeeper-server-4.14.4.jar:4.14.4]
at org.apache.bookkeeper.bookie.storage.ldb.EntryLocationIndex.removeOffsetFromDeletedLedgers(EntryLocationIndex.java:256) ~[org.apache.bookkeeper-bookkeeper-server-4.14.4.jar:4.14.4]
at org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage.lambda$checkpoint$7(SingleDirectoryDbLedgerStorage.java:638) ~[org.apache.bookkeeper-bookkeeper-server-4.14.4.jar:4.14.4]
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [?:1.8.0_342]
at java.util.concurrent.FutureTask.run(FutureTask.java:266) [?:1.8.0_342]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) [?:1.8.0_342]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) [?:1.8.0_342]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_342]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_342]
at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.74.Final.jar:4.1.74.Final]
at java.lang.Thread.run(Thread.java:750) [?:1.8.0_342]
Caused by: org.rocksdb.RocksDBException: block checksum mismatch: expected 459167422, got 2336252423 in data/bookkeeper/ledgers/current/locations/000423.sst offset 612461 size 32254
at org.rocksdb.RocksDB.compactRange(Native Method) ~[org.rocksdb-rocksdbjni-6.10.2.jar:?]
at org.rocksdb.RocksDB.compactRange(RocksDB.java:3167) ~[org.rocksdb-rocksdbjni-6.10.2.jar:?]
at org.rocksdb.RocksDB.compactRange(RocksDB.java:3135) ~[org.rocksdb-rocksdbjni-6.10.2.jar:?]
at org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB.compact(KeyValueStorageRocksDB.java:293) ~[org.apache.bookkeeper-bookkeeper-server-4.14.4.jar:4.14.4]
... 10 more
2022-10-20T09:54:23,026+0000 [SyncThread-7-1] ERROR org.apache.bookkeeper.bookie.SyncThread - Exception flushing ledgers
java.io.IOException: Failed to flush RocksDB batch
at org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB$RocksDBBatch.flush(KeyValueStorageRocksDB.java:451) ~[org.apache.bookkeeper-bookkeeper-server-4.14.4.jar:4.14.4]
at org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage.checkpoint(SingleDirectoryDbLedgerStorage.java:621) ~[org.apache.bookkeeper-bookkeeper-server-4.14.4.jar:4.14.4]
at org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage.checkpoint(DbLedgerStorage.java:231) ~[org.apache.bookkeeper-bookkeeper-server-4.14.4.jar:4.14.4]
at org.apache.bookkeeper.bookie.SyncThread.checkpoint(SyncThread.java:152) ~[org.apache.bookkeeper-bookkeeper-server-4.14.4.jar:4.14.4]
at org.apache.bookkeeper.bookie.SyncThread.lambda$doCheckpoint$0(SyncThread.java:99) ~[org.apache.bookkeeper-bookkeeper-server-4.14.4.jar:4.14.4]
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [?:1.8.0_342]
at java.util.concurrent.FutureTask.run(FutureTask.java:266) [?:1.8.0_342]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) [?:1.8.0_342]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) [?:1.8.0_342]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_342]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_342]
at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.74.Final.jar:4.1.74.Final]
at java.lang.Thread.run(Thread.java:750) [?:1.8.0_342]
Caused by: org.rocksdb.RocksDBException: block checksum mismatch: expected 459167422, got 2336252423 in data/bookkeeper/ledgers/current/locations/000423.sst offset 612461 size 32254
at org.rocksdb.RocksDB.write0(Native Method) ~[org.rocksdb-rocksdbjni-6.10.2.jar:?]
at org.rocksdb.RocksDB.write(RocksDB.java:1586) ~[org.rocksdb-rocksdbjni-6.10.2.jar:?]
at org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB$RocksDBBatch.flush(KeyValueStorageRocksDB.java:449) ~[org.apache.bookkeeper-bookkeeper-server-4.14.4.jar:4.14.4]
... 12 more
2022-10-20T09:54:28,204+0000 [main-EventThread] INFO org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager - Latch countdown due to ZK event: WatchedEvent state:SyncConnected type:NodeChildrenChanged path:/ledgers/underreplication/locks
2022-10-20T09:54:28,206+0000 [main-EventThread] INFO org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager - Latch countdown due to ZK event: WatchedEvent state:SyncConnected type:NodeChildrenChanged path:/ledgers/underreplication/locks
2022-10-20T09:55:23,019+0000 [SyncThread-7-1] ERROR org.apache.bookkeeper.bookie.SyncThread - Exception flushing ledgers
java.io.IOException: Failed to flush RocksDB batch
at org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB$RocksDBBatch.flush(KeyValueStorageRocksDB.java:451) ~[org.apache.bookkeeper-bookkeeper-server-4.14.4.jar:4.14.4]
at org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage.checkpoint(SingleDirectoryDbLedgerStorage.java:621) ~[org.apache.bookkeeper-bookkeeper-server-4.14.4.jar:4.14.4]
at org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage.checkpoint(DbLedgerStorage.java:231) ~[org.apache.bookkeeper-bookkeeper-server-4.14.4.jar:4.14.4]
at org.apache.bookkeeper.bookie.SyncThread.checkpoint(SyncThread.java:152) ~[org.apache.bookkeeper-bookkeeper-server-4.14.4.jar:4.14.4]
at org.apache.bookkeeper.bookie.SyncThread.lambda$doCheckpoint$0(SyncThread.java:99) ~[org.apache.bookkeeper-bookkeeper-server-4.14.4.jar:4.14.4]
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [?:1.8.0_342]
at java.util.concurrent.FutureTask.run(FutureTask.java:266) [?:1.8.0_342]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) [?:1.8.0_342]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) [?:1.8.0_342]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_342]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_342]
at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.74.Final.jar:4.1.74.Final]
at java.lang.Thread.run(Thread.java:750) [?:1.8.0_342]
Caused by: org.rocksdb.RocksDBException: block checksum mismatch: expected 459167422, got 2336252423 in data/bookkeeper/ledgers/current/locations/000423.sst offset 612461 size 32254
at org.rocksdb.RocksDB.write0(Native Method) ~[org.rocksdb-rocksdbjni-6.10.2.jar:?]
at org.rocksdb.RocksDB.write(RocksDB.java:1586) ~[org.rocksdb-rocksdbjni-6.10.2.jar:?]
at org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB$RocksDBBatch.flush(KeyValueStorageRocksDB.java:449) ~[org.apache.bookkeeper-bookkeeper-server-4.14.4.jar:4.14.4]
... 12 more
broker
2022-10-20T13:34:33,566+0000 [pulsar-ph-kafka-79-21] INFO org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - Opening managed ledger public/kafka/persistent/hourly-datetime-Ingest-partition-0
2022-10-20T13:34:33,569+0000 [bookkeeper-ml-scheduler-OrderedScheduler-11-0] INFO org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [public/kafka/persistent/hourly-datetime-Ingest-partition-0] Closing managed ledger
2022-10-20T13:34:33,569+0000 [bookkeeper-ml-scheduler-OrderedScheduler-11-0] INFO org.apache.pulsar.broker.PulsarService - No ledger offloader configured, using NULL instance
2022-10-20T13:34:33,569+0000 [bookkeeper-ml-scheduler-OrderedScheduler-11-0] INFO org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - Opening managed ledger public/kafka/persistent/hourly-datetime-Ingest
2022-10-20T13:34:33,574+0000 [bookkeeper-ml-scheduler-OrderedScheduler-11-0] INFO org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [public/kafka/persistent/hourly-datetime-Ingest] Closing managed ledger
2022-10-20T13:34:33,574+0000 [bookkeeper-ml-scheduler-OrderedScheduler-11-0] ERROR io.streamnative.pulsar.handlers.kop.KafkaTopicManager - [[id: 0x864f8b07, L:/192.168.3.3:9092 - R:/192.168.3.2:33020]]Get empty non-partitioned topic for name persistent://public/kafka/hourly-datetime-Ingest
2022-10-20T13:34:33,574+0000 [bookkeeper-ml-scheduler-OrderedScheduler-11-0] ERROR io.streamnative.pulsar.handlers.kop.KafkaTopicManager - [[id: 0x54c1cef3, L:/192.168.3.3:9092 - R:/192.168.3.2:33024]]Get empty non-partitioned topic for name persistent://public/kafka/hourly-datetime-Ingest
2022-10-20T13:34:33,675+0000 [pulsar-ph-kafka-79-21] INFO org.apache.pulsar.broker.PulsarService - No ledger offloader configured, using NULL instance
2022-10-20T13:34:33,675+0000 [pulsar-ph-kafka-79-21] INFO org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - Opening managed ledger public/kafka/persistent/hourly-datetime-Ingest-partition-0
2022-10-20T13:34:33,679+0000 [bookkeeper-ml-scheduler-OrderedScheduler-11-0] INFO org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [public/kafka/persistent/hourly-datetime-Ingest-partition-0] Closing managed ledger
2022-10-20T13:34:33,679+0000 [bookkeeper-ml-scheduler-OrderedScheduler-11-0] INFO org.apache.pulsar.broker.PulsarService - No ledger offloader configured, using NULL instance
2022-10-20T13:34:33,679+0000 [bookkeeper-ml-scheduler-OrderedScheduler-11-0] INFO org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - Opening managed ledger public/kafka/persistent/hourly-datetime-Ingest
2022-10-20T13:34:33,684+0000 [bookkeeper-ml-scheduler-OrderedScheduler-11-0] INFO org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [public/kafka/persistent/hourly-datetime-Ingest] Closing managed ledger
2022-10-20T13:34:33,684+0000 [bookkeeper-ml-scheduler-OrderedScheduler-11-0] ERROR io.streamnative.pulsar.handlers.kop.KafkaTopicManager - [[id: 0x864f8b07, L:/192.168.3.3:9092 - R:/192.168.3.2:33020]]Get empty non-partitioned topic for name persistent://public/kafka/hourly-datetime-Ingest
2022-10-20T13:34:33,684+0000 [bookkeeper-ml-scheduler-OrderedScheduler-11-0] ERROR io.streamnative.pulsar.handlers.kop.KafkaTopicManager - [[id: 0x54c1cef3, L:/192.168.3.3:9092 - R:/192.168.3.2:33024]]Get empty non-partitioned topic for name persistent://public/kafka/hourly-datetime-Ingest
2022-10-20T13:34:33,685+0000 [pulsar-web-35-9] INFO org.eclipse.jetty.server.RequestLog - 192.168.3.3 - - [20/Oct/2022:13:34:33 +0000] "GET /admin/v2/persistent/public/kafka/hourly-datetime-Ingest/partitions HTTP/1.1" 200 16 "-" "Pulsar-Java-v2.10.0" 1
Any update on this issue? I encountered the same error. Pulsar-2.10.1
Thanks.
Hi there, I'm facing a similar issue in Pulsar-2.9.5, bookie ledger utilization increased to 99.9%. But I can't see any ledger information in error logs there Could some one please tell me how to get over this issue? any workaround?
Details logs: Several days ago, a lot of "Entering Safepoint region..." "Leaving safepoint region..." in logs After a day, I found this "Exception ledger flush" / "Error in Rocksdb put" starting in logs. And 4 hours later, it turned to a lots "Error during flush" Above logs happen until "Ledger directory ... is out-of-space"
This issue we also recently faced within our cluster. It happens if you restart any bookie without touching autorecovery. At that time if there is any ledger is trying to replicate it and if you don't have enough writeable bookie available. The replication goes into infinite loop even later all the bookie is available. Yes, this is a bug which needs a permanent fix. The temporary fix we applied figured out how many ledgers are making noise. Just make a note of all faulty ledgers and remove it safely. Afterthat the storage will start making free.
BUG REPORT
Problem happens after reboot bookie(Bare metal), i can't remember a little bit about the detailed steps at the time.
Found that disk usage continued growth:
Then check the bookie log, found some ledger replicate error log:
This won't always happen, I also restarted bookie before. I don't know why lost these four entries[E4139 - E4144] in bookie and ledger. And this is dangerous that ledger replicate so many times.
After we disable autorecovery and find the topic use this ledger, then delete the topic, entry log was successfully cleaned up