This is an automated email from the ASF dual-hosted git repository.

achennaka pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
     new 3d5497cf2 [cfile] clean up on IndexBlock{Builder,Iterator,Reader}
3d5497cf2 is described below

commit 3d5497cf299ee18f937efb9176190c35a33a9215
Author: Alexey Serbin <ale...@apache.org>
AuthorDate: Wed Oct 25 15:15:13 2023 -0700

    [cfile] clean up on IndexBlock{Builder,Iterator,Reader}
    
    This patch contains the following updates:
    
      * the code in IndexBlockReader::Parse() now catches corruption
        of index block's trailer in a more robust manner
      * fixed index overruns and possible memory corruption in
        IndexBlockReader::GetKeyPointer() when working with not-so-expected
        input for an empty BTree file and when the data read from the index
        file is corrupted
      * removed a few unused fields
      * a few class member functions became static
      * added PREDICT_{TRUE,FALSE} macros for better branch prediction
        where appropriate
      * fixed const-correctness
      * changed CHECK() to DCHECK() for the code paths where the
        assertions might trigger due to the variation of control paths,
        but independent of the input data
      * added more DCHECK() assertions where appropriate
      * updated the code to conform to the project's current style guide
      * addressed a few Clang-Tidy warnings
      * other minor updates
    
    I also added a test to cover various error paths in
    IndexBlockReader::Parse() when parsing corrupted index blocks.
    
    Change-Id: If83dd132b577a481a2ddaa09e2657639f8b92c7d
    Reviewed-on: http://gerrit.cloudera.org:8080/20690
    Reviewed-by: Ashwani Raina <ara...@cloudera.com>
    Reviewed-by: Abhishek Chennaka <achenn...@cloudera.com>
    Tested-by: Abhishek Chennaka <achenn...@cloudera.com>
---
 src/kudu/cfile/index-test.cc  | 242 ++++++++++++++++++++++++++++++++++----
 src/kudu/cfile/index_block.cc | 263 +++++++++++++++++++++++++-----------------
 src/kudu/cfile/index_block.h  |  62 +++++-----
 src/kudu/cfile/index_btree.cc | 150 ++++++++++++------------
 src/kudu/cfile/index_btree.h  |  62 +++++-----
 5 files changed, 508 insertions(+), 271 deletions(-)

diff --git a/src/kudu/cfile/index-test.cc b/src/kudu/cfile/index-test.cc
index e676d75ee..0b61b8738 100644
--- a/src/kudu/cfile/index-test.cc
+++ b/src/kudu/cfile/index-test.cc
@@ -26,13 +26,17 @@
 #include <gtest/gtest.h>
 
 #include "kudu/cfile/block_pointer.h"
+#include "kudu/cfile/cfile.pb.h"
 #include "kudu/cfile/cfile_util.h"
 #include "kudu/cfile/index_block.h"
 #include "kudu/common/common.pb.h"
 #include "kudu/common/key_encoder.h"
 #include "kudu/gutil/endian.h"
+#include "kudu/util/coding.h"
+#include "kudu/util/coding-inl.h"
 #include "kudu/util/faststring.h"
 #include "kudu/util/hexdump.h"
+#include "kudu/util/protobuf_util.h"
 #include "kudu/util/slice.h"
 #include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
