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

ASF GitHub Bot commented on FLINK-8500:
---------------------------------------

Github user FredTing commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5958#discussion_r186479026
  
    --- Diff: 
flink-core/src/main/java/org/apache/flink/api/common/serialization/DeserializationSchema.java
 ---
    @@ -42,14 +42,22 @@
     @Public
     public interface DeserializationSchema<T> extends Serializable, 
ResultTypeQueryable<T> {
     
    +   /**
    +    * @deprecated Use {@link #deserialize(ConsumerRecordMetaInfo)} .
    +    */
    +   @Deprecated
    +   T deserialize(byte[] message) throws IOException;
    +
        /**
         * Deserializes the byte message.
         *
    -    * @param message The message, as a byte array.
    +    * @param consumerRecordMetaInfossage The message, as a {@link 
ConsumerRecordMetaInfo}.
         *
         * @return The deserialized message as an object (null if the message 
cannot be deserialized).
         */
    -   T deserialize(byte[] message) throws IOException;
    +   default T deserialize(ConsumerRecordMetaInfo 
consumerRecordMetaInfossage) throws IOException {
    --- End diff --
    
    I agree that it's probably better to make separate `DeserializationSchema` 
classes, for each connector type. But for now I think this is a relative easy 
fix without breaking the Flink API for the custom deserializers. There is 
already some discussion about redesigning the connectors (see issue 5479) with 
a `common connector framework` in mind. I think that would be a good place to 
decide what to do with a shared `DeserializationSchema`.


> 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)

Reply via email to