[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-20 Thread Hannu Valtonen (JIRA)

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

Hannu Valtonen commented on KAFKA-4686:
---

Oh yeah, thought to mention that just before the payload msg thing occurs, 
there were a few SSL disconnect messages in the Kafka server log. (It would be 
great BTW if it gave some info on which client/from which IP disconnected)

WARN Failed to send SSL Close message  
(org.apache.kafka.common.network.SslTransportLayer)
java.io.IOException: Broken pipe
at sun.nio.ch.FileDispatcherImpl.write0(Native Method)
at sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47)
at sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93)
at sun.nio.ch.IOUtil.write(IOUtil.java:65)
at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:471)
at 
org.apache.kafka.common.network.SslTransportLayer.flush(SslTransportLayer.java:195)
at 
org.apache.kafka.common.network.SslTransportLayer.close(SslTransportLayer.java:163)
at org.apache.kafka.common.utils.Utils.closeAll(Utils.java:690)
at org.apache.kafka.common.network.KafkaChannel.close(KafkaChannel.java:47)
at org.apache.kafka.common.network.Selector.close(Selector.java:487)
at org.apache.kafka.common.network.Selector.pollSelectionKeys(Selector.java:368)
at org.apache.kafka.common.network.Selector.poll(Selector.java:291)
at kafka.network.Processor.poll(SocketServer.scala:476)
at kafka.network.Processor.run(SocketServer.scala:416)
at java.lang.Thread.run(Thread.java:745)

