You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by mp...@apache.org on 2017/03/07 00:56:37 UTC

[1/2] kudu git commit: env: Add support for getting FS capacity

Repository: kudu
Updated Branches:
  refs/heads/master 45548c90c -> 578bf84bb


env: Add support for getting FS capacity

We need a way to get FS capacity in a follow-up patch. This change adds
that capability and changes the Env API to allow for fetching both
capacity and free bytes with a single call. This also allows us to fetch
both values with a single syscall. This API is inspired by
boost::filesystem::space().

This patch also removes the difference in the "free" space returned when
this API is invoked as a superuser vs. a non-privileged user. Now, only
the space available to non-privileged processes is returned. This is
also consistent with the boost API and makes the API more predictable.

Change-Id: Id43275876d3352f5cf943e24ed4513b9f2c131aa
Reviewed-on: http://gerrit.cloudera.org:8080/6255
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>
Reviewed-by: David Ribeiro Alves <dr...@apache.org>


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

Branch: refs/heads/master
Commit: 573e232de129bc9b6afff524bfae7f4e5bfc9e7b
Parents: 45548c9
Author: Mike Percy <mp...@apache.org>
Authored: Thu Mar 2 12:39:52 2017 -0800
Committer: Mike Percy <mp...@apache.org>
Committed: Tue Mar 7 00:56:13 2017 +0000

----------------------------------------------------------------------
 src/kudu/util/env-test.cc  | 41 +++++++++++++++++++++++++++--------------
 src/kudu/util/env.h        | 14 ++++++++++----
 src/kudu/util/env_posix.cc | 11 ++++-------
 src/kudu/util/env_util.cc  | 15 ++++++++-------
 4 files changed, 49 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/573e232d/src/kudu/util/env-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env-test.cc b/src/kudu/util/env-test.cc
index 01e0db6..a8e5f70 100644
--- a/src/kudu/util/env-test.cc
+++ b/src/kudu/util/env-test.cc
@@ -25,6 +25,7 @@
 #include <gtest/gtest.h>
 
 #include "kudu/gutil/bind.h"
+#include "kudu/gutil/strings/human_readable.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/strings/util.h"
 #include "kudu/util/env.h"
@@ -802,28 +803,40 @@ TEST_F(TestEnv, TestTempRWFile) {
   ASSERT_OK(env_->DeleteFile(path));
 }
 
