TakaHiR07 opened a new issue, #4114:
URL: https://github.com/apache/bookkeeper/issues/4114

   **BUG REPORT**
   
   
![企业微信截图_d9a06c8b-515f-4182-a516-b3be87900ba5](https://github.com/apache/bookkeeper/assets/13505225/66d37806-5155-4052-bda8-36b22ceeb239)
   
   When bookie encounter rocksdb IOException, it would cause flush process keep 
running and failing continuously. Since flush of entry is successful, flush of 
rocksdb is failed, finally the disk would be write full. 
   
   I think the reason is IOException is just caught and log error. Then the 
flush process is finish but flush is actually not success. So the entry is 
still in writeCache, waiting for the next time flush. And then the next time 
flush throw the same IOException as before.
   
   **I guess that keep doing unsuccessful flush is not reasonable, and we'd 
better throw the IOException of rocksdb and trigger shutdown bookie ?** 
   
   
   The Relevant code is :
   
   
https://github.com/apache/bookkeeper/blob/3221aa30924825cb4c1a5b00fb68dec44712946e/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java#L831
   
   
https://github.com/apache/bookkeeper/blob/3221aa30924825cb4c1a5b00fb68dec44712946e/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SyncThread.java#L159-L182
   
   
https://github.com/apache/bookkeeper/blob/3221aa30924825cb4c1a5b00fb68dec44712946e/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SyncThread.java#L107-L112
   
   This case log is as following:
   ```
   00:00:02.479 [SyncThread-7-1] INFO  
org.apache.bookkeeper.bookie.EntryLogManagerForSingleEntryLog - Synced entry 
logger 565941 to disk.
   00:00:02.491 [SyncThread-7-1] INFO  
org.apache.bookkeeper.bookie.EntryLogManagerForSingleEntryLog - Synced entry 
logger 565942 to disk.
   00:00:02.491 [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:479)
 ~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
           at 
org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage.checkpoint(SingleDirectoryDbLedgerStorage.java:669)
 ~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
           at 
org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage.checkpoint(DbLedgerStorage.java:231)
 ~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
           at 
org.apache.bookkeeper.bookie.SyncThread.checkpoint(SyncThread.java:152) 
~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
           at 
org.apache.bookkeeper.bookie.SyncThread.lambda$doCheckpoint$0(SyncThread.java:99)
 ~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
           at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) ~[?:?]
           at java.util.concurrent.FutureTask.run(FutureTask.java:264) ~[?:?]
           at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
 ~[?:?]
           at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) 
~[?:?]
           at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) 
~[?:?]
           at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
 ~[io.netty-netty-common-4.1.66.Final.jar:4.1.66.Final]
           at java.lang.Thread.run(Thread.java:834) ~[?:?]
   Caused by: org.rocksdb.RocksDBException: While pread offset 17481728 len 
21533: /data7/pulsar-bookie/ledger/current/locations/490882.sst: Input/output 
error
           at org.rocksdb.RocksDB.write0(Native Method) 
~[org.rocksdb-rocksdbjni-6.16.4.jar:?]
           at org.rocksdb.RocksDB.write(RocksDB.java:1784) 
~[org.rocksdb-rocksdbjni-6.16.4.jar:?]
           at 
org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB$RocksDBBatch.flush(KeyValueStorageRocksDB.java:477)
 ~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
           ... 11 more
   00:00:02.491 [bookie-io-1-4] INFO  
org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage - Write 
cache is full, triggering flush
   00:00:04.675 [db-storage-9-1] INFO  
org.apache.bookkeeper.bookie.EntryLogManagerBase - Creating a new entry log 
file : createNewLog = false, reachEntryLogLimit = true
   
   ......
   
   
   00:00:13.104 [db-storage-9-1] ERROR 
org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage - Error 
during flush
   java.io.IOException: Failed to flush RocksDB batch
           at 
org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB$RocksDBBatch.flush(KeyValueStorageRocksDB.java:479)
 ~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
           at 
org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage.checkpoint(SingleDirectoryDbLedgerStorage.java:669)
 ~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
           at 
org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage.flush(SingleDirectoryDbLedgerStorage.java:748)
 ~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
           at 
org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage.lambda$triggerFlushAndAddEntry$5(SingleDirectoryDbLedgerStorage.java:383)
 ~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
           at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) 
~[?:?]
           at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) 
~[?:?]
           at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
 ~[io.netty-netty-common-4.1.66.Final.jar:4.1.66.Final]
           at java.lang.Thread.run(Thread.java:834) ~[?:?]
   Caused by: org.rocksdb.RocksDBException: While pread offset 17481728 len 
21533: /data7/pulsar-bookie/ledger/current/locations/490882.sst: Input/output 
error
           at org.rocksdb.RocksDB.write0(Native Method) 
~[org.rocksdb-rocksdbjni-6.16.4.jar:?]
           at org.rocksdb.RocksDB.write(RocksDB.java:1784) 
~[org.rocksdb-rocksdbjni-6.16.4.jar:?]
           at 
org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB$RocksDBBatch.flush(KeyValueStorageRocksDB.java:477)
 ~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
           ... 7 more
   00:00:13.105 [bookie-io-1-5] INFO  
org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage - Write 
cache is full, triggering flush
   
   ......
   
   00:00:23.339 [db-storage-9-1] ERROR 
org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage - Error 
during flush
   java.io.IOException: Failed to flush RocksDB batch
           at 
org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB$RocksDBBatch.flush(KeyValueStorageRocksDB.java:479)
 ~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
           at 
org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage.checkpoint(SingleDirectoryDbLedgerStorage.java:669)
 ~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
           at 
org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage.flush(SingleDirectoryDbLedgerStorage.java:748)
 ~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
           at 
org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage.lambda$triggerFlushAndAddEntry$5(SingleDirectoryDbLedgerStorage.java:383)
 ~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
           at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) 
~[?:?]
           at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) 
~[?:?]
           at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
 ~[io.netty-netty-common-4.1.66.Final.jar:4.1.66.Final]
           at java.lang.Thread.run(Thread.java:834) ~[?:?]
   Caused by: org.rocksdb.RocksDBException: While pread offset 17481728 len 
21533: /data7/pulsar-bookie/ledger/current/locations/490882.sst: Input/output 
error
           at org.rocksdb.RocksDB.write0(Native Method) 
~[org.rocksdb-rocksdbjni-6.16.4.jar:?]
           at org.rocksdb.RocksDB.write(RocksDB.java:1784) 
~[org.rocksdb-rocksdbjni-6.16.4.jar:?]
           at 
org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB$RocksDBBatch.flush(KeyValueStorageRocksDB.java:477)
 ~[org.apache.bookkeeper-bookkeeper-server-4.14.1.jar:4.14.1]
           ... 7 more
   ```
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to