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

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

GitHub user pnowojski opened a pull request:

    https://github.com/apache/flink/pull/4928

    [FLINK-7732][kafka-consumer] Do not commit to kafka Flink's sentinel offsets

    ## What is the purpose of the change
    
    This pull request fixes a bug in all versions of `KafkaConsumer`s, that 
resulted in application crash, if snapshot before `KafkaConsumer` was able to 
asynchronously properly initialize.
    
    ## Verifying this change
    
    This change adds additional unit test in `AbstractFetcherTest` and a check 
state to ensure/enforce contract that internal sentinel offsets will not be 
committed to Kafka. 
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (yes / **no**)
      - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: (yes / **no**)
      - The serializers: (yes / **no** / don't know)
      - The runtime per-record code paths (performance sensitive): (yes / 
**no** / don't know)
      - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (yes / **no**)
      - If yes, how is the feature documented? (**not applicable** / docs / 
JavaDocs / not documented)
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/pnowojski/flink f7732

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/4928.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #4928
    
----
commit 9baea16ca69c27e5153808ce01592a47091c3131
Author: Piotr Nowojski <[email protected]>
Date:   2017-10-31T14:38:32Z

    [FLINK-7732][kafka-consumer] Do not commit to kafka Flink's sentinel offsets

----


> Test instability in Kafka end-to-end test (invalid Kafka offset)
> ----------------------------------------------------------------
>
>                 Key: FLINK-7732
>                 URL: https://issues.apache.org/jira/browse/FLINK-7732
>             Project: Flink
>          Issue Type: Bug
>          Components: Kafka Connector, Tests
>    Affects Versions: 1.4.0
>            Reporter: Nico Kruber
>            Assignee: Piotr Nowojski
>            Priority: Blocker
>              Labels: test-stability
>             Fix For: 1.4.0
>
>
> In a test run with unrelated changes in the network stack, the Kafa 
> end-to-end test was failing with an invalid offset:
> {code}
> 2017-09-28 06:34:10,736 INFO  org.apache.kafka.common.utils.AppInfoParser     
>               - Kafka version : 0.10.2.1
> 2017-09-28 06:34:10,744 INFO  org.apache.kafka.common.utils.AppInfoParser     
>               - Kafka commitId : e89bffd6b2eff799
> 2017-09-28 06:34:14,549 INFO  
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator  - Discovered 
> coordinator testing-gce-f67551b8-8867-41c5-b36e-4196d0f11e30:9092 (id: 
> 2147483647 rack: null) for group myconsumer.
> 2017-09-28 06:34:14,573 INFO  
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator  - Marking 
> the coordinator testing-gce-f67551b8-8867-41c5-b36e-4196d0f11e30:9092 (id: 
> 2147483647 rack: null) dead for group myconsumer
> 2017-09-28 06:34:14,686 INFO  
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator  - Discovered 
> coordinator testing-gce-f67551b8-8867-41c5-b36e-4196d0f11e30:9092 (id: 
> 2147483647 rack: null) for group myconsumer.
> 2017-09-28 06:34:14,687 INFO  
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator  - Marking 
> the coordinator testing-gce-f67551b8-8867-41c5-b36e-4196d0f11e30:9092 (id: 
> 2147483647 rack: null) dead for group myconsumer
> 2017-09-28 06:34:14,792 INFO  
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator  - Discovered 
> coordinator testing-gce-f67551b8-8867-41c5-b36e-4196d0f11e30:9092 (id: 
> 2147483647 rack: null) for group myconsumer.
> 2017-09-28 06:34:15,068 INFO  
> org.apache.flink.runtime.state.DefaultOperatorStateBackend    - 
> DefaultOperatorStateBackend snapshot (In-Memory Stream Factory, synchronous 
> part) in thread Thread[Async calls on Source: Custom Source -> Map -> Sink: 
> Unnamed (1/1),5,Flink Task Threads] took 948 ms.
> 2017-09-28 06:34:15,164 WARN  
> org.apache.flink.streaming.connectors.kafka.internal.Kafka09Fetcher  - 
> Committing offsets to Kafka failed. This does not compromise Flink's 
> checkpoints.
> java.lang.IllegalArgumentException: Invalid offset: -915623761772
>       at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.sendOffsetCommitRequest(ConsumerCoordinator.java:687)
>       at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.doCommitOffsetsAsync(ConsumerCoordinator.java:531)
>       at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.commitOffsetsAsync(ConsumerCoordinator.java:499)
>       at 
> org.apache.kafka.clients.consumer.KafkaConsumer.commitAsync(KafkaConsumer.java:1181)
>       at 
> org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread.run(KafkaConsumerThread.java:217)
> 2017-09-28 06:34:15,171 ERROR 
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase  - Async 
> Kafka commit failed.
> java.lang.IllegalArgumentException: Invalid offset: -915623761772
>       at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.sendOffsetCommitRequest(ConsumerCoordinator.java:687)
>       at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.doCommitOffsetsAsync(ConsumerCoordinator.java:531)
>       at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.commitOffsetsAsync(ConsumerCoordinator.java:499)
>       at 
> org.apache.kafka.clients.consumer.KafkaConsumer.commitAsync(KafkaConsumer.java:1181)
>       at 
> org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread.run(KafkaConsumerThread.java:217)
> {code}
> https://travis-ci.org/apache/flink/jobs/280722829
> [~pnowojski] did a first analysis that revealed this:
> In 
> org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java:229 
> this is being sent:
> {{long offsetToCommit = lastProcessedOffset + 1;}}
> {{lastProcessedOffset}} comes from:
> {{org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase#snapshotState}}
>  either lines 741 or 749
> The value that we see is strangely similiar to 
> {{org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionStateSentinel#GROUP_OFFSET}}
> {code}
> /**
>      * Magic number that defines the partition should start from its 
> committed group offset in Kafka.
>      *
>      * <p>This is used as a placeholder so that the actual committed group 
> offset can be evaluated lazily
>      * when the partition will actually start to be read by the consumer.
>      */
>     public static final long GROUP_OFFSET = -915623761773L;
> {code}
> This {{GROUP_OFFSET}} clearly was not intended to be committed, however 
> probably there is some race between this "lazy evaluation" and checkpoints 
> (which trigger committing offsets to kafka).



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to