@@ -43,17 +47,17 @@ using std::unique_ptr;
 namespace kudu {
 namespace cfile {
 
-Status SearchInReaderString(const IndexBlockReader &reader,
-                            const string &search_key,
-                            BlockPointer *ptr, Slice *match) {
+Status SearchInReaderString(const IndexBlockReader& reader,
+                            const string& search_key,
+                            BlockPointer* ptr,
+                            Slice* match) {
 
   static faststring dst;
 
   unique_ptr<IndexBlockIterator> iter(reader.NewIterator());
   dst.clear();
   KeyEncoderTraits<BINARY, faststring>::Encode(search_key, &dst);
-  Status s = iter->SeekAtOrBefore(Slice(dst));
-  RETURN_NOT_OK(s);
+  RETURN_NOT_OK(iter->SeekAtOrBefore(Slice(dst)));
 
   *ptr = iter->GetCurrentBlockPointer();
   *match = iter->GetCurrentKey();
@@ -61,17 +65,17 @@ Status SearchInReaderString(const IndexBlockReader &reader,
 }
 
 
-Status SearchInReaderUint32(const IndexBlockReader &reader,
+Status SearchInReaderUint32(const IndexBlockReader& reader,
                             uint32_t search_key,
-                            BlockPointer *ptr, Slice *match) {
+                            BlockPointer* ptr,
+                            Slice* match) {
 
   static faststring dst;
 
   unique_ptr<IndexBlockIterator> iter(reader.NewIterator());
   dst.clear();
   KeyEncoderTraits<UINT32, faststring>::Encode(search_key, &dst);
-  Status s = iter->SeekAtOrBefore(Slice(dst));
-  RETURN_NOT_OK(s);
+  RETURN_NOT_OK(iter->SeekAtOrBefore(Slice(dst)));
 
   *ptr = iter->GetCurrentBlockPointer();
   *match = iter->GetCurrentKey();
@@ -79,32 +83,35 @@ Status SearchInReaderUint32(const IndexBlockReader &reader,
 }
 
 // Expects a Slice containing a big endian encoded int
-static uint32_t SliceAsUInt32(const Slice &slice) {
-  CHECK_EQ(slice.size(), 4);
-  uint32_t val;
+static uint32_t SliceAsUInt32(const Slice& slice) {
+  CHECK_EQ(4, slice.size());
+  uint32_t val = 0;
   memcpy(&val, slice.data(), slice.size());
   val = BigEndian::FromHost32(val);
   return val;
 }
 
-static void AddToIndex(IndexBlockBuilder *idx, uint32_t val,
-                       const BlockPointer &block_pointer) {
-
+static void AddToIndex(IndexBlockBuilder* idx,
+                       uint32_t val,
+                       const BlockPointer& block_pointer) {
   static faststring dst;
   dst.clear();
   KeyEncoderTraits<UINT32, faststring>::Encode(val, &dst);
   idx->Add(Slice(dst), block_pointer);
 }
 
+static void AddEmptyKeyToIndex(IndexBlockBuilder* idx,
+                               const BlockPointer& block_pointer) {
+  idx->Add({}, block_pointer);
+}
 
 // Test IndexBlockBuilder and IndexReader with integers
 TEST(TestIndexBuilder, TestIndexWithInts) {
 
   // Encode an index block.
-  WriterOptions opts;
-  IndexBlockBuilder idx(&opts, true);
+  IndexBlockBuilder idx(true);
 
-  const int EXPECTED_NUM_ENTRIES = 4;
+  static constexpr int EXPECTED_NUM_ENTRIES = 4;
 
   uint32_t i;
 
@@ -204,8 +211,7 @@ TEST(TestIndexBuilder, TestIndexWithInts) {
 }
 
 TEST(TestIndexBlock, TestIndexBlockWithStrings) {
-  WriterOptions opts;
-  IndexBlockBuilder idx(&opts, true);
+  IndexBlockBuilder idx(true);
 
   // Insert data for "hello-10" through "hello-40" by 10s
   const int EXPECTED_NUM_ENTRIES = 4;
@@ -302,8 +308,7 @@ TEST(TestIndexBlock, TestIndexBlockWithStrings) {
 // Test seeking around using the IndexBlockIterator class
 TEST(TestIndexBlock, TestIterator) {
   // Encode an index block with 1000 entries.
-  WriterOptions opts;
-  IndexBlockBuilder idx(&opts, true);
+  IndexBlockBuilder idx(true);
 
   for (int i = 0; i < 1000; i++) {
     uint32_t key = i * 10;
@@ -340,6 +345,199 @@ TEST(TestIndexBlock, TestIterator) {
   ASSERT_TRUE(iter->HasNext());
 }
 
+// Parse an empty index block and make sure the IndexBlockReader's API works
+// as expected.
+TEST(TestIndexBlock, EmptyBlock) {
+  IndexBlockBuilder idx(true);
+  ASSERT_TRUE(idx.empty());
+  ASSERT_EQ(0, idx.count());
+  Slice key;
+  const auto s = idx.GetFirstKey(&key);
+  ASSERT_TRUE(s.IsNotFound()) << s.ToString();
+
+  const Slice b = idx.Finish();
+  IndexBlockReader reader;
+  ASSERT_OK(reader.Parse(b));
+  ASSERT_TRUE(reader.IsLeaf());
+  ASSERT_EQ(0, reader.Count());
+  unique_ptr<IndexBlockIterator> it(reader.NewIterator());
+  ASSERT_FALSE(it->HasNext());
+  {
+    const auto s = it->SeekToIndex(0);
+    ASSERT_TRUE(s.IsNotFound()) << s.ToString();
+  }
+  {
+    const auto s = it->Next();
+    ASSERT_TRUE(s.IsNotFound()) << s.ToString();
+  }
+  {
+    const Slice key;
+    const auto s = it->SeekAtOrBefore(key);
+    ASSERT_TRUE(s.IsNotFound()) << s.ToString();
+  }
+}
+
+// Test how IndexBlockBuilder and IndexBlockReader work with empty keys
+// in an index block.
+TEST(TestIndexBlock, EmptyKeys) {
+  // All the sanity checks in the parser are based on min/max estimates,
+  // and there might be variations in the size of the encoded fields because
+  // of varint encoding, where higher values consume more space when 
serialized.
+  // That's exercised below by using different number of blocks in the 
generated
+  // index file.
+
+  // One empty key.
+  {
+    IndexBlockBuilder idx(true);
+    AddEmptyKeyToIndex(&idx, BlockPointer(0, 1));
+    const Slice b = idx.Finish();
+
+    IndexBlockReader reader;
+    ASSERT_OK(reader.Parse(b));
+  }
+
+  // Several empty keys.
+  {
+    IndexBlockBuilder idx(true);
+    for (auto i = 0; i < 8; ++i) {
+      AddEmptyKeyToIndex(&idx, BlockPointer(i, 1));
+    }
+    const Slice b = idx.Finish();
+
+    IndexBlockReader reader;
+    ASSERT_OK(reader.Parse(b));
+  }
+
+  // Many empty keys.
+  {
+    IndexBlockBuilder idx(true);
+    for (auto i = 0; i < 65536; ++i) {
+      AddEmptyKeyToIndex(&idx, BlockPointer(i, 1));
+    }
+    const Slice b = idx.Finish();
+
+    IndexBlockReader reader;
+    ASSERT_OK(reader.Parse(b));
+  }
+}
+
+// Corrupt the trailer or its size in the block and try to parse the block.
+TEST(TestIndexBlock, CorruptedTrailer) {
+  // Data chunk is too small.
+  {
+    uint8_t buf[3];
+    Slice b(buf, sizeof(buf));
+    IndexBlockReader reader;
+    const auto s = reader.Parse(b);
+    ASSERT_TRUE(s.IsCorruption()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(), "index block too small");
+  }
+
+  // The trailer's size is too small.
+  {
+    IndexBlockBuilder idx(true);
+    AddToIndex(&idx, 0, BlockPointer(1, 1024));
+    const Slice b = idx.Finish();
+
+    const uint8_t* trailer_size_ptr = b.data() + b.size() - sizeof(uint32_t);
+
+    faststring buf;
+    InlinePutFixed32(&buf, 3);
+    memmove(const_cast<uint8_t*>(trailer_size_ptr), buf.data(), buf.size());
+
+    IndexBlockReader reader;
+    const auto s = reader.Parse(b);
+    ASSERT_TRUE(s.IsCorruption()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(), "3: invalid index block trailer size");
+  }
+
+  // The trailer's size is too big.
+  {
+    IndexBlockBuilder idx(true);
+    AddToIndex(&idx, 1, BlockPointer(2, 1024));
+    const Slice b = idx.Finish();
+
+    const uint8_t* trailer_size_ptr = b.data() + b.size() - sizeof(uint32_t);
+
+    faststring buf;
+    InlinePutFixed32(&buf, 1234);
+    memmove(const_cast<uint8_t*>(trailer_size_ptr), buf.data(), buf.size());
+
+    IndexBlockReader reader;
+    const auto s = reader.Parse(b);
+    ASSERT_TRUE(s.IsCorruption()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(), "1234: invalid index block trailer 
size");
+  }
+
+  // The number of key offsets too high for the actual amount of data stored.
+  {
+    IndexBlockBuilder idx(true);
+    AddToIndex(&idx, 2, BlockPointer(3, 1024));
+    const Slice b = idx.Finish();
+
+    const uint8_t* trailer_size_ptr = b.data() + b.size() - sizeof(uint32_t);
+    const size_t trailer_size = DecodeFixed32(trailer_size_ptr);
+    const uint8_t* trailer_ptr = trailer_size_ptr - trailer_size;
+    IndexBlockTrailerPB t;
+    ASSERT_TRUE(t.ParseFromArray(trailer_ptr, trailer_size));
+    t.set_num_entries(5);
+
+    // To make sure writing over the new serialized message doesn't require
+    // updating the encoded trailer size, check that the new serialized
+    // PB message is the same size as the original message.
+    ASSERT_EQ(trailer_size, t.ByteSizeLong());
+
+    faststring buf;
+    ASSERT_TRUE(AppendPBToString(t, &buf));
+    memmove(const_cast<uint8_t*>(trailer_ptr), buf.data(), buf.size());
+
+    IndexBlockReader reader;
+    const auto s = reader.Parse(b);
+    ASSERT_TRUE(s.IsCorruption()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(), "5: too many entries in trailer");
+  }
+
+  // Negative number of key offsets.
+  {
+    IndexBlockBuilder idx(true);
+    AddToIndex(&idx, 3, BlockPointer(4, 1024));
+    const Slice b = idx.Finish();
+
+    IndexBlockTrailerPB t;
+    {
+      const uint8_t* trailer_size_ptr = b.data() + b.size() - sizeof(uint32_t);
+      const size_t trailer_size = DecodeFixed32(trailer_size_ptr);
+      const uint8_t* trailer_ptr = trailer_size_ptr - trailer_size;
+      ASSERT_TRUE(t.ParseFromArray(trailer_ptr, trailer_size));
+    }
+
+    IndexBlockTrailerPB tc(t);
+    tc.set_num_entries(-3);
+    // The new trailer serialized into a bigger message due to negative value
+    // of the int32_t field.
+    ASSERT_GT(tc.ByteSizeLong(), t.ByteSizeLong());
+
+    unique_ptr<uint8_t[]> bc_data(
+        new uint8_t[b.size() + tc.ByteSizeLong() - t.ByteSizeLong()]);
+    Slice bc(b);
+    bc.relocate(bc_data.get());
+
+    faststring buf;
+    ASSERT_TRUE(AppendPBToString(tc, &buf));
+    InlinePutFixed32(&buf, tc.ByteSizeLong());
+
+    // Overwrite the trailer and the trailer size.
+    memmove(bc.mutable_data() + bc.size() - tc.ByteSizeLong() - 
sizeof(uint32_t),
+            buf.data(),
+            buf.size());
+
+    IndexBlockReader reader;
+    const auto s = reader.Parse(bc);
+    ASSERT_TRUE(s.IsCorruption()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(), "-3: bad number of entries in trailer");
+  }
+}
+
 TEST(TestIndexKeys, TestGetSeparatingKey) {
   // Test example cases
   Slice left = "";
diff --git a/src/kudu/cfile/index_block.cc b/src/kudu/cfile/index_block.cc
index ecfef4bf3..c003d1e3e 100644
--- a/src/kudu/cfile/index_block.cc
+++ b/src/kudu/cfile/index_block.cc
@@ -19,7 +19,6 @@
 
 #include <cstdint>
 #include <ostream>
-#include <string>
 
 #include <glog/logging.h>
 
@@ -30,24 +29,28 @@
 #include "kudu/util/pb_util.h"
 #include "kudu/util/protobuf_util.h"
 
+using strings::Substitute;
+
 namespace kudu {
 namespace cfile {
 
-inline void SliceEncode(const Slice &key, faststring *buf) {
+inline void SliceEncode(const Slice& key, faststring* buf) {
   InlinePutVarint32(buf, key.size());
   buf->append(key.data(), key.size());
 }
 
-inline const uint8_t *SliceDecode(const uint8_t *encoded_ptr, const uint8_t 
*limit,
-                         Slice *retptr) {
+inline const uint8_t* SliceDecode(
+    const uint8_t* encoded_ptr,
+    const uint8_t* limit,
+    Slice* retptr) {
   uint32_t len;
-  const uint8_t *data_start = GetVarint32Ptr(encoded_ptr, limit, &len);
-  if (data_start == nullptr) {
+  const uint8_t* data_start = GetVarint32Ptr(encoded_ptr, limit, &len);
+  if (PREDICT_FALSE(!data_start)) {
     // bad varint
     return nullptr;
   }
 
-  if (data_start + len > limit) {
+  if (PREDICT_FALSE(data_start + len > limit)) {
     // length extends past end of valid area
     return nullptr;
   }
@@ -56,28 +59,23 @@ inline const uint8_t *SliceDecode(const uint8_t 
*encoded_ptr, const uint8_t *lim
   return data_start + len;
 }
 
-IndexBlockBuilder::IndexBlockBuilder(
-  const WriterOptions *options,
-  bool is_leaf)
-  : options_(options),
-    finished_(false),
-    is_leaf_(is_leaf) {
+IndexBlockBuilder::IndexBlockBuilder(bool is_leaf)
+    : finished_(false),
+      is_leaf_(is_leaf) {
 }
 
+void IndexBlockBuilder::Add(const Slice& keyptr,
+                            const BlockPointer& ptr) {
+  DCHECK(!finished_) << "Must Reset() after Finish() before more Add()";
 
-void IndexBlockBuilder::Add(const Slice &keyptr,
-                            const BlockPointer &ptr) {
-  DCHECK(!finished_) <<
-    "Must Reset() after Finish() before more Add()";
-
-  size_t entry_offset = buffer_.size();
+  const size_t entry_offset = buffer_.size();
   SliceEncode(keyptr, &buffer_);
   ptr.EncodeTo(&buffer_);
   entry_offsets_.push_back(entry_offset);
 }
 
 Slice IndexBlockBuilder::Finish() {
-  CHECK(!finished_) << "already called Finish()";
+  DCHECK(!finished_) << "already called Finish()";
 
   for (uint32_t off : entry_offsets_) {
     InlinePutFixed32(&buffer_, off);
@@ -85,8 +83,8 @@ Slice IndexBlockBuilder::Finish() {
 
   IndexBlockTrailerPB trailer;
   trailer.set_num_entries(entry_offsets_.size());
-  trailer.set_type(
-    is_leaf_ ? IndexBlockTrailerPB::LEAF : IndexBlockTrailerPB::INTERNAL);
+  trailer.set_type(is_leaf_ ? IndexBlockTrailerPB::LEAF
+                            : IndexBlockTrailerPB::INTERNAL);
   AppendPBToString(trailer, &buffer_);
 
   InlinePutFixed32(&buffer_, trailer.GetCachedSize());
@@ -95,28 +93,23 @@ Slice IndexBlockBuilder::Finish() {
   return Slice(buffer_);
 }
 
-
 // Return the key of the first entry in this index block.
-Status IndexBlockBuilder::GetFirstKey(Slice *key) const {
-  // TODO: going to need to be able to pass an arena or something
+Status IndexBlockBuilder::GetFirstKey(Slice* key) const {
+  // TODO(todd): going to need to be able to pass an arena or something
   // for slices, which need to copy
-
-  if (entry_offsets_.empty()) {
+  if (PREDICT_FALSE(entry_offsets_.empty())) {
     return Status::NotFound("no keys in builder");
   }
-
-  bool success = nullptr != SliceDecode(buffer_.data(),buffer_.data() + 
buffer_.size(),key);
-
-  if (success) {
-    return Status::OK();
-  } else {
-    return Status::Corruption("Unable to decode first key");
+  if (PREDICT_FALSE(!SliceDecode(
+          buffer_.data(), buffer_.data() + buffer_.size(), key))) {
+    return Status::Corruption("unable to decode first key");
   }
+  return Status::OK();
 }
 
 size_t IndexBlockBuilder::EstimateEncodedSize() const {
   // the actual encoded index entries
-  int size = buffer_.size();
+  auto size = buffer_.size();
 
   // entry offsets
   size += sizeof(uint32_t) * entry_offsets_.size();
@@ -129,48 +122,76 @@ size_t IndexBlockBuilder::EstimateEncodedSize() const {
 }
 
 // Construct a reader.
-// After construtoin, call
+// After construction, call Parse() to read the data.
 IndexBlockReader::IndexBlockReader()
-  : parsed_(false) {
+    : key_offsets_(nullptr),
+      parsed_(false) {
 }
 
 void IndexBlockReader::Reset() {
-  data_ = Slice();
+  data_.clear();
+  trailer_.Clear();
+  key_offsets_ = nullptr;
   parsed_ = false;
 }
 
-Status IndexBlockReader::Parse(const Slice &data) {
-  parsed_ = false;
+Status IndexBlockReader::Parse(const Slice& data) {
   data_ = data;
 
-
-  if (data_.size() < sizeof(uint32_t)) {
+  // The code below parses the data of an index block, checking for invariants
+  // based on index block layout and protobuf serialization and encoding rules.
+  //
+  // For details on protobuf's serialization and encoding, see [2] and [3].
+  // For details on Kudu's index block layout, see [3].
+  //
+  // [1] https://protobuf.dev/programming-guides/encoding/
+  // [2] 
https://stackoverflow.com/questions/30915704/maximum-serialized-protobuf-message-size
+  // [3] 
https://github.com/apache/kudu/blob/master/docs/design-docs/cfile.md#cfile-index
+  if (PREDICT_FALSE(data_.size() < sizeof(uint32_t))) {
     return Status::Corruption("index block too small");
   }
 
-  const uint8_t *trailer_size_ptr =
-    data_.data() + data_.size() - sizeof(uint32_t);
-  uint32_t trailer_size = DecodeFixed32(trailer_size_ptr);
-
-  size_t max_size = trailer_size_ptr - data_.data();
-  if (trailer_size <= 0 ||
-      trailer_size > max_size) {
-    std::string err = strings::Substitute(
-        "invalid index block trailer size: $0", trailer_size);
-    return Status::Corruption(err);
+  const uint8_t* trailer_size_ptr =
+      data_.data() + data_.size() - sizeof(uint32_t);
+  const size_t trailer_size = DecodeFixed32(trailer_size_ptr);
+
+  // A serialized IndexBlockTrailerPB message cannot be shorter than four 
bytes.
+  // In protobuf, each serialized field contains a tag followed by some data.
+  // The tag is at least one byte. As for the serialized fields of the
+  // IndexBlockTrailerPB message, it contains at least two required fields now:
+  //   * int32_t num_entries: at least one byte serialized with varint encoding
+  //   * BlockType type: enum, at least one byte serialized
+  // So, the total for the minimum length of these two fields serialized
+  // in protobuf format is four bytes: (1 + 1) + (1 + 1).
+  if (PREDICT_FALSE(trailer_size < 4 ||
+                    trailer_size > trailer_size_ptr - data_.data())) {
+    return Status::Corruption(Substitute(
+        "$0: invalid index block trailer size", trailer_size));
   }
 
-  const uint8_t *trailer_ptr = trailer_size_ptr - trailer_size;
+  const uint8_t* trailer_ptr = trailer_size_ptr - trailer_size;
+  if (PREDICT_FALSE(!trailer_.ParseFromArray(trailer_ptr, trailer_size))) {
+    return Status::Corruption("unable to parse trailer",
+                              trailer_.InitializationErrorString());
+  }
 
-  bool success = trailer_.ParseFromArray(trailer_ptr, trailer_size);
-  if (!success) {
-    return Status::Corruption(
-      "unable to parse trailer",
-      trailer_.InitializationErrorString());
+  const auto num_entries = trailer_.num_entries();
+  if (PREDICT_FALSE(num_entries < 0)) {
+    return Status::Corruption(Substitute(
+        "$0: bad number of entries in trailer", num_entries));
   }
 
-  key_offsets_ = trailer_ptr - sizeof(uint32_t) * trailer_.num_entries();
-  CHECK(trailer_ptr >= data_.data());
+  key_offsets_ = trailer_ptr - sizeof(uint32_t) * num_entries;
+  // Each entry is three bytes at least (integers use varint encoding):
+  //   * key
+  //     ** the size of the key: at least one byte
+  //     ** the key's data: zero or more bytes (zero bytes for an empty key)
+  //   * the offset of the block: at least one byte
+  //   * the size of the block: at least one byte
+  if (PREDICT_FALSE(key_offsets_ < data_.data() + 3 * num_entries)) {
+    return Status::Corruption(Substitute(
+        "$0: too many entries in trailer", num_entries));
+  }
 
   VLOG(2) << "Parsed index trailer: " << pb_util::SecureDebugString(trailer_);
 
@@ -179,70 +200,94 @@ Status IndexBlockReader::Parse(const Slice &data) {
 }
 
 size_t IndexBlockReader::Count() const {
-  CHECK(parsed_) << "not parsed";
+  DCHECK(parsed_) << "not parsed";
   return trailer_.num_entries();
 }
 
-IndexBlockIterator *IndexBlockReader::NewIterator() const {
-  CHECK(parsed_) << "not parsed";
+IndexBlockIterator* IndexBlockReader::NewIterator() const {
+  DCHECK(parsed_) << "not parsed";
   return new IndexBlockIterator(this);
 }
 
-bool IndexBlockReader::IsLeaf() {
+bool IndexBlockReader::IsLeaf() const {
+  DCHECK(parsed_) << "not parsed";
   return trailer_.type() == IndexBlockTrailerPB::LEAF;
 }
 
-int IndexBlockReader::CompareKey(int idx_in_block,
-                                 const Slice &search_key) const {
-  const uint8_t *key_ptr, *limit;
-  GetKeyPointer(idx_in_block, &key_ptr, &limit);
+int IndexBlockReader::CompareKey(size_t idx_in_block,
+                                 const Slice& search_key) const {
+  const uint8_t* key_ptr = nullptr;
+  const uint8_t* limit = nullptr;
+  if (auto s = GetKeyPointer(idx_in_block, &key_ptr, &limit);
+      PREDICT_FALSE(!s.ok())) {
+    LOG(WARNING) << Substitute("failed to position in block: $0", 
s.ToString());
+    return 0;
+  }
   Slice this_slice;
-  if (PREDICT_FALSE(SliceDecode(key_ptr, limit, &this_slice) == nullptr)) {
-    LOG(WARNING)<< "Invalid data in block!";
+  if (PREDICT_FALSE(!SliceDecode(key_ptr, limit, &this_slice))) {
+    LOG(WARNING) << Substitute("invalid data in block at index $0", 
idx_in_block);
     return 0;
   }
 
   return this_slice.compare(search_key);
 }
 
-Status IndexBlockReader::ReadEntry(size_t idx, Slice *key, BlockPointer 
*block_ptr) const {
-  if (idx >= trailer_.num_entries()) {
-    return Status::NotFound("Invalid index");
+Status IndexBlockReader::ReadEntry(size_t idx,
+                                   Slice* key,
+                                   BlockPointer* block_ptr) const {
+  DCHECK(parsed_) << "not parsed";
+  if (PREDICT_FALSE(idx >= trailer_.num_entries())) {
+    return Status::NotFound(Substitute("$0: invalid index", idx));
   }
 
   // At 'ptr', data is encoded as follows:
   // <key> <block offset> <block length>
 
-  const uint8_t *ptr, *limit;
+  const uint8_t* ptr = nullptr;
+  const uint8_t* limit = nullptr;
   GetKeyPointer(idx, &ptr, &limit);
 
   ptr = SliceDecode(ptr, limit, key);
-  if (ptr == nullptr) {
-    return Status::Corruption("Invalid key in index");
+  if (PREDICT_FALSE(!ptr)) {
+    return Status::Corruption(Substitute("invalid key in index $0", idx));
   }
 
   return block_ptr->DecodeFrom(ptr, data_.data() + data_.size());
 }
 
-void IndexBlockReader::GetKeyPointer(int idx_in_block, const uint8_t **ptr,
-                                     const uint8_t **limit) const {
-  size_t offset_in_block = DecodeFixed32(
-    &key_offsets_[idx_in_block * sizeof(uint32_t)]);
+Status IndexBlockReader::GetKeyPointer(size_t idx_in_block,
+                                       const uint8_t** ptr,
+                                       const uint8_t** limit) const {
+  DCHECK(parsed_) << "not parsed";
+  if (PREDICT_FALSE(trailer_.num_entries() <= idx_in_block)) {
+    return Status::NotFound(Substitute("$0: no such index", idx_in_block));
+  }
+  DCHECK(key_offsets_);
+  const size_t offset_in_block = DecodeFixed32(
+      &key_offsets_[idx_in_block * sizeof(uint32_t)]);
+  if (PREDICT_FALSE(data_.data() + offset_in_block >= key_offsets_)) {
+    return Status::Corruption(Substitute(
+        "$0: invalid block offset at index $1", offset_in_block, 
idx_in_block));
+  }
   *ptr = data_.data() + offset_in_block;
 
-  int next_idx = idx_in_block + 1;
-
-  if (PREDICT_FALSE(next_idx >= trailer_.num_entries())) {
-    DCHECK(next_idx == Count()) << "Bad index: " << idx_in_block
-                                << " Count: " << Count();
-    // last key in block: limit is the beginning of the offsets array
-    *limit = key_offsets_;
-  } else {
-    // otherwise limit is the beginning of the next key
-    offset_in_block = DecodeFixed32(
-      &key_offsets_[next_idx * sizeof(uint32_t)]);
+  const size_t next_idx = idx_in_block + 1;
+  if (PREDICT_TRUE(next_idx < trailer_.num_entries())) {
+    // The limit is the beginning of the next key.
+    const size_t offset_in_block = DecodeFixed32(
+        &key_offsets_[next_idx * sizeof(uint32_t)]);
+    if (PREDICT_FALSE(data_.data() + offset_in_block >= key_offsets_)) {
+      return Status::Corruption(Substitute(
+          "$0: invalid block offset at index $1", offset_in_block, next_idx));
+    }
     *limit = data_.data() + offset_in_block;
+  } else {
+    // For the last key in block, the limit is the beginning of the offsets 
array.
+    DCHECK(next_idx == Count()) << Substitute("bad index: $0 count: $1",
+                                              idx_in_block, Count());
+    *limit = key_offsets_;
   }
+  return Status::OK();
 }
 
 void IndexBlockBuilder::Reset() {
@@ -251,22 +296,25 @@ void IndexBlockBuilder::Reset() {
   finished_ = false;
 }
 
-IndexBlockIterator::IndexBlockIterator(const IndexBlockReader *reader)
-  : reader_(reader),
-    cur_idx_(-1),
-    seeked_(false) {
+IndexBlockIterator::IndexBlockIterator(const IndexBlockReader* reader)
+    : reader_(reader),
+      cur_idx_(-1),
+      seeked_(false) {
 }
 
 void IndexBlockIterator::Reset() {
-  seeked_ = false;
   cur_idx_ = -1;
+  cur_key_.clear();
+  cur_ptr_ = BlockPointer();
+  seeked_ = false;
 }
 
-Status IndexBlockIterator::SeekAtOrBefore(const Slice &search_key) {
+Status IndexBlockIterator::SeekAtOrBefore(const Slice& search_key) {
+  const auto num_entries = reader_->Count();
   size_t left = 0;
-  size_t right = reader_->Count() - 1;
+  size_t right = num_entries > 0 ? num_entries - 1 : 0;
   while (left < right) {
-    int mid = (left + right + 1) / 2;
+    size_t mid = (left + right + 1) / 2;
 
     int compare = reader_->CompareKey(mid, search_key);
     if (compare < 0) {  // mid < search
@@ -280,8 +328,7 @@ Status IndexBlockIterator::SeekAtOrBefore(const Slice 
&search_key) {
   }
 
   // closest is now 'left'
-  int compare = reader_->CompareKey(left, search_key);
-  if (compare > 0) {
+  if (reader_->CompareKey(left, search_key) > 0) {
     // The last midpoint was still greater than the
     // provided key, which implies that the key is
     // lower than the lowest in the block.
@@ -298,21 +345,27 @@ Status IndexBlockIterator::SeekToIndex(size_t idx) {
   return s;
 }
 
+// Unsigned cur_idx_ overflow is intended and works as expected when
+// cur_idx_ has its initial value, so quell UBSAN warnings.
+ATTRIBUTE_NO_SANITIZE_INTEGER
 bool IndexBlockIterator::HasNext() const {
   return cur_idx_ + 1 < reader_->Count();
 }
 
+// Unsigned cur_idx_ overflow is intended and works as expected when
+// cur_idx_ has its initial value, so quell UBSAN warnings.
+ATTRIBUTE_NO_SANITIZE_INTEGER
 Status IndexBlockIterator::Next() {
   return SeekToIndex(cur_idx_ + 1);
 }
 
-const BlockPointer &IndexBlockIterator::GetCurrentBlockPointer() const {
-  CHECK(seeked_) << "not seeked";
+const BlockPointer& IndexBlockIterator::GetCurrentBlockPointer() const {
+  DCHECK(seeked_) << "not seeked";
   return cur_ptr_;
 }
 
-const Slice IndexBlockIterator::GetCurrentKey() const {
-  CHECK(seeked_) << "not seeked";
+const Slice& IndexBlockIterator::GetCurrentKey() const {
+  DCHECK(seeked_) << "not seeked";
   return cur_key_;
 }
 
diff --git a/src/kudu/cfile/index_block.h b/src/kudu/cfile/index_block.h
index 045c0101e..fafd18f4b 100644
--- a/src/kudu/cfile/index_block.h
+++ b/src/kudu/cfile/index_block.h
@@ -15,8 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef KUDU_CFILE_INDEX_BLOCK_H
-#define KUDU_CFILE_INDEX_BLOCK_H
+#pragma once
 
 #include <cstddef>
 #include <cstdint>
@@ -35,19 +34,16 @@ namespace cfile {
 // Forward decl.
 class IndexBlockIterator;
 
-struct WriterOptions;
-
 // Index Block Builder for a particular key type.
 // This works like the rest of the builders in the cfile package.
 // After repeatedly calling Add(), call Finish() to encode it
 // into a Slice, then you may Reset to re-use buffers.
 class IndexBlockBuilder {
  public:
-  explicit IndexBlockBuilder(const WriterOptions *options,
-                             bool is_leaf);
+  explicit IndexBlockBuilder(bool is_leaf);
 
   // Append an entry into the index.
-  void Add(const Slice &key, const BlockPointer &ptr);
+  void Add(const Slice& key, const BlockPointer& ptr);
 
   // Finish the current index block.
   // Returns a fully encoded Slice including the data
@@ -58,14 +54,19 @@ class IndexBlockBuilder {
 
   // Return the key of the first entry in this index block.
   // The pointed-to data is only valid until the next call to this builder.
-  Status GetFirstKey(Slice *key) const;
+  Status GetFirstKey(Slice* key) const;
 
-  // Return the number of entries already added to this index
-  // block.
+  // Return the number of entries already added to this index block.
   size_t count() const {
     return entry_offsets_.size();
   }
 
+  // Return 'true' if there aren't any entries added to this index block,
+  // 'false' otherwise.
+  bool empty() const {
+    return entry_offsets_.empty();
+  }
+
   // Return an estimate of the post-encoding size of this
   // index block. This estimate should be conservative --
   // it will over-estimate rather than under-estimate, and
@@ -78,16 +79,11 @@ class IndexBlockBuilder {
  private:
   DISALLOW_COPY_AND_ASSIGN(IndexBlockBuilder);
 
-#ifdef __clang__
-  __attribute__((__unused__))
-#endif
-  const WriterOptions *options_;
-
   // Is the builder currently between Finish() and Reset()
   bool finished_;
 
   // Is this a leaf block?
-  bool is_leaf_;
+  const bool is_leaf_;
 
   faststring buffer_;
   std::vector<uint32_t> entry_offsets_;
@@ -106,34 +102,35 @@ class IndexBlockReader {
   //
   // Note: this does not copy the data, so the slice must
   // remain valid for the lifetime of the reader (or until the next Parse 
call).
-  Status Parse(const Slice &data);
+  Status Parse(const Slice& data);
 
   size_t Count() const;
 
-  IndexBlockIterator *NewIterator() const;
+  IndexBlockIterator* NewIterator() const;
 
-  bool IsLeaf();
+  bool IsLeaf() const;
 
  private:
   friend class IndexBlockIterator;
 
-  int CompareKey(int idx_in_block, const Slice &search_key) const;
+  int CompareKey(size_t idx_in_block, const Slice& search_key) const;
 
-  Status ReadEntry(size_t idx, Slice *key, BlockPointer *block_ptr) const;
+  Status ReadEntry(size_t idx, Slice* key, BlockPointer* block_ptr) const;
 
-  // Set *ptr to the beginning of the index data for the given index
-  // entry.
+  // Set *ptr to the beginning of the index data for the given index entry.
   // Set *limit to the 'limit' pointer for that entry (i.e a pointer
   // beyond which the data no longer is part of that entry).
   //   - *limit can be used to prevent overrunning in the case of a
   //     corrupted length varint or length prefix
-  void GetKeyPointer(int idx_in_block, const uint8_t **ptr, const uint8_t 
**limit) const;
+  // Return Status::NotFound() if the given index isn't present.
+  // Return Status::Corruption() if inconsistency in block offset is detected.
+  Status GetKeyPointer(size_t idx_in_block,
+                       const uint8_t** ptr,
+                       const uint8_t** limit) const;
 
-  static const int kMaxTrailerSize = 64*1024;
   Slice data_;
-
   IndexBlockTrailerPB trailer_;
-  const uint8_t *key_offsets_;
+  const uint8_t* key_offsets_;
   bool parsed_;
 
   DISALLOW_COPY_AND_ASSIGN(IndexBlockReader);
@@ -141,7 +138,7 @@ class IndexBlockReader {
 
 class IndexBlockIterator {
  public:
-  explicit IndexBlockIterator(const IndexBlockReader *reader);
+  explicit IndexBlockIterator(const IndexBlockReader* reader);
 
   // Reset the state of this iterator. This should be used
   // after the associated 'reader' object parses a different block.
@@ -157,7 +154,7 @@ class IndexBlockIterator {
   // If this function returns an error, then the state of this
   // iterator is undefined (i.e it may or may not have moved
   // since the previous call)
-  Status SeekAtOrBefore(const Slice &search_key);
+  Status SeekAtOrBefore(const Slice& search_key);
 
   Status SeekToIndex(size_t idx);
 
@@ -165,12 +162,12 @@ class IndexBlockIterator {
 
   Status Next();
 
-  const BlockPointer &GetCurrentBlockPointer() const;
+  const BlockPointer& GetCurrentBlockPointer() const;
 
-  const Slice GetCurrentKey() const;
+  const Slice& GetCurrentKey() const;
 
  private:
-  const IndexBlockReader *reader_;
+  const IndexBlockReader* reader_;
   size_t cur_idx_;
   Slice cur_key_;
   BlockPointer cur_ptr_;
@@ -181,4 +178,3 @@ class IndexBlockIterator {
 
 } // namespace cfile
 } // namespace kudu
-#endif
diff --git a/src/kudu/cfile/index_btree.cc b/src/kudu/cfile/index_btree.cc
index 887cbcaf7..942ff389c 100644
--- a/src/kudu/cfile/index_btree.cc
+++ b/src/kudu/cfile/index_btree.cc
@@ -33,6 +33,7 @@
 #include "kudu/cfile/cfile_writer.h"
 #include "kudu/cfile/index_block.h"
 #include "kudu/fs/block_id.h"
+#include "kudu/gutil/port.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/debug-util.h"
@@ -47,31 +48,30 @@ namespace kudu {
 namespace cfile {
 
 IndexTreeBuilder::IndexTreeBuilder(
-  const WriterOptions *options,
-  CFileWriter *writer) :
-  options_(options),
-  writer_(writer) {
+    const WriterOptions* options,
+    CFileWriter* writer)
+    : options_(options),
+      writer_(writer) {
   idx_blocks_.emplace_back(CreateBlockBuilder(true));
 }
 
-
-IndexBlockBuilder *IndexTreeBuilder::CreateBlockBuilder(bool is_leaf) {
-  return new IndexBlockBuilder(options_, is_leaf);
+IndexBlockBuilder* IndexTreeBuilder::CreateBlockBuilder(bool is_leaf) {
+  return new IndexBlockBuilder(is_leaf);
 }
 
-Status IndexTreeBuilder::Append(const Slice &key,
-                                const BlockPointer &block) {
-  return Append(key, block, 0);
+Status IndexTreeBuilder::Append(const Slice& key,
+                                const BlockPointer& block_ptr) {
+  return Append(key, block_ptr, 0);
 }
 
-Status IndexTreeBuilder::Append(
-  const Slice &key, const BlockPointer &block_ptr,
-  size_t level) {
+Status IndexTreeBuilder::Append(const Slice& key,
+                                const BlockPointer& block_ptr,
+                                size_t level) {
   if (level >= idx_blocks_.size()) {
     // Need to create a new level
-    CHECK(level == idx_blocks_.size()) <<
-      "trying to create level " << level << " but size is only "
-                                << idx_blocks_.size();
+    DCHECK(level == idx_blocks_.size()) <<
+          Substitute("trying to create level $0 but size is only $1",
+                     level, idx_blocks_.size());
     VLOG(1) << "Creating level-" << level << " in index b-tree";
     idx_blocks_.emplace_back(CreateBlockBuilder(false));
   }
@@ -79,35 +79,31 @@ Status IndexTreeBuilder::Append(
   IndexBlockBuilder* idx_block = idx_blocks_[level].get();
   idx_block->Add(key, block_ptr);
 
-  // This index block is full, and there are at least two entries,
-  // flush it.
-  size_t est_size = idx_block->EstimateEncodedSize();
-  if (est_size > options_->index_block_size &&
-      idx_block->count() > 1) {
+  // If this index block is full and there are at least two entries,
+  // then flush it.
+  if (idx_block->count() > 1 &&
+      idx_block->EstimateEncodedSize() > options_->index_block_size) {
     RETURN_NOT_OK(FinishBlockAndPropagate(level));
   }
 
   return Status::OK();
 }
 
-
-Status IndexTreeBuilder::Finish(BTreeInfoPB *info) {
-  // Now do the same for the positional index blocks, starting
-  // with leaf
-  VLOG(1) << "flushing tree, b-tree has " <<
-    idx_blocks_.size() << " levels";
+Status IndexTreeBuilder::Finish(BTreeInfoPB* info) {
+  DCHECK(!idx_blocks_.empty());
+  // Now do the same for the positional index blocks, starting with leaf.
+  VLOG(1) << "flushing tree, b-tree has " << idx_blocks_.size() << " levels";
 
   // Flush all but the root of the index.
-  for (size_t i = 0; i < idx_blocks_.size() - 1; i++) {
-    RETURN_NOT_OK(FinishBlockAndPropagate(i));
+  for (size_t i = 1; i < idx_blocks_.size(); ++i) {
+    RETURN_NOT_OK(FinishBlockAndPropagate(i - 1));
   }
 
   // Flush the root
-  int root_level = idx_blocks_.size() - 1;
+  const size_t root_level = idx_blocks_.size() - 1;
   BlockPointer ptr;
-  Status s = FinishAndWriteBlock(root_level, &ptr);
-  if (!s.ok()) {
-    return s.CloneAndPrepend("Unable to flush root index block");
+  if (auto s = FinishAndWriteBlock(root_level, &ptr); PREDICT_FALSE(!s.ok())) {
+    return s.CloneAndPrepend("unable to flush root index block");
   }
 
   VLOG(1) << "Flushed root index block: " << ptr.ToString();
@@ -117,6 +113,7 @@ Status IndexTreeBuilder::Finish(BTreeInfoPB *info) {
 }
 
 Status IndexTreeBuilder::FinishBlockAndPropagate(size_t level) {
+  DCHECK_LT(level, idx_blocks_.size());
   IndexBlockBuilder* idx_block = idx_blocks_[level].get();
 
   // If the block doesn't have any data in it, we don't need to
@@ -124,8 +121,8 @@ Status IndexTreeBuilder::FinishBlockAndPropagate(size_t 
level) {
   // This happens if a lower-level block fills up exactly,
   // and then the file completes.
   //
-  // TODO: add a test case which exercises this explicitly.
-  if (idx_block->count() == 0) {
+  // TODO(todd): add a test case which exercises this explicitly.
+  if (idx_block->empty()) {
     return Status::OK();
   }
 
@@ -135,18 +132,15 @@ Status IndexTreeBuilder::FinishBlockAndPropagate(size_t 
level) {
 
   // Get the first key of the finished block.
   Slice first_in_idx_block;
-  Status s = idx_block->GetFirstKey(&first_in_idx_block);
-
-  if (!s.ok()) {
-    LOG(ERROR) << "Unable to get first key of level-" << level
-               << " index block: " << s.ToString() << std::endl
-               << GetStackTrace();
+  if (auto s = idx_block->GetFirstKey(&first_in_idx_block); 
PREDICT_FALSE(!s.ok())) {
+    LOG(ERROR) << Substitute(
+        "unable to get first key of level-$0 index block: $1\n$2",
+        level, s.ToString(), GetStackTrace());
     return s;
   }
 
   // Add to higher-level index.
-  RETURN_NOT_OK(Append(first_in_idx_block, idx_block_ptr,
-                       level + 1));
+  RETURN_NOT_OK(Append(first_in_idx_block, idx_block_ptr, level + 1));
 
   // Finally, reset the block we just wrote. It's important to wait until
   // here to do this, since the first_in_idx_block data may point to internal
@@ -159,13 +153,12 @@ Status IndexTreeBuilder::FinishBlockAndPropagate(size_t 
level) {
 // Finish the current block at the given level, writing it
 // to the file. Return the location of the written block
 // in 'written'.
-Status IndexTreeBuilder::FinishAndWriteBlock(size_t level, BlockPointer 
*written) {
-  IndexBlockBuilder* idx_block = idx_blocks_[level].get();
-  vector<Slice> v { idx_block->Finish() };
-  Status s = writer_->AddBlock(std::move(v), written, "index block");
-  if (!s.ok()) {
-    LOG(ERROR) << "Unable to append level-" << level << " index "
-               << "block to file";
+Status IndexTreeBuilder::FinishAndWriteBlock(size_t level, BlockPointer* 
written) {
+  DCHECK_LT(level, idx_blocks_.size());
+  vector<Slice> v { idx_blocks_[level].get()->Finish() };
+  if (auto s = writer_->AddBlock(std::move(v), written, "index block");
+      PREDICT_FALSE(!s.ok())) {
+    LOG(ERROR) << Substitute("unable to append level-$0 index block to file", 
level);
     return s;
   }
 
@@ -189,15 +182,16 @@ struct IndexTreeIterator::SeekedIndex {
   IndexBlockIterator iter;
 };
 
-IndexTreeIterator::IndexTreeIterator(const IOContext* io_context, const 
CFileReader *reader,
-                                     const BlockPointer &root_blockptr)
+IndexTreeIterator::IndexTreeIterator(const IOContext* io_context,
+                                     const CFileReader* reader,
+                                     const BlockPointer& root_blockptr)
     : reader_(reader),
       root_block_(root_blockptr),
       io_context_(io_context) {
 }
 IndexTreeIterator::~IndexTreeIterator() = default;
 
-Status IndexTreeIterator::SeekAtOrBefore(const Slice &search_key) {
+Status IndexTreeIterator::SeekAtOrBefore(const Slice& search_key) {
   return SeekDownward(search_key, root_block_, 0);
 }
 
@@ -205,7 +199,7 @@ Status IndexTreeIterator::SeekToFirst() {
   return SeekToFirstDownward(root_block_, 0);
 }
 
-bool IndexTreeIterator::HasNext() {
+bool IndexTreeIterator::HasNext() const {
   for (int i = seeked_indexes_.size(); i > 0; i--) {
     if (seeked_indexes_[i - 1]->iter.HasNext())
       return true;
@@ -214,7 +208,7 @@ bool IndexTreeIterator::HasNext() {
 }
 
 Status IndexTreeIterator::Next() {
-  CHECK(!seeked_indexes_.empty()) << "not seeked";
+  DCHECK(!seeked_indexes_.empty()) << "not seeked";
 
   // Start at the bottom level of the BTree, calling Next(),
   // until one succeeds. If any does not succeed, then
@@ -249,34 +243,34 @@ Status IndexTreeIterator::Next() {
   return Status::OK();
 }
 
-const Slice IndexTreeIterator::GetCurrentKey() const {
+const Slice& IndexTreeIterator::GetCurrentKey() const {
   return seeked_indexes_.back()->iter.GetCurrentKey();
 }
 
-const BlockPointer &IndexTreeIterator::GetCurrentBlockPointer() const {
+const BlockPointer& IndexTreeIterator::GetCurrentBlockPointer() const {
   return seeked_indexes_.back()->iter.GetCurrentBlockPointer();
 }
 
-IndexBlockIterator *IndexTreeIterator::BottomIter() {
+IndexBlockIterator* IndexTreeIterator::BottomIter() {
   return &seeked_indexes_.back()->iter;
 }
 
-IndexBlockReader *IndexTreeIterator::BottomReader() {
+const IndexBlockReader* IndexTreeIterator::BottomReader() const {
   return &seeked_indexes_.back()->reader;
 }
 
-IndexBlockIterator *IndexTreeIterator::seeked_iter(int depth) {
+IndexBlockIterator* IndexTreeIterator::seeked_iter(size_t depth) {
+  DCHECK_LT(depth, seeked_indexes_.size());
   return &seeked_indexes_[depth]->iter;
 }
 
-IndexBlockReader *IndexTreeIterator::seeked_reader(int depth) {
+const IndexBlockReader* IndexTreeIterator::seeked_reader(size_t depth) const {
+  DCHECK_LT(depth, seeked_indexes_.size());
   return &seeked_indexes_[depth]->reader;
 }
 
-Status IndexTreeIterator::LoadBlock(const BlockPointer &block,
-                                    int depth) {
-
-  SeekedIndex *seeked;
+Status IndexTreeIterator::LoadBlock(const BlockPointer& block, size_t depth) {
+  SeekedIndex* seeked;
   if (depth < seeked_indexes_.size()) {
     // We have a cached instance from previous seek.
     seeked = seeked_indexes_[depth].get();
@@ -294,7 +288,7 @@ Status IndexTreeIterator::LoadBlock(const BlockPointer 
&block,
     seeked->iter.Reset();
   } else {
     // No cached instance, make a new one.
-    seeked_indexes_.emplace_back(new SeekedIndex());
+    seeked_indexes_.emplace_back(new SeekedIndex);
     seeked = seeked_indexes_.back().get();
   }
 
@@ -307,16 +301,15 @@ Status IndexTreeIterator::LoadBlock(const BlockPointer 
&block,
                         Substitute("failed to parse index block in block $0 at 
$1",
                                    reader_->block_id().ToString(),
                                    block.ToString()));
-
   return Status::OK();
 }
 
-Status IndexTreeIterator::SeekDownward(const Slice &search_key, const 
BlockPointer &in_block,
-                                       int cur_depth) {
-
+Status IndexTreeIterator::SeekDownward(const Slice& search_key,
+                                       const BlockPointer& in_block,
+                                       size_t cur_depth) {
   // Read the block.
   RETURN_NOT_OK(LoadBlock(in_block, cur_depth));
-  IndexBlockIterator *iter = seeked_iter(cur_depth);
+  IndexBlockIterator* iter = seeked_iter(cur_depth);
 
   RETURN_NOT_OK(iter->SeekAtOrBefore(search_key));
 
@@ -330,10 +323,11 @@ Status IndexTreeIterator::SeekDownward(const Slice 
&search_key, const BlockPoint
   return SeekDownward(search_key, iter->GetCurrentBlockPointer(), cur_depth + 
1);
 }
 
-Status IndexTreeIterator::SeekToFirstDownward(const BlockPointer &in_block, 
int cur_depth) {
+Status IndexTreeIterator::SeekToFirstDownward(const BlockPointer& in_block,
+                                              size_t cur_depth) {
   // Read the block.
   RETURN_NOT_OK(LoadBlock(in_block, cur_depth));
-  IndexBlockIterator *iter = seeked_iter(cur_depth);
+  IndexBlockIterator* iter = seeked_iter(cur_depth);
 
   RETURN_NOT_OK(iter->SeekToIndex(0));
 
@@ -343,18 +337,16 @@ Status IndexTreeIterator::SeekToFirstDownward(const 
BlockPointer &in_block, int
   if (seeked_reader(cur_depth)->IsLeaf()) {
     seeked_indexes_.resize(cur_depth + 1);
     return Status::OK();
-  } else {
-    return SeekToFirstDownward(iter->GetCurrentBlockPointer(), cur_depth + 1);
   }
+  return SeekToFirstDownward(iter->GetCurrentBlockPointer(), cur_depth + 1);
 }
 
-IndexTreeIterator *IndexTreeIterator::IndexTreeIterator::Create(
+IndexTreeIterator* IndexTreeIterator::IndexTreeIterator::Create(
     const IOContext* io_context,
-    const CFileReader *reader,
-    const BlockPointer &root_blockptr) {
+    const CFileReader* reader,
+    const BlockPointer& root_blockptr) {
   return new IndexTreeIterator(io_context, reader, root_blockptr);
 }
 
-
 } // namespace cfile
 } // namespace kudu
diff --git a/src/kudu/cfile/index_btree.h b/src/kudu/cfile/index_btree.h
index 108c7067c..53340639e 100644
--- a/src/kudu/cfile/index_btree.h
+++ b/src/kudu/cfile/index_btree.h
@@ -15,8 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef KUDU_CFILE_INDEX_BTREE_H
-#define KUDU_CFILE_INDEX_BTREE_H
+#pragma once
 
 #include <cstddef>
 #include <memory>
@@ -44,18 +43,17 @@ struct WriterOptions;
 class IndexTreeBuilder {
  public:
   explicit IndexTreeBuilder(
-    const WriterOptions *options,
-    CFileWriter *writer);
+    const WriterOptions* options,
+    CFileWriter* writer);
 
   // Append the given key into the index.
-  // The key is copied into the builder's internal
-  // memory.
-  Status Append(const Slice &key, const BlockPointer &block);
-  Status Finish(BTreeInfoPB *info);
+  // The key is copied into the builder's internal memory.
+  Status Append(const Slice& key, const BlockPointer& block_ptr);
+  Status Finish(BTreeInfoPB* info);
+
  private:
-  IndexBlockBuilder *CreateBlockBuilder(bool is_leaf);
-  Status Append(const Slice &key, const BlockPointer &block_ptr,
-                size_t level);
+  static IndexBlockBuilder* CreateBlockBuilder(bool is_leaf);
+  Status Append(const Slice& key, const BlockPointer& block_ptr, size_t level);
 
   // Finish the current block at the given index level, and then
   // propagate by inserting this block into the next higher-up
@@ -65,10 +63,10 @@ class IndexTreeBuilder {
   // Finish the current block at the given level, writing it
   // to the file. Return the location of the written block
   // in 'written'.
-  Status FinishAndWriteBlock(size_t level, BlockPointer *written);
+  Status FinishAndWriteBlock(size_t level, BlockPointer* written);
 
-  const WriterOptions *options_;
-  CFileWriter *writer_;
+  const WriterOptions* options_;
+  CFileWriter* writer_;
 
   std::vector<std::unique_ptr<IndexBlockBuilder>> idx_blocks_;
 
@@ -77,21 +75,21 @@ class IndexTreeBuilder {
 
 class IndexTreeIterator {
  public:
-  explicit IndexTreeIterator(
+  IndexTreeIterator(
       const fs::IOContext* io_context,
-      const CFileReader *reader,
-      const BlockPointer &root_blockptr);
+      const CFileReader* reader,
+      const BlockPointer& root_blockptr);
   ~IndexTreeIterator();
 
   Status SeekToFirst();
-  Status SeekAtOrBefore(const Slice &search_key);
-  bool HasNext();
+  Status SeekAtOrBefore(const Slice& search_key);
+  bool HasNext() const;
   Status Next();
 
   // The slice key at which the iterator
   // is currently seeked to.
-  const Slice GetCurrentKey() const;
-  const BlockPointer &GetCurrentBlockPointer() const;
+  const Slice& GetCurrentKey() const;
+  const BlockPointer& GetCurrentBlockPointer() const;
 
   static IndexTreeIterator* Create(
     const fs::IOContext* io_context,
@@ -103,20 +101,21 @@ class IndexTreeIterator {
   }
 
  private:
-  IndexBlockIterator *BottomIter();
-  IndexBlockReader *BottomReader();
-  IndexBlockIterator *seeked_iter(int depth);
-  IndexBlockReader *seeked_reader(int depth);
-  Status LoadBlock(const BlockPointer &block, int depth);
-  Status SeekDownward(const Slice &search_key, const BlockPointer &in_block,
-                      int cur_depth);
-  Status SeekToFirstDownward(const BlockPointer &in_block, int cur_depth);
+  IndexBlockIterator* BottomIter();
+  const IndexBlockReader* BottomReader() const;
+  IndexBlockIterator* seeked_iter(size_t depth);
+  const IndexBlockReader* seeked_reader(size_t depth) const;
+  Status LoadBlock(const BlockPointer& block, size_t depth);
+  Status SeekDownward(const Slice& search_key,
+                      const BlockPointer& in_block,
+                      size_t cur_depth);
+  Status SeekToFirstDownward(const BlockPointer& in_block, size_t cur_depth);
 
   struct SeekedIndex;
 
-  const CFileReader *reader_;
+  const CFileReader* reader_;
 
-  BlockPointer root_block_;
+  const BlockPointer root_block_;
 
   std::vector<std::unique_ptr<SeekedIndex>> seeked_indexes_;
 
@@ -127,4 +126,3 @@ class IndexTreeIterator {
 
 } // namespace cfile
 } // namespace kudu
-#endif

Reply via email to