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

Data is not deleted after expiration due to connected readers #5621

Merged
merged 6 commits into from
Nov 12, 2019

Conversation

sijie
Copy link
Member

@sijie sijie commented Nov 11, 2019

Problem

A problem is observed when stress testing pulsar using 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.

Screen Shot 2019-11-11 at 9 18 00 PM

17:38:50.349 [pulsar-io-22-7] INFO  org.apache.pulsar.broker.service.ServerCnx - New connection from /192.168.0.1:42568
17:38:50.353 [pulsar-io-22-7] INFO  org.apache.pulsar.broker.service.ServerCnx - [/192.168.0.1: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

Fixes #5558

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

Add a unit test to simulate the mixture of durable and non-durable cursors, and verify the fix address the problem.

*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.

*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.
@sijie sijie added type/bug The PR fixed a bug or issue reported a bug area/client labels Nov 11, 2019
@sijie sijie added this to the 2.4.3 milestone Nov 11, 2019
@sijie sijie self-assigned this Nov 11, 2019
@sijie
Copy link
Member Author

sijie commented Nov 11, 2019

this pull request is now ready for review. The unit test is added to reproduce the issue and used for verifying the fix works.

@sijie
Copy link
Member Author

sijie commented Nov 11, 2019

This pull request also fixes the root cause of #5558

@sijie
Copy link
Member Author

sijie commented Nov 12, 2019

run java8 tests

@codelipenghui
Copy link
Contributor

Seems failed unit tests are related to this change.

@sijie
Copy link
Member Author

sijie commented Nov 12, 2019

@codelipenghui fixed the NonDurableCursorTest

@codelipenghui
Copy link
Contributor

run java8 tests

@codelipenghui
Copy link
Contributor

run cpp tests

@sijie
Copy link
Member Author

sijie commented Nov 12, 2019

retest this please

@sijie
Copy link
Member Author

sijie commented Nov 12, 2019

run integration tests

@codelipenghui
Copy link
Contributor

run java8 tests
run integration tests

@sijie
Copy link
Member Author

sijie commented Nov 12, 2019

run java8 tests

@sijie sijie merged commit 3e7cb68 into apache:master Nov 12, 2019
@sijie sijie modified the milestones: 2.4.3, 2.4.2 Nov 12, 2019
wolfstudy pushed a commit that referenced this pull request Nov 20, 2019
* Data is not deleted after expiration due to connected readers

*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.

*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.

(cherry picked from commit 3e7cb68)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area/client type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
3 participants