> 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.(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.(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

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-20 Thread Hannu Valtonen (JIRA)

[ 
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:  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.(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/.log 
--deep-iteration
Dumping 
/srv/kafka/6c999dfb-231c-4e8d-9c5b-19010c550a87-msgs-0/.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.(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:6c9

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-16 Thread Jason Gustafson (JIRA)

[ 
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.(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.(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

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-16 Thread Hannu Valtonen (JIRA)

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

Hannu Valtonen commented on KAFKA-4686:
---

Log compaction was not used in our case. Our solution was just to delete the 
whole topic (which BTW unusually required a restart of one of the Kafka nodes 
to go through) since the data in it could be recreated at will.

> 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.(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.(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.(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)
> 

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-16 Thread Jason Gustafson (JIRA)

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

Jason Gustafson commented on KAFKA-4686:


[~Ormod] Thanks. Do you recall if the topic experiencing this problem was using 
compaction or not? How were you able to resolve the issue?

> 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.(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.(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.(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.s

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-16 Thread Hannu Valtonen (JIRA)

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

Hannu Valtonen commented on KAFKA-4686:
---

We've periodically seen the same on Kafka 0.10.1.1 with the kafka-python 
client. The messages in the topics in question have been compressed with snappy.

> 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.(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.(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.(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(Trav

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Rodrigo Queiroz Saramago (JIRA)

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

Rodrigo Queiroz Saramago commented on KAFKA-4686:
-

{noformat:title=broker001}
[root@zupme-qa-1a-kafka001 kafka]# bin/kafka-run-class.sh 
kafka.tools.DumpLogSegments --files 
/data/kafka_vol02/zupme-gateway-12/0145.log --deep-iteration
Dumping /data/kafka_vol02/zupme-gateway-12/0145.log
Starting offset: 145
offset: 145 position: 0 CreateTime: -1 isvalid: true payloadsize: 5150 magic: 1 
compresscodec: NoCompressionCodec crc: 1460190106
{noformat}

{noformat:title=broker002}
[root@zupme-qa-1a-kafka002 kafka]# bin/kafka-run-class.sh 
kafka.tools.DumpLogSegments --files 
/data/kafka_vol01/zupme-gateway-12/0145.log --deep-iteration
Dumping /data/kafka_vol01/zupme-gateway-12/0145.log
Starting offset: 145
offset: 145 position: 0 CreateTime: -1 isvalid: true payloadsize: 5150 magic: 1 
compresscodec: NoCompressionCodec crc: 1460190106
{noformat}

{noformat:title=broker003}
[root@zupme-qa-1a-kafka003 kafka]# bin/kafka-run-class.sh 
kafka.tools.DumpLogSegments --files 
/data/kafka_vol01/zupme-gateway-12/.log --deep-iteration
Dumping /data/kafka_vol01/zupme-gateway-12/.log
Starting offset: 0
{noformat}


> 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.(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.(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 K

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Jason Gustafson (JIRA)

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

Jason Gustafson commented on KAFKA-4686:


[~rodrigo.saramago] Thanks for the updates! One more request: can you use the 
same commands on the log files only and include "--deep-iteration"? 

> 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.(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.(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.(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(Traversab

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Rodrigo Queiroz Saramago (JIRA)

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

Rodrigo Queiroz Saramago commented on KAFKA-4686:
-

Sure, the results are below:

{noformat:title=broker001}
[root@zupme-qa-1a-kafka001 kafka]# bin/kafka-run-class.sh 
kafka.tools.DumpLogSegments --files 
/data/kafka_vol02/zupme-gateway-12/0145.log 
Dumping /data/kafka_vol02/zupme-gateway-12/0145.log
Starting offset: 145
offset: 145 position: 0 CreateTime: -1 isvalid: true payloadsize: 1541 magic: 1 
compresscodec: GZIPCompressionCodec crc: 227440706
{noformat}

{noformat:title=broker002}
[root@zupme-qa-1a-kafka002 kafka]# bin/kafka-run-class.sh 
kafka.tools.DumpLogSegments --files 
/data/kafka_vol01/zupme-gateway-12/0145.log 
Dumping /data/kafka_vol01/zupme-gateway-12/0145.log
Starting offset: 145
offset: 145 position: 0 CreateTime: -1 isvalid: true payloadsize: 1541 magic: 1 
compresscodec: GZIPCompressionCodec crc: 227440706
{noformat}

{noformat:title=broker003}
[root@zupme-qa-1a-kafka003 kafka]# bin/kafka-run-class.sh 
kafka.tools.DumpLogSegments --files 
/data/kafka_vol01/zupme-gateway-12/.log 
Dumping /data/kafka_vol01/zupme-gateway-12/.log
Starting offset: 0
{noformat}


> 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.(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.(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 KafkaServerStarta

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Jason Gustafson (JIRA)

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

Jason Gustafson commented on KAFKA-4686:


Can you try using DumpLogSegments on the log files directly. For example: 
{{bin/kafka-run-class.sh kafka.tools.DumpLogSegments --files 
/data/kafka_vol01/zupme-gateway-12/.log}}.

> 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.(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.(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.(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.col

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Rodrigo Queiroz Saramago (JIRA)

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

Rodrigo Queiroz Saramago commented on KAFKA-4686:
-

Running the Dumplog without --deep-iteraction gives the same result:

{noformat}
[root@zupme-qa-1a-kafka003 kafka]# bin/kafka-run-class.sh 
kafka.tools.DumpLogSegments --files /data/kafka_vol01/zupme-gateway-12/*
Dumping /data/kafka_vol01/zupme-gateway-12/.index
Exception in thread "main" java.util.NoSuchElementException
at kafka.utils.IteratorTemplate.next(IteratorTemplate.scala:37)
at scala.collection.IterableLike$class.head(IterableLike.scala:107)
at kafka.message.MessageSet.head(MessageSet.scala:71)
at 
kafka.tools.DumpLogSegments$$anonfun$kafka$tools$DumpLogSegments$$dumpIndex$1.apply$mcVI$sp(DumpLogSegments.scala:148)
at scala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)
at 
kafka.tools.DumpLogSegments$.kafka$tools$DumpLogSegments$$dumpIndex(DumpLogSegments.scala:145)
at 
kafka.tools.DumpLogSegments$$anonfun$main$1.apply(DumpLogSegments.scala:99)
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)
{noformat}

{noformat}
[root@zupme-qa-1a-kafka001 kafka]# bin/kafka-run-class.sh 
kafka.tools.DumpLogSegment>
Dumping /data/kafka_vol02/zupme-gateway-12/0145.index
offset: 145 position: 0
{noformat}

{noformat}
[root@zupme-qa-1a-kafka002 kafka]# bin/kafka-run-class.sh 
kafka.tools.DumpLogSegments --files /data/kafka_vol01/zupme-gateway-12/*
Dumping /data/kafka_vol01/zupme-gateway-12/0145.index
[2017-02-07 16:38:08,419] DEBUG Loaded index file 
/data/kafka_vol01/zupme-gateway-12/0145.index with maxEntries = 
1310720, maxIndexSize = -1, entries = 1310720, lastOffset = 145, file position 
= 10485760 (kafka.log.OffsetIndex)
offset: 145 position: 0
{noformat}

> 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.(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.(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.ut

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Rodrigo Queiroz Saramago (JIRA)

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

Rodrigo Queiroz Saramago commented on KAFKA-4686:
-

No, there are no more files in kafka data directory:

[root@zupme-qa-1a-kafka003~]$ ls -l /data/kafka_vol01/zupme-gateway-12
total 0
-rw-r--r-- 1 kafka kafka 10485760 Jan 24 16:43 .index
-rw-r--r-- 1 kafka kafka0 Jan 24 16:43 .log
-rw-r--r-- 1 kafka kafka 10485756 Jan 24 16:43 .timeindex

[root@zupme-qa-1a-kafka001 ~]$ ls -l /data/kafka_vol02/zupme-gateway-12
total 4
-rw-r--r-- 1 kafka kafka 10485760 Feb  2 15:50 0145.index
-rw-r--r-- 1 kafka kafka 1575 Feb  2 15:50 0145.log
-rw-r--r-- 1 kafka kafka 10485756 Feb  2 15:50 0145.timeindex

[root@zupme-qa-1a-kafka002 ~]$ ls -l /data/kafka_vol01/zupme-gateway-12 
 
total 4
-rw-r--r-- 1 kafka kafka 10485760 Feb  7 07:03 0145.index
-rw-r--r-- 1 kafka kafka 1575 Feb  7 07:03 0145.log
-rw-r--r-- 1 kafka kafka 10485756 Feb  7 07:03 0145.timeindex

For clients we are using as producers:
nodejs: [https://github:oleksiyk/kafka] version 3.1.0
golang:  [https://github.com/confluentinc/confluent-kafka-go] version 0.9.2

Consumers are using logstash with 
[https://github.com/logstash-plugins/logstash-input-kafka] version 6.2.4. But I 
also get this problem testing with kafka native console-consumer in version 
0.10.1.1

.

> 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.(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.(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 S

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Jason Gustafson (JIRA)

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

Jason Gustafson commented on KAFKA-4686:


One additional question: which clients and which versions do you use?

> 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.(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.(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.(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 

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Jason Gustafson (JIRA)

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

Jason Gustafson commented on KAFKA-4686:


[~rodrigo.saramago] Are there no files with the ".log" extension under 
"/data/kafka_vol01/zupme-gateway-12/"? Can you try the command without 
including "--deep-iteration"?

> 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.(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.(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.(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$Wi

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-02-07 Thread Rodrigo Queiroz Saramago (JIRA)

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

Rodrigo Queiroz Saramago commented on KAFKA-4686:
-

Hi [~hachikuji], no, we are using the default broker behavior, that's set 
compression.type to producer 
(https://kafka.apache.org/documentation.html#brokerconfigs). So we only set the 
compression on the producers (gzip in that case).

Before upgrade we are using kafka version 0.10.1.0. No,I can't able to start 
the broker again after this failure, the broker always fail after that.

I try to run the command that you mention, but the command only return some 
index on others brokers, like 001 or 002, the failed broker (003) gives-me the 
error below:

[root@zupme-qa-1a-kafka003 kafka]# bin/kafka-run-class.sh 
kafka.tools.DumpLogSegments --files /data/kafka_vol01/zupme-gateway-12/* 
--deep-iteration
Dumping /data/kafka_vol01/zupme-gateway-12/.index
Exception in thread "main" java.util.NoSuchElementException
at kafka.utils.IteratorTemplate.next(IteratorTemplate.scala:37)
at scala.collection.IterableLike$class.head(IterableLike.scala:107)
at kafka.message.MessageSet.head(MessageSet.scala:71)
at 
kafka.tools.DumpLogSegments$$anonfun$kafka$tools$DumpLogSegments$$dumpIndex$1.apply$mcVI$sp(DumpLogSegments.scala:148)
at scala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)
at 
kafka.tools.DumpLogSegments$.kafka$tools$DumpLogSegments$$dumpIndex(DumpLogSegments.scala:145)
at 
kafka.tools.DumpLogSegments$$anonfun$main$1.apply(DumpLogSegments.scala:99)
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)

On broker 002 the following message is displayed:
[root@zupme-qa-1a-kafka002 kafka]# bin/kafka-run-class.sh 
kafka.tools.DumpLogSegments --files /data/kafka_vol01/zupme-gateway-12/* 
--deep-iteration  
Dumping /data/kafka_vol01/zupme-gateway-12/0145.index
[2017-02-07 14:13:00,733] DEBUG Loaded index file 
/data/kafka_vol01/zupme-gateway-12/0145.index with maxEntries = 
1310720, maxIndexSize = -1, entries = 1310720, lastOffset = 145, file position 
= 10485760 (kafka.log.OffsetIndex)
offset: 145 position: 0

And on broker 001:
[root@zupme-qa-1a-kafka001 kafka]#  bin/kafka-run-class.sh 
kafka.tools.DumpLogSegments --files /data/kafka_vol02/zupme-gateway-12/* 
--deep-iteration 
Dumping /data/kafka_vol02/zupme-gateway-12/0145.index
offset: 145 position: 0

What this logs means?


> 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.(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.collect

[jira] [Commented] (KAFKA-4686) Null Message payload is shutting down broker

2017-01-31 Thread Jason Gustafson (JIRA)

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

Jason Gustafson commented on KAFKA-4686:


[~rodrigo.saramago] Thanks for the updates. I spotted this error which actually 
tells us which partition has the corrupt message:
{code}
[2017-01-30 07:03:34,149] ERROR [ReplicaFetcherThread-0-1003], Error due to  
(kafka.server.ReplicaFetcherThread)
kafka.common.KafkaException: error processing data for partition 
[zupme-gateway,12] offset 106
at 
kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1$$anonfun$apply$2.apply(AbstractFetcherThread.scala:170)
at 
kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1$$anonfun$apply$2.apply(AbstractFetcherThread.scala:141)
at scala.Option.foreach(Option.scala:257)
at 
kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1.apply(AbstractFetcherThread.scala:141)
at 
kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1.apply(AbstractFetcherThread.scala:138)
at 
scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
at 
kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2.apply$mcV$sp(AbstractFetcherThread.scala:138)
at 
kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2.apply(AbstractFetcherThread.scala:138)
at 
kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2.apply(AbstractFetcherThread.scala:138)
at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:234)
at 
kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:136)
at 
kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:103)
at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)
Caused by: kafka.common.KafkaException: Message payload is null: Message(magic 
= 0, attributes = 1, crc = 3187236729, key = null, payload = null)
at 
kafka.message.ByteBufferMessageSet$$anon$1.(ByteBufferMessageSet.scala:90)
at 
kafka.message.ByteBufferMessageSet$.deepIterator(ByteBufferMessageSet.scala:85)
at 
kafka.message.ByteBufferMessageSet$$anon$2.makeNextOuter(ByteBufferMessageSet.scala:370)
at 
kafka.message.ByteBufferMessageSet$$anon$2.makeNext(ByteBufferMessageSet.scala:383)
at 
kafka.message.ByteBufferMessageSet$$anon$2.makeNext(ByteBufferMessageSet.scala:338)
at 
kafka.utils.IteratorTemplate.maybeComputeNext(IteratorTemplate.scala:64)
at kafka.utils.IteratorTemplate.hasNext(IteratorTemplate.scala:56)
at kafka.message.MessageSet.toString(MessageSet.scala:105)
at java.lang.String.valueOf(String.java:2994)
at 
scala.collection.mutable.StringBuilder.append(StringBuilder.scala:200)
at kafka.log.Log.append(Log.scala:395)
at 
kafka.server.ReplicaFetcherThread.processPartitionData(ReplicaFetcherThread.scala:130)
at 
kafka.server.ReplicaFetcherThread.processPartitionData(ReplicaFetcherThread.scala:42)
at 
kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$2$$anonfun$apply$mcV$sp$1$$anonfun$apply$2.apply(AbstractFetcherThread.scala:153)
... 13 more
{code}
This may not be the only partition which is corrupt, but at least we can 
analyze it a bit more. A couple questions:

1. Do you know if compaction has been enabled on the topic "zupme-gateway"? 
2. What version of Kafka were you on before upgrading to 0.10.1.1? Were you 
ever able to successfully start the brokers running 0.10.1.1?

Also, could use the DumpLogSegments utility (bin/kafka-run-class 
kafka.tools.DumpLogSegments) on all of the segments for partition 12 of 
"zupme-gateway"? Judging by the offset from that message, it should be the 
first segment which is corrupt. If you run with the "--deep-iteration" option, 
you will probably see the same "Message payload is null" error, but we should 
be able to still do the shallow iteration if you leave that option out. Please 
upload the results if possible.

> 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