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

Hannu Valtonen commented on KAFKA-4686:
---------------------------------------

Hit this again in a test enviroment of ours: 
Kafka server log snippet: (this repeats ad infinitum)

[2017-02-20 12:19:16,931] ERROR [KafkaApi-0] Error when handling request Name: 
FetchRequest; Version: 0; CorrelationId: 4; ClientId: kafka-python; ReplicaId: 
-1; MaxWait: 9999 ms; MinBytes: 1 bytes; MaxBytes:2147483647 bytes; 
RequestInfo: 
([6c999dfb-231c-4e8d-9c5b-19010c550a87-msgs,0],PartitionFetchInfo(0,16777216)) 
(kafka.server.KafkaApis)
kafka.common.KafkaException: Message payload is null: Message(magic = 0, 
attributes = 2, crc = 1331266146, key = null, payload = null)
at 
kafka.message.ByteBufferMessageSet$$anon$1.<init>(ByteBufferMessageSet.scala:90)
at 
kafka.message.ByteBufferMessageSet$.deepIterator(ByteBufferMessageSet.scala:85)
at 
kafka.log.FileMessageSet$$anonfun$toMessageFormat$1.apply(FileMessageSet.scala:276)
at 
kafka.log.FileMessageSet$$anonfun$toMessageFormat$1.apply(FileMessageSet.scala:269)
at scala.collection.Iterator$class.foreach(Iterator.scala:893)
at kafka.utils.IteratorTemplate.foreach(IteratorTemplate.scala:30)
at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
at kafka.message.MessageSet.foreach(MessageSet.scala:71)
at kafka.log.FileMessageSet.toMessageFormat(FileMessageSet.scala:269)
at kafka.server.KafkaApis$$anonfun$29.apply(KafkaApis.scala:490)
at kafka.server.KafkaApis$$anonfun$29.apply(KafkaApis.scala:477)
at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at 
scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
at scala.collection.AbstractTraversable.map(Traversable.scala:104)
at 
kafka.server.KafkaApis.kafka$server$KafkaApis$$sendResponseCallback$3(KafkaApis.scala:477)
at 
kafka.server.KafkaApis$$anonfun$handleFetchRequest$1.apply(KafkaApis.scala:542)
at 
kafka.server.KafkaApis$$anonfun$handleFetchRequest$1.apply(KafkaApis.scala:542)
at kafka.server.ReplicaManager.fetchMessages(ReplicaManager.scala:497)
at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:534)
at kafka.server.KafkaApis.handle(KafkaApis.scala:79)
at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:60)
at java.lang.Thread.run(Thread.java:745)

Then the dumping you requested of the topic: 

/opt/kafka/bin/kafka-run-class.sh kafka.tools.DumpLogSegments --files 
/srv/kafka/6c999dfb-231c-4e8d-9c5b-19010c550a87-msgs-0/00000000000000000000.log 
--deep-iteration
Dumping 
/srv/kafka/6c999dfb-231c-4e8d-9c5b-19010c550a87-msgs-0/00000000000000000000.log
Starting offset: 0
offset: 0 position: 0 CreateTime: -1 isvalid: true payloadsize: 9438 magic: 1 
compresscodec: NoCompressionCodec crc: 3724646734
offset: 1 position: 7227 CreateTime: -1 isvalid: true payloadsize: 9571 magic: 
1 compresscodec: NoCompressionCodec crc: 942316008
offset: 2 position: 14504 CreateTime: -1 isvalid: true payloadsize: 9571 magic: 
1 compresscodec: NoCompressionCodec crc: 2440026867
offset: 3 position: 21781 CreateTime: -1 isvalid: true payloadsize: 9571 magic: 
1 compresscodec: NoCompressionCodec crc: 2902450727
Exception in thread "main" kafka.common.KafkaException: Message payload is 
null: Message(magic = 0, attributes = 2, crc = 1331266146, key = null, payload 
= null)
        at 
kafka.message.ByteBufferMessageSet$$anon$1.<init>(ByteBufferMessageSet.scala:90)
        at 
kafka.message.ByteBufferMessageSet$.deepIterator(ByteBufferMessageSet.scala:85)
        at 
kafka.tools.DumpLogSegments$.kafka$tools$DumpLogSegments$$getIterator(DumpLogSegments.scala:352)
        at 
kafka.tools.DumpLogSegments$$anonfun$kafka$tools$DumpLogSegments$$dumpLog$1.apply(DumpLogSegments.scala:311)
        at 
kafka.tools.DumpLogSegments$$anonfun$kafka$tools$DumpLogSegments$$dumpLog$1.apply(DumpLogSegments.scala:310)
        at scala.collection.Iterator$class.foreach(Iterator.scala:893)
        at kafka.utils.IteratorTemplate.foreach(IteratorTemplate.scala:30)
        at 
kafka.tools.DumpLogSegments$.kafka$tools$DumpLogSegments$$dumpLog(DumpLogSegments.scala:310)
        at 
kafka.tools.DumpLogSegments$$anonfun$main$1.apply(DumpLogSegments.scala:96)
        at 
kafka.tools.DumpLogSegments$$anonfun$main$1.apply(DumpLogSegments.scala:92)
        at 
scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
        at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
        at kafka.tools.DumpLogSegments$.main(DumpLogSegments.scala:92)
        at kafka.tools.DumpLogSegments.main(DumpLogSegments.scala)

kafka-topics.sh --describe for the topic:

Topic:6c999dfb-231c-4e8d-9c5b-19010c550a87-msgs PartitionCount:1        
ReplicationFactor:2     Configs:
        Topic: 6c999dfb-231c-4e8d-9c5b-19010c550a87-msgs        Partition: 0    
Leader: 0       Replicas: 0,1   Isr: 0,1

In this case the messages were produced with: 
(https://github.com/mumrah/kafka-python/)

Hope this helps.

> 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