You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ad...@apache.org on 2019/12/09 04:16:03 UTC

[kudu] 03/05: fs: rename PathInstance-related things

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

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

commit b5e07be0598d5836ddff05996a4b1274cd8184ec
Author: Andrew Wong <aw...@apache.org>
AuthorDate: Thu Dec 5 00:28:14 2019 -0800

    fs: rename PathInstance-related things
    
    I'm expecting the directory management code to be reused for WAL
    directories, and I've sometimes found it mildly annoying that we have
    different nomenclature for path instances, directories, and block
    managers in the same area (mostly in the PathInstances). So thought I'd
    rename some things now to make some things more straight-forward:
    
    * PathInstanceMetadataFile -> DirInstanceMetadataFile
    * PathInstanceMetadataPB -> DirInstanceMetadataPB
    * PathSetPB -> DirSetPB
    * block_manager_type -> dir_type (in the context of instance files only)
    * block_manager_util -> dir_util
    
    Along the way, I modernized our usage of the DataDir threadpool when
    calling DataDirManager::Open().
    
    Change-Id: I89049fafe149d535d584e7d4cd2497c298c16043
    Reviewed-on: http://gerrit.cloudera.org:8080/14862
    Tested-by: Kudu Jenkins
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
---
 src/kudu/fs/CMakeLists.txt                         |  4 +-
 src/kudu/fs/block_manager-test.cc                  |  2 +-
 src/kudu/fs/data_dirs.cc                           | 63 +++++++++++-----------
 src/kudu/fs/data_dirs.h                            | 24 ++++-----
 ...block_manager_util-test.cc => dir_util-test.cc} | 37 ++++++-------
 src/kudu/fs/{block_manager_util.cc => dir_util.cc} | 42 +++++++--------
 src/kudu/fs/{block_manager_util.h => dir_util.h}   | 20 +++----
 src/kudu/fs/error_manager.h                        |  2 +-
 src/kudu/fs/file_block_manager.cc                  |  4 +-
 src/kudu/fs/fs.proto                               | 38 ++++++-------
 src/kudu/fs/fs_manager-test.cc                     |  8 +--
 src/kudu/fs/log_block_manager.cc                   |  8 +--
 src/kudu/fs/log_block_manager.h                    |  2 +-
 13 files changed, 124 insertions(+), 130 deletions(-)

