[jira] [Comment Edited] (HDFS-15098) Add SM4 encryption method for HDFS

2020-05-12 Thread Andrea (Jira)


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

Andrea edited comment on HDFS-15098 at 5/12/20, 1:31 PM:
-

[~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 cipher when execute " 
hadoop key create "

So, what idea do you think?

 

Cheers! 

 

 

 

 


was (Author: andrea_julianos_one):
[~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 = 

[jira] [Commented] (HDFS-15098) Add SM4 encryption method for HDFS

2020-05-12 Thread Andrea (Jira)


[ 
https://issues.apache.org/jira/browse/HDFS-15098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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



[jira] [Comment Edited] (HDFS-15098) Add SM4 encryption method for HDFS

2020-05-12 Thread Andrea (Jira)


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

Andrea edited comment on HDFS-15098 at 5/12/20, 6:44 AM:
-

[~weichiu] [~zZtai]

Hi, the message is KMS server side. I can know that " 
java.security.NoSuchAlgorithmException: SM4 KeyGenerator not available" is 
important. but there is nothing about  SM4 KeyGenerator in this patch.

openssl1.1.1 is Adaptable, and bcprov-ext-jdk15on-165.jar was put in 
JDK8_HOME/jre/lib/ext, and add info to java.security.

but Configure Hadoop KMS, I hava no info for how to  set it.

Thank you for watch. cheers

 
{code:java}
// code placeholder
User keyAdmin1 (auth:SIMPLE) request POST http://localhost:16000/kms/v1/keys 
caused exception.
java.lang.reflect.UndeclaredThrowableException
at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1930)
at org.apache.hadoop.crypto.key.kms.server.KMS.createKey(KMS.java:148)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:606)
at 
com.sun.jersey.spi.container.JavaMethodInvokerFactory$1.invoke(JavaMethodInvokerFactory.java:60)
at 
com.sun.jersey.server.impl.model.method.dispatch.AbstractResourceMethodDispatchProvider$ResponseOutInvoker._dispatch(AbstractResourceMethodDispatchProvider.java:205)
at 
com.sun.jersey.server.impl.model.method.dispatch.ResourceJavaMethodDispatcher.dispatch(ResourceJavaMethodDispatcher.java:75)
at 
com.sun.jersey.server.impl.uri.rules.HttpMethodRule.accept(HttpMethodRule.java:288)
at 
com.sun.jersey.server.impl.uri.rules.RightHandPathRule.accept(RightHandPathRule.java:147)
at 
com.sun.jersey.server.impl.uri.rules.ResourceClassRule.accept(ResourceClassRule.java:108)
at 
com.sun.jersey.server.impl.uri.rules.RightHandPathRule.accept(RightHandPathRule.java:147)
at 
com.sun.jersey.server.impl.uri.rules.RootResourceClassesRule.accept(RootResourceClassesRule.java:84)
at 
com.sun.jersey.server.impl.application.WebApplicationImpl._handleRequest(WebApplicationImpl.java:1469)
at 
com.sun.jersey.server.impl.application.WebApplicationImpl._handleRequest(WebApplicationImpl.java:1400)
at 
com.sun.jersey.server.impl.application.WebApplicationImpl.handleRequest(WebApplicationImpl.java:1349)
at 
com.sun.jersey.server.impl.application.WebApplicationImpl.handleRequest(WebApplicationImpl.java:1339)
at 
com.sun.jersey.spi.container.servlet.WebComponent.service(WebComponent.java:416)
at 
com.sun.jersey.spi.container.servlet.ServletContainer.service(ServletContainer.java:537)
at 
com.sun.jersey.spi.container.servlet.ServletContainer.service(ServletContainer.java:699)
at javax.servlet.http.HttpServlet.service(HttpServlet.java:723)
at 
org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:290)
at 
org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:206)
at 
org.apache.hadoop.crypto.key.kms.server.KMSMDCFilter.doFilter(KMSMDCFilter.java:84)
at 
org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:235)
at 
org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:206)
at 
org.apache.hadoop.security.authentication.server.AuthenticationFilter.doFilter(AuthenticationFilter.java:631)
at 
org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationFilter.doFilter(DelegationTokenAuthenticationFilter.java:301)
at 
org.apache.hadoop.security.authentication.server.AuthenticationFilter.doFilter(AuthenticationFilter.java:579)
at 
org.apache.hadoop.crypto.key.kms.server.KMSAuthenticationFilter.doFilter(KMSAuthenticationFilter.java:130)
at 
org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:235)
at 
org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:206)
at 
org.apache.catalina.core.StandardWrapperValve.invoke(StandardWrapperValve.java:233)
at 
org.apache.catalina.core.StandardContextValve.invoke(StandardContextValve.java:191)
at 
org.apache.catalina.core.StandardHostValve.invoke(StandardHostValve.java:127)
at 
org.apache.catalina.valves.ErrorReportValve.invoke(ErrorReportValve.java:103)
at 
org.apache.catalina.core.StandardEngineValve.invoke(StandardEngineValve.java:109)
at 
org.apache.catalina.connector.CoyoteAdapter.service(CoyoteAdapter.java:293)
at 

