apache / pulsar

Apache Pulsar - distributed pub-sub messaging system
https://pulsar.apache.org/
Apache License 2.0
14.23k stars 3.58k forks source link

[Bug] Failed to publish message when using DbLedgerStorage in the pulsar standalone #18062

Open RobertIndie opened 2 years ago

RobertIndie commented 2 years ago

Search before asking

Version

Master branch, at least 08f5766d95034ce27c44ee30e4734d2a8f078e11

Minimal reproduce step

What did you expect to see?

Success to publish large messages

What did you see instead?

Throw these exceptions with dead loop:

2022-10-17T11:16:58,804+0800 [BookieWriteThreadPool-OrderedExecutor-0-0] WARN  org.apache.bookkeeper.common.util.OrderedExecutor - Runnable WriteEntry(-1, -1):class org.apache.bookkeeper.proto.WriteEntryProcessor took too long 10002535 micros to execute.
2022-10-17T11:16:58,806+0800 [BookKeeperClientWorker-OrderedExecutor-0-0] WARN  org.apache.bookkeeper.client.PendingAddOp - Failed to write entry (1, 0): Too many requests to the same Bookie
2022-10-17T11:16:58,818+0800 [BookKeeperClientWorker-OrderedExecutor-0-0] WARN  org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl - Failed to find 1 bookies : excludeBookies [<Bookie:127.0.0.1:3181>], allBookies [<Bookie:127.0.0.1:3181>].
2022-10-17T11:16:58,818+0800 [BookKeeperClientWorker-OrderedExecutor-0-0] WARN  org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl - Failed to choose a bookie: excluded [<Bookie:127.0.0.1:3181>], fallback to choose bookie randomly from the cluster.
2022-10-17T11:16:58,819+0800 [BookKeeperClientWorker-OrderedExecutor-0-0] WARN  org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl - Failed to find 1 bookies : excludeBookies [<Bookie:127.0.0.1:3181>], allBookies [<Bookie:127.0.0.1:3181>].
2022-10-17T11:16:58,819+0800 [BookKeeperClientWorker-OrderedExecutor-0-0] WARN  org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl - Failed to find 1 bookies : excludeBookies [<Bookie:127.0.0.1:3181>], allBookies [<Bookie:127.0.0.1:3181>].
2022-10-17T11:16:58,819+0800 [BookKeeperClientWorker-OrderedExecutor-0-0] WARN  org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl - Failed to choose a bookie: excluded [<Bookie:127.0.0.1:3181>], fallback to choose bookie randomly from the cluster.
2022-10-17T11:16:58,822+0800 [BookKeeperClientWorker-OrderedExecutor-0-0] WARN  org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl - Failed to find 1 bookies : excludeBookies [<Bookie:127.0.0.1:3181>], allBookies [<Bookie:127.0.0.1:3181>].
2022-10-17T11:16:58,822+0800 [BookKeeperClientWorker-OrderedExecutor-0-0] ERROR org.apache.bookkeeper.client.MetadataUpdateLoop - UpdateLoop(ledgerId=1,loopId=44f16df5) Exception updating
org.apache.bookkeeper.client.BKException$BKNotEnoughBookiesException: Not enough non-faulty bookies available
    at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.selectRandomInternal(RackawareEnsemblePlacementPolicyImpl.java:780) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.selectRandom(RackawareEnsemblePlacementPolicyImpl.java:697) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.selectFromNetworkLocation(RackawareEnsemblePlacementPolicyImpl.java:586) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy.selectFromNetworkLocation(RackawareEnsemblePlacementPolicy.java:206) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.selectFromNetworkLocation(RackawareEnsemblePlacementPolicyImpl.java:546) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy.selectFromNetworkLocation(RackawareEnsemblePlacementPolicy.java:227) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.replaceBookie(RackawareEnsemblePlacementPolicyImpl.java:474) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy.replaceBookie(RackawareEnsemblePlacementPolicy.java:120) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.BookieWatcherImpl.replaceBookie(BookieWatcherImpl.java:334) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.EnsembleUtils.replaceBookiesInEnsemble(EnsembleUtils.java:71) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.LedgerHandle.lambda$ensembleChangeLoop$2(LedgerHandle.java:1949) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.MetadataUpdateLoop.writeLoop(MetadataUpdateLoop.java:134) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.MetadataUpdateLoop.run(MetadataUpdateLoop.java:123) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.LedgerHandle.ensembleChangeLoop(LedgerHandle.java:1968) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.LedgerHandle.handleBookieFailure(LedgerHandle.java:1917) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:378) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.proto.PerChannelBookieClient$AddCompletion.writeComplete(PerChannelBookieClient.java:2177) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.proto.PerChannelBookieClient$AddCompletion.handleResponse(PerChannelBookieClient.java:2234) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.proto.PerChannelBookieClient$AddCompletion.handleV2Response(PerChannelBookieClient.java:2213) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadV2ResponseCallback.safeRun(PerChannelBookieClient.java:1391) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) ~[bookkeeper-common-4.15.1.jar:4.15.1]
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) ~[?:?]
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) ~[?:?]
    at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) ~[netty-common-4.1.77.Final.jar:4.1.77.Final]
    at java.lang.Thread.run(Thread.java:833) ~[?:?]
