[fix][broker] Fix partitioned topic auto deletion http request timeout
Fixes https://github.com/apache/pulsar/issues/24879
Motivation
Partitioned-topic auto deletion call chain:
- gc process(invoked on broker0)
- delete partitioned-topic admin api(invoked on broker0)
- delete topic-partition-0 admin api(invoked on broker0)
If brokerClient's connectionsPerBroker is 1, then broker0 wait itself to release the connection, the result is timeout. So we check topic-partition-0 existence first to avoid connection pool deadlock.
See also: https://github.com/apache/pulsar/issues/24879#issuecomment-3475290097
Modifications
When calling delete partitioned-topic admin api, first check topic partitions existence to avoid calling admin api by broker itself again, which would help partitioned-topic auto deletion in gc process.
Verifying this change
- [x] Make sure that the change passes the CI checks.
This change is already covered by existing tests, such as (please describe tests).
InactiveTopicDeleteTest#testWhenSubPartitionNotDelete() test method.
https://github.com/apache/pulsar/blob/1ca17972459095278e2b5f7ed7fd55c8921d8826/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/InactiveTopicDeleteTest.java#L113-L136
But the default connectionsPerBroker is 16, which can not reproduce the problem.
https://github.com/apache/pulsar/blob/1ca17972459095278e2b5f7ed7fd55c8921d8826/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java#L48-L51
Does this pull request potentially affect one of the following parts:
If the box was checked, please highlight the changes
- [ ] Dependencies (add or upgrade a dependency)
- [ ] The public API
- [ ] The schema
- [ ] The default values of configurations
- [ ] The threading model
- [ ] The binary protocol
- [x] The REST endpoints
- [ ] The admin CLI options
- [ ] The metrics
- [ ] Anything that affects deployment
Documentation
- [ ]
doc - [ ]
doc-required - [x]
doc-not-needed - [ ]
doc-complete
Matching PR in forked repository
PR in forked repository: https://github.com/oneby-wang/pulsar/pull/8
Could you please add a test for this pr ?
Hi, @Technoboy-, thanks for reminding me. This PR is covered by existing test, see InactiveTopicDeleteTest#testWhenSubPartitionNotDelete() test method.
But I configured the admin connectionsPerBroker param of PulsarService incorrectly before, now it is correct, see commit: https://github.com/apache/pulsar/pull/24912/commits/edaff51ceba81462b5f95b5e593e94ccfc57b96c.
Without this PR's fix, InactiveTopicDeleteTest#testWhenSubPartitionNotDelete() test method will fail due to assert timeout, because admin client connection pool(just 1 connection) is exhausted due to connection pool deadlock, and test failure messages like this:
org.awaitility.core.ConditionTimeoutException: Assertion condition defined as a org.apache.pulsar.broker.service.InactiveTopicDeleteTest expected [false] but found [true] within 10 seconds.
at org.awaitility.core.ConditionAwaiter.await(ConditionAwaiter.java:167)
at org.awaitility.core.AssertionCondition.await(AssertionCondition.java:119)
at org.awaitility.core.AssertionCondition.await(AssertionCondition.java:31)
at org.awaitility.core.ConditionFactory.until(ConditionFactory.java:985)
at org.awaitility.core.ConditionFactory.untilAsserted(ConditionFactory.java:769)
This PR fixes the issue, so it can pass now.
Beyond this PR's issue , one more issue exists in https://github.com/apache/pulsar/issues/24879.
Race condition when deleting partitioned topic, see https://github.com/apache/pulsar/issues/24879#issuecomment-3440341666. I haven't figured out the side effects of the race condition yet, and haven't found a good solution to solve the race condition. It seems like a distributed lock or cas operation is needed, because multi brokers may fire the GC process at the same time, but this solution needs more discussion.