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

[improve] [broker] Add additionalSystemCursorNames ignore list for TTL check #22614

Merged
Show file tree
Hide file tree
Changes from 5 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -652,6 +652,12 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece
)
private int ttlDurationDefaultInSeconds = 0;

@FieldContext(
category = CATEGORY_POLICIES,
doc = "System subscriptions that will be ignored by ttl check. Default is empty."
lhotari marked this conversation as resolved.
Show resolved Hide resolved
)
private Set<String> systemCursorNames = Set.of("__compaction");

@FieldContext(
category = CATEGORY_POLICIES,
dynamic = true,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.CancellationException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
Expand Down Expand Up @@ -278,6 +279,7 @@ protected TopicStatsHelper initialValue() {
private final ExecutorService orderedExecutor;

private volatile CloseFutures closeFutures;
private Set<String> systemCursorNames = new TreeSet<>();

@Getter
private final PersistentTopicMetrics persistentTopicMetrics = new PersistentTopicMetrics();
Expand Down Expand Up @@ -413,6 +415,7 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS
} else {
shadowSourceTopic = null;
}
systemCursorNames = brokerService.pulsar().getConfiguration().getSystemCursorNames();
}

@Override
Expand Down Expand Up @@ -1933,7 +1936,7 @@ public void checkMessageExpiry() {
int messageTtlInSeconds = topicPolicies.getMessageTTLInSeconds().get();
if (messageTtlInSeconds != 0) {
subscriptions.forEach((__, sub) -> {
if (!isCompactionSubscription(sub.getName())) {
if (systemCursorNames.contains(sub.getName())) {
lhotari marked this conversation as resolved.
Show resolved Hide resolved
sub.expireMessages(messageTtlInSeconds);
}
});
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,15 +23,23 @@
import static org.mockito.Mockito.verify;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNotNull;
import static org.testng.Assert.assertTrue;

import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import lombok.Cleanup;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
import org.apache.pulsar.broker.service.persistent.PersistentSubscription;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.client.api.MessageId;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.client.impl.MessageIdImpl;
import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats.CursorStats;
import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats;
Expand Down Expand Up @@ -138,4 +146,92 @@ public void testTTLPoliciesUpdate() throws Exception {
topicRefMock.onUpdate(topicPolicies);
verify(topicRefMock, times(2)).checkMessageExpiry();
}

@Test
public void testTtlFilteredByIgnoreSubscriptions() throws Exception {
String topicName = "persistent://prop/ns-abc/testTTLFilteredByIgnoreSubscriptions";
String subName = "__SUB_FILTER";
cleanup();
Set<String> ignoredSubscriptions = new HashSet<>();
ignoredSubscriptions.add(subName);
int defaultTtl = 5;
conf.setSubscriptionsIgnoredByTtl(ignoredSubscriptions);
conf.setTtlDurationDefaultInSeconds(defaultTtl);
super.baseSetup();

pulsarClient.newConsumer(Schema.STRING).topic(topicName).subscriptionName(subName)
.subscribe().close();

@Cleanup
org.apache.pulsar.client.api.Producer<String> producer = pulsarClient.newProducer(Schema.STRING)
.enableBatching(false).topic(topicName).create();

final int messages = 10;

for (int i = 0; i < messages; i++) {
String message = "my-message-" + i;
producer.send(message);
}
producer.close();

Optional<Topic> topic = pulsar.getBrokerService().getTopicReference(topicName);
assertTrue(topic.isPresent());
PersistentSubscription subscription = (PersistentSubscription) topic.get().getSubscription(subName);

Thread.sleep((defaultTtl - 1) * 1000);
topic.get().checkMessageExpiry();
// Wait the message expire task done and make sure the message does not expire early.
Thread.sleep(1000);
assertEquals(subscription.getNumberOfEntriesInBacklog(false), 10);
Thread.sleep(2000);
topic.get().checkMessageExpiry();
// Wait the message expire task done.
retryStrategically((test) -> subscription.getNumberOfEntriesInBacklog(false) == 0, 5, 200);
// The message should not expire because the subscription is ignored.
assertEquals(subscription.getNumberOfEntriesInBacklog(false), 10);

conf.setSubscriptionsIgnoredByTtl(new TreeSet<>());
}

@Test
public void testTtlWithoutIgnoreSubscriptions() throws Exception {
String topicName = "persistent://prop/ns-abc/testTTLWithoutIgnoreSubscriptions";
String subName = "__SUB_FILTER";
cleanup();
int defaultTtl = 5;
conf.setTtlDurationDefaultInSeconds(defaultTtl);
conf.setBrokerDeleteInactiveTopicsEnabled(false);
super.baseSetup();

pulsarClient.newConsumer(Schema.STRING).topic(topicName).subscriptionName(subName)
.subscribe().close();

@Cleanup
org.apache.pulsar.client.api.Producer<String> producer = pulsarClient.newProducer(Schema.STRING)
.enableBatching(false).topic(topicName).create();

final int messages = 10;

for (int i = 0; i < messages; i++) {
String message = "my-message-" + i;
producer.send(message);
}
producer.close();

Optional<Topic> topic = pulsar.getBrokerService().getTopicReference(topicName);
assertTrue(topic.isPresent());
PersistentSubscription subscription = (PersistentSubscription) topic.get().getSubscription(subName);

Thread.sleep((defaultTtl - 1) * 1000);
topic.get().checkMessageExpiry();
// Wait the message expire task done and make sure the message does not expire early.
Thread.sleep(1000);
assertEquals(subscription.getNumberOfEntriesInBacklog(false), 10);
Thread.sleep(2000);
topic.get().checkMessageExpiry();
// Wait the message expire task done and make sure the message expired.
retryStrategically((test) -> subscription.getNumberOfEntriesInBacklog(false) == 0, 5, 200);
assertEquals(subscription.getNumberOfEntriesInBacklog(false), 0);
}

}
Loading