You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by aw...@apache.org on 2019/12/05 08:07:27 UTC

[kudu] branch master updated: KUDU-2993: don't require update_dirs to fix directory inconsistencies

This is an automated email from the ASF dual-hosted git repository.

awong pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
     new 83f65ff  KUDU-2993: don't require update_dirs to fix directory inconsistencies
83f65ff is described below

commit 83f65ffc187638b376a06b7fc3888369764e9333
Author: Andrew Wong <aw...@apache.org>
AuthorDate: Sun Nov 17 23:39:39 2019 -0800

    KUDU-2993: don't require update_dirs to fix directory inconsistencies
    
    This patch removes the ENFORCE_CONSISTENCY behavior when opening the
    DataDirManager. By default, the FS layout will be opened with the new
    UPDATE_AND_IGNORE_FAILURE mode, wherein:
    - We update the PIMFs if we notice any are missing or their metadata is
      not consistent with the actual set of directory UUIDs.
    - We tolerate failures when creating and updating the PIMFs.
    
    This also maintains the previous UPDATE_ON_DISK behavior as
    UPDATE_AND_ERROR_ON_FAILURE, wherein a disk failure during the update
    would halt any further updates and revert any metadata changes thus far.
    This is only used by the 'update_dirs' tool to maintain existing
    behavior.
    
    Since we now rewrite the PIMFs to be consistent by default, the
    "integrity check" is now gone. This check was previously useful to
    ensure that the 'all_uuids' fields matched for every PIMF, which ensured
    that every data directory that was expected to exist actually existed.
    This was important for a couple reasons:
    - When a single missing data directory spelled failure for the entire
      node, starting up with even a single "inconsistent" directory would
      break all tablets on the tserver.
    - The file block manager requires that the UUID indexes used by the
      DataDirManager are static. These indexes are defined by the ordering
      of the UUIDs in the PIMFs, so we used the integrity check to ensure
      the ordering was consistent across PIMFs.
    
    Now that Kudu tablets can start up with missing directories, the first
    reason isn't particularly enticing.
    
    The second is trickier to work around. To work around it, I've kept the
    essence of the UUID indexing for the file block manager, though I've
    made the "integrity checking" virtually non-existent. For the log block
    manager, I've made the UUID indexing much simpler: rather than relying
    on the integrity check, we'll now always assign a PIMF a UUID, even if
    we couldn't read one from disk.
    
    Tests:
    - Updated a few tests that previously enforced consistency among PIMFs
      to instead check for the correct instance-updating behavior.
    - Added a test to check that failures while updating the PIMFs don't
      stop us from opening the FS layout.
    - Added a test that checks that the adding/removing behavior on a
      tserver affects and fails tablets as expected.
    - Added a test to make sure that this doesn't completely break the file
      block manager. Given we don't expect heavy usage of the FBM, I didn't
      do extensive testing when the PIMFs are tampered with.
    - Added a test to ensure we don't regress the rollback behavior of
      the 'update_dirs' tool in the face of a disk failure.
    
    Change-Id: Ic3027e7edb5c60e96ced6160fec1a380b38353a5
    Reviewed-on: http://gerrit.cloudera.org:8080/14760
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
    Tested-by: Kudu Jenkins
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
---
 src/kudu/fs/block_manager-test.cc                  |  67 ++-
 src/kudu/fs/block_manager_util-test.cc             | 117 +----
 src/kudu/fs/block_manager_util.cc                  | 230 +++++----
 src/kudu/fs/block_manager_util.h                   |  49 +-
 src/kudu/fs/data_dirs.cc                           | 539 ++++++++++-----------
 src/kudu/fs/data_dirs.h                            | 100 ++--
 src/kudu/fs/error_manager.h                        |   2 +-
 src/kudu/fs/fs_manager-test.cc                     | 340 +++++++++----
 src/kudu/fs/fs_manager.cc                          |  30 +-
 src/kudu/fs/fs_manager.h                           |  24 +-
 .../integration-tests/open-readonly-fs-itest.cc    |   2 +
 src/kudu/server/server_base.cc                     |   6 +-
 src/kudu/tools/tool_action_fs.cc                   |  18 +-
 src/kudu/tools/tool_action_local_replica.cc        |   1 +
 src/kudu/tserver/tablet_server-test.cc             |  36 +-
 15 files changed, 863 insertions(+), 698 deletions(-)

diff --git a/src/kudu/fs/block_manager-test.cc b/src/kudu/fs/block_manager-test.cc
index ae1194d..7f19903 100644
--- a/src/kudu/fs/block_manager-test.cc
+++ b/src/kudu/fs/block_manager-test.cc
@@ -15,25 +15,27 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "kudu/fs/block_manager.h"
+
+#include <stdlib.h>
+
 #include <algorithm>
 #include <cstddef>
 #include <cstdint>
+#include <initializer_list>
 #include <memory>
 #include <ostream>
-#include <stdlib.h>
 #include <string>
 #include <thread>
 #include <unordered_set>
 #include <vector>
 
 #include <gflags/gflags.h>
-#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 #include <google/protobuf/util/message_differencer.h>
 #include <gtest/gtest.h>
 
 #include "kudu/fs/block_id.h"
-#include "kudu/fs/block_manager.h"
 #include "kudu/fs/data_dirs.h"
 #include "kudu/fs/error_manager.h"
 #include "kudu/fs/file_block_manager.h"
