[ 
https://issues.apache.org/jira/browse/BEAM-1573?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15906595#comment-15906595
 ] 

peay edited comment on BEAM-1573 at 3/12/17 5:23 PM:
-----------------------------------------------------

[~rangadi] I have made some good progress, with running the deserializer within 
Kafka on the consumer thread. An issue with this is how to implement 
{{approxBacklogInBytes}}. The current implementation actually deserializes 
manually (i.e., outside Kafka and the consumer thread) in {{advance}}. The 0.10 
API has {{ConsumeRecord.serializedKeySize()}} and 
{{ConsumerRecord.serializedValueSize()}} which would be perfect to handle that 
computation, but the 0.9 API does not have it. What would you suggest?


was (Author: peay):
[~rangadi] I have made some good progress, with running the deserializer within 
Kafka on the consumer thread. An issue with this is how to implement 
{{approxBacklogInBytes}}. The current implementation actually deserializes 
manually (i.e., outside Kafka and the consumer thread) in `advance`. The 0.10 
API has {{ConsumeRecord.serializedKeySize()}} and 
{{ConsumerRecord.serializedValueSize()}} which would be perfect to handle that 
computation, but the 0.9 API does not have it. What would you suggest?

> KafkaIO does not allow using Kafka serializers and deserializers
> ----------------------------------------------------------------
>
>                 Key: BEAM-1573
>                 URL: https://issues.apache.org/jira/browse/BEAM-1573
>             Project: Beam
>          Issue Type: Improvement
>          Components: sdk-java-extensions
>    Affects Versions: 0.4.0, 0.5.0
>            Reporter: peay
>            Assignee: Raghu Angadi
>            Priority: Minor
>             Fix For: Not applicable
>
>
> KafkaIO does not allow to override the serializer and deserializer settings 
> of the Kafka consumer and producers it uses internally. Instead, it allows to 
> set a `Coder`, and has a simple Kafka serializer/deserializer wrapper class 
> that calls the coder.
> I appreciate that allowing to use Beam coders is good and consistent with the 
> rest of the system. However, is there a reason to completely disallow to use 
> custom Kafka serializers instead?
> This is a limitation when working with an Avro schema registry for instance, 
> which requires custom serializers. One can write a `Coder` that wraps a 
> custom Kafka serializer, but that means two levels of un-necessary wrapping.
> In addition, the `Coder` abstraction is not equivalent to Kafka's 
> `Serializer` which gets the topic name as input. Using a `Coder` wrapper 
> would require duplicating the output topic setting in the argument to 
> `KafkaIO` and when building the wrapper, which is not elegant and error prone.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to