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/04/14 18:59:23 UTC

[1/2] kudu git commit: [mini-kdc] adapted config for krb5 1.15

Repository: kudu
Updated Branches:
  refs/heads/master b204477a4 -> 83ce8e753


[mini-kdc] adapted config for krb5 1.15

Starting with version 1.15 of Kerberos5, the krb5kdc by default listens
on TCP ports as well if not specified otherwise.  However, the mini-kdc
logic assumes the daemon listens only on UDP ports.  Explicit setting of
the kdc_tcp_ports configuration property to an empty string in krb.conf
disables opening of the TCP ports by krb5kdc.

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

Branch: refs/heads/master
Commit: b89dc9a28e7f49a1b429ca0b976643f3b16d1563
Parents: b204477
Author: Alexey Serbin <as...@cloudera.com>
Authored: Wed Apr 12 17:07:33 2017 -0700
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Thu Apr 13 22:40:51 2017 +0000

----------------------------------------------------------------------
 java/kudu-client/src/test/java/org/apache/kudu/client/MiniKdc.java | 1 +
 src/kudu/security/test/mini_kdc.cc                                 | 1 +
 2 files changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/b89dc9a2/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKdc.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKdc.java b/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKdc.java
index e58ebc1..d3db0aa 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKdc.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKdc.java
@@ -262,6 +262,7 @@ public class MiniKdc implements Closeable {
     List<String> contents = ImmutableList.of(
         "[kdcdefaults]",
         "   kdc_ports = " + options.port,
+        "   kdc_tcp_ports = \"\"",
 
         "[realms]",
         options.realm + " = {",

http://git-wip-us.apache.org/repos/asf/kudu/blob/b89dc9a2/src/kudu/security/test/mini_kdc.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/test/mini_kdc.cc b/src/kudu/security/test/mini_kdc.cc
index 1df96ac..297bb95 100644
--- a/src/kudu/security/test/mini_kdc.cc
+++ b/src/kudu/security/test/mini_kdc.cc
@@ -204,6 +204,7 @@ Status MiniKdc::CreateKdcConf() const {
   static const string kFileTemplate = R"(
 [kdcdefaults]
 kdc_ports = $2
+kdc_tcp_ports = ""
 
 [realms]
 $1 = {


[2/2] kudu git commit: [security] avoid sparse seq numbers in TokenSigner

Posted by al...@apache.org.
[security] avoid sparse seq numbers in TokenSigner

Changed the way how TokenSigner assigns sequence numbers.  The new way
of assigning sequence numbers allows to avoid sparse sequence numbers
for the CheckNeedKey()-try-to-store-key-AddKey() sequence if
the 'try-to-store-key' part fails.

Added unit test for that and some other scenarios as well.

Change-Id: Ib84f3d4f2596b3e75d1e0132f1e3362812d5799e
Reviewed-on: http://gerrit.cloudera.org:8080/6329
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/83ce8e75
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/83ce8e75
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/83ce8e75

Branch: refs/heads/master
Commit: 83ce8e7534b8576781be3bc21e5ab6747e0eb032
Parents: b89dc9a
Author: Alexey Serbin <as...@cloudera.com>
Authored: Thu Mar 9 10:46:50 2017 -0800
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Fri Apr 14 18:58:24 2017 +0000

----------------------------------------------------------------------
 src/kudu/security/token-test.cc       | 213 ++++++++++++++++++++++++++++-
 src/kudu/security/token_signer.cc     |  34 +++--
 src/kudu/security/token_signer.h      |  53 ++++---
 src/kudu/security/token_signing_key.h |   4 +
 4 files changed, 273 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/83ce8e75/src/kudu/security/token-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/token-test.cc b/src/kudu/security/token-test.cc
index 6fc2772..81bc8b9 100644
--- a/src/kudu/security/token-test.cc
+++ b/src/kudu/security/token-test.cc
@@ -81,6 +81,32 @@ Status GenerateTokenSigningKey(int64_t seq_num,
   return Status::OK();
 }
 
+void CheckAndAddNextKey(int iter_num,
+                        TokenSigner* signer,
+                        int64_t* key_seq_num) {
+  ASSERT_NE(nullptr, signer);
+  ASSERT_NE(nullptr, key_seq_num);
+  int64_t seq_num;
+  {
+    std::unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(signer->CheckNeedKey(&key));
+    ASSERT_NE(nullptr, key.get());
+    seq_num = key->key_seq_num();
+  }
+
+  for (int i = 0; i < iter_num; ++i) {
+    std::unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(signer->CheckNeedKey(&key));
+    ASSERT_NE(nullptr, key.get());
+    ASSERT_EQ(seq_num, key->key_seq_num());
+    if (i + 1 == iter_num) {
+      // Finally, add the key to the TokenSigner.
+      ASSERT_OK(signer->AddKey(std::move(key)));
+    }
+  }
+  *key_seq_num = seq_num;
+}
+
 } // anonymous namespace
 
 class TokenTest : public KuduTest {
@@ -114,7 +140,190 @@ TEST_F(TokenTest, TestInit) {
   ASSERT_TRUE(token.has_signature());
 }
 
-TEST_F(TokenTest, TestGenerateAuthToken) {
+// Verify that TokenSigner does not allow 'holes' in the sequence numbers
+// of the generated keys. The idea is to not allow sequences like '1, 5, 6'.
+// In general, calling the CheckNeedKey() method multiple times and then calling
+// the AddKey() method once should advance the key sequence number only by 1
+// regardless of number CheckNeedKey() calls.
+//
+// This is to make sure that the sequence numbers are not sparse in case if
+// running scenarios CheckNeedKey()-try-to-store-key-AddKey() over and over
+// again, given that the 'try-to-store-key' part can fail sometimes.
+TEST_F(TokenTest, TestTokenSignerNonSparseSequenceNumbers) {
+  static const int kIterNum = 3;
+  static const int64_t kAuthnTokenValiditySeconds = 1;
+  static const int64_t kKeyRotationSeconds = 1;
+
+  TokenSigner signer(kAuthnTokenValiditySeconds, kKeyRotationSeconds);
+
+  int64_t seq_num_first_key;
+  NO_FATALS(CheckAndAddNextKey(kIterNum, &signer, &seq_num_first_key));
+
+  SleepFor(MonoDelta::FromSeconds(kKeyRotationSeconds + 1));
+
+  int64_t seq_num_second_key;
+  NO_FATALS(CheckAndAddNextKey(kIterNum, &signer, &seq_num_second_key));
+
+  ASSERT_EQ(seq_num_first_key + 1, seq_num_second_key);
+}
+
+// Verify the behavior of the TokenSigner::ImportKeys() method. In general,
+// it should tolerate mix of expired and non-expired keys, even if their
+// sequence numbers are intermixed: keys with greater sequence numbers could
+// be already expired but keys with lesser sequence numbers could be still
+// valid. The idea is to correctly import TSKs generated with different
+// validity period settings. This is to address scenarios when the system
+// was run with long authn token validity interval and then switched to
+// a shorter one.
+//
+// After importing keys, the TokenSigner should contain only the valid ones.
+// In addition, the sequence number of the very first key generated after the
+// 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);
+  const TokenVerifier& verifier(signer.verifier());
+
+  static const int64_t kExpiredKeySeqNum = 100;
+  static const int64_t kKeySeqNum = kExpiredKeySeqNum - 1;
+  {
+    // 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));
+    string private_key_str_der;
+    ASSERT_OK(private_key.ToString(&private_key_str_der, DataFormat::DER));
+    TokenSigningPrivateKeyPB pb;
+    pb.set_rsa_key_der(private_key_str_der);
+    pb.set_key_seq_num(kExpiredKeySeqNum);
+    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());
+    ASSERT_TRUE(public_keys.empty());
+  }
+
+  {
+    // 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));
+    string private_key_str_der;
+    ASSERT_OK(private_key.ToString(&private_key_str_der, DataFormat::DER));
+    TokenSigningPrivateKeyPB pb;
+    pb.set_rsa_key_der(private_key_str_der);
+    pb.set_key_seq_num(kKeySeqNum);
+    // 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));
+
+    ASSERT_OK(signer.ImportKeys({pb}));
+  }
+
+  {
+    // Check the result of importing keys.
+    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));
+    ASSERT_TRUE(token.has_signature());
+    ASSERT_TRUE(token.has_signing_key_seq_num());
+    EXPECT_EQ(kKeySeqNum, token.signing_key_seq_num());
+  }
+
+  {
+    std::unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    ASSERT_NE(nullptr, key.get());
+    ASSERT_EQ(kExpiredKeySeqNum + 1, key->key_seq_num());
+    ASSERT_OK(signer.AddKey(std::move(key)));
+    bool has_rotated = false;
+    ASSERT_OK(signer.TryRotateKey(&has_rotated));
+    ASSERT_TRUE(has_rotated);
+  }
+  {
+    // Check the result of generating the new key: the identifier of the new key
+    // should be +1 increment from the identifier of the expired imported key.
+    vector<TokenSigningPublicKeyPB> public_keys(verifier.ExportKeys());
+    ASSERT_EQ(2, public_keys.size());
+    EXPECT_EQ(kKeySeqNum, public_keys[0].key_seq_num());
+    EXPECT_EQ(kExpiredKeySeqNum + 1, public_keys[1].key_seq_num());
+  }
+
+  // At this point the new key should be used to sign tokens.
+  SignedTokenPB token = MakeUnsignedToken(WallTime_Now());
+  ASSERT_OK(signer.SignToken(&token));
+  ASSERT_TRUE(token.has_signature());
+  ASSERT_TRUE(token.has_signing_key_seq_num());
+  EXPECT_EQ(kExpiredKeySeqNum + 1, token.signing_key_seq_num());
+}
+
+// The AddKey() method should not allow to add a key with the sequence number
+// less or equal to the sequence number of the most 'recent' key.
+TEST_F(TokenTest, TestAddKeyConstraints) {
+  {
+    TokenSigner signer(1, 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);
+    std::unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    ASSERT_NE(nullptr, key.get());
+    const int64_t key_seq_num = key->key_seq_num();
+    key->key_seq_num_ = key_seq_num - 1;
+    Status s = signer.AddKey(std::move(key));
+    ASSERT_TRUE(s.IsInvalidArgument()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(),
+                        ": invalid key sequence number, should be at least ");
+  }
+  {
+    TokenSigner signer(1, 1);
+    static const int64_t kKeySeqNum = 100;
+    PrivateKey private_key;
+    ASSERT_OK(GeneratePrivateKey(512, &private_key));
+    string private_key_str_der;
+    ASSERT_OK(private_key.ToString(&private_key_str_der, DataFormat::DER));
+    TokenSigningPrivateKeyPB pb;
+    pb.set_rsa_key_der(private_key_str_der);
+    pb.set_key_seq_num(kKeySeqNum);
+    // Make the key already expired.
+    pb.set_expire_unix_epoch_seconds(WallTime_Now() - 1);
+    ASSERT_OK(signer.ImportKeys({pb}));
+
+    std::unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(signer.CheckNeedKey(&key));
+    ASSERT_NE(nullptr, key.get());
+    const int64_t key_seq_num = key->key_seq_num();
+    ASSERT_GT(key_seq_num, kKeySeqNum);
+    key->key_seq_num_ = kKeySeqNum;
+    Status s = signer.AddKey(std::move(key));
+    ASSERT_TRUE(s.IsInvalidArgument()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(),
+                        ": invalid key sequence number, should be at least ");
+  }
+}
+
+TEST_F(TokenTest, TestGenerateAuthTokenNoUserName) {
   TokenSigner signer(10, 10);
   SignedTokenPB signed_token_pb;
   const Status& s = signer.GenerateAuthnToken("", &signed_token_pb);
@@ -210,7 +419,7 @@ TEST_F(TokenTest, TestTokenSignerAddKeys) {
   }
 }
 
-// Test how test rotation works.
+// 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);

http://git-wip-us.apache.org/repos/asf/kudu/blob/83ce8e75/src/kudu/security/token_signer.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/token_signer.cc b/src/kudu/security/token_signer.cc
index 3cfa382..a60d116 100644
--- a/src/kudu/security/token_signer.cc
+++ b/src/kudu/security/token_signer.cc
@@ -27,6 +27,7 @@
 
 #include <gflags/gflags.h>
 
+#include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/walltime.h"
 #include "kudu/security/openssl_util.h"
 #include "kudu/security/token.pb.h"
@@ -47,6 +48,7 @@ using std::string;
 using std::unique_lock;
 using std::unique_ptr;
 using std::vector;
+using strings::Substitute;
 
 namespace kudu {
 namespace security {
@@ -60,7 +62,7 @@ TokenSigner::TokenSigner(int64_t authn_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_),
-      next_key_seq_num_(0) {
+      last_key_seq_num_(-1) {
   CHECK_GE(key_rotation_seconds_, 0);
   CHECK_GE(authn_token_validity_seconds_, 0);
   CHECK(verifier_);
@@ -85,8 +87,13 @@ Status TokenSigner::ImportKeys(const vector<TokenSigningPrivateKeyPB>& keys) {
     const int64_t key_seq_num = key.key_seq_num();
     unique_ptr<TokenSigningPrivateKey> tsk(new TokenSigningPrivateKey(key));
 
-    // Advance the key sequence number, if needed.
-    next_key_seq_num_ = std::max(next_key_seq_num_, key_seq_num + 1);
+    // Advance the key sequence number, if needed. For the use case when the
+    // history of keys sequence numbers is important, the generated keys are
+    // persisted when TokenSigner is active and then the keys are imported from
+    // the store when TokenSigner is initialized (e.g., on restart). It's
+    // crucial to take into account sequence numbers of all previously persisted
+    // keys even if they have expired at the moment of importing.
+    last_key_seq_num_ = std::max(last_key_seq_num_, key_seq_num);
     const int64_t key_expire_time = tsk->expire_time();
     if (key_expire_time <= now) {
       // Do nothing else with an expired TSK.
@@ -172,7 +179,7 @@ Status TokenSigner::CheckNeedKey(unique_ptr<TokenSigningPrivateKey>* tsk) const
   unique_lock<RWMutex> l(lock_);
   if (tsk_deque_.empty()) {
     // No active key: need a new one.
-    const int64 key_seq_num = next_key_seq_num_++;
+    const int64 key_seq_num = last_key_seq_num_ + 1;
     const int64 key_expiration = now + key_validity_seconds_;
     // Generation of cryptographically strong key takes many CPU cycles;
     // do not want to block other parallel activity.
@@ -204,7 +211,7 @@ Status TokenSigner::CheckNeedKey(unique_ptr<TokenSigningPrivateKey>* tsk) const
   const auto key_creation_time = key->expire_time() - key_validity_seconds_;
   if (key_creation_time + key_rotation_seconds_ <= now) {
     // It's time to create and start propagating next key.
-    const int64 key_seq_num = next_key_seq_num_++;
+    const int64 key_seq_num = last_key_seq_num_ + 1;
     const int64 key_expiration = now + key_validity_seconds_;
     // Generation of cryptographically strong key takes many CPU cycles:
     // do not want to block other parallel activity.
@@ -219,16 +226,21 @@ Status TokenSigner::CheckNeedKey(unique_ptr<TokenSigningPrivateKey>* tsk) const
 
 Status TokenSigner::AddKey(unique_ptr<TokenSigningPrivateKey> tsk) {
   CHECK(tsk);
-  const int64_t seq_num = tsk->key_seq_num();
-  if (seq_num < 0) {
-    return Status::InvalidArgument("invalid key sequence number");
-  }
+  const int64_t key_seq_num = tsk->key_seq_num();
   if (tsk->expire_time() <= WallTime_Now()) {
     return Status::InvalidArgument("key has already expired");
   }
 
   lock_guard<RWMutex> l(lock_);
-  next_key_seq_num_ = std::max(next_key_seq_num_, seq_num + 1);
+  if (key_seq_num < last_key_seq_num_ + 1) {
+    // The AddKey() method is designed for adding new keys: that should be done
+    // using CheckNeedKey()/AddKey() sequence. Use the ImportKeys() method
+    // for importing keys in bulk.
+    return Status::InvalidArgument(
+        Substitute("$0: invalid key sequence number, should be at least $1",
+                   key_seq_num, last_key_seq_num_ + 1));
+  }
+  last_key_seq_num_ = std::max(last_key_seq_num_, key_seq_num);
   // Register the public part of the key in TokenVerifier first.
   TokenSigningPublicKeyPB public_key_pb;
   tsk->ExportPublicKeyPB(&public_key_pb);
@@ -245,7 +257,7 @@ Status TokenSigner::TryRotateKey(bool* has_rotated) {
     *has_rotated = false;
   }
   if (tsk_deque_.size() < 2) {
-    // No next key to rotate to.
+    // There isn't next key to rotate to.
     return Status::OK();
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/83ce8e75/src/kudu/security/token_signer.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/token_signer.h b/src/kudu/security/token_signer.h
index a09a075..3714b90 100644
--- a/src/kudu/security/token_signer.h
+++ b/src/kudu/security/token_signer.h
@@ -51,7 +51,7 @@ class TokenVerifier;
 // (a.k.a. next key) is not used. Rather, the second-most-recent key, if exists,
 // is used. This ensures that there is plenty of time to transmit the public
 // part of the new TSK to all TokenVerifiers (e.g. on other servers via
-// heatbeats or by other means), before the new key enters usage.
+// heartbeats or by other means), before the new key enters usage.
 //
 // On a fresh instance, with only one key, there is no "second most recent"
 // key. Thus, we fall back to signing tokens with the only available key.
@@ -64,7 +64,7 @@ class TokenVerifier;
 // key rotation is performed more frequently than the validity period
 // of the key, so that at any given point in time there are several valid keys.
 //
-// Below is the lifecycle of a TSK (token signing key):
+// Below is the life cycle of a TSK (token signing key):
 //
 //      <---AAAAA===============>
 //      ^                       ^
@@ -213,25 +213,46 @@ class TokenSigner {
 
   // Check whether it's time to generate and add a new key. If so, the new key
   // is generated and output into the 'tsk' parameter so it's possible to
-  // examine and otherwise process the key as needed (e.g. store it).
-  // After that, use AddKey() method to actually add the key into the
-  // TokenSigner's key queue.
+  // examine and process the key as needed (e.g. store it). After that, use the
+  // AddKey() method to actually add the key into the TokenSigner's key queue.
   //
-  // Every non-null key returned by this methods has key sequence number.
-  // The key sequence number always increases with newly generated keys.
+  // Every non-null key returned by this method has key sequence number.
   // It's not a problem to call this method multiple times but call the AddKey()
   // method only once, effectively discarding all the generated keys except for
-  // the key passed to the AddKey() call as a parameter. In other words,
-  // it's possible and not a problem to have 'holes' in the key sequence
-  // numbers. Other components working with verification of the signed tokens
-  // should take that into account.
+  // the key passed to the AddKey() call as a parameter. The key sequence number
+  // always increments with every newly added key (i.e. every successful call of
+  // the AddKey() method). The result key number sequence would not contain
+  // any 'holes'.
+  //
+  // In other words, sequence of calls like
+  //
+  //   CheckNeedKey(k);
+  //   CheckNeedKey(k);
+  //   ...
+  //   CheckNeedKey(k);
+  //   AddKey(k);
+  //
+  // would increase the key sequence number just by 1. Due to that fact, the
+  // following sequence of calls to CheckNeedKey()/AddKey() would work fine:
+  //
+  //   CheckNeedKey(k0);
+  //   AddKey(k0);
+  //   CheckNeedKey(k1);
+  //   AddKey(k1);
+  //
+  // but the sequence below would fail at AddKey(k1):
+  //
+  //   CheckNeedKey(k0);
+  //   CheckNeedKey(k1);
+  //   AddKey(k0);
+  //   AddKey(k1);
   //
   // See the class comment above for more information about the intended usage.
   Status CheckNeedKey(std::unique_ptr<TokenSigningPrivateKey>* tsk) const
       WARN_UNUSED_RESULT;
 
   // Add the new key into the token signing keys queue. Call TryRotateKey()
-  // to make this key active when it's time.
+  // to make the newly added key active when it's time.
   //
   // See the class comment above for more information about the intended usage.
   Status AddKey(std::unique_ptr<TokenSigningPrivateKey> tsk) WARN_UNUSED_RESULT;
@@ -282,12 +303,8 @@ class TokenSigner {
   // Protects next_seq_num_ and tsk_deque_ members.
   mutable RWMutex lock_;
 
-  // The sequence number to assign to next generated key.
-  // It's allowable to have 'holes' in the key sequence numbers, i.e. it's
-  // acceptable to have sequence numbers which do not correspond to any
-  // existing TSK. The only crucial point is to keep the key sequence numbers
-  // increasing.
-  mutable int64_t next_key_seq_num_;
+  // The sequence number of the last generated/imported key.
+  int64_t last_key_seq_num_;
 
   // The currently active key is in the front of the queue,
   // the newly added ones are pushed into back of the queue.

http://git-wip-us.apache.org/repos/asf/kudu/blob/83ce8e75/src/kudu/security/token_signing_key.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/token_signing_key.h b/src/kudu/security/token_signing_key.h
index 2ae6713..b3cd658 100644
--- a/src/kudu/security/token_signing_key.h
+++ b/src/kudu/security/token_signing_key.h
@@ -19,6 +19,8 @@
 #include <memory>
 #include <string>
 
+#include <gtest/gtest_prod.h>
+
 #include "kudu/gutil/macros.h"
 #include "kudu/security/crypto.h"
 #include "kudu/security/openssl_util.h"
@@ -81,6 +83,8 @@ class TokenSigningPrivateKey {
   int64_t expire_time() const { return expire_time_; }
 
  private:
+  FRIEND_TEST(TokenTest, TestAddKeyConstraints);
+
   std::unique_ptr<PrivateKey> key_;
   // The 'private_key_der_' is a serialized 'key_' in DER format: just a cache.
   std::string private_key_der_;