HDFS-12716. 'dfs.datanode.failed.volumes.tolerated' to support minimum number of volumes to be available. Contributed by Ranith Sardar and usharani
This commit is contained in:
parent
63e08ec071
commit
3108d27edd
|
@ -293,6 +293,8 @@ public class DataNode extends ReconfigurableBase
|
|||
" and rolling upgrades.";
|
||||
|
||||
static final int CURRENT_BLOCK_FORMAT_VERSION = 1;
|
||||
public static final int MAX_VOLUME_FAILURE_TOLERATED_LIMIT = -1;
|
||||
public static final String MAX_VOLUME_FAILURES_TOLERATED_MSG = "should be greater than -1";
|
||||
|
||||
/** A list of property that are reconfigurable at runtime. */
|
||||
private static final List<String> RECONFIGURABLE_PROPERTIES =
|
||||
|
@ -1389,10 +1391,11 @@ public class DataNode extends ReconfigurableBase
|
|||
|
||||
int volFailuresTolerated = dnConf.getVolFailuresTolerated();
|
||||
int volsConfigured = dnConf.getVolsConfigured();
|
||||
if (volFailuresTolerated < 0 || volFailuresTolerated >= volsConfigured) {
|
||||
if (volFailuresTolerated < 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 0 or >= "
|
||||
+ ". Value configured is either greater than -1 or >= "
|
||||
+ "to the number of configured volumes (" + volsConfigured + ").");
|
||||
}
|
||||
|
||||
|
|
|
@ -28,6 +28,7 @@ import com.google.common.util.concurrent.ListenableFuture;
|
|||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
||||
|
@ -153,10 +154,11 @@ public class DatasetVolumeChecker {
|
|||
|
||||
lastAllVolumesCheck = timer.monotonicNow() - minDiskCheckGapMs;
|
||||
|
||||
if (maxVolumeFailuresTolerated < 0) {
|
||||
if (maxVolumeFailuresTolerated < DataNode.MAX_VOLUME_FAILURE_TOLERATED_LIMIT) {
|
||||
throw new DiskErrorException("Invalid value configured for "
|
||||
+ DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY + " - "
|
||||
+ maxVolumeFailuresTolerated + " (should be non-negative)");
|
||||
+ maxVolumeFailuresTolerated + " "
|
||||
+ DataNode.MAX_VOLUME_FAILURES_TOLERATED_MSG);
|
||||
}
|
||||
|
||||
delegateChecker = new ThrottledAsyncChecker<>(
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
|
||||
import org.apache.hadoop.hdfs.server.datanode.StorageLocation.CheckContext;
|
||||
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
||||
|
@ -105,10 +106,11 @@ public class StorageLocationChecker {
|
|||
DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY,
|
||||
DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT);
|
||||
|
||||
if (maxVolumeFailuresTolerated < 0) {
|
||||
if (maxVolumeFailuresTolerated < DataNode.MAX_VOLUME_FAILURE_TOLERATED_LIMIT) {
|
||||
throw new DiskErrorException("Invalid value configured for "
|
||||
+ DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY + " - "
|
||||
+ maxVolumeFailuresTolerated + " (should be non-negative)");
|
||||
+ maxVolumeFailuresTolerated + " "
|
||||
+ DataNode.MAX_VOLUME_FAILURES_TOLERATED_MSG);
|
||||
}
|
||||
|
||||
this.timer = timer;
|
||||
|
@ -213,12 +215,22 @@ public class StorageLocationChecker {
|
|||
}
|
||||
}
|
||||
|
||||
if (failedLocations.size() > maxVolumeFailuresTolerated) {
|
||||
throw new DiskErrorException("Too many failed volumes - "
|
||||
+ "current valid volumes: " + goodLocations.size()
|
||||
+ ", volumes configured: " + dataDirs.size()
|
||||
+ ", volumes failed: " + failedLocations.size()
|
||||
+ ", volume failures tolerated: " + maxVolumeFailuresTolerated);
|
||||
if (maxVolumeFailuresTolerated == DataNode.MAX_VOLUME_FAILURE_TOLERATED_LIMIT) {
|
||||
if (dataDirs.size() == failedLocations.size()) {
|
||||
throw new DiskErrorException("Too many failed volumes - "
|
||||
+ "current valid volumes: " + goodLocations.size()
|
||||
+ ", volumes configured: " + dataDirs.size()
|
||||
+ ", volumes failed: " + failedLocations.size()
|
||||
+ ", volume failures tolerated: " + maxVolumeFailuresTolerated);
|
||||
}
|
||||
} else {
|
||||
if (failedLocations.size() > maxVolumeFailuresTolerated) {
|
||||
throw new DiskErrorException("Too many failed volumes - "
|
||||
+ "current valid volumes: " + goodLocations.size()
|
||||
+ ", volumes configured: " + dataDirs.size()
|
||||
+ ", volumes failed: " + failedLocations.size()
|
||||
+ ", volume failures tolerated: " + maxVolumeFailuresTolerated);
|
||||
}
|
||||
}
|
||||
|
||||
if (goodLocations.size() == 0) {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -1276,6 +1276,8 @@
|
|||
<description>The number of volumes that are allowed to
|
||||
fail before a datanode stops offering service. By default
|
||||
any volume failure will cause a datanode to shutdown.
|
||||
The range of the value is -1 now, -1 represents the minimum
|
||||
of volume valids is 1.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
|
|
|
@ -201,7 +201,11 @@ public class TestDataNodeVolumeFailureToleration {
|
|||
@Test
|
||||
public void testVolumeAndTolerableConfiguration() throws Exception {
|
||||
// Check if Block Pool Service exit for an invalid conf value.
|
||||
testVolumeConfig(-1, 0, false, true);
|
||||
testVolumeConfig(-2, 0, false, true);
|
||||
// Test for one good volume at least
|
||||
testVolumeConfig(-1, 0, true, true);
|
||||
testVolumeConfig(-1, 1, true, true);
|
||||
testVolumeConfig(-1, 2, false, true);
|
||||
|
||||
// Ditto if the value is too big.
|
||||
testVolumeConfig(100, 0, false, true);
|
||||
|
|
Loading…
Reference in New Issue