sijie opened a new pull request #5621: Data is not deleted after expiration due to connected readers URL: https://github.com/apache/pulsar/pull/5621 *Problem* A problem is observed when stress testing pulsar using [pulsar-flink](https://github.com/streamnative/pulsar-flink) - No matter what TTL or retention setting is used, the data is never cleaned up. So the stress test ends up failing due to disk filled up. The root cause of the problem is described as below. when a reader is opened using `MessageId.earliest`, a non-durable cursor with position (-1, -2) is added to the cursor heap. The position `(-1, -2)` in the heap is never updated because non-durable cursors are never advanced when mark-deletions happen. So the slowest cursor position is always `(-1, -2)`, thus causing no ledger can be deleted even they are expired or over quota. <img width="720" alt="Screen Shot 2019-11-11 at 9 18 00 PM" src="https://user-images.githubusercontent.com/1217863/68605093-cc039f00-04e6-11ea-8796-2232579a1661.png"> ``` 17:38:50.349 [pulsar-io-22-7] INFO org.apache.pulsar.broker.service.ServerCnx - New connection from /172.22.248.233:42568 17:38:50.353 [pulsar-io-22-7] INFO org.apache.pulsar.broker.service.ServerCnx - [/172.22.248.233:42568] Subscribing on topic persistent://flink- pressure-test/pressure-test-ack-2/topic-1kb-partition-0 / reader-b668b71c03 17:38:50.354 [pulsar-io-22-7] INFO org.apache.pulsar.broker.service.persistent.PersistentTopic - [persistent://flink-pressure-test/pressure-test-ack- 2/topic-1kb-partition-0][reader-b668b71c03] Creating non-durable subscription at msg id -1:-1:-1:-1 17:38:50.354 [pulsar-io-22-7] INFO org.apache.bookkeeper.mledger.impl.NonDurableCursorImpl - [flink-pressure-test/pressure-test-ack-2/persistent/ topic-1kb-partition-0] Created non-durable cursor read-position=82042:0 mark-delete-position=-1:-2 17:38:50.354 [pulsar-io-22-7] INFO org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [flink-pressure-test/pressure-test-ack-2/persistent/topic- 1kb-partition-0] Opened new cursor: NonDurableCursorImpl{ledger=flink-pressure-test/pressure-test-ack-2/persistent/topic-1kb-partition-0, ackPos=-1:- 2, readPos=82042:0} 17:38:50.354 [pulsar-io-22-7] INFO org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [flink-pressure-test/pressure-test-ack-2/persistent/topic- 1kb-partition-0-reader-b668b71c03] Rewind from 82042:0 to 82042:0 ``` *Motivation* Fix the problem to make sure Pulsar honor to TTL and retention settings. *Modifications* - Fix the `startPosition` when PersistentTopic opens a non-durable cursor on `MessageId.earliest`. So the `startPosition` is (-1, -1) not (-1, -2). - Fix the `NonDurableCursorImpl` constructor to check if the position in the ledger of `MessageId.earliest`. If the provided position is in the `earliest` ledger, the mark-deleted position will be set to the previous position of first position. - Fix the `NonDurableCursorImpl` to advance ledger cursor when mark-deletion happens on a non-durable cursor. *Verify this change* Unit tests are coming. <!-- ### Contribution Checklist - Name the pull request in the form "[Issue XYZ][component] Title of the pull request", where *XYZ* should be replaced by the actual issue number. Skip *Issue XYZ* if there is no associated github issue for this pull request. Skip *component* if you are unsure about which is the best component. E.g. `[docs] Fix typo in produce method`. - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review. - Each pull request should address only one issue, not mix up code from multiple issues. - Each commit in the pull request has a meaningful commit message - Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below. **(The sections below can be removed for hotfixes of typos)** --> *(If this PR fixes a github issue, please add `Fixes #<xyz>`.)* Fixes #<xyz> *(or if this PR is one task of a github issue, please add `Master Issue: #<xyz>` to link to the master issue.)* Master Issue: #<xyz> ### Motivation *Explain here the context, and why you're making that change. What is the problem you're trying to solve.* ### Modifications *Describe the modifications you've done.* ### Verifying this change - [ ] Make sure that the change passes the CI checks. *(Please pick either of the following options)* This change is a trivial rework / code cleanup without any test coverage. *(or)* This change is already covered by existing tests, such as *(please describe tests)*. *(or)* This change added tests and can be verified as follows: *(example:)* - *Added integration tests for end-to-end deployment with large payloads (10MB)* - *Extended integration test for recovery after broker failure* ### Does this pull request potentially affect one of the following parts: *If `yes` was chosen, please highlight the changes* - Dependencies (does it add or upgrade a dependency): (yes / no) - The public API: (yes / no) - The schema: (yes / no / don't know) - The default values of configurations: (yes / no) - The wire protocol: (yes / no) - The rest endpoints: (yes / no) - The admin cli options: (yes / no) - Anything that affects deployment: (yes / no / don't know) ### Documentation - Does this pull request introduce a new feature? (yes / no) - If yes, how is the feature documented? (not applicable / docs / JavaDocs / not documented) - If a feature is not applicable for documentation, explain why? - If a feature is not documented yet in this PR, please create a followup issue for adding the documentation
---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services
