You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "hasnain-db (via GitHub)" <gi...@apache.org> on 2023/10/16 15:53:55 UTC

[PR] [SPARK-45541][CORE] Add SSLFactory [spark]

hasnain-db opened a new pull request, #43386:
URL: https://github.com/apache/spark/pull/43386

   ### What changes were proposed in this pull request?
   
   As titled - add a factory which supports creating SSL engines, and a corresponding builder for it. This will be used in a follow up PR by the `TransportContext` and related files to add SSL support.
   
   ### Why are the changes needed?
   
   We need a mechanism to initialize the appropriate SSL implementation with the configured settings (such as protocol, ciphers, etc) for RPC SSL support.
   
   ### Does this PR introduce _any_ user-facing change?
   
   No
   
   ### How was this patch tested?
   
   existing tests. This will be more thoroughly tested in a follow up PR which adds callsites to it. It has been integration tested as part of https://github.com/apache/spark/pull/42685
   
   ### Was this patch authored or co-authored using generative AI tooling?
   
   No


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45541][CORE] Add SSLFactory [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm closed pull request #43386: [SPARK-45541][CORE] Add SSLFactory
URL: https://github.com/apache/spark/pull/43386


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45541][CORE] Add SSLFactory [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm commented on code in PR #43386:
URL: https://github.com/apache/spark/pull/43386#discussion_r1367691734


##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,471 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private static final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");
+      }
+    }
+    return SslProvider.JDK;
+  }
+
+  public void destroy() {
+    if (trustManagers != null) {
+      for (int i = 0; i < trustManagers.length; i++) {
+        if (trustManagers[i] instanceof ReloadingX509TrustManager) {
+          try {
+            ((ReloadingX509TrustManager) trustManagers[i]).destroy();
+          } catch (InterruptedException ex) {
+            logger.info("Interrupted while destroying trust manager: " + ex.toString(), ex);
+          }
+        }
+      }
+      trustManagers = null;
+    }
+
+    keyManagers = null;
+    jdkSslContext = null;
+    nettyClientSslContext = null;
+    nettyServerSslContext = null;
+    requestedProtocol = null;
+    requestedCiphers = null;
+  }
+
+  /**
+   * Builder class to construct instances of {@link SSLFactory} with specific options
+   */
+  public static class Builder {
+    private String requestedProtocol;
+    private String[] requestedCiphers;
+    private File keyStore;
+    private String keyStorePassword;
+    private File privateKey;
+    private String keyPassword;
+    private File certChain;
+    private File trustStore;
+    private String trustStorePassword;
+    private boolean trustStoreReloadingEnabled;
+    private int trustStoreReloadIntervalMs;
+    private boolean openSslEnabled;
+
+    /**
+     * Sets the requested protocol, i.e., "TLSv1.2", "TLSv1.1", etc
+     *
+     * @param requestedProtocol The requested protocol
+     * @return The builder object
+     */
+    public Builder requestedProtocol(String requestedProtocol) {
+      this.requestedProtocol = requestedProtocol == null ? "TLSv1.2" : requestedProtocol;
+      return this;
+    }
+
+    /**
+     * Sets the requested cipher suites, i.e., "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", etc
+     *
+     * @param requestedCiphers The requested ciphers
+     * @return The builder object
+     */
+    public Builder requestedCiphers(String[] requestedCiphers) {
+      this.requestedCiphers = requestedCiphers;
+      return this;
+    }
+
+    /**
+     * Sets the Keystore and Keystore password
+     *
+     * @param keyStore The key store file to use
+     * @param keyStorePassword The password for the key store
+     * @return The builder object
+     */
+    public Builder keyStore(File keyStore, String keyStorePassword) {
+      this.keyStore = keyStore;
+      this.keyStorePassword = keyStorePassword;
+      return this;
+    }
+
+    /**
+     * Sets a PKCS#8 private key file in PEM format
+     *
+     * @param privateKey The private key file to use
+     * @return The builder object
+     */
+    public Builder privateKey(File privateKey) {
+      this.privateKey = privateKey;
+      return this;
+    }
+
+    /**
+     * Sets the Key password
+     *
+     * @param keyPassword The password for the private key
+     * @return The builder object
+     */
+    public Builder keyPassword(String keyPassword) {
+      this.keyPassword = keyPassword;
+      return this;
+    }
+
+    /**
+     * Sets a X.509 certificate chain file in PEM format
+     *
+     * @param certChain The certificate chain file to use
+     * @return The builder object
+     */
+    public Builder certChain(File certChain) {
+      this.certChain = certChain;
+      return this;
+    }
+
+    /**
+     * @param enabled Whether to use the OpenSSL implementation
+     * @return The builder object
+     */
+    public Builder openSslEnabled(boolean enabled) {
+      this.openSslEnabled = enabled;
+      return this;
+    }
+
+    /**
+     * Sets the trust-store, trust-store password, whether to use a Reloading TrustStore,
+     * and the trust-store reload interval, if enabled
+     *
+     * @param trustStore The trust store file to use
+     * @param trustStorePassword The password for the trust store
+     * @param trustStoreReloadingEnabled Whether trust store reloading is enabled
+     * @param trustStoreReloadIntervalMs The interval at which to reload the trust store file
+     * @return The builder object
+     */
+    public Builder trustStore(
+        File trustStore, String trustStorePassword,
+        boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs) {
+      this.trustStore = trustStore;
+      this.trustStorePassword = trustStorePassword;
+      this.trustStoreReloadingEnabled = trustStoreReloadingEnabled;
+      this.trustStoreReloadIntervalMs = trustStoreReloadIntervalMs;
+      return this;
+    }
+
+    /**
+     * Builds our {@link SSLFactory}
+     *
+     * @return The built {@link SSLFactory}
+     */
+    public SSLFactory build() {
+      return new SSLFactory(this);
+    }
+  }
+
+  /**
+   * Returns an initialized {@link SSLContext}
+   *
+   * @param requestedProtocol The requested protocol to use
+   * @param keyManagers The list of key managers to use
+   * @param trustManagers The list of trust managers to use
+   * @return The built {@link SSLContext}
+   * @throws GeneralSecurityException
+   */
+  private static SSLContext createSSLContext(
+      String requestedProtocol,
+      KeyManager[] keyManagers,
+      TrustManager[] trustManagers) throws GeneralSecurityException {
+    SSLContext sslContext = SSLContext.getInstance(requestedProtocol);
+    sslContext.init(keyManagers, trustManagers, null);
+    return sslContext;
+  }
+
+  /**
+   * Creates a new {@link SSLEngine}.
+   * Note that currently client auth is not supported
+   *
+   * @param isClient Whether the engine is used in a client context
+   * @param allocator The {@link ByteBufAllocator to use}
+   * @return A valid {@link SSLEngine}.
+   */
+  public SSLEngine createSSLEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine = createEngine(isClient, allocator);
+    engine.setUseClientMode(isClient);
+    engine.setNeedClientAuth(false);
+    engine.setEnabledProtocols(enabledProtocols(engine, requestedProtocol));
+    engine.setEnabledCipherSuites(enabledCipherSuites(engine, requestedCiphers));
+    return engine;
+  }
+
+  private SSLEngine createEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine;
+    if (isClient) {
+      if (nettyClientSslContext != null) {
+        engine = nettyClientSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    } else {
+      if (nettyServerSslContext != null) {
+        engine = nettyServerSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    }
+    return engine;
+  }
+
+  private static TrustManager[] credulousTrustStoreManagers() {
+    return new TrustManager[]{new X509TrustManager() {
+      @Override
+      public void checkClientTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public void checkServerTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public X509Certificate[] getAcceptedIssuers() {
+        return null;
+      }
+    }};
+  }
+
+  private static TrustManager[] trustStoreManagers(
+      File trustStore, String trustStorePassword,
+      boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs)
+          throws IOException, GeneralSecurityException {
+    if (trustStore == null || !trustStore.exists()) {
+      return credulousTrustStoreManagers();
+    } else {
+      if (trustStoreReloadingEnabled) {
+        ReloadingX509TrustManager reloading = new ReloadingX509TrustManager(
+          KeyStore.getDefaultType(), trustStore, trustStorePassword, trustStoreReloadIntervalMs);
+        reloading.init();
+        return new TrustManager[]{reloading};
+      } else {
+        return defaultTrustManagers(trustStore, trustStorePassword);
+      }
+    }
+  }
+
+  private static TrustManager[] defaultTrustManagers(File trustStore, String trustStorePassword)
+      throws IOException, KeyStoreException, CertificateException, NoSuchAlgorithmException {
+    try (InputStream input = Files.asByteSource(trustStore).openStream()) {
+      KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+      char[] passwordCharacters = trustStorePassword != null?
+        trustStorePassword.toCharArray() : null;
+      ks.load(input, passwordCharacters);
+      TrustManagerFactory tmf = TrustManagerFactory.getInstance(
+        TrustManagerFactory.getDefaultAlgorithm());
+      tmf.init(ks);
+      return tmf.getTrustManagers();
+    }
+  }
+
+  private static KeyManager[] keyManagers(File keyStore, String keyStorePassword)
+      throws NoSuchAlgorithmException, CertificateException,
+          KeyStoreException, IOException, UnrecoverableKeyException {
+    KeyManagerFactory factory = KeyManagerFactory.getInstance(
+      KeyManagerFactory.getDefaultAlgorithm());
+    char[] passwordCharacters = keyStorePassword != null? keyStorePassword.toCharArray() : null;
+    factory.init(loadKeyStore(keyStore, keyStorePassword), passwordCharacters);

Review Comment:
   You can pass `passwordCharacters` to `loadKeyStore`, no ? It is a private method and so no external mods to the array ?
   
   Or can either `factory.init` or `KeyStore.load` change it ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45541][CORE] Add SSLFactory [spark]

Posted by "hasnain-db (via GitHub)" <gi...@apache.org>.
hasnain-db commented on code in PR #43386:
URL: https://github.com/apache/spark/pull/43386#discussion_r1367797191


##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,471 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private static final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");
+      }
+    }
+    return SslProvider.JDK;
+  }
+
+  public void destroy() {
+    if (trustManagers != null) {
+      for (int i = 0; i < trustManagers.length; i++) {
+        if (trustManagers[i] instanceof ReloadingX509TrustManager) {
+          try {
+            ((ReloadingX509TrustManager) trustManagers[i]).destroy();
+          } catch (InterruptedException ex) {
+            logger.info("Interrupted while destroying trust manager: " + ex.toString(), ex);
+          }
+        }
+      }
+      trustManagers = null;
+    }
+
+    keyManagers = null;
+    jdkSslContext = null;
+    nettyClientSslContext = null;
+    nettyServerSslContext = null;
+    requestedProtocol = null;
+    requestedCiphers = null;
+  }
+
+  /**
+   * Builder class to construct instances of {@link SSLFactory} with specific options
+   */
+  public static class Builder {
+    private String requestedProtocol;
+    private String[] requestedCiphers;
+    private File keyStore;
+    private String keyStorePassword;
+    private File privateKey;
+    private String keyPassword;
+    private File certChain;
+    private File trustStore;
+    private String trustStorePassword;
+    private boolean trustStoreReloadingEnabled;
+    private int trustStoreReloadIntervalMs;
+    private boolean openSslEnabled;
+
+    /**
+     * Sets the requested protocol, i.e., "TLSv1.2", "TLSv1.1", etc
+     *
+     * @param requestedProtocol The requested protocol
+     * @return The builder object
+     */
+    public Builder requestedProtocol(String requestedProtocol) {
+      this.requestedProtocol = requestedProtocol == null ? "TLSv1.2" : requestedProtocol;
+      return this;
+    }
+
+    /**
+     * Sets the requested cipher suites, i.e., "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", etc
+     *
+     * @param requestedCiphers The requested ciphers
+     * @return The builder object
+     */
+    public Builder requestedCiphers(String[] requestedCiphers) {
+      this.requestedCiphers = requestedCiphers;
+      return this;
+    }
+
+    /**
+     * Sets the Keystore and Keystore password
+     *
+     * @param keyStore The key store file to use
+     * @param keyStorePassword The password for the key store
+     * @return The builder object
+     */
+    public Builder keyStore(File keyStore, String keyStorePassword) {
+      this.keyStore = keyStore;
+      this.keyStorePassword = keyStorePassword;
+      return this;
+    }
+
+    /**
+     * Sets a PKCS#8 private key file in PEM format
+     *
+     * @param privateKey The private key file to use
+     * @return The builder object
+     */
+    public Builder privateKey(File privateKey) {
+      this.privateKey = privateKey;
+      return this;
+    }
+
+    /**
+     * Sets the Key password
+     *
+     * @param keyPassword The password for the private key
+     * @return The builder object
+     */
+    public Builder keyPassword(String keyPassword) {
+      this.keyPassword = keyPassword;
+      return this;
+    }
+
+    /**
+     * Sets a X.509 certificate chain file in PEM format
+     *
+     * @param certChain The certificate chain file to use
+     * @return The builder object
+     */
+    public Builder certChain(File certChain) {
+      this.certChain = certChain;
+      return this;
+    }
+
+    /**
+     * @param enabled Whether to use the OpenSSL implementation
+     * @return The builder object
+     */
+    public Builder openSslEnabled(boolean enabled) {
+      this.openSslEnabled = enabled;
+      return this;
+    }
+
+    /**
+     * Sets the trust-store, trust-store password, whether to use a Reloading TrustStore,
+     * and the trust-store reload interval, if enabled
+     *
+     * @param trustStore The trust store file to use
+     * @param trustStorePassword The password for the trust store
+     * @param trustStoreReloadingEnabled Whether trust store reloading is enabled
+     * @param trustStoreReloadIntervalMs The interval at which to reload the trust store file
+     * @return The builder object
+     */
+    public Builder trustStore(
+        File trustStore, String trustStorePassword,
+        boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs) {
+      this.trustStore = trustStore;
+      this.trustStorePassword = trustStorePassword;
+      this.trustStoreReloadingEnabled = trustStoreReloadingEnabled;
+      this.trustStoreReloadIntervalMs = trustStoreReloadIntervalMs;
+      return this;
+    }
+
+    /**
+     * Builds our {@link SSLFactory}
+     *
+     * @return The built {@link SSLFactory}
+     */
+    public SSLFactory build() {
+      return new SSLFactory(this);
+    }
+  }
+
+  /**
+   * Returns an initialized {@link SSLContext}
+   *
+   * @param requestedProtocol The requested protocol to use
+   * @param keyManagers The list of key managers to use
+   * @param trustManagers The list of trust managers to use
+   * @return The built {@link SSLContext}
+   * @throws GeneralSecurityException
+   */
+  private static SSLContext createSSLContext(
+      String requestedProtocol,
+      KeyManager[] keyManagers,
+      TrustManager[] trustManagers) throws GeneralSecurityException {
+    SSLContext sslContext = SSLContext.getInstance(requestedProtocol);
+    sslContext.init(keyManagers, trustManagers, null);
+    return sslContext;
+  }
+
+  /**
+   * Creates a new {@link SSLEngine}.
+   * Note that currently client auth is not supported
+   *
+   * @param isClient Whether the engine is used in a client context
+   * @param allocator The {@link ByteBufAllocator to use}
+   * @return A valid {@link SSLEngine}.
+   */
+  public SSLEngine createSSLEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine = createEngine(isClient, allocator);
+    engine.setUseClientMode(isClient);
+    engine.setNeedClientAuth(false);
+    engine.setEnabledProtocols(enabledProtocols(engine, requestedProtocol));
+    engine.setEnabledCipherSuites(enabledCipherSuites(engine, requestedCiphers));
+    return engine;
+  }
+
+  private SSLEngine createEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine;
+    if (isClient) {
+      if (nettyClientSslContext != null) {
+        engine = nettyClientSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    } else {
+      if (nettyServerSslContext != null) {
+        engine = nettyServerSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    }
+    return engine;
+  }
+
+  private static TrustManager[] credulousTrustStoreManagers() {
+    return new TrustManager[]{new X509TrustManager() {
+      @Override
+      public void checkClientTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public void checkServerTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public X509Certificate[] getAcceptedIssuers() {
+        return null;
+      }
+    }};
+  }
+
+  private static TrustManager[] trustStoreManagers(
+      File trustStore, String trustStorePassword,
+      boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs)
+          throws IOException, GeneralSecurityException {
+    if (trustStore == null || !trustStore.exists()) {
+      return credulousTrustStoreManagers();
+    } else {
+      if (trustStoreReloadingEnabled) {
+        ReloadingX509TrustManager reloading = new ReloadingX509TrustManager(
+          KeyStore.getDefaultType(), trustStore, trustStorePassword, trustStoreReloadIntervalMs);
+        reloading.init();
+        return new TrustManager[]{reloading};
+      } else {
+        return defaultTrustManagers(trustStore, trustStorePassword);
+      }
+    }
+  }
+
+  private static TrustManager[] defaultTrustManagers(File trustStore, String trustStorePassword)
+      throws IOException, KeyStoreException, CertificateException, NoSuchAlgorithmException {
+    try (InputStream input = Files.asByteSource(trustStore).openStream()) {
+      KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+      char[] passwordCharacters = trustStorePassword != null?
+        trustStorePassword.toCharArray() : null;
+      ks.load(input, passwordCharacters);
+      TrustManagerFactory tmf = TrustManagerFactory.getInstance(
+        TrustManagerFactory.getDefaultAlgorithm());
+      tmf.init(ks);
+      return tmf.getTrustManagers();
+    }
+  }
+
+  private static KeyManager[] keyManagers(File keyStore, String keyStorePassword)
+      throws NoSuchAlgorithmException, CertificateException,
+          KeyStoreException, IOException, UnrecoverableKeyException {
+    KeyManagerFactory factory = KeyManagerFactory.getInstance(
+      KeyManagerFactory.getDefaultAlgorithm());
+    char[] passwordCharacters = keyStorePassword != null? keyStorePassword.toCharArray() : null;
+    factory.init(loadKeyStore(keyStore, keyStorePassword), passwordCharacters);

Review Comment:
   good catch, fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45541][CORE] Add SSLFactory [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm commented on code in PR #43386:
URL: https://github.com/apache/spark/pull/43386#discussion_r1366403143


##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,470 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private static final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");
+      }
+    }
+    return SslProvider.JDK;
+  }
+
+  public void destroy() {
+    if (trustManagers != null) {
+      for (int i = 0; i < trustManagers.length; i++) {
+        if (trustManagers[i] instanceof ReloadingX509TrustManager) {
+          try {
+            ((ReloadingX509TrustManager) trustManagers[i]).destroy();
+          } catch (InterruptedException ex) {
+            logger.info("Interrupted while destroying trust manager: " + ex.toString(), ex);
+          }
+        }
+      }
+      trustManagers = null;
+    }
+
+    keyManagers = null;
+    jdkSslContext = null;
+    nettyClientSslContext = null;
+    nettyServerSslContext = null;
+    requestedProtocol = null;
+    requestedCiphers = null;
+  }
+
+  /**
+   * Builder class to construct instances of {@link SSLFactory} with specific options
+   */
+  public static class Builder {
+    private String requestedProtocol;
+    private String[] requestedCiphers;
+    private File keyStore;
+    private String keyStorePassword;
+    private File privateKey;
+    private String keyPassword;
+    private File certChain;
+    private File trustStore;
+    private String trustStorePassword;
+    private boolean trustStoreReloadingEnabled;
+    private int trustStoreReloadIntervalMs;
+    private boolean openSslEnabled;
+
+    /**
+     * Sets the requested protocol, i.e., "TLSv1.2", "TLSv1.1", etc
+     *
+     * @param requestedProtocol The requested protocol
+     * @return The builder object
+     */
+    public Builder requestedProtocol(String requestedProtocol) {
+      this.requestedProtocol = requestedProtocol == null ? "TLSv1.2" : requestedProtocol;
+      return this;
+    }
+
+    /**
+     * Sets the requested cipher suites, i.e., "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", etc
+     *
+     * @param requestedCiphers The requested ciphers
+     * @return The builder object
+     */
+    public Builder requestedCiphers(String[] requestedCiphers) {
+      this.requestedCiphers = requestedCiphers;
+      return this;
+    }
+
+    /**
+     * Sets the Keystore and Keystore password
+     *
+     * @param keyStore The key store file to use
+     * @param keyStorePassword The password for the key store
+     * @return The builder object
+     */
+    public Builder keyStore(File keyStore, String keyStorePassword) {
+      this.keyStore = keyStore;
+      this.keyStorePassword = keyStorePassword;
+      return this;
+    }
+
+    /**
+     * Sets a PKCS#8 private key file in PEM format
+     *
+     * @param privateKey The private key file to use
+     * @return The builder object
+     */
+    public Builder privateKey(File privateKey) {
+      this.privateKey = privateKey;
+      return this;
+    }
+
+    /**
+     * Sets the Key password
+     *
+     * @param keyPassword The password for the private key
+     * @return The builder object
+     */
+    public Builder keyPassword(String keyPassword) {
+      this.keyPassword = keyPassword;
+      return this;
+    }
+
+    /**
+     * Sets a X.509 certificate chain file in PEM format
+     *
+     * @param certChain The certificate chain file to use
+     * @return The builder object
+     */
+    public Builder certChain(File certChain) {
+      this.certChain = certChain;
+      return this;
+    }
+
+    /**
+     * @param enabled Whether to use the OpenSSL implementation
+     * @return The builder object
+     */
+    public Builder openSslEnabled(boolean enabled) {
+      this.openSslEnabled = enabled;
+      return this;
+    }
+
+    /**
+     * Sets the trust-store, trust-store password, whether to use a Reloading TrustStore,
+     * and the trust-store reload interval, if enabled
+     *
+     * @param trustStore The trust store file to use
+     * @param trustStorePassword The password for the trust store
+     * @param trustStoreReloadingEnabled Whether trust store reloading is enabled
+     * @param trustStoreReloadIntervalMs The interval at which to reload the trust store file
+     * @return The builder object
+     */
+    public Builder trustStore(
+        File trustStore, String trustStorePassword,
+        boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs) {
+      this.trustStore = trustStore;
+      this.trustStorePassword = trustStorePassword;
+      this.trustStoreReloadingEnabled = trustStoreReloadingEnabled;
+      this.trustStoreReloadIntervalMs = trustStoreReloadIntervalMs;
+      return this;
+    }
+
+    /**
+     * Builds our {@link SSLFactory}
+     *
+     * @return The built {@link SSLFactory}
+     */
+    public SSLFactory build() {
+      return new SSLFactory(this);
+    }
+  }
+
+  /**
+   * Returns an initialized {@link SSLContext}
+   *
+   * @param requestedProtocol The requested protocol to use
+   * @param keyManagers The list of key managers to use
+   * @param trustManagers The list of trust managers to use
+   * @return The built {@link SSLContext}
+   * @throws GeneralSecurityException
+   */
+  private static SSLContext createSSLContext(
+      String requestedProtocol,
+      KeyManager[] keyManagers,
+      TrustManager[] trustManagers) throws GeneralSecurityException {
+    SSLContext sslContext = SSLContext.getInstance(requestedProtocol);
+    sslContext.init(keyManagers, trustManagers, null);
+    return sslContext;
+  }
+
+  /**
+   * Creates a new {@link SSLEngine}.
+   * Note that currently client auth is not supported
+   *
+   * @param isClient Whether the engine is used in a client context
+   * @param allocator The {@link ByteBufAllocator to use}
+   * @return A valid {@link SSLEngine}.
+   */
+  public SSLEngine createSSLEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine = createEngine(isClient, allocator);
+    engine.setUseClientMode(isClient);
+    engine.setNeedClientAuth(false);
+    engine.setEnabledProtocols(enabledProtocols(engine, requestedProtocol));
+    engine.setEnabledCipherSuites(enabledCipherSuites(engine, requestedCiphers));
+    return engine;
+  }
+
+  private SSLEngine createEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine;
+    if (isClient) {
+      if (nettyClientSslContext != null) {
+        engine = nettyClientSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    } else {
+      if (nettyServerSslContext != null) {
+        engine = nettyServerSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    }
+    return engine;
+  }
+
+  private static TrustManager[] credulousTrustStoreManagers() {
+    return new TrustManager[]{new X509TrustManager() {
+      @Override
+      public void checkClientTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public void checkServerTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public X509Certificate[] getAcceptedIssuers() {
+        return null;
+      }
+    }};
+  }
+
+  private static TrustManager[] trustStoreManagers(
+      File trustStore, String trustStorePassword,
+      boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs)
+          throws IOException, GeneralSecurityException {
+    if (trustStore == null || !trustStore.exists()) {
+      return credulousTrustStoreManagers();
+    } else {
+      if (trustStoreReloadingEnabled) {
+        ReloadingX509TrustManager reloading = new ReloadingX509TrustManager(
+          KeyStore.getDefaultType(), trustStore, trustStorePassword, trustStoreReloadIntervalMs);
+        reloading.init();
+        return new TrustManager[]{reloading};
+      } else {
+        return defaultTrustManagers(trustStore, trustStorePassword);
+      }
+    }
+  }
+
+  private static TrustManager[] defaultTrustManagers(File trustStore, String trustStorePassword)
+      throws IOException, KeyStoreException, CertificateException, NoSuchAlgorithmException {
+    try (InputStream input = Files.asByteSource(trustStore).openStream()) {
+      KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+      ks.load(input, null != trustStorePassword ? trustStorePassword.toCharArray() : null);
+      TrustManagerFactory tmf = TrustManagerFactory.getInstance(
+        TrustManagerFactory.getDefaultAlgorithm());
+      tmf.init(ks);
+      return tmf.getTrustManagers();
+    }
+  }
+
+  private static KeyManager[] keyManagers(File keyStore, String keyStorePassword)
+      throws NoSuchAlgorithmException, CertificateException,
+          KeyStoreException, IOException, UnrecoverableKeyException {
+    KeyManagerFactory factory = KeyManagerFactory.getInstance(
+      KeyManagerFactory.getDefaultAlgorithm());
+    factory.init(
+      loadKeyStore(keyStore, keyStorePassword),
+      keyStorePassword != null ? keyStorePassword.toCharArray() : null
+    );
+
+    return factory.getKeyManagers();
+  }
+
+  private static KeyStore loadKeyStore(File keyStore, String keyStorePassword)
+      throws KeyStoreException, IOException, CertificateException, NoSuchAlgorithmException {
+    if (keyStore == null) {
+      throw new KeyStoreException(
+        "keyStore cannot be null. Please configure spark.ssl.rpc.keyStore");
+    }
+
+    KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+    FileInputStream fin = new FileInputStream(keyStore);
+    try {
+      ks.load(fin, keyStorePassword != null ? keyStorePassword.toCharArray() : null);
+      return ks;
+    } finally {
+      JavaUtils.closeQuietly(fin);
+    }
+  }
+
+  private static String[] enabledProtocols(SSLEngine engine, String requestedProtocol) {
+    String[] supportedProtocols = engine.getSupportedProtocols();
+    String[] defaultProtocols = {"TLSv1.3", "TLSv1.2"};
+    String[] enabledProtocols =
+      ((requestedProtocol == null || requestedProtocol.isEmpty()) ?
+        defaultProtocols : new String[]{requestedProtocol});
+
+    List<String> protocols = addIfSupported(supportedProtocols, enabledProtocols);
+    if (!protocols.isEmpty()) {
+      return protocols.toArray(new String[protocols.size()]);
+    } else {
+      return supportedProtocols;
+    }
+  }
+
+  private static String[] enabledCipherSuites(
+      String[] supportedCiphers, String[] defaultCiphers, String[] requestedCiphers) {
+    String[] baseCiphers = new String[]{
+      // We take ciphers from the mozilla modern list first (for TLS 1.3):
+      // https://wiki.mozilla.org/Security/Server_Side_TLS
+      "TLS_CHACHA20_POLY1305_SHA256",
+      "TLS_AES_128_GCM_SHA256",
+      "TLS_AES_256_GCM_SHA384",
+      // Next we have the TLS1.2 ciphers for intermediate compatibility (since JDK8 does not
+      // support TLS1.3)
+      "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256",
+      "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384",
+      "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256",
+      "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384",
+      "TLS_ECDHE_ECDSA_WITH_CHACHA20_POLY1305_SHA256",
+      "TLS_ECDHE_RSA_WITH_CHACHA20_POLY1305_SHA256",
+      "TLS_DHE_RSA_WITH_AES_128_GCM_SHA256",
+      "TLS_DHE_RSA_WITH_AES_256_GCM_SHA384"
+    };
+    String[] enabledCiphers =
+      ((requestedCiphers == null || requestedCiphers.length == 0) ? baseCiphers : requestedCiphers);
+
+    List<String> ciphers = addIfSupported(supportedCiphers, enabledCiphers);
+    if (!ciphers.isEmpty()) {
+      return ciphers.toArray(new String[ciphers.size()]);
+    } else {
+      // Use the default from JDK as fallback.
+      return defaultCiphers;
+    }
+  }
+
+  private static String[] enabledCipherSuites(SSLEngine engine, String[] requestedCiphers) {
+    return enabledCipherSuites(
+      engine.getSupportedCipherSuites(), engine.getEnabledCipherSuites(), requestedCiphers);
+  }
+
+  private static List<String> addIfSupported(String[] supported, String... names) {

Review Comment:
   You can always do it in this method as well :-)
   Essentially, I am not suggesting to change what user specifies - just that this impl has quadratic complexity, which can be optimized trivially.
   



##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,470 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private static final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");
+      }
+    }
+    return SslProvider.JDK;
+  }
+
+  public void destroy() {
+    if (trustManagers != null) {
+      for (int i = 0; i < trustManagers.length; i++) {
+        if (trustManagers[i] instanceof ReloadingX509TrustManager) {
+          try {
+            ((ReloadingX509TrustManager) trustManagers[i]).destroy();
+          } catch (InterruptedException ex) {
+            logger.info("Interrupted while destroying trust manager: " + ex.toString(), ex);
+          }
+        }
+      }
+      trustManagers = null;
+    }
+
+    keyManagers = null;
+    jdkSslContext = null;
+    nettyClientSslContext = null;
+    nettyServerSslContext = null;
+    requestedProtocol = null;
+    requestedCiphers = null;
+  }
+
+  /**
+   * Builder class to construct instances of {@link SSLFactory} with specific options
+   */
+  public static class Builder {
+    private String requestedProtocol;
+    private String[] requestedCiphers;
+    private File keyStore;
+    private String keyStorePassword;
+    private File privateKey;
+    private String keyPassword;
+    private File certChain;
+    private File trustStore;
+    private String trustStorePassword;
+    private boolean trustStoreReloadingEnabled;
+    private int trustStoreReloadIntervalMs;
+    private boolean openSslEnabled;
+
+    /**
+     * Sets the requested protocol, i.e., "TLSv1.2", "TLSv1.1", etc
+     *
+     * @param requestedProtocol The requested protocol
+     * @return The builder object
+     */
+    public Builder requestedProtocol(String requestedProtocol) {
+      this.requestedProtocol = requestedProtocol == null ? "TLSv1.2" : requestedProtocol;
+      return this;
+    }
+
+    /**
+     * Sets the requested cipher suites, i.e., "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", etc
+     *
+     * @param requestedCiphers The requested ciphers
+     * @return The builder object
+     */
+    public Builder requestedCiphers(String[] requestedCiphers) {
+      this.requestedCiphers = requestedCiphers;
+      return this;
+    }
+
+    /**
+     * Sets the Keystore and Keystore password
+     *
+     * @param keyStore The key store file to use
+     * @param keyStorePassword The password for the key store
+     * @return The builder object
+     */
+    public Builder keyStore(File keyStore, String keyStorePassword) {
+      this.keyStore = keyStore;
+      this.keyStorePassword = keyStorePassword;
+      return this;
+    }
+
+    /**
+     * Sets a PKCS#8 private key file in PEM format
+     *
+     * @param privateKey The private key file to use
+     * @return The builder object
+     */
+    public Builder privateKey(File privateKey) {
+      this.privateKey = privateKey;
+      return this;
+    }
+
+    /**
+     * Sets the Key password
+     *
+     * @param keyPassword The password for the private key
+     * @return The builder object
+     */
+    public Builder keyPassword(String keyPassword) {
+      this.keyPassword = keyPassword;
+      return this;
+    }
+
+    /**
+     * Sets a X.509 certificate chain file in PEM format
+     *
+     * @param certChain The certificate chain file to use
+     * @return The builder object
+     */
+    public Builder certChain(File certChain) {
+      this.certChain = certChain;
+      return this;
+    }
+
+    /**
+     * @param enabled Whether to use the OpenSSL implementation
+     * @return The builder object
+     */
+    public Builder openSslEnabled(boolean enabled) {
+      this.openSslEnabled = enabled;
+      return this;
+    }
+
+    /**
+     * Sets the trust-store, trust-store password, whether to use a Reloading TrustStore,
+     * and the trust-store reload interval, if enabled
+     *
+     * @param trustStore The trust store file to use
+     * @param trustStorePassword The password for the trust store
+     * @param trustStoreReloadingEnabled Whether trust store reloading is enabled
+     * @param trustStoreReloadIntervalMs The interval at which to reload the trust store file
+     * @return The builder object
+     */
+    public Builder trustStore(
+        File trustStore, String trustStorePassword,
+        boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs) {
+      this.trustStore = trustStore;
+      this.trustStorePassword = trustStorePassword;
+      this.trustStoreReloadingEnabled = trustStoreReloadingEnabled;
+      this.trustStoreReloadIntervalMs = trustStoreReloadIntervalMs;
+      return this;
+    }
+
+    /**
+     * Builds our {@link SSLFactory}
+     *
+     * @return The built {@link SSLFactory}
+     */
+    public SSLFactory build() {
+      return new SSLFactory(this);
+    }
+  }
+
+  /**
+   * Returns an initialized {@link SSLContext}
+   *
+   * @param requestedProtocol The requested protocol to use
+   * @param keyManagers The list of key managers to use
+   * @param trustManagers The list of trust managers to use
+   * @return The built {@link SSLContext}
+   * @throws GeneralSecurityException
+   */
+  private static SSLContext createSSLContext(
+      String requestedProtocol,
+      KeyManager[] keyManagers,
+      TrustManager[] trustManagers) throws GeneralSecurityException {
+    SSLContext sslContext = SSLContext.getInstance(requestedProtocol);
+    sslContext.init(keyManagers, trustManagers, null);
+    return sslContext;
+  }
+
+  /**
+   * Creates a new {@link SSLEngine}.
+   * Note that currently client auth is not supported
+   *
+   * @param isClient Whether the engine is used in a client context
+   * @param allocator The {@link ByteBufAllocator to use}
+   * @return A valid {@link SSLEngine}.
+   */
+  public SSLEngine createSSLEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine = createEngine(isClient, allocator);
+    engine.setUseClientMode(isClient);
+    engine.setNeedClientAuth(false);
+    engine.setEnabledProtocols(enabledProtocols(engine, requestedProtocol));
+    engine.setEnabledCipherSuites(enabledCipherSuites(engine, requestedCiphers));
+    return engine;
+  }
+
+  private SSLEngine createEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine;
+    if (isClient) {
+      if (nettyClientSslContext != null) {
+        engine = nettyClientSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    } else {
+      if (nettyServerSslContext != null) {
+        engine = nettyServerSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    }
+    return engine;
+  }
+
+  private static TrustManager[] credulousTrustStoreManagers() {
+    return new TrustManager[]{new X509TrustManager() {
+      @Override
+      public void checkClientTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public void checkServerTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public X509Certificate[] getAcceptedIssuers() {
+        return null;
+      }
+    }};
+  }
+
+  private static TrustManager[] trustStoreManagers(
+      File trustStore, String trustStorePassword,
+      boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs)
+          throws IOException, GeneralSecurityException {
+    if (trustStore == null || !trustStore.exists()) {
+      return credulousTrustStoreManagers();
+    } else {
+      if (trustStoreReloadingEnabled) {
+        ReloadingX509TrustManager reloading = new ReloadingX509TrustManager(
+          KeyStore.getDefaultType(), trustStore, trustStorePassword, trustStoreReloadIntervalMs);
+        reloading.init();
+        return new TrustManager[]{reloading};
+      } else {
+        return defaultTrustManagers(trustStore, trustStorePassword);
+      }
+    }
+  }
+
+  private static TrustManager[] defaultTrustManagers(File trustStore, String trustStorePassword)
+      throws IOException, KeyStoreException, CertificateException, NoSuchAlgorithmException {
+    try (InputStream input = Files.asByteSource(trustStore).openStream()) {
+      KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+      ks.load(input, null != trustStorePassword ? trustStorePassword.toCharArray() : null);
+      TrustManagerFactory tmf = TrustManagerFactory.getInstance(
+        TrustManagerFactory.getDefaultAlgorithm());
+      tmf.init(ks);
+      return tmf.getTrustManagers();
+    }
+  }
+
+  private static KeyManager[] keyManagers(File keyStore, String keyStorePassword)
+      throws NoSuchAlgorithmException, CertificateException,
+          KeyStoreException, IOException, UnrecoverableKeyException {
+    KeyManagerFactory factory = KeyManagerFactory.getInstance(
+      KeyManagerFactory.getDefaultAlgorithm());
+    factory.init(
+      loadKeyStore(keyStore, keyStorePassword),
+      keyStorePassword != null ? keyStorePassword.toCharArray() : null
+    );
+
+    return factory.getKeyManagers();
+  }
+
+  private static KeyStore loadKeyStore(File keyStore, String keyStorePassword)
+      throws KeyStoreException, IOException, CertificateException, NoSuchAlgorithmException {
+    if (keyStore == null) {
+      throw new KeyStoreException(
+        "keyStore cannot be null. Please configure spark.ssl.rpc.keyStore");
+    }
+
+    KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+    FileInputStream fin = new FileInputStream(keyStore);
+    try {
+      ks.load(fin, keyStorePassword != null ? keyStorePassword.toCharArray() : null);
+      return ks;
+    } finally {
+      JavaUtils.closeQuietly(fin);
+    }
+  }
+
+  private static String[] enabledProtocols(SSLEngine engine, String requestedProtocol) {
+    String[] supportedProtocols = engine.getSupportedProtocols();
+    String[] defaultProtocols = {"TLSv1.3", "TLSv1.2"};
+    String[] enabledProtocols =
+      ((requestedProtocol == null || requestedProtocol.isEmpty()) ?
+        defaultProtocols : new String[]{requestedProtocol});
+
+    List<String> protocols = addIfSupported(supportedProtocols, enabledProtocols);
+    if (!protocols.isEmpty()) {
+      return protocols.toArray(new String[protocols.size()]);
+    } else {
+      return supportedProtocols;
+    }
+  }
+
+  private static String[] enabledCipherSuites(
+      String[] supportedCiphers, String[] defaultCiphers, String[] requestedCiphers) {
+    String[] baseCiphers = new String[]{
+      // We take ciphers from the mozilla modern list first (for TLS 1.3):
+      // https://wiki.mozilla.org/Security/Server_Side_TLS
+      "TLS_CHACHA20_POLY1305_SHA256",
+      "TLS_AES_128_GCM_SHA256",
+      "TLS_AES_256_GCM_SHA384",
+      // Next we have the TLS1.2 ciphers for intermediate compatibility (since JDK8 does not
+      // support TLS1.3)
+      "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256",
+      "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384",
+      "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256",
+      "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384",
+      "TLS_ECDHE_ECDSA_WITH_CHACHA20_POLY1305_SHA256",
+      "TLS_ECDHE_RSA_WITH_CHACHA20_POLY1305_SHA256",
+      "TLS_DHE_RSA_WITH_AES_128_GCM_SHA256",
+      "TLS_DHE_RSA_WITH_AES_256_GCM_SHA384"
+    };
+    String[] enabledCiphers =
+      ((requestedCiphers == null || requestedCiphers.length == 0) ? baseCiphers : requestedCiphers);
+
+    List<String> ciphers = addIfSupported(supportedCiphers, enabledCiphers);
+    if (!ciphers.isEmpty()) {
+      return ciphers.toArray(new String[ciphers.size()]);
+    } else {
+      // Use the default from JDK as fallback.
+      return defaultCiphers;
+    }
+  }
+
+  private static String[] enabledCipherSuites(SSLEngine engine, String[] requestedCiphers) {
+    return enabledCipherSuites(
+      engine.getSupportedCipherSuites(), engine.getEnabledCipherSuites(), requestedCiphers);
+  }
+
+  private static List<String> addIfSupported(String[] supported, String... names) {

Review Comment:
   You can always do it in this method as well :-)
   Essentially, I am not suggesting to change what user specifies - just that this impl has quadratic complexity, which can be optimized trivially.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45541][CORE] Add SSLFactory [spark]

