[
https://issues.apache.org/jira/browse/FLINK-7978?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Aljoscha Krettek closed FLINK-7978.
-----------------------------------
Resolution: Fixed
Fixed on release-1.4 in
3cbf467ebdf639df4d7d4da78b7bc2929aa4b5d9
460e27aeb5e246aff0f8137448441c315123608c
Fixed on master in
2949dc43b238b7f689571f007fd3346de3b89ed9
d3aa3f0729e42d48820b3786f463eadc409ece4f
> Kafka011 exactly-once Producer sporadically fails to commit under high
> parallelism
> ----------------------------------------------------------------------------------
>
> Key: FLINK-7978
> URL: https://issues.apache.org/jira/browse/FLINK-7978
> Project: Flink
> Issue Type: Bug
> Components: Kafka Connector
> Affects Versions: 1.4.0
> Reporter: Gary Yao
> Assignee: Piotr Nowojski
> Priority: Blocker
> Fix For: 1.4.0
>
>
> The Kafka011 exactly-once producer sporadically fails to commit/confirm the
> first checkpoint. The behavior seems to be easier reproduced under high job
> parallelism.
> *Logs/Stacktrace*
> {noformat}
> 10:24:35,347 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator
> - Completed checkpoint 1 (191029 bytes in 1435 ms).
> 10:24:35,349 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 2/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-12], transactionStartTime=1509787474588} from
> checkpoint 1
> 10:24:35,349 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 1/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-8], transactionStartTime=1509787474393} from
> checkpoint 1
> 10:24:35,349 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 0/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-4], transactionStartTime=1509787474448} from
> checkpoint 1
> 10:24:35,350 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 6/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-34], transactionStartTime=1509787474742} from
> checkpoint 1
> 10:24:35,350 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 4/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-23], transactionStartTime=1509787474777} from
> checkpoint 1
> 10:24:35,353 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 10/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-52], transactionStartTime=1509787474930} from
> checkpoint 1
> 10:24:35,350 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 7/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-35], transactionStartTime=1509787474659} from
> checkpoint 1
> 10:24:35,349 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 5/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-25], transactionStartTime=1509787474652} from
> checkpoint 1
> 10:24:35,361 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 18/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-92], transactionStartTime=1509787475043} from
> checkpoint 1
> 10:24:35,349 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 3/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-15], transactionStartTime=1509787474590} from
> checkpoint 1
> 10:24:35,361 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 13/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-67], transactionStartTime=1509787474962} from
> checkpoint 1
> 10:24:35,359 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 20/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-104], transactionStartTime=1509787474654} from
> checkpoint 1
> 10:24:35,359 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 19/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-96], transactionStartTime=1509787474655} from
> checkpoint 1
> 10:24:35,358 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 17/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-89], transactionStartTime=1509787474389} from
> checkpoint 1
> 10:24:35,358 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 16/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-81], transactionStartTime=1509787474983} from
> checkpoint 1
> 10:24:35,358 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 15/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-78], transactionStartTime=1509787474782} from
> checkpoint 1
> 10:24:35,382 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 28/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-144], transactionStartTime=1509787475011} from
> checkpoint 1
> 10:24:35,358 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 14/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-74], transactionStartTime=1509787474590} from
> checkpoint 1
> 10:24:35,356 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 11/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-56], transactionStartTime=1509787474451} from
> checkpoint 1
> 10:24:35,353 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 12/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-63], transactionStartTime=1509787475127} from
> checkpoint 1
> 10:24:35,351 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 8/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-41], transactionStartTime=1509787474807} from
> checkpoint 1
> 10:24:35,351 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 9/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-45], transactionStartTime=1509787474659} from
> checkpoint 1
> 10:24:35,368 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 31/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-159], transactionStartTime=1509787474781} from
> checkpoint 1
> 10:24:35,368 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 29/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-149], transactionStartTime=1509787474330} from
> checkpoint 1
> 10:24:35,368 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 27/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-139], transactionStartTime=1509787474438} from
> checkpoint 1
> 10:24:35,367 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 26/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-134], transactionStartTime=1509787474437} from
> checkpoint 1
> 10:24:35,367 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 23/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-119], transactionStartTime=1509787474637} from
> checkpoint 1
> 10:24:35,367 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 22/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-114], transactionStartTime=1509787475143} from
> checkpoint 1
> 10:24:35,367 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 21/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-109], transactionStartTime=1509787474237} from
> checkpoint 1
> 10:24:35,388 INFO org.apache.flink.runtime.taskmanager.Task
> - Attempting to fail task externally Sink: kafka-sink-1509787467330
> (30/32) (46216003624d2cffd6cd7b424b66a0b9).
> 10:24:35,388 INFO org.apache.flink.runtime.taskmanager.Task
> - Sink: kafka-sink-1509787467330 (30/32)
> (46216003624d2cffd6cd7b424b66a0b9) switched from RUNNING to FAILED.
> java.lang.RuntimeException: Error while confirming checkpoint
> at org.apache.flink.runtime.taskmanager.Task$3.run(Task.java:1254)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: 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.
> 10:24:35,394 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 25/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-129], transactionStartTime=1509787474563} from
> checkpoint 1
> 10:24:35,402 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 30/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-154], transactionStartTime=1509787474659} from
> checkpoint 1
> 10:24:35,394 INFO
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction -
> FlinkKafkaProducer011 24/32 - checkpoint 1 complete, committing transaction
> TransactionHolder{handle=KafkaTransactionState [transactionalId=Sink:
> kafka-sink-1509787467330-124], transactionStartTime=1509787474637} from
> checkpoint 1
> 10:24:35,414 INFO org.apache.flink.runtime.taskmanager.Task
> - Triggering cancellation of task code Sink: kafka-sink-1509787467330
> (30/32) (46216003624d2cffd6cd7b424b66a0b9).
> 10:24:35,421 INFO
> org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer -
> Flushing new partitions
> 10:24:35,422 INFO org.apache.kafka.clients.producer.KafkaProducer
> - Closing the Kafka producer with timeoutMillis = 9223372036854775807 ms.
> 10:24:35,423 INFO org.apache.flink.runtime.taskmanager.Task
> - Freeing task resources for Sink: kafka-sink-1509787467330 (30/32)
> (46216003624d2cffd6cd7b424b66a0b9).
> 10:24:35,441 INFO org.apache.flink.runtime.taskmanager.Task
> - Ensuring all FileSystem streams are closed for task Sink:
> kafka-sink-1509787467330 (30/32) (46216003624d2cffd6cd7b424b66a0b9) [FAILED]
> 10:24:35,443 INFO org.apache.flink.runtime.taskmanager.TaskManager
> - Un-registering task and sending final execution state FAILED to
> JobManager for task Sink: kafka-sink-1509787467330
> (46216003624d2cffd6cd7b424b66a0b9)
> 10:24:35,444 INFO org.apache.flink.runtime.executiongraph.ExecutionGraph
> - Sink: kafka-sink-1509787467330 (30/32)
> (46216003624d2cffd6cd7b424b66a0b9) switched from RUNNING to FAILED.
> java.lang.RuntimeException: Error while confirming checkpoint
> at org.apache.flink.runtime.taskmanager.Task$3.run(Task.java:1254)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: 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.
> 10:24:35,444 INFO org.apache.flink.runtime.executiongraph.ExecutionGraph
> - Job Flink Streaming Java API Skeleton (58a76026ade37503dadebb28ffaafd1b)
> switched from state RUNNING to FAILING.
> java.lang.RuntimeException: Error while confirming checkpoint
> at org.apache.flink.runtime.taskmanager.Task$3.run(Task.java:1254)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: 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.
> 10:24:35,444 INFO org.apache.flink.runtime.client.JobSubmissionClientActor
> - 11/04/2017 10:24:35 Sink: kafka-sink-1509787467330(30/32)
> switched to FAILED
> java.lang.RuntimeException: Error while confirming checkpoint
> at org.apache.flink.runtime.taskmanager.Task$3.run(Task.java:1254)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: 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.
> 11/04/2017 10:24:35 Sink: kafka-sink-1509787467330(30/32) switched to
> FAILED
> java.lang.RuntimeException: Error while confirming checkpoint
> at org.apache.flink.runtime.taskmanager.Task$3.run(Task.java:1254)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: 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.
> 10:24:35,445 INFO org.apache.flink.runtime.client.JobSubmissionClientActor
> - 11/04/2017 10:24:35 Job execution switched to status FAILING.
> java.lang.RuntimeException: Error while confirming checkpoint
> at org.apache.flink.runtime.taskmanager.Task$3.run(Task.java:1254)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: 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.
> 11/04/2017 10:24:35 Job execution switched to status FAILING.
> java.lang.RuntimeException: Error while confirming checkpoint
> at org.apache.flink.runtime.taskmanager.Task$3.run(Task.java:1254)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: 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.
> 10:24:35,451 INFO org.apache.flink.runtime.executiongraph.ExecutionGraph
> - Source: Collection Source (1/1) (b46e03bc27ee6458ef2c282ef0f3c426)
> switched from RUNNING to CANCELING.
> {noformat}
> *How to reproduce*
> Run example job and restart the job until the first commit/checkpoint
> confirmation fails:
> https://github.com/GJL/flink-kafka011-producer-test/blob/master/src/main/java/com/garyyao/StreamingJob.java
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)