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/01/29 20:14:39 UTC

[1/2] kudu git commit: client: fix UBSAN error in failover calculation

Repository: kudu
Updated Branches:
  refs/heads/master 9941b56bf -> 54081ce18


client: fix UBSAN error in failover calculation

A misplaced typecast caused the following:

/data/somelongdirectorytoavoidrpathissues/src/kudu/src/kudu/client/scanner-internal.cc:433:72:
runtime error: unsigned integer overflow: 4 - 5 cannot be represented in type 'unsigned long'

This patch simply rearranges the expression a bit to avoid underflow
possibilities.

Change-Id: Icd137f86443bc4af59f8a4290a45151afda6c090
Reviewed-on: http://gerrit.cloudera.org:8080/9132
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin <as...@cloudera.com>
Reviewed-by: Dan Burkert <da...@cloudera.com>


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

Branch: refs/heads/master
Commit: 471e0a137cfe33736d8c2b5a23ad6102bcb22e79
Parents: 9941b56
Author: Todd Lipcon <to...@apache.org>
Authored: Wed Jan 24 22:35:08 2018 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Mon Jan 29 19:36:26 2018 +0000

----------------------------------------------------------------------
 src/kudu/client/scanner-internal.cc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/471e0a13/src/kudu/client/scanner-internal.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/scanner-internal.cc b/src/kudu/client/scanner-internal.cc
