You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by da...@apache.org on 2016/06/02 00:52:54 UTC

[2/5] incubator-kudu git commit: Replace boost::{lock, unique_lock, mutex} with std lib equivalents

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/tablet/mvcc.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/mvcc.cc b/src/kudu/tablet/mvcc.cc
index f972379..2ce0237 100644
--- a/src/kudu/tablet/mvcc.cc
+++ b/src/kudu/tablet/mvcc.cc
@@ -15,10 +15,11 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "kudu/tablet/mvcc.h"
+
 #include <algorithm>
-#include <boost/thread/locks.hpp>
-#include <boost/thread/mutex.hpp>
 #include <glog/logging.h>
+#include <mutex>
 
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/mathlimits.h"
@@ -27,7 +28,6 @@
 #include "kudu/gutil/strings/strcat.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/server/logical_clock.h"
-#include "kudu/tablet/mvcc.h"
 #include "kudu/util/countdown_latch.h"
 #include "kudu/util/debug/trace_event.h"
 #include "kudu/util/stopwatch.h"
@@ -45,7 +45,7 @@ MvccManager::MvccManager(const scoped_refptr<server::Clock>& clock)
 Timestamp MvccManager::StartTransaction() {
   while (true) {
     Timestamp now = clock_->Now();
-    boost::lock_guard<LockType> l(lock_);
+    std::lock_guard<LockType> l(lock_);
     if (PREDICT_TRUE(InitTransactionUnlocked(now))) {
       return now;
     }
@@ -56,7 +56,7 @@ Timestamp MvccManager::StartTransaction() {
 }
 
 Timestamp MvccManager::StartTransactionAtLatest() {
-  boost::lock_guard<LockType> l(lock_);
+  std::lock_guard<LockType> l(lock_);
   Timestamp now_latest = clock_->NowLatest();
   while (PREDICT_FALSE(!InitTransactionUnlocked(now_latest))) {
     now_latest = clock_->NowLatest();
@@ -76,7 +76,7 @@ Timestamp MvccManager::StartTransactionAtLatest() {
 }
 
 Status MvccManager::StartTransactionAtTimestamp(Timestamp timestamp) {
-  boost::lock_guard<LockType> l(lock_);
+  std::lock_guard<LockType> l(lock_);
   if (PREDICT_FALSE(cur_snap_.IsCommitted(timestamp))) {
     return Status::IllegalState(
         strings::Substitute("Timestamp: $0 is already committed. Current Snapshot: $1",
@@ -91,7 +91,7 @@ Status MvccManager::StartTransactionAtTimestamp(Timestamp timestamp) {
 }
 
 void MvccManager::StartApplyingTransaction(Timestamp timestamp) {
-  boost::lock_guard<LockType> l(lock_);
+  std::lock_guard<LockType> l(lock_);
   auto it = timestamps_in_flight_.find(timestamp.value());
   if (PREDICT_FALSE(it == timestamps_in_flight_.end())) {
     LOG(FATAL) << "Cannot mark timestamp " << timestamp.ToString() << " as APPLYING: "
@@ -131,7 +131,7 @@ bool MvccManager::InitTransactionUnlocked(const Timestamp& timestamp) {
 }
 
 void MvccManager::CommitTransaction(Timestamp timestamp) {
-  boost::lock_guard<LockType> l(lock_);
+  std::lock_guard<LockType> l(lock_);
   bool was_earliest = false;
   CommitTransactionUnlocked(timestamp, &was_earliest);
 
@@ -149,7 +149,7 @@ void MvccManager::CommitTransaction(Timestamp timestamp) {
 }
 
 void MvccManager::AbortTransaction(Timestamp timestamp) {
-  boost::lock_guard<LockType> l(lock_);
+  std::lock_guard<LockType> l(lock_);
 
   // Remove from our in-flight list.
   TxnState old_state = RemoveInFlightAndGetStateUnlocked(timestamp);
@@ -164,7 +164,7 @@ void MvccManager::AbortTransaction(Timestamp timestamp) {
 }
 
 void MvccManager::OfflineCommitTransaction(Timestamp timestamp) {
-  boost::lock_guard<LockType> l(lock_);
+  std::lock_guard<LockType> l(lock_);
 
   // Commit the transaction, but do not adjust 'all_committed_before_', that will
   // be done with a separate OfflineAdjustCurSnap() call.
@@ -226,7 +226,7 @@ void MvccManager::AdvanceEarliestInFlightTimestamp() {
 }
 
 void MvccManager::OfflineAdjustSafeTime(Timestamp safe_time) {
-  boost::lock_guard<LockType> l(lock_);
+  std::lock_guard<LockType> l(lock_);
 
   // No more transactions will start with a ts that is lower than or equal
   // to 'safe_time', so we adjust the snapshot accordingly.
@@ -302,7 +302,7 @@ Status MvccManager::WaitUntil(WaitFor wait_for, Timestamp ts,
     waiting_state.latch = &latch;
     waiting_state.wait_for = wait_for;
 
-    boost::lock_guard<LockType> l(lock_);
+    std::lock_guard<LockType> l(lock_);
     if (IsDoneWaitingUnlocked(waiting_state)) return Status::OK();
     waiters_.push_back(&waiting_state);
   }
@@ -311,7 +311,7 @@ Status MvccManager::WaitUntil(WaitFor wait_for, Timestamp ts,
   }
   // We timed out. We need to clean up our entry in the waiters_ array.
 
-  boost::lock_guard<LockType> l(lock_);
+  std::lock_guard<LockType> l(lock_);
   // It's possible that while we were re-acquiring the lock, we did get
   // notified. In that case, we have no cleanup to do.
   if (waiting_state.latch->count() == 0) {
@@ -355,7 +355,7 @@ bool MvccManager::AnyApplyingAtOrBeforeUnlocked(Timestamp ts) const {
 }
 
 void MvccManager::TakeSnapshot(MvccSnapshot *snap) const {
-  boost::lock_guard<LockType> l(lock_);
+  std::lock_guard<LockType> l(lock_);
   *snap = cur_snap_;
 }
 
@@ -379,7 +379,7 @@ void MvccManager::WaitForApplyingTransactionsToCommit() const {
   // Find the highest timestamp of an APPLYING transaction.
   Timestamp wait_for = Timestamp::kMin;
   {
-    boost::lock_guard<LockType> l(lock_);
+    std::lock_guard<LockType> l(lock_);
     for (const InFlightMap::value_type entry : timestamps_in_flight_) {
       if (entry.second == APPLYING) {
         wait_for = Timestamp(std::max(entry.first, wait_for.value()));
@@ -399,22 +399,22 @@ void MvccManager::WaitForApplyingTransactionsToCommit() const {
 }
 
 bool MvccManager::AreAllTransactionsCommitted(Timestamp ts) const {
-  boost::lock_guard<LockType> l(lock_);
+  std::lock_guard<LockType> l(lock_);
   return AreAllTransactionsCommittedUnlocked(ts);
 }
 
 int MvccManager::CountTransactionsInFlight() const {
-  boost::lock_guard<LockType> l(lock_);
+  std::lock_guard<LockType> l(lock_);
   return timestamps_in_flight_.size();
 }
 
 Timestamp MvccManager::GetCleanTimestamp() const {
-  boost::lock_guard<LockType> l(lock_);
+  std::lock_guard<LockType> l(lock_);
   return cur_snap_.all_committed_before_;
 }
 
 void MvccManager::GetApplyingTransactionsTimestamps(std::vector<Timestamp>* timestamps) const {
-  boost::lock_guard<LockType> l(lock_);
+  std::lock_guard<LockType> l(lock_);
   timestamps->reserve(timestamps_in_flight_.size());
   for (const InFlightMap::value_type entry : timestamps_in_flight_) {
     if (entry.second == APPLYING) {

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/tablet/rowset.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/rowset.h b/src/kudu/tablet/rowset.h
index 4890410..6e55bf0 100644
--- a/src/kudu/tablet/rowset.h
+++ b/src/kudu/tablet/rowset.h
@@ -17,8 +17,8 @@
 #ifndef KUDU_TABLET_ROWSET_H
 #define KUDU_TABLET_ROWSET_H
 
-#include <boost/thread/mutex.hpp>
 #include <memory>
+#include <mutex>
 #include <string>
 #include <vector>
 
@@ -119,7 +119,7 @@ class RowSet {
   // Return the lock used for including this DiskRowSet in a compaction.
   // This prevents multiple compactions and flushes from trying to include
   // the same rowset.
-  virtual boost::mutex *compact_flush_lock() = 0;
+  virtual std::mutex *compact_flush_lock() = 0;
 
   // Returns the metadata associated with this rowset.
   virtual std::shared_ptr<RowSetMetadata> metadata() = 0;
@@ -157,7 +157,7 @@ class RowSet {
     // the compaction selection has finished because only one thread
     // makes compaction selection at a time on a given Tablet due to
     // Tablet::compact_select_lock_.
-    boost::mutex::scoped_try_lock try_lock(*compact_flush_lock());
+    std::unique_lock<std::mutex> try_lock(*compact_flush_lock(), std::try_to_lock);
     return try_lock.owns_lock();
   }
 
@@ -283,7 +283,7 @@ class DuplicatingRowSet : public RowSet {
   std::shared_ptr<RowSetMetadata> metadata() OVERRIDE;
 
   // A flush-in-progress rowset should never be selected for compaction.
-  boost::mutex *compact_flush_lock() OVERRIDE {
+  std::mutex *compact_flush_lock() OVERRIDE {
     LOG(FATAL) << "Cannot be compacted";
     return NULL;
   }

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/tablet/tablet.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet.cc b/src/kudu/tablet/tablet.cc
index d12343c..59873c1 100644
--- a/src/kudu/tablet/tablet.cc
+++ b/src/kudu/tablet/tablet.cc
@@ -15,13 +15,14 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "kudu/tablet/tablet.h"
+
 #include <algorithm>
-#include <boost/bind.hpp>
-#include <boost/thread/locks.hpp>
-#include <boost/thread/mutex.hpp>
+#include <boost/thread/shared_mutex.hpp>
 #include <iterator>
 #include <limits>
 #include <memory>
+#include <mutex>
 #include <ostream>
 #include <unordered_set>
 #include <utility>
@@ -50,7 +51,6 @@
 #include "kudu/tablet/rowset_info.h"
 #include "kudu/tablet/rowset_tree.h"
 #include "kudu/tablet/svg_dump.h"
-#include "kudu/tablet/tablet.h"
 #include "kudu/tablet/tablet_metrics.h"
 #include "kudu/tablet/tablet_mm_ops.h"
 #include "kudu/tablet/transactions/alter_schema_transaction.h"
@@ -204,7 +204,7 @@ Tablet::~Tablet() {
 
 Status Tablet::Open() {
   TRACE_EVENT0("tablet", "Tablet::Open");
-  boost::lock_guard<rw_spinlock> lock(component_lock_);
+  std::lock_guard<rw_spinlock> lock(component_lock_);
   CHECK_EQ(state_, kInitialized) << "already open";
   CHECK(schema()->has_column_ids());
 
@@ -245,7 +245,7 @@ void Tablet::MarkFinishedBootstrapping() {
 void Tablet::Shutdown() {
   UnregisterMaintenanceOps();
 
-  boost::lock_guard<rw_spinlock> lock(component_lock_);
+  std::lock_guard<rw_spinlock> lock(component_lock_);
   components_ = nullptr;
   state_ = kShutdown;
 
@@ -665,7 +665,7 @@ void Tablet::ModifyRowSetTree(const RowSetTree& old_tree,
 
 void Tablet::AtomicSwapRowSets(const RowSetVector &old_rowsets,
                                const RowSetVector &new_rowsets) {
-  boost::lock_guard<rw_spinlock> lock(component_lock_);
+  std::lock_guard<rw_spinlock> lock(component_lock_);
   AtomicSwapRowSetsUnlocked(old_rowsets, new_rowsets);
 }
 
@@ -690,7 +690,7 @@ Status Tablet::DoMajorDeltaCompaction(const vector<ColumnId>& col_ids,
 
 Status Tablet::Flush() {
   TRACE_EVENT1("tablet", "Tablet::Flush", "id", tablet_id());
-  boost::lock_guard<Semaphore> lock(rowsets_flush_sem_);
+  std::lock_guard<Semaphore> lock(rowsets_flush_sem_);
   return FlushUnlocked();
 }
 
@@ -700,7 +700,7 @@ Status Tablet::FlushUnlocked() {
   shared_ptr<MemRowSet> old_mrs;
   {
     // Create a new MRS with the latest schema.
-    boost::lock_guard<rw_spinlock> lock(component_lock_);
+    std::lock_guard<rw_spinlock> lock(component_lock_);
     RETURN_NOT_OK(ReplaceMemRowSetUnlocked(&input, &old_mrs));
   }
 
@@ -717,12 +717,11 @@ Status Tablet::ReplaceMemRowSetUnlocked(RowSetsInCompaction *compaction,
   *old_ms = components_->memrowset;
   // Mark the memrowset rowset as locked, so compactions won't consider it
   // for inclusion in any concurrent compactions.
-  shared_ptr<boost::mutex::scoped_try_lock> ms_lock(
-    new boost::mutex::scoped_try_lock(*((*old_ms)->compact_flush_lock())));
-  CHECK(ms_lock->owns_lock());
+  std::unique_lock<std::mutex> ms_lock(*(*old_ms)->compact_flush_lock(), std::try_to_lock);
+  CHECK(ms_lock.owns_lock());
 
   // Add to compaction.
-  compaction->AddRowSet(*old_ms, ms_lock);
+  compaction->AddRowSet(*old_ms, std::move(ms_lock));
 
   shared_ptr<MemRowSet> new_mrs(new MemRowSet(next_mrs_id_++, *schema(), log_anchor_registry_.get(),
                                 mem_tracker_));
@@ -813,7 +812,7 @@ Status Tablet::AlterSchema(AlterSchemaTransactionState *tx_state) {
   // Prevent any concurrent flushes. Otherwise, we run into issues where
   // we have an MRS in the rowset tree, and we can't alter its schema
   // in-place.
-  boost::lock_guard<Semaphore> lock(rowsets_flush_sem_);
+  std::lock_guard<Semaphore> lock(rowsets_flush_sem_);
 
   // If the current version >= new version, there is nothing to do.
   bool same_schema = schema()->Equals(*tx_state->schema());
@@ -856,7 +855,7 @@ Status Tablet::RewindSchemaForBootstrap(const Schema& new_schema,
 
   metadata_->SetSchema(new_schema, schema_version);
   {
-    boost::lock_guard<rw_spinlock> lock(component_lock_);
+    std::lock_guard<rw_spinlock> lock(component_lock_);
 
     shared_ptr<MemRowSet> old_mrs = components_->memrowset;
     shared_ptr<RowSetTree> old_rowsets = components_->rowsets;
@@ -914,7 +913,7 @@ CompactRowSetsOp::CompactRowSetsOp(Tablet* tablet)
 }
 
 void CompactRowSetsOp::UpdateStats(MaintenanceOpStats* stats) {
-  boost::lock_guard<simple_spinlock> l(lock_);
+  std::lock_guard<simple_spinlock> l(lock_);
 
   // Any operation that changes the on-disk row layout invalidates the
   // cached stats.
@@ -938,7 +937,7 @@ void CompactRowSetsOp::UpdateStats(MaintenanceOpStats* stats) {
 }
 
 bool CompactRowSetsOp::Prepare() {
-  boost::lock_guard<simple_spinlock> l(lock_);
+  std::lock_guard<simple_spinlock> l(lock_);
   // Invalidate the cached stats so that another section of the tablet can
   // be compacted concurrently.
   //
@@ -978,7 +977,7 @@ MinorDeltaCompactionOp::MinorDeltaCompactionOp(Tablet* tablet)
 }
 
 void MinorDeltaCompactionOp::UpdateStats(MaintenanceOpStats* stats) {
-  boost::lock_guard<simple_spinlock> l(lock_);
+  std::lock_guard<simple_spinlock> l(lock_);
 
   // Any operation that changes the number of REDO files invalidates the
   // cached stats.
@@ -1012,7 +1011,7 @@ void MinorDeltaCompactionOp::UpdateStats(MaintenanceOpStats* stats) {
 }
 
 bool MinorDeltaCompactionOp::Prepare() {
-  boost::lock_guard<simple_spinlock> l(lock_);
+  std::lock_guard<simple_spinlock> l(lock_);
   // Invalidate the cached stats so that another rowset in the tablet can
   // be delta compacted concurrently.
   //
@@ -1050,7 +1049,7 @@ MajorDeltaCompactionOp::MajorDeltaCompactionOp(Tablet* tablet)
 }
 
 void MajorDeltaCompactionOp::UpdateStats(MaintenanceOpStats* stats) {
-  boost::lock_guard<simple_spinlock> l(lock_);
+  std::lock_guard<simple_spinlock> l(lock_);
 
   // Any operation that changes the size of the on-disk data invalidates the
   // cached stats.
@@ -1088,7 +1087,7 @@ void MajorDeltaCompactionOp::UpdateStats(MaintenanceOpStats* stats) {
 }
 
 bool MajorDeltaCompactionOp::Prepare() {
-  boost::lock_guard<simple_spinlock> l(lock_);
+  std::lock_guard<simple_spinlock> l(lock_);
   // Invalidate the cached stats so that another rowset in the tablet can
   // be delta compacted concurrently.
   //
@@ -1126,7 +1125,7 @@ Status Tablet::PickRowSetsToCompact(RowSetsInCompaction *picked,
     rowsets_copy = components_->rowsets;
   }
 
-  boost::lock_guard<boost::mutex> compact_lock(compact_select_lock_);
+  std::lock_guard<std::mutex> compact_lock(compact_select_lock_);
   CHECK_EQ(picked->num_rowsets(), 0);
 
   unordered_set<RowSet*> picked_set;
@@ -1156,14 +1155,13 @@ Status Tablet::PickRowSetsToCompact(RowSetsInCompaction *picked,
     // compaction from selecting this same rowset, and also ensures that
     // we don't select a rowset which is currently in the middle of being
     // flushed.
-    shared_ptr<boost::mutex::scoped_try_lock> lock(
-      new boost::mutex::scoped_try_lock(*rs->compact_flush_lock()));
-    CHECK(lock->owns_lock()) << rs->ToString() << " appeared available for "
+    std::unique_lock<std::mutex> lock(*rs->compact_flush_lock(), std::try_to_lock);
+    CHECK(lock.owns_lock()) << rs->ToString() << " appeared available for "
       "compaction when inputs were selected, but was unable to lock its "
       "compact_flush_lock to prepare for compaction.";
 
     // Push the lock on our scoped list, so we unlock when done.
-    picked->AddRowSet(rs, lock);
+    picked->AddRowSet(rs, std::move(lock));
   }
 
   // When we iterated through the current rowsets, we should have found all of the
@@ -1332,7 +1330,7 @@ Status Tablet::DoCompactionOrFlush(const RowSetsInCompaction &input, int64_t mrs
     TRACE_EVENT0("tablet", "Swapping DuplicatingRowSet");
     // Taking component_lock_ in write mode ensures that no new transactions
     // can StartApplying() (or snapshot components_) during this block.
-    boost::lock_guard<rw_spinlock> lock(component_lock_);
+    std::lock_guard<rw_spinlock> lock(component_lock_);
     AtomicSwapRowSetsUnlocked(input.rowsets(), { inprogress_rowset });
 
     // NOTE: transactions may *commit* in between these two lines.
@@ -1478,7 +1476,7 @@ void Tablet::UpdateCompactionStats(MaintenanceOpStats* stats) {
   }
 
   {
-    boost::lock_guard<boost::mutex> compact_lock(compact_select_lock_);
+    std::lock_guard<std::mutex> compact_lock(compact_select_lock_);
     WARN_NOT_OK(compaction_policy_->PickRowSets(*rowsets_copy, &picked_set_ignored, &quality, NULL),
                 Substitute("Couldn't determine compaction quality for $0", tablet_id()));
   }
@@ -1723,17 +1721,17 @@ Status Tablet::CompactWorstDeltas(RowSet::DeltaCompactionType type) {
   CHECK_EQ(state_, kOpen);
   shared_ptr<RowSet> rs;
   // We're required to grab the rowset's compact_flush_lock under the compact_select_lock_.
-  shared_ptr<boost::mutex::scoped_try_lock> lock;
+  std::unique_lock<std::mutex> lock;
   double perf_improv;
   {
     // We only want to keep the selection lock during the time we look at rowsets to compact.
     // The returned rowset is guaranteed to be available to lock since locking must be done
     // under this lock.
-    boost::lock_guard<boost::mutex> compact_lock(compact_select_lock_);
+    std::lock_guard<std::mutex> compact_lock(compact_select_lock_);
     perf_improv = GetPerfImprovementForBestDeltaCompactUnlocked(type, &rs);
     if (rs) {
-      lock.reset(new boost::mutex::scoped_try_lock(*rs->compact_flush_lock()));
-      CHECK(lock->owns_lock());
+      lock = std::unique_lock<std::mutex>(*rs->compact_flush_lock(), std::try_to_lock);
+      CHECK(lock.owns_lock());
     } else {
       return Status::OK();
     }
@@ -1754,13 +1752,13 @@ Status Tablet::CompactWorstDeltas(RowSet::DeltaCompactionType type) {
 
 double Tablet::GetPerfImprovementForBestDeltaCompact(RowSet::DeltaCompactionType type,
                                                              shared_ptr<RowSet>* rs) const {
-  boost::lock_guard<boost::mutex> compact_lock(compact_select_lock_);
+  std::lock_guard<std::mutex> compact_lock(compact_select_lock_);
   return GetPerfImprovementForBestDeltaCompactUnlocked(type, rs);
 }
 
 double Tablet::GetPerfImprovementForBestDeltaCompactUnlocked(RowSet::DeltaCompactionType type,
                                                              shared_ptr<RowSet>* rs) const {
-  boost::mutex::scoped_try_lock cs_lock(compact_select_lock_);
+  std::unique_lock<std::mutex> cs_lock(compact_select_lock_, std::try_to_lock);
   DCHECK(!cs_lock.owns_lock());
   scoped_refptr<TabletComponents> comps;
   GetComponents(&comps);
@@ -1793,7 +1791,7 @@ void Tablet::PrintRSLayout(ostream* o) {
     boost::shared_lock<rw_spinlock> lock(component_lock_);
     rowsets_copy = components_->rowsets;
   }
-  boost::lock_guard<boost::mutex> compact_lock(compact_select_lock_);
+  std::lock_guard<std::mutex> compact_lock(compact_select_lock_);
   // Run the compaction policy in order to get its log and highlight those
   // rowsets which would be compacted next.
   vector<string> log;

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/tablet/tablet.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet.h b/src/kudu/tablet/tablet.h
index 58175af..c5c666b 100644
--- a/src/kudu/tablet/tablet.h
+++ b/src/kudu/tablet/tablet.h
@@ -17,14 +17,14 @@
 #ifndef KUDU_TABLET_TABLET_H
 #define KUDU_TABLET_TABLET_H
 
+#include <boost/thread/shared_mutex.hpp>
 #include <iosfwd>
 #include <map>
 #include <memory>
+#include <mutex>
 #include <string>
 #include <vector>
 
-#include <boost/thread/shared_mutex.hpp>
-
 #include "kudu/common/iterator.h"
 #include "kudu/common/schema.h"
 #include "kudu/gutil/atomicops.h"
@@ -555,7 +555,7 @@ class Tablet {
   // Lock protecting the selection of rowsets for compaction.
   // Only one thread may run the compaction selection algorithm at a time
   // so that they don't both try to select the same rowset.
-  mutable boost::mutex compact_select_lock_;
+  mutable std::mutex compact_select_lock_;
 
   // We take this lock when flushing the tablet's rowsets in Tablet::Flush.  We
   // don't want to have two flushes in progress at once, in case the one which

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/tablet/tablet_bootstrap.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_bootstrap.cc b/src/kudu/tablet/tablet_bootstrap.cc
index ab4422a..6c26f65 100644
--- a/src/kudu/tablet/tablet_bootstrap.cc
+++ b/src/kudu/tablet/tablet_bootstrap.cc
@@ -72,7 +72,6 @@ DECLARE_int32(max_clock_sync_error_usec);
 namespace kudu {
 namespace tablet {
 
-using boost::shared_lock;
 using consensus::ALTER_SCHEMA_OP;
 using consensus::CHANGE_CONFIG_OP;
 using consensus::ChangeConfigRecordPB;
@@ -376,7 +375,7 @@ TabletStatusListener::~TabletStatusListener() {
 void TabletStatusListener::StatusMessage(const string& status) {
   LOG(INFO) << "T " << tablet_id() << " P " << meta_->fs_manager()->uuid() << ": "
             << status;
-  boost::lock_guard<boost::shared_mutex> l(lock_);
+  std::lock_guard<boost::shared_mutex> l(lock_);
   last_status_ = status;
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/tablet/tablet_metadata.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_metadata.cc b/src/kudu/tablet/tablet_metadata.cc
index ef4c8d2..e484471 100644
--- a/src/kudu/tablet/tablet_metadata.cc
+++ b/src/kudu/tablet/tablet_metadata.cc
@@ -18,9 +18,9 @@
 #include "kudu/tablet/tablet_metadata.h"
 
 #include <algorithm>
-#include <gflags/gflags.h>
 #include <boost/optional.hpp>
-#include <boost/thread/locks.hpp>
+#include <gflags/gflags.h>
+#include <mutex>
 #include <string>
 
 #include "kudu/common/wire_protocol.h"
@@ -164,7 +164,7 @@ Status TabletMetadata::DeleteTabletData(TabletDataState delete_type,
   // We also set the state in our persisted metadata to indicate that
   // we have been deleted.
   {
-    boost::lock_guard<LockType> l(data_lock_);
+    std::lock_guard<LockType> l(data_lock_);
     for (const shared_ptr<RowSetMetadata>& rsmd : rowsets_) {
       AddOrphanedBlocksUnlocked(rsmd->GetAllBlocks());
     }
@@ -187,7 +187,7 @@ Status TabletMetadata::DeleteTabletData(TabletDataState delete_type,
 }
 
 Status TabletMetadata::DeleteSuperBlock() {
-  boost::lock_guard<LockType> l(data_lock_);
+  std::lock_guard<LockType> l(data_lock_);
   if (!orphaned_blocks_.empty()) {
     return Status::InvalidArgument("The metadata for tablet " + tablet_id_ +
                                    " still references orphaned blocks. "
@@ -271,7 +271,7 @@ Status TabletMetadata::LoadFromSuperBlock(const TabletSuperBlockPB& superblock)
           << superblock.DebugString();
 
   {
-    boost::lock_guard<LockType> l(data_lock_);
+    std::lock_guard<LockType> l(data_lock_);
 
     // Verify that the tablet id matches with the one in the protobuf
     if (superblock.tablet_id() != tablet_id_) {
@@ -348,14 +348,14 @@ Status TabletMetadata::UpdateAndFlush(const RowSetMetadataIds& to_remove,
                                       const RowSetMetadataVector& to_add,
                                       int64_t last_durable_mrs_id) {
   {
-    boost::lock_guard<LockType> l(data_lock_);
+    std::lock_guard<LockType> l(data_lock_);
     RETURN_NOT_OK(UpdateUnlocked(to_remove, to_add, last_durable_mrs_id));
   }
   return Flush();
 }
 
 void TabletMetadata::AddOrphanedBlocks(const vector<BlockId>& blocks) {
-  boost::lock_guard<LockType> l(data_lock_);
+  std::lock_guard<LockType> l(data_lock_);
   AddOrphanedBlocksUnlocked(blocks);
 }
 
@@ -388,7 +388,7 @@ void TabletMetadata::DeleteOrphanedBlocks(const vector<BlockId>& blocks) {
 
   // Remove the successfully-deleted blocks from the set.
   {
-    boost::lock_guard<LockType> l(data_lock_);
+    std::lock_guard<LockType> l(data_lock_);
     for (const BlockId& b : deleted) {
       orphaned_blocks_.erase(b);
     }
@@ -396,14 +396,14 @@ void TabletMetadata::DeleteOrphanedBlocks(const vector<BlockId>& blocks) {
 }
 
 void TabletMetadata::PinFlush() {
-  boost::lock_guard<LockType> l(data_lock_);
+  std::lock_guard<LockType> l(data_lock_);
   CHECK_GE(num_flush_pins_, 0);
   num_flush_pins_++;
   VLOG(1) << "Number of flush pins: " << num_flush_pins_;
 }
 
 Status TabletMetadata::UnPinFlush() {
-  boost::unique_lock<LockType> l(data_lock_);
+  std::unique_lock<LockType> l(data_lock_);
   CHECK_GT(num_flush_pins_, 0);
   num_flush_pins_--;
   if (needs_flush_) {
@@ -421,7 +421,7 @@ Status TabletMetadata::Flush() {
   vector<BlockId> orphaned;
   TabletSuperBlockPB pb;
   {
-    boost::lock_guard<LockType> l(data_lock_);
+    std::lock_guard<LockType> l(data_lock_);
     CHECK_GE(num_flush_pins_, 0);
     if (num_flush_pins_ > 0) {
       needs_flush_ = true;
@@ -519,7 +519,7 @@ Status TabletMetadata::ReadSuperBlockFromDisk(TabletSuperBlockPB* superblock) co
 
 Status TabletMetadata::ToSuperBlock(TabletSuperBlockPB* super_block) const {
   // acquire the lock so that rowsets_ doesn't get changed until we're finished.
-  boost::lock_guard<LockType> l(data_lock_);
+  std::lock_guard<LockType> l(data_lock_);
   return ToSuperBlockUnlocked(super_block, rowsets_);
 }
 
@@ -576,7 +576,7 @@ const RowSetMetadata *TabletMetadata::GetRowSetForTests(int64_t id) const {
 }
 
 RowSetMetadata *TabletMetadata::GetRowSetForTests(int64_t id) {
-  boost::lock_guard<LockType> l(data_lock_);
+  std::lock_guard<LockType> l(data_lock_);
   for (const shared_ptr<RowSetMetadata>& rowset_meta : rowsets_) {
     if (rowset_meta->id() == id) {
       return rowset_meta.get();
@@ -587,7 +587,7 @@ RowSetMetadata *TabletMetadata::GetRowSetForTests(int64_t id) {
 
 void TabletMetadata::SetSchema(const Schema& schema, uint32_t version) {
   gscoped_ptr<Schema> new_schema(new Schema(schema));
-  boost::lock_guard<LockType> l(data_lock_);
+  std::lock_guard<LockType> l(data_lock_);
   SetSchemaUnlocked(std::move(new_schema), version);
 }
 
@@ -606,24 +606,24 @@ void TabletMetadata::SetSchemaUnlocked(gscoped_ptr<Schema> new_schema, uint32_t
 }
 
 void TabletMetadata::SetTableName(const string& table_name) {
-  boost::lock_guard<LockType> l(data_lock_);
+  std::lock_guard<LockType> l(data_lock_);
   table_name_ = table_name;
 }
 
 string TabletMetadata::table_name() const {
-  boost::lock_guard<LockType> l(data_lock_);
+  std::lock_guard<LockType> l(data_lock_);
   DCHECK_NE(state_, kNotLoadedYet);
   return table_name_;
 }
 
 uint32_t TabletMetadata::schema_version() const {
-  boost::lock_guard<LockType> l(data_lock_);
+  std::lock_guard<LockType> l(data_lock_);
   DCHECK_NE(state_, kNotLoadedYet);
   return schema_version_;
 }
 
 void TabletMetadata::set_tablet_data_state(TabletDataState state) {
-  boost::lock_guard<LockType> l(data_lock_);
+  std::lock_guard<LockType> l(data_lock_);
   tablet_data_state_ = state;
 }
 
@@ -632,7 +632,7 @@ string TabletMetadata::LogPrefix() const {
 }
 
 TabletDataState TabletMetadata::tablet_data_state() const {
-  boost::lock_guard<LockType> l(data_lock_);
+  std::lock_guard<LockType> l(data_lock_);
   return tablet_data_state_;
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/tablet/tablet_peer.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_peer.cc b/src/kudu/tablet/tablet_peer.cc
index 25a72d8..47aefc8 100644
--- a/src/kudu/tablet/tablet_peer.cc
+++ b/src/kudu/tablet/tablet_peer.cc
@@ -18,12 +18,12 @@
 #include "kudu/tablet/tablet_peer.h"
 
 #include <algorithm>
+#include <gflags/gflags.h>
+#include <mutex>
 #include <string>
 #include <utility>
 #include <vector>
 
-#include <gflags/gflags.h>
-
 #include "kudu/consensus/consensus.h"
 #include "kudu/consensus/consensus_meta.h"
 #include "kudu/consensus/local_consensus.h"
@@ -113,7 +113,7 @@ TabletPeer::TabletPeer(const scoped_refptr<TabletMetadata>& meta,
       mark_dirty_clbk_(std::move(mark_dirty_clbk)) {}
 
 TabletPeer::~TabletPeer() {
-  boost::lock_guard<simple_spinlock> lock(lock_);
+  std::lock_guard<simple_spinlock> lock(lock_);
   // We should either have called Shutdown(), or we should have never called
   // Init().
   CHECK(!tablet_)
@@ -139,7 +139,7 @@ Status TabletPeer::Init(const shared_ptr<Tablet>& tablet,
       METRIC_op_prepare_run_time.Instantiate(metric_entity));
 
   {
-    boost::lock_guard<simple_spinlock> lock(lock_);
+    std::lock_guard<simple_spinlock> lock(lock_);
     CHECK_EQ(BOOTSTRAPPING, state_);
     tablet_ = tablet;
     clock_ = clock;
@@ -197,7 +197,7 @@ Status TabletPeer::Start(const ConsensusBootstrapInfo& bootstrap_info) {
 
   RETURN_NOT_OK(consensus_->Start(bootstrap_info));
   {
-    boost::lock_guard<simple_spinlock> lock(lock_);
+    std::lock_guard<simple_spinlock> lock(lock_);
     CHECK_EQ(state_, BOOTSTRAPPING);
     state_ = RUNNING;
   }
@@ -261,7 +261,7 @@ void TabletPeer::Shutdown() {
 
   // Only mark the peer as SHUTDOWN when all other components have shut down.
   {
-    boost::lock_guard<simple_spinlock> lock(lock_);
+    std::lock_guard<simple_spinlock> lock(lock_);
     // Release mem tracker resources.
     consensus_.reset();
     tablet_.reset();
@@ -272,7 +272,7 @@ void TabletPeer::Shutdown() {
 void TabletPeer::WaitUntilShutdown() {
   while (true) {
     {
-      boost::lock_guard<simple_spinlock> lock(lock_);
+      std::lock_guard<simple_spinlock> lock(lock_);
       if (state_ == SHUTDOWN) {
         return;
       }
@@ -283,7 +283,7 @@ void TabletPeer::WaitUntilShutdown() {
 
 Status TabletPeer::CheckRunning() const {
   {
-    boost::lock_guard<simple_spinlock> lock(lock_);
+    std::lock_guard<simple_spinlock> lock(lock_);
     if (state_ != RUNNING) {
       return Status::IllegalState(Substitute("The tablet is not in a running state: $0",
                                              TabletStatePB_Name(state_)));
@@ -301,7 +301,7 @@ Status TabletPeer::WaitUntilConsensusRunning(const MonoDelta& timeout) {
     bool has_consensus = false;
     TabletStatePB cached_state;
     {
-      boost::lock_guard<simple_spinlock> lock(lock_);
+      std::lock_guard<simple_spinlock> lock(lock_);
       cached_state = state_;
       if (consensus_) {
         has_consensus = true; // consensus_ is a set-once object.
@@ -348,7 +348,7 @@ Status TabletPeer::SubmitAlterSchema(unique_ptr<AlterSchemaTransactionState> sta
 }
 
 void TabletPeer::GetTabletStatusPB(TabletStatusPB* status_pb_out) const {
-  boost::lock_guard<simple_spinlock> lock(lock_);
+  std::lock_guard<simple_spinlock> lock(lock_);
   DCHECK(status_pb_out != nullptr);
   DCHECK(status_listener_.get() != nullptr);
   status_pb_out->set_tablet_id(status_listener_->tablet_id());
@@ -378,7 +378,7 @@ Status TabletPeer::RunLogGC() {
 }
 
 string TabletPeer::HumanReadableState() const {
-  boost::lock_guard<simple_spinlock> lock(lock_);
+  std::lock_guard<simple_spinlock> lock(lock_);
   TabletDataState data_state = meta_->tablet_data_state();
   // If failed, any number of things could have gone wrong.
   if (state_ == FAILED) {
@@ -479,7 +479,7 @@ Status TabletPeer::GetGCableDataSize(int64_t* retention_size) const {
 
 Status TabletPeer::StartReplicaTransaction(const scoped_refptr<ConsensusRound>& round) {
   {
-    boost::lock_guard<simple_spinlock> lock(lock_);
+    std::lock_guard<simple_spinlock> lock(lock_);
     if (state_ != RUNNING && state_ != BOOTSTRAPPING) {
       return Status::IllegalState(TabletStatePB_Name(state_));
     }

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/tablet/tablet_peer.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_peer.h b/src/kudu/tablet/tablet_peer.h
index 25a9ec6..20f03e3 100644
--- a/src/kudu/tablet/tablet_peer.h
+++ b/src/kudu/tablet/tablet_peer.h
@@ -20,6 +20,7 @@
 
 #include <map>
 #include <memory>
+#include <mutex>
 #include <string>
 #include <vector>
 
@@ -122,27 +123,27 @@ class TabletPeer : public RefCountedThreadSafe<TabletPeer>,
       const scoped_refptr<consensus::ConsensusRound>& round) OVERRIDE;
 
   consensus::Consensus* consensus() {
-    boost::lock_guard<simple_spinlock> lock(lock_);
+    std::lock_guard<simple_spinlock> lock(lock_);
     return consensus_.get();
   }
 
   scoped_refptr<consensus::Consensus> shared_consensus() const {
-    boost::lock_guard<simple_spinlock> lock(lock_);
+    std::lock_guard<simple_spinlock> lock(lock_);
     return consensus_;
   }
 
   Tablet* tablet() const {
-    boost::lock_guard<simple_spinlock> lock(lock_);
+    std::lock_guard<simple_spinlock> lock(lock_);
     return tablet_.get();
   }
 
   std::shared_ptr<Tablet> shared_tablet() const {
-    boost::lock_guard<simple_spinlock> lock(lock_);
+    std::lock_guard<simple_spinlock> lock(lock_);
     return tablet_;
   }
 
   const TabletStatePB state() const {
-    boost::lock_guard<simple_spinlock> lock(lock_);
+    std::lock_guard<simple_spinlock> lock(lock_);
     return state_;
   }
 
@@ -160,7 +161,7 @@ class TabletPeer : public RefCountedThreadSafe<TabletPeer>,
 
   // Sets the tablet to a BOOTSTRAPPING state, indicating it is starting up.
   void SetBootstrapping() {
-    boost::lock_guard<simple_spinlock> lock(lock_);
+    std::lock_guard<simple_spinlock> lock(lock_);
     CHECK_EQ(NOT_STARTED, state_);
     state_ = BOOTSTRAPPING;
   }
@@ -168,14 +169,14 @@ class TabletPeer : public RefCountedThreadSafe<TabletPeer>,
   // sets the tablet state to FAILED additionally setting the error to the provided
   // one.
   void SetFailed(const Status& error) {
-    boost::lock_guard<simple_spinlock> lock(lock_);
+    std::lock_guard<simple_spinlock> lock(lock_);
     state_ = FAILED;
     error_ = error;
   }
 
   // Returns the error that occurred, when state is FAILED.
   Status error() const {
-    boost::lock_guard<simple_spinlock> lock(lock_);
+    std::lock_guard<simple_spinlock> lock(lock_);
     return error_;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/tablet/tablet_peer_mm_ops.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_peer_mm_ops.cc b/src/kudu/tablet/tablet_peer_mm_ops.cc
index c8bc920..a1fc4df 100644
--- a/src/kudu/tablet/tablet_peer_mm_ops.cc
+++ b/src/kudu/tablet/tablet_peer_mm_ops.cc
@@ -18,11 +18,11 @@
 #include "kudu/tablet/tablet_peer_mm_ops.h"
 
 #include <algorithm>
+#include <gflags/gflags.h>
 #include <map>
+#include <mutex>
 #include <string>
 
-#include <gflags/gflags.h>
-
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/tablet/maintenance_manager.h"
 #include "kudu/tablet/tablet_metrics.h"
@@ -90,7 +90,7 @@ void FlushOpPerfImprovementPolicy::SetPerfImprovementForFlush(MaintenanceOpStats
 //
 
 void FlushMRSOp::UpdateStats(MaintenanceOpStats* stats) {
-  boost::lock_guard<simple_spinlock> l(lock_);
+  std::lock_guard<simple_spinlock> l(lock_);
 
   map<int64_t, int64_t> max_idx_to_segment_size;
   if (tablet_peer_->tablet()->MemRowSetEmpty() ||
@@ -99,8 +99,7 @@ void FlushMRSOp::UpdateStats(MaintenanceOpStats* stats) {
   }
 
   {
-    boost::unique_lock<Semaphore> lock(tablet_peer_->tablet()->rowsets_flush_sem_,
-                                       boost::defer_lock);
+    std::unique_lock<Semaphore> lock(tablet_peer_->tablet()->rowsets_flush_sem_, std::defer_lock);
     stats->set_runnable(lock.try_lock());
   }
 
@@ -129,7 +128,7 @@ void FlushMRSOp::Perform() {
                         Substitute("FlushMRS failed on $0", tablet_peer_->tablet_id()));
 
   {
-    boost::lock_guard<simple_spinlock> l(lock_);
+    std::lock_guard<simple_spinlock> l(lock_);
     time_since_flush_.start();
   }
   tablet_peer_->tablet()->rowsets_flush_sem_.unlock();
@@ -148,7 +147,7 @@ scoped_refptr<AtomicGauge<uint32_t> > FlushMRSOp::RunningGauge() const {
 //
 
 void FlushDeltaMemStoresOp::UpdateStats(MaintenanceOpStats* stats) {
-  boost::lock_guard<simple_spinlock> l(lock_);
+  std::lock_guard<simple_spinlock> l(lock_);
   int64_t dms_size;
   int64_t retention_size;
   map<int64_t, int64_t> max_idx_to_segment_size;
@@ -178,7 +177,7 @@ void FlushDeltaMemStoresOp::Perform() {
                   Substitute("Failed to flush DMS on $0",
                              tablet_peer_->tablet()->tablet_id()));
   {
-    boost::lock_guard<simple_spinlock> l(lock_);
+    std::lock_guard<simple_spinlock> l(lock_);
     time_since_flush_.start();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/tablet/transactions/transaction_driver.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/transactions/transaction_driver.cc b/src/kudu/tablet/transactions/transaction_driver.cc
index cf065cf..30ec10e 100644
--- a/src/kudu/tablet/transactions/transaction_driver.cc
+++ b/src/kudu/tablet/transactions/transaction_driver.cc
@@ -17,6 +17,8 @@
 
 #include "kudu/tablet/transactions/transaction_driver.h"
 
+#include <mutex>
+
 #include "kudu/consensus/consensus.h"
 #include "kudu/gutil/strings/strcat.h"
 #include "kudu/tablet/tablet_peer.h"
@@ -72,7 +74,7 @@ Status TransactionDriver::Init(gscoped_ptr<Transaction> transaction,
   transaction_ = std::move(transaction);
 
   if (type == consensus::REPLICA) {
-    boost::lock_guard<simple_spinlock> lock(opid_lock_);
+    std::lock_guard<simple_spinlock> lock(opid_lock_);
     op_id_copy_ = transaction_->state()->op_id();
     DCHECK(op_id_copy_.IsInitialized());
     replication_state_ = REPLICATING;
@@ -95,7 +97,7 @@ Status TransactionDriver::Init(gscoped_ptr<Transaction> transaction,
 }
 
 consensus::OpId TransactionDriver::GetOpId() {
-  boost::lock_guard<simple_spinlock> lock(opid_lock_);
+  std::lock_guard<simple_spinlock> lock(opid_lock_);
   return op_id_copy_;
 }
 
@@ -112,7 +114,7 @@ Transaction::TransactionType TransactionDriver::tx_type() const {
 }
 
 string TransactionDriver::ToString() const {
-  boost::lock_guard<simple_spinlock> lock(lock_);
+  std::lock_guard<simple_spinlock> lock(lock_);
   return ToStringUnlocked();
 }
 
@@ -176,7 +178,7 @@ Status TransactionDriver::PrepareAndStart() {
   // phase.
   ReplicationState repl_state_copy;
   {
-    boost::lock_guard<simple_spinlock> lock(lock_);
+    std::lock_guard<simple_spinlock> lock(lock_);
     CHECK_EQ(prepare_state_, NOT_PREPARED);
     prepare_state_ = PREPARED;
     repl_state_copy = replication_state_;
@@ -194,14 +196,14 @@ Status TransactionDriver::PrepareAndStart() {
       // Trigger the consensus replication.
 
       {
-        boost::lock_guard<simple_spinlock> lock(lock_);
+        std::lock_guard<simple_spinlock> lock(lock_);
         replication_state_ = REPLICATING;
         replication_start_time_ = MonoTime::Now(MonoTime::FINE);
       }
       Status s = consensus_->Replicate(mutable_state()->consensus_round());
 
       if (PREDICT_FALSE(!s.ok())) {
-        boost::lock_guard<simple_spinlock> lock(lock_);
+        std::lock_guard<simple_spinlock> lock(lock_);
         CHECK_EQ(replication_state_, REPLICATING);
         transaction_status_ = s;
         replication_state_ = REPLICATION_FAILED;
@@ -237,7 +239,7 @@ void TransactionDriver::HandleFailure(const Status& s) {
   ReplicationState repl_state_copy;
 
   {
-    boost::lock_guard<simple_spinlock> lock(lock_);
+    std::lock_guard<simple_spinlock> lock(lock_);
     transaction_status_ = s;
     repl_state_copy = replication_state_;
   }
@@ -271,7 +273,7 @@ void TransactionDriver::ReplicationFinished(const Status& status) {
 
   ADOPT_TRACE(trace());
   {
-    boost::lock_guard<simple_spinlock> op_id_lock(opid_lock_);
+    std::lock_guard<simple_spinlock> op_id_lock(opid_lock_);
     // TODO: it's a bit silly that we have three copies of the opid:
     // one here, one in ConsensusRound, and one in TransactionState.
 
@@ -283,7 +285,7 @@ void TransactionDriver::ReplicationFinished(const Status& status) {
   MonoDelta replication_duration;
   PrepareState prepare_state_copy;
   {
-    boost::lock_guard<simple_spinlock> lock(lock_);
+    std::lock_guard<simple_spinlock> lock(lock_);
     CHECK_EQ(replication_state_, REPLICATING);
     if (status.ok()) {
       replication_state_ = REPLICATED;
@@ -315,7 +317,7 @@ void TransactionDriver::Abort(const Status& status) {
 
   ReplicationState repl_state_copy;
   {
-    boost::lock_guard<simple_spinlock> lock(lock_);
+    std::lock_guard<simple_spinlock> lock(lock_);
     repl_state_copy = replication_state_;
     transaction_status_ = status;
   }
@@ -332,7 +334,7 @@ void TransactionDriver::Abort(const Status& status) {
 
 Status TransactionDriver::ApplyAsync() {
   {
-    boost::unique_lock<simple_spinlock> lock(lock_);
+    std::unique_lock<simple_spinlock> lock(lock_);
     DCHECK_EQ(prepare_state_, PREPARED);
     if (transaction_status_.ok()) {
       DCHECK_EQ(replication_state_, REPLICATED);
@@ -361,7 +363,7 @@ void TransactionDriver::ApplyTask() {
   ADOPT_TRACE(trace());
 
   {
-    boost::lock_guard<simple_spinlock> lock(lock_);
+    std::lock_guard<simple_spinlock> lock(lock_);
     DCHECK_EQ(replication_state_, REPLICATED);
     DCHECK_EQ(prepare_state_, PREPARED);
   }
@@ -427,7 +429,7 @@ void TransactionDriver::Finalize() {
   // TODO: this is an ugly hack so that the Release() call doesn't delete the
   // object while we still hold the lock.
   scoped_refptr<TransactionDriver> ref(this);
-  boost::lock_guard<simple_spinlock> lock(lock_);
+  std::lock_guard<simple_spinlock> lock(lock_);
   transaction_->Finish(Transaction::COMMITTED);
   mutable_state()->completion_callback()->TransactionCompleted();
   txn_tracker_->Release(this);
@@ -473,7 +475,7 @@ std::string TransactionDriver::LogPrefix() const {
   string ts_string;
 
   {
-    boost::lock_guard<simple_spinlock> lock(lock_);
+    std::lock_guard<simple_spinlock> lock(lock_);
     repl_state_copy = replication_state_;
     prep_state_copy = prepare_state_;
     ts_string = state()->has_timestamp() ? state()->timestamp().ToString() : "No timestamp";

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/tablet/transactions/write_transaction.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/transactions/write_transaction.cc b/src/kudu/tablet/transactions/write_transaction.cc
index fce343a..e7a6ba5 100644
--- a/src/kudu/tablet/transactions/write_transaction.cc
+++ b/src/kudu/tablet/transactions/write_transaction.cc
@@ -45,7 +45,6 @@ TAG_FLAG(tablet_inject_latency_on_apply_write_txn_ms, runtime);
 namespace kudu {
 namespace tablet {
 
-using boost::bind;
 using consensus::ReplicateMsg;
 using consensus::CommitMsg;
 using consensus::DriverType;

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/tserver/scanners.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/scanners.cc b/src/kudu/tserver/scanners.cc
index 11ac9b3..13c588f 100644
--- a/src/kudu/tserver/scanners.cc
+++ b/src/kudu/tserver/scanners.cc
@@ -17,8 +17,9 @@
 #include "kudu/tserver/scanners.h"
 
 #include <boost/bind.hpp>
-#include <boost/thread/locks.hpp>
+#include <boost/thread/shared_mutex.hpp>
 #include <gflags/gflags.h>
+#include <mutex>
 
 #include "kudu/common/iterator.h"
 #include "kudu/common/scan_spec.h"
@@ -116,7 +117,7 @@ void ScannerManager::NewScanner(const scoped_refptr<TabletPeer>& tablet_peer,
     scanner->reset(new Scanner(id, tablet_peer, requestor_string, metrics_.get()));
 
     ScannerMapStripe& stripe = GetStripeByScannerId(id);
-    boost::lock_guard<boost::shared_mutex> l(stripe.lock_);
+    std::lock_guard<boost::shared_mutex> l(stripe.lock_);
     success = InsertIfNotPresent(&stripe.scanners_by_id_, id, *scanner);
   }
 }
@@ -129,7 +130,7 @@ bool ScannerManager::LookupScanner(const string& scanner_id, SharedScanner* scan
 
 bool ScannerManager::UnregisterScanner(const string& scanner_id) {
   ScannerMapStripe& stripe = GetStripeByScannerId(scanner_id);
-  boost::lock_guard<boost::shared_mutex> l(stripe.lock_);
+  std::lock_guard<boost::shared_mutex> l(stripe.lock_);
   return stripe.scanners_by_id_.erase(scanner_id) > 0;
 }
 
@@ -155,7 +156,7 @@ void ScannerManager::RemoveExpiredScanners() {
   MonoDelta scanner_ttl = MonoDelta::FromMilliseconds(FLAGS_scanner_ttl_ms);
 
   for (ScannerMapStripe* stripe : scanner_maps_) {
-    boost::lock_guard<boost::shared_mutex> l(stripe->lock_);
+    std::lock_guard<boost::shared_mutex> l(stripe->lock_);
     for (auto it = stripe->scanners_by_id_.begin(); it != stripe->scanners_by_id_.end();) {
       SharedScanner& scanner = it->second;
       MonoDelta time_live =
@@ -197,13 +198,13 @@ Scanner::~Scanner() {
 }
 
 void Scanner::UpdateAccessTime() {
-  boost::lock_guard<simple_spinlock> l(lock_);
+  std::lock_guard<simple_spinlock> l(lock_);
   last_access_time_ = MonoTime::Now(MonoTime::COARSE);
 }
 
 void Scanner::Init(gscoped_ptr<RowwiseIterator> iter,
                    gscoped_ptr<ScanSpec> spec) {
-  boost::lock_guard<simple_spinlock> l(lock_);
+  std::lock_guard<simple_spinlock> l(lock_);
   CHECK(!iter_) << "Already initialized";
   iter_.reset(iter.release());
   spec_.reset(spec.release());

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/tserver/scanners.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/scanners.h b/src/kudu/tserver/scanners.h
index aeab8b1..e114623 100644
--- a/src/kudu/tserver/scanners.h
+++ b/src/kudu/tserver/scanners.h
@@ -19,6 +19,7 @@
 
 #include <boost/thread/shared_mutex.hpp>
 #include <memory>
+#include <mutex>
 #include <string>
 #include <unordered_map>
 #include <utility>
@@ -180,7 +181,7 @@ class Scanner {
   // Once a Scanner is initialized, it is safe to assume that iter() and spec()
   // return non-NULL for the lifetime of the Scanner object.
   bool IsInitialized() const {
-    boost::lock_guard<simple_spinlock> l(lock_);
+    std::lock_guard<simple_spinlock> l(lock_);
     return iter_ != NULL;
   }
 
@@ -224,19 +225,19 @@ class Scanner {
 
   // Returns the current call sequence ID of the scanner.
   uint32_t call_seq_id() const {
-    boost::lock_guard<simple_spinlock> l(lock_);
+    std::lock_guard<simple_spinlock> l(lock_);
     return call_seq_id_;
   }
 
   // Increments the call sequence ID.
   void IncrementCallSeqId() {
-    boost::lock_guard<simple_spinlock> l(lock_);
+    std::lock_guard<simple_spinlock> l(lock_);
     call_seq_id_ += 1;
   }
 
   // Return the delta from the last time this scan was updated to 'now'.
   MonoDelta TimeSinceLastAccess(const MonoTime& now) const {
-    boost::lock_guard<simple_spinlock> l(lock_);
+    std::lock_guard<simple_spinlock> l(lock_);
     return now.GetDeltaSince(last_access_time_);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/tserver/ts_tablet_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/ts_tablet_manager.cc b/src/kudu/tserver/ts_tablet_manager.cc
index 1276135..fcd46e7 100644
--- a/src/kudu/tserver/ts_tablet_manager.cc
+++ b/src/kudu/tserver/ts_tablet_manager.cc
@@ -18,12 +18,12 @@
 #include "kudu/tserver/ts_tablet_manager.h"
 
 #include <algorithm>
+#include <boost/bind.hpp>
 #include <boost/optional.hpp>
-#include <boost/thread/locks.hpp>
-#include <boost/thread/mutex.hpp>
 #include <boost/thread/shared_mutex.hpp>
 #include <glog/logging.h>
 #include <memory>
+#include <mutex>
 #include <string>
 #include <vector>
 
@@ -205,7 +205,7 @@ Status TSTabletManager::Init() {
   for (const scoped_refptr<TabletMetadata>& meta : metas) {
     scoped_refptr<TransitionInProgressDeleter> deleter;
     {
-      boost::lock_guard<rw_spinlock> lock(lock_);
+      std::lock_guard<rw_spinlock> lock(lock_);
       CHECK_OK(StartTabletStateTransitionUnlocked(meta->tablet_id(), "opening tablet", &deleter));
     }
 
@@ -215,7 +215,7 @@ Status TSTabletManager::Init() {
   }
 
   {
-    boost::lock_guard<rw_spinlock> lock(lock_);
+    std::lock_guard<rw_spinlock> lock(lock_);
     state_ = MANAGER_RUNNING;
   }
 
@@ -265,7 +265,7 @@ Status TSTabletManager::CreateNewTablet(const string& table_id,
   {
     // acquire the lock in exclusive mode as we'll add a entry to the
     // transition_in_progress_ set if the lookup fails.
-    boost::lock_guard<rw_spinlock> lock(lock_);
+    std::lock_guard<rw_spinlock> lock(lock_);
     TRACE("Acquired tablet manager lock");
 
     // Sanity check that the tablet isn't already registered.
@@ -354,7 +354,7 @@ Status TSTabletManager::StartRemoteBootstrap(
   bool replacing_tablet = false;
   scoped_refptr<TransitionInProgressDeleter> deleter;
   {
-    boost::lock_guard<rw_spinlock> lock(lock_);
+    std::lock_guard<rw_spinlock> lock(lock_);
     if (LookupTabletUnlocked(tablet_id, &old_tablet_peer)) {
       meta = old_tablet_peer->tablet_metadata();
       replacing_tablet = true;
@@ -488,7 +488,7 @@ Status TSTabletManager::DeleteTablet(
   {
     // Acquire the lock in exclusive mode as we'll add a entry to the
     // transition_in_progress_ map.
-    boost::lock_guard<rw_spinlock> lock(lock_);
+    std::lock_guard<rw_spinlock> lock(lock_);
     TRACE("Acquired tablet manager lock");
     RETURN_NOT_OK(CheckRunningUnlocked(error_code));
 
@@ -552,7 +552,7 @@ Status TSTabletManager::DeleteTablet(
 
   // We only remove DELETED tablets from the tablet map.
   if (delete_type == TABLET_DATA_DELETED) {
-    boost::lock_guard<rw_spinlock> lock(lock_);
+    std::lock_guard<rw_spinlock> lock(lock_);
     RETURN_NOT_OK(CheckRunningUnlocked(error_code));
     CHECK_EQ(1, tablet_map_.erase(tablet_id)) << tablet_id;
     InsertOrDie(&perm_deleted_tablet_ids_, tablet_id);
@@ -701,7 +701,7 @@ void TSTabletManager::OpenTablet(const scoped_refptr<TabletMetadata>& meta,
 
 void TSTabletManager::Shutdown() {
   {
-    boost::lock_guard<rw_spinlock> lock(lock_);
+    std::lock_guard<rw_spinlock> lock(lock_);
     switch (state_) {
       case MANAGER_QUIESCING: {
         VLOG(1) << "Tablet manager shut down already in progress..";
@@ -740,7 +740,7 @@ void TSTabletManager::Shutdown() {
   apply_pool_->Shutdown();
 
   {
-    boost::lock_guard<rw_spinlock> l(lock_);
+    std::lock_guard<rw_spinlock> l(lock_);
     // We don't expect anyone else to be modifying the map after we start the
     // shut down process.
     CHECK_EQ(tablet_map_.size(), peers_to_shutdown.size())
@@ -754,7 +754,7 @@ void TSTabletManager::Shutdown() {
 void TSTabletManager::RegisterTablet(const std::string& tablet_id,
                                      const scoped_refptr<TabletPeer>& tablet_peer,
                                      RegisterTabletPeerMode mode) {
-  boost::lock_guard<rw_spinlock> lock(lock_);
+  std::lock_guard<rw_spinlock> lock(lock_);
   // If we are replacing a tablet peer, we delete the existing one first.
   if (mode == REPLACEMENT_PEER && tablet_map_.erase(tablet_id) != 1) {
     LOG(FATAL) << "Unable to remove previous tablet peer " << tablet_id << ": not registered!";
@@ -806,7 +806,7 @@ void TSTabletManager::GetTabletPeers(vector<scoped_refptr<TabletPeer> >* tablet_
 }
 
 void TSTabletManager::MarkTabletDirty(const std::string& tablet_id, const std::string& reason) {
-  boost::lock_guard<rw_spinlock> lock(lock_);
+  std::lock_guard<rw_spinlock> lock(lock_);
   MarkDirtyUnlocked(tablet_id, reason);
 }
 
@@ -903,7 +903,7 @@ void TSTabletManager::GenerateFullTabletReport(TabletReportPB* report) {
 }
 
 void TSTabletManager::MarkTabletReportAcknowledged(const TabletReportPB& report) {
-  boost::lock_guard<rw_spinlock> l(lock_);
+  std::lock_guard<rw_spinlock> l(lock_);
 
   int32_t acked_seq = report.sequence_number();
   CHECK_LT(acked_seq, next_report_seq_);
@@ -1009,7 +1009,7 @@ TransitionInProgressDeleter::TransitionInProgressDeleter(
     : in_progress_(map), lock_(lock), entry_(std::move(entry)) {}
 
 TransitionInProgressDeleter::~TransitionInProgressDeleter() {
-  boost::lock_guard<rw_spinlock> lock(*lock_);
+  std::lock_guard<rw_spinlock> lock(*lock_);
   CHECK(in_progress_->erase(entry_));
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/tserver/ts_tablet_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/ts_tablet_manager.h b/src/kudu/tserver/ts_tablet_manager.h
index 9cdb830..5d5dbbf 100644
--- a/src/kudu/tserver/ts_tablet_manager.h
+++ b/src/kudu/tserver/ts_tablet_manager.h
@@ -18,7 +18,7 @@
 #define KUDU_TSERVER_TS_TABLET_MANAGER_H
 
 #include <boost/optional/optional_fwd.hpp>
-#include <boost/thread/locks.hpp>
+#include <boost/thread/shared_mutex.hpp>
 #include <gtest/gtest_prod.h>
 #include <memory>
 #include <string>

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/twitter-demo/insert_consumer.cc
----------------------------------------------------------------------
diff --git a/src/kudu/twitter-demo/insert_consumer.cc b/src/kudu/twitter-demo/insert_consumer.cc
index 370b8cc..2371f7a 100644
--- a/src/kudu/twitter-demo/insert_consumer.cc
+++ b/src/kudu/twitter-demo/insert_consumer.cc
@@ -17,8 +17,8 @@
 
 #include "kudu/twitter-demo/insert_consumer.h"
 
-#include <boost/thread/locks.hpp>
 #include <glog/logging.h>
+#include <mutex>
 #include <string>
 #include <time.h>
 #include <vector>
@@ -96,7 +96,7 @@ InsertConsumer::~InsertConsumer() {
 }
 
 void InsertConsumer::BatchFinished(const Status& s) {
-  boost::lock_guard<simple_spinlock> l(lock_);
+  std::lock_guard<simple_spinlock> l(lock_);
   request_pending_ = false;
   if (!s.ok()) {
     bool overflow;
@@ -145,7 +145,7 @@ void InsertConsumer::ConsumeJSON(const Slice& json_slice) {
   // instead of the manual batching here
   bool do_flush = false;
   {
-    boost::lock_guard<simple_spinlock> l(lock_);
+    std::lock_guard<simple_spinlock> l(lock_);
     if (!request_pending_) {
       request_pending_ = true;
       do_flush = true;

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/twitter-demo/twitter_streamer.cc
----------------------------------------------------------------------
diff --git a/src/kudu/twitter-demo/twitter_streamer.cc b/src/kudu/twitter-demo/twitter_streamer.cc
index d9bfcee..99a1cb0 100644
--- a/src/kudu/twitter-demo/twitter_streamer.cc
+++ b/src/kudu/twitter-demo/twitter_streamer.cc
@@ -17,10 +17,10 @@
 
 #include "kudu/twitter-demo/twitter_streamer.h"
 
-#include <boost/thread/locks.hpp>
 #include <curl/curl.h>
 #include <gflags/gflags.h>
 #include <glog/logging.h>
+#include <mutex>
 #include <string.h>
 #include <string>
 #include <thread>
@@ -132,7 +132,7 @@ void TwitterStreamer::StreamThread() {
   Status s = DoStreaming();
   if (!s.ok()) {
     LOG(ERROR) << "Streaming thread failed: " << s.ToString();
-    boost::lock_guard<boost::mutex> l(lock_);
+    std::lock_guard<std::mutex> l(lock_);
     stream_status_ = s;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/twitter-demo/twitter_streamer.h
----------------------------------------------------------------------
diff --git a/src/kudu/twitter-demo/twitter_streamer.h b/src/kudu/twitter-demo/twitter_streamer.h
index 6412fa5..df292bd 100644
--- a/src/kudu/twitter-demo/twitter_streamer.h
+++ b/src/kudu/twitter-demo/twitter_streamer.h
@@ -17,7 +17,7 @@
 #ifndef KUDU_TWITTER_DEMO_TWITTER_STREAMER_H
 #define KUDU_TWITTER_DEMO_TWITTER_STREAMER_H
 
-#include <boost/thread/mutex.hpp>
+#include <mutex>
 #include <thread>
 
 #include "kudu/util/faststring.h"
@@ -52,7 +52,7 @@ class TwitterStreamer {
   size_t DataReceived(const Slice& data);
 
   thread thread_;
-  boost::mutex lock_;
+  std::mutex lock_;
   Status stream_status_;
 
   faststring recv_buf_;

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/util/boost_mutex_utils.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/boost_mutex_utils.h b/src/kudu/util/boost_mutex_utils.h
index 41e475b..6f6390b 100644
--- a/src/kudu/util/boost_mutex_utils.h
+++ b/src/kudu/util/boost_mutex_utils.h
@@ -18,8 +18,8 @@
 #define KUDU_BOOST_MUTEX_UTILS_H
 
 
-// Similar to boost::lock_guard except that it takes
-// a lock pointer, and checks against NULL. If the
+// Similar to std::lock_guard except that it takes
+// a lock pointer, and checks against nullptr. If the
 // pointer is NULL, does nothing. Otherwise guards
 // with the lock.
 template<class LockType>
@@ -27,13 +27,13 @@ class lock_guard_maybe {
  public:
   explicit lock_guard_maybe(LockType *l) :
     lock_(l) {
-    if (l != NULL) {
+    if (l != nullptr) {
       l->lock();
     }
   }
 
   ~lock_guard_maybe() {
-    if (lock_ != NULL) {
+    if (lock_ != nullptr) {
       lock_->unlock();
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/util/locks.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/locks.h b/src/kudu/util/locks.h
index 28b4092..423f533 100644
--- a/src/kudu/util/locks.h
+++ b/src/kudu/util/locks.h
@@ -231,7 +231,7 @@ class percpu_rwlock {
   padded_lock *locks_;
 };
 
-// Simpler version of boost::lock_guard. Only supports the basic object
+// Simpler version of std::lock_guard. Only supports the basic object
 // lifecycle and defers any error checking to the underlying mutex.
 template <typename Mutex>
 class lock_guard {

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/util/mt-threadlocal-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/mt-threadlocal-test.cc b/src/kudu/util/mt-threadlocal-test.cc
index c3479ff..1d4d6b3 100644
--- a/src/kudu/util/mt-threadlocal-test.cc
+++ b/src/kudu/util/mt-threadlocal-test.cc
@@ -15,8 +15,8 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <boost/thread/locks.hpp>
 #include <glog/logging.h>
+#include <mutex>
 #include <unordered_set>
 
 #include "kudu/gutil/macros.h"
@@ -86,14 +86,14 @@ class Counter {
       registry_(CHECK_NOTNULL(registry)),
       val_(val) {
     LOG(INFO) << "Counter::~Counter(): tid = " << tid_ << ", addr = " << this << ", val = " << val_;
-    boost::lock_guard<RegistryLockType> reg_lock(*registry_->get_lock());
+    std::lock_guard<RegistryLockType> reg_lock(*registry_->get_lock());
     CHECK(registry_->RegisterUnlocked(this));
   }
 
   ~Counter() {
     LOG(INFO) << "Counter::~Counter(): tid = " << tid_ << ", addr = " << this << ", val = " << val_;
-    boost::lock_guard<RegistryLockType> reg_lock(*registry_->get_lock());
-    boost::lock_guard<CounterLockType> self_lock(lock_);
+    std::lock_guard<RegistryLockType> reg_lock(*registry_->get_lock());
+    std::lock_guard<CounterLockType> self_lock(lock_);
     LOG(INFO) << tid_ << ": deleting self from registry...";
     CHECK(registry_->UnregisterUnlocked(this));
   }
@@ -143,7 +143,7 @@ static void RegisterCounterAndLoopIncr(CounterRegistry* registry,
   reader_ready->Wait();
   // Now rock & roll on the counting loop.
   for (int i = 0; i < kTargetCounterVal; i++) {
-    boost::lock_guard<CounterLockType> l(*counter->get_lock());
+    std::lock_guard<CounterLockType> l(*counter->get_lock());
     counter->IncrementUnlocked();
   }
   // Let the reader know we're ready for him to verify our counts.
@@ -157,11 +157,11 @@ static void RegisterCounterAndLoopIncr(CounterRegistry* registry,
 static uint64_t Iterate(CounterRegistry* registry, int expected_counters) {
   uint64_t sum = 0;
   int seen_counters = 0;
-  boost::lock_guard<RegistryLockType> l(*registry->get_lock());
+  std::lock_guard<RegistryLockType> l(*registry->get_lock());
   for (Counter* counter : *registry->GetCountersUnlocked()) {
     uint64_t value;
     {
-      boost::lock_guard<CounterLockType> l(*counter->get_lock());
+      std::lock_guard<CounterLockType> l(*counter->get_lock());
       value = counter->GetValueUnlocked();
     }
     LOG(INFO) << "tid " << counter->tid() << " (counter " << counter << "): " << value;

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/util/oid_generator.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/oid_generator.cc b/src/kudu/util/oid_generator.cc
index 5c680b4..da7acd8 100644
--- a/src/kudu/util/oid_generator.cc
+++ b/src/kudu/util/oid_generator.cc
@@ -15,6 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <mutex>
 #include <string>
 
 #include "kudu/gutil/stringprintf.h"
@@ -23,7 +24,7 @@
 namespace kudu {
 
 string ObjectIdGenerator::Next() {
-  lock_guard<LockType> l(&oid_lock_);
+  std::lock_guard<LockType> l(oid_lock_);
   boost::uuids::uuid oid = oid_generator_();
   const uint8_t *uuid = oid.data;
   return StringPrintf("%02x%02x%02x%02x%02x%02x%02x%02x%02x%02x%02x%02x%02x%02x%02x%02x",

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/util/rw_semaphore-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/rw_semaphore-test.cc b/src/kudu/util/rw_semaphore-test.cc
index 19324e9..8c0ddd3 100644
--- a/src/kudu/util/rw_semaphore-test.cc
+++ b/src/kudu/util/rw_semaphore-test.cc
@@ -15,8 +15,9 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <boost/thread/locks.hpp>
+#include <boost/thread/shared_mutex.hpp>
 #include <gtest/gtest.h>
+#include <mutex>
 #include <thread>
 #include <vector>
 
@@ -39,7 +40,7 @@ struct SharedState {
 void Writer(SharedState* state) {
   int i = 0;
   while (true) {
-    boost::lock_guard<rw_semaphore> l(state->sem);
+    std::lock_guard<rw_semaphore> l(state->sem);
     state->int_var += (i++);
     if (state->done) {
       break;
@@ -77,7 +78,7 @@ TEST(RWSemaphoreTest, TestBasicOperation) {
 
   // Signal them to stop.
   {
-    boost::lock_guard<rw_semaphore> l(s.sem);
+    std::lock_guard<rw_semaphore> l(s.sem);
     s.done = true;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/util/rwc_lock-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/rwc_lock-test.cc b/src/kudu/util/rwc_lock-test.cc
index e3959c0..99c8071 100644
--- a/src/kudu/util/rwc_lock-test.cc
+++ b/src/kudu/util/rwc_lock-test.cc
@@ -15,7 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <boost/thread/locks.hpp>
+#include <mutex>
 #include <string>
 #include <thread>
 #include <vector>
@@ -57,19 +57,19 @@ struct LockHoldersCount {
   }
 
   void AdjustReaders(int delta) {
-    boost::lock_guard<simple_spinlock> l(lock);
+    std::lock_guard<simple_spinlock> l(lock);
     num_readers += delta;
     CheckInvariants();
   }
 
   void AdjustWriters(int delta) {
-    boost::lock_guard<simple_spinlock> l(lock);
+    std::lock_guard<simple_spinlock> l(lock);
     num_writers += delta;
     CheckInvariants();
   }
 
   void AdjustCommitters(int delta) {
-    boost::lock_guard<simple_spinlock> l(lock);
+    std::lock_guard<simple_spinlock> l(lock);
     num_committers += delta;
     CheckInvariants();
   }

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/util/throttler.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/throttler.cc b/src/kudu/util/throttler.cc
index 0eb4e9c..6207bdb 100644
--- a/src/kudu/util/throttler.cc
+++ b/src/kudu/util/throttler.cc
@@ -17,6 +17,8 @@
 
 #include "kudu/util/throttler.h"
 
+#include <mutex>
+
 namespace kudu {
 
 Throttler::Throttler(MonoTime now, uint64_t op_rate, uint64_t byte_rate, double burst_factor) :
@@ -33,7 +35,7 @@ bool Throttler::Take(MonoTime now, uint64_t op, uint64_t byte) {
   if (op_refill_ == 0 && byte_refill_ == 0) {
     return true;
   }
-  boost::lock_guard<simple_spinlock> lock(lock_);
+  std::lock_guard<simple_spinlock> lock(lock_);
   Refill(now);
   if ((op_refill_ == 0 || op <= op_token_) &&
       (byte_refill_ == 0 || byte <= byte_token_)) {

http://git-wip-us.apache.org/repos/asf/incubator-kudu/blob/cfa9a99f/src/kudu/util/throttler.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/throttler.h b/src/kudu/util/throttler.h
index 239778a..7be7ba1 100644
--- a/src/kudu/util/throttler.h
+++ b/src/kudu/util/throttler.h
@@ -19,8 +19,6 @@
 
 #include <algorithm>
 
-#include <boost/thread/locks.hpp>
-
 #include "kudu/gutil/macros.h"
 #include "kudu/util/locks.h"
 #include "kudu/util/monotime.h"