HDFS-10279. Improve validation of the configured number of tolerated failed volumes. Contributed by Lin Yiqun.

This commit is contained in:
Andrew Wang 2016-04-13 16:39:50 -07:00
parent 192112d5a2
commit 314aa21a89
4 changed files with 46 additions and 13 deletions

View File

@ -114,6 +114,9 @@ public class DNConf {
// Allow LAZY_PERSIST writes from non-local clients?
private final boolean allowNonLocalLazyPersist;
private final int volFailuresTolerated;
private final int volsConfigured;
public DNConf(Configuration conf) {
this.conf = conf;
socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
@ -238,6 +241,13 @@ public class DNConf {
this.bpReadyTimeout = conf.getLong(
DFS_DATANODE_BP_READY_TIMEOUT_KEY,
DFS_DATANODE_BP_READY_TIMEOUT_DEFAULT);
this.volFailuresTolerated =
conf.getInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY,
DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT);
String[] dataDirs =
conf.getTrimmedStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
this.volsConfigured = (dataDirs == null) ? 0 : dataDirs.length;
}
// We get minimumNameNodeVersion via a method so it can be mocked out in tests.
@ -363,4 +373,12 @@ public class DNConf {
public long getLifelineIntervalMs() {
return lifelineIntervalMs;
}
public int getVolFailuresTolerated() {
return volFailuresTolerated;
}
public int getVolsConfigured() {
return volsConfigured;
}
}

View File

@ -1268,6 +1268,15 @@ public class DataNode extends ReconfigurableBase
LOG.info("Starting DataNode with maxLockedMemory = " +
dnConf.maxLockedMemory);
int volFailuresTolerated = dnConf.getVolFailuresTolerated();
int volsConfigured = dnConf.getVolsConfigured();
if (volFailuresTolerated < 0 || volFailuresTolerated >= volsConfigured) {
throw new DiskErrorException("Invalid value configured for "
+ "dfs.datanode.failed.volumes.tolerated - " + volFailuresTolerated
+ ". Value configured is either less than 0 or >= "
+ "to the number of configured volumes (" + volsConfigured + ").");
}
storage = new DataStorage();
// global DN settings

View File

@ -268,24 +268,15 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
this.smallBufferSize = DFSUtilClient.getSmallBufferSize(conf);
// The number of volumes required for operation is the total number
// of volumes minus the number of failed volumes we can tolerate.
volFailuresTolerated =
conf.getInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY,
DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT);
volFailuresTolerated = datanode.getDnConf().getVolFailuresTolerated();
String[] dataDirs = conf.getTrimmedStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
Collection<StorageLocation> dataLocations = DataNode.getStorageLocations(conf);
List<VolumeFailureInfo> volumeFailureInfos = getInitialVolumeFailureInfos(
dataLocations, storage);
int volsConfigured = (dataDirs == null) ? 0 : dataDirs.length;
int volsConfigured = datanode.getDnConf().getVolsConfigured();
int volsFailed = volumeFailureInfos.size();
if (volFailuresTolerated < 0 || volFailuresTolerated >= volsConfigured) {
throw new DiskErrorException("Invalid value configured for "
+ "dfs.datanode.failed.volumes.tolerated - " + volFailuresTolerated
+ ". Value configured is either less than 0 or >= "
+ "to the number of configured volumes (" + volsConfigured + ").");
}
if (volsFailed > volFailuresTolerated) {
throw new DiskErrorException("Too many failed volumes - "
+ "current valid volumes: " + storage.getNumStorageDirs()

View File

@ -34,6 +34,8 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@ -229,9 +231,22 @@ public class TestDataNodeVolumeFailureToleration {
prepareDirToFail(dirs[i]);
}
restartDatanodes(volumesTolerated, manageDfsDirs);
assertEquals(expectedBPServiceState, cluster.getDataNodes().get(0)
.isBPServiceAlive(cluster.getNamesystem().getBlockPoolId()));
} catch (DiskErrorException e) {
GenericTestUtils.assertExceptionContains("Invalid value configured for "
+ "dfs.datanode.failed.volumes.tolerated", e);
} finally {
boolean bpServiceState;
// If the datanode not registered successfully,
// because the invalid value configured for tolerated volumes
if (cluster.getDataNodes().size() == 0) {
bpServiceState = false;
} else {
bpServiceState =
cluster.getDataNodes().get(0)
.isBPServiceAlive(cluster.getNamesystem().getBlockPoolId());
}
assertEquals(expectedBPServiceState, bpServiceState);
for (File dir : dirs) {
FileUtil.chmod(dir.toString(), "755");
}