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/01/12 08:23:26 UTC

[2/8] kudu git commit: KUDU-1741: Keep MiniCluster::Restart consistent with ExternalMiniCluster::Restart

KUDU-1741: Keep MiniCluster::Restart consistent with ExternalMiniCluster::Restart

MiniCluster and ExternalMiniCluster utility classes follow opposite
semantics for restart. For eg, ExternalMiniCluster::Restart() expects
that all the nodes must be shutdown before we restart, whereas
MiniCluster::Restart() expects that all servers are up and running
so that it can internally execute shutdown and start in that order.

This patch changes the semantics of MiniCluster::Restart() to follow
that of ExternalMiniCluster::Restart() to keep them consistent and
the existing tests are modified to follow the new semantics of
MiniCluster::Restart().

Change-Id: Iad33b7c46bfca3f277ccbca7d0420272f06a6633
Reviewed-on: http://gerrit.cloudera.org:8080/5598
Tested-by: Kudu Jenkins
Reviewed-by: Jean-Daniel Cryans <jd...@apache.org>
Reviewed-by: Mike Percy <mp...@apache.org>
Reviewed-by: Alexey Serbin <as...@cloudera.com>


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

Branch: refs/heads/master
Commit: 4114ba9bf1cdfbd03eef2a13d6e51b6094527997
Parents: 78feb33
Author: Dinesh Bhat <di...@cloudera.com>
Authored: Wed Jan 4 12:09:11 2017 -0800
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Wed Jan 11 02:06:56 2017 +0000

----------------------------------------------------------------------
 src/kudu/client/client-test.cc                         |  3 +++
 src/kudu/integration-tests/alter_table-test.cc         |  1 +
 src/kudu/integration-tests/create-table-stress-test.cc |  1 +
 src/kudu/integration-tests/fuzz-itest.cc               |  1 +
 src/kudu/integration-tests/registration-test.cc        |  2 ++
 src/kudu/master/master-test.cc                         |  2 ++
 src/kudu/master/mini_master.cc                         | 11 ++++-------
 src/kudu/master/mini_master.h                          |  3 ++-
 src/kudu/tserver/mini_tablet_server.cc                 |  3 +--
 9 files changed, 17 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/4114ba9b/src/kudu/client/client-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-test.cc b/src/kudu/client/client-test.cc
