|
@@ -18,28 +18,32 @@
|
|
|
package org.apache.hadoop.hdfs;
|
|
|
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
|
|
|
-
|
|
|
-import static org.junit.Assert.*;
|
|
|
+import static org.junit.Assert.assertEquals;
|
|
|
+import static org.junit.Assert.assertTrue;
|
|
|
+import static org.junit.Assert.fail;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
-import java.io.OutputStream;
|
|
|
-import java.net.InetSocketAddress;
|
|
|
+import java.net.URI;
|
|
|
import java.net.URISyntaxException;
|
|
|
|
|
|
+import org.apache.commons.logging.Log;
|
|
|
+import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
+import org.apache.hadoop.util.StringUtils;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
public class TestDFSClientFailover {
|
|
|
|
|
|
+ private static final Log LOG = LogFactory.getLog(TestDFSClientFailover.class);
|
|
|
+
|
|
|
private static final Path TEST_FILE = new Path("/tmp/failover-test-file");
|
|
|
private static final int FILE_LENGTH_TO_VERIFY = 100;
|
|
|
|
|
@@ -49,8 +53,9 @@ public class TestDFSClientFailover {
|
|
|
@Before
|
|
|
public void setUpCluster() throws IOException {
|
|
|
cluster = new MiniDFSCluster.Builder(conf)
|
|
|
- .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(2))
|
|
|
+ .nnTopology(MiniDFSNNTopology.simpleHATopology())
|
|
|
.build();
|
|
|
+ cluster.transitionToActive(0);
|
|
|
cluster.waitActive();
|
|
|
}
|
|
|
|
|
@@ -58,34 +63,22 @@ public class TestDFSClientFailover {
|
|
|
public void tearDownCluster() throws IOException {
|
|
|
cluster.shutdown();
|
|
|
}
|
|
|
-
|
|
|
- // TODO(HA): This test should probably be made to fail if a client fails over
|
|
|
- // to talk to an NN with a different block pool id. Once failover between
|
|
|
- // active/standy in a single block pool is implemented, this test should be
|
|
|
- // changed to exercise that.
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Make sure that client failover works when an active NN dies and the standby
|
|
|
+ * takes over.
|
|
|
+ */
|
|
|
@Test
|
|
|
public void testDfsClientFailover() throws IOException, URISyntaxException {
|
|
|
- InetSocketAddress nnAddr1 = cluster.getNameNode(0).getNameNodeAddress();
|
|
|
- InetSocketAddress nnAddr2 = cluster.getNameNode(1).getNameNodeAddress();
|
|
|
-
|
|
|
- ClientProtocol nn1 = DFSUtil.createNamenode(nnAddr1, conf);
|
|
|
- ClientProtocol nn2 = DFSUtil.createNamenode(nnAddr2, conf);
|
|
|
-
|
|
|
- DFSClient dfsClient1 = new DFSClient(null, nn1, conf, null);
|
|
|
- DFSClient dfsClient2 = new DFSClient(null, nn2, conf, null);
|
|
|
-
|
|
|
- OutputStream out1 = dfsClient1.create(TEST_FILE.toString(), false);
|
|
|
- OutputStream out2 = dfsClient2.create(TEST_FILE.toString(), false);
|
|
|
- AppendTestUtil.write(out1, 0, FILE_LENGTH_TO_VERIFY);
|
|
|
- AppendTestUtil.write(out2, 0, FILE_LENGTH_TO_VERIFY);
|
|
|
- out1.close();
|
|
|
- out2.close();
|
|
|
-
|
|
|
FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
|
|
|
|
|
|
- AppendTestUtil.check(fs, TEST_FILE, FILE_LENGTH_TO_VERIFY);
|
|
|
- cluster.getNameNode(0).stop();
|
|
|
- AppendTestUtil.check(fs, TEST_FILE, FILE_LENGTH_TO_VERIFY);
|
|
|
+ DFSTestUtil.createFile(fs, TEST_FILE,
|
|
|
+ FILE_LENGTH_TO_VERIFY, (short)1, 1L);
|
|
|
+
|
|
|
+ assertEquals(fs.getFileStatus(TEST_FILE).getLen(), FILE_LENGTH_TO_VERIFY);
|
|
|
+ cluster.shutdownNameNode(0);
|
|
|
+ cluster.transitionToActive(1);
|
|
|
+ assertEquals(fs.getFileStatus(TEST_FILE).getLen(), FILE_LENGTH_TO_VERIFY);
|
|
|
|
|
|
// Check that it functions even if the URL becomes canonicalized
|
|
|
// to include a port number.
|
|
@@ -115,4 +108,28 @@ public class TestDFSClientFailover {
|
|
|
"does not use port information", ioe);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Make sure that a helpful error message is shown if a proxy provider is
|
|
|
+ * configured for a given URI, but no actual addresses are configured for that
|
|
|
+ * URI.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testFailureWithMisconfiguredHaNNs() throws Exception {
|
|
|
+ String logicalHost = "misconfigured-ha-uri";
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.set(DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "." + logicalHost,
|
|
|
+ ConfiguredFailoverProxyProvider.class.getName());
|
|
|
+
|
|
|
+ URI uri = new URI("hdfs://" + logicalHost + "/test");
|
|
|
+ try {
|
|
|
+ FileSystem.get(uri, conf).exists(new Path("/test"));
|
|
|
+ fail("Successfully got proxy provider for misconfigured FS");
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ LOG.info("got expected exception", ioe);
|
|
|
+ assertTrue("expected exception did not contain helpful message",
|
|
|
+ StringUtils.stringifyException(ioe).contains(
|
|
|
+ "Could not find any configured addresses for URI " + uri));
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|