You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by an...@apache.org on 2020/01/23 12:43:58 UTC

[zookeeper] branch master updated: ZOOKEEPER-3698: fixing NoRouteToHostException when starting large cluster locally

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

andor pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/zookeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new b4c5a7f  ZOOKEEPER-3698: fixing NoRouteToHostException when starting large cluster locally
b4c5a7f is described below

commit b4c5a7fdd0f97aa5a9bf18a345b1cee5b0da967a
Author: Mate Szalay-Beko <sz...@gmail.com>
AuthorDate: Thu Jan 23 13:43:39 2020 +0100

    ZOOKEEPER-3698: fixing NoRouteToHostException when starting large cluster locally
    
    When we tested RC 3.6.0, we had a problem of starting ZooKeeper cluster with large
    number (11+) of ensemble members locally on mac. We found exceptions in the logs
    when the new MultiAddress feature tries to filter the unreachable hosts from the
    address list. This involves the calling of the InetAddress.isReachable method with
    a default timeout of 500ms, which goes down to a native call in java and basically
    try to do a ping (an ICMP echo request) to the host. Naturally, the localhost should
    be always reachable.
    
    The problem was that on mac we have the ICMP rate limit set to 250 by default.
    
    In this patch we:
    - changed the reachability check behavior by disabling the check if there is only
    a single address provided (so we wouldn't be able to filter the unreachable
    addresses anyway).
    - added and documented a configuration parameter to disable the reachability check
    for testing. (default: enabled)
    - added and documented a configuration parameter to set the timeout for the
    reachability checks. (default: 1000ms)
    
    Author: Mate Szalay-Beko <sz...@gmail.com>
    
    Reviewers: eolivelli@apache.org, andor@apache.org
    
    Closes #1228 from symat/ZOOKEEPER-3698-branch-3.6
    
    (cherry picked from commit 8352f78e752a019387c1eb6095085152e7d1d4ad)
    Signed-off-by: Andor Molnar <an...@apache.org>
---
 .../src/main/resources/markdown/zookeeperAdmin.md  | 29 ++++++++++++++++++++++
 .../apache/zookeeper/server/quorum/Learner.java    |  7 +++++-
 .../zookeeper/server/quorum/MultipleAddresses.java | 18 +++++++++++++-
 .../zookeeper/server/quorum/QuorumCnxManager.java  | 26 +++++++++++--------
 .../apache/zookeeper/server/quorum/QuorumPeer.java | 22 ++++++++++++++++
 .../zookeeper/server/quorum/QuorumPeerConfig.java  | 21 ++++++++++++++++
 .../zookeeper/server/quorum/QuorumPeerMain.java    |  2 ++
 .../server/quorum/MultipleAddressesTest.java       | 29 ++++++++++++++++++++++
 .../server/quorum/QuorumPeerMainTest.java          |  2 +-
 9 files changed, 143 insertions(+), 13 deletions(-)

diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md
index 3fa57ae..6b6113f 100644
--- a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md
+++ b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md
@@ -1567,6 +1567,19 @@ and [SASL authentication for ZooKeeper](https://cwiki.apache.org/confluence/disp
     You can instruct ZooKeeper to remove the realm from the client principal name during authentication.
     (e.g. the zk/myhost@EXAMPLE.COM client principal will be authenticated in ZooKeeper as zk/myhost)
     Default: false
+
+* *multiAddress.reachabilityCheckTimeoutMs* :
+    (Java system property: **zookeeper.multiAddress.reachabilityCheckTimeoutMs**)
+    **New in 3.6.0:**
+    Since ZooKeeper 3.6.0 you can also [specify multiple addresses](#id_multi_address) 
+    for each ZooKeeper server instance (this can increase availability when multiple physical 
+    network interfaces can be used parallel in the cluster). ZooKeeper will perform ICMP ECHO requests
+    or try to establish a TCP connection on port 7 (Echo) of the destination host in order to find 
+    the reachable addresses. This happens only if you provide multiple addresses in the configuration.
+    In this property you can set the timeout in millisecs for the reachability check. The check happens 
+    in parallel for the different addresses, so the timeout you set here is the maximum time will be taken
+    by checking the reachability of all addresses.
+    The default value is **1000**.
     
 
 <a name="Experimental+Options%2FFeatures"></a>
@@ -1644,6 +1657,22 @@ the variable does.
     ZAB protocol and the Fast Leader Election protocol. Default
     value is **false**.
 
+* *multiAddress.reachabilityCheckEnabled* :
+    (Java system property: **zookeeper.multiAddress.reachabilityCheckEnabled**)
+    **New in 3.6.0:**
+    Since ZooKeeper 3.6.0 you can also [specify multiple addresses](#id_multi_address) 
+    for each ZooKeeper server instance (this can increase availability when multiple physical 
+    network interfaces can be used parallel in the cluster). ZooKeeper will perform ICMP ECHO requests
+    or try to establish a TCP connection on port 7 (Echo) of the destination host in order to find 
+    the reachable addresses. This happens only if you provide multiple addresses in the configuration.
+    The reachable check can fail if you hit some ICMP rate-limitation, (e.g. on MacOS) when you try to 
+    start a large (e.g. 11+) ensemble members cluster on a single machine for testing. 
+    
+    Default value is **true**. By setting this parameter to 'false' you can disable the reachability checks. 
+    Please note, disabling the reachability check will cause the cluster not to be able to reconfigure 
+    itself properly during network problems, so the disabling is advised only during testing. 
+
+
 <a name="Disabling+data+directory+autocreation"></a>
 
 #### Disabling data directory autocreation
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java
index ad9b0bb..2fd2db4 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java
@@ -266,7 +266,12 @@ public class Learner {
     protected void connectToLeader(MultipleAddresses multiAddr, String hostname) throws IOException {
 
         this.leaderAddr = multiAddr;
-        Set<InetSocketAddress> addresses = multiAddr.getAllReachableAddresses();
+        Set<InetSocketAddress> addresses;
+        if (self.isMultiAddressReachabilityCheckEnabled()) {
+            addresses = multiAddr.getAllReachableAddresses();
+        } else {
+            addresses = multiAddr.getAllAddresses();
+        }
         ExecutorService executor = Executors.newFixedThreadPool(addresses.size());
         CountDownLatch latch = new CountDownLatch(addresses.size());
         AtomicReference<Socket> socket = new AtomicReference<>(null);
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java
index 730b182..3ee63fa 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java
@@ -40,7 +40,7 @@ import java.util.stream.Collectors;
  * See ZOOKEEPER-3188 for a discussion of this feature.
  */
 public final class MultipleAddresses {
-    private static final Duration DEFAULT_TIMEOUT = Duration.ofMillis(500);
+    public static final Duration DEFAULT_TIMEOUT = Duration.ofMillis(1000);
 
     private static Set<InetSocketAddress> newConcurrentHashSet() {
         return Collections.newSetFromMap(new ConcurrentHashMap<>());
@@ -150,6 +150,12 @@ public final class MultipleAddresses {
      */
     public InetSocketAddress getReachableOrOne() {
         InetSocketAddress address;
+
+        // if there is only a single address provided then we don't do any reachability check
+        if (addresses.size() == 1) {
+            return getOne();
+        }
+
         try {
             address = getReachableAddress();
         } catch (NoRouteToHostException e) {
@@ -179,6 +185,16 @@ public final class MultipleAddresses {
         return addresses.iterator().next();
     }
 
+
+    /**
+     * Returns the number of addresses in the set.
+     *
+     * @return the number of addresses.
+     */
+    public int size() {
+        return addresses.size();
+    }
+
     private boolean checkIfAddressIsReachable(InetSocketAddress address) {
         if (address.isUnresolved()) {
             return false;
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
index 55c1dc2..a92fc52 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
@@ -35,6 +35,7 @@ import java.net.SocketTimeoutException;
 import java.nio.BufferUnderflowException;
 import java.nio.ByteBuffer;
 import java.nio.channels.UnresolvedAddressException;
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Enumeration;
@@ -534,7 +535,8 @@ public class QuorumCnxManager {
                 try {
                     InitialMessage init = InitialMessage.parse(protocolVersion, din);
                     sid = init.sid;
-                    electionAddr = new MultipleAddresses(init.electionAddr);
+                    electionAddr = new MultipleAddresses(init.electionAddr,
+                        Duration.ofMillis(self.getMultiAddressReachabilityCheckTimeoutMs()));
                 } catch (InitialMessage.InitialMessageException ex) {
                     LOG.error(ex.toString());
                     closeSocket(sock);
@@ -637,10 +639,12 @@ public class QuorumCnxManager {
     synchronized boolean connectOne(long sid, MultipleAddresses electionAddr) {
         if (senderWorkerMap.get(sid) != null) {
             LOG.debug("There is a connection already for server {}", sid);
-            // since ZOOKEEPER-3188 we can use multiple election addresses to reach a server. It is possible, that the
-            // one we are using is already dead and we need to clean-up, so when we will create a new connection
-            // then we will choose an other one, which is actually reachable
-            senderWorkerMap.get(sid).asyncValidateIfSocketIsStillReachable();
+            if (electionAddr.size() > 1 && self.isMultiAddressReachabilityCheckEnabled()) {
+                // since ZOOKEEPER-3188 we can use multiple election addresses to reach a server. It is possible, that the
+                // one we are using is already dead and we need to clean-up, so when we will create a new connection
+                // then we will choose an other one, which is actually reachable
+                senderWorkerMap.get(sid).asyncValidateIfSocketIsStillReachable();
+            }
             return true;
         }
 
@@ -653,7 +657,7 @@ public class QuorumCnxManager {
                 sock = new Socket();
             }
             setSockOpts(sock);
-            sock.connect(electionAddr.getReachableAddress(), cnxTO);
+            sock.connect(electionAddr.getReachableOrOne(), cnxTO);
             if (sock instanceof SSLSocket) {
                 SSLSocket sslSock = (SSLSocket) sock;
                 sslSock.startHandshake();
@@ -706,10 +710,12 @@ public class QuorumCnxManager {
     synchronized void connectOne(long sid) {
         if (senderWorkerMap.get(sid) != null) {
             LOG.debug("There is a connection already for server {}", sid);
-            // since ZOOKEEPER-3188 we can use multiple election addresses to reach a server. It is possible, that the
-            // one we are using is already dead and we need to clean-up, so when we will create a new connection
-            // then we will choose an other one, which is actually reachable
-            senderWorkerMap.get(sid).asyncValidateIfSocketIsStillReachable();
+            if (self.isMultiAddressReachabilityCheckEnabled()) {
+                // since ZOOKEEPER-3188 we can use multiple election addresses to reach a server. It is possible, that the
+                // one we are using is already dead and we need to clean-up, so when we will create a new connection
+                // then we will choose an other one, which is actually reachable
+                senderWorkerMap.get(sid).asyncValidateIfSocketIsStillReachable();
+            }
             return;
         }
         synchronized (self.QV_LOCK) {
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java
index c6375cd..977f719 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java
@@ -161,6 +161,28 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
         this.observerMasterPort = observerMasterPort;
     }
 
+    private int multiAddressReachabilityCheckTimeoutMs = (int) MultipleAddresses.DEFAULT_TIMEOUT.toMillis();
+
+    public int getMultiAddressReachabilityCheckTimeoutMs() {
+        return multiAddressReachabilityCheckTimeoutMs;
+    }
+
+    public void setMultiAddressReachabilityCheckTimeoutMs(int multiAddressReachabilityCheckTimeoutMs) {
+        this.multiAddressReachabilityCheckTimeoutMs = multiAddressReachabilityCheckTimeoutMs;
+        LOG.info("multiAddress.reachabilityCheckTimeoutMs set to {}", multiAddressReachabilityCheckTimeoutMs);
+    }
+
+    private boolean multiAddressReachabilityCheckEnabled = true;
+
+    public boolean isMultiAddressReachabilityCheckEnabled() {
+        return multiAddressReachabilityCheckEnabled;
+    }
+
+    public void setMultiAddressReachabilityCheckEnabled(boolean multiAddressReachabilityCheckEnabled) {
+        this.multiAddressReachabilityCheckEnabled = multiAddressReachabilityCheckEnabled;
+        LOG.info("multiAddress.reachabilityCheckEnabled set to {}", multiAddressReachabilityCheckEnabled);
+    }
+
     public static class QuorumServer {
 
         public MultipleAddresses addr = new MultipleAddresses();
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
index 99fc2ed..102d699 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
@@ -120,6 +120,15 @@ public class QuorumPeerConfig {
     protected String quorumServerLoginContext = QuorumAuth.QUORUM_SERVER_SASL_LOGIN_CONTEXT_DFAULT_VALUE;
     protected int quorumCnxnThreadsSize;
 
+    // multi address related configs
+    private boolean multiAddressReachabilityCheckEnabled =
+      Boolean.parseBoolean(System.getProperty("zookeeper.multiAddress.reachabilityCheckEnabled",
+                                              "true"));
+    private int multiAddressReachabilityCheckTimeoutMs =
+      Integer.parseInt(System.getProperty("zookeeper.multiAddress.reachabilityCheckTimeoutMs",
+                                          String.valueOf(MultipleAddresses.DEFAULT_TIMEOUT.toMillis())));
+
+
     /**
      * Minimum snapshot retain count.
      * @see org.apache.zookeeper.server.PurgeTxnLog#purge(File, File, int)
@@ -389,6 +398,10 @@ public class QuorumPeerConfig {
             } else if (key.startsWith("metricsProvider.")) {
                 String keyForMetricsProvider = key.substring(16);
                 metricsProviderConfiguration.put(keyForMetricsProvider, value);
+            } else if (key.equals("multiAddress.reachabilityCheckTimeoutMs")) {
+                multiAddressReachabilityCheckTimeoutMs = Integer.parseInt(value);
+            } else if (key.equals("multiAddress.reachabilityCheckEnabled")) {
+                multiAddressReachabilityCheckEnabled = Boolean.parseBoolean(value);
             } else {
                 System.setProperty("zookeeper." + key, value);
             }
@@ -926,6 +939,14 @@ public class QuorumPeerConfig {
         return quorumListenOnAllIPs;
     }
 
+    public boolean isMultiAddressReachabilityCheckEnabled() {
+        return multiAddressReachabilityCheckEnabled;
+    }
+
+    public int getMultiAddressReachabilityCheckTimeoutMs() {
+        return multiAddressReachabilityCheckTimeoutMs;
+    }
+
     public static boolean isStandaloneEnabled() {
         return standaloneEnabled;
     }
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java
index 054a47d..04aec05 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java
@@ -204,6 +204,8 @@ public class QuorumPeerMain {
             if (config.sslQuorumReloadCertFiles) {
                 quorumPeer.getX509Util().enableCertFileReloading();
             }
+            quorumPeer.setMultiAddressReachabilityCheckEnabled(config.isMultiAddressReachabilityCheckEnabled());
+            quorumPeer.setMultiAddressReachabilityCheckTimeoutMs(config.getMultiAddressReachabilityCheckTimeoutMs());
 
             // sets quorum sasl authentication configurations
             quorumPeer.setQuorumSaslEnabled(config.quorumEnableSasl);
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java
index 9920320..e8f08c6 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java
@@ -24,6 +24,7 @@ import java.net.NoRouteToHostException;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -109,6 +110,26 @@ public class MultipleAddressesTest {
     }
 
     @Test
+    public void testGetReachableOrOneWithSingleReachableAddress() {
+        InetSocketAddress reachableAddress = new InetSocketAddress("127.0.0.1", PortAssignment.unique());
+
+        MultipleAddresses multipleAddresses = new MultipleAddresses(Collections.singletonList(reachableAddress));
+        InetSocketAddress actualReturnedAddress = multipleAddresses.getReachableOrOne();
+
+        Assert.assertEquals(reachableAddress, actualReturnedAddress);
+    }
+
+    @Test
+    public void testGetReachableOrOneWithSingleUnreachableAddress() {
+        InetSocketAddress unreachableAddress = new InetSocketAddress("unreachable.address.zookeeper.apache.com", 1234);
+
+        MultipleAddresses multipleAddresses = new MultipleAddresses(Collections.singletonList(unreachableAddress));
+        InetSocketAddress actualReturnedAddress = multipleAddresses.getReachableOrOne();
+
+        Assert.assertEquals(unreachableAddress, actualReturnedAddress);
+    }
+
+    @Test
     public void testRecreateSocketAddresses() throws UnknownHostException {
         List<InetSocketAddress> searchedAddresses = Arrays.stream(InetAddress.getAllByName("google.com"))
                 .map(addr -> new InetSocketAddress(addr, 222)).collect(Collectors.toList());
@@ -180,6 +201,14 @@ public class MultipleAddressesTest {
         Assert.assertNotEquals(multipleAddresses, multipleAddressesNotEquals);
     }
 
+    @Test
+    public void testSize() {
+        List<InetSocketAddress> addresses = getAddressList();
+        MultipleAddresses multipleAddresses = new MultipleAddresses(addresses);
+
+        Assert.assertEquals(PORTS_AMOUNT, multipleAddresses.size());
+    }
+
     public List<Integer> getPortList() {
         return IntStream.range(0, PORTS_AMOUNT).mapToObj(i -> PortAssignment.unique()).collect(Collectors.toList());
     }
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
index d9666e1..beb3bb9 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
@@ -470,7 +470,7 @@ public class QuorumPeerMainTest extends QuorumPeerTestBase {
         LineNumberReader r = new LineNumberReader(new StringReader(os.toString()));
         String line;
         boolean found = false;
-        Pattern p = Pattern.compile(".*None of the addresses .* are reachable for sid 2");
+        Pattern p = Pattern.compile(".*Cannot open channel to .* at election address .*");
         while ((line = r.readLine()) != null) {
             found = p.matcher(line).matches();
             if (found) {