You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by br...@apache.org on 2021/04/01 15:07:45 UTC

[lucene-solr] branch branch_8x updated: SOLR-15217: Use shardsWhitelist in ReplicationHandler.

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new b81252f  SOLR-15217: Use shardsWhitelist in ReplicationHandler.
b81252f is described below

commit b81252f1639da0eda670da9ef6bf12e3811e82d2
Author: Bruno Roustant <33...@users.noreply.github.com>
AuthorDate: Thu Apr 1 17:07:17 2021 +0200

    SOLR-15217: Use shardsWhitelist in ReplicationHandler.
---
 solr/CHANGES.txt                                   |  2 ++
 .../java/org/apache/solr/handler/IndexFetcher.java | 36 ++++++++++++++++++++--
 .../handler/component/HttpShardHandlerFactory.java |  8 ++---
 .../solr/handler/TestReplicationHandler.java       | 22 +++++++++++++
 .../TestReplicationHandlerDiskOverFlow.java        |  3 ++
 5 files changed, 63 insertions(+), 8 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b53ec5e0..2c6aa89 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -48,6 +48,8 @@ Improvements
 
 * SOLR-15155: Let CloudHttp2SolrClient accept an external Http2SolrClient Builder (Tomás Fernández Löbbe)
 
+* SOLR-15217: Use shardsWhitelist in ReplicationHandler. (Bruno Roustant)
+
 Optimizations
 ---------------------
 * SOLR-15079: Block Collapse - Faster collapse code when groups are co-located via Block Join style nested doc indexing.
diff --git a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
index a97ff33..da2846a 100644
--- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
+++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
@@ -78,6 +78,7 @@ 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.Replica;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -90,6 +91,8 @@ import org.apache.solr.core.DirectoryFactory;
 import org.apache.solr.core.DirectoryFactory.DirContext;
 import org.apache.solr.core.IndexDeletionPolicyWrapper;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.component.HttpShardHandlerFactory;
+import org.apache.solr.handler.component.ShardHandlerFactory;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.search.SolrIndexSearcher;
@@ -248,7 +251,7 @@ public class IndexFetcher {
       leaderUrl = leaderUrl.substring(0, leaderUrl.length()-12);
       log.warn("'leaderUrl' must be specified without the {} suffix", ReplicationHandler.PATH);
     }
-    this.leaderUrl = leaderUrl;
+    setLeaderUrl(leaderUrl);
 
     this.replicationHandler = handler;
     String compress = (String) initArgs.get(COMPRESSION);
@@ -271,7 +274,34 @@ public class IndexFetcher {
     String httpBasicAuthPassword = (String) initArgs.get(HttpClientUtil.PROP_BASIC_AUTH_PASS);
     myHttpClient = createHttpClient(solrCore, httpBasicAuthUser, httpBasicAuthPassword, useExternalCompression);
   }
