kafka-jira mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jason Gustafson (JIRA)" <j...@apache.org>
Subject [jira] [Resolved] (KAFKA-5747) Broker crashes on startup when trying to parse empty snapshot files
Date Mon, 21 Aug 2017 17:24:00 GMT

     [ https://issues.apache.org/jira/browse/KAFKA-5747?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

Jason Gustafson resolved KAFKA-5747.
------------------------------------
    Resolution: Fixed

Issue resolved by pull request 3688
[https://github.com/apache/kafka/pull/3688]

> Broker crashes on startup when trying to parse empty snapshot files
> -------------------------------------------------------------------
>
>                 Key: KAFKA-5747
>                 URL: https://issues.apache.org/jira/browse/KAFKA-5747
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.11.0.0
>            Reporter: Lukasz Mierzwa
>            Assignee: Jason Gustafson
>             Fix For: 0.11.0.1
>
>
> A broker server crash can sometime result in empty snapshot files on disk (depending
on FS, barrier setting etc), when Kafka tries to parse such files it crashes, gets restarted
and crashes again, this happens until you remove empty snapshot files with:
> {noformat}
> find /logs/dir -name \*.snapshot -size 0 -delete
> {noformat}
> Log:
> {noformat}
> Aug 15 22:52:11 localhost kafka[23681]: INFO Recovering unflushed segment 0 in log __consumer_offsets-16.
(kafka.log.Log)
> Aug 15 22:52:11 localhost kafka[23681]: INFO Loading producer state from offset 1207
for partition __consumer_offsets-16 with message format version 0 (kafka.log.Log)
> Aug 15 22:52:11 localhost kafka[23681]: INFO Completed load of log __consumer_offsets-16
with 1 log segments, log start offset 0 and log end offset 1207 in 15 ms (kafka.log.Log)
> Aug 15 22:52:11 localhost kafka[23681]: WARN Found a corrupted index file due to requirement
failed: Corrupt index found, index file (/disk/data/kafka/mycluster/mytopic-64/00000000300519800823.index)
has non-zero size but the last offset is 300519800823 which is no larger than the base offset
300519800823.}. deleting /disk/data/kafka/mycluster/mytopic-64/00000000300519800823.timeindex,
/disk/data/kafka/mycluster/mytopic-64/00000000300519800823.index, and /disk/data/kafka/mycluster/mytopic-64/00000000300519800823.txnindex
and rebuilding index... (kafka.log.Log)
> Aug 15 22:52:11 localhost kafka[23681]: INFO Loading producer state from snapshot file
00000000300519800823.snapshot for partition mytopic-64 (kafka.log.ProducerStateManager)
> Aug 15 22:52:11 localhost kafka[23681]: ERROR There was an error in one of the threads
during logs loading: org.apache.kafka.common.protocol.types.SchemaException: Error reading
field 'version': java.nio.BufferUnderflowException (kafka.log.LogManager)
> Aug 15 22:52:11 localhost kafka[23681]: FATAL [Kafka Server 10139], Fatal error during
KafkaServer startup. Prepare to shutdown (kafka.server.KafkaServer)
> Aug 15 22:52:11 localhost kafka[23681]: org.apache.kafka.common.protocol.types.SchemaException:
Error reading field 'version': java.nio.BufferUnderflowException
> Aug 15 22:52:11 localhost kafka[23681]:         at org.apache.kafka.common.protocol.types.Schema.read(Schema.java:76)
> Aug 15 22:52:11 localhost kafka[23681]:         at kafka.log.ProducerStateManager$.readSnapshot(ProducerStateManager.scala:289)
> Aug 15 22:52:11 localhost kafka[23681]:         at kafka.log.ProducerStateManager.loadFromSnapshot(ProducerStateManager.scala:440)
> Aug 15 22:52:11 localhost kafka[23681]:         at kafka.log.ProducerStateManager.truncateAndReload(ProducerStateManager.scala:499)
> Aug 15 22:52:11 localhost kafka[23681]:         at kafka.log.Log.recoverSegment(Log.scala:327)
> Aug 15 22:52:11 localhost kafka[23681]:         at kafka.log.Log.$anonfun$loadSegmentFiles$3(Log.scala:314)
> Aug 15 22:52:11 localhost kafka[23681]:         at scala.collection.TraversableLike$WithFilter.$anonfun$foreach$1(TraversableLike.scala:789)
> Aug 15 22:52:11 localhost kafka[23681]:         at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:32)
> Aug 15 22:52:11 localhost kafka[23681]:         at scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:29)
> Aug 15 22:52:11 localhost kafka[23681]:         at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:191)
> Aug 15 22:52:11 localhost kafka[23681]:         at scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:788)
> Aug 15 22:52:11 localhost kafka[23681]:         at kafka.log.Log.loadSegmentFiles(Log.scala:272)
> Aug 15 22:52:11 localhost kafka[23681]:         at kafka.log.Log.loadSegments(Log.scala:376)
> Aug 15 22:52:11 localhost kafka[23681]:         at kafka.log.Log.<init>(Log.scala:179)
> Aug 15 22:52:11 localhost kafka[23681]:         at kafka.log.Log$.apply(Log.scala:1581)
> Aug 15 22:52:11 localhost kafka[23681]:         at kafka.log.LogManager.$anonfun$loadLogs$12(LogManager.scala:172)
> Aug 15 22:52:11 localhost kafka[23681]:         at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:57)
> Aug 15 22:52:11 localhost kafka[23681]:         at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> Aug 15 22:52:11 localhost kafka[23681]:         at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> Aug 15 22:52:11 localhost kafka[23681]:         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> Aug 15 22:52:11 localhost kafka[23681]:         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> Aug 15 22:52:11 localhost kafka[23681]:         at java.lang.Thread.run(Thread.java:748)
> Aug 15 22:52:11 localhost kafka[23681]: INFO [Kafka Server 10139], shutting down (kafka.server.KafkaServer)
> Aug 15 22:52:11 localhost kafka[23681]: INFO Terminate ZkClient event thread. (org.I0Itec.zkclient.ZkEventThread)
> Aug 15 22:52:11 localhost kafka[23681]: INFO Session: 0x15de80e0a37004c closed (org.apache.zookeeper.ZooKeeper)
> Aug 15 22:52:11 localhost kafka[23681]: INFO EventThread shut down for session: 0x15de80e0a37004c
(org.apache.zookeeper.ClientCnxn)
> Aug 15 22:52:11 localhost kafka[23681]: INFO [Kafka Server 10139], shut down completed
(kafka.server.KafkaServer)
> Aug 15 22:52:11 localhost kafka[23681]: FATAL Exiting Kafka. (kafka.server.KafkaServerStartable)
> Aug 15 22:52:11 localhost kafka[23681]: INFO [Kafka Server 10139], shutting down (kafka.server.KafkaServer)
> Aug 15 22:52:12 localhost systemd[1]: kafka.service: Main process exited, code=exited,
status=1/FAILURE
> Aug 15 22:52:12 localhost systemd[1]: kafka.service: Unit entered failed state.
> Aug 15 22:52:12 localhost systemd[1]: kafka.service: Failed with result 'exit-code'.
> {noformat}
> Happens with 0.10 & 0.11



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Mime
View raw message