|
@@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
@@ -48,6 +49,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
+import org.apache.hadoop.util.StringUtils;
|
|
|
import org.apache.log4j.Level;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Test;
|
|
@@ -71,6 +73,38 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
|
|
|
private static final int FLUSH_POS
|
|
|
= 9*DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT + 1;
|
|
|
+ static {
|
|
|
+ System.out.println("NUM_DATA_BLOCKS = " + NUM_DATA_BLOCKS);
|
|
|
+ System.out.println("NUM_PARITY_BLOCKS= " + NUM_PARITY_BLOCKS);
|
|
|
+ System.out.println("CELL_SIZE = " + CELL_SIZE
|
|
|
+ + " (=" + StringUtils.TraditionalBinaryPrefix.long2String(CELL_SIZE, "B", 2) + ")");
|
|
|
+ System.out.println("BLOCK_SIZE = " + BLOCK_SIZE
|
|
|
+ + " (=" + StringUtils.TraditionalBinaryPrefix.long2String(BLOCK_SIZE, "B", 2) + ")");
|
|
|
+ System.out.println("BLOCK_GROUP_SIZE = " + BLOCK_GROUP_SIZE
|
|
|
+ + " (=" + StringUtils.TraditionalBinaryPrefix.long2String(BLOCK_GROUP_SIZE, "B", 2) + ")");
|
|
|
+ }
|
|
|
+
|
|
|
+ static List<Integer> newLengths() {
|
|
|
+ final List<Integer> lengths = new ArrayList<>();
|
|
|
+ lengths.add(FLUSH_POS + 2);
|
|
|
+ for(int b = 0; b <= 2; b++) {
|
|
|
+ for(int c = 0; c < STRIPES_PER_BLOCK*NUM_DATA_BLOCKS; c++) {
|
|
|
+ for(int delta = -1; delta <= 1; delta++) {
|
|
|
+ final int length = b*BLOCK_GROUP_SIZE + c*CELL_SIZE + delta;
|
|
|
+ System.out.println(lengths.size() + ": length=" + length
|
|
|
+ + ", (b, c, d) = (" + b + ", " + c + ", " + delta + ")");
|
|
|
+ lengths.add(length);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return lengths;
|
|
|
+ }
|
|
|
+
|
|
|
+ private static final List<Integer> LENGTHS = newLengths();
|
|
|
+
|
|
|
+ static int getLength(int i) {
|
|
|
+ return LENGTHS.get(i);
|
|
|
+ }
|
|
|
|
|
|
private MiniDFSCluster cluster;
|
|
|
private DistributedFileSystem dfs;
|
|
@@ -96,50 +130,49 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
return (byte)pos;
|
|
|
}
|
|
|
|
|
|
- private void initConf(Configuration conf){
|
|
|
+ private HdfsConfiguration newHdfsConfiguration() {
|
|
|
+ final HdfsConfiguration conf = new HdfsConfiguration();
|
|
|
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 6000L);
|
|
|
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
|
|
|
+ return conf;
|
|
|
}
|
|
|
|
|
|
- private void initConfWithBlockToken(Configuration conf) {
|
|
|
- conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
|
|
|
- conf.setInt("ipc.client.connect.max.retries", 0);
|
|
|
- // Set short retry timeouts so this test runs faster
|
|
|
- conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10);
|
|
|
- }
|
|
|
-
|
|
|
- @Test(timeout=240000)
|
|
|
- public void testDatanodeFailure() throws Exception {
|
|
|
- final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE);
|
|
|
- HdfsConfiguration conf = new HdfsConfiguration();
|
|
|
- initConf(conf);
|
|
|
+ void runTest(final int length) {
|
|
|
+ final HdfsConfiguration conf = newHdfsConfiguration();
|
|
|
for (int dn = 0; dn < 9; dn++) {
|
|
|
try {
|
|
|
setup(conf);
|
|
|
- cluster.startDataNodes(conf, 1, true, null, null);
|
|
|
- cluster.waitActive();
|
|
|
- runTest(new Path(dir, "file" + dn), length, length / 2, dn, false);
|
|
|
+ runTest(length, dn, false, conf);
|
|
|
} catch (Exception e) {
|
|
|
- LOG.error("failed, dn=" + dn + ", length=" + length);
|
|
|
- throw e;
|
|
|
+ final String err = "failed, dn=" + dn + ", length=" + length
|
|
|
+ + StringUtils.stringifyException(e);
|
|
|
+ LOG.error(err);
|
|
|
+ Assert.fail(err);
|
|
|
} finally {
|
|
|
tearDown();
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test(timeout=240000)
|
|
|
+ public void testDatanodeFailure56() throws Exception {
|
|
|
+ runTest(getLength(56));
|
|
|
+ }
|
|
|
+
|
|
|
@Test(timeout=240000)
|
|
|
public void testBlockTokenExpired() throws Exception {
|
|
|
final int length = NUM_DATA_BLOCKS * (BLOCK_SIZE - CELL_SIZE);
|
|
|
- HdfsConfiguration conf = new HdfsConfiguration();
|
|
|
- initConf(conf);
|
|
|
- initConfWithBlockToken(conf);
|
|
|
+ final HdfsConfiguration conf = newHdfsConfiguration();
|
|
|
+
|
|
|
+ conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
|
|
|
+ conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
|
|
|
+ // Set short retry timeouts so this test runs faster
|
|
|
+ conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10);
|
|
|
for (int dn = 0; dn < 9; dn += 2) {
|
|
|
try {
|
|
|
setup(conf);
|
|
|
- cluster.startDataNodes(conf, 1, true, null, null);
|
|
|
- cluster.waitActive();
|
|
|
- runTest(new Path(dir, "file" + dn), length, length / 2, dn, true);
|
|
|
+ runTest(length, dn, true, conf);
|
|
|
} catch (Exception e) {
|
|
|
LOG.error("failed, dn=" + dn + ", length=" + length);
|
|
|
throw e;
|
|
@@ -229,19 +262,41 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- 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
|
|
|
- + ", dnIndex=" + dnIndex);
|
|
|
- Preconditions.checkArgument(killPos < length);
|
|
|
- Preconditions.checkArgument(killPos > FLUSH_POS);
|
|
|
+ private void runTest(final int length, final int dnIndex,
|
|
|
+ final boolean tokenExpire, final HdfsConfiguration conf) {
|
|
|
+ try {
|
|
|
+ runTest(length, length/2, dnIndex, tokenExpire, conf);
|
|
|
+ } catch(Exception e) {
|
|
|
+ LOG.info("FAILED", e);
|
|
|
+ Assert.fail(StringUtils.stringifyException(e));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void runTest(final int length, final int killPos,
|
|
|
+ final int dnIndex, final boolean tokenExpire,
|
|
|
+ final HdfsConfiguration conf) throws Exception {
|
|
|
+ if (killPos <= FLUSH_POS) {
|
|
|
+ LOG.warn("killPos=" + killPos + " <= FLUSH_POS=" + FLUSH_POS
|
|
|
+ + ", length=" + length + ", dnIndex=" + dnIndex);
|
|
|
+ return; //skip test
|
|
|
+ }
|
|
|
+ Preconditions.checkArgument(length > killPos,
|
|
|
+ "length=%s <= killPos=%s", length, killPos);
|
|
|
+
|
|
|
+ // start a datanode now, will kill one later
|
|
|
+ cluster.startDataNodes(conf, 1, true, null, null);
|
|
|
+ cluster.waitActive();
|
|
|
+
|
|
|
+ final Path p = new Path(dir, "dn" + dnIndex + "len" + length + "kill" + killPos);
|
|
|
final String fullPath = p.toString();
|
|
|
+ LOG.info("fullPath=" + fullPath);
|
|
|
|
|
|
- final NameNode nn = cluster.getNameNode();
|
|
|
- final BlockManager bm = nn.getNamesystem().getBlockManager();
|
|
|
- final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager();
|
|
|
|
|
|
if (tokenExpire) {
|
|
|
+ final NameNode nn = cluster.getNameNode();
|
|
|
+ final BlockManager bm = nn.getNamesystem().getBlockManager();
|
|
|
+ final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager();
|
|
|
+
|
|
|
// set a short token lifetime (1 second)
|
|
|
SecurityTestUtil.setBlockTokenLifetime(sm, 1000L);
|
|
|
}
|
|
@@ -265,7 +320,7 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
waitTokenExpires(out);
|
|
|
}
|
|
|
|
|
|
- StripedFileTestUtil.killDatanode(cluster, stripedOut, dnIndex, pos);
|
|
|
+ killDatanode(cluster, stripedOut, dnIndex, pos);
|
|
|
killed = true;
|
|
|
}
|
|
|
|
|
@@ -301,6 +356,40 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
|
|
|
}
|
|
|
|
|
|
+ static DatanodeInfo getDatanodes(StripedDataStreamer streamer) {
|
|
|
+ for(;;) {
|
|
|
+ DatanodeInfo[] datanodes = streamer.getNodes();
|
|
|
+ if (datanodes == null) {
|
|
|
+ // try peeking following block.
|
|
|
+ final LocatedBlock lb = streamer.peekFollowingBlock();
|
|
|
+ if (lb != null) {
|
|
|
+ datanodes = lb.getLocations();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ if (datanodes != null) {
|
|
|
+ Assert.assertEquals(1, datanodes.length);
|
|
|
+ Assert.assertNotNull(datanodes[0]);
|
|
|
+ return datanodes[0];
|
|
|
+ }
|
|
|
+
|
|
|
+ try {
|
|
|
+ Thread.sleep(100);
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
+ Assert.fail(StringUtils.stringifyException(ie));
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ static void 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());
|
|
|
+ }
|
|
|
+
|
|
|
static void checkData(DistributedFileSystem dfs, String src, int length,
|
|
|
int killedDnIndex, long oldGS) throws IOException {
|
|
|
List<List<LocatedBlock>> blockGroupList = new ArrayList<>();
|
|
@@ -314,7 +403,7 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
final long gs = firstBlock.getBlock().getGenerationStamp();
|
|
|
final String s = "gs=" + gs + ", oldGS=" + oldGS;
|
|
|
LOG.info(s);
|
|
|
- Assert.assertTrue(s, gs > oldGS);
|
|
|
+ Assert.assertTrue(s, gs >= oldGS);
|
|
|
|
|
|
LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
|
|
|
(LocatedStripedBlock) firstBlock,
|
|
@@ -342,7 +431,7 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
final int numCellInBlock = (numCellInGroup - 1)/NUM_DATA_BLOCKS
|
|
|
+ (j <= lastCellIndex? 1: 0);
|
|
|
final int blockSize = numCellInBlock*CELL_SIZE
|
|
|
- + (isLastGroup && i == lastCellIndex? lastCellSize - CELL_SIZE: 0);
|
|
|
+ + (isLastGroup && j == lastCellIndex? lastCellSize - CELL_SIZE: 0);
|
|
|
|
|
|
final byte[] blockBytes = new byte[blockSize];
|
|
|
if (i < NUM_DATA_BLOCKS) {
|
|
@@ -352,7 +441,8 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
}
|
|
|
|
|
|
final LocatedBlock lb = blockList.get(i);
|
|
|
- LOG.info("XXX i=" + i + ", lb=" + lb);
|
|
|
+ LOG.info("i,j=" + i + ", " + j + ", numCellInBlock=" + numCellInBlock
|
|
|
+ + ", blockSize=" + blockSize + ", lb=" + lb);
|
|
|
if (lb == null) {
|
|
|
continue;
|
|
|
}
|
|
@@ -410,4 +500,35 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ public static abstract class TestBase {
|
|
|
+ static final long TIMEOUT = 240000;
|
|
|
+
|
|
|
+ int getBase() {
|
|
|
+ final String name = getClass().getSimpleName();
|
|
|
+ int i = name.length() - 1;
|
|
|
+ for(; i >= 0 && Character.isDigit(name.charAt(i)); i--);
|
|
|
+ return Integer.parseInt(name.substring(i + 1));
|
|
|
+ }
|
|
|
+
|
|
|
+ private final TestDFSStripedOutputStreamWithFailure test
|
|
|
+ = new TestDFSStripedOutputStreamWithFailure();
|
|
|
+ private void run(int offset) {
|
|
|
+ final int i = offset + getBase();
|
|
|
+ final int length = getLength(i);
|
|
|
+ System.out.println("Run test " + i + ", length=" + length);
|
|
|
+ test.runTest(length);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test(timeout=TIMEOUT) public void test0() {run(0);}
|
|
|
+ @Test(timeout=TIMEOUT) public void test1() {run(1);}
|
|
|
+ @Test(timeout=TIMEOUT) public void test2() {run(2);}
|
|
|
+ @Test(timeout=TIMEOUT) public void test3() {run(3);}
|
|
|
+ @Test(timeout=TIMEOUT) public void test4() {run(4);}
|
|
|
+ @Test(timeout=TIMEOUT) public void test5() {run(5);}
|
|
|
+ @Test(timeout=TIMEOUT) public void test6() {run(6);}
|
|
|
+ @Test(timeout=TIMEOUT) public void test7() {run(7);}
|
|
|
+ @Test(timeout=TIMEOUT) public void test8() {run(8);}
|
|
|
+ @Test(timeout=TIMEOUT) public void test9() {run(9);}
|
|
|
+ }
|
|
|
}
|