|
@@ -18,8 +18,11 @@
|
|
|
package org.apache.hadoop.hdfs.protocol.datatransfer.sasl;
|
|
|
|
|
|
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_ENCRYPT_DATA_OVERWRITE_DOWNSTREAM_NEW_QOP_KEY;
|
|
|
import static org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil.*;
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
+import java.io.ByteArrayInputStream;
|
|
|
import java.io.DataInputStream;
|
|
|
import java.io.DataOutputStream;
|
|
|
import java.io.IOException;
|
|
@@ -31,6 +34,7 @@ import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
|
|
|
+import javax.crypto.SecretKey;
|
|
|
import javax.security.auth.callback.Callback;
|
|
|
import javax.security.auth.callback.CallbackHandler;
|
|
|
import javax.security.auth.callback.NameCallback;
|
|
@@ -39,6 +43,7 @@ import javax.security.auth.callback.UnsupportedCallbackException;
|
|
|
import javax.security.sasl.RealmCallback;
|
|
|
import javax.security.sasl.RealmChoiceCallback;
|
|
|
|
|
|
+import javax.security.sasl.Sasl;
|
|
|
import org.apache.commons.codec.binary.Base64;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -54,6 +59,7 @@ import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
|
|
|
import org.apache.hadoop.security.SaslPropertiesResolver;
|
|
|
import org.apache.hadoop.security.SecurityUtil;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
+import org.apache.hadoop.security.token.SecretManager;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -83,6 +89,10 @@ public class SaslDataTransferClient {
|
|
|
private final SaslPropertiesResolver saslPropsResolver;
|
|
|
private final TrustedChannelResolver trustedChannelResolver;
|
|
|
|
|
|
+ // Store the most recent successfully negotiated QOP,
|
|
|
+ // for testing purpose only
|
|
|
+ private String targetQOP;
|
|
|
+
|
|
|
/**
|
|
|
* Creates a new SaslDataTransferClient. This constructor is used in cases
|
|
|
* where it is not relevant to track if a secure client did a fallback to
|
|
@@ -140,7 +150,7 @@ public class SaslDataTransferClient {
|
|
|
DataEncryptionKey encryptionKey = !trustedChannelResolver.isTrusted() ?
|
|
|
encryptionKeyFactory.newDataEncryptionKey() : null;
|
|
|
IOStreamPair ios = send(socket.getInetAddress(), underlyingOut,
|
|
|
- underlyingIn, encryptionKey, accessToken, datanodeId);
|
|
|
+ underlyingIn, encryptionKey, accessToken, datanodeId, null);
|
|
|
return ios != null ? ios : new IOStreamPair(underlyingIn, underlyingOut);
|
|
|
}
|
|
|
|
|
@@ -180,8 +190,19 @@ public class SaslDataTransferClient {
|
|
|
InputStream underlyingIn, DataEncryptionKeyFactory encryptionKeyFactory,
|
|
|
Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
|
|
|
throws IOException {
|
|
|
+ return socketSend(socket, underlyingOut, underlyingIn, encryptionKeyFactory,
|
|
|
+ accessToken, datanodeId, null);
|
|
|
+ }
|
|
|
+
|
|
|
+ public IOStreamPair socketSend(
|
|
|
+ Socket socket, OutputStream underlyingOut, InputStream underlyingIn,
|
|
|
+ DataEncryptionKeyFactory encryptionKeyFactory,
|
|
|
+ Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId,
|
|
|
+ SecretKey secretKey)
|
|
|
+ throws IOException {
|
|
|
IOStreamPair ios = checkTrustAndSend(socket.getInetAddress(), underlyingOut,
|
|
|
- underlyingIn, encryptionKeyFactory, accessToken, datanodeId);
|
|
|
+ underlyingIn, encryptionKeyFactory, accessToken, datanodeId,
|
|
|
+ secretKey);
|
|
|
return ios != null ? ios : new IOStreamPair(underlyingIn, underlyingOut);
|
|
|
}
|
|
|
|
|
@@ -203,17 +224,26 @@ public class SaslDataTransferClient {
|
|
|
DataEncryptionKeyFactory encryptionKeyFactory,
|
|
|
Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
|
|
|
throws IOException {
|
|
|
+ return checkTrustAndSend(addr, underlyingOut, underlyingIn,
|
|
|
+ encryptionKeyFactory, accessToken, datanodeId, null);
|
|
|
+ }
|
|
|
+
|
|
|
+ private IOStreamPair checkTrustAndSend(
|
|
|
+ InetAddress addr, OutputStream underlyingOut, InputStream underlyingIn,
|
|
|
+ DataEncryptionKeyFactory encryptionKeyFactory,
|
|
|
+ Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId,
|
|
|
+ SecretKey secretKey)
|
|
|
+ throws IOException {
|
|
|
boolean localTrusted = trustedChannelResolver.isTrusted();
|
|
|
boolean remoteTrusted = trustedChannelResolver.isTrusted(addr);
|
|
|
- LOG.debug("SASL encryption trust check: localHostTrusted = {}, "
|
|
|
+ LOG.info("SASL encryption trust check: localHostTrusted = {}, "
|
|
|
+ "remoteHostTrusted = {}", localTrusted, remoteTrusted);
|
|
|
-
|
|
|
if (!localTrusted || !remoteTrusted) {
|
|
|
// The encryption key factory only returns a key if encryption is enabled.
|
|
|
- DataEncryptionKey encryptionKey = encryptionKeyFactory
|
|
|
- .newDataEncryptionKey();
|
|
|
+ DataEncryptionKey encryptionKey =
|
|
|
+ encryptionKeyFactory.newDataEncryptionKey();
|
|
|
return send(addr, underlyingOut, underlyingIn, encryptionKey, accessToken,
|
|
|
- datanodeId);
|
|
|
+ datanodeId, secretKey);
|
|
|
} else {
|
|
|
LOG.debug(
|
|
|
"SASL client skipping handshake on trusted connection for addr = {}, "
|
|
@@ -237,13 +267,14 @@ public class SaslDataTransferClient {
|
|
|
*/
|
|
|
private IOStreamPair send(InetAddress addr, OutputStream underlyingOut,
|
|
|
InputStream underlyingIn, DataEncryptionKey encryptionKey,
|
|
|
- Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
|
|
|
+ Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId,
|
|
|
+ SecretKey secretKey)
|
|
|
throws IOException {
|
|
|
if (encryptionKey != null) {
|
|
|
LOG.debug("SASL client doing encrypted handshake for addr = {}, "
|
|
|
+ "datanodeId = {}", addr, datanodeId);
|
|
|
return getEncryptedStreams(addr, underlyingOut, underlyingIn,
|
|
|
- encryptionKey);
|
|
|
+ encryptionKey, accessToken, secretKey);
|
|
|
} else if (!UserGroupInformation.isSecurityEnabled()) {
|
|
|
LOG.debug("SASL client skipping handshake in unsecured configuration for "
|
|
|
+ "addr = {}, datanodeId = {}", addr, datanodeId);
|
|
@@ -264,7 +295,8 @@ public class SaslDataTransferClient {
|
|
|
LOG.debug(
|
|
|
"SASL client doing general handshake for addr = {}, datanodeId = {}",
|
|
|
addr, datanodeId);
|
|
|
- return getSaslStreams(addr, underlyingOut, underlyingIn, accessToken);
|
|
|
+ return getSaslStreams(addr, underlyingOut, underlyingIn,
|
|
|
+ accessToken, secretKey);
|
|
|
} else {
|
|
|
// It's a secured cluster using non-privileged ports, but no SASL. The
|
|
|
// only way this can happen is if the DataNode has
|
|
@@ -287,11 +319,20 @@ public class SaslDataTransferClient {
|
|
|
* @throws IOException for any error
|
|
|
*/
|
|
|
private IOStreamPair getEncryptedStreams(InetAddress addr,
|
|
|
- OutputStream underlyingOut,
|
|
|
- InputStream underlyingIn, DataEncryptionKey encryptionKey)
|
|
|
+ OutputStream underlyingOut, InputStream underlyingIn,
|
|
|
+ DataEncryptionKey encryptionKey,
|
|
|
+ Token<BlockTokenIdentifier> accessToken,
|
|
|
+ SecretKey secretKey)
|
|
|
throws IOException {
|
|
|
Map<String, String> saslProps = createSaslPropertiesForEncryption(
|
|
|
encryptionKey.encryptionAlgorithm);
|
|
|
+ if (secretKey != null) {
|
|
|
+ LOG.debug("DataNode overwriting downstream QOP" +
|
|
|
+ saslProps.get(Sasl.QOP));
|
|
|
+ byte[] newSecret = SecretManager.createPassword(saslProps.get(Sasl.QOP)
|
|
|
+ .getBytes(Charsets.UTF_8), secretKey);
|
|
|
+ accessToken.setDNHandshakeSecret(newSecret);
|
|
|
+ }
|
|
|
|
|
|
LOG.debug("Client using encryption algorithm {}",
|
|
|
encryptionKey.encryptionAlgorithm);
|
|
@@ -301,7 +342,7 @@ public class SaslDataTransferClient {
|
|
|
CallbackHandler callbackHandler = new SaslClientCallbackHandler(userName,
|
|
|
password);
|
|
|
return doSaslHandshake(addr, underlyingOut, underlyingIn, userName,
|
|
|
- saslProps, callbackHandler);
|
|
|
+ saslProps, callbackHandler, accessToken);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -370,6 +411,11 @@ public class SaslDataTransferClient {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ public String getTargetQOP() {
|
|
|
+ return targetQOP;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Sends client SASL negotiation for general-purpose handshake.
|
|
|
*
|
|
@@ -382,16 +428,36 @@ public class SaslDataTransferClient {
|
|
|
*/
|
|
|
private IOStreamPair getSaslStreams(InetAddress addr,
|
|
|
OutputStream underlyingOut, InputStream underlyingIn,
|
|
|
- Token<BlockTokenIdentifier> accessToken)
|
|
|
+ Token<BlockTokenIdentifier> accessToken,
|
|
|
+ SecretKey secretKey)
|
|
|
throws IOException {
|
|
|
Map<String, String> saslProps = saslPropsResolver.getClientProperties(addr);
|
|
|
|
|
|
+ // secretKey != null only happens when this is called by DN
|
|
|
+ // sending to downstream DN. If called from client, this will be null,
|
|
|
+ // as there is no key for client to generate mac instance.
|
|
|
+ // So that, if a different QOP is desired for inter-DN communication,
|
|
|
+ // the check below will use new QOP to create a secret, which includes
|
|
|
+ // the new QOP.
|
|
|
+ if (secretKey != null) {
|
|
|
+ String newQOP = conf
|
|
|
+ .get(DFS_ENCRYPT_DATA_OVERWRITE_DOWNSTREAM_NEW_QOP_KEY);
|
|
|
+ if (newQOP != null) {
|
|
|
+ saslProps.put(Sasl.QOP, newQOP);
|
|
|
+ }
|
|
|
+ LOG.debug("DataNode overwriting downstream QOP " +
|
|
|
+ saslProps.get(Sasl.QOP));
|
|
|
+ byte[] newSecret = SecretManager.createPassword(
|
|
|
+ saslProps.get(Sasl.QOP).getBytes(Charsets.UTF_8), secretKey);
|
|
|
+ accessToken.setDNHandshakeSecret(newSecret);
|
|
|
+ }
|
|
|
+ targetQOP = saslProps.get(Sasl.QOP);
|
|
|
String userName = buildUserName(accessToken);
|
|
|
char[] password = buildClientPassword(accessToken);
|
|
|
CallbackHandler callbackHandler = new SaslClientCallbackHandler(userName,
|
|
|
password);
|
|
|
return doSaslHandshake(addr, underlyingOut, underlyingIn, userName,
|
|
|
- saslProps, callbackHandler);
|
|
|
+ saslProps, callbackHandler, accessToken);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -435,8 +501,8 @@ public class SaslDataTransferClient {
|
|
|
*/
|
|
|
private IOStreamPair doSaslHandshake(InetAddress addr,
|
|
|
OutputStream underlyingOut, InputStream underlyingIn, String userName,
|
|
|
- Map<String, String> saslProps,
|
|
|
- CallbackHandler callbackHandler) throws IOException {
|
|
|
+ Map<String, String> saslProps, CallbackHandler callbackHandler,
|
|
|
+ Token<BlockTokenIdentifier> accessToken) throws IOException {
|
|
|
|
|
|
DataOutputStream out = new DataOutputStream(underlyingOut);
|
|
|
DataInputStream in = new DataInputStream(underlyingIn);
|
|
@@ -449,7 +515,22 @@ public class SaslDataTransferClient {
|
|
|
|
|
|
try {
|
|
|
// Start of handshake - "initial response" in SASL terminology.
|
|
|
- sendSaslMessage(out, new byte[0]);
|
|
|
+ // The handshake secret can be null, this happens when client is running
|
|
|
+ // a new version but the cluster does not have this feature. In which case
|
|
|
+ // there will be no encrypted secret sent from NN.
|
|
|
+ byte[] handshakeSecret = accessToken.getDnHandshakeSecret();
|
|
|
+ if (handshakeSecret == null || handshakeSecret.length == 0) {
|
|
|
+ LOG.debug("Handshake secret is null, sending without "
|
|
|
+ + "handshake secret.");
|
|
|
+ sendSaslMessage(out, new byte[0]);
|
|
|
+ } else {
|
|
|
+ LOG.debug("Sending handshake secret.");
|
|
|
+ BlockTokenIdentifier identifier = new BlockTokenIdentifier();
|
|
|
+ identifier.readFields(new DataInputStream(
|
|
|
+ new ByteArrayInputStream(accessToken.getIdentifier())));
|
|
|
+ String bpid = identifier.getBlockPoolId();
|
|
|
+ sendSaslMessageHandshakeSecret(out, new byte[0], handshakeSecret, bpid);
|
|
|
+ }
|
|
|
|
|
|
// step 1
|
|
|
byte[] remoteResponse = readSaslMessage(in);
|