Hi Kevin,

Which Kafka client version are you using? (=What is effectively bundled
into your application jar?)

On Wed, Aug 4, 2021 at 5:56 PM Kevin Lam <kevin....@shopify.com> wrote:

> Thanks Matthias.
>
> I just tried this backport (https://github.com/apache/flink/pull/16693)
> and got the following error, with the reproduction I described in
> https://lists.apache.org/thread.html/r528102e08d19d3ae446e5df75710009128c736735c0aaf310f95abeb%40%3Cuser.flink.apache.org%3E
> (ie. starting job with exactly_once, waited for some checkpoints to
> complete, and killed a task manager, job fails to recover from checkpoint)
>
> 2021-08-04 11:46:38
> java.lang.RuntimeException: Incompatible KafkaProducer version
> at
> org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer.createProducerIdAndEpoch(FlinkKafkaInternalProducer.java:299)
> at
> org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer.resumeTransaction(FlinkKafkaInternalProducer.java:233)
> at
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.recoverAndCommit(FlinkKafkaProducer.java:1029)
> at
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.recoverAndCommit(FlinkKafkaProducer.java:99)
> at
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction.recoverAndCommitInternal(TwoPhaseCommitSinkFunction.java:414)
> at
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction.initializeState(TwoPhaseCommitSinkFunction.java:364)
> at
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.initializeState(FlinkKafkaProducer.java:1195)
> at
> org.apache.flink.streaming.util.functions.StreamingFunctionUtils.tryRestoreFunction(StreamingFunctionUtils.java:189)
> at
> org.apache.flink.streaming.util.functions.StreamingFunctionUtils.restoreFunctionState(StreamingFunctionUtils.java:171)
> at
> org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.initializeState(AbstractUdfStreamOperator.java:96)
> at
> org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.initializeOperatorState(StreamOperatorStateHandler.java:118)
> at
> org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:290)
> at
> org.apache.flink.streaming.runtime.tasks.OperatorChain.initializeStateAndOpenOperators(OperatorChain.java:436)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.restoreGates(StreamTask.java:582)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.call(StreamTaskActionExecutor.java:55)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.executeRestore(StreamTask.java:562)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.runWithCleanUpOnFail(StreamTask.java:647)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.restore(StreamTask.java:537)
> at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:759)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:566)
> at java.base/java.lang.Thread.run(Unknown Source)
> Caused by: java.lang.ClassNotFoundException:
> org.apache.kafka.clients.producer.internals.ProducerIdAndEpoch
> at java.base/java.net.URLClassLoader.findClass(Unknown Source)
> at java.base/java.lang.ClassLoader.loadClass(Unknown Source)
> at
> org.apache.flink.util.FlinkUserCodeClassLoader.loadClassWithoutExceptionHandling(FlinkUserCodeClassLoader.java:64)
> at
> org.apache.flink.util.ChildFirstClassLoader.loadClassWithoutExceptionHandling(ChildFirstClassLoader.java:74)
> at
> org.apache.flink.util.FlinkUserCodeClassLoader.loadClass(FlinkUserCodeClassLoader.java:48)
> at java.base/java.lang.ClassLoader.loadClass(Unknown Source)
> at java.base/java.lang.Class.forName0(Native Method)
> at java.base/java.lang.Class.forName(Unknown Source)
> at
> org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer.createProducerIdAndEpoch(FlinkKafkaInternalProducer.java:291)
> ... 20 more
>
> Any ideas?
>
> On Tue, Aug 3, 2021 at 10:34 AM Schwalbe Matthias <
> matthias.schwa...@viseca.ch> wrote:
>
>> I don’t think that this risk is there, I guess Fabian a quick response …
>> see also the later comments in the ticket.
>>
>>
>>
>> Thias
>>
>>
>>
>> *From:* Kevin Lam <kevin....@shopify.com>
>> *Sent:* Dienstag, 3. August 2021 15:56
>> *To:* Schwalbe Matthias <matthias.schwa...@viseca.ch>
>> *Cc:* user <user@flink.apache.org>; fabianp...@ververica.com
>> *Subject:* Re: FW: Error using KafkaProducer EXACTLY_ONCE semantic +
>> TaskManager Failure
>>
>>
>>
>> Thank you for the replies!
>>
>>
>>
>> Thias, I will look into that issue and the workaround. Fabian Paul had
>> replied on https://issues.apache.org/jira/browse/FLINK-23509 mentioning
>> that our workaround could result in data loss--is that a risk with the
>> workaround?
>>
>>
>>
>> Fabian Paul, yes here is some more information:
>>
>>
>>
>> Checkpoint config (see [0] for more):
>>
>> Frequency: 2 minutes
>>
>> No concurrent checkpoints
>>
>> Checkpoint duration roughly
>>
>>
>>
>> Overall parallelism 13 - 70 depending on the operator
>>
>>
>>
>> The full exception available in the job manager logs and exception tab in
>> the UI:
>>
>>
>>
>> ```
>>
>> 2021-08-03 09:53:42
>> org.apache.kafka.common.KafkaException: Unexpected error in
>> InitProducerIdResponse; 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.
>> at
>> org.apache.kafka.clients.producer.internals.TransactionManager$InitProducerIdHandler.handleResponse(TransactionManager.java:1352)
>> at
>> org.apache.kafka.clients.producer.internals.TransactionManager$TxnRequestHandler.onComplete(TransactionManager.java:1260)
>> at
>> org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)
>> at
>> org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:572)
>> at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:564)
>> at
>> org.apache.kafka.clients.producer.internals.Sender.maybeSendAndPollTransactionalRequest(Sender.java:414)
>> at
>> org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:312)
>> at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:239)
>> at java.base/java.lang.Thread.run(Unknown Source)
>>
>> ```
>>
>>
>>
>> I saw that you have recently opened
>> https://github.com/apache/flink/pull/16692, will this be fixing the
>> issue?
>>
>>
>>
>> [0]
>>
>> [image: image.png]
>>
>>
>>
>> On Tue, Aug 3, 2021 at 8:27 AM Schwalbe Matthias <
>> matthias.schwa...@viseca.ch> wrote:
>>
>> … I just realized my former answer went only to Kevin directly.
>>
>> (Still exercising with the mail list 😊 )
>>
>>
>>
>> This was my original answer, there is a workaround until the respective
>> ticket is solved.
>>
>>
>>
>> Hope this helps
>>
>>
>>
>> Thias
>>
>>
>>
>> *From:* Schwalbe Matthias
>> *Sent:* Freitag, 30. Juli 2021 08:13
>> *To:* Kevin Lam <kevin....@shopify.com>
>> *Subject:* RE: Error using KafkaProducer EXACTLY_ONCE semantic +
>> TaskManager Failure
>>
>>
>>
>> Hi Kevin,
>>
>>
>>
>> I’ve had this exactly the same way. It has got to do with a regression
>> bug in FlinkKafkaInternalProducer#resumeTransaction
>> <https://github.com/apache/flink/blob/f06faf13930f2e8acccf1e04e2c250b85bdbf48e/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/FlinkKafkaInternalProducer.java#L229>
>>  that
>> surfaces for too new versions of Kafka client.
>>
>>
>>
>> I’ve created a bug ticket:
>> https://issues.apache.org/jira/browse/FLINK-23509 and there propose a
>> work around until the problem is fixed.
>>
>> Also these references are related to the same problem:
>>
>> https://issues.apache.org/jira/browse/FLINK-16419, and
>> https://lists.apache.org/list.html?user@flink.apache.org:lte=2M:recover%20from%20svaepoint
>> <https://lists.apache.org/list.html?user%40flink.apache.org:lte=2M:recover%20from%20svaepoint>
>>
>>
>>
>> Feel free to get back to me for clarifications 😊
>>
>>
>>
>> Thias
>>
>>
>>
>> *From:* Kevin Lam <kevin....@shopify.com>
>> *Sent:* Donnerstag, 29. Juli 2021 19:32
>> *To:* user <user@flink.apache.org>
>> *Subject:* Error using KafkaProducer EXACTLY_ONCE semantic + TaskManager
>> Failure
>>
>>
>>
>> Hi user@,
>>
>>
>>
>> We're developing a Flink application, and using the *FlinkKafkaProducer*
>> *.**Semantic**.**EXACTLY_ONCE* producer semantic to output records to a
>> Kafka topic in an exactly-once way. We run our flink application on
>> kubernetes.
>>
>>
>>
>> I've observed that if a task manager fails (I've simulated this by
>> killing a task-manager pod), the job will not recover or restore from the
>> most recent checkpoint, and will instead enter a crash loop with the
>> following types of errors [0]. If I try the same experiment (run the job,
>> kill a task manager pod) with the AT_LEAST_ONCE semantic, the job recovers
>> using the most recent checkpoint as expected.
>>
>>
>>
>> We've set the transaction.timeout.ms to be 1 hour on both the broker and
>> producer side.
>>
>>
>>
>> Any insights into what we could be doing wrong or what's going on are
>> appreciated.
>>
>>
>>
>> Thanks in advance!
>>
>>
>>
>> [0]: ```
>>
>> 2021-07-28 16:55:32
>> org.apache.kafka.common.KafkaException: Unexpected error in
>> InitProducerIdResponse; 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.
>> at
>> org.apache.kafka.clients.producer.internals.TransactionManager$InitProducerIdHandler.handleResponse(TransactionManager.java:1352)
>> at
>> org.apache.kafka.clients.producer.internals.TransactionManager$TxnRequestHandler.onComplete(TransactionManager.java:1260)
>> at
>> org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:109)
>> at
>> org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:572)
>> at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:564)
>> at
>> org.apache.kafka.clients.producer.internals.Sender.maybeSendAndPollTransactionalRequest(Sender.java:414)
>> at
>> org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:312)
>> at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:239)
>> at java.base/java.lang.Thread.run(Unknown Source)
>>
>> ```
>>
>> Diese Nachricht ist ausschliesslich für den Adressaten bestimmt und
>> beinhaltet unter Umständen vertrauliche Mitteilungen. Da die
>> Vertraulichkeit von e-Mail-Nachrichten nicht gewährleistet werden kann,
>> übernehmen wir keine Haftung für die Gewährung der Vertraulichkeit und
>> Unversehrtheit dieser Mitteilung. Bei irrtümlicher Zustellung bitten wir
>> Sie um Benachrichtigung per e-Mail und um Löschung dieser Nachricht sowie
>> eventueller Anhänge. Jegliche unberechtigte Verwendung oder Verbreitung
>> dieser Informationen ist streng verboten.
>>
>> This message is intended only for the named recipient and may contain
>> confidential or privileged information. As the confidentiality of email
>> communication cannot be guaranteed, we do not accept any responsibility for
>> the confidentiality and the intactness of this message. If you have
>> received it in error, please advise the sender by return e-mail and delete
>> this message and any attachments. Any unauthorised use or dissemination of
>> this information is strictly prohibited.
>>
>> Diese Nachricht ist ausschliesslich für den Adressaten bestimmt und
>> beinhaltet unter Umständen vertrauliche Mitteilungen. Da die
>> Vertraulichkeit von e-Mail-Nachrichten nicht gewährleistet werden kann,
>> übernehmen wir keine Haftung für die Gewährung der Vertraulichkeit und
>> Unversehrtheit dieser Mitteilung. Bei irrtümlicher Zustellung bitten wir
>> Sie um Benachrichtigung per e-Mail und um Löschung dieser Nachricht sowie
>> eventueller Anhänge. Jegliche unberechtigte Verwendung oder Verbreitung
>> dieser Informationen ist streng verboten.
>>
>> This message is intended only for the named recipient and may contain
>> confidential or privileged information. As the confidentiality of email
>> communication cannot be guaranteed, we do not accept any responsibility for
>> the confidentiality and the intactness of this message. If you have
>> received it in error, please advise the sender by return e-mail and delete
>> this message and any attachments. Any unauthorised use or dissemination of
>> this information is strictly prohibited.
>>
>

Reply via email to