Browse Source

HADOOP-19152. Do not hard code security providers. (#6739) (#7439)

Co-authored-by: Tsz-Wo Nicholas Sze <szetszwo@apache.org>
Signed-off-by: Chris Nauroth <cnauroth@apache.org>
Cheng Pan 2 months ago
parent
commit
b3ae9d1eb6

+ 77 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoUtils.java

@@ -0,0 +1,77 @@
+/*
+ * 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 java.security.Provider;
+import java.security.Security;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.store.LogExactlyOnce;
+
+/** Utility methods for the crypto related features. */
+@InterfaceAudience.Private
+public final class CryptoUtils {
+  static final Logger LOG = LoggerFactory.getLogger(CryptoUtils.class);
+  private static final LogExactlyOnce LOG_FAILED_TO_LOAD_CLASS = new LogExactlyOnce(LOG);
+  private static final LogExactlyOnce LOG_FAILED_TO_ADD_PROVIDER = new LogExactlyOnce(LOG);
+
+  private static final String BOUNCY_CASTLE_PROVIDER_CLASS
+      = "org.bouncycastle.jce.provider.BouncyCastleProvider";
+  static final String BOUNCY_CASTLE_PROVIDER_NAME = "BC";
+
+  /**
+   * Get the security provider value specified in
+   * {@link CommonConfigurationKeysPublic#HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY}
+   * from the given conf.
+   *
+   * @param conf the configuration
+   * @return the configured provider, if there is any; otherwise, return an empty string.
+   */
+  public static String getJceProvider(Configuration conf) {
+    final String provider = conf.getTrimmed(
+        CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY, "");
+    final boolean autoAdd = conf.getBoolean(
+        CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_AUTO_ADD_KEY,
+        CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_AUTO_ADD_DEFAULT);
+
+    // For backward compatible, auto-add BOUNCY_CASTLE_PROVIDER_CLASS when the provider is "BC".
+    if (autoAdd && BOUNCY_CASTLE_PROVIDER_NAME.equals(provider)) {
+      try {
+        // Use reflection in order to avoid statically loading the class.
+        final Class<?> clazz = Class.forName(BOUNCY_CASTLE_PROVIDER_CLASS);
+        Security.addProvider((Provider) clazz.getConstructor().newInstance());
+        LOG.debug("Successfully added security provider {}", provider);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Trace", new Throwable());
+        }
+      } catch (ClassNotFoundException e) {
+        LOG_FAILED_TO_LOAD_CLASS.warn("Failed to load " + BOUNCY_CASTLE_PROVIDER_CLASS, e);
+      } catch (Exception e) {
+        LOG_FAILED_TO_ADD_PROVIDER.warn("Failed to add security provider for {}", provider, e);
+      }
+    }
+    return provider;
+  }
+
+  private CryptoUtils() { }
+}

+ 3 - 12
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/JceCtrCryptoCodec.java

@@ -17,7 +17,6 @@
  */
  */
 package org.apache.hadoop.crypto;
 package org.apache.hadoop.crypto;
 
 
-import org.bouncycastle.jce.provider.BouncyCastleProvider;
 import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -27,13 +26,11 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;
 import java.security.GeneralSecurityException;
 import java.security.GeneralSecurityException;
 import java.security.SecureRandom;
 import java.security.SecureRandom;
-import java.security.Security;
 import javax.crypto.Cipher;
 import javax.crypto.Cipher;
 import javax.crypto.spec.IvParameterSpec;
 import javax.crypto.spec.IvParameterSpec;
 import javax.crypto.spec.SecretKeySpec;
 import javax.crypto.spec.SecretKeySpec;
 import org.slf4j.Logger;
 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_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_JAVA_SECURE_RANDOM_ALGORITHM_DEFAULT;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_JAVA_SECURE_RANDOM_ALGORITHM_DEFAULT;
 
 
@@ -48,10 +45,6 @@ public abstract class JceCtrCryptoCodec extends CryptoCodec{
     return provider;
     return provider;
   }
   }
 
 
