Return-Path: Delivered-To: apmail-hadoop-zookeeper-commits-archive@minotaur.apache.org Received: (qmail 79704 invoked from network); 4 Aug 2009 18:11:43 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 4 Aug 2009 18:11:43 -0000 Received: (qmail 96995 invoked by uid 500); 4 Aug 2009 18:11:48 -0000 Delivered-To: apmail-hadoop-zookeeper-commits-archive@hadoop.apache.org Received: (qmail 96974 invoked by uid 500); 4 Aug 2009 18:11:48 -0000 Mailing-List: contact zookeeper-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: zookeeper-dev@ Delivered-To: mailing list zookeeper-commits@hadoop.apache.org Received: (qmail 96964 invoked by uid 99); 4 Aug 2009 18:11:48 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 04 Aug 2009 18:11:48 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.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; Tue, 04 Aug 2009 18:11:38 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id 087912388892; Tue, 4 Aug 2009 18:11:17 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r800895 - in /hadoop/zookeeper/trunk: CHANGES.txt src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java src/java/test/org/apache/zookeeper/test/FLEZeroWeightTest.java Date: Tue, 04 Aug 2009 18:11:16 -0000 To: zookeeper-commits@hadoop.apache.org From: mahadev@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20090804181117.087912388892@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: mahadev Date: Tue Aug 4 18:11:16 2009 New Revision: 800895 URL: http://svn.apache.org/viewvc?rev=800895&view=rev Log: ZOOKEEPER-491. Prevent zero-weight servers from being elected. (flavio via mahadev) Added: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEZeroWeightTest.java Modified: hadoop/zookeeper/trunk/CHANGES.txt hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java Modified: hadoop/zookeeper/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=800895&r1=800894&r2=800895&view=diff ============================================================================== --- hadoop/zookeeper/trunk/CHANGES.txt (original) +++ hadoop/zookeeper/trunk/CHANGES.txt Tue Aug 4 18:11:16 2009 @@ -38,6 +38,9 @@ ZOOKEEPER-480. FLE should perform leader check when node is not leading and add vote of follower (flavio via mahadev) + ZOOKEEPER-491. Prevent zero-weight servers from being elected. (flavio via + mahadev) + IMPROVEMENTS: ZOOKEEPER-473. cleanup junit tests to eliminate false positives due to "socket reuse" and failure to close client (phunt via mahadev) Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java?rev=800895&r1=800894&r2=800895&view=diff ============================================================================== --- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java (original) +++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java Tue Aug 4 18:11:16 2009 @@ -577,8 +577,7 @@ tmpTimeOut : maxNotificationInterval); LOG.info("Notification time out: " + notTimeout); } - else { - //notTimeout = finalizeWait; + else{ switch (n.state) { case LOOKING: // If notification > current, replace and send messages out @@ -607,7 +606,11 @@ } LOG.info("Adding vote"); - recvset.put(n.sid, new Vote(n.leader, n.zxid, n.epoch)); + /* + * Skip zero-weight servers + */ + if(self.getQuorumVerifier().getWeight(n.sid) != 0) + recvset.put(n.sid, new Vote(n.leader, n.zxid, n.epoch)); //If have received from all nodes, then terminate if (self.quorumPeers.size() == recvset.size()) { Added: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEZeroWeightTest.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEZeroWeightTest.java?rev=800895&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEZeroWeightTest.java (added) +++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLEZeroWeightTest.java Tue Aug 4 18:11:16 2009 @@ -0,0 +1,181 @@ +/** + * 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.ByteArrayInputStream; +import java.io.File; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Properties; +import java.util.Random; + +import junit.framework.TestCase; + +import org.apache.log4j.Logger; +import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.server.quorum.FastLeaderElection; +import org.apache.zookeeper.server.quorum.QuorumPeer; +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.quorum.flexible.QuorumHierarchical; +import org.junit.Before; +import org.junit.Test; + +public class FLEZeroWeightTest extends TestCase { + private static final Logger LOG = Logger.getLogger(HierarchicalQuorumTest.class); + + Properties qp; + + int count; + HashMap peers; + ArrayList threads; + File tmpdir[]; + int port[]; + Object finalObj; + + volatile Vote votes[]; + volatile boolean leaderDies; + volatile long leader = -1; + Random rand = new Random(); + + + @Before + @Override + protected void setUp() throws Exception { + count = 9; + + peers = new HashMap(count); + threads = new ArrayList(count); + votes = new Vote[count]; + tmpdir = new File[count]; + port = new int[count]; + finalObj = new Object(); + + String config = "group.1=0:1:2\n" + + "group.2=3:4:5\n" + + "group.3=6:7:8\n" + + "weight.0=1\n" + + "weight.1=1\n" + + "weight.2=1\n" + + "weight.3=0\n" + + "weight.4=0\n" + + "weight.5=0\n" + + "weight.6=0\n" + + "weight.7=0\n" + + "weight.8=0"; + + ByteArrayInputStream is = new ByteArrayInputStream(config.getBytes()); + this.qp = new Properties(); + qp.load(is); + + LOG.info("SetUp " + getName()); + } + + protected void tearDown() throws Exception { + for(int i = 0; i < threads.size(); i++) { + LEThread leThread = threads.get(i); + ((FastLeaderElection) leThread.peer.getElectionAlg()).shutdown(); + // shutdown() has to be explicitly called for every thread to + // make sure that resources are freed properly and all fixed network ports + // are available for other test cases + leThread.peer.shutdown(); + } + LOG.info("FINISHED " + getName()); + } + + class LEThread extends Thread { + int i; + QuorumPeer peer; + //int peerRound = 1; + + LEThread(QuorumPeer peer, int i) { + this.i = i; + this.peer = peer; + LOG.info("Constructor: " + getName()); + } + + public void run() { + try { + Vote v = null; + while(true){ + + //while(true) { + peer.setPeerState(ServerState.LOOKING); + LOG.info("Going to call leader election."); + v = peer.getElectionAlg().lookForLeader(); + if(v == null){ + LOG.info("Thread " + i + " got a null vote"); + return; + } + + /* + * A real zookeeper would take care of setting the current vote. Here + * we do it manually. + */ + peer.setCurrentVote(v); + + LOG.info("Finished election: " + i + ", " + v.id); + votes[i] = v; + + if((peer.getPeerState() == ServerState.LEADING) && + (peer.getId() > 2)) fail("Elected zero-weight server"); + + if((peer.getPeerState() == ServerState.FOLLOWING) || + (peer.getPeerState() == ServerState.LEADING)) break; + } + LOG.debug("Thread " + i + " votes " + v); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + } + + @Test + public void testHierarchicalQuorum() throws Exception { + FastLeaderElection le[] = new FastLeaderElection[count]; + + LOG.info("TestHierarchicalQuorum: " + getName()+ ", " + count); + for(int i = 0; i < count; i++) { + peers.put(Long.valueOf(i), + new QuorumServer(i, + new InetSocketAddress(PortAssignment.unique()), + new InetSocketAddress(PortAssignment.unique()))); + tmpdir[i] = ClientBase.createTmpDir(); + port[i] = PortAssignment.unique(); + } + + for(int i = 0; i < le.length; i++) { + QuorumHierarchical hq = new QuorumHierarchical(qp); + QuorumPeer peer = new QuorumPeer(peers, tmpdir[i], tmpdir[i], port[i], 3, i, 2, 2, 2, hq); + peer.startLeaderElection(); + LEThread thread = new LEThread(peer, i); + thread.start(); + threads.add(thread); + } + LOG.info("Started threads " + getName()); + + for(int i = 0; i < threads.size(); i++) { + threads.get(i).join(15000); + if (threads.get(i).isAlive()) { + fail("Threads didn't join"); + } + } + } +} \ No newline at end of file