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/03/25 01:23:42 UTC

[kudu] branch master updated: jwt: introduce MiniOidc

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 400d25681 jwt: introduce MiniOidc
400d25681 is described below

commit 400d25681135b233c943a1b37d55718261578ff2
Author: Zoltan Chovan <zc...@cloudera.com>
AuthorDate: Thu Feb 2 10:39:23 2023 +0100

    jwt: introduce MiniOidc
    
    This patch takes some existing test utilities and encapsulates them into
    the new MiniOidc class. The MiniOidc serves the purpose of being the
    OpenID Connect Discovery endpoint, as the host for JWKSs of each
    account, and the dispenser of JWTs.
    
    This encapsulation will be useful in testing JWTs from non-C++ tests
    that typically rely on exposing Mini* components via control shell.
    
    Co-authored-by: Andrew Wong <aw...@apache.org>
    
    Change-Id: I26e9b3bcd0946adbe4642a19c5ef1124e39632c6
    Reviewed-on: http://gerrit.cloudera.org:8080/18473
    Tested-by: Kudu Jenkins
    Reviewed-by: Alexey Serbin <al...@apache.org>
---
 src/kudu/util/CMakeLists.txt   |  12 +++
 src/kudu/util/jwt-util-test.cc | 192 +++++++++++------------------------------
 src/kudu/util/jwt-util.cc      |   7 +-
 src/kudu/util/jwt-util.h       |   6 +-
 src/kudu/util/mini_oidc.cc     | 163 ++++++++++++++++++++++++++++++++++
 src/kudu/util/mini_oidc.h      |  93 ++++++++++++++++++++
 6 files changed, 323 insertions(+), 150 deletions(-)

diff --git a/src/kudu/util/CMakeLists.txt b/src/kudu/util/CMakeLists.txt
index 5223d9ef6..bf7d8a843 100644
--- a/src/kudu/util/CMakeLists.txt
+++ b/src/kudu/util/CMakeLists.txt
@@ -353,6 +353,17 @@ target_link_libraries(kudu_curl_util
   glog
   gutil)
 
+#######################################
+# mini_oidc
+#######################################
+set (MINI_OIDC_SRCS mini_oidc.cc)
+add_library(mini_oidc ${MINI_OIDC_SRCS})
+target_link_libraries(mini_oidc
+  server_process
+  gutil
+  kudu_test_util
+  kudu_util)
+
 #######################################
 # kudu_jwt_util
 #######################################
@@ -630,6 +641,7 @@ ADD_KUDU_TEST(jwt-util-test)
 if(NOT NO_TESTS)
   target_link_libraries(jwt-util-test
     kudu_jwt_util
+    mini_oidc
     server_process)
 endif()
 
diff --git a/src/kudu/util/jwt-util-test.cc b/src/kudu/util/jwt-util-test.cc
index 9ccc23cf0..bb7cc2dbe 100644
--- a/src/kudu/util/jwt-util-test.cc
+++ b/src/kudu/util/jwt-util-test.cc
@@ -36,7 +36,6 @@
 #include <jwt-cpp/traits/kazuho-picojson/defaults.h>
 #include <jwt-cpp/traits/kazuho-picojson/traits.h>
 
-#include "kudu/gutil/map-util.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/server/webserver.h"
 #include "kudu/server/webserver_options.h"
@@ -44,6 +43,7 @@
 #include "kudu/util/env.h"
 #include "kudu/util/jwt-util-internal.h"
 #include "kudu/util/jwt_test_certs.h"
+#include "kudu/util/mini_oidc.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/scoped_cleanup.h"
 #include "kudu/util/slice.h"
@@ -58,6 +58,7 @@ using std::unique_ptr;
 using std::unordered_map;
 using std::vector;
 using strings::Substitute;
+using kudu::MiniOidc;
 
 /// Utility class for creating a file that will be automatically deleted upon test
 /// completion.
