HDFS-15125. Pull back HDFS-11353, HDFS-13993, HDFS-13945, and HDFS-14324
to branch-2.10. Contributed by Jim Brennan.
This commit is contained in:
parent
dc010b9844
commit
c5d43b65a9
|
@ -939,8 +939,7 @@ public class TestDataNodeHotSwapVolumes {
|
||||||
*/
|
*/
|
||||||
@Test(timeout=60000)
|
@Test(timeout=60000)
|
||||||
public void testDirectlyReloadAfterCheckDiskError()
|
public void testDirectlyReloadAfterCheckDiskError()
|
||||||
throws IOException, TimeoutException, InterruptedException,
|
throws Exception {
|
||||||
ReconfigurationException {
|
|
||||||
// The test uses DataNodeTestUtils#injectDataDirFailure() to simulate
|
// The test uses DataNodeTestUtils#injectDataDirFailure() to simulate
|
||||||
// volume failures which is currently not supported on Windows.
|
// volume failures which is currently not supported on Windows.
|
||||||
assumeTrue(!Path.WINDOWS);
|
assumeTrue(!Path.WINDOWS);
|
||||||
|
@ -959,11 +958,7 @@ public class TestDataNodeHotSwapVolumes {
|
||||||
|
|
||||||
DataNodeTestUtils.injectDataDirFailure(dirToFail);
|
DataNodeTestUtils.injectDataDirFailure(dirToFail);
|
||||||
// Call and wait DataNode to detect disk failure.
|
// Call and wait DataNode to detect disk failure.
|
||||||
long lastDiskErrorCheck = dn.getLastDiskErrorCheck();
|
DataNodeTestUtils.waitForDiskError(dn, failedVolume);
|
||||||
dn.checkDiskErrorAsync(failedVolume);
|
|
||||||
while (dn.getLastDiskErrorCheck() == lastDiskErrorCheck) {
|
|
||||||
Thread.sleep(100);
|
|
||||||
}
|
|
||||||
|
|
||||||
createFile(new Path("/test1"), 32, (short)2);
|
createFile(new Path("/test1"), 32, (short)2);
|
||||||
assertEquals(used, failedVolume.getDfsUsed());
|
assertEquals(used, failedVolume.getDfsUsed());
|
||||||
|
|
|
@ -35,16 +35,15 @@ import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.TimeoutException;
|
|
||||||
|
|
||||||
|
import org.apache.commons.io.FileUtils;
|
||||||
|
import org.apache.commons.io.filefilter.TrueFileFilter;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.conf.ReconfigurationException;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.FileUtil;
|
import org.apache.hadoop.fs.FileUtil;
|
||||||
import org.apache.hadoop.fs.FsTracer;
|
import org.apache.hadoop.fs.FsTracer;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hdfs.BlockReader;
|
import org.apache.hadoop.hdfs.BlockReader;
|
||||||
import org.apache.hadoop.hdfs.client.impl.BlockReaderFactory;
|
|
||||||
import org.apache.hadoop.hdfs.ClientContext;
|
import org.apache.hadoop.hdfs.ClientContext;
|
||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||||
|
@ -52,6 +51,7 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
|
||||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||||
import org.apache.hadoop.hdfs.RemotePeerFactory;
|
import org.apache.hadoop.hdfs.RemotePeerFactory;
|
||||||
|
import org.apache.hadoop.hdfs.client.impl.BlockReaderFactory;
|
||||||
import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
|
import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
|
||||||
import org.apache.hadoop.hdfs.net.Peer;
|
import org.apache.hadoop.hdfs.net.Peer;
|
||||||
import org.apache.hadoop.hdfs.protocol.Block;
|
import org.apache.hadoop.hdfs.protocol.Block;
|
||||||
|
@ -75,20 +75,17 @@ import org.apache.hadoop.net.NetUtils;
|
||||||
import org.apache.hadoop.security.token.Token;
|
import org.apache.hadoop.security.token.Token;
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
import org.apache.hadoop.util.Shell;
|
import org.apache.hadoop.util.Shell;
|
||||||
|
|
||||||
import org.apache.commons.io.FileUtils;
|
|
||||||
import org.apache.commons.io.filefilter.TrueFileFilter;
|
|
||||||
|
|
||||||
import com.google.common.base.Supplier;
|
|
||||||
|
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.internal.AssumptionViolatedException;
|
import org.junit.internal.AssumptionViolatedException;
|
||||||
|
import org.junit.rules.Timeout;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import com.google.common.base.Supplier;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Fine-grain testing of block files and locations after volume failure.
|
* Fine-grain testing of block files and locations after volume failure.
|
||||||
*/
|
*/
|
||||||
|
@ -114,6 +111,10 @@ public class TestDataNodeVolumeFailure {
|
||||||
// block id to BlockLocs
|
// block id to BlockLocs
|
||||||
final Map<String, BlockLocs> block_map = new HashMap<String, BlockLocs> ();
|
final Map<String, BlockLocs> block_map = new HashMap<String, BlockLocs> ();
|
||||||
|
|
||||||
|
// specific the timeout for entire test class
|
||||||
|
@Rule
|
||||||
|
public Timeout timeout = new Timeout(120 * 1000);
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setUp() throws Exception {
|
public void setUp() throws Exception {
|
||||||
// bring up a cluster of 2
|
// bring up a cluster of 2
|
||||||
|
@ -229,7 +230,7 @@ public class TestDataNodeVolumeFailure {
|
||||||
*/
|
*/
|
||||||
@Test(timeout=150000)
|
@Test(timeout=150000)
|
||||||
public void testFailedVolumeBeingRemovedFromDataNode()
|
public void testFailedVolumeBeingRemovedFromDataNode()
|
||||||
throws InterruptedException, IOException, TimeoutException {
|
throws Exception {
|
||||||
// The test uses DataNodeTestUtils#injectDataDirFailure() to simulate
|
// The test uses DataNodeTestUtils#injectDataDirFailure() to simulate
|
||||||
// volume failures which is currently not supported on Windows.
|
// volume failures which is currently not supported on Windows.
|
||||||
assumeTrue(!Path.WINDOWS);
|
assumeTrue(!Path.WINDOWS);
|
||||||
|
@ -241,7 +242,8 @@ public class TestDataNodeVolumeFailure {
|
||||||
File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
|
File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
|
||||||
DataNodeTestUtils.injectDataDirFailure(dn0Vol1);
|
DataNodeTestUtils.injectDataDirFailure(dn0Vol1);
|
||||||
DataNode dn0 = cluster.getDataNodes().get(0);
|
DataNode dn0 = cluster.getDataNodes().get(0);
|
||||||
checkDiskErrorSync(dn0, DataNodeTestUtils.getVolume(dn0, dn0Vol1));
|
DataNodeTestUtils.waitForDiskError(dn0,
|
||||||
|
DataNodeTestUtils.getVolume(dn0, dn0Vol1));
|
||||||
|
|
||||||
// Verify dn0Vol1 has been completely removed from DN0.
|
// Verify dn0Vol1 has been completely removed from DN0.
|
||||||
// 1. dn0Vol1 is removed from DataStorage.
|
// 1. dn0Vol1 is removed from DataStorage.
|
||||||
|
@ -287,28 +289,13 @@ public class TestDataNodeVolumeFailure {
|
||||||
assertFalse(dataDirStrs[0].contains(dn0Vol1.getAbsolutePath()));
|
assertFalse(dataDirStrs[0].contains(dn0Vol1.getAbsolutePath()));
|
||||||
}
|
}
|
||||||
|
|
||||||
private static void checkDiskErrorSync(DataNode dn, FsVolumeSpi volume)
|
|
||||||
throws InterruptedException {
|
|
||||||
final long lastDiskErrorCheck = dn.getLastDiskErrorCheck();
|
|
||||||
dn.checkDiskErrorAsync(volume);
|
|
||||||
// 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
|
* Test DataNode stops when the number of failed volumes exceeds
|
||||||
* dfs.datanode.failed.volumes.tolerated .
|
* dfs.datanode.failed.volumes.tolerated .
|
||||||
*/
|
*/
|
||||||
@Test(timeout=10000)
|
@Test(timeout=10000)
|
||||||
public void testDataNodeShutdownAfterNumFailedVolumeExceedsTolerated()
|
public void testDataNodeShutdownAfterNumFailedVolumeExceedsTolerated()
|
||||||
throws InterruptedException, IOException {
|
throws Exception {
|
||||||
if (Shell.WINDOWS) {
|
if (Shell.WINDOWS) {
|
||||||
// The test uses DataNodeTestUtils#injectDataDirFailure() to simulate
|
// The test uses DataNodeTestUtils#injectDataDirFailure() to simulate
|
||||||
// volume failures which is currently not supported on Windows.
|
// volume failures which is currently not supported on Windows.
|
||||||
|
@ -320,11 +307,14 @@ public class TestDataNodeVolumeFailure {
|
||||||
final File dn0Vol2 = new File(dataDir, "data" + (2 * 0 + 2));
|
final File dn0Vol2 = new File(dataDir, "data" + (2 * 0 + 2));
|
||||||
DataNodeTestUtils.injectDataDirFailure(dn0Vol1, dn0Vol2);
|
DataNodeTestUtils.injectDataDirFailure(dn0Vol1, dn0Vol2);
|
||||||
DataNode dn0 = cluster.getDataNodes().get(0);
|
DataNode dn0 = cluster.getDataNodes().get(0);
|
||||||
checkDiskErrorSync(dn0, DataNodeTestUtils.getVolume(dn0, dn0Vol1));
|
DataNodeTestUtils.waitForDiskError(dn0,
|
||||||
checkDiskErrorSync(dn0, DataNodeTestUtils.getVolume(dn0, dn0Vol2));
|
DataNodeTestUtils.getVolume(dn0, dn0Vol1));
|
||||||
|
DataNodeTestUtils.waitForDiskError(dn0,
|
||||||
|
DataNodeTestUtils.getVolume(dn0, dn0Vol2));
|
||||||
|
|
||||||
// DN0 should stop after the number of failure disks exceed tolerated
|
// DN0 should stop after the number of failure disks exceed tolerated
|
||||||
// value (1).
|
// value (1).
|
||||||
|
dn0.checkDiskError();
|
||||||
assertFalse(dn0.shouldRun());
|
assertFalse(dn0.shouldRun());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -333,7 +323,7 @@ public class TestDataNodeVolumeFailure {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void testVolumeFailureRecoveredByHotSwappingVolume()
|
public void testVolumeFailureRecoveredByHotSwappingVolume()
|
||||||
throws InterruptedException, ReconfigurationException, IOException {
|
throws Exception {
|
||||||
if (Shell.WINDOWS) {
|
if (Shell.WINDOWS) {
|
||||||
// The test uses DataNodeTestUtils#injectDataDirFailure() to simulate
|
// The test uses DataNodeTestUtils#injectDataDirFailure() to simulate
|
||||||
// volume failures which is currently not supported on Windows.
|
// volume failures which is currently not supported on Windows.
|
||||||
|
@ -348,7 +338,8 @@ public class TestDataNodeVolumeFailure {
|
||||||
|
|
||||||
// Fail dn0Vol1 first.
|
// Fail dn0Vol1 first.
|
||||||
DataNodeTestUtils.injectDataDirFailure(dn0Vol1);
|
DataNodeTestUtils.injectDataDirFailure(dn0Vol1);
|
||||||
checkDiskErrorSync(dn0, DataNodeTestUtils.getVolume(dn0, dn0Vol1));
|
DataNodeTestUtils.waitForDiskError(dn0,
|
||||||
|
DataNodeTestUtils.getVolume(dn0, dn0Vol1));
|
||||||
|
|
||||||
// Hot swap out the failure volume.
|
// Hot swap out the failure volume.
|
||||||
String dataDirs = dn0Vol2.getPath();
|
String dataDirs = dn0Vol2.getPath();
|
||||||
|
@ -367,7 +358,8 @@ public class TestDataNodeVolumeFailure {
|
||||||
// Fail dn0Vol2. Now since dn0Vol1 has been fixed, DN0 has sufficient
|
// Fail dn0Vol2. Now since dn0Vol1 has been fixed, DN0 has sufficient
|
||||||
// resources, thus it should keep running.
|
// resources, thus it should keep running.
|
||||||
DataNodeTestUtils.injectDataDirFailure(dn0Vol2);
|
DataNodeTestUtils.injectDataDirFailure(dn0Vol2);
|
||||||
checkDiskErrorSync(dn0, DataNodeTestUtils.getVolume(dn0, dn0Vol2));
|
DataNodeTestUtils.waitForDiskError(dn0,
|
||||||
|
DataNodeTestUtils.getVolume(dn0, dn0Vol2));
|
||||||
assertTrue(dn0.shouldRun());
|
assertTrue(dn0.shouldRun());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -377,7 +369,7 @@ public class TestDataNodeVolumeFailure {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void testTolerateVolumeFailuresAfterAddingMoreVolumes()
|
public void testTolerateVolumeFailuresAfterAddingMoreVolumes()
|
||||||
throws InterruptedException, ReconfigurationException, IOException {
|
throws Exception {
|
||||||
if (Shell.WINDOWS) {
|
if (Shell.WINDOWS) {
|
||||||
// The test uses DataNodeTestUtils#injectDataDirFailure() to simulate
|
// The test uses DataNodeTestUtils#injectDataDirFailure() to simulate
|
||||||
// volume failures which is currently not supported on Windows.
|
// volume failures which is currently not supported on Windows.
|
||||||
|
@ -400,12 +392,15 @@ public class TestDataNodeVolumeFailure {
|
||||||
|
|
||||||
// Fail dn0Vol1 first and hot swap it.
|
// Fail dn0Vol1 first and hot swap it.
|
||||||
DataNodeTestUtils.injectDataDirFailure(dn0Vol1);
|
DataNodeTestUtils.injectDataDirFailure(dn0Vol1);
|
||||||
checkDiskErrorSync(dn0, DataNodeTestUtils.getVolume(dn0, dn0Vol1));
|
DataNodeTestUtils.waitForDiskError(dn0,
|
||||||
|
DataNodeTestUtils.getVolume(dn0, dn0Vol1));
|
||||||
assertTrue(dn0.shouldRun());
|
assertTrue(dn0.shouldRun());
|
||||||
|
|
||||||
// Fail dn0Vol2, now dn0 should stop, because we only tolerate 1 disk failure.
|
// Fail dn0Vol2, now dn0 should stop, because we only tolerate 1 disk failure.
|
||||||
DataNodeTestUtils.injectDataDirFailure(dn0Vol2);
|
DataNodeTestUtils.injectDataDirFailure(dn0Vol2);
|
||||||
checkDiskErrorSync(dn0, DataNodeTestUtils.getVolume(dn0, dn0Vol2));
|
DataNodeTestUtils.waitForDiskError(dn0,
|
||||||
|
DataNodeTestUtils.getVolume(dn0, dn0Vol2));
|
||||||
|
dn0.checkDiskError();
|
||||||
assertFalse(dn0.shouldRun());
|
assertFalse(dn0.shouldRun());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -441,10 +436,8 @@ public class TestDataNodeVolumeFailure {
|
||||||
@Override
|
@Override
|
||||||
public Boolean get() {
|
public Boolean get() {
|
||||||
// underReplicatedBlocks are due to failed volumes
|
// underReplicatedBlocks are due to failed volumes
|
||||||
int underReplicatedBlocks = BlockManagerTestUtil
|
long underReplicatedBlocks = bm.getUnderReplicatedNotMissingBlocks() +
|
||||||
.checkHeartbeatAndGetUnderReplicatedBlocksCount(
|
bm.getPendingReplicationBlocksCount();
|
||||||
cluster.getNamesystem(), bm);
|
|
||||||
|
|
||||||
if (underReplicatedBlocks > 0) {
|
if (underReplicatedBlocks > 0) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
|
@ -55,7 +55,9 @@ import org.apache.log4j.Level;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.Timeout;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test reporting of DN volume failure counts and metrics.
|
* Test reporting of DN volume failure counts and metrics.
|
||||||
|
@ -83,6 +85,10 @@ public class TestDataNodeVolumeFailureReporting {
|
||||||
// a datanode to be considered dead by the namenode.
|
// a datanode to be considered dead by the namenode.
|
||||||
final int WAIT_FOR_DEATH = 15000;
|
final int WAIT_FOR_DEATH = 15000;
|
||||||
|
|
||||||
|
// specific the timeout for entire test class
|
||||||
|
@Rule
|
||||||
|
public Timeout timeout = new Timeout(120 * 1000);
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setUp() throws Exception {
|
public void setUp() throws Exception {
|
||||||
// These tests use DataNodeTestUtils#injectDataDirFailure() to simulate
|
// These tests use DataNodeTestUtils#injectDataDirFailure() to simulate
|
||||||
|
@ -207,13 +213,13 @@ public class TestDataNodeVolumeFailureReporting {
|
||||||
DFSTestUtil.createFile(fs, file3, 1024, (short)3, 1L);
|
DFSTestUtil.createFile(fs, file3, 1024, (short)3, 1L);
|
||||||
DFSTestUtil.waitReplication(fs, file3, (short)2);
|
DFSTestUtil.waitReplication(fs, file3, (short)2);
|
||||||
|
|
||||||
// The DN should consider itself dead
|
|
||||||
DFSTestUtil.waitForDatanodeDeath(dns.get(2));
|
|
||||||
|
|
||||||
// And report two failed volumes
|
// And report two failed volumes
|
||||||
checkFailuresAtDataNode(dns.get(2), 2, true, dn3Vol1.getAbsolutePath(),
|
checkFailuresAtDataNode(dns.get(2), 2, true, dn3Vol1.getAbsolutePath(),
|
||||||
dn3Vol2.getAbsolutePath());
|
dn3Vol2.getAbsolutePath());
|
||||||
|
|
||||||
|
// The DN should consider itself dead
|
||||||
|
DFSTestUtil.waitForDatanodeDeath(dns.get(2));
|
||||||
|
|
||||||
// The NN considers the DN dead
|
// The NN considers the DN dead
|
||||||
DFSTestUtil.waitForDatanodeStatus(dm, 2, 1, 2,
|
DFSTestUtil.waitForDatanodeStatus(dm, 2, 1, 2,
|
||||||
origCapacity - (4*dnCapacity), WAIT_FOR_HEARTBEATS);
|
origCapacity - (4*dnCapacity), WAIT_FOR_HEARTBEATS);
|
||||||
|
|
|
@ -39,7 +39,9 @@ import org.apache.hadoop.test.GenericTestUtils;
|
||||||
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.Timeout;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test the ability of a DN to tolerate volume failures.
|
* Test the ability of a DN to tolerate volume failures.
|
||||||
|
@ -58,6 +60,10 @@ public class TestDataNodeVolumeFailureToleration {
|
||||||
// a datanode to be considered dead by the namenode.
|
// a datanode to be considered dead by the namenode.
|
||||||
final int WAIT_FOR_DEATH = 15000;
|
final int WAIT_FOR_DEATH = 15000;
|
||||||
|
|
||||||
|
// specific the timeout for entire test class
|
||||||
|
@Rule
|
||||||
|
public Timeout timeout = new Timeout(120 * 1000);
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setUp() throws Exception {
|
public void setUp() throws Exception {
|
||||||
conf = new HdfsConfiguration();
|
conf = new HdfsConfiguration();
|
||||||
|
|
Loading…
Reference in New Issue