|
@@ -237,6 +237,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
final FsDatasetCache cacheManager;
|
|
|
private final Configuration conf;
|
|
|
private final int volFailuresTolerated;
|
|
|
+ private final int volsConfigured;
|
|
|
private volatile boolean fsRunning;
|
|
|
|
|
|
final ReplicaMap volumeMap;
|
|
@@ -285,15 +286,32 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
List<VolumeFailureInfo> volumeFailureInfos = getInitialVolumeFailureInfos(
|
|
|
dataLocations, storage);
|
|
|
|
|
|
- int volsConfigured = datanode.getDnConf().getVolsConfigured();
|
|
|
+ volsConfigured = datanode.getDnConf().getVolsConfigured();
|
|
|
int volsFailed = volumeFailureInfos.size();
|
|
|
|
|
|
- if (volsFailed > volFailuresTolerated) {
|
|
|
- throw new DiskErrorException("Too many failed volumes - "
|
|
|
- + "current valid volumes: " + storage.getNumStorageDirs()
|
|
|
- + ", volumes configured: " + volsConfigured
|
|
|
- + ", volumes failed: " + volsFailed
|
|
|
- + ", volume failures tolerated: " + volFailuresTolerated);
|
|
|
+ if (volFailuresTolerated < DataNode.MAX_VOLUME_FAILURE_TOLERATED_LIMIT
|
|
|
+ || volFailuresTolerated >= volsConfigured) {
|
|
|
+ throw new DiskErrorException("Invalid value configured for "
|
|
|
+ + "dfs.datanode.failed.volumes.tolerated - " + volFailuresTolerated
|
|
|
+ + ". Value configured is either less than maxVolumeFailureLimit or greater than "
|
|
|
+ + "to the number of configured volumes (" + volsConfigured + ").");
|
|
|
+ }
|
|
|
+ if (volFailuresTolerated == DataNode.MAX_VOLUME_FAILURE_TOLERATED_LIMIT) {
|
|
|
+ if (volsConfigured == volsFailed) {
|
|
|
+ throw new DiskErrorException(
|
|
|
+ "Too many failed volumes - " + "current valid volumes: "
|
|
|
+ + storage.getNumStorageDirs() + ", volumes configured: "
|
|
|
+ + volsConfigured + ", volumes failed: " + volsFailed
|
|
|
+ + ", volume failures tolerated: " + volFailuresTolerated);
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ if (volsFailed > volFailuresTolerated) {
|
|
|
+ throw new DiskErrorException(
|
|
|
+ "Too many failed volumes - " + "current valid volumes: "
|
|
|
+ + storage.getNumStorageDirs() + ", volumes configured: "
|
|
|
+ + volsConfigured + ", volumes failed: " + volsFailed
|
|
|
+ + ", volume failures tolerated: " + volFailuresTolerated);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
storageMap = new ConcurrentHashMap<String, DatanodeStorage>();
|
|
@@ -597,7 +615,13 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
*/
|
|
|
@Override // FsDatasetSpi
|
|
|
public boolean hasEnoughResource() {
|
|
|
- return getNumFailedVolumes() <= volFailuresTolerated;
|
|
|
+ if (volFailuresTolerated == DataNode.MAX_VOLUME_FAILURE_TOLERATED_LIMIT) {
|
|
|
+ // If volFailuresTolerated configured maxVolumeFailureLimit then minimum
|
|
|
+ // one volume is required.
|
|
|
+ return volumes.getVolumes().size() >= 1;
|
|
|
+ } else {
|
|
|
+ return getNumFailedVolumes() <= volFailuresTolerated;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|