Posted by "hasnain-db (via GitHub)" <gi...@apache.org>.
hasnain-db commented on PR #43386:
URL: https://github.com/apache/spark/pull/43386#issuecomment-1773993372

   thanks! some jobs failed, rerunning them so hopefully there is a green CI in the morning


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45541][CORE] Add SSLFactory [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm commented on PR #43386:
URL: https://github.com/apache/spark/pull/43386#issuecomment-1774029570

   The test failure is unrelated to this PR.
   Merging to master.
   
   Thanks for fixing this @hasnain-db !


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45541][CORE] Add SSLFactory [spark]

Posted by "hasnain-db (via GitHub)" <gi...@apache.org>.
hasnain-db commented on code in PR #43386:
URL: https://github.com/apache/spark/pull/43386#discussion_r1366078225


##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .sessionCacheSize(0)
+      .sessionTimeout(0)
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");
+      }
+    }
+    return SslProvider.JDK;
+  }
+
+  public void destroy() {
+    if (trustManagers != null) {
+      for (int i = 0; i < trustManagers.length; i++) {
+        if (trustManagers[i] instanceof ReloadingX509TrustManager) {
+          try {
+            ((ReloadingX509TrustManager) trustManagers[i]).destroy();
+          } catch (InterruptedException ex) {
+            logger.error("Interrupted while destroying trust manager: " + ex.toString(), ex);
+          }
+        }
+      }
+      trustManagers = null;
+    }
+
+    keyManagers = null;
+    jdkSslContext = null;
+    nettyClientSslContext = null;
+    nettyServerSslContext = null;
+    requestedProtocol = null;
+    requestedCiphers = null;
+  }
+
+  /**
+   * Builder class to construct instances of {@link SSLFactory} with specific options
+   */
+  public static class Builder {
+    private String requestedProtocol;
+    private String[] requestedCiphers;
+    private File keyStore;
+    private String keyStorePassword;
+    private File privateKey;
+    private String keyPassword;
+    private File certChain;
+    private File trustStore;
+    private String trustStorePassword;
+    private boolean trustStoreReloadingEnabled;
+    private int trustStoreReloadIntervalMs;
+    private boolean openSslEnabled;
+
+    /**
+     * Sets the requested protocol, i.e., "TLSv1.2", "TLSv1.1", etc
+     *
+     * @param requestedProtocol The requested protocol
+     * @return The builder object
+     */
+    public Builder requestedProtocol(String requestedProtocol) {
+      this.requestedProtocol = requestedProtocol == null ? "TLSv1.2" : requestedProtocol;
+      return this;
+    }
+
+    /**
+     * Sets the requested cipher suites, i.e., "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", etc
+     *
+     * @param requestedCiphers The requested ciphers
+     * @return The builder object
+     */
+    public Builder requestedCiphers(String[] requestedCiphers) {
+      this.requestedCiphers = requestedCiphers;
+      return this;
+    }
+
+    /**
+     * Sets the Keystore and Keystore password
+     *
+     * @param keyStore The key store file to use
+     * @param keyStorePassword The password for the key store
+     * @return The builder object
+     */
+    public Builder keyStore(File keyStore, String keyStorePassword) {
+      this.keyStore = keyStore;
+      this.keyStorePassword = keyStorePassword;
+      return this;
+    }
+
+    /**
+     * Sets a PKCS#8 private key file in PEM format
+     *
+     * @param privateKey The private key file to use
+     * @return The builder object
+     */
+    public Builder privateKey(File privateKey) {
+      this.privateKey = privateKey;
+      return this;
+    }
+
+    /**
+     * Sets the Key password
+     *
+     * @param keyPassword The password for the private key
+     * @return The builder object
+     */
+    public Builder keyPassword(String keyPassword) {
+      this.keyPassword = keyPassword;
+      return this;
+    }
+
+    /**
+     * Sets a X.509 certificate chain file in PEM format
+     *
+     * @param certChain The certificate chain file to use
+     * @return The builder object
+     */
+    public Builder certChain(File certChain) {
+      this.certChain = certChain;
+      return this;
+    }
+
+    /**
+     * @param enabled Whether to use the OpenSSL implementation
+     * @return The builder object
+     */
+    public Builder openSslEnabled(boolean enabled) {
+      this.openSslEnabled = enabled;
+      return this;
+    }
+
+    /**
+     * Sets the trust-store, trust-store password, whether to use a Reloading TrustStore,
+     * and the trust-store reload interval, if enabled
+     *
+     * @param trustStore The trust store file to use
+     * @param trustStorePassword The password for the trust store
+     * @param trustStoreReloadingEnabled Whether trust store reloading is enabled
+     * @param trustStoreReloadIntervalMs The interval at which to reload the trust store file
+     * @return The builder object
+     */
+    public Builder trustStore(
+        File trustStore, String trustStorePassword,
+        boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs) {
+      this.trustStore = trustStore;
+      this.trustStorePassword = trustStorePassword;
+      this.trustStoreReloadingEnabled = trustStoreReloadingEnabled;
+      this.trustStoreReloadIntervalMs = trustStoreReloadIntervalMs;
+      return this;
+    }
+
+    /**
+     * Builds our {@link SSLFactory}
+     *
+     * @return The built {@link SSLFactory}
+     */
+    public SSLFactory build() {
+      return new SSLFactory(this);
+    }
+  }
+
+  /**
+   * Returns an initialized {@link SSLContext}
+   *
+   * @param requestedProtocol The requested protocol to use
+   * @param keyManagers The list of key managers to use
+   * @param trustManagers The list of trust managers to use
+   * @return The built {@link SSLContext}
+   * @throws GeneralSecurityException
+   */
+  private static SSLContext createSSLContext(
+      String requestedProtocol,
+      KeyManager[] keyManagers,
+      TrustManager[] trustManagers) throws GeneralSecurityException {
+    SSLContext sslContext = getSSLContextInstance(requestedProtocol);
+    sslContext.init(keyManagers, trustManagers, null);
+    return sslContext;
+  }
+
+  /**
+   * Get the {@link SSLContext} for the specified <tt>requestedProtocol</tt>
+   * if available, or the default {@link SSLContext}
+   * @param requestedProtocol The protocol to use
+   * @return The built {@link SSLContext}
+   * @throws NoSuchAlgorithmException
+   */
+  private static SSLContext getSSLContextInstance(String requestedProtocol)
+      throws NoSuchAlgorithmException {
+    SSLContext context = null;
+    try {
+      context = SSLContext.getInstance(requestedProtocol);
+    } catch (Exception e) {
+      context = SSLContext.getDefault();

Review Comment:
   adding a note: I was worried about not catching the `null` case when the user doesn't provide a protocol but I remembered the builder handles this. So we should propagate the exception and I'll make the code do that.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45541][CORE] Add SSLFactory [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm commented on code in PR #43386:
URL: https://github.com/apache/spark/pull/43386#discussion_r1366403180


##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,470 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private static final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");
+      }
+    }
+    return SslProvider.JDK;
+  }
+
+  public void destroy() {
+    if (trustManagers != null) {
+      for (int i = 0; i < trustManagers.length; i++) {
+        if (trustManagers[i] instanceof ReloadingX509TrustManager) {
+          try {
+            ((ReloadingX509TrustManager) trustManagers[i]).destroy();
+          } catch (InterruptedException ex) {
+            logger.info("Interrupted while destroying trust manager: " + ex.toString(), ex);
+          }
+        }
+      }
+      trustManagers = null;
+    }
+
+    keyManagers = null;
+    jdkSslContext = null;
+    nettyClientSslContext = null;
+    nettyServerSslContext = null;
+    requestedProtocol = null;
+    requestedCiphers = null;
+  }
+
+  /**
+   * Builder class to construct instances of {@link SSLFactory} with specific options
+   */
+  public static class Builder {
+    private String requestedProtocol;
+    private String[] requestedCiphers;
+    private File keyStore;
+    private String keyStorePassword;
+    private File privateKey;
+    private String keyPassword;
+    private File certChain;
+    private File trustStore;
+    private String trustStorePassword;
+    private boolean trustStoreReloadingEnabled;
+    private int trustStoreReloadIntervalMs;
+    private boolean openSslEnabled;
+
+    /**
+     * Sets the requested protocol, i.e., "TLSv1.2", "TLSv1.1", etc
+     *
+     * @param requestedProtocol The requested protocol
+     * @return The builder object
+     */
+    public Builder requestedProtocol(String requestedProtocol) {
+      this.requestedProtocol = requestedProtocol == null ? "TLSv1.2" : requestedProtocol;
+      return this;
+    }
+
+    /**
+     * Sets the requested cipher suites, i.e., "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", etc
+     *
+     * @param requestedCiphers The requested ciphers
+     * @return The builder object
+     */
+    public Builder requestedCiphers(String[] requestedCiphers) {
+      this.requestedCiphers = requestedCiphers;
+      return this;
+    }
+
+    /**
+     * Sets the Keystore and Keystore password
+     *
+     * @param keyStore The key store file to use
+     * @param keyStorePassword The password for the key store
+     * @return The builder object
+     */
+    public Builder keyStore(File keyStore, String keyStorePassword) {
+      this.keyStore = keyStore;
+      this.keyStorePassword = keyStorePassword;
+      return this;
+    }
+
+    /**
+     * Sets a PKCS#8 private key file in PEM format
+     *
+     * @param privateKey The private key file to use
+     * @return The builder object
+     */
+    public Builder privateKey(File privateKey) {
+      this.privateKey = privateKey;
+      return this;
+    }
+
+    /**
+     * Sets the Key password
+     *
+     * @param keyPassword The password for the private key
+     * @return The builder object
+     */
+    public Builder keyPassword(String keyPassword) {
+      this.keyPassword = keyPassword;
+      return this;
+    }
+
+    /**
+     * Sets a X.509 certificate chain file in PEM format
+     *
+     * @param certChain The certificate chain file to use
+     * @return The builder object
+     */
+    public Builder certChain(File certChain) {
+      this.certChain = certChain;
+      return this;
+    }
+
+    /**
+     * @param enabled Whether to use the OpenSSL implementation
+     * @return The builder object
+     */
+    public Builder openSslEnabled(boolean enabled) {
+      this.openSslEnabled = enabled;
+      return this;
+    }
+
+    /**
+     * Sets the trust-store, trust-store password, whether to use a Reloading TrustStore,
+     * and the trust-store reload interval, if enabled
+     *
+     * @param trustStore The trust store file to use
+     * @param trustStorePassword The password for the trust store
+     * @param trustStoreReloadingEnabled Whether trust store reloading is enabled
+     * @param trustStoreReloadIntervalMs The interval at which to reload the trust store file
+     * @return The builder object
+     */
+    public Builder trustStore(
+        File trustStore, String trustStorePassword,
+        boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs) {
+      this.trustStore = trustStore;
+      this.trustStorePassword = trustStorePassword;
+      this.trustStoreReloadingEnabled = trustStoreReloadingEnabled;
+      this.trustStoreReloadIntervalMs = trustStoreReloadIntervalMs;
+      return this;
+    }
+
+    /**
+     * Builds our {@link SSLFactory}
+     *
+     * @return The built {@link SSLFactory}
+     */
+    public SSLFactory build() {
+      return new SSLFactory(this);
+    }
+  }
+
+  /**
+   * Returns an initialized {@link SSLContext}
+   *
+   * @param requestedProtocol The requested protocol to use
+   * @param keyManagers The list of key managers to use
+   * @param trustManagers The list of trust managers to use
+   * @return The built {@link SSLContext}
+   * @throws GeneralSecurityException
+   */
+  private static SSLContext createSSLContext(
+      String requestedProtocol,
+      KeyManager[] keyManagers,
+      TrustManager[] trustManagers) throws GeneralSecurityException {
+    SSLContext sslContext = SSLContext.getInstance(requestedProtocol);
+    sslContext.init(keyManagers, trustManagers, null);
+    return sslContext;
+  }
+
+  /**
+   * Creates a new {@link SSLEngine}.
+   * Note that currently client auth is not supported
+   *
+   * @param isClient Whether the engine is used in a client context
+   * @param allocator The {@link ByteBufAllocator to use}
+   * @return A valid {@link SSLEngine}.
+   */
+  public SSLEngine createSSLEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine = createEngine(isClient, allocator);
+    engine.setUseClientMode(isClient);
+    engine.setNeedClientAuth(false);
+    engine.setEnabledProtocols(enabledProtocols(engine, requestedProtocol));
+    engine.setEnabledCipherSuites(enabledCipherSuites(engine, requestedCiphers));
+    return engine;
+  }
+
+  private SSLEngine createEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine;
+    if (isClient) {
+      if (nettyClientSslContext != null) {
+        engine = nettyClientSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    } else {
+      if (nettyServerSslContext != null) {
+        engine = nettyServerSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    }
+    return engine;
+  }
+
+  private static TrustManager[] credulousTrustStoreManagers() {
+    return new TrustManager[]{new X509TrustManager() {
+      @Override
+      public void checkClientTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public void checkServerTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public X509Certificate[] getAcceptedIssuers() {
+        return null;
+      }
+    }};
+  }
+
+  private static TrustManager[] trustStoreManagers(
+      File trustStore, String trustStorePassword,
+      boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs)
+          throws IOException, GeneralSecurityException {
+    if (trustStore == null || !trustStore.exists()) {
+      return credulousTrustStoreManagers();
+    } else {
+      if (trustStoreReloadingEnabled) {
+        ReloadingX509TrustManager reloading = new ReloadingX509TrustManager(
+          KeyStore.getDefaultType(), trustStore, trustStorePassword, trustStoreReloadIntervalMs);
+        reloading.init();
+        return new TrustManager[]{reloading};
+      } else {
+        return defaultTrustManagers(trustStore, trustStorePassword);
+      }
+    }
+  }
+
+  private static TrustManager[] defaultTrustManagers(File trustStore, String trustStorePassword)
+      throws IOException, KeyStoreException, CertificateException, NoSuchAlgorithmException {
+    try (InputStream input = Files.asByteSource(trustStore).openStream()) {
+      KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+      ks.load(input, null != trustStorePassword ? trustStorePassword.toCharArray() : null);
+      TrustManagerFactory tmf = TrustManagerFactory.getInstance(
+        TrustManagerFactory.getDefaultAlgorithm());
+      tmf.init(ks);
+      return tmf.getTrustManagers();
+    }
+  }
+
+  private static KeyManager[] keyManagers(File keyStore, String keyStorePassword)
+      throws NoSuchAlgorithmException, CertificateException,
+          KeyStoreException, IOException, UnrecoverableKeyException {
+    KeyManagerFactory factory = KeyManagerFactory.getInstance(
+      KeyManagerFactory.getDefaultAlgorithm());
+    factory.init(
+      loadKeyStore(keyStore, keyStorePassword),
+      keyStorePassword != null ? keyStorePassword.toCharArray() : null
+    );
+
+    return factory.getKeyManagers();
+  }
+
+  private static KeyStore loadKeyStore(File keyStore, String keyStorePassword)
+      throws KeyStoreException, IOException, CertificateException, NoSuchAlgorithmException {
+    if (keyStore == null) {
+      throw new KeyStoreException(
+        "keyStore cannot be null. Please configure spark.ssl.rpc.keyStore");
+    }
+
+    KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+    FileInputStream fin = new FileInputStream(keyStore);
+    try {
+      ks.load(fin, keyStorePassword != null ? keyStorePassword.toCharArray() : null);
+      return ks;
+    } finally {
+      JavaUtils.closeQuietly(fin);
+    }
+  }
+
+  private static String[] enabledProtocols(SSLEngine engine, String requestedProtocol) {
+    String[] supportedProtocols = engine.getSupportedProtocols();
+    String[] defaultProtocols = {"TLSv1.3", "TLSv1.2"};
+    String[] enabledProtocols =
+      ((requestedProtocol == null || requestedProtocol.isEmpty()) ?
+        defaultProtocols : new String[]{requestedProtocol});
+
+    List<String> protocols = addIfSupported(supportedProtocols, enabledProtocols);
+    if (!protocols.isEmpty()) {
+      return protocols.toArray(new String[protocols.size()]);
+    } else {
+      return supportedProtocols;
+    }
+  }
+
+  private static String[] enabledCipherSuites(
+      String[] supportedCiphers, String[] defaultCiphers, String[] requestedCiphers) {
+    String[] baseCiphers = new String[]{
+      // We take ciphers from the mozilla modern list first (for TLS 1.3):
+      // https://wiki.mozilla.org/Security/Server_Side_TLS
+      "TLS_CHACHA20_POLY1305_SHA256",
+      "TLS_AES_128_GCM_SHA256",
+      "TLS_AES_256_GCM_SHA384",
+      // Next we have the TLS1.2 ciphers for intermediate compatibility (since JDK8 does not
+      // support TLS1.3)
+      "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256",
+      "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384",
+      "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256",
+      "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384",
+      "TLS_ECDHE_ECDSA_WITH_CHACHA20_POLY1305_SHA256",
+      "TLS_ECDHE_RSA_WITH_CHACHA20_POLY1305_SHA256",
+      "TLS_DHE_RSA_WITH_AES_128_GCM_SHA256",
+      "TLS_DHE_RSA_WITH_AES_256_GCM_SHA384"
+    };
+    String[] enabledCiphers =
+      ((requestedCiphers == null || requestedCiphers.length == 0) ? baseCiphers : requestedCiphers);
+
+    List<String> ciphers = addIfSupported(supportedCiphers, enabledCiphers);
+    if (!ciphers.isEmpty()) {
+      return ciphers.toArray(new String[ciphers.size()]);
+    } else {
+      // Use the default from JDK as fallback.
+      return defaultCiphers;
+    }
+  }
+
+  private static String[] enabledCipherSuites(SSLEngine engine, String[] requestedCiphers) {
+    return enabledCipherSuites(
+      engine.getSupportedCipherSuites(), engine.getEnabledCipherSuites(), requestedCiphers);
+  }
+
+  private static List<String> addIfSupported(String[] supported, String... names) {

Review Comment:
   You can always do it in this method as well :-)
   Essentially, I am not suggesting to change what user specifies - just that this impl has quadratic complexity, which can be optimized trivially.
   I do expect this not to be a serious issue (hence a nit) given the cardinality of the array



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45541][CORE] Add SSLFactory [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm commented on code in PR #43386:
URL: https://github.com/apache/spark/pull/43386#discussion_r1367691734


##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,471 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private static final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");
+      }
+    }
+    return SslProvider.JDK;
+  }
+
+  public void destroy() {
+    if (trustManagers != null) {
+      for (int i = 0; i < trustManagers.length; i++) {
+        if (trustManagers[i] instanceof ReloadingX509TrustManager) {
+          try {
+            ((ReloadingX509TrustManager) trustManagers[i]).destroy();
+          } catch (InterruptedException ex) {
+            logger.info("Interrupted while destroying trust manager: " + ex.toString(), ex);
+          }
+        }
+      }
+      trustManagers = null;
+    }
+
+    keyManagers = null;
+    jdkSslContext = null;
+    nettyClientSslContext = null;
+    nettyServerSslContext = null;
+    requestedProtocol = null;
+    requestedCiphers = null;
+  }
+
+  /**
+   * Builder class to construct instances of {@link SSLFactory} with specific options
+   */
+  public static class Builder {
+    private String requestedProtocol;
+    private String[] requestedCiphers;
+    private File keyStore;
+    private String keyStorePassword;
+    private File privateKey;
+    private String keyPassword;
+    private File certChain;
+    private File trustStore;
+    private String trustStorePassword;
+    private boolean trustStoreReloadingEnabled;
+    private int trustStoreReloadIntervalMs;
+    private boolean openSslEnabled;
+
+    /**
+     * Sets the requested protocol, i.e., "TLSv1.2", "TLSv1.1", etc
+     *
+     * @param requestedProtocol The requested protocol
+     * @return The builder object
+     */
+    public Builder requestedProtocol(String requestedProtocol) {
+      this.requestedProtocol = requestedProtocol == null ? "TLSv1.2" : requestedProtocol;
+      return this;
+    }
+
+    /**
+     * Sets the requested cipher suites, i.e., "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", etc
+     *
+     * @param requestedCiphers The requested ciphers
+     * @return The builder object
+     */
+    public Builder requestedCiphers(String[] requestedCiphers) {
+      this.requestedCiphers = requestedCiphers;
+      return this;
+    }
+
+    /**
+     * Sets the Keystore and Keystore password
+     *
+     * @param keyStore The key store file to use
+     * @param keyStorePassword The password for the key store
+     * @return The builder object
+     */
+    public Builder keyStore(File keyStore, String keyStorePassword) {
+      this.keyStore = keyStore;
+      this.keyStorePassword = keyStorePassword;
+      return this;
+    }
+
+    /**
+     * Sets a PKCS#8 private key file in PEM format
+     *
+     * @param privateKey The private key file to use
+     * @return The builder object
+     */
+    public Builder privateKey(File privateKey) {
+      this.privateKey = privateKey;
+      return this;
+    }
+
+    /**
+     * Sets the Key password
+     *
+     * @param keyPassword The password for the private key
+     * @return The builder object
+     */
+    public Builder keyPassword(String keyPassword) {
+      this.keyPassword = keyPassword;
+      return this;
+    }
+
+    /**
+     * Sets a X.509 certificate chain file in PEM format
+     *
+     * @param certChain The certificate chain file to use
+     * @return The builder object
+     */
+    public Builder certChain(File certChain) {
+      this.certChain = certChain;
+      return this;
+    }
+
+    /**
+     * @param enabled Whether to use the OpenSSL implementation
+     * @return The builder object
+     */
+    public Builder openSslEnabled(boolean enabled) {
+      this.openSslEnabled = enabled;
+      return this;
+    }
+
+    /**
+     * Sets the trust-store, trust-store password, whether to use a Reloading TrustStore,
+     * and the trust-store reload interval, if enabled
+     *
+     * @param trustStore The trust store file to use
+     * @param trustStorePassword The password for the trust store
+     * @param trustStoreReloadingEnabled Whether trust store reloading is enabled
+     * @param trustStoreReloadIntervalMs The interval at which to reload the trust store file
+     * @return The builder object
+     */
+    public Builder trustStore(
+        File trustStore, String trustStorePassword,
+        boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs) {
+      this.trustStore = trustStore;
+      this.trustStorePassword = trustStorePassword;
+      this.trustStoreReloadingEnabled = trustStoreReloadingEnabled;
+      this.trustStoreReloadIntervalMs = trustStoreReloadIntervalMs;
+      return this;
+    }
+
+    /**
+     * Builds our {@link SSLFactory}
+     *
+     * @return The built {@link SSLFactory}
+     */
+    public SSLFactory build() {
+      return new SSLFactory(this);
+    }
+  }
+
+  /**
+   * Returns an initialized {@link SSLContext}
+   *
+   * @param requestedProtocol The requested protocol to use
+   * @param keyManagers The list of key managers to use
+   * @param trustManagers The list of trust managers to use
+   * @return The built {@link SSLContext}
+   * @throws GeneralSecurityException
+   */
+  private static SSLContext createSSLContext(
+      String requestedProtocol,
+      KeyManager[] keyManagers,
+      TrustManager[] trustManagers) throws GeneralSecurityException {
+    SSLContext sslContext = SSLContext.getInstance(requestedProtocol);
+    sslContext.init(keyManagers, trustManagers, null);
+    return sslContext;
+  }
+
+  /**
+   * Creates a new {@link SSLEngine}.
+   * Note that currently client auth is not supported
+   *
+   * @param isClient Whether the engine is used in a client context
+   * @param allocator The {@link ByteBufAllocator to use}
+   * @return A valid {@link SSLEngine}.
+   */
+  public SSLEngine createSSLEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine = createEngine(isClient, allocator);
+    engine.setUseClientMode(isClient);
+    engine.setNeedClientAuth(false);
+    engine.setEnabledProtocols(enabledProtocols(engine, requestedProtocol));
+    engine.setEnabledCipherSuites(enabledCipherSuites(engine, requestedCiphers));
+    return engine;
+  }
+
+  private SSLEngine createEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine;
+    if (isClient) {
+      if (nettyClientSslContext != null) {
+        engine = nettyClientSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    } else {
+      if (nettyServerSslContext != null) {
+        engine = nettyServerSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    }
+    return engine;
+  }
+
+  private static TrustManager[] credulousTrustStoreManagers() {
+    return new TrustManager[]{new X509TrustManager() {
+      @Override
+      public void checkClientTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public void checkServerTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public X509Certificate[] getAcceptedIssuers() {
+        return null;
+      }
+    }};
+  }
+
+  private static TrustManager[] trustStoreManagers(
+      File trustStore, String trustStorePassword,
+      boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs)
+          throws IOException, GeneralSecurityException {
+    if (trustStore == null || !trustStore.exists()) {
+      return credulousTrustStoreManagers();
+    } else {
+      if (trustStoreReloadingEnabled) {
+        ReloadingX509TrustManager reloading = new ReloadingX509TrustManager(
+          KeyStore.getDefaultType(), trustStore, trustStorePassword, trustStoreReloadIntervalMs);
+        reloading.init();
+        return new TrustManager[]{reloading};
+      } else {
+        return defaultTrustManagers(trustStore, trustStorePassword);
+      }
+    }
+  }
+
+  private static TrustManager[] defaultTrustManagers(File trustStore, String trustStorePassword)
+      throws IOException, KeyStoreException, CertificateException, NoSuchAlgorithmException {
+    try (InputStream input = Files.asByteSource(trustStore).openStream()) {
+      KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+      char[] passwordCharacters = trustStorePassword != null?
+        trustStorePassword.toCharArray() : null;
+      ks.load(input, passwordCharacters);
+      TrustManagerFactory tmf = TrustManagerFactory.getInstance(
+        TrustManagerFactory.getDefaultAlgorithm());
+      tmf.init(ks);
+      return tmf.getTrustManagers();
+    }
+  }
+
+  private static KeyManager[] keyManagers(File keyStore, String keyStorePassword)
+      throws NoSuchAlgorithmException, CertificateException,
+          KeyStoreException, IOException, UnrecoverableKeyException {
+    KeyManagerFactory factory = KeyManagerFactory.getInstance(
+      KeyManagerFactory.getDefaultAlgorithm());
+    char[] passwordCharacters = keyStorePassword != null? keyStorePassword.toCharArray() : null;
+    factory.init(loadKeyStore(keyStore, keyStorePassword), passwordCharacters);

Review Comment:
   You can pass `passwordCharacters` to `loadKeyStore`, no ? It is a private method and so no external mods to the array



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45541][CORE] Add SSLFactory [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm commented on code in PR #43386:
URL: https://github.com/apache/spark/pull/43386#discussion_r1366403180


##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,470 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private static final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");
+      }
+    }
+    return SslProvider.JDK;
+  }
+
+  public void destroy() {
+    if (trustManagers != null) {
+      for (int i = 0; i < trustManagers.length; i++) {
+        if (trustManagers[i] instanceof ReloadingX509TrustManager) {
+          try {
+            ((ReloadingX509TrustManager) trustManagers[i]).destroy();
+          } catch (InterruptedException ex) {
+            logger.info("Interrupted while destroying trust manager: " + ex.toString(), ex);
+          }
+        }
+      }
+      trustManagers = null;
+    }
+
+    keyManagers = null;
+    jdkSslContext = null;
+    nettyClientSslContext = null;
+    nettyServerSslContext = null;
+    requestedProtocol = null;
+    requestedCiphers = null;
+  }
+
+  /**
+   * Builder class to construct instances of {@link SSLFactory} with specific options
+   */
+  public static class Builder {
+    private String requestedProtocol;
+    private String[] requestedCiphers;
+    private File keyStore;
+    private String keyStorePassword;
+    private File privateKey;
+    private String keyPassword;
+    private File certChain;
+    private File trustStore;
+    private String trustStorePassword;
+    private boolean trustStoreReloadingEnabled;
+    private int trustStoreReloadIntervalMs;
+    private boolean openSslEnabled;
+
+    /**
+     * Sets the requested protocol, i.e., "TLSv1.2", "TLSv1.1", etc
+     *
+     * @param requestedProtocol The requested protocol
+     * @return The builder object
+     */
+    public Builder requestedProtocol(String requestedProtocol) {
+      this.requestedProtocol = requestedProtocol == null ? "TLSv1.2" : requestedProtocol;
+      return this;
+    }
+
+    /**
+     * Sets the requested cipher suites, i.e., "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", etc
+     *
+     * @param requestedCiphers The requested ciphers
+     * @return The builder object
+     */
+    public Builder requestedCiphers(String[] requestedCiphers) {
+      this.requestedCiphers = requestedCiphers;
+      return this;
+    }
+
+    /**
+     * Sets the Keystore and Keystore password
+     *
+     * @param keyStore The key store file to use
+     * @param keyStorePassword The password for the key store
+     * @return The builder object
+     */
+    public Builder keyStore(File keyStore, String keyStorePassword) {
+      this.keyStore = keyStore;
+      this.keyStorePassword = keyStorePassword;
+      return this;
+    }
+
+    /**
+     * Sets a PKCS#8 private key file in PEM format
+     *
+     * @param privateKey The private key file to use
+     * @return The builder object
+     */
+    public Builder privateKey(File privateKey) {
+      this.privateKey = privateKey;
+      return this;
+    }
+
+    /**
+     * Sets the Key password
+     *
+     * @param keyPassword The password for the private key
+     * @return The builder object
+     */
+    public Builder keyPassword(String keyPassword) {
+      this.keyPassword = keyPassword;
+      return this;
+    }
+
+    /**
+     * Sets a X.509 certificate chain file in PEM format
+     *
+     * @param certChain The certificate chain file to use
+     * @return The builder object
+     */
+    public Builder certChain(File certChain) {
+      this.certChain = certChain;
+      return this;
+    }
+
+    /**
+     * @param enabled Whether to use the OpenSSL implementation
+     * @return The builder object
+     */
+    public Builder openSslEnabled(boolean enabled) {
+      this.openSslEnabled = enabled;
+      return this;
+    }
+
+    /**
+     * Sets the trust-store, trust-store password, whether to use a Reloading TrustStore,
+     * and the trust-store reload interval, if enabled
+     *
+     * @param trustStore The trust store file to use
+     * @param trustStorePassword The password for the trust store
+     * @param trustStoreReloadingEnabled Whether trust store reloading is enabled
+     * @param trustStoreReloadIntervalMs The interval at which to reload the trust store file
+     * @return The builder object
+     */
+    public Builder trustStore(
+        File trustStore, String trustStorePassword,
+        boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs) {
+      this.trustStore = trustStore;
+      this.trustStorePassword = trustStorePassword;
+      this.trustStoreReloadingEnabled = trustStoreReloadingEnabled;
+      this.trustStoreReloadIntervalMs = trustStoreReloadIntervalMs;
+      return this;
+    }
+
+    /**
+     * Builds our {@link SSLFactory}
+     *
+     * @return The built {@link SSLFactory}
+     */
+    public SSLFactory build() {
+      return new SSLFactory(this);
+    }
+  }
+
+  /**
+   * Returns an initialized {@link SSLContext}
+   *
+   * @param requestedProtocol The requested protocol to use
+   * @param keyManagers The list of key managers to use
+   * @param trustManagers The list of trust managers to use
+   * @return The built {@link SSLContext}
+   * @throws GeneralSecurityException
+   */
+  private static SSLContext createSSLContext(
+      String requestedProtocol,
+      KeyManager[] keyManagers,
+      TrustManager[] trustManagers) throws GeneralSecurityException {
+    SSLContext sslContext = SSLContext.getInstance(requestedProtocol);
+    sslContext.init(keyManagers, trustManagers, null);
+    return sslContext;
+  }
+
+  /**
+   * Creates a new {@link SSLEngine}.
+   * Note that currently client auth is not supported
+   *
+   * @param isClient Whether the engine is used in a client context
+   * @param allocator The {@link ByteBufAllocator to use}
+   * @return A valid {@link SSLEngine}.
+   */
+  public SSLEngine createSSLEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine = createEngine(isClient, allocator);
+    engine.setUseClientMode(isClient);
+    engine.setNeedClientAuth(false);
+    engine.setEnabledProtocols(enabledProtocols(engine, requestedProtocol));
+    engine.setEnabledCipherSuites(enabledCipherSuites(engine, requestedCiphers));
+    return engine;
+  }
+
+  private SSLEngine createEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine;
+    if (isClient) {
+      if (nettyClientSslContext != null) {
+        engine = nettyClientSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    } else {
+      if (nettyServerSslContext != null) {
+        engine = nettyServerSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    }
+    return engine;
+  }
+
+  private static TrustManager[] credulousTrustStoreManagers() {
+    return new TrustManager[]{new X509TrustManager() {
+      @Override
+      public void checkClientTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public void checkServerTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public X509Certificate[] getAcceptedIssuers() {
+        return null;
+      }
+    }};
+  }
+
+  private static TrustManager[] trustStoreManagers(
+      File trustStore, String trustStorePassword,
+      boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs)
+          throws IOException, GeneralSecurityException {
+    if (trustStore == null || !trustStore.exists()) {
+      return credulousTrustStoreManagers();
+    } else {
+      if (trustStoreReloadingEnabled) {
+        ReloadingX509TrustManager reloading = new ReloadingX509TrustManager(
+          KeyStore.getDefaultType(), trustStore, trustStorePassword, trustStoreReloadIntervalMs);
+        reloading.init();
+        return new TrustManager[]{reloading};
+      } else {
+        return defaultTrustManagers(trustStore, trustStorePassword);
+      }
+    }
+  }
+
+  private static TrustManager[] defaultTrustManagers(File trustStore, String trustStorePassword)
+      throws IOException, KeyStoreException, CertificateException, NoSuchAlgorithmException {
+    try (InputStream input = Files.asByteSource(trustStore).openStream()) {
+      KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+      ks.load(input, null != trustStorePassword ? trustStorePassword.toCharArray() : null);
+      TrustManagerFactory tmf = TrustManagerFactory.getInstance(
+        TrustManagerFactory.getDefaultAlgorithm());
+      tmf.init(ks);
+      return tmf.getTrustManagers();
+    }
+  }
+
+  private static KeyManager[] keyManagers(File keyStore, String keyStorePassword)
+      throws NoSuchAlgorithmException, CertificateException,
+          KeyStoreException, IOException, UnrecoverableKeyException {
+    KeyManagerFactory factory = KeyManagerFactory.getInstance(
+      KeyManagerFactory.getDefaultAlgorithm());
+    factory.init(
+      loadKeyStore(keyStore, keyStorePassword),
+      keyStorePassword != null ? keyStorePassword.toCharArray() : null
+    );
+
+    return factory.getKeyManagers();
+  }
+
+  private static KeyStore loadKeyStore(File keyStore, String keyStorePassword)
+      throws KeyStoreException, IOException, CertificateException, NoSuchAlgorithmException {
+    if (keyStore == null) {
+      throw new KeyStoreException(
+        "keyStore cannot be null. Please configure spark.ssl.rpc.keyStore");
+    }
+
+    KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+    FileInputStream fin = new FileInputStream(keyStore);
+    try {
+      ks.load(fin, keyStorePassword != null ? keyStorePassword.toCharArray() : null);
+      return ks;
+    } finally {
+      JavaUtils.closeQuietly(fin);
+    }
+  }
+
+  private static String[] enabledProtocols(SSLEngine engine, String requestedProtocol) {
+    String[] supportedProtocols = engine.getSupportedProtocols();
+    String[] defaultProtocols = {"TLSv1.3", "TLSv1.2"};
+    String[] enabledProtocols =
+      ((requestedProtocol == null || requestedProtocol.isEmpty()) ?
+        defaultProtocols : new String[]{requestedProtocol});
+
+    List<String> protocols = addIfSupported(supportedProtocols, enabledProtocols);
+    if (!protocols.isEmpty()) {
+      return protocols.toArray(new String[protocols.size()]);
+    } else {
+      return supportedProtocols;
+    }
+  }
+
+  private static String[] enabledCipherSuites(
+      String[] supportedCiphers, String[] defaultCiphers, String[] requestedCiphers) {
+    String[] baseCiphers = new String[]{
+      // We take ciphers from the mozilla modern list first (for TLS 1.3):
+      // https://wiki.mozilla.org/Security/Server_Side_TLS
+      "TLS_CHACHA20_POLY1305_SHA256",
+      "TLS_AES_128_GCM_SHA256",
+      "TLS_AES_256_GCM_SHA384",
+      // Next we have the TLS1.2 ciphers for intermediate compatibility (since JDK8 does not
+      // support TLS1.3)
+      "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256",
+      "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384",
+      "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256",
+      "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384",
+      "TLS_ECDHE_ECDSA_WITH_CHACHA20_POLY1305_SHA256",
+      "TLS_ECDHE_RSA_WITH_CHACHA20_POLY1305_SHA256",
+      "TLS_DHE_RSA_WITH_AES_128_GCM_SHA256",
+      "TLS_DHE_RSA_WITH_AES_256_GCM_SHA384"
+    };
+    String[] enabledCiphers =
+      ((requestedCiphers == null || requestedCiphers.length == 0) ? baseCiphers : requestedCiphers);
+
+    List<String> ciphers = addIfSupported(supportedCiphers, enabledCiphers);
+    if (!ciphers.isEmpty()) {
+      return ciphers.toArray(new String[ciphers.size()]);
+    } else {
+      // Use the default from JDK as fallback.
+      return defaultCiphers;
+    }
+  }
+
+  private static String[] enabledCipherSuites(SSLEngine engine, String[] requestedCiphers) {
+    return enabledCipherSuites(
+      engine.getSupportedCipherSuites(), engine.getEnabledCipherSuites(), requestedCiphers);
+  }
+
+  private static List<String> addIfSupported(String[] supported, String... names) {

Review Comment:
   You can always do it in this method as well :-)
   Essentially, I am not suggesting to change what user specifies - just that this impl has quadratic complexity, which can be optimized trivially.
   I do expect this not to be a serious issue (hence the nit) given the cardinality of the array



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45541][CORE] Add SSLFactory [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm commented on code in PR #43386:
URL: https://github.com/apache/spark/pull/43386#discussion_r1363202697


##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .sessionCacheSize(0)
+      .sessionTimeout(0)
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");

Review Comment:
   QQ: If openssl is requested, but it is not available on subset of nodes (perhaps due to some installation issues, etc - given this is depending on native libraries), what will be the impact on the spark application ?
   Will it continue to work fine with some subset on openssl and others on jdk provider ? or is there any potential concerns ?



##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private final Logger logger = LoggerFactory.getLogger(SSLFactory.class);

Review Comment:
   ```suggestion
     private static final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
   ```



##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .sessionCacheSize(0)
+      .sessionTimeout(0)

Review Comment:
   It is `0` by default, in order to use the default value.
   
   ```suggestion
   ```
   
   Note - if we are explicitly setting it to `0` to indicate unlimited timeout and cache - add a comment to that effect.



##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .sessionCacheSize(0)
+      .sessionTimeout(0)
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");
+      }
+    }
+    return SslProvider.JDK;
+  }
+
+  public void destroy() {
+    if (trustManagers != null) {
+      for (int i = 0; i < trustManagers.length; i++) {
+        if (trustManagers[i] instanceof ReloadingX509TrustManager) {
+          try {
+            ((ReloadingX509TrustManager) trustManagers[i]).destroy();
+          } catch (InterruptedException ex) {
+            logger.error("Interrupted while destroying trust manager: " + ex.toString(), ex);

Review Comment:
   `info` instead ?



##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .sessionCacheSize(0)
+      .sessionTimeout(0)
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");
+      }
+    }
+    return SslProvider.JDK;
+  }
+
+  public void destroy() {
+    if (trustManagers != null) {
+      for (int i = 0; i < trustManagers.length; i++) {
+        if (trustManagers[i] instanceof ReloadingX509TrustManager) {
+          try {
+            ((ReloadingX509TrustManager) trustManagers[i]).destroy();
+          } catch (InterruptedException ex) {
+            logger.error("Interrupted while destroying trust manager: " + ex.toString(), ex);
+          }
+        }
+      }
+      trustManagers = null;
+    }
+
+    keyManagers = null;
+    jdkSslContext = null;
+    nettyClientSslContext = null;
+    nettyServerSslContext = null;
+    requestedProtocol = null;
+    requestedCiphers = null;
+  }
+
+  /**
+   * Builder class to construct instances of {@link SSLFactory} with specific options
+   */
+  public static class Builder {
+    private String requestedProtocol;
+    private String[] requestedCiphers;
+    private File keyStore;
+    private String keyStorePassword;
+    private File privateKey;
+    private String keyPassword;
+    private File certChain;
+    private File trustStore;
+    private String trustStorePassword;
+    private boolean trustStoreReloadingEnabled;
+    private int trustStoreReloadIntervalMs;
+    private boolean openSslEnabled;
+
+    /**
+     * Sets the requested protocol, i.e., "TLSv1.2", "TLSv1.1", etc
+     *
+     * @param requestedProtocol The requested protocol
+     * @return The builder object
+     */
+    public Builder requestedProtocol(String requestedProtocol) {
+      this.requestedProtocol = requestedProtocol == null ? "TLSv1.2" : requestedProtocol;
+      return this;
+    }
+
+    /**
+     * Sets the requested cipher suites, i.e., "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", etc
+     *
+     * @param requestedCiphers The requested ciphers
+     * @return The builder object
+     */
+    public Builder requestedCiphers(String[] requestedCiphers) {
+      this.requestedCiphers = requestedCiphers;
+      return this;
+    }
+
+    /**
+     * Sets the Keystore and Keystore password
+     *
+     * @param keyStore The key store file to use
+     * @param keyStorePassword The password for the key store
+     * @return The builder object
+     */
+    public Builder keyStore(File keyStore, String keyStorePassword) {
+      this.keyStore = keyStore;
+      this.keyStorePassword = keyStorePassword;
+      return this;
+    }
+
+    /**
+     * Sets a PKCS#8 private key file in PEM format
+     *
+     * @param privateKey The private key file to use
+     * @return The builder object
+     */
+    public Builder privateKey(File privateKey) {
+      this.privateKey = privateKey;
+      return this;
+    }
+
+    /**
+     * Sets the Key password
+     *
+     * @param keyPassword The password for the private key
+     * @return The builder object
+     */
+    public Builder keyPassword(String keyPassword) {
+      this.keyPassword = keyPassword;
+      return this;
+    }
+
+    /**
+     * Sets a X.509 certificate chain file in PEM format
+     *
+     * @param certChain The certificate chain file to use
+     * @return The builder object
+     */
+    public Builder certChain(File certChain) {
+      this.certChain = certChain;
+      return this;
+    }
+
+    /**
+     * @param enabled Whether to use the OpenSSL implementation
+     * @return The builder object
+     */
+    public Builder openSslEnabled(boolean enabled) {
+      this.openSslEnabled = enabled;
+      return this;
+    }
+
+    /**
+     * Sets the trust-store, trust-store password, whether to use a Reloading TrustStore,
+     * and the trust-store reload interval, if enabled
+     *
+     * @param trustStore The trust store file to use
+     * @param trustStorePassword The password for the trust store
+     * @param trustStoreReloadingEnabled Whether trust store reloading is enabled
+     * @param trustStoreReloadIntervalMs The interval at which to reload the trust store file
+     * @return The builder object
+     */
+    public Builder trustStore(
+        File trustStore, String trustStorePassword,
+        boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs) {
+      this.trustStore = trustStore;
+      this.trustStorePassword = trustStorePassword;
+      this.trustStoreReloadingEnabled = trustStoreReloadingEnabled;
+      this.trustStoreReloadIntervalMs = trustStoreReloadIntervalMs;
+      return this;
+    }
+
+    /**
+     * Builds our {@link SSLFactory}
+     *
+     * @return The built {@link SSLFactory}
+     */
+    public SSLFactory build() {
+      return new SSLFactory(this);
+    }
+  }
+
+  /**
+   * Returns an initialized {@link SSLContext}
+   *
+   * @param requestedProtocol The requested protocol to use
+   * @param keyManagers The list of key managers to use
+   * @param trustManagers The list of trust managers to use
+   * @return The built {@link SSLContext}
+   * @throws GeneralSecurityException
+   */
+  private static SSLContext createSSLContext(
+      String requestedProtocol,
+      KeyManager[] keyManagers,
+      TrustManager[] trustManagers) throws GeneralSecurityException {
+    SSLContext sslContext = getSSLContextInstance(requestedProtocol);
+    sslContext.init(keyManagers, trustManagers, null);
+    return sslContext;
+  }
+
+  /**
+   * Get the {@link SSLContext} for the specified <tt>requestedProtocol</tt>
+   * if available, or the default {@link SSLContext}
+   * @param requestedProtocol The protocol to use
+   * @return The built {@link SSLContext}
+   * @throws NoSuchAlgorithmException
+   */
+  private static SSLContext getSSLContextInstance(String requestedProtocol)
+      throws NoSuchAlgorithmException {
+    SSLContext context = null;
+    try {
+      context = SSLContext.getInstance(requestedProtocol);
+    } catch (Exception e) {
+      context = SSLContext.getDefault();
+    }
+    return context;
+  }
+
+  /**
+   * Creates a new {@link SSLEngine}.
+   * Note that currently client auth is not supported
+   *
+   * @param isClient Whether the engine is used in a client context
+   * @param allocator The {@link ByteBufAllocator to use}
+   * @return A valid {@link SSLEngine}.
+   */
+  public SSLEngine createSSLEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine = createEngine(isClient, allocator);
+    engine.setUseClientMode(isClient);
+    engine.setNeedClientAuth(false);
+    engine.setEnabledProtocols(enabledProtocols(engine, requestedProtocol));
+    engine.setEnabledCipherSuites(enabledCipherSuites(engine, requestedCiphers));
+    return engine;
+  }
+
+  private SSLEngine createEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine;
+    if (isClient) {
+      if (nettyClientSslContext != null) {
+        engine = nettyClientSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    } else {
+      if (nettyServerSslContext != null) {
+        engine = nettyServerSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    }
+    return engine;
+  }
+
+  private static TrustManager[] credulousTrustStoreManagers() {
+    return new TrustManager[]{new X509TrustManager() {
+      @Override
+      public void checkClientTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public void checkServerTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public X509Certificate[] getAcceptedIssuers() {
+        return null;
+      }
+    }};
+  }
+
+  private static TrustManager[] trustStoreManagers(
+      File trustStore, String trustStorePassword,
+      boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs)
+          throws IOException, GeneralSecurityException {
+    if (trustStore == null || !trustStore.exists()) {
+      return credulousTrustStoreManagers();
+    } else {
+      if (trustStorePassword == null) {
+        throw new KeyStoreException(
+          "trustStorePassword cannot be null. Please configure spark.ssl.rpc.trustStorePassword");
+      }
+
+      if (trustStoreReloadingEnabled) {
+        ReloadingX509TrustManager reloading = new ReloadingX509TrustManager(
+          KeyStore.getDefaultType(), trustStore, trustStorePassword, trustStoreReloadIntervalMs);
+        reloading.init();
+        return new TrustManager[]{reloading};
+      } else {
+        return defaultTrustManagers(trustStore, trustStorePassword);
+      }
+    }
+  }
+
+  private static TrustManager[] defaultTrustManagers(File trustStore, String trustStorePassword)
+      throws IOException, KeyStoreException, CertificateException, NoSuchAlgorithmException {
+    InputStream input = Files.asByteSource(trustStore).openStream();
+    try {
+      KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+      ks.load(input, trustStorePassword.toCharArray());
+      TrustManagerFactory tmf = TrustManagerFactory.getInstance(
+        TrustManagerFactory.getDefaultAlgorithm());
+      tmf.init(ks);
+      return tmf.getTrustManagers();
+    } finally {
+      JavaUtils.closeQuietly(input);
+    }

Review Comment:
   Use try-with-resources instead
   
   ```suggestion
       try (InputStream input = Files.asByteSource(trustStore).openStream()) {
         KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
         ks.load(input, null != trustStorePassword ? trustStorePassword.toCharArray() : null);
         TrustManagerFactory tmf = TrustManagerFactory.getInstance(
           TrustManagerFactory.getDefaultAlgorithm());
         tmf.init(ks);
         return tmf.getTrustManagers();
       }
   ```



##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .sessionCacheSize(0)
+      .sessionTimeout(0)
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");
+      }
+    }
+    return SslProvider.JDK;
+  }
+
+  public void destroy() {
+    if (trustManagers != null) {
+      for (int i = 0; i < trustManagers.length; i++) {
+        if (trustManagers[i] instanceof ReloadingX509TrustManager) {
+          try {
+            ((ReloadingX509TrustManager) trustManagers[i]).destroy();
+          } catch (InterruptedException ex) {
+            logger.error("Interrupted while destroying trust manager: " + ex.toString(), ex);
+          }
+        }
+      }
+      trustManagers = null;
+    }
+
+    keyManagers = null;
+    jdkSslContext = null;
+    nettyClientSslContext = null;
+    nettyServerSslContext = null;
+    requestedProtocol = null;
+    requestedCiphers = null;
+  }
+
+  /**
+   * Builder class to construct instances of {@link SSLFactory} with specific options
+   */
+  public static class Builder {
+    private String requestedProtocol;
+    private String[] requestedCiphers;
+    private File keyStore;
+    private String keyStorePassword;
+    private File privateKey;
+    private String keyPassword;
+    private File certChain;
+    private File trustStore;
+    private String trustStorePassword;
+    private boolean trustStoreReloadingEnabled;
+    private int trustStoreReloadIntervalMs;
+    private boolean openSslEnabled;
+
+    /**
+     * Sets the requested protocol, i.e., "TLSv1.2", "TLSv1.1", etc
+     *
+     * @param requestedProtocol The requested protocol
+     * @return The builder object
+     */
+    public Builder requestedProtocol(String requestedProtocol) {
+      this.requestedProtocol = requestedProtocol == null ? "TLSv1.2" : requestedProtocol;
+      return this;
+    }
+
+    /**
+     * Sets the requested cipher suites, i.e., "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", etc
+     *
+     * @param requestedCiphers The requested ciphers
+     * @return The builder object
+     */
+    public Builder requestedCiphers(String[] requestedCiphers) {
+      this.requestedCiphers = requestedCiphers;
+      return this;
+    }
+
+    /**
+     * Sets the Keystore and Keystore password
+     *
+     * @param keyStore The key store file to use
+     * @param keyStorePassword The password for the key store
+     * @return The builder object
+     */
+    public Builder keyStore(File keyStore, String keyStorePassword) {
+      this.keyStore = keyStore;
+      this.keyStorePassword = keyStorePassword;
+      return this;
+    }
+
+    /**
+     * Sets a PKCS#8 private key file in PEM format
+     *
+     * @param privateKey The private key file to use
+     * @return The builder object
+     */
+    public Builder privateKey(File privateKey) {
+      this.privateKey = privateKey;
+      return this;
+    }
+
+    /**
+     * Sets the Key password
+     *
+     * @param keyPassword The password for the private key
+     * @return The builder object
+     */
+    public Builder keyPassword(String keyPassword) {
+      this.keyPassword = keyPassword;
+      return this;
+    }
+
+    /**
+     * Sets a X.509 certificate chain file in PEM format
+     *
+     * @param certChain The certificate chain file to use
+     * @return The builder object
+     */
+    public Builder certChain(File certChain) {
+      this.certChain = certChain;
+      return this;
+    }
+
+    /**
+     * @param enabled Whether to use the OpenSSL implementation
+     * @return The builder object
+     */
+    public Builder openSslEnabled(boolean enabled) {
+      this.openSslEnabled = enabled;
+      return this;
+    }
+
+    /**
+     * Sets the trust-store, trust-store password, whether to use a Reloading TrustStore,
+     * and the trust-store reload interval, if enabled
+     *
+     * @param trustStore The trust store file to use
+     * @param trustStorePassword The password for the trust store
+     * @param trustStoreReloadingEnabled Whether trust store reloading is enabled
+     * @param trustStoreReloadIntervalMs The interval at which to reload the trust store file
+     * @return The builder object
+     */
+    public Builder trustStore(
+        File trustStore, String trustStorePassword,
+        boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs) {
+      this.trustStore = trustStore;
+      this.trustStorePassword = trustStorePassword;
+      this.trustStoreReloadingEnabled = trustStoreReloadingEnabled;
+      this.trustStoreReloadIntervalMs = trustStoreReloadIntervalMs;
+      return this;
+    }
+
+    /**
+     * Builds our {@link SSLFactory}
+     *
+     * @return The built {@link SSLFactory}
+     */
+    public SSLFactory build() {
+      return new SSLFactory(this);
+    }
+  }
+
+  /**
+   * Returns an initialized {@link SSLContext}
+   *
+   * @param requestedProtocol The requested protocol to use
+   * @param keyManagers The list of key managers to use
+   * @param trustManagers The list of trust managers to use
+   * @return The built {@link SSLContext}
+   * @throws GeneralSecurityException
+   */
+  private static SSLContext createSSLContext(
+      String requestedProtocol,
+      KeyManager[] keyManagers,
+      TrustManager[] trustManagers) throws GeneralSecurityException {
+    SSLContext sslContext = getSSLContextInstance(requestedProtocol);
+    sslContext.init(keyManagers, trustManagers, null);
+    return sslContext;
+  }
+
+  /**
+   * Get the {@link SSLContext} for the specified <tt>requestedProtocol</tt>
+   * if available, or the default {@link SSLContext}
+   * @param requestedProtocol The protocol to use
+   * @return The built {@link SSLContext}
+   * @throws NoSuchAlgorithmException
+   */
+  private static SSLContext getSSLContextInstance(String requestedProtocol)
+      throws NoSuchAlgorithmException {
+    SSLContext context = null;
+    try {
+      context = SSLContext.getInstance(requestedProtocol);
+    } catch (Exception e) {
+      context = SSLContext.getDefault();
+    }
+    return context;
+  }
+
+  /**
+   * Creates a new {@link SSLEngine}.
+   * Note that currently client auth is not supported
+   *
+   * @param isClient Whether the engine is used in a client context
+   * @param allocator The {@link ByteBufAllocator to use}
+   * @return A valid {@link SSLEngine}.
+   */
+  public SSLEngine createSSLEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine = createEngine(isClient, allocator);
+    engine.setUseClientMode(isClient);
+    engine.setNeedClientAuth(false);
+    engine.setEnabledProtocols(enabledProtocols(engine, requestedProtocol));
+    engine.setEnabledCipherSuites(enabledCipherSuites(engine, requestedCiphers));
+    return engine;
+  }
+
+  private SSLEngine createEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine;
+    if (isClient) {
+      if (nettyClientSslContext != null) {
+        engine = nettyClientSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    } else {
+      if (nettyServerSslContext != null) {
+        engine = nettyServerSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    }
+    return engine;
+  }
+
+  private static TrustManager[] credulousTrustStoreManagers() {
+    return new TrustManager[]{new X509TrustManager() {
+      @Override
+      public void checkClientTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public void checkServerTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public X509Certificate[] getAcceptedIssuers() {
+        return null;
+      }
+    }};
+  }
+
+  private static TrustManager[] trustStoreManagers(
+      File trustStore, String trustStorePassword,
+      boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs)
+          throws IOException, GeneralSecurityException {
+    if (trustStore == null || !trustStore.exists()) {
+      return credulousTrustStoreManagers();
+    } else {
+      if (trustStorePassword == null) {
+        throw new KeyStoreException(
+          "trustStorePassword cannot be null. Please configure spark.ssl.rpc.trustStorePassword");
+      }
+
+      if (trustStoreReloadingEnabled) {
+        ReloadingX509TrustManager reloading = new ReloadingX509TrustManager(
+          KeyStore.getDefaultType(), trustStore, trustStorePassword, trustStoreReloadIntervalMs);
+        reloading.init();
+        return new TrustManager[]{reloading};
+      } else {
+        return defaultTrustManagers(trustStore, trustStorePassword);
+      }
+    }
+  }
+
+  private static TrustManager[] defaultTrustManagers(File trustStore, String trustStorePassword)
+      throws IOException, KeyStoreException, CertificateException, NoSuchAlgorithmException {
+    InputStream input = Files.asByteSource(trustStore).openStream();
+    try {
+      KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+      ks.load(input, trustStorePassword.toCharArray());
+      TrustManagerFactory tmf = TrustManagerFactory.getInstance(
+        TrustManagerFactory.getDefaultAlgorithm());
+      tmf.init(ks);
+      return tmf.getTrustManagers();
+    } finally {
+      JavaUtils.closeQuietly(input);
+    }
+  }
+
+  private static KeyManager[] keyManagers(File keyStore, String keyStorePassword)
+      throws NoSuchAlgorithmException, CertificateException,
+          KeyStoreException, IOException, UnrecoverableKeyException {
+    KeyManagerFactory factory = KeyManagerFactory.getInstance(
+      KeyManagerFactory.getDefaultAlgorithm());
+    factory.init(
+      loadKeyStore(keyStore, keyStorePassword),
+      (keyStorePassword != null ? keyStorePassword.toCharArray() : null)

Review Comment:
   super nit:
   ```suggestion
         keyStorePassword != null ? keyStorePassword.toCharArray() : null
   ```



##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private final Logger logger = LoggerFactory.getLogger(SSLFactory.class);

Review Comment:
   This was missed out in `ReloadingX509TrustManager` and `SslMessageEncoder` as well, can you update them also ? Thanks



##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .sessionCacheSize(0)
+      .sessionTimeout(0)
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");
+      }
+    }
+    return SslProvider.JDK;
+  }
+
+  public void destroy() {
+    if (trustManagers != null) {
+      for (int i = 0; i < trustManagers.length; i++) {
+        if (trustManagers[i] instanceof ReloadingX509TrustManager) {
+          try {
+            ((ReloadingX509TrustManager) trustManagers[i]).destroy();
+          } catch (InterruptedException ex) {
+            logger.error("Interrupted while destroying trust manager: " + ex.toString(), ex);
+          }
+        }
+      }
+      trustManagers = null;
+    }
+
+    keyManagers = null;
+    jdkSslContext = null;
+    nettyClientSslContext = null;
+    nettyServerSslContext = null;
+    requestedProtocol = null;
+    requestedCiphers = null;
+  }
+
+  /**
+   * Builder class to construct instances of {@link SSLFactory} with specific options
+   */
+  public static class Builder {
+    private String requestedProtocol;
+    private String[] requestedCiphers;
+    private File keyStore;
+    private String keyStorePassword;
+    private File privateKey;
+    private String keyPassword;
+    private File certChain;
+    private File trustStore;
+    private String trustStorePassword;
+    private boolean trustStoreReloadingEnabled;
+    private int trustStoreReloadIntervalMs;
+    private boolean openSslEnabled;
+
+    /**
+     * Sets the requested protocol, i.e., "TLSv1.2", "TLSv1.1", etc
+     *
+     * @param requestedProtocol The requested protocol
+     * @return The builder object
+     */
+    public Builder requestedProtocol(String requestedProtocol) {
+      this.requestedProtocol = requestedProtocol == null ? "TLSv1.2" : requestedProtocol;
+      return this;
+    }
+
+    /**
+     * Sets the requested cipher suites, i.e., "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", etc
+     *
+     * @param requestedCiphers The requested ciphers
+     * @return The builder object
+     */
+    public Builder requestedCiphers(String[] requestedCiphers) {
+      this.requestedCiphers = requestedCiphers;
+      return this;
+    }
+
+    /**
+     * Sets the Keystore and Keystore password
+     *
+     * @param keyStore The key store file to use
+     * @param keyStorePassword The password for the key store
+     * @return The builder object
+     */
+    public Builder keyStore(File keyStore, String keyStorePassword) {
+      this.keyStore = keyStore;
+      this.keyStorePassword = keyStorePassword;
+      return this;
+    }
+
+    /**
+     * Sets a PKCS#8 private key file in PEM format
+     *
+     * @param privateKey The private key file to use
+     * @return The builder object
+     */
+    public Builder privateKey(File privateKey) {
+      this.privateKey = privateKey;
+      return this;
+    }
+
+    /**
+     * Sets the Key password
+     *
+     * @param keyPassword The password for the private key
+     * @return The builder object
+     */
+    public Builder keyPassword(String keyPassword) {
+      this.keyPassword = keyPassword;
+      return this;
+    }
+
+    /**
+     * Sets a X.509 certificate chain file in PEM format
+     *
+     * @param certChain The certificate chain file to use
+     * @return The builder object
+     */
+    public Builder certChain(File certChain) {
+      this.certChain = certChain;
+      return this;
+    }
+
+    /**
+     * @param enabled Whether to use the OpenSSL implementation
+     * @return The builder object
+     */
+    public Builder openSslEnabled(boolean enabled) {
+      this.openSslEnabled = enabled;
+      return this;
+    }
+
+    /**
+     * Sets the trust-store, trust-store password, whether to use a Reloading TrustStore,
+     * and the trust-store reload interval, if enabled
+     *
+     * @param trustStore The trust store file to use
+     * @param trustStorePassword The password for the trust store
+     * @param trustStoreReloadingEnabled Whether trust store reloading is enabled
+     * @param trustStoreReloadIntervalMs The interval at which to reload the trust store file
+     * @return The builder object
+     */
+    public Builder trustStore(
+        File trustStore, String trustStorePassword,
+        boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs) {
+      this.trustStore = trustStore;
+      this.trustStorePassword = trustStorePassword;
+      this.trustStoreReloadingEnabled = trustStoreReloadingEnabled;
+      this.trustStoreReloadIntervalMs = trustStoreReloadIntervalMs;
+      return this;
+    }
+
+    /**
+     * Builds our {@link SSLFactory}
+     *
+     * @return The built {@link SSLFactory}
+     */
+    public SSLFactory build() {
+      return new SSLFactory(this);
+    }
+  }
+
+  /**
+   * Returns an initialized {@link SSLContext}
+   *
+   * @param requestedProtocol The requested protocol to use
+   * @param keyManagers The list of key managers to use
+   * @param trustManagers The list of trust managers to use
+   * @return The built {@link SSLContext}
+   * @throws GeneralSecurityException
+   */
+  private static SSLContext createSSLContext(
+      String requestedProtocol,
+      KeyManager[] keyManagers,
+      TrustManager[] trustManagers) throws GeneralSecurityException {
+    SSLContext sslContext = getSSLContextInstance(requestedProtocol);
+    sslContext.init(keyManagers, trustManagers, null);
+    return sslContext;
+  }
+
+  /**
+   * Get the {@link SSLContext} for the specified <tt>requestedProtocol</tt>
+   * if available, or the default {@link SSLContext}
+   * @param requestedProtocol The protocol to use
+   * @return The built {@link SSLContext}
+   * @throws NoSuchAlgorithmException
+   */
+  private static SSLContext getSSLContextInstance(String requestedProtocol)
+      throws NoSuchAlgorithmException {
+    SSLContext context = null;
+    try {
+      context = SSLContext.getInstance(requestedProtocol);
+    } catch (Exception e) {
+      context = SSLContext.getDefault();

Review Comment:
   Discuss: If the request protocol is invalid or unsupported, should we propagate the exception instead ?



##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .sessionCacheSize(0)
+      .sessionTimeout(0)
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");
+      }
+    }
+    return SslProvider.JDK;
+  }
+
+  public void destroy() {
+    if (trustManagers != null) {
+      for (int i = 0; i < trustManagers.length; i++) {
+        if (trustManagers[i] instanceof ReloadingX509TrustManager) {
+          try {
+            ((ReloadingX509TrustManager) trustManagers[i]).destroy();
+          } catch (InterruptedException ex) {
+            logger.error("Interrupted while destroying trust manager: " + ex.toString(), ex);
+          }
+        }
+      }
+      trustManagers = null;
+    }
+
+    keyManagers = null;
+    jdkSslContext = null;
+    nettyClientSslContext = null;
+    nettyServerSslContext = null;
+    requestedProtocol = null;
+    requestedCiphers = null;
+  }
+
+  /**
+   * Builder class to construct instances of {@link SSLFactory} with specific options
+   */
+  public static class Builder {
+    private String requestedProtocol;
+    private String[] requestedCiphers;
+    private File keyStore;
+    private String keyStorePassword;
+    private File privateKey;
+    private String keyPassword;
+    private File certChain;
+    private File trustStore;
+    private String trustStorePassword;
+    private boolean trustStoreReloadingEnabled;
+    private int trustStoreReloadIntervalMs;
+    private boolean openSslEnabled;
+
+    /**
+     * Sets the requested protocol, i.e., "TLSv1.2", "TLSv1.1", etc
+     *
+     * @param requestedProtocol The requested protocol
+     * @return The builder object
+     */
+    public Builder requestedProtocol(String requestedProtocol) {
+      this.requestedProtocol = requestedProtocol == null ? "TLSv1.2" : requestedProtocol;
+      return this;
+    }
+
+    /**
+     * Sets the requested cipher suites, i.e., "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", etc
+     *
+     * @param requestedCiphers The requested ciphers
+     * @return The builder object
+     */
+    public Builder requestedCiphers(String[] requestedCiphers) {
+      this.requestedCiphers = requestedCiphers;
+      return this;
+    }
+
+    /**
+     * Sets the Keystore and Keystore password
+     *
+     * @param keyStore The key store file to use
+     * @param keyStorePassword The password for the key store
+     * @return The builder object
+     */
+    public Builder keyStore(File keyStore, String keyStorePassword) {
+      this.keyStore = keyStore;
+      this.keyStorePassword = keyStorePassword;
+      return this;
+    }
+
+    /**
+     * Sets a PKCS#8 private key file in PEM format
+     *
+     * @param privateKey The private key file to use
+     * @return The builder object
+     */
+    public Builder privateKey(File privateKey) {
+      this.privateKey = privateKey;
+      return this;
+    }
+
+    /**
+     * Sets the Key password
+     *
+     * @param keyPassword The password for the private key
+     * @return The builder object
+     */
+    public Builder keyPassword(String keyPassword) {
+      this.keyPassword = keyPassword;
+      return this;
+    }
+
+    /**
+     * Sets a X.509 certificate chain file in PEM format
+     *
+     * @param certChain The certificate chain file to use
+     * @return The builder object
+     */
+    public Builder certChain(File certChain) {
+      this.certChain = certChain;
+      return this;
+    }
+
+    /**
+     * @param enabled Whether to use the OpenSSL implementation
+     * @return The builder object
+     */
+    public Builder openSslEnabled(boolean enabled) {
+      this.openSslEnabled = enabled;
+      return this;
+    }
+
+    /**
+     * Sets the trust-store, trust-store password, whether to use a Reloading TrustStore,
+     * and the trust-store reload interval, if enabled
+     *
+     * @param trustStore The trust store file to use
+     * @param trustStorePassword The password for the trust store
+     * @param trustStoreReloadingEnabled Whether trust store reloading is enabled
+     * @param trustStoreReloadIntervalMs The interval at which to reload the trust store file
+     * @return The builder object
+     */
+    public Builder trustStore(
+        File trustStore, String trustStorePassword,
+        boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs) {
+      this.trustStore = trustStore;
+      this.trustStorePassword = trustStorePassword;
+      this.trustStoreReloadingEnabled = trustStoreReloadingEnabled;
+      this.trustStoreReloadIntervalMs = trustStoreReloadIntervalMs;
+      return this;
+    }
+
+    /**
+     * Builds our {@link SSLFactory}
+     *
+     * @return The built {@link SSLFactory}
+     */
+    public SSLFactory build() {
+      return new SSLFactory(this);
+    }
+  }
+
+  /**
+   * Returns an initialized {@link SSLContext}
+   *
+   * @param requestedProtocol The requested protocol to use
+   * @param keyManagers The list of key managers to use
+   * @param trustManagers The list of trust managers to use
+   * @return The built {@link SSLContext}
+   * @throws GeneralSecurityException
+   */
+  private static SSLContext createSSLContext(
+      String requestedProtocol,
+      KeyManager[] keyManagers,
+      TrustManager[] trustManagers) throws GeneralSecurityException {
+    SSLContext sslContext = getSSLContextInstance(requestedProtocol);
+    sslContext.init(keyManagers, trustManagers, null);
+    return sslContext;
+  }
+
+  /**
+   * Get the {@link SSLContext} for the specified <tt>requestedProtocol</tt>
+   * if available, or the default {@link SSLContext}
+   * @param requestedProtocol The protocol to use
+   * @return The built {@link SSLContext}
+   * @throws NoSuchAlgorithmException
+   */
+  private static SSLContext getSSLContextInstance(String requestedProtocol)
+      throws NoSuchAlgorithmException {
+    SSLContext context = null;
+    try {
+      context = SSLContext.getInstance(requestedProtocol);
+    } catch (Exception e) {
+      context = SSLContext.getDefault();
+    }
+    return context;
+  }
+
+  /**
+   * Creates a new {@link SSLEngine}.
+   * Note that currently client auth is not supported
+   *
+   * @param isClient Whether the engine is used in a client context
+   * @param allocator The {@link ByteBufAllocator to use}
+   * @return A valid {@link SSLEngine}.
+   */
+  public SSLEngine createSSLEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine = createEngine(isClient, allocator);
+    engine.setUseClientMode(isClient);
+    engine.setNeedClientAuth(false);
+    engine.setEnabledProtocols(enabledProtocols(engine, requestedProtocol));
+    engine.setEnabledCipherSuites(enabledCipherSuites(engine, requestedCiphers));
+    return engine;
+  }
+
+  private SSLEngine createEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine;
+    if (isClient) {
+      if (nettyClientSslContext != null) {
+        engine = nettyClientSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    } else {
+      if (nettyServerSslContext != null) {
+        engine = nettyServerSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    }
+    return engine;
+  }
+
+  private static TrustManager[] credulousTrustStoreManagers() {
+    return new TrustManager[]{new X509TrustManager() {
+      @Override
+      public void checkClientTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public void checkServerTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public X509Certificate[] getAcceptedIssuers() {
+        return null;
+      }
+    }};
+  }
+
+  private static TrustManager[] trustStoreManagers(
+      File trustStore, String trustStorePassword,
+      boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs)
+          throws IOException, GeneralSecurityException {
+    if (trustStore == null || !trustStore.exists()) {
+      return credulousTrustStoreManagers();
+    } else {
+      if (trustStorePassword == null) {
+        throw new KeyStoreException(
+          "trustStorePassword cannot be null. Please configure spark.ssl.rpc.trustStorePassword");
+      }
+
+      if (trustStoreReloadingEnabled) {
+        ReloadingX509TrustManager reloading = new ReloadingX509TrustManager(
+          KeyStore.getDefaultType(), trustStore, trustStorePassword, trustStoreReloadIntervalMs);
+        reloading.init();
+        return new TrustManager[]{reloading};
+      } else {
+        return defaultTrustManagers(trustStore, trustStorePassword);
+      }
+    }
+  }
+
+  private static TrustManager[] defaultTrustManagers(File trustStore, String trustStorePassword)
+      throws IOException, KeyStoreException, CertificateException, NoSuchAlgorithmException {
+    InputStream input = Files.asByteSource(trustStore).openStream();
+    try {
+      KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+      ks.load(input, trustStorePassword.toCharArray());
+      TrustManagerFactory tmf = TrustManagerFactory.getInstance(
+        TrustManagerFactory.getDefaultAlgorithm());
+      tmf.init(ks);
+      return tmf.getTrustManagers();
+    } finally {
+      JavaUtils.closeQuietly(input);
+    }
+  }
+
+  private static KeyManager[] keyManagers(File keyStore, String keyStorePassword)
+      throws NoSuchAlgorithmException, CertificateException,
+          KeyStoreException, IOException, UnrecoverableKeyException {
+    KeyManagerFactory factory = KeyManagerFactory.getInstance(
+      KeyManagerFactory.getDefaultAlgorithm());
+    factory.init(
+      loadKeyStore(keyStore, keyStorePassword),
+      (keyStorePassword != null ? keyStorePassword.toCharArray() : null)
+    );
+
+    return factory.getKeyManagers();
+  }
+
+  private static KeyStore loadKeyStore(File keyStore, String keyStorePassword)
+      throws KeyStoreException, IOException, CertificateException, NoSuchAlgorithmException {
+    if (keyStore == null) {
+      throw new KeyStoreException(
+        "keyStore cannot be null. Please configure spark.ssl.rpc.keyStore");
+    }
+
+    if (keyStorePassword == null) {
+      throw new KeyStoreException(
+        "keyStorePassword cannot be null. Please configure spark.ssl.rpc.keyStorePassword");
+    }
+
+    KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+    FileInputStream fin = new FileInputStream(keyStore);
+    try {
+      ks.load(fin, keyStorePassword.toCharArray());
+      return ks;
+    } finally {
+      JavaUtils.closeQuietly(fin);
+    }
+  }
+
+  private static String[] enabledProtocols(SSLEngine engine, String requestedProtocol) {
+    String[] supportedProtocols = engine.getSupportedProtocols();
+    String[] defaultProtocols = {"TLSv1.3", "TLSv1.2"};
+    String[] enabledProtocols =
+      ((requestedProtocol == null || requestedProtocol.isEmpty()) ?
+        defaultProtocols : new String[]{requestedProtocol});
+    List<String> protocols = new ArrayList<String>();
+
+    addIfSupported(supportedProtocols, protocols, enabledProtocols);
+    if (!protocols.isEmpty()) {
+      return protocols.toArray(new String[protocols.size()]);
+    } else {
+      return supportedProtocols;
+    }
+  }
+
+  private static String[] enabledCipherSuites(
+      String[] supportedCiphers, String[] defaultCiphers, String[] requestedCiphers) {
+    String[] baseCiphers = new String[]{
+      // We take ciphers from the mozilla modern list first (for TLS 1.3):
+      // https://wiki.mozilla.org/Security/Server_Side_TLS
+      "TLS_CHACHA20_POLY1305_SHA256",
+      "TLS_AES_128_GCM_SHA256",
+      "TLS_AES_256_GCM_SHA384",
+      // Next we have the TLS1.2 ciphers for intermediate compatibility (since JDK8 does not
+      // support TLS1.3)
+      "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256",
+      "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384",
+      "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256",
+      "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384",
+      "TLS_ECDHE_ECDSA_WITH_CHACHA20_POLY1305_SHA256",
+      "TLS_ECDHE_RSA_WITH_CHACHA20_POLY1305_SHA256",
+      "TLS_DHE_RSA_WITH_AES_128_GCM_SHA256",
+      "TLS_DHE_RSA_WITH_AES_256_GCM_SHA384"
+    };
+    String[] enabledCiphers =
+      ((requestedCiphers == null || requestedCiphers.length == 0) ? baseCiphers : requestedCiphers);
+    List<String> ciphers = new ArrayList<String>();
+
+    addIfSupported(supportedCiphers, ciphers, enabledCiphers);
+    if (!ciphers.isEmpty()) {
+      return ciphers.toArray(new String[ciphers.size()]);
+    } else {
+      // Use the default from JDK as fallback.
+      return defaultCiphers;
+    }
+  }
+
+  private static String[] enabledCipherSuites(SSLEngine engine, String[] requestedCiphers) {
+    return enabledCipherSuites(
+      engine.getSupportedCipherSuites(), engine.getEnabledCipherSuites(), requestedCiphers);
+  }
+
+  private static void addIfSupported(String[] supported, List<String> enabled, String... names) {

Review Comment:
   nit:
   ```suggestion
     private static List<String> addIfSupported(Set<String> supported, String... names) {
   ```



##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .sessionCacheSize(0)
+      .sessionTimeout(0)
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");
+      }
+    }
+    return SslProvider.JDK;
+  }
+
+  public void destroy() {
+    if (trustManagers != null) {
+      for (int i = 0; i < trustManagers.length; i++) {
+        if (trustManagers[i] instanceof ReloadingX509TrustManager) {
+          try {
+            ((ReloadingX509TrustManager) trustManagers[i]).destroy();
+          } catch (InterruptedException ex) {
+            logger.error("Interrupted while destroying trust manager: " + ex.toString(), ex);
+          }
+        }
+      }
+      trustManagers = null;
+    }
+
+    keyManagers = null;
+    jdkSslContext = null;
+    nettyClientSslContext = null;
+    nettyServerSslContext = null;
+    requestedProtocol = null;
+    requestedCiphers = null;
+  }
+
+  /**
+   * Builder class to construct instances of {@link SSLFactory} with specific options
+   */
+  public static class Builder {
+    private String requestedProtocol;
+    private String[] requestedCiphers;
+    private File keyStore;
+    private String keyStorePassword;
+    private File privateKey;
+    private String keyPassword;
+    private File certChain;
+    private File trustStore;
+    private String trustStorePassword;
+    private boolean trustStoreReloadingEnabled;
+    private int trustStoreReloadIntervalMs;
+    private boolean openSslEnabled;
+
+    /**
+     * Sets the requested protocol, i.e., "TLSv1.2", "TLSv1.1", etc
+     *
+     * @param requestedProtocol The requested protocol
+     * @return The builder object
+     */
+    public Builder requestedProtocol(String requestedProtocol) {
+      this.requestedProtocol = requestedProtocol == null ? "TLSv1.2" : requestedProtocol;
+      return this;
+    }
+
+    /**
+     * Sets the requested cipher suites, i.e., "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", etc
+     *
+     * @param requestedCiphers The requested ciphers
+     * @return The builder object
+     */
+    public Builder requestedCiphers(String[] requestedCiphers) {
+      this.requestedCiphers = requestedCiphers;
+      return this;
+    }
+
+    /**
+     * Sets the Keystore and Keystore password
+     *
+     * @param keyStore The key store file to use
+     * @param keyStorePassword The password for the key store
+     * @return The builder object
+     */
+    public Builder keyStore(File keyStore, String keyStorePassword) {
+      this.keyStore = keyStore;
+      this.keyStorePassword = keyStorePassword;
+      return this;
+    }
+
+    /**
+     * Sets a PKCS#8 private key file in PEM format
+     *
+     * @param privateKey The private key file to use
+     * @return The builder object
+     */
+    public Builder privateKey(File privateKey) {
+      this.privateKey = privateKey;
+      return this;
+    }
+
+    /**
+     * Sets the Key password
+     *
+     * @param keyPassword The password for the private key
+     * @return The builder object
+     */
+    public Builder keyPassword(String keyPassword) {
+      this.keyPassword = keyPassword;
+      return this;
+    }
+
+    /**
+     * Sets a X.509 certificate chain file in PEM format
+     *
+     * @param certChain The certificate chain file to use
+     * @return The builder object
+     */
+    public Builder certChain(File certChain) {
+      this.certChain = certChain;
+      return this;
+    }
+
+    /**
+     * @param enabled Whether to use the OpenSSL implementation
+     * @return The builder object
+     */
+    public Builder openSslEnabled(boolean enabled) {
+      this.openSslEnabled = enabled;
+      return this;
+    }
+
+    /**
+     * Sets the trust-store, trust-store password, whether to use a Reloading TrustStore,
+     * and the trust-store reload interval, if enabled
+     *
+     * @param trustStore The trust store file to use
+     * @param trustStorePassword The password for the trust store
+     * @param trustStoreReloadingEnabled Whether trust store reloading is enabled
+     * @param trustStoreReloadIntervalMs The interval at which to reload the trust store file
+     * @return The builder object
+     */
+    public Builder trustStore(
+        File trustStore, String trustStorePassword,
+        boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs) {
+      this.trustStore = trustStore;
+      this.trustStorePassword = trustStorePassword;
+      this.trustStoreReloadingEnabled = trustStoreReloadingEnabled;
+      this.trustStoreReloadIntervalMs = trustStoreReloadIntervalMs;
+      return this;
+    }
+
+    /**
+     * Builds our {@link SSLFactory}
+     *
+     * @return The built {@link SSLFactory}
+     */
+    public SSLFactory build() {
+      return new SSLFactory(this);
+    }
+  }
+
+  /**
+   * Returns an initialized {@link SSLContext}
+   *
+   * @param requestedProtocol The requested protocol to use
+   * @param keyManagers The list of key managers to use
+   * @param trustManagers The list of trust managers to use
+   * @return The built {@link SSLContext}
+   * @throws GeneralSecurityException
+   */
+  private static SSLContext createSSLContext(
+      String requestedProtocol,
+      KeyManager[] keyManagers,
+      TrustManager[] trustManagers) throws GeneralSecurityException {
+    SSLContext sslContext = getSSLContextInstance(requestedProtocol);
+    sslContext.init(keyManagers, trustManagers, null);
+    return sslContext;
+  }
+
+  /**
+   * Get the {@link SSLContext} for the specified <tt>requestedProtocol</tt>
+   * if available, or the default {@link SSLContext}
+   * @param requestedProtocol The protocol to use
+   * @return The built {@link SSLContext}
+   * @throws NoSuchAlgorithmException
+   */
+  private static SSLContext getSSLContextInstance(String requestedProtocol)
+      throws NoSuchAlgorithmException {
+    SSLContext context = null;
+    try {
+      context = SSLContext.getInstance(requestedProtocol);
+    } catch (Exception e) {
+      context = SSLContext.getDefault();
+    }
+    return context;
+  }
+
+  /**
+   * Creates a new {@link SSLEngine}.
+   * Note that currently client auth is not supported
+   *
+   * @param isClient Whether the engine is used in a client context
+   * @param allocator The {@link ByteBufAllocator to use}
+   * @return A valid {@link SSLEngine}.
+   */
+  public SSLEngine createSSLEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine = createEngine(isClient, allocator);
+    engine.setUseClientMode(isClient);
+    engine.setNeedClientAuth(false);
+    engine.setEnabledProtocols(enabledProtocols(engine, requestedProtocol));
+    engine.setEnabledCipherSuites(enabledCipherSuites(engine, requestedCiphers));
+    return engine;
+  }
+
+  private SSLEngine createEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine;
+    if (isClient) {
+      if (nettyClientSslContext != null) {
+        engine = nettyClientSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    } else {
+      if (nettyServerSslContext != null) {
+        engine = nettyServerSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    }
+    return engine;
+  }
+
+  private static TrustManager[] credulousTrustStoreManagers() {
+    return new TrustManager[]{new X509TrustManager() {
+      @Override
+      public void checkClientTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public void checkServerTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public X509Certificate[] getAcceptedIssuers() {
+        return null;
+      }
+    }};
+  }
+
+  private static TrustManager[] trustStoreManagers(
+      File trustStore, String trustStorePassword,
+      boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs)
+          throws IOException, GeneralSecurityException {
+    if (trustStore == null || !trustStore.exists()) {
+      return credulousTrustStoreManagers();
+    } else {
+      if (trustStorePassword == null) {

Review Comment:
   Both keystore and trust store does not need a password - in which case, `null` is used.
   Drop the `null` check ? (and handle for key store as well)
   
   (We will need to handle `null` password in `ReloadingX509TrustManager`, etc as well)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45541][CORE] Add SSLFactory [spark]

Posted by "hasnain-db (via GitHub)" <gi...@apache.org>.
hasnain-db commented on code in PR #43386:
URL: https://github.com/apache/spark/pull/43386#discussion_r1364167962


##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .sessionCacheSize(0)
+      .sessionTimeout(0)
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");

Review Comment:
   it should work fine, will just be slower on the nodes running the JDK provider



##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,503 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .sessionCacheSize(0)
+      .sessionTimeout(0)
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");
+      }
+    }
+    return SslProvider.JDK;
+  }
+
+  public void destroy() {
+    if (trustManagers != null) {
+      for (int i = 0; i < trustManagers.length; i++) {
+        if (trustManagers[i] instanceof ReloadingX509TrustManager) {
+          try {
+            ((ReloadingX509TrustManager) trustManagers[i]).destroy();
+          } catch (InterruptedException ex) {
+            logger.error("Interrupted while destroying trust manager: " + ex.toString(), ex);
+          }
+        }
+      }
+      trustManagers = null;
+    }
+
+    keyManagers = null;
+    jdkSslContext = null;
+    nettyClientSslContext = null;
+    nettyServerSslContext = null;
+    requestedProtocol = null;
+    requestedCiphers = null;
+  }
+
+  /**
+   * Builder class to construct instances of {@link SSLFactory} with specific options
+   */
+  public static class Builder {
+    private String requestedProtocol;
+    private String[] requestedCiphers;
+    private File keyStore;
+    private String keyStorePassword;
+    private File privateKey;
+    private String keyPassword;
+    private File certChain;
+    private File trustStore;
+    private String trustStorePassword;
+    private boolean trustStoreReloadingEnabled;
+    private int trustStoreReloadIntervalMs;
+    private boolean openSslEnabled;
+
+    /**
+     * Sets the requested protocol, i.e., "TLSv1.2", "TLSv1.1", etc
+     *
+     * @param requestedProtocol The requested protocol
+     * @return The builder object
+     */
+    public Builder requestedProtocol(String requestedProtocol) {
+      this.requestedProtocol = requestedProtocol == null ? "TLSv1.2" : requestedProtocol;
+      return this;
+    }
+
+    /**
+     * Sets the requested cipher suites, i.e., "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", etc
+     *
+     * @param requestedCiphers The requested ciphers
+     * @return The builder object
+     */
+    public Builder requestedCiphers(String[] requestedCiphers) {
+      this.requestedCiphers = requestedCiphers;
+      return this;
+    }
+
+    /**
+     * Sets the Keystore and Keystore password
+     *
+     * @param keyStore The key store file to use
+     * @param keyStorePassword The password for the key store
+     * @return The builder object
+     */
+    public Builder keyStore(File keyStore, String keyStorePassword) {
+      this.keyStore = keyStore;
+      this.keyStorePassword = keyStorePassword;
+      return this;
+    }
+
+    /**
+     * Sets a PKCS#8 private key file in PEM format
+     *
+     * @param privateKey The private key file to use
+     * @return The builder object
+     */
+    public Builder privateKey(File privateKey) {
+      this.privateKey = privateKey;
+      return this;
+    }
+
+    /**
+     * Sets the Key password
+     *
+     * @param keyPassword The password for the private key
+     * @return The builder object
+     */
+    public Builder keyPassword(String keyPassword) {
+      this.keyPassword = keyPassword;
+      return this;
+    }
+
+    /**
+     * Sets a X.509 certificate chain file in PEM format
+     *
+     * @param certChain The certificate chain file to use
+     * @return The builder object
+     */
+    public Builder certChain(File certChain) {
+      this.certChain = certChain;
+      return this;
+    }
+
+    /**
+     * @param enabled Whether to use the OpenSSL implementation
+     * @return The builder object
+     */
+    public Builder openSslEnabled(boolean enabled) {
+      this.openSslEnabled = enabled;
+      return this;
+    }
+
+    /**
+     * Sets the trust-store, trust-store password, whether to use a Reloading TrustStore,
+     * and the trust-store reload interval, if enabled
+     *
+     * @param trustStore The trust store file to use
+     * @param trustStorePassword The password for the trust store
+     * @param trustStoreReloadingEnabled Whether trust store reloading is enabled
+     * @param trustStoreReloadIntervalMs The interval at which to reload the trust store file
+     * @return The builder object
+     */
+    public Builder trustStore(
+        File trustStore, String trustStorePassword,
+        boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs) {
+      this.trustStore = trustStore;
+      this.trustStorePassword = trustStorePassword;
+      this.trustStoreReloadingEnabled = trustStoreReloadingEnabled;
+      this.trustStoreReloadIntervalMs = trustStoreReloadIntervalMs;
+      return this;
+    }
+
+    /**
+     * Builds our {@link SSLFactory}
+     *
+     * @return The built {@link SSLFactory}
+     */
+    public SSLFactory build() {
+      return new SSLFactory(this);
+    }
+  }
+
+  /**
+   * Returns an initialized {@link SSLContext}
+   *
+   * @param requestedProtocol The requested protocol to use
+   * @param keyManagers The list of key managers to use
+   * @param trustManagers The list of trust managers to use
+   * @return The built {@link SSLContext}
+   * @throws GeneralSecurityException
+   */
+  private static SSLContext createSSLContext(
+      String requestedProtocol,
+      KeyManager[] keyManagers,
+      TrustManager[] trustManagers) throws GeneralSecurityException {
+    SSLContext sslContext = getSSLContextInstance(requestedProtocol);
+    sslContext.init(keyManagers, trustManagers, null);
+    return sslContext;
+  }
+
+  /**
+   * Get the {@link SSLContext} for the specified <tt>requestedProtocol</tt>
+   * if available, or the default {@link SSLContext}
+   * @param requestedProtocol The protocol to use
+   * @return The built {@link SSLContext}
+   * @throws NoSuchAlgorithmException
+   */
+  private static SSLContext getSSLContextInstance(String requestedProtocol)
+      throws NoSuchAlgorithmException {
+    SSLContext context = null;
+    try {
+      context = SSLContext.getInstance(requestedProtocol);
+    } catch (Exception e) {
+      context = SSLContext.getDefault();

Review Comment:
   I think this makes sense. will do.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45541][CORE] Add SSLFactory [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm commented on code in PR #43386:
URL: https://github.com/apache/spark/pull/43386#discussion_r1367691734


##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,471 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private static final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");
+      }
+    }
+    return SslProvider.JDK;
+  }
+
+  public void destroy() {
+    if (trustManagers != null) {
+      for (int i = 0; i < trustManagers.length; i++) {
+        if (trustManagers[i] instanceof ReloadingX509TrustManager) {
+          try {
+            ((ReloadingX509TrustManager) trustManagers[i]).destroy();
+          } catch (InterruptedException ex) {
+            logger.info("Interrupted while destroying trust manager: " + ex.toString(), ex);
+          }
+        }
+      }
+      trustManagers = null;
+    }
+
+    keyManagers = null;
+    jdkSslContext = null;
+    nettyClientSslContext = null;
+    nettyServerSslContext = null;
+    requestedProtocol = null;
+    requestedCiphers = null;
+  }
+
+  /**
+   * Builder class to construct instances of {@link SSLFactory} with specific options
+   */
+  public static class Builder {
+    private String requestedProtocol;
+    private String[] requestedCiphers;
+    private File keyStore;
+    private String keyStorePassword;
+    private File privateKey;
+    private String keyPassword;
+    private File certChain;
+    private File trustStore;
+    private String trustStorePassword;
+    private boolean trustStoreReloadingEnabled;
+    private int trustStoreReloadIntervalMs;
+    private boolean openSslEnabled;
+
+    /**
+     * Sets the requested protocol, i.e., "TLSv1.2", "TLSv1.1", etc
+     *
+     * @param requestedProtocol The requested protocol
+     * @return The builder object
+     */
+    public Builder requestedProtocol(String requestedProtocol) {
+      this.requestedProtocol = requestedProtocol == null ? "TLSv1.2" : requestedProtocol;
+      return this;
+    }
+
+    /**
+     * Sets the requested cipher suites, i.e., "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", etc
+     *
+     * @param requestedCiphers The requested ciphers
+     * @return The builder object
+     */
+    public Builder requestedCiphers(String[] requestedCiphers) {
+      this.requestedCiphers = requestedCiphers;
+      return this;
+    }
+
+    /**
+     * Sets the Keystore and Keystore password
+     *
+     * @param keyStore The key store file to use
+     * @param keyStorePassword The password for the key store
+     * @return The builder object
+     */
+    public Builder keyStore(File keyStore, String keyStorePassword) {
+      this.keyStore = keyStore;
+      this.keyStorePassword = keyStorePassword;
+      return this;
+    }
+
+    /**
+     * Sets a PKCS#8 private key file in PEM format
+     *
+     * @param privateKey The private key file to use
+     * @return The builder object
+     */
+    public Builder privateKey(File privateKey) {
+      this.privateKey = privateKey;
+      return this;
+    }
+
+    /**
+     * Sets the Key password
+     *
+     * @param keyPassword The password for the private key
+     * @return The builder object
+     */
+    public Builder keyPassword(String keyPassword) {
+      this.keyPassword = keyPassword;
+      return this;
+    }
+
+    /**
+     * Sets a X.509 certificate chain file in PEM format
+     *
+     * @param certChain The certificate chain file to use
+     * @return The builder object
+     */
+    public Builder certChain(File certChain) {
+      this.certChain = certChain;
+      return this;
+    }
+
+    /**
+     * @param enabled Whether to use the OpenSSL implementation
+     * @return The builder object
+     */
+    public Builder openSslEnabled(boolean enabled) {
+      this.openSslEnabled = enabled;
+      return this;
+    }
+
+    /**
+     * Sets the trust-store, trust-store password, whether to use a Reloading TrustStore,
+     * and the trust-store reload interval, if enabled
+     *
+     * @param trustStore The trust store file to use
+     * @param trustStorePassword The password for the trust store
+     * @param trustStoreReloadingEnabled Whether trust store reloading is enabled
+     * @param trustStoreReloadIntervalMs The interval at which to reload the trust store file
+     * @return The builder object
+     */
+    public Builder trustStore(
+        File trustStore, String trustStorePassword,
+        boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs) {
+      this.trustStore = trustStore;
+      this.trustStorePassword = trustStorePassword;
+      this.trustStoreReloadingEnabled = trustStoreReloadingEnabled;
+      this.trustStoreReloadIntervalMs = trustStoreReloadIntervalMs;
+      return this;
+    }
+
+    /**
+     * Builds our {@link SSLFactory}
+     *
+     * @return The built {@link SSLFactory}
+     */
+    public SSLFactory build() {
+      return new SSLFactory(this);
+    }
+  }
+
+  /**
+   * Returns an initialized {@link SSLContext}
+   *
+   * @param requestedProtocol The requested protocol to use
+   * @param keyManagers The list of key managers to use
+   * @param trustManagers The list of trust managers to use
+   * @return The built {@link SSLContext}
+   * @throws GeneralSecurityException
+   */
+  private static SSLContext createSSLContext(
+      String requestedProtocol,
+      KeyManager[] keyManagers,
+      TrustManager[] trustManagers) throws GeneralSecurityException {
+    SSLContext sslContext = SSLContext.getInstance(requestedProtocol);
+    sslContext.init(keyManagers, trustManagers, null);
+    return sslContext;
+  }
+
+  /**
+   * Creates a new {@link SSLEngine}.
+   * Note that currently client auth is not supported
+   *
+   * @param isClient Whether the engine is used in a client context
+   * @param allocator The {@link ByteBufAllocator to use}
+   * @return A valid {@link SSLEngine}.
+   */
+  public SSLEngine createSSLEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine = createEngine(isClient, allocator);
+    engine.setUseClientMode(isClient);
+    engine.setNeedClientAuth(false);
+    engine.setEnabledProtocols(enabledProtocols(engine, requestedProtocol));
+    engine.setEnabledCipherSuites(enabledCipherSuites(engine, requestedCiphers));
+    return engine;
+  }
+
+  private SSLEngine createEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine;
+    if (isClient) {
+      if (nettyClientSslContext != null) {
+        engine = nettyClientSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    } else {
+      if (nettyServerSslContext != null) {
+        engine = nettyServerSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    }
+    return engine;
+  }
+
+  private static TrustManager[] credulousTrustStoreManagers() {
+    return new TrustManager[]{new X509TrustManager() {
+      @Override
+      public void checkClientTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public void checkServerTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public X509Certificate[] getAcceptedIssuers() {
+        return null;
+      }
+    }};
+  }
+
+  private static TrustManager[] trustStoreManagers(
+      File trustStore, String trustStorePassword,
+      boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs)
+          throws IOException, GeneralSecurityException {
+    if (trustStore == null || !trustStore.exists()) {
+      return credulousTrustStoreManagers();
+    } else {
+      if (trustStoreReloadingEnabled) {
+        ReloadingX509TrustManager reloading = new ReloadingX509TrustManager(
+          KeyStore.getDefaultType(), trustStore, trustStorePassword, trustStoreReloadIntervalMs);
+        reloading.init();
+        return new TrustManager[]{reloading};
+      } else {
+        return defaultTrustManagers(trustStore, trustStorePassword);
+      }
+    }
+  }
+
+  private static TrustManager[] defaultTrustManagers(File trustStore, String trustStorePassword)
+      throws IOException, KeyStoreException, CertificateException, NoSuchAlgorithmException {
+    try (InputStream input = Files.asByteSource(trustStore).openStream()) {
+      KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+      char[] passwordCharacters = trustStorePassword != null?
+        trustStorePassword.toCharArray() : null;
+      ks.load(input, passwordCharacters);
+      TrustManagerFactory tmf = TrustManagerFactory.getInstance(
+        TrustManagerFactory.getDefaultAlgorithm());
+      tmf.init(ks);
+      return tmf.getTrustManagers();
+    }
+  }
+
+  private static KeyManager[] keyManagers(File keyStore, String keyStorePassword)
+      throws NoSuchAlgorithmException, CertificateException,
+          KeyStoreException, IOException, UnrecoverableKeyException {
+    KeyManagerFactory factory = KeyManagerFactory.getInstance(
+      KeyManagerFactory.getDefaultAlgorithm());
+    char[] passwordCharacters = keyStorePassword != null? keyStorePassword.toCharArray() : null;
+    factory.init(loadKeyStore(keyStore, keyStorePassword), passwordCharacters);

Review Comment:
   You can pass `passwordCharacters` to `loadKeyStore`, no ? It is a private method and so no external mods to the array ?
   
   Or can either `factory.init` or `KeyStore.load` change it ? (IIRC no, but want to confirm)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45541][CORE] Add SSLFactory [spark]

Posted by "hasnain-db (via GitHub)" <gi...@apache.org>.
hasnain-db commented on PR #43386:
URL: https://github.com/apache/spark/pull/43386#issuecomment-1765433747

   cc @mridulm @JoshRosen now that tests are green


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45541][CORE] Add SSLFactory [spark]

