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/05/01 03:59:01 UTC

kudu git commit: env: Always read fully when reading files

Repository: kudu
Updated Branches:
  refs/heads/master 948770425 -> 3663e9a9d


env: Always read fully when reading files

In KUDU-9 env_util::ReadFully was added to ensure
short reads were retried until all data was read.

Later RWFile was implemented with \u201cread fully\u201d behavior by default.
(see a15c795360e32885c00442efacd2a345f993f425)

Given we almost always use ReadFully or expect the data to be
fully read regardless, this patch moves the \u201cread fully\u201d behavior
into the Read function so it is always used.

Change-Id: I735cd4cfca3c355226266ef4f0fdb57bf59dfe69
Reviewed-on: http://gerrit.cloudera.org:8080/6758
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/3663e9a9
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/3663e9a9
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/3663e9a9

Branch: refs/heads/master
Commit: 3663e9a9dd8a1b1a9f77a4bb1cbdec9159c313bf
Parents: 9487704
Author: Grant Henke <gr...@gmail.com>
Authored: Wed Apr 26 16:24:14 2017 -0500
Committer: Adar Dembo <ad...@cloudera.com>
Committed: Mon May 1 03:54:21 2017 +0000

----------------------------------------------------------------------
 src/kudu/consensus/log_util.cc                 | 24 +++----
 src/kudu/fs/file_block_manager.cc              |  2 +-
 src/kudu/fs/log_block_manager-test.cc          |  4 +-
 src/kudu/tserver/tablet_copy_client-test.cc    |  4 +-
 src/kudu/tserver/tablet_copy_service-test.cc   |  4 +-
 src/kudu/tserver/tablet_copy_source_session.cc |  2 +-
 src/kudu/tserver/tablet_copy_source_session.h  |  6 +-
 src/kudu/util/env-test.cc                      | 62 +++++-----------
 src/kudu/util/env_posix.cc                     | 79 +++++++++++----------
 src/kudu/util/env_util.cc                      | 36 ----------
 src/kudu/util/env_util.h                       | 15 ----
 src/kudu/util/pb_util-test.cc                  |  2 +-
 src/kudu/util/pb_util.cc                       |  2 +-
 13 files changed, 78 insertions(+), 164 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/3663e9a9/src/kudu/consensus/log_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log_util.cc b/src/kudu/consensus/log_util.cc
