You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ab...@apache.org on 2023/04/14 14:53:58 UTC

[kudu] branch master updated (08bd36a87 -> a3a7c97be)

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

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


    from 08bd36a87 [Python] Refactor tests to use assertRaisesRegex
     new 3d56a26b9 jwt: add control points for test binaries
     new 8595384de [jwt] Verify JWKS URL server TLS certificate by default
     new a3a7c97be jwt: Java client usage of JWTs

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../org/apache/kudu/backup/TestKuduBackup.scala    |   6 +-
 .../java/org/apache/kudu/client/Negotiator.java    |  24 ++++
 .../org/apache/kudu/client/SecurityContext.java    |  24 ++++
 .../java/org/apache/kudu/test/KuduTestHarness.java |   4 +
 .../apache/kudu/test/cluster/MiniKuduCluster.java  |  45 ++++++-
 .../org/apache/kudu/test/TestMiniKuduCluster.java  |  40 ++++++
 src/kudu/integration-tests/CMakeLists.txt          |   3 +-
 src/kudu/integration-tests/security-itest.cc       | 127 +++++++++++++++++
 src/kudu/mini-cluster/CMakeLists.txt               |   1 +
 src/kudu/mini-cluster/external_mini_cluster.cc     |   7 +-
 src/kudu/security/test/test_certs.cc               | 150 +++++++++++++++++++++
 src/kudu/security/test/test_certs.h                |   7 +
 src/kudu/server/server_base.cc                     |  17 ++-
 src/kudu/tools/tool.proto                          |  10 +-
 src/kudu/tools/tool_action_test.cc                 |  24 ++++
 src/kudu/util/jwt-util-internal.h                  |  11 +-
 src/kudu/util/jwt-util-test.cc                     |  54 ++++----
 src/kudu/util/jwt-util.cc                          |  38 ++++--
 src/kudu/util/jwt-util.h                           |  19 ++-
 src/kudu/util/mini_oidc.cc                         |  32 ++++-
 src/kudu/util/mini_oidc.h                          |   7 +
 21 files changed, 590 insertions(+), 60 deletions(-)


[kudu] 03/03: jwt: Java client usage of JWTs

Posted by ab...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit a3a7c97be031f8fc32402e430eff1a89c19dbdfb
Author: Zoltan Chovan <zc...@cloudera.com>
AuthorDate: Thu Apr 13 19:26:04 2023 +0200

    jwt: Java client usage of JWTs
    
    This change introduces modifications to the Negotiator and the
    SecurityContext  classes to be able to handle JWT authentication from
    the Java client.
    
    Also adding a test where the client is set up with a JWT that was
    generated by the MiniOIDC that starts along with the
    ExternalMiniCluster.
    
    Change-Id: Id02dcd7ee57a838f4763500e78053e21aac21c09
    Reviewed-on: http://gerrit.cloudera.org:8080/18477
    Reviewed-by: Attila Bukor <ab...@apache.org>
    Tested-by: Attila Bukor <ab...@apache.org>
---
 .../java/org/apache/kudu/client/Negotiator.java    | 24 +++++++++++++
 .../org/apache/kudu/client/SecurityContext.java    | 24 +++++++++++++
 .../java/org/apache/kudu/test/KuduTestHarness.java |  4 +++
 .../org/apache/kudu/test/TestMiniKuduCluster.java  | 41 +++++++++++++++++-----
 4 files changed, 84 insertions(+), 9 deletions(-)

diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java b/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java
index 5fe628598..f17dfce4e 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java
@@ -82,6 +82,7 @@ import org.apache.kudu.rpc.RpcHeader.AuthenticationTypePB;
 import org.apache.kudu.rpc.RpcHeader.NegotiatePB;
 import org.apache.kudu.rpc.RpcHeader.NegotiatePB.NegotiateStep;
 import org.apache.kudu.rpc.RpcHeader.RpcFeatureFlag;
+import org.apache.kudu.security.Token.JwtRawPB;
 import org.apache.kudu.security.Token.SignedTokenPB;
 import org.apache.kudu.util.SecurityUtil;
 
@@ -176,6 +177,7 @@ public class Negotiator extends SimpleChannelInboundHandler<CallResponse> {
    * ensure that it doesn't change over the course of a negotiation attempt.
    */
   private final SignedTokenPB authnToken;
+  private final JwtRawPB jsonWebToken;
 
   private enum AuthnTokenNotUsedReason {
     NONE_AVAILABLE("no token is available"),
@@ -267,6 +269,9 @@ public class Negotiator extends SimpleChannelInboundHandler<CallResponse> {
       this.authnToken = null;
       this.authnTokenNotUsedReason = AuthnTokenNotUsedReason.NONE_AVAILABLE;
     }
+    JwtRawPB jwt = securityContext.getJsonWebToken();
+    // TODO(zchovan): also guard this on the presence of certs.
+    this.jsonWebToken = jwt;
   }
 
   public void sendHello(ChannelHandlerContext ctx) {
@@ -558,6 +563,12 @@ public class Negotiator extends SimpleChannelInboundHandler<CallResponse> {
               "but client had no valid token");
         }
         break;
+      case JWT:
+        if (jsonWebToken == null) {
+          throw new IllegalArgumentException("server chose JWT authentication " +
+              "but client had no valid JWT");
+        }
+        break;
       default:
         throw new IllegalArgumentException("server chose bad authn type " + chosenAuthnType);
     }
@@ -745,6 +756,9 @@ public class Negotiator extends SimpleChannelInboundHandler<CallResponse> {
       case TOKEN:
         sendTokenExchange(ctx);
         break;
+      case JWT:
+        sendJwtExchange(ctx);
+        break;
       default:
         throw new AssertionError("unreachable");
     }
@@ -764,6 +778,16 @@ public class Negotiator extends SimpleChannelInboundHandler<CallResponse> {
     sendSaslMessage(ctx, builder.build());
   }
 
