|
@@ -24,23 +24,25 @@ import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.List;
|
|
|
+import java.util.Random;
|
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
+import org.apache.commons.logging.impl.Log4JLogger;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.SecurityTestUtil;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
@@ -59,6 +61,9 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
static {
|
|
|
GenericTestUtils.setLogLevel(DFSOutputStream.LOG, Level.ALL);
|
|
|
GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL);
|
|
|
+ GenericTestUtils.setLogLevel(DFSClient.LOG, Level.ALL);
|
|
|
+ ((Log4JLogger)LogFactory.getLog(BlockPlacementPolicy.class))
|
|
|
+ .getLogger().setLevel(Level.ALL);
|
|
|
}
|
|
|
|
|
|
private static final int NUM_DATA_BLOCKS = StripedFileTestUtil.NUM_DATA_BLOCKS;
|
|
@@ -134,6 +139,7 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
private DistributedFileSystem dfs;
|
|
|
private final Path dir = new Path("/"
|
|
|
+ TestDFSStripedOutputStreamWithFailure.class.getSimpleName());
|
|
|
+ private final Random random = new Random();
|
|
|
|
|
|
private void setup(Configuration conf) throws IOException {
|
|
|
final int numDNs = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
|
|
@@ -153,7 +159,8 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
private HdfsConfiguration newHdfsConfiguration() {
|
|
|
final HdfsConfiguration conf = new HdfsConfiguration();
|
|
|
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
|
|
- conf.setLong(HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 6000L);
|
|
|
+ conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY,
|
|
|
+ false);
|
|
|
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
|
|
|
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
|
|
|
return conf;
|
|
@@ -164,11 +171,31 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
runTest(getLength(56));
|
|
|
}
|
|
|
|
|
|
+ @Test(timeout=240000)
|
|
|
+ public void testDatanodeFailureRandomLength() throws Exception {
|
|
|
+ int lenIndex = random.nextInt(LENGTHS.size());
|
|
|
+ LOG.info("run testMultipleDatanodeFailureRandomLength with length index: "
|
|
|
+ + lenIndex);
|
|
|
+ runTest(getLength(lenIndex));
|
|
|
+ }
|
|
|
+
|
|
|
@Test(timeout=240000)
|
|
|
public void testMultipleDatanodeFailure56() throws Exception {
|
|
|
runTestWithMultipleFailure(getLength(56));
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Randomly pick a length and run tests with multiple data failures
|
|
|
+ * TODO: enable this later
|
|
|
+ */
|
|
|
+ //@Test(timeout=240000)
|
|
|
+ public void testMultipleDatanodeFailureRandomLength() throws Exception {
|
|
|
+ int lenIndex = random.nextInt(LENGTHS.size());
|
|
|
+ LOG.info("run testMultipleDatanodeFailureRandomLength with length index: "
|
|
|
+ + lenIndex);
|
|
|
+ runTestWithMultipleFailure(getLength(lenIndex));
|
|
|
+ }
|
|
|
+
|
|
|
@Test(timeout=240000)
|
|
|
public void testBlockTokenExpired() throws Exception {
|
|
|
final int length = NUM_DATA_BLOCKS * (BLOCK_SIZE - CELL_SIZE);
|
|
@@ -208,11 +235,10 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
}
|
|
|
cluster.restartNameNodes();
|
|
|
cluster.triggerHeartbeats();
|
|
|
- DatanodeInfo[] info = dfs.getClient().datanodeReport(
|
|
|
- DatanodeReportType.LIVE);
|
|
|
+ DatanodeInfo[] info = dfs.getClient().datanodeReport(DatanodeReportType.LIVE);
|
|
|
assertEquals("Mismatches number of live Dns ", numDatanodes, info.length);
|
|
|
final Path dirFile = new Path(dir, "ecfile");
|
|
|
- FSDataOutputStream out = null;
|
|
|
+ FSDataOutputStream out;
|
|
|
try {
|
|
|
out = dfs.create(dirFile, true);
|
|
|
out.write("something".getBytes());
|
|
@@ -262,6 +288,7 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
final HdfsConfiguration conf = newHdfsConfiguration();
|
|
|
for (int dn = 0; dn < 9; dn++) {
|
|
|
try {
|
|
|
+ LOG.info("runTest: dn=" + dn + ", length=" + length);
|
|
|
setup(conf);
|
|
|
runTest(length, new int[]{length/2}, new int[]{dn}, false);
|
|
|
} catch (Throwable e) {
|
|
@@ -277,10 +304,11 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
|
|
|
void runTestWithMultipleFailure(final int length) throws Exception {
|
|
|
final HdfsConfiguration conf = newHdfsConfiguration();
|
|
|
- for(int i=0;i<dnIndexSuite.length;i++){
|
|
|
- int[] dnIndex = dnIndexSuite[i];
|
|
|
+ for (int[] dnIndex : dnIndexSuite) {
|
|
|
int[] killPos = getKillPositions(length, dnIndex.length);
|
|
|
try {
|
|
|
+ LOG.info("runTestWithMultipleFailure: length==" + length + ", killPos="
|
|
|
+ + Arrays.toString(killPos) + ", dnIndex=" + Arrays.toString(dnIndex));
|
|
|
setup(conf);
|
|
|
runTest(length, killPos, dnIndex, false);
|
|
|
} catch (Throwable e) {
|
|
@@ -334,6 +362,8 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
|
|
|
long firstGS = -1; // first GS of this block group which never proceeds blockRecovery
|
|
|
long oldGS = -1; // the old GS before bumping
|
|
|
+ List<Long> gsList = new ArrayList<>();
|
|
|
+ final List<DatanodeInfo> killedDN = new ArrayList<>();
|
|
|
int numKilled=0;
|
|
|
for(; pos.get() < length; ) {
|
|
|
final int i = pos.getAndIncrement();
|
|
@@ -353,7 +383,7 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
waitTokenExpires(out);
|
|
|
}
|
|
|
|
|
|
- killDatanode(cluster, stripedOut, dnIndex[numKilled], pos);
|
|
|
+ killedDN.add(killDatanode(cluster, stripedOut, dnIndex[numKilled], pos));
|
|
|
numKilled++;
|
|
|
}
|
|
|
|
|
@@ -363,20 +393,18 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
firstGS = getGenerationStamp(stripedOut);
|
|
|
oldGS = firstGS;
|
|
|
}
|
|
|
+ if (i > 0 && (i + 1) % BLOCK_GROUP_SIZE == 0) {
|
|
|
+ gsList.add(oldGS);
|
|
|
+ }
|
|
|
}
|
|
|
+ gsList.add(oldGS);
|
|
|
out.close();
|
|
|
assertEquals(dnIndex.length, numKilled);
|
|
|
|
|
|
- short expectedReported = StripedFileTestUtil.getRealTotalBlockNum(length);
|
|
|
- for(int idx :dnIndex) {
|
|
|
- if (length > idx * CELL_SIZE || idx >= NUM_DATA_BLOCKS) {
|
|
|
- expectedReported--;
|
|
|
- }
|
|
|
- }
|
|
|
- DFSTestUtil.waitReplication(dfs, p, expectedReported);
|
|
|
+ StripedFileTestUtil.waitBlockGroupsReported(dfs, fullPath, numKilled);
|
|
|
|
|
|
cluster.triggerBlockReports();
|
|
|
- StripedFileTestUtil.checkData(dfs, p, length, dnIndex, oldGS);
|
|
|
+ StripedFileTestUtil.checkData(dfs, p, length, killedDN, gsList);
|
|
|
}
|
|
|
|
|
|
static void write(FSDataOutputStream out, int i) throws IOException {
|
|
@@ -389,8 +417,7 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
|
|
|
static long getGenerationStamp(DFSStripedOutputStream out)
|
|
|
throws IOException {
|
|
|
- DFSTestUtil.flushBuffer(out);
|
|
|
- final long gs = DFSTestUtil.flushInternal(out).getGenerationStamp();
|
|
|
+ final long gs = out.getBlock().getGenerationStamp();
|
|
|
LOG.info("getGenerationStamp returns " + gs);
|
|
|
return gs;
|
|
|
}
|
|
@@ -421,12 +448,15 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- static void killDatanode(MiniDFSCluster cluster, DFSStripedOutputStream out,
|
|
|
- final int dnIndex, final AtomicInteger pos) {
|
|
|
+ static DatanodeInfo killDatanode(MiniDFSCluster cluster,
|
|
|
+ DFSStripedOutputStream out, final int dnIndex, final AtomicInteger pos) {
|
|
|
final StripedDataStreamer s = out.getStripedDataStreamer(dnIndex);
|
|
|
final DatanodeInfo datanode = getDatanodes(s);
|
|
|
LOG.info("killDatanode " + dnIndex + ": " + datanode + ", pos=" + pos);
|
|
|
- cluster.stopDataNode(datanode.getXferAddr());
|
|
|
+ if (datanode != null) {
|
|
|
+ cluster.stopDataNode(datanode.getXferAddr());
|
|
|
+ }
|
|
|
+ return datanode;
|
|
|
}
|
|
|
|
|
|
|