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

[2/2] kudu git commit: KUDU-1856: always truncate containers when they get full

KUDU-1856: always truncate containers when they get full

On a production cluster, I found that XFS's speculative preallocation
feature often winds up permanently growing LBM containers without increasing
their file sizes. It's not clear why this happens; I don't think we're
running afoul of any of the conditions spelled out in the XFS FAQ [1].
Nevertheless, on this cluster the preallocated space accounted for almost a
third of the total data consumption, so we need to address it.

This patch changes the existing LBM container truncation behavior such that
at startup, the container's file size no longer determines whether to
truncate. Instead, all full containers are truncated unconditionally. The
assumption here is that a no-op truncate should be free.

We don't always run on XFS, and triggering its speculative preallocation is
hard. But, we can approximate it by passing FALLOC_FL_KEEP_SIZE to
fallocate(), which grows the file without changing its size.

1. http://xfs.org/index.php/XFS_FAQ#Q:_Is_speculative_preallocation_permanent.3F

Change-Id: Ic959c59489a08f92efa2df5c85b22e56740f1346
Reviewed-on: http://gerrit.cloudera.org:8080/5852
Tested-by: Kudu Jenkins
Reviewed-by: Mike Percy <mp...@apache.org>
Reviewed-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/master
Commit: 0430ea630dd60b4bde2456e4d82f8e991188cc96
Parents: 88a6e14
Author: Adar Dembo <ad...@cloudera.com>
Authored: Tue Jan 31 19:34:17 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Thu Feb 9 03:44:09 2017 +0000

----------------------------------------------------------------------
 src/kudu/fs/block_manager-test.cc | 62 ++++++++++++++++++++++------------
 src/kudu/fs/data_dirs.cc          |  2 +-
 src/kudu/fs/fs_manager.cc         |  5 ++-
 src/kudu/fs/log_block_manager.cc  | 10 ++++--
 src/kudu/util/env.h               | 11 +++++-
 src/kudu/util/env_posix.cc        | 12 +++++--
 src/kudu/util/file_cache.cc       |  4 +--
 7 files changed, 75 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/0430ea63/src/kudu/fs/block_manager-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/block_manager-test.cc b/src/kudu/fs/block_manager-test.cc
index 9802b8b..425cd05 100644
--- a/src/kudu/fs/block_manager-test.cc
+++ b/src/kudu/fs/block_manager-test.cc
@@ -1053,34 +1053,54 @@ TEST_F(LogBlockManagerTest, TestPreallocationAndTruncation) {
   string fname;
   NO_FATALS(GetOnlyContainerDataFile(&fname));
   uint64_t size_after_append;
-  ASSERT_OK(env_->GetFileSize(fname, &size_after_append));
+  ASSERT_OK(env_->GetFileSizeOnDisk(fname, &size_after_append));
   ASSERT_EQ(FLAGS_log_container_preallocate_bytes, size_after_append);
 
   // Close it. The extra preallocated space should be truncated off the file.
   ASSERT_OK(writer->Close());
   uint64_t size_after_close;
-  ASSERT_OK(env_->GetFileSize(fname, &size_after_close));
+  ASSERT_OK(env_->GetFileSizeOnDisk(fname, &size_after_close));
   ASSERT_EQ(FLAGS_log_container_max_size, size_after_close);
 
-  // For the sake of testing, artificially double the file's size.
-  unique_ptr<RWFile> data_file;
-  RWFileOptions opts;
-  opts.mode = Env::OPEN_EXISTING;
-  ASSERT_OK(env_->NewRWFile(opts, fname, &data_file));
-  ASSERT_OK(data_file->PreAllocate(size_after_close, size_after_close));
-  uint64_t size_after_preallocate;
-  ASSERT_OK(env_->GetFileSize(fname, &size_after_preallocate));
-  ASSERT_EQ(size_after_close * 2, size_after_preallocate);
-
-  // Now reopen the block manager. It should notice that the container grew
-  // and truncate the extra preallocated space off again.
-  ASSERT_OK(ReopenBlockManager(scoped_refptr<MetricEntity>(),
-                               shared_ptr<MemTracker>(),
-                               { this->test_dir_ },
-                               false));
-  uint64_t size_after_reopen;
-  ASSERT_OK(env_->GetFileSize(fname, &size_after_reopen));
-  ASSERT_EQ(FLAGS_log_container_max_size, size_after_reopen);
+  // Now test the same startup behavior by artificially growing the file
+  // and reopening the block manager.
+  //
+  // Try preallocating in two ways: once with a change to the file size and
+  // once without. The second way serves as a proxy for XFS's speculative
+  // preallocation behavior, described in KUDU-1856.
+  for (RWFile::PreAllocateMode mode : {RWFile::CHANGE_FILE_SIZE,
+                                       RWFile::DONT_CHANGE_FILE_SIZE}) {
+    LOG(INFO) << "Pass " << mode;
+    unique_ptr<RWFile> data_file;
+    RWFileOptions opts;
+    opts.mode = Env::OPEN_EXISTING;
+    ASSERT_OK(env_->NewRWFile(opts, fname, &data_file));
+    ASSERT_OK(data_file->PreAllocate(size_after_close, size_after_close, mode));
+    uint64_t size_after_preallocate;
+    ASSERT_OK(env_->GetFileSizeOnDisk(fname, &size_after_preallocate));
+    ASSERT_EQ(size_after_close * 2, size_after_preallocate);
+
+    if (mode == RWFile::DONT_CHANGE_FILE_SIZE) {
+      // Some older versions of ext4 (such as on el6) do not appear to truncate
+      // unwritten preallocated space that extends beyond the file size. Let's
+      // coax them by writing a single byte into that space.
+      //
+      // Note: this doesn't invalidate the usefulness of this test, as it's
+      // quite possible for us to have written a little bit of data into XFS's
+      // speculative preallocated area.
+      ASSERT_OK(data_file->Write(size_after_close, "a"));
+    }
+
+    // Now reopen the block manager. It should notice that the container grew
+    // and truncate the extra preallocated space off again.
+    ASSERT_OK(ReopenBlockManager(scoped_refptr<MetricEntity>(),
+                                 shared_ptr<MemTracker>(),
+                                 { this->test_dir_ },
+                                 false));
+    uint64_t size_after_reopen;
+    ASSERT_OK(env_->GetFileSizeOnDisk(fname, &size_after_reopen));
+    ASSERT_EQ(FLAGS_log_container_max_size, size_after_reopen);
+  }
 }
 
 TYPED_TEST(BlockManagerTest, TestDiskSpaceCheck) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/0430ea63/src/kudu/fs/data_dirs.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/data_dirs.cc b/src/kudu/fs/data_dirs.cc
