You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by aw...@apache.org on 2020/04/09 04:03:03 UTC

[kudu] 02/03: cfile: change BlockBuilder API to yield a vector of Slices

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

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

commit 40e4d4f52168690cfbeaeaccc8537558f29f28d2
Author: Bankim Bhavsar <ba...@cloudera.com>
AuthorDate: Thu Apr 2 22:01:08 2020 -0700

    cfile: change BlockBuilder API to yield a vector of Slices
    
    When blocks are appended to cfiles at the IO layer, we already have the
    ability to write multiple slices using a vectored IO. Prior to this
    patch, the BlockBuilder API was restricted to returning a single slice,
    whereas it would be more convenient in some cases to be able to return
    multiple slices (eg separating the header from the data).
    
    This new functionality is used by BinaryDictBlockBuilder to avoid an
    extra copy in Finish().
    
    Change-Id: Ifc7a5f148a4a43cedac2428f4c1a18d0f93a10db
    Reviewed-on: http://gerrit.cloudera.org:8080/15042
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
    Tested-by: Kudu Jenkins
---
 src/kudu/cfile/binary_dict_block.cc   | 29 +++++++----------
 src/kudu/cfile/binary_dict_block.h    |  6 ++--
 src/kudu/cfile/binary_plain_block.cc  | 17 ++++++----
 src/kudu/cfile/binary_plain_block.h   |  9 ++----
 src/kudu/cfile/binary_prefix_block.cc | 35 +++++++-------------
 src/kudu/cfile/binary_prefix_block.h  | 15 ++-------
 src/kudu/cfile/block_encodings.h      | 12 ++++---
 src/kudu/cfile/bshuf_block.cc         |  6 ++--
 src/kudu/cfile/bshuf_block.h          |  7 ++--
 src/kudu/cfile/cfile_writer.cc        |  7 ++--
 src/kudu/cfile/encoding-test.cc       | 60 +++++++++++++++++++++++++----------
 src/kudu/cfile/plain_bitmap_block.h   |  5 +--
 src/kudu/cfile/plain_block.h          |  5 +--
 src/kudu/cfile/rle_block.h            |  9 +++---
 14 files changed, 115 insertions(+), 107 deletions(-)

diff --git a/src/kudu/cfile/binary_dict_block.cc b/src/kudu/cfile/binary_dict_block.cc
index a97c32f..c307439 100644
--- a/src/kudu/cfile/binary_dict_block.cc
+++ b/src/kudu/cfile/binary_dict_block.cc
@@ -19,6 +19,7 @@
 
 #include <limits>
 #include <ostream>
+#include <utility>
 #include <vector>
 
 #include <glog/logging.h>
@@ -27,14 +28,12 @@
 #include "kudu/cfile/bshuf_block.h"
 #include "kudu/cfile/cfile.pb.h"
 #include "kudu/cfile/cfile_reader.h"
-#include "kudu/cfile/cfile_util.h"
 #include "kudu/cfile/cfile_writer.h"
 #include "kudu/common/column_materialization_context.h"
 #include "kudu/common/column_predicate.h"
 #include "kudu/common/columnblock.h"
 #include "kudu/common/common.pb.h"
 #include "kudu/common/rowblock.h"
-#include "kudu/common/schema.h"
 #include "kudu/common/types.h"
 #include "kudu/gutil/casts.h"
 #include "kudu/gutil/map-util.h"
@@ -45,6 +44,8 @@
 #include "kudu/util/coding-inl.h"
 #include "kudu/util/memory/arena.h"
 