[jira] [Comment Edited] (HDFS-15098) Add SM4 encryption method for HDFS

2020-05-12 Thread Andrea (Jira)


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

Andrea edited comment on HDFS-15098 at 5/12/20, 6:39 AM:
-

[~weichiu] [~zZtai]

Hi, the message is KMS server side. I can know that " 
java.security.NoSuchAlgorithmException: SM4 KeyGenerator not available" is 
important. but there is nothing about  SM4 KeyGenerator in this patch.

openssl1.1.1 is Adaptable, and bcprov-ext-jdk15on-165.jar was put in 
JDK8_HOME/jre/lib/ext, and add info to java.security.

Thank you for watch. cheers

 
{code:java}
// code placeholder
User keyAdmin1 (auth:SIMPLE) request POST http://localhost:16000/kms/v1/keys 
caused exception.
java.lang.reflect.UndeclaredThrowableException
at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1930)
at org.apache.hadoop.crypto.key.kms.server.KMS.createKey(KMS.java:148)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:606)
at 
com.sun.jersey.spi.container.JavaMethodInvokerFactory$1.invoke(JavaMethodInvokerFactory.java:60)
at 
com.sun.jersey.server.impl.model.method.dispatch.AbstractResourceMethodDispatchProvider$ResponseOutInvoker._dispatch(AbstractResourceMethodDispatchProvider.java:205)
at 
com.sun.jersey.server.impl.model.method.dispatch.ResourceJavaMethodDispatcher.dispatch(ResourceJavaMethodDispatcher.java:75)
at 
com.sun.jersey.server.impl.uri.rules.HttpMethodRule.accept(HttpMethodRule.java:288)
at 
com.sun.jersey.server.impl.uri.rules.RightHandPathRule.accept(RightHandPathRule.java:147)
at 
com.sun.jersey.server.impl.uri.rules.ResourceClassRule.accept(ResourceClassRule.java:108)
at 
com.sun.jersey.server.impl.uri.rules.RightHandPathRule.accept(RightHandPathRule.java:147)
at 
com.sun.jersey.server.impl.uri.rules.RootResourceClassesRule.accept(RootResourceClassesRule.java:84)
at 
com.sun.jersey.server.impl.application.WebApplicationImpl._handleRequest(WebApplicationImpl.java:1469)
at 
com.sun.jersey.server.impl.application.WebApplicationImpl._handleRequest(WebApplicationImpl.java:1400)
at 
com.sun.jersey.server.impl.application.WebApplicationImpl.handleRequest(WebApplicationImpl.java:1349)
at 
com.sun.jersey.server.impl.application.WebApplicationImpl.handleRequest(WebApplicationImpl.java:1339)
at 
com.sun.jersey.spi.container.servlet.WebComponent.service(WebComponent.java:416)
at 
com.sun.jersey.spi.container.servlet.ServletContainer.service(ServletContainer.java:537)
at 
com.sun.jersey.spi.container.servlet.ServletContainer.service(ServletContainer.java:699)
at javax.servlet.http.HttpServlet.service(HttpServlet.java:723)
at 
org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:290)
at 
org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:206)
at 
org.apache.hadoop.crypto.key.kms.server.KMSMDCFilter.doFilter(KMSMDCFilter.java:84)
at 
org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:235)
at 
org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:206)
at 
org.apache.hadoop.security.authentication.server.AuthenticationFilter.doFilter(AuthenticationFilter.java:631)
at 
org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationFilter.doFilter(DelegationTokenAuthenticationFilter.java:301)
at 
org.apache.hadoop.security.authentication.server.AuthenticationFilter.doFilter(AuthenticationFilter.java:579)
at 
org.apache.hadoop.crypto.key.kms.server.KMSAuthenticationFilter.doFilter(KMSAuthenticationFilter.java:130)
at 
org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:235)
at 
org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:206)
at 
org.apache.catalina.core.StandardWrapperValve.invoke(StandardWrapperValve.java:233)
at 
org.apache.catalina.core.StandardContextValve.invoke(StandardContextValve.java:191)
at 
org.apache.catalina.core.StandardHostValve.invoke(StandardHostValve.java:127)
at 
org.apache.catalina.valves.ErrorReportValve.invoke(ErrorReportValve.java:103)
at 
org.apache.catalina.core.StandardEngineValve.invoke(StandardEngineValve.java:109)
at 
org.apache.catalina.connector.CoyoteAdapter.service(CoyoteAdapter.java:293)
at 
org.apache.coyote.http11.Http11Processor.process(Http11Processor.java:859)
at 

