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/08/25 16:57:23 UTC

[GitHub] [spark] hasnain-db opened a new pull request, #42685: [WIP][SPARK-44937][CORE] Add SSL/TLS support for RPC and Shuffle communications

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

   ### What changes were proposed in this pull request?
   
   This PR adds support for SSL/TLS based communication for Spark RPCs and block transfers - providing an alternative to the existing encryption / authentication implementation documented at https://spark.apache.org/docs/latest/security.html#spark-rpc-communication-protocol-between-spark-processes
   
   This is based on an existing PR from 2015: https://github.com/apache/spark/pull/9416 - with some refactoring and a number of updates to make it work with changes to Spark since.
   
   I understand this is a large PR, so I've broken this down by a high level summary of changes and a suggested review order:
   
   * Add a dependency on `netty-tcnative-boringssl-static` which provides support for faster TLS communication
   * Add documentation for the new functionality in `docs/security.md` which describe the new flags and configuration.
   * Extend `SSLOptions` and `TransportConf` to support the new flags for this feature 
   * Extend `TransportContext` to optionally add an SSL based handler if configured, and make similar changes in the transport client and server factories
   * Add a new API to `ManagedBuffer` to convert objects to a Netty object for SSL encoding (since we can't use zero-copy transfers, we can't use `convertToNetty()` directly)
   * Add some helper classes for communication:
     * `EncryptedMessageWithHeader` and `SSLMessageEncoder` are quite similar to the existing variants but just different enough that it was hard to consolidate
     * `SSLFactory` to create the JDK / Netty SSL handlers as appropriate. This handles configuration of the protocols, ciphers, etc
     * `ReloadingX509TrustManager` to support trust store reloading. 
   * Change `SecurityManager` to disable the existing authentication/encryption mechanisms if this new feature is enabled
   * Update `SparkConf` and `CommandUtils` to pass passwords via env variables if needed, to preserve security guarantees (similar to the existing SSL password propagation)
   * Update almost all constructor callsites of `SparkTransportConf` to propagate SSL options
   * Add tests:
     * Add test keys + certificates in a bunch of places
     * Add an `SSLSampleConfigs` class for sample configurations used in tests
     * Add tests for a bunch of the new classes + features
     * Add new tests for almost all modules that create a `TransportContext` which rerun the same tests but with this new feature enabled (this caught a bunch of bugs). This will ensure features are compatible with SSL going forward
   
   
   ### Why are the changes needed?
   
   Spark currently does not support TLS/SSL for RPC and block transfers. It is helpful to have this as an alternative encryption method for users which may need to use more standard encryption mechanisms instead of one that is more internal to spark.
   
   
   ### Does this PR introduce _any_ user-facing change?
   
   Yes, this includes new configuration options and updates associated documentation. Besides that, though, all other aspects of Spark should remain unchanged (modulo performance differences).
   
   
   ### How was this patch tested?
   
   Added a bunch of unit tests that pass. 
   
   I also ran some queries locally to ensure they still work.
   
   I verified traffic was encrypted using TLS using two mechanisms:
   
   * Enabled trace level logging for Netty and JDK SSL and saw logs confirming TLS handshakes were happening
   * I ran wireshark on my machine and snooped on traffic while sending queries shuffling a fixed string. Without any encryption, I could find that string in the network traffic. With this encryption enabled, that string did not show up, and wireshark logs confirmed a TLS handshake was happening.
   
   ### 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] [WIP][SPARK-44937][CORE] Add SSL/TLS support for RPC and Shuffle communications [spark]

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

   The overall feature has been merged.


-- 
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


[GitHub] [spark] mridulm commented on pull request #42685: [WIP][SPARK-44937][CORE] Add SSL/TLS support for RPC and Shuffle communications

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

   +CC @otterc 


-- 
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


[GitHub] [spark] JoshRosen commented on a diff in pull request #42685: [WIP][SPARK-44937][CORE] Add SSL/TLS support for RPC and Shuffle communications

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


