TLS/SSL

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

TLS is enabled by using a tls address such as tls:// (the default being tcp, which sends data unauthenticated and unencrypted). In TLS mode, all cluster endpoints must present a valid TLS certificate signed by a given Certificate Authority (CA). It is generally recommended to use a custom CA for your organization, as it will allow signing certificates 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 an endpoint’s certificate has been signed by the correct authority;

  • a certificate file for each endpoint, which is presented to other endpoints so as to achieve mutual authentication;

  • a private key file, which is the cryptographic means to prove to other 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 into a single file (you can then just specify the certificate parameter and leave the private key parameter absent).

It is up to you whether each endpoint uses a different certificate and private key, or whether all endpoints share the same, or whether each endpoint kind (Client, Scheduler, Worker) gets its own certificate / key pair. Unless you have extraordinary requirements, however, the CA certificate should 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 which algorithms are considered secure; this setting’s value should be an OpenSSL cipher string;

  • whether to require encryption, to avoid using plain TCP communications by 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 and dask-worker;

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

    from distributed import Client
    from distributed.security import Security
    
    sec = Security(tls_ca_file='cluster_ca.pem',
                   tls_client_cert='cli_cert.pem',
                   tls_client_key='cli_key.pem',
                   require_encryption=True)
    
    client = Client(..., security=sec)
    

Security policy

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

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

Performance implications

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

See also

A study of AES-NI acceleration shows recent x86 CPUs can AES-encrypt more than 1 GB per second on 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 be overridden in the constructor.

Parameters
require_encryptionbool, optional

Whether TLS encryption is required for all connections.

tls_ca_filestr, optional

Path to a CA certificate file encoded in PEM format.

tls_ciphersstr, optional

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

tls_client_certstr, optional

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

tls_client_keystr, optional

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

tls_scheduler_certstr, optional

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

tls_scheduler_keystr, optional

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

tls_worker_certstr, optional

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

tls_worker_keystr, optional

Path to a key file for a worker, encoded in PEM format. Alternatively, the key may be appended to the cert file, and this parameter be omitted.

get_connection_args(self, role)[source]

Get the connection_args argument for a connect() call with the given role.

get_listen_args(self, role)[source]

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

get_tls_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 securing communication for all roles in a Dask cluster. These keys/certs exist only in memory, and are stored in this object.

This method requires the library cryptography be installed.