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 2023/04/03 18:38:58 UTC

[kudu] branch master updated: jwt: Additional test

This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
     new df4cd8195 jwt: Additional test
df4cd8195 is described below

commit df4cd81953e70af53aa6fee436dbc81db6f1fe96
Author: Zoltan Chovan <zc...@cloudera.com>
AuthorDate: Thu Sep 29 13:59:26 2022 +0200

    jwt: Additional test
    
    * Added JWT scenarios to the negiotiation test
    * Updated the security-itest with JWT expiration scenario
    * Updated MiniOidc to set expires_at and not_before properties of the
    generated JWTs
    * Added methods to be able to manually create JWT and JWKS files
    
    Change-Id: I1977c80b70fd9628ac800671f6cf16e9fa96c0f0
    Reviewed-on: http://gerrit.cloudera.org:8080/19156
    Tested-by: Kudu Jenkins
    Reviewed-by: Alexey Serbin <al...@apache.org>
---
 src/kudu/integration-tests/security-itest.cc   |  28 +-
 src/kudu/mini-cluster/external_mini_cluster.cc |   6 +-
 src/kudu/rpc/CMakeLists.txt                    |   4 +-
 src/kudu/rpc/negotiation-test.cc               | 396 +++++++++++++++++++++++--
 src/kudu/rpc/negotiation.cc                    |  11 +-
 src/kudu/rpc/rpc-test-base.h                   |   1 +
 src/kudu/rpc/server_negotiation.cc             |   1 -
 src/kudu/server/server_base.cc                 |   2 +-
 src/kudu/util/CMakeLists.txt                   |  19 +-
 src/kudu/util/jwt-util-test.cc                 |  12 +-
 src/kudu/util/jwt_test_certs.cc                |  46 +++
 src/kudu/util/jwt_test_certs.h                 |  11 +
 src/kudu/util/mini_oidc.cc                     |  57 ++--
 src/kudu/util/mini_oidc.h                      |   6 +-
 14 files changed, 519 insertions(+), 81 deletions(-)

