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

Jason Gustafson commented on KAFKA-4686:
----------------------------------------

I'm at a loss how this can happen. I tried to create a compressed message set 
with a null value and send it to the broker, but it was properly rejected. The 
only other paths that messages can be written to the log is by the log cleaner 
and the group metadata manager (excluding replication which depends on a 
successful initial write to the log), but both seem to have been ruled out and 
the validation appears correct in any case. 

Interestingly, I did find that it is possible to write an empty compressed 
message set to the log. In this case, the value of the wrapper message would 
not be null (as is the case here), but it wouldn't contain any messages. I will 
create a separate JIRA for this. This could be somehow related to this problem, 
but I haven't thought of how yet.

[~Ormod] If you hit this again, can you try to the DumpLogSegments utility 
against the log segment with the invalid data? It might also help to see the 
topic and client configs as well to know the client version.

> Null Message payload is shutting down broker
> --------------------------------------------
>
>                 Key: KAFKA-4686
>                 URL: https://issues.apache.org/jira/browse/KAFKA-4686
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.10.1.1
>         Environment: Amazon Linux AMI release 2016.03 kernel 
> 4.4.19-29.55.amzn1.x86_64
>            Reporter: Rodrigo Queiroz Saramago
>             Fix For: 0.10.3.0
>
>         Attachments: KAFKA-4686-NullMessagePayloadError.tar.xz, 
> kafkaServer.out
>
>
> Hello, I have a test environment with 3 brokers and 1 zookeeper nodes, in 
> which clients connect using two-way ssl authentication. I use kafka version 
> 0.10.1.1, the system works as expected for a while, but if the broker goes 
> down and then is restarted, something got corrupted and is not possible start 
> broker again, it always fails with the same error. What this error mean? What 
> can I do in this case? Is this the expected behavior?
> [2017-01-23 07:03:28,927] ERROR There was an error in one of the threads 
> during logs loading: kafka.common.KafkaException: Message payload is null: 
> Message(magic = 0, attributes = 1, crc = 4122289508, key = null, payload = 
> null) (kafka.log.LogManager)
> [2017-01-23 07:03:28,929] FATAL Fatal error during KafkaServer startup. 
> Prepare to shutdown (kafka.server.KafkaServer)
> kafka.common.KafkaException: Message payload is null: Message(magic = 0, 
> attributes = 1, crc = 4122289508, key = null, payload = null)
>     at 
> kafka.message.ByteBufferMessageSet$$anon$1.<init>(ByteBufferMessageSet.scala:90)
>     at 
> kafka.message.ByteBufferMessageSet$.deepIterator(ByteBufferMessageSet.scala:85)
>     at kafka.message.MessageAndOffset.firstOffset(MessageAndOffset.scala:33)
>     at kafka.log.LogSegment.recover(LogSegment.scala:223)
>     at kafka.log.Log$$anonfun$loadSegments$4.apply(Log.scala:218)
>     at kafka.log.Log$$anonfun$loadSegments$4.apply(Log.scala:179)
>     at 
> scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:733)
>     at 
> scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
>     at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
>     at 
> scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:732)
>     at kafka.log.Log.loadSegments(Log.scala:179)
>     at kafka.log.Log.<init>(Log.scala:108)
>     at 
> kafka.log.LogManager$$anonfun$loadLogs$2$$anonfun$3$$anonfun$apply$10$$anonfun$apply$1.apply$mcV$sp(LogManager.scala:151)
>     at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:58)
>     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:1142)
>     at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>     at java.lang.Thread.run(Thread.java:745)
> [2017-01-23 07:03:28,946] INFO shutting down (kafka.server.KafkaServer)
> [2017-01-23 07:03:28,949] INFO Terminate ZkClient event thread. 
> (org.I0Itec.zkclient.ZkEventThread)
> [2017-01-23 07:03:28,954] INFO EventThread shut down for session: 
> 0x159bd458ae70008 (org.apache.zookeeper.ClientCnxn)
> [2017-01-23 07:03:28,954] INFO Session: 0x159bd458ae70008 closed 
> (org.apache.zookeeper.ZooKeeper)
> [2017-01-23 07:03:28,957] INFO shut down completed (kafka.server.KafkaServer)
> [2017-01-23 07:03:28,959] FATAL Fatal error during KafkaServerStartable 
> startup. Prepare to shutdown (kafka.server.KafkaServerStartable)
> kafka.common.KafkaException: Message payload is null: Message(magic = 0, 
> attributes = 1, crc = 4122289508, key = null, payload = null)
>     at 
> kafka.message.ByteBufferMessageSet$$anon$1.<init>(ByteBufferMessageSet.scala:90)
>     at 
> kafka.message.ByteBufferMessageSet$.deepIterator(ByteBufferMessageSet.scala:85)
>     at kafka.message.MessageAndOffset.firstOffset(MessageAndOffset.scala:33)
>     at kafka.log.LogSegment.recover(LogSegment.scala:223)
>     at kafka.log.Log$$anonfun$loadSegments$4.apply(Log.scala:218)
>     at kafka.log.Log$$anonfun$loadSegments$4.apply(Log.scala:179)
>     at 
> scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:733)
>     at 
> scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
>     at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
>     at 
> scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:732)
>     at kafka.log.Log.loadSegments(Log.scala:179)
>     at kafka.log.Log.<init>(Log.scala:108)
>     at 
> kafka.log.LogManager$$anonfun$loadLogs$2$$anonfun$3$$anonfun$apply$10$$anonfun$apply$1.apply$mcV$sp(LogManager.scala:151)
>     at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:58)
>     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:1142)
>     at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>     at java.lang.Thread.run(Thread.java:745)
> [2017-01-23 07:03:28,961] INFO shutting down (kafka.server.KafkaServer)



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to