You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ja...@apache.org on 2019/02/06 08:17:42 UTC

[lucene-solr] branch branch_8x updated (a552be9 -> d28ec13)

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

janhoy pushed a change to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git.


    from a552be9  SOLR-13193: LeaderTragicEventTest doesn't catch NoSuchFileException when corrupting files
     new ceb7380  SOLR-12121: JWT Token authentication plugin with OpenID Connect implicit flow login through Admin UI
     new 0b3ebdd  SOLR-12121: Fix test fails in JWTAuthPluginIntegrationTest.testMetrics
     new d28ec13  SOLR-12121: Use a different JSON parser for JWTAuthPluginTest after upgrade to Hadoop3

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:
 lucene/ivy-versions.properties                     |   2 +
 solr/CHANGES.txt                                   |   3 +
 solr/NOTICE.txt                                    |  18 +
 solr/core/ivy.xml                                  |   3 +
 .../apache/solr/security/AuthenticationPlugin.java |   1 +
 .../org/apache/solr/security/JWTAuthPlugin.java    | 709 +++++++++++++++++++++
 .../org/apache/solr/security/JWTPrincipal.java     |  88 +++
 .../solr/security/JWTPrincipalWithUserRoles.java   |  71 +++
 .../apache/solr/security/VerifiedUserRoles.java    |  33 +
 .../solr/security/jwt_plugin_jwk_security.json     |  14 +
 .../solr/security/jwt_plugin_jwk_url_security.json |   6 +
 .../solr/security/jwt_well-known-config.json       |  84 +++
 .../solr/security/BasicAuthIntegrationTest.java    |  71 +--
 .../solr/security/BasicAuthStandaloneTest.java     |   2 +-
 .../security/JWTAuthPluginIntegrationTest.java     | 254 ++++++++
 .../apache/solr/security/JWTAuthPluginTest.java    | 405 ++++++++++++
 .../security/PKIAuthenticationIntegrationTest.java |   1 -
 solr/licenses/jose4j-0.6.4.jar.sha1                |   1 +
 ...{ant-LICENSE-ASL.txt => jose4j-LICENSE-ASL.txt} |   0
 solr/licenses/jose4j-NOTICE.txt                    |  13 +
 .../authentication-and-authorization-plugins.adoc  |   6 +-
 .../src/basic-authentication-plugin.adoc           |   2 +-
 .../src/jwt-authentication-plugin.adoc             | 194 ++++++
 .../apispec/cluster.security.JwtAuth.Commands.json |  18 +
 .../apache/solr/cloud/SolrCloudAuthTestCase.java   | 117 +++-
 solr/webapp/web/index.html                         |   1 +
 solr/webapp/web/js/angular/app.js                  |  14 +-
 solr/webapp/web/js/angular/controllers/login.js    | 169 ++++-
 solr/webapp/web/js/angular/controllers/unknown.js  |  37 ++
 solr/webapp/web/js/angular/services.js             |  46 ++
 solr/webapp/web/partials/login.html                |  64 ++
 solr/webapp/web/partials/unknown.html              |  23 +
 32 files changed, 2379 insertions(+), 91 deletions(-)
 create mode 100644 solr/core/src/java/org/apache/solr/security/JWTAuthPlugin.java
 create mode 100644 solr/core/src/java/org/apache/solr/security/JWTPrincipal.java
 create mode 100644 solr/core/src/java/org/apache/solr/security/JWTPrincipalWithUserRoles.java
 create mode 100644 solr/core/src/java/org/apache/solr/security/VerifiedUserRoles.java
 create mode 100644 solr/core/src/test-files/solr/security/jwt_plugin_jwk_security.json
 create mode 100644 solr/core/src/test-files/solr/security/jwt_plugin_jwk_url_security.json
 create mode 100644 solr/core/src/test-files/solr/security/jwt_well-known-config.json
 create mode 100644 solr/core/src/test/org/apache/solr/security/JWTAuthPluginIntegrationTest.java
 create mode 100644 solr/core/src/test/org/apache/solr/security/JWTAuthPluginTest.java
 create mode 100644 solr/licenses/jose4j-0.6.4.jar.sha1
 copy solr/licenses/{ant-LICENSE-ASL.txt => jose4j-LICENSE-ASL.txt} (100%)
 create mode 100644 solr/licenses/jose4j-NOTICE.txt
 create mode 100644 solr/solr-ref-guide/src/jwt-authentication-plugin.adoc
 create mode 100644 solr/solrj/src/resources/apispec/cluster.security.JwtAuth.Commands.json
 create mode 100644 solr/webapp/web/js/angular/controllers/unknown.js
 create mode 100644 solr/webapp/web/partials/unknown.html


[lucene-solr] 01/03: SOLR-12121: JWT Token authentication plugin with OpenID Connect implicit flow login through Admin UI

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

janhoy pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit ceb73803a66fdc5550d80982dcee9ada295febd0
Author: Jan Høydahl <ja...@apache.org>
AuthorDate: Wed Jan 23 13:03:01 2019 +0100

    SOLR-12121: JWT Token authentication plugin with OpenID Connect implicit flow login through Admin UI
    
    (cherry picked from commit ea2c8ba38e32a9f1e7d11cf3687c5469bfd6414c)
---
 lucene/ivy-versions.properties                     |   2 +
 solr/CHANGES.txt                                   |   3 +
 solr/NOTICE.txt                                    |  18 +
 solr/core/ivy.xml                                  |   3 +
 .../apache/solr/security/AuthenticationPlugin.java |   1 +
 .../org/apache/solr/security/JWTAuthPlugin.java    | 709 +++++++++++++++++++++
 .../org/apache/solr/security/JWTPrincipal.java     |  88 +++
 .../solr/security/JWTPrincipalWithUserRoles.java   |  71 +++
 .../apache/solr/security/VerifiedUserRoles.java    |  33 +
 .../solr/security/jwt_plugin_jwk_security.json     |  14 +
 .../solr/security/jwt_plugin_jwk_url_security.json |   6 +
 .../solr/security/jwt_well-known-config.json       |  84 +++
 .../solr/security/BasicAuthIntegrationTest.java    |  71 +--
 .../solr/security/BasicAuthStandaloneTest.java     |   2 +-
 .../security/JWTAuthPluginIntegrationTest.java     | 256 ++++++++
 .../apache/solr/security/JWTAuthPluginTest.java    | 406 ++++++++++++
 .../security/PKIAuthenticationIntegrationTest.java |   1 -
 solr/licenses/jose4j-0.6.4.jar.sha1                |   1 +
 solr/licenses/jose4j-LICENSE-ASL.txt               | 272 ++++++++
 solr/licenses/jose4j-NOTICE.txt                    |  13 +
 .../authentication-and-authorization-plugins.adoc  |   6 +-
 .../src/basic-authentication-plugin.adoc           |   2 +-
 .../src/jwt-authentication-plugin.adoc             | 194 ++++++
 .../apispec/cluster.security.JwtAuth.Commands.json |  18 +
 .../apache/solr/cloud/SolrCloudAuthTestCase.java   |  82 +++
 solr/webapp/web/index.html                         |   1 +
 solr/webapp/web/js/angular/app.js                  |  14 +-
 solr/webapp/web/js/angular/controllers/login.js    | 169 ++++-
 solr/webapp/web/js/angular/controllers/unknown.js  |  37 ++
 solr/webapp/web/js/angular/services.js             |  46 ++
 solr/webapp/web/partials/login.html                |  64 ++
 solr/webapp/web/partials/unknown.html              |  23 +
 32 files changed, 2637 insertions(+), 73 deletions(-)

diff --git a/lucene/ivy-versions.properties b/lucene/ivy-versions.properties
index 8eea8e9..2d46772 100644
--- a/lucene/ivy-versions.properties
+++ b/lucene/ivy-versions.properties
@@ -201,6 +201,8 @@ org.apache.tika.version = 1.19.1
 
 /org.aspectj/aspectjrt = 1.8.0
 
+/org.bitbucket.b_c/jose4j = 0.6.4
+
 org.bouncycastle.version = 1.60
 /org.bouncycastle/bcmail-jdk15on = ${org.bouncycastle.version}
 /org.bouncycastle/bcpkix-jdk15on = ${org.bouncycastle.version}
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 27aebd0..5ad8d54 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -28,6 +28,8 @@ Velocity 1.7 and Velocity Tools 2.0
 Apache ZooKeeper 3.4.13
 Jetty 9.4.14.v20181114
 
+Upgrade Notes
+----------------------
 
 New Features
 ----------------------
@@ -43,6 +45,7 @@ Improvements
 * SOLR-12999: Index replication could delete segments before downloading segments from master if there is not enough
   disk space (noble)
 
+* SOLR-12121: JWT Token authentication plugin with OpenID Connect implicit flow login through Admin UI (janhoy)
 
 ==================  8.0.0 ==================
 
diff --git a/solr/NOTICE.txt b/solr/NOTICE.txt
index 48fdb10..daeddbb 100644
--- a/solr/NOTICE.txt
+++ b/solr/NOTICE.txt
@@ -573,3 +573,21 @@ Copyright 2013 The MITRE Corporation. All Rights Reserved.
   subject to the Rights in Noncommercial Computer Software
   and Noncommercial Computer Software Documentation
   Clause 252.227-7014 (JUN 1995)
+
+=========================================================================
+==     Jose4j Notice                                                   ==
+=========================================================================
+
+jose4j
+Copyright 2012-2015 Brian Campbell
+
+EcdsaUsingShaAlgorithm contains code for converting the concatenated
+R & S values of the signature to and from DER, which was originally
+derived from the Apache Santuario XML Security library's SignatureECDSA
+implementation. http://santuario.apache.org/
+
+The Base64 implementation in this software was derived from the
+Apache Commons Codec project. http://commons.apache.org/proper/commons-codec/
+
+JSON processing in this software was derived from the JSON.simple toolkit.
+https://code.google.com/p/json-simple/
\ No newline at end of file
diff --git a/solr/core/ivy.xml b/solr/core/ivy.xml
index e695bc5..37370cf 100644
--- a/solr/core/ivy.xml
+++ b/solr/core/ivy.xml
@@ -136,6 +136,9 @@
 
     <dependency org="org.rrd4j" name="rrd4j" rev="${/org.rrd4j/rrd4j}" conf="compile"/>
 
+    <!-- JWT Auth plugin -->
+    <dependency org="org.bitbucket.b_c" name="jose4j" rev="${/org.bitbucket.b_c/jose4j}" conf="compile"/>
+
     <exclude org="*" ext="*" matcher="regexp" type="${ivy.exclude.types}"/> 
   </dependencies>
 </ivy-module>
