|
@@ -18,6 +18,7 @@
|
|
|
|
|
|
package org.apache.hadoop.hdfs.security.token.block;
|
|
package org.apache.hadoop.hdfs.security.token.block;
|
|
|
|
|
|
|
|
+import com.google.common.base.Charsets;
|
|
import java.io.ByteArrayInputStream;
|
|
import java.io.ByteArrayInputStream;
|
|
import java.io.DataInputStream;
|
|
import java.io.DataInputStream;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
@@ -34,6 +35,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
|
|
import org.apache.hadoop.io.WritableUtils;
|
|
import org.apache.hadoop.io.WritableUtils;
|
|
|
|
+import org.apache.hadoop.ipc.Server;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.security.token.SecretManager;
|
|
import org.apache.hadoop.security.token.SecretManager;
|
|
import org.apache.hadoop.security.token.Token;
|
|
import org.apache.hadoop.security.token.Token;
|
|
@@ -75,6 +77,7 @@ public class BlockTokenSecretManager extends
|
|
|
|
|
|
private final int intRange;
|
|
private final int intRange;
|
|
private final int nnRangeStart;
|
|
private final int nnRangeStart;
|
|
|
|
+ private final boolean shouldWrapQOP;
|
|
|
|
|
|
private final SecureRandom nonceGenerator = new SecureRandom();
|
|
private final SecureRandom nonceGenerator = new SecureRandom();
|
|
|
|
|
|
@@ -92,7 +95,7 @@ public class BlockTokenSecretManager extends
|
|
public BlockTokenSecretManager(long keyUpdateInterval,
|
|
public BlockTokenSecretManager(long keyUpdateInterval,
|
|
long tokenLifetime, String blockPoolId, String encryptionAlgorithm) {
|
|
long tokenLifetime, String blockPoolId, String encryptionAlgorithm) {
|
|
this(false, keyUpdateInterval, tokenLifetime, blockPoolId,
|
|
this(false, keyUpdateInterval, tokenLifetime, blockPoolId,
|
|
- encryptionAlgorithm, 0, 1);
|
|
|
|
|
|
+ encryptionAlgorithm, 0, 1, false);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -104,19 +107,29 @@ public class BlockTokenSecretManager extends
|
|
* @param blockPoolId block pool ID
|
|
* @param blockPoolId block pool ID
|
|
* @param encryptionAlgorithm encryption algorithm to use
|
|
* @param encryptionAlgorithm encryption algorithm to use
|
|
* @param numNNs number of namenodes possible
|
|
* @param numNNs number of namenodes possible
|
|
|
|
+ * @param shouldWrapQOP should wrap QOP in the block access token
|
|
*/
|
|
*/
|
|
public BlockTokenSecretManager(long keyUpdateInterval,
|
|
public BlockTokenSecretManager(long keyUpdateInterval,
|
|
long tokenLifetime, int nnIndex, int numNNs, String blockPoolId,
|
|
long tokenLifetime, int nnIndex, int numNNs, String blockPoolId,
|
|
- String encryptionAlgorithm) {
|
|
|
|
- this(true, keyUpdateInterval, tokenLifetime, blockPoolId, encryptionAlgorithm, nnIndex, numNNs);
|
|
|
|
|
|
+ String encryptionAlgorithm, boolean shouldWrapQOP) {
|
|
|
|
+ this(true, keyUpdateInterval, tokenLifetime, blockPoolId,
|
|
|
|
+ encryptionAlgorithm, nnIndex, numNNs, shouldWrapQOP);
|
|
Preconditions.checkArgument(nnIndex >= 0);
|
|
Preconditions.checkArgument(nnIndex >= 0);
|
|
Preconditions.checkArgument(numNNs > 0);
|
|
Preconditions.checkArgument(numNNs > 0);
|
|
setSerialNo(new SecureRandom().nextInt());
|
|
setSerialNo(new SecureRandom().nextInt());
|
|
generateKeys();
|
|
generateKeys();
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ public BlockTokenSecretManager(long keyUpdateInterval,
|
|
|
|
+ long tokenLifetime, int nnIndex, int numNNs, String blockPoolId,
|
|
|
|
+ String encryptionAlgorithm) {
|
|
|
|
+ this(keyUpdateInterval, tokenLifetime, nnIndex, numNNs, blockPoolId,
|
|
|
|
+ encryptionAlgorithm, false);
|
|
|
|
+ }
|
|
|
|
+
|
|
private BlockTokenSecretManager(boolean isMaster, long keyUpdateInterval,
|
|
private BlockTokenSecretManager(boolean isMaster, long keyUpdateInterval,
|
|
- long tokenLifetime, String blockPoolId, String encryptionAlgorithm, int nnIndex, int numNNs) {
|
|
|
|
|
|
+ long tokenLifetime, String blockPoolId, String encryptionAlgorithm,
|
|
|
|
+ int nnIndex, int numNNs, boolean shouldWrapQOP) {
|
|
this.intRange = Integer.MAX_VALUE / numNNs;
|
|
this.intRange = Integer.MAX_VALUE / numNNs;
|
|
this.nnRangeStart = intRange * nnIndex;
|
|
this.nnRangeStart = intRange * nnIndex;
|
|
this.isMaster = isMaster;
|
|
this.isMaster = isMaster;
|
|
@@ -125,6 +138,7 @@ public class BlockTokenSecretManager extends
|
|
this.allKeys = new HashMap<Integer, BlockKey>();
|
|
this.allKeys = new HashMap<Integer, BlockKey>();
|
|
this.blockPoolId = blockPoolId;
|
|
this.blockPoolId = blockPoolId;
|
|
this.encryptionAlgorithm = encryptionAlgorithm;
|
|
this.encryptionAlgorithm = encryptionAlgorithm;
|
|
|
|
+ this.shouldWrapQOP = shouldWrapQOP;
|
|
this.timer = new Timer();
|
|
this.timer = new Timer();
|
|
generateKeys();
|
|
generateKeys();
|
|
}
|
|
}
|
|
@@ -253,6 +267,12 @@ public class BlockTokenSecretManager extends
|
|
ExtendedBlock block, EnumSet<BlockTokenIdentifier.AccessMode> modes) throws IOException {
|
|
ExtendedBlock block, EnumSet<BlockTokenIdentifier.AccessMode> modes) throws IOException {
|
|
BlockTokenIdentifier id = new BlockTokenIdentifier(userId, block
|
|
BlockTokenIdentifier id = new BlockTokenIdentifier(userId, block
|
|
.getBlockPoolId(), block.getBlockId(), modes);
|
|
.getBlockPoolId(), block.getBlockId(), modes);
|
|
|
|
+ if (shouldWrapQOP) {
|
|
|
|
+ String qop = Server.getEstablishedQOP();
|
|
|
|
+ if (qop != null) {
|
|
|
|
+ id.setHandshakeMsg(qop.getBytes(Charsets.UTF_8));
|
|
|
|
+ }
|
|
|
|
+ }
|
|
return new Token<BlockTokenIdentifier>(id, this);
|
|
return new Token<BlockTokenIdentifier>(id, this);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -431,18 +451,6 @@ public class BlockTokenSecretManager extends
|
|
return createPassword(nonce, key.getKey());
|
|
return createPassword(nonce, key.getKey());
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Encrypt the given message with the current block key, using the current
|
|
|
|
- * block key.
|
|
|
|
- *
|
|
|
|
- * @param message the message to be encrypted.
|
|
|
|
- * @return the secret created by encrypting the given message.
|
|
|
|
- */
|
|
|
|
- public byte[] secretGen(byte[] message) {
|
|
|
|
- return createPassword(message, currentKey.getKey());
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @VisibleForTesting
|
|
|
|
public BlockKey getCurrentKey() {
|
|
public BlockKey getCurrentKey() {
|
|
return currentKey;
|
|
return currentKey;
|
|
}
|
|
}
|