Skip to content

Conversation

@chenhongSZ
Copy link
Contributor

@chenhongSZ chenhongSZ commented Sep 20, 2023

Fixes #21202

Motivation

fix the local and global topic policies override each other.

Modifications

  1. divide variable org.apache.pulsar.common.policies.data.PolicyHierarchyValue.topicValue into globalTopicValue and variable localTopicValue
  2. replay both of local and global policies in initPolicesCache.
  3. local and global topic policies using the respective key for compaction.

Documentation

  • doc
  • doc-required
  • doc-not-needed
  • doc-complete

Matching PR in forked repository

PR in forked repository:

@github-actions
Copy link

@chenhongSZ Please add the following content to your PR description and select a checkbox:

- [ ] `doc` <!-- Your PR contains doc changes -->
- [ ] `doc-required` <!-- Your PR changes impact docs and you will update later -->
- [ ] `doc-not-needed` <!-- Your PR changes do not impact docs -->
- [ ] `doc-complete` <!-- Docs have been already added -->

@github-actions github-actions bot added doc-not-needed Your PR changes do not impact docs and removed doc-label-missing labels Sep 20, 2023
Copy link
Member

@mattisonchao mattisonchao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you please add a test to verify the changes and explain how you can avoid the compactor to override local policy with global policy(same event key) :)

@chenhongSZ
Copy link
Contributor Author

@mattisonchao This pr does not fix the problem you mentioned(same event key), do I need to submit another pr to fix it or fix it together in this pr?

@Technoboy-
Copy link
Contributor

topicValue in PolicyHierarchyValue could only be local policies.

policiesCache.forEach(((topicName, topicPolicies) -> {
if (listeners.get(topicName) != null) {
for (TopicPolicyListener<TopicPolicies> listener : listeners.get(topicName)) {
try {
listener.onUpdate(topicPolicies);
} catch (Throwable error) {
log.error("[{}] call listener error.", topicName, error);
}
}
}
}));

@chenhongSZ
Copy link
Contributor Author

@Technoboy-
There are two places where the topicValue in PolicyHierarchyValue will be modified

  1. Initialization(you mentioned above), the current implementation is only replay the local policy. I think we should replay both of local and global policies to avoid losing information.
  2. update in org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService#readMorePolicies (mentioned in this issue)
    image

@Technoboy-
Copy link
Contributor

2. update in org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService#readMorePolicies (mentioned in this issue)

ah, good catch. please raise a new pr to fixing it.

@chenhongSZ
Copy link
Contributor Author

there are three issues here. right?

  1. local and global topic policies using the same variable org.apache.pulsar.common.policies.data.PolicyHierarchyValue.topicValue, there's have chance to overwrite each other
  2. during init policesCache, only replay the local policy message, global policies will not take effect.
  3. global and local topic policies using the same key. it will be compacted by the compactor.

I fixed the first two in this pr, the third issue I'll fix with another pr.

@Technoboy- @aloyszhang PTAL

