kudu git commit: [raft_consensus-itest] fix TestElectionMetrics flake

2018-07-11 Thread alexey
Repository: kudu
Updated Branches:
  refs/heads/master 03a3c6812 -> 077fe5762


[raft_consensus-itest] fix TestElectionMetrics flake

This patch fixes flakiness in the RaftConsensusITest.TestElectionMetrics
scenario.  The original TestElectionMetrics scenario was split into
two parts TestElectionMetricsPart[1,2] to separate tests that assume
no leader election happen and depend on the leader failure detection
mechanism, correspondingly.

Prior to this patch, the TestElectionMetrics scenario could fail if a
leader election happened inadvertently.

Before (1 out of 12 failed):
  http://dist-test.cloudera.org/job?job_id=aserbin.1530681412.7635

After  (not a single failure in 1K run):
  http://dist-test.cloudera.org/job?job_id=aserbin.1530918429.120803

Change-Id: I073c9989a6d5d5dc1eb104120a89d38cfce2ac6e
Reviewed-on: http://gerrit.cloudera.org:8080/10887
Tested-by: Alexey Serbin 
Reviewed-by: Will Berkeley 
Reviewed-by: Attila Bukor 


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

Branch: refs/heads/master
Commit: 077fe576288bfaa1aadc42596b3594c65ecf16e2
Parents: 03a3c68
Author: Alexey Serbin 
Authored: Thu Jul 5 13:39:06 2018 -0700
Committer: Alexey Serbin 
Committed: Wed Jul 11 19:42:22 2018 +

--
 src/kudu/consensus/raft_consensus.cc|  10 ++
 .../integration-tests/raft_consensus-itest.cc   | 136 +++
 2 files changed, 120 insertions(+), 26 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kudu/blob/077fe576/src/kudu/consensus/raft_consensus.cc