2022-10-17T11:16:58,860+0800 [BookKeeperClientWorker-OrderedExecutor-0-0] WARN  org.apache.bookkeeper.client.LedgerHandle - [EnsembleChange(ledger:1, change-id:0000000001)][attempt:1] Exception changing ensemble
org.apache.bookkeeper.client.BKException$BKNotEnoughBookiesException: Not enough non-faulty bookies available
    at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.selectRandomInternal(RackawareEnsemblePlacementPolicyImpl.java:780) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.selectRandom(RackawareEnsemblePlacementPolicyImpl.java:697) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.selectFromNetworkLocation(RackawareEnsemblePlacementPolicyImpl.java:586) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy.selectFromNetworkLocation(RackawareEnsemblePlacementPolicy.java:206) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.selectFromNetworkLocation(RackawareEnsemblePlacementPolicyImpl.java:546) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy.selectFromNetworkLocation(RackawareEnsemblePlacementPolicy.java:227) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.replaceBookie(RackawareEnsemblePlacementPolicyImpl.java:474) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy.replaceBookie(RackawareEnsemblePlacementPolicy.java:120) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.BookieWatcherImpl.replaceBookie(BookieWatcherImpl.java:334) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.EnsembleUtils.replaceBookiesInEnsemble(EnsembleUtils.java:71) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.LedgerHandle.lambda$ensembleChangeLoop$2(LedgerHandle.java:1949) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.MetadataUpdateLoop.writeLoop(MetadataUpdateLoop.java:134) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.MetadataUpdateLoop.run(MetadataUpdateLoop.java:123) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.LedgerHandle.ensembleChangeLoop(LedgerHandle.java:1968) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.LedgerHandle.handleBookieFailure(LedgerHandle.java:1917) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:378) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.proto.PerChannelBookieClient$AddCompletion.writeComplete(PerChannelBookieClient.java:2177) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.proto.PerChannelBookieClient$AddCompletion.handleResponse(PerChannelBookieClient.java:2234) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.proto.PerChannelBookieClient$AddCompletion.handleV2Response(PerChannelBookieClient.java:2213) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.proto.PerChannelBookieClient$ReadV2ResponseCallback.safeRun(PerChannelBookieClient.java:1391) ~[bookkeeper-server-4.15.1.jar:4.15.1]
    at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) ~[bookkeeper-common-4.15.1.jar:4.15.1]
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) ~[?:?]
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) ~[?:?]
    at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) ~[netty-common-4.1.77.Final.jar:4.1.77.Final]
    at java.lang.Thread.run(Thread.java:833) ~[?:?]
