Repository: spark
Updated Branches:
  refs/heads/master eb48edf9c -> b30a7d28b


[SPARK-23572][DOCS] Bring "security.md" up to date.

This change basically rewrites the security documentation so that it's
up to date with new features, more correct, and more complete.

Because security is such an important feature, I chose to move all the
relevant configuration documentation to the security page, instead of
having them peppered all over the place in the configuration page. This
allows an almost one-stop shop for security configuration in Spark. The
only exceptions are some YARN-specific minor features which I left in
the YARN page.

I also re-organized the page's topics, since they didn't make a lot of
sense. You had kerberos features described inside paragraphs talking
about UI access control, and other oddities. It should be easier now
to find information about specific Spark security features. I also
enabled TOCs for both the Security and YARN pages, since that makes it
easier to see what is covered.

I removed most of the comments from the SecurityManager javadoc since
they just replicated information in the security doc, with different
levels of out-of-dateness.

Author: Marcelo Vanzin <[email protected]>

Closes #20742 from vanzin/SPARK-23572.


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

Branch: refs/heads/master
Commit: b30a7d28b399950953d4b112c57d4c9b9ab223e9
Parents: eb48edf
Author: Marcelo Vanzin <[email protected]>
Authored: Mon Mar 26 12:45:45 2018 -0700
Committer: Marcelo Vanzin <[email protected]>
Committed: Mon Mar 26 12:45:45 2018 -0700

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 .../org/apache/spark/SecurityManager.scala      | 144 +----
 docs/configuration.md                           | 359 +----------
 docs/monitoring.md                              |  40 +-
 docs/running-on-yarn.md                         | 203 +++---
 docs/security.md                                | 629 ++++++++++++++++---
 6 files changed, 673 insertions(+), 703 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/b30a7d28/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 3908590..e4c44d0 100644
--- a/.gitignore
+++ b/.gitignore
@@ -76,6 +76,7 @@ streaming-tests.log
 target/
 unit-tests.log
 work/
+docs/.jekyll-metadata
 
 # For Hive
 TempStatsStore/

http://git-wip-us.apache.org/repos/asf/spark/blob/b30a7d28/core/src/main/scala/org/apache/spark/SecurityManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala 
b/core/src/main/scala/org/apache/spark/SecurityManager.scala
index da1c89c..09ec893 100644
--- a/core/src/main/scala/org/apache/spark/SecurityManager.scala
+++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala
@@ -42,148 +42,10 @@ import org.apache.spark.util.Utils
  * should access it from that. There are some cases where the SparkEnv hasn't 
been
  * initialized yet and this class must be instantiated directly.
  *
