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 2016/04/02 01:24:51 UTC

incubator-kudu git commit: env: Implement RWFile::Truncate()

Repository: incubator-kudu
Updated Branches:
  refs/heads/master c6e709f6e -> 1f014124d


env: Implement RWFile::Truncate()

This can replace direct calls to truncate(1) and ftruncate(1).

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


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

Branch: refs/heads/master
Commit: 1f014124d1dbd4a546976721062cfef251a19187
Parents: c6e709f
Author: Mike Percy <mp...@apache.org>
Authored: Thu Mar 31 16:24:33 2016 -0700
Committer: Mike Percy <mp...@apache.org>
Committed: Fri Apr 1 23:24:17 2016 +0000

----------------------------------------------------------------------
 src/kudu/util/env-test.cc      | 38 +++++++++++++++++++++++++++++++++++++
 src/kudu/util/env.h            | 29 ++++++++++++++++++++++------
 src/kudu/util/env_posix.cc     | 26 +++++++++++++++++++++++--
 src/kudu/util/memenv/memenv.cc |  8 ++++++++
 src/kudu/util/pb_util-test.cc  | 10 ++++++----
 5 files changed, 99 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/1f014124/src/kudu/util/env-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env-test.cc b/src/kudu/util/env-test.cc
index 03e2503..5079542 100644
--- a/src/kudu/util/env-test.cc
+++ b/src/kudu/util/env-test.cc
@@ -54,6 +54,7 @@ using std::string;
 using std::vector;
 
 static const uint64_t kOneMb = 1024 * 1024;
