|
@@ -20,17 +20,21 @@ package org.apache.hadoop.hdfs.server.namenode.ha;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
|
+import org.apache.hadoop.net.MockDomainNameResolver;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.BeforeClass;
|
|
|
+import org.junit.Rule;
|
|
|
import org.junit.Test;
|
|
|
+import org.junit.rules.ExpectedException;
|
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
|
import org.mockito.stubbing.Answer;
|
|
|
import org.slf4j.event.Level;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.net.InetAddress;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.net.URI;
|
|
|
import java.net.URISyntaxException;
|
|
@@ -40,6 +44,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
+import static org.junit.Assert.assertNull;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.when;
|
|
@@ -54,6 +59,7 @@ public class TestConfiguredFailoverProxyProvider {
|
|
|
private int rpcPort = 8020;
|
|
|
private URI ns1Uri;
|
|
|
private URI ns2Uri;
|
|
|
+ private URI ns3Uri;
|
|
|
private String ns1;
|
|
|
private String ns1nn1Hostname = "machine1.foo.bar";
|
|
|
private InetSocketAddress ns1nn1 =
|
|
@@ -71,8 +77,12 @@ public class TestConfiguredFailoverProxyProvider {
|
|
|
private String ns2nn3Hostname = "router3.foo.bar";
|
|
|
private InetSocketAddress ns2nn3 =
|
|
|
new InetSocketAddress(ns2nn3Hostname, rpcPort);
|
|
|
+ private String ns3;
|
|
|
private static final int NUM_ITERATIONS = 50;
|
|
|
|
|
|
+ @Rule
|
|
|
+ public final ExpectedException exception = ExpectedException.none();
|
|
|
+
|
|
|
@BeforeClass
|
|
|
public static void setupClass() throws Exception {
|
|
|
GenericTestUtils.setLogLevel(RequestHedgingProxyProvider.LOG, Level.TRACE);
|
|
@@ -120,10 +130,41 @@ public class TestConfiguredFailoverProxyProvider {
|
|
|
HdfsClientConfigKeys.Failover.RANDOM_ORDER + "." + ns2,
|
|
|
true);
|
|
|
|
|
|
- conf.set(HdfsClientConfigKeys.DFS_NAMESERVICES, ns1 + "," + ns2);
|
|
|
+ ns3 = "mycluster-3-" + Time.monotonicNow();
|
|
|
+ ns3Uri = new URI("hdfs://" + ns3);
|
|
|
+
|
|
|
+ conf.set(HdfsClientConfigKeys.DFS_NAMESERVICES,
|
|
|
+ String.join(",", ns1, ns2, ns3));
|
|
|
conf.set("fs.defaultFS", "hdfs://" + ns1);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Add more DNS related settings to the passed in configuration.
|
|
|
+ * @param config Configuration file to add settings to.
|
|
|
+ */
|
|
|
+ private void addDNSSettings(Configuration config, boolean hostResolvable) {
|
|
|
+ config.set(
|
|
|
+ HdfsClientConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX + "." + ns3, "nn");
|
|
|
+ String domain = hostResolvable
|
|
|
+ ? MockDomainNameResolver.DOMAIN
|
|
|
+ : MockDomainNameResolver.UNKNOW_DOMAIN;
|
|
|
+ config.set(
|
|
|
+ HdfsClientConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY + "." + ns3 + ".nn",
|
|
|
+ domain + ":" + rpcPort);
|
|
|
+ config.set(
|
|
|
+ HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + "." + ns3,
|
|
|
+ ConfiguredFailoverProxyProvider.class.getName());
|
|
|
+ config.setBoolean(
|
|
|
+ HdfsClientConfigKeys.Failover.RESOLVE_ADDRESS_NEEDED_KEY + "." + ns3,
|
|
|
+ true);
|
|
|
+ config.set(
|
|
|
+ HdfsClientConfigKeys.Failover.RESOLVE_SERVICE_KEY + "." + ns3,
|
|
|
+ MockDomainNameResolver.class.getName());
|
|
|
+ config.setBoolean(
|
|
|
+ HdfsClientConfigKeys.Failover.RANDOM_ORDER + "." + ns3,
|
|
|
+ true);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Tests getProxy with random.order configuration set to false.
|
|
|
* This expects the proxy order to be consistent every time a new
|
|
@@ -209,6 +250,98 @@ public class TestConfiguredFailoverProxyProvider {
|
|
|
nn1Count.get() + nn2Count.get() + nn3Count.get());
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testResolveDomainNameUsingDNS() throws Exception {
|
|
|
+ Configuration dnsConf = new Configuration(conf);
|
|
|
+ addDNSSettings(dnsConf, true);
|
|
|
+
|
|
|
+ // Mock ClientProtocol
|
|
|
+ Map<InetSocketAddress, ClientProtocol> proxyMap = new HashMap<>();
|
|
|
+ final AtomicInteger nn1Count = addClientMock(
|
|
|
+ MockDomainNameResolver.BYTE_ADDR_1, proxyMap);
|
|
|
+ final AtomicInteger nn2Count = addClientMock(
|
|
|
+ MockDomainNameResolver.BYTE_ADDR_2, proxyMap);
|
|
|
+
|
|
|
+ // Get a client multiple times
|
|
|
+ final Map<String, AtomicInteger> proxyResults = new HashMap<>();
|
|
|
+ for (int i = 0; i < NUM_ITERATIONS; i++) {
|
|
|
+ @SuppressWarnings("resource")
|
|
|
+ ConfiguredFailoverProxyProvider<ClientProtocol> provider =
|
|
|
+ new ConfiguredFailoverProxyProvider<>(
|
|
|
+ dnsConf, ns3Uri, ClientProtocol.class, createFactory(proxyMap));
|
|
|
+ ClientProtocol proxy = provider.getProxy().proxy;
|
|
|
+ String proxyAddress = provider.getProxy().proxyInfo;
|
|
|
+
|
|
|
+ if (proxyResults.containsKey(proxyAddress)) {
|
|
|
+ proxyResults.get(proxyAddress).incrementAndGet();
|
|
|
+ } else {
|
|
|
+ proxyResults.put(proxyAddress, new AtomicInteger(1));
|
|
|
+ }
|
|
|
+ proxy.getStats();
|
|
|
+ }
|
|
|
+
|
|
|
+ // Check we got the proper addresses
|
|
|
+ assertEquals(2, proxyResults.size());
|
|
|
+ assertTrue(
|
|
|
+ "nn1 wasn't returned: " + proxyResults,
|
|
|
+ proxyResults.containsKey(
|
|
|
+ "/" + MockDomainNameResolver.ADDR_1 + ":8020"));
|
|
|
+ assertTrue(
|
|
|
+ "nn2 wasn't returned: " + proxyResults,
|
|
|
+ proxyResults.containsKey(
|
|
|
+ "/" + MockDomainNameResolver.ADDR_2 + ":8020"));
|
|
|
+
|
|
|
+ // Check that the Namenodes were invoked
|
|
|
+ assertEquals(NUM_ITERATIONS, nn1Count.get() + nn2Count.get());
|
|
|
+ assertTrue("nn1 was selected too much:" + nn1Count.get(),
|
|
|
+ nn1Count.get() < NUM_ITERATIONS);
|
|
|
+ assertTrue("nn1 should have been selected: " + nn1Count.get(),
|
|
|
+ nn1Count.get() > 0);
|
|
|
+ assertTrue("nn2 was selected too much:" + nn2Count.get(),
|
|
|
+ nn2Count.get() < NUM_ITERATIONS);
|
|
|
+ assertTrue(
|
|
|
+ "nn2 should have been selected: " + nn2Count.get(),
|
|
|
+ nn2Count.get() > 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testResolveDomainNameUsingDNSUnknownHost() throws Exception {
|
|
|
+ Configuration dnsConf = new Configuration(conf);
|
|
|
+ addDNSSettings(dnsConf, false);
|
|
|
+
|
|
|
+ Map<InetSocketAddress, ClientProtocol> proxyMap = new HashMap<>();
|
|
|
+ exception.expect(RuntimeException.class);
|
|
|
+ ConfiguredFailoverProxyProvider<ClientProtocol> provider =
|
|
|
+ new ConfiguredFailoverProxyProvider<>(
|
|
|
+ dnsConf, ns3Uri, ClientProtocol.class, createFactory(proxyMap));
|
|
|
+
|
|
|
+ assertNull("failover proxy cannot be created due to unknownhost",
|
|
|
+ provider);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Add a ClientProtocol mock for the proxy.
|
|
|
+ * @param addr IP address for the destination.
|
|
|
+ * @param proxyMap Map containing the client for each target address.
|
|
|
+ * @return The counter for the number of calls to this target.
|
|
|
+ * @throws Exception If the client cannot be created.
|
|
|
+ */
|
|
|
+ private AtomicInteger addClientMock(
|
|
|
+ byte[] addr, Map<InetSocketAddress, ClientProtocol> proxyMap)
|
|
|
+ throws Exception {
|
|
|
+
|
|
|
+ final AtomicInteger counter = new AtomicInteger(0);
|
|
|
+ InetAddress inetAddr = InetAddress.getByAddress(addr);
|
|
|
+ InetSocketAddress inetSockerAddr =
|
|
|
+ new InetSocketAddress(inetAddr, rpcPort);
|
|
|
+
|
|
|
+ final ClientProtocol cpMock = mock(ClientProtocol.class);
|
|
|
+ when(cpMock.getStats()).thenAnswer(createAnswer(counter, 1));
|
|
|
+ proxyMap.put(inetSockerAddr, cpMock);
|
|
|
+
|
|
|
+ return counter;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* createAnswer creates an Answer for using with the ClientProtocol mocks.
|
|
|
* @param counter counter to increment
|