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 2017/06/22 22:37:21 UTC

kudu git commit: [mini_{master,tablet_server}] micro clean-up

Repository: kudu
Updated Branches:
  refs/heads/master d2b820594 -> d62394293


[mini_{master,tablet_server}] micro clean-up

A micro clean-up on mini_master and mini_tablet_server:
  * remove unused FsManager member from MiniTabletServer
  * use std::unique_ptr instead of gscoped_ptr
  * get rid of started_ member
  * removed redundant CHECK() statements
  * other unsorted micro-updates

Change-Id: I9fc34b6f8ff94098551e567f4aefa4e92a907f53
Reviewed-on: http://gerrit.cloudera.org:8080/7265
Tested-by: Kudu Jenkins
Reviewed-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/d6239429
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/d6239429
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/d6239429

Branch: refs/heads/master
Commit: d623942935e22aa2f22815a9b37397d84ffba46f
Parents: d2b8205
Author: Alexey Serbin <as...@cloudera.com>
Authored: Thu Jun 22 11:16:13 2017 -0700
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Thu Jun 22 22:35:17 2017 +0000

----------------------------------------------------------------------
 .../master_cert_authority-itest.cc              |  4 +-
 src/kudu/integration-tests/mini_cluster.cc      |  2 +-
 src/kudu/master/mini_master.cc                  | 36 ++++++-----------
 src/kudu/master/mini_master.h                   | 15 +++----
 src/kudu/tserver/mini_tablet_server.cc          | 42 +++++++++-----------
 src/kudu/tserver/mini_tablet_server.h           | 18 ++++-----
 6 files changed, 46 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/d6239429/src/kudu/integration-tests/master_cert_authority-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/master_cert_authority-itest.cc b/src/kudu/integration-tests/master_cert_authority-itest.cc
