You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2021/09/08 12:48:16 UTC

[GitHub] [pulsar] codelipenghui commented on issue #11962: PIP 94: Message converter at broker level

codelipenghui commented on issue #11962:
URL: https://github.com/apache/pulsar/issues/11962#issuecomment-915208660


   @BewareMyPower 
   > Once the converter was configured, before dispatching messages to Pulsar consumers, the converter would check if the buffer needs to be converted and then perform the conversion if necessary. 
   
   We should avoid converting(serialization and deserialization) the data at the broker side, this will put a very heavy burden on the broker GC. In my opinion, we should do the data converting at the client-side, we can have a diverse data format implementation and by default, the Pulsar client only has the Pulsar data format processor. if users want to consume the data with Kafka data format, they can add a separate dependency such as `org.apache.pulsar:kafka-format-converter`.
   
   The data to the Kafka client, should not be considered by the Pulsar broker, the KoP should handle it. If the data with Kafka format, KoP can send it directly to the Kafka consumer, if the data with Pulsar format, KoP needs to convert the data to the Kafka format.
   
   For the storage layer(BookKeeper and tiered storage), the data might be read directly bypass the broker such as PulsarSQL, Flink(In the future). This is also be considered if we are doing the data conversion at the broker side, we might need another implementation to read the data with multiple data formats from the BookKeeper/Tiered Storage. 
   
   @wangjialing218 
    > Just a idea. Current there is one ManagedLedger(ledger) associated with PersistentTopic. Could we add another ManagedLedger(kafkaLedger) associated with the topic.
   
   We can't use multiple managed ledgers for a topic, this will break the FIFO semantics if you have Kafka producers and Pulsar producers publishing data to the topic and Kafka consumers, Pulsar consumers to consume the data from the topic.
   And to read data repeatedly, we need to ensure the same reading order.
   
   @eolivelli 
   > do we need do add a Converter on the Publish side ?
   
   If the KoP wants to convert the data at the publishing path, KoP(Or other protocol handlers) can implement directly, any reason introduces the converter at the broker for the data publishing? And I think if using kafka format for KoP, KoP will convert the data on the publish side for now, this is an inefficient way.


-- 
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-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org