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 2020/03/06 05:12:39 UTC

[kudu] 03/03: util: remove all gscoped_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

commit fd4c4771d1a146b75c4648f1807b0e3417c42bf9
Author: Adar Dembo <ad...@cloudera.com>
AuthorDate: Wed Mar 4 18:37:09 2020 -0800

    util: remove all gscoped_ptr
    
    Except from callback_bind-test.cc, which tests gscoped_ptr functionality.
    
    Change-Id: Ia110feea85b6ca43ba53d2d4783f47000794e196
    Reviewed-on: http://gerrit.cloudera.org:8080/15377
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
    Tested-by: Adar Dembo <ad...@cloudera.com>
---
 src/kudu/client/batcher.cc                    | 48 +++++++++-----------
 src/kudu/client/batcher.h                     |  4 +-
 src/kudu/client/scan_predicate-internal.h     |  7 ++-
 src/kudu/tablet/concurrent_btree.h            |  8 +---
 src/kudu/tools/table_scanner.cc               |  3 +-
 src/kudu/tools/table_scanner.h                |  1 +
 src/kudu/util/bloom_filter.h                  |  9 ++--
 src/kudu/util/compression/compression-test.cc | 12 ++---
 src/kudu/util/crc-test.cc                     | 25 ++++++-----
 src/kudu/util/debug/trace_event_impl.cc       | 64 +++++++++++++--------------
 src/kudu/util/debug/trace_event_impl.h        | 26 +++++------
 src/kudu/util/env_posix.cc                    | 11 +++--
 src/kudu/util/hdr_histogram.cc                |  6 +--
 src/kudu/util/hdr_histogram.h                 |  4 +-
 src/kudu/util/inline_slice-test.cc            |  6 ++-
 src/kudu/util/kernel_stack_watchdog.h         |  7 +--
 src/kudu/util/memory/arena.h                  |  9 +---
 src/kudu/util/memory/memory.h                 | 12 ++---
 src/kudu/util/metrics.h                       |  9 ++--
 src/kudu/util/mutex.h                         |  4 +-
 src/kudu/util/nvm_cache.cc                    | 21 ++++-----
 src/kudu/util/nvm_cache.h                     |  2 +
 src/kudu/util/object_pool-test.cc             |  1 -
 src/kudu/util/object_pool.h                   | 12 ++---
 src/kudu/util/path_util.cc                    | 19 ++++++--
 src/kudu/util/path_util.h                     |  4 +-
 src/kudu/util/protoc-gen-insertions.cc        |  9 ++--
 src/kudu/util/trace.h                         |  1 -
 src/kudu/util/user.cc                         | 12 ++---
 29 files changed, 167 insertions(+), 189 deletions(-)

diff --git a/src/kudu/client/batcher.cc b/src/kudu/client/batcher.cc
index 20ff371..ba1d18b 100644
--- a/src/kudu/client/batcher.cc
+++ b/src/kudu/client/batcher.cc
@@ -21,9 +21,7 @@
 #include <memory>
 #include <mutex>
 #include <ostream>
-#include <set>
 #include <string>
-#include <type_traits>
 #include <unordered_map>
 #include <utility>
 #include <vector>
@@ -49,7 +47,6 @@
 #include "kudu/gutil/atomic_refcount.h"
 #include "kudu/gutil/bind.h"
 #include "kudu/gutil/bind_helpers.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/strings/substitute.h"
@@ -66,8 +63,25 @@
 #include "kudu/util/logging.h"
 #include "kudu/util/pb_util.h"
 
-using std::pair;
-using std::set;
+namespace kudu {
+namespace rpc {
+class Messenger;
+}  // namespace rpc
+}  // namespace kudu
+
+using kudu::pb_util::SecureDebugString;
+using kudu::pb_util::SecureShortDebugString;
+using kudu::rpc::CredentialsPolicy;
+using kudu::rpc::ErrorStatusPB;
+using kudu::rpc::Messenger;
+using kudu::rpc::RequestTracker;
+using kudu::rpc::ResponseCallback;
+using kudu::rpc::RetriableRpc;
+using kudu::rpc::RetriableRpcStatus;
+using kudu::security::SignedTokenPB;
+using kudu::tserver::WriteRequestPB;
+using kudu::tserver::WriteResponsePB;
+using kudu::tserver::WriteResponsePB_PerRowErrorPB;
 using std::shared_ptr;
 using std::string;
 using std::unique_ptr;
