|
@@ -20,41 +20,30 @@ package org.apache.hadoop.ha;
|
|
|
import static org.junit.Assert.*;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
-import java.net.InetSocketAddress;
|
|
|
-import java.util.concurrent.CountDownLatch;
|
|
|
-import java.util.concurrent.TimeUnit;
|
|
|
+import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
|
-import org.apache.hadoop.ha.HAServiceProtocol;
|
|
|
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
|
|
|
-import org.apache.hadoop.ha.HealthCheckFailedException;
|
|
|
import org.apache.hadoop.ha.HealthMonitor.Callback;
|
|
|
import org.apache.hadoop.ha.HealthMonitor.State;
|
|
|
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
-import org.mockito.Mockito;
|
|
|
|
|
|
public class TestHealthMonitor {
|
|
|
private static final Log LOG = LogFactory.getLog(
|
|
|
TestHealthMonitor.class);
|
|
|
|
|
|
- /* bogus address to pass to constructor - never used */
|
|
|
- private static final InetSocketAddress BOGUS_ADDR =
|
|
|
- new InetSocketAddress(1);
|
|
|
-
|
|
|
- private HAServiceProtocol mockProxy;
|
|
|
-
|
|
|
/** How many times has createProxy been called */
|
|
|
- private volatile CountDownLatch createProxyLatch;
|
|
|
-
|
|
|
- /** Should throw an IOE when trying to connect */
|
|
|
- private volatile boolean shouldThrowOnCreateProxy = false;
|
|
|
+ private AtomicInteger createProxyCount = new AtomicInteger(0);
|
|
|
+ private volatile boolean throwOOMEOnCreate = false;
|
|
|
|
|
|
private HealthMonitor hm;
|
|
|
+
|
|
|
+ private DummyHAService svc;
|
|
|
|
|
|
@Before
|
|
|
public void setupHM() throws InterruptedException, IOException {
|
|
@@ -63,30 +52,21 @@ public class TestHealthMonitor {
|
|
|
conf.setInt(CommonConfigurationKeys.HA_HM_CHECK_INTERVAL_KEY, 50);
|
|
|
conf.setInt(CommonConfigurationKeys.HA_HM_CONNECT_RETRY_INTERVAL_KEY, 50);
|
|
|
conf.setInt(CommonConfigurationKeys.HA_HM_SLEEP_AFTER_DISCONNECT_KEY, 50);
|
|
|
- mockProxy = Mockito.mock(HAServiceProtocol.class);
|
|
|
- Mockito.doReturn(new HAServiceStatus(HAServiceState.ACTIVE))
|
|
|
- .when(mockProxy).getServiceStatus();
|
|
|
|
|
|
- hm = new HealthMonitor(conf, BOGUS_ADDR) {
|
|
|
+ svc = new DummyHAService(HAServiceState.ACTIVE, null);
|
|
|
+ hm = new HealthMonitor(conf, svc) {
|
|
|
@Override
|
|
|
protected HAServiceProtocol createProxy() throws IOException {
|
|
|
- createProxyLatch.countDown();
|
|
|
- if (shouldThrowOnCreateProxy) {
|
|
|
- throw new IOException("can't connect");
|
|
|
+ createProxyCount.incrementAndGet();
|
|
|
+ if (throwOOMEOnCreate) {
|
|
|
+ throw new OutOfMemoryError("oome");
|
|
|
}
|
|
|
- return mockProxy;
|
|
|
+ return super.createProxy();
|
|
|
}
|
|
|
};
|
|
|
-
|
|
|
- createProxyLatch = new CountDownLatch(1);
|
|
|
-
|
|
|
LOG.info("Starting health monitor");
|
|
|
hm.start();
|
|
|
|
|
|
- LOG.info("Waiting for proxy to be created");
|
|
|
- assertTrue(createProxyLatch.await(2000, TimeUnit.MILLISECONDS));
|
|
|
- createProxyLatch = null;
|
|
|
-
|
|
|
LOG.info("Waiting for HEALTHY signal");
|
|
|
waitForState(hm, HealthMonitor.State.SERVICE_HEALTHY);
|
|
|
}
|
|
@@ -94,27 +74,26 @@ public class TestHealthMonitor {
|
|
|
@Test(timeout=15000)
|
|
|
public void testMonitor() throws Exception {
|
|
|
LOG.info("Mocking bad health check, waiting for UNHEALTHY");
|
|
|
- Mockito.doThrow(new HealthCheckFailedException("Fake health check failure"))
|
|
|
- .when(mockProxy).monitorHealth();
|
|
|
+ svc.isHealthy = false;
|
|
|
waitForState(hm, HealthMonitor.State.SERVICE_UNHEALTHY);
|
|
|
|
|
|
LOG.info("Returning to healthy state, waiting for HEALTHY");
|
|
|
- Mockito.doNothing().when(mockProxy).monitorHealth();
|
|
|
+ svc.isHealthy = true;
|
|
|
waitForState(hm, HealthMonitor.State.SERVICE_HEALTHY);
|
|
|
|
|
|
LOG.info("Returning an IOException, as if node went down");
|
|
|
// should expect many rapid retries
|
|
|
- createProxyLatch = new CountDownLatch(3);
|
|
|
- shouldThrowOnCreateProxy = true;
|
|
|
- Mockito.doThrow(new IOException("Connection lost (fake)"))
|
|
|
- .when(mockProxy).monitorHealth();
|
|
|
+ int countBefore = createProxyCount.get();
|
|
|
+ svc.actUnreachable = true;
|
|
|
waitForState(hm, HealthMonitor.State.SERVICE_NOT_RESPONDING);
|
|
|
- assertTrue("Monitor should retry if createProxy throws an IOE",
|
|
|
- createProxyLatch.await(1000, TimeUnit.MILLISECONDS));
|
|
|
+
|
|
|
+ // Should retry several times
|
|
|
+ while (createProxyCount.get() < countBefore + 3) {
|
|
|
+ Thread.sleep(10);
|
|
|
+ }
|
|
|
|
|
|
LOG.info("Returning to healthy state, waiting for HEALTHY");
|
|
|
- shouldThrowOnCreateProxy = false;
|
|
|
- Mockito.doNothing().when(mockProxy).monitorHealth();
|
|
|
+ svc.actUnreachable = false;
|
|
|
waitForState(hm, HealthMonitor.State.SERVICE_HEALTHY);
|
|
|
|
|
|
hm.shutdown();
|
|
@@ -129,8 +108,8 @@ public class TestHealthMonitor {
|
|
|
@Test(timeout=15000)
|
|
|
public void testHealthMonitorDies() throws Exception {
|
|
|
LOG.info("Mocking RTE in health monitor, waiting for FAILED");
|
|
|
- Mockito.doThrow(new OutOfMemoryError())
|
|
|
- .when(mockProxy).monitorHealth();
|
|
|
+ throwOOMEOnCreate = true;
|
|
|
+ svc.actUnreachable = true;
|
|
|
waitForState(hm, HealthMonitor.State.HEALTH_MONITOR_FAILED);
|
|
|
hm.shutdown();
|
|
|
hm.join();
|
|
@@ -151,8 +130,7 @@ public class TestHealthMonitor {
|
|
|
}
|
|
|
});
|
|
|
LOG.info("Mocking bad health check, waiting for UNHEALTHY");
|
|
|
- Mockito.doThrow(new HealthCheckFailedException("Fake health check failure"))
|
|
|
- .when(mockProxy).monitorHealth();
|
|
|
+ svc.isHealthy = false;
|
|
|
waitForState(hm, HealthMonitor.State.HEALTH_MONITOR_FAILED);
|
|
|
}
|
|
|
|