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

    https://github.com/apache/flink/pull/3031#discussion_r97935696
  
    --- Diff: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java
 ---
    @@ -175,34 +176,115 @@ protected AbstractFetcher(
        // 
------------------------------------------------------------------------
     
        /**
    -    * Takes a snapshot of the partition offsets.
    +    * Takes a snapshot of the partition offsets and watermarks.
         * 
         * <p>Important: This method mus be called under the checkpoint lock.
         * 
    -    * @return A map from partition to current offset.
    +    * @return A map from partition to current offset and watermark.
         */
    -   public HashMap<KafkaTopicPartition, Long> snapshotCurrentState() {
    +   public HashMap<KafkaTopicPartition, Tuple2<Long, Long>> 
snapshotCurrentState() {
                // this method assumes that the checkpoint lock is held
                assert Thread.holdsLock(checkpointLock);
     
    -           HashMap<KafkaTopicPartition, Long> state = new 
HashMap<>(allPartitions.length);
    -           for (KafkaTopicPartitionState<?> partition : 
subscribedPartitions()) {
    -                   state.put(partition.getKafkaTopicPartition(), 
partition.getOffset());
    +           HashMap<KafkaTopicPartition, Tuple2<Long, Long>> state = new 
HashMap<>(allPartitions.length);
    +
    +           switch (timestampWatermarkMode) {
    +
    +                   case NO_TIMESTAMPS_WATERMARKS: {
    +
    +                           for (KafkaTopicPartitionState<KPH> partition : 
allPartitions) {
    +                                   
state.put(partition.getKafkaTopicPartition(), Tuple2.of(partition.getOffset(), 
Long.MIN_VALUE));
    +                           }
    +
    +                           return state;
    +                   }
    +
    +                   case PERIODIC_WATERMARKS: {
    +                           
KafkaTopicPartitionStateWithPeriodicWatermarks<T, KPH> [] partitions =
    +                                   
(KafkaTopicPartitionStateWithPeriodicWatermarks<T, KPH> []) allPartitions;
    +
    +                           for 
(KafkaTopicPartitionStateWithPeriodicWatermarks<T, KPH> partition : partitions) 
{
    +                                   
state.put(partition.getKafkaTopicPartition(), Tuple2.of(partition.getOffset(), 
partition.getCurrentWatermarkTimestamp()));
    +                           }
    +
    +                           return state;
    +                   }
    +
    +                   case PUNCTUATED_WATERMARKS: {
    +                           
KafkaTopicPartitionStateWithPunctuatedWatermarks<T, KPH> [] partitions =
    +                                   
(KafkaTopicPartitionStateWithPunctuatedWatermarks<T, KPH> []) allPartitions;
    +
    +                           for 
(KafkaTopicPartitionStateWithPunctuatedWatermarks<T, KPH> partition : 
partitions) {
    +                                   
state.put(partition.getKafkaTopicPartition(), Tuple2.of(partition.getOffset(), 
partition.getCurrentPartitionWatermark()));
    +                           }
    +
    +                           return state;
    +                   }
    +
    +                   default:
    +                           // cannot happen, add this as a guard for the 
future
    +                           throw new RuntimeException();
    --- End diff --
    
    Would be good to have a reason message here.


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