From 3741e446010f4d2f8be4de64134d479964f318bf Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Tue, 20 Nov 2018 05:33:09 -0800 Subject: [PATCH 01/63] [ZOOKEEPER-2778] QuorumPeer: encapsulate quorum/election/client addresses in an AddressTuple held through an AtomicReference --- .../zookeeper/server/quorum/QuorumPeer.java | 58 ++++++++----------- 1 file changed, 23 insertions(+), 35 deletions(-) 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 0d8a0126f33..7bc8ea6fac7 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 @@ -42,6 +42,7 @@ import java.util.Properties; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import javax.security.sasl.SaslException; @@ -121,6 +122,18 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider */ private ZKDatabase zkDb; + public static class AddressTuple { + public final InetSocketAddress quorumAddr; + public final InetSocketAddress electionAddr; + public final InetSocketAddress clientAddr; + + public AddressTuple(InetSocketAddress quorumAddr, InetSocketAddress electionAddr, InetSocketAddress clientAddr) { + this.quorumAddr = quorumAddr; + this.electionAddr = electionAddr; + this.clientAddr = clientAddr; + } + } + public static class QuorumServer { public InetSocketAddress addr = null; @@ -723,16 +736,14 @@ public synchronized ServerState getPeerState(){ DatagramSocket udpSocket; - private InetSocketAddress myQuorumAddr; - private InetSocketAddress myElectionAddr = null; - private InetSocketAddress myClientAddr = null; + private final AtomicReference myAddrs = new AtomicReference<>(); /** * Resolves hostname for a given server ID. * * This method resolves hostname for a given server ID in both quorumVerifer * and lastSeenQuorumVerifier. If the server ID matches the local server ID, - * it also updates myQuorumAddr and myElectionAddr. + * it also updates myAddrs. */ public void recreateSocketAddresses(long id) { QuorumVerifier qv = getQuorumVerifier(); @@ -741,8 +752,7 @@ public void recreateSocketAddresses(long id) { if (qs != null) { qs.recreateSocketAddresses(); if (id == getId()) { - setQuorumAddress(qs.addr); - setElectionAddress(qs.electionAddr); + setAddrs(qs.addr, qs.electionAddr, qs.clientAddr); } } } @@ -756,39 +766,19 @@ public void recreateSocketAddresses(long id) { } public InetSocketAddress getQuorumAddress(){ - synchronized (QV_LOCK) { - return myQuorumAddr; - } + return myAddrs.get().quorumAddr; } - public void setQuorumAddress(InetSocketAddress addr){ - synchronized (QV_LOCK) { - myQuorumAddr = addr; - } - } - public InetSocketAddress getElectionAddress(){ - synchronized (QV_LOCK) { - return myElectionAddr; - } + return myAddrs.get().electionAddr; } - public void setElectionAddress(InetSocketAddress addr){ - synchronized (QV_LOCK) { - myElectionAddr = addr; - } - } - public InetSocketAddress getClientAddress(){ - synchronized (QV_LOCK) { - return myClientAddr; - } + return myAddrs.get().clientAddr; } - public void setClientAddress(InetSocketAddress addr){ - synchronized (QV_LOCK) { - myClientAddr = addr; - } + public void setAddrs(InetSocketAddress quorumAddr, InetSocketAddress electionAddr, InetSocketAddress clientAddr){ + myAddrs.set(new AddressTuple(quorumAddr, electionAddr, clientAddr)); } private int electionType; @@ -953,7 +943,7 @@ synchronized public void startLeaderElection() { //} if (electionType == 0) { try { - udpSocket = new DatagramSocket(myQuorumAddr.getPort()); + udpSocket = new DatagramSocket(getQuorumAddress().getPort()); responder = new ResponderThread(); responder.start(); } catch (SocketException e) { @@ -1631,9 +1621,7 @@ public QuorumVerifier setQuorumVerifier(QuorumVerifier qv, boolean writeToDisk){ } QuorumServer qs = qv.getAllMembers().get(getId()); if (qs != null) { - setQuorumAddress(qs.addr); - setElectionAddress(qs.electionAddr); - setClientAddress(qs.clientAddr); + setAddrs(qs.addr, qs.electionAddr, qs.clientAddr); } return prevQV; } From 2214c819306797d71a20e496f55c5b1fa88a313c Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Wed, 21 Nov 2018 08:53:54 +0000 Subject: [PATCH 02/63] [ZOOKEEPER-2778] QuorumPeer/QuorumCnxManager: address deadlock and visibility issues --- .../server/quorum/QuorumCnxManager.java | 3 +- .../zookeeper/server/quorum/QuorumPeer.java | 123 +++++++++++------- 2 files changed, 81 insertions(+), 45 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 53a99062714..30fe76dd834 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 @@ -410,7 +410,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 = self.getElectionAddress().getHostString() + ":" + self.getElectionAddress().getPort(); + final InetSocketAddress electionAddr = self.getElectionAddress(); + String addr = electionAddr.getHostString() + ":" + electionAddr.getPort(); byte[] addr_bytes = addr.getBytes(); dout.writeInt(addr_bytes.length); dout.write(addr_bytes); 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 7bc8ea6fac7..c693a66004a 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 @@ -109,7 +109,11 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider LocalPeerBean jmxLocalPeerBean; private Map jmxRemotePeerBean; LeaderElectionBean jmxLeaderElectionBean; - private QuorumCnxManager qcm; + + // The QuorumCnxManager is held through an AtomicReference to ensure cross-thread visibility + // of updates; see the implementation comment at setLastSeenQuorumVerifier(). + private AtomicReference qcmRef = new AtomicReference<>(); + QuorumAuthServer authServer; QuorumAuthLearner authLearner; @@ -468,10 +472,10 @@ public int getQuorumSize(){ */ //last committed quorum verifier - public QuorumVerifier quorumVerifier; + private QuorumVerifier quorumVerifier; //last proposed quorum verifier - public QuorumVerifier lastSeenQuorumVerifier = null; + private QuorumVerifier lastSeenQuorumVerifier = null; // Lock object that guard access to quorumVerifier and lastSeenQuorumVerifier. final Object QV_LOCK = new Object(); @@ -765,20 +769,44 @@ public void recreateSocketAddresses(long id) { } } - public InetSocketAddress getQuorumAddress(){ + InetSocketAddress getQuorumAddress(){ + try { + synchronized (QV_LOCK) { + while (myAddrs.get() == null) { + QV_LOCK.wait(); + } + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } return myAddrs.get().quorumAddr; } - public InetSocketAddress getElectionAddress(){ + InetSocketAddress getElectionAddress(){ + try { + synchronized (QV_LOCK) { + while (myAddrs.get() == null) { + QV_LOCK.wait(); + } + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } return myAddrs.get().electionAddr; } - public InetSocketAddress getClientAddress(){ - return myAddrs.get().clientAddr; + private InetSocketAddress getClientAddress(){ + final AddressTuple addrs = myAddrs.get(); + return (addrs == null) ? null : addrs.clientAddr; } - public void setAddrs(InetSocketAddress quorumAddr, InetSocketAddress electionAddr, InetSocketAddress clientAddr){ - myAddrs.set(new AddressTuple(quorumAddr, electionAddr, clientAddr)); + private void setAddrs(InetSocketAddress quorumAddr, InetSocketAddress electionAddr, InetSocketAddress clientAddr){ + synchronized (QV_LOCK) { + myAddrs.set(new AddressTuple(quorumAddr, electionAddr, clientAddr)); + QV_LOCK.notifyAll(); + } } private int electionType; @@ -1059,7 +1087,8 @@ protected Election createElectionAlgorithm(int electionAlgorithm){ le = new AuthFastLeaderElection(this, true); break; case 3: - qcm = createCnxnManager(); + QuorumCnxManager qcm = createCnxnManager(); + qcmRef.set(qcm); QuorumCnxManager.Listener listener = qcm.listener; if(listener != null){ listener.start(); @@ -1526,18 +1555,6 @@ public QuorumVerifier getLastSeenQuorumVerifier(){ } } - private void connectNewPeers(){ - synchronized (QV_LOCK) { - if (qcm != null && quorumVerifier != null && lastSeenQuorumVerifier != null) { - Map committedView = quorumVerifier.getAllMembers(); - for (Entry e : lastSeenQuorumVerifier.getAllMembers().entrySet()) { - if (e.getKey() != getId() && !committedView.containsKey(e.getKey())) - qcm.connectOne(e.getKey()); - } - } - } - } - public synchronized void restartLeaderElection(QuorumVerifier qvOLD, QuorumVerifier qvNEW){ if (qvOLD == null || !qvOLD.equals(qvNEW)) { LOG.warn("Restarting Leader Election"); @@ -1556,32 +1573,50 @@ public String getNextDynamicConfigFilename() { } public void setLastSeenQuorumVerifier(QuorumVerifier qv, boolean writeToDisk){ - synchronized (QV_LOCK) { - if (lastSeenQuorumVerifier != null && lastSeenQuorumVerifier.getVersion() > qv.getVersion()) { - LOG.error("setLastSeenQuorumVerifier called with stale config " + qv.getVersion() + - ". Current version: " + quorumVerifier.getVersion()); + // If qcm is non-null, we may call qcm.connectOne(), which will take the lock on qcm + // and then take QV_LOCK. Take the locks in the same order to ensure that we don't + // deadlock against other callers of connectOne(). If qcmRef gets set in another + // thread while we're inside the synchronized block, that does no harm; if we didn't + // take a lock on qcm (because it was null when we sampled it), we won't call + // connectOne() on it. (Use of an AtomicReference is enough to guarantee visibility + // of updates that provably happen in another thread before entering this method.) + QuorumCnxManager qcm = qcmRef.get(); + Object outerLockObject = (qcm != null) ? qcm : QV_LOCK; + synchronized (outerLockObject) { + synchronized (QV_LOCK) { + if (lastSeenQuorumVerifier != null && lastSeenQuorumVerifier.getVersion() > qv.getVersion()) { + LOG.error("setLastSeenQuorumVerifier called with stale config " + qv.getVersion() + + ". Current version: " + quorumVerifier.getVersion()); + } + // assuming that a version uniquely identifies a configuration, so if + // version is the same, nothing to do here. + if (lastSeenQuorumVerifier != null && + lastSeenQuorumVerifier.getVersion() == qv.getVersion()) { + return; + } + lastSeenQuorumVerifier = qv; - } - // assuming that a version uniquely identifies a configuration, so if - // version is the same, nothing to do here. - if (lastSeenQuorumVerifier != null && - lastSeenQuorumVerifier.getVersion() == qv.getVersion()) { - return; - } - lastSeenQuorumVerifier = qv; - connectNewPeers(); - if (writeToDisk) { - try { - String fileName = getNextDynamicConfigFilename(); - if (fileName != null) { - QuorumPeerConfig.writeDynamicConfig(fileName, qv, true); + if (qcm != null && quorumVerifier != null && lastSeenQuorumVerifier != null) { + Map committedView = quorumVerifier.getAllMembers(); + for (Entry e : lastSeenQuorumVerifier.getAllMembers().entrySet()) { + if (e.getKey() != getId() && !committedView.containsKey(e.getKey())) + qcm.connectOne(e.getKey()); + } + } + + if (writeToDisk) { + try { + String fileName = getNextDynamicConfigFilename(); + if (fileName != null) { + QuorumPeerConfig.writeDynamicConfig(fileName, qv, true); + } + } catch (IOException e) { + LOG.error("Error writing next dynamic config file to disk: ", e.getMessage()); } - } catch (IOException e) { - LOG.error("Error writing next dynamic config file to disk: ", e.getMessage()); } } } - } + } public QuorumVerifier setQuorumVerifier(QuorumVerifier qv, boolean writeToDisk){ synchronized (QV_LOCK) { @@ -1800,7 +1835,7 @@ public boolean isRunning() { * get reference to QuorumCnxManager */ public QuorumCnxManager getQuorumCnxManager() { - return qcm; + return qcmRef.get(); } private long readLongFromFile(String name) throws IOException { File file = new File(logFactory.getSnapDir(), name); From 792cb225bfe2a0cd32177f90838e3955d22c781e Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Wed, 21 Nov 2018 09:01:45 +0000 Subject: [PATCH 03/63] [ZOOKEEPER-2778] QuorumPeer: add fast path for already-non-null quorum/election address --- .../zookeeper/server/quorum/QuorumPeer.java | 20 +++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) 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 c693a66004a..241101e2b54 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 @@ -770,31 +770,43 @@ public void recreateSocketAddresses(long id) { } InetSocketAddress getQuorumAddress(){ + AddressTuple addrs = myAddrs.get(); + if (addrs != null) { + return addrs.quorumAddr; + } try { synchronized (QV_LOCK) { - while (myAddrs.get() == null) { + addrs = myAddrs.get(); + while (addrs == null) { QV_LOCK.wait(); + addrs = myAddrs.get(); } } } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new RuntimeException(e); } - return myAddrs.get().quorumAddr; + return addrs.quorumAddr; } InetSocketAddress getElectionAddress(){ + AddressTuple addrs = myAddrs.get(); + if (addrs != null) { + return addrs.electionAddr; + } try { synchronized (QV_LOCK) { - while (myAddrs.get() == null) { + addrs = myAddrs.get(); + while (addrs == null) { QV_LOCK.wait(); + addrs = myAddrs.get(); } } } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new RuntimeException(e); } - return myAddrs.get().electionAddr; + return addrs.electionAddr; } private InetSocketAddress getClientAddress(){ From 3e116f85a0d48a3b2c26f0a0048158c5bebe5424 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Wed, 21 Nov 2018 17:13:14 +0000 Subject: [PATCH 04/63] [ZOOKEEPER-2778] QuorumPeer: fixes from code review --- .../zookeeper/server/quorum/QuorumPeer.java | 56 +++++++++---------- 1 file changed, 27 insertions(+), 29 deletions(-) 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 241101e2b54..ce67bc5dc4b 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 @@ -769,10 +769,10 @@ public void recreateSocketAddresses(long id) { } } - InetSocketAddress getQuorumAddress(){ + private AddressTuple getAddrs(){ AddressTuple addrs = myAddrs.get(); if (addrs != null) { - return addrs.quorumAddr; + return addrs; } try { synchronized (QV_LOCK) { @@ -781,35 +781,23 @@ InetSocketAddress getQuorumAddress(){ QV_LOCK.wait(); addrs = myAddrs.get(); } + return addrs; } } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new RuntimeException(e); } - return addrs.quorumAddr; + } + + public InetSocketAddress getQuorumAddress(){ + return getAddrs().quorumAddr; } - InetSocketAddress getElectionAddress(){ - AddressTuple addrs = myAddrs.get(); - if (addrs != null) { - return addrs.electionAddr; - } - try { - synchronized (QV_LOCK) { - addrs = myAddrs.get(); - while (addrs == null) { - QV_LOCK.wait(); - addrs = myAddrs.get(); - } - } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - return addrs.electionAddr; + public InetSocketAddress getElectionAddress(){ + return getAddrs().electionAddr; } - private InetSocketAddress getClientAddress(){ + public InetSocketAddress getClientAddress(){ final AddressTuple addrs = myAddrs.get(); return (addrs == null) ? null : addrs.clientAddr; } @@ -1584,6 +1572,21 @@ public String getNextDynamicConfigFilename() { return configFilename + QuorumPeerConfig.nextDynamicConfigFileSuffix; } + // On entry to this method, qcm must be non-null and the locks on both qcm and QV_LOCK + // must be held. We don't want quorumVerifier/lastSeenQuorumVerifier to change out from + // under us, so we have to hold QV_LOCK; and since the call to qcm.connectOne() will take + // the lock on qcm (and take QV_LOCK again inside that), the caller needs to have taken + // qcm outside QV_LOCK to avoid a deadlock against other callers of qcm.connectOne(). + private void connectNewPeers(QuorumCnxManager qcm){ + if (quorumVerifier != null && lastSeenQuorumVerifier != null) { + Map committedView = quorumVerifier.getAllMembers(); + for (Entry e : lastSeenQuorumVerifier.getAllMembers().entrySet()) { + if (e.getKey() != getId() && !committedView.containsKey(e.getKey())) + qcm.connectOne(e.getKey()); + } + } + } + public void setLastSeenQuorumVerifier(QuorumVerifier qv, boolean writeToDisk){ // If qcm is non-null, we may call qcm.connectOne(), which will take the lock on qcm // and then take QV_LOCK. Take the locks in the same order to ensure that we don't @@ -1607,13 +1610,8 @@ public void setLastSeenQuorumVerifier(QuorumVerifier qv, boolean writeToDisk){ return; } lastSeenQuorumVerifier = qv; - - if (qcm != null && quorumVerifier != null && lastSeenQuorumVerifier != null) { - Map committedView = quorumVerifier.getAllMembers(); - for (Entry e : lastSeenQuorumVerifier.getAllMembers().entrySet()) { - if (e.getKey() != getId() && !committedView.containsKey(e.getKey())) - qcm.connectOne(e.getKey()); - } + if (qcm != null) { + connectNewPeers(qcm); } if (writeToDisk) { From 5a994ca40fe550c747621634991c7dc636ca0b85 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Wed, 21 Nov 2018 17:19:44 +0000 Subject: [PATCH 05/63] [ZOOKEEPER-2778] QuorumPeer: fix access to newly private data members from ReconfigTest --- .../src/test/java/org/apache/zookeeper/test/ReconfigTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 a050f7a9f10..fb0e5f01b38 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 @@ -844,7 +844,7 @@ public void testQuorumSystemChange() throws Exception { testNormalOperation(zkArr[4], zkArr[5]); for (int i = 1; i <= 5; i++) { - if (!(qu.getPeer(i).peer.quorumVerifier instanceof QuorumHierarchical)) + if (!(qu.getPeer(i).peer.getQuorumVerifier() instanceof QuorumHierarchical)) Assert.fail("peer " + i + " doesn't think the quorum system is Hieararchical!"); } @@ -881,7 +881,7 @@ public void testQuorumSystemChange() throws Exception { testNormalOperation(zkArr[1], zkArr[2]); for (int i = 1; i <= 2; i++) { - if (!(qu.getPeer(i).peer.quorumVerifier instanceof QuorumMaj)) + if (!(qu.getPeer(i).peer.getQuorumVerifier() instanceof QuorumMaj)) Assert.fail("peer " + i + " doesn't think the quorum system is a majority quorum system!"); From 7940892416f5a4a6806e875902be585e2323945f Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Wed, 21 Nov 2018 19:48:49 +0000 Subject: [PATCH 06/63] [ZOOKEEPER-2778] LeaderBeanTest: set up mock QuorumVerifier so that addresses get set --- .../server/quorum/LeaderBeanTest.java | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java index 31cef798597..99d5b5d6cd5 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java @@ -21,10 +21,13 @@ import org.apache.jute.OutputArchive; import org.apache.jute.Record; import org.apache.zookeeper.common.X509Exception; +import org.apache.zookeeper.PortAssignment; import org.apache.zookeeper.server.Request; import org.apache.zookeeper.server.ZKDatabase; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; +import org.apache.zookeeper.server.quorum.QuorumPeer.LearnerType; +import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; import org.apache.zookeeper.server.util.SerializeUtils; import org.apache.zookeeper.test.ClientBase; import org.apache.zookeeper.txn.TxnHeader; @@ -36,6 +39,10 @@ import java.io.File; import java.io.IOException; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.util.HashMap; +import java.util.Map; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; @@ -43,6 +50,7 @@ import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class LeaderBeanTest { private Leader leader; @@ -54,8 +62,22 @@ public class LeaderBeanTest { @Before public void setUp() throws IOException, X509Exception { qp = new QuorumPeer(); + long myId = qp.getId(); + + int clientPort = PortAssignment.unique(); + Map peersView = new HashMap(); + InetAddress clientIP = InetAddress.getLoopbackAddress(); + + peersView.put(Long.valueOf(myId), + new QuorumServer(myId, new InetSocketAddress(clientIP, PortAssignment.unique()), + new InetSocketAddress(clientIP, PortAssignment.unique()), + new InetSocketAddress(clientIP, clientPort), LearnerType.PARTICIPANT)); + QuorumVerifier quorumVerifierMock = mock(QuorumVerifier.class); + when(quorumVerifierMock.getAllMembers()).thenReturn(peersView); + qp.setQuorumVerifier(quorumVerifierMock, false); + File tmpDir = ClientBase.createTmpDir(); fileTxnSnapLog = new FileTxnSnapLog(new File(tmpDir, "data"), new File(tmpDir, "data_txnlog")); From 74139995127cc9f2c5f4e86680cc91d573234405 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Thu, 22 Nov 2018 19:21:19 +0000 Subject: [PATCH 07/63] [ZOOKEEPER-2778] QuorumPeer: warn when clobbering existing election algorithm --- .../org/apache/zookeeper/server/quorum/QuorumPeer.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) 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 ce67bc5dc4b..52910670000 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 @@ -126,7 +126,7 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider */ private ZKDatabase zkDb; - public static class AddressTuple { + public static final class AddressTuple { public final InetSocketAddress quorumAddr; public final InetSocketAddress electionAddr; public final InetSocketAddress clientAddr; @@ -1088,7 +1088,10 @@ protected Election createElectionAlgorithm(int electionAlgorithm){ break; case 3: QuorumCnxManager qcm = createCnxnManager(); - qcmRef.set(qcm); + if (!qcmRef.compareAndSet(null, qcm)) { + LOG.warn("Clobbering already-set QuorumCnxManager (restarting leader election?)"); + qcmRef.set(qcm); + } QuorumCnxManager.Listener listener = qcm.listener; if(listener != null){ listener.start(); From 6a40535b4a54464d1dbae1d890f3f2702f7ec4bc Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Thu, 22 Nov 2018 19:29:27 +0000 Subject: [PATCH 08/63] [ZOOKEEPER-2778] QuorumPeer: halt old QCM when clobbering existing election algorithm --- .../java/org/apache/zookeeper/server/quorum/QuorumPeer.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 52910670000..5a66a22403e 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 @@ -1088,9 +1088,10 @@ protected Election createElectionAlgorithm(int electionAlgorithm){ break; case 3: QuorumCnxManager qcm = createCnxnManager(); - if (!qcmRef.compareAndSet(null, qcm)) { + QuorumCnxManager oldQcm = qcmRef.getAndSet(qcm); + if (oldQcm != null) { LOG.warn("Clobbering already-set QuorumCnxManager (restarting leader election?)"); - qcmRef.set(qcm); + oldQcm.halt(); } QuorumCnxManager.Listener listener = qcm.listener; if(listener != null){ From 8cd20ebc2b3e7b80679b09f7564304d2321c6491 Mon Sep 17 00:00:00 2001 From: Fangmin Lyu Date: Thu, 15 Nov 2018 09:46:51 -0800 Subject: [PATCH 09/63] ZOOKEEPER-1818: Correctly handle potentially inconsistent zxid/electionEpoch and peerEpoch during leader election --- .../server/quorum/FastLeaderElection.java | 48 +++---- .../zookeeper/server/quorum/QuorumPeer.java | 32 +++++ .../apache/zookeeper/server/quorum/Vote.java | 40 +++++- .../server/quorum/FLEOutOfElectionTest.java | 136 ++++++++++++++++++ 4 files changed, 222 insertions(+), 34 deletions(-) create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FLEOutOfElectionTest.java diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java index e3cb0454cc5..401ca0521bd 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java @@ -70,14 +70,6 @@ public class FastLeaderElection implements Election { final static int maxNotificationInterval = 60000; - /** - * This value is passed to the methods that check the quorum - * majority of an established ensemble for those values that - * should not be taken into account in the comparison - * (electionEpoch and zxid). - */ - final static int IGNOREVALUE = -1; - /** * Connection manager. Fast leader election uses TCP for * communication between peers, and QuorumCnxManager manages @@ -732,7 +724,7 @@ protected boolean totalOrderPredicate(long newId, long newZxid, long newEpoch, l * @param vote * Identifier of the vote received last */ - private boolean termPredicate(HashMap votes, Vote vote) { + protected boolean termPredicate(Map votes, Vote vote) { SyncedLearnerTracker voteSet = new SyncedLearnerTracker(); voteSet.addQuorumVerifier(self.getQuorumVerifier()); if (self.getLastSeenQuorumVerifier() != null @@ -765,8 +757,8 @@ private boolean termPredicate(HashMap votes, Vote vote) { * @param leader leader id * @param electionEpoch epoch id */ - private boolean checkLeader( - HashMap votes, + protected boolean checkLeader( + Map votes, long leader, long electionEpoch){ @@ -966,6 +958,7 @@ else if (validVoter(n.sid) && validVoter(n.leader)) { ", proposed election epoch=0x" + Long.toHexString(n.electionEpoch)); } + // don't care about the version if it's in LOOKING state recvset.put(n.sid, new Vote(n.leader, n.zxid, n.electionEpoch, n.peerEpoch)); if (termPredicate(recvset, @@ -989,9 +982,9 @@ else if (validVoter(n.sid) && validVoter(n.leader)) { if (n == null) { self.setPeerState((proposedLeader == self.getId()) ? ServerState.LEADING: learningState()); - Vote endVote = new Vote(proposedLeader, - proposedZxid, proposedEpoch); + proposedZxid, logicalclock.get(), + proposedEpoch); leaveInstance(endVote); return endVote; } @@ -1008,13 +1001,13 @@ else if (validVoter(n.sid) && validVoter(n.leader)) { */ if(n.electionEpoch == logicalclock.get()){ recvset.put(n.sid, new Vote(n.leader, n.zxid, n.electionEpoch, n.peerEpoch)); - if(termPredicate(recvset, new Vote(n.leader, + if(termPredicate(recvset, new Vote(n.version, n.leader, n.zxid, n.electionEpoch, n.peerEpoch, n.state)) && checkLeader(outofelection, n.leader, n.electionEpoch)) { self.setPeerState((n.leader == self.getId()) ? ServerState.LEADING: learningState()); - - Vote endVote = new Vote(n.leader, n.zxid, n.peerEpoch); + Vote endVote = new Vote(n.leader, + n.zxid, n.electionEpoch, n.peerEpoch); leaveInstance(endVote); return endVote; } @@ -1023,28 +1016,19 @@ && checkLeader(outofelection, n.leader, n.electionEpoch)) { /* * Before joining an established ensemble, verify that * a majority are following the same leader. - * Only peer epoch is used to check that the votes come - * from the same ensemble. This is because there is at - * least one corner case in which the ensemble can be - * created with inconsistent zxid and election epoch - * info. However, given that only one ensemble can be - * running at a single point in time and that each - * epoch is used only once, using only the epoch to - * compare the votes is sufficient. - * - * @see https://issues.apache.org/jira/browse/ZOOKEEPER-1732 */ - outofelection.put(n.sid, new Vote(n.leader, - IGNOREVALUE, IGNOREVALUE, n.peerEpoch, n.state)); - if (termPredicate(outofelection, new Vote(n.leader, - IGNOREVALUE, IGNOREVALUE, n.peerEpoch, n.state)) - && checkLeader(outofelection, n.leader, IGNOREVALUE)) { + outofelection.put(n.sid, new Vote(n.version, n.leader, + n.zxid, n.electionEpoch, n.peerEpoch, n.state)); + if (termPredicate(outofelection, new Vote(n.version, n.leader, + n.zxid, n.electionEpoch, n.peerEpoch, n.state)) + && checkLeader(outofelection, n.leader, n.electionEpoch)) { synchronized(this){ logicalclock.set(n.electionEpoch); self.setPeerState((n.leader == self.getId()) ? ServerState.LEADING: learningState()); } - Vote endVote = new Vote(n.leader, n.zxid, n.peerEpoch); + Vote endVote = new Vote(n.leader, n.zxid, + n.electionEpoch, n.peerEpoch); leaveInstance(endVote); return endVote; } 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 5a66a22403e..830284283d3 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 @@ -2051,6 +2051,38 @@ private boolean updateVote(long designatedLeader, long zxid){ /** * Updates leader election info to avoid inconsistencies when * a new server tries to join the ensemble. + * + * Here is the inconsistency scenario we try to solve by updating the peer + * epoch after following leader: + * + * Let's say we have an ensemble with 3 servers z1, z2 and z3. + * + * 1. z1, z2 were following z3 with peerEpoch to be 0xb8, the new epoch is + * 0xb9, aka current accepted epoch on disk. + * 2. z2 get restarted, which will use 0xb9 as it's peer epoch when loading + * the current accept epoch from disk. + * 3. z2 received notification from z1 and z3, which is following z3 with + * epoch 0xb8, so it started following z3 again with peer epoch 0xb8. + * 4. before z2 successfully connected to z3, z3 get restarted with new + * epoch 0xb9. + * 5. z2 will retry around a few round (default 5s) before giving up, + * meanwhile it will report z3 as leader. + * 6. z1 restarted, and looking with peer epoch 0xb9. + * 7. z1 voted z3, and z3 was elected as leader again with peer epoch 0xb9. + * 8. z2 successfully connected to z3 before giving up, but with peer + * epoch 0xb8. + * 9. z1 get restarted, looking for leader with peer epoch 0xba, but cannot + * join, because z2 is reporting peer epoch 0xb8, while z3 is reporting + * 0xb9. + * + * By updating the election vote after actually following leader, we can + * avoid this kind of stuck happened. + * + * Btw, the zxid and electionEpoch could be inconsistent because of the same + * reason, it's better to update these as well after syncing with leader, but + * that required protocol change which is non trivial. This problem is worked + * around by skipping comparing the zxid and electionEpoch when counting for + * votes for out of election servers during looking for leader. * * @see https://issues.apache.org/jira/browse/ZOOKEEPER-1732 */ diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Vote.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Vote.java index 8152c66b39c..7af997b95d2 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Vote.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Vote.java @@ -125,11 +125,47 @@ public boolean equals(Object o) { return false; } Vote other = (Vote) o; - return (id == other.id + + if ((state == ServerState.LOOKING) || + (other.state == ServerState.LOOKING)) { + return (id == other.id && zxid == other.zxid && electionEpoch == other.electionEpoch && peerEpoch == other.peerEpoch); - + } else { + /* + * There are two things going on in the logic below: + * + * 1. skip comparing the zxid and electionEpoch for votes for servers + * out of election. + * + * Need to skip those because they can be inconsistent due to + * scenarios described in QuorumPeer.updateElectionVote. + * + * And given that only one ensemble can be running at a single point + * in time and that each epoch is used only once, using only id and + * epoch to compare the votes is sufficient. + * + * 2. skip comparing peerEpoch if if we're running with mixed ensemble + * with (version > 0x0) and without the change (version = 0x0) + * introduced in ZOOKEEPER-1732. + * + * The server running with and without ZOOKEEPER-1732 will return + * different peerEpoch. During rolling upgrades, it's possible + * that 2/5 servers are returning epoch 1, while the other 2/5 + * are returning epoch 2, the other server need to ignore the + * peerEpoch to be able to join it. + * + * {@see https://issues.apache.org/jira/browse/ZOOKEEPER-1732} + * {@see https://issues.apache.org/jira/browse/ZOOKEEPER-1805} + */ + if ((version > 0x0) ^ (other.version > 0x0)) { + return id == other.id; + } else { + return (id == other.id + && peerEpoch == other.peerEpoch); + } + } } @Override diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FLEOutOfElectionTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FLEOutOfElectionTest.java new file mode 100644 index 00000000000..c80a2186bfb --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FLEOutOfElectionTest.java @@ -0,0 +1,136 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.zookeeper.server.quorum; + +import java.io.File; +import java.net.InetSocketAddress; +import java.util.HashMap; +import java.util.Map; + +import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.server.quorum.QuorumPeer; +import org.apache.zookeeper.server.quorum.FastLeaderElection.Notification; +import org.apache.zookeeper.server.quorum.Vote; +import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; +import org.apache.zookeeper.server.quorum.QuorumPeer.ServerState; +import org.apache.zookeeper.server.util.ZxidUtils; +import org.apache.zookeeper.test.ClientBase; +import org.apache.zookeeper.test.FLETest; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +/** + * Test FastLeaderElection with out of election servers. + */ +public class FLEOutOfElectionTest { + + private FastLeaderElection fle; + + @Before + public void setUp() throws Exception { + File tmpdir = ClientBase.createTmpDir(); + Map peers = new HashMap(); + for(int i = 0; i < 5; i++) { + peers.put(Long.valueOf(i), new QuorumServer(Long.valueOf(i), + new InetSocketAddress("127.0.0.1", PortAssignment.unique()))); + } + QuorumPeer peer = new QuorumPeer(peers, tmpdir, tmpdir, + PortAssignment.unique(), 3, 3, 1000, 2, 2); + fle = new FastLeaderElection(peer, peer.createCnxnManager()); + } + + @Test + public void testIgnoringZxidElectionEpoch() { + Map votes = new HashMap(); + votes.put(0L, new Vote(0x1, 4L, ZxidUtils.makeZxid(1, 1), 1, 2, ServerState.FOLLOWING)); + votes.put(1L, new Vote(0x1, 4L, ZxidUtils.makeZxid(1, 2), 1, 2, ServerState.FOLLOWING)); + votes.put(3L, new Vote(0x1, 4L, ZxidUtils.makeZxid(2, 1), 2, 2, ServerState.FOLLOWING)); + votes.put(4L, new Vote(0x1, 4L, ZxidUtils.makeZxid(2, 1), 2, 2, ServerState.LEADING)); + + Assert.assertTrue(fle.termPredicate(votes, + new Vote(4L, ZxidUtils.makeZxid(2, 1), 2, 2, ServerState.FOLLOWING))); + } + + @Test + public void testElectionWIthDifferentVersion() { + Map votes = new HashMap(); + votes.put(0L, new Vote(0x1, 4L, ZxidUtils.makeZxid(1, 1), 1, 1, ServerState.FOLLOWING)); + votes.put(1L, new Vote(0x1, 4L, ZxidUtils.makeZxid(1, 1), 1, 1, ServerState.FOLLOWING)); + votes.put(3L, new Vote(4L, ZxidUtils.makeZxid(2, 1), 2, 2, ServerState.FOLLOWING)); + votes.put(4L, new Vote(4L, ZxidUtils.makeZxid(2, 1), 2, 2, ServerState.LEADING)); + + Assert.assertTrue(fle.termPredicate(votes, + new Vote(4L, ZxidUtils.makeZxid(2, 1), 2, 2, ServerState.FOLLOWING))); + } + + @Test + public void testLookingNormal() { + Map votes = new HashMap(); + votes.put(0L, new Vote(4L, ZxidUtils.makeZxid(2, 1), 1, 1, ServerState.LOOKING)); + votes.put(1L, new Vote(4L, ZxidUtils.makeZxid(2, 1), 1, 1, ServerState.LOOKING)); + votes.put(3L, new Vote(4L, ZxidUtils.makeZxid(2, 1), 1, 1, ServerState.LOOKING)); + votes.put(4L, new Vote(4L, ZxidUtils.makeZxid(2, 1), 1, 1, ServerState.LEADING)); + + Assert.assertTrue(fle.termPredicate(votes, + new Vote(4L, ZxidUtils.makeZxid(2, 1), 1, 1, ServerState.LOOKING))); + } + + @Test + public void testLookingDiffRounds() { + HashMap votes = new HashMap(); + votes.put(0L, new Vote(4L, ZxidUtils.makeZxid(1, 1), 1, 1, ServerState.LOOKING)); + votes.put(1L, new Vote(4L, ZxidUtils.makeZxid(2, 1), 2, 2, ServerState.LOOKING)); + votes.put(3L, new Vote(4L, ZxidUtils.makeZxid(2, 1), 3, 2, ServerState.LOOKING)); + votes.put(4L, new Vote(4L, ZxidUtils.makeZxid(2, 1), 3, 2, ServerState.LEADING)); + + Assert.assertFalse(fle.termPredicate(votes, + new Vote(4L, ZxidUtils.makeZxid(2, 1), 2, 2, ServerState.LOOKING))); + } + + @Test + public void testOutofElection() { + HashMap outofelection = new HashMap(); + + outofelection.put(1L, new Vote(0x0, 5, ZxidUtils.makeZxid(15, 0), 0xa, 0x17, ServerState.FOLLOWING)); + outofelection.put(2L, new Vote(0x0, 5, ZxidUtils.makeZxid(15, 0), 0xa, 0x17, ServerState.FOLLOWING)); + outofelection.put(4L, new Vote(0x1, 5, ZxidUtils.makeZxid(15, 0), 0xa, 0x18, ServerState.FOLLOWING)); + Vote vote = new Vote(0x1, 5, ZxidUtils.makeZxid(15, 0), 0xa, 0x18, ServerState.LEADING); + outofelection.put(5L, vote); + + Notification n = new Notification(); + n.version = vote.getVersion(); + n.leader = vote.getId(); + n.zxid = vote.getZxid(); + n.electionEpoch = vote.getElectionEpoch(); + n.state = vote.getState(); + n.peerEpoch = vote.getPeerEpoch(); + n.sid = 5L; + + // Set the logical clock to 1 on fle instance of server 3. + fle.logicalclock.set(0x1); + + Assert.assertTrue("Termination predicate failed", + fle.termPredicate(outofelection, new Vote(n.version, n.leader, + n.zxid, n.electionEpoch, n.peerEpoch, n.state))); + + Assert.assertTrue("Leader check failed", fle.checkLeader(outofelection, + n.leader, n.electionEpoch)); + } +} From c717bfbd2ba98851ae684ad3985c19132c2dd8e6 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Wed, 21 Nov 2018 20:31:16 +0000 Subject: [PATCH 10/63] ZOOKEEPER-1636: cleanup completion list of a failed multi request (from Thawan Kooburat) --- .../zookeeper-client-c/src/zookeeper.c | 17 ++++- .../zookeeper-client-c/tests/TestMulti.cc | 68 +++++++++++++++++++ 2 files changed, 84 insertions(+), 1 deletion(-) diff --git a/zookeeper-client/zookeeper-client-c/src/zookeeper.c b/zookeeper-client/zookeeper-client-c/src/zookeeper.c index 6daf4a75c12..fc045552fba 100644 --- a/zookeeper-client/zookeeper-client-c/src/zookeeper.c +++ b/zookeeper-client/zookeeper-client-c/src/zookeeper.c @@ -2608,6 +2608,17 @@ completion_list_t *dequeue_completion(completion_head_t *list) return cptr; } +// cleanup completion list of a failed multi request +static void cleanup_failed_multi(zhandle_t *zh, int xid, int rc, completion_list_t *cptr) { + completion_list_t *entry; + completion_head_t *clist = &cptr->c.clist; + while ((entry = dequeue_completion(clist)) != NULL) { + // Fake failed response for all sub-requests + deserialize_response(zh, entry->c.type, xid, 1, rc, entry, NULL); + destroy_completion_entry(entry); + } +} + static int deserialize_multi(zhandle_t *zh, int xid, completion_list_t *cptr, struct iarchive *ia) { int rc = 0; @@ -2737,8 +2748,12 @@ static void deserialize_response(zhandle_t *zh, int type, int xid, int failed, i case COMPLETION_MULTI: LOG_DEBUG(LOGCALLBACK(zh), "Calling COMPLETION_MULTI for xid=%#x failed=%d rc=%d", cptr->xid, failed, rc); - rc = deserialize_multi(zh, xid, cptr, ia); assert(cptr->c.void_result); + if (failed) { + cleanup_failed_multi(zh, xid, rc, cptr); + } else { + rc = deserialize_multi(zh, xid, cptr, ia); + } cptr->c.void_result(rc, cptr->data); break; default: diff --git a/zookeeper-client/zookeeper-client-c/tests/TestMulti.cc b/zookeeper-client/zookeeper-client-c/tests/TestMulti.cc index 0ee9566ffc0..ec1096df765 100644 --- a/zookeeper-client/zookeeper-client-c/tests/TestMulti.cc +++ b/zookeeper-client/zookeeper-client-c/tests/TestMulti.cc @@ -178,6 +178,7 @@ class Zookeeper_multi : public CPPUNIT_NS::TestFixture CPPUNIT_TEST(testCheck); CPPUNIT_TEST(testWatch); CPPUNIT_TEST(testSequentialNodeCreateInAsyncMulti); + CPPUNIT_TEST(testBigAsyncMulti); CPPUNIT_TEST_SUITE_END(); static void watcher(zhandle_t *, int type, int state, const char *path,void*v){ @@ -248,6 +249,16 @@ class Zookeeper_multi : public CPPUNIT_NS::TestFixture count++; } + static void multi_completion_fn_rc(int rc, const void *data) { + count++; + *((int*) data) = rc; + } + + static void create_completion_fn_rc(int rc, const char* value, const void *data) { + count++; + *((int*) data) = rc; + } + static void waitForMultiCompletion(int seconds) { time_t expires = time(0) + seconds; while(count == 0 && time(0) < expires) { @@ -654,6 +665,63 @@ class Zookeeper_multi : public CPPUNIT_NS::TestFixture // wait for multi completion in doMultiInWatch waitForMultiCompletion(5); } + + /** + * ZOOKEEPER-1636: If request is too large, the server will cut the + * connection without sending response packet. The client will try to + * process completion on multi request and eventually cause SIGSEGV + */ + void testBigAsyncMulti() { + int rc; + int callback_rc = (int) ZOK; + watchctx_t ctx; + zhandle_t *zk = createClient(&ctx); + + // The request should be more than 1MB which exceeds the default + // jute.maxbuffer and causes the server to drop client connection + const int iteration = 500; + const int type_count = 3; + const int nops = iteration * type_count; + char buff[1024]; + + zoo_op_result_t results[nops]; + zoo_op_t ops[nops]; + struct Stat* s[nops]; + int index = 0; + + // Test that we deliver error to 3 types of sub-request + for (int i = 0; i < iteration; ++i) { + zoo_set_op_init(&ops[index++], "/x", buff, sizeof(buff), -1, s[i]); + zoo_create_op_init(&ops[index++], "/x", buff, sizeof(buff), + &ZOO_OPEN_ACL_UNSAFE, ZOO_SEQUENCE, NULL, 0); + zoo_delete_op_init(&ops[index++], "/x", -1); + } + + rc = zoo_amulti(zk, nops, ops, results, multi_completion_fn_rc, + + &callback_rc); + CPPUNIT_ASSERT_EQUAL((int) ZOK, rc); + + waitForMultiCompletion(10); + // With the bug, we will get SIGSEGV before reaching this point + CPPUNIT_ASSERT_EQUAL((int) ZCONNECTIONLOSS, callback_rc); + + // Make sure that all sub-request completions get processed + for (int i = 0; i < nops; ++i) { + CPPUNIT_ASSERT_EQUAL((int) ZCONNECTIONLOSS, results[i].err); + } + + // The handle should be able to recover itself. + ctx.waitForConnected(zk); + + // Try to submit another async request to see if it get processed + // correctly + rc = zoo_acreate(zk, "/target", "", 0, &ZOO_OPEN_ACL_UNSAFE, 0, + create_completion_fn_rc, &callback_rc); + CPPUNIT_ASSERT_EQUAL((int) ZOK, rc); + + waitForMultiCompletion(10); + CPPUNIT_ASSERT_EQUAL((int) ZOK, callback_rc); + } /** * ZOOKEEPER-1624: PendingChanges of create sequential node request didn't From 6ad24aec81b20d3c32e80be26a20ce2ffc016253 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Wed, 21 Nov 2018 23:09:55 +0000 Subject: [PATCH 11/63] ZOOKEEPER-2488: Synchronized access to shuttingDownLE in QuorumPeer --- .../zookeeper/server/quorum/QuorumPeer.java | 22 +++++++++++-------- 1 file changed, 13 insertions(+), 9 deletions(-) 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 830284283d3..fa6a456f2d7 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 @@ -1214,10 +1214,12 @@ public void run() { }; try { roZkMgr.start(); - reconfigFlagClear(); - if (shuttingDownLE) { - shuttingDownLE = false; - startLeaderElection(); + synchronized (this) { + reconfigFlagClear(); + if (shuttingDownLE) { + shuttingDownLE = false; + startLeaderElection(); + } } setCurrentVote(makeLEStrategy().lookForLeader()); } catch (Exception e) { @@ -1231,11 +1233,13 @@ public void run() { } } else { try { - reconfigFlagClear(); - if (shuttingDownLE) { - shuttingDownLE = false; - startLeaderElection(); - } + synchronized (this) { + reconfigFlagClear(); + if (shuttingDownLE) { + shuttingDownLE = false; + startLeaderElection(); + } + } setCurrentVote(makeLEStrategy().lookForLeader()); } catch (Exception e) { LOG.warn("Unexpected exception", e); From 6889548765addb3bcd5b0f577b07f1e9fc4f6fd1 Mon Sep 17 00:00:00 2001 From: Andor Molnar Date: Mon, 19 Nov 2018 17:25:52 +0100 Subject: [PATCH 12/63] ZOOKEEPER-3193. Refactor SaslAuthFail test to use single class. Use CountDownLatch to sync with watcher. --- .../test/SaslAuthFailNotifyTest.java | 98 ------------------- .../zookeeper/test/SaslAuthFailTest.java | 46 ++++++--- 2 files changed, 32 insertions(+), 112 deletions(-) delete mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslAuthFailNotifyTest.java diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslAuthFailNotifyTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslAuthFailNotifyTest.java deleted file mode 100644 index 2b00d862dfe..00000000000 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslAuthFailNotifyTest.java +++ /dev/null @@ -1,98 +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.test; - -import java.io.File; -import java.io.FileWriter; -import java.io.IOException; -import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.TestableZooKeeper; -import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.ZooKeeper; -import org.apache.zookeeper.Watcher.Event.KeeperState; -import org.apache.zookeeper.ZooDefs.Ids; -import org.junit.Test; -import org.junit.Assert; - -public class SaslAuthFailNotifyTest extends ClientBase { - static { - System.setProperty("zookeeper.authProvider.1","org.apache.zookeeper.server.auth.SASLAuthenticationProvider"); - System.setProperty("zookeeper.allowSaslFailedClients","true"); - - try { - File tmpDir = createTmpDir(); - File saslConfFile = new File(tmpDir, "jaas.conf"); - FileWriter fwriter = new FileWriter(saslConfFile); - - fwriter.write("" + - "Server {\n" + - " org.apache.zookeeper.server.auth.DigestLoginModule required\n" + - " user_super=\"test\";\n" + - "};\n" + - "Client {\n" + - " org.apache.zookeeper.server.auth.DigestLoginModule required\n" + - " username=\"super\"\n" + - " password=\"test1\";\n" + // NOTE: wrong password ('test' != 'test1') : this is to test SASL authentication failure. - "};" + "\n"); - fwriter.close(); - System.setProperty("java.security.auth.login.config",saslConfFile.getAbsolutePath()); - } - catch (IOException e) { - // could not create tmp directory to hold JAAS conf file. - } - } - - private AtomicInteger authFailed = new AtomicInteger(0); - - @Override - protected TestableZooKeeper createClient(String hp) - throws IOException, InterruptedException - { - MyWatcher watcher = new MyWatcher(); - return createClient(watcher, hp); - } - - private class MyWatcher extends CountdownWatcher { - @Override - public synchronized void process(WatchedEvent event) { - if (event.getState() == KeeperState.AuthFailed) { - synchronized(authFailed) { - authFailed.incrementAndGet(); - authFailed.notify(); - } - } - else { - super.process(event); - } - } - } - - @Test - public void testBadSaslAuthNotifiesWatch() throws Exception { - ZooKeeper zk = createClient(); - // wait for authFailed event from client's EventThread. - synchronized(authFailed) { - authFailed.wait(); - } - Assert.assertEquals(authFailed.get(),1); - zk.close(); - } -} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslAuthFailTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslAuthFailTest.java index 33a505ea2a7..d0847fb09cf 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslAuthFailTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslAuthFailTest.java @@ -18,19 +18,18 @@ package org.apache.zookeeper.test; -import java.io.File; -import java.io.FileWriter; -import java.io.IOException; -import java.util.concurrent.atomic.AtomicInteger; - import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.TestableZooKeeper; import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.Watcher.Event.KeeperState; import org.apache.zookeeper.ZooDefs.Ids; -import org.junit.Test; +import org.apache.zookeeper.ZooKeeper; import org.junit.Assert; +import org.junit.Test; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.util.concurrent.CountDownLatch; public class SaslAuthFailTest extends ClientBase { static { @@ -59,18 +58,37 @@ public class SaslAuthFailTest extends ClientBase { // could not create tmp directory to hold JAAS conf file. } } + + private CountDownLatch authFailed = new CountDownLatch(1); + + private class MyWatcher extends CountdownWatcher { + @Override + public synchronized void process(WatchedEvent event) { + if (event.getState() == KeeperState.AuthFailed) { + authFailed.countDown(); + } + else { + super.process(event); + } + } + } @Test - public void testAuthFail() throws Exception { - ZooKeeper zk = createClient(); - try { + public void testAuthFail() { + try (ZooKeeper zk = createClient()) { zk.create("/path1", null, Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT); Assert.fail("Should have gotten exception."); - } catch(Exception e ) { + } catch (Exception e) { // ok, exception as expected. LOG.info("Got exception as expected: " + e); - } finally { - zk.close(); + } + } + + @Test + public void testBadSaslAuthNotifiesWatch() throws Exception { + try (ZooKeeper ignored = createClient(new MyWatcher(), hostPort)) { + // wait for authFailed event from client's EventThread. + authFailed.await(); } } } From 21fe9a3639a5f5ee554b1712c96c03b35cfa00f0 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Wed, 21 Nov 2018 21:33:01 +0000 Subject: [PATCH 13/63] Bump library versions, fix 'ant package-native tar' targets --- build.xml | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) diff --git a/build.xml b/build.xml index 0216a113af7..2bcc1215266 100644 --- a/build.xml +++ b/build.xml @@ -43,7 +43,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + @@ -53,8 +53,8 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - - + + @@ -998,17 +998,9 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - - - - - - - - From c60deca87b7081be5b7eb1fb714ebb4aa6b52011 Mon Sep 17 00:00:00 2001 From: Andor Molnar Date: Wed, 23 May 2018 06:39:31 +0200 Subject: [PATCH 14/63] ZOOKEEPER-3034: update autoconf for cppunit >= 1.14.0 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Roll-up (with typo fix) of Andor Molnár's patches (motivated by Ubuntu 18.04 support issues): * ZOOKEEPER-3034. Added cppunit.m4, because from libcppunit-dev package doesn't ship from 1.14.0 * ZOOKEEPER-3034. Use pkg-config to setup CPPUNIT * ZOOKEEPER-3034. Define macro to support static linking in pkg-config fix --- .../zookeeper-client-c/configure.ac | 23 +++++++++++-------- .../zookeeper-client-c/src/zookeeper.c | 22 ------------------ 2 files changed, 13 insertions(+), 32 deletions(-) diff --git a/zookeeper-client/zookeeper-client-c/configure.ac b/zookeeper-client/zookeeper-client-c/configure.ac index f4ddd35c356..90a0de37532 100644 --- a/zookeeper-client/zookeeper-client-c/configure.ac +++ b/zookeeper-client/zookeeper-client-c/configure.ac @@ -25,17 +25,20 @@ DX_INIT_DOXYGEN([zookeeper],[c-doc.Doxyfile],[docs]) AM_INIT_AUTOMAKE([-Wall foreign]) AC_CONFIG_HEADER([config.h]) -# Checks for programs. -AC_ARG_WITH(cppunit, - [ --without-cppunit do not use CPPUNIT]) +AC_DEFUN([ZK_PKG_CONFIG_STATIC], + [AC_REQUIRE([PKG_PROG_PKG_CONFIG])dnl + _save_PKG_CONFIG=$PKG_CONFIG + PKG_CONFIG="$PKG_CONFIG --static" + PKG_CHECK_MODULES([$1], [$2], [$3], [$4], [$5]) + PKG_CONFIG=$_save_PKG_CONFIG[]dnl +]) -if test "$with_cppunit" = "no" ; then - CPPUNIT_PATH="No_CPPUNIT" - CPPUNIT_INCLUDE= - CPPUNIT_LIBS= -else - AM_PATH_CPPUNIT(1.10.2) -fi +# Checks for programs. +AC_ARG_WITH([cppunit], + AS_HELP_STRING([--without-cppunit], [do not use CPPUNIT])) +AS_IF([test "x$without_cppunit" != "xyes"], [ + ZK_PKG_CONFIG_STATIC([CPPUNIT], [cppunit >= 1.10.2]) +]) if test "$CALLER" = "ANT" ; then CPPUNIT_CFLAGS="$CPPUNIT_CFLAGS -DZKSERVER_CMD=\"\\\"${base_dir}/zookeeper-client/zookeeper-client-c/tests/zkServer.sh\\\"\"" diff --git a/zookeeper-client/zookeeper-client-c/src/zookeeper.c b/zookeeper-client/zookeeper-client-c/src/zookeeper.c index fc045552fba..12da16905fb 100644 --- a/zookeeper-client/zookeeper-client-c/src/zookeeper.c +++ b/zookeeper-client/zookeeper-client-c/src/zookeeper.c @@ -253,8 +253,6 @@ static void cleanup_bufs(zhandle_t *zh,int callCompletion,int rc); static int disable_conn_permute=0; // permute enabled by default static struct sockaddr_storage *addr_rw_server = 0; -static __attribute__((unused)) void print_completion_queue(zhandle_t *zh); - static void *SYNCHRONOUS_MARKER = (void*)&SYNCHRONOUS_MARKER; static int isValidPath(const char* path, const int flags); @@ -2530,26 +2528,6 @@ int api_epilog(zhandle_t *zh,int rc) return rc; } -static __attribute__((unused)) void print_completion_queue(zhandle_t *zh) -{ - completion_list_t* cptr; - - if(logLevelsent_requests.head==0) { - fprintf(LOGSTREAM,"empty\n"); - return; - } - - cptr=zh->sent_requests.head; - while(cptr){ - fprintf(LOGSTREAM,"%d,",cptr->xid); - cptr=cptr->next; - } - fprintf(LOGSTREAM,"end\n"); -} - //#ifdef THREADED // IO thread queues session events to be processed by the completion thread static int queue_session_event(zhandle_t *zh, int state) From dffbbe1a1281e948fcdf223001ea00712f8a3637 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Sun, 25 Nov 2018 19:50:21 +0000 Subject: [PATCH 15/63] Allow IPv6 to be disabled in C client even if it appears available Tests that use IPv6 will fail inside Docker on a non-Linux host. * C client: allow disabling of IPv6 via --without-ipv6 flag to configure * build.xml: allow disabling of IPv6 in C client via -Ddisable.ipv6=true --- build.xml | 29 ++++++++++++------- .../zookeeper-client-c/configure.ac | 24 +++++++++++---- 2 files changed, 37 insertions(+), 16 deletions(-) diff --git a/build.xml b/build.xml index 2bcc1215266..b59d930f24e 100644 --- a/build.xml +++ b/build.xml @@ -526,6 +526,9 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> + + + @@ -534,6 +537,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> + @@ -1399,17 +1403,20 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - - - - - - - - - - + + + + + + + + + + + + + + diff --git a/zookeeper-client/zookeeper-client-c/configure.ac b/zookeeper-client/zookeeper-client-c/configure.ac index 90a0de37532..43f634a495c 100644 --- a/zookeeper-client/zookeeper-client-c/configure.ac +++ b/zookeeper-client/zookeeper-client-c/configure.ac @@ -99,6 +99,10 @@ AC_ARG_WITH([syncapi], [AS_HELP_STRING([--with-syncapi],[build with support for SyncAPI [default=yes]])], [],[with_syncapi=yes]) +AC_ARG_WITH([ipv6], + [AS_HELP_STRING([--with-ipv6],[build with support for IPv6 [default=yes]])], + [],[with_ipv6=yes]) + # Checks for libraries. AC_CHECK_LIB([pthread], [pthread_mutex_lock],[have_pthread=yes],[have_pthread=no]) @@ -127,7 +131,7 @@ AC_CHECK_TYPE([nfds_t], [AC_DEFINE([POLL_NFDS_TYPE],[unsigned int],[poll() second argument type])], [#include ]) -AC_MSG_CHECKING([whether to enable ipv6]) +AC_MSG_CHECKING([whether IPv6 support is available]) AC_TRY_RUN([ /* is AF_INET6 available? */ #include @@ -140,13 +144,23 @@ main() exit(0); } ], AC_MSG_RESULT(yes) - ipv6=yes, + have_ipv6=yes, AC_MSG_RESULT(no) - ipv6=no, + have_ipv6=no, AC_MSG_RESULT(no) - ipv6=no) + have_ipv6=no) + +if test "x$with_ipv6" != xno && test "x$have_ipv6" = xno; then + AC_MSG_WARN([cannot build with IPv6 support -- IPv6 not found]) + with_ipv6=no +fi +if test "x$with_ipv6" != xno; then + AC_MSG_NOTICE([building with IPv6 support]) +else + AC_MSG_NOTICE([building without IPv6 support]) +fi -if test x"$ipv6" = xyes; then +if test x"$with_ipv6" = xyes; then USEIPV6="-DZOO_IPV6_ENABLED" AC_SUBST(USEIPV6) fi From c6ba28c8a8cbf6c55f8375348a45d02a4294410a Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Sun, 25 Nov 2018 20:47:48 +0000 Subject: [PATCH 16/63] build.xml: configure C client --with-sock-cloexec --- build.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/build.xml b/build.xml index b59d930f24e..cda50ab83f0 100644 --- a/build.xml +++ b/build.xml @@ -537,6 +537,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> + @@ -1414,6 +1415,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> + From 277156c1454ddab93a20e50958725221a93ea384 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Thu, 22 Nov 2018 08:38:28 +0000 Subject: [PATCH 17/63] Add OneLinerFormatter to get semi-verbose logs with captured stdout/stderr --- build.xml | 3 +- ivy.xml | 1 + .../undefined/testing/OneLinerFormatter.java | 345 ++++++++++++++++++ 3 files changed, 348 insertions(+), 1 deletion(-) create mode 100644 zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java diff --git a/build.xml b/build.xml index cda50ab83f0..cf675ff42d1 100644 --- a/build.xml +++ b/build.xml @@ -38,6 +38,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> + @@ -1299,7 +1300,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + diff --git a/ivy.xml b/ivy.xml index 035bb4f7480..91742cb119a 100644 --- a/ivy.xml +++ b/ivy.xml @@ -63,6 +63,7 @@ + diff --git a/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java b/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java new file mode 100644 index 00000000000..c373161e471 --- /dev/null +++ b/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java @@ -0,0 +1,345 @@ +package com.undefined.testing; + +import java.io.BufferedReader; +import java.io.OutputStream; +import java.io.PrintWriter; +import java.io.StringReader; +import java.io.StringWriter; +import java.text.NumberFormat; +import java.util.Hashtable; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import junit.framework.AssertionFailedError; +import junit.framework.Test; + +import org.apache.tools.ant.taskdefs.optional.junit.JUnitResultFormatter; +import org.apache.tools.ant.taskdefs.optional.junit.JUnitTest; +import org.apache.tools.ant.taskdefs.optional.junit.JUnitVersionHelper; + +import org.apache.tools.ant.util.FileUtils; +import org.apache.tools.ant.util.StringUtils; + +/** + * Prints a single lines of tests to a specified Writer. + * Inspired by the BriefJUnitResultFormatter and + * XMLJUnitResultFormatter. + * + * @see FormatterElement + * @see BriefJUnitResultFormatter + * @see XMLJUnitResultFormatter + */ + +public class OneLinerFormatter implements JUnitResultFormatter { + + private final String TAB_STR = " "; + + private final boolean showCausesLines = true; + // (\w+\.)+(\w+)\((\w+).(?:\w+):(\d+)\) + private final Pattern traceLinePattern = Pattern.compile("(\\w+\\.)+(\\w+)\\((\\w+).(?:\\w+):(\\d+)\\)"); + + /** + * Where to write the log to. + */ + private OutputStream out; + + /** + * Used for writing the results. + */ + private PrintWriter output; + + /** + * Used as part of formatting the results. + */ + private StringWriter results; + + /** + * Used for writing formatted results to. + */ + private PrintWriter resultWriter; + + /** + * Formatter for timings. + */ + private NumberFormat numberFormat = NumberFormat.getInstance(); + + /** + * Output suite has written to System.out + */ + private String systemOutput = null; + + /** + * Output suite has written to System.err + */ + private String systemError = null; + + /** + * tests that failed. + */ + private Hashtable failedTests = new Hashtable(); + /** + * Timing helper. + */ + private Hashtable testStarts = new Hashtable(); + + /** + * Constructor for OneLinerFormatter. + */ + public OneLinerFormatter() { + results = new StringWriter(); + resultWriter = new PrintWriter(results); + } + + /** + * Sets the stream the formatter is supposed to write its results to. + * @param out the output stream to write to + */ + public void setOutput(OutputStream out) { + this.out = out; + output = new PrintWriter(out); + } + + /** + * @see JUnitResultFormatter#setSystemOutput(String) + */ + public void setSystemOutput(String out) { + systemOutput = out; + } + + /** + * @see JUnitResultFormatter#setSystemError(String) + */ + public void setSystemError(String err) { + systemError = err; + } + + /** + * The whole testsuite started. + * @param suite the test suite + */ + public void startTestSuite(JUnitTest suite) { + if (output == null) { + return; // Quick return - no output do nothing. + } + StringBuffer sb = new StringBuffer(StringUtils.LINE_SEP); + sb.append("----------------------------------------------------------"); + sb.append(StringUtils.LINE_SEP); + sb.append("Testsuite: "); + sb.append(suite.getName()); + sb.append(StringUtils.LINE_SEP); + output.write(sb.toString()); + output.flush(); + } + + /** + * The whole testsuite ended. + * @param suite the test suite + */ + public void endTestSuite(JUnitTest suite) { + StringBuffer sb = new StringBuffer("Tests run: "); + sb.append(suite.runCount()); + sb.append(", Failures: "); + sb.append(suite.failureCount()); + sb.append(", Errors: "); + sb.append(suite.errorCount()); + sb.append(", Time elapsed: "); + sb.append(numberFormat.format(suite.getRunTime() / 1000.0)); + sb.append(" sec"); + sb.append(StringUtils.LINE_SEP); + sb.append(StringUtils.LINE_SEP); + + // append the err and output streams to the log + if (systemOutput != null && systemOutput.length() > 0) { + sb.append("------------- Standard Output ---------------") + .append(StringUtils.LINE_SEP) + .append(systemOutput) + .append("------------- ---------------- ---------------") + .append(StringUtils.LINE_SEP); + } + + if (systemError != null && systemError.length() > 0) { + sb.append("------------- Standard Error -----------------") + .append(StringUtils.LINE_SEP) + .append(systemError) + .append("------------- ---------------- ---------------") + .append(StringUtils.LINE_SEP); + } + + if (output != null) { + try { + output.write(sb.toString()); + resultWriter.close(); + output.write(results.toString()); + output.flush(); + } finally { + if (out != System.out && out != System.err) { + FileUtils.close(out); + } + } + } + } + + /** + * A test started. + * @param test a test + */ + public void startTest(Test test) { + testStarts.put(test, new Long(System.currentTimeMillis())); + } + + /** + * A test ended. + * @param test a test + */ + public void endTest(Test test) { + // Fix for bug #5637 - if a junit.extensions.TestSetup is + // used and throws an exception during setUp then startTest + // would never have been called + if (!testStarts.containsKey(test)) { + startTest(test); + } + + boolean failed = failedTests.containsKey(test); + + Long l = (Long) testStarts.get(test); + + output.write("Ran ["); + output.write(((System.currentTimeMillis() - l.longValue()) / 1000.0) + "] "); + output.write(getTestName(test) + " ... " + (failed ? "FAILED" : "OK")); + output.write(StringUtils.LINE_SEP); + output.flush(); + } + + /** + * Interface TestListener for JUnit <= 3.4. + * + *

A Test failed. + * @param test a test + * @param t the exception thrown by the test + */ + public void addFailure(Test test, Throwable t) { + formatError("\tFAILED", test, t); + } + + /** + * Interface TestListener for JUnit > 3.4. + * + *

A Test failed. + * @param test a test + * @param t the assertion failed by the test + */ + public void addFailure(Test test, AssertionFailedError t) { + addFailure(test, (Throwable) t); + } + + /** + * A test caused an error. + * @param test a test + * @param error the error thrown by the test + */ + public void addError(Test test, Throwable error) { + formatError("\tCaused an ERROR", test, error); + } + + /** + * Get test name + * + * @param test a test + * @return test name + */ + protected String getTestName(Test test) { + if (test == null) { + return "null"; + } else { + return /* JUnitVersionHelper.getTestCaseClassName(test) + ": " + */ + JUnitVersionHelper.getTestCaseName(test); + } + } + + /** + * Get test case full class name + * + * @param test a test + * @return test full class name + */ + protected String getTestCaseClassName(Test test) { + if (test == null) { + return "null"; + } else { + return JUnitVersionHelper.getTestCaseClassName(test); + } + } + + /** + * Format the test for printing.. + * @param test a test + * @return the formatted testname + */ + protected String formatTest(Test test) { + if (test == null) { + return "Null Test: "; + } else { + return "Testcase: " + test.toString() + ":"; + } + } + + /** + * Format an error and print it. + * @param type the type of error + * @param test the test that failed + * @param error the exception that the test threw + */ + protected synchronized void formatError(String type, Test test, + Throwable error) { + if (test != null) { + failedTests.put(test, test); + endTest(test); + } + + resultWriter.println(formatTest(test) + type); + resultWriter.println(TAB_STR + "(" + error.getClass().getSimpleName() + "): " + + ((error.getMessage() != null) ? error.getMessage() : error)); + + if (showCausesLines) { + // resultWriter.append(StringUtils.LINE_SEP); + resultWriter.println(filterErrorTrace(test, error)); + } + + resultWriter.println(); + + /* String strace = JUnitTestRunner.getFilteredTrace(error); + resultWriter.println(strace); + resultWriter.println(); */ + } + + protected String filterErrorTrace(Test test, Throwable error) { + String trace = StringUtils.getStackTrace(error); + StringWriter sw = new StringWriter(); + PrintWriter pw = new PrintWriter(sw); + StringReader sr = new StringReader(trace); + BufferedReader br = new BufferedReader(sr); + + String line; + try { + while ((line = br.readLine()) != null) { + if (line.indexOf(getTestCaseClassName(test)) != -1) { + Matcher matcher = traceLinePattern.matcher(line); + // pw.println(matcher + ": " + matcher.find()); + if (matcher.find()) { + pw.print(TAB_STR); + pw.print("(" + matcher.group(3) + ") "); + pw.print(matcher.group(2) + ": "); + pw.println(matcher.group(4)); + } else { + pw.println(line); + } + + } + } + } catch (Exception e) { + return trace; // return the treca unfiltered + } + + return sw.toString(); + } +} From 10454d2fa31a731ca94de291e2b154326b146072 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Sun, 25 Nov 2018 22:23:29 +0000 Subject: [PATCH 18/63] ZOOKEEPER-3046: wait for clients to reconnect after restarting server --- .../zookeeper/test/DisconnectedWatcherTest.java | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/DisconnectedWatcherTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/DisconnectedWatcherTest.java index aa65e219f6b..a36349ea059 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/DisconnectedWatcherTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/DisconnectedWatcherTest.java @@ -57,7 +57,8 @@ public void process(WatchedEvent event) { @Test public void testChildWatcherAutoResetWithChroot() throws Exception { - ZooKeeper zk1 = createClient(); + CountdownWatcher watcher1 = new CountdownWatcher(); + ZooKeeper zk1 = createClient(watcher1); zk1.create("/ch1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); @@ -85,6 +86,7 @@ public void testChildWatcherAutoResetWithChroot() throws Exception { watcher.waitForDisconnected(3000); startServer(); watcher.waitForConnected(3000); + watcher1.waitForConnected(3000); // this should trigger the watch zk1.create("/ch1/youshouldmatter2", null, Ids.OPEN_ACL_UNSAFE, @@ -97,7 +99,8 @@ public void testChildWatcherAutoResetWithChroot() throws Exception { @Test public void testDefaultWatcherAutoResetWithChroot() throws Exception { - ZooKeeper zk1 = createClient(); + CountdownWatcher watcher1 = new CountdownWatcher(); + ZooKeeper zk1 = createClient(watcher1); zk1.create("/ch1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); @@ -124,6 +127,7 @@ public void testDefaultWatcherAutoResetWithChroot() throws Exception { watcher.waitForDisconnected(3000); startServer(); watcher.waitForConnected(3000); + watcher1.waitForConnected(3000); // this should trigger the watch zk1.create("/ch1/youshouldmatter2", null, Ids.OPEN_ACL_UNSAFE, @@ -136,7 +140,8 @@ public void testDefaultWatcherAutoResetWithChroot() throws Exception { @Test public void testDeepChildWatcherAutoResetWithChroot() throws Exception { - ZooKeeper zk1 = createClient(); + CountdownWatcher watcher1 = new CountdownWatcher(); + ZooKeeper zk1 = createClient(watcher1); zk1.create("/ch1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); @@ -166,6 +171,7 @@ public void testDeepChildWatcherAutoResetWithChroot() throws Exception { watcher.waitForDisconnected(3000); startServer(); watcher.waitForConnected(3000); + watcher1.waitForConnected(3000); // this should trigger the watch zk1.create("/ch1/here/we/are/again", null, Ids.OPEN_ACL_UNSAFE, @@ -180,7 +186,8 @@ public void testDeepChildWatcherAutoResetWithChroot() throws Exception { // watches which require multiple SetWatches calls. @Test(timeout = 840000) public void testManyChildWatchersAutoReset() throws Exception { - ZooKeeper zk1 = createClient(); + CountdownWatcher watcher1 = new CountdownWatcher(); + ZooKeeper zk1 = createClient(watcher1); MyWatcher watcher = new MyWatcher(); ZooKeeper zk2 = createClient(watcher); @@ -221,6 +228,7 @@ public void testManyChildWatchersAutoReset() throws Exception { watcher.waitForDisconnected(30000); startServer(); watcher.waitForConnected(30000); + watcher1.waitForConnected(30000); // Trigger the watches and ensure they properly propagate to the client i = 0; From f82707d1f1501ccf7cf4b03f9d112265c92a7290 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Mon, 26 Nov 2018 06:13:06 +0000 Subject: [PATCH 19/63] ZOOKEEPER-3046: factor common setup in DisconnectedWatcherTest tests to a setUp() method --- .../test/DisconnectedWatcherTest.java | 72 ++++++++++--------- 1 file changed, 40 insertions(+), 32 deletions(-) diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/DisconnectedWatcherTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/DisconnectedWatcherTest.java index a36349ea059..6f11fafa66c 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/DisconnectedWatcherTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/DisconnectedWatcherTest.java @@ -28,7 +28,9 @@ import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.Watcher.Event.EventType; import org.apache.zookeeper.ZooDefs.Ids; +import org.junit.After; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,18 +55,36 @@ public void process(WatchedEvent event) { } } + private CountdownWatcher watcher1; + private ZooKeeper zk1; + private MyWatcher watcher2; + private ZooKeeper zk2; + + @Before + public void setUp() throws Exception { + watcher1 = new CountdownWatcher(); + zk1 = createClient(watcher1); + watcher2 = new MyWatcher(); + } + + @After + public void tearDown() throws Exception { + if (zk2 != null) { + zk2.close(); + } + if (zk1 != null) { + zk1.close(); + } + } + // @see jira issue ZOOKEEPER-961 @Test public void testChildWatcherAutoResetWithChroot() throws Exception { - CountdownWatcher watcher1 = new CountdownWatcher(); - ZooKeeper zk1 = createClient(watcher1); - zk1.create("/ch1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - MyWatcher watcher = new MyWatcher(); - ZooKeeper zk2 = createClient(watcher, hostPort + "/ch1"); + zk2 = createClient(watcher2, hostPort + "/ch1"); zk2.getChildren("/", true ); // this call shouldn't trigger any error or watch @@ -74,7 +94,7 @@ public void testChildWatcherAutoResetWithChroot() throws Exception { // this should trigger the watch zk1.create("/ch1/youshouldmatter1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - WatchedEvent e = watcher.events.poll(TIMEOUT, TimeUnit.MILLISECONDS); + WatchedEvent e = watcher2.events.poll(TIMEOUT, TimeUnit.MILLISECONDS); Assert.assertNotNull(e); Assert.assertEquals(EventType.NodeChildrenChanged, e.getType()); Assert.assertEquals("/", e.getPath()); @@ -83,9 +103,9 @@ public void testChildWatcherAutoResetWithChroot() throws Exception { zk2.getChildren("/", childWatcher); stopServer(); - watcher.waitForDisconnected(3000); + watcher2.waitForDisconnected(3000); startServer(); - watcher.waitForConnected(3000); + watcher2.waitForConnected(3000); watcher1.waitForConnected(3000); // this should trigger the watch @@ -99,14 +119,10 @@ public void testChildWatcherAutoResetWithChroot() throws Exception { @Test public void testDefaultWatcherAutoResetWithChroot() throws Exception { - CountdownWatcher watcher1 = new CountdownWatcher(); - ZooKeeper zk1 = createClient(watcher1); - zk1.create("/ch1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - MyWatcher watcher = new MyWatcher(); - ZooKeeper zk2 = createClient(watcher, hostPort + "/ch1"); + zk2 = createClient(watcher2, hostPort + "/ch1"); zk2.getChildren("/", true ); // this call shouldn't trigger any error or watch @@ -116,7 +132,7 @@ public void testDefaultWatcherAutoResetWithChroot() throws Exception { // this should trigger the watch zk1.create("/ch1/youshouldmatter1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - WatchedEvent e = watcher.events.poll(TIMEOUT, TimeUnit.MILLISECONDS); + WatchedEvent e = watcher2.events.poll(TIMEOUT, TimeUnit.MILLISECONDS); Assert.assertNotNull(e); Assert.assertEquals(EventType.NodeChildrenChanged, e.getType()); Assert.assertEquals("/", e.getPath()); @@ -124,15 +140,15 @@ public void testDefaultWatcherAutoResetWithChroot() throws Exception { zk2.getChildren("/", true ); stopServer(); - watcher.waitForDisconnected(3000); + watcher2.waitForDisconnected(3000); startServer(); - watcher.waitForConnected(3000); + watcher2.waitForConnected(3000); watcher1.waitForConnected(3000); // this should trigger the watch zk1.create("/ch1/youshouldmatter2", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - e = watcher.events.poll(TIMEOUT, TimeUnit.MILLISECONDS); + e = watcher2.events.poll(TIMEOUT, TimeUnit.MILLISECONDS); Assert.assertNotNull(e); Assert.assertEquals(EventType.NodeChildrenChanged, e.getType()); Assert.assertEquals("/", e.getPath()); @@ -140,9 +156,6 @@ public void testDefaultWatcherAutoResetWithChroot() throws Exception { @Test public void testDeepChildWatcherAutoResetWithChroot() throws Exception { - CountdownWatcher watcher1 = new CountdownWatcher(); - ZooKeeper zk1 = createClient(watcher1); - zk1.create("/ch1", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); zk1.create("/ch1/here", null, Ids.OPEN_ACL_UNSAFE, @@ -152,14 +165,13 @@ public void testDeepChildWatcherAutoResetWithChroot() throws Exception { zk1.create("/ch1/here/we/are", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - MyWatcher watcher = new MyWatcher(); - ZooKeeper zk2 = createClient(watcher, hostPort + "/ch1/here/we"); + zk2 = createClient(watcher2, hostPort + "/ch1/here/we"); zk2.getChildren("/are", true ); // this should trigger the watch zk1.create("/ch1/here/we/are/now", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - WatchedEvent e = watcher.events.poll(TIMEOUT, TimeUnit.MILLISECONDS); + WatchedEvent e = watcher2.events.poll(TIMEOUT, TimeUnit.MILLISECONDS); Assert.assertNotNull(e); Assert.assertEquals(EventType.NodeChildrenChanged, e.getType()); Assert.assertEquals("/are", e.getPath()); @@ -168,9 +180,9 @@ public void testDeepChildWatcherAutoResetWithChroot() throws Exception { zk2.getChildren("/are", childWatcher); stopServer(); - watcher.waitForDisconnected(3000); + watcher2.waitForDisconnected(3000); startServer(); - watcher.waitForConnected(3000); + watcher2.waitForConnected(3000); watcher1.waitForConnected(3000); // this should trigger the watch @@ -186,11 +198,7 @@ public void testDeepChildWatcherAutoResetWithChroot() throws Exception { // watches which require multiple SetWatches calls. @Test(timeout = 840000) public void testManyChildWatchersAutoReset() throws Exception { - CountdownWatcher watcher1 = new CountdownWatcher(); - ZooKeeper zk1 = createClient(watcher1); - - MyWatcher watcher = new MyWatcher(); - ZooKeeper zk2 = createClient(watcher); + zk2 = createClient(watcher2); // 110 character base path String pathBase = "/long-path-000000000-111111111-222222222-333333333-444444444-" @@ -225,9 +233,9 @@ public void testManyChildWatchersAutoReset() throws Exception { } stopServer(); - watcher.waitForDisconnected(30000); + watcher2.waitForDisconnected(30000); startServer(); - watcher.waitForConnected(30000); + watcher2.waitForConnected(30000); watcher1.waitForConnected(30000); // Trigger the watches and ensure they properly propagate to the client From c948adfde4c56fc9c7752f2c4008c4bfeb4e3517 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Mon, 26 Nov 2018 06:31:23 +0000 Subject: [PATCH 20/63] ZOOKEEPER-3202: Add timing margin to improve reliability of testClientServerSSL() --- .../test/java/org/apache/zookeeper/test/ClientSSLTest.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java index 08ffb4eef58..b5c5d4dec74 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java @@ -93,6 +93,10 @@ public void testClientServerSSL() throws Exception { mt[i].start(); } + // Add some timing margin for the quorum to elect a leader + // (without this margin, timeouts have been observed in parallel test runs) + ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[0], 2 * TIMEOUT); + // Servers have been set up. Now go test if secure connection is successful. for (int i = 0; i < SERVER_COUNT; i++) { Assert.assertTrue("waiting for server " + i + " being up", From e38fc681b17263835d2e738bea9ce2e5e4075026 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Mon, 26 Nov 2018 07:00:48 +0000 Subject: [PATCH 21/63] ZOOKEEPER-3046: call setUp/tearDown from ClientBase in the setUp/tearDown of the subclass --- .../java/org/apache/zookeeper/test/DisconnectedWatcherTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/DisconnectedWatcherTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/DisconnectedWatcherTest.java index 6f11fafa66c..f459919edf3 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/DisconnectedWatcherTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/DisconnectedWatcherTest.java @@ -62,6 +62,7 @@ public void process(WatchedEvent event) { @Before public void setUp() throws Exception { + super.setUp(); watcher1 = new CountdownWatcher(); zk1 = createClient(watcher1); watcher2 = new MyWatcher(); @@ -75,6 +76,7 @@ public void tearDown() throws Exception { if (zk1 != null) { zk1.close(); } + super.tearDown(); } // @see jira issue ZOOKEEPER-961 From 0974f67a18e8bb5beffd08a22ca08b7eb3cdda59 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Mon, 26 Nov 2018 23:50:18 +0000 Subject: [PATCH 22/63] QuorumPeerMainTest: 90-second timeout on testBadPeerAddressInQuorum --- .../org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 57d3df8f5cc..72fa13c1e96 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 @@ -562,7 +562,7 @@ private Servers LaunchServers(int numServers, Integer tickTime) throws IOExcepti /** * Verify handling of bad quorum address */ - @Test + @Test(timeout = 90000) public void testBadPeerAddressInQuorum() throws Exception { ClientBase.setupTestEnv(); From 57e7ef0cbe6d47cabf9a5ab1489330ca255000eb Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Tue, 27 Nov 2018 00:20:05 +0000 Subject: [PATCH 23/63] let the test run take as much as 60 minutes (instead of 15) --- build.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.xml b/build.xml index cf675ff42d1..e5bd1dd3d0b 100644 --- a/build.xml +++ b/build.xml @@ -136,7 +136,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + From 3e5b1bd1d96a0f4f1b2410202549f23c3aa3da82 Mon Sep 17 00:00:00 2001 From: Eric Schwartz Date: Tue, 27 Nov 2018 10:59:37 -0700 Subject: [PATCH 24/63] Adding PID to test runs --- .../undefined/testing/OneLinerFormatter.java | 131 ++++++++++-------- 1 file changed, 71 insertions(+), 60 deletions(-) diff --git a/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java b/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java index c373161e471..854e09c5b55 100644 --- a/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java +++ b/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java @@ -1,5 +1,7 @@ package com.undefined.testing; - + +import java.lang.management.ManagementFactory; + import java.io.BufferedReader; import java.io.OutputStream; import java.io.PrintWriter; @@ -9,17 +11,17 @@ import java.util.Hashtable; import java.util.regex.Matcher; import java.util.regex.Pattern; - + import junit.framework.AssertionFailedError; import junit.framework.Test; - + import org.apache.tools.ant.taskdefs.optional.junit.JUnitResultFormatter; import org.apache.tools.ant.taskdefs.optional.junit.JUnitTest; import org.apache.tools.ant.taskdefs.optional.junit.JUnitVersionHelper; - + import org.apache.tools.ant.util.FileUtils; import org.apache.tools.ant.util.StringUtils; - + /** * Prints a single lines of tests to a specified Writer. * Inspired by the BriefJUnitResultFormatter and @@ -29,50 +31,50 @@ * @see BriefJUnitResultFormatter * @see XMLJUnitResultFormatter */ - + public class OneLinerFormatter implements JUnitResultFormatter { - + private final String TAB_STR = " "; - + private final boolean showCausesLines = true; // (\w+\.)+(\w+)\((\w+).(?:\w+):(\d+)\) private final Pattern traceLinePattern = Pattern.compile("(\\w+\\.)+(\\w+)\\((\\w+).(?:\\w+):(\\d+)\\)"); - + /** * Where to write the log to. */ private OutputStream out; - + /** * Used for writing the results. */ private PrintWriter output; - + /** * Used as part of formatting the results. */ private StringWriter results; - + /** * Used for writing formatted results to. */ private PrintWriter resultWriter; - + /** * Formatter for timings. */ private NumberFormat numberFormat = NumberFormat.getInstance(); - + /** * Output suite has written to System.out */ private String systemOutput = null; - + /** * Output suite has written to System.err */ private String systemError = null; - + /** * tests that failed. */ @@ -81,7 +83,7 @@ public class OneLinerFormatter implements JUnitResultFormatter { * Timing helper. */ private Hashtable testStarts = new Hashtable(); - + /** * Constructor for OneLinerFormatter. */ @@ -89,7 +91,7 @@ public OneLinerFormatter() { results = new StringWriter(); resultWriter = new PrintWriter(results); } - + /** * Sets the stream the formatter is supposed to write its results to. * @param out the output stream to write to @@ -98,21 +100,21 @@ public void setOutput(OutputStream out) { this.out = out; output = new PrintWriter(out); } - + /** * @see JUnitResultFormatter#setSystemOutput(String) */ public void setSystemOutput(String out) { systemOutput = out; } - + /** * @see JUnitResultFormatter#setSystemError(String) */ public void setSystemError(String err) { systemError = err; } - + /** * The whole testsuite started. * @param suite the test suite @@ -123,20 +125,23 @@ public void startTestSuite(JUnitTest suite) { } StringBuffer sb = new StringBuffer(StringUtils.LINE_SEP); sb.append("----------------------------------------------------------"); - sb.append(StringUtils.LINE_SEP); + sb.append(StringUtils.LINE_SEP); sb.append("Testsuite: "); sb.append(suite.getName()); + sb.append("In process: "); + sb.append(ManagementFactory.getRuntimeMXBean().getName()); sb.append(StringUtils.LINE_SEP); output.write(sb.toString()); output.flush(); } - + /** * The whole testsuite ended. * @param suite the test suite */ public void endTestSuite(JUnitTest suite) { StringBuffer sb = new StringBuffer("Tests run: "); + String runtimeName = ManagementFactory.getRuntimeMXBean().getName(); sb.append(suite.runCount()); sb.append(", Failures: "); sb.append(suite.failureCount()); @@ -145,26 +150,32 @@ public void endTestSuite(JUnitTest suite) { sb.append(", Time elapsed: "); sb.append(numberFormat.format(suite.getRunTime() / 1000.0)); sb.append(" sec"); + sb.append("In process: "); + sb.append(ManagementFactory.getRuntimeMXBean().getName()); sb.append(StringUtils.LINE_SEP); sb.append(StringUtils.LINE_SEP); - + // append the err and output streams to the log if (systemOutput != null && systemOutput.length() > 0) { - sb.append("------------- Standard Output ---------------") + sb.append(runtimeName) + .append("------------- Standard Output ---------------") .append(StringUtils.LINE_SEP) .append(systemOutput) + .append(runtimeName) .append("------------- ---------------- ---------------") .append(StringUtils.LINE_SEP); } - + if (systemError != null && systemError.length() > 0) { - sb.append("------------- Standard Error -----------------") + sb.append(runtimeName) + .append("------------- Standard Error -----------------") .append(StringUtils.LINE_SEP) .append(systemError) + .append(runtimeName) .append("------------- ---------------- ---------------") .append(StringUtils.LINE_SEP); } - + if (output != null) { try { output.write(sb.toString()); @@ -178,7 +189,7 @@ public void endTestSuite(JUnitTest suite) { } } } - + /** * A test started. * @param test a test @@ -186,7 +197,7 @@ public void endTestSuite(JUnitTest suite) { public void startTest(Test test) { testStarts.put(test, new Long(System.currentTimeMillis())); } - + /** * A test ended. * @param test a test @@ -198,18 +209,18 @@ public void endTest(Test test) { if (!testStarts.containsKey(test)) { startTest(test); } - + boolean failed = failedTests.containsKey(test); - + Long l = (Long) testStarts.get(test); - + output.write("Ran ["); output.write(((System.currentTimeMillis() - l.longValue()) / 1000.0) + "] "); output.write(getTestName(test) + " ... " + (failed ? "FAILED" : "OK")); output.write(StringUtils.LINE_SEP); output.flush(); } - + /** * Interface TestListener for JUnit <= 3.4. * @@ -220,7 +231,7 @@ public void endTest(Test test) { public void addFailure(Test test, Throwable t) { formatError("\tFAILED", test, t); } - + /** * Interface TestListener for JUnit > 3.4. * @@ -231,7 +242,7 @@ public void addFailure(Test test, Throwable t) { public void addFailure(Test test, AssertionFailedError t) { addFailure(test, (Throwable) t); } - + /** * A test caused an error. * @param test a test @@ -240,10 +251,10 @@ public void addFailure(Test test, AssertionFailedError t) { public void addError(Test test, Throwable error) { formatError("\tCaused an ERROR", test, error); } - + /** - * Get test name - * + * Get test name + * * @param test a test * @return test name */ @@ -255,12 +266,12 @@ protected String getTestName(Test test) { JUnitVersionHelper.getTestCaseName(test); } } - + /** - * Get test case full class name - * + * Get test case full class name + * * @param test a test - * @return test full class name + * @return test full class name */ protected String getTestCaseClassName(Test test) { if (test == null) { @@ -268,8 +279,8 @@ protected String getTestCaseClassName(Test test) { } else { return JUnitVersionHelper.getTestCaseClassName(test); } - } - + } + /** * Format the test for printing.. * @param test a test @@ -282,7 +293,7 @@ protected String formatTest(Test test) { return "Testcase: " + test.toString() + ":"; } } - + /** * Format an error and print it. * @param type the type of error @@ -293,32 +304,32 @@ protected synchronized void formatError(String type, Test test, Throwable error) { if (test != null) { failedTests.put(test, test); - endTest(test); + endTest(test); } - + resultWriter.println(formatTest(test) + type); - resultWriter.println(TAB_STR + "(" + error.getClass().getSimpleName() + "): " + + resultWriter.println(TAB_STR + "(" + error.getClass().getSimpleName() + "): " + ((error.getMessage() != null) ? error.getMessage() : error)); - + if (showCausesLines) { // resultWriter.append(StringUtils.LINE_SEP); resultWriter.println(filterErrorTrace(test, error)); } - + resultWriter.println(); - + /* String strace = JUnitTestRunner.getFilteredTrace(error); resultWriter.println(strace); resultWriter.println(); */ } - + protected String filterErrorTrace(Test test, Throwable error) { String trace = StringUtils.getStackTrace(error); StringWriter sw = new StringWriter(); - PrintWriter pw = new PrintWriter(sw); + PrintWriter pw = new PrintWriter(sw); StringReader sr = new StringReader(trace); - BufferedReader br = new BufferedReader(sr); - + BufferedReader br = new BufferedReader(sr); + String line; try { while ((line = br.readLine()) != null) { @@ -327,19 +338,19 @@ protected String filterErrorTrace(Test test, Throwable error) { // pw.println(matcher + ": " + matcher.find()); if (matcher.find()) { pw.print(TAB_STR); - pw.print("(" + matcher.group(3) + ") "); + pw.print("(" + matcher.group(3) + ") "); pw.print(matcher.group(2) + ": "); pw.println(matcher.group(4)); } else { pw.println(line); } - + } } } catch (Exception e) { return trace; // return the treca unfiltered } - + return sw.toString(); - } + } } From c1a2603a567b6167c6e59c9cc756a88dac208b67 Mon Sep 17 00:00:00 2001 From: Eric Schwartz Date: Tue, 27 Nov 2018 12:00:52 -0700 Subject: [PATCH 25/63] Prefix stdout and stderr with process ID --- .../undefined/testing/OneLinerFormatter.java | 24 ++++++++++++++----- 1 file changed, 18 insertions(+), 6 deletions(-) diff --git a/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java b/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java index 854e09c5b55..4a484da240c 100644 --- a/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java +++ b/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java @@ -8,10 +8,11 @@ import java.io.StringReader; import java.io.StringWriter; import java.text.NumberFormat; +import java.util.Arrays; import java.util.Hashtable; import java.util.regex.Matcher; import java.util.regex.Pattern; - +import java.util.stream.Collectors; import junit.framework.AssertionFailedError; import junit.framework.Test; @@ -128,20 +129,29 @@ public void startTestSuite(JUnitTest suite) { sb.append(StringUtils.LINE_SEP); sb.append("Testsuite: "); sb.append(suite.getName()); - sb.append("In process: "); + sb.append(" In process: "); sb.append(ManagementFactory.getRuntimeMXBean().getName()); sb.append(StringUtils.LINE_SEP); output.write(sb.toString()); output.flush(); } + public String prefixLines(String prefix, String buffer) { + String lines[] = buffer.split("\\r?\\n"); + return Arrays.asList(lines) + .stream() + .map(l -> prefix + l) + .collect(Collectors.joining("\n")); + } + + /** * The whole testsuite ended. * @param suite the test suite */ public void endTestSuite(JUnitTest suite) { StringBuffer sb = new StringBuffer("Tests run: "); - String runtimeName = ManagementFactory.getRuntimeMXBean().getName(); + String runtimeName = " [" + ManagementFactory.getRuntimeMXBean().getName() + "] "; sb.append(suite.runCount()); sb.append(", Failures: "); sb.append(suite.failureCount()); @@ -150,7 +160,7 @@ public void endTestSuite(JUnitTest suite) { sb.append(", Time elapsed: "); sb.append(numberFormat.format(suite.getRunTime() / 1000.0)); sb.append(" sec"); - sb.append("In process: "); + sb.append(" In process: "); sb.append(ManagementFactory.getRuntimeMXBean().getName()); sb.append(StringUtils.LINE_SEP); sb.append(StringUtils.LINE_SEP); @@ -160,7 +170,8 @@ public void endTestSuite(JUnitTest suite) { sb.append(runtimeName) .append("------------- Standard Output ---------------") .append(StringUtils.LINE_SEP) - .append(systemOutput) + .append(prefixLines(runtimeName, systemOutput)) + .append(StringUtils.LINE_SEP) .append(runtimeName) .append("------------- ---------------- ---------------") .append(StringUtils.LINE_SEP); @@ -170,7 +181,8 @@ public void endTestSuite(JUnitTest suite) { sb.append(runtimeName) .append("------------- Standard Error -----------------") .append(StringUtils.LINE_SEP) - .append(systemError) + .append(prefixLines(runtimeName, systemError)) + .append(StringUtils.LINE_SEP) .append(runtimeName) .append("------------- ---------------- ---------------") .append(StringUtils.LINE_SEP); From e02eb705c6550f51ebb860a474ce711ec68c7a24 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Tue, 27 Nov 2018 16:31:52 -0800 Subject: [PATCH 26/63] improve consistency of setUp()/tearDown() methods in tests --- .../server/jersey/RestTestSuite.java | 6 +-- .../org/apache/zookeeper/SaslAuthTest.java | 4 +- .../apache/zookeeper/ServerConfigTest.java | 4 +- .../zookeeper/client/ZKClientConfigTest.java | 2 +- .../common/AtomicFileWritingIdiomTest.java | 2 +- .../apache/zookeeper/common/X509UtilTest.java | 2 +- .../zookeeper/common/ZKTrustManagerTest.java | 2 +- .../server/ByteBufferInputStreamTest.java | 42 ++++++++++++------- .../zookeeper/server/EphemeralTypeTest.java | 4 +- .../server/NIOServerCnxnFactoryTest.java | 4 +- .../server/PrepRequestProcessorTest.java | 6 ++- .../apache/zookeeper/server/PurgeTxnTest.java | 2 +- .../zookeeper/server/ServerStatsTest.java | 2 +- .../server/WatchesPathReportTest.java | 12 ++++-- .../zookeeper/server/WatchesReportTest.java | 12 ++++-- .../zookeeper/server/WatchesSummaryTest.java | 9 ++-- .../server/ZooKeeperServerBeanTest.java | 4 +- .../server/ZooKeeperServerConfTest.java | 9 ++-- .../server/admin/CommandResponseTest.java | 9 ++-- .../server/admin/JettyAdminServerTest.java | 4 +- .../server/persistence/TxnLogToolkitTest.java | 4 +- .../server/quorum/CommitProcessorTest.java | 8 ++-- .../quorum/EphemeralNodeDeletionTest.java | 2 +- .../quorum/FuzzySnapshotRelatedTest.java | 2 +- .../server/quorum/LeaderBeanTest.java | 4 +- .../server/quorum/LeaderWithObserverTest.java | 2 +- .../server/quorum/QuorumSSLTest.java | 4 +- .../server/quorum/RaceConditionTest.java | 2 +- .../server/quorum/ReconfigBackupTest.java | 4 +- .../quorum/ReconfigDuringLeaderSyncTest.java | 4 +- .../quorum/ReconfigFailureCasesTest.java | 2 +- .../server/quorum/ReconfigLegacyTest.java | 2 +- .../server/quorum/ReconfigRecoveryTest.java | 4 +- .../server/quorum/StatCommandTest.java | 2 +- .../server/quorum/StatResetCommandTest.java | 2 +- .../server/quorum/WatchLeakTest.java | 2 +- .../zookeeper/server/quorum/Zab1_0Test.java | 2 +- .../quorum/auth/QuorumAuthUpgradeTest.java | 2 +- .../quorum/auth/QuorumDigestAuthTest.java | 2 +- .../quorum/auth/QuorumKerberosAuthTest.java | 2 +- .../auth/QuorumKerberosHostBasedAuthTest.java | 4 +- .../server/util/VerifyingFileFactoryTest.java | 2 +- .../zookeeper/test/AsyncHammerTest.java | 6 +-- .../test/AtomicFileOutputStreamTest.java | 4 +- .../apache/zookeeper/test/BufferSizeTest.java | 3 +- .../apache/zookeeper/test/ClientSSLTest.java | 4 +- .../zookeeper/test/ClientSkipACLTest.java | 6 +-- .../apache/zookeeper/test/CnxManagerTest.java | 2 +- .../zookeeper/test/NettyNettySuiteBase.java | 4 +- .../zookeeper/test/NioNettySuiteBase.java | 4 +- .../apache/zookeeper/test/OSMXBeanTest.java | 2 +- .../apache/zookeeper/test/ObserverLETest.java | 4 +- .../zookeeper/test/ReconfigExceptionTest.java | 2 +- .../zookeeper/test/ReconfigMisconfigTest.java | 2 +- .../apache/zookeeper/test/ReconfigTest.java | 4 +- .../apache/zookeeper/test/SSLAuthTest.java | 5 ++- .../apache/zookeeper/test/SaslClientTest.java | 4 +- .../zookeeper/test/SaslSuperUserTest.java | 4 +- .../apache/zookeeper/test/StandaloneTest.java | 4 +- .../apache/zookeeper/test/TruncateTest.java | 4 +- .../test/WatchEventWhenAutoResetTest.java | 2 +- .../apache/zookeeper/test/X509AuthTest.java | 2 +- 62 files changed, 156 insertions(+), 121 deletions(-) diff --git a/zookeeper-contrib/zookeeper-contrib-rest/src/test/java/org/apache/zookeeper/server/jersey/RestTestSuite.java b/zookeeper-contrib/zookeeper-contrib-rest/src/test/java/org/apache/zookeeper/server/jersey/RestTestSuite.java index fc69cafee8f..f7a950fd1b0 100644 --- a/zookeeper-contrib/zookeeper-contrib-rest/src/test/java/org/apache/zookeeper/server/jersey/RestTestSuite.java +++ b/zookeeper-contrib/zookeeper-contrib-rest/src/test/java/org/apache/zookeeper/server/jersey/RestTestSuite.java @@ -30,13 +30,13 @@ public class RestTestSuite { @BeforeClass - public static void setUp() { + public static void setUpSuite() throws Exception { // suite setup } @AfterClass - public static void tearDown() { - // suite setup + public static void tearDownSuite() throws Exception { + // suite teardown } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/SaslAuthTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/SaslAuthTest.java index 088fe1f0275..0220f494c8c 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/SaslAuthTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/SaslAuthTest.java @@ -43,7 +43,7 @@ public class SaslAuthTest extends ClientBase { @BeforeClass - public static void init() { + public static void setUpSasl() throws Exception { System.setProperty("zookeeper.authProvider.1", "org.apache.zookeeper.server.auth.SASLAuthenticationProvider"); try { @@ -85,7 +85,7 @@ private static String getJaasFileContent() { } @AfterClass - public static void clean() { + public static void tearDownSasl() throws Exception { System.clearProperty("zookeeper.authProvider.1"); System.clearProperty("java.security.auth.login.config"); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/ServerConfigTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/ServerConfigTest.java index 27faa745b6f..ffa7254fe69 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/ServerConfigTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/ServerConfigTest.java @@ -33,7 +33,7 @@ public class ServerConfigTest { private ServerConfig serverConfig; @Before - public void setUp() { + public void setUp() throws Exception { serverConfig = new ServerConfig(); } @@ -71,4 +71,4 @@ boolean checkEquality(String a, File b) { assertNotNull(b); return new File(a).equals(b); } -} \ No newline at end of file +} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/client/ZKClientConfigTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/client/ZKClientConfigTest.java index 98f7c51bc96..6b6e504eca6 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/client/ZKClientConfigTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/client/ZKClientConfigTest.java @@ -52,7 +52,7 @@ public class ZKClientConfigTest { public Timeout timeout = new Timeout(10, TimeUnit.SECONDS); @BeforeClass - public static void init() { + public static void setUpTestDir() throws Exception { if (!testData.exists()) { testData.mkdirs(); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/common/AtomicFileWritingIdiomTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/common/AtomicFileWritingIdiomTest.java index 477ac8e3331..6615e889d02 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/common/AtomicFileWritingIdiomTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/common/AtomicFileWritingIdiomTest.java @@ -39,7 +39,7 @@ public class AtomicFileWritingIdiomTest extends ZKTestCase { private static File tmpdir; @BeforeClass - public static void createTmpDir() { + public static void createTmpDir() throws Exception { tmpdir = new File("build/test/tmp"); tmpdir.mkdirs(); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/common/X509UtilTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/common/X509UtilTest.java index 6b343c32e78..38512da6880 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/common/X509UtilTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/common/X509UtilTest.java @@ -77,7 +77,7 @@ public void setUp() throws Exception { } @After - public void cleanUp() { + public void tearDown() throws Exception { x509TestContext.clearSystemProperties(x509Util); System.clearProperty(x509Util.getSslOcspEnabledProperty()); System.clearProperty(x509Util.getSslCrlEnabledProperty()); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/common/ZKTrustManagerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/common/ZKTrustManagerTest.java index 32f250b4153..b9436dfaad5 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/common/ZKTrustManagerTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/common/ZKTrustManagerTest.java @@ -83,7 +83,7 @@ public static void removeBouncyCastleProvider() throws Exception { } @Before - public void setup() throws Exception { + public void setUp() throws Exception { mockX509ExtendedTrustManager = mock(X509ExtendedTrustManager.class); mockInetAddress = mock(InetAddress.class); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/ByteBufferInputStreamTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/ByteBufferInputStreamTest.java index 80c20e1f8b9..3288d90ef22 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/ByteBufferInputStreamTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/ByteBufferInputStreamTest.java @@ -31,7 +31,8 @@ public class ByteBufferInputStreamTest extends ZKTestCase { "Apache ZooKeeper".getBytes(Charset.forName("UTF-8")); private static byte[] DATA_BYTES; - @BeforeClass public static void setUpClass() { + @BeforeClass + public static void setUpDataBytes() throws Exception { int len = DATA_BYTES_0.length + 2; DATA_BYTES = new byte[len]; System.arraycopy(DATA_BYTES_0, 0, DATA_BYTES, 0, DATA_BYTES_0.length); @@ -42,20 +43,23 @@ public class ByteBufferInputStreamTest extends ZKTestCase { private ByteBuffer bb; private ByteBufferInputStream in; private byte[] bs; - @Before public void setUp() throws Exception { + @Before + public void setUp() throws Exception { bb = ByteBuffer.wrap(DATA_BYTES); in = new ByteBufferInputStream(bb); bs = new byte[] { (byte) 1, (byte) 2, (byte) 3, (byte) 4 }; } - @Test public void testRead() throws Exception { + @Test + public void testRead() throws Exception { for (int i = 0; i < DATA_BYTES.length; i++) { int b = in.read(); assertEquals(DATA_BYTES[i], (byte) b); } assertEquals(-1, in.read()); } - @Test public void testReadArrayOffsetLength() throws Exception { + @Test + public void testReadArrayOffsetLength() throws Exception { assertEquals(1, in.read(bs, 2, 1)); byte[] expected = new byte[] { (byte) 1, (byte) 2, DATA_BYTES[0], (byte) 4 }; @@ -65,7 +69,8 @@ public class ByteBufferInputStreamTest extends ZKTestCase { public void testReadArrayOffsetLength_LengthTooLarge() throws Exception { in.read(bs, 2, 3); } - @Test public void testReadArrayOffsetLength_HitEndOfStream() + @Test + public void testReadArrayOffsetLength_HitEndOfStream() throws Exception { for (int i = 0; i < DATA_BYTES.length - 1; i++) { in.read(); @@ -76,7 +81,8 @@ public void testReadArrayOffsetLength_LengthTooLarge() throws Exception { (byte) 4 }; assertArrayEquals(expected, bs); } - @Test public void testReadArrayOffsetLength_AtEndOfStream() + @Test + public void testReadArrayOffsetLength_AtEndOfStream() throws Exception { for (int i = 0; i < DATA_BYTES.length; i++) { in.read(); @@ -85,24 +91,28 @@ public void testReadArrayOffsetLength_LengthTooLarge() throws Exception { assertEquals(-1, in.read(bs, 2, 2)); assertArrayEquals(expected, bs); } - @Test public void testReadArrayOffsetLength_0Length() throws Exception { + @Test + public void testReadArrayOffsetLength_0Length() throws Exception { byte[] expected = Arrays.copyOf(bs, bs.length); assertEquals(0, in.read(bs, 2, 0)); assertArrayEquals(expected, bs); } - @Test public void testReadArray() throws Exception { + @Test + public void testReadArray() throws Exception { byte[] expected = Arrays.copyOf(DATA_BYTES, 4); assertEquals(4, in.read(bs)); assertArrayEquals(expected, bs); } - @Test public void testSkip() throws Exception { + @Test + public void testSkip() throws Exception { in.read(); assertEquals(2L, in.skip(2L)); assertEquals(DATA_BYTES[3], in.read()); assertEquals(DATA_BYTES[4], in.read()); } - @Test public void testSkip2() throws Exception { + @Test + public void testSkip2() throws Exception { for (int i = 0; i < DATA_BYTES.length / 2; i++) { in.read(); } @@ -112,20 +122,23 @@ public void testReadArrayOffsetLength_LengthTooLarge() throws Exception { assertEquals(DATA_BYTES[idx++], in.read()); assertEquals(DATA_BYTES[idx++], in.read()); } - @Test public void testNegativeSkip() throws Exception { + @Test + public void testNegativeSkip() throws Exception { in.read(); assertEquals(0L, in.skip(-2L)); assertEquals(DATA_BYTES[1], in.read()); assertEquals(DATA_BYTES[2], in.read()); } - @Test public void testSkip_HitEnd() throws Exception { + @Test + public void testSkip_HitEnd() throws Exception { for (int i = 0; i < DATA_BYTES.length - 1; i++) { in.read(); } assertEquals(1L, in.skip(2L)); assertEquals(-1, in.read()); } - @Test public void testSkip_AtEnd() throws Exception { + @Test + public void testSkip_AtEnd() throws Exception { for (int i = 0; i < DATA_BYTES.length; i++) { in.read(); } @@ -133,7 +146,8 @@ public void testReadArrayOffsetLength_LengthTooLarge() throws Exception { assertEquals(-1, in.read()); } - @Test public void testAvailable() throws Exception { + @Test + public void testAvailable() throws Exception { for (int i = DATA_BYTES.length; i > 0; i--) { assertEquals(i, in.available()); in.read(); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/EphemeralTypeTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/EphemeralTypeTest.java index 5c61ffc05c3..7a734d5f958 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/EphemeralTypeTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/EphemeralTypeTest.java @@ -28,12 +28,12 @@ public class EphemeralTypeTest { @Before - public void setUp() { + public void setUp() throws Exception { System.setProperty(EphemeralType.EXTENDED_TYPES_ENABLED_PROPERTY, "true"); } @After - public void tearDown() { + public void tearDown() throws Exception { System.clearProperty(EphemeralType.EXTENDED_TYPES_ENABLED_PROPERTY); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/NIOServerCnxnFactoryTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/NIOServerCnxnFactoryTest.java index 8020657920b..ed92f9deda2 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/NIOServerCnxnFactoryTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/NIOServerCnxnFactoryTest.java @@ -33,14 +33,14 @@ public class NIOServerCnxnFactoryTest { private NIOServerCnxnFactory factory; @Before - public void setUp() throws IOException { + public void setUp() throws Exception { listenAddress = new InetSocketAddress(PortAssignment.unique()); factory = new NIOServerCnxnFactory(); factory.configure(listenAddress, 100); } @After - public void tearDown() { + public void tearDown() throws Exception { if (factory != null) { factory.shutdown(); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/PrepRequestProcessorTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/PrepRequestProcessorTest.java index 8223583b1fb..d1814e0e1ec 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/PrepRequestProcessorTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/PrepRequestProcessorTest.java @@ -65,7 +65,8 @@ public class PrepRequestProcessorTest extends ClientBase { private Request outcome; @Before - public void setup() throws Exception { + public void setUp() throws Exception { + super.setUp(); File tmpDir = ClientBase.createTmpDir(); ClientBase.setupTestEnv(); zks = new ZooKeeperServer(tmpDir, tmpDir, 3000); @@ -80,13 +81,14 @@ public void setup() throws Exception { } @After - public void teardown() throws Exception { + public void tearDown() throws Exception { if (servcnxnf != null) { servcnxnf.shutdown(); } if (zks != null) { zks.shutdown(); } + super.tearDown(); } @Test diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/PurgeTxnTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/PurgeTxnTest.java index 7898e6eaafe..f0908d97290 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/PurgeTxnTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/PurgeTxnTest.java @@ -57,7 +57,7 @@ public class PurgeTxnTest extends ZKTestCase { private File tmpDir; @After - public void teardown() { + public void tearDown() throws Exception { if (null != tmpDir) { ClientBase.recursiveDelete(tmpDir); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/ServerStatsTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/ServerStatsTest.java index aa207d1b20f..44d8bf0c2aa 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/ServerStatsTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/ServerStatsTest.java @@ -33,7 +33,7 @@ public class ServerStatsTest extends ZKTestCase { private ServerStats.Provider providerMock; @Before - public void setUp() { + public void setUp() throws Exception { providerMock = mock(ServerStats.Provider.class); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/WatchesPathReportTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/WatchesPathReportTest.java index c0b107debba..c5b230faa7f 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/WatchesPathReportTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/WatchesPathReportTest.java @@ -28,7 +28,8 @@ public class WatchesPathReportTest extends ZKTestCase { private Map> m; private WatchesPathReport r; - @Before public void setUp() { + @Before + public void setUp() throws Exception { m = new HashMap>(); Set s = new HashSet(); s.add(101L); @@ -39,12 +40,14 @@ public class WatchesPathReportTest extends ZKTestCase { m.put("path2", s); r = new WatchesPathReport(m); } - @Test public void testHasSessions() { + @Test + public void testHasSessions() { assertTrue(r.hasSessions("path1")); assertTrue(r.hasSessions("path2")); assertFalse(r.hasSessions("path3")); } - @Test public void testGetSessions() { + @Test + public void testGetSessions() { Set s = r.getSessions("path1"); assertEquals(2, s.size()); assertTrue(s.contains(101L)); @@ -54,7 +57,8 @@ public class WatchesPathReportTest extends ZKTestCase { assertTrue(s.contains(201L)); assertNull(r.getSessions("path3")); } - @Test public void testToMap() { + @Test + public void testToMap() { assertEquals(m, r.toMap()); } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/WatchesReportTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/WatchesReportTest.java index 7f0343b329a..83bb4b1ae8a 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/WatchesReportTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/WatchesReportTest.java @@ -28,7 +28,8 @@ public class WatchesReportTest extends ZKTestCase { private Map> m; private WatchesReport r; - @Before public void setUp() { + @Before + public void setUp() throws Exception { m = new HashMap>(); Set s = new HashSet(); s.add("path1a"); @@ -39,12 +40,14 @@ public class WatchesReportTest extends ZKTestCase { m.put(2L, s); r = new WatchesReport(m); } - @Test public void testHasPaths() { + @Test + public void testHasPaths() { assertTrue(r.hasPaths(1L)); assertTrue(r.hasPaths(2L)); assertFalse(r.hasPaths(3L)); } - @Test public void testGetPaths() { + @Test + public void testGetPaths() { Set s = r.getPaths(1L); assertEquals(2, s.size()); assertTrue(s.contains("path1a")); @@ -54,7 +57,8 @@ public class WatchesReportTest extends ZKTestCase { assertTrue(s.contains("path2a")); assertNull(r.getPaths(3L)); } - @Test public void testToMap() { + @Test + public void testToMap() { assertEquals(m, r.toMap()); } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/WatchesSummaryTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/WatchesSummaryTest.java index d679065c5bc..8daa61defa3 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/WatchesSummaryTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/WatchesSummaryTest.java @@ -24,15 +24,18 @@ public class WatchesSummaryTest extends ZKTestCase { private WatchesSummary s; - @Before public void setUp() { + @Before + public void setUp() throws Exception { s = new WatchesSummary(1, 2, 3); } - @Test public void testGetters() { + @Test + public void testGetters() { assertEquals(1, s.getNumConnections()); assertEquals(2, s.getNumPaths()); assertEquals(3, s.getTotalWatches()); } - @Test public void testToMap() { + @Test + public void testToMap() { Map m = s.toMap(); assertEquals(3, m.size()); assertEquals(Integer.valueOf(1), m.get(WatchesSummary.KEY_NUM_CONNECTIONS)); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerBeanTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerBeanTest.java index 08adfdc9fa0..ce20adff233 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerBeanTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerBeanTest.java @@ -37,13 +37,13 @@ public class ZooKeeperServerBeanTest { @Before - public void setup() { + public void setUp() throws Exception { System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, "org.apache.zookeeper.server.NettyServerCnxnFactory"); } @After - public void teardown() throws Exception { + public void tearDown() throws Exception { System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerConfTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerConfTest.java index b53321a950e..67b3628bda2 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerConfTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerConfTest.java @@ -24,10 +24,12 @@ public class ZooKeeperServerConfTest extends ZKTestCase { private ZooKeeperServerConf c; - @Before public void setUp() { + @Before + public void setUp() throws Exception { c = new ZooKeeperServerConf(1, "a", "b", 2, 3, 4, 5, 6L); } - @Test public void testGetters() { + @Test + public void testGetters() { assertEquals(1, c.getClientPort()); assertEquals("a", c.getDataDir()); assertEquals("b", c.getDataLogDir()); @@ -37,7 +39,8 @@ public class ZooKeeperServerConfTest extends ZKTestCase { assertEquals(5, c.getMaxSessionTimeout()); assertEquals(6L, c.getServerId()); } - @Test public void testToMap() { + @Test + public void testToMap() { Map m = c.toMap(); assertEquals(8, m.size()); assertEquals(Integer.valueOf(1), m.get(ZooKeeperServerConf.KEY_CLIENT_PORT)); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/CommandResponseTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/CommandResponseTest.java index ab8c25860a1..7173a5d6344 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/CommandResponseTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/CommandResponseTest.java @@ -27,16 +27,19 @@ public class CommandResponseTest extends ZKTestCase { private CommandResponse r; - @Before public void setUp() throws Exception { + @Before + public void setUp() throws Exception { r = new CommandResponse("makemeasandwich", "makeityourself"); } - @Test public void testGetters() { + @Test + public void testGetters() { assertEquals("makemeasandwich", r.getCommand()); assertEquals("makeityourself", r.getError()); } - @Test public void testMap() { + @Test + public void testMap() { r.put("missing", "sudo"); Map m = new HashMap(); m.put("origin", "xkcd"); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/JettyAdminServerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/JettyAdminServerTest.java index bc8aab67b36..cf505d5fba5 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/JettyAdminServerTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/JettyAdminServerTest.java @@ -36,14 +36,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class JettyAdminServerTest extends ZKTestCase{ +public class JettyAdminServerTest extends ZKTestCase { protected static final Logger LOG = LoggerFactory.getLogger(JettyAdminServerTest.class); private static final String URL_FORMAT = "http://localhost:%d/commands"; private static final int jettyAdminPort = PortAssignment.unique(); @Before - public void enableServer() { + public void setUp() throws Exception { // Override setting in ZKTestCase System.setProperty("zookeeper.admin.enableServer", "true"); System.setProperty("zookeeper.admin.serverPort", "" + jettyAdminPort); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/persistence/TxnLogToolkitTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/persistence/TxnLogToolkitTest.java index 79a69c4e921..a4794c91cc6 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/persistence/TxnLogToolkitTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/persistence/TxnLogToolkitTest.java @@ -48,7 +48,7 @@ public class TxnLogToolkitTest { private File mySnapDir; @Before - public void setUp() throws IOException { + public void setUp() throws Exception { System.setOut(new PrintStream(outContent)); System.setErr(new PrintStream(errContent)); File snapDir = new File(testData, "invalidsnap"); @@ -57,7 +57,7 @@ public void setUp() throws IOException { } @After - public void tearDown() throws IOException { + public void tearDown() throws Exception { System.setOut(System.out); System.setErr(System.err); mySnapDir.setWritable(true); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CommitProcessorTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CommitProcessorTest.java index 1418158ecb1..c2995886ba2 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CommitProcessorTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CommitProcessorTest.java @@ -83,7 +83,7 @@ public class CommitProcessorTest extends ZKTestCase { ArrayList testClients = new ArrayList(); - public void setUp(int numCommitThreads, int numClientThreads) + public void prepareTest(int numCommitThreads, int numClientThreads) throws Exception { System.setProperty( CommitProcessor.ZOOKEEPER_COMMIT_PROC_NUM_WORKER_THREADS, @@ -169,7 +169,7 @@ public void run() { @Test public void testNoCommitWorkers() throws Exception { - setUp(0, 10); + prepareTest(0, 10); synchronized(this) { wait(5000); } @@ -179,7 +179,7 @@ public void testNoCommitWorkers() throws Exception { @Test public void testOneCommitWorker() throws Exception { - setUp(1, 10); + prepareTest(1, 10); synchronized(this) { wait(5000); } @@ -189,7 +189,7 @@ public void testOneCommitWorker() throws Exception { @Test public void testManyCommitWorkers() throws Exception { - setUp(10, 10); + prepareTest(10, 10); synchronized(this) { wait(5000); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/EphemeralNodeDeletionTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/EphemeralNodeDeletionTest.java index 9546c25cc79..4b3e98e8413 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/EphemeralNodeDeletionTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/EphemeralNodeDeletionTest.java @@ -166,7 +166,7 @@ public TestQPMain getTestQPMain() { } @After - public void tearDown() { + public void tearDown() throws Exception { // stop all severs for (int i = 0; i < mt.length; i++) { try { diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FuzzySnapshotRelatedTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FuzzySnapshotRelatedTest.java index 1499c8c889f..cdf47f501bb 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FuzzySnapshotRelatedTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FuzzySnapshotRelatedTest.java @@ -66,7 +66,7 @@ public class FuzzySnapshotRelatedTest extends QuorumPeerTestBase { int followerA; @Before - public void setup() throws Exception { + public void setUp() throws Exception { LOG.info("Start up a 3 server quorum"); final int ENSEMBLE_SERVERS = 3; final int clientPorts[] = new int[ENSEMBLE_SERVERS]; diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java index 99d5b5d6cd5..64981437a13 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java @@ -60,7 +60,7 @@ public class LeaderBeanTest { private QuorumPeer qp; @Before - public void setUp() throws IOException, X509Exception { + public void setUp() throws Exception { qp = new QuorumPeer(); long myId = qp.getId(); @@ -89,7 +89,7 @@ public void setUp() throws IOException, X509Exception { } @After - public void tearDown() throws IOException { + public void tearDown() throws Exception { fileTxnSnapLog.close(); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderWithObserverTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderWithObserverTest.java index 2548acad1cd..8bf8a4e4765 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderWithObserverTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderWithObserverTest.java @@ -61,7 +61,7 @@ observerId, new InetSocketAddress("127.0.0.1", PortAssignment.unique()), } @After - public void tearDown(){ + public void tearDown() throws Exception { leader.shutdown("end of test"); tmpDir.delete(); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumSSLTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumSSLTest.java index b088f47b16c..e73dbf0327f 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumSSLTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumSSLTest.java @@ -156,7 +156,7 @@ public class QuorumSSLTest extends QuorumPeerTestBase { .withLookingForStuckThread(true).build(); @Before - public void setup() throws Exception { + public void setUp() throws Exception { ClientBase.setupTestEnv(); tmpDir = createTmpDir().getAbsolutePath(); @@ -394,7 +394,7 @@ public void setSSLSystemProperties() { } @After - public void cleanUp() throws Exception { + public void tearDown() throws Exception { clearSSLSystemProperties(); if (q1 != null) { q1.shutdown(); 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..34b8273e4f6 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 @@ -89,7 +89,7 @@ public void testRaceConditionBetweenLeaderAndAckRequestProcessor() throws Except } @After - public void tearDown() { + public void tearDown() throws Exception { // stop all severs if (null != mt) { for (int i = 0; i < SERVER_COUNT; i++) { diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigBackupTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigBackupTest.java index 8bc04bd6ec7..9878a5142cc 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigBackupTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigBackupTest.java @@ -60,7 +60,7 @@ public static String getFileContent(File file) throws FileNotFoundException { } @Before - public void setup() { + public void setUp() throws Exception { ClientBase.setupTestEnv(); System.setProperty("zookeeper.DigestAuthenticationProvider.superDigest", "super:D/InIHSb7yEEbrWz8b9l71RjZJU="/* password is 'test'*/); @@ -358,4 +358,4 @@ public void testVersionOfDynamicFilename() throws Exception { mt[i].shutdown(); } } -} \ No newline at end of file +} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigDuringLeaderSyncTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigDuringLeaderSyncTest.java index f350abf05ad..0f3192a4541 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigDuringLeaderSyncTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigDuringLeaderSyncTest.java @@ -50,7 +50,7 @@ public class ReconfigDuringLeaderSyncTest extends QuorumPeerTestBase { private MainThread[] mt; @Before - public void setup() { + public void setUp() throws Exception { System.setProperty("zookeeper.DigestAuthenticationProvider.superDigest", "super:D/InIHSb7yEEbrWz8b9l71RjZJU="/* password is 'test'*/); QuorumPeerConfig.setReconfigEnabled(true); @@ -191,7 +191,7 @@ private static CustomQuorumPeer getCustomQuorumPeer(MainThread mt) { } @After - public void tearDown() { + public void tearDown() throws Exception { // stop all severs if (null != mt) { for (int i = 0; i < mt.length; i++) { 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 8120d0fa28a..201b27427f9 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 @@ -43,7 +43,7 @@ public class ReconfigFailureCasesTest extends QuorumPeerTestBase { private QuorumUtil qu; @Before - public void setup() { + public void setUp() throws Exception { QuorumPeerConfig.setReconfigEnabled(true); System.setProperty("zookeeper.DigestAuthenticationProvider.superDigest", "super:D/InIHSb7yEEbrWz8b9l71RjZJU="/* password is 'test'*/); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigLegacyTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigLegacyTest.java index 80a8bfe13cb..b3d793409ff 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigLegacyTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigLegacyTest.java @@ -43,7 +43,7 @@ public class ReconfigLegacyTest extends QuorumPeerTestBase { private static final int SERVER_COUNT = 3; @Before - public void setup() { + public void setUp() throws Exception { ClientBase.setupTestEnv(); QuorumPeerConfig.setReconfigEnabled(true); System.setProperty("zookeeper.DigestAuthenticationProvider.superDigest", diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java index 4de9b976894..0514f63bd66 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java @@ -33,7 +33,7 @@ public class ReconfigRecoveryTest extends QuorumPeerTestBase { @Before - public void setup() { + public void setUp() throws Exception { QuorumPeerConfig.setReconfigEnabled(true); } @@ -581,4 +581,4 @@ public static StringBuilder generateConfig(int numServers, int[][] ports, } return sb; } -} \ No newline at end of file +} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/StatCommandTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/StatCommandTest.java index eccb6b18495..18a01302e16 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/StatCommandTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/StatCommandTest.java @@ -43,7 +43,7 @@ public class StatCommandTest { private ServerStats.Provider providerMock; @Before - public void setUp() { + public void setUp() throws Exception { outputWriter = new StringWriter(); ServerCnxn serverCnxnMock = mock(ServerCnxn.class); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/StatResetCommandTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/StatResetCommandTest.java index 0c20c5887b4..29de92aa150 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/StatResetCommandTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/StatResetCommandTest.java @@ -42,7 +42,7 @@ public class StatResetCommandTest { private ServerStats serverStats; @Before - public void setUp() { + public void setUp() throws Exception { outputWriter = new StringWriter(); ServerCnxn serverCnxnMock = mock(ServerCnxn.class); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/WatchLeakTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/WatchLeakTest.java index d8667dc14dd..cf740833ac2 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/WatchLeakTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/WatchLeakTest.java @@ -83,7 +83,7 @@ public class WatchLeakTest { private final boolean sessionTimedout; @Before - public void setUp() { + public void setUp() throws Exception { System.setProperty("zookeeper.admin.enableServer", "false"); } 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 046092cb911..b0f7fa4412d 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 @@ -81,7 +81,7 @@ public class Zab1_0Test extends ZKTestCase { System.getProperty("test.data.dir", "build/test/data")); @Before - public void setUp() { + public void setUp() throws Exception { System.setProperty("zookeeper.admin.enableServer", "false"); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthUpgradeTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthUpgradeTest.java index 359324549e2..93fcfdb67a4 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthUpgradeTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthUpgradeTest.java @@ -69,7 +69,7 @@ public void tearDown() throws Exception { } @AfterClass - public static void cleanup() { + public static void tearDownJaas() throws Exception { cleanupJaasConfig(); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumDigestAuthTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumDigestAuthTest.java index 5eebdb335c0..e9945fb0018 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumDigestAuthTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumDigestAuthTest.java @@ -65,7 +65,7 @@ public void tearDown() throws Exception { } @AfterClass - public static void cleanup(){ + public static void tearDownJaas() throws Exception { cleanupJaasConfig(); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java index 2cc56a76794..684fa279856 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java @@ -79,7 +79,7 @@ public void tearDown() throws Exception { } @AfterClass - public static void cleanup() { + public static void tearDownJaas() throws Exception { if(keytabFile != null){ FileUtils.deleteQuietly(keytabFile); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java index fcb76919f1b..6ea76d67321 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java @@ -80,7 +80,7 @@ private static void setupJaasConfigEntries(String hostServerPrincipal, } @BeforeClass - public static void setUp() throws Exception { + public static void setUpPrincipals() throws Exception { // create keytab keytabFile = new File(KerberosTestUtils.getKeytabFile()); @@ -104,7 +104,7 @@ public void tearDown() throws Exception { } @AfterClass - public static void cleanup() { + public static void tearDownJaas() throws Exception { if(keytabFile != null){ FileUtils.deleteQuietly(keytabFile); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/util/VerifyingFileFactoryTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/util/VerifyingFileFactoryTest.java index 0bf5b61dbd5..21b297d7af6 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/util/VerifyingFileFactoryTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/util/VerifyingFileFactoryTest.java @@ -33,7 +33,7 @@ public class VerifyingFileFactoryTest extends ZKTestCase { private Logger log; @Before - public void setUp(){ + public void setUp() throws Exception { log = LoggerFactory.getLogger("TODO: Mock Logging"); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/AsyncHammerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/AsyncHammerTest.java index 1ccdc0bc02d..7958127913d 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/AsyncHammerTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/AsyncHammerTest.java @@ -48,7 +48,7 @@ public class AsyncHammerTest extends ZKTestCase private volatile boolean bang; - public void setUp(boolean withObservers) throws Exception { + public void prepareTest(boolean withObservers) throws Exception { qb.setUp(withObservers); } @@ -167,7 +167,7 @@ public void processResult(int rc, String path, Object ctx) { @Test public void testHammer() throws Exception { - setUp(false); + prepareTest(false); bang = true; LOG.info("Starting hammers"); HammerThread[] hammers = new HammerThread[100]; @@ -199,7 +199,7 @@ public void testHammer() throws Exception { @Test public void testObserversHammer() throws Exception { - setUp(true); + prepareTest(true); bang = true; Thread[] hammers = new Thread[100]; for (int i = 0; i < hammers.length; i++) { diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/AtomicFileOutputStreamTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/AtomicFileOutputStreamTest.java index cbd2b7700dc..ff3bad09e81 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/AtomicFileOutputStreamTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/AtomicFileOutputStreamTest.java @@ -42,12 +42,12 @@ public class AtomicFileOutputStreamTest extends ZKTestCase { private File dstFile; @Before - public void setupTestDir() throws IOException { + public void setUp() throws Exception { testDir = ClientBase.createEmptyTestDir(); dstFile = new File(testDir, "test.txt"); } @After - public void cleanupTestDir() throws IOException { + public void tearDown() throws Exception { ClientBase.recursiveDelete(testDir); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/BufferSizeTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/BufferSizeTest.java index f16429803fe..38c25b532bd 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/BufferSizeTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/BufferSizeTest.java @@ -41,7 +41,8 @@ public class BufferSizeTest extends ClientBase { private ZooKeeper zk; @Before - public void setMaxBuffer() throws IOException, InterruptedException { + public void setUp() throws Exception { + super.setUp(); System.setProperty("jute.maxbuffer", "" + TEST_MAXBUFFER); assertEquals("Can't set jute.maxbuffer!", TEST_MAXBUFFER, BinaryInputArchive.maxBuffer); zk = createClient(); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java index b5c5d4dec74..deb0ebd0520 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java @@ -40,7 +40,7 @@ public class ClientSSLTest extends QuorumPeerTestBase { private ClientX509Util clientX509Util = new ClientX509Util(); @Before - public void setup() { + public void setUp() throws Exception { String testDataPath = System.getProperty("test.data.dir", "build/test/data"); System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, "org.apache.zookeeper.server.NettyServerCnxnFactory"); System.setProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET, "org.apache.zookeeper.ClientCnxnSocketNetty"); @@ -52,7 +52,7 @@ public void setup() { } @After - public void teardown() throws Exception { + public void tearDown() throws Exception { System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY); System.clearProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET); System.clearProperty(ZKClientConfig.SECURE_CLIENT); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSkipACLTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSkipACLTest.java index 2c57ff7a460..651e951082c 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSkipACLTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSkipACLTest.java @@ -23,12 +23,12 @@ public class ClientSkipACLTest extends ClientTest { @BeforeClass - static public void setup() { + static public void setUpSkipACL() throws Exception { System.setProperty("zookeeper.skipACL", "yes"); } @AfterClass - static public void teardown() { + static public void tearDownSkipACL() throws Exception { System.clearProperty("zookeeper.skipACL"); } -} \ No newline at end of file +} 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 a072bc065ce..6a21ad10e30 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 @@ -58,9 +58,9 @@ public class CnxManagerTest extends ZKTestCase { File peerTmpdir[]; int peerQuorumPort[]; int peerClientPort[]; + @Before public void setUp() throws Exception { - this.count = 3; this.peers = new HashMap(count); peerTmpdir = new File[count]; diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/NettyNettySuiteBase.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/NettyNettySuiteBase.java index 684d67a91f2..f1dbdba1196 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/NettyNettySuiteBase.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/NettyNettySuiteBase.java @@ -33,7 +33,7 @@ @RunWith(Suite.class) public class NettyNettySuiteBase { @BeforeClass - public static void setUp() { + public static void setUpNetty() throws Exception { System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, NettyServerCnxnFactory.class.getName()); System.setProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET, @@ -42,7 +42,7 @@ public static void setUp() { } @AfterClass - public static void tearDown() { + public static void tearDownNetty() throws Exception { System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY); System.clearProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/NioNettySuiteBase.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/NioNettySuiteBase.java index 5725c170e4f..f7cf5ce4d03 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/NioNettySuiteBase.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/NioNettySuiteBase.java @@ -31,14 +31,14 @@ @RunWith(Suite.class) public class NioNettySuiteBase { @BeforeClass - public static void setUp() { + public static void setUpNetty() throws Exception { System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, NettyServerCnxnFactory.class.getName()); System.setProperty("zookeeper.admin.enableServer", "false"); } @AfterClass - public static void tearDown() { + public static void tearDownNetty() throws Exception { System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY); } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/OSMXBeanTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/OSMXBeanTest.java index 371c9021d39..933da066433 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/OSMXBeanTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/OSMXBeanTest.java @@ -35,7 +35,7 @@ public class OSMXBeanTest extends ZKTestCase { protected static final Logger LOG = LoggerFactory.getLogger(OSMXBeanTest.class); @Before - public void initialize() { + public void setUp() throws Exception { this.osMbean = new OSMXBean(); Assert.assertNotNull("Could not initialize OSMXBean object!", osMbean); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ObserverLETest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ObserverLETest.java index 123ba0be0bd..cdd838ebe8e 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ObserverLETest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ObserverLETest.java @@ -33,7 +33,7 @@ public class ObserverLETest extends ZKTestCase { final ClientTest ct = new ClientTest(); @Before - public void establishThreeParticipantOneObserverEnsemble() throws Exception { + public void setUp() throws Exception { qb.setUp(true); ct.hostPort = qb.hostPort; ct.setUpAll(); @@ -41,7 +41,7 @@ public void establishThreeParticipantOneObserverEnsemble() throws Exception { } @After - public void shutdownQuorum() throws Exception { + public void tearDown() throws Exception { ct.tearDownAll(); qb.tearDown(); } 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..f59414c7d9c 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 @@ -55,7 +55,7 @@ public class ReconfigExceptionTest extends ZKTestCase { private ZooKeeperAdmin zkAdmin; @Before - public void setup() throws InterruptedException { + public void setUp() throws Exception { System.setProperty(authProvider, superDigest); QuorumPeerConfig.setReconfigEnabled(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..1fb8e122d7d 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 @@ -44,7 +44,7 @@ public class ReconfigMisconfigTest extends ZKTestCase { "user's password on server side first."; @Before - public void setup() throws InterruptedException { + public void setUp() throws Exception { QuorumPeerConfig.setReconfigEnabled(true); // Get a three server quorum. qu = new QuorumUtil(1); 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 fb0e5f01b38..6bb48d11680 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 @@ -55,14 +55,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class ReconfigTest extends ZKTestCase implements DataCallback{ +public class ReconfigTest extends ZKTestCase implements DataCallback { private static final Logger LOG = LoggerFactory .getLogger(ReconfigTest.class); private QuorumUtil qu; @Before - public void setup() { + public void setUp() throws Exception { System.setProperty("zookeeper.DigestAuthenticationProvider.superDigest", "super:D/InIHSb7yEEbrWz8b9l71RjZJU="/* password is 'test'*/); QuorumPeerConfig.setReconfigEnabled(true); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/SSLAuthTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/SSLAuthTest.java index 8fd35bc6185..256b19c65a6 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/SSLAuthTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/SSLAuthTest.java @@ -60,7 +60,8 @@ public void setUp() throws Exception { } @After - public void teardown() throws Exception { + public void tearDown() throws Exception { + super.tearDown(); System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY); System.clearProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET); System.clearProperty(ZKClientConfig.SECURE_CLIENT); @@ -102,4 +103,4 @@ public void testMisconfiguration() throws Exception { Assert.assertFalse("Missing SSL configuration should not result in successful connection", watcher.clientConnected.await(1000, TimeUnit.MILLISECONDS)); } -} \ No newline at end of file +} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslClientTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslClientTest.java index 95bf2f6af3d..519cc0a8dc7 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslClientTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslClientTest.java @@ -32,13 +32,13 @@ public class SaslClientTest extends ZKTestCase { private String existingPropertyValue = null; @Before - public void setUp() { + public void setUp() throws Exception { existingPropertyValue = System .getProperty(ZKClientConfig.ENABLE_CLIENT_SASL_KEY); } @After - public void tearDown() { + public void tearDown() throws Exception { // Restore the System property if it was set previously if (existingPropertyValue != null) { System.setProperty(ZKClientConfig.ENABLE_CLIENT_SASL_KEY, diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslSuperUserTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslSuperUserTest.java index 894c0f5d108..8f87407acc3 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslSuperUserTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslSuperUserTest.java @@ -46,7 +46,7 @@ public class SaslSuperUserTest extends ClientBase { private static String oldSuperUser; @BeforeClass - public static void setupStatic() throws Exception { + public static void setUpSaslJaas() throws Exception { oldAuthProvider = System.setProperty("zookeeper.authProvider.1","org.apache.zookeeper.server.auth.SASLAuthenticationProvider"); File tmpDir = createTmpDir(); @@ -70,7 +70,7 @@ public static void setupStatic() throws Exception { } @AfterClass - public static void cleanupStatic() { + public static void tearDownSaslJaas() throws Exception { if (oldAuthProvider != null) { System.setProperty("zookeeper.authProvider.1", oldAuthProvider); } else { diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/StandaloneTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/StandaloneTest.java index 3d57b6c5c10..8eea236ee9a 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/StandaloneTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/StandaloneTest.java @@ -44,12 +44,12 @@ /** * Standalone server tests. */ -public class StandaloneTest extends QuorumPeerTestBase implements Watcher{ +public class StandaloneTest extends QuorumPeerTestBase implements Watcher { protected static final Logger LOG = LoggerFactory.getLogger(StandaloneTest.class); @Before - public void setup() { + public void setUp() throws Exception { System.setProperty("zookeeper.DigestAuthenticationProvider.superDigest", "super:D/InIHSb7yEEbrWz8b9l71RjZJU="/* password is 'test'*/); QuorumPeerConfig.setReconfigEnabled(true); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/TruncateTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/TruncateTest.java index cae2d9f53ae..3004250e437 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/TruncateTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/TruncateTest.java @@ -54,14 +54,14 @@ public class TruncateTest extends ZKTestCase { File dataDir1, dataDir2, dataDir3; @Before - public void setUp() throws IOException { + public void setUp() throws Exception { dataDir1 = ClientBase.createTmpDir(); dataDir2 = ClientBase.createTmpDir(); dataDir3 = ClientBase.createTmpDir(); } @After - public void tearDown() { + public void tearDown() throws Exception { ClientBase.recursiveDelete(dataDir1); ClientBase.recursiveDelete(dataDir2); ClientBase.recursiveDelete(dataDir3); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/WatchEventWhenAutoResetTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/WatchEventWhenAutoResetTest.java index a7604638173..076bbd750e8 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/WatchEventWhenAutoResetTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/WatchEventWhenAutoResetTest.java @@ -90,7 +90,7 @@ private ZooKeeper createClient(QuorumUtil qu, int id) throws IOException { } @Before - public void setUp() { + public void setUp() throws Exception { System.setProperty("zookeeper.admin.enableServer", "false"); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/X509AuthTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/X509AuthTest.java index 4982cf3391b..b8e0196986a 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/X509AuthTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/X509AuthTest.java @@ -54,7 +54,7 @@ public class X509AuthTest extends ZKTestCase { private static TestCertificate unknownCert; @Before - public void setUp() { + public void setUp() throws Exception { System.setProperty("zookeeper.X509AuthenticationProvider.superUser", "CN=SUPER"); System.setProperty("zookeeper.ssl.keyManager", From 614020c28dd1da68b6b5689db6cdb2ed603e286e Mon Sep 17 00:00:00 2001 From: Eric Schwartz Date: Wed, 28 Nov 2018 11:39:12 -0700 Subject: [PATCH 27/63] Trying another formatter-- adding debug output to OneLinerFormatter to ensure it's not used --- build.xml | 160 +++++++++--------- .../undefined/testing/OneLinerFormatter.java | 2 + 2 files changed, 82 insertions(+), 80 deletions(-) diff --git a/build.xml b/build.xml index e5bd1dd3d0b..38204fbade2 100644 --- a/build.xml +++ b/build.xml @@ -17,7 +17,7 @@ limitations under the License. --> - - + @@ -86,7 +86,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + @@ -206,7 +206,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - @@ -222,14 +222,14 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - - + + - + - @@ -291,7 +291,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + @@ -306,8 +306,8 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - - + + @@ -321,7 +321,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> javacchome="${ivy.javacc.lib}" /> - + - + @@ -369,13 +369,13 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + - @@ -384,10 +384,10 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + - @@ -397,12 +397,12 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + - + @@ -414,7 +414,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + @@ -488,7 +488,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + @@ -578,7 +578,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + @@ -622,17 +622,17 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + - + - - + + @@ -643,16 +643,16 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + - - + @@ -665,11 +665,11 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + - + - @@ -683,7 +683,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + @@ -706,11 +706,11 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + - + - @@ -739,7 +739,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + @@ -757,7 +757,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + @@ -775,7 +775,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + @@ -786,7 +786,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - @@ -813,9 +813,9 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + - + @@ -825,8 +825,8 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + tofile="${dist.maven.dir}/${final.name}.jar"/> + @@ -848,7 +848,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> + tofile="${dist.maven.dir}/${final.name}-tests.jar"/> @@ -889,7 +889,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + @@ -931,7 +931,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + @@ -941,12 +941,12 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + tofile="${dist.maven.dir}/${final.name}.jar"/> + - + @@ -962,8 +962,8 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - - + + @@ -1129,7 +1129,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + @@ -1138,7 +1138,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + @@ -1150,11 +1150,11 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + The version is ${zookeeper-pom.version} - + @@ -1175,7 +1175,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + @@ -1185,9 +1185,9 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + - + @@ -1300,7 +1300,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + @@ -1352,7 +1352,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + @@ -1371,11 +1371,11 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + - + @@ -1385,7 +1385,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + @@ -1567,10 +1567,10 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - - @@ -1580,7 +1580,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - @@ -1613,7 +1613,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - @@ -1786,12 +1786,12 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - - @@ -1876,17 +1876,17 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + - + Date: Wed, 28 Nov 2018 10:55:20 -0800 Subject: [PATCH 28/63] fix thread safety problems in OneLinerFormatter --- .../undefined/testing/OneLinerFormatter.java | 55 ++++++++----------- 1 file changed, 24 insertions(+), 31 deletions(-) diff --git a/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java b/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java index b214548aeea..57aa6e40762 100644 --- a/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java +++ b/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java @@ -35,11 +35,12 @@ public class OneLinerFormatter implements JUnitResultFormatter { - private final String TAB_STR = " "; + private final static String TAB_STR = " "; - private final boolean showCausesLines = true; // (\w+\.)+(\w+)\((\w+).(?:\w+):(\d+)\) - private final Pattern traceLinePattern = Pattern.compile("(\\w+\\.)+(\\w+)\\((\\w+).(?:\\w+):(\\d+)\\)"); + private final static Pattern traceLinePattern = Pattern.compile("(\\w+\\.)+(\\w+)\\((\\w+).(?:\\w+):(\\d+)\\)"); + + private final static boolean showCausesLines = true; /** * Where to write the log to. @@ -97,7 +98,7 @@ public OneLinerFormatter() { * Sets the stream the formatter is supposed to write its results to. * @param out the output stream to write to */ - public void setOutput(OutputStream out) { + public synchronized void setOutput(OutputStream out) { this.out = out; output = new PrintWriter(out); } @@ -105,14 +106,14 @@ public void setOutput(OutputStream out) { /** * @see JUnitResultFormatter#setSystemOutput(String) */ - public void setSystemOutput(String out) { + public synchronized void setSystemOutput(String out) { systemOutput = out; } /** * @see JUnitResultFormatter#setSystemError(String) */ - public void setSystemError(String err) { + public synchronized void setSystemError(String err) { systemError = err; } @@ -120,7 +121,7 @@ public void setSystemError(String err) { * The whole testsuite started. * @param suite the test suite */ - public void startTestSuite(JUnitTest suite) { + public synchronized void startTestSuite(JUnitTest suite) { if (output == null) { return; // Quick return - no output do nothing. } @@ -138,7 +139,7 @@ public void startTestSuite(JUnitTest suite) { output.flush(); } - public String prefixLines(String prefix, String buffer) { + public final static String prefixLines(String prefix, String buffer) { String lines[] = buffer.split("\\r?\\n"); return Arrays.asList(lines) .stream() @@ -151,7 +152,7 @@ public String prefixLines(String prefix, String buffer) { * The whole testsuite ended. * @param suite the test suite */ - public void endTestSuite(JUnitTest suite) { + public synchronized void endTestSuite(JUnitTest suite) { StringBuffer sb = new StringBuffer("Tests run: "); String runtimeName = " [" + ManagementFactory.getRuntimeMXBean().getName() + "] "; sb.append(suite.runCount()); @@ -208,7 +209,7 @@ public void endTestSuite(JUnitTest suite) { * A test started. * @param test a test */ - public void startTest(Test test) { + public synchronized void startTest(Test test) { testStarts.put(test, new Long(System.currentTimeMillis())); } @@ -216,7 +217,7 @@ public void startTest(Test test) { * A test ended. * @param test a test */ - public void endTest(Test test) { + public synchronized void endTest(Test test) { // Fix for bug #5637 - if a junit.extensions.TestSetup is // used and throws an exception during setUp then startTest // would never have been called @@ -228,6 +229,8 @@ public void endTest(Test test) { Long l = (Long) testStarts.get(test); + String runtimeName = " [" + ManagementFactory.getRuntimeMXBean().getName() + "] "; + output.write(runtimeName); output.write("Ran ["); output.write(((System.currentTimeMillis() - l.longValue()) / 1000.0) + "] "); output.write(getTestName(test) + " ... " + (failed ? "FAILED" : "OK")); @@ -235,17 +238,6 @@ public void endTest(Test test) { output.flush(); } - /** - * Interface TestListener for JUnit <= 3.4. - * - *

A Test failed. - * @param test a test - * @param t the exception thrown by the test - */ - public void addFailure(Test test, Throwable t) { - formatError("\tFAILED", test, t); - } - /** * Interface TestListener for JUnit > 3.4. * @@ -253,8 +245,8 @@ public void addFailure(Test test, Throwable t) { * @param test a test * @param t the assertion failed by the test */ - public void addFailure(Test test, AssertionFailedError t) { - addFailure(test, (Throwable) t); + public synchronized void addFailure(Test test, AssertionFailedError t) { + formatError("\tFAILED", test, (Throwable) t); } /** @@ -262,7 +254,7 @@ public void addFailure(Test test, AssertionFailedError t) { * @param test a test * @param error the error thrown by the test */ - public void addError(Test test, Throwable error) { + public synchronized void addError(Test test, Throwable error) { formatError("\tCaused an ERROR", test, error); } @@ -272,7 +264,7 @@ public void addError(Test test, Throwable error) { * @param test a test * @return test name */ - protected String getTestName(Test test) { + private final static String getTestName(Test test) { if (test == null) { return "null"; } else { @@ -287,7 +279,7 @@ protected String getTestName(Test test) { * @param test a test * @return test full class name */ - protected String getTestCaseClassName(Test test) { + private final static String getTestCaseClassName(Test test) { if (test == null) { return "null"; } else { @@ -300,7 +292,7 @@ protected String getTestCaseClassName(Test test) { * @param test a test * @return the formatted testname */ - protected String formatTest(Test test) { + private final static String formatTest(Test test) { if (test == null) { return "Null Test: "; } else { @@ -314,7 +306,7 @@ protected String formatTest(Test test) { * @param test the test that failed * @param error the exception that the test threw */ - protected synchronized void formatError(String type, Test test, + private final void formatError(String type, Test test, Throwable error) { if (test != null) { failedTests.put(test, test); @@ -337,17 +329,18 @@ protected synchronized void formatError(String type, Test test, resultWriter.println(); */ } - protected String filterErrorTrace(Test test, Throwable error) { + private final static String filterErrorTrace(Test test, Throwable error) { String trace = StringUtils.getStackTrace(error); StringWriter sw = new StringWriter(); PrintWriter pw = new PrintWriter(sw); StringReader sr = new StringReader(trace); BufferedReader br = new BufferedReader(sr); + String testCaseClassName = getTestCaseClassName(test); String line; try { while ((line = br.readLine()) != null) { - if (line.indexOf(getTestCaseClassName(test)) != -1) { + if (line.indexOf(testCaseClassName) != -1) { Matcher matcher = traceLinePattern.matcher(line); // pw.println(matcher + ": " + matcher.find()); if (matcher.find()) { From 1324e223017cdfb0de09cb6cf3cd574ed75d0f33 Mon Sep 17 00:00:00 2001 From: Eric Schwartz Date: Wed, 28 Nov 2018 12:00:00 -0700 Subject: [PATCH 29/63] re-enabling OneLinerFormatter --- build.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.xml b/build.xml index 38204fbade2..27f6f08ff57 100644 --- a/build.xml +++ b/build.xml @@ -1300,7 +1300,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + From 290f6487b6c83c4cbe3a2afd2b0761c4aebd305f Mon Sep 17 00:00:00 2001 From: Eric Schwartz Date: Wed, 28 Nov 2018 12:24:30 -0700 Subject: [PATCH 30/63] Flush OneLinerFormatter more often, to see if we can isolate if, or where, it is crashing --- .../com/undefined/testing/OneLinerFormatter.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java b/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java index 57aa6e40762..5d9d4b1c51e 100644 --- a/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java +++ b/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java @@ -168,6 +168,10 @@ public synchronized void endTestSuite(JUnitTest suite) { sb.append(StringUtils.LINE_SEP); sb.append(StringUtils.LINE_SEP); + output.write(sb.toString()); + output.flush(); + sb = new StringBuffer(); + // append the err and output streams to the log if (systemOutput != null && systemOutput.length() > 0) { sb.append(runtimeName) @@ -178,6 +182,10 @@ public synchronized void endTestSuite(JUnitTest suite) { .append(runtimeName) .append("------------- ---------------- ---------------") .append(StringUtils.LINE_SEP); + + output.write(sb.toString()); + output.flush(); + sb = new StringBuffer(); } if (systemError != null && systemError.length() > 0) { @@ -189,6 +197,10 @@ public synchronized void endTestSuite(JUnitTest suite) { .append(runtimeName) .append("------------- ---------------- ---------------") .append(StringUtils.LINE_SEP); + + output.write(sb.toString()); + output.flush(); + sb = new StringBuffer(); } if (output != null) { From 970d4cadb49b39df4f82702121d079392559a09a Mon Sep 17 00:00:00 2001 From: Eric Schwartz Date: Wed, 28 Nov 2018 12:56:35 -0700 Subject: [PATCH 31/63] Adding a bunch of printlns to QuorumZxidSyncTest to figure out what's crashing --- .../zookeeper/test/QuorumZxidSyncTest.java | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumZxidSyncTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumZxidSyncTest.java index 6e46edc3e32..7855eea2e06 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumZxidSyncTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumZxidSyncTest.java @@ -36,6 +36,7 @@ public class QuorumZxidSyncTest extends ZKTestCase { @Before public void setUp() throws Exception { + System.out.println("QuorumZxidSyncTest#setUp()"); qb.setUp(); } @@ -44,6 +45,7 @@ public void setUp() throws Exception { */ @Test public void testBehindLeader() throws Exception { + System.out.println("QuorumZxidSyncTest#testBehindLeader() - start"); // crank up the epoch numbers ClientBase.waitForServerUp(qb.hostPort, 10000); ClientBase.waitForServerUp(qb.hostPort, 10000); @@ -53,9 +55,11 @@ public void process(WatchedEvent event) { zk.create("/0", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); zk.close(); qb.shutdownServers(); + System.out.println("QuorumZxidSyncTest#testBehindLeader() - shutdown #1"); qb.startServers(); ClientBase.waitForServerUp(qb.hostPort, 10000); qb.shutdownServers(); + System.out.println("QuorumZxidSyncTest#testBehindLeader() - shutdown #2"); qb.startServers(); ClientBase.waitForServerUp(qb.hostPort, 10000); zk = new ZooKeeper(qb.hostPort, 10000, new Watcher() { @@ -64,6 +68,7 @@ public void process(WatchedEvent event) { zk.create("/1", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); zk.close(); qb.shutdownServers(); + System.out.println("QuorumZxidSyncTest#testBehindLeader() - shutdown #3"); qb.startServers(); ClientBase.waitForServerUp(qb.hostPort, 10000); qb.shutdownServers(); @@ -75,6 +80,7 @@ public void process(WatchedEvent event) { zk.create("/2", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); zk.close(); qb.shutdownServers(); + System.out.println("QuorumZxidSyncTest#testBehindLeader() - shutdown #4"); deleteFiles(qb.s1dir); deleteFiles(qb.s2dir); deleteFiles(qb.s3dir); @@ -89,6 +95,7 @@ public void process(WatchedEvent event) { String hostPort = "127.0.0.1:" + qb.s5.getClientPort(); Assert.assertFalse("Servers came up, but shouldn't have since it's ahead of leader", ClientBase.waitForServerUp(hostPort, 10000)); + System.out.println("QuorumZxidSyncTest#testBehindLeader() - stop"); } private void deleteFiles(File f) { @@ -104,6 +111,7 @@ private void deleteFiles(File f) { */ @Test public void testLateLogs() throws Exception { + System.out.println("QuorumZxidSyncTest#testLateLogs() - start"); // crank up the epoch numbers ClientBase.waitForServerUp(qb.hostPort, 10000); ClientBase.waitForServerUp(qb.hostPort, 10000); @@ -113,9 +121,11 @@ public void process(WatchedEvent event) { zk.create("/0", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); zk.close(); qb.shutdownServers(); + System.out.println("QuorumZxidSyncTest#testLateLogs() - shutdown #1"); qb.startServers(); ClientBase.waitForServerUp(qb.hostPort, 10000); qb.shutdownServers(); + System.out.println("QuorumZxidSyncTest#testLateLogs() - shutdown #2"); qb.startServers(); ClientBase.waitForServerUp(qb.hostPort, 10000); zk = new ZooKeeper(qb.hostPort, 10000, new Watcher() { @@ -124,9 +134,11 @@ public void process(WatchedEvent event) { zk.create("/1", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); zk.close(); qb.shutdownServers(); + System.out.println("QuorumZxidSyncTest#testLateLogs() - shutdown #3"); qb.startServers(); ClientBase.waitForServerUp(qb.hostPort, 10000); qb.shutdownServers(); + System.out.println("QuorumZxidSyncTest#testLateLogs() - shutdown #4"); deleteLogs(qb.s1dir); deleteLogs(qb.s2dir); deleteLogs(qb.s3dir); @@ -140,6 +152,7 @@ public void process(WatchedEvent event) { zk.create("/2", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); zk.close(); qb.shutdownServers(); + System.out.println("QuorumZxidSyncTest#testLateLogs() - shutdown #5"); qb.startServers(); ClientBase.waitForServerUp(qb.hostPort, 10000); zk = new ZooKeeper(qb.hostPort, 10000, new Watcher() { @@ -153,6 +166,7 @@ public void process(WatchedEvent event) { } zk.close(); Assert.assertTrue("Didn't see /2 (went back in time)", saw2); + System.out.println("QuorumZxidSyncTest#testLateLogs() - start"); } private void deleteLogs(File f) { @@ -166,6 +180,8 @@ private void deleteLogs(File f) { @After public void tearDown() throws Exception { + System.out.println("QuorumZxidSyncTest#tearDown() - start"); qb.tearDown(); + System.out.println("QuorumZxidSyncTest#tearDown() - stop"); } } From 3de8163fc9117dc6cfff672b40752cb59cfaa7fd Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Wed, 28 Nov 2018 14:46:11 -0800 Subject: [PATCH 32/63] Clearer assertion failures in QuorumPeerMainTest --- .../server/quorum/QuorumPeerMainTest.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) 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 72fa13c1e96..30481a6c0a2 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 @@ -215,8 +215,8 @@ public void testEarlyLeaderAbandonment() throws Exception { // just make sure that we actually did get it in process at the // leader - Assert.assertTrue(outstanding.size() == 1); - Assert.assertTrue(((Proposal) outstanding.values().iterator().next()).request.getHdr().getType() == OpCode.create); + Assert.assertEquals(1, outstanding.size()); + Assert.assertEquals(OpCode.create, ((Proposal) outstanding.values().iterator().next()).request.getHdr().getType()); // make sure it has a chance to write it to disk Thread.sleep(1000); mt[leader].shutdown(); @@ -241,9 +241,9 @@ public void testEarlyLeaderAbandonment() throws Exception { for (int i = 0; i < SERVER_COUNT; i++) { for (int j = 0; j < SERVER_COUNT; j++) { if (i == leader) { - Assert.assertTrue((j == leader ? ("Leader (" + leader + ")") : ("Follower " + j)) + " should not have /zk" + i, zk[j].exists("/zk" + i, false) == null); + Assert.assertNull((j == leader ? ("Leader (" + leader + ")") : ("Follower " + j)) + " should not have /zk" + i, zk[j].exists("/zk" + i, false)); } else { - Assert.assertTrue((j == leader ? ("Leader (" + leader + ")") : ("Follower " + j)) + " does not have /zk" + i, zk[j].exists("/zk" + i, false) != null); + Assert.assertNotNull((j == leader ? ("Leader (" + leader + ")") : ("Follower " + j)) + " does not have /zk" + i, zk[j].exists("/zk" + i, false)); } } } @@ -378,7 +378,7 @@ public void testElectionFraud() throws IOException, InterruptedException { // find a follower int falseLeader = (trueLeader + 1) % numServers; - Assert.assertTrue("All servers should join the quorum", servers.mt[falseLeader].main.quorumPeer.follower != null); + Assert.assertNotNull("All servers should join the quorum", servers.mt[falseLeader].main.quorumPeer.follower); // 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 @@ -405,7 +405,7 @@ public void testElectionFraud() throws IOException, InterruptedException { waitForOne(servers.zk[falseLeader], States.CONNECTED); // and ensure trueLeader is still the leader - Assert.assertTrue(servers.mt[trueLeader].main.quorumPeer.leader != null); + Assert.assertNotNull(servers.mt[trueLeader].main.quorumPeer.leader); // Look through the logs for output that indicates the falseLeader is LEADING, then LOOKING, then FOLLOWING LineNumberReader r = new LineNumberReader(new StringReader(os.toString())); @@ -1074,13 +1074,13 @@ public void testLeaderOutOfView() throws Exception { // Node 1 must be started first, before quorum is formed, to trigger the attempted invalid connection to 3 svrs.mt[0].start(); QuorumPeer quorumPeer1 = waitForQuorumPeer(svrs.mt[0], CONNECTION_TIMEOUT); - Assert.assertTrue(quorumPeer1.getPeerState() == QuorumPeer.ServerState.LOOKING); + Assert.assertEquals(QuorumPeer.ServerState.LOOKING, quorumPeer1.getPeerState()); // Node 3 started second to avoid 1 and 2 forming a quorum before 3 starts up int highestServerIndex = numServers - 1; svrs.mt[highestServerIndex].start(); QuorumPeer quorumPeer3 = waitForQuorumPeer(svrs.mt[highestServerIndex], CONNECTION_TIMEOUT); - Assert.assertTrue(quorumPeer3.getPeerState() == QuorumPeer.ServerState.LOOKING); + Assert.assertEquals(QuorumPeer.ServerState.LOOKING, quorumPeer3.getPeerState()); // Node 2 started last, kicks off leader election for (int i = 1; i < highestServerIndex; i++) { @@ -1093,10 +1093,10 @@ public void testLeaderOutOfView() throws Exception { ClientBase.waitForServerUp("127.0.0.1:" + svrs.clientPorts[i], CONNECTION_TIMEOUT)); } - Assert.assertTrue(svrs.mt[0].getQuorumPeer().getPeerState() == QuorumPeer.ServerState.LOOKING); + Assert.assertEquals(QuorumPeer.ServerState.LOOKING, svrs.mt[0].getQuorumPeer().getPeerState()); Assert.assertTrue(svrs.mt[highestServerIndex].getQuorumPeer().getPeerState() == QuorumPeer.ServerState.LEADING); for (int i = 1; i < highestServerIndex; i++) { - Assert.assertTrue(svrs.mt[i].getQuorumPeer().getPeerState() == QuorumPeer.ServerState.FOLLOWING); + Assert.assertEquals(QuorumPeer.ServerState.FOLLOWING, svrs.mt[i].getQuorumPeer().getPeerState()); } // Look through the logs for output that indicates Node 1 is LEADING or FOLLOWING From 27ecd1291c6ac2e3833fbdb74ac65e017cb9b3a0 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Thu, 29 Nov 2018 00:12:21 +0000 Subject: [PATCH 33/63] Clean up (a little better) after tests in QuorumPeerMainTest --- .../server/quorum/QuorumPeerMainTest.java | 216 ++++++++++-------- 1 file changed, 118 insertions(+), 98 deletions(-) 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 30481a6c0a2..cdb150b7f40 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 @@ -82,6 +82,9 @@ public void tearDown() throws Exception { return; } for (int i = 0; i < numServers; i++) { + if (i < servers.zk.length) { + servers.zk[i].close(); + } if (i < servers.mt.length) { servers.mt[i].shutdown(); } @@ -109,40 +112,42 @@ public void testQuorum() throws Exception { q1.start(); q2.start(); - Assert.assertTrue("waiting for server 1 being up", - ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP1, - CONNECTION_TIMEOUT)); - Assert.assertTrue("waiting for server 2 being up", - ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP2, - CONNECTION_TIMEOUT)); - QuorumPeer quorumPeer = q1.main.quorumPeer; - - int tickTime = quorumPeer.getTickTime(); - Assert.assertEquals( - "Default value of minimumSessionTimeOut is not considered", - tickTime * 2, quorumPeer.getMinSessionTimeout()); - Assert.assertEquals( - "Default value of maximumSessionTimeOut is not considered", - tickTime * 20, quorumPeer.getMaxSessionTimeout()); + try { + Assert.assertTrue("waiting for server 1 being up", + ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP1, + CONNECTION_TIMEOUT)); + Assert.assertTrue("waiting for server 2 being up", + ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP2, + CONNECTION_TIMEOUT)); + QuorumPeer quorumPeer = q1.main.quorumPeer; - ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT_QP1, - ClientBase.CONNECTION_TIMEOUT, this); - waitForOne(zk, States.CONNECTED); - zk.create("/foo_q1", "foobar1".getBytes(), Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT); - Assert.assertEquals(new String(zk.getData("/foo_q1", null, null)), "foobar1"); - zk.close(); + int tickTime = quorumPeer.getTickTime(); + Assert.assertEquals( + "Default value of minimumSessionTimeOut is not considered", + tickTime * 2, quorumPeer.getMinSessionTimeout()); + Assert.assertEquals( + "Default value of maximumSessionTimeOut is not considered", + tickTime * 20, quorumPeer.getMaxSessionTimeout()); - zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT_QP2, - ClientBase.CONNECTION_TIMEOUT, this); - waitForOne(zk, States.CONNECTED); - zk.create("/foo_q2", "foobar2".getBytes(), Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT); - Assert.assertEquals(new String(zk.getData("/foo_q2", null, null)), "foobar2"); - zk.close(); + ZooKeeper zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT_QP1, + ClientBase.CONNECTION_TIMEOUT, this); + waitForOne(zk, States.CONNECTED); + zk.create("/foo_q1", "foobar1".getBytes(), Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + Assert.assertEquals(new String(zk.getData("/foo_q1", null, null)), "foobar1"); + zk.close(); - q1.shutdown(); - q2.shutdown(); + zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT_QP2, + ClientBase.CONNECTION_TIMEOUT, this); + waitForOne(zk, States.CONNECTED); + zk.create("/foo_q2", "foobar2".getBytes(), Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + Assert.assertEquals(new String(zk.getData("/foo_q2", null, null)), "foobar2"); + zk.close(); + } finally { + q1.shutdown(); + q2.shutdown(); + } Assert.assertTrue("waiting for server 1 down", ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP1, @@ -175,83 +180,91 @@ public void testEarlyLeaderAbandonment() throws Exception { zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i], ClientBase.CONNECTION_TIMEOUT, this); } - waitForAll(zk, States.CONNECTED); - - // we need to shutdown and start back up to make sure that the create session isn't the first transaction since - // that is rather innocuous. - for (int i = 0; i < SERVER_COUNT; i++) { - mt[i].shutdown(); - } - - waitForAll(zk, States.CONNECTING); - - for (int i = 0; i < SERVER_COUNT; i++) { - mt[i].start(); - // Recreate a client session since the previous session was not persisted. - zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i], ClientBase.CONNECTION_TIMEOUT, this); - } - - waitForAll(zk, States.CONNECTED); - + try { + waitForAll(zk, States.CONNECTED); - // ok lets find the leader and kill everything else, we have a few - // seconds, so it should be plenty of time - int leader = -1; - Map outstanding = null; - for (int i = 0; i < SERVER_COUNT; i++) { - if (mt[i].main.quorumPeer.leader == null) { + // we need to shutdown and start back up to make sure that the create session isn't the first transaction since + // that is rather innocuous. + for (int i = 0; i < SERVER_COUNT; i++) { mt[i].shutdown(); - } else { - leader = i; - outstanding = mt[leader].main.quorumPeer.leader.outstandingProposals; } - } - try { - zk[leader].create("/zk" + leader, "zk".getBytes(), Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT); - Assert.fail("create /zk" + leader + " should have failed"); - } catch (KeeperException e) {} + waitForAll(zk, States.CONNECTING); - // just make sure that we actually did get it in process at the - // leader - Assert.assertEquals(1, outstanding.size()); - Assert.assertEquals(OpCode.create, ((Proposal) outstanding.values().iterator().next()).request.getHdr().getType()); - // make sure it has a chance to write it to disk - Thread.sleep(1000); - mt[leader].shutdown(); - waitForAll(zk, States.CONNECTING); - for (int i = 0; i < SERVER_COUNT; i++) { - if (i != leader) { + for (int i = 0; i < SERVER_COUNT; i++) { + zk[i].close(); mt[i].start(); - } - } - for (int i = 0; i < SERVER_COUNT; i++) { - if (i != leader) { // Recreate a client session since the previous session was not persisted. zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i], ClientBase.CONNECTION_TIMEOUT, this); - waitForOne(zk[i], States.CONNECTED); - zk[i].create("/zk" + i, "zk".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); } - } - mt[leader].start(); - waitForAll(zk, States.CONNECTED); - // make sure everything is consistent - for (int i = 0; i < SERVER_COUNT; i++) { - for (int j = 0; j < SERVER_COUNT; j++) { - if (i == leader) { - Assert.assertNull((j == leader ? ("Leader (" + leader + ")") : ("Follower " + j)) + " should not have /zk" + i, zk[j].exists("/zk" + i, false)); + waitForAll(zk, States.CONNECTED); + + // ok lets find the leader and kill everything else, we have a few + // seconds, so it should be plenty of time + int leader = -1; + Map outstanding = null; + for (int i = 0; i < SERVER_COUNT; i++) { + if (mt[i].main.quorumPeer.leader == null) { + mt[i].shutdown(); } else { - Assert.assertNotNull((j == leader ? ("Leader (" + leader + ")") : ("Follower " + j)) + " does not have /zk" + i, zk[j].exists("/zk" + i, false)); + leader = i; + outstanding = mt[leader].main.quorumPeer.leader.outstandingProposals; } } - } - for (int i = 0; i < SERVER_COUNT; i++) { - zk[i].close(); - } - for (int i = 0; i < SERVER_COUNT; i++) { - mt[i].shutdown(); + + try { + zk[leader].create("/zk" + leader, "zk".getBytes(), Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + Assert.fail("create /zk" + leader + " should have failed"); + } catch (KeeperException e) {} + + // just make sure that we actually did get it in process at the + // leader + Assert.assertEquals(1, outstanding.size()); + Assert.assertEquals(OpCode.create, ((Proposal) outstanding.values().iterator().next()).request.getHdr().getType()); + // make sure it has a chance to write it to disk + Thread.sleep(1000); + mt[leader].shutdown(); + + waitForAll(zk, States.CONNECTING); + + for (int i = 0; i < SERVER_COUNT; i++) { + if (i != leader) { + zk[i].close(); + mt[i].start(); + } + } + for (int i = 0; i < SERVER_COUNT; i++) { + if (i != leader) { + // Recreate a client session since the previous session was not persisted. + zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i], ClientBase.CONNECTION_TIMEOUT, this); + waitForOne(zk[i], States.CONNECTED); + zk[i].create("/zk" + i, "zk".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + } + } + + mt[leader].start(); + + waitForAll(zk, States.CONNECTED); + + // make sure everything is consistent + for (int i = 0; i < SERVER_COUNT; i++) { + for (int j = 0; j < SERVER_COUNT; j++) { + if (i == leader) { + Assert.assertNull((j == leader ? ("Leader (" + leader + ")") : ("Follower " + j)) + " should not have /zk" + i, zk[j].exists("/zk" + i, false)); + } else { + Assert.assertNotNull((j == leader ? ("Leader (" + leader + ")") : ("Follower " + j)) + " does not have /zk" + i, zk[j].exists("/zk" + i, false)); + } + } + } + } finally { + for (int i = 0; i < SERVER_COUNT; i++) { + zk[i].close(); + } + for (int i = 0; i < SERVER_COUNT; i++) { + mt[i].shutdown(); + } } } @@ -264,6 +277,7 @@ public void testEarlyLeaderAbandonment() throws Exception { public void testHighestZxidJoinLate() throws Exception { numServers = 3; servers = LaunchServers(numServers); + String path = "/hzxidtest"; int leader = servers.findLeader(); @@ -885,6 +899,9 @@ public void testMinMaxSessionTimeOut() throws Exception { minSessionTimeOut, quorumPeer.getMinSessionTimeout()); Assert.assertEquals("maximumSessionTimeOut is not considered", maxSessionTimeOut, quorumPeer.getMaxSessionTimeout()); + + q1.shutdown(); + q2.shutdown(); } /** @@ -927,6 +944,9 @@ public void testWithOnlyMinSessionTimeout() throws Exception { minSessionTimeOut, quorumPeer.getMinSessionTimeout()); Assert.assertEquals("maximumSessionTimeOut is wrong", maxSessionTimeOut, quorumPeer.getMaxSessionTimeout()); + + q1.shutdown(); + q2.shutdown(); } @Test @@ -1242,8 +1262,8 @@ public void testInconsistentDueToNewLeaderOrder() throws Exception { // start servers MainThread[] mt = new MainThread[ENSEMBLE_SERVERS]; - ZooKeeper zk[] = new ZooKeeper[ENSEMBLE_SERVERS]; - Context contexts[] = new Context[ENSEMBLE_SERVERS]; + ZooKeeper[] zk = new ZooKeeper[ENSEMBLE_SERVERS]; + Context[] contexts = new Context[ENSEMBLE_SERVERS]; for (int i = 0; i < ENSEMBLE_SERVERS; i++) { final Context context = new Context(); contexts[i] = context; @@ -1383,8 +1403,8 @@ public void start() { } finally { System.clearProperty(LearnerHandler.FORCE_SNAP_SYNC); for (int i = 0; i < ENSEMBLE_SERVERS; i++) { - mt[i].shutdown(); zk[i].close(); + mt[i].shutdown(); } } } From 2a9158be63fa7ac7c162690c330d106f127bd67b Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Thu, 29 Nov 2018 00:59:10 +0000 Subject: [PATCH 34/63] fix typo in registerLog4jMBeans() --- .../src/main/java/org/apache/zookeeper/jmx/ManagedUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/jmx/ManagedUtil.java b/zookeeper-server/src/main/java/org/apache/zookeeper/jmx/ManagedUtil.java index 43451b03133..ba741b89686 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/jmx/ManagedUtil.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/jmx/ManagedUtil.java @@ -71,7 +71,7 @@ public static void registerLog4jMBeans() throws JMException { // org.apache.log4j.jmx.HierarchyDynamicMBean hdm = new org.apache.log4j.jmx.HierarchyDynamicMBean(); Object hdm = Class.forName("org.apache.log4j.jmx.HierarchyDynamicMBean").getDeclaredConstructor().newInstance(); - ObjectName mbo = new ObjectName("log4j:hiearchy=default"); + ObjectName mbo = new ObjectName("log4j:hierarchy=default"); mbs.registerMBean(hdm, mbo); // Add the root logger to the Hierarchy MBean From ef8c7815e5582aaa1881c2842cb7aad4c3ca7457 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Thu, 29 Nov 2018 00:59:40 +0000 Subject: [PATCH 35/63] TestReadOnlyClient.cc: wait for connection before attempting zoo_get() --- .../zookeeper-client-c/tests/TestReadOnlyClient.cc | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/zookeeper-client/zookeeper-client-c/tests/TestReadOnlyClient.cc b/zookeeper-client/zookeeper-client-c/tests/TestReadOnlyClient.cc index d73f1896cfe..d45095f6200 100644 --- a/zookeeper-client/zookeeper-client-c/tests/TestReadOnlyClient.cc +++ b/zookeeper-client/zookeeper-client-c/tests/TestReadOnlyClient.cc @@ -92,7 +92,9 @@ class Zookeeper_readOnly : public CPPUNIT_NS::TestFixture { ZOO_READONLY); watch.zh = zh; CPPUNIT_ASSERT(zh != 0); - sleep(1); + + CPPUNIT_ASSERT(watch.waitForConnected(zh)); + int len = 1024; char buf[len]; int res = zoo_get(zh, "/", 0, buf, &len, 0); From 683b1d5d624b3cf8524014328a3b228077c002ca Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Thu, 29 Nov 2018 01:13:21 +0000 Subject: [PATCH 36/63] close ZooKeeper client handles in testInitialConfigHasPositiveVersion --- .../src/test/java/org/apache/zookeeper/test/ReconfigTest.java | 3 +++ 1 file changed, 3 insertions(+) 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 6bb48d11680..720c0feee05 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 @@ -903,6 +903,9 @@ public void testInitialConfigHasPositiveVersion() throws Exception { long version = qv.getVersion(); Assert.assertTrue(version == 0x100000000L); } + for (ZooKeeper zk : zkArr) + if (zk != null) + zk.close(); } /** From fd15e75259cf6d458470d27dbdd9f7dcbe59e8f3 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Thu, 29 Nov 2018 02:10:06 +0000 Subject: [PATCH 37/63] correct println --- .../test/java/org/apache/zookeeper/test/QuorumZxidSyncTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumZxidSyncTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumZxidSyncTest.java index 7855eea2e06..6e7aa0663d2 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumZxidSyncTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumZxidSyncTest.java @@ -166,7 +166,7 @@ public void process(WatchedEvent event) { } zk.close(); Assert.assertTrue("Didn't see /2 (went back in time)", saw2); - System.out.println("QuorumZxidSyncTest#testLateLogs() - start"); + System.out.println("QuorumZxidSyncTest#testLateLogs() - stop"); } private void deleteLogs(File f) { From 64ea3b1ffb15c1f5454c3d77e6ceca1c932fbc77 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Thu, 29 Nov 2018 03:52:28 +0000 Subject: [PATCH 38/63] Include test case class name in 'Ran' lines --- .../src/test/java/com/undefined/testing/OneLinerFormatter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java b/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java index 5d9d4b1c51e..b1180619ca4 100644 --- a/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java +++ b/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java @@ -280,7 +280,7 @@ private final static String getTestName(Test test) { if (test == null) { return "null"; } else { - return /* JUnitVersionHelper.getTestCaseClassName(test) + ": " + */ + return JUnitVersionHelper.getTestCaseClassName(test) + ":" + JUnitVersionHelper.getTestCaseName(test); } } From 0aa260500c5f2931bd5313d38ae8b1d89d5f3bc2 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Thu, 29 Nov 2018 17:43:06 +0000 Subject: [PATCH 39/63] add ant-junit4 dependency --- ivy.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/ivy.xml b/ivy.xml index 91742cb119a..797df71e089 100644 --- a/ivy.xml +++ b/ivy.xml @@ -64,6 +64,7 @@ + From 43f811b65ccb942024323c1ec9dab1f32804dd11 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Thu, 29 Nov 2018 17:47:55 +0000 Subject: [PATCH 40/63] Add runtest.sh --- runtest.sh | 3 +++ 1 file changed, 3 insertions(+) create mode 100755 runtest.sh diff --git a/runtest.sh b/runtest.sh new file mode 100755 index 00000000000..c6f4bca0191 --- /dev/null +++ b/runtest.sh @@ -0,0 +1,3 @@ +#!/bin/bash -x + +exec java -Xmx512m -Dtest.junit.threads=1 -Dbuild.test.dir=build/test/tmp -Dlog4j.configuration=file:conf/log4j.properties -Dtest.data.dir=build/test/data -Dzookeeper.DigestAuthenticationProvider.superDigest=super:D/InIHSb7yEEbrWz8b9l71RjZJU= -classpath build/test/classes:build/test/lib/accessors-smart-1.2.jar:build/test/lib/ant-1.10.5.jar:build/test/lib/ant-junit-1.10.5.jar:build/test/lib/ant-junit4-1.10.5.jar:build/test/lib/ant-launcher-1.10.5.jar:build/test/lib/antlr-2.7.7.jar:build/test/lib/antlr4-runtime-4.5.3.jar:build/test/lib/asm-5.0.4.jar:build/test/lib/audience-annotations-0.5.0.jar:build/test/lib/bcpkix-jdk15on-1.60.jar:build/test/lib/bcprov-jdk15on-1.60.jar:build/test/lib/checkstyle-7.1.2.jar:build/test/lib/commons-beanutils-1.9.3.jar:build/test/lib/commons-cli-1.2.jar:build/test/lib/commons-collections-3.2.2.jar:build/test/lib/commons-io-2.6.jar:build/test/lib/commons-logging-1.2.jar:build/test/lib/guava-19.0.jar:build/test/lib/hamcrest-all-1.3.jar:build/test/lib/hamcrest-core-1.3.jar:build/test/lib/jackson-annotations-2.9.0.jar:build/test/lib/jackson-core-2.9.7.jar:build/test/lib/jackson-databind-2.9.7.jar:build/test/lib/javax.servlet-api-3.1.0.jar:build/test/lib/jcip-annotations-1.0-1.jar:build/test/lib/jetty-http-9.4.14.v20181114.jar:build/test/lib/jetty-io-9.4.14.v20181114.jar:build/test/lib/jetty-security-9.4.14.v20181114.jar:build/test/lib/jetty-server-9.4.14.v20181114.jar:build/test/lib/jetty-servlet-9.4.14.v20181114.jar:build/test/lib/jetty-util-9.4.14.v20181114.jar:build/test/lib/jline-2.11.jar:build/test/lib/json-smart-2.3.jar:build/test/lib/junit-4.12.jar:build/test/lib/kerb-admin-1.1.0.jar:build/test/lib/kerb-client-1.1.0.jar:build/test/lib/kerb-common-1.1.0.jar:build/test/lib/kerb-core-1.1.0.jar:build/test/lib/kerb-crypto-1.1.0.jar:build/test/lib/kerb-identity-1.1.0.jar:build/test/lib/kerb-server-1.1.0.jar:build/test/lib/kerb-simplekdc-1.1.0.jar:build/test/lib/kerb-util-1.1.0.jar:build/test/lib/kerby-asn1-1.1.0.jar:build/test/lib/kerby-config-1.1.0.jar:build/test/lib/kerby-pkix-1.1.0.jar:build/test/lib/kerby-util-1.1.0.jar:build/test/lib/kerby-xdr-1.1.0.jar:build/test/lib/log4j-1.2.17.jar:build/test/lib/mockito-all-1.8.5.jar:build/test/lib/netty-3.10.6.Final.jar:build/test/lib/nimbus-jose-jwt-4.41.2.jar:build/test/lib/slf4j-api-1.7.25.jar:build/test/lib/slf4j-log4j12-1.7.25.jar:build/test/lib/token-provider-1.1.0.jar:build/classes org.apache.tools.ant.taskdefs.optional.junit.JUnitTestRunner $1 skipNonTests=false filtertrace=true haltOnError=false haltOnFailure=false showoutput=true outputtoformatters=true logfailedtests=true threadid=0 logtestlistenerevents=true formatter=com.undefined.testing.OneLinerFormatter From d5e84d222675d7185b213125289813c851f3e642 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Thu, 29 Nov 2018 18:21:24 +0000 Subject: [PATCH 41/63] fix leaked client in QuorumPeerMainTest.testElectionFraud --- .../org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java | 1 + 1 file changed, 1 insertion(+) 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 cdb150b7f40..6bcd353fa03 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 @@ -413,6 +413,7 @@ public void testElectionFraud() throws IOException, InterruptedException { servers.mt[falseLeader].main.quorumPeer.startLeaderElection(); // The previous client connection to falseLeader likely closed, create a new one + servers.zk[falseLeader].close(); servers.zk[falseLeader] = new ZooKeeper("127.0.0.1:" + servers.mt[falseLeader].getClientPort(), ClientBase.CONNECTION_TIMEOUT, this); // Wait for falseLeader to rejoin the quorum From 56e40b0a5e8b2c0dcdd4033e8f9a43dbd3868d10 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Thu, 29 Nov 2018 11:27:32 -0800 Subject: [PATCH 42/63] QuorumPeer: add retry delay in busy-wait cycle in LOOKING state --- .../apache/zookeeper/server/quorum/QuorumPeer.java | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) 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 fa6a456f2d7..e3c7d5f6c7f 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 @@ -1221,7 +1221,12 @@ public void run() { startLeaderElection(); } } - setCurrentVote(makeLEStrategy().lookForLeader()); + Vote vote = makeLEStrategy().lookForLeader(); + setCurrentVote(vote); + if (vote == null) { + // Upper-bound retry delay to 2 seconds + sleep(Math.min(2000, tickTime)); + } } catch (Exception e) { LOG.warn("Unexpected exception", e); setPeerState(ServerState.LOOKING); @@ -1240,7 +1245,12 @@ public void run() { startLeaderElection(); } } - setCurrentVote(makeLEStrategy().lookForLeader()); + Vote vote = makeLEStrategy().lookForLeader(); + setCurrentVote(vote); + if (vote == null) { + // Upper-bound retry delay to 2 seconds + sleep(Math.min(2000, tickTime)); + } } catch (Exception e) { LOG.warn("Unexpected exception", e); setPeerState(ServerState.LOOKING); From 7c4e3ed717be7b19712ea2c1ed29a63b46fa45bd Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Thu, 29 Nov 2018 20:18:13 +0000 Subject: [PATCH 43/63] PemReaderTest: Use ZKParameterized test runner --- .../src/test/java/org/apache/zookeeper/util/PemReaderTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/util/PemReaderTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/util/PemReaderTest.java index d0d3dc7eef0..7924c2e8401 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/util/PemReaderTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/util/PemReaderTest.java @@ -31,12 +31,14 @@ import org.apache.zookeeper.common.KeyStoreFileType; import org.apache.zookeeper.common.X509KeyType; import org.apache.zookeeper.common.X509TestContext; +import org.apache.zookeeper.ZKParameterized; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @RunWith(Parameterized.class) +@Parameterized.UseParametersRunnerFactory(ZKParameterized.RunnerFactory.class) public class PemReaderTest extends BaseX509ParameterizedTestCase { @Parameterized.Parameters From 4c23f4e8a267fa419e28fbc94114eab4217de607 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Thu, 29 Nov 2018 21:00:42 +0000 Subject: [PATCH 44/63] take noise out of ChrootClientTest --- .../java/org/apache/zookeeper/test/ChrootClientTest.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ChrootClientTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ChrootClientTest.java index e0286e16e37..e4156422841 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ChrootClientTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ChrootClientTest.java @@ -32,12 +32,8 @@ public class ChrootClientTest extends ClientTest { public void setUp() throws Exception { String hp = hostPort; hostPort = hostPort + "/chrootclienttest"; - - System.out.println(hostPort); super.setUp(); - LOG.info("STARTING " + getTestName()); - ZooKeeper zk = createClient(hp); try { zk.create("/chrootclienttest", null, Ids.OPEN_ACL_UNSAFE, @@ -46,9 +42,4 @@ public void setUp() throws Exception { zk.close(); } } - - @Test - public void testPing() throws Exception { - // not necessary to repeat this, expensive and not chroot related - } } From b74dd402504a44a940a9a3d96654e798b9b030b5 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Fri, 30 Nov 2018 02:52:05 +0000 Subject: [PATCH 45/63] QuorumPeer.run(): back off when an operation (e. g., followLeader()) fails --- .../zookeeper/server/quorum/Follower.java | 5 +- .../zookeeper/server/quorum/Observer.java | 6 +- .../zookeeper/server/quorum/QuorumPeer.java | 63 ++++++++++++++++--- 3 files changed, 62 insertions(+), 12 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 65086017aac..e52de116fdf 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 @@ -60,9 +60,10 @@ public String toString() { /** * the main method called by the follower to follow the leader * + * @return true if it ran to completion, false otherwise * @throws InterruptedException */ - void followLeader() throws InterruptedException { + boolean followLeader() throws InterruptedException { self.end_fle = Time.currentElapsedTime(); long electionTimeTaken = self.end_fle - self.start_fle; self.setElectionTimeTaken(electionTimeTaken); @@ -102,10 +103,12 @@ void followLeader() throws InterruptedException { // clear pending revalidations pendingRevalidations.clear(); + return false; } } finally { zk.unregisterJMX((Learner)this); } + return true; } /** diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Observer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Observer.java index f0f724e5e6f..e3ea66297e5 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Observer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Observer.java @@ -57,9 +57,11 @@ public String toString() { /** * the main method called by the observer to observe the leader + * + * @return true if it ran to completion, false otherwise * @throws Exception */ - void observeLeader() throws Exception { + boolean observeLeader() throws Exception { zk.registerJMX(new ObserverBean(this, zk), self.jmxLocalPeerBean); try { @@ -87,10 +89,12 @@ void observeLeader() throws Exception { // clear pending revalidations pendingRevalidations.clear(); + return false; } } finally { zk.unregisterJMX(this); } + return true; } /** 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 e3c7d5f6c7f..a8a86a9ee08 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 @@ -1227,9 +1227,16 @@ public void run() { // Upper-bound retry delay to 2 seconds sleep(Math.min(2000, tickTime)); } + } catch (InterruptedException e) { + LOG.warn("Interrupted by exception", e); } catch (Exception e) { LOG.warn("Unexpected exception", e); setPeerState(ServerState.LOOKING); + try { + sleep(Math.min(2000, tickTime)); + } catch (InterruptedException ie) { + LOG.warn("Interrupted by exception", ie); + } } finally { // If the thread is in the the grace period, interrupt // to come out of waiting. @@ -1251,9 +1258,16 @@ public void run() { // Upper-bound retry delay to 2 seconds sleep(Math.min(2000, tickTime)); } + } catch (InterruptedException e) { + LOG.warn("Interrupted by exception", e); } catch (Exception e) { LOG.warn("Unexpected exception", e); setPeerState(ServerState.LOOKING); + try { + sleep(Math.min(2000, tickTime)); + } catch (InterruptedException ie) { + LOG.warn("Interrupted by exception", ie); + } } } break; @@ -1261,26 +1275,48 @@ public void run() { try { LOG.info("OBSERVING"); setObserver(makeObserver(logFactory)); - observer.observeLeader(); + boolean succeeded = observer.observeLeader(); + if (!succeeded) { + LOG.warn("Failed to observe the leader"); + sleep(Math.min(2000, tickTime)); + } + } catch (InterruptedException e) { + LOG.warn("Interrupted by exception", e); } catch (Exception e) { - LOG.warn("Unexpected exception",e ); + LOG.warn("Unexpected exception", e); + try { + sleep(Math.min(2000, tickTime)); + } catch (InterruptedException ie) { + LOG.warn("Interrupted by exception", ie); + } } finally { observer.shutdown(); setObserver(null); - updateServerState(); + updateServerState(); } break; case FOLLOWING: try { - LOG.info("FOLLOWING"); + LOG.info("FOLLOWING"); setFollower(makeFollower(logFactory)); - follower.followLeader(); + boolean succeeded = follower.followLeader(); + if (!succeeded) { + LOG.warn("Failed to follow the leader"); + sleep(Math.min(2000, tickTime)); + } + } catch (InterruptedException e) { + LOG.warn("Interrupted by exception", e); } catch (Exception e) { - LOG.warn("Unexpected exception",e); + LOG.warn("Unexpected exception", e); + try { + sleep(Math.min(2000, tickTime)); + } catch (InterruptedException ie) { + LOG.warn("Interrupted by exception", ie); + } } finally { - follower.shutdown(); - setFollower(null); - updateServerState(); + follower.shutdown(); + setFollower(null); + updateServerState(); } break; case LEADING: @@ -1289,8 +1325,15 @@ public void run() { setLeader(makeLeader(logFactory)); leader.lead(); setLeader(null); + } catch (InterruptedException e) { + LOG.warn("Interrupted by exception", e); } catch (Exception e) { - LOG.warn("Unexpected exception",e); + LOG.warn("Unexpected exception", e); + try { + sleep(Math.min(2000, tickTime)); + } catch (InterruptedException ie) { + LOG.warn("Interrupted by exception", ie); + } } finally { if (leader != null) { leader.shutdown("Forcing shutdown"); From ab929f9d4cd5c9a25263a47a3e1d882cc3d32e82 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Fri, 30 Nov 2018 03:44:32 +0000 Subject: [PATCH 46/63] 200ms is sufficient delay between QuorumPeer.run() attempts --- .../zookeeper/server/quorum/QuorumPeer.java | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) 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 a8a86a9ee08..ef453f330d7 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 @@ -1224,8 +1224,8 @@ public void run() { Vote vote = makeLEStrategy().lookForLeader(); setCurrentVote(vote); if (vote == null) { - // Upper-bound retry delay to 2 seconds - sleep(Math.min(2000, tickTime)); + // Upper-bound retry delay to 0.2 seconds + sleep(Math.min(200, tickTime)); } } catch (InterruptedException e) { LOG.warn("Interrupted by exception", e); @@ -1233,7 +1233,7 @@ public void run() { LOG.warn("Unexpected exception", e); setPeerState(ServerState.LOOKING); try { - sleep(Math.min(2000, tickTime)); + sleep(Math.min(200, tickTime)); } catch (InterruptedException ie) { LOG.warn("Interrupted by exception", ie); } @@ -1255,8 +1255,8 @@ public void run() { Vote vote = makeLEStrategy().lookForLeader(); setCurrentVote(vote); if (vote == null) { - // Upper-bound retry delay to 2 seconds - sleep(Math.min(2000, tickTime)); + // Upper-bound retry delay to 0.2 seconds + sleep(Math.min(200, tickTime)); } } catch (InterruptedException e) { LOG.warn("Interrupted by exception", e); @@ -1264,7 +1264,7 @@ public void run() { LOG.warn("Unexpected exception", e); setPeerState(ServerState.LOOKING); try { - sleep(Math.min(2000, tickTime)); + sleep(Math.min(200, tickTime)); } catch (InterruptedException ie) { LOG.warn("Interrupted by exception", ie); } @@ -1278,14 +1278,14 @@ public void run() { boolean succeeded = observer.observeLeader(); if (!succeeded) { LOG.warn("Failed to observe the leader"); - sleep(Math.min(2000, tickTime)); + sleep(Math.min(200, tickTime)); } } catch (InterruptedException e) { LOG.warn("Interrupted by exception", e); } catch (Exception e) { LOG.warn("Unexpected exception", e); try { - sleep(Math.min(2000, tickTime)); + sleep(Math.min(200, tickTime)); } catch (InterruptedException ie) { LOG.warn("Interrupted by exception", ie); } @@ -1302,14 +1302,14 @@ public void run() { boolean succeeded = follower.followLeader(); if (!succeeded) { LOG.warn("Failed to follow the leader"); - sleep(Math.min(2000, tickTime)); + sleep(Math.min(200, tickTime)); } } catch (InterruptedException e) { LOG.warn("Interrupted by exception", e); } catch (Exception e) { LOG.warn("Unexpected exception", e); try { - sleep(Math.min(2000, tickTime)); + sleep(Math.min(200, tickTime)); } catch (InterruptedException ie) { LOG.warn("Interrupted by exception", ie); } @@ -1330,7 +1330,7 @@ public void run() { } catch (Exception e) { LOG.warn("Unexpected exception", e); try { - sleep(Math.min(2000, tickTime)); + sleep(Math.min(200, tickTime)); } catch (InterruptedException ie) { LOG.warn("Interrupted by exception", ie); } From 8e03898904f1aa15e3d30d18a8a097c6416bad92 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Fri, 30 Nov 2018 04:38:08 +0000 Subject: [PATCH 47/63] Fix cross-thread visibility/synchronization issues in QuorumPeerTestBase? --- .../zookeeper/server/quorum/QuorumPeerMain.java | 2 +- .../server/quorum/QuorumPeerTestBase.java | 17 +++++++++-------- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java index 7c22d8157c7..49ed5f3f03c 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java @@ -69,7 +69,7 @@ public class QuorumPeerMain { private static final String USAGE = "Usage: QuorumPeerMain configfile"; - protected QuorumPeer quorumPeer; + protected volatile QuorumPeer quorumPeer; /** * To start the replicated server specify the configuration file name on diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java index ffc00f39a20..6d80e00b2e6 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java @@ -58,10 +58,11 @@ public void process(WatchedEvent event) { } public static class TestQPMain extends QuorumPeerMain { - public void shutdown() { + synchronized public void shutdown() { // ensure it closes - in particular wait for thread to exit - if (quorumPeer != null) { - QuorumBase.shutdown(quorumPeer); + QuorumPeer myPeer = quorumPeer; + if (myPeer != null) { + QuorumBase.shutdown(myPeer); } } } @@ -300,7 +301,7 @@ public MainThread(int myid, int clientPort, String quorumCfgSection, this(myid, clientPort, quorumCfgSection, otherConfigs, 4000); } - Thread currentThread; + volatile Thread currentThread; synchronized public void start() { main = getTestQPMain(); @@ -325,22 +326,22 @@ public void run() { } } - public void shutdown() throws InterruptedException { + synchronized public void shutdown() throws InterruptedException { Thread t = currentThread; if (t != null && t.isAlive()) { main.shutdown(); - t.join(500); + t.join(2000); } } - public void join(long timeout) throws InterruptedException { + synchronized public void join(long timeout) throws InterruptedException { Thread t = currentThread; if (t != null) { t.join(timeout); } } - public boolean isAlive() { + synchronized public boolean isAlive() { Thread t = currentThread; return t != null && t.isAlive(); } From cf6d6525c11c4b64234c30bc9902cc78f8521ed8 Mon Sep 17 00:00:00 2001 From: Eric Schwartz Date: Fri, 30 Nov 2018 10:24:37 -0700 Subject: [PATCH 48/63] Don't allocate as much memory in OneLinerFormatter#prefixLines --- .../undefined/testing/OneLinerFormatter.java | 28 +++++++++++-------- 1 file changed, 16 insertions(+), 12 deletions(-) diff --git a/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java b/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java index b1180619ca4..77dc3d70cb7 100644 --- a/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java +++ b/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java @@ -13,6 +13,7 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; +import java.util.stream.Stream; import junit.framework.AssertionFailedError; import junit.framework.Test; @@ -126,7 +127,6 @@ public synchronized void startTestSuite(JUnitTest suite) { return; // Quick return - no output do nothing. } StringBuffer sb = new StringBuffer(StringUtils.LINE_SEP); - sb.append("USING ONELINERTESTFORMATTER"); sb.append(StringUtils.LINE_SEP); sb.append("----------------------------------------------------------"); sb.append(StringUtils.LINE_SEP); @@ -139,12 +139,12 @@ public synchronized void startTestSuite(JUnitTest suite) { output.flush(); } - public final static String prefixLines(String prefix, String buffer) { + public final static void prefixLines(StringBuffer sb, String prefix, String buffer) { String lines[] = buffer.split("\\r?\\n"); - return Arrays.asList(lines) - .stream() - .map(l -> prefix + l) - .collect(Collectors.joining("\n")); + for(int i = 0; i < lines.length; i++) { + sb.append(prefix) + .append(lines[i]); + } } @@ -176,9 +176,11 @@ public synchronized void endTestSuite(JUnitTest suite) { if (systemOutput != null && systemOutput.length() > 0) { sb.append(runtimeName) .append("------------- Standard Output ---------------") - .append(StringUtils.LINE_SEP) - .append(prefixLines(runtimeName, systemOutput)) - .append(StringUtils.LINE_SEP) + .append(StringUtils.LINE_SEP); + + prefixLines(sb, runtimeName, systemOutput); + + sb.append(StringUtils.LINE_SEP) .append(runtimeName) .append("------------- ---------------- ---------------") .append(StringUtils.LINE_SEP); @@ -191,9 +193,11 @@ public synchronized void endTestSuite(JUnitTest suite) { if (systemError != null && systemError.length() > 0) { sb.append(runtimeName) .append("------------- Standard Error -----------------") - .append(StringUtils.LINE_SEP) - .append(prefixLines(runtimeName, systemError)) - .append(StringUtils.LINE_SEP) + .append(StringUtils.LINE_SEP); + + prefixLines(sb, runtimeName, systemError); + + sb.append(StringUtils.LINE_SEP) .append(runtimeName) .append("------------- ---------------- ---------------") .append(StringUtils.LINE_SEP); From da784f5eb3c8283fda44c769a5b4aeba7e36a9de Mon Sep 17 00:00:00 2001 From: Eric Schwartz Date: Fri, 30 Nov 2018 13:50:00 -0700 Subject: [PATCH 49/63] Forgot to add a newline between lines in prefixLines --- .../src/test/java/com/undefined/testing/OneLinerFormatter.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java b/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java index 77dc3d70cb7..5c64efc1fd6 100644 --- a/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java +++ b/zookeeper-server/src/test/java/com/undefined/testing/OneLinerFormatter.java @@ -143,7 +143,8 @@ public final static void prefixLines(StringBuffer sb, String prefix, String buff String lines[] = buffer.split("\\r?\\n"); for(int i = 0; i < lines.length; i++) { sb.append(prefix) - .append(lines[i]); + .append(lines[i]) + .append(StringUtils.LINE_SEP); } } From d203be47169b1c64e100cb44a738f9a1c8abc8d5 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Sat, 1 Dec 2018 20:19:09 +0000 Subject: [PATCH 50/63] fix shutdown/restart race conditions exposed in ReconfigRollingRestartCompatibilityTest.testRollingRestartWithoutMembershipChange --- .../zookeeper/server/NIOServerCnxn.java | 32 +++++++++++++------ .../zookeeper/server/ServerCnxnFactory.java | 2 +- .../server/quorum/FastLeaderElection.java | 9 ++++-- .../zookeeper/server/quorum/QuorumPeer.java | 16 ++++++---- .../server/quorum/QuorumPeerTestBase.java | 2 +- ...configRollingRestartCompatibilityTest.java | 30 ++++++++++++++--- .../quorum/auth/QuorumAuthUpgradeTest.java | 2 +- 7 files changed, 68 insertions(+), 25 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/NIOServerCnxn.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/NIOServerCnxn.java index 95efbaec530..c95fbe90abb 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/NIOServerCnxn.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/NIOServerCnxn.java @@ -77,7 +77,7 @@ public class NIOServerCnxn extends ServerCnxn { private int sessionTimeout; - private final ZooKeeperServer zkServer; + private volatile ZooKeeperServer zkServer; /** * The number of requests that have been submitted but not yet responded to. @@ -95,11 +95,11 @@ public class NIOServerCnxn extends ServerCnxn { public NIOServerCnxn(ZooKeeperServer zk, SocketChannel sock, SelectionKey sk, NIOServerCnxnFactory factory, SelectorThread selectorThread) throws IOException { - this.zkServer = zk; this.sock = sock; this.sk = sk; this.factory = factory; this.selectorThread = selectorThread; + this.sessionTimeout = factory.sessionlessCnxnTimeout; if (this.factory.login != null) { this.zooKeeperSaslServer = new ZooKeeperSaslServer(factory.login); } @@ -114,7 +114,10 @@ public NIOServerCnxn(ZooKeeperServer zk, SocketChannel sock, InetAddress addr = ((InetSocketAddress) sock.socket() .getRemoteSocketAddress()).getAddress(); authInfo.add(new Id("ip", addr.getHostAddress())); - this.sessionTimeout = factory.sessionlessCnxnTimeout; + /* This write to a volatile member forces visibility of all of the + * above to other threads. + */ + this.zkServer = zk; } /* Send close connection packet to the client, doIO will eventually @@ -377,7 +380,10 @@ void doIO(SelectionKey k) throws InterruptedException { } private void readRequest() throws IOException { - zkServer.processPacket(this, incomingBuffer); + ZooKeeperServer server = zkServer; + if (server != null) { + server.processPacket(this, incomingBuffer); + } } // Only called as callback from zkServer.processPacket() @@ -563,7 +569,11 @@ private boolean readLength(SelectionKey k) throws IOException { * @return true if the server is running, false otherwise. */ boolean isZKServerRunning() { - return zkServer != null && zkServer.isRunning(); + ZooKeeperServer server = zkServer; + if (server == null) { + return false; + } + return server.isRunning(); } public long getOutstandingRequests() { @@ -593,13 +603,14 @@ public String toString() { * Close the cnxn and remove it from the factory cnxns list. */ @Override - public void close() { + synchronized public void close() { if (!factory.removeCnxn(this)) { return; } - if (zkServer != null) { - zkServer.removeCnxn(this); + ZooKeeperServer server = zkServer; + if (server != null) { + server.removeCnxn(this); } if (sk != null) { @@ -775,10 +786,11 @@ public InetAddress getSocketAddress() { @Override protected ServerStats serverStats() { - if (zkServer == null) { + ZooKeeperServer server = zkServer; + if (server == null) { return null; } - return zkServer.serverStats(); + return server.serverStats(); } @Override diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerCnxnFactory.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerCnxnFactory.java index dbe47a261b7..c0a1e8e3bf7 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerCnxnFactory.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerCnxnFactory.java @@ -109,7 +109,7 @@ public abstract void startup(ZooKeeperServer zkServer, boolean startServer) public abstract void start(); - protected ZooKeeperServer zkServer; + protected volatile ZooKeeperServer zkServer; final public void setZooKeeperServer(ZooKeeperServer zks) { this.zkServer = zks; if (zks != null) { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java index 401ca0521bd..b31a217c029 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java @@ -495,13 +495,11 @@ void process(ToSend m) { this.wsThread = new Thread(this.ws, "WorkerSender[myid=" + self.getId() + "]"); - this.wsThread.setDaemon(true); this.wr = new WorkerReceiver(manager); this.wrThread = new Thread(this.wr, "WorkerReceiver[myid=" + self.getId() + "]"); - this.wrThread.setDaemon(true); } /** @@ -518,6 +516,13 @@ void start(){ void halt(){ this.ws.stop = true; this.wr.stop = true; + try { + this.wsThread.join(); + this.wrThread.join(); + } catch (InterruptedException e) { + LOG.warn("Interrupted Exception while waiting for thread to stop: " + + e.toString()); + } } } 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 ef453f330d7..87bbe3a8768 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 @@ -1386,13 +1386,9 @@ private synchronized void updateServerState(){ public void shutdown() { running = false; - if (leader != null) { - leader.shutdown("quorum Peer shutdown"); - } - if (follower != null) { - follower.shutdown(); - } + shutdownServerCnxnFactory(); + if(udpSocket != null) { udpSocket.close(); } @@ -1407,6 +1403,14 @@ public void shutdown() { this.interrupt(); getElectionAlg().shutdown(); } + + if (leader != null) { + leader.shutdown("quorum Peer shutdown"); + } + if (follower != null) { + follower.shutdown(); + } + try { zkDb.close(); } catch (IOException ie) { diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java index 6d80e00b2e6..1bc2d3e0d48 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java @@ -330,7 +330,7 @@ synchronized public void shutdown() throws InterruptedException { Thread t = currentThread; if (t != null && t.isAlive()) { main.shutdown(); - t.join(2000); + t.join(10000); } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigRollingRestartCompatibilityTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigRollingRestartCompatibilityTest.java index 65ec469eea8..cc99ecc6ce2 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigRollingRestartCompatibilityTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/ReconfigRollingRestartCompatibilityTest.java @@ -121,6 +121,12 @@ public void testNoLocalDynamicConfigAndBackupFiles() for (int i = 0; i < serverCount; i++) { mt[i].shutdown(); } + + for (int i = 0; i < serverCount; ++i) { + Assert.assertTrue("waiting for server " + i + " being down", + ClientBase.waitForServerDown("127.0.0.1:" + clientPorts.get(i), + CONNECTION_TIMEOUT)); + } } @Test(timeout = 60000) @@ -141,9 +147,13 @@ public void testRollingRestartWithoutMembershipChange() throws Exception { } for (int i = 0; i < serverCount; ++i) { - Assert.assertTrue("waiting for server " + i + " being up", - ClientBase.waitForServerUp("127.0.0.1:" + clientPorts.get(i), - CONNECTION_TIMEOUT)); + while (mt[i].main == null || mt[i].main.quorumPeer == null) { + Thread.sleep(200); + } + Assert.assertTrue("waiting for server " + i + " being leader or follower", + ClientBase.waitForServerState(mt[i].main.quorumPeer, + CONNECTION_TIMEOUT, + QuorumStats.Provider.LEADING_STATE, QuorumStats.Provider.FOLLOWING_STATE)); } for (int i = 0; i < serverCount; ++i) { @@ -156,6 +166,12 @@ public void testRollingRestartWithoutMembershipChange() throws Exception { for (int i = 0; i < serverCount; i++) { mt[i].shutdown(); } + + for (int i = 0; i < serverCount; ++i) { + Assert.assertTrue("waiting for server " + i + " being down", + ClientBase.waitForServerDown("127.0.0.1:" + clientPorts.get(i), + CONNECTION_TIMEOUT)); + } } @Test(timeout = 90000) @@ -220,9 +236,15 @@ public void testRollingRestartWithMembershipChange() throws Exception { verifyQuorumMembers(mt[i], expectedConfigs); } - for (int i = 0; i < serverCount; ++i) { + for (int i = 0; i < 5; ++i) { mt[i].shutdown(); } + + for (int i = 0; i < 5; ++i) { + Assert.assertTrue("waiting for server " + i + " being down", + ClientBase.waitForServerDown("127.0.0.1:" + clientPorts.get(i), + CONNECTION_TIMEOUT)); + } } // Verify each quorum peer has expected config in its config zNode. diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthUpgradeTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthUpgradeTest.java index 93fcfdb67a4..256639628e7 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthUpgradeTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthUpgradeTest.java @@ -229,7 +229,7 @@ private void restartServer(Map authConfigs, int index, LOG.info("Restarting server myid=" + index); MainThread m = shutdown(index); startServer(m, authConfigs); - Assert.assertTrue("waiting for server" + index + "being up", + Assert.assertTrue("waiting for server " + index + " being up", ClientBase.waitForServerUp("127.0.0.1:" + m.getClientPort(), ClientBase.CONNECTION_TIMEOUT)); watcher.waitForConnected(ClientTest.CONNECTION_TIMEOUT); From c7d7db36175a1f6319dad4a555c284bd441218cc Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Sun, 2 Dec 2018 05:30:06 +0000 Subject: [PATCH 51/63] FastLeaderElection: implement Messenger.halt() so that it can safely be called from the WorkerReceiver thread --- .../server/quorum/FastLeaderElection.java | 44 +++++++++++++------ 1 file changed, 31 insertions(+), 13 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java index b31a217c029..f6b53ce892a 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java @@ -210,11 +210,13 @@ protected class Messenger { class WorkerReceiver extends ZooKeeperThread { volatile boolean stop; QuorumCnxManager manager; + volatile boolean halted; WorkerReceiver(QuorumCnxManager manager) { super("WorkerReceiver"); this.stop = false; this.manager = manager; + this.halted = false; } public void run() { @@ -223,7 +225,7 @@ public void run() { while (!stop) { // Sleeps on receive try { - response = manager.pollRecvQueue(3000, TimeUnit.MILLISECONDS); + response = manager.pollRecvQueue(500, TimeUnit.MILLISECONDS); if(response == null) continue; // The current protocol and two previous generations all send at least 28 bytes @@ -291,6 +293,7 @@ public void run() { if (!rqv.equals(curQV)) { LOG.info("restarting leader election"); self.shuttingDownLE = true; + halted = true; self.getElectionAlg().shutdown(); break; @@ -426,8 +429,10 @@ public void run() { } catch (InterruptedException e) { LOG.warn("Interrupted Exception while waiting for new message" + e.toString()); + break; } } + halted = true; LOG.info("WorkerReceiver is down"); } } @@ -450,7 +455,7 @@ class WorkerSender extends ZooKeeperThread { public void run() { while (!stop) { try { - ToSend m = sendqueue.poll(3000, TimeUnit.MILLISECONDS); + ToSend m = sendqueue.poll(500, TimeUnit.MILLISECONDS); if(m == null) continue; process(m); @@ -483,6 +488,7 @@ void process(ToSend m) { WorkerReceiver wr; Thread wsThread = null; Thread wrThread = null; + boolean halted = false; /** * Constructor of class Messenger. @@ -500,12 +506,13 @@ void process(ToSend m) { this.wrThread = new Thread(this.wr, "WorkerReceiver[myid=" + self.getId() + "]"); + this.wrThread.setDaemon(true); } /** * Starts instances of WorkerSender and WorkerReceiver */ - void start(){ + void start() { this.wsThread.start(); this.wrThread.start(); } @@ -513,18 +520,29 @@ void start(){ /** * Stops instances of WorkerSender and WorkerReceiver */ - void halt(){ - this.ws.stop = true; - this.wr.stop = true; - try { - this.wsThread.join(); - this.wrThread.join(); - } catch (InterruptedException e) { - LOG.warn("Interrupted Exception while waiting for thread to stop: " + - e.toString()); + void halt() { + synchronized (this) { + if (halted) { + return; + } + this.ws.stop = true; + this.wr.stop = true; + try { + this.wsThread.join(); + // Don't attempt to join wrThread, since we might be in wrThread itself + // via its call to FastLeaderElection.shutdown(). Instead, verify that + // the WorkerReceiver has marked itself "halted", which will already be + // true on entry in the case where we're in wrThread itself. + while (!this.wr.halted) { + Thread.sleep(100); + } + } catch (InterruptedException e) { + LOG.warn("Interrupted Exception while waiting for thread to stop: " + + e.toString()); + } + halted = true; } } - } QuorumPeer self; From 13c53fe9b7a6a0949f3f085fed7763b5ad310bdb Mon Sep 17 00:00:00 2001 From: Eric Schwartz Date: Mon, 3 Dec 2018 12:20:29 -0700 Subject: [PATCH 52/63] WIP upgrading mockito & adding spies to QuorumPeerMainTest --- build.xml | 3 ++- ivy.xml | 10 +++++----- runtest.sh | 2 +- .../server/quorum/QuorumPeerMainTest.java | 14 +++++++++++--- .../server/quorum/QuorumPeerTestBase.java | 5 +++++ 5 files changed, 24 insertions(+), 10 deletions(-) diff --git a/build.xml b/build.xml index 27f6f08ff57..55c8ee24ca2 100644 --- a/build.xml +++ b/build.xml @@ -40,7 +40,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + @@ -237,6 +237,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> + diff --git a/ivy.xml b/ivy.xml index 797df71e089..996a476940b 100644 --- a/ivy.xml +++ b/ivy.xml @@ -66,7 +66,7 @@ - @@ -74,7 +74,7 @@ - @@ -85,13 +85,13 @@ - - - diff --git a/runtest.sh b/runtest.sh index c6f4bca0191..7c6b7d4bef5 100755 --- a/runtest.sh +++ b/runtest.sh @@ -1,3 +1,3 @@ #!/bin/bash -x -exec java -Xmx512m -Dtest.junit.threads=1 -Dbuild.test.dir=build/test/tmp -Dlog4j.configuration=file:conf/log4j.properties -Dtest.data.dir=build/test/data -Dzookeeper.DigestAuthenticationProvider.superDigest=super:D/InIHSb7yEEbrWz8b9l71RjZJU= -classpath build/test/classes:build/test/lib/accessors-smart-1.2.jar:build/test/lib/ant-1.10.5.jar:build/test/lib/ant-junit-1.10.5.jar:build/test/lib/ant-junit4-1.10.5.jar:build/test/lib/ant-launcher-1.10.5.jar:build/test/lib/antlr-2.7.7.jar:build/test/lib/antlr4-runtime-4.5.3.jar:build/test/lib/asm-5.0.4.jar:build/test/lib/audience-annotations-0.5.0.jar:build/test/lib/bcpkix-jdk15on-1.60.jar:build/test/lib/bcprov-jdk15on-1.60.jar:build/test/lib/checkstyle-7.1.2.jar:build/test/lib/commons-beanutils-1.9.3.jar:build/test/lib/commons-cli-1.2.jar:build/test/lib/commons-collections-3.2.2.jar:build/test/lib/commons-io-2.6.jar:build/test/lib/commons-logging-1.2.jar:build/test/lib/guava-19.0.jar:build/test/lib/hamcrest-all-1.3.jar:build/test/lib/hamcrest-core-1.3.jar:build/test/lib/jackson-annotations-2.9.0.jar:build/test/lib/jackson-core-2.9.7.jar:build/test/lib/jackson-databind-2.9.7.jar:build/test/lib/javax.servlet-api-3.1.0.jar:build/test/lib/jcip-annotations-1.0-1.jar:build/test/lib/jetty-http-9.4.14.v20181114.jar:build/test/lib/jetty-io-9.4.14.v20181114.jar:build/test/lib/jetty-security-9.4.14.v20181114.jar:build/test/lib/jetty-server-9.4.14.v20181114.jar:build/test/lib/jetty-servlet-9.4.14.v20181114.jar:build/test/lib/jetty-util-9.4.14.v20181114.jar:build/test/lib/jline-2.11.jar:build/test/lib/json-smart-2.3.jar:build/test/lib/junit-4.12.jar:build/test/lib/kerb-admin-1.1.0.jar:build/test/lib/kerb-client-1.1.0.jar:build/test/lib/kerb-common-1.1.0.jar:build/test/lib/kerb-core-1.1.0.jar:build/test/lib/kerb-crypto-1.1.0.jar:build/test/lib/kerb-identity-1.1.0.jar:build/test/lib/kerb-server-1.1.0.jar:build/test/lib/kerb-simplekdc-1.1.0.jar:build/test/lib/kerb-util-1.1.0.jar:build/test/lib/kerby-asn1-1.1.0.jar:build/test/lib/kerby-config-1.1.0.jar:build/test/lib/kerby-pkix-1.1.0.jar:build/test/lib/kerby-util-1.1.0.jar:build/test/lib/kerby-xdr-1.1.0.jar:build/test/lib/log4j-1.2.17.jar:build/test/lib/mockito-all-1.8.5.jar:build/test/lib/netty-3.10.6.Final.jar:build/test/lib/nimbus-jose-jwt-4.41.2.jar:build/test/lib/slf4j-api-1.7.25.jar:build/test/lib/slf4j-log4j12-1.7.25.jar:build/test/lib/token-provider-1.1.0.jar:build/classes org.apache.tools.ant.taskdefs.optional.junit.JUnitTestRunner $1 skipNonTests=false filtertrace=true haltOnError=false haltOnFailure=false showoutput=true outputtoformatters=true logfailedtests=true threadid=0 logtestlistenerevents=true formatter=com.undefined.testing.OneLinerFormatter +exec java -Xmx512m -Dtest.junit.threads=1 -Dbuild.test.dir=build/test/tmp -Dlog4j.configuration=file:conf/log4j.properties -Dtest.data.dir=build/test/data -Dzookeeper.DigestAuthenticationProvider.superDigest=super:D/InIHSb7yEEbrWz8b9l71RjZJU= -classpath build/test/classes:build/test/lib/accessors-smart-1.2.jar:build/test/lib/ant-1.10.5.jar:build/test/lib/ant-junit-1.10.5.jar:build/test/lib/ant-junit4-1.10.5.jar:build/test/lib/ant-launcher-1.10.5.jar:build/test/lib/antlr-2.7.7.jar:build/test/lib/antlr4-runtime-4.5.3.jar:build/test/lib/asm-5.0.4.jar:build/test/lib/audience-annotations-0.5.0.jar:build/test/lib/bcpkix-jdk15on-1.60.jar:build/test/lib/bcprov-jdk15on-1.60.jar:build/test/lib/checkstyle-7.1.2.jar:build/test/lib/commons-beanutils-1.9.3.jar:build/test/lib/commons-cli-1.2.jar:build/test/lib/commons-collections-3.2.2.jar:build/test/lib/commons-io-2.6.jar:build/test/lib/commons-logging-1.2.jar:build/test/lib/guava-19.0.jar:build/test/lib/hamcrest-all-1.3.jar:build/test/lib/hamcrest-core-1.3.jar:build/test/lib/jackson-annotations-2.9.0.jar:build/test/lib/jackson-core-2.9.7.jar:build/test/lib/jackson-databind-2.9.7.jar:build/test/lib/javax.servlet-api-3.1.0.jar:build/test/lib/jcip-annotations-1.0-1.jar:build/test/lib/jetty-http-9.4.14.v20181114.jar:build/test/lib/jetty-io-9.4.14.v20181114.jar:build/test/lib/jetty-security-9.4.14.v20181114.jar:build/test/lib/jetty-server-9.4.14.v20181114.jar:build/test/lib/jetty-servlet-9.4.14.v20181114.jar:build/test/lib/jetty-util-9.4.14.v20181114.jar:build/test/lib/jline-2.11.jar:build/test/lib/json-smart-2.3.jar:build/test/lib/junit-4.12.jar:build/test/lib/kerb-admin-1.1.0.jar:build/test/lib/kerb-client-1.1.0.jar:build/test/lib/kerb-common-1.1.0.jar:build/test/lib/kerb-core-1.1.0.jar:build/test/lib/kerb-crypto-1.1.0.jar:build/test/lib/kerb-identity-1.1.0.jar:build/test/lib/kerb-server-1.1.0.jar:build/test/lib/kerb-simplekdc-1.1.0.jar:build/test/lib/kerb-util-1.1.0.jar:build/test/lib/kerby-asn1-1.1.0.jar:build/test/lib/kerby-config-1.1.0.jar:build/test/lib/kerby-pkix-1.1.0.jar:build/test/lib/kerby-util-1.1.0.jar:build/test/lib/kerby-xdr-1.1.0.jar:build/test/lib/log4j-1.2.17.jar:build/test/lib/mockito-core-2.23.4.jar:build/test/lib/byte-buddy-agent-1.9.3.jar:build/test/lib/byte-buddy-1.9.3.jar:build/test/lib/objenesis-2.6.jar:build/test/lib/netty-3.10.6.Final.jar:build/test/lib/nimbus-jose-jwt-4.41.2.jar:build/test/lib/slf4j-api-1.7.25.jar:build/test/lib/slf4j-log4j12-1.7.25.jar:build/test/lib/token-provider-1.1.0.jar:build/classes org.apache.tools.ant.taskdefs.optional.junit.JUnitTestRunner "$@" skipNonTests=false filtertrace=true haltOnError=false haltOnFailure=false showoutput=true outputtoformatters=true logfailedtests=true threadid=0 logtestlistenerevents=true formatter=com.undefined.testing.OneLinerFormatter 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 6bcd353fa03..880e48dd98f 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 @@ -23,6 +23,8 @@ import static org.mockito.Matchers.any; import static org.mockito.Mockito.doCallRealMethod; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.io.ByteArrayOutputStream; @@ -104,7 +106,7 @@ public void testQuorum() throws Exception { String quorumCfgSection = "server.1=127.0.0.1:" + PortAssignment.unique() + ":" + PortAssignment.unique() + ";" + CLIENT_PORT_QP1 - + "\nserver.2=127.0.0.1:" + PortAssignment.unique() + + "\nserver.2=127.0.0.1:" + PortAssignment.unique() + ":" + PortAssignment.unique() + ";" + CLIENT_PORT_QP2; MainThread q1 = new MainThread(1, CLIENT_PORT_QP1, quorumCfgSection); @@ -394,6 +396,9 @@ public void testElectionFraud() throws IOException, InterruptedException { int falseLeader = (trueLeader + 1) % numServers; Assert.assertNotNull("All servers should join the quorum", servers.mt[falseLeader].main.quorumPeer.follower); + // Attach a spy to the QuorumPeer of the false leader + servers.mt[falseLeader].main.setQuorumPeer(spy(servers.mt[falseLeader].main.quorumPeer)); + // to keep the quorum peer running and force it to go into the looking state, we kill leader election // and close the connection to the leader servers.mt[falseLeader].main.quorumPeer.electionAlg.shutdown(); @@ -413,7 +418,6 @@ public void testElectionFraud() throws IOException, InterruptedException { servers.mt[falseLeader].main.quorumPeer.startLeaderElection(); // The previous client connection to falseLeader likely closed, create a new one - servers.zk[falseLeader].close(); servers.zk[falseLeader] = new ZooKeeper("127.0.0.1:" + servers.mt[falseLeader].getClientPort(), ClientBase.CONNECTION_TIMEOUT, this); // Wait for falseLeader to rejoin the quorum @@ -422,6 +426,10 @@ public void testElectionFraud() throws IOException, InterruptedException { // and ensure trueLeader is still the leader Assert.assertNotNull(servers.mt[trueLeader].main.quorumPeer.leader); + verify(servers.mt[falseLeader].main.quorumPeer).setPeerState(ServerState.LEADING); + verify(servers.mt[falseLeader].main.quorumPeer).setPeerState(ServerState.LOOKING); + verify(servers.mt[falseLeader].main.quorumPeer).setPeerState(ServerState.FOLLOWING); + // Look through the logs for output that indicates the falseLeader is LEADING, then LOOKING, then FOLLOWING LineNumberReader r = new LineNumberReader(new StringReader(os.toString())); Pattern leading = Pattern.compile(".*myid=" + falseLeader + ".*LEADING.*"); @@ -1213,7 +1221,7 @@ private QuorumPeer waitForQuorumPeer(MainThread mainThread, int timeout) throws } } } - + private Proposal findProposalOfType(Map proposals, int type) { for (Proposal proposal : proposals.values()) { if (proposal.request.getHdr().getType() == type) { diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java index 1bc2d3e0d48..f1ad81b7dcd 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java @@ -65,6 +65,11 @@ synchronized public void shutdown() { QuorumBase.shutdown(myPeer); } } + + + public void setQuorumPeer(QuorumPeer newPeer) { + this.quorumPeer = newPeer; + } } public static class MainThread implements Runnable { From a8f13a5307ef6c9fc0d9659308854a6056bfbf32 Mon Sep 17 00:00:00 2001 From: Eric Schwartz Date: Mon, 3 Dec 2018 13:22:24 -0700 Subject: [PATCH 53/63] Updating build.xml and ivy.xml to track new dependencies --- build.xml | 4 ++-- ivy.xml | 10 +++++++--- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/build.xml b/build.xml index 55c8ee24ca2..d4bd4f73b1a 100644 --- a/build.xml +++ b/build.xml @@ -41,6 +41,8 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> + + @@ -236,8 +238,6 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - - diff --git a/ivy.xml b/ivy.xml index 996a476940b..788607fbb82 100644 --- a/ivy.xml +++ b/ivy.xml @@ -46,7 +46,7 @@ - + - + + + + + From 906b56b754ad443066f29ab6fb916399de1736e8 Mon Sep 17 00:00:00 2001 From: Eric Schwartz Date: Mon, 3 Dec 2018 13:28:53 -0700 Subject: [PATCH 54/63] Import ServerState from QuorumPeer --- .../zookeeper/server/quorum/QuorumPeerMainTest.java | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) 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 880e48dd98f..eb997e71c68 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 @@ -23,9 +23,9 @@ import static org.mockito.Matchers.any; import static org.mockito.Mockito.doCallRealMethod; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.spy; import java.io.ByteArrayOutputStream; import java.io.File; @@ -63,6 +63,7 @@ import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; import org.apache.zookeeper.server.quorum.Leader.Proposal; +import org.apache.zookeeper.server.quorum.QuorumPeer.ServerState; import org.apache.zookeeper.test.ClientBase; import org.junit.After; import org.junit.Assert; @@ -394,7 +395,11 @@ public void testElectionFraud() throws IOException, InterruptedException { // find a follower int falseLeader = (trueLeader + 1) % numServers; - Assert.assertNotNull("All servers should join the quorum", servers.mt[falseLeader].main.quorumPeer.follower); + Assert.assertTrue("Waiting for falseLeader to become a follower", + ClientBase.waitForServerState(servers.mt[falseLeader].main.quorumPeer, + CONNECTION_TIMEOUT, + QuorumStats.Provider.LEADING_STATE, QuorumStats.Provider.FOLLOWING_STATE)); + // Assert.assertNotNull("All servers should join the quorum", servers.mt[falseLeader].main.quorumPeer.follower); // Attach a spy to the QuorumPeer of the false leader servers.mt[falseLeader].main.setQuorumPeer(spy(servers.mt[falseLeader].main.quorumPeer)); @@ -418,6 +423,7 @@ public void testElectionFraud() throws IOException, InterruptedException { servers.mt[falseLeader].main.quorumPeer.startLeaderElection(); // The previous client connection to falseLeader likely closed, create a new one + servers.zk[falseLeader].close(); servers.zk[falseLeader] = new ZooKeeper("127.0.0.1:" + servers.mt[falseLeader].getClientPort(), ClientBase.CONNECTION_TIMEOUT, this); // Wait for falseLeader to rejoin the quorum From bedce584ccc88f709b2291b543a59fb6e7d83d1d Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Mon, 3 Dec 2018 14:20:54 -0800 Subject: [PATCH 55/63] WIP verify testElectionFraud using observe/spy --- .../zookeeper/server/quorum/QuorumPeer.java | 12 +++ .../server/quorum/QuorumPeerMain.java | 4 +- .../quorum/EphemeralNodeDeletionTest.java | 2 +- .../quorum/FuzzySnapshotRelatedTest.java | 2 +- .../server/quorum/QuorumPeerMainTest.java | 78 +++++++++++-------- .../server/quorum/QuorumPeerTestBase.java | 11 ++- .../server/quorum/RaceConditionTest.java | 2 +- 7 files changed, 68 insertions(+), 43 deletions(-) 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 87bbe3a8768..fa946a2730b 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 @@ -126,6 +126,10 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider */ private ZKDatabase zkDb; + public static class PeerStateObserver { + public void observe(ServerState oldState, ServerState newState) {} + } + public static final class AddressTuple { public final InetSocketAddress quorumAddr; public final InetSocketAddress electionAddr; @@ -719,10 +723,18 @@ public void run() { } private ServerState state = ServerState.LOOKING; + private PeerStateObserver stateObserver = null; private boolean reconfigFlag = false; // indicates that a reconfig just committed + public synchronized void setPeerStateObserver(PeerStateObserver observer) { + stateObserver = observer; + } + public synchronized void setPeerState(ServerState newState){ + if (stateObserver != null) { + stateObserver.observe(state, newState); + } state=newState; } public synchronized void reconfigFlagSet(){ diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java index 49ed5f3f03c..1f29eb6ba8c 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java @@ -157,7 +157,7 @@ public void runFromConfig(QuorumPeerConfig config) true); } - quorumPeer = getQuorumPeer(); + quorumPeer = createQuorumPeer(); quorumPeer.setTxnFactory(new FileTxnSnapLog( config.getDataLogDir(), config.getDataDir())); @@ -208,7 +208,7 @@ public void runFromConfig(QuorumPeerConfig config) } // @VisibleForTesting - protected QuorumPeer getQuorumPeer() throws SaslException { + protected QuorumPeer createQuorumPeer() throws SaslException { return new QuorumPeer(); } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/EphemeralNodeDeletionTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/EphemeralNodeDeletionTest.java index 4b3e98e8413..f9316f669a1 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/EphemeralNodeDeletionTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/EphemeralNodeDeletionTest.java @@ -228,7 +228,7 @@ public void setInjectError(boolean injectError) { static class MockTestQPMain extends TestQPMain { @Override - protected QuorumPeer getQuorumPeer() throws SaslException { + protected QuorumPeer createQuorumPeer() throws SaslException { return new CustomQuorumPeer(); } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FuzzySnapshotRelatedTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FuzzySnapshotRelatedTest.java index cdf47f501bb..2e256b8888a 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FuzzySnapshotRelatedTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FuzzySnapshotRelatedTest.java @@ -316,7 +316,7 @@ static interface NodeSerializeListener { static class CustomizedQPMain extends TestQPMain { @Override - protected QuorumPeer getQuorumPeer() throws SaslException { + protected QuorumPeer createQuorumPeer() throws SaslException { return new QuorumPeer() { @Override public void setZKDatabase(ZKDatabase database) { 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 eb997e71c68..163b1e9e7ae 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 @@ -122,7 +122,7 @@ public void testQuorum() throws Exception { Assert.assertTrue("waiting for server 2 being up", ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP2, CONNECTION_TIMEOUT)); - QuorumPeer quorumPeer = q1.main.quorumPeer; + QuorumPeer quorumPeer = q1.getQuorumPeer(); int tickTime = quorumPeer.getTickTime(); Assert.assertEquals( @@ -208,11 +208,11 @@ public void testEarlyLeaderAbandonment() throws Exception { int leader = -1; Map outstanding = null; for (int i = 0; i < SERVER_COUNT; i++) { - if (mt[i].main.quorumPeer.leader == null) { + if (mt[i].getQuorumPeer().leader == null) { mt[i].shutdown(); } else { leader = i; - outstanding = mt[leader].main.quorumPeer.leader.outstandingProposals; + outstanding = mt[leader].getQuorumPeer().leader.outstandingProposals; } } @@ -395,32 +395,31 @@ public void testElectionFraud() throws IOException, InterruptedException { // find a follower int falseLeader = (trueLeader + 1) % numServers; - Assert.assertTrue("Waiting for falseLeader to become a follower", - ClientBase.waitForServerState(servers.mt[falseLeader].main.quorumPeer, - CONNECTION_TIMEOUT, - QuorumStats.Provider.LEADING_STATE, QuorumStats.Provider.FOLLOWING_STATE)); - // Assert.assertNotNull("All servers should join the quorum", servers.mt[falseLeader].main.quorumPeer.follower); - - // Attach a spy to the QuorumPeer of the false leader - servers.mt[falseLeader].main.setQuorumPeer(spy(servers.mt[falseLeader].main.quorumPeer)); + Assert.assertNotNull("All servers should join the quorum", servers.mt[falseLeader].getQuorumPeer().follower); // to keep the quorum peer running and force it to go into the looking state, we kill leader election // and close the connection to the leader - servers.mt[falseLeader].main.quorumPeer.electionAlg.shutdown(); - servers.mt[falseLeader].main.quorumPeer.follower.getSocket().close(); + servers.mt[falseLeader].getQuorumPeer().electionAlg.shutdown(); + servers.mt[falseLeader].getQuorumPeer().follower.getSocket().close(); // wait for the falseLeader to disconnect waitForOne(servers.zk[falseLeader], States.CONNECTING); + waitForPeerState(servers.mt[falseLeader], QuorumPeer.ServerState.LOOKING); + + // Attach a spy to the PeerStateObserver hook of the QuorumPeer of the false leader + QuorumPeer.PeerStateObserver observer = spy(new QuorumPeer.PeerStateObserver()); + servers.mt[falseLeader].getQuorumPeer().setPeerStateObserver(observer); // convince falseLeader that it is the leader - servers.mt[falseLeader].main.quorumPeer.setPeerState(QuorumPeer.ServerState.LEADING); + servers.mt[falseLeader].getQuorumPeer().setPeerState(QuorumPeer.ServerState.LEADING); // provide time for the falseleader to realize no followers have connected // (this is twice the timeout used in Leader#getEpochToPropose) - Thread.sleep(2 * servers.mt[falseLeader].main.quorumPeer.initLimit * servers.mt[falseLeader].main.quorumPeer.tickTime); + Thread.sleep(2 * servers.mt[falseLeader].getQuorumPeer().initLimit * servers.mt[falseLeader].getQuorumPeer().tickTime); // Restart leader election - servers.mt[falseLeader].main.quorumPeer.startLeaderElection(); + servers.mt[falseLeader].getQuorumPeer().startLeaderElection(); + LOG.info("restarted leader election"); // The previous client connection to falseLeader likely closed, create a new one servers.zk[falseLeader].close(); @@ -430,11 +429,11 @@ public void testElectionFraud() throws IOException, InterruptedException { waitForOne(servers.zk[falseLeader], States.CONNECTED); // and ensure trueLeader is still the leader - Assert.assertNotNull(servers.mt[trueLeader].main.quorumPeer.leader); + Assert.assertNotNull(servers.mt[trueLeader].getQuorumPeer().leader); - verify(servers.mt[falseLeader].main.quorumPeer).setPeerState(ServerState.LEADING); - verify(servers.mt[falseLeader].main.quorumPeer).setPeerState(ServerState.LOOKING); - verify(servers.mt[falseLeader].main.quorumPeer).setPeerState(ServerState.FOLLOWING); + verify(observer).observe(ServerState.LOOKING, ServerState.LEADING); + verify(observer).observe(ServerState.LEADING, ServerState.LOOKING); + verify(observer).observe(ServerState.LOOKING, ServerState.FOLLOWING); // Look through the logs for output that indicates the falseLeader is LEADING, then LOOKING, then FOLLOWING LineNumberReader r = new LineNumberReader(new StringReader(os.toString())); @@ -462,6 +461,17 @@ public void testElectionFraud() throws IOException, InterruptedException { Assert.assertTrue("falseLeader never rejoins the quorum", foundFollowing); } + public static void waitForPeerState(MainThread mt, QuorumPeer.ServerState state) throws InterruptedException { + int iterations = ClientBase.CONNECTION_TIMEOUT / 500; + while (mt.getQuorumPeer().getPeerState() != state) { + if (iterations-- == 0) { + throw new RuntimeException("Waiting too long " + mt.getQuorumPeer().getPeerState() + " != " + state); + } + LOG.info("still waiting for peer to reach " + mt.getQuorumPeer().getPeerState()); + Thread.sleep(500); + } + } + public static void waitForOne(ZooKeeper zk, States state) throws InterruptedException { int iterations = ClientBase.CONNECTION_TIMEOUT / 500; while (zk.getState() != state) { @@ -538,7 +548,7 @@ public void restartClient(int clientIndex, Watcher watcher) throws IOException, public int findLeader() { for (int i = 0; i < mt.length; i++) { - if (mt[i].main.quorumPeer.leader != null) { + if (mt[i].getQuorumPeer().leader != null) { return i; } } @@ -908,7 +918,7 @@ public void testMinMaxSessionTimeOut() throws Exception { .waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP2, CONNECTION_TIMEOUT)); - QuorumPeer quorumPeer = q1.main.quorumPeer; + QuorumPeer quorumPeer = q1.getQuorumPeer(); Assert.assertEquals("minimumSessionTimeOut is not considered", minSessionTimeOut, quorumPeer.getMinSessionTimeout()); @@ -952,7 +962,7 @@ public void testWithOnlyMinSessionTimeout() throws Exception { .waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP2, CONNECTION_TIMEOUT)); - QuorumPeer quorumPeer = q1.main.quorumPeer; + QuorumPeer quorumPeer = q1.getQuorumPeer(); final int maxSessionTimeOut = quorumPeer.tickTime * 20; Assert.assertEquals("minimumSessionTimeOut is not considered", @@ -983,9 +993,9 @@ public void testFailedTxnAsPartOfQuorumLoss() throws Exception { // 2. kill all followers int leader = servers.findLeader(); - Map outstanding = servers.mt[leader].main.quorumPeer.leader.outstandingProposals; + Map outstanding = servers.mt[leader].getQuorumPeer().leader.outstandingProposals; // increase the tick time to delay the leader going to looking - servers.mt[leader].main.quorumPeer.tickTime = LEADER_TIMEOUT_MS; + servers.mt[leader].getQuorumPeer().tickTime = LEADER_TIMEOUT_MS; LOG.warn("LEADER {}", leader); for (int i = 0; i < SERVER_COUNT; i++) { @@ -1042,14 +1052,14 @@ public void testFailedTxnAsPartOfQuorumLoss() throws Exception { // 6. wait for the leader to quit due to not enough followers and come back up as a part of the new quorum LOG.info("Waiting for leader {} to timeout followers", leader); sleepTime = 0; - Follower f = servers.mt[leader].main.quorumPeer.follower; + Follower f = servers.mt[leader].getQuorumPeer().follower; while (f == null || !f.isRunning()) { if (sleepTime > LEADER_TIMEOUT_MS * 2) { - Assert.fail("Took too long for old leader to time out " + servers.mt[leader].main.quorumPeer.getPeerState()); + Assert.fail("Took too long for old leader to time out " + servers.mt[leader].getQuorumPeer().getPeerState()); } Thread.sleep(100); sleepTime += 100; - f = servers.mt[leader].main.quorumPeer.follower; + f = servers.mt[leader].getQuorumPeer().follower; } int newLeader = servers.findLeader(); @@ -1174,7 +1184,7 @@ public void testDataDirAndDataLogDir() throws Exception { qpMain.runFromConfig(configMock); // Assert - FileTxnSnapLog txnFactory = qpMain.getQuorumPeer().getTxnFactory(); + FileTxnSnapLog txnFactory = qpMain.createQuorumPeer().getTxnFactory(); Assert.assertEquals(Paths.get(dataLogDir.getAbsolutePath(), "version-2").toString(), txnFactory.getDataDir().getAbsolutePath()); Assert.assertEquals(Paths.get(dataDir.getAbsolutePath(), "version-2").toString(), txnFactory.getSnapDir().getAbsolutePath()); } finally { @@ -1191,7 +1201,7 @@ private class InjectableQuorumPeerMain extends QuorumPeerMain { } @Override - protected QuorumPeer getQuorumPeer() { + protected QuorumPeer createQuorumPeer() { return qp; } } @@ -1301,7 +1311,7 @@ public TestQPMain getTestQPMain() { int leaderId = -1; int followerA = -1; for (int i = 0; i < ENSEMBLE_SERVERS; i++) { - if (mt[i].main.quorumPeer.leader != null) { + if (mt[i].getQuorumPeer().leader != null) { leaderId = i; } else if (followerA == -1) { followerA = i; @@ -1324,7 +1334,7 @@ public TestQPMain getTestQPMain() { LOG.info("created node {} with value {}", nodePath, initialValue); CustomQuorumPeer leaderQuorumPeer = - (CustomQuorumPeer) mt[leaderId].main.quorumPeer; + (CustomQuorumPeer) mt[leaderId].getQuorumPeer(); // 4. on the customized leader catch the startForwarding call // (without synchronized), set the node to value v1, then @@ -1375,7 +1385,7 @@ public void start() { // 6. exit follower A after taking snapshot CustomQuorumPeer followerAQuorumPeer = - ((CustomQuorumPeer) mt[followerA].main.quorumPeer); + ((CustomQuorumPeer) mt[followerA].getQuorumPeer()); LOG.info("set exit when ack new leader packet on {}", followerA); contexts[followerA].exitWhenAckNewLeader = true; CountDownLatch latch = new CountDownLatch(1); @@ -1451,7 +1461,7 @@ public CustomizedQPMain(Context context) { } @Override - protected QuorumPeer getQuorumPeer() throws SaslException { + protected QuorumPeer createQuorumPeer() throws SaslException { return new CustomQuorumPeer(context); } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java index f1ad81b7dcd..cd3cb71691b 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java @@ -66,9 +66,12 @@ synchronized public void shutdown() { } } + public QuorumPeer getQuorumPeer() { + return quorumPeer; + } public void setQuorumPeer(QuorumPeer newPeer) { - this.quorumPeer = newPeer; + quorumPeer = newPeer; } } @@ -352,12 +355,12 @@ synchronized public boolean isAlive() { } public void clean() { - ClientBase.recursiveDelete(main.quorumPeer.getTxnFactory() + ClientBase.recursiveDelete(main.getQuorumPeer().getTxnFactory() .getDataDir()); } public boolean isQuorumPeerRunning() { - return main.quorumPeer != null; + return main.getQuorumPeer() != null; } public String getPropFromStaticFile(String key) throws IOException { @@ -367,7 +370,7 @@ public String getPropFromStaticFile(String key) throws IOException { } public QuorumPeer getQuorumPeer() { - return main.quorumPeer; + return main.getQuorumPeer(); } 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 34b8273e4f6..7cd31ffd329 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 @@ -245,7 +245,7 @@ public MockProposalRequestProcessor(LeaderZooKeeperServer zks, RequestProcessor private static class MockTestQPMain extends TestQPMain { @Override - protected QuorumPeer getQuorumPeer() throws SaslException { + protected QuorumPeer createQuorumPeer() throws SaslException { return new CustomQuorumPeer(); } } From f969f5483425a16eba7af6fd495fbb37ec1b5a26 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Mon, 3 Dec 2018 14:25:16 -0800 Subject: [PATCH 56/63] remove log checking from testElectionFraud --- .../server/quorum/QuorumPeerMainTest.java | 129 +++++++----------- 1 file changed, 46 insertions(+), 83 deletions(-) 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 163b1e9e7ae..c8ff2f43167 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 @@ -372,93 +372,56 @@ public void testHighestZxidJoinLate() throws Exception { */ @Test public void testElectionFraud() throws IOException, InterruptedException { - // capture QuorumPeer logging - ByteArrayOutputStream os = new ByteArrayOutputStream(); - WriterAppender appender = getConsoleAppender(os, Level.INFO); - Logger qlogger = Logger.getLogger(QuorumPeer.class); - qlogger.addAppender(appender); - numServers = 3; - // used for assertions later - boolean foundLeading = false; - boolean foundLooking = false; - boolean foundFollowing = false; + // spin up a quorum, we use a small ticktime to make the test run faster + servers = LaunchServers(numServers, 500); - try { - // spin up a quorum, we use a small ticktime to make the test run faster - servers = LaunchServers(numServers, 500); - - // find the leader - int trueLeader = servers.findLeader(); - Assert.assertTrue("There should be a leader", trueLeader >= 0); - - // find a follower - int falseLeader = (trueLeader + 1) % numServers; - Assert.assertNotNull("All servers should join the quorum", servers.mt[falseLeader].getQuorumPeer().follower); - - // to keep the quorum peer running and force it to go into the looking state, we kill leader election - // and close the connection to the leader - servers.mt[falseLeader].getQuorumPeer().electionAlg.shutdown(); - servers.mt[falseLeader].getQuorumPeer().follower.getSocket().close(); - - // wait for the falseLeader to disconnect - waitForOne(servers.zk[falseLeader], States.CONNECTING); - waitForPeerState(servers.mt[falseLeader], QuorumPeer.ServerState.LOOKING); - - // Attach a spy to the PeerStateObserver hook of the QuorumPeer of the false leader - QuorumPeer.PeerStateObserver observer = spy(new QuorumPeer.PeerStateObserver()); - servers.mt[falseLeader].getQuorumPeer().setPeerStateObserver(observer); - - // convince falseLeader that it is the leader - servers.mt[falseLeader].getQuorumPeer().setPeerState(QuorumPeer.ServerState.LEADING); - - // provide time for the falseleader to realize no followers have connected - // (this is twice the timeout used in Leader#getEpochToPropose) - Thread.sleep(2 * servers.mt[falseLeader].getQuorumPeer().initLimit * servers.mt[falseLeader].getQuorumPeer().tickTime); - - // Restart leader election - servers.mt[falseLeader].getQuorumPeer().startLeaderElection(); - LOG.info("restarted leader election"); - - // The previous client connection to falseLeader likely closed, create a new one - servers.zk[falseLeader].close(); - servers.zk[falseLeader] = new ZooKeeper("127.0.0.1:" + servers.mt[falseLeader].getClientPort(), ClientBase.CONNECTION_TIMEOUT, this); - - // Wait for falseLeader to rejoin the quorum - waitForOne(servers.zk[falseLeader], States.CONNECTED); - - // and ensure trueLeader is still the leader - Assert.assertNotNull(servers.mt[trueLeader].getQuorumPeer().leader); - - verify(observer).observe(ServerState.LOOKING, ServerState.LEADING); - verify(observer).observe(ServerState.LEADING, ServerState.LOOKING); - verify(observer).observe(ServerState.LOOKING, ServerState.FOLLOWING); - - // Look through the logs for output that indicates the falseLeader is LEADING, then LOOKING, then FOLLOWING - LineNumberReader r = new LineNumberReader(new StringReader(os.toString())); - Pattern leading = Pattern.compile(".*myid=" + falseLeader + ".*LEADING.*"); - Pattern looking = Pattern.compile(".*myid=" + falseLeader + ".*LOOKING.*"); - Pattern following = Pattern.compile(".*myid=" + falseLeader + ".*FOLLOWING.*"); - - String line; - while ((line = r.readLine()) != null) { - if (!foundLeading) { - foundLeading = leading.matcher(line).matches(); - } else if(!foundLooking) { - foundLooking = looking.matcher(line).matches(); - } else if (following.matcher(line).matches()){ - foundFollowing = true; - break; - } - } - } finally { - qlogger.removeAppender(appender); - } + // find the leader + int trueLeader = servers.findLeader(); + Assert.assertTrue("There should be a leader", trueLeader >= 0); + + // find a follower + int falseLeader = (trueLeader + 1) % numServers; + Assert.assertNotNull("All servers should join the quorum", servers.mt[falseLeader].getQuorumPeer().follower); + + // to keep the quorum peer running and force it to go into the looking state, we kill leader election + // and close the connection to the leader + servers.mt[falseLeader].getQuorumPeer().electionAlg.shutdown(); + servers.mt[falseLeader].getQuorumPeer().follower.getSocket().close(); + + // wait for the falseLeader to disconnect + waitForOne(servers.zk[falseLeader], States.CONNECTING); + waitForPeerState(servers.mt[falseLeader], QuorumPeer.ServerState.LOOKING); + + // Attach a spy to the PeerStateObserver hook of the QuorumPeer of the false leader + QuorumPeer.PeerStateObserver observer = spy(new QuorumPeer.PeerStateObserver()); + servers.mt[falseLeader].getQuorumPeer().setPeerStateObserver(observer); + + // convince falseLeader that it is the leader + servers.mt[falseLeader].getQuorumPeer().setPeerState(QuorumPeer.ServerState.LEADING); + + // provide time for the falseleader to realize no followers have connected + // (this is twice the timeout used in Leader#getEpochToPropose) + Thread.sleep(2 * servers.mt[falseLeader].getQuorumPeer().initLimit * servers.mt[falseLeader].getQuorumPeer().tickTime); + + // Restart leader election + servers.mt[falseLeader].getQuorumPeer().startLeaderElection(); + LOG.info("restarted leader election"); + + // The previous client connection to falseLeader likely closed, create a new one + servers.zk[falseLeader].close(); + servers.zk[falseLeader] = new ZooKeeper("127.0.0.1:" + servers.mt[falseLeader].getClientPort(), ClientBase.CONNECTION_TIMEOUT, this); + + // Wait for falseLeader to rejoin the quorum + waitForOne(servers.zk[falseLeader], States.CONNECTED); + + // and ensure trueLeader is still the leader + Assert.assertNotNull(servers.mt[trueLeader].getQuorumPeer().leader); - Assert.assertTrue("falseLeader never attempts to become leader", foundLeading); - Assert.assertTrue("falseLeader never gives up on leadership", foundLooking); - Assert.assertTrue("falseLeader never rejoins the quorum", foundFollowing); + verify(observer).observe(ServerState.LOOKING, ServerState.LEADING); + verify(observer).observe(ServerState.LEADING, ServerState.LOOKING); + verify(observer).observe(ServerState.LOOKING, ServerState.FOLLOWING); } public static void waitForPeerState(MainThread mt, QuorumPeer.ServerState state) throws InterruptedException { From 351458548ed300388b1a7ae1541205738db3a276 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Mon, 3 Dec 2018 16:44:12 -0800 Subject: [PATCH 57/63] remove log grepping from ReadOnlyModeTest --- .../java/org/apache/zookeeper/ClientCnxn.java | 23 +++- .../apache/zookeeper/ClientCnxnSocket.java | 2 +- .../zookeeper/test/ReadOnlyModeTest.java | 101 +++++++++--------- 3 files changed, 72 insertions(+), 54 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java index ef53edf0a8d..1272e148984 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java @@ -397,7 +397,7 @@ public ClientCnxn(String chrootPath, HostProvider hostProvider, int sessionTimeo readTimeout = sessionTimeout * 2 / 3; readOnly = canBeReadOnly; - sendThread = new SendThread(clientCnxnSocket); + sendThread = new SendThread(this, clientCnxnSocket); eventThread = new EventThread(); this.clientConfig=zooKeeper.getClientConfig(); initRequestTimeout(); @@ -794,12 +794,23 @@ public RWServerFoundException(String msg) { } } + private volatile InetSocketAddress observedRWServerAddress; + + public InetSocketAddress getObservedRWServerAddress() { + return observedRWServerAddress; + } + + public void observeRWServerAddress(InetSocketAddress addr) { + observedRWServerAddress = addr; + } + /** * This class services the outgoing request queue and generates the heart * beats. It also spawns the ReadThread. */ class SendThread extends ZooKeeperThread { private long lastPingSentNs; + private final ClientCnxn clientCnxn; private final ClientCnxnSocket clientCnxnSocket; private Random r = new Random(); private boolean isFirstConnect = true; @@ -924,9 +935,10 @@ else if (serverPath.length() > chrootPath.length()) } } - SendThread(ClientCnxnSocket clientCnxnSocket) { + SendThread(ClientCnxn clientCnxn, ClientCnxnSocket clientCnxnSocket) { super(makeThreadName("-SendThread()")); state = States.CONNECTING; + this.clientCnxn = clientCnxn; this.clientCnxnSocket = clientCnxnSocket; setDaemon(true); } @@ -1050,6 +1062,11 @@ private void sendPing() { private InetSocketAddress rwServerAddress = null; + public void setRWServerAddress(InetSocketAddress addr) { + clientCnxn.observeRWServerAddress(addr); + rwServerAddress = addr; + } + private final static int minPingRwTimeout = 100; private final static int maxPingRwTimeout = 60000; @@ -1324,7 +1341,7 @@ private void pingRwServer() throws RWServerFoundException { pingRwTimeout = minPingRwTimeout; // save the found address so that it's used during the next // connection attempt - rwServerAddress = addr; + setRWServerAddress(addr); throw new RWServerFoundException("Majority server found at " + addr.getHostString() + ":" + addr.getPort()); } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxnSocket.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxnSocket.java index 84abe8471ff..f45c55b1528 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxnSocket.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxnSocket.java @@ -44,7 +44,7 @@ * be provided as an alternative to the NIO socket code. * */ -abstract class ClientCnxnSocket { +public abstract class ClientCnxnSocket { private static final Logger LOG = LoggerFactory.getLogger(ClientCnxnSocket.class); protected boolean initialized; diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ReadOnlyModeTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ReadOnlyModeTest.java index 68c71824056..09caa39eceb 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ReadOnlyModeTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ReadOnlyModeTest.java @@ -19,17 +19,18 @@ package org.apache.zookeeper.test; import java.io.ByteArrayOutputStream; +import java.io.IOException; import java.io.LineNumberReader; import java.io.StringReader; +import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.List; import java.util.regex.Pattern; -import org.apache.log4j.Layout; -import org.apache.log4j.Level; -import org.apache.log4j.Logger; -import org.apache.log4j.WriterAppender; +import org.apache.zookeeper.ClientCnxn; +import org.apache.zookeeper.ClientCnxnSocket; +import org.apache.zookeeper.ClientWatchManager; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NotReadOnlyException; @@ -41,6 +42,7 @@ import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.ZooKeeper.States; +import org.apache.zookeeper.client.HostProvider; import org.apache.zookeeper.common.Time; import org.apache.zookeeper.test.ClientBase.CountdownWatcher; import org.junit.After; @@ -67,6 +69,25 @@ public void tearDown() throws Exception { qu.tearDown(); } + public static class CustomZooKeeper extends ZooKeeper { + public CustomZooKeeper(String connectString, int sessionTimeout, + Watcher watcher, boolean canBeReadOnly) throws IOException { + super(connectString, sessionTimeout, watcher, canBeReadOnly); + } + + public volatile ClientCnxn myCnxn; + + @Override + protected ClientCnxn createConnection(String chrootPath, + HostProvider hostProvider, int sessionTimeout, + ZooKeeper zooKeeper, ClientWatchManager watcher, + ClientCnxnSocket clientCnxnSocket, boolean canBeReadOnly) + throws IOException { + myCnxn = super.createConnection(chrootPath, hostProvider, sessionTimeout, zooKeeper, watcher, clientCnxnSocket, canBeReadOnly); + return myCnxn; + } + } + /** * Test write operations using multi request. */ @@ -254,56 +275,36 @@ public void testSessionEstablishment() throws Exception { @SuppressWarnings("deprecation") @Test(timeout = 90000) public void testSeekForRwServer() throws Exception { - // setup the logger to capture all logs - Layout layout = Logger.getRootLogger().getAppender("CONSOLE") - .getLayout(); - ByteArrayOutputStream os = new ByteArrayOutputStream(); - WriterAppender appender = new WriterAppender(layout, os); - appender.setImmediateFlush(true); - appender.setThreshold(Level.INFO); - Logger zlogger = Logger.getLogger("org.apache.zookeeper"); - zlogger.addAppender(appender); + qu.shutdown(2); + CountdownWatcher watcher = new CountdownWatcher(); + CustomZooKeeper zk = new CustomZooKeeper(qu.getConnString(), + CONNECTION_TIMEOUT, watcher, true); + watcher.waitForConnected(CONNECTION_TIMEOUT); - try { - qu.shutdown(2); - CountdownWatcher watcher = new CountdownWatcher(); - ZooKeeper zk = new ZooKeeper(qu.getConnString(), - CONNECTION_TIMEOUT, watcher, true); - watcher.waitForConnected(CONNECTION_TIMEOUT); - - // if we don't suspend a peer it will rejoin a quorum - qu.getPeer(1).peer.suspend(); - - // start two servers to form a quorum; client should detect this and - // connect to one of them - watcher.reset(); - qu.start(2); - qu.start(3); - ClientBase.waitForServerUp(qu.getConnString(), 2000); - watcher.waitForConnected(CONNECTION_TIMEOUT); - zk.create("/test", "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, - CreateMode.PERSISTENT); - - // resume poor fellow - qu.getPeer(1).peer.resume(); - } finally { - zlogger.removeAppender(appender); - } + // if we don't suspend a peer it will rejoin a quorum + qu.getPeer(1).peer.suspend(); + + // start two servers to form a quorum; client should detect this and + // connect to one of them + watcher.reset(); + qu.start(2); + qu.start(3); + ClientBase.waitForServerUp(qu.getConnString(), 2000); + watcher.waitForConnected(CONNECTION_TIMEOUT); + zk.create("/test", "test".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + + // resume poor fellow + qu.getPeer(1).peer.resume(); - os.close(); - LineNumberReader r = new LineNumberReader(new StringReader(os - .toString())); - String line; - Pattern p = Pattern.compile(".*Majority server found.*"); - boolean found = false; - while ((line = r.readLine()) != null) { - if (p.matcher(line).matches()) { - found = true; + int iterations = ClientBase.CONNECTION_TIMEOUT / 500; + while (zk.myCnxn.getObservedRWServerAddress() == null) { + if (iterations-- == 0) { break; } + LOG.info("still waiting for client to observe a read/write server"); + Thread.sleep(500); } - Assert.assertTrue( - "Majority server wasn't found while connected to r/o server", - found); + Assert.assertNotNull(zk.myCnxn.getObservedRWServerAddress()); } } From a6017b8186b8156db732ef8176a19be6ba1e5f30 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Mon, 26 Nov 2018 23:14:01 +0000 Subject: [PATCH 58/63] WIP switch from log4j 1.2.17 to 2.11.1 --- bin/zkServer.sh | 7 +- build.xml | 14 +- conf/log4j.properties | 65 ---- conf/log4j2.xml | 13 + ivy.xml | 10 +- runtest.sh | 2 +- .../org/apache/zookeeper/jmx/ManagedUtil.java | 120 ------ .../zookeeper/server/ZooKeeperServerMain.java | 7 - .../server/quorum/QuorumPeerMain.java | 7 - .../resources/lib/log4j-1.2.17.LICENSE.txt | 202 ---------- .../server/quorum/QuorumPeerMainTest.java | 351 ++++++------------ .../test/EmptiedSnapshotRecoveryTest.java | 5 +- 12 files changed, 146 insertions(+), 657 deletions(-) delete mode 100644 conf/log4j.properties create mode 100644 conf/log4j2.xml delete mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/jmx/ManagedUtil.java delete mode 100644 zookeeper-server/src/main/resources/lib/log4j-1.2.17.LICENSE.txt diff --git a/bin/zkServer.sh b/bin/zkServer.sh index e1709460b13..062641101da 100755 --- a/bin/zkServer.sh +++ b/bin/zkServer.sh @@ -60,15 +60,10 @@ then then JMXSSL=false fi - if [ "x$JMXLOG4J" = "x" ] - then - JMXLOG4J=true - fi echo "ZooKeeper remote JMX Port set to $JMXPORT" >&2 echo "ZooKeeper remote JMX authenticate set to $JMXAUTH" >&2 echo "ZooKeeper remote JMX ssl set to $JMXSSL" >&2 - echo "ZooKeeper remote JMX log4j set to $JMXLOG4J" >&2 - ZOOMAIN="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.port=$JMXPORT -Dcom.sun.management.jmxremote.authenticate=$JMXAUTH -Dcom.sun.management.jmxremote.ssl=$JMXSSL -Dzookeeper.jmx.log4j.disable=$JMXLOG4J org.apache.zookeeper.server.quorum.QuorumPeerMain" + ZOOMAIN="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.port=$JMXPORT -Dcom.sun.management.jmxremote.authenticate=$JMXAUTH -Dcom.sun.management.jmxremote.ssl=$JMXSSL org.apache.zookeeper.server.quorum.QuorumPeerMain" fi else echo "JMX disabled by user request" >&2 diff --git a/build.xml b/build.xml index d4bd4f73b1a..bda3731a8eb 100644 --- a/build.xml +++ b/build.xml @@ -31,7 +31,7 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + @@ -1251,8 +1251,8 @@ xmlns:cs="antlib:com.puppycrawl.tools.checkstyle.ant"> - + - @@ -105,7 +105,9 @@ - + + + diff --git a/runtest.sh b/runtest.sh index 7c6b7d4bef5..a1632e9f6b7 100755 --- a/runtest.sh +++ b/runtest.sh @@ -1,3 +1,3 @@ #!/bin/bash -x -exec java -Xmx512m -Dtest.junit.threads=1 -Dbuild.test.dir=build/test/tmp -Dlog4j.configuration=file:conf/log4j.properties -Dtest.data.dir=build/test/data -Dzookeeper.DigestAuthenticationProvider.superDigest=super:D/InIHSb7yEEbrWz8b9l71RjZJU= -classpath build/test/classes:build/test/lib/accessors-smart-1.2.jar:build/test/lib/ant-1.10.5.jar:build/test/lib/ant-junit-1.10.5.jar:build/test/lib/ant-junit4-1.10.5.jar:build/test/lib/ant-launcher-1.10.5.jar:build/test/lib/antlr-2.7.7.jar:build/test/lib/antlr4-runtime-4.5.3.jar:build/test/lib/asm-5.0.4.jar:build/test/lib/audience-annotations-0.5.0.jar:build/test/lib/bcpkix-jdk15on-1.60.jar:build/test/lib/bcprov-jdk15on-1.60.jar:build/test/lib/checkstyle-7.1.2.jar:build/test/lib/commons-beanutils-1.9.3.jar:build/test/lib/commons-cli-1.2.jar:build/test/lib/commons-collections-3.2.2.jar:build/test/lib/commons-io-2.6.jar:build/test/lib/commons-logging-1.2.jar:build/test/lib/guava-19.0.jar:build/test/lib/hamcrest-all-1.3.jar:build/test/lib/hamcrest-core-1.3.jar:build/test/lib/jackson-annotations-2.9.0.jar:build/test/lib/jackson-core-2.9.7.jar:build/test/lib/jackson-databind-2.9.7.jar:build/test/lib/javax.servlet-api-3.1.0.jar:build/test/lib/jcip-annotations-1.0-1.jar:build/test/lib/jetty-http-9.4.14.v20181114.jar:build/test/lib/jetty-io-9.4.14.v20181114.jar:build/test/lib/jetty-security-9.4.14.v20181114.jar:build/test/lib/jetty-server-9.4.14.v20181114.jar:build/test/lib/jetty-servlet-9.4.14.v20181114.jar:build/test/lib/jetty-util-9.4.14.v20181114.jar:build/test/lib/jline-2.11.jar:build/test/lib/json-smart-2.3.jar:build/test/lib/junit-4.12.jar:build/test/lib/kerb-admin-1.1.0.jar:build/test/lib/kerb-client-1.1.0.jar:build/test/lib/kerb-common-1.1.0.jar:build/test/lib/kerb-core-1.1.0.jar:build/test/lib/kerb-crypto-1.1.0.jar:build/test/lib/kerb-identity-1.1.0.jar:build/test/lib/kerb-server-1.1.0.jar:build/test/lib/kerb-simplekdc-1.1.0.jar:build/test/lib/kerb-util-1.1.0.jar:build/test/lib/kerby-asn1-1.1.0.jar:build/test/lib/kerby-config-1.1.0.jar:build/test/lib/kerby-pkix-1.1.0.jar:build/test/lib/kerby-util-1.1.0.jar:build/test/lib/kerby-xdr-1.1.0.jar:build/test/lib/log4j-1.2.17.jar:build/test/lib/mockito-core-2.23.4.jar:build/test/lib/byte-buddy-agent-1.9.3.jar:build/test/lib/byte-buddy-1.9.3.jar:build/test/lib/objenesis-2.6.jar:build/test/lib/netty-3.10.6.Final.jar:build/test/lib/nimbus-jose-jwt-4.41.2.jar:build/test/lib/slf4j-api-1.7.25.jar:build/test/lib/slf4j-log4j12-1.7.25.jar:build/test/lib/token-provider-1.1.0.jar:build/classes org.apache.tools.ant.taskdefs.optional.junit.JUnitTestRunner "$@" skipNonTests=false filtertrace=true haltOnError=false haltOnFailure=false showoutput=true outputtoformatters=true logfailedtests=true threadid=0 logtestlistenerevents=true formatter=com.undefined.testing.OneLinerFormatter +exec java -Xmx512m -Dtest.junit.threads=1 -Dbuild.test.dir=build/test/tmp -Dlog4j2.configurationFile=conf/log4j2.xml -Dtest.data.dir=build/test/data -Dzookeeper.DigestAuthenticationProvider.superDigest=super:D/InIHSb7yEEbrWz8b9l71RjZJU= -classpath build/test/classes:build/test/lib/accessors-smart-1.2.jar:build/test/lib/ant-1.10.5.jar:build/test/lib/ant-junit-1.10.5.jar:build/test/lib/ant-junit4-1.10.5.jar:build/test/lib/ant-launcher-1.10.5.jar:build/test/lib/antlr-2.7.7.jar:build/test/lib/antlr4-runtime-4.5.3.jar:build/test/lib/asm-5.0.4.jar:build/test/lib/audience-annotations-0.5.0.jar:build/test/lib/bcpkix-jdk15on-1.60.jar:build/test/lib/bcprov-jdk15on-1.60.jar:build/test/lib/checkstyle-7.1.2.jar:build/test/lib/commons-beanutils-1.9.3.jar:build/test/lib/commons-cli-1.2.jar:build/test/lib/commons-collections-3.2.2.jar:build/test/lib/commons-io-2.6.jar:build/test/lib/commons-logging-1.2.jar:build/test/lib/guava-19.0.jar:build/test/lib/hamcrest-all-1.3.jar:build/test/lib/hamcrest-core-1.3.jar:build/test/lib/jackson-annotations-2.9.0.jar:build/test/lib/jackson-core-2.9.7.jar:build/test/lib/jackson-databind-2.9.7.jar:build/test/lib/javax.servlet-api-3.1.0.jar:build/test/lib/jcip-annotations-1.0-1.jar:build/test/lib/jetty-http-9.4.14.v20181114.jar:build/test/lib/jetty-io-9.4.14.v20181114.jar:build/test/lib/jetty-security-9.4.14.v20181114.jar:build/test/lib/jetty-server-9.4.14.v20181114.jar:build/test/lib/jetty-servlet-9.4.14.v20181114.jar:build/test/lib/jetty-util-9.4.14.v20181114.jar:build/test/lib/jline-2.11.jar:build/test/lib/json-smart-2.3.jar:build/test/lib/junit-4.12.jar:build/test/lib/kerb-admin-1.1.0.jar:build/test/lib/kerb-client-1.1.0.jar:build/test/lib/kerb-common-1.1.0.jar:build/test/lib/kerb-core-1.1.0.jar:build/test/lib/kerb-crypto-1.1.0.jar:build/test/lib/kerb-identity-1.1.0.jar:build/test/lib/kerb-server-1.1.0.jar:build/test/lib/kerb-simplekdc-1.1.0.jar:build/test/lib/kerb-util-1.1.0.jar:build/test/lib/kerby-asn1-1.1.0.jar:build/test/lib/kerby-config-1.1.0.jar:build/test/lib/kerby-pkix-1.1.0.jar:build/test/lib/kerby-util-1.1.0.jar:build/test/lib/kerby-xdr-1.1.0.jar:build/test/lib/log4j-slf4j-impl-2.11.1.jar:build/test/lib/log4j-api-2.11.1.jar:build/test/lib/log4j-core-2.11.1.jar:build/test/lib/mockito-core-2.23.4.jar:build/test/lib/byte-buddy-agent-1.9.3.jar:build/test/lib/byte-buddy-1.9.3.jar:build/test/lib/objenesis-2.6.jar:build/test/lib/netty-3.10.6.Final.jar:build/test/lib/nimbus-jose-jwt-4.41.2.jar:build/test/lib/slf4j-api-1.7.25.jar:build/test/lib/token-provider-1.1.0.jar:build/classes org.apache.tools.ant.taskdefs.optional.junit.JUnitTestRunner "$@" skipNonTests=false filtertrace=true haltOnError=false haltOnFailure=false showoutput=true outputtoformatters=true logfailedtests=true threadid=0 logtestlistenerevents=true formatter=com.undefined.testing.OneLinerFormatter diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/jmx/ManagedUtil.java b/zookeeper-server/src/main/java/org/apache/zookeeper/jmx/ManagedUtil.java deleted file mode 100644 index ba741b89686..00000000000 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/jmx/ManagedUtil.java +++ /dev/null @@ -1,120 +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.jmx; - -import java.util.Enumeration; - -import javax.management.JMException; -import javax.management.MBeanServer; -import javax.management.ObjectName; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Shared utilities - */ -public class ManagedUtil { - private static final Logger LOG = LoggerFactory.getLogger(ManagedUtil.class); - - private static final boolean isLog4jJmxEnabled() { - boolean enabled = false; - - try { - Class.forName("org.apache.log4j.spi.LoggerRepository"); - - if (Boolean.getBoolean("zookeeper.jmx.log4j.disable") == true) { - LOG.info("Log4j found but jmx support is disabled."); - } else { - enabled = true; - LOG.info("Log4j found with jmx enabled."); - } - - } catch (ClassNotFoundException e) { - LOG.info("Log4j not found."); - } - - return enabled; - } - - - /** - * Register the log4j JMX mbeans. Set environment variable - * "zookeeper.jmx.log4j.disable" to true to disable registration. - * @see http://logging.apache.org/log4j/1.2/apidocs/index.html?org/apache/log4j/jmx/package-summary.html - * @throws JMException if registration fails - */ - @SuppressWarnings("rawtypes") - public static void registerLog4jMBeans() throws JMException { - if (isLog4jJmxEnabled()) { - LOG.debug("registerLog4jMBeans()"); - MBeanServer mbs = MBeanRegistry.getInstance().getPlatformMBeanServer(); - - try { - // Create and Register the top level Log4J MBean - // org.apache.log4j.jmx.HierarchyDynamicMBean hdm = new org.apache.log4j.jmx.HierarchyDynamicMBean(); - Object hdm = Class.forName("org.apache.log4j.jmx.HierarchyDynamicMBean").getDeclaredConstructor().newInstance(); - - ObjectName mbo = new ObjectName("log4j:hierarchy=default"); - mbs.registerMBean(hdm, mbo); - - // Add the root logger to the Hierarchy MBean - // org.apache.log4j.Logger rootLogger = - // org.apache.log4j.Logger.getRootLogger(); - Object rootLogger = Class.forName("org.apache.log4j.Logger") - .getMethod("getRootLogger", (Class[]) null) - .invoke(null, (Object[]) null); - - // hdm.addLoggerMBean(rootLogger.getName()); - Object rootLoggerName = rootLogger.getClass() - .getMethod("getName", (Class[]) null) - .invoke(rootLogger, (Object[]) null); - hdm.getClass().getMethod("addLoggerMBean", String.class) - .invoke(hdm, rootLoggerName); - - // Get each logger from the Log4J Repository and add it to the - // Hierarchy MBean created above. - // org.apache.log4j.spi.LoggerRepository r = - // org.apache.log4j.LogManager.getLoggerRepository(); - Object r = Class.forName("org.apache.log4j.LogManager") - .getMethod("getLoggerRepository", (Class[]) null) - .invoke(null, (Object[]) null); - - // Enumeration enumer = r.getCurrentLoggers(); - Enumeration enumer = (Enumeration) r.getClass() - .getMethod("getCurrentLoggers", (Class[]) null) - .invoke(r, (Object[]) null); - - while (enumer.hasMoreElements()) { - Object logger = enumer.nextElement(); - // hdm.addLoggerMBean(logger.getName()); - Object loggerName = logger.getClass() - .getMethod("getName", (Class[]) null) - .invoke(logger, (Object[]) null); - hdm.getClass().getMethod("addLoggerMBean", String.class) - .invoke(hdm, loggerName); - } - } catch (Exception e) { - LOG.error("Problems while registering log4j jmx beans!", e); - throw new JMException(e.toString()); - } - } - } - -} diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerMain.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerMain.java index 12b8dcd9204..c8945fe0475 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerMain.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerMain.java @@ -25,7 +25,6 @@ import javax.management.JMException; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.zookeeper.jmx.ManagedUtil; import org.apache.zookeeper.server.admin.AdminServer; import org.apache.zookeeper.server.admin.AdminServer.AdminServerException; import org.apache.zookeeper.server.admin.AdminServerFactory; @@ -90,12 +89,6 @@ public static void main(String[] args) { protected void initializeAndRun(String[] args) throws ConfigException, IOException, AdminServerException { - try { - ManagedUtil.registerLog4jMBeans(); - } catch (JMException e) { - LOG.warn("Unable to register log4j JMX control", e); - } - ServerConfig config = new ServerConfig(); if (args.length == 1) { config.parse(args[0]); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java index 1f29eb6ba8c..cbb829bf235 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java @@ -25,7 +25,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.zookeeper.jmx.ManagedUtil; import org.apache.zookeeper.server.ServerCnxnFactory; import org.apache.zookeeper.server.ZKDatabase; import org.apache.zookeeper.server.DatadirCleanupManager; @@ -132,12 +131,6 @@ protected void initializeAndRun(String[] args) public void runFromConfig(QuorumPeerConfig config) throws IOException, AdminServerException { - try { - ManagedUtil.registerLog4jMBeans(); - } catch (JMException e) { - LOG.warn("Unable to register log4j JMX control", e); - } - LOG.info("Starting quorum peer"); try { ServerCnxnFactory cnxnFactory = null; diff --git a/zookeeper-server/src/main/resources/lib/log4j-1.2.17.LICENSE.txt b/zookeeper-server/src/main/resources/lib/log4j-1.2.17.LICENSE.txt deleted file mode 100644 index 6279e5206de..00000000000 --- a/zookeeper-server/src/main/resources/lib/log4j-1.2.17.LICENSE.txt +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright 1999-2005 The Apache Software Foundation - - Licensed 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. 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 c8ff2f43167..96cd80f82d3 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 @@ -45,10 +45,6 @@ import javax.security.sasl.SaslException; import org.apache.commons.io.FileUtils; -import org.apache.log4j.Level; -import org.apache.log4j.Logger; -import org.apache.log4j.PatternLayout; -import org.apache.log4j.WriterAppender; import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; @@ -568,53 +564,29 @@ private Servers LaunchServers(int numServers, Integer tickTime) throws IOExcepti public void testBadPeerAddressInQuorum() throws Exception { ClientBase.setupTestEnv(); - // setup the logger to capture all logs - ByteArrayOutputStream os = new ByteArrayOutputStream(); - WriterAppender appender = getConsoleAppender(os, Level.WARN); - Logger qlogger = Logger.getLogger("org.apache.zookeeper.server.quorum"); - qlogger.addAppender(appender); - - try { - final int CLIENT_PORT_QP1 = PortAssignment.unique(); - final int CLIENT_PORT_QP2 = PortAssignment.unique(); - - String quorumCfgSection = - "server.1=127.0.0.1:" + PortAssignment.unique() - + ":" + PortAssignment.unique() + ";" + CLIENT_PORT_QP1 - + "\nserver.2=fee.fii.foo.fum:" + PortAssignment.unique() - + ":" + PortAssignment.unique() + ";" + CLIENT_PORT_QP2; - - MainThread q1 = new MainThread(1, CLIENT_PORT_QP1, quorumCfgSection); - q1.start(); + final int CLIENT_PORT_QP1 = PortAssignment.unique(); + final int CLIENT_PORT_QP2 = PortAssignment.unique(); - boolean isup = - ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP1, - 30000); + String quorumCfgSection = + "server.1=127.0.0.1:" + PortAssignment.unique() + + ":" + PortAssignment.unique() + ";" + CLIENT_PORT_QP1 + + "\nserver.2=fee.fii.foo.fum:" + PortAssignment.unique() + + ":" + PortAssignment.unique() + ";" + CLIENT_PORT_QP2; - Assert.assertFalse("Server never came up", isup); + MainThread q1 = new MainThread(1, CLIENT_PORT_QP1, quorumCfgSection); + q1.start(); - q1.shutdown(); + boolean isup = + ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP1, + 30000); - Assert.assertTrue("waiting for server 1 down", - ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP1, - ClientBase.CONNECTION_TIMEOUT)); + Assert.assertFalse("Server never came up", isup); - } finally { - qlogger.removeAppender(appender); - } + q1.shutdown(); - LineNumberReader r = new LineNumberReader(new StringReader(os.toString())); - String line; - boolean found = false; - Pattern p = - Pattern.compile(".*Cannot open channel to .* at election address .*"); - while ((line = r.readLine()) != null) { - found = p.matcher(line).matches(); - if (found) { - break; - } - } - Assert.assertTrue("complains about host", found); + Assert.assertTrue("waiting for server 1 down", + ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP1, + ClientBase.CONNECTION_TIMEOUT)); } /** @@ -624,82 +596,50 @@ public void testBadPeerAddressInQuorum() throws Exception { public void testInconsistentPeerType() throws Exception { ClientBase.setupTestEnv(); - // setup the logger to capture all logs - ByteArrayOutputStream os = new ByteArrayOutputStream(); - WriterAppender appender = getConsoleAppender(os, Level.INFO); - Logger qlogger = Logger.getLogger("org.apache.zookeeper.server.quorum"); - qlogger.addAppender(appender); - // test the most likely situation only: server is stated as observer in // servers list, but there's no "peerType=observer" token in config - try { - final int CLIENT_PORT_QP1 = PortAssignment.unique(); - final int CLIENT_PORT_QP2 = PortAssignment.unique(); - final int CLIENT_PORT_QP3 = PortAssignment.unique(); - - String quorumCfgSection = - "server.1=127.0.0.1:" + PortAssignment.unique() - + ":" + PortAssignment.unique() + ";" + CLIENT_PORT_QP1 - + "\nserver.2=127.0.0.1:" + PortAssignment.unique() - + ":" + PortAssignment.unique() + ";" + CLIENT_PORT_QP2 - + "\nserver.3=127.0.0.1:" + PortAssignment.unique() - + ":" + PortAssignment.unique() + ":observer" + ";" + CLIENT_PORT_QP3; - - MainThread q1 = new MainThread(1, CLIENT_PORT_QP1, quorumCfgSection); - MainThread q2 = new MainThread(2, CLIENT_PORT_QP2, quorumCfgSection); - MainThread q3 = new MainThread(3, CLIENT_PORT_QP3, quorumCfgSection); - q1.start(); - q2.start(); - q3.start(); + final int CLIENT_PORT_QP1 = PortAssignment.unique(); + final int CLIENT_PORT_QP2 = PortAssignment.unique(); + final int CLIENT_PORT_QP3 = PortAssignment.unique(); - Assert.assertTrue("waiting for server 1 being up", - ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP1, - CONNECTION_TIMEOUT)); - Assert.assertTrue("waiting for server 2 being up", - ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP2, - CONNECTION_TIMEOUT)); - Assert.assertTrue("waiting for server 3 being up", - ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP3, - CONNECTION_TIMEOUT)); + String quorumCfgSection = + "server.1=127.0.0.1:" + PortAssignment.unique() + + ":" + PortAssignment.unique() + ";" + CLIENT_PORT_QP1 + + "\nserver.2=127.0.0.1:" + PortAssignment.unique() + + ":" + PortAssignment.unique() + ";" + CLIENT_PORT_QP2 + + "\nserver.3=127.0.0.1:" + PortAssignment.unique() + + ":" + PortAssignment.unique() + ":observer" + ";" + CLIENT_PORT_QP3; - q1.shutdown(); - q2.shutdown(); - q3.shutdown(); - - Assert.assertTrue("waiting for server 1 down", - ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP1, - ClientBase.CONNECTION_TIMEOUT)); - Assert.assertTrue("waiting for server 2 down", - ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP2, - ClientBase.CONNECTION_TIMEOUT)); - Assert.assertTrue("waiting for server 3 down", - ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP3, - ClientBase.CONNECTION_TIMEOUT)); + MainThread q1 = new MainThread(1, CLIENT_PORT_QP1, quorumCfgSection); + MainThread q2 = new MainThread(2, CLIENT_PORT_QP2, quorumCfgSection); + MainThread q3 = new MainThread(3, CLIENT_PORT_QP3, quorumCfgSection); + q1.start(); + q2.start(); + q3.start(); - } finally { - qlogger.removeAppender(appender); - } + Assert.assertTrue("waiting for server 1 being up", + ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP1, + CONNECTION_TIMEOUT)); + Assert.assertTrue("waiting for server 2 being up", + ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP2, + CONNECTION_TIMEOUT)); + Assert.assertTrue("waiting for server 3 being up", + ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP3, + CONNECTION_TIMEOUT)); - LineNumberReader r = new LineNumberReader(new StringReader(os.toString())); - String line; - boolean warningPresent = false; - boolean defaultedToObserver = false; - Pattern pWarn = - Pattern.compile(".*Peer type from servers list.* doesn't match peerType.*"); - Pattern pObserve = Pattern.compile(".*OBSERVING.*"); - while ((line = r.readLine()) != null) { - if (pWarn.matcher(line).matches()) { - warningPresent = true; - } - if (pObserve.matcher(line).matches()) { - defaultedToObserver = true; - } - if (warningPresent && defaultedToObserver) { - break; - } - } - Assert.assertTrue("Should warn about inconsistent peer type", - warningPresent && defaultedToObserver); + q1.shutdown(); + q2.shutdown(); + q3.shutdown(); + + Assert.assertTrue("waiting for server 1 down", + ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP1, + ClientBase.CONNECTION_TIMEOUT)); + Assert.assertTrue("waiting for server 2 down", + ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP2, + ClientBase.CONNECTION_TIMEOUT)); + Assert.assertTrue("waiting for server 3 down", + ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP3, + ClientBase.CONNECTION_TIMEOUT)); } /** @@ -764,61 +704,36 @@ public void testBadPackets() throws Exception { public void testQuorumDefaults() throws Exception { ClientBase.setupTestEnv(); - // setup the logger to capture all logs - ByteArrayOutputStream os = new ByteArrayOutputStream(); - WriterAppender appender = getConsoleAppender(os, Level.INFO); - appender.setImmediateFlush(true); - Logger zlogger = Logger.getLogger("org.apache.zookeeper"); - zlogger.addAppender(appender); - - try { - final int CLIENT_PORT_QP1 = PortAssignment.unique(); - final int CLIENT_PORT_QP2 = PortAssignment.unique(); - - String quorumCfgSection = - "server.1=127.0.0.1:" + PortAssignment.unique() - + ":" + PortAssignment.unique() + ";" + CLIENT_PORT_QP1 - + "\nserver.2=127.0.0.1:" + PortAssignment.unique() - + ":" + PortAssignment.unique() + ";" + CLIENT_PORT_QP2; + final int CLIENT_PORT_QP1 = PortAssignment.unique(); + final int CLIENT_PORT_QP2 = PortAssignment.unique(); - MainThread q1 = new MainThread(1, CLIENT_PORT_QP1, quorumCfgSection); - MainThread q2 = new MainThread(2, CLIENT_PORT_QP2, quorumCfgSection); - q1.start(); - q2.start(); + String quorumCfgSection = + "server.1=127.0.0.1:" + PortAssignment.unique() + + ":" + PortAssignment.unique() + ";" + CLIENT_PORT_QP1 + + "\nserver.2=127.0.0.1:" + PortAssignment.unique() + + ":" + PortAssignment.unique() + ";" + CLIENT_PORT_QP2; - Assert.assertTrue("waiting for server 1 being up", - ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP1, - CONNECTION_TIMEOUT)); - Assert.assertTrue("waiting for server 2 being up", - ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP2, - CONNECTION_TIMEOUT)); + MainThread q1 = new MainThread(1, CLIENT_PORT_QP1, quorumCfgSection); + MainThread q2 = new MainThread(2, CLIENT_PORT_QP2, quorumCfgSection); + q1.start(); + q2.start(); - q1.shutdown(); - q2.shutdown(); + Assert.assertTrue("waiting for server 1 being up", + ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP1, + CONNECTION_TIMEOUT)); + Assert.assertTrue("waiting for server 2 being up", + ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP2, + CONNECTION_TIMEOUT)); - Assert.assertTrue("waiting for server 1 down", - ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP1, - ClientBase.CONNECTION_TIMEOUT)); - Assert.assertTrue("waiting for server 2 down", - ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP2, - ClientBase.CONNECTION_TIMEOUT)); + q1.shutdown(); + q2.shutdown(); - } finally { - zlogger.removeAppender(appender); - } - os.close(); - LineNumberReader r = new LineNumberReader(new StringReader(os.toString())); - String line; - boolean found = false; - Pattern p = - Pattern.compile(".*FastLeaderElection.*"); - while ((line = r.readLine()) != null) { - found = p.matcher(line).matches(); - if (found) { - break; - } - } - Assert.assertTrue("fastleaderelection used", found); + Assert.assertTrue("waiting for server 1 down", + ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP1, + ClientBase.CONNECTION_TIMEOUT)); + Assert.assertTrue("waiting for server 2 down", + ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP2, + ClientBase.CONNECTION_TIMEOUT)); } /** @@ -1052,78 +967,49 @@ public void testLeaderOutOfView() throws Exception { int numServers = 3; - // used for assertions later - boolean foundLeading = false; - boolean foundFollowing = false; - - // capture QuorumPeer logging - ByteArrayOutputStream os = new ByteArrayOutputStream(); - WriterAppender appender = getConsoleAppender(os, Level.DEBUG); - Logger qlogger = Logger.getLogger("org.apache.zookeeper.server.quorum"); - qlogger.addAppender(appender); - - try { - Servers svrs = new Servers(); - svrs.clientPorts = new int[numServers]; - for (int i = 0; i < numServers; i++) { - svrs.clientPorts[i] = PortAssignment.unique(); - } - - String quorumCfgIncomplete = getUniquePortCfgForId(1) + "\n" + getUniquePortCfgForId(2); - String quorumCfgComplete = quorumCfgIncomplete + "\n" + getUniquePortCfgForId(3); - svrs.mt = new MainThread[3]; - - // Node 1 is started without the leader (3) in its config view - svrs.mt[0] = new MainThread(1, svrs.clientPorts[0], quorumCfgIncomplete); - for (int i = 1; i < numServers; i++) { - svrs.mt[i] = new MainThread(i + 1, svrs.clientPorts[i], quorumCfgComplete); - } - - // Node 1 must be started first, before quorum is formed, to trigger the attempted invalid connection to 3 - svrs.mt[0].start(); - QuorumPeer quorumPeer1 = waitForQuorumPeer(svrs.mt[0], CONNECTION_TIMEOUT); - Assert.assertEquals(QuorumPeer.ServerState.LOOKING, quorumPeer1.getPeerState()); - - // Node 3 started second to avoid 1 and 2 forming a quorum before 3 starts up - int highestServerIndex = numServers - 1; - svrs.mt[highestServerIndex].start(); - QuorumPeer quorumPeer3 = waitForQuorumPeer(svrs.mt[highestServerIndex], CONNECTION_TIMEOUT); - Assert.assertEquals(QuorumPeer.ServerState.LOOKING, quorumPeer3.getPeerState()); + Servers svrs = new Servers(); + svrs.clientPorts = new int[numServers]; + for (int i = 0; i < numServers; i++) { + svrs.clientPorts[i] = PortAssignment.unique(); + } - // Node 2 started last, kicks off leader election - for (int i = 1; i < highestServerIndex; i++) { - svrs.mt[i].start(); - } + String quorumCfgIncomplete = getUniquePortCfgForId(1) + "\n" + getUniquePortCfgForId(2); + String quorumCfgComplete = quorumCfgIncomplete + "\n" + getUniquePortCfgForId(3); + svrs.mt = new MainThread[3]; - // Nodes 2 and 3 now form quorum and fully start. 1 attempts to vote for 3, fails, returns to LOOKING state - for (int i = 1; i < numServers; i++) { - Assert.assertTrue("waiting for server to start", - ClientBase.waitForServerUp("127.0.0.1:" + svrs.clientPorts[i], CONNECTION_TIMEOUT)); - } + // Node 1 is started without the leader (3) in its config view + svrs.mt[0] = new MainThread(1, svrs.clientPorts[0], quorumCfgIncomplete); + for (int i = 1; i < numServers; i++) { + svrs.mt[i] = new MainThread(i + 1, svrs.clientPorts[i], quorumCfgComplete); + } - Assert.assertEquals(QuorumPeer.ServerState.LOOKING, svrs.mt[0].getQuorumPeer().getPeerState()); - Assert.assertTrue(svrs.mt[highestServerIndex].getQuorumPeer().getPeerState() == QuorumPeer.ServerState.LEADING); - for (int i = 1; i < highestServerIndex; i++) { - Assert.assertEquals(QuorumPeer.ServerState.FOLLOWING, svrs.mt[i].getQuorumPeer().getPeerState()); - } + // Node 1 must be started first, before quorum is formed, to trigger the attempted invalid connection to 3 + svrs.mt[0].start(); + QuorumPeer quorumPeer1 = waitForQuorumPeer(svrs.mt[0], CONNECTION_TIMEOUT); + Assert.assertEquals(QuorumPeer.ServerState.LOOKING, quorumPeer1.getPeerState()); - // Look through the logs for output that indicates Node 1 is LEADING or FOLLOWING - LineNumberReader r = new LineNumberReader(new StringReader(os.toString())); - Pattern leading = Pattern.compile(".*myid=1.*QuorumPeer.*LEADING.*"); - Pattern following = Pattern.compile(".*myid=1.*QuorumPeer.*FOLLOWING.*"); + // Node 3 started second to avoid 1 and 2 forming a quorum before 3 starts up + int highestServerIndex = numServers - 1; + svrs.mt[highestServerIndex].start(); + QuorumPeer quorumPeer3 = waitForQuorumPeer(svrs.mt[highestServerIndex], CONNECTION_TIMEOUT); + Assert.assertEquals(QuorumPeer.ServerState.LOOKING, quorumPeer3.getPeerState()); - String line; - while ((line = r.readLine()) != null && !foundLeading && !foundFollowing) { - foundLeading = leading.matcher(line).matches(); - foundFollowing = following.matcher(line).matches(); - } + // Node 2 started last, kicks off leader election + for (int i = 1; i < highestServerIndex; i++) { + svrs.mt[i].start(); + } - } finally { - qlogger.removeAppender(appender); + // Nodes 2 and 3 now form quorum and fully start. 1 attempts to vote for 3, fails, returns to LOOKING state + for (int i = 1; i < numServers; i++) { + Assert.assertTrue("waiting for server to start", + ClientBase.waitForServerUp("127.0.0.1:" + svrs.clientPorts[i], CONNECTION_TIMEOUT)); } - Assert.assertFalse("Corrupt peer should never become leader", foundLeading); - Assert.assertFalse("Corrupt peer should not attempt connection to out of view leader", foundFollowing); + Assert.assertEquals(QuorumPeer.ServerState.LOOKING, svrs.mt[0].getQuorumPeer().getPeerState()); + Assert.assertTrue(svrs.mt[highestServerIndex].getQuorumPeer().getPeerState() == QuorumPeer.ServerState.LEADING); + for (int i = 1; i < highestServerIndex; i++) { + Assert.assertEquals(QuorumPeer.ServerState.FOLLOWING, svrs.mt[i].getQuorumPeer().getPeerState()); + } } @Test @@ -1169,13 +1055,6 @@ protected QuorumPeer createQuorumPeer() { } } - private WriterAppender getConsoleAppender(ByteArrayOutputStream os, Level level) { - String loggingPattern = ((PatternLayout) Logger.getRootLogger().getAppender("CONSOLE").getLayout()).getConversionPattern(); - WriterAppender appender = new WriterAppender(new PatternLayout(loggingPattern), os); - appender.setThreshold(level); - return appender; - } - private String getUniquePortCfgForId(int id) { return String.format("server.%d=127.0.0.1:%d:%d", id, PortAssignment.unique(), PortAssignment.unique()); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/EmptiedSnapshotRecoveryTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/EmptiedSnapshotRecoveryTest.java index 5b2f8a48153..411751c14c3 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/EmptiedSnapshotRecoveryTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/EmptiedSnapshotRecoveryTest.java @@ -24,7 +24,6 @@ import java.util.List; import java.util.LinkedList; -import org.apache.log4j.Logger; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.PortAssignment; import org.apache.zookeeper.WatchedEvent; @@ -39,13 +38,15 @@ import org.apache.zookeeper.server.persistence.FileTxnSnapLog; import org.junit.Assert; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** If snapshots are corrupted to the empty file or deleted, Zookeeper should * not proceed to read its transactiong log files * Test that zxid == -1 in the presence of emptied/deleted snapshots */ public class EmptiedSnapshotRecoveryTest extends ZKTestCase implements Watcher { - private static final Logger LOG = Logger.getLogger(RestoreCommittedLogTest.class); + private static final Logger LOG = LoggerFactory.getLogger(EmptiedSnapshotRecoveryTest.class); private static String HOSTPORT = "127.0.0.1:" + PortAssignment.unique(); private static final int CONNECTION_TIMEOUT = 3000; private static final int N_TRANSACTIONS = 150; From c46df0d3de9c9fb6edb9935903082aabf2b9d24d Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Mon, 3 Dec 2018 21:37:22 -0800 Subject: [PATCH 59/63] restore verification (using a spy) in testLeaderOutOfView --- .../zookeeper/server/quorum/QuorumPeerMainTest.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) 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 96cd80f82d3..f8af939136f 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 @@ -23,9 +23,10 @@ import static org.mockito.Matchers.any; import static org.mockito.Mockito.doCallRealMethod; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.spy; import java.io.ByteArrayOutputStream; import java.io.File; @@ -979,6 +980,7 @@ public void testLeaderOutOfView() throws Exception { // Node 1 is started without the leader (3) in its config view svrs.mt[0] = new MainThread(1, svrs.clientPorts[0], quorumCfgIncomplete); + for (int i = 1; i < numServers; i++) { svrs.mt[i] = new MainThread(i + 1, svrs.clientPorts[i], quorumCfgComplete); } @@ -988,6 +990,10 @@ public void testLeaderOutOfView() throws Exception { QuorumPeer quorumPeer1 = waitForQuorumPeer(svrs.mt[0], CONNECTION_TIMEOUT); Assert.assertEquals(QuorumPeer.ServerState.LOOKING, quorumPeer1.getPeerState()); + // Attach a spy to the PeerStateObserver hook of the QuorumPeer of the node detached from the leader + QuorumPeer.PeerStateObserver observer = spy(new QuorumPeer.PeerStateObserver()); + svrs.mt[0].getQuorumPeer().setPeerStateObserver(observer); + // Node 3 started second to avoid 1 and 2 forming a quorum before 3 starts up int highestServerIndex = numServers - 1; svrs.mt[highestServerIndex].start(); @@ -1010,6 +1016,9 @@ public void testLeaderOutOfView() throws Exception { for (int i = 1; i < highestServerIndex; i++) { Assert.assertEquals(QuorumPeer.ServerState.FOLLOWING, svrs.mt[i].getQuorumPeer().getPeerState()); } + + verify(observer, never()).observe(ServerState.LOOKING, ServerState.LEADING); + verify(observer, never()).observe(ServerState.LOOKING, ServerState.FOLLOWING); } @Test From 663f9821d85d02601e493815ad4359befc1c7471 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Mon, 3 Dec 2018 21:50:06 -0800 Subject: [PATCH 60/63] restore verification (using getQuorumPeer()) in testQuorumDefaults --- .../apache/zookeeper/server/quorum/QuorumPeerMainTest.java | 4 ++++ 1 file changed, 4 insertions(+) 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 f8af939136f..bea526fc0cd 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 @@ -59,6 +59,7 @@ import org.apache.zookeeper.common.X509Exception; import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; +import org.apache.zookeeper.server.quorum.FastLeaderElection; import org.apache.zookeeper.server.quorum.Leader.Proposal; import org.apache.zookeeper.server.quorum.QuorumPeer.ServerState; import org.apache.zookeeper.test.ClientBase; @@ -735,6 +736,9 @@ public void testQuorumDefaults() throws Exception { Assert.assertTrue("waiting for server 2 down", ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP2, ClientBase.CONNECTION_TIMEOUT)); + + Assert.assertTrue("expected FastLeaderElection algorithm", + q1.getQuorumPeer().getElectionAlg() instanceof FastLeaderElection); } /** From 7336737a01253d61ecfda4fe6c8a83097a862ab5 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Mon, 3 Dec 2018 21:59:57 -0800 Subject: [PATCH 61/63] restore verification (using a spy) in testInconsistentPeerType --- .../zookeeper/server/quorum/QuorumPeerMainTest.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) 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 bea526fc0cd..d01f86e2830 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 @@ -615,9 +615,14 @@ public void testInconsistentPeerType() throws Exception { MainThread q1 = new MainThread(1, CLIENT_PORT_QP1, quorumCfgSection); MainThread q2 = new MainThread(2, CLIENT_PORT_QP2, quorumCfgSection); MainThread q3 = new MainThread(3, CLIENT_PORT_QP3, quorumCfgSection); + + q3.start(); + // Attach a spy to the PeerStateObserver hook of the QuorumPeer of the node with inconsistent peerType + QuorumPeer.PeerStateObserver observer = spy(new QuorumPeer.PeerStateObserver()); + q3.getQuorumPeer().setPeerStateObserver(observer); + q1.start(); q2.start(); - q3.start(); Assert.assertTrue("waiting for server 1 being up", ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP1, @@ -642,6 +647,9 @@ public void testInconsistentPeerType() throws Exception { Assert.assertTrue("waiting for server 3 down", ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP3, ClientBase.CONNECTION_TIMEOUT)); + + // Verify that ":observer" in servers list preempts missing "peerType=observer" + verify(observer).observe(ServerState.LOOKING, ServerState.OBSERVING); } /** From 2f6e2780e038b91d4358b1065946cd00a3f66629 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Mon, 3 Dec 2018 22:33:55 -0800 Subject: [PATCH 62/63] Fix mocking error in LearnerHandlerTest --- .../org/apache/zookeeper/server/quorum/LearnerHandlerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerHandlerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerHandlerTest.java index 843c8aa7c95..a2f8098981a 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerHandlerTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerHandlerTest.java @@ -133,7 +133,7 @@ public void setUp() throws Exception { Matchers.anyLong())).thenAnswer(new Answer() { public Object answer(InvocationOnMock invocation) { currentZxid = (Long) invocation.getArguments()[1]; - return 0; + return new Long(0); } }); From b4f35881c1b16bd5f50f340f24097868c8a89032 Mon Sep 17 00:00:00 2001 From: Michael Edwards Date: Tue, 4 Dec 2018 00:11:59 -0800 Subject: [PATCH 63/63] close loophole in testLeaderOutOfView --- .../zookeeper/server/quorum/Learner.java | 1 + .../zookeeper/server/quorum/QuorumPeer.java | 16 +++++++++++ .../server/quorum/QuorumPeerMainTest.java | 28 +++++++++++++------ 3 files changed, 36 insertions(+), 9 deletions(-) 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 667f73ca4a8..551e56b6ac5 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 @@ -292,6 +292,7 @@ protected void connectToLeader(InetSocketAddress addr, String hostname) Thread.sleep(1000); } + self.onConnection(addr); self.authLearner.authenticate(sock, hostname); leaderIs = BinaryInputArchive.getArchive(new BufferedInputStream( 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 fa946a2730b..49e2edb6882 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 @@ -126,6 +126,10 @@ public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider */ private ZKDatabase zkDb; + public static class ConnectionObserver { + public void observe(InetSocketAddress leaderAddr) {} + } + public static class PeerStateObserver { public void observe(ServerState oldState, ServerState newState) {} } @@ -723,20 +727,32 @@ public void run() { } private ServerState state = ServerState.LOOKING; + private ConnectionObserver connObserver = null; private PeerStateObserver stateObserver = null; private boolean reconfigFlag = false; // indicates that a reconfig just committed + public synchronized void setConnectionObserver(ConnectionObserver observer) { + connObserver = observer; + } + public synchronized void setPeerStateObserver(PeerStateObserver observer) { stateObserver = observer; } + public synchronized void onConnection(InetSocketAddress leaderAddr){ + if (connObserver != null) { + connObserver.observe(leaderAddr); + } + } + public synchronized void setPeerState(ServerState newState){ if (stateObserver != null) { stateObserver.observe(state, newState); } state=newState; } + public synchronized void reconfigFlagSet(){ reconfigFlag = true; } 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 d01f86e2830..905cb4c0d10 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 @@ -973,6 +973,15 @@ public void testFailedTxnAsPartOfQuorumLoss() throws Exception { /** * Verify that a node without the leader in its view will not attempt to connect to the leader. + * + * Note that this test was originally written to verify that node 1 never entered the FOLLOWING + * state (or the LEADING state). That's not correct; it's possible for node 2 to make contact + * with node 1 first, send a vote for itself to node 1, and *then* make contact with node 3 and + * update its vote to be for node 3 -- which node 1 will ignore, because node 3 is not in its + * configuration. Node 1 will then (after a timeout) decide that node 2's vote for itself, plus + * node 1's vote for node 2, form a quorum in favor of node 2 as the new leader, and enter the + * FOLLOWING state. (It will attempt to follow node 2, which will not enter LEADING, and thus + * it will never actually succeed in connecting to the node it thinks is the leader.) */ @Test public void testLeaderOutOfView() throws Exception { @@ -1002,9 +1011,9 @@ public void testLeaderOutOfView() throws Exception { QuorumPeer quorumPeer1 = waitForQuorumPeer(svrs.mt[0], CONNECTION_TIMEOUT); Assert.assertEquals(QuorumPeer.ServerState.LOOKING, quorumPeer1.getPeerState()); - // Attach a spy to the PeerStateObserver hook of the QuorumPeer of the node detached from the leader - QuorumPeer.PeerStateObserver observer = spy(new QuorumPeer.PeerStateObserver()); - svrs.mt[0].getQuorumPeer().setPeerStateObserver(observer); + // Attach a spy to the ConnectionObserver hook of the QuorumPeer of node 1 + QuorumPeer.ConnectionObserver observer = spy(new QuorumPeer.ConnectionObserver()); + svrs.mt[0].getQuorumPeer().setConnectionObserver(observer); // Node 3 started second to avoid 1 and 2 forming a quorum before 3 starts up int highestServerIndex = numServers - 1; @@ -1017,20 +1026,21 @@ public void testLeaderOutOfView() throws Exception { svrs.mt[i].start(); } - // Nodes 2 and 3 now form quorum and fully start. 1 attempts to vote for 3, fails, returns to LOOKING state + // Nodes 2 and 3 now form quorum and fully start. 1 attempts to vote for 2, + // fails (but see test method comment!), returns to LOOKING state for (int i = 1; i < numServers; i++) { Assert.assertTrue("waiting for server to start", ClientBase.waitForServerUp("127.0.0.1:" + svrs.clientPorts[i], CONNECTION_TIMEOUT)); } - Assert.assertEquals(QuorumPeer.ServerState.LOOKING, svrs.mt[0].getQuorumPeer().getPeerState()); - Assert.assertTrue(svrs.mt[highestServerIndex].getQuorumPeer().getPeerState() == QuorumPeer.ServerState.LEADING); + waitForPeerState(svrs.mt[highestServerIndex], QuorumPeer.ServerState.LEADING); for (int i = 1; i < highestServerIndex; i++) { - Assert.assertEquals(QuorumPeer.ServerState.FOLLOWING, svrs.mt[i].getQuorumPeer().getPeerState()); + waitForPeerState(svrs.mt[i], QuorumPeer.ServerState.FOLLOWING); } + waitForPeerState(svrs.mt[0], QuorumPeer.ServerState.LOOKING); - verify(observer, never()).observe(ServerState.LOOKING, ServerState.LEADING); - verify(observer, never()).observe(ServerState.LOOKING, ServerState.FOLLOWING); + Thread.sleep(3000); + verify(observer, never()).observe(any(InetSocketAddress.class)); } @Test