diff --git a/src/kudu/fs/CMakeLists.txt b/src/kudu/fs/CMakeLists.txt
index 593a08e..bab8d54 100644
--- a/src/kudu/fs/CMakeLists.txt
+++ b/src/kudu/fs/CMakeLists.txt
@@ -29,8 +29,8 @@ add_library(kudu_fs
   block_id.cc
   block_manager.cc
   block_manager_metrics.cc
-  block_manager_util.cc
   data_dirs.cc
+  dir_util.cc
   error_manager.cc
   file_block_manager.cc
   fs_manager.cc
@@ -54,9 +54,9 @@ target_link_libraries(kudu_fs_test_util
 # Tests
 SET_KUDU_TEST_LINK_LIBS(kudu_fs kudu_fs_test_util)
 ADD_KUDU_TEST(block_manager-test)
-ADD_KUDU_TEST(block_manager_util-test)
 ADD_KUDU_TEST(block_manager-stress-test RUN_SERIAL true)
 ADD_KUDU_TEST(data_dirs-test)
+ADD_KUDU_TEST(dir_util-test)
 ADD_KUDU_TEST(error_manager-test)
 ADD_KUDU_TEST(fs_manager-test)
 if (NOT APPLE)
diff --git a/src/kudu/fs/block_manager-test.cc b/src/kudu/fs/block_manager-test.cc
index 7f19903..31da823 100644
--- a/src/kudu/fs/block_manager-test.cc
+++ b/src/kudu/fs/block_manager-test.cc
@@ -321,7 +321,7 @@ void BlockManagerTest<FileBlockManager>::RunMultipathTest(const vector<string>&
       if (child == "." || child == "..") {
         continue;
       }
-      PathInstanceMetadataPB instance;
+      DirInstanceMetadataPB instance;
       ASSERT_OK(pb_util::ReadPBContainerFromPath(env_,
                                                  JoinPathSegments(path, child),
                                                  &instance));
diff --git a/src/kudu/fs/data_dirs.cc b/src/kudu/fs/data_dirs.cc
index a7f96e0..b7df6ed 100644
--- a/src/kudu/fs/data_dirs.cc
+++ b/src/kudu/fs/data_dirs.cc
@@ -37,7 +37,7 @@
 #include <google/protobuf/stubs/common.h>
 
 #include "kudu/fs/block_manager.h"
-#include "kudu/fs/block_manager_util.h"
+#include "kudu/fs/dir_util.h"
 #include "kudu/fs/fs.pb.h"
 #include "kudu/gutil/bind.h"
 #include "kudu/gutil/integral_types.h"
@@ -165,7 +165,7 @@ DataDir::DataDir(Env* env,
                  DataDirMetrics* metrics,
                  DataDirFsType fs_type,
                  string dir,
-                 unique_ptr<PathInstanceMetadataFile> metadata_file,
+                 unique_ptr<DirInstanceMetadataFile> metadata_file,
                  unique_ptr<ThreadPool> pool)
     : env_(env),
       metrics_(metrics),
@@ -397,7 +397,7 @@ Status DataDirManager::Create() {
   for (const auto& r : canonicalized_data_fs_roots_) {
     RETURN_NOT_OK_PREPEND(r.status, "Could not create directory manager with disks failed");
   }
-  vector<unique_ptr<PathInstanceMetadataFile>> loaded_instances;
+  vector<unique_ptr<DirInstanceMetadataFile>> loaded_instances;
   bool has_existing_instances;
   RETURN_NOT_OK(LoadInstances(&loaded_instances, &has_existing_instances));
   if (has_existing_instances) {
@@ -412,7 +412,7 @@ Status DataDirManager::Create() {
 }
 
 Status DataDirManager::CreateNewDirectoriesAndUpdateInstances(
-    vector<unique_ptr<PathInstanceMetadataFile>> instances) {
+    vector<unique_ptr<DirInstanceMetadataFile>> instances) {
   CHECK(!opts_.read_only);
   CHECK_NE(UpdateInstanceBehavior::DONT_UPDATE, opts_.update_instances);
 
@@ -448,7 +448,7 @@ Status DataDirManager::CreateNewDirectoriesAndUpdateInstances(
   //
   // Note: we don't bother trying to create/update the instance if the file is
   // otherwise unhealthy.
-  vector<unique_ptr<PathInstanceMetadataFile>> healthy_instances;
+  vector<unique_ptr<DirInstanceMetadataFile>> healthy_instances;
   for (auto& instance : instances) {
     if (instance->healthy()) {
       healthy_instances.emplace_back(std::move(instance));
@@ -466,13 +466,13 @@ Status DataDirManager::CreateNewDirectoriesAndUpdateInstances(
 
   // 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;
+  vector<unique_ptr<DirInstanceMetadataFile>> instances_to_update;
   for (auto& instance : healthy_instances) {
     DCHECK(instance->healthy());
-    const auto& path_set = instance->metadata()->path_set();
+    const auto& dir_set = instance->metadata()->dir_set();
     set<string> instance_uuids;
-    for (int i = 0; i < path_set.all_uuids_size(); i++) {
-      InsertIfNotPresent(&instance_uuids, path_set.all_uuids(i));
+    for (int i = 0; i < dir_set.all_uuids_size(); i++) {
+      InsertIfNotPresent(&instance_uuids, dir_set.all_uuids(i));
     }
     // If an instance file disagrees with the expected UUIDs, rewrite it.
     if (all_uuids != instance_uuids) {
@@ -500,7 +500,7 @@ Status DataDirManager::CreateNewDirectoriesAndUpdateInstances(
 }
 
 Status DataDirManager::UpdateHealthyInstances(
-    const vector<unique_ptr<PathInstanceMetadataFile>>& instances_to_update,
+    const vector<unique_ptr<DirInstanceMetadataFile>>& instances_to_update,
     const set<string>& new_all_uuids) {
   unordered_map<string, string> copies_to_restore;
   unordered_set<string> copies_to_delete;
@@ -548,10 +548,10 @@ Status DataDirManager::UpdateHealthyInstances(
     string copy_filename = instance_filename + kTmpInfix;
 
     // Put together the PB and perform the update.
-    PathInstanceMetadataPB new_pb = *instance->metadata();
-    new_pb.mutable_path_set()->mutable_all_uuids()->Clear();
+    DirInstanceMetadataPB new_pb = *instance->metadata();
+    new_pb.mutable_dir_set()->mutable_all_uuids()->Clear();
     for (const auto& uuid : new_all_uuids) {
-      new_pb.mutable_path_set()->add_all_uuids(uuid);
+      new_pb.mutable_dir_set()->add_all_uuids(uuid);
     }
 
     // We're about to update the file; if we fail midway, we should try to
@@ -587,7 +587,7 @@ Status DataDirManager::UpdateHealthyInstances(
 }
 
 Status DataDirManager::LoadInstances(
-    vector<unique_ptr<PathInstanceMetadataFile>>* instance_files,
+    vector<unique_ptr<DirInstanceMetadataFile>>* instance_files,
     bool* has_existing_instances) {
   LockMode lock_mode;
   if (!FLAGS_fs_lock_data_dirs) {
@@ -598,7 +598,7 @@ Status DataDirManager::LoadInstances(
     lock_mode = LockMode::MANDATORY;
   }
   vector<string> missing_roots_tmp;
-  vector<unique_ptr<PathInstanceMetadataFile>> loaded_instances;
+  vector<unique_ptr<DirInstanceMetadataFile>> loaded_instances;
   ObjectIdGenerator gen;
   for (int i = 0; i < canonicalized_data_fs_roots_.size(); i++) {
     const auto& root = canonicalized_data_fs_roots_[i];
@@ -608,8 +608,8 @@ Status DataDirManager::LoadInstances(
     // 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_, std::move(backup_uuid), opts_.block_manager_type,
+    unique_ptr<DirInstanceMetadataFile> instance(
+        new DirInstanceMetadataFile(env_, std::move(backup_uuid), opts_.block_manager_type,
                                      instance_filename));
     if (PREDICT_FALSE(!root.status.ok())) {
       instance->SetInstanceFailed(root.status);
@@ -675,11 +675,11 @@ Status DataDirManager::PopulateDirectoryMaps(const vector<unique_ptr<DataDir>>&
     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);
+        const auto& dir_set = dd->instance()->metadata()->dir_set();
+        VLOG(1) << Substitute("using dir set $0 as reference: $1",
+            dd->instance()->path(), pb_util::SecureDebugString(dir_set));
+        for (int idx = 0; idx < dir_set.all_uuids_size(); idx++) {
+          const string& uuid = dir_set.all_uuids(idx);
           InsertIfNotPresent(&uuid_to_idx, uuid, idx);
         }
         break;
@@ -691,7 +691,7 @@ Status DataDirManager::PopulateDirectoryMaps(const vector<unique_ptr<DataDir>>&
           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
+    // Keep track of any dirs that were not referenced in the dir set. These
     // are presumably from instance files we failed to read. We'll assign them
     // indexes of those that remain.
     vector<DataDir*> unassigned_dirs;
@@ -730,7 +730,7 @@ Status DataDirManager::Open() {
                                               canonicalized_data_fs_roots_.size(), kMaxDataDirs));
   }
 
-  vector<unique_ptr<PathInstanceMetadataFile>> loaded_instances;
+  vector<unique_ptr<DirInstanceMetadataFile>> loaded_instances;
   // Load the instance files from disk.
   bool has_existing_instances;
   RETURN_NOT_OK_PREPEND(LoadInstances(&loaded_instances, &has_existing_instances),
@@ -762,13 +762,6 @@ Status DataDirManager::Open() {
     auto& instance = loaded_instances[i];
     const string data_dir = instance->dir();
 
-    // Create a per-dir thread pool.
-    unique_ptr<ThreadPool> pool;
-    RETURN_NOT_OK(ThreadPoolBuilder(Substitute("data dir $0", i))
-                  .set_max_threads(1)
-                  .set_trace_metric_prefix("data dirs")
-                  .Build(&pool));
-
     // Figure out what filesystem the data directory is on.
     DataDirFsType fs_type = DataDirFsType::OTHER;
     if (instance->healthy()) {
@@ -792,9 +785,15 @@ Status DataDirManager::Open() {
       }
     }
 
+    // Create a per-dir thread pool.
+    unique_ptr<ThreadPool> pool;
+    RETURN_NOT_OK(ThreadPoolBuilder(Substitute("data dir $0", i))
+                  .set_max_threads(1)
+                  .set_trace_metric_prefix("data dirs")
+                  .Build(&pool));
     unique_ptr<DataDir> dd(new DataDir(
         env_, metrics_.get(), fs_type, data_dir, std::move(instance),
-        unique_ptr<ThreadPool>(pool.release())));
+        std::move(pool)));
     dds.emplace_back(std::move(dd));
   }
 
diff --git a/src/kudu/fs/data_dirs.h b/src/kudu/fs/data_dirs.h
index a9188ed..18f989d 100644
--- a/src/kudu/fs/data_dirs.h
+++ b/src/kudu/fs/data_dirs.h
@@ -58,8 +58,8 @@ namespace fs {
 typedef std::unordered_map<int, std::string> UuidByUuidIndexMap;
 typedef std::unordered_map<std::string, int> UuidIndexByUuidMap;
 
-class PathInstanceMetadataFile;
 struct CreateBlockOptions;
+class DirInstanceMetadataFile;
 
 const char kInstanceMetadataFileName[] = "block_manager_instance";
 const char kDataDirName[] = "data";
@@ -68,7 +68,7 @@ namespace internal {
 
 // A DataDirGroup is a group of directories used by an entity for block
 // placement. A group is represented in memory by a list of indices which index
-// into the list of all UUIDs found in a PathSetPB. A group is represented
+// into the list of all UUIDs found in a DirSetPB. A group is represented
 // on-disk as a list of full UUIDs, and as such, when writing or reading from
 // disk, a mapping is needed to translate between index and UUID.
 //
@@ -117,7 +117,7 @@ enum class DataDirFsType {
 // 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
+  // If the data directories don't match the on-disk dir sets, update the
   // on-disk data to match if not in read-only mode.
   UPDATE_AND_IGNORE_FAILURES,
 
@@ -125,8 +125,8 @@ enum UpdateInstanceBehavior {
   // 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.
+  // If the data directories don't match the on-disk dir sets, continue without
+  // updating the on-disk data.
   DONT_UPDATE
 };
 
@@ -144,7 +144,7 @@ class DataDir {
           DataDirMetrics* metrics,
           DataDirFsType fs_type,
           std::string dir,
-          std::unique_ptr<PathInstanceMetadataFile> metadata_file,
+          std::unique_ptr<DirInstanceMetadataFile> metadata_file,
           std::unique_ptr<ThreadPool> pool);
   ~DataDir();
 
@@ -180,7 +180,7 @@ class DataDir {
 
   const std::string& dir() const { return dir_; }
 
-  const PathInstanceMetadataFile* instance() const {
+  const DirInstanceMetadataFile* instance() const {
     return metadata_file_.get();
   }
 
@@ -199,7 +199,7 @@ class DataDir {
   DataDirMetrics* metrics_;
   const DataDirFsType fs_type_;
   const std::string dir_;
-  const std::unique_ptr<PathInstanceMetadataFile> metadata_file_;
+  const std::unique_ptr<DirInstanceMetadataFile> metadata_file_;
   const std::unique_ptr<ThreadPool> pool_;
 
   bool is_shutdown_;
@@ -392,7 +392,7 @@ class DataDirManager {
   // Finds a data directory by uuid index, returning null if it can't be found.
   //
   // More information on uuid indexes and their relation to data directories
-  // can be found next to PathSetPB in fs.proto.
+  // can be found next to DirSetPB in fs.proto.
   DataDir* FindDataDirByUuidIndex(int uuid_idx) const;
 
   // Finds a uuid index by data directory, returning false if it can't be found.
@@ -450,7 +450,7 @@ class DataDirManager {
   // Returns an error if an instance file fails in an irreconcileable way (e.g.
   // the file is locked).
   Status LoadInstances(
-      std::vector<std::unique_ptr<PathInstanceMetadataFile>>* instance_files,
+      std::vector<std::unique_ptr<DirInstanceMetadataFile>>* instance_files,
       bool* has_existing_instances);
 
   // Takes the set of instance files, does some basic verification on them,
@@ -467,7 +467,7 @@ class DataDirManager {
   // 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);
+      std::vector<std::unique_ptr<DirInstanceMetadataFile>> instances);
 
   // Updates the on-disk instance files specified by 'instances_to_update'
   // (presumably those whose 'all_uuids' field doesn't match 'new_all_uuids')
@@ -481,7 +481,7 @@ class DataDirManager {
   // 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::vector<std::unique_ptr<DirInstanceMetadataFile>>& instances_to_update,
       const std::set<std::string>& new_all_uuids);
 
   // Returns a random directory in the data dir group specified in 'opts',
diff --git a/src/kudu/fs/block_manager_util-test.cc b/src/kudu/fs/dir_util-test.cc
similarity index 71%
rename from src/kudu/fs/block_manager_util-test.cc
rename to src/kudu/fs/dir_util-test.cc
index 2f3fbcf..d669188 100644
--- a/src/kudu/fs/block_manager_util-test.cc
+++ b/src/kudu/fs/dir_util-test.cc
@@ -15,19 +15,16 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "kudu/fs/block_manager_util.h"
+#include "kudu/fs/dir_util.h"
 
-#include <memory>
 #include <ostream>
 #include <string>
-#include <vector>
 
 #include <glog/logging.h>
 #include <google/protobuf/repeated_field.h> // IWYU pragma: keep
 #include <gtest/gtest.h>
 
 #include "kudu/fs/fs.pb.h"
-#include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/env.h"
 #include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
@@ -36,11 +33,7 @@
 namespace kudu {
 namespace fs {
 
-using google::protobuf::RepeatedPtrField;
 using std::string;
-using std::unique_ptr;
-using std::vector;
-using strings::Substitute;
 
 TEST_F(KuduTest, Lifecycle) {
   string kType = "asdf";
@@ -49,27 +42,27 @@ TEST_F(KuduTest, Lifecycle) {
 
   // Test that the metadata file was created.
   {
-    PathInstanceMetadataFile file(env_, kUuid, kType, kFileName);
+    DirInstanceMetadataFile 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_, kUuid, kType, kFileName);
+    DirInstanceMetadataFile file(env_, kUuid, kType, kFileName);
     ASSERT_OK(file.LoadFromDisk());
-    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());
-    ASSERT_EQ(1, path_set.all_uuids_size());
-    ASSERT_EQ(kUuid, path_set.all_uuids(0));
+    const DirInstanceMetadataPB* md = file.metadata();
+    ASSERT_EQ(kType, md->dir_type());
+    const DirSetPB& dir_set = md->dir_set();
+    ASSERT_EQ(kUuid, dir_set.uuid());
+    ASSERT_EQ(1, dir_set.all_uuids_size());
+    ASSERT_EQ(kUuid, dir_set.all_uuids(0));
   }
 
   // Test that expecting a different type of block manager fails.
   {
-    PathInstanceMetadataFile file(env_, kUuid, "other type", kFileName);
-    PathInstanceMetadataPB pb;
+    DirInstanceMetadataFile file(env_, kUuid, "other type", kFileName);
+    DirInstanceMetadataPB pb;
     ASSERT_TRUE(file.LoadFromDisk().IsIOError());
   }
 }
@@ -79,10 +72,10 @@ TEST_F(KuduTest, Locking) {
   const string kFileName = GetTestPath("foo");
   string kUuid = "a_uuid";
 
-  PathInstanceMetadataFile file(env_, kUuid, kType, kFileName);
+  DirInstanceMetadataFile file(env_, kUuid, kType, kFileName);
   ASSERT_OK(file.Create({ kUuid }));
 
-  PathInstanceMetadataFile first(env_, "", kType, kFileName);
+  DirInstanceMetadataFile first(env_, "", kType, kFileName);
   ASSERT_OK(first.LoadFromDisk());
   ASSERT_EQ(kUuid, first.uuid());
   ASSERT_OK(first.Lock());
@@ -90,7 +83,7 @@ TEST_F(KuduTest, Locking) {
   // 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);
+    DirInstanceMetadataFile second(env_, "", kType, kFileName);
     CHECK_OK(second.LoadFromDisk());
     CHECK_EQ(kUuid, second.uuid());
     CHECK_OK(second.Lock());
@@ -98,7 +91,7 @@ TEST_F(KuduTest, Locking) {
 
   ASSERT_OK(first.Unlock());
   ASSERT_DEATH({
-    PathInstanceMetadataFile second(env_, "", kType, kFileName);
+    DirInstanceMetadataFile second(env_, "", kType, kFileName);
     CHECK_OK(second.LoadFromDisk());
     CHECK_EQ(kUuid, second.uuid());
     Status s = second.Lock();
diff --git a/src/kudu/fs/block_manager_util.cc b/src/kudu/fs/dir_util.cc
similarity index 89%
rename from src/kudu/fs/block_manager_util.cc
rename to src/kudu/fs/dir_util.cc
index 92b2765..98f4e82 100644
--- a/src/kudu/fs/block_manager_util.cc
+++ b/src/kudu/fs/dir_util.cc
@@ -14,7 +14,7 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-#include "kudu/fs/block_manager_util.h"
+#include "kudu/fs/dir_util.h"
 
 #include <cstdint>
 #include <ostream>
@@ -125,22 +125,22 @@ Status CheckHolePunch(Env* env, const string& path) {
   } \
 } while (0)
 
-PathInstanceMetadataFile::PathInstanceMetadataFile(Env* env,
+DirInstanceMetadataFile::DirInstanceMetadataFile(Env* env,
                                                    string uuid,
-                                                   string block_manager_type,
+                                                   string dir_type,
                                                    string filename)
     : env_(env),
       uuid_(std::move(uuid)),
-      block_manager_type_(std::move(block_manager_type)),
+      dir_type_(std::move(dir_type)),
       filename_(std::move(filename)) {}
 
-PathInstanceMetadataFile::~PathInstanceMetadataFile() {
+DirInstanceMetadataFile::~DirInstanceMetadataFile() {
   if (lock_) {
     WARN_NOT_OK(Unlock(), Substitute("Failed to unlock file $0", filename_));
   }
 }
 
-Status PathInstanceMetadataFile::Create(const set<string>& all_uuids,
+Status DirInstanceMetadataFile::Create(const set<string>& all_uuids,
                                         bool* created_dir) {
   DCHECK(!lock_);
   DCHECK(ContainsKey(all_uuids, uuid_));
@@ -159,7 +159,7 @@ Status PathInstanceMetadataFile::Create(const set<string>& all_uuids,
 
   // If we're initializing the log block manager, check that we support
   // hole-punching.
-  if (block_manager_type_ == "log") {
+  if (dir_type_ == "log") {
     RETURN_NOT_OK_FAIL_INSTANCE_PREPEND(CheckHolePunch(env_, dir_name),
                                         kHolePunchErrorMsg);
   }
@@ -187,17 +187,17 @@ Status PathInstanceMetadataFile::Create(const set<string>& all_uuids,
   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->mutable_all_uuids()->Reserve(all_uuids.size());
+  // Set up the directory set.
+  DirInstanceMetadataPB new_instance;
+  DirSetPB* new_dir_set = new_instance.mutable_dir_set();
+  new_dir_set->set_uuid(uuid_);
+  new_dir_set->mutable_all_uuids()->Reserve(all_uuids.size());
   for (const string& u : all_uuids) {
-    new_path_set->add_all_uuids(u);
+    new_dir_set->add_all_uuids(u);
   }
 
   // And the rest of the metadata.
-  new_instance.set_block_manager_type(block_manager_type_);
+  new_instance.set_dir_type(dir_type_);
   new_instance.set_filesystem_block_size_bytes(block_size);
 
   RETURN_NOT_OK_FAIL_INSTANCE_PREPEND(pb_util::WritePBContainerToPath(
@@ -216,19 +216,19 @@ Status PathInstanceMetadataFile::Create(const set<string>& all_uuids,
   return Status::OK();
 }
 
-Status PathInstanceMetadataFile::LoadFromDisk() {
+Status DirInstanceMetadataFile::LoadFromDisk() {
   DCHECK(!lock_) <<
       "Opening a metadata file that's already locked would release the lock";
 
-  unique_ptr<PathInstanceMetadataPB> pb(new PathInstanceMetadataPB());
+  unique_ptr<DirInstanceMetadataPB> pb(new DirInstanceMetadataPB());
   RETURN_NOT_OK_FAIL_INSTANCE_PREPEND(pb_util::ReadPBContainerFromPath(env_, filename_, pb.get()),
       Substitute("Failed to read metadata file from $0", filename_));
 
-  if (pb->block_manager_type() != block_manager_type_) {
+  if (pb->dir_type() != dir_type_) {
     return Status::IOError(Substitute(
       "existing instance was written using the '$0' format; cannot restart "
       "with a different format type '$1'",
-      pb->block_manager_type(), block_manager_type_));
+      pb->dir_type(), dir_type_));
   }
 
   uint64_t block_size;
@@ -239,12 +239,12 @@ Status PathInstanceMetadataFile::LoadFromDisk() {
         "Expected $0 but was $1", pb->filesystem_block_size_bytes(), block_size));
   }
 
-  uuid_ = pb->path_set().uuid();
+  uuid_ = pb->dir_set().uuid();
   metadata_ = std::move(pb);
   return Status::OK();
 }
 
-Status PathInstanceMetadataFile::Lock() {
+Status DirInstanceMetadataFile::Lock() {
   DCHECK(!lock_);
 
   FileLock* lock;
@@ -256,7 +256,7 @@ Status PathInstanceMetadataFile::Lock() {
   return Status::OK();
 }
 
-Status PathInstanceMetadataFile::Unlock() {
+Status DirInstanceMetadataFile::Unlock() {
   DCHECK(lock_);
 
   RETURN_NOT_OK_FAIL_INSTANCE_PREPEND(env_->UnlockFile(lock_.release()),
diff --git a/src/kudu/fs/block_manager_util.h b/src/kudu/fs/dir_util.h
similarity index 88%
rename from src/kudu/fs/block_manager_util.h
rename to src/kudu/fs/dir_util.h
index 4675a3c..c55436e 100644
--- a/src/kudu/fs/block_manager_util.h
+++ b/src/kudu/fs/dir_util.h
@@ -26,30 +26,30 @@
 
 namespace kudu {
 
+class DirInstanceMetadataPB;
 class Env;
 class FileLock;
-class PathInstanceMetadataPB;
 
 namespace fs {
 
 // Reads and writes block manager instance metadata files.
 //
 // Thread-unsafe; access to this object must be externally synchronized.
-class PathInstanceMetadataFile {
+class DirInstanceMetadataFile {
  public:
   // 'env' must remain valid for the lifetime of this class.
   //
   // '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);
+  DirInstanceMetadataFile(Env* env, std::string uuid, std::string dir_type,
+                          std::string filename);
 
-  ~PathInstanceMetadataFile();
+  ~DirInstanceMetadataFile();
 
   // 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.
+  // 'all_uuids' is all of the unique UUIDs in this instance's dir 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.
   //
@@ -79,7 +79,7 @@ class PathInstanceMetadataFile {
   // 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_'.
-  void SetInstanceFailed(const Status& s = Status::IOError("Path instance failed")) {
+  void SetInstanceFailed(const Status& s = Status::IOError("directory instance failed")) {
     health_status_ = s;
   }
 
@@ -96,7 +96,7 @@ class PathInstanceMetadataFile {
   std::string uuid() const { return uuid_; }
   std::string dir() const { return DirName(filename_); }
   const std::string& path() const { return filename_; }
-  PathInstanceMetadataPB* metadata() const { return metadata_.get(); }
+  DirInstanceMetadataPB* metadata() const { return metadata_.get(); }
 
  private:
   Env* env_;
@@ -111,12 +111,12 @@ class PathInstanceMetadataFile {
   std::string uuid_;
 
   // The type of this instance file.
-  const std::string block_manager_type_;
+  const std::string dir_type_;
 
   // The name of the file associated with this instance.
   const std::string filename_;
 
-  std::unique_ptr<PathInstanceMetadataPB> metadata_;
+  std::unique_ptr<DirInstanceMetadataPB> metadata_;
 
   // In order to prevent multiple Kudu processes from starting up using the
   // same directories, we lock the instance files when starting up.
diff --git a/src/kudu/fs/error_manager.h b/src/kudu/fs/error_manager.h
index 936c91b..046d734 100644
--- a/src/kudu/fs/error_manager.h
+++ b/src/kudu/fs/error_manager.h
@@ -22,8 +22,8 @@
 
 #include <glog/logging.h>
 
-#include "kudu/fs/block_manager_util.h"
 #include "kudu/fs/data_dirs.h"
+#include "kudu/fs/dir_util.h"
 #include "kudu/fs/fs.pb.h"
 #include "kudu/gutil/callback.h"
 #include "kudu/gutil/port.h"
diff --git a/src/kudu/fs/file_block_manager.cc b/src/kudu/fs/file_block_manager.cc
index fcf1352..9ec4471 100644
--- a/src/kudu/fs/file_block_manager.cc
+++ b/src/kudu/fs/file_block_manager.cc
@@ -81,8 +81,8 @@ namespace internal {
 //
 // A block ID uniquely locates a block. Every ID is a uint64_t, broken down
 // into multiple logical components:
-// 1. Bytes 0 (MSB) and 1 identify the block's data dir by path set index. See
-//    fs.proto for more details on path sets.
+// 1. Bytes 0 (MSB) and 1 identify the block's data dir by dir set index. See
+//    fs.proto for more details on dir sets.
 // 2. Bytes 2-7 (LSB) uniquely identify the block within the data dir. As more
 //    and more blocks are created in a data dir, the likelihood of a collision
 //    becomes greater. In the event of a collision, the block manager will
diff --git a/src/kudu/fs/fs.proto b/src/kudu/fs/fs.proto
index 3292240..7b0a44a 100644
--- a/src/kudu/fs/fs.proto
+++ b/src/kudu/fs/fs.proto
@@ -38,32 +38,34 @@ message InstanceMetadataPB {
   // TODO: add a "node type" (TS/Master?)
 }
 
-// Describes a collection of filesystem path instances and the membership of a
-// particular instance in the collection.
+// Describes a collection of filesystem directory instances and the membership
+// of a particular instance in the collection.
 //
-// In a healthy filesystem (see below), a path instance can be referred to via
-// its UUID's position in all_uuids instead of via the UUID itself. This is
+// In a healthy filesystem (see below), a directory instance can be referred to
+// via its UUID's position in all_uuids instead of via the UUID itself. This is
 // useful when there are many such references, as the position is much shorter
 // than the UUID.
-message PathSetPB {
-  // Globally unique identifier for this path instance.
+message DirSetPB {
+  // Globally unique identifier for this directory instance.
   required bytes uuid = 1;
 
-  // All UUIDs in this path instance set. In a healthy filesystem:
-  // 1. There exists an on-disk PathInstanceMetadataPB for each listed UUID, and
-  // 2. Every PathSetPB contains an identical copy of all_uuids.
+  // All UUIDs in this dir instance set. In a healthy filesystem:
+  // 1. There exists an on-disk DirInstanceMetadataPB for each listed UUID, and
+  // 2. Every DirSetPB contains an identical copy of all_uuids.
   repeated bytes all_uuids = 2;
 }
 
-// A filesystem instance can contain multiple paths. One of these structures
-// is persisted in each path when the filesystem instance is created.
-message PathInstanceMetadataPB {
-  // Describes this path instance as well as all of the other path instances
-  // that, taken together, describe a complete set.
-  required PathSetPB path_set = 1;
-
-  // Textual representation of the block manager that formatted this path.
-  required string block_manager_type = 2;
+// A filesystem instance can contain multiple directories. One of these
+// structures is persisted in each directory when the filesystem instance is
+// created.
+message DirInstanceMetadataPB {
+  // Describes this directory instance as well as all of the other directory
+  // instances that, taken together, describe a complete directory set.
+  required DirSetPB dir_set = 1;
+
+  // Textual representation of the directory type for which this directory was
+  // formatted.
+  required string dir_type = 2;
 
   // Block size on the filesystem where this instance was created. If the
   // instance (and its data) are ever copied to another location, the block
diff --git a/src/kudu/fs/fs_manager-test.cc b/src/kudu/fs/fs_manager-test.cc
index 6516228..0a9fc13 100644
--- a/src/kudu/fs/fs_manager-test.cc
+++ b/src/kudu/fs/fs_manager-test.cc
@@ -38,8 +38,8 @@
 #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/dir_util.h"
 #include "kudu/fs/fs.pb.h"
 #include "kudu/fs/fs_report.h"
 #include "kudu/gutil/map-util.h"
@@ -840,7 +840,7 @@ TEST_F(FsManagerTestBase, TestEIOWhileRunningUpdateDirsTool) {
   };
 
   // Helper to collect the contents of the InstanceMetadataPB and
-  // PathInstanceMetadtaPBs we expect to see in 'data_roots'. We'll read the
+  // DirInstanceMetadataPBs 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,
@@ -863,8 +863,8 @@ TEST_F(FsManagerTestBase, TestEIOWhileRunningUpdateDirsTool) {
         InsertOrDie(&instances, instance_path, SecureDebugString(*pb));
       }
 
-      // Collect the contents of the PathInstanceMetadataPB objects.
-      unique_ptr<PathInstanceMetadataPB> bmi_pb(new PathInstanceMetadataPB);
+      // Collect the contents of the DirInstanceMetadataPB objects.
+      unique_ptr<DirInstanceMetadataPB> bmi_pb(new DirInstanceMetadataPB);
       const auto block_manager_instance = JoinPathSegments(
           JoinPathSegments(root, kDataDirName), kInstanceMetadataFileName);
       s = ReadPBContainerFromPath(env_, block_manager_instance, bmi_pb.get());
diff --git a/src/kudu/fs/log_block_manager.cc b/src/kudu/fs/log_block_manager.cc
index dff3870..3789c38 100644
--- a/src/kudu/fs/log_block_manager.cc
+++ b/src/kudu/fs/log_block_manager.cc
@@ -17,10 +17,11 @@
 
 #include "kudu/fs/log_block_manager.h"
 
+#include <errno.h>
+
 #include <algorithm>
 #include <cstddef>
 #include <cstdint>
-#include <errno.h>
 #include <map>
 #include <memory>
 #include <mutex>
@@ -34,12 +35,11 @@
 
 #include <boost/optional/optional.hpp>
 #include <gflags/gflags.h>
-#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 
 #include "kudu/fs/block_manager_metrics.h"
-#include "kudu/fs/block_manager_util.h"
 #include "kudu/fs/data_dirs.h"
+#include "kudu/fs/dir_util.h"
 #include "kudu/fs/error_manager.h"
 #include "kudu/fs/fs.pb.h"
 #include "kudu/fs/fs_report.h"
@@ -541,7 +541,7 @@ class LogBlockContainer: public RefCountedThreadSafe<LogBlockContainer> {
   bool dead() const { return dead_.Load(); }
   const LogBlockManagerMetrics* metrics() const { return metrics_; }
   DataDir* data_dir() const { return data_dir_; }
-  const PathInstanceMetadataPB* instance() const { return data_dir_->instance()->metadata(); }
+  const DirInstanceMetadataPB* instance() const { return data_dir_->instance()->metadata(); }
 
   // Adjusts the number of blocks being written.
   // Positive means increase, negative means decrease.
diff --git a/src/kudu/fs/log_block_manager.h b/src/kudu/fs/log_block_manager.h
index 11c1216..7631042 100644
--- a/src/kudu/fs/log_block_manager.h
+++ b/src/kudu/fs/log_block_manager.h
@@ -33,6 +33,7 @@
 
 #include "kudu/fs/block_id.h"
 #include "kudu/fs/block_manager.h"
+#include "kudu/fs/fs.pb.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/util/atomic.h"
@@ -44,7 +45,6 @@
 
 namespace kudu {
 
-class BlockRecordPB;
 class Env;
 class RWFile;