You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by aw...@apache.org on 2018/10/30 17:26:58 UTC

kudu git commit: KUDU-2542: add initial authorization token impl

Repository: kudu
Updated Branches:
  refs/heads/master 3a77ba131 -> 94e484da8


KUDU-2542: add initial authorization token impl

This patch adds an authorization token that echoes the authentication
token implementation. These tokens contain privileges that will be used
authorize specific tablet server requests.

By in large, tablet server requests are scoped to a single table, and as
such, so are authz tokens. In cases where this is not true (e.g.
ListTablets), a reasonable assumption is that the call is being made via
tooling, and coarse-grained access control should be used instead of
fine-grained. If this ends up being less the case in the future, we can
always amend the authz token to support multiple tables.

The tokens leverage the same token signer as the authentication tokens,
though with the token validity interval configured via the new flag
--authz_token_validity_seconds.

Change-Id: Id28747ec38675abdf50dce1e7c176d29213e370f
Reviewed-on: http://gerrit.cloudera.org:8080/11750
Tested-by: Kudu Jenkins
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/94e484da
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/94e484da
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/94e484da

Branch: refs/heads/master
Commit: 94e484da8686d830f64134400a7fa7b2b3ac5394
Parents: 3a77ba1
Author: Andrew Wong <aw...@cloudera.com>
Authored: Sun Oct 21 23:38:38 2018 -0700
Committer: Andrew Wong <aw...@cloudera.com>
Committed: Tue Oct 30 17:26:18 2018 +0000

----------------------------------------------------------------------
 .../authn_token_expire-itest.cc                 |   3 +
 .../security-unknown-tsk-itest.cc               |   2 +-
 .../integration-tests/token_signer-itest.cc     |   3 +
 src/kudu/master/master.cc                       |   7 +
 src/kudu/rpc/negotiation-test.cc                |   4 +-
 src/kudu/security/token-test.cc                 | 286 ++++++++++++++++---
 src/kudu/security/token.proto                   |  31 ++
 src/kudu/security/token_signer.cc               |  36 ++-
 src/kudu/security/token_signer.h                |  89 ++++--
 9 files changed, 387 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/94e484da/src/kudu/integration-tests/authn_token_expire-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/authn_token_expire-itest.cc b/src/kudu/integration-tests/authn_token_expire-itest.cc
