TLS/SSL

Dask distributed has support for TLS/SSL communication,providing mutual authentication and encryption of communications between clusterendpoints (Clients, Schedulers, and Workers).

TLS is enabled by using a tls address such as tls:// (the defaultbeing tcp, which sends data unauthenticated and unencrypted). InTLS mode, all cluster endpoints must present a valid TLS certificatesigned by a given Certificate Authority (CA). It is generally recommendedto use a custom CA for your organization, as it will allow signingcertificates for internal hostnames or IP addresses.

Parameters

When using TLS, one has to provide additional parameters:

  • a CA certificate(s) file, which allows TLS to decide whether anendpoint’s certificate has been signed by the correct authority;
  • a certificate file for each endpoint, which is presented to otherendpoints so as to achieve mutual authentication;
  • a private key file, which is the cryptographic means to prove toother endpoints that you are the authorized user of a given certificate.

Note

As per OpenSSL’s requirements, all those files should be in PEM format.Also, it is allowed to concatenate the certificate and private key intoa single file (you can then just specify the certificate parameter andleave the private key parameter absent).

It is up to you whether each endpoint uses a different certificate andprivate key, or whether all endpoints share the same, or whether eachendpoint kind (Client, Scheduler, Worker) gets its own certificate / key pair.Unless you have extraordinary requirements, however, the CA certificateshould probably be the same for all endpoints.

One can also pass additional parameters:

  • a set of allowed ciphers, if you have strong requirements as to whichalgorithms are considered secure; this setting’s value should be anOpenSSL cipher string;
  • whether to require encryption, to avoid using plain TCP communicationsby mistake.

All those parameters can be passed in several ways:

  • through the Dask configuration file;

  • if using the command line, through options to dask-scheduler anddask-worker;

  • if using the API, through a Security object. For example, here ishow you might configure a Security object for client use:

  1. from distributed import Client
  2. from distributed.security import Security
  3.  
  4. sec = Security(tls_ca_file='cluster_ca.pem',
  5. tls_client_cert='cli_cert.pem',
  6. tls_client_key='cli_key.pem',
  7. require_encryption=True)
  8.  
  9. client = Client(..., security=sec)

Security policy

Dask always verifies the certificate presented by a remote endpointagainst the configured CA certificate(s). Certificates are verifiedfor both “client” and “server” endpoints (in the TCP sense), ensuringthe endpoints are mutually authenticated. The hostname or IP address forwhich a certificate has been issued is not checked; this should not be anissue if you are using your own internal Certificate Authority.

It is not possible to disable certificate verification, as it would renderthe communications vulnerable to Man-in-the-Middle attacks.

Performance implications

Encryption is fast on recent CPUs, most of which have hardware accelerationfor AES-based encryption. AES is normally selected by the TLS layerunless you have forced the ciphers parameter to something else. However,encryption may still have a non-negligible overhead if you are transferringvery large data over very high speed network links.

See also

A study of AES-NI accelerationshows recent x86 CPUs can AES-encrypt more than 1 GB per secondon each CPU core.

API

  • class distributed.security.Security(**kwargs)[source]
  • Security configuration for a Dask cluster.

Default values are loaded from Dask’s configuration files, and can beoverridden in the constructor.

Parameters:

  • require_encryption:bool, optional
  • Whether TLS encryption is required for all connections.

  • tls_ca_file:str, optional

  • Path to a CA certificate file encoded in PEM format.

  • tls_ciphers:str, optional

  • An OpenSSL cipher string of allowed ciphers. If not provided, thesystem defaults will be used.

  • tls_client_cert:str, optional

  • Path to a certificate file for the client, encoded in PEM format.

  • tls_client_key:str, optional

  • Path to a key file for the client, encoded in PEM format.Alternatively, the key may be appended to the cert file, and thisparameter be omitted.

  • tls_scheduler_cert:str, optional

  • Path to a certificate file for the scheduler, encoded in PEM format.

  • tls_scheduler_key:str, optional

  • Path to a key file for the scheduler, encoded in PEM format.Alternatively, the key may be appended to the cert file, and thisparameter be omitted.

  • tls_worker_cert:str, optional

  • Path to a certificate file for a worker, encoded in PEM format.

  • tls_worker_key:str, optional

  • Path to a key file for a worker, encoded in PEM format.Alternatively, the key may be appended to the cert file, and thisparameter be omitted.
  • getconnection_args(_self, role)[source]
  • Get the connection_args argument for a connect() call withthe given role.

  • getlisten_args(_self, role)[source]

  • Get the connection_args argument for a listen() call withthe given role.

  • gettls_config_for_role(_self, role)[source]

  • Return the TLS configuration for the given role, as a flat dict.

  • classmethod temporary()[source]

  • Create a new temporary Security object.

This creates a new self-signed key/cert pair suitable for securingcommunication for all roles in a Dask cluster. These keys/certs existonly in memory, and are stored in this object.

This method requires the library cryptography be installed.