You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/09/23 04:19:42 UTC
[GitHub] [pulsar] TakaHiR07 opened a new issue, #17812: [Bug][broker] cursor recover to earliest by mistake
TakaHiR07 opened a new issue, #17812:
URL: https://github.com/apache/pulsar/issues/17812
### Search before asking
- [X] I searched in the [issues](https://github.com/apache/pulsar/issues) and found nothing similar.
### Version
server version : pulsar 2.9.3
client version: go client, github.com/Shopify/sarama v1.26.1
### Minimal reproduce step
Our pulsar cluster has large number of producers and consumers. When we do cluster expansion, topic start unload/load.
However, we found that only one group recover cursor to earliest by mistake, the other is correct. All the go client behaviour is the same, it should continue consume from the last consumed offset, shouldn't occur consume from earliest.
The cursor backlog is shown as follow.
![企业微信截图_91711635-9603-46ae-baf8-af85196fdbb3](https://user-images.githubusercontent.com/13505225/191889499-ccb9d1a0-6863-428e-8393-57e7b844dcac.png)
The server log is shown as follow. Sorry for masking topic and group because it contain sensitive information.
I think the ledgerId:22616096 is the actual offset need to recover, but the ledgerId:22254044 is the earliest position. Both the two ledger seems not exist because I can not get them from zk path /ledgers/00/2261/L6096 and /ledgers/00/2225/L4044.
Is it the cursor would recover to earliest once the ledger is not exist ? If true, it is improper.
`11:18:02.167 [pulsar-io-4-29] INFO org.apache.pulsar.broker.service.ServerCnx - Subscribing on topic xxx / xxx
11:18:02.179 [bookkeeper-ml-scheduler-OrderedScheduler-0-0] INFO org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [xxx] Loading cursor xxx
11:18:02.179 [bookkeeper-ml-scheduler-OrderedScheduler-0-0] INFO org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [xxx] Recovering from bookkeeper ledger cursor: xxx
11:18:02.179 [bookkeeper-ml-scheduler-OrderedScheduler-0-0] INFO org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [xxx] Consumer xxx meta-data recover from ledger 22616096
11:18:02.187 [main-EventThread] INFO org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [xxx] Opened ledger xxx. rc=0
11:18:02.202 [BookKeeperClientWorker-OrderedExecutor-12-0] INFO org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [xxx] Cursor xxx recovered to position 22254044:166985
11:18:02.205 [broker-topic-workers-OrderedScheduler-4-0] INFO org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [xxx-xxx] Rewind from 22254044:166986 to 22254044:166986
11:18:02.208 [broker-topic-workers-OrderedScheduler-4-0] INFO org.apache.pulsar.broker.service.persistent.PersistentTopic - [xxx][xxx] Created new subscription for 474
11:18:02.208 [broker-topic-workers-OrderedScheduler-4-0] INFO org.apache.pulsar.broker.service.ServerCnx - Created subscription on topic xxx / xxx
11:18:17.383 [BookKeeperClientWorker-OrderedExecutor-18-0] INFO org.apache.bookkeeper.mledger.impl.MetaStoreImpl - [xxx] [xxx] Updating cursor info ledgerId=22626176 mark-delete=22254044:166985
11:18:17.384 [bookkeeper-ml-scheduler-OrderedScheduler-0-0] INFO org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [xxx] Updated cursor xxx with ledger id 22626176 md-position=22254044:166985 rd-position=22256398:19705
11:18:17.408 [BookKeeperClientWorker-OrderedExecutor-12-0] INFO org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [xxx][xxx] Successfully closed & deleted ledger 22616096 in cursor`
### What did you expect to see?
cursor should recover to correct consumed offset.
### What did you see instead?
cursor do not recover to last consumed position, but recover to earliest
### Anything else?
_No response_
### Are you willing to submit a PR?
- [ ] I'm willing to submit a PR!
--
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: commits-unsubscribe@pulsar.apache.org.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
[GitHub] [pulsar] xiang092689 commented on issue #17812: [Bug][broker] cursor recover to earliest by mistake
Posted by "xiang092689 (via GitHub)" <gi...@apache.org>.
xiang092689 commented on issue #17812:
URL: https://github.com/apache/pulsar/issues/17812#issuecomment-1647335052
We also encountered this problem in version 2.11.0. (w:2,r:2,a:1)
the scene is we reboot a machine which lives 1 bookie and 1 broker
after rebalance, new topic owner open metadata ledger
1. cursor recover failed by LedgerRecoveryException
bk client open metadata ledger failed by (-8,0) then return LedgerRecoveryException (-10)
pulsar catch the exception and initialize cursor with earliest position recorded in zookeeper and set cursor stat as NoLedger
the cursor will close because recover failed
when cursor close, broker will persist md position to zk if cursor stat is not closed or closing.
however, there is an additional action, because open metadata cursor failed, cursor initialize cursorledger as null, cursor ledger in zookeeper will set as -1 at the same time.
2. reset cursor to the earliest
let's go to next recover round.
broker will create a new cursor ledger with the earliest position which is persisted when cursor close in step 1.
here is the whole story
i think broker process is fine, but i think there should be some tolerance when we meet LedgerRecoveryException.
bookeeper client return LedgerRecoveryException when the rc is not "timeout" and "authenticate failed" which covers too much exception and make the problem reproduce easier.
Actually, i don't know how to fix it properly
--
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: commits-unsubscribe@pulsar.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
[GitHub] [pulsar] github-actions[bot] commented on issue #17812: [Bug][broker] cursor recover to earliest by mistake
Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on issue #17812:
URL: https://github.com/apache/pulsar/issues/17812#issuecomment-1288319068
The issue had no activity for 30 days, mark with Stale label.
--
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: commits-unsubscribe@pulsar.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
[GitHub] [pulsar] xiang092689 commented on issue #17812: [Bug][broker] cursor recover to earliest by mistake
Posted by "xiang092689 (via GitHub)" <gi...@apache.org>.
xiang092689 commented on issue #17812:
URL: https://github.com/apache/pulsar/issues/17812#issuecomment-1647473867
And there is another question here.
i set retention time as 0, which means trim process often.
i don't understand why i can consume such huge mount of duplicate messages
--
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: commits-unsubscribe@pulsar.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org