You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by to...@apache.org on 2017/02/01 02:54:15 UTC

[1/3] kudu git commit: [security] introduced crypto-test

Repository: kudu
Updated Branches:
  refs/heads/master d91313d61 -> e67289edf


[security] introduced crypto-test

The new crypto-test module is for test scenarios involving common
non-CA crypto entities in the Kudu security library.  The CA-related
scenarios are kept in the cert_management-test.

Change-Id: Ibace36b6336b650879cf3f9139ccf562f82765a7
Reviewed-on: http://gerrit.cloudera.org:8080/5798
Reviewed-by: Dan Burkert <da...@apache.org>
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/9136c8ad
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/9136c8ad
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/9136c8ad

Branch: refs/heads/master
Commit: 9136c8ad61d3751b8569029e0ad6eaa0a4154db1
Parents: d91313d
Author: Alexey Serbin <as...@cloudera.com>
Authored: Wed Jan 25 19:42:22 2017 -0800
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Wed Feb 1 01:15:33 2017 +0000

----------------------------------------------------------------------
 src/kudu/security/CMakeLists.txt             |   1 +
 src/kudu/security/ca/cert_management-test.cc |  68 +-------
 src/kudu/security/crypto-test.cc             | 188 ++++++++++++++++++++++
 src/kudu/security/test/test_certs.cc         |   2 -
 src/kudu/security/test/test_certs.h          |   3 -
 5 files changed, 192 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/9136c8ad/src/kudu/security/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/security/CMakeLists.txt b/src/kudu/security/CMakeLists.txt
index 6b48d27..2627b77 100644
--- a/src/kudu/security/CMakeLists.txt
+++ b/src/kudu/security/CMakeLists.txt
@@ -106,6 +106,7 @@ if (NOT NO_TESTS)
     ${KUDU_MIN_TEST_LIBS})
 
   ADD_KUDU_TEST(ca/cert_management-test)
+  ADD_KUDU_TEST(crypto-test)
   ADD_KUDU_TEST(test/mini_kdc-test)
   ADD_KUDU_TEST(tls_handshake-test)
   ADD_KUDU_TEST(token-test)

