You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by to...@apache.org on 2016/10/06 00:05:58 UTC

[2/8] kudu git commit: raft_consensus: clean up overrides, protected methods

raft_consensus: clean up overrides, protected methods

RaftConsensus used to have a subclass mock in raft_consensus-test, but
that test is now removed. Given this, we can make some
previously-protected methods private and non-virtual.

I also took this opportunity to modernize the 'override' style to C++11,
and also to remove a function which was essentially duplicate code.

Change-Id: Ib52d3ac40ed68a4ff8b1738897f6ef62f94a843b
Reviewed-on: http://gerrit.cloudera.org:8080/4620
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/master
Commit: bda9b94e88b2602dcc029e763c8a9e0eecff8e08
Parents: 4bcbb4a
Author: Todd Lipcon <to...@apache.org>
Authored: Tue Oct 4 14:34:10 2016 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Oct 5 21:41:36 2016 +0000

----------------------------------------------------------------------
 src/kudu/consensus/raft_consensus.cc            |  9 +-
 src/kudu/consensus/raft_consensus.h             | 98 +++++++++-----------
 .../consensus/raft_consensus_quorum-test.cc     |  2 +-
 3 files changed, 45 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/bda9b94e/src/kudu/consensus/raft_consensus.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/raft_consensus.cc b/src/kudu/consensus/raft_consensus.cc
index 0cb724e..3e4a89b 100644
--- a/src/kudu/consensus/raft_consensus.cc
+++ b/src/kudu/consensus/raft_consensus.cc
@@ -1511,12 +1511,6 @@ void RaftConsensus::Shutdown() {
   shutdown_.Store(true, kMemOrderRelease);
 }
 
