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 2019/01/25 00:58:14 UTC

[kudu] branch master updated: switch all iterators to unique_ptr

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

adar 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 33d3878  switch all iterators to unique_ptr
33d3878 is described below

commit 33d387866d7f1ce2f7b51d8173d6f4833c2416fc
Author: Adar Dembo <ad...@cloudera.com>
AuthorDate: Thu Jan 10 16:18:14 2019 -0800

    switch all iterators to unique_ptr
    
    We've suffered from a longstanding wart in the iterator subsystem: a mix of
    raw and various kinds of smart pointers. The usage of shared_ptr was
    particularly vexing as it suggested that iterators had shared ownership when
    in fact they didn't.
    
    This yak shave of a patch addresses all of those issues by converting all of
    the iterator pointer types to unique_ptr. I snuck in some clang-tidy
    suggestions too, but otherwise the cleanup here should narrowly scoped.
    
    Change-Id: I60cdc49e6a209c72cb33681ae1be89586d739204
    Reviewed-on: http://gerrit.cloudera.org:8080/12222
    Tested-by: Kudu Jenkins
    Reviewed-by: Mike Percy <mp...@apache.org>
---
 src/kudu/cfile/cfile-test-base.h                   | 36 +++++++------
 src/kudu/cfile/cfile-test.cc                       |  6 +--
 src/kudu/cfile/cfile_reader.cc                     |  5 +-
 src/kudu/cfile/cfile_reader.h                      | 10 +---
 src/kudu/common/generic_iterators-test.cc          | 60 +++++++++++++---------
 src/kudu/common/generic_iterators.cc               | 29 ++++++-----
 src/kudu/common/generic_iterators.h                | 18 +++----
 src/kudu/integration-tests/linked_list-test-util.h |  4 +-
 src/kudu/master/sys_catalog.cc                     |  2 +-
 src/kudu/tablet/all_types-scan-correctness-test.cc |  5 +-
 src/kudu/tablet/cfile_set-test.cc                  | 26 +++++-----
 src/kudu/tablet/cfile_set.cc                       | 32 ++++++------
 src/kudu/tablet/cfile_set.h                        | 16 +++---
 src/kudu/tablet/compaction.cc                      | 11 ++--
 src/kudu/tablet/composite-pushdown-test.cc         |  4 +-
 src/kudu/tablet/delta_compaction-test.cc           |  9 ++--
 src/kudu/tablet/delta_compaction.cc                |  5 +-
 src/kudu/tablet/delta_iterator_merger.cc           |  6 +--
 src/kudu/tablet/delta_store.h                      |  2 +-
 src/kudu/tablet/delta_tracker.cc                   |  2 +-
 src/kudu/tablet/delta_tracker.h                    |  2 +-
 src/kudu/tablet/deltafile-test.cc                  | 45 ++++++----------
 src/kudu/tablet/deltafile.cc                       | 13 ++---
 src/kudu/tablet/deltafile.h                        |  2 +-
 src/kudu/tablet/deltamemstore-test.cc              | 18 +++----
 src/kudu/tablet/deltamemstore.cc                   |  6 ++-
 src/kudu/tablet/deltamemstore.h                    |  2 +-
 src/kudu/tablet/diff_scan-test.cc                  |  4 +-
 src/kudu/tablet/diskrowset-test-base.h             |  6 +--
 src/kudu/tablet/diskrowset-test.cc                 |  6 +--
 src/kudu/tablet/diskrowset.cc                      |  7 ++-
 src/kudu/tablet/diskrowset.h                       |  2 +-
 src/kudu/tablet/major_delta_compaction-test.cc     |  6 +--
 src/kudu/tablet/memrowset-test.cc                  | 13 +++--
 src/kudu/tablet/memrowset.cc                       |  7 +--
 src/kudu/tablet/memrowset.h                        |  2 +-
 src/kudu/tablet/mock-rowsets.h                     |  2 +-
 src/kudu/tablet/mt-rowset_delta_compaction-test.cc |  5 +-
 src/kudu/tablet/mt-tablet-test.cc                  | 17 +++---
 src/kudu/tablet/rowset.cc                          |  9 ++--
 src/kudu/tablet/rowset.h                           |  4 +-
 src/kudu/tablet/tablet-decoder-eval-test.cc        | 10 ++--
 src/kudu/tablet/tablet-pushdown-test.cc            |  8 +--
 src/kudu/tablet/tablet-schema-test.cc              |  4 +-
 src/kudu/tablet/tablet-test-base.h                 | 13 ++---
 src/kudu/tablet/tablet-test-util.h                 | 13 +++--
 src/kudu/tablet/tablet-test.cc                     |  6 +--
 src/kudu/tablet/tablet.cc                          | 25 ++++-----
 src/kudu/tablet/tablet.h                           |  8 +--
 src/kudu/tablet/tablet_bootstrap-test.cc           |  6 +--
 src/kudu/tablet/tablet_random_access-test.cc       |  7 +--
 src/kudu/tools/tool_action_fs.cc                   |  5 +-
 src/kudu/tserver/scanners.cc                       |  6 ++-
 src/kudu/tserver/scanners.h                        |  4 +-
 src/kudu/tserver/tablet_server-test-base.cc        |  6 +--
 src/kudu/tserver/tablet_service.cc                 |  5 +-
 src/kudu/tserver/tablet_service.h                  |  4 +-
 57 files changed, 294 insertions(+), 302 deletions(-)

diff --git a/src/kudu/cfile/cfile-test-base.h b/src/kudu/cfile/cfile-test-base.h
index e48d6eb..ae924db 100644
--- a/src/kudu/cfile/cfile-test-base.h
+++ b/src/kudu/cfile/cfile-test-base.h
@@ -18,13 +18,15 @@
 #ifndef KUDU_CFILE_TEST_BASE_H
 #define KUDU_CFILE_TEST_BASE_H
 
-#include <glog/logging.h>
 #include <algorithm>
+#include <cstdlib>
 #include <functional>
-#include <stdlib.h>
+#include <memory>
 #include <string>
 #include <vector>
 
+#include <glog/logging.h>
+
 #include "kudu/cfile/cfile.pb.h"
 #include "kudu/cfile/cfile_reader.h"
 #include "kudu/cfile/cfile_writer.h"
@@ -438,7 +440,7 @@ SumType FastSum(const Indexable &data, size_t n) {
 }
 
 template<DataType Type, typename SumType>
