Skip to content

Commit

Permalink
Standardize UCX config separator to - (#5539)
Browse files Browse the repository at this point in the history
Some of the UCX configurations use `_` whereas others use `-`. This is
confusing so we now standardize everything to `-`. This also fixes an
inconsistency from #5526, where
configuration files used `create-cuda-context`, but the configuration
read was `create_cuda_context`.
  • Loading branch information
pentschev authored Nov 23, 2021
1 parent 97d4e71 commit b9b26ee
Show file tree
Hide file tree
Showing 5 changed files with 9 additions and 9 deletions.
2 changes: 1 addition & 1 deletion distributed/comm/tests/test_ucx.py
Original file line number Diff line number Diff line change
Expand Up @@ -331,7 +331,7 @@ async def test_simple():

@pytest.mark.asyncio
async def test_cuda_context():
with dask.config.set({"distributed.comm.ucx.create_cuda_context": True}):
with dask.config.set({"distributed.comm.ucx.create-cuda-context": True}):
async with LocalCluster(
protocol="ucx", n_workers=1, asynchronous=True
) as cluster:
Expand Down
8 changes: 4 additions & 4 deletions distributed/comm/tests/test_ucx_config.py
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ async def test_ucx_config(cleanup):
"rdmacm": False,
"net-devices": "",
"tcp": True,
"cuda_copy": True,
"cuda-copy": True,
}

with dask.config.set({"distributed.comm.ucx": ucx}):
Expand All @@ -49,7 +49,7 @@ async def test_ucx_config(cleanup):
"rdmacm": False,
"net-devices": "mlx5_0:1",
"tcp": True,
"cuda_copy": False,
"cuda-copy": False,
}

with dask.config.set({"distributed.comm.ucx": ucx}):
Expand All @@ -68,7 +68,7 @@ async def test_ucx_config(cleanup):
"rdmacm": True,
"net-devices": "all",
"tcp": True,
"cuda_copy": True,
"cuda-copy": True,
}

with dask.config.set({"distributed.comm.ucx": ucx}):
Expand All @@ -85,7 +85,7 @@ async def test_ucx_config(cleanup):
"rdmacm": None,
"net-devices": None,
"tcp": None,
"cuda_copy": None,
"cuda-copy": None,
}

with dask.config.set({"distributed.comm.ucx": ucx}):
Expand Down
4 changes: 2 additions & 2 deletions distributed/comm/ucx.py
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ def init_once():
# We ensure the CUDA context is created before initializing UCX. This can't
# be safely handled externally because communications in Dask start before
# preload scripts run.
if dask.config.get("distributed.comm.ucx.create_cuda_context") is True or (
if dask.config.get("distributed.comm.ucx.create-cuda-context") is True or (
"TLS" in ucx_config and "cuda_copy" in ucx_config["TLS"]
):
try:
Expand Down Expand Up @@ -577,7 +577,7 @@ def _scrub_ucx_config():
if any(
[
dask.config.get("distributed.comm.ucx.nvlink"),
dask.config.get("distributed.comm.ucx.cuda_copy"),
dask.config.get("distributed.comm.ucx.cuda-copy"),
]
):
tls = tls + ",cuda_copy"
Expand Down
2 changes: 1 addition & 1 deletion distributed/distributed-schema.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -793,7 +793,7 @@ properties:
description: |
UCX provides access to other transport methods including NVLink and InfiniBand.
properties:
cuda_copy:
cuda-copy:
type: [boolean, 'null']
description: |
Set environment variables to enable CUDA support over UCX. This may be used even if
Expand Down
2 changes: 1 addition & 1 deletion distributed/distributed.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -187,7 +187,7 @@ distributed:
socket-backlog: 2048
recent-messages-log-length: 0 # number of messages to keep for debugging
ucx:
cuda_copy: null # enable cuda-copy
cuda-copy: null # enable cuda-copy
tcp: null # enable tcp
nvlink: null # enable cuda_ipc
infiniband: null # enable Infiniband
Expand Down

0 comments on commit b9b26ee

Please sign in to comment.