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

    https://github.com/apache/flink/pull/2509#discussion_r89664876
  
    --- Diff: 
flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java
 ---
    @@ -218,26 +221,57 @@ public void run() {
                                }
                        }
     
    -                   // seek the consumer to the initial offsets
    +                   List<KafkaTopicPartition> partitionsWithNoOffset = new 
ArrayList<>();
                        for (KafkaTopicPartitionState<TopicPartition> partition 
: subscribedPartitions()) {
                                if (partition.isOffsetDefined()) {
                                        LOG.info("Partition {} has restored 
initial offsets {} from checkpoint / savepoint; seeking the consumer " +
                                                "to position {}", 
partition.getKafkaPartitionHandle(), partition.getOffset(), 
partition.getOffset() + 1);
     
                                        
consumer.seek(partition.getKafkaPartitionHandle(), partition.getOffset() + 1);
                                } else {
    -                                   // for partitions that do not have 
offsets restored from a checkpoint/savepoint,
    -                                   // we need to define our internal 
offset state for them using the initial offsets retrieved from Kafka
    -                                   // by the KafkaConsumer, so that they 
are correctly checkpointed and committed on the next checkpoint
    +                                   
partitionsWithNoOffset.add(partition.getKafkaTopicPartition());
    +                           }
    +                   }
     
    -                                   long fetchedOffset = 
consumer.position(partition.getKafkaPartitionHandle());
    +                   if (partitionsWithNoOffset.size() == 
subscribedPartitions().length) {
    +                           // if all partitions have no initial offsets, 
that means we're starting fresh
    +                           switch (startupMode) {
    +                                   case EARLIEST:
    +                                           LOG.info("Setting starting 
point as earliest offset for partitions {}", partitionsWithNoOffset);
    +
    +                                           
seekPartitionsToBeginning(consumer, 
convertKafkaPartitions(subscribedPartitions()));
    --- End diff --
    
    The problem with this one is that the `seekToBeginning` method broke 
compatibility from 0.8 to 0.9+.
    In 0.8, it's `seekToBeginning(TopicPartition...)` while in 0.9+ it's 
`seekToBeginning(Collection<TopicPartition>)`.
    
    I'll integrate these seek methods into the `KafkaConsumerCallBridge` 
introduced in a recent PR. I'll be inevitable that we must redundantly do the 
Array -> List conversion because our `subscribedPartitions` is an Array, while 
0.9+ methods take an API. For the 0.8 methods, instead of converting the list 
back to an array, I'll just iterate over the list and call 
`seekPartitionsToBeginning` for each one.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to