You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by tf...@apache.org on 2019/01/15 20:54:40 UTC

[lucene-solr] branch branch_7x updated: SOLR-12770: Make it possible to configure a host whitelist for distributed search

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

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


The following commit(s) were added to refs/heads/branch_7x by this push:
     new 6d63958  SOLR-12770: Make it possible to configure a host whitelist for distributed search
6d63958 is described below

commit 6d63958821232699f0a8423d9b21d4915bfba64e
Author: Tomas Fernandez Lobbe <tf...@apache.org>
AuthorDate: Tue Jan 15 11:44:57 2019 -0800

    SOLR-12770: Make it possible to configure a host whitelist for distributed search
---
 solr/CHANGES.txt                                   |   6 +-
 solr/bin/solr.in.cmd                               |   6 +
 solr/bin/solr.in.sh                                |   5 +
 .../solr/handler/component/HttpShardHandler.java   |  38 ++-
 .../handler/component/HttpShardHandlerFactory.java | 204 ++++++++++++++--
 .../solr/handler/component/TermsComponent.java     |  40 +++-
 solr/core/src/test-files/solr/solr.xml             |   1 +
 .../test/org/apache/solr/TestTolerantSearch.java   |   2 +
 .../component/CustomTermsComponentTest.java        | 257 ++++++++++++++++++++
 .../component/DistributedDebugComponentTest.java   |   2 +
 .../handler/component/ShardsWhitelistTest.java     | 264 +++++++++++++++++++++
 .../component/TestHttpShardHandlerFactory.java     | 204 +++++++++++++++-
 .../org/apache/solr/search/TestSmileRequest.java   |   2 +
 .../solr/search/facet/TestJsonFacetRefinement.java |   2 +
 .../apache/solr/search/facet/TestJsonFacets.java   |  38 ++-
 .../apache/solr/search/json/TestJsonRequest.java   |   2 +
 solr/server/solr/solr.xml                          |   1 +
 solr/solr-ref-guide/src/distributed-requests.adoc  |   8 +-
 .../distributed-search-with-index-sharding.adoc    |   3 +
 solr/solr-ref-guide/src/the-terms-component.adoc   |   2 +
 solr/solrj/src/test-files/solrj/solr/solr.xml      |   1 +
 .../apache/solr/BaseDistributedSearchTestCase.java |  12 +
 .../src/java/org/apache/solr/SolrTestCaseHS.java   |  22 ++
 .../src/java/org/apache/solr/SolrTestCaseJ4.java   |  13 +
 .../apache/solr/cloud/MiniSolrCloudCluster.java    |   3 +
 25 files changed, 1102 insertions(+), 36 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 428256d..95f149b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -32,7 +32,8 @@ Jetty 9.4.14.v20181114
 Upgrade Notes
 ----------------------
 
-(No Changes)
+* SOLR-12770: The 'shards' parameter handling logic changes to use a new config element to determine what hosts can be
+  requested. Please see Apache Solr Reference Guide chapter "Distributed Requests" for details, as well as SOLR-12770.
 
 New Features
 ----------------------
@@ -42,6 +43,9 @@ New Features
 
 * SOLR-7896: Add a login page to Admin UI, with initial support for Basic Auth (janhoy)
 
+* SOLR-12770: Make it possible to configure a host whitelist for distributed search
+  (Christine Poerschke, janhoy, Erick Erickson, Tomás Fernández Löbbe)
+
 Bug Fixes
 ----------------------
 * SOLR-12546: CVSResponseWriter omits useDocValuesAsStored=true field when fl=*
diff --git a/solr/bin/solr.in.cmd b/solr/bin/solr.in.cmd
index 4b86d25..639f7e9 100755
--- a/solr/bin/solr.in.cmd
+++ b/solr/bin/solr.in.cmd
@@ -149,3 +149,9 @@ REM  -DzkCredentialsProvider=org.apache.solr.common.cloud.VMParamsSingleSetCrede
 REM  -DzkDigestUsername=admin-user -DzkDigestPassword=CHANGEME-ADMIN-PASSWORD ^
 REM  -DzkDigestReadonlyUsername=readonly-user -DzkDigestReadonlyPassword=CHANGEME-READONLY-PASSWORD
 REM set SOLR_OPTS=%SOLR_OPTS% %SOLR_ZK_CREDS_AND_ACLS%
+
+REM When running Solr in non-cloud mode and if planning to do distributed search (using the "shards" parameter), the
+REM list of hosts needs to be whitelisted or Solr will forbid the request. The whitelist can be configured in solr.xml,
+REM or if you are using the OOTB solr.xml, can be specified using the system property "solr.shardsWhitelist". Alternatively
+REM host checking can be disabled by using the system property "solr.disable.shardsWhitelist"
+REM set SOLR_OPTS="%SOLR_OPTS% -Dsolr.shardsWhitelist=http://localhost:8983,http://localhost:8984"
diff --git a/solr/bin/solr.in.sh b/solr/bin/solr.in.sh
index af1cd7a..453796f 100644
--- a/solr/bin/solr.in.sh
+++ b/solr/bin/solr.in.sh
@@ -178,3 +178,8 @@
 #SOLR_RECOMMENDED_MAX_PROCESSES=
 #SOLR_ULIMIT_CHECKS=
 
+# When running Solr in non-cloud mode and if planning to do distributed search (using the "shards" parameter), the
+# list of hosts needs to be whitelisted or Solr will forbid the request. The whitelist can be configured in solr.xml,
+# or if you are using the OOTB solr.xml, can be specified using the system property "solr.shardsWhitelist". Alternatively
+# host checking can be disabled by using the system property "solr.disable.shardsWhitelist"
+#SOLR_OPTS="$SOLR_OPTS -Dsolr.shardsWhitelist=http://localhost:8983,http://localhost:8984"
diff --git a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
index a548031..1886d93 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
@@ -19,6 +19,7 @@ package org.apache.solr.handler.component;
 import java.lang.invoke.MethodHandles;
 import java.net.ConnectException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -43,6 +44,7 @@ import org.apache.solr.client.solrj.util.ClientUtils;
 import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
@@ -88,7 +90,6 @@ public class HttpShardHandler extends ShardHandler {
     // This is primarily to keep track of what order we should use to query the replicas of a shard
     // so that we use the same replica for all phases of a distributed request.
     shardToURLs = new HashMap<>();
-
   }
 
 
@@ -349,6 +350,13 @@ public class HttpShardHandler extends ShardHandler {
       rb.shards = new String[rb.slices.length];
     }
 
+    HttpShardHandlerFactory.WhitelistHostChecker hostChecker = httpShardHandlerFactory.getWhitelistHostChecker();
+    if (shards != null && zkController == null && hostChecker.isWhitelistHostCheckingEnabled() && !hostChecker.hasExplicitWhitelist()) {
+      throw new SolrException(ErrorCode.FORBIDDEN, "HttpShardHandlerFactory "+HttpShardHandlerFactory.INIT_SHARDS_WHITELIST
+          +" not configured but required (in lieu of ZkController and ClusterState) when using the '"+ShardParams.SHARDS+"' parameter."
+          +HttpShardHandlerFactory.SET_SOLR_DISABLE_SHARDS_WHITELIST_CLUE);
+    }
+
     //
     // Map slices to shards
     //
@@ -372,21 +380,32 @@ public class HttpShardHandler extends ShardHandler {
         if (shortCircuit) {
           rb.isDistrib = false;
           rb.shortCircuitedURL = ZkCoreNodeProps.getCoreUrl(zkController.getBaseUrl(), coreDescriptor.getName());
+          if (hostChecker.isWhitelistHostCheckingEnabled() && hostChecker.hasExplicitWhitelist()) {
+            /*
+             * We only need to check the host whitelist if there is an explicit whitelist (other than all the live nodes)
+             * when the "shards" indicate cluster state elements only
+             */
+            hostChecker.checkWhitelist(clusterState, shards, Arrays.asList(rb.shortCircuitedURL));
+          }
           return;
         }
         // We shouldn't need to do anything to handle "shard.rows" since it was previously meant to be an optimization?
       }
