-
Notifications
You must be signed in to change notification settings - Fork 3.6k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Fix reader skipped remaining compacted data during the topic unloading. #13629
Merged
codelipenghui
merged 1 commit into
apache:master
from
codelipenghui:penghui/fix-skip-compacted-messages
Jan 7, 2022
Merged
Fix reader skipped remaining compacted data during the topic unloading. #13629
codelipenghui
merged 1 commit into
apache:master
from
codelipenghui:penghui/fix-skip-compacted-messages
Jan 7, 2022
Conversation
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
codelipenghui
requested review from
315157973,
BewareMyPower,
gaoran10,
hangc0276 and
merlimat
January 5, 2022 15:54
315157973
approved these changes
Jan 6, 2022
hangc0276
approved these changes
Jan 6, 2022
### Motivation To fix the reader skipping remaining compacted data while the topic been unloaded. apache#11287 fixed the data skipped issue while the reader first time to read the messages with the earliest position. But if the reader has consumed some messages from the compacted ledger but not all, the start position will not be `earliest`, the broker will rewind the cursor for the reader to the next valid position of the original topic. So the remaining messages in the compacted ledger will be skipped. Here are the logs from broker: ``` 10:44:36.035 [bookkeeper-ml-scheduler-OrderedScheduler-4-0] INFO org.apache.pulsar.broker.service.BrokerService - Created topic persistent://xxx/product-full-prod/5126 - dedup is disabled 10:44:36.035 [bookkeeper-ml-scheduler-OrderedScheduler-4-0] INFO org.apache.pulsar.broker.service.persistent.PersistentTopic - [persistent://xxx/product-full-prod/5126][xxx] Creating non-durable subscription at msg id 181759:14:-1:-1 10:44:36.035 [bookkeeper-ml-scheduler-OrderedScheduler-4-0] INFO org.apache.bookkeeper.mledger.impl.NonDurableCursorImpl - [xxx/product-full-prod/persistent/5126] Created non-durable cursor read-position=221199:0 mark-delete-position=181759:13 10:44:36.035 [bookkeeper-ml-scheduler-OrderedScheduler-4-0] INFO org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [xxx/product-full-prod/persistent/5126] Opened new cursor: NonDurableCursorImpl{ledger=xxx/product-full-prod/persistent/5126, ackPos=181759:13, readPos=221199:0} 10:44:36.035 [bookkeeper-ml-scheduler-OrderedScheduler-4-0] INFO org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [xxx/product-full-prod/persistent/5126-xxx] Rewind from 221199:0 to 221199:0 ``` There some many compacted messages after `181759:13`, but the broker will not dispatch them to the reader. The issue also can be reproduced by the unit test that added in this PR. ### Modification If the cursor with `readCompacted = true`, just rewind to the next message of the mark delete position, so that the reader can continue to read the data from the compacted ledger. ### Verification New test added for testing the reader can get all the compacted messages and non-compacted message from the topic during the topic unloading.
codelipenghui
force-pushed
the
penghui/fix-skip-compacted-messages
branch
from
January 7, 2022 01:49
089232e
to
3e2ef0a
Compare
codelipenghui
added a commit
that referenced
this pull request
Jan 10, 2022
…g. (#13629) To fix the reader skipping remaining compacted data while the topic has been unloaded. with the earliest position. But if the reader has consumed some messages from the compacted ledger but not all, the start position will not be `earliest`, the broker will rewind the cursor for the reader to the next valid position of the original topic. So the remaining messages in the compacted ledger will be skipped. Here are the logs from the broker: ``` 10:44:36.035 [bookkeeper-ml-scheduler-OrderedScheduler-4-0] INFO org.apache.pulsar.broker.service.BrokerService - Created topic persistent://xxx/product-full-prod/5126 - dedup is disabled 10:44:36.035 [bookkeeper-ml-scheduler-OrderedScheduler-4-0] INFO org.apache.pulsar.broker.service.persistent.PersistentTopic - [persistent://xxx/product-full-prod/5126][xxx] Creating non-durable subscription at msg id 181759:14:-1:-1 10:44:36.035 [bookkeeper-ml-scheduler-OrderedScheduler-4-0] INFO org.apache.bookkeeper.mledger.impl.NonDurableCursorImpl - [xxx/product-full-prod/persistent/5126] Created non-durable cursor read-position=221199:0 mark-delete-position=181759:13 10:44:36.035 [bookkeeper-ml-scheduler-OrderedScheduler-4-0] INFO org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [xxx/product-full-prod/persistent/5126] Opened new cursor: NonDurableCursorImpl{ledger=xxx/product-full-prod/persistent/5126, ackPos=181759:13, readPos=221199:0} 10:44:36.035 [bookkeeper-ml-scheduler-OrderedScheduler-4-0] INFO org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [xxx/product-full-prod/persistent/5126-xxx] Rewind from 221199:0 to 221199:0 ``` There some many compacted messages after `181759:13`, but the broker will not dispatch them to the reader. The issue also can be reproduced by the unit test that was added in this PR. If the cursor with `readCompacted = true`, just rewind to the next message of the mark delete position, so that the reader can continue to read the data from the compacted ledger. A new test added for testing the reader can get all the compacted messages and non-compacted messages from the topic during the topic unloading. (cherry picked from commit 07f131f)
codelipenghui
added a commit
that referenced
this pull request
Jan 10, 2022
…g. (#13629) ### Motivation To fix the reader skipping remaining compacted data while the topic has been unloaded. #11287 fixed the data skipped issue while the reader first time to read the messages with the earliest position. But if the reader has consumed some messages from the compacted ledger but not all, the start position will not be `earliest`, the broker will rewind the cursor for the reader to the next valid position of the original topic. So the remaining messages in the compacted ledger will be skipped. Here are the logs from the broker: ``` 10:44:36.035 [bookkeeper-ml-scheduler-OrderedScheduler-4-0] INFO org.apache.pulsar.broker.service.BrokerService - Created topic persistent://xxx/product-full-prod/5126 - dedup is disabled 10:44:36.035 [bookkeeper-ml-scheduler-OrderedScheduler-4-0] INFO org.apache.pulsar.broker.service.persistent.PersistentTopic - [persistent://xxx/product-full-prod/5126][xxx] Creating non-durable subscription at msg id 181759:14:-1:-1 10:44:36.035 [bookkeeper-ml-scheduler-OrderedScheduler-4-0] INFO org.apache.bookkeeper.mledger.impl.NonDurableCursorImpl - [xxx/product-full-prod/persistent/5126] Created non-durable cursor read-position=221199:0 mark-delete-position=181759:13 10:44:36.035 [bookkeeper-ml-scheduler-OrderedScheduler-4-0] INFO org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [xxx/product-full-prod/persistent/5126] Opened new cursor: NonDurableCursorImpl{ledger=xxx/product-full-prod/persistent/5126, ackPos=181759:13, readPos=221199:0} 10:44:36.035 [bookkeeper-ml-scheduler-OrderedScheduler-4-0] INFO org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [xxx/product-full-prod/persistent/5126-xxx] Rewind from 221199:0 to 221199:0 ``` There some many compacted messages after `181759:13`, but the broker will not dispatch them to the reader. The issue also can be reproduced by the unit test that was added in this PR. ### Modification If the cursor with `readCompacted = true`, just rewind to the next message of the mark delete position, so that the reader can continue to read the data from the compacted ledger. ### Verification A new test added for testing the reader can get all the compacted messages and non-compacted messages from the topic during the topic unloading. (cherry picked from commit 07f131f)
3 tasks
15 tasks
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Labels
area/broker
area/compaction
cherry-picked/branch-2.7
Archived: 2.7 is end of life
cherry-picked/branch-2.8
Archived: 2.8 is end of life
cherry-picked/branch-2.9
Archived: 2.9 is end of life
doc-not-needed
Your PR changes do not impact docs
release/2.7.5
release/2.8.3
release/2.9.2
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Motivation
To fix the reader skipping remaining compacted data while the topic has been unloaded.
#11287 fixed the data skipped issue while the reader first time to read the messages
with the earliest position. But if the reader has consumed some messages from the
compacted ledger but not all, the start position will not be
earliest
, the brokerwill rewind the cursor for the reader to the next valid position of the original topic.
So the remaining messages in the compacted ledger will be skipped.
Here are the logs from the broker:
There some many compacted messages after
181759:13
, but the broker will not dispatch them to the reader.The issue also can be reproduced by the unit test that was added in this PR.
Modification
If the cursor with
readCompacted = true
, just rewind to the next message of the mark delete position,so that the reader can continue to read the data from the compacted ledger.
Verification
A new test added for testing the reader can get all the compacted messages and non-compacted messages from the topic during the topic unloading.
Documentation
Check the box below or label this PR directly (if you have committer privilege).
Need to update docs?
doc-required
(If you need help on updating docs, create a doc issue)
no-need-doc
(Please explain why)
doc
(If this PR contains doc changes)