diff --git a/src/kudu/integration-tests/security-itest.cc b/src/kudu/integration-tests/security-itest.cc
index 2d520e1b6..cdc642f18 100644
--- a/src/kudu/integration-tests/security-itest.cc
+++ b/src/kudu/integration-tests/security-itest.cc
@@ -513,28 +513,34 @@ void GetFullBinaryPath(string* binary) {
 }
 
 TEST_F(SecurityITest, TestJwtMiniCluster) {
+  SKIP_IF_SLOW_NOT_ALLOWED();
+
   cluster_opts_.enable_kerberos = false;
   cluster_opts_.num_tablet_servers = 0;
   cluster_opts_.enable_client_jwt = true;
   MiniOidcOptions oidc_opts;
   const auto* const kValidAccount = "valid";
   const auto* const kInvalidAccount = "invalid";
+  const uint64_t kLifetimeMs = 1000;
   oidc_opts.account_ids = {
     { kValidAccount, true },
     { kInvalidAccount, false },
   };
-
+  oidc_opts.lifetime_ms = kLifetimeMs;
   cluster_opts_.mini_oidc_options = std::move(oidc_opts);
   ASSERT_OK(StartCluster());
   const auto* const kSubject = "kudu-user";
-  const auto configure_builder_for = [&] (const string& account_id, KuduClientBuilder* b) {
+  const auto configure_builder_for =
+      [&] (const string& account_id, KuduClientBuilder* b, const uint64_t delay_ms) {
     client::AuthenticationCredentialsPB pb;
     security::JwtRawPB jwt = security::JwtRawPB();
-    *jwt.mutable_jwt_data() = MiniOidc::CreateJwt(account_id, kSubject, true);
+    *jwt.mutable_jwt_data() = cluster_->oidc()->CreateJwt(account_id, kSubject, true);
     *pb.mutable_jwt() = std::move(jwt);
     string creds;
     CHECK(pb.SerializeToString(&creds));
 
+    SleepFor(MonoDelta::FromMilliseconds(delay_ms));
+
     for (auto i = 0; i < cluster_->num_masters(); ++i) {
       b->add_master_server_addr(cluster_->master(i)->bound_rpc_addr().ToString());
     }
@@ -545,7 +551,7 @@ TEST_F(SecurityITest, TestJwtMiniCluster) {
   {
     KuduClientBuilder valid_builder;
     shared_ptr<KuduClient> client;
-    configure_builder_for(kValidAccount, &valid_builder);
+    configure_builder_for(kValidAccount, &valid_builder, 0);
     ASSERT_OK(valid_builder.Build(&client));
     vector<string> tables;
     ASSERT_OK(client->ListTables(&tables));
@@ -553,11 +559,19 @@ TEST_F(SecurityITest, TestJwtMiniCluster) {
   {
     KuduClientBuilder invalid_builder;
     shared_ptr<KuduClient> client;
-    configure_builder_for(kInvalidAccount, &invalid_builder);
+    configure_builder_for(kInvalidAccount, &invalid_builder, 0);
     Status s = invalid_builder.Build(&client);
-    ASSERT_FALSE(s.ok()) << s.ToString();
+    ASSERT_TRUE(s.IsRuntimeError());
     ASSERT_STR_CONTAINS(s.ToString(), "FATAL_INVALID_JWT");
   }
+  {
+    KuduClientBuilder timeout_builder;
+    shared_ptr<KuduClient> client;
+    configure_builder_for(kValidAccount, &timeout_builder, 3 * kLifetimeMs);
+    Status s = timeout_builder.Build(&client);
+    ASSERT_TRUE(s.IsRuntimeError());
+    ASSERT_STR_CONTAINS(s.ToString(), "token expired");
+  }
   {
     KuduClientBuilder no_jwt_builder;
     shared_ptr<KuduClient> client;
@@ -566,7 +580,7 @@ TEST_F(SecurityITest, TestJwtMiniCluster) {
     }
     no_jwt_builder.require_authentication(true);
     Status s = no_jwt_builder.Build(&client);
-    ASSERT_FALSE(s.ok()) << s.ToString();
+    ASSERT_TRUE(s. IsNotAuthorized());
     ASSERT_STR_CONTAINS(s.ToString(), "Not authorized");
   }
 }
diff --git a/src/kudu/mini-cluster/external_mini_cluster.cc b/src/kudu/mini-cluster/external_mini_cluster.cc
index 6db49b86e..948508e35 100644
--- a/src/kudu/mini-cluster/external_mini_cluster.cc
+++ b/src/kudu/mini-cluster/external_mini_cluster.cc
@@ -84,6 +84,10 @@
 #include "kudu/util/subprocess.h"
 #include "kudu/util/test_util.h"
 
+namespace kudu {
+class JwtVerifier;
+}  // namespace kudu
+
 using kudu::client::internal::ConnectToClusterRpc;
 #if !defined(NO_CHRONY)
 using kudu::clock::MiniChronyd;
@@ -269,7 +273,7 @@ Status ExternalMiniCluster::Start() {
   gflags::FlagSaver saver;
   FLAGS_dns_addr_resolution_override = dns_overrides_;
 
-  std::shared_ptr<PerAccountKeyBasedJwtVerifier> jwt_verifier = nullptr;
+  std::shared_ptr<JwtVerifier> jwt_verifier = nullptr;
   if (opts_.enable_client_jwt) {
     oidc_.reset(new MiniOidc(opts_.mini_oidc_options));
     RETURN_NOT_OK_PREPEND(oidc_->Start(), "Failed to start OIDC endpoints");
diff --git a/src/kudu/rpc/CMakeLists.txt b/src/kudu/rpc/CMakeLists.txt
index c8d831e64..6fb27b4c3 100644
--- a/src/kudu/rpc/CMakeLists.txt
+++ b/src/kudu/rpc/CMakeLists.txt
@@ -124,7 +124,9 @@ SET_KUDU_TEST_LINK_LIBS(
   mini_kdc
   rpc_header_proto
   rtest_krpc
-  security_test_util)
+  security_test_util
+  kudu_jwt_util
+  jwt_test_certs)
 ADD_KUDU_TEST(exactly_once_rpc-test PROCESSORS 10)
 ADD_KUDU_TEST(mt-rpc-test RUN_SERIAL true)
 ADD_KUDU_TEST(negotiation-test)
diff --git a/src/kudu/rpc/negotiation-test.cc b/src/kudu/rpc/negotiation-test.cc
index 58d4b9cbb..9a635d3e8 100644
--- a/src/kudu/rpc/negotiation-test.cc
+++ b/src/kudu/rpc/negotiation-test.cc
@@ -59,9 +59,12 @@
 #include "kudu/security/token_signing_key.h"
 #include "kudu/security/token_verifier.h"
 #include "kudu/util/env.h"
+#include "kudu/util/jwt-util.h"
+#include "kudu/util/jwt_test_certs.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/net/socket.h"
+#include "kudu/util/path_util.h"
 #include "kudu/util/status.h"
 #include "kudu/util/subprocess.h"
 #include "kudu/util/test_macros.h"
@@ -69,6 +72,10 @@
 #include "kudu/util/trace.h"
 #include "kudu/util/user.h"
 
+namespace kudu {
+class JwtVerifier;
+}  // namespace kudu
+
 // HACK: MIT Kerberos doesn't have any way of determining its version number,
 // but the error messages in krb5-1.10 and earlier are broken due to
 // a bug: http://krbdev.mit.edu/rt/Ticket/Display.html?id=6973
@@ -121,6 +128,9 @@ struct EndpointConfig {
   // For the client, whether the client has the token.
   // For the server, whether the server has the TSK.
   bool token;
+  // For the client, whether the client has the JWT.
+  // For the server, whether the server has a JWTVerifier
+  bool jwt;
   RpcEncryption encryption;
 };
 std::ostream& operator<<(std::ostream& o, EndpointConfig config) {
@@ -128,6 +138,7 @@ std::ostream& operator<<(std::ostream& o, EndpointConfig config) {
   o << "{pki: " << config.pki
     << ", sasl-mechs: [" << JoinMapped(config.sasl_mechs, SaslMechanism::name_of, ", ")
     << "], token: " << bool_string(config.token)
+    << ", jwt: " << bool_string(config.jwt)
     << ", encryption: ";
 
   switch (config.encryption) {
@@ -232,6 +243,24 @@ TEST_P(TestNegotiation, TestNegotiation) {
     ASSERT_OK(token_verifier.ImportKeys(token_signer.verifier().ExportKeys()));
   }
 
+  // Create JWT token
+  // Create jwt_verifier
+  const string jwks_file_name = "keys.jwks";
+  const string jwt_test_dir = GetTestPath("jwt");
+  const string jwt_data = kudu::CreateTestJWT(true);
+  ASSERT_OK(kudu::CreateTestJWKSFile(jwt_test_dir, jwks_file_name));
+  std::shared_ptr<JwtVerifier> jwt_verifier;
+  if (desc.server.jwt) {
+    jwt_verifier = std::make_shared<kudu::KeyBasedJwtVerifier>(
+        JoinPathSegments(jwt_test_dir, jwks_file_name), /* is_local_file */ true);
+    ASSERT_OK(jwt_verifier-> Init());
+  }
+  optional<security::JwtRawPB> jwt_token;
+  if (desc.client.jwt) {
+    jwt_token = security::JwtRawPB();
+    jwt_token->set_jwt_data(jwt_data);
+  }
+
   // Create the listening socket, client socket, and server socket.
   Socket listening_socket;
   Sockaddr server_addr = Sockaddr::Wildcard();
@@ -254,14 +283,14 @@ TEST_P(TestNegotiation, TestNegotiation) {
   ClientNegotiation client_negotiation(std::move(client_socket),
                                        &client_tls_context,
                                        authn_token,
-                                       std::nullopt,
+                                       jwt_token,
                                        desc.client.encryption,
                                        desc.rpc_encrypt_loopback,
                                        "kudu");
   ServerNegotiation server_negotiation(std::move(server_socket),
                                        &server_tls_context,
                                        &token_verifier,
-                                       nullptr,
+                                       jwt_verifier.get(),
                                        desc.server.encryption,
                                        desc.rpc_encrypt_loopback,
                                        "kudu");
@@ -416,12 +445,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::NONE,
             {},
             false,
+            false,
             RpcEncryption::OPTIONAL,
           },
           EndpointConfig {
             PkiConfig::NONE,
             {},
             false,
+            false,
             RpcEncryption::OPTIONAL,
           },
           false,
@@ -440,12 +471,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::NONE,
             { SaslMechanism::PLAIN },
             false,
+            false,
             RpcEncryption::OPTIONAL,
           },
           EndpointConfig {
             PkiConfig::NONE,
             {},
             false,
+            false,
             RpcEncryption::OPTIONAL,
           },
           false,
@@ -464,12 +497,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::NONE,
             { SaslMechanism::PLAIN },
             false,
+            false,
             RpcEncryption::OPTIONAL
           },
           EndpointConfig {
             PkiConfig::NONE,
             { SaslMechanism::PLAIN },
             false,
+            false,
             RpcEncryption::DISABLED,
           },
           false,
@@ -488,12 +523,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::NONE,
             { SaslMechanism::GSSAPI },
             false,
+            false,
             RpcEncryption::OPTIONAL,
           },
           EndpointConfig {
             PkiConfig::NONE,
             { SaslMechanism::GSSAPI },
             false,
+            false,
             RpcEncryption::DISABLED,
           },
           false,
@@ -512,12 +549,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::NONE,
             { SaslMechanism::GSSAPI, SaslMechanism::PLAIN },
             false,
+            false,
             RpcEncryption::OPTIONAL,
           },
           EndpointConfig {
             PkiConfig::NONE,
             { SaslMechanism::GSSAPI, SaslMechanism::PLAIN },
             false,
+            false,
             RpcEncryption::DISABLED,
           },
           false,
@@ -536,12 +575,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::NONE,
             { SaslMechanism::GSSAPI, SaslMechanism::PLAIN },
             false,
+            false,
             RpcEncryption::OPTIONAL,
           },
           EndpointConfig {
             PkiConfig::NONE,
             { SaslMechanism::GSSAPI },
             false,
+            false,
             RpcEncryption::DISABLED,
           },
           false,
@@ -560,12 +601,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::NONE,
             { SaslMechanism::PLAIN },
             false,
+            false,
             RpcEncryption::OPTIONAL,
           },
           EndpointConfig {
             PkiConfig::NONE,
             { SaslMechanism::GSSAPI },
             false,
+            false,
             RpcEncryption::DISABLED,
           },
           false,
@@ -585,12 +628,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::NONE,
             { SaslMechanism::GSSAPI },
             false,
+            false,
             RpcEncryption::OPTIONAL,
           },
           EndpointConfig {
             PkiConfig::SELF_SIGNED,
             { SaslMechanism::GSSAPI },
             false,
+            false,
             RpcEncryption::OPTIONAL,
           },
           false,
@@ -611,12 +656,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::SIGNED,
             { SaslMechanism::GSSAPI },
             false,
+            false,
             RpcEncryption::OPTIONAL,
           },
           EndpointConfig {
             PkiConfig::SELF_SIGNED,
             { SaslMechanism::GSSAPI },
             false,
+            false,
             RpcEncryption::OPTIONAL,
           },
           false,
@@ -635,12 +682,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::NONE,
             { SaslMechanism::PLAIN },
             false,
+            false,
             RpcEncryption::OPTIONAL,
           },
           EndpointConfig {
             PkiConfig::SELF_SIGNED,
             { SaslMechanism::PLAIN },
             false,
+            false,
             RpcEncryption::OPTIONAL,
           },
           false,
