You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by an...@apache.org on 2015/05/19 23:10:17 UTC

svn commit: r1680391 - in /lucene/dev/trunk/solr: ./ bin/ core/src/java/org/apache/solr/core/ core/src/java/org/apache/solr/handler/component/ core/src/java/org/apache/solr/security/ core/src/java/org/apache/solr/servlet/ core/src/java/org/apache/solr/...

Author: anshum
Date: Tue May 19 21:10:16 2015
New Revision: 1680391

URL: http://svn.apache.org/r1680391
Log:
SOLR-7274: Pluggable authentication module in Solr. This defines an interface and a mechanism to create, load, and use an Authentication plugin.

Added:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java   (with props)
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/bin/solr
    lucene/dev/trunk/solr/bin/solr.in.sh
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SolrCLI.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudCluster.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1680391&r1=1680390&r2=1680391&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Tue May 19 21:10:16 2015
@@ -200,6 +200,9 @@ New Features
   without a known content type, as "application/octet-stream", provided it still is in the
   allowed filetypes setting. (ehatcher)
 
+* SOLR-7274: Pluggable authentication module in Solr. This defines an interface and a mechanism to create,
+  load, and use an Authentication plugin. (Noble Paul, Ishan Chattopadhyaya, Gregory Chanan, Anshum Gupta)
+
 Bug Fixes
 ----------------------
 

Modified: lucene/dev/trunk/solr/bin/solr
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/bin/solr?rev=1680391&r1=1680390&r2=1680391&view=diff
==============================================================================
--- lucene/dev/trunk/solr/bin/solr (original)
+++ lucene/dev/trunk/solr/bin/solr Tue May 19 21:10:16 2015
@@ -164,6 +164,12 @@ else
   SOLR_JETTY_CONFIG+=("--module=http")
 fi
 
+# Authentication options
+if [ "$SOLR_AUTHENTICATION_CLIENT_CONFIGURER" != "" ]; then
+  AUTHC_CLIENT_CONFIGURER_ARG="-Dsolr.authentication.httpclient.configurer=$SOLR_AUTHENTICATION_CLIENT_CONFIGURER"
+fi
+AUTHC_OPTS="$AUTHC_CLIENT_CONFIGURER_ARG $SOLR_AUTHENTICATION_OPTS"
+
 # Set the SOLR_TOOL_HOST variable for use when connecting to a running Solr instance
 if [ "$SOLR_HOST" != "" ]; then
   SOLR_TOOL_HOST="$SOLR_HOST"
@@ -414,7 +420,7 @@ function run_tool() {
       "${UNPACK_WAR_CMD[@]}" "$DEFAULT_SERVER_DIR/webapps/solr.war")
   fi
 
-  "$JAVA" $SOLR_SSL_OPTS -Dsolr.install.dir="$SOLR_TIP" \
+  "$JAVA" $SOLR_SSL_OPTS $AUTHC_OPTS -Dsolr.install.dir="$SOLR_TIP" \
     -Dlog4j.configuration="file:$DEFAULT_SERVER_DIR/scripts/cloud-scripts/log4j.properties" \
     -classpath "$DEFAULT_SERVER_DIR/solr-webapp/webapp/WEB-INF/lib/*:$DEFAULT_SERVER_DIR/lib/ext/*" \
     org.apache.solr.util.SolrCLI "$@"
