You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by yc...@apache.org on 2022/05/12 18:41:06 UTC

[hive] branch master updated: HIVE-26071: JWT authentication mechanism for Thrift over HTTP in HiveMetastore (#3233) (Sourabh Goyal, reviewed by Yu-wen, Deng and Sai)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new d30db8cbaf HIVE-26071: JWT authentication mechanism for Thrift over HTTP in HiveMetastore (#3233) (Sourabh Goyal, reviewed by Yu-wen, Deng and Sai)
d30db8cbaf is described below

commit d30db8cbafba110f6519354df7504b36643a8e60
Author: Sourabh Goyal <so...@cloudera.com>
AuthorDate: Thu May 12 11:40:56 2022 -0700

    HIVE-26071: JWT authentication mechanism for Thrift over HTTP in HiveMetastore (#3233) (Sourabh Goyal, reviewed by Yu-wen, Deng and Sai)
    
    What changes were proposed in this pull request?
    This PR is a follow up of #3105. It adds a support for JWT authentication in HiveMetastore server when run in HTTP transport mode.
    
    Why are the changes needed?
    It supports a new authentication mechanism ie JWT in HiveMetastore server.
    
    Does this PR introduce any user-facing change?
    No
    
    How was this patch tested?
    Added new unit tests that cover cases like
    
    successfully authenticating valid JWT
    failing to authenticate expired, invalid JWTs
    
    * Add JWTValidator and URLBasedJWKSProvider code from HS2
    
    Change-Id: I969f57daf640adb16f228e95b1b522f8ffc24ffe
    
    * Add JWT authentication in HiveMetastore
    
    Change-Id: I6d84517a1ee97df492ad3816ec866c0b785ed5ed
    
    * Better error handling for authentication failures. Added integration tests for validating JWT
    
    Change-Id: I6b9da531db4e4a805d8daa1ba6d941c5643bf514
    
    * Added test JWTs for jwt authentication tests
    
    Change-Id: Ice36a703d8af7d4dbf28a48c9bb96127100fd8c7
    
    * moved jwt test keys under jwt directory
    
    Change-Id: I8bf0b4bbc101a0acb3f69bb1963b9c4bcda5b719
    
    * Fixes failures in metastore jwt unit tests
    
    Change-Id: I2877730a34dff7d3184b100ec04031032611838a
    
    * Addresses review comments
    
    Change-Id: I8498e85212476c663cf735211848a28baaa3bad5
    
    * Addresses nits from review comments
    
    Change-Id: Id67588c106104732a0f6e49e5c983cb5f7287c3e
    
    * Added more comments in the code
    
    Change-Id: Ia51f490362985d109778a6a0aa92a281436d5d21
    
    * removes unsed import statement
    
    Change-Id: I94633bdce0db87a9085968dde79d8ff6cd9bf4a3
---
 .../hadoop/hive/metastore/HiveMetaStoreClient.java |  57 +++--
 .../hadoop/hive/metastore/conf/MetastoreConf.java  |  15 +-
 standalone-metastore/metastore-server/pom.xml      |  16 ++
 .../hadoop/hive/metastore/HiveMetaStore.java       |   2 +-
 .../hive/metastore/HmsThriftHttpServlet.java       | 156 ++++++++----
 .../auth/HttpAuthenticationException.java          |  47 ++++
 .../hive/metastore/auth/jwt/JWTValidator.java      | 109 ++++++++
 .../metastore/auth/jwt/URLBasedJWKSProvider.java   |  88 +++++++
 .../TestRemoteHiveMetastoreWithHttpJwt.java        | 283 +++++++++++++++++++++
 .../resources/auth/jwt/jwt-authorized-key.json     |  12 +
 .../resources/auth/jwt/jwt-unauthorized-key.json   |  12 +
 .../resources/auth/jwt/jwt-verification-jwks.json  |  20 ++
 standalone-metastore/pom.xml                       |   3 +
 13 files changed, 746 insertions(+), 74 deletions(-)

diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 651546caa5..db8477d89b 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -80,6 +80,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.http.HttpException;
+import org.apache.http.HttpHeaders;
 import org.apache.http.HttpRequest;
 import org.apache.http.HttpRequestInterceptor;
 import org.apache.http.impl.client.HttpClientBuilder;
@@ -601,32 +602,52 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     client.rename_partition_req(req);
   }
 
+  /*
+  Creates a THttpClient if HTTP mode is enabled. If Client auth mode is set to JWT,
+  then the method fetches JWT from environment variable: HMS_JWT and sets in auth
+  header in http request
+   */
   private THttpClient createHttpClient(URI store, boolean useSSL) throws MetaException,
       TTransportException {
     String path = MetaStoreUtils.getHttpPath(MetastoreConf.getVar(conf, ConfVars.THRIFT_HTTP_PATH));
     String httpUrl = (useSSL ? "https://" : "http://") + store.getHost() + ":" + store.getPort() + path;
 
-    String user = MetastoreConf.getVar(conf, ConfVars.METASTORE_CLIENT_PLAIN_USERNAME);
-    if (user == null || user.equals("")) {
-      try {
-        LOG.debug("No username passed in config " + ConfVars.METASTORE_CLIENT_PLAIN_USERNAME.getHiveName() +
-            ". Trying to get the current user from UGI" );
-        user = UserGroupInformation.getCurrentUser().getShortUserName();
-      } catch (IOException e) {
-        throw new MetaException("Failed to get client username from UGI");
+    HttpClientBuilder httpClientBuilder = HttpClientBuilder.create();
+    String authType = MetastoreConf.getVar(conf, ConfVars.METASTORE_CLIENT_AUTH_MODE);
+    if (authType.equalsIgnoreCase("jwt")) {
+      // fetch JWT token from environment and set it in Auth Header in HTTP request
+      String jwtToken = System.getenv("HMS_JWT");
+      if (jwtToken == null || jwtToken.isEmpty()) {
+        LOG.debug("No jwt token set in environment variable: HMS_JWT");
+        throw new MetaException("For auth mode JWT, valid signed jwt token must be provided in the "
+            + "environment variable HMS_JWT");
       }
+      httpClientBuilder.addInterceptorFirst(new HttpRequestInterceptor() {
+        @Override
+        public void process(HttpRequest httpRequest, HttpContext httpContext)
+            throws HttpException, IOException {
+          httpRequest.addHeader(HttpHeaders.AUTHORIZATION, "Bearer " + jwtToken);
+        }
+      });
+    } else {
+      String user = MetastoreConf.getVar(conf, ConfVars.METASTORE_CLIENT_PLAIN_USERNAME);
+      if (user == null || user.equals("")) {
+        try {
+          user = UserGroupInformation.getCurrentUser().getShortUserName();
+        } catch (IOException e) {
+          throw new MetaException("Failed to get client username from UGI");
+        }
+      }
+      final String httpUser = user;
+      httpClientBuilder.addInterceptorFirst(new HttpRequestInterceptor() {
+        @Override
+        public void process(HttpRequest httpRequest, HttpContext httpContext)
+            throws HttpException, IOException {
+          httpRequest.addHeader(MetaStoreUtils.USER_NAME_HTTP_HEADER, httpUser);
+        }
+      });
     }
-    final String httpUser = user;
     THttpClient tHttpClient;
-    HttpClientBuilder httpClientBuilder = HttpClientBuilder.create();
-    httpClientBuilder.addInterceptorFirst(new HttpRequestInterceptor() {
-      @Override
-      public void process(HttpRequest httpRequest, HttpContext httpContext)
-          throws HttpException, IOException {
-        httpRequest.addHeader(MetaStoreUtils.USER_NAME_HTTP_HEADER, httpUser);
-      }
-    });
-
     try {
       if (useSSL) {
         String trustStorePath = MetastoreConf.getVar(conf, ConfVars.SSL_TRUSTSTORE_PATH).trim();
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
index 04f94212a8..850d675e30 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
@@ -875,7 +875,7 @@ public class MetastoreConf {
             "The special string _HOST will be replaced automatically with the correct host name."),
     THRIFT_METASTORE_AUTHENTICATION("metastore.authentication", "hive.metastore.authentication",
             "NOSASL",
-      new StringSetValidator("NOSASL", "NONE", "LDAP", "KERBEROS", "CUSTOM"),
+      new StringSetValidator("NOSASL", "NONE", "LDAP", "KERBEROS", "CUSTOM", "JWT"),
         "Client authentication types.\n" +
                 "  NONE: no authentication check\n" +
                 "  LDAP: LDAP/AD based authentication\n" +
@@ -883,7 +883,12 @@ public class MetastoreConf {
                 "  CUSTOM: Custom authentication provider\n" +
                 "          (Use with property metastore.custom.authentication.class)\n" +
                 "  CONFIG: username and password is specified in the config" +
-                "  NOSASL:  Raw transport"),
+                "  NOSASL:  Raw transport" +
+                "  JWT:  JSON Web Token authentication via JWT token. Only supported in Http/Https mode"),
+    THRIFT_METASTORE_AUTHENTICATION_JWT_JWKS_URL("metastore.authentication.jwt.jwks.url",
+        "hive.metastore.authentication.jwt.jwks.url", "", "File URL from where URLBasedJWKSProvider "
+        + "in metastore server will try to load JWKS to match a JWT sent in HTTP request header. Used only when "
+        + "Hive metastore server is running in JWT auth mode"),
     METASTORE_CUSTOM_AUTHENTICATION_CLASS("metastore.custom.authentication.class",
             "hive.metastore.custom.authentication.class",
             "",
@@ -1544,9 +1549,11 @@ public class MetastoreConf {
         "If true, the metastore Thrift interface will be secured with SASL. Clients must authenticate with Kerberos."),
     METASTORE_CLIENT_AUTH_MODE("metastore.client.auth.mode",
             "hive.metastore.client.auth.mode", "NOSASL",
-            new StringSetValidator("NOSASL", "PLAIN", "KERBEROS"),
+            new StringSetValidator("NOSASL", "PLAIN", "KERBEROS", "JWT"),
             "If PLAIN, clients will authenticate using plain authentication, by providing username" +
-                    " and password. Any other value is ignored right now but may be used later."),
+                    " and password. Any other value is ignored right now but may be used later."
+                + "If JWT- Supported only in HTTP transport mode. If set, HMS Client will pick the value of JWT from "
+                + "environment variable HMS_JWT and set it in Authorization header in http request"),
     METASTORE_CLIENT_PLAIN_USERNAME("metastore.client.plain.username",
             "hive.metastore.client.plain.username",  "",
         "The username used by the metastore client when " +
diff --git a/standalone-metastore/metastore-server/pom.xml b/standalone-metastore/metastore-server/pom.xml
index a50e9bc416..8ef5116694 100644
--- a/standalone-metastore/metastore-server/pom.xml
+++ b/standalone-metastore/metastore-server/pom.xml
@@ -311,6 +311,22 @@
       <artifactId>curator-test</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>com.nimbusds</groupId>
+      <artifactId>nimbus-jose-jwt</artifactId>
+      <version>${nimbus-jose-jwt.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.pac4j</groupId>
+      <artifactId>pac4j-core</artifactId>
+      <version>${pac4j-core.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.github.tomakehurst</groupId>
+      <artifactId>wiremock-jre8-standalone</artifactId>
+      <version>2.32.0</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
   <profiles>
     <profile>
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index cf0b1ec8c1..d2a5837daa 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -477,7 +477,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     IHMSHandler handler = newRetryingHMSHandler(baseHandler, conf);
     processor = new ThriftHiveMetastore.Processor<>(handler);
     LOG.info("Starting DB backed MetaStore Server with generic processor");
-    TServlet thriftHttpServlet = new HmsThriftHttpServlet(processor, protocolFactory);
+    TServlet thriftHttpServlet = new HmsThriftHttpServlet(processor, protocolFactory, conf);
 
     boolean directSqlEnabled = MetastoreConf.getBoolVar(conf, ConfVars.TRY_DIRECT_SQL);
     HMSHandler.LOG.info("Direct SQL optimization = {}",  directSqlEnabled);
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HmsThriftHttpServlet.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HmsThriftHttpServlet.java
index e58bd5634b..5a1a8d0aba 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HmsThriftHttpServlet.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HmsThriftHttpServlet.java
@@ -17,14 +17,23 @@
 
 package org.apache.hadoop.hive.metastore;
 
+import com.google.common.base.Preconditions;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Enumeration;
-
+import java.util.Optional;
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.auth.HttpAuthenticationException;
+import org.apache.hadoop.hive.metastore.auth.jwt.JWTValidator;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.pac4j.core.context.JEEContext;
+import org.pac4j.core.credentials.TokenCredentials;
+import org.pac4j.core.credentials.extractor.BearerAuthExtractor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -33,81 +42,126 @@ import org.apache.thrift.TProcessor;
 import org.apache.thrift.protocol.TProtocolFactory;
 import org.apache.thrift.server.TServlet;
 
+/*
+Servlet class used by HiveMetastore server when running in HTTP mode.
+If JWT auth is enabled, then the servlet is also responsible for validating
+JWTs sent in the Authorization header in HTTP request.
+ */
 public class HmsThriftHttpServlet extends TServlet {
-
   private static final Logger LOG = LoggerFactory
       .getLogger(HmsThriftHttpServlet.class);
-
   private static final String X_USER = MetaStoreUtils.USER_NAME_HTTP_HEADER;
-
   private final boolean isSecurityEnabled;
+  private final boolean jwtAuthEnabled;
+  private JWTValidator jwtValidator = null;
+  private Configuration conf;
 
   public HmsThriftHttpServlet(TProcessor processor,
-      TProtocolFactory inProtocolFactory, TProtocolFactory outProtocolFactory) {
-    super(processor, inProtocolFactory, outProtocolFactory);
-    // This should ideally be reveiving an instance of the Configuration which is used for the check
+      TProtocolFactory protocolFactory, Configuration conf) {
+    super(processor, protocolFactory);
+    this.conf = conf;
     isSecurityEnabled = UserGroupInformation.isSecurityEnabled();
+    if (MetastoreConf.getVar(conf,
+        ConfVars.THRIFT_METASTORE_AUTHENTICATION).equalsIgnoreCase("jwt")) {
+      jwtAuthEnabled = true;
+    } else {
+      jwtAuthEnabled = false;
+    }
   }
 
-  public HmsThriftHttpServlet(TProcessor processor,
-      TProtocolFactory protocolFactory) {
-    super(processor, protocolFactory);
-    isSecurityEnabled = UserGroupInformation.isSecurityEnabled();
+  public void init() throws ServletException {
+    super.init();
+    if (jwtAuthEnabled) {
+      try {
+        jwtValidator = new JWTValidator(this.conf);
+      } catch (Exception e) {
+        throw new ServletException("Failed to initialize HmsThriftHttpServlet."
+            + " Error: " + e);
+      }
+    }
   }
 
   @Override
   protected void doPost(HttpServletRequest request,
       HttpServletResponse response) throws ServletException, IOException {
-
-    Enumeration<String> headerNames = request.getHeaderNames();
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Logging headers in request");
+      LOG.debug("Logging headers in doPost request");
+      Enumeration<String> headerNames = request.getHeaderNames();
       while (headerNames.hasMoreElements()) {
         String headerName = headerNames.nextElement();
         LOG.debug("Header: [{}], Value: [{}]", headerName,
             request.getHeader(headerName));
       }
     }
-    String userFromHeader = request.getHeader(X_USER);
-    if (userFromHeader == null || userFromHeader.isEmpty()) {
-      LOG.error("No user header: {} found", X_USER);
-      response.sendError(HttpServletResponse.SC_FORBIDDEN,
-          "Header: " + X_USER + " missing in the request");
-      return;
-    }
-
-    // TODO: These should ideally be in some kind of a Cache with Weak referencse.
-    // If HMS were to set up some kind of a session, this would go into the session by having
-    // this filter work with a custom Processor / or set the username into the session
-    // as is done for HS2.
-    // In case of HMS, it looks like each request is independent, and there is no session
-    // information, so the UGI needs to be set up in the Connection layer itself.
-    UserGroupInformation clientUgi;
-    // Temporary, and useless for now. Here only to allow this to work on an otherwise kerberized
-    // server.
-    if (isSecurityEnabled) {
-      LOG.info("Creating proxy user for: {}", userFromHeader);
-      clientUgi = UserGroupInformation.createProxyUser(userFromHeader, UserGroupInformation.getLoginUser());
-    } else {
-      LOG.info("Creating remote user for: {}", userFromHeader);
-      clientUgi = UserGroupInformation.createRemoteUser(userFromHeader);
+    try {
+      String userFromHeader = extractUserName(request, response);
+      UserGroupInformation clientUgi;
+      // Temporary, and useless for now. Here only to allow this to work on an otherwise kerberized
+      // server.
+      if (isSecurityEnabled) {
+        LOG.info("Creating proxy user for: {}", userFromHeader);
+        clientUgi = UserGroupInformation.createProxyUser(userFromHeader, UserGroupInformation.getLoginUser());
+      } else {
+        LOG.info("Creating remote user for: {}", userFromHeader);
+        clientUgi = UserGroupInformation.createRemoteUser(userFromHeader);
+      }
+      PrivilegedExceptionAction<Void> action = new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          HmsThriftHttpServlet.super.doPost(request, response);
+          return null;
+        }
+      };
+      try {
+        clientUgi.doAs(action);
+      } catch (InterruptedException | RuntimeException e) {
+        LOG.error("Exception when executing http request as user: " + clientUgi.getUserName(),
+            e);
+        throw new ServletException(e);
+      }
+    } catch (HttpAuthenticationException e) {
+      response.setStatus(HttpServletResponse.SC_UNAUTHORIZED);
+      response.getWriter().println("Authentication error: " + e.getMessage());
+      // Also log the error message on server side
+      LOG.error("Authentication error: ", e);
     }
-
-
-    PrivilegedExceptionAction<Void> action = new PrivilegedExceptionAction<Void>() {
-      @Override
-      public Void run() throws Exception {
-        HmsThriftHttpServlet.super.doPost(request, response);
-        return null;
+  }
+  private String extractUserName(HttpServletRequest request, HttpServletResponse response)
+      throws HttpAuthenticationException {
+    if (!jwtAuthEnabled) {
+      String userFromHeader = request.getHeader(X_USER);
+      if (userFromHeader == null || userFromHeader.isEmpty()) {
+        throw new HttpAuthenticationException("User header " + X_USER + " missing in request");
       }
-    };
-
+      return userFromHeader;
+    }
+    String signedJwt = extractBearerToken(request, response);
+    if (signedJwt == null) {
+      throw new HttpAuthenticationException("Couldn't find bearer token in the auth header in the request");
+    }
+    String user;
     try {
-      clientUgi.doAs(action);
-    } catch (InterruptedException | RuntimeException e) {
-      LOG.error("Exception when executing http request as user: " + clientUgi.getUserName(),
-          e);
-      throw new ServletException(e);
+      user = jwtValidator.validateJWTAndExtractUser(signedJwt);
+      Preconditions.checkNotNull(user, "JWT needs to contain the user name as subject");
+      Preconditions.checkState(!user.isEmpty(), "User name should not be empty in JWT");
+      LOG.info("Successfully validated and extracted user name {} from JWT in Auth "
+          + "header in the request", user);
+    } catch (Exception e) {
+      throw new HttpAuthenticationException("Failed to validate JWT from Bearer token in "
+          + "Authentication header", e);
     }
+    return user;
+  }
+
+  /**
+   * Extracts the bearer authorization header from the request. If there is no bearer
+   * authorization token, returns null.
+   */
+  private String extractBearerToken(HttpServletRequest request,
+      HttpServletResponse response) {
+    BearerAuthExtractor extractor = new BearerAuthExtractor();
+    Optional<TokenCredentials> tokenCredentials = extractor.extract(new JEEContext(
+        request, response));
+    return tokenCredentials.map(TokenCredentials::getToken).orElse(null);
   }
 }
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/auth/HttpAuthenticationException.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/auth/HttpAuthenticationException.java
new file mode 100644
index 0000000000..dc0cc7c66f
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/auth/HttpAuthenticationException.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed 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. See accompanying LICENSE file.
+ */
+
+package org.apache.hadoop.hive.metastore.auth;
+
+/*
+Encapsulates any exceptions thrown by HiveMetastore server
+when authenticating http requests
+ */
+public class HttpAuthenticationException extends Exception {
+
+  private static final long serialVersionUID = 0;
+
+  /**
+   * @param cause original exception
+   */
+  public HttpAuthenticationException(Throwable cause) {
+    super(cause);
+  }
+
+  /**
+   * @param msg exception message
+   */
+  public HttpAuthenticationException(String msg) {
+    super(msg);
+  }
+
+  /**
+   * @param msg   exception message
+   * @param cause original exception
+   */
+  public HttpAuthenticationException(String msg, Throwable cause) {
+    super(msg, cause);
+  }
+
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/auth/jwt/JWTValidator.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/auth/jwt/JWTValidator.java
new file mode 100644
index 0000000000..d95427c95e
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/auth/jwt/JWTValidator.java
@@ -0,0 +1,109 @@
+/*
+ * 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.
+ */
+
+package org.apache.hadoop.hive.metastore.auth.jwt;
+
+import com.google.common.base.Preconditions;
+import com.nimbusds.jose.JOSEException;
+import com.nimbusds.jose.JWSHeader;
+import com.nimbusds.jose.JWSObject;
+import com.nimbusds.jose.JWSVerifier;
+import com.nimbusds.jose.crypto.factories.DefaultJWSVerifierFactory;
+import com.nimbusds.jose.jwk.AsymmetricJWK;
+import com.nimbusds.jose.jwk.JWK;
+import com.nimbusds.jwt.JWTClaimsSet;
+import com.nimbusds.jwt.SignedJWT;
+import javax.security.sasl.AuthenticationException;
+import org.apache.hadoop.conf.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.security.Key;
+import java.text.ParseException;
+import java.util.Date;
+import java.util.List;
+
+/**
+ * This class is used to validate JWT. JWKS is fetched during instantiation and kept in the memory.
+ * We disallow JWT signature verification with symmetric key, because that means anyone can get the same key
+ * and use it to sign a JWT.
+ * This is cloned from JWTValidator in HS2 so as to NOT have any dependency on HS2 code.
+ */
+public class JWTValidator {
+  private static final Logger LOG = LoggerFactory.getLogger(JWTValidator.class.getName());
+  private static final DefaultJWSVerifierFactory JWS_VERIFIER_FACTORY = new DefaultJWSVerifierFactory();
+  private final URLBasedJWKSProvider jwksProvider;
+  public JWTValidator(Configuration conf) throws IOException, ParseException {
+    this.jwksProvider = new URLBasedJWKSProvider(conf);
+  }
+
+  public String validateJWTAndExtractUser(String signedJwt) throws ParseException, AuthenticationException {
+    Preconditions.checkNotNull(jwksProvider);
+    Preconditions.checkNotNull(signedJwt, "No token found");
+    final SignedJWT parsedJwt = SignedJWT.parse(signedJwt);
+    List<JWK> matchedJWKS = jwksProvider.getJWKs(parsedJwt.getHeader());
+    if (matchedJWKS.isEmpty()) {
+      throw new AuthenticationException("Failed to find matched JWKs with the JWT header: " + parsedJwt.getHeader());
+    }
+
+    // verify signature
+    Exception lastException = null;
+    for (JWK matchedJWK : matchedJWKS) {
+      String keyID = matchedJWK.getKeyID() == null ? "null" : matchedJWK.getKeyID();
+      try {
+        JWSVerifier verifier = getVerifier(parsedJwt.getHeader(), matchedJWK);
+        if (parsedJwt.verify(verifier)) {
+          LOG.debug("Verified JWT {} by JWK {}", parsedJwt.getPayload(), keyID);
+          break;
+        }
+      } catch (Exception e) {
+        lastException = e;
+        LOG.warn("Failed to verify JWT {} by JWK {}", parsedJwt.getPayload(), keyID, e);
+      }
+    }
+    // We use only the last seven characters to let a user can differentiate exceptions for different JWT
+    int startIndex = Math.max(0, signedJwt.length() - 7);
+    String lastSevenChars = signedJwt.substring(startIndex);
+    if (parsedJwt.getState() != JWSObject.State.VERIFIED) {
+      throw new AuthenticationException("Failed to verify the JWT signature (ends with " + lastSevenChars + ")",
+          lastException);
+    }
+
+    // verify claims
+    JWTClaimsSet claimsSet = parsedJwt.getJWTClaimsSet();
+    Date expirationTime = claimsSet.getExpirationTime();
+    if (expirationTime != null) {
+      Date now = new Date();
+      if (now.after(expirationTime)) {
+        LOG.warn("Rejecting an expired JWT: {}", parsedJwt.getPayload());
+        throw new AuthenticationException("JWT (ends with " + lastSevenChars + ") has been expired");
+      }
+    }
+
+    // We assume the subject of claims is the query user
+    return claimsSet.getSubject();
+  }
+
+  private static JWSVerifier getVerifier(JWSHeader header, JWK jwk) throws JOSEException {
+    Preconditions.checkArgument(jwk instanceof AsymmetricJWK,
+        "JWT signature verification with symmetric key is not allowed.");
+    Key key = ((AsymmetricJWK) jwk).toPublicKey();
+    return JWS_VERIFIER_FACTORY.createJWSVerifier(header, key);
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/auth/jwt/URLBasedJWKSProvider.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/auth/jwt/URLBasedJWKSProvider.java
new file mode 100644
index 0000000000..f13254d878
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/auth/jwt/URLBasedJWKSProvider.java
@@ -0,0 +1,88 @@
+/*
+ * 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.
+ */
+
+package org.apache.hadoop.hive.metastore.auth.jwt;
+
+import com.nimbusds.jose.JWSHeader;
+import com.nimbusds.jose.jwk.JWK;
+import com.nimbusds.jose.jwk.JWKMatcher;
+import com.nimbusds.jose.jwk.JWKSelector;
+import com.nimbusds.jose.jwk.JWKSet;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.security.sasl.AuthenticationException;
+import java.io.IOException;
+import java.net.URL;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Provides a way to get JWKS json. HiveMetastore will use this to verify the incoming JWTs.
+ * This is cloned from URLBasedJWKSProvider in HS2 so as to NOT have any dependency on HS2 code.
+ */
+public class URLBasedJWKSProvider {
+
+  private static final Logger LOG = LoggerFactory.getLogger(URLBasedJWKSProvider.class.getName());
+  private final Configuration conf;
+  private List<JWKSet> jwkSets = new ArrayList<>();
+
+  public URLBasedJWKSProvider(Configuration conf) throws IOException, ParseException {
+    this.conf = conf;
+    loadJWKSets();
+  }
+
+  /**
+   * Fetches the JWKS and stores into memory. The JWKS are expected to be in the standard form as defined here -
+   * https://datatracker.ietf.org/doc/html/rfc7517#appendix-A.
+   */
+  private void loadJWKSets() throws IOException, ParseException {
+    String jwksURL = MetastoreConf.getVar(conf, ConfVars.THRIFT_METASTORE_AUTHENTICATION_JWT_JWKS_URL);
+    if (jwksURL == null || jwksURL.isEmpty()) {
+      throw new IOException("Invalid value of property: " +
+          ConfVars.THRIFT_METASTORE_AUTHENTICATION_JWT_JWKS_URL.getHiveName());
+    }
+    String[] jwksURLs = jwksURL.split(",");
+    for (String urlString : jwksURLs) {
+      URL url = new URL(urlString);
+      jwkSets.add(JWKSet.load(url));
+      LOG.info("Loaded JWKS from " + urlString);
+    }
+  }
+
+  /**
+   * Returns filtered JWKS by one or more criteria, such as kid, typ, alg.
+   */
+  public List<JWK> getJWKs(JWSHeader header) throws AuthenticationException {
+    JWKMatcher matcher = JWKMatcher.forJWSHeader(header);
+    if (matcher == null) {
+      throw new AuthenticationException("Unsupported algorithm: " + header.getAlgorithm());
+    }
+
+    List<JWK> jwks = new ArrayList<>();
+    JWKSelector selector = new JWKSelector(matcher);
+    for (JWKSet jwkSet : jwkSets) {
+      jwks.addAll(selector.select(jwkSet));
+    }
+    return jwks;
+  }
+}
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetastoreWithHttpJwt.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetastoreWithHttpJwt.java
new file mode 100644
index 0000000000..c228e94fd4
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetastoreWithHttpJwt.java
@@ -0,0 +1,283 @@
+/*
+ * 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.
+ */
+
+package org.apache.hadoop.hive.metastore;
+
+import com.github.tomakehurst.wiremock.junit.WireMockRule;
+import com.nimbusds.jose.JWSAlgorithm;
+import com.nimbusds.jose.JWSHeader;
+import com.nimbusds.jose.JWSSigner;
+import com.nimbusds.jose.crypto.RSASSASigner;
+import com.nimbusds.jose.jwk.RSAKey;
+import com.nimbusds.jwt.JWTClaimsSet;
+import com.nimbusds.jwt.SignedJWT;
+
+import java.io.File;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.annotation.MetastoreUnitTest;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+import org.apache.thrift.transport.TTransportException;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import java.nio.charset.StandardCharsets;
+import java.util.Date;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static com.github.tomakehurst.wiremock.client.WireMock.get;
+import static com.github.tomakehurst.wiremock.client.WireMock.ok;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/*
+Tests JWT auth in HiveMetastore server.
+ */
+@Category(MetastoreUnitTest.class)
+public class TestRemoteHiveMetastoreWithHttpJwt {
+  private static final Map<String, String> DEFAULTS = new HashMap<>(System.getenv());
+  private static Map<String, String> envMap;
+
+  private static String baseDir = System.getProperty("basedir");
+  private static final File jwtAuthorizedKeyFile =
+      new File(baseDir,"src/test/resources/auth/jwt/jwt-authorized-key.json");
+  private static final File jwtUnauthorizedKeyFile =
+      new File(baseDir,"src/test/resources/auth/jwt/jwt-unauthorized-key.json");
+  private static final File jwtVerificationJWKSFile =
+      new File(baseDir,"src/test/resources/auth/jwt/jwt-verification-jwks.json");
+
+  private static final String USER_1 = "HMS_TEST_USER_1";
+  private static final String TEST_DB_NAME_PREFIX = "HMS_JWT_AUTH_DB";
+  private static final Logger LOG = LoggerFactory.getLogger(TestRemoteHiveMetastoreWithHttpJwt.class);
+  //private static MiniHS2 miniHS2;
+
+  private static final int MOCK_JWKS_SERVER_PORT = 8089;
+  @ClassRule
+  public static final WireMockRule MOCK_JWKS_SERVER = new WireMockRule(MOCK_JWKS_SERVER_PORT);
+
+  /**
+   * This is a hack to make environment variables modifiable.
+   * Ref: https://stackoverflow.com/questions/318239/how-do-i-set-environment-variables-from-java.
+   */
+  @BeforeClass
+  public static void makeEnvModifiable() throws Exception {
+    envMap = new HashMap<>();
+    Class<?> envClass = Class.forName("java.lang.ProcessEnvironment");
+    Field theEnvironmentField = envClass.getDeclaredField("theEnvironment");
+    Field theUnmodifiableEnvironmentField = envClass.getDeclaredField("theUnmodifiableEnvironment");
+    removeStaticFinalAndSetValue(theEnvironmentField, envMap);
+    removeStaticFinalAndSetValue(theUnmodifiableEnvironmentField, envMap);
+  }
+
+  private static void removeStaticFinalAndSetValue(Field field, Object value) throws Exception {
+    field.setAccessible(true);
+    Field modifiersField = Field.class.getDeclaredField("modifiers");
+    modifiersField.setAccessible(true);
+    modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL);
+    field.set(null, value);
+  }
+  private static int port;
+  private static Configuration conf = null;
+
+  public TestRemoteHiveMetastoreWithHttpJwt() {
+    // default constructor
+  }
+
+  @AfterClass
+  public static void stopServices() throws Exception {
+    System.getenv().remove("HMS_JWT");
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    conf = MetastoreConf.newMetastoreConf();
+
+    // set some values to use for getting conf. vars
+    MetastoreConf.setBoolVar(conf, ConfVars.METRICS_ENABLED, true);
+    conf.set("datanucleus.autoCreateTables", "false");
+    conf.set("hive.in.test", "true");
+    MetastoreConf.setVar(conf, ConfVars.METASTORE_METADATA_TRANSFORMER_CLASS, " ");
+
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    MetastoreConf.setLongVar(conf, ConfVars.BATCH_RETRIEVE_MAX, 2);
+    MetastoreConf.setLongVar(conf, ConfVars.LIMIT_PARTITION_REQUEST, 100);
+    MetastoreConf.setVar(conf, ConfVars.STORAGE_SCHEMA_READER_IMPL, "no.such.class");
+
+    setupMockServer();
+    MetastoreConf.setBoolVar(conf, ConfVars.EXECUTE_SET_UGI, false);
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_TRANSPORT_MODE, "http");
+    MetastoreConf.setVar(conf, ConfVars.METASTORE_CLIENT_THRIFT_TRANSPORT_MODE, "http");
+
+    MetastoreConf.setVar(conf, ConfVars.METASTORE_CLIENT_AUTH_MODE, "JWT");
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_METASTORE_AUTHENTICATION, "JWT");
+    startMetastoreServer();
+  }
+
+  private static void startMetastoreServer() throws Exception {
+    port = MetaStoreTestUtils.startMetaStoreWithRetry(HadoopThriftAuthBridge.getBridge(),
+        conf);
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+    System.out.println("Starting MetaStore Server on port " + port);
+  }
+
+  @Before
+  public void initEnvMap() {
+    envMap.clear();
+    envMap.putAll(DEFAULTS);
+  }
+
+  private static void setupMockServer() throws Exception {
+    MOCK_JWKS_SERVER.stubFor(get("/jwks")
+        .willReturn(ok()
+            .withBody(Files.readAllBytes(jwtVerificationJWKSFile.toPath()))));
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_METASTORE_AUTHENTICATION_JWT_JWKS_URL,
+        "http://localhost:" + MOCK_JWKS_SERVER_PORT + "/jwks");
+  }
+
+  /*
+  Tests a valid JWT sent to metastore sever
+   */
+  @Test
+  public void testValidJWT() throws Exception {
+    String validJwtToken = generateJWT(USER_1, jwtAuthorizedKeyFile.toPath(),
+        TimeUnit.MINUTES.toMillis(5));
+    System.getenv().put("HMS_JWT", validJwtToken);
+    String dbName = ("valid_jwt_" + TEST_DB_NAME_PREFIX + "_" + UUID.randomUUID()).toLowerCase();
+    HiveMetaStoreClient client = new HiveMetaStoreClient(conf);
+    try {
+      Database createdDb = new Database();
+      createdDb.setName(dbName);
+      client.createDatabase(createdDb);
+      Database dbFromServer = client.getDatabase(dbName);
+      assertEquals(dbName, dbFromServer.getName());
+    } finally {
+      try {
+        client.dropDatabase(dbName);
+      } catch (Exception e) {
+        LOG.warn("Failed to drop database: " + dbName + ". Error message: " + e);
+      }
+      try {
+        client.close();
+      } catch (Exception e) {
+        LOG.error("Failed to close metastore client");
+      }
+    }
+  }
+
+  /*
+  Tests that an exception is thrown when metastore client (in http mode)
+  sends a expired jwt to metastore server.
+   */
+  @Test(expected = TTransportException.class)
+  public void testExpiredJWT() throws Exception {
+    String validJwtToken = generateJWT(USER_1, jwtAuthorizedKeyFile.toPath(),
+        TimeUnit.MILLISECONDS.toMillis(2));
+    System.getenv().put("HMS_JWT", validJwtToken);
+    String dbName = ("expired_jwt_" + TEST_DB_NAME_PREFIX + "_" + UUID.randomUUID()).toLowerCase();
+    HiveMetaStoreClient client = new HiveMetaStoreClient(conf);
+    try {
+      Thread.sleep(TimeUnit.MILLISECONDS.toMillis(2));
+      Database createdDb = new Database();
+      createdDb.setName(dbName);
+      client.createDatabase(createdDb);
+    } catch (InterruptedException e) {
+      // ignore
+    } finally {
+      try {
+        client.close();
+      } catch (Exception e) {
+        LOG.error("Failed to close metastore client");
+      }
+    }
+  }
+
+  /*
+  Tests that an exception is thrown when metastore client (in http mode) sends an
+  invalid jwt to the metastore server
+   */
+  @Test(expected = TTransportException.class)
+  public void testInvalidJWT() throws Exception {
+    String jwtToken = generateJWT(USER_1, jwtUnauthorizedKeyFile.toPath(),
+        TimeUnit.MINUTES.toMillis(2));
+    System.getenv().put("HMS_JWT", jwtToken);
+    String dbName = ("invalid_jwt_" + TEST_DB_NAME_PREFIX + "_" + UUID.randomUUID()).toLowerCase();
+    HiveMetaStoreClient client = new HiveMetaStoreClient(conf);
+    try {
+      Thread.sleep(TimeUnit.MILLISECONDS.toMillis(2));
+      Database createdDb = new Database();
+      createdDb.setName(dbName);
+      client.createDatabase(createdDb);
+    } catch (InterruptedException e) {
+      // ignore
+    } finally {
+      try {
+        client.close();
+      } catch (Exception e) {
+        LOG.error("Failed to close metastore client");
+      }
+    }
+  }
+
+  private String generateJWT(String user, Path keyFile, long lifeTimeMillis) throws Exception {
+    RSAKey rsaKeyPair = RSAKey.parse(new String(java.nio.file.Files.readAllBytes(keyFile),
+        StandardCharsets.UTF_8));
+
+    // Create RSA-signer with the private key
+    JWSSigner signer = new RSASSASigner(rsaKeyPair);
+
+    JWSHeader header = new JWSHeader
+        .Builder(JWSAlgorithm.RS256)
+        .keyID(rsaKeyPair.getKeyID())
+        .build();
+
+    Date now = new Date();
+    Date expirationTime = new Date(now.getTime() + lifeTimeMillis);
+    JWTClaimsSet claimsSet = new JWTClaimsSet.Builder()
+        .jwtID(UUID.randomUUID().toString())
+        .issueTime(now)
+        .issuer("auth-server")
+        .subject(user)
+        .expirationTime(expirationTime)
+        .claim("custom-claim-or-payload", "custom-claim-or-payload")
+        .build();
+
+    SignedJWT signedJWT = new SignedJWT(header, claimsSet);
+
+    // Compute the RSA signature
+    signedJWT.sign(signer);
+
+    return signedJWT.serialize();
+  }
+
+}
diff --git a/standalone-metastore/metastore-server/src/test/resources/auth/jwt/jwt-authorized-key.json b/standalone-metastore/metastore-server/src/test/resources/auth/jwt/jwt-authorized-key.json
new file mode 100644
index 0000000000..b5b4fb40e7
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/test/resources/auth/jwt/jwt-authorized-key.json
@@ -0,0 +1,12 @@
+{
+  "p": "-8lxjB9JZA44XBLLVGnY20x28uT8NQ1BlbqI0Tlr96An4B_PzgPL5_bFFB7SWs8ehSWn9z2SJfClhQpBLfy-2mXvJek_xgibESIlPXqY9Qrg7-PhRmPs3whyiIsnn8tpPMm2XJ_4n0Y-Yfx4nwErGdy84LiKFMDXPEk2a7ndYWs",
+  "kty": "RSA",
+  "q": "0YAcTLBnTrSUiciE0lliIkAidW0TnHP48v-vJitLEz0d8mlTZ_aeOQJm6CUOqF7BqQv3Z8OK_HYKXfOr7xzUlfROONybUXRFE0LvT5Fjvrq-56QGB6GeFq5i6HKlRcC_8TD6WwUJWIzeYuPqhp_FYIpT4ds131d5VYPKDCdY_dM",
+  "d": "VsxW72idEAtoZQDphvxJ0t54EyRfcIJVB9BZuqnyNTfH-VsaUO3st86w_PMU_i0lmyIc8dkCmwOb8R2pRXDo6UxEYUe5YfBnvn9iYF3Ll2QfPOKfZhDBOfqSjEb1po20is7mXTQORBv3bhSo664pasHItTwDz-KKI-FiIu_PYq0lYihuaedUUMp3MQTvDFulpFWEKzqseBDat07BholvxjzlnBK-Ez3KI9qGH8VIIk5TGW5pVu3cQe1WC8NJOe3xR9vu7XX6xvhVLPP7fvKiXJWJ_I_SagAhR1JW0uDJl_b0CrYYeVUnt_pzvW1BeJGz7ysCXcHlLBUh72XrpW-O7Q",
+  "e": "AQAB",
+  "kid": "123",
+  "qi": "9yk0mg4LY48YS8cvG51wMVfKfEjSbt2ygKxqabdsP-qSVpz-KVJtCmbKa57jm2BaMV_mRBQFodxu4XN58VGsj5MzXC5Jb_CkLeQfkp6ZKvehZhiJn3HF0Kb19u9xPvKDclHpKl-UMM1Pcu8Ww52DOyOYcHa1_SLZ05CcOWvMkS8",
+  "dp": "HYtToYeCSxVIE7W42hzZb1IXmwS3e1ok2fbbWwGL47CNPUU-UwQrBvrzwRqkwDcRc7opbV9yKLWGFohPgZ_onSPc3evyqcAUwfvptr8N96LhJgTtSB8tijYpilAZxCxQGuvoVBIJUFcjtsezN6Uhc5VtLEk7GphOKSrGEfnrOiU",
+  "dq": "tF2uf5v0JT-1DnazW4IWydQblqtlEfKKp3LX8W2egh7BNJ3XcA9UI1LdFAord2u1IXwq8YvZkgdyX3bVVNSmdb_SxIOxuMv4WF_tNry-eku-5iFCC7nqKC7U-rkRb19GIToAoPJSHImTQOJmXKcbQEV3eGDJHdLqpGQFRLdvl38",
+  "n": "zg12QaFTsez1EijOYRFzNZdowOt79ePqxCMQ-EEHynUhEZ6TIDnXfjWfuWocS1qRRglUUbHerEtmACUKPQShaG8uL0ZXiLqDr2QSuqrTtr2VUGesxZc6GiqkZlnWFNu5kSUvtemcKxWl8OLFf-5kNnGW4_4xM6BIwosYZnddfFqQT5IP6iTMZIUIKXxY4s1dadYRIiMteNutro67fhOLKabHkyC6ILE6f6VZsYbb_NXC5yC--7DiC2GYKzy7TKmaczuDfQZVgVY-nL9kTPIdhf334EYHQfYmLdvLc56g8-cxY3xh2GnwAj1JcT2u3hsS4KS05bUFHFnveO5uxIYKMQ"
+}
\ No newline at end of file
diff --git a/standalone-metastore/metastore-server/src/test/resources/auth/jwt/jwt-unauthorized-key.json b/standalone-metastore/metastore-server/src/test/resources/auth/jwt/jwt-unauthorized-key.json
new file mode 100644
index 0000000000..f4845de745
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/test/resources/auth/jwt/jwt-unauthorized-key.json
@@ -0,0 +1,12 @@
+{
+  "p": "wvzuDSY6dIsIJB0UM5BIncN6ui5ee-KHpCmBhh_ia2iX3DluQODEgITw7gDATTDdQsBD-nJLjrqUs5g5Gmt0UgZucXQ5PCt1CK6dLEZCaLivw2fsHYvOKeTkdA49wqLkTc8pkfQs09N-b6NspDDqVJPFffBvFpR_IBFay-xKa5k",
+  "kty": "RSA",
+  "q": "sQzza69VkEmgUm50pEGjgu-OxugOrjcHrjQ42A23YVwAAJ90qPNQa62O7dv5oWmSX2PJ7TgjkzbvtTycLfT_vUeapwfCcJe4WoDg54xF3E35yBvBIwReRiavxf5nWsHEtd5kBg6wRIndGwGUBE91xaLg21spjH7nQKtG9vKeNM8",
+  "d": "UbiPIpr7agQqpM3ERfaXsKNMETyBrIYr3yoggHQ7XQkSPepCgRhE86puRmjf76FtZ3RwpJwjLfO6Ap0fIE9LXXE8otTF9sMnC9fe7odHkEu61Wr3aQM-53dgZoJL7XU53LOo0cNO44SBbw11d2cYlAR3KuCEK7bCLMBOkK1gdxVpgDC7DgxVgnP39bUlf4fA5gQeT5nNGnCWTV4jMVWCyEb0Ck5CvGJp1cCKaMSEvV4j6AM72EkAn8PogTSOJpurRJaTky0De7-ncT2Sv5DCuOIkMhsHqayLbm7a84ORHqsnWpZV85WVW-xxiivkVpqtSDRKCI94pMa9DWszjNJW8Q",
+  "e": "AQAB",
+  "kid": "sig-1642039368",
+  "qi": "CXP_tewCHyXk6PNDcbI0wtXsaWJryOJfMsc7roBCoOwDbTekUFXhOfRmFX5ZTNetRNDpw9nNiQDXt8pyw7UZ-0EhD1cLst1slS__hBi5QEAGo9cUxl3RGeMAFtY9O8B1gjFyKkG5BzdddGBKGQT3Tg23Eyzn6EA_NCw4XAKnkwQ",
+  "dp": "aAdzphZQN595n3LYNU50P59sWeqlRCkuvvnZ_coDDdUGuFr3pKuGix7iP8is0EISuitD2VmjUCnhbhP3202bCKwfvm4Inz58OT6X4mg1xBNMys8mHPla6-UPsY9rie1IKu8suY7xX65FlaA2NT9XtfoE8tUVH5HoZR59N7EAX3k",
+  "dq": "mTkZDO-fgBCH4-7dmS2JIY7KpI897T2IsxVUwH4WXvastd1Jq9FuntGEKYu_HRbtawpEPbzg5M2dY97BVvB5xshKKhWIC8Lx87knapw19XOyIKEMY46rO9DNO-9waNXatH5zV96sY5RgOrgB7j0KMnFEYfIiIgnNfmT8NElB63c",
+  "n": "htq92ltGQrZv19TlhluoqmXjjRXw_NWEd0nPZsWrbLnr8lZ-gOxsjIsDMjb5HNDNmuAS7pg2d_o5ZZAY1sSjKf_EuUPZN-MOej8ZBOtrMxEH7e_t37kYIbbJSuzt55poZdRli6BE8CVDesS4W-wsFZ0MvUazAUADh3onARN7Arf3jwknm5CLafE_JzKrNKZadBElEFEAEu5y9n_SuTlemw3P81lOVmZmjGjfqtPx01O5aV_truMjrQa3NUivu1ihrjvJl0xc3rwJe7qDrfEqgvpBQ-vrAsvg3Jiz5Idj6cU3J0hNtV4ixYxcDQecNlgR7gBeIp3E8BXL1kGOOHYUtw"
+}
\ No newline at end of file
diff --git a/standalone-metastore/metastore-server/src/test/resources/auth/jwt/jwt-verification-jwks.json b/standalone-metastore/metastore-server/src/test/resources/auth/jwt/jwt-verification-jwks.json
new file mode 100644
index 0000000000..a6fd935a0a
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/test/resources/auth/jwt/jwt-verification-jwks.json
@@ -0,0 +1,20 @@
+{
+  "keys": [
+    {
+      "kty": "RSA",
+      "e": "AQAB",
+      "alg": "RS256",
+      "kid": "819d1e61429dd3d3caef129c0ac2bae8c6d46fbc",
+      "use": "sig",
+      "n": "qfR12Bcs_hSL0Y1fN5TYZeUQIFmuVRYa210na81BFj91xxwtICJY6ckZCI3Jf0v2tPLOT_iKVk4WBCZ7AVJVvZqHuttkyrdFROMVTe6DwmcjbbkgACMVildTnHy9xy2KuX-OZsEYzgHuRgfe_Y-JN6LoxBYZx6VoBLpgK-F0Q-0O_bRgZhHifVG4ZzARjhgz0PvBb700GtOTHS6mQIfToPErbgqcowKN9k-mJqJr8xpXSHils-Yw97LHjICZmvA5B8EPNW28DwFOE5JrsPcyrFKOAYl4NcSYQgjl-17TWE5_tFdZ8Lz-srjiPMoHlBjZD1C7aO03LI-_9u8lVsktMw"
+    },
+    {
+      "kty": "RSA",
+      "e": "AQAB",
+      "alg": "RS256",
+      "kid": "123",
+      "use": "sig",
+      "n": "zg12QaFTsez1EijOYRFzNZdowOt79ePqxCMQ-EEHynUhEZ6TIDnXfjWfuWocS1qRRglUUbHerEtmACUKPQShaG8uL0ZXiLqDr2QSuqrTtr2VUGesxZc6GiqkZlnWFNu5kSUvtemcKxWl8OLFf-5kNnGW4_4xM6BIwosYZnddfFqQT5IP6iTMZIUIKXxY4s1dadYRIiMteNutro67fhOLKabHkyC6ILE6f6VZsYbb_NXC5yC--7DiC2GYKzy7TKmaczuDfQZVgVY-nL9kTPIdhf334EYHQfYmLdvLc56g8-cxY3xh2GnwAj1JcT2u3hsS4KS05bUFHFnveO5uxIYKMQ"
+    }
+  ]
+}
\ No newline at end of file
diff --git a/standalone-metastore/pom.xml b/standalone-metastore/pom.xml
index 68f63e3769..b64d68fe21 100644
--- a/standalone-metastore/pom.xml
+++ b/standalone-metastore/pom.xml
@@ -104,6 +104,8 @@
     <caffeine.version>2.8.4</caffeine.version>
     <slf4j.version>1.7.30</slf4j.version>
     <httpcomponents.core.version>4.4.13</httpcomponents.core.version>
+    <pac4j-core.version>4.5.5</pac4j-core.version>
+    <nimbus-jose-jwt.version>9.20</nimbus-jose-jwt.version>
     <!-- Thrift properties -->
     <thrift.home>you-must-set-this-to-run-thrift</thrift.home>
     <thrift.gen.dir>${basedir}/src/gen/thrift</thrift.gen.dir>
@@ -517,6 +519,7 @@
             <exclude>**/metastore_db/**</exclude>
             <exclude>**/test/resources/**/*.ldif</exclude>
             <exclude>**/test/resources/sql/**</exclude>
+            <exclude>**/test/resources/**/*.json</exclude>
           </excludes>
         </configuration>
       </plugin>