You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Apurva Mehta (JIRA)" <ji...@apache.org> on 2017/05/30 21:18:04 UTC
[jira] [Commented] (KAFKA-5339) Transactions system test with hard
broker bounces fails sporadically
[ https://issues.apache.org/jira/browse/KAFKA-5339?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16030147#comment-16030147 ]
Apurva Mehta commented on KAFKA-5339:
-------------------------------------
Another puzzling one:
{noformat}
21:02:37,450] ERROR Uncaught error in kafka producer I/O thread: (org.apache.kafka.clients.producer.internals.Sender)
java.lang.NullPointerException
at org.apache.kafka.clients.producer.internals.Sender.maybeSendTransactionalRequest(Sender.java:305)
at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:193)
at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:154)
at java.lang.Thread.run(Thread.java:748)
[2017-05-30 21:03:00,433] INFO Closing the Kafka producer with timeoutMillis = 9223372036854775807 ms. (org.apache.kafka.clients.producer.KafkaProducer)
{noformat}
This corresponds to this line in the Sender at the time the test was run:
{code:java}
if (nextRequestHandler.isEndTxn() && transactionManager.isCompletingTransaction() && accumulator.hasUnflushedBatches()) {
...
}
{code}
It should be impossible to hit an NPE there, because we already check for NPE's for `nextRequestHandler` and `transactionManager` before entering that line.
> Transactions system test with hard broker bounces fails sporadically
> --------------------------------------------------------------------
>
> Key: KAFKA-5339
> URL: https://issues.apache.org/jira/browse/KAFKA-5339
> Project: Kafka
> Issue Type: Sub-task
> Components: clients, core, producer
> Reporter: Apurva Mehta
> Assignee: Apurva Mehta
> Priority: Blocker
> Labels: exactly-once
> Fix For: 0.11.0.0
>
>
> The transactions hard bounce test occasionally fails because the transactional message copy just seems to hang. In one of the client logs, I noticed:
> {noformat}
> [2017-05-27 20:36:12,596] WARN Got error produce response with correlation id 124 on topic-partition output-topic-0, retrying (2147483646 attempts left). Error: NOT_LEADER_FOR_PARTITION (org.apache.kafka.clients.producer.internals.Sender)
> [2017-05-27 20:36:15,386] ERROR Uncaught error in kafka producer I/O thread: (org.apache.kafka.clients.producer.internals.Sender)
> java.lang.NullPointerException
> at org.apache.kafka.clients.producer.internals.TransactionManager$1.compare(TransactionManager.java:146)
> at org.apache.kafka.clients.producer.internals.TransactionManager$1.compare(TransactionManager.java:143)
> at java.util.PriorityQueue.siftDownUsingComparator(PriorityQueue.java:721)
> at java.util.PriorityQueue.siftDown(PriorityQueue.java:687)
> at java.util.PriorityQueue.poll(PriorityQueue.java:595)
> at org.apache.kafka.clients.producer.internals.TransactionManager.nextRequestHandler(TransactionManager.java:351)
> at org.apache.kafka.clients.producer.internals.Sender.maybeSendTransactionalRequest(Sender.java:303)
> at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:193)
> at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:154)
> at java.lang.Thread.run(Thread.java:748)
> [2017-05-27 20:36:52,007] INFO Closing the Kafka producer with timeoutMillis = 9223372036854775807 ms. (org.apache.kafka.clients.producer.KafkaProducer)
> [2017-05-27 20:36:52,036] INFO Marking the coordinator knode02:9092 (id: 2147483645 rack: null) dead for group transactions-test-consumer-group (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> root@7dcd60017519:/opt/kafka-dev/results/latest/TransactionsTest/test_transactions/failure_mode=hard_bounce.bounce_target=brokers/1#
> {noformat}
> This suggests that the client has gotten to a bad state which is why it stops processing messages, causing the tests to fail.
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)