- * Spark currently supports authentication via a shared secret.
- * Authentication can be configured to be on via the 'spark.authenticate' 
configuration
- * parameter. This parameter controls whether the Spark communication 
protocols do
- * authentication using the shared secret. This authentication is a basic 
handshake to
- * make sure both sides have the same shared secret and are allowed to 
communicate.
- * If the shared secret is not identical they will not be allowed to 
communicate.
- *
- * The Spark UI can also be secured by using javax servlet filters. A user may 
want to
- * secure the UI if it has data that other users should not be allowed to see. 
The javax
- * servlet filter specified by the user can authenticate the user and then 
once the user
- * is logged in, Spark can compare that user versus the view acls to make sure 
they are
- * authorized to view the UI. The configs 'spark.acls.enable', 
'spark.ui.view.acls' and
- * 'spark.ui.view.acls.groups' control the behavior of the acls. Note that the 
person who
- * started the application always has view access to the UI.
- *
- * Spark has a set of individual and group modify acls (`spark.modify.acls`) 
and
- * (`spark.modify.acls.groups`) that controls which users and groups have 
permission to
- * modify a single application. This would include things like killing the 
application.
- * By default the person who started the application has modify access. For 
modify access
- * through the UI, you must have a filter that does authentication in place 
for the modify
- * acls to work properly.
- *
- * Spark also has a set of individual and group admin acls 
(`spark.admin.acls`) and
- * (`spark.admin.acls.groups`) which is a set of users/administrators and 
admin groups
- * who always have permission to view or modify the Spark application.
- *
- * Starting from version 1.3, Spark has partial support for encrypted 
connections with SSL.
- *
- * At this point spark has multiple communication protocols that need to be 
secured and
- * different underlying mechanisms are used depending on the protocol:
- *
- *  - HTTP for broadcast and file server (via HttpServer) ->  Spark currently 
uses Jetty
- *            for the HttpServer. Jetty supports multiple authentication 
mechanisms -
- *            Basic, Digest, Form, Spnego, etc. It also supports multiple 
different login
- *            services - Hash, JAAS, Spnego, JDBC, etc.  Spark currently uses 
the HashLoginService
- *            to authenticate using DIGEST-MD5 via a single user and the 
shared secret.
- *            Since we are using DIGEST-MD5, the shared secret is not passed 
on the wire
- *            in plaintext.
- *
- *            We currently support SSL (https) for this communication protocol 
(see the details
- *            below).
- *
- *            The Spark HttpServer installs the HashLoginServer and configures 
it to DIGEST-MD5.
- *            Any clients must specify the user and password. There is a 
default
- *            Authenticator installed in the SecurityManager to how it does 
the authentication
- *            and in this case gets the user name and password from the 
request.
- *
- *  - BlockTransferService -> The Spark BlockTransferServices uses java nio to 
asynchronously
- *            exchange messages.  For this we use the Java SASL
- *            (Simple Authentication and Security Layer) API and again use 
DIGEST-MD5
- *            as the authentication mechanism. This means the shared secret is 
not passed
- *            over the wire in plaintext.
- *            Note that SASL is pluggable as to what mechanism it uses.  We 
currently use
- *            DIGEST-MD5 but this could be changed to use Kerberos or other in 
the future.
- *            Spark currently supports "auth" for the quality of protection, 
which means
- *            the connection does not support integrity or privacy protection 
(encryption)
- *            after authentication. SASL also supports "auth-int" and 
"auth-conf" which
- *            SPARK could support in the future to allow the user to specify 
the quality
- *            of protection they want. If we support those, the messages will 
also have to
- *            be wrapped and unwrapped via the 
SaslServer/SaslClient.wrap/unwrap API's.
- *
- *            Since the NioBlockTransferService does asynchronous messages 
passing, the SASL
- *            authentication is a bit more complex. A ConnectionManager can be 
both a client
- *            and a Server, so for a particular connection it has to determine 
what to do.
- *            A ConnectionId was added to be able to track connections and is 
used to
- *            match up incoming messages with connections waiting for 
authentication.
- *            The ConnectionManager tracks all the sendingConnections using 
the ConnectionId,
- *            waits for the response from the server, and does the handshake 
before sending
- *            the real message.
- *
- *            The NettyBlockTransferService ensures that SASL authentication 
is performed
- *            synchronously prior to any other communication on a connection. 
This is done in
- *            SaslClientBootstrap on the client side and SaslRpcHandler on the 
server side.
- *
- *  - HTTP for the Spark UI -> the UI was changed to use servlets so that 
javax servlet filters
- *            can be used. Yarn requires a specific AmIpFilter be installed 
for security to work
- *            properly. For non-Yarn deployments, users can write a filter to 
go through their
- *            organization's normal login service. If an authentication filter 
is in place then the
- *            SparkUI can be configured to check the logged in user against 
the list of users who
- *            have view acls to see if that user is authorized.
- *            The filters can also be used for many different purposes. For 
instance filters
- *            could be used for logging, encryption, or compression.
- *
- *  The exact mechanisms used to generate/distribute the shared secret are 
deployment-specific.
- *
- *  For YARN deployments, the secret is automatically generated. The secret is 
placed in the Hadoop
- *  UGI which gets passed around via the Hadoop RPC mechanism. Hadoop RPC can 
be configured to
- *  support different levels of protection. See the Hadoop documentation for 
more details. Each
- *  Spark application on YARN gets a different shared secret.
- *
- *  On YARN, the Spark UI gets configured to use the Hadoop YARN AmIpFilter 
which requires the user
- *  to go through the ResourceManager Proxy. That proxy is there to reduce the 
possibility of web
- *  based attacks through YARN. Hadoop can be configured to use filters to do 
authentication. That
- *  authentication then happens via the ResourceManager Proxy and Spark will 
use that to do
- *  authorization against the view acls.
- *
- *  For other Spark deployments, the shared secret must be specified via the
- *  spark.authenticate.secret config.
- *  All the nodes (Master and Workers) and the applications need to have the 
same shared secret.
- *  This again is not ideal as one user could potentially affect another users 
application.
- *  This should be enhanced in the future to provide better protection.
- *  If the UI needs to be secure, the user needs to install a javax servlet 
filter to do the
- *  authentication. Spark will then use that user to compare against the view 
acls to do
- *  authorization. If not filter is in place the user is generally null and no 
authorization
- *  can take place.
- *
- *  When authentication is being used, encryption can also be enabled by 
setting the option
- *  spark.authenticate.enableSaslEncryption to true. This is only supported by 
communication
- *  channels that use the network-common library, and can be used as an 
alternative to SSL in those
- *  cases.
- *
- *  SSL can be used for encryption for certain communication channels. 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.
- *
- *  All the SSL settings like `spark.ssl.xxx` where `xxx` 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. Use `spark.ssl.yyy.xxx` settings to overwrite 
the global
- *  configuration for particular protocol denoted by `yyy`. Currently `yyy` 
can be only`fs` for
- *  broadcast and file server.
- *
- *  Refer to [[org.apache.spark.SSLOptions]] documentation for the list of
- *  options that can be specified.
- *
- *  SecurityManager initializes SSLOptions objects for different protocols 
separately. SSLOptions
- *  object parses Spark configuration at a given namespace and builds the 
common representation
- *  of SSL settings. SSLOptions is then used to provide protocol-specific 
SSLContextFactory for
- *  Jetty.
- *
- *  SSL must be configured on each node and configured for each component 
involved in
- *  communication using the particular protocol. In YARN clusters, the 
key-store can be prepared on
- *  the client side then distributed and used by the executors as the part of 
the application
- *  (YARN allows the user to deploy files before the application is started).
- *  In standalone deployment, the user needs to provide key-stores and 
configuration
- *  options for master and workers. 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`.
+ * This class implements all of the configuration related to security features 
described
+ * in the "Security" document. Please refer to that document for specific 
features implemented
+ * here.
  */
-
 private[spark] class SecurityManager(
     sparkConf: SparkConf,
     val ioEncryptionKey: Option[Array[Byte]] = None)

http://git-wip-us.apache.org/repos/asf/spark/blob/b30a7d28/docs/configuration.md
----------------------------------------------------------------------
diff --git a/docs/configuration.md b/docs/configuration.md
index e7f2419..2eb6a77 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -712,30 +712,6 @@ Apart from these, the following properties are also 
available, and may be useful
     When we fail to register to the external shuffle service, we will retry 
for maxAttempts times.
   </td>
 </tr>
-<tr>
-  <td><code>spark.io.encryption.enabled</code></td>
-  <td>false</td>
-  <td>
-    Enable IO encryption. Currently supported by all modes except Mesos. It's 
recommended that RPC encryption
-    be enabled when using this feature.
-  </td>
-</tr>
-<tr>
-  <td><code>spark.io.encryption.keySizeBits</code></td>
-  <td>128</td>
-  <td>
-    IO encryption key size in bits. Supported values are 128, 192 and 256.
-  </td>
-</tr>
-<tr>
-  <td><code>spark.io.encryption.keygen.algorithm</code></td>
-  <td>HmacSHA1</td>
-  <td>
-    The algorithm to use when generating the IO encryption key. The supported 
algorithms are
-    described in the KeyGenerator section of the Java Cryptography 
Architecture Standard Algorithm
-    Name Documentation.
-  </td>
-</tr>
 </table>
 
 ### Spark UI
@@ -893,6 +869,23 @@ Apart from these, the following properties are also 
available, and may be useful
     How many dead executors the Spark UI and status APIs remember before 
garbage collecting.
   </td>
 </tr>
+<tr>
+  <td><code>spark.ui.filters</code></td>
+  <td>None</td>
+  <td>
+    Comma separated list of filter class names to apply to the Spark Web UI. 
The filter should be a
+    standard <a 
href="http://docs.oracle.com/javaee/6/api/javax/servlet/Filter.html";>
+    javax servlet Filter</a>.
+
+    <br />Filter parameters can also be specified in the configuration, by 
setting config entries
+    of the form <code>spark.&lt;class name of filter&gt;.param.&lt;param 
name&gt;=&lt;value&gt;</code>
+
+    <br />For example:
+    <br /><code>spark.ui.filters=com.test.filter1</code>
+    <br /><code>spark.com.test.filter1.param.name1=foo</code>
+    <br /><code>spark.com.test.filter1.param.name2=bar</code>
+  </td>
+</tr>
 </table>
 
 ### Compression and Serialization
@@ -1446,6 +1439,15 @@ Apart from these, the following properties are also 
available, and may be useful
     Duration for an RPC remote endpoint lookup operation to wait before timing 
out.
   </td>
 </tr>
+<tr>
+  <td><code>spark.core.connection.ack.wait.timeout</code></td>
+  <td><code>spark.network.timeout</code></td>
+  <td>
+    How long for the connection to wait for ack to occur before timing
+    out and giving up. To avoid unwilling timeout caused by long pause like GC,
+    you can set larger value.
+  </td>
+</tr>
 </table>
 
 ### Scheduling
@@ -1817,313 +1819,8 @@ Apart from these, the following properties are also 
available, and may be useful
 
 ### Security
 
-<table class="table">
-<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
-<tr>
-  <td><code>spark.acls.enable</code></td>
-  <td>false</td>
-  <td>
-    Whether Spark acls should be enabled. If enabled, this checks to see if 
the user has
-    access permissions to view or modify the job.  Note this requires the user 
to be known,
-    so if the user comes across as null no checks are done. Filters can be 
used with the UI
-    to authenticate and set the user.
-  </td>
-</tr>
-<tr>
-  <td><code>spark.admin.acls</code></td>
-  <td>Empty</td>
-  <td>
-    Comma separated list of users/administrators that have view and modify 
access to all Spark jobs.
-    This can be used if you run on a shared cluster and have a set of 
administrators or devs who
-    help debug when things do not work. Putting a "*" in the list means any 
user can have the
-    privilege of admin.
-  </td>
-</tr>
-<tr>
-  <td><code>spark.admin.acls.groups</code></td>
-  <td>Empty</td>
-  <td>
-    Comma separated list of groups that have view and modify access to all 
Spark jobs.
-    This can be used if you have a set of administrators or developers who 
help maintain and debug
-    the underlying infrastructure. Putting a "*" in the list means any user in 
any group can have
-    the privilege of admin. The user groups are obtained from the instance of 
the groups mapping
-    provider specified by <code>spark.user.groups.mapping</code>. Check the 
entry
-    <code>spark.user.groups.mapping</code> for more details.
-  </td>
-</tr>
-<tr>
-  <td><code>spark.user.groups.mapping</code></td>
-  
<td><code>org.apache.spark.security.ShellBasedGroupsMappingProvider</code></td>
-  <td>
-    The list of groups for a user is determined by a group mapping service 
defined by the trait
-    org.apache.spark.security.GroupMappingServiceProvider which can be 
configured by this property.
-    A default unix shell based implementation is provided 
<code>org.apache.spark.security.ShellBasedGroupsMappingProvider</code>
-    which can be specified to resolve a list of groups for a user.
-    <em>Note:</em> This implementation supports only a Unix/Linux based 
environment. Windows environment is
-    currently <b>not</b> supported. However, a new platform/protocol can be 
supported by implementing
-    the trait 
<code>org.apache.spark.security.GroupMappingServiceProvider</code>.
-  </td>
-</tr>
-<tr>
-  <td><code>spark.authenticate</code></td>
-  <td>false</td>
-  <td>
-    Whether Spark authenticates its internal connections. See
-    <code>spark.authenticate.secret</code> if not running on YARN.
-  </td>
-</tr>
-<tr>
-  <td><code>spark.authenticate.secret</code></td>
-  <td>None</td>
-  <td>
-    Set the secret key used for Spark to authenticate between components. This 
needs to be set if
-    not running on YARN and authentication is enabled.
-  </td>
-</tr>
-<tr>
-  <td><code>spark.network.crypto.enabled</code></td>
-  <td>false</td>
-  <td>
-    Enable encryption using the commons-crypto library for RPC and block 
transfer service.
-    Requires <code>spark.authenticate</code> to be enabled.
-  </td>
-</tr>
-<tr>
-  <td><code>spark.network.crypto.keyLength</code></td>
-  <td>128</td>
-  <td>
-    The length in bits of the encryption key to generate. Valid values are 
128, 192 and 256.
-  </td>
-</tr>
-<tr>
-  <td><code>spark.network.crypto.keyFactoryAlgorithm</code></td>
-  <td>PBKDF2WithHmacSHA1</td>
-  <td>
-    The key factory algorithm to use when generating encryption keys. Should 
be one of the
-    algorithms supported by the javax.crypto.SecretKeyFactory class in the JRE 
being used.
-  </td>
-</tr>
-<tr>
-  <td><code>spark.network.crypto.saslFallback</code></td>
-  <td>true</td>
-  <td>
-    Whether to fall back to SASL authentication if authentication fails using 
Spark's internal
-    mechanism. This is useful when the application is connecting to old 
shuffle services that
-    do not support the internal Spark authentication protocol. On the server 
side, this can be
-    used to block older clients from authenticating against a new shuffle 
service.
-  </td>
-</tr>
-<tr>
-  <td><code>spark.network.crypto.config.*</code></td>
-  <td>None</td>
-  <td>
-    Configuration values for the commons-crypto library, such as which cipher 
implementations to
-    use. The config name should be the name of commons-crypto configuration 
without the
-    "commons.crypto" prefix.
-  </td>
-</tr>
-<tr>
-  <td><code>spark.authenticate.enableSaslEncryption</code></td>
-  <td>false</td>
-  <td>
-    Enable encrypted communication when authentication is
-    enabled. This is supported by the block transfer service and the
-    RPC endpoints.
-  </td>
-</tr>
-<tr>
-  <td><code>spark.network.sasl.serverAlwaysEncrypt</code></td>
-  <td>false</td>
-  <td>
-    Disable unencrypted connections for services that support SASL 
authentication.
-  </td>
-</tr>
-<tr>
-  <td><code>spark.core.connection.ack.wait.timeout</code></td>
-  <td><code>spark.network.timeout</code></td>
-  <td>
-    How long for the connection to wait for ack to occur before timing
-    out and giving up. To avoid unwilling timeout caused by long pause like GC,
-    you can set larger value.
-  </td>
-</tr>
-<tr>
-  <td><code>spark.modify.acls</code></td>
-  <td>Empty</td>
-  <td>
-    Comma separated list of users that have modify access to the Spark job. By 
default only the
-    user that started the Spark job has access to modify it (kill it for 
example). Putting a "*" in
-    the list means any user can have access to modify it.
-  </td>
-</tr>
-<tr>
-  <td><code>spark.modify.acls.groups</code></td>
-  <td>Empty</td>
-  <td>
-    Comma separated list of groups that have modify access to the Spark job. 
This can be used if you
-    have a set of administrators or developers from the same team to have 
access to control the job.
-    Putting a "*" in the list means any user in any group has the access to 
modify the Spark job.
-    The user groups are obtained from the instance of the groups mapping 
provider specified by
-    <code>spark.user.groups.mapping</code>. Check the entry 
<code>spark.user.groups.mapping</code>
-    for more details.
-  </td>
-</tr>
-<tr>
-  <td><code>spark.ui.filters</code></td>
-  <td>None</td>
-  <td>
-    Comma separated list of filter class names to apply to the Spark web UI. 
The filter should be a
-    standard <a 
href="http://docs.oracle.com/javaee/6/api/javax/servlet/Filter.html";>
-    javax servlet Filter</a>. Parameters to each filter can also be specified 
by setting a
-    java system property of: <br />
-    <code>spark.&lt;class name of 
filter&gt;.params='param1=value1,param2=value2'</code><br />
-    For example: <br />
-    <code>-Dspark.ui.filters=com.test.filter1</code> <br />
-    <code>-Dspark.com.test.filter1.params='param1=foo,param2=testing'</code>
-  </td>
-</tr>
-<tr>
-  <td><code>spark.ui.view.acls</code></td>
-  <td>Empty</td>
-  <td>
-    Comma separated list of users that have view access to the Spark web ui. 
By default only the
-    user that started the Spark job has view access. Putting a "*" in the list 
means any user can
-    have view access to this Spark job.
-  </td>
-</tr>
-<tr>
-  <td><code>spark.ui.view.acls.groups</code></td>
-  <td>Empty</td>
-  <td>
-    Comma separated list of groups that have view access to the Spark web ui 
to view the Spark Job
-    details. This can be used if you have a set of administrators or 
developers or users who can
-    monitor the Spark job submitted. Putting a "*" in the list means any user 
in any group can view
-    the Spark job details on the Spark web ui. The user groups are obtained 
from the instance of the
-    groups mapping provider specified by 
<code>spark.user.groups.mapping</code>. Check the entry
-    <code>spark.user.groups.mapping</code> for more details.
-  </td>
-</tr>
-</table>
-
-### TLS / SSL
-
-<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>
-            Whether to enable SSL connections on all supported protocols.
-
-            <br />When <code>spark.ssl.enabled</code> is configured, 
<code>spark.ssl.protocol</code>
-            is required.
-
-            <br />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.
-
-            <br />Use <code>spark.ssl.YYY.XXX</code> settings to overwrite the 
global configuration for
-            particular protocol denoted by <code>YYY</code>. Example values 
for <code>YYY</code>
-            include <code>fs</code>, <code>ui</code>, <code>standalone</code>, 
and
-            <code>historyServer</code>.  See <a 
href="security.html#ssl-configuration">SSL
-            Configuration</a> for details on hierarchical SSL configuration 
for services.
-        </td>
-    </tr>
-    <tr>
-        <td><code>spark.ssl.[namespace].port</code></td>
-        <td>None</td>
-        <td>
-            The port where the SSL service will listen on.
-
-            <br />The port must be defined within a namespace configuration; 
see
-            <a href="security.html#ssl-configuration">SSL Configuration</a> 
for the available
-            namespaces.
-
-            <br />When not set, the SSL port will be derived from the non-SSL 
port for the
-            same service. A value of "0" will make the service bind to an 
ephemeral port.
-        </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.
-            Note: If not set, it will use the default cipher suites of JVM.
-        </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.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.keyStoreType</code></td>
-        <td>JKS</td>
-        <td>
-            The type of the key-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.needClientAuth</code></td>
-        <td>false</td>
-        <td>
-            Set true if SSL needs client authentication.
-        </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.trustStoreType</code></td>
-        <td>JKS</td>
-        <td>
-            The type of the trust-store.
-        </td>
-    </tr>
-</table>
-
+Please refer to the [Security](security.html) page for available options on 
how to secure different
+Spark subsystems.
 
 ### Spark SQL
 

http://git-wip-us.apache.org/repos/asf/spark/blob/b30a7d28/docs/monitoring.md
----------------------------------------------------------------------
diff --git a/docs/monitoring.md b/docs/monitoring.md
index d5f7ffc..01736c7 100644
--- a/docs/monitoring.md
+++ b/docs/monitoring.md
@@ -80,7 +80,10 @@ The history server can be configured as follows:
   </tr>
 </table>
 
-### Spark configuration options
+### Spark History Server Configuration Options
+
+Security options for the Spark History Server are covered more detail in the
+[Security](security.html#web-ui) page.
 
 <table class="table">
   <tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
@@ -161,41 +164,6 @@ The history server can be configured as follows:
     </td>
   </tr>
   <tr>
-    <td>spark.history.ui.acls.enable</td>
-    <td>false</td>
-    <td>
-      Specifies whether acls should be checked to authorize users viewing the 
applications.
-      If enabled, access control checks are made regardless of what the 
individual application had
-      set for <code>spark.ui.acls.enable</code> when the application was run. 
The application owner
-      will always have authorization to view their own application and any 
users specified via
-      <code>spark.ui.view.acls</code> and groups specified via 
<code>spark.ui.view.acls.groups</code>
-      when the application was run will also have authorization to view that 
application.
-      If disabled, no access control checks are made.
-    </td>
-  </tr>
-  <tr>
-    <td>spark.history.ui.admin.acls</td>
-    <td>empty</td>
-    <td>
-      Comma separated list of users/administrators that have view access to 
all the Spark applications in
-      history server. By default only the users permitted to view the 
application at run-time could
-      access the related application history, with this, configured 
users/administrators could also
-      have the permission to access it.
-      Putting a "*" in the list means any user can have the privilege of admin.
-    </td>
-  </tr>
-  <tr>
-    <td>spark.history.ui.admin.acls.groups</td>
-    <td>empty</td>
-    <td>
-      Comma separated list of groups that have view access to all the Spark 
applications in
-      history server. By default only the groups permitted to view the 
application at run-time could
-      access the related application history, with this, configured groups 
could also
-      have the permission to access it.
-      Putting a "*" in the list means any group can have the privilege of 
admin.
-    </td>
-  </tr>
-  <tr>
     <td>spark.history.fs.cleaner.enabled</td>
     <td>false</td>
     <td>

http://git-wip-us.apache.org/repos/asf/spark/blob/b30a7d28/docs/running-on-yarn.md
----------------------------------------------------------------------
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index c010af3..e07759a 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -2,6 +2,8 @@
 layout: global
 title: Running Spark on YARN
 ---
+* This will become a table of contents (this text will be scraped).
+{:toc}
 
 Support for running on [YARN (Hadoop
 
NextGen)](http://hadoop.apache.org/docs/stable/hadoop-yarn/hadoop-yarn-site/YARN.html)
@@ -217,8 +219,8 @@ To use a custom metrics.properties for the application 
master and executors, upd
   <td><code>spark.yarn.dist.forceDownloadSchemes</code></td>
   <td><code>(none)</code></td>
   <td>
-    Comma-separated list of schemes for which files will be downloaded to the 
local disk prior to 
-    being added to YARN's distributed cache. For use in cases where the YARN 
service does not 
+    Comma-separated list of schemes for which files will be downloaded to the 
local disk prior to
+    being added to YARN's distributed cache. For use in cases where the YARN 
service does not
     support schemes that are supported by Spark, like http, https and ftp.
   </td>
 </tr>
@@ -266,19 +268,6 @@ To use a custom metrics.properties for the application 
master and executors, upd
   </td>
 </tr>
 <tr>
-  <td><code>spark.yarn.access.hadoopFileSystems</code></td>
-  <td>(none)</td>
-  <td>
-    A comma-separated list of secure Hadoop filesystems your Spark application 
is going to access. For
-    example, 
<code>spark.yarn.access.hadoopFileSystems=hdfs://nn1.com:8032,hdfs://nn2.com:8032,
-    webhdfs://nn3.com:50070</code>. The Spark application must have access to 
the filesystems listed
-    and Kerberos must be properly configured to be able to access them (either 
in the same realm
-    or in a trusted realm). Spark acquires security tokens for each of the 
filesystems so that
-    the Spark application can access those remote Hadoop filesystems. 
<code>spark.yarn.access.namenodes</code>
-    is deprecated, please use this instead.
-  </td>
-</tr>
-<tr>
   <td><code>spark.yarn.appMasterEnv.[EnvironmentVariableName]</code></td>
   <td>(none)</td>
   <td>
@@ -374,31 +363,6 @@ To use a custom metrics.properties for the application 
master and executors, upd
   </td>
 </tr>
 <tr>
-  <td><code>spark.yarn.keytab</code></td>
-  <td>(none)</td>
-  <td>
-  The full path to the file that contains the keytab for the principal 
specified above.
-  This keytab will be copied to the node running the YARN Application Master 
via the Secure Distributed Cache,
-  for renewing the login tickets and the delegation tokens periodically. 
(Works also with the "local" master)
-  </td>
-</tr>
-<tr>
-  <td><code>spark.yarn.principal</code></td>
-  <td>(none)</td>
-  <td>
-  Principal to be used to login to KDC, while running on secure HDFS. (Works 
also with the "local" master)
-  </td>
-</tr>
-<tr>
-  <td><code>spark.yarn.kerberos.relogin.period</code></td>
-  <td>1m</td>
-  <td>
-  How often to check whether the kerberos TGT should be renewed. This should 
be set to a value
-  that is shorter than the TGT renewal period (or the TGT lifetime if TGT 
renewal is not enabled).
-  The default value should be enough for most deployments.
-  </td>
-</tr>
-<tr>
   <td><code>spark.yarn.config.gatewayPath</code></td>
   <td>(none)</td>
   <td>
@@ -425,17 +389,6 @@ To use a custom metrics.properties for the application 
master and executors, upd
   </td>
 </tr>
 <tr>
-  <td><code>spark.security.credentials.${service}.enabled</code></td>
-  <td><code>true</code></td>
-  <td>
-  Controls whether to obtain credentials for services when security is enabled.
-  By default, credentials for all supported services are retrieved when those 
services are
-  configured, but it's possible to disable that behavior if it somehow 
conflicts with the
-  application being run. For further details please see
-  [Running in a Secure 
Cluster](running-on-yarn.html#running-in-a-secure-cluster)
-  </td>
-</tr>
-<tr>
   <td><code>spark.yarn.rolledLog.includePattern</code></td>
   <td>(none)</td>
   <td>
@@ -468,48 +421,104 @@ To use a custom metrics.properties for the application 
master and executors, upd
 - The `--files` and `--archives` options support specifying file names with 
the # similar to Hadoop. For example you can specify: `--files 
localtest.txt#appSees.txt` and this will upload the file you have locally named 
`localtest.txt` into HDFS but this will be linked to by the name `appSees.txt`, 
and your application should use the name as `appSees.txt` to reference it when 
running on YARN.
 - The `--jars` option allows the `SparkContext.addJar` function to work if you 
