apache / pulsar

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

[Bug] Topic ledger id will rollback in some cases #20094

Open janmee opened 1 year ago

janmee commented 1 year ago

Search before asking

Version

2.11.0

Minimal reproduce step

in my environment deployment: Ensemble: 3, write quorum: 3, ack quorum: 2 in Machine A : brokerA, bookieA in Machine B: brokerB, bookieB in Machine C: brokerC, bookieC

1、If Machine A goes offline for a minute, the ownership of the topic in broker A will be moved to broker B. Broker B will fence the old ledger X and use the new ledger Y to write data. 2、Ledger X on bookie B and bookie C is deleted by the GC thread. 3、After Machine A recovers from the network outage, and a period of time later, Machine B goes offline. The topic ownership will be transferred back to broker A, and broker A, for some reason, continues to use ledger X to write data. At this point, ledger X can be written successfully and returned to the client, causing message ID rollback.

What did you expect to see?

in the end, the topic owned by broker A should use ledger id larger than ledger Y

What did you see instead?

broker A use ledger X to write entry

Anything else?

some log in broker A: 2023-04-12T14:28:49,875+0800 [pulsar-io-4-3] DEBUG org.apache.pulsar.broker.service.ServerCnx - [/10.14.30.111:41134] Received send message request. producer: 0:38699 region1:default:38699 size: 65, partition key is: key38799, ordering key is null 2023-04-12T14:28:49,875+0800 [pulsar-io-4-3] DEBUG org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [longhaul/test/persistent/testTopic] asyncAddEntry size=65 state=LedgerOpened 2023-04-12T14:28:49,875+0800 [BookKeeperClientWorker-OrderedExecutor-3-0] DEBUG org.apache.bookkeeper.mledger.impl.OpAddEntry - Created new OpAddEntry OpAddEntry{mlName=longhaul/test/persistent/testTopic, ledgerId=null, entryId=-1, startTime=67089676632364916, dataLength=65} 2023-04-12T14:28:49,875+0800 [BookKeeperClientWorker-OrderedExecutor-3-0] DEBUG org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [longhaul/test/persistent/testTopic] Write into current ledger lh=13810 entries=16600 network down: 2023-04-12T14:29:06,538+0800 [ZKC-connect-executor-0-SendThread(10.14.30.107:2181)] WARN org.apache.zookeeper.ClientCnxn - Session 0x681d1fc5c69d0016 for server 10.14.30.107/10.14.30.107:2181, Closing socket connection. Attempting reconnect except it is a SessionExpiredException java.net.SocketException: Network is unreachable 2023-04-12T14:29:21,303+0800 [BookKeeperClientWorker-OrderedExecutor-1-0] WARN org.apache.bookkeeper.client.PendingAddOp - Failed to write entry (13811, 185) to bookie (2, 10.14.30.111:11381): Bookie operation timeout 2023-04-12T14:29:53,302+0800 [ZKC-connect-executor-0-SendThread(10.14.30.107:2181)] ERROR org.apache.bookkeeper.client.MetadataUpdateLoop - UpdateLoop(ledgerId=13811,loopId=75e44191) Error writing metadata to store org.apache.bookkeeper.client.BKException$ZKException: Error while using ZooKeeper .....

