BewareMyPower commented on code in PR #22182: URL: https://github.com/apache/pulsar/pull/22182#discussion_r1531556737
########## pip/pip-344.md: ########## @@ -0,0 +1,87 @@ +# PIP-344: Correct the behavior of the public API pulsarClient.getPartitionsForTopic(topicName) + +# Background knowledge + +**Topic auto-creation** +- The partitioned topic auto-creation is dependent on `pulsarClient.getPartitionsForTopic` + - It triggers partitioned metadata creation by `pulsarClient.getPartitionsForTopic` + - And triggers the topic partition creation by producers' registration and consumers' registration. +- When calling `pulsarClient.getPartitionsForTopic(topicName)`, Pulsar will automatically create the partitioned topic metadata if it does not exist, either using `HttpLookupService` or `BinaryProtoLookupService`. + +**Now `pulsarClient.getPartitionsForTopic`'s behavior** +| case | broker allow `auto-create` | param allow <br> `create if not exists` | non-partitioned topic | partitioned topic | current behavior | +| --- | --- | --- | --- | --- | --- | +| 1 | `true/false` | `true/false` | `exists: true` | | REST API: `partitions: 0`<br> Client API: `partitions: 0` | +| 2 | `true/false` | `true/false` | | `exists: true` <br> `partitions: 3` | REST API: `partitions: 3`<br> Client API: `partitions: 3` | +| 3 | `true` | `true` | | | REST API: <br> - `create new: true` <br> - `partitions: 3` <br> Client API: <br> - `create new: true` <br> - `partitions: 3` <br> | +| 4 | `true` | `false` | | | REST API: <br> - `create new: false` <br> - `partitions: 0` <br> Client API: <br> not support <br> | +| 5 | `false` | `true` | | | REST API: <br> - `create new: false` <br> - `partitions: 0` <br> Client API: <br> - `create new: false` <br> - `partitions: 0` <br> | Review Comment: The table is not clear: - Which broker configuration do you mean by "broker allow `auto-create`"? - Which client configuration do you mean by "param allow `create if not exists`"? I see no extra configuration in the `getPartitionsForTopic` API. - Which REST API did you mean? Could you show the specific API? ########## pip/pip-344.md: ########## @@ -0,0 +1,87 @@ +# PIP-344: Correct the behavior of the public API pulsarClient.getPartitionsForTopic(topicName) + +# Background knowledge + +**Topic auto-creation** +- The partitioned topic auto-creation is dependent on `pulsarClient.getPartitionsForTopic` + - It triggers partitioned metadata creation by `pulsarClient.getPartitionsForTopic` + - And triggers the topic partition creation by producers' registration and consumers' registration. +- When calling `pulsarClient.getPartitionsForTopic(topicName)`, Pulsar will automatically create the partitioned topic metadata if it does not exist, either using `HttpLookupService` or `BinaryProtoLookupService`. + +**Now `pulsarClient.getPartitionsForTopic`'s behavior** +| case | broker allow `auto-create` | param allow <br> `create if not exists` | non-partitioned topic | partitioned topic | current behavior | +| --- | --- | --- | --- | --- | --- | +| 1 | `true/false` | `true/false` | `exists: true` | | REST API: `partitions: 0`<br> Client API: `partitions: 0` | +| 2 | `true/false` | `true/false` | | `exists: true` <br> `partitions: 3` | REST API: `partitions: 3`<br> Client API: `partitions: 3` | +| 3 | `true` | `true` | | | REST API: <br> - `create new: true` <br> - `partitions: 3` <br> Client API: <br> - `create new: true` <br> - `partitions: 3` <br> | +| 4 | `true` | `false` | | | REST API: <br> - `create new: false` <br> - `partitions: 0` <br> Client API: <br> not support <br> | +| 5 | `false` | `true` | | | REST API: <br> - `create new: false` <br> - `partitions: 0` <br> Client API: <br> - `create new: false` <br> - `partitions: 0` <br> | + +# Motivation + +The param `create if not exists` of the Client API is always `true.` + +- For case 4 of `pulsarClient.getPartitionsForTopic`'s behavior, it always tries to create the partitioned metadata, but the API name is `getxxx`. +- For case 5 of `pulsarClient.getPartitionsForTopic`'s behavior, it returns a `0` partitioned metadata, but the topic does not exist. For the correct behavior of this case, we had discussed [here](https://github.com/apache/pulsar/issues/8813) before. + +# Goals + +Correct the behaviors of case 4 and case 5. + +- Do not create the partitioned metadata when calling `pulsarClient.getPartitionsForTopic`. The partitioned metadata will only be created when consumers/producers are trying to register. +- Instead of returning a `0` partitioned metadata, respond to a not found error when calling `pulsarClient.getPartitionsForTopic` if the topic does not exist. + +# Detailed Design + +## Public-facing Changes + +When you call the public API `pulsarClient.getPartitionsForTopic`, pulsar will not create the partitioned metadata anymore. +- [flink-connector-pulsar](https://github.com/apache/flink-connector-pulsar/blob/main/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/ProducerRegister.java#L221-L227) is using this API to create partitioned topic metadata. Review Comment: It should be included in the **Motivation** section. Users should not depend on the implementation details. We should provide a new API to `PulsarClient` for this purpose like ```java // The future will be completed with the number of partitions CompletableFuture<Integer> createPartitionedTopicMetadataIfMissed(String topic); ``` or ```java CompletableFuture<Void> createPartitionedTopicMetadata(String topic, int numPartitions); ``` ########## pip/pip-344.md: ########## @@ -0,0 +1,87 @@ +# PIP-344: Correct the behavior of the public API pulsarClient.getPartitionsForTopic(topicName) + +# Background knowledge + +**Topic auto-creation** +- The partitioned topic auto-creation is dependent on `pulsarClient.getPartitionsForTopic` + - It triggers partitioned metadata creation by `pulsarClient.getPartitionsForTopic` + - And triggers the topic partition creation by producers' registration and consumers' registration. Review Comment: What do you mean here? I don't think the partitioned topic auto-creation is dependent on `PulsarClient.getPartitionsForTopic`. From the following description, it seems that the `PulsarClient.getPartitionsForTopic` API could create the partition metadata automatically. ########## pip/pip-344.md: ########## @@ -0,0 +1,87 @@ +# PIP-344: Correct the behavior of the public API pulsarClient.getPartitionsForTopic(topicName) + +# Background knowledge + +**Topic auto-creation** +- The partitioned topic auto-creation is dependent on `pulsarClient.getPartitionsForTopic` + - It triggers partitioned metadata creation by `pulsarClient.getPartitionsForTopic` + - And triggers the topic partition creation by producers' registration and consumers' registration. +- When calling `pulsarClient.getPartitionsForTopic(topicName)`, Pulsar will automatically create the partitioned topic metadata if it does not exist, either using `HttpLookupService` or `BinaryProtoLookupService`. + +**Now `pulsarClient.getPartitionsForTopic`'s behavior** +| case | broker allow `auto-create` | param allow <br> `create if not exists` | non-partitioned topic | partitioned topic | current behavior | +| --- | --- | --- | --- | --- | --- | +| 1 | `true/false` | `true/false` | `exists: true` | | REST API: `partitions: 0`<br> Client API: `partitions: 0` | +| 2 | `true/false` | `true/false` | | `exists: true` <br> `partitions: 3` | REST API: `partitions: 3`<br> Client API: `partitions: 3` | +| 3 | `true` | `true` | | | REST API: <br> - `create new: true` <br> - `partitions: 3` <br> Client API: <br> - `create new: true` <br> - `partitions: 3` <br> | +| 4 | `true` | `false` | | | REST API: <br> - `create new: false` <br> - `partitions: 0` <br> Client API: <br> not support <br> | +| 5 | `false` | `true` | | | REST API: <br> - `create new: false` <br> - `partitions: 0` <br> Client API: <br> - `create new: false` <br> - `partitions: 0` <br> | + +# Motivation + +The param `create if not exists` of the Client API is always `true.` + +- For case 4 of `pulsarClient.getPartitionsForTopic`'s behavior, it always tries to create the partitioned metadata, but the API name is `getxxx`. +- For case 5 of `pulsarClient.getPartitionsForTopic`'s behavior, it returns a `0` partitioned metadata, but the topic does not exist. For the correct behavior of this case, we had discussed [here](https://github.com/apache/pulsar/issues/8813) before. + +# Goals + +Correct the behaviors of case 4 and case 5. + +- Do not create the partitioned metadata when calling `pulsarClient.getPartitionsForTopic`. The partitioned metadata will only be created when consumers/producers are trying to register. +- Instead of returning a `0` partitioned metadata, respond to a not found error when calling `pulsarClient.getPartitionsForTopic` if the topic does not exist. + +# Detailed Design + +## Public-facing Changes + +When you call the public API `pulsarClient.getPartitionsForTopic`, pulsar will not create the partitioned metadata anymore. +- [flink-connector-pulsar](https://github.com/apache/flink-connector-pulsar/blob/main/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/ProducerRegister.java#L221-L227) is using this API to create partitioned topic metadata. + +### Public API +**LookupService.java** +``` +// This API existed before. Not change it, thus ensuring compatibility. +CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadata(TopicName topicName); + ++ // A new API that contains an additional param "createIfAutoCreationEnabled." ++ CompletableFuture<PartitionedTopicMetadata> getPartitionedTopicMetadata(TopicName topicName, boolean createIfAutoCreationEnabled); Review Comment: `LookupService` is a public interface under the `pulsar-client` module, not the `pulsar-client-api` module. We should not encourage users to leverage such APIs because the semantics might change in the next minor release. So I suggest just changing the semantic of the original `getPartitionedTopicMetadata` rather than adding a new API and leaving the old API never used internally. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@pulsar.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org