Hi Arvid, I had 5.5.2 bundled into my application jar. I was able to get the https://github.com/apache/flink/pull/16693 working by ensuring that kafka-clients==2.4.1 was used just now. Thanks!!
On Wed, Aug 4, 2021 at 1:04 PM Arvid Heise <ar...@apache.org> wrote: > 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. >>> >>