-RaftPeerPB::Role RaftConsensus::GetActiveRole() const {
-  ReplicaState::UniqueLock lock;
-  CHECK_OK(state_->LockForRead(&lock));
-  return state_->GetActiveRoleUnlocked();
-}
-
 OpId RaftConsensus::GetLatestOpIdFromLog() {
   OpId id;
   log_->GetLatestEntryOpId(&id);
@@ -1675,8 +1669,7 @@ Status RaftConsensus::RequestVoteRespondVoteGranted(const VoteRequestPB* request
 RaftPeerPB::Role RaftConsensus::role() const {
   ReplicaState::UniqueLock lock;
   CHECK_OK(state_->LockForRead(&lock));
-  return GetConsensusRole(state_->GetPeerUuid(),
-                          state_->ConsensusStateUnlocked(CONSENSUS_CONFIG_ACTIVE));
+  return state_->GetActiveRoleUnlocked();
 }
 
 std::string RaftConsensus::LogPrefixUnlocked() {

http://git-wip-us.apache.org/repos/asf/kudu/blob/bda9b94e/src/kudu/consensus/raft_consensus.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/raft_consensus.h b/src/kudu/consensus/raft_consensus.h
index 887201b..07ef186 100644
--- a/src/kudu/consensus/raft_consensus.h
+++ b/src/kudu/consensus/raft_consensus.h
@@ -91,69 +91,64 @@ class RaftConsensus : public Consensus,
 
   virtual ~RaftConsensus();
 
-  virtual Status Start(const ConsensusBootstrapInfo& info) OVERRIDE;
+  Status Start(const ConsensusBootstrapInfo& info) override;
 
-  virtual bool IsRunning() const OVERRIDE;
+  bool IsRunning() const override;
 
   // Emulates an election by increasing the term number and asserting leadership
   // in the configuration by sending a NO_OP to other peers.
   // This is NOT safe to use in a distributed configuration with failure detection
   // enabled, as it could result in a split-brain scenario.
-  virtual Status EmulateElection() OVERRIDE;
+  Status EmulateElection() override;
 
-  virtual Status StartElection(ElectionMode mode) OVERRIDE;
+  Status StartElection(ElectionMode mode) override;
 
-  virtual Status WaitUntilLeaderForTests(const MonoDelta& timeout) OVERRIDE;
+  Status WaitUntilLeaderForTests(const MonoDelta& timeout) override;
 
-  virtual Status StepDown(LeaderStepDownResponsePB* resp) OVERRIDE;
+  Status StepDown(LeaderStepDownResponsePB* resp) override;
 
   // Call StartElection(), log a warning if the call fails (usually due to
   // being shut down).
   void ReportFailureDetected(const std::string& name, const Status& msg);
 
-  virtual Status Replicate(const scoped_refptr<ConsensusRound>& round) OVERRIDE;
+  Status Replicate(const scoped_refptr<ConsensusRound>& round) override;
 
-  virtual Status CheckLeadershipAndBindTerm(const scoped_refptr<ConsensusRound>& round) OVERRIDE;
+  Status CheckLeadershipAndBindTerm(const scoped_refptr<ConsensusRound>& round) override;
 
-  virtual Status Update(const ConsensusRequestPB* request,
-                        ConsensusResponsePB* response) OVERRIDE;
+  Status Update(const ConsensusRequestPB* request,
+                ConsensusResponsePB* response) override;
 
-  virtual Status RequestVote(const VoteRequestPB* request,
-                             VoteResponsePB* response) OVERRIDE;
+  Status RequestVote(const VoteRequestPB* request,
+                     VoteResponsePB* response) override;
 
-  virtual Status ChangeConfig(const ChangeConfigRequestPB& req,
-                              const StatusCallback& client_cb,
-                              boost::optional<tserver::TabletServerErrorPB::Code>* error_code)
-                              OVERRIDE;
+  Status ChangeConfig(const ChangeConfigRequestPB& req,
+                      const StatusCallback& client_cb,
+                      boost::optional<tserver::TabletServerErrorPB::Code>* error_code) override;
 
-  virtual RaftPeerPB::Role role() const OVERRIDE;
+  Status GetLastOpId(OpIdType type, OpId* id) override;
 
-  virtual std::string peer_uuid() const OVERRIDE;
+  RaftPeerPB::Role role() const override;
 
-  virtual std::string tablet_id() const OVERRIDE;
+  std::string peer_uuid() const override;
 
-  virtual ConsensusStatePB ConsensusState(ConsensusConfigType type) const OVERRIDE;
+  std::string tablet_id() const override;
 
-  virtual RaftConfigPB CommittedConfig() const OVERRIDE;
+  ConsensusStatePB ConsensusState(ConsensusConfigType type) const override;
 
-  virtual void DumpStatusHtml(std::ostream& out) const OVERRIDE;
+  RaftConfigPB CommittedConfig() const override;
 
-  virtual void Shutdown() OVERRIDE;
+  void DumpStatusHtml(std::ostream& out) const override;
 
-  // Makes this peer advance it's term (and step down if leader), for tests.
-  virtual Status AdvanceTermForTests(int64_t new_term);
+  void Shutdown() override;
 
-  // Return the active (as opposed to committed) role.
-  RaftPeerPB::Role GetActiveRole() const;
+  // Makes this peer advance it's term (and step down if leader), for tests.
+  Status AdvanceTermForTests(int64_t new_term);
 
   // Returns the replica state for tests. This should never be used outside of
   // tests, in particular calling the LockFor* methods on the returned object
   // can cause consensus to deadlock.
   ReplicaState* GetReplicaStateForTests();
 
-  virtual Status GetLastOpId(OpIdType type, OpId* id) OVERRIDE;
-
-
   //------------------------------------------------------------
   // PeerMessageQueueObserver implementation
   //------------------------------------------------------------
@@ -171,25 +166,6 @@ class RaftConsensus : public Consensus,
 
   log::RetentionIndexes GetRetentionIndexes() override;
 
- protected:
-  // Trigger that a non-Transaction ConsensusRound has finished replication.
-  // If the replication was successful, an status will be OK. Otherwise, it
-  // may be Aborted or some other error status.
-  // If 'status' is OK, write a Commit message to the local WAL based on the
-  // type of message it is.
-  // The 'client_cb' will be invoked at the end of this execution.
-  virtual void NonTxRoundReplicationFinished(ConsensusRound* round,
-                                             const StatusCallback& client_cb,
-                                             const Status& status);
-
-  // As a leader, append a new ConsensusRound to the queue.
-  // Only virtual and protected for mocking purposes.
-  virtual Status AppendNewRoundToQueueUnlocked(const scoped_refptr<ConsensusRound>& round);
-
-  // As a follower, start a consensus round not associated with a Transaction.
-  // Only virtual and protected for mocking purposes.
-  virtual Status StartConsensusOnlyRoundUnlocked(const ReplicateRefPtr& msg);
-
  private:
   friend class ReplicaState;
   friend class RaftConsensusQuorumTest;
@@ -288,12 +264,6 @@ class RaftConsensus : public Consensus,
   // and also truncate the LogCache accordingly.
   void TruncateAndAbortOpsAfterUnlocked(int64_t truncate_after_index);
 
-  // Pushes a new Raft configuration to a majority of peers. Contrary to write operations,
-  // this actually waits for the commit round to reach a majority of peers, so that we know
-  // we can proceed. If this returns Status::OK(), a majority of peers have accepted the new
-  // configuration. The peer cannot perform any additional operations until this succeeds.
-  Status PushConfigurationToPeersUnlocked(const RaftConfigPB& new_config);
-
   // Returns the most recent OpId written to the Log.
   OpId GetLatestOpIdFromLog();
 
@@ -432,6 +402,24 @@ class RaftConsensus : public Consensus,
                              const RaftConfigPB& committed_config,
                              const std::string& reason);
 
+  // Trigger that a non-Transaction ConsensusRound has finished replication.
+  // If the replication was successful, an status will be OK. Otherwise, it
+  // may be Aborted or some other error status.
+  // If 'status' is OK, write a Commit message to the local WAL based on the
+  // type of message it is.
+  // The 'client_cb' will be invoked at the end of this execution.
+  void NonTxRoundReplicationFinished(ConsensusRound* round,
+                                     const StatusCallback& client_cb,
+                                     const Status& status);
+
+  // As a leader, append a new ConsensusRound to the queue.
+  // Only virtual and protected for mocking purposes.
+  Status AppendNewRoundToQueueUnlocked(const scoped_refptr<ConsensusRound>& round);
+
+  // As a follower, start a consensus round not associated with a Transaction.
+  // Only virtual and protected for mocking purposes.
+  Status StartConsensusOnlyRoundUnlocked(const ReplicateRefPtr& msg);
+
   // Threadpool for constructing requests to peers, handling RPC callbacks,
   // etc.
   gscoped_ptr<ThreadPool> thread_pool_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/bda9b94e/src/kudu/consensus/raft_consensus_quorum-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/raft_consensus_quorum-test.cc b/src/kudu/consensus/raft_consensus_quorum-test.cc
index 9d63266..362a230 100644
--- a/src/kudu/consensus/raft_consensus_quorum-test.cc
+++ b/src/kudu/consensus/raft_consensus_quorum-test.cc
@@ -78,7 +78,7 @@ void DoNothing(const string& s) {
 Status WaitUntilLeaderForTests(RaftConsensus* raft) {
   MonoTime deadline = MonoTime::Now() + MonoDelta::FromSeconds(15);
   while (MonoTime::Now() < deadline) {
-    if (raft->GetActiveRole() == RaftPeerPB::LEADER) {
+    if (raft->role() == RaftPeerPB::LEADER) {
       return Status::OK();
     }
     SleepFor(MonoDelta::FromMilliseconds(10));