|
@@ -17,6 +17,8 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs;
|
|
|
|
|
|
+import static org.junit.Assert.assertEquals;
|
|
|
+
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
@@ -33,6 +35,7 @@ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
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.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
|
|
@@ -40,6 +43,7 @@ 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.datanode.DataNode;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
@@ -145,6 +149,86 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test(timeout = 90000)
|
|
|
+ public void testAddBlockWhenNoSufficientDataBlockNumOfNodes()
|
|
|
+ throws IOException {
|
|
|
+ HdfsConfiguration conf = new HdfsConfiguration();
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
|
|
+ try {
|
|
|
+ setup(conf);
|
|
|
+ ArrayList<DataNode> dataNodes = cluster.getDataNodes();
|
|
|
+ // shutdown few datanodes to avoid getting sufficient data blocks number
|
|
|
+ // of datanodes
|
|
|
+ int killDns = dataNodes.size() / 2;
|
|
|
+ int numDatanodes = dataNodes.size() - killDns;
|
|
|
+ for (int i = 0; i < killDns; i++) {
|
|
|
+ cluster.stopDataNode(i);
|
|
|
+ }
|
|
|
+ cluster.restartNameNodes();
|
|
|
+ cluster.triggerHeartbeats();
|
|
|
+ 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;
|
|
|
+ try {
|
|
|
+ out = dfs.create(dirFile, true);
|
|
|
+ out.write("something".getBytes());
|
|
|
+ out.flush();
|
|
|
+ out.close();
|
|
|
+ Assert.fail("Failed to validate available dns against blkGroupSize");
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ // expected
|
|
|
+ GenericTestUtils.assertExceptionContains("Failed: the number of "
|
|
|
+ + "remaining blocks = 5 < the number of data blocks = 6", ioe);
|
|
|
+ DFSStripedOutputStream dfsout = (DFSStripedOutputStream) out
|
|
|
+ .getWrappedStream();
|
|
|
+
|
|
|
+ // get leading streamer and verify the last exception
|
|
|
+ StripedDataStreamer datastreamer = dfsout.getStripedDataStreamer(0);
|
|
|
+ try {
|
|
|
+ datastreamer.getLastException().check(true);
|
|
|
+ Assert.fail("Failed to validate available dns against blkGroupSize");
|
|
|
+ } catch (IOException le) {
|
|
|
+ GenericTestUtils.assertExceptionContains(
|
|
|
+ "Failed to get datablocks number of nodes from"
|
|
|
+ + " namenode: blockGroupSize= 9, blocks.length= "
|
|
|
+ + numDatanodes, le);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ tearDown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test(timeout = 90000)
|
|
|
+ public void testAddBlockWhenNoSufficientParityNumOfNodes() throws IOException {
|
|
|
+ HdfsConfiguration conf = new HdfsConfiguration();
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
|
|
+ try {
|
|
|
+ setup(conf);
|
|
|
+ ArrayList<DataNode> dataNodes = cluster.getDataNodes();
|
|
|
+ // shutdown few data nodes to avoid writing parity blocks
|
|
|
+ int killDns = (NUM_PARITY_BLOCKS - 1);
|
|
|
+ int numDatanodes = dataNodes.size() - killDns;
|
|
|
+ for (int i = 0; i < killDns; i++) {
|
|
|
+ cluster.stopDataNode(i);
|
|
|
+ }
|
|
|
+ cluster.restartNameNodes();
|
|
|
+ cluster.triggerHeartbeats();
|
|
|
+ DatanodeInfo[] info = dfs.getClient().datanodeReport(
|
|
|
+ DatanodeReportType.LIVE);
|
|
|
+ assertEquals("Mismatches number of live Dns ", numDatanodes, info.length);
|
|
|
+ Path srcPath = new Path(dir, "testAddBlockWhenNoSufficientParityNodes");
|
|
|
+ int fileLength = HdfsConstants.BLOCK_STRIPED_CELL_SIZE - 1000;
|
|
|
+ final byte[] expected = StripedFileTestUtil.generateBytes(fileLength);
|
|
|
+ DFSTestUtil.writeFile(dfs, srcPath, new String(expected));
|
|
|
+ StripedFileTestUtil.verifySeek(dfs, srcPath, fileLength);
|
|
|
+ } finally {
|
|
|
+ tearDown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private void runTest(final Path p, final int length, final int killPos,
|
|
|
final int dnIndex, final boolean tokenExpire) throws Exception {
|
|
|
LOG.info("p=" + p + ", length=" + length + ", killPos=" + killPos
|