--
diff --git a/src/kudu/consensus/raft_consensus.cc 
b/src/kudu/consensus/raft_consensus.cc
index c70abd9..f81637e 100644
--- a/src/kudu/consensus/raft_consensus.cc
+++ b/src/kudu/consensus/raft_consensus.cc
@@ -1320,6 +1320,16 @@ Status RaftConsensus::UpdateReplica(const 
ConsensusRequestPB* request,
 
 last_leader_communication_time_micros_ = GetMonoTimeMicros();
 
+// Reset the 'failed_elections_since_stable_leader' metric now that we've
+// accepted an update from the established leader. This is done in addition
+// to the reset of the value in SetLeaderUuidUnlocked() because there is
+// a potential race between resetting the failed elections count in
+// SetLeaderUuidUnlocked() and incrementing after a failed election
+// if another replica was elected leader in an election concurrent with
+// the one called by this replica.
+failed_elections_since_stable_leader_ = 0;
+
num_failed_elections_metric_->set_value(failed_elections_since_stable_leader_);
+
 // We update the lag metrics here in addition to after appending to the 
queue so the
 // metrics get updated even when the operation is rejected.
 
queue_->UpdateLastIndexAppendedToLeader(request->last_idx_appended_to_leader());

http://git-wip-us.apache.org/repos/asf/kudu/blob/077fe576/src/kudu/integration-tests/raft_consensus-itest.cc
--
diff --git a/src/kudu/integration-tests/raft_consensus-itest.cc 
b/src/kudu/integration-tests/raft_consensus-itest.cc
index d1425d0..f7be807 100644
--- a/src/kudu/integration-tests/raft_consensus-itest.cc
+++ b/src/kudu/integration-tests/raft_consensus-itest.cc
@@ -2670,61 +2670,145 @@ int64_t GetTimeSinceLastLeaderHeartbeat(const 
ExternalTabletServer* ets,
   return ret;
 }
 
-TEST_F(RaftConsensusITest, TestElectionMetrics) {
+// Test for election-related metrics with the leader failure detection 
disabled.
+// This avoids inadvertent leader elections that might race with the tests
+// of the metric's behavior.
+TEST_F(RaftConsensusITest, TestElectionMetricsFailureDetectionDisabled) {
+  constexpr auto kNumReplicas = 3;
+  constexpr auto kNumTservers = 3;
   const vector kTsFlags = {
 // Make leader elections faster so we can test
 // failed_elections_since_stable_leader faster.
 "--raft_heartbeat_interval_ms=100",
+
+// For stability reasons, this scenario uses the 'manual election' mode
+// and assumes no leader change before restarting tablet servers later on.
+"--enable_leader_failure_detection=false",
+  };
+  const vector kMasterFlags = {
+// Corresponding flag for the tserver's 
'--enable_leader_failure_detection'.
+"--catalog_manager_wait_for_new_tablets_to_elect_leader=false",
   };
+  const auto kTimeout = MonoDelta::FromSeconds(30);
 
-  FLAGS_num_replicas = 3;
-  FLAGS_num_tablet_servers = 3;
-  NO_FATALS(BuildAndStart(kTsFlags));
+  FLAGS_num_replicas = kNumReplicas;
+  FLAGS_num_tablet_servers = kNumTservers;
+  NO_FATALS(BuildAndStart(kTsFlags, kMasterFlags));
 
   

kudu git commit: [tools] sanity check for ScheduleReplicaMove()

2018-07-11 Thread alexey
Repository: kudu
Updated Branches:
  refs/heads/master cee17c03b -> 2692ac4ad


[tools] sanity check for ScheduleReplicaMove()

This change introduces a couple of improvements into the implementation
of the kudu::tools::ScheduleReplicaMove() function, used by the
kudu CLI tool's 'kudu tablet change_config move_replica' and
'kudu cluster rebalance' sub-commands:

  * use CAS semantics when changing tablet Raft configuration
  * avoid setting the REPLACE attribute if it's set already

The former protects against unexpected Raft configuration changes
in the middle of the replica movement process.  The latter helps in
situations when a configuration change initiated by a prior run of
the rebalancing tool failed at later stages -- with this change,
there is no need to reset the source replica's attributes separately
if such situation would not resolve on its own (e.g., when the REPLACE
attribute is set on a leader replica).

This changelist does not add corresponding tests since the existing
ConcurrentRebalancersTest.TwoConcurrentRebalancers scenario provides
enough coverage.  As a result of this change, the test scenario
became more stable when running with --stress_cpu_threads=16 flag.

before (24 out of 256 failed):
  http://dist-test.cloudera.org/job?job_id=aserbin.1531287387.25723

after (none of 256 failed):
  http://dist-test.cloudera.org/job?job_id=aserbin.1531288786.42701

Change-Id: Ie311b4bb2dbe3e5f1e86cb1364039b71d7c08019
Reviewed-on: http://gerrit.cloudera.org:8080/10920
Tested-by: Kudu Jenkins
Reviewed-by: Will Berkeley 


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

Branch: refs/heads/master
Commit: 2692ac4ad750e29e9ca146df3b378a72a1e84a0b
Parents: cee17c0
Author: Alexey Serbin 
Authored: Tue Jul 10 22:04:32 2018 -0700
Committer: Alexey Serbin 
Committed: Thu Jul 12 03:07:42 2018 +

--
 src/kudu/tools/kudu-admin-test.cc   |  7 ---
 src/kudu/tools/tool_replica_util.cc | 30 ++
 2 files changed, 26 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kudu/blob/2692ac4a/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 e084753..21bae3b 100644
--- a/src/kudu/tools/kudu-admin-test.cc
+++ b/src/kudu/tools/kudu-admin-test.cc
@@ -1867,13 +1867,6 @@ TEST_P(ConcurrentRebalancersTest, 
TwoConcurrentRebalancers) {
   {
 string out;
 string err;
-// TODO(aserbin): sometimes, when replica movement fails because of
-//   concurrent rebalancers or other reasons, the REPLACE attribute is left
-//   in replica's Raft config. In such cases, rebalancing fails because
-//   it cannot make progress due to the semantics of the ChangeConfig()
-//   method, returning error
-// 'Invalid argument: must modify a field when calling MODIFY_PEER'
-//   in attempt to set REPLACE attribute again.
 const auto s = RunKuduTool(tool_args, , );
 ASSERT_TRUE(s.ok()) << s.ToString() << ": " << err;
 ASSERT_STR_CONTAINS(out, "rebalancing is complete: cluster is balanced")

http://git-wip-us.apache.org/repos/asf/kudu/blob/2692ac4a/src/kudu/tools/tool_replica_util.cc
--
diff --git a/src/kudu/tools/tool_replica_util.cc 
b/src/kudu/tools/tool_replica_util.cc
index 3001350..e06fbcd 100644
--- a/src/kudu/tools/tool_replica_util.cc
+++ b/src/kudu/tools/tool_replica_util.cc
@@ -347,23 +347,44 @@ Status ScheduleReplicaMove(const vector& 
master_addresses,
   // Get information on current replication scheme: the move scenario depends
   // on the replication scheme used.
   bool is_343_scheme;
+  ConsensusStatePB cstate;
   RETURN_NOT_OK(GetConsensusState(proxy, tablet_id, leader_uuid,
   client->default_admin_operation_timeout(),
-  nullptr, _343_scheme));
+  , _343_scheme));
+  // Sanity check: the target replica should not be present in the config.
+  // Anyway, ChangeConfig() RPC would return an error in that case, but this
+  // pre-condition allows us to short-circuit that.
+  for (const auto& peer : cstate.committed_config().peers()) {
+if (peer.permanent_uuid() == to_ts_uuid) {
+  return Status::IllegalState(Substitute(
+  "tablet $0: replica $1 is already present", tablet_id, to_ts_uuid));
+}
+  }
+
+  const auto cas_opid_idx = cstate.committed_config().opid_index();
+
   // The pre- KUDU-1097 way of moving a replica involves first adding a new
   // replica and then 

kudu git commit: [quorum_util-test] scenario for node decommissioning

2018-07-11 Thread alexey
Repository: kudu
Updated Branches:
  refs/heads/master 077fe5762 -> 4eefc8d4e


[quorum_util-test] scenario for node decommissioning

Added an additional unit test and a scenario to ensure the current
implementation of the 3-4-3 replica management scheme handles
decommissioning of multiple tablet servers (or nodes) as expected.

Change-Id: I46e0946c474095095e9a8ccd84580e945dd4921b
Reviewed-on: http://gerrit.cloudera.org:8080/10889
Reviewed-by: Alexey Serbin 
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: 4eefc8d4e362806957cf92de12e1c456a8b01a7d
Parents: 077fe57
Author: Alexey Serbin 
Authored: Fri Jul 6 20:19:53 2018 -0700
Committer: Alexey Serbin 
Committed: Wed Jul 11 20:36:22 2018 +

--
 src/kudu/consensus/quorum_util-test.cc | 120 
 1 file changed, 120 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/kudu/blob/4eefc8d4/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 46776e0..a32e368 100644
--- a/src/kudu/consensus/quorum_util-test.cc
+++ b/src/kudu/consensus/quorum_util-test.cc
@@ -1490,6 +1490,23 @@ TEST_P(QuorumUtilHealthPolicyParamTest, 
MultipleReplicasWithReplaceAttribute) {
 }
 EXPECT_FALSE(ShouldAddReplica(config, 3, policy));
   }
+  {
+RaftConfigPB config;
+AddPeer(, "A", V, '+', {{"REPLACE", true}});
+AddPeer(, "B", V, '+', {{"REPLACE", true}});
+AddPeer(, "C", V, '+', {{"REPLACE", true}});
+AddPeer(, "D", N, '+', {{"PROMOTE", true}});
+AddPeer(, "E", N, '+', {{"PROMOTE", true}});
+AddPeer(, "F", N, '+', {{"PROMOTE", true}});
+
+for (const string& leader_replica : { "A", "B", "C" }) {
+  // All non-voters are in good shape and not a single one has been
+  // promoted yet.
+  ASSERT_FALSE(ShouldEvictReplica(config, leader_replica, 3, policy));
+}
+// No more replicas are needed for the replacement.
+EXPECT_FALSE(ShouldAddReplica(config, 3, policy));
+  }
 }
 
 // Verify logic of the kudu::consensus::ShouldEvictReplica(), anticipating
@@ -1777,5 +1794,108 @@ TEST(QuorumUtilTest, ReplicaHealthFlapping) {
   EXPECT_FALSE(ShouldAddReplica(config, kReplicationFactor, kPolicy));
 }
 
+// A scenario to simulate the process of migrating all replicas of a tablet,
+// where all replicas are marked for replacement simultaneously. This is a
+// possible scenario when decommissioning multiple tablet servers/nodes at 
once.
+TEST(QuorumUtilTest, ReplaceAllTabletReplicas) {
+  constexpr auto kReplicationFactor = 3;
+  constexpr auto kPolicy = MajorityHealthPolicy::HONOR;
+
+  // The initial tablet report after the tablet replica 'A' has started and
+  // become the leader.
+  RaftConfigPB config;
+  AddPeer(, "A", V, '+', {{"REPLACE", true}});
+  AddPeer(, "B", V, '+', {{"REPLACE", true}});
+  AddPeer(, "C", V, '+', {{"REPLACE", true}});
+  EXPECT_FALSE(ShouldEvictReplica(config, "A", kReplicationFactor, kPolicy));
+  EXPECT_TRUE(ShouldAddReplica(config, kReplicationFactor, kPolicy));
+
+  // First non-voter replica added.
+  AddPeer(, "D", N, '?', {{"PROMOTE", true}});
+  EXPECT_FALSE(ShouldEvictReplica(config, "A", kReplicationFactor, kPolicy));
+  EXPECT_TRUE(ShouldAddReplica(config, kReplicationFactor, kPolicy));
+
+  // Second non-voter replica added.
+  AddPeer(, "E", N, '?', {{"PROMOTE", true}});
+  EXPECT_FALSE(ShouldEvictReplica(config, "A", kReplicationFactor, kPolicy));
+  EXPECT_TRUE(ShouldAddReplica(config, kReplicationFactor, kPolicy));
+
+  // Third non-voter replica added.
+  AddPeer(, "F", N, '?', {{"PROMOTE", true}});
+  EXPECT_FALSE(ShouldEvictReplica(config, "A", kReplicationFactor, kPolicy));
+  EXPECT_FALSE(ShouldAddReplica(config, kReplicationFactor, kPolicy));
+
+  SetPeerHealth(, "D", '+');
+  EXPECT_FALSE(ShouldEvictReplica(config, "A", kReplicationFactor, kPolicy));
+  EXPECT_FALSE(ShouldAddReplica(config, kReplicationFactor, kPolicy));
+
+  // Replica 'D' catches up with the leader's WAL and gets promoted.
+  PromotePeer(, "D");
+  string to_evict;
+  ASSERT_TRUE(ShouldEvictReplica(config, "A", kReplicationFactor, kPolicy, 
_evict));
+  EXPECT_TRUE(to_evict == "B" || to_evict == "C");
+  EXPECT_FALSE(ShouldAddReplica(config, kReplicationFactor, kPolicy));
+
+  // Evicting the replica selected by ShouldEvictReplica() above.
+  RemovePeer(, to_evict);
+  EXPECT_FALSE(ShouldEvictReplica(config, "A", kReplicationFactor, kPolicy));
+  EXPECT_FALSE(ShouldAddReplica(config, kReplicationFactor, kPolicy));
+
+  // Non-voter 

kudu git commit: KUDU-2191: downcase/normalize table names during DDL

2018-07-11 Thread danburkert
Repository: kudu
Updated Branches:
  refs/heads/master 4eefc8d4e -> cee17c03b


KUDU-2191: downcase/normalize table names during DDL

This is a followup to 7b048b8dbe which changed the catalog manager to be
case preserving, but insensitive on lookup when the HMS integration is
enabled. It turns out this was only possible because the HMS failed to
downcase/normalize table names in notification log events[1]. This is an
oversight, and probably could be considered a bug, and HMS developers
have suggested that Kudu should not rely on it.

After a lot of consideration I haven't been able to come up with a way
to keep the case preserving semantics without changes to the HMS APIs,
so instead this commit throws in the towel and adopts HMS-style case
normalization during CREATE TABLE and ALTER TABLE RENAME operations.

Existing tables with uppercase table names will not be altered
automatically (this is consistent with the current handling of non-ascii
chars in table names), so the upgrade CLI tool will be extended in a
follow up commit to handle this.

[1] In particular, renaming a table is problematic if the notification
log listener events doesn't preserve case due to how the catalog manager
/ notification log listener handles table renames.

Change-Id: Ie32a209d9d85851562691ddbc30f7dd02886bad7
Reviewed-on: http://gerrit.cloudera.org:8080/10903
Tested-by: Kudu Jenkins
Reviewed-by: Adar Dembo 


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

Branch: refs/heads/master
Commit: cee17c03bc30037bf2a7d97c6bb3aa39cec34a4c
Parents: 4eefc8d
Author: Dan Burkert 
Authored: Tue Jul 10 11:07:03 2018 -0700
Committer: Dan Burkert 
Committed: Wed Jul 11 23:01:51 2018 +

--
 src/kudu/integration-tests/master_hms-itest.cc | 19 ++---
 src/kudu/master/catalog_manager.cc | 91 +
 2 files changed, 46 insertions(+), 64 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kudu/blob/cee17c03/src/kudu/integration-tests/master_hms-itest.cc
--
diff --git a/src/kudu/integration-tests/master_hms-itest.cc 
b/src/kudu/integration-tests/master_hms-itest.cc
index 4b639f5..2fd415c 100644
--- a/src/kudu/integration-tests/master_hms-itest.cc
+++ b/src/kudu/integration-tests/master_hms-itest.cc
@@ -499,23 +499,16 @@ TEST_F(MasterHmsTest, TestUppercaseIdentifiers) {
 ASSERT_EQ(name, table->name());
   }
 
-  // Listing tables shows the preserved case.
+  // Listing tables shows the normalized case.
   vector tables;
   ASSERT_OK(client_->ListTables());
-  ASSERT_EQ(tables, vector({ "default.MyTable" }));
+  ASSERT_EQ(tables, vector({ "default.mytable" }));
 
   // Rename the table to the same normalized name, but with a different case.
   unique_ptr table_alterer;
   table_alterer.reset(client_->NewTableAlterer("default.mytable"));
-  ASSERT_OK(table_alterer->RenameTo("DEFAULT.MYTABLE")->Alter());
-  NO_FATALS(CheckTable("default", "MyTable"));
-  NO_FATALS(CheckTable("default", "mytable"));
-  NO_FATALS(CheckTable("default", "MYTABLE"));
-
-  // The master should retain the new case.
-  tables.clear();
-  ASSERT_OK(client_->ListTables());
-  ASSERT_EQ(tables, vector({ "DEFAULT.MYTABLE" }));
+  Status s = table_alterer->RenameTo("DEFAULT.MYTABLE")->Alter();
+  ASSERT_TRUE(s.IsAlreadyPresent()) << s.ToString();
 
   // Rename the table to something different.
   table_alterer.reset(client_->NewTableAlterer("DEFAULT.MYTABLE"));
@@ -530,10 +523,10 @@ TEST_F(MasterHmsTest, TestUppercaseIdentifiers) {
 NO_FATALS(CheckTable("default", "AbC"));
   });
 
-  // Listing tables shows the preserved case.
+  // Listing tables shows the normalized case.
   tables.clear();
   ASSERT_OK(client_->ListTables());
-  ASSERT_EQ(tables, vector({ "default.AbC" }));
+  ASSERT_EQ(tables, vector({ "default.abc" }));
 
   // Drop the table.
   ASSERT_OK(client_->DeleteTable("DEFAULT.abc"));

http://git-wip-us.apache.org/repos/asf/kudu/blob/cee17c03/src/kudu/master/catalog_manager.cc
--
diff --git a/src/kudu/master/catalog_manager.cc 
b/src/kudu/master/catalog_manager.cc
index 8f58924..b10a4cc 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -1363,10 +1363,9 @@ Status CatalogManager::CreateTable(const 
CreateTableRequestPB* orig_req,
   // a. Validate the user request.
   Schema client_schema;
   RETURN_NOT_OK(SchemaFromPB(req.schema(), _schema));
-  const string& table_name = req.name();
-  string normalized_table_name = NormalizeTableName(table_name);
+  string normalized_table_name = NormalizeTableName(req.name());