[ 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/00000000000000000000.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/00000000000000000145.index [2017-02-07 14:13:00,733] DEBUG Loaded index file /data/kafka_vol01/zupme-gateway-12/00000000000000000145.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/00000000000000000145.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.<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)