|
@@ -31,6 +31,7 @@ import java.io.InputStream;
|
|
|
import java.io.OutputStream;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.net.SocketTimeoutException;
|
|
|
+import java.net.URI;
|
|
|
import java.security.MessageDigest;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
@@ -50,6 +51,7 @@ import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
+import org.apache.hadoop.hdfs.client.HdfsUtils;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
@@ -769,9 +771,11 @@ public class TestDFSClientRetries extends TestCase {
|
|
|
.build();
|
|
|
try {
|
|
|
cluster.waitActive();
|
|
|
+ final DistributedFileSystem dfs = cluster.getFileSystem();
|
|
|
+ final URI uri = dfs.getUri();
|
|
|
+ assertTrue(HdfsUtils.isHealthy(uri));
|
|
|
|
|
|
//create a file
|
|
|
- final DistributedFileSystem dfs = cluster.getFileSystem();
|
|
|
final long length = 1L << 20;
|
|
|
final Path file1 = new Path(dir, "foo");
|
|
|
DFSTestUtil.createFile(dfs, file1, length, numDatanodes, 20120406L);
|
|
@@ -781,7 +785,9 @@ public class TestDFSClientRetries extends TestCase {
|
|
|
assertEquals(length, s1.getLen());
|
|
|
|
|
|
//shutdown namenode
|
|
|
+ assertTrue(HdfsUtils.isHealthy(uri));
|
|
|
cluster.shutdownNameNode(0);
|
|
|
+ assertFalse(HdfsUtils.isHealthy(uri));
|
|
|
|
|
|
//namenode is down, create another file in a thread
|
|
|
final Path file3 = new Path(dir, "file");
|
|
@@ -806,8 +812,10 @@ public class TestDFSClientRetries extends TestCase {
|
|
|
try {
|
|
|
//sleep, restart, and then wait active
|
|
|
TimeUnit.SECONDS.sleep(30);
|
|
|
+ assertFalse(HdfsUtils.isHealthy(uri));
|
|
|
cluster.restartNameNode(0, false);
|
|
|
cluster.waitActive();
|
|
|
+ assertTrue(HdfsUtils.isHealthy(uri));
|
|
|
} catch (Exception e) {
|
|
|
exceptions.add(e);
|
|
|
}
|
|
@@ -823,7 +831,9 @@ public class TestDFSClientRetries extends TestCase {
|
|
|
assertEquals(dfs.getFileChecksum(file1), dfs.getFileChecksum(file3));
|
|
|
|
|
|
//enter safe mode
|
|
|
+ assertTrue(HdfsUtils.isHealthy(uri));
|
|
|
dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
|
|
+ assertFalse(HdfsUtils.isHealthy(uri));
|
|
|
|
|
|
//leave safe mode in a new thread
|
|
|
new Thread(new Runnable() {
|
|
@@ -832,7 +842,9 @@ public class TestDFSClientRetries extends TestCase {
|
|
|
try {
|
|
|
//sleep and then leave safe mode
|
|
|
TimeUnit.SECONDS.sleep(30);
|
|
|
+ assertFalse(HdfsUtils.isHealthy(uri));
|
|
|
dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
|
|
|
+ assertTrue(HdfsUtils.isHealthy(uri));
|
|
|
} catch (Exception e) {
|
|
|
exceptions.add(e);
|
|
|
}
|
|
@@ -844,6 +856,8 @@ public class TestDFSClientRetries extends TestCase {
|
|
|
DFSTestUtil.createFile(dfs, file2, length, numDatanodes, 20120406L);
|
|
|
assertEquals(dfs.getFileChecksum(file1), dfs.getFileChecksum(file2));
|
|
|
|
|
|
+ assertTrue(HdfsUtils.isHealthy(uri));
|
|
|
+
|
|
|
//make sure it won't retry on exceptions like FileNotFoundException
|
|
|
final Path nonExisting = new Path(dir, "nonExisting");
|
|
|
LOG.info("setPermission: " + nonExisting);
|