If a partitioned topic lacks metadata, that indicates an unusual and inconsistent state.
For the demo, I understand your intention to use a partitioned one after the partitioned topic is deleted, right now it should be the non-partitioned topic. I'm open to adding a switch to verify topic consistency, but I don’t think that’s the right long-term direction. As the downstream, the priority should be aligning with upstream changes rather than trying to prevent them. Let's proceed with reverting #24118 from branch-3.0, branch-4.3, and branch-4.0. We'll keep this feature only in Pulsar@4.1. Thanks, Zixuan Yunze Xu <x...@apache.org> 于2025年4月23日周三 15:50写道: > Hi Zixuan, > > The use case is that the client side does not care about how many > partitions the topic has. Creating a partitioned topic for this use > case just acts as a convenient approach to create and delete all these > partitions, as well as querying the total count and the whole > properties. > > However, the partition metadata is not required for this use case. > Actually we don't have to respect the number of partitions. For > example, given a topic with 50 partitions, creating 50 producers and > 50 consumers on that partitioned topic might bring too much pressure > on an application without much CPU. We can configure the number of > partitions manually, here is an example that explains the use case > > ```java > // Before the restart (numPartitions is configured statically) > final var numPartitions = 50; > admin.topics().createPartitionedTopic(topic, 50); > final var producers = new ArrayList<Producer<byte[]>>(); > for (int i = 0; i < numPartitions; i++) { > producers.add(pulsarClient.newProducer().topic(topic + > TopicName.PARTITIONED_TOPIC_SUFFIX + i).create()); > } > // Now there are 50 producers > for (var producer : producers) { > producer.close(); > } > producers.clear(); > // After the restart (we changed the numPartitions config from 50 to 4) > admin.topics().deletePartitionedTopic(topic, true); > final var newNumPartitions = 4; > for (int i = 0; i < newNumPartitions; i++) { > producers.add(pulsarClient.newProducer().topic(topic + > TopicName.PARTITIONED_TOPIC_SUFFIX + i).create()); > } > // Now there are only 4 producers > ``` > > In the code above, deleting the partitioned topic can clear old data > and avoid new partitions being deleted by deleting the partitioned > topic. > > Thanks, > yunze > > On Wed, Apr 23, 2025 at 11:47 AM Zixuan Liu <zix...@apache.org> wrote: > > > > I noticed that the following KoP tests need to be updated: > > OffsetResetTest.java, GroupCoordinatorTest.java, and > > TransactionStateManagerTest.java. The breakage is limited to test > scenarios > > only. > > > It has been valid and allowed to create producers or readers > > directly on partitions if the partition count is known in advance > > > > PR #24118 does not break this feature. If the partition count (i.e., > > partitioned metadata) is known, the broker still allows producers and > > consumers to create partitions as expected. > > > There is a trick that this partitioned topic can be deleted by force > > and then producers will reconnect the partitions to continue > > producing. However, with this change, the producers will fail to > > reconnect due to `NotAllowedException`, Sometimes, users just want to > > discard huge legacy data. Regarding the loss of partition metadata, it > > can be recreated by creating the same partitioned topic again. I admit > > it's a tricky solution. > > > > In cases where a partitioned topic is forcibly deleted while the producer > > or consumer is online, the client will retry and may end up creating a > > non-partitioned topic with `-partition-`. Since the partitioned metadata > is > > deleted, the client will no longer recognize it as a partitioned topic. > > Upon restart, producers and consumers won’t be able to publish or consume > > messages as expected from a partitioned topic. > > PR #24118 can prevent this scenario. It enforces that users must > explicitly > > create the partitioned metadata. > > > > I also discuss this issue on slack: > > https://apache-pulsar.slack.com/archives/C5ZSVEN4E/p1742803935952389 > > > > Once we've concluded this discussion, I'll review your PR. Thanks again > for > > the fix! > > Thanks, > > Zixuan > > > > Yunze Xu <x...@apache.org> 于2025年4月23日周三 00:34写道: > > > > > Hi all, > > > > > > I prepared a quick fix: https://github.com/apache/pulsar/pull/24195 so > > > that we can fix the regression quickly in 3.0, 3.3 and 4.0 releases. I > > > devoted my time addressing this breaking change because it's my > > > responsibility that I missed the previous open discussion in the ML > > > list. But I didn't have more time to write a PIP and go through the > > > PIP process for that. > > > > > > Thanks, > > > Yunze > > > > > > On Tue, Apr 22, 2025 at 11:24 PM Yunze Xu <x...@apache.org> wrote: > > > > > > > > Hi Zixuan, > > > > > > > > If you're maintaining your own private KoP fork and upgrading the > > > > Pulsar dependency to include this PR, all tests in > > > > `GroupCoordinatorTest` fail without changes. I didn't have a chance > to > > > > look into all the failed tests for now. But at least, there are 37 > > > > tests failing due to the same reason. I've verified that some can be > > > > fixed by creating a partitioned topic manually. > > > > > > > > Even out of the scope of KoP, Pulsar's partitioned topic can be used > > > > as a few partitions and a metadata that represents the partition > > > > count. It has been valid and allowed to create producers or readers > > > > directly on partitions if the partition count is known in advance > > > > (e.g. via the admin API or UI). > > > > - producer0 on `topic-partition-0` > > > > - producer1 on `topic-partition-1` > > > > - ... > > > > > > > > There is a trick that this partitioned topic can be deleted by force > > > > and then producers will reconnect the partitions to continue > > > > producing. However, with this change, the producers will fail to > > > > reconnect due to `NotAllowedException`, Sometimes, users just want to > > > > discard huge legacy data. Regarding the loss of partition metadata, > it > > > > can be recreated by creating the same partitioned topic again. I > admit > > > > it's a tricky solution. > > > > > > > > Thanks, > > > > Yunze > > > > > > > > > > > > > > > > > > > > On Tue, Apr 22, 2025 at 10:33 PM Zixuan Liu <zix...@apache.org> > wrote: > > > > > > > > > > I'm aware that #24118 breaks some test cases, but I believe those > test > > > > > cases were incorrect. I also flagged this PR as introducing > breaking > > > > > changes in the mailing list, please see > > > > > https://lists.apache.org/thread/g1jq0vkq0wt28mdnbg63bpw8fpx9ml42 > > > > > > > > > > I'd like to better understand the scope of the breakage. Is it > limited > > > to > > > > > internal KoP usage—such as test cases where developers use > > > > > partitioned-topic format without metadata? Or does it also affect > > > users who > > > > > might be doing this in practice? In our organization, we also > maintain > > > KoP > > > > > and have updated our test cases accordingly. > > > > > > > > > > Personally, I believe that the partitioned topic should have > associated > > > > > metadata. If someone uses the partitioned format without metadata, > it > > > feels > > > > > more like a workaround or an unsupported edge case. > > > > > > > > > > I'm open to discussion. If this behavior is something users > actually > > > depend > > > > > on, then we should talk about how to handle it more gracefully. > > > > > > > > > > Thanks, > > > > > Zixuan > > > >