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

[Broker] Apply PolicyHierarchyValue to inactiveTopicPolicies #12687

Merged
merged 2 commits into from
Nov 11, 2021
Merged
Show file tree
Hide file tree
Changes from all 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 @@ -51,10 +51,9 @@
import org.apache.pulsar.broker.stats.prometheus.metrics.Summary;
import org.apache.pulsar.broker.systopic.SystemTopicClient;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.InactiveTopicDeleteMode;
import org.apache.pulsar.common.policies.data.HierarchyTopicPolicies;
import org.apache.pulsar.common.policies.data.InactiveTopicPolicies;
import org.apache.pulsar.common.policies.data.Policies;
import org.apache.pulsar.common.policies.data.PolicyHierarchyValue;
import org.apache.pulsar.common.policies.data.PublishRate;
import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy;
import org.apache.pulsar.common.policies.data.TopicPolicies;
Expand Down Expand Up @@ -82,8 +81,7 @@ public abstract class AbstractTopic implements Topic {

protected volatile boolean isFenced;

// Inactive topic policies
protected InactiveTopicPolicies inactiveTopicPolicies = new InactiveTopicPolicies();
protected final HierarchyTopicPolicies topicPolicies;

// Timestamp of when this topic was last seen active
protected volatile long lastActive;
Expand All @@ -106,10 +104,6 @@ public abstract class AbstractTopic implements Topic {

protected volatile int maxUnackedMessagesOnConsumerAppilied = 0;

@VisibleForTesting
@Getter
protected volatile PolicyHierarchyValue<Integer> maxSubscriptionsPerTopic;

protected volatile PublishRateLimiter topicPublishRateLimiter;

protected volatile ResourceGroupPublishLimiter resourceGroupPublishLimiter;
Expand Down Expand Up @@ -143,23 +137,22 @@ public AbstractTopic(String topic, BrokerService brokerService) {
this.brokerService = brokerService;
this.producers = new ConcurrentHashMap<>();
this.isFenced = false;
this.replicatorPrefix = brokerService.pulsar().getConfiguration().getReplicatorPrefix();
this.inactiveTopicPolicies.setDeleteWhileInactive(brokerService.pulsar().getConfiguration()
.isBrokerDeleteInactiveTopicsEnabled());
this.inactiveTopicPolicies.setMaxInactiveDurationSeconds(brokerService.pulsar().getConfiguration()
.getBrokerDeleteInactiveTopicsMaxInactiveDurationSeconds());
this.inactiveTopicPolicies.setInactiveTopicDeleteMode(brokerService.pulsar().getConfiguration()
.getBrokerDeleteInactiveTopicsMode());
this.topicMaxMessageSizeCheckIntervalMs = TimeUnit.SECONDS.toMillis(brokerService.pulsar().getConfiguration()
.getMaxMessageSizeCheckIntervalInSeconds());

maxSubscriptionsPerTopic = new PolicyHierarchyValue<>();
maxSubscriptionsPerTopic.updateBrokerValue(brokerService.pulsar().getConfiguration()
.getMaxSubscriptionsPerTopic());
ServiceConfiguration config = brokerService.pulsar().getConfiguration();
this.replicatorPrefix = config.getReplicatorPrefix();


topicPolicies = new HierarchyTopicPolicies();
topicPolicies.getInactiveTopicPolicies().updateBrokerValue(new InactiveTopicPolicies(
config.getBrokerDeleteInactiveTopicsMode(),
config.getBrokerDeleteInactiveTopicsMaxInactiveDurationSeconds(),
config.isBrokerDeleteInactiveTopicsEnabled()));
topicPolicies.getMaxSubscriptionsPerTopic().updateBrokerValue(config.getMaxSubscriptionsPerTopic());

this.topicMaxMessageSizeCheckIntervalMs = TimeUnit.SECONDS.toMillis(
config.getMaxMessageSizeCheckIntervalInSeconds());

this.lastActive = System.nanoTime();
this.preciseTopicPublishRateLimitingEnable =
brokerService.pulsar().getConfiguration().isPreciseTopicPublishRateLimiterEnable();
this.preciseTopicPublishRateLimitingEnable = config.isPreciseTopicPublishRateLimiterEnable();
updatePublishDispatcher(Optional.empty());
}

Expand Down Expand Up @@ -869,30 +862,19 @@ public long getBytesOutCounter() {
}

public boolean isDeleteWhileInactive() {
return this.inactiveTopicPolicies.isDeleteWhileInactive();
return topicPolicies.getInactiveTopicPolicies().get().isDeleteWhileInactive();
}

public boolean deletePartitionedTopicMetadataWhileInactive() {
return brokerService.pulsar().getConfiguration().isBrokerDeleteInactivePartitionedTopicMetadataEnabled();
}

public void setDeleteWhileInactive(boolean deleteWhileInactive) {
this.inactiveTopicPolicies.setDeleteWhileInactive(deleteWhileInactive);
}

protected abstract boolean isTerminated();

private static final Logger log = LoggerFactory.getLogger(AbstractTopic.class);

public InactiveTopicPolicies getInactiveTopicPolicies() {
return inactiveTopicPolicies;
}

public void resetInactiveTopicPolicies(InactiveTopicDeleteMode inactiveTopicDeleteMode
, int maxInactiveDurationSeconds, boolean deleteWhileInactive) {
inactiveTopicPolicies.setInactiveTopicDeleteMode(inactiveTopicDeleteMode);
inactiveTopicPolicies.setMaxInactiveDurationSeconds(maxInactiveDurationSeconds);
inactiveTopicPolicies.setDeleteWhileInactive(deleteWhileInactive);
return topicPolicies.getInactiveTopicPolicies().get();
}

/**
Expand Down Expand Up @@ -953,4 +935,9 @@ protected void updatePublishDispatcher(PublishRate publishRate) {
enableProducerReadForPublishRateLimiting();
}
}

@VisibleForTesting
public HierarchyTopicPolicies getHierarchyTopicPolicies() {
return topicPolicies;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,6 @@
import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.Position;
import org.apache.pulsar.broker.PulsarServerException;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.resources.NamespaceResources;
import org.apache.pulsar.broker.service.AbstractReplicator;
import org.apache.pulsar.broker.service.AbstractTopic;
Expand Down Expand Up @@ -152,11 +151,8 @@ public CompletableFuture<Void> initialize() {
Policies policies = optPolicies.get();
isEncryptionRequired = policies.encryption_required;
isAllowAutoUpdateSchema = policies.is_allow_auto_update_schema;
if (policies.inactive_topic_policies != null) {
inactiveTopicPolicies = policies.inactive_topic_policies;
}
topicPolicies.getInactiveTopicPolicies().updateNamespaceValue(policies.inactive_topic_policies);
setSchemaCompatibilityStrategy(policies);

schemaValidationEnforced = policies.schema_validation_enforced;
}
});
Expand Down Expand Up @@ -866,7 +862,7 @@ public void checkGC() {
// This topic is not included in GC
return;
}
int maxInactiveDurationInSec = inactiveTopicPolicies.getMaxInactiveDurationSeconds();
int maxInactiveDurationInSec = topicPolicies.getInactiveTopicPolicies().get().getMaxInactiveDurationSeconds();
if (isActive()) {
lastActive = System.nanoTime();
} else {
Expand Down Expand Up @@ -979,14 +975,8 @@ public CompletableFuture<Void> onPoliciesUpdate(Policies data) {
});
subscriptions.forEach((subName, sub) -> sub.getConsumers().forEach(Consumer::checkPermissions));

if (data.inactive_topic_policies != null) {
this.inactiveTopicPolicies = data.inactive_topic_policies;
} else {
ServiceConfiguration cfg = brokerService.getPulsar().getConfiguration();
resetInactiveTopicPolicies(cfg.getBrokerDeleteInactiveTopicsMode()
, cfg.getBrokerDeleteInactiveTopicsMaxInactiveDurationSeconds(),
cfg.isBrokerDeleteInactiveTopicsEnabled());
}
this.topicPolicies.getInactiveTopicPolicies().updateNamespaceValue(data.inactive_topic_policies);

return checkReplicationAndRetryOnFailure();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,6 @@
import org.apache.bookkeeper.net.BookieId;
import org.apache.commons.lang3.StringUtils;
import org.apache.pulsar.broker.PulsarServerException;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.resources.NamespaceResources.PartitionedTopicResources;
import org.apache.pulsar.broker.service.AbstractReplicator;
import org.apache.pulsar.broker.service.AbstractTopic;
Expand Down Expand Up @@ -324,9 +323,9 @@ public CompletableFuture<Void> initialize() {
isAllowAutoUpdateSchema = policies.is_allow_auto_update_schema;

schemaValidationEnforced = policies.schema_validation_enforced;
if (policies.inactive_topic_policies != null) {
inactiveTopicPolicies = policies.inactive_topic_policies;
}

topicPolicies.getInactiveTopicPolicies().updateNamespaceValue(policies.inactive_topic_policies);

updateUnackedMessagesAppliedOnSubscription(policies);
updateUnackedMessagesExceededOnConsumer(policies);
}).exceptionally(ex -> {
Expand Down Expand Up @@ -2198,8 +2197,9 @@ public void checkGC() {
// This topic is not included in GC
return;
}
InactiveTopicDeleteMode deleteMode = inactiveTopicPolicies.getInactiveTopicDeleteMode();
int maxInactiveDurationInSec = inactiveTopicPolicies.getMaxInactiveDurationSeconds();
InactiveTopicDeleteMode deleteMode =
topicPolicies.getInactiveTopicPolicies().get().getInactiveTopicDeleteMode();
int maxInactiveDurationInSec = topicPolicies.getInactiveTopicPolicies().get().getMaxInactiveDurationSeconds();
if (isActive(deleteMode)) {
lastActive = System.nanoTime();
} else if (System.nanoTime() - lastActive < TimeUnit.SECONDS.toNanos(maxInactiveDurationInSec)) {
Expand Down Expand Up @@ -2407,24 +2407,16 @@ public CompletableFuture<Void> onPoliciesUpdate(Policies data) {
schemaValidationEnforced = data.schema_validation_enforced;
updateUnackedMessagesAppliedOnSubscription(data);
updateUnackedMessagesExceededOnConsumer(data);
maxSubscriptionsPerTopic.updateNamespaceValue(data.max_subscriptions_per_topic);
this.topicPolicies.getMaxSubscriptionsPerTopic().updateNamespaceValue(data.max_subscriptions_per_topic);

if (data.delayed_delivery_policies != null) {
delayedDeliveryTickTimeMillis = data.delayed_delivery_policies.getTickTime();
delayedDeliveryEnabled = data.delayed_delivery_policies.isActive();
}
//If the topic-level policy already exists, the namespace-level policy cannot override the topic-level policy.
Optional<TopicPolicies> topicPolicies = getTopicPolicies();
if (data.inactive_topic_policies != null) {
if (!topicPolicies.isPresent() || !topicPolicies.get().isInactiveTopicPoliciesSet()) {
this.inactiveTopicPolicies = data.inactive_topic_policies;
}
} else {
ServiceConfiguration cfg = brokerService.getPulsar().getConfiguration();
resetInactiveTopicPolicies(cfg.getBrokerDeleteInactiveTopicsMode()
, cfg.getBrokerDeleteInactiveTopicsMaxInactiveDurationSeconds(),
cfg.isBrokerDeleteInactiveTopicsEnabled());
}

this.topicPolicies.getInactiveTopicPolicies().updateNamespaceValue(data.inactive_topic_policies);

initializeRateLimiterIfNeeded(Optional.ofNullable(data));

Expand Down Expand Up @@ -3096,20 +3088,10 @@ public void onUpdate(TopicPolicies policies) {
updateMaxPublishRate(namespacePolicies.orElse(null));
}

maxSubscriptionsPerTopic.updateTopicValue(policies.getMaxSubscriptionsPerTopic());
topicPolicies.getMaxSubscriptionsPerTopic().updateTopicValue(policies.getMaxSubscriptionsPerTopic());

topicPolicies.getInactiveTopicPolicies().updateTopicValue(policies.getInactiveTopicPolicies());

if (policies.isInactiveTopicPoliciesSet()) {
inactiveTopicPolicies = policies.getInactiveTopicPolicies();
} else if (namespacePolicies.isPresent() && namespacePolicies.get().inactive_topic_policies != null) {
//topic-level policies is null , so use namespace-level
inactiveTopicPolicies = namespacePolicies.get().inactive_topic_policies;
} else {
//namespace-level policies is null , so use broker level
ServiceConfiguration cfg = brokerService.getPulsar().getConfiguration();
resetInactiveTopicPolicies(cfg.getBrokerDeleteInactiveTopicsMode()
, cfg.getBrokerDeleteInactiveTopicsMaxInactiveDurationSeconds(),
cfg.isBrokerDeleteInactiveTopicsEnabled());
}
updateUnackedMessagesAppliedOnSubscription(namespacePolicies.orElse(null));
initializeTopicSubscribeRateLimiterIfNeeded(Optional.ofNullable(policies));
if (this.subscribeRateLimiter.isPresent()) {
Expand Down Expand Up @@ -3180,7 +3162,7 @@ private boolean checkMaxSubscriptionsPerTopicExceed(String subscriptionName) {
return false;
}

Integer maxSubsPerTopic = maxSubscriptionsPerTopic.get();
Integer maxSubsPerTopic = topicPolicies.getMaxSubscriptionsPerTopic().get();

if (maxSubsPerTopic != null && maxSubsPerTopic > 0) {
return subscriptions != null && subscriptions.size() >= maxSubsPerTopic;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2146,7 +2146,7 @@ public void testMaxSubscriptionsPerTopic() throws Exception {
admin.namespaces().setMaxSubscriptionsPerTopic(myNamespace, namespaceLevelMaxSub);
PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topic).get().get();
Awaitility.await().until(() -> Integer.valueOf(namespaceLevelMaxSub)
.equals(persistentTopic.getMaxSubscriptionsPerTopic().getNamespaceValue()));
.equals(persistentTopic.getHierarchyTopicPolicies().getMaxSubscriptionsPerTopic().getNamespaceValue()));

try (PulsarClient client = PulsarClient.builder().operationTimeout(1000, TimeUnit.MILLISECONDS)
.serviceUrl(brokerUrl.toString()).build()) {
Expand All @@ -2173,7 +2173,10 @@ public void testMaxSubscriptionsPerTopic() throws Exception {
}
//Removed namespace-level policy, broker-level should take effect
admin.namespaces().removeMaxSubscriptionsPerTopic(myNamespace);
Awaitility.await().until(() -> persistentTopic.getMaxSubscriptionsPerTopic().getNamespaceValue() == null);
Awaitility.await().until(() -> persistentTopic
.getHierarchyTopicPolicies()
.getMaxSubscriptionsPerTopic()
.getNamespaceValue() == null);
consumerList.add(pulsarClient.newConsumer(Schema.STRING)
.subscriptionName(UUID.randomUUID().toString()).topic(topic).subscribe());
assertEquals(consumerList.size(), brokerLevelMaxSub);
Expand Down
Loading