diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java index 302a56192837..9bdbf1f6c839 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java @@ -605,12 +605,22 @@ public OzoneOutputStream createKey( HddsClientUtils.checkNotNull(keyName, type, factor); String requestId = UUID.randomUUID().toString(); - if(Boolean.valueOf(metadata.get(OzoneConsts.GDPR_FLAG))){ + OmKeyArgs.Builder builder = new OmKeyArgs.Builder() + .setVolumeName(volumeName) + .setBucketName(bucketName) + .setKeyName(keyName) + .setDataSize(size) + .setType(HddsProtos.ReplicationType.valueOf(type.toString())) + .setFactor(HddsProtos.ReplicationFactor.valueOf(factor.getValue())) + .addAllMetadata(metadata) + .setAcls(getAclList()); + + if (Boolean.parseBoolean(metadata.get(OzoneConsts.GDPR_FLAG))) { try{ GDPRSymmetricKey gKey = new GDPRSymmetricKey(new SecureRandom()); - metadata.putAll(gKey.getKeyDetails()); - }catch (Exception e) { - if(e instanceof InvalidKeyException && + builder.addAllMetadata(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 " + @@ -620,18 +630,7 @@ public OzoneOutputStream createKey( } } - OmKeyArgs keyArgs = new OmKeyArgs.Builder() - .setVolumeName(volumeName) - .setBucketName(bucketName) - .setKeyName(keyName) - .setDataSize(size) - .setType(HddsProtos.ReplicationType.valueOf(type.toString())) - .setFactor(HddsProtos.ReplicationFactor.valueOf(factor.getValue())) - .addAllMetadata(metadata) - .setAcls(getAclList()) - .build(); - - OpenKeySession openKey = ozoneManagerClient.openKey(keyArgs); + OpenKeySession openKey = ozoneManagerClient.openKey(builder.build()); return createOutputStream(openKey, requestId, type, factor); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java index 172841fc7771..2163773c7f5a 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java @@ -2693,6 +2693,7 @@ public void testKeyReadWriteForGDPR() throws Exception { text.getBytes().length, STAND_ALONE, ONE, keyMetadata); out.write(text.getBytes()); out.close(); + Assert.assertNull(keyMetadata.get(OzoneConsts.GDPR_SECRET)); //Step 3 OzoneKeyDetails key = bucket.getKey(keyName); @@ -2701,7 +2702,7 @@ public void testKeyReadWriteForGDPR() throws Exception { 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); + Assert.assertNotNull(key.getMetadata().get(OzoneConsts.GDPR_SECRET)); OzoneInputStream is = bucket.readKey(keyName); byte[] fileContent = new byte[text.getBytes().length]; @@ -2725,7 +2726,7 @@ public void testKeyReadWriteForGDPR() throws Exception { //Step 5 key = bucket.getKey(keyName); Assert.assertEquals(keyName, key.getName()); - Assert.assertEquals(null, key.getMetadata().get(OzoneConsts.GDPR_FLAG)); + Assert.assertNull(key.getMetadata().get(OzoneConsts.GDPR_FLAG)); is = bucket.readKey(keyName); fileContent = new byte[text.getBytes().length]; is.read(fileContent);