@@ -104,6 +106,15 @@ namespace fs {
 
 static const char* kTestData = "test data";
 
+template<typename T>
+string block_manager_type();
+
+template<>
+string block_manager_type<FileBlockManager>() { return "file"; }
+
+template<>
+string block_manager_type<LogBlockManager>() { return "log"; }
+
 template <typename T>
 class BlockManagerTest : public KuduTest {
  public:
@@ -143,12 +154,15 @@ class BlockManagerTest : public KuduTest {
   }
 
  protected:
+
   T* CreateBlockManager(const scoped_refptr<MetricEntity>& metric_entity,
                         const shared_ptr<MemTracker>& parent_mem_tracker) {
     if (!dd_manager_) {
+      DataDirManagerOptions opts;
+      opts.block_manager_type = block_manager_type<T>();
       // Create a new directory manager if necessary.
       CHECK_OK(DataDirManager::CreateNewForTests(env_, { test_dir_ },
-          DataDirManagerOptions(), &dd_manager_));
+          opts, &dd_manager_));
     }
     BlockManagerOptions opts;
     opts.metric_entity = metric_entity;
@@ -166,6 +180,7 @@ class BlockManagerTest : public KuduTest {
     // manager first to enforce this.
     bm_.reset();
     DataDirManagerOptions opts;
+    opts.block_manager_type = block_manager_type<T>();
     opts.metric_entity = metric_entity;
     if (create) {
       RETURN_NOT_OK(DataDirManager::CreateNewForTests(
@@ -420,6 +435,50 @@ typedef ::testing::Types<FileBlockManager> BlockManagers;
 #endif
 TYPED_TEST_CASE(BlockManagerTest, BlockManagers);
 
+// Test to make sure that we don't break the file block manager, which depends
+// on a static set of directories to function properly. Internally, the
+// DataDirManager of a file block manager must use a specific ordering for its
+// directory UUID indexes which is persisted with the PIMFs.
+TYPED_TEST(BlockManagerTest, TestOpenWithDifferentDirOrder) {
+  const string path1 = this->GetTestPath("path1");
+  const string path2 = this->GetTestPath("path2");
+  vector<string> paths = { path1, path2 };
+  ASSERT_OK(this->env_->CreateDir(path1));
+  ASSERT_OK(this->env_->CreateDir(path2));
+  ASSERT_OK(this->ReopenBlockManager(scoped_refptr<MetricEntity>(),
+                                     shared_ptr<MemTracker>(),
+                                     paths,
+                                     true /* create */,
+                                     false /* load_test_group */));
+
+  const string kTablet = "tablet";
+  CreateBlockOptions opts({ kTablet });
+  FLAGS_fs_target_data_dirs_per_tablet = 2;
+  ASSERT_OK(this->dd_manager_->CreateDataDirGroup(kTablet));
+
+  // Create a block and keep track of its block id.
+  unique_ptr<BlockCreationTransaction> transaction = this->bm_->NewCreationTransaction();
+  unique_ptr<WritableBlock> written_block;
+  ASSERT_OK(this->bm_->CreateBlock(opts, &written_block));
+  const auto block_id = written_block->id();
+  ASSERT_OK(written_block->Append(kTestData));
+  transaction->AddCreatedBlock(std::move(written_block));
+  ASSERT_OK(transaction->CommitCreatedBlocks());
+
+  // Now reopen the block manager with a different ordering for the data
+  // directories.
+  paths = { path2, path1 };
+  ASSERT_OK(this->ReopenBlockManager(scoped_refptr<MetricEntity>(),
+                                     shared_ptr<MemTracker>(),
+                                     paths,
+                                     false /* create */,
+                                     false /* load_test_group */));
+
+  // We should have no trouble reading the block back.
+  unique_ptr<ReadableBlock> read_block;
+  ASSERT_OK(this->bm_->OpenBlock(block_id, &read_block));
+}
+
 // Test the entire lifecycle of a block.
 TYPED_TEST(BlockManagerTest, EndToEndTest) {
   // Create a block.
diff --git a/src/kudu/fs/block_manager_util-test.cc b/src/kudu/fs/block_manager_util-test.cc
index cad2eb7..2f3fbcf 100644
--- a/src/kudu/fs/block_manager_util-test.cc
+++ b/src/kudu/fs/block_manager_util-test.cc
@@ -15,17 +15,17 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "kudu/fs/block_manager_util.h"
+
 #include <memory>
 #include <ostream>
 #include <string>
-#include <utility>
 #include <vector>
 
 #include <glog/logging.h>
 #include <google/protobuf/repeated_field.h> // IWYU pragma: keep
 #include <gtest/gtest.h>
 
-#include "kudu/fs/block_manager_util.h"
 #include "kudu/fs/fs.pb.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/env.h"
@@ -49,16 +49,16 @@ TEST_F(KuduTest, Lifecycle) {
 
   // Test that the metadata file was created.
   {
-    PathInstanceMetadataFile file(env_, kType, kFileName);
-    ASSERT_OK(file.Create(kUuid, { kUuid }));
+    PathInstanceMetadataFile file(env_, kUuid, kType, kFileName);
+    ASSERT_OK(file.Create({ kUuid }));
   }
   ASSERT_TRUE(env_->FileExists(kFileName));
 
   // Test that we could open and parse it.
   {
-    PathInstanceMetadataFile file(env_, kType, kFileName);
+    PathInstanceMetadataFile file(env_, kUuid, kType, kFileName);
     ASSERT_OK(file.LoadFromDisk());
-    PathInstanceMetadataPB* md = file.metadata();
+    const PathInstanceMetadataPB* md = file.metadata();
     ASSERT_EQ(kType, md->block_manager_type());
     const PathSetPB& path_set = md->path_set();
     ASSERT_EQ(kUuid, path_set.uuid());
@@ -68,7 +68,7 @@ TEST_F(KuduTest, Lifecycle) {
 
   // Test that expecting a different type of block manager fails.
   {
-    PathInstanceMetadataFile file(env_, "other type", kFileName);
+    PathInstanceMetadataFile file(env_, kUuid, "other type", kFileName);
     PathInstanceMetadataPB pb;
     ASSERT_TRUE(file.LoadFromDisk().IsIOError());
   }
@@ -76,26 +76,31 @@ TEST_F(KuduTest, Lifecycle) {
 
 TEST_F(KuduTest, Locking) {
   string kType = "asdf";
-  string kFileName = GetTestPath("foo");
+  const string kFileName = GetTestPath("foo");
   string kUuid = "a_uuid";
 
-  PathInstanceMetadataFile file(env_, kType, kFileName);
-  ASSERT_OK(file.Create(kUuid, { kUuid }));
+  PathInstanceMetadataFile file(env_, kUuid, kType, kFileName);
+  ASSERT_OK(file.Create({ kUuid }));
 
-  PathInstanceMetadataFile first(env_, kType, kFileName);
+  PathInstanceMetadataFile first(env_, "", kType, kFileName);
   ASSERT_OK(first.LoadFromDisk());
+  ASSERT_EQ(kUuid, first.uuid());
   ASSERT_OK(first.Lock());
 
+  // Note: we must use a death test here because file locking is only
+  // disallowed across processes, and death tests spawn child processes.
   ASSERT_DEATH({
-    PathInstanceMetadataFile second(env_, kType, kFileName);
+    PathInstanceMetadataFile second(env_, "", kType, kFileName);
     CHECK_OK(second.LoadFromDisk());
+    CHECK_EQ(kUuid, second.uuid());
     CHECK_OK(second.Lock());
   }, "Could not lock");
 
   ASSERT_OK(first.Unlock());
   ASSERT_DEATH({
-    PathInstanceMetadataFile second(env_, kType, kFileName);
+    PathInstanceMetadataFile second(env_, "", kType, kFileName);
     CHECK_OK(second.LoadFromDisk());
+    CHECK_EQ(kUuid, second.uuid());
     Status s = second.Lock();
     if (s.ok()) {
       LOG(FATAL) << "Lock successfully acquired";
@@ -105,91 +110,5 @@ TEST_F(KuduTest, Locking) {
   }, "Lock successfully acquired");
 }
 
-static void RunCheckIntegrityTest(Env* env,
-                                  const vector<PathSetPB>& path_sets,
-                                  const string& expected_status_string) {
-  vector<unique_ptr<PathInstanceMetadataFile>> instances;
-
-  int i = 0;
-  for (const PathSetPB& ps : path_sets) {
-    unique_ptr<PathInstanceMetadataFile> instance(
-        new PathInstanceMetadataFile(env, "asdf", Substitute("/tmp/$0/instance", i)));
-    unique_ptr<PathInstanceMetadataPB> metadata(new PathInstanceMetadataPB());
-    metadata->set_block_manager_type("asdf");
-    metadata->set_filesystem_block_size_bytes(1);
-    metadata->mutable_path_set()->CopyFrom(ps);
-    instance->SetMetadataForTests(std::move(metadata));
-    instances.emplace_back(std::move(instance));
-    i++;
-  }
-
-  EXPECT_EQ(expected_status_string,
-            PathInstanceMetadataFile::CheckIntegrity(instances).ToString());
-}
-
-TEST_F(KuduTest, CheckIntegrity) {
-  vector<string> uuids = { "fee", "fi", "fo", "fum" };
-  RepeatedPtrField<string> kAllUuids(uuids.begin(), uuids.end());
-
-  // Initialize path_sets to be fully consistent.
-  vector<PathSetPB> path_sets(kAllUuids.size());
-  for (int i = 0; i < path_sets.size(); i++) {
-    PathSetPB& ps = path_sets[i];
-    ps.set_uuid(kAllUuids.Get(i));
-    ps.mutable_all_uuids()->CopyFrom(kAllUuids);
-  }
-
-  {
-    // Test consistent path sets.
-    EXPECT_NO_FATAL_FAILURE(RunCheckIntegrityTest(env_, path_sets, "OK"));
-  }
-  {
-    // Test where two path sets claim the same UUID.
-    vector<PathSetPB> path_sets_copy(path_sets);
-    path_sets_copy[1].set_uuid(path_sets_copy[0].uuid());
-    EXPECT_NO_FATAL_FAILURE(RunCheckIntegrityTest(
-        env_, path_sets_copy,
-        "IO error: Data directories /tmp/0 and /tmp/1 have duplicate instance metadata UUIDs: "
-        "fee"));
-  }
-  {
-    // Test where the path sets have duplicate UUIDs.
-    vector<PathSetPB> path_sets_copy(path_sets);
-    for (PathSetPB& ps : path_sets_copy) {
-      ps.add_all_uuids("fee");
-    }
-    EXPECT_NO_FATAL_FAILURE(RunCheckIntegrityTest(
-        env_, path_sets_copy,
-        "IO error: Data directory /tmp/0 instance metadata path set contains duplicate UUIDs: "
-        "fee,fi,fo,fum,fee"));
-  }
-  {
-    // Test where a path set claims a UUID that isn't in all_uuids.
-    vector<PathSetPB> path_sets_copy(path_sets);
-    path_sets_copy[1].set_uuid("something_else");
-    EXPECT_NO_FATAL_FAILURE(RunCheckIntegrityTest(
-        env_, path_sets_copy,
-        "IO error: Data directory /tmp/1 instance metadata contains unexpected UUID: "
-        "something_else"));
-  }
-  {
-    // Test where a path set claims a different all_uuids.
-    vector<PathSetPB> path_sets_copy(path_sets);
-    path_sets_copy[1].add_all_uuids("another_uuid");
-    EXPECT_NO_FATAL_FAILURE(RunCheckIntegrityTest(
-        env_, path_sets_copy,
-        "IO error: Data directories /tmp/0 and /tmp/1 have different instance metadata UUID sets: "
-        "fee,fi,fo,fum vs fee,fi,fo,fum,another_uuid"));
-  }
-  {
-    // Test removing a path from the set.
-    vector<PathSetPB> path_sets_copy(path_sets);
-    path_sets_copy.resize(1);
-    EXPECT_NO_FATAL_FAILURE(RunCheckIntegrityTest(
-        env_, path_sets_copy,
-        "IO error: 1 data directories provided, but expected 4"));
-  }
-}
-
 } // namespace fs
 } // namespace kudu
diff --git a/src/kudu/fs/block_manager_util.cc b/src/kudu/fs/block_manager_util.cc
index c2afbec..92b2765 100644
--- a/src/kudu/fs/block_manager_util.cc
+++ b/src/kudu/fs/block_manager_util.cc
@@ -19,7 +19,6 @@
 #include <cstdint>
 #include <ostream>
 #include <set>
-#include <unordered_map>
 #include <utility>
 
 #include <gflags/gflags_declare.h>
@@ -28,27 +27,81 @@
 #include "kudu/fs/fs.pb.h"
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/port.h"
-#include "kudu/gutil/strings/join.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/env.h"
+#include "kudu/util/env_util.h"
 #include "kudu/util/path_util.h"
 #include "kudu/util/pb_util.h"
 #include "kudu/util/scoped_cleanup.h"
-#include "kudu/util/test_util_prod.h"
 
 DECLARE_bool(enable_data_block_fsync);
 
-namespace kudu {
-namespace fs {
-
-using pb_util::CreateMode;
+using kudu::pb_util::CreateMode;
 using std::set;
 using std::string;
 using std::unique_ptr;
-using std::unordered_map;
-using std::vector;
 using strings::Substitute;
 
+namespace kudu {
+namespace fs {
+
+namespace {
+
+const char kHolePunchErrorMsg[] =
+    "Error during hole punch test. The log block manager requires a "
+    "filesystem with hole punching support such as ext4 or xfs. On el6, "
+    "kernel version 2.6.32-358 or newer is required. To run without hole "
+    "punching (at the cost of some efficiency and scalability), reconfigure "
+    "Kudu to use the file block manager. Refer to the Kudu documentation for "
+    "more details. WARNING: the file block manager is not suitable for "
+    "production use and should be used only for small-scale evaluation and "
+    "development on systems where hole-punching is not available. It's "
+    "impossible to switch between block managers after data is written to the "
+    "server. Raw error message follows";
+
+Status CheckHolePunch(Env* env, const string& path) {
+  // Arbitrary constants.
+  static uint64_t kFileSize = 4096 * 4;
+  static uint64_t kHoleOffset = 4096;
+  static uint64_t kHoleSize = 8192;
+  static uint64_t kPunchedFileSize = kFileSize - kHoleSize;
+
+  // Open the test file.
+  string filename = JoinPathSegments(path, "hole_punch_test_file");
+  unique_ptr<RWFile> file;
+  RWFileOptions opts;
+  RETURN_NOT_OK(env->NewRWFile(opts, filename, &file));
+
+  // The file has been created; delete it on exit no matter what happens.
+  auto file_deleter = MakeScopedCleanup([&]() {
+    WARN_NOT_OK(env->DeleteFile(filename),
+                "Could not delete file " + filename);
+  });
+
+  // Preallocate it, making sure the file's size is what we'd expect.
+  uint64_t sz;
+  RETURN_NOT_OK(file->PreAllocate(0, kFileSize, RWFile::CHANGE_FILE_SIZE));
+  RETURN_NOT_OK(env->GetFileSizeOnDisk(filename, &sz));
+  if (sz != kFileSize) {
+    return Status::IOError(Substitute(
+        "Unexpected pre-punch file size for $0: expected $1 but got $2",
+        filename, kFileSize, sz));
+  }
+
+  // Punch the hole, testing the file's size again.
+  RETURN_NOT_OK(file->PunchHole(kHoleOffset, kHoleSize));
+  RETURN_NOT_OK(env->GetFileSizeOnDisk(filename, &sz));
+  if (sz != kPunchedFileSize) {
+    return Status::IOError(Substitute(
+        "Unexpected post-punch file size for $0: expected $1 but got $2",
+        filename, kPunchedFileSize, sz));
+  }
+
+  return Status::OK();
+}
+
+} // anonymous namespace
+
 // Evaluates 'status_expr' and if it results in a disk-failure error, logs a
 // message and marks the instance as unhealthy, returning with no error.
 //
@@ -73,9 +126,11 @@ using strings::Substitute;
 } while (0)
 
 PathInstanceMetadataFile::PathInstanceMetadataFile(Env* env,
+                                                   string uuid,
                                                    string block_manager_type,
                                                    string filename)
     : env_(env),
+      uuid_(std::move(uuid)),
       block_manager_type_(std::move(block_manager_type)),
       filename_(std::move(filename)) {}
 
@@ -85,10 +140,29 @@ PathInstanceMetadataFile::~PathInstanceMetadataFile() {
   }
 }
 
-Status PathInstanceMetadataFile::Create(const string& uuid, const vector<string>& all_uuids) {
-  DCHECK(!lock_) <<
-      "Creating a metadata file that's already locked would release the lock";
-  DCHECK(ContainsKey(set<string>(all_uuids.begin(), all_uuids.end()), uuid));
+Status PathInstanceMetadataFile::Create(const set<string>& all_uuids,
+                                        bool* created_dir) {
+  DCHECK(!lock_);
+  DCHECK(ContainsKey(all_uuids, uuid_));
+  const string dir_name = dir();
+
+  bool created;
+  // Create the directory if needed.
+  RETURN_NOT_OK_FAIL_INSTANCE_PREPEND(
+      env_util::CreateDirIfMissing(env_, dir_name, &created),
+      Substitute("Could not create directory $0", dir_name));
+  auto cleanup_dir_on_failure = MakeScopedCleanup([&] {
+    if (created) {
+      WARN_NOT_OK(env_->DeleteDir(dir_name), "Could not remove newly-created directory");
+    }
+  });
+
+  // If we're initializing the log block manager, check that we support
+  // hole-punching.
+  if (block_manager_type_ == "log") {
+    RETURN_NOT_OK_FAIL_INSTANCE_PREPEND(CheckHolePunch(env_, dir_name),
+                                        kHolePunchErrorMsg);
+  }
 
   // Create a temporary file with which to fetch the filesystem's block size.
   //
@@ -98,23 +172,25 @@ Status PathInstanceMetadataFile::Create(const string& uuid, const vector<string>
   // it's more correct to derive it from a file in any case.
   string created_filename;
   string tmp_template = JoinPathSegments(
-      DirName(filename_), Substitute("getblocksize$0.XXXXXX", kTmpInfix));
+      dir_name, Substitute("getblocksize$0.XXXXXX", kTmpInfix));
   unique_ptr<WritableFile> tmp_file;
-  RETURN_NOT_OK(env_->NewTempWritableFile(WritableFileOptions(),
-                                          tmp_template,
-                                          &created_filename, &tmp_file));
+  RETURN_NOT_OK_FAIL_INSTANCE_PREPEND(
+      env_->NewTempWritableFile(WritableFileOptions(),
+                                tmp_template,
+                                &created_filename, &tmp_file),
+      "failed to create temp file while checking block size");
   SCOPED_CLEANUP({
     WARN_NOT_OK(env_->DeleteFile(created_filename),
                 "could not delete temporary file");
   });
   uint64_t block_size;
-  RETURN_NOT_OK(env_->GetBlockSize(created_filename, &block_size));
-
-  PathInstanceMetadataPB new_instance;
+  RETURN_NOT_OK_FAIL_INSTANCE_PREPEND(env_->GetBlockSize(created_filename, &block_size),
+                                      "failed to check block size");
 
   // Set up the path set.
+  PathInstanceMetadataPB new_instance;
   PathSetPB* new_path_set = new_instance.mutable_path_set();
-  new_path_set->set_uuid(uuid);
+  new_path_set->set_uuid(uuid_);
   new_path_set->mutable_all_uuids()->Reserve(all_uuids.size());
   for (const string& u : all_uuids) {
     new_path_set->add_all_uuids(u);
@@ -124,10 +200,20 @@ Status PathInstanceMetadataFile::Create(const string& uuid, const vector<string>
   new_instance.set_block_manager_type(block_manager_type_);
   new_instance.set_filesystem_block_size_bytes(block_size);
 
-  return pb_util::WritePBContainerToPath(
+  RETURN_NOT_OK_FAIL_INSTANCE_PREPEND(pb_util::WritePBContainerToPath(
       env_, filename_, new_instance,
       pb_util::NO_OVERWRITE,
-      FLAGS_enable_data_block_fsync ? pb_util::SYNC : pb_util::NO_SYNC);
+      FLAGS_enable_data_block_fsync ? pb_util::SYNC : pb_util::NO_SYNC),
+      "failed to write PB");
+
+  // Now that we're returning success, we don't need to clean anything up, and
+  // we can indicate to callers there is a new directory to clean up (if
+  // appropriate).
+  cleanup_dir_on_failure.cancel();
+  if (created_dir) {
+    *created_dir = created;
+  }
+  return Status::OK();
 }
 
 Status PathInstanceMetadataFile::LoadFromDisk() {
@@ -140,8 +226,8 @@ Status PathInstanceMetadataFile::LoadFromDisk() {
 
   if (pb->block_manager_type() != block_manager_type_) {
     return Status::IOError(Substitute(
-      "existing data was written using the '$0' block manager; cannot restart "
-      "with a different block manager '$1' without reformatting",
+      "existing instance was written using the '$0' format; cannot restart "
+      "with a different format type '$1'",
       pb->block_manager_type(), block_manager_type_));
   }
 
@@ -153,7 +239,8 @@ Status PathInstanceMetadataFile::LoadFromDisk() {
         "Expected $0 but was $1", pb->filesystem_block_size_bytes(), block_size));
   }
 
-  metadata_.swap(pb);
+  uuid_ = pb->path_set().uuid();
+  metadata_ = std::move(pb);
   return Status::OK();
 }
 
@@ -162,7 +249,7 @@ Status PathInstanceMetadataFile::Lock() {
 
   FileLock* lock;
   RETURN_NOT_OK_FAIL_INSTANCE_PREPEND(env_->LockFile(filename_, &lock),
-                                      "Could not lock block_manager_instance file. Make sure that "
+                                      "Could not lock instance file. Make sure that "
                                       "Kudu is not already running and you are not trying to run "
                                       "Kudu with a different user than before");
   lock_.reset(lock);
@@ -177,94 +264,5 @@ Status PathInstanceMetadataFile::Unlock() {
   return Status::OK();
 }
 
-void PathInstanceMetadataFile::SetMetadataForTests(
-    unique_ptr<PathInstanceMetadataPB> metadata) {
-  DCHECK(IsGTest());
-  metadata_ = std::move(metadata);
-}
-
-Status PathInstanceMetadataFile::CheckIntegrity(
-    const vector<unique_ptr<PathInstanceMetadataFile>>& instances) {
-  CHECK(!instances.empty());
-
-  // Note: although this verification works at the level of UUIDs and instance
-  // files, the (user-facing) error messages are reported in terms of data
-  // directories, because UUIDs and instance files are internal details.
-
-  int first_healthy = -1;
-  for (int i = 0; i < instances.size(); i++) {
-    if (instances[i]->healthy()) {
-      first_healthy = i;
-      break;
-    }
-  }
-  if (first_healthy == -1) {
-    return Status::NotFound("no healthy data directories found");
-  }
-
-  // Map of instance UUID to path instance structure. Tracks duplicate UUIDs.
-  unordered_map<string, PathInstanceMetadataFile*> uuids;
-
-  // Set of UUIDs specified in the path set of the first healthy instance. All
-  // instances will be compared against it to make sure all path sets match.
-  set<string> all_uuids(instances[first_healthy]->metadata()->path_set().all_uuids().begin(),
-                        instances[first_healthy]->metadata()->path_set().all_uuids().end());
-
-  if (all_uuids.size() != instances.size()) {
-    return Status::IOError(
-        Substitute("$0 data directories provided, but expected $1",
-                   instances.size(), all_uuids.size()));
-  }
-
-  for (const auto& instance : instances) {
-    // If the instance has failed (e.g. due to disk failure), there's no
-    // telling what its metadata looks like. Ignore it, and continue checking
-    // integrity across the healthy instances.
-    if (!instance->healthy()) {
-      continue;
-    }
-    const PathSetPB& path_set = instance->metadata()->path_set();
-
-    // Check that the instance's UUID has not been claimed by another instance.
-    PathInstanceMetadataFile** other = InsertOrReturnExisting(
-        &uuids, path_set.uuid(), instance.get());
-    if (other) {
-      return Status::IOError(
-          Substitute("Data directories $0 and $1 have duplicate instance metadata UUIDs",
-                     (*other)->dir(), instance->dir()),
-          path_set.uuid());
-    }
-
-    // Check that the instance's UUID is a member of all_uuids.
-    if (!ContainsKey(all_uuids, path_set.uuid())) {
-      return Status::IOError(
-          Substitute("Data directory $0 instance metadata contains unexpected UUID",
-                     instance->dir()),
-          path_set.uuid());
-    }
-
-    // Check that the instance's UUID set does not contain duplicates.
-    set<string> deduplicated_uuids(path_set.all_uuids().begin(),
-                                   path_set.all_uuids().end());
-    string all_uuids_str = JoinStrings(path_set.all_uuids(), ",");
-    if (deduplicated_uuids.size() != path_set.all_uuids_size()) {
-      return Status::IOError(
-          Substitute("Data directory $0 instance metadata path set contains duplicate UUIDs",
-                     instance->dir()),
-          JoinStrings(path_set.all_uuids(), ","));
-    }
-
-    // Check that the instance's UUID set matches the expected set.
-    if (deduplicated_uuids != all_uuids) {
-      return Status::IOError(
-          Substitute("Data directories $0 and $1 have different instance metadata UUID sets",
-                     instances[0]->dir(), instance->dir()),
-          Substitute("$0 vs $1", JoinStrings(all_uuids, ","), all_uuids_str));
-    }
-  }
-
-  return Status::OK();
-}
-
 } // namespace fs
 } // namespace kudu
diff --git a/src/kudu/fs/block_manager_util.h b/src/kudu/fs/block_manager_util.h
index a9b75aa..4675a3c 100644
--- a/src/kudu/fs/block_manager_util.h
+++ b/src/kudu/fs/block_manager_util.h
@@ -18,8 +18,8 @@
 #pragma once
 
 #include <memory>
+#include <set>
 #include <string>
-#include <vector>
 
 #include "kudu/util/path_util.h"
 #include "kudu/util/status.h"
@@ -38,17 +38,24 @@ namespace fs {
 class PathInstanceMetadataFile {
  public:
   // 'env' must remain valid for the lifetime of this class.
-  PathInstanceMetadataFile(Env* env, std::string block_manager_type,
+  //
+  // 'uuid' is the UUID used for this instance file, though the UUID may be
+  // changed if we read the instance file from disk and find a different UUID.
+  PathInstanceMetadataFile(Env* env, std::string uuid, std::string block_manager_type,
                            std::string filename);
 
   ~PathInstanceMetadataFile();
 
   // Creates, writes, synchronizes, and closes a new instance metadata file.
+  // Fails the PIMF in the case of a failed directory.
+  //
+  // 'all_uuids' is all of the unique UUIDs in this instance's path set.
+  // 'uuid_' must exist in this set. 'created_dir' is set to true if the parent
+  // directory of the file was also created during this process.
   //
-  // 'uuid' is this instance's UUID, and 'all_uuids' is all of the UUIDs in
-  // this instance's path set.
-  Status Create(const std::string& uuid,
-                const std::vector<std::string>& all_uuids);
+  // This should be called if we've already tried to load the instance file but
+  // found it didn't exist.
+  Status Create(const std::set<std::string>& all_uuids, bool* created_dir = nullptr);
 
   // Opens, reads, verifies, and closes an existing instance metadata file.
   //
@@ -69,9 +76,6 @@ class PathInstanceMetadataFile {
   // Unlocks the instance metadata file. Must have been locked to begin with.
   Status Unlock();
 
-  // Sets the metadata file.
-  void SetMetadataForTests(std::unique_ptr<PathInstanceMetadataPB> metadata);
-
   // Sets that the instance failed (e.g. due to a disk failure).
   //
   // If failed, there is no guarantee that the instance will have a 'metadata_'.
@@ -89,21 +93,36 @@ class PathInstanceMetadataFile {
     return health_status_;
   }
 
+  std::string uuid() const { return uuid_; }
   std::string dir() const { return DirName(filename_); }
   const std::string& path() const { return filename_; }
-  PathInstanceMetadataPB* const metadata() const { return metadata_.get(); }
-
-  // Check the integrity of the provided instances' path sets, ignoring any
-  // unhealthy instances.
-  static Status CheckIntegrity(
-      const std::vector<std::unique_ptr<PathInstanceMetadataFile>>& instances);
+  PathInstanceMetadataPB* metadata() const { return metadata_.get(); }
 
  private:
   Env* env_;
+
+  // The UUID of this instance file. This is initialized in the constructor so
+  // it can be used when creating a new instance file. However, it may be
+  // overwritten when loading an existing instance from disk.
+  //
+  // This helps provide the invariant that, healthy or otherwise, every
+  // instance has a valid UUID, which is useful for failed-directory-tracking,
+  // which generally uses UUIDs.
+  std::string uuid_;
+
+  // The type of this instance file.
   const std::string block_manager_type_;
+
+  // The name of the file associated with this instance.
   const std::string filename_;
+
   std::unique_ptr<PathInstanceMetadataPB> metadata_;
+
+  // In order to prevent multiple Kudu processes from starting up using the
+  // same directories, we lock the instance files when starting up.
   std::unique_ptr<FileLock> lock_;
+
+  // The health of the instance file.
   Status health_status_;
 };
 
diff --git a/src/kudu/fs/data_dirs.cc b/src/kudu/fs/data_dirs.cc
index 8142877..a7f96e0 100644
--- a/src/kudu/fs/data_dirs.cc
+++ b/src/kudu/fs/data_dirs.cc
@@ -33,8 +33,8 @@
 #include <vector>
 
 #include <gflags/gflags.h>
-#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
+#include <google/protobuf/stubs/common.h>
 
 #include "kudu/fs/block_manager.h"
 #include "kudu/fs/block_manager_util.h"
@@ -136,59 +136,6 @@ using strings::SubstituteAndAppend;
 
 namespace {
 
-const char kHolePunchErrorMsg[] =
-    "Error during hole punch test. The log block manager requires a "
-    "filesystem with hole punching support such as ext4 or xfs. On el6, "
-    "kernel version 2.6.32-358 or newer is required. To run without hole "
-    "punching (at the cost of some efficiency and scalability), reconfigure "
-    "Kudu to use the file block manager. Refer to the Kudu documentation for "
-    "more details. WARNING: the file block manager is not suitable for "
-    "production use and should be used only for small-scale evaluation and "
-    "development on systems where hole-punching is not available. It's "
-    "impossible to switch between block managers after data is written to the "
-    "server. Raw error message follows";
-
-Status CheckHolePunch(Env* env, const string& path) {
-  // Arbitrary constants.
-  static uint64_t kFileSize = 4096 * 4;
-  static uint64_t kHoleOffset = 4096;
-  static uint64_t kHoleSize = 8192;
-  static uint64_t kPunchedFileSize = kFileSize - kHoleSize;
-
-  // Open the test file.
-  string filename = JoinPathSegments(path, "hole_punch_test_file");
-  unique_ptr<RWFile> file;
-  RWFileOptions opts;
-  RETURN_NOT_OK(env->NewRWFile(opts, filename, &file));
-
-  // The file has been created; delete it on exit no matter what happens.
-  auto file_deleter = MakeScopedCleanup([&]() {
-    WARN_NOT_OK(env->DeleteFile(filename),
-                "Could not delete file " + filename);
-  });
-
-  // Preallocate it, making sure the file's size is what we'd expect.
-  uint64_t sz;
-  RETURN_NOT_OK(file->PreAllocate(0, kFileSize, RWFile::CHANGE_FILE_SIZE));
-  RETURN_NOT_OK(env->GetFileSizeOnDisk(filename, &sz));
-  if (sz != kFileSize) {
-    return Status::IOError(Substitute(
-        "Unexpected pre-punch file size for $0: expected $1 but got $2",
-        filename, kFileSize, sz));
-  }
-
-  // Punch the hole, testing the file's size again.
-  RETURN_NOT_OK(file->PunchHole(kHoleOffset, kHoleSize));
-  RETURN_NOT_OK(env->GetFileSizeOnDisk(filename, &sz));
-  if (sz != kPunchedFileSize) {
-    return Status::IOError(Substitute(
-        "Unexpected post-punch file size for $0: expected $1 but got $2",
-        filename, kPunchedFileSize, sz));
-  }
-
-  return Status::OK();
-}
-
 // Wrapper for env_util::DeleteTmpFilesRecursively that is suitable for parallel
 // execution on a data directory's thread pool (which requires the return value
 // be void).
@@ -352,7 +299,7 @@ Status DataDirGroup::CopyToPB(const UuidByUuidIndexMap& uuid_by_uuid_idx,
 DataDirManagerOptions::DataDirManagerOptions()
     : block_manager_type(FLAGS_block_manager),
       read_only(false),
-      consistency_check(ConsistencyCheckBehavior::ENFORCE_CONSISTENCY) {
+      update_instances(UpdateInstanceBehavior::UPDATE_AND_IGNORE_FAILURES) {
 }
 
 ////////////////////////////////////////////////////////////
@@ -374,8 +321,7 @@ DataDirManager::DataDirManager(Env* env,
       canonicalized_data_fs_roots_(std::move(canonicalized_data_roots)),
       rng_(GetRandomSeed32()) {
   DCHECK_GT(canonicalized_data_fs_roots_.size(), 0);
-  DCHECK(opts_.consistency_check != ConsistencyCheckBehavior::UPDATE_ON_DISK ||
-         !opts_.read_only);
+  DCHECK(opts_.update_instances == UpdateInstanceBehavior::DONT_UPDATE || !opts_.read_only);
 
   if (opts_.metric_entity) {
     metrics_.reset(new DataDirMetrics(opts_.metric_entity));
@@ -447,27 +393,28 @@ Status DataDirManager::CreateNew(Env* env, CanonicalizedRootsList data_fs_roots,
 Status DataDirManager::Create() {
   CHECK(!opts_.read_only);
 
-  // Generate a new UUID for each data directory.
-  ObjectIdGenerator gen;
   vector<string> all_uuids;
-  vector<pair<string, string>> root_uuid_pairs_to_create;
   for (const auto& r : canonicalized_data_fs_roots_) {
     RETURN_NOT_OK_PREPEND(r.status, "Could not create directory manager with disks failed");
-    string uuid = gen.Next();
-    all_uuids.emplace_back(uuid);
-    root_uuid_pairs_to_create.emplace_back(r.path, std::move(uuid));
   }
-  RETURN_NOT_OK_PREPEND(CreateNewDataDirectoriesAndUpdateInstances(
-      std::move(root_uuid_pairs_to_create), {}, std::move(all_uuids)),
+  vector<unique_ptr<PathInstanceMetadataFile>> loaded_instances;
+  bool has_existing_instances;
+  RETURN_NOT_OK(LoadInstances(&loaded_instances, &has_existing_instances));
+  if (has_existing_instances) {
+    return Status::AlreadyPresent("instance files already exist");
+  }
+
+  // If none of the instances exist, we can assume this is a new deployment and
+  // we should try creating some a new set of instance files.
+  RETURN_NOT_OK_PREPEND(CreateNewDirectoriesAndUpdateInstances(std::move(loaded_instances)),
                         "could not create new data directories");
   return Status::OK();
 }
 
-Status DataDirManager::CreateNewDataDirectoriesAndUpdateInstances(
-    vector<pair<string, string>> root_uuid_pairs_to_create,
-    vector<unique_ptr<PathInstanceMetadataFile>> instances_to_update,
-    vector<string> all_uuids) {
+Status DataDirManager::CreateNewDirectoriesAndUpdateInstances(
+    vector<unique_ptr<PathInstanceMetadataFile>> instances) {
   CHECK(!opts_.read_only);
+  CHECK_NE(UpdateInstanceBehavior::DONT_UPDATE, opts_.update_instances);
 
   vector<string> created_dirs;
   vector<string> created_files;
@@ -483,31 +430,63 @@ Status DataDirManager::CreateNewDataDirectoriesAndUpdateInstances(
     }
   });
 
-  // Ensure the data dirs exist and create the instance files.
-  for (const auto& p : root_uuid_pairs_to_create) {
-    string data_dir = JoinPathSegments(p.first, kDataDirName);
-    bool created;
-    RETURN_NOT_OK_PREPEND(env_util::CreateDirIfMissing(env_, data_dir, &created),
-        Substitute("Could not create directory $0", data_dir));
-    if (created) {
-      created_dirs.emplace_back(data_dir);
-    }
+  // First, de-duplicate the instance UUIDs. If we have duplicates, something's
+  // wrong. Maybe an operator manually duplicated some instance files.
+  set<string> all_uuids;
+  for (const auto& instance : instances) {
+    InsertIfNotPresent(&all_uuids, instance->uuid());
+  }
+  if (all_uuids.size() != instances.size()) {
+    return Status::InvalidArgument(
+        Substitute("instance files contain duplicate UUIDs: $0 directories provided, "
+                   "$1 unique UUIDs found ($2)", instances.size(), all_uuids.size(),
+                   JoinStrings(all_uuids, ", ")));
+  }
 
-    if (opts_.block_manager_type == "log") {
-      RETURN_NOT_OK_PREPEND(CheckHolePunch(env_, data_dir), kHolePunchErrorMsg);
+  // Determine which instance files are healthy (and can thus be updated), and
+  // which don't exist. Create any that don't exist.
+  //
+  // Note: we don't bother trying to create/update the instance if the file is
+  // otherwise unhealthy.
+  vector<unique_ptr<PathInstanceMetadataFile>> healthy_instances;
+  for (auto& instance : instances) {
+    if (instance->healthy()) {
+      healthy_instances.emplace_back(std::move(instance));
+      continue;
     }
+    if (instance->health_status().IsNotFound()) {
+      bool created_dir = false;
+      RETURN_NOT_OK(instance->Create(all_uuids, &created_dir));
+      if (created_dir) {
+        created_dirs.emplace_back(instance->dir());
+      }
+      created_files.emplace_back(instance->path());
+    }
+  }
 
-    string instance_filename = JoinPathSegments(data_dir, kInstanceMetadataFileName);
-    PathInstanceMetadataFile metadata(env_, opts_.block_manager_type,
-                                      instance_filename);
-    RETURN_NOT_OK_PREPEND(metadata.Create(p.second, all_uuids), instance_filename);
-    created_files.emplace_back(instance_filename);
+  // Go through the healthy instances and look for instances that don't have
+  // the full complete set of instance UUIDs.
+  vector<unique_ptr<PathInstanceMetadataFile>> instances_to_update;
+  for (auto& instance : healthy_instances) {
+    DCHECK(instance->healthy());
+    const auto& path_set = instance->metadata()->path_set();
+    set<string> instance_uuids;
+    for (int i = 0; i < path_set.all_uuids_size(); i++) {
+      InsertIfNotPresent(&instance_uuids, path_set.all_uuids(i));
+    }
+    // If an instance file disagrees with the expected UUIDs, rewrite it.
+    if (all_uuids != instance_uuids) {
+      instances_to_update.emplace_back(std::move(instance));
+    }
   }
 
-  // Update existing instances, if any.
-  RETURN_NOT_OK_PREPEND(UpdateInstances(
-      std::move(instances_to_update), std::move(all_uuids)),
-                        "could not update existing data directories");
+  // If any of the instance files need to be updated because they didn't match
+  // the expected set of UUIDs, update them now.
+  // Note: Having a consistent set of instance files isn't a correctness
+  // requirement, but it can be useful for degbugging.
+  if (!instances_to_update.empty()) {
+    RETURN_NOT_OK(UpdateHealthyInstances(instances_to_update, all_uuids));
+  }
 
   // Ensure newly created directories are synchronized to disk.
   if (FLAGS_enable_data_block_fsync) {
@@ -520,26 +499,28 @@ Status DataDirManager::CreateNewDataDirectoriesAndUpdateInstances(
   return Status::OK();
 }
 
-Status DataDirManager::UpdateInstances(
-    vector<unique_ptr<PathInstanceMetadataFile>> instances_to_update,
-    vector<string> new_all_uuids) {
-  // Prepare a scoped cleanup for managing instance metadata copies.
+Status DataDirManager::UpdateHealthyInstances(
+    const vector<unique_ptr<PathInstanceMetadataFile>>& instances_to_update,
+    const set<string>& new_all_uuids) {
   unordered_map<string, string> copies_to_restore;
   unordered_set<string> copies_to_delete;
-  auto copy_cleanup = MakeScopedCleanup([&]() {
+  auto cleanup = MakeScopedCleanup([&] {
     for (const auto& f : copies_to_delete) {
-      WARN_NOT_OK(env_->DeleteFile(f), "Could not delete file " + f);
+      WARN_NOT_OK(env_->DeleteFile(f), Substitute("Could not delete file $0", f));
     }
-    for (const auto& f : copies_to_restore) {
-      WARN_NOT_OK(env_->RenameFile(f.first, f.second),
-                  Substitute("Could not restore file $0 from $1", f.second, f.first));
+    for (const auto& copy_and_original : copies_to_restore) {
+      const auto& copy_filename = copy_and_original.first;
+      const auto& original_filename = copy_and_original.second;
+      WARN_NOT_OK(env_->RenameFile(copy_filename, original_filename),
+          Substitute("Could not restore file $0 from $1", original_filename, copy_filename));
     }
   });
-
-  // Make a copy of every existing instance metadata file.
+  // Make a copy of every existing instance metadata file. This is done before
+  // performing any updates, so that if there's a failure while copying,
+  // there's no metadata to restore.
   //
-  // This is done before performing any updates, so that if there's a failure
-  // while copying, there's no metadata to restore.
+  // We'll keep track of the copies so we can delete them on success, or use
+  // them to restore on failure.
   WritableFileOptions opts;
   opts.sync_on_close = true;
   for (const auto& instance : instances_to_update) {
@@ -548,13 +529,17 @@ Status DataDirManager::UpdateInstances(
     }
     const string& instance_filename = instance->path();
     string copy_filename = instance_filename + kTmpInfix;
-    RETURN_NOT_OK_PREPEND(env_util::CopyFile(
-        env_, instance_filename, copy_filename, opts),
-                          "unable to backup existing data directory instance metadata");
+    Status s = env_util::CopyFile(env_, instance_filename, copy_filename, opts);
+    if (PREDICT_FALSE(!s.ok())) {
+      s = s.CloneAndPrepend("unable to backup existing instance file");
+      instance->SetInstanceFailed(s);
+      LOG(WARNING) << s.ToString();
+      continue;
+    }
     InsertOrDie(&copies_to_delete, copy_filename);
   }
 
-  // Update existing instance metadata files with the new value of all_uuids.
+  // Update the instance metadata files with the new set of UUIDs.
   for (const auto& instance : instances_to_update) {
     if (!instance->healthy()) {
       continue;
@@ -562,33 +547,48 @@ Status DataDirManager::UpdateInstances(
     const string& instance_filename = instance->path();
     string copy_filename = instance_filename + kTmpInfix;
 
-    // We've made enough progress on this instance that we should restore its
-    // copy on failure, even if the update below fails. That's because it's a
-    // multi-step process and it's possible for it to return failure despite
-    // the update taking place (e.g. synchronization failure).
-    CHECK_EQ(1, copies_to_delete.erase(copy_filename));
-    InsertOrDie(&copies_to_restore, copy_filename, instance_filename);
-
-    // Perform the update.
+    // Put together the PB and perform the update.
     PathInstanceMetadataPB new_pb = *instance->metadata();
     new_pb.mutable_path_set()->mutable_all_uuids()->Clear();
     for (const auto& uuid : new_all_uuids) {
       new_pb.mutable_path_set()->add_all_uuids(uuid);
     }
-    RETURN_NOT_OK_PREPEND(pb_util::WritePBContainerToPath(
+
+    // We're about to update the file; if we fail midway, we should try to
+    // restore them from our backups if we can.
+    InsertOrDie(&copies_to_restore, copy_filename, instance_filename);
+    CHECK_EQ(1, copies_to_delete.erase(copy_filename));
+    Status s = pb_util::WritePBContainerToPath(
         env_, instance_filename, new_pb, pb_util::OVERWRITE,
-        FLAGS_enable_data_block_fsync ? pb_util::SYNC : pb_util::NO_SYNC),
-                          "unable to overwrite existing data directory instance metadata");
+        FLAGS_enable_data_block_fsync ? pb_util::SYNC : pb_util::NO_SYNC);
+    // We've failed to update for some reason, so restore our original file.
+    // Since we're renaming our copy, we don't have to delete it.
+    if (PREDICT_FALSE(!s.ok())) {
+      s = s.CloneAndPrepend("unable to update instance file");
+      instance->SetInstanceFailed(s);
+      LOG(WARNING) << Substitute("unable to overwrite existing instance file $0: $1",
+                                 instance_filename, s.ToString());
+    }
   }
 
-  // Success; instance metadata copies will be deleted by 'copy_cleanup'.
+  // If we are not tolerating errors (e.g. we're running the update_dirs tool)
+  // and we've hit an error, return now and clean up what we've changed.
+  if (opts_.update_instances == UpdateInstanceBehavior::UPDATE_AND_ERROR_ON_FAILURE) {
+    for (const auto& instance : instances_to_update) {
+      RETURN_NOT_OK_PREPEND(instance->health_status(),
+          "at least one instance file failed to update");
+    }
+  }
+
+  // Success; we only need to delete our copies.
   InsertKeysFromMap(copies_to_restore, &copies_to_delete);
   copies_to_restore.clear();
   return Status::OK();
 }
 
 Status DataDirManager::LoadInstances(
-    vector<unique_ptr<PathInstanceMetadataFile>>* loaded_instances) {
+    vector<unique_ptr<PathInstanceMetadataFile>>* instance_files,
+    bool* has_existing_instances) {
   LockMode lock_mode;
   if (!FLAGS_fs_lock_data_dirs) {
     lock_mode = LockMode::NONE;
@@ -598,14 +598,19 @@ Status DataDirManager::LoadInstances(
     lock_mode = LockMode::MANDATORY;
   }
   vector<string> missing_roots_tmp;
-  vector<unique_ptr<PathInstanceMetadataFile>> loaded_instances_tmp;
+  vector<unique_ptr<PathInstanceMetadataFile>> loaded_instances;
+  ObjectIdGenerator gen;
   for (int i = 0; i < canonicalized_data_fs_roots_.size(); i++) {
     const auto& root = canonicalized_data_fs_roots_[i];
     string data_dir = JoinPathSegments(root.path, kDataDirName);
     string instance_filename = JoinPathSegments(data_dir, kInstanceMetadataFileName);
 
+    // Initialize the instance with a backup UUID. In case the load fails, this
+    // will be the UUID for our instnace.
+    string backup_uuid = gen.Next();
     unique_ptr<PathInstanceMetadataFile> instance(
-        new PathInstanceMetadataFile(env_, opts_.block_manager_type, instance_filename));
+        new PathInstanceMetadataFile(env_, std::move(backup_uuid), opts_.block_manager_type,
+                                     instance_filename));
     if (PREDICT_FALSE(!root.status.ok())) {
       instance->SetInstanceFailed(root.status);
     } else {
@@ -630,90 +635,131 @@ Status DataDirManager::LoadInstances(
         }
       }
     }
-
-    loaded_instances_tmp.emplace_back(std::move(instance));
+    loaded_instances.emplace_back(std::move(instance));
   }
 
   int num_healthy_instances = 0;
-  for (const auto& instance : loaded_instances_tmp) {
+  for (const auto& instance : loaded_instances) {
     if (instance->healthy()) {
       num_healthy_instances++;
     }
   }
-  if (num_healthy_instances == 0) {
-    return Status::NotFound("could not open directory manager; no healthy "
-                            "data directories found");
+  if (has_existing_instances) {
+    *has_existing_instances = num_healthy_instances > 0;
   }
-  loaded_instances->swap(loaded_instances_tmp);
+  instance_files->swap(loaded_instances);
   return Status::OK();
 }
 
-Status DataDirManager::Open() {
-  const int kMaxDataDirs = opts_.block_manager_type == "file" ? (1 << 16) - 1 : kint32max;
-
-  // Find and load existing data directory instances.
-  vector<unique_ptr<PathInstanceMetadataFile>> loaded_instances;
-  RETURN_NOT_OK(LoadInstances(&loaded_instances));
-
-  // Add new or remove existing data directories, if desired.
-  if (opts_.consistency_check == ConsistencyCheckBehavior::UPDATE_ON_DISK) {
-    if (opts_.block_manager_type == "file") {
-      return Status::InvalidArgument(
-          "file block manager may not add or remove data directories");
+Status DataDirManager::PopulateDirectoryMaps(const vector<unique_ptr<DataDir>>& dds) {
+  // Helper lambda to add a directory to the maps.
+  const auto insert_to_maps = [&] (const string& uuid, int idx, DataDir* dd) {
+    if (!dd->instance()->healthy()) {
+      if (metrics_) {
+        metrics_->data_dirs_failed->IncrementBy(1);
+      }
+      InsertOrDie(&failed_data_dirs_, idx);
     }
+    InsertOrDie(&uuid_by_root_, DirName(dd->dir()), uuid);
+    InsertOrDie(&uuid_by_idx_, idx, uuid);
+    InsertOrDie(&idx_by_uuid_, uuid, idx);
+    InsertOrDie(&data_dir_by_uuid_idx_, idx, dd);
+    InsertOrDie(&uuid_idx_by_data_dir_, dd, idx);
+    InsertOrDie(&tablets_by_uuid_idx_map_, idx, {});
+  };
 
-    // Prepare to create new directories and update existing instances. We
-    // must generate a new UUID for each missing root, and update all_uuids in
-    // all existing instances to include those new UUIDs.
-    //
-    // Note: all data directories must be healthy to perform this operation.
-    ObjectIdGenerator gen;
-    vector<string> new_all_uuids;
-    vector<pair<string, string>> root_uuid_pairs_to_create;
-    for (const auto& i : loaded_instances) {
-      if (i->health_status().IsNotFound()) {
-        string uuid = gen.Next();
-        new_all_uuids.emplace_back(uuid);
-        root_uuid_pairs_to_create.emplace_back(DirName(i->dir()), std::move(uuid));
-        continue;
+  if (opts_.block_manager_type == "file") {
+    // When assigning directories for the file block manager, the UUID indexes
+    // must match what exists in the instance files' list of UUIDs.
+    unordered_map<string, int> uuid_to_idx;
+    for (const auto& dd : dds) {
+      // Find a healthy instance file and use its set of UUIDs.
+      if (dd->instance()->healthy()) {
+        const auto& path_set = dd->instance()->metadata()->path_set();
+        VLOG(1) << Substitute("using path set $0 as reference: $1",
+            dd->instance()->path(), pb_util::SecureDebugString(path_set));
+        for (int idx = 0; idx < path_set.all_uuids_size(); idx++) {
+          const string& uuid = path_set.all_uuids(idx);
+          InsertIfNotPresent(&uuid_to_idx, uuid, idx);
+        }
+        break;
       }
-      RETURN_NOT_OK_PREPEND(
-          i->health_status(),
-          "found failed data directory while adding new data directories");
-      new_all_uuids.emplace_back(i->metadata()->path_set().uuid());
     }
-    RETURN_NOT_OK_PREPEND(
-        CreateNewDataDirectoriesAndUpdateInstances(
-            std::move(root_uuid_pairs_to_create),
-            std::move(loaded_instances),
-            std::move(new_all_uuids)),
-            "could not add new data directories");
-
-    // Now that we've created the missing directories, try loading the
-    // directories again.
-    //
-    // Note: 'loaded_instances' must be cleared to unlock the instance files.
-    loaded_instances.clear();
-    RETURN_NOT_OK(LoadInstances(&loaded_instances));
-    for (const auto& i : loaded_instances) {
-      RETURN_NOT_OK_PREPEND(i->health_status(),
-          "found failed data directory after updating data directories");
+    // We should have the same number of UUID assignments as directories.
+    if (dds.size() != uuid_to_idx.size()) {
+      return Status::Corruption(
+          Substitute("instance file is corrupted: $0 unique UUIDs expected, got $1",
+                     dds.size(), uuid_to_idx.size()));
     }
+    // Keep track of any dirs that were not referenced in the path set. These
+    // are presumably from instance files we failed to read. We'll assign them
+    // indexes of those that remain.
+    vector<DataDir*> unassigned_dirs;
+    for (const auto& dd : dds) {
+      const auto& uuid = dd->instance()->uuid();
+      int* idx = FindOrNull(uuid_to_idx, uuid);
+      if (idx) {
+        insert_to_maps(uuid, *idx, dd.get());
+        uuid_to_idx.erase(uuid);
+      } else {
+        LOG(WARNING) << Substitute("instance $0 has unknown UUID $1",
+                                   dd->instance()->path(), uuid);
+        unassigned_dirs.emplace_back(dd.get());
+      }
+    }
+    DCHECK_EQ(unassigned_dirs.size(), uuid_to_idx.size());
+    int unassigned_dir_idx = 0;
+    for (const auto& failed_uuid_and_idx : uuid_to_idx) {
+      insert_to_maps(failed_uuid_and_idx.first, failed_uuid_and_idx.second,
+                     unassigned_dirs[unassigned_dir_idx++]);
+    }
+  } else {
+    // Go through our instances and assign them each a UUID index.
+    for (int idx = 0; idx < dds.size(); idx++) {
+      DataDir* dd = dds[idx].get();
+      insert_to_maps(dd->instance()->uuid(), idx, dd);
+    }
+  }
+  return Status::OK();
+}
+
+Status DataDirManager::Open() {
+  const int kMaxDataDirs = opts_.block_manager_type == "file" ? (1 << 16) - 1 : kint32max;
+  if (canonicalized_data_fs_roots_.size() > kMaxDataDirs) {
+    return Status::InvalidArgument(Substitute("too many directories provided $0, max is $1",
+                                              canonicalized_data_fs_roots_.size(), kMaxDataDirs));
   }
 
-  // Check the integrity of all loaded instances.
-  if (opts_.consistency_check != ConsistencyCheckBehavior::IGNORE_INCONSISTENCY) {
+  vector<unique_ptr<PathInstanceMetadataFile>> loaded_instances;
+  // Load the instance files from disk.
+  bool has_existing_instances;
+  RETURN_NOT_OK_PREPEND(LoadInstances(&loaded_instances, &has_existing_instances),
+      "failed to load instance files");
+  if (!has_existing_instances) {
+    return Status::NotFound(
+        "could not open directory manager, no healthy data directories found");
+  }
+  // Note: the file block manager should not be updated because its block
+  // indexing algorithm depends on a fixed set of directories.
+  if (!opts_.read_only && opts_.block_manager_type != "file" &&
+      opts_.update_instances != UpdateInstanceBehavior::DONT_UPDATE) {
     RETURN_NOT_OK_PREPEND(
-        PathInstanceMetadataFile::CheckIntegrity(loaded_instances),
-        Substitute("could not verify integrity of files: $0",
-                   JoinStrings(GetDataDirs(), ",")));
+        CreateNewDirectoriesAndUpdateInstances(
+            std::move(loaded_instances)),
+            "could not add new data directories");
+    RETURN_NOT_OK_PREPEND(LoadInstances(&loaded_instances, &has_existing_instances),
+                          "failed to load instance files after updating");
+    if (!has_existing_instances) {
+      return Status::IOError(
+          "could not open directory manager, no healthy data directories found");
+    }
   }
 
   // All instances are present and accounted for. Time to create the in-memory
   // data directory structures.
-  int i = 0;
   vector<unique_ptr<DataDir>> dds;
-  for (auto& instance : loaded_instances) {
+  for (int i = 0; i < loaded_instances.size(); i++) {
+    auto& instance = loaded_instances[i];
     const string data_dir = instance->dir();
 
     // Create a per-dir thread pool.
@@ -726,23 +772,30 @@ Status DataDirManager::Open() {
     // Figure out what filesystem the data directory is on.
     DataDirFsType fs_type = DataDirFsType::OTHER;
     if (instance->healthy()) {
-      bool result;
-      RETURN_NOT_OK(env_->IsOnExtFilesystem(data_dir, &result));
-      if (result) {
-        fs_type = DataDirFsType::EXT;
-      } else {
-        RETURN_NOT_OK(env_->IsOnXfsFilesystem(data_dir, &result));
+      bool result = false;
+      Status fs_check = env_->IsOnExtFilesystem(data_dir, &result);
+      if (fs_check.ok()) {
         if (result) {
-          fs_type = DataDirFsType::XFS;
+          fs_type = DataDirFsType::EXT;
+        } else {
+          fs_check = env_->IsOnXfsFilesystem(data_dir, &result);
+          if (fs_check.ok() && result) {
+            fs_type = DataDirFsType::XFS;
+          }
         }
       }
+      // If we hit a disk error, consider the directory failed.
+      if (PREDICT_FALSE(fs_check.IsDiskFailure())) {
+        instance->SetInstanceFailed(fs_check.CloneAndPrepend("failed to check FS type"));
+      } else {
+        RETURN_NOT_OK(fs_check);
+      }
     }
 
     unique_ptr<DataDir> dd(new DataDir(
         env_, metrics_.get(), fs_type, data_dir, std::move(instance),
         unique_ptr<ThreadPool>(pool.release())));
     dds.emplace_back(std::move(dd));
-    i++;
   }
 
   // Use the per-dir thread pools to delete temporary files in parallel.
@@ -755,110 +808,11 @@ Status DataDirManager::Open() {
     dd->WaitOnClosures();
   }
 
-  // Build in-memory maps of on-disk state.
-  UuidByRootMap uuid_by_root;
-  UuidByUuidIndexMap uuid_by_idx;
-  UuidIndexByUuidMap idx_by_uuid;
-  UuidIndexMap dd_by_uuid_idx;
-  ReverseUuidIndexMap uuid_idx_by_dd;
-  TabletsByUuidIndexMap tablets_by_uuid_idx_map;
-  FailedDataDirSet failed_data_dirs;
-
-  const auto insert_to_maps = [&] (int idx, string uuid, DataDir* dd) {
-    InsertOrDie(&uuid_by_root, DirName(dd->dir()), uuid);
-    InsertOrDie(&uuid_by_idx, idx, uuid);
-    InsertOrDie(&idx_by_uuid, uuid, idx);
-    InsertOrDie(&dd_by_uuid_idx, idx, dd);
-    InsertOrDie(&uuid_idx_by_dd, dd, idx);
-    InsertOrDie(&tablets_by_uuid_idx_map, idx, {});
-  };
+  RETURN_NOT_OK(PopulateDirectoryMaps(dds));
+  data_dirs_ = std::move(dds);
 
-  if (opts_.consistency_check != ConsistencyCheckBehavior::IGNORE_INCONSISTENCY) {
-    // If we're not in IGNORE_INCONSISTENCY mode, we're guaranteed that the
-    // healthy instances match from the above integrity check, so we can assign
-    // each healthy directory a UUID in accordance with its instance file.
-    //
-    // A directory may not have been assigned a UUID because its instance file
-    // could not be read, in which case, we track it and assign a UUID to it
-    // later if we can.
-    vector<DataDir*> unassigned_dirs;
-    int first_healthy = -1;
-    for (int dir = 0; dir < dds.size(); dir++) {
-      const auto& dd = dds[dir];
-      if (PREDICT_FALSE(!dd->instance()->healthy())) {
-        // Keep track of failed directories so we can assign them UUIDs later.
-        unassigned_dirs.push_back(dd.get());
-        continue;
-      }
-      if (first_healthy == -1) {
-        first_healthy = dir;
-      }
-      const PathSetPB& path_set = dd->instance()->metadata()->path_set();
-      int idx = -1;
-      for (int i = 0; i < path_set.all_uuids_size(); i++) {
-        if (path_set.uuid() == path_set.all_uuids(i)) {
-          idx = i;
-          break;
-        }
-      }
-      if (idx == -1) {
-        return Status::IOError(Substitute(
-            "corrupt path set for data directory $0: uuid $1 not found in path set",
-            dd->dir(), path_set.uuid()));
-      }
-      if (idx > kMaxDataDirs) {
-        return Status::NotSupported(
-            Substitute("block manager supports a maximum of $0 paths", kMaxDataDirs));
-      }
-      insert_to_maps(idx, path_set.uuid(), dd.get());
-    }
-    CHECK_NE(first_healthy, -1); // Guaranteed by LoadInstances().
-
-    // If the uuid index was not assigned, assign it to a failed directory. Use
-    // the path set from the first healthy instance.
-    PathSetPB path_set = dds[first_healthy]->instance()->metadata()->path_set();
-    int failed_dir_idx = 0;
-    for (int uuid_idx = 0; uuid_idx < path_set.all_uuids_size(); uuid_idx++) {
-      if (!ContainsKey(uuid_by_idx, uuid_idx)) {
-        const string& unassigned_uuid = path_set.all_uuids(uuid_idx);
-        insert_to_maps(uuid_idx, unassigned_uuid, unassigned_dirs[failed_dir_idx]);
-
-        // Record the directory as failed.
-        if (metrics_) {
-          metrics_->data_dirs_failed->IncrementBy(1);
-        }
-        InsertOrDie(&failed_data_dirs, uuid_idx);
-        failed_dir_idx++;
-      }
-    }
-    CHECK_EQ(unassigned_dirs.size(), failed_dir_idx);
-  } else {
-    // If we are in IGNORE_INCONSISTENCY mode, all bets are off. The most we
-    // can do is make a best effort assignment of data dirs to UUIDs based on
-    // the ones that are healthy, and for the sake of completeness, assign
-    // artificial UUIDs to the unhealthy ones.
-    for (int dir = 0; dir < dds.size(); dir++) {
-      DataDir* dd = dds[dir].get();
-      if (dd->instance()->healthy()) {
-        insert_to_maps(dir, dd->instance()->metadata()->path_set().uuid(), dd);
-      } else {
-        insert_to_maps(dir, Substitute("<unknown uuid $0>", dir), dd);
-        InsertOrDie(&failed_data_dirs, dir);
-      }
-    }
-  }
-
-  data_dirs_.swap(dds);
-  uuid_by_idx_.swap(uuid_by_idx);
-  idx_by_uuid_.swap(idx_by_uuid);
-  data_dir_by_uuid_idx_.swap(dd_by_uuid_idx);
-  uuid_idx_by_data_dir_.swap(uuid_idx_by_dd);
-  tablets_by_uuid_idx_map_.swap(tablets_by_uuid_idx_map);
-  failed_data_dirs_.swap(failed_data_dirs);
-  uuid_by_root_.swap(uuid_by_root);
-
-  // From this point onwards, the above in-memory maps must be consistent with
-  // the main path set.
+  // From this point onwards, the in-memory maps are the source of truth about
+  // the state of each data dir.
 
   // Initialize the 'fullness' status of the data directories.
   for (const auto& dd : data_dirs_) {
@@ -876,7 +830,6 @@ Status DataDirManager::Open() {
       return refresh_status;
     }
   }
-
   return Status::OK();
 }
 
diff --git a/src/kudu/fs/data_dirs.h b/src/kudu/fs/data_dirs.h
index 0082f99..a9188ed 100644
--- a/src/kudu/fs/data_dirs.h
+++ b/src/kudu/fs/data_dirs.h
@@ -23,7 +23,6 @@
 #include <set>
 #include <string>
 #include <unordered_map>
-#include <utility>
 #include <vector>
 
 #include <gtest/gtest_prod.h>
@@ -115,19 +114,20 @@ enum class DataDirFsType {
   OTHER
 };
 
-// Defines the behavior of the consistency checks performed when the directory
-// manager is opened.
-enum class ConsistencyCheckBehavior {
-  // If the data directories don't match the on-disk path sets, fail.
-  ENFORCE_CONSISTENCY,
-
+// Defines the behavior when opening a directory manager that has an
+// inconsistent or incomplete set of instance files.
+enum UpdateInstanceBehavior {
   // If the data directories don't match the on-disk path sets, update the
-  // on-disk data to match. The directory manager must not be read-only.
-  UPDATE_ON_DISK,
+  // on-disk data to match if not in read-only mode.
+  UPDATE_AND_IGNORE_FAILURES,
+
+  // Like UPDATE_AND_IGNORE_FAILURES, but will return an error if any of the updates to the
+  // on-disk files fail.
+  UPDATE_AND_ERROR_ON_FAILURE,
 
   // If the data directories don't match the on-disk path sets, continue
   // without updating the on-disk data.
-  IGNORE_INCONSISTENCY
+  DONT_UPDATE
 };
 
 struct DataDirMetrics {
@@ -236,11 +236,11 @@ struct DataDirManagerOptions {
   // Defaults to false.
   bool read_only;
 
-  // The behavior to use when comparing the provided data directories to the
-  // on-disk path sets.
+  // Whether to update the on-disk instances when opening directories if
+  // inconsistencies are detected.
   //
-  // Defaults to ENFORCE_CONSISTENCY.
-  ConsistencyCheckBehavior consistency_check;
+  // Defaults to UPDATE_AND_IGNORE_FAILURES.
+  UpdateInstanceBehavior update_instances;
 };
 
 // Encapsulates knowledge of data directory management on behalf of block
@@ -419,50 +419,70 @@ class DataDirManager {
                  DataDirManagerOptions opts,
                  CanonicalizedRootsList canonicalized_data_roots);
 
-  // Initializes the data directories on disk.
+  // Initializes the data directories on disk. Returns an error if initialized
+  // directories already exist.
   //
-  // Returns an error if initialized directories already exist, or if any of
-  // the directories experience a disk failure.
+  // Note: this doesn't initialize any in-memory state for the directory
+  // manager.
   Status Create();
 
-  // Opens existing data roots from disk and indexes the files found.
+  // Opens existing instance files from disk and indexes the files found.
   //
   // Returns an error if the number of on-disk directories found exceeds the
-  // max allowed, if locks need to be acquired and cannot be, or if the
-  // metadata directory (i.e. the first one) fails to load.
+  // max allowed, if locks need to be acquired and cannot be, or if there are
+  // no healthy directories.
+  //
+  // If appropriate, this will create any missing directories and rewrite
+  // existing instance files to be consistent with each other.
   Status Open();
 
-  // Loads the instance files for each data directory.
+  // Populates the maps to index the given directories.
+  Status PopulateDirectoryMaps(const std::vector<std::unique_ptr<DataDir>>& dds);
+
+  // Loads the instance files for each directory root.
   //
-  // On success, 'loaded_instances' contains loaded instance objects. It also
-  // includes instance files that failed to load because they were missing or
-  // because of a disk failure; they are still considered "loaded" and are
-  // labeled unhealthy internally.
+  // On success, 'instance_files' contains instance objects, including those
+  // that failed to load because they were missing or because of a disk
+  // error; they are still considered "loaded" and are labeled unhealthy
+  // internally. 'has_existing_instances' is set to true if any of the instance
+  // files are healthy.
   //
   // Returns an error if an instance file fails in an irreconcileable way (e.g.
-  // the file is locked), or if none of the instance files are healthy.
+  // the file is locked).
   Status LoadInstances(
-      std::vector<std::unique_ptr<PathInstanceMetadataFile>>* loaded_instances);
+      std::vector<std::unique_ptr<PathInstanceMetadataFile>>* instance_files,
+      bool* has_existing_instances);
 
-  // Initializes new data directories specified by 'root_uuid_pairs_to_create'
-  // and updates the on-disk instance files of data directories specified by
-  // 'instances_to_update' using the contents of 'all_uuids', skipping any
-  // unhealthy instance files.
+  // Takes the set of instance files, does some basic verification on them,
+  // creates any that don't exist on disk, and updates any that have a
+  // different set of UUIDs stored than the expected set.
   //
-  // Returns an error if any disk operations fail.
-  Status CreateNewDataDirectoriesAndUpdateInstances(
-      std::vector<std::pair<std::string, std::string>> root_uuid_pairs_to_create,
-      std::vector<std::unique_ptr<PathInstanceMetadataFile>> instances_to_update,
-      std::vector<std::string> all_uuids);
+  // Returns an error if there is a configuration error, e.g. if the existing
+  // instances believe there should be a different block size.
+  //
+  // If in UPDATE_AND_IGNORE_FAILURES mode, an error is not returned in the event of a disk
+  // error. Instead, it is up to the caller to reload the instance files and
+  // proceed if healthy enough.
+  //
+  // If in UPDATE_AND_ERROR_ON_FAILURE mode, a failure to update instances will
+  // surface as an error.
+  Status CreateNewDirectoriesAndUpdateInstances(
+      std::vector<std::unique_ptr<PathInstanceMetadataFile>> instances);
 
   // Updates the on-disk instance files specified by 'instances_to_update'
+  // (presumably those whose 'all_uuids' field doesn't match 'new_all_uuids')
   // using the contents of 'new_all_uuids', skipping any unhealthy instance
   // files.
   //
-  // Returns an error if any disk operations fail.
-  Status UpdateInstances(
-      std::vector<std::unique_ptr<PathInstanceMetadataFile>> instances_to_update,
-      std::vector<std::string> new_all_uuids);
+  // If in UPDATE_AND_IGNORE_FAILURES mode, this is best effort. If any of the instance
+  // updates fail (e.g. due to a disk error) in this mode, this will log a
+  // warning about the failed updates and return OK.
+  //
+  // If in UPDATE_AND_ERROR_ON_FAILURE mode, any failure will immediately attempt
+  // to clean up any altered state and return with an error.
+  Status UpdateHealthyInstances(
+      const std::vector<std::unique_ptr<PathInstanceMetadataFile>>& instances_to_update,
+      const std::set<std::string>& new_all_uuids);
 
   // Returns a random directory in the data dir group specified in 'opts',
   // giving preference to those with more free space. If there is no room in
diff --git a/src/kudu/fs/error_manager.h b/src/kudu/fs/error_manager.h
index 0ffd636..936c91b 100644
--- a/src/kudu/fs/error_manager.h
+++ b/src/kudu/fs/error_manager.h
@@ -149,7 +149,7 @@ class FsErrorManager {
   // Runs the error notification callback with the UUID of 'dir'.
   void RunErrorNotificationCb(ErrorHandlerType e, const DataDir* dir) const {
     DCHECK_EQ(e, ErrorHandlerType::DISK_ERROR);
-    RunErrorNotificationCb(e, dir->instance()->metadata()->path_set().uuid());
+    RunErrorNotificationCb(e, dir->instance()->uuid());
   }
 
  private:
diff --git a/src/kudu/fs/fs_manager-test.cc b/src/kudu/fs/fs_manager-test.cc
index 59beb4c..6516228 100644
--- a/src/kudu/fs/fs_manager-test.cc
+++ b/src/kudu/fs/fs_manager-test.cc
@@ -15,28 +15,32 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "kudu/fs/fs_manager.h"
+
 #include <sys/stat.h>
 #include <unistd.h>
 
 #include <cstdint>
+#include <initializer_list>
 #include <iostream>
 #include <iterator>
 #include <memory>
 #include <set>
 #include <string>
+#include <unordered_map>
 #include <unordered_set>
 #include <utility>
 #include <vector>
 
 #include <gflags/gflags.h>
-#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 #include <glog/stl_logging.h>
 #include <gtest/gtest.h>
 
 #include "kudu/fs/block_manager.h"
+#include "kudu/fs/block_manager_util.h"
 #include "kudu/fs/data_dirs.h"
-#include "kudu/fs/fs_manager.h"
+#include "kudu/fs/fs.pb.h"
 #include "kudu/fs/fs_report.h"
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/port.h"
@@ -49,16 +53,19 @@
 #include "kudu/util/flags.h"
 #include "kudu/util/oid_generator.h"
 #include "kudu/util/path_util.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/random.h"
 #include "kudu/util/slice.h"
 #include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
-using kudu::fs::ConsistencyCheckBehavior;
+using kudu::pb_util::ReadPBContainerFromPath;
+using kudu::pb_util::SecureDebugString;
 using std::shared_ptr;
 using std::string;
 using std::unique_ptr;
+using std::unordered_map;
 using std::unordered_set;
 using std::vector;
 using strings::Substitute;
@@ -71,6 +78,7 @@ DECLARE_string(env_inject_lock_failure_globs);
 DECLARE_string(umask);
 
 namespace kudu {
+namespace fs {
 
 class FsManagerTestBase : public KuduTest {
  public:
@@ -107,14 +115,14 @@ class FsManagerTestBase : public KuduTest {
     DCHECK_LT(data.size(), sizeof(buffer));
 
     // Test Write
-    unique_ptr<fs::WritableBlock> writer;
+    unique_ptr<WritableBlock> writer;
     ASSERT_OK(fs_manager()->CreateNewBlock({}, &writer));
     ASSERT_OK(writer->Append(data));
     ASSERT_OK(writer->Close());
 
     // Test Read
     Slice result(buffer, data.size());
-    unique_ptr<fs::ReadableBlock> reader;
+    unique_ptr<ReadableBlock> reader;
     ASSERT_OK(fs_manager()->OpenBlock(writer->id(), &reader));
     ASSERT_OK(reader->Read(0, result));
     ASSERT_EQ(0, result.compare(data));
@@ -312,7 +320,6 @@ TEST_F(FsManagerTestBase, TestMetadataDirInDataRoot) {
   // Adding a data dir to the front of the FS root list (i.e. such that the
   // metadata root is no longer at the front) will prevent Kudu from starting.
   opts.data_roots = { GetTestPath("data2"), GetTestPath("data1") };
-  opts.consistency_check = ConsistencyCheckBehavior::UPDATE_ON_DISK;
   ReinitFsManagerWithOpts(opts);
   Status s = fs_manager()->Open();
   ASSERT_STR_CONTAINS(s.ToString(), "could not verify required directory");
@@ -418,6 +425,35 @@ TEST_F(FsManagerTestBase, TestCreateWithFailedDirs) {
                                    "failed to canonicalize");
 }
 
+// Test that if an operator tries to copy an instance file, Kudu will refuse to
+// start up.
+TEST_F(FsManagerTestBase, TestOpenWithDuplicateInstanceFiles) {
+  // First, make a copy of some instance files.
+  WritableFileOptions wr_opts;
+  wr_opts.mode = Env::MUST_CREATE;
+  const string duplicate_test_root = GetTestPath("fs_dup");
+  ASSERT_OK(env_->CreateDir(duplicate_test_root));
+  const string duplicate_instance = JoinPathSegments(
+      duplicate_test_root, FsManager::kInstanceMetadataFileName);
+  ASSERT_OK(env_util::CopyFile(env_, fs_manager()->GetInstanceMetadataPath(fs_root_),
+                               duplicate_instance, wr_opts));
+
+  // Make a copy of the per-directory instance file.
+  const string duplicate_test_dir = JoinPathSegments(duplicate_test_root, kDataDirName);
+  ASSERT_OK(env_->CreateDir(duplicate_test_dir));
+  const string duplicate_dir_instance = JoinPathSegments(
+      duplicate_test_dir, kInstanceMetadataFileName);
+  ASSERT_OK(env_util::CopyFile(env_,
+        fs_manager()->dd_manager()->FindDataDirByUuidIndex(0)->instance()->path(),
+        duplicate_dir_instance, wr_opts));
+
+  // This is disallowed, as each directory should have its own unique UUID.
+  ReinitFsManagerWithPaths(fs_root_, { fs_root_, duplicate_test_root });
+  Status s = fs_manager()->Open();
+  ASSERT_STR_CONTAINS(s.ToString(), "instance files contain duplicate UUIDs") << s.ToString();
+  ASSERT_TRUE(s.IsInvalidArgument()) << s.ToString();
+}
+
 TEST_F(FsManagerTestBase, TestOpenWithNoBlockManagerInstances) {
   // Open a healthy FS layout, sharing the WAL directory with a data directory.
   const string wal_path = GetTestPath("wals");
@@ -429,14 +465,14 @@ TEST_F(FsManagerTestBase, TestOpenWithNoBlockManagerInstances) {
   ASSERT_OK(fs_manager()->Open());
 
   // Now try moving the data directory out of WAL directory.
-  // Even if we're not enforcing consistency, we must be able to find an
-  // existing block manager instance to open the FsManager successfully.
-  for (auto check_behavior : { ConsistencyCheckBehavior::IGNORE_INCONSISTENCY,
-                               ConsistencyCheckBehavior::UPDATE_ON_DISK }) {
+  // We must be able to find an existing block manager instance to open the
+  // FsManager successfully.
+  for (auto check_behavior : { UpdateInstanceBehavior::DONT_UPDATE,
+                               UpdateInstanceBehavior::UPDATE_AND_IGNORE_FAILURES }) {
     FsManagerOpts new_opts;
     new_opts.wal_root = wal_path;
     new_opts.data_roots = { GetTestPath("data") };
-    new_opts.consistency_check = check_behavior;
+    new_opts.update_instances = check_behavior;
     ReinitFsManagerWithOpts(new_opts);
     Status s = fs_manager()->Open();
     ASSERT_STR_CONTAINS(s.ToString(), "no healthy data directories found");
@@ -447,7 +483,7 @@ TEST_F(FsManagerTestBase, TestOpenWithNoBlockManagerInstances) {
     ReinitFsManagerWithOpts(std::move(new_opts));
     s = fs_manager()->Open();
     if (block_manager_type == "file" &&
-        check_behavior == ConsistencyCheckBehavior::UPDATE_ON_DISK) {
+        check_behavior == UpdateInstanceBehavior::UPDATE_AND_IGNORE_FAILURES) {
       ASSERT_TRUE(s.IsInvalidArgument()) << s.ToString();
       ASSERT_STR_CONTAINS(s.ToString(),
           "file block manager may not add or remove data directories");
@@ -466,7 +502,6 @@ TEST_F(FsManagerTestBase, TestOpenWithUnhealthyDataDir) {
   FsManagerOpts opts;
   opts.wal_root = fs_root_;
   opts.data_roots = { fs_root_, new_root };
-  opts.consistency_check = ConsistencyCheckBehavior::UPDATE_ON_DISK;
   ReinitFsManagerWithOpts(opts);
   string new_root_uuid;
   auto s = fs_manager()->Open();
@@ -484,7 +519,6 @@ TEST_F(FsManagerTestBase, TestOpenWithUnhealthyDataDir) {
   // and should list one as failed.
   FLAGS_env_inject_eio_globs = JoinPathSegments(new_root, "**");
   FLAGS_env_inject_eio = 1.0;
-  opts.consistency_check = ConsistencyCheckBehavior::ENFORCE_CONSISTENCY;
   ReinitFsManagerWithOpts(opts);
   s = fs_manager()->Open();
   if (opts.block_manager_type == "file") {
@@ -497,25 +531,16 @@ TEST_F(FsManagerTestBase, TestOpenWithUnhealthyDataDir) {
   ASSERT_OK(s);
   ASSERT_EQ(1, fs_manager()->dd_manager()->GetFailedDataDirs().size());
 
-  // Now remove the new directory on disk. Similarly, Kudu should have no
-  // problem starting up and it should list one failed data directory.
+  // Now remove the new directory from disk. Kudu should start up with the
+  // empty disk and attempt to use it. Upon opening the FS layout, we should
+  // see no failed directories.
   FLAGS_env_inject_eio = 0;
   ASSERT_OK(env_->DeleteRecursively(new_root));
   ReinitFsManagerWithOpts(opts);
   ASSERT_OK(fs_manager()->Open());
-  ASSERT_EQ(1, fs_manager()->dd_manager()->GetFailedDataDirs().size());
-
-  // Now let's simulate the operator replacing the drive. The update tool will
-  // be run and the new directory, even at the same mountpoint, will be
-  // assigned a new UUID.
-  //
-  // At this point, our remaining healthy instance file should know about two
-  // data directories. Kudu should detect one missing and create a new one.
-  // Let's update and ensure we get a new UUID.
-  opts.consistency_check = ConsistencyCheckBehavior::UPDATE_ON_DISK;
-  ReinitFsManagerWithOpts(opts);
-  ASSERT_OK(fs_manager()->Open());
   ASSERT_EQ(0, fs_manager()->dd_manager()->GetFailedDataDirs().size());
+
+  // Even at the same mountpoint, the directory will be assigned a new UUID.
   string new_root_uuid_post_update;
   ASSERT_TRUE(fs_manager()->dd_manager()->FindUuidByRoot(new_root, &new_root_uuid_post_update));
   ASSERT_NE(new_root_uuid, new_root_uuid_post_update);
@@ -524,7 +549,6 @@ TEST_F(FsManagerTestBase, TestOpenWithUnhealthyDataDir) {
   // complaining it couldn't find any healthy data directories.
   FLAGS_env_inject_eio_globs = JoinStrings(JoinPathSegmentsV(opts.data_roots, "**"), ",");
   FLAGS_env_inject_eio = 1.0;
-  opts.consistency_check = ConsistencyCheckBehavior::ENFORCE_CONSISTENCY;
   ReinitFsManagerWithOpts(opts);
   s = fs_manager()->Open();
   ASSERT_TRUE(s.IsNotFound()) << s.ToString();
@@ -581,16 +605,7 @@ TEST_F(FsManagerTestBase, TestOpenWithCanonicalizationFailure) {
   // simulates the mountpoint disappearing.
   ASSERT_OK(env_->DeleteRecursively(dir2));
   ReinitFsManagerWithOpts(opts);
-  ASSERT_OK(fs_manager()->Open());
-  ASSERT_EQ(1, fs_manager()->dd_manager()->GetFailedDataDirs().size());
-
-  // In both of the above failures, the appropriate steps would be to run the
-  // update tool after ensuring the bad mountpoint is replaced with a healthy
-  // one. Until that happens, we won't be able to update the data dirs.
-  opts.consistency_check = ConsistencyCheckBehavior::UPDATE_ON_DISK;
-  ReinitFsManagerWithOpts(opts);
-
-  const auto s = fs_manager()->Open();
+  Status s = fs_manager()->Open();
   if (opts.block_manager_type == "file") {
     ASSERT_TRUE(s.IsInvalidArgument()) << s.ToString();
     ASSERT_STR_CONTAINS(s.ToString(),
@@ -598,9 +613,8 @@ TEST_F(FsManagerTestBase, TestOpenWithCanonicalizationFailure) {
     LOG(INFO) << "Skipping the rest of test, file block manager not supported";
     return;
   }
-
-  ASSERT_TRUE(s.IsNotFound()) << s.ToString();
-  ASSERT_STR_CONTAINS(s.ToString(), "could not add new data directories");
+  ASSERT_OK(s);
+  ASSERT_EQ(1, fs_manager()->dd_manager()->GetFailedDataDirs().size());
 
   // Let's try that again, but with the appropriate mountpoint/directory.
   ASSERT_OK(env_->CreateDir(dir2));
@@ -741,35 +755,24 @@ TEST_F(FsManagerTestBase, TestAddRemoveDataDirs) {
     return;
   }
 
-  // Try to open with a new data dir in the list to be opened; this should fail.
+  // Try to open with a new data dir in the list to be opened; Kudu should
+  // allow for this to happen, creating the necessary data directory.
   const string new_path1 = GetTestPath("new_path1");
   FsManagerOpts opts;
   opts.wal_root = fs_root_;
   opts.data_roots = { fs_root_, new_path1 };
   ReinitFsManagerWithOpts(opts);
-  Status s = fs_manager()->Open();
-  ASSERT_TRUE(s.IsIOError()) << s.ToString();
-  ASSERT_STR_CONTAINS(s.ToString(), "2 data directories provided, but expected 1");
-
-  // This time allow new data directories to be created.
-  opts.consistency_check = ConsistencyCheckBehavior::UPDATE_ON_DISK;
-  ReinitFsManagerWithOpts(opts);
   ASSERT_OK(fs_manager()->Open());
   ASSERT_EQ(2, fs_manager()->dd_manager()->GetDataDirs().size());
+  ASSERT_EQ(0, fs_manager()->dd_manager()->GetFailedDataDirs().size());
 
-  // Try to open with a data dir removed; this should fail.
+  // Try to open with a data dir removed; this should succeed, and Kudu should
+  // open with only a single data directory.
   opts.data_roots = { fs_root_ };
-  opts.consistency_check = ConsistencyCheckBehavior::ENFORCE_CONSISTENCY;
-  ReinitFsManagerWithOpts(opts);
-  s = fs_manager()->Open();
-  ASSERT_TRUE(s.IsIOError());
-  ASSERT_STR_CONTAINS(s.ToString(), "could not verify integrity of files");
-
-  // This time allow data directories to be removed.
-  opts.consistency_check = ConsistencyCheckBehavior::UPDATE_ON_DISK;
   ReinitFsManagerWithOpts(opts);
   ASSERT_OK(fs_manager()->Open());
   ASSERT_EQ(1, fs_manager()->dd_manager()->GetDataDirs().size());
+  ASSERT_EQ(0, fs_manager()->dd_manager()->GetFailedDataDirs().size());
 
   // We should be able to add new directories anywhere in the list.
   const string new_path2 = GetTestPath("new_path2");
@@ -781,23 +784,136 @@ TEST_F(FsManagerTestBase, TestAddRemoveDataDirs) {
 
   // Open the FS layout with an existing, failed data dir; this should be fine,
   // but should report a single failed directory.
-  FLAGS_crash_on_eio = false;
   FLAGS_env_inject_eio = 1.0;
   FLAGS_env_inject_eio_globs = JoinPathSegments(new_path2, "**");
-  opts.consistency_check = ConsistencyCheckBehavior::ENFORCE_CONSISTENCY;
   ReinitFsManagerWithOpts(opts);
   ASSERT_OK(fs_manager()->Open());
   ASSERT_EQ(1, fs_manager()->dd_manager()->GetFailedDataDirs().size());
+}
 
-  // Now try to add a new data dir with an existing, failed data dir; this
-  // should fail.
-  const string new_path3 = GetTestPath("new_path3");
-  opts.consistency_check = ConsistencyCheckBehavior::UPDATE_ON_DISK;
-  opts.data_roots = { fs_root_, new_path2, new_path3 };
+TEST_F(FsManagerTestBase, TestEIOWhileChangingDirs) {
+  if (FLAGS_block_manager == "file") {
+    LOG(INFO) << "The file block manager doesn't support updating directories";
+    return;
+  }
+  const string kTestPathBase = GetTestPath("testpath");
+  const int kMaxDirs = 10;
+  vector<string> all_dirs;
+  for (int i = 0; i < kMaxDirs; i++) {
+    const string dir = Substitute("$0$1", kTestPathBase, i);
+    all_dirs.emplace_back(dir);
+    ASSERT_OK(env_->CreateDir(dir));
+  }
+  FsManagerOpts opts;
+  opts.wal_root = all_dirs[0];
   ReinitFsManagerWithOpts(opts);
-  s = fs_manager()->Open();
-  ASSERT_TRUE(s.IsIOError());
-  ASSERT_STR_CONTAINS(s.ToString(), "found failed data directory");
+  ASSERT_OK(fs_manager()->CreateInitialFileSystemLayout());
+  // Inject failures into the directories. This shouldn't prevent any updating
+  // of instances, and it definitely shouldn't affect startup.
+  vector<string> all_dirs_but_first(++all_dirs.begin(), all_dirs.end());
+  FLAGS_env_inject_eio_globs = JoinStrings(JoinPathSegmentsV(all_dirs_but_first, "**"), ",");
+  FLAGS_env_inject_eio = 0.1;
+  for (int i = 1; i <= kMaxDirs; i++) {
+    // Use an increasing number of dirs so we build up to using all of them.
+    opts.data_roots = all_dirs;
+    opts.data_roots.resize(i);
+    ReinitFsManagerWithOpts(opts);
+    ASSERT_OK(fs_manager()->Open());
+  }
+}
+
+// Unlike the case where we're opening the FsManager for deployment, when
+// running the update_dirs tool (i.e. UPDATE_AND_ERROR_ON_FAILURE mode), Kudu
+// should fail and return an error in the event of a disk failure. When that
+// happens, we should ensure that the our failures to update get rolled back.
+TEST_F(FsManagerTestBase, TestEIOWhileRunningUpdateDirsTool) {
+  if (FLAGS_block_manager == "file") {
+    LOG(INFO) << "The file block manager doesn't support updating directories";
+    return;
+  }
+  const string kTestPathBase = GetTestPath("testpath");
+  // Helper to get a new root.
+  auto create_root = [&] (int i) {
+    const string dir = Substitute("$0$1", kTestPathBase, i);
+    CHECK_OK(env_->CreateDir(dir));
+    return dir;
+  };
+
+  // Helper to collect the contents of the InstanceMetadataPB and
+  // PathInstanceMetadtaPBs we expect to see in 'data_roots'. We'll read the
+  // contents of each instance file from disk and compare them before and after
+  // a botched update of the FsManager.
+  auto get_added_instance_files = [&] (const vector<string>& data_roots,
+                                       unordered_map<string, string>* instance_names_to_contents) {
+    unordered_map<string, string> instances;
+    instance_names_to_contents->clear();
+    // Skip the first root, since we'll be injecting errors into the first
+    // directory, meaning we don't have any guarantees on what that directory's
+    // instance files will be.
+    for (int i = 1; i < data_roots.size(); i++) {
+      const auto& root = data_roots[i];
+      // Collect the contents of the InstanceMetadataPB objects.
+      const auto instance_path = JoinPathSegments(root, FsManager::kInstanceMetadataFileName);
+      unique_ptr<InstanceMetadataPB> pb(new InstanceMetadataPB);
+      Status s = ReadPBContainerFromPath(env_, instance_path, pb.get());
+      if (s.IsNotFound()) {
+        InsertOrDie(&instances, instance_path, "");
+      } else {
+        RETURN_NOT_OK(s);
+        InsertOrDie(&instances, instance_path, SecureDebugString(*pb));
+      }
+
+      // Collect the contents of the PathInstanceMetadataPB objects.
+      unique_ptr<PathInstanceMetadataPB> bmi_pb(new PathInstanceMetadataPB);
+      const auto block_manager_instance = JoinPathSegments(
+          JoinPathSegments(root, kDataDirName), kInstanceMetadataFileName);
+      s = ReadPBContainerFromPath(env_, block_manager_instance, bmi_pb.get());
+      if (s.IsNotFound()) {
+        InsertOrDie(&instances, block_manager_instance, "");
+      } else {
+        RETURN_NOT_OK(s);
+        InsertOrDie(&instances, block_manager_instance, SecureDebugString(*bmi_pb));
+      }
+    }
+    *instance_names_to_contents = std::move(instances);
+    return Status::OK();
+  };
+
+  vector<string> all_roots = { create_root(0) };
+  FsManagerOpts opts;
+  opts.wal_root = all_roots[0];
+  opts.data_roots = all_roots;
+  ReinitFsManagerWithOpts(opts);
+  ASSERT_OK(fs_manager()->CreateInitialFileSystemLayout());
+
+  // Start injecting failures into the first directory as we try updating.
+  FLAGS_env_inject_eio_globs = JoinPathSegments(all_roots[0], "**");
+  FLAGS_env_inject_eio = 0.05;
+  unordered_map<string, string> instance_files_before_update;
+  ASSERT_EVENTUALLY([&] {
+    {
+      // First, collect the contents of our instance files so we can compare
+      // against their contents after failing.
+      google::FlagSaver saver;
+      FLAGS_env_inject_eio = 0;
+      all_roots.emplace_back(create_root(all_roots.size()));
+      ASSERT_OK(get_added_instance_files(all_roots, &instance_files_before_update));
+    }
+    // Then try to update the directories. We'll keep trying until we fail.
+    opts.update_instances = UpdateInstanceBehavior::UPDATE_AND_ERROR_ON_FAILURE;
+    opts.data_roots = all_roots;
+    ReinitFsManagerWithOpts(opts);
+    Status s = fs_manager()->Open();
+    ASSERT_FALSE(s.ok());
+  });
+
+  // Now that we've failed to add a new directory, let's compare the contents
+  // of the instnace files to ensure that they're unchanged from the point
+  // right before the update.
+  FLAGS_env_inject_eio = 0;
+  unordered_map<string, string> instance_files_after_update;
+  ASSERT_OK(get_added_instance_files(all_roots, &instance_files_after_update));
+  ASSERT_EQ(instance_files_before_update, instance_files_after_update);
 }
 
 TEST_F(FsManagerTestBase, TestReAddRemovedDataDir) {
@@ -808,16 +924,32 @@ TEST_F(FsManagerTestBase, TestReAddRemovedDataDir) {
 
   // Add a new data directory, remove it, and add it back.
   const string new_path1 = GetTestPath("new_path1");
+  FsManagerOpts opts;
+  opts.wal_root = fs_root_;
+  unordered_map<string, string> path_to_uuid;
   for (const auto& data_roots : vector<vector<string>>({{ fs_root_, new_path1 },
                                                         { fs_root_ },
                                                         { fs_root_, new_path1 }})) {
-    FsManagerOpts opts;
-    opts.wal_root = fs_root_;
     opts.data_roots = data_roots;
-    opts.consistency_check = ConsistencyCheckBehavior::UPDATE_ON_DISK;
-    ReinitFsManagerWithOpts(std::move(opts));
+    ReinitFsManagerWithOpts(opts);
     ASSERT_OK(fs_manager()->Open());
-    ASSERT_EQ(data_roots.size(), fs_manager()->dd_manager()->GetDataDirs().size());
+    DataDirManager* dd_manager = fs_manager()->dd_manager();
+    ASSERT_EQ(data_roots.size(), dd_manager->GetDataDirs().size());
+
+    // Since we haven't deleted any directories or instance files, ensure that
+    // our UUIDs match across startups.
+    for (const auto& data_root : data_roots) {
+      string uuid;
+      ASSERT_TRUE(dd_manager->FindUuidByRoot(data_root, &uuid));
+      string* existing_uuid = FindOrNull(path_to_uuid, data_root);
+      if (existing_uuid) {
+        ASSERT_EQ(*existing_uuid, uuid) <<
+            Substitute("Expected $0 to have UUID $1, got $2",
+                       data_root, *existing_uuid, uuid);
+      } else {
+        InsertOrDie(&path_to_uuid, data_root, uuid);
+      }
+    }
   }
 }
 
@@ -851,7 +983,6 @@ TEST_F(FsManagerTestBase, TestCannotRemoveDataDirServingAsMetadataDir) {
   // absence of a defined metadata root, the FsManager will try looking for it
   // in the wal root (not found), and the first data dir (not found).
   opts.data_roots = { opts.data_roots[1] };
-  opts.consistency_check = ConsistencyCheckBehavior::UPDATE_ON_DISK;
   ReinitFsManagerWithOpts(opts);
   Status s = fs_manager()->Open();
   ASSERT_TRUE(s.IsNotFound());
@@ -869,14 +1000,14 @@ TEST_F(FsManagerTestBase, TestAddRemoveSpeculative) {
   FsManagerOpts opts;
   opts.wal_root = fs_root_;
   opts.data_roots = { fs_root_, new_path1 };
-  opts.consistency_check = ConsistencyCheckBehavior::UPDATE_ON_DISK;
+  opts.update_instances = UpdateInstanceBehavior::UPDATE_AND_IGNORE_FAILURES;
   ReinitFsManagerWithOpts(opts);
   ASSERT_OK(fs_manager()->Open());
   ASSERT_EQ(2, fs_manager()->dd_manager()->GetDataDirs().size());
 
   // Create a 'speculative' FsManager with the second data directory removed.
   opts.data_roots = { fs_root_ };
-  opts.consistency_check = ConsistencyCheckBehavior::IGNORE_INCONSISTENCY;
+  opts.update_instances = UpdateInstanceBehavior::DONT_UPDATE;
   ReinitFsManagerWithOpts(opts);
   ASSERT_OK(fs_manager()->Open());
   ASSERT_EQ(1, fs_manager()->dd_manager()->GetDataDirs().size());
@@ -890,28 +1021,34 @@ TEST_F(FsManagerTestBase, TestAddRemoveSpeculative) {
   ASSERT_EQ(1, fs_manager()->dd_manager()->GetFailedDataDirs().size());
 
   // Neither of those attempts should have changed the on-disk state. Verify
-  // this by retrying all three combinations with consistency checking
-  // re-enabled. Only the two-directory case should pass.
-  opts.consistency_check = ConsistencyCheckBehavior::ENFORCE_CONSISTENCY;
-  for (const auto& data_roots : vector<vector<string>>({{ fs_root_ },
-                                                        { fs_root_, new_path1 },
-                                                        { fs_root_, new_path1, new_path2 }})) {
+  // this by retrying all three combinations again.
+  // With three directories, we should see a failed directory still.
+  vector<vector<string>> data_roots_list = { { fs_root_ },
+                                             { fs_root_, new_path1 },
+                                             { fs_root_, new_path1, new_path2 } };
+  for (const auto& data_roots : data_roots_list) {
     opts.data_roots = data_roots;
     ReinitFsManagerWithOpts(opts);
-    Status s = fs_manager()->Open();
-    if (data_roots.size() == 1) {
-      // The first data directory's path set refers to a data directory that
-      // wasn't in data_roots.
-      ASSERT_TRUE(s.IsIOError()) << s.ToString();
-      ASSERT_STR_CONTAINS(s.ToString(), "could not verify integrity of files");
-    } else if (data_roots.size() == 2) {
-      ASSERT_OK(s);
-      ASSERT_EQ(2, fs_manager()->dd_manager()->GetDataDirs().size());
-    } else {
-      // The third data directory has no instance file.
-      ASSERT_TRUE(s.IsIOError()) << s.ToString();
-      ASSERT_STR_CONTAINS(s.ToString(), "3 data directories provided, but expected 2");
-    }
+    ASSERT_OK(fs_manager()->Open());
+    ASSERT_EQ(data_roots.size() == 3 ? 1 : 0,
+              fs_manager()->dd_manager()->GetFailedDataDirs().size());
+  }
+
+  // When we allow ourselves to update the disk instances, each open will
+  // update the on-disk layout.
+  for (const auto& data_roots : data_roots_list) {
+    opts.update_instances = UpdateInstanceBehavior::UPDATE_AND_IGNORE_FAILURES;
+    opts.data_roots = data_roots;
+    ReinitFsManagerWithOpts(opts);
+    ASSERT_OK(fs_manager()->Open());
+    ASSERT_EQ(0, fs_manager()->dd_manager()->GetFailedDataDirs().size());
+
+    // Since the on-disk state has been updated, we should be able to open the
+    // speculative directory with no issues.
+    opts.update_instances = UpdateInstanceBehavior::DONT_UPDATE;
+    ReinitFsManagerWithOpts(opts);
+    ASSERT_OK(fs_manager()->Open());
+    ASSERT_EQ(0, fs_manager()->dd_manager()->GetFailedDataDirs().size());
   }
 }
 
@@ -960,13 +1097,11 @@ TEST_F(FsManagerTestBase, TestAddRemoveDataDirsFuzz) {
       // This value isn't arbitrary: most attempts fail and only some succeed.
       FLAGS_env_inject_eio = 0.01;
 
-      fs_opts.consistency_check = ConsistencyCheckBehavior::UPDATE_ON_DISK;
       ReinitFsManagerWithOpts(fs_opts);
       update_succeeded = fs_manager()->Open().ok();
     }
 
     // Reopen regardless, to ensure that failures didn't corrupt anything.
-    fs_opts.consistency_check = ConsistencyCheckBehavior::ENFORCE_CONSISTENCY;
     ReinitFsManagerWithOpts(fs_opts);
     Status open_status = fs_manager()->Open();
     if (update_succeeded) {
@@ -1019,9 +1154,9 @@ TEST_F(FsManagerTestBase, TestAddRemoveDataDirsFuzz) {
       ASSERT_TRUE(open_status.IsIOError());
       bool repaired = false;
       for (const auto& root : fs_opts.data_roots) {
-        string data_dir = JoinPathSegments(root, fs::kDataDirName);
+        string data_dir = JoinPathSegments(root, kDataDirName);
         string instance = JoinPathSegments(data_dir,
-                                           fs::kInstanceMetadataFileName);
+                                           kInstanceMetadataFileName);
         ASSERT_TRUE(env_->FileExists(instance));
         string copy = instance + kTmpInfix;
         if (env_->FileExists(copy)) {
@@ -1048,11 +1183,12 @@ TEST_F(FsManagerTestBase, TestAncillaryDirsReported) {
   opts.metadata_root = GetTestPath("metadata");
   ReinitFsManagerWithOpts(opts);
   ASSERT_OK(fs_manager()->CreateInitialFileSystemLayout());
-  fs::FsReport report;
+  FsReport report;
   ASSERT_OK(fs_manager()->Open(&report));
   string report_str = report.ToString();
   ASSERT_STR_CONTAINS(report_str, "wal directory: " + opts.wal_root);
   ASSERT_STR_CONTAINS(report_str, "metadata directory: " + opts.metadata_root);
 }
 
+} // namespace fs
 } // namespace kudu
diff --git a/src/kudu/fs/fs_manager.cc b/src/kudu/fs/fs_manager.cc
index cbd4113..356ceda 100644
--- a/src/kudu/fs/fs_manager.cc
+++ b/src/kudu/fs/fs_manager.cc
@@ -17,8 +17,10 @@
 
 #include "kudu/fs/fs_manager.h"
 
+#include <algorithm>
 #include <cinttypes>
 #include <ctime>
+#include <initializer_list>
 #include <iostream>
 #include <unordered_map>
 #include <unordered_set>
@@ -99,7 +101,6 @@ DEFINE_string(fs_metadata_dir, "",
 TAG_FLAG(fs_metadata_dir, stable);
 
 using kudu::fs::BlockManagerOptions;
-using kudu::fs::ConsistencyCheckBehavior;
 using kudu::fs::CreateBlockOptions;
 using kudu::fs::DataDirManager;
 using kudu::fs::DataDirManagerOptions;
@@ -110,6 +111,7 @@ using kudu::fs::FileBlockManager;
 using kudu::fs::FsReport;
 using kudu::fs::LogBlockManager;
 using kudu::fs::ReadableBlock;
+using kudu::fs::UpdateInstanceBehavior;
 using kudu::fs::WritableBlock;
 using kudu::pb_util::SecureDebugString;
 using std::ostream;
@@ -138,7 +140,7 @@ FsManagerOpts::FsManagerOpts()
     metadata_root(FLAGS_fs_metadata_dir),
     block_manager_type(FLAGS_block_manager),
     read_only(false),
-    consistency_check(ConsistencyCheckBehavior::ENFORCE_CONSISTENCY) {
+    update_instances(UpdateInstanceBehavior::UPDATE_AND_IGNORE_FAILURES) {
   data_roots = strings::Split(FLAGS_fs_data_dirs, ",", strings::SkipEmpty());
 }
 
@@ -147,7 +149,7 @@ FsManagerOpts::FsManagerOpts(const string& root)
     data_roots({ root }),
     block_manager_type(FLAGS_block_manager),
     read_only(false),
-    consistency_check(ConsistencyCheckBehavior::ENFORCE_CONSISTENCY) {}
+    update_instances(UpdateInstanceBehavior::UPDATE_AND_IGNORE_FAILURES) {}
 
 FsManager::FsManager(Env* env, const string& root_path)
   : env_(DCHECK_NOTNULL(env)),
@@ -160,8 +162,8 @@ FsManager::FsManager(Env* env, FsManagerOpts opts)
     opts_(std::move(opts)),
     error_manager_(new FsErrorManager()),
     initted_(false) {
-  DCHECK(opts_.consistency_check != ConsistencyCheckBehavior::UPDATE_ON_DISK ||
-         !opts_.read_only);
+DCHECK(opts_.update_instances == UpdateInstanceBehavior::DONT_UPDATE ||
+       !opts_.read_only) << "FsManager can only be for updated if not in read-only mode";
 }
 
 FsManager::~FsManager() {}
@@ -388,10 +390,18 @@ Status FsManager::Open(FsReport* report) {
   });
 
   // Create any missing roots, if desired.
-  if (opts_.consistency_check == ConsistencyCheckBehavior::UPDATE_ON_DISK) {
-    RETURN_NOT_OK_PREPEND(CreateFileSystemRoots(
-        missing_roots, *metadata_, &created_dirs, &created_files),
-                          "unable to create missing filesystem roots");
+  if (!opts_.read_only &&
+      opts_.update_instances != UpdateInstanceBehavior::DONT_UPDATE) {
+    Status s = CreateFileSystemRoots(
+        missing_roots, *metadata_, &created_dirs, &created_files);
+    static const string kUnableToCreateMsg = "unable to create missing filesystem roots";
+    if (opts_.update_instances == UpdateInstanceBehavior::UPDATE_AND_IGNORE_FAILURES) {
+      // We only warn on error here -- regardless of errors, we might be in
+      // good enough shape to open the DataDirManager.
+      WARN_NOT_OK(s, kUnableToCreateMsg);
+    } else if (opts_.update_instances == UpdateInstanceBehavior::UPDATE_AND_ERROR_ON_FAILURE) {
+      RETURN_NOT_OK_PREPEND(s, kUnableToCreateMsg);
+    }
   }
 
   // Open the directory manager if it has not been opened already.
@@ -400,7 +410,7 @@ Status FsManager::Open(FsReport* report) {
     dm_opts.metric_entity = opts_.metric_entity;
     dm_opts.block_manager_type = opts_.block_manager_type;
     dm_opts.read_only = opts_.read_only;
-    dm_opts.consistency_check = opts_.consistency_check;
+    dm_opts.update_instances = opts_.update_instances;
     LOG_TIMING(INFO, "opening directory manager") {
       RETURN_NOT_OK(DataDirManager::OpenExisting(env_,
           canonicalized_data_fs_roots_, std::move(dm_opts), &dd_manager_));
diff --git a/src/kudu/fs/fs_manager.h b/src/kudu/fs/fs_manager.h
index 434e7d8..6c61580 100644
--- a/src/kudu/fs/fs_manager.h
+++ b/src/kudu/fs/fs_manager.h
@@ -21,6 +21,7 @@
 #include <iosfwd>
 #include <memory>
 #include <string>
+#include <utility>
 #include <vector>
 
 #include <boost/optional/optional.hpp>
@@ -49,6 +50,12 @@ class MemTracker;
 namespace fs {
 
 class BlockManager;
+class FsManagerTestBase_TestDuplicatePaths_Test;
+class FsManagerTestBase_TestEIOWhileRunningUpdateDirsTool_Test;
+class FsManagerTestBase_TestIsolatedMetadataDir_Test;
+class FsManagerTestBase_TestMetadataDirInDataRoot_Test;
+class FsManagerTestBase_TestMetadataDirInWALRoot_Test;
+class FsManagerTestBase_TestOpenWithDuplicateInstanceFiles_Test;
 class ReadableBlock;
 class WritableBlock;
 struct CreateBlockOptions;
@@ -109,10 +116,11 @@ struct FsManagerOpts {
   // Defaults to false.
   bool read_only;
 
-  // The behavior to use when comparing 'data_roots' to the on-disk path sets.
+  // Whether to update the on-disk instances when opening directories if
+  // inconsistencies are detected.
   //
-  // Defaults to ENFORCE_CONSISTENCY.
-  fs::ConsistencyCheckBehavior consistency_check;
+  // Defaults to UPDATE_AND_IGNORE_FAILURES.
+  fs::UpdateInstanceBehavior update_instances;
 };
 
 // FsManager provides helpers to read data and metadata files,
@@ -279,10 +287,12 @@ class FsManager {
   void DumpFileSystemTree(std::ostream& out);
 
  private:
-  FRIEND_TEST(FsManagerTestBase, TestDuplicatePaths);
-  FRIEND_TEST(FsManagerTestBase, TestMetadataDirInWALRoot);
-  FRIEND_TEST(FsManagerTestBase, TestMetadataDirInDataRoot);
-  FRIEND_TEST(FsManagerTestBase, TestIsolatedMetadataDir);
+  FRIEND_TEST(fs::FsManagerTestBase, TestDuplicatePaths);
+  FRIEND_TEST(fs::FsManagerTestBase, TestEIOWhileRunningUpdateDirsTool);
+  FRIEND_TEST(fs::FsManagerTestBase, TestIsolatedMetadataDir);
+  FRIEND_TEST(fs::FsManagerTestBase, TestMetadataDirInWALRoot);
+  FRIEND_TEST(fs::FsManagerTestBase, TestMetadataDirInDataRoot);
+  FRIEND_TEST(fs::FsManagerTestBase, TestOpenWithDuplicateInstanceFiles);
   FRIEND_TEST(tserver::MiniTabletServerTest, TestFsLayoutEndToEnd);
   friend class itest::MiniClusterFsInspector; // for access to directory names
 
diff --git a/src/kudu/integration-tests/open-readonly-fs-itest.cc b/src/kudu/integration-tests/open-readonly-fs-itest.cc
index dbcf9ea..461166d 100644
--- a/src/kudu/integration-tests/open-readonly-fs-itest.cc
+++ b/src/kudu/integration-tests/open-readonly-fs-itest.cc
@@ -31,6 +31,7 @@
 #include "kudu/client/shared_ptr.h"
 #include "kudu/client/write_op.h"
 #include "kudu/common/partial_row.h"
+#include "kudu/fs/data_dirs.h"
 #include "kudu/fs/fs_manager.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/mini-cluster/external_mini_cluster.h"
@@ -131,6 +132,7 @@ TEST_F(OpenReadonlyFsITest, TestWriteAndVerify) {
   auto t = std::thread([this, deadline] () {
       FsManagerOpts fs_opts;
       fs_opts.read_only = true;
+      fs_opts.update_instances = fs::UpdateInstanceBehavior::DONT_UPDATE;
       fs_opts.wal_root = cluster_->tablet_server(0)->wal_dir();
       fs_opts.data_roots = cluster_->tablet_server(0)->data_dirs();
       while (MonoTime::Now() < deadline) {
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index 02294fc..08fcfd7 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -28,7 +28,6 @@
 #include <boost/algorithm/string/predicate.hpp>
 #include <boost/optional/optional.hpp>
 #include <gflags/gflags.h>
-#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 
 #include "kudu/clock/clock.h"
@@ -448,16 +447,13 @@ Status ServerBase::Init() {
 
   fs::FsReport report;
   Status s = fs_manager_->Open(&report);
+  // No instance files existed. Try creating a new FS layout.
   if (s.IsNotFound()) {
     LOG(INFO) << "Could not load existing FS layout: " << s.ToString();
     LOG(INFO) << "Attempting to create new FS layout instead";
     is_first_run_ = true;
     s = fs_manager_->CreateInitialFileSystemLayout();
     if (s.IsAlreadyPresent()) {
-      // The operator is likely trying to start up with an extra entry in their
-      // `fs_data_dirs` configuration.
-      LOG(INFO) << "To start Kudu with a different FS layout, the `kudu fs "
-                   "update_dirs` tool must be run first";
       return s.CloneAndPrepend("FS layout already exists; not overwriting existing layout");
     }
     RETURN_NOT_OK_PREPEND(s, "Could not create new FS layout");
diff --git a/src/kudu/tools/tool_action_fs.cc b/src/kudu/tools/tool_action_fs.cc
index c72247f..1f0c6c4 100644
--- a/src/kudu/tools/tool_action_fs.cc
+++ b/src/kudu/tools/tool_action_fs.cc
@@ -29,7 +29,6 @@
 #include <boost/container/vector.hpp>
 #include <boost/optional/optional.hpp>
 #include <gflags/gflags.h>
-#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 
 #include "kudu/cfile/cfile.pb.h"
@@ -108,7 +107,7 @@ using cfile::CFileIterator;
 using cfile::CFileReader;
 using cfile::ReaderOptions;
 using fs::BlockDeletionTransaction;
-using fs::ConsistencyCheckBehavior;
+using fs::UpdateInstanceBehavior;
 using fs::FsReport;
 using fs::ReadableBlock;
 using std::cout;
@@ -128,6 +127,7 @@ namespace {
 Status Check(const RunnerContext& /*context*/) {
   FsManagerOpts fs_opts;
   fs_opts.read_only = !FLAGS_repair;
+  fs_opts.update_instances = UpdateInstanceBehavior::DONT_UPDATE;
   FsManager fs_manager(Env::Default(), std::move(fs_opts));
   FsReport report;
   RETURN_NOT_OK(fs_manager.Open(&report));
@@ -232,6 +232,7 @@ Status Format(const RunnerContext& /*context*/) {
 Status DumpUuid(const RunnerContext& /*context*/) {
   FsManagerOpts fs_opts;
   fs_opts.read_only = true;
+  fs_opts.update_instances = UpdateInstanceBehavior::DONT_UPDATE;
   FsManager fs_manager(Env::Default(), std::move(fs_opts));
   RETURN_NOT_OK(fs_manager.PartialOpen());
   cout << fs_manager.uuid() << endl;
@@ -256,6 +257,7 @@ Status DumpCFile(const RunnerContext& context) {
 
   FsManagerOpts fs_opts;
   fs_opts.read_only = true;
+  fs_opts.update_instances = UpdateInstanceBehavior::DONT_UPDATE;
   FsManager fs_manager(Env::Default(), std::move(fs_opts));
   RETURN_NOT_OK(fs_manager.Open());
 
@@ -287,6 +289,7 @@ Status DumpBlock(const RunnerContext& context) {
 
   FsManagerOpts fs_opts;
   fs_opts.read_only = true;
+  fs_opts.update_instances = UpdateInstanceBehavior::DONT_UPDATE;
   FsManager fs_manager(Env::Default(), std::move(fs_opts));
   RETURN_NOT_OK(fs_manager.Open());
 
@@ -313,6 +316,7 @@ Status DumpBlock(const RunnerContext& context) {
 Status DumpFsTree(const RunnerContext& /*context*/) {
   FsManagerOpts fs_opts;
   fs_opts.read_only = true;
+  fs_opts.update_instances = UpdateInstanceBehavior::DONT_UPDATE;
   FsManager fs_manager(Env::Default(), std::move(fs_opts));
   RETURN_NOT_OK(fs_manager.Open());
 
@@ -323,7 +327,7 @@ Status DumpFsTree(const RunnerContext& /*context*/) {
 Status CheckForTabletsThatWillFailWithUpdate() {
   FsManagerOpts opts;
   opts.read_only = true;
-  opts.consistency_check = ConsistencyCheckBehavior::IGNORE_INCONSISTENCY;
+  opts.update_instances = UpdateInstanceBehavior::DONT_UPDATE;
   FsManager fs(Env::Default(), std::move(opts));
   RETURN_NOT_OK(fs.Open());
 
@@ -368,7 +372,7 @@ Status Update(const RunnerContext& /*context*/) {
 
   // Now perform the update.
   FsManagerOpts opts;
-  opts.consistency_check = ConsistencyCheckBehavior::UPDATE_ON_DISK;
+  opts.update_instances = UpdateInstanceBehavior::UPDATE_AND_ERROR_ON_FAILURE;
   FsManager fs(env, std::move(opts));
   return fs.Open();
 }
@@ -718,6 +722,7 @@ Status List(const RunnerContext& /*context*/) {
 
   FsManagerOpts fs_opts;
   fs_opts.read_only = true;
+  fs_opts.update_instances = UpdateInstanceBehavior::DONT_UPDATE;
   FsManager fs_manager(Env::Default(), std::move(fs_opts));
   RETURN_NOT_OK(fs_manager.Open());
 
@@ -876,7 +881,10 @@ unique_ptr<Mode> BuildFsMode() {
       ActionBuilder("update_dirs", &Update)
       .Description("Updates the set of data directories in an existing Kudu filesystem")
       .ExtraDescription("If a data directory is in use by a tablet and is "
-          "removed, the operation will fail unless --force is also used")
+          "removed, the operation will fail unless --force is also used. "
+          "Starting with Kudu 1.12.0, it is not required to run this tool "
+          "to add or remove directories. This tool is preserved for backwards "
+          "compatibility")
       .AddOptionalParameter("force", boost::none, string("If true, permits "
           "the removal of a data directory that is configured for use by "
           "existing tablets. Those tablets will fail the next time the server "
diff --git a/src/kudu/tools/tool_action_local_replica.cc b/src/kudu/tools/tool_action_local_replica.cc
index 6f3a760..347d1b6 100644
--- a/src/kudu/tools/tool_action_local_replica.cc
+++ b/src/kudu/tools/tool_action_local_replica.cc
@@ -155,6 +155,7 @@ string Indent(int indent) {
 Status FsInit(unique_ptr<FsManager>* fs_manager) {
   FsManagerOpts fs_opts;
   fs_opts.read_only = true;
+  fs_opts.update_instances = fs::UpdateInstanceBehavior::DONT_UPDATE;
   unique_ptr<FsManager> fs_ptr(new FsManager(Env::Default(), fs_opts));
   RETURN_NOT_OK(fs_ptr->Open());
   fs_manager->swap(fs_ptr);
diff --git a/src/kudu/tserver/tablet_server-test.cc b/src/kudu/tserver/tablet_server-test.cc
index a739056..d468190 100644
--- a/src/kudu/tserver/tablet_server-test.cc
+++ b/src/kudu/tserver/tablet_server-test.cc
@@ -22,6 +22,7 @@
 
 #include <algorithm>
 #include <cstdint>
+#include <initializer_list>
 #include <map>
 #include <memory>
 #include <set>
@@ -35,7 +36,6 @@
 #include <boost/bind.hpp>
 #include <boost/optional/optional.hpp>
 #include <gflags/gflags.h>
-#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 #include <google/protobuf/util/message_differencer.h>
 #include <gtest/gtest.h>
@@ -134,6 +134,7 @@ using kudu::tablet::LocalTabletWriter;
 using kudu::tablet::RowSetDataPB;
 using kudu::tablet::Tablet;
 using kudu::tablet::TabletReplica;
+using kudu::tablet::TabletStatePB;
 using kudu::tablet::TabletSuperBlockPB;
 using std::map;
 using std::pair;
@@ -862,6 +863,39 @@ TEST_F(TabletServerTest, TestEIODuringDelete) {
   ASSERT_OK(admin_proxy_->DeleteTablet(req, &resp, &rpc));
 }
 
+// Test that adding a directories enables tablet placement in the new
+// directories, and that removing directories fails tablets that are striped
+// across the removed directories.
+TEST_F(TabletServerTest, TestAddRemoveDirectory) {
+  // Start with multiple data dirs so the dirs are suffixed with numbers, and
+  // so when we remove a data dirs, we'll be using the same set of dirs.
+  NO_FATALS(ShutdownAndRebuildTablet(/*num_data_dirs*/2));
+  const char* kFooTablet1 = "fffffffffffffffffffffffffffffff1";
+  ASSERT_OK(mini_server_->AddTestTablet("footable", kFooTablet1, schema_));
+  ASSERT_OK(WaitForTabletRunning(kFooTablet1));
+
+  // Shut down and restart with a new directory. This is allowed, and the
+  // tablet server will be able to use the new directory if we create a new
+  // tablet.
+  NO_FATALS(ShutdownAndRebuildTablet(/*num_data_dirs*/3));
+  const char* kFooTablet2 = "fffffffffffffffffffffffffffffff2";
+  ASSERT_OK(mini_server_->AddTestTablet("footable", kFooTablet2, schema_));
+  ASSERT_OK(WaitForTabletRunning(kFooTablet2));
+
+  // Now open up again with a our original two directories. The second tablet
+  // should fail because it should have been striped across the third
+  // directory. The first tablet should be unaffected.
+  NO_FATALS(ShutdownAndRebuildTablet(/*num_data_dirs*/2));
+  ASSERT_EVENTUALLY([&] {
+    scoped_refptr<TabletReplica> replica1;
+    ASSERT_TRUE(mini_server_->server()->tablet_manager()->LookupTablet(kFooTablet1, &replica1));
+    ASSERT_EQ(TabletStatePB::RUNNING, replica1->state());
+    scoped_refptr<TabletReplica> replica2;
+    ASSERT_TRUE(mini_server_->server()->tablet_manager()->LookupTablet(kFooTablet2, &replica2));
+    ASSERT_EQ(TabletStatePB::FAILED, replica2->state());
+  });
+}
+
 class TabletServerMaintenanceMemoryPressureTest : public TabletServerTestBase {
  public:
   void SetUp() override {