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

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

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

    https://github.com/apache/flink/pull/3314#discussion_r102668004
  
    --- Diff: 
flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java
 ---
    @@ -373,16 +370,28 @@ else if (partitionsRemoved) {
                                                                
keyPayload.get(keyBytes);
                                                        }
     
    -                                                   final T value = 
deserializer.deserialize(keyBytes, valueBytes, 
    -                                                                   
currentPartition.getTopic(), currentPartition.getPartition(), offset);
    -                                                   
    -                                                   if 
(deserializer.isEndOfStream(value)) {
    -                                                           // remove 
partition from subscribed partitions.
    -                                                           
partitionsIterator.remove();
    -                                                           continue 
partitionsLoop;
    -                                                   }
    -                                                   
    -                                                   owner.emitRecord(value, 
currentPartition, offset);
    +                                                   final Collector<T> 
collector = new Collector<T>() {
    --- End diff --
    
    What I think we should do to solve this correctly:
    
    Buffer the elements collected from the `deserialize` call. The 
`Collector.collect` implementation should simply add the collected element to 
the buffer, and not emit it immediately.
    
    After `deserialize` returns, call `emitRecord` once with all the elements 
in the buffer and the original record's offset. This, of course, would mean we 
need to slightly change the `emitRecord` implementation a bit to something like:
    ```
    void emitRecord(List<T> records, KafkaTopicPartitionState<KPH> 
partitionState, long offset) {
        synchronized (checkpointLock) {
            for (T record : records) {
                sourceContext.collect(record);
            }
            partitionState.setOffset(offset);
        }
    }
    ```
    
    After this, we proceed with the next record and repeat. Note that the 
emitting of all produced elements from record at offset 100L and the update to 
the offset state to 100L happens atomically synchronized on the checkpoint 
lock,  so we can make sure that a checkpoint barrier will only come either 
after or before all the produced records of offset 100, and not in-between.
    
    I think we should also be able to avoid a per-record `Collector` with this 
solution. We can reuse a `Collector` and provide it to the `deserializer` for 
every record, because it's simply only a means to collect elements to the 
internal buffer and we're not calling `emitRecords` in it.


> DeserializationSchema should handle zero or more outputs for every input
> ------------------------------------------------------------------------
>
>                 Key: FLINK-3679
>                 URL: https://issues.apache.org/jira/browse/FLINK-3679
>             Project: Flink
>          Issue Type: Bug
>          Components: DataStream API, Kafka Connector
>            Reporter: Jamie Grier
>            Assignee: Haohui Mai
>
> There are a couple of issues with the DeserializationSchema API that I think 
> should be improved.  This request has come to me via an existing Flink user.
> The main issue is simply that the API assumes that there is a one-to-one 
> mapping between input and outputs.  In reality there are scenarios where one 
> input message (say from Kafka) might actually map to zero or more logical 
> elements in the pipeline.
> Particularly important here is the case where you receive a message from a 
> source (such as Kafka) and say the raw bytes don't deserialize properly.  
> Right now the only recourse is to throw IOException and therefore fail the 
> job.  
> This is definitely not good since bad data is a reality and failing the job 
> is not the right option.  If the job fails we'll just end up replaying the 
> bad data and the whole thing will start again.
> Instead in this case it would be best if the user could just return the empty 
> set.
> The other case is where one input message should logically be multiple output 
> messages.  This case is probably less important since there are other ways to 
> do this but in general it might be good to make the 
> DeserializationSchema.deserialize() method return a collection rather than a 
> single element.
> Maybe we need to support a DeserializationSchema variant that has semantics 
> more like that of FlatMap.



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

Reply via email to