Skip to content
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

Conversation

codelipenghui
Copy link
Contributor

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.

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)

@codelipenghui codelipenghui self-assigned this Jan 5, 2022
@codelipenghui codelipenghui added this to the 2.10.0 milestone Jan 5, 2022
@github-actions github-actions bot added the doc-not-needed Your PR changes do not impact docs label Jan 5, 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 codelipenghui force-pushed the penghui/fix-skip-compacted-messages branch from 089232e to 3e2ef0a Compare January 7, 2022 01:49
@codelipenghui codelipenghui merged commit 07f131f into apache:master Jan 7, 2022
@codelipenghui codelipenghui deleted the penghui/fix-skip-compacted-messages branch January 7, 2022 05:44
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 codelipenghui added the cherry-picked/branch-2.8 Archived: 2.8 is end of life label Jan 10, 2022
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)
@codelipenghui codelipenghui added the cherry-picked/branch-2.9 Archived: 2.9 is end of life label Jan 10, 2022
@codelipenghui codelipenghui restored the penghui/fix-skip-compacted-messages branch May 17, 2022 01:22
@codelipenghui codelipenghui deleted the penghui/fix-skip-compacted-messages branch May 17, 2022 01:29
@Technoboy- Technoboy- added the cherry-picked/branch-2.7 Archived: 2.7 is end of life label Jul 5, 2022
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
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants