Browse Source

ZOOKEEPER-3188: Improve resilience to network

This PR is the rebase of the [previous pull request](https://github.com/apache/zookeeper/pull/730), so all the kudos should go to the original authors...

In [ZOOKEEPER-3188](https://issues.apache.org/jira/browse/ZOOKEEPER-3188) we add ability to specify several addresses for quorum operations. Also added reconnection attempts if connection to leader lost.

In this PR I rebased the changes on the current master, resolving some minor conflicts with:
- [ZOOKEEPER-3296](https://issues.apache.org/jira/browse/ZOOKEEPER-3296): Explicitly closing the sslsocket when it failed handshake to prevent issue where peers cannot join quorum
- [ZOOKEEPER-3320](https://issues.apache.org/jira/browse/ZOOKEEPER-3320): Leader election port stop listen when hostname unresolvable for some time
- [ZOOKEEPER-3385](https://issues.apache.org/jira/browse/ZOOKEEPER-3385): Add admin command to display leader
- [ZOOKEEPER-3386](https://issues.apache.org/jira/browse/ZOOKEEPER-3386): Add admin command to display voting view
- [ZOOKEEPER-3398](https://issues.apache.org/jira/browse/ZOOKEEPER-3398): Learner.connectToLeader() may take too long to time-out

I still want to test the feature manually (e.g. using docker containers with multiple virtual networks / interfaces). The steps to the manual test could be recorded in the [google docs](https://docs.google.com/document/d/1iGVwxeHp57qogwfdodCh9b32P2_kOQaJZ2GDo7j36fI/edit?usp=sharing) as well.

Also I think we could add a few more unit tests where we are using multiple addresses. The current tests are using a single address only.

Also the Zookeeper documentation needs to be changed (e.g. by a follow-up Jira?) to promote the new feature and the new config format (possibly including also the admin command documentation in relation with [ZOOKEEPER-3386](https://issues.apache.org/jira/browse/ZOOKEEPER-3386) and [ZOOKEEPER-3461](https://issues.apache.org/jira/browse/ZOOKEEPER-3461))

Author: Mate Szalay-Beko <szalay.beko.mate@gmail.com>
Author: Mate Szalay-Beko <mszalay@cloudera.com>

Reviewers: eolivelli@apache.org, andor@apache.org

Closes #1048 from symat/ZOOKEEPER-3188 and squashes the following commits:

3c6fc52d7 [Mate Szalay-Beko] Merge remote-tracking branch 'apache/master' into ZOOKEEPER-3188
356882d46 [Mate Szalay-Beko] ZOOKEEPER-3188: document new configuration format for using multiple addresses
45b6c0fdb [Mate Szalay-Beko] Merge remote-tracking branch 'apache/master' into ZOOKEEPER-3188
4b6bcea48 [Mate Szalay-Beko] ZOOKEEPER-3188: MultiAddress unit tests for Quorum TLS and Kerberos/Digest authentication
40bc44c23 [Mate Szalay-Beko] Merge remote-tracking branch 'apache/master' into ZOOKEEPER-3188
f875f5c67 [Mate Szalay-Beko] Merge remote-tracking branch 'apache/master' into ZOOKEEPER-3188
31805e79b [Mate Szalay-Beko] Merge remote-tracking branch 'apache/master' into ZOOKEEPER-3188
0f95678ca [Mate Szalay-Beko] ZOOKEEPER-3188: skip unreachable addresses when Learner connects to Leader
e232c55da [Mate Szalay-Beko] ZOOKEEPER-3188: fix flaky unit MultiAddress unit test
e892d8d8e [Mate Szalay-Beko] Merge remote-tracking branch 'apache/master' into ZOOKEEPER-3188
6f2ab755f [Mate Szalay-Beko] Merge remote-tracking branch 'apache/master' into ZOOKEEPER-3188
2eedf2687 [Mate Szalay-Beko] ZOOKEEPER-3188: fix PR commits; handle case when Leader can not bind to port on startup
483d2fc9c [Mate Szalay-Beko] Merge remote-tracking branch 'apache/master' into ZOOKEEPER-3188
a5d6bcb97 [Mate Szalay-Beko] ZOOKEEPER-3188: support for dynamic reconfig + add more unit tests
ed31d2ce9 [Mate Szalay-Beko] ZOOKEEPER-3188: better shutdown for executors (following PR comments)
8713a5bbf [Mate Szalay-Beko] ZOOKEEPER-3188: add fixes for PR comments
05eae83c0 [Mate Szalay-Beko] Merge remote-tracking branch 'apache/master' into ZOOKEEPER-3188
e823af4b1 [Mate Szalay-Beko] Merge remote-tracking branch 'origin/master' into ZOOKEEPER-3188
de7bad2a6 [Mate Szalay-Beko] Merge remote-tracking branch 'origin/master' into ZOOKEEPER-3188
da98a8da6 [Mate Szalay-Beko] ZOOKEEPER-3188: fix JDK-13 warning
5bd1f4e2c [Mate Szalay-Beko] ZOOKEEPER-3188: supress spotbugs warning
42a52a688 [Mate Szalay-Beko] ZOOKEEPER-3188: improve based on code review comments
6c4220a0d [Mate Szalay-Beko] ZOOKEEPER-3188: fix SendWorker.asyncValidateIfSocketIsStillReachable
5b22432c1 [Mate Szalay-Beko] ZOOKEEPER-3188: fix LeaderElection to work with multiple election addresses
7bfbe7e5c [Mate Szalay-Beko] ZOOKEEPER-3188: Improve resilience to network
Mate Szalay-Beko 5 năm trước cách đây
mục cha
commit
815c8f2130
33 tập tin đã thay đổi với 2159 bổ sung508 xóa
  1. 23 2
      zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md
  2. 21 0
      zookeeper-docs/src/main/resources/markdown/zookeeperReconfig.md
  3. 5 0
      zookeeper-docs/src/main/resources/markdown/zookeeperStarted.md
  4. 4 2
      zookeeper-server/src/main/java/org/apache/zookeeper/server/ObserverBean.java
  5. 43 30
      zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/Commands.java
  6. 6 3
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java
  7. 154 81
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java
  8. 136 61
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java
  9. 6 2
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LocalPeerBean.java
  10. 225 0
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java
  11. 4 7
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverMaster.java
  12. 285 136
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java
  13. 106 85
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java
  14. 8 3
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java
  15. 6 2
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java
  16. 7 2
      zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/RemotePeerBean.java
  17. 6 0
      zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/CommandsTest.java
  18. 48 18
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CnxManagerTest.java
  19. 105 19
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerTest.java
  20. 201 0
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java
  21. 303 0
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainMultiAddressTest.java
  22. 1 2
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
  23. 73 3
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumSSLTest.java
  24. 166 0
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumServerConfigBuilder.java
  25. 2 2
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigFailureCasesTest.java
  26. 20 9
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthTestBase.java
  27. 22 1
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumDigestAuthTest.java
  28. 26 3
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java
  29. 22 0
      zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java
  30. 1 1
      zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumUtil.java
  31. 2 2
      zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigExceptionTest.java
  32. 2 2
      zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigMisconfigTest.java
  33. 120 30
      zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigTest.java

+ 23 - 2
zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md

@@ -202,7 +202,13 @@ ensemble:
   though about a few here:
   Every machine that is part of the ZooKeeper ensemble should know
   about every other machine in the ensemble. You accomplish this with
-  the series of lines of the form **server.id=host:port:port**. The parameters **host** and **port** are straightforward. You attribute the
+  the series of lines of the form **server.id=host:port:port**. 
+  (The parameters **host** and **port** are straightforward, for each server 
+  you need to specify first a Quorum port then a dedicated port for ZooKeeper leader
+  election). 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).
+  You attribute the
   server id to each machine by creating a file named
   *myid*, one for each server, which resides in
   that server's data directory, as specified by the configuration file
@@ -1050,7 +1056,7 @@ of servers -- that is, when deploying clusters of servers.
     >Turning on leader selection is highly recommended when
     you have more than three ZooKeeper servers in an ensemble.
 
-* *server.x=[hostname]:nnnnn[:nnnnn], etc* :
+* *server.x=[hostname]:nnnnn[:nnnnn] etc* :
     (No Java system property)
     servers making up the ZooKeeper ensemble. When the server
     starts up, it determines which server it is by looking for the
@@ -1065,6 +1071,21 @@ of servers -- that is, when deploying clusters of servers.
     The first followers use to connect to the leader, and the second is for
     leader election. If you want to test multiple servers on a single machine, then
     different ports can be used for each server.
+    
+
+    <a name="id_multi_address"></a>
+    Since ZooKeeper 3.6.0 it is possible to specify **multiple addresses** for each
+    ZooKeeper server (see [ZOOKEEPER-3188](https://issues.apache.org/jira/projects/ZOOKEEPER/issues/ZOOKEEPER-3188)).
+    This helps to increase availability and adds network level 
+    resiliency to ZooKeeper. When multiple physical network interfaces are used 
+    for the servers, ZooKeeper is able to bind on all interfaces and runtime switching 
+    to a working interface in case a network error. The different addresses can be specified
+    in the config using a pipe ('|') character. A valid configuration using multiple addresses looks like:
+
+        server.1=zoo1-net1:2888:3888|zoo1-net2:2889:3889
+        server.2=zoo2-net1:2888:3888|zoo2-net2:2889:3889
+        server.3=zoo3-net1:2888:3888|zoo3-net2:2889:3889
+       
 
 * *syncLimit* :
     (No Java system property)

+ 21 - 0
zookeeper-docs/src/main/resources/markdown/zookeeperReconfig.md

@@ -19,6 +19,7 @@ limitations under the License.
 * [Overview](#ch_reconfig_intro)
 * [Changes to Configuration Format](#ch_reconfig_format)
     * [Specifying the client port](#sc_reconfig_clientport)
+    * [Specifying multiple server addresses](#sc_multiaddress)
     * [The standaloneEnabled flag](#sc_reconfig_standaloneEnabled)
     * [The reconfigEnabled flag](#sc_reconfig_reconfigEnabled)
     * [Dynamic configuration file](#sc_reconfig_file)
@@ -109,6 +110,26 @@ Examples of legal server statements:
     server.5 = 125.23.63.23:1234:1235;125.23.63.24:1236
     server.5 = 125.23.63.23:1234:1235:participant;125.23.63.23:1236
 
+
+<a name="sc_multiaddress"></a>
+
+### Specifying multiple server addresses
+
+Since ZooKeeper 3.6.0 it is possible to specify multiple addresses for each
+ZooKeeper server (see [ZOOKEEPER-3188](https://issues.apache.org/jira/projects/ZOOKEEPER/issues/ZOOKEEPER-3188)).
+This helps to increase availability and adds network level 
+resiliency to ZooKeeper. When multiple physical network interfaces are used 
+for the servers, ZooKeeper is able to bind on all interfaces and runtime switching 
+to a working interface in case a network error. The different addresses can be 
+specified in the config using a pipe ('|') character. 
+
+Examples for a valid configurations using multiple addresses:
+
+    server.2=zoo2-net1:2888:3888|zoo2-net2:2889:3889;2188
+    server.2=zoo2-net1:2888:3888|zoo2-net2:2889:3889|zoo2-net3:2890:3890;2188
+    server.2=zoo2-net1:2888:3888|zoo2-net2:2889:3889;zoo2-net1:2188
+    server.2=zoo2-net1:2888:3888:observer|zoo2-net2:2889:3889:observer;2188
+
 <a name="sc_reconfig_standaloneEnabled"></a>
 
 ### The _standaloneEnabled_ flag

+ 5 - 0
zookeeper-docs/src/main/resources/markdown/zookeeperStarted.md

@@ -355,6 +355,11 @@ server have its own machine. It must be a completely separate
 physical server. Multiple virtual machines on the same physical
 host are still vulnerable to the complete failure of that host.
 
+>If you have multiple network interfaces in your ZooKeeper machines,
+you can also instruct ZooKeeper to bind on all of your interfaces and
+automatically switch to a healthy interface in case of a network failure.
+For details, see the [Configuration Parameters](zookeeperAdmin.html#id_multi_address).
+
 <a name="other-optimizations"></a>
 
 ### Other Optimizations

+ 4 - 2
zookeeper-server/src/main/java/org/apache/zookeeper/server/ObserverBean.java

@@ -18,6 +18,7 @@
 
 package org.apache.zookeeper.server;
 
+import java.net.InetSocketAddress;
 import org.apache.zookeeper.server.quorum.Observer;
 import org.apache.zookeeper.server.quorum.ObserverMXBean;
 import org.apache.zookeeper.server.quorum.QuorumPeer;
@@ -49,10 +50,11 @@ public class ObserverBean extends ZooKeeperServerBean implements ObserverMXBean
 
     public String getLearnerMaster() {
         QuorumPeer.QuorumServer learnerMaster = observer.getCurrentLearnerMaster();
-        if (learnerMaster == null || learnerMaster.addr == null) {
+        if (learnerMaster == null || learnerMaster.addr.isEmpty()) {
             return "Unknown";
         }
-        return learnerMaster.addr.getAddress().getHostAddress() + ":" + learnerMaster.addr.getPort();
+        InetSocketAddress address = learnerMaster.addr.getReachableOrOne();
+        return address.getAddress().getHostAddress() + ":" + address.getPort();
     }
 
     public void setLearnerMaster(String learnerMaster) {

+ 43 - 30
zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/Commands.java

@@ -18,7 +18,9 @@
 
 package org.apache.zookeeper.server.admin;
 
-import com.fasterxml.jackson.annotation.JsonAnyGetter;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.net.InetSocketAddress;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
@@ -43,7 +45,9 @@ import org.apache.zookeeper.server.quorum.Follower;
 import org.apache.zookeeper.server.quorum.FollowerZooKeeperServer;
 import org.apache.zookeeper.server.quorum.Leader;
 import org.apache.zookeeper.server.quorum.LeaderZooKeeperServer;
+import org.apache.zookeeper.server.quorum.MultipleAddresses;
 import org.apache.zookeeper.server.quorum.QuorumPeer;
+import org.apache.zookeeper.server.quorum.QuorumPeer.LearnerType;
 import org.apache.zookeeper.server.quorum.QuorumZooKeeperServer;
 import org.apache.zookeeper.server.quorum.ReadOnlyZooKeeperServer;
 import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
@@ -673,7 +677,8 @@ public class Commands {
             CommandResponse response = initializeResponse();
             if (zkServer instanceof QuorumZooKeeperServer) {
                 QuorumPeer peer = ((QuorumZooKeeperServer) zkServer).self;
-                VotingView votingView = new VotingView(peer.getVotingView());
+                Map<Long, QuorumServerView> votingView = peer.getVotingView().entrySet().stream()
+                        .collect(Collectors.toMap(Map.Entry::getKey, e -> new QuorumServerView(e.getValue())));
                 response.put("current_config", votingView);
             } else {
                 response.put("current_config", Collections.emptyMap());
@@ -681,38 +686,45 @@ public class Commands {
             return response;
         }
 
-        private static class VotingView {
-
-            private final Map<Long, String> view;
-
-            VotingView(Map<Long, QuorumPeer.QuorumServer> view) {
-                this.view = view.entrySet()
-                                .stream()
-                                .filter(e -> e.getValue().addr != null)
-                                .collect(Collectors.toMap(
-                                    Map.Entry::getKey,
-                                    e -> String.format(
-                                        "%s:%d%s:%s%s",
-                                        QuorumPeer.QuorumServer.delimitedHostString(e.getValue().addr),
-                                        e.getValue().addr.getPort(),
-                                        e.getValue().electionAddr == null ? "" : ":" + e.getValue().electionAddr.getPort(),
-                                        e.getValue().type.equals(QuorumPeer.LearnerType.PARTICIPANT) ? "participant" : "observer",
-                                        e.getValue().clientAddr == null || e.getValue().isClientAddrFromStatic
-                                            ? ""
-                                            : String.format(
-                                                ";%s:%d",
-                                                QuorumPeer.QuorumServer.delimitedHostString(e.getValue().clientAddr),
-                                                e.getValue().clientAddr.getPort())),
-                                    (v1, v2) -> v1, // cannot get duplicates as this straight draws from the other map
-                                    TreeMap::new));
+        @SuppressFBWarnings(value = "URF_UNREAD_FIELD", justification = "class is used only for JSON serialization")
+        private static class QuorumServerView {
+
+            @JsonProperty
+            private List<String> serverAddresses;
+
+            @JsonProperty
+            private List<String> electionAddresses;
+
+            @JsonProperty
+            private String clientAddress;
+
+            @JsonProperty
+            private String learnerType;
+
+            public QuorumServerView(QuorumPeer.QuorumServer quorumServer) {
+                this.serverAddresses = getMultiAddressString(quorumServer.addr);
+                this.electionAddresses = getMultiAddressString(quorumServer.electionAddr);
+                this.learnerType = quorumServer.type.equals(LearnerType.PARTICIPANT) ? "participant" : "observer";
+                this.clientAddress = getAddressString(quorumServer.clientAddr);
             }
 
-            @JsonAnyGetter
-            public Map<Long, String> getView() {
-                return view;
+            private static List<String> getMultiAddressString(MultipleAddresses multipleAddresses) {
+                if (multipleAddresses == null) {
+                    return Collections.emptyList();
+                }
+
+                return multipleAddresses.getAllAddresses().stream()
+                        .map(QuorumServerView::getAddressString)
+                        .collect(Collectors.toList());
             }
 
-        }
+            private static String getAddressString(InetSocketAddress address) {
+                if (address == null) {
+                    return "";
+                }
+                return String.format("%s:%d", QuorumPeer.QuorumServer.delimitedHostString(address), address.getPort());
+            }
+       }
 
     }
 
@@ -720,6 +732,7 @@ public class Commands {
      * Watch information aggregated by session. Returned Map contains:
      *   - "session_id_to_watched_paths": Map&lt;Long, Set&lt;String&gt;&gt; session ID -&gt; watched paths
      * @see DataTree#getWatches()
+     * @see DataTree#getWatches()
      */
     public static class WatchCommand extends CommandBase {
 

+ 6 - 3
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java

@@ -22,6 +22,7 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import java.io.IOException;
 import java.net.DatagramPacket;
 import java.net.DatagramSocket;
+import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.SocketException;
 import java.nio.ByteBuffer;
@@ -712,7 +713,8 @@ public class AuthFastLeaderElection implements Election {
             }
 
             for (QuorumServer server : self.getVotingView().values()) {
-                InetSocketAddress saddr = new InetSocketAddress(server.addr.getAddress(), port);
+                InetAddress address = server.addr.getReachableOrOne().getAddress();
+                InetSocketAddress saddr = new InetSocketAddress(address, port);
                 addrChallengeMap.put(saddr, new ConcurrentHashMap<Long, Long>());
             }
 
@@ -740,7 +742,7 @@ public class AuthFastLeaderElection implements Election {
 
     private void starter(QuorumPeer self) {
         this.self = self;
-        port = self.getVotingView().get(self.getId()).electionAddr.getPort();
+        port = self.getVotingView().get(self.getId()).electionAddr.getAllPorts().get(0);
         proposedLeader = -1;
         proposedZxid = -1;
 
@@ -763,6 +765,7 @@ public class AuthFastLeaderElection implements Election {
     private void sendNotifications() {
         for (QuorumServer server : self.getView().values()) {
 
+            InetSocketAddress address = self.getView().get(server.id).electionAddr.getReachableOrOne();
             ToSend notmsg = new ToSend(
                 ToSend.mType.notification,
                 AuthFastLeaderElection.sequencer++,
@@ -770,7 +773,7 @@ public class AuthFastLeaderElection implements Election {
                 proposedZxid,
                 logicalclock.get(),
                 QuorumPeer.ServerState.LOOKING,
-                self.getView().get(server.id).electionAddr);
+                address);
 
             sendqueue.offer(notmsg);
         }

+ 154 - 81
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java

@@ -24,7 +24,7 @@ import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
-import java.net.BindException;
+import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.net.Socket;
 import java.net.SocketAddress;
@@ -35,13 +35,22 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
 import javax.security.sasl.SaslException;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooDefs.OpCode;
@@ -268,42 +277,47 @@ public class Leader extends LearnerMaster {
         return qv.containsQuorum(ids);
     }
 
-    private final ServerSocket ss;
+    private final List<ServerSocket> serverSockets = new LinkedList<>();
 
     Leader(QuorumPeer self, LeaderZooKeeperServer zk) throws IOException {
         this.self = self;
         this.proposalStats = new BufferStats();
+
+        Set<InetSocketAddress> addresses;
+        if (self.getQuorumListenOnAllIPs()) {
+            addresses = self.getQuorumAddress().getWildcardAddresses();
+        } else {
+            addresses = self.getQuorumAddress().getAllAddresses();
+        }
+
+        addresses.stream()
+          .map(address -> createServerSocket(address, self.shouldUsePortUnification(), self.isSslQuorum()))
+          .filter(Optional::isPresent)
+          .map(Optional::get)
+          .forEach(serverSockets::add);
+
+        if (serverSockets.isEmpty()) {
+            throw new IOException("Leader failed to initialize any of the following sockets: " + addresses);
+        }
+
+        this.zk = zk;
+    }
+
+    Optional<ServerSocket> createServerSocket(InetSocketAddress address, boolean portUnification, boolean sslQuorum) {
+        ServerSocket serverSocket;
         try {
-            if (self.shouldUsePortUnification() || self.isSslQuorum()) {
-                boolean allowInsecureConnection = self.shouldUsePortUnification();
-                if (self.getQuorumListenOnAllIPs()) {
-                    ss = new UnifiedServerSocket(
-                        self.getX509Util(),
-                        allowInsecureConnection,
-                        self.getQuorumAddress().getPort());
-                } else {
-                    ss = new UnifiedServerSocket(self.getX509Util(), allowInsecureConnection);
-                }
-            } else {
-                if (self.getQuorumListenOnAllIPs()) {
-                    ss = new ServerSocket(self.getQuorumAddress().getPort());
-                } else {
-                    ss = new ServerSocket();
-                }
-            }
-            ss.setReuseAddress(true);
-            if (!self.getQuorumListenOnAllIPs()) {
-                ss.bind(self.getQuorumAddress());
-            }
-        } catch (BindException e) {
-            if (self.getQuorumListenOnAllIPs()) {
-                LOG.error("Couldn't bind to port {}", self.getQuorumAddress().getPort(), e);
+            if (portUnification || sslQuorum) {
+                serverSocket = new UnifiedServerSocket(self.getX509Util(), portUnification);
             } else {
-                LOG.error("Couldn't bind to {}", self.getQuorumAddress(), e);
+                serverSocket = new ServerSocket();
             }
-            throw e;
+            serverSocket.setReuseAddress(true);
+            serverSocket.bind(address);
+            return Optional.of(serverSocket);
+        } catch (IOException e) {
+            LOG.error("Couldn't bind to {}", address.toString(), e);
         }
-        this.zk = zk;
+        return Optional.empty();
     }
 
     /**
@@ -418,66 +432,116 @@ public class Leader extends LearnerMaster {
 
     class LearnerCnxAcceptor extends ZooKeeperCriticalThread {
 
-        private volatile boolean stop = false;
+        private final AtomicBoolean stop = new AtomicBoolean(false);
+        private final AtomicBoolean fail = new AtomicBoolean(false);
 
-        public LearnerCnxAcceptor() {
-            super("LearnerCnxAcceptor-" + ss.getLocalSocketAddress(), zk.getZooKeeperServerListener());
+        LearnerCnxAcceptor() {
+            super("LearnerCnxAcceptor-" + serverSockets.stream()
+                      .map(ServerSocket::getLocalSocketAddress)
+                      .map(Objects::toString)
+                      .collect(Collectors.joining("|")),
+                  zk.getZooKeeperServerListener());
         }
 
         @Override
         public void run() {
-            try {
-                while (!stop) {
-                    Socket s = null;
-                    boolean error = false;
+            if (!stop.get() && !serverSockets.isEmpty()) {
+                ExecutorService executor = Executors.newFixedThreadPool(serverSockets.size());
+                CountDownLatch latch = new CountDownLatch(serverSockets.size());
+
+                serverSockets.forEach(serverSocket ->
+                        executor.submit(new LearnerCnxAcceptorHandler(serverSocket, latch)));
+
+                try {
+                    latch.await();
+                } catch (InterruptedException ie) {
+                    LOG.error("Interrupted while sleeping in LearnerCnxAcceptor.", ie);
+                } finally {
+                    closeSockets();
+                    executor.shutdown();
                     try {
-                        s = ss.accept();
-
-                        // start with the initLimit, once the ack is processed
-                        // in LearnerHandler switch to the syncLimit
-                        s.setSoTimeout(self.tickTime * self.initLimit);
-                        s.setTcpNoDelay(nodelay);
-
-                        BufferedInputStream is = new BufferedInputStream(s.getInputStream());
-                        LearnerHandler fh = new LearnerHandler(s, is, Leader.this);
-                        fh.start();
-                    } catch (SocketException e) {
-                        error = true;
-                        if (stop) {
-                            LOG.warn("exception while shutting down acceptor.", e);
-
-                            // When Leader.shutdown() calls ss.close(),
-                            // the call to accept throws an exception.
-                            // We catch and set stop to true.
-                            stop = true;
-                        } else {
-                            throw e;
-                        }
-                    } catch (SaslException e) {
-                        LOG.error("Exception while connecting to quorum learner", e);
-                        error = true;
-                    } catch (Exception e) {
-                        error = true;
-                        throw e;
-                    } finally {
-                        // Don't leak sockets on errors
-                        if (error && s != null && !s.isClosed()) {
-                            try {
-                                s.close();
-                            } catch (IOException e) {
-                                LOG.warn("Error closing socket", e);
-                            }
+                        if (!executor.awaitTermination(1, TimeUnit.SECONDS)) {
+                            LOG.error("not all the LearnerCnxAcceptorHandler terminated properly");
                         }
+                    } catch (InterruptedException ie) {
+                        LOG.error("Interrupted while terminating LearnerCnxAcceptor.", ie);
                     }
                 }
-            } catch (Exception e) {
-                LOG.warn("Exception while accepting follower", e);
-                handleException(this.getName(), e);
             }
         }
 
         public void halt() {
-            stop = true;
+            stop.set(true);
+            closeSockets();
+        }
+
+        class LearnerCnxAcceptorHandler implements Runnable {
+            private ServerSocket serverSocket;
+            private CountDownLatch latch;
+
+            LearnerCnxAcceptorHandler(ServerSocket serverSocket, CountDownLatch latch) {
+                this.serverSocket = serverSocket;
+                this.latch = latch;
+            }
+
+            @Override
+            public void run() {
+                try {
+                    Thread.currentThread().setName("LearnerCnxAcceptorHandler-" + serverSocket.getLocalSocketAddress());
+
+                    while (!stop.get()) {
+                        acceptConnections();
+                    }
+                } catch (Exception e) {
+                    LOG.warn("Exception while accepting follower", e);
+                    if (fail.compareAndSet(false, true)) {
+                        handleException(getName(), e);
+                        halt();
+                    }
+                } finally {
+                    latch.countDown();
+                }
+            }
+
+            private void acceptConnections() throws IOException {
+                Socket socket = null;
+                boolean error = false;
+                try {
+                    socket = serverSocket.accept();
+
+                    // start with the initLimit, once the ack is processed
+                    // in LearnerHandler switch to the syncLimit
+                    socket.setSoTimeout(self.tickTime * self.initLimit);
+                    socket.setTcpNoDelay(nodelay);
+
+                    BufferedInputStream is = new BufferedInputStream(socket.getInputStream());
+                    LearnerHandler fh = new LearnerHandler(socket, is, Leader.this);
+                    fh.start();
+                } catch (SocketException e) {
+                    error = true;
+                    if (stop.get()) {
+                        LOG.warn("Exception while shutting down acceptor.", e);
+                    } else {
+                        throw e;
+                    }
+                } catch (SaslException e) {
+                    LOG.error("Exception while connecting to quorum learner", e);
+                    error = true;
+                } catch (Exception e) {
+                    error = true;
+                    throw e;
+                } finally {
+                    // Don't leak sockets on errors
+                    if (error && socket != null && !socket.isClosed()) {
+                        try {
+                            socket.close();
+                        } catch (IOException e) {
+                            LOG.warn("Error closing socket: " + socket, e);
+                        }
+                    }
+                }
+            }
+
         }
 
     }
@@ -735,16 +799,13 @@ public class Leader extends LearnerMaster {
 
         if (cnxAcceptor != null) {
             cnxAcceptor.halt();
+        } else {
+            closeSockets();
         }
 
         // NIO should not accept conenctions
         self.setZooKeeperServer(null);
         self.adminServer.setZooKeeperServer(null);
-        try {
-            ss.close();
-        } catch (IOException e) {
-            LOG.warn("Ignoring unexpected exception during close", e);
-        }
         self.closeAllConnections();
         // shutdown the previous zk
         if (zk != null) {
@@ -760,6 +821,18 @@ public class Leader extends LearnerMaster {
         isShutdown = true;
     }
 
+    synchronized void closeSockets() {
+       for (ServerSocket serverSocket : serverSockets) {
+           if (!serverSocket.isClosed()) {
+               try {
+                   serverSocket.close();
+               } catch (IOException e) {
+                   LOG.warn("Ignoring unexpected exception during close {}", serverSocket, e);
+               }
+           }
+       }
+    }
+
     /** In a reconfig operation, this method attempts to find the best leader for next configuration.
      *  If the current leader is a voter in the next configuartion, then it remains the leader.
      *  Otherwise, choose one of the new voters that acked the reconfiguartion, such that it is as

+ 136 - 61
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java

@@ -32,7 +32,13 @@ import java.util.ArrayDeque;
 import java.util.Deque;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 import javax.net.ssl.SSLSocket;
 import org.apache.jute.BinaryInputArchive;
 import org.apache.jute.BinaryOutputArchive;
@@ -76,7 +82,7 @@ public class Learner {
     protected BufferedOutputStream bufferedOutput;
 
     protected Socket sock;
-    protected InetSocketAddress leaderAddr;
+    protected MultipleAddresses leaderAddr;
 
     /**
      * Socket getter
@@ -250,73 +256,38 @@ public class Learner {
      * Establish a connection with the LearnerMaster found by findLearnerMaster.
      * Followers only connect to Leaders, Observers can connect to any active LearnerMaster.
      * Retries until either initLimit time has elapsed or 5 tries have happened.
-     * @param addr - the address of the Peer to connect to.
+     * @param multiAddr - the address of the Peer to connect to.
      * @throws IOException - if the socket connection fails on the 5th attempt
      * if there is an authentication failure while connecting to leader
-     * @throws X509Exception
-     * @throws InterruptedException
      */
-    protected void connectToLeader(InetSocketAddress addr, String hostname) throws IOException, InterruptedException, X509Exception {
-        this.sock = createSocket();
-        this.leaderAddr = addr;
-
-        // leader connection timeout defaults to tickTime * initLimit
-        int connectTimeout = self.tickTime * self.initLimit;
-
-        // but if connectToLearnerMasterLimit is specified, use that value to calculate
-        // timeout instead of using the initLimit value
-        if (self.connectToLearnerMasterLimit > 0) {
-            connectTimeout = self.tickTime * self.connectToLearnerMasterLimit;
-        }
+    protected void connectToLeader(MultipleAddresses multiAddr, String hostname) throws IOException {
 
-        int remainingTimeout;
-        long startNanoTime = nanoTime();
+        this.leaderAddr = multiAddr;
+        Set<InetSocketAddress> addresses = multiAddr.getAllReachableAddresses();
+        ExecutorService executor = Executors.newFixedThreadPool(addresses.size());
+        CountDownLatch latch = new CountDownLatch(addresses.size());
+        AtomicReference<Socket> socket = new AtomicReference<>(null);
+        addresses.stream().map(address -> new LeaderConnector(address, socket, latch)).forEach(executor::submit);
 
-        for (int tries = 0; tries < 5; tries++) {
+        try {
+            latch.await();
+        } catch (InterruptedException e) {
+            LOG.warn("Interrupted while trying to connect to Leader", e);
+        } finally {
+            executor.shutdown();
             try {
-                // recalculate the init limit time because retries sleep for 1000 milliseconds
-                remainingTimeout = connectTimeout - (int) ((nanoTime() - startNanoTime) / 1000000);
-                if (remainingTimeout <= 0) {
-                    LOG.error("connectToLeader exceeded on retries.");
-                    throw new IOException("connectToLeader exceeded on retries.");
-                }
-
-                sockConnect(sock, addr, Math.min(connectTimeout, remainingTimeout));
-                if (self.isSslQuorum()) {
-                    ((SSLSocket) sock).startHandshake();
-                }
-                sock.setTcpNoDelay(nodelay);
-                break;
-            } catch (IOException e) {
-                remainingTimeout = connectTimeout - (int) ((nanoTime() - startNanoTime) / 1000000);
-
-                if (remainingTimeout <= 1000) {
-                    LOG.error(
-                        "Unexpected exception, connectToLeader exceeded. tries={}, remaining init limit={}, connecting to {}",
-                        tries,
-                        remainingTimeout,
-                        addr,
-                        e);
-                    throw e;
-                } else if (tries >= 4) {
-                    LOG.error(
-                        "Unexpected exception, retries exceeded. tries={}, remaining init limit={}, connecting to {}",
-                        tries,
-                        remainingTimeout,
-                        addr,
-                        e);
-                    throw e;
-                } else {
-                    LOG.warn(
-                        "Unexpected exception, tries={}, remaining init limit={}, connecting to {}",
-                        tries,
-                        remainingTimeout,
-                        addr,
-                        e);
-                    this.sock = createSocket();
+                if (!executor.awaitTermination(1, TimeUnit.SECONDS)) {
+                    LOG.error("not all the LeaderConnector terminated properly");
                 }
+            } catch (InterruptedException ie) {
+                LOG.error("Interrupted while terminating LeaderConnector executor.", ie);
             }
-            Thread.sleep(leaderConnectDelayDuringRetryMs);
+        }
+
+        if (socket.get() == null) {
+            throw new IOException("Failed connect to " + multiAddr);
+        } else {
+            sock = socket.get();
         }
 
         self.authLearner.authenticate(sock, hostname);
@@ -326,7 +297,111 @@ public class Learner {
         leaderOs = BinaryOutputArchive.getArchive(bufferedOutput);
     }
 
-    private Socket createSocket() throws X509Exception, IOException {
+    class LeaderConnector implements Runnable {
+
+        private AtomicReference<Socket> socket;
+        private InetSocketAddress address;
+        private CountDownLatch latch;
+
+        LeaderConnector(InetSocketAddress address, AtomicReference<Socket> socket, CountDownLatch latch) {
+            this.address = address;
+            this.socket = socket;
+            this.latch = latch;
+        }
+
+        @Override
+        public void run() {
+            try {
+                Thread.currentThread().setName("LeaderConnector-" + address);
+                Socket sock = connectToLeader();
+
+                if (sock != null && sock.isConnected()) {
+                    if (socket.compareAndSet(null, sock)) {
+                        LOG.info("Successfully connected to leader, using address: {}", address);
+                    } else {
+                        LOG.info("Connection to the leader is already established, close the redundant connection");
+                        sock.close();
+                    }
+                }
+
+            } catch (Exception e) {
+                LOG.error("Failed connect to {}", address, e);
+            } finally {
+                latch.countDown();
+            }
+        }
+
+        private Socket connectToLeader() throws IOException, X509Exception, InterruptedException {
+            Socket sock = createSocket();
+
+            // leader connection timeout defaults to tickTime * initLimit
+            int connectTimeout = self.tickTime * self.initLimit;
+
+            // but if connectToLearnerMasterLimit is specified, use that value to calculate
+            // timeout instead of using the initLimit value
+            if (self.connectToLearnerMasterLimit > 0) {
+                connectTimeout = self.tickTime * self.connectToLearnerMasterLimit;
+            }
+
+            int remainingTimeout;
+            long startNanoTime = nanoTime();
+
+            for (int tries = 0; tries < 5 && socket.get() == null; tries++) {
+                try {
+                    // recalculate the init limit time because retries sleep for 1000 milliseconds
+                    remainingTimeout = connectTimeout - (int) ((nanoTime() - startNanoTime) / 1_000_000);
+                    if (remainingTimeout <= 0) {
+                        LOG.error("connectToLeader exceeded on retries.");
+                        throw new IOException("connectToLeader exceeded on retries.");
+                    }
+
+                    sockConnect(sock, address, Math.min(connectTimeout, remainingTimeout));
+                    if (self.isSslQuorum()) {
+                        ((SSLSocket) sock).startHandshake();
+                    }
+                    sock.setTcpNoDelay(nodelay);
+                    break;
+                } catch (IOException e) {
+                    remainingTimeout = connectTimeout - (int) ((nanoTime() - startNanoTime) / 1_000_000);
+
+                    if (remainingTimeout <= leaderConnectDelayDuringRetryMs) {
+                        LOG.error(
+                          "Unexpected exception, connectToLeader exceeded. tries={}, remaining init limit={}, connecting to {}",
+                          tries,
+                          remainingTimeout,
+                          address,
+                          e);
+                        throw e;
+                    } else if (tries >= 4) {
+                        LOG.error(
+                          "Unexpected exception, retries exceeded. tries={}, remaining init limit={}, connecting to {}",
+                          tries,
+                          remainingTimeout,
+                          address,
+                          e);
+                        throw e;
+                    } else {
+                        LOG.warn(
+                          "Unexpected exception, tries={}, remaining init limit={}, connecting to {}",
+                          tries,
+                          remainingTimeout,
+                          address,
+                          e);
+                        sock = createSocket();
+                    }
+                }
+                Thread.sleep(leaderConnectDelayDuringRetryMs);
+            }
+
+            return sock;
+        }
+    }
+
+    /**
+     * Creating a simple or and SSL socket.
+     * This can be overridden in tests to fake already connected sockets for connectToLeader.
+     */
+    protected Socket createSocket() throws X509Exception, IOException {
         Socket sock;
         if (self.isSslQuorum()) {
             sock = self.getX509Util().createSSLSocket();

+ 6 - 2
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LocalPeerBean.java

@@ -19,6 +19,8 @@
 package org.apache.zookeeper.server.quorum;
 
 import static org.apache.zookeeper.common.NetUtils.formatInetAddr;
+import java.util.stream.Collectors;
+import org.apache.zookeeper.common.NetUtils;
 import org.apache.zookeeper.server.ServerCnxnHelper;
 
 /**
@@ -81,7 +83,8 @@ public class LocalPeerBean extends ServerBean implements LocalPeerMXBean {
     }
 
     public String getQuorumAddress() {
-        return formatInetAddr(peer.getQuorumAddress());
+        return peer.getQuorumAddress().getAllAddresses().stream().map(NetUtils::formatInetAddr)
+                .collect(Collectors.joining("|"));
     }
 
     public int getElectionType() {
@@ -89,7 +92,8 @@ public class LocalPeerBean extends ServerBean implements LocalPeerMXBean {
     }
 
     public String getElectionAddress() {
-        return formatInetAddr(peer.getElectionAddress());
+        return peer.getElectionAddress().getAllAddresses().stream().map(NetUtils::formatInetAddr)
+                .collect(Collectors.joining("|"));
     }
 
     public String getClientAddress() {

+ 225 - 0
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java

@@ -0,0 +1,225 @@
+/**
+ * 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.zookeeper.server.quorum;
+
+import static java.util.Arrays.asList;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.NoRouteToHostException;
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+
+/**
+ * This class allows to store several quorum and electing addresses.
+ *
+ * See ZOOKEEPER-3188 for a discussion of this feature.
+ */
+public final class MultipleAddresses {
+    private static final Duration DEFAULT_TIMEOUT = Duration.ofMillis(500);
+
+    private static Set<InetSocketAddress> newConcurrentHashSet() {
+        return Collections.newSetFromMap(new ConcurrentHashMap<>());
+    }
+
+    private Set<InetSocketAddress> addresses;
+    private final Duration timeout;
+
+    public MultipleAddresses() {
+        this(Collections.emptyList());
+    }
+
+    public MultipleAddresses(Collection<InetSocketAddress> addresses) {
+        this(addresses, DEFAULT_TIMEOUT);
+    }
+
+    public MultipleAddresses(InetSocketAddress address) {
+        this(asList(address), DEFAULT_TIMEOUT);
+    }
+
+    public MultipleAddresses(Collection<InetSocketAddress> addresses, Duration timeout) {
+        this.addresses = newConcurrentHashSet();
+        this.addresses.addAll(addresses);
+        this.timeout = timeout;
+    }
+
+    public boolean isEmpty() {
+        return addresses.isEmpty();
+    }
+
+    /**
+     * Returns all addresses in an unmodifiable set.
+     *
+     * @return set of all InetSocketAddress
+     */
+    public Set<InetSocketAddress> getAllAddresses() {
+        return Collections.unmodifiableSet(addresses);
+    }
+
+    /**
+     * Returns wildcard addresses for all ports
+     *
+     * @return set of InetSocketAddress with wildcards for all ports
+     */
+    public Set<InetSocketAddress> getWildcardAddresses() {
+        return addresses.stream().map(a -> new InetSocketAddress(a.getPort())).collect(Collectors.toSet());
+    }
+
+    /**
+     * Returns all ports
+     *
+     * @return list of all ports
+     */
+    public List<Integer> getAllPorts() {
+        return addresses.stream().map(InetSocketAddress::getPort).distinct().collect(Collectors.toList());
+    }
+
+    /**
+     * Returns distinct list of all host strings
+     *
+     * @return list of all hosts
+     */
+    public List<String> getAllHostStrings() {
+        return addresses.stream().map(InetSocketAddress::getHostString).distinct().collect(Collectors.toList());
+    }
+
+    public void addAddress(InetSocketAddress address) {
+        addresses.add(address);
+    }
+
+    /**
+     * Returns a reachable address. If none is reachable than throws exception.
+     * The function is nondeterministic in the sense that the result of calling this function
+     * twice with the same set of reachable addresses might lead to different results.
+     *
+     * @return address which is reachable.
+     * @throws NoRouteToHostException if none of the addresses are reachable
+     */
+    public InetSocketAddress getReachableAddress() throws NoRouteToHostException {
+        // using parallelStream() + findAny() will help to minimize the time spent on network operations
+        return addresses.parallelStream()
+          .filter(this::checkIfAddressIsReachable)
+          .findAny()
+          .orElseThrow(() -> new NoRouteToHostException("No valid address among " + addresses));
+    }
+
+    /**
+     * Returns a set of all reachable addresses. If none is reachable than returns empty set.
+     *
+     * @return all addresses which are reachable.
+     */
+    public Set<InetSocketAddress> getAllReachableAddresses() {
+        // using parallelStream() will help to minimize the time spent on network operations
+        return addresses.parallelStream()
+          .filter(this::checkIfAddressIsReachable)
+          .collect(Collectors.toSet());
+    }
+
+    /**
+     * Returns a reachable address or an arbitrary one, if none is reachable. It throws an exception
+     * if there are no addresses registered. The function is nondeterministic in the sense that the
+     * result of calling this function twice with the same set of reachable addresses might lead
+     * to different results.
+     *
+     * @return address which is reachable or fist one.
+     * @throws NoSuchElementException if there is no address registered
+     */
+    public InetSocketAddress getReachableOrOne() {
+        InetSocketAddress address;
+        try {
+            address = getReachableAddress();
+        } catch (NoRouteToHostException e) {
+            address = getOne();
+        }
+        return address;
+    }
+
+    /**
+     * Performs a parallel DNS lookup for all addresses.
+     *
+     * If the DNS lookup fails, then address remain unmodified.
+     */
+    public void recreateSocketAddresses() {
+        addresses = addresses.parallelStream()
+          .map(this::recreateSocketAddress)
+          .collect(Collectors.toCollection(MultipleAddresses::newConcurrentHashSet));
+    }
+
+    /**
+     * Returns an address from the set.
+     *
+     * @return address from a set.
+     * @throws NoSuchElementException if there is no address registered
+     */
+    public InetSocketAddress getOne() {
+        return addresses.iterator().next();
+    }
+
+    private boolean checkIfAddressIsReachable(InetSocketAddress address) {
+        if (address.isUnresolved()) {
+            return false;
+        }
+        try {
+            if (address.getAddress().isReachable((int) timeout.toMillis())) {
+                return true;
+            }
+        } catch (IOException ignored) {
+            // ignore, we don't really care if we can't reach it for timeout or for IO problems
+        }
+        return false;
+    }
+
+    private InetSocketAddress recreateSocketAddress(InetSocketAddress address) {
+        try {
+            return new InetSocketAddress(InetAddress.getByName(address.getHostString()), address.getPort());
+        } catch (UnknownHostException e) {
+            return address;
+        }
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        } else if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+
+        MultipleAddresses that = (MultipleAddresses) o;
+        return Objects.equals(addresses, that.addresses);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(addresses);
+    }
+
+    @Override
+    public String toString() {
+        return addresses.stream().map(InetSocketAddress::toString).collect(Collectors.joining("|"));
+    }
+}

+ 4 - 7
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverMaster.java

@@ -22,6 +22,7 @@ import java.io.BufferedInputStream;
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
+import java.net.InetAddress;
 import java.net.ServerSocket;
 import java.net.Socket;
 import java.net.SocketAddress;
@@ -429,23 +430,19 @@ public class ObserverMaster extends LearnerMaster implements Runnable {
         }
         listenerRunning = true;
         int backlog = 10; // dog science
+        InetAddress address = self.getQuorumAddress().getReachableOrOne().getAddress();
         if (self.shouldUsePortUnification() || self.isSslQuorum()) {
             boolean allowInsecureConnection = self.shouldUsePortUnification();
             if (self.getQuorumListenOnAllIPs()) {
                 ss = new UnifiedServerSocket(self.getX509Util(), allowInsecureConnection, port, backlog);
             } else {
-                ss = new UnifiedServerSocket(
-                    self.getX509Util(),
-                    allowInsecureConnection,
-                    port,
-                    backlog,
-                    self.getQuorumAddress().getAddress());
+                ss = new UnifiedServerSocket(self.getX509Util(), allowInsecureConnection, port, backlog, address);
             }
         } else {
             if (self.getQuorumListenOnAllIPs()) {
                 ss = new ServerSocket(port, backlog);
             } else {
-                ss = new ServerSocket(port, backlog, self.getQuorumAddress().getAddress());
+                ss = new ServerSocket(port, backlog, address);
             }
         }
         thread = new Thread(this, "ObserverMaster");

+ 285 - 136
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java

@@ -21,10 +21,13 @@ package org.apache.zookeeper.server.quorum;
 import static org.apache.zookeeper.common.NetUtils.formatInetAddr;
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
+import java.io.Closeable;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.net.InetAddress;
 import java.net.InetSocketAddress;
+import java.net.NoRouteToHostException;
 import java.net.ServerSocket;
 import java.net.Socket;
 import java.net.SocketException;
@@ -32,20 +35,28 @@ import java.net.SocketTimeoutException;
 import java.nio.BufferUnderflowException;
 import java.nio.ByteBuffer;
 import java.nio.channels.UnresolvedAddressException;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Enumeration;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
 import javax.net.ssl.SSLSocket;
+import org.apache.zookeeper.common.NetUtils;
 import org.apache.zookeeper.common.X509Exception;
 import org.apache.zookeeper.server.ExitCode;
 import org.apache.zookeeper.server.ZooKeeperThread;
@@ -59,6 +70,7 @@ import org.apache.zookeeper.util.ServiceUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+
 /**
  * This class implements a connection manager for leader election using TCP. It
  * maintains one connection for every pair of servers. The tricky part is to
@@ -102,7 +114,7 @@ public class QuorumCnxManager {
     /*
      * Protocol identifier used among peers
      */
-    public static final long PROTOCOL_VERSION = -65536L;
+    public static final long PROTOCOL_VERSION = -65535L;
 
     /*
      * Max buffer size to be read from the network.
@@ -125,7 +137,7 @@ public class QuorumCnxManager {
     final Map<Long, QuorumPeer.QuorumServer> view;
     final boolean listenOnAllIPs;
     private ThreadPoolExecutor connectionExecutor;
-    private final Set<Long> inprogressConnections = Collections.synchronizedSet(new HashSet<Long>());
+    private final Set<Long> inprogressConnections = Collections.synchronizedSet(new HashSet<>());
     private QuorumAuthServer authServer;
     private QuorumAuthLearner authLearner;
     private boolean quorumSaslAuthEnabled;
@@ -186,11 +198,11 @@ public class QuorumCnxManager {
     public static class InitialMessage {
 
         public Long sid;
-        public InetSocketAddress electionAddr;
+        public List<InetSocketAddress> electionAddr;
 
-        InitialMessage(Long sid, InetSocketAddress address) {
+        InitialMessage(Long sid, List<InetSocketAddress> addresses) {
             this.sid = sid;
-            this.electionAddr = address;
+            this.electionAddr = addresses;
         }
 
         @SuppressWarnings("serial")
@@ -223,33 +235,41 @@ public class QuorumCnxManager {
                 throw new InitialMessageException("Read only %s bytes out of %s sent by server %s", num_read, remaining, sid);
             }
 
-            String addr = new String(b);
-            String[] host_port;
-            try {
-                host_port = ConfigUtils.getHostAndPort(addr);
-            } catch (ConfigException e) {
-                throw new InitialMessageException("Badly formed address: %s", addr);
-            }
+            String[] addressStrings = new String(b).split("\\|");
+            List<InetSocketAddress> addresses = new ArrayList<>(addressStrings.length);
+            for (String addr : addressStrings) {
 
-            if (host_port.length != 2) {
-                throw new InitialMessageException("Badly formed address: %s", addr);
-            }
+                String[] host_port;
+                try {
+                    host_port = ConfigUtils.getHostAndPort(addr);
+                } catch (ConfigException e) {
+                    throw new InitialMessageException("Badly formed address: %s", addr);
+                }
 
-            int port;
-            try {
-                port = Integer.parseInt(host_port[1]);
-            } catch (NumberFormatException e) {
-                throw new InitialMessageException("Bad port number: %s", host_port[1]);
-            } catch (ArrayIndexOutOfBoundsException e) {
-                throw new InitialMessageException("No port number in: %s", addr);
+                if (host_port.length != 2) {
+                    throw new InitialMessageException("Badly formed address: %s", addr);
+                }
+
+                int port;
+                try {
+                    port = Integer.parseInt(host_port[1]);
+                } catch (NumberFormatException e) {
+                    throw new InitialMessageException("Bad port number: %s", host_port[1]);
+                } catch (ArrayIndexOutOfBoundsException e) {
+                    throw new InitialMessageException("No port number in: %s", addr);
+                }
+                addresses.add(new InetSocketAddress(host_port[0], port));
             }
 
-            return new InitialMessage(sid, new InetSocketAddress(host_port[0], port));
+            return new InitialMessage(sid, addresses);
         }
 
     }
 
-    public QuorumCnxManager(QuorumPeer self, final long mySid, Map<Long, QuorumPeer.QuorumServer> view, QuorumAuthServer authServer, QuorumAuthLearner authLearner, int socketTimeout, boolean listenOnAllIPs, int quorumCnxnThreadsSize, boolean quorumSaslAuthEnabled) {
+    public QuorumCnxManager(QuorumPeer self, final long mySid, Map<Long, QuorumPeer.QuorumServer> view,
+        QuorumAuthServer authServer, QuorumAuthLearner authLearner, int socketTimeout, boolean listenOnAllIPs,
+        int quorumCnxnThreadsSize, boolean quorumSaslAuthEnabled) {
+
         this.recvQueue = new CircularBlockingQueue<>(RECV_CAPACITY);
         this.queueSendMap = new ConcurrentHashMap<>();
         this.senderWorkerMap = new ConcurrentHashMap<>();
@@ -274,7 +294,9 @@ public class QuorumCnxManager {
         listener.setName("QuorumPeerListener");
     }
 
-    private void initializeAuth(final long mySid, final QuorumAuthServer authServer, final QuorumAuthLearner authLearner, final int quorumCnxnThreadsSize, final boolean quorumSaslAuthEnabled) {
+    private void initializeAuth(final long mySid, final QuorumAuthServer authServer,
+        final QuorumAuthLearner authLearner, final int quorumCnxnThreadsSize, final boolean quorumSaslAuthEnabled) {
+
         this.authServer = authServer;
         this.authLearner = authLearner;
         this.quorumSaslAuthEnabled = quorumSaslAuthEnabled;
@@ -311,7 +333,8 @@ public class QuorumCnxManager {
         LOG.debug("Opening channel to server {}", sid);
         Socket sock = new Socket();
         setSockOpts(sock);
-        sock.connect(self.getVotingView().get(sid).electionAddr, cnxTO);
+        InetSocketAddress address = self.getVotingView().get(sid).electionAddr.getReachableOrOne();
+        sock.connect(address, cnxTO);
         initiateConnection(sock, sid);
     }
 
@@ -324,10 +347,10 @@ public class QuorumCnxManager {
             startConnection(sock, sid);
         } catch (IOException e) {
             LOG.error(
-                "Exception while connecting, id: {}, addr: {}, closing learner connection",
-                sid,
-                sock.getRemoteSocketAddress(),
-                e);
+              "Exception while connecting, id: {}, addr: {}, closing learner connection",
+              sid,
+              sock.getRemoteSocketAddress(),
+              e);
             closeSocket(sock);
         }
     }
@@ -395,7 +418,8 @@ public class QuorumCnxManager {
             // represents protocol version (in other words - message type)
             dout.writeLong(PROTOCOL_VERSION);
             dout.writeLong(self.getId());
-            String addr = formatInetAddr(self.getElectionAddress());
+            String addr = self.getElectionAddress().getAllAddresses().stream()
+                    .map(NetUtils::formatInetAddr).collect(Collectors.joining("|"));
             byte[] addr_bytes = addr.getBytes();
             dout.writeInt(addr_bytes.length);
             dout.write(addr_bytes);
@@ -500,7 +524,7 @@ public class QuorumCnxManager {
 
     private void handleConnection(Socket sock, DataInputStream din) throws IOException {
         Long sid = null, protocolVersion = null;
-        InetSocketAddress electionAddr = null;
+        MultipleAddresses electionAddr = null;
 
         try {
             protocolVersion = din.readLong();
@@ -510,7 +534,7 @@ public class QuorumCnxManager {
                 try {
                     InitialMessage init = InitialMessage.parse(protocolVersion, din);
                     sid = init.sid;
-                    electionAddr = init.electionAddr;
+                    electionAddr = new MultipleAddresses(init.electionAddr);
                 } catch (InitialMessage.InitialMessageException ex) {
                     LOG.error(ex.toString());
                     closeSocket(sock);
@@ -610,9 +634,13 @@ public class QuorumCnxManager {
      *  @param sid  server id
      *  @return boolean success indication
      */
-    synchronized boolean connectOne(long sid, InetSocketAddress electionAddr) {
+    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();
             return true;
         }
 
@@ -625,7 +653,7 @@ public class QuorumCnxManager {
                 sock = new Socket();
             }
             setSockOpts(sock);
-            sock.connect(electionAddr, cnxTO);
+            sock.connect(electionAddr.getReachableAddress(), cnxTO);
             if (sock instanceof SSLSocket) {
                 SSLSocket sslSock = (SSLSocket) sock;
                 sslSock.startHandshake();
@@ -635,7 +663,8 @@ public class QuorumCnxManager {
                          sslSock.getSession().getCipherSuite());
             }
 
-            LOG.debug("Connected to server {}", sid);
+            LOG.debug("Connected to server {} using election address: {}:{}",
+                      sid, sock.getInetAddress(), sock.getPort());
             // Sends connection request asynchronously if the quorum
             // sasl authentication is enabled. This is required because
             // sasl server authentication process may take few seconds to
@@ -658,6 +687,10 @@ public class QuorumCnxManager {
             LOG.warn("Cannot open secure channel to {} at election address {}", sid, electionAddr, e);
             closeSocket(sock);
             return false;
+        } catch (NoRouteToHostException e) {
+            LOG.warn("None of the addresses ({}) are reachable for sid {}", electionAddr, sid, e);
+            closeSocket(sock);
+            return false;
         } catch (IOException e) {
             LOG.warn("Cannot open channel to {} at election address {}", sid, electionAddr, e);
             closeSocket(sock);
@@ -673,6 +706,10 @@ 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();
             return;
         }
         synchronized (self.QV_LOCK) {
@@ -818,7 +855,7 @@ public class QuorumCnxManager {
     }
 
     /**
-     * Thread to listen on some port
+     * Thread to listen on some ports
      */
     public class Listener extends ZooKeeperThread {
 
@@ -827,25 +864,30 @@ public class QuorumCnxManager {
 
         private final int portBindMaxRetry;
         private Runnable socketBindErrorHandler = () -> ServiceUtils.requestSystemExit(ExitCode.UNABLE_TO_BIND_QUORUM_PORT.getValue());
-        volatile ServerSocket ss = null;
+        private List<ListenerHandler> listenerHandlers;
+        private final AtomicBoolean socketException;
+
 
         public Listener() {
             // During startup of thread, thread name will be overridden to
             // specific election address
             super("ListenerThread");
 
+            socketException = new AtomicBoolean(false);
+
             // maximum retry count while trying to bind to election port
             // see ZOOKEEPER-3320 for more details
-            final Integer maxRetry = Integer.getInteger(ELECTION_PORT_BIND_RETRY, DEFAULT_PORT_BIND_MAX_RETRY);
+            final Integer maxRetry = Integer.getInteger(ELECTION_PORT_BIND_RETRY,
+                    DEFAULT_PORT_BIND_MAX_RETRY);
             if (maxRetry >= 0) {
                 LOG.info("Election port bind maximum retries is {}", maxRetry == 0 ? "infinite" : maxRetry);
                 portBindMaxRetry = maxRetry;
             } else {
                 LOG.info(
-                    "'{}' contains invalid value: {}(must be >= 0). Use default value of {} instead.",
-                    ELECTION_PORT_BIND_RETRY,
-                    maxRetry,
-                    DEFAULT_PORT_BIND_MAX_RETRY);
+                  "'{}' contains invalid value: {}(must be >= 0). Use default value of {} instead.",
+                  ELECTION_PORT_BIND_RETRY,
+                  maxRetry,
+                  DEFAULT_PORT_BIND_MAX_RETRY);
                 portBindMaxRetry = DEFAULT_PORT_BIND_MAX_RETRY;
             }
         }
@@ -857,106 +899,54 @@ public class QuorumCnxManager {
             this.socketBindErrorHandler = errorHandler;
         }
 
-        /**
-         * Sleeps on accept().
-         */
         @Override
         public void run() {
-            int numRetries = 0;
-            InetSocketAddress addr;
-            Socket client = null;
-            Exception exitException = null;
-            while ((!shutdown) && (portBindMaxRetry == 0 || numRetries < portBindMaxRetry)) {
-                try {
-                    if (self.shouldUsePortUnification()) {
-                        LOG.info("Creating TLS-enabled quorum server socket");
-                        ss = new UnifiedServerSocket(self.getX509Util(), true);
-                    } else if (self.isSslQuorum()) {
-                        LOG.info("Creating TLS-only quorum server socket");
-                        ss = new UnifiedServerSocket(self.getX509Util(), false);
-                    } else {
-                        ss = new ServerSocket();
-                    }
+            if (!shutdown) {
+                Set<InetSocketAddress> addresses;
 
-                    ss.setReuseAddress(true);
+                if (self.getQuorumListenOnAllIPs()) {
+                    addresses = self.getElectionAddress().getWildcardAddresses();
+                } else {
+                    addresses = self.getElectionAddress().getAllAddresses();
+                }
 
-                    if (self.getQuorumListenOnAllIPs()) {
-                        int port = self.getElectionAddress().getPort();
-                        addr = new InetSocketAddress(port);
-                    } else {
-                        // Resolve hostname for this server in case the
-                        // underlying ip address has changed.
-                        self.recreateSocketAddresses(self.getId());
-                        addr = self.getElectionAddress();
-                    }
-                    LOG.info("My election bind port: {}", addr.toString());
-                    setName(addr.toString());
-                    ss.bind(addr);
-                    while (!shutdown) {
+                CountDownLatch latch = new CountDownLatch(addresses.size());
+                listenerHandlers = addresses.stream().map(address ->
+                                new ListenerHandler(address, self.shouldUsePortUnification(), self.isSslQuorum(), latch))
+                        .collect(Collectors.toList());
+
+                ExecutorService executor = Executors.newFixedThreadPool(addresses.size());
+                listenerHandlers.forEach(executor::submit);
+
+                try {
+                    latch.await();
+                } catch (InterruptedException ie) {
+                    LOG.error("Interrupted while sleeping. Ignoring exception", ie);
+                } finally {
+                    // Clean up for shutdown.
+                    for (ListenerHandler handler : listenerHandlers) {
                         try {
-                            client = ss.accept();
-                            setSockOpts(client);
-                            LOG.info("Received connection request {}", formatInetAddr((InetSocketAddress) client.getRemoteSocketAddress()));
-                            // Receive and handle the connection request
-                            // asynchronously if the quorum sasl authentication is
-                            // enabled. This is required because sasl server
-                            // authentication process may take few seconds to finish,
-                            // this may delay next peer connection requests.
-                            if (quorumSaslAuthEnabled) {
-                                receiveConnectionAsync(client);
-                            } else {
-                                receiveConnection(client);
-                            }
-                            numRetries = 0;
-                        } catch (SocketTimeoutException e) {
-                            LOG.warn(
-                                "The socket is listening for the election accepted "
-                                    + "and it timed out unexpectedly, but will retry."
-                                    + "see ZOOKEEPER-2836");
+                            handler.close();
+                        } catch (IOException ie) {
+                            // Don't log an error for shutdown.
+                            LOG.debug("Error closing server socket", ie);
                         }
                     }
-                } catch (IOException e) {
-                    if (shutdown) {
-                        break;
-                    }
-                    LOG.error("Exception while listening", e);
-                    exitException = e;
-                    numRetries++;
-                    try {
-                        ss.close();
-                        Thread.sleep(1000);
-                    } catch (IOException ie) {
-                        LOG.error("Error closing server socket", ie);
-                    } catch (InterruptedException ie) {
-                        LOG.error("Interrupted while sleeping. Ignoring exception", ie);
-                    }
-                    closeSocket(client);
                 }
             }
+
             LOG.info("Leaving listener");
             if (!shutdown) {
                 LOG.error(
-                    "As I'm leaving the listener thread after {} errors. "
-                        + "I won't be able to participate in leader election any longer: {}."
-                        + "Use {} property to increase retry count.",
-                          numRetries,
-                          formatInetAddr(self.getElectionAddress()),
-                          ELECTION_PORT_BIND_RETRY);
-
-                if (exitException instanceof SocketException) {
-                    // After leaving listener thread, the host cannot join the
-                    // quorum anymore, this is a severe error that we cannot
-                    // recover from, so we need to exit
+                  "As I'm leaving the listener thread, I won't be able to participate in leader election any longer: {}",
+                  self.getElectionAddress().getAllAddresses().stream()
+                    .map(NetUtils::formatInetAddr)
+                    .collect(Collectors.joining("|")));
+                if (socketException.get()) {
+                    // After leaving listener thread, the host cannot join the quorum anymore,
+                    // this is a severe error that we cannot recover from, so we need to exit
                     socketBindErrorHandler.run();
                 }
-            } else if (ss != null) {
-                // Clean up for shutdown.
-                try {
-                    ss.close();
-                } catch (IOException ie) {
-                    // Don't log an error for shutdown.
-                    LOG.debug("Error closing server socket", ie);
-                }
             }
         }
 
@@ -964,14 +954,145 @@ public class QuorumCnxManager {
          * Halts this listener thread.
          */
         void halt() {
-            try {
-                LOG.debug("Trying to close listener: {}", ss);
-                if (ss != null) {
-                    LOG.debug("Closing listener: {}", QuorumCnxManager.this.mySid);
-                    ss.close();
+            LOG.debug("Trying to close listeners");
+            if (listenerHandlers != null) {
+                LOG.debug("Closing listener: {}", QuorumCnxManager.this.mySid);
+                for (ListenerHandler handler : listenerHandlers) {
+                    try {
+                        handler.close();
+                    } catch (IOException e) {
+                        LOG.warn("Exception when shutting down listener: ", e);
+                    }
                 }
-            } catch (IOException e) {
-                LOG.warn("Exception when shutting down listener", e);
+            }
+        }
+
+        class ListenerHandler implements Runnable, Closeable {
+            private ServerSocket serverSocket;
+            private InetSocketAddress address;
+            private boolean portUnification;
+            private boolean sslQuorum;
+            private CountDownLatch latch;
+
+            ListenerHandler(InetSocketAddress address, boolean portUnification, boolean sslQuorum,
+                            CountDownLatch latch) {
+                this.address = address;
+                this.portUnification = portUnification;
+                this.sslQuorum = sslQuorum;
+                this.latch = latch;
+            }
+
+            /**
+             * Sleeps on acceptConnections().
+             */
+            @Override
+            public void run() {
+                try {
+                    Thread.currentThread().setName("ListenerHandler-" + address);
+                    acceptConnections();
+                    try {
+                        close();
+                    } catch (IOException e) {
+                        LOG.warn("Exception when shutting down listener: ", e);
+                    }
+                } catch (Exception e) {
+                    // Output of unexpected exception, should never happen
+                    LOG.error("Unexpected error ", e);
+                } finally {
+                    latch.countDown();
+                }
+            }
+
+            @Override
+            public synchronized void close() throws IOException {
+                if (serverSocket != null && !serverSocket.isClosed()) {
+                    LOG.debug("Trying to close listeners: {}", serverSocket);
+                    serverSocket.close();
+                }
+            }
+
+            /**
+             * Sleeps on accept().
+             */
+            private void acceptConnections() {
+                int numRetries = 0;
+                Socket client = null;
+
+                while ((!shutdown) && (portBindMaxRetry == 0 || numRetries < portBindMaxRetry)) {
+                    try {
+                        serverSocket = createNewServerSocket();
+                        LOG.info("My election bind port: {}", address.toString());
+                        while (!shutdown) {
+                            try {
+                                client = serverSocket.accept();
+                                setSockOpts(client);
+                                LOG.info("Received connection request {}", client.getRemoteSocketAddress());
+                                // Receive and handle the connection request
+                                // asynchronously if the quorum sasl authentication is
+                                // enabled. This is required because sasl server
+                                // authentication process may take few seconds to finish,
+                                // this may delay next peer connection requests.
+                                if (quorumSaslAuthEnabled) {
+                                    receiveConnectionAsync(client);
+                                } else {
+                                    receiveConnection(client);
+                                }
+                                numRetries = 0;
+                            } catch (SocketTimeoutException e) {
+                                LOG.warn("The socket is listening for the election accepted "
+                                        + "and it timed out unexpectedly, but will retry."
+                                        + "see ZOOKEEPER-2836");
+                            }
+                        }
+                    } catch (IOException e) {
+                        if (shutdown) {
+                            break;
+                        }
+
+                        LOG.error("Exception while listening", e);
+
+                        if (e instanceof SocketException) {
+                            socketException.set(true);
+                        }
+
+                        numRetries++;
+                        try {
+                            close();
+                            Thread.sleep(1000);
+                        } catch (IOException ie) {
+                            LOG.error("Error closing server socket", ie);
+                        } catch (InterruptedException ie) {
+                            LOG.error("Interrupted while sleeping. Ignoring exception", ie);
+                        }
+                        closeSocket(client);
+                    }
+                }
+                if (!shutdown) {
+                    LOG.error(
+                      "Leaving listener thread for address {} after {} errors. Use {} property to increase retry count.",
+                      formatInetAddr(address),
+                      numRetries,
+                      ELECTION_PORT_BIND_RETRY);
+                }
+            }
+
+            private ServerSocket createNewServerSocket() throws IOException {
+                ServerSocket socket;
+
+                if (portUnification) {
+                    LOG.info("Creating TLS-enabled quorum server socket");
+                    socket = new UnifiedServerSocket(self.getX509Util(), true);
+                } else if (sslQuorum) {
+                    LOG.info("Creating TLS-only quorum server socket");
+                    socket = new UnifiedServerSocket(self.getX509Util(), false);
+                } else {
+                    socket = new ServerSocket();
+                }
+
+                socket.setReuseAddress(true);
+                socket.bind(address);
+
+                return socket;
             }
         }
 
@@ -989,6 +1110,7 @@ public class QuorumCnxManager {
         RecvWorker recvWorker;
         volatile boolean running = true;
         DataOutputStream dout;
+        AtomicBoolean ongoingAsyncValidation = new AtomicBoolean(false);
 
         /**
          * An instance of this thread receives messages to send
@@ -1129,6 +1251,33 @@ public class QuorumCnxManager {
             LOG.warn("Send worker leaving thread id {} my id = {}", sid, self.getId());
         }
 
+
+        public void asyncValidateIfSocketIsStillReachable() {
+            if (ongoingAsyncValidation.compareAndSet(false, true)) {
+                new Thread(() -> {
+                    LOG.debug("validate if destination address is reachable for sid {}", sid);
+                    if (sock != null) {
+                        InetAddress address = sock.getInetAddress();
+                        try {
+                            if (address.isReachable(500)) {
+                                LOG.debug("destination address {} is reachable for sid {}", address.toString(), sid);
+                                ongoingAsyncValidation.set(false);
+                                return;
+                            }
+                        } catch (NullPointerException | IOException ignored) {
+                        }
+                        LOG.warn(
+                          "destination address {} not reachable anymore, shutting down the SendWorker for sid {}",
+                          address.toString(),
+                          sid);
+                        this.finish();
+                    }
+                }).start();
+            } else {
+                LOG.debug("validation of destination address for sid {} is skipped (it is already running)", sid);
+            }
+        }
+
     }
 
     /**

+ 106 - 85
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java

@@ -31,12 +31,13 @@ import java.net.DatagramPacket;
 import java.net.DatagramSocket;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -45,6 +46,8 @@ import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
 import javax.security.sasl.SaslException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException.BadArgumentsException;
@@ -136,11 +139,11 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
 
     public static final class AddressTuple {
 
-        public final InetSocketAddress quorumAddr;
-        public final InetSocketAddress electionAddr;
+        public final MultipleAddresses quorumAddr;
+        public final MultipleAddresses electionAddr;
         public final InetSocketAddress clientAddr;
 
-        public AddressTuple(InetSocketAddress quorumAddr, InetSocketAddress electionAddr, InetSocketAddress clientAddr) {
+        public AddressTuple(MultipleAddresses quorumAddr, MultipleAddresses electionAddr, InetSocketAddress clientAddr) {
             this.quorumAddr = quorumAddr;
             this.electionAddr = electionAddr;
             this.clientAddr = clientAddr;
@@ -160,9 +163,9 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
 
     public static class QuorumServer {
 
-        public InetSocketAddress addr = null;
+        public MultipleAddresses addr = new MultipleAddresses();
 
-        public InetSocketAddress electionAddr = null;
+        public MultipleAddresses electionAddr = new MultipleAddresses();
 
         public InetSocketAddress clientAddr = null;
 
@@ -200,54 +203,41 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
          * unmodified.
          */
         public void recreateSocketAddresses() {
-            if (this.addr == null) {
+            if (this.addr.isEmpty()) {
                 LOG.warn("Server address has not been initialized");
                 return;
             }
-            if (this.electionAddr == null) {
+            if (this.electionAddr.isEmpty()) {
                 LOG.warn("Election address has not been initialized");
                 return;
             }
-            String host = this.addr.getHostString();
-            InetAddress address = null;
-            try {
-                address = InetAddress.getByName(host);
-            } catch (UnknownHostException ex) {
-                LOG.warn("Failed to resolve address: {}", host, ex);
-                return;
-            }
-            LOG.debug("Resolved address for {}: {}", host, address);
-            int port = this.addr.getPort();
-            this.addr = new InetSocketAddress(address, port);
-            port = this.electionAddr.getPort();
-            this.electionAddr = new InetSocketAddress(address, port);
-        }
-
-        private void setType(String s) throws ConfigException {
-            if (s.toLowerCase().equals("observer")) {
-                type = LearnerType.OBSERVER;
-            } else if (s.toLowerCase().equals("participant")) {
-                type = LearnerType.PARTICIPANT;
-            } else {
-                throw new ConfigException("Unrecognised peertype: " + s);
+            this.addr.recreateSocketAddresses();
+            this.electionAddr.recreateSocketAddresses();
+        }
+
+        private LearnerType getType(String s) throws ConfigException {
+            switch (s.trim().toLowerCase()) {
+                case "observer":
+                    return LearnerType.OBSERVER;
+                case "participant":
+                    return LearnerType.PARTICIPANT;
+                default:
+                    throw new ConfigException("Unrecognised peertype: " + s);
             }
         }
 
         private static final String wrongFormat =
             " does not have the form server_config or server_config;client_config"
-            + " where server_config is host:port:port or host:port:port:type and client_config is port or host:port";
+            + " where server_config is the pipe separated list of host:port:port or host:port:port:type"
+            + " and client_config is port or host:port";
 
         public QuorumServer(long sid, String addressStr) throws ConfigException {
-            // LOG.warn("sid = " + sid + " addressStr = " + addressStr);
             this.id = sid;
+            LearnerType newType = null;
             String[] serverClientParts = addressStr.split(";");
-            String[] serverParts = ConfigUtils.getHostAndPort(serverClientParts[0]);
-            if ((serverClientParts.length > 2) || (serverParts.length < 3) || (serverParts.length > 4)) {
-                throw new ConfigException(addressStr + wrongFormat);
-            }
+            String[] serverAddresses = serverClientParts[0].split("\\|");
 
             if (serverClientParts.length == 2) {
-                //LOG.warn("ClientParts: " + serverClientParts[1]);
                 String[] clientParts = ConfigUtils.getHostAndPort(serverClientParts[1]);
                 if (clientParts.length > 2) {
                     throw new ConfigException(addressStr + wrongFormat);
@@ -257,35 +247,56 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
                 hostname = (clientParts.length == 2) ? clientParts[0] : "0.0.0.0";
                 try {
                     clientAddr = new InetSocketAddress(hostname, Integer.parseInt(clientParts[clientParts.length - 1]));
-                    //LOG.warn("Set clientAddr to " + clientAddr);
                 } catch (NumberFormatException e) {
                     throw new ConfigException("Address unresolved: " + hostname + ":" + clientParts[clientParts.length - 1]);
                 }
             }
 
-            // server_config should be either host:port:port or host:port:port:type
-            try {
-                addr = new InetSocketAddress(serverParts[0], Integer.parseInt(serverParts[1]));
-            } catch (NumberFormatException e) {
-                throw new ConfigException("Address unresolved: " + serverParts[0] + ":" + serverParts[1]);
-            }
-            try {
-                electionAddr = new InetSocketAddress(serverParts[0], Integer.parseInt(serverParts[2]));
-            } catch (NumberFormatException e) {
-                throw new ConfigException("Address unresolved: " + serverParts[0] + ":" + serverParts[2]);
-            }
+            for (String serverAddress : serverAddresses) {
+                String serverParts[] = ConfigUtils.getHostAndPort(serverAddress);
+                if ((serverClientParts.length > 2) || (serverParts.length < 3)
+                        || (serverParts.length > 4)) {
+                    throw new ConfigException(addressStr + wrongFormat);
+                }
 
-            if (addr.getPort() == electionAddr.getPort()) {
-                throw new ConfigException(
-                    "Client and election port must be different! Please update the configuration file on server."
-                    + sid);
-            }
+                // server_config should be either host:port:port or host:port:port:type
+                InetSocketAddress tempAddress;
+                InetSocketAddress tempElectionAddress;
+                try {
+                    tempAddress = new InetSocketAddress(serverParts[0], Integer.parseInt(serverParts[1]));
+                    addr.addAddress(tempAddress);
+                } catch (NumberFormatException e) {
+                    throw new ConfigException("Address unresolved: " + serverParts[0] + ":" + serverParts[1]);
+                }
+                try {
+                    tempElectionAddress = new InetSocketAddress(serverParts[0], Integer.parseInt(serverParts[2]));
+                    electionAddr.addAddress(tempElectionAddress);
+                } catch (NumberFormatException e) {
+                    throw new ConfigException("Address unresolved: " + serverParts[0] + ":" + serverParts[2]);
+                }
 
-            if (serverParts.length == 4) {
-                setType(serverParts[3]);
+                if (tempAddress.getPort() == tempElectionAddress.getPort()) {
+                    throw new ConfigException("Client and election port must be different! Please update the "
+                            + "configuration file on server." + sid);
+                }
+
+                if (serverParts.length == 4) {
+                    LearnerType tempType = getType(serverParts[3]);
+                    if (newType == null) {
+                        newType = tempType;
+                    }
+
+                    if (newType != tempType) {
+                        throw new ConfigException("Multiple addresses should have similar roles: " + type + " vs " + tempType);
+                    }
+                }
+
+                this.hostname = serverParts[0];
             }
 
-            this.hostname = serverParts[0];
+            if (newType != null) {
+                type = newType;
+            }
 
             setMyAddrs();
         }
@@ -296,8 +307,12 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
 
         public QuorumServer(long id, InetSocketAddress addr, InetSocketAddress electionAddr, InetSocketAddress clientAddr, LearnerType type) {
             this.id = id;
-            this.addr = addr;
-            this.electionAddr = electionAddr;
+            if (addr != null) {
+                this.addr.addAddress(addr);
+            }
+            if (electionAddr != null) {
+                this.electionAddr.addAddress(electionAddr);
+            }
             this.type = type;
             this.clientAddr = clientAddr;
 
@@ -305,10 +320,10 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
         }
 
         private void setMyAddrs() {
-            this.myAddrs = new ArrayList<InetSocketAddress>();
-            this.myAddrs.add(this.addr);
+            this.myAddrs = new ArrayList<>();
+            this.myAddrs.addAll(this.addr.getAllAddresses());
             this.myAddrs.add(this.clientAddr);
-            this.myAddrs.add(this.electionAddr);
+            this.myAddrs.addAll(this.electionAddr.getAllAddresses());
             this.myAddrs = excludedSpecialAddresses(this.myAddrs);
         }
 
@@ -323,27 +338,31 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
 
         public String toString() {
             StringWriter sw = new StringWriter();
-            //addr should never be null, but just to make sure
-            if (addr != null) {
-                sw.append(delimitedHostString(addr));
-                sw.append(":");
-                sw.append(String.valueOf(addr.getPort()));
-            }
-            if (electionAddr != null) {
-                sw.append(":");
-                sw.append(String.valueOf(electionAddr.getPort()));
+
+            List<InetSocketAddress> addrList = new LinkedList<>(addr.getAllAddresses());
+            List<InetSocketAddress> electionAddrList = new LinkedList<>(electionAddr.getAllAddresses());
+
+            if (addrList.size() > 0 && electionAddrList.size() > 0) {
+                addrList.sort(Comparator.comparing(InetSocketAddress::getHostString));
+                electionAddrList.sort(Comparator.comparing(InetSocketAddress::getHostString));
+                sw.append(IntStream.range(0, addrList.size()).mapToObj(i -> String.format("%s:%d:%d",
+                        delimitedHostString(addrList.get(i)), addrList.get(i).getPort(), electionAddrList.get(i).getPort()))
+                        .collect(Collectors.joining("|")));
             }
+
             if (type == LearnerType.OBSERVER) {
                 sw.append(":observer");
             } else if (type == LearnerType.PARTICIPANT) {
                 sw.append(":participant");
             }
+
             if (clientAddr != null && !isClientAddrFromStatic) {
                 sw.append(";");
                 sw.append(delimitedHostString(clientAddr));
                 sw.append(":");
                 sw.append(String.valueOf(clientAddr.getPort()));
             }
+
             return sw.toString();
         }
 
@@ -366,20 +385,19 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
             if ((qs.id != id) || (qs.type != type)) {
                 return false;
             }
-            if (!checkAddressesEqual(addr, qs.addr)) {
+            if (!addr.equals(qs.addr)) {
                 return false;
             }
-            if (!checkAddressesEqual(electionAddr, qs.electionAddr)) {
+            if (!electionAddr.equals(qs.electionAddr)) {
                 return false;
             }
             return checkAddressesEqual(clientAddr, qs.clientAddr);
         }
 
         public void checkAddressDuplicate(QuorumServer s) throws BadArgumentsException {
-            List<InetSocketAddress> otherAddrs = new ArrayList<InetSocketAddress>();
-            otherAddrs.add(s.addr);
+            List<InetSocketAddress> otherAddrs = new ArrayList<>(s.addr.getAllAddresses());
             otherAddrs.add(s.clientAddr);
-            otherAddrs.add(s.electionAddr);
+            otherAddrs.addAll(s.electionAddr.getAllAddresses());
             otherAddrs = excludedSpecialAddresses(otherAddrs);
 
             for (InetSocketAddress my : this.myAddrs) {
@@ -814,9 +832,9 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
         return syncMode.get();
     }
 
-    public void setLeaderAddressAndId(InetSocketAddress addr, long newId) {
+    public void setLeaderAddressAndId(MultipleAddresses addr, long newId) {
         if (addr != null) {
-            leaderAddress.set(addr.getHostString());
+            leaderAddress.set(String.join("|", addr.getAllHostStrings()));
         } else {
             leaderAddress.set(null);
         }
@@ -908,11 +926,11 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
         }
     }
 
-    public InetSocketAddress getQuorumAddress() {
+    public MultipleAddresses getQuorumAddress() {
         return getAddrs().quorumAddr;
     }
 
-    public InetSocketAddress getElectionAddress() {
+    public MultipleAddresses getElectionAddress() {
         return getAddrs().electionAddr;
     }
 
@@ -921,7 +939,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
         return (addrs == null) ? null : addrs.clientAddr;
     }
 
-    private void setAddrs(InetSocketAddress quorumAddr, InetSocketAddress electionAddr, InetSocketAddress clientAddr) {
+    private void setAddrs(MultipleAddresses quorumAddr, MultipleAddresses electionAddr, InetSocketAddress clientAddr) {
         synchronized (QV_LOCK) {
             myAddrs.set(new AddressTuple(quorumAddr, electionAddr, clientAddr));
             QV_LOCK.notifyAll();
@@ -2192,7 +2210,8 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
         observerMasters.clear();
         StringBuilder sb = new StringBuilder();
         for (QuorumServer server : quorumVerifier.getVotingMembers().values()) {
-            InetSocketAddress addr = new InetSocketAddress(server.addr.getAddress(), observerMasterPort);
+            InetAddress address = server.addr.getReachableOrOne().getAddress();
+            InetSocketAddress addr = new InetSocketAddress(address, observerMasterPort);
             observerMasters.add(new QuorumServer(server.id, addr));
             sb.append(addr).append(",");
         }
@@ -2247,9 +2266,11 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider
             }
             for (QuorumServer server : observerMasters) {
                 if (sid == null) {
-                    String serverAddr = server.addr.getAddress().getHostAddress() + ':' + server.addr.getPort();
-                    if (serverAddr.startsWith(desiredMaster)) {
-                        return server;
+                    for (InetSocketAddress address : server.addr.getAllAddresses()) {
+                        String serverAddr = address.getAddress().getHostAddress() + ':' + address.getPort();
+                        if (serverAddr.startsWith(desiredMaster)) {
+                            return server;
+                        }
                     }
                 } else {
                     if (sid.equals(server.id)) {

+ 8 - 3
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java

@@ -21,7 +21,9 @@ package org.apache.zookeeper.server.quorum;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.nio.ByteBuffer;
+import java.util.Objects;
 import java.util.function.BiConsumer;
+import java.util.stream.Collectors;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.MultiOperationRecord;
@@ -45,7 +47,8 @@ public abstract class QuorumZooKeeperServer extends ZooKeeperServer {
     public final QuorumPeer self;
     protected UpgradeableSessionTracker upgradeableSessionTracker;
 
-    protected QuorumZooKeeperServer(FileTxnSnapLog logFactory, int tickTime, int minSessionTimeout, int maxSessionTimeout, int listenBacklog, ZKDatabase zkDb, QuorumPeer self) {
+    protected QuorumZooKeeperServer(FileTxnSnapLog logFactory, int tickTime, int minSessionTimeout,
+                                    int maxSessionTimeout, int listenBacklog, ZKDatabase zkDb, QuorumPeer self) {
         super(logFactory, tickTime, minSessionTimeout, maxSessionTimeout, listenBacklog, zkDb, self.getInitialConfig());
         this.self = self;
     }
@@ -170,9 +173,11 @@ public abstract class QuorumZooKeeperServer extends ZooKeeperServer {
         pwriter.print("electionAlg=");
         pwriter.println(self.getElectionType());
         pwriter.print("electionPort=");
-        pwriter.println(self.getElectionAddress().getPort());
+        pwriter.println(self.getElectionAddress().getAllPorts()
+                .stream().map(Objects::toString).collect(Collectors.joining("|")));
         pwriter.print("quorumPort=");
-        pwriter.println(self.getQuorumAddress().getPort());
+        pwriter.println(self.getQuorumAddress().getAllPorts()
+                        .stream().map(Objects::toString).collect(Collectors.joining("|")));
         pwriter.print("peerType=");
         pwriter.println(self.getLearnerType().ordinal());
         pwriter.println("membership: ");

+ 6 - 2
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java

@@ -19,6 +19,8 @@
 package org.apache.zookeeper.server.quorum;
 
 import java.io.PrintWriter;
+import java.util.Objects;
+import java.util.stream.Collectors;
 import org.apache.zookeeper.jmx.MBeanRegistry;
 import org.apache.zookeeper.server.DataTreeBean;
 import org.apache.zookeeper.server.FinalRequestProcessor;
@@ -170,9 +172,11 @@ public class ReadOnlyZooKeeperServer extends ZooKeeperServer {
         pwriter.print("electionAlg=");
         pwriter.println(self.getElectionType());
         pwriter.print("electionPort=");
-        pwriter.println(self.getElectionAddress().getPort());
+        pwriter.println(self.getElectionAddress().getAllPorts()
+                .stream().map(Objects::toString).collect(Collectors.joining("|")));
         pwriter.print("quorumPort=");
-        pwriter.println(self.getQuorumAddress().getPort());
+        pwriter.println(self.getQuorumAddress().getAllPorts()
+                .stream().map(Objects::toString).collect(Collectors.joining("|")));
         pwriter.print("peerType=");
         pwriter.println(self.getLearnerType().ordinal());
     }

+ 7 - 2
zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/RemotePeerBean.java

@@ -18,6 +18,7 @@
 
 package org.apache.zookeeper.server.quorum;
 
+import java.util.stream.Collectors;
 import org.apache.zookeeper.jmx.ZKMBeanInfo;
 
 /**
@@ -46,11 +47,15 @@ public class RemotePeerBean implements RemotePeerMXBean, ZKMBeanInfo {
     }
 
     public String getQuorumAddress() {
-        return peer.addr.getHostString() + ":" + peer.addr.getPort();
+        return peer.addr.getAllAddresses().stream()
+                .map(address -> String.format("%s:%d", address.getHostString(), address.getPort()))
+                .collect(Collectors.joining("|"));
     }
 
     public String getElectionAddress() {
-        return peer.electionAddr.getHostString() + ":" + peer.electionAddr.getPort();
+        return peer.electionAddr.getAllAddresses().stream()
+                .map(address -> String.format("%s:%d", address.getHostString(), address.getPort()))
+                .collect(Collectors.joining("|"));
     }
 
     public String getClientAddress() {

+ 6 - 0
zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/CommandsTest.java

@@ -242,6 +242,12 @@ public class CommandsTest extends ClientBase {
         testCommand("watch_summary", new Field("num_connections", Integer.class), new Field("num_paths", Integer.class), new Field("num_total_watches", Integer.class));
     }
 
+    @Test
+    public void testVotingViewCommand() throws IOException, InterruptedException {
+        testCommand("voting_view",
+                    new Field("current_config", Map.class));
+    }
+
     @Test
     public void testConsCommandSecureOnly() {
         // Arrange

+ 48 - 18
zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CnxManagerTest.java

@@ -18,6 +18,7 @@
 
 package org.apache.zookeeper.server.quorum;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
@@ -35,11 +36,12 @@ import java.net.SocketAddress;
 import java.nio.ByteBuffer;
 import java.nio.channels.SocketChannel;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.Random;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import javax.net.ssl.HandshakeCompletedListener;
@@ -61,6 +63,7 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+
 public class CnxManagerTest extends ZKTestCase {
 
     protected static final Logger LOG = LoggerFactory.getLogger(FLENewEpochTest.class);
@@ -196,14 +199,17 @@ public class CnxManagerTest extends ZKTestCase {
 
     @Test
     public void testCnxManagerTimeout() throws Exception {
-        Random rand = new Random();
-        byte b = (byte) rand.nextInt();
+        int address = ThreadLocalRandom.current().nextInt(1, 255);
         int deadPort = PortAssignment.unique();
-        String deadAddress = "10.1.1." + b;
+        String deadAddress = "10.1.1." + address;
 
         LOG.info("This is the dead address I'm trying: {}", deadAddress);
 
-        peers.put(2L, new QuorumServer(2, new InetSocketAddress(deadAddress, deadPort), new InetSocketAddress(deadAddress, PortAssignment.unique()), new InetSocketAddress(deadAddress, PortAssignment.unique())));
+        peers.put(2L,
+                  new QuorumServer(2,
+                                   new InetSocketAddress(deadAddress, deadPort),
+                                   new InetSocketAddress(deadAddress, PortAssignment.unique()),
+                                   new InetSocketAddress(deadAddress, PortAssignment.unique())));
         peerTmpdir[2] = ClientBase.createTmpDir();
 
         QuorumPeer peer = new QuorumPeer(peers, peerTmpdir[1], peerTmpdir[1], peerClientPort[1], 3, 1, 1000, 2, 2, 2);
@@ -219,7 +225,7 @@ public class CnxManagerTest extends ZKTestCase {
         cnxManager.toSend(2L, createMsg(ServerState.LOOKING.ordinal(), 1, -1, 1));
         long end = Time.currentElapsedTime();
 
-        if ((end - begin) > 6000) {
+        if ((end - begin) > 10_000) {
             fail("Waited more than necessary");
         }
         cnxManager.halt();
@@ -245,15 +251,15 @@ public class CnxManagerTest extends ZKTestCase {
             LOG.error("Null listener when initializing cnx manager");
         }
 
-        int port = peers.get(peer.getId()).electionAddr.getPort();
-        LOG.info("Election port: {}", port);
+        InetSocketAddress address = peers.get(peer.getId()).electionAddr.getReachableOrOne();
+        LOG.info("Election port: {}", address.getPort());
 
         Thread.sleep(1000);
 
         SocketChannel sc = SocketChannel.open();
-        sc.socket().connect(peers.get(1L).electionAddr, 5000);
+        sc.socket().connect(address, 5000);
 
-        InetSocketAddress otherAddr = peers.get(2L).electionAddr;
+        InetSocketAddress otherAddr = peers.get(2L).electionAddr.getReachableOrOne();
         DataOutputStream dout = new DataOutputStream(sc.socket().getOutputStream());
         dout.writeLong(QuorumCnxManager.PROTOCOL_VERSION);
         dout.writeLong(2);
@@ -303,7 +309,7 @@ public class CnxManagerTest extends ZKTestCase {
         final QuorumPeer peer = new QuorumPeer(unresolvablePeers, ClientBase.createTmpDir(), ClientBase.createTmpDir(), 2181, 3, myid, 1000, 2, 2, 2);
         final QuorumCnxManager cnxManager = peer.createCnxnManager();
         final QuorumCnxManager.Listener listener = cnxManager.listener;
-        final AtomicBoolean errorHappend = new AtomicBoolean();
+        final AtomicBoolean errorHappend = new AtomicBoolean(false);
         listener.setSocketBindErrorHandler(() -> errorHappend.set(true));
         listener.start();
         // listener thread should stop and throws error which notify QuorumPeer about error.
@@ -335,13 +341,13 @@ public class CnxManagerTest extends ZKTestCase {
         } else {
             LOG.error("Null listener when initializing cnx manager");
         }
-        int port = peers.get(peer.getId()).electionAddr.getPort();
-        LOG.info("Election port: {}", port);
+        InetSocketAddress address = peers.get(peer.getId()).electionAddr.getReachableOrOne();
+        LOG.info("Election port: {}", address.getPort());
 
         Thread.sleep(1000);
 
         SocketChannel sc = SocketChannel.open();
-        sc.socket().connect(peers.get(1L).electionAddr, 5000);
+        sc.socket().connect(address, 5000);
 
         /*
          * Write id (3.4.6 protocol). This previously caused a NPE in
@@ -382,12 +388,12 @@ public class CnxManagerTest extends ZKTestCase {
         } else {
             LOG.error("Null listener when initializing cnx manager");
         }
-        int port = peers.get(peer.getId()).electionAddr.getPort();
-        LOG.info("Election port: {}", port);
+        InetSocketAddress address = peers.get(peer.getId()).electionAddr.getReachableOrOne();
+        LOG.info("Election port: {}", address.getPort());
         Thread.sleep(1000);
 
         Socket sock = new Socket();
-        sock.connect(peers.get(1L).electionAddr, 5000);
+        sock.connect(address, 5000);
         long begin = Time.currentElapsedTime();
         // Read without sending data. Verify timeout.
         cnxManager.receiveConnection(sock);
@@ -619,7 +625,7 @@ public class CnxManagerTest extends ZKTestCase {
         } catch (InitialMessage.InitialMessageException ex) {
         }
 
-        // good message
+        // good message, single election address
         try {
 
             hostport = "10.0.0.2:3888";
@@ -632,6 +638,30 @@ public class CnxManagerTest extends ZKTestCase {
             // now parse it
             din = new DataInputStream(new ByteArrayInputStream(bos.toByteArray()));
             msg = InitialMessage.parse(QuorumCnxManager.PROTOCOL_VERSION, din);
+            assertEquals(Long.valueOf(5), msg.sid);
+            assertEquals(Arrays.asList(new InetSocketAddress("10.0.0.2", 3888)), msg.electionAddr);
+        } catch (InitialMessage.InitialMessageException ex) {
+            fail(ex.toString());
+        }
+
+        // good message, multiple election addresses (ZOOKEEPER-3188)
+        try {
+
+            hostport = "1.1.1.1:9999|2.2.2.2:8888|3.3.3.3:7777";
+            bos = new ByteArrayOutputStream();
+            dout = new DataOutputStream(bos);
+            dout.writeLong(5L); // sid
+            dout.writeInt(hostport.getBytes().length);
+            dout.writeBytes(hostport);
+
+            // now parse it
+            din = new DataInputStream(new ByteArrayInputStream(bos.toByteArray()));
+            msg = InitialMessage.parse(QuorumCnxManager.PROTOCOL_VERSION, din);
+            assertEquals(Long.valueOf(5), msg.sid);
+            assertEquals(Arrays.asList(new InetSocketAddress("1.1.1.1", 9999),
+                                       new InetSocketAddress("2.2.2.2", 8888),
+                                       new InetSocketAddress("3.3.3.3", 7777)),
+                         msg.electionAddr);
         } catch (InitialMessage.InitialMessageException ex) {
             fail(ex.toString());
         }

+ 105 - 19
zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerTest.java

@@ -18,9 +18,13 @@
 
 package org.apache.zookeeper.server.quorum;
 
+import static java.util.Arrays.asList;
+import static java.util.Collections.emptySet;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 import java.io.BufferedOutputStream;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -30,10 +34,13 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Set;
 import org.apache.jute.BinaryInputArchive;
 import org.apache.jute.BinaryOutputArchive;
 import org.apache.zookeeper.ZKTestCase;
 import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.common.X509Exception;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.server.ZKDatabase;
 import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
@@ -71,17 +78,19 @@ public class LearnerTest extends ZKTestCase {
 
     }
 
-    static class TimeoutLearner extends Learner {
+    static class TestLearner extends Learner {
 
-        int passSocketConnectOnAttempt = 10;
-        int socketConnectAttempt = 0;
-        long timeMultiplier = 0;
+        private int passSocketConnectOnAttempt = 10;
+        private int socketConnectAttempt = 0;
+        private long timeMultiplier = 0;
+        private Socket socketToBeCreated = null;
+        private Set<InetSocketAddress> unreachableAddresses = emptySet();
 
-        public void setTimeMultiplier(long multiplier) {
+        private void setTimeMultiplier(long multiplier) {
             timeMultiplier = multiplier;
         }
 
-        public void setPassConnectAttempt(int num) {
+        private void setPassConnectAttempt(int num) {
             passSocketConnectOnAttempt = num;
         }
 
@@ -89,22 +98,39 @@ public class LearnerTest extends ZKTestCase {
             return socketConnectAttempt * timeMultiplier;
         }
 
-        protected int getSockConnectAttempt() {
+        private int getSockConnectAttempt() {
             return socketConnectAttempt;
         }
 
+        private void setSocketToBeCreated(Socket socketToBeCreated) {
+            this.socketToBeCreated = socketToBeCreated;
+        }
+
+        private void setUnreachableAddresses(Set<InetSocketAddress> unreachableAddresses) {
+            this.unreachableAddresses = unreachableAddresses;
+        }
+
         @Override
         protected void sockConnect(Socket sock, InetSocketAddress addr, int timeout) throws IOException {
-            if (++socketConnectAttempt < passSocketConnectOnAttempt) {
-                throw new IOException("Test injected Socket.connect() error.");
+            synchronized (this) {
+                if (++socketConnectAttempt < passSocketConnectOnAttempt || unreachableAddresses.contains(addr)) {
+                    throw new IOException("Test injected Socket.connect() error.");
+                }
             }
         }
 
+        @Override
+        protected Socket createSocket() throws X509Exception, IOException {
+            if (socketToBeCreated != null) {
+                return socketToBeCreated;
+            }
+            return super.createSocket();
+        }
     }
 
     @Test(expected = IOException.class)
     public void connectionRetryTimeoutTest() throws Exception {
-        Learner learner = new TimeoutLearner();
+        Learner learner = new TestLearner();
         learner.self = new QuorumPeer();
         learner.self.setTickTime(2000);
         learner.self.setInitLimit(5);
@@ -114,11 +140,12 @@ public class LearnerTest extends ZKTestCase {
         InetSocketAddress addr = new InetSocketAddress(1111);
 
         // we expect this to throw an IOException since we're faking socket connect errors every time
-        learner.connectToLeader(addr, "");
+        learner.connectToLeader(new MultipleAddresses(addr), "");
     }
+
     @Test
     public void connectionInitLimitTimeoutTest() throws Exception {
-        TimeoutLearner learner = new TimeoutLearner();
+        TestLearner learner = new TestLearner();
         learner.self = new QuorumPeer();
         learner.self.setTickTime(2000);
         learner.self.setInitLimit(5);
@@ -128,24 +155,83 @@ public class LearnerTest extends ZKTestCase {
         InetSocketAddress addr = new InetSocketAddress(1111);
 
         // pretend each connect attempt takes 4000 milliseconds
-        learner.setTimeMultiplier((long) 4000 * 1000000);
+        learner.setTimeMultiplier((long) 4000 * 1000_000);
 
         learner.setPassConnectAttempt(5);
 
         // we expect this to throw an IOException since we're faking socket connect errors every time
         try {
-            learner.connectToLeader(addr, "");
+            learner.connectToLeader(new MultipleAddresses(addr), "");
             fail("should have thrown IOException!");
         } catch (IOException e) {
             //good, wanted to see that, let's make sure we ran out of time
-            assertTrue(learner.nanoTime() > 2000 * 5 * 1000000);
+            assertTrue(learner.nanoTime() > 2000 * 5 * 1000_000);
             assertEquals(3, learner.getSockConnectAttempt());
         }
     }
 
+    @Test
+    public void shouldTryMultipleAddresses() throws Exception {
+        TestLearner learner = new TestLearner();
+        learner.self = new QuorumPeer();
+        learner.self.setTickTime(2000);
+        learner.self.setInitLimit(5);
+        learner.self.setSyncLimit(2);
+
+        // this addr won't even be used since we fake the Socket.connect
+        InetSocketAddress addrA = new InetSocketAddress(1111);
+        InetSocketAddress addrB = new InetSocketAddress(2222);
+        InetSocketAddress addrC = new InetSocketAddress(3333);
+        InetSocketAddress addrD = new InetSocketAddress(4444);
+
+        // we will never pass (don't allow successful socker.connect) during this test
+        learner.setPassConnectAttempt(100);
+
+        // we expect this to throw an IOException since we're faking socket connect errors every time
+        try {
+            learner.connectToLeader(new MultipleAddresses(asList(addrA, addrB, addrC, addrD)), "");
+            fail("should have thrown IOException!");
+        } catch (IOException e) {
+            //good, wanted to see the IOException, let's make sure we tried each address 5 times
+            assertEquals(4 * 5, learner.getSockConnectAttempt());
+        }
+    }
+
+    @Test
+    public void multipleAddressesSomeAreFailing() throws Exception {
+        TestLearner learner = new TestLearner();
+        learner.self = new QuorumPeer();
+        learner.self.setTickTime(2000);
+        learner.self.setInitLimit(5);
+        learner.self.setSyncLimit(2);
+
+        // these addresses won't even be used since we fake the Socket.connect
+        InetSocketAddress addrWorking = new InetSocketAddress(1111);
+        InetSocketAddress addrBadA = new InetSocketAddress(2222);
+        InetSocketAddress addrBadB = new InetSocketAddress(3333);
+        InetSocketAddress addrBadC = new InetSocketAddress(4444);
+
+        // we will emulate socket connection error for each 'bad' address
+        learner.setUnreachableAddresses(new HashSet<>(asList(addrBadA, addrBadB, addrBadC)));
+
+        // all connection attempts should succeed (if it is not an unreachable address)
+        learner.setPassConnectAttempt(0);
+
+        // initialize a mock socket, created by the Learner
+        Socket mockSocket = mock(Socket.class);
+        when(mockSocket.isConnected()).thenReturn(true);
+        learner.setSocketToBeCreated(mockSocket);
+
+
+        // we expect this to not throw an IOException since there is a single working address
+        learner.connectToLeader(new MultipleAddresses(asList(addrBadA, addrBadB, addrBadC, addrWorking)), "");
+
+        assertEquals("Learner connected to the wrong address", learner.getSocket(), mockSocket);
+    }
+
     @Test
     public void connectToLearnerMasterLimitTest() throws Exception {
-        TimeoutLearner learner = new TimeoutLearner();
+        TestLearner learner = new TestLearner();
         learner.self = new QuorumPeer();
         learner.self.setTickTime(2000);
         learner.self.setInitLimit(2);
@@ -153,14 +239,14 @@ public class LearnerTest extends ZKTestCase {
         learner.self.setConnectToLearnerMasterLimit(5);
 
         InetSocketAddress addr = new InetSocketAddress(1111);
-        learner.setTimeMultiplier((long) 4000 * 1000000);
+        learner.setTimeMultiplier((long) 4000 * 1000_000);
         learner.setPassConnectAttempt(5);
 
         try {
-            learner.connectToLeader(addr, "");
+            learner.connectToLeader(new MultipleAddresses(addr), "");
             fail("should have thrown IOException!");
         } catch (IOException e) {
-            assertTrue(learner.nanoTime() > 2000 * 5 * 1000000);
+            assertTrue(learner.nanoTime() > 2000 * 5 * 1000_000);
             assertEquals(3, learner.getSockConnectAttempt());
         }
     }

+ 201 - 0
zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java

@@ -0,0 +1,201 @@
+/**
+ * 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.zookeeper.server.quorum;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.NoRouteToHostException;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.zookeeper.PortAssignment;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class MultipleAddressesTest {
+
+    public static final int PORTS_AMOUNT = 10;
+
+    @Test
+    public void testIsEmpty() {
+        MultipleAddresses multipleAddresses = new MultipleAddresses();
+        Assert.assertTrue(multipleAddresses.isEmpty());
+
+        multipleAddresses.addAddress(new InetSocketAddress(22));
+        Assert.assertFalse(multipleAddresses.isEmpty());
+    }
+
+    @Test
+    public void testGetAllAddresses() {
+        List<InetSocketAddress> addresses = getAddressList();
+        MultipleAddresses multipleAddresses = new MultipleAddresses(addresses);
+
+        Assert.assertTrue(CollectionUtils.isEqualCollection(addresses, multipleAddresses.getAllAddresses()));
+
+        multipleAddresses.addAddress(addresses.get(1));
+        Assert.assertTrue(CollectionUtils.isEqualCollection(addresses, multipleAddresses.getAllAddresses()));
+    }
+
+    @Test
+    public void testGetAllHostStrings() {
+        List<InetSocketAddress> addresses = getAddressList();
+        List<String> hostStrings = getHostStrings(addresses);
+        MultipleAddresses multipleAddresses = new MultipleAddresses(addresses);
+
+        Assert.assertTrue(CollectionUtils.isEqualCollection(hostStrings, multipleAddresses.getAllHostStrings()));
+
+        multipleAddresses.addAddress(addresses.get(addresses.size() - 1));
+        Assert.assertTrue(CollectionUtils.isEqualCollection(hostStrings, multipleAddresses.getAllHostStrings()));
+    }
+
+    @Test
+    public void testGetAllPorts() {
+        List<Integer> ports = getPortList();
+        MultipleAddresses multipleAddresses = new MultipleAddresses(getAddressList(ports));
+
+        Assert.assertTrue(CollectionUtils.isEqualCollection(ports, multipleAddresses.getAllPorts()));
+
+        multipleAddresses.addAddress(new InetSocketAddress("localhost", ports.get(ports.size() - 1)));
+        Assert.assertTrue(CollectionUtils.isEqualCollection(ports, multipleAddresses.getAllPorts()));
+    }
+
+    @Test
+    public void testGetWildcardAddresses() {
+        List<Integer> ports = getPortList();
+        List<InetSocketAddress> addresses = getAddressList(ports);
+        MultipleAddresses multipleAddresses = new MultipleAddresses(addresses);
+        List<InetSocketAddress> allAddresses = ports.stream().map(InetSocketAddress::new).collect(Collectors.toList());
+
+        Assert.assertTrue(CollectionUtils.isEqualCollection(allAddresses, multipleAddresses.getWildcardAddresses()));
+
+        multipleAddresses.addAddress(new InetSocketAddress("localhost", ports.get(ports.size() - 1)));
+        Assert.assertTrue(CollectionUtils.isEqualCollection(allAddresses, multipleAddresses.getWildcardAddresses()));
+    }
+
+    @Test
+    public void testGetValidAddress() throws NoRouteToHostException {
+        List<InetSocketAddress> addresses = getAddressList();
+        MultipleAddresses multipleAddresses = new MultipleAddresses(addresses);
+
+        Assert.assertTrue(addresses.contains(multipleAddresses.getReachableAddress()));
+    }
+
+    @Test(expected = NoRouteToHostException.class)
+    public void testGetValidAddressWithNotValid() throws NoRouteToHostException {
+        MultipleAddresses multipleAddresses = new MultipleAddresses(new InetSocketAddress("10.0.0.1", 22));
+        multipleAddresses.getReachableAddress();
+    }
+
+    @Test
+    public void testRecreateSocketAddresses() throws UnknownHostException {
+        List<InetSocketAddress> searchedAddresses = Arrays.stream(InetAddress.getAllByName("google.com"))
+                .map(addr -> new InetSocketAddress(addr, 222)).collect(Collectors.toList());
+
+        MultipleAddresses multipleAddresses = new MultipleAddresses(searchedAddresses.get(searchedAddresses.size() - 1));
+        List<InetSocketAddress> addresses = new ArrayList<>(multipleAddresses.getAllAddresses());
+
+        Assert.assertEquals(1, addresses.size());
+        Assert.assertEquals(searchedAddresses.get(searchedAddresses.size() - 1), addresses.get(0));
+
+        multipleAddresses.recreateSocketAddresses();
+
+        addresses = new ArrayList<>(multipleAddresses.getAllAddresses());
+        Assert.assertEquals(1, addresses.size());
+        Assert.assertEquals(searchedAddresses.get(0), addresses.get(0));
+    }
+
+    @Test
+    public void testRecreateSocketAddressesWithWrongAddresses() {
+        InetSocketAddress address = new InetSocketAddress("locahost", 222);
+        MultipleAddresses multipleAddresses = new MultipleAddresses(address);
+        multipleAddresses.recreateSocketAddresses();
+
+        Assert.assertEquals(address, multipleAddresses.getOne());
+    }
+
+    @Test
+    public void testAlwaysGetReachableAddress() throws Exception{
+        InetSocketAddress reachableHost = new InetSocketAddress("127.0.0.1", 1234);
+        InetSocketAddress unreachableHost1 = new InetSocketAddress("unreachable1.address.zookeeper.apache.com", 1234);
+        InetSocketAddress unreachableHost2 = new InetSocketAddress("unreachable2.address.zookeeper.apache.com", 1234);
+        InetSocketAddress unreachableHost3 = new InetSocketAddress("unreachable3.address.zookeeper.apache.com", 1234);
+
+        MultipleAddresses multipleAddresses = new MultipleAddresses(
+          Arrays.asList(unreachableHost1, unreachableHost2, unreachableHost3, reachableHost));
+
+        // we call the getReachableAddress() function multiple times, to make sure we
+        // always got back a reachable address and not just a random one
+        for (int i = 0; i < 10; i++) {
+            Assert.assertEquals(reachableHost, multipleAddresses.getReachableAddress());
+        }
+    }
+
+    @Test
+    public void testGetAllReachableAddresses() throws Exception {
+        InetSocketAddress reachableHost1 = new InetSocketAddress("127.0.0.1", 1234);
+        InetSocketAddress reachableHost2 = new InetSocketAddress("127.0.0.1", 2345);
+        InetSocketAddress unreachableHost1 = new InetSocketAddress("unreachable1.address.zookeeper.apache.com", 1234);
+        InetSocketAddress unreachableHost2 = new InetSocketAddress("unreachable2.address.zookeeper.apache.com", 1234);
+
+        MultipleAddresses multipleAddresses = new MultipleAddresses(
+          Arrays.asList(unreachableHost1, unreachableHost2, reachableHost1, reachableHost2));
+
+        Set<InetSocketAddress> reachableHosts = new HashSet<>(Arrays.asList(reachableHost1, reachableHost2));
+        Assert.assertEquals(reachableHosts, multipleAddresses.getAllReachableAddresses());
+    }
+
+    @Test
+    public void testEquals() {
+        List<InetSocketAddress> addresses = getAddressList();
+
+        MultipleAddresses multipleAddresses = new MultipleAddresses(addresses);
+        MultipleAddresses multipleAddressesEquals = new MultipleAddresses(addresses);
+
+        Assert.assertEquals(multipleAddresses, multipleAddressesEquals);
+
+        MultipleAddresses multipleAddressesNotEquals = new MultipleAddresses(getAddressList());
+
+        Assert.assertNotEquals(multipleAddresses, multipleAddressesNotEquals);
+    }
+
+    public List<Integer> getPortList() {
+        return IntStream.range(0, PORTS_AMOUNT).mapToObj(i -> PortAssignment.unique()).collect(Collectors.toList());
+    }
+
+    public List<InetSocketAddress> getAddressList() {
+        return getAddressList(getPortList());
+    }
+
+    public List<InetSocketAddress> getAddressList(List<Integer> ports) {
+        return IntStream.range(0, ports.size())
+                .mapToObj(i -> new InetSocketAddress("127.0.0." + i, ports.get(i))).collect(Collectors.toList());
+    }
+
+    private List<String> getHostStrings(List<InetSocketAddress> addresses) {
+        return IntStream.range(0, addresses.size())
+                .mapToObj(i -> "127.0.0." + i).collect(Collectors.toList());
+    }
+
+}

+ 303 - 0
zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainMultiAddressTest.java

@@ -0,0 +1,303 @@
+/*
+ * 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.zookeeper.server.quorum;
+
+import static org.junit.Assert.assertEquals;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.DummyWatcher;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.admin.ZooKeeperAdmin;
+import org.apache.zookeeper.test.ClientBase;
+import org.apache.zookeeper.test.ReconfigTest;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+
+public class QuorumPeerMainMultiAddressTest extends QuorumPeerTestBase {
+
+  private static final int FIRST_SERVER = 0;
+  private static final int SECOND_SERVER = 1;
+  private static final int THIRD_SERVER = 2;
+  private static final int FIRST_ADDRESS = 0;
+  private static final int SECOND_ADDRESS = 1;
+  private static final String UNREACHABLE_HOST = "invalid.hostname.unreachable.com";
+  private static final String IPV6_LOCALHOST = "[0:0:0:0:0:0:0:1]";
+
+  // IPv4 by default, change to IPV6_LOCALHOST to test with servers binding to IPv6
+  private String hostName = "127.0.0.1";
+
+  private int zNodeId = 0;
+
+  @Before
+  public void setUp() throws Exception {
+    ClientBase.setupTestEnv();
+    System.setProperty("zookeeper.DigestAuthenticationProvider.superDigest", "super:D/InIHSb7yEEbrWz8b9l71RjZJU="/* password is 'test'*/);
+    QuorumPeerConfig.setReconfigEnabled(true);
+
+    // just to get rid of the unrelated 'InstanceAlreadyExistsException' in the logs
+    System.setProperty("zookeeper.jmx.log4j.disable", "true");
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    super.tearDown();
+    System.clearProperty("zookeeper.jmx.log4j.disable");
+  }
+
+
+  @Test
+  public void shouldStartClusterWithMultipleAddresses() throws Exception {
+    // we have three ZK servers, each server has two quorumPort and two electionPort registered
+    QuorumServerConfigBuilder quorumConfig = new QuorumServerConfigBuilder(hostName, 3, 2);
+
+    // we launch the three servers, each server having the same configuration
+    QuorumServerConfigBuilder builderForServer1 = new QuorumServerConfigBuilder(quorumConfig);
+    QuorumServerConfigBuilder builderForServer2 = new QuorumServerConfigBuilder(quorumConfig);
+    QuorumServerConfigBuilder builderForServer3 = new QuorumServerConfigBuilder(quorumConfig);
+    launchServers(Arrays.asList(builderForServer1, builderForServer2, builderForServer3));
+
+    checkIfZooKeeperQuorumWorks(quorumConfig);
+  }
+
+
+  @Test
+  public void shouldStartClusterWithMultipleAddresses_IPv6() throws Exception {
+    hostName = IPV6_LOCALHOST;
+
+    shouldStartClusterWithMultipleAddresses();
+  }
+
+
+  @Test
+  public void shouldStartClusterWhenSomeAddressesAreUnreachable() throws Exception {
+    // we have three ZK servers, each server has two quorumPort and two electionPort registered
+    // in the config we misconfigure one of the addresses for each servers
+    QuorumServerConfigBuilder quorumConfig = new QuorumServerConfigBuilder(hostName, 3, 2)
+      .changeHostName(FIRST_SERVER, SECOND_ADDRESS, UNREACHABLE_HOST)
+      .changeHostName(SECOND_SERVER, SECOND_ADDRESS, UNREACHABLE_HOST)
+      .changeHostName(THIRD_SERVER, SECOND_ADDRESS, UNREACHABLE_HOST);
+
+    // we prepare the same initial config for all the three servers
+    QuorumServerConfigBuilder builderForServer1 = new QuorumServerConfigBuilder(quorumConfig);
+    QuorumServerConfigBuilder builderForServer2 = new QuorumServerConfigBuilder(quorumConfig);
+    QuorumServerConfigBuilder builderForServer3 = new QuorumServerConfigBuilder(quorumConfig);
+
+    // we test here:
+    // - if the Leader can bind to the correct address and not die with BindException or
+    //   SocketException for trying to bind to a wrong address / port
+    // - if the ZK server can 'select' the correct address to connect when trying to form a quorum
+    //   with the other servers
+    launchServers(Arrays.asList(builderForServer1, builderForServer2, builderForServer3));
+
+    checkIfZooKeeperQuorumWorks(quorumConfig);
+  }
+
+
+  @Test
+  public void shouldStartClusterWhenSomeAddressesAreUnreachable_IPv6() throws Exception {
+    hostName = IPV6_LOCALHOST;
+
+    shouldStartClusterWhenSomeAddressesAreUnreachable();
+  }
+
+
+  @Test
+  public void shouldReconfigIncrementallyByAddingMoreAddresses() throws Exception {
+    // we have three ZK servers, each server has two quorumPort and two electionPort registered
+    QuorumServerConfigBuilder initialQuorumConfig = new QuorumServerConfigBuilder(hostName, 3, 2);
+
+    // we launch the three servers, each server should use the same initial config
+    launchServers(Arrays.asList(initialQuorumConfig, initialQuorumConfig, initialQuorumConfig));
+
+    checkIfZooKeeperQuorumWorks(initialQuorumConfig);
+
+    // we create a new config where we add a new address to each server with random available ports
+    QuorumServerConfigBuilder newQuorumConfig = new QuorumServerConfigBuilder(initialQuorumConfig)
+      .addNewServerAddress(FIRST_SERVER);
+
+
+    ZooKeeperAdmin zkAdmin = newZooKeeperAdmin(initialQuorumConfig);
+
+    // initiating a new incremental reconfig, by using the updated ports
+    ReconfigTest.reconfig(zkAdmin, newQuorumConfig.buildAsStringList(), null, null, -1);
+
+    checkIfZooKeeperQuorumWorks(newQuorumConfig);
+  }
+
+
+  @Test
+  public void shouldReconfigIncrementallyByDeletingSomeAddresses() throws Exception {
+    // we have three ZK servers, each server has three quorumPort and three electionPort registered
+    QuorumServerConfigBuilder initialQuorumConfig = new QuorumServerConfigBuilder(hostName, 3, 3);
+
+    // we launch the three servers, each server should use the same initial config
+    launchServers(Arrays.asList(initialQuorumConfig, initialQuorumConfig, initialQuorumConfig));
+
+    checkIfZooKeeperQuorumWorks(initialQuorumConfig);
+
+    // we create a new config where we delete a few address from each server
+    QuorumServerConfigBuilder newQuorumConfig = new QuorumServerConfigBuilder(initialQuorumConfig)
+      .deleteLastServerAddress(FIRST_SERVER)
+      .deleteLastServerAddress(SECOND_SERVER)
+      .deleteLastServerAddress(SECOND_SERVER)
+      .deleteLastServerAddress(THIRD_SERVER);
+
+    ZooKeeperAdmin zkAdmin = newZooKeeperAdmin(initialQuorumConfig);
+
+    // initiating a new incremental reconfig, by using the updated ports
+    ReconfigTest.reconfig(zkAdmin, newQuorumConfig.buildAsStringList(), null, null, -1);
+
+    checkIfZooKeeperQuorumWorks(newQuorumConfig);
+  }
+
+  @Test
+  public void shouldReconfigNonIncrementally() throws Exception {
+    // we have three ZK servers, each server has two quorumPort and two electionPort registered
+    QuorumServerConfigBuilder initialQuorumConfig = new QuorumServerConfigBuilder(hostName, 3, 2);
+
+    // we launch the three servers, each server should use the same initial config
+    launchServers(Arrays.asList(initialQuorumConfig, initialQuorumConfig, initialQuorumConfig));
+
+    checkIfZooKeeperQuorumWorks(initialQuorumConfig);
+
+    // we create a new config where we delete and add a few address for each server
+    QuorumServerConfigBuilder newQuorumConfig = new QuorumServerConfigBuilder(initialQuorumConfig)
+      .deleteLastServerAddress(FIRST_SERVER)
+      .deleteLastServerAddress(SECOND_SERVER)
+      .deleteLastServerAddress(SECOND_SERVER)
+      .deleteLastServerAddress(THIRD_SERVER)
+      .addNewServerAddress(SECOND_SERVER)
+      .addNewServerAddress(THIRD_SERVER);
+
+    ZooKeeperAdmin zkAdmin = newZooKeeperAdmin(initialQuorumConfig);
+
+    // initiating a new non-incremental reconfig, by using the updated ports
+    ReconfigTest.reconfig(zkAdmin, null, null, newQuorumConfig.buildAsStringList(), -1);
+
+    checkIfZooKeeperQuorumWorks(newQuorumConfig);
+  }
+
+
+  @Test
+  public void shouldReconfigIncrementally_IPv6() throws Exception {
+
+    hostName = IPV6_LOCALHOST;
+
+    // we have three ZK servers, each server has two quorumPort and two electionPort registered
+    QuorumServerConfigBuilder initialQuorumConfig = new QuorumServerConfigBuilder(hostName, 3, 2);
+
+    // we launch the three servers, each server should use the same initial config
+    launchServers(Arrays.asList(initialQuorumConfig, initialQuorumConfig, initialQuorumConfig));
+
+    checkIfZooKeeperQuorumWorks(initialQuorumConfig);
+
+    // we create a new config where we delete and add a few address for each server
+    QuorumServerConfigBuilder newQuorumConfig = new QuorumServerConfigBuilder(initialQuorumConfig)
+      .deleteLastServerAddress(FIRST_SERVER)
+      .deleteLastServerAddress(SECOND_SERVER)
+      .deleteLastServerAddress(SECOND_SERVER)
+      .deleteLastServerAddress(THIRD_SERVER)
+      .addNewServerAddress(SECOND_SERVER)
+      .addNewServerAddress(THIRD_SERVER);
+
+    ZooKeeperAdmin zkAdmin = newZooKeeperAdmin(initialQuorumConfig);
+
+    // initiating a new incremental reconfig, by using the updated ports
+    ReconfigTest.reconfig(zkAdmin, newQuorumConfig.buildAsStringList(), null, null, -1);
+
+    checkIfZooKeeperQuorumWorks(newQuorumConfig);
+  }
+
+
+  private void launchServers(List<QuorumServerConfigBuilder> builders) throws IOException, InterruptedException {
+
+    numServers = builders.size();
+
+    servers = new Servers();
+    servers.clientPorts = new int[numServers];
+    servers.mt = new MainThread[numServers];
+    servers.zk = new ZooKeeper[numServers];
+
+    for (int i = 0; i < numServers; i++) {
+      QuorumServerConfigBuilder quorumServerConfigBuilder = builders.get(i);
+      String quorumCfgSection = quorumServerConfigBuilder.build();
+      LOG.info(String.format("starting server %d with quorum config:\n%s", i, quorumCfgSection));
+      servers.clientPorts[i] = quorumServerConfigBuilder.getClientPort(i);
+      servers.mt[i] = new MainThread(i, servers.clientPorts[i], quorumCfgSection);
+      servers.mt[i].start();
+      servers.restartClient(i, this);
+    }
+
+    waitForAll(servers, ZooKeeper.States.CONNECTED);
+
+    for (int i = 0; i < numServers; i++) {
+      servers.zk[i].close(5000);
+    }
+  }
+
+  private void checkIfZooKeeperQuorumWorks(QuorumServerConfigBuilder builder) throws IOException,
+    InterruptedException, KeeperException {
+
+    LOG.info("starting to verify if Quorum works");
+    zNodeId += 1;
+    String zNodePath = "/foo_" + zNodeId;
+    ZooKeeper zk = connectToZkServer(builder, FIRST_SERVER);
+    zk.create(zNodePath, "foobar1".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+    assertEquals(new String(zk.getData(zNodePath, null, null)), "foobar1");
+    zk.close(1000);
+
+
+    zk = connectToZkServer(builder, SECOND_SERVER);
+    assertEquals(new String(zk.getData(zNodePath, null, null)), "foobar1");
+    zk.close(1000);
+
+    zk = connectToZkServer(builder, THIRD_SERVER);
+    assertEquals(new String(zk.getData(zNodePath, null, null)), "foobar1");
+    zk.close(1000);
+
+    LOG.info("Quorum verification finished successfully");
+
+  }
+
+  private ZooKeeper connectToZkServer(QuorumServerConfigBuilder builder, int serverId) throws IOException, InterruptedException {
+    QuorumServerConfigBuilder.ServerAddress server = builder.getServerAddress(serverId, FIRST_ADDRESS);
+    int clientPort = builder.getClientPort(serverId);
+    ZooKeeper zk = new ZooKeeper(server.getHost() + ":" + clientPort, ClientBase.CONNECTION_TIMEOUT, this);
+    waitForOne(zk, ZooKeeper.States.CONNECTED);
+    return zk;
+  }
+
+  private ZooKeeperAdmin newZooKeeperAdmin(
+    QuorumServerConfigBuilder quorumConfig) throws IOException {
+    ZooKeeperAdmin zkAdmin = new ZooKeeperAdmin(
+      hostName + ":" + quorumConfig.getClientPort(FIRST_SERVER),
+      ClientBase.CONNECTION_TIMEOUT,
+      DummyWatcher.INSTANCE);
+    zkAdmin.addAuthInfo("digest", "super:test".getBytes());
+    return zkAdmin;
+  }
+
+
+}

+ 1 - 2
zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java

@@ -378,7 +378,6 @@ public class QuorumPeerMainTest extends QuorumPeerTestBase {
                                                                             != null);
 
             // to keep the quorum peer running and force it to go into the looking state, we kill leader election
-            // and close the connection to the leader
             servers.mt[falseLeader].main.quorumPeer.electionAlg.shutdown();
             servers.mt[falseLeader].main.quorumPeer.follower.getSocket().close();
 
@@ -473,7 +472,7 @@ public class QuorumPeerMainTest extends QuorumPeerTestBase {
         LineNumberReader r = new LineNumberReader(new StringReader(os.toString()));
         String line;
         boolean found = false;
-        Pattern p = Pattern.compile(".*Cannot open channel to .* at election address .*");
+        Pattern p = Pattern.compile(".*None of the addresses .* are reachable for sid 2");
         while ((line = r.readLine()) != null) {
             found = p.matcher(line).matches();
             if (found) {

+ 73 - 3
zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumSSLTest.java

@@ -378,6 +378,8 @@ public class QuorumSSLTest extends QuorumPeerTestBase {
     }
 
     private String generateQuorumConfiguration() {
+        StringBuilder sb = new StringBuilder();
+
         int portQp1 = PortAssignment.unique();
         int portQp2 = PortAssignment.unique();
         int portQp3 = PortAssignment.unique();
@@ -386,9 +388,35 @@ public class QuorumSSLTest extends QuorumPeerTestBase {
         int portLe2 = PortAssignment.unique();
         int portLe3 = PortAssignment.unique();
 
-        return "server.1=127.0.0.1:" + (portQp1) + ":" + (portLe1) + ";" + clientPortQp1
-               + "\n" + "server.2=127.0.0.1:" + (portQp2) + ":" + (portLe2) + ";" + clientPortQp2
-               + "\n" + "server.3=127.0.0.1:" + (portQp3) + ":" + (portLe3) + ";" + clientPortQp3;
+        sb.append(String.format("server.1=127.0.0.1:%d:%d;%d\n", portQp1, portLe1, clientPortQp1));
+        sb.append(String.format("server.2=127.0.0.1:%d:%d;%d\n", portQp2, portLe2, clientPortQp2));
+        sb.append(String.format("server.3=127.0.0.1:%d:%d;%d\n", portQp3, portLe3, clientPortQp3));
+
+        return sb.toString();
+    }
+
+    private String generateMultiAddressQuorumConfiguration() {
+        StringBuilder sb = new StringBuilder();
+
+        int portQp1a = PortAssignment.unique();
+        int portQp1b = PortAssignment.unique();
+        int portQp2a = PortAssignment.unique();
+        int portQp2b = PortAssignment.unique();
+        int portQp3a = PortAssignment.unique();
+        int portQp3b = PortAssignment.unique();
+
+        int portLe1a = PortAssignment.unique();
+        int portLe1b = PortAssignment.unique();
+        int portLe2a = PortAssignment.unique();
+        int portLe2b = PortAssignment.unique();
+        int portLe3a = PortAssignment.unique();
+        int portLe3b = PortAssignment.unique();
+
+        sb.append(String.format("server.1=127.0.0.1:%d:%d|127.0.0.1:%d:%d;%d\n", portQp1a, portLe1a, portQp1b, portLe1b, clientPortQp1));
+        sb.append(String.format("server.2=127.0.0.1:%d:%d|127.0.0.1:%d:%d;%d\n", portQp2a, portLe2a, portQp2b, portLe2b, clientPortQp2));
+        sb.append(String.format("server.3=127.0.0.1:%d:%d|127.0.0.1:%d:%d;%d\n", portQp3a, portLe3a, portQp3b, portLe3b, clientPortQp3));
+
+        return sb.toString();
     }
 
     public void setSSLSystemProperties() {
@@ -449,6 +477,30 @@ public class QuorumSSLTest extends QuorumPeerTestBase {
         assertFalse(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp3, CONNECTION_TIMEOUT));
     }
 
+
+    @Test
+    public void testQuorumSSLWithMultipleAddresses() throws Exception {
+        quorumConfiguration = generateMultiAddressQuorumConfiguration();
+
+        q1 = new MainThread(1, clientPortQp1, quorumConfiguration, SSL_QUORUM_ENABLED);
+        q2 = new MainThread(2, clientPortQp2, quorumConfiguration, SSL_QUORUM_ENABLED);
+
+        q1.start();
+        q2.start();
+
+        assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp1, CONNECTION_TIMEOUT));
+        assertTrue(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp2, CONNECTION_TIMEOUT));
+
+        clearSSLSystemProperties();
+
+        // This server should fail to join the quorum as it is not using ssl.
+        q3 = new MainThread(3, clientPortQp3, quorumConfiguration);
+        q3.start();
+
+        assertFalse(ClientBase.waitForServerUp("127.0.0.1:" + clientPortQp3, CONNECTION_TIMEOUT));
+    }
+
+
     @Test
     public void testRollingUpgrade() throws Exception {
         // Form a quorum without ssl
@@ -544,6 +596,24 @@ public class QuorumSSLTest extends QuorumPeerTestBase {
         testHostnameVerification(badhostnameKeystorePath, false);
     }
 
+    @Test
+    public void testHostnameVerificationForInvalidMultiAddressServerConfig() throws Exception {
+        quorumConfiguration = generateMultiAddressQuorumConfiguration();
+
+        String badhostnameKeystorePath = tmpDir + "/badhost.jks";
+        X509Certificate badHostCert = buildEndEntityCert(
+            defaultKeyPair,
+            rootCertificate,
+            rootKeyPair.getPrivate(),
+            "bleepbloop",
+            "140.211.11.105",
+            null,
+            null);
+        writeKeystore(badHostCert, defaultKeyPair, badhostnameKeystorePath);
+
+        testHostnameVerification(badhostnameKeystorePath, false);
+    }
+
     @Test
     public void testHostnameVerificationWithInvalidIpAddressAndValidHostname() throws Exception {
         String badhostnameKeystorePath = tmpDir + "/badhost.jks";

+ 166 - 0
zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumServerConfigBuilder.java

@@ -0,0 +1,166 @@
+/*
+ * 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.zookeeper.server.quorum;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.zookeeper.PortAssignment;
+
+
+/*
+ * Helper class to build / change Quorum Config String, like:
+ * server.1=127.0.0.1:11228:11231|127.0.0.1:11230:11229;11227
+ * server.2=127.0.0.1:11338:11331|127.0.0.1:11330:11229;11337
+ *
+ */
+public class QuorumServerConfigBuilder {
+
+  // map of (serverId -> clientPort)
+  private final Map<Integer, Integer> clientIds = new HashMap<>();
+
+  // map of (serverId -> (ServerAddress=host,quorumPort,electionPort) )
+  private final Map<Integer, List<ServerAddress>> serverAddresses = new HashMap<>();
+  private final String hostName;
+  private final int numberOfServers;
+
+  public QuorumServerConfigBuilder(String hostName, int numberOfServers, int numberOfServerAddresses) {
+    this.numberOfServers = numberOfServers;
+    this.hostName = hostName;
+    for (int serverId = 0; serverId < numberOfServers; serverId++) {
+      clientIds.put(serverId, PortAssignment.unique());
+
+      List<ServerAddress> addresses = new ArrayList<>();
+      serverAddresses.put(serverId, addresses);
+
+      for (int serverAddressId = 0; serverAddressId < numberOfServerAddresses; serverAddressId++) {
+        addresses.add(new ServerAddress(hostName));
+      }
+
+    }
+  }
+
+  public QuorumServerConfigBuilder(QuorumServerConfigBuilder otherBuilder) {
+    this.numberOfServers = otherBuilder.clientIds.size();
+    this.clientIds.putAll(otherBuilder.clientIds);
+    this.hostName = otherBuilder.hostName;
+    for (int i : otherBuilder.serverAddresses.keySet()) {
+      List<ServerAddress> clonedServerAddresses = otherBuilder.serverAddresses.get(i).stream()
+        .map(ServerAddress::clone).collect(Collectors.toList());
+      this.serverAddresses.put(i, clonedServerAddresses);
+    }
+  }
+
+  public int getClientPort(int serverId) {
+    return clientIds.get(serverId);
+  }
+
+  public ServerAddress getServerAddress(int serverId, int addressId) {
+    return serverAddresses.get(serverId).get(addressId);
+  }
+
+  public QuorumServerConfigBuilder changeHostName(int serverId, int addressId, String hostName) {
+    serverAddresses.get(serverId).get(addressId).setHost(hostName);
+    return this;
+  }
+
+  public QuorumServerConfigBuilder changeQuorumPort(int serverId, int addressId, int quorumPort) {
+    serverAddresses.get(serverId).get(addressId).setQuorumPort(quorumPort);
+    return this;
+  }
+
+  public QuorumServerConfigBuilder changeElectionPort(int serverId, int addressId, int electionPort) {
+    serverAddresses.get(serverId).get(addressId).setElectionPort(electionPort);
+    return this;
+  }
+
+  public QuorumServerConfigBuilder addNewServerAddress(int serverId) {
+    serverAddresses.get(serverId).add(new ServerAddress(hostName));
+    return this;
+  }
+
+  public QuorumServerConfigBuilder deleteLastServerAddress(int serverId) {
+    serverAddresses.get(serverId).remove(serverAddresses.get(serverId).size() - 1);
+    return this;
+  }
+
+  public String build() {
+    return String.join("\n", buildAsStringList());
+  }
+
+  public List<String> buildAsStringList() {
+    List<String> result = new ArrayList<>(numberOfServers);
+
+    for (int serverId = 0; serverId < numberOfServers; serverId++) {
+      String s = serverAddresses.get(serverId).stream()
+        .map(ServerAddress::toString)
+        .collect(Collectors.joining("|"));
+
+      result.add(String.format("server.%d=%s;%d", serverId, s, clientIds.get(serverId)));
+    }
+
+    return result;
+  }
+
+  public static class ServerAddress {
+    private String host;
+    private int quorumPort;
+    private int electionPort;
+
+    private ServerAddress(String host) {
+      this(host, PortAssignment.unique(), PortAssignment.unique());
+
+    }
+
+    private ServerAddress(String host, int quorumPort, int electionPort) {
+      this.host = host;
+      this.quorumPort = quorumPort;
+      this.electionPort = electionPort;
+    }
+
+    public String getHost() {
+      return host;
+    }
+
+    private void setHost(String host) {
+      this.host = host;
+    }
+
+    private void setQuorumPort(int quorumPort) {
+      this.quorumPort = quorumPort;
+    }
+
+    private void setElectionPort(int electionPort) {
+      this.electionPort = electionPort;
+    }
+
+    @Override
+    public ServerAddress clone() {
+      return new ServerAddress(host, quorumPort, electionPort);
+    }
+
+    @Override
+    public String toString() {
+      return String.format("%s:%d:%d", host, quorumPort, electionPort);
+    }
+  }
+}
+

+ 2 - 2
zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigFailureCasesTest.java

@@ -77,8 +77,8 @@ public class ReconfigFailureCasesTest extends QuorumPeerTestBase {
 
         for (int i = 1; i <= 5; i++) {
             members.add("server." + i + "=127.0.0.1:"
-                        + qu.getPeer(i).peer.getQuorumAddress().getPort()
-                        + ":" + qu.getPeer(i).peer.getElectionAddress().getPort()
+                        + qu.getPeer(i).peer.getQuorumAddress().getAllPorts().get(0)
+                        + ":" + qu.getPeer(i).peer.getElectionAddress().getAllPorts().get(0)
                         + ";" + "127.0.0.1:"
                         + qu.getPeer(i).peer.getClientPort());
         }

+ 20 - 9
zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthTestBase.java

@@ -64,12 +64,23 @@ public class QuorumAuthTestBase extends ZKTestCase {
         }
     }
 
+    protected String startQuorum(final int serverCount, Map<String, String> authConfigs,
+        int authServerCount) throws IOException {
+        return this.startQuorum(serverCount, authConfigs, authServerCount, false);
+    }
+
+    protected String startMultiAddressQuorum(final int serverCount, Map<String, String> authConfigs,
+        int authServerCount) throws IOException {
+        return this.startQuorum(serverCount, authConfigs, authServerCount, true);
+    }
+
     protected String startQuorum(
         final int serverCount,
         Map<String, String> authConfigs,
-        int authServerCount) throws IOException {
+        int authServerCount,
+        boolean multiAddress) throws IOException {
         StringBuilder connectStr = new StringBuilder();
-        final int[] clientPorts = startQuorum(serverCount, connectStr, authConfigs, authServerCount);
+        final int[] clientPorts = startQuorum(serverCount, connectStr, authConfigs, authServerCount, multiAddress);
         for (int i = 0; i < serverCount; i++) {
             assertTrue(
                 "waiting for server " + i + " being up",
@@ -78,17 +89,17 @@ public class QuorumAuthTestBase extends ZKTestCase {
         return connectStr.toString();
     }
 
-    protected int[] startQuorum(
-        final int serverCount,
-        StringBuilder connectStr,
-        Map<String, String> authConfigs,
-        int authServerCount) throws IOException {
+    protected int[] startQuorum(final int serverCount, StringBuilder connectStr, Map<String, String> authConfigs,
+        int authServerCount, boolean multiAddress) throws IOException {
         final int[] clientPorts = new int[serverCount];
         StringBuilder sb = new StringBuilder();
         for (int i = 0; i < serverCount; i++) {
             clientPorts[i] = PortAssignment.unique();
-            String server = String.format("server.%d=localhost:%d:%d:participant", i, PortAssignment.unique(), PortAssignment.unique());
-            sb.append(server + "\n");
+            String server = String.format("server.%d=localhost:%d:%d", i, PortAssignment.unique(), PortAssignment.unique());
+            if (multiAddress) {
+                server = server + String.format("|localhost:%d:%d", PortAssignment.unique(), PortAssignment.unique());
+            }
+            sb.append(server + ":participant\n");
             connectStr.append("127.0.0.1:" + clientPorts[i]);
             if (i < serverCount - 1) {
                 connectStr.append(",");

+ 22 - 1
zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumDigestAuthTest.java

@@ -92,6 +92,27 @@ public class QuorumDigestAuthTest extends QuorumAuthTestBase {
         zk.close();
     }
 
+    /**
+     * Test to verify that server is able to start with valid credentials
+     * when using multiple Quorum / Election addresses
+     */
+    @Test(timeout = 30000)
+    public void testValidCredentialsWithMultiAddresses() throws Exception {
+        Map<String, String> authConfigs = new HashMap<String, String>();
+        authConfigs.put(QuorumAuth.QUORUM_SASL_AUTH_ENABLED, "true");
+        authConfigs.put(QuorumAuth.QUORUM_SERVER_SASL_AUTH_REQUIRED, "true");
+        authConfigs.put(QuorumAuth.QUORUM_LEARNER_SASL_AUTH_REQUIRED, "true");
+
+        String connectStr = startMultiAddressQuorum(3, authConfigs, 3);
+        CountdownWatcher watcher = new CountdownWatcher();
+        ZooKeeper zk = new ZooKeeper(connectStr, ClientBase.CONNECTION_TIMEOUT, watcher);
+        watcher.waitForConnected(ClientBase.CONNECTION_TIMEOUT);
+        for (int i = 0; i < 10; i++) {
+            zk.create("/" + i, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        }
+        zk.close();
+    }
+
     /**
      * Test to verify that server is able to start with invalid credentials if
      * the configuration is set to quorum.auth.serverRequireSasl=false.
@@ -126,7 +147,7 @@ public class QuorumDigestAuthTest extends QuorumAuthTestBase {
         authConfigs.put(QuorumAuth.QUORUM_SERVER_SASL_AUTH_REQUIRED, "true");
         authConfigs.put(QuorumAuth.QUORUM_LEARNER_SASL_AUTH_REQUIRED, "true");
         int serverCount = 2;
-        final int[] clientPorts = startQuorum(serverCount, new StringBuilder(), authConfigs, serverCount);
+        final int[] clientPorts = startQuorum(serverCount, new StringBuilder(), authConfigs, serverCount, false);
         for (int i = 0; i < serverCount; i++) {
             boolean waitForServerUp = ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i], QuorumPeerTestBase.TIMEOUT);
             assertFalse("Shouldn't start server with invalid credentials", waitForServerUp);

+ 26 - 3
zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java

@@ -31,7 +31,7 @@ import org.apache.zookeeper.test.ClientBase;
 import org.apache.zookeeper.test.ClientBase.CountdownWatcher;
 import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class QuorumKerberosAuthTest extends KerberosSecurityTestcase {
@@ -70,8 +70,8 @@ public class QuorumKerberosAuthTest extends KerberosSecurityTestcase {
         setupJaasConfig(jaasEntries);
     }
 
-    @Before
-    public void setUp() throws Exception {
+    @BeforeClass
+    public static void setUp() throws Exception {
         // create keytab
         keytabFile = new File(KerberosTestUtils.getKeytabFile());
         String learnerPrincipal = KerberosTestUtils.getLearnerPrincipal();
@@ -119,4 +119,27 @@ public class QuorumKerberosAuthTest extends KerberosSecurityTestcase {
         zk.close();
     }
 
+    /**
+     * Test to verify that server is able to start with valid credentials
+     * when using multiple Quorum / Election addresses
+     */
+    @Test(timeout = 120000)
+    public void testValidCredentialsWithMultiAddresses() throws Exception {
+        String serverPrincipal = KerberosTestUtils.getServerPrincipal();
+        serverPrincipal = serverPrincipal.substring(0, serverPrincipal.lastIndexOf("@"));
+        Map<String, String> authConfigs = new HashMap<String, String>();
+        authConfigs.put(QuorumAuth.QUORUM_SASL_AUTH_ENABLED, "true");
+        authConfigs.put(QuorumAuth.QUORUM_SERVER_SASL_AUTH_REQUIRED, "true");
+        authConfigs.put(QuorumAuth.QUORUM_LEARNER_SASL_AUTH_REQUIRED, "true");
+        authConfigs.put(QuorumAuth.QUORUM_KERBEROS_SERVICE_PRINCIPAL, serverPrincipal);
+        String connectStr = startMultiAddressQuorum(3, authConfigs, 3);
+        CountdownWatcher watcher = new CountdownWatcher();
+        ZooKeeper zk = new ZooKeeper(connectStr, ClientBase.CONNECTION_TIMEOUT, watcher);
+        watcher.waitForConnected(ClientBase.CONNECTION_TIMEOUT);
+        for (int i = 0; i < 10; i++) {
+            zk.create("/" + i, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        }
+        zk.close();
+    }
+
 }

+ 22 - 0
zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java

@@ -142,6 +142,28 @@ public class QuorumKerberosHostBasedAuthTest extends KerberosSecurityTestcase {
         zk.close();
     }
 
+    /**
+     * Test to verify that server is able to start with valid credentials
+     * when using multiple Quorum / Election addresses
+     */
+    @Test(timeout = 120000)
+    public void testValidCredentialsWithMultiAddresses() throws Exception {
+        String serverPrincipal = hostServerPrincipal.substring(0, hostServerPrincipal.lastIndexOf("@"));
+        Map<String, String> authConfigs = new HashMap<String, String>();
+        authConfigs.put(QuorumAuth.QUORUM_SASL_AUTH_ENABLED, "true");
+        authConfigs.put(QuorumAuth.QUORUM_SERVER_SASL_AUTH_REQUIRED, "true");
+        authConfigs.put(QuorumAuth.QUORUM_LEARNER_SASL_AUTH_REQUIRED, "true");
+        authConfigs.put(QuorumAuth.QUORUM_KERBEROS_SERVICE_PRINCIPAL, serverPrincipal);
+        String connectStr = startMultiAddressQuorum(3, authConfigs, 3);
+        CountdownWatcher watcher = new CountdownWatcher();
+        ZooKeeper zk = new ZooKeeper(connectStr, ClientBase.CONNECTION_TIMEOUT, watcher);
+        watcher.waitForConnected(ClientBase.CONNECTION_TIMEOUT);
+        for (int i = 0; i < 10; i++) {
+            zk.create("/" + i, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        }
+        zk.close();
+    }
+
     /**
      * Test to verify that the bad server connection to the quorum should be rejected.
      */

+ 1 - 1
zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumUtil.java

@@ -148,7 +148,7 @@ public class QuorumUtil {
 
         LOG.info("Checking ports {}", hostPort);
         for (String hp : hostPort.split(",")) {
-            assertTrue("waiting for server up", ClientBase.waitForServerUp(hp, ClientBase.CONNECTION_TIMEOUT));
+            assertTrue("waiting for server " + hp + " up", ClientBase.waitForServerUp(hp, ClientBase.CONNECTION_TIMEOUT));
             LOG.info("{} is accepting client connections", hp);
         }
 

+ 2 - 2
zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigExceptionTest.java

@@ -198,9 +198,9 @@ public class ReconfigExceptionTest extends ZKTestCase {
         }
         int followerId = leaderId == 1 ? 2 : 1;
         joiningServers.add("server." + followerId + "=localhost:"
-                           + qu.getPeer(followerId).peer.getQuorumAddress().getPort() /*quorum port*/
+                           + qu.getPeer(followerId).peer.getQuorumAddress().getAllPorts().get(0) /*quorum port*/
                            + ":"
-                           + qu.getPeer(followerId).peer.getElectionAddress().getPort() /*election port*/
+                           + qu.getPeer(followerId).peer.getElectionAddress().getAllPorts().get(0) /*election port*/
                            + ":participant;localhost:"
                            + PortAssignment.unique()/* new client port */);
         zkAdmin.reconfigure(joiningServers, null, null, -1, new Stat());

+ 2 - 2
zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigMisconfigTest.java

@@ -121,9 +121,9 @@ public class ReconfigMisconfigTest extends ZKTestCase {
         int followerId = leaderId == 1 ? 2 : 1;
         joiningServers.add("server." + followerId
                            + "=localhost:"
-                           + qu.getPeer(followerId).peer.getQuorumAddress().getPort() /*quorum port*/
+                           + qu.getPeer(followerId).peer.getQuorumAddress().getAllPorts().get(0) /*quorum port*/
                            + ":"
-                           + qu.getPeer(followerId).peer.getElectionAddress().getPort() /*election port*/
+                           + qu.getPeer(followerId).peer.getElectionAddress().getAllPorts().get(0) /*election port*/
                            + ":participant;localhost:"
                            + PortAssignment.unique()/* new client port */);
         zkAdmin.reconfigure(joiningServers, null, null, -1, new Stat());

+ 120 - 30
zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigTest.java

@@ -18,7 +18,10 @@
 
 package org.apache.zookeeper.test;
 
+import static java.lang.Integer.parseInt;
+import static java.lang.String.format;
 import static java.net.InetAddress.getLoopbackAddress;
+import static java.util.stream.Collectors.toList;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -29,8 +32,14 @@ import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
 import org.apache.zookeeper.AsyncCallback.DataCallback;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.DummyWatcher;
@@ -85,6 +94,7 @@ public class ReconfigTest extends ZKTestCase implements DataCallback {
         long fromConfig) throws KeeperException, InterruptedException {
         byte[] config = null;
         String failure = null;
+        LOG.info("reconfig initiated by the test");
         for (int j = 0; j < 30; j++) {
             try {
                 config = zkAdmin.reconfigure(joiningServers, leavingServers, newMembers, fromConfig, new Stat());
@@ -102,14 +112,24 @@ public class ReconfigTest extends ZKTestCase implements DataCallback {
         }
 
         String configStr = new String(config);
+        List<ServerConfigLine> currentServerConfigs = Arrays.stream(configStr.split("\n"))
+          .map(String::trim)
+          .filter(s->s.startsWith("server"))
+          .map(ServerConfigLine::new)
+          .collect(toList());
+
         if (joiningServers != null) {
             for (String joiner : joiningServers) {
-                assertTrue(configStr.contains(joiner));
+                ServerConfigLine joinerServerConfigLine = new ServerConfigLine(joiner);
+
+                String errorMessage = format("expected joiner config \"%s\" not found in current config:\n%s", joiner, configStr);
+                assertTrue(errorMessage, currentServerConfigs.stream().anyMatch(c -> c.equals(joinerServerConfigLine)));
             }
         }
         if (leavingServers != null) {
             for (String leaving : leavingServers) {
-                assertFalse(configStr.contains("server.".concat(leaving)));
+                String errorMessage = format("leaving server \"%s\" not removed from config: \n%s", leaving, configStr);
+                assertFalse(errorMessage, configStr.contains(format("server.%s=", leaving)));
             }
         }
 
@@ -147,12 +167,12 @@ public class ReconfigTest extends ZKTestCase implements DataCallback {
         String configStr = new String(config);
         if (joiningServers != null) {
             for (String joiner : joiningServers) {
-                assertTrue(configStr.contains(joiner));
+                assertTrue("Config:<" + configStr + ">\n" + joiner, configStr.contains(joiner));
             }
         }
         if (leavingServers != null) {
             for (String leaving : leavingServers) {
-                assertFalse(configStr.contains("server.".concat(leaving)));
+                assertFalse("Config:<" + configStr + ">\n" + leaving, configStr.contains("server.".concat(leaving)));
             }
         }
 
@@ -302,9 +322,9 @@ public class ReconfigTest extends ZKTestCase implements DataCallback {
             // remember this server so we can add it back later
             joiningServers.add("server." + leavingIndex
                                + "=localhost:"
-                               + qu.getPeer(leavingIndex).peer.getQuorumAddress().getPort()
+                               + qu.getPeer(leavingIndex).peer.getQuorumAddress().getAllPorts().get(0)
                                + ":"
-                               + qu.getPeer(leavingIndex).peer.getElectionAddress().getPort()
+                               + qu.getPeer(leavingIndex).peer.getElectionAddress().getAllPorts().get(0)
                                + ":participant;localhost:"
                                + qu.getPeer(leavingIndex).peer.getClientPort());
 
@@ -383,9 +403,9 @@ public class ReconfigTest extends ZKTestCase implements DataCallback {
         joiningServers.add("server."
                            + leavingIndex1
                            + "=localhost:"
-                           + qu.getPeer(leavingIndex1).peer.getQuorumAddress().getPort()
+                           + qu.getPeer(leavingIndex1).peer.getQuorumAddress().getAllPorts().get(0)
                            + ":"
-                           + qu.getPeer(leavingIndex1).peer.getElectionAddress().getPort()
+                           + qu.getPeer(leavingIndex1).peer.getElectionAddress().getAllPorts().get(0)
                            + ":participant;localhost:"
                            + qu.getPeer(leavingIndex1).peer.getClientPort());
 
@@ -393,9 +413,9 @@ public class ReconfigTest extends ZKTestCase implements DataCallback {
         joiningServers.add("server."
                            + leavingIndex2
                            + "=localhost:"
-                           + qu.getPeer(leavingIndex2).peer.getQuorumAddress().getPort()
+                           + qu.getPeer(leavingIndex2).peer.getQuorumAddress().getAllPorts().get(0)
                            + ":"
-                           + qu.getPeer(leavingIndex2).peer.getElectionAddress().getPort()
+                           + qu.getPeer(leavingIndex2).peer.getElectionAddress().getAllPorts().get(0)
                            + ":observer;localhost:"
                            + qu.getPeer(leavingIndex2).peer.getClientPort());
 
@@ -561,9 +581,9 @@ public class ReconfigTest extends ZKTestCase implements DataCallback {
             joiningServers.add("server."
                                + changingIndex
                                + "=localhost:"
-                               + qu.getPeer(changingIndex).peer.getQuorumAddress().getPort()
+                               + qu.getPeer(changingIndex).peer.getQuorumAddress().getAllPorts().get(0)
                                + ":"
-                               + qu.getPeer(changingIndex).peer.getElectionAddress().getPort()
+                               + qu.getPeer(changingIndex).peer.getElectionAddress().getAllPorts().get(0)
                                + ":"
                                + newRole
                                + ";localhost:"
@@ -613,8 +633,8 @@ public class ReconfigTest extends ZKTestCase implements DataCallback {
 
         // modify follower's client port
 
-        int quorumPort = qu.getPeer(followerIndex).peer.getQuorumAddress().getPort();
-        int electionPort = qu.getPeer(followerIndex).peer.getElectionAddress().getPort();
+        int quorumPort = qu.getPeer(followerIndex).peer.getQuorumAddress().getAllPorts().get(0);
+        int electionPort = qu.getPeer(followerIndex).peer.getElectionAddress().getAllPorts().get(0);
         int oldClientPort = qu.getPeer(followerIndex).peer.getClientPort();
         int newClientPort = PortAssignment.unique();
         joiningServers.add("server."
@@ -688,7 +708,7 @@ public class ReconfigTest extends ZKTestCase implements DataCallback {
         joiningServers.add("server." + leaderIndex + "=localhost:"
                            + newQuorumPort
                            + ":"
-                           + qu.getPeer(leaderIndex).peer.getElectionAddress().getPort()
+                           + qu.getPeer(leaderIndex).peer.getElectionAddress().getAllPorts().get(0)
                            + ":participant;localhost:"
                            + qu.getPeer(leaderIndex).peer.getClientPort());
 
@@ -696,7 +716,7 @@ public class ReconfigTest extends ZKTestCase implements DataCallback {
 
         testNormalOperation(zkArr[followerIndex], zkArr[leaderIndex]);
 
-        assertTrue(qu.getPeer(leaderIndex).peer.getQuorumAddress().getPort() == newQuorumPort);
+        assertEquals((int) qu.getPeer(leaderIndex).peer.getQuorumAddress().getAllPorts().get(0), newQuorumPort);
 
         joiningServers.clear();
 
@@ -704,7 +724,7 @@ public class ReconfigTest extends ZKTestCase implements DataCallback {
 
         for (int i = 1; i <= 3; i++) {
             joiningServers.add("server." + i + "=localhost:"
-                               + qu.getPeer(i).peer.getQuorumAddress().getPort()
+                               + qu.getPeer(i).peer.getQuorumAddress().getAllPorts().get(0)
                                + ":"
                                + PortAssignment.unique()
                                + ":participant;localhost:"
@@ -753,8 +773,8 @@ public class ReconfigTest extends ZKTestCase implements DataCallback {
         int reconfigIndex = testLeader ? followerIndex : leaderIndex;
 
         // modify server's client port
-        int quorumPort = qu.getPeer(serverIndex).peer.getQuorumAddress().getPort();
-        int electionPort = qu.getPeer(serverIndex).peer.getElectionAddress().getPort();
+        int quorumPort = qu.getPeer(serverIndex).peer.getQuorumAddress().getAllPorts().get(0);
+        int electionPort = qu.getPeer(serverIndex).peer.getElectionAddress().getAllPorts().get(0);
         int oldClientPort = qu.getPeer(serverIndex).peer.getClientPort();
         int newClientPort = PortAssignment.unique();
 
@@ -846,9 +866,9 @@ public class ReconfigTest extends ZKTestCase implements DataCallback {
 
         for (int i = 1; i <= 5; i++) {
             members.add("server." + i + "=127.0.0.1:"
-                        + qu.getPeer(i).peer.getQuorumAddress().getPort()
+                        + qu.getPeer(i).peer.getQuorumAddress().getAllPorts().get(0)
                         + ":"
-                        + qu.getPeer(i).peer.getElectionAddress().getPort()
+                        + qu.getPeer(i).peer.getElectionAddress().getAllPorts().get(0)
                         + ";"
                         + "127.0.0.1:"
                         + qu.getPeer(i).peer.getClientPort());
@@ -881,9 +901,9 @@ public class ReconfigTest extends ZKTestCase implements DataCallback {
         members.clear();
         for (int i = 1; i <= 3; i++) {
             members.add("server." + i + "=127.0.0.1:"
-                        + qu.getPeer(i).peer.getQuorumAddress().getPort()
+                        + qu.getPeer(i).peer.getQuorumAddress().getAllPorts().get(0)
                         + ":"
-                        + qu.getPeer(i).peer.getElectionAddress().getPort()
+                        + qu.getPeer(i).peer.getElectionAddress().getAllPorts().get(0)
                         + ";"
                         + "127.0.0.1:"
                         + qu.getPeer(i).peer.getClientPort());
@@ -967,9 +987,9 @@ public class ReconfigTest extends ZKTestCase implements DataCallback {
 
         // remember this server so we can add it back later
         joiningServers.add("server." + leavingIndex + "=127.0.0.1:"
-                           + qu.getPeer(leavingIndex).peer.getQuorumAddress().getPort()
+                           + qu.getPeer(leavingIndex).peer.getQuorumAddress().getAllPorts().get(0)
                            + ":"
-                           + qu.getPeer(leavingIndex).peer.getElectionAddress().getPort()
+                           + qu.getPeer(leavingIndex).peer.getElectionAddress().getAllPorts().get(0)
                            + ":participant;127.0.0.1:"
                            + qu.getPeer(leavingIndex).peer.getClientPort());
 
@@ -1051,9 +1071,9 @@ public class ReconfigTest extends ZKTestCase implements DataCallback {
 
         // exactly as it is now, except for role change
         joiningServers.add("server." + changingIndex + "=127.0.0.1:"
-                           + qu.getPeer(changingIndex).peer.getQuorumAddress().getPort()
+                           + qu.getPeer(changingIndex).peer.getQuorumAddress().getAllPorts().get(0)
                            + ":"
-                           + qu.getPeer(changingIndex).peer.getElectionAddress().getPort()
+                           + qu.getPeer(changingIndex).peer.getElectionAddress().getAllPorts().get(0)
                            + ":"
                            + newRole
                            + ";127.0.0.1:"
@@ -1100,7 +1120,7 @@ public class ReconfigTest extends ZKTestCase implements DataCallback {
             JMXEnv.ensureBeanAttribute(beanName, "ClientAddress"));
         assertEquals(
             "Mismatches LearnerType!",
-            qp.getElectionAddress().getHostString() + ":" + qp.getElectionAddress().getPort(),
+            qp.getElectionAddress().getOne().getHostString() + ":" + qp.getElectionAddress().getOne().getPort(),
             JMXEnv.ensureBeanAttribute(beanName, "ElectionAddress"));
         assertEquals(
             "Mismatches PartOfEnsemble!",
@@ -1144,12 +1164,82 @@ public class ReconfigTest extends ZKTestCase implements DataCallback {
             getAddrPortFromBean(beanName, "ClientAddress"));
         assertEquals(
             "Mismatches ElectionAddress!",
-            getNumericalAddrPort(qs.electionAddr.getHostString() + ":" + qs.electionAddr.getPort()),
+            getNumericalAddrPort(qs.electionAddr.getOne().getHostString() + ":" + qs.electionAddr.getOne().getPort()),
             getAddrPortFromBean(beanName, "ElectionAddress"));
         assertEquals(
             "Mismatches QuorumAddress!",
-            getNumericalAddrPort(qs.addr.getHostString() + ":" + qs.addr.getPort()),
+            getNumericalAddrPort(qs.addr.getOne().getHostString() + ":" + qs.addr.getOne().getPort()),
             getAddrPortFromBean(beanName, "QuorumAddress"));
     }
 
+
+    /*
+     * A helper class to parse / compare server address config lines.
+     * Example: server.1=127.0.0.1:11228:11231|127.0.0.1:11230:11229:participant;0.0.0.0:11227
+     */
+    private static class ServerConfigLine {
+        private final int serverId;
+        private Integer clientPort;
+
+        // hostName -> <quorumPort1, quorumPort2>
+        private final Map<String, Set<Integer>> quorumPorts = new HashMap<>();
+
+        // hostName -> <electionPort1, electionPort2>
+        private final Map<String, Set<Integer>> electionPorts = new HashMap<>();
+
+        private ServerConfigLine(String configLine) {
+            String[] parts = configLine.trim().split("=");
+            serverId = parseInt(parts[0].split("\\.")[1]);
+            String[] serverConfig = parts[1].split(";");
+            String[] serverAddresses = serverConfig[0].split("\\|");
+            if (serverConfig.length > 1) {
+                String[] clientParts = serverConfig[1].split(":");
+                if (clientParts.length > 1) {
+                    clientPort = parseInt(clientParts[1]);
+                } else {
+                    clientPort = parseInt(clientParts[0]);
+                }
+            }
+
+            for (String addr : serverAddresses) {
+                // addr like: 127.0.0.1:11230:11229:participant or [0:0:0:0:0:0:0:1]:11346:11347
+                String serverHost;
+                String[] ports;
+                if (addr.contains("[")) {
+                    serverHost = addr.substring(1, addr.indexOf("]"));
+                    ports = addr.substring(addr.indexOf("]") + 2).split(":");
+                } else {
+                    serverHost = addr.substring(0, addr.indexOf(":"));
+                    ports = addr.substring(addr.indexOf(":") + 1).split(":");
+                }
+
+                quorumPorts.computeIfAbsent(serverHost, k -> new HashSet<>()).add(parseInt(ports[0]));
+                if (ports.length > 1) {
+                    electionPorts.computeIfAbsent(serverHost, k -> new HashSet<>()).add(parseInt(ports[1]));
+                }
+            }
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) {
+                return true;
+            }
+            if (o == null || getClass() != o.getClass()) {
+                return false;
+            }
+            ServerConfigLine that = (ServerConfigLine) o;
+            return serverId == that.serverId
+              && Objects.equals(clientPort, that.clientPort)
+              && quorumPorts.equals(that.quorumPorts)
+              && electionPorts.equals(that.electionPorts);
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(serverId, clientPort, quorumPorts, electionPorts);
+        }
+    }
+
+
 }