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

Steve Miller edited comment on KAFKA-1554 at 5/12/15 3:09 PM:
--------------------------------------------------------------

Hm, that seems to have rolled out of the logs at this point -- sorry.  I don't 
think that specific error was an issue: I think that anything that makes Kakfa 
exit after it's created the new segment index but before it's compacted it is 
an issue.

I think that if you do the following, you can reproduce the issue:

* create a new, empty topic, and don't publish anything to it.  For each 
partition in that topic, you should end up with the usual topicname-partition# 
directory somewhere (e.g., 'junk-2'), which will have an empty log file and an 
index file of size 10485760.
* copy the whole junk-2 directory somewhere safe (e.g., /var/tmp/junk-2).
* stop kafka
* cp -pr /var/tmp/junk-2 /whatever/kafka/log/junk-2.old
* start kafka

When you do that, Kafka will fail to start properly, with an error like:

{code}
[2015-05-12 15:00:55,661] FATAL [Kafka Server 1], Fatal error during 
KafkaServer startup. Prepare to shutdown (kafka.server.KafkaServer)
java.lang.NumberFormatException: For input string: "6.old"
        at 
java.lang.NumberFormatException.forInputString(NumberFormatException.java:65)
        at java.lang.Integer.parseInt(Integer.java:492)
        at java.lang.Integer.parseInt(Integer.java:527)
        at 
scala.collection.immutable.StringLike$class.toInt(StringLike.scala:247)
        at scala.collection.immutable.StringOps.toInt(StringOps.scala:30)
        at kafka.log.Log$.parseTopicPartitionName(Log.scala:833)
        at 
kafka.log.LogManager$$anonfun$loadLogs$2$$anonfun$3$$anonfun$apply$7$$anonfun$apply$1.apply$mcV$sp(LogManager.scala:138)
        at kafka.utils.Utils$$anon$1.run(Utils.scala:54)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
        at java.util.concurrent.FutureTask.run(FutureTask.java:262)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:745)
{code}

and you'll also now have many index files of size 10485760.

Note that after a clean shutdown, you don't have any index files of that size, 
so I think what's happening here is something like:

* the code runs that replaces the working index file in any log directory with 
the 10485760-sized files, which Kafka uses (presumably) for the current log 
segment
* something blows up
* the usual code that would run at clean execution to replace the current, 
10485760-sized files with their compacted versions doesn't run because of the 
blowup

leaving these files on disk.  And then at startup Kafka panics because it 
considers them corrupted.

I was able to use these steps on a spare broker we had and I could get this to 
repeat at will. :)


was (Author: stevemil00):
Hm, that seems to have rolled out of the logs at this point -- sorry.  I don't 
think that specific error was an issue: I think that anything that makes Kakfa 
exit after it's created the new segment index but before it's compacted it is 
an issue.

I think that if you do the following, you can reproduce the issue:

* create a new, empty topic, and don't publish anything to it.  For each 
partition in that topic, you should end up with the usual topicname-partition# 
directory somewhere (e.g., 'junk-2', which will have an empty log file and an 
index file of size 10485760.
* copy the whole junk-2 directory somewhere safe (e.g., /var/tmp/junk-2).
* stop kafka
* cp -pr /var/tmp/junk-2 /whatever/kafka/log/junk-2.old
* start kafka

When you do that, Kafka will fail to start properly, with an error like:

{code}
[2015-05-12 15:00:55,661] FATAL [Kafka Server 1], Fatal error during 
KafkaServer startup. Prepare to shutdown (kafka.server.KafkaServer)
java.lang.NumberFormatException: For input string: "6.old"
        at 
java.lang.NumberFormatException.forInputString(NumberFormatException.java:65)
        at java.lang.Integer.parseInt(Integer.java:492)
        at java.lang.Integer.parseInt(Integer.java:527)
        at 
scala.collection.immutable.StringLike$class.toInt(StringLike.scala:247)
        at scala.collection.immutable.StringOps.toInt(StringOps.scala:30)
        at kafka.log.Log$.parseTopicPartitionName(Log.scala:833)
        at 
kafka.log.LogManager$$anonfun$loadLogs$2$$anonfun$3$$anonfun$apply$7$$anonfun$apply$1.apply$mcV$sp(LogManager.scala:138)
        at kafka.utils.Utils$$anon$1.run(Utils.scala:54)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
        at java.util.concurrent.FutureTask.run(FutureTask.java:262)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:745)
{code}

and you'll also now have many index files of size 10485760.

Note that after a clean shutdown, you don't have any index files of that size, 
so I think what's happening here is something like:

* the code runs that replaces the working index file in any log directory with 
the 10485760-sized files, which Kafka uses (presumably) for the current log 
segment
* something blows up
* the usual code that would run at clean execution to replace the current, 
10485760-sized files with their compacted versions doesn't run because of the 
blowup

leaving these files on disk.  And then at startup Kafka panics because it 
considers them corrupted.

I was able to use these steps on a spare broker we had and I could get this to 
repeat at will. :)

> 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