http://git-wip-us.apache.org/repos/asf/kudu/blob/9136c8ad/src/kudu/security/ca/cert_management-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/ca/cert_management-test.cc b/src/kudu/security/ca/cert_management-test.cc
index ff070f6..c4d9084 100644
--- a/src/kudu/security/ca/cert_management-test.cc
+++ b/src/kudu/security/ca/cert_management-test.cc
@@ -50,8 +50,7 @@ class CertManagementTest : public KuduTest {
       ca_private_key_file_(JoinPathSegments(pem_dir_, "ca.pkey.pem")),
       ca_public_key_file_(JoinPathSegments(pem_dir_, "ca.pubkey.pem")),
       ca_exp_cert_file_(JoinPathSegments(pem_dir_, "ca.exp.cert.pem")),
-      ca_exp_private_key_file_(JoinPathSegments(pem_dir_, "ca.exp.pkey.pem")),
-      ca_exp_public_key_file_(JoinPathSegments(pem_dir_, "ca.exp.pubkey.pem")) {
+      ca_exp_private_key_file_(JoinPathSegments(pem_dir_, "ca.exp.pkey.pem")) {
   }
 
   void SetUp() override {
@@ -59,12 +58,9 @@ class CertManagementTest : public KuduTest {
     ASSERT_OK(WriteStringToFile(env_, kCaCert, ca_cert_file_));
     ASSERT_OK(WriteStringToFile(env_, kCaPrivateKey, ca_private_key_file_));
     ASSERT_OK(WriteStringToFile(env_, kCaPublicKey, ca_public_key_file_));
-
     ASSERT_OK(WriteStringToFile(env_, kCaExpiredCert, ca_exp_cert_file_));
     ASSERT_OK(WriteStringToFile(env_, kCaExpiredPrivateKey,
         ca_exp_private_key_file_));
-    ASSERT_OK(WriteStringToFile(env_, kCaExpiredPublicKey,
-        ca_exp_public_key_file_));
   }
 
  protected:
@@ -88,7 +84,7 @@ class CertManagementTest : public KuduTest {
         ::testing::UnitTest::GetInstance()->current_test_info();
     const string comment = string(test_info->test_case_name()) + "." +
       test_info->name();
-    const CertRequestGenerator::Config config = {
+    return {
       "US",               // country
       "CA",               // state
       "San Francisco",    // locality
@@ -99,7 +95,6 @@ class CertManagementTest : public KuduTest {
       hostnames,          // hostnames
       ips,                // ips
     };
-    return config;
   }
 
   // Run multiple threads which do certificate signing request generation
@@ -166,7 +161,6 @@ class CertManagementTest : public KuduTest {
 
   const string ca_exp_cert_file_;
   const string ca_exp_private_key_file_;
-  const string ca_exp_public_key_file_;
 };
 
 // Check input/output of RSA private keys in PEM format.
@@ -196,7 +190,7 @@ TEST_F(CertManagementTest, RsaPublicKeyInputOutputPEM) {
 }
 
 // Check extraction of the public part out from RSA private keys par.
-TEST_F(CertManagementTest, RSAExtractPublicPartFromPrivateKey) {
+TEST_F(CertManagementTest, RsaExtractPublicPartFromPrivateKey) {
   // Load the reference RSA private key.
   PrivateKey private_key;
   ASSERT_OK(private_key.FromString(kCaPrivateKey, DataFormat::PEM));
@@ -423,62 +417,6 @@ TEST_F(CertManagementTest, TestSelfSignedCA) {
   }
 }
 
-// Check the transformation chains for RSA private keys:
-//   internal -> PEM -> internal -> PEM
-//   internal -> DER -> internal -> DER
-TEST_F(CertManagementTest, RsaPrivateKeyFromAndToString) {
-  static const DataFormat kFormats[] = { DataFormat::PEM, DataFormat::DER };
-  static const uint16_t kKeyBits[] = { 256, 512, 1024, 2048, 3072, 4096 };
-
-  for (auto format : kFormats) {
-    for (auto key_bits : kKeyBits) {
-      SCOPED_TRACE(Substitute("key format: $0, key bits: $1",
-                              format == DataFormat::PEM ? "PEM"
-                                                        : "DER", key_bits));
-      PrivateKey key_ref;
-      ASSERT_OK(GeneratePrivateKey(key_bits, &key_ref));
-      string str_key_ref;
-      ASSERT_OK(key_ref.ToString(&str_key_ref, format));
-      PrivateKey key;
-      ASSERT_OK(key.FromString(str_key_ref, format));
-      string str_key;
-      ASSERT_OK(key.ToString(&str_key, format));
-      ASSERT_EQ(str_key_ref, str_key);
-    }
-  }
-}
-
-// Check the transformation chains for RSA public keys:
-//   internal -> PEM -> internal -> PEM
-//   internal -> DER -> internal -> DER
-TEST_F(CertManagementTest, RsaPublicKeyFromAndToString) {
-  static const DataFormat kFormats[] = { DataFormat::PEM, DataFormat::DER };
-  static const uint16_t kKeyBits[] = { 256, 512, 1024, 2048, 3072, 4096 };
-
-  for (auto format : kFormats) {
-    for (auto key_bits : kKeyBits) {
-      SCOPED_TRACE(Substitute("key format: $0, key bits: $1",
-                              format == DataFormat::PEM ? "PEM"
-                                                        : "DER", key_bits));
-      // Generate private RSA key.
-      PrivateKey private_key;
-      ASSERT_OK(GeneratePrivateKey(key_bits, &private_key));
-
-      // Extract public part of the key
-      PublicKey key_ref;
-      ASSERT_OK(private_key.GetPublicKey(&key_ref));
-
-      string str_key_ref;
-      ASSERT_OK(key_ref.ToString(&str_key_ref, format));
-      PublicKey key;
-      ASSERT_OK(key.FromString(str_key_ref, format));
-      string str_key;
-      ASSERT_OK(key.ToString(&str_key, format));
-      ASSERT_EQ(str_key_ref, str_key);
-    }
-  }
-}
-
 // Check the transformation chains for X509 CSRs:
 //   internal -> PEM -> internal -> PEM
 //   internal -> DER -> internal -> DER

http://git-wip-us.apache.org/repos/asf/kudu/blob/9136c8ad/src/kudu/security/crypto-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/crypto-test.cc b/src/kudu/security/crypto-test.cc
new file mode 100644
index 0000000..9eb4386
--- /dev/null
+++ b/src/kudu/security/crypto-test.cc
@@ -0,0 +1,188 @@
+// 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 <cstring>
+#include <functional>
+#include <mutex>
+#include <utility>
+
+#include "kudu/gutil/strings/strip.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/strings/util.h"
+#include "kudu/security/crypto.h"
+#include "kudu/security/openssl_util.h"
+#include "kudu/security/test/test_certs.h"
+#include "kudu/util/env.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+using std::string;
+using strings::Substitute;
+
+namespace kudu {
+namespace security {
+
+// Test for various crypto-related functionality in the security library.
+class CryptoTest : public KuduTest {
+ public:
+  CryptoTest() :
+      pem_dir_(GetTestPath("pem")),
+      private_key_file_(JoinPathSegments(pem_dir_, "private_key.pem")),
+      public_key_file_(JoinPathSegments(pem_dir_, "public_key.pem")),
+      corrupted_private_key_file_(JoinPathSegments(pem_dir_,
+          "corrupted.private_key.pem")),
+      corrupted_public_key_file_(JoinPathSegments(pem_dir_,
+          "corrupted.public_key.pem")) {
+  }
+
+  void SetUp() override {
+    ASSERT_OK(env_->CreateDir(pem_dir_));
+    ASSERT_OK(WriteStringToFile(env_, kCaPrivateKey, private_key_file_));
+    ASSERT_OK(WriteStringToFile(env_, kCaPublicKey, public_key_file_));
+    ASSERT_OK(WriteStringToFile(env_,
+        string(kCaPrivateKey, strlen(kCaPrivateKey) / 2),
+        corrupted_private_key_file_));
+    ASSERT_OK(WriteStringToFile(env_,
+        string(kCaPublicKey, strlen(kCaPublicKey) / 2),
+        corrupted_public_key_file_));
+  }
+
+ protected:
+  template<typename Key>
+  void CheckToAndFromString(const Key& key_ref, DataFormat format) {
+    SCOPED_TRACE(Substitute("ToAndFromString for $0 format",
+                            DataFormatToString(format)));
+    string key_ref_str;
+    ASSERT_OK(key_ref.ToString(&key_ref_str, format));
+    Key key;
+    ASSERT_OK(key.FromString(key_ref_str, format));
+    string key_str;
+    ASSERT_OK(key.ToString(&key_str, format));
+    ASSERT_EQ(key_ref_str, key_str);
+  }
+
+  const string pem_dir_;
+
+  const string private_key_file_;
+  const string public_key_file_;
+  const string corrupted_private_key_file_;
+  const string corrupted_public_key_file_;
+};
+
+// Check input/output of RSA private keys in PEM format.
+TEST_F(CryptoTest, RsaPrivateKeyInputOutputPEM) {
+  PrivateKey key;
+  ASSERT_OK(key.FromFile(private_key_file_, DataFormat::PEM));
+  string key_str;
+  key.ToString(&key_str, DataFormat::PEM);
+  RemoveExtraWhitespace(&key_str);
+
+  string ref_key_str(kCaPrivateKey);
+  RemoveExtraWhitespace(&ref_key_str);
+  EXPECT_EQ(ref_key_str, key_str);
+}
+
+// Check input of corrupted RSA private keys in PEM format.
+TEST_F(CryptoTest, CorruptedRsaPrivateKeyInputPEM) {
+  static const string kFiles[] = {
+      corrupted_private_key_file_,
+      public_key_file_,
+      corrupted_public_key_file_,
+      "/bin/sh"
+  };
+  for (const auto& file : kFiles) {
+    PrivateKey key;
+    const Status s = key.FromFile(file, DataFormat::PEM);
+    EXPECT_TRUE(s.IsRuntimeError()) << s.ToString();
+  }
+}
+
+// Check input/output of RSA public keys in PEM format.
+TEST_F(CryptoTest, RsaPublicKeyInputOutputPEM) {
+  PublicKey key;
+  ASSERT_OK(key.FromFile(public_key_file_, DataFormat::PEM));
+  string key_str;
+  key.ToString(&key_str, DataFormat::PEM);
+  RemoveExtraWhitespace(&key_str);
+
+  string ref_key_str(kCaPublicKey);
+  RemoveExtraWhitespace(&ref_key_str);
+  EXPECT_EQ(ref_key_str, key_str);
+}
+
+// Check input of corrupted RSA public keys in PEM format.
+TEST_F(CryptoTest, CorruptedRsaPublicKeyInputPEM) {
+  static const string kFiles[] = {
+      corrupted_public_key_file_,
+      private_key_file_,
+      corrupted_private_key_file_,
+      "/bin/sh"
+  };
+  for (const auto& file : kFiles) {
+    PublicKey key;
+    const Status s = key.FromFile(file, DataFormat::PEM);
+    EXPECT_TRUE(s.IsRuntimeError()) << s.ToString();
+  }
+}
+
+// Check extraction of the public part out from RSA private keys par.
+TEST_F(CryptoTest, RSAExtractPublicPartFromPrivateKey) {
+  // Load the reference RSA private key.
+  PrivateKey private_key;
+  ASSERT_OK(private_key.FromString(kCaPrivateKey, DataFormat::PEM));
+
+  PublicKey public_key;
+  ASSERT_OK(private_key.GetPublicKey(&public_key));
+  string str_public_key;
+  public_key.ToString(&str_public_key, DataFormat::PEM);
+  RemoveExtraWhitespace(&str_public_key);
+
+  string ref_str_public_key(kCaPublicKey);
+  RemoveExtraWhitespace(&ref_str_public_key);
+  EXPECT_EQ(ref_str_public_key, str_public_key);
+}
+
+class CryptoKeySerDesTest :
+    public CryptoTest,
+    public ::testing::WithParamInterface<DataFormat> {
+};
+
+// Check the transformation chains for RSA public/private keys:
+//   internal -> PEM -> internal -> PEM
+//   internal -> DER -> internal -> DER
+TEST_P(CryptoKeySerDesTest, ToAndFromString) {
+  const auto format = GetParam();
+
+  // Generate private RSA key.
+  PrivateKey private_key;
+  ASSERT_OK(GeneratePrivateKey(2048, &private_key));
+  NO_FATALS(CheckToAndFromString(private_key, format));
+
+  // Extract public part of the key.
+  PublicKey public_key;
+  ASSERT_OK(private_key.GetPublicKey(&public_key));
+  NO_FATALS(CheckToAndFromString(public_key, format));
+}
+
+INSTANTIATE_TEST_CASE_P(
+    DataFormats, CryptoKeySerDesTest,
+    ::testing::Values(DataFormat::DER, DataFormat::PEM));
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/9136c8ad/src/kudu/security/test/test_certs.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/test/test_certs.cc b/src/kudu/security/test/test_certs.cc
index 44d4aef..a7911a3 100644
--- a/src/kudu/security/test/test_certs.cc
+++ b/src/kudu/security/test/test_certs.cc
@@ -27,7 +27,6 @@ using std::string;
 
 namespace kudu {
 namespace security {
-namespace ca {
 
 //
 // The easiest way to create RSA private key and CA self-signed certificate pair
@@ -241,7 +240,6 @@ OwIDAQAB
 -----END PUBLIC KEY-----
 )***";
 
-} // namespace ca
 
 Status CreateTestSSLCerts(const string& dir,
                           string* cert_file,

http://git-wip-us.apache.org/repos/asf/kudu/blob/9136c8ad/src/kudu/security/test/test_certs.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/test/test_certs.h b/src/kudu/security/test/test_certs.h
index f45ea88..cb63ad2 100644
--- a/src/kudu/security/test/test_certs.h
+++ b/src/kudu/security/test/test_certs.h
@@ -22,7 +22,6 @@ namespace kudu {
 class Status;
 
 namespace security {
-namespace ca {
 
 //
 // Set of certificates and private keys used for certificate generation
@@ -45,8 +44,6 @@ extern const char kCaExpiredPrivateKey[];
 // The public part of the abovementioned private key.
 extern const char kCaExpiredPublicKey[];
 
-} // namespace ca
-
 // Creates a matching SSL certificate and private key file in 'dir', returning
 // their paths in '*cert_file' and '*key_file'. The password associated with
 // the private key is stored in '*key_password'.


[2/3] kudu git commit: [security] sign/verify data using RSA key pair

Posted by to...@apache.org.
[security] sign/verify data using RSA key pair

Added functionality to make a signature of a data chunk and
verify signature of a data chunk using RSA keys.  As for now,
supporting SHA256 and SHA512 message digests with test coverage
for SHA512.

Change-Id: I3fa04bb7d09aa416363998e2f8d7ccbdea625e4f
Reviewed-on: http://gerrit.cloudera.org:8080/5805
Reviewed-by: Dan Burkert <da...@apache.org>
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/e6e3029c
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/e6e3029c
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/e6e3029c

Branch: refs/heads/master
Commit: e6e3029c2f460fe838af70e07faa138962ee3b55
Parents: 9136c8a
Author: Alexey Serbin <as...@cloudera.com>
Authored: Thu Jan 26 12:57:35 2017 -0800
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Wed Feb 1 02:01:16 2017 +0000

----------------------------------------------------------------------
 src/kudu/security/crypto-test.cc     |  59 +++++++++++++++++-
 src/kudu/security/crypto.cc          | 100 ++++++++++++++++++++++++++++--
 src/kudu/security/crypto.h           |  20 ++++++
 src/kudu/security/test/test_certs.cc |  53 ++++++++++++++++
 src/kudu/security/test/test_certs.h  |   9 +++
 5 files changed, 234 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/e6e3029c/src/kudu/security/crypto-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/crypto-test.cc b/src/kudu/security/crypto-test.cc
index 9eb4386..e8136f9 100644
--- a/src/kudu/security/crypto-test.cc
+++ b/src/kudu/security/crypto-test.cc
@@ -19,6 +19,7 @@
 #include <functional>
 #include <mutex>
 #include <utility>
+#include <vector>
 
 #include "kudu/gutil/strings/strip.h"
 #include "kudu/gutil/strings/substitute.h"
@@ -31,8 +32,11 @@
 #include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
+#include "kudu/util/url-coding.h"
 
+using std::pair;
 using std::string;
+using std::vector;
 using strings::Substitute;
 
 namespace kudu {
@@ -141,8 +145,8 @@ TEST_F(CryptoTest, CorruptedRsaPublicKeyInputPEM) {
   }
 }
 
-// Check extraction of the public part out from RSA private keys par.
-TEST_F(CryptoTest, RSAExtractPublicPartFromPrivateKey) {
+// Check extraction of the public part from RSA private keys par.
+TEST_F(CryptoTest, RsaExtractPublicPartFromPrivateKey) {
   // Load the reference RSA private key.
   PrivateKey private_key;
   ASSERT_OK(private_key.FromString(kCaPrivateKey, DataFormat::PEM));
@@ -184,5 +188,56 @@ INSTANTIATE_TEST_CASE_P(
     DataFormats, CryptoKeySerDesTest,
     ::testing::Values(DataFormat::DER, DataFormat::PEM));
 
+// Check making crypto signatures against the reference data.
+TEST_F(CryptoTest, MakeVerifySignatureRef) {
+  static const vector<pair<string, string>> kRefElements = {
+    { kDataTiny,    kSignatureTinySHA512 },
+    { kDataShort,   kSignatureShortSHA512 },
+    { kDataLong,    kSignatureLongSHA512 },
+  };
+
+  // Load the reference RSA private key.
+  PrivateKey private_key;
+  ASSERT_OK(private_key.FromString(kCaPrivateKey, DataFormat::PEM));
+
+  // Load the reference RSA public key.
+  PublicKey public_key;
+  ASSERT_OK(public_key.FromString(kCaPublicKey, DataFormat::PEM));
+
+  for (const auto& e : kRefElements) {
+    string sig;
+    ASSERT_OK(private_key.MakeSignature(DigestType::SHA512, e.first, &sig));
+
+    // Ad-hoc verification: check the produced signature matches the reference.
+    string sig_base64;
+    Base64Encode(sig, &sig_base64);
+    EXPECT_EQ(e.second, sig_base64);
+
+    // Verify the signature cryptographically.
+    EXPECT_OK(public_key.VerifySignature(DigestType::SHA512, e.first, sig));
+  }
+}
+
+TEST_F(CryptoTest, VerifySignatureWrongData) {
+  static const vector<string> kRefSignatures = {
+    kSignatureTinySHA512,
+    kSignatureShortSHA512,
+    kSignatureLongSHA512,
+  };
+
+  // Load the reference RSA public key.
+  PublicKey key;
+  ASSERT_OK(key.FromString(kCaPublicKey, DataFormat::PEM));
+
+  for (const auto& e : kRefSignatures) {
+    string signature;
+    ASSERT_TRUE(Base64Decode(e, &signature));
+    Status s = key.VerifySignature(DigestType::SHA512,
+        "non-expected-data", signature);
+    EXPECT_TRUE(s.IsCorruption()) << s.ToString();
+  }
+}
+
+
 } // namespace security
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6e3029c/src/kudu/security/crypto.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/crypto.cc b/src/kudu/security/crypto.cc
index 856ddda..f34c270 100644
--- a/src/kudu/security/crypto.cc
+++ b/src/kudu/security/crypto.cc
@@ -81,9 +81,39 @@ struct RsaPublicKeyTraits : public SslTypeTraits<EVP_PKEY> {
 template<> struct SslTypeTraits<RSA> {
   static constexpr auto free = &RSA_free;
 };
-template<> struct SslTypeTraits<void> {
-    static constexpr auto free = &CRYPTO_free;
-};
+
+namespace {
+
+const char* GetDigestName(DigestType digest_type) {
+  switch (digest_type) {
+    case DigestType::SHA256:
+      return "SHA256";
+    case DigestType::SHA512:
+      return "SHA512";
+  }
+  return "UNKNOWN_DIGEST";
+}
+
+Status PrepareDigest(DigestType digest,
+                     const EVP_MD** md,
+                     EVP_MD_CTX** mctx,
+                     c_unique_ptr<BIO>* bmd_out) {
+  InitializeOpenSSL();
+  const char* md_name = GetDigestName(digest);
+  *md = EVP_get_digestbyname(md_name);
+  if (!*md) {
+    return Status::InvalidArgument(Substitute("error getting digest '$0': $1",
+                                              md_name, GetOpenSSLErrors()));
+  }
+  auto bmd = ssl_make_unique(BIO_new(BIO_f_md()));
+  OPENSSL_RET_NOT_OK(BIO_get_md_ctx(bmd.get(), mctx),
+                     "error initializing message digest context");
+  bmd_out->swap(bmd);
+
+  return Status::OK();
+}
+
+} // anonymous namespace
 
 
 Status PublicKey::FromString(const std::string& data, DataFormat format) {
@@ -106,6 +136,43 @@ Status PublicKey::FromBIO(BIO* bio, DataFormat format) {
       bio, format, &data_);
 }
 
+// Modeled after code in $OPENSSL_ROOT/apps/dgst.c
+Status PublicKey::VerifySignature(DigestType digest,
+                                  const std::string& data,
+                                  const std::string& signature) const {
+  const EVP_MD* md = nullptr;
+  EVP_MD_CTX* mctx = nullptr;
+  c_unique_ptr<BIO> bmd;
+  RETURN_NOT_OK(PrepareDigest(digest, &md, &mctx, &bmd));
+
+  OPENSSL_RET_NOT_OK(
+      EVP_DigestVerifyInit(mctx, nullptr, md, nullptr, GetRawData()),
+      "error initializing verification digest");
+  OPENSSL_RET_NOT_OK(
+      EVP_DigestVerifyUpdate(mctx, data.data(), data.size()),
+      "error verifying data signature");
+#if OPENSSL_VERSION_NUMBER < 0x10002000L
+  unsigned char* sig_data = reinterpret_cast<unsigned char*>(
+      const_cast<char*>(signature.data()));
+#else
+  const unsigned char* sig_data = reinterpret_cast<const unsigned char*>(
+      signature.data());
+#endif
+  // The success is indicated by return code 1. All other values means
+  // either wrong signature or error while performing signature verification.
+  const int rc = EVP_DigestVerifyFinal(mctx, sig_data, signature.size());
+  if (rc < 0 || rc > 1) {
+    return Status::RuntimeError(
+        Substitute("error verifying data signature: $0",
+                   GetOpenSSLErrors()));
+  }
+  if (rc == 0) {
+    return Status::Corruption("data signature verification failed");
+  }
+
+  return Status::OK();
+}
+
 Status PrivateKey::FromString(const std::string& data, DataFormat format) {
   return ::kudu::security::FromString<RawDataType, RsaPrivateKeyTraits>(
       data, format, &data_);
@@ -141,6 +208,29 @@ Status PrivateKey::GetPublicKey(PublicKey* public_key) {
   return Status::OK();
 }
 
+// Modeled after code in $OPENSSL_ROOT/apps/dgst.c
+Status PrivateKey::MakeSignature(DigestType digest,
+                                 const std::string& data,
+                                 std::string* signature) const {
+  CHECK(signature);
+  const EVP_MD* md = nullptr;
+  EVP_MD_CTX* mctx = nullptr;
+  c_unique_ptr<BIO> bmd;
+  RETURN_NOT_OK(PrepareDigest(digest, &md, &mctx, &bmd));
+  OPENSSL_RET_NOT_OK(
+      EVP_DigestSignInit(mctx, nullptr, md, nullptr, GetRawData()),
+      "error initializing signing digest");
+  OPENSSL_RET_NOT_OK(EVP_DigestSignUpdate(mctx, data.data(), data.size()),
+                     "error signing data");
+  size_t sig_len = EVP_PKEY_size(GetRawData());
+  unsigned char buf[4 * 1024];
+  OPENSSL_RET_NOT_OK(EVP_DigestSignFinal(mctx, buf, &sig_len),
+                     "error finalizing data signature");
+  *signature = string(reinterpret_cast<char*>(buf), sig_len);
+
+  return Status::OK();
+}
+
 Status GeneratePrivateKey(int num_bits, PrivateKey* ret) {
   CHECK(ret);
   InitializeOpenSSL();
@@ -152,8 +242,8 @@ Status GeneratePrivateKey(int num_bits, PrivateKey* ret) {
     OPENSSL_RET_NOT_OK(
         RSA_generate_key_ex(rsa.get(), num_bits, bn.get(), nullptr),
         "error generating RSA key");
-    OPENSSL_RET_NOT_OK(EVP_PKEY_set1_RSA(key.get(), rsa.get()),
-        "error assigning RSA key");
+    OPENSSL_RET_NOT_OK(
+        EVP_PKEY_set1_RSA(key.get(), rsa.get()), "error assigning RSA key");
   }
   ret->AdoptRawData(key.release());
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6e3029c/src/kudu/security/crypto.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/crypto.h b/src/kudu/security/crypto.h
index 1da5de3..6063f37 100644
--- a/src/kudu/security/crypto.h
+++ b/src/kudu/security/crypto.h
@@ -31,6 +31,12 @@ class Status;
 
 namespace security {
 
+// Supported message digests for data signing and signature verification.
+enum class DigestType {
+  SHA256,
+  SHA512,
+};
+
 // A class with generic public key interface, but actually it represents
 // an RSA key.
 class PublicKey : public RawDataWrapper<EVP_PKEY> {
@@ -42,6 +48,13 @@ class PublicKey : public RawDataWrapper<EVP_PKEY> {
   Status FromFile(const std::string& fpath, DataFormat format);
 
   Status FromBIO(BIO* bio, DataFormat format);
+
+  // Using the key, verify data signature using the specified message
+  // digest algorithm for signature verification.
+  // The input signature should be in in raw format (i.e. no base64 encoding).
+  Status VerifySignature(DigestType digest,
+                         const std::string& data,
+                         const std::string& signature) const;
 };
 
 // A class with generic private key interface, but actually it represents
@@ -57,6 +70,13 @@ class PrivateKey : public RawDataWrapper<EVP_PKEY> {
 
   // Output the public part of the keypair into the specified placeholder.
   Status GetPublicKey(PublicKey* public_key);
+
+  // Using the key, generate data signature using the specified
+  // message digest algorithm. The result signature is in raw format
+  // (i.e. no base64 encoding).
+  Status MakeSignature(DigestType digest,
+                       const std::string& data,
+                       std::string* signature) const;
 };
 
 // Utility method to generate private keys.

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6e3029c/src/kudu/security/test/test_certs.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/test/test_certs.cc b/src/kudu/security/test/test_certs.cc
index a7911a3..5432906 100644
--- a/src/kudu/security/test/test_certs.cc
+++ b/src/kudu/security/test/test_certs.cc
@@ -240,6 +240,59 @@ OwIDAQAB
 -----END PUBLIC KEY-----
 )***";
 
+//
+// The reference signatures were obtained by using the following sequence:
+//  0. The reference private key was saved into /tmp/ca.pkey.pem file.
+//  1. Put the input data into /tmp/in.txt file.
+//  2. To sign the input data, run
+//    openssl dgst -sign /tmp/ca.pkey.pem -sha512 -out /tmp/out /tmp/in.txt
+//  3. To capture the signature in text format, run
+//    base64 -b 60 /tmp/out
+//
+const char kDataTiny[] = "Tiny";
+const char kSignatureTinySHA512[] =
+    "omtvSpfj9tKo0RdI4zJwasWSQnXl++aKVjhH19ABJCd0haKT8RXNuhnxcbZU"
+    "Y1ILE5F9YjVj+tN/7ah5WQZR5qlJ6GMFfCFBhOzvi/vf5PSbUrFfwFvFD6sq"
+    "Bu0PWdwKM3t8/YFE2HcZWSzGCcasKlG/aw2eQCN3Kdv8QVMlC28CFA/EqQBt"
+    "8Sfye1DLba33SzDpJqR2DduTFrEW2UffumpYIbkEcMwUSBFzfdp5hgWPowFb"
+    "LrnKvyWKpEPMFGQmf5siyXSkbBIfL774tynhWN/lAUWykwXSUfGgi2G0NQvj"
+    "xmuHhbxWpbW/31uMGssw92OfVQ/+aQ4pNmY9GbibcA==";
+
+const char kDataShort[] = "ShortRefInputData";
+const char kSignatureShortSHA512[] =
+    "BHaDipr8ibn40BMD6+DlatKsjbmsGZsJIDlheppBjqv66eBDLKOVjpmpMLl9"
+    "9lXCGUlVS+cNcVP4RPDzXNoXkpzUOJD3UQSnxCAm6tV1eGjD3SHi3fk6PCNc"
+    "MhM/+09fA0WHdIdZm93cpHt6c9MFzB/dUjHJByhQ7Csmz2zdITyMIl3/D+bi"
+    "ocW0aIibk0wNGn/FmXfgFDP+3pBS2bpS0AdFnckX8AqXHFMJnvqKYODpYCW8"
+    "NWFSD1TgZOumu/gzxm+HySPezQ2j9tdR6nb9swfShvN+o0oBVGq5vgtgZMTM"
+    "7Ws+BrasLfvQFkvtGMWB9VeH/rDlGOym8RwUrCIJJQ==";
+
+const char kDataLong[] =
+R"***(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.
+)***";
+const char kSignatureLongSHA512[] =
+    "kc62qPHApVFbueR1xSCQJR5NomqDRzVA+4Xi9egVyfkKgpVhDAYGxbMl8OTY/YCb"
+    "eQuwY+B7RGxF9sj3gvsq/dvrbIjLT3QDhs0bv+lXTtBQ5r9zrals3de0tEFrPoLr"
+    "CkKPhVZaG+zwmUVltfsdlsqvepy6rNW7BocehvgpPTbzxgsZg4nUANsjSy8HBoDb"
+    "xWyfbkMgBY4aWIH1g+wksq1DHzdTNdZCYstupRwVw/ESC+zrFQiZPFeRE/wCSeG/"
+    "bd0L8TcotQHJchZ8THW0rEbuCg79I7Crd1KQYljBpOOhMYZEDEdM9L19JlaMlw+Z"
+    "leyLfL8Bw3wCg9cMfNmQfQ==";
+
 
 Status CreateTestSSLCerts(const string& dir,
                           string* cert_file,

http://git-wip-us.apache.org/repos/asf/kudu/blob/e6e3029c/src/kudu/security/test/test_certs.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/test/test_certs.h b/src/kudu/security/test/test_certs.h
index cb63ad2..763d9ed 100644
--- a/src/kudu/security/test/test_certs.h
+++ b/src/kudu/security/test/test_certs.h
@@ -44,6 +44,15 @@ extern const char kCaExpiredPrivateKey[];
 // The public part of the abovementioned private key.
 extern const char kCaExpiredPublicKey[];
 
+extern const char kDataTiny[];
+extern const char kSignatureTinySHA512[];
+
+extern const char kDataShort[];
+extern const char kSignatureShortSHA512[];
+
+extern const char kDataLong[];
+extern const char kSignatureLongSHA512[];
+
 // Creates a matching SSL certificate and private key file in 'dir', returning
 // their paths in '*cert_file' and '*key_file'. The password associated with
 // the private key is stored in '*key_password'.


[3/3] kudu git commit: [security] avoid crashing when importing invalid TSKs

Posted by to...@apache.org.
[security] avoid crashing when importing invalid TSKs

Kudu typically tries to avoid crashing when validating data from
external processes. In this case the data is from another Kudu server,
but it's still better not to crash.

Change-Id: I51dece30df0ec612417917f6855d57432fe2f7ec
Reviewed-on: http://gerrit.cloudera.org:8080/5843
Reviewed-by: Alexey Serbin <as...@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/e67289ed
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/e67289ed
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/e67289ed

Branch: refs/heads/master
Commit: e67289edfbca4b1d189c1b42406a5a8bc5602fc5
Parents: e6e3029
Author: Dan Burkert <da...@apache.org>
Authored: Tue Jan 31 16:20:41 2017 -0800
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Wed Feb 1 02:20:01 2017 +0000

----------------------------------------------------------------------
 src/kudu/security/token-test.cc     |  8 ++++----
 src/kudu/security/token_verifier.cc | 18 ++++++++++++++----
 src/kudu/security/token_verifier.h  |  3 ++-
 3 files changed, 20 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/e67289ed/src/kudu/security/token-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/token-test.cc b/src/kudu/security/token-test.cc
index d99cc85..0bb751d 100644
--- a/src/kudu/security/token-test.cc
+++ b/src/kudu/security/token-test.cc
@@ -136,7 +136,7 @@ TEST_F(TokenTest, TestEndToEnd_Valid) {
 
   // Try to verify it.
   TokenVerifier verifier;
-  verifier.ImportPublicKeys(signer.GetTokenSigningPublicKeys(0));
+  ASSERT_OK(verifier.ImportPublicKeys(signer.GetTokenSigningPublicKeys(0)));
   ASSERT_EQ(VerificationResult::VALID, verifier.VerifyTokenSignature(token));
 }
 
@@ -147,7 +147,7 @@ TEST_F(TokenTest, TestEndToEnd_InvalidCases) {
   ASSERT_OK(signer.RotateSigningKey());
 
   TokenVerifier verifier;
-  verifier.ImportPublicKeys(signer.GetTokenSigningPublicKeys(0));
+  ASSERT_OK(verifier.ImportPublicKeys(signer.GetTokenSigningPublicKeys(0)));
 
   // Make and sign a token, but corrupt the data in it.
   {
@@ -195,8 +195,8 @@ TEST_F(TokenTest, TestEndToEnd_InvalidCases) {
   FLAGS_token_signing_key_validity_seconds = -10;
   ASSERT_OK(signer.RotateSigningKey());
   ASSERT_OK(signer.RotateSigningKey());
-  verifier.ImportPublicKeys(signer.GetTokenSigningPublicKeys(
-      verifier.GetMaxKnownKeySequenceNumber()));
+  ASSERT_OK(verifier.ImportPublicKeys(signer.GetTokenSigningPublicKeys(
+      verifier.GetMaxKnownKeySequenceNumber())));
   {
     SignedTokenPB token = MakeUnsignedToken(WallTime_Now() + 600);
     ASSERT_OK(signer.SignToken(&token));

http://git-wip-us.apache.org/repos/asf/kudu/blob/e67289ed/src/kudu/security/token_verifier.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/token_verifier.cc b/src/kudu/security/token_verifier.cc
index 51bd398..80744de 100644
--- a/src/kudu/security/token_verifier.cc
+++ b/src/kudu/security/token_verifier.cc
@@ -52,15 +52,24 @@ int64_t TokenVerifier::GetMaxKnownKeySequenceNumber() const {
 
 // Import a set of public keys provided by the token signer (typically
 // running on another node).
-void TokenVerifier::ImportPublicKeys(const vector<TokenSigningPublicKeyPB>& public_keys) {
+Status TokenVerifier::ImportPublicKeys(const vector<TokenSigningPublicKeyPB>& public_keys) {
   // Do the copy construction outside of the lock, to avoid holding the
   // lock while doing lots of allocation.
   vector<unique_ptr<TokenSigningPublicKey>> tsks;
   for (const auto& pb : public_keys) {
     // Sanity check the key.
-    CHECK(pb.has_rsa_key_der());
-    CHECK(pb.has_key_seq_num());
-    CHECK(pb.has_expire_unix_epoch_seconds());
+    if (!pb.has_rsa_key_der()) {
+      return Status::RuntimeError(
+          "token-signing public key message must include the signing key");
+    }
+    if (!pb.has_key_seq_num()) {
+      return Status::RuntimeError(
+          "token-signing public key message must include the signing key sequence number");
+    }
+    if (!pb.has_expire_unix_epoch_seconds()) {
+      return Status::RuntimeError(
+          "token-signing public key message must include an expiration time");
+    }
     tsks.emplace_back(new TokenSigningPublicKey { pb });
   }
 
@@ -68,6 +77,7 @@ void TokenVerifier::ImportPublicKeys(const vector<TokenSigningPublicKeyPB>& publ
   for (auto&& tsk_ptr : tsks) {
     keys_by_seq_.emplace(tsk_ptr->pb().key_seq_num(), std::move(tsk_ptr));
   }
+  return Status::OK();
 }
 
 // Verify the signature on the given token.

http://git-wip-us.apache.org/repos/asf/kudu/blob/e67289ed/src/kudu/security/token_verifier.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/token_verifier.h b/src/kudu/security/token_verifier.h
index d068416..a35af22 100644
--- a/src/kudu/security/token_verifier.h
+++ b/src/kudu/security/token_verifier.h
@@ -61,7 +61,8 @@ class TokenVerifier {
   // Import a set of public keys provided by a TokenSigner instance (which might
   // be running on a remote node). If any public keys already exist with matching key
   // sequence numbers, they are replaced by the new keys.
-  void ImportPublicKeys(const std::vector<TokenSigningPublicKeyPB>& public_keys);
+  Status ImportPublicKeys(const std::vector<TokenSigningPublicKeyPB>& public_keys)
+    WARN_UNUSED_RESULT;
 
   // Verify the signature on the given token.
   VerificationResult VerifyTokenSignature(const SignedTokenPB& signed_token) const;