-TEST_F(TestEnv, TestGetBytesFree) {
+// Test that when we write data to disk we see SpaceInfo.free_bytes go down.
+TEST_F(TestEnv, TestGetSpaceInfoFreeBytes) {
   const string kDataDir = GetTestPath("parent");
   const string kTestFilePath = JoinPathSegments(kDataDir, "testfile");
   const int kFileSizeBytes = 256;
-  int64_t orig_bytes_free;
-  int64_t cur_bytes_free;
   ASSERT_OK(env_->CreateDir(kDataDir));
 
-  // Loop several times in case there are concurrent tests running that are
-  // modifying the filesystem.
-  const int kIters = 100;
-  for (int i = 0; i < kIters; i++) {
+  // Loop in case there are concurrent tests running that are modifying the
+  // filesystem.
+  NO_FATALS(AssertEventually([&] {
     if (env_->FileExists(kTestFilePath)) {
-      ASSERT_OK(env_->DeleteFile(kTestFilePath));
+      ASSERT_OK(env_->DeleteFile(kTestFilePath)); // Clean up the previous iteration.
     }
-    ASSERT_OK(env_->GetBytesFree(kDataDir, &orig_bytes_free));
+    SpaceInfo before_space_info;
+    ASSERT_OK(env_->GetSpaceInfo(kDataDir, &before_space_info));
+
     NO_FATALS(WriteTestFile(env_, kTestFilePath, kFileSizeBytes));
-    ASSERT_OK(env_->GetBytesFree(kDataDir, &cur_bytes_free));
-    if (orig_bytes_free - cur_bytes_free >= kFileSizeBytes) break;
-  }
-  ASSERT_GE(orig_bytes_free - cur_bytes_free, kFileSizeBytes)
-      << "Failed after " << kIters << " attempts";
+
+    SpaceInfo after_space_info;
+    ASSERT_OK(env_->GetSpaceInfo(kDataDir, &after_space_info));
+    ASSERT_GE(before_space_info.free_bytes - after_space_info.free_bytes, kFileSizeBytes);
+  }));
+}
+
+// Basic sanity check for GetSpaceInfo().
+TEST_F(TestEnv, TestGetSpaceInfoBasicInvariants) {
+  string path = GetTestDataDirectory();
+  SpaceInfo space_info;
+  ASSERT_OK(env_->GetSpaceInfo(path, &space_info));
+  ASSERT_GT(space_info.capacity_bytes, 0);
+  ASSERT_LE(space_info.free_bytes, space_info.capacity_bytes);
+  VLOG(1) << "Path " << path << " has capacity "
+          << HumanReadableNumBytes::ToString(space_info.capacity_bytes)
+          << " (" << HumanReadableNumBytes::ToString(space_info.free_bytes) << " free)";
 }
 
 TEST_F(TestEnv, TestChangeDir) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/573e232d/src/kudu/util/env.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/env.h b/src/kudu/util/env.h
index 442a451..bbf8f9c 100644
--- a/src/kudu/util/env.h
+++ b/src/kudu/util/env.h
@@ -36,6 +36,12 @@ struct RandomAccessFileOptions;
 struct RWFileOptions;
 struct WritableFileOptions;
 
+// Returned by Env::GetSpaceInfo().
+struct SpaceInfo {
+  int64_t capacity_bytes; // Capacity of a filesystem, in bytes.
+  int64_t free_bytes;     // Bytes available to non-privileged processes.
+};
+
 class Env {
  public:
   // Governs if/how the file is created.
@@ -191,10 +197,10 @@ class Env {
   // *block_size. fname must exist but it may be a file or a directory.
   virtual Status GetBlockSize(const std::string& fname, uint64_t* block_size) = 0;
 
-  // Determine the number of bytes free on the filesystem specified by 'path'.
-  // "Free space" accounting on the underlying filesystem may be more coarse
-  // than single bytes.
-  virtual Status GetBytesFree(const std::string& path, int64_t* bytes_free) = 0;
+  // Determine the capacity and number of bytes free on the filesystem
+  // specified by 'path'. "Free space" accounting on the underlying filesystem
+  // may be more coarse than single bytes.
+  virtual Status GetSpaceInfo(const std::string& path, SpaceInfo* space_info) = 0;
 
   // Rename file src to target.
   virtual Status RenameFile(const std::string& src,

http://git-wip-us.apache.org/repos/asf/kudu/blob/573e232d/src/kudu/util/env_posix.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_posix.cc b/src/kudu/util/env_posix.cc
index 6158bdc..4da3d47 100644
--- a/src/kudu/util/env_posix.cc
+++ b/src/kudu/util/env_posix.cc
@@ -1020,15 +1020,12 @@ class PosixEnv : public Env {
     return Status::OK();
   }
 
-  virtual Status GetBytesFree(const string& path, int64_t* bytes_free) OVERRIDE {
-    TRACE_EVENT1("io", "PosixEnv::GetBytesFree", "path", path);
+  virtual Status GetSpaceInfo(const string& path, SpaceInfo* space_info) OVERRIDE {
+    TRACE_EVENT1("io", "PosixEnv::GetSpaceInfo", "path", path);
     struct statvfs buf;
     RETURN_NOT_OK(StatVfs(path, &buf));
-    if (geteuid() == 0) {
-      *bytes_free = buf.f_frsize * buf.f_bfree;
-    } else {
-      *bytes_free = buf.f_frsize * buf.f_bavail;
-    }
+    space_info->capacity_bytes = buf.f_frsize * buf.f_blocks;
+    space_info->free_bytes = buf.f_frsize * buf.f_bavail;
     return Status::OK();
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/573e232d/src/kudu/util/env_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_util.cc b/src/kudu/util/env_util.cc
index 178d391..4c7e93c 100644
--- a/src/kudu/util/env_util.cc
+++ b/src/kudu/util/env_util.cc
@@ -125,22 +125,23 @@ Status VerifySufficientDiskSpace(Env *env, const std::string& path,
                                  int64_t requested_bytes, int64_t reserved_bytes) {
   DCHECK_GE(requested_bytes, 0);
 
-  int64_t bytes_free;
-  RETURN_NOT_OK(env->GetBytesFree(path, &bytes_free));
+  SpaceInfo space_info;
+  RETURN_NOT_OK(env->GetSpaceInfo(path, &space_info));
+  int64_t available_bytes = space_info.free_bytes;
 
   // Allow overriding these values by tests.
   if (PREDICT_FALSE(FLAGS_disk_reserved_bytes_free_for_testing > -1)) {
-    bytes_free = FLAGS_disk_reserved_bytes_free_for_testing;
+    available_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, &bytes_free);
+    OverrideBytesFreeWithTestingFlags(path, &available_bytes);
   }
 
-  if (bytes_free - requested_bytes < reserved_bytes) {
+  if (available_bytes - requested_bytes < reserved_bytes) {
     return Status::IOError(Substitute("Insufficient disk space to allocate $0 bytes under path $1 "
-                                      "($2 bytes free vs $3 bytes reserved)",
-                                      requested_bytes, path, bytes_free, reserved_bytes),
+                                      "($2 bytes available vs $3 bytes reserved)",
+                                      requested_bytes, path, available_bytes, reserved_bytes),
                            "", ENOSPC);
   }
   return Status::OK();


[2/2] kudu git commit: Reserve 1% of disk space by default

Posted by mp...@apache.org.
Reserve 1% of disk space by default

Many people are not aware that Kudu is able to reserve a certain amount
of space per disk for non-Kudu usage and they don't set the
corresponding flags. Let's bump the default from 0 to a special value
indicating one percent, which seems like a reasonable default in most
cases.

Change-Id: I578afeefd9a520fd56bfca597c5fcec8f0f3f98d
Reviewed-on: http://gerrit.cloudera.org:8080/6192
Tested-by: Kudu Jenkins
Reviewed-by: Adar Dembo <ad...@cloudera.com>


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

Branch: refs/heads/master
Commit: 578bf84bb24ad386708c12cb598dbcb5c5b5c150
Parents: 573e232
Author: Mike Percy <mp...@apache.org>
Authored: Tue Feb 28 17:23:09 2017 -0800
Committer: Mike Percy <mp...@apache.org>
Committed: Tue Mar 7 00:56:21 2017 +0000

----------------------------------------------------------------------
 src/kudu/consensus/log.cc      | 10 ++++++++--
 src/kudu/fs/data_dirs.cc       | 10 ++++++++--
 src/kudu/util/env_util-test.cc | 32 +++++++++++++++++++++++++-------
 src/kudu/util/env_util.cc      |  6 ++++++
 src/kudu/util/env_util.h       |  3 ++-
 5 files changed, 49 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/578bf84b/src/kudu/consensus/log.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log.cc b/src/kudu/consensus/log.cc
index 716baa0..d51e326 100644
--- a/src/kudu/consensus/log.cc
+++ b/src/kudu/consensus/log.cc
@@ -116,8 +116,14 @@ DEFINE_double(log_inject_io_error_on_preallocate_fraction, 0.0,
 TAG_FLAG(log_inject_io_error_on_preallocate_fraction, unsafe);
 TAG_FLAG(log_inject_io_error_on_preallocate_fraction, runtime);
 
-DEFINE_int64(fs_wal_dir_reserved_bytes, 0,
-             "Number of bytes to reserve on the log directory filesystem for non-Kudu usage");
+DEFINE_int64(fs_wal_dir_reserved_bytes, -1,
+             "Number of bytes to reserve on the log directory filesystem for "
+             "non-Kudu usage. The default, which is represented by -1, is that "
+             "1% of the disk space on each disk will be reserved. Any other "
+             "value specified represents the number of bytes reserved and must "
+             "be greater than or equal to 0. Explicit percentages to reserve "
+             "are not currently supported");
+DEFINE_validator(fs_wal_dir_reserved_bytes, [](const char* /*n*/, int64_t v) { return v >= -1; });
 TAG_FLAG(fs_wal_dir_reserved_bytes, runtime);
 TAG_FLAG(fs_wal_dir_reserved_bytes, evolving);
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/578bf84b/src/kudu/fs/data_dirs.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/data_dirs.cc b/src/kudu/fs/data_dirs.cc
index 8453378..44e9f1b 100644
--- a/src/kudu/fs/data_dirs.cc
+++ b/src/kudu/fs/data_dirs.cc
@@ -51,8 +51,14 @@
 #include "kudu/util/stopwatch.h"
 #include "kudu/util/threadpool.h"
 
-DEFINE_int64(fs_data_dirs_reserved_bytes, 0,
-             "Number of bytes to reserve on each data directory filesystem for non-Kudu usage.");
+DEFINE_int64(fs_data_dirs_reserved_bytes, -1,
+             "Number of bytes to reserve on each data directory filesystem for "
+             "non-Kudu usage. The default, which is represented by -1, is that "
+             "1% of the disk space on each disk will be reserved. Any other "
+             "value specified represents the number of bytes reserved and must "
+             "be greater than or equal to 0. Explicit percentages to reserve "
+             "are not currently supported");
+DEFINE_validator(fs_data_dirs_reserved_bytes, [](const char* /*n*/, int64_t v) { return v >= -1; });
 TAG_FLAG(fs_data_dirs_reserved_bytes, runtime);
 TAG_FLAG(fs_data_dirs_reserved_bytes, evolving);
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/578bf84b/src/kudu/util/env_util-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_util-test.cc b/src/kudu/util/env_util-test.cc
index 7c79068..715f03b 100644
--- a/src/kudu/util/env_util-test.cc
+++ b/src/kudu/util/env_util-test.cc
@@ -33,7 +33,6 @@
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
-DECLARE_int64(disk_reserved_bytes);
 DECLARE_int64(disk_reserved_bytes_free_for_testing);
 
 using std::string;
@@ -47,20 +46,39 @@ namespace env_util {
 class EnvUtilTest: public KuduTest {
 };
 
+// Assert that Status 's' indicates there is not enough space left on the
+// device for the request.
+static void AssertNoSpace(const Status& s) {
+  ASSERT_TRUE(s.IsIOError());
+  ASSERT_EQ(ENOSPC, s.posix_code());
+  ASSERT_STR_CONTAINS(s.ToString(), "Insufficient disk space");
+}
+
 TEST_F(EnvUtilTest, TestDiskSpaceCheck) {
-  const int64_t kRequestedBytes = 0;
+  const int64_t kZeroRequestedBytes = 0;
+  const int64_t kRequestOnePercentReservation = -1;
   int64_t reserved_bytes = 0;
-  ASSERT_OK(VerifySufficientDiskSpace(env_, test_dir_, kRequestedBytes, reserved_bytes));
+  ASSERT_OK(VerifySufficientDiskSpace(env_, test_dir_, kZeroRequestedBytes, reserved_bytes));
+
+  // Check 1% reservation logic. We loop this in case there are other FS
+  // operations happening concurrent with this test.
+  NO_FATALS(AssertEventually([&] {
+    SpaceInfo space_info;
+    ASSERT_OK(env_->GetSpaceInfo(test_dir_, &space_info));
+    // Try for 1 less byte than 1% free. This request should be rejected.
+    int64_t target_free_bytes = (space_info.capacity_bytes / 100) - 1;
+    int64_t bytes_to_request = std::max(0L, space_info.free_bytes - target_free_bytes);
+    NO_FATALS(AssertNoSpace(VerifySufficientDiskSpace(env_, test_dir_, bytes_to_request,
+                                                      kRequestOnePercentReservation)));
+  }));
 
   // Make it seem as if the disk is full and specify that we should have
   // reserved 200 bytes. Even asking for 0 bytes should return an error
   // indicating we are out of space.
   FLAGS_disk_reserved_bytes_free_for_testing = 0;
   reserved_bytes = 200;
-  Status s = VerifySufficientDiskSpace(env_, test_dir_, kRequestedBytes, reserved_bytes);
-  ASSERT_TRUE(s.IsIOError());
-  ASSERT_EQ(ENOSPC, s.posix_code());
-  ASSERT_STR_CONTAINS(s.ToString(), "Insufficient disk space");
+  NO_FATALS(AssertNoSpace(VerifySufficientDiskSpace(env_, test_dir_, kZeroRequestedBytes,
+                                                    reserved_bytes)));
 }
 
 // Ensure that we can recursively create directories using both absolute and

http://git-wip-us.apache.org/repos/asf/kudu/blob/578bf84b/src/kudu/util/env_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_util.cc b/src/kudu/util/env_util.cc
index 4c7e93c..b471951 100644
--- a/src/kudu/util/env_util.cc
+++ b/src/kudu/util/env_util.cc
@@ -123,6 +123,7 @@ 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) {
+  const int64_t kOnePercentReservation = -1;
   DCHECK_GE(requested_bytes, 0);
 
   SpaceInfo space_info;
@@ -138,6 +139,11 @@ Status VerifySufficientDiskSpace(Env *env, const std::string& path,
     OverrideBytesFreeWithTestingFlags(path, &available_bytes);
   }
 
+  // If they requested a one percent reservation, calculate what that is in bytes.
+  if (reserved_bytes == kOnePercentReservation) {
+    reserved_bytes = space_info.capacity_bytes / 100;
+  }
+
   if (available_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)",

http://git-wip-us.apache.org/repos/asf/kudu/blob/578bf84b/src/kudu/util/env_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_util.h b/src/kudu/util/env_util.h
index 884b17c..a2f96a4 100644
--- a/src/kudu/util/env_util.h
+++ b/src/kudu/util/env_util.h
@@ -42,7 +42,8 @@ Status OpenFileForSequential(Env *env, const std::string &path,
 // 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'.
 // 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);