Posted by "hasnain-db (via GitHub)" <gi...@apache.org>.
hasnain-db commented on PR #43386:
URL: https://github.com/apache/spark/pull/43386#issuecomment-1764790080

   test failures looked unrelated, retrying now though.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45541][CORE] Add SSLFactory [spark]

Posted by "hasnain-db (via GitHub)" <gi...@apache.org>.
hasnain-db commented on PR #43386:
URL: https://github.com/apache/spark/pull/43386#issuecomment-1773150321

   will re-request review once CI comes back


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45541][CORE] Add SSLFactory [spark]

Posted by "hasnain-db (via GitHub)" <gi...@apache.org>.
hasnain-db commented on code in PR #43386:
URL: https://github.com/apache/spark/pull/43386#discussion_r1366119840


##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,470 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private static final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");
+      }
+    }
+    return SslProvider.JDK;
+  }
+
+  public void destroy() {
+    if (trustManagers != null) {
+      for (int i = 0; i < trustManagers.length; i++) {
+        if (trustManagers[i] instanceof ReloadingX509TrustManager) {
+          try {
+            ((ReloadingX509TrustManager) trustManagers[i]).destroy();
+          } catch (InterruptedException ex) {
+            logger.info("Interrupted while destroying trust manager: " + ex.toString(), ex);
+          }
+        }
+      }
+      trustManagers = null;
+    }
+
+    keyManagers = null;
+    jdkSslContext = null;
+    nettyClientSslContext = null;
+    nettyServerSslContext = null;
+    requestedProtocol = null;
+    requestedCiphers = null;
+  }
+
+  /**
+   * Builder class to construct instances of {@link SSLFactory} with specific options
+   */
+  public static class Builder {
+    private String requestedProtocol;
+    private String[] requestedCiphers;
+    private File keyStore;
+    private String keyStorePassword;
+    private File privateKey;
+    private String keyPassword;
+    private File certChain;
+    private File trustStore;
+    private String trustStorePassword;
+    private boolean trustStoreReloadingEnabled;
+    private int trustStoreReloadIntervalMs;
+    private boolean openSslEnabled;
+
+    /**
+     * Sets the requested protocol, i.e., "TLSv1.2", "TLSv1.1", etc
+     *
+     * @param requestedProtocol The requested protocol
+     * @return The builder object
+     */
+    public Builder requestedProtocol(String requestedProtocol) {
+      this.requestedProtocol = requestedProtocol == null ? "TLSv1.2" : requestedProtocol;
+      return this;
+    }
+
+    /**
+     * Sets the requested cipher suites, i.e., "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", etc
+     *
+     * @param requestedCiphers The requested ciphers
+     * @return The builder object
+     */
+    public Builder requestedCiphers(String[] requestedCiphers) {
+      this.requestedCiphers = requestedCiphers;
+      return this;
+    }
+
+    /**
+     * Sets the Keystore and Keystore password
+     *
+     * @param keyStore The key store file to use
+     * @param keyStorePassword The password for the key store
+     * @return The builder object
+     */
+    public Builder keyStore(File keyStore, String keyStorePassword) {
+      this.keyStore = keyStore;
+      this.keyStorePassword = keyStorePassword;
+      return this;
+    }
+
+    /**
+     * Sets a PKCS#8 private key file in PEM format
+     *
+     * @param privateKey The private key file to use
+     * @return The builder object
+     */
+    public Builder privateKey(File privateKey) {
+      this.privateKey = privateKey;
+      return this;
+    }
+
+    /**
+     * Sets the Key password
+     *
+     * @param keyPassword The password for the private key
+     * @return The builder object
+     */
+    public Builder keyPassword(String keyPassword) {
+      this.keyPassword = keyPassword;
+      return this;
+    }
+
+    /**
+     * Sets a X.509 certificate chain file in PEM format
+     *
+     * @param certChain The certificate chain file to use
+     * @return The builder object
+     */
+    public Builder certChain(File certChain) {
+      this.certChain = certChain;
+      return this;
+    }
+
+    /**
+     * @param enabled Whether to use the OpenSSL implementation
+     * @return The builder object
+     */
+    public Builder openSslEnabled(boolean enabled) {
+      this.openSslEnabled = enabled;
+      return this;
+    }
+
+    /**
+     * Sets the trust-store, trust-store password, whether to use a Reloading TrustStore,
+     * and the trust-store reload interval, if enabled
+     *
+     * @param trustStore The trust store file to use
+     * @param trustStorePassword The password for the trust store
+     * @param trustStoreReloadingEnabled Whether trust store reloading is enabled
+     * @param trustStoreReloadIntervalMs The interval at which to reload the trust store file
+     * @return The builder object
+     */
+    public Builder trustStore(
+        File trustStore, String trustStorePassword,
+        boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs) {
+      this.trustStore = trustStore;
+      this.trustStorePassword = trustStorePassword;
+      this.trustStoreReloadingEnabled = trustStoreReloadingEnabled;
+      this.trustStoreReloadIntervalMs = trustStoreReloadIntervalMs;
+      return this;
+    }
+
+    /**
+     * Builds our {@link SSLFactory}
+     *
+     * @return The built {@link SSLFactory}
+     */
+    public SSLFactory build() {
+      return new SSLFactory(this);
+    }
+  }
+
+  /**
+   * Returns an initialized {@link SSLContext}
+   *
+   * @param requestedProtocol The requested protocol to use
+   * @param keyManagers The list of key managers to use
+   * @param trustManagers The list of trust managers to use
+   * @return The built {@link SSLContext}
+   * @throws GeneralSecurityException
+   */
+  private static SSLContext createSSLContext(
+      String requestedProtocol,
+      KeyManager[] keyManagers,
+      TrustManager[] trustManagers) throws GeneralSecurityException {
+    SSLContext sslContext = SSLContext.getInstance(requestedProtocol);
+    sslContext.init(keyManagers, trustManagers, null);
+    return sslContext;
+  }
+
+  /**
+   * Creates a new {@link SSLEngine}.
+   * Note that currently client auth is not supported
+   *
+   * @param isClient Whether the engine is used in a client context
+   * @param allocator The {@link ByteBufAllocator to use}
+   * @return A valid {@link SSLEngine}.
+   */
+  public SSLEngine createSSLEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine = createEngine(isClient, allocator);
+    engine.setUseClientMode(isClient);
+    engine.setNeedClientAuth(false);
+    engine.setEnabledProtocols(enabledProtocols(engine, requestedProtocol));
+    engine.setEnabledCipherSuites(enabledCipherSuites(engine, requestedCiphers));
+    return engine;
+  }
+
+  private SSLEngine createEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine;
+    if (isClient) {
+      if (nettyClientSslContext != null) {
+        engine = nettyClientSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    } else {
+      if (nettyServerSslContext != null) {
+        engine = nettyServerSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    }
+    return engine;
+  }
+
+  private static TrustManager[] credulousTrustStoreManagers() {
+    return new TrustManager[]{new X509TrustManager() {
+      @Override
+      public void checkClientTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public void checkServerTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public X509Certificate[] getAcceptedIssuers() {
+        return null;
+      }
+    }};
+  }
+
+  private static TrustManager[] trustStoreManagers(
+      File trustStore, String trustStorePassword,
+      boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs)
+          throws IOException, GeneralSecurityException {
+    if (trustStore == null || !trustStore.exists()) {
+      return credulousTrustStoreManagers();
+    } else {
+      if (trustStoreReloadingEnabled) {
+        ReloadingX509TrustManager reloading = new ReloadingX509TrustManager(
+          KeyStore.getDefaultType(), trustStore, trustStorePassword, trustStoreReloadIntervalMs);
+        reloading.init();
+        return new TrustManager[]{reloading};
+      } else {
+        return defaultTrustManagers(trustStore, trustStorePassword);
+      }
+    }
+  }
+
+  private static TrustManager[] defaultTrustManagers(File trustStore, String trustStorePassword)
+      throws IOException, KeyStoreException, CertificateException, NoSuchAlgorithmException {
+    try (InputStream input = Files.asByteSource(trustStore).openStream()) {
+      KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+      ks.load(input, null != trustStorePassword ? trustStorePassword.toCharArray() : null);
+      TrustManagerFactory tmf = TrustManagerFactory.getInstance(
+        TrustManagerFactory.getDefaultAlgorithm());
+      tmf.init(ks);
+      return tmf.getTrustManagers();
+    }
+  }
+
+  private static KeyManager[] keyManagers(File keyStore, String keyStorePassword)
+      throws NoSuchAlgorithmException, CertificateException,
+          KeyStoreException, IOException, UnrecoverableKeyException {
+    KeyManagerFactory factory = KeyManagerFactory.getInstance(
+      KeyManagerFactory.getDefaultAlgorithm());
+    factory.init(
+      loadKeyStore(keyStore, keyStorePassword),
+      keyStorePassword != null ? keyStorePassword.toCharArray() : null
+    );
+
+    return factory.getKeyManagers();
+  }
+
+  private static KeyStore loadKeyStore(File keyStore, String keyStorePassword)
+      throws KeyStoreException, IOException, CertificateException, NoSuchAlgorithmException {
+    if (keyStore == null) {
+      throw new KeyStoreException(
+        "keyStore cannot be null. Please configure spark.ssl.rpc.keyStore");
+    }
+
+    KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+    FileInputStream fin = new FileInputStream(keyStore);
+    try {
+      ks.load(fin, keyStorePassword != null ? keyStorePassword.toCharArray() : null);
+      return ks;
+    } finally {
+      JavaUtils.closeQuietly(fin);
+    }
+  }
+
+  private static String[] enabledProtocols(SSLEngine engine, String requestedProtocol) {
+    String[] supportedProtocols = engine.getSupportedProtocols();
+    String[] defaultProtocols = {"TLSv1.3", "TLSv1.2"};
+    String[] enabledProtocols =
+      ((requestedProtocol == null || requestedProtocol.isEmpty()) ?
+        defaultProtocols : new String[]{requestedProtocol});
+
+    List<String> protocols = addIfSupported(supportedProtocols, enabledProtocols);
+    if (!protocols.isEmpty()) {
+      return protocols.toArray(new String[protocols.size()]);
+    } else {
+      return supportedProtocols;
+    }
+  }
+
+  private static String[] enabledCipherSuites(
+      String[] supportedCiphers, String[] defaultCiphers, String[] requestedCiphers) {
+    String[] baseCiphers = new String[]{
+      // We take ciphers from the mozilla modern list first (for TLS 1.3):
+      // https://wiki.mozilla.org/Security/Server_Side_TLS
+      "TLS_CHACHA20_POLY1305_SHA256",
+      "TLS_AES_128_GCM_SHA256",
+      "TLS_AES_256_GCM_SHA384",
+      // Next we have the TLS1.2 ciphers for intermediate compatibility (since JDK8 does not
+      // support TLS1.3)
+      "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256",
+      "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384",
+      "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256",
+      "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384",
+      "TLS_ECDHE_ECDSA_WITH_CHACHA20_POLY1305_SHA256",
+      "TLS_ECDHE_RSA_WITH_CHACHA20_POLY1305_SHA256",
+      "TLS_DHE_RSA_WITH_AES_128_GCM_SHA256",
+      "TLS_DHE_RSA_WITH_AES_256_GCM_SHA384"
+    };
+    String[] enabledCiphers =
+      ((requestedCiphers == null || requestedCiphers.length == 0) ? baseCiphers : requestedCiphers);
+
+    List<String> ciphers = addIfSupported(supportedCiphers, enabledCiphers);
+    if (!ciphers.isEmpty()) {
+      return ciphers.toArray(new String[ciphers.size()]);
+    } else {
+      // Use the default from JDK as fallback.
+      return defaultCiphers;
+    }
+  }
+
+  private static String[] enabledCipherSuites(SSLEngine engine, String[] requestedCiphers) {
+    return enabledCipherSuites(
+      engine.getSupportedCipherSuites(), engine.getEnabledCipherSuites(), requestedCiphers);
+  }
+
+  private static List<String> addIfSupported(String[] supported, String... names) {

Review Comment:
   @mridulm I addressed part of your feedback here but left `supported` as a `String[]` instead of a Set. Otherwise for example on line 455 we have to convert `engine.getSupportedCipherSuites()` to a set (which is wonky); and in most cases if the user is configuring it they will likely not repeat a ciphersuite I imagine.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45541][CORE] Add SSLFactory [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm commented on code in PR #43386:
URL: https://github.com/apache/spark/pull/43386#discussion_r1366403180


##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,470 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private static final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");
+      }
+    }
+    return SslProvider.JDK;
+  }
+
+  public void destroy() {
+    if (trustManagers != null) {
+      for (int i = 0; i < trustManagers.length; i++) {
+        if (trustManagers[i] instanceof ReloadingX509TrustManager) {
+          try {
+            ((ReloadingX509TrustManager) trustManagers[i]).destroy();
+          } catch (InterruptedException ex) {
+            logger.info("Interrupted while destroying trust manager: " + ex.toString(), ex);
+          }
+        }
+      }
+      trustManagers = null;
+    }
+
+    keyManagers = null;
+    jdkSslContext = null;
+    nettyClientSslContext = null;
+    nettyServerSslContext = null;
+    requestedProtocol = null;
+    requestedCiphers = null;
+  }
+
+  /**
+   * Builder class to construct instances of {@link SSLFactory} with specific options
+   */
+  public static class Builder {
+    private String requestedProtocol;
+    private String[] requestedCiphers;
+    private File keyStore;
+    private String keyStorePassword;
+    private File privateKey;
+    private String keyPassword;
+    private File certChain;
+    private File trustStore;
+    private String trustStorePassword;
+    private boolean trustStoreReloadingEnabled;
+    private int trustStoreReloadIntervalMs;
+    private boolean openSslEnabled;
+
+    /**
+     * Sets the requested protocol, i.e., "TLSv1.2", "TLSv1.1", etc
+     *
+     * @param requestedProtocol The requested protocol
+     * @return The builder object
+     */
+    public Builder requestedProtocol(String requestedProtocol) {
+      this.requestedProtocol = requestedProtocol == null ? "TLSv1.2" : requestedProtocol;
+      return this;
+    }
+
+    /**
+     * Sets the requested cipher suites, i.e., "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", etc
+     *
+     * @param requestedCiphers The requested ciphers
+     * @return The builder object
+     */
+    public Builder requestedCiphers(String[] requestedCiphers) {
+      this.requestedCiphers = requestedCiphers;
+      return this;
+    }
+
+    /**
+     * Sets the Keystore and Keystore password
+     *
+     * @param keyStore The key store file to use
+     * @param keyStorePassword The password for the key store
+     * @return The builder object
+     */
+    public Builder keyStore(File keyStore, String keyStorePassword) {
+      this.keyStore = keyStore;
+      this.keyStorePassword = keyStorePassword;
+      return this;
+    }
+
+    /**
+     * Sets a PKCS#8 private key file in PEM format
+     *
+     * @param privateKey The private key file to use
+     * @return The builder object
+     */
+    public Builder privateKey(File privateKey) {
+      this.privateKey = privateKey;
+      return this;
+    }
+
+    /**
+     * Sets the Key password
+     *
+     * @param keyPassword The password for the private key
+     * @return The builder object
+     */
+    public Builder keyPassword(String keyPassword) {
+      this.keyPassword = keyPassword;
+      return this;
+    }
+
+    /**
+     * Sets a X.509 certificate chain file in PEM format
+     *
+     * @param certChain The certificate chain file to use
+     * @return The builder object
+     */
+    public Builder certChain(File certChain) {
+      this.certChain = certChain;
+      return this;
+    }
+
+    /**
+     * @param enabled Whether to use the OpenSSL implementation
+     * @return The builder object
+     */
+    public Builder openSslEnabled(boolean enabled) {
+      this.openSslEnabled = enabled;
+      return this;
+    }
+
+    /**
+     * Sets the trust-store, trust-store password, whether to use a Reloading TrustStore,
+     * and the trust-store reload interval, if enabled
+     *
+     * @param trustStore The trust store file to use
+     * @param trustStorePassword The password for the trust store
+     * @param trustStoreReloadingEnabled Whether trust store reloading is enabled
+     * @param trustStoreReloadIntervalMs The interval at which to reload the trust store file
+     * @return The builder object
+     */
+    public Builder trustStore(
+        File trustStore, String trustStorePassword,
+        boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs) {
+      this.trustStore = trustStore;
+      this.trustStorePassword = trustStorePassword;
+      this.trustStoreReloadingEnabled = trustStoreReloadingEnabled;
+      this.trustStoreReloadIntervalMs = trustStoreReloadIntervalMs;
+      return this;
+    }
+
+    /**
+     * Builds our {@link SSLFactory}
+     *
+     * @return The built {@link SSLFactory}
+     */
+    public SSLFactory build() {
+      return new SSLFactory(this);
+    }
+  }
+
+  /**
+   * Returns an initialized {@link SSLContext}
+   *
+   * @param requestedProtocol The requested protocol to use
+   * @param keyManagers The list of key managers to use
+   * @param trustManagers The list of trust managers to use
+   * @return The built {@link SSLContext}
+   * @throws GeneralSecurityException
+   */
+  private static SSLContext createSSLContext(
+      String requestedProtocol,
+      KeyManager[] keyManagers,
+      TrustManager[] trustManagers) throws GeneralSecurityException {
+    SSLContext sslContext = SSLContext.getInstance(requestedProtocol);
+    sslContext.init(keyManagers, trustManagers, null);
+    return sslContext;
+  }
+
+  /**
+   * Creates a new {@link SSLEngine}.
+   * Note that currently client auth is not supported
+   *
+   * @param isClient Whether the engine is used in a client context
+   * @param allocator The {@link ByteBufAllocator to use}
+   * @return A valid {@link SSLEngine}.
+   */
+  public SSLEngine createSSLEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine = createEngine(isClient, allocator);
+    engine.setUseClientMode(isClient);
+    engine.setNeedClientAuth(false);
+    engine.setEnabledProtocols(enabledProtocols(engine, requestedProtocol));
+    engine.setEnabledCipherSuites(enabledCipherSuites(engine, requestedCiphers));
+    return engine;
+  }
+
+  private SSLEngine createEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine;
+    if (isClient) {
+      if (nettyClientSslContext != null) {
+        engine = nettyClientSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    } else {
+      if (nettyServerSslContext != null) {
+        engine = nettyServerSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    }
+    return engine;
+  }
+
+  private static TrustManager[] credulousTrustStoreManagers() {
+    return new TrustManager[]{new X509TrustManager() {
+      @Override
+      public void checkClientTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public void checkServerTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public X509Certificate[] getAcceptedIssuers() {
+        return null;
+      }
+    }};
+  }
+
+  private static TrustManager[] trustStoreManagers(
+      File trustStore, String trustStorePassword,
+      boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs)
+          throws IOException, GeneralSecurityException {
+    if (trustStore == null || !trustStore.exists()) {
+      return credulousTrustStoreManagers();
+    } else {
+      if (trustStoreReloadingEnabled) {
+        ReloadingX509TrustManager reloading = new ReloadingX509TrustManager(
+          KeyStore.getDefaultType(), trustStore, trustStorePassword, trustStoreReloadIntervalMs);
+        reloading.init();
+        return new TrustManager[]{reloading};
+      } else {
+        return defaultTrustManagers(trustStore, trustStorePassword);
+      }
+    }
+  }
+
+  private static TrustManager[] defaultTrustManagers(File trustStore, String trustStorePassword)
+      throws IOException, KeyStoreException, CertificateException, NoSuchAlgorithmException {
+    try (InputStream input = Files.asByteSource(trustStore).openStream()) {
+      KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+      ks.load(input, null != trustStorePassword ? trustStorePassword.toCharArray() : null);
+      TrustManagerFactory tmf = TrustManagerFactory.getInstance(
+        TrustManagerFactory.getDefaultAlgorithm());
+      tmf.init(ks);
+      return tmf.getTrustManagers();
+    }
+  }
+
+  private static KeyManager[] keyManagers(File keyStore, String keyStorePassword)
+      throws NoSuchAlgorithmException, CertificateException,
+          KeyStoreException, IOException, UnrecoverableKeyException {
+    KeyManagerFactory factory = KeyManagerFactory.getInstance(
+      KeyManagerFactory.getDefaultAlgorithm());
+    factory.init(
+      loadKeyStore(keyStore, keyStorePassword),
+      keyStorePassword != null ? keyStorePassword.toCharArray() : null
+    );
+
+    return factory.getKeyManagers();
+  }
+
+  private static KeyStore loadKeyStore(File keyStore, String keyStorePassword)
+      throws KeyStoreException, IOException, CertificateException, NoSuchAlgorithmException {
+    if (keyStore == null) {
+      throw new KeyStoreException(
+        "keyStore cannot be null. Please configure spark.ssl.rpc.keyStore");
+    }
+
+    KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+    FileInputStream fin = new FileInputStream(keyStore);
+    try {
+      ks.load(fin, keyStorePassword != null ? keyStorePassword.toCharArray() : null);
+      return ks;
+    } finally {
+      JavaUtils.closeQuietly(fin);
+    }
+  }
+
+  private static String[] enabledProtocols(SSLEngine engine, String requestedProtocol) {
+    String[] supportedProtocols = engine.getSupportedProtocols();
+    String[] defaultProtocols = {"TLSv1.3", "TLSv1.2"};
+    String[] enabledProtocols =
+      ((requestedProtocol == null || requestedProtocol.isEmpty()) ?
+        defaultProtocols : new String[]{requestedProtocol});
+
+    List<String> protocols = addIfSupported(supportedProtocols, enabledProtocols);
+    if (!protocols.isEmpty()) {
+      return protocols.toArray(new String[protocols.size()]);
+    } else {
+      return supportedProtocols;
+    }
+  }
+
+  private static String[] enabledCipherSuites(
+      String[] supportedCiphers, String[] defaultCiphers, String[] requestedCiphers) {
+    String[] baseCiphers = new String[]{
+      // We take ciphers from the mozilla modern list first (for TLS 1.3):
+      // https://wiki.mozilla.org/Security/Server_Side_TLS
+      "TLS_CHACHA20_POLY1305_SHA256",
+      "TLS_AES_128_GCM_SHA256",
+      "TLS_AES_256_GCM_SHA384",
+      // Next we have the TLS1.2 ciphers for intermediate compatibility (since JDK8 does not
+      // support TLS1.3)
+      "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256",
+      "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384",
+      "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256",
+      "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384",
+      "TLS_ECDHE_ECDSA_WITH_CHACHA20_POLY1305_SHA256",
+      "TLS_ECDHE_RSA_WITH_CHACHA20_POLY1305_SHA256",
+      "TLS_DHE_RSA_WITH_AES_128_GCM_SHA256",
+      "TLS_DHE_RSA_WITH_AES_256_GCM_SHA384"
+    };
+    String[] enabledCiphers =
+      ((requestedCiphers == null || requestedCiphers.length == 0) ? baseCiphers : requestedCiphers);
+
+    List<String> ciphers = addIfSupported(supportedCiphers, enabledCiphers);
+    if (!ciphers.isEmpty()) {
+      return ciphers.toArray(new String[ciphers.size()]);
+    } else {
+      // Use the default from JDK as fallback.
+      return defaultCiphers;
+    }
+  }
+
+  private static String[] enabledCipherSuites(SSLEngine engine, String[] requestedCiphers) {
+    return enabledCipherSuites(
+      engine.getSupportedCipherSuites(), engine.getEnabledCipherSuites(), requestedCiphers);
+  }
+
+  private static List<String> addIfSupported(String[] supported, String... names) {

Review Comment:
   You can always do it in this method as well :-)
   Essentially, I am not suggesting to change what user specifies - just that this impl has quadratic complexity, which can be optimized trivially.
   I do expect this not to be a serious issue (hence a nit) given the cardinality of the array



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45541][CORE] Add SSLFactory [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm commented on code in PR #43386:
URL: https://github.com/apache/spark/pull/43386#discussion_r1366406949


##########
common/network-common/src/main/java/org/apache/spark/network/ssl/SSLFactory.java:
##########
@@ -0,0 +1,470 @@
+/*
+ * 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.spark.network.ssl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+
+import com.google.common.io.Files;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.handler.ssl.OpenSsl;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.handler.ssl.SslProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.util.JavaUtils;
+
+public class SSLFactory {
+  private static final Logger logger = LoggerFactory.getLogger(SSLFactory.class);
+
+  /**
+   * For a configuration specifying keystore/truststore files
+   */
+  private SSLContext jdkSslContext;
+
+  /**
+   * For a configuration specifying a PEM cert chain, and a PEM private key
+   */
+  private SslContext nettyClientSslContext;
+  private SslContext nettyServerSslContext;
+
+  private KeyManager[] keyManagers;
+  private TrustManager[] trustManagers;
+  private String requestedProtocol;
+  private String[] requestedCiphers;
+
+  private SSLFactory(final Builder b) {
+    this.requestedProtocol = b.requestedProtocol;
+    this.requestedCiphers = b.requestedCiphers;
+    try {
+      if (b.certChain != null && b.privateKey != null) {
+        initNettySslContexts(b);
+      } else {
+        initJdkSslContext(b);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("SSLFactory creation failed", e);
+    }
+  }
+
+  private void initJdkSslContext(final Builder b)
+          throws IOException, GeneralSecurityException {
+    this.keyManagers = keyManagers(b.keyStore, b.keyStorePassword);
+    this.trustManagers = trustStoreManagers(
+      b.trustStore, b.trustStorePassword,
+      b.trustStoreReloadingEnabled, b.trustStoreReloadIntervalMs
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException {
+    nettyClientSslContext = SslContextBuilder
+      .forClient()
+      .sslProvider(getSslProvider(b))
+      .trustManager(b.certChain)
+      .build();
+
+    nettyServerSslContext = SslContextBuilder
+      .forServer(b.certChain, b.privateKey, b.keyPassword)
+      .sslProvider(getSslProvider(b))
+      .build();
+  }
+
+  /**
+   * If OpenSSL is requested, this will check if an implementation is available on the local host.
+   * If an implementation is not available it will fall back to the JDK {@link SslProvider}.
+   *
+   * @param b
+   * @return
+   */
+  private SslProvider getSslProvider(Builder b) {
+    if (b.openSslEnabled) {
+      if (OpenSsl.isAvailable()) {
+        return SslProvider.OPENSSL;
+      } else {
+        logger.warn("OpenSSL Provider requested but it is not available, using JDK SSL Provider");
+      }
+    }
+    return SslProvider.JDK;
+  }
+
+  public void destroy() {
+    if (trustManagers != null) {
+      for (int i = 0; i < trustManagers.length; i++) {
+        if (trustManagers[i] instanceof ReloadingX509TrustManager) {
+          try {
+            ((ReloadingX509TrustManager) trustManagers[i]).destroy();
+          } catch (InterruptedException ex) {
+            logger.info("Interrupted while destroying trust manager: " + ex.toString(), ex);
+          }
+        }
+      }
+      trustManagers = null;
+    }
+
+    keyManagers = null;
+    jdkSslContext = null;
+    nettyClientSslContext = null;
+    nettyServerSslContext = null;
+    requestedProtocol = null;
+    requestedCiphers = null;
+  }
+
+  /**
+   * Builder class to construct instances of {@link SSLFactory} with specific options
+   */
+  public static class Builder {
+    private String requestedProtocol;
+    private String[] requestedCiphers;
+    private File keyStore;
+    private String keyStorePassword;
+    private File privateKey;
+    private String keyPassword;
+    private File certChain;
+    private File trustStore;
+    private String trustStorePassword;
+    private boolean trustStoreReloadingEnabled;
+    private int trustStoreReloadIntervalMs;
+    private boolean openSslEnabled;
+
+    /**
+     * Sets the requested protocol, i.e., "TLSv1.2", "TLSv1.1", etc
+     *
+     * @param requestedProtocol The requested protocol
+     * @return The builder object
+     */
+    public Builder requestedProtocol(String requestedProtocol) {
+      this.requestedProtocol = requestedProtocol == null ? "TLSv1.2" : requestedProtocol;
+      return this;
+    }
+
+    /**
+     * Sets the requested cipher suites, i.e., "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", etc
+     *
+     * @param requestedCiphers The requested ciphers
+     * @return The builder object
+     */
+    public Builder requestedCiphers(String[] requestedCiphers) {
+      this.requestedCiphers = requestedCiphers;
+      return this;
+    }
+
+    /**
+     * Sets the Keystore and Keystore password
+     *
+     * @param keyStore The key store file to use
+     * @param keyStorePassword The password for the key store
+     * @return The builder object
+     */
+    public Builder keyStore(File keyStore, String keyStorePassword) {
+      this.keyStore = keyStore;
+      this.keyStorePassword = keyStorePassword;
+      return this;
+    }
+
+    /**
+     * Sets a PKCS#8 private key file in PEM format
+     *
+     * @param privateKey The private key file to use
+     * @return The builder object
+     */
+    public Builder privateKey(File privateKey) {
+      this.privateKey = privateKey;
+      return this;
+    }
+
+    /**
+     * Sets the Key password
+     *
+     * @param keyPassword The password for the private key
+     * @return The builder object
+     */
+    public Builder keyPassword(String keyPassword) {
+      this.keyPassword = keyPassword;
+      return this;
+    }
+
+    /**
+     * Sets a X.509 certificate chain file in PEM format
+     *
+     * @param certChain The certificate chain file to use
+     * @return The builder object
+     */
+    public Builder certChain(File certChain) {
+      this.certChain = certChain;
+      return this;
+    }
+
+    /**
+     * @param enabled Whether to use the OpenSSL implementation
+     * @return The builder object
+     */
+    public Builder openSslEnabled(boolean enabled) {
+      this.openSslEnabled = enabled;
+      return this;
+    }
+
+    /**
+     * Sets the trust-store, trust-store password, whether to use a Reloading TrustStore,
+     * and the trust-store reload interval, if enabled
+     *
+     * @param trustStore The trust store file to use
+     * @param trustStorePassword The password for the trust store
+     * @param trustStoreReloadingEnabled Whether trust store reloading is enabled
+     * @param trustStoreReloadIntervalMs The interval at which to reload the trust store file
+     * @return The builder object
+     */
+    public Builder trustStore(
+        File trustStore, String trustStorePassword,
+        boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs) {
+      this.trustStore = trustStore;
+      this.trustStorePassword = trustStorePassword;
+      this.trustStoreReloadingEnabled = trustStoreReloadingEnabled;
+      this.trustStoreReloadIntervalMs = trustStoreReloadIntervalMs;
+      return this;
+    }
+
+    /**
+     * Builds our {@link SSLFactory}
+     *
+     * @return The built {@link SSLFactory}
+     */
+    public SSLFactory build() {
+      return new SSLFactory(this);
+    }
+  }
+
+  /**
+   * Returns an initialized {@link SSLContext}
+   *
+   * @param requestedProtocol The requested protocol to use
+   * @param keyManagers The list of key managers to use
+   * @param trustManagers The list of trust managers to use
+   * @return The built {@link SSLContext}
+   * @throws GeneralSecurityException
+   */
+  private static SSLContext createSSLContext(
+      String requestedProtocol,
+      KeyManager[] keyManagers,
+      TrustManager[] trustManagers) throws GeneralSecurityException {
+    SSLContext sslContext = SSLContext.getInstance(requestedProtocol);
+    sslContext.init(keyManagers, trustManagers, null);
+    return sslContext;
+  }
+
+  /**
+   * Creates a new {@link SSLEngine}.
+   * Note that currently client auth is not supported
+   *
+   * @param isClient Whether the engine is used in a client context
+   * @param allocator The {@link ByteBufAllocator to use}
+   * @return A valid {@link SSLEngine}.
+   */
+  public SSLEngine createSSLEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine = createEngine(isClient, allocator);
+    engine.setUseClientMode(isClient);
+    engine.setNeedClientAuth(false);
+    engine.setEnabledProtocols(enabledProtocols(engine, requestedProtocol));
+    engine.setEnabledCipherSuites(enabledCipherSuites(engine, requestedCiphers));
+    return engine;
+  }
+
+  private SSLEngine createEngine(boolean isClient, ByteBufAllocator allocator) {
+    SSLEngine engine;
+    if (isClient) {
+      if (nettyClientSslContext != null) {
+        engine = nettyClientSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    } else {
+      if (nettyServerSslContext != null) {
+        engine = nettyServerSslContext.newEngine(allocator);
+      } else {
+        engine = jdkSslContext.createSSLEngine();
+      }
+    }
+    return engine;
+  }
+
+  private static TrustManager[] credulousTrustStoreManagers() {
+    return new TrustManager[]{new X509TrustManager() {
+      @Override
+      public void checkClientTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public void checkServerTrusted(X509Certificate[] x509Certificates, String s)
+        throws CertificateException {
+      }
+
+      @Override
+      public X509Certificate[] getAcceptedIssuers() {
+        return null;
+      }
+    }};
+  }
+
+  private static TrustManager[] trustStoreManagers(
+      File trustStore, String trustStorePassword,
+      boolean trustStoreReloadingEnabled, int trustStoreReloadIntervalMs)
+          throws IOException, GeneralSecurityException {
+    if (trustStore == null || !trustStore.exists()) {
+      return credulousTrustStoreManagers();
+    } else {
+      if (trustStoreReloadingEnabled) {
+        ReloadingX509TrustManager reloading = new ReloadingX509TrustManager(
+          KeyStore.getDefaultType(), trustStore, trustStorePassword, trustStoreReloadIntervalMs);
+        reloading.init();
+        return new TrustManager[]{reloading};
+      } else {
+        return defaultTrustManagers(trustStore, trustStorePassword);
+      }
+    }
+  }
+
+  private static TrustManager[] defaultTrustManagers(File trustStore, String trustStorePassword)
+      throws IOException, KeyStoreException, CertificateException, NoSuchAlgorithmException {
+    try (InputStream input = Files.asByteSource(trustStore).openStream()) {
+      KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+      ks.load(input, null != trustStorePassword ? trustStorePassword.toCharArray() : null);
+      TrustManagerFactory tmf = TrustManagerFactory.getInstance(
+        TrustManagerFactory.getDefaultAlgorithm());
+      tmf.init(ks);
+      return tmf.getTrustManagers();
+    }
+  }
+
+  private static KeyManager[] keyManagers(File keyStore, String keyStorePassword)
+      throws NoSuchAlgorithmException, CertificateException,
+          KeyStoreException, IOException, UnrecoverableKeyException {
+    KeyManagerFactory factory = KeyManagerFactory.getInstance(
+      KeyManagerFactory.getDefaultAlgorithm());
+    factory.init(
+      loadKeyStore(keyStore, keyStorePassword),
+      keyStorePassword != null ? keyStorePassword.toCharArray() : null
+    );

Review Comment:
   nit:
   
   ```suggestion
       char[] passwordCharacters = keyStorePassword != null ? keyStorePassword.toCharArray() : null;
       factory.init(
         loadKeyStore(keyStore, passwordCharacters),
         passwordCharacters
       );
   ```
   (with suitable change to `loadKeyStore` as well)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org