@@ -474,7 +480,7 @@ function stop_solr() {
 
   if [ "$SOLR_PID" != "" ]; then
     echo -e "Sending stop command to Solr running on port $SOLR_PORT ... waiting 5 seconds to allow Jetty process $SOLR_PID to stop gracefully."
-    "$JAVA" $SOLR_SSL_OPTS -jar "$DIR/start.jar" "STOP.PORT=$STOP_PORT" "STOP.KEY=$STOP_KEY" --stop || true
+    "$JAVA" $SOLR_SSL_OPTS $AUTHC_OPTS -jar "$DIR/start.jar" "STOP.PORT=$STOP_PORT" "STOP.KEY=$STOP_KEY" --stop || true
     (sleep 5) &
     spinner $!
     rm -f "$SOLR_PID_DIR/solr-$SOLR_PORT.pid"
@@ -1328,6 +1334,11 @@ function launch_solr() {
     SOLR_OPTS+=($SOLR_SSL_OPTS "$SSL_PORT_PROP")
   fi
 
+  # If authentication system props are set, add them to SOLR_OPTS
+  if [ -n "$AUTHC_OPTS" ]; then
+    SOLR_OPTS+=($AUTHC_OPTS)
+  fi
+
   if $verbose ; then
     echo -e "\nStarting Solr using the following settings:"
     echo -e "    JAVA            = $JAVA"

Modified: lucene/dev/trunk/solr/bin/solr.in.sh
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/bin/solr.in.sh?rev=1680391&r1=1680390&r2=1680391&view=diff
==============================================================================
--- lucene/dev/trunk/solr/bin/solr.in.sh (original)
+++ lucene/dev/trunk/solr/bin/solr.in.sh Tue May 19 21:10:16 2015
@@ -109,4 +109,9 @@ ENABLE_REMOTE_JMX_OPTS="false"
 #SOLR_SSL_CLIENT_KEY_STORE=
 #SOLR_SSL_CLIENT_KEY_STORE_PASSWORD=
 #SOLR_SSL_CLIENT_TRUST_STORE=
-#SOLR_SSL_CLIENT_TRUST_STORE_PASSWORD=
\ No newline at end of file
+#SOLR_SSL_CLIENT_TRUST_STORE_PASSWORD=
+
+# Settings for authentication
+#SOLR_AUTHENTICATION_CLIENT_CONFIGURER=
+#SOLR_AUTHENTICATION_OPTS=
+

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java?rev=1680391&r1=1680390&r2=1680391&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/CoreContainer.java Tue May 19 21:10:16 2015
@@ -31,6 +31,9 @@ import java.util.concurrent.ExecutorServ
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
+
+import org.apache.solr.client.solrj.impl.HttpClientConfigurer;
+import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -40,10 +43,12 @@ import org.apache.solr.handler.RequestHa
 import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.handler.admin.CoreAdminHandler;
 import org.apache.solr.handler.admin.InfoHandler;
+import org.apache.solr.handler.component.HttpShardHandlerFactory;
 import org.apache.solr.handler.component.ShardHandlerFactory;
 import org.apache.solr.logging.LogWatcher;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.security.AuthorizationPlugin;
+import org.apache.solr.security.AuthenticationPlugin;
 import org.apache.solr.update.UpdateShardHandler;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.apache.solr.util.FileUtils;
@@ -66,6 +71,8 @@ public class CoreContainer {
 
   protected AuthorizationPlugin authorizationPlugin;
 
+  protected AuthenticationPlugin authenticationPlugin;
+
   public static class CoreLoadFailure {
 
     public final CoreDescriptor cd;
@@ -110,6 +117,8 @@ public class CoreContainer {
   public static final String COLLECTIONS_HANDLER_PATH = "/admin/collections";
   public static final String INFO_HANDLER_PATH = "/admin/info";
 
+  final public static String AUTHENTICATION_PLUGIN_PROP = "authenticationPlugin";
+
   private PluginBag<SolrRequestHandler> containerHandlers = new PluginBag<>(SolrRequestHandler.class, null);
 
   public ExecutorService getCoreZkRegisterExecutorService() {
@@ -203,6 +212,61 @@ public class CoreContainer {
     }
   }
 
+  private void initializeAuthenticationPlugin() {
+    String pluginClassName = null;
+    Map<String, Object> authenticationConfig = null;
+
+    if (isZooKeeperAware()) {
+      Map securityProps = getZkController().getZkStateReader().getSecurityProps();
+      if (securityProps != null) {
+        authenticationConfig = (Map<String, Object>) securityProps.get("authentication");
+        if (authenticationConfig!=null) {
+          if (authenticationConfig.containsKey("class")) {
+            pluginClassName = String.valueOf(authenticationConfig.get("class"));
+          } else {
+            throw new SolrException(ErrorCode.SERVER_ERROR, "No 'class' specified for authentication in ZK.");
+          }
+        }
+      }
+    }
+
+    if (pluginClassName != null) {
+      log.info("Authentication plugin class obtained from ZK: "+pluginClassName);
+    } else if (System.getProperty(AUTHENTICATION_PLUGIN_PROP) != null) {
+      pluginClassName = System.getProperty(AUTHENTICATION_PLUGIN_PROP);
+      log.info("Authentication plugin class obtained from system property '" +
+          AUTHENTICATION_PLUGIN_PROP + "': " + pluginClassName);
+    } else {
+      log.info("No authentication plugin used.");
+    }
+
+    // Initialize the filter
+    if (pluginClassName != null) {
+      try {
+        Class cl = Class.forName(pluginClassName);
+        authenticationPlugin = (AuthenticationPlugin) cl.newInstance();
+      } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
+        throw new SolrException(ErrorCode.SERVER_ERROR, e);
+      }
+    }
+    if (authenticationPlugin != null) {
+      authenticationPlugin.init(authenticationConfig);
+
+      // Setup HttpClient to use the plugin's configurer for internode communication
+      HttpClientConfigurer configurer = authenticationPlugin.getDefaultConfigurer();
+      HttpClientUtil.setConfigurer(configurer);
+
+      // The default http client of the core container's shardHandlerFactory has already been created and
+      // configured using the default httpclient configurer. We need to reconfigure it using the plugin's
+      // http client configurer to set it up for internode communication.
+      log.info("Reconfiguring the shard handler factory and update shard handler.");
+      if (getShardHandlerFactory() instanceof HttpShardHandlerFactory) {
+        ((HttpShardHandlerFactory)getShardHandlerFactory()).reconfigureHttpClient(configurer);
+      }
+      getUpdateShardHandler().reconfigureHttpClient(configurer);
+    }
+  }
+
   /**
    * This method allows subclasses to construct a CoreContainer
    * without any default init behavior.
@@ -273,6 +337,9 @@ public class CoreContainer {
     log.info("Node Name: " + hostName);
 
     zkSys.initZooKeeper(this, solrHome, cfg.getCloudConfig());
+
+    initializeAuthenticationPlugin();
+
     if (isZooKeeperAware()) {
       intializeAuthorizationPlugin();
     }
@@ -436,6 +503,16 @@ public class CoreContainer {
       log.warn("Exception while closing authorization plugin.", e);
     }
     
+    // It should be safe to close the authentication plugin at this point.
+    try {
+      if(authenticationPlugin != null) {
+        authenticationPlugin.close();
+        authenticationPlugin = null;
+      }
+    } catch (Exception e) {
+      log.warn("Exception while closing authentication plugin.", e);
+    }
+
     org.apache.lucene.util.IOUtils.closeWhileHandlingException(loader); // best effort
   }
 
@@ -927,7 +1004,11 @@ public class CoreContainer {
   public AuthorizationPlugin getAuthorizationPlugin() {
     return authorizationPlugin;
   }
-  
+
+  public AuthenticationPlugin getAuthenticationPlugin() {
+    return authenticationPlugin;
+  }
+
 }
 
 class CloserThread extends Thread {

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java?rev=1680391&r1=1680390&r2=1680391&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java Tue May 19 21:10:16 2015
@@ -21,6 +21,7 @@ import org.apache.http.client.HttpClient
 import org.apache.http.impl.client.DefaultHttpClient;
 import org.apache.http.impl.client.DefaultHttpRequestRetryHandler;
 import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.HttpClientConfigurer;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.client.solrj.impl.LBHttpSolrClient;
 import org.apache.solr.client.solrj.request.QueryRequest;
@@ -160,14 +161,8 @@ public class HttpShardHandlerFactory ext
         new DefaultSolrThreadFactory("httpShardExecutor")
     );
 
-    ModifiableSolrParams clientParams = new ModifiableSolrParams();
-    clientParams.set(HttpClientUtil.PROP_MAX_CONNECTIONS_PER_HOST, maxConnectionsPerHost);
-    clientParams.set(HttpClientUtil.PROP_MAX_CONNECTIONS, maxConnections);
-    clientParams.set(HttpClientUtil.PROP_SO_TIMEOUT, soTimeout);
-    clientParams.set(HttpClientUtil.PROP_CONNECTION_TIMEOUT, connectionTimeout);
-    if (!useRetries) {
-      clientParams.set(HttpClientUtil.PROP_USE_RETRY, false);
-    }
+    ModifiableSolrParams clientParams = getClientParams();
+
     this.defaultClient = HttpClientUtil.createClient(clientParams);
     
     // must come after createClient
@@ -179,6 +174,30 @@ public class HttpShardHandlerFactory ext
     
     this.loadbalancer = createLoadbalancer(defaultClient);
   }
+  
+  protected ModifiableSolrParams getClientParams() {
+    ModifiableSolrParams clientParams = new ModifiableSolrParams();
+    clientParams.set(HttpClientUtil.PROP_MAX_CONNECTIONS_PER_HOST, maxConnectionsPerHost);
+    clientParams.set(HttpClientUtil.PROP_MAX_CONNECTIONS, maxConnections);
+    clientParams.set(HttpClientUtil.PROP_SO_TIMEOUT, soTimeout);
+    clientParams.set(HttpClientUtil.PROP_CONNECTION_TIMEOUT, connectionTimeout);
+    if (!useRetries) {
+      clientParams.set(HttpClientUtil.PROP_USE_RETRY, false);
+    }
+    return clientParams;
+  }
+
+  /**
+   * For an already created internal httpclient, this can be used to configure it 
+   * again. Useful for authentication plugins.
+   * @param configurer an HttpClientConfigurer instance
+   */
+  public void reconfigureHttpClient(HttpClientConfigurer configurer) {
+    log.info("Reconfiguring the default client with: " + configurer);
+    synchronized (this.defaultClient) {
+      configurer.configure((DefaultHttpClient)this.defaultClient, getClientParams());
+    }
+  }
 
   protected ThreadPoolExecutor getThreadPoolExecutor(){
     return this.commExecutor;

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java?rev=1680391&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/security/AuthenticationPlugin.java Tue May 19 21:10:16 2015
@@ -0,0 +1,63 @@
+package org.apache.solr.security;
+
+import javax.servlet.FilterChain;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import java.io.Closeable;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.impl.HttpClientConfigurer;
+
+/*
+ * 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.
+ */
+
+/**
+ * 
+ * @lucene.experimental
+ */
+public abstract class AuthenticationPlugin implements Closeable {
+
+  /**
+   * This is called upon loading up of a plugin, used for setting it up.
+   * @param pluginConfig Config parameters, possibly from a ZK source
+   */
+  public abstract void init(Map<String, Object> pluginConfig);
+ 
+  /**
+   * This method must authenticate the request. Upon a successful authentication, this 
+   * must call the next filter in the filter chain and set the user principal of the request,
+   * or else, upon an error or an authentication failure, throw an exception.
+   * 
+   * @param request the http request
+   * @param response the http response
+   * @param filterChain the servlet filter chain
+   * @throws Exception any exception thrown during the authentication, e.g. 
+   * PriviledgedAccessException
+   */
+  public abstract void doAuthenticate(ServletRequest request, ServletResponse response,
+      FilterChain filterChain) throws Exception;
+
+  /**
+   * 
+   * @return Returns an instance of a HttpClientConfigurer to be used for configuring the
+   * httpclients for use with SolrJ clients.
+   * 
+   * @lucene.experimental
+   */
+  public abstract HttpClientConfigurer getDefaultConfigurer();
+
+}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java?rev=1680391&r1=1680390&r2=1680391&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java Tue May 19 21:10:16 2015
@@ -28,6 +28,8 @@ import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Properties;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -44,6 +46,7 @@ import org.apache.solr.core.NodeConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.core.SolrXmlConfig;
+import org.apache.solr.security.AuthenticationPlugin;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -185,7 +188,18 @@ public class SolrDispatchFilter extends
   
   public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain, boolean retry) throws IOException, ServletException {
     if (!(request instanceof HttpServletRequest)) return;
-    
+
+    AtomicReference<ServletRequest> wrappedRequest = new AtomicReference();
+    if (!authenticateRequest(request, response, wrappedRequest)) { // the response and status code have already been sent
+      return;
+    }
+    if (wrappedRequest.get() != null) {
+      request = wrappedRequest.get();
+    }
+    if (cores.getAuthenticationPlugin() != null) {
+      log.debug("User principal: "+((HttpServletRequest)request).getUserPrincipal());
+    }
+
     // No need to even create the HttpSolrCall object if this path is excluded.
     if(excludePatterns != null) {
       String servletPath = ((HttpServletRequest) request).getServletPath().toString();
@@ -216,4 +230,32 @@ public class SolrDispatchFilter extends
       call.destroy();
     }
   }
+
+  private boolean authenticateRequest(ServletRequest request, ServletResponse response, final AtomicReference<ServletRequest> wrappedRequest) throws IOException {
+    final AtomicBoolean isAuthenticated = new AtomicBoolean(false);
+    AuthenticationPlugin authenticationPlugin = cores.getAuthenticationPlugin();
+    if (authenticationPlugin == null) {
+      return true;
+    } else {
+      try {
+        log.debug("Request to authenticate: "+request+", domain: "+request.getLocalName()+", port: "+request.getLocalPort());
+        // upon successful authentication, this should call the chain's next filter.
+        authenticationPlugin.doAuthenticate(request, response, new FilterChain() {
+          public void doFilter(ServletRequest req, ServletResponse rsp) throws IOException, ServletException {
+            isAuthenticated.set(true);
+            wrappedRequest.set(req);
+          }
+        });
+      } catch (Exception e) {
+        e.printStackTrace();
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Error during request authentication, "+e);
+      }
+    }
+    // failed authentication?
+    if (!isAuthenticated.get()) {
+      response.flushBuffer();
+      return false;
+    }
+    return true;
+  }
 }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java?rev=1680391&r1=1680390&r2=1680391&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java Tue May 19 21:10:16 2015
@@ -20,8 +20,10 @@ package org.apache.solr.update;
 import org.apache.http.client.HttpClient;
 import org.apache.http.conn.ClientConnectionManager;
 import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.DefaultHttpClient;
 import org.apache.http.impl.conn.PoolingClientConnectionManager;
 import org.apache.http.impl.conn.SchemeRegistryFactory;
+import org.apache.solr.client.solrj.impl.HttpClientConfigurer;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -46,32 +48,39 @@ public class UpdateShardHandler {
   
   private final CloseableHttpClient client;
 
+  private final UpdateShardHandlerConfig cfg;
+
   @Deprecated
   public UpdateShardHandler(NodeConfig cfg) {
     this(cfg.getUpdateShardHandlerConfig());
   }
 
   public UpdateShardHandler(UpdateShardHandlerConfig cfg) {
-    
+    this.cfg = cfg;
     clientConnectionManager = new PoolingClientConnectionManager(SchemeRegistryFactory.createSystemDefault());
     if (cfg != null ) {
       clientConnectionManager.setMaxTotal(cfg.getMaxUpdateConnections());
       clientConnectionManager.setDefaultMaxPerRoute(cfg.getMaxUpdateConnectionsPerHost());
     }
-    
-    ModifiableSolrParams params = new ModifiableSolrParams();
+
+    ModifiableSolrParams clientParams = getClientParams();
+    log.info("Creating UpdateShardHandler HTTP client with params: {}", clientParams);
+    client = HttpClientUtil.createClient(clientParams, clientConnectionManager);
+  }
+
+  protected ModifiableSolrParams getClientParams() {
+    ModifiableSolrParams clientParams = new ModifiableSolrParams();
     if (cfg != null) {
-      params.set(HttpClientUtil.PROP_SO_TIMEOUT,
+      clientParams.set(HttpClientUtil.PROP_SO_TIMEOUT,
           cfg.getDistributedSocketTimeout());
-      params.set(HttpClientUtil.PROP_CONNECTION_TIMEOUT,
+      clientParams.set(HttpClientUtil.PROP_CONNECTION_TIMEOUT,
           cfg.getDistributedConnectionTimeout());
     }
     // in the update case, we want to do retries, and to use
     // the default Solr retry handler that createClient will 
     // give us
-    params.set(HttpClientUtil.PROP_USE_RETRY, true);
-    log.info("Creating UpdateShardHandler HTTP client with params: {}", params);
-    client = HttpClientUtil.createClient(params, clientConnectionManager);
+    clientParams.set(HttpClientUtil.PROP_USE_RETRY, true);
+    return clientParams;
   }
   
   
@@ -79,6 +88,13 @@ public class UpdateShardHandler {
     return client;
   }
 
+  public void reconfigureHttpClient(HttpClientConfigurer configurer) {
+    log.info("Reconfiguring the default client with: " + configurer);
+    synchronized (client) {
+      configurer.configure((DefaultHttpClient)client, getClientParams());
+    }
+  }
+
   public ClientConnectionManager getConnectionManager() {
     return clientConnectionManager;
   }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SolrCLI.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SolrCLI.java?rev=1680391&r1=1680390&r2=1680391&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SolrCLI.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/SolrCLI.java Tue May 19 21:10:16 2015
@@ -63,6 +63,7 @@ import org.apache.log4j.Logger;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.HttpClientConfigurer;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.response.QueryResponse;
@@ -174,7 +175,19 @@ public class SolrCLI {
       displayToolOptions(System.err);
       System.exit(1);
     }
-    
+
+    String configurerClassName = System.getProperty("solr.authentication.httpclient.configurer");
+    if (configurerClassName!=null) {
+      try {
+        Class c = Class.forName(configurerClassName);
+        HttpClientConfigurer configurer = (HttpClientConfigurer)c.newInstance();
+        HttpClientUtil.setConfigurer(configurer);
+        log.info("Set HttpClientConfigurer from: "+configurerClassName);
+      } catch (Exception ex) {
+        throw new RuntimeException("Error during loading of configurer '"+configurerClassName+"'.", ex);
+      }
+    }
+
     // Determine the tool
     String toolType = args[0].trim().toLowerCase(Locale.ROOT);
     Tool tool = newTool(toolType);

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java?rev=1680391&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestAuthenticationFramework.java Tue May 19 21:10:16 2015
@@ -0,0 +1,169 @@
+package org.apache.solr.cloud;
+
+/*
+ * 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.
+ */
+
+import javax.servlet.FilterChain;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
+import org.apache.http.HttpException;
+import org.apache.http.HttpRequest;
+import org.apache.http.HttpRequestInterceptor;
+import org.apache.http.impl.client.DefaultHttpClient;
+import org.apache.http.protocol.HttpContext;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.solr.client.solrj.impl.HttpClientConfigurer;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.security.AuthenticationPlugin;
+import org.apache.solr.util.RevertDefaultThreadHandlerRule;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.RuleChain;
+import org.junit.rules.TestRule;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test of the MiniSolrCloudCluster functionality with authentication enabled.
+ */
+@LuceneTestCase.Slow
+@SuppressSysoutChecks(bugUrl = "Solr logs to JUL")
+public class TestAuthenticationFramework extends TestMiniSolrCloudCluster {
+  
+  public TestAuthenticationFramework () {
+    NUM_SERVERS = 5;
+    NUM_SHARDS = 2;
+    REPLICATION_FACTOR = 2;
+  }
+  
+  static String requestUsername = MockAuthenticationPlugin.expectedUsername;
+  static String requestPassword = MockAuthenticationPlugin.expectedPassword;
+  
+  protected final static List<String> brokenLocales =
+      Arrays.asList(
+        "th_TH_TH_#u-nu-thai",
+        "ja_JP_JP_#u-ca-japanese",
+        "hi_IN");
+
+  @Rule
+  public TestRule solrTestRules = RuleChain
+      .outerRule(new SystemPropertiesRestoreRule());
+
+  @ClassRule
+  public static TestRule solrClassRules = RuleChain.outerRule(
+      new SystemPropertiesRestoreRule()).around(
+      new RevertDefaultThreadHandlerRule());
+
+  @Override
+  public void setUp() throws Exception {
+    if (brokenLocales.contains(Locale.getDefault().toString())) {
+      Locale.setDefault(Locale.US);
+    }
+    setupAuthenticationPlugin();
+    super.setUp();
+  }
+  
+  private void setupAuthenticationPlugin() throws Exception {
+    System.setProperty("authenticationPlugin", "org.apache.solr.cloud.TestAuthenticationFramework$MockAuthenticationPlugin");
+  }
+  
+  @Test
+  @Override
+  public void testBasics() throws Exception {
+    requestUsername = MockAuthenticationPlugin.expectedUsername;
+    requestPassword = MockAuthenticationPlugin.expectedPassword;
+    
+    // Should pass
+    testCollectionCreateSearchDelete();
+    
+    requestUsername = MockAuthenticationPlugin.expectedUsername;
+    requestPassword = "junkpassword";
+    
+    // Should fail with 401
+    try {
+      testCollectionCreateSearchDelete();
+      fail("Should've returned a 401 error");
+    } catch (Exception ex) {
+      if (!ex.getMessage().contains("Error 401")) {
+        fail("Should've returned a 401 error");
+      }
+    }
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    System.clearProperty("authenticationPlugin");
+    super.tearDown();
+  }
+  
+  public static class MockAuthenticationPlugin extends AuthenticationPlugin {
+    private static Logger log = LoggerFactory.getLogger(MockAuthenticationPlugin.class);
+
+    public static String expectedUsername = "solr";
+    public static String expectedPassword = "s0lrRocks";
+
+    @Override
+    public void init(Map<String,Object> pluginConfig) {}
+
+    @Override
+    public void doAuthenticate(ServletRequest request, ServletResponse response, FilterChain filterChain)
+        throws Exception {
+      HttpServletRequest httpRequest = (HttpServletRequest)request;
+      String username = httpRequest.getHeader("username");
+      String password = httpRequest.getHeader("password");
+      
+      log.info("Username: "+username+", password: "+password);
+      if(MockAuthenticationPlugin.expectedUsername.equals(username) && MockAuthenticationPlugin.expectedPassword.equals(password))      
+        filterChain.doFilter(request, response);
+      else {
+        ((HttpServletResponse)response).sendError(401, "Unauthorized request");
+      }
+    }
+
+    @Override
+    public HttpClientConfigurer getDefaultConfigurer() {
+      return new MockClientConfigurer();
+    }
+
+    @Override
+    public void close() {}
+    
+    private static class MockClientConfigurer extends HttpClientConfigurer {
+      @Override
+      public void configure(DefaultHttpClient httpClient, SolrParams config) {
+        super.configure(httpClient, config);
+        httpClient.addRequestInterceptor(new HttpRequestInterceptor() {
+          @Override
+          public void process(HttpRequest req, HttpContext rsp) throws HttpException, IOException {
+            req.addHeader("username", requestUsername);
+            req.addHeader("password", requestPassword);
+          }
+        });
+      }
+    }
+  }
+}

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudCluster.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudCluster.java?rev=1680391&r1=1680390&r2=1680391&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudCluster.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudCluster.java Tue May 19 21:10:16 2015
@@ -59,10 +59,16 @@ import java.util.concurrent.atomic.Atomi
 public class TestMiniSolrCloudCluster extends LuceneTestCase {
 
   private static Logger log = LoggerFactory.getLogger(MiniSolrCloudCluster.class);
-  private static final int NUM_SERVERS = 5;
-  private static final int NUM_SHARDS = 2;
-  private static final int REPLICATION_FACTOR = 2;
+  protected int NUM_SERVERS = 5;
+  protected int NUM_SHARDS = 2;
+  protected int REPLICATION_FACTOR = 2;
 
+  public TestMiniSolrCloudCluster () {
+    NUM_SERVERS = 5;
+    NUM_SHARDS = 2;
+    REPLICATION_FACTOR = 2;
+  }
+  
   @Rule
   public TestRule solrTestRules = RuleChain
       .outerRule(new SystemPropertiesRestoreRule());
@@ -79,7 +85,7 @@ public class TestMiniSolrCloudCluster ex
     if (random().nextBoolean()) testCollectionCreateSearchDelete();
   }
     
-  private void testCollectionCreateSearchDelete() throws Exception {
+  protected void testCollectionCreateSearchDelete() throws Exception {
 
     File solrXml = new File(SolrTestCaseJ4.TEST_HOME(), "solr-no-core.xml");
     MiniSolrCloudCluster miniCluster = new MiniSolrCloudCluster(NUM_SERVERS, null, createTempDir().toFile(), solrXml, null, null);