2023-04-12T15:32:06,390+0800 [BookKeeperClientWorker-OrderedExecutor-1-0] DEBUG org.apache.bookkeeper.proto.PerChannelBookieClient - Got Read response from bookie:10.14.30.106:11381 rc:EOK, ledger:13810:entry:16593:entryLength:116 023-04-12T15:32:06,396+0800 [pulsar-io-4-5] DEBUG org.apache.pulsar.broker.service.ServerCnx - [/10.14.30.111:45287] Received send message request. producer: 0:377200 region1:default:377200 size: 67, partition key is: key377300, ordering key is null 2023-04-12T15:32:06,396+0800 [pulsar-io-4-5] DEBUG org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [longhaul/test/persistent/testTopic] asyncAddEntry size=67 state=LedgerOpened 2023-04-12T15:32:06,396+0800 [BookKeeperClientWorker-OrderedExecutor-3-0] DEBUG org.apache.bookkeeper.mledger.impl.OpAddEntry - Created new OpAddEntry OpAddEntry{mlName=longhaul/test/persistent/testTopic, ledgerId=null, entryId=-1, startTime=67093473153018494, dataLength=67} 2023-04-12T15:32:06,396+0800 [BookKeeperClientWorker-OrderedExecutor-3-0] DEBUG org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [longhaul/test/persistent/testTopic] Write into current ledger lh=13810 entries=16601 2023-04-12T15:32:06,402+0800 [BookKeeperClientWorker-OrderedExecutor-1-0] DEBUG org.apache.bookkeeper.proto.PerChannelBookieClient - Got Add response from bookie:11.167.133.184:11381 rc:EOK, ledger:13810:entry:16600 2023-04-12T15:32:06,408+0800 [BookKeeperClientWorker-OrderedExecutor-1-0] DEBUG org.apache.bookkeeper.proto.PerChannelBookieClient - Got Add response from bookie:10.14.30.106:11381 rc:EOK, ledger:13810:entry:16600

log in brokerB: 2023-04-12T14:29:16,455+0800 [pulsar-io-4-7] INFO org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - Opening managed ledger longhaul/test/persistent/testTopic 2023-04-12T14:29:16,458+0800 [bookkeeper-ml-scheduler-OrderedScheduler-3-0] INFO org.apache.pulsar.broker.intercept.ManagedLedgerInterceptorImpl - onManagedLedgerP ropertiesInitialize init timeID:1681280727036996 2023-04-12T14:29:16,458+0800 [bookkeeper-ml-scheduler-OrderedScheduler-3-0] DEBUG org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [longhaul/test/persistent/t estTopic] Opening ledger 13810 2023-04-12T14:29:16,469+0800 [ZKC-connect-executor-0-EventThread] DEBUG org.apache.bookkeeper.client.MetadataUpdateLoop - UpdateLoop(ledgerId=13810,loopId=6a93ef27) success 2023-04-12T14:29:17,289+0800 [BookKeeperClientWorker-OrderedExecutor-1-0] DEBUG org.apache.bookkeeper.proto.PerChannelBookieClient - Got Read response from bookie:1 1.167.133.184:11381 rc:EOK, ledger:13810:entry:16599:entryLength:116 2023-04-12T14:29:17,293+0800 [BookKeeperClientWorker-OrderedExecutor-1-0] DEBUG org.apache.bookkeeper.proto.PerChannelBookieClient - Got Add response from bookie:11 .167.133.184:11381 rc:EOK, ledger:13810:entry:16599 2023-04-12T14:29:17,296+0800 [BookKeeperClientWorker-OrderedExecutor-1-0] DEBUG org.apache.bookkeeper.proto.PerChannelBookieClient - Got Add response from bookie:11 .164.30.111:11381 rc:EOK, ledger:13810:entry:16599 2023-04-12T14:29:17,491+0800 [BookKeeperClientWorker-OrderedExecutor-1-0] DEBUG org.apache.bookkeeper.proto.PerChannelBookieClient - Got Read response from bookie:10.14.30.111:11381 rc:ENOENTRY, ledger:13810:entry:16600:entryLength:0 2023-04-12T14:29:17,491+0800 [BookKeeperClientWorker-OrderedExecutor-1-0] DEBUG org.apache.bookkeeper.client.PendingReadOp - No such entry found on bookie. L13810E16600 bookie: 10.14.30.111:11381 2023-04-12T14:29:17,491+0800 [BookKeeperClientWorker-OrderedExecutor-1-0] INFO org.apache.bookkeeper.client.ReadOnlyLedgerHandle - Closing recovered ledger 13810 a t entry 16599 2023-04-12T14:29:17,500+0800 [BookKeeperClientWorker-OrderedExecutor-3-0] DEBUG org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [longhaul/test/persistent/testTopic] Opened ledger 13810: No problem 2023-04-12T14:29:17,770+0800 [pulsar-io-4-7] DEBUG org.apache.pulsar.broker.service.ServerCnx - [/10.14.30.111:42441] Received send message request. producer: 0:41100 region1:default:41100 size: 65, partition key is: key41200, ordering key is null 2023-04-12T14:29:17,770+0800 [pulsar-io-4-7] DEBUG org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [longhaul/test/persistent/testTopic] asyncAddEntry size=65 state=LedgerOpened 2023-04-12T14:29:17,770+0800 [BookKeeperClientWorker-OrderedExecutor-3-0] DEBUG org.apache.bookkeeper.mledger.impl.OpAddEntry - Created new OpAddEntry OpAddEntry{mlName=longhaul/test/persistent/testTopic, ledgerId=null, entryId=-1, startTime=125088582810719907, dataLength=65} 2023-04-12T14:29:17,770+0800 [BookKeeperClientWorker-OrderedExecutor-3-0] DEBUG org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [longhaul/test/persistent/testTopic] Write into current ledger lh=13814 entries=1 2023-04-12T14:29:26,470+0800 [BookKeeperClientWorker-OrderedExecutor-1-0] INFO org.apache.bookkeeper.client.PendingReadOp - Error: Bookie handle is not available w hile reading L13810 E16592 from bookie: 10.14.30.106:11381 ... 2023-04-12T14:29:26,471+0800 [BookKeeperClientWorker-OrderedExecutor-1-0] INFO org.apache.bookkeeper.client.PendingReadOp - Error: Bookie handle is not available w hile reading L13810 E16599 from bookie: 10.14.30.106:11381

