layout | displayTitle | title |
---|---|---|
global |
Spark Security |
Security |
- This will become a table of contents (this text will be scraped). {:toc}
Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. Spark supports multiple deployments types and each one supports different levels of security. Not all deployment types will be secure in all environments and none are secure by default. Be sure to evaluate your environment, what Spark supports, and take the appropriate measure to secure your Spark deployment.
There are many different types of security concerns. Spark does not necessarily protect against all things. Listed below are some of the things Spark supports. Also check the deployment documentation for the type of deployment you are using for deployment specific settings. Anything not documented, Spark does not support.
Spark currently supports authentication for RPC channels using a shared secret. Authentication can
be turned on by setting the spark.authenticate
configuration parameter.
The exact mechanism used to generate and distribute the shared secret is deployment-specific. Unless
specified below, the secret must be defined by setting the spark.authenticate.secret
config
option. The same secret is shared by all Spark applications and daemons in that case, which limits
the security of these deployments, especially on multi-tenant clusters.
The REST Submission Server and the MesosClusterDispatcher do not support authentication. You should ensure that all network access to the REST API & MesosClusterDispatcher (port 6066 and 7077 respectively by default) are restricted to hosts that are trusted to submit jobs.
For Spark on YARN, 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.
On Kubernetes, Spark will also automatically generate an authentication secret unique to each application. The secret is propagated to executor pods using environment variables. This means that any user that can list pods in the namespace where the Spark application is running can also see their authentication secret. Access control rules should be properly set up by the Kubernetes admin to ensure that Spark authentication is secure.
Property Name | Default | Meaning |
---|---|---|
spark.authenticate |
false | Whether Spark authenticates its internal connections. |
spark.authenticate.secret |
None | The secret key used authentication. See above for when this configuration should be set. |
Alternatively, one can mount authentication secrets using files and Kubernetes secrets that the user mounts into their pods.
Property Name | Default | Meaning |
---|---|---|
spark.authenticate.secret.file |
None | Path pointing to the secret key to use for securing connections. Ensure that the contents of the file have been securely generated. This file is loaded on both the driver and the executors unless other settings override this (see below). |
spark.authenticate.secret.driver.file |
The value of spark.authenticate.secret.file |
When specified, overrides the location that the Spark driver reads to load the secret.
Useful when in client mode, when the location of the secret file may differ in the pod versus
the node the driver is running in. When this is specified,
spark.authenticate.secret.executor.file must be specified so that the driver
and the executors can both use files to load the secret key. Ensure that the contents of the file
on the driver is identical to the contents of the file on the executors.
|
spark.authenticate.secret.executor.file |
The value of spark.authenticate.secret.file |
When specified, overrides the location that the Spark executors read to load the secret.
Useful in client mode, when the location of the secret file may differ in the pod versus
the node the driver is running in. When this is specified,
spark.authenticate.secret.driver.file must be specified so that the driver
and the executors can both use files to load the secret key. Ensure that the contents of the file
on the driver is identical to the contents of the file on the executors.
|
Note that when using files, Spark will not mount these files into the containers for you. It is up you to ensure that the secret files are deployed securely into your containers and that the driver's secret file agrees with the executors' secret file.
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 library, and Spark's configuration system allows access to that library's configuration for advanced users.
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.
The following table describes the different options available for configuring this feature.
Property Name | Default | Meaning |
---|---|---|
spark.network.crypto.enabled |
false | Enable AES-based RPC encryption, including the new authentication protocol added in 2.2.0. |
spark.network.crypto.keyLength |
128 | The length in bits of the encryption key to generate. Valid values are 128, 192 and 256. |
spark.network.crypto.keyFactoryAlgorithm |
PBKDF2WithHmacSHA1 | 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. |
spark.network.crypto.config.* |
None |
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.
|
spark.network.crypto.saslFallback |
true | 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. |
spark.authenticate.enableSaslEncryption |
false | Enable SASL-based encrypted communication. |
spark.network.sasl.serverAlwaysEncrypt |
false | 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. |
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
. It also may not cover temporary files created explicitly by the user.
The following settings cover enabling encryption for data written to disk:
Property Name | Default | Meaning |
---|---|---|
spark.io.encryption.enabled |
false | 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. |
spark.io.encryption.keySizeBits |
128 | IO encryption key size in bits. Supported values are 128, 192 and 256. |
spark.io.encryption.keygen.algorithm |
HmacSHA1 | 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. |
spark.io.encryption.commons.config.* |
None |
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.
|
Enabling authentication for the Web UIs is done using javax servlet filters. 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 privilege. 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 spark.user.groups.mapping
config option, described in the table
below.
The following options control the authentication of Web UIs:
Property Name | Default | Meaning |
---|---|---|
spark.ui.filters |
None | See the Spark UI configuration for how to configure filters. |
spark.acls.enable |
false | 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. |
spark.admin.acls |
None | Comma-separated list of users that have view and modify access to the Spark application. |
spark.admin.acls.groups |
None | Comma-separated list of groups that have view and modify access to the Spark application. |
spark.modify.acls |
None | Comma-separated list of users that have modify access to the Spark application. |
spark.modify.acls.groups |
None | Comma-separated list of groups that have modify access to the Spark application. |
spark.ui.view.acls |
None | Comma-separated list of users that have view access to the Spark application. |
spark.ui.view.acls.groups |
None | Comma-separated list of groups that have view access to the Spark application. |
spark.user.groups.mapping |
org.apache.spark.security.ShellBasedGroupsMappingProvider |
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.
|
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.
Authentication for the SHS Web UI is enabled the same way as for regular applications, using servlet filters.
To enable authorization in the SHS, a few extra options are used:
Property Name | Default | Meaning |
---|---|---|
spark.history.ui.acls.enable |
false |
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 spark.ui.acls.enable . The application owner
will always have authorization to view their own application and any users specified via
spark.ui.view.acls and groups specified via spark.ui.view.acls.groups
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.
|
spark.history.ui.admin.acls |
None | Comma separated list of users that have view access to all the Spark applications in history server. |
spark.history.ui.admin.acls.groups |
None | Comma separated list of groups that have view access to all the Spark applications in history server. |
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.
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 following table describes the SSL configuration namespaces:
Config Namespace | Component |
---|---|
spark.ssl |
The default SSL configuration. These values will apply to all namespaces below, unless explicitly overridden at the namespace level. |
spark.ssl.ui |
Spark application Web UI |
spark.ssl.standalone |
Standalone Master / Worker Web UI |
spark.ssl.historyServer |
History Server Web UI |
The full breakdown of available SSL options can be found below. The ${ns}
placeholder should be
replaced with one of the above namespaces.
Property Name | Default | Meaning |
---|---|---|
${ns}.enabled |
false | Enables SSL. When enabled, ${ns}.ssl.protocol is required. |
${ns}.port |
None |
The port where the SSL service will listen on.
|
${ns}.enabledAlgorithms |
None |
A comma-separated list of ciphers. The specified ciphers must be supported by JVM.
|
${ns}.keyPassword |
None | The password to the private key in the key store. |
${ns}.keyStore |
None | Path to the key store file. The path can be absolute or relative to the directory in which the process is started. |
${ns}.keyStorePassword |
None | Password to the key store. |
${ns}.keyStoreType |
JKS | The type of the key store. |
${ns}.protocol |
None |
TLS protocol to use. The protocol must be supported by JVM.
|
${ns}.needClientAuth |
false | Whether to require client authentication. |
${ns}.trustStore |
None | Path to the trust store file. The path can be absolute or relative to the directory in which the process is started. |
${ns}.trustStorePassword |
None | Password for the trust store. |
${ns}.trustStoreType |
JKS | The type of the trust store. |
Spark also supports retrieving ${ns}.keyPassword
, ${ns}.keyStorePassword
and ${ns}.trustStorePassword
from
Hadoop Credential Providers.
User could store password into credential file and make it accessible by different components, like:
hadoop credential create spark.ssl.keyPassword -value password \
-provider jceks://hdfs@nn1.example.com:9001/user/backup/ssl.jceks
To configure the location of the credential provider, set the hadoop.security.credential.provider.path
config option in the Hadoop configuration used by Spark, like:
<property>
<name>hadoop.security.credential.provider.path</name>
<value>jceks://hdfs@nn1.example.com:9001/user/backup/ssl.jceks</value>
</property>
Or via SparkConf "spark.hadoop.hadoop.security.credential.provider.path=jceks://hdfs@nn1.example.com:9001/user/backup/ssl.jceks".
Key stores can be generated by keytool
program. The reference documentation for this tool for
Java 8 is here.
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
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 underlying file system be configured with security in mind (e.g. by enabling authentication and wire encryption).
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 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.
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.
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.
Property Name | Default | Meaning |
---|---|---|
spark.ui.xXssProtection |
1; mode=block |
Value for HTTP X-XSS-Protection response header. You can choose appropriate value
from below:
|
spark.ui.xContentTypeOptions.enabled |
true |
When enabled, X-Content-Type-Options HTTP response header will be set to "nosniff". |
spark.ui.strictTransportSecurity |
None |
Value for HTTP Strict Transport Security (HSTS) Response Header. You can choose appropriate
value from below and set expire-time accordingly. This option is only used when
SSL/TLS is enabled.
|
Generally speaking, a Spark cluster and its services are not deployed on the public internet. They are generally private services, and should only be accessible within the network of the organization that deploys Spark. Access to the hosts and ports used by Spark services should be limited to origin hosts that need to access the services.
Below are the primary ports that Spark uses for its communication and how to configure those ports.
From | To | Default Port | Purpose | Configuration Setting | Notes |
---|---|---|---|---|---|
Browser | Standalone Master | 8080 | Web UI | spark.master.ui.port / |
Jetty-based. Standalone mode only. |
Browser | Standalone Worker | 8081 | Web UI | spark.worker.ui.port / |
Jetty-based. Standalone mode only. |
Driver / Standalone Worker |
Standalone Master | 7077 | Submit job to cluster / Join cluster |
SPARK_MASTER_PORT |
Set to "0" to choose a port randomly. Standalone mode only. |
External Service | Standalone Master | 6066 | Submit job to cluster via REST API | spark.master.rest.port |
Use spark.master.rest.enabled to enable/disable this service. Standalone mode only. |
Standalone Master | Standalone Worker | (random) | Schedule executors | SPARK_WORKER_PORT |
Set to "0" to choose a port randomly. Standalone mode only. |
From | To | Default Port | Purpose | Configuration Setting | Notes |
---|---|---|---|---|---|
Browser | Application | 4040 | Web UI | spark.ui.port |
Jetty-based |
Browser | History Server | 18080 | Web UI | spark.history.ui.port |
Jetty-based |
Executor / Standalone Master |
Driver | (random) | Connect to application / Notify executor state changes |
spark.driver.port |
Set to "0" to choose a port randomly. |
Executor / Driver | Executor / Driver | (random) | Block Manager port | spark.blockManager.port |
Raw socket via ServerSocketChannel |
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:
Property Name | Default | Meaning |
---|---|---|
spark.security.credentials.${service}.enabled |
true |
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. |
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.
When talking to Hadoop-based services behind Kerberos, it was noted that Spark needs to obtain delegation tokens so that non-local processes can authenticate. These delegation tokens in Kubernetes are stored in Secrets that are shared by the Driver and its Executors. As such, there are three ways of submitting a Kerberos job:
In all cases you must define the environment variable: HADOOP_CONF_DIR
or
spark.kubernetes.hadoop.configMapName.
It also important to note that the KDC needs to be visible from inside the containers.
If a user wishes to use a remote HADOOP_CONF directory, that contains the Hadoop configuration files, this could be
achieved by setting spark.kubernetes.hadoop.configMapName
to a pre-existing ConfigMap.
- Submitting with a $kinit that stores a TGT in the Local Ticket Cache:
/usr/bin/kinit -kt <keytab_file> <username>/<krb5 realm>
/opt/spark/bin/spark-submit \
--deploy-mode cluster \
--class org.apache.spark.examples.HdfsTest \
--master k8s://<KUBERNETES_MASTER_ENDPOINT> \
--conf spark.executor.instances=1 \
--conf spark.app.name=spark-hdfs \
--conf spark.kubernetes.container.image=spark:latest \
--conf spark.kubernetes.kerberos.krb5.path=/etc/krb5.conf \
local:///opt/spark/examples/jars/spark-examples_<VERSION>.jar \
<HDFS_FILE_LOCATION>
- Submitting with a local Keytab and Principal
/opt/spark/bin/spark-submit \
--deploy-mode cluster \
--class org.apache.spark.examples.HdfsTest \
--master k8s://<KUBERNETES_MASTER_ENDPOINT> \
--conf spark.executor.instances=1 \
--conf spark.app.name=spark-hdfs \
--conf spark.kubernetes.container.image=spark:latest \
--conf spark.kerberos.keytab=<KEYTAB_FILE> \
--conf spark.kerberos.principal=<PRINCIPAL> \
--conf spark.kubernetes.kerberos.krb5.path=/etc/krb5.conf \
local:///opt/spark/examples/jars/spark-examples_<VERSION>.jar \
<HDFS_FILE_LOCATION>
- Submitting with pre-populated secrets, that contain the Delegation Token, already existing within the namespace
/opt/spark/bin/spark-submit \
--deploy-mode cluster \
--class org.apache.spark.examples.HdfsTest \
--master k8s://<KUBERNETES_MASTER_ENDPOINT> \
--conf spark.executor.instances=1 \
--conf spark.app.name=spark-hdfs \
--conf spark.kubernetes.container.image=spark:latest \
--conf spark.kubernetes.kerberos.tokenSecret.name=<SECRET_TOKEN_NAME> \
--conf spark.kubernetes.kerberos.tokenSecret.itemKey=<SECRET_ITEM_KEY> \
--conf spark.kubernetes.kerberos.krb5.path=/etc/krb5.conf \
local:///opt/spark/examples/jars/spark-examples_<VERSION>.jar \
<HDFS_FILE_LOCATION>
3b. Submitting like in (3) however specifying a pre-created krb5 ConfigMap and pre-created HADOOP_CONF_DIR
ConfigMap
/opt/spark/bin/spark-submit \
--deploy-mode cluster \
--class org.apache.spark.examples.HdfsTest \
--master k8s://<KUBERNETES_MASTER_ENDPOINT> \
--conf spark.executor.instances=1 \
--conf spark.app.name=spark-hdfs \
--conf spark.kubernetes.container.image=spark:latest \
--conf spark.kubernetes.kerberos.tokenSecret.name=<SECRET_TOKEN_NAME> \
--conf spark.kubernetes.kerberos.tokenSecret.itemKey=<SECRET_ITEM_KEY> \
--conf spark.kubernetes.hadoop.configMapName=<HCONF_CONFIG_MAP_NAME> \
--conf spark.kubernetes.kerberos.krb5.configMapName=<KRB_CONFIG_MAP_NAME> \
local:///opt/spark/examples/jars/spark-examples_<VERSION>.jar \
<HDFS_FILE_LOCATION>
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.
If your applications persist driver logs in client mode by enabling spark.driver.log.persistToDfs.enabled
,
the directory where the driver logs go (spark.driver.log.dfsDir
) 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 driver log files will be created by Spark with permissions such that only the user and group have read and write access.