You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cn...@apache.org on 2015/05/07 06:28:55 UTC
[2/2] hadoop git commit: HDFS-7833. DataNode reconfiguration does not
recalculate valid volumes required,
based on configured failed volumes tolerated. Contributed by Lei (Eddy) Xu.
HDFS-7833. DataNode reconfiguration does not recalculate valid volumes required, based on configured failed volumes tolerated. Contributed by Lei (Eddy) Xu.
(cherry picked from commit 6633a8474d7e92fa028ede8fd6c6e41b6c5887f5)
Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/276e2d33
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/276e2d33
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/276e2d33
Branch: refs/heads/branch-2
Commit: 276e2d3390f1463be723d0d190fceb86621fb138
Parents: d127dd6
Author: cnauroth <cn...@apache.org>
Authored: Wed May 6 21:11:30 2015 -0700
Committer: cnauroth <cn...@apache.org>
Committed: Wed May 6 21:11:44 2015 -0700
----------------------------------------------------------------------
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 4 +
.../datanode/fsdataset/impl/FsDatasetImpl.java | 7 +-
.../datanode/TestDataNodeVolumeFailure.java | 104 +++++++++++++++++--
3 files changed, 105 insertions(+), 10 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hadoop/blob/276e2d33/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 19fd4d5..055814d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -314,6 +314,10 @@ Release 2.8.0 - UNRELEASED
HDFS-2484. checkLease should throw FileNotFoundException when file does
not exist. (Rakesh R via shv)
+ HDFS-7833. DataNode reconfiguration does not recalculate valid volumes
+ required, based on configured failed volumes tolerated.
+ (Lei (Eddy) Xu via cnauroth)
+
Release 2.7.1 - UNRELEASED
INCOMPATIBLE CHANGES
http://git-wip-us.apache.org/repos/asf/hadoop/blob/276e2d33/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index f71ffef..46c488f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -240,7 +240,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
final Daemon lazyWriter;
final FsDatasetCache cacheManager;
private final Configuration conf;
- private final int validVolsRequired;
+ private final int volFailuresTolerated;
private volatile boolean fsRunning;
final ReplicaMap volumeMap;
@@ -271,7 +271,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
this.smallBufferSize = DFSUtil.getSmallBufferSize(conf);
// The number of volumes required for operation is the total number
// of volumes minus the number of failed volumes we can tolerate.
- final int volFailuresTolerated =
+ volFailuresTolerated =
conf.getInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY,
DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT);
@@ -282,7 +282,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
int volsConfigured = (dataDirs == null) ? 0 : dataDirs.length;
int volsFailed = volumeFailureInfos.size();
- this.validVolsRequired = volsConfigured - volFailuresTolerated;
if (volFailuresTolerated < 0 || volFailuresTolerated >= volsConfigured) {
throw new DiskErrorException("Invalid value configured for "
@@ -545,7 +544,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
*/
@Override // FsDatasetSpi
public boolean hasEnoughResource() {
- return volumes.getVolumes().size() >= validVolsRequired;
+ return getNumFailedVolumes() <= volFailuresTolerated;
}
/**
http://git-wip-us.apache.org/repos/asf/hadoop/blob/276e2d33/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
index d38892c..b9c7691 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
@@ -34,6 +34,7 @@ import java.util.Map;
import java.util.concurrent.TimeoutException;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.ReconfigurationException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
@@ -219,12 +220,7 @@ public class TestDataNodeVolumeFailure {
File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
DataNodeTestUtils.injectDataDirFailure(dn0Vol1);
DataNode dn0 = cluster.getDataNodes().get(0);
- long lastDiskErrorCheck = dn0.getLastDiskErrorCheck();
- dn0.checkDiskErrorAsync();
- // Wait checkDiskError thread finish to discover volume failure.
- while (dn0.getLastDiskErrorCheck() == lastDiskErrorCheck) {
- Thread.sleep(100);
- }
+ checkDiskErrorSync(dn0);
// Verify dn0Vol1 has been completely removed from DN0.
// 1. dn0Vol1 is removed from DataStorage.
@@ -270,6 +266,102 @@ public class TestDataNodeVolumeFailure {
assertFalse(dataDirStrs[0].contains(dn0Vol1.getAbsolutePath()));
}
+ private static void checkDiskErrorSync(DataNode dn)
+ throws InterruptedException {
+ final long lastDiskErrorCheck = dn.getLastDiskErrorCheck();
+ dn.checkDiskErrorAsync();
+ // Wait 10 seconds for checkDiskError thread to finish and discover volume
+ // failures.
+ int count = 100;
+ while (count > 0 && dn.getLastDiskErrorCheck() == lastDiskErrorCheck) {
+ Thread.sleep(100);
+ count--;
+ }
+ assertTrue("Disk checking thread does not finish in 10 seconds",
+ count > 0);
+ }
+
+ /**
+ * Test DataNode stops when the number of failed volumes exceeds
+ * dfs.datanode.failed.volumes.tolerated .
+ */
+ @Test(timeout=10000)
+ public void testDataNodeShutdownAfterNumFailedVolumeExceedsTolerated()
+ throws InterruptedException, IOException {
+ // make both data directories to fail on dn0
+ final File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
+ final File dn0Vol2 = new File(dataDir, "data" + (2 * 0 + 2));
+ DataNodeTestUtils.injectDataDirFailure(dn0Vol1, dn0Vol2);
+ DataNode dn0 = cluster.getDataNodes().get(0);
+ checkDiskErrorSync(dn0);
+
+ // DN0 should stop after the number of failure disks exceed tolerated
+ // value (1).
+ assertFalse(dn0.shouldRun());
+ }
+
+ /**
+ * Test that DN does not shutdown, as long as failure volumes being hot swapped.
+ */
+ @Test
+ public void testVolumeFailureRecoveredByHotSwappingVolume()
+ throws InterruptedException, ReconfigurationException, IOException {
+ final File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
+ final File dn0Vol2 = new File(dataDir, "data" + (2 * 0 + 2));
+ final DataNode dn0 = cluster.getDataNodes().get(0);
+ final String oldDataDirs = dn0.getConf().get(
+ DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
+
+ // Fail dn0Vol1 first.
+ DataNodeTestUtils.injectDataDirFailure(dn0Vol1);
+ checkDiskErrorSync(dn0);
+
+ // Hot swap out the failure volume.
+ String dataDirs = dn0Vol2.getPath();
+ dn0.reconfigurePropertyImpl(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
+ dataDirs);
+
+ // Fix failure volume dn0Vol1 and remount it back.
+ DataNodeTestUtils.restoreDataDirFromFailure(dn0Vol1);
+ dn0.reconfigurePropertyImpl(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
+ oldDataDirs);
+
+ // Fail dn0Vol2. Now since dn0Vol1 has been fixed, DN0 has sufficient
+ // resources, thus it should keep running.
+ DataNodeTestUtils.injectDataDirFailure(dn0Vol2);
+ checkDiskErrorSync(dn0);
+ assertTrue(dn0.shouldRun());
+ }
+
+ /**
+ * Test changing the number of volumes does not impact the disk failure
+ * tolerance.
+ */
+ @Test
+ public void testTolerateVolumeFailuresAfterAddingMoreVolumes()
+ throws InterruptedException, ReconfigurationException, IOException {
+ final File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
+ final File dn0Vol2 = new File(dataDir, "data" + (2 * 0 + 2));
+ final File dn0VolNew = new File(dataDir, "data_new");
+ final DataNode dn0 = cluster.getDataNodes().get(0);
+ final String oldDataDirs = dn0.getConf().get(
+ DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
+
+ // Add a new volume to DN0
+ dn0.reconfigurePropertyImpl(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
+ oldDataDirs + "," + dn0VolNew.getAbsolutePath());
+
+ // Fail dn0Vol1 first and hot swap it.
+ DataNodeTestUtils.injectDataDirFailure(dn0Vol1);
+ checkDiskErrorSync(dn0);
+ assertTrue(dn0.shouldRun());
+
+ // Fail dn0Vol2, now dn0 should stop, because we only tolerate 1 disk failure.
+ DataNodeTestUtils.injectDataDirFailure(dn0Vol2);
+ checkDiskErrorSync(dn0);
+ assertFalse(dn0.shouldRun());
+ }
+
/**
* Test that there are under replication blocks after vol failures
*/