index 6d6016a..48331c0 100644
--- a/src/kudu/integration-tests/master_cert_authority-itest.cc
+++ b/src/kudu/integration-tests/master_cert_authority-itest.cc
@@ -115,7 +115,7 @@ class MasterCertAuthorityTest : public KuduTest {
       req.mutable_registration()->CopyFrom(fake_reg);
 
       MiniMaster* m = cluster_->mini_master(i);
-      if (!m->is_running()) {
+      if (!m->is_started()) {
         continue;
       }
       MasterServiceProxy proxy(messenger_, m->bound_rpc_addr());
@@ -145,7 +145,7 @@ class MasterCertAuthorityTest : public KuduTest {
       req.set_csr_der(csr_str);
 
       MiniMaster* m = cluster_->mini_master(i);
-      if (!m->is_running()) {
+      if (!m->is_started()) {
         continue;
       }
       MasterServiceProxy proxy(messenger_, m->bound_rpc_addr());

http://git-wip-us.apache.org/repos/asf/kudu/blob/d6239429/src/kudu/integration-tests/mini_cluster.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/mini_cluster.cc b/src/kudu/integration-tests/mini_cluster.cc
index da57fb0..57d8ea4 100644
--- a/src/kudu/integration-tests/mini_cluster.cc
+++ b/src/kudu/integration-tests/mini_cluster.cc
@@ -306,7 +306,7 @@ Status MiniCluster::GetLeaderMasterIndex(int* idx) const {
   while (MonoTime::Now() < deadline) {
     for (int i = 0; i < num_masters(); i++) {
       master::MiniMaster* mm = mini_master(i);
-      if (!mm->is_running() || mm->master()->IsShutdown()) {
+      if (!mm->is_started() || mm->master()->IsShutdown()) {
         continue;
       }
       master::CatalogManager* catalog = mm->master()->catalog_manager();

http://git-wip-us.apache.org/repos/asf/kudu/blob/d6239429/src/kudu/master/mini_master.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/mini_master.cc b/src/kudu/master/mini_master.cc
index 63188b5..0862869 100644
--- a/src/kudu/master/mini_master.cc
+++ b/src/kudu/master/mini_master.cc
@@ -17,19 +17,24 @@
 
 #include "kudu/master/mini_master.h"
 
+#include <memory>
 #include <string>
+#include <vector>
 
 #include <glog/logging.h>
 
 #include "kudu/fs/fs_manager.h"
 #include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/master/master.h"
 #include "kudu/server/rpc_server.h"
 #include "kudu/server/webserver.h"
-#include "kudu/master/master.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/status.h"
 
+using std::string;
+using std::unique_ptr;
+using std::vector;
 using strings::Substitute;
 
 DECLARE_bool(enable_minidumps);
@@ -39,8 +44,7 @@ namespace kudu {
 namespace master {
 
 MiniMaster::MiniMaster(Env* env, string fs_root, uint16_t rpc_port)
-    : running_(false),
-      env_(env),
+    : env_(env),
       fs_root_(std::move(fs_root)),
       rpc_port_(rpc_port) {
   // Disable minidump handler (we allow only one per process).
@@ -52,32 +56,27 @@ MiniMaster::~MiniMaster() {
 }
 
 Status MiniMaster::Start() {
-  CHECK(!running_);
   FLAGS_rpc_server_allow_ephemeral_ports = true;
   RETURN_NOT_OK(StartOnPorts(rpc_port_, 0));
   return master_->WaitForCatalogManagerInit();
 }
 
 Status MiniMaster::StartDistributedMaster(const vector<uint16_t>& peer_ports) {
-  CHECK(!running_);
   return StartDistributedMasterOnPorts(rpc_port_, 0, peer_ports);
 }
 
 Status MiniMaster::Restart() {
-  CHECK(!running_);
   RETURN_NOT_OK(StartOnPorts(bound_rpc_.port(), bound_http_.port()));
-  CHECK(running_);
   return WaitForCatalogManagerInit();
 }
 
 void MiniMaster::Shutdown() {
-  if (running_) {
+  if (master_) {
     bound_rpc_ = bound_rpc_addr();
     bound_http_ = bound_http_addr();
     master_->Shutdown();
+    master_.reset();
   }
-  running_ = false;
-  master_.reset();
 }
 
 Status MiniMaster::WaitForCatalogManagerInit() const {
@@ -85,17 +84,14 @@ Status MiniMaster::WaitForCatalogManagerInit() const {
 }
 
 const Sockaddr MiniMaster::bound_rpc_addr() const {
-  CHECK(running_);
   return master_->first_rpc_address();
 }
 
 const Sockaddr MiniMaster::bound_http_addr() const {
-  CHECK(running_);
   return master_->first_http_address();
 }
 
 std::string MiniMaster::permanent_uuid() const {
-  CHECK(master_);
   return DCHECK_NOTNULL(master_->fs_manager())->uuid();
 }
 
@@ -104,42 +100,34 @@ std::string MiniMaster::bound_rpc_addr_str() const {
 }
 
 Status MiniMaster::StartOnPorts(uint16_t rpc_port, uint16_t web_port) {
-  CHECK(!running_);
-  CHECK(!master_);
-
   MasterOptions opts;
   return StartOnPorts(rpc_port, web_port, &opts);
 }
 
 Status MiniMaster::StartOnPorts(uint16_t rpc_port, uint16_t web_port,
                                 MasterOptions* opts) {
+  CHECK(!master_);
   opts->rpc_opts.rpc_bind_addresses = Substitute("127.0.0.1:$0", rpc_port);
   opts->webserver_opts.port = web_port;
   opts->fs_opts.wal_path = fs_root_;
   opts->fs_opts.data_paths = { fs_root_ };
 
-  gscoped_ptr<Master> server(new Master(*opts));
+  unique_ptr<Master> server(new Master(*opts));
   RETURN_NOT_OK(server->Init());
   RETURN_NOT_OK(server->StartAsync());
-
   master_.swap(server);
-  running_ = true;
 
   return Status::OK();
 }
 
 Status MiniMaster::StartDistributedMasterOnPorts(uint16_t rpc_port, uint16_t web_port,
                                                  const vector<uint16_t>& peer_ports) {
-  CHECK(!running_);
-  CHECK(!master_);
-
-  MasterOptions opts;
-
   vector<HostPort> peer_addresses;
   for (uint16_t peer_port : peer_ports) {
     HostPort peer_address("127.0.0.1", peer_port);
     peer_addresses.push_back(peer_address);
   }
+  MasterOptions opts;
   opts.master_addresses = peer_addresses;
 
   return StartOnPorts(rpc_port, web_port, &opts);

http://git-wip-us.apache.org/repos/asf/kudu/blob/d6239429/src/kudu/master/mini_master.h
----------------------------------------------------------------------
diff --git a/src/kudu/master/mini_master.h b/src/kudu/master/mini_master.h
index fe9777e..b509b3b 100644
--- a/src/kudu/master/mini_master.h
+++ b/src/kudu/master/mini_master.h
@@ -14,9 +14,10 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-#ifndef KUDU_MASTER_MINI_MASTER_H
-#define KUDU_MASTER_MINI_MASTER_H
 
+#pragma once
+
+#include <memory>
 #include <string>
 #include <vector>
 
@@ -60,7 +61,7 @@ class MiniMaster {
 
   Status WaitForCatalogManagerInit() const;
 
-  bool is_running() const { return running_; }
+  bool is_started() const { return master_ ? true : false; }
 
   const Sockaddr bound_rpc_addr() const;
   const Sockaddr bound_http_addr() const;
@@ -77,22 +78,18 @@ class MiniMaster {
   Status StartOnPorts(uint16_t rpc_port, uint16_t web_port);
 
   Status StartOnPorts(uint16_t rpc_port, uint16_t web_port,
-                      MasterOptions* options);
+                      MasterOptions* opts);
 
   Status StartDistributedMasterOnPorts(uint16_t rpc_port, uint16_t web_port,
                                        const std::vector<uint16_t>& peer_ports);
 
-  bool running_;
-
   ATTRIBUTE_MEMBER_UNUSED Env* const env_;
   const std::string fs_root_;
   const uint16_t rpc_port_;
   Sockaddr bound_rpc_;
   Sockaddr bound_http_;
-  gscoped_ptr<Master> master_;
+  std::unique_ptr<Master> master_;
 };
 
 } // namespace master
 } // namespace kudu
-
-#endif /* KUDU_MASTER_MINI_MASTER_H */

http://git-wip-us.apache.org/repos/asf/kudu/blob/d6239429/src/kudu/tserver/mini_tablet_server.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/mini_tablet_server.cc b/src/kudu/tserver/mini_tablet_server.cc
index 67a2734..9ee60b4 100644
--- a/src/kudu/tserver/mini_tablet_server.cc
+++ b/src/kudu/tserver/mini_tablet_server.cc
@@ -17,8 +17,11 @@
 
 #include "kudu/tserver/mini_tablet_server.h"
 
+#include <string>
 #include <utility>
 
+#include <gflags/gflags_declare.h>
+
 #include "kudu/consensus/metadata.pb.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/tablet/tablet-test-util.h"
@@ -27,22 +30,21 @@
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/status.h"
 
-using std::pair;
+DECLARE_bool(enable_minidumps);
+DECLARE_bool(rpc_server_allow_ephemeral_ports);
 
-using kudu::consensus::RaftPeerPB;
 using kudu::consensus::RaftConfigPB;
+using kudu::consensus::RaftPeerPB;
+using std::pair;
+using std::string;
+using std::unique_ptr;
 using strings::Substitute;
 
-DECLARE_bool(enable_minidumps);
-DECLARE_bool(rpc_server_allow_ephemeral_ports);
-
 namespace kudu {
 namespace tserver {
 
 MiniTabletServer::MiniTabletServer(const string& fs_root,
-                                   uint16_t rpc_port)
-  : started_(false) {
-
+                                   uint16_t rpc_port) {
   // Disable minidump handler (we allow only one per process).
   FLAGS_enable_minidumps = false;
   // Start RPC server on loopback.
@@ -58,14 +60,13 @@ MiniTabletServer::~MiniTabletServer() {
 }
 
 Status MiniTabletServer::Start() {
-  CHECK(!started_);
+  CHECK(!server_);
 
-  gscoped_ptr<TabletServer> server(new TabletServer(opts_));
+  unique_ptr<TabletServer> server(new TabletServer(opts_));
   RETURN_NOT_OK(server->Init());
   RETURN_NOT_OK(server->Start());
-
   server_.swap(server);
-  started_ = true;
+
   return Status::OK();
 }
 
@@ -74,7 +75,7 @@ Status MiniTabletServer::WaitStarted() {
 }
 
 void MiniTabletServer::Shutdown() {
-  if (started_) {
+  if (server_) {
     // Save the bound ports back into the options structure so that, if we restart the
     // server, it will come back on the same address. This is necessary since we don't
     // currently support tablet servers re-registering on different ports (KUDU-418).
@@ -83,17 +84,14 @@ void MiniTabletServer::Shutdown() {
     server_->Shutdown();
     server_.reset();
   }
-  started_ = false;
 }
 
 Status MiniTabletServer::Restart() {
-  CHECK(!started_);
-  RETURN_NOT_OK(Start());
-  return Status::OK();
+  return Start();
 }
 
 RaftConfigPB MiniTabletServer::CreateLocalConfig() const {
-  CHECK(started_) << "Must Start()";
+  CHECK(server_) << "must call Start() first";
   RaftConfigPB config;
   RaftPeerPB* peer = config.add_peers();
   peer->set_permanent_uuid(server_->instance_pb().permanent_uuid());
@@ -113,13 +111,12 @@ Status MiniTabletServer::AddTestTablet(const std::string& table_id,
                                        const std::string& tablet_id,
                                        const Schema& schema,
                                        const RaftConfigPB& config) {
-  CHECK(started_) << "Must Start()";
   Schema schema_with_ids = SchemaBuilder(schema).Build();
   pair<PartitionSchema, Partition> partition = tablet::CreateDefaultPartition(schema_with_ids);
 
   return server_->tablet_manager()->CreateNewTablet(
-    table_id, tablet_id, partition.second, table_id,
-    schema_with_ids, partition.first, config, nullptr);
+      table_id, tablet_id, partition.second, table_id,
+      schema_with_ids, partition.first, config, nullptr);
 }
 
 void MiniTabletServer::FailHeartbeats() {
@@ -127,17 +124,14 @@ void MiniTabletServer::FailHeartbeats() {
 }
 
 const Sockaddr MiniTabletServer::bound_rpc_addr() const {
-  CHECK(started_);
   return server_->first_rpc_address();
 }
 
 const Sockaddr MiniTabletServer::bound_http_addr() const {
-  CHECK(started_);
   return server_->first_http_address();
 }
 
 const string& MiniTabletServer::uuid() const {
-  CHECK(started_);
   return server_->fs_manager()->uuid();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/d6239429/src/kudu/tserver/mini_tablet_server.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/mini_tablet_server.h b/src/kudu/tserver/mini_tablet_server.h
index 34ed945..ffa8b59 100644
--- a/src/kudu/tserver/mini_tablet_server.h
+++ b/src/kudu/tserver/mini_tablet_server.h
@@ -14,8 +14,11 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-#ifndef KUDU_TSERVER_MINI_TABLET_SERVER_H
-#define KUDU_TSERVER_MINI_TABLET_SERVER_H
+
+#pragma once
+
+#include <memory>
+#include <string>
 
 #include "kudu/common/schema.h"
 #include "kudu/gutil/macros.h"
@@ -23,8 +26,6 @@
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/status.h"
 
-#include <string>
-
 namespace kudu {
 
 class FsManager;
@@ -90,19 +91,14 @@ class MiniTabletServer {
   // Return TS uuid.
   const std::string& uuid() const;
 
-  bool is_started() const { return started_; }
+  bool is_started() const { return server_ ? true : false; }
 
   void FailHeartbeats();
 
  private:
-  bool started_;
-
   TabletServerOptions opts_;
-
-  gscoped_ptr<FsManager> fs_manager_;
-  gscoped_ptr<TabletServer> server_;
+  std::unique_ptr<TabletServer> server_;
 };
 
 } // namespace tserver
 } // namespace kudu
-#endif