You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ad...@apache.org on 2017/07/31 20:53:41 UTC

kudu git commit: disk failure: coordinate error handling

Repository: kudu
Updated Branches:
  refs/heads/master 230ed20d5 -> 12efb85e1


disk failure: coordinate error handling

IO errors are spawned when calls to env functions fail. The response to
these errors may vary and may span many layers of abstraction.

For instance, if an EIO were triggered while writing a block, one
response might be to shut down the tablet whose operation just failed.
In order to do this without messing up the layering, a new entity, the
FsErrorManager, is added to coordinate error handling.

Callbacks are registered with the FsErrorManager as responses to errors,
and callers to at-risk functions trigger these callbacks as errors are
spawned.

This patch includes the FsErrorManager and its placement w.r.t. the
BlockManager, FsManager, TSTabletManager, etc. Actual handling and
testing will come in a later patch.

This is a part of a series of patches to handle disk failure. See
section 2.2 of:
https://docs.google.com/document/d/1zZk-vb_ETKUuePcZ9ZqoSK2oPvAAaEV1sjDXes8Pxgk/edit?usp=sharing

Change-Id: Ib3169deada702c527b70fabf9d2223364f9a9ea6
Reviewed-on: http://gerrit.cloudera.org:8080/7029
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Adar Dembo <ad...@cloudera.com>


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

Branch: refs/heads/master
Commit: 12efb85e13391c0230105a47ca949959b2e7639f
Parents: 230ed20
Author: Andrew Wong <aw...@cloudera.com>
Authored: Fri May 26 19:55:34 2017 -0700
Committer: Adar Dembo <ad...@cloudera.com>
Committed: Mon Jul 31 20:52:35 2017 +0000

----------------------------------------------------------------------
 src/kudu/fs/block_manager-stress-test.cc | 11 +++-
 src/kudu/fs/block_manager-test.cc        |  5 +-
 src/kudu/fs/data_dirs.cc                 |  4 ++
 src/kudu/fs/data_dirs.h                  |  3 +
 src/kudu/fs/error_manager.h              | 80 +++++++++++++++++++++++++++
 src/kudu/fs/file_block_manager.cc        |  6 +-
 src/kudu/fs/file_block_manager.h         | 17 +++---
 src/kudu/fs/fs_manager.cc                | 16 +++++-
 src/kudu/fs/fs_manager.h                 | 24 ++++++--
 src/kudu/fs/log_block_manager-test.cc    |  7 ++-
 src/kudu/fs/log_block_manager.cc         |  5 +-
 src/kudu/fs/log_block_manager.h          |  9 ++-
 src/kudu/tserver/tablet_server.cc        |  4 ++
 src/kudu/tserver/ts_tablet_manager.cc    | 12 +++-
 src/kudu/tserver/ts_tablet_manager.h     |  9 ++-
 15 files changed, 184 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/12efb85e/src/kudu/fs/block_manager-stress-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/block_manager-stress-test.cc b/src/kudu/fs/block_manager-stress-test.cc
index ac23296..876602d 100644
--- a/src/kudu/fs/block_manager-stress-test.cc
+++ b/src/kudu/fs/block_manager-stress-test.cc
@@ -22,11 +22,12 @@
 #include <unordered_map>
 #include <vector>
 
-#include "kudu/fs/fs.pb.h"
+#include "kudu/fs/error_manager.h"
 #include "kudu/fs/file_block_manager.h"
+#include "kudu/fs/fs.pb.h"
 #include "kudu/fs/fs_report.h"
-#include "kudu/fs/log_block_manager.h"
 #include "kudu/fs/log_block_manager-test-util.h"
+#include "kudu/fs/log_block_manager.h"
 #include "kudu/gutil/stl_util.h"
 #include "kudu/gutil/strings/split.h"
 #include "kudu/gutil/strings/substitute.h"
