You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
Pulsar is able to manage millions of topics but the number of topics within a single namespace is limited by metadata storage.
For each topic within a namespace there is a ZooKeeper node. Listing topics thus requires listing children of a node, which at around 10K topics hits the limits of ZK.
Goal
This feature will allow a larger number of topics within a namespace by inserting an intermediate layer (buckets) before the topic nodes like /managed-ledgers/tenant/namespace/domain/bucket/topic.
By default this feature will be switched off and would only be enabled on a per namespace basis at the creation of namespaces by setting a policy. This eliminates the need for migrating existing installations to this new scheme.
Buckets will not have correlation with bundles.
API Changes
A new policy numberOfTopicBuckets will be added. The default value, 1 means no bucketing, the current behaviour will be preserved for the namespace. Greater values mean topics will be stored at a path including buckets. Users will not be able to change the number of buckets after the namespace is created.
Implementation
The goal is to implement this feature transparently to the user. Clients will continue to refer to topics by domain://tenant/namespace/topic but pulsar will internally translate to the new persistence naming where necessary. This will happen by supplying the bucket number to TopicName.getPersistenceNamingEncoding and calculating the modified path.
The way metadata stores work will not be affected either.
Assigning topics to buckets will be based on the topic name's hash code's absolute value modulo the number of buckets.
The bulk of the changes necessary for this feature is to make namespace policies available wherever persistence naming is calculated. Where listing of topics within a namespace is necessary, the introduction of the new layer will add some overhead in the form of one requests to the metadata store per bucket instead of a single request. These include checking if the limit on topic number per namespace has been reached.
Since the bottleneck is listing topics, which happens using the managed ledgers' path, there is no need to modify schema storage. Furthermore the structure and content of data currently stored at /managed-ledgers/tenant/namespace/domain/topic will not be changed but will be available at the new path.
Existing namespaces and namespaces created without explicitly activating this feature will not be affected.
Namespaces created with this feature activated can be used just as others.
Rejected alternatives
An alternative approach would be to introduce bucketing globally for all namespaces. This would make metadata structure more homogeneous but would require complex update logic to atomically move topics from their current path to the new place once all brokers are upgraded.
For similar reasons changing the number of buckets is not a goal of this proposal.
Since the proposal intends to solve a problem related to ZK, it could be handled within the ZK based metadata store implementation. This would have to introduce knowledge of what paths mean thus breaking separation of concerns.
The text was updated successfully, but these errors were encountered:
Discussion thread : https://lists.apache.org/thread/zx6s7hyrl2vy7nhdl79wh6gn88kxpd6k
Motivation
Pulsar is able to manage millions of topics but the number of topics within a single namespace is limited by metadata storage.
For each topic within a namespace there is a ZooKeeper node. Listing topics thus requires listing children of a node, which at around 10K topics hits the limits of ZK.
Goal
This feature will allow a larger number of topics within a namespace by inserting an intermediate layer (buckets) before the topic nodes like
/managed-ledgers/tenant/namespace/domain/bucket/topic
.By default this feature will be switched off and would only be enabled on a per namespace basis at the creation of namespaces by setting a policy. This eliminates the need for migrating existing installations to this new scheme.
Buckets will not have correlation with bundles.
API Changes
A new policy
numberOfTopicBuckets
will be added. The default value, 1 means no bucketing, the current behaviour will be preserved for the namespace. Greater values mean topics will be stored at a path including buckets. Users will not be able to change the number of buckets after the namespace is created.Implementation
The goal is to implement this feature transparently to the user. Clients will continue to refer to topics by domain://tenant/namespace/topic but pulsar will internally translate to the new persistence naming where necessary. This will happen by supplying the bucket number to
TopicName.getPersistenceNamingEncoding
and calculating the modified path.The way metadata stores work will not be affected either.
Assigning topics to buckets will be based on the topic name's hash code's absolute value modulo the number of buckets.
The bulk of the changes necessary for this feature is to make namespace policies available wherever persistence naming is calculated. Where listing of topics within a namespace is necessary, the introduction of the new layer will add some overhead in the form of one requests to the metadata store per bucket instead of a single request. These include checking if the limit on topic number per namespace has been reached.
Since the bottleneck is listing topics, which happens using the managed ledgers' path, there is no need to modify schema storage. Furthermore the structure and content of data currently stored at
/managed-ledgers/tenant/namespace/domain/topic
will not be changed but will be available at the new path.Example
Let's consider the following metadata hierarchy:
In case of 3 buckets the same topic metadata would be laid out the following way:
Compatibility
Existing namespaces and namespaces created without explicitly activating this feature will not be affected.
Namespaces created with this feature activated can be used just as others.
Rejected alternatives
An alternative approach would be to introduce bucketing globally for all namespaces. This would make metadata structure more homogeneous but would require complex update logic to atomically move topics from their current path to the new place once all brokers are upgraded.
For similar reasons changing the number of buckets is not a goal of this proposal.
Since the proposal intends to solve a problem related to ZK, it could be handled within the ZK based metadata store implementation. This would have to introduce knowledge of what paths mean thus breaking separation of concerns.
The text was updated successfully, but these errors were encountered: