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 2017/02/22 17:59:27 UTC

[1/2] kudu git commit: [security] security-flags

Repository: kudu
Updated Branches:
  refs/heads/master dc454dcd0 -> dc1e45a9a


[security]  security-flags

This commit introduces, removes, and renames security flags in an effort
to make the flags more consistent, more understandable, and easier to
use from the command line. Affected flags:

--rpc_authentication
    This is a new flag which will apply to Kudu servers and the kudu
    command line tool which will allow operators to configure a policy
    for authentication of RPC connections. The possible values are
    'enabled', 'disabled', and 'required'. Three states are necessary
    (as opposed to just 'disabled' and 'required') in order to provide a
    graceful upgrade path for clusters from unsecured to secured.
    'enabled' is the default. A follow up commit will hook this flag
    into the RPC system to ensure that authentication is enforced as
    necessary.

--rpc_encryption
    This is a new flag which will apply to Kudu servers and the 'kudu'
    command line tool which allows operators to configure a policy for
    encryption on RPC connections. This is a tristate flag for the same
    reasons as outlined in --rpc_authentication. A follow up commit will
    hook this flag into the RPC system to ensure that encryption is
    enforced as necessary.

--server_require_kerberos
    This flag has been removed, and in it's place the --keytab and
    --rpc_authentication=required flags are provided. --keytab is used
    to enable Kerberos authentication on a server, and
    --rpc_authentication=required is used to ensure that all RPCs use
    authentication.

--rpc_certificate_file
--rpc_ssl_server_certificate
    --rpc_certificate_file is replacing --rpc_ssl_server_certificate.
    The latter has a few issues. 1) It's not strictly a server flag, it
    also applies to the kudu CLI tool. 3) It's not consistent with the
    similar --webserver_certificate_file flag.

--rpc_private_key_file
--rpc_ssl_private_key
    --rpc_private_key_file is replacing --rpc_ssl_private_key. Same
    reasons as --rpc_cert.

--rpc_ca_certificate_file
--rpc_ssl_certificate_authority
    --rpc_ca_certificate_file is replacing
    --rpc_ssl_certificate_authority. Same reasons as --rpc_cert.

Change-Id: Iaa53348b8969e83d9f794e1e0553bdec12252d9a
Reviewed-on: http://gerrit.cloudera.org:8080/6052
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>
Reviewed-by: Alexey Serbin <as...@cloudera.com>


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

Branch: refs/heads/master
Commit: 70ea7fb19cbd4474a0c4bc09e84f2f6276a1e917
Parents: dc454dc
Author: Dan Burkert <da...@apache.org>
Authored: Thu Feb 16 16:39:35 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Feb 22 17:09:28 2017 +0000

----------------------------------------------------------------------
 .../org/apache/kudu/client/MiniKuduCluster.java |   4 +-
 .../integration-tests/external_mini_cluster.cc  |   2 +-
 src/kudu/rpc/messenger.cc                       | 111 ++++++++++++++-----
 src/kudu/rpc/messenger.h                        |  21 ++++
 src/kudu/rpc/negotiation.cc                     |  10 +-
 src/kudu/rpc/rpc-test-base.h                    |  16 +--
 src/kudu/rpc/sasl_common.cc                     |   7 +-
 7 files changed, 122 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/70ea7fb1/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java b/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
index fa4fea4..c008b9a 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
@@ -182,7 +182,7 @@ public class MiniKuduCluster implements AutoCloseable {
       if (miniKdc != null) {
         commandLine.add("--keytab=" + keytab);
         commandLine.add("--kerberos_principal=kudu/" + bindHost);
-        commandLine.add("--server_require_kerberos");
+        commandLine.add("--rpc_authentication=required");
       }
 
       commandLine.addAll(extraTserverFlags);
@@ -263,7 +263,7 @@ public class MiniKuduCluster implements AutoCloseable {
       if (miniKdc != null) {
         commandLine.add("--keytab=" + keytab);
         commandLine.add("--kerberos_principal=kudu/" + bindHost);
-        commandLine.add("--server_require_kerberos");
+        commandLine.add("--rpc_authentication=required");
       }
 
       commandLine.addAll(extraMasterFlags);

http://git-wip-us.apache.org/repos/asf/kudu/blob/70ea7fb1/src/kudu/integration-tests/external_mini_cluster.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/external_mini_cluster.cc b/src/kudu/integration-tests/external_mini_cluster.cc
index 4c6dfb1..d6c3d97 100644
--- a/src/kudu/integration-tests/external_mini_cluster.cc
+++ b/src/kudu/integration-tests/external_mini_cluster.cc
@@ -606,7 +606,7 @@ Status ExternalDaemon::EnableKerberos(MiniKdc* kdc, const string& bind_host) {
   extra_env_ = kdc->GetEnvVars();
   extra_flags_.push_back(Substitute("--keytab=$0", ktpath));
   extra_flags_.push_back(Substitute("--kerberos_principal=$0", spn));
-  extra_flags_.push_back("--server_require_kerberos");
+  extra_flags_.push_back("--rpc_authentication=required");
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/70ea7fb1/src/kudu/rpc/messenger.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/messenger.cc b/src/kudu/rpc/messenger.cc
index 591e7d0..5ba0141 100644
--- a/src/kudu/rpc/messenger.cc
+++ b/src/kudu/rpc/messenger.cc
@@ -22,6 +22,7 @@
 #include <sys/types.h>
 #include <unistd.h>
 
+#include <boost/algorithm/string/predicate.hpp>
 #include <gflags/gflags.h>
 #include <glog/logging.h>
 #include <list>
@@ -59,31 +60,50 @@ using std::string;
 using std::shared_ptr;
 using strings::Substitute;
 
-DEFINE_string(rpc_ssl_server_certificate, "", "Path to the SSL certificate to be used for the RPC "
-    "layer.");
-DEFINE_string(rpc_ssl_private_key, "",
-    "Path to the private key to be used to complement the public key present in "
-    "--ssl_server_certificate");
-DEFINE_string(rpc_ssl_certificate_authority, "",
-    "Path to the certificate authority to be used by the client side of the connection to verify "
-    "the validity of the certificate presented by the server.");
+DEFINE_string(rpc_authentication, "optional",
+              "Whether to require RPC connections to authenticate. Must be one "
+              "of 'disabled', 'optional', or 'required'. If 'optional', "
+              "authentication will be used when the remote end supports it. If "
+              "'required', connections which are not able to authenticate "
+              "(because the remote end lacks support) are rejected. Secure "
+              "clusters should use 'required'.");
+DEFINE_string(rpc_encryption, "optional",
+              "Whether to require RPC connections to be encrypted. Must be one "
+              "of 'disabled', 'optional', or 'required'. If 'optional', "
+              "encryption will be used when the remote end supports it. If "
+              "'required', connections which are not able to use encryption "
+              "(because the remote end lacks support) are rejected. If 'disabled', "
+              "encryption will not be used, and RPC authentication "
+              "(--rpc_authentication) must also be disabled as well. "
+              "Secure clusters should use 'required'.");
+TAG_FLAG(rpc_authentication, stable);
+TAG_FLAG(rpc_encryption, stable);
+
+DEFINE_string(rpc_certificate_file, "",
+              "Path to a PEM encoded X509 certificate to use for securing RPC "
+              "connections with SSL/TLS. If set, '--rpc_private_key_file' and "
+              "'--rpc_ca_certificate_file' must be set as well.");
+DEFINE_string(rpc_private_key_file, "",
+              "Path to a PEM encoded private key paired with the certificate "
+              "from '--rpc_certificate_file'");
+DEFINE_string(rpc_ca_certificate_file, "",
+              "Path to the PEM encoded X509 certificate of the trusted external "
+              "certificate authority. The provided certificate should be the root "
+              "issuer of the certificate passed in '--rpc_certificate_file'.");
+
+// Setting TLS certs and keys via CLI flags is only necessary for external
+// PKI-based security, which is not yet production ready. Instead, see
+// internal PKI (ipki) and Kerberos-based authentication.
+TAG_FLAG(rpc_certificate_file, experimental);
+TAG_FLAG(rpc_private_key_file, experimental);
+TAG_FLAG(rpc_ca_certificate_file, experimental);
 
 DEFINE_int32(rpc_default_keepalive_time_ms, 65000,
              "If an RPC connection from a client is idle for this amount of time, the server "
              "will disconnect the client.");
-
-TAG_FLAG(rpc_ssl_server_certificate, experimental);
-TAG_FLAG(rpc_ssl_private_key, experimental);
-TAG_FLAG(rpc_ssl_certificate_authority, experimental);
 TAG_FLAG(rpc_default_keepalive_time_ms, advanced);
 
-DEFINE_bool(server_require_kerberos, false,
-            "Whether to force all inbound RPC connections to authenticate "
-            "with Kerberos");
-// TODO(todd): this flag is too coarse-grained, since secure servers still
-// need to allow non-kerberized connections authenticated by tokens. But
-// it's a useful stop-gap.
-TAG_FLAG(server_require_kerberos, experimental);
+DECLARE_string(keytab);
 
 namespace kudu {
 namespace rpc {
@@ -145,17 +165,52 @@ Status MessengerBuilder::Build(shared_ptr<Messenger> *msgr) {
       new_msgr->AllExternalReferencesDropped();
   });
 
+  if (boost::iequals(FLAGS_rpc_authentication, "required")) {
+    new_msgr->authentication_ = RpcAuthentication::REQUIRED;
+  } else if (boost::iequals(FLAGS_rpc_authentication, "optional")) {
+    new_msgr->authentication_ = RpcAuthentication::OPTIONAL;
+  } else if (boost::iequals(FLAGS_rpc_authentication, "disabled")) {
+    new_msgr->authentication_ = RpcAuthentication::DISABLED;
+  } else {
+    return Status::InvalidArgument(
+        "--rpc_authentication flag must be one of 'required', 'optional', or 'disabled'");
+  }
+
+  if (boost::iequals(FLAGS_rpc_encryption, "required")) {
+    new_msgr->encryption_ = RpcEncryption::REQUIRED;
+  } else if (boost::iequals(FLAGS_rpc_encryption, "optional")) {
+    new_msgr->encryption_ = RpcEncryption::OPTIONAL;
+  } else if (boost::iequals(FLAGS_rpc_encryption, "disabled")) {
+    new_msgr->encryption_ = RpcEncryption::DISABLED;
+  } else {
+    return Status::InvalidArgument(
+        "--rpc_encryption flag must be one of 'required', 'optional', or 'disabled'");
+  }
+
+  if (new_msgr->encryption_ == RpcEncryption::DISABLED &&
+      new_msgr->authentication_ != RpcAuthentication::DISABLED) {
+    return Status::InvalidArgument("RPC authentication (--rpc_authentication) must be disabled "
+                                   "if RPC encryption (--rpc_encryption) is disabled");
+  }
+
   RETURN_NOT_OK(new_msgr->Init());
-  if (enable_inbound_tls_server_uuid_) {
+  if (new_msgr->encryption_ != RpcEncryption::DISABLED && enable_inbound_tls_server_uuid_) {
     auto* tls_context = new_msgr->mutable_tls_context();
-    if (!FLAGS_rpc_ssl_server_certificate.empty() ||
-        !FLAGS_rpc_ssl_private_key.empty() ||
-        !FLAGS_rpc_ssl_certificate_authority.empty()) {
+
+    if (!FLAGS_rpc_certificate_file.empty() &&
+        !FLAGS_rpc_private_key_file.empty() &&
+        !FLAGS_rpc_ca_certificate_file.empty()) {
+
       // TODO(PKI): should we try and enforce that the server UUID and/or
       // hostname is in the subject or alt names of the cert?
-      RETURN_NOT_OK(tls_context->LoadCertificateAuthority(FLAGS_rpc_ssl_certificate_authority));
-      RETURN_NOT_OK(tls_context->LoadCertificateAndKey(FLAGS_rpc_ssl_server_certificate,
-                                                       FLAGS_rpc_ssl_private_key));
+      RETURN_NOT_OK(tls_context->LoadCertificateAuthority(FLAGS_rpc_ca_certificate_file));
+      RETURN_NOT_OK(tls_context->LoadCertificateAndKey(FLAGS_rpc_certificate_file,
+                                                       FLAGS_rpc_private_key_file));
+    } else if (!FLAGS_rpc_certificate_file.empty() ||
+               !FLAGS_rpc_private_key_file.empty() ||
+               !FLAGS_rpc_ca_certificate_file.empty()) {
+      return Status::InvalidArgument("--rpc_certificate_file, --rpc_private_key_file, and "
+                                     "--rpc_ca_certificate_file flags must be set as a group");
     } else {
       RETURN_NOT_OK(tls_context->GenerateSelfSignedCertAndKey(*enable_inbound_tls_server_uuid_));
     }
@@ -213,7 +268,7 @@ Status Messenger::AddAcceptorPool(const Sockaddr &accept_addr,
   // Before listening, if we expect to require Kerberos, we want to verify
   // that everything is set up correctly. This way we'll generate errors on
   // startup rather than later on when we first receive a client connection.
-  if (FLAGS_server_require_kerberos) {
+  if (!FLAGS_keytab.empty()) {
     RETURN_NOT_OK_PREPEND(ServerNegotiation::PreflightCheckGSSAPI(),
                           "GSSAPI/Kerberos not properly configured");
   }
@@ -292,6 +347,8 @@ void Messenger::RegisterInboundSocket(Socket *new_socket, const Sockaddr &remote
 Messenger::Messenger(const MessengerBuilder &bld)
   : name_(bld.name_),
     closing_(false),
+    authentication_(RpcAuthentication::REQUIRED),
+    encryption_(RpcEncryption::REQUIRED),
     tls_context_(new security::TlsContext()),
     token_verifier_(new security::TokenVerifier()),
     rpcz_store_(new RpczStore()),

http://git-wip-us.apache.org/repos/asf/kudu/blob/70ea7fb1/src/kudu/rpc/messenger.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/messenger.h b/src/kudu/rpc/messenger.h
index c232bf5..89dd933 100644
--- a/src/kudu/rpc/messenger.h
+++ b/src/kudu/rpc/messenger.h
@@ -74,6 +74,20 @@ struct AcceptorPoolInfo {
   Sockaddr bind_address_;
 };
 
+// Authentication configuration for RPC connections.
+enum class RpcAuthentication {
+  DISABLED,
+  OPTIONAL,
+  REQUIRED,
+};
+
+// Encryption configuration for RPC connections.
+enum class RpcEncryption {
+  DISABLED,
+  OPTIONAL,
+  REQUIRED,
+};
+
 // Used to construct a Messenger.
 class MessengerBuilder {
  public:
@@ -252,6 +266,13 @@ class Messenger {
 
   bool closing_;
 
+  // Whether to require authentication and encryption on the connections managed
+  // by this messenger.
+  // TODO(PKI): scope these to individual proxies, so that messengers can be
+  // reused by different clients.
+  RpcAuthentication authentication_;
+  RpcEncryption encryption_;
+
   // Pools which are listening on behalf of this messenger.
   // Note that the user may have called Shutdown() on one of these
   // pools, so even though we retain the reference, it may no longer

http://git-wip-us.apache.org/repos/asf/kudu/blob/70ea7fb1/src/kudu/rpc/negotiation.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/negotiation.cc b/src/kudu/rpc/negotiation.cc
index f303179..e044c98 100644
--- a/src/kudu/rpc/negotiation.cc
+++ b/src/kudu/rpc/negotiation.cc
@@ -53,7 +53,7 @@ DEFINE_int32(rpc_negotiation_inject_delay_ms, 0,
              "the RPC negotiation process on the server side.");
 TAG_FLAG(rpc_negotiation_inject_delay_ms, unsafe);
 
-DECLARE_bool(server_require_kerberos);
+DECLARE_string(keytab);
 
 DEFINE_bool(rpc_encrypt_loopback_connections, false,
             "Whether to encrypt data transfer on RPC connections that stay within "
@@ -208,10 +208,12 @@ static Status DoServerNegotiation(Connection* conn, const MonoTime& deadline) {
                                        &messenger->tls_context(),
                                        &messenger->token_verifier());
 
-  if (FLAGS_server_require_kerberos) {
-    RETURN_NOT_OK(server_negotiation.EnableGSSAPI());
-  } else {
+  // TODO(PKI): this should be enabling PLAIN if authn < required, and GSSAPI if
+  // there is a keytab and authn > disabled. Same with client version.
+  if (FLAGS_keytab.empty()) {
     RETURN_NOT_OK(server_negotiation.EnablePlain());
+  } else {
+    RETURN_NOT_OK(server_negotiation.EnableGSSAPI());
   }
   server_negotiation.set_deadline(deadline);
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/70ea7fb1/src/kudu/rpc/rpc-test-base.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/rpc-test-base.h b/src/kudu/rpc/rpc-test-base.h
index 5bdacee..47f209f 100644
--- a/src/kudu/rpc/rpc-test-base.h
+++ b/src/kudu/rpc/rpc-test-base.h
@@ -50,10 +50,6 @@
 #include "kudu/util/test_util.h"
 #include "kudu/util/trace.h"
 
-DECLARE_string(rpc_ssl_server_certificate);
-DECLARE_string(rpc_ssl_private_key);
-DECLARE_string(rpc_ssl_certificate_authority);
-
 namespace kudu { namespace rpc {
 
 using kudu::rpc_test::AddRequestPB;
@@ -367,16 +363,12 @@ class RpcTestBase : public KuduTest {
   std::shared_ptr<Messenger> CreateMessenger(const string &name,
                                              int n_reactors = 1,
                                              bool enable_ssl = false) {
+    MessengerBuilder bld(name);
+
     if (enable_ssl) {
-      std::string server_cert_path = GetTestPath("server-cert.pem");
-      std::string private_key_path = GetTestPath("server-key.pem");
-      CHECK_OK(security::CreateSSLServerCert(server_cert_path));
-      CHECK_OK(security::CreateSSLPrivateKey(private_key_path));
-      FLAGS_rpc_ssl_server_certificate = server_cert_path;
-      FLAGS_rpc_ssl_private_key = private_key_path;
-      FLAGS_rpc_ssl_certificate_authority = server_cert_path;
+      bld.enable_inbound_tls(name);
     }
-    MessengerBuilder bld(name);
+
     bld.set_num_reactors(n_reactors);
     bld.set_connection_keepalive_time(
       MonoDelta::FromMilliseconds(keepalive_time_ms_));

http://git-wip-us.apache.org/repos/asf/kudu/blob/70ea7fb1/src/kudu/rpc/sasl_common.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_common.cc b/src/kudu/rpc/sasl_common.cc
index a0ae7e1..c852675 100644
--- a/src/kudu/rpc/sasl_common.cc
+++ b/src/kudu/rpc/sasl_common.cc
@@ -42,7 +42,7 @@
 
 using std::set;
 
-DECLARE_bool(server_require_kerberos);
+DECLARE_string(keytab);
 
 namespace kudu {
 namespace rpc {
@@ -318,9 +318,10 @@ Status WrapSaslCall(sasl_conn_t* conn, const std::function<int()>& call) {
   g_auth_failure_capture = &err;
 
   // Take the 'kerberos_reinit_lock' here to avoid a possible race with ticket renewal.
-  if (FLAGS_server_require_kerberos) kudu::security::KerberosReinitLock()->ReadLock();
+  bool kerberos_supported = !FLAGS_keytab.empty();
+  if (kerberos_supported) kudu::security::KerberosReinitLock()->ReadLock();
   int rc = call();
-  if (FLAGS_server_require_kerberos) kudu::security::KerberosReinitLock()->ReadUnlock();
+  if (kerberos_supported) kudu::security::KerberosReinitLock()->ReadUnlock();
   g_auth_failure_capture = nullptr;
 
   switch (rc) {


[2/2] kudu git commit: [security] derive TSK params from authn token ones

Posted by al...@apache.org.
[security] derive TSK params from authn token ones

Derive the TSK validity interval from the authn token validity period.
The idea is to have set of parameters which is user-oriented: the authn
token lifetime directly impacts user job lifetimes, etc.

The default validity period for authn tokens is set to 7 days to
mirror other Hadoop ecosystem components (e.g. HBase).

The TSK validity interval is derived from authn token validity period:
tsk_validity = authn_token_validity + tsk_rotation.

Change-Id: I95bc64897ed16becda4ab8de6817695fdb48e9eb
Reviewed-on: http://gerrit.cloudera.org:8080/6071
Tested-by: Kudu Jenkins
Reviewed-by: Dan Burkert <da...@apache.org>


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

Branch: refs/heads/master
Commit: dc1e45a9a8571b6c93f1357a61322ad0d6b6cea9
Parents: 70ea7fb
Author: Alexey Serbin <as...@cloudera.com>
Authored: Sat Feb 18 01:19:04 2017 -0800
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Wed Feb 22 17:58:44 2017 +0000

----------------------------------------------------------------------
 .../integration-tests/token_signer-itest.cc     |  4 +--
 src/kudu/master/master.cc                       | 19 ++++++------
 src/kudu/security/token-test.cc                 | 26 ++++++++--------
 src/kudu/security/token_signer.cc               | 22 ++++++--------
 src/kudu/security/token_signer.h                | 32 ++++++++++++++------
 5 files changed, 58 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/dc1e45a9/src/kudu/integration-tests/token_signer-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/token_signer-itest.cc b/src/kudu/integration-tests/token_signer-itest.cc
index 823169c..20e960d 100644
--- a/src/kudu/integration-tests/token_signer-itest.cc
+++ b/src/kudu/integration-tests/token_signer-itest.cc
@@ -38,7 +38,7 @@
 #include "kudu/util/slice.h"
 #include "kudu/util/test_util.h"
 
-DECLARE_int64(tsk_validity_seconds);
+DECLARE_int64(authn_token_validity_seconds);
 DECLARE_int64(tsk_rotation_seconds);
 
 using std::string;
@@ -54,7 +54,7 @@ namespace master {
 class TokenSignerITest : public KuduTest {
  public:
   TokenSignerITest() {
-    FLAGS_tsk_validity_seconds = 60;
+    FLAGS_authn_token_validity_seconds = 60;
     FLAGS_tsk_rotation_seconds = 20;
 
     // Hard-coded ports for the masters. This is safe, as this unit test

http://git-wip-us.apache.org/repos/asf/kudu/blob/dc1e45a9/src/kudu/master/master.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master.cc b/src/kudu/master/master.cc
index 7c5ed46..753419b 100644
--- a/src/kudu/master/master.cc
+++ b/src/kudu/master/master.cc
@@ -53,20 +53,20 @@ DEFINE_int32(master_registration_rpc_timeout_ms, 1500,
              "Timeout for retrieving master registration over RPC.");
 TAG_FLAG(master_registration_rpc_timeout_ms, experimental);
 
-DEFINE_int64(tsk_validity_seconds, 60 * 60 * 24 * 7,
-             "Number of seconds that a TSK (Token Signing Key) is valid for.");
-TAG_FLAG(tsk_validity_seconds, advanced);
-TAG_FLAG(tsk_validity_seconds, experimental);
-
 DEFINE_int64(tsk_rotation_seconds, 60 * 60 * 24 * 1,
              "Number of seconds between consecutive activations of newly "
              "generated TSKs (Token Signing Keys).");
 TAG_FLAG(tsk_rotation_seconds, advanced);
 TAG_FLAG(tsk_rotation_seconds, experimental);
 
+DEFINE_int64(authn_token_validity_seconds, 60 * 60 * 24 * 7,
+             "Period of time for which an issued authentication token is valid.");
+// TODO(PKI): docs for what actual effect this has, given we don't support
+// token renewal.
+TAG_FLAG(authn_token_validity_seconds, experimental);
+
 using std::min;
 using std::shared_ptr;
-using std::unique_ptr;
 using std::vector;
 
 using kudu::consensus::RaftPeerPB;
@@ -119,9 +119,10 @@ Status Master::Init() {
   cert_authority_.reset(new MasterCertAuthority(fs_manager_->uuid()));
 
   // The TokenSigner loads its keys during catalog manager initialization.
-  token_signer_.reset(new TokenSigner(FLAGS_tsk_validity_seconds,
-                                      FLAGS_tsk_rotation_seconds,
-                                      messenger_->shared_token_verifier()));
+  token_signer_.reset(new TokenSigner(
+      FLAGS_authn_token_validity_seconds,
+      FLAGS_tsk_rotation_seconds,
+      messenger_->shared_token_verifier()));
   state_ = kInitialized;
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/dc1e45a9/src/kudu/security/token-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/token-test.cc b/src/kudu/security/token-test.cc
index 5adeb25..d641214 100644
--- a/src/kudu/security/token-test.cc
+++ b/src/kudu/security/token-test.cc
@@ -87,13 +87,13 @@ class TokenTest : public KuduTest {
  public:
   void SetUp() override {
     KuduTest::SetUp();
-    // Set the keylength smaller to make tests run faster.
+    // Set the key length smaller to make tests run faster.
     FLAGS_tsk_num_rsa_bits = 512;
   }
 };
 
 TEST_F(TokenTest, TestInit) {
-  TokenSigner signer(60, 20, make_shared<TokenVerifier>());
+  TokenSigner signer(10, 10);
   const TokenVerifier& verifier(signer.verifier());
 
   SignedTokenPB token = MakeUnsignedToken(WallTime_Now());
@@ -122,7 +122,7 @@ TEST_F(TokenTest, TestInit) {
 
 TEST_F(TokenTest, TestTokenSignerAddKeys) {
   {
-    TokenSigner signer(60, 20, make_shared<TokenVerifier>());
+    TokenSigner signer(10, 10);
     std::unique_ptr<TokenSigningPrivateKey> key;
     ASSERT_OK(signer.CheckNeedKey(&key));
     // No keys are available yet, so should be able to add.
@@ -137,7 +137,7 @@ TEST_F(TokenTest, TestTokenSignerAddKeys) {
   {
     // Special configuration for TokenSigner: rotation interval is zero,
     // so should be able to add two keys right away.
-    TokenSigner signer(60, 0, make_shared<TokenVerifier>());
+    TokenSigner signer(10, 0);
     std::unique_ptr<TokenSigningPrivateKey> key;
     ASSERT_OK(signer.CheckNeedKey(&key));
     // No keys are available yet, so should be able to add.
@@ -156,9 +156,9 @@ TEST_F(TokenTest, TestTokenSignerAddKeys) {
 
   {
     // Special configuration for TokenSigner: key rotation interval
-    // just one second shorter than the validity interval. It should not
-    // need next key right away, but should need next key after 1 second.
-    TokenSigner signer(60, 1, make_shared<TokenVerifier>());
+    // just one second. It should not need next key right away, but should need
+    // next key after 1 second.
+    TokenSigner signer(10, 1);
     std::unique_ptr<TokenSigningPrivateKey> key;
     ASSERT_OK(signer.CheckNeedKey(&key));
     // No keys are available yet, so should be able to add.
@@ -185,7 +185,7 @@ TEST_F(TokenTest, TestTokenSignerAddKeys) {
 // Test how test rotation works.
 TEST_F(TokenTest, TestTokenSignerSignVerifyExport) {
   // Key rotation interval 0 allows adding 2 keys in a row with no delay.
-  TokenSigner signer(60, 0, make_shared<TokenVerifier>());
+  TokenSigner signer(10, 0);
   const TokenVerifier& verifier(signer.verifier());
 
   // Should start off with no signing keys.
@@ -248,8 +248,10 @@ TEST_F(TokenTest, TestTokenSignerSignVerifyExport) {
 TEST_F(TokenTest, TestExportKeys) {
   // Test that the exported public keys don't contain private key material,
   // and have an appropriate expiration.
-  const int64_t key_exp_seconds = 60;
-  TokenSigner signer(key_exp_seconds, 30, make_shared<TokenVerifier>());
+  const int64_t key_exp_seconds = 20;
+  const int64_t key_rotation_seconds = 10;
+  TokenSigner signer(key_exp_seconds - key_rotation_seconds,
+                     key_rotation_seconds);
   int64_t key_seq_num;
   {
     std::unique_ptr<TokenSigningPrivateKey> key;
@@ -273,7 +275,7 @@ TEST_F(TokenTest, TestExportKeys) {
 // Test that the TokenVerifier can import keys exported by the TokenSigner
 // and then verify tokens signed by it.
 TEST_F(TokenTest, TestEndToEnd_Valid) {
-  TokenSigner signer(60, 20, make_shared<TokenVerifier>());
+  TokenSigner signer(10, 10);
   {
     std::unique_ptr<TokenSigningPrivateKey> key;
     ASSERT_OK(signer.CheckNeedKey(&key));
@@ -296,7 +298,7 @@ TEST_F(TokenTest, TestEndToEnd_Valid) {
 // See VerificationResult.
 TEST_F(TokenTest, TestEndToEnd_InvalidCases) {
   // Key rotation interval 0 allows adding 2 keys in a row with no delay.
-  TokenSigner signer(60, 0, make_shared<TokenVerifier>());
+  TokenSigner signer(10, 0);
   {
     std::unique_ptr<TokenSigningPrivateKey> key;
     ASSERT_OK(signer.CheckNeedKey(&key));

http://git-wip-us.apache.org/repos/asf/kudu/blob/dc1e45a9/src/kudu/security/token_signer.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/token_signer.cc b/src/kudu/security/token_signer.cc
index adf6a51..c4a54d5 100644
--- a/src/kudu/security/token_signer.cc
+++ b/src/kudu/security/token_signer.cc
@@ -36,14 +36,6 @@
 #include "kudu/util/locks.h"
 #include "kudu/util/status.h"
 
-DEFINE_int64(authn_token_validity_seconds, 120,
-             "Period of time for which an issued authentication token is valid.");
-// TODO(PKI): docs for what actual effect this has, given we don't support
-// token renewal.
-// TODO(PKI): this is set extremely low, so that we don't forget to come back to
-// this and add rolling and refetching code.
-TAG_FLAG(authn_token_validity_seconds, experimental);
-
 DEFINE_int32(tsk_num_rsa_bits, 2048,
              "Number of bits used for token signing keys");
 // TODO(PKI) is 1024 enough for TSKs since they rotate frequently?
@@ -60,13 +52,17 @@ using std::vector;
 namespace kudu {
 namespace security {
 
-TokenSigner::TokenSigner(int64_t key_validity_seconds,
+TokenSigner::TokenSigner(int64_t authn_token_validity_seconds,
                          int64_t key_rotation_seconds,
                          shared_ptr<TokenVerifier> verifier)
-    : verifier_(std::move(verifier)),
-      key_validity_seconds_(key_validity_seconds),
+    : verifier_(verifier ? std::move(verifier)
+                         : std::make_shared<TokenVerifier>()),
+      authn_token_validity_seconds_(authn_token_validity_seconds),
       key_rotation_seconds_(key_rotation_seconds),
+      key_validity_seconds_(key_rotation_seconds_ + authn_token_validity_seconds_),
       next_key_seq_num_(0) {
+  CHECK_GE(key_rotation_seconds_, 0);
+  CHECK_GE(authn_token_validity_seconds_, 0);
   CHECK(verifier_);
 }
 
@@ -133,7 +129,7 @@ Status TokenSigner::GenerateAuthnToken(string username,
                                        SignedTokenPB* signed_token) const {
   TokenPB token;
   token.set_expire_unix_epoch_seconds(
-      WallTime_Now() + FLAGS_authn_token_validity_seconds);
+      WallTime_Now() + authn_token_validity_seconds_);
   AuthnTokenPB* authn = token.mutable_authn();
   authn->mutable_username()->assign(std::move(username));
 
@@ -177,7 +173,7 @@ Status TokenSigner::CheckNeedKey(unique_ptr<TokenSigningPrivateKey>* tsk) const
     // It does not make much sense to keep more than two keys in the queue.
     // It's enough to have just one active key and next key ready to be
     // activated when it's time to do so.  However, it does not mean the
-    // process of key refreshement is about to stop once there are two keys
+    // process of key refreshment is about to stop once there are two keys
     // in the queue: the TryRotate() method (which should be called periodically
     // along with CheckNeedKey()/AddKey() pair) will eventually pop the
     // current key out of the keys queue once the key enters its inactive phase.

http://git-wip-us.apache.org/repos/asf/kudu/blob/dc1e45a9/src/kudu/security/token_signer.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/token_signer.h b/src/kudu/security/token_signer.h
index 9f05d71..82aba2b 100644
--- a/src/kudu/security/token_signer.h
+++ b/src/kudu/security/token_signer.h
@@ -135,7 +135,7 @@ class TokenVerifier;
 //
 // Typical usage pattern:
 //
-//    TokenSigner ts;
+//    TokenSigner ts(...);
 //    // Load existing TSKs from the system table.
 //    ...
 //    RETURN_NOT_OK(ts.ImportKeys(...));
@@ -163,14 +163,25 @@ class TokenVerifier;
 //
 class TokenSigner {
  public:
-  // Parameters of the TokenSigner constructor define the TSK rotation schedule.
-  // See the class's comment just above for details.
+  // The 'key_validity_seconds' and 'key_rotation_seconds' parameters define
+  // the schedule of TSK rotation. See the class comment above for details.
   //
   // Any newly imported or generated keys are automatically imported into the
-  // passed 'verifier'.
-  TokenSigner(int64_t key_validity_seconds,
+  // passed 'verifier'. If no verifier passed as a parameter, TokenSigner
+  // creates one on its own. In either case, it's possible to access
+  // the embedded TokenVerifier instance using the verifier() accessor.
+  //
+  // The 'authn_token_validity_seconds' parameter is used to specify validity
+  // interval for the generated authn tokens and with 'key_rotation_seconds'
+  // it defines validity interval of the newly generated TSK:
+  //   key_validity = key_rotation + authn_token_validity.
+  //
+  // That corresponds to the maximum possible token lifetime for the effective
+  // TSK validity and rotation intervals: see the class comment above for
+  // details.
+  TokenSigner(int64_t authn_token_validity_seconds,
               int64_t key_rotation_seconds,
-              std::shared_ptr<TokenVerifier> verifier);
+              std::shared_ptr<TokenVerifier> verifier = nullptr);
   ~TokenSigner();
 
   // Import token signing keys in PB format, notifying TokenVerifier
@@ -236,14 +247,17 @@ class TokenSigner {
 
   std::shared_ptr<TokenVerifier> verifier_;
 
-  // Period of validity for newly created token signing keys. In other words,
-  // the expiration time for a new key is set to (now + key_validity_seconds_).
-  const int64_t key_validity_seconds_;
+  // Validity interval for the generated authn tokens.
+  const int64_t authn_token_validity_seconds_;
 
   // TSK rotation interval: number of seconds between consecutive activations
   // of new token signing keys.
   const int64_t key_rotation_seconds_;
 
+  // Period of validity for newly created token signing keys. In other words,
+  // the expiration time for a new key is set to (now + key_validity_seconds_).
+  const int64_t key_validity_seconds_;
+
   // Protects next_seq_num_ and tsk_deque_ members.
   mutable RWMutex lock_;