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/06/01 20:51:01 UTC

[3/3] kudu git commit: consensus: Add DCHECK(is_locked()) in all unlocked methods

consensus: Add DCHECK(is_locked()) in all unlocked methods

Now that RaftConsensus owns 'lock_' we can easily implement these kinds
of basic hygenic checks.

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


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

Branch: refs/heads/master
Commit: fbaa7dce968e2a6965effaa9d1097c34649d65f4
Parents: 3846861
Author: Mike Percy <mp...@apache.org>
Authored: Sat May 27 15:01:08 2017 -0700
Committer: Mike Percy <mp...@apache.org>
Committed: Thu Jun 1 20:45:17 2017 +0000

----------------------------------------------------------------------
 src/kudu/consensus/raft_consensus.cc | 42 +++++++++++++++++++++++++++++++
 src/kudu/consensus/raft_consensus.h  |  4 +--
 2 files changed, 44 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/fbaa7dce/src/kudu/consensus/raft_consensus.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/raft_consensus.cc b/src/kudu/consensus/raft_consensus.cc
index 6a2c39f..b7f5cdc 100644
--- a/src/kudu/consensus/raft_consensus.cc
+++ b/src/kudu/consensus/raft_consensus.cc
@@ -533,6 +533,8 @@ void RaftConsensus::ReportFailureDetected(const std::string& name, const Status&
 }
 
 Status RaftConsensus::BecomeLeaderUnlocked() {
+  DCHECK(lock_.is_locked());
+
   TRACE_EVENT2("consensus", "RaftConsensus::BecomeLeaderUnlocked",
                "peer", peer_uuid_,
                "tablet", options_.tablet_id);
@@ -566,6 +568,8 @@ Status RaftConsensus::BecomeLeaderUnlocked() {
 }
 
 Status RaftConsensus::BecomeReplicaUnlocked() {
+  DCHECK(lock_.is_locked());
+
   LOG_WITH_PREFIX_UNLOCKED(INFO) << "Becoming Follower/Learner. State: "
                                  << ToStringUnlocked();
 
@@ -610,6 +614,8 @@ Status RaftConsensus::CheckLeadershipAndBindTerm(const scoped_refptr<ConsensusRo
 }
 
 Status RaftConsensus::AppendNewRoundToQueueUnlocked(const scoped_refptr<ConsensusRound>& round) {
+  DCHECK(lock_.is_locked());
+
   *round->replicate_msg()->mutable_id() = queue_->GetNextOpId();
   RETURN_NOT_OK(AddPendingOperationUnlocked(round));
 
@@ -621,6 +627,8 @@ Status RaftConsensus::AppendNewRoundToQueueUnlocked(const scoped_refptr<Consensu
 }
 
 Status RaftConsensus::AddPendingOperationUnlocked(const scoped_refptr<ConsensusRound>& round) {
+  DCHECK(lock_.is_locked());
+
   // If we are adding a pending config change, we need to propagate it to the
   // metadata.
   if (PREDICT_FALSE(round->replicate_msg()->op_type() == CHANGE_CONFIG_OP)) {
@@ -794,6 +802,8 @@ static bool IsConsensusOnlyOperation(OperationType op_type) {
 }
 
 Status RaftConsensus::StartReplicaTransactionUnlocked(const ReplicateRefPtr& msg) {
+  DCHECK(lock_.is_locked());
+
   if (IsConsensusOnlyOperation(msg->get()->op_type())) {
     return StartConsensusOnlyRoundUnlocked(msg);
   }
@@ -841,6 +851,8 @@ std::string RaftConsensus::LeaderRequest::OpsRangeString() const {
 
 void RaftConsensus::DeduplicateLeaderRequestUnlocked(ConsensusRequestPB* rpc_req,
                                                      LeaderRequest* deduplicated_req) {
+  DCHECK(lock_.is_locked());
+
   // TODO(todd): use queue committed index?
   int64_t last_committed_index = pending_.GetCommittedIndex();
 
@@ -903,6 +915,7 @@ void RaftConsensus::DeduplicateLeaderRequestUnlocked(ConsensusRequestPB* rpc_req
 
 Status RaftConsensus::HandleLeaderRequestTermUnlocked(const ConsensusRequestPB* request,
                                                       ConsensusResponsePB* response) {
+  DCHECK(lock_.is_locked());
   // Do term checks first:
   if (PREDICT_FALSE(request->caller_term() != GetCurrentTermUnlocked())) {
 
@@ -928,6 +941,7 @@ Status RaftConsensus::HandleLeaderRequestTermUnlocked(const ConsensusRequestPB*
 
 Status RaftConsensus::EnforceLogMatchingPropertyMatchesUnlocked(const LeaderRequest& req,
                                                                 ConsensusResponsePB* response) {
+  DCHECK(lock_.is_locked());
 
   bool term_mismatch;
   if (pending_.IsOpCommittedOrPending(*req.preceding_opid, &term_mismatch)) {
@@ -967,6 +981,7 @@ Status RaftConsensus::EnforceLogMatchingPropertyMatchesUnlocked(const LeaderRequ
 }
 
 void RaftConsensus::TruncateAndAbortOpsAfterUnlocked(int64_t truncate_after_index) {
+  DCHECK(lock_.is_locked());
   pending_.AbortOpsAfter(truncate_after_index);
   queue_->TruncateOpsAfter(truncate_after_index);
 }
@@ -974,6 +989,7 @@ void RaftConsensus::TruncateAndAbortOpsAfterUnlocked(int64_t truncate_after_inde
 Status RaftConsensus::CheckLeaderRequestUnlocked(const ConsensusRequestPB* request,
                                                  ConsensusResponsePB* response,
                                                  LeaderRequest* deduped_req) {
+  DCHECK(lock_.is_locked());
 
   if (request->has_deprecated_committed_index() ||
       !request->has_all_replicated_index()) {
@@ -1373,6 +1389,7 @@ Status RaftConsensus::UpdateReplica(const ConsensusRequestPB* request,
 }
 
 void RaftConsensus::FillConsensusResponseOKUnlocked(ConsensusResponsePB* response) {
+  DCHECK(lock_.is_locked());
   TRACE("Filling consensus response to leader.");
   response->set_responder_term(GetCurrentTermUnlocked());
   response->mutable_status()->mutable_last_received()->CopyFrom(
@@ -1796,6 +1813,7 @@ OpId RaftConsensus::GetLatestOpIdFromLog() {
 }
 
 Status RaftConsensus::StartConsensusOnlyRoundUnlocked(const ReplicateRefPtr& msg) {
+  DCHECK(lock_.is_locked());
   OperationType op_type = msg->get()->op_type();
   CHECK(IsConsensusOnlyOperation(op_type))
       << "Expected a consensus-only op type, got " << OperationType_Name(op_type)
@@ -1821,6 +1839,7 @@ Status RaftConsensus::AdvanceTermForTests(int64_t new_term) {
 }
 
 std::string RaftConsensus::GetRequestVoteLogPrefixUnlocked(const VoteRequestPB& request) const {
+  DCHECK(lock_.is_locked());
   return Substitute("$0Leader $1election vote request",
                     LogPrefixUnlocked(),
                     request.is_pre_election() ? "pre-" : "");
@@ -1981,6 +2000,7 @@ void RaftConsensus::SetLeaderUuidUnlocked(const string& uuid) {
 Status RaftConsensus::ReplicateConfigChangeUnlocked(const RaftConfigPB& old_config,
                                                     const RaftConfigPB& new_config,
                                                     const StatusCallback& client_cb) {
+  DCHECK(lock_.is_locked());
   auto cc_replicate = new ReplicateMsg();
   cc_replicate->set_op_type(CHANGE_CONFIG_OP);
   ChangeConfigRecordPB* cc_req = cc_replicate->mutable_change_config_record();
@@ -2001,6 +2021,7 @@ Status RaftConsensus::ReplicateConfigChangeUnlocked(const RaftConfigPB& old_conf
 }
 
 Status RaftConsensus::RefreshConsensusQueueAndPeersUnlocked() {
+  DCHECK(lock_.is_locked());
   DCHECK_EQ(RaftPeerPB::LEADER, GetActiveRoleUnlocked());
   const RaftConfigPB& active_config = GetActiveConfigUnlocked();
 
@@ -2264,6 +2285,7 @@ void RaftConsensus::NonTxRoundReplicationFinished(ConsensusRound* round,
 }
 
 void RaftConsensus::CompleteConfigChangeRoundUnlocked(ConsensusRound* round, const Status& status) {
+  DCHECK(lock_.is_locked());
   const OpId& op_id = round->replicate_msg()->id();
 
   if (!status.ok()) {
@@ -2322,6 +2344,7 @@ void RaftConsensus::CompleteConfigChangeRoundUnlocked(ConsensusRound* round, con
 
 
 Status RaftConsensus::EnsureFailureDetectorEnabledUnlocked() {
+  DCHECK(lock_.is_locked());
   if (PREDICT_FALSE(!FLAGS_enable_leader_failure_detection)) {
     return Status::OK();
   }
@@ -2335,6 +2358,7 @@ Status RaftConsensus::EnsureFailureDetectorEnabledUnlocked() {
 }
 
 Status RaftConsensus::EnsureFailureDetectorDisabledUnlocked() {
+  DCHECK(lock_.is_locked());
   if (PREDICT_FALSE(!FLAGS_enable_leader_failure_detection)) {
     return Status::OK();
   }
@@ -2346,6 +2370,7 @@ Status RaftConsensus::EnsureFailureDetectorDisabledUnlocked() {
 }
 
 Status RaftConsensus::ExpireFailureDetectorUnlocked() {
+  DCHECK(lock_.is_locked());
   if (PREDICT_FALSE(!FLAGS_enable_leader_failure_detection)) {
     return Status::OK();
   }
@@ -2354,11 +2379,13 @@ Status RaftConsensus::ExpireFailureDetectorUnlocked() {
 }
 
 Status RaftConsensus::SnoozeFailureDetectorUnlocked() {
+  DCHECK(lock_.is_locked());
   return SnoozeFailureDetectorUnlocked(MonoDelta::FromMicroseconds(0), DO_NOT_LOG);
 }
 
 Status RaftConsensus::SnoozeFailureDetectorUnlocked(const MonoDelta& additional_delta,
                                                     AllowLogging allow_logging) {
+  DCHECK(lock_.is_locked());
   if (PREDICT_FALSE(!FLAGS_enable_leader_failure_detection)) {
     return Status::OK();
   }
@@ -2380,6 +2407,7 @@ MonoDelta RaftConsensus::MinimumElectionTimeout() const {
 }
 
 MonoDelta RaftConsensus::LeaderElectionExpBackoffDeltaUnlocked() {
+  DCHECK(lock_.is_locked());
   // Compute a backoff factor based on how many leader elections have
   // failed since a stable leader was last seen.
   double backoff_factor = pow(1.5, failed_elections_since_stable_leader_ + 1);
@@ -2400,6 +2428,7 @@ MonoDelta RaftConsensus::LeaderElectionExpBackoffDeltaUnlocked() {
 
 Status RaftConsensus::HandleTermAdvanceUnlocked(ConsensusTerm new_term,
                                                 FlushToDisk flush) {
+  DCHECK(lock_.is_locked());
   if (new_term <= GetCurrentTermUnlocked()) {
     return Status::IllegalState(Substitute("Can't advance term to: $0 current term: $1 is higher.",
                                            new_term, GetCurrentTermUnlocked()));
@@ -2434,6 +2463,7 @@ Status RaftConsensus::CheckRunningUnlocked() const {
 }
 
 Status RaftConsensus::CheckActiveLeaderUnlocked() const {
+  DCHECK(lock_.is_locked());
   RaftPeerPB::Role role = GetActiveRoleUnlocked();
   switch (role) {
     case RaftPeerPB::LEADER:
@@ -2449,6 +2479,7 @@ Status RaftConsensus::CheckActiveLeaderUnlocked() const {
 }
 
 ConsensusStatePB RaftConsensus::ConsensusStateUnlocked() const {
+  DCHECK(lock_.is_locked());
   return cmeta_->ToConsensusStatePB();
 }
 
@@ -2494,6 +2525,7 @@ Status RaftConsensus::SetPendingConfigUnlocked(const RaftConfigPB& new_config) {
 }
 
 void RaftConsensus::ClearPendingConfigUnlocked() {
+  DCHECK(lock_.is_locked());
   cmeta_->clear_pending_config();
 }
 
@@ -2573,6 +2605,16 @@ const string& RaftConsensus::GetLeaderUuidUnlocked() const {
   return cmeta_->leader_uuid();
 }
 
+bool RaftConsensus::HasLeaderUnlocked() const {
+  DCHECK(lock_.is_locked());
+  return !GetLeaderUuidUnlocked().empty();
+}
+
+void RaftConsensus::ClearLeaderUnlocked() {
+  DCHECK(lock_.is_locked());
+  SetLeaderUuidUnlocked("");
+}
+
 const bool RaftConsensus::HasVotedCurrentTermUnlocked() const {
   DCHECK(lock_.is_locked());
   return cmeta_->has_voted_for();

http://git-wip-us.apache.org/repos/asf/kudu/blob/fbaa7dce/src/kudu/consensus/raft_consensus.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/raft_consensus.h b/src/kudu/consensus/raft_consensus.h
index b9348c2..f00a8c2 100644
--- a/src/kudu/consensus/raft_consensus.h
+++ b/src/kudu/consensus/raft_consensus.h
@@ -528,8 +528,8 @@ class RaftConsensus : public Consensus,
 
   // Accessors for the leader of the current term.
   const std::string& GetLeaderUuidUnlocked() const;
-  bool HasLeaderUnlocked() const { return !GetLeaderUuidUnlocked().empty(); }
-  void ClearLeaderUnlocked() { SetLeaderUuidUnlocked(""); }
+  bool HasLeaderUnlocked() const;
+  void ClearLeaderUnlocked();
 
   // Return whether this peer has voted in the current term.
   const bool HasVotedCurrentTermUnlocked() const;