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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to