|
@@ -25,7 +25,9 @@ import java.io.FileInputStream;
|
|
|
import java.io.FileOutputStream;
|
|
|
import java.io.IOException;
|
|
|
import java.io.RandomAccessFile;
|
|
|
+import java.net.InetAddress;
|
|
|
import java.net.URI;
|
|
|
+import java.util.ArrayList;
|
|
|
import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
import java.util.Properties;
|
|
@@ -43,18 +45,21 @@ import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
|
|
|
|
+import org.apache.hadoop.hdfs.DFSClient;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
+import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
|
|
|
import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
|
|
|
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.io.MD5Hash;
|
|
|
-import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
+import org.junit.Assert;
|
|
|
|
|
|
/**
|
|
|
* Startup and checkpoint tests
|
|
@@ -491,4 +496,89 @@ public class TestStartup extends TestCase {
|
|
|
namenode.join();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * This test tests hosts include list contains host names. After namenode
|
|
|
+ * restarts, the still alive datanodes should not have any trouble in getting
|
|
|
+ * registrant again.
|
|
|
+ */
|
|
|
+ public void testNNRestart() throws IOException, InterruptedException {
|
|
|
+ MiniDFSCluster cluster = null;
|
|
|
+ FileSystem localFileSys;
|
|
|
+ Path hostsFile;
|
|
|
+ Path excludeFile;
|
|
|
+ Configuration conf = new HdfsConfiguration();
|
|
|
+ int HEARTBEAT_INTERVAL = 1; // heartbeat interval in seconds
|
|
|
+ // Set up the hosts/exclude files.
|
|
|
+ localFileSys = FileSystem.getLocal(conf);
|
|
|
+ Path workingDir = localFileSys.getWorkingDirectory();
|
|
|
+ Path dir = new Path(workingDir, "build/test/data/work-dir/restartnn");
|
|
|
+ hostsFile = new Path(dir, "hosts");
|
|
|
+ excludeFile = new Path(dir, "exclude");
|
|
|
+
|
|
|
+ // Setup conf
|
|
|
+ conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
|
|
|
+ writeConfigFile(localFileSys, excludeFile, null);
|
|
|
+ conf.set(DFSConfigKeys.DFS_HOSTS, hostsFile.toUri().getPath());
|
|
|
+ // write into hosts file
|
|
|
+ ArrayList<String>list = new ArrayList<String>();
|
|
|
+ byte b[] = {127, 0, 0, 1};
|
|
|
+ InetAddress inetAddress = InetAddress.getByAddress(b);
|
|
|
+ list.add(inetAddress.getHostName());
|
|
|
+ writeConfigFile(localFileSys, hostsFile, list);
|
|
|
+ int numNameNodes = 1;
|
|
|
+ int numDatanodes = 1;
|
|
|
+
|
|
|
+ try {
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).numNameNodes(numNameNodes)
|
|
|
+ .numDataNodes(numDatanodes).setupHostsFile(true).build();
|
|
|
+ cluster.waitActive();
|
|
|
+
|
|
|
+ cluster.restartNameNode();
|
|
|
+ NameNode nn = cluster.getNameNode();
|
|
|
+ assertNotNull(nn);
|
|
|
+ Assert.assertTrue(cluster.isDataNodeUp());
|
|
|
+
|
|
|
+ DatanodeInfo[] info = nn.getDatanodeReport(DatanodeReportType.LIVE);
|
|
|
+ for (int i = 0 ; i < 5 && info.length != numDatanodes; i++) {
|
|
|
+ Thread.sleep(HEARTBEAT_INTERVAL * 1000);
|
|
|
+ info = nn.getDatanodeReport(DatanodeReportType.LIVE);
|
|
|
+ }
|
|
|
+ assertEquals("Number of live nodes should be "+numDatanodes, numDatanodes,
|
|
|
+ info.length);
|
|
|
+
|
|
|
+ } catch (IOException e) {
|
|
|
+ fail(StringUtils.stringifyException(e));
|
|
|
+ throw e;
|
|
|
+ } finally {
|
|
|
+ cleanupFile(localFileSys, excludeFile.getParent());
|
|
|
+ if (cluster != null) {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void writeConfigFile(FileSystem localFileSys, Path name,
|
|
|
+ ArrayList<String> nodes) throws IOException {
|
|
|
+ // delete if it already exists
|
|
|
+ if (localFileSys.exists(name)) {
|
|
|
+ localFileSys.delete(name, true);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (nodes != null) {
|
|
|
+ FSDataOutputStream stm = localFileSys.create(name);
|
|
|
+ for (Iterator<String> it = nodes.iterator(); it.hasNext();) {
|
|
|
+ String node = it.next();
|
|
|
+ stm.writeBytes(node);
|
|
|
+ stm.writeBytes("\n");
|
|
|
+ }
|
|
|
+ stm.close();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
|
|
|
+ assertTrue(fileSys.exists(name));
|
|
|
+ fileSys.delete(name, true);
|
|
|
+ assertTrue(!fileSys.exists(name));
|
|
|
+ }
|
|
|
}
|