Repository: kudu
Updated Branches:
  refs/heads/master e9448dac5 -> 7e54a17ec


KUDU-2319 follower masters should be able to verify authn tokens

This patch addresses the issue that follower masters could not accept
authn tokens to authenticate clients because they didn't have keys for
token signature verification.

Added a couple of tests:
  * SecurityMasterAuthTest::FollowerTokenVerificationKeys
  * ConnectToFollowerMasterTest::AuthnTokenVerifierHaveKeys

The former is more a unit test; the latter is an integration test.

Change-Id: Idcc92dd4fae3d555af563d86634c07d3d06147a7
Reviewed-on: http://gerrit.cloudera.org:8080/9373
Reviewed-by: Dan Burkert <danburk...@apache.org>
Tested-by: Alexey Serbin <aser...@cloudera.com>


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

Branch: refs/heads/master
Commit: 7e54a17ec63f3d2a01c21aa53fc8f94adf7f1909
Parents: e9448da
Author: Alexey Serbin <aser...@cloudera.com>
Authored: Tue Feb 20 15:51:15 2018 -0800
Committer: Alexey Serbin <aser...@cloudera.com>
Committed: Fri Mar 2 20:22:34 2018 +0000

----------------------------------------------------------------------
 src/kudu/integration-tests/CMakeLists.txt       |   4 +-
 src/kudu/integration-tests/security-itest.cc    | 112 +++++++++++++++++
 .../security-master-auth-itest.cc               | 124 +++++++++++++++++++
 .../security-master-certificates-itest.cc       | 100 ---------------
 src/kudu/master/catalog_manager.cc              |  85 +++++++++++--
 src/kudu/master/catalog_manager.h               |  26 ++--
 6 files changed, 330 insertions(+), 121 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/7e54a17e/src/kudu/integration-tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/CMakeLists.txt 
b/src/kudu/integration-tests/CMakeLists.txt
index c3922eb..90454a4 100644
--- a/src/kudu/integration-tests/CMakeLists.txt
+++ b/src/kudu/integration-tests/CMakeLists.txt
@@ -95,8 +95,8 @@ ADD_KUDU_TEST(raft_consensus_stress-itest)
 ADD_KUDU_TEST(raft_consensus-itest RUN_SERIAL true)
 ADD_KUDU_TEST(registration-test RESOURCE_LOCK "master-web-port")
 ADD_KUDU_TEST(security-faults-itest)
-ADD_KUDU_TEST(security-itest)
-ADD_KUDU_TEST(security-master-certificates-itest RESOURCE_LOCK 
"master-rpc-ports")
+ADD_KUDU_TEST(security-itest RESOURCE_LOCK "master-rpc-ports")
+ADD_KUDU_TEST(security-master-auth-itest RESOURCE_LOCK "master-rpc-ports")
 ADD_KUDU_TEST(security-unknown-tsk-itest)
 ADD_KUDU_TEST(stop_tablet-itest)
 ADD_KUDU_TEST(table_locations-itest)

http://git-wip-us.apache.org/repos/asf/kudu/blob/7e54a17e/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 cb1fbbc..0d12177 100644
--- a/src/kudu/integration-tests/security-itest.cc
+++ b/src/kudu/integration-tests/security-itest.cc
@@ -34,10 +34,13 @@
 #include "kudu/client/write_op.h"
 #include "kudu/common/partial_row.h"
 #include "kudu/common/wire_protocol.pb.h"
+#include "kudu/consensus/consensus.pb.h"
+#include "kudu/consensus/consensus.proxy.h"
 #include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/master/master.pb.h"
 #include "kudu/master/master.proxy.h"
+#include "kudu/master/sys_catalog.h"
 #include "kudu/mini-cluster/external_mini_cluster.h"
 #include "kudu/mini-cluster/mini_cluster.h"
 #include "kudu/rpc/messenger.h"
