Przeglądaj źródła

HDFS-15098. Add SM4 encryption method for HDFS. Contributed by liusheng

Vinayakumar B 4 lat temu
rodzic
commit
82b86e3754
28 zmienionych plików z 1007 dodań i 362 usunięć
  1. 0 1
      hadoop-common-project/hadoop-common/pom.xml
  2. 0 70
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/AesCtrCryptoCodec.java
  3. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CipherSuite.java
  4. 5 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java
  5. 5 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoStreamUtils.java
  6. 18 118
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/JceAesCtrCryptoCodec.java
  7. 174 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/JceCtrCryptoCodec.java
  8. 65 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/JceSm4CtrCryptoCodec.java
  9. 18 116
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslAesCtrCryptoCodec.java
  10. 27 12
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslCipher.java
  11. 189 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslCtrCryptoCodec.java
  12. 79 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslSm4CtrCryptoCodec.java
  13. 9 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
  14. 7 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/random/OsSecureRandom.java
  15. 17 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
  16. 111 8
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/crypto/OpensslCipher.c
  17. 7 1
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/crypto/org_apache_hadoop_crypto.h
  18. 20 0
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  19. 2 0
      hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md
  20. 88 12
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoCodec.java
  21. 48 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithJceSm4CtrCryptoCodec.java
  22. 10 7
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java
  23. 79 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslSm4CtrCryptoCodec.java
  24. 6 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
  25. 9 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferClient.java
  26. 4 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
  27. 1 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
  28. 7 1
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/TransparentEncryption.md

+ 0 - 1
hadoop-common-project/hadoop-common/pom.xml

@@ -323,7 +323,6 @@
     <dependency>
       <groupId>org.bouncycastle</groupId>
       <artifactId>bcprov-jdk15on</artifactId>
-      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.kerby</groupId>

+ 0 - 70
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/AesCtrCryptoCodec.java

@@ -1,70 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.crypto;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-import com.google.common.base.Preconditions;
-
-import java.io.IOException;
-
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public abstract class AesCtrCryptoCodec extends CryptoCodec {
-
-  protected static final CipherSuite SUITE = CipherSuite.AES_CTR_NOPADDING;
-
-  /**
-   * For AES, the algorithm block is fixed size of 128 bits.
-   * @see http://en.wikipedia.org/wiki/Advanced_Encryption_Standard
-   */
-  private static final int AES_BLOCK_SIZE = SUITE.getAlgorithmBlockSize();
-
-  @Override
-  public CipherSuite getCipherSuite() {
-    return SUITE;
-  }
-  
-  /**
-   * The IV is produced by adding the initial IV to the counter. IV length 
-   * should be the same as {@link #AES_BLOCK_SIZE}
-   */
-  @Override
-  public void calculateIV(byte[] initIV, long counter, byte[] IV) {
-    Preconditions.checkArgument(initIV.length == AES_BLOCK_SIZE);
-    Preconditions.checkArgument(IV.length == AES_BLOCK_SIZE);
-
-    int i = IV.length; // IV length
-    int j = 0; // counter bytes index
-    int sum = 0;
-    while (i-- > 0) {
-      // (sum >>> Byte.SIZE) is the carry for addition
-      sum = (initIV[i] & 0xff) + (sum >>> Byte.SIZE);
-      if (j++ < 8) { // Big-endian, and long is 8 bytes length
-        sum += (byte) counter & 0xff;
-        counter >>>= 8;
-      }
-      IV[i] = (byte) sum;
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-  }
-}

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CipherSuite.java

@@ -28,7 +28,8 @@ import org.apache.hadoop.util.StringUtils;
 @InterfaceAudience.Private
 public enum CipherSuite {
   UNKNOWN("Unknown", 0),
-  AES_CTR_NOPADDING("AES/CTR/NoPadding", 16);
+  AES_CTR_NOPADDING("AES/CTR/NoPadding", 16),
+  SM4_CTR_NOPADDING("SM4/CTR/NoPadding", 16);
 
   private final String name;
   private final int algoBlockSize;

+ 5 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java

@@ -101,7 +101,7 @@ public abstract class CryptoCodec implements Configurable, Closeable {
         HADOOP_SECURITY_CRYPTO_CIPHER_SUITE_DEFAULT);
     return getInstance(conf, CipherSuite.convert(name));
   }
-  
+
   private static List<Class<? extends CryptoCodec>> getCodecClasses(
       Configuration conf, CipherSuite cipherSuite) {
     List<Class<? extends CryptoCodec>> result = Lists.newArrayList();
@@ -112,6 +112,10 @@ public abstract class CryptoCodec implements Configurable, Closeable {
         .HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_KEY)) {
       codecString = conf.get(configName, CommonConfigurationKeysPublic
           .HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_DEFAULT);
+    } else if (configName.equals(CommonConfigurationKeysPublic
+            .HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_SM4_CTR_NOPADDING_KEY)){
+      codecString = conf.get(configName, CommonConfigurationKeysPublic
+              .HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_SM4_CTR_NOPADDING_DEFAULT);
     } else {
       codecString = conf.get(configName);
     }

+ 5 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoStreamUtils.java

@@ -58,10 +58,12 @@ public class CryptoStreamUtils {
         HADOOP_SECURITY_CRYPTO_BUFFER_SIZE_DEFAULT);
   }
   
