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/24 19:25:27 UTC

[3/3] kudu git commit: KUDU-2265 CA-signed server certs for non-leader masters

KUDU-2265 CA-signed server certs for non-leader masters

This changelist addresses KUDU-2265.  Prior to this fix, a non-leader
master had a self-signed server TLS certificates if it hasn't ever
become a leader.

An integration test to verify the new behavior is added as well.

Change-Id: Ia3539d58d10ed319ad1d8686c1259c92822fb710
Reviewed-on: http://gerrit.cloudera.org:8080/9076
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>
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/c497c69b
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/c497c69b
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/c497c69b

Branch: refs/heads/master
Commit: c497c69bb2128a8a62738b9b6bb97a5b83da0323
Parents: e0a743d
Author: Alexey Serbin <as...@cloudera.com>
Authored: Thu Jan 18 23:52:03 2018 -0800
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Wed Jan 24 19:24:32 2018 +0000

----------------------------------------------------------------------
 src/kudu/integration-tests/CMakeLists.txt       |   1 +
 .../security-master-certificates-itest.cc       | 100 +++++++++++++++++++
 src/kudu/master/catalog_manager.cc              |  46 +++++++--
 src/kudu/master/catalog_manager.h               |  30 +++++-
 src/kudu/security/tls_context.cc                |  10 +-
 src/kudu/util/rw_mutex.cc                       |   6 ++
 src/kudu/util/rw_mutex.h                        |   2 +
 7 files changed, 177 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/c497c69b/src/kudu/integration-tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/CMakeLists.txt b/src/kudu/integration-tests/CMakeLists.txt
index 1a10a7d..a6a6938 100644
--- a/src/kudu/integration-tests/CMakeLists.txt
+++ b/src/kudu/integration-tests/CMakeLists.txt
@@ -94,6 +94,7 @@ 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-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/c497c69b/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
new file mode 100644
index 0000000..b521d65
--- /dev/null
+++ b/src/kudu/integration-tests/security-master-certificates-itest.cc
@@ -0,0 +1,100 @@
+// 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/c497c69b/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 1248144..33fd194 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -536,6 +536,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();
+        if (!s.ok()) {
+          LOG(WARNING) << s.ToString()
+                       << ": failed to prepare follower catalog manager, will retry";
+        }
       }
     }
     // Wait for a notification or a timeout expiration.
