Hi Mattison,

1. Should we reject the partitioned topic metadata creation when the topic
> name contains the `-partition-` keywords?


I'm not familiar with all the details but my naive opinion would be that we
should avoid placing constraints on users if we can. In this case that
would mean removing any technical issues that arise due to `-partition-`
being in the topic name, rather than rejecting topics with that string in
the name.

Regards,
Dave



On Thu, Jun 16, 2022 at 4:20 AM mattison chao <mattisonc...@apache.org>
wrote:

> Hello, everyone.
>
> I want to start a discussion about *Reject partitioned topic creation when
> the topic name contains the `-partition-` keyword.*
> Please feel free to express your opinion.
>
> *Background*
>
> We first found this problem two months ago. When the user sets the
> *allowAutoTopicCreation* policy as below:
>
> > {
> > allowAutoTopicCreation: true,
> > allowAutoTopicCreationType: "partitioned",
> > defaultNumPartitions: 1
> >  }
>
>
> This policy will make brokers automatically create the partitioned topic
> when the topic does not exist. But we don't check if users use topic
> names like "persistent://public/default/XXXX-partition-YYY".
> In our logic, this topic will pass the automatically created
> partition topic metadata check. We will make the partitioned metadata like
> this:
>
> Z-NODE:
>
>  paritioned-topics/public/defualt/persistent/XXXX-partition-YYY
>
> {"partitions": 1}
>
>
> The things here stand at the broker's side. Everything looks like we
> expect. Even the topic name includes "-partition-" keywords.
>
> And then, we continue the process. The Pulsar Client(Java) will get this
> metadata and then try to create separate topics with `-partition-{index}`.
> But there have small trick things,
> The client try to invoke
> `TopicName.get(topicName).getPartition(partitionIndex).toString();` method
> to get paritioned topic name. however, we have `partition` keywords check
> in the method `TopicName.get(topicName).getPartition(partitionIndex)`[1].
> If the topic name includes `-partition-` keywords, it will return the
> original topic name directly. So, The client will use this topic name to
> create the consumer.
>
> Here we go. Let's summarize:
>
> 1. We make the topic `persistent://public/default/XXXX-partition-YYY` a
> partitioned topic.
> 2. The client tries to create the topic using
> `persistent://public/default/XXXX-partition-YYY` not
> `persistent://public/default/XXXX-partition-YYY-partition-0`
>
> In this condition, we have s problem for users. If they want to delete this
> topic, awkward things happen.
>
> - If the users want to use delete-partition-topic admin API, The broker
> will return the error "Partitioned Topic Name should not contain
> '-partition-'".
> - If the users want to use delete-topic admin API, The broker will delete
> this topic, but the broker doesn't remove the partition topic metadata.
>
> So, maybe there have many methods to solve this problem. It looks like
> removing the `partition` keywords check for the `delete-partition` admin
> API. But it seems not to make sense. So, we chose to reject this topic
> creation directly and push a PR for it[2].
>
> Things are not over yet. In the past few days. After discussing with
> Penghui Li and Baodi Shi. We found another problem that will affect the
> dead letter queue(DLQ) If we reject topic creation directly. So, I push a
> PR[3] to revert it soon. And then draft this discussion.
>
> Please remember the phenomenon described above and take it into DLQ with
> the partitioned topic.
>
> When we want to create a Share model subscription, for example, that topic
> name is "persistent://public/default/test-topic". And we enabled DLQ.
> The client will try to create a partitioned topic that looks like
> `persistent://public/default/test-topic-partition-0`. And then, when we
> need to use DLQ, the client will use a new producer that topic name follows
> the DLQ default naming format:
>
> topicname-subscriptionname-DLQ =>
> > persistent://public/default/test-topic-partition-0-test-sub-DLQ
>
>
> With the allowAutoTopicCreation policy, we mentioned before. the topic
> `persistent://public/default/test-topic-partition-0-test-sub-DLQ` will
> create the partitioned topic metadata.
>
> Z-NODE:
>
>
>  
> paritioned-topics/public/defualt/persistent/test-topic-partition-0-test-sub-DLQ
>
> {"partitions": 1}
>
>
> And the client will also do the same things we mentioned above.
>
> So, there have some problems:
>
> - The DLQ topic is partitioned, but the client doesn't use
>
> `persistent://public/default/test-topic-partition-0-test-sub-DLQ-partition-0`
> to create this topic.
> - If the auto-creation policy `defaultNumPartitions` > 1, and the
> subscription type is `Exclusive`. the error will occur. Because the
> consumer already exists. Please check this logic.[4]
> - If the auto-creation policy `defaultNumPartitions` > 1, and the
> subscription type is `Shared`. we will create many consumers/producers to
> the same topic. The number is the number of partitions.[4]
> - Because this DLQ is not partitioned topic name in the broker and it has
> partitioned topic metadata. So, If we have some checks in the broker, we
> will get errors. Just like the replicator.[5]
>
>
> *Problem*
> After background explanation, we've got some problems to want to discuss:
>
> 1. Should we reject the partitioned topic metadata creation when the topic
> name contains the `-partition-` keywords?
> 2. If you want to reject the partitioned topic metadata creation, how can
> you handle DLQ?
> 3. If we don't let the client do the `-partition-` keyword check in
> `TopicName.get(topicName).getPartition(partitionIndex).toString();`, is the
> DLQ of the partition what we expect?
> 4. If we want to reject topic creation with `-partition-` keywords and make
> DLQ a non-partitioned topic. could change the DLQ naming format from
> `persistent://public/default/test-topic-partition-0-test-sub-DLQ` to
> `persistent://public/default/test-topic-test-sub-DLQ-partition-0`? because
> we will check it when creating partitioned topic metadata and then make it
> a non-partitioned topic.[6]
>
>
>
> Best,
> Mattison
>
> *Reference*
>
> [1]
>
> https://github.com/apache/pulsar/blob/7576c9303513ef8212452ff64a5a53ec7def6a5b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java#L231-L237
> [2] https://github.com/apache/pulsar/pull/14920
> [3] https://github.com/apache/pulsar/pull/16066
> [4]
>
> https://github.com/apache/pulsar/blob/7576c9303513ef8212452ff64a5a53ec7def6a5b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java#L1049-L1072
> [5]
>
> https://github.com/apache/pulsar/blob/0239d417e054be524e1e3fda9a7937647720289e/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java#L247-L259
> [6]
>
> https://github.com/apache/pulsar/blob/1097dbdf4853218b73dec3be5c83a82c26fe8385/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java#L2618-L2667
>

Reply via email to