@@ -659,12 +708,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::SIGNED,
             { SaslMechanism::GSSAPI },
             false,
+            false,
             RpcEncryption::OPTIONAL,
           },
           EndpointConfig {
             PkiConfig::SIGNED,
             { SaslMechanism::GSSAPI },
             false,
+            false,
             RpcEncryption::OPTIONAL,
           },
           false,
@@ -683,12 +734,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::TRUSTED,
             { },
             true,
+            false,
             RpcEncryption::OPTIONAL,
           },
           EndpointConfig {
             PkiConfig::SIGNED,
             { SaslMechanism::PLAIN },
             true,
+            false,
             RpcEncryption::OPTIONAL,
           },
           false,
@@ -710,12 +763,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::NONE,
             { SaslMechanism::PLAIN },
             true,
+            false,
             RpcEncryption::OPTIONAL,
           },
           EndpointConfig {
             PkiConfig::SIGNED,
             { SaslMechanism::PLAIN },
             true,
+            false,
             RpcEncryption::OPTIONAL,
           },
           false,
@@ -734,12 +789,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::SIGNED,
             { SaslMechanism::PLAIN, SaslMechanism::GSSAPI },
             true,
+            false,
             RpcEncryption::OPTIONAL,
           },
           EndpointConfig {
             PkiConfig::SIGNED,
             { SaslMechanism::PLAIN, SaslMechanism::GSSAPI },
             true,
+            false,
             RpcEncryption::OPTIONAL,
           },
           false,
@@ -758,12 +815,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::NONE,
             { SaslMechanism::PLAIN },
             false,
+            false,
             RpcEncryption::DISABLED,
           },
           EndpointConfig {
             PkiConfig::SIGNED,
             { SaslMechanism::PLAIN },
             false,
+            false,
             RpcEncryption::REQUIRED,
           },
           false,
@@ -782,12 +841,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::NONE,
             { SaslMechanism::PLAIN },
             false,
