You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2018/09/19 20:46:47 UTC

[1/2] kudu git commit: [security] test scenario for KUDU-2580

Repository: kudu
Updated Branches:
  refs/heads/master 42dd8f842 -> ffb1f22a7


[security] test scenario for KUDU-2580

Added a repro test scenario for KUDU-2580.  The test is disabled
since it's failing.  The test will be re-enabled in a follow-up patch
that fixes KUDU-2580.

Change-Id: I58218deef24cca7c524bc61700cd400cdaabd050
Reviewed-on: http://gerrit.cloudera.org:8080/11448
Reviewed-by: Adar Dembo <ad...@cloudera.com>
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/cc5c7601
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/cc5c7601
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/cc5c7601

Branch: refs/heads/master
Commit: cc5c7601c99eea9535f61a4ee4cbd35fe8939d1d
Parents: 42dd8f8
Author: Alexey Serbin <as...@cloudera.com>
Authored: Sun Sep 16 20:53:47 2018 -0700
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Wed Sep 19 20:46:06 2018 +0000

----------------------------------------------------------------------
 .../authn_token_expire-itest.cc                 | 189 +++++++++++++++++--
 1 file changed, 175 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/cc5c7601/src/kudu/integration-tests/authn_token_expire-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/authn_token_expire-itest.cc b/src/kudu/integration-tests/authn_token_expire-itest.cc
index 81fdfc5..835c74e 100644
--- a/src/kudu/integration-tests/authn_token_expire-itest.cc
+++ b/src/kudu/integration-tests/authn_token_expire-itest.cc
@@ -31,18 +31,27 @@
 #include "kudu/client/shared_ptr.h"
 #include "kudu/client/write_op.h"
 #include "kudu/common/partial_row.h"
+#include "kudu/consensus/consensus.pb.h"
+#include "kudu/consensus/consensus.proxy.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/integration-tests/cluster_verifier.h"
 #include "kudu/integration-tests/test_workload.h"
+#include "kudu/master/sys_catalog.h"
 #include "kudu/mini-cluster/external_mini_cluster.h"
+#include "kudu/rpc/rpc_controller.h"
 #include "kudu/tablet/key_value_test_schema.h"
 #include "kudu/util/monotime.h"
+#include "kudu/util/net/net_util.h"
+#include "kudu/util/net/sockaddr.h"
 #include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
 DECLARE_bool(rpc_reopen_outbound_connections);
 
+using kudu::client::sp::shared_ptr;
+using kudu::cluster::ExternalMiniCluster;
+using kudu::cluster::ExternalMiniClusterOptions;
 using std::string;
 using std::unique_ptr;
 using strings::Substitute;