[jira] [Commented] (HDFS-15098) Add SM4 encryption method for HDFS

2020-05-12 Thread Andrea (Jira)


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

Andrea commented on HDFS-15098:
---

[~weichiu] [~zZtai]

Hi, the message is KMS server side. I can know that " 
java.security.NoSuchAlgorithmException: SM4 KeyGenerator not available" is 
important. but there is nothing about  SM4 KeyGenerator in this patch.

openssl1.1.1 is Adaptable 

Thank you for watch. cheers

 
{code:java}
// code placeholder
User keyAdmin1 (auth:SIMPLE) request POST http://localhost:16000/kms/v1/keys 
caused exception.
java.lang.reflect.UndeclaredThrowableException
at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1930)
at org.apache.hadoop.crypto.key.kms.server.KMS.createKey(KMS.java:148)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:606)
at 
com.sun.jersey.spi.container.JavaMethodInvokerFactory$1.invoke(JavaMethodInvokerFactory.java:60)
at 
com.sun.jersey.server.impl.model.method.dispatch.AbstractResourceMethodDispatchProvider$ResponseOutInvoker._dispatch(AbstractResourceMethodDispatchProvider.java:205)
at 
com.sun.jersey.server.impl.model.method.dispatch.ResourceJavaMethodDispatcher.dispatch(ResourceJavaMethodDispatcher.java:75)
at 
com.sun.jersey.server.impl.uri.rules.HttpMethodRule.accept(HttpMethodRule.java:288)
at 
com.sun.jersey.server.impl.uri.rules.RightHandPathRule.accept(RightHandPathRule.java:147)
at 
com.sun.jersey.server.impl.uri.rules.ResourceClassRule.accept(ResourceClassRule.java:108)
at 
com.sun.jersey.server.impl.uri.rules.RightHandPathRule.accept(RightHandPathRule.java:147)
at 
com.sun.jersey.server.impl.uri.rules.RootResourceClassesRule.accept(RootResourceClassesRule.java:84)
at 
com.sun.jersey.server.impl.application.WebApplicationImpl._handleRequest(WebApplicationImpl.java:1469)
at 
com.sun.jersey.server.impl.application.WebApplicationImpl._handleRequest(WebApplicationImpl.java:1400)
at 
com.sun.jersey.server.impl.application.WebApplicationImpl.handleRequest(WebApplicationImpl.java:1349)
at 
com.sun.jersey.server.impl.application.WebApplicationImpl.handleRequest(WebApplicationImpl.java:1339)
at 
com.sun.jersey.spi.container.servlet.WebComponent.service(WebComponent.java:416)
at 
com.sun.jersey.spi.container.servlet.ServletContainer.service(ServletContainer.java:537)
at 
com.sun.jersey.spi.container.servlet.ServletContainer.service(ServletContainer.java:699)
at javax.servlet.http.HttpServlet.service(HttpServlet.java:723)
at 
org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:290)
at 
org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:206)
at 
org.apache.hadoop.crypto.key.kms.server.KMSMDCFilter.doFilter(KMSMDCFilter.java:84)
at 
org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:235)
at 
org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:206)
at 
org.apache.hadoop.security.authentication.server.AuthenticationFilter.doFilter(AuthenticationFilter.java:631)
at 
org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationFilter.doFilter(DelegationTokenAuthenticationFilter.java:301)
at 
org.apache.hadoop.security.authentication.server.AuthenticationFilter.doFilter(AuthenticationFilter.java:579)
at 
org.apache.hadoop.crypto.key.kms.server.KMSAuthenticationFilter.doFilter(KMSAuthenticationFilter.java:130)
at 
org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:235)
at 
org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:206)
at 
org.apache.catalina.core.StandardWrapperValve.invoke(StandardWrapperValve.java:233)
at 
org.apache.catalina.core.StandardContextValve.invoke(StandardContextValve.java:191)
at 
org.apache.catalina.core.StandardHostValve.invoke(StandardHostValve.java:127)
at 
org.apache.catalina.valves.ErrorReportValve.invoke(ErrorReportValve.java:103)
at 
org.apache.catalina.core.StandardEngineValve.invoke(StandardEngineValve.java:109)
at 
org.apache.catalina.connector.CoyoteAdapter.service(CoyoteAdapter.java:293)
at 
org.apache.coyote.http11.Http11Processor.process(Http11Processor.java:859)
at 
org.apache.coyote.http11.Http11Protocol$Http11ConnectionHandler.process(Http11Protocol.java:610)
at 

