Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id B1705200D4B for ; Mon, 27 Nov 2017 20:43:33 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id B0125160C13; Mon, 27 Nov 2017 19:43:33 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id A8B26160BFA for ; Mon, 27 Nov 2017 20:43:32 +0100 (CET) Received: (qmail 12918 invoked by uid 500); 27 Nov 2017 19:43:31 -0000 Mailing-List: contact commits-help@kudu.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@kudu.apache.org Delivered-To: mailing list commits@kudu.apache.org Received: (qmail 12908 invoked by uid 99); 27 Nov 2017 19:43:31 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 27 Nov 2017 19:43:31 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 17D2EDFE1A; Mon, 27 Nov 2017 19:43:30 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: alexey@apache.org To: commits@kudu.apache.org Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: kudu git commit: consensus: Rename ChangeConfigType enum constants Date: Mon, 27 Nov 2017 19:43:30 +0000 (UTC) archived-at: Mon, 27 Nov 2017 19:43:33 -0000 Repository: kudu Updated Branches: refs/heads/master 79a255bbf -> fc143c01d consensus: Rename ChangeConfigType enum constants Since we now have attributes in the RaftConfigPeerPB message, it would be more useful to allow a peer to be modified such that only its attributes change, but not its member type. In order to provide meaningful semantics, that means we should change the constant name for the "modify peer" operation as well. This patch changes CHANGE_REPLICA_TYPE to MODIFY_PEER. While we're in there, let's also rename the other constants to more accurately reflect the terminology we tend to use in this part of the code. There are no functional or wire-protocol changes in this patch. The functional changes to go along with this patch will come in a follow-up patch. Change-Id: I77b838f1b7ddcf176d1a5146e04fa1fddc1042fa Reviewed-on: http://gerrit.cloudera.org:8080/8643 Tested-by: Kudu Jenkins Reviewed-by: Mike Percy Reviewed-by: Alexey Serbin Project: http://git-wip-us.apache.org/repos/asf/kudu/repo Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/fc143c01 Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/fc143c01 Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/fc143c01 Branch: refs/heads/master Commit: fc143c01dd9091ef212f9b572752875f11524567 Parents: 79a255b Author: Mike Percy Authored: Sun Nov 26 21:17:29 2017 -0800 Committer: Alexey Serbin Committed: Mon Nov 27 19:41:44 2017 +0000 ---------------------------------------------------------------------- src/kudu/consensus/consensus.proto | 8 ++++---- src/kudu/consensus/raft_consensus.cc | 14 +++++++------- src/kudu/integration-tests/cluster_itest_util.cc | 6 +++--- src/kudu/integration-tests/cluster_itest_util.h | 4 ++-- .../raft_consensus_nonvoter-itest.cc | 4 ++-- src/kudu/master/catalog_manager.cc | 8 ++++---- src/kudu/tools/tool_action_tablet.cc | 18 +++++++++--------- 7 files changed, 31 insertions(+), 31 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/kudu/blob/fc143c01/src/kudu/consensus/consensus.proto ---------------------------------------------------------------------- diff --git a/src/kudu/consensus/consensus.proto b/src/kudu/consensus/consensus.proto index c8abd5d..6ff4186 100644 --- a/src/kudu/consensus/consensus.proto +++ b/src/kudu/consensus/consensus.proto @@ -118,9 +118,9 @@ message ChangeConfigRecordPB { enum ChangeConfigType { UNKNOWN_CHANGE = 0; - ADD_SERVER = 1; - REMOVE_SERVER = 2; - CHANGE_REPLICA_TYPE = 3; + ADD_PEER = 1; + REMOVE_PEER = 2; + MODIFY_PEER = 3; } // A configuration change request for the tablet with 'tablet_id'. @@ -139,7 +139,7 @@ message ChangeConfigRequestPB { optional ChangeConfigType type = 2; // The peer to add or remove. - // When 'type' == ADD_SERVER, both the permanent_uuid and last_known_addr + // When 'type' == ADD_PEER, both the permanent_uuid and last_known_addr // fields must be set. Otherwise, only the permanent_uuid field is required. optional RaftPeerPB server = 3; http://git-wip-us.apache.org/repos/asf/kudu/blob/fc143c01/src/kudu/consensus/raft_consensus.cc ---------------------------------------------------------------------- diff --git a/src/kudu/consensus/raft_consensus.cc b/src/kudu/consensus/raft_consensus.cc index f1e0650..cdfb0fb 100644 --- a/src/kudu/consensus/raft_consensus.cc +++ b/src/kudu/consensus/raft_consensus.cc @@ -802,7 +802,7 @@ void RaftConsensus::TryRemoveFollowerTask(const string& uuid, ChangeConfigRequestPB req; req.set_tablet_id(options_.tablet_id); req.mutable_server()->set_permanent_uuid(uuid); - req.set_type(REMOVE_SERVER); + req.set_type(REMOVE_PEER); req.set_cas_config_opid_index(committed_config.opid_index()); LOG(INFO) << LogPrefixThreadSafe() << "Attempting to remove follower " << uuid << " from the Raft config. Reason: " << reason; @@ -849,7 +849,7 @@ void RaftConsensus::TryPromoteNonVoterTask(const std::string& peer_uuid, ChangeConfigRequestPB req; req.set_tablet_id(options_.tablet_id); - req.set_type(CHANGE_REPLICA_TYPE); + req.set_type(MODIFY_PEER); req.mutable_server()->set_permanent_uuid(peer_uuid); req.mutable_server()->set_member_type(RaftPeerPB::VOTER); req.mutable_server()->mutable_attrs()->set_promote(false); @@ -1682,7 +1682,7 @@ Status RaftConsensus::ChangeConfig(const ChangeConfigRequestPB& req, new_config.clear_opid_index(); const string& server_uuid = server.permanent_uuid(); switch (type) { - case ADD_SERVER: + case ADD_PEER: // Ensure the server we are adding is not already a member of the configuration. if (IsRaftConfigMember(server_uuid, committed_config)) { return Status::InvalidArgument( @@ -1700,7 +1700,7 @@ Status RaftConsensus::ChangeConfig(const ChangeConfigRequestPB& req, *new_config.add_peers() = server; break; - case REMOVE_SERVER: + case REMOVE_PEER: if (server_uuid == peer_uuid()) { return Status::InvalidArgument( Substitute("Cannot remove peer $0 from the config because it is the leader. " @@ -1716,14 +1716,14 @@ Status RaftConsensus::ChangeConfig(const ChangeConfigRequestPB& req, } break; - case CHANGE_REPLICA_TYPE: + case MODIFY_PEER: if (server.member_type() == RaftPeerPB::UNKNOWN_MEMBER_TYPE) { return Status::InvalidArgument("Cannot change replica type to UNKNOWN_MEMBER_TYPE"); } if (server_uuid == peer_uuid()) { return Status::InvalidArgument( - Substitute("Cannot change the replica type of peer $0 because it is the leader. " - "Force another leader to be elected to modify the type of this replica. " + Substitute("Cannot modify peer $0 because it is the leader. " + "Force another leader to be elected to modify this replica. " "Consensus state: $1", server_uuid, SecureShortDebugString(cmeta_->ToConsensusStatePB()))); http://git-wip-us.apache.org/repos/asf/kudu/blob/fc143c01/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 898394f..ad3bd8c 100644 --- a/src/kudu/integration-tests/cluster_itest_util.cc +++ b/src/kudu/integration-tests/cluster_itest_util.cc @@ -696,7 +696,7 @@ Status AddServer(const TServerDetails* leader, ChangeConfigRequestPB req; req.set_dest_uuid(leader->uuid()); req.set_tablet_id(tablet_id); - req.set_type(consensus::ADD_SERVER); + req.set_type(consensus::ADD_PEER); RaftPeerPB* peer = req.mutable_server(); peer->set_permanent_uuid(replica_to_add->uuid()); peer->set_member_type(member_type); @@ -728,7 +728,7 @@ Status RemoveServer(const TServerDetails* leader, ChangeConfigRequestPB req; req.set_dest_uuid(leader->uuid()); req.set_tablet_id(tablet_id); - req.set_type(consensus::REMOVE_SERVER); + req.set_type(consensus::REMOVE_PEER); if (cas_config_index) { req.set_cas_config_opid_index(*cas_config_index); } @@ -758,7 +758,7 @@ Status ChangeReplicaType(const TServerDetails* leader, ChangeConfigRequestPB req; req.set_dest_uuid(leader->uuid()); req.set_tablet_id(tablet_id); - req.set_type(consensus::CHANGE_REPLICA_TYPE); + req.set_type(consensus::MODIFY_PEER); if (cas_config_index) { req.set_cas_config_opid_index(*cas_config_index); } http://git-wip-us.apache.org/repos/asf/kudu/blob/fc143c01/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 8dcc5d7..23481a5 100644 --- a/src/kudu/integration-tests/cluster_itest_util.h +++ b/src/kudu/integration-tests/cluster_itest_util.h @@ -269,7 +269,7 @@ Status WriteSimpleTestRow(const TServerDetails* replica, const std::string& string_val, const MonoDelta& timeout); -// Run a ConfigChange to ADD_SERVER on 'replica_to_add'. +// Run a ConfigChange to ADD_PEER on 'replica_to_add'. // The RPC request is sent to 'leader'. Status AddServer(const TServerDetails* leader, const std::string& tablet_id, @@ -280,7 +280,7 @@ Status AddServer(const TServerDetails* leader, const boost::optional& cas_config_index = boost::none, tserver::TabletServerErrorPB::Code* error_code = nullptr); -// Run a ConfigChange to REMOVE_SERVER on 'replica_to_remove'. +// Run a ConfigChange to REMOVE_PEER on 'replica_to_remove'. // The RPC request is sent to 'leader'. Status RemoveServer(const TServerDetails* leader, const std::string& tablet_id, http://git-wip-us.apache.org/repos/asf/kudu/blob/fc143c01/src/kudu/integration-tests/raft_consensus_nonvoter-itest.cc ---------------------------------------------------------------------- diff --git a/src/kudu/integration-tests/raft_consensus_nonvoter-itest.cc b/src/kudu/integration-tests/raft_consensus_nonvoter-itest.cc index 45dcc2b..e8882c3 100644 --- a/src/kudu/integration-tests/raft_consensus_nonvoter-itest.cc +++ b/src/kudu/integration-tests/raft_consensus_nonvoter-itest.cc @@ -678,7 +678,7 @@ TEST_F(RaftConsensusNonVoterITest, PromoteAndDemote) { const auto s_demote_str = s_demote.ToString(); ASSERT_TRUE(s_demote.IsInvalidArgument()) << s_demote_str; ASSERT_STR_MATCHES(s_demote_str, - "Cannot change the replica type of peer .* because it is the leader"); + "Cannot modify peer .* because it is the leader"); // It should be impossible to promote a leader replica since it's // already a voter. @@ -687,7 +687,7 @@ TEST_F(RaftConsensusNonVoterITest, PromoteAndDemote) { const auto s_promote_str = s_promote.ToString(); ASSERT_TRUE(s_promote.IsInvalidArgument()) << s_promote_str; ASSERT_STR_MATCHES(s_promote_str, - "Cannot change the replica type of peer .* because it is the leader"); + "Cannot modify peer .* because it is the leader"); } // Demote the replica back. http://git-wip-us.apache.org/repos/asf/kudu/blob/fc143c01/src/kudu/master/catalog_manager.cc ---------------------------------------------------------------------- diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc index 5131bd3..2d05290 100644 --- a/src/kudu/master/catalog_manager.cc +++ b/src/kudu/master/catalog_manager.cc @@ -3106,7 +3106,7 @@ AsyncAddReplicaTask::AsyncAddReplicaTask(Master* master, RaftPeerPB::MemberType member_type, ThreadSafeRandom* rng) : AsyncChangeConfigTask(master, std::move(tablet), std::move(cstate), - consensus::ADD_SERVER), + consensus::ADD_PEER), member_type_(member_type), rng_(rng) { } @@ -3147,7 +3147,7 @@ bool AsyncAddReplicaTask::SendRequest(int attempt) { consensus::ChangeConfigRequestPB req; req.set_dest_uuid(target_ts_desc_->permanent_uuid()); req.set_tablet_id(tablet_->id()); - req.set_type(consensus::ADD_SERVER); + req.set_type(consensus::ADD_PEER); req.set_cas_config_opid_index(cstate_.committed_config().opid_index()); RaftPeerPB* peer = req.mutable_server(); peer->set_permanent_uuid(replacement_replica->permanent_uuid()); @@ -3184,7 +3184,7 @@ AsyncEvictReplicaTask::AsyncEvictReplicaTask(Master* master, ConsensusStatePB cstate, string peer_uuid_to_evict) : AsyncChangeConfigTask(master, std::move(tablet), std::move(cstate), - consensus::REMOVE_SERVER), + consensus::REMOVE_PEER), peer_uuid_to_evict_(std::move(peer_uuid_to_evict)) { } @@ -3205,7 +3205,7 @@ bool AsyncEvictReplicaTask::SendRequest(int attempt) { consensus::ChangeConfigRequestPB req; req.set_dest_uuid(target_ts_desc_->permanent_uuid()); req.set_tablet_id(tablet_->id()); - req.set_type(consensus::REMOVE_SERVER); + req.set_type(consensus::REMOVE_PEER); req.set_cas_config_opid_index(cstate_.committed_config().opid_index()); RaftPeerPB* peer = req.mutable_server(); peer->set_permanent_uuid(peer_uuid_to_evict_); http://git-wip-us.apache.org/repos/asf/kudu/blob/fc143c01/src/kudu/tools/tool_action_tablet.cc ---------------------------------------------------------------------- diff --git a/src/kudu/tools/tool_action_tablet.cc b/src/kudu/tools/tool_action_tablet.cc index 8faa214..1a27ecc 100644 --- a/src/kudu/tools/tool_action_tablet.cc +++ b/src/kudu/tools/tool_action_tablet.cc @@ -163,10 +163,10 @@ Status DoChangeConfig(const vector& master_addresses, const string& replica_uuid, const boost::optional& member_type, ChangeConfigType cc_type) { - if (cc_type == consensus::REMOVE_SERVER && member_type) { + if (cc_type == consensus::REMOVE_PEER && member_type) { return Status::InvalidArgument("cannot supply Raft member type when removing a server"); } - if ((cc_type == consensus::ADD_SERVER || cc_type == consensus::CHANGE_REPLICA_TYPE) && + if ((cc_type == consensus::ADD_PEER || cc_type == consensus::MODIFY_PEER) && !member_type) { return Status::InvalidArgument( "must specify member type when adding a server or changing member type"); @@ -184,7 +184,7 @@ Status DoChangeConfig(const vector& master_addresses, .Build(&client)); // When adding a new server, we need to provide the server's RPC address. - if (cc_type == consensus::ADD_SERVER) { + if (cc_type == consensus::ADD_PEER) { HostPort hp; RETURN_NOT_OK(GetRpcAddressForTS(client, replica_uuid, &hp)); RETURN_NOT_OK(HostPortToPB(hp, peer_pb.mutable_last_known_addr())); @@ -220,7 +220,7 @@ Status ChangeConfig(const RunnerContext& context, ChangeConfigType cc_type) { const string& tablet_id = FindOrDie(context.required_args, kTabletIdArg); const string& replica_uuid = FindOrDie(context.required_args, kTsUuidArg); boost::optional member_type; - if (cc_type == consensus::ADD_SERVER || cc_type == consensus::CHANGE_REPLICA_TYPE) { + if (cc_type == consensus::ADD_PEER || cc_type == consensus::MODIFY_PEER) { const string& replica_type = FindOrDie(context.required_args, kReplicaTypeArg); string uppercase_peer_type; ToUpperCase(replica_type, &uppercase_peer_type); @@ -235,15 +235,15 @@ Status ChangeConfig(const RunnerContext& context, ChangeConfigType cc_type) { } Status AddReplica(const RunnerContext& context) { - return ChangeConfig(context, consensus::ADD_SERVER); + return ChangeConfig(context, consensus::ADD_PEER); } Status ChangeReplicaType(const RunnerContext& context) { - return ChangeConfig(context, consensus::CHANGE_REPLICA_TYPE); + return ChangeConfig(context, consensus::MODIFY_PEER); } Status RemoveReplica(const RunnerContext& context) { - return ChangeConfig(context, consensus::REMOVE_SERVER); + return ChangeConfig(context, consensus::REMOVE_PEER); } Status DoLeaderStepDown(const client::sp::shared_ptr& client, const string& tablet_id, @@ -389,7 +389,7 @@ Status MoveReplica(const RunnerContext &context) { RETURN_NOT_OK_PREPEND(DoKsckForTablet(master_addresses, tablet_id), "ksck pre-move health check failed"); RETURN_NOT_OK(DoChangeConfig(master_addresses, tablet_id, to_ts_uuid, - RaftPeerPB::VOTER, consensus::ADD_SERVER)); + RaftPeerPB::VOTER, consensus::ADD_PEER)); // Wait until the tablet copy completes and the tablet returns to perfect health. MonoDelta copy_timeout = MonoDelta::FromSeconds(FLAGS_move_copy_timeout_sec); @@ -410,7 +410,7 @@ Status MoveReplica(const RunnerContext &context) { "failed changing leadership from the replica to be removed"); } return DoChangeConfig(master_addresses, tablet_id, from_ts_uuid, - boost::none, consensus::REMOVE_SERVER); + boost::none, consensus::REMOVE_PEER); } } // anonymous namespace