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 2017/02/24 13:31:19 UTC

lucene-solr:master: SOLR-9640: Support PKI authentication and SSL in standalone-mode master/slave auth with local security.json

Repository: lucene-solr
Updated Branches:
  refs/heads/master 5eeb8136f -> 95d6fc251


SOLR-9640: Support PKI authentication and SSL in standalone-mode master/slave auth with local security.json


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/95d6fc25
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/95d6fc25
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/95d6fc25

Branch: refs/heads/master
Commit: 95d6fc2512d6525b2354165553f0d6cc4d0d6310
Parents: 5eeb813
Author: Jan H�ydahl <ja...@apache.org>
Authored: Fri Feb 24 14:26:48 2017 +0100
Committer: Jan H�ydahl <ja...@apache.org>
Committed: Fri Feb 24 14:30:42 2017 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../org/apache/solr/core/CoreContainer.java     |   9 +-
 .../solr/security/PKIAuthenticationPlugin.java  |  42 +++++-
 .../org/apache/solr/servlet/HttpSolrCall.java   |   4 +-
 .../apache/solr/servlet/SolrDispatchFilter.java |  11 +-
 .../solr/security/BasicAuthDistributedTest.java | 136 +++++++++++++++++++
 .../security/TestPKIAuthenticationPlugin.java   |  38 +++++-
 .../solr/BaseDistributedSearchTestCase.java     |  37 ++++-
 8 files changed, 260 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/95d6fc25/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 0302615..2c5f0db 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -134,6 +134,8 @@ New Features
   field must both be stored=false, indexed=false, docValues=true. (Ishan Chattopadhyaya, hossman, noble,
   shalin, yonik)
 
+* SOLR-9640: Support PKI authentication and SSL in standalone-mode master/slave auth with local security.json (janhoy)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/95d6fc25/solr/core/src/java/org/apache/solr/core/CoreContainer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index e3977d7..6115562 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -497,7 +497,9 @@ public class CoreContainer {
     hostName = cfg.getNodeName();
 
     zkSys.initZooKeeper(this, solrHome, cfg.getCloudConfig());
-    if(isZooKeeperAware())  pkiAuthenticationPlugin = new PKIAuthenticationPlugin(this, zkSys.getZkController().getNodeName());
+    pkiAuthenticationPlugin = isZooKeeperAware() ?
+        new PKIAuthenticationPlugin(this, zkSys.getZkController().getNodeName()) :
+        new PKIAuthenticationPlugin(this, getNodeNameLocal());
 
     MDCLoggingContext.setNode(this);
 
@@ -618,6 +620,11 @@ public class CoreContainer {
     }
   }
 
+  // Builds a node name to be used with PKIAuth.
+  private String getNodeNameLocal() {
+    return getConfig().getCloudConfig().getHost()+":"+getConfig().getCloudConfig().getSolrHostPort()+"_solr";
+  }
+
   public void securityNodeChanged() {
     log.info("Security node changed, reloading security.json");
     reloadSecurityProperties();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/95d6fc25/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
----------------------------------------------------------------------
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 fdd4408..d185bc9 100644
--- a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
+++ b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
@@ -22,7 +22,9 @@ import javax.servlet.ServletResponse;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletRequestWrapper;
 import java.io.IOException;
+import java.io.UnsupportedEncodingException;
 import java.lang.invoke.MethodHandles;
+import java.net.URLDecoder;
 import java.nio.ByteBuffer;
 import java.security.Principal;
 import java.security.PublicKey;
@@ -193,9 +195,14 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
   }
 
   PublicKey getRemotePublicKey(String nodename) {
-    String url = cores.getZkController().getZkStateReader().getBaseUrlForNodeName(nodename);
+    String url, uri = null;
+    if (cores.isZooKeeperAware()) {
+      url = cores.getZkController().getZkStateReader().getBaseUrlForNodeName(nodename);
+    } else {
+      url = getBaseUrlForNodeNameLocal(nodename);
+    }
     try {
-      String uri = url + PATH + "?wt=json&omitHeader=true";
+      uri += PATH + "?wt=json&omitHeader=true";
       log.debug("Fetching fresh public key from : {}",uri);
       HttpResponse rsp = cores.getUpdateShardHandler().getHttpClient()
           .execute(new HttpGet(uri), HttpClientUtil.createNewHttpClientRequestContext());
@@ -212,12 +219,41 @@ public class PKIAuthenticationPlugin extends AuthenticationPlugin implements Htt
       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 : " + uri, e);
       return null;
     }
 
   }
 