diff --git a/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java b/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
index 21fd5c1..5fd18a1 100644
--- a/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
@@ -44,6 +44,7 @@ import org.eclipse.jetty.client.api.Request;
 public abstract class AuthenticationPlugin implements Closeable, SolrInfoBean, SolrMetricProducer {
 
   final public static String AUTHENTICATION_PLUGIN_PROP = "authenticationPlugin";
+  final public static String HTTP_HEADER_X_SOLR_AUTHDATA = "X-Solr-AuthData";
 
   // Metrics
   private Set<String> metricNames = ConcurrentHashMap.newKeySet();
diff --git a/solr/core/src/java/org/apache/solr/security/JWTAuthPlugin.java b/solr/core/src/java/org/apache/solr/security/JWTAuthPlugin.java
new file mode 100644
index 0000000..692ce75
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/security/JWTAuthPlugin.java
@@ -0,0 +1,709 @@
+/*
+ * 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.solr.security;
+
+import javax.servlet.FilterChain;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletRequestWrapper;
+import javax.servlet.http.HttpServletResponse;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.invoke.MethodHandles;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.Principal;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.StringTokenizer;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.http.HttpHeaders;
+import org.apache.http.HttpRequest;
+import org.apache.http.client.protocol.HttpClientContext;
+import org.apache.http.protocol.HttpContext;
+import org.apache.solr.client.solrj.impl.Http2SolrClient;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SpecProvider;
+import org.apache.solr.common.StringUtils;
+import org.apache.solr.common.util.Base64;
+import org.apache.solr.common.util.CommandOperation;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.common.util.ValidatingJsonMap;
+import org.apache.solr.security.JWTAuthPlugin.JWTAuthenticationResponse.AuthCode;
+import org.eclipse.jetty.client.api.Request;
+import org.jose4j.jwa.AlgorithmConstraints;
+import org.jose4j.jwk.HttpsJwks;
+import org.jose4j.jwk.JsonWebKey;
+import org.jose4j.jwk.JsonWebKeySet;
+import org.jose4j.jwt.JwtClaims;
+import org.jose4j.jwt.MalformedClaimException;
+import org.jose4j.jwt.consumer.InvalidJwtException;
+import org.jose4j.jwt.consumer.JwtConsumer;
+import org.jose4j.jwt.consumer.JwtConsumerBuilder;
+import org.jose4j.keys.resolvers.HttpsJwksVerificationKeyResolver;
+import org.jose4j.keys.resolvers.JwksVerificationKeyResolver;
+import org.jose4j.keys.resolvers.VerificationKeyResolver;
+import org.jose4j.lang.JoseException;
+import org.noggit.JSONUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Authenticaion plugin that finds logged in user by validating the signature of a JWT token
+ */
+public class JWTAuthPlugin extends AuthenticationPlugin implements SpecProvider, ConfigEditablePlugin {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private static final String PARAM_BLOCK_UNKNOWN = "blockUnknown";
+  private static final String PARAM_JWK_URL = "jwkUrl";
+  private static final String PARAM_JWK = "jwk";
+  private static final String PARAM_ISSUER = "iss";
+  private static final String PARAM_AUDIENCE = "aud";
+  private static final String PARAM_REQUIRE_SUBJECT = "requireSub";
+  private static final String PARAM_PRINCIPAL_CLAIM = "principalClaim";
+  private static final String PARAM_REQUIRE_EXPIRATIONTIME = "requireExp";
+  private static final String PARAM_ALG_WHITELIST = "algWhitelist";
+  private static final String PARAM_JWK_CACHE_DURATION = "jwkCacheDur";
+  private static final String PARAM_CLAIMS_MATCH = "claimsMatch";
+  private static final String PARAM_SCOPE = "scope";
+  private static final String PARAM_ADMINUI_SCOPE = "adminUiScope";
+  private static final String PARAM_REDIRECT_URIS = "redirectUris";
+  private static final String PARAM_CLIENT_ID = "clientId";
+  private static final String PARAM_WELL_KNOWN_URL = "wellKnownUrl";
+  private static final String PARAM_AUTHORIZATION_ENDPOINT = "authorizationEndpoint";
+
+  private static final String AUTH_REALM = "solr-jwt";
+  private static final String CLAIM_SCOPE = "scope";
+  private static final long RETRY_INIT_DELAY_SECONDS = 30;
+
+  private static final Set<String> PROPS = ImmutableSet.of(PARAM_BLOCK_UNKNOWN, PARAM_JWK_URL, PARAM_JWK, PARAM_ISSUER,
+      PARAM_AUDIENCE, PARAM_REQUIRE_SUBJECT, PARAM_PRINCIPAL_CLAIM, PARAM_REQUIRE_EXPIRATIONTIME, PARAM_ALG_WHITELIST,
+      PARAM_JWK_CACHE_DURATION, PARAM_CLAIMS_MATCH, PARAM_SCOPE, PARAM_CLIENT_ID, PARAM_WELL_KNOWN_URL, 
+      PARAM_AUTHORIZATION_ENDPOINT, PARAM_ADMINUI_SCOPE, PARAM_REDIRECT_URIS);
+
+  private JwtConsumer jwtConsumer;
+  private String iss;
+  private String aud;
+  private boolean requireSubject;
+  private boolean requireExpirationTime;
+  private List<String> algWhitelist;
+  private VerificationKeyResolver verificationKeyResolver;
+  private String principalClaim;
+  private HashMap<String, Pattern> claimsMatchCompiled;
+  private boolean blockUnknown;
+  private List<String> requiredScopes = new ArrayList<>();
+  private String clientId;
+  private long jwkCacheDuration;
+  private WellKnownDiscoveryConfig oidcDiscoveryConfig;
+  private String confIdpConfigUrl;
+  private Map<String, Object> pluginConfig;
+  private Instant lastInitTime = Instant.now();
+  private String authorizationEndpoint;
+  private String adminUiScope;
+  private List<String> redirectUris;
+
+
+  /**
+   * Initialize plugin
+   */
+  public JWTAuthPlugin() {}
+
+  @Override
+  public void init(Map<String, Object> pluginConfig) {
+    List<String> unknownKeys = pluginConfig.keySet().stream().filter(k -> !PROPS.contains(k)).collect(Collectors.toList());
+    unknownKeys.remove("class");
+    unknownKeys.remove("");
+    if (!unknownKeys.isEmpty()) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Invalid JwtAuth configuration parameter " + unknownKeys); 
+    }
+
+    blockUnknown = Boolean.parseBoolean(String.valueOf(pluginConfig.getOrDefault(PARAM_BLOCK_UNKNOWN, false)));
+    clientId = (String) pluginConfig.get(PARAM_CLIENT_ID);
+    requireSubject = Boolean.parseBoolean(String.valueOf(pluginConfig.getOrDefault(PARAM_REQUIRE_SUBJECT, "true")));
+    requireExpirationTime = Boolean.parseBoolean(String.valueOf(pluginConfig.getOrDefault(PARAM_REQUIRE_EXPIRATIONTIME, "true")));
+    principalClaim = (String) pluginConfig.getOrDefault(PARAM_PRINCIPAL_CLAIM, "sub");
+    confIdpConfigUrl = (String) pluginConfig.get(PARAM_WELL_KNOWN_URL);
+    Object redirectUrisObj = pluginConfig.get(PARAM_REDIRECT_URIS);
+    redirectUris = Collections.emptyList();
+    if (redirectUrisObj != null) {
+      if (redirectUrisObj instanceof String) {
+        redirectUris = Collections.singletonList((String) redirectUrisObj);
+      } else if (redirectUrisObj instanceof List) {
+        redirectUris = (List<String>) redirectUrisObj;
+      }
+    } 
+    
+    if (confIdpConfigUrl != null) {
+      log.debug("Initializing well-known oidc config from {}", confIdpConfigUrl);
+      oidcDiscoveryConfig = WellKnownDiscoveryConfig.parse(confIdpConfigUrl);
+      iss = oidcDiscoveryConfig.getIssuer();
+      authorizationEndpoint = oidcDiscoveryConfig.getAuthorizationEndpoint();
+    }
+    
+    if (pluginConfig.containsKey(PARAM_ISSUER)) {
+      if (iss != null) {
+        log.debug("Explicitly setting required issuer instead of using issuer from well-known config");
+      }
+      iss = (String) pluginConfig.get(PARAM_ISSUER);
+    }
+
+    if (pluginConfig.containsKey(PARAM_AUTHORIZATION_ENDPOINT)) {
+      if (authorizationEndpoint != null) {
+        log.debug("Explicitly setting authorizationEndpoint instead of using issuer from well-known config");
+      }
+      authorizationEndpoint = (String) pluginConfig.get(PARAM_AUTHORIZATION_ENDPOINT);
+    }
+    
+    if (pluginConfig.containsKey(PARAM_AUDIENCE)) {
+      if (clientId != null) {
+        log.debug("Explicitly setting required audience instead of using configured clientId");
+      }
+      aud = (String) pluginConfig.get(PARAM_AUDIENCE);
+    } else {
+      aud = clientId;
+    }
+    
+    algWhitelist = (List<String>) pluginConfig.get(PARAM_ALG_WHITELIST);
+
+    String requiredScopesStr = (String) pluginConfig.get(PARAM_SCOPE);
+    if (!StringUtils.isEmpty(requiredScopesStr)) {
+      requiredScopes = Arrays.asList(requiredScopesStr.split("\\s+"));
+    }
+    
+    adminUiScope = (String) pluginConfig.get(PARAM_ADMINUI_SCOPE);
+    if (adminUiScope == null && requiredScopes.size() > 0) {
+      adminUiScope = requiredScopes.get(0);
+      log.warn("No adminUiScope given, using first scope in 'scope' list as required scope for accessing Admin UI");
+    }
+    
+    if (adminUiScope == null) {
+      adminUiScope = "solr";
+      log.warn("Warning: No adminUiScope provided, fallback to 'solr' as required scope. If this is not correct, the Admin UI login may not work");
+    }
+    
+    Map<String, String> claimsMatch = (Map<String, String>) pluginConfig.get(PARAM_CLAIMS_MATCH);
+    claimsMatchCompiled = new HashMap<>();
+    if (claimsMatch != null) {
+      for (Map.Entry<String, String> entry : claimsMatch.entrySet()) {
+        claimsMatchCompiled.put(entry.getKey(), Pattern.compile(entry.getValue()));
+      }
+    }
+
+    initJwk(pluginConfig);
+
+    lastInitTime = Instant.now();
+  }
+
+  private void initJwk(Map<String, Object> pluginConfig) {
+    this.pluginConfig = pluginConfig;
+    String confJwkUrl = (String) pluginConfig.get(PARAM_JWK_URL);
+    Map<String, Object> confJwk = (Map<String, Object>) pluginConfig.get(PARAM_JWK);
+    jwkCacheDuration = Long.parseLong((String) pluginConfig.getOrDefault(PARAM_JWK_CACHE_DURATION, "3600"));
+
+    jwtConsumer = null;
+    int jwkConfigured = confIdpConfigUrl != null ? 1 : 0;
+    jwkConfigured += confJwkUrl != null ? 1 : 0;
+    jwkConfigured += confJwk != null ? 1 : 0;
+    if (jwkConfigured > 1) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "JWTAuthPlugin needs to configure exactly one of " +
+          PARAM_WELL_KNOWN_URL + ", " + PARAM_JWK_URL + " and " + PARAM_JWK);
+    }
+    if (jwkConfigured == 0) {
+      log.warn("Initialized JWTAuthPlugin without any JWK config. Requests with jwk header will fail.");
+    }
+    if (oidcDiscoveryConfig != null) {
+      String jwkUrl = oidcDiscoveryConfig.getJwksUrl();
+      setupJwkUrl(jwkUrl);
+    } else if (confJwkUrl != null) {
+      setupJwkUrl(confJwkUrl);
+    } else if (confJwk != null) {
+      try {
+        JsonWebKeySet jwks = parseJwkSet(confJwk);
+        verificationKeyResolver = new JwksVerificationKeyResolver(jwks.getJsonWebKeys());
+      } catch (JoseException e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Invalid JWTAuthPlugin configuration, " + PARAM_JWK + " parse error", e);
+      }
+    }
+    initConsumer();
+    log.debug("JWK configured");
+  }
+
+  private void setupJwkUrl(String url) {
+    // The HttpsJwks retrieves and caches keys from a the given HTTPS JWKS endpoint.
+    try {
+      URL jwkUrl = new URL(url);
+      if (!"https".equalsIgnoreCase(jwkUrl.getProtocol())) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, PARAM_JWK_URL + " must be an HTTPS url");
+      }
+    } catch (MalformedURLException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, PARAM_JWK_URL + " must be a valid URL");
+    }
+    HttpsJwks httpsJkws = new HttpsJwks(url);
+    httpsJkws.setDefaultCacheDuration(jwkCacheDuration);
+    verificationKeyResolver = new HttpsJwksVerificationKeyResolver(httpsJkws);
+  }
+
+  JsonWebKeySet parseJwkSet(Map<String, Object> jwkObj) throws JoseException {
+    JsonWebKeySet webKeySet = new JsonWebKeySet();
+    if (jwkObj.containsKey("keys")) {
+      List<Object> jwkList = (List<Object>) jwkObj.get("keys");
+      for (Object jwkO : jwkList) {
+        webKeySet.addJsonWebKey(JsonWebKey.Factory.newJwk((Map<String, Object>) jwkO));
+      }
+    } else {
+      webKeySet = new JsonWebKeySet(JsonWebKey.Factory.newJwk(jwkObj));
+    }
+    return webKeySet;
+  }
+
+  /**
+   * Main authentication method that looks for correct JWT token in the Authorization header
+   */
+  @Override
+  public boolean doAuthenticate(ServletRequest servletRequest, ServletResponse servletResponse, FilterChain filterChain) throws Exception {
+    HttpServletRequest request = (HttpServletRequest) servletRequest;
+    HttpServletResponse response = (HttpServletResponse) servletResponse;
+    
+    String header = request.getHeader(HttpHeaders.AUTHORIZATION);
+
+    if (jwtConsumer == null) {
+      if (header == null && !blockUnknown) {
+        log.info("JWTAuth not configured, but allowing anonymous access since {}==false", PARAM_BLOCK_UNKNOWN);
+        filterChain.doFilter(request, response);
+        numPassThrough.inc();;
+        return true;
+      }
+      // Retry config
+      if (lastInitTime.plusSeconds(RETRY_INIT_DELAY_SECONDS).isAfter(Instant.now())) {
+        log.info("Retrying JWTAuthPlugin initialization (retry delay={}s)", RETRY_INIT_DELAY_SECONDS);
+        init(pluginConfig);
+      }
+      if (jwtConsumer == null) {
+        log.warn("JWTAuth not configured");
+        numErrors.mark();
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "JWTAuth plugin not correctly configured");
+      }
+    }
+
+    JWTAuthenticationResponse authResponse = authenticate(header);
+    switch(authResponse.getAuthCode()) {
+      case AUTHENTICATED:
+        HttpServletRequestWrapper wrapper = new HttpServletRequestWrapper(request) {
+          @Override
+          public Principal getUserPrincipal() {
+            return authResponse.getPrincipal();
+          }
+        };
+        if (!(authResponse.getPrincipal() instanceof JWTPrincipal)) {
+          numErrors.mark();
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "JWTAuth plugin says AUTHENTICATED but no token extracted");
+        }
+        if (log.isDebugEnabled())
+          log.debug("Authentication SUCCESS");
+        filterChain.doFilter(wrapper, response);
+        numAuthenticated.inc();
+        return true;
+
+      case PASS_THROUGH:
+        if (log.isDebugEnabled())
+          log.debug("Unknown user, but allow due to {}=false", PARAM_BLOCK_UNKNOWN);
+        filterChain.doFilter(request, response);
+        numPassThrough.inc();
+        return true;
+
+      case AUTZ_HEADER_PROBLEM:
+      case JWT_PARSE_ERROR:
+        authenticationFailure(response, authResponse.getAuthCode().getMsg(), HttpServletResponse.SC_BAD_REQUEST, BearerWwwAuthErrorCode.invalid_request);
+        numErrors.mark();
+        return false;
+
+      case CLAIM_MISMATCH:
+      case JWT_EXPIRED:
+      case JWT_VALIDATION_EXCEPTION:
+      case PRINCIPAL_MISSING:
+        if (authResponse.getJwtException() != null) {
+          log.warn("Exception: {}", authResponse.getJwtException().getMessage());
+        }
+        authenticationFailure(response, authResponse.getAuthCode().getMsg(), HttpServletResponse.SC_UNAUTHORIZED, BearerWwwAuthErrorCode.invalid_token);
+        numWrongCredentials.inc();
+        return false;
+
+      case SCOPE_MISSING:
+        authenticationFailure(response, authResponse.getAuthCode().getMsg(), HttpServletResponse.SC_UNAUTHORIZED, BearerWwwAuthErrorCode.insufficient_scope);
+        numWrongCredentials.inc();
+        return false;
+        
+      case NO_AUTZ_HEADER:
+      default:
+        authenticationFailure(response, authResponse.getAuthCode().getMsg(), HttpServletResponse.SC_UNAUTHORIZED, null);
+        numMissingCredentials.inc();
+        return false;
+    }
+  }
+
+  /**
+   * Testable authentication method
+   *
+   * @param authorizationHeader the http header "Authentication"
+   * @return AuthenticationResponse object
+   */
+  protected JWTAuthenticationResponse authenticate(String authorizationHeader) {
+    if (authorizationHeader != null) {
+      StringTokenizer st = new StringTokenizer(authorizationHeader);
+      if (st.hasMoreTokens()) {
+        String bearer = st.nextToken();
+        if (bearer.equalsIgnoreCase("Bearer") && st.hasMoreTokens()) {
+          try {
+            String jwtCompact = st.nextToken();
+            try {
+              JwtClaims jwtClaims = jwtConsumer.processToClaims(jwtCompact);
+              String principal = jwtClaims.getStringClaimValue(principalClaim);
+              if (principal == null || principal.isEmpty()) {
+                return new JWTAuthenticationResponse(AuthCode.PRINCIPAL_MISSING, "Cannot identify principal from JWT. Required claim " + principalClaim + " missing. Cannot authenticate");
+              }
+              if (claimsMatchCompiled != null) {
+                for (Map.Entry<String, Pattern> entry : claimsMatchCompiled.entrySet()) {
+                  String claim = entry.getKey();
+                  if (jwtClaims.hasClaim(claim)) {
+                    if (!entry.getValue().matcher(jwtClaims.getStringClaimValue(claim)).matches()) {
+                      return new JWTAuthenticationResponse(AuthCode.CLAIM_MISMATCH,
+                          "Claim " + claim + "=" + jwtClaims.getStringClaimValue(claim)
+                              + " does not match required regular expression " + entry.getValue().pattern());
+                    }
+                  } else {
+                    return new JWTAuthenticationResponse(AuthCode.CLAIM_MISMATCH, "Claim " + claim + " is required but does not exist in JWT");
+                  }
+                }
+              }
+              if (!requiredScopes.isEmpty() && !jwtClaims.hasClaim(CLAIM_SCOPE)) {
+                // Fail if we require scopes but they don't exist
+                return new JWTAuthenticationResponse(AuthCode.CLAIM_MISMATCH, "Claim " + CLAIM_SCOPE + " is required but does not exist in JWT");
+              }
+              Set<String> scopes = Collections.emptySet();
+              Object scopesObj = jwtClaims.getClaimValue(CLAIM_SCOPE);
+              if (scopesObj != null) {
+                if (scopesObj instanceof String) {
+                  scopes = new HashSet<>(Arrays.asList(((String) scopesObj).split("\\s+")));
+                } else if (scopesObj instanceof List) {
+                  scopes = new HashSet<>(jwtClaims.getStringListClaimValue(CLAIM_SCOPE));
+                }
+                // Validate that at least one of the required scopes are present in the scope claim 
+                if (!requiredScopes.isEmpty()) {
+                  if (scopes.stream().noneMatch(requiredScopes::contains)) {
+                    return new JWTAuthenticationResponse(AuthCode.SCOPE_MISSING, "Claim " + CLAIM_SCOPE + " does not contain any of the required scopes: " + requiredScopes);
+                  }
+                }
+                final Set<String> finalScopes = new HashSet<>(scopes);
+                finalScopes.remove("openid"); // Remove standard scope
+                // Pass scopes with principal to signal to any Authorization plugins that user has some verified role claims
+                return new JWTAuthenticationResponse(AuthCode.AUTHENTICATED, new JWTPrincipalWithUserRoles(principal, jwtCompact, jwtClaims.getClaimsMap(), finalScopes));
+              } else {
+                return new JWTAuthenticationResponse(AuthCode.AUTHENTICATED, new JWTPrincipal(principal, jwtCompact, jwtClaims.getClaimsMap()));
+              }
+            } catch (InvalidJwtException e) {
+              // Whether or not the JWT has expired being one common reason for invalidity
+              System.out.println("Exception is " + e.getClass().getName() + ", " + e.getMessage() + ", code=" + e.getErrorDetails().get(0).getErrorCode());
+              if (e.hasExpired()) {
+                return new JWTAuthenticationResponse(AuthCode.JWT_EXPIRED, "Authentication failed due to expired JWT token. Expired at " + e.getJwtContext().getJwtClaims().getExpirationTime());
+              }
+              if (e.getCause() != null && e.getCause() instanceof JoseException && e.getCause().getMessage().contains("Invalid JOSE Compact Serialization")) {
+                return new JWTAuthenticationResponse(AuthCode.JWT_PARSE_ERROR, e.getCause().getMessage());
+              }
+              return new JWTAuthenticationResponse(AuthCode.JWT_VALIDATION_EXCEPTION, e);
+            }
+          } catch (MalformedClaimException e) {
+            return new JWTAuthenticationResponse(AuthCode.JWT_PARSE_ERROR, "Malformed claim, error was: " + e.getMessage());
+          }
+        } else {
+          return new JWTAuthenticationResponse(AuthCode.AUTZ_HEADER_PROBLEM, "Authorization header is not in correct format");
+        }
+      } else {
+        return new JWTAuthenticationResponse(AuthCode.AUTZ_HEADER_PROBLEM, "Authorization header is not in correct format");
+      }
+    } else {
+      // No Authorization header
+      if (blockUnknown) {
+        return new JWTAuthenticationResponse(AuthCode.NO_AUTZ_HEADER, "Missing Authorization header");
+      } else {
+        log.debug("No user authenticated, but blockUnknown=false, so letting request through");
+        return new JWTAuthenticationResponse(AuthCode.PASS_THROUGH);
+      }
+    }
+  }
+
+  private void initConsumer() {
+    JwtConsumerBuilder jwtConsumerBuilder = new JwtConsumerBuilder()
+        .setAllowedClockSkewInSeconds(30); // allow some leeway in validating time based claims to account for clock skew
+    if (iss != null)
+      jwtConsumerBuilder.setExpectedIssuer(iss); // whom the JWT needs to have been issued by
+    if (aud != null) {
+      jwtConsumerBuilder.setExpectedAudience(aud); // to whom the JWT is intended for
+    } else {
+      jwtConsumerBuilder.setSkipDefaultAudienceValidation();
+    }
+    if (requireSubject)
+      jwtConsumerBuilder.setRequireSubject();
+    if (requireExpirationTime)
+      jwtConsumerBuilder.setRequireExpirationTime();
+    if (algWhitelist != null)
+      jwtConsumerBuilder.setJwsAlgorithmConstraints( // only allow the expected signature algorithm(s) in the given context
+          new AlgorithmConstraints(AlgorithmConstraints.ConstraintType.WHITELIST, algWhitelist.toArray(new String[0])));
+    jwtConsumerBuilder.setVerificationKeyResolver(verificationKeyResolver);
+    jwtConsumer = jwtConsumerBuilder.build(); // create the JwtConsumer instance
+  }
+
+  @Override
+  public void close() throws IOException {
+    jwtConsumer = null;
+  }
+
+  @Override
+  public ValidatingJsonMap getSpec() {
+    return Utils.getSpec("cluster.security.JwtAuth.Commands").getSpec();
+  }
+
+  /**
+   * Operate the commands on the latest conf and return a new conf object
+   * If there are errors in the commands , throw a SolrException. return a null
+   * if no changes are to be made as a result of this edit. It is the responsibility
+   * of the implementation to ensure that the returned config is valid . The framework
+   * does no validation of the data
+   *
+   * @param latestConf latest version of config
+   * @param commands the list of command operations to perform
+   */
+  @Override
+  public Map<String, Object> edit(Map<String, Object> latestConf, List<CommandOperation> commands) {
+    for (CommandOperation command : commands) {
+      if (command.name.equals("set-property")) {
+        for (Map.Entry<String, Object> e : command.getDataMap().entrySet()) {
+          if (PROPS.contains(e.getKey())) {
+            latestConf.put(e.getKey(), e.getValue());
+            return latestConf;
+          } else {
+            command.addError("Unknown property " + e.getKey());
+          }
+        }
+      }
+    }
+    if (!CommandOperation.captureErrors(commands).isEmpty()) return null;
+    return latestConf;
+  }
+
+  private enum BearerWwwAuthErrorCode { invalid_request, invalid_token, insufficient_scope};
+
+  private void authenticationFailure(HttpServletResponse response, String message, int httpCode, BearerWwwAuthErrorCode responseError) throws IOException {
+    List<String> wwwAuthParams = new ArrayList<>();
+    wwwAuthParams.add("Bearer realm=\"" + AUTH_REALM + "\"");
+    if (responseError != null) {
+      wwwAuthParams.add("error=\"" + responseError + "\"");
+      wwwAuthParams.add("error_description=\"" + message + "\"");
+    }
+    response.addHeader(HttpHeaders.WWW_AUTHENTICATE, org.apache.commons.lang.StringUtils.join(wwwAuthParams, ", "));
+    response.addHeader(AuthenticationPlugin.HTTP_HEADER_X_SOLR_AUTHDATA, generateAuthDataHeader());
+    response.sendError(httpCode, message);
+    log.info("JWT Authentication attempt failed: {}", message);
+  }
+
+  protected String generateAuthDataHeader() {
+    Map<String,Object> data = new HashMap<>();
+    data.put(PARAM_AUTHORIZATION_ENDPOINT, authorizationEndpoint);
+    data.put("client_id", clientId);
+    data.put("scope", adminUiScope);
+    data.put("redirect_uris", redirectUris);
+    String headerJson = JSONUtil.toJSON(data);
+    return Base64.byteArrayToBase64(headerJson.getBytes(StandardCharsets.UTF_8));
+  }
+
+
+  /**
+   * Response for authentication attempt
+   */
+  static class JWTAuthenticationResponse {
+    private final Principal principal;
+    private String errorMessage;
+    private AuthCode authCode;
+    private InvalidJwtException jwtException;
+  
+    enum AuthCode {
+      PASS_THROUGH("No user, pass through"),             // Returned when no user authentication but block_unknown=false 
+      AUTHENTICATED("Authenticated"),                    // Returned when authentication OK 
+      PRINCIPAL_MISSING("No principal in JWT"),          // JWT token does not contain necessary principal (typically sub)  
+      JWT_PARSE_ERROR("Invalid JWT"),                    // Problems with parsing the JWT itself 
+      AUTZ_HEADER_PROBLEM("Wrong header"),               // The Authorization header exists but is not correct 
+      NO_AUTZ_HEADER("Require authentication"),          // The Authorization header is missing 
+      JWT_EXPIRED("JWT token expired"),                  // JWT token has expired 
+      CLAIM_MISMATCH("Required JWT claim missing"),      // Some required claims are missing or wrong 
+      JWT_VALIDATION_EXCEPTION("JWT validation failed"), // The JWT parser failed validation. More details in exception
+      SCOPE_MISSING("Required scope missing in JWT");    // None of the required scopes were present in JWT
+  
+      public String getMsg() {
+        return msg;
+      }
+  
+      private final String msg;
+  
+      AuthCode(String msg) {
+        this.msg = msg;
+      }
+    }
+  
+    JWTAuthenticationResponse(AuthCode authCode, InvalidJwtException e) {
+      this.authCode = authCode;
+      this.jwtException = e;
+      principal = null;
+      this.errorMessage = e.getMessage();
+    }
+    
+    JWTAuthenticationResponse(AuthCode authCode, String errorMessage) {
+      this.authCode = authCode;
+      this.errorMessage = errorMessage;
+      principal = null;
+    }
+  
+    JWTAuthenticationResponse(AuthCode authCode, Principal principal) {
+      this.authCode = authCode;
+      this.principal = principal;
+    }
+    
+    JWTAuthenticationResponse(AuthCode authCode) {
+      this.authCode = authCode;
+      principal = null;
+    }
+  
+    boolean isAuthenticated() {
+      return authCode.equals(AuthCode.AUTHENTICATED);
+    }
+  
+    public Principal getPrincipal() {
+      return principal;
+    }
+  
+    String getErrorMessage() {
+      return errorMessage;
+    }
+  
+    InvalidJwtException getJwtException() {
+      return jwtException;
+    }
+  
+    AuthCode getAuthCode() {
+      return authCode;
+    }
+  }
+
+  /**
+   * Config object for a OpenId Connect well-known config
+   * Typically exposed through /.well-known/openid-configuration endpoint 
+   */
+  public static class WellKnownDiscoveryConfig {
+    private static Map<String, Object> securityConf;
+  
+    WellKnownDiscoveryConfig(Map<String, Object> securityConf) {
+      WellKnownDiscoveryConfig.securityConf = securityConf;
+    }
+  
+    public static WellKnownDiscoveryConfig parse(String urlString) {
+      try {
+        URL url = new URL(urlString);
+        if (!Arrays.asList("https", "file").contains(url.getProtocol())) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Well-known config URL must be HTTPS or file");
+        }
+        return parse(url.openStream());
+      } catch (MalformedURLException e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Well-known config URL " + urlString + " is malformed", e);
+      } catch (IOException e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Well-known config could not be read from url " + urlString, e);
+      }
+    }
+  
+    public static WellKnownDiscoveryConfig parse(String json, Charset charset) {
+      return parse(new ByteArrayInputStream(json.getBytes(charset)));
+    }
+  
+    public static WellKnownDiscoveryConfig parse(InputStream configStream) {
+      securityConf = (Map<String, Object>) Utils.fromJSON(configStream);
+      return new WellKnownDiscoveryConfig(securityConf);
+    }
+  
+    
+    public String getJwksUrl() {
+      return (String) securityConf.get("jwks_uri");
+    }
+  
+    public String getIssuer() {
+      return (String) securityConf.get("issuer");
+    }
+  
+    public String getAuthorizationEndpoint() {
+      return (String) securityConf.get("authorization_endpoint");
+    }
+    
+    public String getUserInfoEndpoint() {
+      return (String) securityConf.get("userinfo_endpoint");
+    }
+
+    public String getTokenEndpoint() {
+      return (String) securityConf.get("token_endpoint");
+    }
+
+    public List<String> getScopesSupported() {
+      return (List<String>) securityConf.get("scopes_supported");
+    }
+
+    public List<String> getResponseTypesSupported() {
+      return (List<String>) securityConf.get("response_types_supported");
+    }
+  }
+
+  @Override
+  protected boolean interceptInternodeRequest(HttpRequest httpRequest, HttpContext httpContext) {
+    if (httpContext instanceof HttpClientContext) {
+      HttpClientContext httpClientContext = (HttpClientContext) httpContext;
+      if (httpClientContext.getUserToken() instanceof JWTPrincipal) {
+        JWTPrincipal jwtPrincipal = (JWTPrincipal) httpClientContext.getUserToken();
+        httpRequest.setHeader(HttpHeaders.AUTHORIZATION, "Bearer " + jwtPrincipal.getToken());
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  protected boolean interceptInternodeRequest(Request request) {
+    Object userToken = request.getAttributes().get(Http2SolrClient.REQ_PRINCIPAL_KEY);
+    if (userToken instanceof JWTPrincipal) {
+      JWTPrincipal jwtPrincipal = (JWTPrincipal) userToken;
+      request.header(HttpHeaders.AUTHORIZATION, "Bearer " + jwtPrincipal.getToken());
+      return true;
+    }
+    return false;
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/security/JWTPrincipal.java b/solr/core/src/java/org/apache/solr/security/JWTPrincipal.java
new file mode 100644
index 0000000..737f3fa
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/security/JWTPrincipal.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.solr.security;
+
+import java.io.Serializable;
+import java.security.Principal;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.http.util.Args;
+
+/**
+ * Principal object that carries JWT token and claims for authenticated user.
+ */
+public class JWTPrincipal implements Principal, Serializable {
+  private static final long serialVersionUID = 4144666467522831388L;
+  final String username;
+  String token;
+  Map<String,Object> claims;
+
+  /**
+   * User principal with user name as well as one or more roles that he/she belong to
+   * @param username string with user name for user
+   * @param token compact string representation of JWT token
+   * @param claims list of verified JWT claims as a map
+   */
+  public JWTPrincipal(final String username, String token, Map<String,Object> claims) {
+    super();
+    Args.notNull(username, "User name");
+    Args.notNull(token, "JWT token");
+    Args.notNull(claims, "JWT claims");
+    this.token = token;
+    this.claims = claims;
+    this.username = username;
+  }
+
+  @Override
+  public String getName() {
+    return this.username;
+  }
+
+  public String getToken() {
+    return token;
+  }
+
+  public Map<String, Object> getClaims() {
+    return claims;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    JWTPrincipal that = (JWTPrincipal) o;
+    return Objects.equals(username, that.username) &&
+        Objects.equals(token, that.token) &&
+        Objects.equals(claims, that.claims);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(username, token, claims);
+  }
+
+  @Override
+  public String toString() {
+    return "JWTPrincipal{" +
+        "username='" + username + '\'' +
+        ", token='" + "*****" + '\'' +
+        ", claims=" + claims +
+        '}';
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/security/JWTPrincipalWithUserRoles.java b/solr/core/src/java/org/apache/solr/security/JWTPrincipalWithUserRoles.java
new file mode 100644
index 0000000..850dc1f
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/security/JWTPrincipalWithUserRoles.java
@@ -0,0 +1,71 @@
+/*
+ * 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.solr.security;
+
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+import org.apache.http.util.Args;
+
+/**
+ * JWT principal that contains username, token, claims and a list of roles the user has, 
+ * so one can keep track of user-role mappings in an Identity Server external to Solr and 
+ * pass the information to Solr in a signed JWT token. The role information can then be used to authorize
+ * requests without the need to maintain or lookup what roles each user belongs to.
+ */
+public class JWTPrincipalWithUserRoles extends JWTPrincipal implements VerifiedUserRoles {
+  private final Set<String> roles;
+
+  public JWTPrincipalWithUserRoles(final String username, String token, Map<String,Object> claims, Set<String> roles) {
+    super(username, token, claims);
+    Args.notNull(roles, "User roles");
+    this.roles = roles;
+  }
+
+  /**
+   * Gets the list of roles
+   */
+  @Override
+  public Set<String> getVerifiedRoles() {
+    return roles;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (!(o instanceof JWTPrincipalWithUserRoles))
+      return false;
+    JWTPrincipalWithUserRoles that = (JWTPrincipalWithUserRoles) o;
+    return super.equals(o) && roles.equals(that.roles);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(username, token, claims, roles);
+  }
+
+  @Override
+  public String toString() {
+    return "JWTPrincipalWithUserRoles{" +
+        "username='" + username + '\'' +
+        ", token='" + "*****" + '\'' +
+        ", claims=" + claims +
+        ", roles=" + roles +
+        '}';
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/security/VerifiedUserRoles.java b/solr/core/src/java/org/apache/solr/security/VerifiedUserRoles.java
new file mode 100644
index 0000000..ed574e0
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/security/VerifiedUserRoles.java
@@ -0,0 +1,33 @@
+/*
+ * 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.solr.security;
+
+import java.util.Set;
+
+/**
+ * Interface used to pass verified user roles in a Principal object.
+ * An Authorization plugin may check for the presence of verified user
+ * roles on the Principal and choose to use those roles instead of
+ * explicitly configuring roles in config. Such roles may e.g. origin
+ * from a signed and validated JWT token.
+ */
+public interface VerifiedUserRoles {
+  /**
+   * Gets a set of roles that have been verified to belong to a user
+   */
+  Set<String> getVerifiedRoles();
+}
diff --git a/solr/core/src/test-files/solr/security/jwt_plugin_jwk_security.json b/solr/core/src/test-files/solr/security/jwt_plugin_jwk_security.json
new file mode 100644
index 0000000..7daab7a
--- /dev/null
+++ b/solr/core/src/test-files/solr/security/jwt_plugin_jwk_security.json
@@ -0,0 +1,14 @@
+{
+  "authentication": {
+    "class": "solr.JWTAuthPlugin",
+    "blockUnknown": true,
+    "jwk": {
+      "kty": "RSA",
+      "e": "AQAB",
+      "use": "sig",
+      "kid": "test",
+      "alg": "RS256",
+      "n": "jeyrvOaZrmKWjyNXt0myAc_pJ1hNt3aRupExJEx1ewPaL9J9HFgSCjMrYxCB1ETO1NDyZ3nSgjZis-jHHDqBxBjRdq_t1E2rkGFaYbxAyKt220Pwgme_SFTB9MXVrFQGkKyjmQeVmOmV6zM3KK8uMdKQJ4aoKmwBcF5Zg7EZdDcKOFgpgva1Jq-FlEsaJ2xrYDYo3KnGcOHIt9_0NQeLsqZbeWYLxYni7uROFncXYV5FhSJCeR4A_rrbwlaCydGxE0ToC_9HNYibUHlkJjqyUhAgORCbNS8JLCJH8NUi5sDdIawK9GTSyvsJXZ-QHqo4cMUuxWV5AJtaRGghuMUfqQ"
+    }
+  }
+}
\ No newline at end of file
diff --git a/solr/core/src/test-files/solr/security/jwt_plugin_jwk_url_security.json b/solr/core/src/test-files/solr/security/jwt_plugin_jwk_url_security.json
new file mode 100644
index 0000000..74b86ef
--- /dev/null
+++ b/solr/core/src/test-files/solr/security/jwt_plugin_jwk_url_security.json
@@ -0,0 +1,6 @@
+{
+  "authentication" : {
+    "class": "solr.JWTAuthPlugin",
+    "jwkUrl": "https://127.0.0.1:8999/this-will-fail.wks"
+  }
+}
\ No newline at end of file
diff --git a/solr/core/src/test-files/solr/security/jwt_well-known-config.json b/solr/core/src/test-files/solr/security/jwt_well-known-config.json
new file mode 100644
index 0000000..ea6d9f1
--- /dev/null
+++ b/solr/core/src/test-files/solr/security/jwt_well-known-config.json
@@ -0,0 +1,84 @@
+{
+  "issuer":"http://acmepaymentscorp",
+  "authorization_endpoint":"http://acmepaymentscorp/oauth/auz/authorize",
+  "token_endpoint":"http://acmepaymentscorp/oauth/oauth20/token",
+  "userinfo_endpoint":"http://acmepaymentscorp/oauth/userinfo",
+  "jwks_uri":"https://acmepaymentscorp/oauth/jwks",
+  "scopes_supported":[
+    "READ",
+    "WRITE",
+    "DELETE",
+    "openid",
+    "scope",
+    "profile",
+    "email",
+    "address",
+    "phone"
+  ],
+  "response_types_supported":[
+    "code",
+    "code id_token",
+    "code token",
+    "code id_token token",
+    "token",
+    "id_token",
+    "id_token token"
+  ],
+  "grant_types_supported":[
+    "authorization_code",
+    "implicit",
+    "password",
+    "client_credentials",
+    "urn:ietf:params:oauth:grant-type:jwt-bearer"
+  ],
+  "subject_types_supported":[
+    "public"
+  ],
+  "id_token_signing_alg_values_supported":[
+    "HS256",
+    "HS384",
+    "HS512",
+    "RS256",
+    "RS384",
+    "RS512",
+    "ES256",
+    "ES384",
+    "ES512",
+    "PS256",
+    "PS384",
+    "PS512"
+  ],
+  "id_token_encryption_alg_values_supported":[
+    "RSA1_5",
+    "RSA-OAEP",
+    "RSA-OAEP-256",
+    "A128KW",
+    "A192KW",
+    "A256KW",
+    "A128GCMKW",
+    "A192GCMKW",
+    "A256GCMKW",
+    "dir"
+  ],
+  "id_token_encryption_enc_values_supported":[
+    "A128CBC-HS256",
+    "A192CBC-HS384",
+    "A256CBC-HS512",
+    "A128GCM",
+    "A192GCM",
+    "A256GCM"
+  ],
+  "token_endpoint_auth_methods_supported":[
+    "client_secret_post",
+    "client_secret_basic",
+    "client_secret_jwt",
+    "private_key_jwt"
+  ],
+  "token_endpoint_auth_signing_alg_values_supported":[
+    "HS256",
+    "RS256"
+  ],
+  "claims_parameter_supported":false,
+  "request_parameter_supported":false,
+  "request_uri_parameter_supported":false
+}
\ No newline at end of file
diff --git a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
index 39389b6..5233703 100644
--- a/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/security/BasicAuthIntegrationTest.java
@@ -16,9 +16,6 @@
  */
 package org.apache.solr.security;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.Collections.singletonMap;
-
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
@@ -27,23 +24,16 @@ import java.lang.invoke.MethodHandles;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.List;
 import java.util.Map;
-import java.util.Objects;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
-import java.util.function.Predicate;
 
-import org.apache.commons.io.IOUtils;
 import com.codahale.metrics.MetricRegistry;
+import org.apache.commons.io.IOUtils;
 import org.apache.http.HttpResponse;
 import org.apache.http.client.HttpClient;
-import org.apache.http.client.methods.HttpGet;
 import org.apache.http.client.methods.HttpPost;
 import org.apache.http.entity.ByteArrayEntity;
-import org.apache.http.message.AbstractHttpMessage;
-import org.apache.http.message.BasicHeader;
-import org.apache.http.util.EntityUtils;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
@@ -66,9 +56,7 @@ import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.Base64;
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.SolrCLI;
 import org.junit.After;
@@ -77,6 +65,9 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.Collections.singletonMap;
+
 public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -166,7 +157,7 @@ public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
           "}";
 
       HttpPost httpPost = new HttpPost(baseUrl + authcPrefix);
-      setBasicAuthHeader(httpPost, "solr", "SolrRocks");
+      setAuthorizationHeader(httpPost, makeBasicAuthHeader("solr", "SolrRocks"));
       httpPost.setEntity(new ByteArrayEntity(command.getBytes(UTF_8)));
       httpPost.addHeader("Content-Type", "application/json; charset=UTF-8");
       verifySecurityStatus(cl, baseUrl + authcPrefix, "authentication.enabled", "true", 20);
@@ -363,7 +354,7 @@ public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
     HttpPost httpPost;
     HttpResponse r;
     httpPost = new HttpPost(url);
-    setBasicAuthHeader(httpPost, user, pwd);
+    setAuthorizationHeader(httpPost, makeBasicAuthHeader(user, pwd));
     httpPost.setEntity(new ByteArrayEntity(payload.getBytes(UTF_8)));
     httpPost.addHeader("Content-Type", "application/json; charset=UTF-8");
     r = cl.execute(httpPost);
@@ -373,54 +364,6 @@ public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
     Utils.consumeFully(r.getEntity());
   }
 
-  public static void verifySecurityStatus(HttpClient cl, String url, String objPath,
-                                          Object expected, int count) throws Exception {
-    verifySecurityStatus(cl, url, objPath, expected, count, null, null);
-  }
-
-
-  public static void verifySecurityStatus(HttpClient cl, String url, String objPath,
-                                          Object expected, int count, String user, String pwd)
-      throws Exception {
-    boolean success = false;
-    String s = null;
-    List<String> hierarchy = StrUtils.splitSmart(objPath, '/');
-    for (int i = 0; i < count; i++) {
-      HttpGet get = new HttpGet(url);
-      if (user != null) setBasicAuthHeader(get, user, pwd);
-      HttpResponse rsp = cl.execute(get);
-      s = EntityUtils.toString(rsp.getEntity());
-      Map m = null;
-      try {
-        m = (Map) Utils.fromJSONString(s);
-      } catch (Exception e) {
-        fail("Invalid json " + s);
-      }
-      Utils.consumeFully(rsp.getEntity());
-      Object actual = Utils.getObjectByPath(m, true, hierarchy);
-      if (expected instanceof Predicate) {
-        Predicate predicate = (Predicate) expected;
-        if (predicate.test(actual)) {
-          success = true;
-          break;
-        }
-      } else if (Objects.equals(actual == null ? null : String.valueOf(actual), expected)) {
-        success = true;
-        break;
-      }
-      Thread.sleep(50);
-    }
-    assertTrue("No match for " + objPath + " = " + expected + ", full response = " + s, success);
-
-  }
-
-  public static void setBasicAuthHeader(AbstractHttpMessage httpMsg, String user, String pwd) {
-    String userPass = user + ":" + pwd;
-    String encoded = Base64.byteArrayToBase64(userPass.getBytes(UTF_8));
-    httpMsg.setHeader(new BasicHeader("Authorization", "Basic " + encoded));
-    log.info("Added Basic Auth security Header {}",encoded );
-  }
-
   public static Replica getRandomReplica(DocCollection coll, Random random) {
     ArrayList<Replica> l = new ArrayList<>();
 
@@ -433,8 +376,6 @@ public class BasicAuthIntegrationTest extends SolrCloudAuthTestCase {
     return l.isEmpty() ? null : l.get(0);
   }
 
-  protected static final Predicate NOT_NULL_PREDICATE = o -> o != null;
-
   //the password is 'SolrRocks'
   //this could be generated everytime. But , then we will not know if there is any regression
   protected static final String STD_CONF = "{\n" +
diff --git a/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java b/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java
index 1cfd681..381a1fb 100644
--- a/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java
+++ b/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java
@@ -49,7 +49,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.solr.security.BasicAuthIntegrationTest.NOT_NULL_PREDICATE;
+import static org.apache.solr.cloud.SolrCloudAuthTestCase.NOT_NULL_PREDICATE;
 import static org.apache.solr.security.BasicAuthIntegrationTest.STD_CONF;
 import static org.apache.solr.security.BasicAuthIntegrationTest.verifySecurityStatus;
 
diff --git a/solr/core/src/test/org/apache/solr/security/JWTAuthPluginIntegrationTest.java b/solr/core/src/test/org/apache/solr/security/JWTAuthPluginIntegrationTest.java
new file mode 100644
index 0000000..14621e8
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/security/JWTAuthPluginIntegrationTest.java
@@ -0,0 +1,256 @@
+/*
+ * 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.solr.security;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.stream.Collectors;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.http.HttpHeaders;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.entity.ByteArrayEntity;
+import org.apache.http.entity.ContentType;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.impl.HttpClientUtil;
+import org.apache.solr.cloud.SolrCloudAuthTestCase;
+import org.apache.solr.common.util.Pair;
+import org.apache.solr.common.util.Utils;
+import org.jose4j.jwk.PublicJsonWebKey;
+import org.jose4j.jwk.RsaJsonWebKey;
+import org.jose4j.jwk.RsaJwkGenerator;
+import org.jose4j.jws.AlgorithmIdentifiers;
+import org.jose4j.jws.JsonWebSignature;
+import org.jose4j.jwt.JwtClaims;
+import org.jose4j.lang.JoseException;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+/**
+ * Validate that JWT token authentication works in a real cluster.
+ * <p> 
+ * TODO: Test also using SolrJ as client. But that requires a way to set Authorization header on request, see SOLR-13070<br>
+ *       This is also the reason we use {@link org.apache.solr.SolrTestCaseJ4.SuppressSSL} annotation, since we use HttpUrlConnection
+ * </p>
+ */
+@SolrTestCaseJ4.SuppressSSL
+public class JWTAuthPluginIntegrationTest extends SolrCloudAuthTestCase {
+  protected static final int NUM_SERVERS = 2;
+  protected static final int NUM_SHARDS = 2;
+  protected static final int REPLICATION_FACTOR = 1;
+  private final String COLLECTION = "jwtColl";
+  private String jwtTestToken;
+  private String baseUrl;
+  private JsonWebSignature jws;
+  private String jwtTokenWrongSignature;
+
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    
+    configureCluster(NUM_SERVERS)// nodes
+        .withSecurityJson(TEST_PATH().resolve("security").resolve("jwt_plugin_jwk_security.json"))
+        .addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
+        .configure();
+    baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
+
+    String jwkJSON = "{\n" +
+        "  \"kty\": \"RSA\",\n" +
+        "  \"d\": \"i6pyv2z3o-MlYytWsOr3IE1olu2RXZBzjPRBNgWAP1TlLNaphHEvH5aHhe_CtBAastgFFMuP29CFhaL3_tGczkvWJkSveZQN2AHWHgRShKgoSVMspkhOt3Ghha4CvpnZ9BnQzVHnaBnHDTTTfVgXz7P1ZNBhQY4URG61DKIF-JSSClyh1xKuMoJX0lILXDYGGcjVTZL_hci4IXPPTpOJHV51-pxuO7WU5M9252UYoiYyCJ56ai8N49aKIMsqhdGuO4aWUwsGIW4oQpjtce5eEojCprYl-9rDhTwLAFoBtjy6LvkqlR2Ae5dKZYpStljBjK8PJrBvWZjXAEMDdQ8PuQ\",\n" +
+        "  \"e\": \"AQAB\",\n" +
+        "  \"use\": \"sig\",\n" +
+        "  \"kid\": \"test\",\n" +
+        "  \"alg\": \"RS256\",\n" +
+        "  \"n\": \"jeyrvOaZrmKWjyNXt0myAc_pJ1hNt3aRupExJEx1ewPaL9J9HFgSCjMrYxCB1ETO1NDyZ3nSgjZis-jHHDqBxBjRdq_t1E2rkGFaYbxAyKt220Pwgme_SFTB9MXVrFQGkKyjmQeVmOmV6zM3KK8uMdKQJ4aoKmwBcF5Zg7EZdDcKOFgpgva1Jq-FlEsaJ2xrYDYo3KnGcOHIt9_0NQeLsqZbeWYLxYni7uROFncXYV5FhSJCeR4A_rrbwlaCydGxE0ToC_9HNYibUHlkJjqyUhAgORCbNS8JLCJH8NUi5sDdIawK9GTSyvsJXZ-QHqo4cMUuxWV5AJtaRGghuMUfqQ\"\n" +
+        "}";
+
+    PublicJsonWebKey jwk = RsaJsonWebKey.Factory.newPublicJwk(jwkJSON);
+    JwtClaims claims = JWTAuthPluginTest.generateClaims();
+    jws = new JsonWebSignature();
+    jws.setPayload(claims.toJson());
+    jws.setKey(jwk.getPrivateKey());
+    jws.setKeyIdHeaderValue(jwk.getKeyId());
+    jws.setAlgorithmHeaderValue(AlgorithmIdentifiers.RSA_USING_SHA256);
+
+    jwtTestToken = jws.getCompactSerialization();
+    
+    PublicJsonWebKey jwk2 = RsaJwkGenerator.generateJwk(2048);
+    jwk2.setKeyId("k2");
+    JsonWebSignature jws2 = new JsonWebSignature();
+    jws2.setPayload(claims.toJson());
+    jws2.setKey(jwk2.getPrivateKey());
+    jws2.setKeyIdHeaderValue(jwk2.getKeyId());
+    jws2.setAlgorithmHeaderValue(AlgorithmIdentifiers.RSA_USING_SHA256);
+    jwtTokenWrongSignature = jws2.getCompactSerialization();
+
+    cluster.waitForAllNodes(10);
+  }
+
+  @Override
+  @After
+  public void tearDown() throws Exception {
+    shutdownCluster();
+    super.tearDown();
+  }
+
+  @Test(expected = IOException.class)
+  public void infoRequestWithoutToken() throws Exception {
+    get(baseUrl + "/admin/info/system", null);
+  }
+
+  @Test
+  public void testMetrics() throws Exception {
+    boolean isUseV2Api = random().nextBoolean();
+    String authcPrefix = "/admin/authentication";
+    String authzPrefix = "/admin/authorization";
+    if(isUseV2Api){
+      authcPrefix = "/____v2/cluster/security/authentication";
+      authzPrefix = "/____v2/cluster/security/authorization";
+    }
+    String baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
+    CloseableHttpClient cl = HttpClientUtil.createClient(null);
+    
+    createCollection(COLLECTION);
+    
+    // Missing token
+    getAndFail(baseUrl + "/" + COLLECTION + "/query?q=*:*", null);
+    assertAuthMetricsMinimums(2, 1, 0, 0, 1, 0);
+    executeCommand(baseUrl + authcPrefix, cl, "{set-property : { blockUnknown: false}}", jws);
+    verifySecurityStatus(cl, baseUrl + authcPrefix, "authentication/blockUnknown", "false", 20, jws);
+    // Pass through
+    verifySecurityStatus(cl, baseUrl + "/admin/info/key", "key", NOT_NULL_PREDICATE, 20);
+    // Now succeeds since blockUnknown=false 
+    get(baseUrl + "/" + COLLECTION + "/query?q=*:*", null);
+    executeCommand(baseUrl + authcPrefix, cl, "{set-property : { blockUnknown: true}}", null);
+    verifySecurityStatus(cl, baseUrl + authcPrefix, "authentication/blockUnknown", "true", 20, jws);
+
+    assertAuthMetricsMinimums(9, 4, 4, 0, 1, 0);
+    
+    // Wrong Credentials
+    getAndFail(baseUrl + "/" + COLLECTION + "/query?q=*:*", jwtTokenWrongSignature);
+    assertAuthMetricsMinimums(10, 4, 4, 1, 1, 0);
+
+    // JWT parse error
+    getAndFail(baseUrl + "/" + COLLECTION + "/query?q=*:*", "foozzz");
+    assertAuthMetricsMinimums(11, 4, 4, 1, 1, 1);
+    
+    HttpClientUtil.close(cl);
+  }
+
+  @Test
+  public void createCollectionUpdateAndQueryDistributed() throws Exception {
+    // Admin request will use PKI inter-node auth from Overseer, and succeed
+    createCollection(COLLECTION);
+    
+    // Now update three documents
+    assertAuthMetricsMinimums(1, 1, 0, 0, 0, 0);
+    assertPkiAuthMetricsMinimums(12, 12, 0, 0, 0, 0);
+    Pair<String,Integer> result = post(baseUrl + "/" + COLLECTION + "/update?commit=true", "[{\"id\" : \"1\"}, {\"id\": \"2\"}, {\"id\": \"3\"}]", jwtTestToken);
+    assertEquals(Integer.valueOf(200), result.second());
+    assertAuthMetricsMinimums(3, 3, 0, 0, 0, 0);
+    assertPkiAuthMetricsMinimums(13, 13, 0, 0, 0, 0);
+    
+    // First a non distributed query
+    result = get(baseUrl + "/" + COLLECTION + "/query?q=*:*&distrib=false", jwtTestToken);
+    assertEquals(Integer.valueOf(200), result.second());
+    assertAuthMetricsMinimums(4, 4, 0, 0, 0, 0);
+
+    // Now do a distributed query, using JWTAuth for inter-node
+    result = get(baseUrl + "/" + COLLECTION + "/query?q=*:*", jwtTestToken);
+    assertEquals(Integer.valueOf(200), result.second());
+    assertAuthMetricsMinimums(9, 9, 0, 0, 0, 0);
+    
+    // Delete
+    assertEquals(200, get(baseUrl + "/admin/collections?action=DELETE&name=" + COLLECTION, jwtTestToken).second().intValue());
+    assertAuthMetricsMinimums(10, 10, 0, 0, 0, 0);
+    assertPkiAuthMetricsMinimums(15, 15, 0, 0, 0, 0);
+  }
+
+  private void getAndFail(String url, String token) throws IOException {
+    try {
+      get(url, token);
+      fail("Request to " + url + " with token " + token + " should have failed");
+    } catch(Exception e) {}
+  }
+  
+  private Pair<String, Integer> get(String url, String token) throws IOException {
+    URL createUrl = new URL(url);
+    HttpURLConnection createConn = (HttpURLConnection) createUrl.openConnection();
+    if (token != null)
+      createConn.setRequestProperty("Authorization", "Bearer " + token);
+    BufferedReader br2 = new BufferedReader(new InputStreamReader((InputStream) createConn.getContent(), StandardCharsets.UTF_8));
+    String result = br2.lines().collect(Collectors.joining("\n"));
+    int code = createConn.getResponseCode();
+    createConn.disconnect();
+    return new Pair<>(result, code);
+  }
+
+  private Pair<String, Integer> post(String url, String json, String token) throws IOException {
+    URL createUrl = new URL(url);
+    HttpURLConnection con = (HttpURLConnection) createUrl.openConnection();
+    con.setRequestMethod("POST");
+    con.setRequestProperty(HttpHeaders.CONTENT_TYPE, ContentType.APPLICATION_JSON.getMimeType());
+    if (token != null)
+      con.setRequestProperty("Authorization", "Bearer " + token);
+
+    con.setDoOutput(true);
+    OutputStream os = con.getOutputStream();
+    os.write(json.getBytes(StandardCharsets.UTF_8));
+    os.flush();
+    os.close();
+
+    con.connect();
+    BufferedReader br2 = new BufferedReader(new InputStreamReader((InputStream) con.getContent(), StandardCharsets.UTF_8));
+    String result = br2.lines().collect(Collectors.joining("\n"));
+    int code = con.getResponseCode();
+    con.disconnect();
+    return new Pair<>(result, code);
+  }
+
+  private void createCollection(String collectionName) throws IOException {
+    assertEquals(200, get(baseUrl + "/admin/collections?action=CREATE&name=" + collectionName + "&numShards=2", jwtTestToken).second().intValue());
+    cluster.waitForActiveCollection(collectionName, 2, 2);
+  }
+
+  private void executeCommand(String url, HttpClient cl, String payload, JsonWebSignature jws) throws IOException, JoseException {
+    HttpPost httpPost;
+    HttpResponse r;
+    httpPost = new HttpPost(url);
+    if (jws != null)
+      setAuthorizationHeader(httpPost, "Bearer " + jws.getCompactSerialization());
+    httpPost.setEntity(new ByteArrayEntity(payload.getBytes(UTF_8)));
+    httpPost.addHeader("Content-Type", "application/json; charset=UTF-8");
+    r = cl.execute(httpPost);
+    String response = IOUtils.toString(r.getEntity().getContent(), StandardCharsets.UTF_8);
+    assertEquals("Non-200 response code. Response was " + response, 200, r.getStatusLine().getStatusCode());
+    assertFalse("Response contained errors: " + response, response.contains("errorMessages"));
+    Utils.consumeFully(r.getEntity());
+  }
+}
diff --git a/solr/core/src/test/org/apache/solr/security/JWTAuthPluginTest.java b/solr/core/src/test/org/apache/solr/security/JWTAuthPluginTest.java
new file mode 100644
index 0000000..72a908e
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/security/JWTAuthPluginTest.java
@@ -0,0 +1,406 @@
+/*
+ * 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.solr.security;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.security.Principal;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.Base64;
+import org.apache.solr.common.util.Utils;
+import org.jose4j.jwk.RsaJsonWebKey;
+import org.jose4j.jwk.RsaJwkGenerator;
+import org.jose4j.jws.AlgorithmIdentifiers;
+import org.jose4j.jws.JsonWebSignature;
+import org.jose4j.jwt.JwtClaims;
+import org.jose4j.keys.BigEndianBigInteger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mortbay.util.ajax.JSON;
+
+import static org.apache.solr.security.JWTAuthPlugin.JWTAuthenticationResponse.AuthCode.AUTZ_HEADER_PROBLEM;
+import static org.apache.solr.security.JWTAuthPlugin.JWTAuthenticationResponse.AuthCode.NO_AUTZ_HEADER;
+import static org.apache.solr.security.JWTAuthPlugin.JWTAuthenticationResponse.AuthCode.SCOPE_MISSING;
+
+public class JWTAuthPluginTest extends SolrTestCaseJ4 {
+  private static String testHeader;
+  private static String slimHeader;
+  private JWTAuthPlugin plugin;
+  private HashMap<String, Object> testJwk;
+  private static RsaJsonWebKey rsaJsonWebKey;
+  private HashMap<String, Object> testConfig;
+  private HashMap<String, Object> minimalConfig;
+
+
+  @BeforeClass
+  public static void beforeAll() throws Exception {
+    // Generate an RSA key pair, which will be used for signing and verification of the JWT, wrapped in a JWK
+    rsaJsonWebKey = RsaJwkGenerator.generateJwk(2048);
+    rsaJsonWebKey.setKeyId("k1");
+
+    JwtClaims claims = generateClaims();
+    JsonWebSignature jws = new JsonWebSignature();
+    jws.setPayload(claims.toJson());
+    jws.setKey(rsaJsonWebKey.getPrivateKey());
+    jws.setKeyIdHeaderValue(rsaJsonWebKey.getKeyId());
+    jws.setAlgorithmHeaderValue(AlgorithmIdentifiers.RSA_USING_SHA256);
+
+    String testJwt = jws.getCompactSerialization();
+    testHeader = "Bearer" + " " + testJwt;
+    
+    claims.unsetClaim("iss");
+    claims.unsetClaim("aud");
+    claims.unsetClaim("exp");
+    jws.setPayload(claims.toJson());
+    String slimJwt = jws.getCompactSerialization();
+    slimHeader = "Bearer" + " " + slimJwt;
+  }
+
+  static JwtClaims generateClaims() {
+    JwtClaims claims = new JwtClaims();
+    claims.setIssuer("IDServer");  // who creates the token and signs it
+    claims.setAudience("Solr"); // to whom the token is intended to be sent
+    claims.setExpirationTimeMinutesInTheFuture(10); // time when the token will expire (10 minutes from now)
+    claims.setGeneratedJwtId(); // a unique identifier for the token
+    claims.setIssuedAtToNow();  // when the token was issued/created (now)
+    claims.setNotBeforeMinutesInThePast(2); // time before which the token is not yet valid (2 minutes ago)
+    claims.setSubject("solruser"); // the subject/principal is whom the token is about
+    claims.setStringClaim("scope", "solr:read"); 
+    claims.setClaim("name", "Solr User"); // additional claims/attributes about the subject can be added
+    claims.setClaim("customPrincipal", "custom"); // additional claims/attributes about the subject can be added
+    claims.setClaim("claim1", "foo"); // additional claims/attributes about the subject can be added
+    claims.setClaim("claim2", "bar"); // additional claims/attributes about the subject can be added
+    claims.setClaim("claim3", "foo"); // additional claims/attributes about the subject can be added
+    List<String> groups = Arrays.asList("group-one", "other-group", "group-three");
+    claims.setStringListClaim("groups", groups); // multi-valued claims work too and will end up as a JSON array
+    return claims;
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    // Create an auth plugin
+    plugin = new JWTAuthPlugin();
+
+    // Create a JWK config for security.json
+    testJwk = new HashMap<>();
+    testJwk.put("kty", rsaJsonWebKey.getKeyType());
+    testJwk.put("e", BigEndianBigInteger.toBase64Url(rsaJsonWebKey.getRsaPublicKey().getPublicExponent()));
+    testJwk.put("use", rsaJsonWebKey.getUse());
+    testJwk.put("kid", rsaJsonWebKey.getKeyId());
+    testJwk.put("alg", rsaJsonWebKey.getAlgorithm());
+    testJwk.put("n", BigEndianBigInteger.toBase64Url(rsaJsonWebKey.getRsaPublicKey().getModulus()));
+
+    testConfig = new HashMap<>();
+    testConfig.put("class", "org.apache.solr.security.JWTAuthPlugin");
+    testConfig.put("jwk", testJwk);
+    plugin.init(testConfig);
+    
+    minimalConfig = new HashMap<>();
+    minimalConfig.put("class", "org.apache.solr.security.JWTAuthPlugin");
+  }
+
+  @Override
+  @After
+  public void tearDown() throws Exception {
+    super.tearDown();
+    plugin.close();
+  }
+
+  @Test
+  public void initWithoutRequired() {
+    plugin.init(testConfig);
+    assertEquals(AUTZ_HEADER_PROBLEM, plugin.authenticate("foo").getAuthCode());
+  }
+
+  @Test
+  public void initFromSecurityJSONLocalJWK() throws Exception {
+    Path securityJson = TEST_PATH().resolve("security").resolve("jwt_plugin_jwk_security.json");
+    InputStream is = Files.newInputStream(securityJson);
+    Map<String,Object> securityConf = (Map<String, Object>) Utils.fromJSON(is);
+    Map<String, Object> authConf = (Map<String, Object>) securityConf.get("authentication");
+    plugin.init(authConf);
+  }
+
+  @Test
+  public void initFromSecurityJSONUrlJwk() throws Exception {
+    Path securityJson = TEST_PATH().resolve("security").resolve("jwt_plugin_jwk_url_security.json");
+    InputStream is = Files.newInputStream(securityJson);
+    Map<String,Object> securityConf = (Map<String, Object>) Utils.fromJSON(is);
+    Map<String, Object> authConf = (Map<String, Object>) securityConf.get("authentication");
+    plugin.init(authConf);
+
+    JWTAuthPlugin.JWTAuthenticationResponse resp = plugin.authenticate(testHeader);
+    assertEquals(JWTAuthPlugin.JWTAuthenticationResponse.AuthCode.JWT_VALIDATION_EXCEPTION, resp.getAuthCode());
+    assertTrue(resp.getJwtException().getMessage().contains("Connection refused"));
+  }
+
+  @Test
+  public void initWithJwk() {
+    HashMap<String, Object> authConf = new HashMap<>();
+    authConf.put("jwk", testJwk);
+    plugin = new JWTAuthPlugin();
+    plugin.init(authConf);
+  }
+
+  @Test
+  public void initWithJwkUrl() {
+    HashMap<String, Object> authConf = new HashMap<>();
+    authConf.put("jwkUrl", "https://127.0.0.1:9999/foo.jwk");
+    plugin = new JWTAuthPlugin();
+    plugin.init(authConf);
+  }
+
+  @Test
+  public void parseJwkSet() throws Exception {
+    plugin.parseJwkSet(testJwk);
+
+    HashMap<String, Object> testJwks = new HashMap<>();
+    List<Map<String, Object>> keys = new ArrayList<>();
+    keys.add(testJwk);
+    testJwks.put("keys", keys);
+    plugin.parseJwkSet(testJwks);
+  }
+
+  @Test
+  public void authenticateOk() {
+    JWTAuthPlugin.JWTAuthenticationResponse resp = plugin.authenticate(testHeader);
+    assertTrue(resp.isAuthenticated());
+    assertEquals("solruser", resp.getPrincipal().getName());
+  }
+
+  @Test
+  public void authFailedMissingSubject() {
+    testConfig.put("iss", "NA");
+    plugin.init(testConfig);
+    JWTAuthPlugin.JWTAuthenticationResponse resp = plugin.authenticate(testHeader);
+    assertFalse(resp.isAuthenticated());
+    assertEquals(JWTAuthPlugin.JWTAuthenticationResponse.AuthCode.JWT_VALIDATION_EXCEPTION, resp.getAuthCode());
+
+    testConfig.put("iss", "IDServer");
+    plugin.init(testConfig);
+    resp = plugin.authenticate(testHeader);
+    assertTrue(resp.isAuthenticated());
+  }
+
+  @Test
+  public void authFailedMissingAudience() {
+    testConfig.put("aud", "NA");
+    plugin.init(testConfig);
+    JWTAuthPlugin.JWTAuthenticationResponse resp = plugin.authenticate(testHeader);
+    assertFalse(resp.isAuthenticated());
+    assertEquals(JWTAuthPlugin.JWTAuthenticationResponse.AuthCode.JWT_VALIDATION_EXCEPTION, resp.getAuthCode());
+
+    testConfig.put("aud", "Solr");
+    plugin.init(testConfig);
+    resp = plugin.authenticate(testHeader);
+    assertTrue(resp.isAuthenticated());
+  }
+
+  @Test
+  public void authFailedMissingPrincipal() {
+    testConfig.put("principalClaim", "customPrincipal");
+    plugin.init(testConfig);
+    JWTAuthPlugin.JWTAuthenticationResponse resp = plugin.authenticate(testHeader);
+    assertTrue(resp.isAuthenticated());
+
+    testConfig.put("principalClaim", "NA");
+    plugin.init(testConfig);
+    resp = plugin.authenticate(testHeader);
+    assertFalse(resp.isAuthenticated());
+    assertEquals(JWTAuthPlugin.JWTAuthenticationResponse.AuthCode.PRINCIPAL_MISSING, resp.getAuthCode());
+  }
+
+  @Test
+  public void claimMatch() {
+    // all custom claims match regex
+    Map<String, String> shouldMatch = new HashMap<>();
+    shouldMatch.put("claim1", "foo");
+    shouldMatch.put("claim2", "foo|bar");
+    shouldMatch.put("claim3", "f\\w{2}$");
+    testConfig.put("claimsMatch", shouldMatch);
+    plugin.init(testConfig);
+    JWTAuthPlugin.JWTAuthenticationResponse resp = plugin.authenticate(testHeader);
+    assertTrue(resp.isAuthenticated());
+
+    // Required claim does not exist
+    shouldMatch.clear();
+    shouldMatch.put("claim9", "NA");
+    plugin.init(testConfig);
+    resp = plugin.authenticate(testHeader);
+    assertEquals(JWTAuthPlugin.JWTAuthenticationResponse.AuthCode.CLAIM_MISMATCH, resp.getAuthCode());
+
+    // Required claim does not match regex
+    shouldMatch.clear();
+    shouldMatch.put("claim1", "NA");
+    resp = plugin.authenticate(testHeader);
+    assertEquals(JWTAuthPlugin.JWTAuthenticationResponse.AuthCode.CLAIM_MISMATCH, resp.getAuthCode());
+  }
+
+  @Test
+  public void missingIssAudExp() {
+    testConfig.put("requireExp", "false");
+    testConfig.put("requireSub", "false");
+    plugin.init(testConfig);
+    JWTAuthPlugin.JWTAuthenticationResponse resp = plugin.authenticate(slimHeader);
+    assertTrue(resp.isAuthenticated());
+
+    // Missing exp header
+    testConfig.put("requireExp", true);
+    plugin.init(testConfig);
+    resp = plugin.authenticate(slimHeader);
+    assertEquals(JWTAuthPlugin.JWTAuthenticationResponse.AuthCode.JWT_VALIDATION_EXCEPTION, resp.getAuthCode());
+
+    // Missing sub header
+    testConfig.put("requireSub", true);
+    plugin.init(testConfig);
+    resp = plugin.authenticate(slimHeader);
+    assertEquals(JWTAuthPlugin.JWTAuthenticationResponse.AuthCode.JWT_VALIDATION_EXCEPTION, resp.getAuthCode());
+  }
+
+  @Test
+  public void algWhitelist() {
+    testConfig.put("algWhitelist", Arrays.asList("PS384", "PS512"));
+    plugin.init(testConfig);
+    JWTAuthPlugin.JWTAuthenticationResponse resp = plugin.authenticate(testHeader);
+    assertEquals(JWTAuthPlugin.JWTAuthenticationResponse.AuthCode.JWT_VALIDATION_EXCEPTION, resp.getAuthCode());
+    assertTrue(resp.getErrorMessage().contains("not a whitelisted"));
+  }
+
+  @Test
+  public void scope() {
+    testConfig.put("scope", "solr:read solr:admin");
+    plugin.init(testConfig);
+    JWTAuthPlugin.JWTAuthenticationResponse resp = plugin.authenticate(testHeader);
+    assertTrue(resp.isAuthenticated());
+
+    Principal principal = resp.getPrincipal();
+    assertTrue(principal instanceof VerifiedUserRoles);
+    Set<String> roles = ((VerifiedUserRoles)principal).getVerifiedRoles();
+    assertEquals(1, roles.size());
+    assertTrue(roles.contains("solr:read"));
+  }
+
+  @Test
+  public void wrongScope() {
+    testConfig.put("scope", "wrong");
+    plugin.init(testConfig);
+    JWTAuthPlugin.JWTAuthenticationResponse resp = plugin.authenticate(testHeader);
+    assertFalse(resp.isAuthenticated());
+    assertNull(resp.getPrincipal());
+    assertEquals(SCOPE_MISSING, resp.getAuthCode());
+  }
+  
+  @Test
+  public void noHeaderBlockUnknown() {
+    testConfig.put("blockUnknown", true);
+    plugin.init(testConfig);
+    JWTAuthPlugin.JWTAuthenticationResponse resp = plugin.authenticate(null);
+    assertEquals(NO_AUTZ_HEADER, resp.getAuthCode());
+  }
+
+  @Test
+  public void noHeaderNotBlockUnknown() {
+    testConfig.put("blockUnknown", false);
+    plugin.init(testConfig);
+    JWTAuthPlugin.JWTAuthenticationResponse resp = plugin.authenticate(null);
+    assertEquals(JWTAuthPlugin.JWTAuthenticationResponse.AuthCode.PASS_THROUGH, resp.getAuthCode());
+  }
+
+  @Test
+  public void minimalConfigPassThrough() {
+    testConfig.put("blockUnknown", false);
+    plugin.init(testConfig);
+    JWTAuthPlugin.JWTAuthenticationResponse resp = plugin.authenticate(null);
+    assertEquals(JWTAuthPlugin.JWTAuthenticationResponse.AuthCode.PASS_THROUGH, resp.getAuthCode());
+  }
+  
+  @Test
+  public void wellKnownConfig() throws IOException {
+    String wellKnownUrl = TEST_PATH().resolve("security").resolve("jwt_well-known-config.json").toAbsolutePath().toUri().toString();
+    testConfig.put("wellKnownUrl", wellKnownUrl);
+    testConfig.remove("jwk");
+    plugin.init(testConfig);
+    JWTAuthPlugin.JWTAuthenticationResponse resp = plugin.authenticate(null);
+    assertEquals(JWTAuthPlugin.JWTAuthenticationResponse.AuthCode.PASS_THROUGH, resp.getAuthCode());
+  }
+
+  @Test(expected = SolrException.class)
+  public void onlyOneJwkConfig() throws IOException {
+    testConfig.put("jwkUrl", "http://127.0.0.1:45678/.well-known/config");
+    plugin.init(testConfig);
+  }
+
+  @Test(expected = SolrException.class)
+  public void wellKnownConfigNotHttps() throws IOException {
+    testConfig.put("wellKnownUrl", "http://127.0.0.1:45678/.well-known/config");
+    plugin.init(testConfig);
+  }
+
+  @Test(expected = SolrException.class)
+  public void wellKnownConfigNotReachable() {
+    testConfig.put("wellKnownUrl", "https://127.0.0.1:45678/.well-known/config");
+    plugin.init(testConfig);
+  }
+  
+  @Test
+  public void wellKnownConfigFromInputstream() throws IOException {
+    Path configJson = TEST_PATH().resolve("security").resolve("jwt_well-known-config.json");
+    JWTAuthPlugin.WellKnownDiscoveryConfig config = JWTAuthPlugin.WellKnownDiscoveryConfig.parse(Files.newInputStream(configJson));
+    assertEquals("https://acmepaymentscorp/oauth/jwks", config.getJwksUrl());
+  }
+
+  @Test
+  public void wellKnownConfigFromString() throws IOException {
+    Path configJson = TEST_PATH().resolve("security").resolve("jwt_well-known-config.json");
+    String configString = StringUtils.join(Files.readAllLines(configJson), "\n");
+    JWTAuthPlugin.WellKnownDiscoveryConfig config = JWTAuthPlugin.WellKnownDiscoveryConfig.parse(configString, StandardCharsets.UTF_8);
+    assertEquals("https://acmepaymentscorp/oauth/jwks", config.getJwksUrl());
+    assertEquals("http://acmepaymentscorp", config.getIssuer());
+    assertEquals("http://acmepaymentscorp/oauth/auz/authorize", config.getAuthorizationEndpoint());
+    assertEquals(Arrays.asList("READ", "WRITE", "DELETE", "openid", "scope", "profile", "email", "address", "phone"), config.getScopesSupported());
+    assertEquals(Arrays.asList("code", "code id_token", "code token", "code id_token token", "token", "id_token", "id_token token"), config.getResponseTypesSupported());
+  }
+
+  @Test
+  public void xSolrAuthDataHeader() {
+    testConfig.put("adminUiScope", "solr:admin");
+    testConfig.put("authorizationEndpoint", "http://acmepaymentscorp/oauth/auz/authorize");
+    testConfig.put("clientId", "solr-cluster");
+    plugin.init(testConfig);
+    String headerBase64 = plugin.generateAuthDataHeader();
+    String headerJson = new String(Base64.base64ToByteArray(headerBase64), StandardCharsets.UTF_8);
+    Map<String,String> parsed = (Map<String, String>) JSON.parse(headerJson);
+    assertEquals("solr:admin", parsed.get("scope"));
+    assertEquals("http://acmepaymentscorp/oauth/auz/authorize", parsed.get("authorizationEndpoint"));
+    assertEquals("solr-cluster", parsed.get("client_id"));
+  }
+}
\ No newline at end of file
diff --git a/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java b/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
index eb25b83..bff9766 100644
--- a/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/security/PKIAuthenticationIntegrationTest.java
@@ -37,7 +37,6 @@ import org.slf4j.LoggerFactory;
 
 import static java.util.Collections.singletonMap;
 import static org.apache.solr.common.util.Utils.makeMap;
-import static org.apache.solr.security.TestAuthorizationFramework.verifySecurityStatus;
 
 public class PKIAuthenticationIntegrationTest extends SolrCloudAuthTestCase {
 
diff --git a/solr/licenses/jose4j-0.6.4.jar.sha1 b/solr/licenses/jose4j-0.6.4.jar.sha1
new file mode 100644
index 0000000..d4a446e
--- /dev/null
+++ b/solr/licenses/jose4j-0.6.4.jar.sha1
@@ -0,0 +1 @@
+0ee27e0fa2e82f1cce75c70861190730ff174e49
diff --git a/solr/licenses/jose4j-LICENSE-ASL.txt b/solr/licenses/jose4j-LICENSE-ASL.txt
new file mode 100644
index 0000000..ab3182e
--- /dev/null
+++ b/solr/licenses/jose4j-LICENSE-ASL.txt
@@ -0,0 +1,272 @@
+/*
+ *                                 Apache License
+ *                           Version 2.0, January 2004
+ *                        http://www.apache.org/licenses/
+ *
+ *   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+ *
+ *   1. Definitions.
+ *
+ *      "License" shall mean the terms and conditions for use, reproduction,
+ *      and distribution as defined by Sections 1 through 9 of this document.
+ *
+ *      "Licensor" shall mean the copyright owner or entity authorized by
+ *      the copyright owner that is granting the License.
+ *
+ *      "Legal Entity" shall mean the union of the acting entity and all
+ *      other entities that control, are controlled by, or are under common
+ *      control with that entity. For the purposes of this definition,
+ *      "control" means (i) the power, direct or indirect, to cause the
+ *      direction or management of such entity, whether by contract or
+ *      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ *      outstanding shares, or (iii) beneficial ownership of such entity.
+ *
+ *      "You" (or "Your") shall mean an individual or Legal Entity
+ *      exercising permissions granted by this License.
+ *
+ *      "Source" form shall mean the preferred form for making modifications,
+ *      including but not limited to software source code, documentation
+ *      source, and configuration files.
+ *
+ *      "Object" form shall mean any form resulting from mechanical
+ *      transformation or translation of a Source form, including but
+ *      not limited to compiled object code, generated documentation,
+ *      and conversions to other media types.
+ *
+ *      "Work" shall mean the work of authorship, whether in Source or
+ *      Object form, made available under the License, as indicated by a
+ *      copyright notice that is included in or attached to the work
+ *      (an example is provided in the Appendix below).
+ *
+ *      "Derivative Works" shall mean any work, whether in Source or Object
+ *      form, that is based on (or derived from) the Work and for which the
+ *      editorial revisions, annotations, elaborations, or other modifications
+ *      represent, as a whole, an original work of authorship. For the purposes
+ *      of this License, Derivative Works shall not include works that remain
+ *      separable from, or merely link (or bind by name) to the interfaces of,
+ *      the Work and Derivative Works thereof.
+ *
+ *      "Contribution" shall mean any work of authorship, including
+ *      the original version of the Work and any modifications or additions
+ *      to that Work or Derivative Works thereof, that is intentionally
+ *      submitted to Licensor for inclusion in the Work by the copyright owner
+ *      or by an individual or Legal Entity authorized to submit on behalf of
+ *      the copyright owner. For the purposes of this definition, "submitted"
+ *      means any form of electronic, verbal, or written communication sent
+ *      to the Licensor or its representatives, including but not limited to
+ *      communication on electronic mailing lists, source code control systems,
+ *      and issue tracking systems that are managed by, or on behalf of, the
+ *      Licensor for the purpose of discussing and improving the Work, but
+ *      excluding communication that is conspicuously marked or otherwise
+ *      designated in writing by the copyright owner as "Not a Contribution."
+ *
+ *      "Contributor" shall mean Licensor and any individual or Legal Entity
+ *      on behalf of whom a Contribution has been received by Licensor and
+ *      subsequently incorporated within the Work.
+ *
+ *   2. Grant of Copyright License. Subject to the terms and conditions of
+ *      this License, each Contributor hereby grants to You a perpetual,
+ *      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ *      copyright license to reproduce, prepare Derivative Works of,
+ *      publicly display, publicly perform, sublicense, and distribute the
+ *      Work and such Derivative Works in Source or Object form.
+ *
+ *   3. Grant of Patent License. Subject to the terms and conditions of
+ *      this License, each Contributor hereby grants to You a perpetual,
+ *      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ *      (except as stated in this section) patent license to make, have made,
+ *      use, offer to sell, sell, import, and otherwise transfer the Work,
+ *      where such license applies only to those patent claims licensable
+ *      by such Contributor that are necessarily infringed by their
+ *      Contribution(s) alone or by combination of their Contribution(s)
+ *      with the Work to which such Contribution(s) was submitted. If You
+ *      institute patent litigation against any entity (including a
+ *      cross-claim or counterclaim in a lawsuit) alleging that the Work
+ *      or a Contribution incorporated within the Work constitutes direct
+ *      or contributory patent infringement, then any patent licenses
+ *      granted to You under this License for that Work shall terminate
+ *      as of the date such litigation is filed.
+ *
+ *   4. Redistribution. You may reproduce and distribute copies of the
+ *      Work or Derivative Works thereof in any medium, with or without
+ *      modifications, and in Source or Object form, provided that You
+ *      meet the following conditions:
+ *
+ *      (a) You must give any other recipients of the Work or
+ *          Derivative Works a copy of this License; and
+ *
+ *      (b) You must cause any modified files to carry prominent notices
+ *          stating that You changed the files; and
+ *
+ *      (c) You must retain, in the Source form of any Derivative Works
+ *          that You distribute, all copyright, patent, trademark, and
+ *          attribution notices from the Source form of the Work,
+ *          excluding those notices that do not pertain to any part of
+ *          the Derivative Works; and
+ *
+ *      (d) If the Work includes a "NOTICE" text file as part of its
+ *          distribution, then any Derivative Works that You distribute must
+ *          include a readable copy of the attribution notices contained
+ *          within such NOTICE file, excluding those notices that do not
+ *          pertain to any part of the Derivative Works, in at least one
+ *          of the following places: within a NOTICE text file distributed
+ *          as part of the Derivative Works; within the Source form or
+ *          documentation, if provided along with the Derivative Works; or,
+ *          within a display generated by the Derivative Works, if and
+ *          wherever such third-party notices normally appear. The contents
+ *          of the NOTICE file are for informational purposes only and
+ *          do not modify the License. You may add Your own attribution
+ *          notices within Derivative Works that You distribute, alongside
+ *          or as an addendum to the NOTICE text from the Work, provided
+ *          that such additional attribution notices cannot be construed
+ *          as modifying the License.
+ *
+ *      You may add Your own copyright statement to Your modifications and
+ *      may provide additional or different license terms and conditions
+ *      for use, reproduction, or distribution of Your modifications, or
+ *      for any such Derivative Works as a whole, provided Your use,
+ *      reproduction, and distribution of the Work otherwise complies with
+ *      the conditions stated in this License.
+ *
+ *   5. Submission of Contributions. Unless You explicitly state otherwise,
+ *      any Contribution intentionally submitted for inclusion in the Work
+ *      by You to the Licensor shall be under the terms and conditions of
+ *      this License, without any additional terms or conditions.
+ *      Notwithstanding the above, nothing herein shall supersede or modify
+ *      the terms of any separate license agreement you may have executed
+ *      with Licensor regarding such Contributions.
+ *
+ *   6. Trademarks. This License does not grant permission to use the trade
+ *      names, trademarks, service marks, or product names of the Licensor,
+ *      except as required for reasonable and customary use in describing the
+ *      origin of the Work and reproducing the content of the NOTICE file.
+ *
+ *   7. Disclaimer of Warranty. Unless required by applicable law or
+ *      agreed to in writing, Licensor provides the Work (and each
+ *      Contributor provides its Contributions) on an "AS IS" BASIS,
+ *      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ *      implied, including, without limitation, any warranties or conditions
+ *      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ *      PARTICULAR PURPOSE. You are solely responsible for determining the
+ *      appropriateness of using or redistributing the Work and assume any
+ *      risks associated with Your exercise of permissions under this License.
+ *
+ *   8. Limitation of Liability. In no event and under no legal theory,
+ *      whether in tort (including negligence), contract, or otherwise,
+ *      unless required by applicable law (such as deliberate and grossly
+ *      negligent acts) or agreed to in writing, shall any Contributor be
+ *      liable to You for damages, including any direct, indirect, special,
+ *      incidental, or consequential damages of any character arising as a
+ *      result of this License or out of the use or inability to use the
+ *      Work (including but not limited to damages for loss of goodwill,
+ *      work stoppage, computer failure or malfunction, or any and all
+ *      other commercial damages or losses), even if such Contributor
+ *      has been advised of the possibility of such damages.
+ *
+ *   9. Accepting Warranty or Additional Liability. While redistributing
+ *      the Work or Derivative Works thereof, You may choose to offer,
+ *      and charge a fee for, acceptance of support, warranty, indemnity,
+ *      or other liability obligations and/or rights consistent with this
+ *      License. However, in accepting such obligations, You may act only
+ *      on Your own behalf and on Your sole responsibility, not on behalf
+ *      of any other Contributor, and only if You agree to indemnify,
+ *      defend, and hold each Contributor harmless for any liability
+ *      incurred by, or claims asserted against, such Contributor by reason
+ *      of your accepting any such warranty or additional liability.
+ *
+ *   END OF TERMS AND CONDITIONS
+ *
+ *   APPENDIX: How to apply the Apache License to your work.
+ *
+ *      To apply the Apache License to your work, attach the following
+ *      boilerplate notice, with the fields enclosed by brackets "[]"
+ *      replaced with your own identifying information. (Don't include
+ *      the brackets!)  The text should be enclosed in the appropriate
+ *      comment syntax for the file format. We also recommend that a
+ *      file or class name and description of purpose be included on the
+ *      same "printed page" as the copyright notice for easier
+ *      identification within third-party archives.
+ *
+ *   Copyright [yyyy] [name of copyright owner]
+ *
+ *   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.
+ */
+
+W3C® SOFTWARE NOTICE AND LICENSE
+http://www.w3.org/Consortium/Legal/2002/copyright-software-20021231
+
+This work (and included software, documentation such as READMEs, or other
+related items) is being provided by the copyright holders under the following
+license. By obtaining, using and/or copying this work, you (the licensee) agree
+that you have read, understood, and will comply with the following terms and
+conditions.
+
+Permission to copy, modify, and distribute this software and its documentation,
+with or without modification, for any purpose and without fee or royalty is
+hereby granted, provided that you include the following on ALL copies of the
+software and documentation or portions thereof, including modifications:
+
+  1. The full text of this NOTICE in a location viewable to users of the
+     redistributed or derivative work. 
+  2. Any pre-existing intellectual property disclaimers, notices, or terms
+     and conditions. If none exist, the W3C Software Short Notice should be
+     included (hypertext is preferred, text is permitted) within the body
+     of any redistributed or derivative code.
+  3. Notice of any changes or modifications to the files, including the date
+     changes were made. (We recommend you provide URIs to the location from
+     which the code is derived.)
+     
+THIS SOFTWARE AND DOCUMENTATION IS PROVIDED "AS IS," AND COPYRIGHT HOLDERS MAKE
+NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED
+TO, WARRANTIES OF MERCHANTABILITY OR FITNESS FOR ANY PARTICULAR PURPOSE OR THAT
+THE USE OF THE SOFTWARE OR DOCUMENTATION WILL NOT INFRINGE ANY THIRD PARTY
+PATENTS, COPYRIGHTS, TRADEMARKS OR OTHER RIGHTS.
+
+COPYRIGHT HOLDERS WILL NOT BE LIABLE FOR ANY DIRECT, INDIRECT, SPECIAL OR
+CONSEQUENTIAL DAMAGES ARISING OUT OF ANY USE OF THE SOFTWARE OR DOCUMENTATION.
+
+The name and trademarks of copyright holders may NOT be used in advertising or
+publicity pertaining to the software without specific, written prior permission.
+Title to copyright in this software and any associated documentation will at
+all times remain with copyright holders.
+
+____________________________________
+
+This formulation of W3C's notice and license became active on December 31 2002.
+This version removes the copyright ownership notice such that this license can
+be used with materials other than those owned by the W3C, reflects that ERCIM
+is now a host of the W3C, includes references to this specific dated version of
+the license, and removes the ambiguous grant of "use". Otherwise, this version
+is the same as the previous version and is written so as to preserve the Free
+Software Foundation's assessment of GPL compatibility and OSI's certification
+under the Open Source Definition. Please see our Copyright FAQ for common
+questions about using materials from our site, including specific terms and
+conditions for packages like libwww, Amaya, and Jigsaw. Other questions about
+this notice can be directed to site-policy@w3.org.
+ 
+Joseph Reagle <si...@w3.org> 
+
+This license came from: http://www.megginson.com/SAX/copying.html
+  However please note future versions of SAX may be covered 
+  under http://saxproject.org/?selected=pd
+
+SAX2 is Free!
+
+I hereby abandon any property rights to SAX 2.0 (the Simple API for
+XML), and release all of the SAX 2.0 source code, compiled code, and
+documentation contained in this distribution into the Public Domain.
+SAX comes with NO WARRANTY or guarantee of fitness for any
+purpose.
+
+David Megginson, david@megginson.com
+2000-05-05
diff --git a/solr/licenses/jose4j-NOTICE.txt b/solr/licenses/jose4j-NOTICE.txt
new file mode 100644
index 0000000..f68819e
--- /dev/null
+++ b/solr/licenses/jose4j-NOTICE.txt
@@ -0,0 +1,13 @@
+jose4j
+Copyright 2012-2015 Brian Campbell
+
+EcdsaUsingShaAlgorithm contains code for converting the concatenated
+R & S values of the signature to and from DER, which was originally
+derived from the Apache Santuario XML Security library's SignatureECDSA
+implementation. http://santuario.apache.org/
+
+The Base64 implementation in this software was derived from the
+Apache Commons Codec project. http://commons.apache.org/proper/commons-codec/
+
+JSON processing in this software was derived from the JSON.simple toolkit.
+https://code.google.com/p/json-simple/
\ No newline at end of file
diff --git a/solr/solr-ref-guide/src/authentication-and-authorization-plugins.adoc b/solr/solr-ref-guide/src/authentication-and-authorization-plugins.adoc
index 3ed3951..ff21ca4 100644
--- a/solr/solr-ref-guide/src/authentication-and-authorization-plugins.adoc
+++ b/solr/solr-ref-guide/src/authentication-and-authorization-plugins.adoc
@@ -1,5 +1,5 @@
 = Authentication and Authorization Plugins
-:page-children: basic-authentication-plugin, hadoop-authentication-plugin, kerberos-authentication-plugin, rule-based-authorization-plugin
+:page-children: basic-authentication-plugin, hadoop-authentication-plugin, kerberos-authentication-plugin, rule-based-authorization-plugin, jwt-authentication-plugin
 // 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
@@ -125,6 +125,7 @@ Solr has the following implementations of authentication plugins:
 * <<kerberos-authentication-plugin.adoc#kerberos-authentication-plugin,Kerberos Authentication Plugin>>
 * <<basic-authentication-plugin.adoc#basic-authentication-plugin,Basic Authentication Plugin>>
 * <<hadoop-authentication-plugin.adoc#hadoop-authentication-plugin,Hadoop Authentication Plugin>>
+* <<jwt-authentication-plugin.adoc#jwt-authentication-plugin,JWT Authentication Plugin>>
 
 == Authorization
 
@@ -163,7 +164,8 @@ Whenever an authentication plugin is enabled, authentication is also required fo
 
 When authentication is required the Admin UI will presented you with a login dialogue. The authentication plugins currently supported by the Admin UI are:
 
-* `BasicAuthPlugin`
+* <<basic-authentication-plugin.adoc#basic-authentication-plugin,Basic Authentication Plugin>>
+* <<jwt-authentication-plugin.adoc#jwt-authentication-plugin,JWT Authentication Plugin>>
  
 If your plugin of choice is not supported, the Admin UI will still let you perform unrestricted operations, while for restricted operations you will need to interact with Solr by sending HTTP requests instead of through the graphical user interface of the Admin UI. All operations supported by Admin UI can be performed through Solr's RESTful APIs.
 
diff --git a/solr/solr-ref-guide/src/basic-authentication-plugin.adoc b/solr/solr-ref-guide/src/basic-authentication-plugin.adoc
index f6c872e..e3cd24f 100644
--- a/solr/solr-ref-guide/src/basic-authentication-plugin.adoc
+++ b/solr/solr-ref-guide/src/basic-authentication-plugin.adoc
@@ -81,7 +81,7 @@ There are a few things to keep in mind when using the Basic authentication plugi
 * A user who has access to write permissions to `security.json` will be able to modify all the permissions and how users have been assigned permissions. Special care should be taken to only grant access to editing security to appropriate users.
 * Your network should, of course, be secure. Even with Basic authentication enabled, you should not unnecessarily expose Solr to the outside world.
 
-== Editing Authentication Plugin Configuration
+== Editing Basic Authentication Plugin Configuration
 
 An Authentication API allows modifying user IDs and passwords. The API provides an endpoint with specific commands to set user details or delete a user.
 
diff --git a/solr/solr-ref-guide/src/jwt-authentication-plugin.adoc b/solr/solr-ref-guide/src/jwt-authentication-plugin.adoc
new file mode 100644
index 0000000..72f410e
--- /dev/null
+++ b/solr/solr-ref-guide/src/jwt-authentication-plugin.adoc
@@ -0,0 +1,194 @@
+= JWT Authentication Plugin
+// 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.
+
+Solr can support https://en.wikipedia.org/wiki/JSON_Web_Token[JSON Web Token] (JWT) based Bearer authentication with the use of the JWTAuthPlugin. This allows Solr to assert that a user is already authenticated with an external https://en.wikipedia.org/wiki/Identity_provider[Identity Provider] by validating that the JWT formatted https://en.wikipedia.org/wiki/Access_token[access token] is digitally signed by the Identity Provider. The typical use case is to integrate Solr with an https:/ [...]
+
+== Enable JWT Authentication
+
+To use JWT Bearer authentication, the `security.json` file must have an `authentication` part which defines the class being used for authentication along with configuration parameters.
+
+The simplest possible `security.json` for registering the plugin without configuration is:
+
+[source,json]
+----
+{
+  "authentication": {
+    "class":"solr.JWTAuthPlugin"
+  }
+}
+----
+
+The plugin will NOT block anonymous traffic in this mode, since the default for `blockUnknown` is false. It is then possible to start configuring the plugin using REST API calls, which is described below.
+
+== Configuration parameters
+
+[%header,format=csv,separator=;]
+|===
+Key                  ; Description                                             ; Default
+blockUnknown         ; Set to `true` in order to block requests from users without a token  ; `false`
+wellKnownUrl         ; URL to an https://openid.net/specs/openid-connect-discovery-1_0.html[OpenID Connect Discovery] endpoint ; (no default)
+clientId             ; Client identifier for use with OpenID Connect           ; (no default value) Required to authenticate with Admin UI
+realm                ; Name of the authentication realm to echo back in HTTP 401 responses. Will also be displayed in Admin UI login page ; 'solr-jwt'
+scope                ; Whitespace separated list of valid scopes. If configured, the JWT access token MUST contain a `scope` claim with at least one of the listed scopes. Example: `solr:read solr:admin` ;
+jwkUrl               ; An https URL to a https://tools.ietf.org/html/rfc7517[JWK] keys file. ; Auto configured if `wellKnownUrl` is provided
+jwk                  ; As an alternative to `jwkUrl` you may provide a JSON object here containing the public key(s) of the issuer.  ;
+iss                  ; Validates that the `iss` (issuer) claim equals this string        ; Auto configured if `wellKnownUrl` is provided
+aud                  ; Validates that the `aud` (audience) claim equals this string      ; If `clientId` is configured, require `aud` to match it
+requireSub           ; Makes `sub` (subject) claim mandatory                         ; `true`
+requireExp           ; Makes `exp` (expiry time) claim mandatory                     ; `true`
+algWhitelist         ; JSON array with algorithms to accept: `HS256`, `HS384`, `HS512`, `RS256`, `RS384`, `RS512`, `ES256`, `ES384`, `ES512`, `PS256`, `PS384`, `PS512`, `none  ; Default is to allow all algorithms 
+jwkCacheDur          ; Duration of JWK cache in seconds                        ; `3600` (1 hour)
+principalClaim       ; What claim id to pull principal from                    ; `sub`
+claimsMatch          ; JSON object of claims (key) that must match a regular expression (value). Example: `{ "foo" : "A|B" }` will require the `foo` claim to be either "A" or "B". ; (none)
+adminUiScope         ; Define what scope is requested when logging in from Admin UI ; If not defined, the first scope from `scope` parameter is used
+authorizationEndpoint; The URL for the Id Provider's authorization endpoint ; Auto configured if `wellKnownUrl` is provided 
+redirectUris         ; Valid location(s) for redirect after external authentication. Takes a string or array of strings. Must be the base URL of Solr, e.g. https://solr1.example.com:8983/solr/ and must match the list of redirect URIs registered with the Identity Provider beforehand. ; Defaults to empty list, i.e. any node is assumed to be a valid redirect target.
+|===
+
+== More configuration examples
+=== With JWK URL
+To start enforcing authentication for all users, requiring a valid JWT in the `Authorization` header, you need to configure the plugin with one or more https://tools.ietf.org/html/rfc7517[JSON Web Key]s (JWK). This is a JSON document containing the key used to sign/encrypt the JWT. It could be a symmetric or asymmetric key. The JWK can either be fetched (and cached) from an external HTTPS endpoint or specified directly in `security.json`. Below is an example of the former:
+
+[source,json]
+----
+{
+  "authentication": {
+    "class": "solr.JWTAuthPlugin",
+    "blockUnknown": true,
+    "jwkUrl": "https://my.key.server/jwk.json"
+  }
+}
+----
+
+=== With Admin UI support   
+The next example shows configuring using https://openid.net/specs/openid-connect-discovery-1_0.html[OpenID Connect Discovery] with a well-known URI for automatic configuration of many common settings, including ability to use the Admin UI with an OpenID Connect enabled Identity Provider. 
+
+[source,json]
+----
+{
+  "authentication": {
+    "class": "solr.JWTAuthPlugin",
+    "blockUnknown": true,
+    "wellKnownUrl": "https://idp.example.com/.well-known/openid-configuration",
+    "clientId": "xyz",
+    "redirectUri": "https://my.solr.server:8983/solr/"
+  }
+}
+---- 
+
+In this case, `jwkUrl`, `iss` and `authorizationEndpoint` will be automatically configured from the fetched configuration.
+
+=== Complex example
+Let's look at a more complex configuration, this time with a static embedded JWK:
+
+[source,json]
+----
+{
+  "authentication": {
+    "class": "solr.JWTAuthPlugin", <1>
+    "blockUnknown": true, <2>
+    "jwk": { <3>
+      "e": "AQAB",
+      "kid": "k1",
+      "kty": "RSA",
+      "n": "3ZF6wBGPMsLzsS1KLghxaVpZtXD3nTLzDm0c974i9-KNU_1rhhBeiVfS64VfEQmP8SA470jEy7yWcvnz9GvG-YAlm9iOwVF7jLl2awdws0ocFjdSPT3SjPQKzOeMO7G9XqNTkrvoFCn1YAi26fbhhcqkwZDoeTcHQdRN32frzccuPhZrwImApIedroKLlKWv2IvPDnz2Bpe2WWVc2HdoWYqEVD3p_BEy8f-RTSHK3_8kDDF9yAwI9jx7CK1_C-eYxXltm-6rpS5NGyFm0UNTZMxVU28Tl7LX8Vb6CikyCQ9YRCtk_CvpKWmEuKEp9I28KHQNmGkDYT90nt3vjbCXxw"
+    },
+    "clientId": "solr-client-12345", <4>
+    "iss": "https://example.com/idp", <5>
+    "aud": "https://example.com/solr", <6>
+    "principalClaim": "solruid", <7>
+    "claimsMatch": { "foo" : "A|B", "dept" : "IT" }, <8>
+    "scope": "solr:read solr:write solr:admin", <9>
+    "algWhitelist" : [ "RS256", "RS384", "RS512" ] <10>
+  }
+}
+----
+
+Let's comment on this config:
+
+<1> Plugin class
+<2> Make sure to block anyone without a valid token
+<3> Here we pass the JWK inline instead of referring to a URL with `jwkUrl`
+<4> Set the client id registered with Identity Provider
+<5> The issuer claim must match "https://example.com/idp"
+<6> The audience claim must match "https://example.com/solr"
+<7> Fetch the user id from another claim than the default `sub`
+<8> Require that the `roles` claim is one of "A" or "B" and that the `dept` claim is "IT"
+<9> Require one of the scopes `solr:read`, `solr:write` or `solr:admin`
+<10> Only accept RSA algorithms for signatures
+
+
+== Editing JWT Authentication Plugin Configuration
+
+All properties mentioned above can be set or changed using the Config Edit API. You can thus start with a simple configuration with only `class` configured and then configure the rest using the API.
+
+=== Set a config Property
+
+Set properties for the authentication plugin. Each of the configuration keys in the table above can be used as parameter keys for the `set-property` command.
+
+Example:
+
+[.dynamic-tabs]
+--
+[example.tab-pane#jwt-v1set-property]
+====
+[.tab-label]*V1 API*
+
+[source,bash]
+----
+curl http://localhost:8983/solr/admin/authentication -H 'Content-type:application/json' -H 'Authorization: Bearer xxx.yyy.zzz' -d '{"set-property": {"blockUnknown":true, "wellKnownUrl": "https://example.com/.well-knwon/openid-configuration", "scope": "solr:read solr:write"}}'
+----
+====
+
+[example.tab-pane#jwt-v2set-property]
+====
+[.tab-label]*V2 API*
+
+[source,bash]
+----
+curl http://localhost:8983/api/cluster/security/authentication -H 'Content-type:application/json' -H 'Authorization: Bearer xxx.yyy.zzz' -d -d '{"set-property": {"blockUnknown":true, "wellKnownUrl": "https://example.com/.well-knwon/openid-configuration", "scope": "solr:read solr:write"}}'
+----
+====
+--
+
+Insert a valid JWT access token in compact serialization format (`xxx.yyy.zzz` above) to authenticate with Solr once the plugin is active.
+
+== Using clients with JWT Auth 
+
+[#jwt-soljr]
+=== SolrJ
+
+SolrJ does not currently support supplying JWT tokens per request.
+
+[#jwt-curl]
+=== cURL
+
+To authenticate with Solr when using the cURL utility, supply a valid JWT access token in an `Authorization` header, as follows (replace xxxxxx.xxxxxx.xxxxxx with your JWT compact token):
+
+[source,bash]
+----
+curl -H "Authorization: Bearer xxxxxx.xxxxxx.xxxxxx" http://localhost:8983/solr/admin/info/system 
+----
+
+=== Admin UI 
+
+When this plugin is enabled, users will be redirected to a login page in the Admin UI once they attempt to do a restricted action. The page has a button that users will click and be redirected to the Identity Provider's login page. Once authenticated, the user will be redirected back to Solr Admin UI to the last known location. The session will last as long as the JWT token expiry time and is valid for one Solr server only. That means you have to login again when navigating to another So [...]
+
+== Using the Solr Control Script with JWT Auth
+
+The control script (`bin/solr`) does not currently support JWT Auth.
\ No newline at end of file
diff --git a/solr/solrj/src/resources/apispec/cluster.security.JwtAuth.Commands.json b/solr/solrj/src/resources/apispec/cluster.security.JwtAuth.Commands.json
new file mode 100644
index 0000000..e940d2f
--- /dev/null
+++ b/solr/solrj/src/resources/apispec/cluster.security.JwtAuth.Commands.json
@@ -0,0 +1,18 @@
+{
+  "documentation": "https://lucene.apache.org/solr/guide/jwt-authentication-plugin.html",
+  "description": "Modifies the configuration of JWT token authentication.",
+  "methods": [
+    "POST"
+  ],
+  "url": {
+    "paths": [
+      "/cluster/security/authentication"
+    ]
+  },
+  "commands": {
+    "set-property": {
+      "type":"object",
+      "description": "The set-property command lets you set any of the configuration parameters supported by this plugin"
+    }
+  }
+}
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java
index 4bcf8b9..6a7be0b 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java
@@ -17,6 +17,7 @@
 
 package org.apache.solr.cloud;
 
+import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -24,15 +25,30 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
+import java.util.function.Predicate;
 
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.Meter;
 import com.codahale.metrics.Metric;
 import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.Timer;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.message.AbstractHttpMessage;
+import org.apache.http.message.BasicHeader;
+import org.apache.http.util.EntityUtils;
+import org.apache.solr.common.util.Base64;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.Utils;
+import org.jose4j.jws.JsonWebSignature;
+import org.jose4j.lang.JoseException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+
 /**
  * Base test class for cloud tests wanting to track authentication metrics.
  * The assertions provided by this base class require a *minimum* count, not exact count from metrics.
@@ -46,6 +62,7 @@ public class SolrCloudAuthTestCase extends SolrCloudTestCase {
   private static final List<String> AUTH_METRICS_TIMER_KEYS = Collections.singletonList("requestTimes");
   private static final String METRICS_PREFIX_PKI = "SECURITY./authentication/pki.";
   private static final String METRICS_PREFIX = "SECURITY./authentication.";
+  public static final Predicate NOT_NULL_PREDICATE = o -> o != null;
   
   /**
    * Used to check metric counts for PKI auth
@@ -129,4 +146,69 @@ public class SolrCloudAuthTestCase extends SolrCloudTestCase {
     else
       return metrics.stream().mapToLong(l -> ((Counter)l.get(prefix + key)).getCount()).sum();
   }
+  
+  public static void verifySecurityStatus(HttpClient cl, String url, String objPath,
+                                            Object expected, int count) throws Exception {
+    verifySecurityStatus(cl, url, objPath, expected, count, (String)null);
+  }
+
+
+  public static void verifySecurityStatus(HttpClient cl, String url, String objPath,
+                                          Object expected, int count, String user, String pwd)
+      throws Exception {
+    verifySecurityStatus(cl, url, objPath, expected, count, makeBasicAuthHeader(user, pwd));
+  }
+
+  protected void verifySecurityStatus(HttpClient cl, String url, String objPath,
+                                      Object expected, int count, JsonWebSignature jws) throws Exception {
+    verifySecurityStatus(cl, url, objPath, expected, count, getBearerAuthHeader(jws));
+  }
+
+
+  private static void verifySecurityStatus(HttpClient cl, String url, String objPath,
+                                            Object expected, int count, String authHeader) throws IOException, InterruptedException {
+    boolean success = false;
+    String s = null;
+    List<String> hierarchy = StrUtils.splitSmart(objPath, '/');
+    for (int i = 0; i < count; i++) {
+      HttpGet get = new HttpGet(url);
+      if (authHeader != null) setAuthorizationHeader(get, authHeader);
+      HttpResponse rsp = cl.execute(get);
+      s = EntityUtils.toString(rsp.getEntity());
+      Map m = null;
+      try {
+        m = (Map) Utils.fromJSONString(s);
+      } catch (Exception e) {
+        fail("Invalid json " + s);
+      }
+      Utils.consumeFully(rsp.getEntity());
+      Object actual = Utils.getObjectByPath(m, true, hierarchy);
+      if (expected instanceof Predicate) {
+        Predicate predicate = (Predicate) expected;
+        if (predicate.test(actual)) {
+          success = true;
+          break;
+        }
+      } else if (Objects.equals(actual == null ? null : String.valueOf(actual), expected)) {
+        success = true;
+        break;
+      }
+      Thread.sleep(50);
+    }
+    assertTrue("No match for " + objPath + " = " + expected + ", full response = " + s, success);
+  }
+
+  protected static String makeBasicAuthHeader(String user, String pwd) {
+    String userPass = user + ":" + pwd;
+    return "Basic " + Base64.byteArrayToBase64(userPass.getBytes(UTF_8));
+  }
+
+  static String getBearerAuthHeader(JsonWebSignature jws) throws JoseException {
+    return "Bearer " + jws.getCompactSerialization();
+  }
+  
+  public static void setAuthorizationHeader(AbstractHttpMessage httpMsg, String headerString) {
+    httpMsg.setHeader(new BasicHeader("Authorization", headerString));
+    log.info("Added Authorization Header {}", headerString);
+  }
 }
diff --git a/solr/webapp/web/index.html b/solr/webapp/web/index.html
index 6987af7..fcf0f82 100644
--- a/solr/webapp/web/index.html
+++ b/solr/webapp/web/index.html
@@ -85,6 +85,7 @@ limitations under the License.
   <script src="js/angular/controllers/replication.js"></script>
   <script src="js/angular/controllers/schema.js"></script>
   <script src="js/angular/controllers/segments.js"></script>
+  <script src="js/angular/controllers/unknown.js"></script>
   <script src="js/angular/controllers/cluster-suggestions.js"></script>
 
 
diff --git a/solr/webapp/web/js/angular/app.js b/solr/webapp/web/js/angular/app.js
index eb442d8..9abacee 100644
--- a/solr/webapp/web/js/angular/app.js
+++ b/solr/webapp/web/js/angular/app.js
@@ -32,10 +32,18 @@ solrAdminApp.config([
         templateUrl: 'partials/index.html',
         controller: 'IndexController'
       }).
+      when('/unknown', {
+        templateUrl: 'partials/unknown.html',
+        controller: 'UnknownController'
+      }).
       when('/login', {
         templateUrl: 'partials/login.html',
         controller: 'LoginController'
       }).
+      when('/login/:route', {
+        templateUrl: 'partials/login.html',
+        controller: 'LoginController'
+      }).
       when('/~logging', {
         templateUrl: 'partials/logging.html',
         controller: 'LoggingController'
@@ -143,7 +151,8 @@ solrAdminApp.config([
         controller: 'SegmentsController'
       }).
       otherwise({
-        redirectTo: '/'
+        templateUrl: 'partials/unknown.html',
+        controller: 'UnknownController'
       });
 }])
 .constant('Constants', {
@@ -351,7 +360,7 @@ solrAdminApp.config([
         $rootScope.$broadcast('connectionStatusInactive');
       },2000);
     }
-    if (!$location.path().startsWith('/login')) {
+    if (!$location.path().startsWith('/login') && !$location.path().startsWith('/unknown')) {
       sessionStorage.removeItem("http401");
       sessionStorage.removeItem("auth.state");
       sessionStorage.removeItem("auth.statusText");
@@ -379,6 +388,7 @@ solrAdminApp.config([
       var headers = rejection.headers();
       var wwwAuthHeader = headers['www-authenticate'];
       sessionStorage.setItem("auth.wwwAuthHeader", wwwAuthHeader);
+      sessionStorage.setItem("auth.authDataHeader", headers['x-solr-authdata']);
       sessionStorage.setItem("auth.statusText", rejection.statusText);
       sessionStorage.setItem("http401", "true");
       sessionStorage.removeItem("auth.scheme");
diff --git a/solr/webapp/web/js/angular/controllers/login.js b/solr/webapp/web/js/angular/controllers/login.js
index e87f2a8..8127c6f 100644
--- a/solr/webapp/web/js/angular/controllers/login.js
+++ b/solr/webapp/web/js/angular/controllers/login.js
@@ -49,11 +49,107 @@ solrAdminApp.controller('LoginController',
 
         var supportedSchemes = ['Basic', 'Bearer', 'Negotiate'];
         $scope.authSchemeSupported = supportedSchemes.includes(authScheme);
+
+        if (authScheme === 'Bearer') {
+          // Check for OpenId redirect response
+          var errorText = "";
+          $scope.isCallback = false;
+          if ($scope.subPath === 'callback') {
+            $scope.isCallback = true;
+            var hash = $location.hash();
+            var hp = AuthenticationService.decodeHashParams(hash);
+            var expectedState = sessionStorage.getItem("auth.stateRandom") + "_" + sessionStorage.getItem("auth.location");
+            sessionStorage.setItem("auth.state", "error");
+            if (hp['access_token'] && hp['token_type'] && hp['state']) {
+              // Validate state
+              if (hp['state'] !== expectedState) {
+                $scope.error = "Problem with auth callback";
+                console.log("Expected state param " + expectedState + " but got " + hp['state']);
+                errorText += "Invalid values in state parameter. ";
+              }
+              // Validate token type
+              if (hp['token_type'].toLowerCase() !== "bearer") {
+                console.log("Expected token_type param 'bearer', but got " + hp['token_type']);
+                errorText += "Invalid values in token_type parameter. ";
+              }
+              // Unpack ID token and validate nonce, get username
+              if (hp['id_token']) {
+                var idToken = hp['id_token'].split(".");
+                if (idToken.length === 3) {
+                  var payload = AuthenticationService.decodeJwtPart(idToken[1]);
+                  if (!payload['nonce'] || payload['nonce'] !== sessionStorage.getItem("auth.nonce")) {
+                    errorText += "Invalid 'nonce' value, possible attack detected. Please log in again. ";
+                  }
+
+                  if (errorText === "") {
+                    sessionStorage.setItem("auth.username", payload['sub']);
+                    sessionStorage.setItem("auth.header", "Bearer " + hp['access_token']);
+                    sessionStorage.removeItem("auth.statusText");
+                    sessionStorage.removeItem("auth.stateRandom");
+                    sessionStorage.removeItem("auth.wwwAuthHeader");
+                    console.log("User " + payload['sub'] + " is logged in");
+                    var redirectTo = sessionStorage.getItem("auth.location");
+                    console.log("Redirecting to stored location " + redirectTo);
+                    sessionStorage.setItem("auth.state", "authenticated");
+                    sessionStorage.removeItem("http401");
+                    $location.path(redirectTo).hash("");
+                  }
+                } else {
+                  console.log("Expected JWT compact id_token param but got " + idToken);
+                  errorText += "Invalid values in id_token parameter. ";
+                }
+              } else {
+                console.log("Callback was missing the id_token parameter, could not validate nonce.");
+                errorText += "Callback was missing the id_token parameter, could not validate nonce. ";
+              }
+              if (hp['access_token'].split(".").length !== 3) {
+                console.log("Expected JWT compact access_token param but got " + hp['access_token']);
+                errorText += "Invalid values in access_token parameter. ";
+              }
+              if (errorText !== "") {
+                $scope.error = "Problems with OpenID callback";
+                $scope.errorDescription = errorText;
+                $scope.http401 = "true";
+              }
+              // End callback processing
+            } else if (hp['error']) {
+              // The callback had errors
+              console.log("Error received from idp: " + hp['error']);
+              var errorDescriptions = {};
+              errorDescriptions['invalid_request'] = "The request is missing a required parameter, includes an invalid parameter value, includes a parameter more than once, or is otherwise malformed.";
+              errorDescriptions['unauthorized_client'] = "The client is not authorized to request an access token using this method.";
+              errorDescriptions['access_denied'] = "The resource owner or authorization server denied the request.";
+              errorDescriptions['unsupported_response_type'] = "The authorization server does not support obtaining an access token using this method.";
+              errorDescriptions['invalid_scope'] = "The requested scope is invalid, unknown, or malformed.";
+              errorDescriptions['server_error'] = "The authorization server encountered an unexpected condition that prevented it from fulfilling the request.";
+              errorDescriptions['temporarily_unavailable'] = "The authorization server is currently unable to handle the request due to a temporary overloading or maintenance of the server.";
+              $scope.error = "Callback from Id Provider contained error. ";
+              if (hp['error_description']) {
+                $scope.errorDescription = decodeURIComponent(hp['error_description']);
+              } else {
+                $scope.errorDescription = errorDescriptions[hp['error']];
+              }
+              if (hp['error_uri']) {
+                $scope.errorDescription += " More information at " + hp['error_uri'] + ". ";
+              }
+              if (hp['state'] !== expectedState) {
+                $scope.errorDescription += "The state parameter returned from ID Provider did not match the one we sent.";
+              }
+              sessionStorage.setItem("auth.state", "error");
+            }
+          }
+        }
+
+        if (errorText === "" && !$scope.error && authParams) {
+          $scope.error = authParams['error'];
+          $scope.errorDescription = authParams['error_description'];
+          $scope.authData = AuthenticationService.getAuthDataHeader();
+        }
+        
         $scope.authScheme = sessionStorage.getItem("auth.scheme");
         $scope.authRealm = sessionStorage.getItem("auth.realm");
         $scope.wwwAuthHeader = sessionStorage.getItem("auth.wwwAuthHeader");
         $scope.statusText = sessionStorage.getItem("auth.statusText");
-        $scope.authConfig = sessionStorage.getItem("auth.config");
         $scope.authLocation = sessionStorage.getItem("auth.location");
         $scope.authLoggedinUser = sessionStorage.getItem("auth.username");
         $scope.authHeader = sessionStorage.getItem("auth.header");
@@ -69,6 +165,77 @@ solrAdminApp.controller('LoginController',
           $location.path("/");
         };
 
+        $scope.jwtLogin = function () {
+          var stateRandom = Math.random().toString(36).substr(2);
+          sessionStorage.setItem("auth.stateRandom", stateRandom);
+          var authState = stateRandom + "_" + sessionStorage.getItem("auth.location");
+          var authNonce = Math.random().toString(36).substr(2) + Math.random().toString(36).substr(2) + Math.random().toString(36).substr(2);
+          sessionStorage.setItem("auth.nonce", authNonce);
+          var params = {
+            "response_type" : "id_token token",
+            "client_id" : $scope.authData['client_id'],
+            "redirect_uri" : $window.location.href.split('#')[0],
+            "scope" : "openid " + $scope.authData['scope'],
+            "state" : authState,
+            "nonce" : authNonce
+          };
+
+          var endpointBaseUrl = $scope.authData['authorizationEndpoint'];
+          var loc = endpointBaseUrl + "?" + paramsToString(params);
+          console.log("Redirecting to " + loc);
+          sessionStorage.setItem("auth.state", "expectCallback");
+          $window.location.href = loc;
+
+          function paramsToString(params) {
+            var arr = [];
+            for (var p in params) {
+               if( params.hasOwnProperty(p) ) {
+                 arr.push(p + "=" + encodeURIComponent(params[p]));
+               } 
+             }
+             return arr.join("&");
+          }
+        };
+
+        $scope.jwtIsLoginNode = function() {
+          var redirect = $scope.authData ? $scope.authData['redirect_uris'] : undefined;
+          if (redirect && Array.isArray(redirect) && redirect.length > 0) {
+            var isLoginNode = false;
+            redirect.forEach(function(uri) { // Check that current node URL is among the configured callback URIs
+              if ($window.location.href.startsWith(uri)) isLoginNode = true;
+            });
+            return isLoginNode; 
+          } else {
+            return true; // no redirect UIRs configured, all nodes are potential login nodes
+          }
+        };
+
+        $scope.jwtFindLoginNode = function() {
+          var redirect = $scope.authData ? $scope.authData['redirect_uris'] : undefined;
+          if (redirect && Array.isArray(redirect) && redirect.length > 0) {
+            var loginNode = redirect[0];
+            redirect.forEach(function(uri) { // if current node is in list, return its callback uri
+              if ($window.location.href.startsWith(uri)) loginNode = uri;
+            });
+            return loginNode; 
+          } else {
+             return $window.location.href.split('#')[0]; // Return base url of current URL as the url to use 
+          }
+        };
+
+        // Redirect to login node if this is not a valid one
+        $scope.jwtGotoLoginNode = function() {
+          if (!$scope.jwtIsLoginNode()) {
+            $window.location.href = $scope.jwtFindLoginNode();
+          }
+        };
+
+        $scope.jwtLogout = function() {
+          // reset login status
+          AuthenticationService.ClearCredentials();
+          $location.path("/");
+        };
+        
         $scope.isLoggedIn = function() {
           return (sessionStorage.getItem("auth.username") !== null);
         };
diff --git a/solr/webapp/web/js/angular/controllers/unknown.js b/solr/webapp/web/js/angular/controllers/unknown.js
new file mode 100644
index 0000000..2d959e6
--- /dev/null
+++ b/solr/webapp/web/js/angular/controllers/unknown.js
@@ -0,0 +1,37 @@
+/*
+ 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.
+*/
+
+/**
+ * This controller is called whenever no other routes match.
+ * It is a place to intercept to look for special flows such as authentication callbacks (that do not support fragment in URL).
+ * Normal action is to redirect to dashboard "/" if no login is in progress
+ */
+solrAdminApp.controller('UnknownController',
+  ['$scope', '$window', '$routeParams', '$location', 'Constants', 'AuthenticationService',
+    function($scope, $window, $routeParams, $location, Constants, AuthenticationService) {
+      var fragment = $window.location.hash.startsWith("#/") ? $window.location.hash.substring(2) : $window.location.hash;
+      // Check if the URL is actually a callback from Identiy provider 
+      if (AuthenticationService.isJwtCallback(fragment)) {
+        console.log("Detected an authentication callback, redirecting to /#/login/callback");
+        $location.path("/login/callback").hash(fragment);
+      } else {
+        console.log("Redirecting from unknown path " + fragment + " to Dashboard");
+        $location.path("/").hash("");
+      }
+    }
+  ]
+);
diff --git a/solr/webapp/web/js/angular/services.js b/solr/webapp/web/js/angular/services.js
index 3391221..3655639 100644
--- a/solr/webapp/web/js/angular/services.js
+++ b/solr/webapp/web/js/angular/services.js
@@ -281,7 +281,53 @@ solrAdminServices.factory('System',
           sessionStorage.removeItem("auth.username");
           sessionStorage.removeItem("auth.wwwAuthHeader");
           sessionStorage.removeItem("auth.statusText");
+          localStorage.removeItem("auth.stateRandom");
+          sessionStorage.removeItem("auth.nonce");
         };
 
+        service.getAuthDataHeader = function () {
+          try {
+            var header64 = sessionStorage.getItem("auth.authDataHeader");
+            var headerJson = base64.decode(header64);
+            return JSON.parse(headerJson);
+          } catch (e) {
+            console.log("WARN: Wrong or missing X-Solr-AuthData header on 401 response " + e);
+            return null;
+          }
+        };
+
+        service.decodeJwtPart = function (jwtPart) {
+          try {
+            return JSON.parse(base64.urldecode(jwtPart));
+          } catch (e) {
+            console.log("WARN: Invalid format of JWT part: " + e);
+            return {};
+          }
+        };
+
+        service.isJwtCallback = function (hash) {
+          var hp = this.decodeHashParams(hash);
+          // console.log("Decoded hash as " + JSON.stringify(hp, undefined, 2)); // For debugging callbacks
+          return (hp['access_token'] && hp['token_type'] && hp['state']) || hp['error'];
+        };
+        
+        service.decodeHashParams = function(hash) {
+          // access_token, token_type, expires_in, state
+          if (hash == null || hash.length === 0) {
+            return {};
+          }
+          var params = {};
+          var parts = hash.split("&");
+          for (var p in parts) {
+            var kv = parts[p].split("=");
+            if (kv.length === 2) {
+              params[kv[0]] = decodeURIComponent(kv[1]);
+            } else {
+              console.log("Invalid callback URI, got parameter " + parts[p] + " but expected key=value");
+            }
+          }
+          return params;
+        };
+        
         return service;
       }]);
diff --git a/solr/webapp/web/partials/login.html b/solr/webapp/web/partials/login.html
index d5dba14..29c8c71 100644
--- a/solr/webapp/web/partials/login.html
+++ b/solr/webapp/web/partials/login.html
@@ -77,6 +77,70 @@ WWW-Authenticate: {{wwwAuthHeader}}</pre>
     <hr/>
   </div>
   
+  <div ng-show="authScheme === 'Bearer'">
+    <h1>OpenID Connect (JWT) authentication</h1>
+    <div class="login-error" ng-show="statusText || authParamsError || error">
+      {{statusText}} {{authParamsError}} {{error}} {{errorDescription}}
+    </div>
+    <div ng-show="isCallback">
+      Callback from ID Provider received.
+      <p ng-show="error">
+        There were errors during login with ID Provider. Please try again.<br/>
+      </p>
+    </div>
+    <div ng-show="!isLoggedIn() && !isCallback">
+      <p>
+        Solr requires authentication for resource {{authLocation === '/' ? 'Dashboard' : authLocation}}.
+      </p>
+      <div ng-show="authData !== null && jwtIsLoginNode()">
+        <p>
+          Please log in with your Identity Provider (IdP) for realm {{authRealm}}.
+        </p>
+        <p>
+          Clicking the button below, you will be redirected to the authorization endpoint of the ID provider:<br/>
+          {{authData['authorizationEndpoint']}}
+        </p>
+        <br/>
+        <div ng-show="error" class="alert alert-danger">{{error}}</div>
+        <form name="form" ng-submit="jwtLogin()" role="form">
+          <div class="form-actions">
+            <button type="submit" class="btn btn-danger">Redirect to Identity Provider</button>
+          </div>
+        </form>
+      </div>
+      <div ng-show="authData !== null && !jwtIsLoginNode()">
+        <p>
+          In order to log in to the identity provider, you need to load this page from the Solr node registered as callback node:<br/>
+          {{jwtFindLoginNode()}}<br/>
+          After successful login you will be able to navigate to other nodes. 
+        </p>
+        <p>
+          <form name="form" ng-submit="jwtGotoLoginNode()" role="form">
+            <div class="form-actions">
+              <button type="submit" class="btn btn-danger">Redirect to {{jwtFindLoginNode()}}</button>
+            </div>
+          </form>
+        </p>
+      </div>
+      <div class="login-error" ng-show="authData === null">
+        Unable to login. There is a problem with the Solr backend. Please consult Solr logs.
+      </div>
+    </div>
+
+    <div ng-show="isLoggedIn()">
+      <p>
+        Logged in as user {{authLoggedinUser}}. Realm={{authRealm}}.<br/>
+      </p>
+      <br/>
+      <form name="logoutForm" ng-submit="jwtLogout()" role="form" ng-show="isLoggedIn()">
+        <div class="form-actions">
+          <button type="submit" class="btn btn-danger">Logout</button>
+        </div>
+      </form>
+    </div>
+
+  </div>
+  
   <div ng-show="!authSchemeSupported">
     <h1>Authentication scheme not supported</h1>
 
diff --git a/solr/webapp/web/partials/unknown.html b/solr/webapp/web/partials/unknown.html
new file mode 100644
index 0000000..51895ab
--- /dev/null
+++ b/solr/webapp/web/partials/unknown.html
@@ -0,0 +1,23 @@
+<!--
+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.
+-->
+<div id="unknown" class="clearfix">
+
+  <div>
+    Oops, this URL is unknown to us, redirecting you back to Dashboard 
+  </div>
+
+</div>


[lucene-solr] 02/03: SOLR-12121: Fix test fails in JWTAuthPluginIntegrationTest.testMetrics

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

janhoy pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit 0b3ebdd290a3401ee9a8dd6bc0e133cad9718f1f
Author: Jan Høydahl <ja...@apache.org>
AuthorDate: Thu Jan 24 11:41:50 2019 +0100

    SOLR-12121: Fix test fails in JWTAuthPluginIntegrationTest.testMetrics
    
    (cherry picked from commit d09c724a0eaca3666dcb3fbc931bb64eb5f5b72f)
---
 .../security/JWTAuthPluginIntegrationTest.java     |  6 ++--
 .../apache/solr/cloud/SolrCloudAuthTestCase.java   | 35 +++++++++++-----------
 2 files changed, 19 insertions(+), 22 deletions(-)

diff --git a/solr/core/src/test/org/apache/solr/security/JWTAuthPluginIntegrationTest.java b/solr/core/src/test/org/apache/solr/security/JWTAuthPluginIntegrationTest.java
index 14621e8..237ec0d 100644
--- a/solr/core/src/test/org/apache/solr/security/JWTAuthPluginIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/security/JWTAuthPluginIntegrationTest.java
@@ -129,10 +129,8 @@ public class JWTAuthPluginIntegrationTest extends SolrCloudAuthTestCase {
   public void testMetrics() throws Exception {
     boolean isUseV2Api = random().nextBoolean();
     String authcPrefix = "/admin/authentication";
-    String authzPrefix = "/admin/authorization";
     if(isUseV2Api){
       authcPrefix = "/____v2/cluster/security/authentication";
-      authzPrefix = "/____v2/cluster/security/authorization";
     }
     String baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
     CloseableHttpClient cl = HttpClientUtil.createClient(null);
@@ -193,11 +191,11 @@ public class JWTAuthPluginIntegrationTest extends SolrCloudAuthTestCase {
     assertPkiAuthMetricsMinimums(15, 15, 0, 0, 0, 0);
   }
 
-  private void getAndFail(String url, String token) throws IOException {
+  private void getAndFail(String url, String token) {
     try {
       get(url, token);
       fail("Request to " + url + " with token " + token + " should have failed");
-    } catch(Exception e) {}
+    } catch(Exception e) { /* Fall through */ }
   }
   
   private Pair<String, Integer> get(String url, String token) throws IOException {
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java
index 6a7be0b..4a0e6ed 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/SolrCloudAuthTestCase.java
@@ -27,6 +27,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.function.Predicate;
+import java.util.stream.Stream;
 
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.Meter;
@@ -67,7 +68,7 @@ public class SolrCloudAuthTestCase extends SolrCloudTestCase {
   /**
    * Used to check metric counts for PKI auth
    */
-  protected void assertPkiAuthMetricsMinimums(int requests, int authenticated, int passThrough, int failWrongCredentials, int failMissingCredentials, int errors) {
+  protected void assertPkiAuthMetricsMinimums(int requests, int authenticated, int passThrough, int failWrongCredentials, int failMissingCredentials, int errors) throws InterruptedException {
     assertAuthMetricsMinimums(METRICS_PREFIX_PKI, requests, authenticated, passThrough, failWrongCredentials, failMissingCredentials, errors);
   }
   
@@ -76,7 +77,7 @@ public class SolrCloudAuthTestCase extends SolrCloudTestCase {
    * 
    * TODO: many of these params have to be under specified - this should wait a bit to see the desired params and timeout
    */
-  protected void assertAuthMetricsMinimums(int requests, int authenticated, int passThrough, int failWrongCredentials, int failMissingCredentials, int errors) {
+  protected void assertAuthMetricsMinimums(int requests, int authenticated, int passThrough, int failWrongCredentials, int failMissingCredentials, int errors) throws InterruptedException {
     assertAuthMetricsMinimums(METRICS_PREFIX, requests, authenticated, passThrough, failWrongCredentials, failMissingCredentials, errors);
   }
 
@@ -103,17 +104,7 @@ public class SolrCloudAuthTestCase extends SolrCloudTestCase {
    * Common test method to be able to check security from any authentication plugin
    * @param prefix the metrics key prefix, currently "SECURITY./authentication." for basic auth and "SECURITY./authentication/pki." for PKI 
    */
-  private void assertAuthMetricsMinimums(String prefix, int requests, int authenticated, int passThrough, int failWrongCredentials, int failMissingCredentials, int errors) {
-    Map<String, Long> counts = countAuthMetrics(prefix);
-    
-    // check each counter
-    boolean success = isMetricEuqalOrLarger(requests, "requests", counts)
-        & isMetricEuqalOrLarger(authenticated, "authenticated", counts)
-        & isMetricEuqalOrLarger(passThrough, "passThrough", counts)
-        & isMetricEuqalOrLarger(failWrongCredentials, "failWrongCredentials", counts)
-        & isMetricEuqalOrLarger(failMissingCredentials, "failMissingCredentials", counts)
-        & isMetricEuqalOrLarger(errors, "errors", counts);
-    
+  private void assertAuthMetricsMinimums(String prefix, int requests, int authenticated, int passThrough, int failWrongCredentials, int failMissingCredentials, int errors) throws InterruptedException {
     Map<String, Long> expectedCounts = new HashMap<>();
     expectedCounts.put("requests", (long) requests);
     expectedCounts.put("authenticated", (long) authenticated);
@@ -121,6 +112,16 @@ public class SolrCloudAuthTestCase extends SolrCloudTestCase {
     expectedCounts.put("failWrongCredentials", (long) failWrongCredentials);
     expectedCounts.put("failMissingCredentials", (long) failMissingCredentials);
     expectedCounts.put("errors", (long) errors);
+
+    Map<String, Long> counts = countAuthMetrics(prefix);
+    boolean success = isMetricsEqualOrLarger(expectedCounts, counts);
+    if (!success) {
+      log.info("First metrics count assert failed, pausing 2s before re-attempt");
+      Thread.sleep(2000);
+      counts = countAuthMetrics(prefix);
+      success = isMetricsEqualOrLarger(expectedCounts, counts);
+    }
+    
     assertTrue("Expected metric minimums for prefix " + prefix + ": " + expectedCounts + ", but got: " + counts, success);
     
     if (counts.get("requests") > 0) {
@@ -129,11 +130,9 @@ public class SolrCloudAuthTestCase extends SolrCloudTestCase {
     }
   }
 
-  // Check that the actual metric is equal to or greater than the expected value, never less
-  private boolean isMetricEuqalOrLarger(int expected, String key, Map<String, Long> counts) {
-    long cnt = counts.get(key);
-    log.debug("Asserting that auth metrics count ({}) > expected ({})", cnt, expected);
-    return(cnt >= expected);
+  private boolean isMetricsEqualOrLarger(Map<String, Long> expectedCounts, Map<String, Long> actualCounts) {
+    return Stream.of("requests", "authenticated", "passThrough", "failWrongCredentials", "failMissingCredentials", "errors")
+        .allMatch(k -> actualCounts.get(k).intValue() >= expectedCounts.get(k).intValue());
   }
 
   // Have to sum the metrics from all three shards/nodes


[lucene-solr] 03/03: SOLR-12121: Use a different JSON parser for JWTAuthPluginTest after upgrade to Hadoop3

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

janhoy pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit d28ec13b9a977e0fdf1bb10a0fe89c6cb2e2caad
Author: Jan Høydahl <ja...@apache.org>
AuthorDate: Wed Feb 6 09:02:24 2019 +0100

    SOLR-12121: Use a different JSON parser for JWTAuthPluginTest after upgrade to Hadoop3
---
 solr/core/src/test/org/apache/solr/security/JWTAuthPluginTest.java | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)

diff --git a/solr/core/src/test/org/apache/solr/security/JWTAuthPluginTest.java b/solr/core/src/test/org/apache/solr/security/JWTAuthPluginTest.java
index 72a908e..10b049e 100644
--- a/solr/core/src/test/org/apache/solr/security/JWTAuthPluginTest.java
+++ b/solr/core/src/test/org/apache/solr/security/JWTAuthPluginTest.java
@@ -44,7 +44,6 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.mortbay.util.ajax.JSON;
 
 import static org.apache.solr.security.JWTAuthPlugin.JWTAuthenticationResponse.AuthCode.AUTZ_HEADER_PROBLEM;
 import static org.apache.solr.security.JWTAuthPlugin.JWTAuthenticationResponse.AuthCode.NO_AUTZ_HEADER;
@@ -398,7 +397,7 @@ public class JWTAuthPluginTest extends SolrTestCaseJ4 {
     plugin.init(testConfig);
     String headerBase64 = plugin.generateAuthDataHeader();
     String headerJson = new String(Base64.base64ToByteArray(headerBase64), StandardCharsets.UTF_8);
-    Map<String,String> parsed = (Map<String, String>) JSON.parse(headerJson);
+    Map<String,String> parsed = (Map<String, String>) Utils.fromJSONString(headerJson);
     assertEquals("solr:admin", parsed.get("scope"));
     assertEquals("http://acmepaymentscorp/oauth/auz/authorize", parsed.get("authorizationEndpoint"));
     assertEquals("solr-cluster", parsed.get("client_id"));