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

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

GitHub user pnowojski opened a pull request:

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

    [FLINK-8086][kafka] Ignore ProducerFencedException

        ProducerFencedException can happen if we restore twice from the same 
checkpoint
        or if we restore from an old savepoint. In both cases transactional.ids 
that we
        want to recoverAndCommit have been already committed and reused. 
Reusing mean
        that they will be known by Kafka's brokers under newer 
producerId/epochId,
        which will result in ProducerFencedException if we try to commit again 
some
        old (and already committed) transaction.
        
        Ignoring this exception might hide some bugs/issues, because instead of 
failing
        we might have a semi silent (with a warning) data loss.
    
    ## Verifying this change
    
    This change added a `testFailAndRecoverSameCheckpointTwice` for a scenario 
that was previously not tested (and was failing).
    
    ## 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)
      - The S3 file system connector: (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 f8086

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

    https://github.com/apache/flink/pull/5030.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 #5030
    
----
commit d4542339e39adeff69cf18a647c8a5ac0d97969e
Author: Piotr Nowojski <[email protected]>
Date:   2017-11-17T13:31:07Z

    [hotfix][kafka] Improve logging in FlinkKafkaProducer011

commit 642ba6d0c3e204824f1ff69412d70918be9e3ac7
Author: Piotr Nowojski <[email protected]>
Date:   2017-11-17T13:40:30Z

    [FLINK-8086][kafka] Ignore ProducerFencedException during recovery
    
    ProducerFencedException can happen if we restore twice from the same 
checkpoint
    or if we restore from an old savepoint. In both cases transactional.ids 
that we
    want to recoverAndCommit have been already committed and reused. Reusing 
mean
    that they will be known by Kafka's brokers under newer producerId/epochId,
    which will result in ProducerFencedException if we try to commit again some
    old (and already committed) transaction.
    
    Ignoring this exception might hide some bugs/issues, because instead of 
failing
    we might have a semi silent (with a warning) data loss.

----


> FlinkKafkaProducer011 can permanently fail in recovery through 
> ProducerFencedException
> --------------------------------------------------------------------------------------
>
>                 Key: FLINK-8086
>                 URL: https://issues.apache.org/jira/browse/FLINK-8086
>             Project: Flink
>          Issue Type: Bug
>          Components: Kafka Connector
>    Affects Versions: 1.4.0, 1.5.0
>            Reporter: Stefan Richter
>            Assignee: Piotr Nowojski
>            Priority: Blocker
>             Fix For: 1.4.0
>
>
> Chaos monkey test in a cluster environment can permanently bring down our 
> FlinkKafkaProducer011.
> Typically, after a small number of randomly killed TMs, the data generator 
> job is no longer able to recover from a checkpoint because of the following 
> problem:
> org.apache.kafka.common.errors.ProducerFencedException: Producer attempted an 
> operation with an old epoch. Either there is a newer producer with the same 
> transactionalId, or the producer's transaction has been expired by the broker.
> The problem is reproduceable and happened for me in every run after the chaos 
> monkey killed a couple of TMs.



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

Reply via email to