index a3cfaf3..95b8b97 100644
--- a/src/kudu/client/client-test.cc
+++ b/src/kudu/client/client-test.cc
@@ -526,6 +526,7 @@ class ClientTest : public KuduTest {
       if (ts->server()->instance_pb().permanent_uuid() == uuid) {
         if (restart) {
           LOG(INFO) << "Restarting TS at " << ts->bound_rpc_addr().ToString();
+          ts->Shutdown();
           RETURN_NOT_OK(ts->Restart());
           if (wait_started) {
             LOG(INFO) << "Waiting for TS " << ts->bound_rpc_addr().ToString()
@@ -4171,6 +4172,7 @@ TEST_F(ClientTest, TestServerTooBusyRetry) {
   for (int i = 0; i < cluster_->num_tablet_servers(); i++) {
     MiniTabletServer* ts = cluster_->mini_tablet_server(i);
     ts->options()->rpc_opts.service_queue_length = 1;
+    ts->Shutdown();
     ASSERT_OK(ts->Restart());
     ASSERT_OK(ts->WaitStarted());
   }
@@ -4213,6 +4215,7 @@ TEST_F(ClientTest, TestLastErrorEmbeddedInScanTimeoutStatus) {
     FLAGS_log_inject_latency_ms_stddev = 0;
     for (int i = 0; i < cluster_->num_tablet_servers(); i++) {
       MiniTabletServer* ts = cluster_->mini_tablet_server(i);
+      ts->Shutdown();
       ASSERT_OK(ts->Restart());
     }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/4114ba9b/src/kudu/integration-tests/alter_table-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/alter_table-test.cc b/src/kudu/integration-tests/alter_table-test.cc
index 9a08065..8e0e13f 100644
--- a/src/kudu/integration-tests/alter_table-test.cc
+++ b/src/kudu/integration-tests/alter_table-test.cc
@@ -154,6 +154,7 @@ class AlterTableTest : public KuduTest {
   void RestartTabletServer(int idx = 0) {
     tablet_peer_.reset();
     if (cluster_->mini_tablet_server(idx)->server()) {
+      cluster_->mini_tablet_server(idx)->Shutdown();
       ASSERT_OK(cluster_->mini_tablet_server(idx)->Restart());
     } else {
       ASSERT_OK(cluster_->mini_tablet_server(idx)->Start());

http://git-wip-us.apache.org/repos/asf/kudu/blob/4114ba9b/src/kudu/integration-tests/create-table-stress-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/create-table-stress-test.cc b/src/kudu/integration-tests/create-table-stress-test.cc
index 97e7955..c301152 100644
--- a/src/kudu/integration-tests/create-table-stress-test.cc
+++ b/src/kudu/integration-tests/create-table-stress-test.cc
@@ -185,6 +185,7 @@ TEST_F(CreateTableStressTest, RestartMasterDuringCreation) {
   for (int i = 0; i < 3; i++) {
     SleepFor(MonoDelta::FromMicroseconds(500));
     LOG(INFO) << "Restarting master...";
+    cluster_->mini_master()->Shutdown();
     ASSERT_OK(cluster_->mini_master()->Restart());
     ASSERT_OK(cluster_->mini_master()->master()->
         WaitUntilCatalogManagerIsLeaderAndReadyForTests(MonoDelta::FromSeconds(5)));

http://git-wip-us.apache.org/repos/asf/kudu/blob/4114ba9b/src/kudu/integration-tests/fuzz-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/fuzz-itest.cc b/src/kudu/integration-tests/fuzz-itest.cc
index 00cd2ec..b88148d 100644
--- a/src/kudu/integration-tests/fuzz-itest.cc
+++ b/src/kudu/integration-tests/fuzz-itest.cc
@@ -190,6 +190,7 @@ class FuzzTest : public KuduTest {
     tablet_peer_.reset();
     auto ts = cluster_->mini_tablet_server(0);
     if (ts->server()) {
+      ts->Shutdown();
       ASSERT_OK(ts->Restart());
     } else {
       ASSERT_OK(ts->Start());

http://git-wip-us.apache.org/repos/asf/kudu/blob/4114ba9b/src/kudu/integration-tests/registration-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/registration-test.cc b/src/kudu/integration-tests/registration-test.cc
index 4c35c0a..12ea54a 100644
--- a/src/kudu/integration-tests/registration-test.cc
+++ b/src/kudu/integration-tests/registration-test.cc
@@ -133,6 +133,7 @@ TEST_F(RegistrationTest, TestTSRegisters) {
 
   // Restart the master, so it loses the descriptor, and ensure that the
   // hearbeater thread handles re-registering.
+  cluster_->mini_master()->Shutdown();
   ASSERT_OK(cluster_->mini_master()->Restart());
 
   ASSERT_OK(cluster_->WaitForTabletServerCount(1));
@@ -185,6 +186,7 @@ TEST_F(RegistrationTest, TestTabletReports) {
   // Shut down the whole system, bring it back up, and make sure the tablets
   // are reported.
   ts->Shutdown();
+  cluster_->mini_master()->Shutdown();
   ASSERT_OK(cluster_->mini_master()->Restart());
   ASSERT_OK(ts->Start());
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/4114ba9b/src/kudu/master/master-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master-test.cc b/src/kudu/master/master-test.cc
index 0899336..c5979e6 100644
--- a/src/kudu/master/master-test.cc
+++ b/src/kudu/master/master-test.cc
@@ -398,6 +398,7 @@ TEST_F(MasterTest, TestCatalog) {
   ASSERT_OK(CreateTable(kTableName, kTableSchema));
 
   // Restart the master, verify the table still shows up.
+  mini_master_->Shutdown();
   ASSERT_OK(mini_master_->Restart());
   ASSERT_OK(mini_master_->master()->
       WaitUntilCatalogManagerIsLeaderAndReadyForTests(MonoDelta::FromSeconds(5)));
@@ -1055,6 +1056,7 @@ TEST_F(MasterTest, TestMasterMetadataConsistentDespiteFailures) {
 
   // Restart the catalog manager to ensure that it can survive reloading the
   // metadata we wrote to disk.
+  mini_master_->Shutdown();
   ASSERT_OK(mini_master_->Restart());
 
   // Reload the metadata again, this time verifying its consistency.

http://git-wip-us.apache.org/repos/asf/kudu/blob/4114ba9b/src/kudu/master/mini_master.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/mini_master.cc b/src/kudu/master/mini_master.cc
index 9f8b944..927bd24 100644
--- a/src/kudu/master/mini_master.cc
+++ b/src/kudu/master/mini_master.cc
@@ -62,6 +62,8 @@ Status MiniMaster::StartDistributedMaster(const vector<uint16_t>& peer_ports) {
 
 void MiniMaster::Shutdown() {
   if (running_) {
+    bound_rpc_ = bound_rpc_addr();
+    bound_http_ = bound_http_addr();
     master_->Shutdown();
   }
   running_ = false;
@@ -111,13 +113,8 @@ Status MiniMaster::StartDistributedMasterOnPorts(uint16_t rpc_port, uint16_t web
 }
 
 Status MiniMaster::Restart() {
-  CHECK(running_);
-
-  Sockaddr prev_rpc = bound_rpc_addr();
-  Sockaddr prev_http = bound_http_addr();
-  Shutdown();
-
-  RETURN_NOT_OK(StartOnPorts(prev_rpc.port(), prev_http.port()));
+  CHECK(!running_);
+  RETURN_NOT_OK(StartOnPorts(bound_rpc_.port(), bound_http_.port()));
   CHECK(running_);
   return WaitForCatalogManagerInit();
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/4114ba9b/src/kudu/master/mini_master.h
----------------------------------------------------------------------
diff --git a/src/kudu/master/mini_master.h b/src/kudu/master/mini_master.h
index fbd3c80..f807a54 100644
--- a/src/kudu/master/mini_master.h
+++ b/src/kudu/master/mini_master.h
@@ -85,7 +85,8 @@ class MiniMaster {
   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_;
 };
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/4114ba9b/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 92f1ad2..8b617c0 100644
--- a/src/kudu/tserver/mini_tablet_server.cc
+++ b/src/kudu/tserver/mini_tablet_server.cc
@@ -101,8 +101,7 @@ void MiniTabletServer::Shutdown() {
 }
 
 Status MiniTabletServer::Restart() {
-  CHECK(started_);
-  Shutdown();
+  CHECK(!started_);
   RETURN_NOT_OK(Start());
   return Status::OK();
 }