Repository: spark
Updated Branches:
  refs/heads/master ef65cf09b -> cfea30037


http://git-wip-us.apache.org/repos/asf/spark/blob/cfea3003/docs/configuration.md
----------------------------------------------------------------------
diff --git a/docs/configuration.md b/docs/configuration.md
index 08c6bef..62d3fca 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -1242,6 +1242,86 @@ Apart from these, the following properties are also 
available, and may be useful
 </tr>
 </table>
 
+#### Encryption
+
+<table class="table">
+    <tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+    <tr>
+        <td><code>spark.ssl.enabled</code></td>
+        <td>false</td>
+        <td>
+            <p>Whether to enable SSL connections on all supported 
protocols.</p>
+
+            <p>All the SSL settings like <code>spark.ssl.xxx</code> where 
<code>xxx</code> is a
+            particular configuration property, denote the global configuration 
for all the supported
+            protocols. In order to override the global configuration for the 
particular protocol,
+            the properties must be overwritten in the protocol-specific 
namespace.</p>
+
+            <p>Use <code>spark.ssl.YYY.XXX</code> settings to overwrite the 
global configuration for
+            particular protocol denoted by <code>YYY</code>. Currently 
<code>YYY</code> can be
+            either <code>akka</code> for Akka based connections or 
<code>fs</code> for broadcast and
+            file server.</p>
+        </td>
+    </tr>
+    <tr>
+        <td><code>spark.ssl.keyStore</code></td>
+        <td>None</td>
+        <td>
+            A path to a key-store file. The path can be absolute or relative 
to the directory where
+            the component is started in.
+        </td>
+    </tr>
+    <tr>
+        <td><code>spark.ssl.keyStorePassword</code></td>
+        <td>None</td>
+        <td>
+            A password to the key-store.
+        </td>
+    </tr>
+    <tr>
+        <td><code>spark.ssl.keyPassword</code></td>
+        <td>None</td>
+        <td>
+            A password to the private key in key-store.
+        </td>
+    </tr>
+    <tr>
+        <td><code>spark.ssl.trustStore</code></td>
+        <td>None</td>
+        <td>
+            A path to a trust-store file. The path can be absolute or relative 
to the directory
+            where the component is started in.
+        </td>
+    </tr>
+    <tr>
+        <td><code>spark.ssl.trustStorePassword</code></td>
+        <td>None</td>
+        <td>
+            A password to the trust-store.
+        </td>
+    </tr>
+    <tr>
+        <td><code>spark.ssl.protocol</code></td>
+        <td>None</td>
+        <td>
+            A protocol name. The protocol must be supported by JVM. The 
reference list of protocols
+            one can find on <a 
href="https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https";>this</a>
+            page.
+        </td>
+    </tr>
+    <tr>
+        <td><code>spark.ssl.enabledAlgorithms</code></td>
+        <td>Empty</td>
+        <td>
+            A comma separated list of ciphers. The specified ciphers must be 
supported by JVM.
+            The reference list of protocols one can find on
+            <a 
href="https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https";>this</a>
+            page.
+        </td>
+    </tr>
+</table>
+
+
 #### Spark Streaming
 <table class="table">
 <tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>

http://git-wip-us.apache.org/repos/asf/spark/blob/cfea3003/docs/security.md
----------------------------------------------------------------------
diff --git a/docs/security.md b/docs/security.md
index 1e206a1..6e0a54f 100644
--- a/docs/security.md
+++ b/docs/security.md
@@ -20,6 +20,30 @@ Spark allows for a set of administrators to be specified in 
the acls who always
 
 If your applications are using event logging, the directory where the event 
logs go (`spark.eventLog.dir`) should be manually created and have the proper 
permissions set on it. If you want those log files secured, the permissions 
should be set to `drwxrwxrwxt` for that directory. The owner of the directory 
should be the super user who is running the history server and the group 
permissions should be restricted to super user group. This will allow all users 
to write to the directory but will prevent unprivileged users from removing or 
renaming a file unless they own the file or directory. The event log files will 
be created by Spark with permissions such that only the user and group have 
read and write access.
 