-  
+
+  private void setLeaderUrl(String leaderUrl) {
+    if (leaderUrl != null) {
+      ShardHandlerFactory shardHandlerFactory = solrCore.getCoreContainer().getShardHandlerFactory();
+      if (shardHandlerFactory instanceof HttpShardHandlerFactory) {
+        ZkController zkController = solrCore.getCoreContainer().getZkController();
+        ClusterState clusterState = zkController == null ? null : zkController.getClusterState();
+        try {
+          ((HttpShardHandlerFactory) shardHandlerFactory).getWhitelistHostChecker()
+              .checkWhitelist(clusterState, null, Collections.singletonList(leaderUrl));
+        } catch (SolrException e) {
+          // Replace the exception because the message is about the 'shard' parameter, which is not right here.
+          // This code is refactored and cleaned up in 9.x and above.
+          if (e.code() == ErrorCode.BAD_REQUEST.code) {
+            throw new SolrException(ErrorCode.BAD_REQUEST,
+                "Invalid URL syntax in '" + LEADER_URL + "' with value '" + leaderUrl + "'", e);
+          } else {
+            throw new SolrException(SolrException.ErrorCode.FORBIDDEN,
+                "The '" + LEADER_URL + "' parameter value '" + leaderUrl
+                    + "' is not in the '" + HttpShardHandlerFactory.INIT_SHARDS_WHITELIST + "'."
+                    + HttpShardHandlerFactory.SET_SOLR_DISABLE_SHARDS_WHITELIST_CLUE);
+          }
+        }
+      }
+    }
+    this.leaderUrl = leaderUrl;
+  }
+
   @SuppressWarnings({"unchecked"})
   protected <T> T getParameter(@SuppressWarnings({"rawtypes"})NamedList initArgs, String configKey, T defaultValue, StringBuilder sb) {
     T toReturn = defaultValue;
@@ -405,7 +435,7 @@ public class IndexFetcher {
           return IndexFetchResult.LEADER_IS_NOT_ACTIVE;
         }
         if (!replica.getCoreUrl().equals(leaderUrl)) {
-          leaderUrl = replica.getCoreUrl();
+          setLeaderUrl(replica.getCoreUrl());
           log.info("Updated leaderUrl to {}", leaderUrl);
           // TODO: Do we need to set forceReplication = true?
         } else {
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 9f290f4..a2382ea 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
@@ -140,9 +140,9 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
 
   public static final String INIT_SHARDS_WHITELIST = "shardsWhitelist";
 
-  static final String INIT_SOLR_DISABLE_SHARDS_WHITELIST = "solr.disable." + INIT_SHARDS_WHITELIST;
+  public 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";
+  public 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.
@@ -510,7 +510,7 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
      * @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) {
+    public void checkWhitelist(ClusterState clusterState, String shardsParamValue, List<String> shardUrls) {
       if (!whitelistHostCheckingEnabled) {
         return;
       }
@@ -540,8 +540,6 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
           throw new SolrException(ErrorCode.BAD_REQUEST, "Invalid URL syntax in \"shards\" parameter: " + shardsParamValue);
         }
         if (!localWhitelistHosts.contains(url.getHost() + ":" + url.getPort())) {
-          log.warn("The '{}' parameter value '{}' contained value(s) not on the shards whitelist ({}), shardUrl: '{}'"
-              , ShardParams.SHARDS, shardsParamValue, localWhitelistHosts, 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);
diff --git a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
index 62b994f..af889b8 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
@@ -79,6 +79,7 @@ import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.StandardDirectoryFactory;
 import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager;
+import org.apache.solr.handler.component.HttpShardHandlerFactory;
 import org.apache.solr.util.FileUtils;
 import org.apache.solr.util.TestInjection;
 import org.apache.solr.util.TimeOut;
@@ -131,6 +132,7 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
   @Before
   public void setUp() throws Exception {
     super.setUp();
+    systemSetPropertySolrDisableShardsWhitelist("true");
 //    System.setProperty("solr.directoryFactory", "solr.StandardDirectoryFactory");
     // For manual testing only
     // useFactory(null); // force an FS factory.
@@ -160,6 +162,7 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
   @After
   public void tearDown() throws Exception {
     super.tearDown();
+    systemClearPropertySolrDisableShardsWhitelist();
     if (null != leaderJetty) {
       leaderJetty.stop();
       leaderJetty = null;
@@ -303,6 +306,25 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
   }
 
   @Test
+  public void testShardsWhitelist() throws Exception {
+    // Run another test with shards whitelist enabled and whitelist is empty.
+    // Expect an exception because the leader URL is not allowed.
+    systemClearPropertySolrDisableShardsWhitelist();
+    SolrException e = expectThrows(SolrException.class, this::doTestDetails);
+    assertTrue(e.getMessage().contains("is not in the '" + HttpShardHandlerFactory.INIT_SHARDS_WHITELIST + "'"));
+
+    // Set the whitelist to allow the leader URL.
+    // Expect the same test to pass now.
+    String propertyName = "solr.tests." + HttpShardHandlerFactory.INIT_SHARDS_WHITELIST;
+    System.setProperty(propertyName, leaderJetty.getBaseUrl() + "," + followerJetty.getBaseUrl());
+    try {
+      doTestDetails();
+    } finally {
+      System.clearProperty(propertyName);
+    }
+  }
+
+  @Test
   public void doTestDetails() throws Exception {
     followerJetty.stop();
     
diff --git a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerDiskOverFlow.java b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerDiskOverFlow.java
index 9565945..b8f5b18 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerDiskOverFlow.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerDiskOverFlow.java
@@ -37,6 +37,7 @@ import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.handler.component.HttpShardHandlerFactory;
 import org.apache.solr.util.LogLevel;
 import org.junit.After;
 import org.junit.Before;
@@ -78,6 +79,7 @@ public class TestReplicationHandlerDiskOverFlow extends SolrTestCaseJ4 {
     leader.setUp();
     leaderJetty = createAndStartJetty(leader);
     leaderClient = createNewSolrClient(leaderJetty.getLocalPort());
+    System.setProperty("solr.tests." + HttpShardHandlerFactory.INIT_SHARDS_WHITELIST, leaderJetty.getBaseUrl().toString());
 
     follower = new TestReplicationHandler.SolrInstance(createTempDir("solr-instance").toFile(), "follower", leaderJetty.getLocalPort());
     follower.setUp();
@@ -108,6 +110,7 @@ public class TestReplicationHandlerDiskOverFlow extends SolrTestCaseJ4 {
       followerClient.close();
       followerClient = null;
     }
+    System.clearProperty("solr.tests." + HttpShardHandlerFactory.INIT_SHARDS_WHITELIST);
     System.clearProperty("solr.indexfetcher.sotimeout");
     
     IndexFetcher.usableDiskSpaceProvider = originalDiskSpaceprovider;