@@ -96,6 +97,7 @@ class BlockManagerStressTest : public KuduTest {
   BlockManagerStressTest() :
     rand_seed_(SeedRandom()),
     stop_latch_(1),
+    test_error_manager_(new FsErrorManager()),
     test_tablet_name_("test_tablet"),
     total_blocks_written_(0),
     total_bytes_written_(0),
@@ -152,7 +154,7 @@ class BlockManagerStressTest : public KuduTest {
   BlockManager* CreateBlockManager() {
     BlockManagerOptions opts;
     opts.root_paths = data_dirs_;
-    return new T(env_, opts);
+    return new T(env_, test_error_manager_.get(), opts);
   }
 
   void RunTest(double secs) {
@@ -235,6 +237,9 @@ class BlockManagerStressTest : public KuduTest {
   // The block manager.
   gscoped_ptr<BlockManager> bm_;
 
+  // The error manager.
+  unique_ptr<FsErrorManager> test_error_manager_;
+
   // Test group of disk to spread data across.
   DataDirGroupPB test_group_pb_;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/12efb85e/src/kudu/fs/block_manager-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/block_manager-test.cc b/src/kudu/fs/block_manager-test.cc
index 78336d1..72068cb 100644
--- a/src/kudu/fs/block_manager-test.cc
+++ b/src/kudu/fs/block_manager-test.cc
@@ -23,6 +23,7 @@
 #include <google/protobuf/util/message_differencer.h>
 
 #include "kudu/fs/data_dirs.h"
+#include "kudu/fs/error_manager.h"
 #include "kudu/fs/file_block_manager.h"
 #include "kudu/fs/fs.pb.h"
 #include "kudu/fs/fs_report.h"
@@ -90,6 +91,7 @@ class BlockManagerTest : public KuduTest {
   BlockManagerTest() :
     test_tablet_name_("test_tablet"),
     test_block_opts_(CreateBlockOptions({ test_tablet_name_ })),
+    test_error_manager_(new FsErrorManager()),
     bm_(CreateBlockManager(scoped_refptr<MetricEntity>(),
                            shared_ptr<MemTracker>(),
                            { test_dir_ })) {
@@ -132,7 +134,7 @@ class BlockManagerTest : public KuduTest {
     opts.metric_entity = metric_entity;
     opts.parent_mem_tracker = parent_mem_tracker;
     opts.root_paths = paths;
-    return new T(env_, opts);
+    return new T(env_, test_error_manager_.get(), opts);
   }
 
   Status ReopenBlockManager(const scoped_refptr<MetricEntity>& metric_entity,
@@ -185,6 +187,7 @@ class BlockManagerTest : public KuduTest {
   DataDirGroupPB test_group_pb_;
   string test_tablet_name_;
   CreateBlockOptions test_block_opts_;
+  unique_ptr<FsErrorManager> test_error_manager_;
   gscoped_ptr<T> bm_;
 };
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/12efb85e/src/kudu/fs/data_dirs.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/data_dirs.cc b/src/kudu/fs/data_dirs.cc
index 2ec3656..5596b22 100644
--- a/src/kudu/fs/data_dirs.cc
+++ b/src/kudu/fs/data_dirs.cc
@@ -619,6 +619,10 @@ bool DataDirManager::FindUuidIndexByDataDir(DataDir* dir, uint16_t* uuid_idx) co
   return FindCopy(uuid_idx_by_data_dir_, dir, uuid_idx);
 }
 
+bool DataDirManager::FindUuidIndexByUuid(const string& uuid, uint16_t* uuid_idx) const {
+  return FindCopy(idx_by_uuid_, uuid, uuid_idx);
+}
+
 set<string> DataDirManager::FindTabletsByDataDirUuidIdx(uint16_t uuid_idx) {
   DCHECK_LT(uuid_idx, data_dirs_.size());
   shared_lock<rw_spinlock> lock(dir_group_lock_.get_lock());

http://git-wip-us.apache.org/repos/asf/kudu/blob/12efb85e/src/kudu/fs/data_dirs.h
----------------------------------------------------------------------
diff --git a/src/kudu/fs/data_dirs.h b/src/kudu/fs/data_dirs.h
index 7e9af12..817f1a0 100644
--- a/src/kudu/fs/data_dirs.h
+++ b/src/kudu/fs/data_dirs.h
@@ -271,6 +271,9 @@ class DataDirManager {
   bool FindUuidIndexByDataDir(DataDir* dir,
                               uint16_t* uuid_idx) const;
 
+  // Finds a uuid index by UUID, returning false if it can't be found.
+  bool FindUuidIndexByUuid(const std::string& uuid, uint16_t* uuid_idx) const;
+
   // Returns a list of all data dirs.
   const std::vector<std::unique_ptr<DataDir>>& data_dirs() const {
     return data_dirs_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/12efb85e/src/kudu/fs/error_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/fs/error_manager.h b/src/kudu/fs/error_manager.h
new file mode 100644
index 0000000..fff702e
--- /dev/null
+++ b/src/kudu/fs/error_manager.h
@@ -0,0 +1,80 @@
+// 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.
+
+#pragma once
+
+#include <map>
+#include <string>
+
+#include "kudu/fs/block_manager_util.h"
+#include "kudu/fs/data_dirs.h"
+#include "kudu/gutil/callback_forward.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/fault_injection.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+namespace fs {
+
+// Callback to error-handling code. The input string is the UUID a failed
+// component.
+//
+// e.g. the ErrorNotificationCb for disk failure handling takes the UUID of a
+// directory, marks it failed, and shuts down the tablets in that directory.
+typedef Callback<void(const std::string&)> ErrorNotificationCb;
+static void DoNothingErrorNotification(const std::string& /* uuid */) {}
+
+// When certain operations fail, the side effects of the error can span
+// multiple layers, many of which we prefer to keep separate. The FsErrorManager
+// registers and runs error handlers without adding cross-layer dependencies.
+//
+// e.g. the TSTabletManager registers a callback to handle disk failure.
+// Blocks and other entities that may hit disk failures can call it without
+// knowing about the TSTabletManager.
+class FsErrorManager {
+ public:
+  FsErrorManager()
+    : notify_cb_(Bind(DoNothingErrorNotification)) {}
+
+  // Sets the error notification callback.
+  //
+  // This should be called when the callback's callee is initialized.
+  void SetErrorNotificationCb(ErrorNotificationCb cb) {
+    notify_cb_ = std::move(cb);
+  }
+
+  // Resets the error notification callback.
+  //
+  // This must be called before the callback's callee is destroyed.
+  void UnsetErrorNotificationCb() {
+    notify_cb_ = Bind(DoNothingErrorNotification);
+  }
+
+  // Runs the error notification callback.
+  //
+  // 'uuid' is the full UUID of the component that failed.
+  void RunErrorNotificationCb(const std::string& uuid) const {
+    notify_cb_.Run(uuid);
+  }
+
+ private:
+   // Callback to be run when an error occurs.
+   ErrorNotificationCb notify_cb_;
+};
+
+}  // namespace fs
+}  // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/12efb85e/src/kudu/fs/file_block_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/file_block_manager.cc b/src/kudu/fs/file_block_manager.cc
index e7a14f2..20f7cb4 100644
--- a/src/kudu/fs/file_block_manager.cc
+++ b/src/kudu/fs/file_block_manager.cc
@@ -24,6 +24,7 @@
 
 #include "kudu/fs/block_manager_metrics.h"
 #include "kudu/fs/data_dirs.h"
+#include "kudu/fs/error_manager.h"
 #include "kudu/fs/fs_report.h"
 #include "kudu/gutil/strings/numbers.h"
 #include "kudu/gutil/strings/substitute.h"
@@ -528,10 +529,13 @@ bool FileBlockManager::FindBlockPath(const BlockId& block_id,
   return dir != nullptr;
 }
 
-FileBlockManager::FileBlockManager(Env* env, const BlockManagerOptions& opts)
+FileBlockManager::FileBlockManager(Env* env,
+                                   FsErrorManager* error_manager,
+                                   const BlockManagerOptions& opts)
   : env_(DCHECK_NOTNULL(env)),
     read_only_(opts.read_only),
     dd_manager_(env, opts.metric_entity, kBlockManagerType, opts.root_paths),
+    error_manager_(DCHECK_NOTNULL(error_manager)),
     file_cache_("fbm", env_, GetFileCacheCapacityForBlockManager(env_),
                 opts.metric_entity),
     rand_(GetRandomSeed32()),

http://git-wip-us.apache.org/repos/asf/kudu/blob/12efb85e/src/kudu/fs/file_block_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/fs/file_block_manager.h b/src/kudu/fs/file_block_manager.h
index 1a83ffa..ac5c735 100644
--- a/src/kudu/fs/file_block_manager.h
+++ b/src/kudu/fs/file_block_manager.h
@@ -40,6 +40,7 @@ class RandomAccessFile;
 class WritableFile;
 
 namespace fs {
+class FsErrorManager;
 struct FsReport;
 
 namespace internal {
@@ -69,10 +70,9 @@ struct BlockManagerMetrics;
 class FileBlockManager : public BlockManager {
  public:
 
-  // Creates a new in-memory instance of a FileBlockManager.
-  //
-  // 'env' should remain alive for the lifetime of the block manager.
-  FileBlockManager(Env* env, const BlockManagerOptions& opts);
+  // Note: 'env' and 'error_manager' should remain alive for the lifetime of
+  // the block manager.
+  FileBlockManager(Env* env, FsErrorManager* error_manager, const BlockManagerOptions& opts);
 
   virtual ~FileBlockManager();
 
@@ -92,15 +92,15 @@ class FileBlockManager : public BlockManager {
 
   Status GetAllBlockIds(std::vector<BlockId>* block_ids) override;
 
-  DataDirManager* dd_manager() override { return &dd_manager_; };
+  DataDirManager* dd_manager() override { return &dd_manager_; }
 
  private:
   friend class internal::FileBlockLocation;
   friend class internal::FileReadableBlock;
   friend class internal::FileWritableBlock;
 
-  // Synchronizes the metadata for a block with the given id.
-  Status SyncMetadata(const internal::FileBlockLocation& block_id);
+  // Synchronizes the metadata for a block with the given location.
+  Status SyncMetadata(const internal::FileBlockLocation& location);
 
   // Looks up the path of the file backing a particular block ID.
   //
@@ -118,6 +118,9 @@ class FileBlockManager : public BlockManager {
   // Manages and owns all of the block manager's data directories.
   DataDirManager dd_manager_;
 
+  // Manages callbacks used to handle disk failure.
+  FsErrorManager* error_manager_;
+
   // Manages files opened for reading.
   FileCache<RandomAccessFile> file_cache_;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/12efb85e/src/kudu/fs/fs_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/fs_manager.cc b/src/kudu/fs/fs_manager.cc
index f8f0b85..0c43706 100644
--- a/src/kudu/fs/fs_manager.cc
+++ b/src/kudu/fs/fs_manager.cc
@@ -29,6 +29,7 @@
 #include <google/protobuf/message.h>
 
 #include "kudu/fs/block_id.h"
+#include "kudu/fs/error_manager.h"
 #include "kudu/fs/file_block_manager.h"
 #include "kudu/fs/fs.pb.h"
 #include "kudu/fs/log_block_manager.h"
@@ -82,6 +83,7 @@ using kudu::env_util::ScopedFileDeleter;
 using kudu::fs::BlockManagerOptions;
 using kudu::fs::CreateBlockOptions;
 using kudu::fs::DataDirManager;
+using kudu::fs::FsErrorManager;
 using kudu::fs::FileBlockManager;
 using kudu::fs::FsReport;
 using kudu::fs::LogBlockManager;
@@ -124,6 +126,7 @@ FsManager::FsManager(Env* env, const string& root_path)
     wal_fs_root_(root_path),
     data_fs_roots_({ root_path }),
     metric_entity_(nullptr),
+    error_manager_(new FsErrorManager()),
     initted_(false) {
 }
 
@@ -135,12 +138,21 @@ FsManager::FsManager(Env* env,
     data_fs_roots_(opts.data_paths),
     metric_entity_(opts.metric_entity),
     parent_mem_tracker_(opts.parent_mem_tracker),
+    error_manager_(new FsErrorManager()),
     initted_(false) {
 }
 
 FsManager::~FsManager() {
 }
 
+void FsManager::SetErrorNotificationCb(fs::ErrorNotificationCb cb) {
+  error_manager_->SetErrorNotificationCb(std::move(cb));
+}
+
+void FsManager::UnsetErrorNotificationCb() {
+  error_manager_->UnsetErrorNotificationCb();
+}
+
 Status FsManager::Init() {
   if (initted_) {
     return Status::OK();
@@ -225,9 +237,9 @@ void FsManager::InitBlockManager() {
   opts.root_paths = GetDataRootDirs();
   opts.read_only = read_only_;
   if (FLAGS_block_manager == "file") {
-    block_manager_.reset(new FileBlockManager(env_, opts));
+    block_manager_.reset(new FileBlockManager(env_, error_manager_.get(), opts));
   } else if (FLAGS_block_manager == "log") {
-    block_manager_.reset(new LogBlockManager(env_, opts));
+    block_manager_.reset(new LogBlockManager(env_, error_manager_.get(), opts));
   } else {
     LOG(FATAL) << "Invalid block manager: " << FLAGS_block_manager;
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/12efb85e/src/kudu/fs/fs_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/fs/fs_manager.h b/src/kudu/fs/fs_manager.h
index 47992b3..386e4c5 100644
--- a/src/kudu/fs/fs_manager.h
+++ b/src/kudu/fs/fs_manager.h
@@ -27,6 +27,7 @@
 #include <string>
 #include <vector>
 
+#include "kudu/fs/error_manager.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/util/env.h"
 #include "kudu/util/path_util.h"
@@ -39,10 +40,13 @@ class Message;
 
 namespace kudu {
 
+class BlockId;
+class InstanceMetadataPB;
 class MemTracker;
 class MetricEntity;
 
 namespace fs {
+
 class BlockManager;
 class DataDirManager;
 class ReadableBlock;
@@ -54,10 +58,7 @@ struct CreateBlockOptions;
 
 namespace itest {
 class ExternalMiniClusterFsInspector;
-}
-
-class BlockId;
-class InstanceMetadataPB;
+} // namespace itest
 
 struct FsManagerOpts {
   FsManagerOpts();
@@ -118,6 +119,18 @@ class FsManager {
   // on-disk structures.
   Status Open(fs::FsReport* report = nullptr);
 
+  // Registers an error-handling callback with the FsErrorManager.
+  //
+  // If a disk failure is detected, this callback will be invoked with the
+  // relevant DataDir's UUID as its input parameter.
+  void SetErrorNotificationCb(fs::ErrorNotificationCb cb);
+
+  // Unregisters the error-handling callback with the FsErrorManager.
+  //
+  // This must be called before the callback's callee is destroyed. Calls to
+  // this are idempotent and are safe even if a callback has not been set.
+  void UnsetErrorNotificationCb();
+
   // Create the initial filesystem layout. If 'uuid' is provided, uses it as
   // uuid of the filesystem. Otherwise generates one at random.
   //
@@ -269,7 +282,7 @@ class FsManager {
   static const char *kTabletSuperBlockMagicNumber;
   static const char *kConsensusMetadataDirName;
 
-  Env *env_;
+  Env* env_;
 
   // If false, operations that mutate on-disk state are prohibited.
   const bool read_only_;
@@ -296,6 +309,7 @@ class FsManager {
   std::unique_ptr<InstanceMetadataPB> metadata_;
 
   std::unique_ptr<fs::BlockManager> block_manager_;
+  std::unique_ptr<fs::FsErrorManager> error_manager_;
 
   bool initted_;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/12efb85e/src/kudu/fs/log_block_manager-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/log_block_manager-test.cc b/src/kudu/fs/log_block_manager-test.cc
index f3f1cfe..cf0cbb9 100644
--- a/src/kudu/fs/log_block_manager-test.cc
+++ b/src/kudu/fs/log_block_manager-test.cc
@@ -22,10 +22,11 @@
 #include <unordered_set>
 #include <vector>
 
+#include "kudu/fs/error_manager.h"
 #include "kudu/fs/fs.pb.h"
 #include "kudu/fs/fs_report.h"
-#include "kudu/fs/log_block_manager.h"
 #include "kudu/fs/log_block_manager-test-util.h"
+#include "kudu/fs/log_block_manager.h"
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/strings/strip.h"
 #include "kudu/gutil/strings/substitute.h"
@@ -66,6 +67,7 @@ class LogBlockManagerTest : public KuduTest {
   LogBlockManagerTest() :
     test_tablet_name_("test_tablet"),
     test_block_opts_({ test_tablet_name_ }),
+    test_error_manager_(new FsErrorManager()),
     bm_(CreateBlockManager(scoped_refptr<MetricEntity>())) {
   }
 
@@ -84,7 +86,7 @@ class LogBlockManagerTest : public KuduTest {
     BlockManagerOptions opts;
     opts.metric_entity = metric_entity;
     opts.root_paths = { test_dir_ };
-    return new LogBlockManager(env_, opts);
+    return new LogBlockManager(env_, test_error_manager_.get(), opts);
   }
 
   Status ReopenBlockManager(FsReport* report = nullptr) {
@@ -146,6 +148,7 @@ class LogBlockManagerTest : public KuduTest {
   string test_tablet_name_;
   CreateBlockOptions test_block_opts_;
 
+  unique_ptr<FsErrorManager> test_error_manager_;
   unique_ptr<LogBlockManager> bm_;
 
  private:

http://git-wip-us.apache.org/repos/asf/kudu/blob/12efb85e/src/kudu/fs/log_block_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/log_block_manager.cc b/src/kudu/fs/log_block_manager.cc
index 5942e9e..0e9def5 100644
--- a/src/kudu/fs/log_block_manager.cc
+++ b/src/kudu/fs/log_block_manager.cc
@@ -1388,11 +1388,14 @@ const map<int64_t, int64_t> LogBlockManager::kPerFsBlockSizeBlockLimits({
   { 2048, 1353 },
   { 4096, 2721 }});
 
-LogBlockManager::LogBlockManager(Env* env, const BlockManagerOptions& opts)
+LogBlockManager::LogBlockManager(Env* env,
+                                 FsErrorManager* error_manager,
+                                 const BlockManagerOptions& opts)
   : mem_tracker_(MemTracker::CreateTracker(-1,
                                            "log_block_manager",
                                            opts.parent_mem_tracker)),
     dd_manager_(env, opts.metric_entity, kBlockManagerType, opts.root_paths),
+    error_manager_(DCHECK_NOTNULL(error_manager)),
     file_cache_("lbm", env, GetFileCacheCapacityForBlockManager(env),
                 opts.metric_entity),
     blocks_by_block_id_(10,

http://git-wip-us.apache.org/repos/asf/kudu/blob/12efb85e/src/kudu/fs/log_block_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/fs/log_block_manager.h b/src/kudu/fs/log_block_manager.h
index ad6b7cf..d485f99 100644
--- a/src/kudu/fs/log_block_manager.h
+++ b/src/kudu/fs/log_block_manager.h
@@ -47,12 +47,14 @@ class Env;
 class MetricEntity;
 class RWFile;
 class ThreadPool;
+class FsManager;
 
 namespace pb_util {
 class WritablePBContainerFile;
 } // namespace pb_util
 
 namespace fs {
+class FsErrorManager;
 struct FsReport;
 
 namespace internal {
@@ -164,7 +166,9 @@ class LogBlockManager : public BlockManager {
   static const char* kContainerMetadataFileSuffix;
   static const char* kContainerDataFileSuffix;
 
-  LogBlockManager(Env* env, const BlockManagerOptions& opts);
+  // Note: 'env' and 'error_manager' should remain alive for the lifetime of
+  // the block manager.
+  LogBlockManager(Env* env, FsErrorManager* error_manager, const BlockManagerOptions& opts);
 
   virtual ~LogBlockManager();
 
@@ -358,6 +362,9 @@ class LogBlockManager : public BlockManager {
   // Manages and owns all of the block manager's data directories.
   DataDirManager dd_manager_;
 
+  // Manages callbacks used to handle disk failure.
+  FsErrorManager* error_manager_;
+
   // Maps a data directory to an upper bound on the number of blocks that a
   // container residing in that directory should observe, if one is necessary.
   std::unordered_map<const DataDir*,

http://git-wip-us.apache.org/repos/asf/kudu/blob/12efb85e/src/kudu/tserver/tablet_server.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_server.cc b/src/kudu/tserver/tablet_server.cc
index bf98057..818b57f 100644
--- a/src/kudu/tserver/tablet_server.cc
+++ b/src/kudu/tserver/tablet_server.cc
@@ -106,6 +106,9 @@ Status TabletServer::WaitInited() {
 Status TabletServer::Start() {
   CHECK(initted_);
 
+  fs_manager_->SetErrorNotificationCb(Bind(&TSTabletManager::FailTabletsInDataDir,
+                                           Unretained(tablet_manager_.get())));
+
   gscoped_ptr<ServiceIf> ts_service(new TabletServiceImpl(this));
   gscoped_ptr<ServiceIf> admin_service(new TabletServiceAdminImpl(this));
   gscoped_ptr<ServiceIf> consensus_service(new ConsensusServiceImpl(this, tablet_manager_.get()));
@@ -137,6 +140,7 @@ void TabletServer::Shutdown() {
     // 2. Shut down the tserver's subsystems.
     maintenance_manager_->Shutdown();
     WARN_NOT_OK(heartbeater_->Stop(), "Failed to stop TS Heartbeat thread");
+    fs_manager_->UnsetErrorNotificationCb();
     tablet_manager_->Shutdown();
 
     // 3. Shut down generic subsystems.

http://git-wip-us.apache.org/repos/asf/kudu/blob/12efb85e/src/kudu/tserver/ts_tablet_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/ts_tablet_manager.cc b/src/kudu/tserver/ts_tablet_manager.cc
index b564de3..079ad89 100644
--- a/src/kudu/tserver/ts_tablet_manager.cc
+++ b/src/kudu/tserver/ts_tablet_manager.cc
@@ -110,13 +110,12 @@ using consensus::OpId;
 using consensus::RaftConfigPB;
 using consensus::StartTabletCopyRequestPB;
 using consensus::kMinimumTerm;
+using fs::DataDirManager;
 using log::Log;
 using master::ReportedTabletPB;
 using master::TabletReportPB;
-using rpc::ResultTracker;
 using std::shared_ptr;
 using std::string;
-using std::unique_ptr;
 using std::vector;
 using strings::Substitute;
 using tablet::Tablet;
@@ -1067,6 +1066,15 @@ Status TSTabletManager::DeleteTabletData(
   return meta->DeleteSuperBlock();
 }
 
+void TSTabletManager::FailTabletsInDataDir(const string& uuid) {
+  DataDirManager* dd_manager = fs_manager_->dd_manager();
+  uint16_t uuid_idx;
+  CHECK(dd_manager->FindUuidIndexByUuid(uuid, &uuid_idx))
+      << Substitute("No data directory found with UUID $0", uuid);
+  LOG(FATAL) << Substitute("Data directory $0 failed. Disk failure handling not implemented",
+                           dd_manager->FindDataDirByUuidIndex(uuid_idx)->dir());
+}
+
 TransitionInProgressDeleter::TransitionInProgressDeleter(
     TransitionInProgressMap* map, rw_spinlock* lock, string entry)
     : in_progress_(map), lock_(lock), entry_(std::move(entry)) {}

http://git-wip-us.apache.org/repos/asf/kudu/blob/12efb85e/src/kudu/tserver/ts_tablet_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/ts_tablet_manager.h b/src/kudu/tserver/ts_tablet_manager.h
index 3b2ce45..5883771 100644
--- a/src/kudu/tserver/ts_tablet_manager.h
+++ b/src/kudu/tserver/ts_tablet_manager.h
@@ -14,8 +14,8 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-#ifndef KUDU_TSERVER_TS_TABLET_MANAGER_H
-#define KUDU_TSERVER_TS_TABLET_MANAGER_H
+
+#pragma once
 
 #include <boost/optional/optional_fwd.hpp>
 #include <gtest/gtest_prod.h>
@@ -184,6 +184,10 @@ class TSTabletManager : public tserver::TabletReplicaLookupIf {
       const scoped_refptr<consensus::ConsensusMetadataManager>& cmeta_manager,
       tablet::TabletDataState delete_type,
       const boost::optional<consensus::OpId>& last_logged_opid);
+
+  // Forces shutdown of the tablet replicas in the data dir corresponding to 'uuid'.
+  void FailTabletsInDataDir(const std::string& uuid);
+
  private:
   FRIEND_TEST(TsTabletManagerTest, TestPersistBlocks);
 
@@ -338,4 +342,3 @@ class TransitionInProgressDeleter : public RefCountedThreadSafe<TransitionInProg
 
 } // namespace tserver
 } // namespace kudu
-#endif /* KUDU_TSERVER_TS_TABLET_MANAGER_H */