HDFS-3446. HostsFileReader silently ignores bad includes/excludes. Contributed by Matthew Jacobs.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1355583 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
30c8f2bc9c
commit
535fd7de15
|
@ -50,9 +50,6 @@ public class HostsFileReader {
|
|||
|
||||
private void readFileToSet(String filename, Set<String> set) throws IOException {
|
||||
File file = new File(filename);
|
||||
if (!file.exists()) {
|
||||
return;
|
||||
}
|
||||
FileInputStream fis = new FileInputStream(file);
|
||||
BufferedReader reader = null;
|
||||
try {
|
||||
|
|
|
@ -18,8 +18,8 @@
|
|||
package org.apache.hadoop.util;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.FileWriter;
|
||||
import java.util.Set;
|
||||
|
||||
import org.junit.*;
|
||||
import static org.junit.Assert.*;
|
||||
|
@ -37,8 +37,6 @@ public class TestHostsFileReader {
|
|||
File INCLUDES_FILE = new File(HOSTS_TEST_DIR, "dfs.include");
|
||||
String excludesFile = HOSTS_TEST_DIR + "/dfs.exclude";
|
||||
String includesFile = HOSTS_TEST_DIR + "/dfs.include";
|
||||
private Set<String> includes;
|
||||
private Set<String> excludes;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
|
@ -100,6 +98,43 @@ public class TestHostsFileReader {
|
|||
|
||||
}
|
||||
|
||||
/*
|
||||
* Test creating a new HostsFileReader with nonexistent files
|
||||
*/
|
||||
@Test
|
||||
public void testCreateHostFileReaderWithNonexistentFile() throws Exception {
|
||||
try {
|
||||
new HostsFileReader(
|
||||
HOSTS_TEST_DIR + "/doesnt-exist",
|
||||
HOSTS_TEST_DIR + "/doesnt-exist");
|
||||
Assert.fail("Should throw FileNotFoundException");
|
||||
} catch (FileNotFoundException ex) {
|
||||
// Exception as expected
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Test refreshing an existing HostsFileReader with an includes file that no longer exists
|
||||
*/
|
||||
@Test
|
||||
public void testRefreshHostFileReaderWithNonexistentFile() throws Exception {
|
||||
FileWriter efw = new FileWriter(excludesFile);
|
||||
FileWriter ifw = new FileWriter(includesFile);
|
||||
|
||||
efw.close();
|
||||
|
||||
ifw.close();
|
||||
|
||||
HostsFileReader hfp = new HostsFileReader(includesFile, excludesFile);
|
||||
assertTrue(INCLUDES_FILE.delete());
|
||||
try {
|
||||
hfp.refresh();
|
||||
Assert.fail("Should throw FileNotFoundException");
|
||||
} catch (FileNotFoundException ex) {
|
||||
// Exception as expected
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Test for null file
|
||||
*/
|
||||
|
|
|
@ -4,6 +4,9 @@ Release 2.0.1-alpha - UNRELEASED
|
|||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
||||
HDFS-3446. HostsFileReader silently ignores bad includes/excludes
|
||||
(Matthew Jacobs via todd)
|
||||
|
||||
NEW FEATURES
|
||||
|
||||
HDFS-744. Support hsync in HDFS. (Lars Hofhansl via szetszwo)
|
||||
|
|
|
@ -78,6 +78,7 @@ public class TestDecommission {
|
|||
|
||||
// Setup conf
|
||||
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false);
|
||||
conf.set(DFSConfigKeys.DFS_HOSTS, hostsFile.toUri().getPath());
|
||||
conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
|
||||
conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 2000);
|
||||
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, HEARTBEAT_INTERVAL);
|
||||
|
@ -85,6 +86,7 @@ public class TestDecommission {
|
|||
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, 4);
|
||||
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, NAMENODE_REPLICATION_INTERVAL);
|
||||
|
||||
writeConfigFile(hostsFile, null);
|
||||
writeConfigFile(excludeFile, null);
|
||||
}
|
||||
|
||||
|
@ -508,7 +510,6 @@ public class TestDecommission {
|
|||
|
||||
public void testHostsFile(int numNameNodes) throws IOException,
|
||||
InterruptedException {
|
||||
conf.set(DFSConfigKeys.DFS_HOSTS, hostsFile.toUri().getPath());
|
||||
int numDatanodes = 1;
|
||||
cluster = new MiniDFSCluster.Builder(conf)
|
||||
.nnTopology(MiniDFSNNTopology.simpleFederatedTopology(numNameNodes))
|
||||
|
|
|
@ -389,9 +389,12 @@ public class TestBlocksWithNotEnoughRacks {
|
|||
Path workingDir = localFileSys.getWorkingDirectory();
|
||||
Path dir = new Path(workingDir, "build/test/data/temp/decommission");
|
||||
Path excludeFile = new Path(dir, "exclude");
|
||||
Path includeFile = new Path(dir, "include");
|
||||
assertTrue(localFileSys.mkdirs(dir));
|
||||
DFSTestUtil.writeFile(localFileSys, excludeFile, "");
|
||||
DFSTestUtil.writeFile(localFileSys, includeFile, "");
|
||||
conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
|
||||
conf.set(DFSConfigKeys.DFS_HOSTS, includeFile.toUri().getPath());
|
||||
|
||||
// Two blocks and four racks
|
||||
String racks[] = {"/rack1", "/rack1", "/rack2", "/rack2"};
|
||||
|
@ -439,8 +442,11 @@ public class TestBlocksWithNotEnoughRacks {
|
|||
Path workingDir = localFileSys.getWorkingDirectory();
|
||||
Path dir = new Path(workingDir, "build/test/data/temp/decommission");
|
||||
Path excludeFile = new Path(dir, "exclude");
|
||||
Path includeFile = new Path(dir, "include");
|
||||
assertTrue(localFileSys.mkdirs(dir));
|
||||
DFSTestUtil.writeFile(localFileSys, excludeFile, "");
|
||||
DFSTestUtil.writeFile(localFileSys, includeFile, "");
|
||||
conf.set(DFSConfigKeys.DFS_HOSTS, includeFile.toUri().getPath());
|
||||
conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
|
||||
|
||||
// All hosts are on two racks, only one host on /rack2
|
||||
|
|
|
@ -119,11 +119,15 @@ public class NNThroughputBenchmark {
|
|||
"${hadoop.tmp.dir}/dfs/hosts/exclude");
|
||||
File excludeFile = new File(config.get(DFSConfigKeys.DFS_HOSTS_EXCLUDE,
|
||||
"exclude"));
|
||||
if(! excludeFile.exists()) {
|
||||
if(!excludeFile.exists()) {
|
||||
if(!excludeFile.getParentFile().mkdirs())
|
||||
throw new IOException("NNThroughputBenchmark: cannot mkdir " + excludeFile);
|
||||
}
|
||||
new FileOutputStream(excludeFile).close();
|
||||
// set include file
|
||||
config.set(DFSConfigKeys.DFS_HOSTS, "${hadoop.tmp.dir}/dfs/hosts/include");
|
||||
File includeFile = new File(config.get(DFSConfigKeys.DFS_HOSTS, "include"));
|
||||
new FileOutputStream(includeFile).close();
|
||||
// Start the NameNode
|
||||
String[] argv = new String[] {};
|
||||
nameNode = NameNode.createNameNode(argv, config);
|
||||
|
|
|
@ -74,14 +74,16 @@ public class TestDecommissioningStatus {
|
|||
assertTrue(localFileSys.mkdirs(dir));
|
||||
excludeFile = new Path(dir, "exclude");
|
||||
conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
|
||||
conf
|
||||
.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 2000);
|
||||
Path includeFile = new Path(dir, "include");
|
||||
conf.set(DFSConfigKeys.DFS_HOSTS, includeFile.toUri().getPath());
|
||||
conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 2000);
|
||||
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
|
||||
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY,
|
||||
4);
|
||||
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1000);
|
||||
conf.setInt(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 1);
|
||||
writeConfigFile(localFileSys, excludeFile, null);
|
||||
writeConfigFile(localFileSys, includeFile, null);
|
||||
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes).build();
|
||||
cluster.waitActive();
|
||||
|
|
|
@ -507,7 +507,7 @@ public class TestStartup extends TestCase {
|
|||
|
||||
// Setup conf
|
||||
conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
|
||||
writeConfigFile(localFileSys, excludeFile, null);
|
||||
writeConfigFile(localFileSys, excludeFile, null);
|
||||
conf.set(DFSConfigKeys.DFS_HOSTS, hostsFile.toUri().getPath());
|
||||
// write into hosts file
|
||||
ArrayList<String>list = new ArrayList<String>();
|
||||
|
@ -553,15 +553,15 @@ public class TestStartup extends TestCase {
|
|||
localFileSys.delete(name, true);
|
||||
}
|
||||
|
||||
FSDataOutputStream stm = localFileSys.create(name);
|
||||
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();
|
||||
}
|
||||
stm.close();
|
||||
}
|
||||
|
||||
private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
|
||||
|
|
Loading…
Reference in New Issue