Skip to content

Commit

Permalink
Optimize topic policy with HierarchyTopicPolicies about dispatchRate
Browse files Browse the repository at this point in the history
  • Loading branch information
AnonHxy committed Feb 9, 2022
1 parent d22ff4f commit ab16e7e
Show file tree
Hide file tree
Showing 5 changed files with 58 additions and 16 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@
import org.apache.pulsar.common.policies.data.RetentionPolicies;
import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy;
import org.apache.pulsar.common.policies.data.TopicPolicies;
import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl;
import org.apache.pulsar.common.protocol.schema.SchemaData;
import org.apache.pulsar.common.protocol.schema.SchemaVersion;
import org.apache.pulsar.common.util.FutureUtil;
Expand Down Expand Up @@ -155,6 +156,10 @@ public SchemaCompatibilityStrategy getSchemaCompatibilityStrategy() {
return this.topicPolicies.getSchemaCompatibilityStrategy().get();
}

public DispatchRateImpl getDispatchRate() {
return this.topicPolicies.getDispatchRate().get();
}

private SchemaCompatibilityStrategy formatSchemaCompatibilityStrategy(SchemaCompatibilityStrategy strategy) {
return strategy == SchemaCompatibilityStrategy.UNDEFINED ? null : strategy;
}
Expand Down Expand Up @@ -187,6 +192,7 @@ protected void updateTopicPolicy(TopicPolicies data) {
topicPolicies.getDelayedDeliveryEnabled().updateTopicValue(data.getDelayedDeliveryEnabled());
topicPolicies.getDelayedDeliveryTickTimeMillis().updateTopicValue(data.getDelayedDeliveryTickTimeMillis());
topicPolicies.getCompactionThreshold().updateTopicValue(data.getCompactionThreshold());
topicPolicies.getDispatchRate().updateTopicValue(normalize(data.getDispatchRate()));
}

protected void updateTopicPolicyByNamespacePolicy(Policies namespacePolicies) {
Expand Down Expand Up @@ -224,6 +230,25 @@ protected void updateTopicPolicyByNamespacePolicy(Policies namespacePolicies) {
type -> this.topicPolicies.getBackLogQuotaMap().get(type)
.updateNamespaceValue(MapUtils.getObject(namespacePolicies.backlog_quota_map, type)));
updateSchemaCompatibilityStrategyNamespaceValue(namespacePolicies);
updateNamespaceDispatchRate(namespacePolicies, brokerService.getPulsar().getConfig().getClusterName());
}

private void updateNamespaceDispatchRate(Policies namespacePolicies, String cluster) {
DispatchRateImpl dispatchRate = namespacePolicies.topicDispatchRate.get(cluster);
if (dispatchRate == null) {
dispatchRate = namespacePolicies.clusterDispatchRate.get(cluster);
}
topicPolicies.getDispatchRate().updateNamespaceValue(normalize(dispatchRate));
}

private DispatchRateImpl normalize(DispatchRateImpl dispatchRate) {
if (dispatchRate != null
&& (dispatchRate.getDispatchThrottlingRateInMsg() > 0
|| dispatchRate.getDispatchThrottlingRateInByte() > 0)) {
return dispatchRate;
} else {
return null;
}
}

private void updateSchemaCompatibilityStrategyNamespaceValue(Policies namespacePolicies){
Expand Down Expand Up @@ -279,6 +304,15 @@ private void updateTopicPolicyByBrokerConfig() {
}
topicPolicies.getSchemaCompatibilityStrategy()
.updateBrokerValue(formatSchemaCompatibilityStrategy(schemaCompatibilityStrategy));
topicPolicies.getDispatchRate().updateBrokerValue(dispatchRateInBroker(config));
}

private DispatchRateImpl dispatchRateInBroker(ServiceConfiguration config) {
return DispatchRateImpl.builder()
.dispatchThrottlingRateInMsg(config.getDispatchThrottlingRatePerTopicInMsg())
.dispatchThrottlingRateInByte(config.getDispatchThrottlingRatePerTopicInByte())
.ratePeriodInSecond(1)
.build();
}

