You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Aljoscha Krettek (JIRA)" <ji...@apache.org> on 2018/05/02 10:56:00 UTC

[jira] [Commented] (FLINK-8500) Get the timestamp of the Kafka message from kafka consumer(Kafka010Fetcher)

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

Aljoscha Krettek commented on FLINK-8500:
-----------------------------------------

Thanks for opening the PR and pushing the discussion! I think we have to take a step back, though, and reconsider the design of the deserialisation schemes. I don't think we can keep adding ever more schemes when new features come up. I think a simple fix for that is to introduce a deserialisation scheme that accepts a Kafka {{ConsumerRecord}}, all the information that you could ever want to access has to be in there, since it's the only thing we get from Kafka. This would also greatly simply the methods and would unify keyed/non-keyed schemes.

[~tzulitai] What do you think about this? 

> Get the timestamp of the Kafka message from kafka consumer(Kafka010Fetcher)
> ---------------------------------------------------------------------------
>
>                 Key: FLINK-8500
>                 URL: https://issues.apache.org/jira/browse/FLINK-8500
>             Project: Flink
>          Issue Type: Improvement
>          Components: Kafka Connector
>    Affects Versions: 1.4.0
>            Reporter: yanxiaobin
>            Priority: Major
>             Fix For: 1.6.0
>
>         Attachments: image-2018-01-30-14-58-58-167.png, image-2018-01-31-10-48-59-633.png
>
>
> The method deserialize of KeyedDeserializationSchema  needs a parameter 'kafka message timestamp' (from ConsumerRecord) .In some business scenarios, this is useful!
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)