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/08/14 03:29:14 UTC

[kudu] branch master updated: KUDU-2901: Consider the available space when selecting data dirs for tablets and blocks.

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


The following commit(s) were added to refs/heads/master by this push:
     new 01033ab  KUDU-2901: Consider the available space when selecting data dirs for tablets and blocks.
01033ab is described below

commit 01033aba27bb3bd846394849d854cf5f027f9b31
Author: triplesheep <tr...@gmail.com>
AuthorDate: Thu Aug 1 03:33:59 2019 +0000

    KUDU-2901: Consider the available space when selecting data dirs for tablets and blocks.
    
    When creating tablets:
    If randomly selected two dirs have same tablets count, gives preference to the
    one with more free space.
    
    When creating blocks:
    Randomly select two not full dir in tablet's data dir group and then choose
    the dir which has more free space.
    
    Change-Id: I194c4965ee64aed728e3b84e684c04d445cbe529
    Reviewed-on: http://gerrit.cloudera.org:8080/13975
    Tested-by: Kudu Jenkins
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
---
 src/kudu/fs/block_manager-test.cc                | 40 +++--------
 src/kudu/fs/data_dirs-test.cc                    |  4 +-
 src/kudu/fs/data_dirs.cc                         | 86 +++++++++++++++---------
 src/kudu/fs/data_dirs.h                          | 22 ++++--
 src/kudu/fs/file_block_manager.cc                |  2 +-
 src/kudu/fs/log_block_manager.cc                 |  4 +-
 src/kudu/integration-tests/disk_failure-itest.cc |  9 ++-
 src/kudu/util/env_util.cc                        | 18 +++--
 src/kudu/util/env_util.h                         |  9 ++-
 9 files changed, 107 insertions(+), 87 deletions(-)

diff --git a/src/kudu/fs/block_manager-test.cc b/src/kudu/fs/block_manager-test.cc
index c3135f3..7262ff0 100644
--- a/src/kudu/fs/block_manager-test.cc
+++ b/src/kudu/fs/block_manager-test.cc
@@ -74,7 +74,7 @@ DECLARE_uint64(log_container_preallocate_bytes);
 DECLARE_uint64(log_container_max_size);
 DECLARE_int64(fs_data_dirs_reserved_bytes);
 DECLARE_int64(disk_reserved_bytes_free_for_testing);
-DECLARE_int32(fs_data_dirs_full_disk_cache_seconds);
+DECLARE_int32(fs_data_dirs_available_space_cache_seconds);
 DECLARE_int32(fs_target_data_dirs_per_tablet);
 DECLARE_string(block_manager);
 DECLARE_double(env_inject_eio);
@@ -825,53 +825,31 @@ TYPED_TEST(BlockManagerTest, TestDiskSpaceCheck) {
                                      { GetTestDataDirectory() },
                                      false /* create */));
 
-  FLAGS_fs_data_dirs_full_disk_cache_seconds = 0; // Don't cache device fullness.
+  FLAGS_fs_data_dirs_available_space_cache_seconds = 0; // Don't cache device available space.
   FLAGS_fs_data_dirs_reserved_bytes = 1; // Keep at least 1 byte reserved in the FS.
   FLAGS_log_container_preallocate_bytes = 0; // Disable preallocation.
   FLAGS_fs_target_data_dirs_per_tablet = 3; // Use a subset of directories instead of all.
 
-  // Normally, a data dir is checked for fullness only after a block is closed;
-  // if it's now full, the next attempt at block creation will fail. Only when
-  // a data dir was last observed as full is it also checked before block creation.
-  //
-  // This behavior enforces a "soft" limit on disk space consumption but
-  // complicates testing somewhat.
-  bool data_dir_observed_full = false;
-
   int i = 0;