+      
+      if (clusterState == null && zkController != null) {
+        clusterState =  zkController.getClusterState();
+      }
 
 
       for (int i=0; i<rb.shards.length; i++) {
         if (rb.shards[i] != null) {
           final List<String> shardUrls = StrUtils.splitSmart(rb.shards[i], "|", true);
           replicaListTransformer.transform(shardUrls);
+          hostChecker.checkWhitelist(clusterState, shards, shardUrls);
           // And now recreate the | delimited list of equivalent servers
           rb.shards[i] = createSliceShardsStr(shardUrls);
         } else {
-          if (clusterState == null) {
-            clusterState =  zkController.getClusterState();
+          if (slices == null) {
             slices = clusterState.getCollection(cloudDescriptor.getCollectionName()).getSlicesMap();
           }
           String sliceName = rb.slices[i];
@@ -427,6 +446,14 @@ public class HttpShardHandler extends ShardHandler {
 
           final List<String> shardUrls = transformReplicasToShardUrls(replicaListTransformer, eligibleSliceReplicas);
 
+          if (hostChecker.isWhitelistHostCheckingEnabled() && hostChecker.hasExplicitWhitelist()) {
+            /*
+             * We only need to check the host whitelist if there is an explicit whitelist (other than all the live nodes)
+             * when the "shards" indicate cluster state elements only
+             */
+            hostChecker.checkWhitelist(clusterState, shards, shardUrls);
+          }
+
           // And now recreate the | delimited list of equivalent servers
           final String sliceShardsStr = createSliceShardsStr(shardUrls);
           if (sliceShardsStr.isEmpty()) {
@@ -440,6 +467,11 @@ public class HttpShardHandler extends ShardHandler {
           rb.shards[i] = sliceShardsStr;
         }
       }
+    } else {
+      if (shards != null) {
+        // No cloud, verbatim check of shards
+        hostChecker.checkWhitelist(shards, new ArrayList<>(Arrays.asList(shards.split("[,|]"))));
+      }
     }
     String shards_rows = params.get(ShardParams.SHARDS_ROWS);
     if(shards_rows != null) {
diff --git a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
index 1bb1fdb..0789f13 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
@@ -16,6 +16,28 @@
  */
 package org.apache.solr.handler.component;
 
+import static org.apache.solr.util.stats.InstrumentedHttpRequestExecutor.KNOWN_METRIC_NAME_STRATEGIES;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 import org.apache.commons.lang.StringUtils;
 import org.apache.http.client.HttpClient;
 import org.apache.http.impl.client.CloseableHttpClient;
@@ -26,6 +48,8 @@ import org.apache.solr.client.solrj.impl.LBHttpSolrClient.Builder;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -39,8 +63,8 @@ import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrInfoBean;
 import org.apache.solr.metrics.SolrMetricManager;
 import org.apache.solr.metrics.SolrMetricProducer;
-import org.apache.solr.update.UpdateShardHandlerConfig;
 import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.update.UpdateShardHandlerConfig;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.apache.solr.util.stats.HttpClientMetricNameStrategy;
 import org.apache.solr.util.stats.InstrumentedHttpRequestExecutor;
@@ -49,23 +73,6 @@ import org.apache.solr.util.stats.MetricUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.CompletionService;
-import java.util.concurrent.ExecutorCompletionService;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.solr.util.stats.InstrumentedHttpRequestExecutor.KNOWN_METRIC_NAME_STRATEGIES;
-
 
 public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.apache.solr.util.plugin.PluginInfoInitialized, SolrMetricProducer {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -104,6 +111,7 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
   int   permittedLoadBalancerRequestsMinimumAbsolute = 0;
   float permittedLoadBalancerRequestsMaximumFraction = 1.0f;
   boolean accessPolicy = false;
+  private WhitelistHostChecker whitelistHostChecker = null;
 
   private String scheme = null;
 
@@ -139,6 +147,12 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
   // Configure if the threadpool favours fairness over throughput
   static final String INIT_FAIRNESS_POLICY = "fairnessPolicy";
 
+  public static final String INIT_SHARDS_WHITELIST = "shardsWhitelist";
+
+  static final String INIT_SOLR_DISABLE_SHARDS_WHITELIST = "solr.disable." + INIT_SHARDS_WHITELIST;
+
+  static final String SET_SOLR_DISABLE_SHARDS_WHITELIST_CLUE = " set -D"+INIT_SOLR_DISABLE_SHARDS_WHITELIST+"=true to disable shards whitelist checks";
+
   /**
    * Get {@link ShardHandler} that uses the default http client.
    */
@@ -154,6 +168,24 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
     return new HttpShardHandler(this, httpClient);
   }
 
+  /**
+   * Returns this Factory's {@link WhitelistHostChecker}.
+   * This method can be overridden to change the checker implementation.
+   */
+  public WhitelistHostChecker getWhitelistHostChecker() {
+    return this.whitelistHostChecker;
+  }
+
+  @Deprecated // For temporary use by the TermsComponent only.
+  static boolean doGetDisableShardsWhitelist() {
+    return getDisableShardsWhitelist();
+  }
+
+
+  private static boolean getDisableShardsWhitelist() {
+    return Boolean.getBoolean(INIT_SOLR_DISABLE_SHARDS_WHITELIST);
+  }
+
   @Override
   public void init(PluginInfo info) {
     StringBuilder sb = new StringBuilder();
@@ -189,6 +221,9 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
         permittedLoadBalancerRequestsMaximumFraction,
         sb);
     this.accessPolicy = getParameter(args, INIT_FAIRNESS_POLICY, accessPolicy,sb);
+    this.whitelistHostChecker = new WhitelistHostChecker(args == null? null: (String) args.get(INIT_SHARDS_WHITELIST), !getDisableShardsWhitelist());
+    log.info("Host whitelist initialized: {}", this.whitelistHostChecker);
+    
     log.debug("created with {}",sb);
     
     // magic sysprop to make tests reproducible: set by SolrTestCaseJ4.
@@ -480,5 +515,138 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
         manager.registry(registry),
         SolrMetricManager.mkName("httpShardExecutor", expandedScope, "threadPool"));
   }
+  
+  /**
+   * Class used to validate the hosts in the "shards" parameter when doing a distributed
+   * request
+   */
+  public static class WhitelistHostChecker {
+    
+    /**
+     * List of the whitelisted hosts. Elements in the list will be host:port (no protocol or context)
+     */
+    private final Set<String> whitelistHosts;
+    
+    /**
+     * Indicates whether host checking is enabled 
+     */
+    private final boolean whitelistHostCheckingEnabled;
+    
+    public WhitelistHostChecker(String whitelistStr, boolean enabled) {
+      this.whitelistHosts = implGetShardsWhitelist(whitelistStr);
+      this.whitelistHostCheckingEnabled = enabled;
+    }
+    
+    final static Set<String> implGetShardsWhitelist(final String shardsWhitelist) {
+      if (shardsWhitelist != null && !shardsWhitelist.isEmpty()) {
+        return StrUtils.splitSmart(shardsWhitelist, ',')
+            .stream()
+            .map(String::trim)
+            .map((hostUrl) -> {
+              URL url;
+              try {
+                if (!hostUrl.startsWith("http://") && !hostUrl.startsWith("https://")) {
+                  // It doesn't really matter which protocol we set here because we are not going to use it. We just need a full URL.
+                  url = new URL("http://" + hostUrl);
+                } else {
+                  url = new URL(hostUrl);
+                }
+              } catch (MalformedURLException e) {
+                throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Invalid URL syntax in \"" + INIT_SHARDS_WHITELIST + "\": " + shardsWhitelist, e);
+              }
+              if (url.getHost() == null || url.getPort() < 0) {
+                throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Invalid URL syntax in \"" + INIT_SHARDS_WHITELIST + "\": " + shardsWhitelist);
+              }
+              return url.getHost() + ":" + url.getPort();
+            }).collect(Collectors.toSet());
+      }
+      return null;
+    }
+    
+    
+    /**
+     * @see #checkWhitelist(ClusterState, String, List)
+     */
+    protected void checkWhitelist(String shardsParamValue, List<String> shardUrls) {
+      checkWhitelist(null, shardsParamValue, shardUrls);
+    }
+    
+    /**
+     * Checks that all the hosts for all the shards requested in shards parameter exist in the configured whitelist
+     * or in the ClusterState (in case of cloud mode)
+     * 
+     * @param clusterState The up to date ClusterState, can be null in case of non-cloud mode
+     * @param shardsParamValue The original shards parameter
+     * @param shardUrls The list of cores generated from the shards parameter. 
+     */
+    protected void checkWhitelist(ClusterState clusterState, String shardsParamValue, List<String> shardUrls) {
+      if (!whitelistHostCheckingEnabled) {
+        return;
+      }
+      Set<String> localWhitelistHosts;
+      if (whitelistHosts == null && clusterState != null) {
+        // TODO: We could implement caching, based on the version of the live_nodes znode
+        localWhitelistHosts = generateWhitelistFromLiveNodes(clusterState);
+      } else if (whitelistHosts != null) {
+        localWhitelistHosts = whitelistHosts;
+      } else {
+        localWhitelistHosts = Collections.emptySet();
+      }
+      
+      shardUrls.stream().map(String::trim).forEach((shardUrl) -> {
+        URL url;
+        try {
+          if (!shardUrl.startsWith("http://") && !shardUrl.startsWith("https://")) {
+            // It doesn't really matter which protocol we set here because we are not going to use it. We just need a full URL.
+            url = new URL("http://" + shardUrl);
+          } else {
+            url = new URL(shardUrl);
+          }
+        } catch (MalformedURLException e) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Invalid URL syntax in \"shards\" parameter: " + shardsParamValue, e);
+        }
+        if (url.getHost() == null || url.getPort() < 0) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Invalid URL syntax in \"shards\" parameter: " + shardsParamValue);
+        }
+        if (!localWhitelistHosts.contains(url.getHost() + ":" + url.getPort())) {
+          log.warn("The '"+ShardParams.SHARDS+"' parameter value '"+shardsParamValue+"' contained value(s) not on the shards whitelist ("+localWhitelistHosts+"), shardUrl:" + shardUrl);
+          throw new SolrException(ErrorCode.FORBIDDEN,
+              "The '"+ShardParams.SHARDS+"' parameter value '"+shardsParamValue+"' contained value(s) not on the shards whitelist. shardUrl:" + shardUrl + "." +
+                  HttpShardHandlerFactory.SET_SOLR_DISABLE_SHARDS_WHITELIST_CLUE);
+        }
+      });
+    }
+    
+    Set<String> generateWhitelistFromLiveNodes(ClusterState clusterState) {
+      return clusterState
+          .getLiveNodes()
+          .stream()
+          .map((liveNode) -> liveNode.substring(0, liveNode.indexOf('_')))
+          .collect(Collectors.toSet());
+    }
+    
+    public boolean hasExplicitWhitelist() {
+      return this.whitelistHosts != null;
+    }
+    
+    public boolean isWhitelistHostCheckingEnabled() {
+      return whitelistHostCheckingEnabled;
+    }
+    
+    /**
+     * Only to be used by tests
+     */
+    @VisibleForTesting
+    Set<String> getWhitelistHosts() {
+      return this.whitelistHosts;
+    }
 