@@ -50,10 +59,6 @@ using strings::Substitute;
 namespace kudu {
 namespace client {
 
-using cluster::ExternalMiniCluster;
-using cluster::ExternalMiniClusterOptions;
-using sp::shared_ptr;
-
 namespace {
 
 // Create a table with the specified name (no replication) and specified
@@ -96,12 +101,15 @@ Status InsertTestRows(KuduClient* client, KuduTable* table,
 
 class AuthnTokenExpireITestBase : public KuduTest {
  public:
-  AuthnTokenExpireITestBase(int num_tablet_servers,
-                            int64_t token_validity_seconds)
-      : num_tablet_servers_(num_tablet_servers),
-        token_validity_seconds_(token_validity_seconds),
+  AuthnTokenExpireITestBase(int64_t token_validity_seconds,
+                            int num_masters,
+                            int num_tablet_servers)
+      : token_validity_seconds_(token_validity_seconds),
+        num_masters_(num_masters),
+        num_tablet_servers_(num_tablet_servers),
         schema_(client::KuduSchemaFromSchema(CreateKeyValueTestSchema())) {
     cluster_opts_.num_tablet_servers = num_tablet_servers_;
+    cluster_opts_.num_masters = num_masters_;
     cluster_opts_.enable_kerberos = true;
   }
 
@@ -118,8 +126,9 @@ class AuthnTokenExpireITestBase : public KuduTest {
   }
 
  protected:
-  const int num_tablet_servers_;
   const int64_t token_validity_seconds_;
+  const int num_masters_;
+  const int num_tablet_servers_;
   KuduSchema schema_;
   ExternalMiniClusterOptions cluster_opts_;
   shared_ptr<ExternalMiniCluster> cluster_;
@@ -129,7 +138,9 @@ class AuthnTokenExpireITestBase : public KuduTest {
 class AuthnTokenExpireITest : public AuthnTokenExpireITestBase {
  public:
   explicit AuthnTokenExpireITest(int64_t token_validity_seconds = 2)
-      : AuthnTokenExpireITestBase(3, token_validity_seconds) {
+      : AuthnTokenExpireITestBase(token_validity_seconds,
+                                  /*num_masters=*/ 1,
+                                  /*num_tablet_servers=*/ 3) {
     // Masters and tservers inject FATAL_INVALID_AUTHENTICATION_TOKEN errors.
     // The client should retry the operation again and eventually it should
     // succeed even with the high ratio of injected errors.
@@ -340,12 +351,14 @@ TEST_F(AuthnTokenExpireDuringWorkloadITest, InvalidTokenDuringSeparateWorkloads)
 
 // Scenarios to verify that the client automatically re-acquires authn token
 // when receiving ERROR_INVALID_AUTHENTICATION_TOKEN from the servers in case
-// if the client has established a token-based connection to master server.
+// if the client has established a token-based connection to masters.
 class TokenBasedConnectionITest : public AuthnTokenExpireITestBase {
  public:
   TokenBasedConnectionITest()
-      : AuthnTokenExpireITestBase(3, 2) {
-
+      : AuthnTokenExpireITestBase(
+          /*token_validity_seconds=*/ 2,
+          /*num_masters=*/ 1,
+          /*num_tablet_servers=*/ 3) {
     cluster_opts_.extra_master_flags = {
       Substitute("--authn_token_validity_seconds=$0", token_validity_seconds_),
     };
@@ -385,7 +398,7 @@ TEST_F(TokenBasedConnectionITest, ReacquireAuthnToken) {
   ASSERT_OK(cluster_->Restart());
 
   // Perform some operations using already existing token. The crux here is to
-  // establish a connection to the master server where client is authenticated
+  // establish a connection to the master where client is authenticated
   // via the authn token, not Kerberos credentials.
   shared_ptr<KuduTable> table;
   ASSERT_OK(client->OpenTable(table_name, &table));
@@ -401,5 +414,153 @@ TEST_F(TokenBasedConnectionITest, ReacquireAuthnToken) {
   NO_FATALS(cluster_->AssertNoCrashes());
 }
 
+// Test for scenarios involving multiple masters where
+// client-to-non-leader-master connections are closed due to inactivity,
+// but the connection to the former leader master is kept open.
+class MultiMasterIdleConnectionsITest : public AuthnTokenExpireITestBase {
+ public:
+  MultiMasterIdleConnectionsITest()
+      : AuthnTokenExpireITestBase(
+          /*token_validity_seconds=*/ 3,
+          /*num_masters=*/ 3,
+          /*num_tablet_servers=*/ 3) {
+
+    cluster_opts_.extra_master_flags = {
+      // Custom validity interval for authn tokens. The scenario involves
+      // expiration of authn tokens, while the default authn expiration timeout
+      // is 7 days. So, let's make the token validity interval really short.
+      Substitute("--authn_token_validity_seconds=$0", token_validity_seconds_),
+
+      // The default for leader_failure_max_missed_heartbeat_periods 3.0, but
+      // 2.0 is enough to have master leadership stable enough and makes it
+      // run a bit faster.
+      Substitute("--leader_failure_max_missed_heartbeat_periods=$0",
+          master_leader_failure_max_missed_heartbeat_periods_),
+
+      // Custom Raft heartbeat interval between replicas of the systable.
+      // The default it 500ms, but custom setting keeps the test stable enough
+      // while making it a bit faster to complete.
+      Substitute("--raft_heartbeat_interval_ms=$0",
+          master_raft_hb_interval_ms_),
+
+      // The default is 65 seconds, but the test scenario need to run faster.
+      // A multiple of (leader_failure_max_missed_heartbeat_periods *
+      // raft_heartbeat_interval_ms) is good enough, but it's also necessary
+      // it to be greater than token validity interval due to the scenario's
+      // logic.
+      Substitute("--rpc_default_keepalive_time_ms=$0",
+          master_rpc_keepalive_time_ms_),
+    };
+
+    cluster_opts_.extra_tserver_flags = {
+      // Decreasing TS->master heartbeat interval speeds up the test.
+      "--heartbeat_interval_ms=100",
+    };
+  }
+
+  void SetUp() override {
+    AuthnTokenExpireITestBase::SetUp();
+    ASSERT_OK(cluster_->Start());
+  }
+
+ protected:
+  const int master_rpc_keepalive_time_ms_ = 3 * token_validity_seconds_ * 1000 / 2;
+  const int master_raft_hb_interval_ms_ = 250;
+  const double master_leader_failure_max_missed_heartbeat_periods_ = 2.0;
+};
+
+// Verify that Kudu C++ client reacquires authn token in the following scenario:
+//
+//   1. Client is running against a multi-master cluster.
+//   2. Client successfully authenticates and gets an authn token by calling
+//      ConnectToCluster.
+//   3. Client keeps the connection to leader master open, but follower masters
+//      close connections to the client due to inactivity.
+//   4. After the authn token expires, a change in master leadership happens.
+//   5. Client tries to open a table, first making a request to the former
+//      leader master.  However, the former leader returns NOT_THE_LEADER error.
+//
+// In that situation, the client should reconnect to the cluster to get a new
+// authn token. Prior to the KUDU-2580 fix, it didn't, and the test was failing
+// when the client tried to open the test table after master leader re-election:
+//   Timed out: GetTableSchema timed out after deadline expired
+TEST_F(MultiMasterIdleConnectionsITest, DISABLED_ClientReacquiresAuthnToken) {
+  const string kTableName = "keep-connection-to-former-master-leader";
+
+  if (!AllowSlowTests()) {
+    LOG(WARNING) << "test is skipped; set KUDU_ALLOW_SLOW_TESTS=1 to run";
+    return;
+  }
+
+  const auto time_start = MonoTime::Now();
+
+  shared_ptr<KuduClient> client;
+  {
+    KuduClientBuilder builder;
+    builder.default_rpc_timeout(MonoDelta::FromSeconds(5));
+    ASSERT_OK(cluster_->CreateClient(&builder, &client));
+  }
+  ASSERT_OK(CreateTable(client.get(), kTableName, schema_, num_tablet_servers_));
+
+  // Wait for the following events:
+  //   1) authn token expires
+  //   2) connections to non-leader masters close
+
+  const auto time_right_before_token_expiration = time_start +
+      MonoDelta::FromSeconds(token_validity_seconds_);
+  while (MonoTime::Now() < time_right_before_token_expiration) {
+    // Keep the connection to leader master open, time to time making requests
+    // that go to the leader master, but not to other masters in the cluster.
+    //
+    // The leader master might unexpectedly change in the middle of this cycle,
+    // but that would not induce errors in this cycle. The only negative outcome
+    // of that unexpected re-election would be missing the conditions of the
+    // reference scenario, but due to the relative stability of the Raft
+    // leadership role given current parameters for masters Raft consensus,
+    // the reference condition is reproduced in the vast majority of runs.
+    shared_ptr<KuduTable> table;
+    ASSERT_OK(client->OpenTable(kTableName, &table));
+    SleepFor(MonoDelta::FromMilliseconds(250));
+  }
+
+  // Given the relation between the master_rpc_keepalive_time_ms_ and
+  // token_validity_seconds_ parameters, the original authn token should expire
+  // and connections to follower masters should be torn down due to inactivity,
+  // but the connection to the leader master should be kept open after waiting
+  // for additional token expiration interval.
+  SleepFor(MonoDelta::FromSeconds(token_validity_seconds_));
+
+  {
+    int former_leader_master_idx;
+    ASSERT_OK(cluster_->GetLeaderMasterIndex(&former_leader_master_idx));
+    const int leader_idx = (former_leader_master_idx + 1) % num_masters_;
+    ASSERT_EVENTUALLY([&] {
+      consensus::ConsensusServiceProxy proxy(
+          cluster_->messenger(), cluster_->master(leader_idx)->bound_rpc_addr(),
+          cluster_->master(leader_idx)->bound_rpc_hostport().host());
+      consensus::RunLeaderElectionRequestPB req;
+      req.set_tablet_id(master::SysCatalogTable::kSysCatalogTabletId);
+      req.set_dest_uuid(cluster_->master(leader_idx)->uuid());
+      rpc::RpcController rpc;
+      rpc.set_timeout(MonoDelta::FromSeconds(1));
+      consensus::RunLeaderElectionResponsePB resp;
+      ASSERT_OK(proxy.RunLeaderElection(req, &resp, &rpc));
+      int idx;
+      ASSERT_OK(cluster_->GetLeaderMasterIndex(&idx));
+      ASSERT_NE(former_leader_master_idx, idx);
+    });
+  }
+
+  // Try to open the table after leader master re-election. The former leader
+  // responds with NOT_THE_LEADER error even if the authn token has expired
+  // (i.e. the client will not get FATAL_INVALID_AUTHENTICATION_TOKEN error).
+  // That's because the connection between the client and the former leader
+  // master was established in the beginning and kept open during the test.
+  // However, the client should detect that condition and reconnect to the
+  // cluster for a new authn token.
+  shared_ptr<KuduTable> table;
+  ASSERT_OK(client->OpenTable(kTableName, &table));
+}
+
 } // namespace client
 } // namespace kudu


[2/2] kudu git commit: KUDU-2580 [c++ client] authn token reacquisition fix

Posted by al...@apache.org.
KUDU-2580 [c++ client] authn token reacquisition fix

This patch updates the authn token reacquisition logic to handle
the following scenario:

1. Client is running against a multi-master cluster.
2. Client successfully authenticates and gets an authn token by calling
   ConnectToCluster().
3. Client keeps the connection to leader master open, but follower
   masters close connections to the client due to inactivity.
4. After the authn token expires, a change in the master leadership
   happens.
5. Client tries to open a table, making a request to the former leader
   master. The former leader returns NOT_THE_LEADER error.

The original authn token reacquisition logic was straightforwardly
retrying RPCs in case of error responses with code
FATAL_INVALID_AUTHENTICATION_TOKEN only.  The scenario described
above was not handled properly, so the opening table operation would
fail with the following error:
   Timed out: GetTableSchema timed out after deadline expired

Under the hood, the following would happen prior to this patch in
the above scenario when calling the GetTableSchema():
  * Having the connection to the former leader master still open,
    but authn token expired and master re-election selected another
    master as leader, the GetTableSchema() RPC call would hit the
    code of the last 'if ()' closure within the 'for ()' retry cycle
    in the KuduClient::Data::SyncLeaderMasterRpc() method,
    with error code MasterErrorPB::NOT_THE_LEADER.
  * The ConnectToMaster() attempt would return an error with the
    FATAL_INVALID_AUTHENTICATION_TOKEN error code, and the control
    flow will continue with the retries, but the state of the client
    stays the same (i.e. the leader master proxy is not updated).
  * That will happen again and again, until the GetTableSchema() RPC
    call times out.

This patch also enables the ClientReacquiresAuthnToken scenario
of MultiMasterIdleConnectionsITest since it passes with this fix.

Change-Id: I4477d0f2bb36ee5ef580b585cae189a634d5002f
Reviewed-on: http://gerrit.cloudera.org:8080/11449
Reviewed-by: Adar Dembo <ad...@cloudera.com>
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/ffb1f22a
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/ffb1f22a
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/ffb1f22a

Branch: refs/heads/master
Commit: ffb1f22a70c8ef67fd4f8ab8d8a7203020674c4e
Parents: cc5c760
Author: Alexey Serbin <as...@cloudera.com>
Authored: Sun Sep 16 21:58:28 2018 -0700
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Wed Sep 19 20:46:14 2018 +0000

----------------------------------------------------------------------
 src/kudu/client/client-internal.cc              | 56 ++++++++++++++------
 src/kudu/client/client-internal.h               | 12 +++++
 .../authn_token_expire-itest.cc                 |  2 +-
 3 files changed, 53 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/ffb1f22a/src/kudu/client/client-internal.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-internal.cc b/src/kudu/client/client-internal.cc
index 0fb45ad..8300ecc 100644
--- a/src/kudu/client/client-internal.cc
+++ b/src/kudu/client/client-internal.cc
@@ -222,8 +222,7 @@ Status KuduClient::Data::SyncLeaderMasterRpc(
           << "): " << s.ToString();
       if (client->IsMultiMaster()) {
         LOG(INFO) << "Determining the new leader Master and retrying...";
-        WARN_NOT_OK(ConnectToCluster(client, deadline),
-                    "Unable to determine the new leader Master");
+        ReconnectToCluster(client, deadline, ReconnectionReason::OTHER);
         continue;
       }
     }
@@ -233,14 +232,8 @@ Status KuduClient::Data::SyncLeaderMasterRpc(
       if (err && err->has_code() &&
           err->code() == ErrorStatusPB::FATAL_INVALID_AUTHENTICATION_TOKEN) {
         // Assuming the token has expired: it's necessary to get a new one.
-        LOG(INFO) << "Reconnecting to the cluster for a new authn token";
-        const Status connect_status = ConnectToCluster(
-            client, deadline, CredentialsPolicy::PRIMARY_CREDENTIALS);
-        if (PREDICT_FALSE(!connect_status.ok())) {
-          KLOG_EVERY_N_SECS(WARNING, 1)
-              << "Unable to reconnect to the cluster for a new authn token: "
-              << connect_status.ToString();
-        }
+        ReconnectToCluster(client, deadline,
+                           ReconnectionReason::INVALID_AUTHN_TOKEN);
         continue;
       }
     }
@@ -252,8 +245,7 @@ Status KuduClient::Data::SyncLeaderMasterRpc(
           << "): " << s.ToString();
       if (client->IsMultiMaster()) {
         LOG(INFO) << "Determining the new leader Master and retrying...";
-        WARN_NOT_OK(ConnectToCluster(client, deadline),
-                    "Unable to determine the new leader Master");
+        ReconnectToCluster(client, deadline, ReconnectionReason::OTHER);
       }
       continue;
     }
@@ -266,8 +258,7 @@ Status KuduClient::Data::SyncLeaderMasterRpc(
             << "): " << s.ToString();
         if (client->IsMultiMaster()) {
           LOG(INFO) << "Determining the new leader Master and retrying...";
-          WARN_NOT_OK(ConnectToCluster(client, deadline),
-                      "Unable to determine the new leader Master");
+          ReconnectToCluster(client, deadline, ReconnectionReason::OTHER);
         }
         continue;
       } else {
@@ -282,8 +273,7 @@ Status KuduClient::Data::SyncLeaderMasterRpc(
           resp->error().code() == MasterErrorPB::CATALOG_MANAGER_NOT_INITIALIZED) {
         if (client->IsMultiMaster()) {
           KLOG_EVERY_N_SECS(INFO, 1) << "Determining the new leader Master and retrying...";
-          WARN_NOT_OK(ConnectToCluster(client, deadline),
-                      "Unable to determine the new leader Master");
+          ReconnectToCluster(client, deadline, ReconnectionReason::OTHER);
         }
         continue;
       } else {
@@ -831,6 +821,40 @@ void KuduClient::Data::ConnectToClusterAsync(KuduClient* client,
   }
 }
 
+void KuduClient::Data::ReconnectToCluster(KuduClient* client,
+                                          const MonoTime& deadline,
+                                          ReconnectionReason reason) {
+  DCHECK(client);
+  DCHECK(reason == ReconnectionReason::OTHER ||
+         reason == ReconnectionReason::INVALID_AUTHN_TOKEN);
+  if (reason == ReconnectionReason::OTHER) {
+    const auto s = ConnectToCluster(client, deadline,
+                                    CredentialsPolicy::ANY_CREDENTIALS);
+    if (s.ok()) {
+      return;
+    }
+    if (!s.IsNotAuthorized()) {
+      // In case of NotAutorized() error, that's most likely due to invalid
+      // authentication token. That's the only case when it's worth trying
+      // to re-connect to the cluster using primary credentials.
+      //
+      // TODO(aserbin): refactor ConnectToCluster to purge cached master proxy
+      //                in case of NOT_THE_LEADER error and update it to
+      //                handle FATAL_INVALID_AUTHENTICATION_TOKEN error as well.
+      WARN_NOT_OK(s, "Unable to determine the new leader Master");
+      return;
+    }
+  }
+  LOG(INFO) << "Reconnecting to the cluster for a new authn token";
+  const auto connect_status = ConnectToCluster(
+      client, deadline, CredentialsPolicy::PRIMARY_CREDENTIALS);
+  if (PREDICT_FALSE(!connect_status.ok())) {
+    KLOG_EVERY_N_SECS(WARNING, 1)
+        << "Unable to reconnect to the cluster for a new authn token: "
+        << connect_status.ToString();
+  }
+}
+
 HostPort KuduClient::Data::leader_master_hostport() const {
   std::lock_guard<simple_spinlock> l(leader_master_lock_);
   return leader_master_hostport_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/ffb1f22a/src/kudu/client/client-internal.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-internal.h b/src/kudu/client/client-internal.h
index 5958533..e359114 100644
--- a/src/kudu/client/client-internal.h
+++ b/src/kudu/client/client-internal.h
@@ -188,6 +188,18 @@ class KuduClient::Data {
       const MonoTime& deadline,
       rpc::CredentialsPolicy creds_policy = rpc::CredentialsPolicy::ANY_CREDENTIALS);
 
+  // A wrapper around ConnectToCluster() to handle various errors in case
+  // if a call to thought-to-be-leader master fails. First, this method calls
+  // ConnectToCluster() with current client credentials unless
+  // INVALID_AUTHN_TOKEN reason is specified. If the ConnectToCluster() with the
+  // current client credentials fails, call ConnectToCluster() with primary
+  // credentials. The ReconnectionReason is a dedicated enumeration for the
+  // third parameter of the method.
+  enum class ReconnectionReason { INVALID_AUTHN_TOKEN, OTHER };
+  void ReconnectToCluster(KuduClient* client,
+                          const MonoTime& deadline,
+                          ReconnectionReason reason);
+
   std::shared_ptr<master::MasterServiceProxy> master_proxy() const;
 
   HostPort leader_master_hostport() const;

http://git-wip-us.apache.org/repos/asf/kudu/blob/ffb1f22a/src/kudu/integration-tests/authn_token_expire-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/authn_token_expire-itest.cc b/src/kudu/integration-tests/authn_token_expire-itest.cc
index 835c74e..c6dd663 100644
--- a/src/kudu/integration-tests/authn_token_expire-itest.cc
+++ b/src/kudu/integration-tests/authn_token_expire-itest.cc
@@ -484,7 +484,7 @@ class MultiMasterIdleConnectionsITest : public AuthnTokenExpireITestBase {
 // authn token. Prior to the KUDU-2580 fix, it didn't, and the test was failing
 // when the client tried to open the test table after master leader re-election:
 //   Timed out: GetTableSchema timed out after deadline expired
-TEST_F(MultiMasterIdleConnectionsITest, DISABLED_ClientReacquiresAuthnToken) {
+TEST_F(MultiMasterIdleConnectionsITest, ClientReacquiresAuthnToken) {
   const string kTableName = "keep-connection-to-former-master-leader";
 
   if (!AllowSlowTests()) {