You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by jq...@apache.org on 2017/07/22 19:36:02 UTC

[1/5] kafka git commit: KAFKA-4763; Handle disk failure for JBOD (KIP-112)

Repository: kafka
Updated Branches:
  refs/heads/trunk 91b5fc737 -> fc93fb4b6


http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala
index 5d221fe..683b34e 100644
--- a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala
@@ -16,13 +16,13 @@
 */
 package kafka.server
 
+import java.io.File
 import java.util.Properties
 import java.util.concurrent.atomic.AtomicBoolean
 
 import kafka.cluster.{Partition, Replica}
 import kafka.log.Log
-import kafka.server.checkpoints.{LeaderEpochCheckpointFile, LeaderEpochFile}
-import kafka.server.epoch.{LeaderEpochCache, LeaderEpochFileCache}
+import kafka.server.epoch.LeaderEpochCache
 import kafka.utils._
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.metrics.Metrics
@@ -54,9 +54,13 @@ class IsrExpirationTest {
 
   @Before
   def setUp() {
-    replicaManager = new ReplicaManager(configs.head, metrics, time, null, null, null, new AtomicBoolean(false),
-      QuotaFactory.instantiate(configs.head, metrics, time).follower, new BrokerTopicStats,
-      new MetadataCache(configs.head.brokerId))
+    val logManager = EasyMock.createMock(classOf[kafka.log.LogManager])
+    EasyMock.expect(logManager.liveLogDirs).andReturn(Array.empty[File]).anyTimes()
+    EasyMock.replay(logManager)
+
+    replicaManager = new ReplicaManager(configs.head, metrics, time, null, null, logManager, new AtomicBoolean(false),
+      QuotaFactory.instantiate(configs.head, metrics, time).follower, new BrokerTopicStats, new MetadataCache(configs.head.brokerId),
+      new LogDirFailureChannel(configs.head.logDirs.size))
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala
index 3497cc3..20526a1 100755
--- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala
+++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala
@@ -145,10 +145,10 @@ class LeaderElectionTest extends ZooKeeperTestHarness {
       val partitionStates = Map(
         new TopicPartition(topic, partitionId) -> new PartitionState(2, brokerId2, LeaderAndIsr.initialLeaderEpoch,
           Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava, LeaderAndIsr.initialZKVersion,
-          Seq(0, 1).map(Integer.valueOf).asJava)
+          Seq(0, 1).map(Integer.valueOf).asJava, false)
       )
       val requestBuilder = new LeaderAndIsrRequest.Builder(
-          controllerId, staleControllerEpoch, partitionStates.asJava, nodes.toSet.asJava)
+        ApiKeys.LEADER_AND_ISR.latestVersion, controllerId, staleControllerEpoch, partitionStates.asJava, nodes.toSet.asJava)
 
       controllerChannelManager.sendRequest(brokerId2, ApiKeys.LEADER_AND_ISR, requestBuilder,
         staleControllerEpochCallback)

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
index 9383355..e053968 100755
--- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
+++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
@@ -181,7 +181,7 @@ class LogOffsetTest extends ZooKeeperTestHarness {
     AdminUtils.createTopic(zkUtils, topic, 3, 1)
 
     val logManager = server.getLogManager
-    val log = logManager.createLog(new TopicPartition(topic, part), logManager.defaultConfig)
+    val log = logManager.getOrCreateLog(new TopicPartition(topic, part), logManager.defaultConfig)
 
     for (_ <- 0 until 20)
       log.appendAsLeader(TestUtils.singletonRecords(value = Integer.toString(42).getBytes()), leaderEpoch = 0)
@@ -210,7 +210,7 @@ class LogOffsetTest extends ZooKeeperTestHarness {
     AdminUtils.createTopic(zkUtils, topic, 3, 1)
 
     val logManager = server.getLogManager
-    val log = logManager.createLog(new TopicPartition(topic, part), logManager.defaultConfig)
+    val log = logManager.getOrCreateLog(new TopicPartition(topic, part), logManager.defaultConfig)
     for (_ <- 0 until 20)
       log.appendAsLeader(TestUtils.singletonRecords(value = Integer.toString(42).getBytes()), leaderEpoch = 0)
     log.flush()

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala
index d9fe995..0e90121 100644
--- a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala
+++ b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala
@@ -23,7 +23,7 @@ import kafka.common.BrokerEndPointNotAvailableException
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.network.ListenerName
 import org.apache.kafka.common.protocol.{ApiKeys, Errors, SecurityProtocol}
-import org.apache.kafka.common.requests.{PartitionState, UpdateMetadataRequest}
+import org.apache.kafka.common.requests.UpdateMetadataRequest
 import org.apache.kafka.common.requests.UpdateMetadataRequest.{Broker, EndPoint}
 import org.junit.Test
 import org.junit.Assert._
@@ -65,9 +65,9 @@ class MetadataCacheTest {
     }.toSet
 
     val partitionStates = Map(
-      new TopicPartition(topic0, 0) -> new PartitionState(controllerEpoch, 0, 0, asList(0, 1, 3), zkVersion, asList(0, 1, 3)),
-      new TopicPartition(topic0, 1) -> new PartitionState(controllerEpoch, 1, 1, asList(1, 0), zkVersion, asList(1, 2, 0, 4)),
-      new TopicPartition(topic1, 0) -> new PartitionState(controllerEpoch, 2, 2, asList(2, 1), zkVersion, asList(2, 1, 3)))
+      new TopicPartition(topic0, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, 0, 0, asList(0, 1, 3), zkVersion, asList(0, 1, 3), asList()),
+      new TopicPartition(topic0, 1) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, 1, 1, asList(1, 0), zkVersion, asList(1, 2, 0, 4), asList()),
+      new TopicPartition(topic1, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, 2, 2, asList(2, 1), zkVersion, asList(2, 1, 3), asList()))
 
     val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion
     val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch,
@@ -125,7 +125,7 @@ class MetadataCacheTest {
     val leader = 1
     val leaderEpoch = 1
     val partitionStates = Map(
-      new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, asList(0), zkVersion, asList(0)))
+      new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, asList(0), zkVersion, asList(0), asList()))
 
     val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion
     val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch,
@@ -169,7 +169,7 @@ class MetadataCacheTest {
     val isr = asList[Integer](0)
 
     val partitionStates = Map(
-      new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas))
+      new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas, asList()))
 
     val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion
     val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch,
@@ -229,7 +229,7 @@ class MetadataCacheTest {
     val isr = asList[Integer](0, 1)
 
     val partitionStates = Map(
-      new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas))
+      new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas, asList()))
 
     val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion
     val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch,
@@ -282,7 +282,7 @@ class MetadataCacheTest {
     val replicas = asList[Integer](0)
     val isr = asList[Integer](0, 1)
     val partitionStates = Map(
-      new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas))
+      new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas, asList()))
     val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion
     val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, partitionStates.asJava,
       brokers.asJava).build()
@@ -315,7 +315,7 @@ class MetadataCacheTest {
       val replicas = asList[Integer](0)
       val isr = asList[Integer](0, 1)
       val partitionStates = Map(
-        new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas))
+        new TopicPartition(topic, 0) -> new UpdateMetadataRequest.PartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas, asList()))
       val version = ApiKeys.UPDATE_METADATA_KEY.latestVersion
       val updateMetadataRequest = new UpdateMetadataRequest.Builder(version, 2, controllerEpoch, partitionStates.asJava,
         brokers.asJava).build()

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
index b0e81a9..231b180 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
@@ -94,8 +94,8 @@ class ReplicaFetcherThreadTest {
     val result = thread.fetchEpochsFromLeader(Map(t1p0 -> 0, t1p1 -> 0))
 
     val expected = Map(
-      t1p0 -> new EpochEndOffset(Errors.UNKNOWN, UNDEFINED_EPOCH_OFFSET),
-      t1p1 -> new EpochEndOffset(Errors.UNKNOWN, UNDEFINED_EPOCH_OFFSET)
+      t1p0 -> new EpochEndOffset(Errors.UNKNOWN_SERVER_ERROR, UNDEFINED_EPOCH_OFFSET),
+      t1p1 -> new EpochEndOffset(Errors.UNKNOWN_SERVER_ERROR, UNDEFINED_EPOCH_OFFSET)
     )
 
     assertEquals("results from leader epoch request should have undefined offset", expected, result)
@@ -271,7 +271,7 @@ class ReplicaFetcherThreadTest {
     //Define the offsets for the OffsetsForLeaderEpochResponse, these are used for truncation
     val offsetsReply = mutable.Map(
       t1p0 -> new EpochEndOffset(NOT_LEADER_FOR_PARTITION, EpochEndOffset.UNDEFINED_EPOCH_OFFSET),
-      t1p1 -> new EpochEndOffset(UNKNOWN, EpochEndOffset.UNDEFINED_EPOCH_OFFSET)
+      t1p1 -> new EpochEndOffset(UNKNOWN_SERVER_ERROR, EpochEndOffset.UNDEFINED_EPOCH_OFFSET)
     ).asJava
 
     //Create the thread

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
index 2ee08a2..483e708 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
@@ -16,6 +16,7 @@
   */
 package kafka.server
 
+import java.io.File
 import java.util.Properties
 import java.util.concurrent.atomic.AtomicBoolean
 
@@ -178,11 +179,12 @@ class ReplicaManagerQuotasTest {
 
     //Return the same log for each partition as it doesn't matter
     expect(logManager.getLog(anyObject())).andReturn(Some(log)).anyTimes()
+    expect(logManager.liveLogDirs).andReturn(Array.empty[File]).anyTimes()
     replay(logManager)
 
     replicaManager = new ReplicaManager(configs.head, metrics, time, zkUtils, scheduler, logManager,
       new AtomicBoolean(false), QuotaFactory.instantiate(configs.head, metrics, time).follower,
-      new BrokerTopicStats, new MetadataCache(configs.head.brokerId))
+      new BrokerTopicStats, new MetadataCache(configs.head.brokerId), new LogDirFailureChannel(configs.head.logDirs.size))
 
     //create the two replicas
     for ((p, _) <- fetchInfo) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index 5794854..6f2a6a7 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -5,7 +5,7 @@
  * The ASF licenses this file to You under the Apache License, Version 2.0
  * (the "License"); you may not use this file except in compliance with
  * the License.  You may obtain a copy of the License at
- * 
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
@@ -27,7 +27,7 @@ import TestUtils.createBroker
 import kafka.utils.timer.MockTimer
 import org.I0Itec.zkclient.ZkClient
 import org.apache.kafka.common.metrics.Metrics
-import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
 import org.apache.kafka.common.record._
 import org.apache.kafka.common.requests.{IsolationLevel, LeaderAndIsrRequest, PartitionState}
 import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
@@ -48,13 +48,13 @@ class ReplicaManagerTest {
   val metrics = new Metrics
   var zkClient : ZkClient = _
   var zkUtils : ZkUtils = _
-    
+
   @Before
   def setUp() {
     zkClient = EasyMock.createMock(classOf[ZkClient])
     zkUtils = ZkUtils(zkClient, isZkSecurityEnabled = false)
   }
-  
+
   @After
   def tearDown() {
     metrics.close()
@@ -67,7 +67,7 @@ class ReplicaManagerTest {
     val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray)
     val rm = new ReplicaManager(config, metrics, time, zkUtils, new MockScheduler(time), mockLogMgr,
       new AtomicBoolean(false), QuotaFactory.instantiate(config, metrics, time).follower, new BrokerTopicStats,
-      new MetadataCache(config.brokerId))
+      new MetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size))
     try {
       val partition = rm.getOrCreatePartition(new TopicPartition(topic, 1))
       partition.getOrCreateReplica(1)
@@ -86,7 +86,7 @@ class ReplicaManagerTest {
     val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray)
     val rm = new ReplicaManager(config, metrics, time, zkUtils, new MockScheduler(time), mockLogMgr,
       new AtomicBoolean(false), QuotaFactory.instantiate(config, metrics, time).follower, new BrokerTopicStats,
-      new MetadataCache(config.brokerId))
+      new MetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size))
     try {
       val partition = rm.getOrCreatePartition(new TopicPartition(topic, 1))
       partition.getOrCreateReplica(1)
@@ -104,7 +104,7 @@ class ReplicaManagerTest {
     val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray)
     val rm = new ReplicaManager(config, metrics, time, zkUtils, new MockScheduler(time), mockLogMgr,
       new AtomicBoolean(false), QuotaFactory.instantiate(config, metrics, time).follower, new BrokerTopicStats,
-      new MetadataCache(config.brokerId), Option(this.getClass.getName))
+      new MetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), Option(this.getClass.getName))
     try {
       def callback(responseStatus: Map[TopicPartition, PartitionResponse]) = {
         assert(responseStatus.values.head.error == Errors.INVALID_REQUIRED_ACKS)
@@ -137,7 +137,7 @@ class ReplicaManagerTest {
     EasyMock.replay(metadataCache)
     val rm = new ReplicaManager(config, metrics, time, zkUtils, new MockScheduler(time), mockLogMgr,
       new AtomicBoolean(false), QuotaFactory.instantiate(config, metrics, time).follower, new BrokerTopicStats,
-      metadataCache)
+      metadataCache, new LogDirFailureChannel(config.logDirs.size))
 
     try {
       val brokerList = Seq[Integer](0, 1).asJava
@@ -145,8 +145,8 @@ class ReplicaManagerTest {
       val partition = rm.getOrCreatePartition(new TopicPartition(topic, 0))
       partition.getOrCreateReplica(0)
       // Make this replica the leader.
-      val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(0, 0,
-        collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerList)).asJava,
+      val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0,
+        collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerList, false)).asJava,
         Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
       rm.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ())
       rm.getLeaderReplicaIfLocal(new TopicPartition(topic, 0))
@@ -162,8 +162,8 @@ class ReplicaManagerTest {
       assertFalse(fetchResult.isFired)
 
       // Make this replica the follower
-      val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(0, 0,
-        collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 1, 1, brokerList, 0, brokerList)).asJava,
+      val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0,
+        collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 1, 1, brokerList, 0, brokerList, false)).asJava,
         Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
       rm.becomeLeaderOrFollower(1, leaderAndIsrRequest2, (_, _) => ())
 
@@ -186,8 +186,8 @@ class ReplicaManagerTest {
       partition.getOrCreateReplica(0)
 
       // Make this replica the leader.
-      val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(0, 0,
-        collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerList)).asJava,
+      val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0,
+        collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerList, true)).asJava,
         Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
       replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ())
       replicaManager.getLeaderReplicaIfLocal(new TopicPartition(topic, 0))
@@ -271,14 +271,12 @@ class ReplicaManagerTest {
 
     try {
       val brokerList: java.util.List[Integer] = Seq[Integer](0, 1).asJava
-      val brokerSet: java.util.Set[Integer] = Set[Integer](0, 1).asJava
-
       val partition = replicaManager.getOrCreatePartition(new TopicPartition(topic, 0))
       partition.getOrCreateReplica(0)
 
       // Make this replica the leader.
-      val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(0, 0,
-        collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerList)).asJava,
+      val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0,
+        collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerList, true)).asJava,
         Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
       replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ())
       replicaManager.getLeaderReplicaIfLocal(new TopicPartition(topic, 0))
@@ -346,7 +344,7 @@ class ReplicaManagerTest {
     EasyMock.replay(metadataCache)
     val rm = new ReplicaManager(config, metrics, time, zkUtils, new MockScheduler(time), mockLogMgr,
       new AtomicBoolean(false), QuotaFactory.instantiate(config, metrics, time).follower, new BrokerTopicStats,
-      metadataCache, Option(this.getClass.getName))
+      metadataCache, new LogDirFailureChannel(config.logDirs.size), Option(this.getClass.getName))
 
     try {
       val brokerList = Seq[Integer](0, 1, 2).asJava
@@ -355,8 +353,8 @@ class ReplicaManagerTest {
       partition.getOrCreateReplica(0)
 
       // Make this replica the leader.
-      val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(0, 0,
-        collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerList)).asJava,
+      val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, 0, 0,
+        collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerList, false)).asJava,
         Set(new Node(0, "host1", 0), new Node(1, "host2", 1), new Node(2, "host2", 2)).asJava).build()
       rm.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => ())
       rm.getLeaderReplicaIfLocal(new TopicPartition(topic, 0))
@@ -498,7 +496,8 @@ class ReplicaManagerTest {
 
     new ReplicaManager(config, metrics, time, zkUtils, new MockScheduler(time), mockLogMgr,
       new AtomicBoolean(false), QuotaFactory.instantiate(config, metrics, time).follower, new BrokerTopicStats,
-      metadataCache, mockProducePurgatory, mockFetchPurgatory, mockDeleteRecordsPurgatory, Option(this.getClass.getName))
+      metadataCache, new LogDirFailureChannel(config.logDirs.size), mockProducePurgatory, mockFetchPurgatory,
+      mockDeleteRecordsPurgatory, Option(this.getClass.getName))
   }
 
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
index 7c171b0..ecb75ce 100644
--- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
+++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
@@ -180,15 +180,16 @@ class RequestQuotaTest extends BaseRequestTest {
             .setTargetTimes(Map(tp -> (0L: java.lang.Long)).asJava)
 
         case ApiKeys.LEADER_AND_ISR =>
-          new LeaderAndIsrRequest.Builder(brokerId, Int.MaxValue,
-            Map(tp -> new PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava)).asJava,
+          new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, brokerId, Int.MaxValue,
+            Map(tp -> new PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava, true)).asJava,
             Set(new Node(brokerId, "localhost", 0)).asJava)
 
         case ApiKeys.STOP_REPLICA =>
           new StopReplicaRequest.Builder(brokerId, Int.MaxValue, true, Set(tp).asJava)
 
         case ApiKeys.UPDATE_METADATA_KEY =>
-          val partitionState = Map(tp -> new PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava)).asJava
+          val partitionState = Map(tp -> new UpdateMetadataRequest.PartitionState(
+            Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava, Seq.empty[Integer].asJava)).asJava
           val securityProtocol = SecurityProtocol.PLAINTEXT
           val brokers = Set(new UpdateMetadataRequest.Broker(brokerId,
             Seq(new UpdateMetadataRequest.EndPoint("localhost", 0, securityProtocol,

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
index 72d7fc5..a20c659 100644
--- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
+++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
@@ -16,6 +16,8 @@
  */
 package kafka.server
 
+import java.io.File
+
 import kafka.api._
 import kafka.utils._
 import kafka.cluster.Replica
@@ -105,12 +107,13 @@ class SimpleFetchTest {
     // create the log manager that is aware of this mock log
     val logManager = EasyMock.createMock(classOf[kafka.log.LogManager])
     EasyMock.expect(logManager.getLog(topicPartition)).andReturn(Some(log)).anyTimes()
+    EasyMock.expect(logManager.liveLogDirs).andReturn(Array.empty[File]).anyTimes()
     EasyMock.replay(logManager)
 
     // create the replica manager
     replicaManager = new ReplicaManager(configs.head, metrics, time, zkUtils, scheduler, logManager,
       new AtomicBoolean(false), QuotaFactory.instantiate(configs.head, metrics, time).follower, new BrokerTopicStats,
-      new MetadataCache(configs.head.brokerId))
+      new MetadataCache(configs.head.brokerId), new LogDirFailureChannel(configs.head.logDirs.size))
 
     // add the partition with two replicas, both in ISR
     val partition = replicaManager.getOrCreatePartition(new TopicPartition(topic, partitionId))

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/server/checkpoints/OffsetCheckpointFileTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/checkpoints/OffsetCheckpointFileTest.scala b/core/src/test/scala/unit/kafka/server/checkpoints/OffsetCheckpointFileTest.scala
index 4daaa72..f4998f6 100644
--- a/core/src/test/scala/unit/kafka/server/checkpoints/OffsetCheckpointFileTest.scala
+++ b/core/src/test/scala/unit/kafka/server/checkpoints/OffsetCheckpointFileTest.scala
@@ -16,10 +16,10 @@
   */
 package kafka.server.checkpoints
 
-import java.io.IOException
-
+import kafka.server.LogDirFailureChannel
 import kafka.utils.{Logging, TestUtils}
 import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.errors.KafkaStorageException
 import org.junit.Assert._
 import org.junit.Test
 import org.scalatest.junit.JUnitSuite
@@ -89,12 +89,14 @@ class OffsetCheckpointFileTest extends JUnitSuite with Logging {
     assertEquals(Map(), checkpoint.read())
   }
 
-  @Test(expected = classOf[IOException])
+  @Test(expected = classOf[KafkaStorageException])
   def shouldThrowIfVersionIsNotRecognised(): Unit = {
-    val checkpointFile = new CheckpointFile(TestUtils.tempFile(), OffsetCheckpointFile.CurrentVersion + 1,
-      OffsetCheckpointFile.Formatter)
+    val file = TestUtils.tempFile()
+    val logDirFailureChannel = new LogDirFailureChannel(10)
+    val checkpointFile = new CheckpointFile(file, OffsetCheckpointFile.CurrentVersion + 1,
+      OffsetCheckpointFile.Formatter, logDirFailureChannel, file.getParent)
     checkpointFile.write(Seq(new TopicPartition("foo", 5) -> 10L))
-    new OffsetCheckpointFile(checkpointFile.file).read()
+    new OffsetCheckpointFile(checkpointFile.file, logDirFailureChannel).read()
   }
 
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala b/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala
index d004641..e8c08fe 100644
--- a/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala
+++ b/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala
@@ -16,6 +16,7 @@
   */
 package kafka.server.epoch
 
+import java.io.File
 import java.util.concurrent.atomic.AtomicBoolean
 
 import kafka.cluster.Replica
@@ -30,6 +31,7 @@ import org.easymock.EasyMock._
 import org.junit.Assert._
 import org.junit.Test
 
+
 class OffsetsForLeaderEpochTest {
   private val config = TestUtils.createBrokerConfigs(1, TestUtils.MockZkConnect).map(KafkaConfig.fromProps).head
   private val time = new MockTime
@@ -46,14 +48,16 @@ class OffsetsForLeaderEpochTest {
     //Stubs
     val mockLog = createNiceMock(classOf[kafka.log.Log])
     val mockCache = createNiceMock(classOf[kafka.server.epoch.LeaderEpochCache])
+    val logManager = createNiceMock(classOf[kafka.log.LogManager])
     expect(mockCache.endOffsetFor(epochRequested)).andReturn(offset)
     expect(mockLog.leaderEpochCache).andReturn(mockCache).anyTimes()
-    replay(mockCache, mockLog)
+    expect(logManager.liveLogDirs).andReturn(Array.empty[File]).anyTimes()
+    replay(mockCache, mockLog, logManager)
 
     // create a replica manager with 1 partition that has 1 replica
-    val replicaManager = new ReplicaManager(config, metrics, time, null, null, null, new AtomicBoolean(false),
+    val replicaManager = new ReplicaManager(config, metrics, time, null, null, logManager, new AtomicBoolean(false),
       QuotaFactory.instantiate(config, metrics, time).follower, new BrokerTopicStats,
-      new MetadataCache(config.brokerId))
+      new MetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size))
     val partition = replicaManager.getOrCreatePartition(tp)
     val leaderReplica = new Replica(config.brokerId, partition, time, 0, Some(mockLog))
     partition.addReplicaIfNotExists(leaderReplica)
@@ -68,10 +72,14 @@ class OffsetsForLeaderEpochTest {
 
   @Test
   def shouldReturnNoLeaderForPartitionIfThrown(): Unit = {
+    val logManager = createNiceMock(classOf[kafka.log.LogManager])
+    expect(logManager.liveLogDirs).andReturn(Array.empty[File]).anyTimes()
+    replay(logManager)
+
     //create a replica manager with 1 partition that has 0 replica
-    val replicaManager = new ReplicaManager(config, metrics, time, null, null, null, new AtomicBoolean(false),
+    val replicaManager = new ReplicaManager(config, metrics, time, null, null, logManager, new AtomicBoolean(false),
       QuotaFactory.instantiate(config, metrics, time).follower, new BrokerTopicStats,
-      new MetadataCache(config.brokerId))
+      new MetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size))
     replicaManager.getOrCreatePartition(tp)
 
     //Given
@@ -87,10 +95,14 @@ class OffsetsForLeaderEpochTest {
 
   @Test
   def shouldReturnUnknownTopicOrPartitionIfThrown(): Unit = {
+    val logManager = createNiceMock(classOf[kafka.log.LogManager])
+    expect(logManager.liveLogDirs).andReturn(Array.empty[File]).anyTimes()
+    replay(logManager)
+
     //create a replica manager with 0 partition
-    val replicaManager = new ReplicaManager(config, metrics, time, null, null, null, new AtomicBoolean(false),
+    val replicaManager = new ReplicaManager(config, metrics, time, null, null, logManager, new AtomicBoolean(false),
       QuotaFactory.instantiate(config, metrics, time).follower, new BrokerTopicStats,
-      new MetadataCache(config.brokerId))
+      new MetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size))
 
     //Given
     val epochRequested: Integer = 5

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index 4976f52..d4f00a6 100755
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -156,11 +156,12 @@ object TestUtils extends Logging {
     enableSsl: Boolean = false,
     enableSaslPlaintext: Boolean = false,
     enableSaslSsl: Boolean = false,
-    rackInfo: Map[Int, String] = Map()): Seq[Properties] = {
+    rackInfo: Map[Int, String] = Map(),
+    logDirCount: Int = 1): Seq[Properties] = {
     (0 until numConfigs).map { node =>
       createBrokerConfig(node, zkConnect, enableControlledShutdown, enableDeleteTopic, RandomPort,
         interBrokerSecurityProtocol, trustStoreFile, saslProperties, enablePlaintext = enablePlaintext, enableSsl = enableSsl,
-        enableSaslPlaintext = enableSaslPlaintext, enableSaslSsl = enableSaslSsl, rack = rackInfo.get(node))
+        enableSaslPlaintext = enableSaslPlaintext, enableSaslSsl = enableSaslSsl, rack = rackInfo.get(node), logDirCount = logDirCount)
     }
   }
 
@@ -205,7 +206,7 @@ object TestUtils extends Logging {
     enablePlaintext: Boolean = true,
     enableSaslPlaintext: Boolean = false, saslPlaintextPort: Int = RandomPort,
     enableSsl: Boolean = false, sslPort: Int = RandomPort,
-    enableSaslSsl: Boolean = false, saslSslPort: Int = RandomPort, rack: Option[String] = None)
+    enableSaslSsl: Boolean = false, saslSslPort: Int = RandomPort, rack: Option[String] = None, logDirCount: Int = 1)
   : Properties = {
 
     def shouldEnable(protocol: SecurityProtocol) = interBrokerSecurityProtocol.fold(false)(_ == protocol)
@@ -227,7 +228,12 @@ object TestUtils extends Logging {
     val props = new Properties
     if (nodeId >= 0) props.put(KafkaConfig.BrokerIdProp, nodeId.toString)
     props.put(KafkaConfig.ListenersProp, listeners)
-    props.put(KafkaConfig.LogDirProp, TestUtils.tempDir().getAbsolutePath)
+    if (logDirCount > 1) {
+      val logDirs = (1 to logDirCount).toList.map(i => TestUtils.tempDir().getAbsolutePath).mkString(",")
+      props.put(KafkaConfig.LogDirsProp, logDirs)
+    } else {
+      props.put(KafkaConfig.LogDirProp, TestUtils.tempDir().getAbsolutePath)
+    }
     props.put(KafkaConfig.ZkConnectProp, zkConnect)
     props.put(KafkaConfig.ZkConnectionTimeoutMsProp, "10000")
     props.put(KafkaConfig.ReplicaSocketTimeoutMsProp, "1500")
@@ -1010,6 +1016,7 @@ object TestUtils extends Logging {
                        cleanerConfig: CleanerConfig = CleanerConfig(enableCleaner = false),
                        time: MockTime = new MockTime()): LogManager = {
     new LogManager(logDirs = logDirs,
+                   initialOfflineDirs = Array.empty[File],
                    topicConfigs = Map(),
                    defaultConfig = defaultConfig,
                    cleanerConfig = cleanerConfig,
@@ -1022,7 +1029,8 @@ object TestUtils extends Logging {
                    scheduler = time.scheduler,
                    time = time,
                    brokerState = BrokerState(),
-                   brokerTopicStats = new BrokerTopicStats)
+                   brokerTopicStats = new BrokerTopicStats,
+                   logDirFailureChannel = new LogDirFailureChannel(logDirs.size))
   }
 
   @deprecated("This method has been deprecated and it will be removed in a future release.", "0.10.0.0")
@@ -1161,7 +1169,7 @@ object TestUtils extends Logging {
       servers.forall(server => topicPartitions.forall(tp => server.getLogManager().getLog(tp).isEmpty)))
     // ensure that topic is removed from all cleaner offsets
     TestUtils.waitUntilTrue(() => servers.forall(server => topicPartitions.forall { tp =>
-      val checkpoints = server.getLogManager().logDirs.map { logDir =>
+      val checkpoints = server.getLogManager().liveLogDirs.map { logDir =>
         new OffsetCheckpointFile(new File(logDir, "cleaner-offset-checkpoint")).read()
       }
       checkpoints.forall(checkpointsPerLogDir => !checkpointsPerLogDir.contains(tp))

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/docs/upgrade.html
----------------------------------------------------------------------
diff --git a/docs/upgrade.html b/docs/upgrade.html
index f90ced3..37313b6 100644
--- a/docs/upgrade.html
+++ b/docs/upgrade.html
@@ -17,7 +17,54 @@
 
 <script><!--#include virtual="js/templateData.js" --></script>
 
-<script id="upgrade-template" type="text/x-handlebars-template">
+<h4><a id="upgrade_11_1_0" href="#upgrade_11_1_0">Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x, 0.10.2.x or 0.11.0.0 to 0.11.1.0</a></h4>
+<p>Kafka 0.11.1.0 introduces wire protocol changes. By following the recommended rolling upgrade plan below,
+    you guarantee no downtime during the upgrade. However, please review the <a href="#upgrade_1110_notable">notable changes in 0.11.1.0</a> before upgrading.
+</p>
+
+<p><b>For a rolling upgrade:</b></p>
+
+<ol>
+    <li> Update server.properties on all brokers and add the following properties. CURRENT_KAFKA_VERSION refers to the version you
+        are upgrading from. CURRENT_MESSAGE_FORMAT_VERSION refers to the current message format version currently in use. If you have
+        not overridden the message format previously, then CURRENT_MESSAGE_FORMAT_VERSION should be set to match CURRENT_KAFKA_VERSION.
+        <ul>
+            <li>inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2, 0.9.0, 0.10.0, 0.10.1, 0.10.2, 0.11.0).</li>
+        </ul>
+    </li>
+    <li> Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. </li>
+    <li> Once the entire cluster is upgraded, bump the protocol version by editing <code>inter.broker.protocol.version</code> and setting it to 0.11.1.
+    <li> Restart the brokers one by one for the new protocol version to take effect. </li>
+</ol>
+
+<p><b>Additional Upgrade Notes:</b></p>
+
+<ol>
+    <li>If you are willing to accept downtime, you can simply take all the brokers down, update the code and start them back up. They will start
+        with the new protocol by default.</li>
+    <li>Bumping the protocol version and restarting can be done any time after the brokers are upgraded. It does not have to be immediately after.
+        Similarly for the message format version.</li>
+</ol>
+
+<h5><a id="upgrade_1110_notable" href="#upgrade_1100_notable">Notable changes in 0.11.1.0</a></h5>
+<ul>
+    <li>If the <code>inter.broker.protocol.version</code> is 0.11.1 or later, a broker will now stay online to serve replicas
+        on live log directories even if there are offline log directories. A log directory may become offline due to IOException
+        caused by hardware failure. Users need to monitor the per-broker metric <code>offlineLogDirectoryCount</code> to check
+        whether there is offline log directory. </li>
+    <li>Added KafkaStorageException which is a retriable exception. KafkaStorageException will be converted to NotLeaderForPartitionException in the response
+        if the version of client's FetchRequest or ProducerRequest does not support KafkaStorageException. </li>
+</ul>
+
+<h5><a id="upgrade_1110_new_protocols" href="#upgrade_1110_new_protocols">New Protocol Versions</a></h5>
+<ul>
+    <li> <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-112%3A+Handle+disk+failure+for+JBOD">KIP-112</a>: LeaderAndIsrRequest v1 introduces a partition-level <code>is_new</code> field. </li>
+    <li> <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-112%3A+Handle+disk+failure+for+JBOD">KIP-112</a>: UpdateMetadataRequest v4 introduces a partition-level <code>offline_replicas</code> field. </li>
+    <li> <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-112%3A+Handle+disk+failure+for+JBOD">KIP-112</a>: MetadataResponse v5 introduces a partition-level <code>offline_replicas</code> field. </li>
+    <li> <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-112%3A+Handle+disk+failure+for+JBOD">KIP-112</a>: ProduceResponse v4 introduces error code for KafkaStorageException. </li>
+    <li> <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-112%3A+Handle+disk+failure+for+JBOD">KIP-112</a>: FetchResponse v6 introduces error code for KafkaStorageException. </li>
+</ul>
+
 
 <h4><a id="upgrade_1_0_0" href="#upgrade_1_0_0"</h4> Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x, 0.10.2 or 0.11.0.0 to 1.0.0</a></h4>
 <p>1.0.0 is fully compatible with 0.11.0.0. The upgrade can be done one broker at a time by simply bringing it down, updating the code, and restarting it.
@@ -73,7 +120,7 @@ To upgrade from earlier versions, please review the <a href="#upgrade_11_0_0">0.
     Similarly for the message format version.</li>
   <li>It is also possible to enable the 0.11.0 message format on individual topics using the topic admin tool (<code>bin/kafka-topics.sh</code>)
     prior to updating the global setting <code>log.message.format.version</code>.</li>
-  <li>If  you are upgrading from a version prior to 0.10.0, it is NOT necessary to first update the message format to 0.10.0
+  <li>If you are upgrading from a version prior to 0.10.0, it is NOT necessary to first update the message format to 0.10.0
     before you switch to 0.11.0.</li>
 </ol>
 
@@ -98,7 +145,7 @@ To upgrade from earlier versions, please review the <a href="#upgrade_11_0_0">0.
         individual messages is only reduced by the overhead of the batch format. However, there are some subtle implications
         for message format conversion (see <a href="#upgrade_11_message_format">below</a> for more detail). Note also
         that while previously the broker would ensure that at least one message is returned in each fetch request (regardless of the
-        total and partition-level fetch sizes), the same behavior now applies to one message batch.</li> 
+        total and partition-level fetch sizes), the same behavior now applies to one message batch.</li>
     <li>GC log rotation is enabled by default, see KAFKA-3754 for details.</li>
     <li>Deprecated constructors of RecordMetadata, MetricName and Cluster classes have been removed.</li>
     <li>Added user headers support through a new Headers interface providing user headers read and write access.</li>
@@ -151,7 +198,7 @@ To upgrade from earlier versions, please review the <a href="#upgrade_11_0_0">0.
   <li>EoS in Kafka introduces new request APIs and modifies several existing ones. See
     <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging#KIP-98-ExactlyOnceDeliveryandTransactionalMessaging-RPCProtocolSummary">KIP-98</a>
     for the full details</code></li>
-</ol>    
+</ol>
 
 <h5><a id="upgrade_11_message_format" href="#upgrade_11_message_format">Notes on the new message format in 0.11.0</a></h5>
 <p>The 0.11.0 message format includes several major enhancements in order to support better delivery semantics for the producer
@@ -182,7 +229,7 @@ To upgrade from earlier versions, please review the <a href="#upgrade_11_0_0">0.
   are upgraded to the latest 0.11.0 client. Significantly, since the old consumer has been deprecated in 0.11.0.0, it does not support
   the new message format. You must upgrade to use the new consumer to use the new message format without the cost of down-conversion.
   Note that 0.11.0 consumers support backwards compability with brokers 0.10.0 brokers and upward, so it is possible to upgrade the
-  clients first before the brokers. 
+  clients first before the brokers.
 </p>
 
 <h4><a id="upgrade_10_2_0" href="#upgrade_10_2_0">Upgrading from 0.8.x, 0.9.x, 0.10.0.x or 0.10.1.x to 0.10.2.0</a></h4>

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java
index 7cdc180..eb844d3 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java
@@ -16,7 +16,7 @@
  */
 package org.apache.kafka.streams.integration.utils;
 
-import kafka.api.PartitionStateInfo;
+import kafka.api.MetadataPartitionState;
 import kafka.api.Request;
 import kafka.server.KafkaServer;
 import kafka.server.MetadataCache;
@@ -225,13 +225,13 @@ public class IntegrationTestUtils {
             public boolean conditionMet() {
                 for (final KafkaServer server : servers) {
                     final MetadataCache metadataCache = server.apis().metadataCache();
-                    final Option<PartitionStateInfo> partitionInfo =
+                    final Option<MetadataPartitionState> partitionInfo =
                             metadataCache.getPartitionInfo(topic, partition);
                     if (partitionInfo.isEmpty()) {
                         return false;
                     }
-                    final PartitionStateInfo partitionStateInfo = partitionInfo.get();
-                    if (!Request.isValidBrokerId(partitionStateInfo.leaderIsrAndControllerEpoch().leaderAndIsr().leader())) {
+                    final MetadataPartitionState metadataPartitionState = partitionInfo.get();
+                    if (!Request.isValidBrokerId(metadataPartitionState.leaderIsrAndControllerEpoch().leaderAndIsr().leader())) {
                         return false;
                     }
                 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java
index 1a46356..469f9cb 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java
@@ -133,7 +133,7 @@ public class InternalTopicManagerTest {
         @Override
         public MetadataResponse fetchMetadata() {
             Node node = new Node(1, "host1", 1001);
-            MetadataResponse.PartitionMetadata partitionMetadata = new MetadataResponse.PartitionMetadata(Errors.NONE, 1, node, new ArrayList<Node>(), new ArrayList<Node>());
+            MetadataResponse.PartitionMetadata partitionMetadata = new MetadataResponse.PartitionMetadata(Errors.NONE, 1, node, new ArrayList<Node>(), new ArrayList<Node>(), new ArrayList<Node>());
             MetadataResponse.TopicMetadata topicMetadata = new MetadataResponse.TopicMetadata(Errors.NONE, topic, true, Collections.singletonList(partitionMetadata));
             MetadataResponse response = new MetadataResponse(Collections.<Node>singletonList(node), null, MetadataResponse.NO_CONTROLLER_ID,
                 Collections.singletonList(topicMetadata));

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/tests/kafkatest/services/kafka/config.py
----------------------------------------------------------------------
diff --git a/tests/kafkatest/services/kafka/config.py b/tests/kafkatest/services/kafka/config.py
index 1b81b43..462277f 100644
--- a/tests/kafkatest/services/kafka/config.py
+++ b/tests/kafkatest/services/kafka/config.py
@@ -24,7 +24,7 @@ class KafkaConfig(dict):
     DEFAULTS = {
         config_property.PORT: 9092,
         config_property.SOCKET_RECEIVE_BUFFER_BYTES: 65536,
-        config_property.LOG_DIRS: "/mnt/kafka-data-logs",
+        config_property.LOG_DIRS: "/mnt/kafka-data-logs-1,/mnt/kafka-data-logs-2",
         config_property.ZOOKEEPER_CONNECTION_TIMEOUT_MS: 2000
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/tests/kafkatest/services/kafka/config_property.py
----------------------------------------------------------------------
diff --git a/tests/kafkatest/services/kafka/config_property.py b/tests/kafkatest/services/kafka/config_property.py
index a9ba40d..8b64d0e 100644
--- a/tests/kafkatest/services/kafka/config_property.py
+++ b/tests/kafkatest/services/kafka/config_property.py
@@ -44,6 +44,10 @@ MESSAGE_FORMAT_VERSION = "log.message.format.version"
 MESSAGE_TIMESTAMP_TYPE = "message.timestamp.type"
 THROTTLING_REPLICATION_RATE_LIMIT = "replication.quota.throttled.rate"
 
+LOG_FLUSH_INTERVAL_MESSAGE = "log.flush.interval.messages"
+REPLICA_HIGHWATERMARK_CHECKPOINT_INTERVAL_MS = "replica.high.watermark.checkpoint.interval.ms"
+LOG_ROLL_TIME_MS = "log.roll.ms"
+
 """
 From KafkaConfig.scala
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/tests/kafkatest/services/kafka/kafka.py
----------------------------------------------------------------------
diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py
index b22b518..e941a3d 100644
--- a/tests/kafkatest/services/kafka/kafka.py
+++ b/tests/kafkatest/services/kafka/kafka.py
@@ -45,7 +45,9 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
     OPERATIONAL_LOG_INFO_DIR = os.path.join(OPERATIONAL_LOG_DIR, "info")
     OPERATIONAL_LOG_DEBUG_DIR = os.path.join(OPERATIONAL_LOG_DIR, "debug")
     # Kafka log segments etc go here
-    DATA_LOG_DIR = os.path.join(PERSISTENT_ROOT, "kafka-data-logs")
+    DATA_LOG_DIR_PREFIX = os.path.join(PERSISTENT_ROOT, "kafka-data-logs")
+    DATA_LOG_DIR_1 = "%s-1" % (DATA_LOG_DIR_PREFIX)
+    DATA_LOG_DIR_2 = "%s-2" % (DATA_LOG_DIR_PREFIX)
     CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "kafka.properties")
     # Kafka Authorizer
     SIMPLE_AUTHORIZER = "kafka.security.auth.SimpleAclAuthorizer"
@@ -60,8 +62,11 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
         "kafka_operational_logs_debug": {
             "path": OPERATIONAL_LOG_DEBUG_DIR,
             "collect_default": False},
-        "kafka_data": {
-            "path": DATA_LOG_DIR,
+        "kafka_data_1": {
+            "path": DATA_LOG_DIR_1,
+            "collect_default": False},
+        "kafka_data_2": {
+            "path": DATA_LOG_DIR_2,
             "collect_default": False}
     }
 
@@ -150,6 +155,9 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
         else:
             self.minikdc = None
 
+    def alive(self, node):
+        return len(self.pids(node)) > 0
+
     def start(self, add_principals=""):
         self.open_port(self.security_protocol)
         self.open_port(self.interbroker_security_protocol)
@@ -262,7 +270,7 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
         JmxMixin.clean_node(self, node)
         self.security_config.clean_node(node)
         node.account.kill_process("kafka", clean_shutdown=False, allow_fail=True)
-        node.account.ssh("rm -rf /mnt/*", allow_fail=False)
+        node.account.ssh("sudo rm -rf /mnt/*", allow_fail=False)
 
     def create_topic(self, topic_cfg, node=None):
         """Run the admin tool create topic command.
@@ -444,7 +452,7 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
                           len(messages))
         for node in self.nodes:
             # Grab all .log files in directories prefixed with this topic
-            files = node.account.ssh_capture("find %s -regex  '.*/%s-.*/[^/]*.log'" % (KafkaService.DATA_LOG_DIR, topic))
+            files = node.account.ssh_capture("find %s* -regex  '.*/%s-.*/[^/]*.log'" % (KafkaService.DATA_LOG_DIR_PREFIX, topic))
 
             # Check each data file to see if it contains the messages we want
             for log in files:
@@ -471,10 +479,45 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
         self.stop_node(node, clean_shutdown)
         self.start_node(node)
 
+    def isr_idx_list(self, topic, partition=0):
+        """ Get in-sync replica list the given topic and partition.
+        """
+        self.logger.debug("Querying zookeeper to find in-sync replicas for topic %s and partition %d" % (topic, partition))
+        zk_path = "/brokers/topics/%s/partitions/%d/state" % (topic, partition)
+        partition_state = self.zk.query(zk_path)
+
+        if partition_state is None:
+            raise Exception("Error finding partition state for topic %s and partition %d." % (topic, partition))
+
+        partition_state = json.loads(partition_state)
+        self.logger.info(partition_state)
+
+        isr_idx_list = partition_state["isr"]
+        self.logger.info("Isr for topic %s and partition %d is now: %s" % (topic, partition, isr_idx_list))
+        return isr_idx_list
+
+    def replicas(self, topic, partition=0):
+        """ Get the assigned replicas for the given topic and partition.
+        """
+        self.logger.debug("Querying zookeeper to find assigned replicas for topic %s and partition %d" % (topic, partition))
+        zk_path = "/brokers/topics/%s" % (topic)
+        assignemnt = self.zk.query(zk_path)
+
+        if assignemnt is None:
+            raise Exception("Error finding partition state for topic %s and partition %d." % (topic, partition))
+
+        assignemnt = json.loads(assignemnt)
+        self.logger.info(assignemnt)
+
+        replicas = assignemnt["partitions"][str(partition)]
+
+        self.logger.info("Assigned replicas for topic %s and partition %d is now: %s" % (topic, partition, replicas))
+        return [self.get_node(replica) for replica in replicas]
+
     def leader(self, topic, partition=0):
         """ Get the leader replica for the given topic and partition.
         """
-        self.logger.debug("Querying zookeeper to find leader replica for topic: \n%s" % (topic))
+        self.logger.debug("Querying zookeeper to find leader replica for topic %s and partition %d" % (topic, partition))
         zk_path = "/brokers/topics/%s/partitions/%d/state" % (topic, partition)
         partition_state = self.zk.query(zk_path)
 
@@ -541,7 +584,7 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
         self.logger.debug(output)
         return output
 
-    def bootstrap_servers(self, protocol='PLAINTEXT', validate=True):
+    def bootstrap_servers(self, protocol='PLAINTEXT', validate=True, offline_nodes=[]):
         """Return comma-delimited list of brokers in this cluster formatted as HOSTNAME1:PORT1,HOSTNAME:PORT2,...
 
         This is the format expected by many config files.
@@ -552,7 +595,7 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
         if validate and not port_mapping.open:
             raise ValueError("We are retrieving bootstrap servers for the port: %s which is not currently open. - " % str(port_mapping))
 
-        return ','.join([node.account.hostname + ":" + str(port_mapping.number) for node in self.nodes])
+        return ','.join([node.account.hostname + ":" + str(port_mapping.number) for node in self.nodes if node not in offline_nodes])
 
     def controller(self):
         """ Get the controller node

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/tests/kafkatest/services/verifiable_producer.py
----------------------------------------------------------------------
diff --git a/tests/kafkatest/services/verifiable_producer.py b/tests/kafkatest/services/verifiable_producer.py
index 3cf3abd..6ba3e86 100644
--- a/tests/kafkatest/services/verifiable_producer.py
+++ b/tests/kafkatest/services/verifiable_producer.py
@@ -56,7 +56,7 @@ class VerifiableProducer(KafkaPathResolverMixin, VerifiableClientMixin, Backgrou
 
     def __init__(self, context, num_nodes, kafka, topic, max_messages=-1, throughput=100000,
                  message_validator=is_int, compression_types=None, version=DEV_BRANCH, acks=None,
-                 stop_timeout_sec=150, request_timeout_sec=30, log_level="INFO", enable_idempotence=False):
+                 stop_timeout_sec=150, request_timeout_sec=30, log_level="INFO", enable_idempotence=False, offline_nodes=[]):
         """
         :param max_messages is a number of messages to be produced per producer
         :param message_validator checks for an expected format of messages produced. There are
@@ -90,6 +90,7 @@ class VerifiableProducer(KafkaPathResolverMixin, VerifiableClientMixin, Backgrou
         self.stop_timeout_sec = stop_timeout_sec
         self.request_timeout_sec = request_timeout_sec
         self.enable_idempotence = enable_idempotence
+        self.offline_nodes = offline_nodes
 
     def java_class_name(self):
         return "VerifiableProducer"
@@ -184,7 +185,7 @@ class VerifiableProducer(KafkaPathResolverMixin, VerifiableClientMixin, Backgrou
         cmd += " export KAFKA_OPTS=%s;" % self.security_config.kafka_opts
         cmd += " export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\"; " % VerifiableProducer.LOG4J_CONFIG
         cmd += self.impl.exec_cmd(node)
-        cmd += " --topic %s --broker-list %s" % (self.topic, self.kafka.bootstrap_servers(self.security_config.security_protocol))
+        cmd += " --topic %s --broker-list %s" % (self.topic, self.kafka.bootstrap_servers(self.security_config.security_protocol, True, self.offline_nodes))
         if self.max_messages > 0:
             cmd += " --max-messages %s" % str(self.max_messages)
         if self.throughput > 0:

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/tests/kafkatest/tests/core/log_dir_failure_test.py
----------------------------------------------------------------------
diff --git a/tests/kafkatest/tests/core/log_dir_failure_test.py b/tests/kafkatest/tests/core/log_dir_failure_test.py
new file mode 100644
index 0000000..faa13c0
--- /dev/null
+++ b/tests/kafkatest/tests/core/log_dir_failure_test.py
@@ -0,0 +1,177 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from ducktape.utils.util import wait_until
+from ducktape.mark import matrix
+from ducktape.mark.resource import cluster
+from kafkatest.services.kafka import config_property
+from kafkatest.services.zookeeper import ZookeeperService
+from kafkatest.services.kafka import KafkaService
+from kafkatest.services.verifiable_producer import VerifiableProducer
+from kafkatest.services.console_consumer import ConsoleConsumer
+from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
+from kafkatest.utils import is_int
+
+
+def select_node(test, broker_type, topic):
+    """ Discover node of requested type. For leader type, discovers leader for our topic and partition 0
+    """
+    if broker_type == "leader":
+        node = test.kafka.leader(topic, partition=0)
+    elif broker_type == "follower":
+        leader = test.kafka.leader(topic, partition=0)
+        node = [replica for replica in test.kafka.replicas(topic, partition=0) if replica != leader][0]
+    elif broker_type == "controller":
+        node = test.kafka.controller()
+    else:
+        raise Exception("Unexpected broker type %s." % (broker_type))
+
+    return node
+
+
+class LogDirFailureTest(ProduceConsumeValidateTest):
+    """
+    Note that consuming is a bit tricky, at least with console consumer. The goal is to consume all messages
+    (foreach partition) in the topic. In this case, waiting for the last message may cause the consumer to stop
+    too soon since console consumer is consuming multiple partitions from a single thread and therefore we lose
+    ordering guarantees.
+
+    Waiting on a count of consumed messages can be unreliable: if we stop consuming when num_consumed == num_acked,
+    we might exit early if some messages are duplicated (though not an issue here since producer retries==0)
+
+    Therefore rely here on the consumer.timeout.ms setting which times out on the interval between successively
+    consumed messages. Since we run the producer to completion before running the consumer, this is a reliable
+    indicator that nothing is left to consume.
+    """
+
+    def __init__(self, test_context):
+        """:type test_context: ducktape.tests.test.TestContext"""
+        super(LogDirFailureTest, self).__init__(test_context=test_context)
+
+        self.topic1 = "test_topic_1"
+        self.topic2 = "test_topic_2"
+        self.zk = ZookeeperService(test_context, num_nodes=1)
+        self.kafka = KafkaService(test_context,
+                                  num_nodes=3,
+                                  zk=self.zk,
+                                  topics={
+                                      self.topic1: {"partitions": 1, "replication-factor": 3, "configs": {"min.insync.replicas": 2}},
+                                      self.topic2: {"partitions": 1, "replication-factor": 3, "configs": {"min.insync.replicas": 1}}
+                                  },
+                                  # Set log.roll.ms to 3 seconds so that broker will detect disk error sooner when it creates log segment
+                                  # Otherwise broker will still be able to read/write the log file even if the log directory is inaccessible.
+                                  server_prop_overides=[
+                                      [config_property.LOG_FLUSH_INTERVAL_MESSAGE, "5"],
+                                      [config_property.REPLICA_HIGHWATERMARK_CHECKPOINT_INTERVAL_MS, "60000"],
+                                      [config_property.LOG_ROLL_TIME_MS, "3000"]
+                                  ])
+
+        self.producer_throughput = 1000
+        self.num_producers = 1
+        self.num_consumers = 1
+
+    def setUp(self):
+        self.zk.start()
+
+    def min_cluster_size(self):
+        """Override this since we're adding services outside of the constructor"""
+        return super(LogDirFailureTest, self).min_cluster_size() + self.num_producers * 2 + self.num_consumers * 2
+
+    @cluster(num_nodes=9)
+    @matrix(bounce_broker=[False, True], broker_type=["leader", "follower"], security_protocol=["PLAINTEXT"])
+    def test_replication_with_disk_failure(self, bounce_broker, security_protocol, broker_type):
+        """Replication tests.
+        These tests verify that replication provides simple durability guarantees by checking that data acked by
+        brokers is still available for consumption in the face of various failure scenarios.
+
+        Setup: 1 zk, 3 kafka nodes, 1 topic with partitions=3, replication-factor=3, and min.insync.replicas=2
+               and another topic with partitions=3, replication-factor=3, and min.insync.replicas=1
+        
+            - Produce messages in the background
+            - Consume messages in the background
+            - Drive broker failures (shutdown, or bounce repeatedly with kill -15 or kill -9)
+            - When done driving failures, stop producing, and finish consuming
+            - Validate that every acked message was consumed
+        """
+
+        self.kafka.security_protocol = security_protocol
+        self.kafka.interbroker_security_protocol = security_protocol
+        self.kafka.start()
+
+        try:
+            # Initialize producer/consumer for topic1
+            self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic1,
+                                               throughput=self.producer_throughput)
+            self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic1, group_id="test-consumer-group-1",
+                                            new_consumer=False, consumer_timeout_ms=60000, message_validator=is_int)
+            self.start_producer_and_consumer()
+
+            # Get a replica of the partition of topic1 and make its first log directory offline by changing the log dir's permission.
+            # We assume that partition of topic1 is created in the first log directory of respective brokers.
+            broker_node = select_node(self, broker_type, self.topic1)
+            broker_idx = self.kafka.idx(broker_node)
+            assert broker_idx in self.kafka.isr_idx_list(self.topic1), \
+                   "Broker %d should be in isr set %s" % (broker_idx, str(self.kafka.isr_idx_list(self.topic1)))
+
+            self.logger.debug("Making log dir %s inaccessible" % (KafkaService.DATA_LOG_DIR_1))
+            cmd = "chmod a-w %s -R" % (KafkaService.DATA_LOG_DIR_1)
+            broker_node.account.ssh(cmd, allow_fail=False)
+
+            if bounce_broker:
+                self.kafka.restart_node(broker_node, clean_shutdown=True)
+
+            # Verify the following:
+            # 1) The broker with offline log directory is not the leader of the partition of topic1
+            # 2) The broker with offline log directory is not in the ISR
+            # 3) The broker with offline log directory is still online
+            # 4) Messages can still be produced and consumed from topic1
+            wait_until(lambda: self.kafka.leader(self.topic1, partition=0) != broker_node,
+                       timeout_sec=60,
+                       err_msg="Broker %d should not be leader of topic %s and partition 0" % (broker_idx, self.topic1))
+            assert self.kafka.alive(broker_node), "Broker %d should be still online" % (broker_idx)
+            wait_until(lambda: broker_idx not in self.kafka.isr_idx_list(self.topic1),
+                       timeout_sec=60,
+                       err_msg="Broker %d should not be in isr set %s" % (broker_idx, str(self.kafka.isr_idx_list(self.topic1))))
+
+            self.stop_producer_and_consumer()
+            self.validate()
+
+            # Shutdown all other brokers so that the broker with offline log dir is the only online broker
+            offline_nodes = []
+            for node in self.kafka.nodes:
+                if broker_node != node:
+                    offline_nodes.append(node)
+                    self.logger.debug("Hard shutdown broker %d" % (self.kafka.idx(node)))
+                    self.kafka.stop_node(node)
+
+            # Verify the following:
+            # 1) The broker with offline directory is the only in-sync broker of the partition of topic2
+            # 2) Messages can still be produced and consumed from topic2
+            self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic2,
+                                               throughput=self.producer_throughput, offline_nodes=offline_nodes)
+            self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic2, group_id="test-consumer-group-2",
+                                            new_consumer=False, consumer_timeout_ms=60000, message_validator=is_int)
+            self.start_producer_and_consumer()
+
+            assert self.kafka.isr_idx_list(self.topic2) == [broker_idx], \
+                   "In-sync replicas of topic %s and partition 0 should be %s" % (self.topic2, str([broker_idx]))
+
+            self.stop_producer_and_consumer()
+            self.validate()
+
+        except BaseException as e:
+            for s in self.test_context.services:
+                self.mark_for_collect(s)
+            raise

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/tests/kafkatest/tests/core/transactions_test.py
----------------------------------------------------------------------
diff --git a/tests/kafkatest/tests/core/transactions_test.py b/tests/kafkatest/tests/core/transactions_test.py
index 9ccb259..d8f9e5c 100644
--- a/tests/kafkatest/tests/core/transactions_test.py
+++ b/tests/kafkatest/tests/core/transactions_test.py
@@ -219,7 +219,8 @@ class TransactionsTest(Test):
         security_protocol = 'PLAINTEXT'
         self.kafka.security_protocol = security_protocol
         self.kafka.interbroker_security_protocol = security_protocol
-        self.kafka.logs["kafka_data"]["collect_default"] = True
+        self.kafka.logs["kafka_data_1"]["collect_default"] = True
+        self.kafka.logs["kafka_data_2"]["collect_default"] = True
         self.kafka.logs["kafka_operational_logs_debug"]["collect_default"] = True
         self.kafka.start()
         input_messages = self.seed_messages()


[2/5] kafka git commit: KAFKA-4763; Handle disk failure for JBOD (KIP-112)

Posted by jq...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
index d7420dd..e856ca1 100644
--- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
@@ -18,16 +18,13 @@
 package kafka.server
 
 import java.util
-
 import kafka.admin.AdminUtils
 import kafka.api.{FetchRequest => _, _}
 import kafka.cluster.{BrokerEndPoint, Replica}
-import kafka.common.KafkaStorageException
 import kafka.log.LogConfig
 import kafka.server.ReplicaFetcherThread._
 import kafka.server.epoch.LeaderEpochCache
 import org.apache.kafka.common.requests.EpochEndOffset._
-import kafka.utils.Exit
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.internals.FatalExitError
 import org.apache.kafka.common.metrics.Metrics
@@ -35,7 +32,6 @@ import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.record.MemoryRecords
 import org.apache.kafka.common.requests.{EpochEndOffset, FetchResponse, ListOffsetRequest, ListOffsetResponse, OffsetsForLeaderEpochRequest, OffsetsForLeaderEpochResponse, FetchRequest => JFetchRequest}
 import org.apache.kafka.common.utils.Time
-
 import scala.collection.JavaConverters._
 import scala.collection.{Map, mutable}
 
@@ -83,41 +79,35 @@ class ReplicaFetcherThread(name: String,
 
   // process fetched data
   def processPartitionData(topicPartition: TopicPartition, fetchOffset: Long, partitionData: PartitionData) {
-    try {
-      val replica = replicaMgr.getReplica(topicPartition).get
-      val records = partitionData.toRecords
-
-      maybeWarnIfOversizedRecords(records, topicPartition)
-
-      if (fetchOffset != replica.logEndOffset.messageOffset)
-        throw new RuntimeException("Offset mismatch for partition %s: fetched offset = %d, log end offset = %d.".format(topicPartition, fetchOffset, replica.logEndOffset.messageOffset))
-      if (logger.isTraceEnabled)
-        trace("Follower %d has replica log end offset %d for partition %s. Received %d messages and leader hw %d"
-          .format(replica.brokerId, replica.logEndOffset.messageOffset, topicPartition, records.sizeInBytes, partitionData.highWatermark))
-
-      // Append the leader's messages to the log
-      replica.log.get.appendAsFollower(records)
-
-      if (logger.isTraceEnabled)
-        trace("Follower %d has replica log end offset %d after appending %d bytes of messages for partition %s"
-          .format(replica.brokerId, replica.logEndOffset.messageOffset, records.sizeInBytes, topicPartition))
-      val followerHighWatermark = replica.logEndOffset.messageOffset.min(partitionData.highWatermark)
-      val leaderLogStartOffset = partitionData.logStartOffset
-      // for the follower replica, we do not need to keep
-      // its segment base offset the physical position,
-      // these values will be computed upon making the leader
-      replica.highWatermark = new LogOffsetMetadata(followerHighWatermark)
-      replica.maybeIncrementLogStartOffset(leaderLogStartOffset)
-      if (logger.isTraceEnabled)
-        trace(s"Follower ${replica.brokerId} set replica high watermark for partition $topicPartition to $followerHighWatermark")
-      if (quota.isThrottled(topicPartition))
-        quota.record(records.sizeInBytes)
-      replicaMgr.brokerTopicStats.updateReplicationBytesIn(records.sizeInBytes)
-    } catch {
-      case e: KafkaStorageException =>
-        fatal(s"Disk error while replicating data for $topicPartition", e)
-        Exit.halt(1)
-    }
+    val replica = replicaMgr.getReplica(topicPartition).get
+    val records = partitionData.toRecords
+
+    maybeWarnIfOversizedRecords(records, topicPartition)
+
+    if (fetchOffset != replica.logEndOffset.messageOffset)
+      throw new RuntimeException("Offset mismatch for partition %s: fetched offset = %d, log end offset = %d.".format(topicPartition, fetchOffset, replica.logEndOffset.messageOffset))
+    if (logger.isTraceEnabled)
+      trace("Follower %d has replica log end offset %d for partition %s. Received %d messages and leader hw %d"
+        .format(replica.brokerId, replica.logEndOffset.messageOffset, topicPartition, records.sizeInBytes, partitionData.highWatermark))
+
+    // Append the leader's messages to the log
+    replica.log.get.appendAsFollower(records)
+
+    if (logger.isTraceEnabled)
+      trace("Follower %d has replica log end offset %d after appending %d bytes of messages for partition %s"
+        .format(replica.brokerId, replica.logEndOffset.messageOffset, records.sizeInBytes, topicPartition))
+    val followerHighWatermark = replica.logEndOffset.messageOffset.min(partitionData.highWatermark)
+    val leaderLogStartOffset = partitionData.logStartOffset
+    // for the follower replica, we do not need to keep
+    // its segment base offset the physical position,
+    // these values will be computed upon making the leader
+    replica.highWatermark = new LogOffsetMetadata(followerHighWatermark)
+    replica.maybeIncrementLogStartOffset(leaderLogStartOffset)
+    if (logger.isTraceEnabled)
+      trace(s"Follower ${replica.brokerId} set replica high watermark for partition $topicPartition to $followerHighWatermark")
+    if (quota.isThrottled(topicPartition))
+      quota.record(records.sizeInBytes)
+    replicaMgr.brokerTopicStats.updateReplicationBytesIn(records.sizeInBytes)
   }
 
   def maybeWarnIfOversizedRecords(records: MemoryRecords, topicPartition: TopicPartition): Unit = {

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/server/ReplicaManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala
index 853b7c4..40887be 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -16,14 +16,13 @@
  */
 package kafka.server
 
-import java.io.{File, IOException}
+import java.io.File
 import java.util.concurrent.TimeUnit
 import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong}
 
 import com.yammer.metrics.core.Gauge
 import kafka.api._
 import kafka.cluster.{Partition, Replica}
-import kafka.common.KafkaStorageException
 import kafka.controller.KafkaController
 import kafka.log.{Log, LogAppendInfo, LogManager}
 import kafka.metrics.KafkaMetricsGroup
@@ -31,11 +30,12 @@ import kafka.server.QuotaFactory.UnboundedQuota
 import kafka.server.checkpoints.OffsetCheckpointFile
 import kafka.utils._
 import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.common.errors.{ControllerMovedException, CorruptRecordException, InvalidTimestampException, InvalidTopicException, NotEnoughReplicasException, NotLeaderForPartitionException, OffsetOutOfRangeException, PolicyViolationException, _}
+import org.apache.kafka.common.errors.{KafkaStorageException, ControllerMovedException, CorruptRecordException, InvalidTimestampException, InvalidTopicException, NotEnoughReplicasException, NotLeaderForPartitionException, OffsetOutOfRangeException, PolicyViolationException, _}
 import org.apache.kafka.common.internals.Topic
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.protocol.Errors.UNKNOWN_TOPIC_OR_PARTITION
+import org.apache.kafka.common.protocol.Errors.KAFKA_STORAGE_ERROR
 import org.apache.kafka.common.record._
 import org.apache.kafka.common.requests.EpochEndOffset._
 import org.apache.kafka.common.requests.FetchRequest.PartitionData
@@ -123,6 +123,7 @@ object ReplicaManager {
   val HighWatermarkFilename = "replication-offset-checkpoint"
   val IsrChangePropagationBlackOut = 5000L
   val IsrChangePropagationInterval = 60000L
+  val OfflinePartition = new Partition("", -1, null, null, isOffline = true)
 }
 
 class ReplicaManager(val config: KafkaConfig,
@@ -135,6 +136,7 @@ class ReplicaManager(val config: KafkaConfig,
                      quotaManager: ReplicationQuotaManager,
                      val brokerTopicStats: BrokerTopicStats,
                      val metadataCache: MetadataCache,
+                     logDirFailureChannel: LogDirFailureChannel,
                      val delayedProducePurgatory: DelayedOperationPurgatory[DelayedProduce],
                      val delayedFetchPurgatory: DelayedOperationPurgatory[DelayedFetch],
                      val delayedDeleteRecordsPurgatory: DelayedOperationPurgatory[DelayedDeleteRecords],
@@ -150,9 +152,10 @@ class ReplicaManager(val config: KafkaConfig,
            quotaManager: ReplicationQuotaManager,
            brokerTopicStats: BrokerTopicStats,
            metadataCache: MetadataCache,
+           logDirFailureChannel: LogDirFailureChannel,
            threadNamePrefix: Option[String] = None) {
     this(config, metrics, time, zkUtils, scheduler, logManager, isShuttingDown,
-      quotaManager, brokerTopicStats, metadataCache,
+      quotaManager, brokerTopicStats, metadataCache, logDirFailureChannel,
       DelayedOperationPurgatory[DelayedProduce](
         purgatoryName = "Produce", brokerId = config.brokerId,
         purgeInterval = config.producerPurgatoryPurgeIntervalRequests),
@@ -173,13 +176,27 @@ class ReplicaManager(val config: KafkaConfig,
   private val replicaStateChangeLock = new Object
   val replicaFetcherManager = createReplicaFetcherManager(metrics, time, threadNamePrefix, quotaManager)
   private val highWatermarkCheckPointThreadStarted = new AtomicBoolean(false)
-  val highWatermarkCheckpoints = config.logDirs.map(dir => (new File(dir).getAbsolutePath, new OffsetCheckpointFile(new File(dir, ReplicaManager.HighWatermarkFilename)))).toMap
+  @volatile var highWatermarkCheckpoints = logManager.liveLogDirs.map(dir =>
+    (dir.getAbsolutePath, new OffsetCheckpointFile(new File(dir, ReplicaManager.HighWatermarkFilename), logDirFailureChannel))).toMap
+
   private var hwThreadInitialized = false
   this.logIdent = "[Replica Manager on Broker " + localBrokerId + "]: "
   val stateChangeLogger = KafkaController.stateChangeLogger
   private val isrChangeSet: mutable.Set[TopicPartition] = new mutable.HashSet[TopicPartition]()
   private val lastIsrChangeMs = new AtomicLong(System.currentTimeMillis())
   private val lastIsrPropagationMs = new AtomicLong(System.currentTimeMillis())
+  private var logDirFailureHandler: LogDirFailureHandler = null
+
+  private class LogDirFailureHandler(name: String, haltBrokerOnDirFailure: Boolean) extends ShutdownableThread(name) {
+    override def doWork() {
+      val newOfflineLogDir = logDirFailureChannel.takeNextLogFailureEvent()
+      if (haltBrokerOnDirFailure) {
+        fatal(s"Halting broker because dir $newOfflineLogDir is offline")
+        Exit.halt(1)
+      }
+      handleLogDirFailure(newOfflineLogDir)
+    }
+  }
 
   val leaderCount = newGauge(
     "LeaderCount",
@@ -193,6 +210,12 @@ class ReplicaManager(val config: KafkaConfig,
       def value = allPartitions.size
     }
   )
+  val offlineReplicaCount = newGauge(
+    "OfflineReplicaCount",
+    new Gauge[Int] {
+      def value = allPartitions.values.count(_ eq ReplicaManager.OfflinePartition)
+    }
+  )
   val underReplicatedPartitions = newGauge(
     "UnderReplicatedPartitions",
     new Gauge[Int] {
@@ -277,20 +300,30 @@ class ReplicaManager(val config: KafkaConfig,
     // A follower can lag behind leader for up to config.replicaLagTimeMaxMs x 1.5 before it is removed from ISR
     scheduler.schedule("isr-expiration", maybeShrinkIsr _, period = config.replicaLagTimeMaxMs / 2, unit = TimeUnit.MILLISECONDS)
     scheduler.schedule("isr-change-propagation", maybePropagateIsrChanges _, period = 2500L, unit = TimeUnit.MILLISECONDS)
+    val haltBrokerOnFailure = config.interBrokerProtocolVersion < KAFKA_0_11_1_IV0
+    logDirFailureHandler = new LogDirFailureHandler("LogDirFailureHandler", haltBrokerOnFailure)
+    logDirFailureHandler.start()
   }
 
   def stopReplica(topicPartition: TopicPartition, deletePartition: Boolean): Errors  = {
     stateChangeLogger.trace(s"Broker $localBrokerId handling stop replica (delete=$deletePartition) for partition $topicPartition")
     val error = Errors.NONE
     getPartition(topicPartition) match {
-      case Some(_) =>
+      case Some(partition) =>
         if (deletePartition) {
+          if (partition eq ReplicaManager.OfflinePartition)
+            throw new KafkaStorageException(s"Partition $topicPartition is on an offline disk")
           val removedPartition = allPartitions.remove(topicPartition)
           if (removedPartition != null) {
-            removedPartition.delete() // this will delete the local log
-            val topicHasPartitions = allPartitions.keys.exists(tp => topicPartition.topic == tp.topic)
+            val topicHasPartitions = allPartitions.values.exists(partition => topicPartition.topic == partition.topic)
             if (!topicHasPartitions)
               brokerTopicStats.removeMetrics(topicPartition.topic)
+            // this will delete the local log. This call may throw exception if the log is on offline directory
+            removedPartition.delete()
+          } else if (logManager.getLog(topicPartition).isDefined) {
+            // Delete log and corresponding folders in case replica manager doesn't hold them anymore.
+            // This could happen when topic is being deleted while broker is down and recovers.
+            logManager.asyncDelete(topicPartition)
           }
         }
       case None =>
@@ -317,8 +350,14 @@ class ReplicaManager(val config: KafkaConfig,
         // First stop fetchers for all partitions, then stop the corresponding replicas
         replicaFetcherManager.removeFetcherForPartitions(partitions)
         for (topicPartition <- partitions){
-          val error = stopReplica(topicPartition, stopReplicaRequest.deletePartitions)
-          responseMap.put(topicPartition, error)
+          try {
+            val error = stopReplica(topicPartition, stopReplicaRequest.deletePartitions)
+            responseMap.put(topicPartition, error)
+          } catch {
+            case e: KafkaStorageException =>
+              stateChangeLogger.error(s"Broker $localBrokerId ignoring stop replica (delete=${stopReplicaRequest.deletePartitions}) for partition $topicPartition due to storage exception", e)
+              responseMap.put(topicPartition, Errors.KAFKA_STORAGE_ERROR)
+          }
         }
         (responseMap, Errors.NONE)
       }
@@ -332,8 +371,15 @@ class ReplicaManager(val config: KafkaConfig,
     Option(allPartitions.get(topicPartition))
 
   def getReplicaOrException(topicPartition: TopicPartition): Replica = {
-    getReplica(topicPartition).getOrElse {
-      throw new ReplicaNotAvailableException(s"Replica $localBrokerId is not available for partition $topicPartition")
+    getPartition(topicPartition) match {
+      case Some(partition) =>
+        if (partition eq ReplicaManager.OfflinePartition)
+          throw new KafkaStorageException(s"Replica $localBrokerId is in an offline log directory for partition $topicPartition")
+        else
+          partition.getReplica(localBrokerId).getOrElse(
+            throw new ReplicaNotAvailableException(s"Replica $localBrokerId is not available for partition $topicPartition"))
+      case None =>
+        throw new ReplicaNotAvailableException(s"Replica $localBrokerId is not available for partition $topicPartition")
     }
   }
 
@@ -343,7 +389,9 @@ class ReplicaManager(val config: KafkaConfig,
       case None =>
         throw new UnknownTopicOrPartitionException(s"Partition $topicPartition doesn't exist on $localBrokerId")
       case Some(partition) =>
-        partition.leaderReplicaIfLocal match {
+        if (partition eq ReplicaManager.OfflinePartition)
+          throw new KafkaStorageException(s"Partition $topicPartition is in an offline log directory on broker $localBrokerId")
+        else partition.leaderReplicaIfLocal match {
           case Some(leaderReplica) => leaderReplica
           case None =>
             throw new NotLeaderForPartitionException(s"Leader not local for partition $topicPartition on broker $localBrokerId")
@@ -352,10 +400,17 @@ class ReplicaManager(val config: KafkaConfig,
   }
 
   def getReplica(topicPartition: TopicPartition, replicaId: Int): Option[Replica] =
-    getPartition(topicPartition).flatMap(_.getReplica(replicaId))
+    getPartition(topicPartition).filter(_ ne ReplicaManager.OfflinePartition).flatMap(_.getReplica(replicaId))
 
   def getReplica(tp: TopicPartition): Option[Replica] = getReplica(tp, localBrokerId)
 
+  def getLogDir(topicPartition: TopicPartition): Option[String] = {
+    getReplica(topicPartition).flatMap(_.log) match {
+      case Some(log) => Some(log.dir.getParent)
+      case None => None
+    }
+  }
+
   /**
    * Append messages to leader replicas of the partition, and wait for them to be replicated to other replicas;
    * the callback function will be triggered either when timeout or the required acks are satisfied;
@@ -422,8 +477,14 @@ class ReplicaManager(val config: KafkaConfig,
         (topicPartition, LogDeleteRecordsResult(-1L, -1L, Some(new InvalidTopicException(s"Cannot delete records of internal topic ${topicPartition.topic}"))))
       } else {
         try {
-          val partition = getPartition(topicPartition).getOrElse(
-            throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d".format(topicPartition, localBrokerId)))
+          val partition = getPartition(topicPartition) match {
+            case Some(p) =>
+              if (p eq ReplicaManager.OfflinePartition)
+                throw new KafkaStorageException("Partition %s is in an offline log directory on broker %d".format(topicPartition, localBrokerId))
+              p
+            case None =>
+              throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d".format(topicPartition, localBrokerId))
+          }
           val convertedOffset =
             if (requestedOffset == DeleteRecordsRequest.HIGH_WATERMARK) {
               partition.leaderReplicaIfLocal match {
@@ -443,14 +504,11 @@ class ReplicaManager(val config: KafkaConfig,
         } catch {
           // NOTE: Failed produce requests metric is not incremented for known exceptions
           // it is supposed to indicate un-expected failures of a broker in handling a produce request
-          case e: KafkaStorageException =>
-            fatal("Halting due to unrecoverable I/O error while handling DeleteRecordsRequest: ", e)
-            Runtime.getRuntime.halt(1)
-            (topicPartition, null)
           case e@ (_: UnknownTopicOrPartitionException |
                    _: NotLeaderForPartitionException |
                    _: OffsetOutOfRangeException |
                    _: PolicyViolationException |
+                   _: KafkaStorageException |
                    _: NotEnoughReplicasException) =>
             (topicPartition, LogDeleteRecordsResult(-1L, -1L, Some(e)))
           case t: Throwable =>
@@ -543,6 +601,8 @@ class ReplicaManager(val config: KafkaConfig,
           val partitionOpt = getPartition(topicPartition)
           val info = partitionOpt match {
             case Some(partition) =>
+              if (partition eq ReplicaManager.OfflinePartition)
+                throw new KafkaStorageException(s"Partition $topicPartition is in an offline log directory on broker $localBrokerId")
               partition.appendRecordsToLeader(records, isFromClient, requiredAcks)
 
             case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d"
@@ -567,15 +627,12 @@ class ReplicaManager(val config: KafkaConfig,
         } catch {
           // NOTE: Failed produce requests metric is not incremented for known exceptions
           // it is supposed to indicate un-expected failures of a broker in handling a produce request
-          case e: KafkaStorageException =>
-            fatal("Halting due to unrecoverable I/O error while handling produce request: ", e)
-            Exit.halt(1)
-            (topicPartition, null)
           case e@ (_: UnknownTopicOrPartitionException |
                    _: NotLeaderForPartitionException |
                    _: RecordTooLargeException |
                    _: RecordBatchTooLargeException |
                    _: CorruptRecordException |
+                   _: KafkaStorageException |
                    _: InvalidTimestampException) =>
             (topicPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(e)))
           case t: Throwable =>
@@ -747,6 +804,7 @@ class ReplicaManager(val config: KafkaConfig,
         case e@ (_: UnknownTopicOrPartitionException |
                  _: NotLeaderForPartitionException |
                  _: ReplicaNotAvailableException |
+                 _: KafkaStorageException |
                  _: OffsetOutOfRangeException) =>
           LogReadResult(info = FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY),
                         highWatermark = -1L,
@@ -793,7 +851,7 @@ class ReplicaManager(val config: KafkaConfig,
    *  the quota is exceeded and the replica is not in sync.
    */
   def shouldLeaderThrottle(quota: ReplicaQuota, topicPartition: TopicPartition, replicaId: Int): Boolean = {
-    val isReplicaInSync = getPartition(topicPartition).flatMap { partition =>
+    val isReplicaInSync = getPartition(topicPartition).filter(_ ne ReplicaManager.OfflinePartition).flatMap { partition =>
       partition.getReplica(replicaId).map(partition.inSyncReplicas.contains)
     }.getOrElse(false)
     quota.isThrottled(topicPartition) && quota.isQuotaExceeded && !isReplicaInSync
@@ -819,7 +877,8 @@ class ReplicaManager(val config: KafkaConfig,
     }
   }
 
-  def becomeLeaderOrFollower(correlationId: Int,leaderAndISRRequest: LeaderAndIsrRequest,
+  def becomeLeaderOrFollower(correlationId: Int,
+                             leaderAndISRRequest: LeaderAndIsrRequest,
                              onLeadershipChange: (Iterable[Partition], Iterable[Partition]) => Unit): BecomeLeaderOrFollowerResult = {
     leaderAndISRRequest.partitionStates.asScala.foreach { case (topicPartition, stateInfo) =>
       stateChangeLogger.trace("Broker %d received LeaderAndIsr request %s correlation id %d from controller %d epoch %d for partition [%s,%d]"
@@ -842,9 +901,14 @@ class ReplicaManager(val config: KafkaConfig,
         leaderAndISRRequest.partitionStates.asScala.foreach { case (topicPartition, stateInfo) =>
           val partition = getOrCreatePartition(topicPartition)
           val partitionLeaderEpoch = partition.getLeaderEpoch
-          // If the leader epoch is valid record the epoch of the controller that made the leadership decision.
-          // This is useful while updating the isr to maintain the decision maker controller's epoch in the zookeeper path
-          if (partitionLeaderEpoch < stateInfo.leaderEpoch) {
+          if (partition eq ReplicaManager.OfflinePartition) {
+            stateChangeLogger.warn(("Broker %d ignoring LeaderAndIsr request from controller %d with correlation id %d " +
+              "epoch %d for partition [%s,%d] as the local replica for the partition is in an offline log directory")
+              .format(localBrokerId, controllerId, correlationId, leaderAndISRRequest.controllerEpoch, topicPartition.topic, topicPartition.partition))
+            responseMap.put(topicPartition, Errors.KAFKA_STORAGE_ERROR)
+          } else if (partitionLeaderEpoch < stateInfo.leaderEpoch) {
+            // If the leader epoch is valid record the epoch of the controller that made the leadership decision.
+            // This is useful while updating the isr to maintain the decision maker controller's epoch in the zookeeper path
             if(stateInfo.replicas.contains(localBrokerId))
               partitionState.put(partition, stateInfo)
             else {
@@ -878,6 +942,17 @@ class ReplicaManager(val config: KafkaConfig,
         else
           Set.empty[Partition]
 
+        leaderAndISRRequest.partitionStates.asScala.keys.foreach( topicPartition =>
+          /*
+           * If there is offline log directory, a Partition object may have been created by getOrCreatePartition()
+           * before getOrCreateReplica() failed to create local replica due to KafkaStorageException.
+           * In this case ReplicaManager.allPartitions will map this topic-partition to an empty Partition object.
+           * we need to map this topic-partition to OfflinePartition instead.
+           */
+          if (getReplica(topicPartition).isEmpty && (allPartitions.get(topicPartition) ne ReplicaManager.OfflinePartition))
+            allPartitions.put(topicPartition, ReplicaManager.OfflinePartition)
+        )
+
         // we initialize highwatermark thread after the first leaderisrrequest. This ensures that all the partitions
         // have been completely populated before starting the checkpointing there by avoiding weird race conditions
         if (!hwThreadInitialized) {
@@ -885,7 +960,6 @@ class ReplicaManager(val config: KafkaConfig,
           hwThreadInitialized = true
         }
         replicaFetcherManager.shutdownIdleFetcherThreads()
-
         onLeadershipChange(partitionsBecomeLeader, partitionsBecomeFollower)
         BecomeLeaderOrFollowerResult(responseMap, Errors.NONE)
       }
@@ -926,18 +1000,27 @@ class ReplicaManager(val config: KafkaConfig,
       replicaFetcherManager.removeFetcherForPartitions(partitionState.keySet.map(_.topicPartition))
       // Update the partition information to be the leader
       partitionState.foreach{ case (partition, partitionStateInfo) =>
-        if (partition.makeLeader(controllerId, partitionStateInfo, correlationId))
-          partitionsToMakeLeaders += partition
-        else
-          stateChangeLogger.info(("Broker %d skipped the become-leader state change after marking its partition as leader with correlation id %d from " +
-            "controller %d epoch %d for partition %s since it is already the leader for the partition.")
-            .format(localBrokerId, correlationId, controllerId, epoch, partition.topicPartition))
-      }
-      partitionsToMakeLeaders.foreach { partition =>
-        stateChangeLogger.trace(("Broker %d stopped fetchers as part of become-leader request from controller " +
-          "%d epoch %d with correlation id %d for partition %s")
-          .format(localBrokerId, controllerId, epoch, correlationId, partition.topicPartition))
+        try {
+          if (partition.makeLeader(controllerId, partitionStateInfo, correlationId)) {
+            partitionsToMakeLeaders += partition
+            stateChangeLogger.trace(("Broker %d stopped fetchers as part of become-leader request from controller " +
+              "%d epoch %d with correlation id %d for partition %s")
+              .format(localBrokerId, controllerId, epoch, correlationId, partition.topicPartition))
+          } else
+            stateChangeLogger.info(("Broker %d skipped the become-leader state change after marking its partition as leader with correlation id %d from " +
+              "controller %d epoch %d for partition %s since it is already the leader for the partition.")
+              .format(localBrokerId, correlationId, controllerId, epoch, partition.topicPartition))
+        } catch {
+          case e: KafkaStorageException =>
+            stateChangeLogger.error(("Broker %d skipped the become-leader state change with correlation id %d from " +
+              "controller %d epoch %d for partition %s since the replica for the partition is offline due to disk error %s.")
+              .format(localBrokerId, correlationId, controllerId, epoch, partition.topicPartition, e))
+            val dirOpt = getLogDir(new TopicPartition(partition.topic, partition.partitionId))
+            error(s"Error while making broker the leader for partition $partition in dir $dirOpt", e)
+            responseMap.put(new TopicPartition(partition.topic, partition.partitionId), Errors.KAFKA_STORAGE_ERROR)
+        }
       }
+
     } catch {
       case e: Throwable =>
         partitionState.keys.foreach { partition =>
@@ -996,27 +1079,37 @@ class ReplicaManager(val config: KafkaConfig,
 
       // TODO: Delete leaders from LeaderAndIsrRequest
       partitionState.foreach{ case (partition, partitionStateInfo) =>
-        val newLeaderBrokerId = partitionStateInfo.leader
-        metadataCache.getAliveBrokers.find(_.id == newLeaderBrokerId) match {
-          // Only change partition state when the leader is available
-          case Some(_) =>
-            if (partition.makeFollower(controllerId, partitionStateInfo, correlationId))
-              partitionsToMakeFollower += partition
-            else
-              stateChangeLogger.info(("Broker %d skipped the become-follower state change after marking its partition as follower with correlation id %d from " +
-                "controller %d epoch %d for partition %s since the new leader %d is the same as the old leader")
+        try {
+          val newLeaderBrokerId = partitionStateInfo.leader
+          metadataCache.getAliveBrokers.find(_.id == newLeaderBrokerId) match {
+            // Only change partition state when the leader is available
+            case Some(_) =>
+              if (partition.makeFollower(controllerId, partitionStateInfo, correlationId))
+                partitionsToMakeFollower += partition
+              else
+                stateChangeLogger.info(("Broker %d skipped the become-follower state change after marking its partition as follower with correlation id %d from " +
+                  "controller %d epoch %d for partition %s since the new leader %d is the same as the old leader")
+                  .format(localBrokerId, correlationId, controllerId, partitionStateInfo.controllerEpoch,
+                    partition.topicPartition, newLeaderBrokerId))
+            case None =>
+              // The leader broker should always be present in the metadata cache.
+              // If not, we should record the error message and abort the transition process for this partition
+              stateChangeLogger.error(("Broker %d received LeaderAndIsrRequest with correlation id %d from controller" +
+                " %d epoch %d for partition %s but cannot become follower since the new leader %d is unavailable.")
                 .format(localBrokerId, correlationId, controllerId, partitionStateInfo.controllerEpoch,
-                partition.topicPartition, newLeaderBrokerId))
-          case None =>
-            // The leader broker should always be present in the metadata cache.
-            // If not, we should record the error message and abort the transition process for this partition
-            stateChangeLogger.error(("Broker %d received LeaderAndIsrRequest with correlation id %d from controller" +
-              " %d epoch %d for partition %s but cannot become follower since the new leader %d is unavailable.")
-              .format(localBrokerId, correlationId, controllerId, partitionStateInfo.controllerEpoch,
-              partition.topicPartition, newLeaderBrokerId))
-            // Create the local replica even if the leader is unavailable. This is required to ensure that we include
-            // the partition's high watermark in the checkpoint file (see KAFKA-1647)
-            partition.getOrCreateReplica()
+                  partition.topicPartition, newLeaderBrokerId))
+              // Create the local replica even if the leader is unavailable. This is required to ensure that we include
+              // the partition's high watermark in the checkpoint file (see KAFKA-1647)
+              partition.getOrCreateReplica(isNew = partitionStateInfo.isNew)
+          }
+        } catch {
+          case e: KafkaStorageException =>
+            stateChangeLogger.error(("Broker %d skipped the become-follower state change with correlation id %d from " +
+              "controller %d epoch %d for partition [%s,%d] since the replica for the partition is offline due to disk error %s")
+              .format(localBrokerId, correlationId, controllerId, partitionStateInfo.controllerEpoch, partition.topic, partition.partitionId, e))
+            val dirOpt = getLogDir(new TopicPartition(partition.topic, partition.partitionId))
+            error(s"Error while making broker the follower for partition $partition in dir $dirOpt", e)
+            responseMap.put(new TopicPartition(partition.topic, partition.partitionId), Errors.KAFKA_STORAGE_ERROR)
         }
       }
 
@@ -1080,7 +1173,7 @@ class ReplicaManager(val config: KafkaConfig,
 
   private def maybeShrinkIsr(): Unit = {
     trace("Evaluating ISR list of partitions to see which replicas can be removed from the ISR")
-    allPartitions.values.foreach(partition => partition.maybeShrinkIsr(config.replicaLagTimeMaxMs))
+    allPartitions.values.filter(_ ne ReplicaManager.OfflinePartition).foreach(partition => partition.maybeShrinkIsr(config.replicaLagTimeMaxMs))
   }
 
   private def updateFollowerLogReadResults(replicaId: Int, readResults: Seq[(TopicPartition, LogReadResult)]) {
@@ -1088,7 +1181,8 @@ class ReplicaManager(val config: KafkaConfig,
     readResults.foreach { case (topicPartition, readResult) =>
       getPartition(topicPartition) match {
         case Some(partition) =>
-          partition.updateReplicaLogReadResult(replicaId, readResult)
+          if (partition ne ReplicaManager.OfflinePartition)
+            partition.updateReplicaLogReadResult(replicaId, readResult)
 
           // for producer requests with ack > 1, we need to check
           // if they can be unblocked after some follower's log end offsets have moved
@@ -1100,33 +1194,86 @@ class ReplicaManager(val config: KafkaConfig,
   }
 
   private def getLeaderPartitions: List[Partition] =
-    allPartitions.values.filter(_.leaderReplicaIfLocal.isDefined).toList
+    allPartitions.values.filter(partition => (partition ne ReplicaManager.OfflinePartition) && partition.leaderReplicaIfLocal.isDefined).toList
 
   def getLogEndOffset(topicPartition: TopicPartition): Option[Long] = {
-    getPartition(topicPartition).flatMap{ partition =>
-      partition.leaderReplicaIfLocal.map(_.logEndOffset.messageOffset)
+    getPartition(topicPartition) match {
+      case Some(partition) =>
+        if (partition eq ReplicaManager.OfflinePartition)
+          None
+        else
+          partition.leaderReplicaIfLocal.map(_.logEndOffset.messageOffset)
+      case None => None
     }
   }
 
   // Flushes the highwatermark value for all partitions to the highwatermark file
   def checkpointHighWatermarks() {
-    val replicas = allPartitions.values.flatMap(_.getReplica(localBrokerId))
-    val replicasByDir = replicas.filter(_.log.isDefined).groupBy(_.log.get.dir.getParentFile.getAbsolutePath)
+    val replicas = allPartitions.values.filter(_ ne ReplicaManager.OfflinePartition).flatMap(_.getReplica(localBrokerId))
+    val replicasByDir = replicas.filter(_.log.isDefined).groupBy(_.log.get.dir.getParent)
     for ((dir, reps) <- replicasByDir) {
       val hwms = reps.map(r => r.partition.topicPartition -> r.highWatermark.messageOffset).toMap
       try {
-        highWatermarkCheckpoints(dir).write(hwms)
+        highWatermarkCheckpoints.get(dir).foreach(_.write(hwms))
       } catch {
-        case e: IOException =>
-          fatal("Error writing to highwatermark file: ", e)
-          Exit.halt(1)
+        case e: KafkaStorageException =>
+          error(s"Error while writing to highwatermark file in directory $dir", e)
       }
     }
   }
 
+  def handleLogDirFailure(dir: String) {
+    if (!logManager.isLogDirOnline(dir))
+      return
+
+    info(s"Stopping serving replicas in dir $dir")
+    replicaStateChangeLock synchronized {
+      val newOfflinePartitions = allPartitions.values.filter { partition =>
+        if (partition eq ReplicaManager.OfflinePartition)
+          false
+        else partition.getReplica(config.brokerId) match {
+          case Some(replica) =>
+            replica.log.isDefined && replica.log.get.dir.getParent == dir
+          case None => false
+        }
+      }.map(_.topicPartition)
+
+      info(s"Partitions ${newOfflinePartitions.mkString(",")} are offline due to failure on log directory $dir")
+
+      newOfflinePartitions.foreach { topicPartition =>
+        val partition = allPartitions.put(topicPartition, ReplicaManager.OfflinePartition)
+        partition.removePartitionMetrics()
+      }
+
+      newOfflinePartitions.map(_.topic).toSet.foreach { topic: String =>
+        val topicHasPartitions = allPartitions.values.exists(partition => topic == partition.topic)
+        if (!topicHasPartitions)
+          brokerTopicStats.removeMetrics(topic)
+      }
+
+      replicaFetcherManager.removeFetcherForPartitions(newOfflinePartitions.toSet)
+      highWatermarkCheckpoints = highWatermarkCheckpoints.filterKeys(_ != dir)
+      info("Broker %d stopped fetcher for partitions %s because they are in the failed log dir %s"
+        .format(localBrokerId, newOfflinePartitions.mkString(", "), dir))
+    }
+    logManager.handleLogDirFailure(dir)
+    LogDirUtils.propagateLogDirEvent(zkUtils, localBrokerId)
+    info(s"Stopped serving replicas in dir $dir")
+  }
+
+  def removeMetrics() {
+    removeMetric("LeaderCount")
+    removeMetric("PartitionCount")
+    removeMetric("OfflineReplicaCount")
+    removeMetric("UnderReplicatedPartitions")
+  }
+
   // High watermark do not need to be checkpointed only when under unit tests
   def shutdown(checkpointHW: Boolean = true) {
     info("Shutting down")
+    removeMetrics()
+    if (logDirFailureHandler != null)
+      logDirFailureHandler.shutdown()
     replicaFetcherManager.shutdown()
     delayedFetchPurgatory.shutdown()
     delayedProducePurgatory.shutdown()
@@ -1144,7 +1291,10 @@ class ReplicaManager(val config: KafkaConfig,
     requestedEpochInfo.map { case (tp, leaderEpoch) =>
       val epochEndOffset = getPartition(tp) match {
         case Some(partition) =>
-          partition.lastOffsetForLeaderEpoch(leaderEpoch)
+          if (partition eq ReplicaManager.OfflinePartition)
+            new EpochEndOffset(KAFKA_STORAGE_ERROR, UNDEFINED_EPOCH_OFFSET)
+          else
+            partition.lastOffsetForLeaderEpoch(leaderEpoch)
         case None =>
           new EpochEndOffset(UNKNOWN_TOPIC_OR_PARTITION, UNDEFINED_EPOCH_OFFSET)
       }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/server/checkpoints/CheckpointFile.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/checkpoints/CheckpointFile.scala b/core/src/main/scala/kafka/server/checkpoints/CheckpointFile.scala
index cc50620..7b67559 100644
--- a/core/src/main/scala/kafka/server/checkpoints/CheckpointFile.scala
+++ b/core/src/main/scala/kafka/server/checkpoints/CheckpointFile.scala
@@ -18,9 +18,13 @@ package kafka.server.checkpoints
 
 import java.io._
 import java.nio.charset.StandardCharsets
-import java.nio.file.{FileAlreadyExistsException, FileSystems, Files, Paths}
-import kafka.utils.{Exit, Logging}
+import java.nio.file.{FileAlreadyExistsException, Files, Paths}
+
+import kafka.server.LogDirFailureChannel
+import kafka.utils.Logging
+import org.apache.kafka.common.errors.KafkaStorageException
 import org.apache.kafka.common.utils.Utils
+
 import scala.collection.{Seq, mutable}
 
 trait CheckpointFileFormatter[T]{
@@ -29,86 +33,94 @@ trait CheckpointFileFormatter[T]{
   def fromLine(line: String): Option[T]
 }
 
-class CheckpointFile[T](val file: File, version: Int, formatter: CheckpointFileFormatter[T]) extends Logging {
+class CheckpointFile[T](val file: File,
+                        version: Int,
+                        formatter: CheckpointFileFormatter[T],
+                        logDirFailureChannel: LogDirFailureChannel,
+                        logDir: String) extends Logging {
   private val path = file.toPath.toAbsolutePath
   private val tempPath = Paths.get(path.toString + ".tmp")
   private val lock = new Object()
-  
+
   try Files.createFile(file.toPath) // create the file if it doesn't exist
   catch { case _: FileAlreadyExistsException => }
 
   def write(entries: Seq[T]) {
     lock synchronized {
-      // write to temp file and then swap with the existing file
-      val fileOutputStream = new FileOutputStream(tempPath.toFile)
-      val writer = new BufferedWriter(new OutputStreamWriter(fileOutputStream, StandardCharsets.UTF_8))
       try {
-        writer.write(version.toString)
-        writer.newLine()
-
-        writer.write(entries.size.toString)
-        writer.newLine()
+        // write to temp file and then swap with the existing file
+        val fileOutputStream = new FileOutputStream(tempPath.toFile)
+        val writer = new BufferedWriter(new OutputStreamWriter(fileOutputStream, StandardCharsets.UTF_8))
+        try {
+          writer.write(version.toString)
+          writer.newLine()
 
-        entries.foreach { entry =>
-          writer.write(formatter.toLine(entry))
+          writer.write(entries.size.toString)
           writer.newLine()
+
+          entries.foreach { entry =>
+            writer.write(formatter.toLine(entry))
+            writer.newLine()
+          }
+
+          writer.flush()
+          fileOutputStream.getFD().sync()
+        } finally {
+          writer.close()
         }
 
-        writer.flush()
-        fileOutputStream.getFD().sync()
+        Utils.atomicMoveWithFallback(tempPath, path)
       } catch {
-        case e: FileNotFoundException =>
-          if (FileSystems.getDefault.isReadOnly) {
-            fatal(s"Halting writes to checkpoint file (${file.getAbsolutePath}) because the underlying file system is inaccessible: ", e)
-            Exit.halt(1)
-          }
-          throw e
-      } finally {
-        writer.close()
+        case e: IOException =>
+          logDirFailureChannel.maybeAddLogFailureEvent(logDir)
+          throw new KafkaStorageException(s"Error while writing to checkpoint file ${file.getAbsolutePath}", e)
       }
-
-      Utils.atomicMoveWithFallback(tempPath, path)
     }
   }
 
   def read(): Seq[T] = {
     def malformedLineException(line: String) =
       new IOException(s"Malformed line in checkpoint file (${file.getAbsolutePath}): $line'")
-
     lock synchronized {
-      val reader = new BufferedReader(new InputStreamReader(new FileInputStream(file), StandardCharsets.UTF_8))
-      var line: String = null
       try {
-        line = reader.readLine()
-        if (line == null)
-          return Seq.empty
-        line.toInt match {
-          case fileVersion if fileVersion == version =>
-            line = reader.readLine()
-            if (line == null)
-              return Seq.empty
-            val expectedSize = line.toInt
-            val entries = mutable.Buffer[T]()
-            line = reader.readLine()
-            while (line != null) {
-              val entry = formatter.fromLine(line)
-              entry match {
-                case Some(e) =>
-                  entries += e
-                  line = reader.readLine()
-                case _ => throw malformedLineException(line)
+        val reader = new BufferedReader(new InputStreamReader(new FileInputStream(file), StandardCharsets.UTF_8))
+        var line: String = null
+        try {
+          line = reader.readLine()
+          if (line == null)
+            return Seq.empty
+          line.toInt match {
+            case fileVersion if fileVersion == version =>
+              line = reader.readLine()
+              if (line == null)
+                return Seq.empty
+              val expectedSize = line.toInt
+              val entries = mutable.Buffer[T]()
+              line = reader.readLine()
+              while (line != null) {
+                val entry = formatter.fromLine(line)
+                entry match {
+                  case Some(e) =>
+                    entries += e
+                    line = reader.readLine()
+                  case _ => throw malformedLineException(line)
+                }
               }
-            }
-            if (entries.size != expectedSize)
-              throw new IOException(s"Expected $expectedSize entries in checkpoint file (${file.getAbsolutePath}), but found only ${entries.size}")
-            entries
-          case _ =>
-            throw new IOException(s"Unrecognized version of the checkpoint file (${file.getAbsolutePath}): " + version)
+              if (entries.size != expectedSize)
+                throw new IOException(s"Expected $expectedSize entries in checkpoint file (${file.getAbsolutePath}), but found only ${entries.size}")
+              entries
+            case _ =>
+              throw new IOException(s"Unrecognized version of the checkpoint file (${file.getAbsolutePath}): " + version)
+          }
+        } catch {
+          case _: NumberFormatException => throw malformedLineException(line)
+        } finally {
+          reader.close()
         }
       } catch {
-        case _: NumberFormatException => throw malformedLineException(line)
-      } finally {
-        reader.close()
+        case e: IOException =>
+          logDirFailureChannel.maybeAddLogFailureEvent(logDir)
+          throw new KafkaStorageException(s"Error while reading checkpoint file ${file.getAbsolutePath}", e)
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/server/checkpoints/LeaderEpochCheckpointFile.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/checkpoints/LeaderEpochCheckpointFile.scala b/core/src/main/scala/kafka/server/checkpoints/LeaderEpochCheckpointFile.scala
index d32d30f..a8db688 100644
--- a/core/src/main/scala/kafka/server/checkpoints/LeaderEpochCheckpointFile.scala
+++ b/core/src/main/scala/kafka/server/checkpoints/LeaderEpochCheckpointFile.scala
@@ -19,6 +19,7 @@ package kafka.server.checkpoints
 import java.io._
 import java.util.regex.Pattern
 
+import kafka.server.LogDirFailureChannel
 import kafka.server.epoch.EpochEntry
 
 import scala.collection._
@@ -55,10 +56,10 @@ object LeaderEpochCheckpointFile {
 /**
   * This class persists a map of (LeaderEpoch => Offsets) to a file (for a certain replica)
   */
-class LeaderEpochCheckpointFile(val file: File) extends LeaderEpochCheckpoint {
+class LeaderEpochCheckpointFile(val file: File, logDirFailureChannel: LogDirFailureChannel = null) extends LeaderEpochCheckpoint {
   import LeaderEpochCheckpointFile._
 
-  val checkpoint = new CheckpointFile[EpochEntry](file, CurrentVersion, Formatter)
+  val checkpoint = new CheckpointFile[EpochEntry](file, CurrentVersion, Formatter, logDirFailureChannel, file.getParentFile.getParent)
 
   def write(epochs: Seq[EpochEntry]): Unit = checkpoint.write(epochs)
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala b/core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala
index 5f5dc97..9cd0963 100644
--- a/core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala
+++ b/core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala
@@ -19,6 +19,7 @@ package kafka.server.checkpoints
 import java.io._
 import java.util.regex.Pattern
 
+import kafka.server.LogDirFailureChannel
 import kafka.server.epoch.EpochEntry
 import org.apache.kafka.common.TopicPartition
 
@@ -51,9 +52,9 @@ trait OffsetCheckpoint {
 /**
   * This class persists a map of (Partition => Offsets) to a file (for a certain replica)
   */
-class OffsetCheckpointFile(val f: File) {
+class OffsetCheckpointFile(val f: File, logDirFailureChannel: LogDirFailureChannel = null) {
   val checkpoint = new CheckpointFile[(TopicPartition, Long)](f, OffsetCheckpointFile.CurrentVersion,
-    OffsetCheckpointFile.Formatter)
+    OffsetCheckpointFile.Formatter, logDirFailureChannel, f.getParent)
 
   def write(offsets: Map[TopicPartition, Long]): Unit = checkpoint.write(offsets.toSeq)
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/utils/LogDirUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/utils/LogDirUtils.scala b/core/src/main/scala/kafka/utils/LogDirUtils.scala
new file mode 100644
index 0000000..0bbc47d
--- /dev/null
+++ b/core/src/main/scala/kafka/utils/LogDirUtils.scala
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.utils
+
+import kafka.controller.LogDirEventNotificationListener
+import scala.collection.Map
+
+object LogDirUtils extends Logging {
+
+  private val LogDirEventNotificationPrefix = "log_dir_event_"
+  val LogDirFailureEvent = 1
+
+  def propagateLogDirEvent(zkUtils: ZkUtils, brokerId: Int) {
+    val logDirEventNotificationPath: String = zkUtils.createSequentialPersistentPath(
+      ZkUtils.LogDirEventNotificationPath + "/" + LogDirEventNotificationPrefix, logDirFailureEventZkData(brokerId))
+    debug("Added " + logDirEventNotificationPath + " for broker " + brokerId)
+  }
+
+  private def logDirFailureEventZkData(brokerId: Int): String = {
+    Json.encode(Map("version" -> LogDirEventNotificationListener.version, "broker" -> brokerId, "event" -> LogDirFailureEvent))
+  }
+
+  def deleteLogDirEvents(zkUtils: ZkUtils) {
+    val sequenceNumbers = zkUtils.getChildrenParentMayNotExist(ZkUtils.LogDirEventNotificationPath).toSet
+    sequenceNumbers.map(x => zkUtils.deletePath(ZkUtils.LogDirEventNotificationPath + "/" + x))
+  }
+
+  def getBrokerIdFromLogDirEvent(zkUtils: ZkUtils, child: String): Option[Int] = {
+    val changeZnode = ZkUtils.LogDirEventNotificationPath + "/" + child
+    val (jsonOpt, stat) = zkUtils.readDataMaybeNull(changeZnode)
+    if (jsonOpt.isDefined) {
+      val json = Json.parseFull(jsonOpt.get)
+
+      json match {
+        case Some(m) =>
+          val brokerAndEventType = m.asInstanceOf[Map[String, Any]]
+          val brokerId = brokerAndEventType.get("broker").get.asInstanceOf[Int]
+          val eventType = brokerAndEventType.get("event").get.asInstanceOf[Int]
+          if (eventType != LogDirFailureEvent)
+            throw new IllegalArgumentException(s"The event type $eventType in znode $changeZnode is not recognized")
+          Some(brokerId)
+        case None =>
+          error("Invalid LogDirEvent JSON: " + jsonOpt.get + " in ZK: " + changeZnode)
+          None
+      }
+    } else {
+      None
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/utils/ZkUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala
index 0035120..7d3529f 100644
--- a/core/src/main/scala/kafka/utils/ZkUtils.scala
+++ b/core/src/main/scala/kafka/utils/ZkUtils.scala
@@ -51,6 +51,7 @@ object ZkUtils {
   val ControllerPath = "/controller"
   val ControllerEpochPath = "/controller_epoch"
   val IsrChangeNotificationPath = "/isr_change_notification"
+  val LogDirEventNotificationPath = "/log_dir_event_notification"
   val KafkaAclPath = "/kafka-acl"
   val KafkaAclChangesPath = "/kafka-acl-changes"
 
@@ -75,7 +76,8 @@ object ZkUtils {
                               IsrChangeNotificationPath,
                               KafkaAclPath,
                               KafkaAclChangesPath,
-                              ProducerIdBlockPath)
+                              ProducerIdBlockPath,
+                              LogDirEventNotificationPath)
 
   // Important: it is necessary to add any new top level Zookeeper path that contains
   //            sensitive information that should not be world readable to the Seq
@@ -235,7 +237,8 @@ class ZkUtils(val zkClient: ZkClient,
                               DeleteTopicsPath,
                               BrokerSequenceIdPath,
                               IsrChangeNotificationPath,
-                              ProducerIdBlockPath)
+                              ProducerIdBlockPath,
+                              LogDirEventNotificationPath)
 
   // Visible for testing
   val zkPath = new ZkPath(zkClient)

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
index 09ff9be..2b134fe 100644
--- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
@@ -44,7 +44,6 @@ import org.apache.kafka.common.security.auth.KafkaPrincipal
 import org.apache.kafka.common.{Node, TopicPartition, requests}
 import org.junit.Assert._
 import org.junit.{After, Assert, Before, Test}
-
 import scala.collection.JavaConverters._
 import scala.collection.mutable
 import scala.collection.mutable.Buffer
@@ -272,7 +271,8 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
   }
 
   private def createUpdateMetadataRequest = {
-    val partitionState = Map(tp -> new PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava)).asJava
+    val partitionState = Map(tp -> new UpdateMetadataRequest.PartitionState(
+      Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava, Seq.empty[Integer].asJava)).asJava
     val securityProtocol = SecurityProtocol.PLAINTEXT
     val brokers = Set(new requests.UpdateMetadataRequest.Broker(brokerId,
       Seq(new requests.UpdateMetadataRequest.EndPoint("localhost", 0, securityProtocol,
@@ -303,8 +303,8 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
   private def leaveGroupRequest = new LeaveGroupRequest.Builder(group, "").build()
 
   private def leaderAndIsrRequest = {
-    new requests.LeaderAndIsrRequest.Builder(brokerId, Int.MaxValue,
-      Map(tp -> new PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava)).asJava,
+    new requests.LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion, brokerId, Int.MaxValue,
+      Map(tp -> new PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Seq(brokerId).asJava, false)).asJava,
       Set(new Node(brokerId, "localhost", 0)).asJava).build()
   }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
index 921c2b4..5e3c7ab 100644
--- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
+++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
@@ -37,6 +37,7 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness {
   val producerCount: Int
   val consumerCount: Int
   val serverCount: Int
+  var logDirCount: Int = 1
   lazy val producerConfig = new Properties
   lazy val consumerConfig = new Properties
   lazy val serverConfig = new Properties
@@ -46,7 +47,7 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness {
 
   override def generateConfigs = {
     val cfgs = TestUtils.createBrokerConfigs(serverCount, zkConnect, interBrokerSecurityProtocol = Some(securityProtocol),
-      trustStoreFile = trustStoreFile, saslProperties = serverSaslProperties)
+      trustStoreFile = trustStoreFile, saslProperties = serverSaslProperties, logDirCount = logDirCount)
     cfgs.foreach { config =>
       config.setProperty(KafkaConfig.ListenersProp, s"${listenerName.value}://localhost:${TestUtils.RandomPort}")
       config.remove(KafkaConfig.InterBrokerSecurityProtocolProp)
@@ -84,7 +85,7 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness {
                                   saslProperties = this.clientSaslProperties,
                                   props = Some(producerConfig))
   }
-  
+
   def createNewConsumer: KafkaConsumer[Array[Byte], Array[Byte]] = {
       TestUtils.createNewConsumer(brokerList,
                                   securityProtocol = this.securityProtocol,

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/integration/kafka/api/LogDirFailureTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/LogDirFailureTest.scala b/core/src/test/scala/integration/kafka/api/LogDirFailureTest.scala
new file mode 100644
index 0000000..6942df0
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/api/LogDirFailureTest.scala
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.api
+
+import java.util.Collections
+import java.util.concurrent.{ExecutionException, TimeUnit}
+
+import kafka.controller.{OfflineReplica, PartitionAndReplica}
+import kafka.server.KafkaConfig
+import kafka.utils.{CoreUtils, TestUtils, ZkUtils}
+import org.apache.kafka.clients.consumer.KafkaConsumer
+import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord}
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.common.errors.{KafkaStorageException, NotLeaderForPartitionException}
+import org.junit.{Before, Test}
+import org.junit.Assert.assertTrue
+import org.junit.Assert.assertEquals
+
+/**
+  * Test whether clients can producer and consume when there is log directory failure
+  */
+class LogDirFailureTest extends IntegrationTestHarness {
+  val producerCount: Int = 1
+  val consumerCount: Int = 1
+  val serverCount: Int = 2
+  private val topic = "topic"
+
+  this.logDirCount = 2
+  this.producerConfig.setProperty(ProducerConfig.RETRIES_CONFIG, "0")
+  this.producerConfig.setProperty(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, "100")
+  this.serverConfig.setProperty(KafkaConfig.ReplicaHighWatermarkCheckpointIntervalMsProp, "100")
+
+  @Before
+  override def setUp() {
+    super.setUp()
+    TestUtils.createTopic(zkUtils, topic, 1, 2, servers = servers)
+  }
+
+  @Test
+  def testProduceAfterLogDirFailure() {
+
+    val consumer = consumers.head
+    subscribeAndWaitForAssignment(topic, consumer)
+    val producer = producers.head
+    val partition = new TopicPartition(topic, 0)
+    val record = new ProducerRecord(topic, 0, s"key".getBytes, s"value".getBytes)
+
+    val leaderServerId = producer.partitionsFor(topic).get(0).leader().id()
+    val leaderServer = servers.find(_.config.brokerId == leaderServerId).get
+
+    // The first send() should succeed
+    producer.send(record).get()
+    TestUtils.waitUntilTrue(() => {
+      consumer.poll(0).count() == 1
+    }, "Expected the first message", 3000L)
+
+    // Make log directory of the partition on the leader broker inaccessible by replacing it with a file
+    val replica = leaderServer.replicaManager.getReplica(partition)
+    val logDir = replica.get.log.get.dir.getParentFile
+    CoreUtils.swallow(Utils.delete(logDir))
+    logDir.createNewFile()
+    assertTrue(logDir.isFile)
+
+    // Wait for ReplicaHighWatermarkCheckpoint to happen so that the log directory of the topic will be offline
+    TestUtils.waitUntilTrue(() => !leaderServer.logManager.liveLogDirs.contains(logDir), "Expected log directory offline", 3000L)
+    assertTrue(leaderServer.replicaManager.getReplica(partition).isEmpty)
+
+    // The second send() should fail due to either KafkaStorageException or NotLeaderForPartitionException
+    try {
+      producer.send(record).get(6000, TimeUnit.MILLISECONDS)
+      fail("send() should fail with either KafkaStorageException or NotLeaderForPartitionException")
+    } catch {
+      case e: ExecutionException =>
+        e.getCause match {
+          case t: KafkaStorageException =>
+          case t: NotLeaderForPartitionException => // This may happen if ProduceRequest version <= 3
+          case t: Throwable => fail(s"send() should fail with either KafkaStorageException or NotLeaderForPartitionException instead of ${t.toString}")
+        }
+      case e: Throwable => fail(s"send() should fail with either KafkaStorageException or NotLeaderForPartitionException instead of ${e.toString}")
+    }
+
+    // Wait for producer to update metadata for the partition
+    TestUtils.waitUntilTrue(() => {
+      // ProduceResponse may contain KafkaStorageException and trigger metadata update
+      producer.send(record)
+      producer.partitionsFor(topic).get(0).leader().id() != leaderServerId
+    }, "Expected new leader for the partition", 6000L)
+
+    // Consumer should receive some messages
+    TestUtils.waitUntilTrue(() => {
+      consumer.poll(0).count() > 0
+    }, "Expected some messages", 3000L)
+
+    // There should be no remaining LogDirEventNotification znode
+    assertTrue(zkUtils.getChildrenParentMayNotExist(ZkUtils.LogDirEventNotificationPath).isEmpty)
+
+    // The controller should have marked the replica on the original leader as offline
+    val controllerServer = servers.find(_.kafkaController.isActive).get
+    val offlineReplicas = controllerServer.kafkaController.replicaStateMachine.replicasInState(topic, OfflineReplica)
+    assertTrue(offlineReplicas.contains(PartitionAndReplica(topic, 0, leaderServerId)))
+  }
+
+  private def subscribeAndWaitForAssignment(topic: String, consumer: KafkaConsumer[Array[Byte], Array[Byte]]) {
+    consumer.subscribe(Collections.singletonList(topic))
+    TestUtils.waitUntilTrue(() => {
+      consumer.poll(0)
+      !consumer.assignment.isEmpty
+    }, "Expected non-empty assignment")
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala
index 0e57e53..760cc39 100644
--- a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala
+++ b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala
@@ -389,7 +389,7 @@ class TransactionsTest extends KafkaServerTestHarness {
       val recordMetadata = result.get()
       error(s"Missed a producer fenced exception when writing to ${recordMetadata.topic}-${recordMetadata.partition}. Grab the logs!!")
       servers.foreach { server =>
-        error(s"log dirs: ${server.logManager.logDirs.map(_.getAbsolutePath).head}")
+        error(s"log dirs: ${server.logManager.liveLogDirs.map(_.getAbsolutePath).head}")
       }
       fail("Should not be able to send messages from a fenced producer.")
     } catch {
@@ -436,7 +436,7 @@ class TransactionsTest extends KafkaServerTestHarness {
       val recordMetadata = result.get()
       error(s"Missed a producer fenced exception when writing to ${recordMetadata.topic}-${recordMetadata.partition}. Grab the logs!!")
       servers.foreach { case (server) =>
-        error(s"log dirs: ${server.logManager.logDirs.map(_.getAbsolutePath).head}")
+        error(s"log dirs: ${server.logManager.liveLogDirs.map(_.getAbsolutePath).head}")
       }
       fail("Should not be able to send messages from a fenced producer.")
     } catch {

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/integration/kafka/server/ReplicaFetcherThreadFatalErrorTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/server/ReplicaFetcherThreadFatalErrorTest.scala b/core/src/test/scala/integration/kafka/server/ReplicaFetcherThreadFatalErrorTest.scala
index 147e84a..ebe7223 100644
--- a/core/src/test/scala/integration/kafka/server/ReplicaFetcherThreadFatalErrorTest.scala
+++ b/core/src/test/scala/integration/kafka/server/ReplicaFetcherThreadFatalErrorTest.scala
@@ -111,7 +111,7 @@ class ReplicaFetcherThreadFatalErrorTest extends ZooKeeperTestHarness {
 
       override def createReplicaManager(isShuttingDown: AtomicBoolean): ReplicaManager = {
         new ReplicaManager(config, metrics, time, zkUtils, kafkaScheduler, logManager, isShuttingDown,
-          quotaManagers.follower, new BrokerTopicStats, metadataCache) {
+          quotaManagers.follower, new BrokerTopicStats, metadataCache, logDirFailureChannel) {
 
           override protected def createReplicaFetcherManager(metrics: Metrics, time: Time, threadNamePrefix: Option[String],
                                                              quotaManager: ReplicationQuotaManager) =

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
index a8ce17e..24e2920 100644
--- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
@@ -97,7 +97,7 @@ class GroupMetadataManagerTest {
     expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
 
     EasyMock.replay(replicaManager)
-    
+
     groupMetadataManager.loadGroupsAndOffsets(groupMetadataTopicPartition, _ => ())
 
     val group = groupMetadataManager.getGroup(groupId).getOrElse(fail("Group was not loaded into the cache"))
@@ -685,9 +685,9 @@ class GroupMetadataManagerTest {
     assertStoreGroupErrorMapping(Errors.NOT_ENOUGH_REPLICAS, Errors.COORDINATOR_NOT_AVAILABLE)
     assertStoreGroupErrorMapping(Errors.NOT_ENOUGH_REPLICAS_AFTER_APPEND, Errors.COORDINATOR_NOT_AVAILABLE)
     assertStoreGroupErrorMapping(Errors.NOT_LEADER_FOR_PARTITION, Errors.NOT_COORDINATOR)
-    assertStoreGroupErrorMapping(Errors.MESSAGE_TOO_LARGE, Errors.UNKNOWN)
-    assertStoreGroupErrorMapping(Errors.RECORD_LIST_TOO_LARGE, Errors.UNKNOWN)
-    assertStoreGroupErrorMapping(Errors.INVALID_FETCH_SIZE, Errors.UNKNOWN)
+    assertStoreGroupErrorMapping(Errors.MESSAGE_TOO_LARGE, Errors.UNKNOWN_SERVER_ERROR)
+    assertStoreGroupErrorMapping(Errors.RECORD_LIST_TOO_LARGE, Errors.UNKNOWN_SERVER_ERROR)
+    assertStoreGroupErrorMapping(Errors.INVALID_FETCH_SIZE, Errors.UNKNOWN_SERVER_ERROR)
     assertStoreGroupErrorMapping(Errors.CORRUPT_MESSAGE, Errors.CORRUPT_MESSAGE)
   }
 
@@ -1311,7 +1311,7 @@ class GroupMetadataManagerTest {
     EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andStubReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE))
     capturedArgument
   }
-  
+
   private def expectAppendMessage(error: Errors) {
     val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
     EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandlerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandlerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandlerTest.scala
index df2f7df..6323d15 100644
--- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandlerTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandlerTest.scala
@@ -144,7 +144,7 @@ class TransactionMarkerRequestCompletionHandlerTest {
 
   @Test
   def shouldThrowIllegalStateExceptionWhenUnknownError(): Unit = {
-    verifyThrowIllegalStateExceptionOnError(Errors.UNKNOWN)
+    verifyThrowIllegalStateExceptionOnError(Errors.UNKNOWN_SERVER_ERROR)
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala
index 6a35d41..e86e088 100644
--- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala
@@ -309,7 +309,7 @@ class TransactionStateManagerTest {
     transactionManager.addLoadedTransactionsToCache(partitionId, coordinatorEpoch, new Pool[String, TransactionMetadata]())
     transactionManager.putTransactionStateIfNotExists(transactionalId1, txnMetadata1)
 
-    expectedError = Errors.UNKNOWN
+    expectedError = Errors.UNKNOWN_SERVER_ERROR
     var failedMetadata = txnMetadata1.prepareAddPartitions(Set[TopicPartition](new TopicPartition("topic2", 0)), time.milliseconds())
 
     prepareForTxnMessageAppend(Errors.MESSAGE_TOO_LARGE)

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala
index bf36199..d6f0a56 100644
--- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala
@@ -20,7 +20,7 @@ import java.io.File
 import java.nio.file.Files
 import java.util.Properties
 
-import kafka.server.BrokerTopicStats
+import kafka.server.{BrokerTopicStats, LogDirFailureChannel}
 import kafka.utils.{MockTime, Pool, TestUtils}
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.utils.Utils
@@ -110,6 +110,7 @@ abstract class AbstractLogCleanerIntegrationTest {
     new LogCleaner(cleanerConfig,
       logDirs = Array(logDir),
       logs = logMap,
+      logDirFailureChannel = new LogDirFailureChannel(1),
       time = time)
   }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
index 8a119c2..e569b29 100755
--- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
@@ -5,7 +5,7 @@
  * The ASF licenses this file to You under the Apache License, Version 2.0
  * (the "License"); you may not use this file except in compliance with
  * the License.  You may obtain a copy of the License at
- * 
+ *
  *    http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
index b4c1790..f4eabc0 100644
--- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
@@ -218,7 +218,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
   private def createCleanerManager(log: Log): LogCleanerManager = {
     val logs = new Pool[TopicPartition, Log]()
     logs.put(new TopicPartition("log", 0), log)
-    val cleanerManager = new LogCleanerManager(Array(logDir), logs)
+    val cleanerManager = new LogCleanerManager(Array(logDir), logs, null)
     cleanerManager
   }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
index 3e58c4d..689a032 100755
--- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
@@ -5,7 +5,7 @@
  * The ASF licenses this file to You under the Apache License, Version 2.0
  * (the "License"); you may not use this file except in compliance with
  * the License.  You may obtain a copy of the License at
- * 
+ *
  *    http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
@@ -881,7 +881,7 @@ class LogCleanerTest extends JUnitSuite {
     checkSegmentOrder(groups)
   }
 
-  /** 
+  /**
    * Following the loading of a log segment where the index file is zero sized,
    * the index returned would be the base offset.  Sometimes the log file would
    * contain data with offsets in excess of the baseOffset which would cause
@@ -1324,7 +1324,7 @@ class FakeOffsetMap(val slots: Int) extends OffsetMap {
     lastOffset = offset
     map.put(keyFor(key), offset)
   }
-  
+
   override def get(key: ByteBuffer): Long = {
     val k = keyFor(key)
     if(map.containsKey(k))
@@ -1332,9 +1332,9 @@ class FakeOffsetMap(val slots: Int) extends OffsetMap {
     else
       -1L
   }
-  
+
   override def clear(): Unit = map.clear()
-  
+
   override def size: Int = map.size
 
   override def latestOffset: Long = lastOffset

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
index 8b7819f..0826747 100755
--- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
@@ -21,12 +21,10 @@ import java.io._
 import java.util.Properties
 
 import kafka.common._
-import kafka.server.FetchDataInfo
 import kafka.server.checkpoints.OffsetCheckpointFile
 import kafka.utils._
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.errors.OffsetOutOfRangeException
-import org.apache.kafka.common.requests.IsolationLevel
 import org.apache.kafka.common.utils.Utils
 import org.junit.Assert._
 import org.junit.{After, Before, Test}
@@ -52,7 +50,7 @@ class LogManagerTest {
     logDir = TestUtils.tempDir()
     logManager = createLogManager()
     logManager.startup()
-    logDir = logManager.logDirs(0)
+    logDir = logManager.liveLogDirs(0)
   }
 
   @After
@@ -60,7 +58,7 @@ class LogManagerTest {
     if(logManager != null)
       logManager.shutdown()
     Utils.delete(logDir)
-    logManager.logDirs.foreach(Utils.delete)
+    logManager.liveLogDirs.foreach(Utils.delete)
   }
 
   /**
@@ -68,7 +66,7 @@ class LogManagerTest {
    */
   @Test
   def testCreateLog() {
-    val log = logManager.createLog(new TopicPartition(name, 0), logConfig)
+    val log = logManager.getOrCreateLog(new TopicPartition(name, 0), logConfig)
     val logFile = new File(logDir, name + "-0")
     assertTrue(logFile.exists)
     log.appendAsLeader(TestUtils.singletonRecords("test".getBytes()), leaderEpoch = 0)
@@ -90,7 +88,7 @@ class LogManagerTest {
    */
   @Test
   def testCleanupExpiredSegments() {
-    val log = logManager.createLog(new TopicPartition(name, 0), logConfig)
+    val log = logManager.getOrCreateLog(new TopicPartition(name, 0), logConfig)
     var offset = 0L
     for(_ <- 0 until 200) {
       val set = TestUtils.singletonRecords("test".getBytes())
@@ -135,7 +133,7 @@ class LogManagerTest {
     logManager.startup()
 
     // create a log
-    val log = logManager.createLog(new TopicPartition(name, 0), config)
+    val log = logManager.getOrCreateLog(new TopicPartition(name, 0), config)
     var offset = 0L
 
     // add a bunch of messages that should be larger than the retentionSize
@@ -175,7 +173,7 @@ class LogManagerTest {
   def testDoesntCleanLogsWithCompactDeletePolicy() {
     val logProps = new Properties()
     logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact + "," + LogConfig.Delete)
-    val log = logManager.createLog(new TopicPartition(name, 0), LogConfig.fromProps(logConfig.originals, logProps))
+    val log = logManager.getOrCreateLog(new TopicPartition(name, 0), LogConfig.fromProps(logConfig.originals, logProps))
     var offset = 0L
     for (_ <- 0 until 200) {
       val set = TestUtils.singletonRecords("test".getBytes(), key="test".getBytes())
@@ -204,7 +202,7 @@ class LogManagerTest {
 
     logManager = createLogManager()
     logManager.startup()
-    val log = logManager.createLog(new TopicPartition(name, 0), config)
+    val log = logManager.getOrCreateLog(new TopicPartition(name, 0), config)
     val lastFlush = log.lastFlushTime
     for (_ <- 0 until 200) {
       val set = TestUtils.singletonRecords("test".getBytes())
@@ -228,7 +226,7 @@ class LogManagerTest {
 
     // verify that logs are always assigned to the least loaded partition
     for(partition <- 0 until 20) {
-      logManager.createLog(new TopicPartition("test", partition), logConfig)
+      logManager.getOrCreateLog(new TopicPartition("test", partition), logConfig)
       assertEquals("We should have created the right number of logs", partition + 1, logManager.allLogs.size)
       val counts = logManager.allLogs.groupBy(_.dir.getParent).values.map(_.size)
       assertTrue("Load should balance evenly", counts.max <= counts.min + 1)
@@ -286,7 +284,7 @@ class LogManagerTest {
 
   private def verifyCheckpointRecovery(topicPartitions: Seq[TopicPartition],
                                        logManager: LogManager) {
-    val logs = topicPartitions.map(this.logManager.createLog(_, logConfig))
+    val logs = topicPartitions.map(this.logManager.getOrCreateLog(_, logConfig))
     logs.foreach(log => {
       for (_ <- 0 until 50)
         log.appendAsLeader(TestUtils.singletonRecords("test".getBytes()), leaderEpoch = 0)
@@ -294,7 +292,7 @@ class LogManagerTest {
       log.flush()
     })
 
-    logManager.checkpointRecoveryPointOffsets()
+    logManager.checkpointLogRecoveryOffsets()
     val checkpoints = new OffsetCheckpointFile(new File(logDir, logManager.RecoveryPointCheckpointFile)).read()
 
     topicPartitions.zip(logs).foreach {

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
index 79fe220..30ccc8b 100644
--- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
@@ -30,7 +30,7 @@ import scala.collection.JavaConverters._
 import scala.collection._
 
 class LogSegmentTest {
-  
+
   val topicPartition = new TopicPartition("topic", 0)
   val segments = mutable.ArrayBuffer[LogSegment]()
   var logDir: File = _
@@ -52,7 +52,7 @@ class LogSegmentTest {
     segments += seg
     seg
   }
-  
+
   /* create a ByteBufferMessageSet for the given messages starting from the given offset */
   def records(offset: Long, records: String*): MemoryRecords = {
     MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, offset, CompressionType.NONE, TimestampType.CREATE_TIME,

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/log/LogTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala
index 008cd27..2213d09 100755
--- a/core/src/test/scala/unit/kafka/log/LogTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogTest.scala
@@ -254,7 +254,8 @@ class LogTest {
       maxProducerIdExpirationMs = 300000,
       producerIdExpirationCheckIntervalMs = 30000,
       topicPartition = Log.parseTopicPartitionName(logDir),
-      stateManager)
+      producerStateManager = stateManager,
+      logDirFailureChannel = null)
 
     EasyMock.verify(stateManager)
 
@@ -322,7 +323,8 @@ class LogTest {
       maxProducerIdExpirationMs = 300000,
       producerIdExpirationCheckIntervalMs = 30000,
       topicPartition = Log.parseTopicPartitionName(logDir),
-      stateManager)
+      producerStateManager = stateManager,
+      logDirFailureChannel = null)
 
     EasyMock.verify(stateManager)
   }
@@ -356,7 +358,8 @@ class LogTest {
       maxProducerIdExpirationMs = 300000,
       producerIdExpirationCheckIntervalMs = 30000,
       topicPartition = Log.parseTopicPartitionName(logDir),
-      stateManager)
+      producerStateManager = stateManager,
+      logDirFailureChannel = null)
 
     EasyMock.verify(stateManager)
     cleanShutdownFile.delete()
@@ -391,7 +394,8 @@ class LogTest {
       maxProducerIdExpirationMs = 300000,
       producerIdExpirationCheckIntervalMs = 30000,
       topicPartition = Log.parseTopicPartitionName(logDir),
-      stateManager)
+      producerStateManager = stateManager,
+      logDirFailureChannel = null)
 
     EasyMock.verify(stateManager)
     cleanShutdownFile.delete()

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
index b6b40c2..5e63500 100755
--- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
+++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
@@ -24,7 +24,6 @@ import org.apache.kafka.common.utils.Utils
 import org.easymock.EasyMock
 import org.junit._
 import org.junit.Assert._
-import kafka.common._
 import kafka.cluster.Replica
 import kafka.utils.{KafkaScheduler, MockTime, TestUtils, ZkUtils}
 import java.util.concurrent.atomic.AtomicBoolean
@@ -35,24 +34,28 @@ class HighwatermarkPersistenceTest {
 
   val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps)
   val topic = "foo"
+  val zkUtils = EasyMock.createMock(classOf[ZkUtils])
   val logManagers = configs map { config =>
     TestUtils.createLogManager(
       logDirs = config.logDirs.map(new File(_)).toArray,
       cleanerConfig = CleanerConfig())
   }
-    
+
+  val logDirFailureChannels = configs map { config =>
+    new LogDirFailureChannel(config.logDirs.size)
+  }
+
   @After
   def teardown() {
-    for(manager <- logManagers; dir <- manager.logDirs)
+    for(manager <- logManagers; dir <- manager.liveLogDirs)
       Utils.delete(dir)
   }
 
   @Test
   def testHighWatermarkPersistenceSinglePartition() {
     // mock zkclient
-    val zkUtils = EasyMock.createMock(classOf[ZkUtils])
     EasyMock.replay(zkUtils)
-    
+
     // create kafka scheduler
     val scheduler = new KafkaScheduler(2)
     scheduler.startup
@@ -61,7 +64,7 @@ class HighwatermarkPersistenceTest {
     // create replica manager
     val replicaManager = new ReplicaManager(configs.head, metrics, time, zkUtils, scheduler,
       logManagers.head, new AtomicBoolean(false), QuotaFactory.instantiate(configs.head, metrics, time).follower,
-      new BrokerTopicStats, new MetadataCache(configs.head.brokerId))
+      new BrokerTopicStats, new MetadataCache(configs.head.brokerId), logDirFailureChannels.head)
     replicaManager.startup()
     try {
       replicaManager.checkpointHighWatermarks()
@@ -69,7 +72,7 @@ class HighwatermarkPersistenceTest {
       assertEquals(0L, fooPartition0Hw)
       val partition0 = replicaManager.getOrCreatePartition(new TopicPartition(topic, 0))
       // create leader and follower replicas
-      val log0 = logManagers.head.createLog(new TopicPartition(topic, 0), LogConfig())
+      val log0 = logManagers.head.getOrCreateLog(new TopicPartition(topic, 0), LogConfig())
       val leaderReplicaPartition0 = new Replica(configs.head.brokerId, partition0, time, 0, Some(log0))
       partition0.addReplicaIfNotExists(leaderReplicaPartition0)
       val followerReplicaPartition0 = new Replica(configs.last.brokerId, partition0, time)
@@ -96,7 +99,6 @@ class HighwatermarkPersistenceTest {
     val topic1 = "foo1"
     val topic2 = "foo2"
     // mock zkclient
-    val zkUtils = EasyMock.createMock(classOf[ZkUtils])
     EasyMock.replay(zkUtils)
     // create kafka scheduler
     val scheduler = new KafkaScheduler(2)
@@ -106,7 +108,7 @@ class HighwatermarkPersistenceTest {
     // create replica manager
     val replicaManager = new ReplicaManager(configs.head, metrics, time, zkUtils,
       scheduler, logManagers.head, new AtomicBoolean(false), QuotaFactory.instantiate(configs.head, metrics, time).follower,
-      new BrokerTopicStats, new MetadataCache(configs.head.brokerId))
+      new BrokerTopicStats, new MetadataCache(configs.head.brokerId), logDirFailureChannels.head)
     replicaManager.startup()
     try {
       replicaManager.checkpointHighWatermarks()
@@ -114,7 +116,7 @@ class HighwatermarkPersistenceTest {
       assertEquals(0L, topic1Partition0Hw)
       val topic1Partition0 = replicaManager.getOrCreatePartition(new TopicPartition(topic1, 0))
       // create leader log
-      val topic1Log0 = logManagers.head.createLog(new TopicPartition(topic1, 0), LogConfig())
+      val topic1Log0 = logManagers.head.getOrCreateLog(new TopicPartition(topic1, 0), LogConfig())
       // create a local replica for topic1
       val leaderReplicaTopic1Partition0 = new Replica(configs.head.brokerId, topic1Partition0, time, 0, Some(topic1Log0))
       topic1Partition0.addReplicaIfNotExists(leaderReplicaTopic1Partition0)
@@ -130,7 +132,7 @@ class HighwatermarkPersistenceTest {
       // add another partition and set highwatermark
       val topic2Partition0 = replicaManager.getOrCreatePartition(new TopicPartition(topic2, 0))
       // create leader log
-      val topic2Log0 = logManagers.head.createLog(new TopicPartition(topic2, 0), LogConfig())
+      val topic2Log0 = logManagers.head.getOrCreateLog(new TopicPartition(topic2, 0), LogConfig())
       // create a local replica for topic2
       val leaderReplicaTopic2Partition0 =  new Replica(configs.head.brokerId, topic2Partition0, time, 0, Some(topic2Log0))
       topic2Partition0.addReplicaIfNotExists(leaderReplicaTopic2Partition0)
@@ -163,5 +165,5 @@ class HighwatermarkPersistenceTest {
     replicaManager.highWatermarkCheckpoints(new File(replicaManager.config.logDirs.head).getAbsolutePath).read.getOrElse(
       new TopicPartition(topic, partition), 0L)
   }
-  
+
 }


[3/5] kafka git commit: KAFKA-4763; Handle disk failure for JBOD (KIP-112)

Posted by jq...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/log/LogCleaner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala
index 4898d11..27da43b 100644
--- a/core/src/main/scala/kafka/log/LogCleaner.scala
+++ b/core/src/main/scala/kafka/log/LogCleaner.scala
@@ -5,7 +5,7 @@
  * The ASF licenses this file to You under the Apache License, Version 2.0
  * (the "License"); you may not use this file except in compliance with
  * the License.  You may obtain a copy of the License at
- * 
+ *
  *    http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
@@ -17,7 +17,7 @@
 
 package kafka.log
 
-import java.io.File
+import java.io.{File, IOException}
 import java.nio._
 import java.util.Date
 import java.util.concurrent.{CountDownLatch, TimeUnit}
@@ -25,6 +25,7 @@ import java.util.concurrent.{CountDownLatch, TimeUnit}
 import com.yammer.metrics.core.Gauge
 import kafka.common._
 import kafka.metrics.KafkaMetricsGroup
+import kafka.server.LogDirFailureChannel
 import kafka.utils._
 import org.apache.kafka.common.record._
 import org.apache.kafka.common.utils.Time
@@ -38,29 +39,29 @@ import scala.collection.JavaConverters._
 /**
  * The cleaner is responsible for removing obsolete records from logs which have the "compact" retention strategy.
  * A message with key K and offset O is obsolete if there exists a message with key K and offset O' such that O < O'.
- * 
+ *
  * Each log can be thought of being split into two sections of segments: a "clean" section which has previously been cleaned followed by a
  * "dirty" section that has not yet been cleaned. The dirty section is further divided into the "cleanable" section followed by an "uncleanable" section.
  * The uncleanable section is excluded from cleaning. The active log segment is always uncleanable. If there is a
  * compaction lag time set, segments whose largest message timestamp is within the compaction lag time of the cleaning operation are also uncleanable.
  *
  * The cleaning is carried out by a pool of background threads. Each thread chooses the dirtiest log that has the "compact" retention policy
- * and cleans that. The dirtiness of the log is guessed by taking the ratio of bytes in the dirty section of the log to the total bytes in the log. 
- * 
+ * and cleans that. The dirtiness of the log is guessed by taking the ratio of bytes in the dirty section of the log to the total bytes in the log.
+ *
  * To clean a log the cleaner first builds a mapping of key=>last_offset for the dirty section of the log. See kafka.log.OffsetMap for details of
- * the implementation of the mapping. 
- * 
- * Once the key=>offset map is built, the log is cleaned by recopying each log segment but omitting any key that appears in the offset map with a 
+ * the implementation of the mapping.
+ *
+ * Once the key=>offset map is built, the log is cleaned by recopying each log segment but omitting any key that appears in the offset map with a
  * higher offset than what is found in the segment (i.e. messages with a key that appears in the dirty section of the log).
- * 
+ *
  * To avoid segments shrinking to very small sizes with repeated cleanings we implement a rule by which if we will merge successive segments when
  * doing a cleaning if their log and index size are less than the maximum log and index size prior to the clean beginning.
- * 
+ *
  * Cleaned segments are swapped into the log as they become available.
- * 
+ *
  * One nuance that the cleaner must handle is log truncation. If a log is truncated while it is being cleaned the cleaning of that log is aborted.
- * 
- * Messages with null payload are treated as deletes for the purpose of log compaction. This means that they receive special treatment by the cleaner. 
+ *
+ * Messages with null payload are treated as deletes for the purpose of log compaction. This means that they receive special treatment by the cleaner.
  * The cleaner will only retain delete records for a period of time to avoid accumulating space indefinitely. This period of time is configurable on a per-topic
  * basis and is measured from the time the segment enters the clean portion of the log (at which point any prior message with that key has been removed).
  * Delete markers in the clean section of the log that are older than this time will not be retained when log segments are being recopied as part of cleaning.
@@ -88,29 +89,30 @@ import scala.collection.JavaConverters._
 class LogCleaner(val config: CleanerConfig,
                  val logDirs: Array[File],
                  val logs: Pool[TopicPartition, Log],
+                 val logDirFailureChannel: LogDirFailureChannel,
                  time: Time = Time.SYSTEM) extends Logging with KafkaMetricsGroup {
-  
+
   /* for managing the state of partitions being cleaned. package-private to allow access in tests */
-  private[log] val cleanerManager = new LogCleanerManager(logDirs, logs)
+  private[log] val cleanerManager = new LogCleanerManager(logDirs, logs, logDirFailureChannel)
 
   /* a throttle used to limit the I/O of all the cleaner threads to a user-specified maximum rate */
-  private val throttler = new Throttler(desiredRatePerSec = config.maxIoBytesPerSecond, 
-                                        checkIntervalMs = 300, 
-                                        throttleDown = true, 
+  private val throttler = new Throttler(desiredRatePerSec = config.maxIoBytesPerSecond,
+                                        checkIntervalMs = 300,
+                                        throttleDown = true,
                                         "cleaner-io",
                                         "bytes",
                                         time = time)
-  
+
   /* the threads */
   private val cleaners = (0 until config.numThreads).map(new CleanerThread(_))
-  
+
   /* a metric to track the maximum utilization of any thread's buffer in the last cleaning */
-  newGauge("max-buffer-utilization-percent", 
+  newGauge("max-buffer-utilization-percent",
            new Gauge[Int] {
              def value: Int = cleaners.map(_.lastStats).map(100 * _.bufferUtilization).max.toInt
            })
   /* a metric to track the recopy rate of each thread's last cleaning */
-  newGauge("cleaner-recopy-percent", 
+  newGauge("cleaner-recopy-percent",
            new Gauge[Int] {
              def value: Int = {
                val stats = cleaners.map(_.lastStats)
@@ -123,7 +125,7 @@ class LogCleaner(val config: CleanerConfig,
            new Gauge[Int] {
              def value: Int = cleaners.map(_.lastStats).map(_.elapsedSecs).max.toInt
            })
-  
+
   /**
    * Start the background cleaning
    */
@@ -131,7 +133,7 @@ class LogCleaner(val config: CleanerConfig,
     info("Starting the log cleaner")
     cleaners.foreach(_.start())
   }
-  
+
   /**
    * Stop the background cleaning
    */
@@ -139,7 +141,7 @@ class LogCleaner(val config: CleanerConfig,
     info("Shutting down the log cleaner.")
     cleaners.foreach(_.shutdown())
   }
-  
+
   /**
    *  Abort the cleaning of a particular partition, if it's in progress. This call blocks until the cleaning of
    *  the partition is aborted.
@@ -155,6 +157,10 @@ class LogCleaner(val config: CleanerConfig,
     cleanerManager.updateCheckpoints(dataDir, update=None)
   }
 
+  def handleLogDirFailure(dir: String) {
+    cleanerManager.handleLogDirFailure(dir)
+  }
+
   /**
    * Truncate cleaner offset checkpoint for the given partition if its checkpointed offset is larger than the given offset
    */
@@ -197,21 +203,21 @@ class LogCleaner(val config: CleanerConfig,
     }
     isCleaned
   }
-  
+
   /**
    * The cleaner threads do the actual log cleaning. Each thread processes does its cleaning repeatedly by
    * choosing the dirtiest log, cleaning it, and then swapping in the cleaned segments.
    */
   private class CleanerThread(threadId: Int)
     extends ShutdownableThread(name = "kafka-log-cleaner-thread-" + threadId, isInterruptible = false) {
-    
+
     override val loggerName = classOf[LogCleaner].getName
-    
+
     if(config.dedupeBufferSize / config.numThreads > Int.MaxValue)
       warn("Cannot use more than 2G of cleaner buffer space per cleaner thread, ignoring excess buffer space...")
 
     val cleaner = new Cleaner(id = threadId,
-                              offsetMap = new SkimpyOffsetMap(memory = math.min(config.dedupeBufferSize / config.numThreads, Int.MaxValue).toInt, 
+                              offsetMap = new SkimpyOffsetMap(memory = math.min(config.dedupeBufferSize / config.numThreads, Int.MaxValue).toInt,
                                                               hashAlgorithm = config.hashAlgorithm),
                               ioBufferSize = config.ioBufferSize / config.numThreads / 2,
                               maxIoBufferSize = config.maxMessageSize,
@@ -219,7 +225,7 @@ class LogCleaner(val config: CleanerConfig,
                               throttler = throttler,
                               time = time,
                               checkDone = checkDone)
-    
+
     @volatile var lastStats: CleanerStats = new CleanerStats()
     private val backOffWaitLatch = new CountDownLatch(1)
 
@@ -241,7 +247,7 @@ class LogCleaner(val config: CleanerConfig,
     	 backOffWaitLatch.countDown()
     	 awaitShutdown()
      }
-     
+
     /**
      * Clean a log if there is a dirty log available, otherwise sleep for a bit
      */
@@ -258,6 +264,9 @@ class LogCleaner(val config: CleanerConfig,
             endOffset = nextDirtyOffset
           } catch {
             case _: LogCleaningAbortedException => // task can be aborted, let it go.
+            case e: IOException =>
+              error(s"Failed to clean up log for ${cleanable.topicPartition} in dir ${cleanable.log.dir.getParent} due to IOException", e)
+              logDirFailureChannel.maybeAddLogFailureEvent(cleanable.log.dir.getParent)
           } finally {
             cleanerManager.doneCleaning(cleanable.topicPartition, cleanable.log.dir.getParentFile, endOffset)
           }
@@ -275,36 +284,36 @@ class LogCleaner(val config: CleanerConfig,
       if (!cleaned)
         backOffWaitLatch.await(config.backOffMs, TimeUnit.MILLISECONDS)
     }
-    
+
     /**
      * Log out statistics on a single run of the cleaner.
      */
     def recordStats(id: Int, name: String, from: Long, to: Long, stats: CleanerStats) {
       this.lastStats = stats
       def mb(bytes: Double) = bytes / (1024*1024)
-      val message = 
-        "%n\tLog cleaner thread %d cleaned log %s (dirty section = [%d, %d])%n".format(id, name, from, to) + 
-        "\t%,.1f MB of log processed in %,.1f seconds (%,.1f MB/sec).%n".format(mb(stats.bytesRead), 
-                                                                                stats.elapsedSecs, 
-                                                                                mb(stats.bytesRead/stats.elapsedSecs)) + 
-        "\tIndexed %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.mapBytesRead), 
-                                                                                           stats.elapsedIndexSecs, 
-                                                                                           mb(stats.mapBytesRead)/stats.elapsedIndexSecs, 
+      val message =
+        "%n\tLog cleaner thread %d cleaned log %s (dirty section = [%d, %d])%n".format(id, name, from, to) +
+        "\t%,.1f MB of log processed in %,.1f seconds (%,.1f MB/sec).%n".format(mb(stats.bytesRead),
+                                                                                stats.elapsedSecs,
+                                                                                mb(stats.bytesRead/stats.elapsedSecs)) +
+        "\tIndexed %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.mapBytesRead),
+                                                                                           stats.elapsedIndexSecs,
+                                                                                           mb(stats.mapBytesRead)/stats.elapsedIndexSecs,
                                                                                            100 * stats.elapsedIndexSecs/stats.elapsedSecs) +
         "\tBuffer utilization: %.1f%%%n".format(100 * stats.bufferUtilization) +
-        "\tCleaned %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.bytesRead), 
-                                                                                           stats.elapsedSecs - stats.elapsedIndexSecs, 
-                                                                                           mb(stats.bytesRead)/(stats.elapsedSecs - stats.elapsedIndexSecs), 100 * (stats.elapsedSecs - stats.elapsedIndexSecs).toDouble/stats.elapsedSecs) + 
+        "\tCleaned %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.bytesRead),
+                                                                                           stats.elapsedSecs - stats.elapsedIndexSecs,
+                                                                                           mb(stats.bytesRead)/(stats.elapsedSecs - stats.elapsedIndexSecs), 100 * (stats.elapsedSecs - stats.elapsedIndexSecs).toDouble/stats.elapsedSecs) +
         "\tStart size: %,.1f MB (%,d messages)%n".format(mb(stats.bytesRead), stats.messagesRead) +
-        "\tEnd size: %,.1f MB (%,d messages)%n".format(mb(stats.bytesWritten), stats.messagesWritten) + 
-        "\t%.1f%% size reduction (%.1f%% fewer messages)%n".format(100.0 * (1.0 - stats.bytesWritten.toDouble/stats.bytesRead), 
+        "\tEnd size: %,.1f MB (%,d messages)%n".format(mb(stats.bytesWritten), stats.messagesWritten) +
+        "\t%.1f%% size reduction (%.1f%% fewer messages)%n".format(100.0 * (1.0 - stats.bytesWritten.toDouble/stats.bytesRead),
                                                                    100.0 * (1.0 - stats.messagesWritten.toDouble/stats.messagesRead))
       info(message)
       if (stats.invalidMessagesRead > 0) {
         warn("\tFound %d invalid messages during compaction.".format(stats.invalidMessagesRead))
       }
     }
-   
+
   }
 }
 
@@ -327,14 +336,14 @@ private[log] class Cleaner(val id: Int,
                            throttler: Throttler,
                            time: Time,
                            checkDone: (TopicPartition) => Unit) extends Logging {
-  
+
   override val loggerName = classOf[LogCleaner].getName
 
   this.logIdent = "Cleaner " + id + ": "
 
   /* buffer used for read i/o */
   private var readBuffer = ByteBuffer.allocate(ioBufferSize)
-  
+
   /* buffer used for write i/o */
   private var writeBuffer = ByteBuffer.allocate(ioBufferSize)
 
@@ -352,7 +361,7 @@ private[log] class Cleaner(val id: Int,
   private[log] def clean(cleanable: LogToClean): (Long, CleanerStats) = {
     // figure out the timestamp below which it is safe to remove delete tombstones
     // this position is defined to be a configurable time beneath the last modified time of the last clean segment
-    val deleteHorizonMs = 
+    val deleteHorizonMs =
       cleanable.log.logSegments(0, cleanable.firstDirtyOffset).lastOption match {
         case None => 0L
         case Some(seg) => seg.lastModified - cleanable.log.config.deleteRetentionMs

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/log/LogCleanerManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala
index ed0cb69..af8707c 100755
--- a/core/src/main/scala/kafka/log/LogCleanerManager.scala
+++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala
@@ -24,11 +24,13 @@ import java.util.concurrent.locks.ReentrantLock
 import com.yammer.metrics.core.Gauge
 import kafka.common.LogCleaningAbortedException
 import kafka.metrics.KafkaMetricsGroup
+import kafka.server.LogDirFailureChannel
 import kafka.server.checkpoints.OffsetCheckpointFile
 import kafka.utils.CoreUtils._
 import kafka.utils.{Logging, Pool}
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.utils.Time
+import org.apache.kafka.common.errors.KafkaStorageException
 
 import scala.collection.{immutable, mutable}
 
@@ -45,7 +47,9 @@ private[log] case object LogCleaningPaused extends LogCleaningState
  *  While a partition is in the LogCleaningPaused state, it won't be scheduled for cleaning again, until cleaning is
  *  requested to be resumed.
  */
-private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[TopicPartition, Log]) extends Logging with KafkaMetricsGroup {
+private[log] class LogCleanerManager(val logDirs: Array[File],
+                                     val logs: Pool[TopicPartition, Log],
+                                     val logDirFailureChannel: LogDirFailureChannel) extends Logging with KafkaMetricsGroup {
 
   import LogCleanerManager._
 
@@ -53,19 +57,19 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To
 
   // package-private for testing
   private[log] val offsetCheckpointFile = "cleaner-offset-checkpoint"
-  
+
   /* the offset checkpoints holding the last cleaned point for each log */
-  private val checkpoints = logDirs.map(dir => (dir, new OffsetCheckpointFile(new File(dir, offsetCheckpointFile)))).toMap
+  @volatile private var checkpoints = logDirs.map(dir => (dir, new OffsetCheckpointFile(new File(dir, offsetCheckpointFile), logDirFailureChannel))).toMap
 
   /* the set of logs currently being cleaned */
   private val inProgress = mutable.HashMap[TopicPartition, LogCleaningState]()
 
   /* a global lock used to control all access to the in-progress set and the offset checkpoints */
   private val lock = new ReentrantLock
-  
+
   /* for coordinating the pausing and the cleaning of a partition */
   private val pausedCleaningCond = lock.newCondition()
-  
+
   /* a gauge for tracking the cleanable ratio of the dirtiest log */
   @volatile private var dirtiestLogCleanableRatio = 0.0
   newGauge("max-dirty-percent", new Gauge[Int] { def value = (100 * dirtiestLogCleanableRatio).toInt })
@@ -77,8 +81,20 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To
   /**
    * @return the position processed for all logs.
    */
-  def allCleanerCheckpoints: Map[TopicPartition, Long] =
-    checkpoints.values.flatMap(_.read()).toMap
+  def allCleanerCheckpoints: Map[TopicPartition, Long] = {
+    inLock(lock) {
+      checkpoints.values.flatMap(checkpoint => {
+        try {
+          checkpoint.read()
+        } catch {
+          case e: KafkaStorageException =>
+            error(s"Failed to access checkpoint file ${checkpoint.f.getName} in dir ${checkpoint.f.getParentFile.getAbsolutePath}", e)
+            Map.empty[TopicPartition, Long]
+        }
+      }).toMap
+    }
+  }
+
 
    /**
     * Choose the log to clean next and add it to the in-progress set. We recompute this
@@ -217,8 +233,22 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To
   def updateCheckpoints(dataDir: File, update: Option[(TopicPartition,Long)]) {
     inLock(lock) {
       val checkpoint = checkpoints(dataDir)
-      val existing = checkpoint.read().filterKeys(logs.keys) ++ update
-      checkpoint.write(existing)
+      if (checkpoint != null) {
+        try {
+          val existing = checkpoint.read().filterKeys(logs.keys) ++ update
+          checkpoint.write(existing)
+        } catch {
+          case e: KafkaStorageException =>
+            error(s"Failed to access checkpoint file ${checkpoint.f.getName} in dir ${checkpoint.f.getParentFile.getAbsolutePath}", e)
+        }
+      }
+    }
+  }
+
+  def handleLogDirFailure(dir: String) {
+    info(s"Stopping cleaning logs in dir $dir")
+    inLock(lock) {
+      checkpoints = checkpoints.filterKeys(_.getAbsolutePath != dir)
     }
   }
 
@@ -226,10 +256,11 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To
     inLock(lock) {
       if (logs.get(topicPartition).config.compact) {
         val checkpoint = checkpoints(dataDir)
-        val existing = checkpoint.read()
-
-        if (existing.getOrElse(topicPartition, 0L) > offset)
-          checkpoint.write(existing + (topicPartition -> offset))
+        if (checkpoint != null) {
+          val existing = checkpoint.read()
+          if (existing.getOrElse(topicPartition, 0L) > offset)
+            checkpoint.write(existing + (topicPartition -> offset))
+        }
       }
     }
   }
@@ -241,7 +272,7 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To
     inLock(lock) {
       inProgress(topicPartition) match {
         case LogCleaningInProgress =>
-          updateCheckpoints(dataDir,Option(topicPartition, endOffset))
+          updateCheckpoints(dataDir, Option(topicPartition, endOffset))
           inProgress.remove(topicPartition)
         case LogCleaningAborted =>
           inProgress.put(topicPartition, LogCleaningPaused)

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/log/LogManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala
index 2df5241..f459cc1 100755
--- a/core/src/main/scala/kafka/log/LogManager.scala
+++ b/core/src/main/scala/kafka/log/LogManager.scala
@@ -21,29 +21,33 @@ import java.io._
 import java.nio.file.Files
 import java.util.concurrent._
 
+import com.yammer.metrics.core.Gauge
 import kafka.admin.AdminUtils
-import kafka.common.{KafkaException, KafkaStorageException}
+import kafka.common.KafkaException
+import kafka.metrics.KafkaMetricsGroup
 import kafka.server.checkpoints.OffsetCheckpointFile
 import kafka.server.{BrokerState, RecoveringFromUncleanShutdown, _}
 import kafka.utils._
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.utils.Time
-
+import org.apache.kafka.common.errors.KafkaStorageException
 import scala.collection.JavaConverters._
 import scala.collection._
+import scala.collection.mutable.ArrayBuffer
 
 /**
  * The entry point to the kafka log management subsystem. The log manager is responsible for log creation, retrieval, and cleaning.
  * All read and write operations are delegated to the individual log instances.
- * 
+ *
  * The log manager maintains logs in one or more directories. New logs are created in the data directory
  * with the fewest logs. No attempt is made to move partitions after the fact or balance based on
  * size or I/O rate.
- * 
+ *
  * A background thread handles log retention by periodically truncating excess log segments.
  */
 @threadsafe
-class LogManager(val logDirs: Array[File],
+class LogManager(logDirs: Array[File],
+                 initialOfflineDirs: Array[File],
                  val topicConfigs: Map[String, LogConfig], // note that this doesn't get updated after creation
                  val defaultConfig: LogConfig,
                  val cleanerConfig: CleanerConfig,
@@ -56,7 +60,8 @@ class LogManager(val logDirs: Array[File],
                  scheduler: Scheduler,
                  val brokerState: BrokerState,
                  brokerTopicStats: BrokerTopicStats,
-                 time: Time) extends Logging {
+                 logDirFailureChannel: LogDirFailureChannel,
+                 time: Time) extends Logging with KafkaMetricsGroup {
   val RecoveryPointCheckpointFile = "recovery-point-offset-checkpoint"
   val LogStartOffsetCheckpointFile = "log-start-offset-checkpoint"
   val LockFile = ".lock"
@@ -66,140 +71,243 @@ class LogManager(val logDirs: Array[File],
   private val logs = new Pool[TopicPartition, Log]()
   private val logsToBeDeleted = new LinkedBlockingQueue[Log]()
 
-  createAndValidateLogDirs(logDirs)
-  private val dirLocks = lockLogDirs(logDirs)
-  private val recoveryPointCheckpoints = logDirs.map(dir => (dir, new OffsetCheckpointFile(new File(dir, RecoveryPointCheckpointFile)))).toMap
-  private val logStartOffsetCheckpoints = logDirs.map(dir => (dir, new OffsetCheckpointFile(new File(dir, LogStartOffsetCheckpointFile)))).toMap
+  private val _liveLogDirs: ConcurrentLinkedQueue[File] = createAndValidateLogDirs(logDirs, initialOfflineDirs)
+
+  def liveLogDirs: Array[File] = {
+    if (_liveLogDirs.size() == logDirs.size)
+      logDirs
+    else
+      _liveLogDirs.asScala.toArray
+  }
+
+  private val dirLocks = lockLogDirs(liveLogDirs)
+  @volatile private var recoveryPointCheckpoints = liveLogDirs.map(dir =>
+    (dir, new OffsetCheckpointFile(new File(dir, RecoveryPointCheckpointFile), logDirFailureChannel))).toMap
+  @volatile private var logStartOffsetCheckpoints = liveLogDirs.map(dir =>
+    (dir, new OffsetCheckpointFile(new File(dir, LogStartOffsetCheckpointFile), logDirFailureChannel))).toMap
+
+  private def offlineLogDirs = logDirs.filterNot(_liveLogDirs.contains)
+
   loadLogs()
 
+
   // public, so we can access this from kafka.admin.DeleteTopicTest
   val cleaner: LogCleaner =
     if(cleanerConfig.enableCleaner)
-      new LogCleaner(cleanerConfig, logDirs, logs, time = time)
+      new LogCleaner(cleanerConfig, liveLogDirs, logs, logDirFailureChannel, time = time)
     else
       null
-  
+
+  val offlineLogDirectoryCount = newGauge(
+    "OfflineLogDirectoryCount",
+    new Gauge[Int] {
+      def value = offlineLogDirs.length
+    }
+  )
+
+  for (dir <- logDirs) {
+    newGauge(
+      "LogDirectoryOffline",
+      new Gauge[Int] {
+        def value = if (_liveLogDirs.contains(dir)) 0 else 1
+      },
+      Map("logDirectory" -> dir.getAbsolutePath)
+    )
+  }
+
   /**
-   * Create and check validity of the given directories, specifically:
+   * Create and check validity of the given directories that are not in the given offline directories, specifically:
    * <ol>
    * <li> Ensure that there are no duplicates in the directory list
    * <li> Create each directory if it doesn't exist
-   * <li> Check that each path is a readable directory 
+   * <li> Check that each path is a readable directory
    * </ol>
    */
-  private def createAndValidateLogDirs(dirs: Seq[File]) {
+  private def createAndValidateLogDirs(dirs: Seq[File], initialOfflineDirs: Seq[File]): ConcurrentLinkedQueue[File] = {
     if(dirs.map(_.getCanonicalPath).toSet.size < dirs.size)
-      throw new KafkaException("Duplicate log directory found: " + logDirs.mkString(", "))
-    for(dir <- dirs) {
-      if(!dir.exists) {
-        info("Log directory '" + dir.getAbsolutePath + "' not found, creating it.")
-        val created = dir.mkdirs()
-        if(!created)
-          throw new KafkaException("Failed to create data directory " + dir.getAbsolutePath)
+      throw new KafkaException("Duplicate log directory found: " + dirs.mkString(", "))
+
+    val liveLogDirs = new ConcurrentLinkedQueue[File]()
+
+    for (dir <- dirs if !initialOfflineDirs.contains(dir)) {
+      try {
+        if (!dir.exists) {
+          info("Log directory '" + dir.getAbsolutePath + "' not found, creating it.")
+          val created = dir.mkdirs()
+          if (!created)
+            throw new IOException("Failed to create data directory " + dir.getAbsolutePath)
+        }
+        if (!dir.isDirectory || !dir.canRead)
+          throw new IOException(dir.getAbsolutePath + " is not a readable log directory.")
+        liveLogDirs.add(dir)
+      } catch {
+        case e: IOException =>
+          error(s"Failed to create or validate data directory $dir.getAbsolutePath", e)
+      }
+    }
+    if (liveLogDirs.isEmpty) {
+      fatal(s"Shutdown broker because none of the specified log dirs from " + dirs.mkString(", ") + " can be created or validated")
+      Exit.halt(1)
+    }
+
+    liveLogDirs
+  }
+
+  def handleLogDirFailure(dir: String) {
+    info(s"Stopping serving logs in dir $dir")
+    logCreationOrDeletionLock synchronized {
+      _liveLogDirs.remove(new File(dir))
+      if (_liveLogDirs.isEmpty) {
+        fatal(s"Shutdown broker because all log dirs in ${logDirs.mkString(", ")} have failed")
+        Exit.halt(1)
       }
-      if(!dir.isDirectory || !dir.canRead)
-        throw new KafkaException(dir.getAbsolutePath + " is not a readable log directory.")
+
+      recoveryPointCheckpoints = recoveryPointCheckpoints.filterKeys(file => file.getAbsolutePath != dir)
+      logStartOffsetCheckpoints = logStartOffsetCheckpoints.filterKeys(file => file.getAbsolutePath != dir)
+      if (cleaner != null)
+        cleaner.handleLogDirFailure(dir)
+
+      val offlineTopicPartitions = logs.filter { case (tp, log) => log.dir.getParent == dir}.map { case (tp, log) => tp }
+
+      offlineTopicPartitions.foreach(topicPartition => {
+        val removedLog = logs.remove(topicPartition)
+        if (removedLog != null) {
+          removedLog.closeHandlers()
+          removedLog.removeLogMetrics()
+        }
+      })
+      info(s"Partitions ${offlineTopicPartitions.mkString(",")} are offline due to failure on log directory $dir")
+      dirLocks.filter(_.file.getParent == dir).foreach(dir => CoreUtils.swallow(dir.destroy()))
     }
   }
-  
+
   /**
    * Lock all the given directories
    */
   private def lockLogDirs(dirs: Seq[File]): Seq[FileLock] = {
-    dirs.map { dir =>
-      val lock = new FileLock(new File(dir, LockFile))
-      if(!lock.tryLock())
-        throw new KafkaException("Failed to acquire lock on file .lock in " + lock.file.getParentFile.getAbsolutePath + 
-                               ". A Kafka instance in another process or thread is using this directory.")
-      lock
+    dirs.flatMap { dir =>
+      try {
+        val lock = new FileLock(new File(dir, LockFile))
+        if (!lock.tryLock())
+          throw new KafkaException("Failed to acquire lock on file .lock in " + lock.file.getParent +
+            ". A Kafka instance in another process or thread is using this directory.")
+        Some(lock)
+      } catch {
+        case e: IOException =>
+          error(s"Disk error while locking directory $dir", e)
+          logDirFailureChannel.maybeAddLogFailureEvent(dir.getAbsolutePath)
+          None
+      }
+    }
+  }
+
+  private def loadLogs(logDir: File, recoveryPoints: Map[TopicPartition, Long], logStartOffsets: Map[TopicPartition, Long]): Unit = {
+    debug("Loading log '" + logDir.getName + "'")
+    val topicPartition = Log.parseTopicPartitionName(logDir)
+    val config = topicConfigs.getOrElse(topicPartition.topic, defaultConfig)
+    val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L)
+    val logStartOffset = logStartOffsets.getOrElse(topicPartition, 0L)
+
+    val current = Log(
+      dir = logDir,
+      config = config,
+      logStartOffset = logStartOffset,
+      recoveryPoint = logRecoveryPoint,
+      maxProducerIdExpirationMs = maxPidExpirationMs,
+      scheduler = scheduler,
+      time = time,
+      brokerTopicStats = brokerTopicStats)
+
+    if (logDir.getName.endsWith(Log.DeleteDirSuffix)) {
+      this.logsToBeDeleted.add(current)
+    } else {
+      val previous = this.logs.put(topicPartition, current)
+      if (previous != null) {
+        throw new IllegalArgumentException(
+          "Duplicate log directories found: %s, %s!".format(
+            current.dir.getAbsolutePath, previous.dir.getAbsolutePath))
+      }
     }
   }
-  
+
   /**
    * Recover and load all logs in the given data directories
    */
   private def loadLogs(): Unit = {
     info("Loading logs.")
     val startMs = time.milliseconds
-    val threadPools = mutable.ArrayBuffer.empty[ExecutorService]
+    val threadPools = ArrayBuffer.empty[ExecutorService]
+    val offlineDirs = ArrayBuffer.empty[String]
     val jobs = mutable.Map.empty[File, Seq[Future[_]]]
 
-    for (dir <- this.logDirs) {
-      val pool = Executors.newFixedThreadPool(ioThreads)
-      threadPools.append(pool)
-
-      val cleanShutdownFile = new File(dir, Log.CleanShutdownFile)
-
-      if (cleanShutdownFile.exists) {
-        debug(
-          "Found clean shutdown file. " +
-          "Skipping recovery for all logs in data directory: " +
-          dir.getAbsolutePath)
-      } else {
-        // log recovery itself is being performed by `Log` class during initialization
-        brokerState.newState(RecoveringFromUncleanShutdown)
-      }
-
-      var recoveryPoints = Map[TopicPartition, Long]()
-      try {
-        recoveryPoints = this.recoveryPointCheckpoints(dir).read
-      } catch {
-        case e: Exception =>
-          warn("Error occurred while reading recovery-point-offset-checkpoint file of directory " + dir, e)
-          warn("Resetting the recovery checkpoint to 0")
-      }
-
-      var logStartOffsets = Map[TopicPartition, Long]()
+    for (dir <- liveLogDirs) {
       try {
-        logStartOffsets = this.logStartOffsetCheckpoints(dir).read
-      } catch {
-        case e: Exception =>
-          warn("Error occurred while reading log-start-offset-checkpoint file of directory " + dir, e)
-      }
+        val pool = Executors.newFixedThreadPool(ioThreads)
+        threadPools.append(pool)
+
+        val cleanShutdownFile = new File(dir, Log.CleanShutdownFile)
+
+        if (cleanShutdownFile.exists) {
+          debug(
+            "Found clean shutdown file. " +
+              "Skipping recovery for all logs in data directory: " +
+              dir.getAbsolutePath)
+        } else {
+          // log recovery itself is being performed by `Log` class during initialization
+          brokerState.newState(RecoveringFromUncleanShutdown)
+        }
 
-      val jobsForDir = for {
-        dirContent <- Option(dir.listFiles).toList
-        logDir <- dirContent if logDir.isDirectory
-      } yield {
-        CoreUtils.runnable {
-          debug("Loading log '" + logDir.getName + "'")
+        var recoveryPoints = Map[TopicPartition, Long]()
+        try {
+          recoveryPoints = this.recoveryPointCheckpoints(dir).read
+        } catch {
+          case e: Exception =>
+            warn("Error occurred while reading recovery-point-offset-checkpoint file of directory " + dir, e)
+            warn("Resetting the recovery checkpoint to 0")
+        }
 
-          val topicPartition = Log.parseTopicPartitionName(logDir)
-          val config = topicConfigs.getOrElse(topicPartition.topic, defaultConfig)
-          val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L)
-          val logStartOffset = logStartOffsets.getOrElse(topicPartition, 0L)
+        var logStartOffsets = Map[TopicPartition, Long]()
+        try {
+          logStartOffsets = this.logStartOffsetCheckpoints(dir).read
+        } catch {
+          case e: Exception =>
+            warn("Error occurred while reading log-start-offset-checkpoint file of directory " + dir, e)
+        }
 
-          val current = Log(
-            dir = logDir,
-            config = config,
-            logStartOffset = logStartOffset,
-            recoveryPoint = logRecoveryPoint,
-            maxProducerIdExpirationMs = maxPidExpirationMs,
-            scheduler = scheduler,
-            time = time,
-            brokerTopicStats = brokerTopicStats)
-          if (logDir.getName.endsWith(Log.DeleteDirSuffix)) {
-            this.logsToBeDeleted.add(current)
-          } else {
-            val previous = this.logs.put(topicPartition, current)
-            if (previous != null) {
-              throw new IllegalArgumentException(
-                "Duplicate log directories found: %s, %s!".format(
-                  current.dir.getAbsolutePath, previous.dir.getAbsolutePath))
+        val jobsForDir = for {
+          dirContent <- Option(dir.listFiles).toList
+          logDir <- dirContent if logDir.isDirectory
+        } yield {
+          CoreUtils.runnable {
+            try {
+              loadLogs(logDir, recoveryPoints, logStartOffsets)
+            } catch {
+              case e: IOException =>
+                offlineDirs.append(dir.getAbsolutePath)
+                error("Error while loading log dir " + dir.getAbsolutePath, e)
             }
           }
         }
+        jobs(cleanShutdownFile) = jobsForDir.map(pool.submit)
+      } catch {
+        case e: IOException =>
+          offlineDirs.append(dir.getAbsolutePath)
+          error("Error while loading log dir " + dir.getAbsolutePath, e)
       }
-
-      jobs(cleanShutdownFile) = jobsForDir.map(pool.submit)
     }
 
-
     try {
       for ((cleanShutdownFile, dirJobs) <- jobs) {
         dirJobs.foreach(_.get)
-        cleanShutdownFile.delete()
+        try {
+          cleanShutdownFile.delete()
+        } catch {
+          case e: IOException =>
+            offlineDirs.append(cleanShutdownFile.getParent)
+            error(s"Error while deleting the clean shutdown file $cleanShutdownFile", e)
+        }
       }
+      offlineDirs.foreach(logDirFailureChannel.maybeAddLogFailureEvent)
     } catch {
       case e: ExecutionException => {
         error("There was an error in one of the threads during logs loading: " + e.getCause)
@@ -231,7 +339,7 @@ class LogManager(val logDirs: Array[File],
                          period = flushCheckMs,
                          TimeUnit.MILLISECONDS)
       scheduler.schedule("kafka-recovery-point-checkpoint",
-                         checkpointRecoveryPointOffsets _,
+                         checkpointLogRecoveryOffsets _,
                          delay = InitialTaskDelayMs,
                          period = flushRecoveryOffsetCheckpointMs,
                          TimeUnit.MILLISECONDS)
@@ -256,7 +364,12 @@ class LogManager(val logDirs: Array[File],
   def shutdown() {
     info("Shutting down.")
 
-    val threadPools = mutable.ArrayBuffer.empty[ExecutorService]
+    removeMetric("OfflineLogDirectoryCount")
+    for (dir <- logDirs) {
+      removeMetric("LogDirectoryOffline", Map("logDirectory" -> dir.getAbsolutePath))
+    }
+
+    val threadPools = ArrayBuffer.empty[ExecutorService]
     val jobs = mutable.Map.empty[File, Seq[Future[_]]]
 
     // stop the cleaner first
@@ -265,7 +378,7 @@ class LogManager(val logDirs: Array[File],
     }
 
     // close logs in each dir
-    for (dir <- this.logDirs) {
+    for (dir <- liveLogDirs) {
       debug("Flushing and closing logs at " + dir)
 
       val pool = Executors.newFixedThreadPool(ioThreads)
@@ -337,11 +450,12 @@ class LogManager(val logDirs: Array[File],
         }
       }
     }
-    checkpointRecoveryPointOffsets()
+    checkpointLogRecoveryOffsets()
   }
 
   /**
    *  Delete all data in a partition and start the log at the new offset
+   *
    *  @param newOffset The new offset to start the log with
    */
   def truncateFullyAndStartAt(topicPartition: TopicPartition, newOffset: Long) {
@@ -357,15 +471,15 @@ class LogManager(val logDirs: Array[File],
         cleaner.resumeCleaning(topicPartition)
       }
     }
-    checkpointRecoveryPointOffsets()
+    checkpointLogRecoveryOffsets()
   }
 
   /**
-   * Write out the current recovery point for all logs to a text file in the log directory 
+   * Write out the current recovery point for all logs to a text file in the log directory
    * to avoid recovering the whole log on startup.
    */
-  def checkpointRecoveryPointOffsets() {
-    this.logDirs.foreach(checkpointLogRecoveryOffsetsInDir)
+  def checkpointLogRecoveryOffsets() {
+    liveLogDirs.foreach(checkpointLogRecoveryOffsetsInDir)
   }
 
   /**
@@ -373,7 +487,7 @@ class LogManager(val logDirs: Array[File],
    * to avoid exposing data that have been deleted by DeleteRecordsRequest
    */
   def checkpointLogStartOffsets() {
-    this.logDirs.foreach(checkpointLogStartOffsetsInDir)
+    liveLogDirs.foreach(checkpointLogStartOffsetsInDir)
   }
 
   /**
@@ -382,7 +496,13 @@ class LogManager(val logDirs: Array[File],
   private def checkpointLogRecoveryOffsetsInDir(dir: File): Unit = {
     val recoveryPoints = this.logsByDir.get(dir.toString)
     if (recoveryPoints.isDefined) {
-      this.recoveryPointCheckpoints(dir).write(recoveryPoints.get.mapValues(_.recoveryPoint))
+      try {
+        this.recoveryPointCheckpoints.get(dir).foreach(_.write(recoveryPoints.get.mapValues(_.recoveryPoint)))
+      } catch {
+        case e: IOException =>
+          error(s"Disk error while writing to recovery point file in directory $dir", e)
+          logDirFailureChannel.maybeAddLogFailureEvent(dir.getAbsolutePath)
+      }
     }
   }
 
@@ -390,10 +510,17 @@ class LogManager(val logDirs: Array[File],
    * Checkpoint log start offset for all logs in provided directory.
    */
   private def checkpointLogStartOffsetsInDir(dir: File): Unit = {
-    val logs = this.logsByDir.get(dir.toString)
+    val logs = this.logsByDir.get(dir.getAbsolutePath)
     if (logs.isDefined) {
-      this.logStartOffsetCheckpoints(dir).write(
-        logs.get.filter{case (tp, log) => log.logStartOffset > log.logSegments.head.baseOffset}.mapValues(_.logStartOffset))
+      try {
+        this.logStartOffsetCheckpoints.get(dir).foreach(_.write(
+          logs.get.filter { case (tp, log) => log.logStartOffset > log.logSegments.head.baseOffset }.mapValues(_.logStartOffset)
+        ))
+      } catch {
+        case e: IOException =>
+          error(s"Disk error while writing to logStartOffset file in directory $dir", e)
+          logDirFailureChannel.maybeAddLogFailureEvent(dir.getAbsolutePath)
+      }
     }
   }
 
@@ -403,33 +530,47 @@ class LogManager(val logDirs: Array[File],
   def getLog(topicPartition: TopicPartition): Option[Log] = Option(logs.get(topicPartition))
 
   /**
-   * Create a log for the given topic and the given partition
    * If the log already exists, just return a copy of the existing log
+   * Otherwise if isNew=true or if there is no offline log directory, create a log for the given topic and the given partition
+   * Otherwise throw KafkaStorageException
+   *
+   * @param isNew Whether the replica should have existed on the broker or not
+   * @throws KafkaStorageException if isNew=false, log is not found in the cache and there is offline log directory on the broker
    */
-  def createLog(topicPartition: TopicPartition, config: LogConfig): Log = {
+  def getOrCreateLog(topicPartition: TopicPartition, config: LogConfig, isNew: Boolean = false): Log = {
     logCreationOrDeletionLock synchronized {
-      // create the log if it has not already been created in another thread
       getLog(topicPartition).getOrElse {
+        // create the log if it has not already been created in another thread
+        if (!isNew && offlineLogDirs.nonEmpty)
+          throw new KafkaStorageException(s"Can not create log for $topicPartition because log directories ${offlineLogDirs.mkString(",")} are offline")
+
         val dataDir = nextLogDir()
-        val dir = new File(dataDir, topicPartition.topic + "-" + topicPartition.partition)
-        Files.createDirectories(dir.toPath)
-
-        val log = Log(
-          dir = dir,
-          config = config,
-          logStartOffset = 0L,
-          recoveryPoint = 0L,
-          maxProducerIdExpirationMs = maxPidExpirationMs,
-          scheduler = scheduler,
-          time = time,
-          brokerTopicStats = brokerTopicStats)
-        logs.put(topicPartition, log)
-        info("Created log for partition [%s,%d] in %s with properties {%s}."
-          .format(topicPartition.topic,
-            topicPartition.partition,
-            dataDir.getAbsolutePath,
-            config.originals.asScala.mkString(", ")))
-        log
+        try {
+          val dir = new File(dataDir, topicPartition.topic + "-" + topicPartition.partition)
+          Files.createDirectories(dir.toPath)
+
+          val log = Log(
+            dir = dir,
+            config = config,
+            logStartOffset = 0L,
+            recoveryPoint = 0L,
+            maxProducerIdExpirationMs = maxPidExpirationMs,
+            scheduler = scheduler,
+            time = time,
+            brokerTopicStats = brokerTopicStats)
+          logs.put(topicPartition, log)
+
+          info("Created log for partition [%s,%d] in %s with properties {%s}."
+            .format(topicPartition.topic,
+              topicPartition.partition,
+              dataDir.getAbsolutePath,
+              config.originals.asScala.mkString(", ")))
+          log
+        } catch {
+          case e: IOException =>
+            logDirFailureChannel.maybeAddLogFailureEvent(dataDir.getAbsolutePath)
+            throw new KafkaStorageException(s"Error while creating log for $topicPartition in dir ${dataDir.getAbsolutePath}", e)
+        }
       }
     }
   }
@@ -439,30 +580,27 @@ class LogManager(val logDirs: Array[File],
    */
   private def deleteLogs(): Unit = {
     try {
-      var failed = 0
-      while (!logsToBeDeleted.isEmpty && failed < logsToBeDeleted.size()) {
+      while (!logsToBeDeleted.isEmpty) {
         val removedLog = logsToBeDeleted.take()
         if (removedLog != null) {
           try {
             removedLog.delete()
             info(s"Deleted log for partition ${removedLog.topicPartition} in ${removedLog.dir.getAbsolutePath}.")
           } catch {
-            case e: Throwable =>
-              error(s"Exception in deleting $removedLog. Moving it to the end of the queue.", e)
-              failed = failed + 1
-              logsToBeDeleted.put(removedLog)
+            case e: KafkaStorageException =>
+              error(s"Exception while deleting $removedLog in dir ${removedLog.dir.getParent}.", e)
           }
         }
       }
     } catch {
-      case e: Throwable => 
+      case e: Throwable =>
         error(s"Exception in kafka-delete-logs thread.", e)
     }
   }
 
   /**
-    * Rename the directory of the given topic-partition "logdir" as "logdir.uuid.delete" and 
-    * add it in the queue for deletion. 
+    * Rename the directory of the given topic-partition "logdir" as "logdir.uuid.delete" and
+    * add it in the queue for deletion.
     * @param topicPartition TopicPartition that needs to be deleted
     */
   def asyncDelete(topicPartition: TopicPartition) = {
@@ -470,30 +608,38 @@ class LogManager(val logDirs: Array[File],
       logs.remove(topicPartition)
     }
     if (removedLog != null) {
-      //We need to wait until there is no more cleaning task on the log to be deleted before actually deleting it.
-      if (cleaner != null) {
-        cleaner.abortCleaning(topicPartition)
-        cleaner.updateCheckpoints(removedLog.dir.getParentFile)
-      }
-      val dirName = Log.logDeleteDirName(removedLog.name)
-      removedLog.close()
-      val renamedDir = new File(removedLog.dir.getParent, dirName)
-      val renameSuccessful = removedLog.dir.renameTo(renamedDir)
-      if (renameSuccessful) {
-        checkpointLogStartOffsetsInDir(removedLog.dir.getParentFile)
-        removedLog.dir = renamedDir
-        // change the file pointers for log and index file
-        for (logSegment <- removedLog.logSegments) {
-          logSegment.log.setFile(new File(renamedDir, logSegment.log.file.getName))
-          logSegment.index.file = new File(renamedDir, logSegment.index.file.getName)
+      try {
+        //We need to wait until there is no more cleaning task on the log to be deleted before actually deleting it.
+        if (cleaner != null) {
+          cleaner.abortCleaning(topicPartition)
+          cleaner.updateCheckpoints(removedLog.dir.getParentFile)
         }
+        val dirName = Log.logDeleteDirName(removedLog.name)
+        removedLog.close()
+        val renamedDir = new File(removedLog.dir.getParent, dirName)
+        val renameSuccessful = removedLog.dir.renameTo(renamedDir)
+        if (renameSuccessful) {
+          checkpointLogStartOffsetsInDir(removedLog.dir.getParentFile)
+          removedLog.dir = renamedDir
+          // change the file pointers for log and index file
+          for (logSegment <- removedLog.logSegments) {
+            logSegment.log.setFile(new File(renamedDir, logSegment.log.file.getName))
+            logSegment.index.file = new File(renamedDir, logSegment.index.file.getName)
+          }
 
-        logsToBeDeleted.add(removedLog)
-        removedLog.removeLogMetrics()
-        info(s"Log for partition ${removedLog.topicPartition} is renamed to ${removedLog.dir.getAbsolutePath} and is scheduled for deletion")
-      } else {
-        throw new KafkaStorageException("Failed to rename log directory from " + removedLog.dir.getAbsolutePath + " to " + renamedDir.getAbsolutePath)
+          logsToBeDeleted.add(removedLog)
+          removedLog.removeLogMetrics()
+          info(s"Log for partition ${removedLog.topicPartition} is renamed to ${removedLog.dir.getAbsolutePath} and is scheduled for deletion")
+        } else {
+          throw new IOException("Failed to rename log directory from " + removedLog.dir.getAbsolutePath + " to " + renamedDir.getAbsolutePath)
+        }
+      } catch {
+        case e: IOException =>
+          logDirFailureChannel.maybeAddLogFailureEvent(removedLog.dir.getParent)
+          throw new KafkaStorageException(s"Error while deleting $topicPartition in dir ${removedLog.dir.getParent}.", e)
       }
+    } else if (offlineLogDirs.nonEmpty) {
+      throw new KafkaStorageException("Failed to delete log for " + topicPartition + " because it may be in one of the offline directories " + offlineLogDirs.mkString(","))
     }
   }
 
@@ -503,14 +649,14 @@ class LogManager(val logDirs: Array[File],
    * data directory with the fewest partitions.
    */
   private def nextLogDir(): File = {
-    if(logDirs.size == 1) {
-      logDirs(0)
+    if(_liveLogDirs.size == 1) {
+      _liveLogDirs.peek()
     } else {
       // count the number of logs in each parent directory (including 0 for empty directories
       val logCounts = allLogs.groupBy(_.dir.getParent).mapValues(_.size)
-      val zeros = logDirs.map(dir => (dir.getPath, 0)).toMap
+      val zeros = _liveLogDirs.asScala.map(dir => (dir.getPath, 0)).toMap
       val dirCounts = (zeros ++ logCounts).toBuffer
-    
+
       // choose the directory with the least logs in it
       val leastLoaded = dirCounts.sortBy(_._2).head
       new File(leastLoaded._1)
@@ -552,6 +698,13 @@ class LogManager(val logDirs: Array[File],
     }
   }
 
+  def isLogDirOnline(logDir: String): Boolean = {
+    if (!logDirs.exists(_.getAbsolutePath == logDir))
+      throw new RuntimeException(s"Log dir $logDir is not found in the config.")
+
+    _liveLogDirs.contains(new File(logDir))
+  }
+
   /**
    * Flush any log which has exceeded its flush interval and has unwritten messages.
    */
@@ -575,11 +728,13 @@ class LogManager(val logDirs: Array[File],
 
 object LogManager {
   def apply(config: KafkaConfig,
+            initialOfflineDirs: Seq[String],
             zkUtils: ZkUtils,
             brokerState: BrokerState,
             kafkaScheduler: KafkaScheduler,
             time: Time,
-            brokerTopicStats: BrokerTopicStats): LogManager = {
+            brokerTopicStats: BrokerTopicStats,
+            logDirFailureChannel: LogDirFailureChannel): LogManager = {
     val defaultProps = KafkaServer.copyKafkaConfigToLog(config)
     val defaultLogConfig = LogConfig(defaultProps)
 
@@ -598,6 +753,7 @@ object LogManager {
       enableCleaner = config.logCleanerEnable)
 
     new LogManager(logDirs = config.logDirs.map(new File(_)).toArray,
+      initialOfflineDirs = initialOfflineDirs.map(new File(_)).toArray,
       topicConfigs = topicConfigs,
       defaultConfig = defaultLogConfig,
       cleanerConfig = cleanerConfig,
@@ -609,7 +765,8 @@ object LogManager {
       maxPidExpirationMs = config.transactionIdExpirationMs,
       scheduler = kafkaScheduler,
       brokerState = brokerState,
-      time = time,
-      brokerTopicStats = brokerTopicStats)
+      brokerTopicStats = brokerTopicStats,
+      logDirFailureChannel = logDirFailureChannel,
+      time = time)
   }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/log/LogSegment.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala
index 3e4c47d..0449a4a 100755
--- a/core/src/main/scala/kafka/log/LogSegment.scala
+++ b/core/src/main/scala/kafka/log/LogSegment.scala
@@ -20,8 +20,6 @@ import java.io.{File, IOException}
 import java.nio.file.Files
 import java.nio.file.attribute.FileTime
 import java.util.concurrent.TimeUnit
-
-import kafka.common._
 import kafka.metrics.{KafkaMetricsGroup, KafkaTimer}
 import kafka.server.epoch.LeaderEpochCache
 import kafka.server.{FetchDataInfo, LogOffsetMetadata}
@@ -383,28 +381,13 @@ class LogSegment(val log: FileRecords,
 
   /**
    * Change the suffix for the index and log file for this log segment
+   * IOException from this method should be handled by the caller
    */
   def changeFileSuffixes(oldSuffix: String, newSuffix: String) {
-
-    def kafkaStorageException(fileType: String, e: IOException) =
-      new KafkaStorageException(s"Failed to change the $fileType file suffix from $oldSuffix to $newSuffix for log segment $baseOffset", e)
-
-    try log.renameTo(new File(CoreUtils.replaceSuffix(log.file.getPath, oldSuffix, newSuffix)))
-    catch {
-      case e: IOException => throw kafkaStorageException("log", e)
-    }
-    try index.renameTo(new File(CoreUtils.replaceSuffix(index.file.getPath, oldSuffix, newSuffix)))
-    catch {
-      case e: IOException => throw kafkaStorageException("index", e)
-    }
-    try timeIndex.renameTo(new File(CoreUtils.replaceSuffix(timeIndex.file.getPath, oldSuffix, newSuffix)))
-    catch {
-      case e: IOException => throw kafkaStorageException("timeindex", e)
-    }
-    try txnIndex.renameTo(new File(CoreUtils.replaceSuffix(txnIndex.file.getPath, oldSuffix, newSuffix)))
-    catch {
-      case e: IOException => throw kafkaStorageException("txnindex", e)
-    }
+    log.renameTo(new File(CoreUtils.replaceSuffix(log.file.getPath, oldSuffix, newSuffix)))
+    index.renameTo(new File(CoreUtils.replaceSuffix(index.file.getPath, oldSuffix, newSuffix)))
+    timeIndex.renameTo(new File(CoreUtils.replaceSuffix(timeIndex.file.getPath, oldSuffix, newSuffix)))
+    txnIndex.renameTo(new File(CoreUtils.replaceSuffix(txnIndex.file.getPath, oldSuffix, newSuffix)))
   }
 
   /**
@@ -481,9 +464,17 @@ class LogSegment(val log: FileRecords,
   }
 
   /**
+    * Close file handlers used by the log segment but don't write to disk. This is used when the disk may have failed
+    */
+  def closeHandlers() {
+    CoreUtils.swallow(index.closeHandler())
+    CoreUtils.swallow(timeIndex.closeHandler())
+    CoreUtils.swallow(log.closeHandlers())
+    CoreUtils.swallow(txnIndex.close())
+  }
+
+  /**
    * Delete this log segment from the filesystem.
-   *
-   * @throws KafkaStorageException if the delete fails.
    */
   def delete() {
     val deletedLog = log.delete()
@@ -491,13 +482,13 @@ class LogSegment(val log: FileRecords,
     val deletedTimeIndex = timeIndex.delete()
     val deletedTxnIndex = txnIndex.delete()
     if (!deletedLog && log.file.exists)
-      throw new KafkaStorageException("Delete of log " + log.file.getName + " failed.")
+      throw new IOException("Delete of log " + log.file.getName + " failed.")
     if (!deletedIndex && index.file.exists)
-      throw new KafkaStorageException("Delete of index " + index.file.getName + " failed.")
+      throw new IOException("Delete of index " + index.file.getName + " failed.")
     if (!deletedTimeIndex && timeIndex.file.exists)
-      throw new KafkaStorageException("Delete of time index " + timeIndex.file.getName + " failed.")
+      throw new IOException("Delete of time index " + timeIndex.file.getName + " failed.")
     if (!deletedTxnIndex && txnIndex.file.exists)
-      throw new KafkaStorageException("Delete of transaction index " + txnIndex.file.getName + " failed.")
+      throw new IOException("Delete of transaction index " + txnIndex.file.getName + " failed.")
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/log/ProducerStateManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/ProducerStateManager.scala b/core/src/main/scala/kafka/log/ProducerStateManager.scala
index 7cc8e8e..ce56a6c 100644
--- a/core/src/main/scala/kafka/log/ProducerStateManager.scala
+++ b/core/src/main/scala/kafka/log/ProducerStateManager.scala
@@ -638,9 +638,13 @@ class ProducerStateManager(val topicPartition: TopicPartition,
   }
 
   private def listSnapshotFiles: List[File] = {
-    if (logDir.exists && logDir.isDirectory)
-      logDir.listFiles.filter(f => f.isFile && isSnapshotFile(f.getName)).toList
-    else
+    if (logDir.exists && logDir.isDirectory) {
+      val files = logDir.listFiles
+      if (files != null)
+        files.filter(f => f.isFile && isSnapshotFile(f.getName)).toList
+      else
+        List.empty[File]
+    } else
       List.empty[File]
   }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
index b17d255..9b01043 100755
--- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
@@ -18,22 +18,19 @@
 package kafka.server
 
 import java.util.concurrent.locks.ReentrantLock
-
 import kafka.cluster.BrokerEndPoint
-import kafka.consumer.PartitionTopicInfo
 import kafka.utils.{DelayedItem, Pool, ShutdownableThread}
+import org.apache.kafka.common.errors.KafkaStorageException
 import kafka.common.{ClientIdAndBroker, KafkaException}
 import kafka.metrics.KafkaMetricsGroup
 import kafka.utils.CoreUtils.inLock
 import org.apache.kafka.common.errors.CorruptRecordException
 import org.apache.kafka.common.protocol.Errors
 import AbstractFetcherThread._
-
 import scala.collection.{Map, Set, mutable}
 import scala.collection.JavaConverters._
 import java.util.concurrent.TimeUnit
 import java.util.concurrent.atomic.AtomicLong
-
 import com.yammer.metrics.core.Gauge
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.internals.{FatalExitError, PartitionStates}
@@ -198,7 +195,10 @@ abstract class AbstractFetcherThread(name: String,
                       // 2. If the message is corrupt due to a transient state in the log (truncation, partial writes can cause this), we simply continue and
                       // should get fixed in the subsequent fetches
                       logger.error("Found invalid messages during fetch for partition [" + topic + "," + partitionId + "] offset " + currentPartitionFetchState.fetchOffset  + " error " + ime.getMessage)
-                      updatePartitionsWithError(topicPartition);
+                      updatePartitionsWithError(topicPartition)
+                    case e: KafkaStorageException =>
+                      logger.error(s"Error while processing data for partition $topicPartition", e)
+                      updatePartitionsWithError(topicPartition)
                     case e: Throwable =>
                       throw new KafkaException("error processing data for partition [%s,%d] offset %d"
                         .format(topic, partitionId, currentPartitionFetchState.fetchOffset), e)

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala b/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala
index 8630026..8ac9864 100755
--- a/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala
+++ b/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala
@@ -30,7 +30,6 @@ case class BrokerMetadata(brokerId: Int)
   */
 class BrokerMetadataCheckpoint(val file: File) extends Logging {
   private val lock = new Object()
-  Files.deleteIfExists(new File(file + ".tmp").toPath()) // try to delete any existing temp files for cleanliness
 
   def write(brokerMetadata: BrokerMetadata) = {
     lock synchronized {
@@ -57,6 +56,8 @@ class BrokerMetadataCheckpoint(val file: File) extends Logging {
   }
 
   def read(): Option[BrokerMetadata] = {
+    Files.deleteIfExists(new File(file + ".tmp").toPath()) // try to delete any existing temp files for cleanliness
+
     lock synchronized {
       try {
         val brokerMetaProps = new VerifiableProperties(Utils.loadProps(file.getAbsolutePath()))

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/server/DelayedDeleteRecords.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/DelayedDeleteRecords.scala b/core/src/main/scala/kafka/server/DelayedDeleteRecords.scala
index e5b301c..a6a8202 100644
--- a/core/src/main/scala/kafka/server/DelayedDeleteRecords.scala
+++ b/core/src/main/scala/kafka/server/DelayedDeleteRecords.scala
@@ -20,9 +20,7 @@ package kafka.server
 
 import java.util.concurrent.TimeUnit
 
-import com.yammer.metrics.core.Meter
 import kafka.metrics.KafkaMetricsGroup
-import kafka.utils.Pool
 import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.requests.DeleteRecordsResponse
@@ -76,12 +74,16 @@ class DelayedDeleteRecords(delayMs: Long,
       if (status.acksPending) {
         val (lowWatermarkReached, error, lw) = replicaManager.getPartition(topicPartition) match {
           case Some(partition) =>
-            partition.leaderReplicaIfLocal match {
-              case Some(_) =>
-                val leaderLW = partition.lowWatermarkIfLeader
-                (leaderLW >= status.requiredOffset, Errors.NONE, leaderLW)
-              case None =>
-                (false, Errors.NOT_LEADER_FOR_PARTITION, DeleteRecordsResponse.INVALID_LOW_WATERMARK)
+            if (partition eq ReplicaManager.OfflinePartition) {
+              (false, Errors.KAFKA_STORAGE_ERROR, DeleteRecordsResponse.INVALID_LOW_WATERMARK)
+            } else {
+              partition.leaderReplicaIfLocal match {
+                case Some(_) =>
+                  val leaderLW = partition.lowWatermarkIfLeader
+                  (leaderLW >= status.requiredOffset, Errors.NONE, leaderLW)
+                case None =>
+                  (false, Errors.NOT_LEADER_FOR_PARTITION, DeleteRecordsResponse.INVALID_LOW_WATERMARK)
+              }
             }
           case None =>
             (false, Errors.UNKNOWN_TOPIC_OR_PARTITION, DeleteRecordsResponse.INVALID_LOW_WATERMARK)

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/server/DelayedFetch.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala
index 8a9ce02..e478053 100644
--- a/core/src/main/scala/kafka/server/DelayedFetch.scala
+++ b/core/src/main/scala/kafka/server/DelayedFetch.scala
@@ -21,7 +21,7 @@ import java.util.concurrent.TimeUnit
 
 import kafka.metrics.KafkaMetricsGroup
 import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.common.errors.{NotLeaderForPartitionException, UnknownTopicOrPartitionException}
+import org.apache.kafka.common.errors.{NotLeaderForPartitionException, UnknownTopicOrPartitionException, KafkaStorageException}
 import org.apache.kafka.common.requests.FetchRequest.PartitionData
 import org.apache.kafka.common.requests.IsolationLevel
 
@@ -71,6 +71,7 @@ class DelayedFetch(delayMs: Long,
    * Case B: This broker does not know of some partitions it tries to fetch
    * Case C: The fetch offset locates not on the last segment of the log
    * Case D: The accumulated bytes from all the fetching partitions exceeds the minimum bytes
+   * Case E: The partition is in an offline log directory on this broker
    *
    * Upon completion, should return whatever data is available for each valid partition
    */
@@ -117,6 +118,9 @@ class DelayedFetch(delayMs: Long,
             }
           }
         } catch {
+          case _: KafkaStorageException => // Case E
+            debug("Partition %s is in an offline log directory, satisfy %s immediately".format(topicPartition, fetchMetadata))
+            return forceComplete()
           case _: UnknownTopicOrPartitionException => // Case B
             debug("Broker no longer know of %s, satisfy %s immediately".format(topicPartition, fetchMetadata))
             return forceComplete()

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/server/DelayedProduce.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala
index 0ff8d34..0d452cc 100644
--- a/core/src/main/scala/kafka/server/DelayedProduce.scala
+++ b/core/src/main/scala/kafka/server/DelayedProduce.scala
@@ -95,7 +95,10 @@ class DelayedProduce(delayMs: Long,
       if (status.acksPending) {
         val (hasEnough, error) = replicaManager.getPartition(topicPartition) match {
           case Some(partition) =>
-            partition.checkEnoughReplicasReachOffset(status.requiredOffset)
+            if (partition eq ReplicaManager.OfflinePartition)
+              (false, Errors.KAFKA_STORAGE_ERROR)
+            else
+              partition.checkEnoughReplicasReachOffset(status.requiredOffset)
           case None =>
             // Case A
             (false, Errors.UNKNOWN_TOPIC_OR_PARTITION)

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/server/KafkaApis.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala
index 9e9299f..1fb8901 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -27,7 +27,7 @@ import java.util.concurrent.atomic.AtomicInteger
 import kafka.admin.{AdminUtils, RackAwareMode}
 import kafka.api.{ApiVersion, ControlledShutdownRequest, ControlledShutdownResponse, KAFKA_0_11_0_IV0}
 import kafka.cluster.Partition
-import kafka.common.{KafkaStorageException, OffsetAndMetadata, OffsetMetadata, TopicAndPartition}
+import kafka.common.{OffsetAndMetadata, OffsetMetadata, TopicAndPartition}
 import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota}
 import kafka.controller.KafkaController
 import kafka.coordinator.group.{GroupCoordinator, JoinGroupResult}
@@ -36,7 +36,7 @@ import kafka.log.{Log, LogManager, TimestampOffset}
 import kafka.network.{RequestChannel, RequestOrResponseSend}
 import kafka.security.SecurityUtils
 import kafka.security.auth._
-import kafka.utils.{CoreUtils, Exit, Logging, ZKGroupTopicDirs, ZkUtils}
+import kafka.utils.{CoreUtils, Logging, ZKGroupTopicDirs, ZkUtils}
 import org.apache.kafka.common.errors._
 import org.apache.kafka.common.internals.FatalExitError
 import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME, isInternal}
@@ -54,7 +54,7 @@ import org.apache.kafka.common.requests.SaslHandshakeResponse
 import org.apache.kafka.common.resource.{Resource => AdminResource}
 import org.apache.kafka.common.acl.{AccessControlEntry, AclBinding}
 
-import scala.collection._
+import scala.collection.{mutable, _}
 import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
 import scala.util.{Failure, Success, Try}
@@ -144,40 +144,33 @@ class KafkaApis(val requestChannel: RequestChannel,
     val correlationId = request.header.correlationId
     val leaderAndIsrRequest = request.body[LeaderAndIsrRequest]
 
-    try {
-      def onLeadershipChange(updatedLeaders: Iterable[Partition], updatedFollowers: Iterable[Partition]) {
-        // for each new leader or follower, call coordinator to handle consumer group migration.
-        // this callback is invoked under the replica state change lock to ensure proper order of
-        // leadership changes
-        updatedLeaders.foreach { partition =>
-          if (partition.topic == GROUP_METADATA_TOPIC_NAME)
-            groupCoordinator.handleGroupImmigration(partition.partitionId)
-          else if (partition.topic == TRANSACTION_STATE_TOPIC_NAME)
-            txnCoordinator.handleTxnImmigration(partition.partitionId, partition.getLeaderEpoch)
-        }
-
-        updatedFollowers.foreach { partition =>
-          if (partition.topic == GROUP_METADATA_TOPIC_NAME)
-            groupCoordinator.handleGroupEmigration(partition.partitionId)
-          else if (partition.topic == TRANSACTION_STATE_TOPIC_NAME)
-            txnCoordinator.handleTxnEmigration(partition.partitionId, partition.getLeaderEpoch)
-        }
+    def onLeadershipChange(updatedLeaders: Iterable[Partition], updatedFollowers: Iterable[Partition]) {
+      // for each new leader or follower, call coordinator to handle consumer group migration.
+      // this callback is invoked under the replica state change lock to ensure proper order of
+      // leadership changes
+      updatedLeaders.foreach { partition =>
+        if (partition.topic == GROUP_METADATA_TOPIC_NAME)
+          groupCoordinator.handleGroupImmigration(partition.partitionId)
+        else if (partition.topic == TRANSACTION_STATE_TOPIC_NAME)
+          txnCoordinator.handleTxnImmigration(partition.partitionId, partition.getLeaderEpoch)
       }
 
-      if (authorize(request.session, ClusterAction, Resource.ClusterResource)) {
-        val result = replicaManager.becomeLeaderOrFollower(correlationId, leaderAndIsrRequest, onLeadershipChange)
-        val leaderAndIsrResponse = new LeaderAndIsrResponse(result.error, result.responseMap.asJava)
-        sendResponseExemptThrottle(RequestChannel.Response(request, leaderAndIsrResponse))
-      } else {
-        val result = leaderAndIsrRequest.partitionStates.asScala.keys.map((_, Errors.CLUSTER_AUTHORIZATION_FAILED)).toMap
-        sendResponseMaybeThrottle(request, _ =>
-          new LeaderAndIsrResponse(Errors.CLUSTER_AUTHORIZATION_FAILED, result.asJava))
+      updatedFollowers.foreach { partition =>
+        if (partition.topic == GROUP_METADATA_TOPIC_NAME)
+          groupCoordinator.handleGroupEmigration(partition.partitionId)
+        else if (partition.topic == TRANSACTION_STATE_TOPIC_NAME)
+          txnCoordinator.handleTxnEmigration(partition.partitionId, partition.getLeaderEpoch)
       }
-    } catch {
-      case e: FatalExitError => throw e
-      case e: KafkaStorageException =>
-        fatal("Disk error during leadership change.", e)
-        Exit.halt(1)
+    }
+
+    if (authorize(request.session, ClusterAction, Resource.ClusterResource)) {
+      val result = replicaManager.becomeLeaderOrFollower(correlationId, leaderAndIsrRequest, onLeadershipChange)
+      val leaderAndIsrResponse = new LeaderAndIsrResponse(result.error, result.responseMap.asJava)
+      sendResponseExemptThrottle(RequestChannel.Response(request, leaderAndIsrResponse))
+    } else {
+      val result = leaderAndIsrRequest.partitionStates.asScala.keys.map((_, Errors.CLUSTER_AUTHORIZATION_FAILED)).toMap
+      sendResponseMaybeThrottle(request, _ =>
+        new LeaderAndIsrResponse(Errors.CLUSTER_AUTHORIZATION_FAILED, result.asJava))
     }
   }
 
@@ -681,7 +674,9 @@ class KafkaApis(val requestChannel: RequestChannel,
       } catch {
         // NOTE: UnknownTopicOrPartitionException and NotLeaderForPartitionException are special cased since these error messages
         // are typically transient and there is no value in logging the entire stack trace for the same
-        case e @ ( _ : UnknownTopicOrPartitionException | _ : NotLeaderForPartitionException) =>
+        case e @ (_ : UnknownTopicOrPartitionException |
+                  _ : NotLeaderForPartitionException |
+                  _ : KafkaStorageException) =>
           debug("Offset request with correlation id %d from client %s on partition %s failed due to %s".format(
             correlationId, clientId, topicPartition, e.getMessage))
           (topicPartition, new ListOffsetResponse.PartitionData(Errors.forException(e), List[JLong]().asJava))
@@ -750,6 +745,7 @@ class KafkaApis(val requestChannel: RequestChannel,
           // would have received a clear exception and there is no value in logging the entire stack trace for the same
           case e @ (_ : UnknownTopicOrPartitionException |
                     _ : NotLeaderForPartitionException |
+                    _ : KafkaStorageException |
                     _ : UnsupportedForMessageFormatException) =>
             debug(s"Offset request with correlation id $correlationId from client $clientId on " +
                 s"partition $topicPartition failed due to ${e.getMessage}")
@@ -1527,7 +1523,7 @@ class KafkaApis(val requestChannel: RequestChannel,
           case e: Exception =>
             error(s"Received an exception while trying to update the offsets cache on transaction marker append", e)
             val updatedErrors = new ConcurrentHashMap[TopicPartition, Errors]()
-            successfulOffsetsPartitions.foreach(updatedErrors.put(_, Errors.UNKNOWN))
+            successfulOffsetsPartitions.foreach(updatedErrors.put(_, Errors.UNKNOWN_SERVER_ERROR))
             updateErrors(producerId, updatedErrors)
         }
       }
@@ -1865,7 +1861,7 @@ class KafkaApis(val requestChannel: RequestChannel,
                 Some(new AclDeletionResult(ApiError.fromThrowable(throwable), aclBinding))
             }
           }.asJava
-          
+
           filterResponseMap.put(i, new AclFilterResponse(deletionResults))
         }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/server/KafkaServer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala
index cc34e14..fc9e4b8 100755
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -47,7 +47,7 @@ import org.apache.kafka.common.utils.{AppInfoParser, Time}
 import org.apache.kafka.common.{ClusterResource, Node}
 
 import scala.collection.JavaConverters._
-import scala.collection.{Map, mutable}
+import scala.collection.{Seq, Map, mutable}
 
 object KafkaServer {
   // Copy the subset of properties that are relevant to Logs
@@ -110,6 +110,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP
   var socketServer: SocketServer = null
   var requestHandlerPool: KafkaRequestHandlerPool = null
 
+  var logDirFailureChannel: LogDirFailureChannel = null
   var logManager: LogManager = null
 
   var replicaManager: ReplicaManager = null
@@ -195,7 +196,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP
         info(s"Cluster ID = $clusterId")
 
         /* generate brokerId */
-        config.brokerId =  getBrokerId
+        val (brokerId, initialOfflineDirs) = getBrokerIdAndOfflineDirs
+        config.brokerId = brokerId
         this.logIdent = "[Kafka Server " + config.brokerId + "], "
 
         /* create and configure metrics */
@@ -211,8 +213,10 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP
         quotaManagers = QuotaFactory.instantiate(config, metrics, time)
         notifyClusterListeners(kafkaMetricsReporters ++ reporters.asScala)
 
+        logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size)
+
         /* start log manager */
-        logManager = LogManager(config, zkUtils, brokerState, kafkaScheduler, time, brokerTopicStats)
+        logManager = LogManager(config, initialOfflineDirs, zkUtils, brokerState, kafkaScheduler, time, brokerTopicStats, logDirFailureChannel)
         logManager.startup()
 
         metadataCache = new MetadataCache(config.brokerId)
@@ -307,7 +311,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP
 
   protected def createReplicaManager(isShuttingDown: AtomicBoolean): ReplicaManager =
     new ReplicaManager(config, metrics, time, zkUtils, kafkaScheduler, logManager, isShuttingDown, quotaManagers.follower,
-      brokerTopicStats, metadataCache)
+      brokerTopicStats, metadataCache, logDirFailureChannel)
 
   private def initZk(): ZkUtils = {
     info(s"Connecting to zookeeper on ${config.zkConnect}")
@@ -582,7 +586,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP
       if (shutdownLatch.getCount > 0 && isShuttingDown.compareAndSet(false, true)) {
         CoreUtils.swallow(controlledShutdown())
         brokerState.newState(BrokerShuttingDown)
-        
+
         if (socketServer != null)
           CoreUtils.swallow(socketServer.shutdown())
         if (requestHandlerPool != null)
@@ -651,16 +655,25 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP
     * <li> config has broker.id and there is no meta.properties file, creates new meta.properties and stores broker.id
     * <ol>
     *
-    * @return A brokerId.
+    * The log directories whose meta.properties can not be accessed due to IOException will be returned to the caller
+    *
+    * @return A 2-tuple containing the brokerId and a sequence of offline log directories.
     */
-  private def getBrokerId: Int =  {
+  private def getBrokerIdAndOfflineDirs: (Int, Seq[String]) =  {
     var brokerId = config.brokerId
     val brokerIdSet = mutable.HashSet[Int]()
+    val offlineDirs = mutable.ArrayBuffer.empty[String]
 
     for (logDir <- config.logDirs) {
-      val brokerMetadataOpt = brokerMetadataCheckpoints(logDir).read()
-      brokerMetadataOpt.foreach { brokerMetadata =>
-        brokerIdSet.add(brokerMetadata.brokerId)
+      try {
+        val brokerMetadataOpt = brokerMetadataCheckpoints(logDir).read()
+        brokerMetadataOpt.foreach { brokerMetadata =>
+          brokerIdSet.add(brokerMetadata.brokerId)
+        }
+      } catch {
+        case e : IOException =>
+          offlineDirs += logDir
+          error(s"Fail to read ${brokerMetaPropsFile} under log directory ${logDir}", e)
       }
     }
 
@@ -678,16 +691,17 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP
     else if(brokerIdSet.size == 1) // pick broker.id from meta.properties
       brokerId = brokerIdSet.last
 
-    brokerId
+
+    (brokerId, offlineDirs)
   }
 
   private def checkpointBrokerId(brokerId: Int) {
     var logDirsWithoutMetaProps: List[String] = List()
 
-    for (logDir <- config.logDirs) {
-      val brokerMetadataOpt = brokerMetadataCheckpoints(logDir).read()
+    for (logDir <- logManager.liveLogDirs) {
+      val brokerMetadataOpt = brokerMetadataCheckpoints(logDir.getAbsolutePath).read()
       if(brokerMetadataOpt.isEmpty)
-          logDirsWithoutMetaProps ++= List(logDir)
+        logDirsWithoutMetaProps ++= List(logDir.getAbsolutePath)
     }
 
     for(logDir <- logDirsWithoutMetaProps) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/server/LogDirFailureChannel.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/LogDirFailureChannel.scala b/core/src/main/scala/kafka/server/LogDirFailureChannel.scala
new file mode 100644
index 0000000..23d9986
--- /dev/null
+++ b/core/src/main/scala/kafka/server/LogDirFailureChannel.scala
@@ -0,0 +1,55 @@
+/**
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *    http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+
+
+package kafka.server
+
+import java.util.concurrent.{ArrayBlockingQueue, ConcurrentHashMap}
+
+/*
+ * LogDirFailureChannel allows an external thread to block waiting for new offline log dir.
+ *
+ * LogDirFailureChannel should be a singleton object which can be accessed by any class that does disk-IO operation.
+ * If IOException is encountered while accessing a log directory, the corresponding class can insert the the log directory name
+ * to the LogDirFailureChannel using maybeAddLogFailureEvent(). Then a thread which is blocked waiting for new offline log directories
+ * can take the name of the new offline log directory out of the LogDirFailureChannel and handles the log failure properly.
+ *
+ */
+class LogDirFailureChannel(logDirNum: Int) {
+
+  private val offlineLogDirs = new ConcurrentHashMap[String, String]
+  private val logDirFailureEvent = new ArrayBlockingQueue[String](logDirNum)
+
+  /*
+   * If the given logDir is not already offline, add it to the
+   * set of offline log dirs and enqueue it to the logDirFailureEvent queue
+   */
+  def maybeAddLogFailureEvent(logDir: String): Unit = {
+    if (offlineLogDirs.putIfAbsent(logDir, logDir) == null) {
+      logDirFailureEvent.add(logDir)
+    }
+  }
+
+  /*
+   * Get the next offline log dir from logDirFailureEvent queue.
+   * The method will wait if necessary until a new offline log directory becomes available
+   */
+  def takeNextLogFailureEvent(): String = {
+    logDirFailureEvent.take()
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/server/MetadataCache.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala
index 466645b..2c28df7 100755
--- a/core/src/main/scala/kafka/server/MetadataCache.scala
+++ b/core/src/main/scala/kafka/server/MetadataCache.scala
@@ -31,7 +31,7 @@ import org.apache.kafka.common.internals.Topic
 import org.apache.kafka.common.{Node, TopicPartition}
 import org.apache.kafka.common.network.ListenerName
 import org.apache.kafka.common.protocol.Errors
-import org.apache.kafka.common.requests.{MetadataResponse, PartitionState, UpdateMetadataRequest}
+import org.apache.kafka.common.requests.{MetadataResponse, UpdateMetadataRequest}
 
 /**
  *  A cache for the state (e.g., current leader) of each partition. This cache is updated through
@@ -39,7 +39,7 @@ import org.apache.kafka.common.requests.{MetadataResponse, PartitionState, Updat
  */
 class MetadataCache(brokerId: Int) extends Logging {
   private val stateChangeLogger = KafkaController.stateChangeLogger
-  private val cache = mutable.Map[String, mutable.Map[Int, PartitionStateInfo]]()
+  private val cache = mutable.Map[String, mutable.Map[Int, MetadataPartitionState]]()
   private var controllerId: Option[Int] = None
   private val aliveBrokers = mutable.Map[Int, Broker]()
   private val aliveNodes = mutable.Map[Int, collection.Map[ListenerName, Node]]()
@@ -73,12 +73,13 @@ class MetadataCache(brokerId: Int) extends Logging {
 
         val replicas = partitionState.allReplicas
         val replicaInfo = getEndpoints(replicas, listenerName, errorUnavailableEndpoints)
+        val offlineReplicaInfo = getEndpoints(partitionState.offlineReplicas, listenerName, errorUnavailableEndpoints)
 
         maybeLeader match {
           case None =>
             debug(s"Error while fetching metadata for $topicPartition: leader not available")
             new MetadataResponse.PartitionMetadata(Errors.LEADER_NOT_AVAILABLE, partitionId, Node.noNode(),
-              replicaInfo.asJava, java.util.Collections.emptyList())
+              replicaInfo.asJava, java.util.Collections.emptyList(), offlineReplicaInfo.asJava)
 
           case Some(leader) =>
             val isr = leaderAndIsr.isr
@@ -89,15 +90,15 @@ class MetadataCache(brokerId: Int) extends Logging {
                 s"following brokers ${replicas.filterNot(replicaInfo.map(_.id).contains).mkString(",")}")
 
               new MetadataResponse.PartitionMetadata(Errors.REPLICA_NOT_AVAILABLE, partitionId, leader,
-                replicaInfo.asJava, isrInfo.asJava)
+                replicaInfo.asJava, isrInfo.asJava, offlineReplicaInfo.asJava)
             } else if (isrInfo.size < isr.size) {
               debug(s"Error while fetching metadata for $topicPartition: in sync replica information not available for " +
                 s"following brokers ${isr.filterNot(isrInfo.map(_.id).contains).mkString(",")}")
               new MetadataResponse.PartitionMetadata(Errors.REPLICA_NOT_AVAILABLE, partitionId, leader,
-                replicaInfo.asJava, isrInfo.asJava)
+                replicaInfo.asJava, isrInfo.asJava, offlineReplicaInfo.asJava)
             } else {
               new MetadataResponse.PartitionMetadata(Errors.NONE, partitionId, leader, replicaInfo.asJava,
-                isrInfo.asJava)
+                isrInfo.asJava, offlineReplicaInfo.asJava)
             }
         }
       }
@@ -147,14 +148,14 @@ class MetadataCache(brokerId: Int) extends Logging {
 
   private def addOrUpdatePartitionInfo(topic: String,
                                        partitionId: Int,
-                                       stateInfo: PartitionStateInfo) {
+                                       stateInfo: MetadataPartitionState) {
     inWriteLock(partitionMetadataLock) {
       val infos = cache.getOrElseUpdate(topic, mutable.Map())
       infos(partitionId) = stateInfo
     }
   }
 
-  def getPartitionInfo(topic: String, partitionId: Int): Option[PartitionStateInfo] = {
+  def getPartitionInfo(topic: String, partitionId: Int): Option[MetadataPartitionState] = {
     inReadLock(partitionMetadataLock) {
       cache.get(topic).flatMap(_.get(partitionId))
     }
@@ -223,10 +224,10 @@ class MetadataCache(brokerId: Int) extends Logging {
     }
   }
 
-  private def partitionStateToPartitionStateInfo(partitionState: PartitionState): PartitionStateInfo = {
+  private def partitionStateToPartitionStateInfo(partitionState: UpdateMetadataRequest.PartitionState): MetadataPartitionState = {
     val leaderAndIsr = LeaderAndIsr(partitionState.leader, partitionState.leaderEpoch, partitionState.isr.asScala.map(_.toInt).toList, partitionState.zkVersion)
     val leaderInfo = LeaderIsrAndControllerEpoch(leaderAndIsr, partitionState.controllerEpoch)
-    PartitionStateInfo(leaderInfo, partitionState.replicas.asScala.map(_.toInt))
+    MetadataPartitionState(leaderInfo, partitionState.replicas.asScala.map(_.toInt), partitionState.offlineReplicas.asScala.map(_.toInt))
   }
 
   def contains(topic: String): Boolean = {


[5/5] kafka git commit: KAFKA-4763; Handle disk failure for JBOD (KIP-112)

Posted by jq...@apache.org.
KAFKA-4763; Handle disk failure for JBOD (KIP-112)

Author: Dong Lin <li...@gmail.com>

Reviewers: Jiangjie Qin <be...@gmail.com>, Jun Rao <ju...@gmail.com>, Ismael Juma <is...@juma.me.uk>, Onur Karaman <ok...@linkedin.com>

Closes #2929 from lindong28/KAFKA-4763


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/fc93fb4b
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/fc93fb4b
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/fc93fb4b

Branch: refs/heads/trunk
Commit: fc93fb4b6116e809f2b69ddb2f7e0f12548fef51
Parents: 91b5fc7
Author: Dong Lin <li...@gmail.com>
Authored: Sat Jul 22 12:35:32 2017 -0700
Committer: Jiangjie Qin <be...@gmail.com>
Committed: Sat Jul 22 12:35:32 2017 -0700

----------------------------------------------------------------------
 .../clients/consumer/internals/Fetcher.java     |   5 +-
 .../org/apache/kafka/common/PartitionInfo.java  |  21 +-
 .../common/errors/KafkaStorageException.java    |  50 ++
 .../apache/kafka/common/protocol/Errors.java    |  21 +-
 .../apache/kafka/common/protocol/Protocol.java  | 138 +++++-
 .../apache/kafka/common/record/FileRecords.java |   7 +
 .../kafka/common/requests/FetchResponse.java    |  11 +-
 .../common/requests/LeaderAndIsrRequest.java    |  12 +-
 .../kafka/common/requests/MetadataRequest.java  |   1 +
 .../kafka/common/requests/MetadataResponse.java |  57 ++-
 .../kafka/common/requests/PartitionState.java   |   8 +-
 .../kafka/common/requests/ProduceRequest.java   |   2 +
 .../kafka/common/requests/ProduceResponse.java  |   9 +-
 .../common/requests/UpdateMetadataRequest.java  |  58 ++-
 .../internals/AbstractCoordinatorTest.java      |   4 +-
 .../internals/ConsumerCoordinatorTest.java      |   4 +-
 .../clients/consumer/internals/FetcherTest.java |   3 +-
 .../internals/TransactionManagerTest.java       |   2 +-
 .../apache/kafka/common/PartitionInfoTest.java  |  15 +-
 .../kafka/common/protocol/ErrorsTest.java       |   4 +-
 .../common/requests/RequestResponseTest.java    |  21 +-
 .../main/scala/kafka/admin/AdminClient.scala    |   2 +-
 .../src/main/scala/kafka/admin/AdminUtils.scala |   2 +-
 core/src/main/scala/kafka/api/ApiVersion.scala  |  12 +-
 .../src/main/scala/kafka/api/LeaderAndIsr.scala |  15 +-
 .../main/scala/kafka/cluster/Partition.scala    | 119 +++--
 .../kafka/common/KafkaStorageException.scala    |   6 +-
 .../controller/ControllerChannelManager.scala   |  51 +-
 .../kafka/controller/ControllerState.scala      |  11 +-
 .../kafka/controller/KafkaController.scala      | 169 +++++--
 .../controller/PartitionLeaderSelector.scala    |  13 +-
 .../controller/PartitionStateMachine.scala      |   7 +-
 .../kafka/controller/ReplicaStateMachine.scala  |   5 +-
 .../kafka/controller/TopicDeletionManager.scala |   2 +-
 .../group/GroupMetadataManager.scala            |  10 +-
 .../transaction/TransactionCoordinator.scala    |   2 +-
 .../transaction/TransactionStateManager.scala   |   5 +-
 .../main/scala/kafka/log/AbstractIndex.scala    |   6 +
 core/src/main/scala/kafka/log/Log.scala         | 495 ++++++++++---------
 core/src/main/scala/kafka/log/LogCleaner.scala  | 107 ++--
 .../scala/kafka/log/LogCleanerManager.scala     |  59 ++-
 core/src/main/scala/kafka/log/LogManager.scala  | 491 +++++++++++-------
 core/src/main/scala/kafka/log/LogSegment.scala  |  47 +-
 .../scala/kafka/log/ProducerStateManager.scala  |  10 +-
 .../kafka/server/AbstractFetcherThread.scala    |  10 +-
 .../kafka/server/BrokerMetadataCheckpoint.scala |   3 +-
 .../kafka/server/DelayedDeleteRecords.scala     |  18 +-
 .../main/scala/kafka/server/DelayedFetch.scala  |   6 +-
 .../scala/kafka/server/DelayedProduce.scala     |   5 +-
 .../src/main/scala/kafka/server/KafkaApis.scala |  70 ++-
 .../main/scala/kafka/server/KafkaServer.scala   |  42 +-
 .../kafka/server/LogDirFailureChannel.scala     |  55 +++
 .../main/scala/kafka/server/MetadataCache.scala |  21 +-
 .../kafka/server/ReplicaFetcherThread.scala     |  68 ++-
 .../scala/kafka/server/ReplicaManager.scala     | 296 ++++++++---
 .../server/checkpoints/CheckpointFile.scala     | 124 ++---
 .../checkpoints/LeaderEpochCheckpointFile.scala |   5 +-
 .../checkpoints/OffsetCheckpointFile.scala      |   5 +-
 .../main/scala/kafka/utils/LogDirUtils.scala    |  66 +++
 core/src/main/scala/kafka/utils/ZkUtils.scala   |   7 +-
 .../kafka/api/AuthorizerIntegrationTest.scala   |   8 +-
 .../kafka/api/IntegrationTestHarness.scala      |   5 +-
 .../kafka/api/LogDirFailureTest.scala           | 126 +++++
 .../kafka/api/TransactionsTest.scala            |   4 +-
 .../ReplicaFetcherThreadFatalErrorTest.scala    |   2 +-
 .../group/GroupMetadataManagerTest.scala        |  10 +-
 ...tionMarkerRequestCompletionHandlerTest.scala |   2 +-
 .../TransactionStateManagerTest.scala           |   2 +-
 .../log/AbstractLogCleanerIntegrationTest.scala |   3 +-
 .../kafka/log/LogCleanerIntegrationTest.scala   |   2 +-
 .../unit/kafka/log/LogCleanerManagerTest.scala  |   2 +-
 .../scala/unit/kafka/log/LogCleanerTest.scala   |  10 +-
 .../scala/unit/kafka/log/LogManagerTest.scala   |  22 +-
 .../scala/unit/kafka/log/LogSegmentTest.scala   |   4 +-
 .../src/test/scala/unit/kafka/log/LogTest.scala |  12 +-
 .../server/HighwatermarkPersistenceTest.scala   |  26 +-
 .../unit/kafka/server/ISRExpirationTest.scala   |  14 +-
 .../unit/kafka/server/LeaderElectionTest.scala  |   4 +-
 .../scala/unit/kafka/server/LogOffsetTest.scala |   4 +-
 .../unit/kafka/server/MetadataCacheTest.scala   |  18 +-
 .../kafka/server/ReplicaFetcherThreadTest.scala |   6 +-
 .../kafka/server/ReplicaManagerQuotasTest.scala |   4 +-
 .../unit/kafka/server/ReplicaManagerTest.scala  |  43 +-
 .../unit/kafka/server/RequestQuotaTest.scala    |   7 +-
 .../unit/kafka/server/SimpleFetchTest.scala     |   5 +-
 .../checkpoints/OffsetCheckpointFileTest.scala  |  14 +-
 .../epoch/OffsetsForLeaderEpochTest.scala       |  26 +-
 .../test/scala/unit/kafka/utils/TestUtils.scala |  20 +-
 docs/upgrade.html                               |  57 ++-
 .../integration/utils/IntegrationTestUtils.java |   8 +-
 .../internals/InternalTopicManagerTest.java     |   2 +-
 tests/kafkatest/services/kafka/config.py        |   2 +-
 .../kafkatest/services/kafka/config_property.py |   4 +
 tests/kafkatest/services/kafka/kafka.py         |  59 ++-
 tests/kafkatest/services/verifiable_producer.py |   5 +-
 .../tests/core/log_dir_failure_test.py          | 177 +++++++
 tests/kafkatest/tests/core/transactions_test.py |   3 +-
 97 files changed, 2526 insertions(+), 1096 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
index f5ebac5..87f04b7 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
@@ -864,7 +864,8 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
                     log.trace("Updating last stable offset for partition {} to {}", tp, partition.lastStableOffset);
                     subscriptions.updateLastStableOffset(tp, partition.lastStableOffset);
                 }
-            } else if (error == Errors.NOT_LEADER_FOR_PARTITION) {
+            } else if (error == Errors.NOT_LEADER_FOR_PARTITION ||
+                       error == Errors.KAFKA_STORAGE_ERROR) {
                 log.debug("Error in fetch for partition {}: {}", tp, error.exceptionName());
                 this.metadata.requestUpdate();
             } else if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION) {
@@ -884,7 +885,7 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
             } else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) {
                 log.warn("Not authorized to read from topic {}.", tp.topic());
                 throw new TopicAuthorizationException(Collections.singleton(tp.topic()));
-            } else if (error == Errors.UNKNOWN) {
+            } else if (error == Errors.UNKNOWN_SERVER_ERROR) {
                 log.warn("Unknown error fetching data for topic-partition {}", tp);
             } else {
                 throw new IllegalStateException("Unexpected error code " + error.code() + " while fetching data");

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java
index b351116..38e4f67 100644
--- a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java
+++ b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java
@@ -17,7 +17,7 @@
 package org.apache.kafka.common;
 
 /**
- * Information about a topic-partition.
+ * This is used to describe per-partition state in the MetadataResponse.
  */
 public class PartitionInfo {
 
@@ -26,13 +26,20 @@ public class PartitionInfo {
     private final Node leader;
     private final Node[] replicas;
     private final Node[] inSyncReplicas;
+    private final Node[] offlineReplicas;
 
+    // Used only by tests
     public PartitionInfo(String topic, int partition, Node leader, Node[] replicas, Node[] inSyncReplicas) {
+        this(topic, partition, leader, replicas, inSyncReplicas, new Node[0]);
+    }
+
+    public PartitionInfo(String topic, int partition, Node leader, Node[] replicas, Node[] inSyncReplicas, Node[] offlineReplicas) {
         this.topic = topic;
         this.partition = partition;
         this.leader = leader;
         this.replicas = replicas;
         this.inSyncReplicas = inSyncReplicas;
+        this.offlineReplicas = offlineReplicas;
     }
 
     /**
@@ -71,14 +78,22 @@ public class PartitionInfo {
         return inSyncReplicas;
     }
 
+    /**
+     * The subset of the replicas that are offline
+     */
+    public Node[] offlineReplicas() {
+        return offlineReplicas;
+    }
+
     @Override
     public String toString() {
-        return String.format("Partition(topic = %s, partition = %d, leader = %s, replicas = %s, isr = %s)",
+        return String.format("Partition(topic = %s, partition = %d, leader = %s, replicas = %s, isr = %s, offlineReplicas = %s)",
                              topic,
                              partition,
                              leader == null ? "none" : leader.idString(),
                              formatNodeIds(replicas),
-                             formatNodeIds(inSyncReplicas));
+                             formatNodeIds(inSyncReplicas),
+                             formatNodeIds(offlineReplicas));
     }
 
     /* Extract the node ids from each item in the array and format for display */

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/main/java/org/apache/kafka/common/errors/KafkaStorageException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/KafkaStorageException.java b/clients/src/main/java/org/apache/kafka/common/errors/KafkaStorageException.java
new file mode 100644
index 0000000..00c7cee
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/errors/KafkaStorageException.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.errors;
+
+/**
+ * Miscellaneous disk-related IOException occurred when handling a request.
+ * Client should request metadata update and retry if the response shows KafkaStorageException
+ *
+ * Here are the guidelines on how to handle KafkaStorageException and IOException:
+ *
+ * 1) If the server has not finished loading logs, IOException does not need to be converted to KafkaStorageException
+ * 2) After the server has finished loading logs, IOException should be caught and trigger LogDirFailureChannel.maybeAddLogFailureEvent
+ *    Then the IOException should either be swallowed and logged, or be converted and re-thrown as KafkaStorageException
+ * 3) It is preferred for IOException to be caught in Log rather than in ReplicaManager or LogSegment.
+ *
+ */
+public class KafkaStorageException extends InvalidMetadataException {
+
+    private static final long serialVersionUID = 1L;
+
+    public KafkaStorageException() {
+        super();
+    }
+
+    public KafkaStorageException(String message) {
+        super(message);
+    }
+
+    public KafkaStorageException(Throwable cause) {
+        super(cause);
+    }
+
+    public KafkaStorageException(String message, Throwable cause) {
+        super(message, cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
index ae8d161..19acfd6 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
@@ -44,6 +44,7 @@ import org.apache.kafka.common.errors.InvalidTimestampException;
 import org.apache.kafka.common.errors.InvalidTopicException;
 import org.apache.kafka.common.errors.InvalidTxnStateException;
 import org.apache.kafka.common.errors.InvalidTxnTimeoutException;
+import org.apache.kafka.common.errors.KafkaStorageException;
 import org.apache.kafka.common.errors.LeaderNotAvailableException;
 import org.apache.kafka.common.errors.NetworkException;
 import org.apache.kafka.common.errors.NotControllerException;
@@ -84,10 +85,15 @@ import java.util.Map;
  * This class contains all the client-server errors--those errors that must be sent from the server to the client. These
  * are thus part of the protocol. The names can be changed but the error code cannot.
  *
+ * Note that client library will convert an unknown error code to the non-retriable UnknownServerException if the client library
+ * version is old and does not recognize the newly-added error code. Therefore when a new server-side error is added,
+ * we may need extra logic to convert the new error code to another existing error code before sending the response back to
+ * the client if the request version suggests that the client may not recognize the new error code.
+ *
  * Do not add exceptions that occur only on the client or only on the server here.
  */
 public enum Errors {
-    UNKNOWN(-1, "The server experienced an unexpected error when processing the request",
+    UNKNOWN_SERVER_ERROR(-1, "The server experienced an unexpected error when processing the request",
         new ApiExceptionBuilder() {
             @Override
             public ApiException build(String message) {
@@ -495,7 +501,14 @@ public enum Errors {
             public ApiException build(String message) {
                 return new OperationNotAttemptedException(message);
             }
-        });
+    }),
+    KAFKA_STORAGE_ERROR(56, "Disk error when trying to access log file on the disk.",
+        new ApiExceptionBuilder() {
+            @Override
+            public ApiException build(String message) {
+                return new KafkaStorageException(message);
+            }
+    });
 
     private interface ApiExceptionBuilder {
         ApiException build(String message);
@@ -588,7 +601,7 @@ public enum Errors {
             return error;
         } else {
             log.warn("Unexpected error code: {}.", code);
-            return UNKNOWN;
+            return UNKNOWN_SERVER_ERROR;
         }
     }
 
@@ -604,7 +617,7 @@ public enum Errors {
                 return error;
             clazz = clazz.getSuperclass();
         }
-        return UNKNOWN;
+        return UNKNOWN_SERVER_ERROR;
     }
 
     private static String toHtml() {

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
index 383332b..329d99b 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
@@ -78,6 +78,9 @@ public class Protocol {
                                                                           "topics that don't exist will be created by the broker. " +
                                                                           "Otherwise, no topics will be created by the broker."));
 
+    /* The v5 metadata request is the same as v4. An additional field for offline_replicas has been added to the v5 metadata response */
+    public static final Schema METADATA_REQUEST_V5 = METADATA_REQUEST_V4;
+
     public static final Schema METADATA_BROKER_V0 = new Schema(new Field("node_id", INT32, "The broker id."),
                                                    new Field("host", STRING, "The hostname of the broker."),
                                                    new Field("port", INT32,
@@ -121,12 +124,40 @@ public class Protocol {
 
     public static final Schema PARTITION_METADATA_V1 = PARTITION_METADATA_V0;
 
+    // PARTITION_METADATA_V2 added a per-partition offline_replicas field. This field specifies the list of replicas that are offline.
+    public static final Schema PARTITION_METADATA_V2 = new Schema(new Field("partition_error_code",
+                                                                            INT16,
+                                                                            "The error code for the partition, if any."),
+                                                                  new Field("partition_id",
+                                                                            INT32,
+                                                                            "The id of the partition."),
+                                                                  new Field("leader",
+                                                                            INT32,
+                                                                            "The id of the broker acting as leader for this partition."),
+                                                                  new Field("replicas",
+                                                                            new ArrayOf(INT32),
+                                                                            "The set of all nodes that host this partition."),
+                                                                  new Field("isr",
+                                                                            new ArrayOf(INT32),
+                                                                            "The set of nodes that are in sync with the leader for this partition."),
+                                                                  new Field("offline_replicas",
+                                                                            new ArrayOf(INT32),
+                                                                            "The set of offline replicas of this partition."));
+
     public static final Schema TOPIC_METADATA_V1 = new Schema(new Field("topic_error_code", INT16, "The error code for the given topic."),
                                                               new Field("topic", STRING, "The name of the topic"),
                                                               new Field("is_internal", BOOLEAN,
                                                                   "Indicates if the topic is considered a Kafka internal topic"),
                                                               new Field("partition_metadata", new ArrayOf(PARTITION_METADATA_V1),
-                                                                "Metadata for each partition of the topic."));
+                                                                  "Metadata for each partition of the topic."));
+
+    // TOPIC_METADATA_V2 added a per-partition offline_replicas field. This field specifies the list of replicas that are offline.
+    public static final Schema TOPIC_METADATA_V2 = new Schema(new Field("topic_error_code", INT16, "The error code for the given topic."),
+                                                              new Field("topic", STRING, "The name of the topic"),
+                                                              new Field("is_internal", BOOLEAN,
+                                                                  "Indicates if the topic is considered a Kafka internal topic"),
+                                                              new Field("partition_metadata", new ArrayOf(PARTITION_METADATA_V2),
+                                                                  "Metadata for each partition of the topic."));
 
     public static final Schema METADATA_RESPONSE_V1 = new Schema(new Field("brokers", new ArrayOf(METADATA_BROKER_V1),
                                                                     "Host and port information for all brokers."),
@@ -154,8 +185,19 @@ public class Protocol {
 
     public static final Schema METADATA_RESPONSE_V4 = METADATA_RESPONSE_V3;
 
-    public static final Schema[] METADATA_REQUEST = {METADATA_REQUEST_V0, METADATA_REQUEST_V1, METADATA_REQUEST_V2, METADATA_REQUEST_V3, METADATA_REQUEST_V4};
-    public static final Schema[] METADATA_RESPONSE = {METADATA_RESPONSE_V0, METADATA_RESPONSE_V1, METADATA_RESPONSE_V2, METADATA_RESPONSE_V3, METADATA_RESPONSE_V4};
+    // METADATA_RESPONSE_V5 added a per-partition offline_replicas field. This field specifies the list of replicas that are offline.
+    public static final Schema METADATA_RESPONSE_V5 = new Schema(
+        newThrottleTimeField(),
+        new Field("brokers", new ArrayOf(METADATA_BROKER_V1),
+            "Host and port information for all brokers."),
+        new Field("cluster_id", NULLABLE_STRING,
+            "The cluster id that this broker belongs to."),
+        new Field("controller_id", INT32,
+            "The broker id of the controller broker."),
+        new Field("topic_metadata", new ArrayOf(TOPIC_METADATA_V2)));
+
+    public static final Schema[] METADATA_REQUEST = {METADATA_REQUEST_V0, METADATA_REQUEST_V1, METADATA_REQUEST_V2, METADATA_REQUEST_V3, METADATA_REQUEST_V4, METADATA_REQUEST_V5};
+    public static final Schema[] METADATA_RESPONSE = {METADATA_RESPONSE_V0, METADATA_RESPONSE_V1, METADATA_RESPONSE_V2, METADATA_RESPONSE_V3, METADATA_RESPONSE_V4, METADATA_RESPONSE_V5};
 
     /* Produce api */
 
@@ -205,6 +247,13 @@ public class Protocol {
             new Field("timeout", INT32, "The time to await a response in ms."),
             new Field("topic_data", new ArrayOf(TOPIC_PRODUCE_DATA_V0)));
 
+    /**
+     * The body of PRODUCE_REQUEST_V4 is the same as PRODUCE_REQUEST_V3.
+     * The version number is bumped up to indicate that the client supports KafkaStorageException.
+     * The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 3
+     */
+    public static final Schema PRODUCE_REQUEST_V4 = PRODUCE_REQUEST_V3;
+
     public static final Schema PRODUCE_RESPONSE_V1 = new Schema(new Field("responses",
                                                                           new ArrayOf(new Schema(new Field("topic", STRING),
                                                                                                  new Field("partition_responses",
@@ -236,10 +285,18 @@ public class Protocol {
                                                                                                                             "If LogAppendTime is used for the topic, the timestamp will be " +
                                                                                                                             "the broker local time when the messages are appended."))))))),
                                                                 newThrottleTimeField());
+
     public static final Schema PRODUCE_RESPONSE_V3 = PRODUCE_RESPONSE_V2;
 
-    public static final Schema[] PRODUCE_REQUEST = {PRODUCE_REQUEST_V0, PRODUCE_REQUEST_V1, PRODUCE_REQUEST_V2, PRODUCE_REQUEST_V3};
-    public static final Schema[] PRODUCE_RESPONSE = {PRODUCE_RESPONSE_V0, PRODUCE_RESPONSE_V1, PRODUCE_RESPONSE_V2, PRODUCE_RESPONSE_V3};
+    /**
+     * The body of PRODUCE_RESPONSE_V4 is the same as PRODUCE_RESPONSE_V3.
+     * The version number is bumped up to indicate that the client supports KafkaStorageException.
+     * The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 3
+     */
+    public static final Schema PRODUCE_RESPONSE_V4 = PRODUCE_RESPONSE_V3;
+
+    public static final Schema[] PRODUCE_REQUEST = {PRODUCE_REQUEST_V0, PRODUCE_REQUEST_V1, PRODUCE_REQUEST_V2, PRODUCE_REQUEST_V3, PRODUCE_REQUEST_V4};
+    public static final Schema[] PRODUCE_RESPONSE = {PRODUCE_RESPONSE_V0, PRODUCE_RESPONSE_V1, PRODUCE_RESPONSE_V2, PRODUCE_RESPONSE_V3, PRODUCE_RESPONSE_V4};
 
     /* Offset commit api */
     public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
@@ -666,6 +723,13 @@ public class Protocol {
                     new ArrayOf(FETCH_REQUEST_TOPIC_V5),
                     "Topics to fetch in the order provided."));
 
+    /**
+     * The body of FETCH_REQUEST_V6 is the same as FETCH_REQUEST_V5.
+     * The version number is bumped up to indicate that the client supports KafkaStorageException.
+     * The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 5
+     */
+    public static final Schema FETCH_REQUEST_V6 = FETCH_REQUEST_V5;
+
     public static final Schema FETCH_RESPONSE_PARTITION_HEADER_V0 = new Schema(new Field("partition",
                                                                                          INT32,
                                                                                          "Topic partition id."),
@@ -692,7 +756,6 @@ public class Protocol {
     public static final Schema FETCH_RESPONSE_V2 = FETCH_RESPONSE_V1;
     public static final Schema FETCH_RESPONSE_V3 = FETCH_RESPONSE_V2;
 
-
     // The v4 Fetch Response adds features for transactional consumption (the aborted transaction list and the
     // last stable offset). It also exposes messages with magic v2 (along with older formats).
     private static final Schema FETCH_RESPONSE_ABORTED_TRANSACTION_V4 = new Schema(
@@ -759,8 +822,15 @@ public class Protocol {
             newThrottleTimeField(),
             new Field("responses", new ArrayOf(FETCH_RESPONSE_TOPIC_V5)));
 
-    public static final Schema[] FETCH_REQUEST = {FETCH_REQUEST_V0, FETCH_REQUEST_V1, FETCH_REQUEST_V2, FETCH_REQUEST_V3, FETCH_REQUEST_V4, FETCH_REQUEST_V5};
-    public static final Schema[] FETCH_RESPONSE = {FETCH_RESPONSE_V0, FETCH_RESPONSE_V1, FETCH_RESPONSE_V2, FETCH_RESPONSE_V3, FETCH_RESPONSE_V4, FETCH_RESPONSE_V5};
+    /**
+     * The body of FETCH_RESPONSE_V6 is the same as FETCH_RESPONSE_V5.
+     * The version number is bumped up to indicate that the client supports KafkaStorageException.
+     * The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 5
+     */
+    public static final Schema FETCH_RESPONSE_V6 = FETCH_RESPONSE_V5;
+
+    public static final Schema[] FETCH_REQUEST = {FETCH_REQUEST_V0, FETCH_REQUEST_V1, FETCH_REQUEST_V2, FETCH_REQUEST_V3, FETCH_REQUEST_V4, FETCH_REQUEST_V5, FETCH_REQUEST_V6};
+    public static final Schema[] FETCH_RESPONSE = {FETCH_RESPONSE_V0, FETCH_RESPONSE_V1, FETCH_RESPONSE_V2, FETCH_RESPONSE_V3, FETCH_RESPONSE_V4, FETCH_RESPONSE_V5, FETCH_RESPONSE_V6};
 
     /* List groups api */
     public static final Schema LIST_GROUPS_REQUEST_V0 = new Schema();
@@ -1039,6 +1109,19 @@ public class Protocol {
                        new Field("zk_version", INT32, "The ZK version."),
                        new Field("replicas", new ArrayOf(INT32), "The replica ids."));
 
+    // LEADER_AND_ISR_REQUEST_PARTITION_STATE_V1 added a per-partition is_new field.
+    // This field specifies whether the replica should have existed on the broker or not.
+    public static final Schema LEADER_AND_ISR_REQUEST_PARTITION_STATE_V1 =
+            new Schema(new Field("topic", STRING, "Topic name."),
+                       new Field("partition", INT32, "Topic partition id."),
+                       new Field("controller_epoch", INT32, "The controller epoch."),
+                       new Field("leader", INT32, "The broker id for the leader."),
+                       new Field("leader_epoch", INT32, "The leader epoch."),
+                       new Field("isr", new ArrayOf(INT32), "The in sync replica ids."),
+                       new Field("zk_version", INT32, "The ZK version."),
+                       new Field("replicas", new ArrayOf(INT32), "The replica ids."),
+                       new Field("is_new", BOOLEAN, "Whether the replica should have existed on the broker or not"));
+
     public static final Schema LEADER_AND_ISR_REQUEST_LIVE_LEADER_V0 =
             new Schema(new Field("id", INT32, "The broker id."),
                        new Field("host", STRING, "The hostname of the broker."),
@@ -1050,6 +1133,13 @@ public class Protocol {
                                                                                 new ArrayOf(LEADER_AND_ISR_REQUEST_PARTITION_STATE_V0)),
                                                                       new Field("live_leaders", new ArrayOf(LEADER_AND_ISR_REQUEST_LIVE_LEADER_V0)));
 
+    // LEADER_AND_ISR_REQUEST_V1 added a per-partition is_new field. This field specifies whether the replica should have existed on the broker or not.
+    public static final Schema LEADER_AND_ISR_REQUEST_V1 = new Schema(new Field("controller_id", INT32, "The controller id."),
+                                                                      new Field("controller_epoch", INT32, "The controller epoch."),
+                                                                      new Field("partition_states",
+                                                                                new ArrayOf(LEADER_AND_ISR_REQUEST_PARTITION_STATE_V1)),
+                                                                      new Field("live_leaders", new ArrayOf(LEADER_AND_ISR_REQUEST_LIVE_LEADER_V0)));
+
     public static final Schema LEADER_AND_ISR_RESPONSE_PARTITION_V0 = new Schema(new Field("topic", STRING, "Topic name."),
                                                                                  new Field("partition", INT32, "Topic partition id."),
                                                                                  new Field("error_code", INT16, "Error code."));
@@ -1058,8 +1148,11 @@ public class Protocol {
                                                                        new Field("partitions",
                                                                                  new ArrayOf(LEADER_AND_ISR_RESPONSE_PARTITION_V0)));
 
-    public static final Schema[] LEADER_AND_ISR_REQUEST = {LEADER_AND_ISR_REQUEST_V0};
-    public static final Schema[] LEADER_AND_ISR_RESPONSE = {LEADER_AND_ISR_RESPONSE_V0};
+    // LeaderAndIsrResponse V1 may receive KAFKA_STORAGE_ERROR in the response
+    public static final Schema LEADER_AND_ISR_RESPONSE_V1 = LEADER_AND_ISR_RESPONSE_V0;
+
+    public static final Schema[] LEADER_AND_ISR_REQUEST = {LEADER_AND_ISR_REQUEST_V0, LEADER_AND_ISR_REQUEST_V1};
+    public static final Schema[] LEADER_AND_ISR_RESPONSE = {LEADER_AND_ISR_RESPONSE_V0, LEADER_AND_ISR_RESPONSE_V1};
 
     /* Replica api */
     public static final Schema STOP_REPLICA_REQUEST_PARTITION_V0 = new Schema(new Field("topic", STRING, "Topic name."),
@@ -1141,6 +1234,18 @@ public class Protocol {
 
     public static final Schema UPDATE_METADATA_REQUEST_PARTITION_STATE_V3 = UPDATE_METADATA_REQUEST_PARTITION_STATE_V2;
 
+    // UPDATE_METADATA_REQUEST_PARTITION_STATE_V4 added a per-partition offline_replicas field. This field specifies the list of replicas that are offline.
+    public static final Schema UPDATE_METADATA_REQUEST_PARTITION_STATE_V4 =
+            new Schema(new Field("topic", STRING, "Topic name."),
+                       new Field("partition", INT32, "Topic partition id."),
+                       new Field("controller_epoch", INT32, "The controller epoch."),
+                       new Field("leader", INT32, "The broker id for the leader."),
+                       new Field("leader_epoch", INT32, "The leader epoch."),
+                       new Field("isr", new ArrayOf(INT32), "The in sync replica ids."),
+                       new Field("zk_version", INT32, "The ZK version."),
+                       new Field("replicas", new ArrayOf(INT32), "The replica ids."),
+                       new Field("offline_replicas", new ArrayOf(INT32), "The offline replica ids"));
+
     public static final Schema UPDATE_METADATA_REQUEST_END_POINT_V3 =
             new Schema(new Field("port", INT32, "The port on which the broker accepts requests."),
                     new Field("host", STRING, "The hostname of the broker."),
@@ -1158,12 +1263,21 @@ public class Protocol {
                     new Field("partition_states", new ArrayOf(UPDATE_METADATA_REQUEST_PARTITION_STATE_V3)),
                     new Field("live_brokers", new ArrayOf(UPDATE_METADATA_REQUEST_BROKER_V3)));
 
+    // UPDATE_METADATA_REQUEST_V4 added a per-partition offline_replicas field. This field specifies the list of replicas that are offline.
+    public static final Schema UPDATE_METADATA_REQUEST_V4 =
+            new Schema(new Field("controller_id", INT32, "The controller id."),
+                    new Field("controller_epoch", INT32, "The controller epoch."),
+                    new Field("partition_states", new ArrayOf(UPDATE_METADATA_REQUEST_PARTITION_STATE_V4)),
+                    new Field("live_brokers", new ArrayOf(UPDATE_METADATA_REQUEST_BROKER_V3)));
+
     public static final Schema UPDATE_METADATA_RESPONSE_V3 = UPDATE_METADATA_RESPONSE_V2;
 
+    public static final Schema UPDATE_METADATA_RESPONSE_V4 = UPDATE_METADATA_RESPONSE_V3;
+
     public static final Schema[] UPDATE_METADATA_REQUEST = {UPDATE_METADATA_REQUEST_V0, UPDATE_METADATA_REQUEST_V1,
-        UPDATE_METADATA_REQUEST_V2, UPDATE_METADATA_REQUEST_V3};
+        UPDATE_METADATA_REQUEST_V2, UPDATE_METADATA_REQUEST_V3, UPDATE_METADATA_REQUEST_V4};
     public static final Schema[] UPDATE_METADATA_RESPONSE = {UPDATE_METADATA_RESPONSE_V0, UPDATE_METADATA_RESPONSE_V1,
-        UPDATE_METADATA_RESPONSE_V2, UPDATE_METADATA_RESPONSE_V3};
+        UPDATE_METADATA_RESPONSE_V2, UPDATE_METADATA_RESPONSE_V3, UPDATE_METADATA_RESPONSE_V4};
 
     /* SASL handshake api */
     public static final Schema SASL_HANDSHAKE_REQUEST_V0 = new Schema(

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java
index 35431d8..a898634 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java
@@ -172,6 +172,13 @@ public class FileRecords extends AbstractRecords implements Closeable {
     }
 
     /**
+     * Close file handlers used by the FileChannel but don't write to disk. This is used when the disk may have failed
+     */
+    public void closeHandlers() throws IOException {
+        channel.close();
+    }
+
+    /**
      * Delete this message set from the filesystem
      * @return True iff this message set was deleted.
      */

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java
index 824a76f..f0f516c 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java
@@ -203,7 +203,7 @@ public class FetchResponse extends AbstractResponse {
                 long logStartOffset = INVALID_LOG_START_OFFSET;
                 if (partitionResponseHeader.hasField(LOG_START_OFFSET_KEY_NAME))
                     logStartOffset = partitionResponseHeader.getLong(LOG_START_OFFSET_KEY_NAME);
-                
+
                 Records records = partitionResponse.getRecords(RECORD_SET_KEY_NAME);
 
                 List<AbortedTransaction> abortedTransactions = null;
@@ -326,10 +326,17 @@ public class FetchResponse extends AbstractResponse {
             List<Struct> partitionArray = new ArrayList<>();
             for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.partitions.entrySet()) {
                 PartitionData fetchPartitionData = partitionEntry.getValue();
+                short errorCode = fetchPartitionData.error.code();
+                // If consumer sends FetchRequest V5 or earlier, the client library is not guaranteed to recognize the error code
+                // for KafkaStorageException. In this case the client library will translate KafkaStorageException to
+                // UnknownServerException which is not retriable. We can ensure that consumer will update metadata and retry
+                // by converting the KafkaStorageException to NotLeaderForPartitionException in the response if FetchRequest version <= 5
+                if (errorCode == Errors.KAFKA_STORAGE_ERROR.code() && version <= 5)
+                    errorCode = Errors.NOT_LEADER_FOR_PARTITION.code();
                 Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
                 Struct partitionDataHeader = partitionData.instance(PARTITION_HEADER_KEY_NAME);
                 partitionDataHeader.set(PARTITION_KEY_NAME, partitionEntry.getKey());
-                partitionDataHeader.set(ERROR_CODE_KEY_NAME, fetchPartitionData.error.code());
+                partitionDataHeader.set(ERROR_CODE_KEY_NAME, errorCode);
                 partitionDataHeader.set(HIGH_WATERMARK_KEY_NAME, fetchPartitionData.highWatermark);
 
                 if (partitionDataHeader.hasField(LAST_STABLE_OFFSET_KEY_NAME)) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java
index 1fdb4a2..733c9af 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java
@@ -45,6 +45,7 @@ public class LeaderAndIsrRequest extends AbstractRequest {
     private static final String ISR_KEY_NAME = "isr";
     private static final String ZK_VERSION_KEY_NAME = "zk_version";
     private static final String REPLICAS_KEY_NAME = "replicas";
+    private static final String IS_NEW_KEY_NAME = "is_new";
 
     // live_leaders key names
     private static final String END_POINT_ID_KEY_NAME = "id";
@@ -57,9 +58,9 @@ public class LeaderAndIsrRequest extends AbstractRequest {
         private final Map<TopicPartition, PartitionState> partitionStates;
         private final Set<Node> liveLeaders;
 
-        public Builder(int controllerId, int controllerEpoch,
+        public Builder(short version, int controllerId, int controllerEpoch,
                        Map<TopicPartition, PartitionState> partitionStates, Set<Node> liveLeaders) {
-            super(ApiKeys.LEADER_AND_ISR);
+            super(ApiKeys.LEADER_AND_ISR, version);
             this.controllerId = controllerId;
             this.controllerEpoch = controllerEpoch;
             this.partitionStates = partitionStates;
@@ -121,10 +122,10 @@ public class LeaderAndIsrRequest extends AbstractRequest {
             List<Integer> replicas = new ArrayList<>(replicasArray.length);
             for (Object r : replicasArray)
                 replicas.add((Integer) r);
+            boolean isNew = partitionStateData.hasField(IS_NEW_KEY_NAME) ? partitionStateData.getBoolean(IS_NEW_KEY_NAME) : false;
 
-            PartitionState partitionState = new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas);
+            PartitionState partitionState = new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas, isNew);
             partitionStates.put(new TopicPartition(topic, partition), partitionState);
-
         }
 
         Set<Node> leaders = new HashSet<>();
@@ -162,6 +163,8 @@ public class LeaderAndIsrRequest extends AbstractRequest {
             partitionStateData.set(ISR_KEY_NAME, partitionState.isr.toArray());
             partitionStateData.set(ZK_VERSION_KEY_NAME, partitionState.zkVersion);
             partitionStateData.set(REPLICAS_KEY_NAME, partitionState.replicas.toArray());
+            if (partitionStateData.hasField(IS_NEW_KEY_NAME))
+                partitionStateData.set(IS_NEW_KEY_NAME, partitionState.isNew);
             partitionStatesData.add(partitionStateData);
         }
         struct.set(PARTITION_STATES_KEY_NAME, partitionStatesData.toArray());
@@ -188,6 +191,7 @@ public class LeaderAndIsrRequest extends AbstractRequest {
         short versionId = version();
         switch (versionId) {
             case 0:
+            case 1:
                 return new LeaderAndIsrResponse(Errors.NONE, responses);
             default:
                 throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java
index 0493f3d..8aa2fc3 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java
@@ -135,6 +135,7 @@ public class MetadataRequest extends AbstractRequest {
                 return new MetadataResponse(Collections.<Node>emptyList(), null, MetadataResponse.NO_CONTROLLER_ID, topicMetadatas);
             case 3:
             case 4:
+            case 5:
                 return new MetadataResponse(throttleTimeMs, Collections.<Node>emptyList(), null, MetadataResponse.NO_CONTROLLER_ID, topicMetadatas);
             default:
                 throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java
index b798764..10f5c13 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java
@@ -80,6 +80,7 @@ public class MetadataResponse extends AbstractResponse {
     private static final String LEADER_KEY_NAME = "leader";
     private static final String REPLICAS_KEY_NAME = "replicas";
     private static final String ISR_KEY_NAME = "isr";
+    private static final String OFFLINE_REPLICAS_KEY_NAME = "offline_replicas";
 
     private final int throttleTimeMs;
     private final Collection<Node> brokers;
@@ -149,26 +150,18 @@ public class MetadataResponse extends AbstractResponse {
                 int partition = partitionInfo.getInt(PARTITION_KEY_NAME);
                 int leader = partitionInfo.getInt(LEADER_KEY_NAME);
                 Node leaderNode = leader == -1 ? null : brokers.get(leader);
-                Object[] replicas = (Object[]) partitionInfo.get(REPLICAS_KEY_NAME);
 
-                List<Node> replicaNodes = new ArrayList<>(replicas.length);
-                for (Object replicaNodeId : replicas) {
-                    if (brokers.containsKey(replicaNodeId))
-                        replicaNodes.add(brokers.get(replicaNodeId));
-                    else
-                        replicaNodes.add(new Node((int) replicaNodeId, "", -1));
-                }
+                Object[] replicas = (Object[]) partitionInfo.get(REPLICAS_KEY_NAME);
+                List<Node> replicaNodes = convertToNodes(brokers, replicas);
 
                 Object[] isr = (Object[]) partitionInfo.get(ISR_KEY_NAME);
-                List<Node> isrNodes = new ArrayList<>(isr.length);
-                for (Object isrNode : isr) {
-                    if (brokers.containsKey(isrNode))
-                        isrNodes.add(brokers.get(isrNode));
-                    else
-                        isrNodes.add(new Node((int) isrNode, "", -1));
-                }
+                List<Node> isrNodes = convertToNodes(brokers, isr);
+
+                Object[] offlineReplicas = partitionInfo.hasField(OFFLINE_REPLICAS_KEY_NAME) ?
+                    (Object[]) partitionInfo.get(OFFLINE_REPLICAS_KEY_NAME) : new Object[0];
+                List<Node> offlineNodes = convertToNodes(brokers, offlineReplicas);
 
-                partitionMetadata.add(new PartitionMetadata(partitionError, partition, leaderNode, replicaNodes, isrNodes));
+                partitionMetadata.add(new PartitionMetadata(partitionError, partition, leaderNode, replicaNodes, isrNodes, offlineNodes));
             }
 
             topicMetadata.add(new TopicMetadata(topicError, topic, isInternal, partitionMetadata));
@@ -179,6 +172,16 @@ public class MetadataResponse extends AbstractResponse {
         this.topicMetadata = topicMetadata;
     }
 
+    private List<Node> convertToNodes(Map<Integer, Node> brokers, Object[] brokerIds) {
+        List<Node> nodes = new ArrayList<>(brokerIds.length);
+        for (Object brokerId : brokerIds)
+            if (brokers.containsKey(brokerId))
+                nodes.add(brokers.get(brokerId));
+            else
+                nodes.add(new Node((int) brokerId, "", -1));
+        return nodes;
+    }
+
     private Node getControllerNode(int controllerId, Collection<Node> brokers) {
         for (Node broker : brokers) {
             if (broker.id() == controllerId)
@@ -256,7 +259,8 @@ public class MetadataResponse extends AbstractResponse {
                             partitionMetadata.partition,
                             partitionMetadata.leader,
                             partitionMetadata.replicas.toArray(new Node[0]),
-                            partitionMetadata.isr.toArray(new Node[0])));
+                            partitionMetadata.isr.toArray(new Node[0]),
+                            partitionMetadata.offlineReplicas.toArray(new Node[0])));
             }
         }
 
@@ -334,23 +338,27 @@ public class MetadataResponse extends AbstractResponse {
 
     }
 
+    // This is used to describe per-partition state in the MetadataResponse
     public static class PartitionMetadata {
         private final Errors error;
         private final int partition;
         private final Node leader;
         private final List<Node> replicas;
         private final List<Node> isr;
+        private final List<Node> offlineReplicas;
 
         public PartitionMetadata(Errors error,
                                  int partition,
                                  Node leader,
                                  List<Node> replicas,
-                                 List<Node> isr) {
+                                 List<Node> isr,
+                                 List<Node> offlineReplicas) {
             this.error = error;
             this.partition = partition;
             this.leader = leader;
             this.replicas = replicas;
             this.isr = isr;
+            this.offlineReplicas = offlineReplicas;
         }
 
         public Errors error() {
@@ -373,6 +381,10 @@ public class MetadataResponse extends AbstractResponse {
             return isr;
         }
 
+        public List<Node> offlineReplicas() {
+            return offlineReplicas;
+        }
+
         @Override
         public String toString() {
             return "(type=PartitionMetadata," +
@@ -380,7 +392,8 @@ public class MetadataResponse extends AbstractResponse {
                     ", partition=" + partition +
                     ", leader=" + leader +
                     ", replicas=" + Utils.join(replicas, ",") +
-                    ", isr=" + Utils.join(isr, ",") + ')';
+                    ", isr=" + Utils.join(isr, ",") +
+                    ", offlineReplicas=" + Utils.join(offlineReplicas, ",") + ')';
         }
     }
 
@@ -433,6 +446,12 @@ public class MetadataResponse extends AbstractResponse {
                 for (Node node : partitionMetadata.isr)
                     isr.add(node.id());
                 partitionData.set(ISR_KEY_NAME, isr.toArray());
+                if (partitionData.hasField(OFFLINE_REPLICAS_KEY_NAME)) {
+                    ArrayList<Integer> offlineReplicas = new ArrayList<>(partitionMetadata.offlineReplicas.size());
+                    for (Node node : partitionMetadata.offlineReplicas)
+                        offlineReplicas.add(node.id());
+                    partitionData.set(OFFLINE_REPLICAS_KEY_NAME, offlineReplicas.toArray());
+                }
                 partitionMetadataArray.add(partitionData);
 
             }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/main/java/org/apache/kafka/common/requests/PartitionState.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/PartitionState.java b/clients/src/main/java/org/apache/kafka/common/requests/PartitionState.java
index 394a60f..be303d9 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/PartitionState.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/PartitionState.java
@@ -20,6 +20,7 @@ import org.apache.kafka.common.utils.Utils;
 
 import java.util.List;
 
+// This is used to describe per-partition state in the LeaderAndIsrRequest
 public class PartitionState {
     public final int controllerEpoch;
     public final int leader;
@@ -27,14 +28,16 @@ public class PartitionState {
     public final List<Integer> isr;
     public final int zkVersion;
     public final List<Integer> replicas;
+    public final boolean isNew;
 
-    public PartitionState(int controllerEpoch, int leader, int leaderEpoch, List<Integer> isr, int zkVersion, List<Integer> replicas) {
+    public PartitionState(int controllerEpoch, int leader, int leaderEpoch, List<Integer> isr, int zkVersion, List<Integer> replicas, boolean isNew) {
         this.controllerEpoch = controllerEpoch;
         this.leader = leader;
         this.leaderEpoch = leaderEpoch;
         this.isr = isr;
         this.zkVersion = zkVersion;
         this.replicas = replicas;
+        this.isNew = isNew;
     }
 
     @Override
@@ -44,6 +47,7 @@ public class PartitionState {
                 ", leaderEpoch=" + leaderEpoch +
                 ", isr=" + Utils.join(isr, ",") +
                 ", zkVersion=" + zkVersion +
-                ", replicas=" + Utils.join(replicas, ",") + ")";
+                ", replicas=" + Utils.join(replicas, ",") +
+                ", isNew=" + isNew + ")";
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
index 3d696c1..089d199 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
@@ -246,6 +246,7 @@ public class ProduceRequest extends AbstractRequest {
             case 1:
             case 2:
             case 3:
+            case 4:
                 return new ProduceResponse(responseMap, throttleTimeMs);
             default:
                 throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
@@ -307,6 +308,7 @@ public class ProduceRequest extends AbstractRequest {
                 return RecordBatch.MAGIC_VALUE_V1;
 
             case 3:
+            case 4:
                 return RecordBatch.MAGIC_VALUE_V2;
 
             default:

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
index d42f1c6..fdfba8b 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
@@ -122,9 +122,16 @@ public class ProduceResponse extends AbstractResponse {
             List<Struct> partitionArray = new ArrayList<>();
             for (Map.Entry<Integer, PartitionResponse> partitionEntry : entry.getValue().entrySet()) {
                 PartitionResponse part = partitionEntry.getValue();
+                short errorCode = part.error.code();
+                // If producer sends ProduceRequest V3 or earlier, the client library is not guaranteed to recognize the error code
+                // for KafkaStorageException. In this case the client library will translate KafkaStorageException to
+                // UnknownServerException which is not retriable. We can ensure that producer will update metadata and retry
+                // by converting the KafkaStorageException to NotLeaderForPartitionException in the response if ProduceRequest version <= 3
+                if (errorCode == Errors.KAFKA_STORAGE_ERROR.code() && version <= 3)
+                    errorCode = Errors.NOT_LEADER_FOR_PARTITION.code();
                 Struct partStruct = topicData.instance(PARTITION_RESPONSES_KEY_NAME)
                         .set(PARTITION_KEY_NAME, partitionEntry.getKey())
-                        .set(ERROR_CODE_KEY_NAME, part.error.code())
+                        .set(ERROR_CODE_KEY_NAME, errorCode)
                         .set(BASE_OFFSET_KEY_NAME, part.baseOffset);
                 if (partStruct.hasField(LOG_APPEND_TIME_KEY_NAME))
                     partStruct.set(LOG_APPEND_TIME_KEY_NAME, part.logAppendTime);

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java
index 8f9b592..1e20866 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java
@@ -27,6 +27,7 @@ import org.apache.kafka.common.utils.Utils;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -74,6 +75,43 @@ public class UpdateMetadataRequest extends AbstractRequest {
         }
     }
 
+    public static final class PartitionState {
+        public final int controllerEpoch;
+        public final int leader;
+        public final int leaderEpoch;
+        public final List<Integer> isr;
+        public final int zkVersion;
+        public final List<Integer> replicas;
+        public final List<Integer> offlineReplicas;
+
+        public PartitionState(int controllerEpoch,
+                              int leader,
+                              int leaderEpoch,
+                              List<Integer> isr,
+                              int zkVersion,
+                              List<Integer> replicas,
+                              List<Integer> offlineReplicas) {
+            this.controllerEpoch = controllerEpoch;
+            this.leader = leader;
+            this.leaderEpoch = leaderEpoch;
+            this.isr = isr;
+            this.zkVersion = zkVersion;
+            this.replicas = replicas;
+            this.offlineReplicas = offlineReplicas;
+        }
+
+        @Override
+        public String toString() {
+            return "PartitionState(controllerEpoch=" + controllerEpoch +
+                ", leader=" + leader +
+                ", leaderEpoch=" + leaderEpoch +
+                ", isr=" + Arrays.toString(isr.toArray()) +
+                ", zkVersion=" + zkVersion +
+                ", replicas=" + Arrays.toString(replicas.toArray()) +
+                ", offlineReplicas=" + Arrays.toString(replicas.toArray()) + ")";
+        }
+    }
+
     public static final class Broker {
         public final int id;
         public final List<EndPoint> endPoints;
@@ -129,6 +167,7 @@ public class UpdateMetadataRequest extends AbstractRequest {
     private static final String ISR_KEY_NAME = "isr";
     private static final String ZK_VERSION_KEY_NAME = "zk_version";
     private static final String REPLICAS_KEY_NAME = "replicas";
+    private static final String OFFLINE_REPLICAS_KEY_NAME = "offline_replicas";
 
     // Broker key names
     private static final String BROKER_ID_KEY_NAME = "id";
@@ -146,8 +185,8 @@ public class UpdateMetadataRequest extends AbstractRequest {
     private final Map<TopicPartition, PartitionState> partitionStates;
     private final Set<Broker> liveBrokers;
 
-    private UpdateMetadataRequest(short version, int controllerId, int controllerEpoch, Map<TopicPartition,
-            PartitionState> partitionStates, Set<Broker> liveBrokers) {
+    private UpdateMetadataRequest(short version, int controllerId, int controllerEpoch,
+                                  Map<TopicPartition, PartitionState> partitionStates, Set<Broker> liveBrokers) {
         super(version);
         this.controllerId = controllerId;
         this.controllerEpoch = controllerEpoch;
@@ -178,9 +217,16 @@ public class UpdateMetadataRequest extends AbstractRequest {
             for (Object r : replicasArray)
                 replicas.add((Integer) r);
 
-            PartitionState partitionState = new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas);
-            partitionStates.put(new TopicPartition(topic, partition), partitionState);
+            List<Integer> offlineReplicas = new ArrayList<>();
+            if (partitionStateData.hasField(OFFLINE_REPLICAS_KEY_NAME)) {
+                Object[] offlineReplicasArray = partitionStateData.getArray(OFFLINE_REPLICAS_KEY_NAME);
+                for (Object r : offlineReplicasArray)
+                    offlineReplicas.add((Integer) r);
+            }
 
+            PartitionState partitionState =
+                new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas, offlineReplicas);
+            partitionStates.put(new TopicPartition(topic, partition), partitionState);
         }
 
         Set<Broker> liveBrokers = new HashSet<>();
@@ -245,6 +291,8 @@ public class UpdateMetadataRequest extends AbstractRequest {
             partitionStateData.set(ISR_KEY_NAME, partitionState.isr.toArray());
             partitionStateData.set(ZK_VERSION_KEY_NAME, partitionState.zkVersion);
             partitionStateData.set(REPLICAS_KEY_NAME, partitionState.replicas.toArray());
+            if (partitionStateData.hasField(OFFLINE_REPLICAS_KEY_NAME))
+              partitionStateData.set(OFFLINE_REPLICAS_KEY_NAME, partitionState.offlineReplicas.toArray());
             partitionStatesData.add(partitionStateData);
         }
         struct.set(PARTITION_STATES_KEY_NAME, partitionStatesData.toArray());
@@ -286,7 +334,7 @@ public class UpdateMetadataRequest extends AbstractRequest {
     @Override
     public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
         short versionId = version();
-        if (versionId <= 3)
+        if (versionId <= 4)
             return new UpdateMetadataResponse(Errors.forException(e));
         else
             throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java
index afebd9d..dd1c79a 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java
@@ -124,7 +124,7 @@ public class AbstractCoordinatorTest {
                     throw e;
                 return false;
             }
-        }, heartbeatResponse(Errors.UNKNOWN));
+        }, heartbeatResponse(Errors.UNKNOWN_SERVER_ERROR));
 
         try {
             coordinator.ensureActiveGroup();
@@ -499,7 +499,7 @@ public class AbstractCoordinatorTest {
                     heartbeatReceived.set(true);
                 return isHeartbeatRequest;
             }
-        }, heartbeatResponse(Errors.UNKNOWN));
+        }, heartbeatResponse(Errors.UNKNOWN_SERVER_ERROR));
         return heartbeatReceived;
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
index fc0ddff..3c1b411 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
@@ -621,7 +621,7 @@ public class ConsumerCoordinatorTest {
 
         // join initially, but let coordinator rebalance on sync
         client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE));
-        client.prepareResponse(syncGroupResponse(Collections.<TopicPartition>emptyList(), Errors.UNKNOWN));
+        client.prepareResponse(syncGroupResponse(Collections.<TopicPartition>emptyList(), Errors.UNKNOWN_SERVER_ERROR));
         coordinator.joinGroupIfNeeded();
     }
 
@@ -1267,7 +1267,7 @@ public class ConsumerCoordinatorTest {
         coordinator.ensureCoordinatorReady();
 
         // sync commit with invalid partitions should throw if we have no callback
-        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.UNKNOWN)), false);
+        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.UNKNOWN_SERVER_ERROR)), false);
         coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), Long.MAX_VALUE);
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
index c0edcfd..bb3162a 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
@@ -1979,7 +1979,8 @@ public class FetcherTest {
                         partitionInfo.partition(),
                         partitionInfo.leader(),
                         Arrays.asList(partitionInfo.replicas()),
-                        Arrays.asList(partitionInfo.inSyncReplicas())));
+                        Arrays.asList(partitionInfo.inSyncReplicas()),
+                        Arrays.asList(partitionInfo.offlineReplicas())));
             }
         }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java
index b655743..9bac895 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java
@@ -1758,7 +1758,7 @@ public class TransactionManagerTest {
 
         TransactionalRequestResult abortResult = transactionManager.beginAbort();
 
-        prepareAddOffsetsToTxnResponse(Errors.UNKNOWN, consumerGroupId, pid, epoch);
+        prepareAddOffsetsToTxnResponse(Errors.UNKNOWN_SERVER_ERROR, consumerGroupId, pid, epoch);
         sender.run(time.milliseconds());  // Send AddOffsetsToTxnRequest
         assertFalse(abortResult.isCompleted());
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/test/java/org/apache/kafka/common/PartitionInfoTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/PartitionInfoTest.java b/clients/src/test/java/org/apache/kafka/common/PartitionInfoTest.java
index 7836023..7511d64 100644
--- a/clients/src/test/java/org/apache/kafka/common/PartitionInfoTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/PartitionInfoTest.java
@@ -20,20 +20,21 @@ import org.junit.Assert;
 import org.junit.Test;
 
 public class PartitionInfoTest {
-    
+
     @Test
     public void testToString() {
         String topic = "sample";
         int partition = 0;
         Node leader = new Node(0, "localhost", 9092);
         Node r1 = new Node(1, "localhost", 9093);
-        Node r2 = new Node(2, "localhost", 9094); 
+        Node r2 = new Node(2, "localhost", 9094);
         Node[] replicas = new Node[] {leader, r1, r2};
-        Node[] inSyncReplicas = new Node[] {leader, r1, r2};
-        PartitionInfo partitionInfo = new PartitionInfo(topic, partition, leader, replicas, inSyncReplicas);
-        
-        String expected = String.format("Partition(topic = %s, partition = %d, leader = %s, replicas = %s, isr = %s)",
-                topic, partition, leader.idString(), "[0,1,2]", "[0,1,2]");
+        Node[] inSyncReplicas = new Node[] {leader, r1};
+        Node[] offlineReplicas = new Node[] {r2};
+        PartitionInfo partitionInfo = new PartitionInfo(topic, partition, leader, replicas, inSyncReplicas, offlineReplicas);
+
+        String expected = String.format("Partition(topic = %s, partition = %d, leader = %s, replicas = %s, isr = %s, offlineReplicas = %s)",
+                topic, partition, leader.idString(), "[0,1,2]", "[0,1]", "[2]");
         Assert.assertEquals(expected, partitionInfo.toString());
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/test/java/org/apache/kafka/common/protocol/ErrorsTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/ErrorsTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/ErrorsTest.java
index 4a96ac4..e424384 100644
--- a/clients/src/test/java/org/apache/kafka/common/protocol/ErrorsTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/protocol/ErrorsTest.java
@@ -74,12 +74,12 @@ public class ErrorsTest {
     @Test
     public void testForExceptionDefault() {
         Errors error = Errors.forException(new ApiException());
-        assertEquals("forException should default to unknown", Errors.UNKNOWN, error);
+        assertEquals("forException should default to unknown", Errors.UNKNOWN_SERVER_ERROR, error);
     }
 
     @Test
     public void testExceptionName() {
-        String exceptionName = Errors.UNKNOWN.exceptionName();
+        String exceptionName = Errors.UNKNOWN_SERVER_ERROR.exceptionName();
         assertEquals("org.apache.kafka.common.errors.UnknownServerException", exceptionName);
         exceptionName = Errors.NONE.exceptionName();
         assertNull(exceptionName);

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
index a3c277f..a1c2a83 100644
--- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
@@ -723,10 +723,11 @@ public class RequestResponseTest {
         Node node = new Node(1, "host1", 1001);
         List<Node> replicas = asList(node);
         List<Node> isr = asList(node);
+        List<Node> offlineReplicas = asList();
 
         List<MetadataResponse.TopicMetadata> allTopicMetadata = new ArrayList<>();
         allTopicMetadata.add(new MetadataResponse.TopicMetadata(Errors.NONE, "__consumer_offsets", true,
-                asList(new MetadataResponse.PartitionMetadata(Errors.NONE, 1, node, replicas, isr))));
+                asList(new MetadataResponse.PartitionMetadata(Errors.NONE, 1, node, replicas, isr, offlineReplicas))));
         allTopicMetadata.add(new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, "topic2", false,
                 Collections.<MetadataResponse.PartitionMetadata>emptyList()));
 
@@ -807,18 +808,17 @@ public class RequestResponseTest {
         List<Integer> isr = asList(1, 2);
         List<Integer> replicas = asList(1, 2, 3, 4);
         partitionStates.put(new TopicPartition("topic5", 105),
-                new PartitionState(0, 2, 1, new ArrayList<>(isr), 2, replicas));
+                new PartitionState(0, 2, 1, new ArrayList<>(isr), 2, replicas, false));
         partitionStates.put(new TopicPartition("topic5", 1),
-                new PartitionState(1, 1, 1, new ArrayList<>(isr), 2, replicas));
+                new PartitionState(1, 1, 1, new ArrayList<>(isr), 2, replicas, false));
         partitionStates.put(new TopicPartition("topic20", 1),
-                new PartitionState(1, 0, 1, new ArrayList<>(isr), 2, replicas));
+                new PartitionState(1, 0, 1, new ArrayList<>(isr), 2, replicas, false));
 
         Set<Node> leaders = Utils.mkSet(
                 new Node(0, "test0", 1223),
                 new Node(1, "test1", 1223)
         );
-
-        return new LeaderAndIsrRequest.Builder(1, 10, partitionStates, leaders).build();
+        return new LeaderAndIsrRequest.Builder(ApiKeys.LEADER_AND_ISR.latestVersion(), 1, 10, partitionStates, leaders).build();
     }
 
     private LeaderAndIsrResponse createLeaderAndIsrResponse() {
@@ -828,15 +828,16 @@ public class RequestResponseTest {
     }
 
     private UpdateMetadataRequest createUpdateMetadataRequest(int version, String rack) {
-        Map<TopicPartition, PartitionState> partitionStates = new HashMap<>();
+        Map<TopicPartition, UpdateMetadataRequest.PartitionState> partitionStates = new HashMap<>();
         List<Integer> isr = asList(1, 2);
         List<Integer> replicas = asList(1, 2, 3, 4);
+        List<Integer> offlineReplicas = asList();
         partitionStates.put(new TopicPartition("topic5", 105),
-                new PartitionState(0, 2, 1, new ArrayList<>(isr), 2, replicas));
+            new UpdateMetadataRequest.PartitionState(0, 2, 1, isr, 2, replicas, offlineReplicas));
         partitionStates.put(new TopicPartition("topic5", 1),
-                new PartitionState(1, 1, 1, new ArrayList<>(isr), 2, replicas));
+            new UpdateMetadataRequest.PartitionState(1, 1, 1, isr, 2, replicas, offlineReplicas));
         partitionStates.put(new TopicPartition("topic20", 1),
-                new PartitionState(1, 0, 1, new ArrayList<>(isr), 2, replicas));
+            new UpdateMetadataRequest.PartitionState(1, 0, 1, isr, 2, replicas, offlineReplicas));
 
         SecurityProtocol plaintext = SecurityProtocol.PLAINTEXT;
         List<UpdateMetadataRequest.EndPoint> endPoints1 = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/admin/AdminClient.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/admin/AdminClient.scala b/core/src/main/scala/kafka/admin/AdminClient.scala
index 7bd626a..2baed02 100644
--- a/core/src/main/scala/kafka/admin/AdminClient.scala
+++ b/core/src/main/scala/kafka/admin/AdminClient.scala
@@ -66,7 +66,7 @@ class AdminClient(val time: Time,
       } finally {
         pendingFutures.asScala.foreach { future =>
           try {
-            future.raise(Errors.UNKNOWN)
+            future.raise(Errors.UNKNOWN_SERVER_ERROR)
           } catch {
             case _: IllegalStateException => // It is OK if the future has been completed
           }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/admin/AdminUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala
index 923ceb7..7de85e4 100644
--- a/core/src/main/scala/kafka/admin/AdminUtils.scala
+++ b/core/src/main/scala/kafka/admin/AdminUtils.scala
@@ -408,7 +408,7 @@ object AdminUtils extends Logging with AdminUtilities {
     zkUtils.pathExists(getTopicPath(topic))
 
   def getBrokerMetadatas(zkUtils: ZkUtils, rackAwareMode: RackAwareMode = RackAwareMode.Enforced,
-                        brokerList: Option[Seq[Int]] = None): Seq[BrokerMetadata] = {
+                         brokerList: Option[Seq[Int]] = None): Seq[BrokerMetadata] = {
     val allBrokers = zkUtils.getAllBrokersInCluster()
     val brokers = brokerList.map(brokerIds => allBrokers.filter(b => brokerIds.contains(b.id))).getOrElse(allBrokers)
     val brokersWithRack = brokers.filter(_.rack.nonEmpty)

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/api/ApiVersion.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala b/core/src/main/scala/kafka/api/ApiVersion.scala
index 62d5d12..05658dd 100644
--- a/core/src/main/scala/kafka/api/ApiVersion.scala
+++ b/core/src/main/scala/kafka/api/ApiVersion.scala
@@ -69,7 +69,10 @@ object ApiVersion {
     "0.11.0-IV1" -> KAFKA_0_11_0_IV1,
     // Introduced leader epoch fetches to the replica fetcher via KIP-101
     "0.11.0-IV2" -> KAFKA_0_11_0_IV2,
-    "0.11.0" -> KAFKA_0_11_0_IV2
+    "0.11.0" -> KAFKA_0_11_0_IV2,
+    // Introduced LeaderAndIsrRequest V1, UpdateMetadataRequest V4 and FetchRequest V6 via KIP-112
+    "0.11.1-IV0" -> KAFKA_0_11_1_IV0,
+    "0.11.1" -> KAFKA_0_11_1_IV0
   )
 
   private val versionPattern = "\\.".r
@@ -171,3 +174,10 @@ case object KAFKA_0_11_0_IV2 extends ApiVersion {
   val messageFormatVersion: Byte = RecordBatch.MAGIC_VALUE_V2
   val id: Int = 12
 }
+
+case object KAFKA_0_11_1_IV0 extends ApiVersion {
+  val version: String = "0.11.1-IV0"
+  val messageFormatVersion: Byte = RecordBatch.MAGIC_VALUE_V2
+  val id: Int = 13
+}
+

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/api/LeaderAndIsr.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/LeaderAndIsr.scala b/core/src/main/scala/kafka/api/LeaderAndIsr.scala
index 474d7a0..e92dc33 100644
--- a/core/src/main/scala/kafka/api/LeaderAndIsr.scala
+++ b/core/src/main/scala/kafka/api/LeaderAndIsr.scala
@@ -50,13 +50,26 @@ case class LeaderAndIsr(leader: Int,
   }
 }
 
-case class PartitionStateInfo(leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, allReplicas: Seq[Int]) {
+case class LeaderAndIsrPartitionState(leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, allReplicas: Seq[Int], isNew: Boolean) {
 
   override def toString: String = {
     val partitionStateInfo = new StringBuilder
     partitionStateInfo.append("(LeaderAndIsrInfo:" + leaderIsrAndControllerEpoch.toString)
     partitionStateInfo.append(",ReplicationFactor:" + allReplicas.size + ")")
     partitionStateInfo.append(",AllReplicas:" + allReplicas.mkString(",") + ")")
+    partitionStateInfo.append(",isNew:" + isNew + ")")
+    partitionStateInfo.toString()
+  }
+}
+
+case class MetadataPartitionState(leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch, allReplicas: Seq[Int], offlineReplicas: Seq[Int]) {
+
+  override def toString: String = {
+    val partitionStateInfo = new StringBuilder
+    partitionStateInfo.append("(LeaderAndIsrInfo:" + leaderIsrAndControllerEpoch.toString)
+    partitionStateInfo.append(",ReplicationFactor:" + allReplicas.size + ")")
+    partitionStateInfo.append(",AllReplicas:" + allReplicas.mkString(",") + ")")
+    partitionStateInfo.append(",OfflineReplicas:" + offlineReplicas.mkString(",") + ")")
     partitionStateInfo.toString()
   }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/cluster/Partition.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala
index ebf3140..2c4767b 100755
--- a/core/src/main/scala/kafka/cluster/Partition.scala
+++ b/core/src/main/scala/kafka/cluster/Partition.scala
@@ -16,9 +16,8 @@
  */
 package kafka.cluster
 
-import java.io.IOException
-import java.util.concurrent.locks.ReentrantReadWriteLock
 
+import java.util.concurrent.locks.ReentrantReadWriteLock
 import com.yammer.metrics.core.Gauge
 import kafka.admin.AdminUtils
 import kafka.api.LeaderAndIsr
@@ -46,12 +45,15 @@ import scala.collection.JavaConverters._
 class Partition(val topic: String,
                 val partitionId: Int,
                 time: Time,
-                replicaManager: ReplicaManager) extends Logging with KafkaMetricsGroup {
+                replicaManager: ReplicaManager,
+                val isOffline: Boolean = false) extends Logging with KafkaMetricsGroup {
+
   val topicPartition = new TopicPartition(topic, partitionId)
 
-  private val localBrokerId = replicaManager.config.brokerId
-  private val logManager = replicaManager.logManager
-  private val zkUtils = replicaManager.zkUtils
+  // Do not use replicaManager if this partition is ReplicaManager.OfflinePartition
+  private val localBrokerId = if (!isOffline) replicaManager.config.brokerId else -1
+  private val logManager = if (!isOffline) replicaManager.logManager else null
+  private val zkUtils = if (!isOffline) replicaManager.zkUtils else null
   private val assignedReplicaMap = new Pool[Int, Replica]
   // The read lock is only required when multiple reads are executed and needs to be in a consistent manner
   private val leaderIsrUpdateLock = new ReentrantReadWriteLock
@@ -71,56 +73,59 @@ class Partition(val topic: String,
   private def isReplicaLocal(replicaId: Int) : Boolean = replicaId == localBrokerId
   val tags = Map("topic" -> topic, "partition" -> partitionId.toString)
 
-  newGauge("UnderReplicated",
-    new Gauge[Int] {
-      def value = {
-        if (isUnderReplicated) 1 else 0
-      }
-    },
-    tags
-  )
-
-  newGauge("InSyncReplicasCount",
-    new Gauge[Int] {
-      def value = {
-        if (isLeaderReplicaLocal) inSyncReplicas.size else 0
-      }
-    },
-    tags
-  )
-
-  newGauge("ReplicasCount",
-    new Gauge[Int] {
-      def value = {
-        if (isLeaderReplicaLocal) assignedReplicas.size else 0
-      }
-    },
-    tags
-  )
-
-  newGauge("LastStableOffsetLag",
-    new Gauge[Long] {
-      def value = {
-        leaderReplicaIfLocal.map { replica =>
-          replica.highWatermark.messageOffset - replica.lastStableOffset.messageOffset
-        }.getOrElse(0)
-      }
-    },
-    tags
-  )
+  // Do not create metrics if this partition is ReplicaManager.OfflinePartition
+  if (!isOffline) {
+    newGauge("UnderReplicated",
+      new Gauge[Int] {
+        def value = {
+          if (isUnderReplicated) 1 else 0
+        }
+      },
+      tags
+    )
+
+    newGauge("InSyncReplicasCount",
+      new Gauge[Int] {
+        def value = {
+          if (isLeaderReplicaLocal) inSyncReplicas.size else 0
+        }
+      },
+      tags
+    )
+
+    newGauge("ReplicasCount",
+      new Gauge[Int] {
+        def value = {
+          if (isLeaderReplicaLocal) assignedReplicas.size else 0
+        }
+      },
+      tags
+    )
+
+    newGauge("LastStableOffsetLag",
+      new Gauge[Long] {
+        def value = {
+          leaderReplicaIfLocal.map { replica =>
+            replica.highWatermark.messageOffset - replica.lastStableOffset.messageOffset
+          }.getOrElse(0)
+        }
+      },
+      tags
+    )
+  }
 
   private def isLeaderReplicaLocal: Boolean = leaderReplicaIfLocal.isDefined
 
   def isUnderReplicated: Boolean =
     isLeaderReplicaLocal && inSyncReplicas.size < assignedReplicas.size
 
-  def getOrCreateReplica(replicaId: Int = localBrokerId): Replica = {
+  def getOrCreateReplica(replicaId: Int = localBrokerId, isNew: Boolean = false): Replica = {
     assignedReplicaMap.getAndMaybePut(replicaId, {
       if (isReplicaLocal(replicaId)) {
         val config = LogConfig.fromProps(logManager.defaultConfig.originals,
                                          AdminUtils.fetchEntityConfig(zkUtils, ConfigType.Topic, topic))
-        val log = logManager.createLog(topicPartition, config)
-        val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParentFile.getAbsolutePath)
+        val log = logManager.getOrCreateLog(topicPartition, config, isNew)
+        val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParent)
         val offsetMap = checkpoint.read
         if (!offsetMap.contains(topicPartition))
           info(s"No checkpointed highwatermark is found for partition $topicPartition")
@@ -151,14 +156,8 @@ class Partition(val topic: String,
       assignedReplicaMap.clear()
       inSyncReplicas = Set.empty[Replica]
       leaderReplicaIdOpt = None
-      try {
-        logManager.asyncDelete(topicPartition)
-        removePartitionMetrics()
-      } catch {
-        case e: IOException =>
-          fatal(s"Error deleting the log for partition $topicPartition", e)
-          Exit.halt(1)
-      }
+      removePartitionMetrics()
+      logManager.asyncDelete(topicPartition)
     }
   }
 
@@ -176,7 +175,7 @@ class Partition(val topic: String,
       // to maintain the decision maker controller's epoch in the zookeeper path
       controllerEpoch = partitionStateInfo.controllerEpoch
       // add replicas that are new
-      val newInSyncReplicas = partitionStateInfo.isr.asScala.map(r => getOrCreateReplica(r)).toSet
+      val newInSyncReplicas = partitionStateInfo.isr.asScala.map(r => getOrCreateReplica(r, partitionStateInfo.isNew)).toSet
       // remove assigned replicas that have been removed by the controller
       (assignedReplicas.map(_.brokerId) -- allReplicas).foreach(removeReplica)
       inSyncReplicas = newInSyncReplicas
@@ -185,7 +184,7 @@ class Partition(val topic: String,
 
       //We cache the leader epoch here, persisting it only if it's local (hence having a log dir)
       leaderEpoch = partitionStateInfo.leaderEpoch
-      allReplicas.foreach(id => getOrCreateReplica(id))
+      allReplicas.foreach(id => getOrCreateReplica(id, partitionStateInfo.isNew))
 
       zkVersion = partitionStateInfo.zkVersion
       val isNewLeader =
@@ -230,7 +229,7 @@ class Partition(val topic: String,
       // to maintain the decision maker controller's epoch in the zookeeper path
       controllerEpoch = partitionStateInfo.controllerEpoch
       // add replicas that are new
-      allReplicas.foreach(r => getOrCreateReplica(r))
+      allReplicas.foreach(r => getOrCreateReplica(r, partitionStateInfo.isNew))
       // remove assigned replicas that have been removed by the controller
       (assignedReplicas.map(_.brokerId) -- allReplicas).foreach(removeReplica)
       inSyncReplicas = Set.empty[Replica]
@@ -557,7 +556,7 @@ class Partition(val topic: String,
   /**
    * remove deleted log metrics
    */
-  private def removePartitionMetrics() {
+  def removePartitionMetrics() {
     removeMetric("UnderReplicated", tags)
     removeMetric("InSyncReplicasCount", tags)
     removeMetric("ReplicasCount", tags)
@@ -565,12 +564,12 @@ class Partition(val topic: String,
   }
 
   override def equals(that: Any): Boolean = that match {
-    case other: Partition => partitionId == other.partitionId && topic == other.topic
+    case other: Partition => partitionId == other.partitionId && topic == other.topic && isOffline == other.isOffline
     case _ => false
   }
 
   override def hashCode: Int =
-    31 + topic.hashCode + 17 * partitionId
+    31 + topic.hashCode + 17 * partitionId + (if (isOffline) 1 else 0)
 
   override def toString: String = {
     val partitionString = new StringBuilder

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/common/KafkaStorageException.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/common/KafkaStorageException.scala b/core/src/main/scala/kafka/common/KafkaStorageException.scala
index 21dd583..e0ecff3 100644
--- a/core/src/main/scala/kafka/common/KafkaStorageException.scala
+++ b/core/src/main/scala/kafka/common/KafkaStorageException.scala
@@ -17,8 +17,10 @@
 package kafka.common
 
 /**
- * Kafka exception caused by real IOs.
-*/
+ * Kafka exception caused by disk-related IOException
+ * This class is deprecated and will be replaced by org.apache.kafka.common.errors.KafkaStorageException
+ */
+@Deprecated
 class KafkaStorageException(message: String, t: Throwable) extends RuntimeException(message, t) {
   def this(message: String) = this(message, null)
   def this(t: Throwable) = this("", t)


[4/5] kafka git commit: KAFKA-4763; Handle disk failure for JBOD (KIP-112)

Posted by jq...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
index ee8fa1e..bb3b14b 100755
--- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
+++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
@@ -31,7 +31,7 @@ import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.network._
 import org.apache.kafka.common.protocol.{ApiKeys, SecurityProtocol}
 import org.apache.kafka.common.requests.UpdateMetadataRequest.EndPoint
-import org.apache.kafka.common.requests.{UpdateMetadataRequest, _}
+import org.apache.kafka.common.requests._
 import org.apache.kafka.common.security.JaasContext
 import org.apache.kafka.common.utils.Time
 import org.apache.kafka.common.{Node, TopicPartition, requests}
@@ -281,10 +281,10 @@ class RequestSendThread(val controllerId: Int,
 class ControllerBrokerRequestBatch(controller: KafkaController) extends  Logging {
   val controllerContext = controller.controllerContext
   val controllerId: Int = controller.config.brokerId
-  val leaderAndIsrRequestMap = mutable.Map.empty[Int, mutable.Map[TopicPartition, PartitionStateInfo]]
+  val leaderAndIsrRequestMap = mutable.Map.empty[Int, mutable.Map[TopicPartition, LeaderAndIsrPartitionState]]
   val stopReplicaRequestMap = mutable.Map.empty[Int, Seq[StopReplicaRequestInfo]]
   val updateMetadataRequestBrokerSet = mutable.Set.empty[Int]
-  val updateMetadataRequestPartitionInfoMap = mutable.Map.empty[TopicPartition, PartitionStateInfo]
+  val updateMetadataRequestPartitionInfoMap = mutable.Map.empty[TopicPartition, MetadataPartitionState]
   private val stateChangeLogger = KafkaController.stateChangeLogger
 
   def newBatch() {
@@ -310,12 +310,13 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends  Logging
 
   def addLeaderAndIsrRequestForBrokers(brokerIds: Seq[Int], topic: String, partition: Int,
                                        leaderIsrAndControllerEpoch: LeaderIsrAndControllerEpoch,
-                                       replicas: Seq[Int], callback: AbstractResponse => Unit = null) {
+                                       replicas: Seq[Int], isNew: Boolean = false) {
     val topicPartition = new TopicPartition(topic, partition)
 
     brokerIds.filter(_ >= 0).foreach { brokerId =>
       val result = leaderAndIsrRequestMap.getOrElseUpdate(brokerId, mutable.Map.empty)
-      result.put(topicPartition, PartitionStateInfo(leaderIsrAndControllerEpoch, replicas))
+      val alreadyNew = result.get(topicPartition).exists(_.isNew)
+      result.put(topicPartition, LeaderAndIsrPartitionState(leaderIsrAndControllerEpoch, replicas, isNew || alreadyNew))
     }
 
     addUpdateMetadataRequestForBrokers(controllerContext.liveOrShuttingDownBrokerIds.toSeq,
@@ -345,7 +346,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends  Logging
       leaderIsrAndControllerEpochOpt match {
         case Some(l @ LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)) =>
           val replicas = controllerContext.partitionReplicaAssignment(partition)
-
+          val offlineReplicas = replicas.filter(!controllerContext.isReplicaOnline(_, partition))
           val leaderIsrAndControllerEpoch = if (beingDeleted) {
             val leaderDuringDelete = LeaderAndIsr.duringDelete(leaderAndIsr.isr)
             LeaderIsrAndControllerEpoch(leaderDuringDelete, controllerEpoch)
@@ -353,7 +354,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends  Logging
             l
           }
 
-          val partitionStateInfo = PartitionStateInfo(leaderIsrAndControllerEpoch, replicas)
+          val partitionStateInfo = MetadataPartitionState(leaderIsrAndControllerEpoch, replicas, offlineReplicas)
           updateMetadataRequestPartitionInfoMap.put(new TopicPartition(partition.topic, partition.partition), partitionStateInfo)
 
         case None =>
@@ -379,8 +380,12 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends  Logging
 
   def sendRequestsToBrokers(controllerEpoch: Int) {
     try {
-      leaderAndIsrRequestMap.foreach { case (broker, partitionStateInfos) =>
-        partitionStateInfos.foreach { case (topicPartition, state) =>
+      val leaderAndIsrRequestVersion: Short =
+        if (controller.config.interBrokerProtocolVersion >= KAFKA_0_11_1_IV0) 1
+        else 0
+
+      leaderAndIsrRequestMap.foreach { case (broker, leaderAndIsrPartitionStates) =>
+        leaderAndIsrPartitionStates.foreach { case (topicPartition, state) =>
           val typeOfRequest =
             if (broker == state.leaderIsrAndControllerEpoch.leaderAndIsr.leader) "become-leader"
             else "become-follower"
@@ -389,20 +394,21 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends  Logging
                                                                    state.leaderIsrAndControllerEpoch, broker,
                                                                    topicPartition.topic, topicPartition.partition))
         }
-        val leaderIds = partitionStateInfos.map(_._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader).toSet
+        val leaderIds = leaderAndIsrPartitionStates.map(_._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader).toSet
         val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)).map {
           _.getNode(controller.config.interBrokerListenerName)
         }
-        val partitionStates = partitionStateInfos.map { case (topicPartition, partitionStateInfo) =>
-          val LeaderIsrAndControllerEpoch(leaderIsr, controllerEpoch) = partitionStateInfo.leaderIsrAndControllerEpoch
+        val partitionStates = leaderAndIsrPartitionStates.map { case (topicPartition, leaderAndIsrPartitionState) =>
+          val LeaderIsrAndControllerEpoch(leaderIsr, controllerEpoch) = leaderAndIsrPartitionState.leaderIsrAndControllerEpoch
           val partitionState = new requests.PartitionState(controllerEpoch, leaderIsr.leader,
             leaderIsr.leaderEpoch, leaderIsr.isr.map(Integer.valueOf).asJava, leaderIsr.zkVersion,
-            partitionStateInfo.allReplicas.map(Integer.valueOf).asJava)
+            leaderAndIsrPartitionState.allReplicas.map(Integer.valueOf).asJava, leaderAndIsrPartitionState.isNew)
           topicPartition -> partitionState
         }
-        val leaderAndIsrRequest = new LeaderAndIsrRequest.Builder(controllerId, controllerEpoch, partitionStates.asJava,
-          leaders.asJava)
-        controller.sendRequest(broker, ApiKeys.LEADER_AND_ISR, leaderAndIsrRequest)
+        val leaderAndIsrRequestBuilder = new LeaderAndIsrRequest.Builder(leaderAndIsrRequestVersion, controllerId,
+          controllerEpoch, partitionStates.asJava, leaders.asJava)
+        controller.sendRequest(broker, ApiKeys.LEADER_AND_ISR, leaderAndIsrRequestBuilder,
+          (r: AbstractResponse) => controller.eventManager.put(controller.LeaderAndIsrResponseReceived(r, broker)))
       }
       leaderAndIsrRequestMap.clear()
 
@@ -411,20 +417,21 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends  Logging
         updateMetadataRequestBrokerSet.toString(), p._1)))
       val partitionStates = updateMetadataRequestPartitionInfoMap.map { case (topicPartition, partitionStateInfo) =>
         val LeaderIsrAndControllerEpoch(leaderIsr, controllerEpoch) = partitionStateInfo.leaderIsrAndControllerEpoch
-        val partitionState = new requests.PartitionState(controllerEpoch, leaderIsr.leader,
+        val partitionState = new UpdateMetadataRequest.PartitionState(controllerEpoch, leaderIsr.leader,
           leaderIsr.leaderEpoch, leaderIsr.isr.map(Integer.valueOf).asJava, leaderIsr.zkVersion,
-          partitionStateInfo.allReplicas.map(Integer.valueOf).asJava)
+          partitionStateInfo.allReplicas.map(Integer.valueOf).asJava, partitionStateInfo.offlineReplicas.map(Integer.valueOf).asJava)
         topicPartition -> partitionState
       }
 
-      val version: Short =
-        if (controller.config.interBrokerProtocolVersion >= KAFKA_0_10_2_IV0) 3
+      val updateMetadataRequestVersion: Short =
+        if (controller.config.interBrokerProtocolVersion >= KAFKA_0_11_1_IV0) 4
+        else if (controller.config.interBrokerProtocolVersion >= KAFKA_0_10_2_IV0) 3
         else if (controller.config.interBrokerProtocolVersion >= KAFKA_0_10_0_IV1) 2
         else if (controller.config.interBrokerProtocolVersion >= KAFKA_0_9_0) 1
         else 0
 
       val updateMetadataRequest = {
-        val liveBrokers = if (version == 0) {
+        val liveBrokers = if (updateMetadataRequestVersion == 0) {
           // Version 0 of UpdateMetadataRequest only supports PLAINTEXT.
           controllerContext.liveOrShuttingDownBrokers.map { broker =>
             val securityProtocol = SecurityProtocol.PLAINTEXT
@@ -441,7 +448,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends  Logging
             new UpdateMetadataRequest.Broker(broker.id, endPoints.asJava, broker.rack.orNull)
           }
         }
-        new UpdateMetadataRequest.Builder(version, controllerId, controllerEpoch, partitionStates.asJava,
+        new UpdateMetadataRequest.Builder(updateMetadataRequestVersion, controllerId, controllerEpoch, partitionStates.asJava,
           liveBrokers.asJava)
       }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/controller/ControllerState.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/ControllerState.scala b/core/src/main/scala/kafka/controller/ControllerState.scala
index 2f690bb..74029b1 100644
--- a/core/src/main/scala/kafka/controller/ControllerState.scala
+++ b/core/src/main/scala/kafka/controller/ControllerState.scala
@@ -78,6 +78,15 @@ object ControllerState {
     def value = 9
   }
 
+  case object LeaderAndIsrResponseReceived extends ControllerState {
+    def value = 10
+  }
+
+  case object LogDirChange extends ControllerState {
+    def value = 11
+  }
+
   val values: Seq[ControllerState] = Seq(Idle, ControllerChange, BrokerChange, TopicChange, TopicDeletion,
-    PartitionReassignment, AutoLeaderBalance, ManualLeaderBalance, ControlledShutdown, IsrChange)
+    PartitionReassignment, AutoLeaderBalance, ManualLeaderBalance, ControlledShutdown, IsrChange, LeaderAndIsrResponseReceived,
+    LogDirChange)
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/controller/KafkaController.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala
index ff47f14..3a61a59 100644
--- a/core/src/main/scala/kafka/controller/KafkaController.scala
+++ b/core/src/main/scala/kafka/controller/KafkaController.scala
@@ -33,7 +33,7 @@ import org.I0Itec.zkclient.{IZkChildListener, IZkDataListener, IZkStateListener}
 import org.apache.kafka.common.errors.{BrokerNotAvailableException, ControllerMovedException}
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.protocol.{ApiKeys, Errors}
-import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, StopReplicaResponse}
+import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, StopReplicaResponse, LeaderAndIsrResponse}
 import org.apache.kafka.common.utils.Time
 import org.apache.zookeeper.Watcher.Event.KeeperState
 
@@ -52,6 +52,7 @@ class ControllerContext(val zkUtils: ZkUtils) {
   var partitionReplicaAssignment: mutable.Map[TopicAndPartition, Seq[Int]] = mutable.Map.empty
   var partitionLeadershipInfo: mutable.Map[TopicAndPartition, LeaderIsrAndControllerEpoch] = mutable.Map.empty
   val partitionsBeingReassigned: mutable.Map[TopicAndPartition, ReassignedPartitionsContext] = new mutable.HashMap
+  val replicasOnOfflineDirs: mutable.Map[Int, Set[TopicAndPartition]] = mutable.HashMap.empty
 
   private var liveBrokersUnderlying: Set[Broker] = Set.empty
   private var liveBrokerIdsUnderlying: Set[Int] = Set.empty
@@ -75,6 +76,14 @@ class ControllerContext(val zkUtils: ZkUtils) {
     }.toSet
   }
 
+  def isReplicaOnline(brokerId: Int, topicAndPartition: TopicAndPartition, includeShuttingDownBrokers: Boolean = false): Boolean = {
+    val brokerOnline = {
+      if (includeShuttingDownBrokers) liveOrShuttingDownBrokerIds.contains(brokerId)
+      else liveBrokerIds.contains(brokerId)
+    }
+    brokerOnline && !replicasOnOfflineDirs.getOrElse(brokerId, Set.empty).contains(topicAndPartition)
+  }
+
   def replicasOnBrokers(brokerIds: Set[Int]): Set[PartitionAndReplica] = {
     brokerIds.flatMap { brokerId =>
       partitionReplicaAssignment.collect {
@@ -98,7 +107,9 @@ class ControllerContext(val zkUtils: ZkUtils) {
     partitionReplicaAssignment.keySet.filter(topicAndPartition => topicAndPartition.topic == topic)
 
   def allLiveReplicas(): Set[PartitionAndReplica] = {
-    replicasOnBrokers(liveBrokerIds)
+    replicasOnBrokers(liveBrokerIds).filter { partitionAndReplica =>
+      isReplicaOnline(partitionAndReplica.replica, TopicAndPartition(partitionAndReplica.topic, partitionAndReplica.partition))
+    }
   }
 
   def replicasForPartition(partitions: collection.Set[TopicAndPartition]): collection.Set[PartitionAndReplica] = {
@@ -175,6 +186,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met
   private val partitionReassignmentListener = new PartitionReassignmentListener(this, eventManager)
   private val preferredReplicaElectionListener = new PreferredReplicaElectionListener(this, eventManager)
   private val isrChangeNotificationListener = new IsrChangeNotificationListener(this, eventManager)
+  private val logDirEventNotificationListener = new LogDirEventNotificationListener(this, eventManager)
 
   @volatile private var activeControllerId = -1
   @volatile private var offlinePartitionCount = 0
@@ -248,6 +260,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met
     info("Broker %d starting become controller state transition".format(config.brokerId))
     readControllerEpochFromZookeeper()
     incrementControllerEpoch()
+    LogDirUtils.deleteLogDirEvents(zkUtils)
 
     // before reading source of truth from zookeeper, register the listeners to get broker/topic callbacks
     registerPartitionReassignmentListener()
@@ -256,6 +269,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met
     registerTopicChangeListener()
     registerTopicDeletionListener()
     registerBrokerChangeListener()
+    registerLogDirEventNotificationListener()
 
     initializeControllerContext()
     val (topicsToBeDeleted, topicsIneligibleForDeletion) = fetchTopicDeletionsInProgress()
@@ -299,6 +313,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met
     deregisterIsrChangeNotificationListener()
     deregisterPartitionReassignmentListener()
     deregisterPreferredReplicaElectionListener()
+    deregisterLogDirEventNotificationListener()
 
     // reset topic deletion manager
     topicDeletionManager.reset()
@@ -329,6 +344,17 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met
    */
   def isActive: Boolean = activeControllerId == config.brokerId
 
+  /*
+   * This callback is invoked by the controller's LogDirEventNotificationListener with the list of broker ids who
+   * have experienced new log directory failures. In response the controller should send LeaderAndIsrRequest
+   * to all these brokers to query the state of their replicas
+   */
+  def onBrokerLogDirFailure(brokerIds: Seq[Int]) {
+    // send LeaderAndIsrRequest for all replicas on those brokers to see if they are still online.
+    val replicasOnBrokers = controllerContext.replicasOnBrokers(brokerIds.toSet)
+    replicaStateMachine.handleStateChanges(replicasOnBrokers, OnlineReplica)
+  }
+
   /**
    * This callback is invoked by the replica state machine's broker change listener, with the list of newly started
    * brokers as input. It does the following -
@@ -345,6 +371,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met
    */
   def onBrokerStartup(newBrokers: Seq[Int]) {
     info("New broker startup callback for %s".format(newBrokers.mkString(",")))
+    newBrokers.foreach(controllerContext.replicasOnOfflineDirs.remove)
     val newBrokersSet = newBrokers.toSet
     // send update metadata request to all live and shutting down brokers. Old brokers will get to know of the new
     // broker via this update.
@@ -374,46 +401,55 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met
     }
   }
 
-  /**
+  /*
    * This callback is invoked by the replica state machine's broker change listener with the list of failed brokers
-   * as input. It does the following -
-   * 1. Mark partitions with dead leaders as offline
-   * 2. Triggers the OnlinePartition state change for all new/offline partitions
-   * 3. Invokes the OfflineReplica state change on the input list of newly started brokers
-   * 4. If no partitions are effected then send UpdateMetadataRequest to live or shutting down brokers
-   *
-   * Note that we don't need to refresh the leader/isr cache for all topic/partitions at this point.  This is because
-   * the partition state machine will refresh our cache for us when performing leader election for all new/offline
-   * partitions coming online.
+   * as input. It will call onReplicaBecomeOffline(...) with the list of replicas on those failed brokers as input.
    */
   def onBrokerFailure(deadBrokers: Seq[Int]) {
     info("Broker failure callback for %s".format(deadBrokers.mkString(",")))
+    deadBrokers.foreach(controllerContext.replicasOnOfflineDirs.remove)
     val deadBrokersThatWereShuttingDown =
       deadBrokers.filter(id => controllerContext.shuttingDownBrokerIds.remove(id))
     info("Removed %s from list of shutting down brokers.".format(deadBrokersThatWereShuttingDown))
-    val deadBrokersSet = deadBrokers.toSet
-    // trigger OfflinePartition state for all partitions whose current leader is one amongst the dead brokers
+    val allReplicasOnDeadBrokers = controllerContext.replicasOnBrokers(deadBrokers.toSet)
+    onReplicasBecomeOffline(allReplicasOnDeadBrokers)
+  }
+
+  /**
+    * This method marks the given replicas as offline. It does the following -
+    * 1. Mark the given partitions as offline
+    * 2. Triggers the OnlinePartition state change for all new/offline partitions
+    * 3. Invokes the OfflineReplica state change on the input list of newly offline replicas
+    * 4. If no partitions are affected then send UpdateMetadataRequest to live or shutting down brokers
+    *
+    * Note that we don't need to refresh the leader/isr cache for all topic/partitions at this point.  This is because
+    * the partition state machine will refresh our cache for us when performing leader election for all new/offline
+    * partitions coming online.
+    */
+  def onReplicasBecomeOffline(newOfflineReplicas: Set[PartitionAndReplica]): Unit = {
+    val (newOfflineReplicasForDeletion, newOfflineReplicasNotForDeletion) =
+      newOfflineReplicas.partition(p => topicDeletionManager.isTopicQueuedUpForDeletion(p.topic))
+
     val partitionsWithoutLeader = controllerContext.partitionLeadershipInfo.filter(partitionAndLeader =>
-      deadBrokersSet.contains(partitionAndLeader._2.leaderAndIsr.leader) &&
+      !controllerContext.isReplicaOnline(partitionAndLeader._2.leaderAndIsr.leader, partitionAndLeader._1) &&
         !topicDeletionManager.isTopicQueuedUpForDeletion(partitionAndLeader._1.topic)).keySet
+
+    // trigger OfflinePartition state for all partitions whose current leader is one amongst the newOfflineReplicas
     partitionStateMachine.handleStateChanges(partitionsWithoutLeader, OfflinePartition)
     // trigger OnlinePartition state changes for offline or new partitions
     partitionStateMachine.triggerOnlinePartitionStateChange()
-    // filter out the replicas that belong to topics that are being deleted
-    var allReplicasOnDeadBrokers = controllerContext.replicasOnBrokers(deadBrokersSet)
-    val activeReplicasOnDeadBrokers = allReplicasOnDeadBrokers.filterNot(p => topicDeletionManager.isTopicQueuedUpForDeletion(p.topic))
-    // handle dead replicas
-    replicaStateMachine.handleStateChanges(activeReplicasOnDeadBrokers, OfflineReplica)
-    // check if topic deletion state for the dead replicas needs to be updated
-    val replicasForTopicsToBeDeleted = allReplicasOnDeadBrokers.filter(p => topicDeletionManager.isTopicQueuedUpForDeletion(p.topic))
-    if(replicasForTopicsToBeDeleted.nonEmpty) {
+    // trigger OfflineReplica state change for those newly offline replicas
+    replicaStateMachine.handleStateChanges(newOfflineReplicasNotForDeletion, OfflineReplica)
+
+    // fail deletion of topics that affected by the offline replicas
+    if (newOfflineReplicasForDeletion.nonEmpty) {
       // it is required to mark the respective replicas in TopicDeletionFailed state since the replica cannot be
-      // deleted when the broker is down. This will prevent the replica from being in TopicDeletionStarted state indefinitely
+      // deleted when its log directory is offline. This will prevent the replica from being in TopicDeletionStarted state indefinitely
       // since topic deletion cannot be retried until at least one replica is in TopicDeletionStarted state
-      topicDeletionManager.failReplicaDeletion(replicasForTopicsToBeDeleted)
+      topicDeletionManager.failReplicaDeletion(newOfflineReplicasForDeletion)
     }
 
-    // If broker failure did not require leader re-election, inform brokers of failed broker
+    // If replica failure did not require leader re-election, inform brokers of the offline replica
     // Note that during leader re-election, brokers update their metadata
     if (partitionsWithoutLeader.isEmpty) {
       sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq)
@@ -724,10 +760,11 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met
 
   private def fetchTopicDeletionsInProgress(): (Set[String], Set[String]) = {
     val topicsToBeDeleted = zkUtils.getChildrenParentMayNotExist(ZkUtils.DeleteTopicsPath).toSet
-    val topicsWithReplicasOnDeadBrokers = controllerContext.partitionReplicaAssignment.filter { case (_, replicas) =>
-      replicas.exists(r => !controllerContext.liveBrokerIds.contains(r)) }.keySet.map(_.topic)
+    val topicsWithOfflineReplicas = controllerContext.partitionReplicaAssignment.filter { case (partition, replicas) =>
+      replicas.exists(r => !controllerContext.isReplicaOnline(r, partition))
+    }.keySet.map(_.topic)
     val topicsForWhichPartitionReassignmentIsInProgress = controllerContext.partitionsBeingReassigned.keySet.map(_.topic)
-    val topicsIneligibleForDeletion = topicsWithReplicasOnDeadBrokers | topicsForWhichPartitionReassignmentIsInProgress
+    val topicsIneligibleForDeletion = topicsWithOfflineReplicas | topicsForWhichPartitionReassignmentIsInProgress
     info("List of topics to be deleted: %s".format(topicsToBeDeleted.mkString(",")))
     info("List of topics ineligible for deletion: %s".format(topicsIneligibleForDeletion.mkString(",")))
     (topicsToBeDeleted, topicsIneligibleForDeletion)
@@ -771,7 +808,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met
       partitionStateMachine.handleStateChanges(Set(topicAndPartition), OnlinePartition, reassignedPartitionLeaderSelector)
     } else {
       // check if the leader is alive or not
-      if (controllerContext.liveBrokerIds.contains(currentLeader)) {
+      if (controllerContext.isReplicaOnline(currentLeader, topicAndPartition)) {
         info("Leader %s for partition %s being reassigned, ".format(currentLeader, topicAndPartition) +
           "is already in the new list of replicas %s and is alive".format(reassignedReplicas.mkString(",")))
         // shrink replication factor and update the leader epoch in zookeeper to use on the next LeaderAndIsrRequest
@@ -909,6 +946,16 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met
     }
   }
 
+  private def registerLogDirEventNotificationListener() = {
+    debug("Registering logDirEventNotificationListener")
+    zkUtils.zkClient.subscribeChildChanges(ZkUtils.LogDirEventNotificationPath, logDirEventNotificationListener)
+  }
+
+  private def deregisterLogDirEventNotificationListener() = {
+    debug("De-registering logDirEventNotificationListener")
+    zkUtils.zkClient.unsubscribeChildChanges(ZkUtils.LogDirEventNotificationPath, logDirEventNotificationListener)
+  }
+
   private def readControllerEpochFromZookeeper() {
     // initialize the controller epoch and zk version by reading from zookeeper
     if(controllerContext.zkUtils.pathExists(ZkUtils.ControllerEpochPath)) {
@@ -1119,7 +1166,7 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met
         topicsNotInPreferredReplica.keys.foreach { topicPartition =>
           // do this check only if the broker is live and there are no partitions being reassigned currently
           // and preferred replica election is not in progress
-          if (controllerContext.liveBrokerIds.contains(leaderBroker) &&
+          if (controllerContext.isReplicaOnline(leaderBroker, topicPartition) &&
             controllerContext.partitionsBeingReassigned.isEmpty &&
             !topicDeletionManager.isTopicQueuedUpForDeletion(topicPartition.topic) &&
             controllerContext.allTopics.contains(topicPartition.topic)) {
@@ -1352,6 +1399,22 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met
 
   }
 
+  case class LogDirEventNotification(sequenceNumbers: Seq[String]) extends ControllerEvent {
+
+    def state = ControllerState.LogDirChange
+
+    override def process(): Unit = {
+      val zkUtils = controllerContext.zkUtils
+      try {
+        val brokerIds = sequenceNumbers.flatMap(LogDirUtils.getBrokerIdFromLogDirEvent(zkUtils, _))
+        onBrokerLogDirFailure(brokerIds)
+      } finally {
+        // delete processed children
+        sequenceNumbers.map(x => zkUtils.deletePath(ZkUtils.LogDirEventNotificationPath + "/" + x))
+      }
+    }
+  }
+
   case class PreferredReplicaLeaderElection(partitions: Set[TopicAndPartition]) extends ControllerEvent {
 
     def state = ControllerState.ManualLeaderBalance
@@ -1447,7 +1510,35 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, time: Time, met
     }
   }
 
-  case class TopicDeletionStopReplicaResult(stopReplicaResponseObj: AbstractResponse, replicaId: Int) extends ControllerEvent {
+  case class LeaderAndIsrResponseReceived(LeaderAndIsrResponseObj: AbstractResponse, brokerId: Int) extends ControllerEvent {
+
+    def state = ControllerState.LeaderAndIsrResponseReceived
+
+    override def process(): Unit = {
+      import JavaConverters._
+      val leaderAndIsrResponse = LeaderAndIsrResponseObj.asInstanceOf[LeaderAndIsrResponse]
+
+      if (leaderAndIsrResponse.error() != Errors.NONE) {
+        stateChangeLogger.error(s"Received error in leaderAndIsrResponse $leaderAndIsrResponse from broker $brokerId")
+        return
+      }
+
+      val offlineReplicas = leaderAndIsrResponse.responses().asScala.filter(_._2 == Errors.KAFKA_STORAGE_ERROR).keys.map(
+        tp => TopicAndPartition(tp.topic(), tp.partition())).toSet
+      val onlineReplicas = leaderAndIsrResponse.responses().asScala.filter(_._2 == Errors.NONE).keys.map(
+        tp => TopicAndPartition(tp.topic(), tp.partition())).toSet
+      val previousOfflineReplicas = controllerContext.replicasOnOfflineDirs.getOrElse(brokerId, Set.empty[TopicAndPartition])
+      val currentOfflineReplicas = previousOfflineReplicas -- onlineReplicas ++ offlineReplicas
+      controllerContext.replicasOnOfflineDirs.put(brokerId, currentOfflineReplicas)
+      val newOfflineReplicas = (currentOfflineReplicas -- previousOfflineReplicas).map(tp => PartitionAndReplica(tp.topic, tp.partition, brokerId))
+      stateChangeLogger.info(s"Mark replicas ${currentOfflineReplicas -- previousOfflineReplicas} on broker $brokerId as offline")
+
+      if (newOfflineReplicas.nonEmpty)
+        onReplicasBecomeOffline(newOfflineReplicas)
+    }
+  }
+
+  case class TopicDeletionStopReplicaResponseReceived(stopReplicaResponseObj: AbstractResponse, replicaId: Int) extends ControllerEvent {
 
     def state = ControllerState.TopicDeletion
 
@@ -1607,6 +1698,20 @@ class TopicChangeListener(controller: KafkaController, eventManager: ControllerE
   }
 }
 
+/**
+  * Called when broker notifies controller of log directory change
+  */
+class LogDirEventNotificationListener(controller: KafkaController, eventManager: ControllerEventManager) extends IZkChildListener with Logging {
+  override def handleChildChange(parentPath: String, currentChilds: java.util.List[String]): Unit = {
+    import JavaConverters._
+    eventManager.put(controller.LogDirEventNotification(currentChilds.asScala))
+  }
+}
+
+object LogDirEventNotificationListener {
+  val version: Long = 1L
+}
+
 class PartitionModificationsListener(controller: KafkaController, eventManager: ControllerEventManager, topic: String) extends IZkDataListener with Logging {
   override def handleDataChange(dataPath: String, data: Any): Unit = {
     eventManager.put(controller.PartitionModifications(topic))

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala
index 54bbb89..e534ff3 100644
--- a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala
+++ b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala
@@ -54,8 +54,8 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext, confi
   def selectLeader(topicAndPartition: TopicAndPartition, currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = {
     controllerContext.partitionReplicaAssignment.get(topicAndPartition) match {
       case Some(assignedReplicas) =>
-        val liveAssignedReplicas = assignedReplicas.filter(r => controllerContext.liveBrokerIds.contains(r))
-        val liveBrokersInIsr = currentLeaderAndIsr.isr.filter(r => controllerContext.liveBrokerIds.contains(r))
+        val liveAssignedReplicas = assignedReplicas.filter(r => controllerContext.isReplicaOnline(r, topicAndPartition))
+        val liveBrokersInIsr = currentLeaderAndIsr.isr.filter(r => controllerContext.isReplicaOnline(r, topicAndPartition))
         val newLeaderAndIsr =
           if (liveBrokersInIsr.isEmpty) {
             // Prior to electing an unclean (i.e. non-ISR) leader, ensure that doing so is not disallowed by the configuration
@@ -63,7 +63,7 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext, confi
             if (!LogConfig.fromProps(config.originals, AdminUtils.fetchEntityConfig(controllerContext.zkUtils,
               ConfigType.Topic, topicAndPartition.topic)).uncleanLeaderElectionEnable) {
               throw new NoReplicaOnlineException(
-                s"No broker in ISR for partition $topicAndPartition is alive. Live brokers are: [${controllerContext.liveBrokerIds}], " +
+                s"No replica in ISR for partition $topicAndPartition is alive. Live brokers are: [${controllerContext.liveBrokerIds}], " +
                   s"ISR brokers are: [${currentLeaderAndIsr.isr.mkString(",")}]"
               )
             }
@@ -111,7 +111,7 @@ class ReassignedPartitionLeaderSelector(controllerContext: ControllerContext) ex
                    currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = {
     val reassignedInSyncReplicas = controllerContext.partitionsBeingReassigned(topicAndPartition).newReplicas
     val newLeaderOpt = reassignedInSyncReplicas.find { r =>
-      controllerContext.liveBrokerIds.contains(r) && currentLeaderAndIsr.isr.contains(r)
+      controllerContext.isReplicaOnline(r, topicAndPartition) && currentLeaderAndIsr.isr.contains(r)
     }
     newLeaderOpt match {
       case Some(newLeader) => (currentLeaderAndIsr.newLeader(newLeader), reassignedInSyncReplicas)
@@ -150,7 +150,7 @@ class PreferredReplicaPartitionLeaderSelector(controllerContext: ControllerConte
       info("Current leader %d for partition %s is not the preferred replica.".format(currentLeader, topicAndPartition) +
         " Triggering preferred replica leader election")
       // check if preferred replica is not the current leader and is alive and in the isr
-      if (controllerContext.liveBrokerIds.contains(preferredReplica) && currentLeaderAndIsr.isr.contains(preferredReplica)) {
+      if (controllerContext.isReplicaOnline(preferredReplica, topicAndPartition) && currentLeaderAndIsr.isr.contains(preferredReplica)) {
         val newLeaderAndIsr = currentLeaderAndIsr.newLeader(preferredReplica)
         (newLeaderAndIsr, assignedReplicas)
       } else {
@@ -174,8 +174,7 @@ class ControlledShutdownLeaderSelector(controllerContext: ControllerContext) ext
                    currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = {
     val currentIsr = currentLeaderAndIsr.isr
     val assignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition)
-    val liveOrShuttingDownBrokerIds = controllerContext.liveOrShuttingDownBrokerIds
-    val liveAssignedReplicas = assignedReplicas.filter(r => liveOrShuttingDownBrokerIds.contains(r))
+    val liveAssignedReplicas = assignedReplicas.filter(r => controllerContext.isReplicaOnline(r, topicAndPartition, true))
 
     val newIsr = currentIsr.filter(brokerId => !controllerContext.shuttingDownBrokerIds.contains(brokerId))
     liveAssignedReplicas.find(newIsr.contains) match {

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala
index 5751e17..5024c02 100755
--- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala
+++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala
@@ -200,7 +200,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
       controllerContext.partitionLeadershipInfo.get(topicPartition) match {
         case Some(currentLeaderIsrAndEpoch) =>
           // else, check if the leader for partition is alive. If yes, it is in Online state, else it is in Offline state
-          if (controllerContext.liveBrokerIds.contains(currentLeaderIsrAndEpoch.leaderAndIsr.leader))
+          if (controllerContext.isReplicaOnline(currentLeaderIsrAndEpoch.leaderAndIsr.leader, topicPartition))
             // leader is alive
             partitionState.put(topicPartition, OnlinePartition)
           else
@@ -227,7 +227,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
    */
   private def initializeLeaderAndIsrForPartition(topicAndPartition: TopicAndPartition) = {
     val replicaAssignment = controllerContext.partitionReplicaAssignment(topicAndPartition).toList
-    val liveAssignedReplicas = replicaAssignment.filter(controllerContext.liveBrokerIds.contains)
+    val liveAssignedReplicas = replicaAssignment.filter(r => controllerContext.isReplicaOnline(r, topicAndPartition))
     liveAssignedReplicas.headOption match {
       case None =>
         val failMsg = s"Controller $controllerId epoch ${controller.epoch} encountered error during state change of " +
@@ -259,7 +259,8 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
             topicAndPartition.topic,
             topicAndPartition.partition,
             leaderIsrAndControllerEpoch,
-            replicaAssignment
+            replicaAssignment,
+            isNew = true
           )
         } catch {
           case _: ZkNodeExistsException =>

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala
index 43fac19..87c53b5 100755
--- a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala
+++ b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala
@@ -149,7 +149,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging {
                   .format(replicaId, topicAndPartition) + "state as it is being requested to become leader")
               brokerRequestBatch.addLeaderAndIsrRequestForBrokers(List(replicaId),
                                                                   topic, partition, leaderIsrAndControllerEpoch,
-                                                                  replicaAssignment)
+                                                                  replicaAssignment, isNew = true)
             case None => // new leader request will be sent to this replica when one gets elected
           }
           replicaState.put(partitionAndReplica, NewReplica)
@@ -283,7 +283,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging {
       val partition = topicPartition.partition
       assignedReplicas.foreach { replicaId =>
         val partitionAndReplica = PartitionAndReplica(topic, partition, replicaId)
-        if (controllerContext.liveBrokerIds.contains(replicaId))
+        if (controllerContext.isReplicaOnline(replicaId, topicPartition))
           replicaState.put(partitionAndReplica, OnlineReplica)
         else
           // mark replicas on dead brokers as failed for topic deletion, if they belong to a topic to be deleted.
@@ -297,6 +297,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging {
   def partitionsAssignedToBroker(topics: Seq[String], brokerId: Int):Seq[TopicAndPartition] = {
     controllerContext.partitionReplicaAssignment.filter(_._2.contains(brokerId)).keySet.toSeq
   }
+
 }
 
 sealed trait ReplicaState {

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala
index e483ac2..325488e 100755
--- a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala
+++ b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala
@@ -299,7 +299,7 @@ class TopicDeletionManager(controller: KafkaController, eventManager: Controller
       debug("Deletion started for replicas %s".format(replicasForDeletionRetry.mkString(",")))
       controller.replicaStateMachine.handleStateChanges(replicasForDeletionRetry, ReplicaDeletionStarted,
         new Callbacks.CallbackBuilder().stopReplicaCallback((stopReplicaResponseObj, replicaId) =>
-          eventManager.put(controller.TopicDeletionStopReplicaResult(stopReplicaResponseObj, replicaId))).build)
+          eventManager.put(controller.TopicDeletionStopReplicaResponseReceived(stopReplicaResponseObj, replicaId))).build)
       if (deadReplicasForTopic.nonEmpty) {
         debug("Dead Replicas (%s) found for topic %s".format(deadReplicasForTopic.mkString(","), topic))
         markTopicIneligibleForDeletion(Set(topic))

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
index 9322ff2..db2bd67 100644
--- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
+++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
@@ -199,7 +199,8 @@ class GroupMetadataManager(brokerId: Int,
                    | Errors.NOT_ENOUGH_REPLICAS_AFTER_APPEND =>
                 Errors.COORDINATOR_NOT_AVAILABLE
 
-              case Errors.NOT_LEADER_FOR_PARTITION =>
+              case Errors.NOT_LEADER_FOR_PARTITION
+                   | Errors.KAFKA_STORAGE_ERROR =>
                 Errors.NOT_COORDINATOR
 
               case Errors.REQUEST_TIMED_OUT =>
@@ -212,7 +213,7 @@ class GroupMetadataManager(brokerId: Int,
                 error(s"Appending metadata message for group ${group.groupId} generation $generationId failed due to " +
                   s"${status.error.exceptionName}, returning UNKNOWN error code to the client")
 
-                Errors.UNKNOWN
+                Errors.UNKNOWN_SERVER_ERROR
 
               case other =>
                 error(s"Appending metadata message for group ${group.groupId} generation $generationId failed " +
@@ -342,7 +343,8 @@ class GroupMetadataManager(brokerId: Int,
                        | Errors.NOT_ENOUGH_REPLICAS_AFTER_APPEND =>
                     Errors.COORDINATOR_NOT_AVAILABLE
 
-                  case Errors.NOT_LEADER_FOR_PARTITION =>
+                  case Errors.NOT_LEADER_FOR_PARTITION
+                       | Errors.KAFKA_STORAGE_ERROR =>
                     Errors.NOT_COORDINATOR
 
                   case Errors.MESSAGE_TOO_LARGE
@@ -695,7 +697,7 @@ class GroupMetadataManager(brokerId: Int,
           val timestampType = TimestampType.CREATE_TIME
           val timestamp = time.milliseconds()
 
-          val partitionOpt = replicaManager.getPartition(appendPartition)
+          val partitionOpt = replicaManager.getPartition(appendPartition).filter(_ ne ReplicaManager.OfflinePartition)
           partitionOpt.foreach { partition =>
             val tombstones = ListBuffer.empty[SimpleRecord]
             removedOffsets.foreach { case (topicPartition, offsetAndMetadata) =>

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala
index 85c19c5..e201e91 100644
--- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala
+++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala
@@ -266,7 +266,7 @@ class TransactionCoordinator(brokerId: Int,
   }
 
   def handleTxnImmigration(txnTopicPartitionId: Int, coordinatorEpoch: Int) {
-      txnManager.loadTransactionsForTxnTopicPartition(txnTopicPartitionId, coordinatorEpoch, txnMarkerChannelManager.addTxnMarkersToSend)
+    txnManager.loadTransactionsForTxnTopicPartition(txnTopicPartitionId, coordinatorEpoch, txnMarkerChannelManager.addTxnMarkersToSend)
   }
 
   def handleTxnEmigration(txnTopicPartitionId: Int, coordinatorEpoch: Int) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala
index e0d5076..394817c 100644
--- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala
+++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala
@@ -485,12 +485,13 @@ class TransactionStateManager(brokerId: Int,
                | Errors.REQUEST_TIMED_OUT => // note that for timed out request we return NOT_AVAILABLE error code to let client retry
             Errors.COORDINATOR_NOT_AVAILABLE
 
-          case Errors.NOT_LEADER_FOR_PARTITION =>
+          case Errors.NOT_LEADER_FOR_PARTITION
+               | Errors.KAFKA_STORAGE_ERROR =>
             Errors.NOT_COORDINATOR
 
           case Errors.MESSAGE_TOO_LARGE
                | Errors.RECORD_LIST_TOO_LARGE =>
-            Errors.UNKNOWN
+            Errors.UNKNOWN_SERVER_ERROR
 
           case other =>
             other

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/log/AbstractIndex.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/AbstractIndex.scala b/core/src/main/scala/kafka/log/AbstractIndex.scala
index bfc6828..d569ad9 100644
--- a/core/src/main/scala/kafka/log/AbstractIndex.scala
+++ b/core/src/main/scala/kafka/log/AbstractIndex.scala
@@ -178,6 +178,12 @@ abstract class AbstractIndex[K, V](@volatile var file: File, val baseOffset: Lon
     trimToValidSize()
   }
 
+  def closeHandler() = {
+    // File handler of the index field will be closed after the mmap is garbage collected
+    CoreUtils.swallow(forceUnmap(mmap))
+    mmap = null
+  }
+
   /**
    * Do a basic sanity check on this index to detect obvious problems
    *

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc93fb4b/core/src/main/scala/kafka/log/Log.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala
index 824d302..0fec75f 100644
--- a/core/src/main/scala/kafka/log/Log.scala
+++ b/core/src/main/scala/kafka/log/Log.scala
@@ -24,11 +24,11 @@ import java.util.concurrent.atomic._
 import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap, TimeUnit}
 
 import kafka.api.KAFKA_0_10_0_IV0
-import kafka.common._
+import kafka.common.{InvalidOffsetException, KafkaException, LongRef}
 import kafka.metrics.KafkaMetricsGroup
-import kafka.server.{BrokerTopicStats, FetchDataInfo, LogOffsetMetadata}
+import kafka.server.{BrokerTopicStats, FetchDataInfo, LogDirFailureChannel, LogOffsetMetadata}
 import kafka.utils._
-import org.apache.kafka.common.errors.{CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException, UnsupportedForMessageFormatException}
+import org.apache.kafka.common.errors.{KafkaStorageException, CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException, UnsupportedForMessageFormatException}
 import org.apache.kafka.common.record._
 import org.apache.kafka.common.requests.{IsolationLevel, ListOffsetRequest}
 
@@ -136,7 +136,8 @@ class Log(@volatile var dir: File,
           val maxProducerIdExpirationMs: Int,
           val producerIdExpirationCheckIntervalMs: Int,
           val topicPartition: TopicPartition,
-          val producerStateManager: ProducerStateManager) extends Logging with KafkaMetricsGroup {
+          val producerStateManager: ProducerStateManager,
+          logDirFailureChannel: LogDirFailureChannel) extends Logging with KafkaMetricsGroup {
 
   import kafka.log.Log._
 
@@ -164,7 +165,7 @@ class Log(@volatile var dir: File,
    * temporary abuse seems justifiable and saves us from scanning the log after deletion to find the first offsets
    * of each ongoing transaction in order to compute a new first unstable offset. It is possible, however,
    * that this could result in disagreement between replicas depending on when they began replicating the log.
-   * In the worst case, the LSO could be seen by a consumer to go backwards. 
+   * In the worst case, the LSO could be seen by a consumer to go backwards.
    */
   @volatile var firstUnstableOffset: Option[LogOffsetMetadata] = None
 
@@ -233,7 +234,7 @@ class Log(@volatile var dir: File,
     // create the log directory if it doesn't exist
     Files.createDirectories(dir.toPath)
     new LeaderEpochFileCache(topicPartition, () => logEndOffsetMetadata,
-      new LeaderEpochCheckpointFile(LeaderEpochFile.newFile(dir)))
+      new LeaderEpochCheckpointFile(LeaderEpochFile.newFile(dir), logDirFailureChannel))
   }
 
   private def removeTempFilesAndCollectSwapFiles(): Set[File] = {
@@ -266,6 +267,7 @@ class Log(@volatile var dir: File,
     swapFiles
   }
 
+  // This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
   private def loadSegmentFiles(): Unit = {
     // load segments in ascending order because transactional data from one segment may depend on the
     // segments that come before it
@@ -340,6 +342,7 @@ class Log(@volatile var dir: File,
     bytesTruncated
   }
 
+  // This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
   private def completeSwapOperations(swapFiles: Set[File]): Unit = {
     for (swapFile <- swapFiles) {
       val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, ""))
@@ -366,7 +369,8 @@ class Log(@volatile var dir: File,
     }
   }
 
-  /* Load the log segments from the log files on disk */
+  // Load the log segments from the log files on disk
+  // This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
   private def loadSegments() {
     // first do a pass through the files in the log directory and remove any temporary files
     // and find any interrupted swap operations
@@ -383,14 +387,14 @@ class Log(@volatile var dir: File,
     if(logSegments.isEmpty) {
       // no existing segments, create a new mutable segment beginning at offset 0
       segments.put(0L, new LogSegment(dir = dir,
-                                     startOffset = 0,
-                                     indexIntervalBytes = config.indexInterval,
-                                     maxIndexSize = config.maxIndexSize,
-                                     rollJitterMs = config.randomSegmentJitter,
-                                     time = time,
-                                     fileAlreadyExists = false,
-                                     initFileSize = this.initFileSize(),
-                                     preallocate = config.preallocate))
+                                      startOffset = 0,
+                                      indexIntervalBytes = config.indexInterval,
+                                      maxIndexSize = config.maxIndexSize,
+                                      rollJitterMs = config.randomSegmentJitter,
+                                      time = time,
+                                      fileAlreadyExists = false,
+                                      initFileSize = this.initFileSize(),
+                                      preallocate = config.preallocate))
     } else if (!dir.getAbsolutePath.endsWith(Log.DeleteDirSuffix)) {
       recoverLog()
       // reset the index size of the currently active log segment to allow more entries
@@ -403,6 +407,7 @@ class Log(@volatile var dir: File,
     nextOffsetMetadata = new LogOffsetMetadata(messageOffset, activeSegment.baseOffset, activeSegment.size)
   }
 
+  // This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded
   private def recoverLog() {
     // if we have the clean shutdown marker, skip recovery
     if(hasCleanShutdownFile) {
@@ -532,6 +537,16 @@ class Log(@volatile var dir: File,
   }
 
   /**
+   * Close file handlers used by log but don't write to disk. This is used when the disk may have failed
+   */
+  def closeHandlers() {
+    debug(s"Closing handlers of log $name")
+    lock synchronized {
+      logSegments.foreach(_.closeHandlers())
+    }
+  }
+
+  /**
    * Append this message set to the active segment of the log, assigning offsets and Partition Leader Epochs
    * @param records The records to append
    * @param isFromClient Whether or not this append is from a producer
@@ -566,16 +581,16 @@ class Log(@volatile var dir: File,
    * @return Information about the appended messages including the first and last offset.
    */
   private def append(records: MemoryRecords, isFromClient: Boolean, assignOffsets: Boolean, leaderEpoch: Int): LogAppendInfo = {
-    val appendInfo = analyzeAndValidateRecords(records, isFromClient = isFromClient)
+    maybeHandleIOException(s"Error while appending records to $topicPartition in dir ${dir.getParent}") {
+      val appendInfo = analyzeAndValidateRecords(records, isFromClient = isFromClient)
 
-    // return if we have no valid messages or if this is a duplicate of the last appended entry
-    if (appendInfo.shallowCount == 0)
-      return appendInfo
+      // return if we have no valid messages or if this is a duplicate of the last appended entry
+      if (appendInfo.shallowCount == 0)
+        return appendInfo
 
-    // trim any invalid bytes or partial messages before appending it to the on-disk log
-    var validRecords = trimInvalidBytes(records, appendInfo)
+      // trim any invalid bytes or partial messages before appending it to the on-disk log
+      var validRecords = trimInvalidBytes(records, appendInfo)
 
-    try {
       // they are valid, insert them in the log
       lock synchronized {
 
@@ -695,8 +710,6 @@ class Log(@volatile var dir: File,
 
         appendInfo
       }
-    } catch {
-      case e: IOException => throw new KafkaStorageException("I/O exception in append to log '%s'".format(name), e)
     }
   }
 
@@ -730,12 +743,14 @@ class Log(@volatile var dir: File,
     // We don't have to write the log start offset to log-start-offset-checkpoint immediately.
     // The deleteRecordsOffset may be lost only if all in-sync replicas of this broker are shutdown
     // in an unclean manner within log.flush.start.offset.checkpoint.interval.ms. The chance of this happening is low.
-    lock synchronized {
-      if (offset > logStartOffset) {
-        logStartOffset = offset
-        leaderEpochCache.clearAndFlushEarliest(logStartOffset)
-        producerStateManager.truncateHead(logStartOffset)
-        updateFirstUnstableOffset()
+    maybeHandleIOException(s"Exception while increasing log start offset for $topicPartition to $offset in dir ${dir.getParent}") {
+      lock synchronized {
+        if (offset > logStartOffset) {
+          logStartOffset = offset
+          leaderEpochCache.clearAndFlushEarliest(logStartOffset)
+          producerStateManager.truncateHead(logStartOffset)
+          updateFirstUnstableOffset()
+        }
       }
     }
   }
@@ -892,64 +907,66 @@ class Log(@volatile var dir: File,
    */
   def read(startOffset: Long, maxLength: Int, maxOffset: Option[Long] = None, minOneMessage: Boolean = false,
            isolationLevel: IsolationLevel): FetchDataInfo = {
-    trace("Reading %d bytes from offset %d in log %s of length %d bytes".format(maxLength, startOffset, name, size))
-
-    // Because we don't use lock for reading, the synchronization is a little bit tricky.
-    // We create the local variables to avoid race conditions with updates to the log.
-    val currentNextOffsetMetadata = nextOffsetMetadata
-    val next = currentNextOffsetMetadata.messageOffset
-    if (startOffset == next) {
-      val abortedTransactions =
-        if (isolationLevel == IsolationLevel.READ_COMMITTED) Some(List.empty[AbortedTransaction])
-        else None
-      return FetchDataInfo(currentNextOffsetMetadata, MemoryRecords.EMPTY, firstEntryIncomplete = false,
-        abortedTransactions = abortedTransactions)
-    }
+    maybeHandleIOException(s"Exception while reading from $topicPartition in dir ${dir.getParent}") {
+      trace("Reading %d bytes from offset %d in log %s of length %d bytes".format(maxLength, startOffset, name, size))
+
+      // Because we don't use lock for reading, the synchronization is a little bit tricky.
+      // We create the local variables to avoid race conditions with updates to the log.
+      val currentNextOffsetMetadata = nextOffsetMetadata
+      val next = currentNextOffsetMetadata.messageOffset
+      if (startOffset == next) {
+        val abortedTransactions =
+          if (isolationLevel == IsolationLevel.READ_COMMITTED) Some(List.empty[AbortedTransaction])
+          else None
+        return FetchDataInfo(currentNextOffsetMetadata, MemoryRecords.EMPTY, firstEntryIncomplete = false,
+          abortedTransactions = abortedTransactions)
+      }
 
-    var segmentEntry = segments.floorEntry(startOffset)
-
-    // return error on attempt to read beyond the log end offset or read below log start offset
-    if (startOffset > next || segmentEntry == null || startOffset < logStartOffset)
-      throw new OffsetOutOfRangeException("Request for offset %d but we only have log segments in the range %d to %d.".format(startOffset, logStartOffset, next))
-
-    // Do the read on the segment with a base offset less than the target offset
-    // but if that segment doesn't contain any messages with an offset greater than that
-    // continue to read from successive segments until we get some messages or we reach the end of the log
-    while(segmentEntry != null) {
-      val segment = segmentEntry.getValue
-
-      // If the fetch occurs on the active segment, there might be a race condition where two fetch requests occur after
-      // the message is appended but before the nextOffsetMetadata is updated. In that case the second fetch may
-      // cause OffsetOutOfRangeException. To solve that, we cap the reading up to exposed position instead of the log
-      // end of the active segment.
-      val maxPosition = {
-        if (segmentEntry == segments.lastEntry) {
-          val exposedPos = nextOffsetMetadata.relativePositionInSegment.toLong
-          // Check the segment again in case a new segment has just rolled out.
-          if (segmentEntry != segments.lastEntry)
+      var segmentEntry = segments.floorEntry(startOffset)
+
+      // return error on attempt to read beyond the log end offset or read below log start offset
+      if (startOffset > next || segmentEntry == null || startOffset < logStartOffset)
+        throw new OffsetOutOfRangeException("Request for offset %d but we only have log segments in the range %d to %d.".format(startOffset, logStartOffset, next))
+
+      // Do the read on the segment with a base offset less than the target offset
+      // but if that segment doesn't contain any messages with an offset greater than that
+      // continue to read from successive segments until we get some messages or we reach the end of the log
+      while (segmentEntry != null) {
+        val segment = segmentEntry.getValue
+
+        // If the fetch occurs on the active segment, there might be a race condition where two fetch requests occur after
+        // the message is appended but before the nextOffsetMetadata is updated. In that case the second fetch may
+        // cause OffsetOutOfRangeException. To solve that, we cap the reading up to exposed position instead of the log
+        // end of the active segment.
+        val maxPosition = {
+          if (segmentEntry == segments.lastEntry) {
+            val exposedPos = nextOffsetMetadata.relativePositionInSegment.toLong
+            // Check the segment again in case a new segment has just rolled out.
+            if (segmentEntry != segments.lastEntry)
             // New log segment has rolled out, we can read up to the file end.
+              segment.size
+            else
+              exposedPos
+          } else {
             segment.size
-          else
-            exposedPos
-        } else {
-          segment.size
+          }
         }
-      }
-      val fetchInfo = segment.read(startOffset, maxOffset, maxLength, maxPosition, minOneMessage)
-      if (fetchInfo == null) {
-        segmentEntry = segments.higherEntry(segmentEntry.getKey)
-      } else {
-        return isolationLevel match {
-          case IsolationLevel.READ_UNCOMMITTED => fetchInfo
-          case IsolationLevel.READ_COMMITTED => addAbortedTransactions(startOffset, segmentEntry, fetchInfo)
+        val fetchInfo = segment.read(startOffset, maxOffset, maxLength, maxPosition, minOneMessage)
+        if (fetchInfo == null) {
+          segmentEntry = segments.higherEntry(segmentEntry.getKey)
+        } else {
+          return isolationLevel match {
+            case IsolationLevel.READ_UNCOMMITTED => fetchInfo
+            case IsolationLevel.READ_COMMITTED => addAbortedTransactions(startOffset, segmentEntry, fetchInfo)
+          }
         }
       }
-    }
 
-    // okay we are beyond the end of the last segment with no data fetched although the start offset is in range,
-    // this can happen when all messages with offset larger than start offsets have been deleted.
-    // In this case, we will return the empty set with log end offset metadata
-    FetchDataInfo(nextOffsetMetadata, MemoryRecords.EMPTY)
+      // okay we are beyond the end of the last segment with no data fetched although the start offset is in range,
+      // this can happen when all messages with offset larger than start offsets have been deleted.
+      // In this case, we will return the empty set with log end offset metadata
+      FetchDataInfo(nextOffsetMetadata, MemoryRecords.EMPTY)
+    }
   }
 
   private[log] def collectAbortedTransactions(startOffset: Long, upperBoundOffset: Long): List[AbortedTxn] = {
@@ -1011,35 +1028,37 @@ class Log(@volatile var dir: File,
    *         None if no such message is found.
    */
   def fetchOffsetsByTimestamp(targetTimestamp: Long): Option[TimestampOffset] = {
-    debug(s"Searching offset for timestamp $targetTimestamp")
+    maybeHandleIOException(s"Error while fetching offset by timestamp for $topicPartition in dir ${dir.getParent}") {
+      debug(s"Searching offset for timestamp $targetTimestamp")
 
-    if (config.messageFormatVersion < KAFKA_0_10_0_IV0 &&
+      if (config.messageFormatVersion < KAFKA_0_10_0_IV0 &&
         targetTimestamp != ListOffsetRequest.EARLIEST_TIMESTAMP &&
         targetTimestamp != ListOffsetRequest.LATEST_TIMESTAMP)
-      throw new UnsupportedForMessageFormatException(s"Cannot search offsets based on timestamp because message format version " +
+        throw new UnsupportedForMessageFormatException(s"Cannot search offsets based on timestamp because message format version " +
           s"for partition $topicPartition is ${config.messageFormatVersion} which is earlier than the minimum " +
           s"required version $KAFKA_0_10_0_IV0")
 
-    // Cache to avoid race conditions. `toBuffer` is faster than most alternatives and provides
-    // constant time access while being safe to use with concurrent collections unlike `toArray`.
-    val segmentsCopy = logSegments.toBuffer
-    // For the earliest and latest, we do not need to return the timestamp.
-    if (targetTimestamp == ListOffsetRequest.EARLIEST_TIMESTAMP)
+      // Cache to avoid race conditions. `toBuffer` is faster than most alternatives and provides
+      // constant time access while being safe to use with concurrent collections unlike `toArray`.
+      val segmentsCopy = logSegments.toBuffer
+      // For the earliest and latest, we do not need to return the timestamp.
+      if (targetTimestamp == ListOffsetRequest.EARLIEST_TIMESTAMP)
         return Some(TimestampOffset(RecordBatch.NO_TIMESTAMP, logStartOffset))
-    else if (targetTimestamp == ListOffsetRequest.LATEST_TIMESTAMP)
+      else if (targetTimestamp == ListOffsetRequest.LATEST_TIMESTAMP)
         return Some(TimestampOffset(RecordBatch.NO_TIMESTAMP, logEndOffset))
 
-    val targetSeg = {
-      // Get all the segments whose largest timestamp is smaller than target timestamp
-      val earlierSegs = segmentsCopy.takeWhile(_.largestTimestamp < targetTimestamp)
-      // We need to search the first segment whose largest timestamp is greater than the target timestamp if there is one.
-      if (earlierSegs.length < segmentsCopy.length)
-        Some(segmentsCopy(earlierSegs.length))
-      else
-        None
-    }
+      val targetSeg = {
+        // Get all the segments whose largest timestamp is smaller than target timestamp
+        val earlierSegs = segmentsCopy.takeWhile(_.largestTimestamp < targetTimestamp)
+        // We need to search the first segment whose largest timestamp is greater than the target timestamp if there is one.
+        if (earlierSegs.length < segmentsCopy.length)
+          Some(segmentsCopy(earlierSegs.length))
+        else
+          None
+      }
 
-    targetSeg.flatMap(_.findOffsetByTimestamp(targetTimestamp, logStartOffset))
+      targetSeg.flatMap(_.findOffsetByTimestamp(targetTimestamp, logStartOffset))
+    }
   }
 
   /**
@@ -1070,18 +1089,20 @@ class Log(@volatile var dir: File,
   }
 
   private def deleteSegments(deletable: Iterable[LogSegment]): Int = {
-    val numToDelete = deletable.size
-    if (numToDelete > 0) {
-      // we must always have at least one segment, so if we are going to delete all the segments, create a new one first
-      if (segments.size == numToDelete)
-        roll()
-      lock synchronized {
-        // remove the segments for lookups
-        deletable.foreach(deleteSegment)
-        maybeIncrementLogStartOffset(segments.firstEntry.getValue.baseOffset)
+    maybeHandleIOException(s"Error while deleting segments for $topicPartition in dir ${dir.getParent}") {
+      val numToDelete = deletable.size
+      if (numToDelete > 0) {
+        // we must always have at least one segment, so if we are going to delete all the segments, create a new one first
+        if (segments.size == numToDelete)
+          roll()
+        lock synchronized {
+          // remove the segments for lookups
+          deletable.foreach(deleteSegment)
+          maybeIncrementLogStartOffset(segments.firstEntry.getValue.baseOffset)
+        }
       }
+      numToDelete
     }
-    numToDelete
   }
 
   /**
@@ -1204,58 +1225,60 @@ class Log(@volatile var dir: File,
    * @return The newly rolled segment
    */
   def roll(expectedNextOffset: Long = 0): LogSegment = {
-    val start = time.nanoseconds
-    lock synchronized {
-      val newOffset = math.max(expectedNextOffset, logEndOffset)
-      val logFile = Log.logFile(dir, newOffset)
-      val offsetIdxFile = offsetIndexFile(dir, newOffset)
-      val timeIdxFile = timeIndexFile(dir, newOffset)
-      val txnIdxFile = transactionIndexFile(dir, newOffset)
-      for(file <- List(logFile, offsetIdxFile, timeIdxFile, txnIdxFile) if file.exists) {
-        warn("Newly rolled segment file " + file.getName + " already exists; deleting it first")
-        file.delete()
-      }
-
-      segments.lastEntry() match {
-        case null =>
-        case entry => {
-          val seg = entry.getValue
-          seg.onBecomeInactiveSegment()
-          seg.index.trimToValidSize()
-          seg.timeIndex.trimToValidSize()
-          seg.log.trim()
+    maybeHandleIOException(s"Error while rolling log segment for $topicPartition in dir ${dir.getParent}") {
+      val start = time.nanoseconds
+      lock synchronized {
+        val newOffset = math.max(expectedNextOffset, logEndOffset)
+        val logFile = Log.logFile(dir, newOffset)
+        val offsetIdxFile = offsetIndexFile(dir, newOffset)
+        val timeIdxFile = timeIndexFile(dir, newOffset)
+        val txnIdxFile = transactionIndexFile(dir, newOffset)
+        for (file <- List(logFile, offsetIdxFile, timeIdxFile, txnIdxFile) if file.exists) {
+          warn("Newly rolled segment file " + file.getName + " already exists; deleting it first")
+          file.delete()
         }
-      }
 
-      // take a snapshot of the producer state to facilitate recovery. It is useful to have the snapshot
-      // offset align with the new segment offset since this ensures we can recover the segment by beginning
-      // with the corresponding snapshot file and scanning the segment data. Because the segment base offset
-      // may actually be ahead of the current producer state end offset (which corresponds to the log end offset),
-      // we manually override the state offset here prior to taking the snapshot.
-      producerStateManager.updateMapEndOffset(newOffset)
-      producerStateManager.takeSnapshot()
+        segments.lastEntry() match {
+          case null =>
+          case entry => {
+            val seg = entry.getValue
+            seg.onBecomeInactiveSegment()
+            seg.index.trimToValidSize()
+            seg.timeIndex.trimToValidSize()
+            seg.log.trim()
+          }
+        }
 
-      val segment = new LogSegment(dir,
-                                   startOffset = newOffset,
-                                   indexIntervalBytes = config.indexInterval,
-                                   maxIndexSize = config.maxIndexSize,
-                                   rollJitterMs = config.randomSegmentJitter,
-                                   time = time,
-                                   fileAlreadyExists = false,
-                                   initFileSize = initFileSize,
-                                   preallocate = config.preallocate)
-      val prev = addSegment(segment)
-      if(prev != null)
-        throw new KafkaException("Trying to roll a new log segment for topic partition %s with start offset %d while it already exists.".format(name, newOffset))
-      // We need to update the segment base offset and append position data of the metadata when log rolls.
-      // The next offset should not change.
-      updateLogEndOffset(nextOffsetMetadata.messageOffset)
-      // schedule an asynchronous flush of the old segment
-      scheduler.schedule("flush-log", () => flush(newOffset), delay = 0L)
-
-      info("Rolled new log segment for '" + name + "' in %.0f ms.".format((System.nanoTime - start) / (1000.0*1000.0)))
+        // take a snapshot of the producer state to facilitate recovery. It is useful to have the snapshot
+        // offset align with the new segment offset since this ensures we can recover the segment by beginning
+        // with the corresponding snapshot file and scanning the segment data. Because the segment base offset
+        // may actually be ahead of the current producer state end offset (which corresponds to the log end offset),
+        // we manually override the state offset here prior to taking the snapshot.
+        producerStateManager.updateMapEndOffset(newOffset)
+        producerStateManager.takeSnapshot()
 
-      segment
+        val segment = new LogSegment(dir,
+          startOffset = newOffset,
+          indexIntervalBytes = config.indexInterval,
+          maxIndexSize = config.maxIndexSize,
+          rollJitterMs = config.randomSegmentJitter,
+          time = time,
+          fileAlreadyExists = false,
+          initFileSize = initFileSize,
+          preallocate = config.preallocate)
+        val prev = addSegment(segment)
+        if (prev != null)
+          throw new KafkaException("Trying to roll a new log segment for topic partition %s with start offset %d while it already exists.".format(name, newOffset))
+        // We need to update the segment base offset and append position data of the metadata when log rolls.
+        // The next offset should not change.
+        updateLogEndOffset(nextOffsetMetadata.messageOffset)
+        // schedule an asynchronous flush of the old segment
+        scheduler.schedule("flush-log", () => flush(newOffset), delay = 0L)
+
+        info("Rolled new log segment for '" + name + "' in %.0f ms.".format((System.nanoTime - start) / (1000.0 * 1000.0)))
+
+        segment
+      }
     }
   }
 
@@ -1275,22 +1298,24 @@ class Log(@volatile var dir: File,
    * @param offset The offset to flush up to (non-inclusive); the new recovery point
    */
   def flush(offset: Long) : Unit = {
-    if (offset <= this.recoveryPoint)
-      return
-    debug("Flushing log '" + name + " up to offset " + offset + ", last flushed: " + lastFlushTime + " current time: " +
-          time.milliseconds + " unflushed = " + unflushedMessages)
-    for(segment <- logSegments(this.recoveryPoint, offset))
-      segment.flush()
+    maybeHandleIOException(s"Error while flushing log for $topicPartition in dir ${dir.getParent} with offset $offset") {
+      if (offset <= this.recoveryPoint)
+        return
+      debug("Flushing log '" + name + " up to offset " + offset + ", last flushed: " + lastFlushTime + " current time: " +
+        time.milliseconds + " unflushed = " + unflushedMessages)
+      for (segment <- logSegments(this.recoveryPoint, offset))
+        segment.flush()
 
-    // now that we have flushed, we can cleanup old producer snapshots. However, it is useful to retain
-    // the snapshots from the recent segments in case we need to truncate and rebuild the producer state.
-    // Otherwise, we would always need to rebuild from the earliest segment.
-    producerStateManager.deleteSnapshotsBefore(minSnapshotOffsetToRetain(offset))
+      // now that we have flushed, we can cleanup old producer snapshots. However, it is useful to retain
+      // the snapshots from the recent segments in case we need to truncate and rebuild the producer state.
+      // Otherwise, we would always need to rebuild from the earliest segment.
+      producerStateManager.deleteSnapshotsBefore(minSnapshotOffsetToRetain(offset))
 
-    lock synchronized {
-      if(offset > this.recoveryPoint) {
-        this.recoveryPoint = offset
-        lastflushedTime.set(time.milliseconds)
+      lock synchronized {
+        if (offset > this.recoveryPoint) {
+          this.recoveryPoint = offset
+          lastflushedTime.set(time.milliseconds)
+        }
       }
     }
   }
@@ -1310,11 +1335,13 @@ class Log(@volatile var dir: File,
    * Completely delete this log directory and all contents from the file system with no delay
    */
   private[log] def delete() {
-    lock synchronized {
-      logSegments.foreach(_.delete())
-      segments.clear()
-      leaderEpochCache.clear()
-      Utils.delete(dir)
+    maybeHandleIOException(s"Error while deleting log for $topicPartition in dir ${dir.getParent}") {
+      lock synchronized {
+        logSegments.foreach(_.delete())
+        segments.clear()
+        leaderEpochCache.clear()
+        Utils.delete(dir)
+      }
     }
   }
 
@@ -1344,25 +1371,27 @@ class Log(@volatile var dir: File,
    * @param targetOffset The offset to truncate to, an upper bound on all offsets in the log after truncation is complete.
    */
   private[log] def truncateTo(targetOffset: Long) {
-    if(targetOffset < 0)
-      throw new IllegalArgumentException("Cannot truncate to a negative offset (%d).".format(targetOffset))
-    if(targetOffset >= logEndOffset) {
-      info("Truncating %s to %d has no effect as the largest offset in the log is %d.".format(name, targetOffset, logEndOffset-1))
-      return
-    }
-    info("Truncating log %s to offset %d.".format(name, targetOffset))
-    lock synchronized {
-      if(segments.firstEntry.getValue.baseOffset > targetOffset) {
-        truncateFullyAndStartAt(targetOffset)
-      } else {
-        val deletable = logSegments.filter(segment => segment.baseOffset > targetOffset)
-        deletable.foreach(deleteSegment)
-        activeSegment.truncateTo(targetOffset)
-        updateLogEndOffset(targetOffset)
-        this.recoveryPoint = math.min(targetOffset, this.recoveryPoint)
-        this.logStartOffset = math.min(targetOffset, this.logStartOffset)
-        leaderEpochCache.clearAndFlushLatest(targetOffset)
-        loadProducerState(targetOffset, reloadFromCleanShutdown = false)
+    maybeHandleIOException(s"Error while truncating log to offset $targetOffset for $topicPartition in dir ${dir.getParent}") {
+      if (targetOffset < 0)
+        throw new IllegalArgumentException("Cannot truncate to a negative offset (%d).".format(targetOffset))
+      if (targetOffset >= logEndOffset) {
+        info("Truncating %s to %d has no effect as the largest offset in the log is %d.".format(name, targetOffset, logEndOffset - 1))
+        return
+      }
+      info("Truncating log %s to offset %d.".format(name, targetOffset))
+      lock synchronized {
+        if (segments.firstEntry.getValue.baseOffset > targetOffset) {
+          truncateFullyAndStartAt(targetOffset)
+        } else {
+          val deletable = logSegments.filter(segment => segment.baseOffset > targetOffset)
+          deletable.foreach(deleteSegment)
+          activeSegment.truncateTo(targetOffset)
+          updateLogEndOffset(targetOffset)
+          this.recoveryPoint = math.min(targetOffset, this.recoveryPoint)
+          this.logStartOffset = math.min(targetOffset, this.logStartOffset)
+          leaderEpochCache.clearAndFlushLatest(targetOffset)
+          loadProducerState(targetOffset, reloadFromCleanShutdown = false)
+        }
       }
     }
   }
@@ -1373,28 +1402,30 @@ class Log(@volatile var dir: File,
    *  @param newOffset The new offset to start the log with
    */
   private[log] def truncateFullyAndStartAt(newOffset: Long) {
-    debug(s"Truncate and start log '$name' at offset $newOffset")
-    lock synchronized {
-      val segmentsToDelete = logSegments.toList
-      segmentsToDelete.foreach(deleteSegment)
-      addSegment(new LogSegment(dir,
-                                newOffset,
-                                indexIntervalBytes = config.indexInterval,
-                                maxIndexSize = config.maxIndexSize,
-                                rollJitterMs = config.randomSegmentJitter,
-                                time = time,
-                                fileAlreadyExists = false,
-                                initFileSize = initFileSize,
-                                preallocate = config.preallocate))
-      updateLogEndOffset(newOffset)
-      leaderEpochCache.clearAndFlush()
-
-      producerStateManager.truncate()
-      producerStateManager.updateMapEndOffset(newOffset)
-      updateFirstUnstableOffset()
+    maybeHandleIOException(s"Error while truncating the entire log for $topicPartition in dir ${dir.getParent}") {
+      debug(s"Truncate and start log '$name' at offset $newOffset")
+      lock synchronized {
+        val segmentsToDelete = logSegments.toList
+        segmentsToDelete.foreach(deleteSegment)
+        addSegment(new LogSegment(dir,
+          newOffset,
+          indexIntervalBytes = config.indexInterval,
+          maxIndexSize = config.maxIndexSize,
+          rollJitterMs = config.randomSegmentJitter,
+          time = time,
+          fileAlreadyExists = false,
+          initFileSize = initFileSize,
+          preallocate = config.preallocate))
+        updateLogEndOffset(newOffset)
+        leaderEpochCache.clearAndFlush()
+
+        producerStateManager.truncate()
+        producerStateManager.updateMapEndOffset(newOffset)
+        updateFirstUnstableOffset()
 
-      this.recoveryPoint = math.min(newOffset, this.recoveryPoint)
-      this.logStartOffset = newOffset
+        this.recoveryPoint = math.min(newOffset, this.recoveryPoint)
+        this.logStartOffset = newOffset
+      }
     }
   }
 
@@ -1439,6 +1470,9 @@ class Log(@volatile var dir: File,
    * This allows reads to happen concurrently without synchronization and without the possibility of physically
    * deleting a file while it is being read from.
    *
+   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded
+   * or the immediate caller will catch and handle IOException
+   *
    * @param segment The log segment to schedule for deletion
    */
   private def deleteSegment(segment: LogSegment) {
@@ -1452,13 +1486,18 @@ class Log(@volatile var dir: File,
   /**
    * Perform an asynchronous delete on the given file if it exists (otherwise do nothing)
    *
-   * @throws KafkaStorageException if the file can't be renamed and still exists
+   * This method does not need to convert IOException (thrown from changeFileSuffixes) to KafkaStorageException because
+   * it is either called before all logs are loaded or the caller will catch and handle IOException
+   *
+   * @throws IOException if the file can't be renamed and still exists
    */
   private def asyncDeleteSegment(segment: LogSegment) {
     segment.changeFileSuffixes("", Log.DeletedFileSuffix)
     def deleteSeg() {
       info("Deleting segment %d from log %s.".format(segment.baseOffset, name))
-      segment.delete()
+      maybeHandleIOException(s"Error while deleting segments for $topicPartition in dir ${dir.getParent}") {
+        segment.delete()
+      }
     }
     scheduler.schedule("delete-file", deleteSeg _, delay = config.fileDeleteDelayMs)
   }
@@ -1467,6 +1506,9 @@ class Log(@volatile var dir: File,
    * Swap a new segment in place and delete one or more existing segments in a crash-safe manner. The old segments will
    * be asynchronously deleted.
    *
+   * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded
+   * or the caller will catch and handle IOException
+   *
    * The sequence of operations is:
    * <ol>
    *   <li> Cleaner creates new segment with suffix .cleaned and invokes replaceSegments().
@@ -1524,6 +1566,16 @@ class Log(@volatile var dir: File,
    */
   def addSegment(segment: LogSegment) = this.segments.put(segment.baseOffset, segment)
 
+  private def maybeHandleIOException[T](msg: => String)(fun: => T): T = {
+    try {
+      fun
+    } catch {
+      case e: IOException =>
+        logDirFailureChannel.maybeAddLogFailureEvent(dir.getParent)
+        throw new KafkaStorageException(msg, e)
+    }
+  }
+
 }
 
 /**
@@ -1574,11 +1626,12 @@ object Log {
             brokerTopicStats: BrokerTopicStats,
             time: Time = Time.SYSTEM,
             maxProducerIdExpirationMs: Int = 60 * 60 * 1000,
-            producerIdExpirationCheckIntervalMs: Int = 10 * 60 * 1000): Log = {
+            producerIdExpirationCheckIntervalMs: Int = 10 * 60 * 1000,
+            logDirFailureChannel: LogDirFailureChannel = null): Log = {
     val topicPartition = Log.parseTopicPartitionName(dir)
     val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs)
     new Log(dir, config, logStartOffset, recoveryPoint, scheduler, brokerTopicStats, time, maxProducerIdExpirationMs,
-      producerIdExpirationCheckIntervalMs, topicPartition, producerStateManager)
+      producerIdExpirationCheckIntervalMs, topicPartition, producerStateManager, logDirFailureChannel)
   }
 
   /**