@@ -933,158 +934,61 @@ TEST(JwtUtilTest, VerifyJWKSUrl) {
 }
 
 namespace {
-
-// $0: account_id
-// $1: jwks_uri
-const char kDiscoveryFormat[] = R"({
-    "issuer": "auth0/$0",
-    "token_endpoint": "dummy.endpoint.com",
-    "response_types_supported": [
-        "id_token"
-    ],
-    "claims_supported": [
-        "sub",
-        "aud",
-        "iss",
-        "exp"
-    ],
-    "subject_types_supported": [
-        "public"
-    ],
-    "id_token_signing_alg_values_supported": [
-        "RS256"
-    ],
-    "jwks_uri": "$1"
-})";
-
-void JWKSDiscoveryHandler(const Webserver::WebRequest& req,
-                          Webserver::PrerenderedWebResponse* resp,
-                          const JWKSMockServer& jwks_server) {
-  const auto* account_id = FindOrNull(req.parsed_args, "accountid");
-  if (!account_id) {
-    resp->output << "expected 'accountId' query";
-    resp->status_code = HttpStatusCode::BadRequest;
-    return;
-  }
-  resp->output << Substitute(kDiscoveryFormat, *account_id,
-                             jwks_server.url_for_account(*account_id));
-  resp->status_code = HttpStatusCode::Ok;
-}
-
 const char kValidAccount[] = "new-phone";
 const char kInvalidAccount[] = "who-is-this";
-const char kMissingAccount[] = "no-where";
-
-class JWKSDiscoveryEndpointMockServer {
- public:
-  Status Start() {
-    unordered_map<string, string> account_id_to_resp({
-        {
-          // Create an account that has valid keys.
-          kValidAccount,
-          Substitute(kJwksRsaFileFormat, kKid1, "RS256",
-              kRsaPubKeyJwkN, kRsaPubKeyJwkE, kKid2, "RS256", kRsaInvalidPubKeyJwkN,
-              kRsaPubKeyJwkE),
-        },
-        {
-          // The keys associated with this account are invalid.
-          kInvalidAccount,
-          Substitute(kJwksRsaFileFormat, kKid1, "RS256",
-              kRsaInvalidPubKeyJwkN, kRsaPubKeyJwkE, kKid2, "RS256",
-              kRsaInvalidPubKeyJwkN, kRsaPubKeyJwkE),
-        },
-    });
-    RETURN_NOT_OK(jwks_server_.StartWithAccounts(account_id_to_resp));
-
-    WebserverOptions opts;
-    opts.port = 0;
-    webserver_.reset(new Webserver(opts));
-    webserver_->RegisterPrerenderedPathHandler(
-        "/.well-known/openid-configuration'", "openid-configuration",
-        // Pass the 'accountId' query arguments to return a response that
-        // points to the JWKS endpoint for the account.
-        [this] (const Webserver::WebRequest& req, Webserver::PrerenderedWebResponse* resp) {
-          JWKSDiscoveryHandler(req, resp, jwks_server_);
-        },
-        /*is_styled*/false, /*is_on_nav_bar*/false);
-    RETURN_NOT_OK(webserver_->Start());
-    vector<Sockaddr> addrs;
-    RETURN_NOT_OK(webserver_->GetBoundAddresses(&addrs));
-    RETURN_NOT_OK(addr_.ParseString("127.0.0.1", addrs[0].port()));
-    url_ = Substitute("http://$0/.well-known/openid-configuration'", addr_.ToString());
-    return Status::OK();
-  }
-
-  const string& url() const {
-    return url_;
-  }
- private:
-  unique_ptr<Webserver> webserver_;
-  JWKSMockServer jwks_server_;
-  string url_;
-  Sockaddr addr_;
-};
-
+const char kMissingAccount[] = "no-one";
 } // anonymous namespace
 
