You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by md...@apache.org on 2022/05/31 15:42:25 UTC

[lucene-solr] branch branch_8_11 updated: SOLR-15965 Use proper signatures for SolrAuth (#2641)

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

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


The following commit(s) were added to refs/heads/branch_8_11 by this push:
     new a236335e11d SOLR-15965 Use proper signatures for SolrAuth (#2641)
a236335e11d is described below

commit a236335e11da126e5ba424c881388fd2f4d6d7f4
Author: Mike Drob <md...@apache.org>
AuthorDate: Tue May 31 08:42:20 2022 -0700

    SOLR-15965 Use proper signatures for SolrAuth (#2641)
    
    Internode communication secured by PKI Authentication has changed formats. This commit allows for smoother upgrades from 8.x to 8.11 to 9.0
---
 solr/CHANGES.txt                                   |   2 +
 .../solr/security/PKIAuthenticationPlugin.java     | 223 ++++++++++++++++-----
 .../apache/solr/servlet/SolrDispatchFilter.java    |   3 +-
 .../src/java/org/apache/solr/util/CryptoKeys.java  |  27 ++-
 .../solr/security/TestPKIAuthenticationPlugin.java |  65 ++++--
 5 files changed, 243 insertions(+), 77 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index c981e96c855..2040f7435a9 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -60,6 +60,8 @@ Bug Fixes
 
 * SOLR-16215: Escape query characters in Solr SQL Array UDF functions (Kiran Chitturi via Timothy Potter)
 
+* SOLR-15965: Use better signatures for the PKI Authentication plugin. (Mike Drob)
+
 ==================  8.11.1 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
diff --git a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
index 9d1f9987f5c..81a652392a5 100644
--- a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
@@ -21,19 +21,26 @@ 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.IOException;
 import java.lang.invoke.MethodHandles;
 import java.nio.ByteBuffer;
+import java.security.InvalidKeyException;
 import java.security.Principal;
 import java.security.PublicKey;
+import java.security.SignatureException;
+import java.time.Instant;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.http.HttpEntity;
 import org.apache.http.HttpException;
+import org.apache.http.HttpHeaders;
 import org.apache.http.HttpRequest;
 import org.apache.http.HttpRequestInterceptor;
 import org.apache.http.HttpResponse;
@@ -61,6 +68,9 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 
 public class PKIAuthenticationPlugin extends AuthenticationPlugin implements HttpClientBuilderPlugin {
 
+  public static final String ACCEPT_VERSIONS = "solr.pki.acceptVersions";
+  public static final String SEND_VERSION = "solr.pki.sendVersion";
+
   /**
    * Mark the current thread as a server thread and set a flag in SolrRequestInfo to indicate you want
    * to send a request as the server identity instead of as the authenticated user.
@@ -89,11 +99,13 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
   private final Map<String, PublicKey> keyCache = new ConcurrentHashMap<>();
   private final PublicKeyHandler publicKeyHandler;
   private final CoreContainer cores;
-  private final int MAX_VALIDITY = Integer.parseInt(System.getProperty("pkiauth.ttl", "15000"));
+  private static final int MAX_VALIDITY = Integer.getInteger("pkiauth.ttl", 15000);
   private final String myNodeName;
   private final HttpHeaderClientInterceptor interceptor = new HttpHeaderClientInterceptor();
   private boolean interceptorRegistered = false;
 
+  private boolean acceptPkiV1 = false;
+
   public boolean isInterceptorRegistered(){
     return interceptorRegistered;
   }
@@ -102,6 +114,21 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
     this.publicKeyHandler = publicKeyHandler;
     this.cores = cores;
     myNodeName = nodeName;
+
+    Set<String> knownPkiVersions = new HashSet<>();
+    knownPkiVersions.add("v1");
+    knownPkiVersions.add("v2");
+    // In branch_8 default accept v1,v2 with option to accept only v2
+    String[] versions = System.getProperty(ACCEPT_VERSIONS, "v1,v2").split(",");
+    for (String version : versions) {
+      if (knownPkiVersions.contains(version) == false) {
+        log.warn("Unknown protocol version [{}] specified in {}", version, ACCEPT_VERSIONS);
+      }
+      if ("v1".equals(version)) {
+        // don't log a warning about using deprecated v1 in branch_8
+        acceptPkiV1 = true;
+      }
+    }
   }
 
   @Override
@@ -120,49 +147,60 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
       return true;
     }
     long receivedTime = System.currentTimeMillis();
-    String header = ((HttpServletRequest) request).getHeader(HEADER);
-    if (header == null) {
-      //this must not happen
-      log.error("No SolrAuth header present");
-      numMissingCredentials.inc();
-      filterChain.doFilter(request, response);
-      return true;
+    PKIHeaderData headerData = null;
+    String headerV2 = ((HttpServletRequest) request).getHeader(HEADER_V2);
+    String headerV1 = ((HttpServletRequest) request).getHeader(HEADER);
+    if (headerV2 != null) {
+      // Try V2 first
+      int nodeNameEnd = headerV2.indexOf(' ');
+      if (nodeNameEnd <= 0) {
+        // Do not log the value as it is likely gibberish
+        return sendError(response, true, "Could not parse node name from SolrAuthV2 header.");
+      }
+      headerData = decipherHeaderV2(headerV2, headerV2.substring(0, nodeNameEnd));
+    } else if (headerV1 != null && acceptPkiV1) {
+      List<String> authInfo = StrUtils.splitWS(headerV1, false);
+      if (authInfo.size() != 2) {
+        // We really shouldn't be logging and returning this, but we did it before so keep that
+        return sendError(response, false, "Invalid SolrAuth header: " + headerV1);
+      }
+      headerData = decipherHeader(authInfo.get(0), authInfo.get(1));
     }
 
-    List<String> authInfo = StrUtils.splitWS(header, false);
-    if (authInfo.size() < 2) {
-      log.error("Invalid SolrAuth Header {}", header);
-      numErrors.mark();
-      filterChain.doFilter(request, response);
-      return true;
+    if (headerData == null) {
+      return sendError(response, true, "Could not load principal from SolrAuthV2 header.");
     }
 
-    String nodeName = authInfo.get(0);
-    String cipher = authInfo.get(1);
-
-    PKIHeaderData decipher = decipherHeader(nodeName, cipher);
-    if (decipher == null) {
-      log.error("Could not decipher a header {} . No principal set", header);
-      numMissingCredentials.inc();
-      filterChain.doFilter(request, response);
-      return true;
-    }
-    if ((receivedTime - decipher.timestamp) > MAX_VALIDITY) {
-      log.error("Invalid key request timestamp: {} , received timestamp: {} , TTL: {}", decipher.timestamp, receivedTime, MAX_VALIDITY);
-      numErrors.mark();
-      filterChain.doFilter(request, response);
-      return true;
+    long elapsed = receivedTime - headerData.timestamp;
+    if (elapsed > MAX_VALIDITY) {
+      return sendError(response, true, "Expired key request timestamp, elapsed=" + elapsed);
     }
 
-    final Principal principal = "$".equals(decipher.userName) ?
+    final Principal principal = "$".equals(headerData.userName) ?
         SU :
-        new BasicUserPrincipal(decipher.userName);
+        new BasicUserPrincipal(headerData.userName);
 
     numAuthenticated.inc();
     filterChain.doFilter(getWrapper((HttpServletRequest) request, principal), response);
     return true;
   }
 
+  /**
+   * Set the response header errors, possibly log something and return false for failed authentication
+   * @param response the response to set error status with
+   * @param v2 whether this authentication used the v1 or v2 header (true if v2)
+   * @param message the message to log and send back to client. do not include anyhting sensitive here about server state
+   * @return false to chain with calls from authenticate
+   */
+  private boolean sendError(ServletResponse response, boolean v2, String message) throws IOException {
+    numErrors.mark();
+    log.error(message);
+    HttpServletResponse httpResponse = (HttpServletResponse) response;
+    httpResponse.setHeader(HttpHeaders.WWW_AUTHENTICATE, v2 ? HEADER_V2 : HEADER);
+    httpResponse.sendError(HttpServletResponse.SC_UNAUTHORIZED, message);
+    return false;
+  }
+
   private static HttpServletRequestWrapper getWrapper(final HttpServletRequest request, final Principal principal) {
     return new HttpServletRequestWrapper(request) {
       @Override
@@ -175,12 +213,66 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
   public static class PKIHeaderData {
     String userName;
     long timestamp;
+
+    @Override
+    public String toString() {
+      return "PKIHeaderData{" +
+          "userName='" + userName + '\'' +
+          ", timestamp=" + timestamp +
+          '}';
+    }
+  }
+
+  private PKIHeaderData decipherHeaderV2(String header, String nodeName) {
+    PublicKey key = keyCache.get(nodeName);
+    if (key == null) {
+      log.debug("No key available for node: {} fetching now ", nodeName);
+      key = getRemotePublicKey(nodeName);
+      log.debug("public key obtained {} ", key);
+    }
+
+    int sigStart = header.lastIndexOf(' ');
+
+    String data = header.substring(0, sigStart);
+    byte[] sig = Base64.base64ToByteArray(header.substring(sigStart + 1));
+    PKIHeaderData rv = validateSignature(data, sig, key);
+    if (rv == null) {
+      log.warn("Failed to verify signature, trying after refreshing the key ");
+      key = getRemotePublicKey(nodeName);
+      rv = validateSignature(data, sig, key);
+    }
+
+    return rv;
+  }
+
+  private PKIHeaderData validateSignature(String data, byte[] sig, PublicKey key) {
+    try {
+      if (CryptoKeys.verifySha256(data.getBytes(UTF_8), sig, key)) {
+        int timestampStart = data.lastIndexOf(' ');
+        PKIHeaderData rv = new PKIHeaderData();
+        String ts = data.substring(timestampStart + 1);
+        try {
+          rv.timestamp = Long.parseLong(ts);
+        } catch (NumberFormatException e) {
+          log.error("SolrAuthV2 header error, cannot parse {} as timestamp", ts);
+          return null;
+        }
+        rv.userName = data.substring(data.indexOf(' ') + 1, timestampStart);
+        return rv;
+      } else {
+        log.warn("Signature verification failed, signature or checksum does not match");
+        return null;
+      }
+    } catch (InvalidKeyException | SignatureException e) {
+      log.error("Signature validation failed, likely key error");
+      return null;
+    }
   }
 
   private PKIHeaderData decipherHeader(String nodeName, String cipherBase64) {
     PublicKey key = keyCache.get(nodeName);
     if (key == null) {
-      log.debug("No key available for node : {} fetching now ", nodeName);
+      log.debug("No key available for node: {} fetching now ", nodeName);
       key = getRemotePublicKey(nodeName);
       log.debug("public key obtained {} ", key);
     }
@@ -223,13 +315,18 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
     }
   }
 
+  /**
+   * Fetch the public key for a remote Solr node and store it in our key cache, replacing any existing entries.
+   * @param nodename the node to fetch a key from
+   * @return the public key
+   */
   PublicKey getRemotePublicKey(String nodename) {
     if (!cores.getZkController().getZkStateReader().getClusterState().getLiveNodes().contains(nodename)) return null;
     String url = cores.getZkController().getZkStateReader().getBaseUrlForNodeName(nodename);
     HttpEntity entity = null;
     try {
       String uri = url + PublicKeyHandler.PATH + "?wt=json&omitHeader=true";
-      log.debug("Fetching fresh public key from : {}",uri);
+      log.debug("Fetching fresh public key from: {}",uri);
       HttpResponse rsp = cores.getUpdateShardHandler().getDefaultHttpClient()
           .execute(new HttpGet(uri), HttpClientUtil.createNewHttpClientRequestContext());
       entity  = rsp.getEntity();
@@ -238,16 +335,16 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
       Map m = (Map) Utils.fromJSON(bytes);
       String key = (String) m.get("key");
       if (key == null) {
-        log.error("No key available from {} {}", url, PublicKeyHandler.PATH);
+        log.error("No key available from {}{}", url, PublicKeyHandler.PATH);
         return null;
       } else {
-        log.info("New Key obtained from  node: {} / {}", nodename, key);
+        log.info("New key obtained from  node={}, key={}", nodename, key);
       }
       PublicKey pubKey = CryptoKeys.deserializeX509PublicKey(key);
       keyCache.put(nodename, pubKey);
       return pubKey;
     } catch (Exception e) {
-      log.error("Exception trying to get public key from : {}", url, e);
+      log.error("Exception trying to get public key from: {}", url, e);
       return null;
     } finally {
       Utils.consumeFully(entity);
@@ -269,7 +366,12 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
           if (log.isDebugEnabled()) {
             log.debug("{} secures this internode request", this.getClass().getSimpleName());
           }
-          generateToken().ifPresent(s -> request.header(HEADER, myNodeName + " " + s));
+          // Reversed from branch_9
+          if ("v2".equals(System.getProperty(SEND_VERSION))) {
+            generateTokenV2().ifPresent(s -> request.header(HEADER_V2, s));
+          } else {
+            generateToken().ifPresent(s -> request.header(HEADER, s));
+          }
         } else {
           if (log.isDebugEnabled()) {
             log.debug("{} secures this internode request", cores.getAuthenticationPlugin().getClass().getSimpleName());
@@ -314,42 +416,67 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
     }
   }
 
-  @SuppressForbidden(reason = "Needs currentTimeMillis to set current time in header")
-  private Optional<String> generateToken() {
+  private String getUser() {
     SolrRequestInfo reqInfo = getRequestInfo();
-    String usr;
     if (reqInfo != null && !reqInfo.useServerToken()) {
       Principal principal = reqInfo.getUserPrincipal();
       if (principal == null) {
         log.debug("generateToken: principal is null");
         //this had a request but not authenticated
         //so we don't not need to set a principal
-        return Optional.empty();
+        return null;
       } else {
-        usr = principal.getName();
+        assert principal.getName() != null;
+        return principal.getName();
       }
     } else {
       if (!isSolrThread()) {
         //if this is not running inside a Solr threadpool (as in testcases)
         // then no need to add any header
         log.debug("generateToken: not a solr (server) thread");
-        return Optional.empty();
+        return null;
       }
       //this request seems to be originated from Solr itself
-      usr = "$"; //special name to denote the user is the node itself
+      return "$"; //special name to denote the user is the node itself
     }
+  }
 
-    String s = usr + " " + System.currentTimeMillis();
+  @SuppressForbidden(reason = "Needs currentTimeMillis to set current time in header")
+  private Optional<String> generateToken() {
+    String usr = getUser();
+    if (usr == null) {
+      return Optional.empty();
+    }
 
+    String s = usr + " " + System.currentTimeMillis();
     byte[] payload = s.getBytes(UTF_8);
     byte[] payloadCipher = publicKeyHandler.keyPair.encrypt(ByteBuffer.wrap(payload));
     String base64Cipher = Base64.byteArrayToBase64(payloadCipher);
     log.trace("generateToken: usr={} token={}", usr, base64Cipher);
-    return Optional.of(base64Cipher);
+    return Optional.of(myNodeName + " " + base64Cipher);
+  }
+
+  private Optional<String> generateTokenV2() {
+    String user = getUser();
+    if (user == null) {
+      return Optional.empty();
+    }
+
+    String s = myNodeName + " " + user + " " + Instant.now().toEpochMilli();
+
+    byte[] payload = s.getBytes(UTF_8);
+    byte[] signature = publicKeyHandler.keyPair.signSha256(payload);
+    String base64Signature = Base64.byteArrayToBase64(signature);
+    return Optional.of(s + " " + base64Signature);
   }
 
   void setHeader(HttpRequest httpRequest) {
-    generateToken().ifPresent(s -> httpRequest.setHeader(HEADER, myNodeName + " " + s));
+    // Reversed from branch_9
+    if ("v2".equals(System.getProperty(SEND_VERSION))) {
+      generateTokenV2().ifPresent(s -> httpRequest.setHeader(HEADER_V2, s));
+    } else {
+      generateToken().ifPresent(s -> httpRequest.setHeader(HEADER, s));
+    }
   }
 
   boolean isSolrThread() {
@@ -366,11 +493,13 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
     interceptorRegistered = false;
   }
 
+  @VisibleForTesting
   public String getPublicKey() {
     return publicKeyHandler.getPublicKey();
   }
 
   public static final String HEADER = "SolrAuth";
+  public static final String HEADER_V2 = "SolrAuthV2";
   public static final String NODE_IS_USER = "$";
   // special principal to denote the cluster member
   private static final Principal SU = new BasicUserPrincipal("$");
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index b21bfdef1ef..9f0a7b3f275 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -508,7 +508,8 @@ public class SolrDispatchFilter extends BaseSolrFilter {
         return true;
       }
       String header = request.getHeader(PKIAuthenticationPlugin.HEADER);
-      if (header != null && cores.getPkiAuthenticationSecurityBuilder() != null)
+      String headerV2 = request.getHeader(PKIAuthenticationPlugin.HEADER_V2);
+      if ((header != null || headerV2 != null) && cores.getPkiAuthenticationSecurityBuilder() != null)
         authenticationPlugin = cores.getPkiAuthenticationSecurityBuilder();
       try {
         if (log.isDebugEnabled()) {
diff --git a/solr/core/src/java/org/apache/solr/util/CryptoKeys.java b/solr/core/src/java/org/apache/solr/util/CryptoKeys.java
index ba176789310..9a5abdea034 100644
--- a/solr/core/src/java/org/apache/solr/util/CryptoKeys.java
+++ b/solr/core/src/java/org/apache/solr/util/CryptoKeys.java
@@ -326,6 +326,17 @@ public final class CryptoKeys {
 
   }
 
+  public static boolean verifySha256(byte[] data, byte[] sig, PublicKey key) throws InvalidKeyException, SignatureException {
+    try {
+      Signature signature = Signature.getInstance("SHA256withRSA");
+      signature.initVerify(key);
+      signature.update(data);
+      return signature.verify(sig);
+    } catch (NoSuchAlgorithmException e) {
+      throw new InternalError("SHA256withRSA must be supported by the JVM.");
+    }
+  }
+
   public static class RSAKeyPair {
     private final String pubKeyStr;
     private final PublicKey publicKey;
@@ -372,19 +383,21 @@ public final class CryptoKeys {
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,e);
       }
     }
-    public byte[] signSha256(byte[] bytes) throws InvalidKeyException, SignatureException {
+    public byte[] signSha256(byte[] bytes) {
       Signature dsa = null;
       try {
         dsa = Signature.getInstance("SHA256withRSA");
       } catch (NoSuchAlgorithmException e) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+        throw new InternalError("SHA256withRSA is required to be supported by the JVM.", e);
+      }
+      try {
+        dsa.initSign(privateKey);
+        dsa.update(bytes,0,bytes.length);
+        return dsa.sign();
+      } catch (InvalidKeyException | SignatureException e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error generating PKI Signature", e);
       }
-      dsa.initSign(privateKey);
-      dsa.update(bytes,0,bytes.length);
-      return dsa.sign();
-
     }
-
   }
 
   public static void main(String[] args) throws Exception {
diff --git a/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java b/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
index a25b3913667..0816512c5a1 100644
--- a/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
+++ b/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
@@ -19,12 +19,15 @@ package org.apache.solr.security;
 import javax.servlet.FilterChain;
 import javax.servlet.ServletRequest;
 import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
 import java.nio.ByteBuffer;
 import java.security.Principal;
 import java.security.PublicKey;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.time.Instant;
+
+import org.apache.http.HttpHeaders;
 import org.apache.solr.common.util.Base64;
 import java.util.concurrent.atomic.AtomicReference;
 
@@ -33,15 +36,17 @@ import org.apache.http.auth.BasicUserPrincipal;
 import org.apache.http.message.BasicHttpRequest;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.core.CoreContainer;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.util.CryptoKeys;
+import org.mockito.ArgumentMatchers;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.Mockito.any;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
@@ -51,8 +56,8 @@ public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
 
     Map<String, PublicKey> remoteKeys = new ConcurrentHashMap<>();
 
-    public MockPKIAuthenticationPlugin(CoreContainer cores, String node) {
-      super(cores, node, new PublicKeyHandler());
+    public MockPKIAuthenticationPlugin(String node) {
+      super(null, node, new PublicKeyHandler());
     }
 
     @Override
@@ -71,6 +76,7 @@ public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
     }
   }
 
+  String headerKey = PKIAuthenticationPlugin.HEADER;
   final AtomicReference<Header> header = new AtomicReference<>();
   final AtomicReference<ServletRequest> wrappedRequestByFilter = new AtomicReference<>();
   final FilterChain filterChain = (servletRequest, servletResponse) -> wrappedRequestByFilter.set(servletRequest);
@@ -82,7 +88,12 @@ public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
     AtomicReference<Principal> principal = new AtomicReference<>();
     String nodeName = "node_x_233";
 
-    final MockPKIAuthenticationPlugin mock = new MockPKIAuthenticationPlugin(null, nodeName);
+    if (random().nextBoolean()) {
+      headerKey = PKIAuthenticationPlugin.HEADER_V2;
+      System.setProperty(PKIAuthenticationPlugin.SEND_VERSION, "v2");
+    }
+
+    final MockPKIAuthenticationPlugin mock = new MockPKIAuthenticationPlugin(nodeName);
     LocalSolrQueryRequest localSolrQueryRequest = new LocalSolrQueryRequest(null, new ModifiableSolrParams()) {
       @Override
       public Principal getUserPrincipal() {
@@ -97,27 +108,16 @@ public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
     mock.solrRequestInfo = new SolrRequestInfo(localSolrQueryRequest, new SolrQueryResponse());
     BasicHttpRequest request = new BasicHttpRequest("GET", "http://localhost:56565");
     mock.setHeader(request);
-    header.set(request.getFirstHeader(PKIAuthenticationPlugin.HEADER));
+    header.set(request.getFirstHeader(headerKey));
     assertNotNull(header.get());
     assertTrue(header.get().getValue().startsWith(nodeName));
     HttpServletRequest mockReq = createMockRequest(header);
-    mock.authenticate(mockReq, null, filterChain);
+    assertTrue(mock.authenticate(mockReq, null, filterChain));
 
     assertNotNull(wrappedRequestByFilter.get());
     assertNotNull(((HttpServletRequest) wrappedRequestByFilter.get()).getUserPrincipal());
     assertEquals(username, ((HttpServletRequest) wrappedRequestByFilter.get()).getUserPrincipal().getName());
 
-    //test 2
-    principal.set(null); // no user
-    header.set(null);
-    wrappedRequestByFilter.set(null);//
-    request = new BasicHttpRequest("GET", "http://localhost:56565");
-    mock.setHeader(request);
-    assertNull(request.getFirstHeader(PKIAuthenticationPlugin.HEADER));
-    mock.authenticate(mockReq, null, filterChain);
-    assertNotNull(wrappedRequestByFilter.get());
-    assertNull(((HttpServletRequest) wrappedRequestByFilter.get()).getUserPrincipal());
-
     //test 3 . No user request . Request originated from Solr
     //create pub key in advance because it can take time and it should be
     //created before the header is set
@@ -127,16 +127,16 @@ public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
     wrappedRequestByFilter.set(null);
     request = new BasicHttpRequest("GET", "http://localhost:56565");
     mock.setHeader(request);
-    header.set(request.getFirstHeader(PKIAuthenticationPlugin.HEADER));
+    header.set(request.getFirstHeader(headerKey));
     assertNotNull(header.get());
     assertTrue(header.get().getValue().startsWith(nodeName));
 
-    mock.authenticate(mockReq, null, filterChain);
+    assertTrue(mock.authenticate(mockReq, null, filterChain));
     assertNotNull(wrappedRequestByFilter.get());
     assertEquals("$", ((HttpServletRequest) wrappedRequestByFilter.get()).getUserPrincipal().getName());
 
     /*test4 mock the restart of a node*/
-    MockPKIAuthenticationPlugin mock1 = new MockPKIAuthenticationPlugin(null, nodeName) {
+    MockPKIAuthenticationPlugin mock1 = new MockPKIAuthenticationPlugin(nodeName) {
       int called = 0;
       @Override
       PublicKey getRemotePublicKey(String nodename) {
@@ -148,11 +148,32 @@ public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
       }
     };
 
-    mock1.authenticate(mockReq, null,filterChain );
+    assertTrue(mock1.authenticate(mockReq, null,filterChain));
     assertNotNull(wrappedRequestByFilter.get());
     assertEquals("$", ((HttpServletRequest) wrappedRequestByFilter.get()).getUserPrincipal().getName());
     mock1.close();
     mock.close();
+
+    /* test protocol mismatch */
+    wrappedRequestByFilter.set(null);
+
+    System.setProperty(PKIAuthenticationPlugin.SEND_VERSION, "v1");
+    System.setProperty(PKIAuthenticationPlugin.ACCEPT_VERSIONS, "v2");
+    MockPKIAuthenticationPlugin mock2 = new MockPKIAuthenticationPlugin(nodeName);
+
+    principal.set(new BasicUserPrincipal("solr"));
+    mock2.solrRequestInfo = new SolrRequestInfo(localSolrQueryRequest, new SolrQueryResponse());
+    mock2.setHeader(request);
+
+    HttpServletResponse response = mock(HttpServletResponse.class);
+    assertFalse("Should have failed authentication", mock2.authenticate(mockReq, response, filterChain));
+
+    verify(response).setHeader(HttpHeaders.WWW_AUTHENTICATE, PKIAuthenticationPlugin.HEADER_V2);
+    verify(response).sendError(ArgumentMatchers.eq(401), anyString());
+
+    assertNull("Should not have proceeded after authentication failure", wrappedRequestByFilter.get());
+
+    mock2.close();
   }
 
   public void testParseCipher() {
@@ -228,7 +249,7 @@ public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
   private HttpServletRequest createMockRequest(final AtomicReference<Header> header) {
     HttpServletRequest mockReq = mock(HttpServletRequest.class);
     when(mockReq.getHeader(any(String.class))).then(invocation -> {
-      if (PKIAuthenticationPlugin.HEADER.equals(invocation.getArgument(0))) {
+      if (headerKey.equals(invocation.getArgument(0))) {
         if (header.get() == null) return null;
         return header.get().getValue();
       } else return null;