Changing the topic creation flow and optimize heartbeat topic not trigger compaction.#14643
Conversation
| .thenCompose(__ -> persistentTopic.checkReplication()) | ||
| .thenCompose(v -> { | ||
| // Also check dedup status | ||
| return persistentTopic.checkDeduplicationStatus(); |
There was a problem hiding this comment.
For convenient review, a little explanation here:
Move preCreateSubscriptionForCompactionIfNeeded after initialize. because in preCreateSubscriptionForCompactionIfNeeded it will retrieve the compaction threshold, it could only get the value at the broker level. Only after initialize, namespace and topic level values will load into the topic. So it's the reason to do this. Besides, I think preCreateSubscriptionForCompactionIfNeeded should stay after initialize is more accurate .
|
For convenient review, a little explanation here: |
eolivelli
left a comment
There was a problem hiding this comment.
I am fine with this PR but the scope is larger than only the heartbeat topic: we are changing the topic creation flow.
Please update the title accordingly or split the patch into two parts
Thanks @eolivelli . I have changed the title. |
|
Could you help review this ?@Jason918 @hangc0276 |
…gger compaction. (#14643) ### Motivation When create persistent topic, create compaction subscription(line-1395) is before topic `initialize`(line-1397), it's better to do this after `initialize`: https://github.com/apache/pulsar/blob/ad2cc2d38280b7dd0f056ee981ec8d3b157e3526/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java#L1391-L1398 If we change this part, we can optimize the heartbeat topic not trigger the compaction. #13611 has made the heartbeat topic as a system topic to avoid deleting by GC. However, system topic is compacted by default. But the heartbeat topic sends msg without a key, so trigger compaction is meaningful. So it's better to skip the heartbeat topic to do the compaction. ### Modification - Move `preCreateSubscriptionForCompactionIfNeeded` after `initialize`. - Add heartbeat topic not trigger compaction. (cherry picked from commit d02bd73)
…gger compaction. (apache#14643) ### Motivation When create persistent topic, create compaction subscription(line-1395) is before topic `initialize`(line-1397), it's better to do this after `initialize`: https://github.com/apache/pulsar/blob/ad2cc2d38280b7dd0f056ee981ec8d3b157e3526/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java#L1391-L1398 If we change this part, we can optimize the heartbeat topic not trigger the compaction. apache#13611 has made the heartbeat topic as a system topic to avoid deleting by GC. However, system topic is compacted by default. But the heartbeat topic sends msg without a key, so trigger compaction is meaningful. So it's better to skip the heartbeat topic to do the compaction. ### Modification - Move `preCreateSubscriptionForCompactionIfNeeded` after `initialize`. - Add heartbeat topic not trigger compaction.
- backport relevant change from apache#14643 to branch-2.7
@Technoboy- Do we need to cherry-pick/backport this PR to the branches which include #13611 / #13816 ? |
|
@Technoboy- Do we need to cherry-pick/backport this PR to the branches which include #13611 / #13816 ? |
Motivation
When create persistent topic, create compaction subscription(line-1395) is before topic
initialize(line-1397), it's better to do this afterinitialize:pulsar/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
Lines 1391 to 1398 in ad2cc2d
If we change this part, we can optimize the heartbeat topic not trigger the compaction.
#13611 has made the heartbeat topic as a system topic to avoid deleting by GC. However, system topic is compacted by default. But the heartbeat topic sends msg without a key, so trigger compaction is meaningful. So it's better to skip the heartbeat topic to do the compaction.
Modification
preCreateSubscriptionForCompactionIfNeededafterinitialize.Documentation
no-need-doc