You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by Uwe Schindler <uw...@thetaphi.de> on 2016/12/19 18:36:13 UTC

RE: [1/2] lucene-solr:master: SOLR-9513: Generic Hadoop authentication plugins, GenericHadoopAuthPlugin and ConfigurableInternodeAuthHadoopPlugin

Hi,

Something went wrong with the Java 9 assume in TestSolrCloudWithHadoopAuthPlugin. It should just ignore the test, but for some reason it fails the test. See recent Jenkins failures!

Uwe

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

> -----Original Message-----
> From: ishan@apache.org [mailto:ishan@apache.org]
> Sent: Monday, December 19, 2016 4:16 PM
> To: commits@lucene.apache.org
> Subject: [1/2] lucene-solr:master: SOLR-9513: Generic Hadoop authentication
> plugins, GenericHadoopAuthPlugin and
> ConfigurableInternodeAuthHadoopPlugin
> 
> Repository: lucene-solr
> Updated Branches:
>   refs/heads/master 321c6f090 -> a1a8b2864
> 
> 
> http://git-wip-us.apache.org/repos/asf/lucene-
> solr/blob/a1a8b286/solr/core/src/test/org/apache/solr/security/hadoop/Te
> stImpersonationWithHadoopAuth.java
> ----------------------------------------------------------------------
> diff --git
> a/solr/core/src/test/org/apache/solr/security/hadoop/TestImpersonationWi
> thHadoopAuth.java
> b/solr/core/src/test/org/apache/solr/security/hadoop/TestImpersonationW
> ithHadoopAuth.java
> new file mode 100644
> index 0000000..ed8397b
> --- /dev/null
> +++
> b/solr/core/src/test/org/apache/solr/security/hadoop/TestImpersonationW
> ithHadoopAuth.java
> @@ -0,0 +1,215 @@
> +/*
> + * 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.hadoop;
> +
> +import static
> org.apache.solr.security.HttpParamDelegationTokenPlugin.USER_PARAM;
> +import static org.apache.solr.security.hadoop.ImpersonationUtil.*;
> +
> +import java.net.InetAddress;
> +import java.nio.charset.Charset;
> +import java.nio.file.Files;
> +import java.nio.file.Path;
> +import java.util.HashMap;
> +import java.util.Map;
> +
> +import org.apache.lucene.util.Constants;
> +import org.apache.solr.client.solrj.SolrClient;
> +import org.apache.solr.client.solrj.embedded.JettySolrRunner;
> +import org.apache.solr.client.solrj.impl.HttpSolrClient;
> +import org.apache.solr.client.solrj.request.CollectionAdminRequest;
> +import org.apache.solr.cloud.SolrCloudTestCase;
> +import org.apache.solr.common.params.ModifiableSolrParams;
> +import org.apache.solr.common.util.Utils;
> +import org.apache.solr.security.HadoopAuthPlugin;
> +import org.apache.solr.servlet.SolrRequestParsers;
> +import org.junit.AfterClass;
> +import org.junit.BeforeClass;
> +import org.junit.Test;
> +
> +public class TestImpersonationWithHadoopAuth  extends
> SolrCloudTestCase {
> +  protected static final int NUM_SERVERS = 2;
> +  private static final boolean defaultAddRequestHeadersToContext =
> +      SolrRequestParsers.DEFAULT.isAddRequestHeadersToContext();
> +
> +  @SuppressWarnings("unchecked")
> +  @BeforeClass
> +  public static void setupClass() throws Exception {
> +    assumeFalse("Hadoop does not work on Windows",
> Constants.WINDOWS);
> +
> +    InetAddress loopback = InetAddress.getLoopbackAddress();
> +    Path securityJsonPath =
> TEST_PATH().resolve("security").resolve("hadoop_simple_auth_with_delegat
> ion.json");
> +    String securityJson = new String(Files.readAllBytes(securityJsonPath),
> Charset.defaultCharset());
> +
> +    Map<String, Object> securityConfig = (Map<String,
> Object>)Utils.fromJSONString(securityJson);
> +    Map<String, Object> authConfig = (Map<String,
> Object>)securityConfig.get("authentication");
> +    Map<String,String> proxyUserConfigs = (Map<String,String>) authConfig
> +        .getOrDefault(HadoopAuthPlugin.PROXY_USER_CONFIGS, new
> HashMap<>());
> +    proxyUserConfigs.put("proxyuser.noGroups.hosts", "*");
> +    proxyUserConfigs.put("proxyuser.anyHostAnyUser.hosts", "*");
> +    proxyUserConfigs.put("proxyuser.anyHostAnyUser.groups", "*");
> +    proxyUserConfigs.put("proxyuser.wrongHost.hosts", "1.1.1.1.1.1");
> +    proxyUserConfigs.put("proxyuser.wrongHost.groups", "*");
> +    proxyUserConfigs.put("proxyuser.noHosts.groups", "*");
> +    proxyUserConfigs.put("proxyuser.localHostAnyGroup.hosts",
> +        loopback.getCanonicalHostName() + "," + loopback.getHostName() + ","
> + loopback.getHostAddress());
> +    proxyUserConfigs.put("proxyuser.localHostAnyGroup.groups", "*");
> +    proxyUserConfigs.put("proxyuser.bogusGroup.hosts", "*");
> +    proxyUserConfigs.put("proxyuser.bogusGroup.groups",
> "__some_bogus_group");
> +    proxyUserConfigs.put("proxyuser.anyHostUsersGroup.groups",
> ImpersonationUtil.getUsersFirstGroup());
> +    proxyUserConfigs.put("proxyuser.anyHostUsersGroup.hosts", "*");
> +
> +    authConfig.put(HadoopAuthPlugin.PROXY_USER_CONFIGS,
> proxyUserConfigs);
> +
> +    SolrRequestParsers.DEFAULT.setAddRequestHeadersToContext(true);
> +    System.setProperty("collectionsHandler",
> ImpersonatorCollectionsHandler.class.getName());
> +
> +    configureCluster(NUM_SERVERS)// nodes
> +        .withSecurityJson(Utils.toJSONString(securityConfig))
> +        .addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-
> minimal").resolve("conf"))
> +        .configure();
> +  }
> +
> +  @AfterClass
> +  public static void tearDownClass() throws Exception {
> +
> SolrRequestParsers.DEFAULT.setAddRequestHeadersToContext(defaultAddR
> equestHeadersToContext);
> +    System.clearProperty("collectionsHandler");
> +  }
> +
> +  private SolrClient newSolrClient() {
> +    return new HttpSolrClient.Builder(
> +        cluster.getJettySolrRunner(0).getBaseUrl().toString()).build();
> +  }
> +
> +  @Test
> +  public void testProxyNoConfigGroups() throws Exception {
> +    try (SolrClient solrClient = newSolrClient()) {
> +      solrClient.request(getProxyRequest("noGroups","bar"));
> +      fail("Expected RemoteSolrException");
> +    }
> +    catch (HttpSolrClient.RemoteSolrException ex) {
> +      assertTrue(ex.getLocalizedMessage(),
> ex.getMessage().contains(getExpectedGroupExMsg("noGroups", "bar")));
> +    }
> +  }
> +
> +  @Test
> +  public void testProxyWrongHost() throws Exception {
> +    try (SolrClient solrClient = newSolrClient()) {
> +      solrClient.request(getProxyRequest("wrongHost","bar"));
> +      fail("Expected RemoteSolrException");
> +    }
> +    catch (HttpSolrClient.RemoteSolrException ex) {
> +
> assertTrue(ex.getMessage().contains(getExpectedHostExMsg("wrongHost")))
> ;
> +    }
> +  }
> +
> +  @Test
> +  public void testProxyNoConfigHosts() throws Exception {
> +    try (SolrClient solrClient = newSolrClient()) {
> +      solrClient.request(getProxyRequest("noHosts","bar"));
> +      fail("Expected RemoteSolrException");
> +    }
> +    catch (HttpSolrClient.RemoteSolrException ex) {
> +
> assertTrue(ex.getMessage().contains(getExpectedHostExMsg("noHosts")));
> +    }
> +  }
> +
> +  @Test
> +  public void testProxyValidateAnyHostAnyUser() throws Exception {
> +    try (SolrClient solrClient = newSolrClient()) {
> +      solrClient.request(getProxyRequest("anyHostAnyUser", "bar"));
> +      assertTrue(ImpersonatorCollectionsHandler.called.get());
> +    }
> +  }
> +
> +  @Test
> +  public void testProxyInvalidProxyUser() throws Exception {
> +    try (SolrClient solrClient = newSolrClient()) {
> +      // wrong direction, should fail
> +      solrClient.request(getProxyRequest("bar","anyHostAnyUser"));
> +      fail("Expected RemoteSolrException");
> +    }
> +    catch (HttpSolrClient.RemoteSolrException ex) {
> +      assertTrue(ex.getMessage().contains(getExpectedGroupExMsg("bar",
> "anyHostAnyUser")));
> +    }
> +  }
> +
> +  @Test
> +  public void testProxyValidateHost() throws Exception {
> +    try (SolrClient solrClient = newSolrClient()) {
> +      solrClient.request(getProxyRequest("localHostAnyGroup", "bar"));
> +      assertTrue(ImpersonatorCollectionsHandler.called.get());
> +    }
> +  }
> +
> +  @Test
> +  public void testProxyValidateGroup() throws Exception {
> +    try (SolrClient solrClient = newSolrClient()) {
> +      solrClient.request(getProxyRequest("anyHostUsersGroup",
> System.getProperty("user.name")));
> +      assertTrue(ImpersonatorCollectionsHandler.called.get());
> +    }
> +  }
> +
> +  @Test
> +  public void testProxyInvalidGroup() throws Exception {
> +    try (SolrClient solrClient = newSolrClient()) {
> +      solrClient.request(getProxyRequest("bogusGroup","bar"));
> +      fail("Expected RemoteSolrException");
> +    }
> +    catch (HttpSolrClient.RemoteSolrException ex) {
> +
> assertTrue(ex.getMessage().contains(getExpectedGroupExMsg("bogusGroup
> ", "bar")));
> +    }
> +  }
> +
> +  @Test
> +  public void testProxyNullProxyUser() throws Exception {
> +    try (SolrClient solrClient = newSolrClient()) {
> +      solrClient.request(getProxyRequest("","bar"));
> +      fail("Expected RemoteSolrException");
> +    }
> +    catch (HttpSolrClient.RemoteSolrException ex) {
> +      // this exception is specific to our implementation, don't check a specific
> message.
> +    }
> +  }
> +
> +  @Test
> +  @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/HADOOP-
> 9893")
> +  public void testForwarding() throws Exception {
> +    String collectionName = "forwardingCollection";
> +
> +    // create collection
> +    CollectionAdminRequest.Create create =
> CollectionAdminRequest.createCollection(collectionName, "conf1",
> +        1, 1);
> +    try (SolrClient solrClient = newSolrClient()) {
> +      create.process(solrClient);
> +    }
> +
> +    // try a command to each node, one of them must be forwarded
> +    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
> +      HttpSolrClient client =
> +          new HttpSolrClient.Builder(jetty.getBaseUrl().toString() + "/" +
> collectionName).build();
> +      try {
> +        ModifiableSolrParams params = new ModifiableSolrParams();
> +        params.set("q", "*:*");
> +        params.set(USER_PARAM, "user");
> +        client.query(params);
> +      } finally {
> +        client.close();
> +      }
> +    }
> +  }
> +
> +}
> 
> http://git-wip-us.apache.org/repos/asf/lucene-
> solr/blob/a1a8b286/solr/core/src/test/org/apache/solr/security/hadoop/Te
> stSolrCloudWithHadoopAuthPlugin.java
> ----------------------------------------------------------------------
> diff --git
> a/solr/core/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithHa
> doopAuthPlugin.java
> b/solr/core/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithHa
> doopAuthPlugin.java
> new file mode 100644
> index 0000000..960fd9a
> --- /dev/null
> +++
> b/solr/core/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithHa
> doopAuthPlugin.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.hadoop;
> +
> +import java.io.File;
> +import java.nio.charset.StandardCharsets;
> +
> +import org.apache.commons.io.FileUtils;
> +import org.apache.lucene.util.Constants;
> +import org.apache.solr.client.solrj.SolrQuery;
> +import org.apache.solr.client.solrj.impl.CloudSolrClient;
> +import org.apache.solr.client.solrj.request.CollectionAdminRequest;
> +import org.apache.solr.client.solrj.response.QueryResponse;
> +import org.apache.solr.cloud.AbstractDistribZkTestBase;
> +import org.apache.solr.cloud.KerberosTestServices;
> +import org.apache.solr.cloud.SolrCloudTestCase;
> +import org.apache.solr.common.SolrInputDocument;
> +import org.junit.AfterClass;
> +import org.junit.BeforeClass;
> +import org.junit.Test;
> +
> +public class TestSolrCloudWithHadoopAuthPlugin extends
> SolrCloudTestCase {
> +  protected static final int NUM_SERVERS = 1;
> +  protected static final int NUM_SHARDS = 1;
> +  protected static final int REPLICATION_FACTOR = 1;
> +  private static KerberosTestServices kerberosTestServices;
> +
> +  @BeforeClass
> +  public static void setupClass() throws Exception {
> +    assumeFalse("Hadoop does not work on Windows",
> Constants.WINDOWS);
> +    assumeFalse("FIXME: SOLR-8182: This test fails under Java 9",
> Constants.JRE_IS_MINIMUM_JAVA9);
> +
> +    setupMiniKdc();
> +
> +    configureCluster(NUM_SERVERS)// nodes
> +
> .withSecurityJson(TEST_PATH().resolve("security").resolve("hadoop_kerbero
> s_config.json"))
> +        .addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-
> minimal").resolve("conf"))
> +        .configure();
> +  }
> +
> +  @AfterClass
> +  public static void tearDownClass() throws Exception {
> +    System.clearProperty("java.security.auth.login.config");
> +    System.clearProperty("solr.kerberos.principal");
> +    System.clearProperty("solr.kerberos.keytab");
> +    System.clearProperty("solr.kerberos.name.rules");
> +    System.clearProperty("solr.jaas.debug");
> +    kerberosTestServices.stop();
> +    kerberosTestServices = null;
> +  }
> +
> +  private static void setupMiniKdc() throws Exception {
> +    System.setProperty("solr.jaas.debug", "true");
> +    String kdcDir = createTempDir()+File.separator+"minikdc";
> +    String solrClientPrincipal = "solr";
> +    File keytabFile = new File(kdcDir, "keytabs");
> +    kerberosTestServices = KerberosTestServices.builder()
> +        .withKdc(new File(kdcDir))
> +        .withJaasConfiguration(solrClientPrincipal, keytabFile, "SolrClient")
> +        .build();
> +    String solrServerPrincipal = "HTTP/127.0.0.1";
> +    kerberosTestServices.start();
> +    kerberosTestServices.getKdc().createPrincipal(keytabFile,
> solrServerPrincipal, solrClientPrincipal);
> +
> +    String jaas = "SolrClient {\n"
> +        + " com.sun.security.auth.module.Krb5LoginModule required\n"
> +        + " useKeyTab=true\n"
> +        + " keyTab=\"" + keytabFile.getAbsolutePath() + "\"\n"
> +        + " storeKey=true\n"
> +        + " useTicketCache=false\n"
> +        + " doNotPrompt=true\n"
> +        + " debug=true\n"
> +        + " principal=\"" + solrClientPrincipal + "\";\n"
> +        + "};";
> +
> +    String jaasFilePath = kdcDir+File.separator+"jaas-client.conf";
> +    FileUtils.write(new File(jaasFilePath), jaas, StandardCharsets.UTF_8);
> +    System.setProperty("java.security.auth.login.config", jaasFilePath);
> +    System.setProperty("solr.kerberos.jaas.appname", "SolrClient"); // Get
> this app name from the jaas file
> +
> +    System.setProperty("solr.kerberos.principal", solrServerPrincipal);
> +    System.setProperty("solr.kerberos.keytab",
> keytabFile.getAbsolutePath());
> +    // Extracts 127.0.0.1 from HTTP/127.0.0.1@EXAMPLE.COM
> +    System.setProperty("solr.kerberos.name.rules",
> "RULE:[1:$1@$0](.*EXAMPLE.COM)s/@.*//"
> +        + "\nRULE:[2:$2@$0](.*EXAMPLE.COM)s/@.*//"
> +        + "\nDEFAULT"
> +        );
> +  }
> +
> +  @Test
> +  public void testBasics() throws Exception {
> +    testCollectionCreateSearchDelete();
> +    // sometimes run a second test e.g. to test collection create-delete-create
> scenario
> +    if (random().nextBoolean()) testCollectionCreateSearchDelete();
> +  }
> +
> +  protected void testCollectionCreateSearchDelete() throws Exception {
> +    CloudSolrClient solrClient = cluster.getSolrClient();
> +    String collectionName = "testkerberoscollection";
> +
> +    // create collection
> +    CollectionAdminRequest.Create create =
> CollectionAdminRequest.createCollection(collectionName, "conf1",
> +        NUM_SHARDS, REPLICATION_FACTOR);
> +    create.process(solrClient);
> +
> +    SolrInputDocument doc = new SolrInputDocument();
> +    doc.setField("id", "1");
> +    solrClient.add(collectionName, doc);
> +    solrClient.commit(collectionName);
> +
> +    SolrQuery query = new SolrQuery();
> +    query.setQuery("*:*");
> +    QueryResponse rsp = solrClient.query(collectionName, query);
> +    assertEquals(1, rsp.getResults().getNumFound());
> +
> +    CollectionAdminRequest.Delete deleteReq =
> CollectionAdminRequest.deleteCollection(collectionName);
> +    deleteReq.process(solrClient);
> +    AbstractDistribZkTestBase.waitForCollectionToDisappear(collectionName,
> +        solrClient.getZkStateReader(), true, true, 330);
> +  }
> +
> +}
> 
> http://git-wip-us.apache.org/repos/asf/lucene-
> solr/blob/a1a8b286/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Htt
> pClientBuilderFactory.java
> ----------------------------------------------------------------------
> diff --git
> a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientBuilderFact
> ory.java
> b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientBuilderFact
> ory.java
> new file mode 100644
> index 0000000..77c4a94
> --- /dev/null
> +++
> b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientBuilderFact
> ory.java
> @@ -0,0 +1,41 @@
> +/*
> + * 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.client.solrj.impl;
> +
> +import java.io.Closeable;
> +import java.util.Optional;
> +
> +/**
> + * Factory interface for configuring {@linkplain SolrHttpClientBuilder}. This
> + * relies on the internal HttpClient implementation and is subject to
> + * change.
> + *
> + * @lucene.experimental
> + **/
> +public interface HttpClientBuilderFactory extends Closeable {
> +
> +  /**
> +   * This method configures the {@linkplain SolrHttpClientBuilder} by
> overriding the
> +   * configuration of passed SolrHttpClientBuilder or as a new instance.
> +   *
> +   * @param builder The instance of the {@linkplain SolrHttpClientBuilder}
> which should
> +   *                by configured (optional).
> +   * @return the {@linkplain SolrHttpClientBuilder}
> +   */
> +  public SolrHttpClientBuilder
> getHttpClientBuilder(Optional<SolrHttpClientBuilder> builder);
> +
> +}
> 
> http://git-wip-us.apache.org/repos/asf/lucene-
> solr/blob/a1a8b286/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Kr
> b5HttpClientBuilder.java
> ----------------------------------------------------------------------
> diff --git
> a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder
> .java
> b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder
> .java
> index 39cc2dc..7f3cf29 100644
> ---
> a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder
> .java
> +++
> b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder
> .java
> @@ -21,6 +21,7 @@ import java.security.Principal;
>  import java.util.Arrays;
>  import java.util.HashSet;
>  import java.util.Locale;
> +import java.util.Optional;
>  import java.util.Set;
> 
>  import javax.security.auth.login.AppConfigurationEntry;
> @@ -46,7 +47,7 @@ import org.slf4j.LoggerFactory;
>  /**
>   * Kerberos-enabled SolrHttpClientBuilder
>   */
> -public class Krb5HttpClientBuilder  {
> +public class Krb5HttpClientBuilder implements HttpClientBuilderFactory {
> 
>    public static final String LOGIN_CONFIG_PROP =
> "java.security.auth.login.config";
>    private static final Logger logger =
> LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
> @@ -73,7 +74,12 @@ public class Krb5HttpClientBuilder  {
>    public void close() {
>      HttpClientUtil.removeRequestInterceptor(bufferedEntityInterceptor);
>    }
> -
> +
> +  @Override
> +  public SolrHttpClientBuilder
> getHttpClientBuilder(Optional<SolrHttpClientBuilder> builder) {
> +    return builder.isPresent() ? getBuilder(builder.get()) : getBuilder();
> +  }
> +
>    public SolrHttpClientBuilder getBuilder(SolrHttpClientBuilder builder) {
>      if (System.getProperty(LOGIN_CONFIG_PROP) != null) {
>        String configValue = System.getProperty(LOGIN_CONFIG_PROP);
> 
> http://git-wip-us.apache.org/repos/asf/lucene-
> solr/blob/a1a8b286/solr/test-
> framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
> ----------------------------------------------------------------------
> diff --git a/solr/test-
> framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
> b/solr/test-
> framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
> index 5ebdfb7..81e1f22 100644
> --- a/solr/test-
> framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
> +++ b/solr/test-
> framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
> @@ -28,6 +28,7 @@ import java.util.Collection;
>  import java.util.Collections;
>  import java.util.List;
>  import java.util.Objects;
> +import java.util.Optional;
>  import java.util.Random;
>  import java.util.SortedMap;
>  import java.util.concurrent.Callable;
> @@ -68,6 +69,7 @@ public class MiniSolrCloudCluster {
>        "  <str name=\"shareSchema\">${shareSchema:false}</str>\n" +
>        "  <str
> name=\"configSetBaseDir\">${configSetBaseDir:configsets}</str>\n" +
>        "  <str name=\"coreRootDirectory\">${coreRootDirectory:.}</str>\n" +
> +      "  <str
> name=\"collectionsHandler\">${collectionsHandler:solr.CollectionsHandler}<
> /str>\n" +
>        "\n" +
>        "  <shardHandlerFactory name=\"shardHandlerFactory\"
> class=\"HttpShardHandlerFactory\">\n" +
>        "    <str name=\"urlScheme\">${urlScheme:}</str>\n" +
> @@ -180,8 +182,30 @@ public class MiniSolrCloudCluster {
>     *
>     * @throws Exception if there was an error starting the cluster
>     */
> -  public MiniSolrCloudCluster(int numServers, Path baseDir, String solrXml,
> JettyConfig jettyConfig, ZkTestServer zkTestServer) throws Exception {
> +  public MiniSolrCloudCluster(int numServers, Path baseDir, String solrXml,
> JettyConfig jettyConfig,
> +      ZkTestServer zkTestServer) throws Exception {
> +    this(numServers, baseDir, solrXml, jettyConfig, zkTestServer,
> Optional.empty());
> +  }
> 
> +  /**
> +   * Create a MiniSolrCloudCluster.
> +   * Note - this constructor visibility is changed to package protected so as to
> +   * discourage its usage. Ideally *new* functionality should use {@linkplain
> SolrCloudTestCase}
> +   * to configure any additional parameters.
> +   *
> +   * @param numServers number of Solr servers to start
> +   * @param baseDir base directory that the mini cluster should be run from
> +   * @param solrXml solr.xml file to be uploaded to ZooKeeper
> +   * @param jettyConfig Jetty configuration
> +   * @param zkTestServer ZkTestServer to use.  If null, one will be created
> +   * @param securityJson A string representation of security.json file
> (optional).
> +   *
> +   * @throws Exception if there was an error starting the cluster
> +   */
> +   MiniSolrCloudCluster(int numServers, Path baseDir, String solrXml,
> JettyConfig jettyConfig,
> +      ZkTestServer zkTestServer, Optional<String> securityJson) throws
> Exception {
> +
> +    Objects.requireNonNull(securityJson);
>      this.baseDir = Objects.requireNonNull(baseDir);
>      this.jettyConfig = Objects.requireNonNull(jettyConfig);
> 
> @@ -202,6 +226,9 @@ public class MiniSolrCloudCluster {
>        if (jettyConfig.sslConfig != null && jettyConfig.sslConfig.isSSLMode()) {
>          zkClient.makePath("/solr" + ZkStateReader.CLUSTER_PROPS,
> "{'urlScheme':'https'}".getBytes(StandardCharsets.UTF_8), true);
>        }
> +      if (securityJson.isPresent()) { // configure Solr security
> +        zkClient.makePath("/solr/security.json",
> securityJson.get().getBytes(Charset.defaultCharset()), true);
> +      }
>      }
> 
>      // tell solr to look in zookeeper for solr.xml
> 
> http://git-wip-us.apache.org/repos/asf/lucene-
> solr/blob/a1a8b286/solr/test-
> framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
> ----------------------------------------------------------------------
> diff --git a/solr/test-
> framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
> b/solr/test-
> framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
> index 9cc4a22..34dc8ac 100644
> --- a/solr/test-
> framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
> +++ b/solr/test-
> framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
> @@ -26,6 +26,7 @@ import java.util.Collections;
>  import java.util.HashMap;
>  import java.util.List;
>  import java.util.Map;
> +import java.util.Optional;
>  import java.util.concurrent.TimeUnit;
>  import java.util.concurrent.atomic.AtomicReference;
>  import java.util.function.Predicate;
> @@ -90,6 +91,7 @@ public class SolrCloudTestCase extends SolrTestCaseJ4 {
>      private final Path baseDir;
>      private String solrxml =
> MiniSolrCloudCluster.DEFAULT_CLOUD_SOLR_XML;
>      private JettyConfig jettyConfig = buildJettyConfig("/solr");
> +    private Optional<String> securityJson = Optional.empty();
> 
>      private List<Config> configs = new ArrayList<>();
>      private Map<String, String> clusterProperties = new HashMap<>();
> @@ -133,6 +135,32 @@ public class SolrCloudTestCase extends
> SolrTestCaseJ4 {
>      }
> 
>      /**
> +     * Configure the specified security.json for the {@linkplain
> MiniSolrCloudCluster}
> +     *
> +     * @param securityJson The path specifying the security.json file
> +     * @return the instance of {@linkplain Builder}
> +     */
> +    public Builder withSecurityJson(Path securityJson) {
> +      try {
> +        this.securityJson = Optional.of(new
> String(Files.readAllBytes(securityJson), Charset.defaultCharset()));
> +      } catch (IOException e) {
> +        throw new RuntimeException(e);
> +      }
> +      return this;
> +    }
> +
> +    /**
> +     * Configure the specified security.json for the {@linkplain
> MiniSolrCloudCluster}
> +     *
> +     * @param securityJson The string specifying the security.json
> configuration
> +     * @return the instance of {@linkplain Builder}
> +     */
> +    public Builder withSecurityJson(String securityJson) {
> +      this.securityJson = Optional.of(securityJson);
> +      return this;
> +    }
> +
> +    /**
>       * Upload a collection config before tests start
>       * @param configName the config name
>       * @param configPath the path to the config files
> @@ -157,7 +185,7 @@ public class SolrCloudTestCase extends
> SolrTestCaseJ4 {
>       * @throws Exception if an error occurs on startup
>       */
>      public void configure() throws Exception {
> -      cluster = new MiniSolrCloudCluster(nodeCount, baseDir, solrxml,
> jettyConfig);
> +      cluster = new MiniSolrCloudCluster(nodeCount, baseDir, solrxml,
> jettyConfig, null, securityJson);
>        CloudSolrClient client = cluster.getSolrClient();
>        for (Config config : configs) {
> 
> ((ZkClientClusterStateProvider)client.getClusterStateProvider()).uploadConfig
> (config.path, config.name);


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


RE: [1/2] lucene-solr:master: SOLR-9513: Generic Hadoop authentication plugins, GenericHadoopAuthPlugin and ConfigurableInternodeAuthHadoopPlugin

Posted by Uwe Schindler <uw...@thetaphi.de>.
Hi,

 

Yeah that’s a similar test setup. Windows is also detected with assumeFalse(Constants.WINDOWS), which seems to break for this test.

 

Uwe

 

-----

Uwe Schindler

Achterdiek 19, D-28357 Bremen

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

eMail: uwe@thetaphi.de

 

From: Kevin Risden [mailto:compuwizard123@gmail.com] 
Sent: Monday, December 19, 2016 7:39 PM
To: dev@lucene.apache.org
Subject: Re: [1/2] lucene-solr:master: SOLR-9513: Generic Hadoop authentication plugins, GenericHadoopAuthPlugin and ConfigurableInternodeAuthHadoopPlugin

 

Failed on Java 8 too.

 

Build:  <https://jenkins.thetaphi.de/job/Lucene-Solr-master-Windows/6296/> https://jenkins.thetaphi.de/job/Lucene-Solr-master-Windows/6296/
Java: 32bit/jdk1.8.0_112 -server -XX:+UseG1GC

3 tests failed.
FAILED:  junit.framework.TestSuite.org <http://junit.framework.TestSuite.org> .apache.solr.security.hadoop.TestSolrCloudWithHadoopAuthPlugin




Kevin Risden

 

On Mon, Dec 19, 2016 at 12:36 PM, Uwe Schindler <uwe@thetaphi.de <ma...@thetaphi.de> > wrote:

Hi,

Something went wrong with the Java 9 assume in TestSolrCloudWithHadoopAuthPlugin. It should just ignore the test, but for some reason it fails the test. See recent Jenkins failures!

Uwe

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

> -----Original Message-----
> From: ishan@apache.org <ma...@apache.org>  [mailto:ishan@apache.org <ma...@apache.org> ]
> Sent: Monday, December 19, 2016 4:16 PM
> To: commits@lucene.apache.org <ma...@lucene.apache.org> 
> Subject: [1/2] lucene-solr:master: SOLR-9513: Generic Hadoop authentication
> plugins, GenericHadoopAuthPlugin and
> ConfigurableInternodeAuthHadoopPlugin
>

> Repository: lucene-solr
> Updated Branches:
>   refs/heads/master 321c6f090 -> a1a8b2864
>
>
> http://git-wip-us.apache.org/repos/asf/lucene-
> solr/blob/a1a8b286/solr/core/src/test/org/apache/solr/security/hadoop/Te
> stImpersonationWithHadoopAuth.java
> ----------------------------------------------------------------------
> diff --git
> a/solr/core/src/test/org/apache/solr/security/hadoop/TestImpersonationWi
> thHadoopAuth.java
> b/solr/core/src/test/org/apache/solr/security/hadoop/TestImpersonationW
> ithHadoopAuth.java
> new file mode 100644
> index 0000000..ed8397b
> --- /dev/null
> +++
> b/solr/core/src/test/org/apache/solr/security/hadoop/TestImpersonationW
> ithHadoopAuth.java
> @@ -0,0 +1,215 @@
> +/*
> + * 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.hadoop;
> +
> +import static
> org.apache.solr.security.HttpParamDelegationTokenPlugin.USER_PARAM;
> +import static org.apache.solr.security.hadoop.ImpersonationUtil.*;
> +
> +import java.net.InetAddress;
> +import java.nio.charset.Charset;
> +import java.nio.file.Files;
> +import java.nio.file.Path;
> +import java.util.HashMap;
> +import java.util.Map;
> +
> +import org.apache.lucene.util.Constants;
> +import org.apache.solr.client.solrj.SolrClient;
> +import org.apache.solr.client.solrj.embedded.JettySolrRunner;
> +import org.apache.solr.client.solrj.impl.HttpSolrClient;
> +import org.apache.solr.client.solrj.request.CollectionAdminRequest;
> +import org.apache.solr.cloud.SolrCloudTestCase;
> +import org.apache.solr.common.params.ModifiableSolrParams;
> +import org.apache.solr.common.util.Utils;
> +import org.apache.solr.security.HadoopAuthPlugin;
> +import org.apache.solr.servlet.SolrRequestParsers;
> +import org.junit.AfterClass;
> +import org.junit.BeforeClass;
> +import org.junit.Test;
> +
> +public class TestImpersonationWithHadoopAuth  extends
> SolrCloudTestCase {
> +  protected static final int NUM_SERVERS = 2;
> +  private static final boolean defaultAddRequestHeadersToContext =
> +      SolrRequestParsers.DEFAULT.isAddRequestHeadersToContext();
> +
> +  @SuppressWarnings("unchecked")
> +  @BeforeClass
> +  public static void setupClass() throws Exception {
> +    assumeFalse("Hadoop does not work on Windows",
> Constants.WINDOWS);
> +
> +    InetAddress loopback = InetAddress.getLoopbackAddress();
> +    Path securityJsonPath =
> TEST_PATH().resolve("security").resolve("hadoop_simple_auth_with_delegat
> ion.json");
> +    String securityJson = new String(Files.readAllBytes(securityJsonPath),
> Charset.defaultCharset());
> +
> +    Map<String, Object> securityConfig = (Map<String,
> Object>)Utils.fromJSONString(securityJson);
> +    Map<String, Object> authConfig = (Map<String,
> Object>)securityConfig.get("authentication");
> +    Map<String,String> proxyUserConfigs = (Map<String,String>) authConfig
> +        .getOrDefault(HadoopAuthPlugin.PROXY_USER_CONFIGS, new
> HashMap<>());
> +    proxyUserConfigs.put("proxyuser.noGroups.hosts", "*");
> +    proxyUserConfigs.put("proxyuser.anyHostAnyUser.hosts", "*");
> +    proxyUserConfigs.put("proxyuser.anyHostAnyUser.groups", "*");
> +    proxyUserConfigs.put("proxyuser.wrongHost.hosts", "1.1.1.1.1.1");
> +    proxyUserConfigs.put("proxyuser.wrongHost.groups", "*");
> +    proxyUserConfigs.put("proxyuser.noHosts.groups", "*");
> +    proxyUserConfigs.put("proxyuser.localHostAnyGroup.hosts",
> +        loopback.getCanonicalHostName() + "," + loopback.getHostName() + ","
> + loopback.getHostAddress());
> +    proxyUserConfigs.put("proxyuser.localHostAnyGroup.groups", "*");
> +    proxyUserConfigs.put("proxyuser.bogusGroup.hosts", "*");
> +    proxyUserConfigs.put("proxyuser.bogusGroup.groups",
> "__some_bogus_group");
> +    proxyUserConfigs.put("proxyuser.anyHostUsersGroup.groups",
> ImpersonationUtil.getUsersFirstGroup());
> +    proxyUserConfigs.put("proxyuser.anyHostUsersGroup.hosts", "*");
> +
> +    authConfig.put(HadoopAuthPlugin.PROXY_USER_CONFIGS,
> proxyUserConfigs);
> +
> +    SolrRequestParsers.DEFAULT.setAddRequestHeadersToContext(true);
> +    System.setProperty("collectionsHandler",
> ImpersonatorCollectionsHandler.class.getName());
> +
> +    configureCluster(NUM_SERVERS)// nodes
> +        .withSecurityJson(Utils.toJSONString(securityConfig))
> +        .addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-
> minimal").resolve("conf"))
> +        .configure();
> +  }
> +
> +  @AfterClass
> +  public static void tearDownClass() throws Exception {
> +
> SolrRequestParsers.DEFAULT.setAddRequestHeadersToContext(defaultAddR
> equestHeadersToContext);
> +    System.clearProperty("collectionsHandler");
> +  }
> +
> +  private SolrClient newSolrClient() {
> +    return new HttpSolrClient.Builder(
> +        cluster.getJettySolrRunner(0).getBaseUrl().toString()).build();
> +  }
> +
> +  @Test
> +  public void testProxyNoConfigGroups() throws Exception {
> +    try (SolrClient solrClient = newSolrClient()) {
> +      solrClient.request(getProxyRequest("noGroups","bar"));
> +      fail("Expected RemoteSolrException");
> +    }
> +    catch (HttpSolrClient.RemoteSolrException ex) {
> +      assertTrue(ex.getLocalizedMessage(),
> ex.getMessage().contains(getExpectedGroupExMsg("noGroups", "bar")));
> +    }
> +  }
> +
> +  @Test
> +  public void testProxyWrongHost() throws Exception {
> +    try (SolrClient solrClient = newSolrClient()) {
> +      solrClient.request(getProxyRequest("wrongHost","bar"));
> +      fail("Expected RemoteSolrException");
> +    }
> +    catch (HttpSolrClient.RemoteSolrException ex) {
> +
> assertTrue(ex.getMessage().contains(getExpectedHostExMsg("wrongHost")))
> ;
> +    }
> +  }
> +
> +  @Test
> +  public void testProxyNoConfigHosts() throws Exception {
> +    try (SolrClient solrClient = newSolrClient()) {
> +      solrClient.request(getProxyRequest("noHosts","bar"));
> +      fail("Expected RemoteSolrException");
> +    }
> +    catch (HttpSolrClient.RemoteSolrException ex) {
> +
> assertTrue(ex.getMessage().contains(getExpectedHostExMsg("noHosts")));
> +    }
> +  }
> +
> +  @Test
> +  public void testProxyValidateAnyHostAnyUser() throws Exception {
> +    try (SolrClient solrClient = newSolrClient()) {
> +      solrClient.request(getProxyRequest("anyHostAnyUser", "bar"));
> +      assertTrue(ImpersonatorCollectionsHandler.called.get());
> +    }
> +  }
> +
> +  @Test
> +  public void testProxyInvalidProxyUser() throws Exception {
> +    try (SolrClient solrClient = newSolrClient()) {
> +      // wrong direction, should fail
> +      solrClient.request(getProxyRequest("bar","anyHostAnyUser"));
> +      fail("Expected RemoteSolrException");
> +    }
> +    catch (HttpSolrClient.RemoteSolrException ex) {
> +      assertTrue(ex.getMessage().contains(getExpectedGroupExMsg("bar",
> "anyHostAnyUser")));
> +    }
> +  }
> +
> +  @Test
> +  public void testProxyValidateHost() throws Exception {
> +    try (SolrClient solrClient = newSolrClient()) {
> +      solrClient.request(getProxyRequest("localHostAnyGroup", "bar"));
> +      assertTrue(ImpersonatorCollectionsHandler.called.get());
> +    }
> +  }
> +
> +  @Test
> +  public void testProxyValidateGroup() throws Exception {
> +    try (SolrClient solrClient = newSolrClient()) {
> +      solrClient.request(getProxyRequest("anyHostUsersGroup",
> System.getProperty("user.name <http://user.name> ")));
> +      assertTrue(ImpersonatorCollectionsHandler.called.get());
> +    }
> +  }
> +
> +  @Test
> +  public void testProxyInvalidGroup() throws Exception {
> +    try (SolrClient solrClient = newSolrClient()) {
> +      solrClient.request(getProxyRequest("bogusGroup","bar"));
> +      fail("Expected RemoteSolrException");
> +    }
> +    catch (HttpSolrClient.RemoteSolrException ex) {
> +
> assertTrue(ex.getMessage().contains(getExpectedGroupExMsg("bogusGroup
> ", "bar")));
> +    }
> +  }
> +
> +  @Test
> +  public void testProxyNullProxyUser() throws Exception {
> +    try (SolrClient solrClient = newSolrClient()) {
> +      solrClient.request(getProxyRequest("","bar"));
> +      fail("Expected RemoteSolrException");
> +    }
> +    catch (HttpSolrClient.RemoteSolrException ex) {
> +      // this exception is specific to our implementation, don't check a specific
> message.
> +    }
> +  }
> +
> +  @Test
> +  @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/HADOOP-
> 9893")
> +  public void testForwarding() throws Exception {
> +    String collectionName = "forwardingCollection";
> +
> +    // create collection
> +    CollectionAdminRequest.Create create =
> CollectionAdminRequest.createCollection(collectionName, "conf1",
> +        1, 1);
> +    try (SolrClient solrClient = newSolrClient()) {
> +      create.process(solrClient);
> +    }
> +
> +    // try a command to each node, one of them must be forwarded
> +    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
> +      HttpSolrClient client =
> +          new HttpSolrClient.Builder(jetty.getBaseUrl().toString() + "/" +
> collectionName).build();
> +      try {
> +        ModifiableSolrParams params = new ModifiableSolrParams();
> +        params.set("q", "*:*");
> +        params.set(USER_PARAM, "user");
> +        client.query(params);
> +      } finally {
> +        client.close();
> +      }
> +    }
> +  }
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/lucene-
> solr/blob/a1a8b286/solr/core/src/test/org/apache/solr/security/hadoop/Te
> stSolrCloudWithHadoopAuthPlugin.java
> ----------------------------------------------------------------------
> diff --git
> a/solr/core/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithHa
> doopAuthPlugin.java
> b/solr/core/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithHa
> doopAuthPlugin.java
> new file mode 100644
> index 0000000..960fd9a
> --- /dev/null
> +++
> b/solr/core/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithHa
> doopAuthPlugin.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.hadoop;
> +
> +import java.io.File;
> +import java.nio.charset.StandardCharsets;
> +
> +import org.apache.commons.io <http://org.apache.commons.io> .FileUtils;
> +import org.apache.lucene.util.Constants;
> +import org.apache.solr.client.solrj.SolrQuery;
> +import org.apache.solr.client.solrj.impl.CloudSolrClient;
> +import org.apache.solr.client.solrj.request.CollectionAdminRequest;
> +import org.apache.solr.client.solrj.response.QueryResponse;
> +import org.apache.solr.cloud.AbstractDistribZkTestBase;
> +import org.apache.solr.cloud.KerberosTestServices;
> +import org.apache.solr.cloud.SolrCloudTestCase;
> +import org.apache.solr.common.SolrInputDocument;
> +import org.junit.AfterClass;
> +import org.junit.BeforeClass;
> +import org.junit.Test;
> +
> +public class TestSolrCloudWithHadoopAuthPlugin extends
> SolrCloudTestCase {
> +  protected static final int NUM_SERVERS = 1;
> +  protected static final int NUM_SHARDS = 1;
> +  protected static final int REPLICATION_FACTOR = 1;
> +  private static KerberosTestServices kerberosTestServices;
> +
> +  @BeforeClass
> +  public static void setupClass() throws Exception {
> +    assumeFalse("Hadoop does not work on Windows",
> Constants.WINDOWS);
> +    assumeFalse("FIXME: SOLR-8182: This test fails under Java 9",
> Constants.JRE_IS_MINIMUM_JAVA9);
> +
> +    setupMiniKdc();
> +
> +    configureCluster(NUM_SERVERS)// nodes
> +
> .withSecurityJson(TEST_PATH().resolve("security").resolve("hadoop_kerbero
> s_config.json"))
> +        .addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-
> minimal").resolve("conf"))
> +        .configure();
> +  }
> +
> +  @AfterClass
> +  public static void tearDownClass() throws Exception {
> +    System.clearProperty("java.security.auth.login.config");
> +    System.clearProperty("solr.kerberos.principal");
> +    System.clearProperty("solr.kerberos.keytab");
> +    System.clearProperty("solr.kerberos.name.rules");
> +    System.clearProperty("solr.jaas.debug");
> +    kerberosTestServices.stop();
> +    kerberosTestServices = null;
> +  }
> +
> +  private static void setupMiniKdc() throws Exception {
> +    System.setProperty("solr.jaas.debug", "true");
> +    String kdcDir = createTempDir()+File.separator+"minikdc";
> +    String solrClientPrincipal = "solr";
> +    File keytabFile = new File(kdcDir, "keytabs");
> +    kerberosTestServices = KerberosTestServices.builder()
> +        .withKdc(new File(kdcDir))
> +        .withJaasConfiguration(solrClientPrincipal, keytabFile, "SolrClient")
> +        .build();
> +    String solrServerPrincipal = "HTTP/127.0.0.1 <http://127.0.0.1> ";
> +    kerberosTestServices.start();
> +    kerberosTestServices.getKdc().createPrincipal(keytabFile,
> solrServerPrincipal, solrClientPrincipal);
> +
> +    String jaas = "SolrClient {\n"
> +        + " com.sun.security.auth.module.Krb5LoginModule required\n"
> +        + " useKeyTab=true\n"
> +        + " keyTab=\"" + keytabFile.getAbsolutePath() + "\"\n"
> +        + " storeKey=true\n"
> +        + " useTicketCache=false\n"
> +        + " doNotPrompt=true\n"
> +        + " debug=true\n"
> +        + " principal=\"" + solrClientPrincipal + "\";\n"
> +        + "};";
> +
> +    String jaasFilePath = kdcDir+File.separator+"jaas-client.conf";
> +    FileUtils.write(new File(jaasFilePath), jaas, StandardCharsets.UTF_8);
> +    System.setProperty("java.security.auth.login.config", jaasFilePath);
> +    System.setProperty("solr.kerberos.jaas.appname", "SolrClient"); // Get
> this app name from the jaas file
> +
> +    System.setProperty("solr.kerberos.principal", solrServerPrincipal);
> +    System.setProperty("solr.kerberos.keytab",
> keytabFile.getAbsolutePath());
> +    // Extracts 127.0.0.1 from HTTP/127.0.0.1@EXAMPLE.COM <ma...@EXAMPLE.COM> 
> +    System.setProperty("solr.kerberos.name.rules",
> "RULE:[1:$1@$0](.* <http://EXAMPLE.COM> EXAMPLE.COM) <mailto:s/@.*//> s/@.*//"
> +        + "\nRULE:[2:$2@$0](.* <http://EXAMPLE.COM> EXAMPLE.COM) <mailto:s/@.*//> s/@.*//"
> +        + "\nDEFAULT"
> +        );
> +  }
> +
> +  @Test
> +  public void testBasics() throws Exception {
> +    testCollectionCreateSearchDelete();
> +    // sometimes run a second test e.g. to test collection create-delete-create
> scenario
> +    if (random().nextBoolean()) testCollectionCreateSearchDelete();
> +  }
> +
> +  protected void testCollectionCreateSearchDelete() throws Exception {
> +    CloudSolrClient solrClient = cluster.getSolrClient();
> +    String collectionName = "testkerberoscollection";
> +
> +    // create collection
> +    CollectionAdminRequest.Create create =
> CollectionAdminRequest.createCollection(collectionName, "conf1",
> +        NUM_SHARDS, REPLICATION_FACTOR);
> +    create.process(solrClient);
> +
> +    SolrInputDocument doc = new SolrInputDocument();
> +    doc.setField("id", "1");
> +    solrClient.add(collectionName, doc);
> +    solrClient.commit(collectionName);
> +
> +    SolrQuery query = new SolrQuery();
> +    query.setQuery("*:*");
> +    QueryResponse rsp = solrClient.query(collectionName, query);
> +    assertEquals(1, rsp.getResults().getNumFound());
> +
> +    CollectionAdminRequest.Delete deleteReq =
> CollectionAdminRequest.deleteCollection(collectionName);
> +    deleteReq.process(solrClient);
> +    AbstractDistribZkTestBase.waitForCollectionToDisappear(collectionName,
> +        solrClient.getZkStateReader(), true, true, 330);
> +  }
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/lucene-
> solr/blob/a1a8b286/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Htt
> pClientBuilderFactory.java
> ----------------------------------------------------------------------
> diff --git
> a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientBuilderFact
> ory.java
> b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientBuilderFact
> ory.java
> new file mode 100644
> index 0000000..77c4a94
> --- /dev/null
> +++
> b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpClientBuilderFact
> ory.java
> @@ -0,0 +1,41 @@
> +/*
> + * 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.client.solrj.impl;
> +
> +import java.io.Closeable;
> +import java.util.Optional;
> +
> +/**
> + * Factory interface for configuring {@linkplain SolrHttpClientBuilder}. This
> + * relies on the internal HttpClient implementation and is subject to
> + * change.
> + *
> + * @lucene.experimental
> + **/
> +public interface HttpClientBuilderFactory extends Closeable {
> +
> +  /**
> +   * This method configures the {@linkplain SolrHttpClientBuilder} by
> overriding the
> +   * configuration of passed SolrHttpClientBuilder or as a new instance.
> +   *
> +   * @param builder The instance of the {@linkplain SolrHttpClientBuilder}
> which should
> +   *                by configured (optional).
> +   * @return the {@linkplain SolrHttpClientBuilder}
> +   */
> +  public SolrHttpClientBuilder
> getHttpClientBuilder(Optional<SolrHttpClientBuilder> builder);
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/lucene-
> solr/blob/a1a8b286/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Kr
> b5HttpClientBuilder.java
> ----------------------------------------------------------------------
> diff --git
> a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder
> .java
> b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder
> .java
> index 39cc2dc..7f3cf29 100644
> ---
> a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder
> .java
> +++
> b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/Krb5HttpClientBuilder
> .java
> @@ -21,6 +21,7 @@ import java.security.Principal;
>  import java.util.Arrays;
>  import java.util.HashSet;
>  import java.util.Locale;
> +import java.util.Optional;
>  import java.util.Set;
>
>  import javax.security.auth.login.AppConfigurationEntry;
> @@ -46,7 +47,7 @@ import org.slf4j.LoggerFactory;
>  /**
>   * Kerberos-enabled SolrHttpClientBuilder
>   */
> -public class Krb5HttpClientBuilder  {
> +public class Krb5HttpClientBuilder implements HttpClientBuilderFactory {
>
>    public static final String LOGIN_CONFIG_PROP =
> "java.security.auth.login.config";
>    private static final Logger logger =
> LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
> @@ -73,7 +74,12 @@ public class Krb5HttpClientBuilder  {
>    public void close() {
>      HttpClientUtil.removeRequestInterceptor(bufferedEntityInterceptor);
>    }
> -
> +
> +  @Override
> +  public SolrHttpClientBuilder
> getHttpClientBuilder(Optional<SolrHttpClientBuilder> builder) {
> +    return builder.isPresent() ? getBuilder(builder.get()) : getBuilder();
> +  }
> +
>    public SolrHttpClientBuilder getBuilder(SolrHttpClientBuilder builder) {
>      if (System.getProperty(LOGIN_CONFIG_PROP) != null) {
>        String configValue = System.getProperty(LOGIN_CONFIG_PROP);
>
> http://git-wip-us.apache.org/repos/asf/lucene-
> solr/blob/a1a8b286/solr/test-
> framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
> ----------------------------------------------------------------------
> diff --git a/solr/test-
> framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
> b/solr/test-
> framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
> index 5ebdfb7..81e1f22 100644
> --- a/solr/test-
> framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
> +++ b/solr/test-
> framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
> @@ -28,6 +28,7 @@ import java.util.Collection;
>  import java.util.Collections;
>  import java.util.List;
>  import java.util.Objects;
> +import java.util.Optional;
>  import java.util.Random;
>  import java.util.SortedMap;
>  import java.util.concurrent.Callable;
> @@ -68,6 +69,7 @@ public class MiniSolrCloudCluster {
>        "  <str name=\"shareSchema\">${shareSchema:false}</str>\n" +
>        "  <str
> name=\"configSetBaseDir\">${configSetBaseDir:configsets}</str>\n" +
>        "  <str name=\"coreRootDirectory\">${coreRootDirectory:.}</str>\n" +
> +      "  <str
> name=\"collectionsHandler\">${collectionsHandler:solr.CollectionsHandler}<
> /str>\n" +
>        "\n" +
>        "  <shardHandlerFactory name=\"shardHandlerFactory\"
> class=\"HttpShardHandlerFactory\">\n" +
>        "    <str name=\"urlScheme\">${urlScheme:}</str>\n" +
> @@ -180,8 +182,30 @@ public class MiniSolrCloudCluster {
>     *
>     * @throws Exception if there was an error starting the cluster
>     */
> -  public MiniSolrCloudCluster(int numServers, Path baseDir, String solrXml,
> JettyConfig jettyConfig, ZkTestServer zkTestServer) throws Exception {
> +  public MiniSolrCloudCluster(int numServers, Path baseDir, String solrXml,
> JettyConfig jettyConfig,
> +      ZkTestServer zkTestServer) throws Exception {
> +    this(numServers, baseDir, solrXml, jettyConfig, zkTestServer,
> Optional.empty());
> +  }
>
> +  /**
> +   * Create a MiniSolrCloudCluster.
> +   * Note - this constructor visibility is changed to package protected so as to
> +   * discourage its usage. Ideally *new* functionality should use {@linkplain
> SolrCloudTestCase}
> +   * to configure any additional parameters.
> +   *
> +   * @param numServers number of Solr servers to start
> +   * @param baseDir base directory that the mini cluster should be run from
> +   * @param solrXml solr.xml file to be uploaded to ZooKeeper
> +   * @param jettyConfig Jetty configuration
> +   * @param zkTestServer ZkTestServer to use.  If null, one will be created
> +   * @param securityJson A string representation of security.json file
> (optional).
> +   *
> +   * @throws Exception if there was an error starting the cluster
> +   */
> +   MiniSolrCloudCluster(int numServers, Path baseDir, String solrXml,
> JettyConfig jettyConfig,
> +      ZkTestServer zkTestServer, Optional<String> securityJson) throws
> Exception {
> +
> +    Objects.requireNonNull(securityJson);
>      this.baseDir = Objects.requireNonNull(baseDir);
>      this.jettyConfig = Objects.requireNonNull(jettyConfig);
>
> @@ -202,6 +226,9 @@ public class MiniSolrCloudCluster {
>        if (jettyConfig.sslConfig != null && jettyConfig.sslConfig.isSSLMode()) {
>          zkClient.makePath("/solr" + ZkStateReader.CLUSTER_PROPS,
> "{'urlScheme':'https'}".getBytes(StandardCharsets.UTF_8), true);
>        }
> +      if (securityJson.isPresent()) { // configure Solr security
> +        zkClient.makePath("/solr/security.json",
> securityJson.get().getBytes(Charset.defaultCharset()), true);
> +      }
>      }
>
>      // tell solr to look in zookeeper for solr.xml
>
> http://git-wip-us.apache.org/repos/asf/lucene-
> solr/blob/a1a8b286/solr/test-
> framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
> ----------------------------------------------------------------------
> diff --git a/solr/test-
> framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
> b/solr/test-
> framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
> index 9cc4a22..34dc8ac 100644
> --- a/solr/test-
> framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
> +++ b/solr/test-
> framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
> @@ -26,6 +26,7 @@ import java.util.Collections;
>  import java.util.HashMap;
>  import java.util.List;
>  import java.util.Map;
> +import java.util.Optional;
>  import java.util.concurrent.TimeUnit;
>  import java.util.concurrent.atomic.AtomicReference;
>  import java.util.function.Predicate;
> @@ -90,6 +91,7 @@ public class SolrCloudTestCase extends SolrTestCaseJ4 {
>      private final Path baseDir;
>      private String solrxml =
> MiniSolrCloudCluster.DEFAULT_CLOUD_SOLR_XML;
>      private JettyConfig jettyConfig = buildJettyConfig("/solr");
> +    private Optional<String> securityJson = Optional.empty();
>
>      private List<Config> configs = new ArrayList<>();
>      private Map<String, String> clusterProperties = new HashMap<>();
> @@ -133,6 +135,32 @@ public class SolrCloudTestCase extends
> SolrTestCaseJ4 {
>      }
>
>      /**
> +     * Configure the specified security.json for the {@linkplain
> MiniSolrCloudCluster}
> +     *
> +     * @param securityJson The path specifying the security.json file
> +     * @return the instance of {@linkplain Builder}
> +     */
> +    public Builder withSecurityJson(Path securityJson) {
> +      try {
> +        this.securityJson = Optional.of(new
> String(Files.readAllBytes(securityJson), Charset.defaultCharset()));
> +      } catch (IOException e) {
> +        throw new RuntimeException(e);
> +      }
> +      return this;
> +    }
> +
> +    /**
> +     * Configure the specified security.json for the {@linkplain
> MiniSolrCloudCluster}
> +     *
> +     * @param securityJson The string specifying the security.json
> configuration
> +     * @return the instance of {@linkplain Builder}
> +     */
> +    public Builder withSecurityJson(String securityJson) {
> +      this.securityJson = Optional.of(securityJson);
> +      return this;
> +    }
> +
> +    /**
>       * Upload a collection config before tests start
>       * @param configName the config name
>       * @param configPath the path to the config files
> @@ -157,7 +185,7 @@ public class SolrCloudTestCase extends
> SolrTestCaseJ4 {
>       * @throws Exception if an error occurs on startup
>       */
>      public void configure() throws Exception {
> -      cluster = new MiniSolrCloudCluster(nodeCount, baseDir, solrxml,
> jettyConfig);
> +      cluster = new MiniSolrCloudCluster(nodeCount, baseDir, solrxml,
> jettyConfig, null, securityJson);
>        CloudSolrClient client = cluster.getSolrClient();
>        for (Config config : configs) {
>
> ((ZkClientClusterStateProvider)client.getClusterStateProvider()).uploadConfig
> (config.path, config.name <http://config.name> );



---------------------------------------------------------------------
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: [1/2] lucene-solr:master: SOLR-9513: Generic Hadoop authentication plugins, GenericHadoopAuthPlugin and ConfigurableInternodeAuthHadoopPlugin

Posted by Kevin Risden <co...@gmail.com>.
Failed on Java 8 too.

Build: https://jenkins.thetaphi.de/job/Lucene-Solr-master-Windows/6296/
Java: 32bit/jdk1.8.0_112 -server -XX:+UseG1GC

3 tests failed.
FAILED:  junit.framework.TestSuite.org.apache.solr.security.hadoop.
TestSolrCloudWithHadoopAuthPlugin

Kevin Risden

On Mon, Dec 19, 2016 at 12:36 PM, Uwe Schindler <uw...@thetaphi.de> wrote:

> Hi,
>
> Something went wrong with the Java 9 assume in
> TestSolrCloudWithHadoopAuthPlugin. It should just ignore the test, but
> for some reason it fails the test. See recent Jenkins failures!
>
> Uwe
>
> -----
> Uwe Schindler
> Achterdiek 19, D-28357 Bremen
> http://www.thetaphi.de
> eMail: uwe@thetaphi.de
>
> > -----Original Message-----
> > From: ishan@apache.org [mailto:ishan@apache.org]
> > Sent: Monday, December 19, 2016 4:16 PM
> > To: commits@lucene.apache.org
> > Subject: [1/2] lucene-solr:master: SOLR-9513: Generic Hadoop
> authentication
> > plugins, GenericHadoopAuthPlugin and
> > ConfigurableInternodeAuthHadoopPlugin
> >
> > Repository: lucene-solr
> > Updated Branches:
> >   refs/heads/master 321c6f090 -> a1a8b2864
> >
> >
> > http://git-wip-us.apache.org/repos/asf/lucene-
> > solr/blob/a1a8b286/solr/core/src/test/org/apache/solr/security/hadoop/Te
> > stImpersonationWithHadoopAuth.java
> > ----------------------------------------------------------------------
> > diff --git
> > a/solr/core/src/test/org/apache/solr/security/hadoop/TestImpersonationWi
> > thHadoopAuth.java
> > b/solr/core/src/test/org/apache/solr/security/hadoop/TestImpersonationW
> > ithHadoopAuth.java
> > new file mode 100644
> > index 0000000..ed8397b
> > --- /dev/null
> > +++
> > b/solr/core/src/test/org/apache/solr/security/hadoop/TestImpersonationW
> > ithHadoopAuth.java
> > @@ -0,0 +1,215 @@
> > +/*
> > + * 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.hadoop;
> > +
> > +import static
> > org.apache.solr.security.HttpParamDelegationTokenPlugin.USER_PARAM;
> > +import static org.apache.solr.security.hadoop.ImpersonationUtil.*;
> > +
> > +import java.net.InetAddress;
> > +import java.nio.charset.Charset;
> > +import java.nio.file.Files;
> > +import java.nio.file.Path;
> > +import java.util.HashMap;
> > +import java.util.Map;
> > +
> > +import org.apache.lucene.util.Constants;
> > +import org.apache.solr.client.solrj.SolrClient;
> > +import org.apache.solr.client.solrj.embedded.JettySolrRunner;
> > +import org.apache.solr.client.solrj.impl.HttpSolrClient;
> > +import org.apache.solr.client.solrj.request.CollectionAdminRequest;
> > +import org.apache.solr.cloud.SolrCloudTestCase;
> > +import org.apache.solr.common.params.ModifiableSolrParams;
> > +import org.apache.solr.common.util.Utils;
> > +import org.apache.solr.security.HadoopAuthPlugin;
> > +import org.apache.solr.servlet.SolrRequestParsers;
> > +import org.junit.AfterClass;
> > +import org.junit.BeforeClass;
> > +import org.junit.Test;
> > +
> > +public class TestImpersonationWithHadoopAuth  extends
> > SolrCloudTestCase {
> > +  protected static final int NUM_SERVERS = 2;
> > +  private static final boolean defaultAddRequestHeadersToContext =
> > +      SolrRequestParsers.DEFAULT.isAddRequestHeadersToContext();
> > +
> > +  @SuppressWarnings("unchecked")
> > +  @BeforeClass
> > +  public static void setupClass() throws Exception {
> > +    assumeFalse("Hadoop does not work on Windows",
> > Constants.WINDOWS);
> > +
> > +    InetAddress loopback = InetAddress.getLoopbackAddress();
> > +    Path securityJsonPath =
> > TEST_PATH().resolve("security").resolve("hadoop_simple_auth_with_delegat
> > ion.json");
> > +    String securityJson = new String(Files.readAllBytes(
> securityJsonPath),
> > Charset.defaultCharset());
> > +
> > +    Map<String, Object> securityConfig = (Map<String,
> > Object>)Utils.fromJSONString(securityJson);
> > +    Map<String, Object> authConfig = (Map<String,
> > Object>)securityConfig.get("authentication");
> > +    Map<String,String> proxyUserConfigs = (Map<String,String>)
> authConfig
> > +        .getOrDefault(HadoopAuthPlugin.PROXY_USER_CONFIGS, new
> > HashMap<>());
> > +    proxyUserConfigs.put("proxyuser.noGroups.hosts", "*");
> > +    proxyUserConfigs.put("proxyuser.anyHostAnyUser.hosts", "*");
> > +    proxyUserConfigs.put("proxyuser.anyHostAnyUser.groups", "*");
> > +    proxyUserConfigs.put("proxyuser.wrongHost.hosts", "1.1.1.1.1.1");
> > +    proxyUserConfigs.put("proxyuser.wrongHost.groups", "*");
> > +    proxyUserConfigs.put("proxyuser.noHosts.groups", "*");
> > +    proxyUserConfigs.put("proxyuser.localHostAnyGroup.hosts",
> > +        loopback.getCanonicalHostName() + "," + loopback.getHostName()
> + ","
> > + loopback.getHostAddress());
> > +    proxyUserConfigs.put("proxyuser.localHostAnyGroup.groups", "*");
> > +    proxyUserConfigs.put("proxyuser.bogusGroup.hosts", "*");
> > +    proxyUserConfigs.put("proxyuser.bogusGroup.groups",
> > "__some_bogus_group");
> > +    proxyUserConfigs.put("proxyuser.anyHostUsersGroup.groups",
> > ImpersonationUtil.getUsersFirstGroup());
> > +    proxyUserConfigs.put("proxyuser.anyHostUsersGroup.hosts", "*");
> > +
> > +    authConfig.put(HadoopAuthPlugin.PROXY_USER_CONFIGS,
> > proxyUserConfigs);
> > +
> > +    SolrRequestParsers.DEFAULT.setAddRequestHeadersToContext(true);
> > +    System.setProperty("collectionsHandler",
> > ImpersonatorCollectionsHandler.class.getName());
> > +
> > +    configureCluster(NUM_SERVERS)// nodes
> > +        .withSecurityJson(Utils.toJSONString(securityConfig))
> > +        .addConfig("conf1", TEST_PATH().resolve("
> configsets").resolve("cloud-
> > minimal").resolve("conf"))
> > +        .configure();
> > +  }
> > +
> > +  @AfterClass
> > +  public static void tearDownClass() throws Exception {
> > +
> > SolrRequestParsers.DEFAULT.setAddRequestHeadersToContext(defaultAddR
> > equestHeadersToContext);
> > +    System.clearProperty("collectionsHandler");
> > +  }
> > +
> > +  private SolrClient newSolrClient() {
> > +    return new HttpSolrClient.Builder(
> > +        cluster.getJettySolrRunner(0).getBaseUrl().toString()).build();
> > +  }
> > +
> > +  @Test
> > +  public void testProxyNoConfigGroups() throws Exception {
> > +    try (SolrClient solrClient = newSolrClient()) {
> > +      solrClient.request(getProxyRequest("noGroups","bar"));
> > +      fail("Expected RemoteSolrException");
> > +    }
> > +    catch (HttpSolrClient.RemoteSolrException ex) {
> > +      assertTrue(ex.getLocalizedMessage(),
> > ex.getMessage().contains(getExpectedGroupExMsg("noGroups", "bar")));
> > +    }
> > +  }
> > +
> > +  @Test
> > +  public void testProxyWrongHost() throws Exception {
> > +    try (SolrClient solrClient = newSolrClient()) {
> > +      solrClient.request(getProxyRequest("wrongHost","bar"));
> > +      fail("Expected RemoteSolrException");
> > +    }
> > +    catch (HttpSolrClient.RemoteSolrException ex) {
> > +
> > assertTrue(ex.getMessage().contains(getExpectedHostExMsg("wrongHost")))
> > ;
> > +    }
> > +  }
> > +
> > +  @Test
> > +  public void testProxyNoConfigHosts() throws Exception {
> > +    try (SolrClient solrClient = newSolrClient()) {
> > +      solrClient.request(getProxyRequest("noHosts","bar"));
> > +      fail("Expected RemoteSolrException");
> > +    }
> > +    catch (HttpSolrClient.RemoteSolrException ex) {
> > +
> > assertTrue(ex.getMessage().contains(getExpectedHostExMsg("noHosts")));
> > +    }
> > +  }
> > +
> > +  @Test
> > +  public void testProxyValidateAnyHostAnyUser() throws Exception {
> > +    try (SolrClient solrClient = newSolrClient()) {
> > +      solrClient.request(getProxyRequest("anyHostAnyUser", "bar"));
> > +      assertTrue(ImpersonatorCollectionsHandler.called.get());
> > +    }
> > +  }
> > +
> > +  @Test
> > +  public void testProxyInvalidProxyUser() throws Exception {
> > +    try (SolrClient solrClient = newSolrClient()) {
> > +      // wrong direction, should fail
> > +      solrClient.request(getProxyRequest("bar","anyHostAnyUser"));
> > +      fail("Expected RemoteSolrException");
> > +    }
> > +    catch (HttpSolrClient.RemoteSolrException ex) {
> > +      assertTrue(ex.getMessage().contains(getExpectedGroupExMsg("bar",
> > "anyHostAnyUser")));
> > +    }
> > +  }
> > +
> > +  @Test
> > +  public void testProxyValidateHost() throws Exception {
> > +    try (SolrClient solrClient = newSolrClient()) {
> > +      solrClient.request(getProxyRequest("localHostAnyGroup", "bar"));
> > +      assertTrue(ImpersonatorCollectionsHandler.called.get());
> > +    }
> > +  }
> > +
> > +  @Test
> > +  public void testProxyValidateGroup() throws Exception {
> > +    try (SolrClient solrClient = newSolrClient()) {
> > +      solrClient.request(getProxyRequest("anyHostUsersGroup",
> > System.getProperty("user.name")));
> > +      assertTrue(ImpersonatorCollectionsHandler.called.get());
> > +    }
> > +  }
> > +
> > +  @Test
> > +  public void testProxyInvalidGroup() throws Exception {
> > +    try (SolrClient solrClient = newSolrClient()) {
> > +      solrClient.request(getProxyRequest("bogusGroup","bar"));
> > +      fail("Expected RemoteSolrException");
> > +    }
> > +    catch (HttpSolrClient.RemoteSolrException ex) {
> > +
> > assertTrue(ex.getMessage().contains(getExpectedGroupExMsg("bogusGroup
> > ", "bar")));
> > +    }
> > +  }
> > +
> > +  @Test
> > +  public void testProxyNullProxyUser() throws Exception {
> > +    try (SolrClient solrClient = newSolrClient()) {
> > +      solrClient.request(getProxyRequest("","bar"));
> > +      fail("Expected RemoteSolrException");
> > +    }
> > +    catch (HttpSolrClient.RemoteSolrException ex) {
> > +      // this exception is specific to our implementation, don't check
> a specific
> > message.
> > +    }
> > +  }
> > +
> > +  @Test
> > +  @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/HADOOP-
> > 9893")
> > +  public void testForwarding() throws Exception {
> > +    String collectionName = "forwardingCollection";
> > +
> > +    // create collection
> > +    CollectionAdminRequest.Create create =
> > CollectionAdminRequest.createCollection(collectionName, "conf1",
> > +        1, 1);
> > +    try (SolrClient solrClient = newSolrClient()) {
> > +      create.process(solrClient);
> > +    }
> > +
> > +    // try a command to each node, one of them must be forwarded
> > +    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
> > +      HttpSolrClient client =
> > +          new HttpSolrClient.Builder(jetty.getBaseUrl().toString() +
> "/" +
> > collectionName).build();
> > +      try {
> > +        ModifiableSolrParams params = new ModifiableSolrParams();
> > +        params.set("q", "*:*");
> > +        params.set(USER_PARAM, "user");
> > +        client.query(params);
> > +      } finally {
> > +        client.close();
> > +      }
> > +    }
> > +  }
> > +
> > +}
> >
> > http://git-wip-us.apache.org/repos/asf/lucene-
> > solr/blob/a1a8b286/solr/core/src/test/org/apache/solr/security/hadoop/Te
> > stSolrCloudWithHadoopAuthPlugin.java
> > ----------------------------------------------------------------------
> > diff --git
> > a/solr/core/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithHa
> > doopAuthPlugin.java
> > b/solr/core/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithHa
> > doopAuthPlugin.java
> > new file mode 100644
> > index 0000000..960fd9a
> > --- /dev/null
> > +++
> > b/solr/core/src/test/org/apache/solr/security/hadoop/TestSolrCloudWithHa
> > doopAuthPlugin.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.hadoop;
> > +
> > +import java.io.File;
> > +import java.nio.charset.StandardCharsets;
> > +
> > +import org.apache.commons.io.FileUtils;
> > +import org.apache.lucene.util.Constants;
> > +import org.apache.solr.client.solrj.SolrQuery;
> > +import org.apache.solr.client.solrj.impl.CloudSolrClient;
> > +import org.apache.solr.client.solrj.request.CollectionAdminRequest;
> > +import org.apache.solr.client.solrj.response.QueryResponse;
> > +import org.apache.solr.cloud.AbstractDistribZkTestBase;
> > +import org.apache.solr.cloud.KerberosTestServices;
> > +import org.apache.solr.cloud.SolrCloudTestCase;
> > +import org.apache.solr.common.SolrInputDocument;
> > +import org.junit.AfterClass;
> > +import org.junit.BeforeClass;
> > +import org.junit.Test;
> > +
> > +public class TestSolrCloudWithHadoopAuthPlugin extends
> > SolrCloudTestCase {
> > +  protected static final int NUM_SERVERS = 1;
> > +  protected static final int NUM_SHARDS = 1;
> > +  protected static final int REPLICATION_FACTOR = 1;
> > +  private static KerberosTestServices kerberosTestServices;
> > +
> > +  @BeforeClass
> > +  public static void setupClass() throws Exception {
> > +    assumeFalse("Hadoop does not work on Windows",
> > Constants.WINDOWS);
> > +    assumeFalse("FIXME: SOLR-8182: This test fails under Java 9",
> > Constants.JRE_IS_MINIMUM_JAVA9);
> > +
> > +    setupMiniKdc();
> > +
> > +    configureCluster(NUM_SERVERS)// nodes
> > +
> > .withSecurityJson(TEST_PATH().resolve("security").resolve("
> hadoop_kerbero
> > s_config.json"))
> > +        .addConfig("conf1", TEST_PATH().resolve("
> configsets").resolve("cloud-
> > minimal").resolve("conf"))
> > +        .configure();
> > +  }
> > +
> > +  @AfterClass
> > +  public static void tearDownClass() throws Exception {
> > +    System.clearProperty("java.security.auth.login.config");
> > +    System.clearProperty("solr.kerberos.principal");
> > +    System.clearProperty("solr.kerberos.keytab");
> > +    System.clearProperty("solr.kerberos.name.rules");
> > +    System.clearProperty("solr.jaas.debug");
> > +    kerberosTestServices.stop();
> > +    kerberosTestServices = null;
> > +  }
> > +
> > +  private static void setupMiniKdc() throws Exception {
> > +    System.setProperty("solr.jaas.debug", "true");
> > +    String kdcDir = createTempDir()+File.separator+"minikdc";
> > +    String solrClientPrincipal = "solr";
> > +    File keytabFile = new File(kdcDir, "keytabs");
> > +    kerberosTestServices = KerberosTestServices.builder()
> > +        .withKdc(new File(kdcDir))
> > +        .withJaasConfiguration(solrClientPrincipal, keytabFile,
> "SolrClient")
> > +        .build();
> > +    String solrServerPrincipal = "HTTP/127.0.0.1";
> > +    kerberosTestServices.start();
> > +    kerberosTestServices.getKdc().createPrincipal(keytabFile,
> > solrServerPrincipal, solrClientPrincipal);
> > +
> > +    String jaas = "SolrClient {\n"
> > +        + " com.sun.security.auth.module.Krb5LoginModule required\n"
> > +        + " useKeyTab=true\n"
> > +        + " keyTab=\"" + keytabFile.getAbsolutePath() + "\"\n"
> > +        + " storeKey=true\n"
> > +        + " useTicketCache=false\n"
> > +        + " doNotPrompt=true\n"
> > +        + " debug=true\n"
> > +        + " principal=\"" + solrClientPrincipal + "\";\n"
> > +        + "};";
> > +
> > +    String jaasFilePath = kdcDir+File.separator+"jaas-client.conf";
> > +    FileUtils.write(new File(jaasFilePath), jaas,
> StandardCharsets.UTF_8);
> > +    System.setProperty("java.security.auth.login.config",
> jaasFilePath);
> > +    System.setProperty("solr.kerberos.jaas.appname", "SolrClient"); //
> Get
> > this app name from the jaas file
> > +
> > +    System.setProperty("solr.kerberos.principal", solrServerPrincipal);
> > +    System.setProperty("solr.kerberos.keytab",
> > keytabFile.getAbsolutePath());
> > +    // Extracts 127.0.0.1 from HTTP/127.0.0.1@EXAMPLE.COM
> > +    System.setProperty("solr.kerberos.name.rules",
> > "RULE:[1:$1@$0](.*EXAMPLE.COM)s/@.*//"
> > +        + "\nRULE:[2:$2@$0](.*EXAMPLE.COM)s/@.*//"
> > +        + "\nDEFAULT"
> > +        );
> > +  }
> > +
> > +  @Test
> > +  public void testBasics() throws Exception {
> > +    testCollectionCreateSearchDelete();
> > +    // sometimes run a second test e.g. to test collection
> create-delete-create
> > scenario
> > +    if (random().nextBoolean()) testCollectionCreateSearchDelete();
> > +  }
> > +
> > +  protected void testCollectionCreateSearchDelete() throws Exception {
> > +    CloudSolrClient solrClient = cluster.getSolrClient();
> > +    String collectionName = "testkerberoscollection";
> > +
> > +    // create collection
> > +    CollectionAdminRequest.Create create =
> > CollectionAdminRequest.createCollection(collectionName, "conf1",
> > +        NUM_SHARDS, REPLICATION_FACTOR);
> > +    create.process(solrClient);
> > +
> > +    SolrInputDocument doc = new SolrInputDocument();
> > +    doc.setField("id", "1");
> > +    solrClient.add(collectionName, doc);
> > +    solrClient.commit(collectionName);
> > +
> > +    SolrQuery query = new SolrQuery();
> > +    query.setQuery("*:*");
> > +    QueryResponse rsp = solrClient.query(collectionName, query);
> > +    assertEquals(1, rsp.getResults().getNumFound());
> > +
> > +    CollectionAdminRequest.Delete deleteReq =
> > CollectionAdminRequest.deleteCollection(collectionName);
> > +    deleteReq.process(solrClient);
> > +    AbstractDistribZkTestBase.waitForCollectionToDisappear(
> collectionName,
> > +        solrClient.getZkStateReader(), true, true, 330);
> > +  }
> > +
> > +}
> >
> > http://git-wip-us.apache.org/repos/asf/lucene-
> > solr/blob/a1a8b286/solr/solrj/src/java/org/apache/solr/
> client/solrj/impl/Htt
> > pClientBuilderFactory.java
> > ----------------------------------------------------------------------
> > diff --git
> > a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/
> HttpClientBuilderFact
> > ory.java
> > b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/
> HttpClientBuilderFact
> > ory.java
> > new file mode 100644
> > index 0000000..77c4a94
> > --- /dev/null
> > +++
> > b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/
> HttpClientBuilderFact
> > ory.java
> > @@ -0,0 +1,41 @@
> > +/*
> > + * 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.client.solrj.impl;
> > +
> > +import java.io.Closeable;
> > +import java.util.Optional;
> > +
> > +/**
> > + * Factory interface for configuring {@linkplain
> SolrHttpClientBuilder}. This
> > + * relies on the internal HttpClient implementation and is subject to
> > + * change.
> > + *
> > + * @lucene.experimental
> > + **/
> > +public interface HttpClientBuilderFactory extends Closeable {
> > +
> > +  /**
> > +   * This method configures the {@linkplain SolrHttpClientBuilder} by
> > overriding the
> > +   * configuration of passed SolrHttpClientBuilder or as a new instance.
> > +   *
> > +   * @param builder The instance of the {@linkplain
> SolrHttpClientBuilder}
> > which should
> > +   *                by configured (optional).
> > +   * @return the {@linkplain SolrHttpClientBuilder}
> > +   */
> > +  public SolrHttpClientBuilder
> > getHttpClientBuilder(Optional<SolrHttpClientBuilder> builder);
> > +
> > +}
> >
> > http://git-wip-us.apache.org/repos/asf/lucene-
> > solr/blob/a1a8b286/solr/solrj/src/java/org/apache/solr/
> client/solrj/impl/Kr
> > b5HttpClientBuilder.java
> > ----------------------------------------------------------------------
> > diff --git
> > a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/
> Krb5HttpClientBuilder
> > .java
> > b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/
> Krb5HttpClientBuilder
> > .java
> > index 39cc2dc..7f3cf29 100644
> > ---
> > a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/
> Krb5HttpClientBuilder
> > .java
> > +++
> > b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/
> Krb5HttpClientBuilder
> > .java
> > @@ -21,6 +21,7 @@ import java.security.Principal;
> >  import java.util.Arrays;
> >  import java.util.HashSet;
> >  import java.util.Locale;
> > +import java.util.Optional;
> >  import java.util.Set;
> >
> >  import javax.security.auth.login.AppConfigurationEntry;
> > @@ -46,7 +47,7 @@ import org.slf4j.LoggerFactory;
> >  /**
> >   * Kerberos-enabled SolrHttpClientBuilder
> >   */
> > -public class Krb5HttpClientBuilder  {
> > +public class Krb5HttpClientBuilder implements HttpClientBuilderFactory {
> >
> >    public static final String LOGIN_CONFIG_PROP =
> > "java.security.auth.login.config";
> >    private static final Logger logger =
> > LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
> > @@ -73,7 +74,12 @@ public class Krb5HttpClientBuilder  {
> >    public void close() {
> >      HttpClientUtil.removeRequestInterceptor(bufferedEntityInterceptor);
> >    }
> > -
> > +
> > +  @Override
> > +  public SolrHttpClientBuilder
> > getHttpClientBuilder(Optional<SolrHttpClientBuilder> builder) {
> > +    return builder.isPresent() ? getBuilder(builder.get()) :
> getBuilder();
> > +  }
> > +
> >    public SolrHttpClientBuilder getBuilder(SolrHttpClientBuilder
> builder) {
> >      if (System.getProperty(LOGIN_CONFIG_PROP) != null) {
> >        String configValue = System.getProperty(LOGIN_CONFIG_PROP);
> >
> > http://git-wip-us.apache.org/repos/asf/lucene-
> > solr/blob/a1a8b286/solr/test-
> > framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
> > ----------------------------------------------------------------------
> > diff --git a/solr/test-
> > framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
> > b/solr/test-
> > framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
> > index 5ebdfb7..81e1f22 100644
> > --- a/solr/test-
> > framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
> > +++ b/solr/test-
> > framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
> > @@ -28,6 +28,7 @@ import java.util.Collection;
> >  import java.util.Collections;
> >  import java.util.List;
> >  import java.util.Objects;
> > +import java.util.Optional;
> >  import java.util.Random;
> >  import java.util.SortedMap;
> >  import java.util.concurrent.Callable;
> > @@ -68,6 +69,7 @@ public class MiniSolrCloudCluster {
> >        "  <str name=\"shareSchema\">${shareSchema:false}</str>\n" +
> >        "  <str
> > name=\"configSetBaseDir\">${configSetBaseDir:configsets}</str>\n" +
> >        "  <str name=\"coreRootDirectory\">${coreRootDirectory:.}</str>\n"
> +
> > +      "  <str
> > name=\"collectionsHandler\">${collectionsHandler:solr.
> CollectionsHandler}<
> > /str>\n" +
> >        "\n" +
> >        "  <shardHandlerFactory name=\"shardHandlerFactory\"
> > class=\"HttpShardHandlerFactory\">\n" +
> >        "    <str name=\"urlScheme\">${urlScheme:}</str>\n" +
> > @@ -180,8 +182,30 @@ public class MiniSolrCloudCluster {
> >     *
> >     * @throws Exception if there was an error starting the cluster
> >     */
> > -  public MiniSolrCloudCluster(int numServers, Path baseDir, String
> solrXml,
> > JettyConfig jettyConfig, ZkTestServer zkTestServer) throws Exception {
> > +  public MiniSolrCloudCluster(int numServers, Path baseDir, String
> solrXml,
> > JettyConfig jettyConfig,
> > +      ZkTestServer zkTestServer) throws Exception {
> > +    this(numServers, baseDir, solrXml, jettyConfig, zkTestServer,
> > Optional.empty());
> > +  }
> >
> > +  /**
> > +   * Create a MiniSolrCloudCluster.
> > +   * Note - this constructor visibility is changed to package protected
> so as to
> > +   * discourage its usage. Ideally *new* functionality should use
> {@linkplain
> > SolrCloudTestCase}
> > +   * to configure any additional parameters.
> > +   *
> > +   * @param numServers number of Solr servers to start
> > +   * @param baseDir base directory that the mini cluster should be run
> from
> > +   * @param solrXml solr.xml file to be uploaded to ZooKeeper
> > +   * @param jettyConfig Jetty configuration
> > +   * @param zkTestServer ZkTestServer to use.  If null, one will be
> created
> > +   * @param securityJson A string representation of security.json file
> > (optional).
> > +   *
> > +   * @throws Exception if there was an error starting the cluster
> > +   */
> > +   MiniSolrCloudCluster(int numServers, Path baseDir, String solrXml,
> > JettyConfig jettyConfig,
> > +      ZkTestServer zkTestServer, Optional<String> securityJson) throws
> > Exception {
> > +
> > +    Objects.requireNonNull(securityJson);
> >      this.baseDir = Objects.requireNonNull(baseDir);
> >      this.jettyConfig = Objects.requireNonNull(jettyConfig);
> >
> > @@ -202,6 +226,9 @@ public class MiniSolrCloudCluster {
> >        if (jettyConfig.sslConfig != null && jettyConfig.sslConfig.isSSLMode())
> {
> >          zkClient.makePath("/solr" + ZkStateReader.CLUSTER_PROPS,
> > "{'urlScheme':'https'}".getBytes(StandardCharsets.UTF_8), true);
> >        }
> > +      if (securityJson.isPresent()) { // configure Solr security
> > +        zkClient.makePath("/solr/security.json",
> > securityJson.get().getBytes(Charset.defaultCharset()), true);
> > +      }
> >      }
> >
> >      // tell solr to look in zookeeper for solr.xml
> >
> > http://git-wip-us.apache.org/repos/asf/lucene-
> > solr/blob/a1a8b286/solr/test-
> > framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
> > ----------------------------------------------------------------------
> > diff --git a/solr/test-
> > framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
> > b/solr/test-
> > framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
> > index 9cc4a22..34dc8ac 100644
> > --- a/solr/test-
> > framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
> > +++ b/solr/test-
> > framework/src/java/org/apache/solr/cloud/SolrCloudTestCase.java
> > @@ -26,6 +26,7 @@ import java.util.Collections;
> >  import java.util.HashMap;
> >  import java.util.List;
> >  import java.util.Map;
> > +import java.util.Optional;
> >  import java.util.concurrent.TimeUnit;
> >  import java.util.concurrent.atomic.AtomicReference;
> >  import java.util.function.Predicate;
> > @@ -90,6 +91,7 @@ public class SolrCloudTestCase extends SolrTestCaseJ4 {
> >      private final Path baseDir;
> >      private String solrxml =
> > MiniSolrCloudCluster.DEFAULT_CLOUD_SOLR_XML;
> >      private JettyConfig jettyConfig = buildJettyConfig("/solr");
> > +    private Optional<String> securityJson = Optional.empty();
> >
> >      private List<Config> configs = new ArrayList<>();
> >      private Map<String, String> clusterProperties = new HashMap<>();
> > @@ -133,6 +135,32 @@ public class SolrCloudTestCase extends
> > SolrTestCaseJ4 {
> >      }
> >
> >      /**
> > +     * Configure the specified security.json for the {@linkplain
> > MiniSolrCloudCluster}
> > +     *
> > +     * @param securityJson The path specifying the security.json file
> > +     * @return the instance of {@linkplain Builder}
> > +     */
> > +    public Builder withSecurityJson(Path securityJson) {
> > +      try {
> > +        this.securityJson = Optional.of(new
> > String(Files.readAllBytes(securityJson), Charset.defaultCharset()));
> > +      } catch (IOException e) {
> > +        throw new RuntimeException(e);
> > +      }
> > +      return this;
> > +    }
> > +
> > +    /**
> > +     * Configure the specified security.json for the {@linkplain
> > MiniSolrCloudCluster}
> > +     *
> > +     * @param securityJson The string specifying the security.json
> > configuration
> > +     * @return the instance of {@linkplain Builder}
> > +     */
> > +    public Builder withSecurityJson(String securityJson) {
> > +      this.securityJson = Optional.of(securityJson);
> > +      return this;
> > +    }
> > +
> > +    /**
> >       * Upload a collection config before tests start
> >       * @param configName the config name
> >       * @param configPath the path to the config files
> > @@ -157,7 +185,7 @@ public class SolrCloudTestCase extends
> > SolrTestCaseJ4 {
> >       * @throws Exception if an error occurs on startup
> >       */
> >      public void configure() throws Exception {
> > -      cluster = new MiniSolrCloudCluster(nodeCount, baseDir, solrxml,
> > jettyConfig);
> > +      cluster = new MiniSolrCloudCluster(nodeCount, baseDir, solrxml,
> > jettyConfig, null, securityJson);
> >        CloudSolrClient client = cluster.getSolrClient();
> >        for (Config config : configs) {
> >
> > ((ZkClientClusterStateProvider)client.getClusterStateProvider()).
> uploadConfig
> > (config.path, config.name);
>
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
> For additional commands, e-mail: dev-help@lucene.apache.org
>
>