+  protected String getBaseUrlForNodeNameLocal(String nodeName) {
+    final int _offset = nodeName.indexOf("_");
+    if (_offset < 0) {
+      throw new IllegalArgumentException("nodeName does not contain expected '_' seperator: " + nodeName);
+    }
+    final String hostAndPort = nodeName.substring(0,_offset);
+    try {
+      final String path = URLDecoder.decode(nodeName.substring(1+_offset), "UTF-8");
+      // TODO: Find a better way of resolving urlScheme when not using ZK?
+      String urlScheme = resolveUrlScheme();
+      return urlScheme + "://" + hostAndPort + (path.isEmpty() ? "" : ("/" + path));
+    } catch (UnsupportedEncodingException e) {
+      throw new IllegalStateException("JVM Does not seem to support UTF-8", e);
+    }
+  }
+
+  /**
+   * Resolve urlScheme first from sysProp "urlScheme", if not set or invalid value, peek at ssl sysProps
+   * @return "https" if SSL is enabled, else "http"
+   */
+  protected static String resolveUrlScheme() {
+    String urlScheme = System.getProperty("urlScheme");
+    if (urlScheme != null && urlScheme.matches("https?")) {
+      return urlScheme;
+    } else {
+      return System.getProperty("solr.jetty.keystore") == null ? "http" : "https";
+    }
+  }
+
   @Override
   public SolrHttpClientBuilder getHttpClientBuilder(SolrHttpClientBuilder builder) {
     HttpClientUtil.addRequestInterceptor(interceptor);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/95d6fc25/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
index 4f6bae0..0dfb0ea 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -560,7 +560,7 @@ public class HttpSolrCall {
   }
 
   private boolean shouldAuthorize() {
-    if(PKIAuthenticationPlugin.PATH.equals(path)) return false;
+    if(path != null && path.endsWith(PKIAuthenticationPlugin.PATH)) return false;
     //admin/info/key is the path where public key is exposed . it is always unsecured
     if (cores.getPkiAuthenticationPlugin() != null && req.getUserPrincipal() != null) {
       boolean b = cores.getPkiAuthenticationPlugin().needsAuthorization(req);
@@ -1081,7 +1081,7 @@ public class HttpSolrCall {
           response.delete(response.length() - 1, response.length());
         
         response.append("], Path: [").append(resource).append("]");
-        response.append(" path : ").append(path).append(" params :").append(solrReq.getParams());
+        response.append(" path : ").append(path).append(" params :").append(solrReq == null ? null : solrReq.getParams());
         return response.toString();
       }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/95d6fc25/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
----------------------------------------------------------------------
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 ce65069..4ce57b0 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -402,11 +402,11 @@ public class SolrDispatchFilter extends BaseSolrFilter {
     if (authenticationPlugin == null) {
       return true;
     } else {
-      // /admin/info/key must be always open. see SOLR-9188
-      // tests work only w/ getPathInfo
-      //otherwise it's just enough to have getServletPath()
-      if (PKIAuthenticationPlugin.PATH.equals(((HttpServletRequest) request).getServletPath()) ||
-          PKIAuthenticationPlugin.PATH.equals(((HttpServletRequest) request).getPathInfo())) return true;
+      String requestUri = ((HttpServletRequest) request).getRequestURI();
+      if (requestUri != null && requestUri.endsWith(PKIAuthenticationPlugin.PATH)) {
+        log.debug("Passthrough of pki URL " + requestUri);
+        return true;
+      }
       String header = ((HttpServletRequest) request).getHeader(PKIAuthenticationPlugin.HEADER);
       if (header != null && cores.getPkiAuthenticationPlugin() != null)
         authenticationPlugin = cores.getPkiAuthenticationPlugin();
@@ -418,7 +418,6 @@ public class SolrDispatchFilter extends BaseSolrFilter {
           wrappedRequest.set(req);
         });
       } catch (Exception e) {
-        log.info("Error authenticating", e);
         throw new SolrException(ErrorCode.SERVER_ERROR, "Error during request authentication, ", e);
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/95d6fc25/solr/core/src/test/org/apache/solr/security/BasicAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/security/BasicAuthDistributedTest.java b/solr/core/src/test/org/apache/solr/security/BasicAuthDistributedTest.java
new file mode 100644
index 0000000..e35e369
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/security/BasicAuthDistributedTest.java
@@ -0,0 +1,136 @@
+/*
+ * 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.nio.file.Files;
+import java.nio.file.Paths;
+
+import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.solr.BaseDistributedSearchTestCase;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.admin.SecurityConfHandler;
+import org.apache.solr.handler.admin.SecurityConfHandlerLocalForTesting;
+import org.apache.solr.util.LogLevel;
+import org.junit.Test;
+
+/**
+ * Tests basicAuth in a multi shard env
+ */
+@Slow
+public class BasicAuthDistributedTest extends BaseDistributedSearchTestCase {
+  public BasicAuthDistributedTest() {
+    super();
+    schemaString = "schema.xml";
+  }
+
+  private SecurityConfHandlerLocalForTesting securityConfHandler;
+
+  @Test
+  @LogLevel("org.apache.solr=DEBUG")
+  public void test() throws Exception {
+    index();
+    testAuth();
+  }
+
+  private void index() throws Exception {
+    del("*:*");
+    indexr(id, "1", "text", "doc one");
+    indexr(id, "2", "text", "doc two");
+    indexr(id, "3", "text", "doc three");
+    indexr(id, "4", "text", "doc four");
+    indexr(id, "5", "text", "doc five");
+
+    commit();  // try to ensure there's more than one segment
+
+    indexr(id, "6", "text", "doc six");
+    indexr(id, "7", "text", "doc seven");
+    indexr(id, "8", "text", "doc eight");
+    indexr(id, "9", "text", "doc nine");
+    indexr(id, "10", "text", "doc ten");
+
+    commit();
+
+    handle.clear();
+    handle.put("QTime", SKIPVAL);
+    handle.put("timestamp", SKIPVAL);
+    handle.put("maxScore", SKIPVAL);
+    handle.put("_version_", SKIPVAL);
+  }
+
+  private void testAuth() throws Exception {
+    QueryResponse rsp = query("q","text:doc", "fl", "id,text", "sort", "id asc");
+    assertEquals(10, rsp.getResults().getNumFound());
+
+    // Enable authentication
+    for (JettySolrRunner j : jettys) {
+      writeSecurityJson(j.getCoreContainer());
+    }
+
+    HttpSolrClient.RemoteSolrException expected = expectThrows(HttpSolrClient.RemoteSolrException.class, () -> {
+      query("q","text:doc-fail", "fl", "id,text", "sort", "id asc");
+    });
+    assertEquals(401, expected.code());
+
+    // Add auth
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.add("q", "text:doc").add("fl", "id,text").add("sort", "id asc");
+    QueryRequest req = new QueryRequest(params);
+    req.setBasicAuthCredentials("solr", "SolrRocks");
+    rsp = req.process(clients.get(0), null);
+    if (jettys.size() > 1) {
+      assertTrue(rsp.getResults().getNumFound() < 10);
+      rsp = query(true, params, "solr", "SolrRocks");
+    }
+    assertEquals(10, rsp.getResults().getNumFound());
+
+    // Disable auth
+    for (JettySolrRunner j : jettys) {
+      deleteSecurityJson(j.getCoreContainer());
+    }
+
+  }
+
+  private void deleteSecurityJson(CoreContainer coreContainer) throws IOException {
+    securityConfHandler = new SecurityConfHandlerLocalForTesting(coreContainer);
+    Files.delete(Paths.get(coreContainer.getSolrHome()).resolve("security.json"));
+    coreContainer.securityNodeChanged();
+  }
+
+  private void writeSecurityJson(CoreContainer coreContainer) throws IOException {
+    securityConfHandler = new SecurityConfHandlerLocalForTesting(coreContainer);
+    securityConfHandler.persistConf(new SecurityConfHandler.SecurityConfig()
+        .setData(Utils.fromJSONString(ALL_CONF.replaceAll("'", "\""))));
+    coreContainer.securityNodeChanged();
+  }
+
+  protected static final String ALL_CONF = "{\n" +
+      "  'authentication':{\n" +
+      "    'blockUnknown':true,\n" +
+      "    'class':'solr.BasicAuthPlugin',\n" +
+      "    'credentials':{'solr':'orwp2Ghgj39lmnrZOTm7Qtre1VqHFDfwAEzr0ApbN3Y= Ju5osoAqOX8iafhWpPP01E5P+sg8tK8tHON7rCYZRRw='}},\n" +
+      "  'authorization':{\n" +
+      "    'class':'solr.RuleBasedAuthorizationPlugin',\n" +
+      "    'user-role':{'solr':'admin'},\n" +
+      "    'permissions':[{'name':'all','role':'admin'}]}}";
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/95d6fc25/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
----------------------------------------------------------------------
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 a5a279f..90c5bd2 100644
--- a/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
+++ b/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.java
@@ -35,7 +35,11 @@ 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 static org.mockito.Mockito.*;
+import org.junit.Test;
+
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
 
@@ -141,10 +145,38 @@ public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
     mock1.doAuthenticate(mockReq, null,filterChain );
     assertNotNull(wrappedRequestByFilter.get());
     assertEquals("$", ((HttpServletRequest) wrappedRequestByFilter.get()).getUserPrincipal().getName());
+  }
 
+  @Test
+  public void testGetBaseUrlForNodeNameLocal() {
+    synchronized (this) {
+      final MockPKIAuthenticationPlugin mock = new MockPKIAuthenticationPlugin(null, "myName");
+      System.clearProperty("solr.jetty.keystore");
+      assertEquals("http://my.host:9876/solr2", mock.getBaseUrlForNodeNameLocal("my.host:9876_solr2"));
+      System.setProperty("solr.jetty.keystore", "foo");
+      assertEquals("https://my.host:9876/solr2", mock.getBaseUrlForNodeNameLocal("my.host:9876_solr2"));
+      System.clearProperty("solr.jetty.keystore");
+    }
+  }
 
-
-
+  @Test
+  public void testResolveUrlScheme() {
+    synchronized (this) {
+      System.clearProperty("urlScheme");
+      System.clearProperty("solr.jetty.keystore");
+      assertEquals("http", MockPKIAuthenticationPlugin.resolveUrlScheme());
+      System.setProperty("urlScheme", "http");
+      assertEquals("http", MockPKIAuthenticationPlugin.resolveUrlScheme());
+      System.setProperty("urlScheme", "https");
+      assertEquals("https", MockPKIAuthenticationPlugin.resolveUrlScheme());
+      System.setProperty("urlScheme", "ftp");
+      System.clearProperty("solr.jetty.keystore");
+      assertEquals("http", MockPKIAuthenticationPlugin.resolveUrlScheme());
+      System.setProperty("solr.jetty.keystore", "foo");
+      assertEquals("https", MockPKIAuthenticationPlugin.resolveUrlScheme());
+      System.clearProperty("urlScheme");
+      System.clearProperty("solr.jetty.keystore");
+    }
   }
 
   private HttpServletRequest createMockRequest(final AtomicReference<Header> header) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/95d6fc25/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java b/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
index 8c6eb60..bbfc048 100644
--- a/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
+++ b/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
@@ -50,6 +50,7 @@ import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.embedded.JettyConfig;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.client.solrj.response.UpdateResponse;
@@ -558,6 +559,12 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
     return rsp;
   }
 
+  protected QueryResponse queryServer(QueryRequest req) throws IOException, SolrServerException {
+    int which = r.nextInt(clients.size());
+    SolrClient client = clients.get(which);
+    return req.process(client, null);
+  }
+
   /**
    * Sets distributed params.
    * Returns the QueryResponse from {@link #queryServer},
@@ -591,18 +598,31 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
    * Returns the QueryResponse from {@link #queryServer}  
    */
   protected QueryResponse query(boolean setDistribParams, SolrParams p) throws Exception {
+    return query(setDistribParams, p, null, null);
+  }
+
+  /**
+   * Returns the QueryResponse from {@link #queryServer}
+   * @param setDistribParams whether to do a distributed request
+   * @param user basic auth username (set to null if not in use)
+   * @param pass basic auth password (set to null if not in use)
+   * @return the query response
+   */
+  protected QueryResponse query(boolean setDistribParams, SolrParams p, String user, String pass) throws Exception {
     
     final ModifiableSolrParams params = new ModifiableSolrParams(p);
 
     // TODO: look into why passing true causes fails
     params.set("distrib", "false");
-    final QueryResponse controlRsp = controlClient.query(params);
+    QueryRequest req = generateQueryRequest(params, user, pass);
+    final QueryResponse controlRsp = req.process(controlClient, null);
     validateControlData(controlRsp);
 
     params.remove("distrib");
     if (setDistribParams) setDistributedParams(params);
+    req = generateQueryRequest(params, user, pass);
 
-    QueryResponse rsp = queryServer(params);
+    QueryResponse rsp = queryServer(req);
 
     compareResponses(rsp, controlRsp);
 
@@ -617,7 +637,8 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
               int which = r.nextInt(clients.size());
               SolrClient client = clients.get(which);
               try {
-                QueryResponse rsp = client.query(new ModifiableSolrParams(params));
+                QueryRequest qreq = generateQueryRequest(new ModifiableSolrParams(params), user, pass);
+                QueryResponse rsp = qreq.process(client, null);
                 if (verifyStress) {
                   compareResponses(rsp, controlRsp);
                 }
@@ -636,7 +657,15 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
     }
     return rsp;
   }
-  
+
+  private QueryRequest generateQueryRequest(ModifiableSolrParams params, String user, String pass) {
+    QueryRequest req = new QueryRequest(params);
+    if (user != null && pass != null) {
+      req.setBasicAuthCredentials(user, pass);
+    }
+    return req;
+  }
+
   public QueryResponse queryAndCompare(SolrParams params, SolrClient... clients) throws SolrServerException, IOException {
     return queryAndCompare(params, Arrays.<SolrClient>asList(clients));
   }


RE: lucene-solr:master: SOLR-9640: Support PKI authentication and SSL in standalone-mode master/slave auth with local security.json

Posted by Uwe Schindler <uw...@thetaphi.de>.
Thanks!

 

-----

Uwe Schindler

Achterdiek 19, D-28357 Bremen

http://www.thetaphi.de <http://www.thetaphi.de/> 

eMail: uwe@thetaphi.de

 

From: Jan Høydahl [mailto:jan.asf@cominvent.com] 
Sent: Saturday, February 25, 2017 12:49 AM
To: dev@lucene.apache.org
Subject: Re: lucene-solr:master: SOLR-9640: Support PKI authentication and SSL in standalone-mode master/slave auth with local security.json

 

Reverted.

 

--
Jan Høydahl, search solution architect
Cominvent AS - www.cominvent.com <http://www.cominvent.com> 

 

25. feb. 2017 kl. 00.13 skrev Jan Høydahl <jan.asf@cominvent.com <ma...@cominvent.com> >:

 

Looking…

 

--
Jan Høydahl, search solution architect
Cominvent AS - www.cominvent.com <http://www.cominvent.com/> 

 

24. feb. 2017 kl. 19.48 skrev Uwe Schindler <uwe@thetaphi.de <ma...@thetaphi.de> >:

 

I have the feeling this broke Jenkins. Millions of NPEs with JDK 8u121:

 <https://jenkins.thetaphi.de/job/Lucene-Solr-master-Linux/19042/console> https://jenkins.thetaphi.de/job/Lucene-Solr-master-Linux/19042/console

130 test failures by NPE in org.apache.solr.core.CoreContainer.getNodeNameLocal()

-----
Uwe Schindler
Achterdiek 19, D-28357 Bremen
 <http://www.thetaphi.de/> http://www.thetaphi.de
eMail:  <ma...@thetaphi.de> uwe@thetaphi.de




-----Original Message-----
From: janhoy@apache.org <ma...@apache.org>  [mailto:janhoy@apache.org]
Sent: Friday, February 24, 2017 2:31 PM
To: commits@lucene.apache.org <ma...@lucene.apache.org> 
Subject: lucene-solr:master: SOLR-9640: Support PKI authentication and SSL
in standalone-mode master/slave auth with local security.json

Repository: lucene-solr
Updated Branches:
 refs/heads/master 5eeb8136f -> 95d6fc251


SOLR-9640: Support PKI authentication and SSL in standalone-mode
master/slave auth with local security.json


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-
solr/commit/95d6fc25
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/95d6fc25
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/95d6fc25

Branch: refs/heads/master
Commit: 95d6fc2512d6525b2354165553f0d6cc4d0d6310
Parents: 5eeb813
Author: Jan HC8ydahl <janhoy@apache.org <ma...@apache.org> >
Authored: Fri Feb 24 14:26:48 2017 +0100
Committer: Jan HC8ydahl <janhoy@apache.org <ma...@apache.org> >
Committed: Fri Feb 24 14:30:42 2017 +0100

----------------------------------------------------------------------
solr/CHANGES.txt                                |   2 +
.../org/apache/solr/core/CoreContainer.java     |   9 +-
.../solr/security/PKIAuthenticationPlugin.java  |  42 +++++-
.../org/apache/solr/servlet/HttpSolrCall.java   |   4 +-
.../apache/solr/servlet/SolrDispatchFilter.java |  11 +-
.../solr/security/BasicAuthDistributedTest.java | 136 +++++++++++++++++++
.../security/TestPKIAuthenticationPlugin.java   |  38 +++++-
.../solr/BaseDistributedSearchTestCase.java     |  37 ++++-
8 files changed, 260 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-
solr/blob/95d6fc25/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 0302615..2c5f0db 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -134,6 +134,8 @@ New Features
  field must both be stored=false, indexed=false, docValues=true. (Ishan
Chattopadhyaya, hossman, noble,
  shalin, yonik)

+* SOLR-9640: Support PKI authentication and SSL in standalone-mode
master/slave auth with local security.json (janhoy)
+
Bug Fixes
----------------------


http://git-wip-us.apache.org/repos/asf/lucene-
solr/blob/95d6fc25/solr/core/src/java/org/apache/solr/core/CoreContainer.
java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index e3977d7..6115562 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -497,7 +497,9 @@ public class CoreContainer {
    hostName = cfg.getNodeName();

    zkSys.initZooKeeper(this, solrHome, cfg.getCloudConfig());
-    if(isZooKeeperAware())  pkiAuthenticationPlugin = new
PKIAuthenticationPlugin(this, zkSys.getZkController().getNodeName());
+    pkiAuthenticationPlugin = isZooKeeperAware() ?
+        new PKIAuthenticationPlugin(this,
zkSys.getZkController().getNodeName()) :
+        new PKIAuthenticationPlugin(this, getNodeNameLocal());

    MDCLoggingContext.setNode(this);

@@ -618,6 +620,11 @@ public class CoreContainer {
    }
  }

+  // Builds a node name to be used with PKIAuth.
+  private String getNodeNameLocal() {
+    return
getConfig().getCloudConfig().getHost()+":"+getConfig().getCloudConfig().getS
olrHostPort()+"_solr";
+  }
+
  public void securityNodeChanged() {
    log.info("Security node changed, reloading security.json");
    reloadSecurityProperties();

http://git-wip-us.apache.org/repos/asf/lucene-
solr/blob/95d6fc25/solr/core/src/java/org/apache/solr/security/PKIAuthenti
cationPlugin.java
----------------------------------------------------------------------
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 fdd4408..d185bc9 100644
---
a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
+++
b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
@@ -22,7 +22,9 @@ import javax.servlet.ServletResponse;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletRequestWrapper;
import java.io.IOException;
+import java.io.UnsupportedEncodingException;
import java.lang.invoke.MethodHandles;
+import java.net.URLDecoder;
import java.nio.ByteBuffer;
import java.security.Principal;
import java.security.PublicKey;
@@ -193,9 +195,14 @@ public class PKIAuthenticationPlugin extends
AuthenticationPlugin implements Htt
  }

  PublicKey getRemotePublicKey(String nodename) {
-    String url =
cores.getZkController().getZkStateReader().getBaseUrlForNodeName(nodena
me);
+    String url, uri = null;
+    if (cores.isZooKeeperAware()) {
+      url =
cores.getZkController().getZkStateReader().getBaseUrlForNodeName(nodena
me);
+    } else {
+      url = getBaseUrlForNodeNameLocal(nodename);
+    }
    try {
-      String uri = url + PATH + "?wt=json&omitHeader=true";
+      uri += PATH + "?wt=json&omitHeader=true";
      log.debug("Fetching fresh public key from : {}",uri);
      HttpResponse rsp = cores.getUpdateShardHandler().getHttpClient()
          .execute(new HttpGet(uri),
HttpClientUtil.createNewHttpClientRequestContext());
@@ -212,12 +219,41 @@ public class PKIAuthenticationPlugin extends
AuthenticationPlugin implements Htt
      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 : " + uri, e);
      return null;
    }

  }

+  protected String getBaseUrlForNodeNameLocal(String nodeName) {
+    final int _offset = nodeName.indexOf("_");
+    if (_offset < 0) {
+      throw new IllegalArgumentException("nodeName does not contain
expected '_' seperator: " + nodeName);
+    }
+    final String hostAndPort = nodeName.substring(0,_offset);
+    try {
+      final String path = URLDecoder.decode(nodeName.substring(1+_offset),
"UTF-8");
+      // TODO: Find a better way of resolving urlScheme when not using ZK?
+      String urlScheme = resolveUrlScheme();
+      return urlScheme + "://" + hostAndPort + (path.isEmpty() ? "" : ("/" +
path));
+    } catch (UnsupportedEncodingException e) {
+      throw new IllegalStateException("JVM Does not seem to support UTF-8",
e);
+    }
+  }
+
+  /**
+   * Resolve urlScheme first from sysProp "urlScheme", if not set or invalid
value, peek at ssl sysProps
+   * @return "https" if SSL is enabled, else "http"
+   */
+  protected static String resolveUrlScheme() {
+    String urlScheme = System.getProperty("urlScheme");
+    if (urlScheme != null && urlScheme.matches("https?")) {
+      return urlScheme;
+    } else {
+      return System.getProperty("solr.jetty.keystore") == null ? "http" :
"https";
+    }
+  }
+
  @Override
  public SolrHttpClientBuilder getHttpClientBuilder(SolrHttpClientBuilder
builder) {
    HttpClientUtil.addRequestInterceptor(interceptor);

 <http://git-wip-us.apache.org/repos/asf/lucene-> http://git-wip-us.apache.org/repos/asf/lucene-
solr/blob/95d6fc25/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.j
ava
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
index 4f6bae0..0dfb0ea 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -560,7 +560,7 @@ public class HttpSolrCall {
  }

  private boolean shouldAuthorize() {
-    if(PKIAuthenticationPlugin.PATH.equals(path)) return false;
+    if(path != null && path.endsWith(PKIAuthenticationPlugin.PATH)) return
false;
    //admin/info/key is the path where public key is exposed . it is always
unsecured
    if (cores.getPkiAuthenticationPlugin() != null && req.getUserPrincipal() !=
null) {
      boolean b = cores.getPkiAuthenticationPlugin().needsAuthorization(req);
@@ -1081,7 +1081,7 @@ public class HttpSolrCall {
          response.delete(response.length() - 1, response.length());

        response.append("], Path: [").append(resource).append("]");
-        response.append(" path : ").append(path).append(" params
:").append(solrReq.getParams());
+        response.append(" path : ").append(path).append(" params
:").append(solrReq == null ? null : solrReq.getParams());
        return response.toString();
      }


 <http://git-wip-us.apache.org/repos/asf/lucene-> http://git-wip-us.apache.org/repos/asf/lucene-
solr/blob/95d6fc25/solr/core/src/java/org/apache/solr/servlet/SolrDispatch
Filter.java
----------------------------------------------------------------------
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 ce65069..4ce57b0 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -402,11 +402,11 @@ public class SolrDispatchFilter extends
BaseSolrFilter {
    if (authenticationPlugin == null) {
      return true;
    } else {
-      // /admin/info/key must be always open. see SOLR-9188
-      // tests work only w/ getPathInfo
-      //otherwise it's just enough to have getServletPath()
-      if (PKIAuthenticationPlugin.PATH.equals(((HttpServletRequest)
request).getServletPath()) ||
-          PKIAuthenticationPlugin.PATH.equals(((HttpServletRequest)
request).getPathInfo())) return true;
+      String requestUri = ((HttpServletRequest) request).getRequestURI();
+      if (requestUri != null &&
requestUri.endsWith(PKIAuthenticationPlugin.PATH)) {
+        log.debug("Passthrough of pki URL " + requestUri);
+        return true;
+      }
      String header = ((HttpServletRequest)
request).getHeader(PKIAuthenticationPlugin.HEADER);
      if (header != null && cores.getPkiAuthenticationPlugin() != null)
        authenticationPlugin = cores.getPkiAuthenticationPlugin();
@@ -418,7 +418,6 @@ public class SolrDispatchFilter extends BaseSolrFilter {
          wrappedRequest.set(req);
        });
      } catch (Exception e) {
-        log.info("Error authenticating", e);
        throw new SolrException(ErrorCode.SERVER_ERROR, "Error during
request authentication, ", e);
      }
    }

 <http://git-wip-us.apache.org/repos/asf/lucene-> http://git-wip-us.apache.org/repos/asf/lucene-
solr/blob/95d6fc25/solr/core/src/test/org/apache/solr/security/BasicAuthDi
stributedTest.java
----------------------------------------------------------------------
diff --git
a/solr/core/src/test/org/apache/solr/security/BasicAuthDistributedTest.java
b/solr/core/src/test/org/apache/solr/security/BasicAuthDistributedTest.java
new file mode 100644
index 0000000..e35e369
--- /dev/null
+++
b/solr/core/src/test/org/apache/solr/security/BasicAuthDistributedTest.java
@@ -0,0 +1,136 @@
+/*
+ * 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> 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.nio.file.Files;
+import java.nio.file.Paths;
+
+import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.solr.BaseDistributedSearchTestCase;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.admin.SecurityConfHandler;
+import org.apache.solr.handler.admin.SecurityConfHandlerLocalForTesting;
+import org.apache.solr.util.LogLevel;
+import org.junit.Test;
+
+/**
+ * Tests basicAuth in a multi shard env
+ */
+@Slow
+public class BasicAuthDistributedTest extends
BaseDistributedSearchTestCase {
+  public BasicAuthDistributedTest() {
+    super();
+    schemaString = "schema.xml";
+  }
+
+  private SecurityConfHandlerLocalForTesting securityConfHandler;
+
+  @Test
+  @LogLevel("org.apache.solr=DEBUG")
+  public void test() throws Exception {
+    index();
+    testAuth();
+  }
+
+  private void index() throws Exception {
+    del("*:*");
+    indexr(id, "1", "text", "doc one");
+    indexr(id, "2", "text", "doc two");
+    indexr(id, "3", "text", "doc three");
+    indexr(id, "4", "text", "doc four");
+    indexr(id, "5", "text", "doc five");
+
+    commit();  // try to ensure there's more than one segment
+
+    indexr(id, "6", "text", "doc six");
+    indexr(id, "7", "text", "doc seven");
+    indexr(id, "8", "text", "doc eight");
+    indexr(id, "9", "text", "doc nine");
+    indexr(id, "10", "text", "doc ten");
+
+    commit();
+
+    handle.clear();
+    handle.put("QTime", SKIPVAL);
+    handle.put("timestamp", SKIPVAL);
+    handle.put("maxScore", SKIPVAL);
+    handle.put("_version_", SKIPVAL);
+  }
+
+  private void testAuth() throws Exception {
+    QueryResponse rsp = query("q","text:doc", "fl", "id,text", "sort", "id asc");
+    assertEquals(10, rsp.getResults().getNumFound());
+
+    // Enable authentication
+    for (JettySolrRunner j : jettys) {
+      writeSecurityJson(j.getCoreContainer());
+    }
+
+    HttpSolrClient.RemoteSolrException expected =
expectThrows(HttpSolrClient.RemoteSolrException.class, () -> {
+      query("q","text:doc-fail", "fl", "id,text", "sort", "id asc");
+    });
+    assertEquals(401, expected.code());
+
+    // Add auth
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.add("q", "text:doc").add("fl", "id,text").add("sort", "id asc");
+    QueryRequest req = new QueryRequest(params);
+    req.setBasicAuthCredentials("solr", "SolrRocks");
+    rsp = req.process(clients.get(0), null);
+    if (jettys.size() > 1) {
+      assertTrue(rsp.getResults().getNumFound() < 10);
+      rsp = query(true, params, "solr", "SolrRocks");
+    }
+    assertEquals(10, rsp.getResults().getNumFound());
+
+    // Disable auth
+    for (JettySolrRunner j : jettys) {
+      deleteSecurityJson(j.getCoreContainer());
+    }
+
+  }
+
+  private void deleteSecurityJson(CoreContainer coreContainer) throws
IOException {
+    securityConfHandler = new
SecurityConfHandlerLocalForTesting(coreContainer);
+
Files.delete(Paths.get(coreContainer.getSolrHome()).resolve("security.json"))
;
+    coreContainer.securityNodeChanged();
+  }
+
+  private void writeSecurityJson(CoreContainer coreContainer) throws
IOException {
+    securityConfHandler = new
SecurityConfHandlerLocalForTesting(coreContainer);
+    securityConfHandler.persistConf(new
SecurityConfHandler.SecurityConfig()
+        .setData(Utils.fromJSONString(ALL_CONF.replaceAll("'", "\""))));
+    coreContainer.securityNodeChanged();
+  }
+
+  protected static final String ALL_CONF = "{\n" +
+      "  'authentication':{\n" +
+      "    'blockUnknown':true,\n" +
+      "    'class':'solr.BasicAuthPlugin',\n" +
+      "
'credentials':{'solr':'orwp2Ghgj39lmnrZOTm7Qtre1VqHFDfwAEzr0ApbN3Y=
Ju5osoAqOX8iafhWpPP01E5P+sg8tK8tHON7rCYZRRw='}},\n" +
+      "  'authorization':{\n" +
+      "    'class':'solr.RuleBasedAuthorizationPlugin',\n" +
+      "    'user-role':{'solr':'admin'},\n" +
+      "    'permissions':[{'name':'all','role':'admin'}]}}";
+}

 <http://git-wip-us.apache.org/repos/asf/lucene-> http://git-wip-us.apache.org/repos/asf/lucene-
solr/blob/95d6fc25/solr/core/src/test/org/apache/solr/security/TestPKIAuth
enticationPlugin.java
----------------------------------------------------------------------
diff --git
a/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.ja
va
b/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.j
ava
index a5a279f..90c5bd2 100644
---
a/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.ja
va
+++
b/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.j
ava
@@ -35,7 +35,11 @@ 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 static org.mockito.Mockito.*;
+import org.junit.Test;
+
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;

public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {

@@ -141,10 +145,38 @@ public class TestPKIAuthenticationPlugin extends
SolrTestCaseJ4 {
    mock1.doAuthenticate(mockReq, null,filterChain );
    assertNotNull(wrappedRequestByFilter.get());
    assertEquals("$", ((HttpServletRequest)
wrappedRequestByFilter.get()).getUserPrincipal().getName());
+  }

+  @Test
+  public void testGetBaseUrlForNodeNameLocal() {
+    synchronized (this) {
+      final MockPKIAuthenticationPlugin mock = new
MockPKIAuthenticationPlugin(null, "myName");
+      System.clearProperty("solr.jetty.keystore");
+      assertEquals(" <http://my.host:9876/solr2> http://my.host:9876/solr2",
mock.getBaseUrlForNodeNameLocal("my.host:9876_solr2"));
+      System.setProperty("solr.jetty.keystore", "foo");
+      assertEquals(" <https://my.host:9876/solr2> https://my.host:9876/solr2",
mock.getBaseUrlForNodeNameLocal("my.host:9876_solr2"));
+      System.clearProperty("solr.jetty.keystore");
+    }
+  }

-
-
+  @Test
+  public void testResolveUrlScheme() {
+    synchronized (this) {
+      System.clearProperty("urlScheme");
+      System.clearProperty("solr.jetty.keystore");
+      assertEquals("http", MockPKIAuthenticationPlugin.resolveUrlScheme());
+      System.setProperty("urlScheme", "http");
+      assertEquals("http", MockPKIAuthenticationPlugin.resolveUrlScheme());
+      System.setProperty("urlScheme", "https");
+      assertEquals("https", MockPKIAuthenticationPlugin.resolveUrlScheme());
+      System.setProperty("urlScheme", "ftp");
+      System.clearProperty("solr.jetty.keystore");
+      assertEquals("http", MockPKIAuthenticationPlugin.resolveUrlScheme());
+      System.setProperty("solr.jetty.keystore", "foo");
+      assertEquals("https", MockPKIAuthenticationPlugin.resolveUrlScheme());
+      System.clearProperty("urlScheme");
+      System.clearProperty("solr.jetty.keystore");
+    }
  }

  private HttpServletRequest createMockRequest(final
AtomicReference<Header> header) {

 <http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/95d6fc25/solr/test-> http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/95d6fc25/solr/test-
framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
----------------------------------------------------------------------
diff --git a/solr/test-
framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
b/solr/test-
framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
index 8c6eb60..bbfc048 100644
--- a/solr/test-
framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
+++ b/solr/test-
framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
@@ -50,6 +50,7 @@ import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.embedded.JettyConfig;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.client.solrj.response.UpdateResponse;
@@ -558,6 +559,12 @@ public abstract class BaseDistributedSearchTestCase
extends SolrTestCaseJ4 {
    return rsp;
  }

+  protected QueryResponse queryServer(QueryRequest req) throws
IOException, SolrServerException {
+    int which = r.nextInt(clients.size());
+    SolrClient client = clients.get(which);
+    return req.process(client, null);
+  }
+
  /**
   * Sets distributed params.
   * Returns the QueryResponse from {@link #queryServer},
@@ -591,18 +598,31 @@ public abstract class
BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
   * Returns the QueryResponse from {@link #queryServer}
   */
  protected QueryResponse query(boolean setDistribParams, SolrParams p)
throws Exception {
+    return query(setDistribParams, p, null, null);
+  }
+
+  /**
+   * Returns the QueryResponse from {@link #queryServer}
+   * @param setDistribParams whether to do a distributed request
+   * @param user basic auth username (set to null if not in use)
+   * @param pass basic auth password (set to null if not in use)
+   * @return the query response
+   */
+  protected QueryResponse query(boolean setDistribParams, SolrParams p,
String user, String pass) throws Exception {

    final ModifiableSolrParams params = new ModifiableSolrParams(p);

    // TODO: look into why passing true causes fails
    params.set("distrib", "false");
-    final QueryResponse controlRsp = controlClient.query(params);
+    QueryRequest req = generateQueryRequest(params, user, pass);
+    final QueryResponse controlRsp = req.process(controlClient, null);
    validateControlData(controlRsp);

    params.remove("distrib");
    if (setDistribParams) setDistributedParams(params);
+    req = generateQueryRequest(params, user, pass);

-    QueryResponse rsp = queryServer(params);
+    QueryResponse rsp = queryServer(req);

    compareResponses(rsp, controlRsp);

@@ -617,7 +637,8 @@ public abstract class BaseDistributedSearchTestCase
extends SolrTestCaseJ4 {
              int which = r.nextInt(clients.size());
              SolrClient client = clients.get(which);
              try {
-                QueryResponse rsp = client.query(new
ModifiableSolrParams(params));
+                QueryRequest qreq = generateQueryRequest(new
ModifiableSolrParams(params), user, pass);
+                QueryResponse rsp = qreq.process(client, null);
                if (verifyStress) {
                  compareResponses(rsp, controlRsp);
                }
@@ -636,7 +657,15 @@ public abstract class BaseDistributedSearchTestCase
extends SolrTestCaseJ4 {
    }
    return rsp;
  }
-
+
+  private QueryRequest generateQueryRequest(ModifiableSolrParams
params, String user, String pass) {
+    QueryRequest req = new QueryRequest(params);
+    if (user != null && pass != null) {
+      req.setBasicAuthCredentials(user, pass);
+    }
+    return req;
+  }
+
  public QueryResponse queryAndCompare(SolrParams params, SolrClient...
clients) throws SolrServerException, IOException {
    return queryAndCompare(params, Arrays.<SolrClient>asList(clients));
  }



---------------------------------------------------------------------
To unsubscribe, e-mail:  <ma...@lucene.apache.org> dev-unsubscribe@lucene.apache.org
For additional commands, e-mail:  <ma...@lucene.apache.org> dev-help@lucene.apache.org

 

 


Re: lucene-solr:master: SOLR-9640: Support PKI authentication and SSL in standalone-mode master/slave auth with local security.json

Posted by Jan Høydahl <ja...@cominvent.com>.
Reverted.

--
Jan Høydahl, search solution architect
Cominvent AS - www.cominvent.com

> 25. feb. 2017 kl. 00.13 skrev Jan Høydahl <ja...@cominvent.com>:
> 
> Looking…
> 
> --
> Jan Høydahl, search solution architect
> Cominvent AS - www.cominvent.com <http://www.cominvent.com/>
> 
>> 24. feb. 2017 kl. 19.48 skrev Uwe Schindler <uwe@thetaphi.de <ma...@thetaphi.de>>:
>> 
>> I have the feeling this broke Jenkins. Millions of NPEs with JDK 8u121:
>> 
>> https://jenkins.thetaphi.de/job/Lucene-Solr-master-Linux/19042/console <https://jenkins.thetaphi.de/job/Lucene-Solr-master-Linux/19042/console>
>> 
>> 130 test failures by NPE in org.apache.solr.core.CoreContainer.getNodeNameLocal()
>> 
>> -----
>> Uwe Schindler
>> Achterdiek 19, D-28357 Bremen
>> http://www.thetaphi.de <http://www.thetaphi.de/>
>> eMail: uwe@thetaphi.de <ma...@thetaphi.de>
>> 
>>> -----Original Message-----
>>> From: janhoy@apache.org <ma...@apache.org> [mailto:janhoy@apache.org <ma...@apache.org>]
>>> Sent: Friday, February 24, 2017 2:31 PM
>>> To: commits@lucene.apache.org <ma...@lucene.apache.org>
>>> Subject: lucene-solr:master: SOLR-9640: Support PKI authentication and SSL
>>> in standalone-mode master/slave auth with local security.json
>>> 
>>> Repository: lucene-solr
>>> Updated Branches:
>>>  refs/heads/master 5eeb8136f -> 95d6fc251
>>> 
>>> 
>>> SOLR-9640: Support PKI authentication and SSL in standalone-mode
>>> master/slave auth with local security.json
>>> 
>>> 
>>> Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo <http://git-wip-us.apache.org/repos/asf/lucene-solr/repo>
>>> Commit: http://git-wip-us.apache.org/repos/asf/lucene- <http://git-wip-us.apache.org/repos/asf/lucene->
>>> solr/commit/95d6fc25
>>> Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/95d6fc25 <http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/95d6fc25>
>>> Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/95d6fc25 <http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/95d6fc25>
>>> 
>>> Branch: refs/heads/master
>>> Commit: 95d6fc2512d6525b2354165553f0d6cc4d0d6310
>>> Parents: 5eeb813
>>> Author: Jan HC8ydahl <janhoy@apache.org <ma...@apache.org>>
>>> Authored: Fri Feb 24 14:26:48 2017 +0100
>>> Committer: Jan HC8ydahl <janhoy@apache.org <ma...@apache.org>>
>>> Committed: Fri Feb 24 14:30:42 2017 +0100
>>> 
>>> ----------------------------------------------------------------------
>>> solr/CHANGES.txt                                |   2 +
>>> .../org/apache/solr/core/CoreContainer.java     |   9 +-
>>> .../solr/security/PKIAuthenticationPlugin.java  |  42 +++++-
>>> .../org/apache/solr/servlet/HttpSolrCall.java   |   4 +-
>>> .../apache/solr/servlet/SolrDispatchFilter.java |  11 +-
>>> .../solr/security/BasicAuthDistributedTest.java | 136 +++++++++++++++++++
>>> .../security/TestPKIAuthenticationPlugin.java   |  38 +++++-
>>> .../solr/BaseDistributedSearchTestCase.java     |  37 ++++-
>>> 8 files changed, 260 insertions(+), 19 deletions(-)
>>> ----------------------------------------------------------------------
>>> 
>>> 
>>> http://git-wip-us.apache.org/repos/asf/lucene- <http://git-wip-us.apache.org/repos/asf/lucene->
>>> solr/blob/95d6fc25/solr/CHANGES.txt
>>> ----------------------------------------------------------------------
>>> diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
>>> index 0302615..2c5f0db 100644
>>> --- a/solr/CHANGES.txt
>>> +++ b/solr/CHANGES.txt
>>> @@ -134,6 +134,8 @@ New Features
>>>   field must both be stored=false, indexed=false, docValues=true. (Ishan
>>> Chattopadhyaya, hossman, noble,
>>>   shalin, yonik)
>>> 
>>> +* SOLR-9640: Support PKI authentication and SSL in standalone-mode
>>> master/slave auth with local security.json (janhoy)
>>> +
>>> Bug Fixes
>>> ----------------------
>>> 
>>> 
>>> http://git-wip-us.apache.org/repos/asf/lucene-
>>> solr/blob/95d6fc25/solr/core/src/java/org/apache/solr/core/CoreContainer.
>>> java
>>> ----------------------------------------------------------------------
>>> diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
>>> b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
>>> index e3977d7..6115562 100644
>>> --- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
>>> +++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
>>> @@ -497,7 +497,9 @@ public class CoreContainer {
>>>     hostName = cfg.getNodeName();
>>> 
>>>     zkSys.initZooKeeper(this, solrHome, cfg.getCloudConfig());
>>> -    if(isZooKeeperAware())  pkiAuthenticationPlugin = new
>>> PKIAuthenticationPlugin(this, zkSys.getZkController().getNodeName());
>>> +    pkiAuthenticationPlugin = isZooKeeperAware() ?
>>> +        new PKIAuthenticationPlugin(this,
>>> zkSys.getZkController().getNodeName()) :
>>> +        new PKIAuthenticationPlugin(this, getNodeNameLocal());
>>> 
>>>     MDCLoggingContext.setNode(this);
>>> 
>>> @@ -618,6 +620,11 @@ public class CoreContainer {
>>>     }
>>>   }
>>> 
>>> +  // Builds a node name to be used with PKIAuth.
>>> +  private String getNodeNameLocal() {
>>> +    return
>>> getConfig().getCloudConfig().getHost()+":"+getConfig().getCloudConfig().getS
>>> olrHostPort()+"_solr";
>>> +  }
>>> +
>>>   public void securityNodeChanged() {
>>>     log.info("Security node changed, reloading security.json");
>>>     reloadSecurityProperties();
>>> 
>>> http://git-wip-us.apache.org/repos/asf/lucene-
>>> solr/blob/95d6fc25/solr/core/src/java/org/apache/solr/security/PKIAuthenti
>>> cationPlugin.java
>>> ----------------------------------------------------------------------
>>> 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 fdd4408..d185bc9 100644
>>> ---
>>> a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
>>> +++
>>> b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
>>> @@ -22,7 +22,9 @@ import javax.servlet.ServletResponse;
>>> import javax.servlet.http.HttpServletRequest;
>>> import javax.servlet.http.HttpServletRequestWrapper;
>>> import java.io.IOException;
>>> +import java.io.UnsupportedEncodingException;
>>> import java.lang.invoke.MethodHandles;
>>> +import java.net.URLDecoder;
>>> import java.nio.ByteBuffer;
>>> import java.security.Principal;
>>> import java.security.PublicKey;
>>> @@ -193,9 +195,14 @@ public class PKIAuthenticationPlugin extends
>>> AuthenticationPlugin implements Htt
>>>   }
>>> 
>>>   PublicKey getRemotePublicKey(String nodename) {
>>> -    String url =
>>> cores.getZkController().getZkStateReader().getBaseUrlForNodeName(nodena
>>> me);
>>> +    String url, uri = null;
>>> +    if (cores.isZooKeeperAware()) {
>>> +      url =
>>> cores.getZkController().getZkStateReader().getBaseUrlForNodeName(nodena
>>> me);
>>> +    } else {
>>> +      url = getBaseUrlForNodeNameLocal(nodename);
>>> +    }
>>>     try {
>>> -      String uri = url + PATH + "?wt=json&omitHeader=true";
>>> +      uri += PATH + "?wt=json&omitHeader=true";
>>>       log.debug("Fetching fresh public key from : {}",uri);
>>>       HttpResponse rsp = cores.getUpdateShardHandler().getHttpClient()
>>>           .execute(new HttpGet(uri),
>>> HttpClientUtil.createNewHttpClientRequestContext());
>>> @@ -212,12 +219,41 @@ public class PKIAuthenticationPlugin extends
>>> AuthenticationPlugin implements Htt
>>>       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 : " + uri, e);
>>>       return null;
>>>     }
>>> 
>>>   }
>>> 
>>> +  protected String getBaseUrlForNodeNameLocal(String nodeName) {
>>> +    final int _offset = nodeName.indexOf("_");
>>> +    if (_offset < 0) {
>>> +      throw new IllegalArgumentException("nodeName does not contain
>>> expected '_' seperator: " + nodeName);
>>> +    }
>>> +    final String hostAndPort = nodeName.substring(0,_offset);
>>> +    try {
>>> +      final String path = URLDecoder.decode(nodeName.substring(1+_offset),
>>> "UTF-8");
>>> +      // TODO: Find a better way of resolving urlScheme when not using ZK?
>>> +      String urlScheme = resolveUrlScheme();
>>> +      return urlScheme + "://" + hostAndPort + (path.isEmpty() ? "" : ("/" +
>>> path));
>>> +    } catch (UnsupportedEncodingException e) {
>>> +      throw new IllegalStateException("JVM Does not seem to support UTF-8",
>>> e);
>>> +    }
>>> +  }
>>> +
>>> +  /**
>>> +   * Resolve urlScheme first from sysProp "urlScheme", if not set or invalid
>>> value, peek at ssl sysProps
>>> +   * @return "https" if SSL is enabled, else "http"
>>> +   */
>>> +  protected static String resolveUrlScheme() {
>>> +    String urlScheme = System.getProperty("urlScheme");
>>> +    if (urlScheme != null && urlScheme.matches("https?")) {
>>> +      return urlScheme;
>>> +    } else {
>>> +      return System.getProperty("solr.jetty.keystore") == null ? "http" :
>>> "https";
>>> +    }
>>> +  }
>>> +
>>>   @Override
>>>   public SolrHttpClientBuilder getHttpClientBuilder(SolrHttpClientBuilder
>>> builder) {
>>>     HttpClientUtil.addRequestInterceptor(interceptor);
>>> 
>>> http://git-wip-us.apache.org/repos/asf/lucene-
>>> solr/blob/95d6fc25/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.j
>>> ava
>>> ----------------------------------------------------------------------
>>> diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
>>> b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
>>> index 4f6bae0..0dfb0ea 100644
>>> --- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
>>> +++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
>>> @@ -560,7 +560,7 @@ public class HttpSolrCall {
>>>   }
>>> 
>>>   private boolean shouldAuthorize() {
>>> -    if(PKIAuthenticationPlugin.PATH.equals(path)) return false;
>>> +    if(path != null && path.endsWith(PKIAuthenticationPlugin.PATH)) return
>>> false;
>>>     //admin/info/key is the path where public key is exposed . it is always
>>> unsecured
>>>     if (cores.getPkiAuthenticationPlugin() != null && req.getUserPrincipal() !=
>>> null) {
>>>       boolean b = cores.getPkiAuthenticationPlugin().needsAuthorization(req);
>>> @@ -1081,7 +1081,7 @@ public class HttpSolrCall {
>>>           response.delete(response.length() - 1, response.length());
>>> 
>>>         response.append("], Path: [").append(resource).append("]");
>>> -        response.append(" path : ").append(path).append(" params
>>> :").append(solrReq.getParams());
>>> +        response.append(" path : ").append(path).append(" params
>>> :").append(solrReq == null ? null : solrReq.getParams());
>>>         return response.toString();
>>>       }
>>> 
>>> 
>>> http://git-wip-us.apache.org/repos/asf/lucene-
>>> solr/blob/95d6fc25/solr/core/src/java/org/apache/solr/servlet/SolrDispatch
>>> Filter.java
>>> ----------------------------------------------------------------------
>>> 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 ce65069..4ce57b0 100644
>>> --- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
>>> +++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
>>> @@ -402,11 +402,11 @@ public class SolrDispatchFilter extends
>>> BaseSolrFilter {
>>>     if (authenticationPlugin == null) {
>>>       return true;
>>>     } else {
>>> -      // /admin/info/key must be always open. see SOLR-9188
>>> -      // tests work only w/ getPathInfo
>>> -      //otherwise it's just enough to have getServletPath()
>>> -      if (PKIAuthenticationPlugin.PATH.equals(((HttpServletRequest)
>>> request).getServletPath()) ||
>>> -          PKIAuthenticationPlugin.PATH.equals(((HttpServletRequest)
>>> request).getPathInfo())) return true;
>>> +      String requestUri = ((HttpServletRequest) request).getRequestURI();
>>> +      if (requestUri != null &&
>>> requestUri.endsWith(PKIAuthenticationPlugin.PATH)) {
>>> +        log.debug("Passthrough of pki URL " + requestUri);
>>> +        return true;
>>> +      }
>>>       String header = ((HttpServletRequest)
>>> request).getHeader(PKIAuthenticationPlugin.HEADER);
>>>       if (header != null && cores.getPkiAuthenticationPlugin() != null)
>>>         authenticationPlugin = cores.getPkiAuthenticationPlugin();
>>> @@ -418,7 +418,6 @@ public class SolrDispatchFilter extends BaseSolrFilter {
>>>           wrappedRequest.set(req);
>>>         });
>>>       } catch (Exception e) {
>>> -        log.info("Error authenticating", e);
>>>         throw new SolrException(ErrorCode.SERVER_ERROR, "Error during
>>> request authentication, ", e);
>>>       }
>>>     }
>>> 
>>> http://git-wip-us.apache.org/repos/asf/lucene-
>>> solr/blob/95d6fc25/solr/core/src/test/org/apache/solr/security/BasicAuthDi
>>> stributedTest.java
>>> ----------------------------------------------------------------------
>>> diff --git
>>> a/solr/core/src/test/org/apache/solr/security/BasicAuthDistributedTest.java
>>> b/solr/core/src/test/org/apache/solr/security/BasicAuthDistributedTest.java
>>> new file mode 100644
>>> index 0000000..e35e369
>>> --- /dev/null
>>> +++
>>> b/solr/core/src/test/org/apache/solr/security/BasicAuthDistributedTest.java
>>> @@ -0,0 +1,136 @@
>>> +/*
>>> + * 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.nio.file.Files;
>>> +import java.nio.file.Paths;
>>> +
>>> +import org.apache.lucene.util.LuceneTestCase.Slow;
>>> +import org.apache.solr.BaseDistributedSearchTestCase;
>>> +import org.apache.solr.client.solrj.embedded.JettySolrRunner;
>>> +import org.apache.solr.client.solrj.impl.HttpSolrClient;
>>> +import org.apache.solr.client.solrj.request.QueryRequest;
>>> +import org.apache.solr.client.solrj.response.QueryResponse;
>>> +import org.apache.solr.common.params.ModifiableSolrParams;
>>> +import org.apache.solr.common.util.Utils;
>>> +import org.apache.solr.core.CoreContainer;
>>> +import org.apache.solr.handler.admin.SecurityConfHandler;
>>> +import org.apache.solr.handler.admin.SecurityConfHandlerLocalForTesting;
>>> +import org.apache.solr.util.LogLevel;
>>> +import org.junit.Test;
>>> +
>>> +/**
>>> + * Tests basicAuth in a multi shard env
>>> + */
>>> +@Slow
>>> +public class BasicAuthDistributedTest extends
>>> BaseDistributedSearchTestCase {
>>> +  public BasicAuthDistributedTest() {
>>> +    super();
>>> +    schemaString = "schema.xml";
>>> +  }
>>> +
>>> +  private SecurityConfHandlerLocalForTesting securityConfHandler;
>>> +
>>> +  @Test
>>> +  @LogLevel("org.apache.solr=DEBUG")
>>> +  public void test() throws Exception {
>>> +    index();
>>> +    testAuth();
>>> +  }
>>> +
>>> +  private void index() throws Exception {
>>> +    del("*:*");
>>> +    indexr(id, "1", "text", "doc one");
>>> +    indexr(id, "2", "text", "doc two");
>>> +    indexr(id, "3", "text", "doc three");
>>> +    indexr(id, "4", "text", "doc four");
>>> +    indexr(id, "5", "text", "doc five");
>>> +
>>> +    commit();  // try to ensure there's more than one segment
>>> +
>>> +    indexr(id, "6", "text", "doc six");
>>> +    indexr(id, "7", "text", "doc seven");
>>> +    indexr(id, "8", "text", "doc eight");
>>> +    indexr(id, "9", "text", "doc nine");
>>> +    indexr(id, "10", "text", "doc ten");
>>> +
>>> +    commit();
>>> +
>>> +    handle.clear();
>>> +    handle.put("QTime", SKIPVAL);
>>> +    handle.put("timestamp", SKIPVAL);
>>> +    handle.put("maxScore", SKIPVAL);
>>> +    handle.put("_version_", SKIPVAL);
>>> +  }
>>> +
>>> +  private void testAuth() throws Exception {
>>> +    QueryResponse rsp = query("q","text:doc", "fl", "id,text", "sort", "id asc");
>>> +    assertEquals(10, rsp.getResults().getNumFound());
>>> +
>>> +    // Enable authentication
>>> +    for (JettySolrRunner j : jettys) {
>>> +      writeSecurityJson(j.getCoreContainer());
>>> +    }
>>> +
>>> +    HttpSolrClient.RemoteSolrException expected =
>>> expectThrows(HttpSolrClient.RemoteSolrException.class, () -> {
>>> +      query("q","text:doc-fail", "fl", "id,text", "sort", "id asc");
>>> +    });
>>> +    assertEquals(401, expected.code());
>>> +
>>> +    // Add auth
>>> +    ModifiableSolrParams params = new ModifiableSolrParams();
>>> +    params.add("q", "text:doc").add("fl", "id,text").add("sort", "id asc");
>>> +    QueryRequest req = new QueryRequest(params);
>>> +    req.setBasicAuthCredentials("solr", "SolrRocks");
>>> +    rsp = req.process(clients.get(0), null);
>>> +    if (jettys.size() > 1) {
>>> +      assertTrue(rsp.getResults().getNumFound() < 10);
>>> +      rsp = query(true, params, "solr", "SolrRocks");
>>> +    }
>>> +    assertEquals(10, rsp.getResults().getNumFound());
>>> +
>>> +    // Disable auth
>>> +    for (JettySolrRunner j : jettys) {
>>> +      deleteSecurityJson(j.getCoreContainer());
>>> +    }
>>> +
>>> +  }
>>> +
>>> +  private void deleteSecurityJson(CoreContainer coreContainer) throws
>>> IOException {
>>> +    securityConfHandler = new
>>> SecurityConfHandlerLocalForTesting(coreContainer);
>>> +
>>> Files.delete(Paths.get(coreContainer.getSolrHome()).resolve("security.json"))
>>> ;
>>> +    coreContainer.securityNodeChanged();
>>> +  }
>>> +
>>> +  private void writeSecurityJson(CoreContainer coreContainer) throws
>>> IOException {
>>> +    securityConfHandler = new
>>> SecurityConfHandlerLocalForTesting(coreContainer);
>>> +    securityConfHandler.persistConf(new
>>> SecurityConfHandler.SecurityConfig()
>>> +        .setData(Utils.fromJSONString(ALL_CONF.replaceAll("'", "\""))));
>>> +    coreContainer.securityNodeChanged();
>>> +  }
>>> +
>>> +  protected static final String ALL_CONF = "{\n" +
>>> +      "  'authentication':{\n" +
>>> +      "    'blockUnknown':true,\n" +
>>> +      "    'class':'solr.BasicAuthPlugin',\n" +
>>> +      "
>>> 'credentials':{'solr':'orwp2Ghgj39lmnrZOTm7Qtre1VqHFDfwAEzr0ApbN3Y=
>>> Ju5osoAqOX8iafhWpPP01E5P+sg8tK8tHON7rCYZRRw='}},\n" +
>>> +      "  'authorization':{\n" +
>>> +      "    'class':'solr.RuleBasedAuthorizationPlugin',\n" +
>>> +      "    'user-role':{'solr':'admin'},\n" +
>>> +      "    'permissions':[{'name':'all','role':'admin'}]}}";
>>> +}
>>> 
>>> http://git-wip-us.apache.org/repos/asf/lucene-
>>> solr/blob/95d6fc25/solr/core/src/test/org/apache/solr/security/TestPKIAuth
>>> enticationPlugin.java
>>> ----------------------------------------------------------------------
>>> diff --git
>>> a/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.ja
>>> va
>>> b/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.j
>>> ava
>>> index a5a279f..90c5bd2 100644
>>> ---
>>> a/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.ja
>>> va
>>> +++
>>> b/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.j
>>> ava
>>> @@ -35,7 +35,11 @@ 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 static org.mockito.Mockito.*;
>>> +import org.junit.Test;
>>> +
>>> +import static org.mockito.Mockito.any;
>>> +import static org.mockito.Mockito.mock;
>>> +import static org.mockito.Mockito.when;
>>> 
>>> public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
>>> 
>>> @@ -141,10 +145,38 @@ public class TestPKIAuthenticationPlugin extends
>>> SolrTestCaseJ4 {
>>>     mock1.doAuthenticate(mockReq, null,filterChain );
>>>     assertNotNull(wrappedRequestByFilter.get());
>>>     assertEquals("$", ((HttpServletRequest)
>>> wrappedRequestByFilter.get()).getUserPrincipal().getName());
>>> +  }
>>> 
>>> +  @Test
>>> +  public void testGetBaseUrlForNodeNameLocal() {
>>> +    synchronized (this) {
>>> +      final MockPKIAuthenticationPlugin mock = new
>>> MockPKIAuthenticationPlugin(null, "myName");
>>> +      System.clearProperty("solr.jetty.keystore");
>>> +      assertEquals("http://my.host:9876/solr2",
>>> mock.getBaseUrlForNodeNameLocal("my.host:9876_solr2"));
>>> +      System.setProperty("solr.jetty.keystore", "foo");
>>> +      assertEquals("https://my.host:9876/solr2",
>>> mock.getBaseUrlForNodeNameLocal("my.host:9876_solr2"));
>>> +      System.clearProperty("solr.jetty.keystore");
>>> +    }
>>> +  }
>>> 
>>> -
>>> -
>>> +  @Test
>>> +  public void testResolveUrlScheme() {
>>> +    synchronized (this) {
>>> +      System.clearProperty("urlScheme");
>>> +      System.clearProperty("solr.jetty.keystore");
>>> +      assertEquals("http", MockPKIAuthenticationPlugin.resolveUrlScheme());
>>> +      System.setProperty("urlScheme", "http");
>>> +      assertEquals("http", MockPKIAuthenticationPlugin.resolveUrlScheme());
>>> +      System.setProperty("urlScheme", "https");
>>> +      assertEquals("https", MockPKIAuthenticationPlugin.resolveUrlScheme());
>>> +      System.setProperty("urlScheme", "ftp");
>>> +      System.clearProperty("solr.jetty.keystore");
>>> +      assertEquals("http", MockPKIAuthenticationPlugin.resolveUrlScheme());
>>> +      System.setProperty("solr.jetty.keystore", "foo");
>>> +      assertEquals("https", MockPKIAuthenticationPlugin.resolveUrlScheme());
>>> +      System.clearProperty("urlScheme");
>>> +      System.clearProperty("solr.jetty.keystore");
>>> +    }
>>>   }
>>> 
>>>   private HttpServletRequest createMockRequest(final
>>> AtomicReference<Header> header) {
>>> 
>>> http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/95d6fc25/solr/test-
>>> framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
>>> ----------------------------------------------------------------------
>>> diff --git a/solr/test-
>>> framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
>>> b/solr/test-
>>> framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
>>> index 8c6eb60..bbfc048 100644
>>> --- a/solr/test-
>>> framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
>>> +++ b/solr/test-
>>> framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
>>> @@ -50,6 +50,7 @@ import org.apache.solr.client.solrj.SolrServerException;
>>> import org.apache.solr.client.solrj.embedded.JettyConfig;
>>> import org.apache.solr.client.solrj.embedded.JettySolrRunner;
>>> import org.apache.solr.client.solrj.impl.HttpSolrClient;
>>> +import org.apache.solr.client.solrj.request.QueryRequest;
>>> import org.apache.solr.client.solrj.request.UpdateRequest;
>>> import org.apache.solr.client.solrj.response.QueryResponse;
>>> import org.apache.solr.client.solrj.response.UpdateResponse;
>>> @@ -558,6 +559,12 @@ public abstract class BaseDistributedSearchTestCase
>>> extends SolrTestCaseJ4 {
>>>     return rsp;
>>>   }
>>> 
>>> +  protected QueryResponse queryServer(QueryRequest req) throws
>>> IOException, SolrServerException {
>>> +    int which = r.nextInt(clients.size());
>>> +    SolrClient client = clients.get(which);
>>> +    return req.process(client, null);
>>> +  }
>>> +
>>>   /**
>>>    * Sets distributed params.
>>>    * Returns the QueryResponse from {@link #queryServer},
>>> @@ -591,18 +598,31 @@ public abstract class
>>> BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
>>>    * Returns the QueryResponse from {@link #queryServer}
>>>    */
>>>   protected QueryResponse query(boolean setDistribParams, SolrParams p)
>>> throws Exception {
>>> +    return query(setDistribParams, p, null, null);
>>> +  }
>>> +
>>> +  /**
>>> +   * Returns the QueryResponse from {@link #queryServer}
>>> +   * @param setDistribParams whether to do a distributed request
>>> +   * @param user basic auth username (set to null if not in use)
>>> +   * @param pass basic auth password (set to null if not in use)
>>> +   * @return the query response
>>> +   */
>>> +  protected QueryResponse query(boolean setDistribParams, SolrParams p,
>>> String user, String pass) throws Exception {
>>> 
>>>     final ModifiableSolrParams params = new ModifiableSolrParams(p);
>>> 
>>>     // TODO: look into why passing true causes fails
>>>     params.set("distrib", "false");
>>> -    final QueryResponse controlRsp = controlClient.query(params);
>>> +    QueryRequest req = generateQueryRequest(params, user, pass);
>>> +    final QueryResponse controlRsp = req.process(controlClient, null);
>>>     validateControlData(controlRsp);
>>> 
>>>     params.remove("distrib");
>>>     if (setDistribParams) setDistributedParams(params);
>>> +    req = generateQueryRequest(params, user, pass);
>>> 
>>> -    QueryResponse rsp = queryServer(params);
>>> +    QueryResponse rsp = queryServer(req);
>>> 
>>>     compareResponses(rsp, controlRsp);
>>> 
>>> @@ -617,7 +637,8 @@ public abstract class BaseDistributedSearchTestCase
>>> extends SolrTestCaseJ4 {
>>>               int which = r.nextInt(clients.size());
>>>               SolrClient client = clients.get(which);
>>>               try {
>>> -                QueryResponse rsp = client.query(new
>>> ModifiableSolrParams(params));
>>> +                QueryRequest qreq = generateQueryRequest(new
>>> ModifiableSolrParams(params), user, pass);
>>> +                QueryResponse rsp = qreq.process(client, null);
>>>                 if (verifyStress) {
>>>                   compareResponses(rsp, controlRsp);
>>>                 }
>>> @@ -636,7 +657,15 @@ public abstract class BaseDistributedSearchTestCase
>>> extends SolrTestCaseJ4 {
>>>     }
>>>     return rsp;
>>>   }
>>> -
>>> +
>>> +  private QueryRequest generateQueryRequest(ModifiableSolrParams
>>> params, String user, String pass) {
>>> +    QueryRequest req = new QueryRequest(params);
>>> +    if (user != null && pass != null) {
>>> +      req.setBasicAuthCredentials(user, pass);
>>> +    }
>>> +    return req;
>>> +  }
>>> +
>>>   public QueryResponse queryAndCompare(SolrParams params, SolrClient...
>>> clients) throws SolrServerException, IOException {
>>>     return queryAndCompare(params, Arrays.<SolrClient>asList(clients));
>>>   }
>> 
>> 
>> ---------------------------------------------------------------------
>> To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org <ma...@lucene.apache.org>
>> For additional commands, e-mail: dev-help@lucene.apache.org <ma...@lucene.apache.org>


Re: lucene-solr:master: SOLR-9640: Support PKI authentication and SSL in standalone-mode master/slave auth with local security.json

Posted by Jan Høydahl <ja...@cominvent.com>.
Looking…

--
Jan Høydahl, search solution architect
Cominvent AS - www.cominvent.com

> 24. feb. 2017 kl. 19.48 skrev Uwe Schindler <uw...@thetaphi.de>:
> 
> I have the feeling this broke Jenkins. Millions of NPEs with JDK 8u121:
> 
> https://jenkins.thetaphi.de/job/Lucene-Solr-master-Linux/19042/console <https://jenkins.thetaphi.de/job/Lucene-Solr-master-Linux/19042/console>
> 
> 130 test failures by NPE in org.apache.solr.core.CoreContainer.getNodeNameLocal()
> 
> -----
> Uwe Schindler
> Achterdiek 19, D-28357 Bremen
> http://www.thetaphi.de <http://www.thetaphi.de/>
> eMail: uwe@thetaphi.de <ma...@thetaphi.de>
> 
>> -----Original Message-----
>> From: janhoy@apache.org <ma...@apache.org> [mailto:janhoy@apache.org <ma...@apache.org>]
>> Sent: Friday, February 24, 2017 2:31 PM
>> To: commits@lucene.apache.org <ma...@lucene.apache.org>
>> Subject: lucene-solr:master: SOLR-9640: Support PKI authentication and SSL
>> in standalone-mode master/slave auth with local security.json
>> 
>> Repository: lucene-solr
>> Updated Branches:
>>  refs/heads/master 5eeb8136f -> 95d6fc251
>> 
>> 
>> SOLR-9640: Support PKI authentication and SSL in standalone-mode
>> master/slave auth with local security.json
>> 
>> 
>> Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
>> Commit: http://git-wip-us.apache.org/repos/asf/lucene-
>> solr/commit/95d6fc25
>> Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/95d6fc25
>> Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/95d6fc25
>> 
>> Branch: refs/heads/master
>> Commit: 95d6fc2512d6525b2354165553f0d6cc4d0d6310
>> Parents: 5eeb813
>> Author: Jan HC8ydahl <janhoy@apache.org <ma...@apache.org>>
>> Authored: Fri Feb 24 14:26:48 2017 +0100
>> Committer: Jan HC8ydahl <janhoy@apache.org <ma...@apache.org>>
>> Committed: Fri Feb 24 14:30:42 2017 +0100
>> 
>> ----------------------------------------------------------------------
>> solr/CHANGES.txt                                |   2 +
>> .../org/apache/solr/core/CoreContainer.java     |   9 +-
>> .../solr/security/PKIAuthenticationPlugin.java  |  42 +++++-
>> .../org/apache/solr/servlet/HttpSolrCall.java   |   4 +-
>> .../apache/solr/servlet/SolrDispatchFilter.java |  11 +-
>> .../solr/security/BasicAuthDistributedTest.java | 136 +++++++++++++++++++
>> .../security/TestPKIAuthenticationPlugin.java   |  38 +++++-
>> .../solr/BaseDistributedSearchTestCase.java     |  37 ++++-
>> 8 files changed, 260 insertions(+), 19 deletions(-)
>> ----------------------------------------------------------------------
>> 
>> 
>> http://git-wip-us.apache.org/repos/asf/lucene-
>> solr/blob/95d6fc25/solr/CHANGES.txt
>> ----------------------------------------------------------------------
>> diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
>> index 0302615..2c5f0db 100644
>> --- a/solr/CHANGES.txt
>> +++ b/solr/CHANGES.txt
>> @@ -134,6 +134,8 @@ New Features
>>   field must both be stored=false, indexed=false, docValues=true. (Ishan
>> Chattopadhyaya, hossman, noble,
>>   shalin, yonik)
>> 
>> +* SOLR-9640: Support PKI authentication and SSL in standalone-mode
>> master/slave auth with local security.json (janhoy)
>> +
>> Bug Fixes
>> ----------------------
>> 
>> 
>> http://git-wip-us.apache.org/repos/asf/lucene-
>> solr/blob/95d6fc25/solr/core/src/java/org/apache/solr/core/CoreContainer.
>> java
>> ----------------------------------------------------------------------
>> diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
>> b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
>> index e3977d7..6115562 100644
>> --- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
>> +++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
>> @@ -497,7 +497,9 @@ public class CoreContainer {
>>     hostName = cfg.getNodeName();
>> 
>>     zkSys.initZooKeeper(this, solrHome, cfg.getCloudConfig());
>> -    if(isZooKeeperAware())  pkiAuthenticationPlugin = new
>> PKIAuthenticationPlugin(this, zkSys.getZkController().getNodeName());
>> +    pkiAuthenticationPlugin = isZooKeeperAware() ?
>> +        new PKIAuthenticationPlugin(this,
>> zkSys.getZkController().getNodeName()) :
>> +        new PKIAuthenticationPlugin(this, getNodeNameLocal());
>> 
>>     MDCLoggingContext.setNode(this);
>> 
>> @@ -618,6 +620,11 @@ public class CoreContainer {
>>     }
>>   }
>> 
>> +  // Builds a node name to be used with PKIAuth.
>> +  private String getNodeNameLocal() {
>> +    return
>> getConfig().getCloudConfig().getHost()+":"+getConfig().getCloudConfig().getS
>> olrHostPort()+"_solr";
>> +  }
>> +
>>   public void securityNodeChanged() {
>>     log.info("Security node changed, reloading security.json");
>>     reloadSecurityProperties();
>> 
>> http://git-wip-us.apache.org/repos/asf/lucene-
>> solr/blob/95d6fc25/solr/core/src/java/org/apache/solr/security/PKIAuthenti
>> cationPlugin.java
>> ----------------------------------------------------------------------
>> 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 fdd4408..d185bc9 100644
>> ---
>> a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
>> +++
>> b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
>> @@ -22,7 +22,9 @@ import javax.servlet.ServletResponse;
>> import javax.servlet.http.HttpServletRequest;
>> import javax.servlet.http.HttpServletRequestWrapper;
>> import java.io.IOException;
>> +import java.io.UnsupportedEncodingException;
>> import java.lang.invoke.MethodHandles;
>> +import java.net.URLDecoder;
>> import java.nio.ByteBuffer;
>> import java.security.Principal;
>> import java.security.PublicKey;
>> @@ -193,9 +195,14 @@ public class PKIAuthenticationPlugin extends
>> AuthenticationPlugin implements Htt
>>   }
>> 
>>   PublicKey getRemotePublicKey(String nodename) {
>> -    String url =
>> cores.getZkController().getZkStateReader().getBaseUrlForNodeName(nodena
>> me);
>> +    String url, uri = null;
>> +    if (cores.isZooKeeperAware()) {
>> +      url =
>> cores.getZkController().getZkStateReader().getBaseUrlForNodeName(nodena
>> me);
>> +    } else {
>> +      url = getBaseUrlForNodeNameLocal(nodename);
>> +    }
>>     try {
>> -      String uri = url + PATH + "?wt=json&omitHeader=true";
>> +      uri += PATH + "?wt=json&omitHeader=true";
>>       log.debug("Fetching fresh public key from : {}",uri);
>>       HttpResponse rsp = cores.getUpdateShardHandler().getHttpClient()
>>           .execute(new HttpGet(uri),
>> HttpClientUtil.createNewHttpClientRequestContext());
>> @@ -212,12 +219,41 @@ public class PKIAuthenticationPlugin extends
>> AuthenticationPlugin implements Htt
>>       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 : " + uri, e);
>>       return null;
>>     }
>> 
>>   }
>> 
>> +  protected String getBaseUrlForNodeNameLocal(String nodeName) {
>> +    final int _offset = nodeName.indexOf("_");
>> +    if (_offset < 0) {
>> +      throw new IllegalArgumentException("nodeName does not contain
>> expected '_' seperator: " + nodeName);
>> +    }
>> +    final String hostAndPort = nodeName.substring(0,_offset);
>> +    try {
>> +      final String path = URLDecoder.decode(nodeName.substring(1+_offset),
>> "UTF-8");
>> +      // TODO: Find a better way of resolving urlScheme when not using ZK?
>> +      String urlScheme = resolveUrlScheme();
>> +      return urlScheme + "://" + hostAndPort + (path.isEmpty() ? "" : ("/" +
>> path));
>> +    } catch (UnsupportedEncodingException e) {
>> +      throw new IllegalStateException("JVM Does not seem to support UTF-8",
>> e);
>> +    }
>> +  }
>> +
>> +  /**
>> +   * Resolve urlScheme first from sysProp "urlScheme", if not set or invalid
>> value, peek at ssl sysProps
>> +   * @return "https" if SSL is enabled, else "http"
>> +   */
>> +  protected static String resolveUrlScheme() {
>> +    String urlScheme = System.getProperty("urlScheme");
>> +    if (urlScheme != null && urlScheme.matches("https?")) {
>> +      return urlScheme;
>> +    } else {
>> +      return System.getProperty("solr.jetty.keystore") == null ? "http" :
>> "https";
>> +    }
>> +  }
>> +
>>   @Override
>>   public SolrHttpClientBuilder getHttpClientBuilder(SolrHttpClientBuilder
>> builder) {
>>     HttpClientUtil.addRequestInterceptor(interceptor);
>> 
>> http://git-wip-us.apache.org/repos/asf/lucene-
>> solr/blob/95d6fc25/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.j
>> ava
>> ----------------------------------------------------------------------
>> diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
>> b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
>> index 4f6bae0..0dfb0ea 100644
>> --- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
>> +++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
>> @@ -560,7 +560,7 @@ public class HttpSolrCall {
>>   }
>> 
>>   private boolean shouldAuthorize() {
>> -    if(PKIAuthenticationPlugin.PATH.equals(path)) return false;
>> +    if(path != null && path.endsWith(PKIAuthenticationPlugin.PATH)) return
>> false;
>>     //admin/info/key is the path where public key is exposed . it is always
>> unsecured
>>     if (cores.getPkiAuthenticationPlugin() != null && req.getUserPrincipal() !=
>> null) {
>>       boolean b = cores.getPkiAuthenticationPlugin().needsAuthorization(req);
>> @@ -1081,7 +1081,7 @@ public class HttpSolrCall {
>>           response.delete(response.length() - 1, response.length());
>> 
>>         response.append("], Path: [").append(resource).append("]");
>> -        response.append(" path : ").append(path).append(" params
>> :").append(solrReq.getParams());
>> +        response.append(" path : ").append(path).append(" params
>> :").append(solrReq == null ? null : solrReq.getParams());
>>         return response.toString();
>>       }
>> 
>> 
>> http://git-wip-us.apache.org/repos/asf/lucene-
>> solr/blob/95d6fc25/solr/core/src/java/org/apache/solr/servlet/SolrDispatch
>> Filter.java
>> ----------------------------------------------------------------------
>> 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 ce65069..4ce57b0 100644
>> --- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
>> +++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
>> @@ -402,11 +402,11 @@ public class SolrDispatchFilter extends
>> BaseSolrFilter {
>>     if (authenticationPlugin == null) {
>>       return true;
>>     } else {
>> -      // /admin/info/key must be always open. see SOLR-9188
>> -      // tests work only w/ getPathInfo
>> -      //otherwise it's just enough to have getServletPath()
>> -      if (PKIAuthenticationPlugin.PATH.equals(((HttpServletRequest)
>> request).getServletPath()) ||
>> -          PKIAuthenticationPlugin.PATH.equals(((HttpServletRequest)
>> request).getPathInfo())) return true;
>> +      String requestUri = ((HttpServletRequest) request).getRequestURI();
>> +      if (requestUri != null &&
>> requestUri.endsWith(PKIAuthenticationPlugin.PATH)) {
>> +        log.debug("Passthrough of pki URL " + requestUri);
>> +        return true;
>> +      }
>>       String header = ((HttpServletRequest)
>> request).getHeader(PKIAuthenticationPlugin.HEADER);
>>       if (header != null && cores.getPkiAuthenticationPlugin() != null)
>>         authenticationPlugin = cores.getPkiAuthenticationPlugin();
>> @@ -418,7 +418,6 @@ public class SolrDispatchFilter extends BaseSolrFilter {
>>           wrappedRequest.set(req);
>>         });
>>       } catch (Exception e) {
>> -        log.info("Error authenticating", e);
>>         throw new SolrException(ErrorCode.SERVER_ERROR, "Error during
>> request authentication, ", e);
>>       }
>>     }
>> 
>> http://git-wip-us.apache.org/repos/asf/lucene-
>> solr/blob/95d6fc25/solr/core/src/test/org/apache/solr/security/BasicAuthDi
>> stributedTest.java
>> ----------------------------------------------------------------------
>> diff --git
>> a/solr/core/src/test/org/apache/solr/security/BasicAuthDistributedTest.java
>> b/solr/core/src/test/org/apache/solr/security/BasicAuthDistributedTest.java
>> new file mode 100644
>> index 0000000..e35e369
>> --- /dev/null
>> +++
>> b/solr/core/src/test/org/apache/solr/security/BasicAuthDistributedTest.java
>> @@ -0,0 +1,136 @@
>> +/*
>> + * 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.nio.file.Files;
>> +import java.nio.file.Paths;
>> +
>> +import org.apache.lucene.util.LuceneTestCase.Slow;
>> +import org.apache.solr.BaseDistributedSearchTestCase;
>> +import org.apache.solr.client.solrj.embedded.JettySolrRunner;
>> +import org.apache.solr.client.solrj.impl.HttpSolrClient;
>> +import org.apache.solr.client.solrj.request.QueryRequest;
>> +import org.apache.solr.client.solrj.response.QueryResponse;
>> +import org.apache.solr.common.params.ModifiableSolrParams;
>> +import org.apache.solr.common.util.Utils;
>> +import org.apache.solr.core.CoreContainer;
>> +import org.apache.solr.handler.admin.SecurityConfHandler;
>> +import org.apache.solr.handler.admin.SecurityConfHandlerLocalForTesting;
>> +import org.apache.solr.util.LogLevel;
>> +import org.junit.Test;
>> +
>> +/**
>> + * Tests basicAuth in a multi shard env
>> + */
>> +@Slow
>> +public class BasicAuthDistributedTest extends
>> BaseDistributedSearchTestCase {
>> +  public BasicAuthDistributedTest() {
>> +    super();
>> +    schemaString = "schema.xml";
>> +  }
>> +
>> +  private SecurityConfHandlerLocalForTesting securityConfHandler;
>> +
>> +  @Test
>> +  @LogLevel("org.apache.solr=DEBUG")
>> +  public void test() throws Exception {
>> +    index();
>> +    testAuth();
>> +  }
>> +
>> +  private void index() throws Exception {
>> +    del("*:*");
>> +    indexr(id, "1", "text", "doc one");
>> +    indexr(id, "2", "text", "doc two");
>> +    indexr(id, "3", "text", "doc three");
>> +    indexr(id, "4", "text", "doc four");
>> +    indexr(id, "5", "text", "doc five");
>> +
>> +    commit();  // try to ensure there's more than one segment
>> +
>> +    indexr(id, "6", "text", "doc six");
>> +    indexr(id, "7", "text", "doc seven");
>> +    indexr(id, "8", "text", "doc eight");
>> +    indexr(id, "9", "text", "doc nine");
>> +    indexr(id, "10", "text", "doc ten");
>> +
>> +    commit();
>> +
>> +    handle.clear();
>> +    handle.put("QTime", SKIPVAL);
>> +    handle.put("timestamp", SKIPVAL);
>> +    handle.put("maxScore", SKIPVAL);
>> +    handle.put("_version_", SKIPVAL);
>> +  }
>> +
>> +  private void testAuth() throws Exception {
>> +    QueryResponse rsp = query("q","text:doc", "fl", "id,text", "sort", "id asc");
>> +    assertEquals(10, rsp.getResults().getNumFound());
>> +
>> +    // Enable authentication
>> +    for (JettySolrRunner j : jettys) {
>> +      writeSecurityJson(j.getCoreContainer());
>> +    }
>> +
>> +    HttpSolrClient.RemoteSolrException expected =
>> expectThrows(HttpSolrClient.RemoteSolrException.class, () -> {
>> +      query("q","text:doc-fail", "fl", "id,text", "sort", "id asc");
>> +    });
>> +    assertEquals(401, expected.code());
>> +
>> +    // Add auth
>> +    ModifiableSolrParams params = new ModifiableSolrParams();
>> +    params.add("q", "text:doc").add("fl", "id,text").add("sort", "id asc");
>> +    QueryRequest req = new QueryRequest(params);
>> +    req.setBasicAuthCredentials("solr", "SolrRocks");
>> +    rsp = req.process(clients.get(0), null);
>> +    if (jettys.size() > 1) {
>> +      assertTrue(rsp.getResults().getNumFound() < 10);
>> +      rsp = query(true, params, "solr", "SolrRocks");
>> +    }
>> +    assertEquals(10, rsp.getResults().getNumFound());
>> +
>> +    // Disable auth
>> +    for (JettySolrRunner j : jettys) {
>> +      deleteSecurityJson(j.getCoreContainer());
>> +    }
>> +
>> +  }
>> +
>> +  private void deleteSecurityJson(CoreContainer coreContainer) throws
>> IOException {
>> +    securityConfHandler = new
>> SecurityConfHandlerLocalForTesting(coreContainer);
>> +
>> Files.delete(Paths.get(coreContainer.getSolrHome()).resolve("security.json"))
>> ;
>> +    coreContainer.securityNodeChanged();
>> +  }
>> +
>> +  private void writeSecurityJson(CoreContainer coreContainer) throws
>> IOException {
>> +    securityConfHandler = new
>> SecurityConfHandlerLocalForTesting(coreContainer);
>> +    securityConfHandler.persistConf(new
>> SecurityConfHandler.SecurityConfig()
>> +        .setData(Utils.fromJSONString(ALL_CONF.replaceAll("'", "\""))));
>> +    coreContainer.securityNodeChanged();
>> +  }
>> +
>> +  protected static final String ALL_CONF = "{\n" +
>> +      "  'authentication':{\n" +
>> +      "    'blockUnknown':true,\n" +
>> +      "    'class':'solr.BasicAuthPlugin',\n" +
>> +      "
>> 'credentials':{'solr':'orwp2Ghgj39lmnrZOTm7Qtre1VqHFDfwAEzr0ApbN3Y=
>> Ju5osoAqOX8iafhWpPP01E5P+sg8tK8tHON7rCYZRRw='}},\n" +
>> +      "  'authorization':{\n" +
>> +      "    'class':'solr.RuleBasedAuthorizationPlugin',\n" +
>> +      "    'user-role':{'solr':'admin'},\n" +
>> +      "    'permissions':[{'name':'all','role':'admin'}]}}";
>> +}
>> 
>> http://git-wip-us.apache.org/repos/asf/lucene-
>> solr/blob/95d6fc25/solr/core/src/test/org/apache/solr/security/TestPKIAuth
>> enticationPlugin.java
>> ----------------------------------------------------------------------
>> diff --git
>> a/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.ja
>> va
>> b/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.j
>> ava
>> index a5a279f..90c5bd2 100644
>> ---
>> a/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.ja
>> va
>> +++
>> b/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.j
>> ava
>> @@ -35,7 +35,11 @@ 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 static org.mockito.Mockito.*;
>> +import org.junit.Test;
>> +
>> +import static org.mockito.Mockito.any;
>> +import static org.mockito.Mockito.mock;
>> +import static org.mockito.Mockito.when;
>> 
>> public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
>> 
>> @@ -141,10 +145,38 @@ public class TestPKIAuthenticationPlugin extends
>> SolrTestCaseJ4 {
>>     mock1.doAuthenticate(mockReq, null,filterChain );
>>     assertNotNull(wrappedRequestByFilter.get());
>>     assertEquals("$", ((HttpServletRequest)
>> wrappedRequestByFilter.get()).getUserPrincipal().getName());
>> +  }
>> 
>> +  @Test
>> +  public void testGetBaseUrlForNodeNameLocal() {
>> +    synchronized (this) {
>> +      final MockPKIAuthenticationPlugin mock = new
>> MockPKIAuthenticationPlugin(null, "myName");
>> +      System.clearProperty("solr.jetty.keystore");
>> +      assertEquals("http://my.host:9876/solr2",
>> mock.getBaseUrlForNodeNameLocal("my.host:9876_solr2"));
>> +      System.setProperty("solr.jetty.keystore", "foo");
>> +      assertEquals("https://my.host:9876/solr2",
>> mock.getBaseUrlForNodeNameLocal("my.host:9876_solr2"));
>> +      System.clearProperty("solr.jetty.keystore");
>> +    }
>> +  }
>> 
>> -
>> -
>> +  @Test
>> +  public void testResolveUrlScheme() {
>> +    synchronized (this) {
>> +      System.clearProperty("urlScheme");
>> +      System.clearProperty("solr.jetty.keystore");
>> +      assertEquals("http", MockPKIAuthenticationPlugin.resolveUrlScheme());
>> +      System.setProperty("urlScheme", "http");
>> +      assertEquals("http", MockPKIAuthenticationPlugin.resolveUrlScheme());
>> +      System.setProperty("urlScheme", "https");
>> +      assertEquals("https", MockPKIAuthenticationPlugin.resolveUrlScheme());
>> +      System.setProperty("urlScheme", "ftp");
>> +      System.clearProperty("solr.jetty.keystore");
>> +      assertEquals("http", MockPKIAuthenticationPlugin.resolveUrlScheme());
>> +      System.setProperty("solr.jetty.keystore", "foo");
>> +      assertEquals("https", MockPKIAuthenticationPlugin.resolveUrlScheme());
>> +      System.clearProperty("urlScheme");
>> +      System.clearProperty("solr.jetty.keystore");
>> +    }
>>   }
>> 
>>   private HttpServletRequest createMockRequest(final
>> AtomicReference<Header> header) {
>> 
>> http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/95d6fc25/solr/test-
>> framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
>> ----------------------------------------------------------------------
>> diff --git a/solr/test-
>> framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
>> b/solr/test-
>> framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
>> index 8c6eb60..bbfc048 100644
>> --- a/solr/test-
>> framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
>> +++ b/solr/test-
>> framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
>> @@ -50,6 +50,7 @@ import org.apache.solr.client.solrj.SolrServerException;
>> import org.apache.solr.client.solrj.embedded.JettyConfig;
>> import org.apache.solr.client.solrj.embedded.JettySolrRunner;
>> import org.apache.solr.client.solrj.impl.HttpSolrClient;
>> +import org.apache.solr.client.solrj.request.QueryRequest;
>> import org.apache.solr.client.solrj.request.UpdateRequest;
>> import org.apache.solr.client.solrj.response.QueryResponse;
>> import org.apache.solr.client.solrj.response.UpdateResponse;
>> @@ -558,6 +559,12 @@ public abstract class BaseDistributedSearchTestCase
>> extends SolrTestCaseJ4 {
>>     return rsp;
>>   }
>> 
>> +  protected QueryResponse queryServer(QueryRequest req) throws
>> IOException, SolrServerException {
>> +    int which = r.nextInt(clients.size());
>> +    SolrClient client = clients.get(which);
>> +    return req.process(client, null);
>> +  }
>> +
>>   /**
>>    * Sets distributed params.
>>    * Returns the QueryResponse from {@link #queryServer},
>> @@ -591,18 +598,31 @@ public abstract class
>> BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
>>    * Returns the QueryResponse from {@link #queryServer}
>>    */
>>   protected QueryResponse query(boolean setDistribParams, SolrParams p)
>> throws Exception {
>> +    return query(setDistribParams, p, null, null);
>> +  }
>> +
>> +  /**
>> +   * Returns the QueryResponse from {@link #queryServer}
>> +   * @param setDistribParams whether to do a distributed request
>> +   * @param user basic auth username (set to null if not in use)
>> +   * @param pass basic auth password (set to null if not in use)
>> +   * @return the query response
>> +   */
>> +  protected QueryResponse query(boolean setDistribParams, SolrParams p,
>> String user, String pass) throws Exception {
>> 
>>     final ModifiableSolrParams params = new ModifiableSolrParams(p);
>> 
>>     // TODO: look into why passing true causes fails
>>     params.set("distrib", "false");
>> -    final QueryResponse controlRsp = controlClient.query(params);
>> +    QueryRequest req = generateQueryRequest(params, user, pass);
>> +    final QueryResponse controlRsp = req.process(controlClient, null);
>>     validateControlData(controlRsp);
>> 
>>     params.remove("distrib");
>>     if (setDistribParams) setDistributedParams(params);
>> +    req = generateQueryRequest(params, user, pass);
>> 
>> -    QueryResponse rsp = queryServer(params);
>> +    QueryResponse rsp = queryServer(req);
>> 
>>     compareResponses(rsp, controlRsp);
>> 
>> @@ -617,7 +637,8 @@ public abstract class BaseDistributedSearchTestCase
>> extends SolrTestCaseJ4 {
>>               int which = r.nextInt(clients.size());
>>               SolrClient client = clients.get(which);
>>               try {
>> -                QueryResponse rsp = client.query(new
>> ModifiableSolrParams(params));
>> +                QueryRequest qreq = generateQueryRequest(new
>> ModifiableSolrParams(params), user, pass);
>> +                QueryResponse rsp = qreq.process(client, null);
>>                 if (verifyStress) {
>>                   compareResponses(rsp, controlRsp);
>>                 }
>> @@ -636,7 +657,15 @@ public abstract class BaseDistributedSearchTestCase
>> extends SolrTestCaseJ4 {
>>     }
>>     return rsp;
>>   }
>> -
>> +
>> +  private QueryRequest generateQueryRequest(ModifiableSolrParams
>> params, String user, String pass) {
>> +    QueryRequest req = new QueryRequest(params);
>> +    if (user != null && pass != null) {
>> +      req.setBasicAuthCredentials(user, pass);
>> +    }
>> +    return req;
>> +  }
>> +
>>   public QueryResponse queryAndCompare(SolrParams params, SolrClient...
>> clients) throws SolrServerException, IOException {
>>     return queryAndCompare(params, Arrays.<SolrClient>asList(clients));
>>   }
> 
> 
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org <ma...@lucene.apache.org>
> For additional commands, e-mail: dev-help@lucene.apache.org <ma...@lucene.apache.org>

RE: lucene-solr:master: SOLR-9640: Support PKI authentication and SSL in standalone-mode master/slave auth with local security.json

Posted by Uwe Schindler <uw...@thetaphi.de>.
I have the feeling this broke Jenkins. Millions of NPEs with JDK 8u121:

https://jenkins.thetaphi.de/job/Lucene-Solr-master-Linux/19042/console

130 test failures by NPE in org.apache.solr.core.CoreContainer.getNodeNameLocal()

-----
Uwe Schindler
Achterdiek 19, D-28357 Bremen
http://www.thetaphi.de
eMail: uwe@thetaphi.de

> -----Original Message-----
> From: janhoy@apache.org [mailto:janhoy@apache.org]
> Sent: Friday, February 24, 2017 2:31 PM
> To: commits@lucene.apache.org
> Subject: lucene-solr:master: SOLR-9640: Support PKI authentication and SSL
> in standalone-mode master/slave auth with local security.json
> 
> Repository: lucene-solr
> Updated Branches:
>   refs/heads/master 5eeb8136f -> 95d6fc251
> 
> 
> SOLR-9640: Support PKI authentication and SSL in standalone-mode
> master/slave auth with local security.json
> 
> 
> Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
> Commit: http://git-wip-us.apache.org/repos/asf/lucene-
> solr/commit/95d6fc25
> Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/95d6fc25
> Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/95d6fc25
> 
> Branch: refs/heads/master
> Commit: 95d6fc2512d6525b2354165553f0d6cc4d0d6310
> Parents: 5eeb813
> Author: Jan HC8ydahl <ja...@apache.org>
> Authored: Fri Feb 24 14:26:48 2017 +0100
> Committer: Jan HC8ydahl <ja...@apache.org>
> Committed: Fri Feb 24 14:30:42 2017 +0100
> 
> ----------------------------------------------------------------------
>  solr/CHANGES.txt                                |   2 +
>  .../org/apache/solr/core/CoreContainer.java     |   9 +-
>  .../solr/security/PKIAuthenticationPlugin.java  |  42 +++++-
>  .../org/apache/solr/servlet/HttpSolrCall.java   |   4 +-
>  .../apache/solr/servlet/SolrDispatchFilter.java |  11 +-
>  .../solr/security/BasicAuthDistributedTest.java | 136 +++++++++++++++++++
>  .../security/TestPKIAuthenticationPlugin.java   |  38 +++++-
>  .../solr/BaseDistributedSearchTestCase.java     |  37 ++++-
>  8 files changed, 260 insertions(+), 19 deletions(-)
> ----------------------------------------------------------------------
> 
> 
> http://git-wip-us.apache.org/repos/asf/lucene-
> solr/blob/95d6fc25/solr/CHANGES.txt
> ----------------------------------------------------------------------
> diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
> index 0302615..2c5f0db 100644
> --- a/solr/CHANGES.txt
> +++ b/solr/CHANGES.txt
> @@ -134,6 +134,8 @@ New Features
>    field must both be stored=false, indexed=false, docValues=true. (Ishan
> Chattopadhyaya, hossman, noble,
>    shalin, yonik)
> 
> +* SOLR-9640: Support PKI authentication and SSL in standalone-mode
> master/slave auth with local security.json (janhoy)
> +
>  Bug Fixes
>  ----------------------
> 
> 
> http://git-wip-us.apache.org/repos/asf/lucene-
> solr/blob/95d6fc25/solr/core/src/java/org/apache/solr/core/CoreContainer.
> java
> ----------------------------------------------------------------------
> diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
> b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
> index e3977d7..6115562 100644
> --- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
> +++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
> @@ -497,7 +497,9 @@ public class CoreContainer {
>      hostName = cfg.getNodeName();
> 
>      zkSys.initZooKeeper(this, solrHome, cfg.getCloudConfig());
> -    if(isZooKeeperAware())  pkiAuthenticationPlugin = new
> PKIAuthenticationPlugin(this, zkSys.getZkController().getNodeName());
> +    pkiAuthenticationPlugin = isZooKeeperAware() ?
> +        new PKIAuthenticationPlugin(this,
> zkSys.getZkController().getNodeName()) :
> +        new PKIAuthenticationPlugin(this, getNodeNameLocal());
> 
>      MDCLoggingContext.setNode(this);
> 
> @@ -618,6 +620,11 @@ public class CoreContainer {
>      }
>    }
> 
> +  // Builds a node name to be used with PKIAuth.
> +  private String getNodeNameLocal() {
> +    return
> getConfig().getCloudConfig().getHost()+":"+getConfig().getCloudConfig().getS
> olrHostPort()+"_solr";
> +  }
> +
>    public void securityNodeChanged() {
>      log.info("Security node changed, reloading security.json");
>      reloadSecurityProperties();
> 
> http://git-wip-us.apache.org/repos/asf/lucene-
> solr/blob/95d6fc25/solr/core/src/java/org/apache/solr/security/PKIAuthenti
> cationPlugin.java
> ----------------------------------------------------------------------
> 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 fdd4408..d185bc9 100644
> ---
> a/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
> +++
> b/solr/core/src/java/org/apache/solr/security/PKIAuthenticationPlugin.java
> @@ -22,7 +22,9 @@ import javax.servlet.ServletResponse;
>  import javax.servlet.http.HttpServletRequest;
>  import javax.servlet.http.HttpServletRequestWrapper;
>  import java.io.IOException;
> +import java.io.UnsupportedEncodingException;
>  import java.lang.invoke.MethodHandles;
> +import java.net.URLDecoder;
>  import java.nio.ByteBuffer;
>  import java.security.Principal;
>  import java.security.PublicKey;
> @@ -193,9 +195,14 @@ public class PKIAuthenticationPlugin extends
> AuthenticationPlugin implements Htt
>    }
> 
>    PublicKey getRemotePublicKey(String nodename) {
> -    String url =
> cores.getZkController().getZkStateReader().getBaseUrlForNodeName(nodena
> me);
> +    String url, uri = null;
> +    if (cores.isZooKeeperAware()) {
> +      url =
> cores.getZkController().getZkStateReader().getBaseUrlForNodeName(nodena
> me);
> +    } else {
> +      url = getBaseUrlForNodeNameLocal(nodename);
> +    }
>      try {
> -      String uri = url + PATH + "?wt=json&omitHeader=true";
> +      uri += PATH + "?wt=json&omitHeader=true";
>        log.debug("Fetching fresh public key from : {}",uri);
>        HttpResponse rsp = cores.getUpdateShardHandler().getHttpClient()
>            .execute(new HttpGet(uri),
> HttpClientUtil.createNewHttpClientRequestContext());
> @@ -212,12 +219,41 @@ public class PKIAuthenticationPlugin extends
> AuthenticationPlugin implements Htt
>        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 : " + uri, e);
>        return null;
>      }
> 
>    }
> 
> +  protected String getBaseUrlForNodeNameLocal(String nodeName) {
> +    final int _offset = nodeName.indexOf("_");
> +    if (_offset < 0) {
> +      throw new IllegalArgumentException("nodeName does not contain
> expected '_' seperator: " + nodeName);
> +    }
> +    final String hostAndPort = nodeName.substring(0,_offset);
> +    try {
> +      final String path = URLDecoder.decode(nodeName.substring(1+_offset),
> "UTF-8");
> +      // TODO: Find a better way of resolving urlScheme when not using ZK?
> +      String urlScheme = resolveUrlScheme();
> +      return urlScheme + "://" + hostAndPort + (path.isEmpty() ? "" : ("/" +
> path));
> +    } catch (UnsupportedEncodingException e) {
> +      throw new IllegalStateException("JVM Does not seem to support UTF-8",
> e);
> +    }
> +  }
> +
> +  /**
> +   * Resolve urlScheme first from sysProp "urlScheme", if not set or invalid
> value, peek at ssl sysProps
> +   * @return "https" if SSL is enabled, else "http"
> +   */
> +  protected static String resolveUrlScheme() {
> +    String urlScheme = System.getProperty("urlScheme");
> +    if (urlScheme != null && urlScheme.matches("https?")) {
> +      return urlScheme;
> +    } else {
> +      return System.getProperty("solr.jetty.keystore") == null ? "http" :
> "https";
> +    }
> +  }
> +
>    @Override
>    public SolrHttpClientBuilder getHttpClientBuilder(SolrHttpClientBuilder
> builder) {
>      HttpClientUtil.addRequestInterceptor(interceptor);
> 
> http://git-wip-us.apache.org/repos/asf/lucene-
> solr/blob/95d6fc25/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.j
> ava
> ----------------------------------------------------------------------
> diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
> b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
> index 4f6bae0..0dfb0ea 100644
> --- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
> +++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
> @@ -560,7 +560,7 @@ public class HttpSolrCall {
>    }
> 
>    private boolean shouldAuthorize() {
> -    if(PKIAuthenticationPlugin.PATH.equals(path)) return false;
> +    if(path != null && path.endsWith(PKIAuthenticationPlugin.PATH)) return
> false;
>      //admin/info/key is the path where public key is exposed . it is always
> unsecured
>      if (cores.getPkiAuthenticationPlugin() != null && req.getUserPrincipal() !=
> null) {
>        boolean b = cores.getPkiAuthenticationPlugin().needsAuthorization(req);
> @@ -1081,7 +1081,7 @@ public class HttpSolrCall {
>            response.delete(response.length() - 1, response.length());
> 
>          response.append("], Path: [").append(resource).append("]");
> -        response.append(" path : ").append(path).append(" params
> :").append(solrReq.getParams());
> +        response.append(" path : ").append(path).append(" params
> :").append(solrReq == null ? null : solrReq.getParams());
>          return response.toString();
>        }
> 
> 
> http://git-wip-us.apache.org/repos/asf/lucene-
> solr/blob/95d6fc25/solr/core/src/java/org/apache/solr/servlet/SolrDispatch
> Filter.java
> ----------------------------------------------------------------------
> 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 ce65069..4ce57b0 100644
> --- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
> +++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
> @@ -402,11 +402,11 @@ public class SolrDispatchFilter extends
> BaseSolrFilter {
>      if (authenticationPlugin == null) {
>        return true;
>      } else {
> -      // /admin/info/key must be always open. see SOLR-9188
> -      // tests work only w/ getPathInfo
> -      //otherwise it's just enough to have getServletPath()
> -      if (PKIAuthenticationPlugin.PATH.equals(((HttpServletRequest)
> request).getServletPath()) ||
> -          PKIAuthenticationPlugin.PATH.equals(((HttpServletRequest)
> request).getPathInfo())) return true;
> +      String requestUri = ((HttpServletRequest) request).getRequestURI();
> +      if (requestUri != null &&
> requestUri.endsWith(PKIAuthenticationPlugin.PATH)) {
> +        log.debug("Passthrough of pki URL " + requestUri);
> +        return true;
> +      }
>        String header = ((HttpServletRequest)
> request).getHeader(PKIAuthenticationPlugin.HEADER);
>        if (header != null && cores.getPkiAuthenticationPlugin() != null)
>          authenticationPlugin = cores.getPkiAuthenticationPlugin();
> @@ -418,7 +418,6 @@ public class SolrDispatchFilter extends BaseSolrFilter {
>            wrappedRequest.set(req);
>          });
>        } catch (Exception e) {
> -        log.info("Error authenticating", e);
>          throw new SolrException(ErrorCode.SERVER_ERROR, "Error during
> request authentication, ", e);
>        }
>      }
> 
> http://git-wip-us.apache.org/repos/asf/lucene-
> solr/blob/95d6fc25/solr/core/src/test/org/apache/solr/security/BasicAuthDi
> stributedTest.java
> ----------------------------------------------------------------------
> diff --git
> a/solr/core/src/test/org/apache/solr/security/BasicAuthDistributedTest.java
> b/solr/core/src/test/org/apache/solr/security/BasicAuthDistributedTest.java
> new file mode 100644
> index 0000000..e35e369
> --- /dev/null
> +++
> b/solr/core/src/test/org/apache/solr/security/BasicAuthDistributedTest.java
> @@ -0,0 +1,136 @@
> +/*
> + * 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.nio.file.Files;
> +import java.nio.file.Paths;
> +
> +import org.apache.lucene.util.LuceneTestCase.Slow;
> +import org.apache.solr.BaseDistributedSearchTestCase;
> +import org.apache.solr.client.solrj.embedded.JettySolrRunner;
> +import org.apache.solr.client.solrj.impl.HttpSolrClient;
> +import org.apache.solr.client.solrj.request.QueryRequest;
> +import org.apache.solr.client.solrj.response.QueryResponse;
> +import org.apache.solr.common.params.ModifiableSolrParams;
> +import org.apache.solr.common.util.Utils;
> +import org.apache.solr.core.CoreContainer;
> +import org.apache.solr.handler.admin.SecurityConfHandler;
> +import org.apache.solr.handler.admin.SecurityConfHandlerLocalForTesting;
> +import org.apache.solr.util.LogLevel;
> +import org.junit.Test;
> +
> +/**
> + * Tests basicAuth in a multi shard env
> + */
> +@Slow
> +public class BasicAuthDistributedTest extends
> BaseDistributedSearchTestCase {
> +  public BasicAuthDistributedTest() {
> +    super();
> +    schemaString = "schema.xml";
> +  }
> +
> +  private SecurityConfHandlerLocalForTesting securityConfHandler;
> +
> +  @Test
> +  @LogLevel("org.apache.solr=DEBUG")
> +  public void test() throws Exception {
> +    index();
> +    testAuth();
> +  }
> +
> +  private void index() throws Exception {
> +    del("*:*");
> +    indexr(id, "1", "text", "doc one");
> +    indexr(id, "2", "text", "doc two");
> +    indexr(id, "3", "text", "doc three");
> +    indexr(id, "4", "text", "doc four");
> +    indexr(id, "5", "text", "doc five");
> +
> +    commit();  // try to ensure there's more than one segment
> +
> +    indexr(id, "6", "text", "doc six");
> +    indexr(id, "7", "text", "doc seven");
> +    indexr(id, "8", "text", "doc eight");
> +    indexr(id, "9", "text", "doc nine");
> +    indexr(id, "10", "text", "doc ten");
> +
> +    commit();
> +
> +    handle.clear();
> +    handle.put("QTime", SKIPVAL);
> +    handle.put("timestamp", SKIPVAL);
> +    handle.put("maxScore", SKIPVAL);
> +    handle.put("_version_", SKIPVAL);
> +  }
> +
> +  private void testAuth() throws Exception {
> +    QueryResponse rsp = query("q","text:doc", "fl", "id,text", "sort", "id asc");
> +    assertEquals(10, rsp.getResults().getNumFound());
> +
> +    // Enable authentication
> +    for (JettySolrRunner j : jettys) {
> +      writeSecurityJson(j.getCoreContainer());
> +    }
> +
> +    HttpSolrClient.RemoteSolrException expected =
> expectThrows(HttpSolrClient.RemoteSolrException.class, () -> {
> +      query("q","text:doc-fail", "fl", "id,text", "sort", "id asc");
> +    });
> +    assertEquals(401, expected.code());
> +
> +    // Add auth
> +    ModifiableSolrParams params = new ModifiableSolrParams();
> +    params.add("q", "text:doc").add("fl", "id,text").add("sort", "id asc");
> +    QueryRequest req = new QueryRequest(params);
> +    req.setBasicAuthCredentials("solr", "SolrRocks");
> +    rsp = req.process(clients.get(0), null);
> +    if (jettys.size() > 1) {
> +      assertTrue(rsp.getResults().getNumFound() < 10);
> +      rsp = query(true, params, "solr", "SolrRocks");
> +    }
> +    assertEquals(10, rsp.getResults().getNumFound());
> +
> +    // Disable auth
> +    for (JettySolrRunner j : jettys) {
> +      deleteSecurityJson(j.getCoreContainer());
> +    }
> +
> +  }
> +
> +  private void deleteSecurityJson(CoreContainer coreContainer) throws
> IOException {
> +    securityConfHandler = new
> SecurityConfHandlerLocalForTesting(coreContainer);
> +
> Files.delete(Paths.get(coreContainer.getSolrHome()).resolve("security.json"))
> ;
> +    coreContainer.securityNodeChanged();
> +  }
> +
> +  private void writeSecurityJson(CoreContainer coreContainer) throws
> IOException {
> +    securityConfHandler = new
> SecurityConfHandlerLocalForTesting(coreContainer);
> +    securityConfHandler.persistConf(new
> SecurityConfHandler.SecurityConfig()
> +        .setData(Utils.fromJSONString(ALL_CONF.replaceAll("'", "\""))));
> +    coreContainer.securityNodeChanged();
> +  }
> +
> +  protected static final String ALL_CONF = "{\n" +
> +      "  'authentication':{\n" +
> +      "    'blockUnknown':true,\n" +
> +      "    'class':'solr.BasicAuthPlugin',\n" +
> +      "
> 'credentials':{'solr':'orwp2Ghgj39lmnrZOTm7Qtre1VqHFDfwAEzr0ApbN3Y=
> Ju5osoAqOX8iafhWpPP01E5P+sg8tK8tHON7rCYZRRw='}},\n" +
> +      "  'authorization':{\n" +
> +      "    'class':'solr.RuleBasedAuthorizationPlugin',\n" +
> +      "    'user-role':{'solr':'admin'},\n" +
> +      "    'permissions':[{'name':'all','role':'admin'}]}}";
> +}
> 
> http://git-wip-us.apache.org/repos/asf/lucene-
> solr/blob/95d6fc25/solr/core/src/test/org/apache/solr/security/TestPKIAuth
> enticationPlugin.java
> ----------------------------------------------------------------------
> diff --git
> a/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.ja
> va
> b/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.j
> ava
> index a5a279f..90c5bd2 100644
> ---
> a/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.ja
> va
> +++
> b/solr/core/src/test/org/apache/solr/security/TestPKIAuthenticationPlugin.j
> ava
> @@ -35,7 +35,11 @@ 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 static org.mockito.Mockito.*;
> +import org.junit.Test;
> +
> +import static org.mockito.Mockito.any;
> +import static org.mockito.Mockito.mock;
> +import static org.mockito.Mockito.when;
> 
>  public class TestPKIAuthenticationPlugin extends SolrTestCaseJ4 {
> 
> @@ -141,10 +145,38 @@ public class TestPKIAuthenticationPlugin extends
> SolrTestCaseJ4 {
>      mock1.doAuthenticate(mockReq, null,filterChain );
>      assertNotNull(wrappedRequestByFilter.get());
>      assertEquals("$", ((HttpServletRequest)
> wrappedRequestByFilter.get()).getUserPrincipal().getName());
> +  }
> 
> +  @Test
> +  public void testGetBaseUrlForNodeNameLocal() {
> +    synchronized (this) {
> +      final MockPKIAuthenticationPlugin mock = new
> MockPKIAuthenticationPlugin(null, "myName");
> +      System.clearProperty("solr.jetty.keystore");
> +      assertEquals("http://my.host:9876/solr2",
> mock.getBaseUrlForNodeNameLocal("my.host:9876_solr2"));
> +      System.setProperty("solr.jetty.keystore", "foo");
> +      assertEquals("https://my.host:9876/solr2",
> mock.getBaseUrlForNodeNameLocal("my.host:9876_solr2"));
> +      System.clearProperty("solr.jetty.keystore");
> +    }
> +  }
> 
> -
> -
> +  @Test
> +  public void testResolveUrlScheme() {
> +    synchronized (this) {
> +      System.clearProperty("urlScheme");
> +      System.clearProperty("solr.jetty.keystore");
> +      assertEquals("http", MockPKIAuthenticationPlugin.resolveUrlScheme());
> +      System.setProperty("urlScheme", "http");
> +      assertEquals("http", MockPKIAuthenticationPlugin.resolveUrlScheme());
> +      System.setProperty("urlScheme", "https");
> +      assertEquals("https", MockPKIAuthenticationPlugin.resolveUrlScheme());
> +      System.setProperty("urlScheme", "ftp");
> +      System.clearProperty("solr.jetty.keystore");
> +      assertEquals("http", MockPKIAuthenticationPlugin.resolveUrlScheme());
> +      System.setProperty("solr.jetty.keystore", "foo");
> +      assertEquals("https", MockPKIAuthenticationPlugin.resolveUrlScheme());
> +      System.clearProperty("urlScheme");
> +      System.clearProperty("solr.jetty.keystore");
> +    }
>    }
> 
>    private HttpServletRequest createMockRequest(final
> AtomicReference<Header> header) {
> 
> http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/95d6fc25/solr/test-
> framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
> ----------------------------------------------------------------------
> diff --git a/solr/test-
> framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
> b/solr/test-
> framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
> index 8c6eb60..bbfc048 100644
> --- a/solr/test-
> framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
> +++ b/solr/test-
> framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
> @@ -50,6 +50,7 @@ import org.apache.solr.client.solrj.SolrServerException;
>  import org.apache.solr.client.solrj.embedded.JettyConfig;
>  import org.apache.solr.client.solrj.embedded.JettySolrRunner;
>  import org.apache.solr.client.solrj.impl.HttpSolrClient;
> +import org.apache.solr.client.solrj.request.QueryRequest;
>  import org.apache.solr.client.solrj.request.UpdateRequest;
>  import org.apache.solr.client.solrj.response.QueryResponse;
>  import org.apache.solr.client.solrj.response.UpdateResponse;
> @@ -558,6 +559,12 @@ public abstract class BaseDistributedSearchTestCase
> extends SolrTestCaseJ4 {
>      return rsp;
>    }
> 
> +  protected QueryResponse queryServer(QueryRequest req) throws
> IOException, SolrServerException {
> +    int which = r.nextInt(clients.size());
> +    SolrClient client = clients.get(which);
> +    return req.process(client, null);
> +  }
> +
>    /**
>     * Sets distributed params.
>     * Returns the QueryResponse from {@link #queryServer},
> @@ -591,18 +598,31 @@ public abstract class
> BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
>     * Returns the QueryResponse from {@link #queryServer}
>     */
>    protected QueryResponse query(boolean setDistribParams, SolrParams p)
> throws Exception {
> +    return query(setDistribParams, p, null, null);
> +  }
> +
> +  /**
> +   * Returns the QueryResponse from {@link #queryServer}
> +   * @param setDistribParams whether to do a distributed request
> +   * @param user basic auth username (set to null if not in use)
> +   * @param pass basic auth password (set to null if not in use)
> +   * @return the query response
> +   */
> +  protected QueryResponse query(boolean setDistribParams, SolrParams p,
> String user, String pass) throws Exception {
> 
>      final ModifiableSolrParams params = new ModifiableSolrParams(p);
> 
>      // TODO: look into why passing true causes fails
>      params.set("distrib", "false");
> -    final QueryResponse controlRsp = controlClient.query(params);
> +    QueryRequest req = generateQueryRequest(params, user, pass);
> +    final QueryResponse controlRsp = req.process(controlClient, null);
>      validateControlData(controlRsp);
> 
>      params.remove("distrib");
>      if (setDistribParams) setDistributedParams(params);
> +    req = generateQueryRequest(params, user, pass);
> 
> -    QueryResponse rsp = queryServer(params);
> +    QueryResponse rsp = queryServer(req);
> 
>      compareResponses(rsp, controlRsp);
> 
> @@ -617,7 +637,8 @@ public abstract class BaseDistributedSearchTestCase
> extends SolrTestCaseJ4 {
>                int which = r.nextInt(clients.size());
>                SolrClient client = clients.get(which);
>                try {
> -                QueryResponse rsp = client.query(new
> ModifiableSolrParams(params));
> +                QueryRequest qreq = generateQueryRequest(new
> ModifiableSolrParams(params), user, pass);
> +                QueryResponse rsp = qreq.process(client, null);
>                  if (verifyStress) {
>                    compareResponses(rsp, controlRsp);
>                  }
> @@ -636,7 +657,15 @@ public abstract class BaseDistributedSearchTestCase
> extends SolrTestCaseJ4 {
>      }
>      return rsp;
>    }
> -
> +
> +  private QueryRequest generateQueryRequest(ModifiableSolrParams
> params, String user, String pass) {
> +    QueryRequest req = new QueryRequest(params);
> +    if (user != null && pass != null) {
> +      req.setBasicAuthCredentials(user, pass);
> +    }
> +    return req;
> +  }
> +
>    public QueryResponse queryAndCompare(SolrParams params, SolrClient...
> clients) throws SolrServerException, IOException {
>      return queryAndCompare(params, Arrays.<SolrClient>asList(clients));
>    }


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org