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