+1 Thanks, Hang
PengHui Li <peng...@apache.org> 于2022年4月25日周一 09:20写道: > > +1 > > Penghui > > On Thu, Apr 21, 2022 at 9:17 PM Andras Beni > <andras.b...@streamnative.io.invalid> wrote: > > > Hi everyone, > > > > I've just created a proposal that will help scaling up the number of topics > > per namespace. > > It's available at https://github.com/apache/pulsar/issues/15254 and is > > copied below. > > Let me know what you think. > > > > Thanks, > > Andras > > > > 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. > > > > 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 multiple requests to the > > metadata store. These include checking if the limit on topic number per > > namespace has been reached. > > Example > > > > Let's consider the following metadata hierarchy: > > > > managed-ledgers > > \- tenant > > \- namespace > > \- persistent > > +- nptopic1 > > +- nptopic2 > > +- ptopic-partition-0 > > +- ptopic-partition-1 > > +- ptopic-partition-2 > > \- ptopic-partition-3 > > > > In case of 3 buckets the same topic metadata would be laid out the > > following way: > > > > managed-ledgers > > \- tenant > > \- namespace > > \- persistent > > +- $0 > > | +- ptopic-partition-0 > > | \- ptopic-partition-3 > > +- $1 > > | +- nptopic2 > > | \- ptopic-partition-1 > > \- $2 > > +- nptopic1 > > \- ptopic-partition-2 > > > > 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. > >