You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by mp...@apache.org on 2017/08/30 01:55:56 UTC

kudu git commit: KUDU-2118. Fully shut down TabletReplica on delete

Repository: kudu
Updated Branches:
  refs/heads/master 1e332fe6e -> 2f78643e4


KUDU-2118. Fully shut down TabletReplica on delete

After the various lifecycle changes implemented in
5bca7d8ba185d62952fb3e3163cbe88d20453da0 we were left with a case where
we fail to fully shut down TABLET_DATA_DELETED replicas before
dereferencing them. Because LeaderElection via ElectionDecisionCallback
will hold a reference to RaftConsensus while a vote is taking place, the
leader election callback running on the reactor thread may destroy the
RaftConsensus object. If that occurs without RaftConsensus already being
in a fully-shutdown state, the RaftConsensus destructor runs Shutdown()
before destoying the object, which takes a lock and triggers an
assertion via AssertWaitAllowed(). This crashes the server.

This patch fixes the issue by calling Shutdown() on TABLET_DATA_DELETED
replicas before dereferencing them in TSTabletManager.

A concurrency test is included that spawns threads to start elections
while the tablets are being deleted. This test pretty reliably fails
without the included fix, and is solidly stable with the fix.

Change-Id: I4a0ea46f220a10d4de680fc73d45d14426e0395e
Reviewed-on: http://gerrit.cloudera.org:8080/7883
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/2f78643e
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/2f78643e
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/2f78643e

Branch: refs/heads/master
Commit: 2f78643e4979fc8a9499498aa04c7f4ffa0deb61
Parents: 1e332fe
Author: Mike Percy <mp...@apache.org>
Authored: Tue Aug 29 13:44:46 2017 -0700
Committer: Mike Percy <mp...@apache.org>
Committed: Wed Aug 30 01:55:35 2017 +0000

----------------------------------------------------------------------
 .../integration-tests/delete_tablet-itest.cc    | 55 +++++++++++++++++++-
 src/kudu/tserver/ts_tablet_manager.cc           |  9 ++--
 2 files changed, 59 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/2f78643e/src/kudu/integration-tests/delete_tablet-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/delete_tablet-itest.cc b/src/kudu/integration-tests/delete_tablet-itest.cc
index 447a4cf..0dce3fb 100644
--- a/src/kudu/integration-tests/delete_tablet-itest.cc
+++ b/src/kudu/integration-tests/delete_tablet-itest.cc
@@ -15,13 +15,17 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <atomic>
 #include <cstdint>
 #include <memory>
+#include <string>
+#include <thread>
 #include <unordered_map>
 #include <vector>
 
 #include <boost/optional/optional.hpp>
 #include <gflags/gflags_declare.h>
+#include <glog/stl_logging.h>
 #include <gtest/gtest.h>
 
 #include "kudu/gutil/ref_counted.h"
@@ -43,7 +47,11 @@
 
 DECLARE_int64(fs_wal_dir_reserved_bytes);
 
+using kudu::tablet::TABLET_DATA_DELETED;
 using kudu::tablet::TabletReplica;
