Return-Path: X-Original-To: apmail-zookeeper-commits-archive@www.apache.org Delivered-To: apmail-zookeeper-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id DA142953B for ; Fri, 13 Jan 2012 23:29:08 +0000 (UTC) Received: (qmail 6580 invoked by uid 500); 13 Jan 2012 23:29:08 -0000 Delivered-To: apmail-zookeeper-commits-archive@zookeeper.apache.org Received: (qmail 6512 invoked by uid 500); 13 Jan 2012 23:29:08 -0000 Mailing-List: contact commits-help@zookeeper.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ Delivered-To: mailing list commits@zookeeper.apache.org Received: (qmail 6504 invoked by uid 99); 13 Jan 2012 23:29:07 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 13 Jan 2012 23:29:07 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 13 Jan 2012 23:29:04 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id A5E4123888D2 for ; Fri, 13 Jan 2012 23:28:42 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1231389 - in /zookeeper/trunk: CHANGES.txt src/java/main/org/apache/zookeeper/server/quorum/Leader.java src/java/test/org/apache/zookeeper/test/ClientBase.java src/java/test/org/apache/zookeeper/test/ObserverLETest.java Date: Fri, 13 Jan 2012 23:28:42 -0000 To: commits@zookeeper.apache.org From: henry@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20120113232842.A5E4123888D2@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: henry Date: Fri Jan 13 23:28:41 2012 New Revision: 1231389 URL: http://svn.apache.org/viewvc?rev=1231389&view=rev Log: ZOOKEEPER-1294. One of the zookeeper server is not accepting any requests (Kavita Sharma via henryr) Added: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ObserverLETest.java Modified: zookeeper/trunk/CHANGES.txt zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.java zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java Modified: zookeeper/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1231389&r1=1231388&r2=1231389&view=diff ============================================================================== --- zookeeper/trunk/CHANGES.txt (original) +++ zookeeper/trunk/CHANGES.txt Fri Jan 13 23:28:41 2012 @@ -94,6 +94,8 @@ BUGFIXES: ZOOKEEPER-1050. zooinspector shell scripts do not work (Will Johnson via phunt) + + ZOOKEEPER-1294. One of the zookeeper server is not accepting any requests (Kavita Sharma via henryr) IMPROVEMENTS: Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.java URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.java?rev=1231389&r1=1231388&r2=1231389&view=diff ============================================================================== --- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.java (original) +++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/Leader.java Fri Jan 13 23:28:41 2012 @@ -308,7 +308,7 @@ public class Leader { cnxAcceptor.setName("LearnerCnxAcceptor-" + ss.getLocalSocketAddress()); cnxAcceptor.start(); - long epoch = getEpochToPropose(self.getId(), self.getAcceptedEpoch()); + long epoch = getEpochToPropose(self.getId(), self.getAcceptedEpoch()); zk.setZxid(ZxidUtils.makeZxid(epoch, 0)); @@ -377,15 +377,15 @@ public class Leader { if (!tickSkip) { self.tick++; } - int syncedCount = 0; HashSet syncedSet = new HashSet(); // lock on the followers when we use it. syncedSet.add(self.getId()); synchronized (learners) { for (LearnerHandler f : learners) { - if (f.synced()) { - syncedCount++; + // Synced set is used to check we have a supporting quorum, so only + // PARTICIPANT, not OBSERVER, learners should be used + if (f.synced() && f.getLearnerType() == LearnerType.PARTICIPANT) { syncedSet.add(f.getSid()); } f.ping(); @@ -395,7 +395,7 @@ public class Leader { //if (!tickSkip && syncedCount < self.quorumPeers.size() / 2) { // Lost quorum, shutdown // TODO: message is wrong unless majority quorums used - shutdown("Only " + syncedCount + " followers, need " + shutdown("Only " + syncedSet.size() + " followers, need " + (self.getVotingView().size() / 2)); // make sure the order is the same! // the leader goes to looking @@ -770,7 +770,7 @@ public class Leader { } connectingFollowers.add(sid); QuorumVerifier verifier = self.getQuorumVerifier(); - if (connectingFollowers.contains(self.getId()) && + if (connectingFollowers.contains(self.getId()) && verifier.containsQuorum(connectingFollowers)) { waitingForNewEpoch = false; self.setAcceptedEpoch(epoch); Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java?rev=1231389&r1=1231388&r2=1231389&view=diff ============================================================================== --- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java (original) +++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java Fri Jan 13 23:28:41 2012 @@ -53,6 +53,8 @@ import org.apache.zookeeper.server.Serve import org.apache.zookeeper.server.ZKDatabase; import org.apache.zookeeper.server.ZooKeeperServer; import org.apache.zookeeper.server.persistence.FileTxnLog; +import org.apache.zookeeper.server.quorum.QuorumPeer; + import static org.apache.zookeeper.client.FourLetterWordMain.send4LetterWord; import org.junit.After; import org.junit.Assert; @@ -263,6 +265,23 @@ public abstract class ClientBase extends return false; } + public static boolean waitForServerState(QuorumPeer qp, int timeout, + String serverState) { + long start = System.currentTimeMillis(); + while (true) { + try { + Thread.sleep(250); + } catch (InterruptedException e) { + // ignore + } + if (qp.getServerState().equals(serverState)) + return true; + if (System.currentTimeMillis() > start + timeout) { + return false; + } + } + } + static void verifyThreadTerminated(Thread thread, long millis) throws InterruptedException { Added: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ObserverLETest.java URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ObserverLETest.java?rev=1231389&view=auto ============================================================================== --- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ObserverLETest.java (added) +++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ObserverLETest.java Fri Jan 13 23:28:41 2012 @@ -0,0 +1,76 @@ +/** + * 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 static org.junit.Assert.*; + +import java.util.Arrays; + +import org.apache.zookeeper.ZKTestCase; +import org.apache.zookeeper.server.quorum.QuorumPeer; +import org.apache.zookeeper.server.quorum.QuorumStats; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class ObserverLETest extends ZKTestCase { + final QuorumBase qb = new QuorumBase(); + final ClientTest ct = new ClientTest(); + + @Before + public void establishThreeParticipantOneObserverEnsemble() throws Exception { + qb.setUp(true); + ct.hostPort = qb.hostPort; + ct.setUpAll(); + qb.s5.shutdown(); + } + + @After + public void shutdownQuorum() throws Exception { + ct.tearDownAll(); + qb.tearDown(); + } + + /** + * See ZOOKEEPER-1294. Confirms that an observer will not support the quorum + * of a leader by forming a 5-node, 2-observer ensemble (so quorum size is 2). + * When all but the leader and one observer are shut down, the leader should + * enter the 'looking' state, not stay in the 'leading' state. + */ + @Test + public void testLEWithObserver() throws Exception { + QuorumPeer leader = null; + for (QuorumPeer server : Arrays.asList(qb.s1, qb.s2, qb.s3)) { + if (server.getServerState().equals( + QuorumStats.Provider.FOLLOWING_STATE)) { + server.shutdown(); + assertTrue("Waiting for server down", ClientBase + .waitForServerDown("127.0.0.1:" + + server.getClientPort(), + ClientBase.CONNECTION_TIMEOUT)); + } else { + assertNull("More than one leader found", leader); + leader = server; + } + } + assertTrue("Leader is not in Looking state", ClientBase + .waitForServerState(leader, ClientBase.CONNECTION_TIMEOUT, + QuorumStats.Provider.LOOKING_STATE)); + } + +}