[ https://issues.apache.org/jira/browse/KAFKA-540?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Jun Rao updated KAFKA-540: -------------------------- Attachment: kafka-540_v2.patch Attach patch v2. 1. changed to KafkaStorageException. 2. I think we should call halt on real IO exception. This will make sure that kafkaserver.shutdown is not called and will force us to do log recovery on restart, which is the right thing to do. Not sure what scala exit does, but it's being deprecated anyway. Also, removed calling exit in handleOffsetRequest. It feels weird to shut down a server on a read only operation. > log.append() should halt on IOException > --------------------------------------- > > Key: KAFKA-540 > URL: https://issues.apache.org/jira/browse/KAFKA-540 > Project: Kafka > Issue Type: Bug > Components: core > Affects Versions: 0.8 > Reporter: Jun Rao > Assignee: Jun Rao > Priority: Blocker > Labels: bugs > Fix For: 0.8 > > Attachments: kafka-540.patch, kafka-540_v2.patch > > Original Estimate: 24h > Remaining Estimate: 24h > > See the following entry in the broker log in a system test run. We > interrupted the ReplicaFetcherThread during shutdown. However, log.append > halts the system when we hit the interrupted exception. The fix is not to > halt the system in log.append and just pass on the exception. The caller can > decide what to do. > [2012-10-03 15:08:53,124] FATAL [Kafka Log on Broker 2], Halting due to > unrecoverable I/O error while handling producer request (kafka.log.Log) > java.nio.channels.ClosedByInterruptException > at > java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:184) > at sun.nio.ch.FileChannelImpl.write(FileChannelImpl.java:203) > at > kafka.message.ByteBufferMessageSet.writeTo(ByteBufferMessageSet.scala:128) > at kafka.log.FileMessageSet.append(FileMessageSet.scala:155) > at kafka.log.LogSegment.append(LogSegment.scala:60) > at kafka.log.Log.liftedTree1$1(Log.scala:282) > at kafka.log.Log.append(Log.scala:270) > at > kafka.server.ReplicaFetcherThread.processPartitionData(ReplicaFetcherThread.scala:42) > at > kafka.server.AbstractFetcherThread$$anonfun$doWork$5.apply(AbstractFetcherThread.scala:105) > at > kafka.server.AbstractFetcherThread$$anonfun$doWork$5.apply(AbstractFetcherThread.scala:98) > at scala.collection.immutable.Map$Map1.foreach(Map.scala:105) > at > kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:98) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:50) > ( -- 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