-  public void setProvider(String provider) {
-    this.provider = provider;
-  }
-
   public void calculateIV(byte[] initIV, long counter,
   public void calculateIV(byte[] initIV, long counter,
                             byte[] iv, int blockSize) {
                             byte[] iv, int blockSize) {
     Preconditions.checkArgument(initIV.length == blockSize);
     Preconditions.checkArgument(initIV.length == blockSize);
@@ -82,17 +75,15 @@ public abstract class JceCtrCryptoCodec extends CryptoCodec{
 
 
   public void setConf(Configuration conf) {
   public void setConf(Configuration conf) {
     this.conf = conf;
     this.conf = conf;
-    setProvider(conf.get(HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY));
-    if (BouncyCastleProvider.PROVIDER_NAME.equals(provider)) {
-      Security.addProvider(new BouncyCastleProvider());
-    }
+    this.provider = CryptoUtils.getJceProvider(conf);
+
     final String secureRandomAlg =
     final String secureRandomAlg =
           conf.get(
           conf.get(
               HADOOP_SECURITY_JAVA_SECURE_RANDOM_ALGORITHM_KEY,
               HADOOP_SECURITY_JAVA_SECURE_RANDOM_ALGORITHM_KEY,
               HADOOP_SECURITY_JAVA_SECURE_RANDOM_ALGORITHM_DEFAULT);
               HADOOP_SECURITY_JAVA_SECURE_RANDOM_ALGORITHM_DEFAULT);
 
 
     try {
     try {
-      random = (provider != null)
+      random = (provider != null && !provider.isEmpty())
             ? SecureRandom.getInstance(secureRandomAlg, provider)
             ? SecureRandom.getInstance(secureRandomAlg, provider)
             : SecureRandom.getInstance(secureRandomAlg);
             : SecureRandom.getInstance(secureRandomAlg);
     } catch(GeneralSecurityException e) {
     } catch(GeneralSecurityException e) {

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

@@ -26,7 +26,6 @@ import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
 import java.io.OutputStreamWriter;
 import java.nio.charset.StandardCharsets;
 import java.nio.charset.StandardCharsets;
 import java.security.NoSuchAlgorithmException;
 import java.security.NoSuchAlgorithmException;
-import java.security.Security;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.Date;
 import java.util.Date;
@@ -35,17 +34,16 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Objects;
 
 
-import org.bouncycastle.jce.provider.BouncyCastleProvider;
 import com.google.gson.stream.JsonReader;
 import com.google.gson.stream.JsonReader;
 import com.google.gson.stream.JsonWriter;
 import com.google.gson.stream.JsonWriter;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.CryptoUtils;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 
 
 import javax.crypto.KeyGenerator;
 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;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_JCEKS_KEY_SERIALFILTER;
 
 
 /**
 /**
@@ -410,10 +408,7 @@ public abstract class KeyProvider implements Closeable {
                       JCEKS_KEY_SERIALFILTER_DEFAULT);
                       JCEKS_KEY_SERIALFILTER_DEFAULT);
       System.setProperty(JCEKS_KEY_SERIAL_FILTER, serialFilter);
       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());
-    }
+    CryptoUtils.getJceProvider(conf);
   }
   }
 
 
   /**
   /**

+ 20 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/package-info.java

@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+/** Crypto related classes. */
+package org.apache.hadoop.crypto;

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

@@ -773,6 +773,9 @@ public class CommonConfigurationKeysPublic {
    */
    */
   public static final String HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY =
   public static final String HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY =
     "hadoop.security.crypto.jce.provider";
     "hadoop.security.crypto.jce.provider";
+  public static final String HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_AUTO_ADD_KEY =
+      "hadoop.security.crypto.jce.provider.auto-add";
+  public static final boolean HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_AUTO_ADD_DEFAULT = true;
   /**
   /**
    * @see
    * @see
    * <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">
    * <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">

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

@@ -3633,6 +3633,21 @@ The switch to turn S3A auditing on or off.
   <value></value>
   <value></value>
   <description>
   <description>
     The JCE provider name used in CryptoCodec.
     The JCE provider name used in CryptoCodec.
+    If this value is set, the corresponding provider must be added to the provider list.
+    The provider may be added statically in the java.security file, or
+    dynamically by calling the java.security.Security.addProvider(..) method, or
+    automatically (only for org.bouncycastle.jce.provider.BouncyCastleProvider)
+    by setting "hadoop.security.crypto.jce.provider.auto-add" to true
+  </description>
+</property>
+
+<property>
+  <name>hadoop.security.crypto.jce.provider.auto-add</name>
+  <value>true</value>
+  <description>
+    Automatically add the org.bouncycastle.jce.provider.BouncyCastleProvider
+    when the value in "hadoop.security.crypto.jce.provider" is set
+    to BouncyCastleProvider.PROVIDER_NAME.
   </description>
   </description>
 </property>
 </property>
 
 

+ 89 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoUtils.java

@@ -0,0 +1,89 @@
+/*
+ * 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.test.GenericTestUtils;
+import org.assertj.core.api.Assertions;
+import org.bouncycastle.jce.provider.BouncyCastleProvider;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.event.Level;
+
+import java.security.Provider;
+import java.security.Security;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_AUTO_ADD_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_AUTO_ADD_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY;
+
+/** Test {@link CryptoUtils}. */
+public class TestCryptoUtils {
+  static {
+    GenericTestUtils.setLogLevel(CryptoUtils.LOG, Level.TRACE);
+  }
+
+  @Test(timeout = 1_000)
+  public void testProviderName() {
+    Assert.assertEquals(CryptoUtils.BOUNCY_CASTLE_PROVIDER_NAME, BouncyCastleProvider.PROVIDER_NAME);
+  }
+
+  static void assertRemoveProvider() {
+    Security.removeProvider(BouncyCastleProvider.PROVIDER_NAME);
+    Assert.assertNull(Security.getProvider(BouncyCastleProvider.PROVIDER_NAME));
+  }
+
+  static void assertSetProvider(Configuration conf) {
+    conf.set(HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY, CryptoUtils.BOUNCY_CASTLE_PROVIDER_NAME);
+    final String providerFromConf = CryptoUtils.getJceProvider(conf);
+    Assert.assertEquals(CryptoUtils.BOUNCY_CASTLE_PROVIDER_NAME, providerFromConf);
+  }
+
+  @Test(timeout = 5_000)
+  public void testAutoAddDisabled() {
+    assertRemoveProvider();
+
+    final Configuration conf = new Configuration();
+    conf.setBoolean(HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_AUTO_ADD_KEY, false);
+
+    assertSetProvider(conf);
+
+    Assert.assertNull(Security.getProvider(BouncyCastleProvider.PROVIDER_NAME));
+  }
+
+  @Test(timeout = 5_000)
+  public void testAutoAddEnabled() {
+    assertRemoveProvider();
+
+    final Configuration conf = new Configuration();
+    Assertions.assertThat(conf.get(HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_AUTO_ADD_KEY))
+        .describedAs("conf: " + HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_AUTO_ADD_KEY)
+        .isEqualToIgnoringCase("true");
+    Assert.assertTrue(HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_AUTO_ADD_DEFAULT);
+
+    conf.set(HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY, CryptoUtils.BOUNCY_CASTLE_PROVIDER_NAME);
+    final String providerFromConf = CryptoUtils.getJceProvider(conf);
+    Assert.assertEquals(CryptoUtils.BOUNCY_CASTLE_PROVIDER_NAME, providerFromConf);
+
+    final Provider provider = Security.getProvider(BouncyCastleProvider.PROVIDER_NAME);
+    Assertions.assertThat(provider)
+        .isInstanceOf(BouncyCastleProvider.class);
+
+    assertRemoveProvider();
+  }
+}