|
@@ -25,22 +25,27 @@ 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.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
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.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
|
|
|
+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.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.After;
|
|
|
import org.junit.Assert;
|
|
|
-import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
import com.google.common.base.Preconditions;
|
|
@@ -63,17 +68,13 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
private static final int FLUSH_POS
|
|
|
= 9*DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT + 1;
|
|
|
|
|
|
- private final HdfsConfiguration conf = new HdfsConfiguration();
|
|
|
private MiniDFSCluster cluster;
|
|
|
private DistributedFileSystem dfs;
|
|
|
private final Path dir = new Path("/"
|
|
|
+ TestDFSStripedOutputStreamWithFailure.class.getSimpleName());
|
|
|
|
|
|
-
|
|
|
- @Before
|
|
|
- public void setup() throws IOException {
|
|
|
+ private void setup(Configuration conf) throws IOException {
|
|
|
final int numDNs = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
|
|
|
- conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
|
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
|
|
|
cluster.waitActive();
|
|
|
dfs = cluster.getFileSystem();
|
|
@@ -81,8 +82,7 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
dfs.createErasureCodingZone(dir, null, 0);
|
|
|
}
|
|
|
|
|
|
- @After
|
|
|
- public void tearDown() {
|
|
|
+ private void tearDown() {
|
|
|
if (cluster != null) {
|
|
|
cluster.shutdown();
|
|
|
}
|
|
@@ -92,89 +92,76 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
return (byte)pos;
|
|
|
}
|
|
|
|
|
|
- @Test(timeout=120000)
|
|
|
- public void testDatanodeFailure0() {
|
|
|
- final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE);
|
|
|
- final int dn = 0;
|
|
|
- runTest("file" + dn, length, dn);
|
|
|
- }
|
|
|
-
|
|
|
- @Test(timeout=120000)
|
|
|
- public void testDatanodeFailure1() {
|
|
|
- final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE);
|
|
|
- final int dn = 1;
|
|
|
- runTest("file" + dn, length, dn);
|
|
|
- }
|
|
|
-
|
|
|
- @Test(timeout=120000)
|
|
|
- public void testDatanodeFailure2() {
|
|
|
- final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE);
|
|
|
- final int dn = 2;
|
|
|
- runTest("file" + dn, length, dn);
|
|
|
- }
|
|
|
-
|
|
|
- @Test(timeout=120000)
|
|
|
- public void testDatanodeFailure3() {
|
|
|
- final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE);
|
|
|
- final int dn = 3;
|
|
|
- runTest("file" + dn, length, dn);
|
|
|
- }
|
|
|
-
|
|
|
- @Test(timeout=120000)
|
|
|
- public void testDatanodeFailure4() {
|
|
|
- final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE);
|
|
|
- final int dn = 4;
|
|
|
- runTest("file" + dn, length, dn);
|
|
|
- }
|
|
|
-
|
|
|
- @Test(timeout=120000)
|
|
|
- public void testDatanodeFailure5() {
|
|
|
- final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE);
|
|
|
- final int dn = 5;
|
|
|
- runTest("file" + dn, length, dn);
|
|
|
- }
|
|
|
-
|
|
|
- @Test(timeout=120000)
|
|
|
- public void testDatanodeFailure6() {
|
|
|
- final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE);
|
|
|
- final int dn = 6;
|
|
|
- runTest("file" + dn, length, dn);
|
|
|
+ private void initConf(Configuration conf){
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
|
|
|
}
|
|
|
|
|
|
- @Test(timeout=120000)
|
|
|
- public void testDatanodeFailure7() {
|
|
|
- final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE);
|
|
|
- final int dn = 7;
|
|
|
- runTest("file" + dn, length, dn);
|
|
|
+ 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=120000)
|
|
|
- public void testDatanodeFailure8() {
|
|
|
+ @Test(timeout=240000)
|
|
|
+ public void testDatanodeFailure() throws Exception {
|
|
|
final int length = NUM_DATA_BLOCKS*(BLOCK_SIZE - CELL_SIZE);
|
|
|
- final int dn = 8;
|
|
|
- runTest("file" + dn, length, dn);
|
|
|
+ HdfsConfiguration conf = new HdfsConfiguration();
|
|
|
+ initConf(conf);
|
|
|
+ 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);
|
|
|
+ } catch (Exception e) {
|
|
|
+ LOG.error("failed, dn=" + dn + ", length=" + length);
|
|
|
+ throw e;
|
|
|
+ } finally {
|
|
|
+ tearDown();
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
- private void runTest(final String src, final int length, final int dnIndex) {
|
|
|
- try {
|
|
|
- cluster.startDataNodes(conf, 1, true, null, null);
|
|
|
- cluster.waitActive();
|
|
|
-
|
|
|
- runTest(new Path(dir, src), length, length/2, dnIndex);
|
|
|
- } catch(Exception e) {
|
|
|
- LOG.info("FAILED", e);
|
|
|
- Assert.fail(StringUtils.stringifyException(e));
|
|
|
+ @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);
|
|
|
+ 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);
|
|
|
+ } catch (Exception e) {
|
|
|
+ LOG.error("failed, dn=" + dn + ", length=" + length);
|
|
|
+ throw e;
|
|
|
+ } finally {
|
|
|
+ tearDown();
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
|
private void runTest(final Path p, final int length, final int killPos,
|
|
|
- final int dnIndex) throws Exception {
|
|
|
+ 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);
|
|
|
final String fullPath = p.toString();
|
|
|
|
|
|
+ final NameNode nn = cluster.getNameNode();
|
|
|
+ final BlockManager bm = nn.getNamesystem().getBlockManager();
|
|
|
+ final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager();
|
|
|
+
|
|
|
+ if (tokenExpire) {
|
|
|
+ // set a short token lifetime (1 second)
|
|
|
+ SecurityTestUtil.setBlockTokenLifetime(sm, 1000L);
|
|
|
+ }
|
|
|
+
|
|
|
final AtomicInteger pos = new AtomicInteger();
|
|
|
final FSDataOutputStream out = dfs.create(p);
|
|
|
final DFSStripedOutputStream stripedOut
|
|
@@ -189,6 +176,11 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
Assert.assertTrue(oldGS != -1);
|
|
|
Assert.assertEquals(oldGS, gs);
|
|
|
|
|
|
+ if (tokenExpire) {
|
|
|
+ DFSTestUtil.flushInternal(stripedOut);
|
|
|
+ waitTokenExpires(out);
|
|
|
+ }
|
|
|
+
|
|
|
killDatanode(cluster, stripedOut, dnIndex, pos);
|
|
|
killed = true;
|
|
|
}
|
|
@@ -348,4 +340,14 @@ public class TestDFSStripedOutputStreamWithFailure {
|
|
|
killedDnIndex - dataBlockBytes.length);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ private void waitTokenExpires(FSDataOutputStream out) throws IOException {
|
|
|
+ Token<BlockTokenIdentifier> token = DFSTestUtil.getBlockToken(out);
|
|
|
+ while (!SecurityTestUtil.isBlockTokenExpired(token)) {
|
|
|
+ try {
|
|
|
+ Thread.sleep(10);
|
|
|
+ } catch (InterruptedException ignored) {
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|