flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-4616) Kafka consumer doesn't store last emmited watermarks per partition in state
Date Thu, 26 Jan 2017 06:21:25 GMT

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

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

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

    https://github.com/apache/flink/pull/3031#discussion_r97758477
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
---
    @@ -345,39 +345,39 @@ public void snapshotState(FunctionSnapshotContext context) throws
Exception {
     			LOG.debug("snapshotState() called on closed source");
     		} else {
     
    -			offsetsStateForCheckpoint.clear();
    +			offsetsAndWatermarksStateForCheckpoint.clear();
     
     			final AbstractFetcher<?, ?> fetcher = this.kafkaFetcher;
     			if (fetcher == null) {
     				// the fetcher has not yet been initialized, which means we need to return the
    -				// originally restored offsets or the assigned partitions
    +				// originally restored offsets and watermarks or the assigned partitions
     
    -				if (restoreToOffset != null) {
    +				if (restoreToOffsetAndWatermark != null) {
     
    -					for (Map.Entry<KafkaTopicPartition, Long> kafkaTopicPartitionLongEntry : restoreToOffset.entrySet())
{
    -						offsetsStateForCheckpoint.add(
    -								Tuple2.of(kafkaTopicPartitionLongEntry.getKey(), kafkaTopicPartitionLongEntry.getValue()));
    +					for (Map.Entry<KafkaTopicPartition, Tuple2<Long, Long>> kafkaTopicPartitionOffsetAndWatermark
: restoreToOffsetAndWatermark.entrySet()) {
    +						offsetsAndWatermarksStateForCheckpoint.add(
    +								Tuple2.of(kafkaTopicPartitionOffsetAndWatermark.getKey(), kafkaTopicPartitionOffsetAndWatermark.getValue()));
    --- End diff --
    
    Having a specific checkpoint state object will also be helpful for code readability in
situations like this one (it's quite tricky to understand quickly what the key / value refers
to, as well as some of the `f0`, `f1` calls in other parts of the PR. I know the previous
code used `f0` and `f1` also, but I think it's a good opportunity to improve that).


> Kafka consumer doesn't store last emmited watermarks per partition in state
> ---------------------------------------------------------------------------
>
>                 Key: FLINK-4616
>                 URL: https://issues.apache.org/jira/browse/FLINK-4616
>             Project: Flink
>          Issue Type: Bug
>          Components: Kafka Connector
>    Affects Versions: 1.1.1
>            Reporter: Yuri Makhno
>            Assignee: Roman Maier
>
> Kafka consumers stores in state only kafka offsets and doesn't store last emmited watermarks,
this may go to wrong state when checkpoint is restored:
> Let's say our watermark is (timestamp - 10) and in case we have the following messages
queue results will be different after checkpoint restore and during normal processing:
> A(ts = 30)
> B(ts = 35)
> ------ checkpoint goes here
> C(ts=15) -- this one should be filtered by next time window
> D(ts=60)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message