+static const uint64_t kTwoMb = 2 * kOneMb;
 
 class TestEnv : public KuduTest {
  public:
@@ -334,6 +335,43 @@ TEST_F(TestEnv, TestHolePunch) {
   ASSERT_EQ(size_on_disk - punch_amount, new_size_on_disk);
 }
 
+TEST_F(TestEnv, TestTruncate) {
+  LOG(INFO) << "Testing Truncate()";
+  string test_path = GetTestPath("test_env_wf");
+  gscoped_ptr<RWFile> file;
+  ASSERT_OK(env_->NewRWFile(test_path, &file));
+  uint64_t size;
+  ASSERT_OK(file->Size(&size));
+  ASSERT_EQ(0, size);
+
+  // Truncate to 2 MB (up).
+  ASSERT_OK(file->Truncate(kTwoMb));
+  ASSERT_OK(file->Size(&size));
+  ASSERT_EQ(kTwoMb, size);
+  ASSERT_OK(env_->GetFileSize(test_path, &size));
+  ASSERT_EQ(kTwoMb, size);
+
+  // Truncate to 1 MB (down).
+  ASSERT_OK(file->Truncate(kOneMb));
+  ASSERT_OK(file->Size(&size));
+  ASSERT_EQ(kOneMb, size);
+  ASSERT_OK(env_->GetFileSize(test_path, &size));
+  ASSERT_EQ(kOneMb, size);
+
+  ASSERT_OK(file->Close());
+
+  // Read the whole file. Ensure it is all zeroes.
+  gscoped_ptr<RandomAccessFile> raf;
+  ASSERT_OK(env_->NewRandomAccessFile(test_path, &raf));
+  Slice s;
+  gscoped_ptr<uint8_t[]> scratch(new uint8_t[size]);
+  ASSERT_OK(env_util::ReadFully(raf.get(), 0, size, &s, scratch.get()));
+  const uint8_t* data = s.data();
+  for (int i = 0; i < size; i++) {
+    ASSERT_EQ(0, data[i]) << "Not null at position " << i;
+  }
+}
+
 class ShortReadRandomAccessFile : public RandomAccessFile {
  public:
   explicit ShortReadRandomAccessFile(shared_ptr<RandomAccessFile> wrapped)

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/1f014124/src/kudu/util/env.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/env.h b/src/kudu/util/env.h
index 0d90ef3..5b3ff77 100644
--- a/src/kudu/util/env.h
+++ b/src/kudu/util/env.h
@@ -350,12 +350,6 @@ class WritableFile {
   WritableFile() { }
   virtual ~WritableFile();
 
-  // Pre-allocates 'size' bytes for the file in the underlying filesystem.
-  // size bytes are added to the current pre-allocated size or to the current
-  // offset, whichever is bigger. In no case is the file truncated by this
-  // operation.
-  virtual Status PreAllocate(uint64_t size) = 0;
-
   virtual Status Append(const Slice& data) = 0;
 
   // If possible, uses scatter-gather I/O to efficiently append
@@ -365,6 +359,18 @@ class WritableFile {
   // implementation source code (e.g., env_posix.cc)
   virtual Status AppendVector(const std::vector<Slice>& data_vector) = 0;
 
+  // Pre-allocates 'size' bytes for the file in the underlying filesystem.
+  // size bytes are added to the current pre-allocated size or to the current
+  // offset, whichever is bigger. In no case is the file truncated by this
+  // operation.
+  //
+  // On some implementations, preallocation is done without initializing the
+  // contents of the data blocks (as opposed to writing zeroes), requiring no
+  // IO to the data blocks.
+  //
+  // In no case is the file truncated by this operation.
+  virtual Status PreAllocate(uint64_t size) = 0;
+
   virtual Status Close() = 0;
 
   // Flush all dirty data (not metadata) to disk.
@@ -437,9 +443,20 @@ class RWFile {
   // beginning at 'offset'. It is safe to preallocate the same range
   // repeatedly; this is an idempotent operation.
   //
+  // On some implementations, preallocation is done without initializing the
+  // contents of the data blocks (as opposed to writing zeroes), requiring no
+  // IO to the data blocks. On such implementations, this is much faster than
+  // 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;
 
+  // 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
+  // size, the file length is extended, and the extended portion will contain
+  // null bytes ('\0').
+  virtual Status Truncate(uint64_t length) = 0;
+
   // Deallocates space given by 'offset' and length' from the file,
   // effectively "punching a hole" in it. The space will be reclaimed by
   // the filesystem and reads to that range will return zeroes. Useful

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/1f014124/src/kudu/util/env_posix.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_posix.cc b/src/kudu/util/env_posix.cc
index d568f76..fe964fa 100644
--- a/src/kudu/util/env_posix.cc
+++ b/src/kudu/util/env_posix.cc
@@ -65,6 +65,10 @@
 #define fread_unlocked fread
 #endif
 
+#define RETRY_ON_EINTR(ret, expr) do { \
+  ret = (expr); \
+} while ((ret == -1) && (errno == EINTR));
+
 // See KUDU-588 for details.
 DEFINE_bool(writable_file_use_fsync, false,
             "Use fsync(2) instead of fdatasync(2) for synchronizing dirty "
@@ -122,7 +126,9 @@ int fallocate(int fd, int mode, off_t offset, off_t len) {
 
   if (stat.st_size < size) {
     // fcntl does not change the file size, so set it if necessary.
-    return ftruncate(fd, size);
+    int ret;
+    RETRY_ON_EINTR(ret, ftruncate(fd, size));
+    return ret;
   }
   return 0;
 }
@@ -349,7 +355,9 @@ class PosixWritableFile : public WritableFile {
     // If we've allocated more space than we used, truncate to the
     // actual size of the file and perform Sync().
     if (filesize_ < pre_allocated_size_) {
-      if (ftruncate(fd_, filesize_) < 0) {
+      int ret;
+      RETRY_ON_EINTR(ret, ftruncate(fd_, filesize_));
+      if (ret != 0) {
         s = IOError(filename_, errno);
         pending_sync_ = true;
       }
@@ -553,6 +561,20 @@ class PosixRWFile : public RWFile {
     return Status::OK();
   }
 
+  virtual Status Truncate(uint64_t length) OVERRIDE {
+    TRACE_EVENT2("io", "PosixRWFile::Truncate", "path", filename_, "length", length);
+    ThreadRestrictions::AssertIOAllowed();
+    int ret;
+    RETRY_ON_EINTR(ret, ftruncate(fd_, length));
+    if (ret != 0) {
+      int err = errno;
+      return Status::IOError(Substitute("Unable to truncate file $0", filename_),
+                             Substitute("ftruncate() failed: $0", ErrnoToString(err)),
+                             err);
+    }
+    return Status::OK();
+  }
+
   virtual Status PunchHole(uint64_t offset, size_t length) OVERRIDE {
 #if defined(__linux__)
     TRACE_EVENT1("io", "PosixRWFile::PunchHole", "path", filename_);

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/1f014124/src/kudu/util/memenv/memenv.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/memenv/memenv.cc b/src/kudu/util/memenv/memenv.cc
index 92b2007..1af79df 100644
--- a/src/kudu/util/memenv/memenv.cc
+++ b/src/kudu/util/memenv/memenv.cc
@@ -94,6 +94,10 @@ class FileState : public RefCountedThreadSafe<FileState> {
     return s;
   }
 
+  Status Truncate(uint64_t length) {
+    return Status::NotSupported("Not implemented");
+  }
+
   Status Append(const Slice& data) {
     return AppendRaw(data.data(), data.size());
   }
@@ -309,6 +313,10 @@ class RWFileImpl : public RWFile {
     return Status::OK();
   }
 
+  virtual Status Truncate(uint64_t length) OVERRIDE {
+    return file_->Truncate(length);
+  }
+
   virtual Status PunchHole(uint64_t offset, size_t length) OVERRIDE {
     return Status::OK();
   }

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/1f014124/src/kudu/util/pb_util-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/pb_util-test.cc b/src/kudu/util/pb_util-test.cc
index 5e746e4..3696f51 100644
--- a/src/kudu/util/pb_util-test.cc
+++ b/src/kudu/util/pb_util-test.cc
@@ -199,10 +199,12 @@ TEST_F(TestPBUtil, TestPBContainerCorruption) {
   ASSERT_OK(CreateKnownGoodContainerFile());
   uint64_t known_good_size = 0;
   ASSERT_OK(env_->GetFileSize(path_, &known_good_size));
-  int ret = truncate(path_.c_str(), known_good_size - 2);
-  if (ret != 0) {
-    PLOG(ERROR) << "truncate() of file " << path_ << " failed";
-    FAIL();
+  {
+    gscoped_ptr<RWFile> file;
+    RWFileOptions opts;
+    opts.mode = Env::OPEN_EXISTING;
+    ASSERT_OK(env_->NewRWFile(opts, path_, &file));
+    ASSERT_OK(file->Truncate(known_good_size - 2));
   }
   s = ReadPBContainerFromPath(env_.get(), path_, &test_pb);
   ASSERT_TRUE(s.IsCorruption()) << "Should be incorrect size: " << path_ << ": " << s.ToString();