pulsar
pulsar copied to clipboard
[Bug] [ml] Incorrect behavior of Topic Retention Policy
Search before asking
- [X] I searched in the issues and found nothing similar.
Read release policy
- [X] I understand that unsupported versions don't get bug fixes. I will attempt to reproduce the issue on a supported version of Pulsar client and Pulsar broker.
Version
master branch
Minimal reproduce step
public void testRetentionInMB() throws Exception {
@Cleanup("shutdown")
ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc);
ManagedLedgerConfig config = new ManagedLedgerConfig();
config.setRetentionSizeInMB(2);
config.setRetentionTime(-1, TimeUnit.SECONDS);
config.setMaxEntriesPerLedger(2);
config.setMinimumRolloverTime(0, TimeUnit.SECONDS);
@Cleanup
ManagedLedgerImpl ml = (ManagedLedgerImpl) factory.open("retention_test_ledger", config);
@Cleanup
ManagedCursor c1 = ml.openCursor("c1");
// 512K per entry, 2 entries per ledger, 1 ledger has 1MB data.
byte[] content = new byte[1024 * 512];
for (int i = 0; i < 21; i++) {
ml.addEntry(content);
}
// should be 11 ledgers.
assertEquals(ml.ledgers.size(), (21 / 2) + 1);
long firstKey = ml.ledgers.firstKey();
// ack 5 ledgers.
c1.markDelete(new PositionImpl(firstKey + 5, 0));
// trigger trim ledgers manually
CompletableFuture<Void> f = new CompletableFuture<>();
ml.trimConsumedLedgersInBackground(false, f);
f.get();
}
- For the test, we set the retention policy by size(2MB); 2 entries per ledger and each entry 512KB, each ledger is 1MB.
- Add 21 entries to the ManagedLedger, it will open 11 ledgers(I assume these ledger ID are [1, 11]).
- Set markDeletePosition to 6:0, which means [1, 5] ledgers are consumed
- Trigger trim ledger task manually
- Wait until trim task finish
What did you expect to see?
Ledgers [1, 3] deleted.
What did you see instead?
Ledgers [1, 5] deleted
Anything else?
No response
Are you willing to submit a PR?
- [ ] I'm willing to submit a PR!
@lhotari PTAL
Good description @dao-jun . This makes sense.
We need to ask the original developer of the solution for clarification whether this is a bug or not. I guess it depends a lot on how the "retention" settings are defined.
@lhotari According to https://pulsar.apache.org/docs/3.2.x/cookbooks-retention-expiry/#retention-policies and ManagedLedgerImpl#internalTrimLedgers, delete a ledger or not should be based on Total size of consumed ledgers
, base on Total size of all the ledgers
will lead to ManagedLedgerImpl#isLedgerRetentionOverSizeQuota always true
(in some cases). This is not in line with its intended meaning
/cc @merlimat @hangc0276 @codelipenghui @poorbarcode @mattisonchao @BewareMyPower @Technoboy-
@lhotari According to https://pulsar.apache.org/docs/3.2.x/cookbooks-retention-expiry/#retention-policies and ManagedLedgerImpl#internalTrimLedgers, delete a ledger or not should be based on
Total size of consumed ledgers
, base onTotal size of all the ledgers
will lead to ManagedLedgerImpl#isLedgerRetentionOverSizeQuota alwaystrue
(in some cases). This is not in line with its intended meaning
+1 @dao-jun the reported issue seems to be a bug based on this information.
The definition of the retention settings was explicitly mentioned.
The retention settings apply to all messages on topics that do not have any subscriptions, or to messages that have been acknowledged by all subscriptions. The retention policy settings do not affect unacknowledged messages on topics with subscriptions. The unacknowledged messages are controlled by the backlog quota.
See: https://github.com/apache/pulsar/pull/2206/files#diff-63f327b2f634d152b3ab8528d0d4ba49eb39276482ae60eff3f182ea2e1037d6
https://github.com/apache/pulsar/pull/14546
In the first version doc, it says: messages in all topics in the namespace, even acknowledged messages, will be retained.
This is a doc issue, since https://github.com/apache/pulsar/pull/5482/files