Skip to content

Conversation

@wangjialing218
Copy link
Contributor

@wangjialing218 wangjialing218 commented Jan 6, 2021

Motivation

Fixes #9057

When current ledger closed, if there is no incoming traffic, the read position of the cursor is still point to the last entry of the closed ledger, that casue the slowestReaderLedgerId point to the closed ledger in internalTrimConsumedLedgers() and fail to delete the closed ledger.

Modifications

When close current ledger, if cursor's mark delete position point to the last entry of current ledger, move the read position to the new created ledger.

Verifying this change

add test case: testDeletionAfterLedgerClosedAndRetention()

@wangjialing218
Copy link
Contributor Author

/pulsarbot run-failure-checks

@wangjialing218 wangjialing218 force-pushed the branch-refine-retention branch from 55f31df to 15e0c84 Compare January 7, 2021 05:56
@wangjialing218
Copy link
Contributor Author

/pulsarbot run-failure-checks

2 similar comments
@wangjialing218
Copy link
Contributor Author

/pulsarbot run-failure-checks

@wangjialing218
Copy link
Contributor Author

/pulsarbot run-failure-checks

wangjialing added 2 commits January 8, 2021 11:31
update read position when mark delete position point to last entry

update read position when mark delete position point to last entry

fix the test case for rollover current ledger
@wangjialing218 wangjialing218 force-pushed the branch-refine-retention branch from 1116ae1 to 103a5da Compare January 8, 2021 04:49
@wangjialing218
Copy link
Contributor Author

/pulsarbot run-failure-checks

@codelipenghui codelipenghui merged commit 0e5c536 into apache:master Jan 10, 2021
codelipenghui pushed a commit that referenced this pull request Jan 14, 2021
Fixes #9057

When current ledger closed, if there is no incoming traffic, the read position of the cursor is still point to the last entry of the closed ledger,  that casue the `slowestReaderLedgerId` point to the closed ledger in `internalTrimConsumedLedgers()` and fail to delete the closed ledger.

When close current ledger, if cursor's mark delete position point to the last entry of current ledger, move the read position to the new created ledger.

add test case: testDeletionAfterLedgerClosedAndRetention()

(cherry picked from commit 0e5c536)
@codelipenghui codelipenghui added the cherry-picked/branch-2.7 Archived: 2.7 is end of life label Jan 14, 2021
freeznet pushed a commit to streamnative/pulsar-archived that referenced this pull request Jan 14, 2021
Fixes apache#9057

When current ledger closed, if there is no incoming traffic, the read position of the cursor is still point to the last entry of the closed ledger,  that casue the `slowestReaderLedgerId` point to the closed ledger in `internalTrimConsumedLedgers()` and fail to delete the closed ledger.

When close current ledger, if cursor's mark delete position point to the last entry of current ledger, move the read position to the new created ledger.

add test case: testDeletionAfterLedgerClosedAndRetention()

(cherry picked from commit 0e5c536)
codelipenghui pushed a commit that referenced this pull request Jan 28, 2021
Fixes #9057

When current ledger closed, if there is no incoming traffic, the read position of the cursor is still point to the last entry of the closed ledger,  that casue the `slowestReaderLedgerId` point to the closed ledger in `internalTrimConsumedLedgers()` and fail to delete the closed ledger.

When close current ledger, if cursor's mark delete position point to the last entry of current ledger, move the read position to the new created ledger.

add test case: testDeletionAfterLedgerClosedAndRetention()

(cherry picked from commit 0e5c536)
merlimat pushed a commit to merlimat/pulsar that referenced this pull request Apr 6, 2021
### Motivation
Fixes apache#9057

When current ledger closed, if there is no incoming traffic, the read position of the cursor is still point to the last entry of the closed ledger,  that casue the `slowestReaderLedgerId` point to the closed ledger in `internalTrimConsumedLedgers()` and fail to delete the closed ledger.

### Modifications
When close current ledger, if cursor's mark delete position point to the last entry of current ledger, move the read position to the new created ledger. 

### Verifying this change
add test case: testDeletionAfterLedgerClosedAndRetention()
codelipenghui pushed a commit that referenced this pull request May 23, 2021
…0087)

This pull request resolves #10086 

## Motivation

Although #7111 and #9136 solved the problems of 
1. current ledger is full and cannot be rolled
2. cursor has subscribed to an expired ledger, which makes the cleaning thread unable to clean up

respectively. However, when we close and open a managed-ledger(such like brokers shutdown unexpectedly or topics unload due to rebalance), the managed-ledger will create an empty ledger after initialization causing the current cleanup logic failed to take effect. Therefore I think we need a more unified entrance to solve the cursor update problem, and then further solve the problem of clearing expired ledgers.

## Expected Behavior

1. Able to cleanup expired data after managed-ledger re-open
2. A more unified entrance to implement cursor update

## Modification

1. move the cursor update logic to `internalTrimConsumedLedgers` from `ManagedLedgerImpl`'s callback `createComplete`
2. update `lastConfirmedEntry` if necessary when updating cursors
yangl pushed a commit to yangl/pulsar that referenced this pull request Jun 23, 2021
…ache#10087)

This pull request resolves apache#10086 

## Motivation

Although apache#7111 and apache#9136 solved the problems of 
1. current ledger is full and cannot be rolled
2. cursor has subscribed to an expired ledger, which makes the cleaning thread unable to clean up

respectively. However, when we close and open a managed-ledger(such like brokers shutdown unexpectedly or topics unload due to rebalance), the managed-ledger will create an empty ledger after initialization causing the current cleanup logic failed to take effect. Therefore I think we need a more unified entrance to solve the cursor update problem, and then further solve the problem of clearing expired ledgers.

## Expected Behavior

1. Able to cleanup expired data after managed-ledger re-open
2. A more unified entrance to implement cursor update

## Modification

1. move the cursor update logic to `internalTrimConsumedLedgers` from `ManagedLedgerImpl`'s callback `createComplete`
2. update `lastConfirmedEntry` if necessary when updating cursors
bharanic-dev pushed a commit to bharanic-dev/pulsar that referenced this pull request Mar 18, 2022
…ache#10087)

This pull request resolves apache#10086 

## Motivation

Although apache#7111 and apache#9136 solved the problems of 
1. current ledger is full and cannot be rolled
2. cursor has subscribed to an expired ledger, which makes the cleaning thread unable to clean up

respectively. However, when we close and open a managed-ledger(such like brokers shutdown unexpectedly or topics unload due to rebalance), the managed-ledger will create an empty ledger after initialization causing the current cleanup logic failed to take effect. Therefore I think we need a more unified entrance to solve the cursor update problem, and then further solve the problem of clearing expired ledgers.

## Expected Behavior

1. Able to cleanup expired data after managed-ledger re-open
2. A more unified entrance to implement cursor update

## Modification

1. move the cursor update logic to `internalTrimConsumedLedgers` from `ManagedLedgerImpl`'s callback `createComplete`
2. update `lastConfirmedEntry` if necessary when updating cursors
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Closed ledger did not deleted after retention period when no incoming traffic

3 participants