log in bookieB and bookieC, we can find the same message: 2023-04-12T14:29:17,291+0800 [BookieJournal-11381] DEBUG org.apache.bookkeeper.bookie.Journal - Written and queuing for flush Ledger: 13810 Entry: 16599 2023-04-12T14:29:17,291+0800 [ForceWriteThread] DEBUG org.apache.bookkeeper.bookie.JournalChannel - Journal ForceWrite 2023-04-12T14:29:17,296+0800 [bookie-journal-callback-4] DEBUG org.apache.bookkeeper.bookie.Journal - Acknowledge Ledger: 13810, Entry: 16599 2023-04-12T14:29:17,490+0800 [BookieHighPriorityThread-OrderedExecutor-1-0] DEBUG org.apache.bookkeeper.proto.ReadEntryProcessor - Received new read request: Op(2)[ Ledger:13810,Entry:16600] 2023-04-12T14:29:17,490+0800 [BookieHighPriorityThread-OrderedExecutor-1-0] WARN org.apache.bookkeeper.proto.ReadEntryProcessor - Ledger: 13810 fenced by: /10.14 .30.104:36794 2023-04-12T14:29:17,490+0800 [BookieHighPriorityThread-OrderedExecutor-1-0] DEBUG org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage - Set fenc ed. ledger: 13810 2023-04-12T14:29:17,490+0800 [BookieHighPriorityThread-OrderedExecutor-1-0] DEBUG org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage - Get Entr y: 13810@16600 2023-04-12T14:29:17,490+0800 [BookieHighPriorityThread-OrderedExecutor-1-0] DEBUG org.apache.bookkeeper.bookie.storage.ldb.EntryLocationIndex - Entry not found 1381 0@16600 in db index ... 2023-04-12T14:52:19,980+0800 [GarbageCollectorThread-9-1] DEBUG org.apache.bookkeeper.bookie.GarbageCollectorThread - delete ledger : 13810 2023-04-12T14:52:19,980+0800 [GarbageCollectorThread-9-1] DEBUG org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage - Deleting ledger 13810 2023-04-12T14:52:19,980+0800 [GarbageCollectorThread-9-1] DEBUG org.apache.bookkeeper.bookie.storage.ldb.LedgerMetadataIndex - Removed ledger 13810 ... 2023-04-12T15:32:06,397+0800 [bookie-io-11-2] DEBUG org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage - Set master key. ledger: 13810 2023-04-12T15:32:06,397+0800 [bookie-io-11-2] DEBUG org.apache.bookkeeper.bookie.storage.ldb.LedgerMetadataIndex - Inserting new ledger 13810 2023-04-12T15:32:06,397+0800 [bookie-io-11-2] DEBUG org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage - isFenced. ledger: 13810 2023-04-12T15:32:06,397+0800 [bookie-io-11-2] DEBUG org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage - hasLimboState. ledger: 13810 2023-04-12T15:32:06,398+0800 [bookie-io-11-2] DEBUG org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage - Add entry. 13810@16600, lac = 16599 2023-04-12T15:32:06,399+0800 [BookieJournal-11381] DEBUG org.apache.bookkeeper.bookie.Journal - Written and queuing for flush Ledger: 13810 Entry: 16600 2023-04-12T15:32:06,399+0800 [ForceWriteThread] DEBUG org.apache.bookkeeper.bookie.JournalChannel - Journal ForceWrite 2023-04-12T15:32:06,403+0800 [bookie-journal-callback-5] DEBUG org.apache.bookkeeper.bookie.Journal - Acknowledge Ledger: 13810, Entry: 16600

