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-4723) Unify behaviour of committed offsets to Kafka / ZK for Kafka 0.8 and 0.9 consumer
Date Tue, 04 Oct 2016 17:07:22 GMT

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

ASF GitHub Bot commented on FLINK-4723:

Github user StephanEwen commented on the issue:

    Looks quite good. I would suggest one change, though:
    Can we avoid copying the offsets in the checkpoint into a new map (with increment by one)
and passing that to the ZooKeeper Offset Committer or the Kafka Offset Committer? I am just
not a big fan of copying things back and forth (especially in "prepareSnaoshot()", which we
want to keep as lightweight as possible). Instead, can we have the contract that the offset
committers always commit "+1" from the value they get (pretty much as it was in the 0.9 committer
after FLINK-4618)?
    Concerning the tests, is the stability issue fixed there?
    What I frequently do is push the same commit to 10 different newly created branches to
keep Travis busy over night with 10 test runs and see if I see a stability issue.

> Unify behaviour of committed offsets to Kafka / ZK for Kafka 0.8 and 0.9 consumer
> ---------------------------------------------------------------------------------
>                 Key: FLINK-4723
>                 URL: https://issues.apache.org/jira/browse/FLINK-4723
>             Project: Flink
>          Issue Type: Improvement
>          Components: Kafka Connector
>            Reporter: Tzu-Li (Gordon) Tai
>            Assignee: Tzu-Li (Gordon) Tai
>             Fix For: 1.2.0, 1.1.3
> The proper "behaviour" of the offsets committed back to Kafka / ZK should be "the next
offset that consumers should read (in Kafka terms, the 'position')".
> This is already fixed for the 0.9 consumer by FLINK-4618, by incrementing the committed
offsets back to Kafka by the 0.9 by 1, so that the internal {{KafkaConsumer}} picks up the
correct start position when committed offsets are present. This fix was required because the
start position from committed offsets was implicitly determined with Kafka 0.9 APIs.
> However, since the 0.8 consumer handles offset committing and start position using Flink's
own {{ZookeeperOffsetHandler}} and not Kafka's high-level APIs, the 0.8 consumer did not require
a fix.
> I propose to still unify the behaviour of committed offsets across 0.8 and 0.9 to the
definition above.
> Otherwise, if users in any case first uses the 0.8 consumer to read data and have Flink-committed
offsets in ZK, and then uses a high-level 0.8 Kafka consumer to read the same topic in a non-Flink
application, the first record will be duplicate (because, like described above, Kafka high-level
consumers expect the committed offsets to be "the next record to process" and not "the last
processed record").
> This requires incrementing the committed ZK offsets in 0.8 to also be incremented by
1, and changing how Flink internal offsets are initialized with accordance to the acquired
ZK offsets.

This message was sent by Atlassian JIRA

View raw message