+            false,
             RpcEncryption::REQUIRED,
           },
           EndpointConfig {
             PkiConfig::SIGNED,
             { SaslMechanism::PLAIN },
             false,
+            false,
             RpcEncryption::DISABLED,
           },
           false,
@@ -806,12 +867,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::EXTERNALLY_SIGNED,
             { SaslMechanism::GSSAPI },
             false,
+            false,
             RpcEncryption::REQUIRED,
           },
           EndpointConfig {
             PkiConfig::EXTERNALLY_SIGNED,
             { SaslMechanism::GSSAPI },
             false,
+            false,
             RpcEncryption::REQUIRED,
           },
           false,
@@ -830,12 +893,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::EXTERNALLY_SIGNED,
             { SaslMechanism::GSSAPI },
             false,
+            false,
             RpcEncryption::OPTIONAL,
           },
           EndpointConfig {
             PkiConfig::SIGNED,
             { SaslMechanism::GSSAPI },
             false,
+            false,
             RpcEncryption::REQUIRED,
           },
           false,
@@ -854,12 +919,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::NONE,
             { SaslMechanism::GSSAPI },
             false,
+            false,
             RpcEncryption::REQUIRED,
           },
           EndpointConfig {
             PkiConfig::EXTERNALLY_SIGNED,
             { SaslMechanism::GSSAPI },
             false,
+            false,
             RpcEncryption::REQUIRED,
           },
           false,
@@ -878,12 +945,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::EXTERNALLY_SIGNED,
             { SaslMechanism::GSSAPI, SaslMechanism::PLAIN },
             false,
+            false,
             RpcEncryption::REQUIRED,
           },
           EndpointConfig {
             PkiConfig::EXTERNALLY_SIGNED,
             { SaslMechanism::PLAIN },
             false,
+            false,
             RpcEncryption::REQUIRED,
           },
           false,
@@ -902,12 +971,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::SIGNED,
             { SaslMechanism::GSSAPI },
             false,
+            false,
             RpcEncryption::DISABLED,
           },
           EndpointConfig {
             PkiConfig::EXTERNALLY_SIGNED,
             { SaslMechanism::GSSAPI },
             false,
+            false,
             RpcEncryption::REQUIRED,
           },
           false,
@@ -926,12 +997,14 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
             PkiConfig::SIGNED,
             { SaslMechanism::GSSAPI },
             false,
+            false,
             RpcEncryption::REQUIRED,
           },
           EndpointConfig {
             PkiConfig::EXTERNALLY_SIGNED,
             { SaslMechanism::GSSAPI },
             false,
+            false,
             RpcEncryption::REQUIRED,
           },
           false,
@@ -948,16 +1021,18 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
         // connection from public routable IP
         NegotiationDescriptor {
             EndpointConfig {
-                PkiConfig::NONE,
-                { SaslMechanism::PLAIN },
-                false,
-                RpcEncryption::OPTIONAL
+              PkiConfig::NONE,
+              { SaslMechanism::PLAIN },
+              false,
+              false,
+              RpcEncryption::OPTIONAL
             },
             EndpointConfig {
-                PkiConfig::NONE,
-                { SaslMechanism::PLAIN },
-                false,
-                RpcEncryption::OPTIONAL
+              PkiConfig::NONE,
+              { SaslMechanism::PLAIN },
+              false,
+              false,
+              RpcEncryption::OPTIONAL
             },
             true,
             false,
@@ -972,27 +1047,274 @@ INSTANTIATE_TEST_SUITE_P(NegotiationCombinations,
         // server: GSSAPI, TLS required, externally-signed cert
         // connection from public routable IP
         NegotiationDescriptor {
-            EndpointConfig {
-                PkiConfig::EXTERNALLY_SIGNED,
-                { SaslMechanism::GSSAPI },
-                false,
-                RpcEncryption::REQUIRED,
-            },
-            EndpointConfig {
-                PkiConfig::EXTERNALLY_SIGNED,
-                { SaslMechanism::GSSAPI },
-                false,
-                RpcEncryption::REQUIRED,
-            },
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          true,
+          // true as no longer a loopback connection.
+          true,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          true,
+        },
+
+
+        // client: JWT, TLS required
+        // server: JWT, TLS required
+        // connecting with JWT only
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { },
+            false,
             true,
-            // true as no longer a loopback connection.
+            RpcEncryption::REQUIRED
+          },
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { },
+            false,
             true,
-            Status::OK(),
-            Status::OK(),
-            AuthenticationType::SASL,
-            SaslMechanism::GSSAPI,
+            RpcEncryption::REQUIRED
+          },
+          true,
+          true,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::JWT,
+          SaslMechanism::INVALID,
+          true,
+        },
+
+
+        // client: JWT, TLS required, self-signed cert
+        // server: JWT, TLS required, self-signed cert
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::SELF_SIGNED,
+            { },
+            false,
             true,