+using std::vector;
+
 namespace kudu {
 namespace cfile {
 
@@ -62,10 +63,6 @@ BinaryDictBlockBuilder::BinaryDictBlockBuilder(const WriterOptions* options)
 }
 
 void BinaryDictBlockBuilder::Reset() {
-  buffer_.clear();
-  buffer_.resize(kMaxHeaderSize);
-  buffer_.reserve(options_->storage_attributes.cfile_block_size);
-
   if (mode_ == kCodeWordMode &&
       dict_block_.IsBlockFull()) {
     mode_ = kPlainBinaryMode;
@@ -77,17 +74,16 @@ void BinaryDictBlockBuilder::Reset() {
   finished_ = false;
 }
 
-Slice BinaryDictBlockBuilder::Finish(rowid_t ordinal_pos) {
+void BinaryDictBlockBuilder::Finish(rowid_t ordinal_pos, vector<Slice>* slices) {
   finished_ = true;
 
-  InlineEncodeFixed32(&buffer_[0], mode_);
-
-  // TODO: if we could modify the the Finish() API a little bit, we can
-  // avoid an extra memory copy (buffer_.append(..))
-  Slice data_slice = data_builder_->Finish(ordinal_pos);
-  buffer_.append(data_slice.data(), data_slice.size());
+  header_buffer_.resize(sizeof(int32_t));
+  InlineEncodeFixed32(&header_buffer_[0], mode_);
 
-  return Slice(buffer_);
+  vector<Slice> data_slices;
+  data_builder_->Finish(ordinal_pos, &data_slices);
+  data_slices.insert(data_slices.begin(), Slice(header_buffer_));
+  *slices = std::move(data_slices);
 }
 
 // The current block is considered full when the the size of data block
@@ -153,10 +149,9 @@ int BinaryDictBlockBuilder::Add(const uint8_t* vals, size_t count) {
 }
 
 Status BinaryDictBlockBuilder::AppendExtraInfo(CFileWriter* c_writer, CFileFooterPB* footer) {
-  Slice dict_slice = dict_block_.Finish(0);
+  vector<Slice> dict_v;
+  dict_block_.Finish(0, &dict_v);
 
-  std::vector<Slice> dict_v;
-  dict_v.push_back(dict_slice);
 
   BlockPointer ptr;
   Status s = c_writer->AppendDictBlock(dict_v, &ptr, "Append dictionary block");
diff --git a/src/kudu/cfile/binary_dict_block.h b/src/kudu/cfile/binary_dict_block.h
index ceb044e..a8d35e8 100644
--- a/src/kudu/cfile/binary_dict_block.h
+++ b/src/kudu/cfile/binary_dict_block.h
@@ -38,6 +38,7 @@
 #include <cstddef>
 #include <cstdint>
 #include <memory>
+#include <vector>
 
 #include <sparsehash/dense_hash_map>
 
@@ -89,7 +90,7 @@ class BinaryDictBlockBuilder final : public BlockBuilder {
 
   int Add(const uint8_t* vals, size_t count) OVERRIDE;
 
-  Slice Finish(rowid_t ordinal_pos) OVERRIDE;
+  void Finish(rowid_t ordinal_pos, std::vector<Slice>* slices) OVERRIDE;
 
   void Reset() OVERRIDE;
 
@@ -107,7 +108,8 @@ class BinaryDictBlockBuilder final : public BlockBuilder {
   ATTRIBUTE_COLD
   bool AddToDict(Slice val, uint32_t* codeword);
 
-  faststring buffer_;
+  // Buffer used in Finish() for holding the encoded header.
+  faststring header_buffer_;
   bool finished_;
   const WriterOptions* options_;
 
diff --git a/src/kudu/cfile/binary_plain_block.cc b/src/kudu/cfile/binary_plain_block.cc
index 4c1721c..18ec85d 100644
--- a/src/kudu/cfile/binary_plain_block.cc
+++ b/src/kudu/cfile/binary_plain_block.cc
@@ -20,6 +20,7 @@
 #include <algorithm>
 #include <cstdint>
 #include <ostream>
+#include <vector>
 
 #include <glog/logging.h>
 
@@ -39,6 +40,8 @@
 #include "kudu/util/hexdump.h"
 #include "kudu/util/memory/arena.h"
 
+using std::vector;
+
 namespace kudu {
 namespace cfile {
 
@@ -52,11 +55,11 @@ BinaryPlainBlockBuilder::BinaryPlainBlockBuilder(const WriterOptions *options)
 void BinaryPlainBlockBuilder::Reset() {
   offsets_.clear();
   buffer_.clear();
-  buffer_.resize(kMaxHeaderSize);
   buffer_.reserve(options_->storage_attributes.cfile_block_size);
+  buffer_.resize(kHeaderSize);
 
-  size_estimate_ = kMaxHeaderSize;
-  end_of_data_offset_ = kMaxHeaderSize;
+  size_estimate_ = kHeaderSize;
+  end_of_data_offset_ = kHeaderSize;
   finished_ = false;
 }
 
@@ -64,7 +67,7 @@ bool BinaryPlainBlockBuilder::IsBlockFull() const {
   return size_estimate_ > options_->storage_attributes.cfile_block_size;
 }
 
-Slice BinaryPlainBlockBuilder::Finish(rowid_t ordinal_pos) {
+void BinaryPlainBlockBuilder::Finish(rowid_t ordinal_pos, vector<Slice>* slices) {
   finished_ = true;
 
   size_t offsets_pos = buffer_.size();
@@ -79,7 +82,7 @@ Slice BinaryPlainBlockBuilder::Finish(rowid_t ordinal_pos) {
     coding::AppendGroupVarInt32Sequence(&buffer_, 0, &offsets_[0], offsets_.size());
   }
 
-  return Slice(buffer_);
+  *slices = { Slice(buffer_) };
 }
 
 int BinaryPlainBlockBuilder::Add(const uint8_t *vals, size_t count) {
@@ -131,8 +134,8 @@ Status BinaryPlainBlockBuilder::GetKeyAtIdx(void *key_void, int idx) const {
   }
 
   if (PREDICT_FALSE(offsets_.size() == 1)) {
-    *slice = Slice(&buffer_[kMaxHeaderSize],
-                   end_of_data_offset_ - kMaxHeaderSize);
+    *slice = Slice(&buffer_[kHeaderSize],
+                   end_of_data_offset_ - kHeaderSize);
   } else if (idx + 1 == offsets_.size()) {
     *slice = Slice(&buffer_[offsets_[idx]],
                    end_of_data_offset_ - offsets_[idx]);
diff --git a/src/kudu/cfile/binary_plain_block.h b/src/kudu/cfile/binary_plain_block.h
index 5a9e203..d70f779 100644
--- a/src/kudu/cfile/binary_plain_block.h
+++ b/src/kudu/cfile/binary_plain_block.h
@@ -63,12 +63,7 @@ class BinaryPlainBlockBuilder final : public BlockBuilder {
 
   int Add(const uint8_t *vals, size_t count) OVERRIDE;
 
-  // Return a Slice which represents the encoded data.
-  //
-  // This Slice points to internal data of this class
-  // and becomes invalid after the builder is destroyed
-  // or after Finish() is called again.
-  Slice Finish(rowid_t ordinal_pos) OVERRIDE;
+  void Finish(rowid_t ordinal_pos, std::vector<Slice>* slices) override;
 
   void Reset() OVERRIDE;
 
@@ -87,7 +82,7 @@ class BinaryPlainBlockBuilder final : public BlockBuilder {
   Status GetLastKey(void* key) const OVERRIDE;
 
   // Length of a header.
-  static const size_t kMaxHeaderSize = sizeof(uint32_t) * 3;
+  static constexpr size_t kHeaderSize = sizeof(uint32_t) * 3;
 
  private:
   faststring buffer_;
diff --git a/src/kudu/cfile/binary_prefix_block.cc b/src/kudu/cfile/binary_prefix_block.cc
index 5157a45..4d6da61 100644
--- a/src/kudu/cfile/binary_prefix_block.cc
+++ b/src/kudu/cfile/binary_prefix_block.cc
@@ -22,6 +22,7 @@
 #include <cstdint>
 #include <ostream>
 #include <string>
+#include <vector>
 
 #include <glog/logging.h>
 
@@ -47,6 +48,7 @@ namespace cfile {
 
 using kudu::coding::AppendGroupVarInt32;
 using std::string;
+using std::vector;
 using strings::Substitute;
 
 ////////////////////////////////////////////////////////////
@@ -84,7 +86,6 @@ void BinaryPrefixBlockBuilder::Reset() {
   vals_since_restart_ = 0;
 
   buffer_.clear();
-  buffer_.resize(kHeaderReservedLength);
   buffer_.reserve(options_->storage_attributes.cfile_block_size);
 
   restarts_.clear();
@@ -96,25 +97,13 @@ bool BinaryPrefixBlockBuilder::IsBlockFull() const {
   return buffer_.size() > options_->storage_attributes.cfile_block_size;
 }
 
-Slice BinaryPrefixBlockBuilder::Finish(rowid_t ordinal_pos) {
+void BinaryPrefixBlockBuilder::Finish(rowid_t ordinal_pos, vector<Slice>* slices) {
   CHECK(!finished_) << "already finished";
-  DCHECK_GE(buffer_.size(), kHeaderReservedLength);
 
-  faststring header(kHeaderReservedLength);
-
-  AppendGroupVarInt32(&header, val_count_, ordinal_pos,
+  header_buf_.clear();
+  AppendGroupVarInt32(&header_buf_, val_count_, ordinal_pos,
                       options_->block_restart_interval, 0);
 
-  int header_encoded_len = header.size();
-
-  // Copy the header into the buffer at the right spot.
-  // Since the header is likely shorter than the amount of space
-  // reserved for it, need to find where it fits:
-  int header_offset = kHeaderReservedLength - header_encoded_len;
-  DCHECK_GE(header_offset, 0);
-  uint8_t *header_dst = buffer_.data() + header_offset;
-  strings::memcpy_inlined(header_dst, header.data(), header_encoded_len);
-
   // Serialize the restart points.
   // Note that the values stored in restarts_ are relative to the
   // start of the *buffer*, which is not the same as the start of
@@ -123,15 +112,14 @@ Slice BinaryPrefixBlockBuilder::Finish(rowid_t ordinal_pos) {
                   + restarts_.size() * sizeof(uint32_t) // the data
                   + sizeof(uint32_t)); // the restart count);
   for (uint32_t restart : restarts_) {
-    DCHECK_GE(static_cast<int>(restart), header_offset);
-    uint32_t relative_to_block = restart - header_offset;
-    VLOG(2) << "appending restart " << relative_to_block;
-    InlinePutFixed32(&buffer_, relative_to_block);
+    // The encoded offsets are relative to the start of the block,
+    // inclusive of the header.
+    InlinePutFixed32(&buffer_, restart + header_buf_.size());
   }
   InlinePutFixed32(&buffer_, restarts_.size());
 
   finished_ = true;
-  return Slice(&buffer_[header_offset], buffer_.size() - header_offset);
+  *slices = { Slice(header_buf_), Slice(buffer_) };
 }
 
 int BinaryPrefixBlockBuilder::Add(const uint8_t *vals, size_t count) {
@@ -193,8 +181,9 @@ Status BinaryPrefixBlockBuilder::GetFirstKey(void *key) const {
     return Status::NotFound("no keys in data block");
   }
 
-  const uint8_t *p = &buffer_[kHeaderReservedLength];
-  uint32_t shared, non_shared;
+  const uint8_t *p = &buffer_[0];
+  uint32_t shared;
+  uint32_t non_shared;
   p = DecodeEntryLengths(p, &buffer_[buffer_.size()], &shared, &non_shared);
   if (p == nullptr) {
     return Status::Corruption("Could not decode first entry in string block");
diff --git a/src/kudu/cfile/binary_prefix_block.h b/src/kudu/cfile/binary_prefix_block.h
index 2f51650..817f327 100644
--- a/src/kudu/cfile/binary_prefix_block.h
+++ b/src/kudu/cfile/binary_prefix_block.h
@@ -51,12 +51,7 @@ class BinaryPrefixBlockBuilder final : public BlockBuilder {
 
   int Add(const uint8_t *vals, size_t count) OVERRIDE;
 
-  // Return a Slice which represents the encoded data.
-  //
-  // This Slice points to internal data of this class
-  // and becomes invalid after the builder is destroyed
-  // or after Finish() is called again.
-  Slice Finish(rowid_t ordinal_pos) OVERRIDE;
+  void Finish(rowid_t ordinal_pos, std::vector<Slice>* slices) override;
 
   void Reset() OVERRIDE;
 
@@ -71,6 +66,7 @@ class BinaryPrefixBlockBuilder final : public BlockBuilder {
   Status GetLastKey(void *key) const OVERRIDE;
 
  private:
+  faststring header_buf_;
   faststring buffer_;
   faststring last_val_;
 
@@ -82,13 +78,6 @@ class BinaryPrefixBlockBuilder final : public BlockBuilder {
   bool finished_;
 
   const WriterOptions *options_;
-
-  // Maximum length of a header.
-  // We leave this much space at the start of the buffer before
-  // accumulating any data, so we can later fill in the variable-length
-  // header.
-  // Currently four varints, so maximum is 20 bytes
-  static const size_t kHeaderReservedLength = 20;
 };
 
 // Decoder for BINARY type, PREFIX encoding
diff --git a/src/kudu/cfile/block_encodings.h b/src/kudu/cfile/block_encodings.h
index f7b7c10..08a9e96 100644
--- a/src/kudu/cfile/block_encodings.h
+++ b/src/kudu/cfile/block_encodings.h
@@ -19,6 +19,7 @@
 #define KUDU_CFILE_BLOCK_ENCODINGS_H
 
 #include <algorithm>
+#include <vector>
 #include <stdint.h>
 #include <glog/logging.h>
 
@@ -58,12 +59,13 @@ class BlockBuilder {
   // than requested if the block is full.
   virtual int Add(const uint8_t *vals, size_t count) = 0;
 
-  // Return a Slice which represents the encoded data.
+  // Return one or more Slices which represents the encoded data.
+  // The multiple slices will be concatenated when appended to the file.
   //
-  // This Slice points to internal data of this class
-  // and becomes invalid after the builder is destroyed
-  // or after Finish() is called again.
-  virtual Slice Finish(rowid_t ordinal_pos) = 0;
+  // These Slices may point to internal data of this class
+  // and can become invalid after the builder is destroyed
+  // or after any other method is called.
+  virtual void Finish(rowid_t ordinal_pos, std::vector<Slice>* slices) = 0;
 
   // Reset the internal state of the encoder.
   //
diff --git a/src/kudu/cfile/bshuf_block.cc b/src/kudu/cfile/bshuf_block.cc
index 7317103..d2dbfb3 100644
--- a/src/kudu/cfile/bshuf_block.cc
+++ b/src/kudu/cfile/bshuf_block.cc
@@ -21,6 +21,8 @@
 
 #include "kudu/gutil/port.h"
 
+using std::vector;
+
 namespace kudu {
 namespace cfile {
 
@@ -53,7 +55,7 @@ void AbortWithBitShuffleError(int64_t val) {
 // It dynamically switches the element size to UINT16 or UINT8 depending on the values
 // in the current block.
 template<>
-Slice BShufBlockBuilder<UINT32>::Finish(rowid_t ordinal_pos) {
+void BShufBlockBuilder<UINT32>::Finish(rowid_t ordinal_pos, vector<Slice>* slices) {
   RememberFirstAndLastKey();
   uint32_t max_value = 0;
   for (int i = 0; i < count_; i++) {
@@ -84,7 +86,7 @@ Slice BShufBlockBuilder<UINT32>::Finish(rowid_t ordinal_pos) {
     ret = Finish(ordinal_pos, sizeof(uint32_t));
     InlineEncodeFixed32(ret.mutable_data() + 16, sizeof(uint32_t));
   }
-  return ret;
+  *slices = { ret };
 }
 
 // Template specialization for UINT32.
diff --git a/src/kudu/cfile/bshuf_block.h b/src/kudu/cfile/bshuf_block.h
index 60c1545..387a6fc 100644
--- a/src/kudu/cfile/bshuf_block.h
+++ b/src/kudu/cfile/bshuf_block.h
@@ -29,6 +29,7 @@
 #include <cstring>
 #include <cstdint>
 #include <ostream>
+#include <vector>
 
 #include <glog/logging.h>
 
@@ -151,9 +152,9 @@ class BShufBlockBuilder final : public BlockBuilder {
     return Status::OK();
   }
 
-  Slice Finish(rowid_t ordinal_pos) OVERRIDE {
+  void Finish(rowid_t ordinal_pos, std::vector<Slice>* slices) OVERRIDE {
     RememberFirstAndLastKey();
-    return Finish(ordinal_pos, size_of_type);
+    *slices = { Finish(ordinal_pos, size_of_type) };
   }
 
  private:
@@ -224,7 +225,7 @@ class BShufBlockBuilder final : public BlockBuilder {
 };
 
 template<>
-Slice BShufBlockBuilder<UINT32>::Finish(rowid_t ordinal_pos);
+void BShufBlockBuilder<UINT32>::Finish(rowid_t ordinal_pos, std::vector<Slice>* slices);
 
 template<DataType Type>
 class BShufBlockDecoder final : public BlockDecoder {
diff --git a/src/kudu/cfile/cfile_writer.cc b/src/kudu/cfile/cfile_writer.cc
index 2921bf1..2cebb73 100644
--- a/src/kudu/cfile/cfile_writer.cc
+++ b/src/kudu/cfile/cfile_writer.cc
@@ -18,6 +18,7 @@
 #include "kudu/cfile/cfile_writer.h"
 
 #include <functional>
+#include <iterator>
 #include <numeric>
 #include <ostream>
 #include <utility>
@@ -373,8 +374,8 @@ Status CFileWriter::FinishCurDataBlock() {
 
   // The current data block is full, need to push it
   // into the file, and add to index
-  Slice data = data_block_->Finish(first_elem_ord);
-  VLOG(2) << " actual size=" << data.size();
+  vector<Slice> data_slices;
+  data_block_->Finish(first_elem_ord, &data_slices);
 
   uint8_t key_tmp_space[typeinfo_->size()];
   if (validx_builder_ != nullptr) {
@@ -394,7 +395,7 @@ Status CFileWriter::FinishCurDataBlock() {
     v.emplace_back(null_headers.data(), null_headers.size());
     v.push_back(non_null_bitmap);
   }
-  v.push_back(data);
+  std::move(data_slices.begin(), data_slices.end(), std::back_inserter(v));
   Status s = AppendRawBlock(v, first_elem_ord,
                             reinterpret_cast<const void *>(key_tmp_space),
                             Slice(last_key_),
diff --git a/src/kudu/cfile/encoding-test.cc b/src/kudu/cfile/encoding-test.cc
index f2527bc..cb25c1b 100644
--- a/src/kudu/cfile/encoding-test.cc
+++ b/src/kudu/cfile/encoding-test.cc
@@ -27,6 +27,7 @@
 #include <memory>
 #include <ostream>
 #include <string>
+#include <type_traits>
 #include <vector>
 
 #include <glog/logging.h>
@@ -47,6 +48,7 @@
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/stringprintf.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/faststring.h"
 #include "kudu/util/group_varint-inl.h"
 #include "kudu/util/hexdump.h"
 #include "kudu/util/int128.h"
@@ -93,9 +95,9 @@ class TestEncoding : public KuduTest {
   //
   // The strings are generated using the provided 'formatter' function.
   template<class BuilderType>
-  static Slice CreateBinaryBlock(BuilderType *sbb,
-                                 int num_items,
-                                 const std::function<string(int)>& formatter) {
+  Slice CreateBinaryBlock(BuilderType *sbb,
+                          int num_items,
+                          const std::function<string(int)>& formatter) {
     vector<string> to_insert(num_items);
     vector<Slice> slices;
     for (int i = 0; i < num_items; i++) {
@@ -114,10 +116,32 @@ class TestEncoding : public KuduTest {
     }
 
     CHECK_EQ(slices.size(), sbb->Count());
-    return sbb->Finish(12345L);
+    return FinishAndMakeContiguous(sbb, 12345L);
   }
 
-  WriterOptions* NewWriterOptions() {
+  // Concatenate the given slices into 'contiguous_buf_' and return a new slice.
+  Slice MakeContiguous(const vector<Slice>& slices) {
+    if (slices.size() == 1) {
+      return slices[0];
+    }
+    // Concat the slices into a local buffer, since the block decoders and
+    // tests expect contiguous data.
+    contiguous_buf_.clear();
+    for (const auto& s : slices) {
+      contiguous_buf_.append(s.data(), s.size());
+    }
+
+    return Slice(contiguous_buf_);
+  }
+
+  template<class BuilderType>
+  Slice FinishAndMakeContiguous(BuilderType* b, int ord_val) {
+    vector<Slice> slices;
+    b->Finish(ord_val, &slices);
+    return MakeContiguous(slices);
+  }
+
+  static WriterOptions* NewWriterOptions() {
     auto ret = new WriterOptions();
     ret->storage_attributes.cfile_block_size = 256 * 1024;
     return ret;
@@ -338,21 +362,22 @@ class TestEncoding : public KuduTest {
 
   template<class BlockBuilderType, class BlockDecoderType, DataType IntType>
   void DoSeekTest(BlockBuilderType* ibb, int num_ints, int num_queries, bool verify) {
-    // TODO : handle and verify seeking inside a run for testing RLE
+    // TODO(Alex Feinberg) : handle and verify seeking inside a run for testing RLE
     typedef typename TypeTraits<IntType>::cpp_type CppType;
 
-    const CppType kBase =
-        std::numeric_limits<CppType>::is_signed ? -6 : 6;
+    const CppType kBase = std::is_signed<CppType>::value ? -6 : 6;
 
     CppType data[num_ints];
-    for (CppType i = 0; i < num_ints; i++) {
+    // CppType may be uint8, int16 etc. and hence clang-tidy warning about loop variable smaller
+    // than num_ints data-type int. But callers supply correct values within range.
+    for (CppType i = 0; i < num_ints; i++) { // NOLINT(bugprone-too-small-loop-variable)
       data[i] = kBase + i * 2;
     }
     const CppType max_seek_target = data[num_ints - 1] + 1;
 
     CHECK_EQ(num_ints, ibb->Add(reinterpret_cast<uint8_t *>(&data[0]),
                                num_ints));
-    Slice s = ibb->Finish(0);
+    Slice s = FinishAndMakeContiguous(ibb, 0);
     LOG(INFO) << "Created " << TypeTraits<IntType>::name() << " block with " << num_ints << " ints"
               << " (" << s.size() << " bytes)";
     BlockDecoderType ibd(s);
@@ -400,7 +425,7 @@ class TestEncoding : public KuduTest {
   void TestEmptyBlockEncodeDecode() {
     unique_ptr<WriterOptions> opts(NewWriterOptions());
     BlockBuilderType bb(opts.get());
-    Slice s = bb.Finish(0);
+    Slice s = FinishAndMakeContiguous(&bb, 0);
     ASSERT_GT(s.size(), 0);
     LOG(INFO) << "Encoded size for 0 items: " << s.size();
 
@@ -419,7 +444,7 @@ class TestEncoding : public KuduTest {
     BlockBuilder pbb(opts.get());
 
     pbb.Add(reinterpret_cast<const uint8_t *>(src), size);
-    Slice s = pbb.Finish(kOrdinalPosBase);
+    Slice s = FinishAndMakeContiguous(&pbb, kOrdinalPosBase);
 
     LOG(INFO) << "Encoded size for 10k elems: " << s.size();
 
@@ -531,7 +556,7 @@ class TestEncoding : public KuduTest {
 
     ibb->Add(reinterpret_cast<const uint8_t *>(&to_insert[0]),
              to_insert.size());
-    Slice s = ibb->Finish(kOrdinalPosBase);
+    Slice s = FinishAndMakeContiguous(ibb, kOrdinalPosBase);
 
     // Check GetFirstKey() and GetLastKey().
     CppType key;
@@ -613,7 +638,7 @@ class TestEncoding : public KuduTest {
 
     vector<uint8_t> to_insert;
     for (int i = 0; i < 10003; ) {
-      int run_size = random() % 100;
+      int run_size = static_cast<int>(random() % 100);
       bool val = random() % 2;
       for (int j = 0; j < run_size; j++) {
         to_insert.push_back(val);
@@ -625,7 +650,7 @@ class TestEncoding : public KuduTest {
     BuilderType bb(opts.get());
     bb.Add(reinterpret_cast<const uint8_t *>(&to_insert[0]),
            to_insert.size());
-    Slice s = bb.Finish(kOrdinalPosBase);
+    Slice s = FinishAndMakeContiguous(&bb, kOrdinalPosBase);
 
     DecoderType bd(s);
     ASSERT_OK(bd.ParseHeader());
@@ -677,6 +702,7 @@ class TestEncoding : public KuduTest {
   }
 
   Arena arena_;
+  faststring contiguous_buf_;
 };
 
 TEST_F(TestEncoding, TestPlainBlockEncoder) {
@@ -763,7 +789,7 @@ TEST_F(TestEncoding, TestRleIntBlockEncoder) {
   }
   ibb.Add(reinterpret_cast<const uint8_t *>(ints.get()), 10000);
 
-  Slice s = ibb.Finish(12345);
+  Slice s = FinishAndMakeContiguous(&ibb, 12345);
   LOG(INFO) << "RLE Encoded size for 10k ints: " << s.size();
 
   ibb.Reset();
@@ -772,7 +798,7 @@ TEST_F(TestEncoding, TestRleIntBlockEncoder) {
     ints[i] = 0;
   }
   ibb.Add(reinterpret_cast<const uint8_t *>(ints.get()), 100);
-  s = ibb.Finish(12345);
+  s = FinishAndMakeContiguous(&ibb, 12345);
   ASSERT_EQ(14UL, s.size());
 }
 
diff --git a/src/kudu/cfile/plain_bitmap_block.h b/src/kudu/cfile/plain_bitmap_block.h
index 0691a18..b7fa2dc 100644
--- a/src/kudu/cfile/plain_bitmap_block.h
+++ b/src/kudu/cfile/plain_bitmap_block.h
@@ -20,6 +20,7 @@
 
 #include <algorithm>
 #include <string>
+#include <vector>
 
 #include "kudu/cfile/block_encodings.h"
 #include "kudu/cfile/cfile_util.h"
@@ -65,11 +66,11 @@ class PlainBitMapBlockBuilder final : public BlockBuilder {
     return count;
   }
 
-  virtual Slice Finish(rowid_t ordinal_pos) OVERRIDE {
+  virtual void Finish(rowid_t ordinal_pos, std::vector<Slice>* slices) OVERRIDE {
     InlineEncodeFixed32(&buf_[0], count_);
     InlineEncodeFixed32(&buf_[4], ordinal_pos);
     writer_.Flush(false);
-    return Slice(buf_);
+    *slices = { Slice(buf_) };
   }
 
   virtual void Reset() OVERRIDE {
diff --git a/src/kudu/cfile/plain_block.h b/src/kudu/cfile/plain_block.h
index 5c11534..d4ba8a6 100644
--- a/src/kudu/cfile/plain_block.h
+++ b/src/kudu/cfile/plain_block.h
@@ -19,6 +19,7 @@
 
 #include <algorithm>
 #include <string>
+#include <vector>
 
 #include "kudu/cfile/block_encodings.h"
 #include "kudu/cfile/cfile_util.h"
@@ -66,10 +67,10 @@ class PlainBlockBuilder final : public BlockBuilder {
     return buffer_.size() > options_->storage_attributes.cfile_block_size;
   }
 
-  virtual Slice Finish(rowid_t ordinal_pos) OVERRIDE {
+  virtual void Finish(rowid_t ordinal_pos, std::vector<Slice>* slices) OVERRIDE {
     InlineEncodeFixed32(&buffer_[0], count_);
     InlineEncodeFixed32(&buffer_[4], ordinal_pos);
-    return Slice(buffer_);
+    *slices = { Slice(buffer_) };
   }
 
   virtual void Reset() OVERRIDE {
diff --git a/src/kudu/cfile/rle_block.h b/src/kudu/cfile/rle_block.h
index 7595494..dfc325a 100644
--- a/src/kudu/cfile/rle_block.h
+++ b/src/kudu/cfile/rle_block.h
@@ -20,6 +20,7 @@
 
 #include <algorithm>
 #include <string>
+#include <vector>
 
 #include "kudu/gutil/port.h"
 #include "kudu/cfile/block_encodings.h"
@@ -70,11 +71,11 @@ class RleBitMapBlockBuilder final : public BlockBuilder {
     return encoder_.len() > options_->storage_attributes.cfile_block_size;
   }
 
-  virtual Slice Finish(rowid_t ordinal_pos) OVERRIDE {
+  virtual void Finish(rowid_t ordinal_pos, std::vector<Slice>* slices) OVERRIDE {
     InlineEncodeFixed32(&buf_[0], count_);
     InlineEncodeFixed32(&buf_[4], ordinal_pos);
     encoder_.Flush();
-    return Slice(buf_);
+    *slices = { buf_ };
   }
 
   virtual void Reset() OVERRIDE {
@@ -250,11 +251,11 @@ class RleIntBlockBuilder final : public BlockBuilder {
     return count;
   }
 
-  virtual Slice Finish(rowid_t ordinal_pos) OVERRIDE {
+  virtual void Finish(rowid_t ordinal_pos, std::vector<Slice>* slices) OVERRIDE {
     InlineEncodeFixed32(&buf_[0], count_);
     InlineEncodeFixed32(&buf_[4], ordinal_pos);
     rle_encoder_.Flush();
-    return Slice(buf_);
+    *slices = { Slice(buf_) };
   }
 
   virtual void Reset() OVERRIDE {