|
@@ -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));
|
|
|
+
|
|
|
+ }
|
|
|
}
|