+    @Override
+    public String toString() {
+      return "WhitelistHostChecker [whitelistHosts=" + whitelistHosts + ", whitelistHostCheckingEnabled="
+          + whitelistHostCheckingEnabled + "]";
+    }
+    
+  }
+  
 }
diff --git a/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java b/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java
index b7a1f56..cd388df 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java
@@ -19,10 +19,10 @@ package org.apache.solr.handler.component;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Locale;
 import java.util.regex.Pattern;
-
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
@@ -38,6 +38,8 @@ import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.mutable.MutableValue;
 import org.apache.solr.client.solrj.response.TermsResponse;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.ShardParams;
 import org.apache.solr.common.params.SolrParams;
@@ -45,6 +47,7 @@ import org.apache.solr.common.params.TermsParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.handler.component.HttpShardHandlerFactory.WhitelistHostChecker;
 import org.apache.solr.request.SimpleFacets.CountPair;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.SchemaField;
@@ -76,6 +79,20 @@ public class TermsComponent extends SearchComponent {
   public static final int UNLIMITED_MAX_COUNT = -1;
   public static final String COMPONENT_NAME = "terms";
 
+  // This needs to be created here too, because Solr doesn't call init(...) on default components. Bug?
+  private WhitelistHostChecker whitelistHostChecker = new WhitelistHostChecker(
+      null, 
+      !HttpShardHandlerFactory.doGetDisableShardsWhitelist());
+
+  @Override
+  public void init( NamedList args )
+  {
+    super.init(args);
+    whitelistHostChecker = new WhitelistHostChecker(
+        (String) args.get(HttpShardHandlerFactory.INIT_SHARDS_WHITELIST), 
+        !HttpShardHandlerFactory.doGetDisableShardsWhitelist());
+  }
+  
   @Override
   public void prepare(ResponseBuilder rb) throws IOException {
     SolrParams params = rb.req.getParams();
@@ -95,10 +112,30 @@ public class TermsComponent extends SearchComponent {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No shards.qt parameter specified");
       }
       List<String> lst = StrUtils.splitSmart(shards, ",", true);
+      checkShardsWhitelist(rb, lst);
       rb.shards = lst.toArray(new String[lst.size()]);
     }
   }
 
