You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by to...@apache.org on 2016/12/01 01:15:06 UTC

[3/9] kudu git commit: KUDU-1764: truncate preallocated space off of full lbm containers

KUDU-1764: truncate preallocated space off of full lbm containers

The LBM maintains a preallocation window at the end of each container, but
it erroneously leaves the window intact even as a container becomes full.

This patch addresses that in two ways:
1. The window is truncated when a container becomes full, and
2. It is truncated at startup after a full container is loaded.

The first trims the window in real time, while the second handles both
existing deployments and the edge case where a tserver has crashed mid-trim.

The implementation uses total_bytes_written_ as the truncation offset, which
means it still leaves last_block_size % filesystem_block_size bytes behind.
This is equivalent to the internal fragmentation caused by aligning block
offsets to the nearest filesystem block size, so I don't think it matters.

I snuck in a change to remove the "filesystem does not support
preallocation" behavior from block-manager-test; it's not possible to create
an LBM on a filesystem sans hole punching; such a filesystem should support
preallocation by definition.

Change-Id: Ib9173f955e53d096bfd9d3ddacf4294846cff11a
Reviewed-on: http://gerrit.cloudera.org:8080/5254
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>
Tested-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/4aacaf6f
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/4aacaf6f
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/4aacaf6f

Branch: refs/heads/master
Commit: 4aacaf6fc57b26195634c09c0433a24f5d4a778e
Parents: 8524516
Author: Adar Dembo <ad...@cloudera.com>
Authored: Mon Nov 28 17:31:29 2016 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Nov 30 21:24:01 2016 +0000

----------------------------------------------------------------------
 src/kudu/fs/block_manager-test.cc | 65 +++++++++++++++++++++++++++-------
 src/kudu/fs/log_block_manager.cc  | 39 ++++++++++++++++++++
 2 files changed, 92 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/4aacaf6f/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 b484a4b..a854717 100644
