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/09/09 03:43:33 UTC

kudu git commit: KUDU-2141. master: Remove DCHECK when tablet report has no opid_index

Repository: kudu
Updated Branches:
  refs/heads/master 9f0649448 -> 3f4972408


KUDU-2141. master: Remove DCHECK when tablet report has no opid_index

Commit 2108767bf5331e0f3beccd56a987cb413cca380a made it possible for a
tombstoned replica to report an empty committed config with no
opid_index associated with it. The master should handle such a tablet by
simply ignoring it.

The included modification to tombstoned_voting-itest.cc causes the test
to fail without the included changes to catalog_manager.cc

Change-Id: Ia99936b11b49a4bd70dbc065f2d16136eb5b8bda
Reviewed-on: http://gerrit.cloudera.org:8080/8019
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Kudu Jenkins
Reviewed-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/3f497240
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/3f497240
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/3f497240

Branch: refs/heads/master
Commit: 3f4972408ede6232436e60bcc8a86837411ea617
Parents: 9f06494
Author: Mike Percy <mp...@apache.org>
Authored: Fri Sep 8 18:36:43 2017 -0700
Committer: Mike Percy <mp...@apache.org>
Committed: Sat Sep 9 03:43:12 2017 +0000

----------------------------------------------------------------------
 .../tombstoned_voting-itest.cc                  | 23 +++++++++++++++++++-
 src/kudu/master/catalog_manager.cc              | 18 +++++++--------
 2 files changed, 31 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/3f497240/src/kudu/integration-tests/tombstoned_voting-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/tombstoned_voting-itest.cc b/src/kudu/integration-tests/tombstoned_voting-itest.cc
index 815540d..978f7fc 100644
--- a/src/kudu/integration-tests/tombstoned_voting-itest.cc
+++ b/src/kudu/integration-tests/tombstoned_voting-itest.cc
@@ -34,11 +34,14 @@
 #include "kudu/integration-tests/test_workload.h"
 #include "kudu/master/master.pb.h"
 #include "kudu/tablet/metadata.pb.h"
+#include "kudu/util/metrics.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
+METRIC_DECLARE_histogram(handler_latency_kudu_master_MasterService_TSHeartbeat);
+
 using kudu::consensus::MakeOpId;
 using kudu::itest::TServerDetails;
 using kudu::tablet::TABLET_DATA_COPYING;
@@ -52,10 +55,18 @@ namespace kudu {
 class TombstonedVotingITest : public ExternalMiniClusterITestBase {
 };
 
+Status CountMasterHeartbeatsRecvd(ExternalMaster* m, int64_t* count) {
+  return m->GetInt64Metric(&METRIC_ENTITY_server,
+                           "kudu.master",
+                           &METRIC_handler_latency_kudu_master_MasterService_TSHeartbeat,
+                           "total_count",
+                           count);
+}
+
 // Test that a replica that crashes during a first-time tablet copy after
 // persisting a superblock but before persisting a cmeta file will be
 // tombstoned and able to vote after the tablet server is restarted.
-// See KUDU-2123.
+// See KUDU-2123. This also serves as a regression test for KUDU-2141.
 TEST_F(TombstonedVotingITest, TestTombstonedReplicaWithoutCMetaCanVote) {
   const MonoDelta kTimeout = MonoDelta::FromSeconds(30);
 
@@ -161,6 +172,16 @@ TEST_F(TombstonedVotingITest, TestTombstonedReplicaWithoutCMetaCanVote) {
     // After voting yes, cmeta should exist.
     ASSERT_FALSE(inspect_->DoesConsensusMetaExistForTabletOnTS(new_replica_idx, tablet_id));
   });
+
+  // Ensure we have no crash due to the new config being reported. This is a
+  // regression test for KUDU-2141.
+  ASSERT_OK(cluster_->master()->Restart());
+  ASSERT_EVENTUALLY([&] {
+    int64_t heartbeat_count;
+    ASSERT_OK(CountMasterHeartbeatsRecvd(cluster_->master(), &heartbeat_count));
+    ASSERT_GE(heartbeat_count, 5); // Wait for 5 heartbeats from the only live TS.
+  });
+  NO_FATALS(cluster_->AssertNoCrashes());
 }
 
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/3f497240/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 4619129..8e22d43 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -2549,6 +2549,15 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
   if (report.has_consensus_state()) {
     ConsensusStatePB cstate = report.consensus_state();
 
+    // The Master only processes reports for replicas with committed consensus
+    // configurations since it needs the committed index to only cache the most
+    // up-to-date config. Since it's possible for TOMBSTONED replicas with no
+    // ConsensusMetadata on disk to be reported as having no committed config
+    // opid_index, we skip over those replicas.
+    if (!cstate.committed_config().has_opid_index()) {
+      return Status::OK();
+    }
+
     // If the reported leader is not a member of the committed config, then we
     // disregard the leader state.
     if (cstate.has_leader_uuid() &&
@@ -2573,15 +2582,6 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
                                             "Tablet reported with an active leader");
     }
 
-    // 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())) {
-      string msg = Substitute("Missing opid_index in reported config: $0",
-                              SecureDebugString(report));
-      LOG(DFATAL) << msg;
-      return Status::InvalidArgument(msg);
-    }
-
     bool modified_cstate = false;
     if (cstate.committed_config().opid_index() > prev_cstate.committed_config().opid_index() ||
         (cstate.has_leader_uuid() &&