-void TimeReadFileForDataType(gscoped_ptr<CFileIterator> &iter, int &count) {
+void TimeReadFileForDataType(CFileIterator* iter, int* count) {
   ScopedColumnBlock<Type> cb(8192);
   SelectionVector sel(cb.nrows());
   ColumnMaterializationContext ctx(0, nullptr, &cb, &sel);
@@ -448,11 +450,11 @@ void TimeReadFileForDataType(gscoped_ptr<CFileIterator> &iter, int &count) {
     size_t n = cb.nrows();
     ASSERT_OK_FAST(iter->CopyNextValues(&n, &ctx));
     sum += FastSum<ScopedColumnBlock<Type>, SumType>(cb, n);
-    count += n;
+    *count += n;
     cb.arena()->Reset();
   }
   LOG(INFO)<< "Sum: " << sum;
-  LOG(INFO)<< "Count: " << count;
+  LOG(INFO)<< "Count: " << *count;
 }
 
 template<DataType Type>
@@ -485,7 +487,7 @@ void TimeReadFile(FsManager* fs_manager, const BlockId& block_id, size_t *count_
   std::unique_ptr<CFileReader> reader;
   ASSERT_OK(CFileReader::Open(std::move(source), ReaderOptions(), &reader));
 
-  gscoped_ptr<CFileIterator> iter;
+  std::unique_ptr<CFileIterator> iter;
   ASSERT_OK(reader->NewIterator(&iter, CFileReader::CACHE_BLOCK, nullptr));
   ASSERT_OK(iter->SeekToOrdinal(0));
 
@@ -494,57 +496,57 @@ void TimeReadFile(FsManager* fs_manager, const BlockId& block_id, size_t *count_
   switch (reader->type_info()->physical_type()) {
     case UINT8:
     {
-      TimeReadFileForDataType<UINT8, uint64_t>(iter, count);
+      TimeReadFileForDataType<UINT8, uint64_t>(iter.get(), &count);
       break;
     }
     case INT8:
     {
-      TimeReadFileForDataType<INT8, int64_t>(iter, count);
+      TimeReadFileForDataType<INT8, int64_t>(iter.get(), &count);
       break;
     }
     case UINT16:
     {
-      TimeReadFileForDataType<UINT16, uint64_t>(iter, count);
+      TimeReadFileForDataType<UINT16, uint64_t>(iter.get(), &count);
       break;
     }
     case INT16:
     {
-      TimeReadFileForDataType<INT16, int64_t>(iter, count);
+      TimeReadFileForDataType<INT16, int64_t>(iter.get(), &count);
       break;
     }
     case UINT32:
     {
-      TimeReadFileForDataType<UINT32, uint64_t>(iter, count);
+      TimeReadFileForDataType<UINT32, uint64_t>(iter.get(), &count);
       break;
     }
     case INT32:
     {
-      TimeReadFileForDataType<INT32, int64_t>(iter, count);
+      TimeReadFileForDataType<INT32, int64_t>(iter.get(), &count);
       break;
     }
     case UINT64:
     {
-      TimeReadFileForDataType<UINT64, uint64_t>(iter, count);
+      TimeReadFileForDataType<UINT64, uint64_t>(iter.get(), &count);
       break;
     }
     case INT64:
     {
-      TimeReadFileForDataType<INT64, int64_t>(iter, count);
+      TimeReadFileForDataType<INT64, int64_t>(iter.get(), &count);
       break;
     }
     case INT128:
     {
-      TimeReadFileForDataType<INT128, int128_t>(iter, count);
+      TimeReadFileForDataType<INT128, int128_t>(iter.get(), &count);
       break;
     }
     case FLOAT:
     {
-      TimeReadFileForDataType<FLOAT, float>(iter, count);
+      TimeReadFileForDataType<FLOAT, float>(iter.get(), &count);
       break;
     }
     case DOUBLE:
     {
-      TimeReadFileForDataType<DOUBLE, double>(iter, count);
+      TimeReadFileForDataType<DOUBLE, double>(iter.get(), &count);
       break;
     }
     case STRING:
diff --git a/src/kudu/cfile/cfile-test.cc b/src/kudu/cfile/cfile-test.cc
index a524155..78bdefd 100644
--- a/src/kudu/cfile/cfile-test.cc
+++ b/src/kudu/cfile/cfile-test.cc
@@ -118,7 +118,7 @@ class TestCFile : public CFileTestBase {
     ASSERT_OK(CFileReader::Open(std::move(block), ReaderOptions(), &reader));
 
     BlockPointer ptr;
-    gscoped_ptr<CFileIterator> iter;
+    unique_ptr<CFileIterator> iter;
     ASSERT_OK(reader->NewIterator(&iter, CFileReader::CACHE_BLOCK, nullptr));
 
     ASSERT_OK(iter->SeekToOrdinal(5000));
@@ -201,7 +201,7 @@ class TestCFile : public CFileTestBase {
     ASSERT_OK(CFileReader::Open(std::move(block), ReaderOptions(), &reader));
     ASSERT_EQ(DataGeneratorType::kDataType, reader->type_info()->type());
 
-    gscoped_ptr<CFileIterator> iter;
+    unique_ptr<CFileIterator> iter;
     ASSERT_OK(reader->NewIterator(&iter, CFileReader::CACHE_BLOCK, nullptr));
 
     Arena arena(8192);
@@ -619,7 +619,7 @@ void TestCFile::TestReadWriteStrings(EncodingType encoding,
 
   BlockPointer ptr;
 
-  gscoped_ptr<CFileIterator> iter;
+  unique_ptr<CFileIterator> iter;
   ASSERT_OK(reader->NewIterator(&iter, CFileReader::CACHE_BLOCK, nullptr));
 
   Arena arena(1024);
diff --git a/src/kudu/cfile/cfile_reader.cc b/src/kudu/cfile/cfile_reader.cc
index 637d202..12794a0 100644
--- a/src/kudu/cfile/cfile_reader.cc
+++ b/src/kudu/cfile/cfile_reader.cc
@@ -596,9 +596,10 @@ void CFileReader::HandleCorruption(const fs::IOContext* io_context) const {
       ErrorHandlerType::CFILE_CORRUPTION, io_context->tablet_id);
 }
 
-Status CFileReader::NewIterator(CFileIterator** iter, CacheControl cache_control,
+Status CFileReader::NewIterator(unique_ptr<CFileIterator>* iter,
+                                CacheControl cache_control,
                                 const IOContext* io_context) {
-  *iter = new CFileIterator(this, cache_control, io_context);
+  iter->reset(new CFileIterator(this, cache_control, io_context));
   return Status::OK();
 }
 
diff --git a/src/kudu/cfile/cfile_reader.h b/src/kudu/cfile/cfile_reader.h
index 2e57f1f..9df42ee 100644
--- a/src/kudu/cfile/cfile_reader.h
+++ b/src/kudu/cfile/cfile_reader.h
@@ -100,17 +100,9 @@ class CFileReader {
   };
 
   // Can be called before Init().
-  Status NewIterator(CFileIterator** iter,
+  Status NewIterator(std::unique_ptr<CFileIterator>* iter,
                      CacheControl cache_control,
                      const fs::IOContext* io_context);
-  Status NewIterator(gscoped_ptr<CFileIterator>* iter,
-                     CacheControl cache_control,
-                     const fs::IOContext* io_context) {
-    CFileIterator* iter_ptr;
-    RETURN_NOT_OK(NewIterator(&iter_ptr, cache_control, io_context));
-    (*iter).reset(iter_ptr);
-    return Status::OK();
-  }
 
   // Reads the data block pointed to by `ptr`. Will pull the data block from
   // the block cache if it exists, and reads from the filesystem block
diff --git a/src/kudu/common/generic_iterators-test.cc b/src/kudu/common/generic_iterators-test.cc
index 735836e..4561dad 100644
--- a/src/kudu/common/generic_iterators-test.cc
+++ b/src/kudu/common/generic_iterators-test.cc
@@ -54,8 +54,6 @@ DEFINE_int32(num_lists, 3, "Number of lists to merge");
 DEFINE_int32(num_rows, 1000, "Number of entries per list");
 DEFINE_int32(num_iters, 1, "Number of times to run merge");
 
-using std::make_shared;
-using std::shared_ptr;
 using std::string;
 using std::unique_ptr;
 using std::vector;
@@ -162,10 +160,12 @@ class VectorIterator : public ColumnwiseIterator {
 
 // Test that empty input to a merger behaves correctly.
 TEST(TestMergeIterator, TestMergeEmpty) {
-  shared_ptr<RowwiseIterator> iter(
-    new MaterializingIterator(
-        shared_ptr<ColumnwiseIterator>(new VectorIterator({}))));
-  MergeIterator merger({ std::move(iter) });
+  unique_ptr<RowwiseIterator> iter(
+      new MaterializingIterator(
+          unique_ptr<ColumnwiseIterator>(new VectorIterator({}))));
+  vector<unique_ptr<RowwiseIterator>> input;
+  input.emplace_back(std::move(iter));
+  MergeIterator merger(std::move(input));
   ASSERT_OK(merger.Init(nullptr));
   ASSERT_FALSE(merger.HasNext());
 }
@@ -175,10 +175,12 @@ TEST(TestMergeIterator, TestMergeEmpty) {
 TEST(TestMergeIterator, TestMergeEmptyViaSelectionVector) {
   SelectionVector sv(3);
   sv.SetAllFalse();
-  shared_ptr<VectorIterator> vec(new VectorIterator({ 1, 2, 3 }));
+  unique_ptr<VectorIterator> vec(new VectorIterator({ 1, 2, 3 }));
   vec->set_selection_vector(&sv);
-  shared_ptr<RowwiseIterator> iter(new MaterializingIterator(std::move(vec)));
-  MergeIterator merger({ std::move(iter) });
+  unique_ptr<RowwiseIterator> iter(new MaterializingIterator(std::move(vec)));
+  vector<unique_ptr<RowwiseIterator>> input;
+  input.emplace_back(std::move(iter));
+  MergeIterator merger(std::move(input));
   ASSERT_OK(merger.Init(nullptr));
   ASSERT_FALSE(merger.HasNext());
 }
@@ -242,13 +244,15 @@ void TestMerge(const TestIntRangePredicate &predicate) {
   VLOG(1) << "Predicate expects " << expected.size() << " results: " << expected;
 
   for (int trial = 0; trial < FLAGS_num_iters; trial++) {
-    vector<shared_ptr<RowwiseIterator>> to_merge;
+    vector<unique_ptr<RowwiseIterator>> to_merge;
     for (const auto& e : all_ints) {
-      shared_ptr<VectorIterator> vec_it(new VectorIterator(e.ints));
+      unique_ptr<VectorIterator> vec_it(new VectorIterator(e.ints));
       vec_it->set_block_size(10);
       vec_it->set_selection_vector(e.sv.get());
-      shared_ptr<RowwiseIterator> mat_it(new MaterializingIterator(std::move(vec_it)));
-      shared_ptr<RowwiseIterator> un_it(new UnionIterator({ std::move(mat_it) }));
+      unique_ptr<RowwiseIterator> mat_it(new MaterializingIterator(std::move(vec_it)));
+      vector<unique_ptr<RowwiseIterator>> to_union;
+      to_union.emplace_back(std::move(mat_it));
+      unique_ptr<RowwiseIterator> un_it(new UnionIterator(std::move(to_union)));
       to_merge.emplace_back(std::move(un_it));
     }
 
@@ -316,8 +320,8 @@ TEST(TestMaterializingIterator, TestMaterializingPredicatePushdown) {
     ints[i] = i;
   }
 
-  shared_ptr<VectorIterator> colwise(new VectorIterator(std::move(ints)));
-  MaterializingIterator materializing(colwise);
+  unique_ptr<VectorIterator> colwise(new VectorIterator(std::move(ints)));
+  MaterializingIterator materializing(std::move(colwise));
   ASSERT_OK(materializing.Init(&spec));
   ASSERT_EQ(0, spec.predicates().size()) << "Iterator should have pushed down predicate";
 
@@ -349,16 +353,18 @@ TEST(TestPredicateEvaluatingIterator, TestPredicateEvaluation) {
 
   // Set up a MaterializingIterator with pushdown disabled, so that the
   // PredicateEvaluatingIterator will wrap it and do evaluation.
-  shared_ptr<VectorIterator> colwise(new VectorIterator(std::move(ints)));
-  MaterializingIterator *materializing = new MaterializingIterator(colwise);
+  unique_ptr<VectorIterator> colwise(new VectorIterator(std::move(ints)));
+  unique_ptr<MaterializingIterator> materializing(
+      new MaterializingIterator(std::move(colwise)));
   materializing->disallow_pushdown_for_tests_ = true;
 
   // Wrap it in another iterator to do the evaluation
-  shared_ptr<RowwiseIterator> outer_iter(materializing);
+  const MaterializingIterator* mat_iter_addr = materializing.get();
+  unique_ptr<RowwiseIterator> outer_iter(std::move(materializing));
   ASSERT_OK(PredicateEvaluatingIterator::InitAndMaybeWrap(&outer_iter, &spec));
 
   ASSERT_NE(reinterpret_cast<uintptr_t>(outer_iter.get()),
-            reinterpret_cast<uintptr_t>(materializing))
+            reinterpret_cast<uintptr_t>(mat_iter_addr))
     << "Iterator pointer should differ after wrapping";
 
   PredicateEvaluatingIterator *pred_eval = down_cast<PredicateEvaluatingIterator *>(
@@ -388,11 +394,15 @@ TEST(TestPredicateEvaluatingIterator, TestDontWrapWhenNoPredicates) {
   ScanSpec spec;
 
   vector<uint32_t> ints;
-  shared_ptr<VectorIterator> colwise(new VectorIterator(std::move(ints)));
-  shared_ptr<RowwiseIterator> materializing(new MaterializingIterator(colwise));
-  shared_ptr<RowwiseIterator> outer_iter(materializing);
+  unique_ptr<VectorIterator> colwise(new VectorIterator(std::move(ints)));
+  unique_ptr<MaterializingIterator> materializing(
+      new MaterializingIterator(std::move(colwise)));
+  const MaterializingIterator* mat_iter_addr = materializing.get();
+  unique_ptr<RowwiseIterator> outer_iter(std::move(materializing));
   ASSERT_OK(PredicateEvaluatingIterator::InitAndMaybeWrap(&outer_iter, &spec));
-  ASSERT_EQ(outer_iter, materializing) << "InitAndMaybeWrap should not have wrapped iter";
+  ASSERT_EQ(reinterpret_cast<uintptr_t>(outer_iter.get()),
+            reinterpret_cast<uintptr_t>(mat_iter_addr))
+      << "InitAndMaybeWrap should not have wrapped iter";
 }
 
 // Test row-wise iterator which does nothing.
@@ -451,7 +461,7 @@ TEST(TestPredicateEvaluatingIterator, TestPredicateEvaluationOrder) {
     spec.AddPredicate(b_equality);
     spec.AddPredicate(c_equality);
 
-    shared_ptr<RowwiseIterator> iter = make_shared<DummyIterator>(schema);
+    unique_ptr<RowwiseIterator> iter(new DummyIterator(schema));
     ASSERT_OK(PredicateEvaluatingIterator::InitAndMaybeWrap(&iter, &spec));
 
     PredicateEvaluatingIterator* pred_eval = down_cast<PredicateEvaluatingIterator*>(iter.get());
@@ -465,7 +475,7 @@ TEST(TestPredicateEvaluatingIterator, TestPredicateEvaluationOrder) {
     spec.AddPredicate(a_equality);
     spec.AddPredicate(c_equality);
 
-    shared_ptr<RowwiseIterator> iter = make_shared<DummyIterator>(schema);
+    unique_ptr<RowwiseIterator> iter(new DummyIterator(schema));
     ASSERT_OK(PredicateEvaluatingIterator::InitAndMaybeWrap(&iter, &spec));
 
     PredicateEvaluatingIterator* pred_eval = down_cast<PredicateEvaluatingIterator*>(iter.get());
diff --git a/src/kudu/common/generic_iterators.cc b/src/kudu/common/generic_iterators.cc
index 1e7c236..27c0515 100644
--- a/src/kudu/common/generic_iterators.cc
+++ b/src/kudu/common/generic_iterators.cc
@@ -66,7 +66,7 @@ TAG_FLAG(materializing_iterator_decoder_eval, runtime);
 namespace kudu {
 namespace {
 void AddIterStats(const RowwiseIterator& iter,
-                  std::vector<IteratorStats>* stats) {
+                  vector<IteratorStats>* stats) {
   vector<IteratorStats> iter_stats;
   iter.GetIteratorStats(&iter_stats);
   DCHECK_EQ(stats->size(), iter_stats.size());
@@ -88,7 +88,7 @@ static const int kMergeRowBuffer = 1000;
 // such that all returned rows are valid.
 class MergeIterState {
  public:
-  explicit MergeIterState(shared_ptr<RowwiseIterator> iter) :
+  explicit MergeIterState(unique_ptr<RowwiseIterator> iter) :
       iter_(std::move(iter)),
       arena_(1024),
       read_block_(iter_->schema(), kMergeRowBuffer, &arena_),
@@ -120,7 +120,7 @@ class MergeIterState {
   Status Advance();
 
   // Add statistics about the underlying iterator to the given vector.
-  void AddStats(std::vector<IteratorStats>* stats) const {
+  void AddStats(vector<IteratorStats>* stats) const {
     AddIterStats(*iter_, stats);
   }
 
@@ -143,7 +143,7 @@ class MergeIterState {
     return rows_advanced_ == rows_valid_;
   }
 
-  shared_ptr<RowwiseIterator> iter_;
+  unique_ptr<RowwiseIterator> iter_;
   Arena arena_;
   RowBlock read_block_;
 
@@ -211,9 +211,9 @@ Status MergeIterState::PullNextBlock() {
   return Status::OK();
 }
 
-MergeIterator::MergeIterator(vector<shared_ptr<RowwiseIterator>> iters)
+MergeIterator::MergeIterator(vector<unique_ptr<RowwiseIterator>> iters)
     : initted_(false),
-      orig_iters_(std::move(iters)),
+      orig_iters_(move(iters)),
       num_orig_iters_(orig_iters_.size()) {
   CHECK_GT(orig_iters_.size(), 0);
 }
@@ -382,7 +382,7 @@ void MergeIterator::GetIteratorStats(vector<IteratorStats>* stats) const {
 // Union iterator
 ////////////////////////////////////////////////////////////
 
-UnionIterator::UnionIterator(vector<shared_ptr<RowwiseIterator>> iters)
+UnionIterator::UnionIterator(vector<unique_ptr<RowwiseIterator>> iters)
   : initted_(false),
     iters_(std::make_move_iterator(iters.begin()),
            std::make_move_iterator(iters.end())) {
@@ -476,14 +476,14 @@ void UnionIterator::PopFront() {
 string UnionIterator::ToString() const {
   string s;
   s.append("Union(");
-  s += JoinMapped(iters_, [](const shared_ptr<RowwiseIterator>& it) {
+  s += JoinMapped(iters_, [](const unique_ptr<RowwiseIterator>& it) {
       return it->ToString();
     }, ",");
   s.append(")");
   return s;
 }
 
-void UnionIterator::GetIteratorStats(std::vector<IteratorStats>* stats) const {
+void UnionIterator::GetIteratorStats(vector<IteratorStats>* stats) const {
   CHECK(initted_);
   shared_lock<rw_spinlock> l(iters_lock_);
   *stats = finished_iter_stats_by_col_;
@@ -496,7 +496,7 @@ void UnionIterator::GetIteratorStats(std::vector<IteratorStats>* stats) const {
 // Materializing iterator
 ////////////////////////////////////////////////////////////
 
-MaterializingIterator::MaterializingIterator(shared_ptr<ColumnwiseIterator> iter)
+MaterializingIterator::MaterializingIterator(unique_ptr<ColumnwiseIterator> iter)
     : iter_(move(iter)),
       disallow_pushdown_for_tests_(!FLAGS_materializing_iterator_do_pushdown),
       disallow_decoder_eval_(!FLAGS_materializing_iterator_decoder_eval) {
@@ -631,19 +631,20 @@ string MaterializingIterator::ToString() const {
 // PredicateEvaluatingIterator
 ////////////////////////////////////////////////////////////
 
-PredicateEvaluatingIterator::PredicateEvaluatingIterator(shared_ptr<RowwiseIterator> base_iter)
+PredicateEvaluatingIterator::PredicateEvaluatingIterator(unique_ptr<RowwiseIterator> base_iter)
     : base_iter_(move(base_iter)) {
 }
 
 Status PredicateEvaluatingIterator::InitAndMaybeWrap(
-  shared_ptr<RowwiseIterator> *base_iter, ScanSpec *spec) {
+  unique_ptr<RowwiseIterator> *base_iter, ScanSpec *spec) {
   RETURN_NOT_OK((*base_iter)->Init(spec));
 
   if (spec != nullptr && !spec->predicates().empty()) {
     // Underlying iterator did not accept all predicates. Wrap it.
-    shared_ptr<RowwiseIterator> wrapper(new PredicateEvaluatingIterator(*base_iter));
+    unique_ptr<RowwiseIterator> wrapper(
+        new PredicateEvaluatingIterator(std::move(*base_iter)));
     CHECK_OK(wrapper->Init(spec));
-    base_iter->swap(wrapper);
+    *base_iter = std::move(wrapper);
   }
   return Status::OK();
 }
diff --git a/src/kudu/common/generic_iterators.h b/src/kudu/common/generic_iterators.h
index bc3ac63..5edb0ee 100644
--- a/src/kudu/common/generic_iterators.h
+++ b/src/kudu/common/generic_iterators.h
@@ -54,7 +54,7 @@ class MergeIterator : public RowwiseIterator {
   //
   // Note: the iterators must be constructed using a projection that includes
   // all key columns; otherwise a CHECK will fire at initialization time.
-  explicit MergeIterator(std::vector<std::shared_ptr<RowwiseIterator>> iters);
+  explicit MergeIterator(std::vector<std::unique_ptr<RowwiseIterator>> iters);
 
   virtual ~MergeIterator();
 
@@ -83,7 +83,7 @@ class MergeIterator : public RowwiseIterator {
 
   // Holds the subiterators until Init is called, at which point this is cleared.
   // This is required because we can't create a MergeIterState of an uninitialized iterator.
-  std::vector<std::shared_ptr<RowwiseIterator>> orig_iters_;
+  std::vector<std::unique_ptr<RowwiseIterator>> orig_iters_;
 
   // See UnionIterator::states_lock_ for details on locking. This follows the same
   // pattern.
@@ -114,7 +114,7 @@ class UnionIterator : public RowwiseIterator {
   // Constructs a UnionIterator of the given iterators.
   //
   // The iterators must have matching schemas and should not yet be initialized.
-  explicit UnionIterator(std::vector<std::shared_ptr<RowwiseIterator>> iters);
+  explicit UnionIterator(std::vector<std::unique_ptr<RowwiseIterator>> iters);
 
   Status Init(ScanSpec *spec) OVERRIDE;
 
@@ -156,7 +156,7 @@ class UnionIterator : public RowwiseIterator {
   // it's the only thread which might write. However, it does need to acquire in write
   // mode when changing 'iters_'.
   mutable rw_spinlock iters_lock_;
-  std::deque<std::shared_ptr<RowwiseIterator>> iters_;
+  std::deque<std::unique_ptr<RowwiseIterator>> iters_;
 
   // Statistics (keyed by projection column index) accumulated so far by any
   // fully-consumed sub-iterators.
@@ -177,7 +177,7 @@ class UnionIterator : public RowwiseIterator {
 // batch, then other columns may avoid doing any IO.
 class MaterializingIterator : public RowwiseIterator {
  public:
-  explicit MaterializingIterator(std::shared_ptr<ColumnwiseIterator> iter);
+  explicit MaterializingIterator(std::unique_ptr<ColumnwiseIterator> iter);
 
   // Initialize the iterator, performing predicate pushdown as described above.
   Status Init(ScanSpec *spec) OVERRIDE;
@@ -202,7 +202,7 @@ class MaterializingIterator : public RowwiseIterator {
 
   Status MaterializeBlock(RowBlock *dst);
 
-  std::shared_ptr<ColumnwiseIterator> iter_;
+  std::unique_ptr<ColumnwiseIterator> iter_;
 
   // List of (column index, predicate) in order of most to least selective, with
   // ties broken by the index.
@@ -228,7 +228,7 @@ class PredicateEvaluatingIterator : public RowwiseIterator {
   //
   // POSTCONDITION: spec->predicates().empty()
   // POSTCONDITION: base_iter and its wrapper are initialized
-  static Status InitAndMaybeWrap(std::shared_ptr<RowwiseIterator> *base_iter,
+  static Status InitAndMaybeWrap(std::unique_ptr<RowwiseIterator> *base_iter,
                                  ScanSpec *spec);
 
   // Initialize the iterator.
@@ -254,12 +254,12 @@ class PredicateEvaluatingIterator : public RowwiseIterator {
   // Construct the evaluating iterator.
   // This is only called from ::InitAndMaybeWrap()
   // REQUIRES: base_iter is already Init()ed.
-  explicit PredicateEvaluatingIterator(std::shared_ptr<RowwiseIterator> base_iter);
+  explicit PredicateEvaluatingIterator(std::unique_ptr<RowwiseIterator> base_iter);
 
   FRIEND_TEST(TestPredicateEvaluatingIterator, TestPredicateEvaluation);
   FRIEND_TEST(TestPredicateEvaluatingIterator, TestPredicateEvaluationOrder);
 
-  std::shared_ptr<RowwiseIterator> base_iter_;
+  std::unique_ptr<RowwiseIterator> base_iter_;
 
   // List of predicates in order of most to least selective, with
   // ties broken by the column index.
diff --git a/src/kudu/integration-tests/linked_list-test-util.h b/src/kudu/integration-tests/linked_list-test-util.h
index f4ecb59..1fb71fa 100644
--- a/src/kudu/integration-tests/linked_list-test-util.h
+++ b/src/kudu/integration-tests/linked_list-test-util.h
@@ -693,10 +693,10 @@ Status LinkedListTester::VerifyLinkedListLocal(const tablet::Tablet* tablet,
   const Schema* tablet_schema = tablet->schema();
   // Cannot use schemas with col indexes in a scan (assertions fire).
   Schema projection(tablet_schema->columns(), tablet_schema->num_key_columns());
-  gscoped_ptr<RowwiseIterator> iter;
+  std::unique_ptr<RowwiseIterator> iter;
   RETURN_NOT_OK_PREPEND(tablet->NewRowIterator(projection, &iter),
                         "Cannot create new row iterator");
-  RETURN_NOT_OK_PREPEND(iter->Init(NULL), "Cannot initialize row iterator");
+  RETURN_NOT_OK_PREPEND(iter->Init(nullptr), "Cannot initialize row iterator");
 
   Arena arena(1024);
   RowBlock block(projection, 100, &arena);
diff --git a/src/kudu/master/sys_catalog.cc b/src/kudu/master/sys_catalog.cc
index 4f9630a..c9bc5ad 100644
--- a/src/kudu/master/sys_catalog.cc
+++ b/src/kudu/master/sys_catalog.cc
@@ -622,7 +622,7 @@ Status SysCatalogTable::ProcessRows(
   ScanSpec spec;
   spec.AddPredicate(pred);
 
-  gscoped_ptr<RowwiseIterator> iter;
+  unique_ptr<RowwiseIterator> iter;
   RETURN_NOT_OK(tablet_replica_->tablet()->NewRowIterator(schema_, &iter));
   RETURN_NOT_OK(iter->Init(&spec));
 
diff --git a/src/kudu/tablet/all_types-scan-correctness-test.cc b/src/kudu/tablet/all_types-scan-correctness-test.cc
index cdf9e3d..19f3232 100644
--- a/src/kudu/tablet/all_types-scan-correctness-test.cc
+++ b/src/kudu/tablet/all_types-scan-correctness-test.cc
@@ -19,6 +19,7 @@
 #include <cinttypes>
 #include <cstddef>
 #include <cstdint>
+#include <memory>
 #include <ostream>
 #include <string>
 #include <unordered_map>
@@ -34,7 +35,6 @@
 #include "kudu/common/scan_spec.h"
 #include "kudu/common/schema.h"
 #include "kudu/common/types.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/stringprintf.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/tablet/local_tablet_writer.h"
@@ -46,6 +46,7 @@
 #include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
 
+using std::unique_ptr;
 using strings::Substitute;
 
 namespace kudu {
@@ -305,7 +306,7 @@ public:
     AutoReleasePool pool;
     *count = 0;
     spec.OptimizeScan(schema, &arena, &pool, true);
-    gscoped_ptr<RowwiseIterator> iter;
+    unique_ptr<RowwiseIterator> iter;
     ASSERT_OK(tablet()->NewRowIterator(schema, &iter));
     ASSERT_OK(iter->Init(&spec));
     ASSERT_TRUE(spec.predicates().empty()) << "Should have accepted all predicate.";
diff --git a/src/kudu/tablet/cfile_set-test.cc b/src/kudu/tablet/cfile_set-test.cc
index fc99c66..4d28a86 100644
--- a/src/kudu/tablet/cfile_set-test.cc
+++ b/src/kudu/tablet/cfile_set-test.cc
@@ -43,7 +43,6 @@
 #include "kudu/common/rowid.h"
 #include "kudu/common/scan_spec.h"
 #include "kudu/common/schema.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/integral_types.h"
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/stringprintf.h"
@@ -52,6 +51,7 @@
 #include "kudu/tablet/tablet-test-util.h"
 #include "kudu/util/auto_release_pool.h"
 #include "kudu/util/bloom_filter.h"
+#include "kudu/util/hash.pb.h"
 #include "kudu/util/mem_tracker.h"
 #include "kudu/util/memory/arena.h"
 #include "kudu/util/slice.h"
@@ -62,6 +62,7 @@ DECLARE_int32(cfile_default_block_size);
 
 using std::shared_ptr;
 using std::string;
+using std::unique_ptr;
 using std::vector;
 
 namespace kudu {
@@ -179,8 +180,8 @@ class TestCFileSet : public KuduRowSetTest {
                        int32_t lower,
                        int32_t upper) {
     // Create iterator.
-    shared_ptr<CFileSet::Iterator> cfile_iter(fileset->NewIterator(&schema_, nullptr));
-    gscoped_ptr<RowwiseIterator> iter(new MaterializingIterator(cfile_iter));
+    unique_ptr<CFileSet::Iterator> cfile_iter(fileset->NewIterator(&schema_, nullptr));
+    unique_ptr<RowwiseIterator> iter(new MaterializingIterator(std::move(cfile_iter)));
 
     // Create a scan with a range predicate on the key column.
     ScanSpec spec;
@@ -215,8 +216,8 @@ class TestCFileSet : public KuduRowSetTest {
                              vector<size_t> target) {
     LOG(INFO) << "predicates size: " << predicates.size();
     // Create iterator.
-    shared_ptr<CFileSet::Iterator> cfile_iter(fileset->NewIterator(&schema_, nullptr));
-    gscoped_ptr<RowwiseIterator> iter(new MaterializingIterator(cfile_iter));
+    unique_ptr<CFileSet::Iterator> cfile_iter(fileset->NewIterator(&schema_, nullptr));
+    unique_ptr<RowwiseIterator> iter(new MaterializingIterator(std::move(cfile_iter)));
     LOG(INFO) << "Target size: " << target.size();
     // Create a scan with a range predicate on the key column.
     ScanSpec spec;
@@ -279,7 +280,7 @@ TEST_F(TestCFileSet, TestPartiallyMaterialize) {
   shared_ptr<CFileSet> fileset;
   ASSERT_OK(CFileSet::Open(rowset_meta_, MemTracker::GetRootTracker(), nullptr, &fileset));
 
-  gscoped_ptr<CFileSet::Iterator> iter(fileset->NewIterator(&schema_, nullptr));
+  unique_ptr<CFileSet::Iterator> iter(fileset->NewIterator(&schema_, nullptr));
   ASSERT_OK(iter->Init(nullptr));
 
   Arena arena(4096);
@@ -361,8 +362,8 @@ TEST_F(TestCFileSet, TestIteratePartialSchema) {
 
   Schema new_schema;
   ASSERT_OK(schema_.CreateProjectionByNames({ "c0", "c2" }, &new_schema));
-  shared_ptr<CFileSet::Iterator> cfile_iter(fileset->NewIterator(&new_schema, nullptr));
-  gscoped_ptr<RowwiseIterator> iter(new MaterializingIterator(cfile_iter));
+  unique_ptr<CFileSet::Iterator> cfile_iter(fileset->NewIterator(&new_schema, nullptr));
+  unique_ptr<RowwiseIterator> iter(new MaterializingIterator(std::move(cfile_iter)));
 
   ASSERT_OK(iter->Init(nullptr));
 
@@ -393,8 +394,9 @@ TEST_F(TestCFileSet, TestRangeScan) {
   ASSERT_OK(CFileSet::Open(rowset_meta_, MemTracker::GetRootTracker(), nullptr, &fileset));
 
   // Create iterator.
-  shared_ptr<CFileSet::Iterator> cfile_iter(fileset->NewIterator(&schema_, nullptr));
-  gscoped_ptr<RowwiseIterator> iter(new MaterializingIterator(cfile_iter));
+  unique_ptr<CFileSet::Iterator> cfile_iter(fileset->NewIterator(&schema_, nullptr));
+  CFileSet::Iterator* cfile_iter_raw = cfile_iter.get();
+  unique_ptr<RowwiseIterator> iter(new MaterializingIterator(std::move(cfile_iter)));
   Schema key_schema = schema_.CreateKeyProjection();
   Arena arena(1024);
   AutoReleasePool pool;
@@ -411,8 +413,8 @@ TEST_F(TestCFileSet, TestRangeScan) {
   // Check that the bounds got pushed as index bounds.
   // Since the key column is the rowidx * 2, we need to divide the integer bounds
   // back down.
-  EXPECT_EQ(lower / 2, cfile_iter->lower_bound_idx_);
-  EXPECT_EQ(upper / 2, cfile_iter->upper_bound_idx_);
+  EXPECT_EQ(lower / 2, cfile_iter_raw->lower_bound_idx_);
+  EXPECT_EQ(upper / 2, cfile_iter_raw->upper_bound_idx_);
 
   // Read all the results.
   vector<string> results;
diff --git a/src/kudu/tablet/cfile_set.cc b/src/kudu/tablet/cfile_set.cc
index 5f07581..6ddac50 100644
--- a/src/kudu/tablet/cfile_set.cc
+++ b/src/kudu/tablet/cfile_set.cc
@@ -45,7 +45,6 @@
 #include "kudu/fs/block_manager.h"
 #include "kudu/fs/fs_manager.h"
 #include "kudu/gutil/dynamic_annotations.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/port.h"
@@ -221,15 +220,19 @@ CFileReader* CFileSet::key_index_reader() const {
   return FindOrDie(readers_by_col_id_, key_col_id).get();
 }
 
-Status CFileSet::NewColumnIterator(ColumnId col_id, CFileReader::CacheControl cache_blocks,
-                                   const fs::IOContext* io_context, CFileIterator **iter) const {
+Status CFileSet::NewColumnIterator(ColumnId col_id,
+                                   CFileReader::CacheControl cache_blocks,
+                                   const fs::IOContext* io_context,
+                                   unique_ptr<CFileIterator>* iter) const {
   return FindOrDie(readers_by_col_id_, col_id)->NewIterator(iter, cache_blocks,
                                                             io_context);
 }
 
-CFileSet::Iterator* CFileSet::NewIterator(const Schema* projection,
-                                          const IOContext* io_context) const {
-  return new CFileSet::Iterator(shared_from_this(), projection, io_context);
+unique_ptr<CFileSet::Iterator> CFileSet::NewIterator(
+    const Schema* projection,
+    const IOContext* io_context) const {
+  return unique_ptr<CFileSet::Iterator>(
+      new CFileSet::Iterator(shared_from_this(), projection, io_context));
 }
 
 Status CFileSet::CountRows(const IOContext* io_context, rowid_t *count) const {
@@ -297,11 +300,9 @@ Status CFileSet::FindRow(const RowSetKeyProbe &probe,
   }
 
   stats->keys_consulted++;
-  CFileIterator *key_iter = nullptr;
+  unique_ptr<CFileIterator> key_iter;
   RETURN_NOT_OK(NewKeyIterator(io_context, &key_iter));
 
-  unique_ptr<CFileIterator> key_iter_scoped(key_iter); // free on return
-
   bool exact;
   Status s = key_iter->SeekAtOrAfter(probe.encoded_key(), &exact);
   if (s.IsNotFound() || (s.ok() && !exact)) {
@@ -325,7 +326,8 @@ Status CFileSet::CheckRowPresent(const RowSetKeyProbe& probe, const IOContext* i
   return Status::OK();
 }
 
-Status CFileSet::NewKeyIterator(const IOContext* io_context, CFileIterator** key_iter) const {
+Status CFileSet::NewKeyIterator(const IOContext* io_context,
+                                unique_ptr<CFileIterator>* key_iter) const {
   RETURN_NOT_OK(key_index_reader()->Init(io_context));
   return key_index_reader()->NewIterator(key_iter, CFileReader::CACHE_BLOCK, io_context);
 }
@@ -364,11 +366,11 @@ Status CFileSet::Iterator::CreateColumnIterators(const ScanSpec* spec) {
                                                             col_schema.read_default_value()));
       continue;
     }
-    CFileIterator *iter;
+    unique_ptr<CFileIterator> iter;
     RETURN_NOT_OK_PREPEND(base_data_->NewColumnIterator(col_id, cache_blocks, io_context_, &iter),
                           Substitute("could not create iterator for column $0",
                                      projection_->column(proj_col_idx).ToString()));
-    ret_iters.emplace_back(iter);
+    ret_iters.emplace_back(std::move(iter));
   }
 
   col_iters_.swap(ret_iters);
@@ -380,10 +382,8 @@ Status CFileSet::Iterator::Init(ScanSpec *spec) {
 
   RETURN_NOT_OK(base_data_->CountRows(io_context_, &row_count_));
 
-  // Setup Key Iterator
-  CFileIterator *tmp;
-  RETURN_NOT_OK(base_data_->NewKeyIterator(io_context_, &tmp));
-  key_iter_.reset(tmp);
+  // Setup key iterator.
+  RETURN_NOT_OK(base_data_->NewKeyIterator(io_context_, &key_iter_));
 
   // Setup column iterators.
   RETURN_NOT_OK(CreateColumnIterators(spec));
diff --git a/src/kudu/tablet/cfile_set.h b/src/kudu/tablet/cfile_set.h
index 54a854b..a66697b 100644
--- a/src/kudu/tablet/cfile_set.h
+++ b/src/kudu/tablet/cfile_set.h
@@ -33,7 +33,6 @@
 #include "kudu/common/iterator.h"
 #include "kudu/common/rowid.h"
 #include "kudu/common/schema.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/port.h"
@@ -81,14 +80,14 @@ class CFileSet : public std::enable_shared_from_this<CFileSet> {
 
   // Create an iterator with the given projection. 'projection' must remain valid
   // for the lifetime of the returned iterator.
-  virtual Iterator* NewIterator(const Schema* projection,
-                                const fs::IOContext* io_context) const;
+  std::unique_ptr<Iterator> NewIterator(const Schema* projection,
+                                        const fs::IOContext* io_context) const;
 
   Status CountRows(const fs::IOContext* io_context, rowid_t *count) const;
 
   // See RowSet::GetBounds
-  virtual Status GetBounds(std::string* min_encoded_key,
-                           std::string* max_encoded_key) const;
+  Status GetBounds(std::string* min_encoded_key,
+                   std::string* max_encoded_key) const;
 
   // The on-disk size, in bytes, of this cfile set's ad hoc index.
   // Returns 0 if there is no ad hoc index.
@@ -143,8 +142,9 @@ class CFileSet : public std::enable_shared_from_this<CFileSet> {
   Status NewColumnIterator(ColumnId col_id,
                            cfile::CFileReader::CacheControl cache_blocks,
                            const fs::IOContext* io_context,
-                           cfile::CFileIterator **iter) const;
-  Status NewKeyIterator(const fs::IOContext* io_context, cfile::CFileIterator** key_iter) const;
+                           std::unique_ptr<cfile::CFileIterator>* iter) const;
+  Status NewKeyIterator(const fs::IOContext* io_context,
+                        std::unique_ptr<cfile::CFileIterator>* key_iter) const;
 
   // Return the CFileReader responsible for reading the key index.
   // (the ad-hoc reader for composite keys, otherwise the key column reader)
@@ -246,7 +246,7 @@ class CFileSet::Iterator : public ColumnwiseIterator {
   const Schema* projection_;
 
   // Iterator for the key column in the underlying data.
-  gscoped_ptr<cfile::CFileIterator> key_iter_;
+  std::unique_ptr<cfile::CFileIterator> key_iter_;
   std::vector<std::unique_ptr<cfile::ColumnIterator>> col_iters_;
 
   bool initted_;
diff --git a/src/kudu/tablet/compaction.cc b/src/kudu/tablet/compaction.cc
index e409d05..7912d08 100644
--- a/src/kudu/tablet/compaction.cc
+++ b/src/kudu/tablet/compaction.cc
@@ -23,7 +23,6 @@
 #include <memory>
 #include <ostream>
 #include <string>
-#include <type_traits>
 #include <unordered_set>
 #include <vector>
 
@@ -181,7 +180,7 @@ class MemRowSetCompactionInput : public CompactionInput {
   DISALLOW_COPY_AND_ASSIGN(MemRowSetCompactionInput);
   gscoped_ptr<RowBlock> row_block_;
 
-  gscoped_ptr<MemRowSet::Iterator> iter_;
+  unique_ptr<MemRowSet::Iterator> iter_;
 
   // Arena used to store the projected undo/redo mutations of the current block.
   Arena arena_;
@@ -196,7 +195,7 @@ class MemRowSetCompactionInput : public CompactionInput {
 // CompactionInput yielding rows and mutations from an on-disk DiskRowSet.
 class DiskRowSetCompactionInput : public CompactionInput {
  public:
-  DiskRowSetCompactionInput(gscoped_ptr<RowwiseIterator> base_iter,
+  DiskRowSetCompactionInput(unique_ptr<RowwiseIterator> base_iter,
                             unique_ptr<DeltaIterator> redo_delta_iter,
                             unique_ptr<DeltaIterator> undo_delta_iter)
       : base_iter_(std::move(base_iter)),
@@ -260,7 +259,7 @@ class DiskRowSetCompactionInput : public CompactionInput {
 
  private:
   DISALLOW_COPY_AND_ASSIGN(DiskRowSetCompactionInput);
-  gscoped_ptr<RowwiseIterator> base_iter_;
+  unique_ptr<RowwiseIterator> base_iter_;
   unique_ptr<DeltaIterator> redo_delta_iter_;
   unique_ptr<DeltaIterator> undo_delta_iter_;
 
@@ -853,8 +852,8 @@ Status CompactionInput::Create(const DiskRowSet &rowset,
                                gscoped_ptr<CompactionInput>* out) {
   CHECK(projection->has_column_ids());
 
-  shared_ptr<ColumnwiseIterator> base_cwise(rowset.base_data_->NewIterator(projection, io_context));
-  gscoped_ptr<RowwiseIterator> base_iter(new MaterializingIterator(base_cwise));
+  unique_ptr<ColumnwiseIterator> base_cwise(rowset.base_data_->NewIterator(projection, io_context));
+  unique_ptr<RowwiseIterator> base_iter(new MaterializingIterator(std::move(base_cwise)));
 
   // Creates a DeltaIteratorMerger that will only include the relevant REDO deltas.
   RowIteratorOptions redo_opts;
diff --git a/src/kudu/tablet/composite-pushdown-test.cc b/src/kudu/tablet/composite-pushdown-test.cc
index 80c1d53..4698d8a 100644
--- a/src/kudu/tablet/composite-pushdown-test.cc
+++ b/src/kudu/tablet/composite-pushdown-test.cc
@@ -32,7 +32,6 @@
 #include "kudu/common/partial_row.h"
 #include "kudu/common/scan_spec.h"
 #include "kudu/common/schema.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/stringprintf.h"
 #include "kudu/tablet/local_tablet_writer.h"
@@ -45,6 +44,7 @@
 #include "kudu/util/test_macros.h"
 
 using std::string;
+using std::unique_ptr;
 using std::vector;
 
 namespace kudu {
@@ -115,7 +115,7 @@ class CompositePushdownTest : public KuduTabletTest {
   void ScanTablet(ScanSpec *spec, vector<string> *results, const char *descr) {
     SCOPED_TRACE(descr);
 
-    gscoped_ptr<RowwiseIterator> iter;
+    unique_ptr<RowwiseIterator> iter;
     ASSERT_OK(tablet()->NewRowIterator(client_schema_, &iter));
     ASSERT_OK(iter->Init(spec));
     ASSERT_TRUE(spec->predicates().empty()) << "Should have accepted all predicates";
diff --git a/src/kudu/tablet/delta_compaction-test.cc b/src/kudu/tablet/delta_compaction-test.cc
index 21be212..165526c 100644
--- a/src/kudu/tablet/delta_compaction-test.cc
+++ b/src/kudu/tablet/delta_compaction-test.cc
@@ -209,13 +209,12 @@ TEST_F(TestDeltaCompaction, TestMergeMultipleSchemas) {
 
   shared_ptr<DeltaFileReader> dfr;
   ASSERT_OK(GetDeltaFileReader(block_id, &dfr));
-  DeltaIterator* raw_iter;
-  ASSERT_OK(dfr->NewDeltaIterator(opts, &raw_iter));
-  gscoped_ptr<DeltaIterator> scoped_iter(raw_iter);
+  unique_ptr<DeltaIterator> iter;
+  ASSERT_OK(dfr->NewDeltaIterator(opts, &iter));
 
   vector<string> results;
-  ASSERT_OK(DebugDumpDeltaIterator(REDO, scoped_iter.get(), merge_schema,
-                                          ITERATE_OVER_ALL_ROWS, &results));
+  ASSERT_OK(DebugDumpDeltaIterator(REDO, iter.get(), merge_schema,
+                                   ITERATE_OVER_ALL_ROWS, &results));
   for (const string &str : results) {
     VLOG(1) << str;
   }
diff --git a/src/kudu/tablet/delta_compaction.cc b/src/kudu/tablet/delta_compaction.cc
index f597152..4bb54a9 100644
--- a/src/kudu/tablet/delta_compaction.cc
+++ b/src/kudu/tablet/delta_compaction.cc
@@ -117,9 +117,10 @@ string MajorDeltaCompaction::ColumnNamesToString() const {
 Status MajorDeltaCompaction::FlushRowSetAndDeltas(const IOContext* io_context) {
   CHECK_EQ(state_, kInitialized);
 
-  shared_ptr<ColumnwiseIterator> old_base_data_cwise(base_data_->NewIterator(&partial_schema_,
+  unique_ptr<ColumnwiseIterator> old_base_data_cwise(base_data_->NewIterator(&partial_schema_,
                                                                              io_context));
-  gscoped_ptr<RowwiseIterator> old_base_data_rwise(new MaterializingIterator(old_base_data_cwise));
+  unique_ptr<RowwiseIterator> old_base_data_rwise(new MaterializingIterator(
+      std::move(old_base_data_cwise)));
 
   ScanSpec spec;
   spec.set_cache_blocks(false);
diff --git a/src/kudu/tablet/delta_iterator_merger.cc b/src/kudu/tablet/delta_iterator_merger.cc
index 58d14da..8bdf648 100644
--- a/src/kudu/tablet/delta_iterator_merger.cc
+++ b/src/kudu/tablet/delta_iterator_merger.cc
@@ -159,15 +159,15 @@ Status DeltaIteratorMerger::Create(
   vector<unique_ptr<DeltaIterator> > delta_iters;
 
   for (const shared_ptr<DeltaStore> &store : stores) {
-    DeltaIterator* raw_iter;
-    Status s = store->NewDeltaIterator(opts, &raw_iter);
+    unique_ptr<DeltaIterator> iter;
+    Status s = store->NewDeltaIterator(opts, &iter);
     if (s.IsNotFound()) {
       continue;
     }
     RETURN_NOT_OK_PREPEND(s, Substitute("Could not create iterator for store $0",
                                         store->ToString()));
 
-    delta_iters.push_back(unique_ptr<DeltaIterator>(raw_iter));
+    delta_iters.emplace_back(std::move(iter));
   }
 
   if (delta_iters.size() == 1) {
diff --git a/src/kudu/tablet/delta_store.h b/src/kudu/tablet/delta_store.h
index 269f353..63d0657 100644
--- a/src/kudu/tablet/delta_store.h
+++ b/src/kudu/tablet/delta_store.h
@@ -77,7 +77,7 @@ class DeltaStore {
   // returns Status::NotFound if the mutations within this delta store
   // cannot include the snapshot.
   virtual Status NewDeltaIterator(const RowIteratorOptions& opts,
-                                  DeltaIterator** iterator) const = 0;
+                                  std::unique_ptr<DeltaIterator>* iterator) const = 0;
 
   // Set *deleted to true if the latest update for the given row is a deletion.
   virtual Status CheckRowDeleted(rowid_t row_idx, const fs::IOContext* io_context,
diff --git a/src/kudu/tablet/delta_tracker.cc b/src/kudu/tablet/delta_tracker.cc
index 8f41a3e..dd3c788 100644
--- a/src/kudu/tablet/delta_tracker.cc
+++ b/src/kudu/tablet/delta_tracker.cc
@@ -623,7 +623,7 @@ Status DeltaTracker::NewDeltaFileIterator(
 
 Status DeltaTracker::WrapIterator(const shared_ptr<CFileSet::Iterator> &base,
                                   const RowIteratorOptions& opts,
-                                  gscoped_ptr<ColumnwiseIterator>* out) const {
+                                  unique_ptr<ColumnwiseIterator>* out) const {
   unique_ptr<DeltaIterator> iter;
   RETURN_NOT_OK(NewDeltaIterator(opts, &iter));
 
diff --git a/src/kudu/tablet/delta_tracker.h b/src/kudu/tablet/delta_tracker.h
index edac431..36aa59e 100644
--- a/src/kudu/tablet/delta_tracker.h
+++ b/src/kudu/tablet/delta_tracker.h
@@ -91,7 +91,7 @@ class DeltaTracker {
 
   Status WrapIterator(const std::shared_ptr<CFileSet::Iterator> &base,
                       const RowIteratorOptions& opts,
-                      gscoped_ptr<ColumnwiseIterator>* out) const;
+                      std::unique_ptr<ColumnwiseIterator>* out) const;
 
   // Enum used for NewDeltaIterator() and CollectStores() below.
   // Determines whether all types of stores should be considered,
diff --git a/src/kudu/tablet/deltafile-test.cc b/src/kudu/tablet/deltafile-test.cc
index 5307579..65aefa7 100644
--- a/src/kudu/tablet/deltafile-test.cc
+++ b/src/kudu/tablet/deltafile-test.cc
@@ -43,7 +43,6 @@
 #include "kudu/fs/block_manager.h"
 #include "kudu/fs/fs-test-util.h"
 #include "kudu/fs/fs_manager.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/tablet/delta_key.h"
@@ -146,7 +145,7 @@ class TestDeltaFile : public KuduTest {
     return DeltaFileReader::Open(std::move(block), REDO, ReaderOptions(), out);
   }
 
-  Status OpenDeltaFileIterator(const BlockId& block_id, gscoped_ptr<DeltaIterator>* out) {
+  Status OpenDeltaFileIterator(const BlockId& block_id, unique_ptr<DeltaIterator>* out) {
     shared_ptr<DeltaFileReader> reader;
     RETURN_NOT_OK(OpenDeltaFileReader(block_id, &reader));
     return OpenDeltaFileIteratorFromReader(REDO, reader, out);
@@ -154,16 +153,13 @@ class TestDeltaFile : public KuduTest {
 
   Status OpenDeltaFileIteratorFromReader(DeltaType type,
                                          const shared_ptr<DeltaFileReader>& reader,
-                                         gscoped_ptr<DeltaIterator>* out) {
+                                         unique_ptr<DeltaIterator>* out) {
     RowIteratorOptions opts;
     opts.snap_to_include = type == REDO ?
                 MvccSnapshot::CreateSnapshotIncludingAllTransactions() :
                 MvccSnapshot::CreateSnapshotIncludingNoTransactions();
     opts.projection = &schema_;
-    DeltaIterator* raw_iter;
-    RETURN_NOT_OK(reader->NewDeltaIterator(opts, &raw_iter));
-    out->reset(raw_iter);
-    return Status::OK();
+    return reader->NewDeltaIterator(opts, out);
   }
 
   void VerifyTestFile() {
@@ -172,7 +168,7 @@ class TestDeltaFile : public KuduTest {
     ASSERT_EQ(((FLAGS_last_row_to_update - FLAGS_first_row_to_update) / 2) + 1,
               reader->delta_stats().update_count_for_col_id(schema_.column_id(0)));
     ASSERT_EQ(0, reader->delta_stats().delete_count());
-    gscoped_ptr<DeltaIterator> it;
+    unique_ptr<DeltaIterator> it;
     Status s = OpenDeltaFileIteratorFromReader(REDO, reader, &it);
     if (s.IsNotFound()) {
       FAIL() << "Iterator fell outside of the range of an include-all snapshot";
@@ -221,7 +217,7 @@ class TestDeltaFile : public KuduTest {
   }
 
  protected:
-  gscoped_ptr<FsManager> fs_manager_;
+  unique_ptr<FsManager> fs_manager_;
   Schema schema_;
   Arena arena_;
   BlockId test_block_;
@@ -230,7 +226,7 @@ class TestDeltaFile : public KuduTest {
 TEST_F(TestDeltaFile, TestDumpDeltaFileIterator) {
   WriteTestFile();
 
-  gscoped_ptr<DeltaIterator> it;
+  unique_ptr<DeltaIterator> it;
   Status s = OpenDeltaFileIterator(test_block_, &it);
   if (s.IsNotFound()) {
     FAIL() << "Iterator fell outside of the range of an include-all snapshot";
@@ -251,7 +247,7 @@ TEST_F(TestDeltaFile, TestDumpDeltaFileIterator) {
 
 TEST_F(TestDeltaFile, TestWriteDeltaFileIteratorToFile) {
   WriteTestFile();
-  gscoped_ptr<DeltaIterator> it;
+  unique_ptr<DeltaIterator> it;
   Status s = OpenDeltaFileIterator(test_block_, &it);
   if (s.IsNotFound()) {
     FAIL() << "Iterator fell outside of the range of an include-all snapshot";
@@ -303,7 +299,7 @@ TEST_F(TestDeltaFile, TestCollectMutations) {
   WriteTestFile();
 
   {
-    gscoped_ptr<DeltaIterator> it;
+    unique_ptr<DeltaIterator> it;
     Status s = OpenDeltaFileIterator(test_block_, &it);
     if (s.IsNotFound()) {
       FAIL() << "Iterator fell outside of the range of an include-all snapshot";
@@ -344,32 +340,26 @@ TEST_F(TestDeltaFile, TestSkipsDeltasOutOfRange) {
   shared_ptr<DeltaFileReader> reader;
   ASSERT_OK(OpenDeltaFileReader(test_block_, &reader));
 
-  gscoped_ptr<DeltaIterator> iter;
-
   RowIteratorOptions opts;
   opts.projection = &schema_;
 
   // should skip
   opts.snap_to_include = MvccSnapshot(Timestamp(9));
   ASSERT_FALSE(opts.snap_to_include.MayHaveCommittedTransactionsAtOrAfter(Timestamp(10)));
-  DeltaIterator* raw_iter = nullptr;
-  Status s = reader->NewDeltaIterator(opts, &raw_iter);
+  unique_ptr<DeltaIterator> iter;
+  Status s = reader->NewDeltaIterator(opts, &iter);
   ASSERT_TRUE(s.IsNotFound());
-  ASSERT_TRUE(raw_iter == nullptr);
+  ASSERT_EQ(nullptr, iter);
 
   // should include
-  raw_iter = nullptr;
   opts.snap_to_include = MvccSnapshot(Timestamp(15));
-  ASSERT_OK(reader->NewDeltaIterator(opts, &raw_iter));
-  ASSERT_TRUE(raw_iter != nullptr);
-  iter.reset(raw_iter);
+  ASSERT_OK(reader->NewDeltaIterator(opts, &iter));
+  ASSERT_NE(nullptr, iter);
 
   // should include
-  raw_iter = nullptr;
   opts.snap_to_include = MvccSnapshot(Timestamp(21));
-  ASSERT_OK(reader->NewDeltaIterator(opts, &raw_iter));
-  ASSERT_TRUE(raw_iter != nullptr);
-  iter.reset(raw_iter);
+  ASSERT_OK(reader->NewDeltaIterator(opts, &iter));
+  ASSERT_NE(nullptr, iter);
 }
 
 TEST_F(TestDeltaFile, TestLazyInit) {
@@ -507,14 +497,13 @@ TYPED_TEST(DeltaTypeTestDeltaFile, BenchmarkPrepareAndApply) {
     RowIteratorOptions opts;
     opts.projection = &projection;
     opts.snap_to_include = MvccSnapshot(ts);
-    DeltaIterator* raw_iter;
-    Status s = reader->NewDeltaIterator(opts, &raw_iter);
+    unique_ptr<DeltaIterator> iter;
+    Status s = reader->NewDeltaIterator(opts, &iter);
     if (s.IsNotFound()) {
       ASSERT_STR_CONTAINS(s.ToString(), "MvccSnapshot outside the range of this delta");
       continue;
     }
     ASSERT_OK(s);
-    unique_ptr<DeltaIterator> iter(raw_iter);
     ASSERT_OK(iter->Init(nullptr));
 
     // Scan from the iterator as if we were a DeltaApplier.
diff --git a/src/kudu/tablet/deltafile.cc b/src/kudu/tablet/deltafile.cc
index 30bb01b..9d5bc01 100644
--- a/src/kudu/tablet/deltafile.cc
+++ b/src/kudu/tablet/deltafile.cc
@@ -323,7 +323,7 @@ Status DeltaFileReader::CloneForDebugging(FsManager* fs_manager,
 }
 
 Status DeltaFileReader::NewDeltaIterator(const RowIteratorOptions& opts,
-                                         DeltaIterator** iterator) const {
+                                         unique_ptr<DeltaIterator>* iterator) const {
   if (IsRelevantForSnapshots(opts.snap_to_exclude, opts.snap_to_include)) {
     if (VLOG_IS_ON(2)) {
       if (!init_once_.init_succeeded()) {
@@ -348,9 +348,9 @@ Status DeltaFileReader::NewDeltaIterator(const RowIteratorOptions& opts,
     // during its first seek.
     auto s_this = const_cast<DeltaFileReader*>(this)->shared_from_this();
     if (delta_type_ == REDO) {
-      *iterator = new DeltaFileIterator<REDO>(std::move(s_this), opts);
+      iterator->reset(new DeltaFileIterator<REDO>(std::move(s_this), opts));
     } else {
-      *iterator = new DeltaFileIterator<UNDO>(std::move(s_this), opts);
+      iterator->reset(new DeltaFileIterator<UNDO>(std::move(s_this), opts));
     }
     return Status::OK();
   }
@@ -377,16 +377,13 @@ Status DeltaFileReader::CheckRowDeleted(rowid_t row_idx, const IOContext* io_con
   RowIteratorOptions opts;
   opts.projection = &empty_schema;
   opts.io_context = io_context;
-  DeltaIterator* raw_iter;
-  Status s = NewDeltaIterator(opts, &raw_iter);
+  unique_ptr<DeltaIterator> iter;
+  Status s = NewDeltaIterator(opts, &iter);
   if (s.IsNotFound()) {
     *deleted = false;
     return Status::OK();
   }
   RETURN_NOT_OK(s);
-
-  gscoped_ptr<DeltaIterator> iter(raw_iter);
-
   ScanSpec spec;
   RETURN_NOT_OK(iter->Init(&spec));
   RETURN_NOT_OK(iter->SeekToOrdinal(row_idx));
diff --git a/src/kudu/tablet/deltafile.h b/src/kudu/tablet/deltafile.h
index e330ce5..a93d5a3 100644
--- a/src/kudu/tablet/deltafile.h
+++ b/src/kudu/tablet/deltafile.h
@@ -159,7 +159,7 @@ class DeltaFileReader : public DeltaStore,
 
   // See DeltaStore::NewDeltaIterator(...)
   Status NewDeltaIterator(const RowIteratorOptions& opts,
-                          DeltaIterator** iterator) const OVERRIDE;
+                          std::unique_ptr<DeltaIterator>* iterator) const OVERRIDE;
 
   // See DeltaStore::CheckRowDeleted
   virtual Status CheckRowDeleted(rowid_t row_idx,
diff --git a/src/kudu/tablet/deltamemstore-test.cc b/src/kudu/tablet/deltamemstore-test.cc
index 89ee619..ba7adfe 100644
--- a/src/kudu/tablet/deltamemstore-test.cc
+++ b/src/kudu/tablet/deltamemstore-test.cc
@@ -138,13 +138,12 @@ class TestDeltaMemStore : public KuduTest {
     RowIteratorOptions opts;
     opts.projection = &single_col_projection;
     opts.snap_to_include = snapshot;
-    DeltaIterator* raw_iter;
-    Status s = dms_->NewDeltaIterator(opts, &raw_iter);
+    unique_ptr<DeltaIterator> iter;
+    Status s = dms_->NewDeltaIterator(opts, &iter);
     if (s.IsNotFound()) {
       return;
     }
     ASSERT_OK(s);
-    gscoped_ptr<DeltaIterator> iter(raw_iter);
     ASSERT_OK(iter->Init(nullptr));
     ASSERT_OK(iter->SeekToOrdinal(row_idx));
     ASSERT_OK(iter->PrepareBatch(cb->nrows(), DeltaIterator::PREPARE_FOR_APPLY));
@@ -549,14 +548,13 @@ TEST_F(TestDeltaMemStore, TestIteratorDoesUpdates) {
   opts.projection = &schema_;
   // TODO(todd): test snapshot reads from different points
   opts.snap_to_include = MvccSnapshot(mvcc_);
-  DeltaIterator* raw_iter;
-  Status s = dms_->NewDeltaIterator(opts, &raw_iter);
+
+  unique_ptr<DeltaIterator> iter;
+  Status s = dms_->NewDeltaIterator(opts, &iter);
   if (s.IsNotFound()) {
     FAIL() << "Iterator fell outside of the range of the snapshot";
   }
   ASSERT_OK(s);
-
-  unique_ptr<DeltaIterator> iter(raw_iter);
   ASSERT_OK(iter->Init(nullptr));
 
   int block_start_row = 50;
@@ -599,14 +597,12 @@ TEST_F(TestDeltaMemStore, TestCollectMutations) {
   RowIteratorOptions opts;
   opts.projection = &schema_;
   opts.snap_to_include = MvccSnapshot(mvcc_);
-  DeltaIterator* raw_iter;
-  Status s =  dms_->NewDeltaIterator(opts, &raw_iter);
+  unique_ptr<DeltaIterator> iter;
+  Status s =  dms_->NewDeltaIterator(opts, &iter);
   if (s.IsNotFound()) {
     FAIL() << "Iterator fell outside of the range of the snapshot";
   }
   ASSERT_OK(s);
-
-  unique_ptr<DeltaIterator> iter(raw_iter);
   ASSERT_OK(iter->Init(nullptr));
   ASSERT_OK(iter->SeekToOrdinal(0));
   ASSERT_OK(iter->PrepareBatch(kBatchSize, DeltaIterator::PREPARE_FOR_COLLECT));
diff --git a/src/kudu/tablet/deltamemstore.cc b/src/kudu/tablet/deltamemstore.cc
index c2a8f0d..6f65810 100644
--- a/src/kudu/tablet/deltamemstore.cc
+++ b/src/kudu/tablet/deltamemstore.cc
@@ -17,6 +17,7 @@
 
 #include "kudu/tablet/deltamemstore.h"
 
+#include <memory>
 #include <ostream>
 #include <utility>
 
@@ -44,6 +45,7 @@ using fs::IOContext;
 using log::LogAnchorRegistry;
 using std::string;
 using std::shared_ptr;
+using std::unique_ptr;
 using std::vector;
 using strings::Substitute;
 
@@ -142,8 +144,8 @@ Status DeltaMemStore::FlushToFile(DeltaFileWriter *dfw,
 }
 
 Status DeltaMemStore::NewDeltaIterator(const RowIteratorOptions& opts,
-                                       DeltaIterator** iterator) const {
-  *iterator = new DMSIterator(shared_from_this(), opts);
+                                       unique_ptr<DeltaIterator>* iterator) const {
+  iterator->reset(new DMSIterator(shared_from_this(), opts));
   return Status::OK();
 }
 
diff --git a/src/kudu/tablet/deltamemstore.h b/src/kudu/tablet/deltamemstore.h
index 3cc02a0..b741d85 100644
--- a/src/kudu/tablet/deltamemstore.h
+++ b/src/kudu/tablet/deltamemstore.h
@@ -119,7 +119,7 @@ class DeltaMemStore : public DeltaStore,
   // returns Status::NotFound if the mutations within this delta store
   // cannot include the snapshot.
   virtual Status NewDeltaIterator(const RowIteratorOptions& opts,
-                                  DeltaIterator** iterator) const OVERRIDE;
+                                  std::unique_ptr<DeltaIterator>* iterator) const OVERRIDE;
 
   virtual Status CheckRowDeleted(rowid_t row_idx, const fs::IOContext* io_context,
                                  bool* deleted) const OVERRIDE;
diff --git a/src/kudu/tablet/diff_scan-test.cc b/src/kudu/tablet/diff_scan-test.cc
index 01bf63f..bd4ae26 100644
--- a/src/kudu/tablet/diff_scan-test.cc
+++ b/src/kudu/tablet/diff_scan-test.cc
@@ -27,7 +27,6 @@
 #include "kudu/common/iterator.h"
 #include "kudu/common/scan_spec.h"
 #include "kudu/common/schema.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/tablet/local_tablet_writer.h"
 #include "kudu/tablet/mvcc.h"
 #include "kudu/tablet/rowset.h"
@@ -38,6 +37,7 @@
 #include "kudu/util/test_macros.h"
 
 using std::string;
+using std::unique_ptr;
 using std::vector;
 
 namespace kudu {
@@ -91,7 +91,7 @@ TEST_F(DiffScanTest, TestDiffScan) {
   auto projection = tablet->schema()->CopyWithoutColumnIds();
   opts.projection = &projection;
 
-  gscoped_ptr<RowwiseIterator> row_iterator;
+  unique_ptr<RowwiseIterator> row_iterator;
   ASSERT_OK(tablet->NewRowIterator(std::move(opts),
                                    &row_iterator));
   ASSERT_TRUE(row_iterator);
diff --git a/src/kudu/tablet/diskrowset-test-base.h b/src/kudu/tablet/diskrowset-test-base.h
index ec3f4b3..d26ee03 100644
--- a/src/kudu/tablet/diskrowset-test-base.h
+++ b/src/kudu/tablet/diskrowset-test-base.h
@@ -214,7 +214,7 @@ class TestRowSet : public KuduRowSetTest {
     Schema proj_val = CreateProjection(schema_, { "val" });
     RowIteratorOptions opts;
     opts.projection = &proj_val;
-    gscoped_ptr<RowwiseIterator> row_iter;
+    std::unique_ptr<RowwiseIterator> row_iter;
     CHECK_OK(rs.NewRowIterator(opts, &row_iter));
     CHECK_OK(row_iter->Init(nullptr));
     Arena arena(1024);
@@ -262,7 +262,7 @@ class TestRowSet : public KuduRowSetTest {
 
     RowIteratorOptions opts;
     opts.projection = &schema_;
-    gscoped_ptr<RowwiseIterator> row_iter;
+    std::unique_ptr<RowwiseIterator> row_iter;
     CHECK_OK(rs.NewRowIterator(opts, &row_iter));
     CHECK_OK(row_iter->Init(&spec));
     std::vector<std::string> rows;
@@ -277,7 +277,7 @@ class TestRowSet : public KuduRowSetTest {
                                 int expected_rows, bool do_log = true) {
     RowIteratorOptions opts;
     opts.projection = &schema;
-    gscoped_ptr<RowwiseIterator> row_iter;
+    std::unique_ptr<RowwiseIterator> row_iter;
     CHECK_OK(rs.NewRowIterator(opts, &row_iter));
     CHECK_OK(row_iter->Init(nullptr));
 
diff --git a/src/kudu/tablet/diskrowset-test.cc b/src/kudu/tablet/diskrowset-test.cc
index b10c277..01549cd 100644
--- a/src/kudu/tablet/diskrowset-test.cc
+++ b/src/kudu/tablet/diskrowset-test.cc
@@ -438,7 +438,7 @@ TEST_F(TestRowSet, TestFlushedUpdatesRespectMVCC) {
     RowIteratorOptions opts;
     opts.projection = &schema_;
     opts.snap_to_include = snaps[i];
-    gscoped_ptr<RowwiseIterator> iter;
+    unique_ptr<RowwiseIterator> iter;
     ASSERT_OK(rs->NewRowIterator(opts, &iter));
     string data = InitAndDumpIterator(iter.get());
     EXPECT_EQ(StringPrintf(R"((string key="row", uint32 val=%d))", i + 1), data);
@@ -453,7 +453,7 @@ TEST_F(TestRowSet, TestFlushedUpdatesRespectMVCC) {
     RowIteratorOptions opts;
     opts.projection = &schema_;
     opts.snap_to_include = snaps[i];
-    gscoped_ptr<RowwiseIterator> iter;
+    unique_ptr<RowwiseIterator> iter;
     ASSERT_OK(rs->NewRowIterator(opts, &iter));
     string data = InitAndDumpIterator(iter.get());
     EXPECT_EQ(StringPrintf(R"((string key="row", uint32 val=%d))", i + 1), data);
@@ -785,7 +785,7 @@ TEST_P(DiffScanRowSetTest, TestFuzz) {
     opts.projection = &projection;
     opts.snap_to_exclude = MvccSnapshot(ts1);
     opts.snap_to_include = MvccSnapshot(ts2);
-    gscoped_ptr<RowwiseIterator> iter;
+    unique_ptr<RowwiseIterator> iter;
     ASSERT_OK(rs->NewRowIterator(opts, &iter));
     ASSERT_OK(iter->Init(nullptr));
 
diff --git a/src/kudu/tablet/diskrowset.cc b/src/kudu/tablet/diskrowset.cc
index 066385d..6b6e84c 100644
--- a/src/kudu/tablet/diskrowset.cc
+++ b/src/kudu/tablet/diskrowset.cc
@@ -656,17 +656,16 @@ Status DiskRowSet::NewMajorDeltaCompaction(const vector<ColumnId>& col_ids,
 }
 
 Status DiskRowSet::NewRowIterator(const RowIteratorOptions& opts,
-                                  gscoped_ptr<RowwiseIterator>* out) const {
+                                  unique_ptr<RowwiseIterator>* out) const {
   DCHECK(open_);
   shared_lock<rw_spinlock> l(component_lock_);
 
   shared_ptr<CFileSet::Iterator> base_iter(base_data_->NewIterator(opts.projection,
                                                                    opts.io_context));
-  gscoped_ptr<ColumnwiseIterator> col_iter;
+  unique_ptr<ColumnwiseIterator> col_iter;
   RETURN_NOT_OK(delta_tracker_->WrapIterator(base_iter, opts, &col_iter));
 
-  out->reset(new MaterializingIterator(
-      shared_ptr<ColumnwiseIterator>(col_iter.release())));
+  out->reset(new MaterializingIterator(std::move(col_iter)));
   return Status::OK();
 }
 
diff --git a/src/kudu/tablet/diskrowset.h b/src/kudu/tablet/diskrowset.h
index e83f81c..65ed8b3 100644
--- a/src/kudu/tablet/diskrowset.h
+++ b/src/kudu/tablet/diskrowset.h
@@ -356,7 +356,7 @@ class DiskRowSet : public RowSet {
   // Read functions.
   ////////////////////
   virtual Status NewRowIterator(const RowIteratorOptions& opts,
-                                gscoped_ptr<RowwiseIterator>* out) const override;
+                                std::unique_ptr<RowwiseIterator>* out) const override;
 
   virtual Status NewCompactionInput(const Schema* projection,
                                     const MvccSnapshot &snap,
diff --git a/src/kudu/tablet/major_delta_compaction-test.cc b/src/kudu/tablet/major_delta_compaction-test.cc
index 69f5c07..24af96f 100644
--- a/src/kudu/tablet/major_delta_compaction-test.cc
+++ b/src/kudu/tablet/major_delta_compaction-test.cc
@@ -20,7 +20,6 @@
 #include <memory>
 #include <ostream>
 #include <string>
-#include <unordered_set>
 #include <vector>
 
 #include <gflags/gflags_declare.h>
@@ -32,7 +31,6 @@
 #include "kudu/common/partial_row.h"
 #include "kudu/common/schema.h"
 #include "kudu/fs/io_context.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/stringprintf.h"
 #include "kudu/gutil/strings/substitute.h"
@@ -48,7 +46,7 @@ DECLARE_double(cfile_inject_corruption);
 
 using std::shared_ptr;
 using std::string;
-using std::unordered_set;
+using std::unique_ptr;
 using std::vector;
 
 namespace kudu {
@@ -160,7 +158,7 @@ class TestMajorDeltaCompaction : public KuduRowSetTest {
       RowIteratorOptions opts;
       opts.projection = &client_schema_;
       opts.snap_to_include = snap;
-      gscoped_ptr<RowwiseIterator> row_iter;
+      unique_ptr<RowwiseIterator> row_iter;
       ASSERT_OK(tablet()->NewRowIterator(std::move(opts), &row_iter));
       ASSERT_OK(row_iter->Init(nullptr));
 
diff --git a/src/kudu/tablet/memrowset-test.cc b/src/kudu/tablet/memrowset-test.cc
index 1823f14..4cd3b25 100644
--- a/src/kudu/tablet/memrowset-test.cc
+++ b/src/kudu/tablet/memrowset-test.cc
@@ -40,7 +40,6 @@
 #include "kudu/consensus/log_anchor_registry.h"
 #include "kudu/consensus/opid.pb.h"
 #include "kudu/consensus/opid_util.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/stringprintf.h"
 #include "kudu/gutil/strings/strcat.h"
@@ -96,9 +95,9 @@ class TestMemRowSet : public KuduTest {
 
  protected:
   // Check that the given row in the memrowset contains the given data.
-  void CheckValue(const shared_ptr<MemRowSet> &mrs, string key,
-                  const string &expected_row) {
-    gscoped_ptr<MemRowSet::Iterator> iter(mrs->NewIterator());
+  void CheckValue(const shared_ptr<MemRowSet>& mrs, const string& key,
+                  const string& expected_row) {
+    unique_ptr<MemRowSet::Iterator> iter(mrs->NewIterator());
     ASSERT_OK(iter->Init(nullptr));
 
     Slice keystr_slice(key);
@@ -200,7 +199,7 @@ class TestMemRowSet : public KuduTest {
   }
 
   int ScanAndCount(MemRowSet* mrs, const RowIteratorOptions& opts) {
-    gscoped_ptr<MemRowSet::Iterator> iter(mrs->NewIterator(opts));
+    unique_ptr<MemRowSet::Iterator> iter(mrs->NewIterator(opts));
     CHECK_OK(iter->Init(nullptr));
 
     Arena arena(1024);
@@ -268,7 +267,7 @@ TEST_F(TestMemRowSet, TestInsertAndIterate) {
 
   ASSERT_EQ(2, mrs->entry_count());
 
-  gscoped_ptr<MemRowSet::Iterator> iter(mrs->NewIterator());
+  unique_ptr<MemRowSet::Iterator> iter(mrs->NewIterator());
   ASSERT_OK(iter->Init(nullptr));
 
   // The first row returned from the iterator should
@@ -337,7 +336,7 @@ TEST_F(TestMemRowSet, TestInsertAndIterateCompoundKey) {
 
   ASSERT_EQ(3, mrs->entry_count());
 
-  gscoped_ptr<MemRowSet::Iterator> iter(mrs->NewIterator());
+  unique_ptr<MemRowSet::Iterator> iter(mrs->NewIterator());
   ASSERT_OK(iter->Init(nullptr));
 
   // The first row returned from the iterator should
diff --git a/src/kudu/tablet/memrowset.cc b/src/kudu/tablet/memrowset.cc
index 2045d18..4bb7294 100644
--- a/src/kudu/tablet/memrowset.cc
+++ b/src/kudu/tablet/memrowset.cc
@@ -53,6 +53,7 @@ TAG_FLAG(mrs_use_codegen, hidden);
 
 using std::shared_ptr;
 using std::string;
+using std::unique_ptr;
 using std::vector;
 
 namespace kudu { namespace tablet {
@@ -132,8 +133,8 @@ MemRowSet::~MemRowSet() {
 }
 
 Status MemRowSet::DebugDump(vector<string> *lines) {
-  gscoped_ptr<Iterator> iter(NewIterator());
-  RETURN_NOT_OK(iter->Init(NULL));
+  unique_ptr<Iterator> iter(NewIterator());
+  RETURN_NOT_OK(iter->Init(nullptr));
   while (iter->HasNext()) {
     MRSRow row = iter->GetCurrentRow();
     LOG_STRING(INFO, lines)
@@ -300,7 +301,7 @@ MemRowSet::Iterator *MemRowSet::NewIterator() const {
 }
 
 Status MemRowSet::NewRowIterator(const RowIteratorOptions& opts,
-                                 gscoped_ptr<RowwiseIterator>* out) const {
+                                 unique_ptr<RowwiseIterator>* out) const {
   out->reset(NewIterator(opts));
   return Status::OK();
 }
diff --git a/src/kudu/tablet/memrowset.h b/src/kudu/tablet/memrowset.h
index 5803a7e..c32f1c5 100644
--- a/src/kudu/tablet/memrowset.h
+++ b/src/kudu/tablet/memrowset.h
@@ -324,7 +324,7 @@ class MemRowSet : public RowSet,
 
   // Alias to conform to DiskRowSet interface
   virtual Status NewRowIterator(const RowIteratorOptions& opts,
-                                gscoped_ptr<RowwiseIterator>* out) const override;
+                                std::unique_ptr<RowwiseIterator>* out) const override;
 
   // Create compaction input.
   virtual Status NewCompactionInput(const Schema* projection,
diff --git a/src/kudu/tablet/mock-rowsets.h b/src/kudu/tablet/mock-rowsets.h
index 49d9d2e..01576d5 100644
--- a/src/kudu/tablet/mock-rowsets.h
+++ b/src/kudu/tablet/mock-rowsets.h
@@ -55,7 +55,7 @@ class MockRowSet : public RowSet {
     return Status::OK();
   }
   virtual Status NewRowIterator(const RowIteratorOptions& /*opts*/,
-                                gscoped_ptr<RowwiseIterator>* /*out*/) const OVERRIDE {
+                                std::unique_ptr<RowwiseIterator>* /*out*/) const OVERRIDE {
     LOG(FATAL) << "Unimplemented";
     return Status::OK();
   }
diff --git a/src/kudu/tablet/mt-rowset_delta_compaction-test.cc b/src/kudu/tablet/mt-rowset_delta_compaction-test.cc
index a9ff5f9..a1501db 100644
--- a/src/kudu/tablet/mt-rowset_delta_compaction-test.cc
+++ b/src/kudu/tablet/mt-rowset_delta_compaction-test.cc
@@ -29,12 +29,12 @@
 #include "kudu/common/rowblock.h"
 #include "kudu/common/schema.h"
 #include "kudu/gutil/atomicops.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/tablet/diskrowset-test-base.h"
 #include "kudu/tablet/diskrowset.h"
 #include "kudu/tablet/rowset.h"
+#include "kudu/tablet/tablet-test-util.h"
 #include "kudu/tablet/tablet.pb.h"
 #include "kudu/util/memory/arena.h"
 #include "kudu/util/monotime.h"
@@ -56,6 +56,7 @@ DEFINE_int32(num_seconds_per_thread, kDefaultNumSecondsPerThread,
              "Minimum number of seconds each thread should work");
 
 using std::shared_ptr;
+using std::unique_ptr;
 using std::vector;
 
 namespace kudu {
@@ -111,7 +112,7 @@ class TestMultiThreadedRowSetDeltaCompaction : public TestRowSet {
     RowBlock dst(schema_, 1000, &arena);
     RowIteratorOptions opts;
     opts.projection = &schema_;
-    gscoped_ptr<RowwiseIterator> iter;
+    unique_ptr<RowwiseIterator> iter;
     ASSERT_OK(rs->NewRowIterator(opts, &iter));
     uint32_t expected = NoBarrier_Load(&update_counter_);
     ASSERT_OK(iter->Init(nullptr));
diff --git a/src/kudu/tablet/mt-tablet-test.cc b/src/kudu/tablet/mt-tablet-test.cc
index 446d1c2..f5a15e9 100644
--- a/src/kudu/tablet/mt-tablet-test.cc
+++ b/src/kudu/tablet/mt-tablet-test.cc
@@ -35,13 +35,13 @@
 #include "kudu/common/rowblock.h"
 #include "kudu/common/rowid.h"
 #include "kudu/common/schema.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/tablet/local_tablet_writer.h"
 #include "kudu/tablet/rowset.h"
 #include "kudu/tablet/tablet-harness.h"
 #include "kudu/tablet/tablet-test-base.h"
+#include "kudu/tablet/tablet-test-util.h"
 #include "kudu/tablet/tablet.h"
 #include "kudu/util/countdown_latch.h"
 #include "kudu/util/faststring.h"
@@ -77,6 +77,7 @@ DEFINE_double(flusher_backoff, 2.0f, "Ratio to backoff the flusher thread");
 DEFINE_int32(flusher_initial_frequency_ms, 30, "Number of ms to wait between flushes");
 
 using std::shared_ptr;
+using std::unique_ptr;
 
 namespace kudu {
 namespace tablet {
@@ -95,7 +96,7 @@ class MultiThreadedTabletTest : public TabletTestBase<SETUP> {
     superclass::SetUp();
 
     // Warm up code cache with all the projections we'll be using.
-    gscoped_ptr<RowwiseIterator> iter;
+    unique_ptr<RowwiseIterator> iter;
     CHECK_OK(tablet()->NewRowIterator(client_schema_, &iter));
     uint64_t count;
     CHECK_OK(tablet()->CountRows(&count));
@@ -152,9 +153,9 @@ class MultiThreadedTabletTest : public TabletTestBase<SETUP> {
     KuduPartialRow row(&client_schema_);
 
     while (running_insert_count_.count() > 0) {
-      gscoped_ptr<RowwiseIterator> iter;
+      unique_ptr<RowwiseIterator> iter;
       CHECK_OK(tablet()->NewRowIterator(client_schema_, &iter));
-      CHECK_OK(iter->Init(NULL));
+      CHECK_OK(iter->Init(nullptr));
 
       while (iter->HasNext() && running_insert_count_.count() > 0) {
         tmp_arena.Reset();
@@ -220,9 +221,9 @@ class MultiThreadedTabletTest : public TabletTestBase<SETUP> {
     int max_iters = FLAGS_num_insert_threads * max_rows / 10;
 
     while (running_insert_count_.count() > 0) {
-      gscoped_ptr<RowwiseIterator> iter;
+      unique_ptr<RowwiseIterator> iter;
       CHECK_OK(tablet()->NewRowIterator(client_schema_, &iter));
-      CHECK_OK(iter->Init(NULL));
+      CHECK_OK(iter->Init(nullptr));
 
       for (int i = 0; i < max_iters && iter->HasNext(); i++) {
         CHECK_OK(iter->NextBlock(&block));
@@ -254,9 +255,9 @@ class MultiThreadedTabletTest : public TabletTestBase<SETUP> {
 
     int64_t sum = 0;
 
-    gscoped_ptr<RowwiseIterator> iter;
+    unique_ptr<RowwiseIterator> iter;
     CHECK_OK(tablet()->NewRowIterator(valcol_projection_, &iter));
-    CHECK_OK(iter->Init(NULL));
+    CHECK_OK(iter->Init(nullptr));
 
     while (iter->HasNext()) {
       arena.Reset();
diff --git a/src/kudu/tablet/rowset.cc b/src/kudu/tablet/rowset.cc
index 3f54456..2e5f4d1 100644
--- a/src/kudu/tablet/rowset.cc
+++ b/src/kudu/tablet/rowset.cc
@@ -32,6 +32,7 @@
 
 using std::shared_ptr;
 using std::string;
+using std::unique_ptr;
 using std::vector;
 using strings::Substitute;
 
@@ -84,20 +85,20 @@ string DuplicatingRowSet::ToString() const {
 }
 
 Status DuplicatingRowSet::NewRowIterator(const RowIteratorOptions& opts,
-                                         gscoped_ptr<RowwiseIterator>* out) const {
+                                         unique_ptr<RowwiseIterator>* out) const {
   // Use the original rowset.
   if (old_rowsets_.size() == 1) {
     return old_rowsets_[0]->NewRowIterator(opts, out);
   }
   // Union or merge between them
 
-  vector<shared_ptr<RowwiseIterator> > iters;
+  vector<unique_ptr<RowwiseIterator>> iters;
   for (const shared_ptr<RowSet> &rowset : old_rowsets_) {
-    gscoped_ptr<RowwiseIterator> iter;
+    unique_ptr<RowwiseIterator> iter;
     RETURN_NOT_OK_PREPEND(rowset->NewRowIterator(opts, &iter),
                           Substitute("Could not create iterator for rowset $0",
                                      rowset->ToString()));
-    iters.push_back(shared_ptr<RowwiseIterator>(iter.release()));
+    iters.emplace_back(std::move(iter));
   }
 
   switch (opts.order) {
diff --git a/src/kudu/tablet/rowset.h b/src/kudu/tablet/rowset.h
index e2a1ae6..ed99122 100644
--- a/src/kudu/tablet/rowset.h
+++ b/src/kudu/tablet/rowset.h
@@ -143,7 +143,7 @@ class RowSet {
   //
   // The returned iterator is not Initted.
   virtual Status NewRowIterator(const RowIteratorOptions& opts,
-                                gscoped_ptr<RowwiseIterator>* out) const = 0;
+                                std::unique_ptr<RowwiseIterator>* out) const = 0;
 
   // Create the input to be used for a compaction.
   //
@@ -391,7 +391,7 @@ class DuplicatingRowSet : public RowSet {
                          bool *present, ProbeStats* stats) const OVERRIDE;
 
   virtual Status NewRowIterator(const RowIteratorOptions& opts,
-                                gscoped_ptr<RowwiseIterator>* out) const OVERRIDE;
+                                std::unique_ptr<RowwiseIterator>* out) const OVERRIDE;
 
   virtual Status NewCompactionInput(const Schema* projection,
                                     const MvccSnapshot &snap,
diff --git a/src/kudu/tablet/tablet-decoder-eval-test.cc b/src/kudu/tablet/tablet-decoder-eval-test.cc
index ed1a482..eb4ff7c 100644
--- a/src/kudu/tablet/tablet-decoder-eval-test.cc
+++ b/src/kudu/tablet/tablet-decoder-eval-test.cc
@@ -36,7 +36,6 @@
 #include "kudu/common/rowblock.h"
 #include "kudu/common/scan_spec.h"
 #include "kudu/common/schema.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/stringprintf.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/tablet/local_tablet_writer.h"
@@ -56,11 +55,12 @@ DEFINE_int32(decoder_eval_test_lower, 0, "Lower bound on the predicate [lower, u
 DEFINE_int32(decoder_eval_test_upper, 50, "Upper bound on the predicate [lower, upper)");
 DEFINE_int32(decoder_eval_test_strlen, 10, "Number of strings per cell");
 
+using std::unique_ptr;
+using strings::Substitute;
+
 namespace kudu {
 namespace tablet {
 
-using strings::Substitute;
-
 enum Setup {
 #ifdef ADDRESS_SANITIZER
   EMPTY = 0, SMALL = 100, MEDIUM = 3000, LARGE = 10000
@@ -173,7 +173,7 @@ public:
     spec.AddPredicate(string_pred);
     spec.OptimizeScan(schema_, &arena, &pool, true);
     ScanSpec orig_spec = spec;
-    gscoped_ptr<RowwiseIterator> iter;
+    unique_ptr<RowwiseIterator> iter;
     ASSERT_OK(tablet()->NewRowIterator(client_schema_, &iter));
     spec = orig_spec;
     ASSERT_OK(iter->Init(&spec));
@@ -251,7 +251,7 @@ public:
     spec.AddPredicate(string_pred_b);
     spec.OptimizeScan(schema_, &arena, &pool, true);
     ScanSpec orig_spec = spec;
-    gscoped_ptr<RowwiseIterator> iter;
+    unique_ptr<RowwiseIterator> iter;
     ASSERT_OK(tablet()->NewRowIterator(client_schema_, &iter));
     spec = orig_spec;
     ASSERT_OK(iter->Init(&spec));
diff --git a/src/kudu/tablet/tablet-pushdown-test.cc b/src/kudu/tablet/tablet-pushdown-test.cc
index 55fb5ef..c9bee93 100644
--- a/src/kudu/tablet/tablet-pushdown-test.cc
+++ b/src/kudu/tablet/tablet-pushdown-test.cc
@@ -34,7 +34,6 @@
 #include "kudu/common/row.h"
 #include "kudu/common/scan_spec.h"
 #include "kudu/common/schema.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/stringprintf.h"
 #include "kudu/tablet/local_tablet_writer.h"
 #include "kudu/tablet/tablet-test-util.h"
@@ -47,6 +46,7 @@
 #include "kudu/util/test_util.h"
 
 using std::string;
+using std::unique_ptr;
 using std::vector;
 
 namespace kudu {
@@ -107,7 +107,7 @@ class TabletPushdownTest : public KuduTabletTest,
     AutoReleasePool pool;
     spec.OptimizeScan(schema_, &arena, &pool, true);
 
-    gscoped_ptr<RowwiseIterator> iter;
+    unique_ptr<RowwiseIterator> iter;
     ASSERT_OK(tablet()->NewRowIterator(client_schema_, &iter));
     ASSERT_OK(iter->Init(&spec));
     ASSERT_TRUE(spec.predicates().empty()) << "Should have accepted all predicates";
@@ -175,7 +175,7 @@ class TabletPushdownTest : public KuduTabletTest,
     spec.OptimizeScan(schema_, &arena, &pool, true);
 
     Schema empty_schema(std::vector<ColumnSchema>(), 0);
-    gscoped_ptr<RowwiseIterator> iter;
+    unique_ptr<RowwiseIterator> iter;
     ASSERT_OK(tablet()->NewRowIterator(empty_schema, &iter));
     ASSERT_OK(iter->Init(&spec));
     ASSERT_TRUE(spec.predicates().empty()) << "Should have accepted all predicates";
@@ -266,7 +266,7 @@ TEST_F(TabletSparsePushdownTest, Kudu2231) {
   int32_t value = 50;
   spec.AddPredicate(ColumnPredicate::Equality(schema_.column(1), &value));
 
-  gscoped_ptr<RowwiseIterator> iter;
+  unique_ptr<RowwiseIterator> iter;
   ASSERT_OK(tablet()->NewRowIterator(client_schema_, &iter));
   ASSERT_OK(iter->Init(&spec));
   ASSERT_TRUE(spec.predicates().empty()) << "Should have accepted all predicates";
diff --git a/src/kudu/tablet/tablet-schema-test.cc b/src/kudu/tablet/tablet-schema-test.cc
index 4b428e7..94dfa36 100644
--- a/src/kudu/tablet/tablet-schema-test.cc
+++ b/src/kudu/tablet/tablet-schema-test.cc
@@ -29,7 +29,6 @@
 #include "kudu/common/iterator.h"
 #include "kudu/common/partial_row.h"
 #include "kudu/common/schema.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/tablet/local_tablet_writer.h"
 #include "kudu/tablet/tablet-test-util.h"
@@ -41,6 +40,7 @@
 
 using std::pair;
 using std::string;
+using std::unique_ptr;
 using std::vector;
 using strings::Substitute;
 
@@ -125,7 +125,7 @@ TEST_F(TestTabletSchema, TestRead) {
 
   InsertRows(client_schema_, 0, kNumRows);
 
-  gscoped_ptr<RowwiseIterator> iter;
+  unique_ptr<RowwiseIterator> iter;
   ASSERT_OK(tablet()->NewRowIterator(projection, &iter));
 
   Status s = iter->Init(nullptr);
diff --git a/src/kudu/tablet/tablet-test-base.h b/src/kudu/tablet/tablet-test-base.h
index d6977f6..ef98ef1 100644
--- a/src/kudu/tablet/tablet-test-base.h
+++ b/src/kudu/tablet/tablet-test-base.h
@@ -19,6 +19,7 @@
 
 #include <algorithm>
 #include <limits>
+#include <memory>
 #include <string>
 #include <unordered_set>
 #include <vector>
@@ -411,7 +412,7 @@ class TabletTestBase : public KuduTabletTest {
 
   void VerifyTestRowsWithVerifier(int32_t first_row, uint64_t expected_count,
                                   const boost::optional<TestRowVerifier>& verifier) {
-    gscoped_ptr<RowwiseIterator> iter;
+    std::unique_ptr<RowwiseIterator> iter;
     ASSERT_OK(tablet()->NewRowIterator(client_schema_, &iter));
     VerifyTestRowsWithRowIteratorAndVerifier(first_row, expected_count, std::move(iter), verifier);
   }
@@ -422,15 +423,15 @@ class TabletTestBase : public KuduTabletTest {
     RowIteratorOptions opts;
     opts.projection = &client_schema_;
     opts.snap_to_include = MvccSnapshot(timestamp);
-    gscoped_ptr<RowwiseIterator> iter;
+    std::unique_ptr<RowwiseIterator> iter;
     ASSERT_OK(tablet()->NewRowIterator(std::move(opts), &iter));
     VerifyTestRowsWithRowIteratorAndVerifier(first_row, expected_count, std::move(iter), verifier);
   }
 
   void VerifyTestRowsWithRowIteratorAndVerifier(int32_t first_row, uint64_t expected_row_count,
-                                                gscoped_ptr<RowwiseIterator> iter,
+                                                std::unique_ptr<RowwiseIterator> iter,
                                                 const boost::optional<TestRowVerifier>& verifier) {
-    ASSERT_OK(iter->Init(NULL));
+    ASSERT_OK(iter->Init(nullptr));
     int batch_size = std::max<size_t>(1, std::min<size_t>(expected_row_count / 10,
                                                           4L * 1024 * 1024 / schema_.byte_size()));
     Arena arena(32*1024);
@@ -491,9 +492,9 @@ class TabletTestBase : public KuduTabletTest {
   // into the given vector. This is only useful in tests which insert
   // a very small number of rows.
   Status IterateToStringList(std::vector<std::string> *out) {
-    gscoped_ptr<RowwiseIterator> iter;
+    std::unique_ptr<RowwiseIterator> iter;
     RETURN_NOT_OK(this->tablet()->NewRowIterator(this->client_schema_, &iter));
-    RETURN_NOT_OK(iter->Init(NULL));
+    RETURN_NOT_OK(iter->Init(nullptr));
     return kudu::tablet::IterateToStringList(iter.get(), out);
   }
 
diff --git a/src/kudu/tablet/tablet-test-util.h b/src/kudu/tablet/tablet-test-util.h
index 22651af..d44d2bd 100644
--- a/src/kudu/tablet/tablet-test-util.h
+++ b/src/kudu/tablet/tablet-test-util.h
@@ -244,7 +244,7 @@ static inline void CollectRowsForSnapshots(
     RowIteratorOptions opts;
     opts.projection = &schema;
     opts.snap_to_include = snapshot;
-    gscoped_ptr<RowwiseIterator> iter;
+    std::unique_ptr<RowwiseIterator> iter;
     ASSERT_OK(tablet->NewRowIterator(std::move(opts), &iter));
     ASSERT_OK(iter->Init(nullptr));
     auto collector = new std::vector<std::string>();
@@ -271,7 +271,7 @@ static inline void VerifySnapshotsHaveSameResult(
     RowIteratorOptions opts;
     opts.projection = &schema;
     opts.snap_to_include = snapshot;
-    gscoped_ptr<RowwiseIterator> iter;
+    std::unique_ptr<RowwiseIterator> iter;
     ASSERT_OK(tablet->NewRowIterator(std::move(opts), &iter));
     ASSERT_OK(iter->Init(nullptr));
     std::vector<std::string> collector;
@@ -293,7 +293,7 @@ static inline void VerifySnapshotsHaveSameResult(
 static inline Status DumpRowSet(const RowSet& rs,
                                 const RowIteratorOptions& opts,
                                 std::vector<std::string>* out) {
-  gscoped_ptr<RowwiseIterator> iter;
+  std::unique_ptr<RowwiseIterator> iter;
   RETURN_NOT_OK(rs.NewRowIterator(opts, &iter));
   RETURN_NOT_OK(iter->Init(nullptr));
   RETURN_NOT_OK(IterateToStringList(iter.get(), out));
@@ -314,7 +314,7 @@ static inline std::string InitAndDumpIterator(RowwiseIterator* iter) {
 static inline Status DumpTablet(const Tablet& tablet,
                                 const Schema& projection,
                                 std::vector<std::string>* out) {
-  gscoped_ptr<RowwiseIterator> iter;
+  std::unique_ptr<RowwiseIterator> iter;
   RETURN_NOT_OK(tablet.NewRowIterator(projection, &iter));
   RETURN_NOT_OK(iter->Init(nullptr));
   std::vector<std::string> rows;
@@ -914,15 +914,14 @@ void RunDeltaFuzzTest(const DeltaStore& store,
     SCOPED_TRACE(Substitute("Timestamps: [$0,$1)",
                                      lower_ts ? lower_ts->ToString() : "INF",
                                      upper_ts.ToString()));
-    DeltaIterator* raw_iter;
-    Status s = store.NewDeltaIterator(opts, &raw_iter);
+    std::unique_ptr<DeltaIterator> iter;
+    Status s = store.NewDeltaIterator(opts, &iter);
     if (s.IsNotFound()) {
       ASSERT_STR_CONTAINS(s.ToString(), "MvccSnapshot outside the range of this delta");
       ASSERT_TRUE(mirror->CheckAllDeltasCulled(lower_ts, upper_ts));
       continue;
     }
     ASSERT_OK(s);
-    std::unique_ptr<DeltaIterator> iter(raw_iter);
     ASSERT_OK(iter->Init(nullptr));
 
     // Run tests in batches, in case there's some bug related to batching.
diff --git a/src/kudu/tablet/tablet-test.cc b/src/kudu/tablet/tablet-test.cc
index c9da697..457a51a 100644
--- a/src/kudu/tablet/tablet-test.cc
+++ b/src/kudu/tablet/tablet-test.cc
@@ -485,7 +485,7 @@ TYPED_TEST(TestTablet, TestRowIteratorSimple) {
   CHECK_OK(this->InsertTestRow(&writer, kInMemRowSet, 0));
 
   // Now iterate the tablet and make sure the rows show up
-  gscoped_ptr<RowwiseIterator> iter;
+  unique_ptr<RowwiseIterator> iter;
   ASSERT_OK(this->tablet()->NewRowIterator(this->client_schema_, &iter));
   ASSERT_OK(iter->Init(nullptr));
 
@@ -557,7 +557,7 @@ TYPED_TEST(TestTablet, TestRowIteratorOrdered) {
       opts.projection = &this->client_schema_;
       opts.snap_to_include = snap;
       opts.order = ORDERED;
-      gscoped_ptr<RowwiseIterator> iter;
+      unique_ptr<RowwiseIterator> iter;
       ASSERT_OK(this->tablet()->NewRowIterator(std::move(opts), &iter));
       ASSERT_OK(iter->Init(nullptr));
 
@@ -658,7 +658,7 @@ TYPED_TEST(TestTablet, TestRowIteratorComplex) {
   }
 
   // Now iterate the tablet and make sure the rows show up.
-  gscoped_ptr<RowwiseIterator> iter;
+  unique_ptr<RowwiseIterator> iter;
   const Schema& schema = this->client_schema_;
   ASSERT_OK(this->tablet()->NewRowIterator(schema, &iter));
   ASSERT_OK(iter->Init(nullptr));
diff --git a/src/kudu/tablet/tablet.cc b/src/kudu/tablet/tablet.cc
index a6343d6..42804cf 100644
--- a/src/kudu/tablet/tablet.cc
+++ b/src/kudu/tablet/tablet.cc
@@ -177,6 +177,7 @@ using std::ostream;
 using std::pair;
 using std::shared_ptr;
 using std::string;
+using std::unique_ptr;
 using std::unordered_set;
 using std::vector;
 using strings::Substitute;
@@ -401,7 +402,7 @@ void Tablet::SplitKeyRange(const EncodedKey* start_key,
 }
 
 Status Tablet::NewRowIterator(const Schema& projection,
-                              gscoped_ptr<RowwiseIterator>* iter) const {
+                              unique_ptr<RowwiseIterator>* iter) const {
   RowIteratorOptions opts;
   // Yield current rows.
   opts.snap_to_include = MvccSnapshot(mvcc_);
@@ -410,7 +411,7 @@ Status Tablet::NewRowIterator(const Schema& projection,
 }
 
 Status Tablet::NewRowIterator(RowIteratorOptions opts,
-                              gscoped_ptr<RowwiseIterator>* iter) const {
+                              unique_ptr<RowwiseIterator>* iter) const {
   RETURN_IF_STOPPED_OR_CHECK_STATE(kOpen);
   if (metrics_) {
     metrics_->scans_started->Increment();
@@ -1830,18 +1831,18 @@ Status Tablet::DebugDump(vector<string> *lines) {
 Status Tablet::CaptureConsistentIterators(
     const RowIteratorOptions& opts,
     const ScanSpec* spec,
-    vector<shared_ptr<RowwiseIterator>>* iters) const {
+    vector<unique_ptr<RowwiseIterator>>* iters) const {
 
   shared_lock<rw_spinlock> l(component_lock_);
   RETURN_IF_STOPPED_OR_CHECK_STATE(kOpen);
 
   // Construct all the iterators locally first, so that if we fail
   // in the middle, we don't modify the output arguments.
-  vector<shared_ptr<RowwiseIterator>> ret;
+  vector<unique_ptr<RowwiseIterator>> ret;
 
 
   // Grab the memrowset iterator.
-  gscoped_ptr<RowwiseIterator> ms_iter;
+  unique_ptr<RowwiseIterator> ms_iter;
   RETURN_NOT_OK(components_->memrowset->NewRowIterator(opts, &ms_iter));
   ret.emplace_back(ms_iter.release());
 
@@ -1855,28 +1856,28 @@ Status Tablet::CaptureConsistentIterators(
     vector<RowSet*> interval_sets;
     components_->rowsets->FindRowSetsIntersectingInterval(lower_bound, upper_bound, &interval_sets);
     for (const RowSet *rs : interval_sets) {
-      gscoped_ptr<RowwiseIterator> row_it;
+      unique_ptr<RowwiseIterator> row_it;
       RETURN_NOT_OK_PREPEND(rs->NewRowIterator(opts, &row_it),
                             Substitute("Could not create iterator for rowset $0",
                                        rs->ToString()));
-      ret.emplace_back(row_it.release());
+      ret.emplace_back(std::move(row_it));
     }
-    ret.swap(*iters);
+    *iters = std::move(ret);
     return Status::OK();
   }
 
   // If there are no encoded predicates of the primary keys, then
   // fall back to grabbing all rowset iterators.
   for (const shared_ptr<RowSet> &rs : components_->rowsets->all_rowsets()) {
-    gscoped_ptr<RowwiseIterator> row_it;
+    unique_ptr<RowwiseIterator> row_it;
     RETURN_NOT_OK_PREPEND(rs->NewRowIterator(opts, &row_it),
                           Substitute("Could not create iterator for rowset $0",
                                      rs->ToString()));
-    ret.emplace_back(row_it.release());
+    ret.emplace_back(std::move(row_it));
   }
 
   // Swap results into the parameters.
-  ret.swap(*iters);
+  *iters = std::move(ret);
   return Status::OK();
 }
 
@@ -2458,7 +2459,7 @@ Status Tablet::Iterator::Init(ScanSpec *spec) {
 
   RETURN_NOT_OK(tablet_->GetMappedReadProjection(projection_, &projection_));
 
-  vector<shared_ptr<RowwiseIterator>> iters;
+  vector<unique_ptr<RowwiseIterator>> iters;
   RETURN_NOT_OK(tablet_->CaptureConsistentIterators(opts_, spec, &iters));
   TRACE_COUNTER_INCREMENT("rowset_iterators", iters.size());
 
diff --git a/src/kudu/tablet/tablet.h b/src/kudu/tablet/tablet.h
index 9853191..62f7cb4 100644
--- a/src/kudu/tablet/tablet.h
+++ b/src/kudu/tablet/tablet.h
@@ -193,7 +193,7 @@ class Tablet {
   // state of this tablet.
   // The returned iterator is not initialized.
   Status NewRowIterator(const Schema& projection,
-                        gscoped_ptr<RowwiseIterator>* iter) const;
+                        std::unique_ptr<RowwiseIterator>* iter) const;
 
   // Create a new row iterator using specific iterator options.
   //
@@ -207,7 +207,7 @@ class Tablet {
   // because the iterator constructs and holds the relevant instance of that
   // object as a member variable.
   Status NewRowIterator(RowIteratorOptions opts,
-                        gscoped_ptr<RowwiseIterator>* iter) const;
+                        std::unique_ptr<RowwiseIterator>* iter) const;
 
   // Flush the current MemRowSet for this tablet to disk. This swaps
   // in a new (initially empty) MemRowSet in its place.
@@ -583,7 +583,7 @@ class Tablet {
   // lifetime of the returned iterators.
   Status CaptureConsistentIterators(const RowIteratorOptions& opts,
                                     const ScanSpec* spec,
-                                    std::vector<std::shared_ptr<RowwiseIterator> >* iters) const;
+                                    std::vector<std::unique_ptr<RowwiseIterator>>* iters) const;
 
   Status PickRowSetsToCompact(RowSetsInCompaction *picked,
                               CompactFlags flags) const;
@@ -809,7 +809,7 @@ class Tablet::Iterator : public RowwiseIterator {
   fs::IOContext io_context_;
   Schema projection_;
   RowIteratorOptions opts_;
-  gscoped_ptr<RowwiseIterator> iter_;
+  std::unique_ptr<RowwiseIterator> iter_;
 };
 
 // Structure which represents the components of the tablet's storage.
diff --git a/src/kudu/tablet/tablet_bootstrap-test.cc b/src/kudu/tablet/tablet_bootstrap-test.cc
index ea53e44..256043a 100644
--- a/src/kudu/tablet/tablet_bootstrap-test.cc
+++ b/src/kudu/tablet/tablet_bootstrap-test.cc
@@ -198,11 +198,7 @@ class BootstrapTest : public LogTestBase {
 
   void IterateTabletRows(const Tablet* tablet,
                          vector<string>* results) {
-    gscoped_ptr<RowwiseIterator> iter;
-    // TODO: there seems to be something funny with timestamps in this test.
-    // Unless we explicitly scan at a snapshot including all timestamps, we don't
-    // see the bootstrapped operation. This is likely due to KUDU-138 -- perhaps
-    // we aren't properly setting up the clock after bootstrap.
+    unique_ptr<RowwiseIterator> iter;
     RowIteratorOptions opts;
     opts.projection = &schema_;
     ASSERT_OK(tablet->NewRowIterator(std::move(opts), &iter));
diff --git a/src/kudu/tablet/tablet_random_access-test.cc b/src/kudu/tablet/tablet_random_access-test.cc
index a7c831e..d8fc442 100644
--- a/src/kudu/tablet/tablet_random_access-test.cc
+++ b/src/kudu/tablet/tablet_random_access-test.cc
@@ -25,10 +25,10 @@
 #include <boost/bind.hpp> // IWYU pragma: keep
 #include <boost/optional/optional.hpp>
 #include <boost/optional/optional_io.hpp>
-#include <glog/logging.h>
-#include <gtest/gtest.h>
 #include <gflags/gflags.h>
 #include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
 
 #include "kudu/common/column_predicate.h"
 #include "kudu/common/common.pb.h"
@@ -64,6 +64,7 @@ DECLARE_int32(deltafile_default_block_size);
 
 using boost::optional;
 using std::string;
+using std::unique_ptr;
 using std::vector;
 
 namespace kudu {
@@ -270,7 +271,7 @@ class TestRandomAccess : public KuduTabletTest {
   optional<ExpectedKeyValueRow> GetRow(int key) {
     ScanSpec spec;
     const Schema& schema = this->client_schema_;
-    gscoped_ptr<RowwiseIterator> iter;
+    unique_ptr<RowwiseIterator> iter;
     CHECK_OK(this->tablet()->NewRowIterator(schema, &iter));
     auto pred_one = ColumnPredicate::Equality(schema.column(0), &key);
     spec.AddPredicate(pred_one);
diff --git a/src/kudu/tools/tool_action_fs.cc b/src/kudu/tools/tool_action_fs.cc
index 5da0319..88072e7 100644
--- a/src/kudu/tools/tool_action_fs.cc
+++ b/src/kudu/tools/tool_action_fs.cc
@@ -15,8 +15,6 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "kudu/tools/tool_action.h"
-
 #include <algorithm>
 #include <cstdint>
 #include <iostream>
@@ -66,6 +64,7 @@
 #include "kudu/tablet/rowset_metadata.h"
 #include "kudu/tablet/tablet.pb.h"
 #include "kudu/tablet/tablet_metadata.h"
+#include "kudu/tools/tool_action.h"
 #include "kudu/tools/tool_action_common.h"
 #include "kudu/util/compression/compression.pb.h"
 #include "kudu/util/env.h"
@@ -270,7 +269,7 @@ Status DumpCFile(const RunnerContext& context) {
   }
 
   if (FLAGS_print_rows) {
-    gscoped_ptr<CFileIterator> it;
+    unique_ptr<CFileIterator> it;
     RETURN_NOT_OK(reader->NewIterator(&it, CFileReader::DONT_CACHE_BLOCK, nullptr));
     RETURN_NOT_OK(it->SeekToFirst());
 
diff --git a/src/kudu/tserver/scanners.cc b/src/kudu/tserver/scanners.cc
index 60f69fb..e1562e8 100644
--- a/src/kudu/tserver/scanners.cc
+++ b/src/kudu/tserver/scanners.cc
@@ -18,6 +18,7 @@
 
 #include <algorithm>
 #include <cstdint>
+#include <memory>
 #include <mutex>
 #include <ostream>
 
@@ -64,6 +65,7 @@ METRIC_DEFINE_gauge_size(server, active_scanners,
                          "Number of scanners that are currently active");
 
 using std::string;
+using std::unique_ptr;
 using std::vector;
 using strings::Substitute;
 
@@ -341,11 +343,11 @@ void Scanner::UpdateAccessTime() {
   last_access_time_ = MonoTime::Now();
 }
 
-void Scanner::Init(gscoped_ptr<RowwiseIterator> iter,
+void Scanner::Init(unique_ptr<RowwiseIterator> iter,
                    gscoped_ptr<ScanSpec> spec) {
   std::lock_guard<simple_spinlock> l(lock_);
   CHECK(!iter_) << "Already initialized";
-  iter_.reset(iter.release());
+  iter_ = std::move(iter);
   spec_.reset(spec.release());
 }
 
diff --git a/src/kudu/tserver/scanners.h b/src/kudu/tserver/scanners.h
index 354ec90..3f03bd3 100644
--- a/src/kudu/tserver/scanners.h
+++ b/src/kudu/tserver/scanners.h
@@ -201,7 +201,7 @@ class Scanner {
 
   // Attach an actual iterator and a ScanSpec to this Scanner.
   // Takes ownership of 'iter' and 'spec'.
-  void Init(gscoped_ptr<RowwiseIterator> iter,
+  void Init(std::unique_ptr<RowwiseIterator> iter,
             gscoped_ptr<ScanSpec> spec);
 
   // Return true if the scanner has been initialized (i.e has an iterator).
@@ -364,7 +364,7 @@ class Scanner {
   // schema used by the iterator.
   gscoped_ptr<Schema> client_projection_schema_;
 
-  gscoped_ptr<RowwiseIterator> iter_;
+  std::unique_ptr<RowwiseIterator> iter_;
 
   AutoReleasePool autorelease_pool_;
 
diff --git a/src/kudu/tserver/tablet_server-test-base.cc b/src/kudu/tserver/tablet_server-test-base.cc
index 59b26e9..9c46103 100644
--- a/src/kudu/tserver/tablet_server-test-base.cc
+++ b/src/kudu/tserver/tablet_server-test-base.cc
@@ -38,7 +38,6 @@
 #include "kudu/common/wire_protocol.h"
 #include "kudu/common/wire_protocol.pb.h"
 #include "kudu/consensus/raft_consensus.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/stringprintf.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/rpc/messenger.h"
@@ -73,6 +72,7 @@ using kudu::pb_util::SecureDebugString;
 using kudu::pb_util::SecureShortDebugString;
 using kudu::rpc::RpcController;
 using std::string;
+using std::unique_ptr;
 using std::vector;
 using strings::Substitute;
 
@@ -402,9 +402,9 @@ Status TabletServerTestBase::ShutdownAndRebuildTablet(int num_data_dirs) {
 // Verifies that a set of expected rows (key, value) is present in the tablet.
 void TabletServerTestBase::VerifyRows(const Schema& schema,
                                       const vector<KeyValue>& expected) {
-  gscoped_ptr<RowwiseIterator> iter;
+  unique_ptr<RowwiseIterator> iter;
   ASSERT_OK(tablet_replica_->tablet()->NewRowIterator(schema, &iter));
-  ASSERT_OK(iter->Init(NULL));
+  ASSERT_OK(iter->Init(nullptr));
 
   int batch_size = std::max<int>(1,
      std::min<int>(expected.size() / 10,
diff --git a/src/kudu/tserver/tablet_service.cc b/src/kudu/tserver/tablet_service.cc
index 40c855d..9834f8b 100644
--- a/src/kudu/tserver/tablet_service.cc
+++ b/src/kudu/tserver/tablet_service.cc
@@ -56,6 +56,7 @@
 #include "kudu/consensus/replica_management.pb.h"
 #include "kudu/consensus/time_manager.h"
 #include "kudu/gutil/casts.h"
+#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/ref_counted.h"
@@ -1899,7 +1900,7 @@ Status TabletServiceImpl::HandleNewScanRequest(TabletReplica* replica,
   }
   projection = projection_builder.BuildWithoutIds();
 
-  gscoped_ptr<RowwiseIterator> iter;
+  unique_ptr<RowwiseIterator> iter;
   // Preset the error code for when creating the iterator on the tablet fails
   TabletServerErrorPB::Code tmp_error_code = TabletServerErrorPB::MISMATCHED_SCHEMA;
 
@@ -2233,7 +2234,7 @@ Status TabletServiceImpl::HandleScanAtSnapshot(const NewScanRequestPB& scan_pb,
                                                const Schema& projection,
                                                Tablet* tablet,
                                                consensus::TimeManager* time_manager,
-                                               gscoped_ptr<RowwiseIterator>* iter,
+                                               unique_ptr<RowwiseIterator>* iter,
                                                Timestamp* snap_timestamp) {
   switch (scan_pb.read_mode()) {
     case READ_AT_SNAPSHOT: // Fallthrough intended
diff --git a/src/kudu/tserver/tablet_service.h b/src/kudu/tserver/tablet_service.h
index e4d67f9..bc4c2b0 100644
--- a/src/kudu/tserver/tablet_service.h
+++ b/src/kudu/tserver/tablet_service.h
@@ -18,10 +18,10 @@
 #define KUDU_TSERVER_TABLET_SERVICE_H
 
 #include <cstdint>
+#include <memory>
 #include <string>
 
 #include "kudu/consensus/consensus.service.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/port.h"
 #include "kudu/tserver/tserver.pb.h"
 #include "kudu/tserver/tserver_admin.service.h"
@@ -156,7 +156,7 @@ class TabletServiceImpl : public TabletServerServiceIf {
                               const Schema& projection,
                               tablet::Tablet* tablet,
                               consensus::TimeManager* time_manager,
-                              gscoped_ptr<RowwiseIterator>* iter,
+                              std::unique_ptr<RowwiseIterator>* iter,
                               Timestamp* snap_timestamp);
 
   // Validates the given timestamp is not so far in the future that