You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by mi...@apache.org on 2014/08/24 07:44:26 UTC
svn commit: r1620112 - in /zookeeper/branches/branch-3.5: ./
src/java/test/org/apache/zookeeper/server/quorum/
src/java/test/org/apache/zookeeper/test/
Author: michim
Date: Sun Aug 24 05:44:25 2014
New Revision: 1620112
URL: http://svn.apache.org/r1620112
Log:
ZOOKEEPER-2017 New tests for reconfig failure cases (Alexander Shraer and Hongchao Deng via michim)
Added:
zookeeper/branches/branch-3.5/src/java/test/org/apache/zookeeper/server/quorum/ReconfigFailureCases.java
Modified:
zookeeper/branches/branch-3.5/CHANGES.txt
zookeeper/branches/branch-3.5/src/java/test/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java
zookeeper/branches/branch-3.5/src/java/test/org/apache/zookeeper/server/quorum/StandaloneDisabledTest.java
zookeeper/branches/branch-3.5/src/java/test/org/apache/zookeeper/test/ReconfigTest.java
Modified: zookeeper/branches/branch-3.5/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.5/CHANGES.txt?rev=1620112&r1=1620111&r2=1620112&view=diff
==============================================================================
--- zookeeper/branches/branch-3.5/CHANGES.txt (original)
+++ zookeeper/branches/branch-3.5/CHANGES.txt Sun Aug 24 05:44:25 2014
@@ -1055,6 +1055,9 @@ IMPROVEMENTS:
ZOOKEEPER-1986. refactor log trace on touchSession
(Hongchao Deng via phunt)
+ ZOOKEEPER-2017 New tests for reconfig failure cases (Alexander Shraer and
+ Hongchao Deng via michim)
+
headers
Release 3.4.0 -
Added: zookeeper/branches/branch-3.5/src/java/test/org/apache/zookeeper/server/quorum/ReconfigFailureCases.java
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.5/src/java/test/org/apache/zookeeper/server/quorum/ReconfigFailureCases.java?rev=1620112&view=auto
==============================================================================
--- zookeeper/branches/branch-3.5/src/java/test/org/apache/zookeeper/server/quorum/ReconfigFailureCases.java (added)
+++ zookeeper/branches/branch-3.5/src/java/test/org/apache/zookeeper/server/quorum/ReconfigFailureCases.java Sun Aug 24 05:44:25 2014
@@ -0,0 +1,269 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.quorum;
+
+import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.NewConfigNoQuorum;
+import org.apache.zookeeper.PortAssignment;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.test.ClientBase;
+import org.apache.zookeeper.test.QuorumUtil;
+import org.apache.zookeeper.test.ReconfigTest;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ReconfigFailureCases extends QuorumPeerTestBase {
+
+ private QuorumUtil qu;
+
+ @After
+ public void tearDown() throws Exception {
+ if (qu != null) {
+ qu.tearDown();
+ }
+ }
+
+ /*
+ * Tests that an incremental reconfig fails if the current config is hiearchical.
+ */
+ @Test
+ public void testIncrementalReconfigInvokedOnHiearchicalQS() throws Exception {
+ qu = new QuorumUtil(2); // create 5 servers
+ qu.disableJMXTest = true;
+ qu.startAll();
+ ZooKeeper[] zkArr = ReconfigTest.createHandles(qu);
+
+ ArrayList<String> members = new ArrayList<String>();
+ members.add("group.1=3:4:5");
+ members.add("group.2=1:2");
+ members.add("weight.1=0");
+ members.add("weight.2=0");
+ members.add("weight.3=1");
+ members.add("weight.4=1");
+ members.add("weight.5=1");
+
+ for (int i = 1; i <= 5; i++) {
+ members.add("server." + i + "=127.0.0.1:"
+ + qu.getPeer(i).peer.getQuorumAddress().getPort() + ":"
+ + qu.getPeer(i).peer.getElectionAddress().getPort() + ";"
+ + "127.0.0.1:" + qu.getPeer(i).peer.getClientPort());
+ }
+
+ // Change the quorum system from majority to hierarchical.
+ ReconfigTest.reconfig(zkArr[1], null, null, members, -1);
+ ReconfigTest.testNormalOperation(zkArr[1], zkArr[2]);
+
+ // Attempt an incremental reconfig.
+ List<String> leavingServers = new ArrayList<String>();
+ leavingServers.add("3");
+ try {
+ zkArr[1].reconfig(null, leavingServers, null, -1, null);
+ Assert.fail("Reconfig should have failed since the current config isn't Majority QS");
+ } catch (KeeperException.BadArgumentsException e) {
+ // We expect this to happen.
+ } catch (Exception e) {
+ Assert.fail("Should have been BadArgumentsException!");
+ }
+
+ ReconfigTest.closeAllHandles(zkArr);
+ }
+
+ /*
+ * Test that a reconfiguration fails if the proposed change would leave the
+ * cluster with less than 2 participants (StandaloneEnabled = true).
+ * StandaloneDisabledTest.java (startSingleServerTest) checks that if
+ * StandaloneEnabled = false its legal to remove all but one remaining
+ * server.
+ */
+ @Test
+ public void testTooFewRemainingPariticipants() throws Exception {
+ qu = new QuorumUtil(1); // create 3 servers
+ qu.disableJMXTest = true;
+ qu.startAll();
+ ZooKeeper[] zkArr = ReconfigTest.createHandles(qu);
+
+ List<String> leavingServers = new ArrayList<String>();
+ leavingServers.add("2");
+ leavingServers.add("3");
+ try {
+ zkArr[1].reconfig(null, leavingServers, null, -1, null);
+ Assert.fail("Reconfig should have failed since the current config version is not 8");
+ } catch (KeeperException.BadArgumentsException e) {
+ // We expect this to happen.
+ } catch (Exception e) {
+ Assert.fail("Should have been BadArgumentsException!");
+ }
+
+ ReconfigTest.closeAllHandles(zkArr);
+ }
+
+ /*
+ * Tests that a conditional reconfig fails if the specified version doesn't correspond
+ * to the version of the current config.
+ */
+ @Test
+ public void testReconfigVersionConditionFails() throws Exception {
+ qu = new QuorumUtil(1); // create 3 servers
+ qu.disableJMXTest = true;
+ qu.startAll();
+ ZooKeeper[] zkArr = ReconfigTest.createHandles(qu);
+
+ List<String> leavingServers = new ArrayList<String>();
+ leavingServers.add("3");
+ try {
+ zkArr[1].reconfig(null, leavingServers, null, 8, null);
+ Assert.fail("Reconfig should have failed since the current config version is not 8");
+ } catch (KeeperException.BadVersionException e) {
+ // We expect this to happen.
+ } catch (Exception e) {
+ Assert.fail("Should have been BadVersionException!");
+ }
+
+ ReconfigTest.closeAllHandles(zkArr);
+ }
+
+ /*
+ * Tests that if a quorum of a new config is synced with the leader and a reconfig
+ * is allowed to start but then the new quorum is lost, the leader will time out and
+ * we go to leader election.
+ */
+ @Test
+ public void testLeaderTimesoutOnNewQuorum() throws Exception {
+ qu = new QuorumUtil(1); // create 3 servers
+ qu.disableJMXTest = true;
+ qu.startAll();
+ ZooKeeper[] zkArr = ReconfigTest.createHandles(qu);
+
+ List<String> leavingServers = new ArrayList<String>();
+ leavingServers.add("3");
+ qu.shutdown(2);
+ try {
+ // Since we just shut down server 2, its still considered "synced"
+ // by the leader, which allows us to start the reconfig
+ // (PrepRequestProcessor checks that a quorum of the new
+ // config is synced before starting a reconfig).
+ // We try to remove server 3, which requires a quorum of {1,2,3}
+ // (we have that) and of {1,2}, but 2 is down so we won't get a
+ // quorum of new config ACKs.
+ zkArr[1].reconfig(null, leavingServers, null, -1, null);
+ Assert.fail("Reconfig should have failed since we don't have quorum of new config");
+ } catch (KeeperException.ConnectionLossException e) {
+ // We expect leader to lose quorum of proposed config and time out
+ } catch (Exception e) {
+ Assert.fail("Should have been ConnectionLossException!");
+ }
+
+ // The leader should time out and remaining servers should go into
+ // LOOKING state. A new leader won't be established since that
+ // would require completing the reconfig, which is not possible while
+ // 2 is down.
+ Assert.assertEquals(QuorumStats.Provider.LOOKING_STATE,
+ qu.getPeer(1).peer.getServerState());
+ Assert.assertEquals(QuorumStats.Provider.LOOKING_STATE,
+ qu.getPeer(3).peer.getServerState());
+ ReconfigTest.closeAllHandles(zkArr);
+ }
+
+ /*
+ * Converting an observer into a participant may sometimes fail with a
+ * NewConfigNoQuorum exception. This test-case demonstrates the scenario.
+ * Current configuration is (A, B, C, D), where A, B and C are participant
+ * and D is an observer. Suppose that B has crashed (or never booted). If a
+ * reconfiguration is submitted where D is said to become a participant, it
+ * will fail with NewConfigNoQuorum since in this configuration, a majority
+ * of voters in the new configuration (any 3 voters), must be connected and
+ * up-to-date with the leader. An observer cannot acknowledge the history
+ * prefix sent during reconfiguration, and therefore it does not count towards
+ * these 3 required servers and the reconfiguration will be aborted. In case
+ * this happens, a client can achieve the same task by two reconfig commands:
+ * first invoke a reconfig to remove D from the configuration and then invoke a
+ * second command to add it back as a participant (follower). During the
+ * intermediate state D is a non-voting follower and can ACK the state
+ * transfer performed during the second reconfig command.
+ */
+ @Test
+ public void testObserverToParticipantConversionFails() throws Exception {
+ ClientBase.setupTestEnv();
+
+ final int SERVER_COUNT = 4;
+ int[][] ports = ReconfigRecoveryTest.generatePorts(SERVER_COUNT);
+
+ // generate old config string
+ HashSet<Integer> observers = new HashSet<Integer>();
+ observers.add(3);
+ StringBuilder sb = ReconfigRecoveryTest.generateConfig(SERVER_COUNT, ports, observers);
+ String currentQuorumCfgSection = sb.toString();
+ String nextQuorumCfgSection = currentQuorumCfgSection.replace("observer", "participant");
+
+ MainThread mt[] = new MainThread[SERVER_COUNT];
+ ZooKeeper zk[] = new ZooKeeper[SERVER_COUNT];
+
+ // Server 0 stays down
+ for (int i = 1; i < SERVER_COUNT; i++) {
+ mt[i] = new MainThread(i, ports[i][2], currentQuorumCfgSection,
+ true, "100000000");
+ mt[i].start();
+ zk[i] = new ZooKeeper("127.0.0.1:" + ports[i][2],
+ ClientBase.CONNECTION_TIMEOUT, this);
+ }
+
+ for (int i = 1; i < SERVER_COUNT; i++) {
+ Assert.assertTrue("waiting for server " + i + " being up",
+ ClientBase.waitForServerUp("127.0.0.1:" + ports[i][2],
+ CONNECTION_TIMEOUT * 2));
+ }
+
+ try {
+ zk[1].reconfig("", "", nextQuorumCfgSection, -1, new Stat());
+ Assert.fail("Reconfig should have failed with NewConfigNoQuorum");
+ } catch (NewConfigNoQuorum e) {
+ // This is expected case since server 0 is down and 3 can't vote
+ // (observer in current role) and we need 3 votes from 0, 1, 2, 3,
+ } catch (Exception e) {
+ Assert.fail("Reconfig should have failed with NewConfigNoQuorum");
+ }
+ // In this scenario to change 3's role to participant we need to remove it first
+ ArrayList<String> leavingServers = new ArrayList<String>();
+ leavingServers.add("3");
+ ReconfigTest.reconfig(zk[1], null, leavingServers, null, -1);
+ ReconfigTest.testNormalOperation(zk[2], zk[3]);
+ ReconfigTest.testServerHasConfig(zk[3], null, leavingServers);
+
+ // Now we're adding it back as a participant and everything should work.
+ List<String> newMembers = Arrays.asList(nextQuorumCfgSection.split("\n"));
+ ReconfigTest.reconfig(zk[1], null, null, newMembers, -1);
+ ReconfigTest.testNormalOperation(zk[2], zk[3]);
+ for (int i = 1; i < SERVER_COUNT; i++) {
+ ReconfigTest.testServerHasConfig(zk[i], newMembers, null);
+ }
+ for (int i = 1; i < SERVER_COUNT; i++) {
+ zk[i].close();
+ mt[i].shutdown();
+ }
+ }
+}
Modified: zookeeper/branches/branch-3.5/src/java/test/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.5/src/java/test/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java?rev=1620112&r1=1620111&r2=1620112&view=diff
==============================================================================
--- zookeeper/branches/branch-3.5/src/java/test/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java (original)
+++ zookeeper/branches/branch-3.5/src/java/test/org/apache/zookeeper/server/quorum/ReconfigRecoveryTest.java Sun Aug 24 05:44:25 2014
@@ -548,7 +548,7 @@ public class ReconfigRecoveryTest extend
/*
* Generates 3 ports per server
*/
- private int[][] generatePorts(int numServers) {
+ public static int[][] generatePorts(int numServers) {
int[][] ports = new int[numServers][];
for (int i = 0; i < numServers; i++) {
ports[i] = new int[3];
@@ -563,7 +563,7 @@ public class ReconfigRecoveryTest extend
* Creates a configuration string for servers 0..numServers-1 Ids in
* observerIds correspond to observers, other ids are for participants.
*/
- private StringBuilder generateConfig(int numServers, int[][] ports,
+ public static StringBuilder generateConfig(int numServers, int[][] ports,
HashSet<Integer> observerIds) {
StringBuilder sb = new StringBuilder();
for (int i = 0; i < numServers; i++) {
Modified: zookeeper/branches/branch-3.5/src/java/test/org/apache/zookeeper/server/quorum/StandaloneDisabledTest.java
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.5/src/java/test/org/apache/zookeeper/server/quorum/StandaloneDisabledTest.java?rev=1620112&r1=1620111&r2=1620112&view=diff
==============================================================================
--- zookeeper/branches/branch-3.5/src/java/test/org/apache/zookeeper/server/quorum/StandaloneDisabledTest.java (original)
+++ zookeeper/branches/branch-3.5/src/java/test/org/apache/zookeeper/server/quorum/StandaloneDisabledTest.java Sun Aug 24 05:44:25 2014
@@ -70,7 +70,7 @@ public class StandaloneDisabledTest exte
LOG.info("Configuration after adding 2 followers:\n"
+ new String(zkHandles[leaderId].getConfig(this, new Stat())));
- //shutdown leader- quorum should still exist
+ //shutdown leader- quorum should still exist
shutDownServer(leaderId);
ReconfigTest.testNormalOperation(zkHandles[follower1], zkHandles[follower2]);
@@ -92,6 +92,18 @@ public class StandaloneDisabledTest exte
LOG.info("Configuration after removing leader and follower 1:\n"
+ new String(zkHandles[follower2].getConfig(this, new Stat())));
+ // Try to remove follower2, which is the only remaining server. This should fail.
+ reconfigServers.clear();
+ reconfigServers.add(Integer.toString(follower2));
+ try {
+ zkHandles[follower2].reconfig(null, reconfigServers, null, -1, new Stat());
+ Assert.fail("reconfig completed successfully even though there is no quorum up in new config!");
+ } catch (KeeperException.BadArgumentsException e) {
+ // This is expected.
+ } catch (Exception e) {
+ Assert.fail("Should have been BadArgumentsException!");
+ }
+
//Add two participants and change them to observers to check
//that we can reconfigure down to one participant with observers.
ArrayList<String> observerStrings = new ArrayList<String>();
@@ -240,4 +252,4 @@ public class StandaloneDisabledTest exte
Assert.assertFalse("Observer was able to start by itself!",
ClientBase.waitForServerUp("127.0.0.1:" + clientPort, CONNECTION_TIMEOUT));
}
-}
\ No newline at end of file
+}
Modified: zookeeper/branches/branch-3.5/src/java/test/org/apache/zookeeper/test/ReconfigTest.java
URL: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.5/src/java/test/org/apache/zookeeper/test/ReconfigTest.java?rev=1620112&r1=1620111&r2=1620112&view=diff
==============================================================================
--- zookeeper/branches/branch-3.5/src/java/test/org/apache/zookeeper/test/ReconfigTest.java (original)
+++ zookeeper/branches/branch-3.5/src/java/test/org/apache/zookeeper/test/ReconfigTest.java Sun Aug 24 05:44:25 2014
@@ -171,7 +171,7 @@ public class ReconfigTest extends ZKTest
return leaderId;
}
- private ZooKeeper[] createHandles(QuorumUtil qu) throws IOException {
+ public static ZooKeeper[] createHandles(QuorumUtil qu) throws IOException {
// create an extra handle, so we can index the handles from 1 to qu.ALL
// using the server id.
ZooKeeper[] zkArr = new ZooKeeper[qu.ALL + 1];
@@ -187,7 +187,7 @@ public class ReconfigTest extends ZKTest
return zkArr;
}
- private void closeAllHandles(ZooKeeper[] zkArr) throws InterruptedException {
+ public static void closeAllHandles(ZooKeeper[] zkArr) throws InterruptedException {
for (ZooKeeper zk : zkArr)
if (zk != null)
zk.close();
@@ -409,43 +409,6 @@ public class ReconfigTest extends ZKTest
}
@Test
- public void testLeaderTimesoutOnNewQuorum() throws Exception {
- qu = new QuorumUtil(1); // create 3 servers
- qu.disableJMXTest = true;
- qu.startAll();
- ZooKeeper[] zkArr = createHandles(qu);
-
- List<String> leavingServers = new ArrayList<String>();
- leavingServers.add("3");
- qu.shutdown(2);
- try {
- // Since we just shut down server 2, its still considered "synced"
- // by the leader, which allows us to start the reconfig
- // (PrepRequestProcessor checks that a quorum of the new
- // config is synced before starting a reconfig).
- // We try to remove server 3, which requires a quorum of {1,2,3}
- // (we have that) and of {1,2}, but 2 is down so we won't get a
- // quorum of new config ACKs.
- zkArr[1].reconfig(null, leavingServers, null, -1, null);
- Assert.fail("Reconfig should have failed since we don't have quorum of new config");
- } catch (KeeperException.ConnectionLossException e) {
- // We expect leader to loose quorum of proposed config and time out
- } catch (Exception e) {
- Assert.fail("Should have been ConnectionLossException!");
- }
-
- // The leader should time out and remaining servers should go into
- // LOOKING state. A new leader won't be established since that
- // would require completing the reconfig, which is not possible while
- // 2 is down.
- Assert.assertEquals(QuorumStats.Provider.LOOKING_STATE,
- qu.getPeer(1).peer.getServerState());
- Assert.assertEquals(QuorumStats.Provider.LOOKING_STATE,
- qu.getPeer(3).peer.getServerState());
- closeAllHandles(zkArr);
- }
-
- @Test
public void testRemoveOneAsynchronous() throws Exception {
qu = new QuorumUtil(2);
qu.disableJMXTest = true;