-
Notifications
You must be signed in to change notification settings - Fork 3.6k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[improve] [broker] Improve cache handling for partitioned topic metadata when doing lookup #21063
[improve] [broker] Improve cache handling for partitioned topic metadata when doing lookup #21063
Conversation
It means we disabled the cache. Right? Could you please write a unit test to demonstrate it? btw, it looks like a clean cache way still 100% ensures it, because zk allowed the client to read from a follower. maybe we can consider the sync method of metadata? |
No, only refresh the cache if the write conflicts; the probability is very low.
Since the test is too hard to write, I wrote a simple test
The internal method |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good catch!
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
Outdated
Show resolved
Hide resolved
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
Outdated
Show resolved
Hide resolved
…is auto create partitioned topic
dbd0e23
to
f831ed0
Compare
…ata when doing lookup (#21063) Motivation: If we set `allowAutoTopicCreationType` to `PARTITIONED`, the flow of the create topic progress is like the below: 1. `Client-side`: Lookup topic to get partitioned topic metadata to create a producer. 1. `Broker-side`: Create partitioned topic metadata. 1. `Broker-side`: response `{"partitions":3}`. 1. `Client-side`: Create separate connections for each partition of the topic. 1. `Broker-side`: Receive 3 connect requests and create 3 partition-topics. In the `step 2` above, the flow of the progress is like the below: 1. Check the policy of topic auto-creation( the policy is `{allowAutoTopicCreationType=PARTITIONED, defaultNumPartitions=3}` ) 1. Check the partitioned topic metadata already exists. 1. Try to create the partitioned topic metadata if it does not exist. 1. If created failed by the partitioned topic metadata already exists( maybe another broker is also creating now), read partitioned topic metadata from the metadata store and respond to the client. There is a race condition that makes the client get non-partitioned metadata of the topic: | time | `broker-1` | `broker-2` | | --- | --- | --- | | 1 | get policy: `PARTITIONED, 3` | get policy: `PARTITIONED, 3` | | 2 | check the partitioned topic metadata already exists | Check the partitioned topic metadata already exists | | 3 | Partitioned topic metadata does not exist, the metadata cache will cache an empty optional for the path | Partitioned topic metadata does not exist, the metadata cache will cache an empty optional for the path | | 4 | | succeed create the partitioned topic metadata | | 5 | Receive a ZK node changed event to invalidate the cache of the partitioned topic metadata | | 6 | Creating the metadata failed due to it already exists | | 7 | Read the partitioned topic metadata again | If `step-5` is executed later than `step-7`, `broker-1` will get an empty optional from the cache of the partitioned topic metadata and respond non-partitioned metadata to the client. **What thing would make the `step-5` is executed later than `step-7`?** Provide a scenario: Such as the issue that the PR #20303 fixed, it makes `zk operation` and `zk node changed notifications` executed in different threads: `main-thread of ZK client` and `metadata store thread`. Therefore, the mechanism of the lookup partitioned topic metadata is fragile and we need to optimize it. Modifications: Before reading the partitioned topic metadata again, refresh the cache first. (cherry picked from commit d099ac4)
…ata when doing lookup (#21063) Motivation: If we set `allowAutoTopicCreationType` to `PARTITIONED`, the flow of the create topic progress is like the below: 1. `Client-side`: Lookup topic to get partitioned topic metadata to create a producer. 1. `Broker-side`: Create partitioned topic metadata. 1. `Broker-side`: response `{"partitions":3}`. 1. `Client-side`: Create separate connections for each partition of the topic. 1. `Broker-side`: Receive 3 connect requests and create 3 partition-topics. In the `step 2` above, the flow of the progress is like the below: 1. Check the policy of topic auto-creation( the policy is `{allowAutoTopicCreationType=PARTITIONED, defaultNumPartitions=3}` ) 1. Check the partitioned topic metadata already exists. 1. Try to create the partitioned topic metadata if it does not exist. 1. If created failed by the partitioned topic metadata already exists( maybe another broker is also creating now), read partitioned topic metadata from the metadata store and respond to the client. There is a race condition that makes the client get non-partitioned metadata of the topic: | time | `broker-1` | `broker-2` | | --- | --- | --- | | 1 | get policy: `PARTITIONED, 3` | get policy: `PARTITIONED, 3` | | 2 | check the partitioned topic metadata already exists | Check the partitioned topic metadata already exists | | 3 | Partitioned topic metadata does not exist, the metadata cache will cache an empty optional for the path | Partitioned topic metadata does not exist, the metadata cache will cache an empty optional for the path | | 4 | | succeed create the partitioned topic metadata | | 5 | Receive a ZK node changed event to invalidate the cache of the partitioned topic metadata | | 6 | Creating the metadata failed due to it already exists | | 7 | Read the partitioned topic metadata again | If `step-5` is executed later than `step-7`, `broker-1` will get an empty optional from the cache of the partitioned topic metadata and respond non-partitioned metadata to the client. **What thing would make the `step-5` is executed later than `step-7`?** Provide a scenario: Such as the issue that the PR #20303 fixed, it makes `zk operation` and `zk node changed notifications` executed in different threads: `main-thread of ZK client` and `metadata store thread`. Therefore, the mechanism of the lookup partitioned topic metadata is fragile and we need to optimize it. Modifications: Before reading the partitioned topic metadata again, refresh the cache first. (cherry picked from commit d099ac4)
…ata when doing lookup (#21063) Motivation: If we set `allowAutoTopicCreationType` to `PARTITIONED`, the flow of the create topic progress is like the below: 1. `Client-side`: Lookup topic to get partitioned topic metadata to create a producer. 1. `Broker-side`: Create partitioned topic metadata. 1. `Broker-side`: response `{"partitions":3}`. 1. `Client-side`: Create separate connections for each partition of the topic. 1. `Broker-side`: Receive 3 connect requests and create 3 partition-topics. In the `step 2` above, the flow of the progress is like the below: 1. Check the policy of topic auto-creation( the policy is `{allowAutoTopicCreationType=PARTITIONED, defaultNumPartitions=3}` ) 1. Check the partitioned topic metadata already exists. 1. Try to create the partitioned topic metadata if it does not exist. 1. If created failed by the partitioned topic metadata already exists( maybe another broker is also creating now), read partitioned topic metadata from the metadata store and respond to the client. There is a race condition that makes the client get non-partitioned metadata of the topic: | time | `broker-1` | `broker-2` | | --- | --- | --- | | 1 | get policy: `PARTITIONED, 3` | get policy: `PARTITIONED, 3` | | 2 | check the partitioned topic metadata already exists | Check the partitioned topic metadata already exists | | 3 | Partitioned topic metadata does not exist, the metadata cache will cache an empty optional for the path | Partitioned topic metadata does not exist, the metadata cache will cache an empty optional for the path | | 4 | | succeed create the partitioned topic metadata | | 5 | Receive a ZK node changed event to invalidate the cache of the partitioned topic metadata | | 6 | Creating the metadata failed due to it already exists | | 7 | Read the partitioned topic metadata again | If `step-5` is executed later than `step-7`, `broker-1` will get an empty optional from the cache of the partitioned topic metadata and respond non-partitioned metadata to the client. **What thing would make the `step-5` is executed later than `step-7`?** Provide a scenario: Such as the issue that the PR #20303 fixed, it makes `zk operation` and `zk node changed notifications` executed in different threads: `main-thread of ZK client` and `metadata store thread`. Therefore, the mechanism of the lookup partitioned topic metadata is fragile and we need to optimize it. Modifications: Before reading the partitioned topic metadata again, refresh the cache first. (cherry picked from commit d099ac4)
…ata when doing lookup (apache#21063) Motivation: If we set `allowAutoTopicCreationType` to `PARTITIONED`, the flow of the create topic progress is like the below: 1. `Client-side`: Lookup topic to get partitioned topic metadata to create a producer. 1. `Broker-side`: Create partitioned topic metadata. 1. `Broker-side`: response `{"partitions":3}`. 1. `Client-side`: Create separate connections for each partition of the topic. 1. `Broker-side`: Receive 3 connect requests and create 3 partition-topics. In the `step 2` above, the flow of the progress is like the below: 1. Check the policy of topic auto-creation( the policy is `{allowAutoTopicCreationType=PARTITIONED, defaultNumPartitions=3}` ) 1. Check the partitioned topic metadata already exists. 1. Try to create the partitioned topic metadata if it does not exist. 1. If created failed by the partitioned topic metadata already exists( maybe another broker is also creating now), read partitioned topic metadata from the metadata store and respond to the client. There is a race condition that makes the client get non-partitioned metadata of the topic: | time | `broker-1` | `broker-2` | | --- | --- | --- | | 1 | get policy: `PARTITIONED, 3` | get policy: `PARTITIONED, 3` | | 2 | check the partitioned topic metadata already exists | Check the partitioned topic metadata already exists | | 3 | Partitioned topic metadata does not exist, the metadata cache will cache an empty optional for the path | Partitioned topic metadata does not exist, the metadata cache will cache an empty optional for the path | | 4 | | succeed create the partitioned topic metadata | | 5 | Receive a ZK node changed event to invalidate the cache of the partitioned topic metadata | | 6 | Creating the metadata failed due to it already exists | | 7 | Read the partitioned topic metadata again | If `step-5` is executed later than `step-7`, `broker-1` will get an empty optional from the cache of the partitioned topic metadata and respond non-partitioned metadata to the client. **What thing would make the `step-5` is executed later than `step-7`?** Provide a scenario: Such as the issue that the PR apache#20303 fixed, it makes `zk operation` and `zk node changed notifications` executed in different threads: `main-thread of ZK client` and `metadata store thread`. Therefore, the mechanism of the lookup partitioned topic metadata is fragile and we need to optimize it. Modifications: Before reading the partitioned topic metadata again, refresh the cache first. (cherry picked from commit d099ac4)
…ata when doing lookup (apache#21063) Motivation: If we set `allowAutoTopicCreationType` to `PARTITIONED`, the flow of the create topic progress is like the below: 1. `Client-side`: Lookup topic to get partitioned topic metadata to create a producer. 1. `Broker-side`: Create partitioned topic metadata. 1. `Broker-side`: response `{"partitions":3}`. 1. `Client-side`: Create separate connections for each partition of the topic. 1. `Broker-side`: Receive 3 connect requests and create 3 partition-topics. In the `step 2` above, the flow of the progress is like the below: 1. Check the policy of topic auto-creation( the policy is `{allowAutoTopicCreationType=PARTITIONED, defaultNumPartitions=3}` ) 1. Check the partitioned topic metadata already exists. 1. Try to create the partitioned topic metadata if it does not exist. 1. If created failed by the partitioned topic metadata already exists( maybe another broker is also creating now), read partitioned topic metadata from the metadata store and respond to the client. There is a race condition that makes the client get non-partitioned metadata of the topic: | time | `broker-1` | `broker-2` | | --- | --- | --- | | 1 | get policy: `PARTITIONED, 3` | get policy: `PARTITIONED, 3` | | 2 | check the partitioned topic metadata already exists | Check the partitioned topic metadata already exists | | 3 | Partitioned topic metadata does not exist, the metadata cache will cache an empty optional for the path | Partitioned topic metadata does not exist, the metadata cache will cache an empty optional for the path | | 4 | | succeed create the partitioned topic metadata | | 5 | Receive a ZK node changed event to invalidate the cache of the partitioned topic metadata | | 6 | Creating the metadata failed due to it already exists | | 7 | Read the partitioned topic metadata again | If `step-5` is executed later than `step-7`, `broker-1` will get an empty optional from the cache of the partitioned topic metadata and respond non-partitioned metadata to the client. **What thing would make the `step-5` is executed later than `step-7`?** Provide a scenario: Such as the issue that the PR apache#20303 fixed, it makes `zk operation` and `zk node changed notifications` executed in different threads: `main-thread of ZK client` and `metadata store thread`. Therefore, the mechanism of the lookup partitioned topic metadata is fragile and we need to optimize it. Modifications: Before reading the partitioned topic metadata again, refresh the cache first. (cherry picked from commit d099ac4) (cherry picked from commit 2e534d2)
…ata when doing lookup (apache#21063) Motivation: If we set `allowAutoTopicCreationType` to `PARTITIONED`, the flow of the create topic progress is like the below: 1. `Client-side`: Lookup topic to get partitioned topic metadata to create a producer. 1. `Broker-side`: Create partitioned topic metadata. 1. `Broker-side`: response `{"partitions":3}`. 1. `Client-side`: Create separate connections for each partition of the topic. 1. `Broker-side`: Receive 3 connect requests and create 3 partition-topics. In the `step 2` above, the flow of the progress is like the below: 1. Check the policy of topic auto-creation( the policy is `{allowAutoTopicCreationType=PARTITIONED, defaultNumPartitions=3}` ) 1. Check the partitioned topic metadata already exists. 1. Try to create the partitioned topic metadata if it does not exist. 1. If created failed by the partitioned topic metadata already exists( maybe another broker is also creating now), read partitioned topic metadata from the metadata store and respond to the client. There is a race condition that makes the client get non-partitioned metadata of the topic: | time | `broker-1` | `broker-2` | | --- | --- | --- | | 1 | get policy: `PARTITIONED, 3` | get policy: `PARTITIONED, 3` | | 2 | check the partitioned topic metadata already exists | Check the partitioned topic metadata already exists | | 3 | Partitioned topic metadata does not exist, the metadata cache will cache an empty optional for the path | Partitioned topic metadata does not exist, the metadata cache will cache an empty optional for the path | | 4 | | succeed create the partitioned topic metadata | | 5 | Receive a ZK node changed event to invalidate the cache of the partitioned topic metadata | | 6 | Creating the metadata failed due to it already exists | | 7 | Read the partitioned topic metadata again | If `step-5` is executed later than `step-7`, `broker-1` will get an empty optional from the cache of the partitioned topic metadata and respond non-partitioned metadata to the client. **What thing would make the `step-5` is executed later than `step-7`?** Provide a scenario: Such as the issue that the PR apache#20303 fixed, it makes `zk operation` and `zk node changed notifications` executed in different threads: `main-thread of ZK client` and `metadata store thread`. Therefore, the mechanism of the lookup partitioned topic metadata is fragile and we need to optimize it. Modifications: Before reading the partitioned topic metadata again, refresh the cache first. (cherry picked from commit d099ac4) (cherry picked from commit 2e534d2)
…ata when doing lookup (apache#21063) Motivation: If we set `allowAutoTopicCreationType` to `PARTITIONED`, the flow of the create topic progress is like the below: 1. `Client-side`: Lookup topic to get partitioned topic metadata to create a producer. 1. `Broker-side`: Create partitioned topic metadata. 1. `Broker-side`: response `{"partitions":3}`. 1. `Client-side`: Create separate connections for each partition of the topic. 1. `Broker-side`: Receive 3 connect requests and create 3 partition-topics. In the `step 2` above, the flow of the progress is like the below: 1. Check the policy of topic auto-creation( the policy is `{allowAutoTopicCreationType=PARTITIONED, defaultNumPartitions=3}` ) 1. Check the partitioned topic metadata already exists. 1. Try to create the partitioned topic metadata if it does not exist. 1. If created failed by the partitioned topic metadata already exists( maybe another broker is also creating now), read partitioned topic metadata from the metadata store and respond to the client. There is a race condition that makes the client get non-partitioned metadata of the topic: | time | `broker-1` | `broker-2` | | --- | --- | --- | | 1 | get policy: `PARTITIONED, 3` | get policy: `PARTITIONED, 3` | | 2 | check the partitioned topic metadata already exists | Check the partitioned topic metadata already exists | | 3 | Partitioned topic metadata does not exist, the metadata cache will cache an empty optional for the path | Partitioned topic metadata does not exist, the metadata cache will cache an empty optional for the path | | 4 | | succeed create the partitioned topic metadata | | 5 | Receive a ZK node changed event to invalidate the cache of the partitioned topic metadata | | 6 | Creating the metadata failed due to it already exists | | 7 | Read the partitioned topic metadata again | If `step-5` is executed later than `step-7`, `broker-1` will get an empty optional from the cache of the partitioned topic metadata and respond non-partitioned metadata to the client. **What thing would make the `step-5` is executed later than `step-7`?** Provide a scenario: Such as the issue that the PR apache#20303 fixed, it makes `zk operation` and `zk node changed notifications` executed in different threads: `main-thread of ZK client` and `metadata store thread`. Therefore, the mechanism of the lookup partitioned topic metadata is fragile and we need to optimize it. Modifications: Before reading the partitioned topic metadata again, refresh the cache first. (cherry picked from commit d099ac4) (cherry picked from commit 2e534d2)
Motivation
If we set
allowAutoTopicCreationType
toPARTITIONED
, the flow of the create topic progress is like the below:Client-side
: Lookup topic to get partitioned topic metadata to create a producer.Broker-side
: Create partitioned topic metadata.Broker-side
: response{"partitions":3}
.Client-side
: Create separate connections for each partition of the topic.Broker-side
: Receive 3 connect requests and create 3 partition-topics.In the
step 2
above, the flow of the progress is like the below:{allowAutoTopicCreationType=PARTITIONED, defaultNumPartitions=3}
)There is a race condition that makes the client get non-partitioned metadata of the topic:
broker-1
broker-2
PARTITIONED, 3
PARTITIONED, 3
If
step-5
is executed later thanstep-7
,broker-1
will get an empty optional from the cache of the partitioned topic metadata and respond non-partitioned metadata to the client.What thing would make the
step-5
is executed later thanstep-7
?Provide a scenario: Such as the issue that the PR #20303 fixed, it makes
zk operation
andzk node changed notifications
executed in different threads:main-thread of ZK client
andmetadata store thread
.Therefore, the mechanism of the lookup partitioned topic metadata is fragile and we need to optimize it.
We hit the issue above when using the
release: 2.10.3
, the log is hereWhen we create the topic
public/default/persistent/TP_52
, the broker responds to the client a non-partitioned topic metadata:Total number of partitions for topic persistent://public/default/TP_52 is 0
Modifications
Before reading the partitioned topic metadata again, refresh the cache first.
Documentation
doc
doc-required
doc-not-needed
doc-complete
Matching PR in forked repository
PR in forked repository: x