index 92948c9..f26f588 100644
--- a/src/kudu/client/scanner-internal.cc
+++ b/src/kudu/client/scanner-internal.cc
@@ -430,7 +430,7 @@ Status KuduScanner::Data::OpenTablet(const string& partition_key,
     ts_ = CHECK_NOTNULL(ts);
     proxy_ = ts_->proxy();
 
-    bool allow_time_for_failover = static_cast<int>(candidates.size()) - blacklist->size() > 1;
+    bool allow_time_for_failover = candidates.size() > blacklist->size() + 1;
     ScanRpcStatus scan_status = SendScanRpc(deadline, allow_time_for_failover);
     if (scan_status.result == ScanRpcStatus::OK) {
       last_error_ = Status::OK();


[2/2] kudu git commit: KUDU-1927: no half-baked responses on ConnectToMaster

Posted by al...@apache.org.
KUDU-1927: no half-baked responses on ConnectToMaster

Do not send half-baked responses to clients from a master that declares
itself a leader if its catalog manager/CA authority/TokenSigner
are not in proper state.

While working on AuthTokenIssuingTest.ChannelConfidentiality test,
I noticed that implementation of MasterServiceImpl::ConnectToMaster()
allowed for getting a success response without proper authn/security
information in case if the master hasn't been established as a leader
yet.  By examining the code further I found that could happen not
only at the very first startup of a master, but also during master
re-elections in case of multi-master setup.

If a legit client connects to master but it does not get CA cert and
authn token, it might be a situation when it works flawlessly with
masters and tablet servers using its Kerberos credentials, but the
exported authentication credentials contain neither CA cert nor authn
token.  The latter is very surprising in cases when the credentials
are later imported by other Kudu client applications that do not have
Kerberos credentials in their environment.  If so, the client is not
able to connect to a secured Kudu cluster at all.

Added a dedicated integration test to verify that the new implementation
does not allow for such a behavior.  In addition, updated the already
existing AuthTokenIssuingTest.ChannelConfidentiality test so now it
provides some coverage for the new code as well.

Change-Id: Ib9c906863f5f0e1995041281b122135e1b2cd3a4
Reviewed-on: http://gerrit.cloudera.org:8080/9052
Reviewed-by: Todd Lipcon <to...@apache.org>
Tested-by: Kudu Jenkins
Reviewed-by: Dan Burkert <da...@cloudera.com>


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

Branch: refs/heads/master
Commit: 54081ce18ffff42a2a04f55d892aba3523ba9df9
Parents: 471e0a1
Author: Alexey Serbin <as...@cloudera.com>
Authored: Wed Jan 17 18:18:16 2018 -0800
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Mon Jan 29 20:13:57 2018 +0000

----------------------------------------------------------------------
 src/kudu/client/client.h                        |   1 +
 .../master_cert_authority-itest.cc              | 154 +++++++++++++++++--
 src/kudu/integration-tests/security-itest.cc    |  19 ---
 src/kudu/master/catalog_manager.cc              |  13 +-
 src/kudu/master/master_cert_authority.cc        |  17 +-
 src/kudu/master/master_cert_authority.h         |  11 +-
 src/kudu/master/master_service.cc               |  45 +++---
 7 files changed, 197 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/54081ce1/src/kudu/client/client.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/client.h b/src/kudu/client/client.h
index b8dd7ec..a9d09cf 100644
--- a/src/kudu/client/client.h
+++ b/src/kudu/client/client.h
@@ -527,6 +527,7 @@ class KUDU_EXPORT KuduClient : public sp::enable_shared_from_this<KuduClient> {
   friend class internal::RemoteTablet;
   friend class internal::RemoteTabletServer;
   friend class internal::WriteRpc;
+  friend class ConnectToClusterBaseTest;
   friend class ClientTest;
   friend class KuduClientBuilder;
   friend class KuduPartitionerBuilder;

http://git-wip-us.apache.org/repos/asf/kudu/blob/54081ce1/src/kudu/integration-tests/master_cert_authority-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/master_cert_authority-itest.cc b/src/kudu/integration-tests/master_cert_authority-itest.cc
index 806b870..4687477 100644
--- a/src/kudu/integration-tests/master_cert_authority-itest.cc
+++ b/src/kudu/integration-tests/master_cert_authority-itest.cc
@@ -15,23 +15,33 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <algorithm>
+#include <cstdint>
+#include <iterator>
 #include <memory>
+#include <ostream>
 #include <string>
+#include <thread>
 #include <vector>
 
 #include <boost/optional/optional.hpp>
 #include <gflags/gflags_declare.h>
+#include <glog/logging.h>
 #include <gtest/gtest.h>
 
+#include "kudu/client/client-internal.h"
+#include "kudu/client/client.h"
+#include "kudu/client/shared_ptr.h"
 #include "kudu/common/common.pb.h"
 #include "kudu/common/wire_protocol.pb.h"
 #include "kudu/consensus/replica_management.pb.h"
-#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/strings/substitute.h"
 #include "kudu/master/master.h"
 #include "kudu/master/master.pb.h"
 #include "kudu/master/master.proxy.h"
 #include "kudu/master/master_cert_authority.h"
 #include "kudu/master/mini_master.h"
+#include "kudu/mini-cluster/external_mini_cluster.h"
 #include "kudu/mini-cluster/internal_mini_cluster.h"
 #include "kudu/rpc/messenger.h"
 #include "kudu/rpc/rpc_controller.h"
@@ -39,8 +49,12 @@
 #include "kudu/security/cert.h"
 #include "kudu/security/crypto.h"
 #include "kudu/security/openssl_util.h"
+#include "kudu/security/tls_context.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/monotime.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/pb_util.h"
+#include "kudu/util/scoped_cleanup.h"
 #include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
@@ -48,16 +62,21 @@
 
 DECLARE_bool(raft_prepare_replacement_before_eviction);
 
+using kudu::cluster::ExternalMiniCluster;
+using kudu::cluster::ExternalMiniClusterOptions;
 using kudu::cluster::InternalMiniCluster;
 using kudu::cluster::InternalMiniClusterOptions;
 using kudu::consensus::ReplicaManagementInfoPB;
-using kudu::security::ca::CertRequestGenerator;
 using kudu::security::Cert;
 using kudu::security::CertSignRequest;
 using kudu::security::DataFormat;
 using kudu::security::PrivateKey;
+using kudu::security::ca::CertRequestGenerator;
+using std::back_inserter;
+using std::copy;
 using std::string;
-using std::shared_ptr;
+using std::vector;
+using strings::Substitute;
 
 
 namespace kudu {
@@ -70,8 +89,7 @@ class MasterCertAuthorityTest : public KuduTest {
     // runs under a resource lock (see CMakeLists.txt in this directory).
     // TODO(aserbin): we should have a generic method to obtain n free ports.
     opts_.master_rpc_ports = { 11010, 11011, 11012 };
-
-    opts_.num_masters = num_masters_ = opts_.master_rpc_ports.size();
+    opts_.num_masters = opts_.master_rpc_ports.size();
   }
 
   void SetUp() override {
@@ -198,11 +216,9 @@ class MasterCertAuthorityTest : public KuduTest {
  protected:
   static const char kFakeTsUUID[];
 
-  int num_masters_;
   InternalMiniClusterOptions opts_;
-  gscoped_ptr<InternalMiniCluster> cluster_;
-
-  shared_ptr<rpc::Messenger> messenger_;
+  std::unique_ptr<InternalMiniCluster> cluster_;
+  std::shared_ptr<rpc::Messenger> messenger_;
 };
 const char MasterCertAuthorityTest::kFakeTsUUID[] = "fake-ts-uuid";
 
@@ -340,4 +356,124 @@ TEST_F(MasterCertAuthorityTest, MasterLeaderSignsCSR) {
 }
 
 } // namespace master
+
+namespace client {
+
+class ConnectToClusterBaseTest : public KuduTest {
+ public:
+  ConnectToClusterBaseTest(int run_time_seconds,
+                           int latency_ms,
+                           vector<uint16_t> master_ports)
+      : run_time_seconds_(run_time_seconds),
+        latency_ms_(latency_ms) {
+    cluster_opts_.master_rpc_ports = std::move(master_ports);
+    cluster_opts_.num_masters = cluster_opts_.master_rpc_ports.size();
+  }
+
+  void SetUp() override {
+    KuduTest::SetUp();
+    cluster_.reset(new ExternalMiniCluster(cluster_opts_));
+  }
+
+  void ConnectToCluster() {
+    const MonoDelta timeout(MonoDelta::FromSeconds(run_time_seconds_));
+    KuduClientBuilder builder;
+    builder.default_admin_operation_timeout(timeout);
+    builder.default_rpc_timeout(timeout);
+    client::sp::shared_ptr<KuduClient> client;
+    ASSERT_OK(cluster_->CreateClient(&builder, &client));
+    ASSERT_EQ(1, client->data_->messenger_-> tls_context().trusted_cert_count_for_tests());
+    ASSERT_NE(boost::none, client->data_->messenger_->authn_token());
+  }
+
+  void Run() {
+    const MonoTime t_stop = MonoTime::Now() + MonoDelta::FromSeconds(run_time_seconds_);
+    CountDownLatch stop_latch(1);
+    std::thread clear_latency_thread([&]{
+      // Allow the test client to connect to the cluster (avoid timing out).
+      const MonoTime clear_latency = t_stop -
+          MonoDelta::FromMilliseconds(1000L * run_time_seconds_ / 4);
+      stop_latch.WaitUntil(clear_latency);
+      for (auto i = 0; i < cluster_->num_masters(); ++i) {
+        CHECK_OK(cluster_->SetFlag(cluster_->master(i),
+            "catalog_manager_inject_latency_load_ca_info_ms", "0"));
+      }
+    });
+    SCOPED_CLEANUP({
+      clear_latency_thread.join();
+    });
+    while (MonoTime::Now() < t_stop) {
+      NO_FATALS(ConnectToCluster());
+    }
+    stop_latch.CountDown();
+    NO_FATALS(cluster_->AssertNoCrashes());
+  }
+
+ protected:
+  const int run_time_seconds_;
+  const int latency_ms_;
+  ExternalMiniClusterOptions cluster_opts_;
+  std::shared_ptr<ExternalMiniCluster> cluster_;
+};
+
+// Test for KUDU-1927 in single-master configuration: verify that Kudu client
+// successfully connects to Kudu cluster and always have CA certificate and
+// authn token once connected. The test injects random latency into the process
+// of loading the CA record from the system table. There is a high chance that
+// during start-up the master server responds with ServiceUnavailable to
+// ConnectToCluster RPC sent by the client. The client should retry in that
+// case, connecting to the cluster eventually. Once successfully connected,
+// the client must have Kudu IPKI CA certificate and authn token.
+class SingleMasterConnectToClusterTest : public ConnectToClusterBaseTest {
+ public:
+  SingleMasterConnectToClusterTest()
+      : ConnectToClusterBaseTest(5, 2500, { 11020 }) {
+    // Add master-only flags.
+    cluster_opts_.extra_master_flags.push_back(Substitute(
+        "--catalog_manager_inject_latency_load_ca_info_ms=$0", latency_ms_));
+  }
+};
+
+// Test for KUDU-1927 in multi-master configuration: verify that Kudu client
+// successfully connects to Kudu cluster and always has CA certificate and
+// authn token once connected. The test injects random latency into the process
+// of loading the CA record from the system table. In addition, it uses short
+// timeouts for leader failure detection. Due to many re-election events,
+// sometimes elected master servers respond with ServiceUnavailable to
+// ConnectToCluster RPC sent by the client. The client should retry in that
+// case, connecting to the cluster eventually. Once successfully connected,
+// the client must have Kudu IPKI CA certificate and authn token.
+class MultiMasterConnectToClusterTest : public ConnectToClusterBaseTest {
+ public:
+  MultiMasterConnectToClusterTest()
+      : ConnectToClusterBaseTest(120, 2000, { 11030, 11031, 11032 }) {
+    constexpr int kHbIntervalMs = 100;
+    // Add master-only flags.
+    const vector<string> master_flags = {
+      Substitute("--catalog_manager_inject_latency_load_ca_info_ms=$0", latency_ms_),
+      "--raft_enable_pre_election=false",
+      Substitute("--leader_failure_exp_backoff_max_delta_ms=$0", kHbIntervalMs * 3),
+      "--leader_failure_max_missed_heartbeat_periods=1.0",
+      Substitute("--raft_heartbeat_interval_ms=$0", kHbIntervalMs),
+    };
+    copy(master_flags.begin(), master_flags.end(),
+         back_inserter(cluster_opts_.extra_master_flags));
+  }
+};
+
+TEST_F(SingleMasterConnectToClusterTest, ConnectToCluster) {
+  ASSERT_OK(cluster_->Start());
+  Run();
+}
+
+TEST_F(MultiMasterConnectToClusterTest, ConnectToCluster) {
+  if (!AllowSlowTests()) {
+    LOG(WARNING) << "test is skipped; set KUDU_ALLOW_SLOW_TESTS=1 to run";
+    return;
+  }
+  ASSERT_OK(cluster_->Start());
+  Run();
+}
+
+} // namespace client
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/54081ce1/src/kudu/integration-tests/security-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/security-itest.cc b/src/kudu/integration-tests/security-itest.cc
index 6fb76bf..cb1fbbc 100644
--- a/src/kudu/integration-tests/security-itest.cc
+++ b/src/kudu/integration-tests/security-itest.cc
@@ -370,25 +370,6 @@ TEST_P(AuthTokenIssuingTest, ChannelConfidentiality) {
   // so the connection is not considered a 'loopback' one.
   FLAGS_local_ip_for_outbound_sockets = "127.0.0.1";
 
-  // Current implementation of MasterServiceImpl::ConnectToMaster() allows to
-  // get a success response without proper security information in case if the
-  // master hasn't been established as a leader yet. As a temporary workaround,
-  // make sure the master is sending back the necessary info before going any
-  // further with the scenario which is sensitive to that issue.
-  //
-  // TODO(aserbin): fix the issue with MasterServiceImpl::ConnectToMaster()
-  //                and remove this ASSERT_EVENTUALLY() block.
-  ASSERT_EVENTUALLY([&] {
-    client::sp::shared_ptr<KuduClient> client;
-    ASSERT_OK(cluster_->CreateClient(nullptr, &client));
-
-    string authn_creds;
-    ASSERT_OK(client->ExportAuthenticationCredentials(&authn_creds));
-    client::AuthenticationCredentialsPB pb;
-    ASSERT_TRUE(pb.ParseFromString(authn_creds));
-    ASSERT_GE(pb.ca_cert_ders_size(), 1);
-  });
-
   // In current implementation, KuduClientBuilder calls ConnectToCluster() on
   // the newly created instance of the KuduClient.
   client::sp::shared_ptr<KuduClient> client;

http://git-wip-us.apache.org/repos/asf/kudu/blob/54081ce1/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 33fd194..ec01f18 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -219,6 +219,14 @@ DEFINE_bool(catalog_manager_fail_ts_rpcs, false,
 TAG_FLAG(catalog_manager_fail_ts_rpcs, hidden);
 TAG_FLAG(catalog_manager_fail_ts_rpcs, runtime);
 
+DEFINE_int32(catalog_manager_inject_latency_load_ca_info_ms, 0,
+             "Injects a random sleep between 0 and this many milliseconds "
+             "while reading CA info from the system table. "
+             "This is a test-only flag, do not use in production.");
+TAG_FLAG(catalog_manager_inject_latency_load_ca_info_ms, hidden);
+TAG_FLAG(catalog_manager_inject_latency_load_ca_info_ms, runtime);
+TAG_FLAG(catalog_manager_inject_latency_load_ca_info_ms, unsafe);
+
 DEFINE_int32(catalog_manager_inject_latency_prior_tsk_write_ms, 0,
              "Injects a random sleep between 0 and this many milliseconds "
              "prior to writing newly generated TSK into the system table. "
@@ -780,7 +788,7 @@ Status CatalogManager::InitCertAuthority() {
     unique_ptr<Cert> cert(new Cert);
 
     // Generate new private key and corresponding CA certificate.
-    RETURN_NOT_OK(master_->cert_authority()->Generate(key.get(), cert.get()));
+    RETURN_NOT_OK(MasterCertAuthority::Generate(key.get(), cert.get()));
     // If the leadership was lost, writing into the system table fails.
     RETURN_NOT_OK(StoreCertAuthorityInfo(*key, *cert));
     // Once the CA information is persisted, it's necessary to initialize
@@ -796,6 +804,7 @@ Status CatalogManager::InitCertAuthority() {
 // private key and certificate.
 Status CatalogManager::InitCertAuthorityWith(
     unique_ptr<PrivateKey> key, unique_ptr<Cert> cert) {
+
   leader_lock_.AssertAcquired();
 
   auto* ca = master_->cert_authority();
@@ -820,6 +829,8 @@ Status CatalogManager::LoadCertAuthorityInfo(unique_ptr<PrivateKey>* key,
                                              unique_ptr<Cert>* cert) {
   leader_lock_.AssertAcquired();
 
+  MAYBE_INJECT_RANDOM_LATENCY(FLAGS_catalog_manager_inject_latency_load_ca_info_ms);
+
   SysCertAuthorityEntryPB info;
   RETURN_NOT_OK(sys_catalog_->GetCertAuthorityEntry(&info));
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/54081ce1/src/kudu/master/master_cert_authority.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master_cert_authority.cc b/src/kudu/master/master_cert_authority.cc
index ab91283..8467e4a 100644
--- a/src/kudu/master/master_cert_authority.cc
+++ b/src/kudu/master/master_cert_authority.cc
@@ -63,16 +63,8 @@ TAG_FLAG(ipki_server_cert_expiration_seconds, experimental);
 namespace kudu {
 namespace master {
 
-MasterCertAuthority::MasterCertAuthority(string server_uuid)
-    : server_uuid_(std::move(server_uuid)) {
-}
-
-MasterCertAuthority::~MasterCertAuthority() {
-}
-
-// Generate
 Status MasterCertAuthority::Generate(security::PrivateKey* key,
-                                     security::Cert* cert) const {
+                                     security::Cert* cert) {
   CHECK(key);
   CHECK(cert);
   // Create a key and cert for the self-signed CA.
@@ -84,6 +76,13 @@ Status MasterCertAuthority::Generate(security::PrivateKey* key,
                                 cert);
 }
 
+MasterCertAuthority::MasterCertAuthority(string server_uuid)
+    : server_uuid_(std::move(server_uuid)) {
+}
+
+MasterCertAuthority::~MasterCertAuthority() {
+}
+
 Status MasterCertAuthority::Init(unique_ptr<PrivateKey> key,
                                  unique_ptr<Cert> cert) {
   CHECK(key);

http://git-wip-us.apache.org/repos/asf/kudu/blob/54081ce1/src/kudu/master/master_cert_authority.h
----------------------------------------------------------------------
diff --git a/src/kudu/master/master_cert_authority.h b/src/kudu/master/master_cert_authority.h
index a25438e..a1a279f 100644
--- a/src/kudu/master/master_cert_authority.h
+++ b/src/kudu/master/master_cert_authority.h
@@ -53,16 +53,13 @@ class MasterCertAuthorityTest;
 // This class is thread-safe after initialization.
 class MasterCertAuthority {
  public:
+  // Generate a private key and corresponding self-signed root CA certificate
+  // bound to the aggregated server UUID.
+  static Status Generate(security::PrivateKey* key, security::Cert* cert);
+
   explicit MasterCertAuthority(std::string server_uuid);
   virtual ~MasterCertAuthority();
 
-  // Generate a private key and corresponding self-signed root CA certificate
-  // bound to the aggregated server UUID. Does not require Init() to be called.
-  // Calling this method does not have side-effects on the instance, i.e.
-  // even this method has been called on object, it's still necessary
-  // to call Init() prior to calling SignServerCSR() method.
-  Status Generate(security::PrivateKey* key, security::Cert* cert) const;
-
   // Initializes the MasterCertAuthority with the given private key
   // and CA certificate. This method is called when the master server
   // is elected as a leader -- upon that event it initializes the certificate

http://git-wip-us.apache.org/repos/asf/kudu/blob/54081ce1/src/kudu/master/master_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master_service.cc b/src/kudu/master/master_service.cc
index 2e50fe9..5229448 100644
--- a/src/kudu/master/master_service.cc
+++ b/src/kudu/master/master_service.cc
@@ -30,6 +30,7 @@
 #include "kudu/common/common.pb.h"
 #include "kudu/common/wire_protocol.h"
 #include "kudu/common/wire_protocol.pb.h"
+#include "kudu/consensus/metadata.pb.h"
 #include "kudu/consensus/replica_management.pb.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/master/catalog_manager.h"
@@ -455,8 +456,6 @@ void MasterServiceImpl::ConnectToMaster(const ConnectToMasterRequestPB* /*req*/,
   if (!l.CheckIsInitializedOrRespond(resp, rpc)) {
     return;
   }
-  auto role = server_->catalog_manager()->Role();
-  resp->set_role(role);
 
   // Set the info about the other masters, so that the client can verify
   // it has the full set of info.
@@ -470,13 +469,8 @@ void MasterServiceImpl::ConnectToMaster(const ConnectToMasterRequestPB* /*req*/,
     }
   }
 
-  if (l.leader_status().ok()) {
-    // TODO(KUDU-1924): it seems there is some window when 'role' is LEADER but
-    // in fact we aren't done initializing (and we don't have a CA cert).
-    // In that case, if we respond with the 'LEADER' role to a client, but
-    // don't pass back the CA cert, then the client won't be able to trust
-    // anyone... seems like a potential race bug for clients who connect
-    // exactly as the leader is changing.
+  const bool is_leader = l.leader_status().ok();
+  if (is_leader) {
     resp->add_ca_cert_der(server_->cert_authority()->ca_cert_der());
 
     // Issue an authentication token for the caller, unless they are
@@ -485,20 +479,35 @@ void MasterServiceImpl::ConnectToMaster(const ConnectToMasterRequestPB* /*req*/,
     // client over a non-confidential channel.
     if (rpc->is_confidential() &&
         rpc->remote_user().authenticated_by() != rpc::RemoteUser::AUTHN_TOKEN) {
+      string username = rpc->remote_user().username();
+      if (username.empty()) {
+        static const char* const kErrMsg = "missing name of the remote user";
+        StatusToPB(Status::InvalidArgument(kErrMsg),
+                   resp->mutable_error()->mutable_status());
+        resp->mutable_error()->set_code(MasterErrorPB::UNKNOWN_ERROR);
+        rpc->RespondSuccess();
+        KLOG_EVERY_N_SECS(WARNING, 60) << Substitute("invalid request from $0: $1",
+                                                     rpc->requestor_string(), kErrMsg);
+        return;
+      }
+
       SignedTokenPB authn_token;
-      Status s = server_->token_signer()->GenerateAuthnToken(
-          rpc->remote_user().username(),
-          &authn_token);
+      Status s = server_->token_signer()->GenerateAuthnToken(username, &authn_token);
       if (!s.ok()) {
-        KLOG_EVERY_N_SECS(WARNING, 1)
-            << "Unable to generate signed token for " << rpc->requestor_string()
-            << ": " << s.ToString();
-      } else {
-        // TODO(todd): this might be a good spot for some auditing code?
-        resp->mutable_authn_token()->Swap(&authn_token);
+        LOG(FATAL) << Substitute("unable to generate signed token for $0: $1",
+                                 rpc->requestor_string(), s.ToString());
       }
+
+      // TODO(todd): this might be a good spot for some auditing code?
+      resp->mutable_authn_token()->Swap(&authn_token);
     }
   }
+
+  // Rather than consulting the current consensus role, instead base it
+  // on the catalog manager's view. This prevents us from advertising LEADER
+  // until we have taken over all the associated responsibilities.
+  resp->set_role(is_leader ? consensus::RaftPeerPB::LEADER
+                           : consensus::RaftPeerPB::FOLLOWER);
   rpc->RespondSuccess();
 }