--- a/src/kudu/fs/block_manager-test.cc
+++ b/src/kudu/fs/block_manager-test.cc
@@ -326,26 +326,22 @@ void BlockManagerTest<LogBlockManager>::RunLogContainerPreallocationTest() {
   ASSERT_OK(written_block->Append(kTestData));
   ASSERT_OK(written_block->Close());
 
-  // We expect the container size to either be equal to the test data size (if
-  // preallocation isn't supported) or equal to the preallocation amount, which
-  // we know is greater than the test data size.
+  // We expect the container size to be equal to the preallocation amount,
+  // which we know is greater than the test data size.
   string container_data_filename;
   NO_FATALS(GetOnlyContainerDataFile(&container_data_filename));
   uint64_t size;
   ASSERT_OK(env_->GetFileSizeOnDisk(container_data_filename, &size));
-  ASSERT_TRUE(size == kTestData.size() ||
-              size == FLAGS_log_container_preallocate_bytes);
+  ASSERT_EQ(FLAGS_log_container_preallocate_bytes, size);
 
-  // Upon writing a second block, we'd expect the container to either double in
-  // size (without preallocation) or remain the same size (with preallocation).
+  // Upon writing a second block, we'd expect the container to remain the same
+  // size.
   ASSERT_OK(bm_->CreateBlock(&written_block));
   ASSERT_OK(written_block->Append(kTestData));
   ASSERT_OK(written_block->Close());
   NO_FATALS(GetOnlyContainerDataFile(&container_data_filename));
   ASSERT_OK(env_->GetFileSizeOnDisk(container_data_filename, &size));
-  ASSERT_TRUE(size == kTestData.size() * 2 ||
-              size == FLAGS_log_container_preallocate_bytes);
-
+  ASSERT_EQ(FLAGS_log_container_preallocate_bytes, size);
 
   // Now reopen the block manager and create another block. The block manager
   // should be smart enough to reuse the previously preallocated amount.
@@ -358,8 +354,7 @@ void BlockManagerTest<LogBlockManager>::RunLogContainerPreallocationTest() {
   ASSERT_OK(written_block->Close());
   NO_FATALS(GetOnlyContainerDataFile(&container_data_filename));
   ASSERT_OK(env_->GetFileSizeOnDisk(container_data_filename, &size));
-  ASSERT_TRUE(size == kTestData.size() * 3 ||
-              size == FLAGS_log_container_preallocate_bytes);
+  ASSERT_EQ(FLAGS_log_container_preallocate_bytes, size);
 }
 
 template <>
@@ -1027,6 +1022,52 @@ TEST_F(LogBlockManagerTest, TestAppendExceedsPreallocation) {
   ASSERT_OK(writer->Append("hello world"));
 }
 
+TEST_F(LogBlockManagerTest, TestPreallocationAndTruncation) {
+  RETURN_NOT_LOG_BLOCK_MANAGER();
+
+  // Ensure preallocation window is greater than the container size itself.
+  FLAGS_log_container_max_size = 1024 * 1024;
+  FLAGS_log_container_preallocate_bytes = 32 * 1024 * 1024;
+
+  // Fill up one container.
+  gscoped_ptr<WritableBlock> writer;
+  ASSERT_OK(bm_->CreateBlock(&writer));
+  unique_ptr<uint8_t[]> data(new uint8_t[FLAGS_log_container_max_size]);
+  memset(data.get(), 0, FLAGS_log_container_max_size);
+  ASSERT_OK(writer->Append({ data.get(), FLAGS_log_container_max_size } ));
+  string fname;
+  NO_FATALS(GetOnlyContainerDataFile(&fname));
+  uint64_t size_after_append;
+  ASSERT_OK(env_->GetFileSize(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_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);
+}
+
 TYPED_TEST(BlockManagerTest, TestDiskSpaceCheck) {
   // Reopen the block manager with metrics enabled.
   MetricRegistry registry;

http://git-wip-us.apache.org/repos/asf/kudu/blob/4aacaf6f/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 874a282..cce6d84 100644
--- a/src/kudu/fs/log_block_manager.cc
+++ b/src/kudu/fs/log_block_manager.cc
@@ -234,6 +234,16 @@ class LogBlockContainer {
   // TODO(unknown): Add support to synchronize just a range.
   Status SyncMetadata();
 
+  // Truncates this container's data file to 'total_bytes_written_' if it is
+  // full. This effectively removes any preallocated but unused space.
+  //
+  // Should be called only when 'total_bytes_written_' is up-to-date with
+  // respect to the data on disk (i.e. after the container's records have
+  // been loaded), otherwise data may be lost!
+  //
+  // This function is thread unsafe.
+  Status TruncateDataToTotalBytesWritten();
+
   // Reads the container's metadata from disk, sanity checking and
   // returning the records.
   Status ReadContainerRecords(deque<BlockRecordPB>* records) const;
@@ -438,6 +448,15 @@ Status LogBlockContainer::Open(LogBlockManager* block_manager,
   return Status::OK();
 }
 
+Status LogBlockContainer::TruncateDataToTotalBytesWritten() {
+  if (full() && preallocated_offset_ > total_bytes_written_) {
+    VLOG(2) << Substitute("Truncating container $0 to offset $1",
+                          ToString(), total_bytes_written_);
+    RETURN_NOT_OK(data_file_->Truncate(total_bytes_written_));
+  }
+  return Status::OK();
+}
+
 Status LogBlockContainer::ReadContainerRecords(deque<BlockRecordPB>* records) const {
   string metadata_path = metadata_file_->filename();
   unique_ptr<RandomAccessFile> metadata_reader;
@@ -539,6 +558,14 @@ Status LogBlockContainer::FinishBlock(const Status& s, WritableBlock* block) {
                                      block->BytesAppended()));
   UpdateBytesWritten(0);
 
+  // Truncate the container if it's now full; any left over preallocated space
+  // is no longer needed.
+  //
+  // Note that this take places _after_ the container has been synced to disk.
+  // That's OK; truncation isn't needed for correctness, and in the event of a
+  // crash, it will be retried at startup.
+  RETURN_NOT_OK(TruncateDataToTotalBytesWritten());
+
   if (full() && block_manager()->metrics()) {
     block_manager()->metrics()->full_containers->Increment();
   }
@@ -1479,6 +1506,18 @@ void LogBlockManager::OpenDataDir(DataDir* dir,
       ProcessBlockRecord(r, container.get(), &blocks_in_container);
       max_block_id = std::max(max_block_id, r.block_id().id());
     }
+
+    // Having processed the block records, it is now safe to truncate the
+    // preallocated space off of the end of the container. This is a no-op for
+    // non-full containers, where excess preallocated space is expected to be
+    // (eventually) used.
+    s = container->TruncateDataToTotalBytesWritten();
+    if (!s.ok()) {
+      *result_status = s.CloneAndPrepend(Substitute(
+          "Could not truncate container $0", container->ToString()));
+      return;
+    }
+
     next_block_id_.StoreMax(max_block_id + 1);
 
     // Under the lock, merge this map into the main block map and add