if (hasMore) {
reader.readNextAsync().thenAccept(msg -> {
refreshTopicPoliciesCache(msg);
notifyListener(msg);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's best to wait for initPolicesCache to complete before notifying the listeners.

Copy link
Contributor Author

@chenhongSZ chenhongSZ Sep 22, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

make sense to me, I have fixed it. PTAL

@chenhongSZ chenhongSZ requested a review from coderzc September 23, 2023 05:28
@chenhongSZ
Copy link
Contributor Author

Could you please add a test to verify the changes and explain how you can avoid the compactor to override local policy with global policy(same event key) :)

I have fixed this issue and added a test case in my latest commit, PTAL, thanks
CC @Technoboy- @aloyszhang @coderzc

# Conflicts:
#	pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java
@chenhongSZ
Copy link
Contributor Author

/pulsarbot run-failure-checks

return CompletableFuture.completedFuture(null).thenRunAsync(() -> {
TopicPoliciesService topicPoliciesService = brokerService.getPulsar().getTopicPoliciesService();
onUpdate(topicPoliciesService.getLocalTopicPoliciesIfExists(TopicName.getPartitionedTopicName(topic)));
onUpdate(topicPoliciesService.getGlobalTopicPoliciesIfExists(TopicName.getPartitionedTopicName(topic)));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This change fixed the bug that the global policies were discarded when loading a topic up. We also need to add a test to cover this fix. The test flow should be like this:

  • create a topic and unload it
  • set global policies
  • start a consumer/producer to load the topic up
  • ensure the global policies affect the topic

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

thanks for the reminder, I've added a test for this change in the latest commit, PTAL

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wrote a test for this case, could you add it into current PR?

@Test
public void testGetTopicPoliciesWhenDeleteTopicPolicy2() throws Exception {
    final String tpName = BrokerTestUtil.newUniqueName("persistent://" + myNamespace + "/tp");
    final String subscriptionName = "s1";
    final int dispatchThrottlingRateInMsg = 1000;
    admin.topics().createNonPartitionedTopic(tpName);
    PersistentTopic persistentTopic1 =
            (PersistentTopic) pulsar.getBrokerService().getTopic(tpName, false).join().get();
    admin.topics().createSubscription(tpName, subscriptionName, MessageId.earliest);
    Producer producer = pulsarClient.newProducer().topic(tpName).create();

    // Set global policy.
    DispatchRate dispatchRate = new DispatchRateImpl(dispatchThrottlingRateInMsg, 1, false, 1);
    admin.topicPolicies(true).setDispatchRate(tpName, dispatchRate);

    // Assert policy was affected.
    Awaitility.await().ignoreExceptions().untilAsserted(() -> {
        HierarchyTopicPolicies policies = persistentTopic1.getHierarchyTopicPolicies();
        assertNotNull(policies);
        assertEquals(policies.getDispatchRate().getTopicValue().getDispatchThrottlingRateInMsg(),
                dispatchThrottlingRateInMsg);
        DispatchRate dispatchRateAp = admin.topicPolicies(true).getDispatchRate(tpName, true);
        assertEquals(dispatchRateAp.getDispatchThrottlingRateInMsg(), dispatchThrottlingRateInMsg);
    });

    // Unload topic and check again.
    admin.topics().unload(tpName);
    // Wait topic load complete.
    producer.send("1".getBytes(StandardCharsets.UTF_8));
    PersistentTopic persistentTopic2 =
            (PersistentTopic) pulsar.getBrokerService().getTopic(tpName, false).join().get();
    // Assert policy was affected.
    Awaitility.await().ignoreExceptions().untilAsserted(() -> {
        HierarchyTopicPolicies policies = persistentTopic2.getHierarchyTopicPolicies();
        assertNotNull(policies);
        assertEquals(policies.getDispatchRate().getTopicValue().getDispatchThrottlingRateInMsg(),
                dispatchThrottlingRateInMsg);
        DispatchRate dispatchRateAp = admin.topicPolicies(true).getDispatchRate(tpName, true);
        assertEquals(dispatchRateAp.getDispatchThrottlingRateInMsg(), dispatchThrottlingRateInMsg);
    });

    // cleanup.
    producer.close();
    admin.topics().delete(tpName, false);
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

}

@Test
public void testInitPolicesCacheAndNotifyListenersAfterCompaction() throws Exception {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A new channel to trace the comment #21212 (review)

@mattisonchao
Could you please add a test to verify the changes and explain how you can avoid the compactor to override local policy with global policy(same event key) :)

Agree with @mattisonchao , the test flow might be like this:

  • set local policies
  • set global policies
  • wait for the compaction task to finish
  • read the change_event topic to verify there are two events for this topic. Or find a way to start a new SystemTopicBasedTopicPoliciesService and verify the two caches (policiesCache and globalPoliciesCache) each has one item for this topic

Copy link
Contributor Author

@chenhongSZ chenhongSZ Sep 27, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This test case(testInitPolicesCacheAndNotifyListenersAfterCompaction) is exactly what you described, please correct me if I have misunderstood.

Copy link
Contributor

@poorbarcode poorbarcode Oct 8, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I do not think the test you provided can cover the case. Could you try to run the test below?

private void triggerAndWaitNewTopicCompaction(String topicName) throws Exception {
    PersistentTopic tp =
            (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).join().get();
    // Wait for the old task finish.
    Awaitility.await().untilAsserted(() -> {
        CompletableFuture<Long> compactionTask = WhiteboxImpl.getInternalState(tp, "currentCompaction");
        assertTrue(compactionTask == null || compactionTask.isDone());
    });
    // Trigger a new task.
    tp.triggerCompaction();
    // Wait for the new task finish.
    Awaitility.await().untilAsserted(() -> {
        CompletableFuture<Long> compactionTask = WhiteboxImpl.getInternalState(tp, "currentCompaction");
        assertTrue(compactionTask == null || compactionTask.isDone());
    });
}

/***
 * It is not a thread safety method, something will go to a wrong pointer if there is a task is trying to load a
 * topic policies.
 */
private void clearTopicPoliciesCache() {
    TopicPoliciesService topicPoliciesService = pulsar.getTopicPoliciesService();
    if (topicPoliciesService instanceof TopicPoliciesService.TopicPoliciesServiceDisabled) {
        return;
    }
    assertTrue(topicPoliciesService instanceof SystemTopicBasedTopicPoliciesService);

    Map<NamespaceName, CompletableFuture<Void>> policyCacheInitMap =
            WhiteboxImpl.getInternalState(topicPoliciesService, "policyCacheInitMap");
    for (CompletableFuture<Void> future : policyCacheInitMap.values()) {
        future.join();
    }
    Map<TopicName, TopicPolicies> policiesCache =
            WhiteboxImpl.getInternalState(topicPoliciesService, "policiesCache");
    Map<TopicName, TopicPolicies> globalPoliciesCache =
            WhiteboxImpl.getInternalState(topicPoliciesService, "globalPoliciesCache");

    policyCacheInitMap.clear();
    policiesCache.clear();
    globalPoliciesCache.clear();
}

@Test
public void testLocalPolicyAffectAfterCompaction() throws Exception {
    final String tpName = BrokerTestUtil.newUniqueName("persistent://" + myNamespace + "/tp");
    final String tpNameChangeEvents = "persistent://" + myNamespace + "/" + NAMESPACE_EVENTS_LOCAL_NAME;
    final String subscriptionName = "s1";
    final int rateMsgLocal = 2000;
    final int rateMsgGlobal = 1000;
    admin.topics().createNonPartitionedTopic(tpName);
    admin.topics().createSubscription(tpName, subscriptionName, MessageId.earliest);

    // Set global policy and local policy.
    DispatchRate dispatchRateLocal = new DispatchRateImpl(rateMsgLocal, 1, false, 1);
    DispatchRate dispatchRateGlobal = new DispatchRateImpl(rateMsgGlobal, 1, false, 1);
    admin.topicPolicies(false).setDispatchRate(tpName, dispatchRateLocal);
    admin.topicPolicies(true).setDispatchRate(tpName, dispatchRateGlobal);

    // Trigger __change_events compaction and clear topic policies cache.
    triggerAndWaitNewTopicCompaction(tpNameChangeEvents);
    clearTopicPoliciesCache();

    // Reload the topic policies.
    // Verify the local policies was affected.
    Optional<TopicPolicies> topicPoliciesOptional =
            pulsar.getTopicPoliciesService().getTopicPoliciesAsync(TopicName.get(tpName)).join();
    assertTrue(topicPoliciesOptional.isPresent());
    assertEquals(topicPoliciesOptional.get().getDispatchRate().getDispatchThrottlingRateInMsg(),
            rateMsgLocal);

    // cleanup.
    admin.topics().delete(tpName, false);
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the test case(testInitPolicesCacheAndNotifyListenersAfterCompaction) can cover the fix about compaction, the test case will fail without this fix.
I also added the test cases you provided above, and it's passed.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@chenhongSZ

Sorry, there is a mistake in the test. Could you change the line like below? Thanks

private void triggerAndWaitNewTopicCompaction(String topicName) throws Exception {
    PersistentTopic tp =
            (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).join().get();
    // Wait for the old task finish.
    Awaitility.await().untilAsserted(() -> {
        CompletableFuture<Long> compactionTask = WhiteboxImpl.getInternalState(tp, "currentCompaction");
        assertTrue(compactionTask == null || compactionTask.isDone());
    });
    // Trigger a new task.
    tp.triggerCompaction();
    // Wait for the new task finish.
    Awaitility.await().untilAsserted(() -> {
        CompletableFuture<Long> compactionTask = WhiteboxImpl.getInternalState(tp, "currentCompaction");
        assertTrue(compactionTask == null || compactionTask.isDone());
    });
}

@Test
public void testLocalPolicyAffectAfterCompaction() throws Exception {
    final String tpName = BrokerTestUtil.newUniqueName("persistent://" + myNamespace + "/tp");
    final String tpNameChangeEvents = "persistent://" + myNamespace + "/" + NAMESPACE_EVENTS_LOCAL_NAME;
    final String subscriptionName = "s1";
    final int rateMsgLocal = 2000;
    final int rateMsgGlobal = 1000;
    admin.topics().createNonPartitionedTopic(tpName);
    admin.topics().createSubscription(tpName, subscriptionName, MessageId.earliest);

    // Set global policy and local policy.
    DispatchRate dispatchRateLocal = new DispatchRateImpl(rateMsgLocal, 1, false, 1);
    DispatchRate dispatchRateGlobal = new DispatchRateImpl(rateMsgGlobal, 1, false, 1);
    admin.topicPolicies(false).setDispatchRate(tpName, dispatchRateLocal);
    admin.topicPolicies(true).setDispatchRate(tpName, dispatchRateGlobal);

    // Trigger __change_events compaction.
    triggerAndWaitNewTopicCompaction(tpNameChangeEvents);

    // Create a new SystemTopicBasedTopicPoliciesService and verify the local policies was affected.
    Optional<TopicPolicies> topicPoliciesOptional =
            new SystemTopicBasedTopicPoliciesService(pulsar).getTopicPoliciesAsync(TopicName.get(tpName)).join();
    assertTrue(topicPoliciesOptional.isPresent());
    assertEquals(topicPoliciesOptional.get().getDispatchRate().getDispatchThrottlingRateInMsg(),
            rateMsgLocal);

    // cleanup.
    admin.topics().delete(tpName, false);
}

Copy link
Contributor

@poorbarcode poorbarcode Oct 8, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@mattisonchao

The global topic policy and the local topic policy use the same message key, and they will overwrite each other by topic compression.

The changes below solved this issue.

https://github.com/apache/pulsar/pull/21212/files#diff-9d2948d863c111e4be6d508a1c573667a1326b98c4314e917ba9e344bb61dc27R665-L635

截屏2023-10-09 00 11 31

@chenhongSZ

But this fix also changed the rule of key building, which makes it not forward-compatible. For example:

  • The user updates local policies, and Pulsar sends a message with the key "persistent://public/default/topic_1"
  • The user upgrades the Pulsar cluster to the newest version.
  • The user updates local policies, and Pulsar sends a message with the key "persistent/public/default/topic_1/false"
  • The user updates global policies, and Pulsar sends a message with the key "persistent/public/default/topic_1/true"

At this time, there should be three messages in the system topic, but the behavior will be correct even if it is not expected, but the behavior will be wrong after the steps below.

  • Delete the topic
    • Pulsar deletes the key "persistent/public/default/topic_1/false"
    • Pulsar deletes the key "persistent/public/default/topic_1/true"
  • Create a new topic with the same name as the old one.

Currently, the message whose key is "persistent://public/default/topic_1" is still there.

I think once the new key is generated, the data of the old key should be deleted. And could you also add a test for this case?

By the way, the original rule is persistent://xxx, do not change it to persistent/xxx.

Copy link
Member

@mattisonchao mattisonchao Oct 8, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please negotiate this special key name. Do not make it casually. :)

Plus, please care about the compatibility of this change.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I will fix the compatibility problem and add a test later.
I didn't change the rules of the key building, just added a property, because we need to get the topic name from the key
original: persistent://tenant/namespace/topic/
new:persistent://tenant/namespace/topic/isGlobal

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@chenhongSZ

Sorry, there is a mistake in the test. Could you change the line like below? Thanks

private void triggerAndWaitNewTopicCompaction(String topicName) throws Exception {
    PersistentTopic tp =
            (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).join().get();
    // Wait for the old task finish.
    Awaitility.await().untilAsserted(() -> {
        CompletableFuture<Long> compactionTask = WhiteboxImpl.getInternalState(tp, "currentCompaction");
        assertTrue(compactionTask == null || compactionTask.isDone());
    });
    // Trigger a new task.
    tp.triggerCompaction();
    // Wait for the new task finish.
    Awaitility.await().untilAsserted(() -> {
        CompletableFuture<Long> compactionTask = WhiteboxImpl.getInternalState(tp, "currentCompaction");
        assertTrue(compactionTask == null || compactionTask.isDone());
    });
}

@Test
public void testLocalPolicyAffectAfterCompaction() throws Exception {
    final String tpName = BrokerTestUtil.newUniqueName("persistent://" + myNamespace + "/tp");
    final String tpNameChangeEvents = "persistent://" + myNamespace + "/" + NAMESPACE_EVENTS_LOCAL_NAME;
    final String subscriptionName = "s1";
    final int rateMsgLocal = 2000;
    final int rateMsgGlobal = 1000;
    admin.topics().createNonPartitionedTopic(tpName);
    admin.topics().createSubscription(tpName, subscriptionName, MessageId.earliest);

    // Set global policy and local policy.
    DispatchRate dispatchRateLocal = new DispatchRateImpl(rateMsgLocal, 1, false, 1);
    DispatchRate dispatchRateGlobal = new DispatchRateImpl(rateMsgGlobal, 1, false, 1);
    admin.topicPolicies(false).setDispatchRate(tpName, dispatchRateLocal);
    admin.topicPolicies(true).setDispatchRate(tpName, dispatchRateGlobal);

    // Trigger __change_events compaction.
    triggerAndWaitNewTopicCompaction(tpNameChangeEvents);

    // Create a new SystemTopicBasedTopicPoliciesService and verify the local policies was affected.
    Optional<TopicPolicies> topicPoliciesOptional =
            new SystemTopicBasedTopicPoliciesService(pulsar).getTopicPoliciesAsync(TopicName.get(tpName)).join();
    assertTrue(topicPoliciesOptional.isPresent());
    assertEquals(topicPoliciesOptional.get().getDispatchRate().getDispatchThrottlingRateInMsg(),
            rateMsgLocal);

    // cleanup.
    admin.topics().delete(tpName, false);
}

fixed

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I've fixed the compatibility problem and added a test to cover this change. PTAL

@coderzc
Copy link
Member

coderzc commented Sep 28, 2023

Also, we need to clean the global topic policy in cleanCacheAndCloseReader.

private void cleanCacheAndCloseReader(@Nonnull NamespaceName namespace, boolean cleanOwnedBundlesCount) {
CompletableFuture<SystemTopicClient.Reader<PulsarEvent>> readerFuture = readerCaches.remove(namespace);
if (cleanOwnedBundlesCount) {
ownedBundlesCountPerNamespace.remove(namespace);
}
if (readerFuture != null && !readerFuture.isCompletedExceptionally()) {
readerFuture.thenCompose(SystemTopicClient.Reader::closeAsync)
.exceptionally(ex -> {
log.warn("[{}] Close change_event reader fail.", namespace, ex);
return null;
});
}
policyCacheInitMap.compute(namespace, (k, v) -> {
policiesCache.entrySet().removeIf(entry -> Objects.equals(entry.getKey().getNamespaceObject(), namespace));
return null;
});
}

@chenhongSZ
Copy link
Contributor Author

Also, we need to clean the global topic policy in cleanCacheAndCloseReader.

private void cleanCacheAndCloseReader(@Nonnull NamespaceName namespace, boolean cleanOwnedBundlesCount) {
CompletableFuture<SystemTopicClient.Reader<PulsarEvent>> readerFuture = readerCaches.remove(namespace);
if (cleanOwnedBundlesCount) {
ownedBundlesCountPerNamespace.remove(namespace);
}
if (readerFuture != null && !readerFuture.isCompletedExceptionally()) {
readerFuture.thenCompose(SystemTopicClient.Reader::closeAsync)
.exceptionally(ex -> {
log.warn("[{}] Close change_event reader fail.", namespace, ex);
return null;
});
}
policyCacheInitMap.compute(namespace, (k, v) -> {
policiesCache.entrySet().removeIf(entry -> Objects.equals(entry.getKey().getNamespaceObject(), namespace));
return null;
});
}

fixed and added a test to cover this change, PTAL

}

@Test
public void testGetTopicPoliciesWhenDeleteTopicPolicy2() throws Exception {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry, I forgot to rename this method. Please help to rename it to "testGlobalPolicyAfterUnloadTopic"

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

}

@Test
public void testInitPolicesCacheAndNotifyListenersAfterCompaction() throws Exception {
Copy link
Contributor

@poorbarcode poorbarcode Oct 8, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I do not think the test you provided can cover the case. Could you try to run the test below?

private void triggerAndWaitNewTopicCompaction(String topicName) throws Exception {
    PersistentTopic tp =
            (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).join().get();
    // Wait for the old task finish.
    Awaitility.await().untilAsserted(() -> {
        CompletableFuture<Long> compactionTask = WhiteboxImpl.getInternalState(tp, "currentCompaction");
        assertTrue(compactionTask == null || compactionTask.isDone());
    });
    // Trigger a new task.
    tp.triggerCompaction();
    // Wait for the new task finish.
    Awaitility.await().untilAsserted(() -> {
        CompletableFuture<Long> compactionTask = WhiteboxImpl.getInternalState(tp, "currentCompaction");
        assertTrue(compactionTask == null || compactionTask.isDone());
    });
}

/***
 * It is not a thread safety method, something will go to a wrong pointer if there is a task is trying to load a
 * topic policies.
 */
private void clearTopicPoliciesCache() {
    TopicPoliciesService topicPoliciesService = pulsar.getTopicPoliciesService();
    if (topicPoliciesService instanceof TopicPoliciesService.TopicPoliciesServiceDisabled) {
        return;
    }
    assertTrue(topicPoliciesService instanceof SystemTopicBasedTopicPoliciesService);

    Map<NamespaceName, CompletableFuture<Void>> policyCacheInitMap =
            WhiteboxImpl.getInternalState(topicPoliciesService, "policyCacheInitMap");
    for (CompletableFuture<Void> future : policyCacheInitMap.values()) {
        future.join();
    }
    Map<TopicName, TopicPolicies> policiesCache =
            WhiteboxImpl.getInternalState(topicPoliciesService, "policiesCache");
    Map<TopicName, TopicPolicies> globalPoliciesCache =
            WhiteboxImpl.getInternalState(topicPoliciesService, "globalPoliciesCache");

    policyCacheInitMap.clear();
    policiesCache.clear();
    globalPoliciesCache.clear();
}

@Test
public void testLocalPolicyAffectAfterCompaction() throws Exception {
    final String tpName = BrokerTestUtil.newUniqueName("persistent://" + myNamespace + "/tp");
    final String tpNameChangeEvents = "persistent://" + myNamespace + "/" + NAMESPACE_EVENTS_LOCAL_NAME;
    final String subscriptionName = "s1";
    final int rateMsgLocal = 2000;
    final int rateMsgGlobal = 1000;
    admin.topics().createNonPartitionedTopic(tpName);
    admin.topics().createSubscription(tpName, subscriptionName, MessageId.earliest);

    // Set global policy and local policy.
    DispatchRate dispatchRateLocal = new DispatchRateImpl(rateMsgLocal, 1, false, 1);
    DispatchRate dispatchRateGlobal = new DispatchRateImpl(rateMsgGlobal, 1, false, 1);
    admin.topicPolicies(false).setDispatchRate(tpName, dispatchRateLocal);
    admin.topicPolicies(true).setDispatchRate(tpName, dispatchRateGlobal);

    // Trigger __change_events compaction and clear topic policies cache.
    triggerAndWaitNewTopicCompaction(tpNameChangeEvents);
    clearTopicPoliciesCache();

    // Reload the topic policies.
    // Verify the local policies was affected.
    Optional<TopicPolicies> topicPoliciesOptional =
            pulsar.getTopicPoliciesService().getTopicPoliciesAsync(TopicName.get(tpName)).join();
    assertTrue(topicPoliciesOptional.isPresent());
    assertEquals(topicPoliciesOptional.get().getDispatchRate().getDispatchThrottlingRateInMsg(),
            rateMsgLocal);

    // cleanup.
    admin.topics().delete(tpName, false);
}

Copy link
Member

@mattisonchao mattisonchao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm sorry, but I don't think this change suits me.

There are some reasons are as follows:

  1. Apache Pulsar has the broker/namespace/topic level policy. The global topic policy is just a particular topic policy based on the geo-replication. We should refrain from introducing this concept into the topic entity.
  2. We should encapsulate this concept into topic policy service, which can decide to give the topic a global or local policy because the topic doesn't care if the policy is global or not.

Therefore, I would like to let [SystemTopicBasedTopicPoliciesService.java](https://github.com/apache/pulsar/pull/21212/files#diff-9d2948d863c111e4be6d508a1c573667a1326b98c4314e917ba9e344bb61dc27) decide to update global or local policy to the topic to avoid isGlobal param everywhere.

WDYT?
/cc @codelipenghui @Technoboy- @poorbarcode

@mattisonchao
Copy link
Member

Plus, we didn't support the global policy very well. There are some issues I know so far.

  1. The global topic policy and the local topic policy use the same message key, and they will overwrite each other by topic compression.
  2. We did not use the global policy correctly. In some places, we only get local policies.

@chenhongSZ
Copy link
Contributor Author

I'm sorry, but I don't think this change suits me.

There are some reasons are as follows:

  1. Apache Pulsar has the broker/namespace/topic level policy. The global topic policy is just a particular topic policy based on the geo-replication. We should refrain from introducing this concept into the topic entity.
  2. We should encapsulate this concept into topic policy service, which can decide to give the topic a global or local policy because the topic doesn't care if the policy is global or not.

Therefore, I would like to let [SystemTopicBasedTopicPoliciesService.java](https://github.com/apache/pulsar/pull/21212/files#diff-9d2948d863c111e4be6d508a1c573667a1326b98c4314e917ba9e344bb61dc27) decide to update global or local policy to the topic to avoid isGlobal param everywhere.

WDYT? /cc @codelipenghui @Technoboy- @poorbarcode

I agree, It is indeed we should encapsulate this concept into topic policy service. I will make some changes as below:

  1. rollback PolicyHierarchyValue to its previous state (use topicValue replace localTopicValue and globalTopicValue)
  2. merge the local and global topic policies into one TopicPolicies at SystemTopicBasedTopicPoliciesService before init topic and notify topic. therefore org.apache.pulsar.broker.service.persistent.PersistentTopic#onUpdate method doesn't care if the policy is global or not. but the org.apache.pulsar.common.policies.data.TopicPolicies#isGlobal becomes meaningless in the topic context. so we should not use this variable in the topic context.
    Any suggestions?

@chenhongSZ
Copy link
Contributor Author

I'm sorry, but I don't think this change suits me.
There are some reasons are as follows:

  1. Apache Pulsar has the broker/namespace/topic level policy. The global topic policy is just a particular topic policy based on the geo-replication. We should refrain from introducing this concept into the topic entity.
  2. We should encapsulate this concept into topic policy service, which can decide to give the topic a global or local policy because the topic doesn't care if the policy is global or not.

Therefore, I would like to let [SystemTopicBasedTopicPoliciesService.java](https://github.com/apache/pulsar/pull/21212/files#diff-9d2948d863c111e4be6d508a1c573667a1326b98c4314e917ba9e344bb61dc27) decide to update global or local policy to the topic to avoid isGlobal param everywhere.
WDYT? /cc @codelipenghui @Technoboy- @poorbarcode

I agree, It is indeed we should encapsulate this concept into topic policy service. I will make some changes as below:

  1. rollback PolicyHierarchyValue to its previous state (use topicValue replace localTopicValue and globalTopicValue)
  2. merge the local and global topic policies into one TopicPolicies at SystemTopicBasedTopicPoliciesService before init topic and notify topic. therefore org.apache.pulsar.broker.service.persistent.PersistentTopic#onUpdate method doesn't care if the policy is global or not. but the org.apache.pulsar.common.policies.data.TopicPolicies#isGlobal becomes meaningless in the topic context. so we should not use this variable in the topic context.
    Any suggestions?

those changes have been done in my latest commit, PTAL, thanks @mattisonchao

@mattisonchao
Copy link
Member

@chenhongSZ Thanks for your great work! I will review it today.

@mattisonchao
Copy link
Member

Sorry for some other tasks. I have to postpone this review.

}

TopicPolicies topicPolicies = new TopicPolicies();
BeanUtil.copyProperties(globalTopicPolicies, topicPolicies, true);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh, I think the original implementation is better

@Technoboy- Technoboy- modified the milestones: 3.2.0, 3.3.0 Dec 22, 2023
@chenhongSZ chenhongSZ closed this Mar 26, 2024
@chenhongSZ chenhongSZ deleted the fix-topicPolicyPriority branch March 26, 2024 06:24
@chenhongSZ chenhongSZ restored the fix-topicPolicyPriority branch March 26, 2024 06:24
@chenhongSZ chenhongSZ deleted the fix-topicPolicyPriority branch July 25, 2024 11:58
@chenhongSZ chenhongSZ restored the fix-topicPolicyPriority branch July 25, 2024 11:59
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

doc-not-needed Your PR changes do not impact docs ready-to-test

Projects

None yet

Development

Successfully merging this pull request may close these issues.

[Bug] The global topic policy config override the local topic policy config

6 participants