You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2018/01/17 03:19:19 UTC

[1/2] kudu git commit: mini-cluster: rename data_root to cluster_root

Repository: kudu
Updated Branches:
  refs/heads/master b0d27d3f5 -> c40e0587b


mini-cluster: rename data_root to cluster_root

I found that the name data_root for the root directory in which to place
the cluster's data and logs was a confusing name because:
  1. we use "data" here loosely to describe any old bytes, encompassing
     not only data blocks, but WALs and logs as well, and
  2. the concept of a "data root" already exists to mean a
     user-specified location to place data blocks (i.e. an entry in
     fs_data_dirs).

As such, this patch renames External- and InternalMiniClusterOptions'
"data_root" members to "cluster_root".

Change-Id: Id8ccc9e232f47a684d6b9226fd84639c4c94d0c3
Reviewed-on: http://gerrit.cloudera.org:8080/9033
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Andrew Wong <aw...@cloudera.com>


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

Branch: refs/heads/master
Commit: 5806b807a712116e79e8a67dc4dcc55d51ea6b2a
Parents: b0d27d3
Author: Andrew Wong <an...@g.ucla.edu>
Authored: Tue Jan 16 08:44:24 2018 -0800
Committer: Andrew Wong <aw...@cloudera.com>
Committed: Wed Jan 17 01:06:54 2018 +0000

----------------------------------------------------------------------
 src/kudu/benchmarks/tpch/tpch1.cc              |  2 +-
 src/kudu/benchmarks/tpch/tpch_real_world.cc    |  2 +-
 src/kudu/integration-tests/ts_itest-base.cc    |  4 ++--
 src/kudu/mini-cluster/external_mini_cluster.cc | 22 ++++++++++-----------
 src/kudu/mini-cluster/external_mini_cluster.h  | 10 +++++-----
 src/kudu/mini-cluster/internal_mini_cluster.cc | 14 ++++++-------
 src/kudu/mini-cluster/internal_mini_cluster.h  |  4 ++--
 src/kudu/tools/kudu-tool-test.cc               |  4 ++--
 src/kudu/tools/tool.proto                      |  2 +-
 src/kudu/tools/tool_action_test.cc             | 20 +++++++++----------
 10 files changed, 42 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/5806b807/src/kudu/benchmarks/tpch/tpch1.cc
