[ 
https://issues.apache.org/jira/browse/HDFS-15098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17105426#comment-17105426
 ] 

Andrea commented on HDFS-15098:
-------------------------------

[~weichiu] [~zZtai]

Hi, I modified the method of KeyProvider.java that called generateKey(int size, 
String algorithm). like this 

 
{code:java}
// code placeholder
protected byte[] generateKey(int size, String algorithm)
    throws NoSuchAlgorithmException {
  algorithm = getAlgorithm(algorithm);
  KeyGenerator keyGenerator = KeyGenerator.getInstance(algorithm);
  keyGenerator.init(size);
  byte[] key = keyGenerator.generateKey().getEncoded();
  return key;
}

====>
protected byte[] generateKey(int size, String algorithm)
    throws NoSuchAlgorithmException {
  if("SM4/CTR/NoPadding".equals(algorithm)){
       algorithm = "AES/CTR/NoPadding"
   };
  algorithm = getAlgorithm(algorithm);
  KeyGenerator keyGenerator = KeyGenerator.getInstance(algorithm);
  keyGenerator.init(size);
  byte[] key = keyGenerator.generateKey().getEncoded();
  return key;
}

{code}
and run "hadoop key create key5 -cipher 'SM4/CTR/NoPadding' -size 128 -provider 
kms://http@localhost:16000/kms "

 

I get a result like

 
{code:java}
// code placeholder
key5 has been successfully created with options 
Options{cipher='SM4/CTR/NoPadding', bitLength=128, description='null', 
attributes=null}.
KMSClientProvider[http://localhost:16000/kms/v1/] has been updated.
{code}
 

 

Now, I temporary fixed a bug, when I run "hadoop fs -put file /encryptZone" 
that print console info : "Now Codec is OpensslSm4CryptoCodec",

In the past, I get the  console info is : "Now Codec is 
OpensslOpensslAesCtrCryptoCodec", that I used this patch.

the console info in DFSClient.java is 

 
{code:java}
// code placeholder
private static CryptoCodec getCryptoCodec(Configuration conf,
    FileEncryptionInfo feInfo) throws IOException {
  final CipherSuite suite = feInfo.getCipherSuite();
  if (suite.equals(CipherSuite.UNKNOWN)) {
    throw new IOException("NameNode specified unknown CipherSuite with ID "
        + suite.getUnknownValue() + ", cannot instantiate CryptoCodec.");
  }

  final CryptoCodec codec = CryptoCodec.getInstance(conf, suite);

  if (codec instanceof OpensslAesCtrCryptoCodec) {
    System.out.println("Now Codec is OpensslAesCtrCryptoCodec");
  }
  if (codec instanceof OpensslSm4CtrCryptoCodec) {
    System.out.println("Now Codec is OpensslSm4CtrCryptoCodec");
  }
  if (codec instanceof JceAesCtrCryptoCodec) {
    System.out.println("Now Codec is JceAesCtrCryptoCodec");
  }

{code}
It Seems like the method of PBHelper.java(Hadoop-hdfs),  "convert(CipherSuite 
suite)" or convert(CipherSuiteProto proto),   They are still received 
AES/CTR/NoPadding , If you do not specify  SM4 as the policy  when execute " 
hadoop key create "

So, what idea do you think?

 

Cheers! 

 

 

 

 

> Add SM4 encryption method for HDFS
> ----------------------------------
>
>                 Key: HDFS-15098
>                 URL: https://issues.apache.org/jira/browse/HDFS-15098
>             Project: Hadoop HDFS
>          Issue Type: New Feature
>    Affects Versions: 3.4.0
>            Reporter: liusheng
>            Assignee: zZtai
>            Priority: Major
>              Labels: sm4
>         Attachments: HDFS-15098.001.patch, HDFS-15098.002.patch, 
> HDFS-15098.003.patch
>
>
> SM4 (formerly SMS4)is a block cipher used in the Chinese National Standard 
> for Wireless LAN WAPI (Wired Authentication and Privacy Infrastructure).
> SM4 was a cipher proposed to for the IEEE 802.11i standard, but has so far 
> been rejected by ISO. One of the reasons for the rejection has been 
> opposition to the WAPI fast-track proposal by the IEEE. please see:
> [https://en.wikipedia.org/wiki/SM4_(cipher)]



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to