##########
common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java:
##########
@@ -325,7 +325,10 @@ public TransportResponseHandler getHandler() {
 
   @Override
   public void close() {
-    // close is a local operation and should finish with milliseconds; timeout just to be safe
+    // Mark the connection as timed out, so we do not return a connection that's being closed
+    // from the TransportClientFactory if closing takes some time (e.g. with SSL)
+    this.timedOut = true;

Review Comment:
   This change is necessary for SSL because SSL connections can take a long time to close, but I think it's also okay to have it apply for regular unencrypted connections (as you've done here): if we have a client that is in the process of closing then marking it as timed out during that close helps to avoid a race condition in which a pooled client might be handed to a requester only to immediately close. 👍 



##########
common/network-common/src/main/java/org/apache/spark/network/ssl/ReloadingX509TrustManager.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * A {@link TrustManager} implementation that reloads its configuration when
+ * the truststore file on disk changes.
+ * This implementation is based entirely on the
+ * org.apache.hadoop.security.ssl.ReloadingX509TrustManager class in the Apache Hadoop Encrypted
+ * Shuffle implementation.
+ *
+ * @see <a href="https://hadoop.apache.org/docs/current/hadoop-mapreduce-client/hadoop-mapreduce-client-core/EncryptedShuffle.html">Hadoop MapReduce Next Generation - Encrypted Shuffle</a>
+ */
+public final class ReloadingX509TrustManager

Review Comment:
   Bundling this as source avoids the need to add `hadoop-common` dependencies to `common/network-common`. Currently it looks like nothing under `common/` depends on Hadoop stuff. Based on https://github.com/c9n/hadoop/blob/master/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/ReloadingX509TrustManager.java it looks like this code hasn't changed in over a decade, so this is unlikely to pose maintainability risks.
   
   My read of https://infra.apache.org/licensing-howto.html is that this is permissible via 
   
   - https://infra.apache.org/licensing-howto.html#alv2-dep
   - https://infra.apache.org/licensing-howto.html#bundle-asf-product
   
   and I don't see any `NOTICE` portions in `hadoop-common` that specifically pertain to this file, so I believe we're clear from a license perspective 👍 



##########
common/network-common/src/main/java/org/apache/spark/network/ssl/ReloadingX509TrustManager.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * A {@link TrustManager} implementation that reloads its configuration when
+ * the truststore file on disk changes.
+ * This implementation is based entirely on the
+ * org.apache.hadoop.security.ssl.ReloadingX509TrustManager class in the Apache Hadoop Encrypted
+ * Shuffle implementation.
+ *
+ * @see <a href="https://hadoop.apache.org/docs/current/hadoop-mapreduce-client/hadoop-mapreduce-client-core/EncryptedShuffle.html">Hadoop MapReduce Next Generation - Encrypted Shuffle</a>
+ */
+public final class ReloadingX509TrustManager

Review Comment:
   Bundling this as source avoids the need to add `hadoop-common` dependencies to `common/network-common`. Currently it looks like nothing under `common/` depends on Hadoop stuff. Based on https://github.com/c9n/hadoop/blob/master/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/ReloadingX509TrustManager.java it looks like this code hasn't changed in over a decade, so this is unlikely to pose maintainability risks.
   
   My read of https://infra.apache.org/licensing-howto.html is that this is permissible via 
   
   - https://infra.apache.org/licensing-howto.html#alv2-dep
   - https://infra.apache.org/licensing-howto.html#bundle-asf-product
   
   and I don't see any `NOTICE` portions in `hadoop-common` that specifically pertain to this file, so I believe we're clear from a license perspective 👍 



##########
common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java:
##########
@@ -325,7 +325,10 @@ public TransportResponseHandler getHandler() {
 
   @Override
   public void close() {
-    // close is a local operation and should finish with milliseconds; timeout just to be safe
+    // Mark the connection as timed out, so we do not return a connection that's being closed
+    // from the TransportClientFactory if closing takes some time (e.g. with SSL)
+    this.timedOut = true;

Review Comment:
   This change is necessary for SSL because SSL connections can take a long time to close, but I think it's also okay to have it apply for regular unencrypted connections (as you've done here): if we have a client that is in the process of closing then marking it as timed out during that close helps to avoid a race condition in which a pooled client might be handed to a requester only to immediately close. 👍 



-- 
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] [WIP][SPARK-44937][CORE] Add SSL/TLS support for RPC and Shuffle communications [spark]

Posted by "mridulm (via GitHub)" <gi...@apache.org>.
mridulm closed pull request #42685: [WIP][SPARK-44937][CORE] Add SSL/TLS support for RPC and Shuffle communications
URL: https://github.com/apache/spark/pull/42685


-- 
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


[GitHub] [spark] turp1twin commented on pull request #42685: [WIP][SPARK-44937][CORE] Add SSL/TLS support for RPC and Shuffle communications

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

   @hasnain-db Wow, this a blast from the past.. Happy to review... Cheers!


-- 
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


[GitHub] [spark] mridulm commented on pull request #42685: [WIP][SPARK-44937][CORE] Add SSL/TLS support for RPC and Shuffle communications

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

   Thanks for working on this @hasnain-db , this is a very nice adding to spark !
   Given the size of the PR, can we split this up to make it easier to review ?


-- 
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


[GitHub] [spark] hasnain-db commented on a diff in pull request #42685: [WIP][SPARK-44937][CORE] Add SSL/TLS support for RPC and Shuffle communications

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


##########
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 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 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 java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyManagementException;
+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 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.trustStoreReloadInterval
+    );
+    this.jdkSslContext = createSSLContext(requestedProtocol, keyManagers, trustManagers);
+  }
+
+  private void initNettySslContexts(final Builder b)
+          throws SSLException, NoSuchAlgorithmException {
+    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) {
+          ((ReloadingX509TrustManager) trustManagers[i]).destroy();
+        }
+      }
+      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 trustStoreReloadInterval;
+    private boolean openSslEnabled;
+
+    /**
+     * Sets the requested protocol, i.e., "TLSv1.2", "TLSv1.1", etc
+     *
+     * @param requestedProtocol
+     * @return
+     */
+    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
+     * @return
+     */
+    public Builder requestedCiphers(String[] requestedCiphers) {
+      this.requestedCiphers = requestedCiphers;
+      return this;
+    }
+
+    /**
+     * Sets the Keystore and Keystore password
+     *
+     * @param keyStore
+     * @param keyStorePassword
+     * @return
+     */
+    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
+     * @return
+     */
+    public Builder privateKey(File privateKey) {
+      this.privateKey = privateKey;
+      return this;
+    }
+
+    /**
+     * Sets the Key password
+     *
+     * @param keyPassword
+     * @return
+     */
+    public Builder keyPassword(String keyPassword) {
+      this.keyPassword = keyPassword;
+      return this;
+    }
+
+    /**
+     * Sets a X.509 certificate chain file in PEM format
+     *
+     * @param certChain
+     * @return
+     */
+    public Builder certChain(File certChain) {
+      this.certChain = certChain;
+      return this;
+    }
+
+    /**
+     * @param enabled
+     * @return
+     */
+    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
+     * @param trustStorePassword
+     * @param trustStoreReloadingEnabled
+     * @param trustStoreReloadInterval
+     * @return
+     */
+    public Builder trustStore(
+        File trustStore, String trustStorePassword,
+        boolean trustStoreReloadingEnabled, int trustStoreReloadInterval) {
+      this.trustStore = trustStore;
+      this.trustStorePassword = trustStorePassword;
+      this.trustStoreReloadingEnabled = trustStoreReloadingEnabled;
+      this.trustStoreReloadInterval = trustStoreReloadInterval;
+      return this;
+    }
+
+    /**
+     * Builds our {@link SSLFactory}
+     *
+     * @return
+     */
+    public SSLFactory build() {
+      return new SSLFactory(this);
+    }
+  }
+
+  /**
+   * Returns an initialized {@link SSLContext}
+   *
+   * @param requestedProtocol
+   * @param keyManagers
+   * @param trustManagers
+   * @return
+   * @throws IOException
+   * @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 {@linnk SSLContext}
+   * @param requestedProtocol
+   * @return
+   * @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
+   *
+   * @return
+   * @throws NoSuchAlgorithmException
+   * @throws UnrecoverableKeyException
+   * @throws CertificateException
+   * @throws KeyStoreException
+   * @throws IOException
+   * @throws KeyManagementException
+   */
+  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 trustStoreReloadInterval)
+          throws IOException, GeneralSecurityException {
+    if (trustStore == null || !trustStore.exists()) {
+      return credulousTrustStoreManagers();
+    } else {
+      if (trustStorePassword == null) {
+        throw new KeyStoreException("trustStorePassword cannot be null");
+      }
+
+      if (trustStoreReloadingEnabled) {
+        ReloadingX509TrustManager reloading = new ReloadingX509TrustManager(
+          KeyStore.getDefaultType(), trustStore, trustStorePassword, trustStoreReloadInterval);
+        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");
+    }
+
+    if (keyStorePassword == null) {
+      throw new KeyStoreException("keyStorePassword cannot be null");
+    }
+
+    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.2)

Review Comment:
   err, this should be TLS1.3, will fix.



-- 
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


[GitHub] [spark] hasnain-db commented on pull request #42685: [WIP][SPARK-44937][CORE] Add SSL/TLS support for RPC and Shuffle communications

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

   cc: @JoshRosen, @mridulm, @vanzin and @turp1twin as potential reviewers to look at this PR based on blame. Happy to discuss/share more context, I understand this is a big PR. I'm looking to gather high level feedback so I can address any concerns and work towards getting this merged. Thank you in advance!


-- 
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


[GitHub] [spark] mridulm commented on pull request #42685: [WIP][SPARK-44937][CORE] Add SSL/TLS support for RPC and Shuffle communications

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

   > Just to confirm (since I'm not sure this repo has support for stacked PRs - if there is, please link me to an example) - you're proposing I put up one PR, get it approved and merged, then put up the second PR, and so on, right (since most changes depend on each other).
   
   If there are independent PR's, those can be done in parallel.
   If there is a interdependency, they would end up being serial, agree


-- 
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


[GitHub] [spark] hasnain-db commented on pull request #42685: [WIP][SPARK-44937][CORE] Add SSL/TLS support for RPC and Shuffle communications

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

   I wonder if the failing test is flaky, it consistently passes for me locally


-- 
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


[GitHub] [spark] hasnain-db commented on pull request #42685: [WIP][SPARK-44937][CORE] Add SSL/TLS support for RPC and Shuffle communications

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

   Thanks @mridulm ! Happy to do that. I can think of a nice split in line with the bullet points listed in the summary here.
   
   Just to confirm (since I'm not sure this repo has support for stacked PRs - if there is, please link me to an example) - you're proposing I put up one PR, get it approved and merged, then put up the second PR, and so on, right (since most changes depend on each other).


-- 
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