index c6dd663..bdfdb3f 100644
--- a/src/kudu/integration-tests/authn_token_expire-itest.cc
+++ b/src/kudu/integration-tests/authn_token_expire-itest.cc
@@ -153,6 +153,7 @@ class AuthnTokenExpireITest : public AuthnTokenExpireITestBase {
       // a new authn token re-acquisitions and retried RPCs.
       "--tsk_rotation_seconds=1",
       Substitute("--authn_token_validity_seconds=$0", token_validity_seconds_),
+      Substitute("--authz_token_validity_seconds=$0", token_validity_seconds_),
     };
 
     cluster_opts_.extra_tserver_flags = {
@@ -361,6 +362,7 @@ class TokenBasedConnectionITest : public AuthnTokenExpireITestBase {
           /*num_tablet_servers=*/ 3) {
     cluster_opts_.extra_master_flags = {
       Substitute("--authn_token_validity_seconds=$0", token_validity_seconds_),
+      Substitute("--authz_token_validity_seconds=$0", token_validity_seconds_),
     };
 
     cluster_opts_.extra_tserver_flags = {
@@ -430,6 +432,7 @@ class MultiMasterIdleConnectionsITest : public AuthnTokenExpireITestBase {
       // expiration of authn tokens, while the default authn expiration timeout
       // is 7 days. So, let's make the token validity interval really short.
       Substitute("--authn_token_validity_seconds=$0", token_validity_seconds_),
+      Substitute("--authz_token_validity_seconds=$0", token_validity_seconds_),
 
       // The default for leader_failure_max_missed_heartbeat_periods 3.0, but
       // 2.0 is enough to have master leadership stable enough and makes it

http://git-wip-us.apache.org/repos/asf/kudu/blob/94e484da/src/kudu/integration-tests/security-unknown-tsk-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/security-unknown-tsk-itest.cc b/src/kudu/integration-tests/security-unknown-tsk-itest.cc
index bf6416f..4bc7f44 100644
--- a/src/kudu/integration-tests/security-unknown-tsk-itest.cc
+++ b/src/kudu/integration-tests/security-unknown-tsk-itest.cc
@@ -161,7 +161,7 @@ class SecurityUnknownTskTest : public KuduTest {
       return Status::RuntimeError("failed to serialize token data");
     }
 
-    TokenSigner forger(1, 1);
+    TokenSigner forger(1, 1, 1);
     RETURN_NOT_OK(forger.ImportKeys({ tsk }));
     return forger.SignToken(new_signed_token);
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/94e484da/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 d5468de..d02c758 100644
--- a/src/kudu/integration-tests/token_signer-itest.cc
+++ b/src/kudu/integration-tests/token_signer-itest.cc
@@ -43,6 +43,7 @@
 #include "kudu/util/test_util.h"
 
 DECLARE_int64(authn_token_validity_seconds);
+DECLARE_int64(authz_token_validity_seconds);
 DECLARE_int64(tsk_rotation_seconds);
 DECLARE_int32(heartbeat_interval_ms);
 
@@ -63,6 +64,7 @@ class TokenSignerITest : public KuduTest {
  public:
   TokenSignerITest() {
     FLAGS_authn_token_validity_seconds = authn_token_validity_seconds_;
+    FLAGS_authz_token_validity_seconds = authz_token_validity_seconds_;
     FLAGS_tsk_rotation_seconds = tsk_rotation_seconds_;
 
     opts_.num_masters = 3;
@@ -121,6 +123,7 @@ class TokenSignerITest : public KuduTest {
 
  protected:
   const int64_t authn_token_validity_seconds_ = 20;
+  const int64_t authz_token_validity_seconds_ = 20;
   const int64_t tsk_rotation_seconds_ = 20;
 
   InternalMiniClusterOptions opts_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/94e484da/src/kudu/master/master.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master.cc b/src/kudu/master/master.cc
index 61efd76..ae33cd2 100644
--- a/src/kudu/master/master.cc
+++ b/src/kudu/master/master.cc
@@ -79,6 +79,12 @@ DEFINE_int64(authn_token_validity_seconds, 60 * 60 * 24 * 7,
              "validity period expires.");
 TAG_FLAG(authn_token_validity_seconds, experimental);
 
+DEFINE_int64(authz_token_validity_seconds, 60 * 5,
+             "Period of time for which an issued authorization token is valid. "
+             "Clients will automatically attempt to reacquire a token after the "
+             "validity period expires.");
+TAG_FLAG(authz_token_validity_seconds, experimental);
+
 DECLARE_bool(hive_metastore_sasl_enabled);
 DECLARE_string(keytab_file);
 
@@ -160,6 +166,7 @@ Status Master::Init() {
   // The TokenSigner loads its keys during catalog manager initialization.
   token_signer_.reset(new TokenSigner(
       FLAGS_authn_token_validity_seconds,
+      FLAGS_authz_token_validity_seconds,
       FLAGS_tsk_rotation_seconds,
       messenger_->shared_token_verifier()));
   state_ = kInitialized;

http://git-wip-us.apache.org/repos/asf/kudu/blob/94e484da/src/kudu/rpc/negotiation-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/negotiation-test.cc b/src/kudu/rpc/negotiation-test.cc
index 976f590..c463df7 100644
--- a/src/kudu/rpc/negotiation-test.cc
+++ b/src/kudu/rpc/negotiation-test.cc
@@ -199,7 +199,9 @@ TEST_P(TestNegotiation, TestNegotiation) {
   FLAGS_rpc_encrypt_loopback_connections = desc.rpc_encrypt_loopback;
 
   // Generate an optional client token and server token verifier.
-  TokenSigner token_signer(60, 20, std::make_shared<TokenVerifier>());
+  // Note: the authz token validity period doesn't matter because we're only
+  // concerned with authenticating the connection.
+  TokenSigner token_signer(60, 0, 20, std::make_shared<TokenVerifier>());
   {
     unique_ptr<TokenSigningPrivateKey> key;
     ASSERT_OK(token_signer.CheckNeedKey(&key));

http://git-wip-us.apache.org/repos/asf/kudu/blob/94e484da/src/kudu/security/token-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/token-test.cc b/src/kudu/security/token-test.cc
index 7172a51..6332c18 100644
--- a/src/kudu/security/token-test.cc
+++ b/src/kudu/security/token-test.cc
@@ -18,7 +18,9 @@
 #include <cstdint>
 #include <deque>
 #include <memory>
+#include <ostream>
 #include <string>
+#include <thread>
 #include <utility>
 #include <vector>
 
@@ -26,6 +28,7 @@
 #include <glog/logging.h>
 #include <gtest/gtest.h>
 
+#include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/walltime.h"
 #include "kudu/security/crypto.h"
 #include "kudu/security/openssl_util.h"
@@ -33,23 +36,69 @@
 #include "kudu/security/token_signer.h"
 #include "kudu/security/token_signing_key.h"
 #include "kudu/security/token_verifier.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/logging.h"
 #include "kudu/util/monotime.h"
+#include "kudu/util/pb_util.h"
 #include "kudu/util/status.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 
 DECLARE_int32(tsk_num_rsa_bits);
 
+using kudu::pb_util::SecureDebugString;
 using std::string;
-using std::make_shared;
+using std::thread;
 using std::unique_ptr;
 using std::vector;
+using strings::Substitute;
 
 namespace kudu {
 namespace security {
 
 namespace {
 
+// Dummy variables to use when their values don't matter much.
+const int kNumBits = 512;
+const int64_t kTokenValiditySeconds = 10;
+const char kUser[] = "user";
+
+// Repeatedly signs tokens and attempts to rotate TSKs until the active TSK's
+// sequence number passes `seq_num`, returning the last token signed by the TSK
+// at `seq_num`. This token is roughly the last possible token signed in the
+// TSK's activity interval.
+// The TokenGenerator 'generate_token' is a lambda that fills in a
+// SignedTokenPB and returns a Status.
+template <class TokenGenerator>
+Status SignUntilRotatePast(TokenSigner* signer, TokenGenerator generate_token,
+                           const string& token_type, int64_t seq_num,
+                           SignedTokenPB* last_signed_by_tsk) {
+  SignedTokenPB last_signed;
+  RETURN_NOT_OK_PREPEND(generate_token(&last_signed),
+      Substitute("Failed to generate first $0 token", token_type));
+  DCHECK_EQ(seq_num, last_signed.signing_key_seq_num())
+      << Substitute("Unexpected starting seq_num for $0 token", token_type);
+
+  auto cur_seq_num = seq_num;
+  while (cur_seq_num == seq_num) {
+    SleepFor(MonoDelta::FromMilliseconds(50));
+    KLOG_EVERY_N_SECS(INFO, 1) <<
+        Substitute("Generating $0 token for activity interval $1", token_type, seq_num);
+    RETURN_NOT_OK_PREPEND(signer->TryRotateKey(), "Failed to attempt to rotate key");
+    SignedTokenPB signed_token;
+    RETURN_NOT_OK_PREPEND(generate_token(&signed_token),
+        Substitute("Failed to generate $0 token", token_type));
+    // We want to return the last token signed by the `seq_num` TSK, so only
+    // update it when appropriate.
+    cur_seq_num = signed_token.signing_key_seq_num();
+    if (cur_seq_num == seq_num) {
+      last_signed = std::move(signed_token);
+    }
+  }
+  *last_signed_by_tsk = std::move(last_signed);
+  return Status::OK();
+}
+
 SignedTokenPB MakeUnsignedToken(int64_t expiration) {
   SignedTokenPB ret;
   TokenPB token;
@@ -70,7 +119,7 @@ SignedTokenPB MakeIncompatibleToken() {
 // Generate public key as a string in DER format for tests.
 Status GeneratePublicKeyStrDer(string* ret) {
   PrivateKey private_key;
-  RETURN_NOT_OK(GeneratePrivateKey(512, &private_key));
+  RETURN_NOT_OK(GeneratePrivateKey(kNumBits, &private_key));
   PublicKey public_key;
   RETURN_NOT_OK(private_key.GetPublicKey(&public_key));
   string public_key_str_der;
@@ -85,7 +134,7 @@ Status GenerateTokenSigningKey(int64_t seq_num,
                                unique_ptr<TokenSigningPrivateKey>* tsk) {
   {
     unique_ptr<PrivateKey> private_key(new PrivateKey);
-    RETURN_NOT_OK(GeneratePrivateKey(512, private_key.get()));
+    RETURN_NOT_OK(GeneratePrivateKey(kNumBits, private_key.get()));
     tsk->reset(new TokenSigningPrivateKey(
         seq_num, expire_time_seconds, std::move(private_key)));
   }
@@ -124,7 +173,7 @@ class TokenTest : public KuduTest {
 };
 
 TEST_F(TokenTest, TestInit) {
-  TokenSigner signer(10, 10);
+  TokenSigner signer(kTokenValiditySeconds, kTokenValiditySeconds, 10);
   const TokenVerifier& verifier(signer.verifier());
 
   SignedTokenPB token = MakeUnsignedToken(WallTime_Now());
@@ -133,7 +182,7 @@ TEST_F(TokenTest, TestInit) {
 
   static const int64_t kKeySeqNum = 100;
   PrivateKey private_key;
-  ASSERT_OK(GeneratePrivateKey(512, &private_key));
+  ASSERT_OK(GeneratePrivateKey(kNumBits, &private_key));
   string private_key_str_der;
   ASSERT_OK(private_key.ToString(&private_key_str_der, DataFormat::DER));
   TokenSigningPrivateKeyPB pb;
@@ -163,9 +212,10 @@ TEST_F(TokenTest, TestInit) {
 TEST_F(TokenTest, TestTokenSignerNonSparseSequenceNumbers) {
   static const int kIterNum = 3;
   static const int64_t kAuthnTokenValiditySeconds = 1;
+  static const int64_t kAuthzTokenValiditySeconds = 1;
   static const int64_t kKeyRotationSeconds = 1;
 
-  TokenSigner signer(kAuthnTokenValiditySeconds, kKeyRotationSeconds);
+  TokenSigner signer(kAuthnTokenValiditySeconds, kAuthzTokenValiditySeconds, kKeyRotationSeconds);
 
   int64_t seq_num_first_key;
   NO_FATALS(CheckAndAddNextKey(kIterNum, &signer, &seq_num_first_key));
@@ -192,12 +242,10 @@ TEST_F(TokenTest, TestTokenSignerNonSparseSequenceNumbers) {
 // import should be greater than any sequence number the TokenSigner has seen
 // during the import.
 TEST_F(TokenTest, TestTokenSignerAddKeyAfterImport) {
-  static const int64_t kAuthnTokenValiditySeconds = 8;
   static const int64_t kKeyRotationSeconds = 8;
-  static const int64_t kKeyValiditySeconds =
-      kAuthnTokenValiditySeconds + 2 * kKeyRotationSeconds;
 
-  TokenSigner signer(kAuthnTokenValiditySeconds, kKeyRotationSeconds);
+  TokenSigner signer(kTokenValiditySeconds, kTokenValiditySeconds, kKeyRotationSeconds);
+  const int64_t key_validity_seconds = signer.key_validity_seconds_;
   const TokenVerifier& verifier(signer.verifier());
 
   static const int64_t kExpiredKeySeqNum = 100;
@@ -206,7 +254,7 @@ TEST_F(TokenTest, TestTokenSignerAddKeyAfterImport) {
     // First, try to import already expired key to check that internal key
     // sequence number advances correspondingly.
     PrivateKey private_key;
-    ASSERT_OK(GeneratePrivateKey(512, &private_key));
+    ASSERT_OK(GeneratePrivateKey(kNumBits, &private_key));
     string private_key_str_der;
     ASSERT_OK(private_key.ToString(&private_key_str_der, DataFormat::DER));
     TokenSigningPrivateKeyPB pb;
@@ -215,9 +263,7 @@ TEST_F(TokenTest, TestTokenSignerAddKeyAfterImport) {
     pb.set_expire_unix_epoch_seconds(WallTime_Now() - 1);
 
     ASSERT_OK(signer.ImportKeys({pb}));
-  }
 
-  {
     // Check the result of importing keys: there should be no keys because
     // the only one we tried to import was already expired.
     vector<TokenSigningPublicKeyPB> public_keys(verifier.ExportKeys());
@@ -228,7 +274,7 @@ TEST_F(TokenTest, TestTokenSignerAddKeyAfterImport) {
     // Now import valid (not yet expired) key, but with sequence number less
     // than of the expired key.
     PrivateKey private_key;
-    ASSERT_OK(GeneratePrivateKey(512, &private_key));
+    ASSERT_OK(GeneratePrivateKey(kNumBits, &private_key));
     string private_key_str_der;
     ASSERT_OK(private_key.ToString(&private_key_str_der, DataFormat::DER));
     TokenSigningPrivateKeyPB pb;
@@ -237,19 +283,17 @@ TEST_F(TokenTest, TestTokenSignerAddKeyAfterImport) {
     // Set the TSK's expiration time: make the key valid but past its activity
     // interval.
     pb.set_expire_unix_epoch_seconds(
-        WallTime_Now() + (kKeyValiditySeconds - 2 * kKeyRotationSeconds - 1));
+        WallTime_Now() + (key_validity_seconds - 2 * kKeyRotationSeconds - 1));
 
     ASSERT_OK(signer.ImportKeys({pb}));
-  }
 
-  {
-    // Check the result of importing keys.
+    // Check the result of importing keys. The lower sequence number is
+    // accepted, even though we previously imported a key with a higher
+    // sequence number that was expired.
     vector<TokenSigningPublicKeyPB> public_keys(verifier.ExportKeys());
     ASSERT_EQ(1, public_keys.size());
     ASSERT_EQ(kKeySeqNum, public_keys[0].key_seq_num());
-  }
 
-  {
     // The newly imported key should be used to sign tokens.
     SignedTokenPB token = MakeUnsignedToken(WallTime_Now());
     ASSERT_OK(signer.SignToken(&token));
@@ -289,14 +333,18 @@ TEST_F(TokenTest, TestTokenSignerAddKeyAfterImport) {
 // less or equal to the sequence number of the most 'recent' key.
 TEST_F(TokenTest, TestAddKeyConstraints) {
   {
-    TokenSigner signer(1, 1);
+    // If a signer has not created a TSK yet, it will create a key, and will
+    // happily accept the generated key.
+    TokenSigner signer(kTokenValiditySeconds, kTokenValiditySeconds, 1);
     std::unique_ptr<TokenSigningPrivateKey> key;
     ASSERT_OK(signer.CheckNeedKey(&key));
     ASSERT_NE(nullptr, key.get());
     ASSERT_OK(signer.AddKey(std::move(key)));
   }
   {
-    TokenSigner signer(1, 1);
+    // If the key sequence number added to the signer isn't monotonically
+    // increasing, the signer will complain.
+    TokenSigner signer(kTokenValiditySeconds, kTokenValiditySeconds, 1);
     std::unique_ptr<TokenSigningPrivateKey> key;
     ASSERT_OK(signer.CheckNeedKey(&key));
     ASSERT_NE(nullptr, key.get());
@@ -308,10 +356,11 @@ TEST_F(TokenTest, TestAddKeyConstraints) {
                         ": invalid key sequence number, should be at least ");
   }
   {
-    TokenSigner signer(1, 1);
+    // Test importing expired keys. The signer should be OK with it.
+    TokenSigner signer(kTokenValiditySeconds, kTokenValiditySeconds, 1);
     static const int64_t kKeySeqNum = 100;
     PrivateKey private_key;
-    ASSERT_OK(GeneratePrivateKey(512, &private_key));
+    ASSERT_OK(GeneratePrivateKey(kNumBits, &private_key));
     string private_key_str_der;
     ASSERT_OK(private_key.ToString(&private_key_str_der, DataFormat::DER));
     TokenSigningPrivateKeyPB pb;
@@ -321,6 +370,8 @@ TEST_F(TokenTest, TestAddKeyConstraints) {
     pb.set_expire_unix_epoch_seconds(WallTime_Now() - 1);
     ASSERT_OK(signer.ImportKeys({pb}));
 
+    // Generated keys thereafter are expected to have higher sequence numbers
+    // than the imported expired keys.
     std::unique_ptr<TokenSigningPrivateKey> key;
     ASSERT_OK(signer.CheckNeedKey(&key));
     ASSERT_NE(nullptr, key.get());
@@ -334,21 +385,52 @@ TEST_F(TokenTest, TestAddKeyConstraints) {
   }
 }
 
-TEST_F(TokenTest, TestGenerateAuthTokenNoUserName) {
-  TokenSigner signer(10, 10);
+TEST_F(TokenTest, TestGenerateAuthnTokenNoUserName) {
+  TokenSigner signer(kTokenValiditySeconds, kTokenValiditySeconds, 10);
   SignedTokenPB signed_token_pb;
   const Status& s = signer.GenerateAuthnToken("", &signed_token_pb);
   EXPECT_TRUE(s.IsInvalidArgument()) << s.ToString();
   ASSERT_STR_CONTAINS(s.ToString(), "no username provided for authn token");
 }
 
+TEST_F(TokenTest, TestGenerateAuthzToken) {
+  // We cannot generate tokens with no username associated with it.
+  std::shared_ptr<TokenVerifier> verifier(new TokenVerifier());
+  TokenSigner signer(kTokenValiditySeconds, kTokenValiditySeconds, 10, verifier);
+  TablePrivilegePB table_privilege;
+  SignedTokenPB signed_token_pb;
+  Status s = signer.GenerateAuthzToken("", table_privilege, &signed_token_pb);
+  EXPECT_TRUE(s.IsInvalidArgument()) << s.ToString();
+  ASSERT_STR_CONTAINS(s.ToString(), "no username provided for authz token");
+
+  // Generated tokens will have the specified privileges.
+  const string kAuthorized = "authzed";
+  unique_ptr<TokenSigningPrivateKey> key;
+  ASSERT_OK(signer.CheckNeedKey(&key));
+  ASSERT_NE(nullptr, key.get());
+  ASSERT_OK(signer.AddKey(std::move(key)));
+  ASSERT_OK(signer.GenerateAuthzToken(kAuthorized,
+                                      table_privilege,
+                                      &signed_token_pb));
+  ASSERT_TRUE(signed_token_pb.has_token_data());
+  TokenPB token_pb;
+  ASSERT_EQ(VerificationResult::VALID,
+            verifier->VerifyTokenSignature(signed_token_pb, &token_pb));
+  ASSERT_TRUE(token_pb.has_authz());
+  ASSERT_EQ(kAuthorized, token_pb.authz().username());
+  ASSERT_TRUE(token_pb.authz().has_table_privilege());
+  ASSERT_EQ(SecureDebugString(table_privilege),
+            SecureDebugString(token_pb.authz().table_privilege()));
+}
+
 TEST_F(TokenTest, TestIsCurrentKeyValid) {
-  static const int64_t kAuthnTokenValiditySeconds = 1;
+  // This test sleeps for a key validity period, so set it up to be short.
+  static const int64_t kShortTokenValiditySeconds = 1;
   static const int64_t kKeyRotationSeconds = 1;
-  static const int64_t kKeyValiditySeconds =
-      kAuthnTokenValiditySeconds + 2 * kKeyRotationSeconds;
 
-  TokenSigner signer(kAuthnTokenValiditySeconds, kKeyRotationSeconds);
+  TokenSigner signer(kShortTokenValiditySeconds, kShortTokenValiditySeconds, kKeyRotationSeconds);
+  static const int64_t key_validity_seconds = signer.key_validity_seconds_;
+
   EXPECT_FALSE(signer.IsCurrentKeyValid());
   {
     std::unique_ptr<TokenSigningPrivateKey> key;
@@ -358,7 +440,7 @@ TEST_F(TokenTest, TestIsCurrentKeyValid) {
     ASSERT_OK(signer.AddKey(std::move(key)));
   }
   EXPECT_TRUE(signer.IsCurrentKeyValid());
-  SleepFor(MonoDelta::FromSeconds(kKeyValiditySeconds));
+  SleepFor(MonoDelta::FromSeconds(key_validity_seconds));
   // The key should expire after its validity interval.
   EXPECT_FALSE(signer.IsCurrentKeyValid());
 
@@ -369,7 +451,7 @@ TEST_F(TokenTest, TestIsCurrentKeyValid) {
 
 TEST_F(TokenTest, TestTokenSignerAddKeys) {
   {
-    TokenSigner signer(10, 10);
+    TokenSigner signer(kTokenValiditySeconds, kTokenValiditySeconds, 10);
     std::unique_ptr<TokenSigningPrivateKey> key;
     ASSERT_OK(signer.CheckNeedKey(&key));
     // No keys are available yet, so should be able to add.
@@ -384,7 +466,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(10, 0);
+    TokenSigner signer(kTokenValiditySeconds, kTokenValiditySeconds, 0);
     std::unique_ptr<TokenSigningPrivateKey> key;
     ASSERT_OK(signer.CheckNeedKey(&key));
     // No keys are available yet, so should be able to add.
@@ -406,7 +488,7 @@ TEST_F(TokenTest, TestTokenSignerAddKeys) {
     // It should not need next key right away, but should need next key after
     // the rotation interval.
     static const int64_t kKeyRotationIntervalSeconds = 8;
-    TokenSigner signer(10, kKeyRotationIntervalSeconds);
+    TokenSigner signer(kTokenValiditySeconds, kTokenValiditySeconds, kKeyRotationIntervalSeconds);
     std::unique_ptr<TokenSigningPrivateKey> key;
     ASSERT_OK(signer.CheckNeedKey(&key));
     // No keys are available yet, so should be able to add.
@@ -433,7 +515,7 @@ TEST_F(TokenTest, TestTokenSignerAddKeys) {
 // Test how key rotation works.
 TEST_F(TokenTest, TestTokenSignerSignVerifyExport) {
   // Key rotation interval 0 allows adding 2 keys in a row with no delay.
-  TokenSigner signer(10, 0);
+  TokenSigner signer(kTokenValiditySeconds, kTokenValiditySeconds, 0);
   const TokenVerifier& verifier(signer.verifier());
 
   // Should start off with no signing keys.
@@ -482,7 +564,7 @@ TEST_F(TokenTest, TestTokenSignerSignVerifyExport) {
   ASSERT_EQ(0, verifier.ExportKeys(next_signing_key_seq_num).size());
 
   // The first key should be used for signing: the next one is saved
-  // for the next round.
+  // for the next rotation.
   {
     SignedTokenPB token = MakeUnsignedToken(WallTime_Now());
     ASSERT_OK(signer.SignToken(&token));
@@ -498,7 +580,7 @@ TEST_F(TokenTest, TestExportKeys) {
   // and have an appropriate expiration.
   const int64_t key_exp_seconds = 30;
   const int64_t key_rotation_seconds = 10;
-  TokenSigner signer(key_exp_seconds - 2 * key_rotation_seconds,
+  TokenSigner signer(key_exp_seconds - 2 * key_rotation_seconds, 0,
                      key_rotation_seconds);
   int64_t key_seq_num;
   {
@@ -523,7 +605,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(10, 10);
+  TokenSigner signer(kTokenValiditySeconds, kTokenValiditySeconds, 10);
   {
     std::unique_ptr<TokenSigningPrivateKey> key;
     ASSERT_OK(signer.CheckNeedKey(&key));
@@ -546,7 +628,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(10, 0);
+  TokenSigner signer(kTokenValiditySeconds, kTokenValiditySeconds, 0);
   {
     std::unique_ptr<TokenSigningPrivateKey> key;
     ASSERT_OK(signer.CheckNeedKey(&key));
@@ -673,5 +755,131 @@ TEST_F(TokenTest, TestTokenVerifierImportKeys) {
   }
 }
 
+// Test using different token validity intervals.
+TEST_F(TokenTest, TestVaryingTokenValidityIntervals) {
+  constexpr int kShortValiditySeconds = 2;
+  const int kLongValiditySeconds = kShortValiditySeconds * 3;
+  std::shared_ptr<TokenVerifier> verifier(new TokenVerifier());
+  TokenSigner signer(kLongValiditySeconds, kShortValiditySeconds, 10, verifier);
+  unique_ptr<TokenSigningPrivateKey> key;
+  ASSERT_OK(signer.CheckNeedKey(&key));
+  ASSERT_NE(nullptr, key.get());
+  ASSERT_OK(signer.AddKey(std::move(key)));
+
+  const TablePrivilegePB table_privilege;
+  SignedTokenPB signed_authn;
+  SignedTokenPB signed_authz;
+  ASSERT_OK(signer.GenerateAuthnToken(kUser, &signed_authn));
+  ASSERT_OK(signer.GenerateAuthzToken(kUser, table_privilege, &signed_authz));
+  TokenPB authn_token;
+  TokenPB authz_token;
+  ASSERT_EQ(VerificationResult::VALID, verifier->VerifyTokenSignature(signed_authn, &authn_token));
+  ASSERT_EQ(VerificationResult::VALID, verifier->VerifyTokenSignature(signed_authz, &authz_token));
+
+  // Wait for the authz validity interval to pass and verify its expiration.
+  SleepFor(MonoDelta::FromSeconds(1 + kShortValiditySeconds));
+  EXPECT_EQ(VerificationResult::VALID, verifier->VerifyTokenSignature(signed_authn, &authn_token));
+  EXPECT_EQ(VerificationResult::EXPIRED_TOKEN,
+            verifier->VerifyTokenSignature(signed_authz, &authz_token));
+
+  // Wait for the authn validity interval to pass and verify its expiration.
+  SleepFor(MonoDelta::FromSeconds(kLongValiditySeconds - kShortValiditySeconds));
+  EXPECT_EQ(VerificationResult::EXPIRED_TOKEN,
+            verifier->VerifyTokenSignature(signed_authn, &authn_token));
+  EXPECT_EQ(VerificationResult::EXPIRED_TOKEN,
+            verifier->VerifyTokenSignature(signed_authz, &authz_token));
+}
+
+// Test to check the invariant that all tokens signed within a TSK's activity
+// interval must be expired by the end of the TSK's validity interval.
+TEST_F(TokenTest, TestKeyValidity) {
+  if (!AllowSlowTests()) {
+    LOG(WARNING) << "test is skipped; set KUDU_ALLOW_SLOW_TESTS=1 to run";
+    return;
+  }
+  // Note: this test's runtime is roughly the length of a key-validity
+  // interval, which is determined by the token validity intervals and the key
+  // rotation interval.
+  const int kShortValiditySeconds = 2;
+  const int kLongValiditySeconds = 6;
+  const int kKeyRotationSeconds = 5;
+  std::shared_ptr<TokenVerifier> verifier(new TokenVerifier());
+  TokenSigner signer(kLongValiditySeconds, kShortValiditySeconds, kKeyRotationSeconds, verifier);
+  unique_ptr<TokenSigningPrivateKey> key;
+  ASSERT_OK(signer.CheckNeedKey(&key));
+  ASSERT_NE(nullptr, key.get());
+  ASSERT_OK(signer.AddKey(std::move(key)));
+
+  // First, start a countdown for the first TSK's validity interval. Any token
+  // signed during the first TSK's activity interval must be expired once this
+  // latch counts down.
+  vector<thread> threads;
+  CountDownLatch first_tsk_validity_latch(1);
+  const double key_validity_seconds = signer.key_validity_seconds_;
+  threads.emplace_back([&first_tsk_validity_latch, key_validity_seconds] {
+    SleepFor(MonoDelta::FromSeconds(key_validity_seconds));
+    LOG(INFO) << Substitute("First TSK's validity interval of $0 secs has finished!",
+                            key_validity_seconds);
+    first_tsk_validity_latch.CountDown();
+  });
+
+  // Set up a second TSK so our threads can rotate TSKs when the time comes.
+  while (true) {
+    KLOG_EVERY_N_SECS(INFO, 1) << "Waiting for a second key...";
+    unique_ptr<TokenSigningPrivateKey> tsk;
+    ASSERT_OK(signer.CheckNeedKey(&tsk));
+    if (tsk) {
+      LOG(INFO) << "Added second key!";
+      ASSERT_OK(signer.AddKey(std::move(tsk)));
+      break;
+    }
+    SleepFor(MonoDelta::FromMilliseconds(50));
+  }
+
+  // Utility lambda to check that the token is expired.
+  const auto verify_expired = [&verifier] (const SignedTokenPB& signed_token,
+                                           const string& token_type) {
+    TokenPB token_pb;
+    const auto result = verifier->VerifyTokenSignature(signed_token, &token_pb);
+    const auto expire_secs = token_pb.expire_unix_epoch_seconds();
+    ASSERT_EQ(VerificationResult::EXPIRED_TOKEN, result)
+        << Substitute("$0 token expires at $1, currently: $2",
+                      token_type, expire_secs, WallTime_Now());
+  };
+
+  // Create a thread that repeatedly signs new authn tokens, returning the
+  // final one signed by TSK with seq_num 0. At the end of the key validity
+  // period, this token will not be valid.
+  vector<SignedTokenPB> tsks(2);
+  vector<Status> results(2);
+  threads.emplace_back([&] {
+    results[0] = SignUntilRotatePast(&signer,
+        [&] (SignedTokenPB* signed_token) {
+          return signer.GenerateAuthnToken(kUser, signed_token);
+        },
+        "authn", 0, &tsks[0]);
+    first_tsk_validity_latch.Wait();
+  });
+
+  // Do the same for authz tokens.
+  threads.emplace_back([&] {
+    SignedTokenPB last_signed_by_first_tsk;
+    results[1] = SignUntilRotatePast(&signer,
+        [&] (SignedTokenPB* signed_token) {
+          return signer.GenerateAuthzToken(kUser, TablePrivilegePB(), signed_token);
+        },
+        "authz", 0, &tsks[1]);
+    first_tsk_validity_latch.Wait();
+  });
+
+  for (auto& t : threads) {
+    t.join();
+  }
+  EXPECT_OK(results[0]);
+  EXPECT_OK(results[1]);
+  NO_FATALS(verify_expired(tsks[0], "authn"));
+  NO_FATALS(verify_expired(tsks[1], "authz"));
+}
+
 } // namespace security
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/94e484da/src/kudu/security/token.proto
----------------------------------------------------------------------
diff --git a/src/kudu/security/token.proto b/src/kudu/security/token.proto
index e27ccdb..ea63ce2 100644
--- a/src/kudu/security/token.proto
+++ b/src/kudu/security/token.proto
@@ -21,11 +21,42 @@ option java_package = "org.apache.kudu.security";
 
 import "kudu/util/pb_util.proto";
 
+message ColumnPrivilegePB {
+  // If set, the user has privileges to select and apply predicates on the
+  // column during scans.
+  optional bool scan_privilege = 1;
+};
+
+message TablePrivilegePB {
+  // The ID of the table to which the privileges apply.
+  optional string table_id = 1;
+
+  // If set, the user is authorized to select and apply predicates to all
+  // columns when scanning the table, and `column_privileges` is ignored. If
+  // unset, the user may only scan and apply predicates to columns with the
+  // privileges specified in `column_privileges`.
+  optional bool scan_privilege = 2;
+
+  // If set, the user is authorized to insert rows into the table.
+  optional bool insert_privilege= 3;
+
+  // If set, the user is authorized to update rows in the table.
+  optional bool update_privilege = 4;
+
+  // If set, the user is authorized to delete rows in the table.
+  optional bool delete_privilege = 5;
+
+  // Per-column privileges, indexed by column ID.
+  map<int32, ColumnPrivilegePB> column_privileges = 6;
+};
+
 message AuthnTokenPB {
   optional string username = 1;
 };
 
 message AuthzTokenPB {
+  optional string username = 1;
+  optional TablePrivilegePB table_privilege = 2;
 };
 
 message TokenPB {

http://git-wip-us.apache.org/repos/asf/kudu/blob/94e484da/src/kudu/security/token_signer.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/token_signer.cc b/src/kudu/security/token_signer.cc
index 08c84be..aaf1be8 100644
--- a/src/kudu/security/token_signer.cc
+++ b/src/kudu/security/token_signer.cc
@@ -56,17 +56,21 @@ namespace kudu {
 namespace security {
 
 TokenSigner::TokenSigner(int64_t authn_token_validity_seconds,
+                         int64_t authz_token_validity_seconds,
                          int64_t key_rotation_seconds,
                          shared_ptr<TokenVerifier> verifier)
     : verifier_(verifier ? std::move(verifier)
                          : std::make_shared<TokenVerifier>()),
       authn_token_validity_seconds_(authn_token_validity_seconds),
+      authz_token_validity_seconds_(authz_token_validity_seconds),
       key_rotation_seconds_(key_rotation_seconds),
       // The TSK propagation interval is equal to the rotation interval.
-      key_validity_seconds_(2 * key_rotation_seconds_ + authn_token_validity_seconds_),
+      key_validity_seconds_(2 * key_rotation_seconds_ +
+          std::max(authn_token_validity_seconds_, authz_token_validity_seconds)),
       last_key_seq_num_(-1) {
   CHECK_GE(key_rotation_seconds_, 0);
   CHECK_GE(authn_token_validity_seconds_, 0);
+  CHECK_GE(authz_token_validity_seconds_, 0);
   CHECK(verifier_);
 }
 
@@ -134,6 +138,27 @@ Status TokenSigner::ImportKeys(const vector<TokenSigningPrivateKeyPB>& keys) {
   return Status::OK();
 }
 
+Status TokenSigner::GenerateAuthzToken(string username,
+                                       TablePrivilegePB privilege,
+                                       SignedTokenPB* signed_token) const {
+  if (username.empty()) {
+    return Status::InvalidArgument("no username provided for authz token");
+  }
+  TokenPB token;
+  token.set_expire_unix_epoch_seconds(WallTime_Now() + authz_token_validity_seconds_);
+  AuthzTokenPB* authz = token.mutable_authz();
+  authz->set_username(std::move(username));
+  *authz->mutable_table_privilege() = std::move(privilege);
+
+  SignedTokenPB ret;
+  if (!token.SerializeToString(ret.mutable_token_data())) {
+    return Status::RuntimeError("could not serialize authz token");
+  }
+  RETURN_NOT_OK(SignToken(&ret));
+  *signed_token = std::move(ret);
+  return Status::OK();
+}
+
 Status TokenSigner::GenerateAuthnToken(string username,
                                        SignedTokenPB* signed_token) const {
   if (username.empty()) {
@@ -162,7 +187,7 @@ Status TokenSigner::SignToken(SignedTokenPB* token) const {
     return Status::IllegalState("no token signing key");
   }
   const TokenSigningPrivateKey* key = tsk_deque_.front().get();
-  RETURN_NOT_OK_PREPEND(key->Sign(token), "could not sign authn token");
+  RETURN_NOT_OK_PREPEND(key->Sign(token), "could not sign token");
   return Status::OK();
 }
 
@@ -194,9 +219,10 @@ Status TokenSigner::CheckNeedKey(unique_ptr<TokenSigningPrivateKey>* tsk) const
     // 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 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.
+    // in the queue: the TryRotateKey() 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.
     tsk->reset();
     return Status::OK();
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/94e484da/src/kudu/security/token_signer.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/token_signer.h b/src/kudu/security/token_signer.h
index df1e3eb..36f5a65 100644
--- a/src/kudu/security/token_signer.h
+++ b/src/kudu/security/token_signer.h
@@ -33,6 +33,7 @@ class Status;
 
 namespace security {
 class SignedTokenPB;
+class TablePrivilegePB;
 class TokenSigningPrivateKey;
 class TokenSigningPrivateKeyPB;
 class TokenVerifier;
@@ -74,16 +75,18 @@ class TokenVerifier;
 // '-' propagation interval
 //       The TSK is already created but not yet used to sign tokens. However,
 //       its public part is already being sent to the components which
-//       may be involved in validation of tokens signed by the key.
+//       may be involved in validation of tokens signed by the key. This is
+//       exactly 'tsk_propagation_interval' below.
 //
 // 'A' activity interval
 //       The TSK is used to sign tokens. It's assumed that the components which
-//       are involved in token verification have already received
-//       the corresponding public part of the TSK.
+//       are involved in token verification have already received the
+//       corresponding public part of the TSK.
 //
 // '=' inactivity interval
-//       The TSK is no longer used to sign tokens. However, it's still sent
-//       to other components which validate token signatures.
+//       The TSK is no longer used to sign tokens. However, its public part is
+//       still sent to other components and can be used to validate token
+//       signatures.
 //
 // Shortly after the TSK's expiration the token signing components stop
 // propagating its public part.
@@ -98,8 +101,8 @@ class TokenVerifier;
 //       implications, so it's worth considering rolling twice at startup.
 //
 // For example, consider the following configuration for token signing keys:
-//   validity period:      4 days
-//   rotation interval:    1 days
+//   key validity period:  4 days
+//   rotation interval:    1 day
 //   propagation interval: 1 day
 //
 // Day      1    2    3    4    5    6    7    8
@@ -111,24 +114,34 @@ class TokenVerifier;
 //                              ...............
 // authn token:                     <**********>
 //
-// 'A' indicates the 'Originator Usage Period' (a.k.a. 'Activity Interval'),
-// i.e. the period in which the key is being used to sign tokens.
+// 'A' indicates 'Activity Interval', i.e. the period during which the key is
+// being used to sign tokens. In cryptographic terms, this is the 'Originator
+// Usage Period'. Note that the Activity Interval is identically the rotation
+// interval -- a key is active for some amount of time, after which, we rotate.
 //
-// '<...>' indicates the 'Recipient Usage Period': the period in which
-// the verifier may get tokens signed by the TSK and should consider them
-// for verification. The start of the recipient usage period is not crucial
-// in that regard, but the end of that period is -- after the TSK is expired,
-// a verifier should consider tokens signed by that TSK invalid and stop
-// accepting them even if the token signature is correct and the expiration.
+// '<...>' in cryptographic terms, indicates the 'Recipient Usage Period': the
+// period during which the public part of a key is used to sign tokens, and
+// verifiers will consider tokens signed by the given key as valid. At the end
+// of this period, a verifier should consider tokens signed by the given TSK
+// invalid and stop accepting them, even if the token signature is correct. The
+// start of the period is not crucial to the validity of a token, so we don't
+// perform verification for it.
 //
 // '<***>' indicates the validity interval for an authn token.
 //
-// When configuring key rotation and authn token validity interval durations,
+// When configuring key rotation and token validity interval durations,
 // consider the following constraint:
 //
-//   max_token_validity < tsk_validity_period -
+// Eq 1.
+//   max_token_validity = tsk_validity_period -
 //       (tsk_propagation_interval + tsk_rotation_interval)
 //
+// Note how if the validity period for a token created at the end of the
+// Activity Interval were to extend any farther than the above
+// 'max_token_validity', it would be considered valid beyond the end of the
+// 'tsk_validity_period', which would break the constraint that the token only
+// be valid within the TSK's validity period.
+//
 // The idea is that the token validity interval should be contained in the
 // corresponding TSK's validity interval. If the TSK is already expired at the
 // time of token verification, the token is considered invalid and the
@@ -139,7 +152,7 @@ class TokenVerifier;
 //
 // * A TSK is issued at 00:00:00 on day 4.
 // * An authn token generated and signed by current/active TSK at 23:59:59 on
-//   day 6. That's at the very end of the TSK's activity interval.
+//   day 5. That's at the very end of the TSK's activity interval.
 // * From the diagram above it's clear that if the authn token validity
 //   interval were set to something longer than TSK inactivity interval
 //   (which is 2 days with for the specified parameters), an attempt to verify
@@ -179,23 +192,29 @@ class TokenVerifier;
 //
 class TokenSigner {
  public:
-  // The 'key_validity_seconds' and 'key_rotation_seconds' parameters define
-  // the schedule of TSK rotation. See the class comment above for details.
+  // The token validity 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'. 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 = 2 * key_rotation + authn_token_validity.
+  // The 'authn_token_validity_seconds' and 'authz_token_validity_seconds'
+  // parameters are used to specify validity intervals for the generated tokens
+  // and with 'key_rotation_seconds' it defines validity interval of the newly
+  // generated TSK:
+  //
+  // Eq 2.
+  //   key_validity =
+  //      2 * key_rotation + max(authn_token_validity, authz_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.
+  // This selects the 'max_token_validity' in Eq 1 as the higher of the authn
+  // and authz token validity intervals, and based on that, calculates the
+  // effective TSK validity period based on the provided rotation interval.
+  // See the above class comment for details.
   TokenSigner(int64_t authn_token_validity_seconds,
+              int64_t authz_token_validity_seconds,
               int64_t key_rotation_seconds,
               std::shared_ptr<TokenVerifier> verifier = nullptr);
   ~TokenSigner();
@@ -266,6 +285,16 @@ class TokenSigner {
   // See the class comment above for more information about the intended usage.
   Status TryRotateKey(bool* has_rotated = nullptr) WARN_UNUSED_RESULT;
 
+  // Populates 'signed_token' with a signed authorization token with the given
+  // 'username' and table privilege. Returns an error if 'username' is empty,
+  // or if the created authn token could not be serialized for some reason.
+  Status GenerateAuthzToken(std::string username,
+                            TablePrivilegePB privilege,
+                            SignedTokenPB* signed_token) const WARN_UNUSED_RESULT;
+
+  // Populates 'signed_token' with a signed authentication token with the given
+  // 'username'. Returns an error if 'username' is empty, or if the created
+  // authz token could not be serialized for some reason.
   Status GenerateAuthnToken(std::string username,
                             SignedTokenPB* signed_token) const WARN_UNUSED_RESULT;
 
@@ -279,6 +308,9 @@ class TokenSigner {
 
  private:
   FRIEND_TEST(TokenTest, TestEndToEnd_InvalidCases);
+  FRIEND_TEST(TokenTest, TestIsCurrentKeyValid);
+  FRIEND_TEST(TokenTest, TestTokenSignerAddKeyAfterImport);
+  FRIEND_TEST(TokenTest, TestKeyValidity);
 
   static Status GenerateSigningKey(int64_t key_seq_num,
                                    int64_t key_expiration,
@@ -286,8 +318,9 @@ class TokenSigner {
 
   std::shared_ptr<TokenVerifier> verifier_;
 
-  // Validity interval for the generated authn tokens.
+  // Validity intervals for the generated tokens.
   const int64_t authn_token_validity_seconds_;
+  const int64_t authz_token_validity_seconds_;
 
   // TSK rotation interval: number of seconds between consecutive activations
   // of new token signing keys. Note that in current implementation it defines