pulsar
pulsar copied to clipboard
[fix][broker] Fix ResourceGroups loading
Motivation
When org.apache.pulsar.broker.resourcegroup.ResourceGroupConfigListener#loadAllResourceGroups
is unfinished, the org.apache.pulsar.broker.resourcegroup.ResourceGroupNamespaceConfigListener
will be created, which will bind the namespace and resource groups, but loadAllResourceGroups
is async loading, so cause the ResourceGroupNamespaceConfigListener
does not find the ResourceGroup
from the ResourceGroupService cache.
2023-12-21T09:41:18,749+0000 [ForkJoinPool.commonPool-worker-2] ERROR org.apache.pulsar.broker.resourcegroup.ResourceGroupNamespaceConfigListener - Failed to register namespace my-tenant/my-namespace with resource group rg-1
org.apache.pulsar.client.admin.PulsarAdminException: Resource group does not exist: rg-1
at org.apache.pulsar.broker.resourcegroup.ResourceGroupService.checkResourceGroupExists(ResourceGroupService.java:417) ~[org.apache.pulsar-pulsar-broker-3.1.1.jar:3.1.1]
at org.apache.pulsar.broker.resourcegroup.ResourceGroupService.registerNameSpace(ResourceGroupService.java:249) ~[org.apache.pulsar-pulsar-broker-3.1.1.jar:3.1.1]
at org.apache.pulsar.broker.resourcegroup.ResourceGroupNamespaceConfigListener.reconcileNamespaceResourceGroup(ResourceGroupNamespaceConfigListener.java:127) ~[org.apache.pulsar-pulsar-broker-3.1.1.jar:3.1.1]
at org.apache.pulsar.broker.resourcegroup.ResourceGroupNamespaceConfigListener.lambda$updateNamespaceResourceGroup$0(ResourceGroupNamespaceConfigListener.java:69) ~[org.apache.pulsar-pulsar-broker-3.1.1.jar:3.1.1]
at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:863) ~[?:?]
at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:841) ~[?:?]
at java.util.concurrent.CompletableFuture$Completion.exec(CompletableFuture.java:483) ~[?:?]
at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:373) ~[?:?]
at java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(ForkJoinPool.java:1182) ~[?:?]
at java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:1655) ~[?:?]
at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1622) ~[?:?]
at java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:165) ~[?:?]
This process is completed asynchronously, and if it fails, an error will not be thrown to prevent the pulsar from working.
Ultimately, the rate limit failed, due to resourceGroup
is null
, please see org.apache.pulsar.broker.service.AbstractTopic#updateResourceGroupLimiter(org.apache.pulsar.common.policies.data.Policies)
:
public void updateResourceGroupLimiter(@Nonnull Policies namespacePolicies) {
// ....
String rgName = namespacePolicies.resource_group_name;
if (rgName != null) {
final ResourceGroup resourceGroup =
brokerService.getPulsar().getResourceGroupServiceManager().resourceGroupGet(rgName);
if (resourceGroup != null) {
this.resourceGroupRateLimitingEnabled = true;
this.resourceGroupPublishLimiter = resourceGroup.getResourceGroupPublishLimiter();
log.info("Using resource group {} rate limiter for topic {}", rgName, topic);
}
}
// ....
}
Solution: Waiting for the resource groups to cache, and then new ResourceGroupNamespaceConfigListener
.
Modifications
- Waiting for the
ResourceSerivce
to start successfully, if the startup fails, pulsar needs to be restarted by the users.- Compared to the failure of pulsar startup, I prefer the rate limiter to work properly.
- Make
loadAllResourceGroups
return theCompletableFuture<Void>
, and wait for the return value to be done in the constructor. - Make
loadAllNamespaceResourceGroups
return theCompletableFuture<Void>
, and wait for the return value to be done in the constructor.
Verifying this change
Two tests have been added to verify the fail to load the resource.
Documentation
- [ ]
doc
- [ ]
doc-required
- [x]
doc-not-needed
- [ ]
doc-complete
/pulsarbot rerun-failure-checks
/pulsarbot rerun-failure-checks
/pulsarbot rerun-failure-checks
/pulsarbot rerun-failure-checks
Codecov Report
Attention: Patch coverage is 89.18919%
with 4 lines
in your changes are missing coverage. Please review.
Project coverage is 73.65%. Comparing base (
bbc6224
) to head (68bb959
). Report is 75 commits behind head on master.
Additional details and impacted files
@@ Coverage Diff @@
## master #21781 +/- ##
============================================
+ Coverage 73.57% 73.65% +0.08%
+ Complexity 32624 32197 -427
============================================
Files 1877 1887 +10
Lines 139502 139502
Branches 15299 15293 -6
============================================
+ Hits 102638 102754 +116
+ Misses 28908 28783 -125
- Partials 7956 7965 +9
Flag | Coverage Δ | |
---|---|---|
inttests | 26.86% <40.54%> (+2.28%) |
:arrow_up: |
systests | 24.38% <40.54%> (+0.05%) |
:arrow_up: |
unittests | 72.93% <89.18%> (+0.08%) |
:arrow_up: |
Flags with carried forward coverage won't be shown. Click here to find out more.
Files | Coverage Δ | |
---|---|---|
...sar/broker/resourcegroup/ResourceGroupService.java | 73.36% <100.00%> (-0.20%) |
:arrow_down: |
...ker/resourcegroup/ResourceGroupConfigListener.java | 83.14% <88.88%> (+5.68%) |
:arrow_up: |
Add related issue https://github.com/apache/pulsar/issues/15113 and related fix https://github.com/apache/pulsar/pull/15126
https://github.com/apache/pulsar/pull/15126 has been closed, so this PR needs to be reviewed again.
The comment has been resolved and can be reviewed again.
/pulsarbot rerun-failure-checks
/pulsarbot rerun-failure-checks