index c31c58a..17778b4 100644
--- a/src/kudu/consensus/log_util.cc
+++ b/src/kudu/consensus/log_util.cc
@@ -69,7 +69,6 @@ namespace kudu {
 namespace log {
 
 using consensus::OpId;
-using env_util::ReadFully;
 using std::vector;
 using std::shared_ptr;
 using std::unique_ptr;
@@ -416,8 +415,8 @@ Status ReadableLogSegment::ReadHeader() {
   LogSegmentHeaderPB header;
 
   // Read and parse the log segment header.
-  RETURN_NOT_OK_PREPEND(ReadFully(readable_file_.get(), kLogSegmentHeaderMagicAndHeaderLength,
-                                  header_size, &header_slice, header_space),
+  RETURN_NOT_OK_PREPEND(readable_file_->Read(kLogSegmentHeaderMagicAndHeaderLength,
+                                             header_size, &header_slice, header_space),
                         "Unable to read fully");
 
   RETURN_NOT_OK_PREPEND(pb_util::ParseFromArray(&header,
@@ -440,8 +439,7 @@ Status ReadableLogSegment::ReadHeader() {
 Status ReadableLogSegment::ReadHeaderMagicAndHeaderLength(uint32_t *len) {
   uint8_t scratch[kLogSegmentHeaderMagicAndHeaderLength];
   Slice slice;
-  RETURN_NOT_OK(ReadFully(readable_file_.get(), 0, kLogSegmentHeaderMagicAndHeaderLength,
-                          &slice, scratch));
+  RETURN_NOT_OK(readable_file_->Read(0, kLogSegmentHeaderMagicAndHeaderLength, &slice, scratch));
   RETURN_NOT_OK(ParseHeaderMagicAndHeaderLength(slice, len));
   return Status::OK();
 }
@@ -526,8 +524,8 @@ Status ReadableLogSegment::ReadFooter() {
   LogSegmentFooterPB footer;
 
   // Read and parse the log segment footer.
-  RETURN_NOT_OK_PREPEND(ReadFully(readable_file_.get(), footer_offset,
-                                  footer_size, &footer_slice, footer_space),
+  RETURN_NOT_OK_PREPEND(readable_file_->Read(footer_offset, footer_size,
+                                             &footer_slice, footer_space),
                         "Footer not found. Could not read fully.");
 
   RETURN_NOT_OK_PREPEND(pb_util::ParseFromArray(&footer,
@@ -544,11 +542,8 @@ Status ReadableLogSegment::ReadFooterMagicAndFooterLength(uint32_t *len) {
   Slice slice;
 
   CHECK_GT(file_size(), kLogSegmentFooterMagicAndFooterLength);
-  RETURN_NOT_OK(ReadFully(readable_file_.get(),
-                          file_size() - kLogSegmentFooterMagicAndFooterLength,
-                          kLogSegmentFooterMagicAndFooterLength,
-                          &slice,
-                          scratch));
+  RETURN_NOT_OK(readable_file_->Read(file_size() - kLogSegmentFooterMagicAndFooterLength,
+                                     kLogSegmentFooterMagicAndFooterLength, &slice, scratch));
 
   RETURN_NOT_OK(ParseFooterMagicAndFooterLength(slice, len));
   return Status::OK();
@@ -606,7 +601,7 @@ Status ReadableLogSegment::ScanForValidEntryHeaders(int64_t offset, bool* has_va
        offset += kChunkSize - entry_header_size()) {
     int rem = std::min<int64_t>(file_size() - offset, kChunkSize);
     Slice chunk;
-    RETURN_NOT_OK(ReadFully(readable_file().get(), offset, rem, &chunk, &buf[0]));
+    RETURN_NOT_OK(readable_file()->Read(offset, rem, &chunk, &buf[0]));
 
     // Optimization for the case where a chunk is all zeros -- this is common in the
     // case of pre-allocated files. This avoids a lot of redundant CRC calculation.
@@ -647,8 +642,7 @@ Status ReadableLogSegment::ReadEntryHeader(int64_t *offset, EntryHeader* header)
   const size_t header_size = entry_header_size();
   uint8_t scratch[header_size];
   Slice slice;
-  RETURN_NOT_OK_PREPEND(ReadFully(readable_file().get(), *offset, header_size,
-                                  &slice, scratch),
+  RETURN_NOT_OK_PREPEND(readable_file()->Read(*offset, header_size, &slice, scratch),
                         "Could not read log entry header");
 
   if (PREDICT_FALSE(!DecodeEntryHeader(slice, header))) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/3663e9a9/src/kudu/fs/file_block_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/file_block_manager.cc b/src/kudu/fs/file_block_manager.cc
index 642f9ca..713e7b1 100644
--- a/src/kudu/fs/file_block_manager.cc
+++ b/src/kudu/fs/file_block_manager.cc
@@ -447,7 +447,7 @@ Status FileReadableBlock::Read(uint64_t offset, size_t length,
                                Slice* result, uint8_t* scratch) const {
   DCHECK(!closed_.Load());
 
-  RETURN_NOT_OK(env_util::ReadFully(reader_.get(), offset, length, result, scratch));
+  RETURN_NOT_OK(reader_->Read(offset, length, result, scratch));
   if (block_manager_->metrics_) {
     block_manager_->metrics_->total_bytes_read->IncrementBy(length);
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/3663e9a9/src/kudu/fs/log_block_manager-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/log_block_manager-test.cc b/src/kudu/fs/log_block_manager-test.cc
index 0f01fc5..e906e97 100644
--- a/src/kudu/fs/log_block_manager-test.cc
+++ b/src/kudu/fs/log_block_manager-test.cc
@@ -30,14 +30,12 @@
 #include "kudu/gutil/strings/strip.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/strings/util.h"
-#include "kudu/util/env_util.h"
 #include "kudu/util/metrics.h"
 #include "kudu/util/path_util.h"
 #include "kudu/util/pb_util.h"
 #include "kudu/util/random.h"
 #include "kudu/util/test_util.h"
 
-using kudu::env_util::ReadFully;
 using kudu::pb_util::ReadablePBContainerFile;
 using std::string;
 using std::unique_ptr;
@@ -512,7 +510,7 @@ TEST_F(LogBlockManagerTest, TestMetadataTruncation) {
   ASSERT_OK(env_->NewRandomAccessFile(metadata_path, &meta_file));
   Slice result;
   gscoped_ptr<uint8_t[]> scratch(new uint8_t[latest_meta_size]);
-  ASSERT_OK(ReadFully(meta_file.get(), 0, latest_meta_size, &result, scratch.get()));
+  ASSERT_OK(meta_file->Read(0, latest_meta_size, &result, scratch.get()));
   string data = result.ToString();
   // Flip the high bit of the length field, which is a 4-byte little endian
   // unsigned integer. This will cause the length field to represent a large

http://git-wip-us.apache.org/repos/asf/kudu/blob/3663e9a9/src/kudu/tserver/tablet_copy_client-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_client-test.cc b/src/kudu/tserver/tablet_copy_client-test.cc
index 9547ee4..040acc3 100644
--- a/src/kudu/tserver/tablet_copy_client-test.cc
+++ b/src/kudu/tserver/tablet_copy_client-test.cc
@@ -81,8 +81,8 @@ Status TabletCopyClientTest::CompareFileContents(const string& path1, const stri
   faststring scratch1, scratch2;
   scratch1.resize(size1);
   scratch2.resize(size2);
-  RETURN_NOT_OK(env_util::ReadFully(file1.get(), 0, size1, &slice1, scratch1.data()));
-  RETURN_NOT_OK(env_util::ReadFully(file2.get(), 0, size2, &slice2, scratch2.data()));
+  RETURN_NOT_OK(file1->Read(0, size1, &slice1, scratch1.data()));
+  RETURN_NOT_OK(file2->Read(0, size2, &slice2, scratch2.data()));
   int result = strings::fastmemcmp_inlined(slice1.data(), slice2.data(), size1);
   if (result != 0) {
     return Status::Corruption("Files do not match");

http://git-wip-us.apache.org/repos/asf/kudu/blob/3663e9a9/src/kudu/tserver/tablet_copy_service-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_service-test.cc b/src/kudu/tserver/tablet_copy_service-test.cc
index fc88155..c71e49e 100644
--- a/src/kudu/tserver/tablet_copy_service-test.cc
+++ b/src/kudu/tserver/tablet_copy_service-test.cc
@@ -32,7 +32,6 @@
 #include "kudu/tserver/tserver_service.pb.h"
 #include "kudu/tserver/tserver_service.proxy.h"
 #include "kudu/util/crc.h"
-#include "kudu/util/env_util.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/pb_util.h"
 #include "kudu/util/stopwatch.h"
@@ -50,7 +49,6 @@ namespace tserver {
 using consensus::MaximumOpId;
 using consensus::MinimumOpId;
 using consensus::OpIdEquals;
-using env_util::ReadFully;
 using log::ReadableLogSegment;
 using rpc::ErrorStatusPB;
 using rpc::RpcController;
@@ -458,7 +456,7 @@ TEST_F(TabletCopyServiceTest, TestFetchLog) {
   int64_t size = segment->file_size();
   scratch.resize(size);
   Slice slice;
-  ASSERT_OK(ReadFully(segment->readable_file().get(), 0, size, &slice, scratch.data()));
+  ASSERT_OK(segment->readable_file()->Read(0, size, &slice, scratch.data()));
 
   AssertDataEqual(slice.data(), slice.size(), resp.chunk());
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/3663e9a9/src/kudu/tserver/tablet_copy_source_session.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_source_session.cc b/src/kudu/tserver/tablet_copy_source_session.cc
index c357673..fe9917d 100644
--- a/src/kudu/tserver/tablet_copy_source_session.cc
+++ b/src/kudu/tserver/tablet_copy_source_session.cc
@@ -218,7 +218,7 @@ static Status ReadFileChunkToBuf(const Info* info,
   data->resize(response_data_size);
   uint8_t* buf = reinterpret_cast<uint8_t*>(const_cast<char*>(data->data()));
   Slice slice;
-  Status s = info->ReadFully(offset, response_data_size, &slice, buf);
+  Status s = info->Read(offset, response_data_size, &slice, buf);
   if (PREDICT_FALSE(!s.ok())) {
     s = s.CloneAndPrepend(
         Substitute("Unable to read existing file for $0", data_name));

http://git-wip-us.apache.org/repos/asf/kudu/blob/3663e9a9/src/kudu/tserver/tablet_copy_source_session.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_source_session.h b/src/kudu/tserver/tablet_copy_source_session.h
index 7c6ba44..20f5a2e 100644
--- a/src/kudu/tserver/tablet_copy_source_session.h
+++ b/src/kudu/tserver/tablet_copy_source_session.h
@@ -58,8 +58,8 @@ struct ImmutableRandomAccessFileInfo {
                                 int64_t size)
       : readable(std::move(readable)), size(size) {}
 
-  Status ReadFully(uint64_t offset, int64_t size, Slice* data, uint8_t* scratch) const {
-    return env_util::ReadFully(readable.get(), offset, size, data, scratch);
+  Status Read(uint64_t offset, int64_t size, Slice* data, uint8_t* scratch) const {
+    return readable->Read(offset, size, data, scratch);
   }
 };
 
@@ -75,7 +75,7 @@ struct ImmutableReadableBlockInfo {
     size(size) {
   }
 
-  Status ReadFully(uint64_t offset, int64_t size, Slice* data, uint8_t* scratch) const {
+  Status Read(uint64_t offset, int64_t size, Slice* data, uint8_t* scratch) const {
     return readable->Read(offset, size, data, scratch);
   }
 };

http://git-wip-us.apache.org/repos/asf/kudu/blob/3663e9a9/src/kudu/util/env-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env-test.cc b/src/kudu/util/env-test.cc
index c326dcc..ee0b42a 100644
--- a/src/kudu/util/env-test.cc
+++ b/src/kudu/util/env-test.cc
@@ -50,6 +50,7 @@
 #endif
 
 DECLARE_bool(never_fsync);
+DECLARE_int32(env_inject_short_read_bytes);
 
 namespace kudu {
 
@@ -136,8 +137,7 @@ class TestEnv : public KuduTest {
   void ReadAndVerifyTestData(RandomAccessFile* raf, size_t offset, size_t n) {
     unique_ptr<uint8_t[]> scratch(new uint8_t[n]);
     Slice s;
-    ASSERT_OK(env_util::ReadFully(raf, offset, n, &s,
-                                         scratch.get()));
+    ASSERT_OK(raf->Read(offset, n, &s, scratch.get()));
     ASSERT_EQ(n, s.size());
     ASSERT_NO_FATAL_FAILURE(VerifyTestData(s, offset));
   }
@@ -180,7 +180,7 @@ class TestEnv : public KuduTest {
         }
         if (!fast) {
           // Verify as write. Note: this requires that file is pre-allocated, otherwise
-          // the ReadFully() fails with EINVAL.
+          // the Read() fails with EINVAL.
           ASSERT_NO_FATAL_FAILURE(ReadAndVerifyTestData(raf.get(), num_slices * slice_size * i,
                                                         num_slices * slice_size));
         }
@@ -369,47 +369,13 @@ TEST_F(TestEnv, TestTruncate) {
   ASSERT_OK(env_->NewRandomAccessFile(test_path, &raf));
   Slice s;
   unique_ptr<uint8_t[]> scratch(new uint8_t[size]);
-  ASSERT_OK(env_util::ReadFully(raf.get(), 0, size, &s, scratch.get()));
+  ASSERT_OK(raf->Read(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)
-      : wrapped_(std::move(wrapped)) {}
-
-  virtual Status Read(uint64_t offset, size_t n, Slice* result,
-                      uint8_t *scratch) const OVERRIDE {
-    CHECK_GT(n, 0);
-    // Divide the requested amount of data by a small integer,
-    // and issue the shorter read to the underlying file.
-    int short_n = n / ((rand() % 3) + 1);
-    if (short_n == 0) {
-      short_n = 1;
-    }
-
-    VLOG(1) << "Reading " << short_n << " instead of " << n;
-
-    return wrapped_->Read(offset, short_n, result, scratch);
-  }
-
-  virtual Status Size(uint64_t *size) const OVERRIDE {
-    return wrapped_->Size(size);
-  }
-
-  virtual const string& filename() const OVERRIDE { return wrapped_->filename(); }
-
-  virtual size_t memory_footprint() const OVERRIDE {
-    return wrapped_->memory_footprint();
-  }
-
- private:
-  const shared_ptr<RandomAccessFile> wrapped_;
-};
-
 // Write 'size' bytes of data to a file, with a simple pattern stored in it.
 static void WriteTestFile(Env* env, const string& path, size_t size) {
   shared_ptr<WritableFile> wf;
@@ -436,22 +402,26 @@ TEST_F(TestEnv, TestReadFully) {
   shared_ptr<RandomAccessFile> raf;
   ASSERT_OK(env_util::OpenFileForRandom(env, kTestPath, &raf));
 
-  ShortReadRandomAccessFile sr_raf(raf);
-
   const int kReadLength = 10000;
   Slice s;
   unique_ptr<uint8_t[]> scratch(new uint8_t[kReadLength]);
 
-  // Verify that ReadFully reads the whole requested data.
-  ASSERT_OK(env_util::ReadFully(&sr_raf, 0, kReadLength, &s, scratch.get()));
+  // Force a short read to half the data length
+  FLAGS_env_inject_short_read_bytes = kReadLength / 2;
+
+  // Verify that Read fully reads the whole requested data.
+  ASSERT_OK(raf->Read(0, kReadLength, &s, scratch.get()));
   ASSERT_EQ(s.data(), scratch.get()) << "Should have returned a contiguous copy";
   ASSERT_EQ(kReadLength, s.size());
 
   // Verify that the data read was correct.
   VerifyTestData(s, 0);
 
-  // Verify that ReadFully fails with an IOError at EOF.
-  Status status = env_util::ReadFully(&sr_raf, kFileSize - 100, 200, &s, scratch.get());
+  // Turn short reads off again
+  FLAGS_env_inject_short_read_bytes = 0;
+
+  // Verify that Read fails with an IOError at EOF.
+  Status status = raf->Read(kFileSize - 100, 200, &s, scratch.get());
   ASSERT_FALSE(status.ok());
   ASSERT_TRUE(status.IsIOError());
   ASSERT_STR_CONTAINS(status.ToString(), "EOF");
@@ -539,7 +509,7 @@ TEST_F(TestEnv, TestReopen) {
   ASSERT_EQ(first.length() + second.length(), size);
   Slice s;
   uint8_t scratch[size];
-  ASSERT_OK(env_util::ReadFully(reader.get(), 0, size, &s, scratch));
+  ASSERT_OK(reader->Read(0, size, &s, scratch));
   ASSERT_EQ(first + second, s.ToString());
 }
 
@@ -669,7 +639,7 @@ TEST_F(TestEnv, TestGlob) {
 
   for (const auto& matcher : matchers) {
     SCOPED_TRACE(Substitute("pattern: $0, expected matches: $1",
-                            matcher.first, matcher.second));
+                                     matcher.first, matcher.second));
     vector<string> matches;
     ASSERT_OK(env_->Glob(JoinPathSegments(dir, matcher.first), &matches));
     ASSERT_EQ(matcher.second, matches.size());

http://git-wip-us.apache.org/repos/asf/kudu/blob/3663e9a9/src/kudu/util/env_posix.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_posix.cc b/src/kudu/util/env_posix.cc
index 7f26f2f..11a0054 100644
--- a/src/kudu/util/env_posix.cc
+++ b/src/kudu/util/env_posix.cc
@@ -119,6 +119,10 @@ DEFINE_double(env_inject_io_error_on_write_or_preallocate, 0.0,
               "Fraction of the time that write or preallocate operations will fail");
 TAG_FLAG(env_inject_io_error_on_write_or_preallocate, hidden);
 
+DEFINE_int32(env_inject_short_read_bytes, 0,
+             "The number of bytes less than the requested bytes to read");
+TAG_FLAG(env_inject_short_read_bytes, hidden);
+
 using base::subtle::Atomic64;
 using base::subtle::Barrier_AtomicIncrement;
 using std::string;
@@ -191,7 +195,7 @@ class ScopedFdCloser {
   int fd_;
 };
 
-static Status IOError(const std::string& context, int err_number) {
+Status IOError(const std::string& context, int err_number) {
   switch (err_number) {
     case ENOENT:
       return Status::NotFound(context, ErrnoToString(err_number), err_number);
@@ -209,7 +213,7 @@ static Status IOError(const std::string& context, int err_number) {
   return Status::IOError(context, ErrnoToString(err_number), err_number);
 }
 
-static Status DoSync(int fd, const string& filename) {
+Status DoSync(int fd, const string& filename) {
   ThreadRestrictions::AssertIOAllowed();
   if (FLAGS_never_fsync) return Status::OK();
   if (FLAGS_env_use_fsync) {
@@ -228,7 +232,7 @@ static Status DoSync(int fd, const string& filename) {
   return Status::OK();
 }
 
-static Status DoOpen(const string& filename, Env::CreateMode mode, int* fd) {
+Status DoOpen(const string& filename, Env::CreateMode mode, int* fd) {
   ThreadRestrictions::AssertIOAllowed();
   int flags = O_RDWR;
   switch (mode) {
@@ -251,6 +255,39 @@ static Status DoOpen(const string& filename, Env::CreateMode mode, int* fd) {
   return Status::OK();
 }
 
+Status DoRead(int fd, const string& filename, uint64_t offset, size_t length,
+                     Slice* result, uint8_t *scratch) {
+  ThreadRestrictions::AssertIOAllowed();
+  size_t rem = length;
+  uint8_t* dst = scratch;
+  while (rem > 0) {
+    size_t req = rem;
+    // Inject a short read for testing
+    if (PREDICT_FALSE(FLAGS_env_inject_short_read_bytes > 0 && req == length)) {
+      DCHECK_LT(FLAGS_env_inject_short_read_bytes, req);
+      req -= FLAGS_env_inject_short_read_bytes;
+    }
+    ssize_t r;
+    RETRY_ON_EINTR(r, pread(fd, dst, req, offset));
+    if (r < 0) {
+      // An error: return a non-ok status.
+      return IOError(filename, errno);
+    }
+    if (r == 0) {
+      // EOF
+      return Status::IOError(Substitute("EOF trying to read $0 bytes at offset $1",
+                                        length, offset));
+    }
+    DCHECK_LE(r, rem);
+    dst += r;
+    rem -= r;
+    offset += r;
+  }
+  DCHECK_EQ(0, rem);
+  *result = Slice(scratch, length);
+  return Status::OK();
+}
+
 class PosixSequentialFile: public SequentialFile {
  private:
   std::string filename_;
@@ -303,16 +340,7 @@ class PosixRandomAccessFile: public RandomAccessFile {
 
   virtual Status Read(uint64_t offset, size_t n, Slice* result,
                       uint8_t *scratch) const OVERRIDE {
-    ThreadRestrictions::AssertIOAllowed();
-    Status s;
-    ssize_t r;
-    RETRY_ON_EINTR(r, pread(fd_, scratch, n, offset));
-    if (r < 0) {
-      // An error: return a non-ok status.
-      s = IOError(filename_, errno);
-    }
-    *result = Slice(scratch, r);
-    return s;
+    return DoRead(fd_, filename_, offset, n, result, scratch);
   }
 
   virtual Status Size(uint64_t *size) const OVERRIDE {
@@ -555,30 +583,7 @@ class PosixRWFile : public RWFile {
 
   virtual Status Read(uint64_t offset, size_t length,
                       Slice* result, uint8_t* scratch) const OVERRIDE {
-    ThreadRestrictions::AssertIOAllowed();
-    int rem = length;
-    uint8_t* dst = scratch;
-    while (rem > 0) {
-      ssize_t r;
-      RETRY_ON_EINTR(r, pread(fd_, dst, rem, offset));
-      if (r < 0) {
-        // An error: return a non-ok status.
-        return IOError(filename_, errno);
-      }
-      Slice this_result(dst, r);
-      DCHECK_LE(this_result.size(), rem);
-      if (this_result.size() == 0) {
-        // EOF
-        return Status::IOError(Substitute("EOF trying to read $0 bytes at offset $1",
-                                          length, offset));
-      }
-      dst += this_result.size();
-      rem -= this_result.size();
-      offset += this_result.size();
-    }
-    DCHECK_EQ(0, rem);
-    *result = Slice(scratch, length);
-    return Status::OK();
+    return DoRead(fd_, filename_, offset, length, result, scratch);
   }
 
   virtual Status Write(uint64_t offset, const Slice& data) OVERRIDE {

http://git-wip-us.apache.org/repos/asf/kudu/blob/3663e9a9/src/kudu/util/env_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_util.cc b/src/kudu/util/env_util.cc
index b471951..72e1b10 100644
--- a/src/kudu/util/env_util.cc
+++ b/src/kudu/util/env_util.cc
@@ -153,42 +153,6 @@ Status VerifySufficientDiskSpace(Env *env, const std::string& path,
   return Status::OK();
 }
 
-Status ReadFully(RandomAccessFile* file, uint64_t offset, size_t n,
-                 Slice* result, uint8_t* scratch) {
-
-  bool first_read = true;
-
-  int rem = n;
-  uint8_t* dst = scratch;
-  while (rem > 0) {
-    Slice this_result;
-    RETURN_NOT_OK(file->Read(offset, rem, &this_result, dst));
-    DCHECK_LE(this_result.size(), rem);
-    if (this_result.size() == 0) {
-      // EOF
-      return Status::IOError(Substitute("EOF trying to read $0 bytes at offset $1",
-                                        n, offset));
-    }
-
-    if (first_read && this_result.size() == n) {
-      // If it's the first read, we can return a zero-copy array.
-      *result = this_result;
-      return Status::OK();
-    }
-    first_read = false;
-
-    // Otherwise, we're going to have to do more reads and stitch
-    // each read together.
-    this_result.relocate(dst);
-    dst += this_result.size();
-    rem -= this_result.size();
-    offset += this_result.size();
-  }
-  DCHECK_EQ(0, rem);
-  *result = Slice(scratch, n);
-  return Status::OK();
-}
-
 Status CreateDirIfMissing(Env* env, const string& path, bool* created) {
   Status s = env->CreateDir(path);
   if (created != nullptr) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/3663e9a9/src/kudu/util/env_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/env_util.h b/src/kudu/util/env_util.h
index a2f96a4..5c93584 100644
--- a/src/kudu/util/env_util.h
+++ b/src/kudu/util/env_util.h
@@ -47,21 +47,6 @@ Status OpenFileForSequential(Env *env, const std::string &path,
 Status VerifySufficientDiskSpace(Env *env, const std::string& path,
                                  int64_t requested_bytes, int64_t reserved_bytes);
 
-// Read exactly 'n' bytes from the given file. If fewer than 'n' bytes
-// are read, returns an IOError. This differs from the underlying
-// RandomAccessFile::Read(), which may return a "short read".
-//
-// Similar to RandomAccessFile::Read(), '*result' is modified to point
-// to the bytes which were read. These bytes may be a copy placed in
-// the 'scratch' buffer, or result may point into the underlying file
-// (e.g. via mmap or other zero-copy mechanism).
-//
-// NOTE: even if this returns an error, some data _may_ be read into
-// the provided scratch buffer, but no guarantee that that will be the
-// case.
-Status ReadFully(RandomAccessFile* file, uint64_t offset, size_t n,
-                 Slice* result, uint8_t* scratch);
-
 // Creates the directory given by 'path', unless it already exists.
 //
 // If 'created' is not NULL, sets it to true if the directory was

http://git-wip-us.apache.org/repos/asf/kudu/blob/3663e9a9/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 a4b63a8..c1e81c7 100644
--- a/src/kudu/util/pb_util-test.cc
+++ b/src/kudu/util/pb_util-test.cc
@@ -149,7 +149,7 @@ Status TestPBUtil::BitFlipFileByteRange(const string& path, uint64_t offset, uin
     Slice slice;
     faststring scratch;
     scratch.resize(size);
-    RETURN_NOT_OK(env_util::ReadFully(file.get(), 0, size, &slice, scratch.data()));
+    RETURN_NOT_OK(file->Read(0, size, &slice, scratch.data()));
     buf.append(slice.data(), slice.size());
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/3663e9a9/src/kudu/util/pb_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/pb_util.cc b/src/kudu/util/pb_util.cc
index b159144..f8f8d5e 100644
--- a/src/kudu/util/pb_util.cc
+++ b/src/kudu/util/pb_util.cc
@@ -185,7 +185,7 @@ Status NonShortRead(T* file, uint64_t offset, uint64_t length, Slice* result, ui
 template<>
 Status NonShortRead<RandomAccessFile>(RandomAccessFile* file, uint64_t offset, uint64_t length,
                                       Slice* result, uint8_t* scratch) {
-  return env_util::ReadFully(file, offset, length, result, scratch);
+  return file->Read(offset, length, result, scratch);
 }
 
 template<>