Repository: spark
Updated Branches:
  refs/heads/branch-1.4 ffc793a6c -> eab1d16a7


[SPARK-7756] [CORE] More robust SSL options processing.

Subset the enabled algorithms in an SSLOptions to the elements that are 
supported by the protocol provider.

Update the list of ciphers in the sample config to include modern algorithms, 
and specify both Oracle and IBM names.  In practice the user would either 
specify their own chosen cipher suites, or specify none, and delegate the 
decision to the provider.

Author: Tim Ellison <[email protected]>

Closes #7043 from tellison/SSLEnhancements and squashes the following commits:

034efa5 [Tim Ellison] Ensure Java imports are grouped and ordered by package.
3797f8b [Tim Ellison] Remove unnecessary use of Option to improve clarity, and 
fix import style ordering.
4b5c89f [Tim Ellison] More robust SSL options processing.

(cherry picked from commit 2ed0c0ac4686ea779f98713978e37b97094edc1c)
Signed-off-by: Sean Owen <[email protected]>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/eab1d16a
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/eab1d16a
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/eab1d16a

Branch: refs/heads/branch-1.4
Commit: eab1d16a7abebbf901fcfe7e997ac015ed4e4cf7
Parents: ffc793a
Author: Tim Ellison <[email protected]>
Authored: Tue Jun 30 13:49:52 2015 +0100
Committer: Sean Owen <[email protected]>
Committed: Tue Jun 30 13:50:07 2015 +0100

----------------------------------------------------------------------
 .../scala/org/apache/spark/SSLOptions.scala     | 43 +++++++++++++++++---
 .../org/apache/spark/SSLOptionsSuite.scala      | 20 ++++++---
 .../org/apache/spark/SSLSampleConfigs.scala     | 24 ++++++++---
 .../org/apache/spark/SecurityManagerSuite.scala | 21 +++++++---
 4 files changed, 85 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/eab1d16a/core/src/main/scala/org/apache/spark/SSLOptions.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SSLOptions.scala 
b/core/src/main/scala/org/apache/spark/SSLOptions.scala
index 2cdc167..32df42d 100644
--- a/core/src/main/scala/org/apache/spark/SSLOptions.scala
+++ b/core/src/main/scala/org/apache/spark/SSLOptions.scala
@@ -17,7 +17,9 @@
 
 package org.apache.spark
 
-import java.io.File
+import java.io.{File, FileInputStream}
+import java.security.{KeyStore, NoSuchAlgorithmException}
+import javax.net.ssl.{KeyManager, KeyManagerFactory, SSLContext, TrustManager, 
TrustManagerFactory}
 
 import com.typesafe.config.{Config, ConfigFactory, ConfigValueFactory}
 import org.eclipse.jetty.util.ssl.SslContextFactory
@@ -38,7 +40,7 @@ import org.eclipse.jetty.util.ssl.SslContextFactory
  * @param trustStore          a path to the trust-store file
  * @param trustStorePassword  a password to access the trust-store file
  * @param protocol            SSL protocol (remember that SSLv3 was 
compromised) supported by Java
- * @param enabledAlgorithms   a set of encryption algorithms to use
+ * @param enabledAlgorithms   a set of encryption algorithms that may be used
  */
 private[spark] case class SSLOptions(
     enabled: Boolean = false,
@@ -48,7 +50,8 @@ private[spark] case class SSLOptions(
     trustStore: Option[File] = None,
     trustStorePassword: Option[String] = None,
     protocol: Option[String] = None,
-    enabledAlgorithms: Set[String] = Set.empty) {
+    enabledAlgorithms: Set[String] = Set.empty)
+    extends Logging {
 
   /**
    * Creates a Jetty SSL context factory according to the SSL settings 
represented by this object.
@@ -63,7 +66,7 @@ private[spark] case class SSLOptions(
       trustStorePassword.foreach(sslContextFactory.setTrustStorePassword)
       keyPassword.foreach(sslContextFactory.setKeyManagerPassword)
       protocol.foreach(sslContextFactory.setProtocol)
-      sslContextFactory.setIncludeCipherSuites(enabledAlgorithms.toSeq: _*)
+      sslContextFactory.setIncludeCipherSuites(supportedAlgorithms.toSeq: _*)
 
       Some(sslContextFactory)
     } else {
@@ -94,7 +97,7 @@ private[spark] case class SSLOptions(
         .withValue("akka.remote.netty.tcp.security.protocol",
           ConfigValueFactory.fromAnyRef(protocol.getOrElse("")))
         .withValue("akka.remote.netty.tcp.security.enabled-algorithms",
-          ConfigValueFactory.fromIterable(enabledAlgorithms.toSeq))
+          ConfigValueFactory.fromIterable(supportedAlgorithms.toSeq))
         .withValue("akka.remote.netty.tcp.enable-ssl",
           ConfigValueFactory.fromAnyRef(true)))
     } else {
@@ -102,6 +105,36 @@ private[spark] case class SSLOptions(
     }
   }
 
+  /*
+   * The supportedAlgorithms set is a subset of the enabledAlgorithms that
+   * are supported by the current Java security provider for this protocol.
+   */
+  private val supportedAlgorithms: Set[String] = {
+    var context: SSLContext = null
+    try {
+      context = SSLContext.getInstance(protocol.orNull)
+      /* The set of supported algorithms does not depend upon the keys, trust, 
or
+         rng, although they will influence which algorithms are eventually 
used. */
+      context.init(null, null, null)
+    } catch {
+      case npe: NullPointerException =>
+        logDebug("No SSL protocol specified")
+        context = SSLContext.getDefault
+      case nsa: NoSuchAlgorithmException =>
+        logDebug(s"No support for requested SSL protocol ${protocol.get}")
+        context = SSLContext.getDefault
+    }
+
+    val providerAlgorithms = 
context.getServerSocketFactory.getSupportedCipherSuites.toSet
+
+    // Log which algorithms we are discarding
+    (enabledAlgorithms &~ providerAlgorithms).foreach { cipher =>
+      logDebug(s"Discarding unsupported cipher $cipher")
+    }
+
+    enabledAlgorithms & providerAlgorithms
+  }
+
   /** Returns a string representation of this SSLOptions with all the 
passwords masked. */
   override def toString: String = s"SSLOptions{enabled=$enabled, " +
       s"keyStore=$keyStore, keyStorePassword=${keyStorePassword.map(_ => 
"xxx")}, " +

