From 4980e746e9be15e5fe6736b6c496b8faea737fd7 Mon Sep 17 00:00:00 2001 From: Till Riedel Date: Sat, 4 Dec 2021 16:56:42 +0100 Subject: [PATCH] add temporary security if set to True and test warning (#527) xfail pbs due to unshared home --- dask_jobqueue/core.py | 15 +++++++++++-- dask_jobqueue/tests/test_jobqueue_core.py | 27 +++++++++++++++++++++++ 2 files changed, 40 insertions(+), 2 deletions(-) diff --git a/dask_jobqueue/core.py b/dask_jobqueue/core.py index de1338d6..baf54146 100644 --- a/dask_jobqueue/core.py +++ b/dask_jobqueue/core.py @@ -20,6 +20,7 @@ from distributed.deploy.spec import ProcessInterface, SpecCluster from distributed.deploy.local import nprocesses_nthreads from distributed.scheduler import Scheduler +from distributed.security import Security logger = logging.getLogger(__name__) @@ -75,8 +76,9 @@ scheduler is started locally asynchronous : bool Whether or not to run this cluster object with the async/await syntax - security : Security - A dask.distributed security object if you're using TLS/SSL + security : Security or Bool + A dask.distributed security object if you're using TLS/SSL. If True, + temporary self-signed credentials will be created automatically. scheduler_options : dict Used to pass additional arguments to Dask Scheduler. For example use ``scheduler_options={'dashboard_address': ':12435'}`` to specify which @@ -509,6 +511,15 @@ def __init__( if protocol is None and security is not None: protocol = "tls://" + if security is True: + try: + security = Security.temporary() + except ImportError: + raise ImportError( + "In order to use TLS without pregenerated certificates `cryptography` is required," + "please install it using either pip or conda" + ) + default_scheduler_options = { "protocol": protocol, "dashboard_address": ":8787", diff --git a/dask_jobqueue/tests/test_jobqueue_core.py b/dask_jobqueue/tests/test_jobqueue_core.py index 7a05e4cd..971908ec 100644 --- a/dask_jobqueue/tests/test_jobqueue_core.py +++ b/dask_jobqueue/tests/test_jobqueue_core.py @@ -370,6 +370,7 @@ def test_wrong_parameter_error(Cluster): @pytest.mark.xfail_env({"htcondor": "#535 no shared filesystem in htcondor ci"}) @pytest.mark.xfail_env({"slurm": "#535 no shared filesystem in slurm ci"}) +@pytest.mark.filterwarnings("error:Using a temporary security object:UserWarning") def test_security(EnvSpecificCluster, loop): dirname = os.path.dirname(__file__) key = os.path.join(dirname, "key.pem") @@ -457,3 +458,29 @@ def test_security_temporary(EnvSpecificCluster, loop): assert result == 11 # TODO assert not any([os.path.exists(f) for f in [keyfile, certfile, cafile]]) + + +@pytest.mark.xfail_env({"htcondor": "#535 no shared filesystem in htcondor ci"}) +@pytest.mark.xfail_env({"slurm": "#535 no shared filesystem in slurm ci"}) +@pytest.mark.xfail_env({"pbs": "current directory (pbsuser home) not shared"}) +def test_security_temporary_defaults(EnvSpecificCluster, loop): + # test automatic behaviour if security is true and shared_temp_directory not set + with pytest.warns(UserWarning, match="shared_temp_directory"), EnvSpecificCluster( + cores=1, + memory="100MB", + security=True, + protocol="tls", + loop=loop, # for some reason (bug?) using the loop fixture requires using a new test case + ) as cluster: + assert cluster.security + assert cluster.scheduler_spec["options"]["security"] == cluster.security + job_script = cluster.job_script() + assert "--tls-key" in job_script + assert "--tls-cert" in job_script + assert "--tls-ca-file" in job_script + + cluster.scale(jobs=1) + with Client(cluster) as client: + future = client.submit(lambda x: x + 1, 10) + result = future.result(timeout=30) + assert result == 11