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

Neha Narkhede updated KAFKA-669:
--------------------------------

    Comment: was deleted

(was: This issue was caused by KAFKA-654 on the leader broker 3. The root cause 
was leftover on disk state from previous runs of KAFKA-654. Basically, the 
broker was leader for a few partitions that didn't have a 0.log file, but had 
an non-empty and full 0.index file. Due to this, when it receives a produce 
request, it tries to roll the log and fails. This uncovered a potential 
recovery bug. Ideally, since the brokers were hard killed, the recovery process 
should detect this issue and fix itself)
    
> Irrecoverable error on leader while rolling to a new segment
> ------------------------------------------------------------
>
>                 Key: KAFKA-669
>                 URL: https://issues.apache.org/jira/browse/KAFKA-669
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.8
>            Reporter: Neha Narkhede
>            Priority: Blocker
>              Labels: bugs
>             Fix For: 0.8
>
>
> I setup a test cluster of 5 brokers and setup migration tool to send data for 
> ~1000 partitions to this cluster. I see  a bunch of 
> FailedToSendMessageExceptions on the producer. After tracing some of these 
> failed requests, I found that the broker successfully completed the requests, 
> yet sends a ProducerResponseStatus(-1, -1) to the producer. The impact is the 
> producer marks the messages for those partitions as failed, retries and 
> eventually gives up. This seems like a bug on the broker. 
> The producer's log says -
> [2012-12-11 23:00:22,179] DEBUG Producer sending messages with correlation id 
> 60626 for topics Map([SpamReportEvent,0] -> 
> ByteBufferMessageSet(MessageAndOffset(Message(magic = 2, attributes = 0, crc 
> = 2412809952, key = null,
>  payload = java.nio.HeapByteBuffer[pos=0 lim=159 cap=159]),0), )) to broker 3 
> on 172.20.72.46:9092 (kafka.producer.async.DefaultEventHandler)
> [2012-12-11 23:00:22,179] INFO Connected to 172.20.72.46:9092 for producing 
> (kafka.producer.SyncProducer)
> [2012-12-11 23:00:22,185] DEBUG Producer sent messages for topics 
> Map([SpamReportEvent,0] -> 
> ByteBufferMessageSet(MessageAndOffset(Message(magic = 2, attributes = 0, crc 
> = 2412809952, key = null, payload = java.nio.HeapByteBuffer[pos=0 lim=159 
> cap=159]),0), )) to broker 3 on 
> (kafka.producer.async.DefaultEventHandler)
> [2012-12-11 23:00:22,185] DEBUG Produce request with correlation id 60626 
> failed due to response ProducerResponse(0,60
> 626,Map([SpamReportEvent,0] -> ProducerResponseStatus(-1,-1))). List of 
> failed topic partitions is [CommGe
> nericMessageSendEvent,1] (kafka.producer.async.DefaultEventHandler)
> [2012-12-11 23:00:22,285] INFO Fetching metadata with correlation id 60627 
> for 1 topic(s) Set(SpamReportEvent) (kafka.client.ClientUtils$)
> [2012-12-11 23:00:22,312] ERROR Failed to send the following requests with 
> correlation id 60627: 
> ArrayBuffer(KeyedMessage(SpamReportEvent,null,Message(magic = 2, attributes = 
> 0, crc = 2412809952, key = null, payload = java.n
> io.HeapByteBuffer[pos=0 lim=159 cap=159]))) 
> (kafka.producer.async.DefaultEventHandler)
> [2012-12-11 23:00:22,314] ERROR Error in handling batch of 200 events 
> (kafka.producer.async.ProducerSendThread)
> kafka.common.FailedToSendMessageException: Failed to send messages after 3 
> tries.
>         at 
> kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:73)
>         at 
> kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
>         at 
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
>         at 
> kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
>         at scala.collection.immutable.Stream.foreach(Stream.scala:254)
>         at 
> kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
>         at 
> kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44)
> Tracing produce request with correlation id 60626 on broker 3 -
> [2012-12-11 23:00:22,180] TRACE Received request with correlation id 60626 
> from client : ProducerRequest(0,60626,,0,3000,Map([SpamReportEvent,0] -> 
> ByteBufferMessageSet(MessageAndOffset(Message(magic = 2, attributes = 0, crc 
> = 2412809952, key = null, payload = java.nio.HeapByteBuffer[pos=0 lim=159 
> cap=159]),0), ))) (kafka.network.RequestChannel$)
> [2012-12-11 23:00:22,180] TRACE Handling ProducerRequest with correlation id 
> 60626 from client : ProducerRequest(0,60626,,0,3000,Map([SpamReportEvent,0] 
> -> ByteBufferMessageSet(MessageAndOffset(Message(magic = 2, attributes = 0, 
> crc = 2412809952, key = null, payload = java.nio.HeapByteBuffer[pos=0 lim=159 
> cap=159]),0), ))) (kafka.request.logger)
> [2012-12-11 23:00:22,184] TRACE Completed request with correlation id 60626 
> and client : ProducerRequest(0,60626,,0,3000,Map([SpamReportEvent,0] -> 
> ByteBufferMessageSet(MessageAndOffset(Message(magic = 2, attributes = 0, crc 
> = 2412809952, key = null, payload = java.nio.HeapByteBuffer[pos=0 lim=159 
> cap=159]),0), ))), totalTime:4, queueTime:0, localTime:4, remoteTime:0, 
> sendTime:0 (kafka.network.RequestChannel$)
> [2012-12-12 02:06:06,081] ERROR [KafkaApi-3] Error processing ProducerRequest 
> on SpamReportEvent:0 (kafka.server.KafkaApis)
> kafka.common.KafkaException: Trying to roll a new log segment for topic 
> partition SpamReportEvent-0 with start offset 0 while it already exists
>         at kafka.log.Log.rollToOffset(Log.scala:465)
>         at kafka.log.Log.roll(Log.scala:443)
>         at kafka.log.Log.maybeRoll(Log.scala:431)
>         at kafka.log.Log.append(Log.scala:258)
>         at 
> kafka.server.KafkaApis$$anonfun$appendToLocalLog$2.apply(KafkaApis.scala:246)
>         at 
> kafka.server.KafkaApis$$anonfun$appendToLocalLog$2.apply(KafkaApis.scala:239)
>         at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:206)
>         at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:206)
>         at scala.collection.immutable.Map$Map1.foreach(Map.scala:105)
>         at 
> scala.collection.TraversableLike$class.map(TraversableLike.scala:206)
>         at scala.collection.immutable.Map$Map1.map(Map.scala:93)
>         at kafka.server.KafkaApis.appendToLocalLog(KafkaApis.scala:239)
>         at kafka.server.KafkaApis.handleProducerRequest(KafkaApis.scala:179)
>         at kafka.server.KafkaApis.handle(KafkaApis.scala:56)
>         at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:41)
>         at java.lang.Thread.run(Thread.java:619)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to