-TEST(JwtUtilTest, VerifyJWKSDiscoveryEndpoint) {
-  JWKSDiscoveryEndpointMockServer discovery_endpoint;
-  ASSERT_OK(discovery_endpoint.Start());
-  PerAccountKeyBasedJwtVerifier jwt_verifier(discovery_endpoint.url());
-  {
-    auto valid_user_token =
-        jwt::create()
-            .set_issuer(Substitute("auth0/$0", kValidAccount))
-            .set_type("JWT")
-            .set_algorithm("RS256")
-            .set_key_id(kKid1)
-            .set_subject(kValidAccount)
-            .sign(jwt::algorithm::rs256(kRsaPubKeyPem, kRsaPrivKeyPem, "", ""));
-    string subject;
-    ASSERT_OK(jwt_verifier.VerifyToken(valid_user_token, &subject));
-    ASSERT_EQ(kValidAccount, subject);
-  }
-  {
-    auto invalid_user_token =
-        jwt::create()
-            .set_issuer(Substitute("auth0/$0", kInvalidAccount))
-            .set_type("JWT")
-            .set_algorithm("RS256")
-            .set_key_id(kKid1)
-            .set_subject(kInvalidAccount)
-            .sign(jwt::algorithm::rs256(kRsaPubKeyPem, kRsaPrivKeyPem, "", ""));
-    string subject;
-    Status s = jwt_verifier.VerifyToken(invalid_user_token, &subject);
-    ASSERT_TRUE(s.IsNotAuthorized()) << s.ToString();
-  }
-  {
-    auto missing_user_token =
-        jwt::create()
-            .set_issuer(Substitute("auth0/$0", kMissingAccount))
-            .set_type("JWT")
-            .set_algorithm("RS256")
-            .set_key_id(kKid1)
-            .set_subject(kMissingAccount)
-            .sign(jwt::algorithm::rs256(kRsaPubKeyPem, kRsaPrivKeyPem, "", ""));
-    string subject;
-    Status s = jwt_verifier.VerifyToken(missing_user_token, &subject);
-    ASSERT_TRUE(s.IsRemoteError()) << s.ToString();
+TEST(JwtUtilTest, VerifyOIDCDiscoveryEndpoint) {
+  MiniOidcOptions opts;
+  opts.account_ids = {
+    { kValidAccount, /*is_valid*/true },
+    { kInvalidAccount, /*is_valid*/false },
+  };
+  MiniOidc oidc(std::move(opts));
+  ASSERT_OK(oidc.Start());
+  const PerAccountKeyBasedJwtVerifier jwt_verifier(oidc.url());
+
+  // Create and verify a token on the happy path.
+  const string kSubject = "kudu";
+  auto valid_user_token =
+      MiniOidc::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 "
+       "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" },
+  };
+
+  for (const auto& [jwt, msg] : invalid_jwts) {
+    string invalid_subject;
+    const Status s = jwt_verifier.VerifyToken(jwt, &invalid_subject);
+    EXPECT_FALSE(s.ok()) << Substitute("failed case $0: $1", msg, s.ToString());
   }
 }
 
 TEST(JwtUtilTest, VerifyJWKSDiscoveryEndpointMultipleClients) {
-  JWKSDiscoveryEndpointMockServer discovery_endpoint;
-  ASSERT_OK(discovery_endpoint.Start());
-  PerAccountKeyBasedJwtVerifier jwt_verifier(discovery_endpoint.url());
+  MiniOidcOptions opts;
+  opts.account_ids = {
+      { kValidAccount, /*is_valid*/true }
+  };
+  MiniOidc oidc(std::move(opts));
+  ASSERT_OK(oidc.Start());
+  PerAccountKeyBasedJwtVerifier jwt_verifier(oidc.url());
+
   {
+    const string kSubject = "kudu";
     auto valid_user_token =
-        jwt::create()
-            .set_issuer(Substitute("auth0/$0", kValidAccount))
-            .set_type("JWT")
-            .set_algorithm("RS256")
-            .set_key_id(kKid1)
-            .set_subject(kValidAccount)
-            .sign(jwt::algorithm::rs256(kRsaPubKeyPem, kRsaPrivKeyPem, "", ""));
+        MiniOidc::CreateJwt(kValidAccount, kSubject, /*is_valid*/true);
+    string subject;
+    ASSERT_OK(jwt_verifier.VerifyToken(valid_user_token, &subject));
+    ASSERT_EQ(kSubject, subject);
 
     int constexpr n = 8;
     std::vector<std::thread> threads;
@@ -1095,7 +999,7 @@ TEST(JwtUtilTest, VerifyJWKSDiscoveryEndpointMultipleClients) {
       threads.emplace_back([&](){
         string subject;
         CHECK_OK(jwt_verifier.VerifyToken(valid_user_token, &subject));
-        CHECK_EQ(kValidAccount, subject);
+        CHECK_EQ(kSubject, subject);
         latch.CountDown();
       });
     }
diff --git a/src/kudu/util/jwt-util.cc b/src/kudu/util/jwt-util.cc
index 78280ae1e..cff633ad8 100644
--- a/src/kudu/util/jwt-util.cc
+++ b/src/kudu/util/jwt-util.cc
@@ -968,10 +968,11 @@ Status PerAccountKeyBasedJwtVerifier::JWTHelperForToken(const JWTHelper::JWTDeco
     }
   }
 
-  // Otherwise, use the Discovery Endpoint to determine what 'jwks_uri' to use.
+  // Otherwise, use the OIDC Discovery Endpoint to determine what 'jwks_uri' to
+  // use.
   kudu::EasyCurl curl;
   kudu::faststring dst;
-  const auto discovery_endpoint = Substitute("$0?accountId=$1", discovery_base_, account_id);
+  const auto discovery_endpoint = Substitute("$0?accountId=$1", oidc_uri_, account_id);
   curl.set_timeout(
       kudu::MonoDelta::FromSeconds(static_cast<int64_t>(FLAGS_jwks_pulling_timeout_s)));
   curl.set_verify_peer(false);
@@ -1017,7 +1018,7 @@ Status PerAccountKeyBasedJwtVerifier::JWTHelperForToken(const JWTHelper::JWTDeco
 
 Status PerAccountKeyBasedJwtVerifier::Init() {
   for (auto& [account_id, verifier] : jwt_by_account_id_) {
-    verifier->Init(Substitute("$0?accountId=$1", discovery_base_, account_id),
+    verifier->Init(Substitute("$0?accountId=$1", oidc_uri_, account_id),
                    /*is_local_file*/false);
   }
   return Status::OK();
diff --git a/src/kudu/util/jwt-util.h b/src/kudu/util/jwt-util.h
index c98ce2b58..e25bbd8d4 100644
--- a/src/kudu/util/jwt-util.h
+++ b/src/kudu/util/jwt-util.h
@@ -117,8 +117,8 @@ class KeyBasedJwtVerifier : public JwtVerifier {
 
 class PerAccountKeyBasedJwtVerifier : public JwtVerifier {
  public:
-  explicit PerAccountKeyBasedJwtVerifier(std::string  jwks_uri)
-      : discovery_base_(std::move(jwks_uri)) {}
+  explicit PerAccountKeyBasedJwtVerifier(std::string oidc_uri)
+      : oidc_uri_(std::move(oidc_uri)) {}
 
   ~PerAccountKeyBasedJwtVerifier() override = default;
 
@@ -131,7 +131,7 @@ class PerAccountKeyBasedJwtVerifier : public JwtVerifier {
   // Returns an error if the token doesn't contain the appropriate fields.
   Status JWTHelperForToken(const JWTHelper::JWTDecodedToken& token, JWTHelper** helper) const;
 
-  std::string discovery_base_;
+  const std::string oidc_uri_;
   // Marked as mutable so that PerAccountKeyBasedJwtVerifier::JWTHelperForToken is able to emplace
   // new JWTHelpers in it.
   mutable std::unordered_map<std::string, std::shared_ptr<JWTHelper>> jwt_by_account_id_;
diff --git a/src/kudu/util/mini_oidc.cc b/src/kudu/util/mini_oidc.cc
new file mode 100644
index 000000000..803ed1e14
--- /dev/null
+++ b/src/kudu/util/mini_oidc.cc
@@ -0,0 +1,163 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+#include "kudu/util/mini_oidc.h"
+
+#include <functional>
+#include <ostream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <jwt-cpp/jwt.h>
+#include <jwt-cpp/traits/kazuho-picojson/defaults.h>
+#include <jwt-cpp/traits/kazuho-picojson/traits.h>
+
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/server/webserver.h"
+#include "kudu/server/webserver_options.h"
+#include "kudu/util/jwt_test_certs.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/web_callback_registry.h"
+
+using std::string;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+
+namespace {
+
+// $0: account_id
+// $1: jwks_uri
+const char kDiscoveryFormat[] = R"({
+    "issuer": "auth0/$0",
+    "token_endpoint": "dummy.endpoint.com",
+    "response_types_supported": [
+        "id_token"
+    ],
+    "claims_supported": [
+        "sub",
+        "aud",
+        "iss",
+        "exp"
+    ],
+    "subject_types_supported": [
+        "public"
+    ],
+    "id_token_signing_alg_values_supported": [
+        "RS256"
+    ],
+    "jwks_uri": "$1"
+})";
+
+void OidcDiscoveryHandler(const Webserver::WebRequest& req,
+                          Webserver::PrerenderedWebResponse* resp,
+                          const string& jwks_server_url) {
+  const auto* account_id = FindOrNull(req.parsed_args, "accountid");
+  if (!account_id) {
+    resp->output << "expected 'accountId' query";
+    resp->status_code = HttpStatusCode::BadRequest;
+    return;
+  }
+  resp->output << Substitute(kDiscoveryFormat, *account_id,
+                             Substitute("$0/$1", jwks_server_url, *account_id));
+  resp->status_code = HttpStatusCode::Ok;
+}
+
+} // anonymous namespace
+
+MiniOidc::MiniOidc(MiniOidcOptions options) : options_(std::move(options)) {
+  if (options_.lifetime_ms == 0) {
+    // Setting default TTL for JWTs to 60 minutes
+    options_.lifetime_ms = 3600000;
+  }
+}
+
+// Explicitly defined outside the header to ensure users of the header don't
+// need to include member destructors.
+MiniOidc::~MiniOidc() {
+  Stop();
+}
+
+Status MiniOidc::Start() {
+  // Start the JWKS server and register path handlers for each of the accounts
+  // we've been configured to server.
+  WebserverOptions jwks_opts;
+  jwks_opts.port = 0;
+  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) {
+          // 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;
+        },
+        /*is_styled*/false, /*is_on_nav_bar*/false);
+  }
+  RETURN_NOT_OK(jwks_server_->Start());
+  vector<Sockaddr> bound_addrs;
+  Sockaddr addr;
+  RETURN_NOT_OK(jwks_server_->GetBoundAddresses(&bound_addrs));
+  RETURN_NOT_OK(addr.ParseString(bound_addrs[0].host(), bound_addrs[0].port()));
+  string const jwks_url = Substitute("http://$0/jwks", addr.ToString());
+
+  // Now start the OIDC Discovery server that points to the JWKS endpoints.
+  WebserverOptions oidc_opts;
+  oidc_opts.port = 0;
+  oidc_server_.reset(new Webserver(oidc_opts));
+  oidc_server_->RegisterPrerenderedPathHandler(
+      "/.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) {
+        OidcDiscoveryHandler(req, resp, jwks_url);
+      },
+      /*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));
+  RETURN_NOT_OK(addr.ParseString(bound_addrs[0].host(), bound_addrs[0].port()));
+  oidc_url_ = Substitute("http://$0/.well-known/openid-configuration", addr.ToString());
+  return Status::OK();
+}
+
+void MiniOidc::Stop() {
+  if (oidc_server_) {
+    oidc_server_->Stop();
+  }
+  if (jwks_server_) {
+    jwks_server_->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, "", ""));
+}
+
+} // namespace kudu
diff --git a/src/kudu/util/mini_oidc.h b/src/kudu/util/mini_oidc.h
new file mode 100644
index 000000000..82f09ba35
--- /dev/null
+++ b/src/kudu/util/mini_oidc.h
@@ -0,0 +1,93 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+#pragma once
+
+#include <cstdint>
+#include <memory>
+#include <string>
+#include <unordered_map>
+
+#include "kudu/gutil/port.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class Webserver;
+
+struct MiniOidcOptions {
+  // Port to bind to.
+  //
+  // Default: 0 (ephemeral port).
+  uint16_t port = 0;
+
+  // Lifetime of generated JWTs in milliseconds
+  uint64_t lifetime_ms = 0;
+
+  // Maps account IDs to add to whether or not to create JWKS with invalid keys.
+  std::unordered_map<std::string, bool> account_ids;
+};
+
+// Serves the following endpoints for testing a cluster:
+//
+// OIDC Discovery Endpoint: <url>/.well-known/openid-configuration/<account-id>
+// - Returns a JSON document containing the JWKS endpoint associated with the
+//   given account ID, among other metadata
+// - See https://swagger.io/docs/specification/authentication/openid-connect-discovery/
+//   for more details
+//
+// JWKS Endpoint: <url>/jwks/<account-id>
+// - Returns a JSON document containing the JWKS associated with the given
+//   account ID
+// - See https://auth0.com/docs/secure/tokens/json-web-tokens/json-web-key-sets
+//   for more details
+//
+// Verification of a JWT associated with a given account ID (typically denoted
+// as the final segment of the 'iss' field in the JWT payload) comprises of the
+// following steps:
+// - Query the OIDC Discovery Endpoint, specifying an 'accountId' argument from
+//   the JWT.
+// - The endpoint returns a document containing the field 'jwks_uri', the JWKS
+//   Endpoint.
+// - Query the JWKS Endpoint returns the JWKS associated with the account ID.
+// - The JWKS is used to verify the JWT.
+class MiniOidc {
+ public:
+  explicit MiniOidc(MiniOidcOptions options);
+  ~MiniOidc();
+
+  Status Start() WARN_UNUSED_RESULT;
+  void Stop();
+
+  // 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);
+  const std::string& url() const {
+    return oidc_url_;
+  }
+ private:
+  MiniOidcOptions options_;
+
+  std::unique_ptr<Webserver> oidc_server_;
+
+  std::string oidc_url_;
+
+  std::unique_ptr<Webserver> jwks_server_;
+};
+
+} // namespace kudu