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 2017/08/25 22:46:25 UTC

kudu git commit: catalog_manager: don't log deleted tables/tablets at startup

Repository: kudu
Updated Branches:
  refs/heads/master 48aba1a86 -> ddb95e5fd


catalog_manager: don't log deleted tables/tablets at startup

On long-lived clusters this list can get quite long.

While I was in the area, I made some other cosmetic improvements:
- Applying strings::Substitute() to perforated LOG statements.
- Replacing 'OVERRIDE' with 'override'.
- Removing 'virtual' from overridden methods.
- Removing unnecessary std:: prefixes.
- Adding virtual destructors to {Table,Tablet}Visitor.
- A few other miscellaneous tweaks.

The new virtual destructors caused sys_catalog-test to fail in a bizarre
way: the catalog manager ran ~TabletLoader defined in sys_catalog-test.cc
rather than the implicit one from catalog_manager.cc. I fixed this by
renaming the sys_catalog-test {Table,Tablet}Loader classes; not quite sure
why there was no collision at link-time in the first place.

Change-Id: I557ebfe9d3ce1663e7f7c189f0087bdd29a09306
Reviewed-on: http://gerrit.cloudera.org:8080/7826
Reviewed-by: Alexey Serbin <as...@cloudera.com>
Reviewed-by: Todd Lipcon <to...@apache.org>
Tested-by: Adar Dembo <ad...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/ddb95e5f
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/ddb95e5f
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/ddb95e5f

Branch: refs/heads/master
Commit: ddb95e5fdfa06c1e099df640786c494ab841c9fd
Parents: 48aba1a
Author: Adar Dembo <ad...@cloudera.com>
Authored: Thu Aug 24 20:02:43 2017 -0700
Committer: Adar Dembo <ad...@cloudera.com>
Committed: Fri Aug 25 22:45:02 2017 +0000