private EnumSet<SubType> subTypeStringsToEnumSet(Set<String> getSubscriptionTypesEnabled) {
Expand Down Expand Up @@ -1092,4 +1126,9 @@ public void updateBrokerSubscriptionTypesEnabled() {
topicPolicies.getSubscriptionTypesEnabled().updateBrokerValue(
subTypeStringsToEnumSet(brokerService.pulsar().getConfiguration().getSubscriptionTypesEnabled()));
}

public void updateBrokerDispatchRate() {
topicPolicies.getDispatchRate().updateBrokerValue(
dispatchRateInBroker(brokerService.pulsar().getConfiguration()));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -2221,6 +2221,9 @@ private void updateTopicMessageDispatchRate() {
this.pulsar().getExecutor().execute(() -> {
// update message-rate for each topic
forEachTopic(topic -> {
if (topic instanceof AbstractTopic) {
((AbstractTopic) topic).updateBrokerDispatchRate();
}
if (topic.getDispatchRateLimiter().isPresent()) {
topic.getDispatchRateLimiter().get().updateDispatchRate();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -169,6 +169,12 @@ private DispatchRate createDispatchRate() {
* broker-level
*/
public void updateDispatchRate() {
switch (type) {
case TOPIC:
updateDispatchRate(topic.getDispatchRate());
return;
}

Optional<DispatchRate> dispatchRate = getTopicPolicyDispatchRate(brokerService, topicName, type);
if (!dispatchRate.isPresent()) {
getPoliciesDispatchRateAsync(brokerService).thenAccept(dispatchRateOp -> {
Expand Down Expand Up @@ -457,7 +463,7 @@ public long getDispatchRateOnByte() {
}


private static boolean isDispatchRateEnabled(DispatchRate dispatchRate) {
public static boolean isDispatchRateEnabled(DispatchRate dispatchRate) {
return dispatchRate != null && (dispatchRate.getDispatchThrottlingRateInMsg() > 0
|| dispatchRate.getDispatchThrottlingRateInByte() > 0);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -257,7 +257,6 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS
this.replicators = new ConcurrentOpenHashMap<>(16, 1);
this.backloggedCursorThresholdEntries =
brokerService.pulsar().getConfiguration().getManagedLedgerCursorBackloggedThreshold();
initializeRateLimiterIfNeeded(Optional.empty());
registerTopicPolicyListener();

this.compactedTopic = new CompactedTopicImpl(brokerService.pulsar().getBookKeeperClient());
Expand Down Expand Up @@ -318,6 +317,8 @@ public CompletableFuture<Void> initialize() {

this.updateTopicPolicyByNamespacePolicy(policies);

initializeRateLimiterIfNeeded(Optional.empty());

this.isEncryptionRequired = policies.encryption_required;

isAllowAutoUpdateSchema = policies.is_allow_auto_update_schema;
Expand Down Expand Up @@ -358,8 +359,8 @@ public CompletableFuture<Void> initialize() {
private void initializeRateLimiterIfNeeded(Optional<Policies> policies) {
synchronized (dispatchRateLimiter) {
// dispatch rate limiter for topic
if (!dispatchRateLimiter.isPresent() && DispatchRateLimiter
.isDispatchRateNeeded(brokerService, policies, topic, Type.TOPIC)) {
if (!dispatchRateLimiter.isPresent()
&& DispatchRateLimiter.isDispatchRateEnabled(topicPolicies.getDispatchRate().get())) {
this.dispatchRateLimiter = Optional.of(new DispatchRateLimiter(this, Type.TOPIC));
}
boolean isDispatchRateNeeded = SubscribeRateLimiter.isDispatchRateNeeded(brokerService, policies, topic);
Expand Down Expand Up @@ -2425,11 +2426,7 @@ public CompletableFuture<Void> onPoliciesUpdate(Policies data) {
CompletableFuture<Void> dedupFuture = checkDeduplicationStatus();
CompletableFuture<Void> persistentPoliciesFuture = checkPersistencePolicies();
// update rate-limiter if policies updated
if (this.dispatchRateLimiter.isPresent()) {
if (!topicPolicies.isPresent() || !topicPolicies.get().isDispatchRateSet()) {
dispatchRateLimiter.get().onPoliciesUpdate(data);
}
}
dispatchRateLimiter.ifPresent(DispatchRateLimiter::updateDispatchRate);
if (this.subscribeRateLimiter.isPresent()) {
subscribeRateLimiter.get().onPoliciesUpdate(data);
}
Expand Down Expand Up @@ -3027,13 +3024,7 @@ public void onUpdate(TopicPolicies policies) {
Optional<Policies> namespacePolicies = getNamespacePolicies();
initializeTopicDispatchRateLimiterIfNeeded(policies);

dispatchRateLimiter.ifPresent(limiter -> {
if (policies.isDispatchRateSet()) {
dispatchRateLimiter.get().updateDispatchRate(policies.getDispatchRate());
} else {
dispatchRateLimiter.get().updateDispatchRate();
}
});
dispatchRateLimiter.ifPresent(DispatchRateLimiter::updateDispatchRate);

List<CompletableFuture<Void>> consumerCheckFutures = new ArrayList<>();
subscriptions.forEach((subName, sub) -> sub.getConsumers().forEach(consumer -> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import lombok.Getter;
import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType;
import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType;
import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl;

/**
* Topic policy hierarchy value container.
Expand All @@ -50,6 +51,7 @@ public class HierarchyTopicPolicies {
final PolicyHierarchyValue<Long> delayedDeliveryTickTimeMillis;
final PolicyHierarchyValue<Integer> maxConsumersPerSubscription;
final PolicyHierarchyValue<SchemaCompatibilityStrategy> schemaCompatibilityStrategy;
final PolicyHierarchyValue<DispatchRateImpl> dispatchRate;

public HierarchyTopicPolicies() {
replicationClusters = new PolicyHierarchyValue<>();
Expand All @@ -73,5 +75,6 @@ public HierarchyTopicPolicies() {
delayedDeliveryTickTimeMillis = new PolicyHierarchyValue<>();
compactionThreshold = new PolicyHierarchyValue<>();
schemaCompatibilityStrategy = new PolicyHierarchyValue<>();
dispatchRate = new PolicyHierarchyValue<>();
}
}

0 comments on commit ab16e7e

Please sign in to comment.