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

Steve Miller commented on KAFKA-1554:
-------------------------------------

The foo-<n>.old thing was just a way I knew I could reproduce the failure.

I'm suggesting that if Kafka ever restarts and has an issue on its way up, and 
it exits before it's fully up and running, it'll leave a ton of corrupted index 
files as a result.  That is, anything strange that happens in someone's 
environment, that causes a restart that isn't happy, will ultimately cause 
index corruption of this sort.

Maybe the original poster didn't end up in this state because of an unexpected 
restart of the sort I'm describing, but the current behavior seems worth 
fixing.  I'm betting there was an unexpected startup issue in that person's 
case but I admit I might be wrong. :)

Did that help?

> Corrupt index found on clean startup
> ------------------------------------
>
>                 Key: KAFKA-1554
>                 URL: https://issues.apache.org/jira/browse/KAFKA-1554
>             Project: Kafka
>          Issue Type: Bug
>          Components: core
>    Affects Versions: 0.8.1
>         Environment: ubuntu 12.04, oracle jdk 1.7
>            Reporter: Alexis Midon
>            Assignee: Mayuresh Gharat
>            Priority: Critical
>             Fix For: 0.9.0
>
>         Attachments: KAFKA-1554.patch
>
>
> On a clean start up, corrupted index files are found.
> After investigations, it appears that some pre-allocated index files are not 
> "compacted" correctly and the end of the file is full of zeroes.
> As a result, on start up, the last relative offset is zero which yields an 
> offset equal to the base offset.
> The workaround is to delete all index files of size 10MB (the size of the 
> pre-allocated files), and restart. Index files will be re-created.
> {code}
> find $your_data_directory -size 10485760c -name *.index #-delete
> {code}
> This is issue might be related/similar to 
> https://issues.apache.org/jira/browse/KAFKA-1112
> {code}
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,696 
> INFO main kafka.server.KafkaServer.info - [Kafka Server 847605514], starting
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,698 
> INFO main kafka.server.KafkaServer.info - [Kafka Server 847605514], 
> Connecting to zookeeper on 
> zk-main0.XXX:2181,zk-main1.XXX:2181,zk-main2.XXXX:2181/production/kafka/main
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,708 
> INFO 
> ZkClient-EventThread-14-zk-main0.XXX.com:2181,zk-main1.XXX.com:2181,zk-main2.XXX.com:2181,zk-main3.XXX.com:2181,zk-main4.XXX.com:2181/production/kafka/main
>  org.I0Itec.zkclient.ZkEventThread.run - Starting ZkClient event thread.
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,714 
> INFO main org.apache.zookeeper.ZooKeeper.logEnv - Client 
> environment:zookeeper.version=3.3.3-1203054, built on 11/17/2011 05:47 GMT
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,714 
> INFO main org.apache.zookeeper.ZooKeeper.logEnv - Client 
> environment:host.name=i-6b948138.inst.aws.airbnb.com
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,714 
> INFO main org.apache.zookeeper.ZooKeeper.logEnv - Client 
> environment:java.version=1.7.0_55
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,715 
> INFO main org.apache.zookeeper.ZooKeeper.logEnv - Client 
> environment:java.vendor=Oracle Corporation
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,715 
> INFO main org.apache.zookeeper.ZooKeeper.logEnv - Client 
> environment:java.home=/usr/lib/jvm/jre-7-oracle-x64/jre
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,715 
> INFO main org.apache.zookeeper.ZooKeeper.logEnv - Client 
> environment:java.class.path=libs/snappy-java-1.0.5.jar:libs/scala-library-2.10.1.jar:libs/slf4j-api-1.7.2.jar:libs/jopt-simple-3.2.jar:libs/metrics-annotation-2.2.0.jar:libs/log4j-1.2.15.jar:libs/kafka_2.10-0.8.1.jar:libs/zkclient-0.3.jar:libs/zookeeper-3.3.4.jar:libs/metrics-core-2.2.0.jar
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,715 
> INFO main org.apache.zookeeper.ZooKeeper.logEnv - Client 
> environment:java.library.path=/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,716 
> INFO main org.apache.zookeeper.ZooKeeper.logEnv - Client 
> environment:java.io.tmpdir=/tmp
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,716 
> INFO main org.apache.zookeeper.ZooKeeper.logEnv - Client 
> environment:java.compiler=<NA>
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,716 
> INFO main org.apache.zookeeper.ZooKeeper.logEnv - Client 
> environment:os.name=Linux
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,716 
> INFO main org.apache.zookeeper.ZooKeeper.logEnv - Client 
> environment:os.arch=amd64
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,717 
> INFO main org.apache.zookeeper.ZooKeeper.logEnv - Client 
> environment:os.version=3.2.0-61-virtual
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,717 
> INFO main org.apache.zookeeper.ZooKeeper.logEnv - Client 
> environment:user.name=kafka
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,717 
> INFO main org.apache.zookeeper.ZooKeeper.logEnv - Client 
> environment:user.home=/srv/kafka
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,717 
> INFO main org.apache.zookeeper.ZooKeeper.logEnv - Client 
> environment:user.dir=/srv/kafka/kafka_2.10-0.8.1
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,718 
> INFO main org.apache.zookeeper.ZooKeeper.<init> - Initiating client 
> connection, 
> connectString=zk-main0.XXX.com:2181,zk-main1.XXX.com:2181,zk-main2.XXX.com:2181,zk-main3.XXX.com:2181,zk-main4.XXX.com:2181/production/kafka/main
>  sessionTimeout=6000 watcher=org.I0Itec.zkclient.ZkClient@4758af63
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,733 
> INFO main-SendThread() org.apache.zookeeper.ClientCnxn.startConnect - Opening 
> socket connection to server zk-main1.XXX.com/10.12.135.61:2181
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,738 
> INFO main-SendThread(zk-main1.XXX.com:2181) 
> org.apache.zookeeper.ClientCnxn.primeConnection - Socket connection 
> established to zk-main1.XXX.com/10.12.135.61:2181, initiating session
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,745 
> INFO main-SendThread(zk-main1.XXX.com:2181) 
> org.apache.zookeeper.ClientCnxn.readConnectResult - Session establishment 
> complete on server zk-main1.XXX.com/10.12.135.61:2181, sessionid = 
> 0x646838f07761601, negotiated timeout = 6000
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,747 
> INFO main-EventThread org.I0Itec.zkclient.ZkClient.processStateChanged - 
> zookeeper state changed (SyncConnected)
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,961 
> INFO main kafka.log.LogManager.info - Found clean shutdown file. Skipping 
> recovery for all logs in data directory '/mnt/kafka_logs'
> 2014-07-11T00:53:17+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:17,962 
> INFO main kafka.log.LogManager.info - Loading log 'flog-30'
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg 2014-07-11 - 00:53:18,349 
> FATAL main kafka.server.KafkaServerStartable.fatal - Fatal error during 
> KafkaServerStable startup. Prepare to shutdown
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg 
> java.lang.IllegalArgumentException: - requirement failed: Corrupt index 
> found, index file (/mnt/kafka_logs/flog-30/00000000000121158146.index) has 
> non-zero size but the last offset is 121158146 and the base offset is 
> 121158146
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> scala.Predef$.require(Predef.scala:233)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> kafka.log.OffsetIndex.sanityCheck(OffsetIndex.scala:352)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> kafka.log.Log$$anonfun$loadSegments$5.apply(Log.scala:159)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> kafka.log.Log$$anonfun$loadSegments$5.apply(Log.scala:158)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> scala.collection.Iterator$class.foreach(Iterator.scala:727)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> scala.collection.AbstractIterable.foreach(Iterable.scala:54)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> kafka.log.Log.loadSegments(Log.scala:158)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> kafka.log.Log.<init>(Log.scala:64)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> kafka.log.LogManager$$anonfun$loadLogs$1$$anonfun$apply$4.apply(LogManager.scala:118)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> kafka.log.LogManager$$anonfun$loadLogs$1$$anonfun$apply$4.apply(LogManager.scala:113)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:105)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> kafka.log.LogManager$$anonfun$loadLogs$1.apply(LogManager.scala:113)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> kafka.log.LogManager$$anonfun$loadLogs$1.apply(LogManager.scala:105)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> scala.collection.mutable.WrappedArray.foreach(WrappedArray.scala:34)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> kafka.log.LogManager.loadLogs(LogManager.scala:105)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> kafka.log.LogManager.<init>(LogManager.scala:57)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> kafka.server.KafkaServer.createLogManager(KafkaServer.scala:275)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> kafka.server.KafkaServer.startup(KafkaServer.scala:72)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> kafka.server.KafkaServerStartable.startup(KafkaServerStartable.scala:34)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> kafka.Kafka$.main(Kafka.scala:46)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.emerg  -    at 
> kafka.Kafka.main(Kafka.scala)
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:18,351 
> INFO main kafka.server.KafkaServer.info - [Kafka Server 847605514], shutting 
> down
> 2014-07-11T00:53:18+00:00 i-6b948138 local3.info 2014-07-11 - 00:53:18,353 
> INFO 
> ZkClient-EventThread-14-zk-main0.XXX.com:2181,zk-main1.XXX.com:2181,zk-main2.XXX.com:2181,zk-main3.XXX.com:2181,zk-main4.XXX.com:2181/production/kafka/main
>  org.I0Itec.zkclient.ZkEventThread.run - Terminate ZkClient event thread.
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to