+using std::atomic;
+using std::string;
+using std::thread;
 using std::vector;
 
 namespace kudu {
@@ -53,7 +61,7 @@ class DeleteTabletITest : public MiniClusterITestBase {
 
 // Test deleting a failed replica. Regression test for KUDU-1607.
 TEST_F(DeleteTabletITest, TestDeleteFailedReplica) {
-  NO_FATALS(StartCluster(1)); // 1 TS.
+  NO_FATALS(StartCluster(/*num_tablet_servers=*/ 1));
   TestWorkload workload(cluster_.get());
   workload.set_num_replicas(1);
   workload.Setup();
@@ -111,4 +119,49 @@ TEST_F(DeleteTabletITest, TestDeleteFailedReplica) {
   });
 }
 
+// Test that a leader election will not cause a crash when a tablet is deleted.
+// Regression test for KUDU-2118.
+TEST_F(DeleteTabletITest, TestLeaderElectionDuringDeleteTablet) {
+  const MonoDelta kTimeout = MonoDelta::FromSeconds(30);
+  const int kNumTablets = 10;
+  NO_FATALS(StartCluster());
+  TestWorkload workload(cluster_.get());
+  workload.set_num_tablets(kNumTablets);
+  workload.Setup();
+  vector<string> tablet_ids;
+  ASSERT_EVENTUALLY([&] {
+    tablet_ids = cluster_->mini_tablet_server(0)->ListTablets();
+    ASSERT_EQ(kNumTablets, tablet_ids.size()) << tablet_ids;
+  });
+
+  // Start threads that start leader elections.
+  vector<thread> threads;
+  atomic<bool> running(true);
+  auto* ts = ts_map_[cluster_->mini_tablet_server(0)->uuid()];
+  for (const string& tablet_id : tablet_ids) {
+    threads.emplace_back([ts, tablet_id, &running, &kTimeout] {
+      while (running) {
+        itest::StartElection(ts, tablet_id, kTimeout); // Ignore result.
+      }
+    });
+  }
+
+  // Sequentially delete all of the tablets.
+  for (const string& tablet_id : tablet_ids) {
+    ASSERT_OK(itest::DeleteTablet(ts, tablet_id, TABLET_DATA_DELETED, boost::none, kTimeout));
+  }
+
+  // Wait until all tablets are deleted.
+  ASSERT_EVENTUALLY([&] {
+    tablet_ids = cluster_->mini_tablet_server(0)->ListTablets();
+    ASSERT_EQ(0, tablet_ids.size()) << tablet_ids;
+  });
+
+  // Stop all the election threads.
+  running = false;
+  for (auto& t : threads) {
+    t.join();
+  }
+}
+
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/2f78643e/src/kudu/tserver/ts_tablet_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/ts_tablet_manager.cc b/src/kudu/tserver/ts_tablet_manager.cc
index 094e873..cde1f71 100644
--- a/src/kudu/tserver/ts_tablet_manager.cc
+++ b/src/kudu/tserver/ts_tablet_manager.cc
@@ -658,8 +658,8 @@ Status TSTabletManager::DeleteTablet(
   // If the tablet is already deleted, the CAS check isn't possible because
   // consensus and therefore the log is not available.
   TabletDataState data_state = replica->tablet_metadata()->tablet_data_state();
-  bool tablet_deleted = (data_state == TABLET_DATA_DELETED ||
-                         data_state == TABLET_DATA_TOMBSTONED);
+  bool tablet_already_deleted = (data_state == TABLET_DATA_DELETED ||
+                                 data_state == TABLET_DATA_TOMBSTONED);
 
   // They specified an "atomic" delete. Check the committed config's opid_index.
   // TODO(mpercy): There's actually a race here between the check and shutdown,
@@ -667,7 +667,7 @@ Status TSTabletManager::DeleteTablet(
   // restarting the tablet if the local replica committed a higher config change
   // op during that time, or potentially something else more invasive.
   shared_ptr<RaftConsensus> consensus = replica->shared_consensus();
-  if (cas_config_opid_index_less_or_equal && !tablet_deleted) {
+  if (cas_config_opid_index_less_or_equal && !tablet_already_deleted) {
     if (!consensus) {
       *error_code = TabletServerErrorPB::TABLET_NOT_RUNNING;
       return Status::IllegalState("Raft Consensus not available. Tablet shutting down");
@@ -708,8 +708,9 @@ Status TSTabletManager::DeleteTablet(
 
   replica->SetStatusMessage("Deleted tablet blocks from disk");
 
-  // We only remove DELETED tablets from the tablet map.
+  // Only DELETED tablets are fully shut down and removed from the tablet map.
   if (delete_type == TABLET_DATA_DELETED) {
+    replica->Shutdown();
     std::lock_guard<rw_spinlock> lock(lock_);
     RETURN_NOT_OK(CheckRunningUnlocked(error_code));
     CHECK_EQ(1, tablet_map_.erase(tablet_id)) << tablet_id;