2022-10-17T11:16:58,942+0800 [BookKeeperClientWorker-OrderedExecutor-0-0] ERROR org.apache.bookkeeper.client.LedgerHandle - Closing ledger 1 due to NotEnoughBookiesException: Not enough non-faulty bookies available
2022-10-17T11:16:58,943+0800 [BookKeeperClientWorker-OrderedExecutor-0-0] ERROR org.apache.bookkeeper.client.PendingAddOp - Write of ledger entry to quorum failed: L1 E0
2022-10-17T11:16:58,943+0800 [BookKeeperClientWorker-OrderedExecutor-0-0] ERROR org.apache.bookkeeper.client.PendingAddOp - Write of ledger entry to quorum failed: L1 E1
2022-10-17T11:16:58,943+0800 [BookKeeperClientWorker-OrderedExecutor-0-0] ERROR org.apache.bookkeeper.client.PendingAddOp - Write of ledger entry to quorum failed: L1 E2
2022-10-17T11:16:58,944+0800 [BookKeeperClientWorker-OrderedExecutor-0-0] ERROR org.apache.bookkeeper.client.PendingAddOp - Write of ledger entry to quorum failed: L1 E3

Anything else?

The problem may be related to this configuration:

ledgerStorageClass=org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage

It works fine when I set it to use SortedLedgerStorage.

For more context see: https://github.com/apache/pulsar/pull/17985#issue-1402829309

Are you willing to submit a PR?

RobertIndie commented 2 years ago

I think I have found the root cause.

The root cause may from the WriteCache. If the size of the entry exceeds half of the dbStorage_writeCacheMaxSizeMb, the error will occur.

And here is the detail: From the log we can see that the write cache is full when writing the first chunk of the large message:

2022-10-18T14:55:04,033+0800 [pulsar-io-19-1] DEBUG org.apache.pulsar.broker.service.ServerCnx - [/127.0.0.1:60392] Received send message request. producer: 0:0 standalone-14-0:0 size: 5242922, partition key is: null, ordering key is null
2022-10-18T14:55:04,042+0800 [pulsar-io-19-1] DEBUG org.apache.pulsar.broker.service.ServerCnx - [/127.0.0.1:60392] Received send message request. producer: 0:0 standalone-14-0:0 size: 5242922, partition key is: null, ordering key is null
2022-10-18T14:55:04,056+0800 [BookieWriteThreadPool-OrderedExecutor-0-0] DEBUG org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage - Set master key. ledger: 15
2022-10-18T14:55:04,058+0800 [BookieWriteThreadPool-OrderedExecutor-0-0] DEBUG org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage - isFenced. ledger: 15
2022-10-18T14:55:04,059+0800 [BookieWriteThreadPool-OrderedExecutor-0-0] DEBUG org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage - hasLimboState. ledger: 15
2022-10-18T14:55:04,059+0800 [BookieWriteThreadPool-OrderedExecutor-0-0] DEBUG org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage - Add entry. 15@0, lac = -1
2022-10-18T14:55:04,059+0800 [BookieWriteThreadPool-OrderedExecutor-0-0] INFO  org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage - Write cache is full, triggering flush
2022-10-18T14:55:04,068+0800 [pulsar-io-19-1] DEBUG org.apache.pulsar.broker.service.ServerCnx - [/127.0.0.1:60392] Received send message request. producer: 0:0 standalone-14-0:0 size: 5242922, partition key is: null, ordering key is null
2022-10-18T14:55:04,070+0800 [pulsar-io-19-1] DEBUG org.apache.pulsar.broker.service.ServerCnx - [/127.0.0.1:60392] Received send message request. producer: 0:0 standalone-14-0:0 size: 166, partition key is: null, ordering key is null

But in fact, no entries were written successfully.

org.apache.bookkeeper.client.BKException$BKNotEnoughBookiesException: Not enough non-faulty bookies available
...

If the first entry already exceeds the maxCacheSize of the WriteCache, it will not be added to the WriteCache because of this logic:

https://github.com/apache/bookkeeper/blob/30bdedc25a59aa7d4df3f5c0962095a574f0d653/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCache.java#L149-L151

And the maxCacheSize is half of the dbStorage_writeCacheMaxSizeMb: https://github.com/apache/bookkeeper/blob/30bdedc25a59aa7d4df3f5c0962095a574f0d653/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java#L169

So we need to make sure dbStorage_writeCacheMaxSizeMb is greater than 2 * maxEntrySize.

I think it's a bug from the WriteCache. I will give a fix later.

github-actions[bot] commented 1 year ago

The issue had no activity for 30 days, mark with Stale label.