|
@@ -19,6 +19,7 @@
|
|
|
package org.apache.hadoop.hdfs.security.token.block;
|
|
|
|
|
|
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
|
|
|
+import static org.junit.Assert.assertArrayEquals;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertNotEquals;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
@@ -73,6 +74,7 @@ import org.apache.hadoop.security.SaslInputStream;
|
|
|
import org.apache.hadoop.security.SaslRpcClient;
|
|
|
import org.apache.hadoop.security.SaslRpcServer;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
+import org.apache.hadoop.security.token.SecretManager;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
@@ -88,6 +90,7 @@ import org.mockito.stubbing.Answer;
|
|
|
import com.google.protobuf.BlockingService;
|
|
|
import com.google.protobuf.RpcController;
|
|
|
import com.google.protobuf.ServiceException;
|
|
|
+import org.apache.hadoop.fs.StorageType;
|
|
|
|
|
|
/** Unit tests for block tokens */
|
|
|
public class TestBlockToken {
|
|
@@ -102,7 +105,9 @@ public class TestBlockToken {
|
|
|
GenericTestUtils.setLogLevel(SaslInputStream.LOG, Level.ALL);
|
|
|
}
|
|
|
|
|
|
- /** Directory where we can count our open file descriptors under Linux */
|
|
|
+ /**
|
|
|
+ * Directory where we can count our open file descriptors under Linux
|
|
|
+ */
|
|
|
static final File FD_DIR = new File("/proc/self/fd/");
|
|
|
|
|
|
final long blockKeyUpdateInterval = 10 * 60 * 1000; // 10 mins
|
|
@@ -124,7 +129,7 @@ public class TestBlockToken {
|
|
|
final BlockTokenIdentifier ident;
|
|
|
|
|
|
public GetLengthAnswer(BlockTokenSecretManager sm,
|
|
|
- BlockTokenIdentifier ident) {
|
|
|
+ BlockTokenIdentifier ident) {
|
|
|
this.sm = sm;
|
|
|
this.ident = ident;
|
|
|
}
|
|
@@ -145,7 +150,8 @@ public class TestBlockToken {
|
|
|
LOG.info("Got: " + id.toString());
|
|
|
assertTrue("Received BlockTokenIdentifier is wrong", ident.equals(id));
|
|
|
sm.checkAccess(id, null, PBHelperClient.convert(req.getBlock()),
|
|
|
- BlockTokenIdentifier.AccessMode.WRITE);
|
|
|
+ BlockTokenIdentifier.AccessMode.WRITE,
|
|
|
+ new StorageType[]{StorageType.DEFAULT});
|
|
|
result = id.getBlockId();
|
|
|
}
|
|
|
return GetReplicaVisibleLengthResponseProto.newBuilder()
|
|
@@ -154,10 +160,11 @@ public class TestBlockToken {
|
|
|
}
|
|
|
|
|
|
private BlockTokenIdentifier generateTokenId(BlockTokenSecretManager sm,
|
|
|
- ExtendedBlock block,
|
|
|
- EnumSet<BlockTokenIdentifier.AccessMode> accessModes)
|
|
|
- throws IOException {
|
|
|
- Token<BlockTokenIdentifier> token = sm.generateToken(block, accessModes);
|
|
|
+ ExtendedBlock block,
|
|
|
+ EnumSet<BlockTokenIdentifier.AccessMode> accessModes,
|
|
|
+ StorageType... storageTypes) throws IOException {
|
|
|
+ Token<BlockTokenIdentifier> token = sm.generateToken(block, accessModes,
|
|
|
+ storageTypes);
|
|
|
BlockTokenIdentifier id = sm.createIdentifier();
|
|
|
id.readFields(new DataInputStream(new ByteArrayInputStream(token
|
|
|
.getIdentifier())));
|
|
@@ -169,12 +176,31 @@ public class TestBlockToken {
|
|
|
BlockTokenSecretManager sm = new BlockTokenSecretManager(
|
|
|
blockKeyUpdateInterval, blockTokenLifetime, 0, 1, "fake-pool", null,
|
|
|
enableProtobuf);
|
|
|
+ TestWritable.testWritable(generateTokenId(sm, block3,
|
|
|
+ EnumSet.noneOf(BlockTokenIdentifier.AccessMode.class),
|
|
|
+ StorageType.DEFAULT));
|
|
|
+ TestWritable.testWritable(generateTokenId(sm, block3,
|
|
|
+ EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE),
|
|
|
+ StorageType.DEFAULT));
|
|
|
+ TestWritable.testWritable(generateTokenId(sm, block3,
|
|
|
+ EnumSet.allOf(BlockTokenIdentifier.AccessMode.class),
|
|
|
+ StorageType.DEFAULT));
|
|
|
TestWritable.testWritable(generateTokenId(sm, block1,
|
|
|
- EnumSet.allOf(BlockTokenIdentifier.AccessMode.class)));
|
|
|
+ EnumSet.allOf(BlockTokenIdentifier.AccessMode.class),
|
|
|
+ StorageType.DEFAULT));
|
|
|
TestWritable.testWritable(generateTokenId(sm, block2,
|
|
|
- EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE)));
|
|
|
+ EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE),
|
|
|
+ StorageType.DEFAULT));
|
|
|
+ TestWritable.testWritable(generateTokenId(sm, block3,
|
|
|
+ EnumSet.noneOf(BlockTokenIdentifier.AccessMode.class),
|
|
|
+ StorageType.DEFAULT));
|
|
|
+ // We must be backwards compatible when adding storageType
|
|
|
TestWritable.testWritable(generateTokenId(sm, block3,
|
|
|
- EnumSet.noneOf(BlockTokenIdentifier.AccessMode.class)));
|
|
|
+ EnumSet.noneOf(BlockTokenIdentifier.AccessMode.class),
|
|
|
+ (StorageType[]) null));
|
|
|
+ TestWritable.testWritable(generateTokenId(sm, block3,
|
|
|
+ EnumSet.noneOf(BlockTokenIdentifier.AccessMode.class),
|
|
|
+ StorageType.EMPTY_ARRAY));
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -187,29 +213,37 @@ public class TestBlockToken {
|
|
|
testWritable(true);
|
|
|
}
|
|
|
|
|
|
+ private static void checkAccess(BlockTokenSecretManager m,
|
|
|
+ Token<BlockTokenIdentifier> t, ExtendedBlock blk,
|
|
|
+ BlockTokenIdentifier.AccessMode mode) throws SecretManager.InvalidToken {
|
|
|
+ m.checkAccess(t, null, blk, mode, new StorageType[]{ StorageType.DEFAULT });
|
|
|
+ }
|
|
|
+
|
|
|
private void tokenGenerationAndVerification(BlockTokenSecretManager master,
|
|
|
- BlockTokenSecretManager slave) throws Exception {
|
|
|
+ BlockTokenSecretManager slave, StorageType... storageTypes)
|
|
|
+ throws Exception {
|
|
|
// single-mode tokens
|
|
|
for (BlockTokenIdentifier.AccessMode mode : BlockTokenIdentifier.AccessMode
|
|
|
.values()) {
|
|
|
// generated by master
|
|
|
Token<BlockTokenIdentifier> token1 = master.generateToken(block1,
|
|
|
- EnumSet.of(mode));
|
|
|
- master.checkAccess(token1, null, block1, mode);
|
|
|
- slave.checkAccess(token1, null, block1, mode);
|
|
|
+ EnumSet.of(mode), storageTypes);
|
|
|
+ checkAccess(master, token1, block1, mode);
|
|
|
+ checkAccess(slave, token1, block1, mode);
|
|
|
// generated by slave
|
|
|
Token<BlockTokenIdentifier> token2 = slave.generateToken(block2,
|
|
|
- EnumSet.of(mode));
|
|
|
- master.checkAccess(token2, null, block2, mode);
|
|
|
- slave.checkAccess(token2, null, block2, mode);
|
|
|
+ EnumSet.of(mode), storageTypes);
|
|
|
+ checkAccess(master, token2, block2, mode);
|
|
|
+ checkAccess(slave, token2, block2, mode);
|
|
|
}
|
|
|
// multi-mode tokens
|
|
|
Token<BlockTokenIdentifier> mtoken = master.generateToken(block3,
|
|
|
- EnumSet.allOf(BlockTokenIdentifier.AccessMode.class));
|
|
|
+ EnumSet.allOf(BlockTokenIdentifier.AccessMode.class),
|
|
|
+ storageTypes);
|
|
|
for (BlockTokenIdentifier.AccessMode mode : BlockTokenIdentifier.AccessMode
|
|
|
.values()) {
|
|
|
- master.checkAccess(mtoken, null, block3, mode);
|
|
|
- slave.checkAccess(mtoken, null, block3, mode);
|
|
|
+ checkAccess(master, mtoken, block3, mode);
|
|
|
+ checkAccess(slave, mtoken, block3, mode);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -224,13 +258,19 @@ public class TestBlockToken {
|
|
|
enableProtobuf);
|
|
|
ExportedBlockKeys keys = masterHandler.exportKeys();
|
|
|
slaveHandler.addKeys(keys);
|
|
|
- tokenGenerationAndVerification(masterHandler, slaveHandler);
|
|
|
+ tokenGenerationAndVerification(masterHandler, slaveHandler,
|
|
|
+ StorageType.DEFAULT);
|
|
|
+ tokenGenerationAndVerification(masterHandler, slaveHandler, null);
|
|
|
// key updating
|
|
|
masterHandler.updateKeys();
|
|
|
- tokenGenerationAndVerification(masterHandler, slaveHandler);
|
|
|
+ tokenGenerationAndVerification(masterHandler, slaveHandler,
|
|
|
+ StorageType.DEFAULT);
|
|
|
+ tokenGenerationAndVerification(masterHandler, slaveHandler, null);
|
|
|
keys = masterHandler.exportKeys();
|
|
|
slaveHandler.addKeys(keys);
|
|
|
- tokenGenerationAndVerification(masterHandler, slaveHandler);
|
|
|
+ tokenGenerationAndVerification(masterHandler, slaveHandler,
|
|
|
+ StorageType.DEFAULT);
|
|
|
+ tokenGenerationAndVerification(masterHandler, slaveHandler, null);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -274,7 +314,8 @@ public class TestBlockToken {
|
|
|
blockKeyUpdateInterval, blockTokenLifetime, 0, 1, "fake-pool", null,
|
|
|
enableProtobuf);
|
|
|
Token<BlockTokenIdentifier> token = sm.generateToken(block3,
|
|
|
- EnumSet.allOf(BlockTokenIdentifier.AccessMode.class));
|
|
|
+ EnumSet.allOf(BlockTokenIdentifier.AccessMode.class),
|
|
|
+ new StorageType[]{StorageType.DEFAULT});
|
|
|
|
|
|
final Server server = createMockDatanode(sm, token, conf);
|
|
|
|
|
@@ -323,7 +364,8 @@ public class TestBlockToken {
|
|
|
blockKeyUpdateInterval, blockTokenLifetime, 0, 1, "fake-pool", null,
|
|
|
enableProtobuf);
|
|
|
Token<BlockTokenIdentifier> token = sm.generateToken(block3,
|
|
|
- EnumSet.allOf(BlockTokenIdentifier.AccessMode.class));
|
|
|
+ EnumSet.allOf(BlockTokenIdentifier.AccessMode.class),
|
|
|
+ new StorageType[]{StorageType.DEFAULT});
|
|
|
|
|
|
final Server server = createMockDatanode(sm, token, conf);
|
|
|
server.start();
|
|
@@ -409,14 +451,19 @@ public class TestBlockToken {
|
|
|
|
|
|
ExportedBlockKeys keys = masterHandler.exportKeys();
|
|
|
bpMgr.addKeys(bpid, keys);
|
|
|
- tokenGenerationAndVerification(masterHandler, bpMgr.get(bpid));
|
|
|
-
|
|
|
+ tokenGenerationAndVerification(masterHandler, bpMgr.get(bpid),
|
|
|
+ StorageType.DEFAULT);
|
|
|
+ tokenGenerationAndVerification(masterHandler, bpMgr.get(bpid), null);
|
|
|
// Test key updating
|
|
|
masterHandler.updateKeys();
|
|
|
- tokenGenerationAndVerification(masterHandler, bpMgr.get(bpid));
|
|
|
+ tokenGenerationAndVerification(masterHandler, bpMgr.get(bpid),
|
|
|
+ StorageType.DEFAULT);
|
|
|
+ tokenGenerationAndVerification(masterHandler, bpMgr.get(bpid), null);
|
|
|
keys = masterHandler.exportKeys();
|
|
|
bpMgr.addKeys(bpid, keys);
|
|
|
- tokenGenerationAndVerification(masterHandler, bpMgr.get(bpid));
|
|
|
+ tokenGenerationAndVerification(masterHandler, bpMgr.get(bpid),
|
|
|
+ StorageType.DEFAULT);
|
|
|
+ tokenGenerationAndVerification(masterHandler, bpMgr.get(bpid), null);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -492,7 +539,8 @@ public class TestBlockToken {
|
|
|
blockKeyUpdateInterval, blockTokenLifetime, 0, 1, "fake-pool", null,
|
|
|
useProto);
|
|
|
Token<BlockTokenIdentifier> token = sm.generateToken(block1,
|
|
|
- EnumSet.noneOf(BlockTokenIdentifier.AccessMode.class));
|
|
|
+ EnumSet.noneOf(BlockTokenIdentifier.AccessMode.class),
|
|
|
+ new StorageType[]{StorageType.DEFAULT});
|
|
|
final byte[] tokenBytes = token.getIdentifier();
|
|
|
BlockTokenIdentifier legacyToken = new BlockTokenIdentifier();
|
|
|
BlockTokenIdentifier protobufToken = new BlockTokenIdentifier();
|
|
@@ -547,7 +595,6 @@ public class TestBlockToken {
|
|
|
|
|
|
dib.reset(emptyIdentBytes, emptyIdentBytes.length);
|
|
|
readToken.readFields(dib);
|
|
|
- assertTrue(invalidProtobufMessage);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -557,7 +604,8 @@ public class TestBlockToken {
|
|
|
blockKeyUpdateInterval, blockTokenLifetime, 0, 1, "fake-pool", null,
|
|
|
useProto);
|
|
|
Token<BlockTokenIdentifier> token = sm.generateToken(block1,
|
|
|
- EnumSet.noneOf(BlockTokenIdentifier.AccessMode.class));
|
|
|
+ EnumSet.noneOf(BlockTokenIdentifier.AccessMode.class),
|
|
|
+ StorageType.EMPTY_ARRAY);
|
|
|
final byte[] tokenBytes = token.getIdentifier();
|
|
|
BlockTokenIdentifier legacyToken = new BlockTokenIdentifier();
|
|
|
BlockTokenIdentifier protobufToken = new BlockTokenIdentifier();
|
|
@@ -594,7 +642,7 @@ public class TestBlockToken {
|
|
|
assertEquals(protobufToken, readToken);
|
|
|
}
|
|
|
|
|
|
- public void testCraftedProtobufBlockTokenIdentifier(
|
|
|
+ private void testCraftedProtobufBlockTokenIdentifier(
|
|
|
BlockTokenIdentifier identifier, boolean expectIOE,
|
|
|
boolean expectRTE) throws IOException {
|
|
|
DataOutputBuffer dob = new DataOutputBuffer(4096);
|
|
@@ -631,20 +679,27 @@ public class TestBlockToken {
|
|
|
dib.reset(identBytes, identBytes.length);
|
|
|
readToken.readFieldsProtobuf(dib);
|
|
|
assertEquals(protobufToken, readToken);
|
|
|
+ assertEquals(identifier, readToken);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testCraftedProtobufBlockTokenBytesIsProtobuf() throws
|
|
|
- IOException {
|
|
|
+ public void testEmptyProtobufBlockTokenBytesIsProtobuf() throws IOException {
|
|
|
// Empty BlockTokenIdentifiers throw IOException
|
|
|
BlockTokenIdentifier identifier = new BlockTokenIdentifier();
|
|
|
testCraftedProtobufBlockTokenIdentifier(identifier, true, false);
|
|
|
+ }
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testCraftedProtobufBlockTokenBytesIsProtobuf() throws
|
|
|
+ IOException {
|
|
|
/* Parsing BlockTokenIdentifier with expiryDate
|
|
|
* 2017-02-09 00:12:35,072+0100 will throw IOException.
|
|
|
* However, expiryDate of
|
|
|
* 2017-02-09 00:12:35,071+0100 will throw NegativeArraySizeException.
|
|
|
*/
|
|
|
+ BlockTokenIdentifier identifier = new BlockTokenIdentifier("user",
|
|
|
+ "blockpool", 123, EnumSet.allOf(BlockTokenIdentifier.AccessMode.class),
|
|
|
+ new StorageType[]{StorageType.DISK, StorageType.ARCHIVE}, true);
|
|
|
Calendar cal = new GregorianCalendar();
|
|
|
cal.set(2017, 1, 9, 0, 12, 35);
|
|
|
long datetime = cal.getTimeInMillis();
|
|
@@ -656,4 +711,61 @@ public class TestBlockToken {
|
|
|
identifier.setExpiryDate(datetime);
|
|
|
testCraftedProtobufBlockTokenIdentifier(identifier, true, false);
|
|
|
}
|
|
|
+
|
|
|
+ private BlockTokenIdentifier writeAndReadBlockToken(
|
|
|
+ BlockTokenIdentifier identifier) throws IOException {
|
|
|
+ DataOutputBuffer dob = new DataOutputBuffer(4096);
|
|
|
+ DataInputBuffer dib = new DataInputBuffer();
|
|
|
+ identifier.write(dob);
|
|
|
+ byte[] identBytes = Arrays.copyOf(dob.getData(), dob.getLength());
|
|
|
+
|
|
|
+ BlockTokenIdentifier readToken = new BlockTokenIdentifier();
|
|
|
+
|
|
|
+ dib.reset(identBytes, identBytes.length);
|
|
|
+ readToken.readFields(dib);
|
|
|
+ assertEquals(identifier, readToken);
|
|
|
+ return readToken;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testEmptyBlockTokenSerialization() throws IOException {
|
|
|
+ BlockTokenIdentifier ident = new BlockTokenIdentifier();
|
|
|
+ BlockTokenIdentifier ret = writeAndReadBlockToken(ident);
|
|
|
+ assertEquals(ret.getExpiryDate(), 0);
|
|
|
+ assertEquals(ret.getKeyId(), 0);
|
|
|
+ assertEquals(ret.getUserId(), null);
|
|
|
+ assertEquals(ret.getBlockPoolId(), null);
|
|
|
+ assertEquals(ret.getBlockId(), 0);
|
|
|
+ assertEquals(ret.getAccessModes(),
|
|
|
+ EnumSet.noneOf(BlockTokenIdentifier.AccessMode.class));
|
|
|
+ assertArrayEquals(ret.getStorageTypes(), StorageType.EMPTY_ARRAY);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void testBlockTokenSerialization(boolean useProto) throws
|
|
|
+ IOException {
|
|
|
+ EnumSet<BlockTokenIdentifier.AccessMode> accessModes =
|
|
|
+ EnumSet.allOf(BlockTokenIdentifier.AccessMode.class);
|
|
|
+ StorageType[] storageTypes =
|
|
|
+ new StorageType[]{StorageType.RAM_DISK, StorageType.SSD,
|
|
|
+ StorageType.DISK, StorageType.ARCHIVE};
|
|
|
+ BlockTokenIdentifier ident = new BlockTokenIdentifier("user", "bpool",
|
|
|
+ 123, accessModes, storageTypes, useProto);
|
|
|
+ ident.setExpiryDate(1487080345L);
|
|
|
+ BlockTokenIdentifier ret = writeAndReadBlockToken(ident);
|
|
|
+ assertEquals(ret.getExpiryDate(), 1487080345L);
|
|
|
+ assertEquals(ret.getKeyId(), 0);
|
|
|
+ assertEquals(ret.getUserId(), "user");
|
|
|
+ assertEquals(ret.getBlockPoolId(), "bpool");
|
|
|
+ assertEquals(ret.getBlockId(), 123);
|
|
|
+ assertEquals(ret.getAccessModes(),
|
|
|
+ EnumSet.allOf(BlockTokenIdentifier.AccessMode.class));
|
|
|
+ assertArrayEquals(ret.getStorageTypes(), storageTypes);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBlockTokenSerialization() throws IOException {
|
|
|
+ testBlockTokenSerialization(false);
|
|
|
+ testBlockTokenSerialization(true);
|
|
|
+ }
|
|
|
+
|
|
|
}
|