-        }
+            RpcEncryption::REQUIRED
+          },
+          EndpointConfig {
+            PkiConfig::SELF_SIGNED,
+            {  },
+            false,
+            true,
+            RpcEncryption::REQUIRED
+          },
+          true,
+          true,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::JWT,
+          SaslMechanism::INVALID,
+          true,
+        },
+
+
+        // client: GSSAPI, JWT, TLS required
+        // server: JWT, TLS required
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            true,
+            RpcEncryption::REQUIRED
+          },
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::PLAIN },
+            false,
+            true,
+            RpcEncryption::REQUIRED
+          },
+          true,
+          true,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::JWT,
+          SaslMechanism::INVALID,
+          true,
+        },
+
+
+        // client: JWT, TLS disabled
+        // server: JWT, TLS required
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { },
+            false,
+            true,
+            RpcEncryption::DISABLED
+          },
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { },
+            false,
+            true,
+            RpcEncryption::REQUIRED
+          },
+          true,
+          true,
+          Status::NotAuthorized(".*client does not support required TLS encryption"),
+          Status::NotAuthorized(""),
+          AuthenticationType::JWT,
+          SaslMechanism::INVALID,
+          true,
+        },
+
+
+        // client: GSSAPI, JWT, TLS required
+        // server: JWT, TLS disabled
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            true,
+            RpcEncryption::REQUIRED
+          },
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::PLAIN },
+            false,
+            true,
+            RpcEncryption::DISABLED
+          },
+          true,
+          true,
+          Status::NotAuthorized(".*server does not support required TLS encryption"),
+          Status::NetworkError(""),
+          AuthenticationType::JWT,
+          SaslMechanism::INVALID,
+          true,
+       },
+
+
+       // client: JWT, TLS required
+       // server: JWT, TLS optional
+       NegotiationDescriptor {
+           EndpointConfig {
+               PkiConfig::EXTERNALLY_SIGNED,
+               { },
+               false,
+               true,
+               RpcEncryption::REQUIRED
+           },
+           EndpointConfig {
+               PkiConfig::EXTERNALLY_SIGNED,
+               { },
+               false,
+               true,
+               RpcEncryption::OPTIONAL
+           },
+           true,
+           true,
+           Status::OK(),
+           Status::OK(),
+           AuthenticationType::JWT,
+           SaslMechanism::INVALID,
+           true,
+       },
+
+
+       // client: GSSAPI, JWT, TLS optional
+       // server: JWT, TLS required
+       NegotiationDescriptor {
+           EndpointConfig {
+               PkiConfig::EXTERNALLY_SIGNED,
+               { SaslMechanism::GSSAPI },
+               false,
+               true,
+               RpcEncryption::OPTIONAL
+           },
+           EndpointConfig {
+               PkiConfig::EXTERNALLY_SIGNED,
+               { SaslMechanism::PLAIN },
+               false,
+               true,
+               RpcEncryption::REQUIRED
+           },
+           true,
+           true,
+           Status::OK(),
+           Status::OK(),
+           AuthenticationType::JWT,
+           SaslMechanism::INVALID,
+           true,
+       },
+
+
+       // client: GSSAPI, TLS required
+       // server: GSSAPI, JWT, TLS required
+       NegotiationDescriptor {
+           EndpointConfig {
+               PkiConfig::EXTERNALLY_SIGNED,
+               { SaslMechanism::GSSAPI },
+               false,
+               false,
+               RpcEncryption::REQUIRED
+           },
+           EndpointConfig {
+               PkiConfig::EXTERNALLY_SIGNED,
+               { SaslMechanism::GSSAPI },
+               false,
+               true,
+               RpcEncryption::OPTIONAL
+           },
+           true,
+           true,
+           Status::OK(),
+           Status::OK(),
+               AuthenticationType::SASL,
+               SaslMechanism::GSSAPI,
+           true,
+       },
+
+
+       // client: GSSAPI, JWT, TLS required
+       // server: GSSAPI, TLS required
+       NegotiationDescriptor {
+           EndpointConfig {
+               PkiConfig::EXTERNALLY_SIGNED,
+               { SaslMechanism::GSSAPI },
+               false,
+               true,
+               RpcEncryption::REQUIRED
+           },
+           EndpointConfig {
+               PkiConfig::EXTERNALLY_SIGNED,
+               { SaslMechanism::GSSAPI },
+               false,
+               false,
+               RpcEncryption::REQUIRED
+           },
+           true,
+           true,
+           Status::OK(),
+           Status::OK(),
+           AuthenticationType::SASL,
+           SaslMechanism::GSSAPI,
+           true,
+       }
+
 ));
 
 // A "Callable" that takes a socket for use with starting a thread.