-  /** AES/CTR/NoPadding is required */
+  /** AES/CTR/NoPadding or SM4/CTR/NoPadding is required. */
   public static void checkCodec(CryptoCodec codec) {
-    if (codec.getCipherSuite() != CipherSuite.AES_CTR_NOPADDING) {
-      throw new UnsupportedCodecException("AES/CTR/NoPadding is required");
+    if (codec.getCipherSuite() != CipherSuite.AES_CTR_NOPADDING &&
+            codec.getCipherSuite() != CipherSuite.SM4_CTR_NOPADDING) {
+      throw new UnsupportedCodecException(
+          "AES/CTR/NoPadding or SM4/CTR/NoPadding is required");
     }
   }
 

+ 18 - 118
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/JceAesCtrCryptoCodec.java

@@ -17,149 +17,49 @@
  */
 package org.apache.hadoop.crypto;
 
-import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.security.GeneralSecurityException;
-import java.security.SecureRandom;
-
 import javax.crypto.Cipher;
-import javax.crypto.spec.IvParameterSpec;
-import javax.crypto.spec.SecretKeySpec;
-
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-
-import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY;
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_JAVA_SECURE_RANDOM_ALGORITHM_KEY;
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_JAVA_SECURE_RANDOM_ALGORITHM_DEFAULT;
-
 /**
  * Implement the AES-CTR crypto codec using JCE provider.
  */
 @InterfaceAudience.Private
-public class JceAesCtrCryptoCodec extends AesCtrCryptoCodec {
+public class JceAesCtrCryptoCodec extends JceCtrCryptoCodec {
+
   private static final Logger LOG =
       LoggerFactory.getLogger(JceAesCtrCryptoCodec.class.getName());
-  
-  private Configuration conf;
-  private String provider;
-  private SecureRandom random;
 
   public JceAesCtrCryptoCodec() {
   }
-  
+
+  @Override
+  public Logger getLogger() {
+    return LOG;
+  }
+
   @Override
-  public Configuration getConf() {
-    return conf;
+  public CipherSuite getCipherSuite() {
+    return CipherSuite.AES_CTR_NOPADDING;
   }
-  
+
   @Override
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-    provider = conf.get(HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY);
-    final String secureRandomAlg = conf.get(
-        HADOOP_SECURITY_JAVA_SECURE_RANDOM_ALGORITHM_KEY, 
-        HADOOP_SECURITY_JAVA_SECURE_RANDOM_ALGORITHM_DEFAULT);
-    try {
-      random = (provider != null) ? 
-          SecureRandom.getInstance(secureRandomAlg, provider) : 
-            SecureRandom.getInstance(secureRandomAlg);
-    } catch (GeneralSecurityException e) {
-      LOG.warn(e.getMessage());
-      random = new SecureRandom();
-    }
+  public void calculateIV(byte[] initIV, long counter, byte[] iv) {
+    super.calculateIV(initIV, counter, iv,
+            getCipherSuite().getAlgorithmBlockSize());
   }
 
   @Override
   public Encryptor createEncryptor() throws GeneralSecurityException {
-    return new JceAesCtrCipher(Cipher.ENCRYPT_MODE, provider);
+    return new JceCtrCipher(Cipher.ENCRYPT_MODE, getProvider(),
+            getCipherSuite(), "AES");
   }
 
   @Override
   public Decryptor createDecryptor() throws GeneralSecurityException {
-    return new JceAesCtrCipher(Cipher.DECRYPT_MODE, provider);
-  }
-  
-  @Override
-  public void generateSecureRandom(byte[] bytes) {
-    random.nextBytes(bytes);
-  }  
-  
-  private static class JceAesCtrCipher implements Encryptor, Decryptor {
-    private final Cipher cipher;
-    private final int mode;
-    private boolean contextReset = false;
-    
-    public JceAesCtrCipher(int mode, String provider) 
-        throws GeneralSecurityException {
-      this.mode = mode;
-      if (provider == null || provider.isEmpty()) {
-        cipher = Cipher.getInstance(SUITE.getName());
-      } else {
-        cipher = Cipher.getInstance(SUITE.getName(), provider);
-      }
-    }
-
-    @Override
-    public void init(byte[] key, byte[] iv) throws IOException {
-      Preconditions.checkNotNull(key);
-      Preconditions.checkNotNull(iv);
-      contextReset = false;
-      try {
-        cipher.init(mode, new SecretKeySpec(key, "AES"), 
-            new IvParameterSpec(iv));
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
-    }
-
-    /**
-     * AES-CTR will consume all of the input data. It requires enough space in 
-     * the destination buffer to encrypt entire input buffer.
-     */
-    @Override
-    public void encrypt(ByteBuffer inBuffer, ByteBuffer outBuffer)
-        throws IOException {
-      process(inBuffer, outBuffer);
-    }
-    
-    /**
-     * AES-CTR will consume all of the input data. It requires enough space in
-     * the destination buffer to decrypt entire input buffer.
-     */
-    @Override
-    public void decrypt(ByteBuffer inBuffer, ByteBuffer outBuffer)
-        throws IOException {
-      process(inBuffer, outBuffer);
-    }
-    
-    private void process(ByteBuffer inBuffer, ByteBuffer outBuffer)
-        throws IOException {
-      try {
-        int inputSize = inBuffer.remaining();
-        // Cipher#update will maintain crypto context.
-        int n = cipher.update(inBuffer, outBuffer);
-        if (n < inputSize) {
-          /**
-           * Typically code will not get here. Cipher#update will consume all 
-           * input data and put result in outBuffer. 
-           * Cipher#doFinal will reset the crypto context.
-           */
-          contextReset = true;
-          cipher.doFinal(inBuffer, outBuffer);
-        }
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
-    }
-    
-    @Override
-    public boolean isContextReset() {
-      return contextReset;
-    }
+    return new JceCtrCipher(Cipher.DECRYPT_MODE, getProvider(),
+            getCipherSuite(), "AES");
   }
 }

+ 174 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/JceCtrCryptoCodec.java

@@ -0,0 +1,174 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto;
+
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.GeneralSecurityException;
+import java.security.SecureRandom;
+import java.security.Security;
+import javax.crypto.Cipher;
+import javax.crypto.spec.IvParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.slf4j.Logger;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_JAVA_SECURE_RANDOM_ALGORITHM_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_JAVA_SECURE_RANDOM_ALGORITHM_DEFAULT;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public abstract class JceCtrCryptoCodec extends CryptoCodec{
+  private Configuration conf;
+  private String provider;
+  private SecureRandom random;
+
+  public String getProvider() {
+    return provider;
+  }
+
+  public void setProvider(String provider) {
+    this.provider = provider;
+  }
+
+  public void calculateIV(byte[] initIV, long counter,
+                            byte[] iv, int blockSize) {
+    Preconditions.checkArgument(initIV.length == blockSize);
+    Preconditions.checkArgument(iv.length == blockSize);
+
+    int i = iv.length; // IV length
+    int j = 0; // counter bytes index
+    int sum = 0;
+    while(i-- > 0) {
+    // (sum >>> Byte.SIZE) is the carry for condition
+      sum = (initIV[i] & 0xff) + (sum >>> Byte.SIZE);
+      if (j++ < 8) { // Big-endian, and long is 8 bytes length
+        sum += (byte) counter & 0xff;
+        counter >>>= 8;
+      }
+      iv[i] = (byte) sum;
+    }
+  }
+
+  public void close() throws IOException {
+  }
+
+  protected abstract Logger getLogger();
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+    setProvider(conf.get(HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY));
+    if (BouncyCastleProvider.PROVIDER_NAME.equals(provider)) {
+      Security.addProvider(new BouncyCastleProvider());
+    }
+    final String secureRandomAlg =
+          conf.get(
+              HADOOP_SECURITY_JAVA_SECURE_RANDOM_ALGORITHM_KEY,
+              HADOOP_SECURITY_JAVA_SECURE_RANDOM_ALGORITHM_DEFAULT);
+
+    try {
+      random = (provider != null)
+            ? SecureRandom.getInstance(secureRandomAlg, provider)
+            : SecureRandom.getInstance(secureRandomAlg);
+    } catch(GeneralSecurityException e) {
+      getLogger().warn(e.getMessage());
+      random = new SecureRandom();
+    }
+  }
+
+  @Override
+  public void generateSecureRandom(byte[] bytes) {
+    random.nextBytes(bytes);
+  }
+
+  protected static class JceCtrCipher implements Encryptor, Decryptor {
+    private final Cipher cipher;
+    private final int mode;
+    private String name;
+    private boolean contextReset = false;
+
+    public JceCtrCipher(int mode, String provider,
+                        CipherSuite suite, String name)
+            throws GeneralSecurityException {
+
+      this.mode = mode;
+      this.name = name;
+      if(provider == null || provider.isEmpty()) {
+        cipher = Cipher.getInstance(suite.getName());
+      } else {
+        cipher = Cipher.getInstance(suite.getName(), provider);
+      }
+    }
+
+    public void init(byte[] key, byte[] iv) throws IOException {
+      Preconditions.checkNotNull(key);
+      Preconditions.checkNotNull(iv);
+      contextReset = false;
+      try {
+        cipher.init(mode, new SecretKeySpec(key, name),
+                      new IvParameterSpec(iv));
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+    }
+
+    public void encrypt(ByteBuffer inBuffer, ByteBuffer outBuffer)
+          throws IOException {
+      process(inBuffer, outBuffer);
+    }
+
+    public void decrypt(ByteBuffer inBuffer, ByteBuffer outBuffer)
+          throws IOException {
+      process(inBuffer, outBuffer);
+    }
+
+    public void process(ByteBuffer inBuffer, ByteBuffer outBuffer)
+          throws IOException {
+      try {
+        int inputSize = inBuffer.remaining();
+        // Cipher#update will maintain crypto context.
+        int n = cipher.update(inBuffer, outBuffer);
+        if (n < inputSize) {
+          /**
+           * Typically code will not get here. Cipher#update will consume all
+           * input data and put result in outBuffer.
+           * Cipher#doFinal will reset the crypto context.
+           */
+          contextReset = true;
+          cipher.doFinal(inBuffer, outBuffer);
+        }
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+    }
+
+    public boolean isContextReset() {
+      return contextReset;
+    }
+  }
+}

+ 65 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/JceSm4CtrCryptoCodec.java

@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import javax.crypto.Cipher;
+import java.security.GeneralSecurityException;
+
+/**
+ * Implement the SM4-CTR crypto codec using JCE provider.
+ */
+@InterfaceAudience.Private
+public class JceSm4CtrCryptoCodec extends JceCtrCryptoCodec {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(JceSm4CtrCryptoCodec.class.getName());
+
+  public JceSm4CtrCryptoCodec() {
+  }
+
+  @Override
+  public Logger getLogger() {
+    return LOG;
+  }
+
+  @Override
+  public CipherSuite getCipherSuite() {
+    return CipherSuite.SM4_CTR_NOPADDING;
+  }
+
+  @Override
+  public void calculateIV(byte[] initIV, long counter, byte[] iv) {
+    super.calculateIV(initIV, counter, iv,
+            getCipherSuite().getAlgorithmBlockSize());
+  }
+
+  @Override
+  public Encryptor createEncryptor() throws GeneralSecurityException {
+    return new JceCtrCipher(Cipher.ENCRYPT_MODE, getProvider(),
+            getCipherSuite(), "SM4");
+  }
+
+  @Override
+  public Decryptor createDecryptor() throws GeneralSecurityException {
+    return new JceCtrCipher(Cipher.DECRYPT_MODE, getProvider(),
+            getCipherSuite(), "SM4");
+  }
+}

+ 18 - 116
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslAesCtrCryptoCodec.java

@@ -17,35 +17,21 @@
  */
 package org.apache.hadoop.crypto;
 
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_SECURE_RANDOM_IMPL_KEY;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.security.GeneralSecurityException;
-import java.security.SecureRandom;
-import java.util.Random;
-
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.crypto.random.OpensslSecureRandom;
-import org.apache.hadoop.util.ReflectionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.security.GeneralSecurityException;
+
 /**
  * Implement the AES-CTR crypto codec using JNI into OpenSSL.
  */
 @InterfaceAudience.Private
-public class OpensslAesCtrCryptoCodec extends AesCtrCryptoCodec {
+public class OpensslAesCtrCryptoCodec extends OpensslCtrCryptoCodec {
+
   private static final Logger LOG =
-      LoggerFactory.getLogger(OpensslAesCtrCryptoCodec.class.getName());
+          LoggerFactory.getLogger(OpensslAesCtrCryptoCodec.class.getName());
 
-  private Configuration conf;
-  private Random random;
-  
   public OpensslAesCtrCryptoCodec() {
     String loadingFailureReason = OpensslCipher.getLoadingFailureReason();
     if (loadingFailureReason != null) {
@@ -54,114 +40,30 @@ public class OpensslAesCtrCryptoCodec extends AesCtrCryptoCodec {
   }
 
   @Override
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-    final Class<? extends Random> klass = conf.getClass(
-        HADOOP_SECURITY_SECURE_RANDOM_IMPL_KEY, OpensslSecureRandom.class,
-        Random.class);
-    try {
-      random = ReflectionUtils.newInstance(klass, conf);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Using " + klass.getName() + " as random number generator.");
-      }
-    } catch (Exception e) {
-      LOG.info("Unable to use " + klass.getName() + ".  Falling back to " +
-          "Java SecureRandom.", e);
-      this.random = new SecureRandom();
-    }
+  public Logger getLogger() {
+    return LOG;
   }
 
   @Override
-  public Configuration getConf() {
-    return conf;
+  public CipherSuite getCipherSuite() {
+    return CipherSuite.AES_CTR_NOPADDING;
   }
 
   @Override
-  public Encryptor createEncryptor() throws GeneralSecurityException {
-    return new OpensslAesCtrCipher(OpensslCipher.ENCRYPT_MODE);
+  public void calculateIV(byte[] initIV, long counter, byte[] iv) {
+    super.calculateIV(initIV, counter, iv,
+            getCipherSuite().getAlgorithmBlockSize());
   }
 
   @Override
-  public Decryptor createDecryptor() throws GeneralSecurityException {
-    return new OpensslAesCtrCipher(OpensslCipher.DECRYPT_MODE);
-  }
-  
-  @Override
-  public void generateSecureRandom(byte[] bytes) {
-    random.nextBytes(bytes);
+  public Encryptor createEncryptor() throws GeneralSecurityException {
+    return new OpensslCtrCipher(OpensslCipher.ENCRYPT_MODE,
+            getCipherSuite());
   }
 
   @Override
-  public void close() throws IOException {
-    try {
-      Closeable r = (Closeable) this.random;
-      r.close();
-    } catch (ClassCastException e) {
-    }
-    super.close();
-  }
-
-  private static class OpensslAesCtrCipher implements Encryptor, Decryptor {
-    private final OpensslCipher cipher;
-    private final int mode;
-    private boolean contextReset = false;
-    
-    public OpensslAesCtrCipher(int mode) throws GeneralSecurityException {
-      this.mode = mode;
-      cipher = OpensslCipher.getInstance(SUITE.getName());
-    }
-
-    @Override
-    public void init(byte[] key, byte[] iv) throws IOException {
-      Preconditions.checkNotNull(key);
-      Preconditions.checkNotNull(iv);
-      contextReset = false;
-      cipher.init(mode, key, iv);
-    }
-    
-    /**
-     * AES-CTR will consume all of the input data. It requires enough space in 
-     * the destination buffer to encrypt entire input buffer.
-     */
-    @Override
-    public void encrypt(ByteBuffer inBuffer, ByteBuffer outBuffer)
-        throws IOException {
-      process(inBuffer, outBuffer);
-    }
-    
-    /**
-     * AES-CTR will consume all of the input data. It requires enough space in
-     * the destination buffer to decrypt entire input buffer.
-     */
-    @Override
-    public void decrypt(ByteBuffer inBuffer, ByteBuffer outBuffer)
-        throws IOException {
-      process(inBuffer, outBuffer);
-    }
-    
-    private void process(ByteBuffer inBuffer, ByteBuffer outBuffer)
-        throws IOException {
-      try {
-        int inputSize = inBuffer.remaining();
-        // OpensslCipher#update will maintain crypto context.
-        int n = cipher.update(inBuffer, outBuffer);
-        if (n < inputSize) {
-          /**
-           * Typically code will not get here. OpensslCipher#update will 
-           * consume all input data and put result in outBuffer. 
-           * OpensslCipher#doFinal will reset the crypto context.
-           */
-          contextReset = true;
-          cipher.doFinal(outBuffer);
-        }
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
-    }
-    
-    @Override
-    public boolean isContextReset() {
-      return contextReset;
-    }
+  public Decryptor createDecryptor() throws GeneralSecurityException {
+    return new OpensslCtrCipher(OpensslCipher.DECRYPT_MODE,
+            getCipherSuite());
   }
 }

+ 27 - 12
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslCipher.java

@@ -45,10 +45,11 @@ public final class OpensslCipher {
       LoggerFactory.getLogger(OpensslCipher.class.getName());
   public static final int ENCRYPT_MODE = 1;
   public static final int DECRYPT_MODE = 0;
-  
-  /** Currently only support AES/CTR/NoPadding. */
+
+  /** Currently only support AES/CTR/NoPadding and SM4/CTR/NoPadding. */
   private enum AlgMode {
-    AES_CTR;
+    AES_CTR,
+    SM4_CTR;
     
     static int get(String algorithm, String mode) 
         throws NoSuchAlgorithmException {
@@ -76,6 +77,7 @@ public final class OpensslCipher {
   private long context = 0;
   private final int alg;
   private final int padding;
+  private long engine;
   
   private static final String loadingFailureReason;
 
@@ -100,10 +102,16 @@ public final class OpensslCipher {
     return loadingFailureReason;
   }
   
-  private OpensslCipher(long context, int alg, int padding) {
+  private OpensslCipher(long context, int alg, int padding, long engine) {
     this.context = context;
     this.alg = alg;
     this.padding = padding;
+    this.engine = engine;
+  }
+
+  public static OpensslCipher getInstance(String transformation)
+      throws NoSuchAlgorithmException, NoSuchPaddingException {
+    return getInstance(transformation, null);
   }
   
   /**
@@ -112,6 +120,8 @@ public final class OpensslCipher {
    * 
    * @param transformation the name of the transformation, e.g., 
    * AES/CTR/NoPadding.
+   * @param engineId the openssl engine to use.if not set,
+   * defalut engine will be used.
    * @return OpensslCipher an <code>OpensslCipher</code> object
    * @throws NoSuchAlgorithmException if <code>transformation</code> is null, 
    * empty, in an invalid format, or if Openssl doesn't implement the 
@@ -119,13 +129,15 @@ public final class OpensslCipher {
    * @throws NoSuchPaddingException if <code>transformation</code> contains 
    * a padding scheme that is not available.
    */
-  public static final OpensslCipher getInstance(String transformation) 
+  public static OpensslCipher getInstance(
+      String transformation, String engineId)
       throws NoSuchAlgorithmException, NoSuchPaddingException {
     Transform transform = tokenizeTransformation(transformation);
     int algMode = AlgMode.get(transform.alg, transform.mode);
     int padding = Padding.get(transform.padding);
     long context = initContext(algMode, padding);
-    return new OpensslCipher(context, algMode, padding);
+    long engine = (engineId != null) ? initEngine(engineId) : 0;
+    return new OpensslCipher(context, algMode, padding, engine);
   }
   
   /** Nested class for algorithm, mode and padding. */
@@ -175,7 +187,7 @@ public final class OpensslCipher {
    * @param iv crypto iv
    */
   public void init(int mode, byte[] key, byte[] iv) {
-    context = init(context, mode, alg, padding, key, iv);
+    context = init(context, mode, alg, padding, key, iv, engine);
   }
   
   /**
@@ -255,8 +267,9 @@ public final class OpensslCipher {
   /** Forcibly clean the context. */
   public void clean() {
     if (context != 0) {
-      clean(context);
+      clean(context, engine);
       context = 0;
+      engine = 0;
     }
   }
 
@@ -273,17 +286,19 @@ public final class OpensslCipher {
   private native static void initIDs();
   
   private native static long initContext(int alg, int padding);
+
+  private native static long initEngine(String engineId);
   
   private native long init(long context, int mode, int alg, int padding, 
-      byte[] key, byte[] iv);
-  
+      byte[] key, byte[] iv, long engineNum);
+
   private native int update(long context, ByteBuffer input, int inputOffset, 
       int inputLength, ByteBuffer output, int outputOffset, int maxOutputLength);
   
   private native int doFinal(long context, ByteBuffer output, int offset, 
       int maxOutputLength);
   
-  private native void clean(long context);
-  
+  private native void clean(long ctx, long engineNum);
+
   public native static String getLibraryName();
 }

+ 189 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslCtrCryptoCodec.java

@@ -0,0 +1,189 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.random.OpensslSecureRandom;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.GeneralSecurityException;
+import java.security.SecureRandom;
+import java.util.Random;
+import org.slf4j.Logger;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_SECURE_RANDOM_IMPL_KEY;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public abstract class OpensslCtrCryptoCodec extends CryptoCodec{
+
+  private Configuration conf;
+  private Random random;
+  private String engineId;
+
+  public String getEngineId() {
+    return engineId;
+  }
+
+  public void setEngineId(String engineId) {
+    this.engineId = engineId;
+  }
+
+  public Random getRandom() {
+    return random;
+  }
+
+  public void setRandom(Random random) {
+    this.random = random;
+  }
+
+
+  public void calculateIV(byte[] initIV, long counter,
+                            byte[] iv, int blockSize) {
+    Preconditions.checkArgument(initIV.length == blockSize);
+    Preconditions.checkArgument(iv.length == blockSize);
+
+    int i = iv.length; // IV length
+    int j = 0; // counter bytes index
+    int sum = 0;
+    while(i-- > 0){
+      // (sum >>> Byte.SIZE) is the carry for condition
+      sum = (initIV[i] & 0xff) + (sum >>> Byte.SIZE);
+      if (j++ < 8) { // Big-endian, and long is 8 bytes length
+        sum += (byte) counter & 0xff;
+        counter >>>= 8;
+      }
+      iv[i] = (byte) sum;
+    }
+  }
+
+  protected abstract Logger getLogger();
+
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+    final Class<? extends Random> klass = conf.getClass(
+            HADOOP_SECURITY_SECURE_RANDOM_IMPL_KEY,
+            OpensslSecureRandom.class,
+            Random.class);
+    try {
+      random = ReflectionUtils.newInstance(klass, conf);
+      getLogger().debug("Using " + klass.getName() +
+                " as random number generator.");
+    } catch (Exception e) {
+      getLogger().info("Unable to use " + klass.getName() +
+              ".  Falling back to " +
+              "Java SecureRandom.", e);
+      this.random = new SecureRandom();
+    }
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void generateSecureRandom(byte[] bytes) {
+    random.nextBytes(bytes);
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (this.random instanceof Closeable) {
+      Closeable r = (Closeable) this.random;
+      IOUtils.cleanupWithLogger(getLogger(), r);
+    }
+  }
+
+  protected static class OpensslCtrCipher implements Encryptor, Decryptor {
+    private final OpensslCipher cipher;
+    private final int mode;
+    private boolean contextReset = false;
+
+    public OpensslCtrCipher(int mode, CipherSuite suite, String engineId)
+                throws GeneralSecurityException {
+      this.mode = mode;
+      cipher = OpensslCipher.getInstance(suite.getName(), engineId);
+    }
+
+    public OpensslCtrCipher(int mode, CipherSuite suite)
+            throws GeneralSecurityException {
+      this.mode = mode;
+      cipher = OpensslCipher.getInstance(suite.getName());
+    }
+
+    @Override
+    public void init(byte[] key, byte[] iv) throws IOException {
+      Preconditions.checkNotNull(key);
+      Preconditions.checkNotNull(iv);
+      contextReset = false;
+      cipher.init(mode, key, iv);
+    }
+
+    /**
+     * AES-CTR will consume all of the input data. It requires enough space in
+     * the destination buffer to encrypt entire input buffer.
+     */
+    @Override
+    public void encrypt(ByteBuffer inBuffer, ByteBuffer outBuffer)
+                throws IOException {
+      process(inBuffer, outBuffer);
+    }
+
+    /**
+     *  AES-CTR will consume all of the input data. It requires enough space in
+     * the destination buffer to decrypt entire input buffer.
+     */
+    @Override
+    public void decrypt(ByteBuffer inBuffer, ByteBuffer outBuffer)
+                throws IOException {
+      process(inBuffer, outBuffer);
+    }
+
+    private void process(ByteBuffer inBuffer, ByteBuffer outBuffer)
+                throws IOException {
+      try {
+        int inputSize = inBuffer.remaining();
+        // OpensslCipher#update will maintain crypto context.
+        int n = cipher.update(inBuffer, outBuffer);
+        if (n < inputSize) {
+        /**
+         * Typically code will not get here. OpensslCipher#update will
+         * consume all input data and put result in outBuffer.
+         * OpensslCipher#doFinal will reset the crypto context.
+         */
+          contextReset = true;
+          cipher.doFinal(outBuffer);
+        }
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+    }
+
+    @Override
+    public boolean isContextReset() {
+      return contextReset;
+    }
+  }
+}

+ 79 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslSm4CtrCryptoCodec.java

@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.security.GeneralSecurityException;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_OPENSSL_ENGINE_ID_KEY;
+
+/**
+ * Implement the SM4-CTR crypto codec using JNI into OpenSSL.
+ */
+@InterfaceAudience.Private
+public class OpensslSm4CtrCryptoCodec extends OpensslCtrCryptoCodec {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OpensslSm4CtrCryptoCodec.class.getName());
+
+  public OpensslSm4CtrCryptoCodec() {
+    String loadingFailureReason = OpensslCipher.getLoadingFailureReason();
+    if (loadingFailureReason != null) {
+      throw new RuntimeException(loadingFailureReason);
+    }
+  }
+
+  @Override
+  public Logger getLogger() {
+    return LOG;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    super.setConf(conf);
+    setEngineId(conf.get(HADOOP_SECURITY_OPENSSL_ENGINE_ID_KEY));
+  }
+
+  @Override
+  public CipherSuite getCipherSuite() {
+    return CipherSuite.SM4_CTR_NOPADDING;
+  }
+
+  @Override
+  public void calculateIV(byte[] initIV, long counter, byte[] iv) {
+    super.calculateIV(initIV, counter, iv,
+            getCipherSuite().getAlgorithmBlockSize());
+  }
+
+  @Override
+  public Encryptor createEncryptor() throws GeneralSecurityException {
+    return new OpensslCtrCipher(OpensslCipher.ENCRYPT_MODE,
+            getCipherSuite(), getEngineId());
+  }
+
+  @Override
+  public Decryptor createDecryptor() throws GeneralSecurityException {
+    return new OpensslCtrCipher(OpensslCipher.DECRYPT_MODE,
+            getCipherSuite(), getEngineId());
+  }
+}

+ 9 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java

@@ -26,12 +26,14 @@ import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
 import java.nio.charset.StandardCharsets;
 import java.security.NoSuchAlgorithmException;
+import java.security.Security;
 import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
 import com.google.gson.stream.JsonReader;
 import com.google.gson.stream.JsonWriter;
 import org.apache.commons.lang3.builder.EqualsBuilder;
@@ -43,6 +45,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 
 import javax.crypto.KeyGenerator;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_JCEKS_KEY_SERIALFILTER;
 
 /**
@@ -409,10 +412,14 @@ public abstract class KeyProvider implements Closeable {
     // java.security.UnrecoverableKeyException in JDK 8u171.
     if(System.getProperty(JCEKS_KEY_SERIAL_FILTER) == null) {
       String serialFilter =
-          conf.get(HADOOP_SECURITY_CRYPTO_JCEKS_KEY_SERIALFILTER,
-              JCEKS_KEY_SERIALFILTER_DEFAULT);
+              conf.get(HADOOP_SECURITY_CRYPTO_JCEKS_KEY_SERIALFILTER,
+                      JCEKS_KEY_SERIALFILTER_DEFAULT);
       System.setProperty(JCEKS_KEY_SERIAL_FILTER, serialFilter);
     }
+    String jceProvider = conf.get(HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY);
+    if (BouncyCastleProvider.PROVIDER_NAME.equals(jceProvider)) {
+      Security.addProvider(new BouncyCastleProvider());
+    }
   }
 
   /**

+ 7 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/random/OsSecureRandom.java

@@ -24,6 +24,7 @@ import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.util.Random;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
@@ -73,7 +74,12 @@ public class OsSecureRandom extends Random implements Closeable, Configurable {
 
   public OsSecureRandom() {
   }
-  
+
+  @VisibleForTesting
+  public boolean isClosed() {
+    return stream == null;
+  }
+
   @Override
   synchronized public void setConf(Configuration conf) {
     this.conf = conf;

+ 17 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java

@@ -21,7 +21,9 @@ package org.apache.hadoop.fs;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.crypto.CipherSuite;
 import org.apache.hadoop.crypto.JceAesCtrCryptoCodec;
+import org.apache.hadoop.crypto.JceSm4CtrCryptoCodec;
 import org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec;
+import org.apache.hadoop.crypto.OpensslSm4CtrCryptoCodec;
 
 /** 
  * This class contains constants for configuration keys used
@@ -701,10 +703,18 @@ public class CommonConfigurationKeysPublic {
       HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_KEY =
       HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX
           + CipherSuite.AES_CTR_NOPADDING.getConfigSuffix();
+  public static final String
+      HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_SM4_CTR_NOPADDING_KEY =
+      HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX
+          + CipherSuite.SM4_CTR_NOPADDING.getConfigSuffix();
   public static final String
       HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_DEFAULT =
       OpensslAesCtrCryptoCodec.class.getName() + "," +
           JceAesCtrCryptoCodec.class.getName();
+  public static final String
+      HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_SM4_CTR_NOPADDING_DEFAULT =
+      OpensslSm4CtrCryptoCodec.class.getName() + "," +
+          JceSm4CtrCryptoCodec.class.getName();
   /**
    * @see
    * <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">
@@ -873,6 +883,13 @@ public class CommonConfigurationKeysPublic {
    * <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">
    * core-default.xml</a>
    */
+  public static final String HADOOP_SECURITY_OPENSSL_ENGINE_ID_KEY =
+          "hadoop.security.openssl.engine.id";
+  /**
+   * @see
+   * <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">
+   * core-default.xml</a>
+   */
   public static final String HADOOP_SECURITY_SECURE_RANDOM_DEVICE_FILE_PATH_KEY = 
     "hadoop.security.random.device.file.path";
   public static final String HADOOP_SECURITY_SECURE_RANDOM_DEVICE_FILE_PATH_DEFAULT = 

+ 111 - 8
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/crypto/OpensslCipher.c

@@ -46,6 +46,13 @@ static int (*dlsym_EVP_CipherUpdate)(EVP_CIPHER_CTX *, unsigned char *,  \
 static int (*dlsym_EVP_CipherFinal_ex)(EVP_CIPHER_CTX *, unsigned char *, int *);
 static EVP_CIPHER * (*dlsym_EVP_aes_256_ctr)(void);
 static EVP_CIPHER * (*dlsym_EVP_aes_128_ctr)(void);
+#if OPENSSL_VERSION_NUMBER >= 0x10100001L
+static EVP_CIPHER * (*dlsym_EVP_sm4_ctr)(void);
+static int (*dlsym_OPENSSL_init_crypto)(uint64_t opts, \
+            const OPENSSL_INIT_SETTINGS *settings);
+static ENGINE * (*dlsym_ENGINE_by_id)(const char *id);
+static int (*dlsym_ENGINE_free)(ENGINE *);
+#endif
 static void *openssl;
 #endif
 
@@ -84,6 +91,18 @@ static __dlsym_EVP_CipherUpdate dlsym_EVP_CipherUpdate;
 static __dlsym_EVP_CipherFinal_ex dlsym_EVP_CipherFinal_ex;
 static __dlsym_EVP_aes_256_ctr dlsym_EVP_aes_256_ctr;
 static __dlsym_EVP_aes_128_ctr dlsym_EVP_aes_128_ctr;
+#if OPENSSL_VERSION_NUMBER >= 0x10101001L
+typedef EVP_CIPHER * (__cdecl *__dlsym_EVP_sm4_ctr)(void);
+typedef int (__cdecl *__dlsym_OPENSSL_init_crypto)(uint64_t opts, \
+             const OPENSSL_INIT_SETTINGS *settings);
+typedef ENGINE * (__cdecl *__dlsym_ENGINE_by_id)(const char *id);
+typedef int (__cdecl *__dlsym_ENGINE_free)(ENGINE *e);
+
+static __dlsym_EVP_sm4_ctr dlsym_EVP_sm4_ctr;
+static __dlsym_OPENSSL_init_crypto dlsym_OPENSSL_init_crypto;
+static __dlsym_ENGINE_by_id dlsym_ENGINE_by_id;
+static __dlsym_ENGINE_free dlsym_ENGINE_free;
+#endif
 static HMODULE openssl;
 #endif
 
@@ -102,6 +121,15 @@ static void loadAesCtr(JNIEnv *env)
 #endif
 }
 
+static void loadSm4Ctr(JNIEnv *env)
+{
+#ifdef UNIX
+#if OPENSSL_VERSION_NUMBER >= 0x10101001L
+   LOAD_DYNAMIC_SYMBOL(dlsym_EVP_sm4_ctr, env, openssl, "EVP_sm4_ctr");
+#endif
+#endif
+}
+
 JNIEXPORT void JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_initIDs
     (JNIEnv *env, jclass clazz)
 {
@@ -153,6 +181,14 @@ JNIEXPORT void JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_initIDs
                       "EVP_CipherUpdate");
   LOAD_DYNAMIC_SYMBOL(dlsym_EVP_CipherFinal_ex, env, openssl,  \
                       "EVP_CipherFinal_ex");
+#if OPENSSL_VERSION_NUMBER >= 0x10101001L
+  LOAD_DYNAMIC_SYMBOL(dlsym_OPENSSL_init_crypto, env, openssl,  \
+                      "OPENSSL_init_crypto");
+  LOAD_DYNAMIC_SYMBOL(dlsym_ENGINE_by_id, env, openssl,  \
+                      "ENGINE_by_id");
+  LOAD_DYNAMIC_SYMBOL(dlsym_ENGINE_free, env, openssl,  \
+                      "ENGINE_free");
+#endif
 #endif
 
 #ifdef WINDOWS
@@ -185,14 +221,31 @@ JNIEXPORT void JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_initIDs
                       env, openssl, "EVP_CipherUpdate");
   LOAD_DYNAMIC_SYMBOL(__dlsym_EVP_CipherFinal_ex, dlsym_EVP_CipherFinal_ex,  \
                       env, openssl, "EVP_CipherFinal_ex");
+#if OPENSSL_VERSION_NUMBER >= 0x10101001L
+  LOAD_DYNAMIC_SYMBOL(__dlsym_OPENSSL_init_crypto, dlsym_OPENSSL_init_crypto,  \
+                      env, openssl, "OPENSSL_init_crypto");
+  LOAD_DYNAMIC_SYMBOL(__dlsym_ENGINE_by_id, dlsym_ENGINE_by_id,  \
+                      env, openssl, "ENGINE_by_id");
+  LOAD_DYNAMIC_SYMBOL(__dlsym_ENGINE_free, dlsym_ENGINE_free,  \
+                      env, openssl, "ENGINE_by_free");
+#endif
 #endif
 
   loadAesCtr(env);
+  loadSm4Ctr(env);
+#if OPENSSL_VERSION_NUMBER >= 0x10101001L
+  int ret = dlsym_OPENSSL_init_crypto(OPENSSL_INIT_LOAD_CONFIG, NULL);
+  if(!ret) {
+    THROW(env, "java/lang/UnsatisfiedLinkError", \
+        "Openssl init crypto failed");
+    return;
+  }
+#endif
   jthrowable jthr = (*env)->ExceptionOccurred(env);
   if (jthr) {
     (*env)->DeleteLocalRef(env, jthr);
     THROW(env, "java/lang/UnsatisfiedLinkError",  \
-        "Cannot find AES-CTR support, is your version of Openssl new enough?");
+        "Cannot find AES-CTR/SM4-CTR support, is your version of Openssl new enough?");
     return;
   }
 }
@@ -200,7 +253,7 @@ JNIEXPORT void JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_initIDs
 JNIEXPORT jlong JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_initContext
     (JNIEnv *env, jclass clazz, jint alg, jint padding)
 {
-  if (alg != AES_CTR) {
+  if (alg != AES_CTR && alg != SM4_CTR) {
     THROW(env, "java/security/NoSuchAlgorithmException", NULL);
     return (jlong)0;
   }
@@ -209,11 +262,27 @@ JNIEXPORT jlong JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_initContext
     return (jlong)0;
   }
   
-  if (dlsym_EVP_aes_256_ctr == NULL || dlsym_EVP_aes_128_ctr == NULL) {
+  if (alg == AES_CTR && (dlsym_EVP_aes_256_ctr == NULL || dlsym_EVP_aes_128_ctr == NULL)) {
     THROW(env, "java/security/NoSuchAlgorithmException",  \
         "Doesn't support AES CTR.");
     return (jlong)0;
   }
+
+  if (alg == SM4_CTR) {
+    int ret = 0;
+#if OPENSSL_VERSION_NUMBER >= 0x10101001L
+    if (dlsym_EVP_sm4_ctr == NULL) {
+      ret = 1;
+    }
+#else
+    ret = 1;
+#endif
+    if (ret) {
+      THROW(env, "java/security/NoSuchAlgorithmException",  \
+              "Doesn't support SM4 CTR.");
+      return (jlong)0;
+    }
+  }
   
   // Create and initialize a EVP_CIPHER_CTX
   EVP_CIPHER_CTX *context = dlsym_EVP_CIPHER_CTX_new();
@@ -225,7 +294,29 @@ JNIEXPORT jlong JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_initContext
   return JLONG(context);
 }
 
-// Only supports AES-CTR currently
+JNIEXPORT jlong JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_initEngine
+    (JNIEnv *env, jclass clazz, jstring engineId)
+{
+  ENGINE *e = NULL;
+
+#if OPENSSL_VERSION_NUMBER >= 0x10101001L
+  if (engineId != NULL) {
+    const char *id = (*env)->GetStringUTFChars(env, engineId, NULL);
+    if (id != NULL) {
+      e = dlsym_ENGINE_by_id(id);
+      (*env)->ReleaseStringUTFChars(env, engineId, id);
+    }
+  }
+#endif
+
+  if (e == NULL) {
+    return (jlong)0;
+  } else {
+    return JLONG(e);
+  }
+}
+
+// Only supports AES-CTR & SM4-CTR currently
 static EVP_CIPHER * getEvpCipher(int alg, int keyLen)
 {
   EVP_CIPHER *cipher = NULL;
@@ -235,13 +326,19 @@ static EVP_CIPHER * getEvpCipher(int alg, int keyLen)
     } else if (keyLen == KEY_LENGTH_128) {
       cipher = dlsym_EVP_aes_128_ctr();
     }
+  } else if (alg == SM4_CTR) {
+    if (keyLen == KEY_LENGTH_128) {
+#if OPENSSL_VERSION_NUMBER >= 0x10101001L
+      cipher = dlsym_EVP_sm4_ctr();
+#endif
+    }
   }
   return cipher;
 }
 
 JNIEXPORT jlong JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_init
     (JNIEnv *env, jobject object, jlong ctx, jint mode, jint alg, jint padding, 
-    jbyteArray key, jbyteArray iv)
+    jbyteArray key, jbyteArray iv, jlong engine)
 {
   int jKeyLen = (*env)->GetArrayLength(env, key);
   int jIvLen = (*env)->GetArrayLength(env, iv);
@@ -275,9 +372,10 @@ JNIEXPORT jlong JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_init
     THROW(env, "java/lang/InternalError", "Cannot get bytes array for iv.");
     return (jlong)0;
   }
-  
+
+  ENGINE *e = LONG_TO_ENGINE(engine);
   int rc = dlsym_EVP_CipherInit_ex(context, getEvpCipher(alg, jKeyLen),  \
-      NULL, (unsigned char *)jKey, (unsigned char *)jIv, mode == ENCRYPT_MODE);
+      e, (unsigned char *)jKey, (unsigned char *)jIv, mode == ENCRYPT_MODE);
   (*env)->ReleaseByteArrayElements(env, key, jKey, 0);
   (*env)->ReleaseByteArrayElements(env, iv, jIv, 0);
   if (rc == 0) {
@@ -406,12 +504,17 @@ JNIEXPORT jint JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_doFinal
 }
 
 JNIEXPORT void JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_clean
-    (JNIEnv *env, jobject object, jlong ctx) 
+    (JNIEnv *env, jobject object, jlong ctx, jlong engine)
 {
   EVP_CIPHER_CTX *context = CONTEXT(ctx);
   if (context) {
     dlsym_EVP_CIPHER_CTX_free(context);
   }
+
+  ENGINE *e = LONG_TO_ENGINE(engine);
+  if (e) {
+    dlsym_ENGINE_free(e);
+  }
 }
 
 JNIEXPORT jstring JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_getLibraryName

+ 7 - 1
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/crypto/org_apache_hadoop_crypto.h

@@ -46,6 +46,11 @@
  */
 #define JLONG(context) ((jlong)((ptrdiff_t)(context)))
 
+/**
+ * A helper macro to convert long to ENGINE.
+ */
+#define LONG_TO_ENGINE(engine) ((ENGINE*)((ptrdiff_t)(engine)))
+
 #define KEY_LENGTH_128 16
 #define KEY_LENGTH_256 32
 #define IV_LENGTH 16
@@ -53,8 +58,9 @@
 #define ENCRYPT_MODE 1
 #define DECRYPT_MODE 0
 
-/** Currently only support AES/CTR/NoPadding. */
+/** Currently only support AES/CTR/NoPadding & SM4/CTR/NoPadding. */
 #define AES_CTR 0
+#define SM4_CTR 1
 #define NOPADDING 0
 #define PKCSPADDING 1
 

+ 20 - 0
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -3308,6 +3308,26 @@
   </description>
 </property>
 
+<property>
+  <name>hadoop.security.crypto.codec.classes.sm4.ctr.nopadding</name>
+  <value>org.apache.hadoop.crypto.OpensslSm4CtrCryptoCodec, org.apache.hadoop.crypto.JceSm4CtrCryptoCodec</value>
+  <description>
+    Comma-separated list of crypto codec implementations for SM4/CTR/NoPadding.
+    The first implementation will be used if available, others are fallbacks.
+  </description>
+</property>
+
+<property>
+  <name>hadoop.security.openssl.engine.id</name>
+  <value></value>
+  <description>
+    The Openssl provided an engine mechanism that allow to specify third-party software
+    encryption library or hardware encryption device for encryption. The engine ID could
+    be vendor defined and will be passed to openssl, more info please see:
+    https://github.com/openssl/openssl/blob/master/README.ENGINE
+  </description>
+</property>
+
 <property>
   <name>hadoop.security.crypto.cipher.suite</name>
   <value>AES/CTR/NoPadding</value>

+ 2 - 0
hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md

@@ -203,6 +203,8 @@ Setting `dfs.encrypt.data.transfer.cipher.suites` to `AES/CTR/NoPadding` activat
 
 AES offers the greatest cryptographic strength and the best performance. At this time, 3DES and RC4 have been used more often in Hadoop clusters.
 
+You can also set `dfs.encrypt.data.transfer.cipher.suites` to `SM4/CTR/NoPadding` to activates SM4 encryption. By default, this is unspecified. The SM4 key bit length can be configured by setting `dfs.encrypt.data.transfer.cipher.key.bitlength` to 128, 192 or 256. The default is 128.
+
 ### Data Encryption on HTTP
 
 Data transfer between Web-console and clients are protected by using SSL(HTTPS). SSL configuration is recommended but not required to configure Hadoop security with Kerberos.

+ 88 - 12
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoCodec.java

@@ -17,6 +17,12 @@
  */
 package org.apache.hadoop.crypto;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.
+    HADOOP_SECURITY_CRYPTO_CIPHER_SUITE_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.
+    HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_SM4_CTR_NOPADDING_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.
+        HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -43,6 +49,7 @@ import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
 import com.google.common.primitives.Longs;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -57,10 +64,14 @@ public class TestCryptoCodec {
   private Configuration conf = new Configuration();
   private int count = 10000;
   private int seed = new Random().nextInt();
-  private final String jceCodecClass = 
+  private final String jceAesCodecClass =
       "org.apache.hadoop.crypto.JceAesCtrCryptoCodec";
-  private final String opensslCodecClass = 
+  private final String jceSm4CodecClass =
+      "org.apache.hadoop.crypto.JceSm4CtrCryptoCodec";
+  private final String opensslAesCodecClass =
       "org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec";
+  private final String opensslSm4CodecClass =
+      "org.apache.hadoop.crypto.OpensslSm4CtrCryptoCodec";
   
   @Before
   public void setUp() throws IOException {
@@ -77,15 +88,49 @@ public class TestCryptoCodec {
       Assume.assumeTrue(false);
     }
     Assert.assertEquals(null, OpensslCipher.getLoadingFailureReason());
-    cryptoCodecTest(conf, seed, 0, jceCodecClass, jceCodecClass, iv);
-    cryptoCodecTest(conf, seed, count, jceCodecClass, jceCodecClass, iv);
-    cryptoCodecTest(conf, seed, count, jceCodecClass, opensslCodecClass, iv);
+    cryptoCodecTest(conf, seed, 0,
+        jceAesCodecClass, jceAesCodecClass, iv);
+    cryptoCodecTest(conf, seed, count,
+        jceAesCodecClass, jceAesCodecClass, iv);
+    cryptoCodecTest(conf, seed, count,
+        jceAesCodecClass, opensslAesCodecClass, iv);
     // Overflow test, IV: xx xx xx xx xx xx xx xx ff ff ff ff ff ff ff ff 
     for(int i = 0; i < 8; i++) {
       iv[8 + i] = (byte) 0xff;
     }
-    cryptoCodecTest(conf, seed, count, jceCodecClass, jceCodecClass, iv);
-    cryptoCodecTest(conf, seed, count, jceCodecClass, opensslCodecClass, iv);
+    cryptoCodecTest(conf, seed, count,
+        jceAesCodecClass, jceAesCodecClass, iv);
+    cryptoCodecTest(conf, seed, count,
+        jceAesCodecClass, opensslAesCodecClass, iv);
+  }
+
+  @Test(timeout=120000)
+  public void testJceSm4CtrCryptoCodec() throws Exception {
+    GenericTestUtils.assumeInNativeProfile();
+    if (!NativeCodeLoader.buildSupportsOpenssl()) {
+      LOG.warn("Skipping test since openSSL library not loaded");
+      Assume.assumeTrue(false);
+    }
+    conf.set(HADOOP_SECURITY_CRYPTO_CIPHER_SUITE_KEY, "SM4/CTR/NoPadding");
+    conf.set(HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_SM4_CTR_NOPADDING_KEY,
+        JceSm4CtrCryptoCodec.class.getName());
+    conf.set(HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY,
+            BouncyCastleProvider.PROVIDER_NAME);
+    Assert.assertEquals(null, OpensslCipher.getLoadingFailureReason());
+    cryptoCodecTest(conf, seed, 0,
+        jceSm4CodecClass, jceSm4CodecClass, iv);
+    cryptoCodecTest(conf, seed, count,
+        jceSm4CodecClass, jceSm4CodecClass, iv);
+    cryptoCodecTest(conf, seed, count,
+        jceSm4CodecClass, opensslSm4CodecClass, iv);
+    // Overflow test, IV: xx xx xx xx xx xx xx xx ff ff ff ff ff ff ff ff
+    for(int i = 0; i < 8; i++) {
+      iv[8 + i] = (byte) 0xff;
+    }
+    cryptoCodecTest(conf, seed, count,
+        jceSm4CodecClass, jceSm4CodecClass, iv);
+    cryptoCodecTest(conf, seed, count,
+        jceSm4CodecClass, opensslSm4CodecClass, iv);
   }
   
   @Test(timeout=120000)
@@ -96,15 +141,46 @@ public class TestCryptoCodec {
       Assume.assumeTrue(false);
     }
     Assert.assertEquals(null, OpensslCipher.getLoadingFailureReason());
-    cryptoCodecTest(conf, seed, 0, opensslCodecClass, opensslCodecClass, iv);
-    cryptoCodecTest(conf, seed, count, opensslCodecClass, opensslCodecClass, iv);
-    cryptoCodecTest(conf, seed, count, opensslCodecClass, jceCodecClass, iv);
+    cryptoCodecTest(conf, seed, 0,
+        opensslAesCodecClass, opensslAesCodecClass, iv);
+    cryptoCodecTest(conf, seed, count,
+        opensslAesCodecClass, opensslAesCodecClass, iv);
+    cryptoCodecTest(conf, seed, count,
+        opensslAesCodecClass, jceAesCodecClass, iv);
     // Overflow test, IV: xx xx xx xx xx xx xx xx ff ff ff ff ff ff ff ff 
     for(int i = 0; i < 8; i++) {
       iv[8 + i] = (byte) 0xff;
     }
-    cryptoCodecTest(conf, seed, count, opensslCodecClass, opensslCodecClass, iv);
-    cryptoCodecTest(conf, seed, count, opensslCodecClass, jceCodecClass, iv);
+    cryptoCodecTest(conf, seed, count,
+        opensslAesCodecClass, opensslAesCodecClass, iv);
+    cryptoCodecTest(conf, seed, count,
+        opensslAesCodecClass, jceAesCodecClass, iv);
+  }
+
+  @Test(timeout=120000)
+  public void testOpensslSm4CtrCryptoCodec() throws Exception {
+    GenericTestUtils.assumeInNativeProfile();
+    if (!NativeCodeLoader.buildSupportsOpenssl()) {
+      LOG.warn("Skipping test since openSSL library not loaded");
+      Assume.assumeTrue(false);
+    }
+    conf.set(HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY,
+            BouncyCastleProvider.PROVIDER_NAME);
+    Assert.assertEquals(null, OpensslCipher.getLoadingFailureReason());
+    cryptoCodecTest(conf, seed, 0,
+        opensslSm4CodecClass, opensslSm4CodecClass, iv);
+    cryptoCodecTest(conf, seed, count,
+        opensslSm4CodecClass, opensslSm4CodecClass, iv);
+    cryptoCodecTest(conf, seed, count,
+        opensslSm4CodecClass, jceSm4CodecClass, iv);
+    // Overflow test, IV: xx xx xx xx xx xx xx xx ff ff ff ff ff ff ff ff
+    for(int i = 0; i < 8; i++) {
+      iv[8 + i] = (byte) 0xff;
+    }
+    cryptoCodecTest(conf, seed, count,
+        opensslSm4CodecClass, opensslSm4CodecClass, iv);
+    cryptoCodecTest(conf, seed, count,
+        opensslSm4CodecClass, jceSm4CodecClass, iv);
   }
   
   private void cryptoCodecTest(Configuration conf, int seed, int count, 

+ 48 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithJceSm4CtrCryptoCodec.java

@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto;
+
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.junit.BeforeClass;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.
+    HADOOP_SECURITY_CRYPTO_CIPHER_SUITE_KEY;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY;
+
+public class TestCryptoStreamsWithJceSm4CtrCryptoCodec extends
+    TestCryptoStreams {
+
+  @BeforeClass
+  public static void init() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(HADOOP_SECURITY_CRYPTO_CIPHER_SUITE_KEY, "SM4/CTR/NoPadding");
+    conf.set(HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY,
+            BouncyCastleProvider.PROVIDER_NAME);
+    conf.set(
+         CommonConfigurationKeysPublic.
+             HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_SM4_CTR_NOPADDING_KEY,
+         JceSm4CtrCryptoCodec.class.getName());
+    codec = CryptoCodec.getInstance(conf);
+    assertThat(JceSm4CtrCryptoCodec.class.getCanonicalName()).
+        isEqualTo(codec.getClass().getCanonicalName());
+  }
+}

+ 10 - 7
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java

@@ -21,14 +21,16 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.random.OsSecureRandom;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.test.Whitebox;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_KEY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CIPHER_SUITE_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_KEY;
 
 public class TestCryptoStreamsWithOpensslAesCtrCryptoCodec 
     extends TestCryptoStreams {
@@ -51,6 +53,7 @@ public class TestCryptoStreamsWithOpensslAesCtrCryptoCodec
   public void testCodecClosesRandom() throws Exception {
     GenericTestUtils.assumeInNativeProfile();
     Configuration conf = new Configuration();
+    conf.set(HADOOP_SECURITY_CRYPTO_CIPHER_SUITE_KEY, "AES/CTR/NoPadding");
     conf.set(HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_AES_CTR_NOPADDING_KEY,
         OpensslAesCtrCryptoCodec.class.getName());
     conf.set(
@@ -61,13 +64,13 @@ public class TestCryptoStreamsWithOpensslAesCtrCryptoCodec
         "Unable to instantiate codec " + OpensslAesCtrCryptoCodec.class
             .getName() + ", is the required " + "version of OpenSSL installed?",
         codecWithRandom);
-    OsSecureRandom random =
-        (OsSecureRandom) Whitebox.getInternalState(codecWithRandom, "random");
+    OsSecureRandom random = (OsSecureRandom)
+            ((OpensslAesCtrCryptoCodec) codecWithRandom).getRandom();
     // trigger the OsSecureRandom to create an internal FileInputStream
     random.nextBytes(new byte[10]);
-    assertNotNull(Whitebox.getInternalState(random, "stream"));
+    assertFalse(random.isClosed());
     // verify closing the codec closes the codec's random's stream.
     codecWithRandom.close();
-    assertNull(Whitebox.getInternalState(random, "stream"));
+    assertTrue(random.isClosed());
   }
 }

+ 79 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslSm4CtrCryptoCodec.java

@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.random.OsSecureRandom;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.
+    HADOOP_SECURITY_CRYPTO_CIPHER_SUITE_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.
+    HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_SM4_CTR_NOPADDING_KEY;
+
+public class TestCryptoStreamsWithOpensslSm4CtrCryptoCodec
+    extends TestCryptoStreams {
+
+  @BeforeClass
+  public static void init() throws Exception {
+    GenericTestUtils.assumeInNativeProfile();
+    Configuration conf = new Configuration();
+    conf.set(HADOOP_SECURITY_CRYPTO_CIPHER_SUITE_KEY, "SM4/CTR/NoPadding");
+    conf.set(HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_SM4_CTR_NOPADDING_KEY,
+            OpensslSm4CtrCryptoCodec.class.getName());
+    codec = CryptoCodec.getInstance(conf);
+    assertNotNull("Unable to instantiate codec " +
+            OpensslSm4CtrCryptoCodec.class.getName() + ", is the required "
+            + "version of OpenSSL installed?", codec);
+    assertEquals(OpensslSm4CtrCryptoCodec.class.getCanonicalName(),
+            codec.getClass().getCanonicalName());
+  }
+
+  @Test
+  public void testCodecClosesRandom() throws Exception {
+    GenericTestUtils.assumeInNativeProfile();
+    Configuration conf = new Configuration();
+    conf.set(HADOOP_SECURITY_CRYPTO_CIPHER_SUITE_KEY, "SM4/CTR/NoPadding");
+    conf.set(HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_SM4_CTR_NOPADDING_KEY,
+            OpensslSm4CtrCryptoCodec.class.getName());
+    conf.set(
+            CommonConfigurationKeysPublic.
+                    HADOOP_SECURITY_SECURE_RANDOM_IMPL_KEY,
+            OsSecureRandom.class.getName());
+    CryptoCodec codecWithRandom = CryptoCodec.getInstance(conf);
+    assertNotNull("Unable to instantiate codec " +
+            OpensslSm4CtrCryptoCodec.class.getName() + ", is the required "
+            + "version of OpenSSL installed?", codecWithRandom);
+    OsSecureRandom random = (OsSecureRandom)
+            ((OpensslSm4CtrCryptoCodec) codecWithRandom).getRandom();
+    // trigger the OsSecureRandom to create an internal FileInputStream
+    random.nextBytes(new byte[10]);
+    assertFalse(random.isClosed());
+    // verify closing the codec closes the codec's random's stream.
+    codecWithRandom.close();
+    assertTrue(random.isClosed());
+  }
+}

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java

@@ -305,20 +305,22 @@ public final class DataTransferSaslUtil {
   public static CipherOption negotiateCipherOption(Configuration conf,
       List<CipherOption> options) throws IOException {
     // Negotiate cipher suites if configured.  Currently, the only supported
-    // cipher suite is AES/CTR/NoPadding, but the protocol allows multiple
-    // values for future expansion.
+    // cipher suite is AES/CTR/NoPadding or SM4/CTR/NoPadding, but the protocol
+    // allows multiple values for future expansion.
     String cipherSuites = conf.get(DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY);
     if (cipherSuites == null || cipherSuites.isEmpty()) {
       return null;
     }
-    if (!cipherSuites.equals(CipherSuite.AES_CTR_NOPADDING.getName())) {
+    if (!cipherSuites.equals(CipherSuite.AES_CTR_NOPADDING.getName()) &&
+        !cipherSuites.equals(CipherSuite.SM4_CTR_NOPADDING.getName())) {
       throw new IOException(String.format("Invalid cipher suite, %s=%s",
           DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY, cipherSuites));
     }
     if (options != null) {
       for (CipherOption option : options) {
         CipherSuite suite = option.getCipherSuite();
-        if (suite == CipherSuite.AES_CTR_NOPADDING) {
+        if (suite == CipherSuite.AES_CTR_NOPADDING ||
+            suite == CipherSuite.SM4_CTR_NOPADDING) {
           int keyLen = conf.getInt(
               DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_KEY,
               DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_DEFAULT) / 8;

+ 9 - 4
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferClient.java

@@ -548,14 +548,19 @@ public class SaslDataTransferClient {
           DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY);
       if (requestedQopContainsPrivacy(saslProps)) {
         // Negotiate cipher suites if configured.  Currently, the only supported
-        // cipher suite is AES/CTR/NoPadding, but the protocol allows multiple
-        // values for future expansion.
+        // cipher suite is AES/CTR/NoPadding or SM4/CTR/Nopadding,
+        // but the protocol allows multiple values for future expansion.
         if (cipherSuites != null && !cipherSuites.isEmpty()) {
-          if (!cipherSuites.equals(CipherSuite.AES_CTR_NOPADDING.getName())) {
+          CipherOption option = null;
+          if (cipherSuites.equals(CipherSuite.AES_CTR_NOPADDING.getName())) {
+            option = new CipherOption(CipherSuite.AES_CTR_NOPADDING);
+          } else if (cipherSuites.equals(
+              CipherSuite.SM4_CTR_NOPADDING.getName())) {
+            option = new CipherOption(CipherSuite.SM4_CTR_NOPADDING);
+          } else {
             throw new IOException(String.format("Invalid cipher suite, %s=%s",
                 DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY, cipherSuites));
           }
-          CipherOption option = new CipherOption(CipherSuite.AES_CTR_NOPADDING);
           cipherOptions = Lists.newArrayListWithCapacity(1);
           cipherOptions.add(option);
         }

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java

@@ -565,6 +565,8 @@ public class PBHelperClient {
     switch (proto) {
     case AES_CTR_NOPADDING:
       return CipherSuite.AES_CTR_NOPADDING;
+    case SM4_CTR_NOPADDING:
+      return CipherSuite.SM4_CTR_NOPADDING;
     default:
       // Set to UNKNOWN and stash the unknown enum value
       CipherSuite suite = CipherSuite.UNKNOWN;
@@ -603,6 +605,8 @@ public class PBHelperClient {
       return HdfsProtos.CipherSuiteProto.UNKNOWN;
     case AES_CTR_NOPADDING:
       return HdfsProtos.CipherSuiteProto.AES_CTR_NOPADDING;
+    case SM4_CTR_NOPADDING:
+      return HdfsProtos.CipherSuiteProto.SM4_CTR_NOPADDING;
     default:
       return null;
     }

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto

@@ -296,6 +296,7 @@ message DataEncryptionKeyProto {
 enum CipherSuiteProto {
     UNKNOWN = 1;
     AES_CTR_NOPADDING = 2;
+    SM4_CTR_NOPADDING = 3;
 }
 
 /**

+ 7 - 1
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/TransparentEncryption.md

@@ -111,11 +111,17 @@ Default: `org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec, org.apache.hadoop.c
 
 Comma-separated list of crypto codec implementations for AES/CTR/NoPadding. The first implementation will be used if available, others are fallbacks.
 
+#### hadoop.security.crypto.codec.classes.sm4.ctr.nopadding
+
+Default: `org.apache.hadoop.crypto.OpensslSm4CtrCryptoCodec, org.apache.hadoop.crypto.JceSm4CtrCryptoCodec`
+
+Comma-separated list of crypto codec implementations for SM4/CTR/NoPadding. The first implementation will be used if available, others are fallbacks.
+
 #### hadoop.security.crypto.cipher.suite
 
 Default: `AES/CTR/NoPadding`
 
-Cipher suite for crypto codec.
+Cipher suite for crypto codec, now AES/CTR/NoPadding and SM4/CTR/NoPadding are supported.
 
 #### hadoop.security.crypto.jce.provider