HDDS-2015. Encrypt/decrypt key using symmetric key while writing/reading
Signed-off-by: Anu Engineer <aengineer@apache.org>
This commit is contained in:
parent
b71a7f195c
commit
b15c116c1e
|
@ -313,8 +313,9 @@ public final class OzoneConsts {
|
|||
public static final int S3_BUCKET_MAX_LENGTH = 64;
|
||||
|
||||
//GDPR
|
||||
public static final String GDPR_FLAG = "gdprEnabled";
|
||||
public static final String GDPR_ALGORITHM_NAME = "AES";
|
||||
public static final int GDPR_RANDOM_SECRET_LENGTH = 32;
|
||||
public static final int GDPR_DEFAULT_RANDOM_SECRET_LENGTH = 16;
|
||||
public static final String GDPR_CHARSET = "UTF-8";
|
||||
public static final String GDPR_LENGTH = "length";
|
||||
public static final String GDPR_SECRET = "secret";
|
||||
|
|
|
@ -82,6 +82,7 @@ import org.apache.hadoop.hdds.scm.protocolPB
|
|||
.StorageContainerLocationProtocolClientSideTranslatorPB;
|
||||
import org.apache.hadoop.hdds.scm.protocolPB
|
||||
.StorageContainerLocationProtocolPB;
|
||||
import org.apache.hadoop.ozone.security.GDPRSymmetricKey;
|
||||
import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
|
||||
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLIdentityType;
|
||||
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType;
|
||||
|
@ -96,9 +97,13 @@ import org.apache.ratis.protocol.ClientId;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.crypto.Cipher;
|
||||
import javax.crypto.CipherInputStream;
|
||||
import javax.crypto.CipherOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.URI;
|
||||
import java.security.InvalidKeyException;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.stream.Collectors;
|
||||
|
@ -601,6 +606,22 @@ public class RpcClient implements ClientProtocol {
|
|||
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
||||
HddsClientUtils.checkNotNull(keyName, type, factor);
|
||||
String requestId = UUID.randomUUID().toString();
|
||||
|
||||
if(Boolean.valueOf(metadata.get(OzoneConsts.GDPR_FLAG))){
|
||||
try{
|
||||
GDPRSymmetricKey gKey = new GDPRSymmetricKey();
|
||||
metadata.putAll(gKey.getKeyDetails());
|
||||
}catch (Exception e) {
|
||||
if(e instanceof InvalidKeyException &&
|
||||
e.getMessage().contains("Illegal key size or default parameters")) {
|
||||
LOG.error("Missing Unlimited Strength Policy jars. Please install " +
|
||||
"Java Cryptography Extension (JCE) Unlimited Strength " +
|
||||
"Jurisdiction Policy Files");
|
||||
}
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
|
||||
.setVolumeName(volumeName)
|
||||
.setBucketName(bucketName)
|
||||
|
@ -1062,6 +1083,22 @@ public class RpcClient implements ClientProtocol {
|
|||
OzoneKMSUtil.getCryptoCodec(conf, feInfo),
|
||||
decrypted.getMaterial(), feInfo.getIV());
|
||||
return new OzoneInputStream(cryptoIn);
|
||||
} else {
|
||||
try{
|
||||
GDPRSymmetricKey gk;
|
||||
Map<String, String> keyInfoMetadata = keyInfo.getMetadata();
|
||||
if(Boolean.valueOf(keyInfoMetadata.get(OzoneConsts.GDPR_FLAG))){
|
||||
gk = new GDPRSymmetricKey(
|
||||
keyInfoMetadata.get(OzoneConsts.GDPR_SECRET),
|
||||
keyInfoMetadata.get(OzoneConsts.GDPR_ALGORITHM)
|
||||
);
|
||||
gk.getCipher().init(Cipher.DECRYPT_MODE, gk.getSecretKey());
|
||||
return new OzoneInputStream(
|
||||
new CipherInputStream(lengthInputStream, gk.getCipher()));
|
||||
}
|
||||
}catch (Exception ex){
|
||||
throw new IOException(ex);
|
||||
}
|
||||
}
|
||||
return new OzoneInputStream(lengthInputStream.getWrappedStream());
|
||||
}
|
||||
|
@ -1099,6 +1136,23 @@ public class RpcClient implements ClientProtocol {
|
|||
decrypted.getMaterial(), feInfo.getIV());
|
||||
return new OzoneOutputStream(cryptoOut);
|
||||
} else {
|
||||
try{
|
||||
GDPRSymmetricKey gk;
|
||||
Map<String, String> openKeyMetadata =
|
||||
openKey.getKeyInfo().getMetadata();
|
||||
if(Boolean.valueOf(openKeyMetadata.get(OzoneConsts.GDPR_FLAG))){
|
||||
gk = new GDPRSymmetricKey(
|
||||
openKeyMetadata.get(OzoneConsts.GDPR_SECRET),
|
||||
openKeyMetadata.get(OzoneConsts.GDPR_ALGORITHM)
|
||||
);
|
||||
gk.getCipher().init(Cipher.ENCRYPT_MODE, gk.getSecretKey());
|
||||
return new OzoneOutputStream(
|
||||
new CipherOutputStream(keyOutputStream, gk.getCipher()));
|
||||
}
|
||||
}catch (Exception ex){
|
||||
throw new IOException(ex);
|
||||
}
|
||||
|
||||
return new OzoneOutputStream(keyOutputStream);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -51,7 +51,7 @@ public class GDPRSymmetricKey {
|
|||
public GDPRSymmetricKey() throws Exception {
|
||||
algorithm = OzoneConsts.GDPR_ALGORITHM_NAME;
|
||||
secret = RandomStringUtils
|
||||
.randomAlphabetic(OzoneConsts.GDPR_RANDOM_SECRET_LENGTH);
|
||||
.randomAlphabetic(OzoneConsts.GDPR_DEFAULT_RANDOM_SECRET_LENGTH);
|
||||
this.secretKey = new SecretKeySpec(
|
||||
secret.getBytes(OzoneConsts.GDPR_CHARSET), algorithm);
|
||||
this.cipher = Cipher.getInstance(algorithm);
|
||||
|
@ -62,8 +62,12 @@ public class GDPRSymmetricKey {
|
|||
* @throws Exception
|
||||
*/
|
||||
public GDPRSymmetricKey(String secret, String algorithm) throws Exception {
|
||||
Preconditions.checkArgument(secret.length() == 32,
|
||||
"Secret must be exactly 32 characters");
|
||||
Preconditions.checkNotNull(secret, "Secret cannot be null");
|
||||
//TODO: When we add feature to allow users to customize the secret length,
|
||||
// we need to update this length check Precondition
|
||||
Preconditions.checkArgument(secret.length() == 16,
|
||||
"Secret must be exactly 16 characters");
|
||||
Preconditions.checkNotNull(algorithm, "Algorithm cannot be null");
|
||||
this.secret = secret;
|
||||
this.algorithm = algorithm;
|
||||
this.secretKey = new SecretKeySpec(
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.ozone.security;
|
||||
|
||||
import org.apache.commons.lang3.RandomStringUtils;
|
||||
import org.apache.hadoop.ozone.OzoneConsts;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -39,7 +40,7 @@ public class TestGDPRSymmetricKey {
|
|||
@Test
|
||||
public void testKeyGenerationWithValidInput() throws Exception {
|
||||
GDPRSymmetricKey gkey = new GDPRSymmetricKey(
|
||||
"ApacheHadoopOzoneIsAnObjectStore",
|
||||
RandomStringUtils.randomAlphabetic(16),
|
||||
OzoneConsts.GDPR_ALGORITHM_NAME);
|
||||
|
||||
Assert.assertTrue(gkey.getCipher().getAlgorithm()
|
||||
|
@ -53,11 +54,11 @@ public class TestGDPRSymmetricKey {
|
|||
public void testKeyGenerationWithInvalidInput() throws Exception {
|
||||
GDPRSymmetricKey gkey = null;
|
||||
try{
|
||||
gkey = new GDPRSymmetricKey("ozone",
|
||||
gkey = new GDPRSymmetricKey(RandomStringUtils.randomAlphabetic(5),
|
||||
OzoneConsts.GDPR_ALGORITHM_NAME);
|
||||
} catch (IllegalArgumentException ex) {
|
||||
Assert.assertTrue(ex.getMessage()
|
||||
.equalsIgnoreCase("Secret must be exactly 32 characters"));
|
||||
.equalsIgnoreCase("Secret must be exactly 16 characters"));
|
||||
Assert.assertTrue(gkey == null);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -74,6 +74,7 @@ import org.apache.hadoop.ozone.container.common.interfaces.Container;
|
|||
import org.apache.hadoop.ozone.container.keyvalue.KeyValueBlockIterator;
|
||||
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
|
||||
import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
|
||||
import org.apache.hadoop.ozone.om.OMMetadataManager;
|
||||
import org.apache.hadoop.ozone.om.OzoneManager;
|
||||
import org.apache.hadoop.ozone.om.exceptions.OMException;
|
||||
import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
|
||||
|
@ -2647,4 +2648,89 @@ public abstract class TestOzoneRpcClientAbstract {
|
|||
Assert.assertEquals(omMultipartUploadCompleteInfo.getKey(), keyName);
|
||||
Assert.assertNotNull(omMultipartUploadCompleteInfo.getHash());
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests GDPR encryption/decryption.
|
||||
* 1. Create GDPR Enabled bucket.
|
||||
* 2. Create a Key in this bucket so it gets encrypted via GDPRSymmetricKey.
|
||||
* 3. Read key and validate the content/metadata is as expected because the
|
||||
* readKey will decrypt using the GDPR Symmetric Key with details from KeyInfo
|
||||
* Metadata.
|
||||
* 4. To check encryption, we forcibly update KeyInfo Metadata and remove the
|
||||
* gdprEnabled flag
|
||||
* 5. When we now read the key, {@link RpcClient} checks for GDPR Flag in
|
||||
* method createInputStream. If the gdprEnabled flag in metadata is set to
|
||||
* true, it decrypts using the GDPRSymmetricKey. Since we removed that flag
|
||||
* from metadata for this key, if will read the encrypted data as-is.
|
||||
* 6. Thus, when we compare this content with expected text, it should
|
||||
* not match as the decryption has not been performed.
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
public void testGDPR() throws Exception {
|
||||
//Step 1
|
||||
String volumeName = UUID.randomUUID().toString();
|
||||
String bucketName = UUID.randomUUID().toString();
|
||||
String keyName = UUID.randomUUID().toString();
|
||||
|
||||
store.createVolume(volumeName);
|
||||
OzoneVolume volume = store.getVolume(volumeName);
|
||||
BucketArgs args = BucketArgs.newBuilder()
|
||||
.addMetadata(OzoneConsts.GDPR_FLAG, "true").build();
|
||||
volume.createBucket(bucketName, args);
|
||||
OzoneBucket bucket = volume.getBucket(bucketName);
|
||||
Assert.assertEquals(bucketName, bucket.getName());
|
||||
Assert.assertNotNull(bucket.getMetadata());
|
||||
Assert.assertEquals("true",
|
||||
bucket.getMetadata().get(OzoneConsts.GDPR_FLAG));
|
||||
|
||||
//Step 2
|
||||
String text = "hello world";
|
||||
Map<String, String> keyMetadata = new HashMap<>();
|
||||
keyMetadata.put(OzoneConsts.GDPR_FLAG, "true");
|
||||
OzoneOutputStream out = bucket.createKey(keyName,
|
||||
text.getBytes().length, STAND_ALONE, ONE, keyMetadata);
|
||||
out.write(text.getBytes());
|
||||
out.close();
|
||||
|
||||
//Step 3
|
||||
OzoneKeyDetails key = bucket.getKey(keyName);
|
||||
|
||||
Assert.assertEquals(keyName, key.getName());
|
||||
Assert.assertEquals("true", key.getMetadata().get(OzoneConsts.GDPR_FLAG));
|
||||
Assert.assertEquals("AES",
|
||||
key.getMetadata().get(OzoneConsts.GDPR_ALGORITHM));
|
||||
Assert.assertTrue(key.getMetadata().get(OzoneConsts.GDPR_SECRET) != null);
|
||||
|
||||
OzoneInputStream is = bucket.readKey(keyName);
|
||||
byte[] fileContent = new byte[text.getBytes().length];
|
||||
is.read(fileContent);
|
||||
Assert.assertTrue(verifyRatisReplication(volumeName, bucketName,
|
||||
keyName, STAND_ALONE,
|
||||
ONE));
|
||||
Assert.assertEquals(text, new String(fileContent));
|
||||
|
||||
//Step 4
|
||||
OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
|
||||
OmKeyInfo omKeyInfo =
|
||||
omMetadataManager.getKeyTable().get(omMetadataManager.getOzoneKey(
|
||||
volumeName, bucketName, keyName));
|
||||
|
||||
omKeyInfo.getMetadata().remove(OzoneConsts.GDPR_FLAG);
|
||||
|
||||
omMetadataManager.getKeyTable().put(omMetadataManager.getOzoneKey(
|
||||
volumeName, bucketName, keyName), omKeyInfo);
|
||||
|
||||
//Step 5
|
||||
key = bucket.getKey(keyName);
|
||||
Assert.assertEquals(keyName, key.getName());
|
||||
Assert.assertEquals(null, key.getMetadata().get(OzoneConsts.GDPR_FLAG));
|
||||
is = bucket.readKey(keyName);
|
||||
fileContent = new byte[text.getBytes().length];
|
||||
is.read(fileContent);
|
||||
|
||||
//Step 6
|
||||
Assert.assertNotEquals(text, new String(fileContent));
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -483,7 +483,6 @@ public class KeyManagerImpl implements KeyManager {
|
|||
OmKeyInfo keyInfo = null;
|
||||
if (keyArgs.getIsMultipartKey()) {
|
||||
keyInfo = prepareMultipartKeyInfo(keyArgs, size, locations, encInfo);
|
||||
//TODO args.getMetadata
|
||||
} else if (metadataManager.getKeyTable().isExist(dbKeyName)) {
|
||||
keyInfo = metadataManager.getKeyTable().get(dbKeyName);
|
||||
// the key already exist, the new blocks will be added as new version
|
||||
|
@ -492,6 +491,9 @@ public class KeyManagerImpl implements KeyManager {
|
|||
keyInfo.addNewVersion(locations, true);
|
||||
keyInfo.setDataSize(size + keyInfo.getDataSize());
|
||||
}
|
||||
if(keyInfo != null) {
|
||||
keyInfo.setMetadata(keyArgs.getMetadata());
|
||||
}
|
||||
return keyInfo;
|
||||
}
|
||||
|
||||
|
@ -556,8 +558,13 @@ public class KeyManagerImpl implements KeyManager {
|
|||
.setDataSize(size)
|
||||
.setReplicationType(type)
|
||||
.setReplicationFactor(factor)
|
||||
.setFileEncryptionInfo(encInfo);
|
||||
.setFileEncryptionInfo(encInfo)
|
||||
.addAllMetadata(keyArgs.getMetadata());
|
||||
builder.setAcls(getAclsForKey(keyArgs, omBucketInfo));
|
||||
|
||||
if(Boolean.valueOf(omBucketInfo.getMetadata().get(OzoneConsts.GDPR_FLAG))) {
|
||||
builder.addMetadata(OzoneConsts.GDPR_FLAG, Boolean.TRUE.toString());
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
|
|
@ -35,6 +35,7 @@ import com.google.common.base.Preconditions;
|
|||
import org.apache.hadoop.ozone.OzoneAcl;
|
||||
import org.apache.hadoop.ozone.om.PrefixManager;
|
||||
import org.apache.hadoop.ozone.om.helpers.BucketEncryptionKeyInfo;
|
||||
import org.apache.hadoop.ozone.om.helpers.KeyValueUtil;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
||||
|
@ -361,7 +362,9 @@ public abstract class OMKeyRequest extends OMClientRequest {
|
|||
.setReplicationType(type)
|
||||
.setReplicationFactor(factor)
|
||||
.setFileEncryptionInfo(encInfo)
|
||||
.setAcls(getAclsForKey(keyArgs, omBucketInfo, prefixManager)).build();
|
||||
.setAcls(getAclsForKey(keyArgs, omBucketInfo, prefixManager))
|
||||
.addAllMetadata(KeyValueUtil.getFromProtobuf(keyArgs.getMetadataList()))
|
||||
.build();
|
||||
}
|
||||
|
||||
private List< OzoneAcl > getAclsForKey(KeyArgs keyArgs,
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.hadoop.io.Text;
|
|||
import org.apache.hadoop.ozone.OzoneAcl;
|
||||
import org.apache.hadoop.ozone.om.OzoneManager;
|
||||
import org.apache.hadoop.ozone.om.exceptions.OMException;
|
||||
import org.apache.hadoop.ozone.om.helpers.KeyValueUtil;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
||||
|
@ -585,6 +586,7 @@ public class OzoneManagerRequestHandler implements RequestHandler {
|
|||
.setMultipartUploadPartNumber(keyArgs.getMultipartNumber())
|
||||
.setAcls(keyArgs.getAclsList().stream().map(a ->
|
||||
OzoneAcl.fromProtobuf(a)).collect(Collectors.toList()))
|
||||
.addAllMetadata(KeyValueUtil.getFromProtobuf(keyArgs.getMetadataList()))
|
||||
.build();
|
||||
if (keyArgs.hasDataSize()) {
|
||||
omKeyArgs.setDataSize(keyArgs.getDataSize());
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.ozone.web.ozShell.keys;
|
|||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.conf.StorageUnit;
|
||||
|
@ -28,6 +29,7 @@ import org.apache.hadoop.hdds.client.ReplicationFactor;
|
|||
import org.apache.hadoop.hdds.client.ReplicationType;
|
||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.ozone.OzoneConsts;
|
||||
import org.apache.hadoop.ozone.client.OzoneBucket;
|
||||
import org.apache.hadoop.ozone.client.OzoneClient;
|
||||
import org.apache.hadoop.ozone.client.OzoneVolume;
|
||||
|
@ -104,9 +106,13 @@ public class PutKeyHandler extends Handler {
|
|||
conf.get(OZONE_REPLICATION_TYPE, OZONE_REPLICATION_TYPE_DEFAULT));
|
||||
OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
|
||||
OzoneBucket bucket = vol.getBucket(bucketName);
|
||||
Map<String, String> keyMetadata = new HashMap<>();
|
||||
if(Boolean.valueOf(bucket.getMetadata().get(OzoneConsts.GDPR_FLAG))){
|
||||
keyMetadata.put(OzoneConsts.GDPR_FLAG, Boolean.TRUE.toString());
|
||||
}
|
||||
OzoneOutputStream outputStream = bucket
|
||||
.createKey(keyName, dataFile.length(), replicationType,
|
||||
replicationFactor, new HashMap<>());
|
||||
replicationFactor, keyMetadata);
|
||||
FileInputStream fileInputStream = new FileInputStream(dataFile);
|
||||
IOUtils.copyBytes(fileInputStream, outputStream, (int) conf
|
||||
.getStorageSize(OZONE_SCM_CHUNK_SIZE_KEY, OZONE_SCM_CHUNK_SIZE_DEFAULT,
|
||||
|
|
Loading…
Reference in New Issue