----------------------------------------------------------------------
diff --git a/src/kudu/benchmarks/tpch/tpch1.cc b/src/kudu/benchmarks/tpch/tpch1.cc
index d8d4a11..2feb823 100644
--- a/src/kudu/benchmarks/tpch/tpch1.cc
+++ b/src/kudu/benchmarks/tpch/tpch1.cc
@@ -261,7 +261,7 @@ int main(int argc, char **argv) {
     kudu::Status s = env->CreateDir(FLAGS_mini_cluster_base_dir);
     CHECK(s.IsAlreadyPresent() || s.ok()) << s.ToString();
     kudu::cluster::InternalMiniClusterOptions options;
-    options.data_root = FLAGS_mini_cluster_base_dir;
+    options.cluster_root = FLAGS_mini_cluster_base_dir;
     cluster.reset(new kudu::cluster::InternalMiniCluster(env, options));
     CHECK_OK(cluster->StartSync());
     master_address = cluster->mini_master()->bound_rpc_addr_str();

http://git-wip-us.apache.org/repos/asf/kudu/blob/5806b807/src/kudu/benchmarks/tpch/tpch_real_world.cc
----------------------------------------------------------------------
diff --git a/src/kudu/benchmarks/tpch/tpch_real_world.cc b/src/kudu/benchmarks/tpch/tpch_real_world.cc
index ce65501..03a25f1 100644
--- a/src/kudu/benchmarks/tpch/tpch_real_world.cc
+++ b/src/kudu/benchmarks/tpch/tpch_real_world.cc
@@ -194,7 +194,7 @@ Status TpchRealWorld::Init() {
 
     ExternalMiniClusterOptions opts;
     opts.num_tablet_servers = 1;
-    opts.data_root = FLAGS_tpch_mini_cluster_base_dir;
+    opts.cluster_root = FLAGS_tpch_mini_cluster_base_dir;
     if (!FLAGS_tpch_path_to_ts_flags_file.empty()) {
       opts.extra_tserver_flags.push_back("--flagfile=" + FLAGS_tpch_path_to_ts_flags_file);
     }

http://git-wip-us.apache.org/repos/asf/kudu/blob/5806b807/src/kudu/integration-tests/ts_itest-base.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/ts_itest-base.cc b/src/kudu/integration-tests/ts_itest-base.cc
index f8344f0..26f4010 100644
--- a/src/kudu/integration-tests/ts_itest-base.cc
+++ b/src/kudu/integration-tests/ts_itest-base.cc
@@ -105,7 +105,7 @@ void TabletServerIntegrationTestBase::AddExtraFlags(
 }
 
 void TabletServerIntegrationTestBase::CreateCluster(
-    const string& data_root_path,
+    const string& cluster_root_path,
     const vector<string>& non_default_ts_flags,
     const vector<string>& non_default_master_flags,
     uint32_t num_data_dirs) {
@@ -118,7 +118,7 @@ void TabletServerIntegrationTestBase::CreateCluster(
 
   cluster::ExternalMiniClusterOptions opts;
   opts.num_tablet_servers = FLAGS_num_tablet_servers;
-  opts.data_root = GetTestPath(data_root_path);
+  opts.cluster_root = GetTestPath(cluster_root_path);
   opts.num_data_dirs = num_data_dirs;
 
   // Enable exactly once semantics for tests.

http://git-wip-us.apache.org/repos/asf/kudu/blob/5806b807/src/kudu/mini-cluster/external_mini_cluster.cc
----------------------------------------------------------------------
diff --git a/src/kudu/mini-cluster/external_mini_cluster.cc b/src/kudu/mini-cluster/external_mini_cluster.cc
index 1495b7b..df5cb12 100644
--- a/src/kudu/mini-cluster/external_mini_cluster.cc
+++ b/src/kudu/mini-cluster/external_mini_cluster.cc
@@ -132,9 +132,9 @@ Status ExternalMiniCluster::HandleOptions() {
     RETURN_NOT_OK(DeduceBinRoot(&opts_.daemon_bin_path));
   }
 
-  if (opts_.data_root.empty()) {
-    // If they don't specify a data root, use the current gtest directory.
-    opts_.data_root = JoinPathSegments(GetTestDataDirectory(), "minicluster-data");
+  if (opts_.cluster_root.empty()) {
+    // If they don't specify a cluster root, use the current gtest directory.
+    opts_.cluster_root = JoinPathSegments(GetTestDataDirectory(), "minicluster-data");
   }
 
   if (opts_.block_manager_type.empty()) {
@@ -157,9 +157,9 @@ Status ExternalMiniCluster::Start() {
                         .Build(&messenger_),
                         "Failed to start Messenger for minicluster");
 
-  Status s = Env::Default()->CreateDir(opts_.data_root);
+  Status s = Env::Default()->CreateDir(opts_.cluster_root);
   if (!s.ok() && !s.IsAlreadyPresent()) {
-    RETURN_NOT_OK_PREPEND(s, "Could not create root dir " + opts_.data_root);
+    RETURN_NOT_OK_PREPEND(s, "Could not create root dir " + opts_.cluster_root);
   }
 
   if (opts_.enable_kerberos) {
@@ -258,13 +258,13 @@ string ExternalMiniCluster::GetBinaryPath(const string& binary) const {
 }
 
 string ExternalMiniCluster::GetLogPath(const string& daemon_id) const {
-  CHECK(!opts_.data_root.empty());
-  return JoinPathSegments(JoinPathSegments(opts_.data_root, daemon_id), "logs");
+  CHECK(!opts_.cluster_root.empty());
+  return JoinPathSegments(JoinPathSegments(opts_.cluster_root, daemon_id), "logs");
 }
 
 string ExternalMiniCluster::GetDataPath(const string& daemon_id,
                                         boost::optional<uint32_t> dir_index) const {
-  CHECK(!opts_.data_root.empty());
+  CHECK(!opts_.cluster_root.empty());
   string data_path = "data";
   if (dir_index) {
     CHECK_LT(*dir_index, opts_.num_data_dirs);
@@ -272,7 +272,7 @@ string ExternalMiniCluster::GetDataPath(const string& daemon_id,
   } else {
     CHECK_EQ(1, opts_.num_data_dirs);
   }
-  return JoinPathSegments(JoinPathSegments(opts_.data_root, daemon_id), data_path);
+  return JoinPathSegments(JoinPathSegments(opts_.cluster_root, daemon_id), data_path);
 }
 
 vector<string> ExternalMiniCluster::GetDataPaths(const string& daemon_id) const {
@@ -287,8 +287,8 @@ vector<string> ExternalMiniCluster::GetDataPaths(const string& daemon_id) const
 }
 
 string ExternalMiniCluster::GetWalPath(const string& daemon_id) const {
-  CHECK(!opts_.data_root.empty());
-  return JoinPathSegments(JoinPathSegments(opts_.data_root, daemon_id), "wal");
+  CHECK(!opts_.cluster_root.empty());
+  return JoinPathSegments(JoinPathSegments(opts_.cluster_root, daemon_id), "wal");
 }
 
 namespace {

http://git-wip-us.apache.org/repos/asf/kudu/blob/5806b807/src/kudu/mini-cluster/external_mini_cluster.h
----------------------------------------------------------------------
diff --git a/src/kudu/mini-cluster/external_mini_cluster.h b/src/kudu/mini-cluster/external_mini_cluster.h
index 0d19bff..17c7c3a 100644
--- a/src/kudu/mini-cluster/external_mini_cluster.h
+++ b/src/kudu/mini-cluster/external_mini_cluster.h
@@ -87,10 +87,10 @@ struct ExternalMiniClusterOptions {
   // Default: 1.
   int num_tablet_servers;
 
-  // Directory in which to store data.
+  // Directory in which to store the cluster's data.
   //
   // Default: "", which auto-generates a unique path for this cluster.
-  std::string data_root;
+  std::string cluster_root;
 
   // Block manager type. Must be either "file" or "log".
   //
@@ -247,8 +247,8 @@ class ExternalMiniCluster : public MiniCluster {
     return hms_.get();
   }
 
-  const std::string& data_root() const {
-    return opts_.data_root;
+  const std::string& cluster_root() const {
+    return opts_.cluster_root;
   }
 
   int num_tablet_servers() const override {
@@ -322,7 +322,7 @@ class ExternalMiniCluster : public MiniCluster {
   std::string GetBinaryPath(const std::string& binary) const;
 
   // Returns the path where 'daemon_id' is expected to store its data, based on
-  // ExternalMiniClusterOptions.data_root if it was provided, or on the
+  // ExternalMiniClusterOptions.cluster_root if it was provided, or on the
   // standard Kudu test directory otherwise.
   // 'dir_index' is an optional numeric suffix to be added to the default path.
   // If it is not specified, the cluster must be configured to use a single data dir.

http://git-wip-us.apache.org/repos/asf/kudu/blob/5806b807/src/kudu/mini-cluster/internal_mini_cluster.cc
----------------------------------------------------------------------
diff --git a/src/kudu/mini-cluster/internal_mini_cluster.cc b/src/kudu/mini-cluster/internal_mini_cluster.cc
index cfd0474..7e342cc 100644
--- a/src/kudu/mini-cluster/internal_mini_cluster.cc
+++ b/src/kudu/mini-cluster/internal_mini_cluster.cc
@@ -73,8 +73,8 @@ InternalMiniCluster::InternalMiniCluster(Env* env, InternalMiniClusterOptions op
   : env_(env),
     opts_(std::move(options)),
     running_(false) {
-  if (opts_.data_root.empty()) {
-    opts_.data_root = JoinPathSegments(GetTestDataDirectory(), "minicluster-data");
+  if (opts_.cluster_root.empty()) {
+    opts_.cluster_root = JoinPathSegments(GetTestDataDirectory(), "minicluster-data");
   }
 }
 
@@ -83,15 +83,15 @@ InternalMiniCluster::~InternalMiniCluster() {
 }
 
 Status InternalMiniCluster::Start() {
-  CHECK(!opts_.data_root.empty()) << "No Fs root was provided";
+  CHECK(!opts_.cluster_root.empty()) << "No cluster root was provided";
   CHECK(!running_);
 
   if (opts_.num_masters > 1) {
     CHECK_GE(opts_.master_rpc_ports.size(), opts_.num_masters);
   }
 
-  if (!env_->FileExists(opts_.data_root)) {
-    RETURN_NOT_OK(env_->CreateDir(opts_.data_root));
+  if (!env_->FileExists(opts_.cluster_root)) {
+    RETURN_NOT_OK(env_->CreateDir(opts_.cluster_root));
   }
 
   // start the masters
@@ -264,11 +264,11 @@ vector<HostPort> InternalMiniCluster::master_rpc_addrs() const {
 }
 
 string InternalMiniCluster::GetMasterFsRoot(int idx) const {
-  return JoinPathSegments(opts_.data_root, Substitute("master-$0-root", idx));
+  return JoinPathSegments(opts_.cluster_root, Substitute("master-$0-root", idx));
 }
 
 string InternalMiniCluster::GetTabletServerFsRoot(int idx) const {
-  return JoinPathSegments(opts_.data_root, Substitute("ts-$0-root", idx));
+  return JoinPathSegments(opts_.cluster_root, Substitute("ts-$0-root", idx));
 }
 
 Status InternalMiniCluster::WaitForTabletServerCount(int count) const {

http://git-wip-us.apache.org/repos/asf/kudu/blob/5806b807/src/kudu/mini-cluster/internal_mini_cluster.h
----------------------------------------------------------------------
diff --git a/src/kudu/mini-cluster/internal_mini_cluster.h b/src/kudu/mini-cluster/internal_mini_cluster.h
index d1e55de..4f73505 100644
--- a/src/kudu/mini-cluster/internal_mini_cluster.h
+++ b/src/kudu/mini-cluster/internal_mini_cluster.h
@@ -70,10 +70,10 @@ struct InternalMiniClusterOptions {
   // Default: 1 (this will place the wals in the same dir)
   int num_data_dirs;
 
-  // Directory in which to store data.
+  // Directory in which to store the cluster's data.
   // Default: "", which auto-generates a unique path for this cluster.
   // The default may only be used from a gtest unit test.
-  std::string data_root;
+  std::string cluster_root;
 
   MiniCluster::BindMode bind_mode;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/5806b807/src/kudu/tools/kudu-tool-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index 0f1a939..2798898 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -1910,7 +1910,7 @@ TEST_P(ControlShellToolTest, TestControlShell) {
   {
     ControlShellRequestPB req;
     ControlShellResponsePB resp;
-    req.mutable_create_cluster()->set_data_root(JoinPathSegments(
+    req.mutable_create_cluster()->set_cluster_root(JoinPathSegments(
         test_dir_, "minicluster-data"));
     req.mutable_create_cluster()->set_num_masters(kNumMasters);
     req.mutable_create_cluster()->set_num_tservers(kNumTservers);
@@ -1922,7 +1922,7 @@ TEST_P(ControlShellToolTest, TestControlShell) {
   {
     ControlShellRequestPB req;
     ControlShellResponsePB resp;
-    req.mutable_create_cluster()->set_data_root(JoinPathSegments(
+    req.mutable_create_cluster()->set_cluster_root(JoinPathSegments(
         test_dir_, "minicluster-data"));
     ASSERT_OK(proto_->SendMessage(req));
     ASSERT_OK(proto_->ReceiveMessage(&resp));

http://git-wip-us.apache.org/repos/asf/kudu/blob/5806b807/src/kudu/tools/tool.proto
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool.proto b/src/kudu/tools/tool.proto
index 1c1e5a3..c591925 100644
--- a/src/kudu/tools/tool.proto
+++ b/src/kudu/tools/tool.proto
@@ -44,7 +44,7 @@ message CreateClusterRequestPB {
   optional bool enable_hive_metastore = 7;
 
   // The directory where the cluster's data and logs should be placed.
-  optional string data_root = 4;
+  optional string cluster_root = 4;
 
   // Any additional gflags for masters or tablet servers. Each should be in a
   // a format that's expected by gflag (i.e. "--foo=bar").

http://git-wip-us.apache.org/repos/asf/kudu/blob/5806b807/src/kudu/tools/tool_action_test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_test.cc b/src/kudu/tools/tool_action_test.cc
index 5e01ae9..c5fc888 100644
--- a/src/kudu/tools/tool_action_test.cc
+++ b/src/kudu/tools/tool_action_test.cc
@@ -68,8 +68,8 @@ using strings::Substitute;
 
 namespace {
 
-Status MakeDataRoot(string* data_root) {
-  // The ExternalMiniCluster can't generate the data root on our behalf because
+Status MakeClusterRoot(string* cluster_root) {
+  // The ExternalMiniCluster can't generate the cluster root on our behalf because
   // we're not running inside a gtest. So we'll use this approach instead,
   // which is what the Java external mini cluster used for a long time.
   const char* tmpdir = getenv("TEST_TMPDIR");
@@ -77,7 +77,7 @@ Status MakeDataRoot(string* data_root) {
   string root = JoinPathSegments(tmpdir_str, "minicluster-data");
   RETURN_NOT_OK(env_util::CreateDirsRecursively(Env::Default(), root));
 
-  *data_root = root;
+  *cluster_root = root;
   return Status::OK();
 }
 
@@ -158,10 +158,10 @@ Status ProcessRequest(const ControlShellRequestPB& req,
       }
       opts.enable_kerberos = cc.enable_kerberos();
       opts.enable_hive_metastore = cc.enable_hive_metastore();
-      if (cc.has_data_root()) {
-        opts.data_root = cc.data_root();
+      if (cc.has_cluster_root()) {
+        opts.cluster_root = cc.cluster_root();
       } else {
-        RETURN_NOT_OK(MakeDataRoot(&opts.data_root));
+        RETURN_NOT_OK(MakeClusterRoot(&opts.cluster_root));
       }
       opts.extra_master_flags.assign(cc.extra_master_flags().begin(),
                                      cc.extra_master_flags().end());
@@ -171,7 +171,7 @@ Status ProcessRequest(const ControlShellRequestPB& req,
         opts.master_rpc_ports = { 11030, 11031, 11032 };
       }
       if (opts.enable_kerberos) {
-        opts.mini_kdc_options.data_root = JoinPathSegments(opts.data_root, "krb5kdc");
+        opts.mini_kdc_options.data_root = JoinPathSegments(opts.cluster_root, "krb5kdc");
       }
 
       cluster->reset(new ExternalMiniCluster(std::move(opts)));
@@ -344,11 +344,11 @@ Status RunControlShell(const RunnerContext& /*context*/) {
     RETURN_NOT_OK(s);
   }
 
-  // Normal exit, clean up data root.
+  // Normal exit, clean up cluster root.
   if (cluster) {
     cluster->Shutdown();
-    WARN_NOT_OK(Env::Default()->DeleteRecursively(cluster->data_root()),
-                "Could not delete data root");
+    WARN_NOT_OK(Env::Default()->DeleteRecursively(cluster->cluster_root()),
+                "Could not delete cluster root");
   }
   return Status::OK();
 }


[2/2] kudu git commit: [catalog_manager] more info if unable to replace a replica

Posted by al...@apache.org.
[catalog_manager] more info if unable to replace a replica

Output actionable warning message when the catalog manager is unable
to find a spot for a replacement replica.  Since the 3-4-3 replication
scheme is now enabled by default, this might be useful in case if
running a cluster with just 3 tablet servers when tables have
replication factor of 3.

Change-Id: Id5f562c6d1ff526daa785ea535e440598c03cd37
Reviewed-on: http://gerrit.cloudera.org:8080/9040
Reviewed-by: Mike Percy <mp...@apache.org>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: c40e0587bf6a6aa55e5bd72dd2dd9356b1507f2e
Parents: 5806b80
Author: Alexey Serbin <as...@cloudera.com>
Authored: Tue Jan 16 15:01:34 2018 -0800
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Wed Jan 17 03:18:11 2018 +0000

----------------------------------------------------------------------
 src/kudu/master/catalog_manager.cc | 38 ++++++++++++++++++++++++++++++---
 src/kudu/master/ts_manager.cc      |  4 ++--
 src/kudu/master/ts_manager.h       |  8 +++----
 3 files changed, 41 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/c40e0587/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index d6b369b..1248144 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -3133,11 +3133,43 @@ bool AsyncAddReplicaTask::SendRequest(int attempt) {
       InsertOrDie(&excluded, ts_desc);
     }
   }
+
   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();
+    auto msg = Substitute("no candidate replacement replica found for tablet $0",
+                          tablet_->ToString());
+    // Check whether it's a situation when a replacement replica cannot be found
+    // due to an inconsistency in cluster configuration. If the tablet has the
+    // replication factor of N, and the cluster is configured to use N->(N+1)->N
+    // replication scheme (see --raft_prepare_replacement_before_eviction flag),
+    // at least N+1 tablet servers should be registered to find a place
+    // for a replacement replica.
+    TSDescriptorVector all_descriptors;
+    master_->ts_manager()->GetAllDescriptors(&all_descriptors);
+    const auto num_tservers_registered = all_descriptors.size();
+
+    auto replication_factor = 0;
+    {
+      TableMetadataLock l(tablet_->table().get(), LockMode::READ);
+      replication_factor = tablet_->table()->metadata().state().pb.num_replicas();
+    }
+    DCHECK_GE(replication_factor, 0);
+    const auto num_tservers_needed =
+        FLAGS_raft_prepare_replacement_before_eviction ? replication_factor + 1
+                                                       : replication_factor;
+    if (num_tservers_registered < num_tservers_needed) {
+      msg += Substitute(
+          "; the total number of registered tablet servers ($0) does not allow "
+          "for replacement of the failed replica: at least $1 tablet servers "
+          "are required", num_tservers_registered, num_tservers_needed);
+      if (FLAGS_raft_prepare_replacement_before_eviction &&
+          num_tservers_registered == replication_factor) {
+        msg +=
+          "; consider either adding an additional tablet server or running "
+          "the cluster with --raft_prepare_replacement_before_eviction=false";
+      }
+    }
+    KLOG_EVERY_N_SECS(WARNING, 60) << LogPrefix() << msg;
     return false;
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/c40e0587/src/kudu/master/ts_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/ts_manager.cc b/src/kudu/master/ts_manager.cc
index 65faad8..8b1aa1c 100644
--- a/src/kudu/master/ts_manager.cc
+++ b/src/kudu/master/ts_manager.cc
@@ -92,13 +92,13 @@ Status TSManager::RegisterTS(const NodeInstancePB& instance,
   return Status::OK();
 }
 
-void TSManager::GetAllDescriptors(vector<shared_ptr<TSDescriptor> > *descs) const {
+void TSManager::GetAllDescriptors(vector<shared_ptr<TSDescriptor>> *descs) const {
   descs->clear();
   shared_lock<rw_spinlock> l(lock_);
   AppendValuesFromMap(servers_by_id_, descs);
 }
 
-void TSManager::GetAllLiveDescriptors(vector<shared_ptr<TSDescriptor> > *descs) const {
+void TSManager::GetAllLiveDescriptors(vector<shared_ptr<TSDescriptor>> *descs) const {
   descs->clear();
 
   shared_lock<rw_spinlock> l(lock_);

http://git-wip-us.apache.org/repos/asf/kudu/blob/c40e0587/src/kudu/master/ts_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/master/ts_manager.h b/src/kudu/master/ts_manager.h
index ecdd5cd..8327203 100644
--- a/src/kudu/master/ts_manager.h
+++ b/src/kudu/master/ts_manager.h
@@ -35,7 +35,7 @@ namespace master {
 
 class TSDescriptor;
 
-typedef std::vector<std::shared_ptr<TSDescriptor> > TSDescriptorVector;
+typedef std::vector<std::shared_ptr<TSDescriptor>> TSDescriptorVector;
 
 // Tracks the servers that the master has heard from, along with their
 // last heartbeat, etc.
@@ -74,11 +74,11 @@ class TSManager {
 
   // Return all of the currently registered TS descriptors into the provided
   // list.
-  void GetAllDescriptors(std::vector<std::shared_ptr<TSDescriptor> >* descs) const;
+  void GetAllDescriptors(std::vector<std::shared_ptr<TSDescriptor>>* descs) const;
 
   // Return all of the currently registered TS descriptors that have sent a
   // heartbeat recently, indicating that they're alive and well.
-  void GetAllLiveDescriptors(std::vector<std::shared_ptr<TSDescriptor> >* descs) const;
+  void GetAllLiveDescriptors(std::vector<std::shared_ptr<TSDescriptor>>* descs) const;
 
   // Get the TS count.
   int GetCount() const;
@@ -87,7 +87,7 @@ class TSManager {
   mutable rw_spinlock lock_;
 
   typedef std::unordered_map<
-    std::string, std::shared_ptr<TSDescriptor> > TSDescriptorMap;
+    std::string, std::shared_ptr<TSDescriptor>> TSDescriptorMap;
   TSDescriptorMap servers_by_id_;
 
   DISALLOW_COPY_AND_ASSIGN(TSManager);