-  for (int free_space : { 0, 2, 0 }) {
+  for (int free_space : { 0, 2 }) {
     FLAGS_disk_reserved_bytes_free_for_testing = free_space;
 
-    for (int attempt = 0; attempt < 3; attempt++) {
+    for (int attempt = 0; attempt < 2; attempt++) {
       unique_ptr<WritableBlock> writer;
       LOG(INFO) << "Attempt #" << ++i;
       Status s = this->bm_->CreateBlock(this->test_block_opts_, &writer);
       if (FLAGS_disk_reserved_bytes_free_for_testing < FLAGS_fs_data_dirs_reserved_bytes) {
-        if (data_dir_observed_full) {
-          // The dir was previously observed as full, so CreateBlock() checked
-          // fullness again and failed.
-          ASSERT_TRUE(s.IsIOError()) << s.ToString();
-          ASSERT_STR_CONTAINS(
-              s.ToString(), "No directories available in test_tablet's directory group");
-        } else {
-          ASSERT_OK(s);
-          ASSERT_OK(writer->Append("test data"));
-          ASSERT_OK(writer->Close());
-
-          // The dir was not previously full so CreateBlock() did not check for
-          // fullness, but given the parameters of the test, we know that the
-          // dir was observed as full at Close().
-          data_dir_observed_full = true;
-        }
-        ASSERT_EQ(1, down_cast<AtomicGauge<uint64_t>*>(
-            entity->FindOrNull(METRIC_data_dirs_full).get())->value());
+        // The dir was previously observed as full, so CreateBlock() checked
+        // fullness again and failed.
+        ASSERT_TRUE(s.IsIOError()) << s.ToString();
+        ASSERT_STR_CONTAINS(
+            s.ToString(), "No directories available in test_tablet's directory group");
       } else {
         // CreateBlock() succeeded regardless of the previously fullness state,
         // and the new state is definitely not full.
         ASSERT_OK(s);
         ASSERT_OK(writer->Append("test data"));
         ASSERT_OK(writer->Close());
-        data_dir_observed_full = false;
         ASSERT_EQ(0, down_cast<AtomicGauge<uint64_t>*>(
             entity->FindOrNull(METRIC_data_dirs_full).get())->value());
       }
diff --git a/src/kudu/fs/data_dirs-test.cc b/src/kudu/fs/data_dirs-test.cc
index f0955bd..078a884 100644
--- a/src/kudu/fs/data_dirs-test.cc
+++ b/src/kudu/fs/data_dirs-test.cc
@@ -53,7 +53,7 @@ using strings::Substitute;
 
 DECLARE_bool(crash_on_eio);
 DECLARE_double(env_inject_eio);
-DECLARE_int32(fs_data_dirs_full_disk_cache_seconds);
+DECLARE_int32(fs_data_dirs_available_space_cache_seconds);
 DECLARE_int32(fs_target_data_dirs_per_tablet);
 DECLARE_int64(disk_reserved_bytes_free_for_testing);
 DECLARE_int64(fs_data_dirs_reserved_bytes);
@@ -188,7 +188,7 @@ TEST_F(DataDirsTest, TestDeleteDataDirGroup) {
 }
 
 TEST_F(DataDirsTest, TestFullDisk) {
-  FLAGS_fs_data_dirs_full_disk_cache_seconds = 0;       // Don't cache device fullness.
+  FLAGS_fs_data_dirs_available_space_cache_seconds = 0; // Don't cache device available space.
   FLAGS_fs_data_dirs_reserved_bytes = 1;                // Reserved space.
   FLAGS_disk_reserved_bytes_free_for_testing = 0;       // Free space.
 
diff --git a/src/kudu/fs/data_dirs.cc b/src/kudu/fs/data_dirs.cc
index ed80c3f..f365e38 100644
--- a/src/kudu/fs/data_dirs.cc
+++ b/src/kudu/fs/data_dirs.cc
@@ -86,11 +86,10 @@ DEFINE_validator(fs_data_dirs_reserved_bytes, [](const char* /*n*/, int64_t v) {
 TAG_FLAG(fs_data_dirs_reserved_bytes, runtime);
 TAG_FLAG(fs_data_dirs_reserved_bytes, evolving);
 
-DEFINE_int32(fs_data_dirs_full_disk_cache_seconds, 30,
-             "Number of seconds we cache the full-disk status in the block manager. "
-             "During this time, writes to the corresponding root path will not be attempted.");
-TAG_FLAG(fs_data_dirs_full_disk_cache_seconds, advanced);
-TAG_FLAG(fs_data_dirs_full_disk_cache_seconds, evolving);
+DEFINE_int32(fs_data_dirs_available_space_cache_seconds, 10,
+             "Number of seconds we cache the available disk space in the block manager. ");
+TAG_FLAG(fs_data_dirs_available_space_cache_seconds, advanced);
+TAG_FLAG(fs_data_dirs_available_space_cache_seconds, evolving);
 
 DEFINE_bool(fs_lock_data_dirs, true,
             "Lock the data directories to prevent concurrent usage. "
@@ -223,7 +222,8 @@ DataDir::DataDir(Env* env,
       metadata_file_(std::move(metadata_file)),
       pool_(std::move(pool)),
       is_shutdown_(false),
-      is_full_(false) {
+      is_full_(false),
+      available_bytes_(0) {
 }
 
 DataDir::~DataDir() {
@@ -253,27 +253,28 @@ void DataDir::WaitOnClosures() {
   pool_->Wait();
 }
 
-Status DataDir::RefreshIsFull(RefreshMode mode) {
+Status DataDir::RefreshAvailableSpace(RefreshMode mode) {
   switch (mode) {
     case RefreshMode::EXPIRED_ONLY: {
       std::lock_guard<simple_spinlock> l(lock_);
-      DCHECK(last_check_is_full_.Initialized());
-      MonoTime expiry = last_check_is_full_ + MonoDelta::FromSeconds(
-          FLAGS_fs_data_dirs_full_disk_cache_seconds);
-      if (!is_full_ || MonoTime::Now() < expiry) {
+      DCHECK(last_space_check_.Initialized());
+      MonoTime expiry = last_space_check_ + MonoDelta::FromSeconds(
+          FLAGS_fs_data_dirs_available_space_cache_seconds);
+      if (MonoTime::Now() < expiry) {
         break;
       }
       FALLTHROUGH_INTENDED; // Root was previously full, check again.
     }
     case RefreshMode::ALWAYS: {
+      int64_t available_bytes_new;
       Status s = env_util::VerifySufficientDiskSpace(
-          env_, dir_, 0, FLAGS_fs_data_dirs_reserved_bytes);
+          env_, dir_, 0, FLAGS_fs_data_dirs_reserved_bytes, &available_bytes_new);
       bool is_full_new;
       if (PREDICT_FALSE(s.IsIOError() && s.posix_code() == ENOSPC)) {
         LOG(WARNING) << Substitute(
             "Insufficient disk space under path $0: creation of new data "
             "blocks under this path can be retried after $1 seconds: $2",
-            dir_, FLAGS_fs_data_dirs_full_disk_cache_seconds, s.ToString());
+            dir_, FLAGS_fs_data_dirs_available_space_cache_seconds, s.ToString());
         s = Status::OK();
         is_full_new = true;
       } else {
@@ -286,7 +287,8 @@ Status DataDir::RefreshIsFull(RefreshMode mode) {
           metrics_->data_dirs_full->IncrementBy(is_full_new ? 1 : -1);
         }
         is_full_ = is_full_new;
-        last_check_is_full_ = MonoTime::Now();
+        last_space_check_ = MonoTime::Now();
+        available_bytes_ = available_bytes_new;
       }
       break;
     }
@@ -860,7 +862,7 @@ Status DataDirManager::Open() {
     if (ContainsKey(failed_data_dirs_, uuid_idx)) {
       continue;
     }
-    Status refresh_status = dd->RefreshIsFull(DataDir::RefreshMode::ALWAYS);
+    Status refresh_status = dd->RefreshAvailableSpace(DataDir::RefreshMode::ALWAYS);
     if (PREDICT_FALSE(!refresh_status.ok())) {
       if (refresh_status.IsDiskFailure()) {
         RETURN_NOT_OK(MarkDataDirFailed(uuid_idx, refresh_status.ToString()));
@@ -951,6 +953,7 @@ Status DataDirManager::GetNextDataDir(const CreateBlockOptions& opts, DataDir**
   const vector<int>* group_uuid_indices;
   vector<int> valid_uuid_indices;
   DataDirGroup* group = nullptr;
+
   if (PREDICT_TRUE(!opts.tablet_id.empty())) {
     // Get the data dir group for the tablet.
     group = FindOrNull(group_by_tablet_map_, opts.tablet_id);
@@ -975,22 +978,31 @@ Status DataDirManager::GetNextDataDir(const CreateBlockOptions& opts, DataDir**
     AppendKeysFromMap(data_dir_by_uuid_idx_, &valid_uuid_indices);
     group_uuid_indices = &valid_uuid_indices;
   }
-  vector<int> random_indices(group_uuid_indices->size());
-  iota(random_indices.begin(), random_indices.end(), 0);
-  shuffle(random_indices.begin(), random_indices.end(), default_random_engine(rng_.Next()));
-
-  // Randomly select a member of the group that is not full.
-  for (int i : random_indices) {
-    int uuid_idx = (*group_uuid_indices)[i];
-    DataDir* candidate = FindOrDie(data_dir_by_uuid_idx_, uuid_idx);
-    Status s = candidate->RefreshIsFull(DataDir::RefreshMode::EXPIRED_ONLY);
+
+  // In a certain group of dirs, randomly choose two not full dirs and select the one
+  // with more free space.
+  vector<DataDir*> candidate_dirs;
+  for (auto uuid_idx : *group_uuid_indices) {
+    DataDir *candidate = FindOrDie(data_dir_by_uuid_idx_, uuid_idx);
+    Status s = candidate->RefreshAvailableSpace(DataDir::RefreshMode::EXPIRED_ONLY);
     WARN_NOT_OK(s, Substitute("failed to refresh fullness of $0", candidate->dir()));
     if (s.ok() && !candidate->is_full()) {
-      *dir = candidate;
-      return Status::OK();
+      candidate_dirs.emplace_back(candidate);
     }
   }
 
+  if (candidate_dirs.size() == 1) {
+    *dir = candidate_dirs[0];
+    return Status::OK();
+  }
+
+  if (candidate_dirs.size() >= 2) {
+    shuffle(candidate_dirs.begin(), candidate_dirs.end(), default_random_engine(rng_.Next()));
+    *dir = (candidate_dirs[0]->available_bytes() > candidate_dirs[1]->available_bytes()) ?
+            candidate_dirs[0] : candidate_dirs[1];
+    return Status::OK();
+  }
+
   // All healthy directories are full. Return an error.
   if (PREDICT_TRUE(!opts.tablet_id.empty())) {
     DCHECK(group);
@@ -1043,7 +1055,7 @@ void DataDirManager::GetDirsForGroupUnlocked(int target_size,
     if (ContainsKey(failed_data_dirs_, e.first)) {
       continue;
     }
-    Status s = e.second->RefreshIsFull(DataDir::RefreshMode::ALWAYS);
+    Status s = e.second->RefreshAvailableSpace(DataDir::RefreshMode::ALWAYS);
     WARN_NOT_OK(s, Substitute("failed to refresh fullness of $0", e.second->dir()));
     if (s.ok() && !e.second->is_full()) {
       // TODO(awong): If a disk is unhealthy at the time of group creation, the
@@ -1054,14 +1066,24 @@ void DataDirManager::GetDirsForGroupUnlocked(int target_size,
   }
   while (group_indices->size() < target_size && !candidate_indices.empty()) {
     shuffle(candidate_indices.begin(), candidate_indices.end(), default_random_engine(rng_.Next()));
-    if (candidate_indices.size() == 1 ||
-        FindOrDie(tablets_by_uuid_idx_map_, candidate_indices[0]).size() <
-            FindOrDie(tablets_by_uuid_idx_map_, candidate_indices[1]).size()) {
+    if (candidate_indices.size() == 1) {
       group_indices->push_back(candidate_indices[0]);
       candidate_indices.erase(candidate_indices.begin());
     } else {
-      group_indices->push_back(candidate_indices[1]);
-      candidate_indices.erase(candidate_indices.begin() + 1);
+      int tablets_in_first = FindOrDie(tablets_by_uuid_idx_map_, candidate_indices[0]).size();
+      int tablets_in_second = FindOrDie(tablets_by_uuid_idx_map_, candidate_indices[1]).size();
+      int selected_index = 0;
+      if (tablets_in_first == tablets_in_second) {
+        int64_t space_in_first = FindOrDie(data_dir_by_uuid_idx_,
+                                           candidate_indices[0])->available_bytes();
+        int64_t space_in_second = FindOrDie(data_dir_by_uuid_idx_,
+                                            candidate_indices[1])->available_bytes();
+        selected_index = space_in_first > space_in_second ? 0 : 1;
+      } else {
+        selected_index = tablets_in_first < tablets_in_second ? 0 : 1;
+      }
+      group_indices->push_back(candidate_indices[selected_index]);
+      candidate_indices.erase(candidate_indices.begin() + selected_index);
     }
   }
 }
diff --git a/src/kudu/fs/data_dirs.h b/src/kudu/fs/data_dirs.h
index 9bc3911..b7f7f0f 100644
--- a/src/kudu/fs/data_dirs.h
+++ b/src/kudu/fs/data_dirs.h
@@ -174,7 +174,7 @@ class DataDir {
     EXPIRED_ONLY,
     ALWAYS,
   };
-  Status RefreshIsFull(RefreshMode mode);
+  Status RefreshAvailableSpace(RefreshMode mode);
 
   DataDirFsType fs_type() const { return fs_type_; }
 
@@ -189,6 +189,11 @@ class DataDir {
     return is_full_;
   }
 
+  int64_t available_bytes() {
+    std::lock_guard<simple_spinlock> l(lock_);
+    return available_bytes_;
+  }
+
  private:
   Env* env_;
   DataDirMetrics* metrics_;
@@ -199,11 +204,14 @@ class DataDir {
 
   bool is_shutdown_;
 
-  // Protects 'last_check_is_full_' and 'is_full_'.
+  // Protects 'last_space_check_', 'is_full_' and available_bytes_.
   mutable simple_spinlock lock_;
-  MonoTime last_check_is_full_;
+  MonoTime last_space_check_;
   bool is_full_;
 
+  // The available bytes of this dir, updated by RefreshAvailableSpace.
+  int64_t available_bytes_;
+
   DISALLOW_COPY_AND_ASSIGN(DataDir);
 };
 
@@ -327,8 +335,9 @@ class DataDirManager {
   // and data dir to tablet set are cleared of all references to the tablet.
   void DeleteDataDirGroup(const std::string& tablet_id);
 
-  // Returns a random directory from the specfied option's data dir group. If
-  // there is no room in the group, returns an error.
+  // Returns a random directory, giving preference to the one with more free
+  // space in the specified option's data dir group. If there is no room in the
+  // group, returns an error.
   Status GetNextDataDir(const CreateBlockOptions& opts, DataDir** dir);
 
   // Finds the set of tablet_ids in the data dir specified by 'uuid_idx' and
@@ -460,7 +469,8 @@ class DataDirManager {
   // Selection is based on "The Power of Two Choices in Randomized Load
   // Balancing", selecting two directories randomly and choosing the one with
   // less load, quantified as the number of unique tablets in the directory.
-  // The resulting behavior fills directories that have fewer tablets stored on
+  // Ties are broken by choosing the directory with more free space. The
+  // resulting behavior fills directories that have fewer tablets stored on
   // them while not completely neglecting those with more tablets.
   //
   // 'group_indices' is an output that stores the list of uuid_indices to be
diff --git a/src/kudu/fs/file_block_manager.cc b/src/kudu/fs/file_block_manager.cc
index 8fee5be..6349579 100644
--- a/src/kudu/fs/file_block_manager.cc
+++ b/src/kudu/fs/file_block_manager.cc
@@ -340,7 +340,7 @@ Status FileWritableBlock::Append(const Slice& data) {
 Status FileWritableBlock::AppendV(ArrayView<const Slice> data) {
   DCHECK(state_ == CLEAN || state_ == DIRTY) << "Invalid state: " << state_;
   RETURN_NOT_OK_HANDLE_ERROR(writer_->AppendV(data));
-  RETURN_NOT_OK_HANDLE_ERROR(location_.data_dir()->RefreshIsFull(
+  RETURN_NOT_OK_HANDLE_ERROR(location_.data_dir()->RefreshAvailableSpace(
       DataDir::RefreshMode::ALWAYS));
   state_ = DIRTY;
 
diff --git a/src/kudu/fs/log_block_manager.cc b/src/kudu/fs/log_block_manager.cc
index 4b47739..e71aa86 100644
--- a/src/kudu/fs/log_block_manager.cc
+++ b/src/kudu/fs/log_block_manager.cc
@@ -1134,7 +1134,7 @@ Status LogBlockContainer::WriteVData(int64_t offset, ArrayView<const Slice> data
                                   return sum + curr.size();
                                 });
   if (offset + data_size > preallocated_offset_) {
-    RETURN_NOT_OK_HANDLE_ERROR(data_dir_->RefreshIsFull(DataDir::RefreshMode::ALWAYS));
+    RETURN_NOT_OK_HANDLE_ERROR(data_dir_->RefreshAvailableSpace(DataDir::RefreshMode::ALWAYS));
   }
   return Status::OK();
 }
@@ -1219,7 +1219,7 @@ Status LogBlockContainer::EnsurePreallocated(int64_t block_start_offset,
     int64_t off = std::max(preallocated_offset_, block_start_offset);
     int64_t len = FLAGS_log_container_preallocate_bytes;
     RETURN_NOT_OK_HANDLE_ERROR(data_file_->PreAllocate(off, len, RWFile::CHANGE_FILE_SIZE));
-    RETURN_NOT_OK_HANDLE_ERROR(data_dir_->RefreshIsFull(DataDir::RefreshMode::ALWAYS));
+    RETURN_NOT_OK_HANDLE_ERROR(data_dir_->RefreshAvailableSpace(DataDir::RefreshMode::ALWAYS));
     VLOG(2) << Substitute("Preallocated $0 bytes at offset $1 in container $2",
                           len, off, ToString());
 
diff --git a/src/kudu/integration-tests/disk_failure-itest.cc b/src/kudu/integration-tests/disk_failure-itest.cc
index 7123820..97350d9 100644
--- a/src/kudu/integration-tests/disk_failure-itest.cc
+++ b/src/kudu/integration-tests/disk_failure-itest.cc
@@ -208,12 +208,15 @@ class DiskErrorITest : public ExternalMiniClusterITestBase,
   FlagList InjectionFlags(ErrorType error, ExternalTabletServer* error_ts) const {
     FlagList injection_flags;
     switch (error) {
-      case ErrorType::DISK_FAILURE:
+      case ErrorType::DISK_FAILURE: {
         // Avoid injecting errors to the first data directory.
-        injection_flags.emplace_back("env_inject_eio_globs",
-            JoinPathSegments(error_ts->data_dirs()[1], "**"));
+        string data_dirs = Substitute("$0,$1",
+                                      JoinPathSegments(error_ts->data_dirs()[1], "**"),
+                                      JoinPathSegments(error_ts->data_dirs()[2], "**"));
+        injection_flags.emplace_back("env_inject_eio_globs", data_dirs);
         injection_flags.emplace_back("env_inject_eio", "1.0");
         break;
+      }
       case ErrorType::CFILE_CORRUPTION:
         injection_flags.emplace_back("cfile_inject_corruption", "1.0");
         break;
diff --git a/src/kudu/util/env_util.cc b/src/kudu/util/env_util.cc
index dde4138..2fc3b1d 100644
--- a/src/kudu/util/env_util.cc
+++ b/src/kudu/util/env_util.cc
@@ -127,22 +127,22 @@ static void OverrideBytesFreeWithTestingFlags(const string& path, int64_t* bytes
   }
 }
 
-Status VerifySufficientDiskSpace(Env *env, const std::string& path,
-                                 int64_t requested_bytes, int64_t reserved_bytes) {
+Status VerifySufficientDiskSpace(Env *env, const std::string& path, int64_t requested_bytes,
+                                 int64_t reserved_bytes, int64_t* available_bytes) {
   const int64_t kOnePercentReservation = -1;
   DCHECK_GE(requested_bytes, 0);
 
   SpaceInfo space_info;
   RETURN_NOT_OK(env->GetSpaceInfo(path, &space_info));
-  int64_t available_bytes = space_info.free_bytes;
+  int64_t free_bytes = space_info.free_bytes;
 
   // Allow overriding these values by tests.
   if (PREDICT_FALSE(FLAGS_disk_reserved_bytes_free_for_testing > -1)) {
-    available_bytes = FLAGS_disk_reserved_bytes_free_for_testing;
+    free_bytes = FLAGS_disk_reserved_bytes_free_for_testing;
   }
   if (PREDICT_FALSE(FLAGS_disk_reserved_override_prefix_1_bytes_free_for_testing != -1 ||
                     FLAGS_disk_reserved_override_prefix_2_bytes_free_for_testing != -1)) {
-    OverrideBytesFreeWithTestingFlags(path, &available_bytes);
+    OverrideBytesFreeWithTestingFlags(path, &free_bytes);
   }
 
   // If they requested a one percent reservation, calculate what that is in bytes.
@@ -150,10 +150,14 @@ Status VerifySufficientDiskSpace(Env *env, const std::string& path,
     reserved_bytes = space_info.capacity_bytes / 100;
   }
 
-  if (available_bytes - requested_bytes < reserved_bytes) {
+  if (available_bytes != nullptr) {
+    *available_bytes = free_bytes;
+  }
+
+  if (free_bytes - requested_bytes < reserved_bytes) {
     return Status::IOError(Substitute("Insufficient disk space to allocate $0 bytes under path $1 "
                                       "($2 bytes available vs $3 bytes reserved)",
-                                      requested_bytes, path, available_bytes, reserved_bytes),
+                                      requested_bytes, path, free_bytes, reserved_bytes),
                            "", ENOSPC);
   }
   return Status::OK();
diff --git a/src/kudu/util/env_util.h b/src/kudu/util/env_util.h
index de1cc3c..bba1e92 100644
--- a/src/kudu/util/env_util.h
+++ b/src/kudu/util/env_util.h
@@ -49,12 +49,15 @@ Status OpenFileForSequential(Env *env, const std::string &path,
                              std::shared_ptr<SequentialFile> *file);
 
 // Returns Status::IOError with POSIX code ENOSPC if there is not sufficient
-// disk space to write 'bytes' bytes to the file system represented by 'path'.
+// disk space to write 'requested_bytes' bytes to the file system represented by 'path'.
 // Otherwise returns OK.
 // If 'reserved_bytes' equals -1, it is interpreted as a 1% reservation. No
 // other values less than 0 are supported at this time.
-Status VerifySufficientDiskSpace(Env *env, const std::string& path,
-                                 int64_t requested_bytes, int64_t reserved_bytes);
+// If 'available_bytes' is not null, it will contain the amount of free disk space (in bytes)
+// in 'path' when the function finishes. This will happen even if the function returns IOError
+// with ENOSPC, but not on any other error.
+Status VerifySufficientDiskSpace(Env *env, const std::string& path, int64_t requested_bytes,
+                                 int64_t reserved_bytes, int64_t* available_bytes = nullptr);
 
 // Creates the directory given by 'path', unless it already exists.
 //