in the log, we can know that: broker A in machine A 14:28:49: key38799 was written into ledger=13810 entryID=16599 14:29:00~14:30:00 : network was down 15:32:06: key377300 was written into ledger=13810 entryID=16600

brokerB in machine B: 14:29:16: Opening managed ledger longhaul/test/persistent/testTopic 14:29:17: Closing recovered ledger 13810 at entry 16599 14:29:17: key41200 was written into ledger=13814 entryID=0 15:32:00~15:33:00: netdown

the ledger id rollback from 13814 to 13810. the question is, why broker A hold the topic ledger after zk session is timeout, how to solve this problem.

Are you willing to submit a PR?

janmee commented 1 year ago

more log for broker A, broker A believes that it is the owner of the topic while broker B works well for client.

2023-04-12T15:17:26,356+0800 [pulsar-io-4-7] INFO org.apache.pulsar.broker.service.ServerCnx - New connection from /10.14.30.111:35123 2023-04-12T15:17:26,357+0800 [pulsar-io-4-7] DEBUG org.apache.pulsar.common.protocol.PulsarDecoder - [/10.14.30.111:35123] Received cmd CONNECT 2023-04-12T15:17:26,357+0800 [pulsar-io-4-7] DEBUG org.apache.pulsar.broker.service.ServerCnx - Received CONNECT from /10.14.30.111:35123, auth enabled: false: has original principal = false, original principal = null 2023-04-12T15:17:26,357+0800 [pulsar-io-4-7] DEBUG org.apache.pulsar.common.protocol.PulsarDecoder - [/10.14.30.111:35123] Received cmd LOOKUP 2023-04-12T15:17:26,357+0800 [pulsar-io-4-7] DEBUG org.apache.pulsar.broker.service.ServerCnx - [persistent://longhaul/test/testTopic] Received Lookup from /10.14.30.111:35123 for 7 requesting listener (none) 2023-04-12T15:17:26,358+0800 [pulsar-io-4-7] DEBUG org.apache.pulsar.broker.namespace.NamespaceService - findBrokerServiceUrl: longhaul/test/0x00000000_0xffffffff - options: LookupOptions(authoritative=false, readOnly=false, loadTopicsInBundle=true, requestHttps=false, advertisedListenerName=null) 2023-04-12T15:17:26,358+0800 [pulsar-io-4-7] DEBUG org.apache.pulsar.broker.namespace.NamespaceService - Namespace bundle longhaul/test/0x00000000_0xffffffff already owned by Optional[NamespaceEphemeralData(nativeUrl=pulsar://10.14.30.106:6650, nativeUrlTls=null, httpUrl=http://10.14.30.106:8080, httpUrlTls=null, disabled=false, advertisedListeners={})] 2023-04-12T15:17:26,358+0800 [pulsar-io-4-7] DEBUG org.apache.pulsar.broker.lookup.TopicLookupBase - [persistent://longhaul/test/testTopic] Lookup result Optional[LookupResult [type=BrokerUrl, lookupData=LookupData{brokerUrl=pulsar://10.14.30.106:6650, brokerUrlTls=null, httpUrl=http://10.14.30.106:8080, httpUrlTls=null}]]

janmee commented 1 year ago

it seem in broke.conf,chose zookeeperSessionExpiredPolicy to shutdown can avoid this issue,but it still a problem in the policie reconnect

github-actions[bot] commented 1 year ago

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