are using it with local files and running in `cluster` mode. It does not need 
to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files.
 
-# Running in a Secure Cluster
+# Kerberos
+
+Standard Kerberos support in Spark is covered in the 
[Security](security.html#kerberos) page.
+
+In YARN mode, when accessing Hadoop file systems, aside from the service 
hosting the user's home
+directory, Spark will also automatically obtain delegation tokens for the 
service hosting the
+staging directory of the Spark application.
+
+If an application needs to interact with other secure Hadoop filesystems, 
their URIs need to be
+explicitly provided to Spark at launch time. This is done by listing them in 
the
+`spark.yarn.access.hadoopFileSystems` property, described in the configuration 
section below.
 
-As covered in [security](security.html), Kerberos is used in a secure Hadoop 
cluster to
-authenticate principals associated with services and clients. This allows 
clients to
-make requests of these authenticated services; the services to grant rights
-to the authenticated principals.
+The YARN integration also supports custom delegation token providers using the 
Java Services
+mechanism (see `java.util.ServiceLoader`). Implementations of
+`org.apache.spark.deploy.yarn.security.ServiceCredentialProvider` can be made 
available to Spark
+by listing their names in the corresponding file in the jar's 
`META-INF/services` directory. These
+providers can be disabled individually by setting 
`spark.security.credentials.{service}.enabled` to
+`false`, where `{service}` is the name of the credential provider.
+
+## YARN-specific Kerberos Configuration
+
+<table class="table">
+<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+<tr>
+  <td><code>spark.yarn.keytab</code></td>
+  <td>(none)</td>
+  <td>
+  The full path to the file that contains the keytab for the principal 
specified above. This keytab
+  will be copied to the node running the YARN Application Master via the YARN 
Distributed Cache, and
+  will be used for renewing the login tickets and the delegation tokens 
periodically. Equivalent to
+  the <code>--keytab</code> command line argument.
+
+  <br /> (Works also with the "local" master.)
+  </td>
+</tr>
+<tr>
+  <td><code>spark.yarn.principal</code></td>
+  <td>(none)</td>
+  <td>
+  Principal to be used to login to KDC, while running on secure clusters. 
Equivalent to the
+  <code>--principal</code> command line argument.
+
+  <br /> (Works also with the "local" master.)
+  </td>
+</tr>
+<tr>
+  <td><code>spark.yarn.access.hadoopFileSystems</code></td>
+  <td>(none)</td>
+  <td>
+    A comma-separated list of secure Hadoop filesystems your Spark application 
is going to access. For
+    example, 
<code>spark.yarn.access.hadoopFileSystems=hdfs://nn1.com:8032,hdfs://nn2.com:8032,
+    webhdfs://nn3.com:50070</code>. The Spark application must have access to 
the filesystems listed
+    and Kerberos must be properly configured to be able to access them (either 
in the same realm
+    or in a trusted realm). Spark acquires security tokens for each of the 
filesystems so that
+    the Spark application can access those remote Hadoop filesystems.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.yarn.kerberos.relogin.period</code></td>
+  <td>1m</td>
+  <td>
+  How often to check whether the kerberos TGT should be renewed. This should 
be set to a value
+  that is shorter than the TGT renewal period (or the TGT lifetime if TGT 
renewal is not enabled).
+  The default value should be enough for most deployments.
+  </td>
+</tr>
+</table>
 
-Hadoop services issue *hadoop tokens* to grant access to the services and data.
-Clients must first acquire tokens for the services they will access and pass 
them along with their
-application as it is launched in the YARN cluster.
+## Troubleshooting Kerberos
 
-For a Spark application to interact with any of the Hadoop filesystem (for 
example hdfs, webhdfs, etc), HBase and Hive, it must acquire the relevant tokens
-using the Kerberos credentials of the user launching the application
-—that is, the principal whose identity will become that of the launched 
Spark application.
+Debugging Hadoop/Kerberos problems can be "difficult". One useful technique is 
to
+enable extra logging of Kerberos operations in Hadoop by setting the 
`HADOOP_JAAS_DEBUG`
+environment variable.
 
-This is normally done at launch time: in a secure cluster Spark will 
automatically obtain a
-token for the cluster's default Hadoop filesystem, and potentially for HBase 
and Hive.
+```bash
+export HADOOP_JAAS_DEBUG=true
+```
 
-An HBase token will be obtained if HBase is in on classpath, the HBase 
configuration declares
-the application is secure (i.e. `hbase-site.xml` sets 
`hbase.security.authentication` to `kerberos`),
-and `spark.security.credentials.hbase.enabled` is not set to `false`.
+The JDK classes can be configured to enable extra logging of their Kerberos and
+SPNEGO/REST authentication via the system properties `sun.security.krb5.debug`
+and `sun.security.spnego.debug=true`
 
-Similarly, a Hive token will be obtained if Hive is on the classpath, its 
configuration
-includes a URI of the metadata store in `"hive.metastore.uris`, and
-`spark.security.credentials.hive.enabled` is not set to `false`.
+```
+-Dsun.security.krb5.debug=true -Dsun.security.spnego.debug=true
+```
 
-If an application needs to interact with other secure Hadoop filesystems, then
-the tokens needed to access these clusters must be explicitly requested at
-launch time. This is done by listing them in the 
`spark.yarn.access.hadoopFileSystems` property.
+All these options can be enabled in the Application Master:
 
 ```
-spark.yarn.access.hadoopFileSystems 
hdfs://ireland.example.org:8020/,webhdfs://frankfurt.example.org:50070/
+spark.yarn.appMasterEnv.HADOOP_JAAS_DEBUG true
+spark.yarn.am.extraJavaOptions -Dsun.security.krb5.debug=true 
-Dsun.security.spnego.debug=true
 ```
 
-Spark supports integrating with other security-aware services through Java 
Services mechanism (see
-`java.util.ServiceLoader`). To do that, implementations of 
`org.apache.spark.deploy.yarn.security.ServiceCredentialProvider`
-should be available to Spark by listing their names in the corresponding file 
in the jar's
-`META-INF/services` directory. These plug-ins can be disabled by setting
-`spark.security.credentials.{service}.enabled` to `false`, where `{service}` 
is the name of
-credential provider.
+Finally, if the log level for `org.apache.spark.deploy.yarn.Client` is set to 
`DEBUG`, the log
+will include a list of all tokens obtained, and their expiry details
 
-## Configuring the External Shuffle Service
+
+# Configuring the External Shuffle Service
 
 To start the Spark Shuffle Service on each `NodeManager` in your YARN cluster, 
follow these
 instructions:
@@ -542,7 +551,7 @@ The following extra configuration options are available 
when the shuffle service
 </tr>
 </table>
 
-## Launching your application with Apache Oozie
+# Launching your application with Apache Oozie
 
 Apache Oozie can launch Spark applications as part of a workflow.
 In a secure cluster, the launched application will need the relevant tokens to 
access the cluster's
@@ -576,35 +585,7 @@ spark.security.credentials.hbase.enabled  false
 
 The configuration option `spark.yarn.access.hadoopFileSystems` must be unset.
 
-## Troubleshooting Kerberos
-
-Debugging Hadoop/Kerberos problems can be "difficult". One useful technique is 
to
-enable extra logging of Kerberos operations in Hadoop by setting the 
`HADOOP_JAAS_DEBUG`
-environment variable.
-
-```bash
-export HADOOP_JAAS_DEBUG=true
-```
-
-The JDK classes can be configured to enable extra logging of their Kerberos and
-SPNEGO/REST authentication via the system properties `sun.security.krb5.debug`
-and `sun.security.spnego.debug=true`
-
-```
--Dsun.security.krb5.debug=true -Dsun.security.spnego.debug=true
-```
-
-All these options can be enabled in the Application Master:
-
-```
-spark.yarn.appMasterEnv.HADOOP_JAAS_DEBUG true
-spark.yarn.am.extraJavaOptions -Dsun.security.krb5.debug=true 
-Dsun.security.spnego.debug=true
-```
-
-Finally, if the log level for `org.apache.spark.deploy.yarn.Client` is set to 
`DEBUG`, the log
-will include a list of all tokens obtained, and their expiry details
-
-## Using the Spark History Server to replace the Spark Web UI
+# Using the Spark History Server to replace the Spark Web UI
 
 It is possible to use the Spark History Server application page as the 
tracking URL for running
 applications when the application UI is disabled. This may be desirable on 
secure clusters, or to

http://git-wip-us.apache.org/repos/asf/spark/blob/b30a7d28/docs/security.md
----------------------------------------------------------------------
diff --git a/docs/security.md b/docs/security.md
index 913d9df..3e5607a 100644
--- a/docs/security.md
+++ b/docs/security.md
@@ -3,41 +3,323 @@ layout: global
 displayTitle: Spark Security
 title: Security
 ---
+* This will become a table of contents (this text will be scraped).
+{:toc}
 
-Spark currently supports authentication via a shared secret. Authentication 
can be configured to be on via the `spark.authenticate` configuration 
parameter. This parameter controls whether the Spark communication protocols do 
authentication using the shared secret. This authentication is a basic 
handshake to make sure both sides have the same shared secret and are allowed 
to communicate. If the shared secret is not identical they will not be allowed 
to communicate. The shared secret is created as follows:
+# Spark RPC
 
-* For Spark on [YARN](running-on-yarn.html) and local deployments, configuring 
`spark.authenticate` to `true` will automatically handle generating and 
distributing the shared secret. Each application will use a unique shared 
secret.
-* For other types of Spark deployments, the Spark parameter 
`spark.authenticate.secret` should be configured on each of the nodes. This 
secret will be used by all the Master/Workers and applications.
+## Authentication
 
-## Web UI
+Spark currently supports authentication for RPC channels using a shared 
secret. Authentication can
+be turned on by setting the `spark.authenticate` configuration parameter.
 
-The Spark UI can be secured by using [javax servlet 
filters](http://docs.oracle.com/javaee/6/api/javax/servlet/Filter.html) via the 
`spark.ui.filters` setting
-and by using [https/SSL](http://en.wikipedia.org/wiki/HTTPS) via [SSL 
settings](security.html#ssl-configuration).
+The exact mechanism used to generate and distribute the shared secret is 
deployment-specific.
 
-### Authentication
+For Spark on [YARN](running-on-yarn.html) and local deployments, Spark will 
automatically handle
+generating and distributing the shared secret. Each application will use a 
unique shared secret. In
+the case of YARN, this feature relies on YARN RPC encryption being enabled for 
the distribution of
+secrets to be secure.
 
-A user may want to secure the UI if it has data that other users should not be 
allowed to see. The javax servlet filter specified by the user can authenticate 
the user and then once the user is logged in, Spark can compare that user 
versus the view ACLs to make sure they are authorized to view the UI. The 
configs `spark.acls.enable`, `spark.ui.view.acls` and 
`spark.ui.view.acls.groups` control the behavior of the ACLs. Note that the 
user who started the application always has view access to the UI.  On YARN, 
the Spark UI uses the standard YARN web application proxy mechanism and will 
authenticate via any installed Hadoop filters.
+For other resource managers, `spark.authenticate.secret` must be configured on 
each of the nodes.
+This secret will be shared by all the daemons and applications, so this 
deployment configuration is
+not as secure as the above, especially when considering multi-tenant clusters.
 
-Spark also supports modify ACLs to control who has access to modify a running 
Spark application. This includes things like killing the application or a task. 
This is controlled by the configs `spark.acls.enable`, `spark.modify.acls` and 
`spark.modify.acls.groups`. Note that if you are authenticating the web UI, in 
order to use the kill button on the web UI it might be necessary to add the 
users in the modify acls to the view acls also. On YARN, the modify acls are 
passed in and control who has modify access via YARN interfaces.
-Spark allows for a set of administrators to be specified in the acls who 
always have view and modify permissions to all the applications. is controlled 
by the configs `spark.admin.acls` and `spark.admin.acls.groups`. This is useful 
on a shared cluster where you might have administrators or support staff who 
help users debug applications.
+<table class="table">
+<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+<tr>
+  <td><code>spark.authenticate</code></td>
+  <td>false</td>
+  <td>Whether Spark authenticates its internal connections.</td>
+</tr>
+<tr>
+  <td><code>spark.authenticate.secret</code></td>
+  <td>None</td>
+  <td>
+    The secret key used authentication. See above for when this configuration 
should be set.
+  </td>
+</tr>
+</table>
+
+## Encryption
 
-## Event Logging
+Spark supports AES-based encryption for RPC connections. For encryption to be 
enabled, RPC
+authentication must also be enabled and properly configured. AES encryption 
uses the
+[Apache Commons Crypto](http://commons.apache.org/proper/commons-crypto/) 
library, and Spark's
+configuration system allows access to that library's configuration for 
advanced users.
 
-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.
+There is also support for SASL-based encryption, although it should be 
considered deprecated. It
+is still required when talking to shuffle services from Spark versions older 
than 2.2.0.
 
-## Encryption
+The following table describes the different options available for configuring 
this feature.
+
+<table class="table">
+<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+<tr>
+  <td><code>spark.network.crypto.enabled</code></td>
+  <td>false</td>
+  <td>
+    Enable AES-based RPC encryption, including the new authentication protocol 
added in 2.2.0.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.network.crypto.keyLength</code></td>
+  <td>128</td>
+  <td>
+    The length in bits of the encryption key to generate. Valid values are 
128, 192 and 256.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.network.crypto.keyFactoryAlgorithm</code></td>
+  <td>PBKDF2WithHmacSHA1</td>
+  <td>
+    The key factory algorithm to use when generating encryption keys. Should 
be one of the
+    algorithms supported by the javax.crypto.SecretKeyFactory class in the JRE 
being used.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.network.crypto.config.*</code></td>
+  <td>None</td>
+  <td>
+    Configuration values for the commons-crypto library, such as which cipher 
implementations to
+    use. The config name should be the name of commons-crypto configuration 
without the
+    <code>commons.crypto</code> prefix.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.network.crypto.saslFallback</code></td>
+  <td>true</td>
+  <td>
+    Whether to fall back to SASL authentication if authentication fails using 
Spark's internal
+    mechanism. This is useful when the application is connecting to old 
shuffle services that
+    do not support the internal Spark authentication protocol. On the shuffle 
service side,
+    disabling this feature will block older clients from authenticating.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.authenticate.enableSaslEncryption</code></td>
+  <td>false</td>
+  <td>
+    Enable SASL-based encrypted communication.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.network.sasl.serverAlwaysEncrypt</code></td>
+  <td>false</td>
+  <td>
+    Disable unencrypted connections for ports using SASL authentication. This 
will deny connections
+    from clients that have authentication enabled, but do not request 
SASL-based encryption.
+  </td>
+</tr>
+</table>
+
+
+# Local Storage Encryption
+
+Spark supports encrypting temporary data written to local disks. This covers 
shuffle files, shuffle
+spills and data blocks stored on disk (for both caching and broadcast 
variables). It does not cover
+encrypting output data generated by applications with APIs such as 
`saveAsHadoopFile` or
+`saveAsTable`.
+
+The following settings cover enabling encryption for data written to disk:
+
+<table class="table">
+<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+<tr>
+  <td><code>spark.io.encryption.enabled</code></td>
+  <td>false</td>
+  <td>
+    Enable local disk I/O encryption. Currently supported by all modes except 
Mesos. It's strongly
+    recommended that RPC encryption be enabled when using this feature.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.io.encryption.keySizeBits</code></td>
+  <td>128</td>
+  <td>
+    IO encryption key size in bits. Supported values are 128, 192 and 256.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.io.encryption.keygen.algorithm</code></td>
+  <td>HmacSHA1</td>
+  <td>
+    The algorithm to use when generating the IO encryption key. The supported 
algorithms are
+    described in the KeyGenerator section of the Java Cryptography 
Architecture Standard Algorithm
+    Name Documentation.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.io.encryption.commons.config.*</code></td>
+  <td>None</td>
+  <td>
+    Configuration values for the commons-crypto library, such as which cipher 
implementations to
+    use. The config name should be the name of commons-crypto configuration 
without the
+    <code>commons.crypto</code> prefix.
+  </td>
+</tr>
+</table>
+
+
+# Web UI
+
+## Authentication and Authorization
+
+Enabling authentication for the Web UIs is done using [javax servlet 
filters](http://docs.oracle.com/javaee/6/api/javax/servlet/Filter.html).
+You will need a filter that implements the authentication method you want to 
deploy. Spark does not
+provide any built-in authentication filters.
+
+Spark also supports access control to the UI when an authentication filter is 
present. Each
+application can be configured with its own separate access control lists 
(ACLs). Spark
+differentiates between "view" permissions (who is allowed to see the 
application's UI), and "modify"
+permissions (who can do things like kill jobs in a running application).
+
+ACLs can be configured for either users or groups. Configuration entries 
accept comma-separated
+lists as input, meaning multiple users or groups can be given the desired 
privileges. This can be
+used if you run on a shared cluster and have a set of administrators or 
developers who need to
+monitor applications they may not have started themselves. A wildcard (`*`) 
added to specific ACL
+means that all users will have the respective pivilege. By default, only the 
user submitting the
+application is added to the ACLs.
+
+Group membership is established by using a configurable group mapping 
provider. The mapper is
+configured using the <code>spark.user.groups.mapping</code> config option, 
described in the table
+below.
+
+The following options control the authentication of Web UIs:
+
+<table class="table">
+<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+<tr>
+  <td><code>spark.ui.filters</code></td>
+  <td>None</td>
+  <td>
+    See the <a href="configuration.html#spark-ui">Spark UI</a> configuration 
for how to configure
+    filters.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.acls.enable</code></td>
+  <td>false</td>
+  <td>
+    Whether UI ACLs should be enabled. If enabled, this checks to see if the 
user has access
+    permissions to view or modify the application. Note this requires the user 
to be authenticated,
+    so if no authentication filter is installed, this option does not do 
anything.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.admin.acls</code></td>
+  <td>None</td>
+  <td>
+    Comma-separated list of users that have view and modify access to the 
Spark application.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.admin.acls.groups</code></td>
+  <td>None</td>
+  <td>
+    Comma-separated list of groups that have view and modify access to the 
Spark application.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.modify.acls</code></td>
+  <td>None</td>
+  <td>
+    Comma-separated list of users that have modify access to the Spark 
application.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.modify.acls.groups</code></td>
+  <td>None</td>
+  <td>
+    Comma-separated list of groups that have modify access to the Spark 
application.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.ui.view.acls</code></td>
+  <td>None</td>
+  <td>
+    Comma-separated list of users that have view access to the Spark 
application.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.ui.view.acls.groups</code></td>
+  <td>None</td>
+  <td>
+    Comma-separated list of groups that have view access to the Spark 
application.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.user.groups.mapping</code></td>
+  
<td><code>org.apache.spark.security.ShellBasedGroupsMappingProvider</code></td>
+  <td>
+    The list of groups for a user is determined by a group mapping service 
defined by the trait
+    <code>org.apache.spark.security.GroupMappingServiceProvider</code>, which 
can be configured by
+    this property.
+
+    <br />By default, a Unix shell-based implementation is used, which 
collects this information
+    from the host OS.
+
+    <br /><em>Note:</em> This implementation supports only Unix/Linux-based 
environments.
+    Windows environment is currently <b>not</b> supported. However, a new 
platform/protocol can
+    be supported by implementing the trait mentioned above.
+  </td>
+</tr>
+</table>
+
+On YARN, the view and modify ACLs are provided to the YARN service when 
submitting applications, and
+control who has the respective privileges via YARN interfaces.
+
+## Spark History Server ACLs
 
-Spark supports SSL for HTTP protocols. SASL encryption is supported for the 
block transfer service
-and the RPC endpoints. Shuffle files can also be encrypted if desired.
+Authentication for the SHS Web UI is enabled the same way as for regular 
applications, using
+servlet filters.
 
-### SSL Configuration
+To enable authorization in the SHS, a few extra options are used:
+
+<table class="table">
+<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+<tr>
+  <td>spark.history.ui.acls.enable</td>
+  <td>false</td>
+  <td>
+    Specifies whether ACLs should be checked to authorize users viewing the 
applications in
+    the history server. If enabled, access control checks are performed 
regardless of what the
+    individual applications had set for <code>spark.ui.acls.enable</code>. The 
application owner
+    will always have authorization to view their own application and any users 
specified via
+    <code>spark.ui.view.acls</code> and groups specified via 
<code>spark.ui.view.acls.groups</code>
+    when the application was run will also have authorization to view that 
application.
+    If disabled, no access control checks are made for any application UIs 
available through
+    the history server.
+  </td>
+</tr>
+<tr>
+  <td>spark.history.ui.admin.acls</td>
+  <td>None</td>
+  <td>
+    Comma separated list of users that have view access to all the Spark 
applications in history
+    server.
+  </td>
+</tr>
+<tr>
+  <td>spark.history.ui.admin.acls.groups</td>
+  <td>None</td>
+  <td>
+    Comma separated list of groups that have view access to all the Spark 
applications in history
+    server.
+  </td>
+</tr>
+</table>
+
+The SHS uses the same options to configure the group mapping provider as 
regular applications.
+In this case, the group mapping provider will apply to all UIs server by the 
SHS, and individual
+application configurations will be ignored.
+
+## SSL Configuration
 
 Configuration for SSL 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. The following table 
describes the
-component-specific configuration namespaces used to override the default 
settings:
+protocols without disabling the ability to configure each one individually. 
The following table
+describes the the SSL configuration namespaces:
 
 <table class="table">
   <tr>
@@ -45,6 +327,13 @@ component-specific configuration namespaces used to 
override the default setting
     <th>Component</th>
   </tr>
   <tr>
+    <td><code>spark.ssl</code></td>
+    <td>
+      The default SSL configuration. These values will apply to all namespaces 
below, unless
+      explicitly overridden at the namespace level.
+    </td>
+  </tr>
+  <tr>
     <td><code>spark.ssl.ui</code></td>
     <td>Spark application Web UI</td>
   </tr>
@@ -58,49 +347,205 @@ component-specific configuration namespaces used to 
override the default setting
   </tr>
 </table>
 
-The full breakdown of available SSL options 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.
+The full breakdown of available SSL options can be found below. The `${ns}` 
placeholder should be
+replaced with one of the above namespaces.
+
+<table class="table">
+<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+  <tr>
+    <td><code>${ns}.enabled</code></td>
+    <td>false</td>
+    <td>Enables SSL. When enabled, <code>${ns}.ssl.protocol</code> is 
required.</td>
+  </tr>
+  <tr>
+    <td><code>${ns}.port</code></td>
+    <td>None</td>
+    <td>
+      The port where the SSL service will listen on.
+
+      <br />The port must be defined within a specific namespace 
configuration. The default
+      namespace is ignored when reading this configuration.
+
+      <br />When not set, the SSL port will be derived from the non-SSL port 
for the
+      same service. A value of "0" will make the service bind to an ephemeral 
port.
+    </td>
+  </tr>
+  <tr>
+    <td><code>${ns}.enabledAlgorithms</code></td>
+    <td>None</td>
+    <td>
+      A comma separated list of ciphers. The specified ciphers must be 
supported by JVM.
+
+      <br />The reference list of protocols can be found in the "JSSE Cipher 
Suite Names" section
+      of the Java security guide. The list for Java 8 can be found at
+      <a 
href="https://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#ciphersuites";>this</a>
+      page.
+
+      <br />Note: If not set, the default cipher suite for the JRE will be 
used.
+    </td>
+  </tr>
+  <tr>
+    <td><code>${ns}.keyPassword</code></td>
+    <td>None</td>
+    <td>
+      The password to the private key in the key store.
+    </td>
+  </tr>
+  <tr>
+    <td><code>${ns}.keyStore</code></td>
+    <td>None</td>
+    <td>
+      Path to the key store file. The path can be absolute or relative to the 
directory in which the
+      process is started.
+    </td>
+  </tr>
+  <tr>
+    <td><code>${ns}.keyStorePassword</code></td>
+    <td>None</td>
+    <td>Password to the key store.</td>
+  </tr>
+  <tr>
+    <td><code>${ns}.keyStoreType</code></td>
+    <td>JKS</td>
+    <td>The type of the key store.</td>
+  </tr>
+  <tr>
+    <td><code>${ns}.protocol</code></td>
+    <td>None</td>
+    <td>
+      TLS protocol to use. The protocol must be supported by JVM.
+
+      <br />The reference list of protocols can be found in the "Additional 
JSSE Standard Names"
+      section of the Java security guide. For Java 8, the list can be found at
+      <a 
href="https://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#jssenames";>this</a>
+      page.
+    </td>
+  </tr>
+  <tr>
+    <td><code>${ns}.needClientAuth</code></td>
+    <td>false</td>
+    <td>Whether to require client authentication.</td>
+  </tr>
+  <tr>
+    <td><code>${ns}.trustStore</code></td>
+    <td>None</td>
+    <td>
+      Path to the trust store file. The path can be absolute or relative to 
the directory in which
+      the process is started.
+    </td>
+  </tr>
+  <tr>
+    <td><code>${ns}.trustStorePassword</code></td>
+    <td>None</td>
+    <td>Password for the trust store.</td>
+  </tr>
+  <tr>
+    <td><code>${ns}.trustStoreType</code></td>
+    <td>JKS</td>
+    <td>The type of the trust store.</td>
+  </tr>
+</table>
+
+## Preparing the key stores
+
+Key stores can be generated by `keytool` program. The reference documentation 
for this tool for
+Java 8 is 
[here](https://docs.oracle.com/javase/8/docs/technotes/tools/unix/keytool.html).
+The most basic steps to configure the key stores and the trust store for a 
Spark Standalone
+deployment mode is as follows:
+
+* Generate a key 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 to the cluster nodes
 
 ### 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.
 
-For long-running apps like Spark Streaming apps to be able to write to HDFS, 
it is possible to pass a principal and keytab to `spark-submit` via the 
`--principal` and `--keytab` parameters respectively. The keytab passed in will 
be copied over to the machine running the Application Master via the Hadoop 
Distributed Cache (securely - if YARN is configured with SSL and HDFS 
encryption is enabled). The Kerberos login will be periodically renewed using 
this principal and keytab and the delegation tokens required for HDFS will be 
generated periodically so the application can continue writing to HDFS.
+To provide a local trust store or key store file to drivers running in cluster 
mode, they can be
+distributed with the application using the `--files` command line argument (or 
the equivalent
+`spark.files` configuration). The files will be placed on the driver's working 
directory, so the TLS
+configuration should just reference the file name with no absolute path.
+
+Distributing local key stores this way may require the files to be staged in 
HDFS (or other similar
+distributed file system used by the cluster), so it's recommended that the 
undelying file system be
+configured with security in mind (e.g. by enabling authentication and wire 
encryption).
 
 ### 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.
+
+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`.
+
+The user may allow the executors to use the SSL settings inherited from the 
worker process. That
+can be accomplished by setting `spark.ssl.useNodeLocalConf` to `true`. In that 
case, the settings
+provided by the user on the client side are not used.
 
 ### Mesos mode
-Mesos 1.3.0 and newer supports `Secrets` primitives as both file-based and 
environment based secrets. Spark allows the specification of file-based and 
environment variable based secrets with the 
`spark.mesos.driver.secret.filenames` and `spark.mesos.driver.secret.envkeys`, 
respectively. Depending on the secret store backend secrets can be passed by 
reference or by value with the `spark.mesos.driver.secret.names` and 
`spark.mesos.driver.secret.values` configuration properties, respectively. 
Reference type secrets are served by the secret store and referred to by name, 
for example `/mysecret`. Value type secrets are passed on the command line and 
translated into their appropriate files or environment variables. 
+Mesos 1.3.0 and newer supports `Secrets` primitives as both file-based and 
environment based
+secrets. Spark allows the specification of file-based and environment variable 
based secrets with
+`spark.mesos.driver.secret.filenames` and `spark.mesos.driver.secret.envkeys`, 
respectively.
 
-### 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:
+Depending on the secret store backend secrets can be passed by reference or by 
value with the
+`spark.mesos.driver.secret.names` and `spark.mesos.driver.secret.values` 
configuration properties,
+respectively.
 
-* 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
+Reference type secrets are served by the secret store and referred to by name, 
for example
+`/mysecret`. Value type secrets are passed on the command line and translated 
into their
+appropriate files or environment variables.
 
-### Configuring SASL Encryption
+## HTTP Security Headers
 
-SASL encryption is currently supported for the block transfer service when 
authentication
-(`spark.authenticate`) is enabled. To enable SASL encryption for an 
application, set
-`spark.authenticate.enableSaslEncryption` to `true` in the application's 
configuration.
+Apache Spark can be configured to include HTTP headers to aid in preventing 
Cross Site Scripting
+(XSS), Cross-Frame Scripting (XFS), MIME-Sniffing, and also to enforce HTTP 
Strict Transport
+Security.
 
-When using an external shuffle service, it's possible to disable unencrypted 
connections by setting
-`spark.network.sasl.serverAlwaysEncrypt` to `true` in the shuffle service's 
configuration. If that
-option is enabled, applications that are not set up to use SASL encryption 
will fail to connect to
-the shuffle service.
+<table class="table">
+<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+<tr>
+  <td><code>spark.ui.xXssProtection</code></td>
+  <td><code>1; mode=block</code></td>
+  <td>
+    Value for HTTP X-XSS-Protection response header. You can choose 
appropriate value
+    from below:
+    <ul>
+      <li><code>0</code> (Disables XSS filtering)</li>
+      <li><code>1</code> (Enables XSS filtering. If a cross-site scripting 
attack is detected,
+        the browser will sanitize the page.)</li>
+      <li><code>1; mode=block</code> (Enables XSS filtering. The browser will 
prevent rendering
+        of the page if an attack is detected.)</li>
+    </ul>
+  </td>
+</tr>
+<tr>
+  <td><code>spark.ui.xContentTypeOptions.enabled</code></td>
+  <td><code>true</code></td>
+  <td>
+    When enabled, X-Content-Type-Options HTTP response header will be set to 
"nosniff".
+  </td>
+  </tr>
+<tr>
+  <td><code>spark.ui.strictTransportSecurity</code></td>
+  <td>None</td>
+  <td>
+    Value for HTTP Strict Transport Security (HSTS) Response Header. You can 
choose appropriate
+    value from below and set <code>expire-time</code> accordingly. This option 
is only used when
+    SSL/TLS is enabled.
+    <ul>
+      <li><code>max-age=&lt;expire-time&gt;</code></li>
+      <li><code>max-age=&lt;expire-time&gt;; includeSubDomains</code></li>
+      <li><code>max-age=&lt;expire-time&gt;; preload</code></li>
+    </ul>
+  </td>
+</tr>
+</table>
 
-## Configuring Ports for Network Security
+
+# Configuring Ports for Network Security
 
 Spark makes heavy use of the network, and some environments have strict 
requirements for using tight
 firewall settings.  Below are the primary ports that Spark uses for its 
communication and how to
 configure those ports.
 
-### Standalone mode only
+## Standalone mode only
 
 <table class="table">
   <tr>
@@ -141,7 +586,7 @@ configure those ports.
   </tr>
 </table>
 
-### All cluster managers
+## All cluster managers
 
 <table class="table">
   <tr>
@@ -182,54 +627,70 @@ configure those ports.
   </tr>
 </table>
 
-### HTTP Security Headers
 
-Apache Spark can be configured to include HTTP Headers which aids in 
preventing Cross 
-Site Scripting (XSS), Cross-Frame Scripting (XFS), MIME-Sniffing and also 
enforces HTTP 
-Strict Transport Security.
+# Kerberos
+
+Spark supports submitting applications in environments that use Kerberos for 
authentication.
+In most cases, Spark relies on the credentials of the current logged in user 
when authenticating
+to Kerberos-aware services. Such credentials can be obtained by logging in to 
the configured KDC
+with tools like `kinit`.
+
+When talking to Hadoop-based services, Spark needs to obtain delegation tokens 
so that non-local
+processes can authenticate. Spark ships with support for HDFS and other Hadoop 
file systems, Hive
+and HBase.
+
+When using a Hadoop filesystem (such HDFS or WebHDFS), Spark will acquire the 
relevant tokens
+for the service hosting the user's home directory.
+
+An HBase token will be obtained if HBase is in the application's classpath, 
and the HBase
+configuration has Kerberos authentication turned 
(`hbase.security.authentication=kerberos`).
+
+Similarly, a Hive token will be obtained if Hive is in the classpath, and the 
configuration includes
+URIs for remote metastore services (`hive.metastore.uris` is not empty).
+
+Delegation token support is currently only supported in YARN and Mesos modes. 
Consult the
+deployment-specific page for more information.
+
+The following options provides finer-grained control for this feature:
 
 <table class="table">
 <tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
 <tr>
-  <td><code>spark.ui.xXssProtection</code></td>
-  <td><code>1; mode=block</code></td>
-  <td>
-    Value for HTTP X-XSS-Protection response header. You can choose 
appropriate value 
-    from below:
-    <ul>
-      <li><code>0</code> (Disables XSS filtering)</li> 
-      <li><code>1</code> (Enables XSS filtering. If a cross-site scripting 
attack is detected, 
-        the browser will sanitize the page.)</li>
-      <li><code>1; mode=block</code> (Enables XSS filtering. The browser will 
prevent rendering 
-        of the page if an attack is detected.)</li>
-    </ul>
-  </td>
-</tr>
-<tr>
-  <td><code>spark.ui.xContentTypeOptions.enabled</code></td>
+  <td><code>spark.security.credentials.${service}.enabled</code></td>
   <td><code>true</code></td>
   <td>
-    When value is set to "true", X-Content-Type-Options HTTP response header 
will be set 
-    to "nosniff". Set "false" to disable.
-  </td>
-  </tr>
-<tr>
-  <td><code>spark.ui.strictTransportSecurity</code></td>
-  <td>None</td>
-  <td>
-    Value for HTTP Strict Transport Security (HSTS) Response Header. You can 
choose appropriate 
-    value from below and set <code>expire-time</code> accordingly, when Spark 
is SSL/TLS enabled.
-    <ul>
-      <li><code>max-age=&lt;expire-time&gt;</code></li>
-      <li><code>max-age=&lt;expire-time&gt;; includeSubDomains</code></li>
-      <li><code>max-age=&lt;expire-time&gt;; preload</code></li>
-    </ul>
+  Controls whether to obtain credentials for services when security is enabled.
+  By default, credentials for all supported services are retrieved when those 
services are
+  configured, but it's possible to disable that behavior if it somehow 
conflicts with the
+  application being run.
   </td>
 </tr>
 </table>
-    
 
-See the [configuration page](configuration.html) for more details on the 
security configuration
-parameters, and <a 
href="{{site.SPARK_GITHUB_URL}}/tree/master/core/src/main/scala/org/apache/spark/SecurityManager.scala">
-<code>org.apache.spark.SecurityManager</code></a> for implementation details 
about security.
+## Long-Running Applications
+
+Long-running applications may run into issues if their run time exceeds the 
maximum delegation
+token lifetime configured in services it needs to access.
+
+Spark supports automatically creating new tokens for these applications when 
running in YARN mode.
+Kerberos credentials need to be provided to the Spark application via the 
`spark-submit` command,
+using the `--principal` and `--keytab` parameters.
+
+The provided keytab will be copied over to the machine running the Application 
Master via the Hadoop
+Distributed Cache. For this reason, it's strongly recommended that both YARN 
and HDFS be secured
+with encryption, at least.
+
+The Kerberos login will be periodically renewed using the provided 
credentials, and new delegation
+tokens for supported will be created.
+
+
+# Event Logging
+
+If your applications are using event logging, the directory where the event 
logs go
+(`spark.eventLog.dir`) should be manually created with proper permissions. To 
secure the log files,
+the directory permissions should be set to `drwxrwxrwxt`. The owner and group 
of the directory
+should correspond to the super user who is running the Spark History Server.
 
+This will allow all users to write to the directory but will prevent 
unprivileged users from
+reading, removing or renaming a file unless they own it. The event log files 
will be created by
+Spark with permissions such that only the user and group have read and write 
access.


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

Reply via email to