You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by wd...@apache.org on 2017/05/17 19:30:47 UTC

[1/2] kudu git commit: Refactor ConsensusStatePB to hold committed and pending configs

Repository: kudu
Updated Branches:
  refs/heads/master 3db551f7b -> 1d74c73e1


http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/tserver/tserver-path-handlers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tserver-path-handlers.cc b/src/kudu/tserver/tserver-path-handlers.cc
index e89016c..4aa9407 100644
--- a/src/kudu/tserver/tserver-path-handlers.cc
+++ b/src/kudu/tserver/tserver-path-handlers.cc
@@ -45,7 +45,6 @@
 #include "kudu/util/url-coding.h"
 
 using kudu::consensus::GetConsensusRole;
-using kudu::consensus::CONSENSUS_CONFIG_COMMITTED;
 using kudu::consensus::ConsensusStatePB;
 using kudu::consensus::RaftPeerPB;
 using kudu::consensus::TransactionStatusPB;
@@ -264,9 +263,7 @@ void TabletServerPathHandlers::HandleTabletsPage(const Webserver::WebRequest& re
           tablet_id_or_link, // $1
           EscapeForHtmlToString(partition), // $2
           EscapeForHtmlToString(replica->HumanReadableState()), mem_bytes, n_bytes, // $3, $4, $5
-          consensus ? ConsensusStatePBToHtml(consensus->
-              ConsensusState(CONSENSUS_CONFIG_COMMITTED))
-                    : "", // $6
+          consensus ? ConsensusStatePBToHtml(consensus->ConsensusState()) : "", // $6
           EscapeForHtmlToString(status.last_status())); // $7
     }
     *output << "<tbody></table>\n</div>\n";
