|
@@ -148,11 +148,11 @@ public class ReconfigRollingRestartCompatibilityTest extends QuorumPeerTestBase
|
|
|
}
|
|
|
|
|
|
@Test(timeout = 90000)
|
|
|
- // This test simulate the use case of change of membership through rolling
|
|
|
- // restart. For a 3 node ensemble we expand it to a 5 node ensemble, verify
|
|
|
+ // This test simulate the use case of change of membership by starting new servers
|
|
|
+ // without dynamic reconfig. For a 3 node ensemble we expand it to a 5 node ensemble, verify
|
|
|
// during the process each node has the expected configuration setting pushed
|
|
|
// via updating local zoo.cfg file.
|
|
|
- public void testRollingRestartWithMembershipChange() throws Exception {
|
|
|
+ public void testExtendingQuorumWithNewMembers() throws Exception {
|
|
|
int serverCount = 3;
|
|
|
String config = generateNewQuorumConfig(serverCount);
|
|
|
QuorumPeerTestBase.MainThread[] mt = new QuorumPeerTestBase.MainThread[serverCount];
|
|
@@ -174,7 +174,7 @@ public class ReconfigRollingRestartCompatibilityTest extends QuorumPeerTestBase
|
|
|
|
|
|
Map<Integer, String> oldServerAddress = new HashMap<>(serverAddress);
|
|
|
List<String> newServers = new ArrayList<>(joiningServers);
|
|
|
- config = updateExistingQuorumConfig(Arrays.asList(3, 4), new ArrayList<Integer>());
|
|
|
+ config = updateExistingQuorumConfig(Arrays.asList(3, 4), new ArrayList<>());
|
|
|
newServers.add(serverAddress.get(3));
|
|
|
newServers.add(serverAddress.get(4));
|
|
|
serverCount = serverAddress.size();
|
|
@@ -209,6 +209,138 @@ public class ReconfigRollingRestartCompatibilityTest extends QuorumPeerTestBase
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testRollingRestartWithExtendedMembershipConfig() throws Exception {
|
|
|
+ // in this test we are performing rolling restart with extended quorum config, see ZOOKEEPER-3829
|
|
|
+
|
|
|
+ // Start a quorum with 3 members
|
|
|
+ int serverCount = 3;
|
|
|
+ String config = generateNewQuorumConfig(serverCount);
|
|
|
+ QuorumPeerTestBase.MainThread[] mt = new QuorumPeerTestBase.MainThread[serverCount];
|
|
|
+ List<String> joiningServers = new ArrayList<>();
|
|
|
+ for (int i = 0; i < serverCount; i++) {
|
|
|
+ mt[i] = new QuorumPeerTestBase.MainThread(i, clientPorts.get(i), config, false);
|
|
|
+ mt[i].start();
|
|
|
+ joiningServers.add(serverAddress.get(i));
|
|
|
+ }
|
|
|
+ for (int i = 0; i < serverCount; i++) {
|
|
|
+ assertTrue("waiting for server " + i + " being up", ClientBase.waitForServerUp("127.0.0.1:" + clientPorts.get(i), CONNECTION_TIMEOUT));
|
|
|
+ }
|
|
|
+ for (int i = 0; i < serverCount; i++) {
|
|
|
+ verifyQuorumConfig(i, joiningServers, null);
|
|
|
+ verifyQuorumMembers(mt[i]);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Create updated config with 4 members
|
|
|
+ List<String> newServers = new ArrayList<>(joiningServers);
|
|
|
+ config = updateExistingQuorumConfig(Arrays.asList(3), new ArrayList<>());
|
|
|
+ newServers.add(serverAddress.get(3));
|
|
|
+ serverCount = serverAddress.size();
|
|
|
+ assertEquals("Server count should be 4 after config update.", serverCount, 4);
|
|
|
+
|
|
|
+ // We are adding one new server to the ensemble. The new server should be started with the new config
|
|
|
+ mt = Arrays.copyOf(mt, mt.length + 1);
|
|
|
+ mt[3] = new QuorumPeerTestBase.MainThread(3, clientPorts.get(3), config, false);
|
|
|
+ mt[3].start();
|
|
|
+ assertTrue("waiting for server 3 being up", ClientBase.waitForServerUp("127.0.0.1:" + clientPorts.get(3), CONNECTION_TIMEOUT));
|
|
|
+ verifyQuorumConfig(3, newServers, null);
|
|
|
+ verifyQuorumMembers(mt[3]);
|
|
|
+
|
|
|
+ // Now we restart the first 3 servers, one-by-one with the new config
|
|
|
+ for (int i = 0; i < 3; i++) {
|
|
|
+ mt[i].shutdown();
|
|
|
+
|
|
|
+ assertTrue(String.format("Timeout during waiting for server %d to go down", i),
|
|
|
+ ClientBase.waitForServerDown("127.0.0.1:" + clientPorts.get(i), ClientBase.CONNECTION_TIMEOUT));
|
|
|
+
|
|
|
+ mt[i] = new QuorumPeerTestBase.MainThread(i, clientPorts.get(i), config, false);
|
|
|
+ mt[i].start();
|
|
|
+ assertTrue("waiting for server " + i + " being up", ClientBase.waitForServerUp("127.0.0.1:" + clientPorts.get(i), CONNECTION_TIMEOUT));
|
|
|
+ verifyQuorumConfig(i, newServers, null);
|
|
|
+ verifyQuorumMembers(mt[i]);
|
|
|
+ }
|
|
|
+
|
|
|
+ // now verify that all nodes can handle traffic
|
|
|
+ for (int i = 0; i < 4; ++i) {
|
|
|
+ ZooKeeper zk = ClientBase.createZKClient("127.0.0.1:" + clientPorts.get(i));
|
|
|
+ ReconfigTest.testNormalOperation(zk, zk, false);
|
|
|
+ }
|
|
|
+
|
|
|
+ for (int i = 0; i < 4; ++i) {
|
|
|
+ mt[i].shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testRollingRestartWithHostAddedAndRemoved() throws Exception {
|
|
|
+ // in this test we are performing rolling restart with a new quorum config,
|
|
|
+ // contains a deleted node and a new node
|
|
|
+
|
|
|
+ // Start a quorum with 3 members
|
|
|
+ int serverCount = 3;
|
|
|
+ String config = generateNewQuorumConfig(serverCount);
|
|
|
+ QuorumPeerTestBase.MainThread[] mt = new QuorumPeerTestBase.MainThread[serverCount];
|
|
|
+ List<String> originalServers = new ArrayList<>();
|
|
|
+ for (int i = 0; i < serverCount; i++) {
|
|
|
+ mt[i] = new QuorumPeerTestBase.MainThread(i, clientPorts.get(i), config, false);
|
|
|
+ mt[i].start();
|
|
|
+ originalServers.add(serverAddress.get(i));
|
|
|
+ }
|
|
|
+ for (int i = 0; i < serverCount; i++) {
|
|
|
+ assertTrue("waiting for server " + i + " being up", ClientBase.waitForServerUp("127.0.0.1:" + clientPorts.get(i), CONNECTION_TIMEOUT));
|
|
|
+ }
|
|
|
+ for (int i = 0; i < serverCount; i++) {
|
|
|
+ verifyQuorumConfig(i, originalServers, null);
|
|
|
+ verifyQuorumMembers(mt[i]);
|
|
|
+ }
|
|
|
+
|
|
|
+ // we are stopping the third server (myid=2)
|
|
|
+ mt[2].shutdown();
|
|
|
+ assertTrue(String.format("Timeout during waiting for server %d to go down", 2),
|
|
|
+ ClientBase.waitForServerDown("127.0.0.1:" + clientPorts.get(2), ClientBase.CONNECTION_TIMEOUT));
|
|
|
+ String leavingServer = originalServers.get(2);
|
|
|
+
|
|
|
+ // Create updated config with the first 2 existing members, but we remove 3rd and add one with different myid
|
|
|
+ config = updateExistingQuorumConfig(Arrays.asList(3), Arrays.asList(2));
|
|
|
+ List<String> newServers = new ArrayList<>(serverAddress.values());
|
|
|
+ serverCount = serverAddress.size();
|
|
|
+ assertEquals("Server count should be 3 after config update.", serverCount, 3);
|
|
|
+
|
|
|
+
|
|
|
+ // We are adding one new server to the ensemble. The new server should be started with the new config
|
|
|
+ mt = Arrays.copyOf(mt, mt.length + 1);
|
|
|
+ mt[3] = new QuorumPeerTestBase.MainThread(3, clientPorts.get(3), config, false);
|
|
|
+ mt[3].start();
|
|
|
+ assertTrue("waiting for server 3 being up", ClientBase.waitForServerUp("127.0.0.1:" + clientPorts.get(3), CONNECTION_TIMEOUT));
|
|
|
+ verifyQuorumConfig(3, newServers, Arrays.asList(leavingServer));
|
|
|
+ verifyQuorumMembers(mt[3]);
|
|
|
+
|
|
|
+ // Now we restart the first 2 servers, one-by-one with the new config
|
|
|
+ for (int i = 0; i < 2; i++) {
|
|
|
+ mt[i].shutdown();
|
|
|
+
|
|
|
+ assertTrue(String.format("Timeout during waiting for server %d to go down", i),
|
|
|
+ ClientBase.waitForServerDown("127.0.0.1:" + clientPorts.get(i), ClientBase.CONNECTION_TIMEOUT));
|
|
|
+
|
|
|
+ mt[i] = new QuorumPeerTestBase.MainThread(i, clientPorts.get(i), config, false);
|
|
|
+ mt[i].start();
|
|
|
+ assertTrue("waiting for server " + i + " being up", ClientBase.waitForServerUp("127.0.0.1:" + clientPorts.get(i), CONNECTION_TIMEOUT));
|
|
|
+ verifyQuorumConfig(i, newServers, null);
|
|
|
+ verifyQuorumMembers(mt[i]);
|
|
|
+ }
|
|
|
+
|
|
|
+ // now verify that all three nodes can handle traffic
|
|
|
+ for (int i : serverAddress.keySet()) {
|
|
|
+ ZooKeeper zk = ClientBase.createZKClient("127.0.0.1:" + clientPorts.get(i));
|
|
|
+ ReconfigTest.testNormalOperation(zk, zk, false);
|
|
|
+ }
|
|
|
+
|
|
|
+ for (int i : serverAddress.keySet()) {
|
|
|
+ mt[i].shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
// Verify each quorum peer has expected config in its config zNode.
|
|
|
private void verifyQuorumConfig(int sid, List<String> joiningServers, List<String> leavingServers) throws Exception {
|
|
|
ZooKeeper zk = ClientBase.createZKClient("127.0.0.1:" + clientPorts.get(sid));
|