HADOOP-11149. Increase the timeout of TestZKFailoverController. Contributed by Steve Loughran.
(cherry picked from commit 579f9030da
)
Conflicts:
hadoop-common-project/hadoop-common/CHANGES.txt
This commit is contained in:
parent
1b8641a3c3
commit
5220d4dc27
|
@ -25,6 +25,9 @@ Release 2.7.4 - UNRELEASED
|
|||
HADOOP-12991. Conflicting default ports in DelegateToFileSystem.
|
||||
(Kai Sasaki via aajisaka)
|
||||
|
||||
HADOOP-11149. Increase the timeout of TestZKFailoverController.
|
||||
(Steve Loughran via wheat9)
|
||||
|
||||
Release 2.7.3 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -102,9 +102,11 @@ public class MiniZKFCCluster {
|
|||
* @throws Exception if either of the services had encountered a fatal error
|
||||
*/
|
||||
public void stop() throws Exception {
|
||||
for (DummyZKFCThread thr : thrs) {
|
||||
if (thr != null) {
|
||||
thr.interrupt();
|
||||
if (thrs != null) {
|
||||
for (DummyZKFCThread thr : thrs) {
|
||||
if (thr != null) {
|
||||
thr.interrupt();
|
||||
}
|
||||
}
|
||||
}
|
||||
if (ctx != null) {
|
||||
|
|
|
@ -34,14 +34,23 @@ import org.apache.zookeeper.KeeperException;
|
|||
import org.apache.zookeeper.ZooKeeper;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.Timeout;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
public class TestZKFailoverController extends ClientBaseWithFixes {
|
||||
private Configuration conf;
|
||||
private MiniZKFCCluster cluster;
|
||||
|
||||
|
||||
/**
|
||||
* Set the timeout for every test
|
||||
*/
|
||||
@Rule
|
||||
public Timeout testTimeout = new Timeout(3 * 60 * 1000);
|
||||
|
||||
// Set up ZK digest-based credentials for the purposes of the tests,
|
||||
// to make sure all of our functionality works with auth and ACLs
|
||||
// present.
|
||||
|
@ -74,11 +83,21 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
|
|||
this.cluster = new MiniZKFCCluster(conf, getServer(serverFactory));
|
||||
}
|
||||
|
||||
@After
|
||||
public void teardown() {
|
||||
if (cluster != null) {
|
||||
try {
|
||||
cluster.stop();
|
||||
} catch (Exception e) {
|
||||
LOG.warn("When stopping the cluster", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
/**
|
||||
* Test that the various command lines for formatting the ZK directory
|
||||
* function correctly.
|
||||
*/
|
||||
@Test(timeout=15000)
|
||||
@Test
|
||||
public void testFormatZK() throws Exception {
|
||||
DummyHAService svc = cluster.getService(1);
|
||||
// Run without formatting the base dir,
|
||||
|
@ -101,14 +120,14 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
|
|||
* Test that if ZooKeeper is not running, the correct error
|
||||
* code is returned.
|
||||
*/
|
||||
@Test(timeout=15000)
|
||||
@Test
|
||||
public void testNoZK() throws Exception {
|
||||
stopServer();
|
||||
DummyHAService svc = cluster.getService(1);
|
||||
assertEquals(ZKFailoverController.ERR_CODE_NO_ZK,
|
||||
runFC(svc));
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testFormatOneClusterLeavesOtherClustersAlone() throws Exception {
|
||||
DummyHAService svc = cluster.getService(1);
|
||||
|
@ -146,7 +165,7 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
|
|||
* Test that automatic failover won't run against a target that hasn't
|
||||
* explicitly enabled the feature.
|
||||
*/
|
||||
@Test(timeout=10000)
|
||||
@Test
|
||||
public void testWontRunWhenAutoFailoverDisabled() throws Exception {
|
||||
DummyHAService svc = cluster.getService(1);
|
||||
svc = Mockito.spy(svc);
|
||||
|
@ -162,7 +181,7 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
|
|||
* Test that, if ACLs are specified in the configuration, that
|
||||
* it sets the ACLs when formatting the parent node.
|
||||
*/
|
||||
@Test(timeout=15000)
|
||||
@Test
|
||||
public void testFormatSetsAcls() throws Exception {
|
||||
// Format the base dir, should succeed
|
||||
DummyHAService svc = cluster.getService(1);
|
||||
|
@ -184,7 +203,7 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
|
|||
* Test that the ZKFC won't run if fencing is not configured for the
|
||||
* local service.
|
||||
*/
|
||||
@Test(timeout=15000)
|
||||
@Test
|
||||
public void testFencingMustBeConfigured() throws Exception {
|
||||
DummyHAService svc = Mockito.spy(cluster.getService(0));
|
||||
Mockito.doThrow(new BadFencingConfigurationException("no fencing"))
|
||||
|
@ -202,31 +221,27 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
|
|||
* transition is used when possible, falling back to fencing when
|
||||
* the graceful approach fails.
|
||||
*/
|
||||
@Test(timeout=15000)
|
||||
@Test
|
||||
public void testAutoFailoverOnBadHealth() throws Exception {
|
||||
try {
|
||||
cluster.start();
|
||||
DummyHAService svc1 = cluster.getService(1);
|
||||
|
||||
LOG.info("Faking svc0 unhealthy, should failover to svc1");
|
||||
cluster.setHealthy(0, false);
|
||||
|
||||
LOG.info("Waiting for svc0 to enter initializing state");
|
||||
cluster.waitForHAState(0, HAServiceState.INITIALIZING);
|
||||
cluster.waitForHAState(1, HAServiceState.ACTIVE);
|
||||
|
||||
LOG.info("Allowing svc0 to be healthy again, making svc1 unreachable " +
|
||||
"and fail to gracefully go to standby");
|
||||
cluster.setUnreachable(1, true);
|
||||
cluster.setHealthy(0, true);
|
||||
|
||||
// Should fail back to svc0 at this point
|
||||
cluster.waitForHAState(0, HAServiceState.ACTIVE);
|
||||
// and fence svc1
|
||||
Mockito.verify(svc1.fencer).fence(Mockito.same(svc1));
|
||||
} finally {
|
||||
cluster.stop();
|
||||
}
|
||||
cluster.start();
|
||||
DummyHAService svc1 = cluster.getService(1);
|
||||
|
||||
LOG.info("Faking svc0 unhealthy, should failover to svc1");
|
||||
cluster.setHealthy(0, false);
|
||||
|
||||
LOG.info("Waiting for svc0 to enter initializing state");
|
||||
cluster.waitForHAState(0, HAServiceState.INITIALIZING);
|
||||
cluster.waitForHAState(1, HAServiceState.ACTIVE);
|
||||
|
||||
LOG.info("Allowing svc0 to be healthy again, making svc1 unreachable " +
|
||||
"and fail to gracefully go to standby");
|
||||
cluster.setUnreachable(1, true);
|
||||
cluster.setHealthy(0, true);
|
||||
|
||||
// Should fail back to svc0 at this point
|
||||
cluster.waitForHAState(0, HAServiceState.ACTIVE);
|
||||
// and fence svc1
|
||||
Mockito.verify(svc1.fencer).fence(Mockito.same(svc1));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -235,120 +250,104 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
|
|||
* transition is used when possible, falling back to fencing when
|
||||
* the graceful approach fails.
|
||||
*/
|
||||
@Test(timeout=15000)
|
||||
@Test
|
||||
public void testAutoFailoverOnBadState() throws Exception {
|
||||
try {
|
||||
cluster.start();
|
||||
DummyHAService svc0 = cluster.getService(0);
|
||||
LOG.info("Faking svc0 to change the state, should failover to svc1");
|
||||
svc0.state = HAServiceState.STANDBY;
|
||||
|
||||
// Should fail back to svc0 at this point
|
||||
cluster.waitForHAState(1, HAServiceState.ACTIVE);
|
||||
} finally {
|
||||
cluster.stop();
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=15000)
|
||||
public void testAutoFailoverOnLostZKSession() throws Exception {
|
||||
try {
|
||||
cluster.start();
|
||||
cluster.start();
|
||||
DummyHAService svc0 = cluster.getService(0);
|
||||
LOG.info("Faking svc0 to change the state, should failover to svc1");
|
||||
svc0.state = HAServiceState.STANDBY;
|
||||
|
||||
// Expire svc0, it should fail over to svc1
|
||||
cluster.expireAndVerifyFailover(0, 1);
|
||||
|
||||
// Expire svc1, it should fail back to svc0
|
||||
cluster.expireAndVerifyFailover(1, 0);
|
||||
|
||||
LOG.info("======= Running test cases second time to test " +
|
||||
"re-establishment =========");
|
||||
// Expire svc0, it should fail over to svc1
|
||||
cluster.expireAndVerifyFailover(0, 1);
|
||||
|
||||
// Expire svc1, it should fail back to svc0
|
||||
cluster.expireAndVerifyFailover(1, 0);
|
||||
} finally {
|
||||
cluster.stop();
|
||||
}
|
||||
// Should fail back to svc0 at this point
|
||||
cluster.waitForHAState(1, HAServiceState.ACTIVE);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAutoFailoverOnLostZKSession() throws Exception {
|
||||
cluster.start();
|
||||
|
||||
// Expire svc0, it should fail over to svc1
|
||||
cluster.expireAndVerifyFailover(0, 1);
|
||||
|
||||
// Expire svc1, it should fail back to svc0
|
||||
cluster.expireAndVerifyFailover(1, 0);
|
||||
|
||||
LOG.info("======= Running test cases second time to test " +
|
||||
"re-establishment =========");
|
||||
// Expire svc0, it should fail over to svc1
|
||||
cluster.expireAndVerifyFailover(0, 1);
|
||||
|
||||
// Expire svc1, it should fail back to svc0
|
||||
cluster.expireAndVerifyFailover(1, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that, if the standby node is unhealthy, it doesn't try to become
|
||||
* active
|
||||
*/
|
||||
@Test(timeout=15000)
|
||||
@Test
|
||||
public void testDontFailoverToUnhealthyNode() throws Exception {
|
||||
cluster.start();
|
||||
|
||||
// Make svc1 unhealthy, and wait for its FC to notice the bad health.
|
||||
cluster.setHealthy(1, false);
|
||||
cluster.waitForHealthState(1, HealthMonitor.State.SERVICE_UNHEALTHY);
|
||||
|
||||
// Expire svc0
|
||||
cluster.getElector(0).preventSessionReestablishmentForTests();
|
||||
try {
|
||||
cluster.start();
|
||||
cluster.expireActiveLockHolder(0);
|
||||
|
||||
// Make svc1 unhealthy, and wait for its FC to notice the bad health.
|
||||
cluster.setHealthy(1, false);
|
||||
cluster.waitForHealthState(1, HealthMonitor.State.SERVICE_UNHEALTHY);
|
||||
|
||||
// Expire svc0
|
||||
cluster.getElector(0).preventSessionReestablishmentForTests();
|
||||
try {
|
||||
cluster.expireActiveLockHolder(0);
|
||||
LOG.info("Expired svc0's ZK session. Waiting a second to give svc1" +
|
||||
" a chance to take the lock, if it is ever going to.");
|
||||
Thread.sleep(1000);
|
||||
|
||||
// Ensure that no one holds the lock.
|
||||
cluster.waitForActiveLockHolder(null);
|
||||
|
||||
LOG.info("Expired svc0's ZK session. Waiting a second to give svc1" +
|
||||
" a chance to take the lock, if it is ever going to.");
|
||||
Thread.sleep(1000);
|
||||
|
||||
// Ensure that no one holds the lock.
|
||||
cluster.waitForActiveLockHolder(null);
|
||||
|
||||
} finally {
|
||||
LOG.info("Allowing svc0's elector to re-establish its connection");
|
||||
cluster.getElector(0).allowSessionReestablishmentForTests();
|
||||
}
|
||||
// svc0 should get the lock again
|
||||
cluster.waitForActiveLockHolder(0);
|
||||
} finally {
|
||||
cluster.stop();
|
||||
LOG.info("Allowing svc0's elector to re-establish its connection");
|
||||
cluster.getElector(0).allowSessionReestablishmentForTests();
|
||||
}
|
||||
// svc0 should get the lock again
|
||||
cluster.waitForActiveLockHolder(0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that the ZKFC successfully quits the election when it fails to
|
||||
* become active. This allows the old node to successfully fail back.
|
||||
*/
|
||||
@Test(timeout=15000)
|
||||
@Test
|
||||
public void testBecomingActiveFails() throws Exception {
|
||||
try {
|
||||
cluster.start();
|
||||
DummyHAService svc1 = cluster.getService(1);
|
||||
|
||||
LOG.info("Making svc1 fail to become active");
|
||||
cluster.setFailToBecomeActive(1, true);
|
||||
|
||||
LOG.info("Faking svc0 unhealthy, should NOT successfully " +
|
||||
"failover to svc1");
|
||||
cluster.setHealthy(0, false);
|
||||
cluster.waitForHealthState(0, State.SERVICE_UNHEALTHY);
|
||||
cluster.waitForActiveLockHolder(null);
|
||||
cluster.start();
|
||||
DummyHAService svc1 = cluster.getService(1);
|
||||
|
||||
|
||||
Mockito.verify(svc1.proxy, Mockito.timeout(2000).atLeastOnce())
|
||||
.transitionToActive(Mockito.<StateChangeRequestInfo>any());
|
||||
LOG.info("Making svc1 fail to become active");
|
||||
cluster.setFailToBecomeActive(1, true);
|
||||
|
||||
cluster.waitForHAState(0, HAServiceState.INITIALIZING);
|
||||
cluster.waitForHAState(1, HAServiceState.STANDBY);
|
||||
|
||||
LOG.info("Faking svc0 healthy again, should go back to svc0");
|
||||
cluster.setHealthy(0, true);
|
||||
cluster.waitForHAState(0, HAServiceState.ACTIVE);
|
||||
cluster.waitForHAState(1, HAServiceState.STANDBY);
|
||||
cluster.waitForActiveLockHolder(0);
|
||||
|
||||
// Ensure that we can fail back to svc1 once it it is able
|
||||
// to become active (e.g the admin has restarted it)
|
||||
LOG.info("Allowing svc1 to become active, expiring svc0");
|
||||
svc1.failToBecomeActive = false;
|
||||
cluster.expireAndVerifyFailover(0, 1);
|
||||
} finally {
|
||||
cluster.stop();
|
||||
}
|
||||
LOG.info("Faking svc0 unhealthy, should NOT successfully " +
|
||||
"failover to svc1");
|
||||
cluster.setHealthy(0, false);
|
||||
cluster.waitForHealthState(0, State.SERVICE_UNHEALTHY);
|
||||
cluster.waitForActiveLockHolder(null);
|
||||
|
||||
|
||||
Mockito.verify(svc1.proxy, Mockito.timeout(2000).atLeastOnce())
|
||||
.transitionToActive(Mockito.<StateChangeRequestInfo>any());
|
||||
|
||||
cluster.waitForHAState(0, HAServiceState.INITIALIZING);
|
||||
cluster.waitForHAState(1, HAServiceState.STANDBY);
|
||||
|
||||
LOG.info("Faking svc0 healthy again, should go back to svc0");
|
||||
cluster.setHealthy(0, true);
|
||||
cluster.waitForHAState(0, HAServiceState.ACTIVE);
|
||||
cluster.waitForHAState(1, HAServiceState.STANDBY);
|
||||
cluster.waitForActiveLockHolder(0);
|
||||
|
||||
// Ensure that we can fail back to svc1 once it it is able
|
||||
// to become active (e.g the admin has restarted it)
|
||||
LOG.info("Allowing svc1 to become active, expiring svc0");
|
||||
svc1.failToBecomeActive = false;
|
||||
cluster.expireAndVerifyFailover(0, 1);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -356,211 +355,183 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
|
|||
* current state, without triggering any failovers, and without
|
||||
* causing the active node to enter standby state.
|
||||
*/
|
||||
@Test(timeout=15000)
|
||||
@Test
|
||||
public void testZooKeeperFailure() throws Exception {
|
||||
try {
|
||||
cluster.start();
|
||||
cluster.start();
|
||||
|
||||
// Record initial ZK sessions
|
||||
long session0 = cluster.getElector(0).getZKSessionIdForTests();
|
||||
long session1 = cluster.getElector(1).getZKSessionIdForTests();
|
||||
// Record initial ZK sessions
|
||||
long session0 = cluster.getElector(0).getZKSessionIdForTests();
|
||||
long session1 = cluster.getElector(1).getZKSessionIdForTests();
|
||||
|
||||
LOG.info("====== Stopping ZK server");
|
||||
stopServer();
|
||||
waitForServerDown(hostPort, CONNECTION_TIMEOUT);
|
||||
|
||||
LOG.info("====== Waiting for services to enter NEUTRAL mode");
|
||||
cluster.waitForElectorState(0,
|
||||
ActiveStandbyElector.State.NEUTRAL);
|
||||
cluster.waitForElectorState(1,
|
||||
ActiveStandbyElector.State.NEUTRAL);
|
||||
LOG.info("====== Stopping ZK server");
|
||||
stopServer();
|
||||
waitForServerDown(hostPort, CONNECTION_TIMEOUT);
|
||||
|
||||
LOG.info("====== Checking that the services didn't change HA state");
|
||||
assertEquals(HAServiceState.ACTIVE, cluster.getService(0).state);
|
||||
assertEquals(HAServiceState.STANDBY, cluster.getService(1).state);
|
||||
|
||||
LOG.info("====== Restarting server");
|
||||
startServer();
|
||||
waitForServerUp(hostPort, CONNECTION_TIMEOUT);
|
||||
LOG.info("====== Waiting for services to enter NEUTRAL mode");
|
||||
cluster.waitForElectorState(0,
|
||||
ActiveStandbyElector.State.NEUTRAL);
|
||||
cluster.waitForElectorState(1,
|
||||
ActiveStandbyElector.State.NEUTRAL);
|
||||
|
||||
// Nodes should go back to their original states, since they re-obtain
|
||||
// the same sessions.
|
||||
cluster.waitForElectorState(0, ActiveStandbyElector.State.ACTIVE);
|
||||
cluster.waitForElectorState(1, ActiveStandbyElector.State.STANDBY);
|
||||
// Check HA states didn't change.
|
||||
cluster.waitForHAState(0, HAServiceState.ACTIVE);
|
||||
cluster.waitForHAState(1, HAServiceState.STANDBY);
|
||||
LOG.info("====== Checking that the services didn't change HA state");
|
||||
assertEquals(HAServiceState.ACTIVE, cluster.getService(0).state);
|
||||
assertEquals(HAServiceState.STANDBY, cluster.getService(1).state);
|
||||
|
||||
// Check they re-used the same sessions and didn't spuriously reconnect
|
||||
assertEquals(session0,
|
||||
cluster.getElector(0).getZKSessionIdForTests());
|
||||
assertEquals(session1,
|
||||
cluster.getElector(1).getZKSessionIdForTests());
|
||||
} finally {
|
||||
cluster.stop();
|
||||
}
|
||||
LOG.info("====== Restarting server");
|
||||
startServer();
|
||||
waitForServerUp(hostPort, CONNECTION_TIMEOUT);
|
||||
|
||||
// Nodes should go back to their original states, since they re-obtain
|
||||
// the same sessions.
|
||||
cluster.waitForElectorState(0, ActiveStandbyElector.State.ACTIVE);
|
||||
cluster.waitForElectorState(1, ActiveStandbyElector.State.STANDBY);
|
||||
// Check HA states didn't change.
|
||||
cluster.waitForHAState(0, HAServiceState.ACTIVE);
|
||||
cluster.waitForHAState(1, HAServiceState.STANDBY);
|
||||
|
||||
// Check they re-used the same sessions and didn't spuriously reconnect
|
||||
assertEquals(session0,
|
||||
cluster.getElector(0).getZKSessionIdForTests());
|
||||
assertEquals(session1,
|
||||
cluster.getElector(1).getZKSessionIdForTests());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that the ZKFC can gracefully cede its active status.
|
||||
*/
|
||||
@Test(timeout=15000)
|
||||
@Test
|
||||
public void testCedeActive() throws Exception {
|
||||
try {
|
||||
cluster.start();
|
||||
DummyZKFC zkfc = cluster.getZkfc(0);
|
||||
// It should be in active to start.
|
||||
assertEquals(ActiveStandbyElector.State.ACTIVE,
|
||||
zkfc.getElectorForTests().getStateForTests());
|
||||
cluster.start();
|
||||
DummyZKFC zkfc = cluster.getZkfc(0);
|
||||
// It should be in active to start.
|
||||
assertEquals(ActiveStandbyElector.State.ACTIVE,
|
||||
zkfc.getElectorForTests().getStateForTests());
|
||||
|
||||
// Ask it to cede active for 3 seconds. It should respond promptly
|
||||
// (i.e. the RPC itself should not take 3 seconds!)
|
||||
ZKFCProtocol proxy = zkfc.getLocalTarget().getZKFCProxy(conf, 5000);
|
||||
long st = Time.now();
|
||||
proxy.cedeActive(3000);
|
||||
long et = Time.now();
|
||||
assertTrue("RPC to cedeActive took " + (et - st) + " ms",
|
||||
et - st < 1000);
|
||||
|
||||
// Should be in "INIT" state since it's not in the election
|
||||
// at this point.
|
||||
assertEquals(ActiveStandbyElector.State.INIT,
|
||||
zkfc.getElectorForTests().getStateForTests());
|
||||
// Ask it to cede active for 3 seconds. It should respond promptly
|
||||
// (i.e. the RPC itself should not take 3 seconds!)
|
||||
ZKFCProtocol proxy = zkfc.getLocalTarget().getZKFCProxy(conf, 5000);
|
||||
long st = Time.now();
|
||||
proxy.cedeActive(3000);
|
||||
long et = Time.now();
|
||||
assertTrue("RPC to cedeActive took " + (et - st) + " ms",
|
||||
et - st < 1000);
|
||||
|
||||
// After the prescribed 3 seconds, should go into STANDBY state,
|
||||
// since the other node in the cluster would have taken ACTIVE.
|
||||
cluster.waitForElectorState(0, ActiveStandbyElector.State.STANDBY);
|
||||
long et2 = Time.now();
|
||||
assertTrue("Should take ~3 seconds to rejoin. Only took " + (et2 - et) +
|
||||
"ms before rejoining.",
|
||||
et2 - et > 2800);
|
||||
} finally {
|
||||
cluster.stop();
|
||||
}
|
||||
// Should be in "INIT" state since it's not in the election
|
||||
// at this point.
|
||||
assertEquals(ActiveStandbyElector.State.INIT,
|
||||
zkfc.getElectorForTests().getStateForTests());
|
||||
|
||||
// After the prescribed 3 seconds, should go into STANDBY state,
|
||||
// since the other node in the cluster would have taken ACTIVE.
|
||||
cluster.waitForElectorState(0, ActiveStandbyElector.State.STANDBY);
|
||||
long et2 = Time.now();
|
||||
assertTrue("Should take ~3 seconds to rejoin. Only took " + (et2 - et) +
|
||||
"ms before rejoining.",
|
||||
et2 - et > 2800);
|
||||
}
|
||||
|
||||
@Test(timeout=25000)
|
||||
|
||||
@Test
|
||||
public void testGracefulFailover() throws Exception {
|
||||
try {
|
||||
cluster.start();
|
||||
cluster.start();
|
||||
|
||||
cluster.waitForActiveLockHolder(0);
|
||||
cluster.getService(1).getZKFCProxy(conf, 5000).gracefulFailover();
|
||||
cluster.waitForActiveLockHolder(1);
|
||||
cluster.waitForActiveLockHolder(0);
|
||||
cluster.getService(1).getZKFCProxy(conf, 5000).gracefulFailover();
|
||||
cluster.waitForActiveLockHolder(1);
|
||||
|
||||
cluster.getService(0).getZKFCProxy(conf, 5000).gracefulFailover();
|
||||
cluster.waitForActiveLockHolder(0);
|
||||
cluster.getService(0).getZKFCProxy(conf, 5000).gracefulFailover();
|
||||
cluster.waitForActiveLockHolder(0);
|
||||
|
||||
Thread.sleep(10000); // allow to quiesce
|
||||
Thread.sleep(10000); // allow to quiesce
|
||||
|
||||
assertEquals(0, cluster.getService(0).fenceCount);
|
||||
assertEquals(0, cluster.getService(1).fenceCount);
|
||||
assertEquals(2, cluster.getService(0).activeTransitionCount);
|
||||
assertEquals(1, cluster.getService(1).activeTransitionCount);
|
||||
} finally {
|
||||
cluster.stop();
|
||||
}
|
||||
assertEquals(0, cluster.getService(0).fenceCount);
|
||||
assertEquals(0, cluster.getService(1).fenceCount);
|
||||
assertEquals(2, cluster.getService(0).activeTransitionCount);
|
||||
assertEquals(1, cluster.getService(1).activeTransitionCount);
|
||||
}
|
||||
|
||||
@Test(timeout=15000)
|
||||
|
||||
@Test
|
||||
public void testGracefulFailoverToUnhealthy() throws Exception {
|
||||
cluster.start();
|
||||
|
||||
cluster.waitForActiveLockHolder(0);
|
||||
|
||||
// Mark it unhealthy, wait for it to exit election
|
||||
cluster.setHealthy(1, false);
|
||||
cluster.waitForElectorState(1, ActiveStandbyElector.State.INIT);
|
||||
|
||||
// Ask for failover, it should fail, because it's unhealthy
|
||||
try {
|
||||
cluster.start();
|
||||
|
||||
cluster.waitForActiveLockHolder(0);
|
||||
|
||||
// Mark it unhealthy, wait for it to exit election
|
||||
cluster.setHealthy(1, false);
|
||||
cluster.waitForElectorState(1, ActiveStandbyElector.State.INIT);
|
||||
|
||||
// Ask for failover, it should fail, because it's unhealthy
|
||||
try {
|
||||
cluster.getService(1).getZKFCProxy(conf, 5000).gracefulFailover();
|
||||
fail("Did not fail to graceful failover to unhealthy service!");
|
||||
} catch (ServiceFailedException sfe) {
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
cluster.getService(1).toString() +
|
||||
" is not currently healthy.", sfe);
|
||||
}
|
||||
} finally {
|
||||
cluster.stop();
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=15000)
|
||||
public void testGracefulFailoverFailBecomingActive() throws Exception {
|
||||
try {
|
||||
cluster.start();
|
||||
|
||||
cluster.waitForActiveLockHolder(0);
|
||||
cluster.setFailToBecomeActive(1, true);
|
||||
|
||||
// Ask for failover, it should fail and report back to user.
|
||||
try {
|
||||
cluster.getService(1).getZKFCProxy(conf, 5000).gracefulFailover();
|
||||
fail("Did not fail to graceful failover when target failed " +
|
||||
"to become active!");
|
||||
} catch (ServiceFailedException sfe) {
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
"Couldn't make " + cluster.getService(1) + " active", sfe);
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
"injected failure", sfe);
|
||||
}
|
||||
|
||||
// No fencing
|
||||
assertEquals(0, cluster.getService(0).fenceCount);
|
||||
assertEquals(0, cluster.getService(1).fenceCount);
|
||||
|
||||
// Service 0 should go back to being active after the failed failover
|
||||
cluster.waitForActiveLockHolder(0);
|
||||
} finally {
|
||||
cluster.stop();
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=15000)
|
||||
public void testGracefulFailoverFailBecomingStandby() throws Exception {
|
||||
try {
|
||||
cluster.start();
|
||||
|
||||
cluster.waitForActiveLockHolder(0);
|
||||
|
||||
// Ask for failover when old node fails to transition to standby.
|
||||
// This should trigger fencing, since the cedeActive() command
|
||||
// still works, but leaves the breadcrumb in place.
|
||||
cluster.setFailToBecomeStandby(0, true);
|
||||
cluster.getService(1).getZKFCProxy(conf, 5000).gracefulFailover();
|
||||
|
||||
// Check that the old node was fenced
|
||||
assertEquals(1, cluster.getService(0).fenceCount);
|
||||
} finally {
|
||||
cluster.stop();
|
||||
fail("Did not fail to graceful failover to unhealthy service!");
|
||||
} catch (ServiceFailedException sfe) {
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
cluster.getService(1).toString() +
|
||||
" is not currently healthy.", sfe);
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=15000)
|
||||
public void testGracefulFailoverFailBecomingStandbyAndFailFence()
|
||||
throws Exception {
|
||||
|
||||
@Test
|
||||
public void testGracefulFailoverFailBecomingActive() throws Exception {
|
||||
cluster.start();
|
||||
|
||||
cluster.waitForActiveLockHolder(0);
|
||||
cluster.setFailToBecomeActive(1, true);
|
||||
|
||||
// Ask for failover, it should fail and report back to user.
|
||||
try {
|
||||
cluster.start();
|
||||
cluster.getService(1).getZKFCProxy(conf, 5000).gracefulFailover();
|
||||
fail("Did not fail to graceful failover when target failed " +
|
||||
"to become active!");
|
||||
} catch (ServiceFailedException sfe) {
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
"Couldn't make " + cluster.getService(1) + " active", sfe);
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
"injected failure", sfe);
|
||||
}
|
||||
|
||||
cluster.waitForActiveLockHolder(0);
|
||||
|
||||
// Ask for failover when old node fails to transition to standby.
|
||||
// This should trigger fencing, since the cedeActive() command
|
||||
// still works, but leaves the breadcrumb in place.
|
||||
cluster.setFailToBecomeStandby(0, true);
|
||||
cluster.setFailToFence(0, true);
|
||||
// No fencing
|
||||
assertEquals(0, cluster.getService(0).fenceCount);
|
||||
assertEquals(0, cluster.getService(1).fenceCount);
|
||||
|
||||
try {
|
||||
cluster.getService(1).getZKFCProxy(conf, 5000).gracefulFailover();
|
||||
fail("Failover should have failed when old node wont fence");
|
||||
} catch (ServiceFailedException sfe) {
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
"Unable to fence " + cluster.getService(0), sfe);
|
||||
}
|
||||
} finally {
|
||||
cluster.stop();
|
||||
// Service 0 should go back to being active after the failed failover
|
||||
cluster.waitForActiveLockHolder(0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGracefulFailoverFailBecomingStandby() throws Exception {
|
||||
cluster.start();
|
||||
|
||||
cluster.waitForActiveLockHolder(0);
|
||||
|
||||
// Ask for failover when old node fails to transition to standby.
|
||||
// This should trigger fencing, since the cedeActive() command
|
||||
// still works, but leaves the breadcrumb in place.
|
||||
cluster.setFailToBecomeStandby(0, true);
|
||||
cluster.getService(1).getZKFCProxy(conf, 5000).gracefulFailover();
|
||||
|
||||
// Check that the old node was fenced
|
||||
assertEquals(1, cluster.getService(0).fenceCount);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGracefulFailoverFailBecomingStandbyAndFailFence()
|
||||
throws Exception {
|
||||
cluster.start();
|
||||
|
||||
cluster.waitForActiveLockHolder(0);
|
||||
|
||||
// Ask for failover when old node fails to transition to standby.
|
||||
// This should trigger fencing, since the cedeActive() command
|
||||
// still works, but leaves the breadcrumb in place.
|
||||
cluster.setFailToBecomeStandby(0, true);
|
||||
cluster.setFailToFence(0, true);
|
||||
|
||||
try {
|
||||
cluster.getService(1).getZKFCProxy(conf, 5000).gracefulFailover();
|
||||
fail("Failover should have failed when old node wont fence");
|
||||
} catch (ServiceFailedException sfe) {
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
"Unable to fence " + cluster.getService(0), sfe);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -568,43 +539,39 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
|
|||
* Test which exercises all of the inputs into ZKFC. This is particularly
|
||||
* useful for running under jcarder to check for lock order violations.
|
||||
*/
|
||||
@Test(timeout=30000)
|
||||
@Test
|
||||
public void testOneOfEverything() throws Exception {
|
||||
try {
|
||||
cluster.start();
|
||||
|
||||
// Failover by session expiration
|
||||
LOG.info("====== Failing over by session expiration");
|
||||
cluster.expireAndVerifyFailover(0, 1);
|
||||
cluster.expireAndVerifyFailover(1, 0);
|
||||
|
||||
// Restart ZK
|
||||
LOG.info("====== Restarting server");
|
||||
stopServer();
|
||||
waitForServerDown(hostPort, CONNECTION_TIMEOUT);
|
||||
startServer();
|
||||
waitForServerUp(hostPort, CONNECTION_TIMEOUT);
|
||||
cluster.start();
|
||||
|
||||
// Failover by bad health
|
||||
cluster.setHealthy(0, false);
|
||||
cluster.waitForHAState(0, HAServiceState.INITIALIZING);
|
||||
cluster.waitForHAState(1, HAServiceState.ACTIVE);
|
||||
cluster.setHealthy(1, true);
|
||||
cluster.setHealthy(0, false);
|
||||
cluster.waitForHAState(1, HAServiceState.ACTIVE);
|
||||
cluster.waitForHAState(0, HAServiceState.INITIALIZING);
|
||||
cluster.setHealthy(0, true);
|
||||
|
||||
cluster.waitForHealthState(0, State.SERVICE_HEALTHY);
|
||||
|
||||
// Graceful failovers
|
||||
cluster.getZkfc(1).gracefulFailoverToYou();
|
||||
cluster.getZkfc(0).gracefulFailoverToYou();
|
||||
} finally {
|
||||
cluster.stop();
|
||||
}
|
||||
// Failover by session expiration
|
||||
LOG.info("====== Failing over by session expiration");
|
||||
cluster.expireAndVerifyFailover(0, 1);
|
||||
cluster.expireAndVerifyFailover(1, 0);
|
||||
|
||||
// Restart ZK
|
||||
LOG.info("====== Restarting server");
|
||||
stopServer();
|
||||
waitForServerDown(hostPort, CONNECTION_TIMEOUT);
|
||||
startServer();
|
||||
waitForServerUp(hostPort, CONNECTION_TIMEOUT);
|
||||
|
||||
// Failover by bad health
|
||||
cluster.setHealthy(0, false);
|
||||
cluster.waitForHAState(0, HAServiceState.INITIALIZING);
|
||||
cluster.waitForHAState(1, HAServiceState.ACTIVE);
|
||||
cluster.setHealthy(1, true);
|
||||
cluster.setHealthy(0, false);
|
||||
cluster.waitForHAState(1, HAServiceState.ACTIVE);
|
||||
cluster.waitForHAState(0, HAServiceState.INITIALIZING);
|
||||
cluster.setHealthy(0, true);
|
||||
|
||||
cluster.waitForHealthState(0, State.SERVICE_HEALTHY);
|
||||
|
||||
// Graceful failovers
|
||||
cluster.getZkfc(1).gracefulFailoverToYou();
|
||||
cluster.getZkfc(0).gracefulFailoverToYou();
|
||||
}
|
||||
|
||||
|
||||
private int runFC(DummyHAService target, String ... args) throws Exception {
|
||||
DummyZKFC zkfc = new DummyZKFC(conf, target);
|
||||
return zkfc.run(args);
|
||||
|
|
Loading…
Reference in New Issue