http://git-wip-us.apache.org/repos/asf/spark/blob/eab1d16a/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala 
b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala
index 376481b..25b79bc 100644
--- a/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala
@@ -18,6 +18,7 @@
 package org.apache.spark
 
 import java.io.File
+import javax.net.ssl.SSLContext
 
 import com.google.common.io.Files
 import org.apache.spark.util.Utils
@@ -29,6 +30,15 @@ class SSLOptionsSuite extends SparkFunSuite with 
BeforeAndAfterAll {
     val keyStorePath = new 
File(this.getClass.getResource("/keystore").toURI).getAbsolutePath
     val trustStorePath = new 
File(this.getClass.getResource("/truststore").toURI).getAbsolutePath
 
+    // Pick two cipher suites that the provider knows about
+    val sslContext = SSLContext.getInstance("TLSv1.2")
+    sslContext.init(null, null, null)
+    val algorithms = sslContext
+      .getServerSocketFactory
+      .getDefaultCipherSuites
+      .take(2)
+      .toSet
+
     val conf = new SparkConf
     conf.set("spark.ssl.enabled", "true")
     conf.set("spark.ssl.keyStore", keyStorePath)
@@ -36,9 +46,8 @@ class SSLOptionsSuite extends SparkFunSuite with 
BeforeAndAfterAll {
     conf.set("spark.ssl.keyPassword", "password")
     conf.set("spark.ssl.trustStore", trustStorePath)
     conf.set("spark.ssl.trustStorePassword", "password")
-    conf.set("spark.ssl.enabledAlgorithms",
-      "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA")
-    conf.set("spark.ssl.protocol", "SSLv3")
+    conf.set("spark.ssl.enabledAlgorithms", algorithms.mkString(","))
+    conf.set("spark.ssl.protocol", "TLSv1.2")
 
     val opts = SSLOptions.parse(conf, "spark.ssl")
 
@@ -52,9 +61,8 @@ class SSLOptionsSuite extends SparkFunSuite with 
BeforeAndAfterAll {
     assert(opts.trustStorePassword === Some("password"))
     assert(opts.keyStorePassword === Some("password"))
     assert(opts.keyPassword === Some("password"))
-    assert(opts.protocol === Some("SSLv3"))
-    assert(opts.enabledAlgorithms ===
-      Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA"))
+    assert(opts.protocol === Some("TLSv1.2"))
+    assert(opts.enabledAlgorithms === algorithms)
   }
 
   test("test resolving property with defaults specified ") {

http://git-wip-us.apache.org/repos/asf/spark/blob/eab1d16a/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala 
b/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala
index 1a099da..33270be 100644
--- a/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala
+++ b/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala
@@ -25,6 +25,20 @@ object SSLSampleConfigs {
     this.getClass.getResource("/untrusted-keystore").toURI).getAbsolutePath
   val trustStorePath = new 
File(this.getClass.getResource("/truststore").toURI).getAbsolutePath
 
+  val enabledAlgorithms =
+    // A reasonable set of TLSv1.2 Oracle security provider suites
+    "TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA384, " +
+    "TLS_RSA_WITH_AES_256_CBC_SHA256, " +
+    "TLS_DHE_RSA_WITH_AES_256_CBC_SHA256, " +
+    "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256, " +
+    "TLS_DHE_RSA_WITH_AES_128_CBC_SHA256, " +
+    // and their equivalent names in the IBM Security provider
+    "SSL_ECDHE_RSA_WITH_AES_256_CBC_SHA384, " +
+    "SSL_RSA_WITH_AES_256_CBC_SHA256, " +
+    "SSL_DHE_RSA_WITH_AES_256_CBC_SHA256, " +
+    "SSL_ECDHE_RSA_WITH_AES_128_CBC_SHA256, " +
+    "SSL_DHE_RSA_WITH_AES_128_CBC_SHA256"
+
   def sparkSSLConfig(): SparkConf = {
     val conf = new SparkConf(loadDefaults = false)
     conf.set("spark.ssl.enabled", "true")
@@ -33,9 +47,8 @@ object SSLSampleConfigs {
     conf.set("spark.ssl.keyPassword", "password")
     conf.set("spark.ssl.trustStore", trustStorePath)
     conf.set("spark.ssl.trustStorePassword", "password")
-    conf.set("spark.ssl.enabledAlgorithms",
-      "SSL_RSA_WITH_RC4_128_SHA, SSL_RSA_WITH_DES_CBC_SHA")
-    conf.set("spark.ssl.protocol", "TLSv1")
+    conf.set("spark.ssl.enabledAlgorithms", enabledAlgorithms)
+    conf.set("spark.ssl.protocol", "TLSv1.2")
     conf
   }
 
@@ -47,9 +60,8 @@ object SSLSampleConfigs {
     conf.set("spark.ssl.keyPassword", "password")
     conf.set("spark.ssl.trustStore", trustStorePath)
     conf.set("spark.ssl.trustStorePassword", "password")
-    conf.set("spark.ssl.enabledAlgorithms",
-      "SSL_RSA_WITH_RC4_128_SHA, SSL_RSA_WITH_DES_CBC_SHA")
-    conf.set("spark.ssl.protocol", "TLSv1")
+    conf.set("spark.ssl.enabledAlgorithms", enabledAlgorithms)
+    conf.set("spark.ssl.protocol", "TLSv1.2")
     conf
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/eab1d16a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala 
b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala
index e9b64aa..f34aefc 100644
--- a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala
@@ -127,6 +127,17 @@ class SecurityManagerSuite extends SparkFunSuite {
 
   test("ssl on setup") {
     val conf = SSLSampleConfigs.sparkSSLConfig()
+    val expectedAlgorithms = Set(
+    "TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA384",
+    "TLS_RSA_WITH_AES_256_CBC_SHA256",
+    "TLS_DHE_RSA_WITH_AES_256_CBC_SHA256",
+    "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256",
+    "TLS_DHE_RSA_WITH_AES_128_CBC_SHA256",
+    "SSL_ECDHE_RSA_WITH_AES_256_CBC_SHA384",
+    "SSL_RSA_WITH_AES_256_CBC_SHA256",
+    "SSL_DHE_RSA_WITH_AES_256_CBC_SHA256",
+    "SSL_ECDHE_RSA_WITH_AES_128_CBC_SHA256",
+    "SSL_DHE_RSA_WITH_AES_128_CBC_SHA256")
 
     val securityManager = new SecurityManager(conf)
 
@@ -143,9 +154,8 @@ class SecurityManagerSuite extends SparkFunSuite {
     assert(securityManager.fileServerSSLOptions.trustStorePassword === 
Some("password"))
     assert(securityManager.fileServerSSLOptions.keyStorePassword === 
Some("password"))
     assert(securityManager.fileServerSSLOptions.keyPassword === 
Some("password"))
-    assert(securityManager.fileServerSSLOptions.protocol === Some("TLSv1"))
-    assert(securityManager.fileServerSSLOptions.enabledAlgorithms ===
-        Set("SSL_RSA_WITH_RC4_128_SHA", "SSL_RSA_WITH_DES_CBC_SHA"))
+    assert(securityManager.fileServerSSLOptions.protocol === Some("TLSv1.2"))
+    assert(securityManager.fileServerSSLOptions.enabledAlgorithms === 
expectedAlgorithms)
 
     assert(securityManager.akkaSSLOptions.trustStore.isDefined === true)
     assert(securityManager.akkaSSLOptions.trustStore.get.getName === 
"truststore")
@@ -154,9 +164,8 @@ class SecurityManagerSuite extends SparkFunSuite {
     assert(securityManager.akkaSSLOptions.trustStorePassword === 
Some("password"))
     assert(securityManager.akkaSSLOptions.keyStorePassword === 
Some("password"))
     assert(securityManager.akkaSSLOptions.keyPassword === Some("password"))
-    assert(securityManager.akkaSSLOptions.protocol === Some("TLSv1"))
-    assert(securityManager.akkaSSLOptions.enabledAlgorithms ===
-        Set("SSL_RSA_WITH_RC4_128_SHA", "SSL_RSA_WITH_DES_CBC_SHA"))
+    assert(securityManager.akkaSSLOptions.protocol === Some("TLSv1.2"))
+    assert(securityManager.akkaSSLOptions.enabledAlgorithms === 
expectedAlgorithms)
   }
 
   test("ssl off setup") {


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to