+## Encryption
+
+Spark supports SSL for Akka and HTTP (for broadcast and file server) 
protocols. However SSL is not supported yet for WebUI and block transfer 
service.
+
+Connection encryption (SSL) configuration is organized hierarchically. The 
user can configure the default SSL settings which will be used for all the 
supported communication protocols unless they are overwritten by 
protocol-specific settings. This way the user can easily provide the common 
settings for all the protocols without disabling the ability to configure each 
one individually. The common SSL settings are at `spark.ssl` namespace in Spark 
configuration, while Akka SSL configuration is at `spark.ssl.akka` and HTTP for 
broadcast and file server SSL configuration is at `spark.ssl.fs`. The full 
breakdown can be found on the [configuration page](configuration.html).
+
+SSL must be configured on each node and configured for each component involved 
in communication using the particular protocol.
+
+### YARN mode
+The key-store can be prepared on the client side and then distributed and used 
by the executors as the part of the application. It is possible because the 
user is able to deploy files before the application is started in YARN by using 
`spark.yarn.dist.files` or `spark.yarn.dist.archives` configuration settings. 
The responsibility for encryption of transferring these files is on YARN side 
and has nothing to do with Spark.
+
+### Standalone mode
+The user needs to provide key-stores and configuration options for master and 
workers. They have to be set by attaching appropriate Java system properties in 
`SPARK_MASTER_OPTS` and in `SPARK_WORKER_OPTS` environment variables, or just 
in `SPARK_DAEMON_JAVA_OPTS`. In this mode, the user may allow the executors to 
use the SSL settings inherited from the worker which spawned that executor. It 
can be accomplished by setting `spark.ssl.useNodeLocalConf` to `true`. If that 
parameter is set, the settings provided by user on the client side, are not 
used by the executors.
+
+### Preparing the key-stores
+Key-stores can be generated by `keytool` program. The reference documentation 
for this tool is
+[here](https://docs.oracle.com/javase/7/docs/technotes/tools/solaris/keytool.html).
 The most basic
+steps to configure the key-stores and the trust-store for the standalone 
deployment mode is as
+follows:
+* Generate a keys pair for each node
+* Export the public key of the key pair to a file on each node
+* Import all exported public keys into a single trust-store
+* Distribute the trust-store over the nodes
+
 ## Configuring Ports for Network Security
 
 Spark makes heavy use of the network, and some environments have strict 
requirements for using tight

http://git-wip-us.apache.org/repos/asf/spark/blob/cfea3003/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala
----------------------------------------------------------------------
diff --git 
a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala 
b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala
index b46df12..9805609 100644
--- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala
@@ -45,7 +45,7 @@ class ExecutorClassLoader(conf: SparkConf, classUri: String, 
parent: ClassLoader
 
   // Hadoop FileSystem object for our URI, if it isn't using HTTP
   var fileSystem: FileSystem = {
-    if (uri.getScheme() == "http") {
+    if (Set("http", "https", "ftp").contains(uri.getScheme)) {
       null
     } else {
       FileSystem.get(uri, SparkHadoopUtil.get.newConfiguration(conf))
@@ -78,13 +78,16 @@ class ExecutorClassLoader(conf: SparkConf, classUri: 
String, parent: ClassLoader
         if (fileSystem != null) {
           fileSystem.open(new Path(directory, pathInDirectory))
         } else {
-          if (SparkEnv.get.securityManager.isAuthenticationEnabled()) {
+          val url = if 
(SparkEnv.get.securityManager.isAuthenticationEnabled()) {
             val uri = new URI(classUri + "/" + urlEncode(pathInDirectory))
             val newuri = Utils.constructURIForAuthentication(uri, 
SparkEnv.get.securityManager)
-            newuri.toURL().openStream()
+            newuri.toURL
           } else {
-            new URL(classUri + "/" + urlEncode(pathInDirectory)).openStream()
+            new URL(classUri + "/" + urlEncode(pathInDirectory))
           }
+
+          Utils.setupSecureURLConnection(url.openConnection(), 
SparkEnv.get.securityManager)
+            .getInputStream
         }
       }
       val bytes = readAndTransformClass(name, inputStream)

http://git-wip-us.apache.org/repos/asf/spark/blob/cfea3003/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala
----------------------------------------------------------------------
diff --git 
a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala
 
b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala
index 716cf2c..7d29ed8 100644
--- 
a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala
+++ 
b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala
@@ -67,8 +67,12 @@ private[streaming] class ReceiverSupervisorImpl(
   private val trackerActor = {
     val ip = env.conf.get("spark.driver.host", "localhost")
     val port = env.conf.getInt("spark.driver.port", 7077)
-    val url = "akka.tcp://%s@%s:%s/user/ReceiverTracker".format(
-      SparkEnv.driverActorSystemName, ip, port)
+    val url = AkkaUtils.address(
+      AkkaUtils.protocol(env.actorSystem),
+      SparkEnv.driverActorSystemName,
+      ip,
+      port,
+      "ReceiverTracker")
     env.actorSystem.actorSelection(url)
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/cfea3003/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
----------------------------------------------------------------------
diff --git 
a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala 
b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index eb328b2..37e98e0 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -244,7 +244,9 @@ private[spark] class ApplicationMaster(
       host: String,
       port: String,
       isDriver: Boolean): Unit = {
-    val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format(
+    
+    val driverUrl = AkkaUtils.address(
+      AkkaUtils.protocol(actorSystem),
       SparkEnv.driverActorSystemName,
       host,
       port,

http://git-wip-us.apache.org/repos/asf/spark/blob/cfea3003/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
----------------------------------------------------------------------
diff --git 
a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala 
b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
index 040406c..0dbb615 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
@@ -34,9 +34,10 @@ import org.apache.hadoop.yarn.util.RackResolver
 
 import org.apache.log4j.{Level, Logger}
 
-import org.apache.spark.{Logging, SecurityManager, SparkConf}
+import org.apache.spark.{SparkEnv, Logging, SecurityManager, SparkConf}
 import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._
 import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
+import org.apache.spark.util.AkkaUtils
 
 /**
  * YarnAllocator is charged with requesting containers from the YARN 
ResourceManager and deciding
@@ -106,7 +107,9 @@ private[yarn] class YarnAllocator(
     new ThreadFactoryBuilder().setNameFormat("ContainerLauncher 
#%d").setDaemon(true).build())
   launcherPool.allowCoreThreadTimeOut(true)
 
-  private val driverUrl = "akka.tcp://sparkDriver@%s:%s/user/%s".format(
+  private val driverUrl = AkkaUtils.address(
+    AkkaUtils.protocol(securityMgr.akkaSSLOptions.enabled),
+    SparkEnv.driverActorSystemName,
     sparkConf.get("spark.driver.host"),
     sparkConf.get("spark.driver.port"),
     CoarseGrainedSchedulerBackend.ACTOR_NAME)


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

Reply via email to