[ 
https://issues.apache.org/jira/browse/KAFKA-19397?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Lucas Brutschy updated KAFKA-19397:
-----------------------------------
    Description: 
Sometimes, current trunk throws the following NPE:

 
{code:java}
[2025-05-29 04:06:05,855] ERROR [kafka-producer-network-thread | 
i-07bbab180f6062ba3-StreamThread-3-producer] [Producer 
clientId=i-07bbab180f6062ba3-StreamThread-3-producer] Uncaught error in request 
completion: (org.apache.kafka.clients.NetworkClient)
java.lang.NullPointerException: Cannot read field "topicPartition" because 
"batch" is null
at 
org.apache.kafka.clients.producer.internals.TransactionManager.handleCompletedBatch(TransactionManager.java:748)
at 
org.apache.kafka.clients.producer.internals.Sender.completeBatch(Sender.java:736)
at 
org.apache.kafka.clients.producer.internals.Sender.completeBatch(Sender.java:710)
at 
org.apache.kafka.clients.producer.internals.Sender.lambda$handleProduceResponse$2(Sender.java:613)
at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
at 
org.apache.kafka.clients.producer.internals.Sender.lambda$handleProduceResponse$3(Sender.java:597)
at java.base/java.lang.Iterable.forEach(Iterable.java:75)
at 
org.apache.kafka.clients.producer.internals.Sender.handleProduceResponse(Sender.java:597)
at 
org.apache.kafka.clients.producer.internals.Sender.lambda$sendProduceRequest$9(Sender.java:895)
at org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:154)
at 
org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:669)
at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:661)
at org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:340)
at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:242)
at java.base/java.lang.Thread.run(Thread.java:840)
 
 
{code}
 

This was discovered in a long-running test, so we do not have a directly 
reproducible test case. However, DEBUG logs are included below, which show the 
sequence of METADATA and PRODUCE requests / responses that seem to cause this.

 

Likely cause is the change here: [https://github.com/apache/kafka/pull/15968]

  was:
Sometimes, current trunk throws the following NPE:



[2025-05-29 04:06:05,855] ERROR [kafka-producer-network-thread | 
i-07bbab180f6062ba3-StreamThread-3-producer] [Producer 
clientId=i-07bbab180f6062ba3-StreamThread-3-producer] Uncaught error in request 
completion: (org.apache.kafka.clients.NetworkClient)
java.lang.NullPointerException: Cannot read field "topicPartition" because 
"batch" is null
        at 
org.apache.kafka.clients.producer.internals.TransactionManager.handleCompletedBatch(TransactionManager.java:748)
        at 
org.apache.kafka.clients.producer.internals.Sender.completeBatch(Sender.java:736)
        at 
org.apache.kafka.clients.producer.internals.Sender.completeBatch(Sender.java:710)
        at 
org.apache.kafka.clients.producer.internals.Sender.lambda$handleProduceResponse$2(Sender.java:613)
        at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
        at 
org.apache.kafka.clients.producer.internals.Sender.lambda$handleProduceResponse$3(Sender.java:597)
        at java.base/java.lang.Iterable.forEach(Iterable.java:75)
        at 
org.apache.kafka.clients.producer.internals.Sender.handleProduceResponse(Sender.java:597)
        at 
org.apache.kafka.clients.producer.internals.Sender.lambda$sendProduceRequest$9(Sender.java:895)
        at 
org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:154)
        at 
org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:669)
        at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:661)
        at 
org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:340)
        at 
org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:242)
        at java.base/java.lang.Thread.run(Thread.java:840)
 

 

This was discovered in a long-running test, so we do not have a directly 
reproducible test case. However, DEBUG logs are included below, which show the 
sequence of METADATA and PRODUCE requests / responses that seem to cause this.

 

Likely cause is the change here: https://github.com/apache/kafka/pull/15968


> TransactionManager.handleCompletedBatch throws NPE
> --------------------------------------------------
>
>                 Key: KAFKA-19397
>                 URL: https://issues.apache.org/jira/browse/KAFKA-19397
>             Project: Kafka
>          Issue Type: Bug
>            Reporter: Lucas Brutschy
>            Assignee: Omnia Ibrahim
>            Priority: Major
>
> Sometimes, current trunk throws the following NPE:
>  
> {code:java}
> [2025-05-29 04:06:05,855] ERROR [kafka-producer-network-thread | 
> i-07bbab180f6062ba3-StreamThread-3-producer] [Producer 
> clientId=i-07bbab180f6062ba3-StreamThread-3-producer] Uncaught error in 
> request completion: (org.apache.kafka.clients.NetworkClient)
> java.lang.NullPointerException: Cannot read field "topicPartition" because 
> "batch" is null
> at 
> org.apache.kafka.clients.producer.internals.TransactionManager.handleCompletedBatch(TransactionManager.java:748)
> at 
> org.apache.kafka.clients.producer.internals.Sender.completeBatch(Sender.java:736)
> at 
> org.apache.kafka.clients.producer.internals.Sender.completeBatch(Sender.java:710)
> at 
> org.apache.kafka.clients.producer.internals.Sender.lambda$handleProduceResponse$2(Sender.java:613)
> at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
> at 
> org.apache.kafka.clients.producer.internals.Sender.lambda$handleProduceResponse$3(Sender.java:597)
> at java.base/java.lang.Iterable.forEach(Iterable.java:75)
> at 
> org.apache.kafka.clients.producer.internals.Sender.handleProduceResponse(Sender.java:597)
> at 
> org.apache.kafka.clients.producer.internals.Sender.lambda$sendProduceRequest$9(Sender.java:895)
> at org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:154)
> at 
> org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:669)
> at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:661)
> at org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:340)
> at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:242)
> at java.base/java.lang.Thread.run(Thread.java:840)
>  
>  
> {code}
>  
> This was discovered in a long-running test, so we do not have a directly 
> reproducible test case. However, DEBUG logs are included below, which show 
> the sequence of METADATA and PRODUCE requests / responses that seem to cause 
> this.
>  
> Likely cause is the change here: [https://github.com/apache/kafka/pull/15968]



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to