@@ -79,24 +93,6 @@ namespace kudu {
 
 class Schema;
 
-namespace rpc {
-class Messenger;
-}
-
-using pb_util::SecureDebugString;
-using pb_util::SecureShortDebugString;
-using rpc::CredentialsPolicy;
-using rpc::ErrorStatusPB;
-using rpc::Messenger;
-using rpc::RequestTracker;
-using rpc::ResponseCallback;
-using rpc::RetriableRpc;
-using rpc::RetriableRpcStatus;
-using security::SignedTokenPB;
-using tserver::WriteRequestPB;
-using tserver::WriteResponsePB;
-using tserver::WriteResponsePB_PerRowErrorPB;
-
 namespace client {
 
 namespace internal {
@@ -187,7 +183,7 @@ struct InFlightOp {
   State state;
 
   // The actual operation.
-  gscoped_ptr<KuduWriteOperation> write_op;
+  unique_ptr<KuduWriteOperation> write_op;
 
   // The tablet the operation is destined for.
   // This is only filled in after passing through the kLookingUpTablet state.
@@ -715,7 +711,7 @@ void Batcher::FlushAsync(KuduStatusCallback* cb) {
 Status Batcher::Add(KuduWriteOperation* write_op) {
   // As soon as we get the op, start looking up where it belongs,
   // so that when the user calls Flush, we are ready to go.
-  gscoped_ptr<InFlightOp> op(new InFlightOp());
+  unique_ptr<InFlightOp> op(new InFlightOp());
   string partition_key;
   RETURN_NOT_OK(write_op->table_->partition_schema().EncodeKey(write_op->row(), &partition_key));
   op->write_op.reset(write_op);
@@ -943,7 +939,7 @@ void Batcher::ProcessWriteResponse(const WriteRpc& rpc,
                  << SecureDebugString(rpc.resp());
       continue;
     }
-    gscoped_ptr<KuduWriteOperation> op = std::move(rpc.ops()[err_pb.row_index()]->write_op);
+    unique_ptr<KuduWriteOperation> op = std::move(rpc.ops()[err_pb.row_index()]->write_op);
     VLOG(2) << "Error on op " << op->ToString() << ": "
             << SecureShortDebugString(err_pb.error());
     Status op_status = StatusFromPB(err_pb.error());
diff --git a/src/kudu/client/batcher.h b/src/kudu/client/batcher.h
index 7128d71..4e72ec2 100644
--- a/src/kudu/client/batcher.h
+++ b/src/kudu/client/batcher.h
@@ -14,8 +14,7 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-#ifndef KUDU_CLIENT_BATCHER_H
-#define KUDU_CLIENT_BATCHER_H
+#pragma once
 
 #include <cstdint>
 #include <mutex>
@@ -237,4 +236,3 @@ class Batcher : public RefCountedThreadSafe<Batcher> {
 } // namespace internal
 } // namespace client
 } // namespace kudu
-#endif /* KUDU_CLIENT_BATCHER_H */
diff --git a/src/kudu/client/scan_predicate-internal.h b/src/kudu/client/scan_predicate-internal.h
index 8ba7a8e..5ee276e 100644
--- a/src/kudu/client/scan_predicate-internal.h
+++ b/src/kudu/client/scan_predicate-internal.h
@@ -14,9 +14,9 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-#ifndef KUDU_CLIENT_SCAN_PREDICATE_INTERNAL_H
-#define KUDU_CLIENT_SCAN_PREDICATE_INTERNAL_H
+#pragma once
 
+#include <memory>
 #include <utility>
 #include <vector>
 
@@ -88,7 +88,7 @@ class ComparisonPredicateData : public KuduPredicate::Data {
 
   ColumnSchema col_;
   KuduPredicate::ComparisonOp op_;
-  gscoped_ptr<KuduValue> val_;
+  std::unique_ptr<KuduValue> val_;
 };
 
 // An InBloomFilter predicate for selecting values present in the vector of Bloom filters.
@@ -211,4 +211,3 @@ class KuduBloomFilter::Data {
 
 } // namespace client
 } // namespace kudu
-#endif /* KUDU_CLIENT_SCAN_PREDICATE_INTERNAL_H */
diff --git a/src/kudu/tablet/concurrent_btree.h b/src/kudu/tablet/concurrent_btree.h
index 4e919d4..57ea0bb 100644
--- a/src/kudu/tablet/concurrent_btree.h
+++ b/src/kudu/tablet/concurrent_btree.h
@@ -39,9 +39,7 @@
 // NOTE: this code disables TSAN for the most part. This is because it uses
 // some "clever" concurrency mechanisms which are difficult to model in TSAN.
 // We instead ensure correctness by heavy stress-testing.
-
-#ifndef KUDU_TABLET_CONCURRENT_BTREE_H
-#define KUDU_TABLET_CONCURRENT_BTREE_H
+#pragma once
 
 #include <algorithm>
 #include <boost/smart_ptr/detail/yield_k.hpp>
@@ -1100,7 +1098,7 @@ class CBTree {
   // Note that this requires iterating through the entire tree,
   // so it is not very efficient.
   size_t count() const {
-    gscoped_ptr<CBTreeIterator<Traits> > iter(NewIterator());
+    std::unique_ptr<CBTreeIterator<Traits>> iter(NewIterator());
     bool exact;
     iter->SeekAtOrAfter(Slice(""), &exact);
     size_t count = 0;
@@ -1862,5 +1860,3 @@ class CBTreeIterator {
 } // namespace btree
 } // namespace tablet
 } // namespace kudu
-
-#endif
diff --git a/src/kudu/tools/table_scanner.cc b/src/kudu/tools/table_scanner.cc
index f50b93a..af34e70 100644
--- a/src/kudu/tools/table_scanner.cc
+++ b/src/kudu/tools/table_scanner.cc
@@ -31,7 +31,6 @@
 #include <boost/bind.hpp>
 #include <boost/optional/optional.hpp>
 #include <gflags/gflags.h>
-#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 #include <rapidjson/document.h>
 
@@ -380,7 +379,7 @@ Status CreateDstTableIfNeeded(const client::sp::shared_ptr<KuduTable>& src_table
   };
 
   // Table schema and replica number.
-  gscoped_ptr<KuduTableCreator> table_creator(dst_client->NewTableCreator());
+  unique_ptr<KuduTableCreator> table_creator(dst_client->NewTableCreator());
   table_creator->table_name(dst_table_name)
       .schema(&dst_table_schema)
       .num_replicas(src_table->num_replicas());
diff --git a/src/kudu/tools/table_scanner.h b/src/kudu/tools/table_scanner.h
index ff6b81b..fc5352e 100644
--- a/src/kudu/tools/table_scanner.h
+++ b/src/kudu/tools/table_scanner.h
@@ -20,6 +20,7 @@
 #include <cstdint>
 #include <functional>
 #include <iosfwd>
+#include <memory>
 #include <string>
 #include <utility>
 #include <vector>
diff --git a/src/kudu/util/bloom_filter.h b/src/kudu/util/bloom_filter.h
index 30cd64c..27c631f 100644
--- a/src/kudu/util/bloom_filter.h
+++ b/src/kudu/util/bloom_filter.h
@@ -14,13 +14,12 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-#ifndef KUDU_UTIL_BLOOM_FILTER_H
-#define KUDU_UTIL_BLOOM_FILTER_H
+#pragma once
 
 #include <cstddef>
 #include <cstdint>
+#include <memory>
 
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/hash/city.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/port.h"
@@ -175,7 +174,7 @@ class BloomFilterBuilder {
   DISALLOW_COPY_AND_ASSIGN(BloomFilterBuilder);
 
   size_t n_bits_;
-  gscoped_array<uint8_t> bitmap_;
+  std::unique_ptr<uint8_t[]> bitmap_;
 
   // The number of hash functions to compute.
   size_t n_hashes_;
@@ -267,5 +266,3 @@ inline bool BloomFilter::MayContainKey(const BloomKeyProbe &probe) const {
 }
 
 } // namespace kudu
-
-#endif
diff --git a/src/kudu/util/compression/compression-test.cc b/src/kudu/util/compression/compression-test.cc
index 9aa8635..df3e8ae 100644
--- a/src/kudu/util/compression/compression-test.cc
+++ b/src/kudu/util/compression/compression-test.cc
@@ -19,6 +19,8 @@
 #include <cstdint>
 #include <cstdlib>
 #include <cstring>
+#include <initializer_list>
+#include <memory>
 #include <ostream>
 #include <string>
 #include <vector>
@@ -26,7 +28,6 @@
 #include <glog/logging.h>
 #include <gtest/gtest.h>
 
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/util/compression/compression.pb.h"
 #include "kudu/util/compression/compression_codec.h"
 #include "kudu/util/random.h"
@@ -36,11 +37,12 @@
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
-namespace kudu {
-
 using std::string;
+using std::unique_ptr;
 using std::vector;
 
+namespace kudu {
+
 class TestCompression : public KuduTest {};
 
 static void TestCompressionCodec(CompressionType compression) {
@@ -60,7 +62,7 @@ static void TestCompressionCodec(CompressionType compression) {
   // Allocate the compression buffer
   size_t max_compressed = codec->MaxCompressedLength(kInputSize);
   ASSERT_LT(max_compressed, (kInputSize * 2));
-  gscoped_array<uint8_t> cbuffer(new uint8_t[max_compressed]);
+  unique_ptr<uint8_t[]> cbuffer(new uint8_t[max_compressed]);
 
   // Compress and uncompress
   ASSERT_OK(codec->Compress(Slice(ibuffer, kInputSize), cbuffer.get(), &compressed));
@@ -102,7 +104,7 @@ static void Benchmark(Random random, CompressionType compression) {
 
   // Allocate the compression buffer.
   size_t max_compressed = codec->MaxCompressedLength(kSliceCount * kInputSize);
-  gscoped_array<uint8_t> cbuffer(new uint8_t[max_compressed]);
+  unique_ptr<uint8_t[]> cbuffer(new uint8_t[max_compressed]);
 
   // Execute Compress.
   size_t compressed;
diff --git a/src/kudu/util/crc-test.cc b/src/kudu/util/crc-test.cc
index cf13268..41c35e1 100644
--- a/src/kudu/util/crc-test.cc
+++ b/src/kudu/util/crc-test.cc
@@ -15,39 +15,42 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "kudu/util/crc.h"
+
 #include <cstdint>
 #include <cstring>
+#include <memory>
 #include <ostream>
+#include <utility>
 #include <string>
 
 #include <glog/logging.h>
 #include <gtest/gtest.h>
 
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/strings/numbers.h"
 #include "kudu/gutil/strings/substitute.h"
-#include "kudu/util/crc.h"
 #include "kudu/util/stopwatch.h"
 #include "kudu/util/test_util.h"
 
+using std::unique_ptr;
+using strings::Substitute;
+
 namespace kudu {
 namespace crc {
 
-using strings::Substitute;
-
 class CrcTest : public KuduTest {
  protected:
 
   // Returns pointer to data which must be deleted by caller.
-  static void GenerateBenchmarkData(const uint8_t** bufptr, size_t* buflen) {
+  static void GenerateBenchmarkData(unique_ptr<uint8_t[]>* bufptr, size_t* buflen) {
     const uint32_t kNumNumbers = 1000000;
     const uint32_t kBytesPerNumber = sizeof(uint32_t);
     const uint32_t kLength = kNumNumbers * kBytesPerNumber;
-    auto buf = new uint8_t[kLength];
+    unique_ptr<uint8_t[]> buf(new uint8_t[kLength]);
     for (uint32_t i = 0; i < kNumNumbers; i++) {
-      memcpy(buf + (i * kBytesPerNumber), &i, kBytesPerNumber);
+      memcpy(buf.get() + (i * kBytesPerNumber), &i, kBytesPerNumber);
     }
-    *bufptr = buf;
+    *bufptr = std::move(buf);
     *buflen = kLength;
   }
 
@@ -82,11 +85,9 @@ TEST_F(CrcTest, TestCRC32C) {
 // Simple benchmark of CRC32C throughput.
 // We should expect about 8 bytes per cycle in throughput on a single core.
 TEST_F(CrcTest, BenchmarkCRC32C) {
-  gscoped_ptr<const uint8_t[]> data;
-  const uint8_t* buf;
+  unique_ptr<uint8_t[]> buf;
   size_t buflen;
   GenerateBenchmarkData(&buf, &buflen);
-  data.reset(buf);
   Crc* crc32c = GetCrc32cInstance();
   int kNumRuns = 1000;
   if (AllowSlowTests()) {
@@ -97,7 +98,7 @@ TEST_F(CrcTest, BenchmarkCRC32C) {
   sw.start();
   for (int i = 0; i < kNumRuns; i++) {
     uint64_t cksum;
-    crc32c->Compute(buf, buflen, &cksum);
+    crc32c->Compute(buf.get(), buflen, &cksum);
   }
   sw.stop();
   CpuTimes elapsed = sw.elapsed();
diff --git a/src/kudu/util/debug/trace_event_impl.cc b/src/kudu/util/debug/trace_event_impl.cc
index 0f1da6e..f642657 100644
--- a/src/kudu/util/debug/trace_event_impl.cc
+++ b/src/kudu/util/debug/trace_event_impl.cc
@@ -14,8 +14,8 @@
 #include <cstring>
 #include <functional>
 #include <list>
+#include <memory>
 #include <sstream>
-#include <type_traits>
 #include <utility>
 #include <vector>
 
@@ -54,15 +54,15 @@ TAG_FLAG(trace_to_console, experimental);
 // The thread buckets for the sampling profiler.
 BASE_EXPORT TRACE_EVENT_API_ATOMIC_WORD g_trace_state[3];
 
-namespace kudu {
-namespace debug {
-
 using base::SpinLockHolder;
-
 using strings::SubstituteAndAppend;
 using std::string;
+using std::unique_ptr;
 using std::vector;
 
+namespace kudu {
+namespace debug {
+
 __thread TraceLog::PerThreadInfo* TraceLog::thread_local_info_ = nullptr;
 
 namespace {
@@ -131,7 +131,7 @@ class TraceBufferRingBuffer : public TraceBuffer {
     STLDeleteElements(&chunks_);
   }
 
-  virtual gscoped_ptr<TraceBufferChunk> GetChunk(size_t* index) OVERRIDE {
+  virtual unique_ptr<TraceBufferChunk> GetChunk(size_t* index) OVERRIDE {
     // Because the number of threads is much less than the number of chunks,
     // the queue should never be empty.
     DCHECK(!QueueIsEmpty());
@@ -150,11 +150,11 @@ class TraceBufferRingBuffer : public TraceBuffer {
     else
       chunk = new TraceBufferChunk(current_chunk_seq_++);
 
-    return gscoped_ptr<TraceBufferChunk>(chunk);
+    return unique_ptr<TraceBufferChunk>(chunk);
   }
 
   virtual void ReturnChunk(size_t index,
-                           gscoped_ptr<TraceBufferChunk> chunk) OVERRIDE {
+                           unique_ptr<TraceBufferChunk> chunk) OVERRIDE {
     // When this method is called, the queue should not be full because it
     // can contain all chunks including the one to be returned.
     DCHECK(!QueueIsFull());
@@ -203,8 +203,8 @@ class TraceBufferRingBuffer : public TraceBuffer {
     return nullptr;
   }
 
-  virtual gscoped_ptr<TraceBuffer> CloneForIteration() const OVERRIDE {
-    gscoped_ptr<ClonedTraceBuffer> cloned_buffer(new ClonedTraceBuffer());
+  virtual unique_ptr<TraceBuffer> CloneForIteration() const OVERRIDE {
+    unique_ptr<ClonedTraceBuffer> cloned_buffer(new ClonedTraceBuffer());
     for (size_t queue_index = queue_head_; queue_index != queue_tail_;
         queue_index = NextQueueIndex(queue_index)) {
       size_t chunk_index = recyclable_chunks_queue_[queue_index];
@@ -213,7 +213,7 @@ class TraceBufferRingBuffer : public TraceBuffer {
       TraceBufferChunk* chunk = chunks_[chunk_index];
       cloned_buffer->chunks_.push_back(chunk ? chunk->Clone().release() : nullptr);
     }
-    return cloned_buffer.PassAs<TraceBuffer>();
+    return std::move(cloned_buffer);
   }
 
  private:
@@ -230,23 +230,23 @@ class TraceBufferRingBuffer : public TraceBuffer {
           chunks_[current_iteration_index_++] : nullptr;
     }
 
-    virtual gscoped_ptr<TraceBufferChunk> GetChunk(size_t* index) OVERRIDE {
+    virtual unique_ptr<TraceBufferChunk> GetChunk(size_t* /*index*/) OVERRIDE {
       NOTIMPLEMENTED();
-      return gscoped_ptr<TraceBufferChunk>();
+      return unique_ptr<TraceBufferChunk>();
     }
-    virtual void ReturnChunk(size_t index,
-                             gscoped_ptr<TraceBufferChunk>) OVERRIDE {
+    virtual void ReturnChunk(size_t /*index*/,
+                             unique_ptr<TraceBufferChunk> /*chunk*/) OVERRIDE {
       NOTIMPLEMENTED();
     }
     virtual bool IsFull() const OVERRIDE { return false; }
     virtual size_t Size() const OVERRIDE { return 0; }
     virtual size_t Capacity() const OVERRIDE { return 0; }
-    virtual TraceEvent* GetEventByHandle(TraceEventHandle handle) OVERRIDE {
+    virtual TraceEvent* GetEventByHandle(TraceEventHandle /*handle*/) OVERRIDE {
       return nullptr;
     }
-    virtual gscoped_ptr<TraceBuffer> CloneForIteration() const OVERRIDE {
+    virtual unique_ptr<TraceBuffer> CloneForIteration() const OVERRIDE {
       NOTIMPLEMENTED();
-      return gscoped_ptr<TraceBuffer>();
+      return unique_ptr<TraceBuffer>();
     }
 
     size_t current_iteration_index_;
@@ -281,7 +281,7 @@ class TraceBufferRingBuffer : public TraceBuffer {
   size_t max_chunks_;
   vector<TraceBufferChunk*> chunks_;
 
-  gscoped_ptr<size_t[]> recyclable_chunks_queue_;
+  unique_ptr<size_t[]> recyclable_chunks_queue_;
   size_t queue_head_;
   size_t queue_tail_;
 
@@ -302,7 +302,7 @@ class TraceBufferVector : public TraceBuffer {
     STLDeleteElements(&chunks_);
   }
 
-  virtual gscoped_ptr<TraceBufferChunk> GetChunk(size_t* index) OVERRIDE {
+  virtual unique_ptr<TraceBufferChunk> GetChunk(size_t* index) OVERRIDE {
     // This function may be called when adding normal events or indirectly from
     // AddMetadataEventsWhileLocked(). We can not DECHECK(!IsFull()) because we
     // have to add the metadata events and flush thread-local buffers even if
@@ -311,13 +311,13 @@ class TraceBufferVector : public TraceBuffer {
     chunks_.push_back(nullptr);  // Put NULL in the slot of a in-flight chunk.
     ++in_flight_chunk_count_;
     // + 1 because zero chunk_seq is not allowed.
-    return gscoped_ptr<TraceBufferChunk>(
+    return unique_ptr<TraceBufferChunk>(
         new TraceBufferChunk(static_cast<uint32_t>(*index) + 1));
   }
 
   virtual void ReturnChunk(size_t index,
-                           gscoped_ptr<TraceBufferChunk> chunk) OVERRIDE {
-    DCHECK_GT(in_flight_chunk_count_, 0u);
+                           unique_ptr<TraceBufferChunk> chunk) OVERRIDE {
+    DCHECK_GT(in_flight_chunk_count_, 0U);
     DCHECK_LT(index, chunks_.size());
     DCHECK(!chunks_[index]);
     --in_flight_chunk_count_;
@@ -356,9 +356,9 @@ class TraceBufferVector : public TraceBuffer {
     return nullptr;
   }
 
-  virtual gscoped_ptr<TraceBuffer> CloneForIteration() const OVERRIDE {
+  virtual unique_ptr<TraceBuffer> CloneForIteration() const OVERRIDE {
     NOTIMPLEMENTED();
-    return gscoped_ptr<TraceBuffer>();
+    return unique_ptr<TraceBuffer>();
   }
 
  private:
@@ -439,12 +439,12 @@ TraceEvent* TraceBufferChunk::AddTraceEvent(size_t* event_index) {
   return &chunk_[*event_index];
 }
 
-gscoped_ptr<TraceBufferChunk> TraceBufferChunk::Clone() const {
-  gscoped_ptr<TraceBufferChunk> cloned_chunk(new TraceBufferChunk(seq_));
+unique_ptr<TraceBufferChunk> TraceBufferChunk::Clone() const {
+  unique_ptr<TraceBufferChunk> cloned_chunk(new TraceBufferChunk(seq_));
   cloned_chunk->next_free_ = next_free_;
   for (size_t i = 0; i < next_free_; ++i)
     cloned_chunk->chunk_[i].CopyFrom(chunk_[i]);
-  return std::move(cloned_chunk);
+  return cloned_chunk;
 }
 
 // A helper class that allows the lock to be acquired in the middle of the scope
@@ -1057,7 +1057,7 @@ class TraceLog::ThreadLocalEventBuffer {
   // Since TraceLog is a leaky singleton, trace_log_ will always be valid
   // as long as the thread exists.
   TraceLog* trace_log_;
-  gscoped_ptr<TraceBufferChunk> chunk_;
+  unique_ptr<TraceBufferChunk> chunk_;
   size_t chunk_index_;
   int generation_;
 
@@ -1616,7 +1616,7 @@ void TraceLog::Flush(const TraceLog::OutputCallback& cb) {
 }
 
 void TraceLog::ConvertTraceEventsToTraceFormat(
-    gscoped_ptr<TraceBuffer> logged_events,
+    unique_ptr<TraceBuffer> logged_events,
     const TraceLog::OutputCallback& flush_output_callback) {
 
   if (flush_output_callback.is_null())
@@ -1649,7 +1649,7 @@ void TraceLog::ConvertTraceEventsToTraceFormat(
 
 void TraceLog::FinishFlush(int generation,
                            const TraceLog::OutputCallback& flush_output_callback) {
-  gscoped_ptr<TraceBuffer> previous_logged_events;
+  unique_ptr<TraceBuffer> previous_logged_events;
 
   if (!CheckGeneration(generation))
     return;
@@ -1667,7 +1667,7 @@ void TraceLog::FinishFlush(int generation,
 
 void TraceLog::FlushButLeaveBufferIntact(
     const TraceLog::OutputCallback& flush_output_callback) {
-  gscoped_ptr<TraceBuffer> previous_logged_events;
+  unique_ptr<TraceBuffer> previous_logged_events;
   {
     SpinLockHolder lock(&lock_);
     if (mode_ == DISABLED || (trace_options_ & RECORD_CONTINUOUSLY) == 0) {
diff --git a/src/kudu/util/debug/trace_event_impl.h b/src/kudu/util/debug/trace_event_impl.h
index 2650e8b..f0eb4c2 100644
--- a/src/kudu/util/debug/trace_event_impl.h
+++ b/src/kudu/util/debug/trace_event_impl.h
@@ -1,13 +1,12 @@
 // Copyright (c) 2012 The Chromium Authors. All rights reserved.
 // Use of this source code is governed by a BSD-style license that can be
 // found in the LICENSE file.
-
-#ifndef KUDU_UTIL_DEBUG_TRACE_EVENT_IMPL_H_
-#define KUDU_UTIL_DEBUG_TRACE_EVENT_IMPL_H_
+#pragma once
 
 #include <cstddef>
 #include <cstdint>
 #include <iosfwd>
+#include <memory>
 #include <stack>
 #include <string>
 #include <unordered_map>
@@ -19,7 +18,6 @@
 #include "kudu/gutil/atomicops.h"
 #include "kudu/gutil/bind_helpers.h"
 #include "kudu/gutil/callback.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/spinlock.h"
@@ -206,7 +204,7 @@ class BASE_EXPORT TraceBufferChunk {
     return &chunk_[index];
   }
 
-  gscoped_ptr<TraceBufferChunk> Clone() const;
+  std::unique_ptr<TraceBufferChunk> Clone() const;
 
   static const size_t kTraceBufferChunkSize = 64;
 
@@ -221,9 +219,9 @@ class BASE_EXPORT TraceBuffer {
  public:
   virtual ~TraceBuffer() {}
 
-  virtual gscoped_ptr<TraceBufferChunk> GetChunk(size_t *index) = 0;
+  virtual std::unique_ptr<TraceBufferChunk> GetChunk(size_t *index) = 0;
   virtual void ReturnChunk(size_t index,
-                           gscoped_ptr<TraceBufferChunk> chunk) = 0;
+                           std::unique_ptr<TraceBufferChunk> chunk) = 0;
 
   virtual bool IsFull() const = 0;
   virtual size_t Size() const = 0;
@@ -233,7 +231,7 @@ class BASE_EXPORT TraceBuffer {
   // For iteration. Each TraceBuffer can only be iterated once.
   virtual const TraceBufferChunk* NextChunk() = 0;
 
-  virtual gscoped_ptr<TraceBuffer> CloneForIteration() const = 0;
+  virtual std::unique_ptr<TraceBuffer> CloneForIteration() const = 0;
 };
 
 // TraceResultBuffer collects and converts trace fragments returned by TraceLog
@@ -615,8 +613,8 @@ class BASE_EXPORT TraceLog {
   TraceEvent* GetEventByHandleInternal(TraceEventHandle handle,
                                        OptionalAutoLock* lock);
 
-  void ConvertTraceEventsToTraceFormat(gscoped_ptr<TraceBuffer> logged_events,
-                                       const OutputCallback& flush_output_callback);
+  static void ConvertTraceEventsToTraceFormat(std::unique_ptr<TraceBuffer> logged_events,
+                                              const OutputCallback& flush_output_callback);
   void FinishFlush(int generation,
                    const OutputCallback& flush_output_callback);
 
@@ -654,7 +652,7 @@ class BASE_EXPORT TraceLog {
   int locked_line_;
   Mode mode_;
   int num_traces_recorded_;
-  gscoped_ptr<TraceBuffer> logged_events_;
+  std::unique_ptr<TraceBuffer> logged_events_;
   AtomicWord /* EventCallback */ event_callback_;
   bool dispatching_to_observer_list_;
   std::vector<EnabledStateObserver*> enabled_state_observer_list_;
@@ -684,7 +682,7 @@ class BASE_EXPORT TraceLog {
   AtomicWord /* Options */ trace_options_;
 
   // Sampling thread handles.
-  gscoped_ptr<TraceSamplingThread> sampling_thread_;
+  std::unique_ptr<TraceSamplingThread> sampling_thread_;
   scoped_refptr<kudu::Thread> sampling_thread_handle_;
 
   CategoryFilter category_filter_;
@@ -708,7 +706,7 @@ class BASE_EXPORT TraceLog {
 
   // For events which can't be added into the thread local buffer, e.g. events
   // from threads without a message loop.
-  gscoped_ptr<TraceBufferChunk> thread_shared_chunk_;
+  std::unique_ptr<TraceBufferChunk> thread_shared_chunk_;
   size_t thread_shared_chunk_index_;
 
   // The generation is incremented whenever tracing is enabled, and incremented
@@ -722,5 +720,3 @@ class BASE_EXPORT TraceLog {
 
 }  // namespace debug
 }  // namespace kudu
-
-#endif  // KUDU_UTIL_DEBUG_TRACE_EVENT_IMPL_H_
diff --git a/src/kudu/util/env_posix.cc b/src/kudu/util/env_posix.cc
index bca9f88..2dc608b 100644
--- a/src/kudu/util/env_posix.cc
+++ b/src/kudu/util/env_posix.cc
@@ -39,7 +39,6 @@
 #include "kudu/gutil/basictypes.h"
 #include "kudu/gutil/bind.h"
 #include "kudu/gutil/bind_helpers.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/once.h"
@@ -206,6 +205,12 @@ const char* const Env::kInjectedFailureStatusMsg = "INJECTED FAILURE";
 
 namespace {
 
+struct FreeDeleter {
+  inline void operator()(void* ptr) const {
+    free(ptr);
+  }
+};
+
 #if defined(__APPLE__)
 // Simulates Linux's fallocate file preallocation API on OS X.
 int fallocate(int fd, int mode, off_t offset, off_t len) {
@@ -1224,7 +1229,7 @@ class PosixEnv : public Env {
   Status GetCurrentWorkingDir(string* cwd) const override {
     TRACE_EVENT0("io", "PosixEnv::GetCurrentWorkingDir");
     ThreadRestrictions::AssertIOAllowed();
-    unique_ptr<char, FreeDeleter> wd(getcwd(NULL, 0));
+    unique_ptr<char[], FreeDeleter> wd(getcwd(nullptr, 0));
     if (!wd) {
       return IOError("getcwd()", errno);
     }
@@ -1514,7 +1519,7 @@ class PosixEnv : public Env {
 
     // FTS requires a non-const copy of the name. strdup it and free() when
     // we leave scope.
-    unique_ptr<char, FreeDeleter> name_dup(strdup(root.c_str()));
+    unique_ptr<char[], FreeDeleter> name_dup(strdup(root.c_str()));
     char *(paths[]) = { name_dup.get(), nullptr };
 
     // FTS_NOCHDIR is important here to make this thread-safe.
diff --git a/src/kudu/util/hdr_histogram.cc b/src/kudu/util/hdr_histogram.cc
index 958d89c..6bf0487 100644
--- a/src/kudu/util/hdr_histogram.cc
+++ b/src/kudu/util/hdr_histogram.cc
@@ -58,8 +58,7 @@ HdrHistogram::HdrHistogram(uint64_t highest_trackable_value, int num_significant
     total_count_(0),
     total_sum_(0),
     min_value_(std::numeric_limits<Atomic64>::max()),
-    max_value_(0),
-    counts_(nullptr) {
+    max_value_(0) {
   Init();
 }
 
@@ -75,8 +74,7 @@ HdrHistogram::HdrHistogram(const HdrHistogram& other)
     total_count_(0),
     total_sum_(0),
     min_value_(std::numeric_limits<Atomic64>::max()),
-    max_value_(0),
-    counts_(nullptr) {
+    max_value_(0) {
   Init();
 
   // Not a consistent snapshot but we try to roughly keep it close.
diff --git a/src/kudu/util/hdr_histogram.h b/src/kudu/util/hdr_histogram.h
index fa139e2..a92f2c9 100644
--- a/src/kudu/util/hdr_histogram.h
+++ b/src/kudu/util/hdr_histogram.h
@@ -50,9 +50,9 @@
 
 #include <cstdint>
 #include <iosfwd>
+#include <memory>
 
 #include "kudu/gutil/atomicops.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/port.h"
 
@@ -203,7 +203,7 @@ class HdrHistogram {
   base::subtle::Atomic64 total_sum_;
   base::subtle::Atomic64 min_value_;
   base::subtle::Atomic64 max_value_;
-  gscoped_array<base::subtle::Atomic64> counts_;
+  std::unique_ptr<base::subtle::Atomic64[]> counts_;
 
   HdrHistogram& operator=(const HdrHistogram& other); // Disable assignment operator.
 };
diff --git a/src/kudu/util/inline_slice-test.cc b/src/kudu/util/inline_slice-test.cc
index 60a0005..bc52c79 100644
--- a/src/kudu/util/inline_slice-test.cc
+++ b/src/kudu/util/inline_slice-test.cc
@@ -17,22 +17,24 @@
 
 #include <cstddef>
 #include <cstdint>
+#include <memory>
 #include <string>
 
 #include <gtest/gtest.h>
 
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/util/inline_slice.h"
 #include "kudu/util/memory/arena.h"
 #include "kudu/util/slice.h"
 
+using std::unique_ptr;
+
 namespace kudu {
 
 template<size_t N>
 static void TestRoundTrip(InlineSlice<N> *slice,
                           Arena *arena,
                           size_t test_size) {
-  gscoped_ptr<uint8_t[]> buf(new uint8_t[test_size]);
+  unique_ptr<uint8_t[]> buf(new uint8_t[test_size]);
   for (int i = 0; i < test_size; i++) {
     buf[i] = i & 0xff;
   }
diff --git a/src/kudu/util/kernel_stack_watchdog.h b/src/kudu/util/kernel_stack_watchdog.h
index 6ec7b50..20053bd 100644
--- a/src/kudu/util/kernel_stack_watchdog.h
+++ b/src/kudu/util/kernel_stack_watchdog.h
@@ -50,8 +50,7 @@
 //
 // Scopes with SCOPED_WATCH_STACK may be nested, but only up to a hard-coded limited depth
 // (currently 8).
-#ifndef KUDU_UTIL_KERNEL_STACK_WATCHDOG_H
-#define KUDU_UTIL_KERNEL_STACK_WATCHDOG_H
+#pragma once
 
 #include <ctime>
 #include <memory>
@@ -62,7 +61,6 @@
 #include <glog/logging.h>
 
 #include "kudu/gutil/atomicops.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/ref_counted.h"
@@ -203,7 +201,7 @@ class KernelStackWatchdog {
 
   // If non-NULL, warnings will be emitted into this vector instead of glog.
   // Used by tests.
-  gscoped_ptr<std::vector<std::string> > log_collector_;
+  std::unique_ptr<std::vector<std::string> > log_collector_;
 
   // Lock protecting log_collector_.
   mutable simple_spinlock log_lock_;
@@ -287,4 +285,3 @@ class ScopedWatchKernelStack {
 };
 
 } // namespace kudu
-#endif /* KUDU_UTIL_KERNEL_STACK_WATCHDOG_H */
diff --git a/src/kudu/util/memory/arena.h b/src/kudu/util/memory/arena.h
index 6d9843b..34e1faa 100644
--- a/src/kudu/util/memory/arena.h
+++ b/src/kudu/util/memory/arena.h
@@ -19,9 +19,7 @@
 //
 //
 // Memory arena for variable-length datatypes and STL collections.
-
-#ifndef KUDU_UTIL_MEMORY_ARENA_H_
-#define KUDU_UTIL_MEMORY_ARENA_H_
+#pragma once
 
 #include <algorithm>
 #include <cstddef>
@@ -37,7 +35,6 @@
 
 #include "kudu/gutil/atomicops.h"
 #include "kudu/gutil/dynamic_annotations.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/strings/stringpiece.h"
@@ -374,7 +371,7 @@ class ArenaBase<THREADSAFE>::Component {
   // This is a no-op in a non-ASAN build.
   void AsanUnpoison(const void* addr, size_t size);
 
-  gscoped_ptr<Buffer> buffer_;
+  std::unique_ptr<Buffer> buffer_;
   uint8_t* const data_;
   typename ArenaTraits<THREADSAFE>::offset_type offset_;
   const size_t size_;
@@ -497,5 +494,3 @@ inline T *ArenaBase<THREADSAFE>::NewObject(Args&&... args) {
 }
 
 }  // namespace kudu
-
-#endif  // KUDU_UTIL_MEMORY_ARENA_H_
diff --git a/src/kudu/util/memory/memory.h b/src/kudu/util/memory/memory.h
index 315631b..5fa60a9 100644
--- a/src/kudu/util/memory/memory.h
+++ b/src/kudu/util/memory/memory.h
@@ -28,9 +28,7 @@
 // be shared between multiple materializations; e.g. you can designate a
 // single allocator per a single user request, thus setting bounds on memory
 // usage on a per-request basis.
-
-#ifndef KUDU_UTIL_MEMORY_MEMORY_H_
-#define KUDU_UTIL_MEMORY_MEMORY_H_
+#pragma once
 
 #include <algorithm>
 #include <cstddef>
@@ -45,7 +43,6 @@
 #include "kudu/util/boost_mutex_utils.h"
 #include "kudu/util/memory/overwrite.h"
 #include "kudu/util/mutex.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/singleton.h"
@@ -649,8 +646,7 @@ class MemoryStatisticsCollectingBufferAllocator : public BufferAllocator {
   virtual void FreeInternal(Buffer* buffer) OVERRIDE;
 
   BufferAllocator* delegate_;
-  gscoped_ptr<MemoryStatisticsCollectorInterface>
-      memory_stats_collector_;
+  std::unique_ptr<MemoryStatisticsCollectorInterface> memory_stats_collector_;
 };
 
 // BufferAllocator which uses MemTracker to keep track of and optionally
@@ -771,7 +767,7 @@ class OwningThreadSafeBufferAllocator
   virtual ~OwningThreadSafeBufferAllocator() {}
 
  private:
-  gscoped_ptr<DelegateAllocatorType> delegate_owned_;
+  std::unique_ptr<DelegateAllocatorType> delegate_owned_;
 };
 
 class ThreadSafeMemoryLimit
@@ -966,5 +962,3 @@ void StaticQuota<thread_safe>::SetQuota(const size_t quota) {
 }
 
 }  // namespace kudu
-
-#endif  // KUDU_UTIL_MEMORY_MEMORY_H_
diff --git a/src/kudu/util/metrics.h b/src/kudu/util/metrics.h
index 752d382..ff665ae 100644
--- a/src/kudu/util/metrics.h
+++ b/src/kudu/util/metrics.h
@@ -14,8 +14,7 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-#ifndef KUDU_UTIL_METRICS_H
-#define KUDU_UTIL_METRICS_H
+#pragma once
 
 /////////////////////////////////////////////////////
 // Kudu Metrics
@@ -232,6 +231,7 @@
 #include <cstdint>
 #include <cstring>
 #include <limits>
+#include <memory>
 #include <mutex>
 #include <string>
 #include <unordered_map>
@@ -244,7 +244,6 @@
 #include "kudu/gutil/bind.h"
 #include "kudu/gutil/callback.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/port.h"
@@ -1468,7 +1467,7 @@ class Histogram : public Metric {
   explicit Histogram(const HistogramPrototype* proto);
   Histogram(const HistogramPrototype* proto, const HdrHistogram& hdr_hist);
 
-  const gscoped_ptr<HdrHistogram> histogram_;
+  const std::unique_ptr<HdrHistogram> histogram_;
   DISALLOW_COPY_AND_ASSIGN(Histogram);
 };
 
@@ -1563,5 +1562,3 @@ inline scoped_refptr<FunctionGauge<T> > MetricEntity::FindOrCreateFunctionGauge(
 }
 
 } // namespace kudu
-
-#endif // KUDU_UTIL_METRICS_H
diff --git a/src/kudu/util/mutex.h b/src/kudu/util/mutex.h
index 9277ac0..3d4d378 100644
--- a/src/kudu/util/mutex.h
+++ b/src/kudu/util/mutex.h
@@ -20,11 +20,11 @@
 #include <pthread.h>
 #include <sys/types.h>
 
+#include <memory>
 #include <string>
 
 #include <glog/logging.h>
 
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/macros.h"
 
 namespace kudu {
@@ -70,7 +70,7 @@ class Mutex {
   // All private data is implicitly protected by native_handle_.
   // Be VERY careful to only access members under that lock.
   pid_t owning_tid_;
-  gscoped_ptr<StackTrace> stack_trace_;
+  std::unique_ptr<StackTrace> stack_trace_;
 #endif
 
   DISALLOW_COPY_AND_ASSIGN(Mutex);
diff --git a/src/kudu/util/nvm_cache.cc b/src/kudu/util/nvm_cache.cc
index d7860ac..c4aa09b 100644
--- a/src/kudu/util/nvm_cache.cc
+++ b/src/kudu/util/nvm_cache.cc
@@ -31,19 +31,16 @@
 #include <vector>
 
 #include <gflags/gflags.h>
-#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 
 #include "kudu/gutil/atomic_refcount.h"
 #include "kudu/gutil/atomicops.h"
 #include "kudu/gutil/bits.h"
 #include "kudu/gutil/dynamic_annotations.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/hash/city.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/ref_counted.h"
-#include "kudu/gutil/stl_util.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/sysinfo.h"
 #include "kudu/util/cache.h"
@@ -569,7 +566,7 @@ int DetermineShardBits() {
 class ShardedLRUCache : public Cache {
  private:
   unique_ptr<CacheMetrics> metrics_;
-  vector<NvmLRUCache*> shards_;
+  vector<unique_ptr<NvmLRUCache>> shards_;
 
   // Number of bits of hash used to determine the shard.
   const int shard_bits_;
@@ -594,14 +591,14 @@ class ShardedLRUCache : public Cache {
     int num_shards = 1 << shard_bits_;
     const size_t per_shard = (capacity + (num_shards - 1)) / num_shards;
     for (int s = 0; s < num_shards; s++) {
-      gscoped_ptr<NvmLRUCache> shard(new NvmLRUCache(vmp_));
+      unique_ptr<NvmLRUCache> shard(new NvmLRUCache(vmp_));
       shard->SetCapacity(per_shard);
-      shards_.push_back(shard.release());
+      shards_.emplace_back(std::move(shard));
     }
   }
 
   virtual ~ShardedLRUCache() {
-    STLDeleteElements(&shards_);
+    shards_.clear();
     // Per the note at the top of this file, our cache is entirely volatile.
     // Hence, when the cache is destructed, we delete the underlying
     // memkind pool.
@@ -638,8 +635,8 @@ class ShardedLRUCache : public Cache {
 
   virtual void SetMetrics(unique_ptr<CacheMetrics> metrics) OVERRIDE {
     metrics_ = std::move(metrics);
-    for (NvmLRUCache* cache : shards_) {
-      cache->SetMetrics(metrics_.get());
+    for (const auto& shard : shards_) {
+      shard->SetMetrics(metrics_.get());
     }
   }
   virtual UniquePendingHandle Allocate(Slice key, int val_len, int charge) OVERRIDE {
@@ -650,9 +647,9 @@ class ShardedLRUCache : public Cache {
     // Try allocating from each of the shards -- if memkind is tight,
     // this can cause eviction, so we might have better luck in different
     // shards.
-    for (NvmLRUCache* cache : shards_) {
+    for (const auto& shard : shards_) {
       UniquePendingHandle ph(static_cast<PendingHandle*>(
-          cache->Allocate(sizeof(LRUHandle) + key_len + val_len)),
+          shard->Allocate(sizeof(LRUHandle) + key_len + val_len)),
           Cache::PendingHandleDeleter(this));
       if (ph) {
         LRUHandle* handle = reinterpret_cast<LRUHandle*>(ph.get());
@@ -677,7 +674,7 @@ class ShardedLRUCache : public Cache {
 
   size_t Invalidate(const InvalidationControl& ctl) override {
     size_t invalidated_count = 0;
-    for (auto& shard: shards_) {
+    for (const auto& shard: shards_) {
       invalidated_count += shard->Invalidate(ctl);
     }
     return invalidated_count;
diff --git a/src/kudu/util/nvm_cache.h b/src/kudu/util/nvm_cache.h
index d6c2762..4a302f2 100644
--- a/src/kudu/util/nvm_cache.h
+++ b/src/kudu/util/nvm_cache.h
@@ -19,6 +19,8 @@
 #include <cstddef>
 #include <string>
 
+#include <glog/logging.h>
+
 #include "kudu/util/cache.h"
 
 namespace kudu {
diff --git a/src/kudu/util/object_pool-test.cc b/src/kudu/util/object_pool-test.cc
index ecfd641..4b995cc 100644
--- a/src/kudu/util/object_pool-test.cc
+++ b/src/kudu/util/object_pool-test.cc
@@ -17,7 +17,6 @@
 
 #include <gtest/gtest.h>
 
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/util/object_pool.h"
 
 namespace kudu {
diff --git a/src/kudu/util/object_pool.h b/src/kudu/util/object_pool.h
index 64d4b5c..df717e6 100644
--- a/src/kudu/util/object_pool.h
+++ b/src/kudu/util/object_pool.h
@@ -17,13 +17,14 @@
 //
 // Simple pool/freelist for objects of the same type, typically used
 // in local context.
-#ifndef KUDU_UTIL_OBJECT_POOL_H
-#define KUDU_UTIL_OBJECT_POOL_H
+#pragma once
+
+#include <cstdint>
+#include <memory>
 
 #include <glog/logging.h>
-#include <stdint.h>
+
 #include "kudu/gutil/manual_constructor.h"
-#include "kudu/gutil/gscoped_ptr.h"
 
 namespace kudu {
 
@@ -46,7 +47,7 @@ template<typename T>
 class ObjectPool {
  public:
   typedef ReturnToPool<T> deleter_type;
-  typedef gscoped_ptr<T, deleter_type> scoped_ptr;
+  typedef std::unique_ptr<T, deleter_type> scoped_ptr;
 
   ObjectPool() :
     free_list_head_(NULL),
@@ -163,4 +164,3 @@ class ReturnToPool {
 
 
 } // namespace kudu
-#endif
diff --git a/src/kudu/util/path_util.cc b/src/kudu/util/path_util.cc
index 6d1c4a5..7ee27fc 100644
--- a/src/kudu/util/path_util.cc
+++ b/src/kudu/util/path_util.cc
@@ -20,7 +20,9 @@
 // Use the POSIX version of dirname(3).
 #include <libgen.h>
 
+#include <cstdlib>
 #include <cstring>
+#include <memory>
 #if defined(__APPLE__)
 #include <mutex>
 #endif // defined(__APPLE__)
@@ -29,7 +31,6 @@
 
 #include <glog/logging.h>
 
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/strings/split.h"
 #include "kudu/gutil/strings/stringpiece.h"
 #include "kudu/gutil/strings/strip.h"
@@ -37,14 +38,24 @@
 #include "kudu/util/status.h"
 #include "kudu/util/subprocess.h"
 
-
 using std::string;
+using std::unique_ptr;
 using std::vector;
 using strings::SkipEmpty;
 using strings::Split;
 
 namespace kudu {
 
+namespace {
+
+struct FreeDeleter {
+  inline void operator()(void* ptr) const {
+    free(ptr);
+  }
+};
+
+} // anonymous namespace
+
 const char kTmpInfix[] = ".kudutmp";
 const char kOldTmpInfix[] = ".tmp";
 
@@ -80,7 +91,7 @@ vector<string> SplitPath(const string& path) {
 }
 
 string DirName(const string& path) {
-  gscoped_ptr<char[], FreeDeleter> path_copy(strdup(path.c_str()));
+  unique_ptr<char[], FreeDeleter> path_copy(strdup(path.c_str()));
 #if defined(__APPLE__)
   static std::mutex lock;
   std::lock_guard<std::mutex> l(lock);
@@ -89,7 +100,7 @@ string DirName(const string& path) {
 }
 
 string BaseName(const string& path) {
-  gscoped_ptr<char[], FreeDeleter> path_copy(strdup(path.c_str()));
+  unique_ptr<char[], FreeDeleter> path_copy(strdup(path.c_str()));
   return basename(path_copy.get());
 }
 
diff --git a/src/kudu/util/path_util.h b/src/kudu/util/path_util.h
index 58211a9..fd1b21f 100644
--- a/src/kudu/util/path_util.h
+++ b/src/kudu/util/path_util.h
@@ -16,8 +16,7 @@
 // under the License.
 //
 // Utility methods for dealing with file paths.
-#ifndef KUDU_UTIL_PATH_UTIL_H
-#define KUDU_UTIL_PATH_UTIL_H
+#pragma once
 
 #include <string>
 #include <vector>
@@ -60,4 +59,3 @@ Status FindExecutable(const std::string& binary,
                       std::string* path) WARN_UNUSED_RESULT;
 
 } // namespace kudu
-#endif /* KUDU_UTIL_PATH_UTIL_H */
diff --git a/src/kudu/util/protoc-gen-insertions.cc b/src/kudu/util/protoc-gen-insertions.cc
index 5d1097e..53c87d2 100644
--- a/src/kudu/util/protoc-gen-insertions.cc
+++ b/src/kudu/util/protoc-gen-insertions.cc
@@ -19,6 +19,7 @@
 // Currently, this just adds an include of protobuf-annotations.h, a file which hooks up
 // the protobuf concurrency annotations to our TSAN annotations.
 
+#include <memory>
 #include <string>
 
 #include <google/protobuf/compiler/code_generator.h>
@@ -27,7 +28,6 @@
 #include <google/protobuf/io/printer.h>
 #include <google/protobuf/io/zero_copy_stream.h>
 
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/strings/strip.h"
 #include "kudu/gutil/strings/substitute.h"
@@ -35,6 +35,7 @@
 using google::protobuf::io::ZeroCopyOutputStream;
 using google::protobuf::io::Printer;
 using std::string;
+using std::unique_ptr;
 
 namespace kudu {
 
@@ -50,13 +51,15 @@ class InsertAnnotations : public ::google::protobuf::compiler::CodeGenerator {
     // Determine the file name we will substitute into.
     string path_no_extension;
     if (!TryStripSuffixString(file->name(), kProtoExtension, &path_no_extension)) {
-      *error = strings::Substitute("file name $0 did not end in $1", file->name(), kProtoExtension);
+      *error = strings::Substitute("file name $0 did not end in $1",
+                                   file->name(), kProtoExtension);
       return false;
     }
     string pb_file = path_no_extension + ".pb.cc";
 
     // Actually insert the new #include
-    gscoped_ptr<ZeroCopyOutputStream> inserter(gen_context->OpenForInsert(pb_file, "includes"));
+    unique_ptr<ZeroCopyOutputStream> inserter(
+        gen_context->OpenForInsert(pb_file, "includes"));
     Printer printer(inserter.get(), '$');
     printer.Print(kIncludeToInsert);
 
diff --git a/src/kudu/util/trace.h b/src/kudu/util/trace.h
index 61e3b13..27d56dd 100644
--- a/src/kudu/util/trace.h
+++ b/src/kudu/util/trace.h
@@ -25,7 +25,6 @@
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/strings/stringpiece.h"
 #include "kudu/gutil/strings/substitute.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/threading/thread_collision_warner.h"
 #include "kudu/gutil/walltime.h"
diff --git a/src/kudu/util/user.cc b/src/kudu/util/user.cc
index e3c7fd4..34347fe 100644
--- a/src/kudu/util/user.cc
+++ b/src/kudu/util/user.cc
@@ -18,12 +18,12 @@
 #include "kudu/util/user.h"
 
 #include <pwd.h>
-#include <string.h>
 #include <unistd.h>
 
-#include <cerrno>
 #include <cstdint>
 #include <cstdlib>
+#include <cstring>
+#include <memory>
 #include <mutex>
 #include <ostream>
 #include <string>
@@ -31,12 +31,12 @@
 
 #include <glog/logging.h>
 
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/util/debug/leakcheck_disabler.h"
 #include "kudu/util/errno.h"
 #include "kudu/util/status.h"
 
 using std::string;
+using std::unique_ptr;
 
 namespace kudu {
 namespace {
@@ -59,11 +59,7 @@ Status DoGetLoggedInUser(string* user_name) {
   int64_t retval = sysconf(_SC_GETPW_R_SIZE_MAX);
   size_t bufsize = retval > 0 ? retval : 16384;
 
-  gscoped_ptr<char[], FreeDeleter> buf(static_cast<char *>(malloc(bufsize)));
-  if (buf.get() == nullptr) {
-    return Status::RuntimeError("malloc failed", ErrnoToString(errno), errno);
-  }
-
+  unique_ptr<char[]> buf(new char[bufsize]);
   int ret = getpwuid_r(getuid(), &pwd, buf.get(), bufsize, &result);
   if (result == nullptr) {
     if (ret == 0) {