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. >