Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@

import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslProvider;
import org.bouncycastle.jce.provider.BouncyCastleProvider;
import org.slf4j.Logger;
Expand All @@ -32,6 +33,7 @@
import java.security.Provider;
import java.security.Security;
import java.time.Duration;
import java.util.concurrent.TimeUnit;

import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DEFAULT_KEY_ALGORITHM;
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DEFAULT_KEY_LEN;
Expand Down Expand Up @@ -459,4 +461,14 @@ private Provider initSecurityProvider(String providerName) {
throw new SecurityException("Unknown security provider:" + provider);
}
}

/**
* Returns max date for which S3 tokens will be valid.
* */
public long getS3TokenMaxDate() {
return getConfiguration().getTimeDuration(
OzoneConfigKeys.OZONE_S3_TOKEN_MAX_LIFETIME_KEY,
OzoneConfigKeys.OZONE_S3_TOKEN_MAX_LIFETIME_KEY_DEFAULT,
TimeUnit.MICROSECONDS);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -373,7 +373,9 @@ public final class OzoneConfigKeys {
"ozone.acl.enabled";
public static final boolean OZONE_ACL_ENABLED_DEFAULT =
false;

public static final String OZONE_S3_TOKEN_MAX_LIFETIME_KEY =
"ozone.s3.token.max.lifetime";
public static final String OZONE_S3_TOKEN_MAX_LIFETIME_KEY_DEFAULT = "3m";
//For technical reasons this is unused and hardcoded to the
// OzoneFileSystem.initialize.
public static final String OZONE_FS_ISOLATED_CLASSLOADER =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,4 +27,10 @@
public interface S3SecretManager {

S3SecretValue getS3Secret(String kerberosID) throws IOException;

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The two methods are confusing a little. Especially as the bigger part of the implementation is duplicated. Would be great to merge them (or use better naming). (Not a blocker, we can address it later).

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Renamed new api to getS3UserSecretString, open to any better name you may suggest. Purpose of both api's is different so consolidating them right not might not be a good option. We can discuss this further in separate jira.

/**
* API to get s3 secret for given awsAccessKey.
* @param awsAccessKey
* */
String getS3UserSecretString(String awsAccessKey) throws IOException;
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,12 +24,16 @@
import org.apache.hadoop.ozone.OmUtils;
import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
import org.apache.hadoop.ozone.security.OzoneSecurityException;
import org.apache.logging.log4j.util.Strings;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;

import static java.nio.charset.StandardCharsets.UTF_8;
import static org.apache.hadoop.ozone.security.OzoneSecurityException.ResultCodes.S3_SECRET_NOT_FOUND;

/**
* S3 Secret manager.
*/
Expand Down Expand Up @@ -58,7 +62,8 @@ public S3SecretManagerImpl(OzoneConfiguration configuration,
public S3SecretValue getS3Secret(String kerberosID) throws IOException {
Preconditions.checkArgument(Strings.isNotBlank(kerberosID),
"kerberosID cannot be null or empty.");
byte[] awsAccessKey = OmUtils.getMD5Digest(kerberosID);
String awsAccessKeyStr = DigestUtils.md5Hex(kerberosID);
byte[] awsAccessKey = awsAccessKeyStr.getBytes(UTF_8);
S3SecretValue result = null;
omMetadataManager.getLock().acquireS3SecretLock(kerberosID);
try {
Expand All @@ -73,10 +78,36 @@ public S3SecretValue getS3Secret(String kerberosID) throws IOException {
result = S3SecretValue.fromProtobuf(
OzoneManagerProtocolProtos.S3Secret.parseFrom(s3Secret));
}
result.setAwsAccessKey(DigestUtils.md5Hex(awsAccessKey));
result.setAwsAccessKey(awsAccessKeyStr);
} finally {
omMetadataManager.getLock().releaseS3SecretLock(kerberosID);
}
LOG.trace("Secret for kerberosID:{},accessKey:{}, proto:{}", kerberosID,
awsAccessKeyStr, result);
return result;
}

@Override
public String getS3UserSecretString(String awsAccessKeyId)
throws IOException {
Preconditions.checkArgument(Strings.isNotBlank(awsAccessKeyId),
"awsAccessKeyId cannot be null or empty.");
LOG.trace("Get secret for awsAccessKey:{}", awsAccessKeyId);

byte[] s3Secret;
omMetadataManager.getLock().acquireS3SecretLock(awsAccessKeyId);
try {
s3Secret = omMetadataManager.getS3SecretTable()
.get(awsAccessKeyId.getBytes(UTF_8));
if (s3Secret == null) {
throw new OzoneSecurityException("S3 secret not found for " +
"awsAccessKeyId " + awsAccessKeyId, S3_SECRET_NOT_FOUND);
}
} finally {
omMetadataManager.getLock().releaseS3SecretLock(awsAccessKeyId);
}

return OzoneManagerProtocolProtos.S3Secret.parseFrom(s3Secret)
.getAwsSecret();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -185,5 +185,6 @@ public enum ResultCodes {

INVALID_KMS_PROVIDER,

TOKEN_CREATION_ERROR
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,8 @@
package org.apache.hadoop.ozone.om.helpers;

import org.apache.commons.codec.digest.DigestUtils;
import org.apache.hadoop.ozone.OmUtils;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;

import java.io.IOException;

/**
* S3Secret to be saved in database.
*/
Expand All @@ -31,11 +28,10 @@ public class S3SecretValue {
private String awsSecret;
private String awsAccessKey;

public S3SecretValue(String kerberosID, String awsSecret) throws IOException {
public S3SecretValue(String kerberosID, String awsSecret) {
this.kerberosID = kerberosID;
this.awsSecret = awsSecret;
this.awsAccessKey =
DigestUtils.md5Hex(OmUtils.getMD5Digest(kerberosID));
this.awsAccessKey = DigestUtils.md5Hex(kerberosID);
}

public String getKerberosID() {
Expand Down Expand Up @@ -63,7 +59,7 @@ public void setAwsAccessKey(String awsAccessKey) {
}

public static S3SecretValue fromProtobuf(
OzoneManagerProtocolProtos.S3Secret s3Secret) throws IOException {
OzoneManagerProtocolProtos.S3Secret s3Secret) {
return new S3SecretValue(s3Secret.getKerberosID(), s3Secret.getAwsSecret());
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,116 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.security;

import org.apache.hadoop.util.StringUtils;
import org.apache.kerby.util.Hex;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import javax.crypto.Mac;
import javax.crypto.spec.SecretKeySpec;
import java.io.UnsupportedEncodingException;
import java.net.URLDecoder;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
import java.security.GeneralSecurityException;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;

/**
* AWS v4 authentication payload validator. For more details refer to AWS
* documentation https://docs.aws.amazon.com/general/latest/gr/
* sigv4-create-canonical-request.html.
**/
final class AWSV4AuthValidator {

private final static Logger LOG =
LoggerFactory.getLogger(AWSV4AuthValidator.class);
private static final String HMAC_SHA256_ALGORITHM = "HmacSHA256";
private static final Charset UTF_8 = Charset.forName("utf-8");

private AWSV4AuthValidator() {
}

private static String urlDecode(String str) {
try {
return URLDecoder.decode(str, UTF_8.name());
} catch (UnsupportedEncodingException e) {
throw new RuntimeException(e);
}
}

public static String hash(String payload) throws NoSuchAlgorithmException {
MessageDigest md = MessageDigest.getInstance("SHA-256");
md.update(payload.getBytes(UTF_8));
return String.format("%064x", new java.math.BigInteger(1, md.digest()));
}

private static byte[] sign(byte[] key, String msg) {
try {
SecretKeySpec signingKey = new SecretKeySpec(key, HMAC_SHA256_ALGORITHM);
Mac mac = Mac.getInstance(HMAC_SHA256_ALGORITHM);
mac.init(signingKey);
return mac.doFinal(msg.getBytes(StandardCharsets.UTF_8));
} catch (GeneralSecurityException gse) {
throw new RuntimeException(gse);
}
}

/**
* Returns signing key.
*
* @param key
* @param strToSign
*
* SignatureKey = HMAC-SHA256(HMAC-SHA256(HMAC-SHA256(HMAC-SHA256("AWS4" +
* "<YourSecretAccessKey>","20130524"),"us-east-1"),"s3"),"aws4_request")
*
* For more details refer to AWS documentation: https://docs.aws.amazon
* .com/AmazonS3/latest/API/sig-v4-header-based-auth.html
*
* */
private static byte[] getSigningKey(String key, String strToSign) {
String[] signData = StringUtils.split(StringUtils.split(strToSign,
'\n')[2], '/');
String dateStamp = signData[0];
String regionName = signData[1];
String serviceName = signData[2];
byte[] kDate = sign(("AWS4" + key).getBytes(UTF_8), dateStamp);
byte[] kRegion = sign(kDate, regionName);
byte[] kService = sign(kRegion, serviceName);
byte[] kSigning = sign(kService, "aws4_request");
LOG.info(Hex.encode(kSigning));
return kSigning;
}

/**
* Validate request by comparing Signature from request. Returns true if
* aws request is legit else returns false.
* Signature = HEX(HMAC_SHA256(key, String to Sign))
*
* For more details refer to AWS documentation: https://docs.aws.amazon.com
* /AmazonS3/latest/API/sigv4-streaming.html
*/
public static boolean validateRequest(String strToSign, String signature,
String userKey) {
String expectedSignature = Hex.encode(sign(getSigningKey(userKey,
strToSign), strToSign));
return expectedSignature.equals(signature);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
import org.apache.hadoop.hdds.security.x509.exceptions.CertificateException;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.ozone.om.S3SecretManager;
import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.apache.hadoop.ozone.security.OzoneSecretStore.OzoneManagerSecretState;
import org.apache.hadoop.ozone.security.OzoneTokenIdentifier.TokenInfo;
Expand All @@ -43,7 +44,9 @@
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;

import static java.nio.charset.StandardCharsets.UTF_8;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TOKEN_EXPIRED;
import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMTokenProto.Type.S3TOKEN;

/**
* SecretManager for Ozone Master. Responsible for signing identifiers with
Expand All @@ -58,6 +61,7 @@ public class OzoneDelegationTokenSecretManager
.getLogger(OzoneDelegationTokenSecretManager.class);
private final Map<OzoneTokenIdentifier, TokenInfo> currentTokens;
private final OzoneSecretStore store;
private final S3SecretManager s3SecretManager;
private Thread tokenRemoverThread;
private final long tokenRemoverScanInterval;
private String omCertificateSerialId;
Expand All @@ -80,12 +84,14 @@ public class OzoneDelegationTokenSecretManager
*/
public OzoneDelegationTokenSecretManager(OzoneConfiguration conf,
long tokenMaxLifetime, long tokenRenewInterval,
long dtRemoverScanInterval, Text service) throws IOException {
long dtRemoverScanInterval, Text service,
S3SecretManager s3SecretManager) throws IOException {
super(new SecurityConfig(conf), tokenMaxLifetime, tokenRenewInterval,
service, LOG);
currentTokens = new ConcurrentHashMap();
this.tokenRemoverScanInterval = dtRemoverScanInterval;
this.store = new OzoneSecretStore(conf);
this.s3SecretManager = s3SecretManager;
loadTokenSecretState(store.loadState());
}

Expand Down Expand Up @@ -279,14 +285,17 @@ public OzoneTokenIdentifier cancelToken(Token<OzoneTokenIdentifier> token,
@Override
public byte[] retrievePassword(OzoneTokenIdentifier identifier)
throws InvalidToken {
if(identifier.getTokenType().equals(S3TOKEN)) {
return validateS3Token(identifier);
}
return validateToken(identifier).getPassword();
}

/**
* Checks if TokenInfo for the given identifier exists in database and if the
* token is expired.
*/
public TokenInfo validateToken(OzoneTokenIdentifier identifier)
private TokenInfo validateToken(OzoneTokenIdentifier identifier)
throws InvalidToken {
TokenInfo info = currentTokens.get(identifier);
if (info == null) {
Expand Down Expand Up @@ -327,6 +336,37 @@ public boolean verifySignature(OzoneTokenIdentifier identifier,
}
}

/**
* Validates if a S3 identifier is valid or not.
* */
private byte[] validateS3Token(OzoneTokenIdentifier identifier)
throws InvalidToken {
LOG.trace("Validating S3Token for identifier:{}", identifier);
String awsSecret;
try {
awsSecret = s3SecretManager.getS3UserSecretString(identifier
.getAwsAccessId());
} catch (IOException e) {
LOG.error("Error while validating S3 identifier:{}",
identifier, e);
throw new InvalidToken("No S3 secret found for S3 identifier:"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Now if InvalidToken is thrown as an exception during invalid/malformed header, then how this will be thrown to the end user s3 request? I don't see any code for it.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Now if token validation fails rpc connection will fail itself. S3 gateway will get an error. Error propagation to client will depend on S3g error handling.

+ identifier);
}

if (awsSecret == null) {
throw new InvalidToken("No S3 secret found for S3 identifier:"
+ identifier);
}

if (AWSV4AuthValidator.validateRequest(identifier.getStrToSign(),
identifier.getSignature(), awsSecret)) {
return identifier.getSignature().getBytes(UTF_8);
}
throw new InvalidToken("Invalid S3 identifier:"
+ identifier);

}

// TODO: handle roll private key/certificate
private synchronized void removeExpiredKeys() {
long now = Time.now();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,6 +99,7 @@ public OzoneSecurityException.ResultCodes getResult() {
*/
public enum ResultCodes {
OM_PUBLIC_PRIVATE_KEY_FILE_NOT_EXIST,
S3_SECRET_NOT_FOUND,
SECRET_MANAGER_HMAC_ERROR
}
}
Loading