|
@@ -17,6 +17,8 @@
|
|
*/
|
|
*/
|
|
package org.apache.hadoop.hdfs.server.namenode.ha;
|
|
package org.apache.hadoop.hdfs.server.namenode.ha;
|
|
|
|
|
|
|
|
+import org.apache.commons.lang3.StringUtils;
|
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.apache.hadoop.thirdparty.com.google.common.base.Joiner;
|
|
import org.apache.hadoop.thirdparty.com.google.common.base.Joiner;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.net.InetSocketAddress;
|
|
import java.net.InetSocketAddress;
|
|
@@ -24,7 +26,10 @@ import java.net.URI;
|
|
import java.util.HashMap;
|
|
import java.util.HashMap;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
|
|
+import java.util.concurrent.ExecutionException;
|
|
|
|
+import java.util.concurrent.Future;
|
|
import java.util.concurrent.TimeUnit;
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
+import java.util.concurrent.TimeoutException;
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.fs.permission.FsAction;
|
|
import org.apache.hadoop.fs.permission.FsAction;
|
|
@@ -36,20 +41,30 @@ import org.apache.hadoop.ipc.RemoteException;
|
|
import org.apache.hadoop.ipc.StandbyException;
|
|
import org.apache.hadoop.ipc.StandbyException;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.tools.GetUserMappingsProtocol;
|
|
import org.apache.hadoop.tools.GetUserMappingsProtocol;
|
|
|
|
+import org.apache.hadoop.util.StopWatch;
|
|
import org.junit.Before;
|
|
import org.junit.Before;
|
|
|
|
+import org.junit.BeforeClass;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
import org.mockito.Mockito;
|
|
import org.mockito.Mockito;
|
|
|
|
+import org.mockito.MockitoAnnotations;
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
import org.mockito.stubbing.Answer;
|
|
import org.mockito.stubbing.Answer;
|
|
|
|
+import org.slf4j.event.Level;
|
|
|
|
|
|
import static org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
|
|
import static org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
|
|
|
|
|
|
import static org.apache.hadoop.hdfs.server.namenode.ha.ObserverReadProxyProvider.*;
|
|
import static org.apache.hadoop.hdfs.server.namenode.ha.ObserverReadProxyProvider.*;
|
|
import static org.junit.Assert.assertArrayEquals;
|
|
import static org.junit.Assert.assertArrayEquals;
|
|
import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertEquals;
|
|
|
|
+import static org.junit.Assert.assertNull;
|
|
|
|
+import static org.junit.Assert.assertTrue;
|
|
import static org.junit.Assert.fail;
|
|
import static org.junit.Assert.fail;
|
|
|
|
+import static org.mockito.Matchers.any;
|
|
|
|
+import static org.mockito.Matchers.anyLong;
|
|
import static org.mockito.Mockito.mock;
|
|
import static org.mockito.Mockito.mock;
|
|
|
|
+import static org.mockito.Mockito.verifyNoMoreInteractions;
|
|
import static org.mockito.Mockito.when;
|
|
import static org.mockito.Mockito.when;
|
|
|
|
+import static org.mockito.Mockito.verify;
|
|
|
|
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -58,30 +73,42 @@ import static org.mockito.Mockito.when;
|
|
* NameNode to communicate with.
|
|
* NameNode to communicate with.
|
|
*/
|
|
*/
|
|
public class TestObserverReadProxyProvider {
|
|
public class TestObserverReadProxyProvider {
|
|
|
|
+ private final static long SLOW_RESPONSE_SLEEP_TIME = TimeUnit.SECONDS.toMillis(5); // 5 s
|
|
|
|
+ private final static long NAMENODE_HA_STATE_PROBE_TIMEOUT_SHORT = TimeUnit.SECONDS.toMillis(2);
|
|
|
|
+ private final static long NAMENODE_HA_STATE_PROBE_TIMEOUT_LONG = TimeUnit.SECONDS.toMillis(25);
|
|
|
|
+ private final GenericTestUtils.LogCapturer proxyLog =
|
|
|
|
+ GenericTestUtils.LogCapturer.captureLogs(ObserverReadProxyProvider.LOG);
|
|
|
|
|
|
private static final LocatedBlock[] EMPTY_BLOCKS = new LocatedBlock[0];
|
|
private static final LocatedBlock[] EMPTY_BLOCKS = new LocatedBlock[0];
|
|
private String ns;
|
|
private String ns;
|
|
private URI nnURI;
|
|
private URI nnURI;
|
|
- private Configuration conf;
|
|
|
|
|
|
|
|
private ObserverReadProxyProvider<ClientProtocol> proxyProvider;
|
|
private ObserverReadProxyProvider<ClientProtocol> proxyProvider;
|
|
private NameNodeAnswer[] namenodeAnswers;
|
|
private NameNodeAnswer[] namenodeAnswers;
|
|
private String[] namenodeAddrs;
|
|
private String[] namenodeAddrs;
|
|
|
|
|
|
|
|
+ @BeforeClass
|
|
|
|
+ public static void setLogLevel() {
|
|
|
|
+ GenericTestUtils.setLogLevel(ObserverReadProxyProvider.LOG, Level.DEBUG);
|
|
|
|
+ }
|
|
|
|
+
|
|
@Before
|
|
@Before
|
|
public void setup() throws Exception {
|
|
public void setup() throws Exception {
|
|
ns = "testcluster";
|
|
ns = "testcluster";
|
|
nnURI = URI.create("hdfs://" + ns);
|
|
nnURI = URI.create("hdfs://" + ns);
|
|
- conf = new Configuration();
|
|
|
|
- conf.set(HdfsClientConfigKeys.DFS_NAMESERVICES, ns);
|
|
|
|
- // Set observer probe retry period to 0. Required by the tests that
|
|
|
|
- // transition observer back and forth
|
|
|
|
- conf.setTimeDuration(
|
|
|
|
- OBSERVER_PROBE_RETRY_PERIOD_KEY, 0, TimeUnit.MILLISECONDS);
|
|
|
|
- conf.setBoolean(HdfsClientConfigKeys.Failover.RANDOM_ORDER, false);
|
|
|
|
}
|
|
}
|
|
|
|
|
|
private void setupProxyProvider(int namenodeCount) throws Exception {
|
|
private void setupProxyProvider(int namenodeCount) throws Exception {
|
|
|
|
+ setupProxyProvider(namenodeCount, new Configuration());
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void setupProxyProvider(int namenodeCount, long nnHAStateProbeTimeout) throws Exception {
|
|
|
|
+ Configuration conf = new Configuration();
|
|
|
|
+ conf.setLong(NAMENODE_HA_STATE_PROBE_TIMEOUT, nnHAStateProbeTimeout);
|
|
|
|
+ setupProxyProvider(namenodeCount, conf);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void setupProxyProvider(int namenodeCount, Configuration conf) throws Exception {
|
|
String[] namenodeIDs = new String[namenodeCount];
|
|
String[] namenodeIDs = new String[namenodeCount];
|
|
namenodeAddrs = new String[namenodeCount];
|
|
namenodeAddrs = new String[namenodeCount];
|
|
namenodeAnswers = new NameNodeAnswer[namenodeCount];
|
|
namenodeAnswers = new NameNodeAnswer[namenodeCount];
|
|
@@ -104,6 +131,12 @@ public class TestObserverReadProxyProvider {
|
|
}
|
|
}
|
|
conf.set(HdfsClientConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX + "." + ns,
|
|
conf.set(HdfsClientConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX + "." + ns,
|
|
Joiner.on(",").join(namenodeIDs));
|
|
Joiner.on(",").join(namenodeIDs));
|
|
|
|
+ conf.set(HdfsClientConfigKeys.DFS_NAMESERVICES, ns);
|
|
|
|
+ // Set observer probe retry period to 0. Required by the tests that
|
|
|
|
+ // transition observer back and forth
|
|
|
|
+ conf.setTimeDuration(
|
|
|
|
+ OBSERVER_PROBE_RETRY_PERIOD_KEY, 0, TimeUnit.MILLISECONDS);
|
|
|
|
+ conf.setBoolean(HdfsClientConfigKeys.Failover.RANDOM_ORDER, false);
|
|
proxyProvider = new ObserverReadProxyProvider<ClientProtocol>(conf, nnURI,
|
|
proxyProvider = new ObserverReadProxyProvider<ClientProtocol>(conf, nnURI,
|
|
ClientProtocol.class,
|
|
ClientProtocol.class,
|
|
new ClientHAProxyFactory<ClientProtocol>() {
|
|
new ClientHAProxyFactory<ClientProtocol>() {
|
|
@@ -145,7 +178,7 @@ public class TestObserverReadProxyProvider {
|
|
}
|
|
}
|
|
};
|
|
};
|
|
ObserverReadProxyProvider<GetUserMappingsProtocol> userProxyProvider =
|
|
ObserverReadProxyProvider<GetUserMappingsProtocol> userProxyProvider =
|
|
- new ObserverReadProxyProvider<>(conf, nnURI,
|
|
|
|
|
|
+ new ObserverReadProxyProvider<>(proxyProvider.conf, nnURI,
|
|
GetUserMappingsProtocol.class, proxyFactory);
|
|
GetUserMappingsProtocol.class, proxyFactory);
|
|
assertArrayEquals(fakeGroups,
|
|
assertArrayEquals(fakeGroups,
|
|
userProxyProvider.getProxy().proxy.getGroupsForUser(fakeUser));
|
|
userProxyProvider.getProxy().proxy.getGroupsForUser(fakeUser));
|
|
@@ -325,6 +358,160 @@ public class TestObserverReadProxyProvider {
|
|
assertHandledBy(1);
|
|
assertHandledBy(1);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Happy case for GetHAServiceStateWithTimeout.
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testGetHAServiceStateWithTimeout() throws Exception {
|
|
|
|
+ proxyLog.clearOutput();
|
|
|
|
+
|
|
|
|
+ setupProxyProvider(1, NAMENODE_HA_STATE_PROBE_TIMEOUT_SHORT);
|
|
|
|
+ final HAServiceState state = HAServiceState.STANDBY;
|
|
|
|
+ NNProxyInfo<ClientProtocol> dummyNNProxyInfo =
|
|
|
|
+ (NNProxyInfo<ClientProtocol>) mock(NNProxyInfo.class);
|
|
|
|
+ Future<HAServiceState> task = mock(Future.class);
|
|
|
|
+ when(task.get(anyLong(), any(TimeUnit.class))).thenReturn(state);
|
|
|
|
+
|
|
|
|
+ HAServiceState state2 = proxyProvider.getHAServiceStateWithTimeout(dummyNNProxyInfo, task);
|
|
|
|
+ assertEquals(state, state2);
|
|
|
|
+ verify(task).get(anyLong(), any(TimeUnit.class));
|
|
|
|
+ verifyNoMoreInteractions(task);
|
|
|
|
+ assertEquals(1, StringUtils.countMatches(proxyLog.getOutput(),
|
|
|
|
+ "HA State for " + dummyNNProxyInfo.proxyInfo + " is " + state));
|
|
|
|
+ proxyLog.clearOutput();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Test TimeoutException for GetHAServiceStateWithTimeout.
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testTimeoutExceptionGetHAServiceStateWithTimeout() throws Exception {
|
|
|
|
+ proxyLog.clearOutput();
|
|
|
|
+
|
|
|
|
+ setupProxyProvider(1, NAMENODE_HA_STATE_PROBE_TIMEOUT_SHORT);
|
|
|
|
+ NNProxyInfo<ClientProtocol> dummyNNProxyInfo =
|
|
|
|
+ (NNProxyInfo<ClientProtocol>) Mockito.mock(NNProxyInfo.class);
|
|
|
|
+ Future<HAServiceState> task = mock(Future.class);
|
|
|
|
+ TimeoutException e = new TimeoutException("Timeout");
|
|
|
|
+ when(task.get(anyLong(), any(TimeUnit.class))).thenThrow(e);
|
|
|
|
+
|
|
|
|
+ HAServiceState state = proxyProvider.getHAServiceStateWithTimeout(dummyNNProxyInfo, task);
|
|
|
|
+ assertNull(state);
|
|
|
|
+ verify(task).get(anyLong(), any(TimeUnit.class));
|
|
|
|
+ verify(task).cancel(true);
|
|
|
|
+ verifyNoMoreInteractions(task);
|
|
|
|
+ assertEquals(1, StringUtils.countMatches(proxyLog.getOutput(),
|
|
|
|
+ "Cancel NN probe task due to timeout for " + dummyNNProxyInfo.proxyInfo));
|
|
|
|
+ proxyLog.clearOutput();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Test InterruptedException for GetHAServiceStateWithTimeout.
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testInterruptedExceptionGetHAServiceStateWithTimeout() throws Exception {
|
|
|
|
+ proxyLog.clearOutput();
|
|
|
|
+
|
|
|
|
+ setupProxyProvider(1, NAMENODE_HA_STATE_PROBE_TIMEOUT_SHORT);
|
|
|
|
+ NNProxyInfo<ClientProtocol> dummyNNProxyInfo =
|
|
|
|
+ (NNProxyInfo<ClientProtocol>) Mockito.mock(NNProxyInfo.class);
|
|
|
|
+ Future<HAServiceState> task = mock(Future.class);
|
|
|
|
+ InterruptedException e = new InterruptedException("Interrupted");
|
|
|
|
+ when(task.get(anyLong(), any(TimeUnit.class))).thenThrow(e);
|
|
|
|
+
|
|
|
|
+ HAServiceState state = proxyProvider.getHAServiceStateWithTimeout(dummyNNProxyInfo, task);
|
|
|
|
+ assertNull(state);
|
|
|
|
+ verify(task).get(anyLong(), any(TimeUnit.class));
|
|
|
|
+ verifyNoMoreInteractions(task);
|
|
|
|
+ assertEquals(1, StringUtils.countMatches(proxyLog.getOutput(),
|
|
|
|
+ "Exception in NN probe task for " + dummyNNProxyInfo.proxyInfo));
|
|
|
|
+ proxyLog.clearOutput();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Test ExecutionException for GetHAServiceStateWithTimeout.
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testExecutionExceptionGetHAServiceStateWithTimeout() throws Exception {
|
|
|
|
+ proxyLog.clearOutput();
|
|
|
|
+
|
|
|
|
+ setupProxyProvider(1, NAMENODE_HA_STATE_PROBE_TIMEOUT_SHORT);
|
|
|
|
+ NNProxyInfo<ClientProtocol> dummyNNProxyInfo =
|
|
|
|
+ (NNProxyInfo<ClientProtocol>) Mockito.mock(NNProxyInfo.class);
|
|
|
|
+ Future<HAServiceState> task = mock(Future.class);
|
|
|
|
+ Exception e = new ExecutionException(new InterruptedException("Interrupted"));
|
|
|
|
+ when(task.get(anyLong(), any(TimeUnit.class))).thenThrow(e);
|
|
|
|
+
|
|
|
|
+ HAServiceState state = proxyProvider.getHAServiceStateWithTimeout(dummyNNProxyInfo, task);
|
|
|
|
+ assertNull(state);
|
|
|
|
+ verify(task).get(anyLong(), any(TimeUnit.class));
|
|
|
|
+ verifyNoMoreInteractions(task);
|
|
|
|
+ assertEquals(1, StringUtils.countMatches(proxyLog.getOutput(),
|
|
|
|
+ "Exception in NN probe task for " + dummyNNProxyInfo.proxyInfo));
|
|
|
|
+ proxyLog.clearOutput();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Test GetHAServiceState when timeout is disabled (test the else { task.get() } code path)
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testGetHAServiceStateWithoutTimeout() throws Exception {
|
|
|
|
+ proxyLog.clearOutput();
|
|
|
|
+ setupProxyProvider(1, 0);
|
|
|
|
+
|
|
|
|
+ final HAServiceState state = HAServiceState.STANDBY;
|
|
|
|
+ NNProxyInfo<ClientProtocol> dummyNNProxyInfo =
|
|
|
|
+ (NNProxyInfo<ClientProtocol>) mock(NNProxyInfo.class);
|
|
|
|
+ Future<HAServiceState> task = mock(Future.class);
|
|
|
|
+ when(task.get()).thenReturn(state);
|
|
|
|
+
|
|
|
|
+ HAServiceState state2 = proxyProvider.getHAServiceStateWithTimeout(dummyNNProxyInfo, task);
|
|
|
|
+ assertEquals(state, state2);
|
|
|
|
+ verify(task).get();
|
|
|
|
+ verifyNoMoreInteractions(task);
|
|
|
|
+ assertEquals(1, StringUtils.countMatches(proxyLog.getOutput(),
|
|
|
|
+ "HA State for " + dummyNNProxyInfo.proxyInfo + " is " + state));
|
|
|
|
+ proxyLog.clearOutput();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Test getHAServiceState when we have a slow NN, using a 25s timeout.
|
|
|
|
+ * This is to verify the old behavior without being able to fast-fail (we can also set
|
|
|
|
+ * namenodeHAStateProbeTimeoutMs to 0 or a negative value and the rest of the test can stay
|
|
|
|
+ * the same).
|
|
|
|
+ *
|
|
|
|
+ * 5-second (SLOW_RESPONSE_SLEEP_TIME) latency is introduced and we expect that latency is added
|
|
|
|
+ * to the READ operation.
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testStandbyGetHAServiceStateLongTimeout() throws Exception {
|
|
|
|
+ setupProxyProvider(4, NAMENODE_HA_STATE_PROBE_TIMEOUT_LONG);
|
|
|
|
+ namenodeAnswers[0].setActiveState();
|
|
|
|
+ namenodeAnswers[1].setSlowNode(true);
|
|
|
|
+ namenodeAnswers[3].setObserverState();
|
|
|
|
+
|
|
|
|
+ StopWatch watch = new StopWatch();
|
|
|
|
+ watch.start();
|
|
|
|
+ doRead();
|
|
|
|
+ long runtime = watch.now(TimeUnit.MILLISECONDS);
|
|
|
|
+ assertTrue("Read operation finished earlier than we expected",
|
|
|
|
+ runtime > SLOW_RESPONSE_SLEEP_TIME);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Test getHAServiceState using a 2s timeout with a slow standby.
|
|
|
|
+ * Fail the test if we don't complete it in 4s.
|
|
|
|
+ */
|
|
|
|
+ @Test(timeout = 4000)
|
|
|
|
+ public void testStandbyGetHAServiceStateTimeout() throws Exception {
|
|
|
|
+ setupProxyProvider(4, NAMENODE_HA_STATE_PROBE_TIMEOUT_SHORT);
|
|
|
|
+ namenodeAnswers[0].setActiveState();
|
|
|
|
+ namenodeAnswers[1].setSlowNode(true);
|
|
|
|
+ namenodeAnswers[3].setObserverState();
|
|
|
|
+
|
|
|
|
+ doRead();
|
|
|
|
+ }
|
|
|
|
+
|
|
private void doRead() throws Exception {
|
|
private void doRead() throws Exception {
|
|
doRead(proxyProvider.getProxy().proxy);
|
|
doRead(proxyProvider.getProxy().proxy);
|
|
}
|
|
}
|
|
@@ -357,6 +544,7 @@ public class TestObserverReadProxyProvider {
|
|
|
|
|
|
private volatile boolean unreachable = false;
|
|
private volatile boolean unreachable = false;
|
|
private volatile boolean retryActive = false;
|
|
private volatile boolean retryActive = false;
|
|
|
|
+ private volatile boolean slowNode = false;
|
|
|
|
|
|
// Standby state by default
|
|
// Standby state by default
|
|
private volatile boolean allowWrites = false;
|
|
private volatile boolean allowWrites = false;
|
|
@@ -370,6 +558,12 @@ public class TestObserverReadProxyProvider {
|
|
if (unreachable) {
|
|
if (unreachable) {
|
|
throw new IOException("Unavailable");
|
|
throw new IOException("Unavailable");
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ // sleep to simulate slow rpc responses.
|
|
|
|
+ if (slowNode) {
|
|
|
|
+ Thread.sleep(SLOW_RESPONSE_SLEEP_TIME);
|
|
|
|
+ }
|
|
|
|
+
|
|
// retryActive should be checked before getHAServiceState.
|
|
// retryActive should be checked before getHAServiceState.
|
|
// Check getHAServiceState first here only because in test,
|
|
// Check getHAServiceState first here only because in test,
|
|
// it relies read call, which relies on getHAServiceState
|
|
// it relies read call, which relies on getHAServiceState
|
|
@@ -416,6 +610,11 @@ public class TestObserverReadProxyProvider {
|
|
this.unreachable = unreachable;
|
|
this.unreachable = unreachable;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ // Whether this node should be slow in rpc response.
|
|
|
|
+ void setSlowNode(boolean slowNode) {
|
|
|
|
+ this.slowNode = slowNode;
|
|
|
|
+ }
|
|
|
|
+
|
|
void setActiveState() {
|
|
void setActiveState() {
|
|
allowReads = true;
|
|
allowReads = true;
|
|
allowWrites = true;
|
|
allowWrites = true;
|