@@ -788,17 +796,16 @@ Status CatalogManager::InitCertAuthority() {
 // private key and certificate.
 Status CatalogManager::InitCertAuthorityWith(
     unique_ptr<PrivateKey> key, unique_ptr<Cert> cert) {
-  leader_lock_.AssertAcquiredForWriting();
+  leader_lock_.AssertAcquired();
+
   auto* ca = master_->cert_authority();
   RETURN_NOT_OK_PREPEND(ca->Init(std::move(key), std::move(cert)),
                         "could not init master CA");
-
   auto* tls = master_->mutable_tls_context();
   RETURN_NOT_OK_PREPEND(tls->AddTrustedCertificate(ca->ca_cert()),
                         "could not trust master CA cert");
   // If we haven't signed our own server cert yet, do so.
-  boost::optional<security::CertSignRequest> csr =
-      tls->GetCsrIfNecessary();
+  boost::optional<security::CertSignRequest> csr = tls->GetCsrIfNecessary();
   if (csr) {
     Cert cert;
     RETURN_NOT_OK_PREPEND(ca->SignServerCSR(*csr, &cert),
@@ -811,7 +818,7 @@ Status CatalogManager::InitCertAuthorityWith(
 
 Status CatalogManager::LoadCertAuthorityInfo(unique_ptr<PrivateKey>* key,
                                              unique_ptr<Cert>* cert) {
-  leader_lock_.AssertAcquiredForWriting();
+  leader_lock_.AssertAcquired();
 
   SysCertAuthorityEntryPB info;
   RETURN_NOT_OK(sys_catalog_->GetCertAuthorityEntry(&info));
@@ -975,6 +982,29 @@ void CatalogManager::PrepareForLeadershipTask() {
   leader_ready_term_ = term;
 }
 
+bool CatalogManager::NeedToPrepareFollower() {
+  return !master_->tls_context().has_signed_cert();
+}
+
+Status CatalogManager::PrepareFollower() {
+  static const char* const kDescription =
+      "acquiring CA information for follower catalog manager";
+
+  leader_lock_.AssertAcquiredForReading();
+
+  unique_ptr<PrivateKey> key;
+  unique_ptr<Cert> cert;
+  Status s = LoadCertAuthorityInfo(&key, &cert).AndThen([&] {
+    return InitCertAuthorityWith(std::move(key), std::move(cert));
+  });
+  if (s.ok()) {
+    LOG_WITH_PREFIX(INFO) << kDescription << ": success";
+  } else {
+    LOG_WITH_PREFIX(WARNING) << kDescription << ": " << s.ToString();
+  }
+  return s;
+}
+
 Status CatalogManager::VisitTablesAndTabletsUnlocked() {
   leader_lock_.AssertAcquiredForWriting();
 
@@ -1013,10 +1043,8 @@ Status CatalogManager::VisitTablesAndTablets() {
 
 Status CatalogManager::InitSysCatalogAsync(bool is_first_run) {
   std::lock_guard<LockType> l(lock_);
-  unique_ptr<SysCatalogTable> new_catalog(
-      new SysCatalogTable(master_,
-                          Bind(&CatalogManager::ElectedAsLeaderCb,
-                               Unretained(this))));
+  unique_ptr<SysCatalogTable> new_catalog(new SysCatalogTable(
+      master_, Bind(&CatalogManager::ElectedAsLeaderCb, Unretained(this))));
   if (is_first_run) {
     RETURN_NOT_OK(new_catalog->CreateNew(master_->fs_manager()));
   } else {

http://git-wip-us.apache.org/repos/asf/kudu/blob/c497c69b/src/kudu/master/catalog_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.h b/src/kudu/master/catalog_manager.h
index 9261cf0..720e6ea 100644
--- a/src/kudu/master/catalog_manager.h
+++ b/src/kudu/master/catalog_manager.h
@@ -412,11 +412,18 @@ class CatalogManager : public tserver::TabletReplicaLookupIf {
   //
   class ScopedLeaderSharedLock {
    public:
-    // Creates a new shared lock, acquiring the catalog manager's leader_lock_
-    // for reading in the process. The lock is released when this object is
-    // destroyed.
+    // Creates a new shared lock, trying to acquire the catalog manager's
+    // leader_lock_ for reading in the process. If acquired, the lock is
+    // released when this object is destroyed.
     //
-    // 'catalog' must outlive this object.
+    // In most common use cases, where write lock semantics is assumed, call
+    // CheckIsInitializedAndIsLeaderOrRespond() to verify that the leader_lock_
+    // has been acquired (as shown in the class-wide comment above). In rare
+    // cases, where both read and write semantics are applicable, use the
+    // combination of CheckIsInitializedOrRespond() and owns_lock() methods
+    // to verify that the leader_lock_ is acquired.
+    //
+    // The object pointed by the 'catalog' parameter must outlive this object.
     explicit ScopedLeaderSharedLock(CatalogManager* catalog);
 
     // General status of the catalog manager. If not OK (e.g. the catalog
@@ -440,6 +447,11 @@ class CatalogManager : public tserver::TabletReplicaLookupIf {
       return leader_status_;
     }
 
+    // Whether the underlying leader lock of the system catalog is acquired.
+    bool owns_lock() const {
+      return leader_shared_lock_.owns_lock();
+    }
+
     // Check whether the consensus configuration term has changed from the term
     // captured at object construction (initial_term_).
     // Requires: leader_status() returns OK().
@@ -660,6 +672,16 @@ 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();
+
   // Clears out the existing metadata ('table_names_map_', 'table_ids_map_',
   // and 'tablet_map_'), loads tables metadata into memory and if successful
   // loads the tablets metadata.

http://git-wip-us.apache.org/repos/asf/kudu/blob/c497c69b/src/kudu/security/tls_context.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_context.cc b/src/kudu/security/tls_context.cc
index e43522c..8e687f3 100644
--- a/src/kudu/security/tls_context.cc
+++ b/src/kudu/security/tls_context.cc
@@ -394,16 +394,16 @@ Status TlsContext::AdoptSignedCert(const Cert& cert) {
   SCOPED_OPENSSL_NO_PENDING_ERRORS;
   unique_lock<RWMutex> lock(lock_);
 
-  // Verify that the appropriate CA certs have been loaded into the context
-  // before we adopt a cert. Otherwise, client connections without the CA cert
-  // available would fail.
-  RETURN_NOT_OK(VerifyCertChainUnlocked(cert));
-
   if (!csr_) {
     // A signed cert has already been adopted.
     return Status::OK();
   }
 
+  // Verify that the appropriate CA certs have been loaded into the context
+  // before we adopt a cert. Otherwise, client connections without the CA cert
+  // available would fail.
+  RETURN_NOT_OK(VerifyCertChainUnlocked(cert));
+
   PublicKey csr_key;
   RETURN_NOT_OK(csr_->GetPublicKey(&csr_key));
   PublicKey cert_key;

http://git-wip-us.apache.org/repos/asf/kudu/blob/c497c69b/src/kudu/util/rw_mutex.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/rw_mutex.cc b/src/kudu/util/rw_mutex.cc
index 59139af..75fc1e7 100644
--- a/src/kudu/util/rw_mutex.cc
+++ b/src/kudu/util/rw_mutex.cc
@@ -140,6 +140,12 @@ bool RWMutex::TryWriteLock() {
 
 #ifndef NDEBUG
 
+void RWMutex::AssertAcquired() const {
+  lock_guard<simple_spinlock> l(tid_lock_);
+  CHECK(ContainsKey(reader_tids_, Env::Default()->gettid()) ||
+        Env::Default()->gettid() == writer_tid_);
+}
+
 void RWMutex::AssertAcquiredForReading() const {
   lock_guard<simple_spinlock> l(tid_lock_);
   CHECK(ContainsKey(reader_tids_, Env::Default()->gettid()));

http://git-wip-us.apache.org/repos/asf/kudu/blob/c497c69b/src/kudu/util/rw_mutex.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/rw_mutex.h b/src/kudu/util/rw_mutex.h
index 84f77d2..bb6168c 100644
--- a/src/kudu/util/rw_mutex.h
+++ b/src/kudu/util/rw_mutex.h
@@ -65,9 +65,11 @@ class RWMutex {
   bool TryWriteLock();
 
 #ifndef NDEBUG
+  void AssertAcquired() const;
   void AssertAcquiredForReading() const;
   void AssertAcquiredForWriting() const;
 #else
+  void AssertAcquired() const {}
   void AssertAcquiredForReading() const {}
   void AssertAcquiredForWriting() const {}
 #endif