@@ -59,6 +62,7 @@
 DECLARE_string(local_ip_for_outbound_sockets);
 
 using kudu::client::KuduClient;
+using kudu::client::KuduClientBuilder;
 using kudu::client::KuduInsert;
 using kudu::client::KuduSchema;
 using kudu::client::KuduSession;
@@ -390,4 +394,112 @@ TEST_P(AuthTokenIssuingTest, ChannelConfidentiality) {
   }
 }
 
+struct ConnectToFollowerMasterTestParams {
+  const string rpc_authentication;
+  const string rpc_encryption;
+};
+class ConnectToFollowerMasterTest :
+    public SecurityITest,
+    public ::testing::WithParamInterface<ConnectToFollowerMasterTestParams> {
+};
+INSTANTIATE_TEST_CASE_P(, ConnectToFollowerMasterTest, ::testing::ValuesIn(
+    vector<ConnectToFollowerMasterTestParams>{
+      { "required", "optional", },
+      { "required", "required", },
+    }
+));
+
+// Test that a client can authenticate against a follower master using
+// authn token. For that, the token verifier at follower masters should have
+// appropriate keys for authn token signature verification.
+TEST_P(ConnectToFollowerMasterTest, AuthnTokenVerifierHaveKeys) {
+  // Want to have control over the master leadership.
+  cluster_opts_.extra_master_flags.emplace_back(
+      "--enable_leader_failure_detection=false");
+  cluster_opts_.num_masters = 3;
+  cluster_opts_.master_rpc_ports = { 11010, 11011, 11012, };
+  const auto& params = GetParam();
+  cluster_opts_.extra_master_flags.emplace_back(
+      Substitute("--rpc_authentication=$0", params.rpc_authentication));
+  cluster_opts_.extra_tserver_flags.emplace_back(
+      Substitute("--rpc_authentication=$0", params.rpc_authentication));
+  cluster_opts_.extra_master_flags.emplace_back(
+      Substitute("--rpc_encryption=$0", params.rpc_encryption));
+  cluster_opts_.extra_tserver_flags.emplace_back(
+      Substitute("--rpc_encryption=$0", params.rpc_encryption));
+  ASSERT_OK(StartCluster());
+
+  const auto& master_host = cluster_->master(0)->bound_rpc_addr().host();
+  {
+    consensus::ConsensusServiceProxy proxy(
+        cluster_->messenger(), cluster_->master(0)->bound_rpc_addr(), 
master_host);
+    consensus::RunLeaderElectionRequestPB req;
+    consensus::RunLeaderElectionResponsePB resp;
+    rpc::RpcController rpc;
+    req.set_tablet_id(master::SysCatalogTable::kSysCatalogTabletId);
+    req.set_dest_uuid(cluster_->master(0)->uuid());
+    rpc.set_timeout(MonoDelta::FromSeconds(10));
+    ASSERT_OK(proxy.RunLeaderElection(req, &resp, &rpc));
+  }
+
+  // Get authentication credentials.
+  string authn_creds;
+  {
+    client::sp::shared_ptr<KuduClient> client;
+    ASSERT_OK(cluster_->CreateClient(nullptr, &client));
+    ASSERT_OK(client->ExportAuthenticationCredentials(&authn_creds));
+  }
+
+  ASSERT_OK(cluster_->kdc()->Kdestroy());
+  // At this point the primary credentials (i.e. Kerberos) are not available.
+
+  // Make sure it's not possible to connect without authn token at this point:
+  // the server side is configured to require authentication.
+  {
+    client::sp::shared_ptr<KuduClient> client;
+    KuduClientBuilder builder;
+    for (auto i = 0; i < cluster_->num_masters(); ++i) {
+      
builder.add_master_server_addr(cluster_->master(i)->bound_rpc_addr().ToString());
+    }
+    const auto s = builder.Build(&client);
+    ASSERT_TRUE(s.IsNotAuthorized()) << s.ToString();
+  }
+
+  // Try to connect to every available follower master, authenticating using
+  // only the authn token. This scenario uses some sort of 'negative' criterion
+  // based on the fact that it's not possible to receive 'configuration error'
+  // status without being sucessfully authenticated first. The configuration
+  // error is returned because the client uses only a single master in its list
+  // of masters' endpoints while trying to connect to a multi-master Kudu 
cluster.
+  ASSERT_EVENTUALLY([&] {
+    for (auto i = 1; i < cluster_->num_masters(); ++i) {
+      client::sp::shared_ptr<KuduClient> client;
+      const auto s = KuduClientBuilder()
+          
.add_master_server_addr(cluster_->master(i)->bound_rpc_addr().ToString())
+          .import_authentication_credentials(authn_creds)
+          .Build(&client);
+      ASSERT_TRUE(s.IsConfigurationError()) << s.ToString();
+      ASSERT_STR_MATCHES(s.ToString(),
+                         ".* Client configured with 1 master.* "
+                         "but cluster indicates it expects 3 master.*");
+    }
+  });
+
+  // Although it's not in the exact scope of this test, make sure it's possible
+  // to connect and perform basic operations (like listing tables) when using
+  // secondary credentials only (i.e. authn token).
+  {
+    client::sp::shared_ptr<KuduClient> client;
+    KuduClientBuilder builder;
+    for (auto i = 0; i < cluster_->num_masters(); ++i) {
+      
builder.add_master_server_addr(cluster_->master(i)->bound_rpc_addr().ToString());
+    }
+    builder.import_authentication_credentials(authn_creds);
+    ASSERT_OK(builder.Build(&client));
+
+    vector<string> tables;
+    ASSERT_OK(client->ListTables(&tables));
+  }
+}
+
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/7e54a17e/src/kudu/integration-tests/security-master-auth-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/security-master-auth-itest.cc 
b/src/kudu/integration-tests/security-master-auth-itest.cc
new file mode 100644
index 0000000..c8eca38
--- /dev/null
+++ b/src/kudu/integration-tests/security-master-auth-itest.cc
@@ -0,0 +1,124 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <memory>
+#include <vector>
+
+#include <gflags/gflags_declare.h>
+#include <gtest/gtest.h>
+
+#include "kudu/consensus/raft_consensus.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/master/catalog_manager.h"
+#include "kudu/master/master.h"
+#include "kudu/master/mini_master.h"
+#include "kudu/master/sys_catalog.h"
+#include "kudu/mini-cluster/internal_mini_cluster.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/security/tls_context.h"
+#include "kudu/security/token_verifier.h"
+#include "kudu/tablet/tablet_replica.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+DECLARE_bool(enable_leader_failure_detection);
+
+using kudu::cluster::InternalMiniCluster;
+using kudu::cluster::InternalMiniClusterOptions;
+using kudu::consensus::RaftConsensus;
+
+using std::unique_ptr;
+
+
+namespace kudu {
+
+class SecurityMasterAuthTest : public KuduTest {
+ public:
+  void SetUp() override {
+    KuduTest::SetUp();
+
+    // This test requires manual system catalog leader election.
+    FLAGS_enable_leader_failure_detection = false;
+
+    InternalMiniClusterOptions opts;
+    opts.master_rpc_ports = { 11010, 11011, 11012, 11013, 11014, };
+    opts.num_masters = opts.master_rpc_ports.size();
+    opts.num_tablet_servers = 0;
+    cluster_.reset(new InternalMiniCluster(env_, opts));
+    ASSERT_OK(cluster_->Start());
+  }
+
+  void TearDown() override {
+    cluster_->Shutdown();
+  }
+
+ protected:
+  unique_ptr<InternalMiniCluster> cluster_;
+};
+
+// This scenario verifies that follower masters get CA-signed certificates
+// even if they haven't run in the leader role yet. In this particular 
scenario,
+// only one of the masters has ever become a leader and the rest have always
+// been followers. This is a test to cover regressions of KUDU-2265, if any.
+TEST_F(SecurityMasterAuthTest, FollowerCertificates) {
+  for (auto i = 0; i < cluster_->num_masters(); ++i) {
+    const auto& tls = cluster_->mini_master(i)->master()->tls_context();
+    // Initially, all master servers have self-signed certs,
+    // but none has CA-signed cert.
+    ASSERT_FALSE(tls.has_signed_cert());
+    ASSERT_TRUE(tls.has_cert());
+  }
+
+  auto* consensus = cluster_->mini_master(0)->master()->catalog_manager()->
+      sys_catalog()->tablet_replica()->consensus();
+  ASSERT_OK(consensus->StartElection(
+      RaftConsensus::ELECT_EVEN_IF_LEADER_IS_ALIVE,
+      RaftConsensus::EXTERNAL_REQUEST));
+
+  // After some time, all masters should have CA-signed certs.
+  ASSERT_EVENTUALLY([&] {
+    for (auto i = 0; i < cluster_->num_masters(); ++i) {
+      const auto& tls = cluster_->mini_master(i)->master()->tls_context();
+      ASSERT_TRUE(tls.has_signed_cert());
+    }
+  });
+}
+
+// This scenario verifies that follower masters get keys for authn token
+// verification even if they haven't run in the leader role yet. In this
+// particular scenario, only one of the masters has ever become a leader and
+// the rest have always been followers. This is a test to cover regressions of
+// KUDU-2319, if any.
+TEST_F(SecurityMasterAuthTest, FollowerTokenVerificationKeys) {
+  auto* consensus = cluster_->mini_master(0)->master()->catalog_manager()->
+      sys_catalog()->tablet_replica()->consensus();
+  ASSERT_OK(consensus->StartElection(
+      RaftConsensus::ELECT_EVEN_IF_LEADER_IS_ALIVE,
+      RaftConsensus::EXTERNAL_REQUEST));
+
+  // After some time, all masters should have keys for token verification.
+  ASSERT_EVENTUALLY([&] {
+    for (auto i = 0; i < cluster_->num_masters(); ++i) {
+      const auto& verifier = cluster_->mini_master(i)->master()->messenger()->
+          token_verifier();
+      ASSERT_LE(0, verifier.GetMaxKnownKeySequenceNumber());
+    }
+  });
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/7e54a17e/src/kudu/integration-tests/security-master-certificates-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/security-master-certificates-itest.cc 
b/src/kudu/integration-tests/security-master-certificates-itest.cc
deleted file mode 100644
index b521d65..0000000
--- a/src/kudu/integration-tests/security-master-certificates-itest.cc
+++ /dev/null
@@ -1,100 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-#include <memory>
-#include <vector>
-
-#include <gflags/gflags_declare.h>
-#include <gtest/gtest.h>
-
-#include "kudu/consensus/raft_consensus.h"
-#include "kudu/gutil/ref_counted.h"
-#include "kudu/master/catalog_manager.h"
-#include "kudu/master/master.h"
-#include "kudu/master/mini_master.h"
-#include "kudu/master/sys_catalog.h"
-#include "kudu/mini-cluster/internal_mini_cluster.h"
-#include "kudu/security/tls_context.h"
-#include "kudu/tablet/tablet_replica.h"
-#include "kudu/util/status.h"
-#include "kudu/util/test_macros.h"
-#include "kudu/util/test_util.h"
-
-DECLARE_bool(enable_leader_failure_detection);
-
-using kudu::cluster::InternalMiniCluster;
-using kudu::cluster::InternalMiniClusterOptions;
-using kudu::consensus::RaftConsensus;
-
-using std::unique_ptr;
-
-
-namespace kudu {
-
-class SecurityMasterCertsTest : public KuduTest {
- public:
-  void SetUp() override {
-    KuduTest::SetUp();
-
-    // This test requires manual system catalog leader election.
-    FLAGS_enable_leader_failure_detection = false;
-
-    InternalMiniClusterOptions opts;
-    opts.master_rpc_ports = { 11010, 11011, 11012, 11013, 11014, };
-    opts.num_masters = opts.master_rpc_ports.size();
-    opts.num_tablet_servers = 0;
-    cluster_.reset(new InternalMiniCluster(env_, opts));
-    ASSERT_OK(cluster_->Start());
-  }
-
-  void TearDown() override {
-    cluster_->Shutdown();
-  }
-
- protected:
-  unique_ptr<InternalMiniCluster> cluster_;
-};
-
-// This scenario verifies that follower masters get CA-signed certificates
-// even if they haven't run in the leader role yet. In this particular 
scenario,
-// only one of the masters has ever become a leader and the rest have always
-// been followers. This is a test to cover regressions of KUDU-2265, if any.
-TEST_F(SecurityMasterCertsTest, FollowerCertificates) {
-  for (auto i = 0; i < cluster_->num_masters(); ++i) {
-    const auto& tls = cluster_->mini_master(i)->master()->tls_context();
-    // Initially, all master servers have self-signed certs,
-    // but none has CA-signed cert.
-    ASSERT_FALSE(tls.has_signed_cert());
-    ASSERT_TRUE(tls.has_cert());
-  }
-
-  auto* consensus = cluster_->mini_master(0)->master()->catalog_manager()->
-      sys_catalog()->tablet_replica()->consensus();
-  ASSERT_OK(consensus->StartElection(
-      RaftConsensus::ELECT_EVEN_IF_LEADER_IS_ALIVE,
-      RaftConsensus::EXTERNAL_REQUEST));
-
-  // After some time, all master servers should have CA-signed certs.
-  ASSERT_EVENTUALLY([&] {
-    for (auto i = 0; i < cluster_->num_masters(); ++i) {
-      const auto& tls = cluster_->mini_master(i)->master()->tls_context();
-      ASSERT_TRUE(tls.has_signed_cert());
-    }
-  });
-}
-
-} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/7e54a17e/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc 
b/src/kudu/master/catalog_manager.cc
index 01f5b0e..974927e 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -107,6 +107,7 @@
 #include "kudu/security/token.pb.h"
 #include "kudu/security/token_signer.h"
 #include "kudu/security/token_signing_key.h"
+#include "kudu/security/token_verifier.h"
 #include "kudu/server/monitored_task.h"
 #include "kudu/tablet/metadata.pb.h"
 #include "kudu/tablet/tablet_replica.h"
@@ -242,6 +243,7 @@ TAG_FLAG(catalog_manager_evict_excess_replicas, runtime);
 
 DECLARE_bool(raft_prepare_replacement_before_eviction);
 DECLARE_bool(raft_attempt_to_replace_replica_without_majority);
+DECLARE_int64(tsk_rotation_seconds);
 
 using base::subtle::NoBarrier_CompareAndSwap;
 using base::subtle::NoBarrier_Load;
@@ -265,6 +267,7 @@ using kudu::security::PrivateKey;
 using kudu::security::TokenSigner;
 using kudu::security::TokenSigningPrivateKey;
 using kudu::security::TokenSigningPrivateKeyPB;
+using kudu::security::TokenSigningPublicKeyPB;
 using kudu::tablet::TABLET_DATA_DELETED;
 using kudu::tablet::TABLET_DATA_TOMBSTONED;
 using kudu::tablet::TabletDataState;
@@ -490,6 +493,7 @@ void CatalogManagerBgTasks::Shutdown() {
 }
 
 void CatalogManagerBgTasks::Run() {
+  MonoTime last_tspk_run;
   while (!NoBarrier_Load(&closing_)) {
     {
       CatalogManager::ScopedLeaderSharedLock l(catalog_manager_);
@@ -544,10 +548,14 @@ void CatalogManagerBgTasks::Run() {
             LOG(FATAL) << err_msg;
           }
         }
-      } else if (catalog_manager_->NeedToPrepareFollower() && l.owns_lock()) {
-        // This is the case of a non-leader catalog manager that has some work
-        // to do in a preparation to run in its current role.
-        Status s = catalog_manager_->PrepareFollower();
+      } else if (l.owns_lock()) {
+        // This is the case of a follower catalog manager running as a part
+        // of master process. To be able to authenticate connecting clients
+        // using their authn tokens, a follower master needs:
+        //  * CA-signed server certificate to authenticate itself to a
+        //    connecting client (otherwise the client wont try to use its 
token)
+        //  * public parts of active TSK keys to verify token signature
+        Status s = catalog_manager_->PrepareFollower(&last_tspk_run);
         if (!s.ok()) {
           LOG(WARNING) << s.ToString()
                        << ": failed to prepare follower catalog manager, will 
retry";
@@ -994,16 +1002,11 @@ void CatalogManager::PrepareForLeadershipTask() {
   leader_ready_term_ = term;
 }
 
-bool CatalogManager::NeedToPrepareFollower() {
-  return !master_->tls_context().has_signed_cert();
-}
-
-Status CatalogManager::PrepareFollower() {
+Status CatalogManager::PrepareFollowerCaInfo() {
   static const char* const kDescription =
       "acquiring CA information for follower catalog manager";
 
-  leader_lock_.AssertAcquiredForReading();
-
+  // Load the CA certificate and CA private key.
   unique_ptr<PrivateKey> key;
   unique_ptr<Cert> cert;
   Status s = LoadCertAuthorityInfo(&key, &cert).AndThen([&] {
@@ -1017,6 +1020,51 @@ Status CatalogManager::PrepareFollower() {
   return s;
 }
 
+Status CatalogManager::PrepareFollowerTokenVerifier() {
+  static const char* const kDescription =
+      "importing token verification keys for follower catalog manager";
+
+  // Load public parts of the existing TSKs.
+  vector<TokenSigningPublicKeyPB> keys;
+  const Status s = LoadTspkEntries(&keys).AndThen([&] {
+    return master_->messenger()->shared_token_verifier()->ImportKeys(keys);
+  });
+  if (!s.ok()) {
+    LOG_WITH_PREFIX(WARNING) << kDescription << ": " << s.ToString();
+    return s;
+  }
+
+  if (keys.empty()) {
+    // In case if no keys are found in the system table it's necessary to 
retry.
+    // Returning non-OK will lead the upper-level logic to call this method
+    // again as soon as possible.
+    return Status::NotFound("no TSK found in the system table");
+  }
+
+  LOG_WITH_PREFIX(INFO) << kDescription
+                        << ": success; most recent TSK sequence number "
+                        << keys.back().key_seq_num();
+  return Status::OK();
+}
+
+Status CatalogManager::PrepareFollower(MonoTime* last_tspk_run) {
+  leader_lock_.AssertAcquiredForReading();
+  // Load the CA certificate and CA private key.
+  if (!master_->tls_context().has_signed_cert()) {
+    RETURN_NOT_OK(PrepareFollowerCaInfo());
+  }
+  // Import keys for authn token verification. A new TSK appear every
+  // tsk_rotation_seconds, so using 1/2 of that interval to avoid edge cases.
+  const auto tsk_rotation_interval =
+      MonoDelta::FromSeconds(FLAGS_tsk_rotation_seconds / 2);
+  const auto now = MonoTime::Now();
+  if (!last_tspk_run->Initialized() || *last_tspk_run + tsk_rotation_interval 
< now) {
+    RETURN_NOT_OK(PrepareFollowerTokenVerifier());
+    *last_tspk_run = now;
+  }
+  return Status::OK();
+}
+
 Status CatalogManager::VisitTablesAndTabletsUnlocked() {
   leader_lock_.AssertAcquiredForWriting();
 
@@ -3826,6 +3874,21 @@ Status CatalogManager::LoadTskEntries(set<string>* 
expired_entry_ids) {
   return master_->token_signer()->ImportKeys(loader.entries());
 }
 
+Status CatalogManager::LoadTspkEntries(vector<TokenSigningPublicKeyPB>* keys) {
+  TskEntryLoader loader;
+  RETURN_NOT_OK(sys_catalog_->VisitTskEntries(&loader));
+  for (const auto& private_key : loader.entries()) {
+    // Extract public parts of the loaded keys for the verifier.
+    TokenSigningPrivateKey tsk(private_key);
+    TokenSigningPublicKeyPB key;
+    tsk.ExportPublicKeyPB(&key);
+    auto key_seq_num = key.key_seq_num();
+    keys->emplace_back(std::move(key));
+    VLOG(2) << "read public part of TSK " << key_seq_num;
+  }
+  return Status::OK();
+}
+
 Status CatalogManager::DeleteTskEntries(const set<string>& entry_ids) {
   leader_lock_.AssertAcquiredForWriting();
   return sys_catalog_->RemoveTskEntries(entry_ids);

http://git-wip-us.apache.org/repos/asf/kudu/blob/7e54a17e/src/kudu/master/catalog_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.h 
b/src/kudu/master/catalog_manager.h
index 720e6ea..0ae32ff 100644
--- a/src/kudu/master/catalog_manager.h
+++ b/src/kudu/master/catalog_manager.h
@@ -73,6 +73,7 @@ class RpcContext;
 namespace security {
 class Cert;
 class PrivateKey;
+class TokenSigningPublicKeyPB;
 } // namespace security
 
 namespace consensus {
@@ -672,15 +673,20 @@ class CatalogManager : public 
tserver::TabletReplicaLookupIf {
   // internal state of this object upon becoming the leader.
   void PrepareForLeadershipTask();
 
-  // Whether this catalog manager needs to prepare for running in the follower
-  // role.
-  bool NeedToPrepareFollower();
-
   // Perform necessary work to prepare for running in the follower role.
-  // Currently, this includes reading the CA information from the system table,
-  // creating TLS server certificate request, signing it with the CA key,
-  // and installing the certificate TLS server certificates.
-  Status PrepareFollower();
+  // Currently, it's about having a means to authenticate clients by authn 
tokens.
+  Status PrepareFollower(MonoTime* last_tspk_run);
+
+  // Prepare CA-related information for the follower catalog manager. 
Currently,
+  // this includes reading the CA information from the system table, creating
+  // TLS server certificate request, signing it with the CA key, and installing
+  // the certificate TLS server certificates.
+  Status PrepareFollowerCaInfo();
+
+  // Read currently active TSK keys from the system table and import their
+  // public parts into the token verifier, so it's possible to verify 
signatures
+  // of authn tokens.
+  Status PrepareFollowerTokenVerifier();
 
   // Clears out the existing metadata ('table_names_map_', 'table_ids_map_',
   // and 'tablet_map_'), loads tables metadata into memory and if successful
@@ -767,6 +773,10 @@ class CatalogManager : public 
tserver::TabletReplicaLookupIf {
   // Once done, initialize TokenSigner with the loaded entries.
   Status LoadTskEntries(std::set<std::string>* expired_entry_ids);
 
+  // Load non-expired TSK entries from the system table, extract the public
+  // part from those, and return them with the 'key' output parameter.
+  Status LoadTspkEntries(std::vector<security::TokenSigningPublicKeyPB>* keys);
+
   // Delete TSK entries with the specified entry identifiers
   // (identifiers correspond to the 'entry_id' column).
   Status DeleteTskEntries(const std::set<std::string>& entry_ids);

Reply via email to