+  protected void checkShardsWhitelist(final ResponseBuilder rb, final List<String> lst) {
+    final List<String> urls = new LinkedList<String>();
+    for (final String ele : lst) {
+      urls.addAll(StrUtils.splitSmart(ele, '|'));
+    }
+    
+    if (whitelistHostChecker.isWhitelistHostCheckingEnabled() && rb.req.getCore().getCoreContainer().getZkController() == null && !whitelistHostChecker.hasExplicitWhitelist()) {
+      throw new SolrException(ErrorCode.FORBIDDEN, "TermsComponent "+HttpShardHandlerFactory.INIT_SHARDS_WHITELIST
+          +" not configured but required when using the '"+ShardParams.SHARDS+"' parameter with the TermsComponent."
+          +HttpShardHandlerFactory.SET_SOLR_DISABLE_SHARDS_WHITELIST_CLUE);
+    } else {
+      ClusterState cs = null;
+      if (rb.req.getCore().getCoreContainer().getZkController() != null) {
+        cs = rb.req.getCore().getCoreContainer().getZkController().getClusterState();
+      }
+      whitelistHostChecker.checkWhitelist(cs, urls.toString(), urls);
+    }
+  }
+
   @Override
   public void process(ResponseBuilder rb) throws IOException {
     SolrParams params = rb.req.getParams();
@@ -687,4 +724,5 @@ public class TermsComponent extends SearchComponent {
   public Category getCategory() {
     return Category.QUERY;
   }
+
 }
diff --git a/solr/core/src/test-files/solr/solr.xml b/solr/core/src/test-files/solr/solr.xml
index f77c9c4..a3dedf0 100644
--- a/solr/core/src/test-files/solr/solr.xml
+++ b/solr/core/src/test-files/solr/solr.xml
@@ -29,6 +29,7 @@
     <str name="urlScheme">${urlScheme:}</str>
     <int name="socketTimeout">${socketTimeout:15000}</int>
     <int name="connTimeout">${connTimeout:15000}</int>
+    <str name="shardsWhitelist">${solr.tests.shardsWhitelist:}</str>
   </shardHandlerFactory>
 
   <transientCoreCacheFactory name="transientCoreCacheFactory" class="TransientSolrCoreCacheFactoryDefault">
diff --git a/solr/core/src/test/org/apache/solr/TestTolerantSearch.java b/solr/core/src/test/org/apache/solr/TestTolerantSearch.java
index 031d942..e4ba4fa 100644
--- a/solr/core/src/test/org/apache/solr/TestTolerantSearch.java
+++ b/solr/core/src/test/org/apache/solr/TestTolerantSearch.java
@@ -56,6 +56,7 @@ public class TestTolerantSearch extends SolrJettyTestBase {
   
   @BeforeClass
   public static void createThings() throws Exception {
+    systemSetPropertySolrDisableShardsWhitelist("true");
     solrHome = createSolrHome();
     createAndStartJetty(solrHome.getAbsolutePath());
     String url = jetty.getBaseUrl().toString();
@@ -105,6 +106,7 @@ public class TestTolerantSearch extends SolrJettyTestBase {
     jetty.stop();
     jetty=null;
     resetExceptionIgnores();
+    systemClearPropertySolrDisableShardsWhitelist();
   }
   
   @SuppressWarnings("unchecked")
diff --git a/solr/core/src/test/org/apache/solr/handler/component/CustomTermsComponentTest.java b/solr/core/src/test/org/apache/solr/handler/component/CustomTermsComponentTest.java
new file mode 100644
index 0000000..b0836f1
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/component/CustomTermsComponentTest.java
@@ -0,0 +1,257 @@
+/*
+ * 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.handler.component;
+
+import static org.hamcrest.CoreMatchers.containsString;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.TermsResponse;
+import org.apache.solr.client.solrj.response.TermsResponse.Term;
+import org.apache.solr.cloud.ConfigRequest;
+import org.apache.solr.cloud.MiniSolrCloudCluster;
+import org.apache.solr.common.util.NamedList;
+import org.junit.Test;
+
+public class CustomTermsComponentTest extends ShardsWhitelistTest {
+
+  public static class CustomTermsComponent extends TermsComponent {
+    
+    public void init( NamedList args )
+    {
+      super.init(args);
+    }
+
+    @Override
+    protected void checkShardsWhitelist(final ResponseBuilder rb, final List<String> lst) {
+      // ignore shards whitelist
+    }
+
+  }
+
+  private static String addCustomHandlerWithTermsComponentConfig(final MiniSolrCloudCluster cluster, final String collection,
+      final String defaultHandlerName, final String shardsWhitelist) throws Exception {
+    return addCustomHandler(cluster, collection, defaultHandlerName, shardsWhitelist);
+  }
+
+  private static String addCustomHandlerWithCustomTermsComponent(final MiniSolrCloudCluster cluster, final String collection,
+      final String defaultHandlerName) throws Exception {
+    return addCustomHandler(cluster, collection, defaultHandlerName, null);
+  }
+
+  private static String addCustomHandler(final MiniSolrCloudCluster cluster, final String collection,
+      final String defaultHandlerName, final String shardsWhitelist) throws Exception {
+
+    // determine custom handler name (the exact name should not matter)
+    final String customHandlerName = defaultHandlerName+"_custom"+random().nextInt();
+
+    // determine custom terms component name (the exact name should not matter)
+    final String customTermsComponentName = TermsComponent.COMPONENT_NAME+"_custom"+random().nextInt();
+
+    // determine terms component class name and attributes
+    final String customTermsComponentClass;
+    final String customTermsComponentAttributesJSON;
+    if (shardsWhitelist != null) {
+      customTermsComponentClass = TermsComponent.class.getName();
+      customTermsComponentAttributesJSON =
+          "    '"+HttpShardHandlerFactory.INIT_SHARDS_WHITELIST+"' : '"+shardsWhitelist+"',\n";
+    } else {
+      customTermsComponentClass = CustomTermsComponent.class.getName();
+      customTermsComponentAttributesJSON = "";
+    }
+
+    // add custom component
+    cluster.getSolrClient().request(
+        new ConfigRequest(
+            "{\n" +
+            "  'add-searchcomponent': {\n" +
+            customTermsComponentAttributesJSON +
+            "    'name': '"+customTermsComponentName+"',\n" +
+            "    'class': '"+customTermsComponentClass+"'\n" +
+            "  }\n" +
+            "}"),
+        collection);
+
+    // add custom handler
+    cluster.getSolrClient().request(
+        new ConfigRequest(
+            "{\n" +
+            "  'add-requesthandler': {\n" +
+            "    'name' : '"+customHandlerName+"',\n" +
+            "    'class' : '"+SearchHandler.class.getName()+"',\n" +
+            "    'components' : [ '"+QueryComponent.COMPONENT_NAME+"', '"+customTermsComponentName+"' ]\n" +
+            "  }\n" +
+            "}"),
+        collection);
+
+    return customHandlerName;
+  }
+
+  @Test
+  @Override
+  public void test() throws Exception {
+    for (final String clusterId : clusterId2cluster.keySet()) {
+      final MiniSolrCloudCluster cluster = clusterId2cluster.get(clusterId);
+      final String collection = COLLECTION_NAME;
+      doTest(cluster, collection);
+    }
+  }
+
+  private static void doTest(final MiniSolrCloudCluster cluster, final String collection) throws Exception {
+
+    // add some documents
+    final String id = "id";
+    final String f1 = "a_t";
+    final String f2 = "b_t";
+    final String v1 = "bee";
+    final String v2 = "buzz";
+    {
+      new UpdateRequest()
+          .add(sdoc(id, 1, f1, v1, f2, v2+" "+v2+" "+v2))
+          .add(sdoc(id, 2, f1, v1+" "+v1, f2, v2+" "+v2))
+          .add(sdoc(id, 3, f1, v1+" "+v1+" "+v1, f2, v2))
+          .commit(cluster.getSolrClient(), collection);
+    }
+
+    // search for the documents' terms ...
+    final String defaultHandlerName = "/select";
+
+    // search with the default handler ...
+    final String shards = findAndCheckTerms(cluster, collection,
+        defaultHandlerName,
+        null, // ... without specifying shards
+        (random().nextBoolean() ? null : f1), v1,
+        (random().nextBoolean() ? null : f2), v2,
+        null);
+
+    // search with the default handler ...
+    findAndCheckTerms(cluster, collection,
+        defaultHandlerName,
+        shards, // ... with specified shards, but all valid
+        (random().nextBoolean() ? null : f1), v1,
+        (random().nextBoolean() ? null : f2), v2,
+        null);
+    
+    ignoreException("not on the shards whitelist");
+    // this case should fail
+    findAndCheckTerms(cluster, collection,
+        defaultHandlerName,
+        shards + ",http://[ff01::114]:33332", // ... with specified shards with one invalid
+        (random().nextBoolean() ? null : f1), v1,
+        (random().nextBoolean() ? null : f2), v2,
+        "No live SolrServers available to handle this request");
+    unIgnoreException("not on the shards whitelist");
+
+    // configure a custom handler ...
+    final String customHandlerName;
+    if (random().nextBoolean()) {
+      // ... with a shards whitelist
+      customHandlerName = addCustomHandlerWithTermsComponentConfig(cluster, collection, defaultHandlerName, shards);
+    } else {
+      // ... with a custom terms component that disregards shards whitelist logic
+      customHandlerName = addCustomHandlerWithCustomTermsComponent(cluster, collection, defaultHandlerName);
+    }
+
+    // search with the custom handler ...
+    findAndCheckTerms(cluster, collection,
+        customHandlerName,
+        shards, // ... with specified shards
+        (random().nextBoolean() ? null : f1), v1,
+        (random().nextBoolean() ? null : f2), v2,
+        null);
+
+  }
+
+  private static String findAndCheckTerms(final MiniSolrCloudCluster cluster, final String collection,
+      String requestHandlerName, String in_shards,
+      String field1, String value1,
+      String field2, String value2,
+      String solrServerExceptionMessagePrefix) throws IOException {
+
+      // compose the query ...
+      final SolrQuery solrQuery =  new SolrQuery("*:*");
+      solrQuery.setRequestHandler(requestHandlerName);
+      solrQuery.add("shards.qt", requestHandlerName);
+      // ... asking for terms ...
+      solrQuery.setTerms(true);
+      if (field1 != null) {
+        solrQuery.addTermsField(field1);
+      }
+      if (field2 != null) {
+        solrQuery.addTermsField(field2);
+      }
+      // ... and shards info ...
+      solrQuery.add("shards.info", "true");
+      // ... passing shards to use (if we have a preference)
+      if (in_shards != null) {
+        solrQuery.add("shards", in_shards);
+      }
+
+      // make the query
+      final QueryResponse queryResponse;
+      try {
+        queryResponse = new QueryRequest(solrQuery)
+            .process(cluster.getSolrClient(), collection);
+        assertNull("expected exception ("+solrServerExceptionMessagePrefix+") not encountered", solrServerExceptionMessagePrefix);
+      } catch (SolrServerException sse) {
+        assertNotNull("unexpectedly caught exception "+sse, solrServerExceptionMessagePrefix);
+        assertTrue(sse.getMessage().startsWith(solrServerExceptionMessagePrefix));
+        assertThat(sse.getCause().getMessage(), containsString("not on the shards whitelist"));
+        return null;
+      }
+
+      // analyse the response ...
+      final TermsResponse termsResponse = queryResponse.getTermsResponse();
+      // ... checking the terms returned ...
+      checkTermsResponse(termsResponse, field1, value1);
+      checkTermsResponse(termsResponse, field2, value2);
+      // ... and assemble info about the shards ...
+      final String out_shards = extractShardAddresses(queryResponse, ",");
+      // ... to return to the caller
+      return out_shards;
+    }
+  
+
+  @SuppressWarnings("unchecked")
+  private static String extractShardAddresses(final QueryResponse queryResponse, final String delimiter) {
+    final StringBuilder sb = new StringBuilder();
+    final NamedList<Object> nl = (NamedList<Object>)queryResponse.getResponse().get("shards.info");
+    assertNotNull(queryResponse.toString(), nl);
+    for (int ii = 0; ii < nl.size(); ++ii) {
+      final String shardAddress = (String)((NamedList<Object>)nl.getVal(ii)).get("shardAddress");
+      if (sb.length() > 0) {
+        sb.append(delimiter);
+      }
+      sb.append(shardAddress);
+    }
+    return sb.toString();
+  }
+
+  private static void checkTermsResponse(TermsResponse termsResponse, String field, String value) {
+    if (field != null) {
+      final List<Term> ttList = termsResponse.getTerms(field);
+      assertEquals(1, ttList.size());
+      assertEquals(value, ttList.get(0).getTerm());
+    }
+  }
+
+}
diff --git a/solr/core/src/test/org/apache/solr/handler/component/DistributedDebugComponentTest.java b/solr/core/src/test/org/apache/solr/handler/component/DistributedDebugComponentTest.java
index f1daf39..5d3c7f4 100644
--- a/solr/core/src/test/org/apache/solr/handler/component/DistributedDebugComponentTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/component/DistributedDebugComponentTest.java
@@ -63,6 +63,7 @@ public class DistributedDebugComponentTest extends SolrJettyTestBase {
   
   @BeforeClass
   public static void createThings() throws Exception {
+    systemSetPropertySolrDisableShardsWhitelist("true");
     solrHome = createSolrHome();
     createAndStartJetty(solrHome.getAbsolutePath());
     String url = jetty.getBaseUrl().toString();
@@ -105,6 +106,7 @@ public class DistributedDebugComponentTest extends SolrJettyTestBase {
     jetty.stop();
     jetty=null;
     resetExceptionIgnores();
+    systemClearPropertySolrDisableShardsWhitelist();
   }
   
   @Test
diff --git a/solr/core/src/test/org/apache/solr/handler/component/ShardsWhitelistTest.java b/solr/core/src/test/org/apache/solr/handler/component/ShardsWhitelistTest.java
new file mode 100644
index 0000000..8aea6eb
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/component/ShardsWhitelistTest.java
@@ -0,0 +1,264 @@
+/*
+ * 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.handler.component;
+
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.CoreMatchers.nullValue;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.cloud.MiniSolrCloudCluster;
+import org.apache.solr.cloud.MultiSolrCloudTestCase;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class ShardsWhitelistTest extends MultiSolrCloudTestCase {
+
+  /**
+   * The cluster with this key will include an explicit list of host whitelisted (all hosts in both the clusters)
+   */
+  private static final String EXPLICIT_CLUSTER_KEY = "explicitCluster";
+  /**
+   * The cluster with this key will not include an explicit list of host whitelisted, will rely on live_nodes
+   */
+  private static final String IMPLICIT_CLUSTER_KEY = "implicitCluster";
+  private static final String EXPLICIT_WHITELIST_PROPERTY = "solr.tests.ShardsWhitelistTest.explicitWhitelist.";
+  protected static final String COLLECTION_NAME = "ShardsWhitelistTestCollection";
+
+  private static int numShards;
+  private static int numReplicas;
+  private static int maxShardsPerNode;
+  private static int nodesPerCluster;
+
+  private static void appendClusterNodes(final StringBuilder sb, final String delimiter,
+      final MiniSolrCloudCluster cluster) {
+    cluster.getJettySolrRunners().forEach((jetty) -> sb.append(jetty.getBaseUrl().toString() + delimiter));
+  }
+
+  @BeforeClass
+  public static void setupClusters() throws Exception {
+
+    final String[] clusterIds = new String[] {IMPLICIT_CLUSTER_KEY, EXPLICIT_CLUSTER_KEY};
+
+    numShards = 2; // +random().nextInt(2);
+    numReplicas = 1; // +random().nextInt(2);
+    maxShardsPerNode = 1; // +random().nextInt(2);
+    nodesPerCluster = (numShards * numReplicas + (maxShardsPerNode - 1)) / maxShardsPerNode;
+
+    final StringBuilder sb = new StringBuilder();
+
+    doSetupClusters(clusterIds,
+        new DefaultClusterCreateFunction() {
+
+          @Override
+          public MiniSolrCloudCluster apply(String clusterId) {
+            try {
+              final MiniSolrCloudCluster cluster = new SolrCloudTestCase.Builder(nodesPerCluster(clusterId),
+                  createTempDir())
+                      .addConfig("conf", configset("cloud-dynamic"))
+                      .withSolrXml(MiniSolrCloudCluster.DEFAULT_CLOUD_SOLR_XML.replace(
+                          MiniSolrCloudCluster.SOLR_TESTS_SHARDS_WHITELIST, EXPLICIT_WHITELIST_PROPERTY + clusterId))
+                      .build();
+              return cluster;
+            } catch (Exception e) {
+              throw new RuntimeException(e);
+            }
+          }
+
+          @Override
+          protected int nodesPerCluster(String clusterId) {
+            return nodesPerCluster;
+          }
+        },
+        new DefaultClusterInitFunction(numShards, numReplicas, maxShardsPerNode) {
+          @Override
+          public void accept(String clusterId, MiniSolrCloudCluster cluster) {
+            appendClusterNodes(sb, ",", cluster);
+            if (clusterId.equals(EXPLICIT_CLUSTER_KEY)) {
+              System.setProperty(EXPLICIT_WHITELIST_PROPERTY + clusterId, sb.toString());
+              for (JettySolrRunner runner : cluster.getJettySolrRunners()) {
+                try {
+                  runner.stop();
+                  runner.start(true);
+                } catch (Exception e) {
+                  throw new RuntimeException("Unable to restart runner", e);
+                }
+              }
+            }
+            doAccept(COLLECTION_NAME, cluster);
+          }
+        });
+  }
+
+  @AfterClass
+  public static void afterTests() {
+    System.clearProperty(EXPLICIT_WHITELIST_PROPERTY + EXPLICIT_CLUSTER_KEY);
+  }
+
+  private HttpShardHandlerFactory getShardHandlerFactory(String clusterId) {
+    return (HttpShardHandlerFactory) clusterId2cluster.get(clusterId).getJettySolrRunner(0).getCoreContainer()
+        .getShardHandlerFactory();
+  }
+
+  @Test
+  public void test() throws Exception {
+    assertThat(getShardHandlerFactory(EXPLICIT_CLUSTER_KEY).getWhitelistHostChecker().getWhitelistHosts(), notNullValue());
+    assertThat(getShardHandlerFactory(IMPLICIT_CLUSTER_KEY).getWhitelistHostChecker().getWhitelistHosts(), nullValue());
+
+    assertThat(getShardHandlerFactory(EXPLICIT_CLUSTER_KEY).getWhitelistHostChecker().hasExplicitWhitelist(), is(true));
+    assertThat(getShardHandlerFactory(IMPLICIT_CLUSTER_KEY).getWhitelistHostChecker().hasExplicitWhitelist(), is(false));
+    for (MiniSolrCloudCluster cluster : clusterId2cluster.values()) {
+      for (JettySolrRunner runner : cluster.getJettySolrRunners()) {
+        URI uri = runner.getBaseUrl().toURI();
+        assertThat(getShardHandlerFactory(EXPLICIT_CLUSTER_KEY).getWhitelistHostChecker().getWhitelistHosts(),
+            hasItem(uri.getHost() + ":" + uri.getPort()));
+      }
+    }
+
+    MiniSolrCloudCluster implicitCluster = clusterId2cluster.get(IMPLICIT_CLUSTER_KEY);
+    MiniSolrCloudCluster explicitCluster = clusterId2cluster.get(EXPLICIT_CLUSTER_KEY);
+
+    for (Map.Entry<String,MiniSolrCloudCluster> entry : clusterId2cluster.entrySet()) {
+      List<SolrInputDocument> docs = new ArrayList<>(10);
+      for (int i = 0; i < 10; i++) {
+        docs.add(new SolrInputDocument("id", entry.getKey() + i));
+      }
+      MiniSolrCloudCluster cluster = entry.getValue();
+      cluster.getSolrClient().add(COLLECTION_NAME, docs);
+      cluster.getSolrClient().commit(COLLECTION_NAME, true, true);
+
+      // test using ClusterState elements
+      assertThat("No shards specified, should work in both clusters",
+          numDocs("*:*", null, cluster), is(10));
+      assertThat("Both shards specified, should work in both clusters",
+          numDocs("*:*", "shard1,shard2", cluster), is(10));
+      assertThat("Both shards specified with collection name, should work in both clusters",
+          numDocs("*:*", COLLECTION_NAME + "_shard1", cluster), is(numDocs("*:*", "shard1", cluster)));
+
+      // test using explicit urls from within the cluster
+      assertThat("Shards has the full URLs, should be allowed since they are internal. Cluster=" + entry.getKey(),
+          numDocs("*:*", getShardUrl("shard1", cluster) + "," + getShardUrl("shard2", cluster), cluster), is(10));
+      assertThat("Full URL without scheme",
+          numDocs("*:*", getShardUrl("shard1", cluster).replaceAll("http://", "") + ","
+              + getShardUrl("shard2", cluster).replaceAll("http://", ""), cluster),
+          is(10));
+
+      // Mix shards with URLs
+      assertThat("Mix URL and cluster state object",
+          numDocs("*:*", "shard1," + getShardUrl("shard2", cluster), cluster), is(10));
+      assertThat("Mix URL and cluster state object",
+          numDocs("*:*", getShardUrl("shard1", cluster) + ",shard2", cluster), is(10));
+    }
+
+    // explicit whitelist includes all the nodes in both clusters. Requests should be allowed to go through
+    assertThat("A request to the explicit cluster with shards that point to the implicit one",
+        numDocs(
+            "id:implicitCluster*",
+            getShardUrl("shard1", implicitCluster) + "," + getShardUrl("shard2", implicitCluster),
+            explicitCluster),
+        is(10));
+
+    assertThat("A request to the explicit cluster with shards that point to the both clusters",
+        numDocs(
+            "*:*",
+            getShardUrl("shard1", implicitCluster)
+                + "," + getShardUrl("shard2", implicitCluster)
+                + "," + getShardUrl("shard1", explicitCluster)
+                + "," + getShardUrl("shard2", explicitCluster),
+            explicitCluster),
+        is(20));
+
+    // Implicit shouldn't allow requests to the other cluster
+    assertForbidden("id:explicitCluster*",
+        getShardUrl("shard1", explicitCluster) + "," + getShardUrl("shard2", explicitCluster),
+        implicitCluster);
+
+    assertForbidden("id:explicitCluster*",
+        "shard1," + getShardUrl("shard2", explicitCluster),
+        implicitCluster);
+
+    assertForbidden("id:explicitCluster*",
+        getShardUrl("shard1", explicitCluster) + ",shard2",
+        implicitCluster);
+
+    assertForbidden("id:explicitCluster*",
+        getShardUrl("shard1", explicitCluster),
+        implicitCluster);
+
+    assertThat("A typical internal request, should be handled locally",
+        numDocs(
+            "id:explicitCluster*",
+            null,
+            implicitCluster,
+            "distrib", "false",
+            "shard.url", getShardUrl("shard2", explicitCluster),
+            "shards.purpose", "64",
+            "isShard", "true"),
+        is(0));
+  }
+
+  private void assertForbidden(String query, String shards, MiniSolrCloudCluster cluster) throws IOException {
+    ignoreException("not on the shards whitelist");
+    try {
+      numDocs(
+          query,
+          shards,
+          cluster);
+      fail("Expecting failure for shards parameter: '" + shards + "'");
+    } catch (SolrServerException e) {
+      assertThat(e.getCause(), instanceOf(SolrException.class));
+      assertThat(((SolrException) e.getCause()).code(), is(SolrException.ErrorCode.FORBIDDEN.code));
+      assertThat(((SolrException) e.getCause()).getMessage(), containsString("not on the shards whitelist"));
+    }
+    unIgnoreException("not on the shards whitelist");
+  }
+
+  private String getShardUrl(String shardName, MiniSolrCloudCluster cluster) {
+    return cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(COLLECTION_NAME)
+        .getSlice(shardName).getReplicas().iterator().next().getCoreUrl();
+  }
+
+  private int numDocs(String queryString, String shardsParamValue, MiniSolrCloudCluster cluster, String... otherParams)
+      throws SolrServerException, IOException {
+    SolrQuery q = new SolrQuery(queryString);
+    if (shardsParamValue != null) {
+      q.set("shards", shardsParamValue);
+    }
+    if (otherParams != null) {
+      assert otherParams.length % 2 == 0;
+      for (int i = 0; i < otherParams.length; i += 2) {
+        q.set(otherParams[i], otherParams[i + 1]);
+      }
+    }
+    return (int) cluster.getSolrClient().query(COLLECTION_NAME, q).getResults().getNumFound();
+  }
+
+}
diff --git a/solr/core/src/test/org/apache/solr/handler/component/TestHttpShardHandlerFactory.java b/solr/core/src/test/org/apache/solr/handler/component/TestHttpShardHandlerFactory.java
index 01aff97..903d594 100644
--- a/solr/core/src/test/org/apache/solr/handler/component/TestHttpShardHandlerFactory.java
+++ b/solr/core/src/test/org/apache/solr/handler/component/TestHttpShardHandlerFactory.java
@@ -16,22 +16,34 @@
  */
 package org.apache.solr.handler.component;
 
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.nullValue;
+
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
-
+import java.util.Set;
 import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.impl.LBHttpSolrClient;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ShardParams;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.core.CoreContainer;
-
-import org.junit.BeforeClass;
+import org.apache.solr.handler.component.HttpShardHandlerFactory.WhitelistHostChecker;
 import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
 
 /**
  * Tests specifying a custom ShardHandlerFactory
@@ -40,6 +52,7 @@ public class TestHttpShardHandlerFactory extends SolrTestCaseJ4 {
 
   private static final String LOAD_BALANCER_REQUESTS_MIN_ABSOLUTE = "solr.tests.loadBalancerRequestsMinimumAbsolute";
   private static final String LOAD_BALANCER_REQUESTS_MAX_FRACTION = "solr.tests.loadBalancerRequestsMaximumFraction";
+  private static final String SHARDS_WHITELIST = "solr.tests.shardsWhitelist";
 
   private static int   expectedLoadBalancerRequestsMinimumAbsolute = 0;
   private static float expectedLoadBalancerRequestsMaximumFraction = 1.0f;
@@ -50,6 +63,7 @@ public class TestHttpShardHandlerFactory extends SolrTestCaseJ4 {
     expectedLoadBalancerRequestsMaximumFraction = (1+random().nextInt(10))/10f; // 0.1 .. 1.0
     System.setProperty(LOAD_BALANCER_REQUESTS_MIN_ABSOLUTE, Integer.toString(expectedLoadBalancerRequestsMinimumAbsolute));
     System.setProperty(LOAD_BALANCER_REQUESTS_MAX_FRACTION, Float.toString(expectedLoadBalancerRequestsMaximumFraction));
+
   }
 
   @AfterClass
@@ -216,4 +230,186 @@ public class TestHttpShardHandlerFactory extends SolrTestCaseJ4 {
     }
   }
 
+  @Test
+  public void getShardsWhitelist() throws Exception {
+    System.setProperty(SHARDS_WHITELIST, "http://abc:8983/,http://def:8984/,");
+    final Path home = Paths.get(TEST_HOME());
+    CoreContainer cc = null;
+    ShardHandlerFactory factory = null;
+    try {
+      cc = CoreContainer.createAndLoad(home, home.resolve("solr.xml"));
+      factory = cc.getShardHandlerFactory();
+      assertTrue(factory instanceof HttpShardHandlerFactory);
+      final HttpShardHandlerFactory httpShardHandlerFactory = ((HttpShardHandlerFactory)factory);
+      assertThat(httpShardHandlerFactory.getWhitelistHostChecker().getWhitelistHosts().size(), is(2));
+      assertThat(httpShardHandlerFactory.getWhitelistHostChecker().getWhitelistHosts(), hasItem("abc:8983"));
+      assertThat(httpShardHandlerFactory.getWhitelistHostChecker().getWhitelistHosts(), hasItem("def:8984"));
+    } finally {
+      if (factory != null) factory.close();
+      if (cc != null) cc.shutdown();
+      System.clearProperty(SHARDS_WHITELIST);
+    }
+  }
+  
+  @Test
+  public void testLiveNodesToHostUrl() throws Exception {
+    Set<String> liveNodes = new HashSet<>(Arrays.asList(new String[]{
+        "1.2.3.4:8983_solr",
+        "1.2.3.4:9000_",
+        "1.2.3.4:9001_solr-2",
+    }));
+    ClusterState cs = new ClusterState(0, liveNodes, new HashMap<>());
+    WhitelistHostChecker checker = new WhitelistHostChecker(null, true);
+    Set<String> hostSet = checker.generateWhitelistFromLiveNodes(cs);
+    assertThat(hostSet.size(), is(3));
+    assertThat(hostSet, hasItem("1.2.3.4:8983"));
+    assertThat(hostSet, hasItem("1.2.3.4:9000"));
+    assertThat(hostSet, hasItem("1.2.3.4:9001"));
+  }
+  
+  @Test
+  public void testWhitelistHostCheckerDisabled() throws Exception {
+    WhitelistHostChecker checker = new WhitelistHostChecker("http://cde:8983", false);
+    checker.checkWhitelist("http://abc-1.com:8983/solr", Arrays.asList(new String[]{"abc-1.com:8983/solr"}));
+    
+    try {
+      checker = new WhitelistHostChecker("http://cde:8983", true);
+      checker.checkWhitelist("http://abc-1.com:8983/solr", Arrays.asList(new String[]{"http://abc-1.com:8983/solr"}));
+      fail("Expecting exception");
+    } catch (SolrException se) {
+      assertThat(se.code(), is(SolrException.ErrorCode.FORBIDDEN.code));
+    }
+  }
+  
+  @Test
+  public void testWhitelistHostCheckerNoInput() throws Exception {
+    assertNull("Whitelist hosts should be null with null input",
+        new WhitelistHostChecker(null, true).getWhitelistHosts());
+    assertNull("Whitelist hosts should be null with empty input",
+        new WhitelistHostChecker("", true).getWhitelistHosts());
+  }
+  
+  @Test
+  public void testWhitelistHostCheckerSingleHost() {
+    WhitelistHostChecker checker = new WhitelistHostChecker("http://abc-1.com:8983/solr", true);
+    checker.checkWhitelist("http://abc-1.com:8983/solr", Arrays.asList(new String[]{"http://abc-1.com:8983/solr"}));
+  }
+  
+  @Test
+  public void testWhitelistHostCheckerMultipleHost() {
+    WhitelistHostChecker checker = new WhitelistHostChecker("http://abc-1.com:8983, http://abc-2.com:8983, http://abc-3.com:8983", true);
+    checker.checkWhitelist("http://abc-1.com:8983/solr", Arrays.asList(new String[]{"http://abc-1.com:8983/solr"}));
+  }
+  
+  @Test
+  public void testWhitelistHostCheckerMultipleHost2() {
+    WhitelistHostChecker checker = new WhitelistHostChecker("http://abc-1.com:8983, http://abc-2.com:8983, http://abc-3.com:8983", true);
+    checker.checkWhitelist("http://abc-1.com:8983/solr", Arrays.asList(new String[]{"http://abc-1.com:8983/solr", "http://abc-2.com:8983/solr"}));
+  }
+  
+  @Test
+  public void testWhitelistHostCheckerNoProtocolInParameter() {
+    WhitelistHostChecker checker = new WhitelistHostChecker("http://abc-1.com:8983, http://abc-2.com:8983, http://abc-3.com:8983", true);
+    checker.checkWhitelist("abc-1.com:8983/solr", Arrays.asList(new String[]{"abc-1.com:8983/solr"}));
+  }
+  
+  @Test
+  public void testWhitelistHostCheckerNonWhitelistedHost1() {
+    WhitelistHostChecker checker = new WhitelistHostChecker("http://abc-1.com:8983, http://abc-2.com:8983, http://abc-3.com:8983", true);
+    try {
+      checker.checkWhitelist("http://abc-1.com:8983/solr", Arrays.asList(new String[]{"http://abc-4.com:8983/solr"}));
+      fail("Expected exception");
+    } catch (SolrException e) {
+      assertThat(e.code(), is(SolrException.ErrorCode.FORBIDDEN.code));
+      assertThat(e.getMessage(), containsString("not on the shards whitelist"));
+    }
+  }
+  
+  @Test
+  public void testWhitelistHostCheckerNonWhitelistedHost2() {
+    WhitelistHostChecker checker = new WhitelistHostChecker("http://abc-1.com:8983, http://abc-2.com:8983, http://abc-3.com:8983", true);
+    try {
+      checker.checkWhitelist("http://abc-1.com:8983/solr", Arrays.asList(new String[]{"http://abc-1.com:8983/solr", "http://abc-4.com:8983/solr"}));
+      fail("Expected exception");
+    } catch (SolrException e) {
+      assertThat(e.code(), is(SolrException.ErrorCode.FORBIDDEN.code));
+      assertThat(e.getMessage(), containsString("not on the shards whitelist"));
+    }
+  }
+  
+  @Test
+  public void testWhitelistHostCheckerNonWhitelistedHostHttps() {
+    WhitelistHostChecker checker = new WhitelistHostChecker("http://abc-1.com:8983, http://abc-2.com:8983, http://abc-3.com:8983", true);
+    checker.checkWhitelist("https://abc-1.com:8983/solr", Arrays.asList(new String[]{"https://abc-1.com:8983/solr"}));
+  }
+  
+  @Test
+  public void testWhitelistHostCheckerInvalidUrl() {
+    WhitelistHostChecker checker = new WhitelistHostChecker("http://abc-1.com:8983, http://abc-2.com:8983, http://abc-3.com:8983", true);
+    try {
+      checker.checkWhitelist("abc_1", Arrays.asList(new String[]{"abc_1"}));
+      fail("Expected exception");
+    } catch (SolrException e) {
+      assertThat(e.code(), is(SolrException.ErrorCode.BAD_REQUEST.code));
+      assertThat(e.getMessage(), containsString("Invalid URL syntax"));
+    }
+  }
+  
+  @Test
+  public void testWhitelistHostCheckerCoreSpecific() {
+    // cores are removed completely so it doesn't really matter if they were set in config
+    WhitelistHostChecker checker = new WhitelistHostChecker("http://abc-1.com:8983/solr/core1, http://abc-2.com:8983/solr2/core2", true);
+    checker.checkWhitelist("http://abc-1.com:8983/solr/core2", Arrays.asList(new String[]{"http://abc-1.com:8983/solr/core2"}));
+  }
+  
+  @Test
+  public void testGetShardsOfWhitelistedHostsUnset() {
+    assertThat(WhitelistHostChecker.implGetShardsWhitelist(null), nullValue());
+  }
+  
+  @Test
+  public void testGetShardsOfWhitelistedHostsEmpty() {
+    assertThat(WhitelistHostChecker.implGetShardsWhitelist(""), nullValue());
+  }
+  
+  @Test
+  public void testGetShardsOfWhitelistedHostsSingle() {
+    assertThat(WhitelistHostChecker.implGetShardsWhitelist("http://abc-1.com:8983/solr/core1").size(), is(1));
+    assertThat(WhitelistHostChecker.implGetShardsWhitelist("http://abc-1.com:8983/solr/core1").iterator().next(), equalTo("abc-1.com:8983"));
+  }
+  
+  @Test
+  public void testGetShardsOfWhitelistedHostsMulti() {
+    assertThat(WhitelistHostChecker.implGetShardsWhitelist("http://abc-1.com:8983/solr/core1,http://abc-1.com:8984/solr").size(), is(2));
+    assertThat(WhitelistHostChecker.implGetShardsWhitelist("http://abc-1.com:8983/solr/core1,http://abc-1.com:8984/solr"), hasItem("abc-1.com:8983"));
+    assertThat(WhitelistHostChecker.implGetShardsWhitelist("http://abc-1.com:8983/solr/core1,http://abc-1.com:8984/solr"), hasItem("abc-1.com:8984"));
+  }
+  
+  @Test
+  public void testGetShardsOfWhitelistedHostsIpv4() {
+    assertThat(WhitelistHostChecker.implGetShardsWhitelist("http://10.0.0.1:8983/solr/core1,http://127.0.0.1:8984/solr").size(), is(2));
+    assertThat(WhitelistHostChecker.implGetShardsWhitelist("http://10.0.0.1:8983/solr/core1,http://127.0.0.1:8984/solr"), hasItem("10.0.0.1:8983"));
+    assertThat(WhitelistHostChecker.implGetShardsWhitelist("http://10.0.0.1:8983/solr/core1,http://127.0.0.1:8984/solr"), hasItem("127.0.0.1:8984"));
+  }
+  
+  @Test
+  public void testGetShardsOfWhitelistedHostsIpv6() {
+    assertThat(WhitelistHostChecker.implGetShardsWhitelist("http://[2001:abc:abc:0:0:123:456:1234]:8983/solr/core1,http://[::1]:8984/solr").size(), is(2));
+    assertThat(WhitelistHostChecker.implGetShardsWhitelist("http://[2001:abc:abc:0:0:123:456:1234]:8983/solr/core1,http://[::1]:8984/solr"), hasItem("[2001:abc:abc:0:0:123:456:1234]:8983"));
+    assertThat(WhitelistHostChecker.implGetShardsWhitelist("http://[2001:abc:abc:0:0:123:456:1234]:8983/solr/core1,http://[::1]:8984/solr"), hasItem("[::1]:8984"));
+  }
+  
+  @Test
+  public void testGetShardsOfWhitelistedHostsHttps() {
+    assertThat(WhitelistHostChecker.implGetShardsWhitelist("https://abc-1.com:8983/solr/core1").size(), is(1));
+    assertThat(WhitelistHostChecker.implGetShardsWhitelist("https://abc-1.com:8983/solr/core1"), hasItem("abc-1.com:8983"));
+  }
+  
+  @Test
+  public void testGetShardsOfWhitelistedHostsNoProtocol() {
+    assertThat(WhitelistHostChecker.implGetShardsWhitelist("abc-1.com:8983/solr"),
+        equalTo(WhitelistHostChecker.implGetShardsWhitelist("http://abc-1.com:8983/solr")));
+    assertThat(WhitelistHostChecker.implGetShardsWhitelist("abc-1.com:8983/solr"),
+        equalTo(WhitelistHostChecker.implGetShardsWhitelist("https://abc-1.com:8983/solr")));
+  }
 }
diff --git a/solr/core/src/test/org/apache/solr/search/TestSmileRequest.java b/solr/core/src/test/org/apache/solr/search/TestSmileRequest.java
index 0bf46e7..765e984 100644
--- a/solr/core/src/test/org/apache/solr/search/TestSmileRequest.java
+++ b/solr/core/src/test/org/apache/solr/search/TestSmileRequest.java
@@ -42,6 +42,7 @@ public class TestSmileRequest extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void beforeTests() throws Exception {
+    systemSetPropertySolrDisableShardsWhitelist("true");
     JSONTestUtil.failRepeatedKeys = true;
     initCore("solrconfig-tlog.xml", "schema_latest.xml");
   }
@@ -59,6 +60,7 @@ public class TestSmileRequest extends SolrTestCaseJ4 {
       servers.stop();
       servers = null;
     }
+    systemClearPropertySolrDisableShardsWhitelist();
   }
 
   @Test
diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java
index e757b66..a333914 100644
--- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java
+++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java
@@ -41,6 +41,7 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS {
 
   @BeforeClass
   public static void beforeTests() throws Exception {
+    systemSetPropertySolrDisableShardsWhitelist("true");
     // we need DVs on point fields to compute stats & facets
     if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
     
@@ -61,6 +62,7 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS {
       servers.stop();
       servers = null;
     }
+    systemClearPropertySolrDisableShardsWhitelist();
   }
 
 
diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
index 8909815..0eccea8 100644
--- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
+++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
@@ -16,6 +16,8 @@
  */
 package org.apache.solr.search.facet;
 
+import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
+import com.tdunning.math.stats.AVLTreeDigest;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -26,21 +28,17 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.atomic.AtomicLong;
-
-import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
-import com.tdunning.math.stats.AVLTreeDigest;
-import org.apache.solr.client.solrj.SolrClient;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.util.hll.HLL;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.JSONTestUtil;
 import org.apache.solr.SolrTestCaseHS;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.macro.MacroExpander;
-
+import org.apache.solr.util.hll.HLL;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -73,14 +71,37 @@ public class TestJsonFacets extends SolrTestCaseHS {
     initCore("solrconfig-tlog.xml","schema_latest.xml");
   }
 
+  /**
+   * Start all servers for cluster, initialize shards whitelist and then restart
+   */
   public static void initServers() throws Exception {
     if (servers == null) {
       servers = new SolrInstances(3, "solrconfig-tlog.xml", "schema_latest.xml");
+      // Set the shards whitelist to all shards plus the fake one used for tolerant test
+      System.setProperty(SOLR_TESTS_SHARDS_WHITELIST, servers.getWhitelistString() + ",http://[ff01::114]:33332");
+      systemSetPropertySolrDisableShardsWhitelist("false");
+      restartServers();
     }
   }
 
+  /**
+   * Restart all configured servers, i.e. configuration will be re-read
+   */
+  public static void restartServers() {
+    servers.slist.forEach(s -> {
+      try {
+        s.stop();
+        s.start();
+      } catch (Exception e) {
+        fail("Exception during server restart: " + e.getMessage());
+      }
+    });
+  }
+
   @AfterClass
   public static void afterTests() throws Exception {
+    System.clearProperty(SOLR_TESTS_SHARDS_WHITELIST);
+    systemClearPropertySolrDisableShardsWhitelist();
     JSONTestUtil.failRepeatedKeys = false;
     FacetFieldProcessorByHashDV.MAXIMUM_STARTING_TABLE_SIZE=origTableSize;
     FacetField.FacetMethod.DEFAULT_METHOD = origDefaultFacetMethod;
@@ -2318,6 +2339,7 @@ public class TestJsonFacets extends SolrTestCaseHS {
   public void doTestPrelimSortingDistrib(final boolean extraAgg, final boolean extraSubFacet) throws Exception {
     // we only use 2 shards, but we also want to to sanity check code paths if one (additional) shard is empty
     final int totalShards = random().nextBoolean() ? 2 : 3;
+    
     final SolrInstances nodes = new SolrInstances(totalShards, "solrconfig-tlog.xml", "schema_latest.xml");
     try {
       final Client client = nodes.getClient(random().nextInt());
@@ -2789,7 +2811,7 @@ public class TestJsonFacets extends SolrTestCaseHS {
   public void testTolerant() throws Exception {
     initServers();
     Client client = servers.getClient(random().nextInt());
-    client.queryDefaults().set("shards", servers.getShards() + ",[ff01::114]:33332:/ignore_exception");
+    client.queryDefaults().set("shards", servers.getShards() + ",[ff01::114]:33332/ignore_exception");
     indexSimple(client);
 
     try {
diff --git a/solr/core/src/test/org/apache/solr/search/json/TestJsonRequest.java b/solr/core/src/test/org/apache/solr/search/json/TestJsonRequest.java
index 6efe840..bcc936d 100644
--- a/solr/core/src/test/org/apache/solr/search/json/TestJsonRequest.java
+++ b/solr/core/src/test/org/apache/solr/search/json/TestJsonRequest.java
@@ -32,6 +32,7 @@ public class TestJsonRequest extends SolrTestCaseHS {
 
   @BeforeClass
   public static void beforeTests() throws Exception {
+    systemSetPropertySolrDisableShardsWhitelist("true");
     JSONTestUtil.failRepeatedKeys = true;
     initCore("solrconfig-tlog.xml","schema_latest.xml");
   }
@@ -49,6 +50,7 @@ public class TestJsonRequest extends SolrTestCaseHS {
       servers.stop();
       servers = null;
     }
+    systemClearPropertySolrDisableShardsWhitelist();
   }
 
   @Test
diff --git a/solr/server/solr/solr.xml b/solr/server/solr/solr.xml
index 68b15ba..795e352 100644
--- a/solr/server/solr/solr.xml
+++ b/solr/server/solr/solr.xml
@@ -48,6 +48,7 @@
     class="HttpShardHandlerFactory">
     <int name="socketTimeout">${socketTimeout:600000}</int>
     <int name="connTimeout">${connTimeout:60000}</int>
+    <str name="shardsWhitelist">${solr.shardsWhitelist:}</str>
   </shardHandlerFactory>
 
 </solr>
diff --git a/solr/solr-ref-guide/src/distributed-requests.adoc b/solr/solr-ref-guide/src/distributed-requests.adoc
index 842a021..dc7d594 100644
--- a/solr/solr-ref-guide/src/distributed-requests.adoc
+++ b/solr/solr-ref-guide/src/distributed-requests.adoc
@@ -85,7 +85,7 @@ To configure the standard search handler, provide a configuration like this in `
 </requestHandler>
 ----
 
-The parameters that can be specified are as follows:
+`HttpShardHandlerFactory` is the only `ShardHandlerFactory` implementation included out of the box with Solr, It accepts the following parameters:
 
 `socketTimeout`::
 The amount of time in ms that a socket is allowed to wait. The default is `0`, where the operating system's default will be used.
@@ -114,6 +114,12 @@ If specified, the thread pool will use a backing queue instead of a direct hando
 `fairnessPolicy`::
 Chooses the JVM specifics dealing with fair policy queuing, if enabled distributed searches will be handled in a First in First out fashion at a cost to throughput. If disabled throughput will be favored over latency. The default is `false`.
 
+`shardsWhitelist`::
+If specified, this lists limits what nodes can be requested in the `shards` request parameter. In cloud mode this whitelist is automatically configured to include all live nodes in the cluster. In standalone mode the whitelist defaults to empty (sharding not allowed). If you need to disable this feature for backwards compatibility, you can set the system property `solr.disable.shardsWhitelist=true`. The value of this parameter is a comma separated list of the nodes that will be whitelist [...]
+`10.0.0.1:8983/solr,10.0.0.1:8984/solr`.
+
+NOTE: In cloud mode, if at least one node is included in the whitelist, then the live_nodes will no longer be used as source for the list. This means that, if you need to do a cross-cluster request using the `shards` parameter in cloud mode (in addition to regular within-cluster requests), you'll need to add all nodes (local cluster + remote nodes) to the whitelist. 
+
 == Configuring statsCache (Distributed IDF)
 
 Document and term statistics are needed in order to calculate relevancy. Solr provides four implementations out of the box when it comes to document stats calculation:
diff --git a/solr/solr-ref-guide/src/distributed-search-with-index-sharding.adoc b/solr/solr-ref-guide/src/distributed-search-with-index-sharding.adoc
index 70324ac..d8a2d76 100644
--- a/solr/solr-ref-guide/src/distributed-search-with-index-sharding.adoc
+++ b/solr/solr-ref-guide/src/distributed-search-with-index-sharding.adoc
@@ -59,6 +59,9 @@ The following components support distributed search:
 * The *Stats* component, which returns simple statistics for numeric fields within the DocSet.
 * The *Debug* component, which helps with debugging.
 
+=== Shards Whitelist
+What nodes are allowed in the `shards` parameter is configurable through the `shardsWhitelist` property in `solr.xml`. This whitelist is automatically configured for SolrCloud but needs explicit configuration for master/slave mode. Read more details in <<distributed-requests.adoc#configuring-the-shardhandlerfactory>>. 
+
 == Limitations to Distributed Search
 
 Distributed searching in Solr has the following limitations:
diff --git a/solr/solr-ref-guide/src/the-terms-component.adoc b/solr/solr-ref-guide/src/the-terms-component.adoc
index 6fe40d9..b2705b1 100644
--- a/solr/solr-ref-guide/src/the-terms-component.adoc
+++ b/solr/solr-ref-guide/src/the-terms-component.adoc
@@ -295,3 +295,5 @@ Specifies the shards in your distributed indexing configuration. For more inform
 
 `shards.qt`::
 Specifies the request handler Solr uses for requests to shards.
+
+Same as with regular distributed search, the `shards` parameter is subject to a host whitelist that has to be configured in the component init parameters using the configuration key `shardsWhitelist` and the list of hosts as values. In the same way as with distributed search, the whitelist will be populated to all live nodes by default when running in SolrCloud mode. If you need to disable this feature for backwards compatibility, you can set the system property `solr.disable.shardsWhite [...]
diff --git a/solr/solrj/src/test-files/solrj/solr/solr.xml b/solr/solrj/src/test-files/solrj/solr/solr.xml
index 6eef53f..0e9f3f4 100644
--- a/solr/solrj/src/test-files/solrj/solr/solr.xml
+++ b/solr/solrj/src/test-files/solrj/solr/solr.xml
@@ -30,6 +30,7 @@
     <str name="urlScheme">${urlScheme:}</str>
     <int name="socketTimeout">${socketTimeout:90000}</int>
     <int name="connTimeout">${connTimeout:15000}</int>
+    <str name="shardsWhitelist">${solr.tests.shardsWhitelist:}</str>
   </shardHandlerFactory>
 
   <solrcloud>
diff --git a/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java b/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
index 4728aa3..82b1b64 100644
--- a/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
+++ b/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
@@ -175,6 +175,18 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
     System.clearProperty("hostContext");
   }
 
+  @SuppressWarnings("deprecation")
+  @BeforeClass
+  public static void setSolrDisableShardsWhitelist() throws Exception {
+    systemSetPropertySolrDisableShardsWhitelist("true");
+  }
+
+  @SuppressWarnings("deprecation")
+  @AfterClass
+  public static void clearSolrDisableShardsWhitelist() throws Exception {
+    systemClearPropertySolrDisableShardsWhitelist();
+  }
+
   private static String getHostContextSuitableForServletContext() {
     String ctx = System.getProperty("hostContext","/solr");
     if ("".equals(ctx)) ctx = "/solr";
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseHS.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseHS.java
index 2da0c84..e175566 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseHS.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseHS.java
@@ -64,6 +64,8 @@ import org.slf4j.LoggerFactory;
 //@LuceneTestCase.SuppressCodecs({"Lucene3x","Lucene40","Lucene41","Lucene42","Lucene45","Appending","Asserting"})
 public class SolrTestCaseHS extends SolrTestCaseJ4 {
   
+  public static final String SOLR_TESTS_SHARDS_WHITELIST = "solr.tests.shardsWhitelist";
+
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   @SafeVarargs
   public static <T> Set<T> set(T... a) {
@@ -468,6 +470,12 @@ public class SolrTestCaseHS extends SolrTestCaseJ4 {
 
       // silly stuff included from solrconfig.snippet.randomindexconfig.xml
       System.setProperty("solr.tests.maxBufferedDocs", String.valueOf(100000));
+      
+      // If we want to run with whitelist list, this must be explicitly set to true for the test
+      // otherwise we disable the check
+      if (System.getProperty(SYSTEM_PROPERTY_SOLR_DISABLE_SHARDS_WHITELIST) == null) {
+        systemSetPropertySolrDisableShardsWhitelist("true");
+      }
 
       jetty.start();
       port = jetty.getLocalPort();
@@ -534,6 +542,20 @@ public class SolrTestCaseHS extends SolrTestCaseJ4 {
     public String getShards() {
       return getShardsParam(slist);
     }
+    
+    public String getWhitelistString() {
+      StringBuilder sb = new StringBuilder();
+      boolean first = true;
+      for (SolrInstance instance : slist) {
+        if (first) {
+          first = false;
+        } else {
+          sb.append(',');
+        }
+        sb.append( instance.getBaseURL().replace("/solr", ""));
+      }
+      return sb.toString();
+    }
 
     public List<SolrClient> getSolrJs() {
       List<SolrClient> solrjs = new ArrayList<>(slist.size());
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
index 9a7fef1..400af1b 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -190,6 +190,9 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
 
   public static final String SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICYFACTORY = "solr.tests.mergePolicyFactory";
 
+  @Deprecated // For backwards compatibility only. Please do not use in new tests.
+  public static final String SYSTEM_PROPERTY_SOLR_DISABLE_SHARDS_WHITELIST = "solr.disable.shardsWhitelist";
+
   protected static String coreName = DEFAULT_TEST_CORENAME;
 
   public static int DEFAULT_CONNECTION_TIMEOUT = 60000;  // default socket connection timeout in ms
@@ -2715,6 +2718,16 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
     System.clearProperty(SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICYFACTORY);
   }
   
+  @Deprecated // For backwards compatibility only. Please do not use in new tests.
+  protected static void systemSetPropertySolrDisableShardsWhitelist(String value) {
+    System.setProperty(SYSTEM_PROPERTY_SOLR_DISABLE_SHARDS_WHITELIST, value);
+  }
+
+  @Deprecated // For backwards compatibility only. Please do not use in new tests.
+  protected static void systemClearPropertySolrDisableShardsWhitelist() {
+    System.clearProperty(SYSTEM_PROPERTY_SOLR_DISABLE_SHARDS_WHITELIST);
+  }
+
   protected <T> T pickRandom(T... options) {
     return options[random().nextInt(options.length)];
   }
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 fd719ed..2fbeba8 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
@@ -82,6 +82,8 @@ public class MiniSolrCloudCluster {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   
+  public static final String SOLR_TESTS_SHARDS_WHITELIST = "solr.tests.shardsWhitelist";
+
   public static final String DEFAULT_CLOUD_SOLR_XML = "<solr>\n" +
       "\n" +
       "  <str name=\"shareSchema\">${shareSchema:false}</str>\n" +
@@ -93,6 +95,7 @@ public class MiniSolrCloudCluster {
       "    <str name=\"urlScheme\">${urlScheme:}</str>\n" +
       "    <int name=\"socketTimeout\">${socketTimeout:90000}</int>\n" +
       "    <int name=\"connTimeout\">${connTimeout:15000}</int>\n" +
+      "    <str name=\"shardsWhitelist\">${"+SOLR_TESTS_SHARDS_WHITELIST+":}</str>\n" +
       "  </shardHandlerFactory>\n" +
       "\n" +
       "  <solrcloud>\n" +