You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mu...@apache.org on 2020/09/24 17:16:19 UTC

[lucene-solr] branch master updated: SOLR-14503: use specified waitForZk val as conn timeout for zk

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

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


The following commit(s) were added to refs/heads/master by this push:
     new ddd1072  SOLR-14503: use specified waitForZk val as conn timeout for zk
ddd1072 is described below

commit ddd10725b00649edc80726c59f9fdf0442adb6c2
Author: Munendra S N <mu...@apache.org>
AuthorDate: Wed Sep 23 16:11:15 2020 +0530

    SOLR-14503: use specified waitForZk val as conn timeout for zk
    
    * Also, consume SOLR_WAIT_FOR_ZK in bin/solr.cmd
---
 solr/CHANGES.txt                                   |  3 ++
 solr/bin/solr.cmd                                  |  4 +++
 .../apache/solr/servlet/SolrDispatchFilter.java    |  2 +-
 .../test/org/apache/solr/cloud/ZkFailoverTest.java | 35 +++++++++++++++++-----
 4 files changed, 35 insertions(+), 9 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 3931744..f54a40e 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -245,6 +245,9 @@ Bug Fixes
 * SOLR-14768: Fix HTTP multipart POST to Solr -- a regression from 8.6.0.
   Many Jetty classes are not classpath-visible from the Solr webapp.  (David Smiley)
 
+* SOLR-14503: Use specified waitForZk value as connection timeout for zookeeper in SolrDispatcherFilter.
+  Also, consume specified SOLR_WAIT_FOR_ZK in bin/solr.cmd (Colvin Cowie via Munendra S N)
+
 Other Changes
 ---------------------
 
diff --git a/solr/bin/solr.cmd b/solr/bin/solr.cmd
index 9698deb..d003dab 100755
--- a/solr/bin/solr.cmd
+++ b/solr/bin/solr.cmd
@@ -1147,6 +1147,10 @@ IF "%SOLR_MODE%"=="solrcloud" (
 
   set "CLOUD_MODE_OPTS=-DzkClientTimeout=!ZK_CLIENT_TIMEOUT!"
 
+  IF NOT "%SOLR_WAIT_FOR_ZK%"=="" (
+    set "CLOUD_MODE_OPTS=!CLOUD_MODE_OPTS! -DwaitForZk=%SOLR_WAIT_FOR_ZK%"
+  )
+
   IF NOT "%ZK_HOST%"=="" (
     set "CLOUD_MODE_OPTS=!CLOUD_MODE_OPTS! -DzkHost=%ZK_HOST%"
   ) ELSE (
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index 5a5317b..c560208 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -296,7 +296,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
     if (!StringUtils.isEmpty(zkHost)) {
       int startUpZkTimeOut = Integer.getInteger("waitForZk", 30);
       startUpZkTimeOut *= 1000;
-      try (SolrZkClient zkClient = new SolrZkClient(zkHost, startUpZkTimeOut)) {
+      try (SolrZkClient zkClient = new SolrZkClient(zkHost, startUpZkTimeOut, startUpZkTimeOut)) {
         if (zkClient.exists("/solr.xml", true)) {
           log.info("solr.xml found in ZooKeeper. Loading...");
           byte[] data = zkClient.getData("/solr.xml", null, null, true);
diff --git a/solr/core/src/test/org/apache/solr/cloud/ZkFailoverTest.java b/solr/core/src/test/org/apache/solr/cloud/ZkFailoverTest.java
index 7c0250f..1f54199 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ZkFailoverTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ZkFailoverTest.java
@@ -25,28 +25,52 @@ import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.zookeeper.KeeperException;
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
 public class ZkFailoverTest extends SolrCloudTestCase {
+  private ZkTestServer zkTestServer;
 
   @BeforeClass
   public static void setupCluster() throws Exception {
-    System.setProperty("waitForZk", "60");
     useFactory("solr.StandardDirectoryFactory");
     configureCluster(2)
         .addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-dynamic").resolve("conf"))
         .configure();
   }
 
+  @AfterClass
+  public static void resetWaitForZk(){
+    System.setProperty("waitForZk", "30");
+  }
+
   public void testRestartZkWhenClusterDown() throws Exception {
     String coll = "coll1";
     CollectionAdminRequest.createCollection(coll, 2, 1).process(cluster.getSolrClient());
     cluster.waitForActiveCollection(coll, 2, 2);
     cluster.getSolrClient().add(coll, new SolrInputDocument("id", "1"));
+    zkTestServer = cluster.getZkServer();
+
+    // This attempt will fail since it will timeout after 1 second
+    System.setProperty("waitForZk", "1");
+    restartSolrAndZk();
+    waitForLiveNodes(0);
+
+    // This attempt will succeed since there will be enough time to connect
+    System.setProperty("waitForZk", "20");
+    restartSolrAndZk();
+    waitForLiveNodes(2);
+    waitForState("Timeout waiting for " + coll, coll, clusterShape(2, 2));
+    QueryResponse rsp = new QueryRequest(new SolrQuery("*:*")).process(cluster.getSolrClient(), coll);
+    assertEquals(1, rsp.getResults().getNumFound());
+    zkTestServer.shutdown();
+  }
+
+  private void restartSolrAndZk()
+      throws Exception {
     for (JettySolrRunner runner : cluster.getJettySolrRunners()) {
       runner.stop();
     }
-    ZkTestServer zkTestServer = cluster.getZkServer();
     zkTestServer.shutdown();
     Thread[] threads = new Thread[cluster.getJettySolrRunners().size()];
     for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
@@ -60,17 +84,12 @@ public class ZkFailoverTest extends SolrCloudTestCase {
         });
       threads[i].start();
     }
-    Thread.sleep(5000);
+    Thread.sleep(2500);
     zkTestServer = new ZkTestServer(zkTestServer.getZkDir(), zkTestServer.getPort());
     zkTestServer.run(false);
     for (Thread thread : threads) {
       thread.join();
     }
-    waitForLiveNodes(2);
-    waitForState("Timeout waiting for " + coll, coll, clusterShape(2, 2));
-    QueryResponse rsp = new QueryRequest(new SolrQuery("*:*")).process(cluster.getSolrClient(), coll);
-    assertEquals(1, rsp.getResults().getNumFound());
-    zkTestServer.shutdown();
   }
 
   private void waitForLiveNodes(int numNodes) throws InterruptedException, KeeperException {