@@ -307,7 +304,8 @@ string TabletServerPathHandlers::ConsensusStatePBToHtml(const ConsensusStatePB&
 
   html << "<ul>\n";
   std::vector<RaftPeerPB> sorted_peers;
-  sorted_peers.assign(cstate.config().peers().begin(), cstate.config().peers().end());
+  sorted_peers.assign(cstate.committed_config().peers().begin(),
+                      cstate.committed_config().peers().end());
   std::sort(sorted_peers.begin(), sorted_peers.end(), &CompareByMemberType);
   for (const RaftPeerPB& peer : sorted_peers) {
     string peer_addr_or_uuid =


[2/2] kudu git commit: Refactor ConsensusStatePB to hold committed and pending configs

Posted by wd...@apache.org.
Refactor ConsensusStatePB to hold committed and pending configs

This patch refactors ConsensusStatePB so it contains both the
committed config and, if there is a pending config, the pending
config. Previously, consumers of the consensus state would ask
for either the committed or active (= pending if there is a pending
config, else the committed config) config. With this change, the
committed and active configs are always included.

The addition of pending info to the ConsensusStatePB will be used
by ksck in a follow-up patch. Additionally, the master may use
this info in the future to be more aware of the health of tablets.

Change-Id: I4bc4bdd9752fc29a7ce2cefcdc95c4366f5353af
Reviewed-on: http://gerrit.cloudera.org:8080/6809
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/1d74c73e
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/1d74c73e
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/1d74c73e

Branch: refs/heads/master
Commit: 1d74c73e12fbe308e59bf1e175aa234e865daac1
Parents: 3db551f
Author: Will Berkeley <wd...@gmail.com>
Authored: Fri May 5 17:28:58 2017 -0400
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed May 17 18:25:53 2017 +0000

----------------------------------------------------------------------
 src/kudu/consensus/consensus.h                  |   4 +-
 src/kudu/consensus/consensus.proto              |   5 +-
 src/kudu/consensus/consensus_meta-test.cc       |  35 ++--
 src/kudu/consensus/consensus_meta.cc            |  30 ++--
 src/kudu/consensus/consensus_meta.h             |  16 +-
 src/kudu/consensus/metadata.proto               |  36 +---
 src/kudu/consensus/quorum_util-test.cc          |  48 ++++--
 src/kudu/consensus/quorum_util.cc               | 167 +++++++++++++------
 src/kudu/consensus/quorum_util.h                |   6 +-
 src/kudu/consensus/raft_consensus.cc            |  11 +-
 src/kudu/consensus/raft_consensus.h             |   2 +-
 src/kudu/consensus/raft_consensus_state.cc      |   6 +-
 src/kudu/consensus/raft_consensus_state.h       |   4 +-
 .../integration-tests/cluster_itest_util.cc     |  23 +--
 src/kudu/integration-tests/cluster_itest_util.h |   1 -
 .../integration-tests/delete_table-itest.cc     |   3 +-
 src/kudu/integration-tests/tablet_copy-itest.cc |   4 +-
 .../ts_tablet_manager-itest.cc                  |   6 +-
 src/kudu/master/catalog_manager.cc              | 119 ++++++-------
 src/kudu/master/master-path-handlers.cc         |  13 +-
 src/kudu/master/master.proto                    |   8 +-
 src/kudu/master/sys_catalog.cc                  |  14 +-
 src/kudu/tools/kudu-admin-test.cc               |  12 +-
 src/kudu/tserver/tablet_copy.proto              |   4 +-
 src/kudu/tserver/tablet_copy_client-test.cc     |   3 +-
 src/kudu/tserver/tablet_copy_client.cc          |  12 +-
 src/kudu/tserver/tablet_copy_client.h           |   2 +-
 src/kudu/tserver/tablet_copy_service.cc         |   2 +-
 src/kudu/tserver/tablet_copy_source_session.cc  |   2 +-
 src/kudu/tserver/tablet_copy_source_session.h   |   6 +-
 src/kudu/tserver/tablet_service.cc              |  13 +-
 src/kudu/tserver/ts_tablet_manager-test.cc      |  14 +-
 src/kudu/tserver/ts_tablet_manager.cc           |   3 +-
 src/kudu/tserver/tserver-path-handlers.cc       |   8 +-
 34 files changed, 335 insertions(+), 307 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/consensus/consensus.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus.h b/src/kudu/consensus/consensus.h
index 8ea313c..7084aff 100644
--- a/src/kudu/consensus/consensus.h
+++ b/src/kudu/consensus/consensus.h
@@ -265,8 +265,8 @@ class Consensus : public RefCountedThreadSafe<Consensus> {
 
   virtual scoped_refptr<TimeManager> time_manager() const = 0;
 
-  // Returns a copy of the committed state of the Consensus system.
-  virtual ConsensusStatePB ConsensusState(ConsensusConfigType type) const = 0;
+  // Returns a copy of the state of the consensus system.
+  virtual ConsensusStatePB ConsensusState() const = 0;
 
   // Returns a copy of the current committed Raft configuration.
   virtual RaftConfigPB CommittedConfig() const = 0;

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/consensus/consensus.proto
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus.proto b/src/kudu/consensus/consensus.proto
index 1888833..93b8f85 100644
--- a/src/kudu/consensus/consensus.proto
+++ b/src/kudu/consensus/consensus.proto
@@ -456,9 +456,6 @@ message GetConsensusStateRequestPB {
 
   // The id of the tablet.
   required bytes tablet_id = 1;
-
-  // Whether to fetch the committed or active consensus state.
-  optional ConsensusConfigType type = 3 [ default = CONSENSUS_CONFIG_COMMITTED ];
 }
 
 message GetConsensusStateResponsePB {
@@ -539,7 +536,7 @@ service ConsensusService {
 
   rpc GetLastOpId(GetLastOpIdRequestPB) returns (GetLastOpIdResponsePB);
 
-  // Returns the committed Consensus state.
+  // Returns the consensus state.
   rpc GetConsensusState(GetConsensusStateRequestPB) returns (GetConsensusStateResponsePB);
 
   // Instruct this server to copy a tablet from another host.

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/consensus/consensus_meta-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_meta-test.cc b/src/kudu/consensus/consensus_meta-test.cc
index 11214a2..33f9e0c 100644
--- a/src/kudu/consensus/consensus_meta-test.cc
+++ b/src/kudu/consensus/consensus_meta-test.cc
@@ -206,27 +206,18 @@ TEST_F(ConsensusMetadataTest, TestToConsensusStatePB) {
   pending_config.set_opid_index(2);
 
   // Set the pending configuration to be one containing the current leader (who is not
-  // in the committed configuration). Ensure that the leader shows up when we ask for
-  // the active consensus state.
+  // in the committed configuration). Ensure that the leader shows up in the pending
+  // configuration.
   cmeta->set_pending_config(pending_config);
   cmeta->set_leader_uuid(peer_uuid);
-  ConsensusStatePB active_cstate = cmeta->ToConsensusStatePB(CONSENSUS_CONFIG_ACTIVE);
-  ASSERT_TRUE(active_cstate.has_leader_uuid());
-  ASSERT_OK(VerifyConsensusState(active_cstate, UNCOMMITTED_QUORUM));
-
-  // Without changing anything, ask for the committed consensus state.
-  // Since the current leader is not a voter in the committed configuration, the
-  // returned consensus state should not list a leader.
-  ConsensusStatePB committed_cstate = cmeta->ToConsensusStatePB(CONSENSUS_CONFIG_COMMITTED);
-  ASSERT_FALSE(committed_cstate.has_leader_uuid());
-  ASSERT_OK(VerifyConsensusState(committed_cstate, COMMITTED_QUORUM));
-
-  // Set a new leader to be a member of the committed configuration. Now the committed
-  // consensus state should list a leader.
+  ConsensusStatePB cstate = cmeta->ToConsensusStatePB();
+  ASSERT_OK(VerifyConsensusState(cstate));
+
+  // Set a new leader to be a member of the committed configuration.
   cmeta->set_leader_uuid("a");
-  ConsensusStatePB new_committed_cstate = cmeta->ToConsensusStatePB(CONSENSUS_CONFIG_COMMITTED);
-  ASSERT_TRUE(new_committed_cstate.has_leader_uuid());
-  ASSERT_OK(VerifyConsensusState(new_committed_cstate, COMMITTED_QUORUM));
+  ConsensusStatePB new_cstate = cmeta->ToConsensusStatePB();
+  ASSERT_TRUE(new_cstate.has_leader_uuid());
+  ASSERT_OK(VerifyConsensusState(new_cstate));
 }
 
 // Helper for TestMergeCommittedConsensusStatePB.
@@ -238,7 +229,7 @@ static void AssertConsensusMergeExpected(const unique_ptr<ConsensusMetadata>& cm
   // a "spec" of these assertions.
   ASSERT_TRUE(!cmeta->has_pending_config());
   ASSERT_EQ(SecureShortDebugString(cmeta->committed_config()),
-            SecureShortDebugString(cstate.config()));
+            SecureShortDebugString(cstate.committed_config()));
   ASSERT_EQ("", cmeta->leader_uuid());
   ASSERT_EQ(expected_term, cmeta->current_term());
   if (expected_voted_for.empty()) {
@@ -267,19 +258,19 @@ TEST_F(ConsensusMetadataTest, TestMergeCommittedConsensusStatePB) {
   // Keep the term and votes because the merged term is lower.
   ConsensusStatePB remote_state;
   remote_state.set_current_term(0);
-  *remote_state.mutable_config() = BuildConfig({ "x", "y", "z" });
+  *remote_state.mutable_committed_config() = BuildConfig({ "x", "y", "z" });
   cmeta->MergeCommittedConsensusStatePB(remote_state);
   NO_FATALS(AssertConsensusMergeExpected(cmeta, remote_state, 1, "e"));
 
   // Same as above because the merged term is the same as the cmeta term.
   remote_state.set_current_term(1);
-  *remote_state.mutable_config() = BuildConfig({ "f", "g", "h" });
+  *remote_state.mutable_committed_config() = BuildConfig({ "f", "g", "h" });
   cmeta->MergeCommittedConsensusStatePB(remote_state);
   NO_FATALS(AssertConsensusMergeExpected(cmeta, remote_state, 1, "e"));
 
   // Higher term, so wipe out the prior state.
   remote_state.set_current_term(2);
-  *remote_state.mutable_config() = BuildConfig({ "i", "j", "k" });
+  *remote_state.mutable_committed_config() = BuildConfig({ "i", "j", "k" });
   cmeta->set_pending_config(pending_config);
   cmeta->MergeCommittedConsensusStatePB(remote_state);
   NO_FATALS(AssertConsensusMergeExpected(cmeta, remote_state, 2, ""));

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/consensus/consensus_meta.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_meta.cc b/src/kudu/consensus/consensus_meta.cc
index 977074c..ec46755 100644
--- a/src/kudu/consensus/consensus_meta.cc
+++ b/src/kudu/consensus/consensus_meta.cc
@@ -162,33 +162,25 @@ RaftPeerPB::Role ConsensusMetadata::active_role() const {
   return active_role_;
 }
 
-ConsensusStatePB ConsensusMetadata::ToConsensusStatePB(ConsensusConfigType type) const {
-  CHECK(type == CONSENSUS_CONFIG_ACTIVE || type == CONSENSUS_CONFIG_COMMITTED)
-      << "Unsupported ConsensusConfigType: " << ConsensusConfigType_Name(type) << ": " << type;
+ConsensusStatePB ConsensusMetadata::ToConsensusStatePB() const {
   ConsensusStatePB cstate;
   cstate.set_current_term(pb_.current_term());
-  if (type == CONSENSUS_CONFIG_ACTIVE) {
-    *cstate.mutable_config() = active_config();
-    cstate.set_leader_uuid(leader_uuid_);
-  } else {
-    *cstate.mutable_config() = committed_config();
-    // It's possible, though unlikely, that a new node from a pending configuration
-    // could be elected leader. Do not indicate a leader in this case.
-    if (PREDICT_TRUE(IsRaftConfigVoter(leader_uuid_, cstate.config()))) {
-      cstate.set_leader_uuid(leader_uuid_);
-    }
+  cstate.set_leader_uuid(leader_uuid_);
+  *cstate.mutable_committed_config() = committed_config();
+  if (has_pending_config()) {
+    *cstate.mutable_pending_config() = pending_config();
   }
   return cstate;
 }
 
-void ConsensusMetadata::MergeCommittedConsensusStatePB(const ConsensusStatePB& committed_cstate) {
-  if (committed_cstate.current_term() > current_term()) {
-    set_current_term(committed_cstate.current_term());
+void ConsensusMetadata::MergeCommittedConsensusStatePB(const ConsensusStatePB& cstate) {
+  if (cstate.current_term() > current_term()) {
+    set_current_term(cstate.current_term());
     clear_voted_for();
   }
 
   set_leader_uuid("");
-  set_committed_config(committed_cstate.config());
+  set_committed_config(cstate.committed_config());
   clear_pending_config();
 }
 
@@ -198,7 +190,7 @@ Status ConsensusMetadata::Flush() {
 
   flush_count_for_tests_++;
   // Sanity test to ensure we never write out a bad configuration.
-  RETURN_NOT_OK_PREPEND(VerifyRaftConfig(pb_.committed_config(), COMMITTED_QUORUM),
+  RETURN_NOT_OK_PREPEND(VerifyRaftConfig(pb_.committed_config(), COMMITTED_CONFIG),
                         "Invalid config in ConsensusMetadata, cannot flush to disk");
 
   // Create directories if needed.
@@ -241,7 +233,7 @@ std::string ConsensusMetadata::LogPrefix() const {
 }
 
 void ConsensusMetadata::UpdateActiveRole() {
-  ConsensusStatePB cstate = ToConsensusStatePB(CONSENSUS_CONFIG_ACTIVE);
+  ConsensusStatePB cstate = ToConsensusStatePB();
   active_role_ = GetConsensusRole(peer_uuid_, cstate);
   VLOG_WITH_PREFIX(1) << "Updating active role to " << RaftPeerPB::Role_Name(active_role_)
                       << ". Consensus state: " << SecureShortDebugString(cstate);

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/consensus/consensus_meta.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_meta.h b/src/kudu/consensus/consensus_meta.h
index 7a6b8f0..f4e3d7a 100644
--- a/src/kudu/consensus/consensus_meta.h
+++ b/src/kudu/consensus/consensus_meta.h
@@ -118,22 +118,22 @@ class ConsensusMetadata {
   // ConsensusStatePB using only the committed configuration. In this case, if the
   // current leader is not a member of the committed configuration, then the
   // leader_uuid field of the returned ConsensusStatePB will be cleared.
-  ConsensusStatePB ToConsensusStatePB(ConsensusConfigType type) const;
+  ConsensusStatePB ToConsensusStatePB() const;
 
-  // Merge the committed consensus state from the source node during remote
-  // bootstrap.
+  // Merge the committed portion of the consensus state from the source node
+  // during remote bootstrap.
   //
   // This method will clear any pending config change, replace the committed
-  // consensus config with the one in 'committed_cstate', and clear the
-  // currently tracked leader.
+  // consensus config with the one in 'cstate', and clear the currently
+  // tracked leader.
   //
-  // It will also check whether the current term passed in 'committed_cstate'
+  // It will also check whether the current term passed in 'cstate'
   // is greater than the currently recorded one. If so, it will update the
   // local current term to match the passed one and it will clear the voting
-  // record for this node. If the current term in 'committed_cstate' is less
+  // record for this node. If the current term in 'cstate' is less
   // than the locally recorded term, the locally recorded term and voting
   // record are not changed.
-  void MergeCommittedConsensusStatePB(const ConsensusStatePB& committed_cstate);
+  void MergeCommittedConsensusStatePB(const ConsensusStatePB& cstate);
 
   // Persist current state of the protobuf to disk.
   Status Flush();

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/consensus/metadata.proto
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/metadata.proto b/src/kudu/consensus/metadata.proto
index b211c43..cc7223d 100644
--- a/src/kudu/consensus/metadata.proto
+++ b/src/kudu/consensus/metadata.proto
@@ -65,20 +65,6 @@ message RaftPeerPB {
   optional HostPortPB last_known_addr = 3;
 }
 
-enum ConsensusConfigType {
-  CONSENSUS_CONFIG_UNKNOWN = 999;
-
-  // Committed consensus config. This includes the consensus configuration that
-  // has been serialized through consensus and committed, thus having a valid
-  // opid_index field set.
-  CONSENSUS_CONFIG_COMMITTED = 1;
-
-  // Active consensus config. This could be a pending consensus config that
-  // has not yet been committed. If the config is not committed, its opid_index
-  // field will not be set.
-  CONSENSUS_CONFIG_ACTIVE = 2;
-}
-
 // A set of peers, serving a single tablet.
 message RaftConfigPB {
   // The index of the operation which serialized this RaftConfigPB through
@@ -115,25 +101,15 @@ message ConsensusStatePB {
   // vote for a peer is not sufficient to assume that that peer has won the
   // election, so we do not update this field based on our vote.
   //
-  // The leader listed here, if any, should always be a member of 'configuration', and
-  // the term that the node is leader of _must_ equal the term listed above in
-  // the 'current_term' field. The Master will use the combination of current
-  // term and leader uuid to determine when to update its cache of the current
-  // leader for client lookup purposes.
-  //
-  // There is a corner case in Raft where a node may be elected leader of a
-  // pending (uncommitted) configuration. In such a case, if the leader of the pending
-  // configuration is not a member of the committed configuration, and it is the committed
-  // configuration that is being reported, then the leader_uuid field should be
-  // cleared by the process filling in the ConsensusStatePB object.
+  // The leader may be a part of the committed or the pending configuration (or both).
   optional string leader_uuid = 2;
 
-  // The peers. In some contexts, this will be the committed configuration,
-  // which will always have configuration.opid_index set. In other contexts, this may
-  // a "pending" configuration, which is active but in the process of being committed.
-  // In any case, initial peership is set on tablet start, so this
+  // The committed peers. Initial peership is set on tablet start, so this
   // field should always be present.
-  required RaftConfigPB config = 3;
+  required RaftConfigPB committed_config = 3;
+
+  // The peers in the pending configuration, if there is one.
+  optional RaftConfigPB pending_config = 4;
 }
 
 // This PB is used to serialize all of the persistent state needed for

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/consensus/quorum_util-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/quorum_util-test.cc b/src/kudu/consensus/quorum_util-test.cc
index 4b9f50d..70e361c 100644
--- a/src/kudu/consensus/quorum_util-test.cc
+++ b/src/kudu/consensus/quorum_util-test.cc
@@ -49,7 +49,7 @@ TEST(QuorumUtilTest, TestMemberExtraction) {
 
   // Basic test for GetRaftConfigLeader().
   ConsensusStatePB cstate;
-  *cstate.mutable_config() = config;
+  *cstate.mutable_committed_config() = config;
   s = GetRaftConfigLeader(cstate, &peer_pb);
   ASSERT_TRUE(s.IsNotFound()) << s.ToString();
   cstate.set_leader_uuid("B");
@@ -59,12 +59,12 @@ TEST(QuorumUtilTest, TestMemberExtraction) {
 
 TEST(QuorumUtilTest, TestDiffConsensusStates) {
   ConsensusStatePB old_cs;
-  SetPeerInfo("A", RaftPeerPB::VOTER, old_cs.mutable_config()->add_peers());
-  SetPeerInfo("B", RaftPeerPB::VOTER, old_cs.mutable_config()->add_peers());
-  SetPeerInfo("C", RaftPeerPB::VOTER, old_cs.mutable_config()->add_peers());
+  SetPeerInfo("A", RaftPeerPB::VOTER, old_cs.mutable_committed_config()->add_peers());
+  SetPeerInfo("B", RaftPeerPB::VOTER, old_cs.mutable_committed_config()->add_peers());
+  SetPeerInfo("C", RaftPeerPB::VOTER, old_cs.mutable_committed_config()->add_peers());
   old_cs.set_current_term(1);
   old_cs.set_leader_uuid("A");
-  old_cs.mutable_config()->set_opid_index(1);
+  old_cs.mutable_committed_config()->set_opid_index(1);
 
   // Simple case of no change.
   EXPECT_EQ("no change",
@@ -84,8 +84,8 @@ TEST(QuorumUtilTest, TestDiffConsensusStates) {
   // Simulate eviction of a peer.
   {
     auto new_cs = old_cs;
-    new_cs.mutable_config()->set_opid_index(2);
-    new_cs.mutable_config()->mutable_peers()->RemoveLast();
+    new_cs.mutable_committed_config()->set_opid_index(2);
+    new_cs.mutable_committed_config()->mutable_peers()->RemoveLast();
 
     EXPECT_EQ("config changed from index 1 to 2, "
               "VOTER C (C.example.com) evicted",
@@ -95,8 +95,8 @@ TEST(QuorumUtilTest, TestDiffConsensusStates) {
   // Simulate addition of a peer.
   {
     auto new_cs = old_cs;
-    new_cs.mutable_config()->set_opid_index(2);
-    SetPeerInfo("D", RaftPeerPB::NON_VOTER, new_cs.mutable_config()->add_peers());
+    new_cs.mutable_committed_config()->set_opid_index(2);
+    SetPeerInfo("D", RaftPeerPB::NON_VOTER, new_cs.mutable_committed_config()->add_peers());
 
     EXPECT_EQ("config changed from index 1 to 2, "
               "NON_VOTER D (D.example.com) added",
@@ -106,8 +106,9 @@ TEST(QuorumUtilTest, TestDiffConsensusStates) {
   // Simulate change of a peer's member type.
   {
     auto new_cs = old_cs;
-    new_cs.mutable_config()->set_opid_index(2);
-    new_cs.mutable_config()->mutable_peers()->Mutable(2)->set_member_type(RaftPeerPB::NON_VOTER);
+    new_cs.mutable_committed_config()->set_opid_index(2);
+    new_cs.mutable_committed_config()
+      ->mutable_peers()->Mutable(2)->set_member_type(RaftPeerPB::NON_VOTER);
 
     EXPECT_EQ("config changed from index 1 to 2, "
               "C (C.example.com) changed from VOTER to NON_VOTER",
@@ -126,6 +127,31 @@ TEST(QuorumUtilTest, TestDiffConsensusStates) {
               DiffConsensusStates(no_leader_cs, new_cs));
   }
 
+  // Simulate gaining a pending config
+  {
+    auto pending_config_cs = old_cs;
+    pending_config_cs.mutable_pending_config();
+    EXPECT_EQ("now has a pending config: ", DiffConsensusStates(old_cs, pending_config_cs));
+  }
+
+  // Simulate losing a pending config
+  {
+    auto pending_config_cs = old_cs;
+    pending_config_cs.mutable_pending_config();
+    EXPECT_EQ("no longer has a pending config: ", DiffConsensusStates(pending_config_cs, old_cs));
+  }
+
+  // Simulate a change in a pending config
+  {
+    auto before_cs = old_cs;
+    SetPeerInfo("A", RaftPeerPB::VOTER, before_cs.mutable_pending_config()->add_peers());
+    auto after_cs = before_cs;
+    after_cs.mutable_pending_config()
+      ->mutable_peers()->Mutable(0)->set_member_type(RaftPeerPB::NON_VOTER);
+
+    EXPECT_EQ("pending config changed, A (A.example.com) changed from VOTER to NON_VOTER",
+              DiffConsensusStates(before_cs, after_cs));
+  }
 }
 
 } // namespace consensus

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/consensus/quorum_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/quorum_util.cc b/src/kudu/consensus/quorum_util.cc
index 17cc8eb..a0c4a41 100644
--- a/src/kudu/consensus/quorum_util.cc
+++ b/src/kudu/consensus/quorum_util.cc
@@ -71,7 +71,7 @@ Status GetRaftConfigLeader(const ConsensusStatePB& cstate, RaftPeerPB* peer_pb)
   if (!cstate.has_leader_uuid() || cstate.leader_uuid().empty()) {
     return Status::NotFound("Consensus config has no leader");
   }
-  return GetRaftConfigMember(cstate.config(), cstate.leader_uuid(), peer_pb);
+  return GetRaftConfigMember(cstate.committed_config(), cstate.leader_uuid(), peer_pb);
 }
 
 bool RemoveFromRaftConfig(RaftConfigPB* config, const string& uuid) {
@@ -104,16 +104,19 @@ int MajoritySize(int num_voters) {
   return (num_voters / 2) + 1;
 }
 
-RaftPeerPB::Role GetConsensusRole(const std::string& uuid,
-                                    const ConsensusStatePB& cstate) {
+RaftPeerPB::Role GetConsensusRole(const std::string& uuid, const ConsensusStatePB& cstate) {
+  // The active config is the pending config if there is one, else it's the committed config.
+  const RaftConfigPB& config = cstate.has_pending_config() ?
+                               cstate.pending_config() :
+                               cstate.committed_config();
   if (cstate.leader_uuid() == uuid) {
-    if (IsRaftConfigVoter(uuid, cstate.config())) {
+    if (IsRaftConfigVoter(uuid, config)) {
       return RaftPeerPB::LEADER;
     }
     return RaftPeerPB::NON_PARTICIPANT;
   }
 
-  for (const RaftPeerPB& peer : cstate.config().peers()) {
+  for (const RaftPeerPB& peer : config.peers()) {
     if (peer.permanent_uuid() == uuid) {
       switch (peer.member_type()) {
         case RaftPeerPB::VOTER:
@@ -174,20 +177,26 @@ Status VerifyRaftConfig(const RaftConfigPB& config, RaftConfigState type) {
   return Status::OK();
 }
 
-Status VerifyConsensusState(const ConsensusStatePB& cstate, RaftConfigState type) {
+Status VerifyConsensusState(const ConsensusStatePB& cstate) {
   if (!cstate.has_current_term()) {
     return Status::IllegalState("ConsensusStatePB missing current_term",
                                 SecureShortDebugString(cstate));
   }
-  if (!cstate.has_config()) {
+  if (!cstate.has_committed_config()) {
     return Status::IllegalState("ConsensusStatePB missing config", SecureShortDebugString(cstate));
   }
-  RETURN_NOT_OK(VerifyRaftConfig(cstate.config(), type));
+  RETURN_NOT_OK(VerifyRaftConfig(cstate.committed_config(), COMMITTED_CONFIG));
+  if (cstate.has_pending_config()) {
+    RETURN_NOT_OK(VerifyRaftConfig(cstate.pending_config(), PENDING_CONFIG));
+  }
 
   if (cstate.has_leader_uuid() && !cstate.leader_uuid().empty()) {
-    if (!IsRaftConfigVoter(cstate.leader_uuid(), cstate.config())) {
+    if (!IsRaftConfigVoter(cstate.leader_uuid(), cstate.committed_config())
+        && cstate.has_pending_config()
+        && !IsRaftConfigVoter(cstate.leader_uuid(), cstate.pending_config())) {
       return Status::IllegalState(
-          Substitute("Leader with UUID $0 is not a VOTER in the config! Consensus state: $1",
+          Substitute("Leader with UUID $0 is not a VOTER in the committed or pending config! "
+                     "Consensus state: $1",
                      cstate.leader_uuid(), SecureShortDebugString(cstate)));
     }
   }
@@ -200,38 +209,95 @@ std::string DiffRaftConfigs(const RaftConfigPB& old_config,
   // Create dummy ConsensusState objects so we can reuse the code
   // from the below function.
   ConsensusStatePB old_state;
-  old_state.mutable_config()->CopyFrom(old_config);
+  old_state.mutable_committed_config()->CopyFrom(old_config);
   ConsensusStatePB new_state;
-  new_state.mutable_config()->CopyFrom(new_config);
+  new_state.mutable_committed_config()->CopyFrom(new_config);
 
   return DiffConsensusStates(old_state, new_state);
 }
 
+namespace {
+
+// A mapping from peer UUID to to <old peer, new peer> pairs.
+typedef map<string, pair<RaftPeerPB, RaftPeerPB>> PeerInfoMap;
+
+bool DiffPeers(const PeerInfoMap& peer_infos,
+               vector<string>* change_strs) {
+  bool changes = false;
+  for (const auto& e : peer_infos) {
+    const auto& old_peer = e.second.first;
+    const auto& new_peer = e.second.second;
+    if (old_peer.has_permanent_uuid() && !new_peer.has_permanent_uuid()) {
+      changes = true;
+      change_strs->push_back(
+        Substitute("$0 $1 ($2) evicted",
+                   RaftPeerPB_MemberType_Name(old_peer.member_type()),
+                   old_peer.permanent_uuid(),
+                   old_peer.last_known_addr().host()));
+    } else if (!old_peer.has_permanent_uuid() && new_peer.has_permanent_uuid()) {
+      changes = true;
+      change_strs->push_back(
+        Substitute("$0 $1 ($2) added",
+                   RaftPeerPB_MemberType_Name(new_peer.member_type()),
+                   new_peer.permanent_uuid(),
+                   new_peer.last_known_addr().host()));
+    } else if (old_peer.has_permanent_uuid() && new_peer.has_permanent_uuid()) {
+      changes = true;
+      if (old_peer.member_type() != new_peer.member_type()) {
+        change_strs->push_back(
+          Substitute("$0 ($1) changed from $2 to $3",
+                     old_peer.permanent_uuid(),
+                     old_peer.last_known_addr().host(),
+                     RaftPeerPB_MemberType_Name(old_peer.member_type()),
+                     RaftPeerPB_MemberType_Name(new_peer.member_type())));
+      }
+    }
+  }
+  return changes;
+}
+
+string PeersString(const RaftConfigPB& config) {
+  vector<string> strs;
+  for (const auto& p : config.peers()) {
+    strs.push_back(Substitute("$0 $1 ($2)",
+                              RaftPeerPB_MemberType_Name(p.member_type()),
+                              p.permanent_uuid(),
+                              p.last_known_addr().host()));
+  }
+  return JoinStrings(strs, ", ");
+}
+
+} // anonymous namespace
+
 string DiffConsensusStates(const ConsensusStatePB& old_state,
                            const ConsensusStatePB& new_state) {
   bool leader_changed = old_state.leader_uuid() != new_state.leader_uuid();
   bool term_changed = old_state.current_term() != new_state.current_term();
-  bool config_changed = old_state.config().opid_index() != new_state.config().opid_index();
+  bool config_changed =
+      old_state.committed_config().opid_index() != new_state.committed_config().opid_index();
+
+  bool pending_config_gained = !old_state.has_pending_config() && new_state.has_pending_config();
+  bool pending_config_lost = old_state.has_pending_config() && !new_state.has_pending_config();
 
   // Construct a map from Peer UUID to '<old peer, new peer>' pairs.
   // Due to the default construction nature of std::map and std::pair, if a peer
   // is present in one configuration but not the other, we'll end up with an empty
   // protobuf in that element of the pair.
-  map<string, pair<RaftPeerPB, RaftPeerPB>> peer_infos;
-  for (const auto& p : old_state.config().peers()) {
-    peer_infos[p.permanent_uuid()].first = p;
+  PeerInfoMap committed_peer_infos;
+  for (const auto& p : old_state.committed_config().peers()) {
+    committed_peer_infos[p.permanent_uuid()].first = p;
   }
-  for (const auto& p : new_state.config().peers()) {
-    peer_infos[p.permanent_uuid()].second = p;
+  for (const auto& p : new_state.committed_config().peers()) {
+    committed_peer_infos[p.permanent_uuid()].second = p;
   }
 
   // Now collect strings representing the changes.
   vector<string> change_strs;
   if (config_changed) {
     change_strs.push_back(
-        Substitute("config changed from index $0 to $1",
-                   old_state.config().opid_index(),
-                   new_state.config().opid_index()));
+      Substitute("config changed from index $0 to $1",
+                 old_state.committed_config().opid_index(),
+                 new_state.committed_config().opid_index()));
   }
 
   if (term_changed) {
@@ -247,42 +313,47 @@ string DiffConsensusStates(const ConsensusStatePB& old_state,
     if (old_state.has_leader_uuid()) {
       old_leader = Substitute("$0 ($1)",
                               old_state.leader_uuid(),
-                              peer_infos[old_state.leader_uuid()].first.last_known_addr().host());
+                              committed_peer_infos[old_state.leader_uuid()].first
+                                  .last_known_addr().host());
     }
     if (new_state.has_leader_uuid()) {
       new_leader = Substitute("$0 ($1)",
                               new_state.leader_uuid(),
-                              peer_infos[new_state.leader_uuid()].second.last_known_addr().host());
+                              committed_peer_infos[new_state.leader_uuid()].second
+                                  .last_known_addr().host());
     }
 
     change_strs.push_back(Substitute("leader changed from $0 to $1",
                                      old_leader, new_leader));
   }
 
-  for (const auto& e : peer_infos) {
-    const auto& old_peer = e.second.first;
-    const auto& new_peer = e.second.second;
-    if (old_peer.has_permanent_uuid() && !new_peer.has_permanent_uuid()) {
-      change_strs.push_back(
-          Substitute("$0 $1 ($2) evicted",
-                     RaftPeerPB_MemberType_Name(old_peer.member_type()),
-                     old_peer.permanent_uuid(),
-                     old_peer.last_known_addr().host()));
-    } else if (!old_peer.has_permanent_uuid() && new_peer.has_permanent_uuid()) {
-      change_strs.push_back(
-          Substitute("$0 $1 ($2) added",
-                     RaftPeerPB_MemberType_Name(new_peer.member_type()),
-                     new_peer.permanent_uuid(),
-                     new_peer.last_known_addr().host()));
-    } else if (old_peer.has_permanent_uuid() && new_peer.has_permanent_uuid()) {
-      if (old_peer.member_type() != new_peer.member_type()) {
-        change_strs.push_back(
-            Substitute("$0 ($1) changed from $2 to $3",
-                       old_peer.permanent_uuid(),
-                       old_peer.last_known_addr().host(),
-                       RaftPeerPB_MemberType_Name(old_peer.member_type()),
-                       RaftPeerPB_MemberType_Name(new_peer.member_type())));
-      }
+  DiffPeers(committed_peer_infos, &change_strs);
+
+  if (pending_config_gained) {
+    change_strs.push_back(Substitute("now has a pending config: $0",
+                                     PeersString(new_state.pending_config())));
+  }
+  if (pending_config_lost) {
+    change_strs.push_back(Substitute("no longer has a pending config: $0",
+                                     PeersString(old_state.pending_config())));
+  }
+
+  // A pending config doesn't have a committed opid_index yet, so we determine if there's a change
+  // by computing the peer differences.
+  if (old_state.has_pending_config() && new_state.has_pending_config()) {
+    PeerInfoMap pending_peer_infos;
+    for (const auto &p : old_state.pending_config().peers()) {
+      pending_peer_infos[p.permanent_uuid()].first = p;
+    }
+    for (const auto &p : new_state.pending_config().peers()) {
+      pending_peer_infos[p.permanent_uuid()].second = p;
+    }
+
+    vector<string> pending_change_strs;
+    if (DiffPeers(pending_peer_infos, &pending_change_strs)) {
+      change_strs.push_back("pending config changed");
+      change_strs.insert(change_strs.end(), pending_change_strs.cbegin(),
+                         pending_change_strs.cend());
     }
   }
 
@@ -299,9 +370,7 @@ string DiffConsensusStates(const ConsensusStatePB& old_state,
                       SecureShortDebugString(new_state));
   }
 
-
   return JoinStrings(change_strs, ", ");
-
 }
 
 }  // namespace consensus

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/consensus/quorum_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/quorum_util.h b/src/kudu/consensus/quorum_util.h
index ed21e9d..9c4796c 100644
--- a/src/kudu/consensus/quorum_util.h
+++ b/src/kudu/consensus/quorum_util.h
@@ -28,8 +28,8 @@ class Status;
 namespace consensus {
 
 enum RaftConfigState {
-  UNCOMMITTED_QUORUM,
-  COMMITTED_QUORUM,
+  PENDING_CONFIG,
+  COMMITTED_CONFIG,
 };
 
 bool IsRaftConfigMember(const std::string& uuid, const RaftConfigPB& config);
@@ -71,7 +71,7 @@ Status VerifyRaftConfig(const RaftConfigPB& config, RaftConfigState type);
 
 // Superset of checks performed by VerifyRaftConfig. Also ensures that the
 // leader is a configuration voter, if it is set, and that a valid term is set.
-Status VerifyConsensusState(const ConsensusStatePB& cstate, RaftConfigState type);
+Status VerifyConsensusState(const ConsensusStatePB& cstate);
 
 // Provide a textual description of the difference between two consensus states,
 // suitable for logging.

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/consensus/raft_consensus.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/raft_consensus.cc b/src/kudu/consensus/raft_consensus.cc
index 91040bb..4c60664 100644
--- a/src/kudu/consensus/raft_consensus.cc
+++ b/src/kudu/consensus/raft_consensus.cc
@@ -1537,10 +1537,9 @@ Status RaftConsensus::ChangeConfig(const ChangeConfigRequestPB& req,
           return Status::InvalidArgument(
               Substitute("Cannot remove peer $0 from the config because it is the leader. "
                          "Force another leader to be elected to remove this server. "
-                         "Active consensus state: $1",
+                         "Consensus state: $1",
                          server_uuid,
-                         SecureShortDebugString(state_->ConsensusStateUnlocked(
-                             CONSENSUS_CONFIG_ACTIVE))));
+                         SecureShortDebugString(state_->ConsensusStateUnlocked())));
         }
         if (!RemoveFromRaftConfig(&new_config, server_uuid)) {
           return Status::NotFound(
@@ -1654,7 +1653,7 @@ Status RaftConsensus::UnsafeChangeConfig(const UnsafeChangeConfigRequestPB& req,
   new_config.set_opid_index(replicate_opid_index);
 
   // Sanity check the new config. 'type' is irrelevant here.
-  Status s = VerifyRaftConfig(new_config, UNCOMMITTED_QUORUM);
+  Status s = VerifyRaftConfig(new_config, PENDING_CONFIG);
   if (!s.ok()) {
     *error_code = TabletServerErrorPB::INVALID_CONFIG;
     return Status::InvalidArgument(Substitute("The resulting new config for tablet $0  "
@@ -1974,10 +1973,10 @@ string RaftConsensus::tablet_id() const {
   return state_->GetOptions().tablet_id;
 }
 
-ConsensusStatePB RaftConsensus::ConsensusState(ConsensusConfigType type) const {
+ConsensusStatePB RaftConsensus::ConsensusState() const {
   ReplicaState::UniqueLock lock;
   CHECK_OK(state_->LockForRead(&lock));
-  return state_->ConsensusStateUnlocked(type);
+  return state_->ConsensusStateUnlocked();
 }
 
 RaftConfigPB RaftConsensus::CommittedConfig() const {

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/consensus/raft_consensus.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/raft_consensus.h b/src/kudu/consensus/raft_consensus.h
index 7d6c633..0c6fa34 100644
--- a/src/kudu/consensus/raft_consensus.h
+++ b/src/kudu/consensus/raft_consensus.h
@@ -138,7 +138,7 @@ class RaftConsensus : public Consensus,
 
   scoped_refptr<TimeManager> time_manager() const override { return time_manager_; }
 
-  ConsensusStatePB ConsensusState(ConsensusConfigType type) const override;
+  ConsensusStatePB ConsensusState() const override;
 
   RaftConfigPB CommittedConfig() const override;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/consensus/raft_consensus_state.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/raft_consensus_state.cc b/src/kudu/consensus/raft_consensus_state.cc
index ebd45e6..e06eea7 100644
--- a/src/kudu/consensus/raft_consensus_state.cc
+++ b/src/kudu/consensus/raft_consensus_state.cc
@@ -117,7 +117,7 @@ Status ReplicaState::CheckActiveLeaderUnlocked() const {
     case RaftPeerPB::LEADER:
       return Status::OK();
     default:
-      ConsensusStatePB cstate = ConsensusStateUnlocked(CONSENSUS_CONFIG_ACTIVE);
+      ConsensusStatePB cstate = ConsensusStateUnlocked();
       return Status::IllegalState(Substitute("Replica $0 is not leader of this config. Role: $1. "
                                              "Consensus state: $2",
                                              peer_uuid_,
@@ -196,7 +196,7 @@ Status ReplicaState::CheckNoConfigChangePendingUnlocked() const {
 
 Status ReplicaState::SetPendingConfigUnlocked(const RaftConfigPB& new_config) {
   DCHECK(update_lock_.is_locked());
-  RETURN_NOT_OK_PREPEND(VerifyRaftConfig(new_config, UNCOMMITTED_QUORUM),
+  RETURN_NOT_OK_PREPEND(VerifyRaftConfig(new_config, PENDING_CONFIG),
                         "Invalid config to set as pending");
   if (!new_config.unsafe_config_change()) {
     CHECK(!cmeta_->has_pending_config())
@@ -227,7 +227,7 @@ Status ReplicaState::SetCommittedConfigUnlocked(const RaftConfigPB& config_to_co
   TRACE_EVENT0("consensus", "ReplicaState::SetCommittedConfigUnlocked");
   DCHECK(update_lock_.is_locked());
   DCHECK(config_to_commit.IsInitialized());
-  RETURN_NOT_OK_PREPEND(VerifyRaftConfig(config_to_commit, COMMITTED_QUORUM),
+  RETURN_NOT_OK_PREPEND(VerifyRaftConfig(config_to_commit, COMMITTED_CONFIG),
                         "Invalid config to set as committed");
 
   // Compare committed with pending configuration, ensure that they are the same.

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/consensus/raft_consensus_state.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/raft_consensus_state.h b/src/kudu/consensus/raft_consensus_state.h
index ead377f..9350ab8 100644
--- a/src/kudu/consensus/raft_consensus_state.h
+++ b/src/kudu/consensus/raft_consensus_state.h
@@ -131,8 +131,8 @@ class ReplicaState {
   Status ShutdownUnlocked() WARN_UNUSED_RESULT;
 
   // Return current consensus state summary.
-  ConsensusStatePB ConsensusStateUnlocked(ConsensusConfigType type) const {
-    return cmeta_->ToConsensusStatePB(type);
+  ConsensusStatePB ConsensusStateUnlocked() const {
+    return cmeta_->ToConsensusStatePB();
   }
 
   // Returns the currently active Raft role.

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/integration-tests/cluster_itest_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/cluster_itest_util.cc b/src/kudu/integration-tests/cluster_itest_util.cc
index 9bc945e..58dd451 100644
--- a/src/kudu/integration-tests/cluster_itest_util.cc
+++ b/src/kudu/integration-tests/cluster_itest_util.cc
@@ -45,15 +45,10 @@
 namespace kudu {
 namespace itest {
 
-using client::KuduClient;
 using client::KuduSchema;
 using client::KuduSchemaBuilder;
-using client::KuduTable;
-using consensus::CONSENSUS_CONFIG_ACTIVE;
-using consensus::CONSENSUS_CONFIG_COMMITTED;
 using consensus::ChangeConfigRequestPB;
 using consensus::ChangeConfigResponsePB;
-using consensus::ConsensusConfigType;
 using consensus::ConsensusStatePB;
 using consensus::CountVoters;
 using consensus::GetConsensusStateRequestPB;
@@ -160,8 +155,7 @@ Status WaitForOpFromCurrentTerm(TServerDetails* replica,
   Status s;
   while (MonoTime::Now() < kDeadline) {
     ConsensusStatePB cstate;
-    s = GetConsensusState(replica, tablet_id, CONSENSUS_CONFIG_ACTIVE, kDeadline - MonoTime::Now(),
-                          &cstate);
+    s = GetConsensusState(replica, tablet_id, kDeadline - MonoTime::Now(), &cstate);
     if (s.ok()) {
       OpId tmp_opid;
       s = GetLastOpIdForReplica(tablet_id, replica, opid_type, kDeadline - MonoTime::Now(),
@@ -300,7 +294,6 @@ Status CreateTabletServerMap(const shared_ptr<MasterServiceProxy>& master_proxy,
 
 Status GetConsensusState(const TServerDetails* replica,
                          const string& tablet_id,
-                         consensus::ConsensusConfigType type,
                          const MonoDelta& timeout,
                          ConsensusStatePB* consensus_state) {
   GetConsensusStateRequestPB req;
@@ -309,7 +302,6 @@ Status GetConsensusState(const TServerDetails* replica,
   controller.set_timeout(timeout);
   req.set_dest_uuid(replica->uuid());
   req.set_tablet_id(tablet_id);
-  req.set_type(type);
 
   RETURN_NOT_OK(replica->consensus_proxy->GetConsensusState(req, &resp, &controller));
   if (resp.has_error()) {
@@ -332,10 +324,9 @@ Status WaitUntilCommittedConfigNumVotersIs(int config_size,
   ConsensusStatePB cstate;
   while (true) {
     MonoDelta remaining_timeout = deadline - MonoTime::Now();
-    s = GetConsensusState(replica, tablet_id, CONSENSUS_CONFIG_COMMITTED,
-                          remaining_timeout, &cstate);
+    s = GetConsensusState(replica, tablet_id, remaining_timeout, &cstate);
     if (s.ok()) {
-      if (CountVoters(cstate.config()) == config_size) {
+      if (CountVoters(cstate.committed_config()) == config_size) {
         return Status::OK();
       }
     }
@@ -363,9 +354,8 @@ Status WaitUntilCommittedConfigOpIdIndexIs(int64_t opid_index,
   ConsensusStatePB cstate;
   while (true) {
     MonoDelta remaining_timeout = deadline - MonoTime::Now();
-    s = GetConsensusState(replica, tablet_id, CONSENSUS_CONFIG_COMMITTED,
-                          remaining_timeout, &cstate);
-    if (s.ok() && cstate.config().opid_index() == opid_index) {
+    s = GetConsensusState(replica, tablet_id, remaining_timeout, &cstate);
+    if (s.ok() && cstate.committed_config().opid_index() == opid_index) {
       return Status::OK();
     }
     if (MonoTime::Now() > deadline) break;
@@ -490,8 +480,7 @@ Status GetReplicaStatusAndCheckIfLeader(const TServerDetails* replica,
                                         const string& tablet_id,
                                         const MonoDelta& timeout) {
   ConsensusStatePB cstate;
-  Status s = GetConsensusState(replica, tablet_id, CONSENSUS_CONFIG_ACTIVE,
-                               timeout, &cstate);
+  Status s = GetConsensusState(replica, tablet_id, timeout, &cstate);
   if (PREDICT_FALSE(!s.ok())) {
     VLOG(1) << "Error getting consensus state from replica: "
             << replica->instance_id.permanent_uuid();

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/integration-tests/cluster_itest_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/cluster_itest_util.h b/src/kudu/integration-tests/cluster_itest_util.h
index 6474ae2..6336232 100644
--- a/src/kudu/integration-tests/cluster_itest_util.h
+++ b/src/kudu/integration-tests/cluster_itest_util.h
@@ -145,7 +145,6 @@ Status WaitUntilAllReplicasHaveOp(const int64_t log_index,
 // Get the consensus state from the given replica.
 Status GetConsensusState(const TServerDetails* replica,
                          const std::string& tablet_id,
-                         consensus::ConsensusConfigType type,
                          const MonoDelta& timeout,
                          consensus::ConsensusStatePB* consensus_state);
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/integration-tests/delete_table-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/delete_table-itest.cc b/src/kudu/integration-tests/delete_table-itest.cc
index e0b0d95..9c78054 100644
--- a/src/kudu/integration-tests/delete_table-itest.cc
+++ b/src/kudu/integration-tests/delete_table-itest.cc
@@ -51,7 +51,6 @@ using kudu::client::KuduSchema;
 using kudu::client::KuduSchemaFromSchema;
 using kudu::client::KuduTable;
 using kudu::client::KuduTableCreator;
-using kudu::consensus::CONSENSUS_CONFIG_COMMITTED;
 using kudu::consensus::ConsensusMetadataPB;
 using kudu::consensus::ConsensusStatePB;
 using kudu::consensus::RaftPeerPB;
@@ -132,7 +131,7 @@ class DeleteTableITest : public ExternalMiniClusterITestBase {
 
 string DeleteTableITest::GetLeaderUUID(const string& ts_uuid, const string& tablet_id) {
   ConsensusStatePB cstate;
-  CHECK_OK(itest::GetConsensusState(ts_map_[ts_uuid], tablet_id, CONSENSUS_CONFIG_COMMITTED,
+  CHECK_OK(itest::GetConsensusState(ts_map_[ts_uuid], tablet_id,
                                     MonoDelta::FromSeconds(10), &cstate));
   return cstate.leader_uuid();
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/integration-tests/tablet_copy-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/tablet_copy-itest.cc b/src/kudu/integration-tests/tablet_copy-itest.cc
index 9b16c4a..d7fde17 100644
--- a/src/kudu/integration-tests/tablet_copy-itest.cc
+++ b/src/kudu/integration-tests/tablet_copy-itest.cc
@@ -53,7 +53,6 @@ DEFINE_int32(test_delete_leader_num_writer_threads, 1,
 using kudu::client::KuduSchema;
 using kudu::client::KuduSchemaFromSchema;
 using kudu::client::KuduTableCreator;
-using kudu::consensus::CONSENSUS_CONFIG_COMMITTED;
 using kudu::itest::TServerDetails;
 using kudu::itest::WaitForNumTabletServers;
 using kudu::tablet::TABLET_DATA_DELETED;
@@ -406,8 +405,7 @@ TEST_F(TabletCopyITest, TestTabletCopyFollowerWithHigherTerm) {
   int64_t term = 0;
   for (int i = 0; i < 1000; i++) {
     consensus::ConsensusStatePB cstate;
-    ASSERT_OK(itest::GetConsensusState(follower_ts, tablet_id, CONSENSUS_CONFIG_COMMITTED,
-                                       timeout, &cstate));
+    ASSERT_OK(itest::GetConsensusState(follower_ts, tablet_id, timeout, &cstate));
     term = cstate.current_term();
     if (term == 2) break;
     SleepFor(MonoDelta::FromMilliseconds(10));

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/integration-tests/ts_tablet_manager-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/ts_tablet_manager-itest.cc b/src/kudu/integration-tests/ts_tablet_manager-itest.cc
index b632050..9d5c6a5 100644
--- a/src/kudu/integration-tests/ts_tablet_manager-itest.cc
+++ b/src/kudu/integration-tests/ts_tablet_manager-itest.cc
@@ -181,11 +181,11 @@ TEST_F(TsTabletManagerITest, TestReportNewLeaderOnLeaderChange) {
       TabletReportPB& report = reports[0];
       ASSERT_EQ(1, report.updated_tablets_size())
           << "Wrong report size:\n" << SecureDebugString(report);
-      ReportedTabletPB reported_tablet = report.updated_tablets(0);
-      ASSERT_TRUE(reported_tablet.has_committed_consensus_state());
+      const ReportedTabletPB& reported_tablet = report.updated_tablets(0);
+      ASSERT_TRUE(reported_tablet.has_consensus_state());
 
       string uuid = tablet_replicas[replica]->permanent_uuid();
-      RaftPeerPB::Role role = GetConsensusRole(uuid, reported_tablet.committed_consensus_state());
+      RaftPeerPB::Role role = GetConsensusRole(uuid, reported_tablet.consensus_state());
       if (replica == new_leader_idx) {
         ASSERT_EQ(RaftPeerPB::LEADER, role)
             << "Tablet report: " << SecureShortDebugString(report);

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index a6763a7..2080b61 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -222,7 +222,6 @@ namespace master {
 using base::subtle::NoBarrier_CompareAndSwap;
 using base::subtle::NoBarrier_Load;
 using cfile::TypeEncodingInfo;
-using consensus::CONSENSUS_CONFIG_COMMITTED;
 using consensus::Consensus;
 using consensus::ConsensusServiceProxy;
 using consensus::ConsensusStatePB;
@@ -744,8 +743,7 @@ Status CatalogManager::ElectedAsLeaderCb() {
 }
 
 Status CatalogManager::WaitUntilCaughtUpAsLeader(const MonoDelta& timeout) {
-  ConsensusStatePB cstate = sys_catalog_->tablet_replica()->consensus()->
-      ConsensusState(CONSENSUS_CONFIG_COMMITTED);
+  ConsensusStatePB cstate = sys_catalog_->tablet_replica()->consensus()->ConsensusState();
   const string& uuid = master_->fs_manager()->uuid();
   if (!cstate.has_leader_uuid() || cstate.leader_uuid() != uuid) {
     return Status::IllegalState(
@@ -928,8 +926,7 @@ void CatalogManager::PrepareForLeadershipTask() {
     shared_lock<LockType> l(lock_);
   }
   const Consensus* consensus = sys_catalog_->tablet_replica()->consensus();
-  const int64_t term_before_wait =
-      consensus->ConsensusState(CONSENSUS_CONFIG_COMMITTED).current_term();
+  const int64_t term_before_wait = consensus->ConsensusState().current_term();
   {
     std::lock_guard<simple_spinlock> l(state_lock_);
     if (leader_ready_term_ == term_before_wait) {
@@ -954,8 +951,7 @@ void CatalogManager::PrepareForLeadershipTask() {
     return;
   }
 
-  const int64_t term =
-      consensus->ConsensusState(CONSENSUS_CONFIG_COMMITTED).current_term();
+  const int64_t term = consensus->ConsensusState().current_term();
   if (term_before_wait != term) {
     // If we got elected leader again while waiting to catch up then we will
     // get another callback to visit the tables and tablets, so bail.
@@ -993,8 +989,7 @@ void CatalogManager::PrepareForLeadershipTask() {
         }
       }
 
-      const int64_t term =
-          consensus.ConsensusState(CONSENSUS_CONFIG_COMMITTED).current_term();
+      const int64_t term = consensus.ConsensusState().current_term();
       if (term != start_term) {
         // If the term has changed we assume the new leader catalog is about
         // to do the necessary work in its leadership preparation task.
@@ -2365,8 +2360,11 @@ bool ShouldTransitionTabletToRunning(const ReportedTabletPB& report) {
     return true;
   }
 
-  // Otherwise, we only transition to RUNNING once a leader is elected.
-  return report.committed_consensus_state().has_leader_uuid();
+  // Otherwise, we only transition to RUNNING once there is a leader that is a
+  // member of the committed configuration.
+  const ConsensusStatePB& cstate = report.consensus_state();
+  return cstate.has_leader_uuid() &&
+      IsRaftConfigMember(cstate.leader_uuid(), cstate.committed_config());
 }
 } // anonymous namespace
 
@@ -2466,11 +2464,11 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
     return Status::OK();
   }
 
-  // The report will not have a committed_consensus_state if it is in the
+  // The report will not have a consensus_state if it is in the
   // middle of starting up, such as during tablet bootstrap.
-  if (report.has_committed_consensus_state()) {
-    const ConsensusStatePB& prev_cstate = tablet_lock.data().pb.committed_consensus_state();
-    ConsensusStatePB cstate = report.committed_consensus_state();
+  if (report.has_consensus_state()) {
+    const ConsensusStatePB& prev_cstate = tablet_lock.data().pb.consensus_state();
+    ConsensusStatePB cstate = report.consensus_state();
 
     // Check if we got a report from a tablet that is no longer part of the raft
     // config. If so, tombstone it. We only tombstone replicas that include a
@@ -2481,17 +2479,24 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
     // in the process of catching up to the log entry where they were added to
     // the config.
     if (FLAGS_master_tombstone_evicted_tablet_replicas &&
-        cstate.config().opid_index() < prev_cstate.config().opid_index() &&
-        !IsRaftConfigMember(ts_desc->permanent_uuid(), prev_cstate.config())) {
+        cstate.committed_config().opid_index() < prev_cstate.committed_config().opid_index() &&
+        !IsRaftConfigMember(ts_desc->permanent_uuid(), prev_cstate.committed_config())) {
       SendDeleteReplicaRequest(report.tablet_id(), TABLET_DATA_TOMBSTONED,
-                               prev_cstate.config().opid_index(),
+                               prev_cstate.committed_config().opid_index(),
                                tablet->table(), ts_desc->permanent_uuid(),
                                Substitute("Replica from old config with index $0 (latest is $1)",
-                                          cstate.config().opid_index(),
-                                          prev_cstate.config().opid_index()));
+                                          cstate.committed_config().opid_index(),
+                                          prev_cstate.committed_config().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() &&
+        !IsRaftConfigMember(cstate.leader_uuid(), cstate.committed_config())) {
+      cstate.clear_leader_uuid();
+    }
+
     // If the tablet was not RUNNING, and we have a leader elected, mark it as RUNNING.
     // We need to wait for a leader before marking a tablet as RUNNING, or else we
     // could incorrectly consider a tablet created when only a minority of its replicas
@@ -2511,13 +2516,13 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
 
     // 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.config().has_opid_index())) {
+    if (PREDICT_FALSE(!cstate.committed_config().has_opid_index())) {
       LOG(DFATAL) << "Missing opid_index in reported config:\n" << SecureDebugString(report);
       return Status::InvalidArgument("Missing opid_index in reported config");
     }
 
     bool modified_cstate = false;
-    if (cstate.config().opid_index() > prev_cstate.config().opid_index() ||
+    if (cstate.committed_config().opid_index() > prev_cstate.committed_config().opid_index() ||
         (cstate.has_leader_uuid() &&
          (!prev_cstate.has_leader_uuid() || cstate.current_term() > prev_cstate.current_term()))) {
 
@@ -2557,7 +2562,7 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
       ReportedTabletPB updated_report;
       if (modified_cstate) {
         updated_report = report;
-        *updated_report.mutable_committed_consensus_state() = cstate;
+        *updated_report.mutable_consensus_state() = cstate;
         final_report = &updated_report;
       }
 
@@ -2565,9 +2570,9 @@ Status CatalogManager::HandleReportedTablet(TSDescriptor* ts_desc,
               << final_report->tablet_id()
               << " from config reported by " << ts_desc->ToString()
               << " to that committed in log index "
-              << final_report->committed_consensus_state().config().opid_index()
+              << final_report->consensus_state().committed_config().opid_index()
               << " with leader state from term "
-              << final_report->committed_consensus_state().current_term();
+              << final_report->consensus_state().current_term();
 
       RETURN_NOT_OK(HandleRaftConfigChanged(*final_report, tablet,
                                             &tablet_lock, &table_lock));
@@ -2607,30 +2612,31 @@ Status CatalogManager::HandleRaftConfigChanged(
     TableMetadataLock* table_lock) {
 
   DCHECK(tablet_lock->is_write_locked());
-  ConsensusStatePB prev_cstate = tablet_lock->mutable_data()->pb.committed_consensus_state();
-  const ConsensusStatePB& cstate = report.committed_consensus_state();
-  *tablet_lock->mutable_data()->pb.mutable_committed_consensus_state() = cstate;
+  ConsensusStatePB prev_cstate = tablet_lock->mutable_data()->pb.consensus_state();
+  const ConsensusStatePB& cstate = report.consensus_state();
+  *tablet_lock->mutable_data()->pb.mutable_consensus_state() = cstate;
 
   if (FLAGS_master_tombstone_evicted_tablet_replicas) {
     unordered_set<string> current_member_uuids;
-    for (const consensus::RaftPeerPB& peer : cstate.config().peers()) {
+    for (const consensus::RaftPeerPB& peer : cstate.committed_config().peers()) {
       InsertOrDie(&current_member_uuids, peer.permanent_uuid());
     }
     // Send a DeleteTablet() request to peers that are not in the new config.
-    for (const consensus::RaftPeerPB& prev_peer : prev_cstate.config().peers()) {
+    for (const consensus::RaftPeerPB& prev_peer : prev_cstate.committed_config().peers()) {
       const string& peer_uuid = prev_peer.permanent_uuid();
       if (!ContainsKey(current_member_uuids, peer_uuid)) {
         SendDeleteReplicaRequest(report.tablet_id(), TABLET_DATA_TOMBSTONED,
-                                 prev_cstate.config().opid_index(), tablet->table(), peer_uuid,
+                                 prev_cstate.committed_config().opid_index(),
+                                 tablet->table(), peer_uuid,
                                  Substitute("TS $0 not found in new config with opid_index $1",
-                                            peer_uuid, cstate.config().opid_index()));
+                                            peer_uuid, cstate.committed_config().opid_index()));
       }
     }
   }
 
   // If the config is under-replicated, add a server to the config.
   if (FLAGS_master_add_server_when_underreplicated &&
-      CountVoters(cstate.config()) < table_lock->data().pb.num_replicas()) {
+      CountVoters(cstate.committed_config()) < table_lock->data().pb.num_replicas()) {
     SendAddServerRequest(tablet, cstate);
   }
 
@@ -2710,16 +2716,16 @@ class PickLeaderReplica : public TSPicker {
     TabletMetadataLock l(tablet_.get(), TabletMetadataLock::READ);
 
     string err_msg;
-    if (!l.data().pb.has_committed_consensus_state()) {
+    if (!l.data().pb.has_consensus_state()) {
       // The tablet is still in the PREPARING state and has no replicas.
       err_msg = Substitute("Tablet $0 has no consensus state",
                            tablet_->tablet_id());
-    } else if (!l.data().pb.committed_consensus_state().has_leader_uuid()) {
+    } else if (!l.data().pb.consensus_state().has_leader_uuid()) {
       // The tablet may be in the midst of a leader election.
       err_msg = Substitute("Tablet $0 consensus state has no leader",
                            tablet_->tablet_id());
     } else {
-      *ts_uuid = l.data().pb.committed_consensus_state().leader_uuid();
+      *ts_uuid = l.data().pb.consensus_state().leader_uuid();
       return Status::OK();
     }
     return Status::NotFound("No leader found", err_msg);
@@ -3011,7 +3017,7 @@ class AsyncCreateReplica : public RetrySpecificTSRpcTask {
     req_.mutable_partition_schema()->CopyFrom(
         table_lock.data().pb.partition_schema());
     req_.mutable_config()->CopyFrom(
-        tablet_lock.data().pb.committed_consensus_state().config());
+        tablet_lock.data().pb.consensus_state().committed_config());
   }
 
   virtual string type_name() const OVERRIDE { return "Create Tablet"; }
@@ -3290,7 +3296,7 @@ string AsyncAddServerTask::description() const {
   return Substitute("AddServer ChangeConfig RPC for tablet $0 "
                     "with cas_config_opid_index $1",
                     tablet_->tablet_id(),
-                    cstate_.config().opid_index());
+                    cstate_.committed_config().opid_index());
 }
 
 bool AsyncAddServerTask::SendRequest(int attempt) {
@@ -3301,12 +3307,13 @@ bool AsyncAddServerTask::SendRequest(int attempt) {
   int64_t latest_index;
   {
     TabletMetadataLock tablet_lock(tablet_.get(), TabletMetadataLock::READ);
-    latest_index = tablet_lock.data().pb.committed_consensus_state().config().opid_index();
+    latest_index = tablet_lock.data().pb.consensus_state()
+      .committed_config().opid_index();
   }
-  if (latest_index > cstate_.config().opid_index()) {
+  if (latest_index > cstate_.committed_config().opid_index()) {
     LOG_WITH_PREFIX(INFO) << "Latest config for has opid_index of " << latest_index
                           << " while this task has opid_index of "
-                          << cstate_.config().opid_index() << ". Aborting task.";
+                          << cstate_.committed_config().opid_index() << ". Aborting task.";
     MarkAborted();
     return false;
   }
@@ -3314,7 +3321,7 @@ bool AsyncAddServerTask::SendRequest(int attempt) {
   // Select the replica we wish to add to the config.
   // Do not include current members of the config.
   unordered_set<string> replica_uuids;
-  for (const RaftPeerPB& peer : cstate_.config().peers()) {
+  for (const RaftPeerPB& peer : cstate_.committed_config().peers()) {
     InsertOrDie(&replica_uuids, peer.permanent_uuid());
   }
   TSDescriptorVector ts_descs;
@@ -3329,7 +3336,7 @@ bool AsyncAddServerTask::SendRequest(int attempt) {
   req_.set_dest_uuid(target_ts_desc_->permanent_uuid());
   req_.set_tablet_id(tablet_->tablet_id());
   req_.set_type(consensus::ADD_SERVER);
-  req_.set_cas_config_opid_index(cstate_.config().opid_index());
+  req_.set_cas_config_opid_index(cstate_.committed_config().opid_index());
   RaftPeerPB* peer = req_.mutable_server();
   peer->set_permanent_uuid(replacement_replica->permanent_uuid());
   ServerRegistrationPB peer_reg;
@@ -3403,7 +3410,7 @@ void CatalogManager::SendDeleteTableRequest(const scoped_refptr<TableInfo>& tabl
 void CatalogManager::SendDeleteTabletRequest(const scoped_refptr<TabletInfo>& tablet,
                                              const TabletMetadataLock& tablet_lock,
                                              const string& deletion_msg) {
-  if (!tablet_lock.data().pb.has_committed_consensus_state()) {
+  if (!tablet_lock.data().pb.has_consensus_state()) {
     // We could end up here if we're deleting a tablet that never made it to
     // the CREATING state. That would mean no replicas were ever assigned, so
     // there's nothing to delete.
@@ -3411,11 +3418,11 @@ void CatalogManager::SendDeleteTabletRequest(const scoped_refptr<TabletInfo>& ta
               << tablet->tablet_id();
     return;
   }
-  const ConsensusStatePB& cstate = tablet_lock.data().pb.committed_consensus_state();
+  const ConsensusStatePB& cstate = tablet_lock.data().pb.consensus_state();
   LOG_WITH_PREFIX(INFO)
-      << "Sending DeleteTablet for " << cstate.config().peers().size()
+      << "Sending DeleteTablet for " << cstate.committed_config().peers().size()
       << " replicas of tablet " << tablet->tablet_id();
-  for (const auto& peer : cstate.config().peers()) {
+  for (const auto& peer : cstate.committed_config().peers()) {
     SendDeleteReplicaRequest(tablet->tablet_id(), TABLET_DATA_DELETED,
                              boost::none, tablet->table(),
                              peer.permanent_uuid(), deletion_msg);
@@ -3773,9 +3780,9 @@ Status CatalogManager::SelectReplicasForTablet(const TSDescriptorVector& ts_desc
 
   // Select the set of replicas for the tablet.
   ConsensusStatePB* cstate = tablet->mutable_metadata()->mutable_dirty()
-          ->pb.mutable_committed_consensus_state();
+          ->pb.mutable_consensus_state();
   cstate->set_current_term(kMinimumTerm);
-  consensus::RaftConfigPB *config = cstate->mutable_config();
+  consensus::RaftConfigPB *config = cstate->mutable_committed_config();
 
   // Maintain ability to downgrade Kudu to a version with LocalConsensus.
   if (nreplicas == 1) {
@@ -3792,7 +3799,7 @@ Status CatalogManager::SelectReplicasForTablet(const TSDescriptorVector& ts_desc
 void CatalogManager::SendCreateTabletRequest(const scoped_refptr<TabletInfo>& tablet,
                                              const TabletMetadataLock& tablet_lock) {
   const consensus::RaftConfigPB& config =
-      tablet_lock.data().pb.committed_consensus_state().config();
+      tablet_lock.data().pb.consensus_state().committed_config();
   tablet->set_last_create_tablet_time(MonoTime::Now());
   for (const RaftPeerPB& peer : config.peers()) {
     AsyncCreateReplica* task = new AsyncCreateReplica(master_,
@@ -3926,11 +3933,11 @@ Status CatalogManager::BuildLocationsForTablet(const scoped_refptr<TabletInfo>&
   }
 
   // Guaranteed because the tablet is RUNNING.
-  DCHECK(l_tablet.data().pb.has_committed_consensus_state());
+  DCHECK(l_tablet.data().pb.has_consensus_state());
 
-  const ConsensusStatePB& cstate = l_tablet.data().pb.committed_consensus_state();
-  for (const consensus::RaftPeerPB& peer : cstate.config().peers()) {
-    // TODO: GetConsensusRole() iterates over all of the peers, making this an
+  const ConsensusStatePB& cstate = l_tablet.data().pb.consensus_state();
+  for (const consensus::RaftPeerPB& peer : cstate.committed_config().peers()) {
+    // TODO(adar): GetConsensusRole() iterates over all of the peers, making this an
     // O(n^2) loop. If replication counts get high, it should be optimized.
     TabletLocationsPB_ReplicaPB* replica_pb = locs_pb->add_replicas();
     replica_pb->set_role(GetConsensusRole(peer.permanent_uuid(), cstate));
@@ -4139,7 +4146,7 @@ CatalogManager::ScopedLeaderSharedLock::ScopedLeaderSharedLock(
 
   // Check if the catalog manager is the leader.
   const ConsensusStatePB cstate = catalog_->sys_catalog_->tablet_replica()->
-      consensus()->ConsensusState(CONSENSUS_CONFIG_COMMITTED);
+      consensus()->ConsensusState();
   initial_term_ = cstate.current_term();
   const string& uuid = catalog_->master_->fs_manager()->uuid();
   if (PREDICT_FALSE(!cstate.has_leader_uuid() || cstate.leader_uuid() != uuid)) {
@@ -4160,7 +4167,7 @@ CatalogManager::ScopedLeaderSharedLock::ScopedLeaderSharedLock(
 bool CatalogManager::ScopedLeaderSharedLock::has_term_changed() const {
   DCHECK(leader_status().ok());
   const ConsensusStatePB cstate = catalog_->sys_catalog_->tablet_replica()->
-      consensus()->ConsensusState(CONSENSUS_CONFIG_COMMITTED);
+      consensus()->ConsensusState();
   return cstate.current_term() != initial_term_;
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/master/master-path-handlers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master-path-handlers.cc b/src/kudu/master/master-path-handlers.cc
index 70793a7..a2ad456 100644
--- a/src/kudu/master/master-path-handlers.cc
+++ b/src/kudu/master/master-path-handlers.cc
@@ -261,10 +261,11 @@ void MasterPathHandlers::HandleTablePage(const Webserver::WebRequest& req,
   for (const scoped_refptr<TabletInfo>& tablet : tablets) {
     vector<pair<string, RaftPeerPB::Role>> sorted_replicas;
     TabletMetadataLock l(tablet.get(), TabletMetadataLock::READ);
+
     summary_states[SysTabletsEntryPB_State_Name(l.data().pb.state())]++;
-    if (l.data().pb.has_committed_consensus_state()) {
-      const ConsensusStatePB& cstate = l.data().pb.committed_consensus_state();
-      for (const auto& peer : cstate.config().peers()) {
+    if (l.data().pb.has_consensus_state()) {
+      const ConsensusStatePB& cstate = l.data().pb.consensus_state();
+      for (const auto& peer : cstate.committed_config().peers()) {
         RaftPeerPB::Role role = GetConsensusRole(peer.permanent_uuid(), cstate);
         string html;
         string location_html;
@@ -466,11 +467,11 @@ class JsonDumper : public TableVisitor, public TabletVisitor {
     jw_->String(SysTabletsEntryPB::State_Name(metadata.state()));
 
     // Dump replica UUIDs
-    if (metadata.has_committed_consensus_state()) {
-      const consensus::ConsensusStatePB& cs = metadata.committed_consensus_state();
+    if (metadata.has_consensus_state()) {
+      const consensus::ConsensusStatePB& cs = metadata.consensus_state();
       jw_->String("replicas");
       jw_->StartArray();
-      for (const RaftPeerPB& peer : cs.config().peers()) {
+      for (const RaftPeerPB& peer : cs.committed_config().peers()) {
         jw_->StartObject();
         jw_->String("type");
         jw_->String(RaftPeerPB::MemberType_Name(peer.member_type()));

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/master/master.proto
----------------------------------------------------------------------
diff --git a/src/kudu/master/master.proto b/src/kudu/master/master.proto
index fa49283..240b866 100644
--- a/src/kudu/master/master.proto
+++ b/src/kudu/master/master.proto
@@ -122,8 +122,8 @@ message SysTabletsEntryPB {
   // Tablet partition.
   optional PartitionPB partition = 7;
 
-  // The latest committed consensus configuration reported to the Master.
-  optional consensus.ConsensusStatePB committed_consensus_state = 3;
+  // The latest consensus state reported to the Master.
+  optional consensus.ConsensusStatePB consensus_state = 3;
 
   // Debug state for the tablet.
   optional State state = 4 [ default = UNKNOWN ];
@@ -211,10 +211,10 @@ message ReportedTabletPB {
   optional tablet.TabletStatePB state = 2 [ default = UNKNOWN ];
   optional tablet.TabletDataState tablet_data_state = 6 [ default = TABLET_DATA_UNKNOWN ];
 
-  // The latest _committed_ consensus state.
+  // The latest consensus state.
   // This will be missing if the tablet is not in a RUNNING state
   // (i.e. if it is BOOTSTRAPPING).
-  optional consensus.ConsensusStatePB committed_consensus_state = 3;
+  optional consensus.ConsensusStatePB consensus_state = 3;
 
   optional AppStatusPB error = 4;
   optional uint32 schema_version = 5;

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/master/sys_catalog.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/sys_catalog.cc b/src/kudu/master/sys_catalog.cc
index 8820580..3aa736c 100644
--- a/src/kudu/master/sys_catalog.cc
+++ b/src/kudu/master/sys_catalog.cc
@@ -63,7 +63,7 @@ DEFINE_double(sys_catalog_fail_during_write, 0.0,
               "Fraction of the time when system table writes will fail");
 TAG_FLAG(sys_catalog_fail_during_write, hidden);
 
-using kudu::consensus::CONSENSUS_CONFIG_COMMITTED;
+using kudu::consensus::COMMITTED_CONFIG;
 using kudu::consensus::ConsensusMetadata;
 using kudu::consensus::ConsensusStatePB;
 using kudu::consensus::RaftConfigPB;
@@ -131,9 +131,9 @@ Status SysCatalogTable::Load(FsManager *fs_manager) {
     RETURN_NOT_OK_PREPEND(ConsensusMetadata::Load(fs_manager, tablet_id,
                                                   fs_manager->uuid(), &cmeta),
                           "Unable to load consensus metadata for tablet " + tablet_id);
-    ConsensusStatePB cstate = cmeta->ToConsensusStatePB(CONSENSUS_CONFIG_COMMITTED);
-    RETURN_NOT_OK(consensus::VerifyConsensusState(
-        cstate, consensus::COMMITTED_QUORUM));
+    ConsensusStatePB cstate = cmeta->ToConsensusStatePB();
+    RETURN_NOT_OK(consensus::VerifyRaftConfig(cstate.committed_config(), COMMITTED_CONFIG));
+    CHECK(!cstate.has_pending_config());
 
     // Make sure the set of masters passed in at start time matches the set in
     // the on-disk cmeta.
@@ -142,7 +142,7 @@ Status SysCatalogTable::Load(FsManager *fs_manager) {
       peer_addrs_from_opts.insert(hp.ToString());
     }
     set<string> peer_addrs_from_disk;
-    for (const auto& p : cstate.config().peers()) {
+    for (const auto& p : cstate.committed_config().peers()) {
       HostPort hp;
       RETURN_NOT_OK(HostPortFromPB(p.last_known_addr(), &hp));
       peer_addrs_from_disk.insert(hp.ToString());
@@ -246,7 +246,7 @@ Status SysCatalogTable::CreateDistributedConfig(const MasterOptions& options,
     }
   }
 
-  RETURN_NOT_OK(consensus::VerifyRaftConfig(resolved_config, consensus::COMMITTED_QUORUM));
+  RETURN_NOT_OK(consensus::VerifyRaftConfig(resolved_config, consensus::COMMITTED_CONFIG));
   VLOG(1) << "Distributed Raft configuration: " << SecureShortDebugString(resolved_config);
 
   *committed_config = resolved_config;
@@ -262,7 +262,7 @@ void SysCatalogTable::SysCatalogStateChanged(const string& tablet_id, const stri
                              << tablet_id << ". Reason: " << reason;
     return;
   }
-  consensus::ConsensusStatePB cstate = consensus->ConsensusState(CONSENSUS_CONFIG_COMMITTED);
+  consensus::ConsensusStatePB cstate = consensus->ConsensusState();
   LOG_WITH_PREFIX(INFO) << "SysCatalogTable state changed. Reason: " << reason << ". "
                         << "Latest consensus state: " << SecureShortDebugString(cstate);
   RaftPeerPB::Role new_role = GetConsensusRole(tablet_replica_->permanent_uuid(), cstate);

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/tools/kudu-admin-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/kudu-admin-test.cc b/src/kudu/tools/kudu-admin-test.cc
index 497857b..e1c8841 100644
--- a/src/kudu/tools/kudu-admin-test.cc
+++ b/src/kudu/tools/kudu-admin-test.cc
@@ -22,6 +22,7 @@
 
 #include "kudu/client/client.h"
 #include "kudu/consensus/opid.pb.h"
+#include "kudu/consensus/quorum_util.h"
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/integration-tests/cluster_itest_util.h"
@@ -39,10 +40,9 @@ using client::KuduClientBuilder;
 using client::KuduSchema;
 using client::KuduTableCreator;
 using client::sp::shared_ptr;
-using consensus::CONSENSUS_CONFIG_COMMITTED;
+using consensus::COMMITTED_OPID;
 using consensus::ConsensusStatePB;
 using consensus::OpId;
-using consensus::COMMITTED_OPID;
 using itest::GetConsensusState;
 using itest::TabletServerMap;
 using itest::TServerDetails;
@@ -955,10 +955,10 @@ Status GetTermFromConsensus(const vector<TServerDetails*>& tservers,
   ConsensusStatePB cstate;
   for (auto& ts : tservers) {
     RETURN_NOT_OK(
-        GetConsensusState(ts, tablet_id,
-                          CONSENSUS_CONFIG_COMMITTED,
-                          MonoDelta::FromSeconds(10), &cstate));
-    if (cstate.has_leader_uuid() && cstate.has_current_term()) {
+        GetConsensusState(ts, tablet_id, MonoDelta::FromSeconds(10), &cstate));
+    if (cstate.has_leader_uuid() &&
+        IsRaftConfigMember(cstate.leader_uuid(), cstate.committed_config()) &&
+        cstate.has_current_term()) {
       *current_term = cstate.current_term();
       return Status::OK();
     }

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/tserver/tablet_copy.proto
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy.proto b/src/kudu/tserver/tablet_copy.proto
index 13d4e60..6260bd5 100644
--- a/src/kudu/tserver/tablet_copy.proto
+++ b/src/kudu/tserver/tablet_copy.proto
@@ -114,9 +114,9 @@ message BeginTabletCopySessionResponsePB {
   // Each WAL segment is keyed by its sequence number.
   repeated uint64 wal_segment_seqnos = 4;
 
-  // A snapshot of the committed Consensus state at the time that the
+  // A snapshot of the consensus state at the time that the
   // tablet copy session was started.
-  required consensus.ConsensusStatePB initial_committed_cstate = 5;
+  required consensus.ConsensusStatePB initial_cstate = 5;
 
   // permanent_uuid of the responding peer.
   optional bytes responder_uuid = 6;

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/tserver/tablet_copy_client-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_client-test.cc b/src/kudu/tserver/tablet_copy_client-test.cc
index 7c70688..b14de82 100644
--- a/src/kudu/tserver/tablet_copy_client-test.cc
+++ b/src/kudu/tserver/tablet_copy_client-test.cc
@@ -51,8 +51,7 @@ class TabletCopyClientTest : public TabletCopyTest {
     client_.reset(new TabletCopyClient(GetTabletId(),
                                        fs_manager_.get(),
                                        messenger_));
-    ASSERT_OK(GetRaftConfigLeader(tablet_replica_->consensus()
-        ->ConsensusState(consensus::CONSENSUS_CONFIG_COMMITTED), &leader_));
+    ASSERT_OK(GetRaftConfigLeader(tablet_replica_->consensus()->ConsensusState(), &leader_));
 
     HostPort host_port;
     ASSERT_OK(HostPortFromPB(leader_.last_known_addr(), &host_port));

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/tserver/tablet_copy_client.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_client.cc b/src/kudu/tserver/tablet_copy_client.cc
index 21c5630..b5450ec 100644
--- a/src/kudu/tserver/tablet_copy_client.cc
+++ b/src/kudu/tserver/tablet_copy_client.cc
@@ -213,7 +213,7 @@ Status TabletCopyClient::Start(const HostPort& copy_source_addr,
   superblock_->set_tablet_data_state(tablet::TABLET_DATA_COPYING);
 
   wal_seqnos_.assign(resp.wal_segment_seqnos().begin(), resp.wal_segment_seqnos().end());
-  remote_committed_cstate_.reset(resp.release_initial_committed_cstate());
+  remote_cstate_.reset(resp.release_initial_cstate());
 
   Schema schema;
   RETURN_NOT_OK_PREPEND(SchemaFromPB(superblock_->schema(), &schema),
@@ -226,13 +226,13 @@ Status TabletCopyClient::Start(const HostPort& copy_source_addr,
     // source peer, even after passing the term check from the caller in
     // SetTabletToReplace().
     int64_t last_logged_term = meta_->tombstone_last_logged_opid().term();
-    if (last_logged_term > remote_committed_cstate_->current_term()) {
+    if (last_logged_term > remote_cstate_->current_term()) {
       return Status::InvalidArgument(
           Substitute("Tablet $0: source peer has term $1 but "
                      "tombstoned replica has last-logged opid with higher term $2. "
                      "Refusing tablet copy from source peer $3",
                      tablet_id_,
-                     remote_committed_cstate_->current_term(),
+                     remote_cstate_->current_term(),
                      last_logged_term,
                      copy_peer_uuid));
     }
@@ -507,14 +507,14 @@ Status TabletCopyClient::WriteConsensusMetadata() {
   if (!cmeta_) {
     unique_ptr<ConsensusMetadata> cmeta;
     return ConsensusMetadata::Create(fs_manager_, tablet_id_, fs_manager_->uuid(),
-                                     remote_committed_cstate_->config(),
-                                     remote_committed_cstate_->current_term(),
+                                     remote_cstate_->committed_config(),
+                                     remote_cstate_->current_term(),
                                      &cmeta);
   }
 
   // Otherwise, update the consensus metadata to reflect the config and term
   // sent by the tablet copy source.
-  cmeta_->MergeCommittedConsensusStatePB(*remote_committed_cstate_);
+  cmeta_->MergeCommittedConsensusStatePB(*remote_cstate_);
   RETURN_NOT_OK(cmeta_->Flush());
 
   if (FLAGS_tablet_copy_save_downloaded_metadata) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/tserver/tablet_copy_client.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_client.h b/src/kudu/tserver/tablet_copy_client.h
index 259196f..85a2149 100644
--- a/src/kudu/tserver/tablet_copy_client.h
+++ b/src/kudu/tserver/tablet_copy_client.h
@@ -215,7 +215,7 @@ class TabletCopyClient {
   uint64_t session_idle_timeout_millis_;
   std::unique_ptr<tablet::TabletSuperBlockPB> remote_superblock_;
   std::unique_ptr<tablet::TabletSuperBlockPB> superblock_;
-  std::unique_ptr<consensus::ConsensusStatePB> remote_committed_cstate_;
+  std::unique_ptr<consensus::ConsensusStatePB> remote_cstate_;
   std::vector<uint64_t> wal_seqnos_;
   int64_t start_time_micros_;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/tserver/tablet_copy_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_service.cc b/src/kudu/tserver/tablet_copy_service.cc
index cc5016b..ef11ecc 100644
--- a/src/kudu/tserver/tablet_copy_service.cc
+++ b/src/kudu/tserver/tablet_copy_service.cc
@@ -153,7 +153,7 @@ void TabletCopyServiceImpl::BeginTabletCopySession(
   resp->set_session_id(session_id);
   resp->set_session_idle_timeout_millis(FLAGS_tablet_copy_idle_timeout_ms);
   resp->mutable_superblock()->CopyFrom(session->tablet_superblock());
-  resp->mutable_initial_committed_cstate()->CopyFrom(session->initial_committed_cstate());
+  resp->mutable_initial_cstate()->CopyFrom(session->initial_cstate());
 
   for (const scoped_refptr<log::ReadableLogSegment>& segment : session->log_segments()) {
     resp->add_wal_segment_seqnos(segment->header().sequence_number());

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/tserver/tablet_copy_source_session.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_source_session.cc b/src/kudu/tserver/tablet_copy_source_session.cc
index 6021e85..2926792 100644
--- a/src/kudu/tserver/tablet_copy_source_session.cc
+++ b/src/kudu/tserver/tablet_copy_source_session.cc
@@ -128,7 +128,7 @@ Status TabletCopySourceSession::Init() {
         "Consensus is not available. Tablet state: $1 ($2)",
         tablet_id, tablet::TabletStatePB_Name(tablet_state), tablet_state));
   }
-  initial_committed_cstate_ = consensus->ConsensusState(consensus::CONSENSUS_CONFIG_COMMITTED);
+  initial_cstate_ = consensus->ConsensusState();
 
   // Re-anchor on the highest OpId that was in the log right before we
   // snapshotted the log segments. This helps ensure that we don't end up in a

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/tserver/tablet_copy_source_session.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_source_session.h b/src/kudu/tserver/tablet_copy_source_session.h
index 2615869..adeb133 100644
--- a/src/kudu/tserver/tablet_copy_source_session.h
+++ b/src/kudu/tserver/tablet_copy_source_session.h
@@ -128,9 +128,9 @@ class TabletCopySourceSession : public RefCountedThreadSafe<TabletCopySourceSess
     return tablet_superblock_;
   }
 
-  const consensus::ConsensusStatePB& initial_committed_cstate() const {
+  const consensus::ConsensusStatePB& initial_cstate() const {
     DCHECK(initted_);
-    return initial_committed_cstate_;
+    return initial_cstate_;
   }
 
   const log::SegmentSequence& log_segments() const {
@@ -186,7 +186,7 @@ class TabletCopySourceSession : public RefCountedThreadSafe<TabletCopySourceSess
 
   tablet::TabletSuperBlockPB tablet_superblock_;
 
-  consensus::ConsensusStatePB initial_committed_cstate_;
+  consensus::ConsensusStatePB initial_cstate_;
 
   // The sequence of log segments that will be sent in the course of this
   // session.

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/tserver/tablet_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_service.cc b/src/kudu/tserver/tablet_service.cc
index 4f3eab8..8ae2e4a 100644
--- a/src/kudu/tserver/tablet_service.cc
+++ b/src/kudu/tserver/tablet_service.cc
@@ -101,10 +101,7 @@ DECLARE_int32(tablet_history_max_age_sec);
 using google::protobuf::RepeatedPtrField;
 using kudu::consensus::ChangeConfigRequestPB;
 using kudu::consensus::ChangeConfigResponsePB;
-using kudu::consensus::CONSENSUS_CONFIG_ACTIVE;
-using kudu::consensus::CONSENSUS_CONFIG_COMMITTED;
 using kudu::consensus::Consensus;
-using kudu::consensus::ConsensusConfigType;
 using kudu::consensus::ConsensusRequestPB;
 using kudu::consensus::ConsensusResponsePB;
 using kudu::consensus::GetLastOpIdRequestPB;
@@ -1053,15 +1050,7 @@ void ConsensusServiceImpl::GetConsensusState(const consensus::GetConsensusStateR
 
   scoped_refptr<Consensus> consensus;
   if (!GetConsensusOrRespond(replica, resp, context, &consensus)) return;
-  ConsensusConfigType type = req->type();
-  if (PREDICT_FALSE(type != CONSENSUS_CONFIG_ACTIVE && type != CONSENSUS_CONFIG_COMMITTED)) {
-    HandleUnknownError(
-        Status::InvalidArgument(Substitute("Unsupported ConsensusConfigType $0 ($1)",
-                                           ConsensusConfigType_Name(type), type)),
-        resp, context);
-    return;
-  }
-  *resp->mutable_cstate() = consensus->ConsensusState(req->type());
+  *resp->mutable_cstate() = consensus->ConsensusState();
   context->RespondSuccess();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/src/kudu/tserver/ts_tablet_manager-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/ts_tablet_manager-test.cc b/src/kudu/tserver/ts_tablet_manager-test.cc
index 7b290bf..570d8dc 100644
--- a/src/kudu/tserver/ts_tablet_manager-test.cc
+++ b/src/kudu/tserver/ts_tablet_manager-test.cc
@@ -156,22 +156,22 @@ static void AssertReportHasUpdatedTablet(const TabletReportPB& report,
                                          const string& tablet_id) {
   ASSERT_GE(report.updated_tablets_size(), 0);
   bool found_tablet = false;
-  for (ReportedTabletPB reported_tablet : report.updated_tablets()) {
+  for (const ReportedTabletPB& reported_tablet : report.updated_tablets()) {
     if (reported_tablet.tablet_id() == tablet_id) {
       found_tablet = true;
-      ASSERT_TRUE(reported_tablet.has_committed_consensus_state());
-      ASSERT_TRUE(reported_tablet.committed_consensus_state().has_current_term())
+      ASSERT_TRUE(reported_tablet.has_consensus_state());
+      ASSERT_TRUE(reported_tablet.consensus_state().has_current_term())
           << SecureShortDebugString(reported_tablet);
-      ASSERT_TRUE(reported_tablet.committed_consensus_state().has_leader_uuid())
+      ASSERT_TRUE(reported_tablet.consensus_state().has_leader_uuid())
           << SecureShortDebugString(reported_tablet);
-      ASSERT_TRUE(reported_tablet.committed_consensus_state().has_config());
-      const RaftConfigPB& committed_config = reported_tablet.committed_consensus_state().config();
+      ASSERT_TRUE(reported_tablet.consensus_state().has_committed_config());
+      const RaftConfigPB& committed_config = reported_tablet.consensus_state().committed_config();
       ASSERT_EQ(kInvalidOpIdIndex, committed_config.opid_index());
       ASSERT_EQ(1, committed_config.peers_size());
       ASSERT_TRUE(committed_config.peers(0).has_permanent_uuid())
           << SecureShortDebugString(reported_tablet);
       ASSERT_EQ(committed_config.peers(0).permanent_uuid(),
-                reported_tablet.committed_consensus_state().leader_uuid())
+                reported_tablet.consensus_state().leader_uuid())
           << SecureShortDebugString(reported_tablet);
     }
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/1d74c73e/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 45c7016..861cd8a 100644
--- a/src/kudu/tserver/ts_tablet_manager.cc
+++ b/src/kudu/tserver/ts_tablet_manager.cc
@@ -948,8 +948,7 @@ void TSTabletManager::CreateReportedTabletPB(const string& tablet_id,
   // We cannot get consensus state information unless the TabletReplica is running.
   scoped_refptr<consensus::Consensus> consensus = replica->shared_consensus();
   if (consensus) {
-    *reported_tablet->mutable_committed_consensus_state() =
-        consensus->ConsensusState(consensus::CONSENSUS_CONFIG_COMMITTED);
+    *reported_tablet->mutable_consensus_state() = consensus->ConsensusState();
   }
 }