[ 
https://issues.apache.org/jira/browse/KAFKA-10645?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17222393#comment-17222393
 ] 

Matthias J. Sax edited comment on KAFKA-10645 at 10/30/20, 1:31 AM:
--------------------------------------------------------------------

[~showuon] While my PR adds an additional check, it is still unclear how a 
`null` header was added in the first place?
{quote}by passing the headers Array/Iterable to the constructor.
{quote}
Yes, but if the input records does not have any headers, the array should have 
zero entries? It's still unclear to me, where a `null` entry comes from?

[~filmac79] I have no idea atm what could cause the 
`ArrayIndexOutOfBoundsException` atm.  In L67 we call `headers.add()` that 
should add the header to the end of the list. It seem that the `ArrayList.add` 
does something funny? There should not be any synchronization issue, as the 
punctuate() and process() are called by the same thread?

Well, there was some other issue with regard to concurrency though: 
https://issues.apache.org/jira/browse/KAFKA-9584 – maybe this is the root cause 
but you just hit a different exception?

 


was (Author: mjsax):
[~showuon] While my PR adds an additional check, it is still unclear how a 
`null` header was added in the first place?
{quote}by passing the headers Array/Iterable to the constructor.
{quote}
Yes, but if the input records does not have any headers, the array should have 
zero entries? It's still unclear to me, there a `null` entry comes from?

[~filmac79] I have no idea atm what could cause the 
`ArrayIndexOutOfBoundsException` atm.  In L67 we call `headers.add()` that 
should add the header to the end of the list. It seem that the `ArrayList.add` 
does something funny? There should not be any synchronization issue, as the 
punctuate() and process() are called by the same thread?

Well, there was some other issue with regard to concurrency though: 
https://issues.apache.org/jira/browse/KAFKA-9584 – maybe this is the root cause 
but you just hit a different exception?

 

> Forwarding a record from a punctuator sometimes it results in a 
> NullPointerException
> ------------------------------------------------------------------------------------
>
>                 Key: KAFKA-10645
>                 URL: https://issues.apache.org/jira/browse/KAFKA-10645
>             Project: Kafka
>          Issue Type: Bug
>          Components: clients
>    Affects Versions: 2.5.0
>            Reporter: Filippo Machi
>            Assignee: Matthias J. Sax
>            Priority: Major
>
> Hello,
>  I am working on a java kafka stream application (v. 2.5.0) running on a 
> kubernetes cluster.
> It´s a springboot application running with java 8.
> With the last upgrade to version 2.5.0 I started to see into the logs some 
> NullPointerException that are happening when forwarding a record from a 
> punctuator. 
>  This is the stacktrace of the exception
> {code:java}
> Caused by: org.apache.kafka.streams.errors.StreamsException: task [2_2] Abort 
> sending since an error caught with a previous record (timestamp 
> 1603721062667) to topic reply-reminder-push-sender due to 
> java.lang.NullPointerException\tat 
> org.apache.kafka.streams.processor.internals.RecordCollectorImpl.send(RecordCollectorImpl.java:240)\tat
>  
> org.apache.kafka.streams.processor.internals.RecordCollectorImpl.send(RecordCollectorImpl.java:111)\tat
>  
> org.apache.kafka.streams.processor.internals.SinkNode.process(SinkNode.java:89)\tat
>  
> org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forward(ProcessorContextImpl.java:201)\tat
>  
> org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forward(ProcessorContextImpl.java:180)\tat
>  
> org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forward(ProcessorContextImpl.java:133)\t...
>  24 common frames omittedCaused by: java.lang.NullPointerException: null\tat 
> org.apache.kafka.common.record.DefaultRecord.sizeOf(DefaultRecord.java:613)\tat
>  
> org.apache.kafka.common.record.DefaultRecord.recordSizeUpperBound(DefaultRecord.java:633)\tat
>  
> org.apache.kafka.common.record.DefaultRecordBatch.estimateBatchSizeUpperBound(DefaultRecordBatch.java:534)\tat
>  
> org.apache.kafka.common.record.AbstractRecords.estimateSizeInBytesUpperBound(AbstractRecords.java:135)\tat
>  
> org.apache.kafka.common.record.AbstractRecords.estimateSizeInBytesUpperBound(AbstractRecords.java:125)\tat
>  
> org.apache.kafka.clients.producer.KafkaProducer.doSend(KafkaProducer.java:914)\tat
>  
> org.apache.kafka.clients.producer.KafkaProducer.send(KafkaProducer.java:862)\tat
>  
> org.apache.kafka.streams.processor.internals.RecordCollectorImpl.send(RecordCollectorImpl.java:181)\t...
>  29 common frames omitted
> {code}
> Checking the code, it looks like it happens calculating the size of the 
> record. There is one header that is null but I don´t think I can control 
> those headers right?
> Thanks a lot



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to