index c710407..8453378 100644
--- a/src/kudu/fs/data_dirs.cc
+++ b/src/kudu/fs/data_dirs.cc
@@ -109,7 +109,7 @@ Status CheckHolePunch(Env* env, const string& path) {
 
   // Preallocate it, making sure the file's size is what we'd expect.
   uint64_t sz;
-  RETURN_NOT_OK(file->PreAllocate(0, kFileSize));
+  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(

http://git-wip-us.apache.org/repos/asf/kudu/blob/0430ea63/src/kudu/fs/fs_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/fs_manager.cc b/src/kudu/fs/fs_manager.cc
index c3a7e99..2fa6e65 100644
--- a/src/kudu/fs/fs_manager.cc
+++ b/src/kudu/fs/fs_manager.cc
@@ -48,6 +48,7 @@
 #include "kudu/util/oid_generator.h"
 #include "kudu/util/path_util.h"
 #include "kudu/util/pb_util.h"
+#include "kudu/util/stopwatch.h"
 
 DEFINE_bool(enable_data_block_fsync, true,
             "Whether to enable fsync() of data blocks, metadata, and their parent directories. "
@@ -247,7 +248,9 @@ Status FsManager::Open() {
     }
   }
 
-  RETURN_NOT_OK(block_manager_->Open());
+  LOG_TIMING(INFO, "opening block manager") {
+    RETURN_NOT_OK(block_manager_->Open());
+  }
   LOG(INFO) << "Opened local filesystem: " << JoinStrings(canonicalized_all_fs_roots_, ",")
             << std::endl << SecureDebugString(*metadata_);
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/kudu/blob/0430ea63/src/kudu/fs/log_block_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/log_block_manager.cc b/src/kudu/fs/log_block_manager.cc
index b77db85..f1d76b1 100644
--- a/src/kudu/fs/log_block_manager.cc
+++ b/src/kudu/fs/log_block_manager.cc
@@ -512,7 +512,13 @@ Status LogBlockContainer::Open(LogBlockManager* block_manager,
 }
 
 Status LogBlockContainer::TruncateDataToTotalBytesWritten() {
-  if (full() && preallocated_offset_ > total_bytes_written_) {
+  // Truncation is performed even if the container's logical file size
+  // (available by proxy via preallocated_offset_) and total_bytes_written_
+  // agree, because XFS's speculative preallocation feature may artificially
+  // enlarge the file without updating its file size.
+  //
+  // See KUDU-1856 for more details.
+  if (full()) {
     VLOG(2) << Substitute("Truncating container $0 to offset $1",
                           ToString(), total_bytes_written_);
     RETURN_NOT_OK(data_file_->Truncate(total_bytes_written_));
@@ -745,7 +751,7 @@ Status LogBlockContainer::EnsurePreallocated(int64_t block_start_offset,
       next_append_length > preallocated_offset_ - block_start_offset) {
     int64_t off = std::max(preallocated_offset_, block_start_offset);
     int64_t len = FLAGS_log_container_preallocate_bytes;
-    RETURN_NOT_OK(data_file_->PreAllocate(off, len));
+    RETURN_NOT_OK(data_file_->PreAllocate(off, len, RWFile::CHANGE_FILE_SIZE));
     RETURN_NOT_OK(data_dir_->RefreshIsFull(DataDir::RefreshMode::ALWAYS));
     VLOG(2) << Substitute("Preallocated $0 bytes at offset $1 in container $2",
                           len, off, ToString());

http://git-wip-us.apache.org/repos/asf/kudu/blob/0430ea63/src/kudu/util/env.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/env.h b/src/kudu/util/env.h
index 78d4bb5..c14d979 100644
--- a/src/kudu/util/env.h
+++ b/src/kudu/util/env.h
@@ -515,7 +515,16 @@ class RWFile {
   // using Truncate() to increase the file size.
   //
   // In no case is the file truncated by this operation.
-  virtual Status PreAllocate(uint64_t offset, size_t length) = 0;
+  //
+  // 'mode' controls whether the file's logical size grows to include the
+  // preallocated space, or whether it remains the same.
+  enum PreAllocateMode {
+    CHANGE_FILE_SIZE,
+    DONT_CHANGE_FILE_SIZE
+  };
+  virtual Status PreAllocate(uint64_t offset,
+                             size_t length,
+                             PreAllocateMode mode) = 0;
 
   // Truncate the file. If 'new_size' is less than the previous file size, the
   // extra data will be lost. If 'new_size' is greater than the previous file

http://git-wip-us.apache.org/repos/asf/kudu/blob/0430ea63/src/kudu/util/env_posix.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_posix.cc b/src/kudu/util/env_posix.cc
index 0a7b4bd..76f0804 100644
--- a/src/kudu/util/env_posix.cc
+++ b/src/kudu/util/env_posix.cc
@@ -134,7 +134,7 @@ namespace {
 #if defined(__APPLE__)
 // Simulates Linux's fallocate file preallocation API on OS X.
 int fallocate(int fd, int mode, off_t offset, off_t len) {
-  CHECK(mode == 0);
+  CHECK_EQ(mode, 0);
   off_t size = offset + len;
 
   struct stat stat;
@@ -601,13 +601,19 @@ class PosixRWFile : public RWFile {
     return Status::OK();
   }
 
-  virtual Status PreAllocate(uint64_t offset, size_t length) OVERRIDE {
+  virtual Status PreAllocate(uint64_t offset,
+                             size_t length,
+                             PreAllocateMode mode) OVERRIDE {
     MAYBE_RETURN_FAILURE(FLAGS_env_inject_io_error_on_write_or_preallocate,
                          Status::IOError(Env::kInjectedFailureStatusMsg));
 
     TRACE_EVENT1("io", "PosixRWFile::PreAllocate", "path", filename_);
     ThreadRestrictions::AssertIOAllowed();
-    if (fallocate(fd_, 0, offset, length) < 0) {
+    int falloc_mode = 0;
+    if (mode == DONT_CHANGE_FILE_SIZE) {
+      falloc_mode = FALLOC_FL_KEEP_SIZE;
+    }
+    if (fallocate(fd_, falloc_mode, offset, length) < 0) {
       if (errno == EOPNOTSUPP) {
         KLOG_FIRST_N(WARNING, 1) << "The filesystem does not support fallocate().";
       } else if (errno == ENOSYS) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/0430ea63/src/kudu/util/file_cache.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/file_cache.cc b/src/kudu/util/file_cache.cc
index d277937..7e3519a 100644
--- a/src/kudu/util/file_cache.cc
+++ b/src/kudu/util/file_cache.cc
@@ -219,10 +219,10 @@ class Descriptor<RWFile> : public RWFile {
     return opened.file()->Write(offset, data);
   }
 
-  Status PreAllocate(uint64_t offset, size_t length) override {
+  Status PreAllocate(uint64_t offset, size_t length, PreAllocateMode mode) override {
     ScopedOpenedDescriptor<RWFile> opened(&base_);
     RETURN_NOT_OK(ReopenFileIfNecessary(&opened));
-    return opened.file()->PreAllocate(offset, length);
+    return opened.file()->PreAllocate(offset, length, mode);
   }
 
   Status Truncate(uint64_t length) override {