+  private void sendJwtExchange(ChannelHandlerContext ctx) {
+    Preconditions.checkNotNull(jsonWebToken);
+    Preconditions.checkNotNull(sslHandshakeFuture);
+    Preconditions.checkState(sslHandshakeFuture.isSuccess());
+    RpcHeader.NegotiatePB.Builder builder = RpcHeader.NegotiatePB.newBuilder()
+        .setStep(NegotiateStep.JWT_EXCHANGE)
+        .setJwtRaw(jsonWebToken);
+    sendSaslMessage(ctx, builder.build());
+  }
+
   private void handleTokenExchangeResponse(ChannelHandlerContext ctx, NegotiatePB response)
       throws SaslException {
     Preconditions.checkArgument(response.getStep() == NegotiateStep.TOKEN_EXCHANGE,
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/SecurityContext.java b/java/kudu-client/src/main/java/org/apache/kudu/client/SecurityContext.java
index 8073c4dd6..8597e0cd9 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/SecurityContext.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/SecurityContext.java
@@ -53,6 +53,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.kudu.client.Client.AuthenticationCredentialsPB;
+import org.apache.kudu.security.Token.JwtRawPB;
 import org.apache.kudu.security.Token.SignedTokenPB;
 import org.apache.kudu.security.Token.TokenPB;
 import org.apache.kudu.util.Pair;
@@ -73,6 +74,10 @@ class SecurityContext {
   @Nullable
   private SignedTokenPB authnToken;
 
+  @GuardedBy("this")
+  @Nullable
+  private JwtRawPB jsonWebToken;
+
   @GuardedBy("this")
   private String realUser;
 
@@ -310,8 +315,10 @@ class SecurityContext {
       }
 
       LOG.debug("Importing authentication credentials with {} authn token, " +
+                "JWT={} , " +
                 "{} cert(s), and realUser={}",
                 pb.hasAuthnToken() ? "one" : "no",
+                pb.hasJwt() ? pb.getJwt() : "<none>",
                 pb.getCaCertDersCount(),
                 pb.hasRealUser() ? pb.getRealUser() : "<none>");
       if (pb.hasAuthnToken()) {
@@ -319,6 +326,10 @@ class SecurityContext {
       }
       trustCertificates(pb.getCaCertDersList());
 
+      if (pb.hasJwt()) {
+        jsonWebToken = pb.getJwt();
+      }
+
       if (pb.hasRealUser()) {
         realUser = pb.getRealUser();
       }
@@ -335,6 +346,11 @@ class SecurityContext {
     return authnToken;
   }
 
+  @Nullable
+  public synchronized JwtRawPB getJsonWebToken() {
+    return jsonWebToken;
+  }
+
   /**
    * Set the token that we will use to authenticate to servers. Replaces any
    * prior token.
@@ -343,6 +359,14 @@ class SecurityContext {
     authnToken = token;
   }
 
+  /**
+   * Set the JWT that we will use to authenticate to the server. Replaces any
+   * prior JWT.
+   */
+  public synchronized void setJsonWebToken(JwtRawPB jwt) {
+    jsonWebToken = jwt;
+  }
+
   /**
    * Create an SSLEngine which will trust all certificates without verification.
    */
diff --git a/java/kudu-test-utils/src/main/java/org/apache/kudu/test/KuduTestHarness.java b/java/kudu-test-utils/src/main/java/org/apache/kudu/test/KuduTestHarness.java
index afa470b2f..30ea8a365 100644
--- a/java/kudu-test-utils/src/main/java/org/apache/kudu/test/KuduTestHarness.java
+++ b/java/kudu-test-utils/src/main/java/org/apache/kudu/test/KuduTestHarness.java
@@ -487,6 +487,10 @@ public class KuduTestHarness extends ExternalResource {
     client = asyncClient.syncClient();
   }
 
+  public String createJwtFor(String accountId, String subject, boolean isValid) throws IOException {
+    return miniCluster.createJwtFor(accountId, subject, isValid);
+  }
+
   /**
    * An annotation that can be added to each test method to
    * define additional master server flags to be used when
diff --git a/java/kudu-test-utils/src/test/java/org/apache/kudu/test/TestMiniKuduCluster.java b/java/kudu-test-utils/src/test/java/org/apache/kudu/test/TestMiniKuduCluster.java
index a36f38d0a..ddfbe6d9b 100644
--- a/java/kudu-test-utils/src/test/java/org/apache/kudu/test/TestMiniKuduCluster.java
+++ b/java/kudu-test-utils/src/test/java/org/apache/kudu/test/TestMiniKuduCluster.java
@@ -26,16 +26,21 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.net.Socket;
 
+import com.google.protobuf.ByteString;
 import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.kudu.client.AsyncKuduClient;
+import org.apache.kudu.client.Client.AuthenticationCredentialsPB;
 import org.apache.kudu.client.HostAndPort;
 import org.apache.kudu.client.KuduClient;
 import org.apache.kudu.client.KuduClient.KuduClientBuilder;
 import org.apache.kudu.client.ListTablesResponse;
 import org.apache.kudu.client.TimeoutTracker;
+import org.apache.kudu.security.Token.JwtRawPB;
 import org.apache.kudu.test.cluster.FakeDNS;
 import org.apache.kudu.test.cluster.MiniKuduCluster;
+import org.apache.kudu.test.cluster.MiniKuduCluster.MiniKuduClusterBuilder;
 import org.apache.kudu.test.junit.RetryRule;
 import org.apache.kudu.tools.Tool.CreateClusterRequestPB.JwksOptionsPB;
 import org.apache.kudu.tools.Tool.CreateClusterRequestPB.MiniOidcOptionsPB;
@@ -49,6 +54,9 @@ public class TestMiniKuduCluster {
   @Rule
   public RetryRule retryRule = new RetryRule();
 
+  @Rule
+  public KuduTestHarness harness;
+
   @Test(timeout = 50000)
   public void test() throws Exception {
     try (MiniKuduCluster cluster = new MiniKuduCluster.MiniKuduClusterBuilder()
@@ -111,16 +119,31 @@ public class TestMiniKuduCluster {
 
   @Test(timeout = 50000)
   public void testJwt() throws Exception {
-    try (MiniKuduCluster cluster = new MiniKuduCluster.MiniKuduClusterBuilder()
-                                                      .numMasterServers(NUM_MASTERS)
-                                                      .numTabletServers(0)
-                                                      .enableClientJwt()
-                                                      .addJwks("account-id", true)
-                                                      .build();
-         KuduClient client = new KuduClientBuilder(cluster.getMasterAddressesAsString()).build()) {
-      String jwt = cluster.createJwtFor("account-id", "subject", true);
-
+    try {
+      MiniKuduClusterBuilder clusterBuilder = new MiniKuduCluster.MiniKuduClusterBuilder()
+              .numMasterServers(NUM_MASTERS)
+              .numTabletServers(0)
+              .enableClientJwt()
+              .addJwks("account-id", true);
+
+      harness = new KuduTestHarness(clusterBuilder);
+      harness.before();
+      harness.startAllMasterServers();
+
+      String jwt = harness.createJwtFor("account-id", "subject", true);
       assertNotNull(jwt);
+      AuthenticationCredentialsPB credentials = AuthenticationCredentialsPB.newBuilder()
+              .setJwt(JwtRawPB.newBuilder()
+                      .setJwtData(ByteString.copyFromUtf8(jwt))
+                      .build())
+              .build();
+
+      AsyncKuduClient c = harness.getAsyncClient();
+      c.importAuthenticationCredentials(credentials.toByteArray());
+      c.getTablesList();
+
+    } catch (Exception e) {
+      throw new RuntimeException(e);
     }
   }
 


[kudu] 01/03: jwt: add control points for test binaries

Posted by ab...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 3d56a26b962d83f3f2789b2f2c7c2967322e98e9
Author: Andrew Wong <aw...@cloudera.com>
AuthorDate: Mon May 2 12:08:50 2022 -0700

    jwt: add control points for test binaries
    
    This hooks the configurations of the MiniOidc into the Java test binary,
    allowing test code to add statically-defined JWKS associated with
    specific account IDs, to be fetched via OIDC Discovery Endpoint.
    
    Change-Id: I489a67e93610467c5a2caabcf3e5603cbb49d118
    Reviewed-on: http://gerrit.cloudera.org:8080/18476
    Tested-by: Kudu Jenkins
    Reviewed-by: Attila Bukor <ab...@apache.org>
---
 .../org/apache/kudu/backup/TestKuduBackup.scala    |  6 +--
 .../apache/kudu/test/cluster/MiniKuduCluster.java  | 45 ++++++++++++++++++++--
 .../org/apache/kudu/test/TestMiniKuduCluster.java  | 17 ++++++++
 src/kudu/tools/tool.proto                          | 10 ++++-
 src/kudu/tools/tool_action_test.cc                 | 24 ++++++++++++
 5 files changed, 94 insertions(+), 8 deletions(-)

diff --git a/java/kudu-backup/src/test/scala/org/apache/kudu/backup/TestKuduBackup.scala b/java/kudu-backup/src/test/scala/org/apache/kudu/backup/TestKuduBackup.scala
index f1421b5ff..0a61f989c 100644
--- a/java/kudu-backup/src/test/scala/org/apache/kudu/backup/TestKuduBackup.scala
+++ b/java/kudu-backup/src/test/scala/org/apache/kudu/backup/TestKuduBackup.scala
@@ -609,7 +609,7 @@ class TestKuduBackup extends KuduTestSuite {
     // Check the range bounds and the hash schema of each range of the restored table.
     val restoredTable = kuduClient.openTable(s"$tableName-restore")
     assertEquals(
-        "[0 <= VALUES < 100 HASH(key) PARTITIONS 2, " +
+      "[0 <= VALUES < 100 HASH(key) PARTITIONS 2, " +
         "100 <= VALUES < 200 HASH(key) PARTITIONS 3]",
       restoredTable.getFormattedRangePartitionsWithHashSchema(10000).toString
     )
@@ -635,7 +635,7 @@ class TestKuduBackup extends KuduTestSuite {
     // Check the range bounds and the hash schema of each range of the restored table.
     val restoredTable = kuduClient.openTable(s"$tableName-restore")
     assertEquals(
-        "[0 <= VALUES < 100 HASH(key) PARTITIONS 2, " +
+      "[0 <= VALUES < 100 HASH(key) PARTITIONS 2, " +
         "100 <= VALUES < 200 HASH(key) PARTITIONS 3, " +
         "200 <= VALUES < 300 HASH(key) PARTITIONS 4]",
       restoredTable.getFormattedRangePartitionsWithHashSchema(10000).toString
@@ -694,7 +694,7 @@ class TestKuduBackup extends KuduTestSuite {
     // Check the range bounds and the hash schema of each range of the restored table.
     val restoredTable = kuduClient.openTable(s"$tableName-restore")
     assertEquals(
-        "[0 <= VALUES < 100 HASH(key) PARTITIONS 2, " +
+      "[0 <= VALUES < 100 HASH(key) PARTITIONS 2, " +
         "100 <= VALUES < 200 HASH(key) PARTITIONS 3, " +
         "200 <= VALUES < 300 HASH(key) PARTITIONS 5, " +
         "300 <= VALUES < 400 HASH(key) PARTITIONS 6]",
diff --git a/java/kudu-test-utils/src/main/java/org/apache/kudu/test/cluster/MiniKuduCluster.java b/java/kudu-test-utils/src/main/java/org/apache/kudu/test/cluster/MiniKuduCluster.java
index cd5eb2acb..31bcdc6f9 100644
--- a/java/kudu-test-utils/src/main/java/org/apache/kudu/test/cluster/MiniKuduCluster.java
+++ b/java/kudu-test-utils/src/main/java/org/apache/kudu/test/cluster/MiniKuduCluster.java
@@ -50,7 +50,10 @@ import org.apache.kudu.test.TempDirUtils;
 import org.apache.kudu.tools.Tool.ControlShellRequestPB;
 import org.apache.kudu.tools.Tool.ControlShellResponsePB;
 import org.apache.kudu.tools.Tool.CreateClusterRequestPB;
+import org.apache.kudu.tools.Tool.CreateClusterRequestPB.JwksOptionsPB;
 import org.apache.kudu.tools.Tool.CreateClusterRequestPB.MiniKdcOptionsPB;
+import org.apache.kudu.tools.Tool.CreateClusterRequestPB.MiniOidcOptionsPB;
+import org.apache.kudu.tools.Tool.CreateJwtRequestPB;
 import org.apache.kudu.tools.Tool.DaemonIdentifierPB;
 import org.apache.kudu.tools.Tool.DaemonInfoPB;
 import org.apache.kudu.tools.Tool.GetKDCEnvVarsRequestPB;
@@ -110,9 +113,11 @@ public final class MiniKuduCluster implements AutoCloseable {
   private final ImmutableList<String> locationInfo;
   private final String clusterRoot;
   private final String principal;
+  private final boolean enableClientJwt;
 
   private MiniKdcOptionsPB kdcOptionsPb;
   private final Common.HmsMode hmsMode;
+  private MiniOidcOptionsPB oidcOptionsPb;
 
   private MiniKuduCluster(boolean enableKerberos,
       int numMasters,
@@ -123,7 +128,9 @@ public final class MiniKuduCluster implements AutoCloseable {
       MiniKdcOptionsPB kdcOptionsPb,
       String clusterRoot,
       Common.HmsMode hmsMode,
-      String principal) {
+      String principal,
+      boolean enableClientJwt,
+      MiniOidcOptionsPB oidcOptionsPb) {
     this.enableKerberos = enableKerberos;
     this.numMasters = numMasters;
     this.numTservers = numTservers;
@@ -133,6 +140,8 @@ public final class MiniKuduCluster implements AutoCloseable {
     this.kdcOptionsPb = kdcOptionsPb;
     this.principal = principal;
     this.hmsMode = hmsMode;
+    this.enableClientJwt = enableClientJwt;
+    this.oidcOptionsPb = oidcOptionsPb;
 
     if (clusterRoot == null) {
       // If a cluster root was not set, create a unique temp directory to use.
@@ -227,7 +236,8 @@ public final class MiniKuduCluster implements AutoCloseable {
         .addAllExtraTserverFlags(extraTserverFlags)
         .setMiniKdcOptions(kdcOptionsPb)
         .setClusterRoot(clusterRoot)
-        .setPrincipal(principal);
+        .setPrincipal(principal)
+        .setMiniOidcOptions(oidcOptionsPb);
 
     // Set up the location mapping command flag if there is location info.
     if (!locationInfo.isEmpty()) {
@@ -323,6 +333,18 @@ public final class MiniKuduCluster implements AutoCloseable {
     return principal;
   }
 
+  public String createJwtFor(String accountId, String subject, boolean isValid) throws IOException {
+    ControlShellResponsePB resp = sendRequestToCluster(ControlShellRequestPB.newBuilder()
+        .setCreateJwt(CreateJwtRequestPB
+            .newBuilder()
+            .setAccountId(accountId)
+            .setSubject(subject)
+            .setIsValidKey(isValid)
+            .build())
+        .build());
+    return resp.getCreateJwt().getJwt();
+  }
+
   /**
    * Starts a master identified by a host and port.
    * Does nothing if the server was already running.
@@ -703,8 +725,10 @@ public final class MiniKuduCluster implements AutoCloseable {
     private final List<String> locationInfo = new ArrayList<>();
     private String clusterRoot = null;
     private String principal = "kudu";
+    private boolean enableClientJwt = false;
 
     private MiniKdcOptionsPB.Builder kdcOptionsPb = MiniKdcOptionsPB.newBuilder();
+    private MiniOidcOptionsPB.Builder oidcOptionsPb = MiniOidcOptionsPB.newBuilder();
     private Common.HmsMode hmsMode = Common.HmsMode.NONE;
 
     public MiniKuduClusterBuilder numMasterServers(int numMasterServers) {
@@ -726,6 +750,11 @@ public final class MiniKuduCluster implements AutoCloseable {
       return this;
     }
 
+    public MiniKuduClusterBuilder enableClientJwt() {
+      enableClientJwt = true;
+      return this;
+    }
+
     public MiniKuduClusterBuilder enableHiveMetastoreIntegration() {
       hmsMode = Common.HmsMode.ENABLE_METASTORE_INTEGRATION;
       return this;
@@ -788,6 +817,15 @@ public final class MiniKuduCluster implements AutoCloseable {
       return this;
     }
 
+    public MiniKuduClusterBuilder addJwks(String accountId, boolean isValid) {
+      this.oidcOptionsPb.addJwksOptions(
+          JwksOptionsPB.newBuilder()
+              .setAccountId(accountId)
+              .setIsValidKey(isValid)
+              .build());
+      return this;
+    }
+
     /**
      * Builds and starts a new {@link MiniKuduCluster} using builder state.
      * @return the newly started {@link MiniKuduCluster}
@@ -798,7 +836,8 @@ public final class MiniKuduCluster implements AutoCloseable {
           new MiniKuduCluster(enableKerberos,
               numMasterServers, numTabletServers,
               extraTabletServerFlags, extraMasterServerFlags, locationInfo,
-              kdcOptionsPb.build(), clusterRoot, hmsMode, principal);
+              kdcOptionsPb.build(), clusterRoot, hmsMode, principal, enableClientJwt,
+              oidcOptionsPb.build());
       try {
         cluster.start();
       } catch (IOException e) {
diff --git a/java/kudu-test-utils/src/test/java/org/apache/kudu/test/TestMiniKuduCluster.java b/java/kudu-test-utils/src/test/java/org/apache/kudu/test/TestMiniKuduCluster.java
index 6844fcdd4..a36f38d0a 100644
--- a/java/kudu-test-utils/src/test/java/org/apache/kudu/test/TestMiniKuduCluster.java
+++ b/java/kudu-test-utils/src/test/java/org/apache/kudu/test/TestMiniKuduCluster.java
@@ -37,6 +37,8 @@ import org.apache.kudu.client.TimeoutTracker;
 import org.apache.kudu.test.cluster.FakeDNS;
 import org.apache.kudu.test.cluster.MiniKuduCluster;
 import org.apache.kudu.test.junit.RetryRule;
+import org.apache.kudu.tools.Tool.CreateClusterRequestPB.JwksOptionsPB;
+import org.apache.kudu.tools.Tool.CreateClusterRequestPB.MiniOidcOptionsPB;
 
 public class TestMiniKuduCluster {
 
@@ -107,6 +109,21 @@ public class TestMiniKuduCluster {
     }
   }
 
+  @Test(timeout = 50000)
+  public void testJwt() throws Exception {
+    try (MiniKuduCluster cluster = new MiniKuduCluster.MiniKuduClusterBuilder()
+                                                      .numMasterServers(NUM_MASTERS)
+                                                      .numTabletServers(0)
+                                                      .enableClientJwt()
+                                                      .addJwks("account-id", true)
+                                                      .build();
+         KuduClient client = new KuduClientBuilder(cluster.getMasterAddressesAsString()).build()) {
+      String jwt = cluster.createJwtFor("account-id", "subject", true);
+
+      assertNotNull(jwt);
+    }
+  }
+
   @Test(timeout = 50000)
   public void testKerberos() throws Exception {
     FakeDNS.getInstance().install();
diff --git a/src/kudu/tools/tool.proto b/src/kudu/tools/tool.proto
index 1fa7878a1..94b20aba3 100644
--- a/src/kudu/tools/tool.proto
+++ b/src/kudu/tools/tool.proto
@@ -72,7 +72,7 @@ message CreateClusterRequestPB {
   // Options pertaining to a single JWKS.
   message JwksOptionsPB {
     optional string account_id = 1;
-    optional bool is_valid_key = 2;
+    optional bool is_valid_key = 2 [ default = true ];
   }
 
   message MiniOidcOptionsPB {
@@ -223,6 +223,7 @@ message ControlShellResponsePB {
     GetMastersResponsePB get_masters = 2;
     GetTServersResponsePB get_tservers = 3;
     GetKDCEnvVarsResponsePB get_kdc_env_vars = 4;
+    CreateJwtResponsePB create_jwt = 5;
   }
 }
 
@@ -249,6 +250,7 @@ message ControlShellRequestPB {
     SetDaemonFlagRequestPB set_daemon_flag = 12;
     PauseDaemonRequestPB pause_daemon = 13;
     ResumeDaemonRequestPB resume_daemon = 14;
+    CreateJwtRequestPB create_jwt = 15;
   }
 }
 
@@ -548,7 +550,7 @@ message TablesInfoPB {
   repeated TableInfoPB tables = 1;
 }
 
-message CreateJwtPB {
+message CreateJwtRequestPB {
   // The account ID with which a JWT will be created.
   optional string account_id = 1;
 
@@ -558,3 +560,7 @@ message CreateJwtPB {
   // Whether or not the returned token should supply a valid key ID.
   optional bool is_valid_key = 3;
 }
+
+message CreateJwtResponsePB {
+  optional string jwt = 1;
+}
diff --git a/src/kudu/tools/tool_action_test.cc b/src/kudu/tools/tool_action_test.cc
index 11153aa98..df49b83b3 100644
--- a/src/kudu/tools/tool_action_test.cc
+++ b/src/kudu/tools/tool_action_test.cc
@@ -43,6 +43,7 @@
 #include "kudu/tools/tool.pb.h"
 #include "kudu/util/env.h"
 #include "kudu/util/env_util.h"
+#include "kudu/util/mini_oidc.h"
 #include "kudu/util/path_util.h"
 #include "kudu/util/pb_util.h"
 #include "kudu/util/status.h"
@@ -179,6 +180,14 @@ Status ProcessRequest(const ControlShellRequestPB& req,
           opts.principal = cc.principal();
         }
       }
+      if (cc.has_mini_oidc_options()) {
+        opts.enable_client_jwt = true;
+        const auto& mini_oidc_opts = cc.mini_oidc_options();
+        for (const auto& jwks_opt : mini_oidc_opts.jwks_options()) {
+          opts.mini_oidc_options.account_ids.emplace(
+              jwks_opt.account_id(), jwks_opt.is_valid_key());
+        }
+      }
 
       cluster->reset(new ExternalMiniCluster(std::move(opts)));
       break;
@@ -341,6 +350,21 @@ Status ProcessRequest(const ControlShellRequestPB& req,
       RETURN_NOT_OK(daemon->Resume());
       break;
     }
+    case ControlShellRequestPB::kCreateJwt:
+    {
+      const auto& r = req.create_jwt();
+      if (!r.has_account_id() || !r.has_subject()) {
+        return Status::InvalidArgument("JWT creation requires account ID and subject");
+      }
+      RETURN_NOT_OK(CheckClusterExists(*cluster));
+      auto* oidc = (*cluster)->oidc();
+      if (!oidc) {
+        return Status::ConfigurationError("mini-cluster must be configured with MiniOidc");
+      }
+      auto jwt = oidc->CreateJwt(r.account_id(), r.subject(), r.is_valid_key());
+      *resp->mutable_create_jwt()->mutable_jwt() = jwt;
+      break;
+    }
     default:
       RETURN_NOT_OK(Status::InvalidArgument("unknown cluster control request"));
   }


[kudu] 02/03: [jwt] Verify JWKS URL server TLS certificate by default

Posted by ab...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 8595384de007963181cca59b0248b85169f18792
Author: Zoltan Chovan <zc...@cloudera.com>
AuthorDate: Mon Apr 3 15:38:50 2023 +0200

    [jwt] Verify JWKS URL server TLS certificate by default
    
    This commit is to pull IMPALA-11922 code into the Kudu jwt handling,
    with some modifications.
    
    This change introduces:
      1. verification of JWKS server TLS certificate by default
      2. jwks_verify_server_certificate Kudu startup flag
    
    Instead of introducing a new flag such as 'jwks_ca_certificate' the
    already existing 'trusted_certificate_file' flag is reused.
    
    The TLS certificate verification is not used in unit-tests, however
    security-itest is set up with the verification enabled.
    
    Change-Id: I0fd7b53d651786bbe57642dd14cd477055b80c78
    Reviewed-on: http://gerrit.cloudera.org:8080/19709
    Reviewed-by: Attila Bukor <ab...@apache.org>
    Tested-by: Kudu Jenkins
---
 src/kudu/integration-tests/CMakeLists.txt      |   3 +-
 src/kudu/integration-tests/security-itest.cc   | 127 +++++++++++++++++++++
 src/kudu/mini-cluster/CMakeLists.txt           |   1 +
 src/kudu/mini-cluster/external_mini_cluster.cc |   7 +-
 src/kudu/security/test/test_certs.cc           | 150 +++++++++++++++++++++++++
 src/kudu/security/test/test_certs.h            |   7 ++
 src/kudu/server/server_base.cc                 |  17 ++-
 src/kudu/util/jwt-util-internal.h              |  11 +-
 src/kudu/util/jwt-util-test.cc                 |  54 ++++-----
 src/kudu/util/jwt-util.cc                      |  38 +++++--
 src/kudu/util/jwt-util.h                       |  19 +++-
 src/kudu/util/mini_oidc.cc                     |  32 ++++--
 src/kudu/util/mini_oidc.h                      |   7 ++
 13 files changed, 421 insertions(+), 52 deletions(-)

diff --git a/src/kudu/integration-tests/CMakeLists.txt b/src/kudu/integration-tests/CMakeLists.txt
index 0e7ca24b5..c390871c0 100644
--- a/src/kudu/integration-tests/CMakeLists.txt
+++ b/src/kudu/integration-tests/CMakeLists.txt
@@ -48,8 +48,7 @@ target_link_libraries(itest_util
   kudu_fs
   kudu_test_util
   kudu_tools_test_util
-  kudu_tools_util
-  security_test_util)
+  kudu_tools_util)
 add_dependencies(itest_util
   kudu-master
   kudu-tserver)
diff --git a/src/kudu/integration-tests/security-itest.cc b/src/kudu/integration-tests/security-itest.cc
index cdc642f18..0b77adbb9 100644
--- a/src/kudu/integration-tests/security-itest.cc
+++ b/src/kudu/integration-tests/security-itest.cc
@@ -57,6 +57,7 @@
 #include "kudu/rpc/rpc_controller.h"
 #include "kudu/security/kinit_context.h"
 #include "kudu/security/test/mini_kdc.h"
+#include "kudu/security/test/test_certs.h"
 #include "kudu/security/token.pb.h"
 #include "kudu/server/server_base.pb.h"
 #include "kudu/server/server_base.proxy.h"
@@ -512,6 +513,7 @@ void GetFullBinaryPath(string* binary) {
   (*binary) = JoinPathSegments(DirName(exe), *binary);
 }
 
+
 TEST_F(SecurityITest, TestJwtMiniCluster) {
   SKIP_IF_SLOW_NOT_ALLOWED();
 
@@ -527,6 +529,22 @@ TEST_F(SecurityITest, TestJwtMiniCluster) {
     { kInvalidAccount, false },
   };
   oidc_opts.lifetime_ms = kLifetimeMs;
+
+  // Set up certificates for the JWKS server
+  string ca_certificate_file;
+  string private_key_file;
+  string certificate_file;
+  ASSERT_OK(kudu::security::CreateTestSSLCertWithChainSignedByRoot(GetTestDataDirectory(),
+                                                                   &certificate_file,
+                                                                   &private_key_file,
+                                                                   &ca_certificate_file));
+  // set the certs and private key for the jwks webserver
+  oidc_opts.private_key_file = private_key_file;
+  oidc_opts.server_certificate = certificate_file;
+  // set the ca_cert (jwks certificate verification is enabled by default)
+  cluster_opts_.extra_master_flags.push_back(Substitute("--trusted_certificate_file=$0",
+                                                        ca_certificate_file));
+
   cluster_opts_.mini_oidc_options = std::move(oidc_opts);
   ASSERT_OK(StartCluster());
   const auto* const kSubject = "kudu-user";
@@ -585,6 +603,115 @@ TEST_F(SecurityITest, TestJwtMiniCluster) {
   }
 }
 
+TEST_F(SecurityITest, TestJwtMiniClusterWithInvalidCert) {
+  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 uint64_t kLifetimeMs = 1000;
+  oidc_opts.account_ids = {
+    { kValidAccount, true }
+  };
+  oidc_opts.lifetime_ms = kLifetimeMs;
+  const auto* const kSubject = "kudu-user";
+
+  // Set up certificates for the JWKS server
+  string ca_certificate_file;
+  string private_key_file;
+  string certificate_file;
+
+  ASSERT_OK(kudu::security::CreateTestSSLExpiredCertWithChainSignedByRoot(GetTestDataDirectory(),
+                                                                   &certificate_file,
+                                                                   &private_key_file,
+                                                                   &ca_certificate_file));
+
+  // set the certs and private key for the jwks webserver
+  oidc_opts.private_key_file = private_key_file;
+  oidc_opts.server_certificate = certificate_file;
+  // set the ca_cert (jwks certificate verification is enabled by default)
+  cluster_opts_.extra_master_flags.push_back(Substitute("--trusted_certificate_file=$0",
+                                                        ca_certificate_file));
+
+  cluster_opts_.mini_oidc_options = std::move(oidc_opts);
+  ASSERT_OK(StartCluster());
+
+  {
+    KuduClientBuilder client_builder;
+    client::AuthenticationCredentialsPB pb;
+    security::JwtRawPB jwt = security::JwtRawPB();
+    *jwt.mutable_jwt_data() = cluster_->oidc()->CreateJwt(kValidAccount, kSubject, true);
+    *pb.mutable_jwt() = std::move(jwt);
+    string creds;
+    CHECK(pb.SerializeToString(&creds));
+
+    for (auto i = 0; i < cluster_->num_masters(); ++i) {
+      client_builder.add_master_server_addr(cluster_->master(i)->bound_rpc_addr().ToString());
+    }
+    client_builder.import_authentication_credentials(creds);
+    client_builder.require_authentication(true);
+
+    shared_ptr<KuduClient> client;
+
+    Status s = client_builder.Build(&client);
+    ASSERT_FALSE(s.ok());
+    ASSERT_STR_CONTAINS(s.ToString(),
+                        "SSL certificate problem: unable to get local issuer certificate");
+  }
+}
+
+TEST_F(SecurityITest, TestJwtMiniClusterWithUntrustedCert) {
+  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 uint64_t kLifetimeMs = 1000;
+  oidc_opts.account_ids = {
+    { kValidAccount, true }
+  };
+  oidc_opts.lifetime_ms = kLifetimeMs;
+  const auto* const kSubject = "kudu-user";
+
+  // Set up certificates for the JWKS server
+  string ca_certificate_file;
+  string private_key_file;
+  string certificate_file;
+  ASSERT_OK(kudu::security::CreateTestSSLCertWithChainSignedByRoot(GetTestDataDirectory(),
+                                                                   &certificate_file,
+                                                                   &private_key_file,
+                                                                   &ca_certificate_file));
+  // set the certs and private key for the jwks webserver
+  // jwks certificate verification is enabled by default, so we won't have to set it
+  oidc_opts.private_key_file = private_key_file;
+  oidc_opts.server_certificate = certificate_file;
+
+  cluster_opts_.mini_oidc_options = std::move(oidc_opts);
+  ASSERT_OK(StartCluster());
+
+  {
+    KuduClientBuilder client_builder;
+    client::AuthenticationCredentialsPB pb;
+    security::JwtRawPB jwt = security::JwtRawPB();
+    *jwt.mutable_jwt_data() = cluster_->oidc()->CreateJwt(kValidAccount, kSubject, true);
+    *pb.mutable_jwt() = std::move(jwt);
+    string creds;
+    CHECK(pb.SerializeToString(&creds));
+
+    for (auto i = 0; i < cluster_->num_masters(); ++i) {
+      client_builder.add_master_server_addr(cluster_->master(i)->bound_rpc_addr().ToString());
+    }
+    client_builder.import_authentication_credentials(creds);
+    client_builder.require_authentication(true);
+
+    shared_ptr<KuduClient> client;
+
+    Status s = client_builder.Build(&client);
+    ASSERT_FALSE(s.ok());
+    ASSERT_STR_CONTAINS(s.ToString(), "SSL peer certificate or SSH remote key was not OK");
+  }
+}
+
 TEST_F(SecurityITest, TestWorldReadableKeytab) {
   const string credentials_name = GetTestPath("insecure.keytab");
   NO_FATALS(CreateWorldReadableFile(credentials_name));
diff --git a/src/kudu/mini-cluster/CMakeLists.txt b/src/kudu/mini-cluster/CMakeLists.txt
index f10497a0d..7748795e6 100644
--- a/src/kudu/mini-cluster/CMakeLists.txt
+++ b/src/kudu/mini-cluster/CMakeLists.txt
@@ -45,6 +45,7 @@ set(MINI_CLUSTER_LIBS
   tserver
   tserver_proto
   tserver_service_proto
+  security_test_util
   wire_protocol_proto)
 
 if (NOT NO_CHRONY)
diff --git a/src/kudu/mini-cluster/external_mini_cluster.cc b/src/kudu/mini-cluster/external_mini_cluster.cc
index 948508e35..0e61192dc 100644
--- a/src/kudu/mini-cluster/external_mini_cluster.cc
+++ b/src/kudu/mini-cluster/external_mini_cluster.cc
@@ -276,8 +276,13 @@ Status ExternalMiniCluster::Start() {
   std::shared_ptr<JwtVerifier> jwt_verifier = nullptr;
   if (opts_.enable_client_jwt) {
     oidc_.reset(new MiniOidc(opts_.mini_oidc_options));
+    // Set up certificates for the JWKS server
     RETURN_NOT_OK_PREPEND(oidc_->Start(), "Failed to start OIDC endpoints");
-    jwt_verifier = std::make_shared<PerAccountKeyBasedJwtVerifier>(oidc_->url());
+    jwt_verifier =
+        std::make_shared<PerAccountKeyBasedJwtVerifier>(oidc_->url(),
+                                                        true,
+                                                        opts_.mini_oidc_options.server_certificate);
+
   }
 
   RETURN_NOT_OK_PREPEND(
diff --git a/src/kudu/security/test/test_certs.cc b/src/kudu/security/test/test_certs.cc
index 321a4ef59..d322e317c 100644
--- a/src/kudu/security/test/test_certs.cc
+++ b/src/kudu/security/test/test_certs.cc
@@ -971,5 +971,155 @@ KH5H1VGmllMdZDHOamHHKA8mEDI4eAKY3HoOS4rfioT8Tks=
   return Status::OK();
 }
 
+// These certificates are the same as used in CreateTestSSLCertWithChainSignedByRoot,
+// except the ca_cert, which is replaced with an expired version.
+Status CreateTestSSLExpiredCertWithChainSignedByRoot(const string& dir,
+                                              string* cert_file,
+                                              string* key_file,
+                                              string* expired_ca_cert_file) {
+
+   const char* kCert = R"(
+-----BEGIN CERTIFICATE-----
+MIIFizCCA3OgAwIBAgICEAAwDQYJKoZIhvcNAQEFBQAwUTEXMBUGA1UEAwwOSW50
+ZXJtZWRpYXRlQ0ExCzAJBgNVBAgMAkNBMQswCQYDVQQGEwJVUzENMAsGA1UECgwE
+QWNtZTENMAsGA1UECwwES3VkdTAeFw0xNzA4MTEyMTM4MDZaFw00NDEyMjYyMTM4
+MDZaMEwxEjAQBgNVBAMMCWxvY2FsaG9zdDELMAkGA1UECAwCQ0ExCzAJBgNVBAYT
+AlVTMQ0wCwYDVQQKDARBY21lMQ0wCwYDVQQLDARLdWR1MIICIjANBgkqhkiG9w0B
+AQEFAAOCAg8AMIICCgKCAgEAqevNYH73n4kARZtMsHRucdKmqVd/xxztMlK5VOor
+ERUBhKVVOw3kpDrN9z80ldIkpOrtrfE7Ame/nA9v4k6P3minPEm1qCA/kvaAodtT
+4HjAkrPc+fto6VO6+aUV6l+ckAV/79lOuc7AutNlvvPtBQQcgOKvlNUSRKwM7ndy
+dO4ZAa+uP9Wtsd0gl8b5F3P8vwevD3a0+iDvwSd3pi2s/BeVgRwvOxJzud8ipZ/A
+ZmZN8Df9nHw5lsqLdNnqHXjTVCNXLnYXQC4gKU56fzyZL595liuefyQxiGY+dCCn
+CpqlSsHboJVC/F3OaQi3xVRTB5l2Nwb149EIadwCF0OulZCuYljJ5y9H2bECXEjP
+e5aOdz9d8W3/T7p9vBKWctToeCpqKXUd+8RPudh0D0sUHuwQ4u4S1K6X+eK+gGhT
+HOnPwt+P8ytG0M463z5Gh9feW9ZDIYoiFckheFBAHxsgDWhjYpFmYireLLXMbyaM
+s5v/AxPNRAsx3vAAd0M0vGOpdgEJ9V1MsKmxkPO/tDC3zmnv6uJhtJfrOAKxwiGC
+fDe4IoSC6H5fTxeAgw6BG5onS1UPLADL8NA/M1y8qiSCZS/5S0cHoJp5AxDfZSSR
+O49ispjqcONRwckcRJ5Pbl0IA+wGyg2DuI9LaqS5kKWp5AE8VCLPz7yepDkRnnjO
+3m8CAwEAAaNyMHAwDAYDVR0TAQH/BAIwADAdBgNVHQ4EFgQUZBZLZZaUfyIK/8B7
+GIIWDqeEvDgwHwYDVR0jBBgwFoAU8KctfaqAq0887CHqDsIC0Rkg7oQwCwYDVR0P
+BAQDAgWgMBMGA1UdJQQMMAoGCCsGAQUFBwMBMA0GCSqGSIb3DQEBBQUAA4ICAQA3
+XJXk9CbzdZUQugPI43LY88g+WjbTJfc/KtPSkHN3GjBBh8C0He7A2tp6Xj/LELmx
+crq62FzcFBnq8/iSdFITaYWRo0V/mXlpv2cpPebtwqbARCXUHGvF4/dGk/kw7uK/
+ohZJbeNySuQmQ5SQyfTdVA30Z0OSZ4jp24jC8uME7L8XOcFDgCRw01QNOISpi/5J
+BqeuFihmu/odYMHiEJdCXqe+4qIFfTh0mbgQ57l/geZm0K8uCEiOdTzSMoO8YdO2
+tm6EGNnc4yrVywjIHIvSy6YtNzd4ZM1a1CkEfPvGwe/wI1DI/zl3aJ721kcMPken
+rgEA4xXTPh6gZNMELIGZfu/mOTCFObe8rrh4QSaW4L+xa/VrLEnQRxuXAYGnmDWF
+e79aA+uXdS4+3OysNgEf4qDBt/ZquS/31DBdfJ59VfXWxp2yxMcGhcfiOdnx2Jy5
+KO8wdpXJA/7uwTJzsjLrIgfZnserOiBwE4luaHhDmKDGNVQvhkMq5tdtMdzuwn3/
+n6P1UwbFPiRGIzEAo0SSC1PRT8phv+5y0B1+gcj/peFymZVE+gRcrv9irVQqUpAY
+Lo9xrClAJ2xx4Ouz1GprKPoHdVyqtgcLXN4Oyi8Tehu96Zf6GytSEfTXsbQp+GgR
+TGRhKnDySjPhLp/uObfVwioyuAyA5mVCwjsZ/cvUUA==
+-----END CERTIFICATE-----
+-----BEGIN CERTIFICATE-----
+MIIHmDCCA4CgAwIBAgICEAAwDQYJKoZIhvcNAQEFBQAwVjELMAkGA1UEBhMCVVMx
+CzAJBgNVBAgMAkNBMQswCQYDVQQHDAJTRjENMAsGA1UECgwEQWNtZTENMAsGA1UE
+CwwES3VkdTEPMA0GA1UEAwwGUk9PVENBMB4XDTE3MDgxMTIxMzUzNVoXDTQ0MTIy
+NzIxMzUzNVowUTEXMBUGA1UEAwwOSW50ZXJtZWRpYXRlQ0ExCzAJBgNVBAgMAkNB
+MQswCQYDVQQGEwJVUzENMAsGA1UECgwEQWNtZTENMAsGA1UECwwES3VkdTCCAiIw
+DQYJKoZIhvcNAQEBBQADggIPADCCAgoCggIBAM1X35LT/eBWBt0Uqqh3DSUyY3K8
+HLIlX3ZXg2Nx6y8yqhw5UGVFZl0uYBDo2DSlTl4sey+AxLIbpQI9ArRA+xqmFynV
+jheB9otudnA8hVwi/e9o+m+VSjG+HPRjSS5hwdPgpJG8DCPSmGyUUFtf3v0NxkUq
+Is+fB5qhQ36aQkI+MwQsSlHR+YrrKKVnE3f911wr9OScQP5KHjrZLQex8OmpWD9G
+v4P9jfVSUwmNEXXjmXDhNG/1R4ofX6HogZR6lBmRNGbcjjWRZQmPrOe9YcdkMLD0
+CdaUyKikqqW6Ilxs7scfuCGqwBWqh66tY18MBMHnt0bL26atTPduKYqulJ1pijio
+DUrzqtAzm7PirqPZ4aOJ9PNjdQs9zH3Zad3pcjfjpdKj4a/asX0st631J5jE6MLB
+LcbAerb/Csr/+tD0TOxwWlA+p/6wPb8ECflQLkvDDEY5BrRGdqYDpEOdm1F9DWQh
+y0RB8rWJMkxC/tTqYHfeaphzCxndLRsZQKVcPiqWCT7b431umIjPaDhsykNlcU3N
+f0V7V/fLY6wwuACngS0BLQuMrXy5FyhmWnUBeWwHfAeTxCkHlF+cVT6wHmeOuGbC
+c1piq7O7puKdC3UjO7Nn+WoOb2B6Qm/dajHpj5myxYJa5tGQGeUnWPwjjMQR557k
+HzugGAzkuG1ASQrhAgMBAAGjdTBzMA8GA1UdEwEB/wQFMAMBAf8wHQYDVR0OBBYE
+FPCnLX2qgKtPPOwh6g7CAtEZIO6EMB8GA1UdIwQYMBaAFE/9XKaDey5kC8f3bCeU
+HW46abboMAsGA1UdDwQEAwIBpjATBgNVHSUEDDAKBggrBgEFBQcDATANBgkqhkiG
+9w0BAQUFAAOCBAEAIaD2yzjTFdn61A4Qi+ek3fBJaDNQZytd0rHb49v3T+mdj/MI
+yShI1qezDFkg2FP1LfNgjuQl/T+g0BloXatAhZ/dj20Y8oN6bmilV+r2YLJbvbTn
+3hI+MxNf3Ue3FmIrwKK3QdkWcDBURpyYaDO71oxPl9QNfdhWCGHB/oWKU2y4Qt/O
+aPy+CmBAZEclX+hsdUBDJG5vuujpv4myCFwpLgFKNQX3XqCPLc4SRjfyla2YmeZv
+j7KKYh8XOWBbBF0BnWD94WzUDIBmFlUfS32aJTvd7tVaWXwH8rGwDfLN8i05UD9G
+zc3uuFH+UdzWVymk/4svKIPlB2nw9vPV8hvRRah0yFN3EQqAF0vQtwVJF/VwtZdg
+ahH0DykYTf7cKtFXE40xB7YgwDLXd3UiXfo3USW28uKqsrO52xYuUTBn+xkilds1
+tNKwtpXFWP2PUk92ficxoqi1cJnHxIIt5HKskFPgfIpzkpR8IM/vsom1a5fn4TT1
+aJbO5FsZTXQMxFLYWiSOMhTZMp3iNduxMYPosngjjKPEIkTQHKkedpF+CAGIMOKE
+BVa0vHyF34laKMMDT8d9yxwBJLqjlBohNsLLZa/Y90ThaMw+QYn/GZATB+7ng+ip
+VdGAQrghsGSxP+47HZ6WgBrlRdUWN1d1tlN2NBMHLucpbra5THGzl5MlaSVBYZb6
+yXI+2lwcTnnEkKv2zoA4ZHWdtLn/b1y4NKNg205TA+sOZcl6B1BgMe/rFuXdZe9Q
+/b6Tjz65qL4y1ByBVBJNhQQairw6cypHzwzC3w6ub1ZXtFqnTlU8fFcHGeOyydYS
+NfoepF0w2v0ounqD+6rN1CH/ERVb4FCEN19HQ3z+rAj19z2h6m/l5QEKI7bz8ghD
+8yxyqJz+L9XpfOo1yZfHQJckilY6BBIGWyeetJBmvkwv2WPt+3pX1u7h5LkvNRj2
+3fItf486zqtzUi+i/E//rS4gD/rRr4a85U8GSfp3LSAbtmfC0LNYUYA9Dcc0LSpl
+9alNuEpBHSHXlCVh4bcOb0L9n5XNdMcUYBo14hQdP0K1G7TounuAXFKYIQeyNyoi
+OAZ+eb7Y2xNnkY/ps/kyhsZgOJyiDZhdcruK3FIUGYlg5aVjQTB8H0c3/5SZnSky
+6779yMKztFXj9ctYU0YyJXWdF0xP/vi1gjQx/hJnDfXFfIOmeJdQSC08BGyK/PeC
+8zAS380bgzOza/eBL6IK0RqytbWgdoLrUQQfa1+f7AQxDDdoOkUenM0HSWjKfCuG
+m1/N7KUDHtnjVIHWqRefTPg1/tQjVY8/zgxN8MyAy+D95y4rawjsJf1dL6c0+zGv
+Wd40Cr+wAdHKN6t/oransoxu0EZ3HcSOI1umFg==
+-----END CERTIFICATE-----
+)";
+  const char* kKey = R"(
+-----BEGIN PRIVATE KEY-----
+MIIJQgIBADANBgkqhkiG9w0BAQEFAASCCSwwggkoAgEAAoICAQCp681gfvefiQBF
+m0ywdG5x0qapV3/HHO0yUrlU6isRFQGEpVU7DeSkOs33PzSV0iSk6u2t8TsCZ7+c
+D2/iTo/eaKc8SbWoID+S9oCh21PgeMCSs9z5+2jpU7r5pRXqX5yQBX/v2U65zsC6
+02W+8+0FBByA4q+U1RJErAzud3J07hkBr64/1a2x3SCXxvkXc/y/B68PdrT6IO/B
+J3emLaz8F5WBHC87EnO53yKln8BmZk3wN/2cfDmWyot02eodeNNUI1cudhdALiAp
+Tnp/PJkvn3mWK55/JDGIZj50IKcKmqVKwduglUL8Xc5pCLfFVFMHmXY3BvXj0Qhp
+3AIXQ66VkK5iWMnnL0fZsQJcSM97lo53P13xbf9Pun28EpZy1Oh4KmopdR37xE+5
+2HQPSxQe7BDi7hLUrpf54r6AaFMc6c/C34/zK0bQzjrfPkaH195b1kMhiiIVySF4
+UEAfGyANaGNikWZiKt4stcxvJoyzm/8DE81ECzHe8AB3QzS8Y6l2AQn1XUywqbGQ
+87+0MLfOae/q4mG0l+s4ArHCIYJ8N7gihILofl9PF4CDDoEbmidLVQ8sAMvw0D8z
+XLyqJIJlL/lLRwegmnkDEN9lJJE7j2KymOpw41HByRxEnk9uXQgD7AbKDYO4j0tq
+pLmQpankATxUIs/PvJ6kORGeeM7ebwIDAQABAoICAATegvYe7U2fCWj1OE9eJsQQ
+O0JjBYBZLdrhT/pE85L7vR1l93lHvqOOI9TP9NvON8qaCNGRNhWtj2oTbytXAPxo
+l1I88n2s3uWBNtJsjIzEKRCLIuvu7mSxR4xb1LLwpnXiEnZ3DbB5YkB4SlQcfVBF
+e+Odm1ZyfKGHJJ+4wIjlQcYwmJevsdiE86glxYGMi1OWDsgsqKb6RqSMUvtqF6jp
+rBkVC61vq+1JnZ6NY2AL0nPtxtCzJptRlol0rSbHDZc9pAPq0mO+bqGAZDY9ME6T
+DVLmURZnnRvBgkylmuPM5qurvnVtkYvVzFJqM4nuDqsLFL4i7uzmUo1mBpFQGTKY
+BNhxyiKB9kNH/98coCZ2COA+y2rLU0kv65dsi40TRtH6YEzYlDM2M8hwTrs8b7Rp
+B07h2PROdPORM/UlKxrpPPhyQ5SC3sgEryOKUJCkeA4H3TSFcLrFVGcBpT+65JgS
+1+LZ4UEodPTY6ofnXI1naOyA5AkK/E2ut0g6YDQBpna1VdNNd7qp5to2OOnZzOKI
+7mZr7mZ0jW+YtAAD2/SaJw754qn+mLl7SvqP727JHY6jiqceDThMh8PKCVbe4rPV
+4jRE98E393HdYunJ3Ep7LBc8foTN8EWTynNJPazL7Vk3i+fRsNJONIuwsFb7EEpg
+g2oqQEPYqDoHtp40g2MRAoIBAQDgFAHLJlX29r3VP6tCI0+J+4jg8qCYtOXTDEzU
+mD8fhgLIu24SSa6/B94CNpRpS6TYwOzfF9Dim7y+0CcNqmrm8n+UyDSbf40jCiSz
+7F570X/8zh272PrtRK9flDD/oKfHXMC+tZFPJmdwpShxCjBxcz1VTTFy/ipUIxof
+kXlD/VW8bu4C0YKcHxM4fNXsZRqP3HFKoNx/f6n0HOx5yk9mx2lWLTV1BsIqL2d5
+nAW/VWvcy+J40M4apIafxfkSNIdjk/MJpctL78egVY5LNZy413MolGNWQLxT54eg
+RptpGcPjt18me03eo5DozB/o/aMw30aqVC4NCW+kzEHBBPX9AoIBAQDCILiGvoRk
+pouZ5kEuCCxL6iJ8dF5dDLq1/afuQIDACAv7rkrxRb5hS1DIcvpQV8Pbu+9bnKno
+tnExFQPeCGC4c8xBx6OChBN8aa2HVsJv23HOp/G5ZSg1q6pmI/j+SDGiHzXkq36Q
+LwFEJc0haMffzPFj6dy/Rvigo/uVidr/teRREuYwWv3ZBUDJ1HFj2RBMdBMJ35lC
+sVP3vESiiyDOQqGbKdJ9Y3HvKZiYKsfOxwBO62kPbq7gIDhaHhl1U8QXXSUZHnfV
+IAUSpcKRpS8h/A+mE+Y626bYtl0RiGj0LvrvWQvPOgk4lQ2jYv05F3kji06sxPS2
+Y34ylLVw9dvbAoIBAC7doF5n3zTu+FdAoMYNcpZOaJt7w4EM3MCeYvdX/GPQeIaZ
+RPVIOec0cweNeM7pBkpbV291oLe0kO5rxK9EBGXXND3e/bnEHLXGalTDTCOjdpxe
+U7O1Nw4m/nMEIJdmd5Dn4lxAx2qBgsL5mBLEactgqeRMZ9pANIQyb0VI/M7ujl8B
+6H/oZ+PVUATRf0CZCMwr8/oC2PtFrTskTYVPffnmHS7r97FJP5TpI0A5FK6m5A9j
+CTPxoBnMbWe/VU+scuCt0fgjl/iC5wKuwjsStHuofCpxlrE0iu8VjrVD7z81J1Za
+ROlcgrXdCfLWtpnZaqdPG42GW7dYUORr4BjJu9UCggEAI5rEvVHsDlnNeOiWQ88T
+8Mh8kr71H7PZ+s8PIc+Kza2sJPkOnbng9Q9PPbR43It8TKzndbICJ8BuekYUc4Ct
+3KbAa8Al6SY4PLVVMmFjQAjLks+SsiIvgch+dEVcwaaUE9wNkmcxy1gTr2APg3Uo
+U4/PJjgaWKq3px7sYbzrAcNmoMgKmAvYSxl/jIT+VwXUy3DunPz5qxXDBMju/bDu
+z2XBJihBhuXaW7cRWbde9jnhgJgEqOPwBwNh0oV6vd4jNPXMfBLuf5Rj2cu1J+lX
+/6+vXxJ/Q4RN0amA4FpYhZCoTYXTeKp4TnxoB/N75iC8AxzlzSJCj8EnwDcuIA23
+yQKCAQEArxwUQGb5nWDmzUhpbGpMjKVHlcXlEW3L0fJRe2QB3Z3WpXMFoEbdJeDh
+xGxbQSFxRXZc/eJMUg9cQQAFFG7lToMRqXrBByWuF8/iGJFAenY3LqMOPvQ0JJ5S
+bIS7aWbAQkliwNI4KdWa236qWCaTjIjUO5qdeEtw8uqim6ERmzZX+XZDoTmLi+uz
+vVjuh/6+wC++3jZXHnCwZ09CKFzkTUwjsYGHGZDKzUzAbcnJ3tmTzwxU294BJ4Qh
+ztQCav9MYv5Ei/hvE2L/UeuB6QF/WOcVRTfh+x2orLRb6s6A6UY3xNfs1gLPcHjV
+QSLJ/7+Bn9NwsoDIaegA7vbs2BrKqg==
+-----END PRIVATE KEY-----
+)";
+
+  *cert_file = JoinPathSegments(dir, "test.cert");
+  *key_file = JoinPathSegments(dir, "test.key");
+  *expired_ca_cert_file = JoinPathSegments(dir, "testchainca.cert");
+
+  RETURN_NOT_OK(WriteStringToFile(Env::Default(), kCert, *cert_file));
+  RETURN_NOT_OK(WriteStringToFile(Env::Default(), kKey, *key_file));
+  RETURN_NOT_OK(WriteStringToFile(Env::Default(), kCaExpiredCert, *expired_ca_cert_file));
+
+  return Status::OK();
+}
+
 } // namespace security
 } // namespace kudu
diff --git a/src/kudu/security/test/test_certs.h b/src/kudu/security/test/test_certs.h
index 7767cb249..8c571ce34 100644
--- a/src/kudu/security/test/test_certs.h
+++ b/src/kudu/security/test/test_certs.h
@@ -82,5 +82,12 @@ Status CreateTestSSLCertWithChainSignedByRoot(const std::string& dir,
                                               std::string* key_file,
                                               std::string* ca_cert_file);
 
+// Same as the CreateTestSSLCertWithPlainKey() except that the 'ca_cert_file' contains
+// an expired certificate.
+Status CreateTestSSLExpiredCertWithChainSignedByRoot(const std::string& dir,
+                                              std::string* cert_file,
+                                              std::string* key_file,
+                                              std::string* expired_ca_cert_file);
+
 } // namespace security
 } // namespace kudu
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index 51045eee0..baf877b2f 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -274,6 +274,17 @@ DEFINE_string(jwks_url, "",
     "URL of the JSON Web Key Set (JWKS) for JWT verification.");
 TAG_FLAG(jwks_url, experimental);
 
+// Enables retrieving the JWKS URL with verifying the presented TLS certificate
+// from the server.
+DEFINE_bool(jwks_verify_server_certificate, true,
+            "Specifies if the TLS certificate of the JWKS server is verified when retrieving "
+            "the JWKS from the specified JWKS URL. A certificate is considered valid if a "
+            "trust chain can be established for it, and the certificate has a common name or "
+            "SAN that matches the server's hostname. This should only be set to false for "
+            "development / testing.");
+TAG_FLAG(jwks_verify_server_certificate, experimental);
+TAG_FLAG(jwks_verify_server_certificate, unsafe);
+
 DEFINE_string(jwks_discovery_endpoint_base, "",
               "Base URL of the Discovery Endpoint that points to a JSON Web Key Set "
               "(JWKS) for JWT verification. Additional query parameters, like 'accountId', "
@@ -320,6 +331,7 @@ DECLARE_string(log_filename);
 DECLARE_string(keytab_file);
 DECLARE_string(principal);
 DECLARE_string(time_source);
+DECLARE_string(trusted_certificate_file);
 
 METRIC_DECLARE_gauge_size(merged_entities_count_of_server);
 METRIC_DEFINE_gauge_int64(server, uptime,
@@ -800,7 +812,10 @@ Status ServerBase::Init() {
   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);
+      jwt_verifier =
+          std::make_shared<PerAccountKeyBasedJwtVerifier>(FLAGS_jwks_url,
+                                                          FLAGS_jwks_verify_server_certificate,
+                                                          FLAGS_trusted_certificate_file);
     } else if (!FLAGS_jwks_file_path.empty()) {
       jwt_verifier = std::make_shared<KeyBasedJwtVerifier>(FLAGS_jwks_file_path, true);
     } else {
diff --git a/src/kudu/util/jwt-util-internal.h b/src/kudu/util/jwt-util-internal.h
index 38c499d24..01be925ba 100644
--- a/src/kudu/util/jwt-util-internal.h
+++ b/src/kudu/util/jwt-util-internal.h
@@ -257,8 +257,9 @@ class JWKSSnapshot final {
   // Download JWKS JSON file from the given URL, then load the public keys if the
   // checksum of JWKS object is changed. If no keys were given in the URL, the internal
   // maps will be empty.
-  Status LoadKeysFromUrl(
-      const std::string& jwks_url, uint64_t cur_jwks_checksum, bool* is_changed);
+  Status LoadKeysFromUrl(const std::string& jwks_url, bool jwks_verify_server_certificate,
+                         uint64_t cur_jwks_hash, bool* is_changed);
+
 
   // Look up the key ID in the internal key maps and returns the key if the lookup was
   // successful, otherwise return nullptr.
@@ -339,7 +340,8 @@ class JWKSMgr {
   // the internal maps will be empty.
   // If the given jwks_uri is a URL, start a working thread which will periodically
   // checks the JWKS URL for updates. This provides support for key rotation.
-  Status Init(const std::string& jwks_uri, bool is_local_file);
+  Status Init(const std::string& jwks_uri, bool jwks_verify_server_certificate,
+              bool is_local_file);
 
   // Returns a read only snapshot of the current JWKS. This function should be called
   // after calling Init().
@@ -361,6 +363,9 @@ class JWKSMgr {
   // JWKS URI.
   std::string jwks_uri_;
 
+  // JWKS insecure TLS
+  bool jwks_verify_server_certificate_;
+
   // The snapshot of the current JWKS. When the checksum of downloaded JWKS json object
   // has been changed, the public keys will be reloaded and the content of this pointer
   // will be atomically swapped.
diff --git a/src/kudu/util/jwt-util-test.cc b/src/kudu/util/jwt-util-test.cc
index ddc210f18..b7d181cd0 100644
--- a/src/kudu/util/jwt-util-test.cc
+++ b/src/kudu/util/jwt-util-test.cc
@@ -100,7 +100,7 @@ TEST(JwtUtilTest, LoadJwksFile) {
       kRsaPubKeyJwkN, kRsaPubKeyJwkE, kKid2, "RS256", kRsaInvalidPubKeyJwkN,
       kRsaPubKeyJwkE));
   JWTHelper jwt_helper;
-  Status status = jwt_helper.Init(jwks_file.Filename(), true);
+  Status status = jwt_helper.Init(jwks_file.Filename());
   EXPECT_OK(status);
   JWKSSnapshotPtr jwks = jwt_helper.GetJWKS();
   ASSERT_FALSE(jwks->IsEmpty());
@@ -131,7 +131,7 @@ TEST(JwtUtilTest, LoadInvalidJwksFiles) {
       "  ]"
       "}"));
   JWTHelper jwt_helper;
-  Status status = jwt_helper.Init(jwks_file->Filename(), true);
+  Status status = jwt_helper.Init(jwks_file->Filename());
   ASSERT_FALSE(status.ok());
   ASSERT_STR_CONTAINS(status.ToString(), "parsing key #0")
       << " Actual error: " << status.ToString();
@@ -150,7 +150,7 @@ TEST(JwtUtilTest, LoadInvalidJwksFiles) {
       "      \"n\": \"sttddbg-_yjXzcFpbMJB1fIFam9lQBeXWbTqzJwbuFbspHMsRowa8FaPw\","
       "      \"e\": \"AQAB\""
       "}"));
-  status = jwt_helper.Init(jwks_file->Filename(), true);
+  status = jwt_helper.Init(jwks_file->Filename());
   ASSERT_FALSE(status.ok());
   ASSERT_STR_CONTAINS(status.ToString(), "Missing a comma or ']' after an array element")
       << " Actual error: " << status.ToString();
@@ -159,7 +159,7 @@ TEST(JwtUtilTest, LoadInvalidJwksFiles) {
   jwks_file.reset(new TempTestDataFile(
       Substitute(kJwksRsaFileFormat, "", "RS256", kRsaPubKeyJwkN, kRsaPubKeyJwkE,
           "", "RS256", kRsaInvalidPubKeyJwkN, kRsaPubKeyJwkE)));
-  status = jwt_helper.Init(jwks_file->Filename(), true);
+  status = jwt_helper.Init(jwks_file->Filename());
   ASSERT_FALSE(status.ok());
   ASSERT_STR_CONTAINS(status.ToString(), "parsing key #0")
       << " Actual error: " << status.ToString();
@@ -169,7 +169,7 @@ TEST(JwtUtilTest, LoadInvalidJwksFiles) {
   // JWKS with empty key value.
   jwks_file.reset(new TempTestDataFile(
       Substitute(kJwksRsaFileFormat, kKid1, "RS256", "", "", kKid2, "RS256", "", "")));
-  status = jwt_helper.Init(jwks_file->Filename(), true);
+  status = jwt_helper.Init(jwks_file->Filename());
   ASSERT_FALSE(status.ok());
   ASSERT_STR_CONTAINS(status.ToString(), "parsing key #0")
       << " Actual error: " << status.ToString();
@@ -184,7 +184,7 @@ TEST(JwtUtilTest, VerifyJwtHS256) {
   TempTestDataFile jwks_file(
       Substitute(kJwksHsFileFormat, kKid1, "HS256", shared_secret));
   JWTHelper jwt_helper;
-  Status status = jwt_helper.Init(jwks_file.Filename(), true);
+  Status status = jwt_helper.Init(jwks_file.Filename());
   EXPECT_OK(status);
   JWKSSnapshotPtr jwks = jwt_helper.GetJWKS();
   EXPECT_OK(status);
@@ -224,7 +224,7 @@ TEST(JwtUtilTest, VerifyJwtHS384) {
   TempTestDataFile jwks_file(
       Substitute(kJwksHsFileFormat, kKid1, "HS384", shared_secret));
   JWTHelper jwt_helper;
-  Status status = jwt_helper.Init(jwks_file.Filename(), true);
+  Status status = jwt_helper.Init(jwks_file.Filename());
   EXPECT_OK(status);
   JWKSSnapshotPtr jwks = jwt_helper.GetJWKS();
   ASSERT_EQ(1, jwks->GetHSKeyNum());
@@ -263,7 +263,7 @@ TEST(JwtUtilTest, VerifyJwtHS512) {
   TempTestDataFile jwks_file(
       Substitute(kJwksHsFileFormat, kKid1, "HS512", shared_secret));
   JWTHelper jwt_helper;
-  Status status = jwt_helper.Init(jwks_file.Filename(), true);
+  Status status = jwt_helper.Init(jwks_file.Filename());
   EXPECT_OK(status);
   JWKSSnapshotPtr jwks = jwt_helper.GetJWKS();
   EXPECT_OK(status);
@@ -301,7 +301,7 @@ TEST(JwtUtilTest, VerifyJwtRS256) {
       kRsaPubKeyJwkN, kRsaPubKeyJwkE, kKid2, "RS256", kRsaInvalidPubKeyJwkN,
       kRsaPubKeyJwkE));
   JWTHelper jwt_helper;
-  Status status = jwt_helper.Init(jwks_file.Filename(), true);
+  Status status = jwt_helper.Init(jwks_file.Filename());
   EXPECT_OK(status);
   JWKSSnapshotPtr jwks = jwt_helper.GetJWKS();
   ASSERT_EQ(2, jwks->GetRSAPublicKeyNum());
@@ -354,7 +354,7 @@ TEST(JwtUtilTest, VerifyJwtRS384) {
       kRsaPubKeyJwkN, kRsaPubKeyJwkE, kKid2, "RS384", kRsaInvalidPubKeyJwkN,
       kRsaPubKeyJwkE));
   JWTHelper jwt_helper;
-  Status status = jwt_helper.Init(jwks_file.Filename(), true);
+  Status status = jwt_helper.Init(jwks_file.Filename());
   EXPECT_OK(status);
   JWKSSnapshotPtr jwks = jwt_helper.GetJWKS();
   ASSERT_EQ(2, jwks->GetRSAPublicKeyNum());
@@ -392,7 +392,7 @@ TEST(JwtUtilTest, VerifyJwtRS512) {
       kRsa512PubKeyJwkN, kRsa512PubKeyJwkE, kKid2, "RS512",
       kRsa512InvalidPubKeyJwkN, kRsa512PubKeyJwkE));
   JWTHelper jwt_helper;
-  Status status = jwt_helper.Init(jwks_file.Filename(), true);
+  Status status = jwt_helper.Init(jwks_file.Filename());
   EXPECT_OK(status);
   JWKSSnapshotPtr jwks = jwt_helper.GetJWKS();
   ASSERT_EQ(2, jwks->GetRSAPublicKeyNum());
@@ -430,7 +430,7 @@ TEST(JwtUtilTest, VerifyJwtPS256) {
       kRsa1024PubKeyJwkN, kRsa1024PubKeyJwkE, kKid2, "PS256",
       kRsaInvalidPubKeyJwkN, kRsaPubKeyJwkE));
   JWTHelper jwt_helper;
-  Status status = jwt_helper.Init(jwks_file.Filename(), true);
+  Status status = jwt_helper.Init(jwks_file.Filename());
   EXPECT_OK(status);
   JWKSSnapshotPtr jwks = jwt_helper.GetJWKS();
   ASSERT_EQ(2, jwks->GetRSAPublicKeyNum());
@@ -468,7 +468,7 @@ TEST(JwtUtilTest, VerifyJwtPS384) {
       kRsa2048PubKeyJwkN, kRsa2048PubKeyJwkE, kKid2, "PS384",
       kRsaInvalidPubKeyJwkN, kRsaPubKeyJwkE));
   JWTHelper jwt_helper;
-  Status status = jwt_helper.Init(jwks_file.Filename(), true);
+  Status status = jwt_helper.Init(jwks_file.Filename());
   EXPECT_OK(status);
   JWKSSnapshotPtr jwks = jwt_helper.GetJWKS();
   ASSERT_EQ(2, jwks->GetRSAPublicKeyNum());
@@ -506,7 +506,7 @@ TEST(JwtUtilTest, VerifyJwtPS512) {
       kRsa4096PubKeyJwkN, kRsa4096PubKeyJwkE, kKid2, "PS512",
       kRsaInvalidPubKeyJwkN, kRsaPubKeyJwkE));
   JWTHelper jwt_helper;
-  Status status = jwt_helper.Init(jwks_file.Filename(), true);
+  Status status = jwt_helper.Init(jwks_file.Filename());
   EXPECT_OK(status);
   JWKSSnapshotPtr jwks = jwt_helper.GetJWKS();
   ASSERT_EQ(2, jwks->GetRSAPublicKeyNum());
@@ -543,7 +543,7 @@ TEST(JwtUtilTest, VerifyJwtES256) {
   TempTestDataFile jwks_file(Substitute(kJwksEcFileFormat, kKid1, "P-256",
       kEcdsa256PubKeyJwkX, kEcdsa256PubKeyJwkY));
   JWTHelper jwt_helper;
-  Status status = jwt_helper.Init(jwks_file.Filename(), true);
+  Status status = jwt_helper.Init(jwks_file.Filename());
   EXPECT_OK(status);
   JWKSSnapshotPtr jwks = jwt_helper.GetJWKS();
   ASSERT_EQ(1, jwks->GetECPublicKeyNum());
@@ -588,7 +588,7 @@ TEST(JwtUtilTest, VerifyJwtES384) {
   TempTestDataFile jwks_file(Substitute(kJwksEcFileFormat, kKid1, "P-384",
       kEcdsa384PubKeyJwkX, kEcdsa384PubKeyJwkY));
   JWTHelper jwt_helper;
-  Status status = jwt_helper.Init(jwks_file.Filename(), true);
+  Status status = jwt_helper.Init(jwks_file.Filename());
   EXPECT_OK(status);
   JWKSSnapshotPtr jwks = jwt_helper.GetJWKS();
   ASSERT_EQ(1, jwks->GetECPublicKeyNum());
@@ -625,7 +625,7 @@ TEST(JwtUtilTest, VerifyJwtES512) {
   TempTestDataFile jwks_file(Substitute(kJwksEcFileFormat, kKid1, "P-521",
       kEcdsa521PubKeyJwkX, kEcdsa521PubKeyJwkY));
   JWTHelper jwt_helper;
-  Status status = jwt_helper.Init(jwks_file.Filename(), true);
+  Status status = jwt_helper.Init(jwks_file.Filename());
   EXPECT_OK(status);
   JWKSSnapshotPtr jwks = jwt_helper.GetJWKS();
   ASSERT_EQ(1, jwks->GetECPublicKeyNum());
@@ -683,7 +683,7 @@ TEST(JwtUtilTest, VerifyJwtFailMismatchingAlgorithms) {
       kRsaPubKeyJwkN, kRsaPubKeyJwkE, kKid2, "RS256", kRsaInvalidPubKeyJwkN,
       kRsaPubKeyJwkE));
   JWTHelper jwt_helper;
-  Status status = jwt_helper.Init(jwks_file.Filename(), true);
+  Status status = jwt_helper.Init(jwks_file.Filename());
   EXPECT_OK(status);
 
   // Create a JWT token, but set mismatching algorithm.
@@ -712,7 +712,7 @@ TEST(JwtUtilTest, VerifyJwtFailKeyNotFound) {
       kRsaPubKeyJwkN, kRsaPubKeyJwkE, kKid2, "RS256", kRsaInvalidPubKeyJwkN,
       kRsaPubKeyJwkE));
   JWTHelper jwt_helper;
-  Status status = jwt_helper.Init(jwks_file.Filename(), true);
+  Status status = jwt_helper.Init(jwks_file.Filename());
   EXPECT_OK(status);
 
   // Create a JWT token with a key ID which can not be found in JWKS.
@@ -740,7 +740,7 @@ TEST(JwtUtilTest, VerifyJwtTokenWithoutKeyId) {
       kRsaPubKeyJwkN, kRsaPubKeyJwkE, kKid2, "RS256", kRsaInvalidPubKeyJwkN,
       kRsaPubKeyJwkE));
   JWTHelper jwt_helper;
-  Status status = jwt_helper.Init(jwks_file.Filename(), true);
+  Status status = jwt_helper.Init(jwks_file.Filename());
   EXPECT_OK(status);
 
   // Create a JWT token without key ID.
@@ -764,7 +764,7 @@ TEST(JwtUtilTest, VerifyJwtFailTokenWithoutKeyId) {
       kRsaPubKeyJwkN, kRsaPubKeyJwkE, kKid2, "RS256", kRsaInvalidPubKeyJwkN,
       kRsaPubKeyJwkE));
   JWTHelper jwt_helper;
-  Status status = jwt_helper.Init(jwks_file.Filename(), true);
+  Status status = jwt_helper.Init(jwks_file.Filename());
   EXPECT_OK(status);
 
   // Create a JWT token without key ID.
@@ -787,7 +787,7 @@ TEST(JwtUtilTest, VerifyJwtFailTokenWithoutSignature) {
       kRsaPubKeyJwkN, kRsaPubKeyJwkE, kKid2, "RS256", kRsaInvalidPubKeyJwkN,
       kRsaPubKeyJwkE));
   JWTHelper jwt_helper;
-  Status status = jwt_helper.Init(jwks_file.Filename(), true);
+  Status status = jwt_helper.Init(jwks_file.Filename());
   EXPECT_OK(status);
 
   // Create a JWT token without signature.
@@ -809,7 +809,7 @@ TEST(JwtUtilTest, VerifyJwtFailExpiredToken) {
       kRsaPubKeyJwkN, kRsaPubKeyJwkE, kKid2, "RS256", kRsaInvalidPubKeyJwkN,
       kRsaPubKeyJwkE));
   JWTHelper jwt_helper;
-  Status status = jwt_helper.Init(jwks_file.Filename(), true);
+  Status status = jwt_helper.Init(jwks_file.Filename());
   EXPECT_OK(status);
 
   // Create a JWT token and sign it with RS256.
@@ -947,7 +947,9 @@ TEST(JwtUtilTest, VerifyOIDCDiscoveryEndpoint) {
   };
   MiniOidc oidc(std::move(opts));
   ASSERT_OK(oidc.Start());
-  const PerAccountKeyBasedJwtVerifier jwt_verifier(oidc.url());
+  const PerAccountKeyBasedJwtVerifier jwt_verifier(oidc.url(),
+                                                   /*jwks_verify_server_certificate*/ false,
+                                                   /*jwks_ca_certificate*/ "");
 
   // Create and verify a token on the happy path.
   const string kSubject = "kudu";
@@ -980,7 +982,9 @@ TEST(JwtUtilTest, VerifyJWKSDiscoveryEndpointMultipleClients) {
   };
   MiniOidc oidc(std::move(opts));
   ASSERT_OK(oidc.Start());
-  PerAccountKeyBasedJwtVerifier jwt_verifier(oidc.url());
+  PerAccountKeyBasedJwtVerifier jwt_verifier(oidc.url(),
+                                             /*jwks_verify_server_certificate*/ false,
+                                             /*jwks_ca_certificate*/ "");
 
   {
     const string kSubject = "kudu";
diff --git a/src/kudu/util/jwt-util.cc b/src/kudu/util/jwt-util.cc
index 6ecd51cd5..5fd7c582b 100644
--- a/src/kudu/util/jwt-util.cc
+++ b/src/kudu/util/jwt-util.cc
@@ -602,14 +602,16 @@ Status JWKSSnapshot::LoadKeysFromFile(const string& jwks_file_path) {
 
 // Download JWKS from the given URL with Kudu's EasyCurl wrapper.
 Status JWKSSnapshot::LoadKeysFromUrl(
-    const std::string& jwks_url, uint64_t cur_jwks_checksum, bool* is_changed) {
+    const std::string& jwks_url, bool jwks_verify_server_certificate, uint64_t cur_jwks_checksum,
+    bool* is_changed) {
   kudu::EasyCurl curl;
   kudu::faststring dst;
   *is_changed = false;
 
   curl.set_timeout(
       kudu::MonoDelta::FromMilliseconds(static_cast<int64_t>(FLAGS_jwks_pulling_timeout_s) * 1000));
-  curl.set_verify_peer(false);
+  curl.set_verify_peer(jwks_verify_server_certificate);
+
   // TODO support CurlAuthType by calling kudu::EasyCurl::set_auth().
   RETURN_NOT_OK_PREPEND(curl.FetchURL(jwks_url, &dst),
       Substitute("Error downloading JWKS from '$0'", jwks_url));
@@ -694,15 +696,19 @@ JWKSMgr::~JWKSMgr() {
   if (jwks_update_thread_ != nullptr) jwks_update_thread_->Join();
 }
 
-Status JWKSMgr::Init(const std::string& jwks_uri, bool is_local_file) {
+Status JWKSMgr::Init(const std::string& jwks_uri, bool jwks_verify_server_certificate,
+                     bool is_local_file) {
   jwks_uri_ = jwks_uri;
+  jwks_verify_server_certificate_ = jwks_verify_server_certificate;
   std::shared_ptr<JWKSSnapshot> new_jwks = std::make_shared<JWKSSnapshot>();
   if (is_local_file) {
     RETURN_NOT_OK_PREPEND(new_jwks->LoadKeysFromFile(jwks_uri), "Failed to load JWKS");
     SetJWKSSnapshot(new_jwks);
   } else {
     bool is_changed = false;
-    RETURN_NOT_OK_PREPEND(new_jwks->LoadKeysFromUrl(jwks_uri, current_jwks_checksum_, &is_changed),
+    RETURN_NOT_OK_PREPEND(new_jwks->LoadKeysFromUrl(jwks_uri, jwks_verify_server_certificate,
+                                                    current_jwks_checksum_,
+                                                    &is_changed),
                           "Failed to load JWKS");
     DCHECK(is_changed);
     if (is_changed) SetJWKSSnapshot(new_jwks);
@@ -734,7 +740,8 @@ void JWKSMgr::UpdateJWKSThread() {
     new_jwks = std::make_shared<JWKSSnapshot>();
     bool is_changed = false;
     Status status =
-        new_jwks->LoadKeysFromUrl(jwks_uri_, current_jwks_checksum_, &is_changed);
+        new_jwks->LoadKeysFromUrl(jwks_uri_, jwks_verify_server_certificate_,
+                                  current_jwks_checksum_, &is_changed);
     if (!status.ok()) {
       LOG(WARNING) << "Failed to update JWKS: " << status.ToString();
     } else if (is_changed) {
@@ -781,9 +788,17 @@ void JWTHelper::TokenDeleter::operator()(JWTHelper::JWTDecodedToken* token) cons
   delete token;
 }
 
-Status JWTHelper::Init(const std::string& jwks_uri, bool is_local_file) {
+Status JWTHelper::Init(const std::string& jwks_file_path) {
+  return Init(jwks_file_path,
+              /*jwks_verify_server_certificate*/ false,
+              /*is_local_file*/ true);
+}
+
+Status JWTHelper::Init(const std::string& jwks_uri, bool jwks_verify_server_certificate,
+                       bool is_local_file) {
   jwks_mgr_.reset(new JWKSMgr());
-  RETURN_NOT_OK(jwks_mgr_->Init(jwks_uri, is_local_file));
+  RETURN_NOT_OK(jwks_mgr_->Init(jwks_uri, jwks_verify_server_certificate,
+                                is_local_file));
   if (!initialized_) initialized_ = true;
   return Status::OK();
 }
@@ -931,7 +946,7 @@ Status JWTHelper::GetCustomClaimUsername(const JWTDecodedToken* decoded_token,
 }
 
 Status KeyBasedJwtVerifier::Init() {
-  return jwt_->Init(jwks_uri_, is_local_file_);
+  return jwt_->Init(jwks_uri_, /*jwks_verify_server_certificate*/ false, is_local_file_);
 }
 
 Status KeyBasedJwtVerifier::VerifyToken(const string& bytes_raw, string* subject) const {
@@ -1004,7 +1019,9 @@ Status PerAccountKeyBasedJwtVerifier::JWTHelperForToken(const JWTHelper::JWTDeco
   // accounts, as it creates a JWKS refresh thread for each account. Group the
   // refreshes into a single thread or threadpool.
   auto new_helper = std::make_shared<JWTHelper>();
-  RETURN_NOT_OK_PREPEND(new_helper->Init(jwks_uri, /*is_local_file*/ false),
+  RETURN_NOT_OK_PREPEND(new_helper->Init(jwks_uri,
+                                         jwks_verify_server_certificate_,
+                                         /*is_local_file*/ false),
                         "Error initializing JWT helper");
 
   {
@@ -1019,7 +1036,8 @@ Status PerAccountKeyBasedJwtVerifier::JWTHelperForToken(const JWTHelper::JWTDeco
 Status PerAccountKeyBasedJwtVerifier::Init() {
   for (auto& [account_id, verifier] : jwt_by_account_id_) {
     RETURN_NOT_OK(verifier->Init(Substitute("$0?accountId=$1", oidc_uri_, account_id),
-                   /*is_local_file*/false));
+                                            jwks_verify_server_certificate_,
+                                            /*is_local_file*/ false));
   }
   return Status::OK();
 }
diff --git a/src/kudu/util/jwt-util.h b/src/kudu/util/jwt-util.h
index e25bbd8d4..daba84eb1 100644
--- a/src/kudu/util/jwt-util.h
+++ b/src/kudu/util/jwt-util.h
@@ -64,7 +64,12 @@ class JWTHelper {
 
   // Load JWKS from a given local JSON file or URL. Returns an error if problems were
   // encountered.
-  Status Init(const std::string& jwks_uri, bool is_local_file);
+  Status Init(const std::string& jwks_uri);
+
+  // Load JWKS from a given local JSON file or URL. Returns an error if problems were
+  // encountered.
+  Status Init(const std::string& jwks_uri, bool jwks_verify_server_certificate,
+              bool is_local_file);
 
   // Decode the given JWT token. The decoding result is stored in decoded_token_out.
   // Return Status::OK if the decoding is successful.
@@ -117,8 +122,11 @@ class KeyBasedJwtVerifier : public JwtVerifier {
 
 class PerAccountKeyBasedJwtVerifier : public JwtVerifier {
  public:
-  explicit PerAccountKeyBasedJwtVerifier(std::string oidc_uri)
-      : oidc_uri_(std::move(oidc_uri)) {}
+  explicit PerAccountKeyBasedJwtVerifier(std::string oidc_uri, bool jwks_verify_server_certificate,
+                                         const std::string jwks_ca_certificate)
+      : oidc_uri_(std::move(oidc_uri)),
+        jwks_verify_server_certificate_(jwks_verify_server_certificate),
+        jwks_ca_certificate_(jwks_ca_certificate) {}
 
   ~PerAccountKeyBasedJwtVerifier() override = default;
 
@@ -132,6 +140,11 @@ class PerAccountKeyBasedJwtVerifier : public JwtVerifier {
   Status JWTHelperForToken(const JWTHelper::JWTDecodedToken& token, JWTHelper** helper) const;
 
   const std::string oidc_uri_;
+
+  const bool jwks_verify_server_certificate_;
+
+  const std::string jwks_ca_certificate_;
+
   // 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
index 1672f33a5..a6296d3a5 100644
--- a/src/kudu/util/mini_oidc.cc
+++ b/src/kudu/util/mini_oidc.cc
@@ -23,6 +23,7 @@
 #include <utility>
 #include <vector>
 
+#include <glog/logging.h>
 #include <jwt-cpp/jwt.h>
 #include <jwt-cpp/traits/kazuho-picojson/defaults.h>
 #include <jwt-cpp/traits/kazuho-picojson/traits.h>
@@ -100,6 +101,10 @@ Status MiniOidc::Start() {
   // we've been configured to server.
   WebserverOptions jwks_opts;
   jwks_opts.port = 0;
+  jwks_opts.bind_interface = "localhost";
+  jwks_opts.certificate_file = options_.server_certificate;
+  jwks_opts.private_key_file = options_.private_key_file;
+
   jwks_server_.reset(new Webserver(jwks_opts));
 
   for (const auto& [account_id, valid] : options_.account_ids) {
@@ -121,16 +126,26 @@ Status MiniOidc::Start() {
         /*is_styled*/ false,
         /*is_on_nav_bar*/ false);
   }
+  LOG(INFO) << "Starting JWKS server";
   RETURN_NOT_OK(jwks_server_->Start());
-  vector<Sockaddr> bound_addrs;
+  vector<Sockaddr> advertised_addrs;
   Sockaddr addr;
-  RETURN_NOT_OK(jwks_server_->GetBoundAddresses(&bound_addrs));
-  RETURN_NOT_OK(addr.ParseString(bound_addrs[0].host(), bound_addrs[0].port()));
-  const string jwks_url = Substitute("http://$0/jwks", addr.ToString());
+  RETURN_NOT_OK(jwks_server_->GetAdvertisedAddresses(&advertised_addrs));
+  // calling ParseString() to verify the address components
+  RETURN_NOT_OK(addr.ParseString(advertised_addrs[0].host(), advertised_addrs[0].port()));
+  string protocol = "https";
+  if (jwks_opts.certificate_file.empty() && jwks_opts.password_file.empty()) {
+    protocol = "http";
+  }
+
+  const string jwks_url = Substitute("$0://localhost:$1/jwks",
+                                     protocol,
+                                     advertised_addrs[0].port());
 
   // Now start the OIDC Discovery server that points to the JWKS endpoints.
   WebserverOptions oidc_opts;
   oidc_opts.port = 0;
+  oidc_opts.bind_interface = "localhost";
   oidc_server_.reset(new Webserver(oidc_opts));
   oidc_server_->RegisterPrerenderedPathHandler(
       "/.well-known/openid-configuration",
@@ -142,10 +157,13 @@ Status MiniOidc::Start() {
       },
       /*is_styled*/ false,
       /*is_on_nav_bar*/ false);
+
+  LOG(INFO) << "Starting OIDC Discovery server";
   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()));
+  advertised_addrs.clear();
+  RETURN_NOT_OK(oidc_server_->GetAdvertisedAddresses(&advertised_addrs));
+  // calling ParseString() to verify the address components
+  RETURN_NOT_OK(addr.ParseString(advertised_addrs[0].host(), advertised_addrs[0].port()));
   oidc_url_ = Substitute("http://$0/.well-known/openid-configuration", addr.ToString());
   return Status::OK();
 }
diff --git a/src/kudu/util/mini_oidc.h b/src/kudu/util/mini_oidc.h
index 74dfbfefd..b4eff0d0e 100644
--- a/src/kudu/util/mini_oidc.h
+++ b/src/kudu/util/mini_oidc.h
@@ -39,6 +39,13 @@ struct MiniOidcOptions {
 
   // Maps account IDs to add to whether or not to create JWKS with invalid keys.
   std::unordered_map<std::string, bool> account_ids;
+
+  // String that contains the server_certificate that is used to establish secure
+  // https connection to the JWKS server.
+  std::string server_certificate;
+
+  // The private key belonging to the server certificate
+  std::string private_key_file;
 };
 
 // Serves the following endpoints for testing a cluster: