From 56e0b6c60c4ccee09e2de8b407705c3f51602419 Mon Sep 17 00:00:00 2001 From: ArtemChernatsky Date: Fri, 23 Nov 2018 18:31:59 +0200 Subject: [PATCH 01/16] added multiple addresses support for quorum --- .../zookeeper/server/ZooKeeperServer.java | 2 +- .../server/quorum/AuthFastLeaderElection.java | 10 +- .../zookeeper/server/quorum/Follower.java | 87 +++--- .../zookeeper/server/quorum/Leader.java | 228 +++++++++------- .../zookeeper/server/quorum/Learner.java | 77 +++--- .../server/quorum/LocalPeerBean.java | 8 +- .../server/quorum/MultipleAddresses.java | 101 +++++++ .../server/quorum/QuorumCnxManager.java | 253 ++++++++++-------- .../zookeeper/server/quorum/QuorumPeer.java | 220 +++++++-------- .../server/quorum/QuorumZooKeeperServer.java | 8 +- .../quorum/ReadOnlyZooKeeperServer.java | 8 +- .../server/quorum/RemotePeerBean.java | 10 +- .../exception/RuntimeConfigException.java | 15 ++ .../exception/RuntimeLearnerException.java | 19 ++ .../RuntimeNoReachableHostException.java | 15 ++ .../zookeeper/server/quorum/LearnerTest.java | 9 +- .../server/quorum/QuorumPeerMainTest.java | 3 +- .../server/quorum/RaceConditionTest.java | 3 +- .../quorum/ReconfigFailureCasesTest.java | 4 +- .../zookeeper/server/quorum/Zab1_0Test.java | 1 + .../apache/zookeeper/test/CnxManagerTest.java | 31 +-- .../org/apache/zookeeper/test/QuorumUtil.java | 2 +- .../zookeeper/test/ReconfigExceptionTest.java | 4 +- .../zookeeper/test/ReconfigMisconfigTest.java | 4 +- .../apache/zookeeper/test/ReconfigTest.java | 64 +++-- 25 files changed, 728 insertions(+), 458 deletions(-) create mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java create mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeConfigException.java create mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeLearnerException.java create mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeNoReachableHostException.java diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java index b417a8f095d..d551489d725 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java @@ -1114,7 +1114,7 @@ public void processPacket(ServerCnxn cnxn, ByteBuffer incomingBuffer) throws IOE // there might be a race condition that it enabled recv after // processing request and then disabled when check throttling. // - // Be aware that we're actually checking the global outstanding + // Be aware that we're actually checking the global outstanding // request before this request. // // It's fine if the IOException thrown before we decrease the count diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java index 93471526d26..a2eb9f9fd97 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java @@ -43,8 +43,6 @@ import org.apache.zookeeper.jmx.MBeanRegistry; import org.apache.zookeeper.server.ZooKeeperThread; -import org.apache.zookeeper.server.quorum.Election; -import org.apache.zookeeper.server.quorum.Vote; import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; import org.apache.zookeeper.server.quorum.QuorumPeer.ServerState; @@ -729,8 +727,7 @@ private void process(ToSend m) { } for (QuorumServer server : self.getVotingView().values()) { - InetSocketAddress saddr = new InetSocketAddress(server.addr - .getAddress(), port); + InetSocketAddress saddr = new InetSocketAddress(server.addr.getValidAddress().getAddress(), port); addrChallengeMap.put(saddr, new ConcurrentHashMap()); } @@ -760,7 +757,7 @@ public AuthFastLeaderElection(QuorumPeer self) { 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; @@ -783,11 +780,10 @@ private void leaveInstance() { private void sendNotifications() { for (QuorumServer server : self.getView().values()) { - ToSend notmsg = new ToSend(ToSend.mType.notification, AuthFastLeaderElection.sequencer++, proposedLeader, proposedZxid, logicalclock.get(), QuorumPeer.ServerState.LOOKING, - self.getView().get(server.id).electionAddr); + self.getView().get(server.id).electionAddr.getValidAddress()); sendqueue.offer(notmsg); } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java index 78ae7aa4fb4..3f37ca5ce23 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java @@ -19,8 +19,10 @@ package org.apache.zookeeper.server.quorum; import java.io.IOException; -import java.net.InetSocketAddress; +import java.io.InterruptedIOException; +import java.net.SocketException; import java.nio.ByteBuffer; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.jute.Record; import org.apache.zookeeper.ZooDefs.OpCode; @@ -42,11 +44,13 @@ public class Follower extends Learner{ private long lastQueued; // This is the same object as this.zk, but we cache the downcast op final FollowerZooKeeperServer fzk; - + private AtomicBoolean reconnect; + Follower(QuorumPeer self,FollowerZooKeeperServer zk) { this.self = self; this.zk=zk; this.fzk = zk; + reconnect = new AtomicBoolean(); } @Override @@ -59,6 +63,10 @@ public String toString() { return sb.toString(); } + public void disableReconnect() { + reconnect.set(false); + } + /** * the main method called by the follower to follow the leader * @@ -74,46 +82,53 @@ void followLeader() throws InterruptedException { self.start_fle = 0; self.end_fle = 0; fzk.registerJMX(new FollowerBean(this, zk), self.jmxLocalPeerBean); + reconnect.set(true); + try { QuorumServer leaderServer = findLeader(); - try { - connectToLeader(leaderServer.addr, leaderServer.hostname); - long newEpochZxid = registerWithLeader(Leader.FOLLOWERINFO); - if (self.isReconfigStateChange()) - throw new Exception("learned about role change"); - //check to see if the leader zxid is lower than ours - //this should never happen but is just a safety check - long newEpoch = ZxidUtils.getEpochFromZxid(newEpochZxid); - if (newEpoch < self.getAcceptedEpoch()) { - LOG.error("Proposed leader epoch " + ZxidUtils.zxidToString(newEpochZxid) - + " is less than our accepted epoch " + ZxidUtils.zxidToString(self.getAcceptedEpoch())); - throw new IOException("Error: Epoch of leader is lower"); - } - long startTime = Time.currentElapsedTime(); + while (reconnect.get()) { try { - syncWithLeader(newEpochZxid); - } finally { - long syncTime = Time.currentElapsedTime() - startTime; - ServerMetrics.FOLLOWER_SYNC_TIME.add(syncTime); - } - QuorumPacket qp = new QuorumPacket(); - while (this.isRunning()) { - readPacket(qp); - processPacket(qp); - } - } catch (Exception e) { - LOG.warn("Exception when following the leader", e); - try { - sock.close(); - } catch (IOException e1) { - e1.printStackTrace(); + connectToLeader(leaderServer.addr, leaderServer.hostname); + long newEpochZxid = registerWithLeader(Leader.FOLLOWERINFO); + if (self.isReconfigStateChange()) + throw new Exception("learned about role change"); + //check to see if the leader zxid is lower than ours + //this should never happen but is just a safety check + long newEpoch = ZxidUtils.getEpochFromZxid(newEpochZxid); + if (newEpoch < self.getAcceptedEpoch()) { + LOG.error("Proposed leader epoch " + ZxidUtils.zxidToString(newEpochZxid) + + " is less than our accepted epoch " + ZxidUtils.zxidToString(self.getAcceptedEpoch())); + throw new IOException("Error: Epoch of leader is lower"); + } + long startTime = Time.currentElapsedTime(); + try {syncWithLeader(newEpochZxid);} finally { + long syncTime = Time.currentElapsedTime() - startTime; + ServerMetrics.FOLLOWER_SYNC_TIME.add(syncTime); + } + QuorumPacket qp = new QuorumPacket(); + while (this.isRunning()) { + readPacket(qp); + processPacket(qp); + } + reconnect.set(false); + } catch (SocketException | InterruptedIOException ignored) { + zk.unregisterJMX(); + LOG.warn("Error when following the leader, reconnecting"); + } catch (Exception e) { + reconnect.set(false); + LOG.warn("Exception when following the leader", e); + try { + sock.close(); + } catch (IOException e1) { + e1.printStackTrace(); + } + + // clear pending revalidations + pendingRevalidations.clear(); } - - // clear pending revalidations - pendingRevalidations.clear(); } } finally { - zk.unregisterJMX((Learner)this); + zk.unregisterJMX((Learner) this); } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java index 0a892b1bbc1..848d211e7ab 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java @@ -20,24 +20,12 @@ import java.io.BufferedInputStream; import java.io.IOException; -import java.net.BindException; -import java.net.ServerSocket; -import java.net.Socket; -import java.net.SocketAddress; -import java.net.SocketException; +import java.net.*; import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; +import java.util.*; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ConcurrentMap; import javax.security.sasl.SaslException; @@ -230,50 +218,48 @@ public boolean isQuorumSynced(QuorumVerifier qv) { } return qv.containsQuorum(ids); } - - private final ServerSocket ss; + + private List ss = Collections.synchronizedList(new LinkedList<>()); Leader(QuorumPeer self,LeaderZooKeeperServer zk) throws IOException, X509Exception { this.self = self; this.proposalStats = new BufferStats(); + + List addresses; + if (self.getQuorumListenOnAllIPs()) + addresses = self.getQuorumAddress().getAllAddressesForAllPorts(); + else + addresses = self.getQuorumAddress().getAllAddresses(); + + for(InetSocketAddress addr : addresses) + ss.add(createServerSocket(addr, self.shouldUsePortUnification(), self.isSslQuorum())); + + this.zk = zk; + this.learnerSnapshotThrottler = createLearnerSnapshotThrottler( + maxConcurrentSnapshots, maxConcurrentSnapshotTimeout); + } + + ServerSocket createServerSocket(InetSocketAddress addr, boolean portUnification, boolean sslQuorum) throws IOException, X509Exception { + ServerSocket temp; try { - if (self.shouldUsePortUnification()) { - if (self.getQuorumListenOnAllIPs()) { - ss = new UnifiedServerSocket(self.getX509Util(), true, self.getQuorumAddress().getPort()); - } else { - ss = new UnifiedServerSocket(self.getX509Util(), true); - } - } else if (self.isSslQuorum()) { - if (self.getQuorumListenOnAllIPs()) { - ss = self.getX509Util().createSSLServerSocket(self.getQuorumAddress().getPort()); - } else { - ss = self.getX509Util().createSSLServerSocket(); - } + if (portUnification) { + temp = new UnifiedServerSocket(self.getX509Util(), true); + } else if (sslQuorum) { + temp = self.getX509Util().createSSLServerSocket(); } else { - if (self.getQuorumListenOnAllIPs()) { - ss = new ServerSocket(self.getQuorumAddress().getPort()); - } else { - ss = new ServerSocket(); - } - } - ss.setReuseAddress(true); - if (!self.getQuorumListenOnAllIPs()) { - ss.bind(self.getQuorumAddress()); + temp = new ServerSocket(); } + + temp.setReuseAddress(true); + temp.bind(addr); + return temp; } catch (X509Exception e) { LOG.error("Failed to setup ssl server socket", e); throw e; } catch (BindException e) { - if (self.getQuorumListenOnAllIPs()) { - LOG.error("Couldn't bind to port " + self.getQuorumAddress().getPort(), e); - } else { - LOG.error("Couldn't bind to " + self.getQuorumAddress(), e); - } + LOG.error("Couldn't bind to " + self.getQuorumAddress(), e); throw e; } - this.zk = zk; - this.learnerSnapshotThrottler = createLearnerSnapshotThrottler( - maxConcurrentSnapshots, maxConcurrentSnapshotTimeout); } /** @@ -387,69 +373,102 @@ public boolean isQuorumSynced(QuorumVerifier qv) { protected final Proposal newLeaderProposal = new Proposal(); class LearnerCnxAcceptor extends ZooKeeperCriticalThread { - private volatile boolean stop = false; + private AtomicBoolean stop; + private AtomicBoolean fail; public LearnerCnxAcceptor() { - super("LearnerCnxAcceptor-" + ss.getLocalSocketAddress(), zk + super("LearnerCnxAcceptor-" + ss, zk .getZooKeeperServerListener()); + stop = new AtomicBoolean(false); + fail = new AtomicBoolean(false); } @Override public void run() { + ExecutorService threadPool = Executors.newCachedThreadPool(); + + if (!stop.get()) { + ss.forEach(ssTemp -> threadPool.submit(new LearnerCnxAcceptorHandler(ssTemp))); + threadPool.shutdown(); + } + try { - while (!stop) { - Socket s = null; - boolean error = false; - 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.info("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 { + while (true) + if (threadPool.awaitTermination(10, TimeUnit.MILLISECONDS)) + break; + } catch (InterruptedException ie) { + LOG.error("Interrupted while sleeping. " + + "Ignoring exception", ie); + } + } + + public void halt() { + stop.set(true); + closeSockets(); + } + + class LearnerCnxAcceptorHandler implements Runnable { + + private ServerSocket ss; + + public LearnerCnxAcceptorHandler(ServerSocket ss) { + this.ss = ss; + } + + @Override + public void run() { + Thread.currentThread().setName("LeaderHandler-" + ss.getInetAddress()); + try { + while (!stop.get()) { + Socket s = null; + boolean error = false; + 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.get()) { + LOG.info("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; - } - } 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); + } 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); + } } } } + } catch (Exception e) { + LOG.warn("Exception while accepting follower", e.getMessage()); + if(!fail.get()) { + handleException(getName(), e); + fail.set(true); + halt(); + } } - } catch (Exception e) { - LOG.warn("Exception while accepting follower", e.getMessage()); - handleException(this.getName(), e); } - } - public void halt() { - stop = true; } } @@ -708,16 +727,13 @@ void shutdown(String reason) { 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) { @@ -734,6 +750,18 @@ void shutdown(String reason) { isShutdown = true; } + void closeSockets() { + for (ServerSocket tempSocket : ss) { + if (!tempSocket.isClosed()) + if (!tempSocket.isClosed()) + try { + tempSocket.close(); + } catch (IOException e) { + LOG.warn("Ignoring unexpected exception during close" + tempSocket, 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 diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java index faaa844ada2..539cf43c40c 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java @@ -39,6 +39,7 @@ import org.apache.jute.OutputArchive; import org.apache.jute.Record; import org.apache.zookeeper.common.X509Exception; +import org.apache.zookeeper.server.quorum.exception.RuntimeLearnerException; import org.apache.zookeeper.server.ExitCode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -243,52 +244,64 @@ protected void sockConnect(Socket sock, InetSocketAddress addr, int timeout) * @throws ConnectException * @throws InterruptedException */ - protected void connectToLeader(InetSocketAddress addr, String hostname) - throws IOException, InterruptedException, X509Exception { + protected void connectToLeader(MultipleAddresses addr, String hostname) + throws IOException, ConnectException, InterruptedException, X509Exception { this.sock = createSocket(); int initLimitTime = self.tickTime * self.initLimit; int remainingInitLimitTime = initLimitTime; long startNanoTime = nanoTime(); + boolean connected = false; + for (int tries = 0; tries < 5; tries++) { - try { - // recalculate the init limit time because retries sleep for 1000 milliseconds - remainingInitLimitTime = initLimitTime - (int)((nanoTime() - startNanoTime) / 1000000); - if (remainingInitLimitTime <= 0) { - LOG.error("initLimit exceeded on retries."); - throw new IOException("initLimit exceeded on retries."); - } + for (InetSocketAddress address : addr.getAllAddresses()) { + try { + // recalculate the init limit time because retries sleep for 1000 milliseconds + remainingInitLimitTime = initLimitTime - (int) ((nanoTime() - startNanoTime) / 1000000); + if (remainingInitLimitTime <= 0) { + LOG.error("initLimit exceeded on retries."); + throw new IOException("initLimit exceeded on retries."); + } + if (address.getAddress().isReachable(100)) { + sockConnect(sock, address, Math.min(self.tickTime * self.syncLimit, remainingInitLimitTime)); + if (self.isSslQuorum()) { + ((SSLSocket) sock).startHandshake(); + } + sock.setTcpNoDelay(nodelay); + connected = true; + break; + } + } catch (IOException e) { + remainingInitLimitTime = initLimitTime - (int) ((nanoTime() - startNanoTime) / 1000000); - sockConnect(sock, addr, Math.min(self.tickTime * self.syncLimit, remainingInitLimitTime)); - if (self.isSslQuorum()) { - ((SSLSocket) sock).startHandshake(); + if (remainingInitLimitTime <= 1000) { + LOG.error("Unexpected exception, initLimit exceeded. tries=" + tries + + ", remaining init limit=" + remainingInitLimitTime + + ", connecting to " + address, e); + throw new RuntimeLearnerException(e); + } else if (tries >= 4) { + LOG.error("Unexpected exception, retries exceeded. tries=" + tries + + ", remaining init limit=" + remainingInitLimitTime + + ", connecting to " + address, e); + throw new RuntimeLearnerException(e); + } else { + LOG.warn("Unexpected exception, tries=" + tries + + ", remaining init limit=" + remainingInitLimitTime + + ", connecting to " + address, e); + this.sock = createSocket(); + } } - sock.setTcpNoDelay(nodelay); + } + if (connected) break; - } catch (IOException e) { - remainingInitLimitTime = initLimitTime - (int)((nanoTime() - startNanoTime) / 1000000); - if (remainingInitLimitTime <= 1000) { - LOG.error("Unexpected exception, initLimit exceeded. tries=" + tries + - ", remaining init limit=" + remainingInitLimitTime + - ", connecting to " + addr,e); - throw e; - } else if (tries >= 4) { - LOG.error("Unexpected exception, retries exceeded. tries=" + tries + - ", remaining init limit=" + remainingInitLimitTime + - ", connecting to " + addr,e); - throw e; - } else { - LOG.warn("Unexpected exception, tries=" + tries + - ", remaining init limit=" + remainingInitLimitTime + - ", connecting to " + addr,e); - this.sock = createSocket(); - } - } Thread.sleep(1000); } + if (!connected) + throw new RuntimeException("Failed connect to " + addr); + self.authLearner.authenticate(sock, hostname); leaderIs = BinaryInputArchive.getArchive(new BufferedInputStream( diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LocalPeerBean.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LocalPeerBean.java index 5bbd80e6d81..7d01c5364cf 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LocalPeerBean.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LocalPeerBean.java @@ -18,6 +18,8 @@ package org.apache.zookeeper.server.quorum; +import org.apache.zookeeper.common.NetUtils; +import java.util.stream.Collectors; import static org.apache.zookeeper.common.NetUtils.formatInetAddr; /** @@ -71,7 +73,8 @@ public String getState() { } public String getQuorumAddress() { - return formatInetAddr(peer.getQuorumAddress()); + return peer.getQuorumAddress().getAllAddresses().stream().map(NetUtils::formatInetAddr) + .collect(Collectors.joining(",")); } public int getElectionType() { @@ -79,7 +82,8 @@ public int getElectionType() { } public String getElectionAddress() { - return formatInetAddr(peer.getElectionAddress()); + return peer.getElectionAddress().getAllAddresses().stream().map(NetUtils::formatInetAddr) + .collect(Collectors.joining(",")); } public String getClientAddress() { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java new file mode 100644 index 00000000000..a36dc2de274 --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java @@ -0,0 +1,101 @@ +package org.apache.zookeeper.server.quorum; + +import org.apache.zookeeper.server.quorum.exception.RuntimeNoReachableHostException; + +import java.io.IOException; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.UnknownHostException; +import java.util.*; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +public class MultipleAddresses { + + private Set addresses; + + public MultipleAddresses() { + addresses = Collections.newSetFromMap(new ConcurrentHashMap<>()); + } + + public MultipleAddresses(List addresses) { + this.addresses = Collections.newSetFromMap(new ConcurrentHashMap<>()); + this.addresses.addAll(addresses); + } + + public MultipleAddresses(InetSocketAddress address) { + addresses = Collections.newSetFromMap(new ConcurrentHashMap<>()); + addresses.add(address); + } + + public boolean isEmpty() { + return addresses.isEmpty(); + } + + public List getAllAddresses() { + return new LinkedList<>(addresses); + } + + public List getAllAddressesForAllPorts() { + return addresses.stream().map(a -> new InetSocketAddress(a.getPort())).distinct().collect(Collectors.toList()); + } + + public List getAllPorts() { + return addresses.stream().map(InetSocketAddress::getPort).collect(Collectors.toList()); + } + + public void addAddress(InetSocketAddress address) { + addresses.add(address); + } + + public InetSocketAddress getValidAddress() { + + for(InetSocketAddress addr : addresses) { + try { + if (addr.getAddress().isReachable(100)) + return addr; + } catch (NullPointerException | IOException e) { + } + } + + throw new RuntimeNoReachableHostException("No valid address among " + addresses); + } + + public void recreateSocketAddresses() { + Set temp = new HashSet<>(); + + for(InetSocketAddress addr : addresses) { + try { + temp.add(new InetSocketAddress(InetAddress.getByName(addr.getHostString()), addr.getPort())); + } catch (UnknownHostException e) { + temp.add(addr); + } + } + + addresses.clear(); + addresses.addAll(temp); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + 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() { + StringBuilder result = new StringBuilder(); + + addresses.forEach(addr -> result.append(String.format("%s.", addr))); + result.deleteCharAt(result.length() - 1); + + return result.toString(); + } +} diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java index 4175f3cbed1..630c2169a0d 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java @@ -18,11 +18,7 @@ package org.apache.zookeeper.server.quorum; -import java.io.BufferedInputStream; -import java.io.BufferedOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; +import java.io.*; import java.net.BindException; import java.net.InetSocketAddress; import java.net.ServerSocket; @@ -32,20 +28,12 @@ import java.nio.BufferUnderflowException; import java.nio.ByteBuffer; import java.nio.channels.UnresolvedAddressException; -import java.util.Collections; -import java.util.Enumeration; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.NoSuchElementException; +import java.util.*; +import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.zookeeper.common.X509Exception; import org.apache.zookeeper.server.ExitCode; @@ -54,6 +42,7 @@ import org.apache.zookeeper.server.ZooKeeperThread; import org.apache.zookeeper.server.quorum.auth.QuorumAuthLearner; import org.apache.zookeeper.server.quorum.auth.QuorumAuthServer; +import org.apache.zookeeper.server.quorum.exception.RuntimeNoReachableHostException; import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -332,7 +321,7 @@ public void testInitiateConnection(long sid) throws Exception { LOG.debug("Opening channel to server " + sid); Socket sock = new Socket(); setSockOpts(sock); - sock.connect(self.getVotingView().get(sid).electionAddr, cnxTO); + sock.connect(self.getVotingView().get(sid).electionAddr.getValidAddress(), cnxTO); initiateConnection(sock, sid); } @@ -415,7 +404,8 @@ private boolean startConnection(Socket sock, Long sid) // represents protocol version (in other words - message type) dout.writeLong(PROTOCOL_VERSION); dout.writeLong(self.getId()); - String addr = formatInetAddr(self.getElectionAddress()); + InetSocketAddress address = self.getElectionAddress().getValidAddress(); + String addr = address.getHostString() + ":" + address.getPort(); byte[] addr_bytes = addr.getBytes(); dout.writeInt(addr_bytes.length); dout.write(addr_bytes); @@ -575,7 +565,7 @@ private void handleConnection(Socket sock, DataInputStream din) closeSocket(sock); if (electionAddr != null) { - connectOne(sid, electionAddr); + connectOne(sid, new MultipleAddresses(electionAddr)); } else { connectOne(sid); } @@ -638,7 +628,7 @@ public void toSend(Long sid, ByteBuffer b) { * @param sid server id * @return boolean success indication */ - synchronized private boolean connectOne(long sid, InetSocketAddress electionAddr){ + synchronized private boolean connectOne(long sid, MultipleAddresses electionAddr){ if (senderWorkerMap.get(sid) != null) { LOG.debug("There is a connection already for server " + sid); return true; @@ -647,16 +637,17 @@ synchronized private boolean connectOne(long sid, InetSocketAddress electionAddr Socket sock = null; try { LOG.debug("Opening channel to server " + sid); + InetSocketAddress address = electionAddr.getValidAddress(); if (self.isSslQuorum()) { SSLSocket sslSock = self.getX509Util().createSSLSocket(); setSockOpts(sslSock); - sslSock.connect(electionAddr, cnxTO); + sslSock.connect(address, cnxTO); sslSock.startHandshake(); sock = sslSock; } else { sock = new Socket(); setSockOpts(sock); - sock.connect(electionAddr, cnxTO); + sock.connect(address, cnxTO); } LOG.debug("Connected to server " + sid); @@ -670,7 +661,7 @@ synchronized private boolean connectOne(long sid, InetSocketAddress electionAddr initiateConnection(sock, sid); } return true; - } catch (UnresolvedAddressException e) { + } catch (RuntimeNoReachableHostException e) { // Sun doesn't include the address that causes this // exception to be thrown, also UAE cannot be wrapped cleanly // so we log the exception in order to capture this critical @@ -850,7 +841,7 @@ private void resetConnectionThreadCount() { */ public class Listener extends ZooKeeperThread { - volatile ServerSocket ss = null; + List listenerHandlers; public Listener() { // During startup of thread, thread name will be overridden to @@ -858,45 +849,105 @@ public Listener() { super("ListenerThread"); } - /** - * Sleeps on accept(). - */ + @Override public void run() { - int numRetries = 0; - InetSocketAddress addr; - Socket client = null; - Exception exitException = null; - while((!shutdown) && (numRetries < 3)){ + while(!shutdown) { + Stream addresses; + + if (self.getQuorumListenOnAllIPs()) + addresses = self.getElectionAddress().getAllAddressesForAllPorts().stream(); + else + addresses = self.getElectionAddress().getAllAddresses().stream(); + + listenerHandlers = addresses + .map(addr -> new ListenerHandler(addr, self.shouldUsePortUnification(), self.isSslQuorum())) + .collect(Collectors.toList()); + + ExecutorService threadPool = Executors.newCachedThreadPool(); + listenerHandlers.forEach(threadPool::submit); + threadPool.shutdown(); + try { - if (self.shouldUsePortUnification()) { - ss = new UnifiedServerSocket(self.getX509Util(), true); - } else if (self.isSslQuorum()) { - ss = self.getX509Util().createSSLServerSocket(); - } else { - ss = new ServerSocket(); - } + while (true) + if (threadPool.awaitTermination(10, TimeUnit.MILLISECONDS)) + break; + } catch (InterruptedException ie) { + LOG.error("Interrupted while sleeping. " + + "Ignoring exception", ie); + } - ss.setReuseAddress(true); + } - 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("Leaving listener"); + if (!shutdown) { + LOG.error("As I'm leaving the listener thread, " + + "I won't be able to participate in leader " + + "election any longer: " + + self.getElectionAddress().getAllAddresses()); + } else if (listenerHandlers != null) { + // Clean up for shutdown. + for (ListenerHandler handler : listenerHandlers) + try { + handler.close(); + } catch (IOException ie) { + // Don't log an error for shutdown. + LOG.debug("Error closing server socket", ie); } - LOG.info("My election bind port: " + addr.toString()); - setName(addr.toString()); - ss.bind(addr); - while (!shutdown) { - try { + } + } + + /** + * Halts this listener thread. + */ + void halt(){ + 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); + } + } + } + + class ListenerHandler implements Runnable, Closeable { + private final Logger LOG = LoggerFactory.getLogger(ListenerHandler.class); + private final static int ATTEMPTS_AMOUNT = 3; + + private ServerSocket ss; + private InetSocketAddress address; + private boolean portUnification; + private boolean sslQuorum; + + ListenerHandler(InetSocketAddress address, boolean portUnification, boolean sslQuorum) { + this.address = address; + this.portUnification = portUnification; + this.sslQuorum = sslQuorum; + } + + /** + * Sleeps on accept(). + */ + @Override + public void run() { + Thread.currentThread().setName("ListenerHandler-" + address); + + int numRetries = 0; + Socket client = null; + + while((!shutdown) && (numRetries < ATTEMPTS_AMOUNT)){ + try { + ss = createNewServerSocket(); + LOG.info("My election bind port: " + address.toString()); + while (!shutdown) { client = ss.accept(); setSockOpts(client); LOG.info("Received connection request " - + formatInetAddr((InetSocketAddress)client.getRemoteSocketAddress())); + + client.getRemoteSocketAddress()); // Receive and handle the connection request // asynchronously if the quorum sasl authentication is // enabled. This is required because sasl server @@ -908,68 +959,56 @@ public void run() { 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 (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|X509Exception e) { + if (shutdown) { + break; + } + LOG.error("Exception while listening", 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); } - } catch (IOException|X509Exception 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, " - + "I won't be able to participate in leader " - + "election any longer: " - + formatInetAddr(self.getElectionAddress())); - if (exitException instanceof BindException) { - // 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 - System.exit(ExitCode.UNABLE_TO_BIND_QUORUM_PORT.getValue()); - } - } else if (ss != null) { - // Clean up for shutdown. - try { + + @Override + public void close() throws IOException { + if(ss != null) { + LOG.debug("Trying to close listeners: " + ss); ss.close(); - } catch (IOException ie) { - // Don't log an error for shutdown. - LOG.debug("Error closing server socket", ie); } } - } - /** - * 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(); + private ServerSocket createNewServerSocket() throws IOException, X509Exception { + ServerSocket temp; + + if (portUnification) { + temp = new UnifiedServerSocket(self.getX509Util(), true); + } else if (sslQuorum) { + temp = self.getX509Util().createSSLServerSocket(); + } else { + temp = new ServerSocket(); } - } catch (IOException e){ - LOG.warn("Exception when shutting down listener: " + e); + + temp.setReuseAddress(true); + temp.bind(address); + + return temp; } + } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java index 7abde4ba991..4350308acbf 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -29,21 +29,15 @@ import java.net.DatagramSocket; import java.net.InetAddress; import java.net.InetSocketAddress; -import java.net.UnknownHostException; +import java.net.SocketException; import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; +import java.util.*; import java.util.Map.Entry; -import java.util.Properties; -import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import javax.security.sasl.SaslException; - import org.apache.zookeeper.KeeperException.BadArgumentsException; import org.apache.zookeeper.common.AtomicFileWritingIdiom; import org.apache.zookeeper.common.AtomicFileWritingIdiom.WriterStatement; @@ -128,9 +122,9 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider private ZKDatabase zkDb; 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; @@ -141,7 +135,7 @@ public static class QuorumServer { public LearnerType type = LearnerType.PARTICIPANT; public boolean isClientAddrFromStatic = false; - + private List myAddrs; public QuorumServer(long id, InetSocketAddress addr, @@ -158,7 +152,7 @@ public QuorumServer(long id, InetSocketAddress addr, public QuorumServer(long id, InetSocketAddress addr) { this(id, addr, (InetSocketAddress)null, (InetSocketAddress)null, LearnerType.PARTICIPANT); } - + public long getId() { return id; } @@ -170,37 +164,33 @@ public long getId() { * 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 { + LearnerType tempType; + + switch (s.toLowerCase()) { + case "observer": + tempType = LearnerType.OBSERVER; + break; + case "participant": + tempType = LearnerType.PARTICIPANT; + break; + default: + throw new ConfigException("Unrecognised peertype: " + s); } + + return tempType; } private static final String wrongFormat = " does not have the form server_config or server_config;client_config"+ @@ -209,56 +199,70 @@ private void setType(String s) throws ConfigException { 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); - } - - if (serverClientParts.length == 2) { - //LOG.warn("ClientParts: " + serverClientParts[1]); - String clientParts[] = ConfigUtils.getHostAndPort(serverClientParts[1]); - if (clientParts.length > 2) { + String serverAddresses[] = serverClientParts[0].split(","); + for(String serverAddress : serverAddresses) { + String serverParts[] = ConfigUtils.getHostAndPort(serverAddress); + if ((serverClientParts.length > 2) || (serverParts.length < 3) + || (serverParts.length > 4)) { throw new ConfigException(addressStr + wrongFormat); } - // is client_config a host:port or just a port - hostname = (clientParts.length == 2) ? clientParts[0] : "0.0.0.0"; + if (serverClientParts.length == 2) { + //LOG.warn("ClientParts: " + serverClientParts[1]); + String clientParts[] = ConfigUtils.getHostAndPort(serverClientParts[1]); + if (clientParts.length > 2) { + throw new ConfigException(addressStr + wrongFormat); + } + + // is client_config a host:port or just a port + 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 + InetSocketAddress tempAddress; + InetSocketAddress tempElectionAddress; try { - clientAddr = new InetSocketAddress(hostname, - Integer.parseInt(clientParts[clientParts.length - 1])); - //LOG.warn("Set clientAddr to " + clientAddr); + tempAddress = new InetSocketAddress(serverParts[0], + Integer.parseInt(serverParts[1])); + addr.addAddress(tempAddress); } catch (NumberFormatException e) { - throw new ConfigException("Address unresolved: " + hostname + ":" + clientParts[clientParts.length - 1]); + 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]); } - } - - // 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]); - } - if(addr.getPort() == electionAddr.getPort()) { + 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) { - setType(serverParts[3]); - } + if (serverParts.length == 4) { + LearnerType tempType = getType(serverParts[3]); + if (newType == null) + newType = tempType; - this.hostname = serverParts[0]; - + if (newType != tempType) + throw new ConfigException("Multiple addresses should have similar roles: " + type + " vs " + tempType); + } + + this.hostname = serverParts[0]; + } + if(newType != null) + type = newType; setMyAddrs(); } @@ -270,8 +274,10 @@ public QuorumServer(long id, InetSocketAddress addr, 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; @@ -279,10 +285,10 @@ public QuorumServer(long id, InetSocketAddress addr, } private void setMyAddrs() { - this.myAddrs = new ArrayList(); - 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); } @@ -298,25 +304,29 @@ private static String delimitedHostString(InetSocketAddress addr) 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())); + + List addrList = addr.getAllAddresses(); + List electionAddrList = 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 (electionAddr!=null){ - sw.append(":"); - sw.append(String.valueOf(electionAddr.getPort())); - } + if (type == LearnerType.OBSERVER) sw.append(":observer"); - else if (type == LearnerType.PARTICIPANT) sw.append(":participant"); + 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(); + + return sw.toString(); } public int hashCode() { @@ -334,18 +344,16 @@ private boolean checkAddressesEqual(InetSocketAddress addr1, InetSocketAddress a public boolean equals(Object o){ if (!(o instanceof QuorumServer)) return false; QuorumServer qs = (QuorumServer)o; - if ((qs.id != id) || (qs.type != type)) return false; - if (!checkAddressesEqual(addr, qs.addr)) return false; - if (!checkAddressesEqual(electionAddr, qs.electionAddr)) return false; - if (!checkAddressesEqual(clientAddr, qs.clientAddr)) return false; - return true; + if ((qs.id != id) || (qs.type != type)) return false; + if (!addr.equals(qs.addr)) return false; + if (!electionAddr.equals(qs.electionAddr)) return false; + return checkAddressesEqual(clientAddr, qs.clientAddr); } public void checkAddressDuplicate(QuorumServer s) throws BadArgumentsException { - List otherAddrs = new ArrayList(); - otherAddrs.add(s.addr); + List 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) { @@ -716,8 +724,8 @@ public synchronized ServerState getPeerState(){ DatagramSocket udpSocket; - private InetSocketAddress myQuorumAddr; - private InetSocketAddress myElectionAddr = null; + private MultipleAddresses myQuorumAddr = new MultipleAddresses(); + private MultipleAddresses myElectionAddr = new MultipleAddresses(); private InetSocketAddress myClientAddr = null; /** @@ -748,25 +756,25 @@ public void recreateSocketAddresses(long id) { } } - public InetSocketAddress getQuorumAddress(){ + public MultipleAddresses getQuorumAddress(){ synchronized (QV_LOCK) { return myQuorumAddr; } } - public void setQuorumAddress(InetSocketAddress addr){ + public void setQuorumAddress(MultipleAddresses addr){ synchronized (QV_LOCK) { myQuorumAddr = addr; } } - public InetSocketAddress getElectionAddress(){ + public MultipleAddresses getElectionAddress(){ synchronized (QV_LOCK) { return myElectionAddr; } } - public void setElectionAddress(InetSocketAddress addr){ + public void setElectionAddress(MultipleAddresses addr){ synchronized (QV_LOCK) { myElectionAddr = addr; } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java index f6e4f1135b5..22188545cb2 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java @@ -20,6 +20,8 @@ import java.io.IOException; import java.io.PrintWriter; import java.nio.ByteBuffer; +import java.util.Objects; +import java.util.stream.Collectors; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; @@ -173,9 +175,11 @@ public void dumpConf(PrintWriter pwriter) { 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: "); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java index 2ef8a5e3ed0..d74186dbbdb 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java +++ b/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; @@ -166,9 +168,11 @@ public void dumpConf(PrintWriter pwriter) { 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()); } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/RemotePeerBean.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/RemotePeerBean.java index 285f11a8593..c6e0185a093 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/RemotePeerBean.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/RemotePeerBean.java @@ -20,6 +20,8 @@ import org.apache.zookeeper.jmx.ZKMBeanInfo; +import java.util.stream.Collectors; + /** * A remote peer bean only provides limited information about the remote peer, * and the peer cannot be managed remotely. @@ -45,11 +47,15 @@ public boolean isHidden() { } 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() { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeConfigException.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeConfigException.java new file mode 100644 index 00000000000..7dc48aac805 --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeConfigException.java @@ -0,0 +1,15 @@ +package org.apache.zookeeper.server.quorum.exception; + +public class RuntimeConfigException extends RuntimeException { + + private static final long serialVersionUID = -9025894204684855418L; + + public RuntimeConfigException() { + super(); + } + + public RuntimeConfigException(String message) { + super(message); + } + +} diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeLearnerException.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeLearnerException.java new file mode 100644 index 00000000000..0ca9c20a824 --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeLearnerException.java @@ -0,0 +1,19 @@ +package org.apache.zookeeper.server.quorum.exception; + +public class RuntimeLearnerException extends RuntimeException { + + private static final long serialVersionUID = 9164642202468819481L; + + public RuntimeLearnerException() { + super(); + } + + public RuntimeLearnerException(Throwable cause) { + super(cause); + } + + public RuntimeLearnerException(String message) { + super(message); + } + +} diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeNoReachableHostException.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeNoReachableHostException.java new file mode 100644 index 00000000000..811038e3245 --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeNoReachableHostException.java @@ -0,0 +1,15 @@ +package org.apache.zookeeper.server.quorum.exception; + +public class RuntimeNoReachableHostException extends RuntimeException { + + private static final long serialVersionUID = -8118892361652577058L; + + public RuntimeNoReachableHostException() { + super(); + } + + public RuntimeNoReachableHostException(String message) { + super(message); + } + +} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerTest.java index 17935505ca7..036aaaf68df 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerTest.java @@ -35,6 +35,7 @@ import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.server.ZKDatabase; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; +import org.apache.zookeeper.server.quorum.exception.RuntimeLearnerException; import org.apache.zookeeper.test.TestUtils; import org.apache.zookeeper.txn.CreateTxn; import org.apache.zookeeper.txn.TxnHeader; @@ -98,7 +99,7 @@ protected void sockConnect(Socket sock, InetSocketAddress addr, int timeout) } } - @Test(expected=IOException.class) + @Test(expected= RuntimeLearnerException.class) public void connectionRetryTimeoutTest() throws Exception { Learner learner = new TimeoutLearner(); learner.self = new QuorumPeer(); @@ -110,7 +111,7 @@ public void connectionRetryTimeoutTest() throws Exception { 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 { @@ -130,9 +131,9 @@ public void connectionInitLimitTimeoutTest() throws Exception { // 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), ""); Assert.fail("should have thrown IOException!"); - } catch (IOException e) { + } catch (RuntimeLearnerException e) { //good, wanted to see that, let's make sure we ran out of time Assert.assertTrue(learner.nanoTime() > 2000*5*1000000); Assert.assertEquals(3, learner.getSockConnectAttempt()); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java index 947e7a505b7..043b9bb94dc 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java @@ -384,8 +384,9 @@ public void testElectionFraud() throws IOException, InterruptedException { Assert.assertTrue("All servers should join the quorum", servers.mt[falseLeader].main.quorumPeer.follower != 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 + // and disable reconnect, close the connection to the leader servers.mt[falseLeader].main.quorumPeer.electionAlg.shutdown(); + servers.mt[falseLeader].main.quorumPeer.follower.disableReconnect(); servers.mt[falseLeader].main.quorumPeer.follower.getSocket().close(); // wait for the falseLeader to disconnect diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/RaceConditionTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/RaceConditionTest.java index ea2a4d38518..17d1cda624e 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/RaceConditionTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/RaceConditionTest.java @@ -36,6 +36,7 @@ import org.apache.zookeeper.server.ZooKeeperServer; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; import org.apache.zookeeper.server.quorum.QuorumPeer.ServerState; +import org.apache.zookeeper.server.quorum.exception.RuntimeLearnerException; import org.apache.zookeeper.test.ClientBase; import org.junit.After; import org.junit.Assert; @@ -174,7 +175,7 @@ protected Follower makeFollower(FileTxnSnapLog logFactory) throws IOException { protected void processPacket(QuorumPacket qp) throws Exception { if (stopPing && qp.getType() == Leader.PING) { LOG.info("Follower skipped ping"); - throw new SocketException("Socket time out while sending the ping response"); + throw new RuntimeLearnerException("Socket time out while sending the ping response"); } else { super.processPacket(qp); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigFailureCasesTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigFailureCasesTest.java index a0b9760ebb6..de0322abc38 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigFailureCasesTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigFailureCasesTest.java @@ -79,8 +79,8 @@ public void testIncrementalReconfigInvokedOnHiearchicalQS() throws Exception { 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()); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/Zab1_0Test.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/Zab1_0Test.java index aabd9d88222..96c52bd6f89 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/Zab1_0Test.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/Zab1_0Test.java @@ -451,6 +451,7 @@ public void run() { conversation.converseWithFollower(ia, oa, follower); } finally { if (follower != null) { + follower.disableReconnect(); follower.shutdown(); } if (followerThread != null) { diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/CnxManagerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/CnxManagerTest.java index 3cee7ba4be2..1d3aa52f845 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/CnxManagerTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/CnxManagerTest.java @@ -31,10 +31,12 @@ import java.util.HashMap; import java.util.Map; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.net.Socket; import org.apache.zookeeper.common.Time; +import org.apache.zookeeper.server.quorum.exception.RuntimeNoReachableHostException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.zookeeper.PortAssignment; @@ -184,12 +186,12 @@ public void testCnxManager() throws Exception { Assert.assertFalse(cnxManager.listener.isAlive()); } - @Test + @Test(expected = RuntimeNoReachableHostException.class) 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); @@ -236,19 +238,18 @@ public void testCnxManagerSpinLock() throws Exception { } 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.getValidAddress(); + 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(new Long(2)).electionAddr; + InetSocketAddress otherAddr = peers.get(2L).electionAddr.getValidAddress(); DataOutputStream dout = new DataOutputStream(sc.socket().getOutputStream()); dout.writeLong(QuorumCnxManager.PROTOCOL_VERSION); - dout.writeLong(new Long(2)); + dout.writeLong(2L); String addr = otherAddr.getHostString()+ ":" + otherAddr.getPort(); byte[] addr_bytes = addr.getBytes(); dout.writeInt(addr_bytes.length); @@ -301,13 +302,13 @@ public void testCnxManagerNPE() throws Exception { } 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.getValidAddress(); + 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 @@ -348,12 +349,12 @@ public void testSocketTimeout() throws Exception { } 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.getValidAddress(); + 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); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumUtil.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumUtil.java index 314171d873a..5a8cea81bc6 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumUtil.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumUtil.java @@ -148,7 +148,7 @@ public void startAll() throws IOException { LOG.info("Checking ports " + hostPort); for (String hp : hostPort.split(",")) { - Assert.assertTrue("waiting for server up", ClientBase.waitForServerUp(hp, + Assert.assertTrue("waiting for server " + hp + " up", ClientBase.waitForServerUp(hp, ClientBase.CONNECTION_TIMEOUT)); LOG.info(hp + " is accepting client connections"); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigExceptionTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigExceptionTest.java index 5eda4b0f429..a564dcdfc91 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigExceptionTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigExceptionTest.java @@ -204,8 +204,8 @@ private boolean reconfigPort() throws KeeperException, InterruptedException { leaderId++; int followerId = leaderId == 1 ? 2 : 1; joiningServers.add("server." + followerId + "=localhost:" - + qu.getPeer(followerId).peer.getQuorumAddress().getPort() /*quorum port*/ - + ":" + qu.getPeer(followerId).peer.getElectionAddress().getPort() /*election port*/ + + qu.getPeer(followerId).peer.getQuorumAddress().getAllPorts().get(0) /*quorum 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()); return true; diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigMisconfigTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigMisconfigTest.java index 1694fcf2a8a..e0e963a8eaf 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigMisconfigTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigMisconfigTest.java @@ -119,8 +119,8 @@ private boolean reconfigPort() throws KeeperException, InterruptedException { leaderId++; int followerId = leaderId == 1 ? 2 : 1; joiningServers.add("server." + followerId + "=localhost:" - + qu.getPeer(followerId).peer.getQuorumAddress().getPort() /*quorum port*/ - + ":" + qu.getPeer(followerId).peer.getElectionAddress().getPort() /*election port*/ + + qu.getPeer(followerId).peer.getQuorumAddress().getAllPorts().get(0) /*quorum 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()); return true; diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigTest.java index 7b39ab155b7..af56b586517 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigTest.java @@ -141,12 +141,13 @@ public static String testServerHasConfig(ZooKeeper zk, String configStr = new String(config); if (joiningServers != null) { for (String joiner : joiningServers) { - Assert.assertTrue(configStr.contains(joiner)); + Assert.assertTrue("Config:<" + configStr + ">\n" + joiner, configStr.contains(joiner)); } } if (leavingServers != null) { for (String leaving : leavingServers) - Assert.assertFalse(configStr.contains("server.".concat(leaving))); + Assert.assertFalse("Config:<" + configStr + ">\n" + leaving, + configStr.contains("server.".concat(leaving))); } return configStr; @@ -291,11 +292,10 @@ public void testRemoveAddOne() throws Exception { 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() + ":participant;localhost:" + + qu.getPeer(leavingIndex).peer.getElectionAddress().getAllPorts().get(0) + + ":participant;localhost:" + qu.getPeer(leavingIndex).peer.getClientPort()); String configStr = reconfig(zkAdmin1, null, leavingServers, null, -1); @@ -368,17 +368,17 @@ public void testRemoveAddTwo() throws Exception { // remember these servers so we can add them back later 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()); // this server will be added back as an observer 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()); @@ -546,11 +546,10 @@ public void testRoleChange() throws Exception { 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() + ":" + newRole + ";localhost:" + + qu.getPeer(changingIndex).peer.getElectionAddress().getAllPorts().get(0) + + ":" + newRole + ";localhost:" + qu.getPeer(changingIndex).peer.getClientPort()); reconfig(zkAdmin1, joiningServers, null, null, -1); @@ -597,8 +596,8 @@ public void testPortChange() throws Exception { // 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." + followerIndex + "=localhost:" + quorumPort @@ -666,7 +665,7 @@ ClientBase.CONNECTION_TIMEOUT, new Watcher() { 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()); @@ -674,8 +673,7 @@ ClientBase.CONNECTION_TIMEOUT, new Watcher() { testNormalOperation(zkArr[followerIndex], zkArr[leaderIndex]); - Assert.assertTrue(qu.getPeer(leaderIndex).peer.getQuorumAddress() - .getPort() == newQuorumPort); + Assert.assertEquals((int) qu.getPeer(leaderIndex).peer.getQuorumAddress().getAllPorts().get(0), newQuorumPort); joiningServers.clear(); @@ -683,7 +681,7 @@ ClientBase.CONNECTION_TIMEOUT, new Watcher() { 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:" + qu.getPeer(i).peer.getClientPort()); } @@ -729,8 +727,8 @@ private void testPortChangeToBlockedPort(boolean testLeader) throws Exception { 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(); @@ -827,8 +825,8 @@ public void testQuorumSystemChange() throws Exception { 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()); } @@ -859,8 +857,8 @@ public void testQuorumSystemChange() throws Exception { 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.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()); } @@ -939,9 +937,9 @@ public void testJMXBeanAfterRemoveAddOne() throws Exception { // 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()); @@ -1017,9 +1015,9 @@ public void testJMXBeanAfterRoleChange() throws Exception { // 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:" + qu.getPeer(changingIndex).peer.getClientPort()); @@ -1056,7 +1054,7 @@ private void assertLocalPeerMXBeanAttributes(QuorumPeer qp, qp.getClientAddress().getHostString() + ":" + qp.getClientAddress().getPort(), JMXEnv.ensureBeanAttribute(beanName, "ClientAddress")); Assert.assertEquals("Mismatches LearnerType!", - qp.getElectionAddress().getHostString() + ":" + qp.getElectionAddress().getPort(), + qp.getElectionAddress().getAllAddresses().get(0).getHostString() + ":" + qp.getElectionAddress().getAllAddresses().get(0).getPort(), JMXEnv.ensureBeanAttribute(beanName, "ElectionAddress")); Assert.assertEquals("Mismatches PartOfEnsemble!", isPartOfEnsemble, JMXEnv.ensureBeanAttribute(beanName, "PartOfEnsemble")); @@ -1094,10 +1092,10 @@ private void assertRemotePeerMXBeanAttributes(QuorumServer qs, getNumericalAddrPort(qs.clientAddr.getHostString() + ":" + qs.clientAddr.getPort()), getAddrPortFromBean(beanName, "ClientAddress") ); Assert.assertEquals("Mismatches ElectionAddress!", - getNumericalAddrPort(qs.electionAddr.getHostString() + ":" + qs.electionAddr.getPort()), + getNumericalAddrPort(qs.electionAddr.getAllAddresses().get(0).getHostString() + ":" + qs.electionAddr.getAllAddresses().get(0).getPort()), getAddrPortFromBean(beanName, "ElectionAddress") ); Assert.assertEquals("Mismatches QuorumAddress!", - getNumericalAddrPort(qs.addr.getHostString() + ":" + qs.addr.getPort()), + getNumericalAddrPort(qs.addr.getAllAddresses().get(0).getHostString() + ":" + qs.addr.getAllAddresses().get(0).getPort()), getAddrPortFromBean(beanName, "QuorumAddress") ); } } From 1b0b78a9b65d394450fa8f48c8eb39dcca3feabe Mon Sep 17 00:00:00 2001 From: ArtemChernatsky Date: Thu, 6 Dec 2018 14:07:03 +0200 Subject: [PATCH 02/16] fixed converting and improved parsing --- .../server/quorum/MultipleAddresses.java | 7 +-- .../zookeeper/server/quorum/QuorumPeer.java | 44 +++++++++---------- 2 files changed, 22 insertions(+), 29 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java index a36dc2de274..55a80a49a0a 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java @@ -91,11 +91,6 @@ public int hashCode() { @Override public String toString() { - StringBuilder result = new StringBuilder(); - - addresses.forEach(addr -> result.append(String.format("%s.", addr))); - result.deleteCharAt(result.length() - 1); - - return result.toString(); + return addresses.stream().map(InetSocketAddress::toString).collect(Collectors.joining(",")); } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java index 4350308acbf..dd8581d3f35 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -29,7 +29,6 @@ import java.net.DatagramSocket; import java.net.InetAddress; import java.net.InetSocketAddress; -import java.net.SocketException; import java.nio.ByteBuffer; import java.util.*; import java.util.Map.Entry; @@ -202,6 +201,25 @@ public QuorumServer(long sid, String addressStr) throws ConfigException { LearnerType newType = null; String serverClientParts[] = addressStr.split(";"); 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); + } + + // is client_config a host:port or just a port + 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]); + } + } + for(String serverAddress : serverAddresses) { String serverParts[] = ConfigUtils.getHostAndPort(serverAddress); if ((serverClientParts.length > 2) || (serverParts.length < 3) @@ -209,37 +227,17 @@ public QuorumServer(long sid, String addressStr) throws ConfigException { throw new ConfigException(addressStr + wrongFormat); } - if (serverClientParts.length == 2) { - //LOG.warn("ClientParts: " + serverClientParts[1]); - String clientParts[] = ConfigUtils.getHostAndPort(serverClientParts[1]); - if (clientParts.length > 2) { - throw new ConfigException(addressStr + wrongFormat); - } - - // is client_config a host:port or just a port - 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 InetSocketAddress tempAddress; InetSocketAddress tempElectionAddress; try { - tempAddress = new InetSocketAddress(serverParts[0], - Integer.parseInt(serverParts[1])); + 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])); + tempElectionAddress = new InetSocketAddress(serverParts[0], Integer.parseInt(serverParts[2])); electionAddr.addAddress(tempElectionAddress); } catch (NumberFormatException e) { throw new ConfigException("Address unresolved: " + serverParts[0] + ":" + serverParts[2]); From 30edf454199a990a13fe4978f6c9262cce3bc62e Mon Sep 17 00:00:00 2001 From: ArtemChernatsky Date: Thu, 6 Dec 2018 14:10:04 +0200 Subject: [PATCH 03/16] fixed ipv6 --- .../org/apache/zookeeper/server/quorum/QuorumCnxManager.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java index 630c2169a0d..1285830bbd6 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java @@ -404,8 +404,7 @@ private boolean startConnection(Socket sock, Long sid) // represents protocol version (in other words - message type) dout.writeLong(PROTOCOL_VERSION); dout.writeLong(self.getId()); - InetSocketAddress address = self.getElectionAddress().getValidAddress(); - String addr = address.getHostString() + ":" + address.getPort(); + String addr = formatInetAddr(self.getElectionAddress().getValidAddress()); byte[] addr_bytes = addr.getBytes(); dout.writeInt(addr_bytes.length); dout.write(addr_bytes); From 9a83af79dbf04ca014895de62930301335001722 Mon Sep 17 00:00:00 2001 From: ArtemChernatsky Date: Wed, 12 Dec 2018 12:51:30 +0200 Subject: [PATCH 04/16] improved work with multiple addresses --- .../zookeeper/server/quorum/Follower.java | 7 +- .../zookeeper/server/quorum/Leader.java | 163 +++++++++-------- .../zookeeper/server/quorum/Learner.java | 28 +-- .../server/quorum/MultipleAddresses.java | 38 +++- .../server/quorum/QuorumCnxManager.java | 170 +++++++++++------- .../server/quorum/MultipleAddressesTest.java | 133 ++++++++++++++ 6 files changed, 372 insertions(+), 167 deletions(-) create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java index 3f37ca5ce23..5a46dc5c085 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java @@ -50,7 +50,7 @@ public class Follower extends Learner{ this.self = self; this.zk=zk; this.fzk = zk; - reconnect = new AtomicBoolean(); + reconnect = new AtomicBoolean(true); } @Override @@ -82,11 +82,10 @@ void followLeader() throws InterruptedException { self.start_fle = 0; self.end_fle = 0; fzk.registerJMX(new FollowerBean(this, zk), self.jmxLocalPeerBean); - reconnect.set(true); try { QuorumServer leaderServer = findLeader(); - while (reconnect.get()) { + do { try { connectToLeader(leaderServer.addr, leaderServer.hostname); long newEpochZxid = registerWithLeader(Leader.FOLLOWERINFO); @@ -126,7 +125,7 @@ void followLeader() throws InterruptedException { // clear pending revalidations pendingRevalidations.clear(); } - } + } while (reconnect.get()); } finally { zk.unregisterJMX((Learner) this); } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java index 848d211e7ab..b9f823a94de 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java @@ -26,6 +26,7 @@ import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; import javax.security.sasl.SaslException; @@ -219,7 +220,7 @@ public boolean isQuorumSynced(QuorumVerifier qv) { return qv.containsQuorum(ids); } - private List ss = Collections.synchronizedList(new LinkedList<>()); + private List serverSockets; Leader(QuorumPeer self,LeaderZooKeeperServer zk) throws IOException, X509Exception { this.self = self; @@ -231,28 +232,30 @@ public boolean isQuorumSynced(QuorumVerifier qv) { else addresses = self.getQuorumAddress().getAllAddresses(); - for(InetSocketAddress addr : addresses) - ss.add(createServerSocket(addr, self.shouldUsePortUnification(), self.isSslQuorum())); + serverSockets = new LinkedList<>(); + for(InetSocketAddress address : addresses) + serverSockets.add(createServerSocket(address, self.shouldUsePortUnification(), self.isSslQuorum())); this.zk = zk; this.learnerSnapshotThrottler = createLearnerSnapshotThrottler( maxConcurrentSnapshots, maxConcurrentSnapshotTimeout); } - ServerSocket createServerSocket(InetSocketAddress addr, boolean portUnification, boolean sslQuorum) throws IOException, X509Exception { - ServerSocket temp; + ServerSocket createServerSocket(InetSocketAddress address, boolean portUnification, boolean sslQuorum) + throws IOException, X509Exception { + ServerSocket serverSocket; try { if (portUnification) { - temp = new UnifiedServerSocket(self.getX509Util(), true); + serverSocket = new UnifiedServerSocket(self.getX509Util(), true); } else if (sslQuorum) { - temp = self.getX509Util().createSSLServerSocket(); + serverSocket = self.getX509Util().createSSLServerSocket(); } else { - temp = new ServerSocket(); + serverSocket = new ServerSocket(); } - temp.setReuseAddress(true); - temp.bind(addr); - return temp; + serverSocket.setReuseAddress(true); + serverSocket.bind(address); + return serverSocket; } catch (X509Exception e) { LOG.error("Failed to setup ssl server socket", e); throw e; @@ -377,7 +380,9 @@ class LearnerCnxAcceptor extends ZooKeeperCriticalThread { private AtomicBoolean fail; public LearnerCnxAcceptor() { - super("LearnerCnxAcceptor-" + ss, zk + super("LearnerCnxAcceptor-" + + serverSockets.stream().map(ServerSocket::getLocalSocketAddress).map(Objects::toString) + .collect(Collectors.joining(",")), zk .getZooKeeperServerListener()); stop = new AtomicBoolean(false); fail = new AtomicBoolean(false); @@ -385,20 +390,21 @@ public LearnerCnxAcceptor() { @Override public void run() { - ExecutorService threadPool = Executors.newCachedThreadPool(); + if (!stop.get() && serverSockets != null) { + ExecutorService executor = Executors.newFixedThreadPool(serverSockets.size()); + CountDownLatch latch = new CountDownLatch(serverSockets.size()); - if (!stop.get()) { - ss.forEach(ssTemp -> threadPool.submit(new LearnerCnxAcceptorHandler(ssTemp))); - threadPool.shutdown(); - } + serverSockets.forEach(serverSocket -> + executor.submit(new LearnerCnxAcceptorHandler(serverSocket, latch))); - try { - while (true) - if (threadPool.awaitTermination(10, TimeUnit.MILLISECONDS)) - break; - } catch (InterruptedException ie) { - LOG.error("Interrupted while sleeping. " + - "Ignoring exception", ie); + try { + latch.await(); + } catch (InterruptedException ie) { + LOG.error("Interrupted while sleeping. " + + "Ignoring exception", ie); + } finally { + closeSockets(); + } } } @@ -408,68 +414,75 @@ public void halt() { } class LearnerCnxAcceptorHandler implements Runnable { + private ServerSocket serverSocket; + private CountDownLatch latch; - private ServerSocket ss; - - public LearnerCnxAcceptorHandler(ServerSocket ss) { - this.ss = ss; + LearnerCnxAcceptorHandler(ServerSocket serverSocket, CountDownLatch latch) { + this.serverSocket = serverSocket; + this.latch = latch; } @Override public void run() { - Thread.currentThread().setName("LeaderHandler-" + ss.getInetAddress()); try { + Thread.currentThread().setName("LearnerCnxAcceptorHandler-" + serverSocket.getLocalSocketAddress()); + while (!stop.get()) { - Socket s = null; - boolean error = false; - 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.get()) { - LOG.info("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 && s != null && !s.isClosed()) { - try { - s.close(); - } catch (IOException e) { - LOG.warn("Error closing socket", e); - } - } - } + acceptConnections(); } } catch (Exception e) { LOG.warn("Exception while accepting follower", e.getMessage()); - if(!fail.get()) { + if (!fail.get()) { handleException(getName(), e); fail.set(true); 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.info("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", e); + } + } } } } + } StateSummary leaderStateSummary; @@ -750,16 +763,16 @@ void shutdown(String reason) { isShutdown = true; } - void closeSockets() { - for (ServerSocket tempSocket : ss) { - if (!tempSocket.isClosed()) - if (!tempSocket.isClosed()) + synchronized void closeSockets() { + if (serverSockets != null) + for (ServerSocket serverSocket : serverSockets) { + if (!serverSocket.isClosed()) try { - tempSocket.close(); + serverSocket.close(); } catch (IOException e) { - LOG.warn("Ignoring unexpected exception during close" + tempSocket, 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. diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java index 539cf43c40c..7429b1869dd 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java @@ -29,6 +29,7 @@ import java.net.Socket; import java.nio.ByteBuffer; import java.util.LinkedList; +import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.concurrent.ConcurrentHashMap; @@ -254,36 +255,35 @@ protected void connectToLeader(MultipleAddresses addr, String hostname) boolean connected = false; + List addresses = addr.getAllAddresses(); for (int tries = 0; tries < 5; tries++) { - for (InetSocketAddress address : addr.getAllAddresses()) { + for (InetSocketAddress address : addresses) { try { // recalculate the init limit time because retries sleep for 1000 milliseconds remainingInitLimitTime = initLimitTime - (int) ((nanoTime() - startNanoTime) / 1000000); if (remainingInitLimitTime <= 0) { LOG.error("initLimit exceeded on retries."); - throw new IOException("initLimit exceeded on retries."); + throw new RuntimeLearnerException("initLimit exceeded on retries."); } - if (address.getAddress().isReachable(100)) { - sockConnect(sock, address, Math.min(self.tickTime * self.syncLimit, remainingInitLimitTime)); - if (self.isSslQuorum()) { - ((SSLSocket) sock).startHandshake(); - } - sock.setTcpNoDelay(nodelay); - connected = true; - break; + sockConnect(sock, address, Math.min(self.tickTime * self.syncLimit, remainingInitLimitTime)); + if (self.isSslQuorum()) { + ((SSLSocket) sock).startHandshake(); } + sock.setTcpNoDelay(nodelay); + connected = true; + break; } catch (IOException e) { remainingInitLimitTime = initLimitTime - (int) ((nanoTime() - startNanoTime) / 1000000); - if (remainingInitLimitTime <= 1000) { + if (remainingInitLimitTime <= 1000 / addresses.size()) { LOG.error("Unexpected exception, initLimit exceeded. tries=" + tries + ", remaining init limit=" + remainingInitLimitTime + - ", connecting to " + address, e); + ", connecting to " + addr, e); throw new RuntimeLearnerException(e); } else if (tries >= 4) { LOG.error("Unexpected exception, retries exceeded. tries=" + tries + ", remaining init limit=" + remainingInitLimitTime + - ", connecting to " + address, e); + ", connecting to " + addr, e); throw new RuntimeLearnerException(e); } else { LOG.warn("Unexpected exception, tries=" + tries + @@ -296,7 +296,7 @@ protected void connectToLeader(MultipleAddresses addr, String hostname) if (connected) break; - Thread.sleep(1000); + Thread.sleep(1000 / addresses.size()); } if (!connected) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java index 55a80a49a0a..6a67ac6b02f 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java @@ -13,19 +13,39 @@ public class MultipleAddresses { private Set addresses; + private int timeout; public MultipleAddresses() { addresses = Collections.newSetFromMap(new ConcurrentHashMap<>()); + timeout = 100; } public MultipleAddresses(List addresses) { + this(addresses, 100); + } + + public MultipleAddresses(InetSocketAddress address) { + this(address, 100); + } + + public MultipleAddresses(List addresses, int timeout) { this.addresses = Collections.newSetFromMap(new ConcurrentHashMap<>()); this.addresses.addAll(addresses); + this.timeout = timeout; } - public MultipleAddresses(InetSocketAddress address) { + public MultipleAddresses(InetSocketAddress address, int timeout) { addresses = Collections.newSetFromMap(new ConcurrentHashMap<>()); addresses.add(address); + this.timeout = timeout; + } + + public int getTimeout() { + return timeout; + } + + public void setTimeout(int timeout) { + this.timeout = timeout; } public boolean isEmpty() { @@ -41,7 +61,7 @@ public List getAllAddressesForAllPorts() { } public List getAllPorts() { - return addresses.stream().map(InetSocketAddress::getPort).collect(Collectors.toList()); + return addresses.stream().map(InetSocketAddress::getPort).distinct().collect(Collectors.toList()); } public void addAddress(InetSocketAddress address) { @@ -50,11 +70,17 @@ public void addAddress(InetSocketAddress address) { public InetSocketAddress getValidAddress() { - for(InetSocketAddress addr : addresses) { + for(int i = 0; i < 3; i++) { + for (InetSocketAddress addr : addresses) { + try { + if (addr.getAddress().isReachable(timeout)) + return addr; + } catch (NullPointerException | IOException ignored) { + } + } try { - if (addr.getAddress().isReachable(100)) - return addr; - } catch (NullPointerException | IOException e) { + Thread.sleep(200); + } catch (InterruptedException ignored) { } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java index 1285830bbd6..d69d0255316 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java @@ -27,14 +27,14 @@ import java.net.SocketTimeoutException; import java.nio.BufferUnderflowException; import java.nio.ByteBuffer; -import java.nio.channels.UnresolvedAddressException; import java.util.*; import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; -import java.util.stream.Stream; +import org.apache.zookeeper.common.NetUtils; import org.apache.zookeeper.common.X509Exception; import org.apache.zookeeper.server.ExitCode; import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException; @@ -841,41 +841,49 @@ private void resetConnectionThreadCount() { public class Listener extends ZooKeeperThread { List listenerHandlers; + private AtomicBoolean bindException; public Listener() { // During startup of thread, thread name will be overridden to // specific election address super("ListenerThread"); + bindException = new AtomicBoolean(false); } @Override public void run() { - while(!shutdown) { - Stream addresses; + if(!shutdown) { + List addresses; if (self.getQuorumListenOnAllIPs()) - addresses = self.getElectionAddress().getAllAddressesForAllPorts().stream(); + addresses = self.getElectionAddress().getAllAddressesForAllPorts(); else - addresses = self.getElectionAddress().getAllAddresses().stream(); + addresses = self.getElectionAddress().getAllAddresses(); - listenerHandlers = addresses - .map(addr -> new ListenerHandler(addr, self.shouldUsePortUnification(), self.isSslQuorum())) + CountDownLatch latch = new CountDownLatch(addresses.size()); + listenerHandlers = addresses.stream().map(address -> + new ListenerHandler(address, self.shouldUsePortUnification(), self.isSslQuorum(), latch)) .collect(Collectors.toList()); - ExecutorService threadPool = Executors.newCachedThreadPool(); - listenerHandlers.forEach(threadPool::submit); - threadPool.shutdown(); + ExecutorService executor = Executors.newFixedThreadPool(addresses.size()); + listenerHandlers.forEach(executor::submit); try { - while (true) - if (threadPool.awaitTermination(10, TimeUnit.MILLISECONDS)) - break; + latch.await(); } catch (InterruptedException ie) { LOG.error("Interrupted while sleeping. " + "Ignoring exception", ie); + } finally { + // Clean up for shutdown. + for (ListenerHandler handler : listenerHandlers) + try { + handler.close(); + } catch (IOException ie) { + // Don't log an error for shutdown. + LOG.debug("Error closing server socket", ie); + } } - } LOG.info("Leaving listener"); @@ -883,23 +891,21 @@ public void run() { LOG.error("As I'm leaving the listener thread, " + "I won't be able to participate in leader " + "election any longer: " - + self.getElectionAddress().getAllAddresses()); - } else if (listenerHandlers != null) { - // Clean up for shutdown. - for (ListenerHandler handler : listenerHandlers) - try { - handler.close(); - } catch (IOException ie) { - // Don't log an error for shutdown. - LOG.debug("Error closing server socket", ie); - } + + self.getElectionAddress().getAllAddresses().stream() + .map(NetUtils::formatInetAddr).collect(Collectors.joining(","))); + if (bindException.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 + System.exit(ExitCode.UNABLE_TO_BIND_QUORUM_PORT.getValue()); + } } } /** * Halts this listener thread. */ - void halt(){ + void halt() { LOG.debug("Trying to close listeners"); if (listenerHandlers != null) { LOG.debug("Closing listener: " @@ -914,63 +920,98 @@ void halt(){ } class ListenerHandler implements Runnable, Closeable { - private final Logger LOG = LoggerFactory.getLogger(ListenerHandler.class); private final static int ATTEMPTS_AMOUNT = 3; - private ServerSocket ss; + private ServerSocket serverSocket; private InetSocketAddress address; private boolean portUnification; private boolean sslQuorum; + private CountDownLatch latch; - ListenerHandler(InetSocketAddress address, boolean portUnification, boolean sslQuorum) { + ListenerHandler(InetSocketAddress address, boolean portUnification, boolean sslQuorum, + CountDownLatch latch) { this.address = address; this.portUnification = portUnification; this.sslQuorum = sslQuorum; + this.latch = latch; } /** - * Sleeps on accept(). + * Sleeps on acceptConnections(). */ @Override public void run() { - Thread.currentThread().setName("ListenerHandler-" + address); + 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) && (numRetries < ATTEMPTS_AMOUNT)){ + while ((!shutdown) && (numRetries < ATTEMPTS_AMOUNT)) { try { - ss = createNewServerSocket(); + serverSocket = createNewServerSocket(); LOG.info("My election bind port: " + address.toString()); while (!shutdown) { - client = ss.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); + 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"); } - 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|X509Exception e) { + } catch (IOException | X509Exception e) { if (shutdown) { break; } + LOG.error("Exception while listening", e); + + if (e instanceof BindException) + bindException.set(true); + numRetries++; try { - ss.close(); + close(); Thread.sleep(1000); } catch (IOException ie) { LOG.error("Error closing server socket", ie); @@ -983,32 +1024,25 @@ public void run() { } } - @Override - public void close() throws IOException { - if(ss != null) { - LOG.debug("Trying to close listeners: " + ss); - ss.close(); - } - } - private ServerSocket createNewServerSocket() throws IOException, X509Exception { - ServerSocket temp; + ServerSocket socket; if (portUnification) { - temp = new UnifiedServerSocket(self.getX509Util(), true); + socket = new UnifiedServerSocket(self.getX509Util(), true); } else if (sslQuorum) { - temp = self.getX509Util().createSSLServerSocket(); + socket = self.getX509Util().createSSLServerSocket(); } else { - temp = new ServerSocket(); + socket = new ServerSocket(); } - temp.setReuseAddress(true); - temp.bind(address); + socket.setReuseAddress(true); + socket.bind(address); - return temp; + return socket; } } + } /** diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java new file mode 100644 index 00000000000..cfd9cb2a64b --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java @@ -0,0 +1,133 @@ +package org.apache.zookeeper.server.quorum; + +import org.apache.commons.collections.CollectionUtils; +import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.server.quorum.exception.RuntimeNoReachableHostException; +import org.junit.Assert; +import org.junit.Test; + +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.UnknownHostException; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +public class MultipleAddressesTest { + + public final static 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 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 testGetAllPorts() { + List 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 testGetAllAddressesForAllPorts() { + List ports = getPortList(); + List addresses = getAddressList(ports); + MultipleAddresses multipleAddresses = new MultipleAddresses(addresses); + List allAddresses = ports.stream().map(InetSocketAddress::new).collect(Collectors.toList()); + + Assert.assertTrue(CollectionUtils.isEqualCollection(allAddresses, multipleAddresses.getAllAddressesForAllPorts())); + + multipleAddresses.addAddress(new InetSocketAddress("localhost", ports.get(ports.size() - 1))); + Assert.assertTrue(CollectionUtils.isEqualCollection(allAddresses, multipleAddresses.getAllAddressesForAllPorts())); + } + + @Test + public void testGetValidAddress() { + List addresses = getAddressList(); + MultipleAddresses multipleAddresses = new MultipleAddresses(addresses); + + Assert.assertTrue(addresses.contains(multipleAddresses.getValidAddress())); + } + + @Test(expected = RuntimeNoReachableHostException.class) + public void testGetValidAddressWithNotValid() { + MultipleAddresses multipleAddresses = new MultipleAddresses(new InetSocketAddress("10.0.0.1", 22)); + multipleAddresses.getValidAddress(); + } + + @Test + public void testRecreateSocketAddresses() throws UnknownHostException { + List 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 addresses = multipleAddresses.getAllAddresses(); + + Assert.assertEquals(1, addresses.size()); + Assert.assertEquals(searchedAddresses.get(searchedAddresses.size() - 1), addresses.get(0)); + + multipleAddresses.recreateSocketAddresses(); + + addresses = 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.getAllAddresses().get(0)); + } + + @Test + public void testEquals() { + List 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 getPortList() { + return IntStream.range(0, PORTS_AMOUNT).mapToObj(i -> PortAssignment.unique()).collect(Collectors.toList()); + } + + public List getAddressList() { + return getAddressList(getPortList()); + } + + public List getAddressList(List ports) { + return IntStream.range(0, ports.size()) + .mapToObj(i -> new InetSocketAddress("127.0.0." + i, ports.get(i))).collect(Collectors.toList()); + } + +} From 818663cbff8b8fe7b5f2cc204985ab9c39b5828c Mon Sep 17 00:00:00 2001 From: ArtemChernatsky Date: Wed, 12 Dec 2018 15:53:56 +0200 Subject: [PATCH 05/16] increased timeout --- .../src/test/java/org/apache/zookeeper/test/ClientBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientBase.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientBase.java index 4cb42ed06dc..680c531f0cd 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientBase.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientBase.java @@ -70,7 +70,7 @@ public abstract class ClientBase extends ZKTestCase { protected static final Logger LOG = LoggerFactory.getLogger(ClientBase.class); - public static int CONNECTION_TIMEOUT = 30000; + public static int CONNECTION_TIMEOUT = 35000; static final File BASETEST = new File(System.getProperty("build.test.dir", "build")); From dd015b9e45f0e5e5b8a2e92576e669101d33feb8 Mon Sep 17 00:00:00 2001 From: ArtemChernatsky Date: Mon, 17 Dec 2018 08:49:14 +0200 Subject: [PATCH 06/16] integrated with master updates --- .../java/org/apache/zookeeper/server/ObserverBean.java | 5 ++++- .../apache/zookeeper/server/quorum/ObserverMaster.java | 4 ++-- .../org/apache/zookeeper/server/quorum/QuorumPeer.java | 10 ++++++---- 3 files changed, 12 insertions(+), 7 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ObserverBean.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ObserverBean.java index 167c96d27d8..923658e1cd6 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ObserverBean.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ObserverBean.java @@ -22,6 +22,8 @@ import org.apache.zookeeper.server.quorum.ObserverMXBean; import org.apache.zookeeper.server.quorum.QuorumPeer; +import java.net.InetSocketAddress; + /** * ObserverBean * @@ -52,7 +54,8 @@ public String getLearnerMaster() { if (learnerMaster == null || learnerMaster.addr == null) { return "Unknown"; } - return learnerMaster.addr.getAddress().getHostAddress() + ":" + learnerMaster.addr.getPort(); + InetSocketAddress address = learnerMaster.addr.getValidAddress(); + return address.getAddress().getHostAddress() + ":" + address.getPort(); } public void setLearnerMaster(String learnerMaster) { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverMaster.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverMaster.java index 07de57be86a..556da7f6fd8 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverMaster.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverMaster.java @@ -445,13 +445,13 @@ synchronized public void start() throws IOException { allowInsecureConnection, port, backlog, - self.getQuorumAddress().getAddress()); + self.getQuorumAddress().getValidAddress().getAddress()); } } else { if (self.getQuorumListenOnAllIPs()) { ss = new ServerSocket(port, backlog); } else { - ss = new ServerSocket(port, backlog, self.getQuorumAddress().getAddress()); + ss = new ServerSocket(port, backlog, self.getQuorumAddress().getValidAddress().getAddress()); } } thread = new Thread(this, "ObserverMaster"); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java index ddefe688e08..b0653b52f68 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -2020,7 +2020,7 @@ private void updateObserverMasterList() { observerMasters.clear(); StringBuilder sb = new StringBuilder(); for (QuorumServer server : quorumVerifier.getVotingMembers().values()) { - InetSocketAddress addr = new InetSocketAddress(server.addr.getAddress(), observerMasterPort); + InetSocketAddress addr = new InetSocketAddress(server.addr.getValidAddress().getAddress(), observerMasterPort); observerMasters.add(new QuorumServer(server.id, addr)); sb.append(addr).append(","); } @@ -2058,9 +2058,11 @@ QuorumServer validateLearnerMaster(String desiredMaster) { } 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)) { From 229a0272e0f1763b9dab511f3df951e9dd6ee2a3 Mon Sep 17 00:00:00 2001 From: ArtemChernatsky Date: Mon, 17 Dec 2018 12:58:24 +0200 Subject: [PATCH 07/16] Revert "increased timeout" This reverts commit 818663cb --- .../src/test/java/org/apache/zookeeper/test/ClientBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientBase.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientBase.java index 680c531f0cd..4cb42ed06dc 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientBase.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientBase.java @@ -70,7 +70,7 @@ public abstract class ClientBase extends ZKTestCase { protected static final Logger LOG = LoggerFactory.getLogger(ClientBase.class); - public static int CONNECTION_TIMEOUT = 35000; + public static int CONNECTION_TIMEOUT = 30000; static final File BASETEST = new File(System.getProperty("build.test.dir", "build")); From 976e6d5e62520f34e72cc7941becef5f1a96ec91 Mon Sep 17 00:00:00 2001 From: ArtemChernatsky Date: Mon, 17 Dec 2018 17:57:26 +0200 Subject: [PATCH 08/16] fixed sockets closing and empty addresses lists --- .../java/org/apache/zookeeper/server/quorum/Follower.java | 6 +++++- .../apache/zookeeper/server/quorum/MultipleAddresses.java | 5 ++--- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java index 402018a199b..281fdacdcc3 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java @@ -126,6 +126,11 @@ void followLeader() throws InterruptedException { } catch (SocketException | InterruptedIOException ignored) { zk.unregisterJMX(); LOG.warn("Error when following the leader, reconnecting"); + try { + sock.close(); + } catch (IOException e1) { + e1.printStackTrace(); + } } catch (Exception e) { reconnect.set(false); LOG.warn("Exception when following the leader", e); @@ -134,7 +139,6 @@ void followLeader() throws InterruptedException { } catch (IOException e1) { e1.printStackTrace(); } - // clear pending revalidations pendingRevalidations.clear(); } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java index 6a67ac6b02f..497cc708c07 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java @@ -88,7 +88,7 @@ public InetSocketAddress getValidAddress() { } public void recreateSocketAddresses() { - Set temp = new HashSet<>(); + Set temp = Collections.newSetFromMap(new ConcurrentHashMap<>()); for(InetSocketAddress addr : addresses) { try { @@ -98,8 +98,7 @@ public void recreateSocketAddresses() { } } - addresses.clear(); - addresses.addAll(temp); + addresses = temp; } @Override From 08f8d94e5ac212f18ca534433167257203f85000 Mon Sep 17 00:00:00 2001 From: ArtemChernatsky Date: Tue, 18 Dec 2018 14:19:03 +0200 Subject: [PATCH 09/16] added comments and improved naming --- .../zookeeper/server/quorum/Leader.java | 2 +- .../server/quorum/MultipleAddresses.java | 32 ++++++++++++++++++- .../server/quorum/QuorumCnxManager.java | 2 +- .../server/quorum/MultipleAddressesTest.java | 6 ++-- 4 files changed, 36 insertions(+), 6 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java index 48a5cc8c661..92a67bee870 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java @@ -242,7 +242,7 @@ public boolean isQuorumSynced(QuorumVerifier qv) { List addresses; if (self.getQuorumListenOnAllIPs()) - addresses = self.getQuorumAddress().getAllAddressesForAllPorts(); + addresses = self.getQuorumAddress().getWildcardAddresses(); else addresses = self.getQuorumAddress().getAllAddresses(); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java index 497cc708c07..e29b7683d6f 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java @@ -10,6 +10,11 @@ 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 class MultipleAddresses { private Set addresses; @@ -52,14 +57,29 @@ public boolean isEmpty() { return addresses.isEmpty(); } + /** + * Returns all addresses. + * + * @return list of all InetSocketAddress + */ public List getAllAddresses() { return new LinkedList<>(addresses); } - public List getAllAddressesForAllPorts() { + /** + * Returns wildcard addresses for all ports + * + * @return list of InetSocketAddress with wildcards for all ports + */ + public List getWildcardAddresses() { return addresses.stream().map(a -> new InetSocketAddress(a.getPort())).distinct().collect(Collectors.toList()); } + /** + * Returns all ports + * + * @return list of all ports + */ public List getAllPorts() { return addresses.stream().map(InetSocketAddress::getPort).distinct().collect(Collectors.toList()); } @@ -68,6 +88,11 @@ public void addAddress(InetSocketAddress address) { addresses.add(address); } + /** + * Returns reachable address. If none is reachable than throws exception. + * + * @return address which is reachable. + */ public InetSocketAddress getValidAddress() { for(int i = 0; i < 3; i++) { @@ -87,6 +112,11 @@ public InetSocketAddress getValidAddress() { throw new RuntimeNoReachableHostException("No valid address among " + addresses); } + /** + * Performs a DNS lookup for addresses. + * + * If the DNS lookup fails, than address remain unmodified. + */ public void recreateSocketAddresses() { Set temp = Collections.newSetFromMap(new ConcurrentHashMap<>()); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java index 211dcf297e8..24a36955ace 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java @@ -857,7 +857,7 @@ public void run() { List addresses; if (self.getQuorumListenOnAllIPs()) - addresses = self.getElectionAddress().getAllAddressesForAllPorts(); + addresses = self.getElectionAddress().getWildcardAddresses(); else addresses = self.getElectionAddress().getAllAddresses(); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java index cfd9cb2a64b..0a539b92e01 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java @@ -50,16 +50,16 @@ public void testGetAllPorts() { } @Test - public void testGetAllAddressesForAllPorts() { + public void testGetWildcardAddresses() { List ports = getPortList(); List addresses = getAddressList(ports); MultipleAddresses multipleAddresses = new MultipleAddresses(addresses); List allAddresses = ports.stream().map(InetSocketAddress::new).collect(Collectors.toList()); - Assert.assertTrue(CollectionUtils.isEqualCollection(allAddresses, multipleAddresses.getAllAddressesForAllPorts())); + Assert.assertTrue(CollectionUtils.isEqualCollection(allAddresses, multipleAddresses.getWildcardAddresses())); multipleAddresses.addAddress(new InetSocketAddress("localhost", ports.get(ports.size() - 1))); - Assert.assertTrue(CollectionUtils.isEqualCollection(allAddresses, multipleAddresses.getAllAddressesForAllPorts())); + Assert.assertTrue(CollectionUtils.isEqualCollection(allAddresses, multipleAddresses.getWildcardAddresses())); } @Test From af05e0b53e36ed9895000345e0897a0348ce40fb Mon Sep 17 00:00:00 2001 From: ArtemChernatsky Date: Mon, 24 Dec 2018 12:07:17 +0200 Subject: [PATCH 10/16] added parallel attempts to connect and check if address is reachable --- .../zookeeper/server/quorum/Follower.java | 103 +++++++----------- .../zookeeper/server/quorum/Learner.java | 102 +++++++++++------ .../server/quorum/MultipleAddresses.java | 53 +++++---- .../exception/RuntimeConfigException.java | 15 --- .../exception/RuntimeLearnerException.java | 19 ---- .../zookeeper/server/quorum/LearnerTest.java | 5 +- .../server/quorum/QuorumPeerMainTest.java | 2 - .../server/quorum/RaceConditionTest.java | 3 +- .../zookeeper/server/quorum/Zab1_0Test.java | 1 - 9 files changed, 146 insertions(+), 157 deletions(-) delete mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeConfigException.java delete mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeLearnerException.java diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java index 281fdacdcc3..12d5964c785 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java @@ -19,10 +19,7 @@ package org.apache.zookeeper.server.quorum; import java.io.IOException; -import java.io.InterruptedIOException; -import java.net.SocketException; import java.nio.ByteBuffer; -import java.util.concurrent.atomic.AtomicBoolean; import org.apache.jute.Record; import org.apache.zookeeper.ZooDefs.OpCode; @@ -44,7 +41,6 @@ public class Follower extends Learner{ private long lastQueued; // This is the same object as this.zk, but we cache the downcast op final FollowerZooKeeperServer fzk; - private AtomicBoolean reconnect; ObserverMaster om; @@ -52,7 +48,6 @@ public class Follower extends Learner{ this.self = self; this.zk=zk; this.fzk = zk; - reconnect = new AtomicBoolean(true); } @Override @@ -65,10 +60,6 @@ public String toString() { return sb.toString(); } - public void disableReconnect() { - reconnect.set(false); - } - /** * the main method called by the follower to follow the leader * @@ -87,62 +78,50 @@ void followLeader() throws InterruptedException { try { QuorumServer leaderServer = findLeader(); - do { + try { + connectToLeader(leaderServer.addr, leaderServer.hostname); + long newEpochZxid = registerWithLeader(Leader.FOLLOWERINFO); + if (self.isReconfigStateChange()) + throw new Exception("learned about role change"); + //check to see if the leader zxid is lower than ours + //this should never happen but is just a safety check + long newEpoch = ZxidUtils.getEpochFromZxid(newEpochZxid); + if (newEpoch < self.getAcceptedEpoch()) { + LOG.error("Proposed leader epoch " + ZxidUtils.zxidToString(newEpochZxid) + + " is less than our accepted epoch " + ZxidUtils.zxidToString(self.getAcceptedEpoch())); + throw new IOException("Error: Epoch of leader is lower"); + } + long startTime = Time.currentElapsedTime(); try { - connectToLeader(leaderServer.addr, leaderServer.hostname); - long newEpochZxid = registerWithLeader(Leader.FOLLOWERINFO); - if (self.isReconfigStateChange()) - throw new Exception("learned about role change"); - //check to see if the leader zxid is lower than ours - //this should never happen but is just a safety check - long newEpoch = ZxidUtils.getEpochFromZxid(newEpochZxid); - if (newEpoch < self.getAcceptedEpoch()) { - LOG.error("Proposed leader epoch " + ZxidUtils.zxidToString(newEpochZxid) - + " is less than our accepted epoch " + ZxidUtils.zxidToString(self.getAcceptedEpoch())); - throw new IOException("Error: Epoch of leader is lower"); - } - long startTime = Time.currentElapsedTime(); - try { - syncWithLeader(newEpochZxid); - } finally { - long syncTime = Time.currentElapsedTime() - startTime; - ServerMetrics.FOLLOWER_SYNC_TIME.add(syncTime); - } - if (self.getObserverMasterPort() > 0) { - LOG.info("Starting ObserverMaster"); + syncWithLeader(newEpochZxid); + } finally { + long syncTime = Time.currentElapsedTime() - startTime; + ServerMetrics.FOLLOWER_SYNC_TIME.add(syncTime); + } + if (self.getObserverMasterPort() > 0) { + LOG.info("Starting ObserverMaster"); - om = new ObserverMaster(self, fzk, self.getObserverMasterPort()); - om.start(); - } else { - om = null; - } - // create a reusable packet to reduce gc impact - QuorumPacket qp = new QuorumPacket(); - while (this.isRunning()) { - readPacket(qp); - processPacket(qp); - } - reconnect.set(false); - } catch (SocketException | InterruptedIOException ignored) { - zk.unregisterJMX(); - LOG.warn("Error when following the leader, reconnecting"); - try { - sock.close(); - } catch (IOException e1) { - e1.printStackTrace(); - } - } catch (Exception e) { - reconnect.set(false); - LOG.warn("Exception when following the leader", e); - try { - sock.close(); - } catch (IOException e1) { - e1.printStackTrace(); - } - // clear pending revalidations - pendingRevalidations.clear(); + om = new ObserverMaster(self, fzk, self.getObserverMasterPort()); + om.start(); + } else { + om = null; } - } while (reconnect.get()); + // create a reusable packet to reduce gc impact + QuorumPacket qp = new QuorumPacket(); + while (this.isRunning()) { + readPacket(qp); + processPacket(qp); + } + } catch (Exception e) { + LOG.warn("Exception when following the leader", e); + try { + sock.close(); + } catch (IOException e1) { + e1.printStackTrace(); + } + // clear pending revalidations + pendingRevalidations.clear(); + } } finally { if (om != null) { om.stop(); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java index 8cb96ca81c0..560ea75b8f1 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java @@ -24,7 +24,6 @@ import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; -import java.net.ConnectException; import java.net.InetSocketAddress; import java.net.Socket; import java.nio.ByteBuffer; @@ -33,6 +32,10 @@ import java.util.Map; import java.util.Map.Entry; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicReference; import org.apache.jute.BinaryInputArchive; import org.apache.jute.BinaryOutputArchive; @@ -40,7 +43,6 @@ import org.apache.jute.OutputArchive; import org.apache.jute.Record; import org.apache.zookeeper.common.X509Exception; -import org.apache.zookeeper.server.quorum.exception.RuntimeLearnerException; import org.apache.zookeeper.server.ExitCode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -251,72 +253,106 @@ protected void sockConnect(Socket sock, InetSocketAddress addr, int timeout) * @param addr - 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 ConnectException * @throws InterruptedException */ protected void connectToLeader(MultipleAddresses addr, String hostname) - throws IOException, ConnectException, InterruptedException, X509Exception { - this.sock = createSocket(); + throws IOException, InterruptedException { - int initLimitTime = self.tickTime * self.initLimit; - int remainingInitLimitTime; - long startNanoTime = nanoTime(); + List addresses = addr.getAllAddresses(); + ExecutorService executor = Executors.newFixedThreadPool(addresses.size()); + CountDownLatch latch = new CountDownLatch(addresses.size()); + AtomicReference socket = new AtomicReference<>(null); + addresses.stream().map(address -> new LeaderConnector(address, socket, latch)).forEach(executor::submit); - boolean connected = false; + latch.await(); - List addresses = addr.getAllAddresses(); - for (int tries = 0; tries < 5; tries++) { - for (InetSocketAddress address : addresses) { + if(socket.get() == null) + throw new IOException("Failed connect to " + addr); + else + sock = socket.get(); + + self.authLearner.authenticate(sock, hostname); + + leaderIs = BinaryInputArchive.getArchive(new BufferedInputStream( + sock.getInputStream())); + bufferedOutput = new BufferedOutputStream(sock.getOutputStream()); + leaderOs = BinaryOutputArchive.getArchive(bufferedOutput); + } + + class LeaderConnector implements Runnable { + + private AtomicReference socket; + private InetSocketAddress address; + private CountDownLatch latch; + + LeaderConnector(InetSocketAddress address, AtomicReference 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() && !socket.compareAndSet(null, sock)) + sock.close(); + + } catch (Exception e) { + LOG.error("Failed to connect to " + address, e); + } finally { + latch.countDown(); + } + } + + private Socket connectToLeader() throws IOException, X509Exception, InterruptedException { + Socket sock = createSocket(); + + int initLimitTime = self.tickTime * self.initLimit; + int remainingInitLimitTime; + 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 remainingInitLimitTime = initLimitTime - (int) ((nanoTime() - startNanoTime) / 1000000); if (remainingInitLimitTime <= 0) { LOG.error("initLimit exceeded on retries."); - throw new RuntimeLearnerException("initLimit exceeded on retries."); + throw new IOException("initLimit exceeded on retries."); } sockConnect(sock, address, Math.min(self.tickTime * self.syncLimit, remainingInitLimitTime)); if (self.isSslQuorum()) { ((SSLSocket) sock).startHandshake(); } sock.setTcpNoDelay(nodelay); - connected = true; break; } catch (IOException e) { remainingInitLimitTime = initLimitTime - (int) ((nanoTime() - startNanoTime) / 1000000); - if (remainingInitLimitTime <= leaderConnectDelayDuringRetryMs / addresses.size()) { + if (remainingInitLimitTime <= leaderConnectDelayDuringRetryMs) { LOG.error("Unexpected exception, initLimit exceeded. tries=" + tries + ", remaining init limit=" + remainingInitLimitTime + - ", connecting to " + addr, e); - throw new RuntimeLearnerException(e); + ", connecting to " + address, e); + throw e; } else if (tries >= 4) { LOG.error("Unexpected exception, retries exceeded. tries=" + tries + ", remaining init limit=" + remainingInitLimitTime + - ", connecting to " + addr, e); - throw new RuntimeLearnerException(e); + ", connecting to " + address, e); + throw e; } else { LOG.warn("Unexpected exception, tries=" + tries + ", remaining init limit=" + remainingInitLimitTime + ", connecting to " + address, e); - this.sock = createSocket(); + sock = createSocket(); } } + Thread.sleep(leaderConnectDelayDuringRetryMs); } - if (connected) - break; - Thread.sleep(leaderConnectDelayDuringRetryMs / addresses.size()); + return sock; } - - if (!connected) - throw new RuntimeException("Failed connect to " + addr); - - self.authLearner.authenticate(sock, hostname); - - leaderIs = BinaryInputArchive.getArchive(new BufferedInputStream( - sock.getInputStream())); - bufferedOutput = new BufferedOutputStream(sock.getOutputStream()); - leaderOs = BinaryOutputArchive.getArchive(bufferedOutput); } private Socket createSocket() throws X509Exception, IOException { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java index e29b7683d6f..2a74429ad9d 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java @@ -1,14 +1,15 @@ package org.apache.zookeeper.server.quorum; import org.apache.zookeeper.server.quorum.exception.RuntimeNoReachableHostException; - import java.io.IOException; import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.UnknownHostException; import java.util.*; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; +import java.util.stream.Stream; /** * This class allows to store several quorum and electing addresses. @@ -94,22 +95,27 @@ public void addAddress(InetSocketAddress address) { * @return address which is reachable. */ public InetSocketAddress getValidAddress() { + AtomicReference address = new AtomicReference<>(null); + getInetSocketAddressStream().forEach(addr -> checkIfAddressIsReachableAndSet(addr, address)); - for(int i = 0; i < 3; i++) { - for (InetSocketAddress addr : addresses) { - try { - if (addr.getAddress().isReachable(timeout)) - return addr; - } catch (NullPointerException | IOException ignored) { - } - } + if(address.get() != null) + return address.get(); + else + throw new RuntimeNoReachableHostException("No valid address among " + addresses); + } + + private void checkIfAddressIsReachableAndSet(InetSocketAddress address, + AtomicReference reachableAddress) { + for(int i = 0; i < 5 && reachableAddress.get() == null; i++) { try { - Thread.sleep(200); - } catch (InterruptedException ignored) { + if(address.getAddress().isReachable((i + 1) * timeout)) { + reachableAddress.compareAndSet(null, address); + break; + } + Thread.sleep(timeout); + } catch (NullPointerException | IOException | InterruptedException ignored) { } } - - throw new RuntimeNoReachableHostException("No valid address among " + addresses); } /** @@ -119,16 +125,23 @@ public InetSocketAddress getValidAddress() { */ public void recreateSocketAddresses() { Set temp = Collections.newSetFromMap(new ConcurrentHashMap<>()); + temp.addAll(getInetSocketAddressStream().map(this::recreateSocketAddress).collect(Collectors.toSet())); + addresses = temp; + } - for(InetSocketAddress addr : addresses) { - try { - temp.add(new InetSocketAddress(InetAddress.getByName(addr.getHostString()), addr.getPort())); - } catch (UnknownHostException e) { - temp.add(addr); - } + private InetSocketAddress recreateSocketAddress(InetSocketAddress address) { + try { + return new InetSocketAddress(InetAddress.getByName(address.getHostString()), address.getPort()); + } catch (UnknownHostException e) { + return address; } + } - addresses = temp; + private Stream getInetSocketAddressStream() { + if(addresses.size() > 1) + return addresses.parallelStream(); + else + return addresses.stream(); } @Override diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeConfigException.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeConfigException.java deleted file mode 100644 index 7dc48aac805..00000000000 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeConfigException.java +++ /dev/null @@ -1,15 +0,0 @@ -package org.apache.zookeeper.server.quorum.exception; - -public class RuntimeConfigException extends RuntimeException { - - private static final long serialVersionUID = -9025894204684855418L; - - public RuntimeConfigException() { - super(); - } - - public RuntimeConfigException(String message) { - super(message); - } - -} diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeLearnerException.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeLearnerException.java deleted file mode 100644 index 0ca9c20a824..00000000000 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeLearnerException.java +++ /dev/null @@ -1,19 +0,0 @@ -package org.apache.zookeeper.server.quorum.exception; - -public class RuntimeLearnerException extends RuntimeException { - - private static final long serialVersionUID = 9164642202468819481L; - - public RuntimeLearnerException() { - super(); - } - - public RuntimeLearnerException(Throwable cause) { - super(cause); - } - - public RuntimeLearnerException(String message) { - super(message); - } - -} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerTest.java index 036aaaf68df..15c083db88a 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerTest.java @@ -35,7 +35,6 @@ import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.server.ZKDatabase; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; -import org.apache.zookeeper.server.quorum.exception.RuntimeLearnerException; import org.apache.zookeeper.test.TestUtils; import org.apache.zookeeper.txn.CreateTxn; import org.apache.zookeeper.txn.TxnHeader; @@ -99,7 +98,7 @@ protected void sockConnect(Socket sock, InetSocketAddress addr, int timeout) } } - @Test(expected= RuntimeLearnerException.class) + @Test(expected = IOException.class) public void connectionRetryTimeoutTest() throws Exception { Learner learner = new TimeoutLearner(); learner.self = new QuorumPeer(); @@ -133,7 +132,7 @@ public void connectionInitLimitTimeoutTest() throws Exception { try { learner.connectToLeader(new MultipleAddresses(addr), ""); Assert.fail("should have thrown IOException!"); - } catch (RuntimeLearnerException e) { + } catch (IOException e) { //good, wanted to see that, let's make sure we ran out of time Assert.assertTrue(learner.nanoTime() > 2000*5*1000000); Assert.assertEquals(3, learner.getSockConnectAttempt()); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java index 043b9bb94dc..b6fecd208e9 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java @@ -384,9 +384,7 @@ public void testElectionFraud() throws IOException, InterruptedException { Assert.assertTrue("All servers should join the quorum", servers.mt[falseLeader].main.quorumPeer.follower != null); // to keep the quorum peer running and force it to go into the looking state, we kill leader election - // and disable reconnect, close the connection to the leader servers.mt[falseLeader].main.quorumPeer.electionAlg.shutdown(); - servers.mt[falseLeader].main.quorumPeer.follower.disableReconnect(); servers.mt[falseLeader].main.quorumPeer.follower.getSocket().close(); // wait for the falseLeader to disconnect diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/RaceConditionTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/RaceConditionTest.java index 17d1cda624e..ea2a4d38518 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/RaceConditionTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/RaceConditionTest.java @@ -36,7 +36,6 @@ import org.apache.zookeeper.server.ZooKeeperServer; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; import org.apache.zookeeper.server.quorum.QuorumPeer.ServerState; -import org.apache.zookeeper.server.quorum.exception.RuntimeLearnerException; import org.apache.zookeeper.test.ClientBase; import org.junit.After; import org.junit.Assert; @@ -175,7 +174,7 @@ protected Follower makeFollower(FileTxnSnapLog logFactory) throws IOException { protected void processPacket(QuorumPacket qp) throws Exception { if (stopPing && qp.getType() == Leader.PING) { LOG.info("Follower skipped ping"); - throw new RuntimeLearnerException("Socket time out while sending the ping response"); + throw new SocketException("Socket time out while sending the ping response"); } else { super.processPacket(qp); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/Zab1_0Test.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/Zab1_0Test.java index 96c52bd6f89..aabd9d88222 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/Zab1_0Test.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/Zab1_0Test.java @@ -451,7 +451,6 @@ public void run() { conversation.converseWithFollower(ia, oa, follower); } finally { if (follower != null) { - follower.disableReconnect(); follower.shutdown(); } if (followerThread != null) { From 5164e2a33b4015954d6a41b8f228484e35bb9d72 Mon Sep 17 00:00:00 2001 From: ArtemChernatsky Date: Wed, 26 Dec 2018 17:40:15 +0200 Subject: [PATCH 11/16] added Apache license headers --- .../server/quorum/MultipleAddresses.java | 18 ++++++++++++++++++ .../RuntimeNoReachableHostException.java | 18 ++++++++++++++++++ .../server/quorum/MultipleAddressesTest.java | 18 ++++++++++++++++++ 3 files changed, 54 insertions(+) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java index 2a74429ad9d..ffccfd799ef 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java @@ -1,3 +1,21 @@ +/** + * 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 org.apache.zookeeper.server.quorum.exception.RuntimeNoReachableHostException; diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeNoReachableHostException.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeNoReachableHostException.java index 811038e3245..2a542494867 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeNoReachableHostException.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeNoReachableHostException.java @@ -1,3 +1,21 @@ +/** + * 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.exception; public class RuntimeNoReachableHostException extends RuntimeException { diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java index 0a539b92e01..46bbffdce21 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java @@ -1,3 +1,21 @@ +/** + * 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 org.apache.commons.collections.CollectionUtils; From a98774ace18b85da033f7e41b6f13414dcc00ac0 Mon Sep 17 00:00:00 2001 From: ArtemChernatsky Date: Sat, 29 Dec 2018 16:33:23 +0200 Subject: [PATCH 12/16] improved exceptions handling --- .../apache/zookeeper/server/ObserverBean.java | 8 ++- .../server/quorum/AuthFastLeaderElection.java | 22 ++++--- .../server/quorum/MultipleAddresses.java | 35 ++++++----- .../server/quorum/ObserverMaster.java | 19 +++--- .../server/quorum/QuorumCnxManager.java | 63 ++++++++++--------- .../zookeeper/server/quorum/QuorumPeer.java | 13 ++-- .../RuntimeNoReachableHostException.java | 33 ---------- .../server/quorum/MultipleAddressesTest.java | 8 +-- .../apache/zookeeper/test/CnxManagerTest.java | 43 ++++++++++--- 9 files changed, 129 insertions(+), 115 deletions(-) delete mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeNoReachableHostException.java diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ObserverBean.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ObserverBean.java index 923658e1cd6..e8975238ce5 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ObserverBean.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ObserverBean.java @@ -23,6 +23,7 @@ import org.apache.zookeeper.server.quorum.QuorumPeer; import java.net.InetSocketAddress; +import java.net.NoRouteToHostException; /** * ObserverBean @@ -54,7 +55,12 @@ public String getLearnerMaster() { if (learnerMaster == null || learnerMaster.addr == null) { return "Unknown"; } - InetSocketAddress address = learnerMaster.addr.getValidAddress(); + InetSocketAddress address; + try { + address = learnerMaster.addr.getValidAddress(); + } catch (NoRouteToHostException e) { + address = learnerMaster.addr.getOne(); + } return address.getAddress().getHostAddress() + ":" + address.getPort(); } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java index a2eb9f9fd97..4a8e5dad823 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java @@ -19,10 +19,7 @@ package org.apache.zookeeper.server.quorum; import java.io.IOException; -import java.net.DatagramPacket; -import java.net.DatagramSocket; -import java.net.InetSocketAddress; -import java.net.SocketException; +import java.net.*; import java.nio.ByteBuffer; import java.util.Collection; import java.util.Collections; @@ -727,7 +724,13 @@ private void process(ToSend m) { } for (QuorumServer server : self.getVotingView().values()) { - InetSocketAddress saddr = new InetSocketAddress(server.addr.getValidAddress().getAddress(), port); + InetAddress address; + try { + address = server.addr.getValidAddress().getAddress(); + } catch (NoRouteToHostException e) { + address = server.addr.getOne().getAddress(); + } + InetSocketAddress saddr = new InetSocketAddress(address, port); addrChallengeMap.put(saddr, new ConcurrentHashMap()); } @@ -780,10 +783,15 @@ private void leaveInstance() { private void sendNotifications() { for (QuorumServer server : self.getView().values()) { + InetSocketAddress address; + try { + address = self.getView().get(server.id).electionAddr.getValidAddress(); + } catch (NoRouteToHostException e) { + address = self.getView().get(server.id).electionAddr.getOne(); + } ToSend notmsg = new ToSend(ToSend.mType.notification, AuthFastLeaderElection.sequencer++, proposedLeader, - proposedZxid, logicalclock.get(), QuorumPeer.ServerState.LOOKING, - self.getView().get(server.id).electionAddr.getValidAddress()); + proposedZxid, logicalclock.get(), QuorumPeer.ServerState.LOOKING, address); sendqueue.offer(notmsg); } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java index ffccfd799ef..85720137873 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java @@ -18,10 +18,10 @@ package org.apache.zookeeper.server.quorum; -import org.apache.zookeeper.server.quorum.exception.RuntimeNoReachableHostException; import java.io.IOException; import java.net.InetAddress; import java.net.InetSocketAddress; +import java.net.NoRouteToHostException; import java.net.UnknownHostException; import java.util.*; import java.util.concurrent.ConcurrentHashMap; @@ -111,19 +111,35 @@ public void addAddress(InetSocketAddress address) { * Returns reachable address. If none is reachable than throws exception. * * @return address which is reachable. + * @throws NoRouteToHostException if none address is reachable */ - public InetSocketAddress getValidAddress() { + public InetSocketAddress getValidAddress() throws NoRouteToHostException { AtomicReference address = new AtomicReference<>(null); getInetSocketAddressStream().forEach(addr -> checkIfAddressIsReachableAndSet(addr, address)); if(address.get() != null) return address.get(); else - throw new RuntimeNoReachableHostException("No valid address among " + addresses); + throw new NoRouteToHostException("No valid address among " + addresses); + } + + /** + * Performs a DNS lookup for addresses. + * + * If the DNS lookup fails, than address remain unmodified. + */ + public void recreateSocketAddresses() { + Set temp = Collections.newSetFromMap(new ConcurrentHashMap<>()); + temp.addAll(getInetSocketAddressStream().map(this::recreateSocketAddress).collect(Collectors.toSet())); + addresses = temp; + } + + public InetSocketAddress getOne() { + return addresses.iterator().next(); } private void checkIfAddressIsReachableAndSet(InetSocketAddress address, - AtomicReference reachableAddress) { + AtomicReference reachableAddress) { for(int i = 0; i < 5 && reachableAddress.get() == null; i++) { try { if(address.getAddress().isReachable((i + 1) * timeout)) { @@ -136,17 +152,6 @@ private void checkIfAddressIsReachableAndSet(InetSocketAddress address, } } - /** - * Performs a DNS lookup for addresses. - * - * If the DNS lookup fails, than address remain unmodified. - */ - public void recreateSocketAddresses() { - Set temp = Collections.newSetFromMap(new ConcurrentHashMap<>()); - temp.addAll(getInetSocketAddressStream().map(this::recreateSocketAddress).collect(Collectors.toSet())); - addresses = temp; - } - private InetSocketAddress recreateSocketAddress(InetSocketAddress address) { try { return new InetSocketAddress(InetAddress.getByName(address.getHostString()), address.getPort()); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverMaster.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverMaster.java index 556da7f6fd8..a4f92239be1 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverMaster.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverMaster.java @@ -26,9 +26,7 @@ import java.io.ByteArrayInputStream; import java.io.DataInputStream; import java.io.IOException; -import java.net.ServerSocket; -import java.net.Socket; -import java.net.SocketAddress; +import java.net.*; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -435,23 +433,24 @@ synchronized public void start() throws IOException { } listenerRunning = true; int backlog = 10; // dog science + InetAddress address; + try { + address = self.getQuorumAddress().getValidAddress().getAddress(); + } catch (NoRouteToHostException e) { + address = self.getQuorumAddress().getOne().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().getValidAddress().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().getValidAddress().getAddress()); + ss = new ServerSocket(port, backlog, address); } } thread = new Thread(this, "ObserverMaster"); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java index 24a36955ace..5c645eb3d7e 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java @@ -19,12 +19,7 @@ package org.apache.zookeeper.server.quorum; import java.io.*; -import java.net.BindException; -import java.net.InetSocketAddress; -import java.net.ServerSocket; -import java.net.Socket; -import java.net.SocketException; -import java.net.SocketTimeoutException; +import java.net.*; import java.nio.BufferUnderflowException; import java.nio.ByteBuffer; import java.util.*; @@ -42,7 +37,6 @@ import org.apache.zookeeper.server.ZooKeeperThread; import org.apache.zookeeper.server.quorum.auth.QuorumAuthLearner; import org.apache.zookeeper.server.quorum.auth.QuorumAuthServer; -import org.apache.zookeeper.server.quorum.exception.RuntimeNoReachableHostException; import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -321,7 +315,13 @@ public void testInitiateConnection(long sid) throws Exception { LOG.debug("Opening channel to server " + sid); Socket sock = new Socket(); setSockOpts(sock); - sock.connect(self.getVotingView().get(sid).electionAddr.getValidAddress(), cnxTO); + InetSocketAddress address; + try { + address = self.getVotingView().get(sid).electionAddr.getValidAddress(); + } catch (NoRouteToHostException e) { + address = self.getVotingView().get(sid).electionAddr.getOne(); + } + sock.connect(address, cnxTO); initiateConnection(sock, sid); } @@ -404,7 +404,13 @@ private boolean startConnection(Socket sock, Long sid) // represents protocol version (in other words - message type) dout.writeLong(PROTOCOL_VERSION); dout.writeLong(self.getId()); - String addr = formatInetAddr(self.getElectionAddress().getValidAddress()); + InetSocketAddress address; + try { + address = self.getElectionAddress().getValidAddress(); + } catch (NoRouteToHostException e) { + address = self.getElectionAddress().getOne(); + } + String addr = formatInetAddr(address); byte[] addr_bytes = addr.getBytes(); dout.writeInt(addr_bytes.length); dout.write(addr_bytes); @@ -636,20 +642,26 @@ synchronized private boolean connectOne(long sid, MultipleAddresses electionAddr Socket sock = null; try { LOG.debug("Opening channel to server " + sid); - InetSocketAddress address = electionAddr.getValidAddress(); + InetSocketAddress address; + try { + address = electionAddr.getValidAddress(); + } catch (NoRouteToHostException e) { + address = electionAddr.getOne(); + } + if (self.isSslQuorum()) { - SSLSocket sslSock = self.getX509Util().createSSLSocket(); - setSockOpts(sslSock); - sslSock.connect(address, cnxTO); - sslSock.startHandshake(); - sock = sslSock; - } else { - sock = new Socket(); - setSockOpts(sock); - sock.connect(address, cnxTO); + SSLSocket sslSock = self.getX509Util().createSSLSocket(); + setSockOpts(sslSock); + sslSock.connect(address, cnxTO); + sslSock.startHandshake(); + sock = sslSock; + } else { + sock = new Socket(); + setSockOpts(sock); + sock.connect(address, cnxTO); - } - LOG.debug("Connected to server " + sid); + } + LOG.debug("Connected to server " + sid); // Sends connection request asynchronously if the quorum // sasl authentication is enabled. This is required because // sasl server authentication process may take few seconds to @@ -660,15 +672,6 @@ synchronized private boolean connectOne(long sid, MultipleAddresses electionAddr initiateConnection(sock, sid); } return true; - } catch (RuntimeNoReachableHostException e) { - // Sun doesn't include the address that causes this - // exception to be thrown, also UAE cannot be wrapped cleanly - // so we log the exception in order to capture this critical - // detail. - LOG.warn("Cannot open channel to " + sid - + " at election address " + electionAddr, e); - closeSocket(sock); - throw e; } catch (X509Exception e) { LOG.warn("Cannot open secure channel to " + sid + " at election address " + electionAddr, e); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java index 120f27859a6..82bb3b573bb 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -25,10 +25,7 @@ import java.io.StringReader; import java.io.StringWriter; import java.io.Writer; -import java.net.DatagramPacket; -import java.net.DatagramSocket; -import java.net.InetAddress; -import java.net.InetSocketAddress; +import java.net.*; import java.nio.ByteBuffer; import java.util.*; import java.util.Map.Entry; @@ -2022,7 +2019,13 @@ private void updateObserverMasterList() { observerMasters.clear(); StringBuilder sb = new StringBuilder(); for (QuorumServer server : quorumVerifier.getVotingMembers().values()) { - InetSocketAddress addr = new InetSocketAddress(server.addr.getValidAddress().getAddress(), observerMasterPort); + InetAddress address; + try { + address = server.addr.getValidAddress().getAddress(); + } catch (NoRouteToHostException e) { + address = server.addr.getOne().getAddress(); + } + InetSocketAddress addr = new InetSocketAddress(address, observerMasterPort); observerMasters.add(new QuorumServer(server.id, addr)); sb.append(addr).append(","); } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeNoReachableHostException.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeNoReachableHostException.java deleted file mode 100644 index 2a542494867..00000000000 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/exception/RuntimeNoReachableHostException.java +++ /dev/null @@ -1,33 +0,0 @@ -/** - * 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.exception; - -public class RuntimeNoReachableHostException extends RuntimeException { - - private static final long serialVersionUID = -8118892361652577058L; - - public RuntimeNoReachableHostException() { - super(); - } - - public RuntimeNoReachableHostException(String message) { - super(message); - } - -} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java index 46bbffdce21..332419897d9 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java @@ -20,12 +20,12 @@ import org.apache.commons.collections.CollectionUtils; import org.apache.zookeeper.PortAssignment; -import org.apache.zookeeper.server.quorum.exception.RuntimeNoReachableHostException; import org.junit.Assert; import org.junit.Test; import java.net.InetAddress; import java.net.InetSocketAddress; +import java.net.NoRouteToHostException; import java.net.UnknownHostException; import java.util.Arrays; import java.util.List; @@ -81,15 +81,15 @@ public void testGetWildcardAddresses() { } @Test - public void testGetValidAddress() { + public void testGetValidAddress() throws NoRouteToHostException { List addresses = getAddressList(); MultipleAddresses multipleAddresses = new MultipleAddresses(addresses); Assert.assertTrue(addresses.contains(multipleAddresses.getValidAddress())); } - @Test(expected = RuntimeNoReachableHostException.class) - public void testGetValidAddressWithNotValid() { + @Test(expected = NoRouteToHostException.class) + public void testGetValidAddressWithNotValid() throws NoRouteToHostException { MultipleAddresses multipleAddresses = new MultipleAddresses(new InetSocketAddress("10.0.0.1", 22)); multipleAddresses.getValidAddress(); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/CnxManagerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/CnxManagerTest.java index 1d3aa52f845..5162e5e241f 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/CnxManagerTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/CnxManagerTest.java @@ -24,19 +24,18 @@ import java.io.DataOutputStream; import java.io.File; import java.net.InetSocketAddress; +import java.net.NoRouteToHostException; import java.nio.ByteBuffer; import java.nio.channels.SocketChannel; import java.util.ArrayList; import java.util.Date; import java.util.HashMap; import java.util.Map; -import java.util.Random; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.net.Socket; import org.apache.zookeeper.common.Time; -import org.apache.zookeeper.server.quorum.exception.RuntimeNoReachableHostException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.zookeeper.PortAssignment; @@ -186,16 +185,15 @@ public void testCnxManager() throws Exception { Assert.assertFalse(cnxManager.listener.isAlive()); } - @Test(expected = RuntimeNoReachableHostException.class) + @Test public void testCnxManagerTimeout() throws Exception { - Random rand = new Random(); int address = ThreadLocalRandom.current().nextInt(1, 255); int deadPort = PortAssignment.unique(); String deadAddress = "10.1.1." + address; LOG.info("This is the dead address I'm trying: " + deadAddress); - peers.put(Long.valueOf(2), + peers.put(2L, new QuorumServer(2, new InetSocketAddress(deadAddress, deadPort), new InetSocketAddress(deadAddress, PortAssignment.unique()), @@ -215,7 +213,7 @@ public void testCnxManagerTimeout() throws Exception { cnxManager.toSend(2L, createMsg(ServerState.LOOKING.ordinal(), 1, -1, 1)); long end = Time.currentElapsedTime(); - if((end - begin) > 6000) Assert.fail("Waited more than necessary"); + if((end - begin) > 10000) Assert.fail("Waited more than necessary"); cnxManager.halt(); Assert.assertFalse(cnxManager.listener.isAlive()); } @@ -238,7 +236,12 @@ public void testCnxManagerSpinLock() throws Exception { } else { LOG.error("Null listener when initializing cnx manager"); } - InetSocketAddress address = peers.get(peer.getId()).electionAddr.getValidAddress(); + InetSocketAddress address; + try { + address = peers.get(peer.getId()).electionAddr.getValidAddress(); + } catch (NoRouteToHostException e) { + address = peers.get(peer.getId()).electionAddr.getOne(); + } LOG.info("Election port: " + address.getPort()); Thread.sleep(1000); @@ -246,7 +249,13 @@ public void testCnxManagerSpinLock() throws Exception { SocketChannel sc = SocketChannel.open(); sc.socket().connect(address, 5000); - InetSocketAddress otherAddr = peers.get(2L).electionAddr.getValidAddress(); + InetSocketAddress otherAddr; + try { + otherAddr = peers.get(2L).electionAddr.getValidAddress(); + } catch (NoRouteToHostException e) { + otherAddr = peers.get(2L).electionAddr.getOne(); + } + DataOutputStream dout = new DataOutputStream(sc.socket().getOutputStream()); dout.writeLong(QuorumCnxManager.PROTOCOL_VERSION); dout.writeLong(2L); @@ -302,7 +311,14 @@ public void testCnxManagerNPE() throws Exception { } else { LOG.error("Null listener when initializing cnx manager"); } - InetSocketAddress address = peers.get(peer.getId()).electionAddr.getValidAddress(); + + InetSocketAddress address; + try { + address = peers.get(peer.getId()).electionAddr.getValidAddress(); + } catch (NoRouteToHostException e) { + address = peers.get(peer.getId()).electionAddr.getOne(); + } + LOG.info("Election port: " + address.getPort()); Thread.sleep(1000); @@ -349,7 +365,14 @@ public void testSocketTimeout() throws Exception { } else { LOG.error("Null listener when initializing cnx manager"); } - InetSocketAddress address = peers.get(peer.getId()).electionAddr.getValidAddress(); + + InetSocketAddress address; + try { + address = peers.get(peer.getId()).electionAddr.getValidAddress(); + } catch (NoRouteToHostException e) { + address = peers.get(peer.getId()).electionAddr.getOne(); + } + LOG.info("Election port: " + address.getPort()); Thread.sleep(1000); From 1e5a2baa221153511e4d2bd0a39e0de1d82685bc Mon Sep 17 00:00:00 2001 From: ArtemChernatsky Date: Fri, 1 Mar 2019 18:12:56 +0200 Subject: [PATCH 13/16] fixed naming, formatting, logging --- .../apache/zookeeper/server/ObserverBean.java | 9 +- .../server/quorum/AuthFastLeaderElection.java | 20 ++-- .../zookeeper/server/quorum/Leader.java | 44 ++++++--- .../zookeeper/server/quorum/Learner.java | 29 +++--- .../server/quorum/MultipleAddresses.java | 76 ++++++++++----- .../server/quorum/ObserverMaster.java | 12 +-- .../server/quorum/QuorumCnxManager.java | 94 ++++++++++--------- .../zookeeper/server/quorum/QuorumPeer.java | 27 ++++-- .../server/quorum/MultipleAddressesTest.java | 17 ++-- .../apache/zookeeper/test/CnxManagerTest.java | 31 +----- .../apache/zookeeper/test/ReconfigTest.java | 8 +- 11 files changed, 198 insertions(+), 169 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ObserverBean.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ObserverBean.java index e8975238ce5..e648f155393 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ObserverBean.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ObserverBean.java @@ -21,9 +21,7 @@ import org.apache.zookeeper.server.quorum.Observer; import org.apache.zookeeper.server.quorum.ObserverMXBean; import org.apache.zookeeper.server.quorum.QuorumPeer; - import java.net.InetSocketAddress; -import java.net.NoRouteToHostException; /** * ObserverBean @@ -55,12 +53,7 @@ public String getLearnerMaster() { if (learnerMaster == null || learnerMaster.addr == null) { return "Unknown"; } - InetSocketAddress address; - try { - address = learnerMaster.addr.getValidAddress(); - } catch (NoRouteToHostException e) { - address = learnerMaster.addr.getOne(); - } + InetSocketAddress address = learnerMaster.addr.getReachableOrOne(); return address.getAddress().getHostAddress() + ":" + address.getPort(); } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java index 4a8e5dad823..e6ce950ebdd 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java @@ -19,7 +19,11 @@ package org.apache.zookeeper.server.quorum; import java.io.IOException; -import java.net.*; +import java.net.DatagramPacket; +import java.net.DatagramSocket; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.SocketException; import java.nio.ByteBuffer; import java.util.Collection; import java.util.Collections; @@ -724,12 +728,7 @@ private void process(ToSend m) { } for (QuorumServer server : self.getVotingView().values()) { - InetAddress address; - try { - address = server.addr.getValidAddress().getAddress(); - } catch (NoRouteToHostException e) { - address = server.addr.getOne().getAddress(); - } + InetAddress address = server.addr.getReachableOrOne().getAddress(); InetSocketAddress saddr = new InetSocketAddress(address, port); addrChallengeMap.put(saddr, new ConcurrentHashMap()); } @@ -783,12 +782,7 @@ private void leaveInstance() { private void sendNotifications() { for (QuorumServer server : self.getView().values()) { - InetSocketAddress address; - try { - address = self.getView().get(server.id).electionAddr.getValidAddress(); - } catch (NoRouteToHostException e) { - address = self.getView().get(server.id).electionAddr.getOne(); - } + InetSocketAddress address = self.getView().get(server.id).electionAddr.getReachableOrOne(); ToSend notmsg = new ToSend(ToSend.mType.notification, AuthFastLeaderElection.sequencer++, proposedLeader, proposedZxid, logicalclock.get(), QuorumPeer.ServerState.LOOKING, address); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java index 92a67bee870..c60352c15f3 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java @@ -24,10 +24,29 @@ import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; -import java.net.*; +import java.net.BindException; +import java.net.InetSocketAddress; +import java.net.ServerSocket; +import java.net.Socket; +import java.net.SocketAddress; +import java.net.SocketException; import java.nio.ByteBuffer; -import java.util.*; -import java.util.concurrent.*; +import java.util.ArrayList; +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.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.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; @@ -240,14 +259,15 @@ public boolean isQuorumSynced(QuorumVerifier qv) { this.self = self; this.proposalStats = new BufferStats(); - List addresses; - if (self.getQuorumListenOnAllIPs()) + Set addresses; + if (self.getQuorumListenOnAllIPs()) { addresses = self.getQuorumAddress().getWildcardAddresses(); - else + } else { addresses = self.getQuorumAddress().getAllAddresses(); + } serverSockets = new LinkedList<>(); - for(InetSocketAddress address : addresses) + for (InetSocketAddress address : addresses) serverSockets.add(createServerSocket(address, self.shouldUsePortUnification(), self.isSslQuorum())); this.zk = zk; @@ -264,7 +284,6 @@ ServerSocket createServerSocket(InetSocketAddress address, boolean portUnificati } else { serverSocket = new ServerSocket(); } - serverSocket.setReuseAddress(true); serverSocket.bind(address); return serverSocket; @@ -440,7 +459,7 @@ public void run() { acceptConnections(); } } catch (Exception e) { - LOG.warn("Exception while accepting follower", e.getMessage()); + LOG.warn("Exception while accepting follower", e); if (!fail.get()) { handleException(getName(), e); fail.set(true); @@ -468,7 +487,7 @@ private void acceptConnections() throws IOException { } catch (SocketException e) { error = true; if (stop.get()) { - LOG.info("exception while shutting down acceptor: " + e); + LOG.info("Exception while shutting down acceptor", e); } else { throw e; } @@ -775,12 +794,13 @@ void shutdown(String reason) { synchronized void closeSockets() { if (serverSockets != null) for (ServerSocket serverSocket : serverSockets) { - if (!serverSocket.isClosed()) + if (!serverSocket.isClosed()) { try { serverSocket.close(); } catch (IOException e) { - LOG.warn("Ignoring unexpected exception during close" + serverSocket, e); + LOG.warn("Ignoring unexpected exception during close {}", serverSocket, e); } + } } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java index 560ea75b8f1..ead4695b6b8 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java @@ -28,9 +28,9 @@ import java.net.Socket; import java.nio.ByteBuffer; import java.util.LinkedList; -import java.util.List; 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; @@ -258,7 +258,7 @@ protected void sockConnect(Socket sock, InetSocketAddress addr, int timeout) protected void connectToLeader(MultipleAddresses addr, String hostname) throws IOException, InterruptedException { - List addresses = addr.getAllAddresses(); + Set addresses = addr.getAllAddresses(); ExecutorService executor = Executors.newFixedThreadPool(addresses.size()); CountDownLatch latch = new CountDownLatch(addresses.size()); AtomicReference socket = new AtomicReference<>(null); @@ -266,10 +266,11 @@ protected void connectToLeader(MultipleAddresses addr, String hostname) latch.await(); - if(socket.get() == null) + if (socket.get() == null) { throw new IOException("Failed connect to " + addr); - else + } else { sock = socket.get(); + } self.authLearner.authenticate(sock, hostname); @@ -297,11 +298,13 @@ public void run() { Thread.currentThread().setName("LeaderConnector-" + address); Socket sock = connectToLeader(); - if(sock != null && sock.isConnected() && !socket.compareAndSet(null, sock)) + if (sock != null && sock.isConnected() && !socket.compareAndSet(null, sock)) { + LOG.info("Connection to the leader is already established, close the redundant connection"); sock.close(); + } } catch (Exception e) { - LOG.error("Failed to connect to " + address, e); + LOG.error("Failed connect to {}", address, e); } finally { latch.countDown(); } @@ -332,19 +335,15 @@ private Socket connectToLeader() throws IOException, X509Exception, InterruptedE remainingInitLimitTime = initLimitTime - (int) ((nanoTime() - startNanoTime) / 1000000); if (remainingInitLimitTime <= leaderConnectDelayDuringRetryMs) { - LOG.error("Unexpected exception, initLimit exceeded. tries=" + tries + - ", remaining init limit=" + remainingInitLimitTime + - ", connecting to " + address, e); + LOG.error("Unexpected exception, initLimit exceeded. tries={}, remaining init limit={}, " + + "connecting to {}", tries, remainingInitLimitTime, address, e); throw e; } else if (tries >= 4) { - LOG.error("Unexpected exception, retries exceeded. tries=" + tries + - ", remaining init limit=" + remainingInitLimitTime + - ", connecting to " + address, e); + LOG.error("Unexpected exception, retries exceeded. tries={}, remaining init limit={}, " + + "connecting to {}", tries, remainingInitLimitTime, address, e); throw e; } else { - LOG.warn("Unexpected exception, tries=" + tries + - ", remaining init limit=" + remainingInitLimitTime + - ", connecting to " + address, e); + LOG.warn("Unexpected exception, tries={}, remaining init limit={}, connecting to {}", tries, remainingInitLimitTime, address, e); sock = createSocket(); } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java index 85720137873..a26b28d0165 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java @@ -6,9 +6,9 @@ * 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 - * + *

    + * 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. @@ -23,7 +23,10 @@ import java.net.InetSocketAddress; import java.net.NoRouteToHostException; import java.net.UnknownHostException; -import java.util.*; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; @@ -35,21 +38,22 @@ * See ZOOKEEPER-3188 for a discussion of this feature. */ public class MultipleAddresses { + private final static int DEFAULT_TIMEOUT = 100; private Set addresses; private int timeout; public MultipleAddresses() { addresses = Collections.newSetFromMap(new ConcurrentHashMap<>()); - timeout = 100; + timeout = DEFAULT_TIMEOUT; } public MultipleAddresses(List addresses) { - this(addresses, 100); + this(addresses, DEFAULT_TIMEOUT); } public MultipleAddresses(InetSocketAddress address) { - this(address, 100); + this(address, DEFAULT_TIMEOUT); } public MultipleAddresses(List addresses, int timeout) { @@ -58,7 +62,7 @@ public MultipleAddresses(List addresses, int timeout) { this.timeout = timeout; } - public MultipleAddresses(InetSocketAddress address, int timeout) { + public MultipleAddresses(InetSocketAddress address, int timeout) { addresses = Collections.newSetFromMap(new ConcurrentHashMap<>()); addresses.add(address); this.timeout = timeout; @@ -79,19 +83,19 @@ public boolean isEmpty() { /** * Returns all addresses. * - * @return list of all InetSocketAddress + * @return set of all InetSocketAddress */ - public List getAllAddresses() { - return new LinkedList<>(addresses); + public Set getAllAddresses() { + return Collections.unmodifiableSet(addresses); } /** * Returns wildcard addresses for all ports * - * @return list of InetSocketAddress with wildcards for all ports + * @return set of InetSocketAddress with wildcards for all ports */ - public List getWildcardAddresses() { - return addresses.stream().map(a -> new InetSocketAddress(a.getPort())).distinct().collect(Collectors.toList()); + public Set getWildcardAddresses() { + return addresses.stream().map(a -> new InetSocketAddress(a.getPort())).collect(Collectors.toSet()); } /** @@ -113,14 +117,30 @@ public void addAddress(InetSocketAddress address) { * @return address which is reachable. * @throws NoRouteToHostException if none address is reachable */ - public InetSocketAddress getValidAddress() throws NoRouteToHostException { + public InetSocketAddress getReachableAddress() throws NoRouteToHostException { AtomicReference address = new AtomicReference<>(null); getInetSocketAddressStream().forEach(addr -> checkIfAddressIsReachableAndSet(addr, address)); - if(address.get() != null) + if (address.get() != null) { return address.get(); - else + } else { throw new NoRouteToHostException("No valid address among " + addresses); + } + } + + /** + * Returns reachable address or first one, if none is reachable. + * + * @return address which is reachable or fist one. + */ + public InetSocketAddress getReachableOrOne() { + InetSocketAddress address; + try { + address = getReachableAddress(); + } catch (NoRouteToHostException e) { + address = getOne(); + } + return address; } /** @@ -134,15 +154,20 @@ public void recreateSocketAddresses() { addresses = temp; } + /** + * Returns first address from set. + * + * @return address from a set. + */ public InetSocketAddress getOne() { return addresses.iterator().next(); } private void checkIfAddressIsReachableAndSet(InetSocketAddress address, AtomicReference reachableAddress) { - for(int i = 0; i < 5 && reachableAddress.get() == null; i++) { + for (int i = 0; i < 5 && reachableAddress.get() == null; i++) { try { - if(address.getAddress().isReachable((i + 1) * timeout)) { + if (address.getAddress().isReachable((i + 1) * timeout)) { reachableAddress.compareAndSet(null, address); break; } @@ -161,16 +186,21 @@ private InetSocketAddress recreateSocketAddress(InetSocketAddress address) { } private Stream getInetSocketAddressStream() { - if(addresses.size() > 1) + if (addresses.size() > 1) { return addresses.parallelStream(); - else + } else { return addresses.stream(); + } } @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } else if (o == null || getClass() != o.getClass()) { + return false; + } + MultipleAddresses that = (MultipleAddresses) o; return Objects.equals(addresses, that.addresses); } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverMaster.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverMaster.java index a4f92239be1..aa610b58b0a 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverMaster.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverMaster.java @@ -26,7 +26,10 @@ import java.io.ByteArrayInputStream; import java.io.DataInputStream; import java.io.IOException; -import java.net.*; +import java.net.InetAddress; +import java.net.ServerSocket; +import java.net.Socket; +import java.net.SocketAddress; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -433,12 +436,7 @@ synchronized public void start() throws IOException { } listenerRunning = true; int backlog = 10; // dog science - InetAddress address; - try { - address = self.getQuorumAddress().getValidAddress().getAddress(); - } catch (NoRouteToHostException e) { - address = self.getQuorumAddress().getOne().getAddress(); - } + InetAddress address = self.getQuorumAddress().getReachableOrOne().getAddress(); if (self.shouldUsePortUnification() || self.isSslQuorum()) { boolean allowInsecureConnection = self.shouldUsePortUnification(); if (self.getQuorumListenOnAllIPs()) { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java index 5c645eb3d7e..a2b739fb27f 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java @@ -18,12 +18,36 @@ package org.apache.zookeeper.server.quorum; -import java.io.*; -import java.net.*; +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.BindException; +import java.net.InetSocketAddress; +import java.net.ServerSocket; +import java.net.Socket; +import java.net.SocketException; +import java.net.SocketTimeoutException; import java.nio.BufferUnderflowException; import java.nio.ByteBuffer; -import java.util.*; -import java.util.concurrent.*; +import java.util.Collections; +import java.util.Enumeration; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.concurrent.ArrayBlockingQueue; +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; @@ -315,12 +339,7 @@ public void testInitiateConnection(long sid) throws Exception { LOG.debug("Opening channel to server " + sid); Socket sock = new Socket(); setSockOpts(sock); - InetSocketAddress address; - try { - address = self.getVotingView().get(sid).electionAddr.getValidAddress(); - } catch (NoRouteToHostException e) { - address = self.getVotingView().get(sid).electionAddr.getOne(); - } + InetSocketAddress address = self.getVotingView().get(sid).electionAddr.getReachableOrOne(); sock.connect(address, cnxTO); initiateConnection(sock, sid); } @@ -404,12 +423,7 @@ private boolean startConnection(Socket sock, Long sid) // represents protocol version (in other words - message type) dout.writeLong(PROTOCOL_VERSION); dout.writeLong(self.getId()); - InetSocketAddress address; - try { - address = self.getElectionAddress().getValidAddress(); - } catch (NoRouteToHostException e) { - address = self.getElectionAddress().getOne(); - } + InetSocketAddress address = self.getElectionAddress().getReachableOrOne(); String addr = formatInetAddr(address); byte[] addr_bytes = addr.getBytes(); dout.writeInt(addr_bytes.length); @@ -642,12 +656,7 @@ synchronized private boolean connectOne(long sid, MultipleAddresses electionAddr Socket sock = null; try { LOG.debug("Opening channel to server " + sid); - InetSocketAddress address; - try { - address = electionAddr.getValidAddress(); - } catch (NoRouteToHostException e) { - address = electionAddr.getOne(); - } + InetSocketAddress address = electionAddr.getReachableOrOne(); if (self.isSslQuorum()) { SSLSocket sslSock = self.getX509Util().createSSLSocket(); @@ -857,12 +866,13 @@ public Listener() { @Override public void run() { if(!shutdown) { - List addresses; + Set addresses; - if (self.getQuorumListenOnAllIPs()) + if (self.getQuorumListenOnAllIPs()) { addresses = self.getElectionAddress().getWildcardAddresses(); - else + } else { addresses = self.getElectionAddress().getAllAddresses(); + } CountDownLatch latch = new CountDownLatch(addresses.size()); listenerHandlers = addresses.stream().map(address -> @@ -875,17 +885,17 @@ public void run() { try { latch.await(); } catch (InterruptedException ie) { - LOG.error("Interrupted while sleeping. " + - "Ignoring exception", ie); + LOG.error("Interrupted while sleeping. Ignoring exception", ie); } finally { // Clean up for shutdown. - for (ListenerHandler handler : listenerHandlers) + for (ListenerHandler handler : listenerHandlers) { try { handler.close(); } catch (IOException ie) { // Don't log an error for shutdown. LOG.debug("Error closing server socket", ie); } + } } } @@ -893,9 +903,9 @@ public void run() { if (!shutdown) { LOG.error("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(","))); + + "election any longer: {}" + , self.getElectionAddress().getAllAddresses().stream().map(NetUtils::formatInetAddr) + .collect(Collectors.joining(","))); if (bindException.get()) { // After leaving listener thread, the host cannot join the // quorum anymore, this is a severe error that we cannot @@ -911,14 +921,14 @@ public void run() { void halt() { LOG.debug("Trying to close listeners"); if (listenerHandlers != null) { - LOG.debug("Closing listener: " - + QuorumCnxManager.this.mySid); - for (ListenerHandler handler : listenerHandlers) + 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); + LOG.warn("Exception when shutting down listener: ", e); } + } } } @@ -950,11 +960,11 @@ public void run() { try { close(); } catch (IOException e) { - LOG.warn("Exception when shutting down listener: " + e); + LOG.warn("Exception when shutting down listener: ", e); } } catch (Exception e) { // Output of unexpected exception, should never happen - LOG.error("Unexpected error " + e); + LOG.error("Unexpected error ", e); } finally { latch.countDown(); } @@ -963,7 +973,7 @@ public void run() { @Override public synchronized void close() throws IOException { if (serverSocket != null && !serverSocket.isClosed()) { - LOG.debug("Trying to close listeners: " + serverSocket); + LOG.debug("Trying to close listeners: {}", serverSocket); serverSocket.close(); } } @@ -978,13 +988,12 @@ private void acceptConnections() { while ((!shutdown) && (numRetries < ATTEMPTS_AMOUNT)) { try { serverSocket = createNewServerSocket(); - LOG.info("My election bind port: " + address.toString()); + LOG.info("My election bind port: {}", address.toString()); while (!shutdown) { try { client = serverSocket.accept(); setSockOpts(client); - LOG.info("Received connection request " - + client.getRemoteSocketAddress()); + 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 @@ -1019,8 +1028,7 @@ private void acceptConnections() { } catch (IOException ie) { LOG.error("Error closing server socket", ie); } catch (InterruptedException ie) { - LOG.error("Interrupted while sleeping. " + - "Ignoring exception", ie); + LOG.error("Interrupted while sleeping. Ignoring exception", ie); } closeSocket(client); } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java index 82bb3b573bb..77c26f47681 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -25,10 +25,22 @@ import java.io.StringReader; import java.io.StringWriter; import java.io.Writer; -import java.net.*; +import java.net.DatagramPacket; +import java.net.DatagramSocket; +import java.net.InetAddress; +import java.net.InetSocketAddress; import java.nio.ByteBuffer; -import java.util.*; +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; +import java.util.Properties; +import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -327,8 +339,8 @@ private static String delimitedHostString(InetSocketAddress addr) public String toString(){ StringWriter sw = new StringWriter(); - List addrList = addr.getAllAddresses(); - List electionAddrList = electionAddr.getAllAddresses(); + List addrList = new LinkedList<>(addr.getAllAddresses()); + List electionAddrList = new LinkedList<>(electionAddr.getAllAddresses()); if(addrList.size() > 0 && electionAddrList.size() > 0) { addrList.sort(Comparator.comparing(InetSocketAddress::getHostString)); @@ -2019,12 +2031,7 @@ private void updateObserverMasterList() { observerMasters.clear(); StringBuilder sb = new StringBuilder(); for (QuorumServer server : quorumVerifier.getVotingMembers().values()) { - InetAddress address; - try { - address = server.addr.getValidAddress().getAddress(); - } catch (NoRouteToHostException e) { - address = server.addr.getOne().getAddress(); - } + InetAddress address = server.addr.getReachableOrOne().getAddress(); InetSocketAddress addr = new InetSocketAddress(address, observerMasterPort); observerMasters.add(new QuorumServer(server.id, addr)); sb.append(addr).append(","); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java index 332419897d9..b1ea4d0bf0e 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java @@ -6,9 +6,9 @@ * 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 - * + *

    + * 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. @@ -27,6 +27,7 @@ import java.net.InetSocketAddress; import java.net.NoRouteToHostException; import java.net.UnknownHostException; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; @@ -85,13 +86,13 @@ public void testGetValidAddress() throws NoRouteToHostException { List addresses = getAddressList(); MultipleAddresses multipleAddresses = new MultipleAddresses(addresses); - Assert.assertTrue(addresses.contains(multipleAddresses.getValidAddress())); + 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.getValidAddress(); + multipleAddresses.getReachableAddress(); } @Test @@ -100,14 +101,14 @@ public void testRecreateSocketAddresses() throws UnknownHostException { .map(addr -> new InetSocketAddress(addr, 222)).collect(Collectors.toList()); MultipleAddresses multipleAddresses = new MultipleAddresses(searchedAddresses.get(searchedAddresses.size() - 1)); - List addresses = multipleAddresses.getAllAddresses(); + List addresses = new ArrayList<>(multipleAddresses.getAllAddresses()); Assert.assertEquals(1, addresses.size()); Assert.assertEquals(searchedAddresses.get(searchedAddresses.size() - 1), addresses.get(0)); multipleAddresses.recreateSocketAddresses(); - addresses = multipleAddresses.getAllAddresses(); + addresses = new ArrayList<>(multipleAddresses.getAllAddresses()); Assert.assertEquals(1, addresses.size()); Assert.assertEquals(searchedAddresses.get(0), addresses.get(0)); } @@ -118,7 +119,7 @@ public void testRecreateSocketAddressesWithWrongAddresses() { MultipleAddresses multipleAddresses = new MultipleAddresses(address); multipleAddresses.recreateSocketAddresses(); - Assert.assertEquals(address, multipleAddresses.getAllAddresses().get(0)); + Assert.assertEquals(address, multipleAddresses.getOne()); } @Test diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/CnxManagerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/CnxManagerTest.java index 5162e5e241f..396a8df19e4 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/CnxManagerTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/CnxManagerTest.java @@ -24,7 +24,6 @@ import java.io.DataOutputStream; import java.io.File; import java.net.InetSocketAddress; -import java.net.NoRouteToHostException; import java.nio.ByteBuffer; import java.nio.channels.SocketChannel; import java.util.ArrayList; @@ -236,12 +235,7 @@ public void testCnxManagerSpinLock() throws Exception { } else { LOG.error("Null listener when initializing cnx manager"); } - InetSocketAddress address; - try { - address = peers.get(peer.getId()).electionAddr.getValidAddress(); - } catch (NoRouteToHostException e) { - address = peers.get(peer.getId()).electionAddr.getOne(); - } + InetSocketAddress address = peers.get(peer.getId()).electionAddr.getReachableOrOne(); LOG.info("Election port: " + address.getPort()); Thread.sleep(1000); @@ -249,12 +243,7 @@ public void testCnxManagerSpinLock() throws Exception { SocketChannel sc = SocketChannel.open(); sc.socket().connect(address, 5000); - InetSocketAddress otherAddr; - try { - otherAddr = peers.get(2L).electionAddr.getValidAddress(); - } catch (NoRouteToHostException e) { - otherAddr = peers.get(2L).electionAddr.getOne(); - } + InetSocketAddress otherAddr = peers.get(2L).electionAddr.getReachableOrOne(); DataOutputStream dout = new DataOutputStream(sc.socket().getOutputStream()); dout.writeLong(QuorumCnxManager.PROTOCOL_VERSION); @@ -312,13 +301,7 @@ public void testCnxManagerNPE() throws Exception { LOG.error("Null listener when initializing cnx manager"); } - InetSocketAddress address; - try { - address = peers.get(peer.getId()).electionAddr.getValidAddress(); - } catch (NoRouteToHostException e) { - address = peers.get(peer.getId()).electionAddr.getOne(); - } - + InetSocketAddress address = peers.get(peer.getId()).electionAddr.getReachableOrOne(); LOG.info("Election port: " + address.getPort()); Thread.sleep(1000); @@ -366,13 +349,7 @@ public void testSocketTimeout() throws Exception { LOG.error("Null listener when initializing cnx manager"); } - InetSocketAddress address; - try { - address = peers.get(peer.getId()).electionAddr.getValidAddress(); - } catch (NoRouteToHostException e) { - address = peers.get(peer.getId()).electionAddr.getOne(); - } - + InetSocketAddress address = peers.get(peer.getId()).electionAddr.getReachableOrOne(); LOG.info("Election port: " + address.getPort()); Thread.sleep(1000); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigTest.java index ea0fa403b54..03c14e4ab09 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ReconfigTest.java @@ -1056,7 +1056,8 @@ private void assertLocalPeerMXBeanAttributes(QuorumPeer qp, qp.getClientAddress().getHostString() + ":" + qp.getClientAddress().getPort(), JMXEnv.ensureBeanAttribute(beanName, "ClientAddress")); Assert.assertEquals("Mismatches LearnerType!", - qp.getElectionAddress().getAllAddresses().get(0).getHostString() + ":" + qp.getElectionAddress().getAllAddresses().get(0).getPort(), + qp.getElectionAddress().getOne().getHostString() + ":" + + qp.getElectionAddress().getOne().getPort(), JMXEnv.ensureBeanAttribute(beanName, "ElectionAddress")); Assert.assertEquals("Mismatches PartOfEnsemble!", isPartOfEnsemble, JMXEnv.ensureBeanAttribute(beanName, "PartOfEnsemble")); @@ -1094,10 +1095,11 @@ private void assertRemotePeerMXBeanAttributes(QuorumServer qs, getNumericalAddrPort(qs.clientAddr.getHostString() + ":" + qs.clientAddr.getPort()), getAddrPortFromBean(beanName, "ClientAddress") ); Assert.assertEquals("Mismatches ElectionAddress!", - getNumericalAddrPort(qs.electionAddr.getAllAddresses().get(0).getHostString() + ":" + qs.electionAddr.getAllAddresses().get(0).getPort()), + getNumericalAddrPort(qs.electionAddr.getOne().getHostString() + ":" + + qs.electionAddr.getOne().getPort()), getAddrPortFromBean(beanName, "ElectionAddress") ); Assert.assertEquals("Mismatches QuorumAddress!", - getNumericalAddrPort(qs.addr.getAllAddresses().get(0).getHostString() + ":" + qs.addr.getAllAddresses().get(0).getPort()), + getNumericalAddrPort(qs.addr.getOne().getHostString() + ":" + qs.addr.getOne().getPort()), getAddrPortFromBean(beanName, "QuorumAddress") ); } } From b8dff5537d498abe58f470f4efcba496f28db735 Mon Sep 17 00:00:00 2001 From: ArtemChernatsky Date: Fri, 1 Mar 2019 18:30:29 +0200 Subject: [PATCH 14/16] fixed formatting --- .../main/java/org/apache/zookeeper/server/quorum/Learner.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java index ead4695b6b8..23c7e4d3d23 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java @@ -343,7 +343,8 @@ private Socket connectToLeader() throws IOException, X509Exception, InterruptedE "connecting to {}", tries, remainingInitLimitTime, address, e); throw e; } else { - LOG.warn("Unexpected exception, tries={}, remaining init limit={}, connecting to {}", tries, remainingInitLimitTime, address, e); + LOG.warn("Unexpected exception, tries={}, remaining init limit={}, connecting to {}", tries, + remainingInitLimitTime, address, e); sock = createSocket(); } } From f2284c8b2fc197c7d2edd3f6fe209b3020647409 Mon Sep 17 00:00:00 2001 From: dhavrilovych Date: Thu, 28 Mar 2019 06:45:59 -0700 Subject: [PATCH 15/16] fixed atomicity --- .../main/java/org/apache/zookeeper/server/quorum/Leader.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java index c60352c15f3..4ca089e851b 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java @@ -460,9 +460,8 @@ public void run() { } } catch (Exception e) { LOG.warn("Exception while accepting follower", e); - if (!fail.get()) { + if (fail.compareAndSet(false, true)) { handleException(getName(), e); - fail.set(true); halt(); } } finally { From 2486aee47e87ae0482ed69a3b4235d7a0d947170 Mon Sep 17 00:00:00 2001 From: dmitriyHavrilovich Date: Mon, 1 Apr 2019 15:01:20 +0300 Subject: [PATCH 16/16] fixed import --- .../org/apache/zookeeper/server/quorum/QuorumCnxManager.java | 1 + 1 file changed, 1 insertion(+) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java index 6728f81c54e..42c6540f860 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java @@ -32,6 +32,7 @@ import java.net.SocketTimeoutException; import java.nio.BufferUnderflowException; import java.nio.ByteBuffer; +import java.nio.channels.UnresolvedAddressException; import java.util.Collections; import java.util.Enumeration; import java.util.HashSet;