[jira] [Commented] (HDFS-15098) Add SM4 encryption method for HDFS

2020-05-11 Thread Andrea (Jira)


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

Andrea commented on HDFS-15098:
---

[~zZtai]   

Hi, I found a bugs, when I input 'hadoop key create key1 -cipher 
'SM4/CTR/NoPadding', I will get a error, it is about 

java.io.IOException: HTTP status [500], exception 
[java.lang.reflect.UndeclaredThrowableException], message [null] 
 at 
org.apache.hadoop.util.HttpExceptionUtils.validateResponse(HttpExceptionUtils.java:159)
 at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider.call(KMSClientProvider.java:540)
 at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider.call(KMSClientProvider.java:490)
 at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider.createKeyInternal(KMSClientProvider.java:664)
 at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider.createKey(KMSClientProvider.java:672)
 at 
org.apache.hadoop.crypto.key.KeyShell$CreateCommand.execute(KeyShell.java:521)
 at org.apache.hadoop.crypto.key.KeyShell.run(KeyShell.java:92)
 at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:70)
 at org.apache.hadoop.crypto.key.KeyShell.main(KeyShell.java:604)

but -cipher 'AES/CTR/NoPadding' can work.

My test env is hadoop2.6.0-cdh5.13.1.

Do you have some idea ?

> 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



[jira] [Comment Edited] (HDFS-15098) Add SM4 encryption method for HDFS

2020-02-24 Thread Andrea (Jira)


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

Andrea edited comment on HDFS-15098 at 2/24/20 11:52 AM:
-

This patch can be used which  hadoop version  and openssl version ?


was (Author: andrea_julianos_one):
This patch can be used whice  hadoop version  and openssl version ?

> 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
>Reporter: liusheng
>Priority: Major
> Attachments: HDFS-15098.001.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



[jira] [Commented] (HDFS-15098) Add SM4 encryption method for HDFS

2020-02-24 Thread Andrea (Jira)


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

Andrea commented on HDFS-15098:
---

This patch can be used whice  hadoop version  and openssl version ?

> 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
>Reporter: liusheng
>Priority: Major
> Attachments: HDFS-15098.001.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