----------------------------------------------------------------------
 src/kudu/master/catalog_manager.cc  | 404 +++++++++++++++----------------
 src/kudu/master/sys_catalog-test.cc |  16 +-
 src/kudu/master/sys_catalog.h       |   3 +
 3 files changed, 204 insertions(+), 219 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/ddb95e5f/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 3936cda..bfbbe7b 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -282,8 +282,8 @@ class TableLoader : public TableVisitor {
     : catalog_manager_(catalog_manager) {
   }
 
-  virtual Status VisitTable(const std::string& table_id,
-                            const SysTablesEntryPB& metadata) OVERRIDE {
+  Status VisitTable(const string& table_id,
+                    const SysTablesEntryPB& metadata) override {
     CHECK(!ContainsKey(catalog_manager_->table_ids_map_, table_id))
           << "Table already exists: " << table_id;
 
@@ -293,15 +293,18 @@ class TableLoader : public TableVisitor {
     l.mutable_data()->pb.CopyFrom(metadata);
 
     // Add the tablet to the IDs map and to the name map (if the table is not deleted).
+    bool is_deleted = l.mutable_data()->is_deleted();
     catalog_manager_->table_ids_map_[table->id()] = table;
-    if (!l.data().is_deleted()) {
+    if (!is_deleted) {
       catalog_manager_->table_names_map_[l.data().name()] = table;
     }
     l.Commit();
 
-    LOG(INFO) << "Loaded metadata for table " << table->ToString();
-    VLOG(1) << "Metadata for table " << table->ToString()
-            << ": " << SecureShortDebugString(metadata);
+    if (!is_deleted) {
+      LOG(INFO) << Substitute("Loaded metadata for table $0", table->ToString());
+    }
+    VLOG(2) << Substitute("Metadata for table $0: $1",
+                          table->ToString(), SecureShortDebugString(metadata));
     return Status::OK();
   }
 
@@ -321,18 +324,19 @@ class TabletLoader : public TabletVisitor {
     : catalog_manager_(catalog_manager) {
   }
 
-  virtual Status VisitTablet(const std::string& table_id,
-                             const std::string& tablet_id,
-                             const SysTabletsEntryPB& metadata) OVERRIDE {
+  Status VisitTablet(const string& table_id,
+                     const string& tablet_id,
+                     const SysTabletsEntryPB& metadata) override {
     // Lookup the table.
     scoped_refptr<TableInfo> table(FindPtrOrNull(
         catalog_manager_->table_ids_map_, table_id));
     if (table == nullptr) {
       // Tables and tablets are always created/deleted in one operation, so
       // this shouldn't be possible.
-      LOG(ERROR) << "Missing Table " << table_id << " required by tablet " << tablet_id;
-      LOG(ERROR) << "Metadata: " << SecureDebugString(metadata);
-      return Status::Corruption("Missing table for tablet: ", tablet_id);
+      string msg = Substitute("Missing table $0 required by tablet $1 (metadata: $2)",
+                              table_id, tablet_id, SecureDebugString(metadata));
+      LOG(ERROR) << msg;
+      return Status::Corruption(msg);
     }
 
     // Set up the tablet info.
@@ -348,11 +352,12 @@ class TabletLoader : public TabletVisitor {
     l.Commit();
     if (!is_deleted) {
       table->AddTablet(tablet);
+      LOG(INFO) << Substitute("Loaded metadata for tablet $0 (table $1)",
+                              tablet_id, table->ToString());
     }
 
-    LOG(INFO) << "Loaded metadata for tablet " << tablet_id
-              << " (table " << table->ToString() << ")";
-    VLOG(2) << "Metadata for tablet " << tablet_id << ": " << SecureShortDebugString(metadata);
+    VLOG(2) << Substitute("Metadata for tablet $0: $1",
+                          tablet_id, SecureShortDebugString(metadata));
     return Status::OK();
   }
 
@@ -984,8 +989,9 @@ void CatalogManager::PrepareForLeadershipTask() {
   if (term_before_wait != term) {
     // If we got elected leader again while waiting to catch up then we will
     // get another callback to visit the tables and tablets, so bail.
-    LOG(INFO) << "Term changed from " << term_before_wait << " to " << term
-        << " while waiting for master leader catchup. Not loading sys catalog metadata";
+    LOG(INFO) << Substitute("Term changed from $0 to $1 while waiting for "
+        "master leader catchup. Not loading sys catalog metadata",
+        term_before_wait, term);
     return;
   }
 
@@ -1011,9 +1017,8 @@ void CatalogManager::PrepareForLeadershipTask() {
         std::lock_guard<simple_spinlock> l(state_lock_);
         if (state_ == kClosing) {
           // Errors on shutdown are not considered fatal.
-          LOG(INFO) << op_description
-                    << " failed due to the shutdown of the catalog: "
-                    << s.ToString();
+          LOG(INFO) << Substitute("$0 failed due to the shutdown of the catalog: $1",
+                                  op_description, s.ToString());
           return s;
         }
       }
@@ -1022,15 +1027,13 @@ void CatalogManager::PrepareForLeadershipTask() {
       if (term != start_term) {
         // If the term has changed we assume the new leader catalog is about
         // to do the necessary work in its leadership preparation task.
-        LOG(INFO) << op_description << " failed; "
-                  << Substitute("change in term detected: $0 vs $1: ",
-                                start_term, term)
-                  << s.ToString();
+        LOG(INFO) << Substitute("$0 failed; change in term detected: $1 vs $2: $3",
+                                op_description, start_term, term, s.ToString());
         return s;
       }
 
       // In all other cases non-OK status is considered fatal.
-      LOG(FATAL) << op_description << " failed: " << s.ToString();
+      LOG(FATAL) << Substitute("$0 failed: $1", op_description, s.ToString());
       return s; // unreachable
     };
 
@@ -1302,8 +1305,8 @@ Status CatalogManager::CreateTable(const CreateTableRequestPB* orig_req,
 
   // Copy the request, so we can fill in some defaults.
   CreateTableRequestPB req = *orig_req;
-  LOG(INFO) << "CreateTable from " << RequestorString(rpc)
-            << ":\n" << SecureDebugString(req);
+  LOG(INFO) << Substitute("Servicing CreateTable request from $0:\n$1",
+                          RequestorString(rpc), SecureDebugString(req));
 
   // Do some fix-up of any defaults specified on columns.
   // Clients are only expected to pass the default value in the 'read_default'
@@ -1608,8 +1611,8 @@ Status CatalogManager::DeleteTable(const DeleteTableRequestPB* req,
   leader_lock_.AssertAcquiredForReading();
   RETURN_NOT_OK(CheckOnline());
 
-  LOG(INFO) << "Servicing DeleteTable request from " << RequestorString(rpc)
-            << ": " << SecureShortDebugString(*req);
+  LOG(INFO) << Substitute("Servicing DeleteTable request from $0:\n$1",
+                          RequestorString(rpc), SecureShortDebugString(*req));
 
   // 1. Look up the table, lock it, and mark it as removed.
   TRACE("Looking up table");
@@ -1689,8 +1692,7 @@ Status CatalogManager::DeleteTable(const DeleteTableRequestPB* req,
   // 8. Send a DeleteTablet() request to each tablet replica in the table.
   SendDeleteTableRequest(table, deletion_msg);
 
-  LOG(INFO) << "Successfully deleted table " << table->ToString()
-            << " per request from " << RequestorString(rpc);
+  VLOG(1) << "Deleted table " << table->ToString();
   return Status::OK();
 }
 
@@ -1945,8 +1947,8 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB* req,
   leader_lock_.AssertAcquiredForReading();
   RETURN_NOT_OK(CheckOnline());
 
-  LOG(INFO) << "Servicing AlterTable request from " << RequestorString(rpc)
-            << ": " << SecureShortDebugString(*req);
+  LOG(INFO) << Substitute("Servicing AlterTable request from $0:\n$1",
+                          RequestorString(rpc), SecureShortDebugString(*req));
 
   RETURN_NOT_OK(CheckOnline());
 
@@ -2352,10 +2354,8 @@ Status CatalogManager::ProcessTabletReport(TSDescriptor* ts_desc,
 
   leader_lock_.AssertAcquiredForReading();
 
-  if (VLOG_IS_ON(2)) {
-    VLOG(2) << "Received tablet report from " <<
-      RequestorString(rpc) << ": " << SecureDebugString(report);
-  }
+  VLOG(2) << Substitute("Received tablet report from $0:\n$1",
+                        RequestorString(rpc), SecureDebugString(report));
 
   // TODO: on a full tablet report, we may want to iterate over the tablets we think
   // the server should have, compare vs the ones being reported, and somehow mark
@@ -2428,8 +2428,6 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
   }
   DCHECK(tablet->table()); // guaranteed by TabletLoader
 
-  VLOG(3) << "tablet report: " << SecureShortDebugString(report);
-
   // TODO: we don't actually need to do the COW here until we see we're going
   // to change the state. Can we change CowedObject to lazily do the copy?
   TableMetadataLock table_lock(tablet->table().get(), TableMetadataLock::READ);
@@ -2441,8 +2439,8 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
       table_lock.data().is_deleted()) {
     report_updates->set_state_msg(tablet_lock.data().pb.state_msg());
     const string msg = tablet_lock.data().pb.state_msg();
-    LOG(INFO) << "Got report from deleted tablet " << tablet->ToString()
-              << " (" << msg << "): Sending delete request for this tablet";
+    LOG(INFO) << Substitute("Got report from deleted tablet $0 ($1): Sending "
+        "delete request for this tablet", tablet->ToString(), msg);
     // TODO: Cancel tablet creation, instead of deleting, in cases where
     // that might be possible (tablet creation timeout & replacement).
     SendDeleteReplicaRequest(tablet->tablet_id(), TABLET_DATA_DELETED,
@@ -2452,9 +2450,9 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
   }
 
   if (!table_lock.data().is_running()) {
-    LOG(INFO) << "Got report from tablet " << tablet->tablet_id()
-              << " for non-running table " << tablet->table()->ToString() << ": "
-              << tablet_lock.data().pb.state_msg();
+    LOG(INFO) << Substitute("Got report from tablet $0 for non-running table $1: $2",
+                            tablet->tablet_id(), tablet->table()->ToString(),
+                            tablet_lock.data().pb.state_msg());
     report_updates->set_state_msg(tablet_lock.data().pb.state_msg());
     return Status::OK();
   }
@@ -2464,17 +2462,14 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
   if (report.has_schema_version() &&
       table_lock.data().pb.version() != report.schema_version()) {
     if (report.schema_version() > table_lock.data().pb.version()) {
-      LOG(ERROR) << "TS " << ts_desc->ToString()
-                 << " has reported a schema version greater than the current one "
-                 << " for tablet " << tablet->ToString()
-                 << ". Expected version " << table_lock.data().pb.version()
-                 << " got " << report.schema_version()
-                 << " (corruption)";
+      LOG(ERROR) << Substitute("TS $0 has reported a schema version greater "
+          "than the current one for tablet $1. Expected version $2 got $3 (corruption)",
+          ts_desc->ToString(), tablet->ToString(),
+          table_lock.data().pb.version(), report.schema_version());
     } else {
-      LOG(INFO) << "TS " << ts_desc->ToString()
-            << " does not have the latest schema for tablet " << tablet->ToString()
-            << ". Expected version " << table_lock.data().pb.version()
-            << " got " << report.schema_version();
+      LOG(INFO) << Substitute("TS $0 does not have the latest schema for tablet $1. "
+          "Expected version $2 got $3", ts_desc->ToString(), tablet->ToString(),
+          table_lock.data().pb.version(), report.schema_version());
     }
     // It's possible that the tablet being reported is a laggy replica, and in fact
     // the leader has already received an AlterTable RPC. That's OK, though --
@@ -2486,8 +2481,8 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
   if (report.has_error()) {
     Status s = StatusFromPB(report.error());
     DCHECK(!s.ok());
-    LOG(WARNING) << "Tablet " << tablet->ToString() << " experienced an error on TS "
-                 << ts_desc->ToString() << ": " << s.ToString();
+    LOG(WARNING) << Substitute("Tablet $0 has failed on TS $1: $2",
+                               tablet->ToString(), ts_desc->ToString(), s.ToString());
     return Status::OK();
   }
 
@@ -2531,12 +2526,12 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
     // forever.
     if (!tablet_lock.data().is_running() && ShouldTransitionTabletToRunning(report)) {
       DCHECK_EQ(SysTabletsEntryPB::CREATING, tablet_lock.data().pb.state())
-          << "Tablet in unexpected state: " << tablet->ToString()
-          << ": " << SecureShortDebugString(tablet_lock.data().pb);
+          << Substitute("Tablet in unexpected state: $0: $1", tablet->ToString(),
+                        SecureShortDebugString(tablet_lock.data().pb));
       // Mark the tablet as running
       // TODO: we could batch the IO onto a background thread, or at least
       // across multiple tablets in the same report.
-      VLOG(1) << "Tablet " << tablet->ToString() << " is now online";
+      VLOG(1) << Substitute("Tablet $0 is now online", tablet->ToString());
       tablet_lock.mutable_data()->set_state(SysTabletsEntryPB::RUNNING,
                                             "Tablet reported with an active leader");
     }
@@ -2544,8 +2539,10 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
     // The Master only accepts committed consensus configurations since it needs the committed index
     // to only cache the most up-to-date config.
     if (PREDICT_FALSE(!cstate.committed_config().has_opid_index())) {
-      LOG(DFATAL) << "Missing opid_index in reported config:\n" << SecureDebugString(report);
-      return Status::InvalidArgument("Missing opid_index in reported config");
+      string msg = Substitute("Missing opid_index in reported config: $0",
+                              SecureDebugString(report));
+      LOG(DFATAL) << msg;
+      return Status::InvalidArgument(msg);
     }
 
     bool modified_cstate = false;
@@ -2579,9 +2576,9 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
 
       // If a replica is reporting a new consensus configuration, update the
       // master's copy of that configuration.
-      LOG(INFO) << "T " << tablet->tablet_id() << " reported consensus state change: "
-                << DiffConsensusStates(prev_cstate, cstate)
-                << ". New consensus state: " << SecureShortDebugString(cstate);
+      LOG(INFO) << Substitute("T $0 reported cstate change: $1. New cstate: $2",
+                              tablet->tablet_id(), DiffConsensusStates(prev_cstate, cstate),
+                              SecureShortDebugString(cstate));
 
       // If we need to change the report, copy the whole thing on the stack
       // rather than const-casting.
@@ -2593,13 +2590,11 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
         final_report = &updated_report;
       }
 
-      VLOG(2) << "Updating consensus configuration for tablet "
-              << final_report->tablet_id()
-              << " from config reported by " << ts_desc->ToString()
-              << " to that committed in log index "
-              << final_report->consensus_state().committed_config().opid_index()
-              << " with leader state from term "
-              << final_report->consensus_state().current_term();
+      VLOG(2) << Substitute("Updating cstate for tablet $0 from config reported by $1 "
+                            "to that committed in log index $2 with leader state from term $3",
+                            final_report->tablet_id(), ts_desc->ToString(),
+                            final_report->consensus_state().committed_config().opid_index(),
+                            final_report->consensus_state().current_term());
 
       RETURN_NOT_OK(HandleRaftConfigChanged(*final_report, tablet,
                                             &tablet_lock, &table_lock));
@@ -2617,8 +2612,8 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
   actions.tablets_to_update.push_back(tablet.get());
   Status s = sys_catalog_->Write(actions);
   if (!s.ok()) {
-    LOG(WARNING) << "Error updating tablets: " << s.ToString() << ". Tablet report was: "
-                 << SecureShortDebugString(report);
+    LOG(ERROR) << Substitute("Error updating tablets: $0. Tablet report was: $1",
+                             s.ToString(), SecureShortDebugString(report));
     return s;
   }
   tablet_lock.Commit();
@@ -2647,7 +2642,7 @@ Status CatalogManager::HandleRaftConfigChanged(
   *tablet_lock->mutable_data()->pb.mutable_consensus_state() = cstate;
 
   if (FLAGS_master_tombstone_evicted_tablet_replicas) {
-    std::unordered_set<string> current_member_uuids;
+    unordered_set<string> current_member_uuids;
     for (const consensus::RaftPeerPB& peer : cstate.committed_config().peers()) {
       InsertOrDie(&current_member_uuids, peer.permanent_uuid());
     }
@@ -2732,7 +2727,7 @@ class PickSpecificUUID : public TSPicker {
   explicit PickSpecificUUID(string ts_uuid)
       : ts_uuid_(std::move(ts_uuid)) {}
 
-  virtual Status PickReplica(string* ts_uuid) OVERRIDE {
+  Status PickReplica(string* ts_uuid) override {
     // Just a straight passthrough.
     *ts_uuid = ts_uuid_;
     return Status::OK();
@@ -2752,7 +2747,7 @@ class PickLeaderReplica : public TSPicker {
     tablet_(tablet) {
   }
 
-  virtual Status PickReplica(string* ts_uuid) OVERRIDE {
+  Status PickReplica(string* ts_uuid) override {
     TabletMetadataLock l(tablet_.get(), TabletMetadataLock::READ);
 
     string err_msg;
@@ -2797,16 +2792,16 @@ class RetryingTSRpcTask : public MonitoredTask {
   Status Run();
 
   // Abort this task.
-  virtual void Abort() OVERRIDE {
+  void Abort() override {
     MarkAborted();
   }
 
-  virtual State state() const OVERRIDE {
+  State state() const override {
     return static_cast<State>(NoBarrier_Load(&state_));
   }
 
-  virtual MonoTime start_timestamp() const OVERRIDE { return start_ts_; }
-  virtual MonoTime completion_timestamp() const OVERRIDE { return end_ts_; }
+  MonoTime start_timestamp() const override { return start_ts_; }
+  MonoTime completion_timestamp() const override { return end_ts_; }
 
  protected:
   // Send an RPC request and register a callback.
@@ -2917,9 +2912,9 @@ Status RetryingTSRpcTask::Run() {
 
 void RetryingTSRpcTask::RpcCallback() {
   if (!rpc_.status().ok()) {
-    LOG(WARNING) << "TS " << target_ts_desc_->ToString() << ": "
-                  << type_name() << " RPC failed for tablet "
-                  << tablet_id() << ": " << rpc_.status().ToString();
+    LOG(WARNING) << Substitute("TS $0: $1 RPC failed for tablet $2: $3",
+                               target_ts_desc_->ToString(), type_name(),
+                               tablet_id(), rpc_.status().ToString());
   } else if (state() != kStateAborted) {
     HandleResponse(attempt_); // Modifies state_.
   }
@@ -2951,21 +2946,20 @@ bool RetryingTSRpcTask::RescheduleWithBackoffDelay() {
   if (delay_millis <= 0) {
     LOG(WARNING) << "Request timed out: " << description();
     MarkFailed();
-  } else {
-    LOG(INFO) << "Scheduling retry of " << description() << " with a delay"
-              << " of " << delay_millis << "ms (attempt = " << attempt_ << ")...";
-    master_->messenger()->ScheduleOnReactor(
-        boost::bind(&RetryingTSRpcTask::RunDelayedTask, this, _1),
-        MonoDelta::FromMilliseconds(delay_millis));
-    return true;
+    return false;
   }
-  return false;
+  LOG(INFO) << Substitute("Scheduling retry of $0 with a delay of $1 ms (attempt = $2)",
+                          description(), delay_millis, attempt_);
+  master_->messenger()->ScheduleOnReactor(
+      boost::bind(&RetryingTSRpcTask::RunDelayedTask, this, _1),
+      MonoDelta::FromMilliseconds(delay_millis));
+  return true;
 }
 
 void RetryingTSRpcTask::RunDelayedTask(const Status& status) {
   if (!status.ok()) {
-    LOG(WARNING) << "Async tablet task " << description() << " failed or was cancelled: "
-                  << status.ToString();
+    LOG(WARNING) << Substitute("Async tablet task $0 failed was cancelled: $1",
+                               description(), status.ToString());
     UnregisterAsyncTask();   // May delete this.
     return;
   }
@@ -2973,7 +2967,8 @@ void RetryingTSRpcTask::RunDelayedTask(const Status& status) {
   string desc = description();  // Save in case we need to log after deletion.
   Status s = Run();             // May delete this.
   if (!s.ok()) {
-    LOG(WARNING) << "Async tablet task " << desc << " failed: " << s.ToString();
+    LOG(WARNING) << Substitute("Async tablet task $0 failed: $1",
+                               desc, s.ToString());
   }
 }
 
@@ -3060,37 +3055,35 @@ class AsyncCreateReplica : public RetrySpecificTSRpcTask {
         tablet_lock.data().pb.consensus_state().committed_config());
   }
 
-  virtual string type_name() const OVERRIDE { return "Create Tablet"; }
+  string type_name() const override { return "Create Tablet"; }
 
-  virtual string description() const OVERRIDE {
+  string description() const override {
     return "CreateTablet RPC for tablet " + tablet_id_ + " on TS " + permanent_uuid_;
   }
 
  protected:
-  virtual string tablet_id() const OVERRIDE { return tablet_id_; }
+  string tablet_id() const override { return tablet_id_; }
 
-  virtual void HandleResponse(int attempt) OVERRIDE {
+  void HandleResponse(int attempt) override {
     if (!resp_.has_error()) {
       MarkComplete();
     } else {
       Status s = StatusFromPB(resp_.error().status());
       if (s.IsAlreadyPresent()) {
-        LOG(INFO) << "CreateTablet RPC for tablet " << tablet_id_
-                  << " on TS " << target_ts_desc_->ToString() << " returned already present: "
-                  << s.ToString();
+        LOG(INFO) << Substitute("CreateTablet RPC for tablet $0 on TS $1 "
+            "returned already present: $2", tablet_id_,
+            target_ts_desc_->ToString(), s.ToString());
         MarkComplete();
       } else {
-        LOG(WARNING) << "CreateTablet RPC for tablet " << tablet_id_
-                     << " on TS " << target_ts_desc_->ToString() << " failed: " << s.ToString();
+        LOG(WARNING) << Substitute("CreateTablet RPC for tablet $0 on TS $1 failed: $2",
+                                   tablet_id_, target_ts_desc_->ToString(), s.ToString());
       }
     }
   }
 
-  virtual bool SendRequest(int attempt) OVERRIDE {
-    VLOG(1) << "Send create tablet request to "
-            << target_ts_desc_->ToString() << ":\n"
-            << " (attempt " << attempt << "):\n"
-            << SecureDebugString(req_);
+  bool SendRequest(int attempt) override {
+    VLOG(1) << Substitute("Send create tablet request to $0 (attempt = $1): $2",
+                          target_ts_desc_->ToString(), attempt, SecureDebugString(req_));
     ts_proxy_->CreateTabletAsync(req_, &resp_, &rpc_,
                                  boost::bind(&AsyncCreateReplica::RpcCallback, this));
     return true;
@@ -3107,7 +3100,7 @@ class AsyncDeleteReplica : public RetrySpecificTSRpcTask {
  public:
   AsyncDeleteReplica(
       Master* master, const string& permanent_uuid,
-      const scoped_refptr<TableInfo>& table, std::string tablet_id,
+      const scoped_refptr<TableInfo>& table, string tablet_id,
       TabletDataState delete_type,
       boost::optional<int64_t> cas_config_opid_index_less_or_equal,
       string reason)
@@ -3118,16 +3111,16 @@ class AsyncDeleteReplica : public RetrySpecificTSRpcTask {
             std::move(cas_config_opid_index_less_or_equal)),
         reason_(std::move(reason)) {}
 
-  virtual string type_name() const OVERRIDE { return "Delete Tablet"; }
+  string type_name() const override { return "Delete Tablet"; }
 
-  virtual string description() const OVERRIDE {
+  string description() const override {
     return tablet_id_ + " Delete Tablet RPC for TS=" + permanent_uuid_;
   }
 
  protected:
-  virtual string tablet_id() const OVERRIDE { return tablet_id_; }
+  string tablet_id() const override { return tablet_id_; }
 
-  virtual void HandleResponse(int attempt) OVERRIDE {
+  void HandleResponse(int attempt) override {
     if (resp_.has_error()) {
       Status status = StatusFromPB(resp_.error().status());
 
@@ -3135,43 +3128,39 @@ class AsyncDeleteReplica : public RetrySpecificTSRpcTask {
       TabletServerErrorPB::Code code = resp_.error().code();
       switch (code) {
         case TabletServerErrorPB::TABLET_NOT_FOUND:
-          LOG(WARNING) << "TS " << target_ts_desc_->ToString()
-                       << ": delete failed for tablet " << tablet_id_
-                       << " because the tablet was not found. No further retry: "
-                       << status.ToString();
+          LOG(WARNING) << Substitute("TS $0: delete failed for tablet $1 "
+              "because the tablet was not found. No further retry: $2",
+              target_ts_desc_->ToString(), tablet_id_, status.ToString());
           MarkComplete();
           break;
         case TabletServerErrorPB::CAS_FAILED:
-          LOG(WARNING) << "TS " << target_ts_desc_->ToString()
-                       << ": delete failed for tablet " << tablet_id_
-                       << " due to a CAS failure. No further retry: " << status.ToString();
+          LOG(WARNING) << Substitute("TS $0: delete failed for tablet $1 "
+              "because of a CAS failure. No further retry: $2",
+              target_ts_desc_->ToString(), tablet_id_, status.ToString());
           MarkComplete();
           break;
         default:
-          LOG(WARNING) << "TS " << target_ts_desc_->ToString()
-                       << ": delete failed for tablet " << tablet_id_
-                       << " with error code " << TabletServerErrorPB::Code_Name(code)
-                       << ": " << status.ToString();
+          LOG(WARNING) << Substitute("TS $0: delete failed for tablet $1 "
+              "with error code $2: $3", target_ts_desc_->ToString(), tablet_id_,
+              TabletServerErrorPB::Code_Name(code), status.ToString());
           break;
       }
     } else {
       master_->catalog_manager()->NotifyTabletDeleteSuccess(permanent_uuid_, tablet_id_);
       if (table_) {
-        LOG(INFO) << "TS " << target_ts_desc_->ToString()
-                  << ": tablet " << tablet_id_
-                  << " (table " << table_->ToString() << ") successfully deleted";
+        LOG(INFO) << Substitute("TS $0: tablet $1 (table $2) successfully deleted",
+                                target_ts_desc_->ToString(), tablet_id_, table_->ToString());
       } else {
-        LOG(WARNING) << "TS " << target_ts_desc_->ToString()
-                     << ": tablet " << tablet_id_
-                     << " did not belong to a known table, but was successfully deleted";
+        LOG(WARNING) << Substitute("TS $0: tablet $1 did not belong to a known table, "
+            "but was successfully deleted", target_ts_desc_->ToString(), tablet_id_);
       }
       MarkComplete();
-      VLOG(1) << "TS " << target_ts_desc_->ToString()
-              << ": delete complete on tablet " << tablet_id_;
+      VLOG(1) << Substitute("TS $0: delete complete on tablet $1",
+                            target_ts_desc_->ToString(), tablet_id_);
     }
   }
 
-  virtual bool SendRequest(int attempt) OVERRIDE {
+  bool SendRequest(int attempt) override {
     tserver::DeleteTabletRequestPB req;
     req.set_dest_uuid(permanent_uuid_);
     req.set_tablet_id(tablet_id_);
@@ -3181,21 +3170,18 @@ class AsyncDeleteReplica : public RetrySpecificTSRpcTask {
       req.set_cas_config_opid_index_less_or_equal(*cas_config_opid_index_less_or_equal_);
     }
 
-    LOG(INFO) << Substitute("Sending DeleteTablet($0) for tablet $1 on $2 "
-                            "($3)",
-                            TabletDataState_Name(delete_type_),
-                            tablet_id_,
-                            target_ts_desc_->ToString(),
-                            reason_);
+    LOG(INFO) << Substitute("Sending DeleteTablet($0) for tablet $1 on $2 ($3)",
+                            TabletDataState_Name(delete_type_), tablet_id_,
+                            target_ts_desc_->ToString(), reason_);
     ts_proxy_->DeleteTabletAsync(req, &resp_, &rpc_,
                                  boost::bind(&AsyncDeleteReplica::RpcCallback, this));
     return true;
   }
 
-  const std::string tablet_id_;
+  const string tablet_id_;
   const TabletDataState delete_type_;
   const boost::optional<int64_t> cas_config_opid_index_less_or_equal_;
-  const std::string reason_;
+  const string reason_;
   tserver::DeleteTabletResponsePB resp_;
 };
 
@@ -3302,16 +3288,16 @@ class AsyncAlterTable : public RetryingTSRpcTask {
       tablet_(tablet) {
   }
 
-  virtual string type_name() const OVERRIDE { return "Alter Table"; }
+  string type_name() const override { return "Alter Table"; }
 
-  virtual string description() const OVERRIDE {
+  string description() const override {
     return tablet_->ToString() + " Alter Table RPC";
   }
 
  private:
-  virtual string tablet_id() const OVERRIDE { return tablet_->tablet_id(); }
+  string tablet_id() const override { return tablet_->tablet_id(); }
 
-  virtual void HandleResponse(int attempt) OVERRIDE {
+  void HandleResponse(int attempt) override {
     if (resp_.has_error()) {
       Status status = StatusFromPB(resp_.error().status());
 
@@ -3320,19 +3306,21 @@ class AsyncAlterTable : public RetryingTSRpcTask {
         case TabletServerErrorPB::TABLET_NOT_FOUND:
         case TabletServerErrorPB::MISMATCHED_SCHEMA:
         case TabletServerErrorPB::TABLET_HAS_A_NEWER_SCHEMA:
-          LOG(WARNING) << "TS " << target_ts_desc_->ToString() << ": alter failed for tablet "
-                       << tablet_->ToString() << " no further retry: " << status.ToString();
+          LOG(WARNING) << Substitute("TS $0: alter failed for tablet $1,"
+              "no further retry: $2", target_ts_desc_->ToString(),
+              tablet_->ToString(), status.ToString());
           MarkComplete();
           break;
         default:
-          LOG(WARNING) << "TS " << target_ts_desc_->ToString() << ": alter failed for tablet "
-                       << tablet_->ToString() << ": " << status.ToString();
+          LOG(WARNING) << Substitute("TS $0: alter failed for tablet $1: $2",
+                                     target_ts_desc_->ToString(), tablet_->ToString(),
+                                     status.ToString());
           break;
       }
     } else {
       MarkComplete();
-      VLOG(1) << "TS " << target_ts_desc_->ToString()
-              << ": alter complete on tablet " << tablet_->ToString();
+      VLOG(1) << Substitute("TS $0: alter complete on tablet $1",
+                            target_ts_desc_->ToString(), tablet_->ToString());
     }
 
     if (state() != kStateComplete) {
@@ -3340,7 +3328,7 @@ class AsyncAlterTable : public RetryingTSRpcTask {
     }
   }
 
-  virtual bool SendRequest(int attempt) OVERRIDE {
+  bool SendRequest(int attempt) override {
     TableMetadataLock l(tablet_->table().get(), TableMetadataLock::READ);
 
     tserver::AlterSchemaRequestPB req;
@@ -3352,9 +3340,8 @@ class AsyncAlterTable : public RetryingTSRpcTask {
 
     l.Unlock();
 
-    VLOG(1) << "Send alter table request to " << target_ts_desc_->ToString()
-            << " (attempt " << attempt << "):\n"
-            << SecureDebugString(req);
+    VLOG(1) << Substitute("Sending alter table request to $0 (attempt $1): $2",
+                          target_ts_desc_->ToString(), attempt, SecureDebugString(req));
     ts_proxy_->AlterSchemaAsync(req, &resp_, &rpc_,
                                 boost::bind(&AsyncAlterTable::RpcCallback, this));
     return true;
@@ -3379,16 +3366,16 @@ class AsyncAddServerTask : public RetryingTSRpcTask {
     deadline_ = MonoTime::Max(); // Never time out.
   }
 
-  virtual string type_name() const OVERRIDE { return "AddServer ChangeConfig"; }
+  string type_name() const override { return "AddServer ChangeConfig"; }
 
-  virtual string description() const OVERRIDE;
+  string description() const override;
 
  protected:
-  virtual bool SendRequest(int attempt) OVERRIDE;
-  virtual void HandleResponse(int attempt) OVERRIDE;
+  bool SendRequest(int attempt) override;
+  void HandleResponse(int attempt) override;
 
  private:
-  virtual string tablet_id() const OVERRIDE { return tablet_->tablet_id(); }
+  string tablet_id() const override { return tablet_->tablet_id(); }
 
   const scoped_refptr<TabletInfo> tablet_;
   const ConsensusStatePB cstate_;
@@ -3408,8 +3395,8 @@ string AsyncAddServerTask::description() const {
 }
 
 bool AsyncAddServerTask::SendRequest(int attempt) {
-  LOG(INFO) << "Sending request for AddServer on tablet " << tablet_->tablet_id()
-            << " (attempt " << attempt << ")";
+  LOG(INFO) << Substitute("Sending request for AddServer on tablet $0 (attempt $1)",
+                          tablet_->tablet_id(), attempt);
 
   // Bail if we're retrying in vain.
   int64_t latest_index;
@@ -3419,9 +3406,9 @@ bool AsyncAddServerTask::SendRequest(int attempt) {
       .committed_config().opid_index();
   }
   if (latest_index > cstate_.committed_config().opid_index()) {
-    LOG_WITH_PREFIX(INFO) << "Latest config for has opid_index of " << latest_index
-                          << " while this task has opid_index of "
-                          << cstate_.committed_config().opid_index() << ". Aborting task.";
+    LOG_WITH_PREFIX(INFO) << Substitute("Latest config for has opid_index of $0 "
+        "while this task has opid_index of $1. Aborting task",
+        latest_index, cstate_.committed_config().opid_index());
     MarkAborted();
     return false;
   }
@@ -3430,7 +3417,7 @@ bool AsyncAddServerTask::SendRequest(int attempt) {
   // Do not include current members of the config.
   TSDescriptorVector ts_descs;
   master_->ts_manager()->GetAllLiveDescriptors(&ts_descs);
-  set<std::shared_ptr<TSDescriptor>> excluded;
+  set<shared_ptr<TSDescriptor>> excluded;
   for (const auto& ts_desc : ts_descs) {
     if (IsRaftConfigMember(ts_desc->permanent_uuid(), cstate_.committed_config())) {
       InsertOrDie(&excluded, ts_desc);
@@ -3438,8 +3425,9 @@ bool AsyncAddServerTask::SendRequest(int attempt) {
   }
   auto replacement_replica = SelectReplica(ts_descs, excluded, rng_);
   if (PREDICT_FALSE(!replacement_replica)) {
-    KLOG_EVERY_N(WARNING, 100) << LogPrefix() << "No candidate replacement replica found "
-                               << "for tablet " << tablet_->ToString();
+    KLOG_EVERY_N(WARNING, 100) << LogPrefix()
+                               << "No candidate replacement replica found for tablet "
+                               << tablet_->ToString();
     return false;
   }
 
@@ -3454,9 +3442,8 @@ bool AsyncAddServerTask::SendRequest(int attempt) {
   CHECK_GT(peer_reg.rpc_addresses_size(), 0);
   *peer->mutable_last_known_addr() = peer_reg.rpc_addresses(0);
   peer->set_member_type(RaftPeerPB::VOTER);
-  VLOG(1) << "Sending AddServer ChangeConfig request to "
-          << target_ts_desc_->ToString() << ":\n"
-          << SecureDebugString(req_);
+  VLOG(1) << Substitute("Sending AddServer ChangeConfig request to $0: $1",
+                        target_ts_desc_->ToString(), SecureDebugString(req_));
   consensus_proxy_->ChangeConfigAsync(req_, &resp_, &rpc_,
                                       boost::bind(&AsyncAddServerTask::RpcCallback, this));
   return true;
@@ -3474,19 +3461,16 @@ void AsyncAddServerTask::HandleResponse(int attempt) {
   // Do not retry on a CAS error, otherwise retry forever or until cancelled.
   switch (resp_.error().code()) {
     case TabletServerErrorPB::CAS_FAILED:
-      LOG_WITH_PREFIX(WARNING) << "ChangeConfig() failed with leader "\
-                               << target_ts_desc_->ToString()
-                               << " due to CAS failure. No further retry: "
-                               << status.ToString();
+      LOG_WITH_PREFIX(WARNING) << Substitute("ChangeConfig() failed with leader $0 "
+          "due to CAS failure. No further retry: $1",
+          target_ts_desc_->ToString(), status.ToString());
       MarkFailed();
       break;
     default:
-      LOG_WITH_PREFIX(INFO) << "ChangeConfig() failed with leader "
-                            << target_ts_desc_->ToString()
-                            << " due to error "
-                            << TabletServerErrorPB::Code_Name(resp_.error().code())
-                            << ". This operation will be retried. Error detail: "
-                            << status.ToString();
+      LOG_WITH_PREFIX(INFO) << Substitute("ChangeConfig() failed with leader $0 "
+          "due to error $1. This operation will be retried. Error detail: $2",
+          target_ts_desc_->ToString(),
+          TabletServerErrorPB::Code_Name(resp_.error().code()), status.ToString());
       break;
   }
 }
@@ -3540,7 +3524,7 @@ void CatalogManager::SendDeleteTabletRequest(const scoped_refptr<TabletInfo>& ta
 }
 
 void CatalogManager::SendDeleteReplicaRequest(
-    const std::string& tablet_id,
+    const string& tablet_id,
     TabletDataState delete_type,
     const boost::optional<int64_t>& cas_config_opid_index_less_or_equal,
     const scoped_refptr<TableInfo>& table,
@@ -3677,8 +3661,8 @@ void CatalogManager::HandleAssignCreatingTablet(TabletInfo* tablet,
 
   // Skip the tablet if the assignment timeout is not yet expired
   if (remaining_timeout_ms > 0) {
-    VLOG(2) << "Tablet " << tablet->ToString() << " still being created. "
-            << remaining_timeout_ms << "ms remain until timeout.";
+    VLOG(2) << Substitute("Tablet $0 still being created. $1ms remain until timeout",
+                          tablet->ToString(), remaining_timeout_ms);
     return;
   }
 
@@ -3688,10 +3672,9 @@ void CatalogManager::HandleAssignCreatingTablet(TabletInfo* tablet,
   // within the timeout. So the tablet will be replaced by a new one.
   scoped_refptr<TabletInfo> replacement = CreateTabletInfo(tablet->table().get(),
                                                            old_info.pb.partition());
-  LOG_WITH_PREFIX(WARNING)
-      << "Tablet " << tablet->ToString() << " was not created within "
-      << "the allowed timeout. Replacing with a new tablet "
-      << replacement->tablet_id();
+  LOG_WITH_PREFIX(WARNING) << Substitute("Tablet $0 was not created within the "
+      "allowed timeout. Replacing with a new tablet $1",
+      tablet->ToString(), replacement->tablet_id());
 
   // Mark old tablet as replaced.
   tablet->mutable_metadata()->mutable_dirty()->set_state(
@@ -3707,9 +3690,9 @@ void CatalogManager::HandleAssignCreatingTablet(TabletInfo* tablet,
   deferred->tablets_to_update.push_back(tablet);
   deferred->tablets_to_add.push_back(replacement.get());
   deferred->needs_create_rpc.push_back(replacement.get());
-  VLOG(1) << "Replaced tablet " << tablet->tablet_id()
-          << " with " << replacement->tablet_id()
-          << " (Table " << tablet->table()->ToString() << ")";
+  VLOG(1) << Substitute("Replaced tablet $0 with $1 (table $2)",
+                        tablet->tablet_id(), replacement->tablet_id(),
+                        tablet->table()->ToString());
 
   new_tablets->emplace_back(std::move(replacement));
 }
@@ -3748,14 +3731,13 @@ Status CatalogManager::HandleTabletSchemaVersionReport(TabletInfo *tablet, uint3
   }
 
   l.Commit();
-  LOG_WITH_PREFIX(INFO)
-      << table->ToString() << " - Alter table completed version="
-      << current_version;
+  LOG_WITH_PREFIX(INFO) << Substitute("$0 alter complete (version $1)",
+                                      table->ToString(), current_version);
   return Status::OK();
 }
 
 Status CatalogManager::ProcessPendingAssignments(
-    const std::vector<scoped_refptr<TabletInfo> >& tablets) {
+    const vector<scoped_refptr<TabletInfo> >& tablets) {
   VLOG(1) << "Processing pending assignments";
 
   // Take write locks on all tablets to be processed, and ensure that they are
@@ -3790,8 +3772,8 @@ Status CatalogManager::ProcessPendingAssignments(
         break;
       }
       default:
-        VLOG(2) << "Nothing to do for tablet " << tablet->tablet_id() << ": state = "
-                << SysTabletsEntryPB_State_Name(t_state);
+        VLOG(2) << Substitute("Nothing to do for tablet $0: $1", tablet->tablet_id(),
+                              SysTabletsEntryPB_State_Name(t_state));
         break;
     }
   }
@@ -4003,7 +3985,7 @@ Status CatalogManager::BuildLocationsForTablet(const scoped_refptr<TabletInfo>&
   return Status::OK();
 }
 
-Status CatalogManager::GetTabletLocations(const std::string& tablet_id,
+Status CatalogManager::GetTabletLocations(const string& tablet_id,
                                           TabletLocationsPB* locs_pb) {
   leader_lock_.AssertAcquiredForReading();
   RETURN_NOT_OK(CheckOnline());
@@ -4142,7 +4124,7 @@ void CatalogManager::DumpState(std::ostream* out) const {
   }
 }
 
-std::string CatalogManager::LogPrefix() const {
+string CatalogManager::LogPrefix() const {
   return Substitute("T $0 P $1: ",
                     sys_catalog_->tablet_replica()->tablet_id(),
                     sys_catalog_->tablet_replica()->permanent_uuid());
@@ -4263,8 +4245,7 @@ INITTED_AND_LEADER_OR_RESPOND(GetTabletLocationsResponsePB);
 // TabletInfo
 ////////////////////////////////////////////////////////////
 
-TabletInfo::TabletInfo(const scoped_refptr<TableInfo>& table,
-                       std::string tablet_id)
+TabletInfo::TabletInfo(const scoped_refptr<TableInfo>& table, string tablet_id)
     : tablet_id_(std::move(tablet_id)),
       table_(table),
       last_create_tablet_time_(MonoTime::Now()),
@@ -4297,7 +4278,7 @@ uint32_t TabletInfo::reported_schema_version() const {
   return reported_schema_version_;
 }
 
-std::string TabletInfo::ToString() const {
+string TabletInfo::ToString() const {
   return Substitute("$0 (table $1)", tablet_id_,
                     (table_ != nullptr ? table_->ToString() : "MISSING"));
 }
@@ -4311,17 +4292,17 @@ void PersistentTabletInfo::set_state(SysTabletsEntryPB::State state, const strin
 // TableInfo
 ////////////////////////////////////////////////////////////
 
-TableInfo::TableInfo(std::string table_id) : table_id_(std::move(table_id)) {}
+TableInfo::TableInfo(string table_id) : table_id_(std::move(table_id)) {}
 
 TableInfo::~TableInfo() {
 }
 
-std::string TableInfo::ToString() const {
+string TableInfo::ToString() const {
   TableMetadataLock l(this, TableMetadataLock::READ);
   return Substitute("$0 [id=$1]", l.data().pb.name(), table_id_);
 }
 
-bool TableInfo::RemoveTablet(const std::string& partition_key_start) {
+bool TableInfo::RemoveTablet(const string& partition_key_start) {
   std::lock_guard<rw_spinlock> l(lock_);
   return EraseKeyReturnValuePtr(&tablet_map_, partition_key_start) != nullptr;
 }
@@ -4355,7 +4336,8 @@ void TableInfo::AddTabletUnlocked(TabletInfo* tablet) {
   if (UpdateReturnCopy(&tablet_map_,
                        tablet->metadata().state().pb.partition().partition_key_start(),
                        tablet, &old)) {
-    VLOG(1) << "Replaced tablet " << old->tablet_id() << " with " << tablet->tablet_id();
+    VLOG(1) << Substitute("Replaced tablet $0 with $1",
+                          old->tablet_id(), tablet->tablet_id());
     // TODO: can we assert that the replaced tablet is not in Running state?
     // May be a little tricky since we don't know whether to look at its committed or
     // uncommitted state.
@@ -4394,9 +4376,9 @@ bool TableInfo::IsAlterInProgress(uint32_t version) const {
   shared_lock<rw_spinlock> l(lock_);
   for (const TableInfo::TabletInfoMap::value_type& e : tablet_map_) {
     if (e.second->reported_schema_version() < version) {
-      VLOG(3) << "Table " << table_id_ << " ALTER in progress due to tablet "
-              << e.second->ToString() << " because reported schema "
-              << e.second->reported_schema_version() << " < expected " << version;
+      VLOG(3) << Substitute("Table $0 ALTER in progress due to tablet $1 "
+          "because reported schema $2 < expected $3", table_id_,
+          e.second->ToString(), e.second->reported_schema_version(), version);
       return true;
     }
   }
@@ -4454,14 +4436,14 @@ void TableInfo::WaitTasksCompletion() {
   }
 }
 
-void TableInfo::GetTaskList(std::vector<scoped_refptr<MonitoredTask> > *ret) {
+void TableInfo::GetTaskList(vector<scoped_refptr<MonitoredTask>>* ret) {
   shared_lock<rw_spinlock> l(lock_);
   for (MonitoredTask* task : pending_tasks_) {
     ret->push_back(make_scoped_refptr(task));
   }
 }
 
-void TableInfo::GetAllTablets(vector<scoped_refptr<TabletInfo> > *ret) const {
+void TableInfo::GetAllTablets(vector<scoped_refptr<TabletInfo>>* ret) const {
   ret->clear();
   shared_lock<rw_spinlock> l(lock_);
   for (const auto& e : tablet_map_) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/ddb95e5f/src/kudu/master/sys_catalog-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/sys_catalog-test.cc b/src/kudu/master/sys_catalog-test.cc
index 19a452f..32ab82f 100644
--- a/src/kudu/master/sys_catalog-test.cc
+++ b/src/kudu/master/sys_catalog-test.cc
@@ -93,10 +93,10 @@ class SysCatalogTest : public KuduTest {
   gscoped_ptr<MasterServiceProxy> proxy_;
 };
 
-class TableLoader : public TableVisitor {
+class TestTableLoader : public TableVisitor {
  public:
-  TableLoader() {}
-  ~TableLoader() { Reset(); }
+  TestTableLoader() {}
+  ~TestTableLoader() { Reset(); }
 
   void Reset() {
     for (TableInfo* ti : tables) {
@@ -134,7 +134,7 @@ static bool MetadatasEqual(C* ti_a, C* ti_b) {
 // Test the sys-catalog tables basic operations (add, update, delete,
 // visit)
 TEST_F(SysCatalogTest, TestSysCatalogTablesOperations) {
-  TableLoader loader;
+  TestTableLoader loader;
   ASSERT_OK(master_->catalog_manager()->sys_catalog()->VisitTables(&loader));
   ASSERT_EQ(0, loader.tables.size());
 
@@ -222,10 +222,10 @@ TEST_F(SysCatalogTest, TestTableInfoCommit) {
   }
 }
 
-class TabletLoader : public TabletVisitor {
+class TestTabletLoader : public TabletVisitor {
  public:
-  TabletLoader() {}
-  ~TabletLoader() { Reset(); }
+  TestTabletLoader() {}
+  ~TestTabletLoader() { Reset(); }
 
   void Reset() {
     for (TabletInfo* ti : tablets) {
@@ -276,7 +276,7 @@ TEST_F(SysCatalogTest, TestSysCatalogTabletsOperations) {
 
   SysCatalogTable* sys_catalog = master_->catalog_manager()->sys_catalog();
 
-  TabletLoader loader;
+  TestTabletLoader loader;
   ASSERT_OK(master_->catalog_manager()->sys_catalog()->VisitTablets(&loader));
   ASSERT_EQ(0, loader.tablets.size());
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/ddb95e5f/src/kudu/master/sys_catalog.h
----------------------------------------------------------------------
diff --git a/src/kudu/master/sys_catalog.h b/src/kudu/master/sys_catalog.h
index 8496f00..efdef38 100644
--- a/src/kudu/master/sys_catalog.h
+++ b/src/kudu/master/sys_catalog.h
@@ -68,12 +68,14 @@ struct MasterOptions;
 // data must be loaded into memory before the tablets data.
 class TableVisitor {
  public:
+  virtual ~TableVisitor() = default;
   virtual Status VisitTable(const std::string& table_id,
                             const SysTablesEntryPB& metadata) = 0;
 };
 
 class TabletVisitor {
  public:
+  virtual ~TabletVisitor() = default;
   virtual Status VisitTablet(const std::string& table_id,
                              const std::string& tablet_id,
                              const SysTabletsEntryPB& metadata) = 0;
@@ -85,6 +87,7 @@ class TabletVisitor {
 // by current or former master leader.
 class TskEntryVisitor {
  public:
+  virtual ~TskEntryVisitor() = default;
   virtual Status Visit(const std::string& entry_id,
                        const SysTskEntryPB& metadata) = 0;
 };