@@ -1220,8 +1542,15 @@ static void RunTimeoutExpectingServer(unique_ptr<Socket> socket) {
   TlsContext tls_context;
   CHECK_OK(tls_context.Init());
   TokenVerifier token_verifier;
+  string jwks_file_name = "keys.jwks";
+  string jwt_test_dir = JoinPathSegments(kudu::GetTestDataDirectory(), "jwt");
+  string jwt_data = kudu::CreateTestJWT(true);
+  ASSERT_OK(kudu::CreateTestJWKSFile(jwt_test_dir, jwks_file_name));
+  kudu::KeyBasedJwtVerifier jwt_verifier(
+      JoinPathSegments(jwt_test_dir, jwks_file_name), true);
+  CHECK_OK(jwt_verifier.Init());
   ServerNegotiation server_negotiation(std::move(socket), &tls_context,
-                                       &token_verifier, nullptr, RpcEncryption::OPTIONAL,
+                                       &token_verifier, &jwt_verifier, RpcEncryption::OPTIONAL,
                                        /* encrypt_loopback */ false, "kudu");
   CHECK_OK(server_negotiation.EnablePlain());
   Status s = server_negotiation.Negotiate();
@@ -1254,8 +1583,15 @@ static void RunTimeoutNegotiationServer(unique_ptr<Socket> socket) {
   TlsContext tls_context;
   CHECK_OK(tls_context.Init());
   TokenVerifier token_verifier;
+  string jwks_file_name = "keys.jwks";
+  string jwt_test_dir = JoinPathSegments(kudu::GetTestDataDirectory(), "jwt");
+  string jwt_data = kudu::CreateTestJWT(true);
+  ASSERT_OK(kudu::CreateTestJWKSFile(jwt_test_dir, jwks_file_name));
+  kudu::KeyBasedJwtVerifier jwt_verifier(
+      JoinPathSegments(jwt_test_dir, jwks_file_name), true);
+  CHECK_OK(jwt_verifier.Init());
   ServerNegotiation server_negotiation(std::move(socket), &tls_context,
-                                       &token_verifier, nullptr, RpcEncryption::OPTIONAL,
+                                       &token_verifier, &jwt_verifier, RpcEncryption::OPTIONAL,
                                        /* encrypt_loopback */ false, "kudu");
   CHECK_OK(server_negotiation.EnablePlain());
   MonoTime deadline = MonoTime::Now() - MonoDelta::FromMilliseconds(100L);
diff --git a/src/kudu/rpc/negotiation.cc b/src/kudu/rpc/negotiation.cc
index 07430c8d9..dd7f7005b 100644
--- a/src/kudu/rpc/negotiation.cc
+++ b/src/kudu/rpc/negotiation.cc
@@ -175,6 +175,7 @@ static Status DoClientNegotiation(Connection* conn,
   // Prefer secondary credentials (such as authn token) if permitted by policy.
   const auto authn_token = (conn->credentials_policy() == CredentialsPolicy::PRIMARY_CREDENTIALS)
       ? std::nullopt : messenger->authn_token();
+  const auto jwt = messenger->jwt();
   ClientNegotiation client_negotiation(conn->release_socket(),
                                        &messenger->tls_context(),
                                        authn_token,
@@ -203,9 +204,10 @@ static Status DoClientNegotiation(Connection* conn,
 
       if (authentication == RpcAuthentication::REQUIRED &&
           !authn_token &&
-          !messenger->tls_context().has_signed_cert()) {
+          !messenger->tls_context().has_signed_cert() &&
+          !jwt) {
         return Status::InvalidArgument(
-            "Kerberos, token, or PKI certificate credentials must be provided in order to "
+            "Kerberos, token, JWT, or PKI certificate credentials must be provided in order to "
             "require authentication for a client");
       }
     }
@@ -234,6 +236,11 @@ static Status DoClientNegotiation(Connection* conn,
   DCHECK(!(!authn_token &&
            client_negotiation.negotiated_authn() == AuthenticationType::TOKEN));
 
+  // Sanity check: if no JWT token was supplied as user credentials,
+  // the negotiated authentication type cannot be AuthenticationType::JWT.
+  DCHECK(!(!jwt &&
+           client_negotiation.negotiated_authn() == AuthenticationType::JWT));
+
   return Status::OK();
 }
 
diff --git a/src/kudu/rpc/rpc-test-base.h b/src/kudu/rpc/rpc-test-base.h
index 0aa46ceb2..745ece846 100644
--- a/src/kudu/rpc/rpc-test-base.h
+++ b/src/kudu/rpc/rpc-test-base.h
@@ -42,6 +42,7 @@
 #include "kudu/util/crc.h"
 #include "kudu/util/env.h"
 #include "kudu/util/faststring.h"
+#include "kudu/util/jwt.h"
 #include "kudu/util/mem_tracker.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/sockaddr.h"
diff --git a/src/kudu/rpc/server_negotiation.cc b/src/kudu/rpc/server_negotiation.cc
index f0eb9fef5..91c636595 100644
--- a/src/kudu/rpc/server_negotiation.cc
+++ b/src/kudu/rpc/server_negotiation.cc
@@ -206,7 +206,6 @@ Status ServerNegotiation::Negotiate() {
 
   // Wait until starting negotiation to check that the socket, tls_context, and
   // token_verifier are not null, since they do not need to be set for
-  // PreflightCheckGSSAPI.
   DCHECK(socket_);
   DCHECK(tls_context_);
   DCHECK(token_verifier_);
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index a20cbfb7e..51045eee0 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -797,7 +797,7 @@ Status ServerBase::Init() {
 
   // Create the Messenger.
   rpc::MessengerBuilder builder(name_);
-  std::shared_ptr<JwtVerifier> jwt_verifier;
+  shared_ptr<JwtVerifier> jwt_verifier = nullptr;
   if (FLAGS_enable_jwt_token_auth) {
     if (!FLAGS_jwks_url.empty()) {
       jwt_verifier = std::make_shared<PerAccountKeyBasedJwtVerifier>(FLAGS_jwks_url);
diff --git a/src/kudu/util/CMakeLists.txt b/src/kudu/util/CMakeLists.txt
index 51ba95ef1..77e01fe95 100644
--- a/src/kudu/util/CMakeLists.txt
+++ b/src/kudu/util/CMakeLists.txt
@@ -356,19 +356,24 @@ target_link_libraries(kudu_curl_util
 #######################################
 # mini_oidc
 #######################################
-set (MINI_OIDC_SRCS mini_oidc.cc jwt_test_certs.cc)
-add_library(mini_oidc ${MINI_OIDC_SRCS})
+add_library(mini_oidc mini_oidc.cc)
 target_link_libraries(mini_oidc
   server_process
   gutil
   kudu_test_util
-  kudu_util)
+  kudu_util
+  jwt_test_certs)
+
+#######################################
+# jwt_test_certs
+#######################################
+add_library(jwt_test_certs jwt_test_certs.cc)
+target_link_libraries(jwt_test_certs)
 
 #######################################
 # kudu_jwt_util
 #######################################
-add_library(kudu_jwt_util
-  jwt-util.cc)
+add_library(kudu_jwt_util jwt-util.cc)
 target_link_libraries(kudu_jwt_util
   gutil
   security
@@ -640,9 +645,9 @@ endif()
 ADD_KUDU_TEST(jwt-util-test)
 if(NOT NO_TESTS)
   target_link_libraries(jwt-util-test
-    kudu_jwt_util
     mini_oidc
-    server_process)
+    server_process
+    jwt_test_certs)
 endif()
 
 #######################################
diff --git a/src/kudu/util/jwt-util-test.cc b/src/kudu/util/jwt-util-test.cc
index bb7cc2dbe..ddc210f18 100644
--- a/src/kudu/util/jwt-util-test.cc
+++ b/src/kudu/util/jwt-util-test.cc
@@ -952,18 +952,18 @@ TEST(JwtUtilTest, VerifyOIDCDiscoveryEndpoint) {
   // Create and verify a token on the happy path.
   const string kSubject = "kudu";
   auto valid_user_token =
-      MiniOidc::CreateJwt(kValidAccount, kSubject, /*is_valid*/true);
+      oidc.CreateJwt(kValidAccount, kSubject, /*is_valid*/true);
   string subject;
   ASSERT_OK(jwt_verifier.VerifyToken(valid_user_token, &subject));
   ASSERT_EQ(kSubject, subject);
 
   // Verify some expected failure scenarios.
   const unordered_map<string, string> invalid_jwts {
-    { MiniOidc::CreateJwt(kInvalidAccount, kSubject, false), "invalid issuer with invalid "
+    { oidc.CreateJwt(kInvalidAccount, kSubject, false), "invalid issuer with invalid "
        "subject" },
-    { MiniOidc::CreateJwt(kInvalidAccount, kSubject, true), "invalid issuer with valid subject" },
-    { MiniOidc::CreateJwt(kValidAccount, kSubject, false), "valid issuer with invalid key id" },
-    { MiniOidc::CreateJwt(kMissingAccount, kSubject, true), "missing account" },
+    { oidc.CreateJwt(kInvalidAccount, kSubject, true), "invalid issuer with valid subject" },
+    { oidc.CreateJwt(kValidAccount, kSubject, false), "valid issuer with invalid key id" },
+    { oidc.CreateJwt(kMissingAccount, kSubject, true), "missing account" },
   };
 
   for (const auto& [jwt, msg] : invalid_jwts) {
@@ -985,7 +985,7 @@ TEST(JwtUtilTest, VerifyJWKSDiscoveryEndpointMultipleClients) {
   {
     const string kSubject = "kudu";
     auto valid_user_token =
-        MiniOidc::CreateJwt(kValidAccount, kSubject, /*is_valid*/true);
+        oidc.CreateJwt(kValidAccount, kSubject, /*is_valid*/true);
     string subject;
     ASSERT_OK(jwt_verifier.VerifyToken(valid_user_token, &subject));
     ASSERT_EQ(kSubject, subject);
diff --git a/src/kudu/util/jwt_test_certs.cc b/src/kudu/util/jwt_test_certs.cc
index 3f1311d0d..b79a1a150 100644
--- a/src/kudu/util/jwt_test_certs.cc
+++ b/src/kudu/util/jwt_test_certs.cc
@@ -16,6 +16,22 @@
 // under the License.
 #include "kudu/util/jwt_test_certs.h"
 
+#include <string>
+
+#include <jwt-cpp/jwt.h>
+#include <jwt-cpp/traits/kazuho-picojson/defaults.h>
+#include <jwt-cpp/traits/kazuho-picojson/traits.h>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/env.h"
+#include "kudu/util/path_util.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+
+using strings::Substitute;
+
+namespace kudu {
+
 const char* kRsaPrivKeyPem = R"(-----BEGIN PRIVATE KEY-----
 MIIEvwIBADANBgkqhkiG9w0BAQEFAASCBKkwggSlAgEAAoIBAQC4ZtdaIrd1BPIJ
 tfnF0TjIK5inQAXZ3XlCrUlJdP+XHwIRxdv1FsN12XyMYO/6ymLmo9ryoQeIrsXB
@@ -329,3 +345,33 @@ const char* kJwksEcFileFormat = R"(
   ]
 })";
 
+std::string CreateTestJWT(bool is_valid) {
+  return jwt::create()
+      .set_issuer(Substitute("auth0/$0", "test_account_id"))
+      .set_type("JWT")
+      .set_algorithm("RS256")
+      .set_key_id(is_valid ? kKid1 : kKid2)
+      .set_subject("test_subject")
+      .sign(jwt::algorithm::rs256(kRsaPubKeyPem, kRsaPrivKeyPem, "", ""));
+}
+
+Status CreateTestJWKSFile(const std::string& dir, const std::string& file_name) {
+  //"keys": [
+  //    { "kty": "RSA", "kid": "$0", "alg": "$1", "n": "$2", "e": "$3" },
+  //    { "kty": "RSA", "kid": "$4", "alg": "$5", "n": "$6", "e": "$7" }
+  //  ]
+  const std::string jwks_content = Substitute(kJwksRsaFileFormat,
+                                              /* kid */ kKid1,
+                                              /* alg */ "RS256",
+                                              /* n */ kRsaPubKeyJwkN,
+                                              /* e */ kRsaPubKeyJwkE,
+                                              /* kid */ kKid2,
+                                              /* alg */ "RS512",
+                                              /* n */ kRsa512PubKeyJwkN,
+                                              /* e */ kRsa512PubKeyJwkE);
+  RETURN_NOT_OK(Env::Default()->CreateDir(dir));
+
+  return WriteStringToFile(Env::Default(), jwks_content,
+                           JoinPathSegments(dir, file_name));
+}
+}  // namespace kudu
diff --git a/src/kudu/util/jwt_test_certs.h b/src/kudu/util/jwt_test_certs.h
index 1c59855f4..67c2a7413 100644
--- a/src/kudu/util/jwt_test_certs.h
+++ b/src/kudu/util/jwt_test_certs.h
@@ -16,6 +16,12 @@
 // under the License.
 #pragma once
 
+#include <string>
+
+#include "kudu/util/status.h"
+
+namespace kudu {
+
 extern const char* kRsaPrivKeyPem;
 extern const char* kRsaPubKeyPem;
 // The public keys in JWK format were converted from PEM formatted crypto keys with
@@ -66,3 +72,8 @@ extern const char* kKid2;
 extern const char* kJwksHsFileFormat;
 extern const char* kJwksRsaFileFormat;
 extern const char* kJwksEcFileFormat;
+
+std::string CreateTestJWT(bool is_valid);
+Status CreateTestJWKSFile(const std::string& dir, const std::string& file_name);
+
+}  // namespace kudu
diff --git a/src/kudu/util/mini_oidc.cc b/src/kudu/util/mini_oidc.cc
index d3e077722..1672f33a5 100644
--- a/src/kudu/util/mini_oidc.cc
+++ b/src/kudu/util/mini_oidc.cc
@@ -16,6 +16,7 @@
 // under the License.
 #include "kudu/util/mini_oidc.h"
 
+#include <chrono>
 #include <functional>
 #include <ostream>
 #include <string>
@@ -74,12 +75,12 @@ void OidcDiscoveryHandler(const Webserver::WebRequest& req,
     resp->status_code = HttpStatusCode::BadRequest;
     return;
   }
-  resp->output << Substitute(kDiscoveryFormat, *account_id,
-                             Substitute("$0/$1", jwks_server_url, *account_id));
+  resp->output << Substitute(
+      kDiscoveryFormat, *account_id, Substitute("$0/$1", jwks_server_url, *account_id));
   resp->status_code = HttpStatusCode::Ok;
 }
 
-} // anonymous namespace
+}  // anonymous namespace
 
 MiniOidc::MiniOidc(MiniOidcOptions options) : options_(std::move(options)) {
   if (options_.lifetime_ms == 0) {
@@ -102,17 +103,23 @@ Status MiniOidc::Start() {
   jwks_server_.reset(new Webserver(jwks_opts));
 
   for (const auto& [account_id, valid] : options_.account_ids) {
-    jwks_server_->RegisterPrerenderedPathHandler(Substitute("/jwks/$0", account_id), account_id,
-        [account_id = account_id, valid = valid] (const Webserver::WebRequest&  /*req*/,
-                                                  Webserver::PrerenderedWebResponse* resp) {
+    jwks_server_->RegisterPrerenderedPathHandler(
+        Substitute("/jwks/$0", account_id),
+        account_id,
+        [account_id = account_id, valid = valid](const Webserver::WebRequest&  /*req*/,
+                                                 Webserver::PrerenderedWebResponse* resp) {
           // NOTE: 'kKid1' points at a valid key, while 'kKid2' points at an
           // invalid key.
-          resp->output << Substitute(kJwksRsaFileFormat, kKid1, "RS256",
-              valid ? kRsaPubKeyJwkN : kRsaInvalidPubKeyJwkN,
-              kRsaPubKeyJwkE, kKid2, "RS256", kRsaInvalidPubKeyJwkN, kRsaPubKeyJwkE),
-          resp->status_code = HttpStatusCode::Ok;
+          resp->output << Substitute(kJwksRsaFileFormat, kKid1,
+                                     "RS256",
+                                     valid ? kRsaPubKeyJwkN : kRsaInvalidPubKeyJwkN,
+                                     kRsaPubKeyJwkE, kKid2,
+                                     "RS256",
+                                     kRsaInvalidPubKeyJwkN, kRsaPubKeyJwkE),
+              resp->status_code = HttpStatusCode::Ok;
         },
-        /*is_styled*/false, /*is_on_nav_bar*/false);
+        /*is_styled*/ false,
+        /*is_on_nav_bar*/ false);
   }
   RETURN_NOT_OK(jwks_server_->Start());
   vector<Sockaddr> bound_addrs;
@@ -126,13 +133,15 @@ Status MiniOidc::Start() {
   oidc_opts.port = 0;
   oidc_server_.reset(new Webserver(oidc_opts));
   oidc_server_->RegisterPrerenderedPathHandler(
-      "/.well-known/openid-configuration", "openid-configuration",
+      "/.well-known/openid-configuration",
+      "openid-configuration",
       // Pass the 'accountId' query arguments to return a response that
       // points to the JWKS endpoint for the account.
-      [jwks_url] (const Webserver::WebRequest& req, Webserver::PrerenderedWebResponse* resp) {
+      [jwks_url](const Webserver::WebRequest& req, Webserver::PrerenderedWebResponse* resp) {
         OidcDiscoveryHandler(req, resp, jwks_url);
       },
-      /*is_styled*/false, /*is_on_nav_bar*/false);
+      /*is_styled*/ false,
+      /*is_on_nav_bar*/ false);
   RETURN_NOT_OK(oidc_server_->Start());
   bound_addrs.clear();
   RETURN_NOT_OK(oidc_server_->GetBoundAddresses(&bound_addrs));
@@ -151,13 +160,17 @@ void MiniOidc::Stop() {
 }
 
 string MiniOidc::CreateJwt(const string& account_id, const string& subject, bool is_valid) {
- return jwt::create()
-     .set_issuer(Substitute("auth0/$0", account_id))
-     .set_type("JWT")
-     .set_algorithm("RS256")
-     .set_key_id(is_valid ? kKid1 : kKid2)
-     .set_subject(subject)
-     .sign(jwt::algorithm::rs256(kRsaPubKeyPem, kRsaPrivKeyPem, "", ""));
+  auto lifetime = std::chrono::milliseconds(options_.lifetime_ms);
+
+  return jwt::create()
+      .set_issuer(Substitute("auth0/$0", account_id))
+      .set_type("JWT")
+      .set_algorithm("RS256")
+      .set_key_id(is_valid ? kKid1 : kKid2)
+      .set_subject(subject)
+      .set_not_before(std::chrono::system_clock::now())
+      .set_expires_at(std::chrono::system_clock::now() + lifetime)
+      .sign(jwt::algorithm::rs256(kRsaPubKeyPem, kRsaPrivKeyPem, "", ""));
 }
 
-} // namespace kudu
+}  // namespace kudu
diff --git a/src/kudu/util/mini_oidc.h b/src/kudu/util/mini_oidc.h
index 82f09ba35..74dfbfefd 100644
--- a/src/kudu/util/mini_oidc.h
+++ b/src/kudu/util/mini_oidc.h
@@ -74,9 +74,9 @@ class MiniOidc {
 
   // Creates a JWT with the given `account_id` and `subject`. If `is_valid` is set to false, the
   // created token will be invalid.
-  static std::string CreateJwt(const std::string& account_id,
-                               const std::string& subject,
-                               bool is_valid);
+  std::string CreateJwt(const std::string& account_id,
+                        const std::string& subject,
+                        bool is_valid);
   const std::string& url() const {
     return oidc_url_;
   }