From c7abf21cf5e7a017fd4032e29a6b28a359cf7675 Mon Sep 17 00:00:00 2001 From: Raymond Douglass Date: Mon, 9 Mar 2020 15:23:53 -0400 Subject: [PATCH 01/67] DOC v0.14 Updates From a8c5ae4f360f56c89662ce69a38fcfff783630e5 Mon Sep 17 00:00:00 2001 From: Paul Taylor Date: Wed, 25 Mar 2020 11:01:22 -0700 Subject: [PATCH 02/67] changelog --- CHANGELOG.rst | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/CHANGELOG.rst b/CHANGELOG.rst index d01f0b5d7..981c97b4b 100644 --- a/CHANGELOG.rst +++ b/CHANGELOG.rst @@ -1,3 +1,7 @@ +0.14 +---- +- Publish branch-0.14 to conda (#262) + 0.13 ---- - Use RMM's `DeviceBuffer` directly (#235) `John Kirkham`_ From 75d0da0fb6534caf13ff7b7b2c782bccd83d92bc Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Sat, 28 Mar 2020 17:06:37 -0700 Subject: [PATCH 03/67] Parse memory_limit in LocalCUDACluster and default to "auto" --- dask_cuda/local_cuda_cluster.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/dask_cuda/local_cuda_cluster.py b/dask_cuda/local_cuda_cluster.py index e9a823450..ad15c9931 100644 --- a/dask_cuda/local_cuda_cluster.py +++ b/dask_cuda/local_cuda_cluster.py @@ -5,6 +5,7 @@ from dask.distributed import LocalCluster from distributed.system import MEMORY_LIMIT from distributed.utils import parse_bytes +from distributed.worker import parse_memory_limit from .device_host_file import DeviceHostFile from .initialize import initialize @@ -124,7 +125,7 @@ def __init__( n_workers=None, threads_per_worker=1, processes=True, - memory_limit=None, + memory_limit="auto", device_memory_limit=None, CUDA_VISIBLE_DEVICES=None, data=None, @@ -144,9 +145,9 @@ def __init__( CUDA_VISIBLE_DEVICES = list(map(int, CUDA_VISIBLE_DEVICES)) if n_workers is None: n_workers = len(CUDA_VISIBLE_DEVICES) - if memory_limit is None: - memory_limit = MEMORY_LIMIT / n_workers - self.host_memory_limit = memory_limit + self.host_memory_limit = parse_memory_limit( + MEMORY_LIMIT, n_threads_per_worker, n_workers + ) self.device_memory_limit = device_memory_limit self.rmm_pool_size = rmm_pool_size From 23b26dca0a723ddf5a488ca20319011cc023c72d Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Sat, 28 Mar 2020 17:12:40 -0700 Subject: [PATCH 04/67] Disable spilling to disk when memory_limit==0 --- dask_cuda/device_host_file.py | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/dask_cuda/device_host_file.py b/dask_cuda/device_host_file.py index 91e4451e4..e462e197b 100644 --- a/dask_cuda/device_host_file.py +++ b/dask_cuda/device_host_file.py @@ -90,7 +90,8 @@ class DeviceHostFile(ZictBase): spills to host cache once filled. memory_limit: int Number of bytes of host memory for host LRU cache, spills to - disk once filled. + disk once filled. Setting this to 0 means unlimited host memory, + implies no spilling to disk. local_directory: path Path where to store serialized objects on disk """ @@ -109,9 +110,12 @@ def __init__( self.disk_func = Func( serialize_bytelist, deserialize_bytes, File(self.disk_func_path) ) - self.host_buffer = Buffer( - self.host_func, self.disk_func, memory_limit, weight=weight - ) + if memory_limit == 0: + self.host_buffer = host_func + else: + self.host_buffer = Buffer( + self.host_func, self.disk_func, memory_limit, weight=weight + ) self.device_keys = set() self.device_func = dict() @@ -121,8 +125,8 @@ def __init__( ) self.device = self.device_buffer.fast.d - self.host = self.host_buffer.fast.d - self.disk = self.host_buffer.slow.d + self.host = host_buffer if memory_limit == 0 else self.host_buffer.fast.d + self.disk = None if memory_limit == 0 else self.host_buffer.slow.d # For Worker compatibility only, where `fast` is host memory buffer self.fast = self.host_buffer.fast From 7c843cd595ae4601c451e150182e8bacf72f9039 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Sun, 29 Mar 2020 03:12:18 -0700 Subject: [PATCH 05/67] Fix usage of `thread_per_worker` argument --- dask_cuda/local_cuda_cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dask_cuda/local_cuda_cluster.py b/dask_cuda/local_cuda_cluster.py index ad15c9931..304d00dfd 100644 --- a/dask_cuda/local_cuda_cluster.py +++ b/dask_cuda/local_cuda_cluster.py @@ -146,7 +146,7 @@ def __init__( if n_workers is None: n_workers = len(CUDA_VISIBLE_DEVICES) self.host_memory_limit = parse_memory_limit( - MEMORY_LIMIT, n_threads_per_worker, n_workers + MEMORY_LIMIT, threads_per_worker, n_workers ) self.device_memory_limit = device_memory_limit From ceecbe6498c1e30cd8aa03221fc4c27046b5d167 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Mon, 30 Mar 2020 06:17:39 -0700 Subject: [PATCH 06/67] Fix wrong references to memory_limit in LocalCUDACluster --- dask_cuda/local_cuda_cluster.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dask_cuda/local_cuda_cluster.py b/dask_cuda/local_cuda_cluster.py index 304d00dfd..43f7e2d49 100644 --- a/dask_cuda/local_cuda_cluster.py +++ b/dask_cuda/local_cuda_cluster.py @@ -146,7 +146,7 @@ def __init__( if n_workers is None: n_workers = len(CUDA_VISIBLE_DEVICES) self.host_memory_limit = parse_memory_limit( - MEMORY_LIMIT, threads_per_worker, n_workers + memory_limit, threads_per_worker, n_workers ) self.device_memory_limit = device_memory_limit @@ -206,7 +206,7 @@ def __init__( super().__init__( n_workers=0, threads_per_worker=threads_per_worker, - memory_limit=memory_limit, + memory_limit=self.host_memory_limit, processes=True, data=data, local_directory=local_directory, From 5c496923b3f2182988420f9af8f0f10007de7566 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Mon, 30 Mar 2020 08:04:19 -0700 Subject: [PATCH 07/67] Fix usage of wrong variable names in device_host_file --- dask_cuda/device_host_file.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dask_cuda/device_host_file.py b/dask_cuda/device_host_file.py index e462e197b..6a4e7cb82 100644 --- a/dask_cuda/device_host_file.py +++ b/dask_cuda/device_host_file.py @@ -111,7 +111,7 @@ def __init__( serialize_bytelist, deserialize_bytes, File(self.disk_func_path) ) if memory_limit == 0: - self.host_buffer = host_func + self.host_buffer = self.host_func else: self.host_buffer = Buffer( self.host_func, self.disk_func, memory_limit, weight=weight @@ -125,11 +125,11 @@ def __init__( ) self.device = self.device_buffer.fast.d - self.host = host_buffer if memory_limit == 0 else self.host_buffer.fast.d + self.host = self.host_buffer if memory_limit == 0 else self.host_buffer.fast.d self.disk = None if memory_limit == 0 else self.host_buffer.slow.d # For Worker compatibility only, where `fast` is host memory buffer - self.fast = self.host_buffer.fast + self.fast = self.host_buffer if memory_limit == 0 else self.host_buffer.fast def __setitem__(self, key, value): if is_device_object(value): From 1ba934e3f5e1193d8cd4f1905aeb54d7f31fbab5 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Mon, 30 Mar 2020 08:10:15 -0700 Subject: [PATCH 08/67] Add spilling tests for `memory_limit=0` --- dask_cuda/tests/test_spill.py | 64 +++++++++++++++++++++++++++++------ 1 file changed, 53 insertions(+), 11 deletions(-) diff --git a/dask_cuda/tests/test_spill.py b/dask_cuda/tests/test_spill.py index dba8e7c9a..bcd13161f 100644 --- a/dask_cuda/tests/test_spill.py +++ b/dask_cuda/tests/test_spill.py @@ -7,6 +7,7 @@ from dask_cuda.device_host_file import DeviceHostFile from distributed import Client, get_worker, wait from distributed.metrics import time +from distributed.sizeof import sizeof from distributed.utils_test import gen_cluster, gen_test, loop # noqa: F401 from distributed.worker import Worker @@ -20,15 +21,24 @@ def device_host_file_size_matches( dhf, total_bytes, device_chunk_overhead=0, serialized_chunk_overhead=1024 ): - byte_sum = dhf.device_buffer.fast.total_weight + dhf.host_buffer.fast.total_weight - file_path = [os.path.join(dhf.disk.directory, safe_key(k)) for k in dhf.disk.keys()] - file_size = [os.path.getsize(f) for f in file_path] - byte_sum += sum(file_size) + byte_sum = dhf.device_buffer.fast.total_weight - # Allow up to chunk_overhead bytes overhead per chunk on disk + # `dhf.host_buffer.fast` is only available when Worker's `memory_limit != 0` + if hasattr(dhf.host_buffer, "fast"): + byte_sum += dhf.host_buffer.fast.total_weight + else: + byte_sum += sum([sizeof(b) for b in dhf.host_buffer.values()]) + + # `dhf.disk` is only available when Worker's `memory_limit != 0` + if dhf.disk is not None: + file_path = [os.path.join(dhf.disk.directory, safe_key(k)) for k in dhf.disk.keys()] + file_size = [os.path.getsize(f) for f in file_path] + byte_sum += sum(file_size) + + # Allow up to chunk_overhead bytes overhead per chunk device_overhead = len(dhf.device) * device_chunk_overhead host_overhead = len(dhf.host) * serialized_chunk_overhead - disk_overhead = len(dhf.disk) * serialized_chunk_overhead + disk_overhead = len(dhf.disk) * serialized_chunk_overhead if dhf.disk is not None else 0 return ( byte_sum >= total_bytes @@ -65,7 +75,7 @@ def delayed_worker_assert(total_size, device_chunk_overhead, serialized_chunk_ov ) -@pytest.mark.xfail(reason="https://github.com/rapidsai/dask-cuda/issues/79") +#@pytest.mark.xfail(reason="https://github.com/rapidsai/dask-cuda/issues/79") @pytest.mark.parametrize( "params", [ @@ -85,6 +95,14 @@ def delayed_worker_assert(total_size, device_chunk_overhead, serialized_chunk_ov "host_pause": None, "spills_to_disk": True, }, + { + "device_memory_limit": 1e9, + "memory_limit": 0, + "host_target": 0.0, + "host_spill": 0.0, + "host_pause": None, + "spills_to_disk": False, + }, ], ) def test_cupy_device_spill(params): @@ -125,7 +143,7 @@ def test_device_spill(client, scheduler, worker): yield client.run(worker_assert, x.nbytes, 1024, 1024) host_chunks = yield client.run(lambda: len(get_worker().data.host)) - disk_chunks = yield client.run(lambda: len(get_worker().data.disk)) + disk_chunks = yield client.run(lambda: len(get_worker().data.disk or list())) for hc, dc in zip(host_chunks.values(), disk_chunks.values()): if params["spills_to_disk"]: assert dc > 0 @@ -155,6 +173,14 @@ def test_device_spill(client, scheduler, worker): "host_pause": None, "spills_to_disk": True, }, + { + "device_memory_limit": 1e9, + "memory_limit": 0, + "host_target": 0.0, + "host_spill": 0.0, + "host_pause": None, + "spills_to_disk": False, + }, ], ) @pytest.mark.asyncio @@ -194,7 +220,7 @@ async def test_cupy_cluster_device_spill(params): await client.run(worker_assert, x.nbytes, 1024, 1024) host_chunks = await client.run(lambda: len(get_worker().data.host)) - disk_chunks = await client.run(lambda: len(get_worker().data.disk)) + disk_chunks = await client.run(lambda: len(get_worker().data.disk or list())) for hc, dc in zip(host_chunks.values(), disk_chunks.values()): if params["spills_to_disk"]: assert dc > 0 @@ -223,6 +249,14 @@ async def test_cupy_cluster_device_spill(params): "host_pause": None, "spills_to_disk": True, }, + { + "device_memory_limit": 1e9, + "memory_limit": 0, + "host_target": 0.0, + "host_spill": 0.0, + "host_pause": None, + "spills_to_disk": False, + }, ], ) def test_cudf_device_spill(params): @@ -267,7 +301,7 @@ def test_device_spill(client, scheduler, worker): del cdf host_chunks = yield client.run(lambda: len(get_worker().data.host)) - disk_chunks = yield client.run(lambda: len(get_worker().data.disk)) + disk_chunks = yield client.run(lambda: len(get_worker().data.disk or list())) for hc, dc in zip(host_chunks.values(), disk_chunks.values()): if params["spills_to_disk"]: assert dc > 0 @@ -304,6 +338,14 @@ def test_device_spill(client, scheduler, worker): "host_pause": None, "spills_to_disk": True, }, + { + "device_memory_limit": 1e9, + "memory_limit": 0, + "host_target": 0.0, + "host_spill": 0.0, + "host_pause": None, + "spills_to_disk": False, + }, ], ) @pytest.mark.asyncio @@ -346,7 +388,7 @@ async def test_cudf_cluster_device_spill(params): del cdf host_chunks = await client.run(lambda: len(get_worker().data.host)) - disk_chunks = await client.run(lambda: len(get_worker().data.disk)) + disk_chunks = await client.run(lambda: len(get_worker().data.disk or list())) for hc, dc in zip(host_chunks.values(), disk_chunks.values()): if params["spills_to_disk"]: assert dc > 0 From 1c3da918fb4aafbc55ad0973d1c384c1d2d827b6 Mon Sep 17 00:00:00 2001 From: John Kirkham Date: Mon, 30 Mar 2020 16:36:18 -0700 Subject: [PATCH 09/67] Raise serialization errors when spilling The default behavior of `serialize` (and thus `serialize_bytes`) is to capture any serialization errors as messages. The result is the error gets raises on deserialization. However for spilling, this means that we fail to spill and raise that error only after trying to restore that data. Instead go ahead and raise the error when trying to spill to avoid thinking content has spilled successfully (when that is not the case) and thus losing it. --- dask_cuda/device_host_file.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dask_cuda/device_host_file.py b/dask_cuda/device_host_file.py index 6a4e7cb82..6b3f24043 100644 --- a/dask_cuda/device_host_file.py +++ b/dask_cuda/device_host_file.py @@ -1,3 +1,4 @@ +import functools import os import numpy @@ -108,7 +109,9 @@ def __init__( self.host_func = dict() self.disk_func = Func( - serialize_bytelist, deserialize_bytes, File(self.disk_func_path) + functools.partial(serialize_bytelist, on_error="raise"), + deserialize_bytes, + File(self.disk_func_path), ) if memory_limit == 0: self.host_buffer = self.host_func From a08859894854aeb2b6fdff4ec60ab56909a04183 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Fri, 3 Apr 2020 11:40:45 -0700 Subject: [PATCH 10/67] Fix dask-cuda-worker memory_limit --- dask_cuda/dask_cuda_worker.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dask_cuda/dask_cuda_worker.py b/dask_cuda/dask_cuda_worker.py index 448218db9..a4102cdbc 100755 --- a/dask_cuda/dask_cuda_worker.py +++ b/dask_cuda/dask_cuda_worker.py @@ -232,6 +232,8 @@ def main( if not nthreads: nthreads = min(1, multiprocessing.cpu_count() // nprocs) + memory_limit = parse_memory_limit(memory_limit, nthreads, total_cores=nprocs) + if pid_file: with open(pid_file, "w") as f: f.write(str(os.getpid())) @@ -324,9 +326,7 @@ def del_pid_file(): "device_memory_limit": get_device_total_memory(index=i) if (device_memory_limit == "auto" or device_memory_limit == int(0)) else parse_bytes(device_memory_limit), - "memory_limit": parse_memory_limit( - memory_limit, nthreads, total_cores=nprocs - ), + "memory_limit": memory_limit, "local_directory": local_directory, }, ), From 69a3288ae25b4c7dd3c0caa739e526ed90bb748a Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Fri, 3 Apr 2020 12:22:31 -0700 Subject: [PATCH 11/67] Add test for dask-cuda-worker memory_limit --- dask_cuda/tests/test_dask_cuda_worker.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/dask_cuda/tests/test_dask_cuda_worker.py b/dask_cuda/tests/test_dask_cuda_worker.py index a29c91e56..63cec983c 100644 --- a/dask_cuda/tests/test_dask_cuda_worker.py +++ b/dask_cuda/tests/test_dask_cuda_worker.py @@ -5,6 +5,7 @@ from distributed import Client from distributed.metrics import time +from distributed.system import MEMORY_LIMIT from distributed.utils_test import loop # noqa: F401 from distributed.utils_test import popen from dask_cuda.utils import get_gpu_count @@ -12,7 +13,7 @@ import pytest -def test_cuda_visible_devices(loop): # noqa: F811 +def test_cuda_visible_devices_and_memory_limit(loop): # noqa: F811 os.environ["CUDA_VISIBLE_DEVICES"] = "2,3,7,8" try: with popen(["dask-scheduler", "--port", "9359", "--no-dashboard"]): @@ -44,6 +45,11 @@ def get_visible_devices(): expected = {"2,3,7,8": 1, "3,7,8,2": 1, "7,8,2,3": 1, "8,2,3,7": 1} for v in result.values(): del expected[v] + + workers = client.scheduler_info()["workers"] + for w in workers.values(): + assert w["memory_limit"] == MEMORY_LIMIT // len(workers) + assert len(expected) == 0 finally: del os.environ["CUDA_VISIBLE_DEVICES"] From f385ef926b2e2c6cc6ef6ec56700a3778915102f Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Tue, 14 Apr 2020 05:04:29 -0700 Subject: [PATCH 12/67] Add NVTX annotations for spilling --- dask_cuda/device_host_file.py | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/dask_cuda/device_host_file.py b/dask_cuda/device_host_file.py index 6b3f24043..f24e530a2 100644 --- a/dask_cuda/device_host_file.py +++ b/dask_cuda/device_host_file.py @@ -20,6 +20,16 @@ from .is_device_object import is_device_object +try: + from cudf._lib.nvtx import annotate +except ImportError: + # NVTX annotations functionality currently exists in cuDF, if cuDF isn't + # installed, `annotate` yields only. + from contextlib import contextmanager + @contextmanager + def annotate(message=None, color="blue", domain=None): + yield + class DeviceSerialized: """ Store device object on the host @@ -65,12 +75,14 @@ def device_deserialize(header, frames): return DeviceSerialized(header["main-header"], parts) +@annotate("SPILL_D2H", color="red", domain="dask_cuda") def device_to_host(obj: object) -> DeviceSerialized: header, frames = serialize(obj, serializers=["dask", "pickle"]) frames = [numpy.asarray(f) for f in frames] return DeviceSerialized(header, frames) +@annotate("SPILL_H2D", color="green", domain="dask_cuda") def host_to_device(s: DeviceSerialized) -> object: return deserialize(s.header, s.parts) From 6b6c2dec54ea9702caa698e4a2be03f0f07ca3ce Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Tue, 14 Apr 2020 08:44:50 -0700 Subject: [PATCH 13/67] Rename and move nvtx_annotate to utils --- dask_cuda/device_host_file.py | 15 +++------------ dask_cuda/utils.py | 11 +++++++++++ 2 files changed, 14 insertions(+), 12 deletions(-) diff --git a/dask_cuda/device_host_file.py b/dask_cuda/device_host_file.py index f24e530a2..fc4f60bd8 100644 --- a/dask_cuda/device_host_file.py +++ b/dask_cuda/device_host_file.py @@ -19,16 +19,7 @@ from zict.common import ZictBase from .is_device_object import is_device_object - -try: - from cudf._lib.nvtx import annotate -except ImportError: - # NVTX annotations functionality currently exists in cuDF, if cuDF isn't - # installed, `annotate` yields only. - from contextlib import contextmanager - @contextmanager - def annotate(message=None, color="blue", domain=None): - yield +from .utils import nvtx_annotate class DeviceSerialized: @@ -75,14 +66,14 @@ def device_deserialize(header, frames): return DeviceSerialized(header["main-header"], parts) -@annotate("SPILL_D2H", color="red", domain="dask_cuda") +@nvtx_annotate("SPILL_D2H", color="red", domain="dask_cuda") def device_to_host(obj: object) -> DeviceSerialized: header, frames = serialize(obj, serializers=["dask", "pickle"]) frames = [numpy.asarray(f) for f in frames] return DeviceSerialized(header, frames) -@annotate("SPILL_H2D", color="green", domain="dask_cuda") +@nvtx_annotate("SPILL_H2D", color="green", domain="dask_cuda") def host_to_device(s: DeviceSerialized) -> object: return deserialize(s.header, s.parts) diff --git a/dask_cuda/utils.py b/dask_cuda/utils.py index 6ab7fb3b8..fbb1f0605 100644 --- a/dask_cuda/utils.py +++ b/dask_cuda/utils.py @@ -8,6 +8,17 @@ import toolz +try: + from cudf._lib.nvtx import annotate as nvtx_annotate +except ImportError: + # NVTX annotations functionality currently exists in cuDF, if cuDF isn't + # installed, `annotate` yields only. + from contextlib import contextmanager + @contextmanager + def nvtx_annotate(message=None, color="blue", domain=None): + yield + + class CPUAffinity: def __init__(self, cores): self.cores = cores From ba1565be755e53125bc51d5dc306c0b9d529f78c Mon Sep 17 00:00:00 2001 From: Raymond Douglass Date: Mon, 27 Apr 2020 16:38:15 -0400 Subject: [PATCH 14/67] Skip existing on conda uploads --- ci/cpu/upload-anaconda.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ci/cpu/upload-anaconda.sh b/ci/cpu/upload-anaconda.sh index d46f2bcc9..4183188d8 100755 --- a/ci/cpu/upload-anaconda.sh +++ b/ci/cpu/upload-anaconda.sh @@ -25,4 +25,4 @@ fi echo "Upload" echo ${UPLOADFILE} -anaconda -t ${MY_UPLOAD_KEY} upload -u ${CONDA_USERNAME:-rapidsai} ${LABEL_OPTION} --force ${UPLOADFILE} +anaconda -t ${MY_UPLOAD_KEY} upload -u ${CONDA_USERNAME:-rapidsai} ${LABEL_OPTION} --skip-existing ${UPLOADFILE} From 8cb1cd94ffe8718b6c21de262b64fe769020d4a6 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Mon, 27 Apr 2020 16:18:43 -0700 Subject: [PATCH 15/67] Add rdmacm support --- dask_cuda/dask_cuda_worker.py | 8 ++++++++ dask_cuda/initialize.py | 9 +++++++++ dask_cuda/local_cuda_cluster.py | 5 +++++ dask_cuda/utils.py | 10 ++++++++++ 4 files changed, 32 insertions(+) diff --git a/dask_cuda/dask_cuda_worker.py b/dask_cuda/dask_cuda_worker.py index a4102cdbc..0976eff7d 100755 --- a/dask_cuda/dask_cuda_worker.py +++ b/dask_cuda/dask_cuda_worker.py @@ -176,6 +176,12 @@ default=False, help="Enable InfiniBand communication", ) +@click.option( + "--enable-rdmacm/--disable-rdmacm", + default=False, + help="Enable RDMA connection manager, " + "currently requires InfiniBand enabled." +) @click.option( "--enable-nvlink/--disable-nvlink", default=False, @@ -214,6 +220,7 @@ def main( enable_tcp_over_ucx, enable_infiniband, enable_nvlink, + enable_rdmacm, net_devices, **kwargs, ): @@ -316,6 +323,7 @@ def del_pid_file(): enable_tcp_over_ucx=enable_tcp_over_ucx, enable_infiniband=enable_infiniband, enable_nvlink=enable_nvlink, + enable_rdmacm=enable_rdmacm, net_devices=net_devices, cuda_device_index=i, ) diff --git a/dask_cuda/initialize.py b/dask_cuda/initialize.py index 04c7d735f..f6308f28f 100644 --- a/dask_cuda/initialize.py +++ b/dask_cuda/initialize.py @@ -40,6 +40,7 @@ def initialize( enable_tcp_over_ucx=False, enable_infiniband=False, enable_nvlink=False, + enable_rdmacm=False, net_devices="", cuda_device_index=None, ): @@ -53,6 +54,7 @@ def initialize( enable_tcp_over_ucx=enable_tcp_over_ucx, enable_infiniband=enable_infiniband, enable_nvlink=enable_nvlink, + enable_rdmacm=enable_rdmacm, net_devices=net_devices, cuda_device_index=cuda_device_index, ) @@ -80,6 +82,12 @@ def initialize( default=False, help="Enable NVLink communication", ) +@click.option( + "--enable-rdmacm/--disable-rdmacm", + default=False, + help="Enable RDMA connection manager, " + "currently requires InfiniBand enabled." +) @click.option( "--net-devices", type=str, @@ -93,6 +101,7 @@ def dask_setup( enable_tcp_over_ucx, enable_infiniband, enable_nvlink, + enable_rdmacm, net_devices, ): if create_cuda_context: diff --git a/dask_cuda/local_cuda_cluster.py b/dask_cuda/local_cuda_cluster.py index 43f7e2d49..72c4526b2 100644 --- a/dask_cuda/local_cuda_cluster.py +++ b/dask_cuda/local_cuda_cluster.py @@ -80,6 +80,9 @@ class LocalCUDACluster(LocalCluster): enable_infiniband: bool Set environment variables to enable UCX InfiniBand support, requires protocol='ucx' and implies enable_tcp_over_ucx=True. + enable_rdmacm: bool + Set environment variables to enable UCX RDMA connection manager support, + requires protocol='ucx' and enable_infiniband=True. enable_nvlink: bool Set environment variables to enable UCX NVLink support, requires protocol='ucx' and implies enable_tcp_over_ucx=True. @@ -134,6 +137,7 @@ def __init__( enable_tcp_over_ucx=False, enable_infiniband=False, enable_nvlink=False, + enable_rdmacm=False, ucx_net_devices=None, rmm_pool_size=None, **kwargs, @@ -216,6 +220,7 @@ def __init__( enable_tcp_over_ucx=enable_tcp_over_ucx, enable_nvlink=enable_nvlink, enable_infiniband=enable_infiniband, + enable_rdmacm=enable_rdmacm, ) }, **kwargs, diff --git a/dask_cuda/utils.py b/dask_cuda/utils.py index fbb1f0605..7d0ade2e3 100644 --- a/dask_cuda/utils.py +++ b/dask_cuda/utils.py @@ -190,6 +190,7 @@ def get_ucx_config( enable_tcp_over_ucx=False, enable_infiniband=False, enable_nvlink=False, + enable_rdmacm=False, net_devices="", cuda_device_index=None, ): @@ -197,6 +198,7 @@ def get_ucx_config( "tcp": None, "infiniband": None, "nvlink": None, + "rdmacm": None, "net-devices": None, "cuda_copy": None, } @@ -208,6 +210,8 @@ def get_ucx_config( ucx_config["infiniband"] = True if enable_nvlink: ucx_config["nvlink"] = True + if enable_rdmacm: + ucx_config["rdmacm"] = True if net_devices is not None and net_devices != "": ucx_config["net-devices"] = get_ucx_net_devices(cuda_device_index, net_devices) @@ -220,6 +224,7 @@ def get_preload_options( enable_tcp_over_ucx=False, enable_infiniband=False, enable_nvlink=False, + enable_rdmacm=False, ucx_net_devices="", cuda_device_index=0, ): @@ -241,6 +246,9 @@ def get_preload_options( enable_infiniband: bool Set environment variables to enable UCX InfiniBand support. Implies enable_tcp=True. + enable_rdmacm: bool + Set environment variables to enable UCX RDMA connection manager support. + Currently requires enable_infiniband=True. enable_nvlink: bool Set environment variables to enable UCX NVLink support. Implies enable_tcp=True. @@ -276,6 +284,8 @@ def get_preload_options( initialize_ucx_argv.append("--enable-tcp-over-ucx") if enable_infiniband: initialize_ucx_argv.append("--enable-infiniband") + if enable_rdmacm: + initialize_ucx_argv.append("--enable-rdmacm") if enable_nvlink: initialize_ucx_argv.append("--enable-nvlink") if ucx_net_devices is not None and ucx_net_devices != "": From 0ed93a477bf7a83c2c984c6796d0e2b3cb44bceb Mon Sep 17 00:00:00 2001 From: Eli Fajardo Date: Wed, 29 Apr 2020 18:35:05 +0000 Subject: [PATCH 16/67] local gpuci build script --- ci/local/README.md | 57 ++++++++++++++++++ ci/local/build.sh | 142 +++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 199 insertions(+) create mode 100644 ci/local/README.md create mode 100755 ci/local/build.sh diff --git a/ci/local/README.md b/ci/local/README.md new file mode 100644 index 000000000..3dc6e0f51 --- /dev/null +++ b/ci/local/README.md @@ -0,0 +1,57 @@ +## Purpose + +This script is designed for developer and contributor use. This tool mimics the actions of gpuCI on your local machine. This allows you to test and even debug your code inside a gpuCI base container before pushing your code as a GitHub commit. +The script can be helpful in locally triaging and debugging RAPIDS continuous integration failures. + +## Requirements + +``` +nvidia-docker +``` + +## Usage + +``` +bash build.sh [-h] [-H] [-s] [-r ] [-i ] +Build and test your local repository using a base gpuCI Docker image + +where: + -H Show this help text + -r Path to repository (defaults to working directory) + -i Use Docker image (default is gpuci/rapidsai-base:cuda10.0-ubuntu16.04-gcc5-py3.6) + -s Skip building and testing and start an interactive shell in a container of the Docker image +``` + +Example Usage: +`bash build.sh -r ~/rapids/dask-cuda -i gpuci/rapidsai-base:cuda10.1-ubuntu16.04-gcc5-py3.6` + +For a full list of available gpuCI docker images, visit our [DockerHub](https://hub.docker.com/r/gpuci/rapidsai-base/tags) page. + +Style Check: +```bash +$ bash ci/local/build.sh -r ~/rapids/dask-cuda -s +$ source activate gdf #Activate gpuCI conda environment +$ cd rapids +$ flake8 python +``` + +## Information + +There are some caveats to be aware of when using this script, especially if you plan on developing from within the container itself. + + +### Docker Image Build Repository + +The docker image will generate build artifacts in a folder on your machine located in the `root` directory of the repository you passed to the script. For the above example, the directory is named `~/rapids/dask-cuda/build_rapidsai-base_cuda10.1-ubuntu16.04-gcc5-py3.6/`. Feel free to remove this directory after the script is finished. + +*Note*: The script *will not* override your local build repository. Your local environment stays in tact. + + +### Where The User is Dumped + +The script will build your repository and run all tests. If any tests fail, it dumps the user into the docker container itself to allow you to debug from within the container. If all the tests pass as expected the container exits and is automatically removed. Remember to exit the container if tests fail and you do not wish to debug within the container itself. + + +### Container File Structure + +Your repository will be located in the `/rapids/` folder of the container. This folder is volume mounted from the local machine. Any changes to the code in this repository are replicated onto the local machine. The `cpp/build` and `python/build` directories within your repository is on a separate mount to avoid conflicting with your local build artifacts. diff --git a/ci/local/build.sh b/ci/local/build.sh new file mode 100755 index 000000000..443e3c148 --- /dev/null +++ b/ci/local/build.sh @@ -0,0 +1,142 @@ +#!/bin/bash + +DOCKER_IMAGE="gpuci/rapidsai-base:cuda10.0-ubuntu16.04-gcc5-py3.6" +REPO_PATH=${PWD} +RAPIDS_DIR_IN_CONTAINER="/rapids" +CPP_BUILD_DIR="cpp/build" +PYTHON_BUILD_DIR="python/build" +CONTAINER_SHELL_ONLY=0 + +SHORTHELP="$(basename "$0") [-h] [-H] [-s] [-r ] [-i ]" +LONGHELP="${SHORTHELP} +Build and test your local repository using a base gpuCI Docker image + +where: + -H Show this help text + -r Path to repository (defaults to working directory) + -i Use Docker image (default is ${DOCKER_IMAGE}) + -s Skip building and testing and start an interactive shell in a container of the Docker image +" + +# Limit GPUs available to container based on CUDA_VISIBLE_DEVICES +if [[ -z "${CUDA_VISIBLE_DEVICES}" ]]; then + NVIDIA_VISIBLE_DEVICES="all" +else + NVIDIA_VISIBLE_DEVICES=${CUDA_VISIBLE_DEVICES} +fi + +while getopts ":hHr:i:s" option; do + case ${option} in + r) + REPO_PATH=${OPTARG} + ;; + i) + DOCKER_IMAGE=${OPTARG} + ;; + s) + CONTAINER_SHELL_ONLY=1 + ;; + h) + echo "${SHORTHELP}" + exit 0 + ;; + H) + echo "${LONGHELP}" + exit 0 + ;; + *) + echo "ERROR: Invalid flag" + echo "${SHORTHELP}" + exit 1 + ;; + esac +done + +REPO_PATH_IN_CONTAINER="${RAPIDS_DIR_IN_CONTAINER}/$(basename "${REPO_PATH}")" +CPP_BUILD_DIR_IN_CONTAINER="${RAPIDS_DIR_IN_CONTAINER}/$(basename "${REPO_PATH}")/${CPP_BUILD_DIR}" +PYTHON_BUILD_DIR_IN_CONTAINER="${RAPIDS_DIR_IN_CONTAINER}/$(basename "${REPO_PATH}")/${PYTHON_BUILD_DIR}" + + +# BASE_CONTAINER_BUILD_DIR is named after the image name, allowing for +# multiple image builds to coexist on the local filesystem. This will +# be mapped to the typical BUILD_DIR inside of the container. Builds +# running in the container generate build artifacts just as they would +# in a bare-metal environment, and the host filesystem is able to +# maintain the host build in BUILD_DIR as well. +# FIXME: Fix the shellcheck complaints +# shellcheck disable=SC2001,SC2005,SC2046 +BASE_CONTAINER_BUILD_DIR=${REPO_PATH}/build_$(echo $(basename "${DOCKER_IMAGE}")|sed -e 's/:/_/g') +CPP_CONTAINER_BUILD_DIR=${BASE_CONTAINER_BUILD_DIR}/cpp +PYTHON_CONTAINER_BUILD_DIR=${BASE_CONTAINER_BUILD_DIR}/python +# Create build directories. This is to ensure correct owner for directories. If +# directories don't exist there is side effect from docker volume mounting creating build +# directories owned by root(volume mount point(s)) +mkdir -p "${REPO_PATH}/${CPP_BUILD_DIR}" +mkdir -p "${REPO_PATH}/${PYTHON_BUILD_DIR}" + +BUILD_SCRIPT="#!/bin/bash +set -e +WORKSPACE=${REPO_PATH_IN_CONTAINER} +PREBUILD_SCRIPT=${REPO_PATH_IN_CONTAINER}/ci/gpu/prebuild.sh +BUILD_SCRIPT=${REPO_PATH_IN_CONTAINER}/ci/gpu/build.sh +cd \${WORKSPACE} +if [ -f \${PREBUILD_SCRIPT} ]; then + source \${PREBUILD_SCRIPT} +fi +yes | source \${BUILD_SCRIPT} +" + +if (( CONTAINER_SHELL_ONLY == 0 )); then + COMMAND="${CPP_BUILD_DIR_IN_CONTAINER}/build.sh || bash" +else + COMMAND="bash" +fi + +# Create the build dir for the container to mount, generate the build script inside of it +mkdir -p "${BASE_CONTAINER_BUILD_DIR}" +mkdir -p "${CPP_CONTAINER_BUILD_DIR}" +mkdir -p "${PYTHON_CONTAINER_BUILD_DIR}" +echo "${BUILD_SCRIPT}" > "${CPP_CONTAINER_BUILD_DIR}/build.sh" +chmod ugo+x "${CPP_CONTAINER_BUILD_DIR}/build.sh" + +# Mount passwd and group files to docker. This allows docker to resolve username and group +# avoiding these nags: +# * groups: cannot find name for group ID ID +# * I have no name!@id:/$ +# For ldap user user information is not present in system /etc/passwd and /etc/group files. +# Hence we generate dummy files for ldap users which docker uses to resolve username and group + +PASSWD_FILE="/etc/passwd" +GROUP_FILE="/etc/group" + +USER_FOUND=$(grep -wc "$(whoami)" < "$PASSWD_FILE") +if [ "$USER_FOUND" == 0 ]; then + echo "Local User not found, LDAP WAR for docker mounts activated. Creating dummy passwd and group" + echo "files to allow docker resolve username and group" + cp "$PASSWD_FILE" /tmp/passwd + PASSWD_FILE="/tmp/passwd" + cp "$GROUP_FILE" /tmp/group + GROUP_FILE="/tmp/group" + echo "$(whoami):x:$(id -u):$(id -g):$(whoami),,,:$HOME:$SHELL" >> "$PASSWD_FILE" + echo "$(whoami):x:$(id -g):" >> "$GROUP_FILE" +fi + +# Run the generated build script in a container +docker pull "${DOCKER_IMAGE}" + +DOCKER_MAJOR=$(docker -v|sed 's/[^[0-9]*\([0-9]*\).*/\1/') +GPU_OPTS="--gpus device=${NVIDIA_VISIBLE_DEVICES}" +if [ "$DOCKER_MAJOR" -lt 19 ] +then + GPU_OPTS="--runtime=nvidia -e NVIDIA_VISIBLE_DEVICES='${NVIDIA_VISIBLE_DEVICES}'" +fi + +docker run --rm -it ${GPU_OPTS} \ + -u "$(id -u)":"$(id -g)" \ + -v "${REPO_PATH}":"${REPO_PATH_IN_CONTAINER}" \ + -v "${CPP_CONTAINER_BUILD_DIR}":"${CPP_BUILD_DIR_IN_CONTAINER}" \ + -v "${PYTHON_CONTAINER_BUILD_DIR}":"${PYTHON_BUILD_DIR_IN_CONTAINER}" \ + -v "$PASSWD_FILE":/etc/passwd:ro \ + -v "$GROUP_FILE":/etc/group:ro \ + --cap-add=SYS_PTRACE \ + "${DOCKER_IMAGE}" bash -c "${COMMAND}" \ No newline at end of file From a50b9dfc38ae3d547b16ff5ba0bd657c294b0bdd Mon Sep 17 00:00:00 2001 From: Eli Fajardo Date: Wed, 29 Apr 2020 18:40:10 +0000 Subject: [PATCH 17/67] update changelog --- CHANGELOG.rst | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.rst b/CHANGELOG.rst index dcbf74757..96e84099b 100644 --- a/CHANGELOG.rst +++ b/CHANGELOG.rst @@ -1,6 +1,7 @@ 0.14 ---- - Publish branch-0.14 to conda (#262) +- Local gpuCI build script (#285) 0.13 ---- From 57760a4ab500c12ef1c09b68196b1ae9e53915ca Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Fri, 1 May 2020 04:44:12 -0700 Subject: [PATCH 18/67] Add new get_host_from_cuda_device utility function --- dask_cuda/utils.py | 49 +++++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 48 insertions(+), 1 deletion(-) diff --git a/dask_cuda/utils.py b/dask_cuda/utils.py index 7d0ade2e3..e97af5512 100644 --- a/dask_cuda/utils.py +++ b/dask_cuda/utils.py @@ -3,17 +3,19 @@ import warnings from multiprocessing import cpu_count +from distributed.utils import get_ip_interface + import numpy as np import pynvml import toolz - try: from cudf._lib.nvtx import annotate as nvtx_annotate except ImportError: # NVTX annotations functionality currently exists in cuDF, if cuDF isn't # installed, `annotate` yields only. from contextlib import contextmanager + @contextmanager def nvtx_annotate(message=None, color="blue", domain=None): yield @@ -295,3 +297,48 @@ def get_preload_options( preload_options["preload_argv"].extend(initialize_ucx_argv) return preload_options + + +def get_host_from_cuda_device(host, cuda_device_index, enable_infiniband, net_devices): + """ + Return the host address for a given CUDA device if enable_infinibad=True + and net_devices='auto', otherwise return the original host parameter. + This function is used to identify the host when InfiniBand interfaces + are available on the system and automatic detection of topologically + closest ones are requested. Currently requires that at least one of the + topologically closest devices is an InfiniBand and has a network address, + such as 'ib0'. + + Parameters + ---------- + host: None or str + Host to use if one cannot be identified automatically, when + net_devices is not "auto" or enable_infiniband=False. + cuda_device_index: int + The index of the CUDA device used to identify the host. + ucx_net_devices: str or callable + A string with value 'auto' to attempt identifying the host, otherwise + returns the value of host argument. + enable_infiniband: bool + True to attempt identifying the host, otherwise return the value of + host argument. + Set environment variables to enable UCX InfiniBand support. Implies + enable_tcp=True. + + Example + ------- + >>> from dask_cuda.utils import get_host_from_cuda_device + >>> get_host_from_cuda_device('127.0.0.1', 0, None, False) + '127.0.0.1' + >>> get_host_from_cuda_device('127.0.0.1', 0, "auto", False) + '127.0.0.1' + >>> get_host_from_cuda_device('127.0.0.1', 0, "auto", True) + '10.33.225.162' + """ + if enable_infiniband and net_devices == "auto": + devices = get_ucx_net_devices(cuda_device_index, "auto").split(",") + for d in devices: + if d.startswith("ib"): + host = get_ip_interface(d) + break + return host From c19876383581a8ce4d0f6016e80c442a71bf8e04 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Fri, 1 May 2020 04:45:17 -0700 Subject: [PATCH 19/67] Add automatic host identification support to dask_cuda_worker --- dask_cuda/dask_cuda_worker.py | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/dask_cuda/dask_cuda_worker.py b/dask_cuda/dask_cuda_worker.py index 0976eff7d..6c0da4f36 100755 --- a/dask_cuda/dask_cuda_worker.py +++ b/dask_cuda/dask_cuda_worker.py @@ -30,6 +30,7 @@ RMMPool, get_cpu_affinity, get_device_total_memory, + get_host_from_cuda_device, get_n_gpus, get_ucx_config, ) @@ -179,8 +180,7 @@ @click.option( "--enable-rdmacm/--disable-rdmacm", default=False, - help="Enable RDMA connection manager, " - "currently requires InfiniBand enabled." + help="Enable RDMA connection manager, currently requires InfiniBand enabled.", ) @click.option( "--enable-nvlink/--disable-nvlink", @@ -310,7 +310,12 @@ def del_pid_file(): loop=loop, resources=resources, memory_limit=memory_limit, - host=host, + host=get_host_from_cuda_device( + host=host, + cuda_device_index=i, + enable_infiniband=enable_infiniband, + net_devices=net_devices, + ), preload=(list(preload) or []) + ["dask_cuda.initialize"], preload_argv=(list(preload_argv) or []) + ["--create-cuda-context"], security=sec, From 72c90b805bc399f5ad5517776d1d76080bd8b515 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Fri, 1 May 2020 05:53:34 -0700 Subject: [PATCH 20/67] Remove deprecated DGX class --- dask_cuda/dgx.py | 93 ------------------------------------------------ 1 file changed, 93 deletions(-) delete mode 100644 dask_cuda/dgx.py diff --git a/dask_cuda/dgx.py b/dask_cuda/dgx.py deleted file mode 100644 index 9d685e96c..000000000 --- a/dask_cuda/dgx.py +++ /dev/null @@ -1,93 +0,0 @@ -import warnings - -from .local_cuda_cluster import LocalCUDACluster - - -class DGX(LocalCUDACluster): - def __init__( - self, - interface=None, - dashboard_address=":8787", - threads_per_worker=1, - silence_logs=True, - CUDA_VISIBLE_DEVICES=None, - protocol=None, - enable_tcp_over_ucx=False, - enable_infiniband=False, - enable_nvlink=False, - ucx_net_devices=None, - **kwargs, - ): - """ A Local Cluster for a DGX 1 machine - - NVIDIA's DGX-1 machine has a complex architecture mapping CPUs, GPUs, and - network hardware. This function creates a local cluster that tries to - respect this hardware as much as possible. - - It creates one Dask worker process per GPU, and assigns each worker process - the correct CPU cores and Network interface cards to maximize performance. - If UCX and UCX-Py are also available, it's possible to use InfiniBand and - NVLink connections for optimal data transfer performance. - - That being said, things aren't perfect. Today a DGX has very high - performance between certain sets of GPUs and not others. A Dask DGX - cluster that uses only certain tightly coupled parts of the computer will - have significantly higher bandwidth than a deployment on the entire thing. - - Parameters - ---------- - interface: str - The external interface used to connect to the scheduler, usually - the ethernet interface is used for connection (not the InfiniBand!). - dashboard_address: str - The address for the scheduler dashboard. Defaults to ":8787". - threads_per_worker: int - Number of threads to be used for each CUDA worker process. - silence_logs: bool - Disable logging for all worker processes - CUDA_VISIBLE_DEVICES: str - String like ``"0,1,2,3"`` or ``[0, 1, 2, 3]`` to restrict activity to - different GPUs - protocol: str - Protocol to use for communication, e.g., "tcp" or "ucx" - enable_tcp_over_ucx: bool - Set environment variables to enable TCP over UCX, even if InfiniBand - and NVLink are not supported or disabled. - enable_infiniband: bool - Set environment variables to enable UCX InfiniBand support, requires - protocol='ucx' and implies enable_tcp_over_ucx=True. - enable_nvlink: bool - Set environment variables to enable UCX NVLink support, requires - protocol='ucx' and implies enable_tcp_over_ucx=True. - - Raises - ------ - TypeError - If enable_infiniband or enable_nvlink is True and protocol is not 'ucx' - - Examples - -------- - >>> from dask_cuda import DGX - >>> from dask.distributed import Client - >>> cluster = DGX() - >>> client = Client(cluster) - """ - warnings.warn( - "DGX is deprecated and will be removed in the next release, please switch " - "to LocalCUDACluster", - DeprecationWarning, - ) - - super().__init__( - interface=interface, - dashboard_address=dashboard_address, - threads_per_worker=threads_per_worker, - silence_logs=silence_logs, - CUDA_VISIBLE_DEVICES=CUDA_VISIBLE_DEVICES, - protocol=protocol, - enable_tcp_over_ucx=enable_tcp_over_ucx, - enable_nvlink=enable_nvlink, - enable_infiniband=enable_infiniband, - ucx_net_devices=ucx_net_devices, - **kwargs, - ) From cc26d604da1a2bd694c0195a88abe16c728cb3b4 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Fri, 1 May 2020 05:53:56 -0700 Subject: [PATCH 21/67] Update DGX tests to use LocalCUDACluster --- dask_cuda/tests/test_dgx.py | 120 +++++++++++++++++------------------- 1 file changed, 58 insertions(+), 62 deletions(-) diff --git a/dask_cuda/tests/test_dgx.py b/dask_cuda/tests/test_dgx.py index 506d9cb03..5d155baa5 100644 --- a/dask_cuda/tests/test_dgx.py +++ b/dask_cuda/tests/test_dgx.py @@ -3,7 +3,7 @@ import subprocess import dask.array as da -from dask_cuda import DGX +from dask_cuda import LocalCUDACluster from distributed import Client import numpy @@ -86,12 +86,11 @@ def _get_dgx_net_devices(): def _test_default(): - with pytest.warns(DeprecationWarning): - with DGX() as cluster: - with Client(cluster): - res = da.from_array(numpy.arange(10000), chunks=(1000,)) - res = res.sum().compute() - assert res == 49995000 + with LocalCUDACluster() as cluster: + with Client(cluster): + res = da.from_array(numpy.arange(10000), chunks=(1000,)) + res = res.sum().compute() + assert res == 49995000 def test_default(): @@ -102,23 +101,22 @@ def test_default(): def _test_tcp_over_ucx(): - with pytest.warns(DeprecationWarning): - with DGX(enable_tcp_over_ucx=True) as cluster: - with Client(cluster) as client: - res = da.from_array(numpy.arange(10000), chunks=(1000,)) - res = res.sum().compute() - assert res == 49995000 + with LocalCUDACluster(enable_tcp_over_ucx=True) as cluster: + with Client(cluster) as client: + res = da.from_array(numpy.arange(10000), chunks=(1000,)) + res = res.sum().compute() + assert res == 49995000 - def check_ucx_options(): - conf = ucp.get_config() - assert "TLS" in conf - assert "tcp" in conf["TLS"] - assert "sockcm" in conf["TLS"] - assert "cuda_copy" in conf["TLS"] - assert "sockcm" in conf["SOCKADDR_TLS_PRIORITY"] - return True + def check_ucx_options(): + conf = ucp.get_config() + assert "TLS" in conf + assert "tcp" in conf["TLS"] + assert "sockcm" in conf["TLS"] + assert "cuda_copy" in conf["TLS"] + assert "sockcm" in conf["SOCKADDR_TLS_PRIORITY"] + return True - assert all(client.run(check_ucx_options).values()) + assert all(client.run(check_ucx_options).values()) def test_tcp_over_ucx(): @@ -129,12 +127,11 @@ def test_tcp_over_ucx(): def _test_tcp_only(): - with pytest.warns(DeprecationWarning): - with DGX(protocol="tcp") as cluster: - with Client(cluster): - res = da.from_array(numpy.arange(10000), chunks=(1000,)) - res = res.sum().compute() - assert res == 49995000 + with LocalCUDACluster(protocol="tcp") as cluster: + with Client(cluster): + res = da.from_array(numpy.arange(10000), chunks=(1000,)) + res = res.sum().compute() + assert res == 49995000 def test_tcp_only(): @@ -151,41 +148,40 @@ def _test_ucx_infiniband_nvlink(enable_infiniband, enable_nvlink): ucx_net_devices = "auto" if enable_infiniband else None - with pytest.warns(DeprecationWarning): - with DGX( - enable_tcp_over_ucx=True, - enable_infiniband=enable_infiniband, - enable_nvlink=enable_nvlink, - ucx_net_devices=ucx_net_devices, - ) as cluster: - with Client(cluster) as client: - res = da.from_array(cupy.arange(10000), chunks=(1000,), asarray=False) - res = res.sum().compute() - assert res == 49995000 - - def check_ucx_options(): - conf = ucp.get_config() - assert "TLS" in conf - assert "tcp" in conf["TLS"] - assert "sockcm" in conf["TLS"] - assert "cuda_copy" in conf["TLS"] - assert "sockcm" in conf["SOCKADDR_TLS_PRIORITY"] - if enable_nvlink: - assert "cuda_ipc" in conf["TLS"] - if enable_infiniband: - assert "rc" in conf["TLS"] - return True - + with LocalCUDACluster( + enable_tcp_over_ucx=True, + enable_infiniband=enable_infiniband, + enable_nvlink=enable_nvlink, + ucx_net_devices=ucx_net_devices, + ) as cluster: + with Client(cluster) as client: + res = da.from_array(cupy.arange(10000), chunks=(1000,), asarray=False) + res = res.sum().compute() + assert res == 49995000 + + def check_ucx_options(): + conf = ucp.get_config() + assert "TLS" in conf + assert "tcp" in conf["TLS"] + assert "sockcm" in conf["TLS"] + assert "cuda_copy" in conf["TLS"] + assert "sockcm" in conf["SOCKADDR_TLS_PRIORITY"] + if enable_nvlink: + assert "cuda_ipc" in conf["TLS"] if enable_infiniband: - assert all( - [ - cluster.worker_spec[k]["options"]["env"]["UCX_NET_DEVICES"] - == net_devices[k] - for k in cluster.worker_spec.keys() - ] - ) - - assert all(client.run(check_ucx_options).values()) + assert "rc" in conf["TLS"] + return True + + if enable_infiniband: + assert all( + [ + cluster.worker_spec[k]["options"]["env"]["UCX_NET_DEVICES"] + == net_devices[k] + for k in cluster.worker_spec.keys() + ] + ) + + assert all(client.run(check_ucx_options).values()) @pytest.mark.parametrize( From 44ebbf6a09a0e1ba30645406015eca9493444a77 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Fri, 1 May 2020 06:19:44 -0700 Subject: [PATCH 22/67] Remove special dask/distributed condition from test_device_host_file --- dask_cuda/tests/test_device_host_file.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/dask_cuda/tests/test_device_host_file.py b/dask_cuda/tests/test_device_host_file.py index 26c41d668..00e96d0a4 100644 --- a/dask_cuda/tests/test_device_host_file.py +++ b/dask_cuda/tests/test_device_host_file.py @@ -154,9 +154,7 @@ def test_serialize_cupy_collection(collection, length, value): else: obj = device_to_host(collection((x,) * length)) - if length > 5: - assert obj.header["serializer"] == "pickle" - elif length > 0: + if length > 0: assert all([h["serializer"] == "dask" for h in obj.header["sub-headers"]]) else: assert obj.header["serializer"] == "dask" From 55e06ab88ad195ffddadd6657de20de018d76f5a Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Fri, 1 May 2020 06:20:53 -0700 Subject: [PATCH 23/67] Remove DGX from __init__.py --- dask_cuda/__init__.py | 1 - dask_cuda/benchmarks/local_cudf_merge.py | 9 +++- .../benchmarks/local_cupy_transpose_sum.py | 48 ++++++++++++++++++- 3 files changed, 54 insertions(+), 4 deletions(-) diff --git a/dask_cuda/__init__.py b/dask_cuda/__init__.py index 1fee6824c..bbc1ac598 100644 --- a/dask_cuda/__init__.py +++ b/dask_cuda/__init__.py @@ -1,5 +1,4 @@ from ._version import get_versions -from .dgx import DGX from .local_cuda_cluster import LocalCUDACluster __version__ = get_versions()["version"] diff --git a/dask_cuda/benchmarks/local_cudf_merge.py b/dask_cuda/benchmarks/local_cudf_merge.py index d40ff6e68..45a2b28fa 100644 --- a/dask_cuda/benchmarks/local_cudf_merge.py +++ b/dask_cuda/benchmarks/local_cudf_merge.py @@ -183,7 +183,7 @@ def main(args): protocol=args.protocol, n_workers=args.n_workers, CUDA_VISIBLE_DEVICES=args.devs, - ucx_net_devices="auto", + ucx_net_devices=args.ucx_net_devices, enable_tcp_over_ucx=enable_tcp_over_ucx, enable_infiniband=enable_infiniband, enable_nvlink=enable_nvlink, @@ -387,6 +387,13 @@ def parse_args(): dest="enable_nvlink", help="Disable NVLink over ucx.", ) + parser.add_argument( + "--ucx-net-devices", + default=None, + type=str, + help="The device to be used for UCX communication, or 'auto'. " + "Ignored if protocol is 'tcp'", + ) parser.set_defaults( enable_tcp_over_ucx=True, enable_infiniband=True, enable_nvlink=True ) diff --git a/dask_cuda/benchmarks/local_cupy_transpose_sum.py b/dask_cuda/benchmarks/local_cupy_transpose_sum.py index 3ccf12da2..bd1b4d66e 100644 --- a/dask_cuda/benchmarks/local_cupy_transpose_sum.py +++ b/dask_cuda/benchmarks/local_cupy_transpose_sum.py @@ -20,8 +20,9 @@ async def run(args): n_workers=len(args.devs.split(",")), CUDA_VISIBLE_DEVICES=args.devs, ucx_net_devices="auto", - enable_infiniband=True, - enable_nvlink=True, + enable_tcp_over_ucx=args.enable_tcp_over_ucx, + enable_infiniband=args.enable_infiniband, + enable_nvlink=args.enable_nvlink, asynchronous=True, ) as cluster: async with Client(cluster, asynchronous=True) as client: @@ -153,7 +154,50 @@ def parse_args(): parser.add_argument( "--no-rmm-pool", action="store_true", help="Disable the RMM memory pool" ) + parser.add_argument( + "--enable-tcp-over-ucx", + action="store_true", + dest="enable_tcp_over_ucx", + help="Enable tcp over ucx.", + ) + parser.add_argument( + "--enable-infiniband", + action="store_true", + dest="enable_infiniband", + help="Enable infiniband over ucx.", + ) + parser.add_argument( + "--enable-nvlink", + action="store_true", + dest="enable_nvlink", + help="Enable NVLink over ucx.", + ) + parser.add_argument( + "--disable-tcp-over-ucx", + action="store_false", + dest="enable_tcp_over_ucx", + help="Disable tcp over ucx.", + ) + parser.add_argument( + "--disable-infiniband", + action="store_false", + dest="enable_infiniband", + help="Disable infiniband over ucx.", + ) + parser.add_argument( + "--disable-nvlink", + action="store_false", + dest="enable_nvlink", + help="Disable NVLink over ucx.", + ) + parser.set_defaults( + enable_tcp_over_ucx=True, enable_infiniband=True, enable_nvlink=True + ) args = parser.parse_args() + if args.protocol == "tcp": + args.enable_tcp_over_ucx = False + args.enable_infinibank = False + args.enable_nvlink = False return args From 8bc75e994432dc1e4d2a1381a8053408fa2c3f43 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Fri, 1 May 2020 06:50:17 -0700 Subject: [PATCH 24/67] Add --ucx-net-devices argument to CuPy benchmark --- dask_cuda/benchmarks/local_cupy_transpose_sum.py | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/dask_cuda/benchmarks/local_cupy_transpose_sum.py b/dask_cuda/benchmarks/local_cupy_transpose_sum.py index bd1b4d66e..cb498d482 100644 --- a/dask_cuda/benchmarks/local_cupy_transpose_sum.py +++ b/dask_cuda/benchmarks/local_cupy_transpose_sum.py @@ -19,7 +19,7 @@ async def run(args): protocol=args.protocol, n_workers=len(args.devs.split(",")), CUDA_VISIBLE_DEVICES=args.devs, - ucx_net_devices="auto", + ucx_net_devices=args.ucx_net_devices, enable_tcp_over_ucx=args.enable_tcp_over_ucx, enable_infiniband=args.enable_infiniband, enable_nvlink=args.enable_nvlink, @@ -190,6 +190,13 @@ def parse_args(): dest="enable_nvlink", help="Disable NVLink over ucx.", ) + parser.add_argument( + "--ucx-net-devices", + default=None, + type=str, + help="The device to be used for UCX communication, or 'auto'. " + "Ignored if protocol is 'tcp'", + ) parser.set_defaults( enable_tcp_over_ucx=True, enable_infiniband=True, enable_nvlink=True ) From 8edfb665e2451595a1b1e994265fd957ab61e236 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Fri, 1 May 2020 11:15:24 -0700 Subject: [PATCH 25/67] Add dask-cuda-worker RDMACM test --- dask_cuda/tests/test_dgx.py | 74 +++++++++++++++++++++++++++---------- 1 file changed, 55 insertions(+), 19 deletions(-) diff --git a/dask_cuda/tests/test_dgx.py b/dask_cuda/tests/test_dgx.py index 5d155baa5..48ad6f3cd 100644 --- a/dask_cuda/tests/test_dgx.py +++ b/dask_cuda/tests/test_dgx.py @@ -10,13 +10,13 @@ import pytest from time import sleep -from distributed.metrics import time -from distributed.utils_test import loop # noqa: F401 -from distributed.utils_test import popen from dask_cuda.utils import get_gpu_count from dask_cuda.initialize import initialize - +from distributed.metrics import time +from distributed.utils import get_ip_interface +from distributed.utils_test import popen from distributed.worker import get_worker +from tornado.ioloop import IOLoop mp = mp.get_context("spawn") ucp = pytest.importorskip("ucp") @@ -201,12 +201,38 @@ def test_ucx_infiniband_nvlink(params): assert not p.exitcode -def test_dask_cuda_worker_ucx_net_devices(loop): # noqa: F811 +def _test_dask_cuda_worker_ucx_net_devices(enable_rdmacm): + loop = IOLoop.current() + + cm_protocol = "rdmacm" if enable_rdmacm else "sockcm" net_devices = _get_dgx_net_devices() + sched_addr = "127.0.0.1" + + # Enable proper variables for scheduler sched_env = os.environ.copy() - sched_env["UCX_TLS"] = "rc,sockcm,tcp,cuda_copy" - sched_env["UCX_SOCKADDR_TLS_PRIORITY"] = "sockcm" + sched_env["DASK_UCX__INFINIBAND"] = "True" + sched_env["DASK_UCX__TCP"] = "True" + + if enable_rdmacm: + sched_env["DASK_UCX__RDMACM"] = "True" + sched_addr = get_ip_interface("ib0") + + sched_url = "ucx://" + sched_addr + ":9379" + + # Enable proper variables for workers + worker_ucx_opts = [ + "--enable-infiniband", + "--net-devices", + "auto", + ] + if enable_rdmacm: + worker_ucx_opts.append("--enable-rdmacm") + + # Enable proper variables for client + initialize( + enable_tcp_over_ucx=True, enable_infiniband=True, enable_rdmacm=enable_rdmacm + ) with subprocess.Popen( [ @@ -214,7 +240,7 @@ def test_dask_cuda_worker_ucx_net_devices(loop): # noqa: F811 "--protocol", "ucx", "--host", - "127.0.0.1", + sched_addr, "--port", "9379", "--no-dashboard", @@ -225,18 +251,9 @@ def test_dask_cuda_worker_ucx_net_devices(loop): # noqa: F811 sleep(5) with subprocess.Popen( - [ - "dask-cuda-worker", - "ucx://127.0.0.1:9379", - "--host", - "127.0.0.1", - "--enable-infiniband", - "--net-devices", - "auto", - "--no-dashboard", - ], + ["dask-cuda-worker", sched_url, "--no-dashboard",] + worker_ucx_opts ) as worker_proc: - with Client("ucx://127.0.0.1:9379", loop=loop) as client: + with Client(sched_url, loop=loop) as client: start = time() while True: @@ -246,6 +263,15 @@ def test_dask_cuda_worker_ucx_net_devices(loop): # noqa: F811 assert time() - start < 10 sleep(0.1) + workers_tls = client.run(lambda: ucp.get_config()["TLS"]) + workers_tls_priority = client.run( + lambda: ucp.get_config()["SOCKADDR_TLS_PRIORITY"] + ) + for tls, tls_priority in zip( + workers_tls.values(), workers_tls_priority.values() + ): + assert cm_protocol in tls + assert cm_protocol in tls_priority worker_net_devices = client.run(lambda: ucp.get_config()["NET_DEVICES"]) cuda_visible_devices = client.run( lambda: os.environ["CUDA_VISIBLE_DEVICES"] @@ -263,3 +289,13 @@ def test_dask_cuda_worker_ucx_net_devices(loop): # noqa: F811 # ensure timely closing. worker_proc.kill() sched_proc.kill() + + +@pytest.mark.parametrize("enable_rdmacm", [False, True]) +def test_dask_cuda_worker_ucx_net_devices(enable_rdmacm): + p = mp.Process( + target=_test_dask_cuda_worker_ucx_net_devices, args=(enable_rdmacm,), + ) + p.start() + p.join() + assert not p.exitcode From 0e4b3eaac17b0ac333ae974ac26e26d0a03a6121 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Fri, 1 May 2020 13:22:49 -0700 Subject: [PATCH 26/67] Add get_host_from_cuda_device test --- dask_cuda/tests/test_dgx.py | 30 +++++++++++++++++++++++++++++- 1 file changed, 29 insertions(+), 1 deletion(-) diff --git a/dask_cuda/tests/test_dgx.py b/dask_cuda/tests/test_dgx.py index 48ad6f3cd..49503d44b 100644 --- a/dask_cuda/tests/test_dgx.py +++ b/dask_cuda/tests/test_dgx.py @@ -10,7 +10,7 @@ import pytest from time import sleep -from dask_cuda.utils import get_gpu_count +from dask_cuda.utils import get_gpu_count, get_host_from_cuda_device from dask_cuda.initialize import initialize from distributed.metrics import time from distributed.utils import get_ip_interface @@ -299,3 +299,31 @@ def test_dask_cuda_worker_ucx_net_devices(enable_rdmacm): p.start() p.join() assert not p.exitcode + + +@pytest.mark.parametrize( + "params", + [ + {"host": "127.0.0.1", "enable_infiniband": False, "net_devices": None}, + {"host": "127.0.0.1", "enable_infiniband": True, "net_devices": None}, + {"host": "127.0.0.1", "enable_infiniband": True, "net_devices": "auto"}, + ], +) +def test_get_host_from_cuda_device(params): + host = params["host"] + enable_infiniband = params["enable_infiniband"] + net_devices = params["net_devices"] + dgx_net_devices = _get_dgx_net_devices() + + for i in range(get_gpu_count()): + ret_host = get_host_from_cuda_device( + host=host, + cuda_device_index=i, + enable_infiniband=enable_infiniband, + net_devices=net_devices, + ) + if enable_infiniband is True and net_devices == "auto": + ib = dgx_net_devices[i].split(",")[1] + assert ret_host == get_ip_interface(ib) + else: + assert ret_host == host From 2a3489ba61509e05d8b322cb86b554bea0918575 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Sat, 2 May 2020 05:31:11 -0700 Subject: [PATCH 27/67] Use get_host_from_cuda_device in LocalCUDACluster --- dask_cuda/local_cuda_cluster.py | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/dask_cuda/local_cuda_cluster.py b/dask_cuda/local_cuda_cluster.py index 72c4526b2..60b2b26a8 100644 --- a/dask_cuda/local_cuda_cluster.py +++ b/dask_cuda/local_cuda_cluster.py @@ -14,6 +14,7 @@ RMMPool, get_cpu_affinity, get_device_total_memory, + get_host_from_cuda_device, get_n_gpus, get_ucx_config, get_ucx_net_devices, @@ -206,6 +207,7 @@ def __init__( raise ValueError("ucx_net_devices can not be an empty string") self.ucx_net_devices = ucx_net_devices self.set_ucx_net_devices = enable_infiniband + self.host = kwargs.get("host", None) super().__init__( n_workers=0, @@ -259,10 +261,18 @@ def new_worker_spec(self): ) if self.set_ucx_net_devices: - net_dev = get_ucx_net_devices( - visible_devices.split(",")[0], self.ucx_net_devices - ) + cuda_device_index = visible_devices.split(",")[0] + + net_dev = get_ucx_net_devices(cuda_device_index, self.ucx_net_devices) if net_dev is not None: spec["options"]["env"]["UCX_NET_DEVICES"] = net_dev + spec["options"]["config"]["ucx"]["net-devices"] = net_dev + + spec["options"]["host"] = get_host_from_cuda_device( + host=self.host, + cuda_device_index=cuda_device_index, + enable_infiniband=self.set_ucx_net_devices, + net_devices=self.ucx_net_devices, + ) return {name: spec} From da84c0fdf28414f6f27312a08f32d093d9bc3c62 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Sat, 2 May 2020 05:31:57 -0700 Subject: [PATCH 28/67] Add RDMACM test for LocalCUDACluster --- dask_cuda/tests/test_dgx.py | 31 +++++++++++++++++++++++++------ 1 file changed, 25 insertions(+), 6 deletions(-) diff --git a/dask_cuda/tests/test_dgx.py b/dask_cuda/tests/test_dgx.py index 49503d44b..b74dab62d 100644 --- a/dask_cuda/tests/test_dgx.py +++ b/dask_cuda/tests/test_dgx.py @@ -141,17 +141,30 @@ def test_tcp_only(): assert not p.exitcode -def _test_ucx_infiniband_nvlink(enable_infiniband, enable_nvlink): +def _test_ucx_infiniband_nvlink(enable_infiniband, enable_nvlink, enable_rdmacm): cupy = pytest.importorskip("cupy") net_devices = _get_dgx_net_devices() + sched_cli_device = "ib0" ucx_net_devices = "auto" if enable_infiniband else None + cm_protocol = "rdmacm" if enable_rdmacm else "sockcm" + host = get_ip_interface(sched_cli_device) if enable_rdmacm else None + + initialize( + enable_tcp_over_ucx=True, + enable_infiniband=enable_infiniband, + enable_nvlink=enable_nvlink, + enable_rdmacm=enable_rdmacm, + net_devices=sched_cli_device, + ) with LocalCUDACluster( + host=host, enable_tcp_over_ucx=True, enable_infiniband=enable_infiniband, enable_nvlink=enable_nvlink, + enable_rdmacm=enable_rdmacm, ucx_net_devices=ucx_net_devices, ) as cluster: with Client(cluster) as client: @@ -163,9 +176,9 @@ def check_ucx_options(): conf = ucp.get_config() assert "TLS" in conf assert "tcp" in conf["TLS"] - assert "sockcm" in conf["TLS"] assert "cuda_copy" in conf["TLS"] - assert "sockcm" in conf["SOCKADDR_TLS_PRIORITY"] + assert cm_protocol in conf["TLS"] + assert cm_protocol in conf["SOCKADDR_TLS_PRIORITY"] if enable_nvlink: assert "cuda_ipc" in conf["TLS"] if enable_infiniband: @@ -187,14 +200,20 @@ def check_ucx_options(): @pytest.mark.parametrize( "params", [ - {"enable_infiniband": False, "enable_nvlink": False}, - {"enable_infiniband": True, "enable_nvlink": True}, + {"enable_infiniband": False, "enable_nvlink": False, "enable_rdmacm": False}, + {"enable_infiniband": True, "enable_nvlink": True, "enable_rdmacm": False}, + {"enable_infiniband": True, "enable_nvlink": False, "enable_rdmacm": True}, + {"enable_infiniband": True, "enable_nvlink": True, "enable_rdmacm": True}, ], ) def test_ucx_infiniband_nvlink(params): p = mp.Process( target=_test_ucx_infiniband_nvlink, - args=(params["enable_infiniband"], params["enable_nvlink"]), + args=( + params["enable_infiniband"], + params["enable_nvlink"], + params["enable_rdmacm"], + ), ) p.start() p.join() From f16ecfd2572df17eeee5cb0e2f46e1ee782fb622 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Sat, 2 May 2020 09:45:53 -0700 Subject: [PATCH 29/67] Default "network" type to disabled in get_ucx_net_devices Having the "network" interface causes issues with RDMACM, using "openfabrics" alone seems to solve the issue. The issue seen in question happens when launching CUDA workers with multiple IB devices and the error is below: wireup_ep.c:486 UCX ERROR no supported sockaddr auxiliary transports found for mlx5_1:1 --- dask_cuda/utils.py | 20 +++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/dask_cuda/utils.py b/dask_cuda/utils.py index e97af5512..1e416c4be 100644 --- a/dask_cuda/utils.py +++ b/dask_cuda/utils.py @@ -155,7 +155,9 @@ def get_device_total_memory(index=0): ).total -def get_ucx_net_devices(cuda_device_index, ucx_net_devices): +def get_ucx_net_devices( + cuda_device_index, ucx_net_devices, get_openfabrics=True, get_network=False +): if cuda_device_index is None and ( callable(ucx_net_devices) or ucx_net_devices == "auto" ): @@ -177,12 +179,16 @@ def get_ucx_net_devices(cuda_device_index, ucx_net_devices): net_dev = "" td = TopologicalDistance() - ibs = td.get_cuda_distances_from_device_index(dev, "openfabrics") - if len(ibs) > 0: - net_dev += ibs[0]["name"] + ":1," - ifnames = td.get_cuda_distances_from_device_index(dev, "network") - if len(ifnames) > 0: - net_dev += ifnames[0]["name"] + if get_openfabrics: + ibs = td.get_cuda_distances_from_device_index(dev, "openfabrics") + if len(ibs) > 0: + net_dev += ibs[0]["name"] + ":1" + if get_network: + ifnames = td.get_cuda_distances_from_device_index(dev, "network") + if len(ifnames) > 0: + if len(net_dev) > 0: + net_dev += "," + net_dev += ifnames[0]["name"] else: net_dev = ucx_net_devices return net_dev From ea893fc73541ec6f1f737490cf248071f51afc35 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Sat, 2 May 2020 15:24:57 -0700 Subject: [PATCH 30/67] Update get_host_from_cuda_device --- dask_cuda/utils.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dask_cuda/utils.py b/dask_cuda/utils.py index 1e416c4be..0a7fc1bb4 100644 --- a/dask_cuda/utils.py +++ b/dask_cuda/utils.py @@ -342,7 +342,9 @@ def get_host_from_cuda_device(host, cuda_device_index, enable_infiniband, net_de '10.33.225.162' """ if enable_infiniband and net_devices == "auto": - devices = get_ucx_net_devices(cuda_device_index, "auto").split(",") + devices = get_ucx_net_devices( + cuda_device_index, "auto", get_network=True + ).split(",") for d in devices: if d.startswith("ib"): host = get_ip_interface(d) From 74018bccbf60df3c6be89d82c18d7385328fe7a5 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Sat, 2 May 2020 15:25:17 -0700 Subject: [PATCH 31/67] Fix RDMACM tests --- dask_cuda/tests/test_dgx.py | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/dask_cuda/tests/test_dgx.py b/dask_cuda/tests/test_dgx.py index b74dab62d..b23954b95 100644 --- a/dask_cuda/tests/test_dgx.py +++ b/dask_cuda/tests/test_dgx.py @@ -145,22 +145,21 @@ def _test_ucx_infiniband_nvlink(enable_infiniband, enable_nvlink, enable_rdmacm) cupy = pytest.importorskip("cupy") net_devices = _get_dgx_net_devices() - sched_cli_device = "ib0" + openfabrics_devices = [d.split(",")[0] for d in net_devices] + network_devices = [d.split(",")[1] for d in net_devices] ucx_net_devices = "auto" if enable_infiniband else None cm_protocol = "rdmacm" if enable_rdmacm else "sockcm" - host = get_ip_interface(sched_cli_device) if enable_rdmacm else None initialize( enable_tcp_over_ucx=True, enable_infiniband=enable_infiniband, enable_nvlink=enable_nvlink, enable_rdmacm=enable_rdmacm, - net_devices=sched_cli_device, ) with LocalCUDACluster( - host=host, + interface="ib0", enable_tcp_over_ucx=True, enable_infiniband=enable_infiniband, enable_nvlink=enable_nvlink, @@ -189,7 +188,7 @@ def check_ucx_options(): assert all( [ cluster.worker_spec[k]["options"]["env"]["UCX_NET_DEVICES"] - == net_devices[k] + == openfabrics_devices[k].split(",")[0] for k in cluster.worker_spec.keys() ] ) @@ -225,6 +224,8 @@ def _test_dask_cuda_worker_ucx_net_devices(enable_rdmacm): cm_protocol = "rdmacm" if enable_rdmacm else "sockcm" net_devices = _get_dgx_net_devices() + openfabrics_devices = [d.split(",")[0] for d in net_devices] + network_devices = [d.split(",")[1] for d in net_devices] sched_addr = "127.0.0.1" @@ -301,7 +302,7 @@ def _test_dask_cuda_worker_ucx_net_devices(enable_rdmacm): ): net_dev = v[0] dev_idx = int(v[1].split(",")[0]) - assert net_dev == net_devices[dev_idx] + assert net_dev == openfabrics_devices[dev_idx] # A dask-worker with UCX protocol will not close until some work # is dispatched, therefore we kill the worker and scheduler to From d3f711f4f8316b59555f530157b04dc3b82d5413 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Sun, 3 May 2020 14:36:11 -0700 Subject: [PATCH 32/67] Replace use of host by interface to select listener IB device --- dask_cuda/dask_cuda_worker.py | 10 +++++----- dask_cuda/local_cuda_cluster.py | 11 +++++------ 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/dask_cuda/dask_cuda_worker.py b/dask_cuda/dask_cuda_worker.py index 6c0da4f36..869d85fdc 100755 --- a/dask_cuda/dask_cuda_worker.py +++ b/dask_cuda/dask_cuda_worker.py @@ -30,9 +30,9 @@ RMMPool, get_cpu_affinity, get_device_total_memory, - get_host_from_cuda_device, get_n_gpus, get_ucx_config, + get_ucx_net_devices, ) logger = logging.getLogger(__name__) @@ -310,11 +310,11 @@ def del_pid_file(): loop=loop, resources=resources, memory_limit=memory_limit, - host=get_host_from_cuda_device( - host=host, + interface=get_ucx_net_devices( cuda_device_index=i, - enable_infiniband=enable_infiniband, - net_devices=net_devices, + ucx_net_devices=net_devices, + get_openfabrics=False, + get_network=True, ), preload=(list(preload) or []) + ["dask_cuda.initialize"], preload_argv=(list(preload_argv) or []) + ["--create-cuda-context"], diff --git a/dask_cuda/local_cuda_cluster.py b/dask_cuda/local_cuda_cluster.py index 60b2b26a8..7e40b6d6a 100644 --- a/dask_cuda/local_cuda_cluster.py +++ b/dask_cuda/local_cuda_cluster.py @@ -14,7 +14,6 @@ RMMPool, get_cpu_affinity, get_device_total_memory, - get_host_from_cuda_device, get_n_gpus, get_ucx_config, get_ucx_net_devices, @@ -268,11 +267,11 @@ def new_worker_spec(self): spec["options"]["env"]["UCX_NET_DEVICES"] = net_dev spec["options"]["config"]["ucx"]["net-devices"] = net_dev - spec["options"]["host"] = get_host_from_cuda_device( - host=self.host, - cuda_device_index=cuda_device_index, - enable_infiniband=self.set_ucx_net_devices, - net_devices=self.ucx_net_devices, + spec["options"]["interface"] = get_ucx_net_devices( + cuda_device_index, + self.ucx_net_devices, + get_openfabrics=False, + get_network=True, ) return {name: spec} From 149db315b19a6f5f62bfcdb481b0464303f93673 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Sun, 3 May 2020 14:42:03 -0700 Subject: [PATCH 33/67] Remove get_host_from_cuda_device This is not necessary anymore, given the utilization of Dask's host argument has been replaced by interface. --- dask_cuda/tests/test_dgx.py | 28 ---------------------- dask_cuda/utils.py | 47 ------------------------------------- 2 files changed, 75 deletions(-) diff --git a/dask_cuda/tests/test_dgx.py b/dask_cuda/tests/test_dgx.py index b23954b95..58b4f5452 100644 --- a/dask_cuda/tests/test_dgx.py +++ b/dask_cuda/tests/test_dgx.py @@ -319,31 +319,3 @@ def test_dask_cuda_worker_ucx_net_devices(enable_rdmacm): p.start() p.join() assert not p.exitcode - - -@pytest.mark.parametrize( - "params", - [ - {"host": "127.0.0.1", "enable_infiniband": False, "net_devices": None}, - {"host": "127.0.0.1", "enable_infiniband": True, "net_devices": None}, - {"host": "127.0.0.1", "enable_infiniband": True, "net_devices": "auto"}, - ], -) -def test_get_host_from_cuda_device(params): - host = params["host"] - enable_infiniband = params["enable_infiniband"] - net_devices = params["net_devices"] - dgx_net_devices = _get_dgx_net_devices() - - for i in range(get_gpu_count()): - ret_host = get_host_from_cuda_device( - host=host, - cuda_device_index=i, - enable_infiniband=enable_infiniband, - net_devices=net_devices, - ) - if enable_infiniband is True and net_devices == "auto": - ib = dgx_net_devices[i].split(",")[1] - assert ret_host == get_ip_interface(ib) - else: - assert ret_host == host diff --git a/dask_cuda/utils.py b/dask_cuda/utils.py index 0a7fc1bb4..e50d8aefc 100644 --- a/dask_cuda/utils.py +++ b/dask_cuda/utils.py @@ -303,50 +303,3 @@ def get_preload_options( preload_options["preload_argv"].extend(initialize_ucx_argv) return preload_options - - -def get_host_from_cuda_device(host, cuda_device_index, enable_infiniband, net_devices): - """ - Return the host address for a given CUDA device if enable_infinibad=True - and net_devices='auto', otherwise return the original host parameter. - This function is used to identify the host when InfiniBand interfaces - are available on the system and automatic detection of topologically - closest ones are requested. Currently requires that at least one of the - topologically closest devices is an InfiniBand and has a network address, - such as 'ib0'. - - Parameters - ---------- - host: None or str - Host to use if one cannot be identified automatically, when - net_devices is not "auto" or enable_infiniband=False. - cuda_device_index: int - The index of the CUDA device used to identify the host. - ucx_net_devices: str or callable - A string with value 'auto' to attempt identifying the host, otherwise - returns the value of host argument. - enable_infiniband: bool - True to attempt identifying the host, otherwise return the value of - host argument. - Set environment variables to enable UCX InfiniBand support. Implies - enable_tcp=True. - - Example - ------- - >>> from dask_cuda.utils import get_host_from_cuda_device - >>> get_host_from_cuda_device('127.0.0.1', 0, None, False) - '127.0.0.1' - >>> get_host_from_cuda_device('127.0.0.1', 0, "auto", False) - '127.0.0.1' - >>> get_host_from_cuda_device('127.0.0.1', 0, "auto", True) - '10.33.225.162' - """ - if enable_infiniband and net_devices == "auto": - devices = get_ucx_net_devices( - cuda_device_index, "auto", get_network=True - ).split(",") - for d in devices: - if d.startswith("ib"): - host = get_ip_interface(d) - break - return host From 83f39e15c99a9adbd7750845de43f79a8ae6cd79 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Mon, 4 May 2020 10:25:50 +0200 Subject: [PATCH 34/67] Apply suggestions from code review Co-authored-by: Benjamin Zaitlen --- dask_cuda/tests/test_dgx.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/dask_cuda/tests/test_dgx.py b/dask_cuda/tests/test_dgx.py index 58b4f5452..299972f3a 100644 --- a/dask_cuda/tests/test_dgx.py +++ b/dask_cuda/tests/test_dgx.py @@ -10,12 +10,10 @@ import pytest from time import sleep -from dask_cuda.utils import get_gpu_count, get_host_from_cuda_device +from dask_cuda.utils import get_gpu_count from dask_cuda.initialize import initialize from distributed.metrics import time -from distributed.utils import get_ip_interface from distributed.utils_test import popen -from distributed.worker import get_worker from tornado.ioloop import IOLoop mp = mp.get_context("spawn") From 95c6be5ea845a2b49dc92c6266b32cf520b8a23c Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Mon, 4 May 2020 10:29:39 +0200 Subject: [PATCH 35/67] Apply more suggestions from code review --- dask_cuda/tests/test_dgx.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/dask_cuda/tests/test_dgx.py b/dask_cuda/tests/test_dgx.py index 299972f3a..dbb843c8f 100644 --- a/dask_cuda/tests/test_dgx.py +++ b/dask_cuda/tests/test_dgx.py @@ -144,7 +144,6 @@ def _test_ucx_infiniband_nvlink(enable_infiniband, enable_nvlink, enable_rdmacm) net_devices = _get_dgx_net_devices() openfabrics_devices = [d.split(",")[0] for d in net_devices] - network_devices = [d.split(",")[1] for d in net_devices] ucx_net_devices = "auto" if enable_infiniband else None cm_protocol = "rdmacm" if enable_rdmacm else "sockcm" @@ -223,7 +222,6 @@ def _test_dask_cuda_worker_ucx_net_devices(enable_rdmacm): cm_protocol = "rdmacm" if enable_rdmacm else "sockcm" net_devices = _get_dgx_net_devices() openfabrics_devices = [d.split(",")[0] for d in net_devices] - network_devices = [d.split(",")[1] for d in net_devices] sched_addr = "127.0.0.1" From 658a77d563095313c212b3542e689bfac540a829 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Mon, 4 May 2020 01:33:38 -0700 Subject: [PATCH 36/67] Fix test_dgx imports --- dask_cuda/tests/test_dgx.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dask_cuda/tests/test_dgx.py b/dask_cuda/tests/test_dgx.py index dbb843c8f..6cee0bb2a 100644 --- a/dask_cuda/tests/test_dgx.py +++ b/dask_cuda/tests/test_dgx.py @@ -13,7 +13,7 @@ from dask_cuda.utils import get_gpu_count from dask_cuda.initialize import initialize from distributed.metrics import time -from distributed.utils_test import popen +from distributed.utils import get_ip_interface from tornado.ioloop import IOLoop mp = mp.get_context("spawn") From c787ba63d14466182c338f71f19ed830ff047452 Mon Sep 17 00:00:00 2001 From: Benjamin Zaitlen Date: Tue, 5 May 2020 15:27:23 -0400 Subject: [PATCH 37/67] initial docs setup --- conda/environments/builddocs_py37.yml | 14 ++ docs/Makefile | 19 +++ docs/source/_static/Architecture.png | Bin 0 -> 148726 bytes docs/source/api.rst | 28 ++++ docs/source/conf.py | 182 ++++++++++++++++++++++++++ docs/source/index.rst | 14 ++ docs/source/install.rst | 91 +++++++++++++ docs/source/quickstart.rst | 27 ++++ docs/source/worker.rst | 106 +++++++++++++++ 9 files changed, 481 insertions(+) create mode 100644 conda/environments/builddocs_py37.yml create mode 100644 docs/Makefile create mode 100644 docs/source/_static/Architecture.png create mode 100644 docs/source/api.rst create mode 100644 docs/source/conf.py create mode 100644 docs/source/index.rst create mode 100644 docs/source/install.rst create mode 100644 docs/source/quickstart.rst create mode 100644 docs/source/worker.rst diff --git a/conda/environments/builddocs_py37.yml b/conda/environments/builddocs_py37.yml new file mode 100644 index 000000000..9526681cb --- /dev/null +++ b/conda/environments/builddocs_py37.yml @@ -0,0 +1,14 @@ +name: dask_cuda_docs +channels: +- rapidsai-nightly +- conda-forge +dependencies: +# required for building docs +- sphinx +- sphinx_rtd_theme +- sphinxcontrib-websupport +- nbsphinx +- numpydoc +- recommonmark +- pandoc=<2.0.0 +- pip \ No newline at end of file diff --git a/docs/Makefile b/docs/Makefile new file mode 100644 index 000000000..69fe55ecf --- /dev/null +++ b/docs/Makefile @@ -0,0 +1,19 @@ +# Minimal makefile for Sphinx documentation +# + +# You can set these variables from the command line. +SPHINXOPTS = +SPHINXBUILD = sphinx-build +SOURCEDIR = source +BUILDDIR = build + +# Put it first so that "make" without argument is like "make help". +help: + @$(SPHINXBUILD) -M help "$(SOURCEDIR)" "$(BUILDDIR)" $(SPHINXOPTS) $(O) + +.PHONY: help Makefile + +# Catch-all target: route all unknown targets to Sphinx using the new +# "make mode" option. $(O) is meant as a shortcut for $(SPHINXOPTS). +%: Makefile + @$(SPHINXBUILD) -M $@ "$(SOURCEDIR)" "$(BUILDDIR)" $(SPHINXOPTS) $(O) \ No newline at end of file diff --git a/docs/source/_static/Architecture.png b/docs/source/_static/Architecture.png new file mode 100644 index 0000000000000000000000000000000000000000..dd09d211c69f3e8c6700b8dfa1f3be9ec97686f6 GIT binary patch literal 148726 zcmeGEby$?$`UVUGqO=U4gyaCyFpP9JC?OzpkATw#UKYaI@}v-RoM{y4HDJ=eZ_aj6@T3h{3gL@<>uxj3W0cdc<^}e z@i@9zLZBidA`sqtkbC#Iffn4Zo(^u$VB8L_On*P*_jBaUUCmsqo!qP)9T={j`^?nQ z-A#g#@v5VL{rmep-K;JC-IIgsuVDcLgk1dv0_EX_{Oj4kt>RZ#MIV{FI@-BkJ^qP< zwVUKU@xL1X_vL^0_V-5ZIS}NZu!q^|J`2Q#o8Pg-&Hq~&|mHU^V)y6 z*RXPQa|CSdVr}-o!Oh$S=ZcmaAt-2+Vu^dbX}c^yP*p4%y+rM81YF7*+|}2yF%0Q*sMhR-`t9e@8%n0APS@6B-^xkv zsb#OXMC{q}$Yfqpu&hRNp@Lf5U8#Vv%8RNZId@L`&tcXn& zeM@@$2g~%a@*49|C(u5CtS%!<*KzR&-52hRwN-tu+c4F$(NS6WdiS3<;6YRDWm^m6 zC)4*HHF{H3I-Xn)&9U-6shL%pdoM9N#Qk%yl731m%_k=&CpV%O=Ff-4JxY0gE0|JH zT=CD_Jprk-7~jqPK!XtXXsrEmMA`a$RU)6TJD9KHpJ$7LL|RO1+pgk9t&Co=%!gZ{ z7(Z`lmSKJSgXs*_q|b}5ryFB_zD`}B9Cs_{W3zL@KR@{?nTD9{*X5`F4qp{3?*CyX zfE6kl;E(0TT9xbm=hZqG2za1=xh7ZqKX!Fxy?}=dIT$l|C04Y5#z0jJnAcay?QTUf z{}CnSkm0gPd&qL?;ruyza^IJ}8h$(DA6=4k6&nkGFmo_sOy&=sMEY8qfRyrkp!ui& zw2Yyw3E;WUh7>6JY)^mKmIYH7!QK|wO1dn5<2CPo!)w`}v9mv-UhAgIoq-(^-^>`v zMM|z`_4e%bfcTzjikQci2HmXL@7Ee^xw2ZFGgsQ*R?GR0>wVlxfAeEWjb65CrZ0MJ zVfN+rW~D=&AAH+qn_#`T^5J_*)(j_E*#3}GD`@w5fnNu#Aa z&SCuGhjr0TQZ2mTamH0#TDUmfhMaHZL+H_ip)DcQ4k=I9;*{v{Lg{^BDUv>36W)7$ zD&OG;WBRT2D_Qzz+9uUBe)f%uPP&4_42hR7gg44tWgQ(Ib$kxKx7JKKP$%&0JV;x~ z_LJ;Q6X2YFF=fAyeR;NmWsQg!%2Nt0srkWC{HdFx{x}LZx&i(33JdErBV|z2#t_DTnys zgxSL3lp`^*#J)!Me&0{A2re`Y|EWxUme1kjnRxI*pU_-J_FTJL$N$u-mw zOP}qhaK8;H_+z5N3^@)yH*o2?+o~jm9vX&{B=R)u4-Y$b-&T02{b=hUzx*(pq*fDW z8T?)u3!}vk-?gJR*D}cU?JC@!tev(HK&Dw^heH+#P9LTVa@&=Ey>TLP_GRvz=g971 z{G#kujM$dW?(oA}=d(m@J$B<1UchTEKybprj7`n^p-B_{u0V9DtUw?`L+yO(kplISs&nz?_37EFwZ*# z$rB$GtQGKsv?pHh47%q@AfGr=@MYG@pMBmUL;B<{bJjMVao?PcaW){kP?%sdFL?_ zhZDnKD<)ZP(3FTk9it%Q8L1JIz>i>F+BjmPtTuAr)2TTlWkHAOI`PMSPo5e+(Gwbk zTx;06;RO%a2%V=XW5?F&@6;AT$5Y0Na&s;)r38g3O6`m{T?kz2s?2>=Ij&<#ou0pA zMjz8|>BTeSIv`%yq0Bxj{&%ne* zI4fp1@<>?JZ|`gEi>>w0t~rV*U0xhDt&B56+`Y%Y%sx3pFQyejOt3=mm`cYz8uu~@ zXd-l0CLn$%J!f8}MUIiFQ0EX2k&5q7mJ*ZB#Uz$rSVj7X^qn{BWP@sP{qNo5Iu=a0 z=moH$?V|=2+&J)GSfh_=;|kd%E2G_~9GiUF#lJ~Ke1w)Ahkt9(CoifkDq=0YGsudC zkUZVc()4E0k{n|l- zgg8+ z((P+>bx|T)&Clt~i@)C(sU-QJKh;irhnV`y)rq0WLmetJ(FhC=5*Tx-LK-A0~&%R8n-S!f~SeA{@jpLO4g?!ws3 zz4%_tc7FlWu+JoXaPEHI3}`b#vF^AY zVOdD-4E#ad!bBDkQ_0qY@J>+`0pw>!RW)wKxvwHx{H0|!34-nW=jY3!McZNIWetc$ zdiAIsd8`!}Iv>-%9h+<_W&f?Za!+)T09|hoF+vaUOOTa?DY2;?nZ@U^^ zfnfsb84i`d&H59(GgjMJt?q&#WRhg;&&T3NkOkhauj{tHAvEKr1)v+n3sRvLErw-9 zxqT*W8xd$MFYOf!*rRABLHMVJZ|0abpJ>Sc|%pmvx*Wtl; zu#?Y_@#n#R-}by+jD{%)_wBu0FdHMGCXV=5snf}U4mERRW0D|?F|1AUc3b!p)d_+h z*huKv5qw3WKq72AYBk8|34c%_Q>^$sM-_Pvc+&!i2uh84XJ986q+l>j84~oG5$3OG zb}uIlBbp&O!gS*)w@V@?>M*6D+#UrNhYClQlt>)r;NarbSe0I7v*X3lmo~fx53j zGS~YZb=-_s#FD}k@u`o0_+6f#(xin$x;tg>$GRD}c(>`CwRfdnplkLS+HDJt`DN0^ zwtpv6Za6?fvt?yz%l>zepyUlOrSw0RWtTUVUsx6CVVVlFz^zOqbVdVN1H@A}+jBY9 zb0Rtg=M=p9hAf9f0tIgeDMq4RY7I1tYts6G^tHam?OwBXOy8JSIkn#mCFRR3b&>_>0ANr`u z%}LW!3{ap++R*!HQP%6{s;3Nx1=$E#TXXCR#|pPPWKd95`ZjkVXDQ@@PIa!M{`DR8 zr1B~koun$bZ}P9iI|l`%O4I|RG{2Aers80Y5qE}iWCKnc6M}D0-rYmplD);d^J`-c)nyqTPuLxI@ zbKrVihsha{6$DR#3be7-^LWx%{^93-^G=L#OD)!j{?!Y zzuo@9P5&&$-*?t?r9Hbq{bxw1HB;0Mo2rY6YVoaL(Qd=2uTx>jq@KK?al?8u!WXpT z_L}}No-XI3!==!34ZnNV360G0>*NC_SyuNO&OLSM-!iQhap-!L#4Q6U+S-T*gDKzJ zF-bw2R8c;x+XIkYo{wU`k{3I%{S|zPN!@V^}!}w;A2vo)>V)3?$oG zn-=eX*-K6~XEnG>=3~RRh#i8kcm-!ZazT6+lu)*UGb<4^uCP@_s2br|Q?V@#-?+gT z;UfN1nJs#BFX5n%7arnit!L8~!=qPmq0k7Ga5mEV$U=jBR=?LTmdrvoe13e>!w8{x z{HFD$Sn-Er9lJSG&q&Wb>V?|rNPN!l<)vK{S~7+q&6grT4uMw`ZB)}hv5m*F3gYm~lL5cw zXYN3{T&c0aIU>&CLvJ^yox#QV5cWj0bp2n6km^-^WU_Qj`ELsWxdY5Si7e{0Cxw?L z5qmZ9sgVeUU0yQ72~r?akK5YDI}s-i@8Q1WkVXcid2J!0b)!y2Cpu>r zGD3*wmlELjNd`1pVl+bb&0oLQFfOITeGyhy(ZANXG)z1$Dw_)IU@L87`i>3F^=c_0 zRcdUIDBhnbMsiSonZ}BiYzqUAwYQ)%t$=Fblla1t!^|Bq*gQqB;-O3iPej4@-4B-f zdELfaDF!R-)Wh@KhJ6t}I4e>Rs^|shoeYRk!uMTH2PBQG$@2C5_aw{Mv1j`Qd3LM@ zeGp$%tB8KpD)nz`Q%BN{uzDlNvT@=3@vcFKQRm9_=aK8fpN2YlG2djpf}XpN8z%cj zk82vxvu1z-M2sU7n7*UIXl|C>8&kEndmitT#+n~!{q5om0b5`oXiC`ItNmI90%X#! zuZl~j`Autu=`IU(1Kk@qbm8k|O}=zE!VYq;i9&XHh7`W&53ejVX^zU7Fk{c)aQ80#agJr*R%AzJaJN^5f@NCdgd^mK(KZ?~UJ@~`1 z>D*<3dO_I7ntJM#Y~#LM7JP^M&H?LlX-+EpmkYrZA@;R1HR=^0+>r(FagKwFNM4EE|S((jAK5Zlx|0eyqu?k)Ovz8($$im06Mp25K$R+NJ(xnvD zv|Fs@-J42&^6A=qS<7{2yFJ&y_i|CpvJ&djZRf|u1$MkK2?0Sw52AH8V9#7L3llV< zYSq40wKFezSV6ugJ;pcd9FfARZ28(7a>Mrj-qyu%_lcOtXx(d)elM4OISglBpP;nG zFz6ED+fquVL?@~Pd^uQimfyw6(#@f5(KdEC4~`k`#Cdo%u5A!S`0SX_oVRPX9HV$5 zS@r{Yu1N(7LwEY8}~u|{a48jD7|+@y{*nO-?bV|f4H=me(CVz8H;`) z4wPaeKK5KF`Y*2qz2!a5a%#aVNUkN_4fO>Jh^GR1SN?p9MQ@w- zb%~i}kKbjb&#!=@x<^%I^_#yD&ZvSv@3r>}i8Ko1Za!xZ?+1?VL~$P>xZwuj(aMw| zqo5~Y`g;4zDkGrFR9=SnVEEY*#1GFh+vmk+FH9L0{F;3A20oI{4oAG3~n!(wqS1ZUs)5Hx1*VHg{BaGu-_2P@2yDA$=&)()EZ+Nji8T{Y^x`GkY_!Wmo1zO42rp>;H$@WOR9__hsDx4Itcq+SB+ ztttj?Y0w-ujC+a}JMVWMSqCTO9qUfBUhu4J(kNID=X-YgeEagP00wtyLSCTw3LnIh zjNg{B6``|U^{Z=CF8KJ6(o*cfeE zggRN{<2ZP~^mASG_mqhyNUK|Fe`K<)cGCX1)QF09+;3>z{`NtS{jV0Qx3Jo70yr>J z<@-W~bDL?Kjy!%4rMP<}0#6KdfmOC(1=LB8T)!rD_MjT?j=Zp%hOd@2 z;Mq&Vt*~WSpdnMRdIUf)WKtk4rA~F9IQ!J~=xrUBXkVBo=@zPUy)2b0-GtG$i2A&} zOD52!>6q_q??PCrR{8+_Hdn`Z4VE#!fA>wQPuL`l!mih8jkWICB~Wn7b7RX26Gley zDL0)n{we?q2GF0cDz%ZEdDOrMH_TG~HnA_rqRJ0_RDUH#0JU^O0b}9Qs(_(wNZ6Y` zat;zL-JP-0b`V(irvSD+oLcvW=|Pe!L+AY?nMI2hfAjJXR!GCgE?4mcWj~| zRK>xV6(M<{MrNPK>OZKM_8M3PsIkCCw}kW@R9!)x6(0F}{etkJoM17^xC zTKB(49g?Y6a+XoX1#J|&Kf9N8w5TE?t~yKj{L=>mqmr5!RxQMV>LjkjqqTxhCE71~ zn0}X;T=6KceXjTJGyJt}0oQY|FpYM}B_N&^;1)wxD!Ovm=^kk(Q;cx#-8Q4{HZtrs zVHO9EFS?vb*~0s8K1#{KB$wW3p;g9erfPwT+;ESjrzij$sOt}WEDb`+-=Rd%a zn?LWAy3mBafF<127nqW}DUx5%tyNuaE?8rF;`yEhg2O!BamThet}`%Jp=g^?QTUlY z==tf(HIwNwS}6+M=W&n3olVa>y9!vh$b znj=qg!HiG6;*3?vq&&go5UCfS1GZsMMEQ3#Rq-FZe-mCxZfWU7^Zv>kLN@G;Q_M@CWKZ4 z*uR6rfBfb5M=N{)_!XK6X#M|bqv3x@^JiL>A)pYX;qQR2&6oIV{Qo2!yKXh`YtH_$ z(f@m|F}7^}0}xCo@3{zB{$Wg2^q2ybWA=Oi;GU&E9>*wfgIG-3QwK4ohQe`hq+; zvo8UdxZC*g{MQ8jw?PB8SO8f-&q3uD%aB(!z;`zYzJHwSITUX}a1g;x!jxD*`FzPwB$SdMfVQ_rB%a- zCM@Cl$gD^8gX&J&m@KwpjfO6l@+CHTcYIW)fUWkx72>i5I7sgvKpUb{^qVC9FTFy} zM_l*?KtCF|4JkAMe5n3q+>^5*yVOK(+@X%ZeukUd)d^ec9V}W zX^@SJ)!3f(YFq!8fBb9diD5xAZe+tvKmn;b0|S;l36p_4QMl)-HRY0p1C2H%(imSK z+0tVfNQ)~rBe5wdra31+>C~w9Cb=-W)#xSs<3f5ZJw?hEF#xd;@D$b0`bI9lO6ets z5_vUbQy6OZvHHIK($^kOw;I)NoLV%G211z*`5t%hW6{Ib2eO22&^+%5aH3Aj26U9K=kX>2!OKTLJ6U z-NC);G9UY%I3^`ut{$tAtgI|^ua_@bZwhK1B}*59zLkbkRN5lqiY1c=fOxDfb<(BI z^R4mdh4Oit0r6=v%QeB%Ll%?E;hIg$xv?5G%deq<{ytM!Z#MbJr4jgeXQBQ+9ez5; zC3{yIBcpV^JT%_^M(v#M$5=WE^gFw8we`6f3Fg6SY@Te1mxpvX^xiv}RG7i5yF~%# zDxN&0iPC=98^&lRe)!bpR7i4@lA|%!%#Yeh%c^Y=+pf@L>Doi8R+X7NpS~gYv*{DJbFjY9(t}4$sg2i zrH^$8DVcOTv@e!v?vkv$~2Qh3*14Qpo1vdV%}=YtHHhCdy@ET5Gb7E|L+ zFr-+-b9b?)Qj|@y)>%$({~7Df2H${qc^W(%@+mkKYA;))~kZ=TKi$m3|<-L($j`W zFH~A+pQ=P|DxUveh5=EY0F}y%^GO1nx!D`$)nTMoo>&Gd23oo@N;Tfk(v780lb87p zaSi2x$C}@nYAi8NYYBA23gB6t+TasikgvYUEtf-pj!JrFFOwz*a9|<^ebv=?ywp+_ zh_-Ng`%P0lQM@XOfRpjjl;Y8ujqCQ8!$w6V1@>fsd0S^S4((fKvB`7D@Z6lP%p1;H z?0)A37VWepp_o3wpf>;l`Ybm{^kOkq&g+5!jTdr^Gwp7?^eJhIpIUT}&0yc9{)X|7 zqgyqL+^WqN<`dMNjrJ4i%~)=X0XMg85 zCXcntE0`3cbUr;Hu&QSsjKN+u)-tR7cm-TK=dDEz}hQgmvwS; zrC9~98!N`r(OyX`$^nQF>WmDj5la_!x6!eyXuspOT;(U2Dx|FAjcS*Bp{3zr{%M^> zHA`G1ineDuT07kTT;=XPxfe3+wxFz)gJ=0G*>@q!WU`_wnHqW%{DRUG#)i<6J4oVz8NV#`p|uq zG4CWdo!#xtlK4_pI%Iq$ASmDZyuNJ8=_v&W0u~(oPW9^mu}dq*$6``RfL{y&i09xd zwoYn4|3N|vAk{eFb)1Fo_+1|PJp*`XJZk$Y%|?3wv@OJ|@N}*3G;I4oqV60h)_Tte z;DppRoo_zd>k}FnxqW%IUnE7e?KS?zrh5fIH(Xmr=<`iSKd~r!DakYRc+*W{@g3<# z-#MQnA{Czc=f_C;OUf)jwO4*L`!acWZ|O>72UNiHjkacIXQj_;=6p}fe}4G5jMxK8 zh;&;`mlsam>_tIW8p46bcKWdIqs}90LPV-8CNz71@4H^raa6h%c1s+6IHRceSk6x8 zE^jK)ILlj40g5Pngoa+K&s(CK68nR0PcD3}#BpMqRazJ40Hz}Fm~nsV%X6gzxcTNP z5p#yE6(sL&e%Y+ipY6XQzhIlyn@gnkkY1fnWTQBS(Z%tcfH^Kk8OY4I+Yb*%*b00R#%Bjn zA>3t+XdBhxl|7(n(5>L6?Na)hf{be7?&Na}_h}e1y0u91;5|IXu4tzGN|d(&OoAs> zp=v7WETG-Ok^c#eR=X@ELeKMgH5Zlq299I2& zzQTQZL6s_tbqzfDeq6g~%$C7^Lz2{r8MCZJahSgh%#?#6@%){{>%X%-6DMqtmswrN zdg&J)K={Hs03zYTPs^9N6{RB1xBvmkli1B2H6FEp_oNI5*5uwo&a0ao?Qd90-T1NX zRK06WfkM;BNa(&bbxO*?=#shvQ#sk>d{A@+CZ_$fJbug+EB{t+qk{zk|)pG z<-pU51GX%ryzohN;f0_KW5QPfY_#=8z@j?Iml46d69ThPVp4B$dc)Ud+EO@wqIxQ? znI_nb+bT18aoTkGWnc@se6(3RN0$Xdgil|~>cSr*ev2_vqsgo0m{ zX>9pHtM{u)$Jbj}K_e-5?_^a3X&<+-O2!d1*p^glKNy4qT1b~zs}?qpafxVwlA0Y8^%lTVS{{l){iQVof7 zL0&uW9n_Kj>~4}l3acQ$(Ev;lX&7DPzW4ahCisc=MMI{6Jqod#C&o96!9S00PF0APn1H`TDuk&>XMUpIn9{?LxJYA8x zOzpfLlQn?b7}VPgN&tGlJ$zr=aHru(e#BE=iN3g*Rhp!zTD@-fB!!D#Ti{~`+ite zvD|(q^0b56kq{^|x;Kh=$#zzS9pIt?qL8WhRuXgc5aApf3zDQ?R?M}S()U%38uDeO zK&FQXp#LtE!DwGYi+g0v859xVfJ<4lGPz~**$Kd<&h+)wiw-9^*^Lc@=X z3Q@`Y3$(Zi0pFrR`5|LWqt{F!xq{@a(0oT0e8US?Hh0{w}*t_|3qGckOwsmJp^%vHb7N~|T=N0V33Nwwi&`i-nH z^pbw*B9u_j@90XmB26!6Oh!u@(R3F$&hT#*I3T0>iY6S+TLbb3rwc0qM%k$7h-No{ z6b}ZZ#Ow9tw<^Gdpt3!nTtf_d@3XEy*L^(>I4Kl*5`B{{Dohzg;^E*6TN?+$)t=kw zc9S2^?Ah6h)WCJp-Ii6kJ$}GZY7Go!F6MZE_MJR2BcBi$M&F(4-gbsndm( z>@$5QjHAT#0$@f@u(rG&8bs(g=S2jWp*+ls@8kiqB6_w>FW@j}watn5VP$sTI{J$0 zKLbF~eUF~jhYZq87)+}!J}uYYrd_Z%nv@Sb>>@@tao0>RZcBMgnkQ#)%94aoM%DGG zT$mV&OP)jR6;3sj(5~`^LIPCm(f=_@mf53_p&;d*y-o4ta+t34pfhUi;l zJr)JPYL5f;hr}x>w~BgUK_?u5g1dMI7u$PM$&w#+J1AEOjE`lrH|hmfIjL0I!}lfe zZlLmh2z=8rh@yX#*JR4&ayV%TnP+}oh!i`D9}QOT(3$>N*aYCVQ3Masd0)zF$y6w- zdL~d5Yi`DkqMUHp0t%vp6alLw6jJO;x3W!Bcb<1#BMHEObX}?KoV9q$=j~rnP|Myx z@EOH%!DJe*WTTEz%3^Q2vh8v2evj%LCizCi)|)O=wlB*gA8{o{-Dj+}@#HttUnG|j z>5x7q%D00(bs51kQ^UG;*WW3;&OelDg@?eDAVCT6W#iN5k#!{Pmf~5d$0mah_0X8u zCT*Y=19@VLdu(0=l{{NL(FEfH*PIcAd|!j-Tis&{vK*ahv)?lW9Xa00L=`JMUrNb0 z)8|K1p$pu?J>v_Tyx%b;(TN$yod;YKl3Y}}a55Q=N!Q+uJ_yTpZaVHlo5f1h+>#QB z7;s{CEN((Fy>aqxbNt!eHvv!-Ns+8=KRR&LWuq*}9X^=#TC?0_j5K}uM3g5`$X@Au zCr5FkaFe$`cz-FOI4`E+;>{~pV2%kqGn_l9PAm(MOX25C0W6^V!TRt}s@B1A)V=;= zL_Z5n4OrKqX3}E8aNFxCk0c*Tz3COWuw6s!AZmzm@2fu^7!9T@u3t`j(sQS5@HE~YsIjP`H5+#PKuI$$#G-AcItDngLe8tv`R1n>^SbEEH3Q4D?C5Qe`Gj?eSr2}s zsOmcH>4OgGeUfSFPY!zIG4@mTwQ321i!ZBfd1r<08NKCl%4w3VvS4hK$}?W+;{1`q zii~JFo$@Q6!qZi{B&CJiNY6f1$dgyCa)83|a%>3SgtVOu2u$<}6htD=ZqZWD-YS>icStBq*-V1Gg(7-(Lo+jgW|jx{ z3(SPEG0sxJ_EwWkC)-npwlB0E)6ltS^hVXl!+KU>Iyu6LcVgE#;QWzUY2Ra(YPOh~ zqp^(!!v?hqRoNvP4f~Pn@A6{0%OGvCrQj$x%Pdcet%HznGW4J$iRBbCG)DWLpx4|d z%!>HynDL#RBNQ`3EAyy?JL6WLV~Xf?a=}JQr|sk{c)!030MK55nnHjRK#MBY3JV0X zR@^7^3o6-KTNZE9bAYtv=p>qfrj9BXK2%3H@~QJ+jrD+aRshVF(rb3PZ&CLHWlbhd zyjBY%%$}Qmqo8>L$km>By&FEd5_%MDd$W55RhAArd&YrPY?2LM=$bqr8miav7+d&$ z-_Y7md_GUua1Cai<(n+chs*Gp&gC1}b4=BNiF8N)+Bpdlhs7pk{uyEeIKB0a)eVM^ zF(5sL>Ix!Rxu-6-^Bi+UP(9YrYpvjiKbsO`Y1M^;`shv-Hjbs;LzOhBhCPsafoDRS_p%coK%U~EILZCl|H{`mh zce|F?1dilRfVXMjC8o!mkH^I&XHwY@=jkytwv`8Yf7iC}(HGAEVyUwofHPjfJCcez z8QBYxzdG8p>-y&M#NhIbeg*yoNTl~%!(6lRXi8N|IZHj{2W(bc>m~1!w)KcStm|}d zrw|}BjSOe+aC<{gX0_E?_Yf3W1}rqVI?|dms>_ijdaV-X6}?`TMpDXkn6Pgywoxv3 zF-m&6!;d}NFq(jNQANx6?p^v6p5Ov6mvkw+WZX8ryeI^|3auj+fEub=^~D05?kq`BlM4 z`Q9P2jYN?24m7y0v+h*R%J-!=O=6TwVDe`vsX%3Lpg#FQw9z z(^ja{jPv$~o3fqXx`{z){gWyW+_7`%52bf+1`XlZ;BqJaj2cYu$Ws#U5Ef*-XK<4J z>RQXzm+dSu+%^=mz5lTXkk%$q7MY3YFv;@WD1IImVSNlXo!P@CK=yu1RPWG@YNt$a z_U_QWaDchO5sS6AikUG47>rsCXufQ)bBD#@;!L6>CJP< zK0PPj+Qlq6)b=@7r?&xP+poDUgpJsTLh-WMv*)DmoOli@*_AMG(E^Uo@{Mea*pm|F z1*e$qdn}Tj4(L#n(o&X3UpI?yAZ7ZQVf4!Rp2QsVq?P&E#z&`#ub@k$#Ip^u`ljg4 zyK3;c0#(#|fw8^xmmiev`xEr6CHE5(iyzG?(UrF%n8g$HRVfwCjM0VJ7yCudiP=D< zfC|9z<~eoBSS89BB-c?%GB}s-JV(WDG7sM)yW|tFutrMl*Jb(T=x})Ns8a>hSxpmG zjG7VRmh=1hy1FXS3j6M7Jm-LNVEp@ON-N2oz3{ZHi015`-0rUXodMA}IL zNc@x$*ukxw z0Un+Aq3kq1I3tcVgY2qa4lr#nxItf-hOZ4n9~VINqShZ~QDQj|Chv8mp)a&Qy51V5 ztZcT<_Bmv)P>Gcl)!{pocT#+9arUZGnuCEU3&ngtN`F8&2Bf(}NW#6kYg%X_gOo@{0Dn#O!-1vn)=rh8 zP5Wt!n?6eH9BlMWHO&Xm()9~bPz_1B_;lFvayzEcC)hd%a+j_&P<|&ATqLIGsNJW6 z8FbFUo{*8za~={|r|$(>dl~D9TNE_LHikTqWtnHgq1(Gk92&*c^#_yr>N^@lI`ifL zbwj`|P%rLrbuPiyis6t%yf6F!-l3YoUgSruuJ%61TQOc-Vus?liF!UWu?hNC)dhm=8YXW*XyFx znwN(85H5pUj+bG4Rcp0YLEQyeHa>e^9`S_JW16*aDJ1!jk;b1TKoRuL?b=)zyiX3G z&!jm?q#0UzZj)d@mw|M+H<+0vFA|}y=a7|RTHHDjP=Sq%S@rxz8RCt&w1yu7+q*-D zJ>UcC4KGO&cRqFOctw#Ng;4;TAE=W`4%*oAGVW8{(}E&nQ(6w64zJtmy<|?%#(SOe zZvs7U1&^G)Bkq`pt6=MRj)DB*MiZ*huXTH;Xk(~{ckb;DYf|KQXg<~4B&+1J4o5CJ z`hd7IPZV|J*a^0ch9Duf@6rZmU z>iuS#1@2*&{zy@aH7bn-dsPE-hsR-M?Rlnz<&QAa0#xAEh!Cbbm)>2;Up1X;lF!&# zFWX33Ddkbe=HB2txz1-Rv=T{q5M=STkXf3^6X9uAo6BV6w-~wg(VFZX#|(QJTgPpA_^W8-0~?5^$ZENF56!ge)^3 ztFHsPGX5kCWUPzO1uj=ue3bc)!Rcs`WFmp;+7MD+IF}>KCn&+hAV9kHI1EL8`D!g4 zq)f#L4S5!rx?{dLH+Q*%pae@Q$94+DI`Yk$Vx?73gxqhry!7g@3KKc9tk7RmYTC=( zUYzY14R+QAVn+AdBpqRX$_=Q-=Z~a9=Gm4>I*r26xDUeGG_6z`XnUfO9MRm! z;|Q~bhHmJNPO$E+V^fh_Hl-6}iGLy(7Mt3%$7tyDulZ{tV0(If<8gf^lUvn!M zRCv<%HW-g=D|)A{9W9n^mL0zw$pji#z8g8NOz2E0MMVI;undfp47ZK;4RcN2qcdxX zc5$VTn-yE4PLP@|a-t8Z@Z(40py|10$K`QyQbGoFAo6JCs8xDfD7smPe2cptXK|te zGH;EC#()poR@>Oe6bULyENDbO1>(X*7ATzZ98dhdO8uId6|1a&IZ>ECZX7+n56b5w)Bcln09u0AZ`62!~N)Ao; z+l7@onQ3={Qgyp^nNHd}6+8(Ae5@IauypEOMl2{*>qM3{9nQ=edqZ~6hnW|KrD-7BJ0GkYmClP!&4mhrOWBbOEEOa~bsZv6f7u6sFxe?rV-pi+6yWL4r5#DdxfV)5BI^545TC z6mIN44eK7FG(LE0c^Dq(E0VPsY^8XK(^=-TX)7Woe^=hP!z0RvL$|<%NgHgmP!MJ^ z0HXwFxJQZP*#p&+@0#|-SL57j& zj)m)(Y3m=F_C$PkKN+R%GYPLdsqH+L_wM%JBO^)s#2{@Ze$qvTJwl3^=9*qknQxB8 zap+?$@-1Qg^}2v$fd5Wk5Mi$P2?FSyHQKSejPLd=V*O#m+ulUK#B%O(W_xP~io7#%w~%^6|70Ei&;%LBpWnmrxZ3`oN{#(ijRvBK@W}?e;pp_ zb=|#`>7?l1$Fwh+T4=gKdi7C!t1SXLIKJU{ztVxc(Jo(b5puzjRaBi+wy6Wu9%7uo z`UNN3ez1G$x?OmuYK=iI)5^B*sj=44@^085Tn`BIB8cw?pcK(p%*5N<9T5sq)-}9p zpAUxmu;$oF2qpg-3Cu3irWs$e(``oeNDzI&Q5E6v*U%w2*eLE3~j85IQ%mbjbms+ciF~6kx$KGFC9Vk5fytU9RaN-cacHa+SpD+saZ}paa+?5j1;wYW!tlbvDlA~=M zJk+T5SIQ5hB-xbxgq)q9!ihH z!_YC1Lp+Ic4mQQm!+bdZf<`C--Y8k?sR zxXSZon+Y_h+F$Wlf{WeJuVYfq-_eDsKBgOrgg2T;XCV!s2Q#a4j z5M7qAFluXBaa-E`sYe!! zeB<^9O2)L9*$BrZRbd@=sghRU?9WGOVydK0WuxmT~t;OpqiV!KLN>Euf} zXeji6?Ms>Q2bAupHv{u8)L0FCJf09X&f#6Ij7BF8%8Mo){_#171bIWQ8+5+o$ZqyC zQbgnTuWZdj-gsRGa`*^S?TLL?>wosxTVtN_ zcn~W{_r(ebQuqRWM~EKp-c_%0$>bBGZaE%Hfr4VTFlIdr3xM!jWie8@8R%^Ip$~X4 zpKqwy3IB-0iLsz`nGkhU?ac0j$>_KB)0{-mEtgSy82C7;1Qm})lW%Vgpc`gfK{DO- zItSSUk>9d8AVw*8{3up4R!%9Z8zS-Sr!W4xH2i>5X&My}fM=XbcoEf^E+qMFkl{wN zVl3jLVD~FA#41W^HiCfd7v0TS+*C1I14)-qW(lV6w5iO+Um7+#9*+eP6UXAMxVF8LB{?MioMpcQko~@6t1>B0vi!D;PPKd z>WC~7&v1y!Jo41v4+yZL@L!B@^a3z|#P9>r(``vyR?`qGWU96-JU2Y=yfceE+SEq& ztU2`Y2s-pxVn?F0h*hA;bDrW`l7oN4dJ}0t-Vd3U7hqXS_W0o;PYI~sK)D5JF+{~R zHBR&Sw_?SdaA?~qH;0>swod>ON6rIAT@SHgm2sU@Immyg@2-xy3?=LPW#{EMI!APk zSwgFlDF-D+9Vr4`0I|14JzlK3ZH6pupcx(j!6wwa03g^f+JHuyCxn9wWgu}7o!3Vx z#6B^pg(i{I6zcavSvC>mecz-k=yY&cQE3vSXpKb#uYW)|eqg)_WYdH7;}{{l(%wA2d%Eza9C8dQP6Eky(K1Wluc}7=0SLYj0~8q_099ZVJ=`#<4>4R6K;w&6a(jc8z@DT#StCEB7y6VG4?>uZ z5`0B-c8>NZH&dE&(7Qd@*8l+t=&*alsO;^FKC2!#ahecRJuv#V2b+`(W#?9bxZrVQ zmF}_92(e_yCLU8P?wdA*tU;Tz^St~1`%TCDfA@y}k(QZY0M3m91+?Yi|LUseM58YQ zqEd$6SQ>uyuLH;;S|X=BH1-&m8zP>U-9($ATp{&;w%LfYeoaQ;MN4r}b4Zq1A{*<; zM({?z8ci7$UHAcql(#oLP>%!A5w`v1*%4iu(uD==zq-gxh5O*#V1lL2 znfa!OUL~pua|OX~cbKzr&%u_fhfj#~!cv&ennhmp`6(#?*73}~+QUIMP5QAOO)gZk z7}h1XY;OApKvJTPRm6FN8@c`}^pvJz^8`;C)8D-qcLL6SprRX+=mJx_a!H_oI;+)p z{^C{O>S5e*QEjs910-5S%+(}@CJ>vpiz`v}D7veSTx*cD)D*5vQ*1^{GlR;WxheAu z&FOVpie}v$6##d{KQdUG!yh$x!ygF>zX9NmnJzV2$w^cb>%NjQYz?@hG=QYlPBj-~ zjy3=^m)?KaR_^-E;`iGLzb9-S^lH|xz+NNrNS`#!=?@9sCNv21J#no1RaeY@*63T{ z`m!xt+qP?J1at(_Il=&n5P&|;xHG0`A#&mE{t?bAgryg^1Ba|e_d88kAl{!*aUws<019ply;uY@hsc;^}%R*Zf zpx|f=b|CatL_VGH;nL0;N3Nx*AyR4=EM8vLMAVfJu#T{1=sopmaIkg!ALvC~0KNEp zF3`kkjT_hMm?rQ!a!U??Ei0b5u1M+vp}UA?v*R)!Pn!WO2~AIy3S?4=9Qt94FV{go z=Q+HcLSLzA`1eKNVRA7#U&MhLaShlK+KZ2>((PLIz-41GX2Sqmp;dcdKR4@|^;K7s z7BJ!}|B~>z*!+5XGW-#W{Ammqr6*M99)0Z<89t)!jQI-;1)L8)Kokp)X_unVLu?~P zB7>gx;&h$@>v_3-*=e|(7f8MOS2d3GfhaA4I1Z?$wL<__x{K!1eQ6=}vo;ig?+-ZA(55o9gPzs? zm4@6B_YCrwVw0M|B4a*Wv-si4*4}#w6`+N{F8cv#xB}UztOQSqcu8Qp{eXC}GwRJ! z3&{q0-gi%sqJyWl2U*Q*#2Ah_=IZ#k2_}2aCYIE7J=5r80aNm6;m_bNIWb(A5R7tY*jUq8RBBBlZMu=X zj2f!^zBI%R9J~$AaTNl13-{4Gi}chN-xkkZ8U*85A|^ss4SXBDMaC8AgV~C9MKlW+?&ODtqruy`j5uH(qTPVHhzNRS^~o8pov=7JK4D?Z$LPFsKwRvjJFk32$%j z;cEpUA)$6aTXUrLX{SZiG`oNePaHj2YKNf8|LhC!fj<#6`#4&yQpDYQ^;cH_Fhwtj zGX?%h3?`0|B~4xtHx4nD3dTl`t44F?j}l)0P4v5bLad8?v;#_|;6#wzL%MO1o2}k3 zOJEoQ;JCcX96%Ce^(Mzd1(S9fm=iHk0~5E{vqfc@PI1zynwY<{IS~L%{($EMs9a4{ zAX^=+y%wmq1|%$j9W20C(%3UC9fCyA*Tjt3cz3dIlatYH^1@NC|p8sKV{->`&k! z6X|khm&=4+-hKp<(J6d(mwjl-yxR#x;c-Rb0Misz-ax0}U-(er|cy0ft1m49j^{*tvNibax_BNq#UpW@;yZKzha};P^C+AzId6{nL}&G zrnY1Vu$3yA+OkCZw?fLA{RVm^ml0qw&ZNnwWfU!PEWZeXQQO3EEdc&90&*ns{Df2rc z(0=P$$l-^J4Qt5^|2j(w{;|OxV9^fW zDWtF+LsD^tT9cG>nN~YT)iiTLd0YmxuGvTbU;vXRboy` zvS2LFFK`GK$-aBkp4Yk>OqJ&rCdk#d9bg}IK1KFI&F-JriXD|VLE!zE;x-1gRoO9% z`~9AArgeUd5B`%+L};TgG|lz(MItBdj|v$k#I{ zTy0IvZmE^R`@35)Z)yDg?Zgm}4QMsHG6UNK8!=*K+og{O^tXT58oBou4S?B!7O%SK zi^IQLwB96=fmc{|o5W_+_p8QI2T)U}!vPq)e#}vscP6}7Tk$;~hB>)=x|W-rpDHZ` z6|g&9!~JvX2Ar2N;4|b$K>Yz?)9@SSE>A4t=2ghiHm4stOMFK&b9*#^VJA3k23fe6 z8fK-0LY!H>rO%odi$^;rk#RGMDyrOWhoipru7|^Gau-F;`UuuJvX^K^$XXLNoW*{p z0mQIiz|;SYBj2ZIv+K!c^O{WTf!r+9F9RZaSLTYQF$4;N?619hej8SA&nx>>^!%2? zx1a}JuF~+u!|}KW+CJ$#&c!VV0r0ZD4*jl;VGP6V&6%+pbML7FgxzLcc{(lU%FH{! zD6_W1jEta0&Ds?}@rNn;HMc>5VmU}7LgW-436*ehb3J3Ka=D|=i^ zR(g{zT>3;Ccj$`2!HQ4TrH>+2a(0%^Vr`QC>_5_DX2A>p)H{=zv%xT7_woMiox z^qI|Rf2O2$v~3GaVJzJErBx$^A0tZ3!bQVnvA)N>6Q1V)&@ASf-Pg7jm+X6qIxn{! z;CZYe{mJKWS$+P;zd$#59(n$nOOFM2X{)3%Ihz==SuHv#SLem2P@xW7JwNES=A(f?Qzm0?d(sGvpsi>swIs9n?XfIx48sqYIL5blaJ zBKaq(BYi+y_B+Dt#G@%_VObhTT0R6a$W-DVdl{(Ro~vJD_8E~!2;Ke(UU$xhe{I;r#!djUr^bxV%u=%b3&NbgS zH@p$Leh#4699BH>rC?>wtQaaaO;gc+xrQsDj(|daP62w~BT8KVV?jaMKo5{42EPRz z&N5ZB)8c-s*Yk$$71cIXW#d#v>j*Np?c!e@+j#*$8U?_+?9`a4)$+GTXZ|nxd@ARU z0XWTRsP8q_CZRrU6gf}z>WUm|;2Zh?J#Med-GW-A3y+vINNgi$8wpIDxL5}!%yK3X z@H+4_Sg)kHtLVf_%U8E1!|595)DL?a|ko_E!eQ@jZ`&A zt_3pzZRN?K&Io-D?u+xfQLW}g=A{61=7z1W>yt(LINnDO{igsE=qeq}F|YIH0iHY< zN4Hw~cV>+rXm-Ua=o+Z5#ASRJfhS}dUuC;(IUI&HuT9ftz3JLp_bO>j53#|ZxWyxb z=bXB;DPg?vuzROc$0Ap0oyEH1H63e3uG6XqnGw zT-Za{R77`+2Hm@d`_sH0P4mo^lU$7Xv)PA7`umFx`kn++wLPA8AX5(WVbjsNevi^>?BZ|^%~0m2qlpc-}TgM!&a(6jbGe_9fsgH25xxpRC; zGcbPA zq8_(nPy89mTCabu-xS?*y>{LR+89qdNu%hvN?hrg9v7x`yC30^kr2;^jnFq>P~M#q zE^Y?;!plXj`=-~c@)WTBTOa3?VMv`g za<4?(SN41Zw|5C1@#O_=%`8k-U*V+Lq|}{;1fawERK{PxOg~9CxcX+Y`PtlFocO)| z;A6O?+exVBSr3;UkMXg%XluuY&QkVjLUfhp&tNk5OWD30WeIY^uiY*g)}u(0-^bny zrQ6|lT+EIqjM32H$Uht*XdbS4u8U>gXP<%(`7jSv?D{w2i7<-pufw5!CwI5%=i^R$ zjOBN|ad4s}Eu(6OKjYtVAo^aq1twx09jn*N`zvH&_I}W5Prh}mn;)~@x`#EJbA~qx zc4T&)7adjO^EmI#=4aOWjC5U6>n|3{fHti70#LT% zzjUCHn|X-ra1Z4%xx5gxud3(yc6LY#@Ir+J$T$7xID*&8DQThilW!*qk%=uDcMd(% z5y@?;ZKJyJU$;D1GA&u}dU)p>Qn9&0>uVb4^vJPcmQXU@jVAnYS8+m~v|E720o+Fm zgR{k-L|`Ynh;`tl_?AW%`$glROW7b2H0R_f{#zG7zw8v|xxb5}|4eQ1{?s(+A}RF~ zq3vNx2eqqCOspJ?ykU{~{#L;I>R__9$4MqE#=gTz~{!H$5qal@1)!Gb;ego0ZCamH-<~?->GlG;E`f zn}Ie1UNFT)RCxs`f>kbd4w|rQ4k~D}#7urJn{xtssT2f{F0e(-g%7K#N_UP17z`{vwE%9lqX)ad;hCU};1=RLSag^hVe0)(xy*c6zjVDGAQnT}a4vK< zQ@dWeCd7P0RoUHb;qdaWsZ-z> z%XWx7m2BdGR1efNw;XdjtLBA0GjS=ABJT{t-^SB@<+Q&s8?kIoe00`d73tty&A)$AaUd;D`nDc?p zqRX4^m0zLka;fREx3OSuvU`tZ1Qe^)X`OdKue~2dyyp=pCC)~7v)uGVRWkk2uFH0bsG)LEN=ft8PUf}QVQX*UzkHN0tLj2^-_(o|=)81f^JW-pW{ zgZJPP+l0j+#nsYPN5vG3WO@g?(^vBI3v^TdY3!Ya8@Pzy7C&*m*1;6IGo~-^aboyk zb6uc9HGg1GeDWVm z8E?XEKeR}iGX3}S6pjT8p(dVW^{>%|fM`bqK0Z+C6M4+>KBiQV#sVvezm+xoijEi* zK=nubS;HS58$B8KWlx^=^Yd?3t`h(=ez%-~QmMR<{-RMw7BbXIjgF2OC;>v>?b=|- z#Xt-KJ;UvyBZ<6Pnrad_DJVH8RGbn4(=VN7O z_)60Imi#iEN6B9HdMK!nF^CvtBiP*BjU-xiU%9!-hkKe&9PwbQpHyKZXkmDPr>07OCd&Uwt3?@WIT%V8`YMrJh8&D&~TyWMwtFtR~Km-DliQb`XdPw>aXuKswKhaj&ZUc5P1AuD06r&TiP8CADi#`~&CL zrObtoc)1Q`T*WLx=cDiY#jnOz7?R|Du*lMKza^EcnK6*gRf-W+nXzr9FJkVM<8&%% zHEv4k59}Rvk(T@J73uq{lRdV>M#yLC9DTWIKV;SyCo^K{2>5Mo7+EM%79FipYQ__$ z{T}t)><~!VEmNmYMP+Btj8C^`BDW+-rDfv{tqzpr+VXD{$*T(=Hwz*O;fYX=t!-(` z`HhL#a`4+0Ngkx{Ka1OPzadOJ6^3cS>5XsED8}&`(FSx)&(Z8XKbqWnZhGCB%@rUz zZ~|`44#PPruiFe1vn()EMqF_@3jOL6CMm+b8$x$KI}KY`9U|0k6~BfoX@%4H{B5%&wY~6O&Y}pbYPzDHfKkq8rn-HR z2d`CjQK%N{cZ{4MJA>Y;SEXu`vAIM{=X}B?YA6NbKVBgt2O3B{<6b}DJOZhRq>57d zo|Dk>95JW2p?T*@!;k|%M+M+cBsQJC-3tP#0^h{i#Lw)LR2?tfQozMU9wG`nx6|K9 z6%Xyo4;CrJ=C8N!C+(LnH{?*Jl+P}Y5WevB)r~p&EarB+T4|ri3&tQz>(MIo<%%iv zxzB8Q?K_L|xt5QEDP7MG;ykD0`mS{@Mg&eVdcUh_>JH0q7-UjrPadTJO)a9j%FAa> zeGOHs9>mz84tvg!C0n+=}J14IJxP!9g1Uw4)B-IAzc8QXZ z!AK}uPKs)ruh|pKV7r$3BVtBUwxyK-w@8k3mgUUjqG5%CYo^}VSnE>Aey~|edFtB6C3ZByvvOuWv2 z{bh_I{qnu<-K6-}s0hWhQ;{mv59k&%krnifN`+=)Cg&tO5{f*a{5-*<5R_U9Hxtl_ z-O%DsS9sG@y<4vtf8?kihy(mJph1Q8uum7yZlBA&!mBui>j;R0ap)jSV*CLl9;WVrAlxH6cXkpYG6IPOaoh%g z7eLxDX>*Lo9ab%xn}k#njuDcYL`?_1c1o!dy_*O7Umh8Mr=sWbd=ahAbjEe)iK*5U zWVt*cJy2n$-9eRI-!<@l1b{&zqMI;PA2ccjgNFb?5L@-FT5L=*qzyFB`&D>JUgh3x z=2?u+2hC=_mX)JVX<{47?IjtI8NR5yl;yVwTsLI(GdBLA>d|Zy)b}p!lxuYw z5>d+h_RagbS7jt`qNTN3rct3dHeZpDIc1##sKPV}elc@*BAn4OZ+G{0UZnQT6MS?i z{B4m(WFeZ4VWEEz$}Y%9^k1_8)_M2jl(c8DO#imj-+$Vu9RlP)pd3DoAFs^bL{064 z?Cj%fCB%}q-Y{Gs;^D2k%oaTCj!Ds#-VOLub%TV(=Ip^nK3g|@Qz*7upSV>fiv#QF zYSsd42NNDjhP>|xGQs)$%&~h~-(G>2)r{utkG4pM1gf|{pHS#pE}*nq5Li!xv4(dk zAq#!D>yK@_IQ5#8oD+s$H^J*jugoSUbXzA&JXdqFrjw3yId4Kf&|yUppzYiu`exRw zNxQkfo7|G$JGyO=_ShY)-?${r0o#%Bdqp^A(H2UHVCg9YB zYOT8UI?-0$BIm=S=XE~8Tb&|4pU7vuyS-1(-gBrfwA0vCB4CRCI;#8HM0=*2nvDq2 zvOE2YmCaw)bW<$zx;k{yQRd5rQ~meB4`#B+AU5bTTPO6o6SQ;GzuZ>@KdzS@nTw_S zp4D&-VI6|R{BEHi=y;)8GILP%>5p{?Gp|tB1&!2w2=jh6a&nUoB4 z6Rk?~9~2_BU3CK}>T5FA-1h+f^n}ct7Rx{5z8XO zc(dnZai<}ve|Uc3`LIqFxKo%K3)ey&ZfVH$5Hst4G4PF~4Z#>O(|>jEa(E*VT>2X2 zpP=}+&mu+uwnxL2#rXW+c=;0zapKVTzWK&tjCJ^(fIzZ=GZ8ZFFG$Nu)TrYWl_k4NJ+SviW9%RyDB&=-iujUscCT*lcZ_%qFmjX*VSml%6c1E^{Y$6MzYbuoCmU-r(7 zn@?|Sm#V^Y*GrFlgX^#pwq7pTxv6*XdD0Gx#7u|QQTuwJt=lh)v6kFK>%S>DO3#d! z`If{tjn(Lo_aI8J#@n2GHeuK@tZwJXzbq8)|LYdxr1*w7ZG7Pr2iVmDbtps}KVzKu zmw}7v3>=(k3A({iU)@aTnKS(^yi0m_7mkr#`Q0C#lHKb4!f4oZs!OGA30jTkwOsjZIGQ zCx@2zzftCt2q9&{VlH2p%aj0*Y7zwG4i|4HZ4O9Ce_tKU{CKdT_Zq%lNl=mH_CAM< zbL89aZ-t{@M`wFN#U#@12)@&c(Awt6BB81u9uv14ctO{56@m0BKD zcL3l%r40zi26O9Zkex})mX6&vsmYqL%JdEMUHu_DyH5m zO-0dqOqZ|8ZZlU4$O|q8f@J*k^JI);SN#T`h`Y8 zdH3we#Q$~aoab+z9c`D(YHXOR=OjLF?FYM6>-s-7iNRLkm)@qOq=n2m_2d?TYDd?Z z4EqoU8{EHIuMb(*`zrK3z~kv68xVOPjA3Hbk>Ee(v{d!1THUM~2S;D7J5WU)#b(uS$b7l<2ee>Rc}w*YUP!7RFf_S;=ymFs`9b8 zf99S=MWa=d6e{HeVz+Mop9mTRlAxhNc9BR9=wtn#SLdTTy+4@Vg_VZkju=EN0LOtp zRqI>YDxK9QB^f$FCLP<4kiGk?B9-Lt<%ya5?RVHy1OAQEkhS}K*{}o^<+r5&b-BOO zsV)}aAzDKvDgMr0|4Cw9aTnTeo(UO&(JS`1Dh{1K*mpngVbws{h8@?X?h}<+Cb$5F zf&chB75W!@WO0;2FW~iGdjAbO|H;<=*F6J2hyk^Q55a1Pg#XK&|0BEm=leNcPe!k zre+pXcHi;mni5zoSJ{Esf4&bS8O~+gYW7>vn%N04d-;k)=~a!2=>=}_jr@M+oqE0? z*zu-%$r~RJPd*aEF~mL~f=W>jU0g~^S;r=7Y4`NuQ$&pGT^lD3<9?!F%WBov&2Q@{U;CX6|*^l-}alPM|rtgDPI&~4RQaqF)_HpCcC~0 z$xuNerR0xhN#o8Zl|Ex!4!g3RUfyG!ykix?o$&Z6W?;r#DwgF+Fj(Gn z% zt$VF8bjqXo$itjZ+bQh2H}dDW{a(&X9wg}4-Ct48EtzHeTCV#{(4d&6r<0R0P?Uga z$Zmb7!mUH!B6_JPDw=y>E?a!&XBgLeTRfE2d3sxZxhjLBctLkHEEp4qCqLTw>zQ@e z)l+rlZtB5h>8ke&1$Am{7&1S_D~nff^MRVL=8fXlL$&V{BARZgGwStoF!Bx}RW*0r zwOXs67jfQTtI_arb_qX|1G- z^3vv)^T(B%cx9Xd6^Z$JgZ}4xKR?-w*v5A9s?Nhsuvwi`_piS+RvzCc&igvw1$*?; z^nAHoHykszoHBalXW4_P*16+^J?Db@K{xRI5}V)J1kUqK)hM+e^&vE0*A$p|9#=89 z1Wsf~UG!Yuf3|j+c0TcCps4z?F7Dgzk`eVw(`E@aGmHGIki46Uu z$whM)Wj94kOiat(LfgwBf$2R(tuUwg{ied1gF}_wC>5~EC@Dsr^Nny?>mPUE6v(;{ zEgnXjbQ~p}Iw|FL@~|zjE!}b9eL!b<@MUSg`7Jv%I;?)2gXeHnQSa3ZT@S9TtJYmiKE-es#k-3TOp?)My+Neh|&$9s&K;EazRC2r3* zn+^7|IjxWJleH}{b%*U**yYb0tLsicj4b-ifo5a6Wey!18y!POC7Ix<+D-dH$ZCqD zx;{8&g}jhx4JIpe5vJrcD&CUKzEES!L;ib2S!1p=UKZF6)h(fqMFka?3w&n@JHBYx zajKqw9$h7cd~*K^ z-duw~R-08NmNvNNX&ZTexRBZ7W=YdNhw||Ka4Gxe>fUjYtD>3yqs|469Q@jRf3ck- zbG141=dgCeNv?a*Xns8zcWIrj=Wpn;S=|r!=GBUAFNDDIS^5)7xzIN1&BvnEFE^JW zr8OQFJxgt`TWYO64*dE%E!I$mi zbTLz-?z}R&4(1CB3uE-ru91HIYqyj&$1M6lMbEFp)cfnrigp^V{2hVz)G4O73b`Gt zlhe6HzdqTFKx_iB6akc6B1_5tfq^XKq?~-p^rXbsLEX&4TX2?u$^Ws=@K^YXJo&%*IFCx3e~t*PKNraYYx;MnyLJ4|+PIt%mT>Ssy=|XQ?vnJ6 z7lQUS5^fx&Bk{QP1PbJN+|3cMw3Nc%vD!O&TQx>G?mFAAL(f-E)!;$}qbhgX4!q36 zFVn_%RY;c{?V;7vs0d^NS8B!)d8&88_R&dWHP#rx5A!`Xd&CynGN3lUM^2{2L$kvZQq2`Cm zZeMM*hh{f|{yy>M^TWM-${3!3=#Ho~NBF6M6)6kcS*ZPD(*0-U*_Lmfs`o9h-X;s- zZcRbWP%x%yC#;o{Xqr{9!I{3m7lK#uN^f^F&z|?Z=F!FJPAxCS(3?8pd(2{E6v1lK)vNM&K*X|MrRAoi44qki5eg=HB- z>Yqvv{9hv|7K+GpIj*I_-T8J01ZgezJ^FOk8v}#=#euXDJB~Q*;&L9w70KCca&e%N z@M+cC@)&!`TTv3v0r$>ggxg5j;>_;852&dmQZ>bvd0zYTU#H#gdRb{mE|>Efi)qK$ z-XQkWVY}~)@wXFX&J~bHcQ{iU;@T%tmKTJufer&ks{b-1${2imMrGPy0`C@Mq`W$y zQwaz1+WtHW#hbxs3d!Jc5gE@x5CV)6C|2bF3oc(^pz|XY$;ed5#W^@c;AKgaxut9c zjX%}NQk|7N(Eku;>Gxi*uAXUd#SFz;w#YC@cNn|h{u&VkFD^#6Qmo{rrKe>a0bN=l zg2^OL*Fw>>ZbiSFm|aM|n`E@ZCiR~ZB9W@A;{^r|XTmCs=MlOQ10Ukhl66x~`xP)(9Y5vl2Q}6i zQ9s)v@QWQOvyt%0wKMv-1$26pOpVb5Z0wB3a8SLfnoIk2b)N~t9wzJ@4_ER^VSC=i z7oV2;2H33#Xfxvki=}gSqO0shaDg(Lzu@3bS>31Y;8_LFnmn$~ntqhPye4zt{ceT!qvB{Cfp_(rB-TDK z_QURHf3>Vcvh0qQ=$G<`{MKr^vo3f3F;7b=29xj(;_b0aEhyau|F z!u2-cZO~qx?lB%nf;_#~L&L7ql$5~3%OgT5_p3E|isgFkfzzwz%+o_Zp2Hb`cgu@8 zzxLZ5^CZ*f+bBNcE&_J+B5q)+OQpjZ?P!|sYDYL7Tw0(@(FwCnpI9mN6d3c=!iCyy;57Nt;DeI^d0)60;_T><(D(|HkDv5F3<(T+ToY1nNc z?G!3joHnILT$qk4PW$7d`aVhBJGM^(1uA3`!oiB|yJ>fc>B(zuR$iH&`_i;Md@j7f zlFMva?+rd2bQi6CdHaI@9hFseF6A%c1+8}Isx+t5A%V+e{mnG#-1E?+(b0Jw4Qgg% zzp#gsH;MGTP0`Yh<-Te?Ncu0xoU&T3_PzBT_$idu?bi~2@Mj&RWQ4l?pv%sNh+Z0S zJ3GCBU|52RaI`$tK3unadj)Otn^tSt-x>V!IH2Gmh2?^JxnCHewuOVt#`$h=6^@8S z?wPGMfOVe75N}q+ii!BV#$l1MWus7v;afG^*Gl4t(+hhS41xpo7|&&ij7D?nz6uW+ z4$j4T;$bA2nL$1GNNVuTdkxWKottO6ew#F(mx%kwBIN(rLRQYy%=rB_9l^=YF@Q$J zyQBk6m1~5#8Q-l%_XIlp)lSe1ivASrEQ*iw^t{z#BGG+14GYfj!Rb;}>-$}4s%5s$ z6St`lH^i+AA3f74|6B5(Wk78&b7>2v#q9=uV{CE^Ml1awrhQVLr2%EKCEM9sG9LL> zp^2o+h&@-;BjvAI*D?{T6yB}CcC_Vt+$NcQOHR$9+^-4QJuA02cKKd zq~*Mnz#8elmgRg?^idMKkMFUyM^-1<>fIZDz1PuCD>GN$Oto;})Zy6LZ@FF+eOS9a zlq^r9USz-?3ygo+?xpL)B&1&!_w58_KR?rJm>O&|bsJ9cICd=6{#N&-_}*aKGGV zc~;l>7PdTpKKZyzepDxMdQgGAU^y7=NVddPw##|bDVpGIt$%jB+-FP>mF@?bm7QOP zz4a%fP-c>kcJ$;QB;G3YrkgZ4#$3JWI~cMQp!jCc<8`etGEti`%;YfP&g(@{2nM!$ zaPe{TojR~Ii7W6}E%J>8gWa4qwqlCcOdAOX1P5SYfO0voPFs$NFPI=t>r@y$0@Vfj zhq)RZ@m0_>ek2ug2=z*@6?blq`guk zwLWW5O4fEiR%AMC-E(0T?Gse9#yzPjHUcFR(tzA%qa=7q$fz&HGlwa-E1G&V#C`?ZAR$ ze%5SfsHh4~&1s|R&@%EbSy?CG-548tPwUM*b5$u-ToK_+#DYgI>pZlhPB7@*RN5iB zh@xw^ww8kF53(R11w#Tt+=TIT8$`2n4)@~oK(!?D!=y5e()-P9c9Dv7Zh5wxE|V9Z zb^wnq=|*0KRGPzoa%+g;rl$n-8x#|!=P@uTM1i#17z#KYyIp`<~WkrCj7y z`daq>8Juo}JY!XF05-hs6=Qkz?gdMAp?)+upOoqzaYoJ&@_X}_7UVQ66KWw1oU$h% zA4~tJ6LRif(?&Pm1uF-6MfNpr#!=IDOC8_op;1Qj*;%_zH=_u->(+M>E*3_8(~Scu zdZo^-o>fcSOISEZ$bHl9NVd5Z+F7tSOrB0kGC50MTA`Dcr*o*DrH%ZQZZT{~d3V(! z_ia&j+oM^A&oTcW_TDNg&Zyni1VV5x+_muF5ZpDmyM#b+cXti$Ubt(J0EKIChv4o` za66TM_da8Hk2CtVZ~C%;QG>5)$@|Xvthrvb8)-V{#FZSqo^jJ(2}hxmRCHg;XLDJb z^n~`?DrZfFyco$CW3-*|2d$NCr~Sc4jQwgPQx{!pZmUZ*1he^*XUQKmV~_7Fjt_*( zG%_#WSFL9~=D)!&N{#W!SMFeQx6}uwXpxV4I63oYg$1i3jp6!@Bf;>*5?4G8vl=}R z#qEeRigWT-e_DNk$OHEMC!B^`2s#~aMp}HvipBGe3WwKU)imCG&>0)fe+4SrAw$Yl zf`2j@!3U-uWEh;tZl`7*%Va+R55ca3qZ%F89gdsUg>SvSV;%?ldu&&P;&t`VD>Ec3 zIUt)hld0-7H!>N)s)#1fuD5d}G>%uPKmM0>ZB-f^(@o%6FtO1PDk@F~F=_7DBU{+X zxlb>n4ZO!`iq+BHK=1_3i^yM*!n5Car3@U@%NGC}?{Owa8^3<0DTm;@53tJ<4;dfd z2%V&Hf_m~*{JzZ+xk+YeT}EI6BI4D7b8Pt21-}iGGYqb1#s{yb-awF}bXVP?Ly_08 zzm*wzW&a%C-JN?da-=H-3W^qX8@BKVX5cFbS@Q%|Ag|9&!ZfKG$t9n@Px*lUZk$@cxm|;D+aJms=jthY zLBbQ`&_qxixnk5ccu!*D@dW_HC7+V%RT^#U;yIbck~SJ0wBs+Lx+?X(EY3aj7?aks z)vo^oc$hyEo*3UCUHwakRp!8Qt_;jyPe^afxC&SzGp|;i61x(?R^l2t zaCN$q3pXyk8d}~z9Uf&>t>N!9t~6VBJy|T~xgDJIm&*d%&kAEV()^T| z1<6vCVnq+V)o-nQwON#-E_&!y)S2U;ABScgWpL^4fT^?0WrpST6!2Kl4Ag!d~a_Y-@CRu&=FlZ48y5#%!P5*HNCtvaLCN zg6P79=GjvhBzisl zgX}PPvQvwsP*Gkg?>EQ$)We5A*2EUpsMcpKww1WY*t>M6U&Md7aXZEY;=lrhICa9T zP5m5frR$D1mYz+tm{tC)o{fZv9H3Pvzb-+wm1D|Tls^+wPlkmrt!=?OgX!YyO) zTXLs{E7GpV5P~zPIVl<&ZT6|P1B-Lqhx9VWAI`ro5bBf$Ye_K3Fk0m3A#yi!%8{Qy zni;FJ$|d{*xeqyLwzwb7;1ql!cupIQoc;d#@=IfQX?|t;QhD&}yFN|eJfIlU-3(#e zJc2iF@v5YR9F@m@rI_>{Fkktj0Yi~yEVxvGD$p?(ET z^ci>C`gzk@uzmz^Zb^LFeA8Jn%U}p8W)?Yh2J$aViV-;%8mk}m*QKGJ$&F`4nT`}A z+!ep}lJslo$6Vy(s{h^dE}kuqgA^@wYRGJZ<@F6KLPs8_PosAGk6DFHgT65v6D@Gw zLLOM{o>M02E6lz|WWK1Ako7hz&GKpvJ&+zhg4cF^nT4dew6ON^O0%V7Ue8AHEIPo! zjka?^+l5 z*gz)<(3fz4hRqTPnLEcMob$)HodpMGC-t6Jv)=P>qPPk(ldey%(T3)FR&{Aefl_F0 z2MaJU)@uZzsBEWCC^`dKY?d*l9E!Y=#w)7GZp`5pdR0%I?Nn|(%~6) zrvK5rdKD)aqio9pSvSDGY1c@J=A+GmHnX;cPbT5kr>dw}Jxt_UT$TvEI*`~yj!-+3 ztlG=PbOq}}nvf+}xL^~6)N?`Bb1OpM+mGtC+TvLEYkL&R_!MYMx00SGG?Tx+;~LG_ z{B8GLG?a8ODJU2sn7Hn9X8Jw197d;Pf1&tg0q#!Iob{-HZ3uIerD>mO+a;VPR}@AA zuTD0nrIF8++Kg<$VnWii@$I7e1?{{!GzV>Tc3N*;@X__2snVlhSmqM_+&h*3fCh5e zQd|o=CtMIJR}=2;bDp2l&;3vNG4D2}?Nd216X4tx&-V^E)zfapOi(0jHO*Y4A2>fg zHK7mwa8{M%tXaHMs$g_%(W~G!V+Z*gQCQs-i70R z?dHywZ7|N+euL0dc;HmaXk|Ru?c(gv+M|x~{JcF72jKQZp=sGL>zDC*bid}YvG{`j znmKtkr}vX=r%B`K$K?3Ie}?UIa4{QmVA<^8<&T56{kKr_kV32wPZj4y@_lbsgaJ^v_Ie%#BEhQm}yn20C> zFss00H9#XPUdI`)($wi3s587r-~ESYqBF!ELtuhg zs8TTAiJyX)Y~%uQ zQXiOY|G=|8a+*FIZEwK=Pxxgi^=G2Ue5kl3LGvuq_3jb8z53l(6F2dd<+rDBqu-z0 zJQmj?TiP}!@^YSu(oY5B(t^-Vol7W#@@XzCu|Ak&j&4gf2cFi7>>yoYjCw^2T`&rzF}Gy>^FaH@>v=2n?{e`kV6- zOuVroYQ!E(vZG{5p(?Ek9n?GuGYzScii*XyBI_YT1(;kIkPfiKa|iu>6xT-zi9!jRkdKCtXN-IWT* zbVwW15?1;N`M3($Zoa}_=D;Ob(c~^u+AdG3k!UyF8Fj5V=6ol1&AYn2@BF*m*())q zwK1vHf2pvDq5+UNoIQFKs=Bu93H~yIG5YmTMT1fd3C`!)B+9N~vui3;z zFfM12Aaf;jIau-_(=Aw{C(4wVVhw`>4r44$46v(L7sscaoNAx2x}-Y<>-aNW1a)vR zcrI;2BSGs2rp% zMSJ~5yP=#JH>C_VePKkxTXH|~)0GbGX^Y7s2#&rPyc&&-I+Cf#O zR<@{H2yb$9FGH5^8N9Ym;jx%se=}N;Vnr#wLd;V-u8*s2Rk_iZz0EE4(c=%#tz8F5^1gJa0Ls)axX__#a`O1(KK~JD$HyXtQf7rE2Z&B+)B+XOoJV z>u;BOlG`AKJ`gccP*8fvVFQ!GpC!oD3bnT*8#4=Yo1Nh4JN>7=dZarJ2}WG41Y5rv zZm;HeRHnHtYlP%z>F*OVuKujBUIxSET91Dxz7m35cv61aEUbA!a{lMhuv}*C<>s$K za})s5zvgXD{D>j(|Bs5p)cS_sEwp0iLBi=zQwC@)=nUEg>0uqs%vstu*Ow*!A`Id$tb)u&aKWx3+-*eawUEF zFpYw2-)X^TAMcUvWjjbB3_$^Za@3whwfq|5^f7| zoJl_{SsHC@Upzs)QhJ(b$lZgx>S9|$B!ePPuQQJLaZmg({3BMYy-~ERlc6!Mq(1x1 zg=%kipPUW)sKax+UDh~rENom2Z%zHzJxRhaS<>Hs#SL+5y9WB^f5@&n3XF72o2L9rv*{WisO5NoCy%RTrs$8`{wZ&WN%`Olrs0P3A-=aze+s#A zAe4V?ADg-OWFb}pw=6z;STr=_A7 z9y5f-#fwAYjFb?8d|@g-$X3I`RNT8i3rfi%SV~$FBQ;m_L%*EJDn+$+0m97(DdF+sh`@7kg4?iR6U+!3zP-Nm~2Ma+a@z(T2Y%ICny@7_pLas`IO< zrulIV(`c?-!kE(ijAl+$XyjX8cSKow87)oTH1<+X|{ zY%?34>zu3^bj(@a@F)b9S-UyRA90dF9D~Pe1@58J+OtL_&84NQ@Xugh=hLvl+33O? z`H$2>PZ=}h^3n|OvuF|_NE#;bKRufsJFX?Mu?RT2?<8Nn;!sfzH%Zh@H zM4co|uHgMbVb^8iuu^OOJ=4-`%RPx*(p<{g=6jX%rG z$ms-@JtNG|9!#McFo{WDcRFx|e$N$8~!sapJmpzvs&DTo6l=`MrJU;I}o0q=OO`b$^a%d)-Q_ z6pan$iN?8lQ}Vu1;kx=5b9Dr^f=xnif!J!*9#Ho;y&FeY2OH>y-9emY7hNypN(D4C zdptr`y!ix0ziTeP{pM-yjqZl1&g~{d4364}n02$b8jE~Tp@cTc>v3AHCV<}1uGaxZ zv5oSOVkTK!SO!vP3J_x_N|h=(H>g@U!TUe-gt{oKam^+~MBE~2PE+q$l}&|(oE{c4Yi-ehd}>v*d#POe_L= zeF;qKa0sG5OT_&TYIRCZ4%HMXr>sV2z)|hYwWy z4%MI_hY28;t>zAu$hbZ4DiOnz21a?j*LyBMb+ss`DblKja}TNEW^MKi%HQDC3JhW^ zZ9407iD$PMOU{pJ6CB8O{x(B#fC^5#kb>$-1_hx(^V_PpR)6qbLZRSRUh8o0S}xre zvC^GN$)lF}2Ui8okdp&H%LN-gKH&g}_o7?+f&cwD{}WjIZ*DCT0|P+ZqobYQlffjL z0(qMM!yo<6ABvbsLs+yC!=OaJ|7Onq>wLlg;Z+be5Jd->juN5cYVXMz+J-rP#2v zxpdhzk)E3^L9Fka&@c9s{Guis3_1p;f~vnRe09Pp`^-jPobjqa5V9sk;hQLFMI%7e zP$UtIBq_2Sg%oN;`M*cOix_AGFvSnaA1|R}tQ7sJ^OO5IRGGkYnXPEjGBh-Hu}@5l z)Z%(A?&l~SA!0KzW_-_A-XLZb4L;a!bu!qPq>84sByt@Sr}BT0{R)EUSyTpd*OF7c z>Sj_(skJfa7-7YdtX1@b!(3GEb6p0U-DjWkCQDfMnM#6!dK;qH%X(AXD9fV6u>k!D z>i5lJcribOIk6xL2uygu4+xxG972tNkBPu$2z&Fm0Np_%V+0|WedTVNDPEUVO}36P z;5#r#3AsAA+&9cLz6qfluCuW1IBd;QP`Cv5Z6S;^Fc2=g-6?H2W`bj?u$B}|?|yb3 zOtIl)e2iQP>1GQaknB>1PLh)pW5|e496gSOf{WggXA0%0$DZ8*TS+oWN`%awDE%nT zTzSi1z27J*XzZJRb&Y>PhB3rMP^b{Sau*W0%HdPnFxjCK?&_}08#xtOjvUeVO1z7b z0JSyYbn6b1;XDq?uFgXT9t+r&BketpY9-NAQx`EGn9;toS6%P23|Zi$uKo-c(J;Cp zYU81$4U%1{qjXx$xEMBQ_f@_Sx&tACE31Q_3|*+JjXxyp0OKbkOpNU8P{=4B4gmFN zU65@ffRIUuH?=MFU@D)ewxo*3s08e49pY2-P>_HPO97LfEcz-UDC*k^9(DXjZeks) zXrw|&)s#uT_nN}fG$6vGehRYQHLfx&t4h+_ltD6^cS&Y9^p>iIz~!9$EBYEBK&Eu8gG2tHR=CH*h5-IlrHD3z1#478#NF zc2&i-hyAr|Ff+ceFE6tQ>k~zdxhme>9lUYHeh&%YhEYSDFssu>M3PO!Z$wZp?b)tg zG?anwd`x6F^ro;F6~_W1z;eKp@pHA<79c;nA@6@nWw4!5OhyvAVnU#Utq|9v*g^OF z^D*#R8pIc)H$r9*Sgeaw-lRTjIudI-nVbcKG7AG21Y*OrjP&M%i)j0ACY?F*!G&d%uZeko1(H z{1|8~Nq+y@VzFENig2e`nJL4sGB2lot(JhGzSj)%YviGja^q8EdmEMV{1EeehY!uW zS)7Ol5R|V%oN5@49(0dyqOhxL6aLr%*=gPzUxWOHNOnNUJWH!m2evwvqvB%3lg|26 zPR+^S0?I1d>XAL;jEfOJHMr=M4p^Gko)8fYQl>lJ1ND(yVWlxU*jZ#+%kpFdk!OS~ z_q+4yCe@M+wSwZD%aB+B0IZKVh@l0C{q6G10An#oM2B2CWr*1mQw#v0-ws(2(9oNQ zr0r~Og2VeSQ!%%+Sv7o%19%?KT&b zdrh{OG(x7>wiKba?I9#6&}l)H0c}R?>6(EXrn<=RZK+~u-L7s;Ix>2(>V$nX{m$rQ znL`{$?Ugu7amc8&szqaxhZ{Do>OMLL{n?pTtDVy;Q|jUDWRWhqP)*Iux^Ggf1HLXL zf3|!iRZYP#_ZdAk(+{NCS#<$R`%bagRsFN7{`!3bbY+{ko4JmTLMdgF5kptf>0^nC z=ND_5)qUHyG$-75If7Z?{3W5+^8NG`diQxb$i-DTG^;c*q~ce>pIp{+mPtjB+*W+`JSm^Vkp=%2H;f>bE!^-1d#SFQSpsCU`-YPX-<`gjI%pvbZMUvlSbtBRoNICIJ zu|2l6()x)nG2ek{wPiB)hT#N`e`csb%10x~*oGF{vJck9!Uy~$RUVo4I&nXlFq)Ji zBqt+5t>hwwg|kT8vfh{DigAT=y8hdblCDaVB+AMiYyQ_2H=o0Wtz8G)G4wK4BVzhg zGgix4YEQk!b2|6HAG^>xRFRt~H2DTmmrn7WZTc#6b{dq9kX|CrqB&Yp_7Fqv(sc0T zMq9>*aGbsoW{sw5guf4kPH7DsxAh0;0_HHhu@^KkZGkf-V8xH^ZmOuc#@ve82lkI~ zs(JolNDmhDGY2gpl6It$po&<%*nQ;**uVx+5d;fB&FWPj;f7wr&)H8)0(Fz)%-`QU zs*c6$$+-Vuww>J%@ZjR?3kqiE5h} zWy#3msk#bl?(L~)0R4TfFkM*W&MrS0=#XQgARa=IhJ2}2zL42p{o|6(g+R{|*NE#l z_UE7#;SeL&BU0$rW$_uiC&a;Hc3-{Dm$BR#m`4GB-fl1-+|MYjVzcoU_2TNuo=xso z?HIEnlLiDeae#N8rHraSC6RdrDc72|-GjhgJQlQeanhZ+qDN$kD*Y@(4s7}HkU&k3*m>K?=1TkC?!F0c4eHfv#AHFxDcyiN^A!|p~Z@-08r{gBn} zSRI8`EWb(Sd2)yI2Cc+CmaJv<&+s8pp6)|MZ}J!Yi@Ie=;UO&uPRYYvkAzyhs63^=|mp< z>b62M&@%PDFU)LNAGjhZU8Q&<)UTaV{Cqk%S!Dkr*@s{f>I5b)7Eu)a$rmDfM_6_H zqb;Np4_LMuS?V@=I8@;)?>*`5%el|LzbcgM^W1^m2k75Y$W(!@EsttCMDrdf8A6`jY z&H_;0s)6yQ-nSYTj_Y0T%4b8lDoI8jwO7?YELM{#b%~{$NWHS})&!zXeD2u(#0$N% z#SfzbU^HA4N0Sv=A@zK$ND3{8;~J@6G0j_bSDm>IkpeKSkk0+!204*9Yide0xRCSK z;Y?y*|4t(Ql-CdBU*CUQwZyl1qH2|t41X6))S%S&@5L0}ixbgtvzktbvYFL%-DeA+ zqacM-HgP>J@4g$3j@<#*Z)?e-%a5oeMZE zaRi5)NpN-dZ2~@!$5XU2YW*S^PLZ*{3gw@H*X=#uzNZ!LOyCsQbV=(Hj&14%I4jm- z>8R~4(2^zpO1qsU<{Q(U(0*FnoVC|>W)e;KO+(z;D&zYi8YBgWaLC8fDQi;2i;Nok zLrJ(O{pX~as(r`|+(6>$=>W1P`C}|)zbK3d5$YcCZ{3n9!(J%^M;msiOl$JT+4ZkK z;Z3&e>NWdull0>07UmR*{cv#DDTp)UL2zlU){AtM$P%v9=u-LPsJ+l1{)ouwEL4h= zsP(dwcM~R-Rbp5m2a(TEJ1k`DMEL+0jOG2`ZROq{eBTJk3}#6BzA*Pr#0sw_wrFIM} zGhHHIUx|WZ_3n{czV{o8E(pP*F%niRJ#8X5LNt?NT6_Mw|EEACXX2Swiz+N98)Egx zZ<(o=UbiW&iH$9^jFe=Ad7>W!B_0uS{$JGrZY~nhsZp0P-)G4&=IkX4*52vP zx^8gi-C0|Z5%JI%F@45Y&I<(PAKC}(w|AshaP7RQVh^|@RtQah$*z`|v%E+u#OKFh zO;?3xIv`8Iv`Oq&4u}3t0^?EN{pj}27-z)T@yfe6{a(jg^Wi>5O^y=tMR76Xp+UDJ z6tR5FzDoe8e00-Q4161HnDj?EOt}o38A9u7ybmNH7h8+W_#1gP zJ!|{&whKS1ahAxLdmXOBSSUh-v?Rjr|6-T{-EGX(Cm~`jto^BSEcy~OodisAvlkHi z!fxNETYLq}j_yX0bm6DW+YZ7}XdQ44JcKmzO4fEqux-S>sU9G0n(sUz4N^ zOCvPWXS>mXR%jiZ2~Dy7swOA?`SNX2uE$lMICR=SgH-@gR7hvoO9Leh|A&HU5&i8=CjHa4fIXSA#3Q~<*q>vB1?rg zJZWnjWd`UAc$=_zo!zq99_mXab;`p3ER%VLP}tg*@r-#(fhN*`lu9 z-Aeb$O7i%orvX@Ku*?)g0Nu=4BbI4_ z?o;!}xM#lXT}}&~XQ!WD4psg8PF` zk;odPn&v%=(JH&!0i-N=a^3?>h=}gWr)fig%@O1nObGqBEo|wla$v7yIZd=hYctJ= z+H21NHK7)T1lnd-tGg_$SUyT$zoVNhLuAN)1RDrNq3pT{ck__O_O&L>scW_ApvxJL zDfeaRNkI*5cmJ`*@f1V|mfgzIgTH(INq%xEMQXo39{4OT^b@ExJtBU@82(B3?X+y; z{VDPH_cF)5%h~{c#U^?o%<@iz?q*g5!X-XfMAY9aza~CLI6^h$r4_(Ez z&oteCZHJ=BI?RfW;7AtCauC2OVR>!lbC;ki6b%ImMD>pL5`l{;Nj|A%Fv;yvjj^ZD?6xY+O0|Sp=`qJDqxnuAFlS8bHB=Y&Ro7*GihQA zl?c;DpXeh1mb^s-PBa>`+3JJ`e_N3J<;enLJFa<v z-&4E4>Pixp)*I~i&Z*PFyu67&qbIhTko3T!!)E$C^u#(*186ZU=K4_JU^Br9^2DZL zG|`bs9g8TX!C*cpSR@(kxyq2y8g;vv_pcre@+-VSX9; z>G?ryNq6yfaI^o!GQC3W<;vWdV!4Kzo2wrR@0&dIN_CjAsyy;Bnr5QK@7Ei5vkhGX z=k`iXlG1DXrT@+XVEx2;rV>3K9N=?DogP2(WJcN(*7OBmPadYEnpUkxonFGn|I#3E$ z@*YH!#p)V6CuYH_h_={YJ}dQ08`FIFvLE*|!F8>9b6hz_>neu1U?^@} zGsrY^aNw8m_i?;7RR15P8-Q_{A%Ut4-oud>k`7sCmg=9ru={iOJhb9w;{&oivBTrB zo8sqY8Y?d`;oy=N6Axzsi&D8}+^l@%ni4%g%?c0|C>H0fdkBZdtR*n34pvCM zd1g;B#8B3}>T4H%bVQ4kld6#=+zAV}0=1P0-5;wrbq|B|~0j3R#~fzN#$!u_*(CBkf1s%J)PNoQcNDzu6ugZh_@`_Tx{&-Iy}`@q*UKIgN-k5QWSwk-!1eyZD`kf zD_{JWma*`Ek+<)*0l+p-{oMDdwaGH=o3>c`wC~RL78ITG z2lxy65z*E#Jwp!sLfDM>j$}thg~olM0^;u9u^M=#V1{%j4>-ch$Yyw+ziVB-?ek#& z2HF`n1GAo~9DU*Fk~GN+^9cxaD^p`@`lt2UL*G(yEALclZ6Lq_X$HB{_8S~>kYQ7pqXXmS@U;xfDjpDp-sf_KTIb`rV_aOUZS`OTjIu#vwEev^|L-Sv7e?>jN3A;J zNIK(5<0s*x{C`dC?*N<oe@bIQhWPh5-h0Muq@}Z3rYoJ8PSPzgm zs;gZGQEKQ&Ef}(X{^}vIme)>B-cB&DC(&Kx=yZ#28ddoXkPf^J0QB>~)h%64o8-@H zvSYVa*gYjA*D_H8ACUL}zdDi)k?q)I5109IT&wrst*cjMwRcWtGY2u&K zW%FnUt=Y&&oRO}M*6=veA}omqbkS;;hjzjM3-hvPC%LD3KVv3x&d zRhhLt10u&WP3=k{y+Zcy=54O&s&di zlN#Y66zYi>!{!H~sCVESbh6vjqKICwANQTM@+4|yq64$9-^b{ayuXhI%EOWWHCiha zMNIjGqW!CY5`SJsH_4=`o9=dZzs2ZY9i$mp&+9)aXKB9jQ`5L@K0_y+_7;rQx}Rm- zFNG<5X+TDD0Z;YAJ^rAYfj+bOJ7U5hA%UTc5O7M}du&Kz08}uepzbNUTtxvX+ z0`6I8q{ENYq}%t5!fsRt(2b=p1i{lp0G!L2r#Zy9a%p_!;Hn{Vt_sudh0a}*);|rj zEn+EcWZovuY)b6}tpygfNWQ1vM+nrDSeNjqPv%Qb94jBMo(O+-4sVLv#YGTXsv4Mi z7oWUpr%*k^aM(4Oj`fy(zcjE@7o+zZmaNjiAF^@ICWOVUXI`RPmq%g5Xk6d`c3wwo zVSXd!Er6sgLN}U~D+$WinZ%~xY0n2nR#klzzA+EGNQuzTLRK~BO2=4$DPSzvyNVhc ze=#X(JEN<6TMeoT{pB6|osY#r%lUm>hJd3a!LY*ru(cGC$VYUCFbI)wm1DWTrB`94 zQPN;ydt7`cT)&E&NYZ!6JQ@KVn8AK;OqvrKDZDKZAi^Gn6u#nEgbK^u&mdGt28qYB zp#W_(ZByMvcYm{Hx<^+Yf}~x+_FI}9Lz_9{OE8bwZwd}bSPeF!8<=-oqV@T|fbyfj z^2Clfi?`D1_{Vwqu~ro+RaWBLNvibDrsU-3)N> zfPEEOSJizNVjDa{SiZ3=f{(8Mmflx@z&yXtj)qOb39v`0hE%Edy(Y1GA`j#(dekQskv-LJ|$*3-4W=09Hf5y_4)HeddmM7+0n z(jF>Usd&-|3%wQVA6Nl#%MdNdLvquBUNO|12%F1{Xrv>AV2eNAY221Bz17vaXg1g+ zYuLFjDPrMN{%>!=Uq}sGOxA0vvipe97Va#z33fToDFpF*_`M^2C{;T!9Z>N-HqG>V zBgYT_Vtqkbo)OOeDj1;Ko;kDOpOyILo@Bj)f`TMEw|m85BWB7)-Vb!2sv9sT(%wc4 zqT}S&MC~_z#>;e>3K+!VE+sJ`jWC|t|3o4X&k@CDH2C~P>}J5c^WA#^($=-Y4QNHhCKu|hF-BIQ zzjdZ+?J%$;uD?mKkeE>zhJMjtix(gtQ1nyKq7hJ6jie^FBr{6a4#5S%>IbZkUps5Z zI#?~Sdu&yNPSdS=`!-&$Zs-n5R8`KeT!#NM1*83(KhxUx2dO+a&^bBm}aFy`^KWeqgUncgL`+Lkk%eT%wP`jH5e(Cq%B zuYOwYB}d7{X0Utf%O>?LfqXAN?Y3j}{X%yuBwa}bp-h$}R)F)9DE$on3S&B(yswqO zJh6A_&vopi$)zg5b6>g^*0E5>-^U#6(A=rP5p~S}fH^RMP)a>oXn!D>;a8o^$xOXW z&TuC?sz>;yFo`#sn_BMox@L5%aK{T*K~IOtwkGe(RwJj!2H{)-A99>y@+mK&e&P6u zhpiWc_d0XZ>L-hWY!V!Xrg2hdMvfxe~NFzG%hYD5cAhUbf_|reRfCyoW?}q}*Y5R^j9_9%* z37;yzG|v@d?c45J^E#E|+lpC8I+3S#!%) z(l}a13;@eO3~fov07jsO4iYIAf#4)^D)M_tGKQ`ggN&7RPMFV|>9hl*w3v6M#X%j{qq;gAMLuqj@AhGJC&PiTw`s3?7~D>bE4Qb`n1 z&})UCewp9LSu7mRNy4jLbY9(SC~Ne$d6Wjq=D_eG!9de5`5!TBfea9w zmQst+glL3f1^$kpS@7gCEFNR%#!OU*2r6cOJc5*IObG0_(k+BXbUs=RDRn%=p+0;$ z6`PcZiGaLP7lOr-ev30En|!c-7_gD^yL3cS=x;`rK<#?!xz)Gb9J>Dssj${6Zx3Pj z6yGa77eCZ|H@Q7NSt;hf28N>w<(2nV-}Ii)^r{jC7_tXZuDH`Vb~WPtz88{5EuB>L zWVY4bHo_?Tfgq9RS5oyf$5pxfu<|AQ=#{WAquaio-niJH5e(rp?AnFSY z{ajXgp`6ki9-PTH%;tL%Ak-BS5*}lDH-36sjGzG4G=(Mfd!-5u(bHO)lT}i=d+{um zf)7!afxSDXmTP@uU|8xFp&4txF&> zYG(FFoSZ9K5F<5TrfLm|BE(@%V_<@CHR^D%Vw8pI>?Dt#>UAukVuc0spZ>DR3;eU; zFj{n^B5hU-n|Kmzm)q^aC3sgTS4CbUh0f#`e=d<8s0<#Ewb#4~M)Fn@g$@ofTqMvo z?vD!*Rl00SaSDHdSQ-$ei*2vk;LIL-=z)>XC*s=O<^a{Oo~ItN3cq4>^B8_P)Z>cBy_=LCQKyX{;up!jq=i_O7Y7}}CHbKEwzdJlZX__S=FA@ov zs3NywpQ-g< z0^D&x09TU6fK#XUVE}jQqbDE=i6nSw;-`@l9m{(+^83sESz>TG{iM6iMwhtiSLuxp zPJfGnf;9Bt?)9mY2Uh=pb$J;`QyI$;qyzhX z7+Lw18|x#O&AY#c8!^1(c=s!)jR4O-?fRo~tDB5odbbgQ0Y@ z<><4~sj_Qpzw`M|kfCnqLHUG~tB|C^l(g$%QSbPFxws{oUNnZjIlZuuextgoRk-Yq zD&*Ol%Z^Ywq004WcvK?`^F!X>xOFw)%!{maBzQRV--0CxRmsVEWH1~>&+E^0(0l|N zW!}pRRx#CNB2Y0;nl}Uh(f*S+S@sP<-4n*kV54*B;NYOvr22c8IxZ54I;;Ig=STNi zLmn39S}2q})5`)gvbS2ZpTc&$oqHc6F}HqqCiB`m;ZIn$atZR=ezum_ShU&Cz871< zjbJf}SUHXs8M)P}7#m`zS9Kmcvm$TsUyV67{G0JzEeCId9AymY{@i5Ste6m_Y(h~U ztG4lmjfGode$Yq6V-{pik|dTS7<3egA?AY!=IoI0sfw{6B`5Vdw{()OIUL4RoO!qV zD)m`C;S;_9-{&$v;%V9y2-fJ>|aXq5)x*DWtv zL7$P$wviq+bTtGbct-8mip|HjB|15h@JLpC!(} z834kVRT>D;<)*leJaq?1>_|!0FSblu)XnL16Wq|PLUz(r{w7S1lay<45K?hc6`bgpo~_XF?M@r z9(5tC-Rh2wQ~qH6f4PRh=##_@Z6($W+%nG%pMod10+MbC^kXF8kE=8FUftocIFpw?jw*(cN85V7As)Of<6D^Ay{ZQV z1?qg?si5B!KFDfPsP~^Ug6$i*@xIw4v9j%b0vD+iGN0HCma5S|=7y)-Q9r>>rAv52 zGmk_JD$to6(p0R8zeC$%HPS;YH>pS}Db>?Ig03?f?cvc5=O>}h322u5`pP4bSpgl?Aif3#ey)rVwl7a}?qzh0vrYogYQg6~P+|>P z;W8o}QF$k((l3(7bH=DGM0@#VoRZUgaTfXANIp$vEX;o~tspG&e}rdUb*r>hBXZnr zw9k5KN{87?Ta3}zhPyF;niZN8&tw52`l+@Y(i?FW_ifh^S z4uqh=A-KD{YjAgWcX#*T?(V_eo#5^kJh;2VFnn{)z3)|hRrlQY*RGn`)!n;$`C7ex z5mZH_jd<&2u{pY}DthTU{UvefxB(&kK-YRf9F)(Nm4VIuPnS?B(nv--u4k2jv~8Uv z+dkz0uG`yeA~10TaTp#44uw>(3RL130!#=-hlOAcJfqiAf>UMJ$<4ahLXSGgUCxlP zpE`QO*rtp_V7pWmNsS;Zia#`c-SY6{Nobzo4+4;2FmXJgeq=T+?_rvqHolU;&z=+s<;T>@1tqL zC0XaPl_&A_yj!Vk5s`VuoJ+bJ4;Q3e(Uy4J{i5RWTt}R5tkJ1}JOZ#CY(=QdXADGJ z2&T~RB&O(+!{dHFey_ksciJ2-u4wsE-M{<#ImY}RPjE4xF3{wiISmw(ESqjjqcfB} z#Zq~zhdIT;*}SkBe>E)eG6SSN&3vr#AU;z^*2KtThmg9&Z+-s-PW>1esey#kGIJXl z*=^{5(M|KsrdAi3$L$>UcRV#=+5MI^VYlCrz?cqu+WP#T!^3O~+>Sl&{ zE*HysiK6?;rY{6*&f{f{NGbO>%5H-h+8q0`tom$AW!bLmB2JphCO>2ft0PBp|BSW+ zWTBq91L2EsD(;A+r_~w!flea-X`2PW*pcsv3Mm0Q(|aj2CHR5Rr^2*Ha{;o8Pbcccef^`nx3svB8%2s?SwL8yxWsZ+IiMN?i|LW=txa0`J& z0q?ri`$dw24*ZxZ(14M@Vuc8oM}jHnrYEVNXu-HdMKfyYbZ&PHjs5WG_wQ`lxhBYH zlNsgvbztB~X*bi;T8*v;%L#P(U56WKoSo9fTAbE9vUM~{G_Ig2Iq1CvUyvj$76=#|ldZu77szPNG&}kJ`oimqFeHp9kgK*Sb)IC?v)g#3B{m9MB%W zh@41#Ms|i5VQ6nKnyqi?gA4?3-=(#lL^K;JAISg=@tW;-hP1i9bx*{|^V=-z=oXq( zkiZgG8r-waUTf>>^;X0i^T2y=vHD-b5X`}UPo*7r!Mc?f_PFp5(=y8LDh`L8(+n&_ zu^{D9|Kuj27Z$6QBf_ty&ozkf;Ce%speDs{@AJykcye%bPVX=cG>z8uT>I1roX#>S zvxTwDCL6d`)V<>|7o*c9Jnp0pM`eB@_oNw@66Q z-N>a3J-KK798$p4J;(*S-9s$*4*qG=P7R)az~tjT$CrFHjkbscJ3I<(OVuxI-S{Do zSV0*nJng6~)@9A$S=n@6@hkC{A;O}HI)Esglo*~bu-+G+o!6&!6aSo`C$sSxL|32Q6@Kejv%dPc}B}0>pMun3bydglI z=9cBlZUdC?g^aoSEg00g9Lp)PTT=?}Ui{27rf0(MikSC^ew^=Mmt5JZqy(25y42v2 zg8R_Ly|0BkTtD&q!v`Th+Go#6lfJQhFF&gyXIs~d(aupS(rpBSVZ9K#M*q2pNJOAq zT+$x=i4h7%$eo|fggCU8)!Kuzyy+`FMyVe0=Tw-!h8`VER1i7b48kbqSw<@$K#aibLk(E$Ryvmi57~ZDN2>#sfSGGbx(nFBmdl*30I3WlE)Uk9N1)iE!r=?|CtM z^()2D&{E0;@@nRW&aJw*P+-VzIKsR{WB>v9GJFVYvBJ?}5@gJZ#i9XMQV3vdv@YqL z%M>Z5w`ouQR&?gIpm=8x5}F7tLQiP4++QeXCQai+iqL6jNRn%pvQz2+o*Bx&LK2T# zQ$2m>pLbOKoJ}xS7wuaN(()LFR^I?^li* zt9= zf^kog*&+c24Wvk9aAI^~xE_jE-^0O2e;QDYvl|qt>S6w0B>{k1~Et8lij_ z!yn|Zp^T|tF1gfu`)zpMbw=pt@hwus^5h{Uk1!<7yR<;&9>G~0D+f4!r`tHH;Xk8v2j;(o|Rg9A52C9G^J%M12VPl+6)J4W-i0+I8YsHCAy3#LvM^U+Vv?a-V1go& zlDLyvg3od1KtZXrD4`R?F+kJ#;0JkmPTNf4726#%BW$AF&Y&DgF!|tUIW98PP7fU1 zVy*+8LJbSOf{|zAM~f|ND`J{Y8*L-`1r!^IniU|dV8QEU1f`wlNKMeugf2sy;a^^? z^RgTNBq2wo5)Q1O6&9Lt$mKm;jFL?Ny_#0& zDXy|pmz7(>nx|e~DD%faP=b~vp|Wx$i%JkmvE0KbRPksyd3fZ0Av2H2G%s+6>#o#N zlh}ESzLlL(HW><0QYkFh+|xqHLIu-DvF;Q3`{QYOZIq@GMBBrHG)Ob}w;7brrU*-d z!5%`QgR=l5MP{f50|+c?yJ$&q1^u}IQS{timjom&-j3T@NW#EuCF<|tjQf$X8pQlS z%N$$G$GUhU9GOjBZ@79@6W>mDf1fr!2E}g z>7HTcja-v*9+V_Bb_O69&8c@r#%_y{ml(fXCu<6^DNc#{*;$Sw^1 zaOw9qS%V3d3Vok|hEY*_cC>d{Rai$JE*5{0{-zC{3A57F@n<$F##FJuXWc4->j~%C zX$F9e=u)V`UtEh+Uy8MULjs(_Y7d~n1+cP=VL((}BvbpkVQvl;2lH<087F_?pU=t4 zIy~&WFbo~lq%7C!*@zROvAdZ_O_Gzj^^z}vhw8h}_ePT# zw|0M7COeZt$M9OahGfBIlZkkjS4LAZBoKhi@XG0Db*h`M?2{?d1&pO;U zy(LN?KP#i4jB|S@OXy9|(Mlz5Z$S0Rr*fp$h~t1Os+XhhLehw-l0^1$$C{J`lI@GF z1)$RP1$KlJ8BE^-b1k6*Y0-#6h%w+?_|=h};0HmZea7&G+xDeZa)?IdGu_4hg6!v; zxKQszc;>J4*`6>BRT&GmF4z&26@-0+1uP~K4!}gw?@2;2G9;+3$`h^p8x>0NBW*yN)P=G=r5O$(X{a@CR_uG==wly!{Vs;L zwtLX(8{?E!hV)Xl{+TVgwdd4k^5T)Xhu`HDqYtZTlE4^J11Yg9UVC2+eqG-o5|5?AS?cz6Jc*3n; zi9^w-@7C1pZgj;LPnnFOO!ve1CSzPio+J8@>Xt;s_+t7t#<{{gx#z_h*qoM5HA4fz zx~-L4Jw>9|82R0Y(O9M;`{spmeJ;eC@i693R@!v&kMtwWiC5FA9LnSbt@eeFrcRDH zkEK{@q3Tgernl(sw`KMRg6&3a1&eLrM}rsfur8d7U+qC99L{31=Irv1JBaw;_)j=p z1WxAfWWtx?quy0fImz)yxIyE-t!Gl=*q7buI6>C*$?o};B)_ep-E@g=cTVju*Nj}T z7@XZqY{lf!w7*4N^=3MRc|cwJ+EJ6B-R*aa;^i3dBoF#`z5zR@tv;;B;pcMWF%n~X zWTBFqb`}e9zvzfPx_n#9$&%!6W&xjWD9{PqgpHU`7OVTgEp2pmoDMby! zIKJ`4&=K~_GuB`#z|K(`qu^kBv2I;cq~08qk`KvcDxX{bp7nq+Nk9_XnLe^5w8L_e zGjK9JG1@)>rHi*AHK~NRB#dXDI$0A{B_g$;*$m1cGiwb`#radX)RBT{uXemAOOEoG z%Z2z^HV}jz2W=k5-taB_F{HN`R~*=oq+ecI0wdSqlaqf4(f{U5gwhGI@i?|y_>+BC z-H(E=ROwc z`e@xpP)|{;DkxFR1ua3leobRz=L%LE)@`!)J}s9GqSn;swt$FJo67Gbr@H&nsnsfN zB2J+HDJxJ=s-~9f+930qLJco_VxPJLkntIRL-)Uoz83V2F@@ccbr=>2(bINR^533;XZB23 zxz3p(6?Jhy9~Cj_qF@v&H~HXRrp~(&jXUjN*RJZJSgu|-m@mF#Vi<<85Zm;~vsO>_ zjO|)K!JI+)!CFLsd9X)zxU%Vy;yWE9v++Z_{$Q78)B5Cgirgc{+$AX*Vf}vB0ZV9m zE4I*KCY?jCCYts7aG+Ces_URH@x!wqo&xZsZJXAYWJsvV zzU#)d+X^{+=~ZH>u(+HGe4SXqIDAI*b8N%uUTjfs9P7-WwU2L{op0^wykqjyd?Q>G z^N{NB@!Lmk%pC{X^rAv`C;YM^%fc&-VX6sPbU_1%5Gq2%P(U5|ET{|murAhIdh5ko zzfYZ#Hpg$9o}VpQ*L2$4-X}C5-|{TJoK65+3R_;^u9H4`!GXRv%H>@3z#p80N$w?Rb#y8Mavl%7mxz0WlU7$fpe~JcmmOBI6b?#=Rq%Z$9VD zhz3AA!HcwfAy(Jf7^k<4XWCDVz;tZh!M3m#K_3$_kTr$&%S9S7_322xE2fOc{C19( zoq&sWoz40bKDX>^Eb#-SP zh_0#_wl=or#@MbZ&MnQ%&^4Uxe^(d3qPgHEcGg)~vZIj!U{8Y)Q>6Xb`z&LcIA>h+ zDNw^#KAW$s7i&@Bkn=e&b&itKOfluhtB@Cy$hP(3k($0=poGEfXxSGb!h^)hsj_v{HS<@4vx6w0DJ3%Ea*rO{;MDJNC*jWc&72Hrxk@QYvvTr%oKUjsF& z<3z9~Q^u#$<#W;H(;c7jMQvb@T4PiLj6v9HMAX#1WiU^6$8Z)(bp(tLF`}3o z92x^pGsyN?AQ+iNi)9v%X?N%*B)(Eng*2b~JEG`#G< zI66u;=vG6LT<;BPn$7n47R3g%jAIGt+?fdRLu)3tJjnAcT#AB4Sx1j+<=?fl<}{4F zx1==B?>yOQtI}avMa2zzLE-O8Cqilg0iNWt zXo1zxrfDNrZhtfqY!7tWWMM1{*F-$O&4xTb%I0G-_CKzsdtC1vhD8K!_Sxj&s?2B> zKIkg)spI67ztU6|v7}{IN+5+VJq@}WQ)D{l0&2$+=GdmyEl52qvVV&QB;c3To?Wg< zCvQKTLr2g$z7*{{UwebVvAvjA51v*8uz8i>H;Y^&@%;DBp(_Pf;}HrZwzk36QJri8 z%}p%w_g%GJh>$8D5{`v5ol&Ep7qP*;l>LAeBfy-G)lo9(j|3daG_sIJWxl;9tW;=q z?7(-F(OBrO;F&IjbwtIQ;<7mjnhNp0LyIT+xEoa`O;GMvTxGse-SnXC4JB4c!jAd^ zca+7(Y|>>{%V!GS?N(y&Amtdr)c1-}U2tumbfEyxN)AUQ`wY|v6~uXDRCzlySVlPg-akS_QNBrN&>Ak` zv3$JyZ4>ynANOc7SKA$^tbjCLtYEKm2I@CY+@u)=zNK2ftRn1HD`8*$1FAI zGuEsFMhi`H$XQRSwH0QB3OJ`EYD}W3cXk|T-ZZXSo3Td zY6T76e8Sg9H#Wn-D4ydMgVpI2WeM2T4vknm4ZqrcE}60##apnU246=6%Z1BD%tg&j zqpJ2!m$vsF<`-XQz*zr~(JK87Oh3ay4_s3tnB%E|R>1&gX3(FigPm3wGw~loIh6zf z`8f=8IRBD_Ov`byS>FxX^^J;*e3WFo$!mST&ALJ1Al7ESE(4Zu_BcBV0cI`)-cJcK z99fVGq9;&qmGp(AEc@*mgcj!mW2s`y;w-gwbO28~6#6GEC(N)&5;8K3D>@F5C+-de zNqmn`%GYNa%5_#hsI=tsf-{Npp*bNeXvte_Y*9?XXS{shH1kR+JuPIGAQ#+4ZQCZi zVp2r&TY4#Q{id?mncv?otz$Rb)fcKXQIpo{3L@mgO*g|koj`%D9b{0`7UqnT6SK`V z_`?y&#;-jIj`E6GGG}PbCw6Dem(UPR{0$@uwH?l}FZ-G#*&FnU#Ka1mt+rJ`xO?m; z$DyOh;jZABXGJZ!gkT>t^rKc0)!eQAR$JsOO>Lt(iZ+qj?H8itb`nQ(+uh{7!U_HK zUK@pbZvsYbOsuLPGt{w)`Kj@)+eVl7584Fjc~#sQpZg@yk+zaL{vB zYZ}qZJVh^&kPuDvZ4dOvokld@I!d5DkY}GlFyM97A<)1?33Qen@*R{jz@uS9=r%Ys;8};T_&x=HZlUqBZc`s#vd(X z98}`hUxh+TZN{_S0sq z^DlKcT6zC(gZ};Bq6>r-JQ$znpI^xc;jYgXleg!`u7o5P= z74~yE|6MP)8PXP2MAa$j==hPb^$`x1SSq2i+*4)0+T|&HmOACr>ni$R)(R5B7tp`b zY`<+)wd^6qtmxxF9}lW?xA2H%bGgUtLsrd9Xus^8x?c6v8H_QY^l}Cn+hk)L(g962 z)*kX~CK)tP`KR74MDXh64lS!pz_s&>)^M2aj$@3<$v6yzrM@}FOUS`B#EJG zb;cEb6SOD$7HXbCCrOb|Ejl+dw4mYJ|^cXJ0_z-E=C`)k&rx z#NkKk`Qd(Sg#L)Sl(ksF8;)~fUBcI!-3HE z-ye0pkvMEJNqJWPl{^G5#6IPoQ|?UU`n{eIe1VxLSh5o-JUb;J;xD2ZIhc?I)NOV1 zEXm89W1QAaeffeD_jV9R5emXU{k~hx0IzKlG0%wt&Mm)dOV4Nbes}jxj@RbgK8|i? zT9bBNNUJe~yHqMfLS>eh_oYZYP7iq0BtW6}nwP{``j-g)n}z@5L!lD6ZLg#6%R&gl z8{lpx4G!tr^V-5nF-Q=!X%XIR<^D~KqY-`Au~AVpUPjYh2~mv=l=%~+g4m5BebIZj z4qv{dxTHd$El88nCfC^*>usIVlmH*Rs7fY+*Z(j>NXf1 z7k|x&Xbw+na?WlEcQN{?9)A8kypi`s`^W$VuCqix>XVLU{w1f-ZVAudnU8;9C!?=m z&vh(voV=y#GYDsy?6zK;C9+xU<7E9arqTi8%26l~Mi~KzYp0~cL#^;X9=xU10hDh< z*@}EcL3%V`w_NH%3cDJQo-rQr18qBhJmv#u^=@{%e)x|LhE|&L<->U6wVEd_(l@Y_ zNp;=}PWOFolI-`|$zS2HnEg`{bEYom{npNNSKEHv6r|Vb!Zel1N*hxH_g_T`p&L@b zQ_x2cR50S%a{bsOpVyO@kOwm_>K`?Xt*^24N^KsdU<>_dL7dPmi1K+f2e?2%FBwqNVp%^Tm?Eua@Obt2aA3UJ@w(2M67g zD5$8Ybdiy;xWB(XWKjT^{DzUmarsSm2O}IzDDyPA70aiyzL~bNpv<$3H{d$D5&9>c zk`Hnqx(%iRf>cjDJ<;Sqm$FGpnD-l>SVsXYWbt-u;~UwYn`fn$S22Hd-hshW%knE0X6Y? zoveteL9kvnw;M49urZtsn~8@s&ub$~wwR_M4?fr=Ev>fnPpaZ?H4=y&_bD4oCq2$N z$yLJqnQt&NTzcuLF#yT-+XO_DDmVtZWI;$Qyuky?E|JX*oN!-t>=yF;lx1c%1QqTg z`{B`!b0$DYP~0ds#7)XN6#EZ=OI1i0MoyNl64dWl=U?fQe_8=yCX$1ZY^sA{Tkj7W zv;OZs{ChRu0?Bw-43U!Os(8+7kBBK7^6+t03*fV1rJI!-4 zSusn$5@*Bd?jxyX^Z4H@2(i-v>CqTkicUI}2y*(1xvfrIuW@@+ahuLotCpr}b%q*O zr|zBtCGGfxAGODk>CSy%FflmDO_CX8^hRid|#!6ZD*fE_WXrO_l?h*&FRVOVtZ}i#g?}1nzf*VB&bV1O!hWo%~RK`0h`eM3QF; zmUP&&AK*KQdGVYvLPN&e^480;$v;T)2SgFudfVFCd=(Yv+BH|X%qhsonro7hHS~47 z2c3?D1abefsrc(rTyU*e)br(w_U+xBdhwz(J{AFifY4g8G<71X?Y0_3bEc<^Y6=b= z3)>w~opeQ2)r{K7j<3WCS|ww*)oB&lzbSq|4R6Eo{wd)QD-9dz7oPok^V=2B9wOd-=9hQL%RS&Z?P|~ z65qe~>AbXSfwQ=a+1&nFnwpv+pzk zlWorG0md$2rwu{SB2tHHj4u^DRY5eXw(Xia zlG|la(WgEJjSQ#w*w>)vv@uV%nxawg<^FH4Z9)C zuA)cxb;dF@xsrhCyp0QuwZ7>cRD2|bd#s87*?LjPfo8QzH@oa{l#sj9?OaN^D~5)_ zQ?mJav3}Zx!$xs&oBY=S9rdW0%S|Yob-k1kjvUJhtxQDxW!LzeyCu$gN~9sY+h47| z_X-h<|FDh})m-&|!n-@OP2^adnIN@EMM^-l@SIytXlpO({h<2*wF;I@bo~d>M_7T> zwsYHf`-Z=0?q(%*yCo!a1Ua7FV*j~7sk8M54TqMQmHP{{%frri^aqoa;2Vk3dv8%t zboKCrh+;HcjBM?sKJP&kdL@7Lu(N2ki<{z3YNy%z8{|u3ohUfLZuqEq1V&m$%0s`o ze%yr0t>&!4f0TdWQVWHqlYsNJq*&|wH|nI8QU$~k`0TD@~Iyrqs?JdEZ<}2 zr`G?@lVpT5YWIu5oYiJ>?==c6aIl?ryAqygt1+S}!}(UODe;bOym%#=HY|Nw+Me;eHuXZZEUZc7mw6eW|9Q zll}3W?O7Y6)nXKyoedBaG?xsZj2lGB+fAnE|Mk%WnjWnXOMk+e)B)6*5Wx3F;W$#idw8iUZ>P-A9&A&I53y8cli_@^#(t$~@zTv69WX&dKaSt)x0if$L=k+by%Wyn^UXzeZA~Q)n)L3tw^FMR5KzJXAHz`cb!1EF>{Mh$1 z4Rwz>yN(1-j(8wg!+}lNhPX^81Z#cgozZdd3=C$+dqeDC^?ju@VuZRPMQti#Pl%_Yn??DBo+R|bPo2nN*E zoOE<_Y#p+U7&JaWhb%ZHO30t9u3Aq;UHDtb;c@Gt|Q?sahQ6-s%d7O;eX5wq^m zS@y&biOlHy77OntQ}?8G{amp9lRocX1}h;p5~b~gkRXr1EYW&vN_EnW+{I!~isgJ; z!Km3;IvI^xmZ{P<4J9=17v&so$r9o9)DqVTvq!#|M5np=l#)6|2VXWMkcTG)f{mMZ zvs86!?>?;$ywyzTh%VhwAVSbI1J7Z>Zndp#z?0nk?9;L=U{s`cNyT!hiyAYOzkE>6 zjy$`eBQDfv@(fAOXuz5&Wx;AT)$tW3$^@@hB|=D|^;k`yS~7{-RSBD%tb|=gszO~w zM<+WmK0aAhBRwrQFR$@uKBKj*o!sGXHk{O{nNdPIm8r67-L?3c=e$r4v5shg(_6|s zvJST2xV61i^HUvvyKjUYNXOmQ9CwTF!m@sTfmj~X@Z=o!+U{@meBPy&3sw{?=R@vO z7~8$9$rpkDiOI&yRPvz4DMtJNfPL7sPBhy6L3*)ixH^QLd* zu-U!$Ys-4J_gXzlEE7ZTdxod^qWj@esa?vjl=a=@1VOrd4|eO6qd0Cu>(m}3Q6*H4 z{{;J6rPDy!3cd)P38bz=CD}+N^Tny$*+hc;@2)T&#DjegDhf}qamhAg>suh?gR-nt zw@IVf=5W4A@yA0B_qBubaKw9yryXEVb^mD;DTM^zWQ#G&A#7an|pGpyPkzX(8*=My~@>Y)>n9N81p2a zHMRh3cC=9Od%sL#F6JBCaG7Z|n{V<5$FIcsS0NxXgvA24qUz}5MV)sollS@ZRP^&T z&fOZ1zKt{1JHqD3hTDa~o<2qJ2G7SwY?}RH!Xw@ZpTkn-LrW}&->uA**}_$wugzwI zWD%rkORZjKUl;Jw;_f0{VB7}raiCXEva3Rh=Qr0xV|TsIwsjH@e zTQ@Gyf6HC=tB%}MZ^)XMpz}yJ4?+hYQKTY?ed4SHxIRv$=naY21*%5G(v6f zg^9y@J4;-Qv~H-2^@$Do{l@hc$Axc1wRf~sV!}p(y`N>>6-D)3Cs4_Wh;FCPLqh)4 zPCwder?bePnlm>x_=>dg^lGP0wU5$=oc9G?JjLl}Y)3@EsKw-vPGp)DI6UdWnGSyJ zoo*6Tp2cl>{y5oc4sN&qeRf!?PcH6~QPhKJEXV)I=zP;ok+Dra&4|OU)Ln*Msa?9? zzKMU(!OIo9?BfU@xMkc|6BEnDy~&sx1N41jyYhb(&NwcpD+@cZgj;pZUhgjr zi;CPrUF_CGRAz7Kn{%ogJ_1X(7zLuUO!gF@x^0gQI&-tK%u}m%toK?hwg!=a-Vg2M z-?`tBghLtxAI3q4zvwy9SSx4wp8cv`=jG)&Ph-C!wP;F7(=1ShU;8ti5Tv z&PSqiq&RPg8XmxZJKOEc*yfdHFFd7p(_?M*m@+7FX-xulVTVU;a)cO4M#?(a$ic`P z{w;%~Kzf*K9FmFQvBFV6%HW5x9zb0Am1n`Pf8 zhdRad$7Ur6aEa8HYqm{#o-ftkcp1U}1X656W9hq#A%cI7<*t)4i{&!5@>msx1U%vwoc${@jS10Zt99j-qVr3$6y%u9_^{~v z?4br%Bu)3~x?P%x*R$C#|K6ayTD(az(?ag&D#bUR>2?@{Wx0PWJ-6Ut7o^l1_!rWyDHvj~$CF|@h8!+zr#UjY)T0(0jl zFj+!6aC@hDQV2Ijvj2^bOCfPuEBJ3;*L4zj4*_`G>k+ZZ^@jL{b33nW(2L&A+oJ%$ zE}4Td?Lf0l?fP4?{nnuEqxaWytJ~YVsZ#y8yLIiJJjxWJjQhi3%-X;Oam-*+5VVuS~;3<7O`WjpEUFZ?vNzRRPzN*R`q<+@21o(z0Dx5HkP zXF17YnH&`O`1UbFMjTf>D026wIxSN~vxjw8nZW&BlEr@^QB#qT^s3Ej+~%AxwZ!=Z#(i{}+$8!x3) zv0&Su9x~jTa;gU#V`dVT2>`QU2bWy(ZJR=$>V(Wq3Sc)Y@Aunx{ z`S7Qu;lx(K-Qu+pFT42pksZ51-_@u@KVJlNySMg`fXW5zhk@)%KF>q)%Xrb97-%-viW*FFyb9(_8$zNdrE_s1Z5!{Jz;I%_sYLY z>(~J3#!ud9kx{M*H_QGCXA9Da2O#iBaG%Ffu}sxrY&{V5d#7-rJ#X<7x>9doJZv>r z`u4hA&aMq-u{+nq`IrqPnpRGA=K0v8&(gVD5pR1PGQd<(!Ku|&Nv!Skd=EUu(B)}n zhx=CB^aA&g<8AW1emH21*i6q^R?7XcOd-M7^kMj>^^=qO_Ym{PIA|bqa=G{QFvB(< zWF{TmD9$+r5Y7|Z7`0YQ_e%Kb>z{Y8m*kK0b#o91Cwn?aO$`nBQsAfk3N0Aq3keh4 z^@UyYE|$JLk5HfC=UXIPm?;zx?4haZzYSnQ6bLuud9*CTf${GB#!bqPvh!`9?Yok8 znf}xSNq@tG7+B25nQT!rW$vwXYCB;Du7wWho08ccSvYCA7Tcu_fV_`73&bVpxEmD_ zwrX2ltNRc)qI76Rly9D}_uCqS`h|#Oc{0V|&$6)mXPO~ciYsrq&(B@eOJ^&uJ@honF32_&tC3UOl^YC;780(((0&G^r;PRM=7i zbJHxA=}+kq_8`s7_P2WV9}aSC@|zrs>{UA1H91OOR~oFJ&uZ{si8GVBcXZKF;=9Nth~5@i32ZJN`ZdklYu< z$Pf;BqC4b1e6I~d=!UED{IzUNsL3>l;1d_e84xPbgP^eh^JMDG4Gl^xc@1yh z5x=sFDXoB4;_0RlA+k+@h-lNb2Ho(>_=jKExpHWLo2z;2_Rz{tVg)?tWQocOl7q}F z@?vHYW+}5HIpe4*p1kR{Q$o(|>AG*x2_bE2LyZCXC4aMtC$a^F2_3utTtj|XA*|So z^v`DMs<``SY1hUu8k zsE7GwgV{5}p($$Y_LQ{@7@~{=cM9;EE?3Tn=M$F=Rz&kX*HwY3oXr$9LH>F=3Z!Wm z>VVMgmr?1%d!p710{>hS=l$z7?Mn_IL48z>_h`1u)|497mzGWLR@8=$>dhlywNCf6 zj^BduqnDz9F{QuzH-Y_-B#|G{jx!lmrn6sK)AdCIV{)8N=Sp&r2KDYhcy32qlob_; zVprGd{H$pk*Wpz1K!to`7gihjsQr!;hjIzc>Pc7D)M2+ar%0tO*_v zc8Pc5Yz-of?gm~<^?jWEYJDFLx5DmNg>5dJzPsA6HV^FI9|oNVJXo7zc#*I8^fzT6 z@Vu{o!h2)b3h4LR@3EV;<{B>OZ%g${e=l`E9q%afDnk`k8QAr{7kr=vT7kg49(83S z#DyDZARn(1^lhYaTdc<88JB-J4v`oAMzfD#anz37o7`yIZL@6Pul7^A4H?e!gEg%F zo2Wq501$yqDjQcg+I@&9KgT&*k1N_sd%m*u0bKA3w;ZP%Jo`ET#){O=6~ydwJPayF82cR4D=5z6b~7ZvFz9! zBzM{TiabupBlV)6HB4_qwUHR}iJoQ`?6Gwe4M5n86)%8`5t*D~(#BrOX5HYC%c?;E z27088gQ>SVjO~YnUsq!g_GR3d4dJlZLvbwB>v{fnQ4gir4D^bGXwFW~vRbFhoN*D< z$9e5h=R|Q|2@F%F$|5mq;CAGJ>0P%O>il()lmKbwfe(ID44m4VwHF#)~@#93fjB)$Fg%KaUmZ8w?r2+T!U>? zs4e8-_EI8B(GQ#p005=7a?bb2Zy8b3Tyw2> zPLJ@8JOSK~8BjHBl!A)0AYUKvgzCti8GI30OJP!+MBG5jhpS|B2WY-?Kx|h zeBALZa@>o-aW9}FQ4F=XW1x)L(|Wt*Cm)mtpX*1n+vYn6$&=8ua=fce1wvhAZ5mf| z+Wr>S?`!D~A6%cqPdMM`bS+!sfIIH%V824rFb`0A!>1i`7~T7rrL|hD<|aGV47R`C zkeuViM&+FiO{i)8tmp+_^R}hey<$!Tv_H#%!H?li%ApoMe3uT#LvY_Yo4=mjxWqT=8L5cmZDJ1j~Om9sF|#Ggm<1U^?1|qjYw`T zyhYKuba!sE$jA0Orzr^~x(W92t$xU3?$dbzcFJ0arr`FY=H@bgBz|RU5bO9}V-Ot= zEH&%ACM=rjY-s3oFrSUreXw<0(Ce9wvs+?B@(eKb$|i8>#@2zuLt8*wMki?Y<)Y(! z2L;?zL;i~5OTsZTE3dIys)#zI2w?4y#mDBT8^F$JdjJ$`-OrOfG&inmTR&VH4`w>> z3;tGyKad3HPQD&Mqjq?}yE6J6)Jkc>hXj~8B^eACGn+?Qegksz^gm{n19A7UvlUb*; z3&B46^eXi$TpZRFcm?YUUBuc01dPaaaBp@vQUiv`pkN^<%kyZmIPCE^AfV|5o+Af- zl~+tgNsHd*l407ibp*Czhur_<;5YKc72*%#u#<+S%uPq$!fJb^DQ#%D87D*cxB-*I z8eaZFNgx?PHt3U}@~P{voBhhI6<-pFOLa-^QcQ$7xf~@R?H;FtOWs%+E$X$2PF0yQ zo_3^pBR**OXZ(ai=#P+K5urV~X@I0`{oJnK=gM*BIIHtfrJ$@Ptju$JU#}?Z^kMi` z&lO(EBBBzSU{1euLi9kKKahN}%3VR1TYBWSLE=UQJAhhUd6PYUhr`Od5Pp(^}VIxxH{ zJ;|KveOicbW@jp=(o?KbKo5@=qc2_H7LjOwO@?rGtjK` zVQc

DWQPIv%to*Y$Qof8_;~eE$Q&&yaO<-PuK7-ljr-*fi-+%D^Y1K~79WnW`7R zwBG~rnJ4I~9TLDIO7LzNa~_C`E(A}y)No%Sub{DN5H{a{y4;JFhRM36P+GbrNM2_n zn!A6BH1iMEfGkpmyl?IVCWbCmt-{gW$b8;yQ_^qTqnK%WqseMbv6e~}DP#D!n&2fXsoM&KlP zLfG>w9Cj}zTjw4d6#ETwc4DlnUfk0iblzIXfRl=&Vc_jJT%@$TkVmJ7c7B0^m zyg@@TE}hE0pq3g3bd?E*L<(KE$qNKSGq0vcc}5gqT_=&wRYC~5XADMsEZR$38#4Wl0JO6TWjING^xoz)r=#AcC$0Mzt zr_*87d?8q}^_;g5*X(Rtw{0D*;ao!u%x#=ye1*X<|2Nc;e|de)nz>&$dtET)@yULdi6$4qYd5PrE!Gagii#-X*+F|%_dub86N=amUWXbe(oW8(>r$YP-7#c zN~9*t3Q!>`fyot#h#rn(loFoWYS-IQ1SG{v_wfd5b*=D5vd{HuoSAZY(E8c8K;y7% zzS%U0o;t;2mB^AAWUrA{~6P zi<|L9z(AtTgZ9hY)Fp?PC{q1|WM!41UfFa*`6F^iz+vglBbQN8=di>)Y?}FzUvMn3 z&iP+q7zUYSNzsqgX_9E~+JBpfb#&l%O4<;gTlL+0&`FWtFznTJ94b!QoyEP8t~9?6hl}?bzDocwkx_O-oH2i&iT_81^P;)yD)rbzlA|i`(HJTh0DL_CWl! z-uYFfmezhv^#&dmBp(hl9{l&$LlhB3IfIyXPl6g_Tlwo0-NiF&Ey^c;M}^<|uNT=s z1${zi9hMNbiI$N&9`D`E|I7XrA*{zJ^yu=nQSYq$xe6j?ys0%5y_#}1r zccmQYlJ@nY^i`JlRagJ3kl$@j0i^pU#~a7xF#MyyeDB+#Cm*SAI^v*+AkY8f>ztz_ zZ{9XPF(;hZwr$&*SQFctc*2P@v2D%7wr$()xZ`Zk^Zed@clYe>UpYCalkTpnZ`F;@ zb=6_d8e?73W|%3QNEVHtcSedL=CIRa{)^o=IIn)d=yJWbi(TKq(^`e4&gS57MjxOa zec^n)B}+*k0>^h{TWx$;Web`gDwwx##}O`&)Oqn7D%sW}xCBTnt@h(~fKLO~5+uy= zpD>2GG+Uw?Atwz9p~y)dJ^Wnh^b$zBn_Tf>0v&#e(|#2a&XV_2?O^q@#v{{gd%@R; z{;RS`>o%nt!own}cblt^sL0eT*|8;Zp`5j$v8XF=8NJc+z6=?tmd%#gPXb~-^lF?r z%}2Cs{yWYYP)jJ1zV)t8UyEFGU#k)<@-q?>NRBd%T8GKq4n}g+F7_f@$}hf=9K*eb z;=V>j+Q%NZ@-()aEqwUAcZnUG72eu2YVy44P?EV4(6YC7(zWH|R8!(`KFx`Rv3NB?`cs6t*X2q{WZ?L?;iyy{$kadKj681ZXU;W7Cb9&T3Z zdR_^e{#$;*-s$&lsA;C>JMR{_DExje5q*Am98l={c+ym9`3Z{I#gEx*i?Hp&cGLev zuuhutC1m_zQc2Vomcx-5;h%LhzeQT+(GsuKa|24!kNn4v0e*E`?yw_6mz}edT+`aT z%RMQt%0Nn`_pm9$mf|>;Lu#k>={=-;u9y}tU$eW{Iy+aqP++V4zwRCSNW23+^d zj>JR%J=-S%fjPoBJbD5HG_-@Dvr1q=-hZIqB9X&{s8^1BYa_&kG(evrFi0Ul@&bzPb$O z_h?s`37~0~OczQVp2RAqLwGPvdQHi|dT!a#A?>hF!AR+n9xdk^^v99ut# zr^o4Wa@}p-8G0QgRT$GO9ff!R{nJv@uw zA)~*RPmkZXky5>t?m>X&BX{&i_)i%c6~ykl$R;CZ5p6=cdIQUs+bRn{J>*{B zRJ$1-R}v3QpO_O@?>$RiG?|#n3M*N&(8z&5BkX2%mhwL5AA7>&8oe{3qTVi0kgFK= ztIZ{B@u5{2Ijk)k-eM8ZT|V5#CF3KI&uzE|h%be!JND(} z{E%B7L^$m{D$dE0~89%R~5I8=t+){Noo31W$ zoo!0w9sd48aObWE(@U&9YGwV9u#A7snh}c3-%xdrhKn1@;fqb?@B{VQ&ZtA>1uzk3 zlS3C=-w=v>P0(1!@2aWE+HM%J~dw{F{uThOAQ_O+F&FhA!f; z6_rRGSEYOF{sQ9JRmkG?OXgQ7fFgrKFRtM~Q%5cO>Dy$*bTqk*JiKJ7q8QZepm#x) zQV*Nrxuz93ZV{QE{@cV8~{6p%Ql^?1eW7E`uai?K5)cwU%Ty{EC+Gz~6K zU)H%p0GGpdvrFRk(ifL&wMa%)_14kKLqUd5dw08^4GqJ!dEd3`Usz0-2c;XZ^5+fd zw_o5jD)%K&Xw0czQUkJ)30#V>HkRN!zY3*pIj!OG8W8@@_MctgZ%5&rH1~e=3i$Pm zb%$jZJ3_dU6vQLB=B;xNjuE@b{)(f>bs}R(yecj@{zBUwE^;GTo^D7r$3NfJOA)5pcQoS7dJ%Q?A&6&0S6d z^(F|M@uH=N$p_t8i&HkA>2d2yyj1@SEw&Z_*Qr6&hpcBr$0gdRfbD1ft3MMv6g}Ow zO?2YLK|7T$m33hhFk&E1>q&5#AETp{C3rqEG>o)CVgorFo0#ef%gIPDYq*ok1>lD< z{}(j790OYk>fEgB$Bf)Hnk?R_ui=2L8|!%wsjWR##^+pyIVB*9{4X$0J~~*HW@smq zP6}Lf?4<_eH=IB%%G?$Qlxc4Kl&5z@HZ3R((y`76TyplF_-T}B8Z=y4OHS9-@hN1x zMc15XnV$*=Ku$`SV#4WmufA)WCDBAWu#V57|N2$@f3z>wAixYIdjBP0vI+KEZka3zz-6EbK8x&%>Gf{|y2A-`>NT^zVOlQHqf5 zduSJ;uBxNRZCA`piRYpF0=7ZP(Gt>Mf#lXmUh!`UeE=mK9ZU3mkbWXCs=4O@RcQWQ z`2{?SZn?2smP+e+&4zCYbyRc*|FqfhrFRF*-oN=y>|*G{9fZ$NYMNKnRG%TZf&Kyw zZC7FKD_>exItS;8F6;QO0~>ioMf*~*a#JGDemU)7RQdM9OR-#MG`nS}(IyXFiN%e3 z&HZKR7`PO_=C}AvvAzGuIMfNieM&9;|Y!v*!CL9|aVLiL7Ht!vW zPW=8E1lk|McEq`KKjh3}MZ7+Pb;yXyemC}$c5~S)3^qqog0lqsXO{CJ0>`F=1~T1y z!=#UJc+n73Wc!=b;5c81nZM_EUsu4JY{`&moy@g`wOmKkDMTJB z`23+MAo67yY=4-<>a<(;VXTgKG(q?)9DUs;3BuveTSq)BP%adGIUUqS{3)?q{~S9D9oU331~3pK)pRjnlaK;Vt} zn05XY1x}gP3SY%tL{Z_9ZWjaWk)OG&t^A=g-VL>A6(P- zNnH05a+Q(J-Q!FqT|bCIsBgE zk<$wvSEU08=M87za;*jx!1L31O&WcnCj~CBrXHF$&mq9cc(cB?~;Z3w?o^AX#8&&z1r;h(l@Rw?}TAxS!`b(z=>yZsQ}+HWuA zdpxCDig(oiZV`cwux~C}?Ll~bybqg*EeKuv-Bm9Ijze(5IEm#TL`WTBuLk)I7feG~ z6L28`m$k&g!ND+Z_0{aaeAsWvta!sHuY30IO8Kpi>S@()VLo1?=5u7bSs(AUGm{}d zNBLO+SWQ4rai{<_w5G7a&-43}RHYwCS}YOixy@8-8yfV{QkbF22Gkuh}7 zH=V+tbV7!_vgY%AKh);gD=w0e|1ald{ENWha-P?7_W9!NtEjX7JJZlCE3L zGv#x?=XrZ42Y6}J1j>3{8cW=&8t8E=F-Os*hdB<-r2R_PnHtY-TGsQs-4o;(i>p)m z{n-F^Q}}>huN7k>CpFcGPs@D5fVx_k)sFQ&!hiIZU`gNTnXo=(#iK@lyav5k^UhVn zU9WJ8iV9Jav3MfW2Ad{6et@{{w4vM&skg(X_o8OAVqyND4q2%-tSex36Q9vdQBO$# z)GgK3+=c&zhCU&{WoX)!^1<__i*5_$;Sg9S}K=vvf_J+del&uf*N&XY+ zIys=?;s*3S*-?LW*ExM157`|5)2{ae*XU&(41pb}2ycW9|NK&jb9fQk$&n%VQw*={ z$4B=CEXj(;R#BvO7s&>w4>{#?v%p;uOfmA8G#i7?`Symn$7%+uDafb9*i9^N6{iBk z2k<6IJHmQy{%EPM$8NY42`DVZ*qz5s;$7l05};lmvN~OWU%JZvZEqpPzMW?n`e^g& zhtB*(pj<82X6exyfO2Gl9Oxy|P##I7>FX0S37vq^b$Gl=qx8);+_z zvB@k%r>0@wJd-1T2Vd(k#)HRSw~EO(i+x{4e^0dBrlf-;txt}|A->Ipk6y1iW{uwd z(`mi;V!}-~u9v~1L_;7m@=-mP8JuF>qsHRgT;&_yH6PL?ujN4Rv~1Si?Sm$vx`U|Z z$KKA{XN2+EB%29PzB5pqi+TC9FDxkpNsDz>XlqD6MdO~|Y>{Wo^#Ck8EJskPJ4kg# zV|3W5UZW)3H!PeIX?DqUT{k|wLy!3lkW0KvkSOAMO+vnC`{ZAwY3SO@1*GMYkoF<|1946 zc;FQuG$P)-QWCIdgcYsG)D^yfdh>0ckEX!I7LCut%A)#?wn!LVQg0NNq|SPzOBZ~UBdQz)wV*hL?kQ83SS}nU)q1HtD1lF z^)B{cGHR26|3DrWa=}{hA&%pgl2j%+G>QomDpI>|S(02a2^UVs>k{&MQ;JOT%TX3v@ONf6BMymN&cBa6$JSf+ zL^?%dbxL~`nZ}gY$3|igmPFDNp}eaGD3i)Z22)2Y%_iw(PRuhfc_b^J`HPGtrGO+e zUvjGSZHrP(2=azlsO6AX{E>A>`~Y)A!NhhZu>#LmZ{xZ|J{Jm{`@Hsi#0Q`qj;wGL zm^>U>-{Y~N>ZJB550NB%hI;w8Q(enhC<4Pyo%q9^YF-MvDt7NmyEfS^sGSY>w9>mh z81M-P=?NB9Dj>73#OWxeieY4oLu~oHLkbz^K|6m`e8l%w<7hQ>1d$|+Wb~?JLl&3A zga1`^_zbQI9gB?1z$?b?SU&BcMg{|)q|Hf#`S^z}!_M0)8JCXyFxLLlU!E^PztdSk z3!Hb>8=IEM!oayCuzuhMs;1$La}=g}xwS6Wwh{zXB*TT0pyAv1m)w3D=gDSu%6a4? zbbPqS^k5p?^zL_aHtfMk!GV};-JGw<`9>QA;nhO`?;giadKo9Vo@n*4tOy5!bo^#6 z+=)J}{3lT~aWR{FsHv%cc{>Lr9%Uc4(*Zc5@i?xMs9=YmcDpi6j_LDMctI)JK+Ua0 zzSgEjc9$fA<>vh5h;6!Wdn$<~2l-XvR zG*OwUbmTplCs%MuO^!Vq5%qPKQr5)My~3mQ{+?9GSm$VQh_PMluFp^?WC{}mvNmybB_nR5CW zxdEo^KYqn3;lffUrGp@6d*s*XXCgN+i=9*7O@M3ic+qc_%@iL)SZo3hM2lj=X7$ak znwZz#h)=VsYUeLL*R>kO9qkrQQz0x=!>na4Di)n?$6kl)beDg&6c^Jf0%LZw_6Zb@ zOn{6R6WbMlIyaJFC`JWdI1(>l67U@qR&#g=J(Me~y~HAk-=yj_RXf~HX04@dc)PQF z9w5-qZB1=rJm8c5+P~2_X+(u;y(@yG;XNG2%n+K8 zJOrx^wf6?AG+VJ*M(?StV}zpKM=@uUO z9+5P6$*dsZjfnURS3Mv8?@><&u_v7Hy@73o249H@c%r)b51Knwi5%>XGiAb+n2BUz zp2*9P`3eLBxv8kQG)Ir=%=&Q{V?gG$OyIJ@-ZQ1o_rdYrm+m&IdU}yYGFX@ym2f8~ z_48>Bax%w8!Y6+dN07#GTMRIY0KMW;B0UfWj}cmd7%o(k_Zkoo+Wzg>e=F#rJ}HC9 zKh2bGbF(>)xm0Fw?24WCSv`{*)6B{X_Shtr-lm2Z;S*5394Sbcu z3qrrrxU`~dsDsoPPHQ^^5x65fWF297H$CExtC?Emq+a3pclNuR5`qNl$xIT_*Z0h;2YzxnZ|oX%{5;5IDd$m7X6+F;o=qm!g&h zyCP=lIQZO44(Omu>4aW-zIM{gK}IME$%TwZ~bI;FoTn`S9*2s1r#|eg8Qp5Rn{24QbNnwaN!blr446YAn)G2 z0oIZo54q9F)O|53ZQtUesZ;+rJ{8Q!<49;ABxA!{+{C0~Qed7kW^3ZIza6afDxZed zIZj!WI?og@D6+2XDIHE;Vx><>P$(roUWW(U zK%&3#fC3~|J6(Y390>n#;+*){{AhRWo#2@0=a>yY0z7n-#HKXIO`+g`D#+{Z#d*nX zYP>s^`}uK)?`$IH==rIeiPKX%LZQSz3-SF#fj*E+@TnRT*Qrk{4Z;W<^;DEQ!Ca@Vw_+j;NWKUe7w zD0=y9{Vj1EE7PrA;L;tHl-RFZPd`x6rqg5cn7GhSUTU}PqfPTP{bHj!JO%dC>)E$> z!gE6pgO=9wrrg@CPU%|}wLKN+^fAjn|DuS3s;h#jYk&?xNelz30-CZVz98wtj0wvY zU{Q8}XILxyVb)O)pkeJY#%#ra*Q!)WIs11-(S>tuy?%QrZq6itYwKeagmzlei3*$k za6-~6Fuu?^TB(&k>PQJF&hA+kjT2S3t@ z{u!-saej{v^~f_7gFCiv#Y(RP<+|$lV2K_v!Y0%>Vs{`H6w_zv*rqOpOx`hCP8?2Y zy(7Q9upBA}F=4(W^A}FOv^=aFV>*pOn3G`e0D~9DMICi-#6SJVfq<1N@uzVI*mh0v zydSgEHqH6nlo2`BT=Yn@A-8$>{r5q2f4USKDyUUi63OGHuupNL&hBu<2|q)_5*+dV z2pwNZ*`P=o?e6^DF%+gdv;bYMMO%*bx^?aDNs}aJk<|~9df&%V?Bl{qs-so2%23>n zyo9zMe~tE8Eb@5rllqR_chHw&bRug(63_A)#5cfJO|FRNoH2q{^qCR-caH2^w96Kz ztXcBwB36hBa}Ya)ecs*SBxVx^%5*<+nAq>T*QTz=K`K>}k5Squr<}lNA}o6KZ6~V$ z>#_=0QKM2{AbQtWstvv9m)4^rO_F+&v_INK2N`f|D$K@-Sg<0tnMM#R$B6jK6WmLP z_I%|=CGy2SGc3YEA#h*ThFhG#&=~!ZDu_bVJ)lwF+7XS5T}lBWzP3+1(%EMkqJCmp zqNThc&42qeBeyu!dffbPbBA2}LQ^bee9D`>`X0IL8;#OAL5@d8&5d{>B3@ukBl-BC zOQL?#;p=5xO^Ri4iBrmjh`(Rm=GY#=96UpxT=7a~-&0*0qb&?_R*#VZaCu3wRl1$9 z4=h?M+d`@L;UuM2K(Z6-rIXTb>I~{CubP=pMP!k{H+7PV9Iq!aiW~;nXQjV!9tN1? zurkr-L#Y%13Hnh|$>tE(K$(OlJkA8){DPcv!M7+}XGvc~mQRhcn*+<-I7R#G?s<#{ zZ>!>xqSu|2oi6_3>xNY0zq1yiPMt&7IEO8wo*}BaUa6Mcogf(H9(x1Y%B@Yzx_mW~ zVl5TISo^5C)c58$N?-Em-N_Qy{Zjl-ZBrtbVQ$jC_l6^Pp_<*`x(q;-tWG8%3}odl z-D3`^a1ux1q~5l{@Fd2gVkX z?qvnX!M~kNt=oG)sH8LgGTxdb=DVKZ{r%Bn9*5$J;^rd$=VVw1D|YY`1_>>v`lyP4 ziC6xQs)R)HzENN#NTBmD{AM@BKeUrGg>#b6ACTz&8KQ|vVZH~uux^Q)=@9pGr8k}N z1MrZ6UKIBflAeyQNuEDkG_f#=gEr6Nr^5+8j!|xS|7;kG8gohDFW39nuijfn&3_e8- z<`d0#TP}{R9bh(gC1H`6!Vfo@IVHUA;AuB{w{o~iWpHa>3(t_K^CI1MwAB*nW4&zj zv}Uqo|KQ0Bo~iP+g)1(%be6FglsNfvEMQ$6>&0(Vy^zf6@K)Nb?z{7#YU2A^%g4qGaiQ*-cwPv){@chNf@%ihfX zGpt@zj9>ikc^)=o46L}nZCYe)=@#COttoQq#W2HQ#dw0N3tb^2$b#_ z*X?rVz+88D%pj}!IpwjSO>lh?+tLbgroEzj2JdbN=|-~4W3yTRN4F~3OvC4;oWevU zk=h;%=j10V4isAf+BKZQMk1@E6bxhon~yu2b!_c5C>fF+a{jsdYH+>x_Ouqe(umq$ zgO2ZrUvR1CO9%#OZFY+Omnl=g+}fu;lSx6<*oWX?bDOa+coJUaV?{JP-7nqm@_BLM zn}Q@o0_-nD@%L>YhMqAUE*P`p-(!m-e03xzXLlO}gtUKEgCrFpK7>2QfOr88)6pUc znkbP|heV-cHF-YuR_W(4UM@OBuAw>dm`^;@)CT<-($$>=GdY#~r*i1d4_+Eb(m||@ zTbP6h1Rla0TA)nJ4~OpP^linQX|g^MW3bXHPp)4{en~;}2o6RtXQ#dLk6`H`hyb&g z6o3|Q6awqedlfc`n=>~ZL2WKuF7lRo>C2&K`ic7FKOY>&D?cClVMZGtCybaEtc>sa z&Ja(93HrE0_}`igT-KOLfGIk^K6{`~DvSLfGhG6|=7_pr0*>O8FRT$g=e*!J;*1C& zrXZ*MM&IRLPa7Y?K|jcQ6<5$-z(fj@o?Pm5LpbwB-jSrM1m zND|8qG${<~rzu#rZ@#{=d8cK1094GA6#rKoDp8(o~yJ zLs0jSQ1uzPR&Oh5Z9pAv9SY0@ecl74Xe#ylJo?$AxgzvRz3l|v z#5^>F#3t!#KK0gf1R%b+Gm<3D9qKo%EdV<_4#D-V2^ZYT+Fa|P@WJF;*V#;po%=dl zPk#_WQ4oaK{oS$)1ryzRt#GqZT;HSL-~coLmW)I!9X!GlnKswEtUy_e7+v43qc%8)GZcHMWXpTqQ8q9 z%f|%wsV>1BbX9vtr9l82Q!61X4$VXQO$mz^y(JsQxGD`twQH9Ub) z=;_VAQCiN2yaJ9vJGNuVRarBxznM?rOzd%MV1GEbc&^Ce2rMbC+a~rqS$V!`s+r^1 zNrWLYt`g^F+3-rbvwTxjKn;id>wCS2b+}K_RF;s44~LBriB2+yPkcR~y)tK@+x+OT zl0+v0cl-(agc7uf!v!H7nF2+A^QL@ZOnEK-B?^Vdajn!fJ=C~B?5$6MxK(Tr)ZX_& zXi6yXgsAO(H%TW>a~0ciABfei!VuW^TM`#P=FSDRlzAs7O$OuXdJ*5oqF;l+nm{#;%`~^tAgejDqVGXOqKu7Ms8srLC<6=*)rwA@&!w>e&a=z*~ z=@!3ix-}poTwh&Wd=>2=Fzs2#9EM(v ze|q5FCix2Hc)O_&JMZ9%N#W00WE^GM>=B?-JH+d8If-hM|4lj4mf@EeV%MEo5bd&T zm17oCV)%Hau?^&4pcBA~hU4tlow%1I=PV`~%pzY!C=q!(YkZYjEYf~LUgNw4v44Vr zRe1$HC~`R2=2BVAD`j;KW;dj`-R(@ukgSOr&}0X2TugHPQhe|7IGvGn*wSEn%|^{P z|H(0QP`i^*M^!qTm}EMBc;FV4rVhsf83|nb3^6=JNg-i10TuS(>8&IH6W)@OV0&HOOrC8-%;+^!9dgHJqs#Pf4s%fWhhzYEXDpVCe!75eYOE(QAVJ9>la$W zIb{u6&gXHe-3Est`QMQ2f;lITn@#f<`1hiqNJbU1@jFlKU3fYMla+|1`AU;x;=ZBf zS?|!{I1$N;aXSm7t7DmA;=Xp>U+lX928@eFFPCmmh#*uJ`Bsy5PYki31i!b>hMF&p z`ZH^+@xeQ^Oc0A2`z7m@4IWd(C;v778vP5(VJd&I0l1bvl-G@5y+J!LdtPGlYaZ_* zGNvc#n%hb4>VA!|M{74)F%N}= zwI43w(g(DTg#iuP85xO*A90HP{{qmWiIOnl`1UNyU@U3aS&(rWFZU}GlbedIEiH(R zaJH9v_70`q(wUA=#7LtG;SXa|M5ep@u`yAO*6U+RcwgoKR_xBjjK&^AksP zroo+EsUHx6-86=-f{?7zR(d4LBT`TkZkbFGUw7wzKLUKG-h{lDzz8*FBpkj2@`Cc0 zy`TOBg2U%8tEfO?KtidboG&AkmWESD7(vhzM;L)6kz$I2K^2vS&)SH7->Cn{&KSR` zUD>exrBEw#`PRt*=$%0*WN`YUGP@FB0n3{>rI@KwZC^aTJTebO zAeOjy?c{K?ErmfSqZvAB03PPTK|c6E>BtJ<1mRF2Sirp}t<(1e|X42;~x(cpY z?(~tZ%9?Y*(sS&{G~z&fgwfa+4DzlC^8&8oDRksQ4+}qNLyy_Lb9ozN4vJh1=|X>#N5 z4Eb|=t>PBgLB>SR@6+j_lH8*K7PAfb-LB*|BBFbE!0Ns3>>hFdf4W;&7?4-l1N7zF z+eYzt0hM6G7Kph>W=~}N5G2}%R&7(yNZsDhfR5f;pdBjmWPVi$kaPT*u$R@edcUbs zpBf>#hGy<4!W5ied&W_Z{)X8DZ?UYj)>w)*k!`?_-;IW)Q^@1R$^dvz7WI4XC0FXGA#>Hr1I z0@MFrwqT60ma5RhQ|6#6$b#DY9JkaQF`u(N>w^+M*y= zn4_8X>8#+Gzi*mxa3tKJT)>PIko@QFh24`x^Pe>hk1NA}6AxT2aAGSeeqXG&iMes& z5fyYX2n2&r;(nHU#6Uq5M`ljm2)r|IoAamhN?YQOZHjlMvroDuYxi9bq$fV=nlTV+ zchfillJ}RaGV$VzX?@=z_XDNCU3Y}n1H(IC^KXxd2i5!KrAjL@ot~HT3F4(2I_>NywKb%r)mlFxS)f zrFEM4nd^2t;$)ns*rg$A1?);b7gV6b|Gwe}~PS z;iUz)KpToc0e3d7n0`p4hU9|UkEDgR+Bbm}gB6wl8<&URDYK3N2KjLAUvTEhkiUoA ziC`JX28aS4a!kD1hPFO z49t!CS_j4?lQe>>Y~*7zHaRPSh8pyn#$U>)TNjo->jig}&Jl*g0G8u&Yqup_*pFE2 zI?Ck^Q)1;Tf8z*F(o(LQ1uSTQII?QWTj{|g85#g~EZ=6W$zv1AGdARuWwN&7p-Iq%go5AOQI_k91sP7h>+$t&HZ@yl8MY>NoUs&4 z58LH!+>P)}N$WtqO46f_yZ`v?e@v0~GyLO?yBg6H-H8ARn?>?6Ys#EIR>75LYF@mp z*E$Y7NG+6Q+|kGCM!xP?tt;6EdMaHNKof@cHNnJnSOTl!hp>?b8rSc7;&O;Ocmy$@ zs`Zy!yEa0d7p&zk{K8Ib-4Lq;PvaT?Ixkx4z;*|r7mndSq=&AJ2L#BCSdfu$nX5x!Chyl z;J1;~w|}T16qmFr`+bs+MT~ zFDVH*m`7R>WZ(>>ia9xL?QV$dNj%jtyRWwITzsSDgN8-*(OZE=!rvdwC5w$^iM!$( zt?y*|t0uQARzaY(m zf}o(=H*ie3DQUJzM)>t(;2( z&p^+4hRbY|m~-BCu@|0uRm-p-Mj2V1?`JhOLI*TRFP`Xn)xx4;aWI6_Nolf<>Y;J= z|G1G0qoP3e_KBL|3f8Yo3$v&9D10Uy$ScdbF4i`LYz;0%iC}2LnFcmTnp7aue79aM*fU zi-0I=eoT0*%Mc>YL)E$4p2lMOx1IfW4nVPBScAEntiX>u8YGU6WLN(!sZZ$>2B%h z^IbWH&O^6nk${IIzaL#6ZNI`9{2O5ZPixNq(?9qW4-B*g@tY@hU=z7qne@kkUSA#| zY(kYADYRT`g6tp4!p!i5uB{QkFOQ72@+|UrBl_OmhC|LaY$1;&!}wBo>%izwJYMi0 zvCaQnkpEei|M~Iw1zdZ@>ztz_C08suH=M|?b-HAxf za8O=&&z6{+I($pu6$}-BoX8>L+K02BjX&qQ=Z^?`P%t!H%m;B%Jkra5&*gt%xc|D( z|MzdxrKpy)knF?T5U+5hA$3-;majkb8o!I+v94Cug!I`Y|MyG(U!y*G3G}^Esg#)cwqDTFtO6>R7p6^d913By-4|%k37%b&_*9-8^9*ee;9E=oE-&1)dk*eUXn+`N!Ka z*NJ4CC`TBzN@s2%t4-zO8hN^Y#c(FmAz8(k~>eQK}Dwx#onurs=KeUz{S@j%V zOnmu;OQ~d}oh|1RE|(b`^Qtu93sSj!Q@`;!bGZhFBvZyriKZmKdmQ>ao1ZWpEG`|2 z;-~B+G!4OD`)Yr9(`-SreBzMk=0cfn8R~Q*Y}bJ&(Ho4(bpgK(b{t8&iEtI2CIO8e zE+6>p^`RnHwKq!4n<79M!9t0A9V;nqlP`%xU1+VsTM+^RVt<`$tv%kEDn{+5#ZNJZ z@GlB-iVB9>zGDWfl1$OYb-|kgyS$g9Kh&(mI*QsY%q-w zzDGHB9hAAd8U_;;JUUuC-l}Trf5FL~V$iw)^q_L7)*wt)8yqey)1|(&Uj+=_Hq4{9 z;-(!lf(&d(Kx7I7n6BXp?T)J9@R^e@YGAo*`~Bwi0Grkyc@3Ttj}2!I3jYThvrfUh zZ<4K&e+N5|GWb(=kki_G-lhPbm%5EF)py7Q%5RY6HwKO1#Y?Fe(V_QUtw-&~3#xJz zu%4m9A2y7?f^{1K#ez+OqCrvuVsD`WicL}YVr%Z_Np?&63w?Ft?Q!rV<0txO?l;_TG&-#w~@#-CLtyOv|Xme0HtobJajxe zJTiNC>*(nCN+!_0m3be{QM(;a2uP0_IEu>sI=%So7NZ_GFWlX&I!pq1K*tL=12bcZLNtkVH!Y`1YMn`p*A^$Ny4^aCmkW>zj!PX^+k`?KGn+O) zqH0}k&b>~EB)V}xyaAtBQDC)NZ&6@2H-mc?@pkoI6;LrN6TF6~e}kvv_Zd#&%LZD~ z#ue{rdLC$)*wrDo8^VPv1+*$|f_M=0zlv@BWXzlSoJ72R*RgG2RvwTAFhc^1`a{~f` zzClXiFDAD-{l24NJDica#Z;4)X3l!+LMM-%w8)nC^Ygpu3d#{07!+5L)sf@@f1X9) zQyEf=rSLSC3Jz0EsZ&b2W9obWJHR}cE6rP)I}n&kayz7BKM`8Q?YqsuD=q8qTE|VlEuJa zqfcgZ;mJn`+HR!8aR+PLT)vA8p_e+YcziA(75pmuVTqL6n~kb4TAmw8*G$~%DJ-1u zLz-@l8--f0F|jqrDSvJBm~V!Gzjfv__~qu<;>I0rUH;7P5AW}jc zI<2$f#)IseqdFqUMhPmLn^QXtJJT}X-z&2-z%Zw2)Kj^y_YfNJJ z6B*ln*H!8a8!Y-l3(*fr1c)c@)n;dn%sIaS5=}1lZ>v0nDipY(_RQ;7km1 zlZnQQra!W@o!Ve*UP%i|@3G{lD_yXf%7S9}(|g-etPfihv|#YWrnoa!$91Ls;^PU4 z7`Br$kSBN+Qc1bd8W_7xD2{2hM!7z5w+!wg+%8s2SDRgB^)_U~&u z9jEMjZ)dVj_+|)r8=9*A)Enw|q*M)aJla&22PH3#a`k>>XWmL`*;V}_C5CPP7~#6R zNz|@4)3xe?2>jr!cMywf$7cHu>-+kk#S5rZBfs#n$mt~Sh{ z6^HdbQvO+FxoG&4PF8U?GkL=|0ghJDF^!R-YNNL%)W9$Y-`4lzZ7728*a!6Qt)e_+ zHK$@KT*BMHxe-7HTC6KfzAlhWfVCZJYU4%ZD(e) zi;sm_4{G$qtkft~WWG{{Wcoo<2tMIs6ctZzW>i!7jV;fm`y%td!5&mm6R=3YG%UR2 z#2I=*UbGc4VNg)5%WS7R7i+IH&L|nK3p6VPWcl$P&VP_ItU==kYV?xBa8D< zX^Q85$Rk=`+$E3(=&GL;)4p_bAX?LXlGx%;z4^*m;u>~~JDraalI_fEHkl^z^x9q# zleB-Na##s0bzD+xY728a72UzoJHF$&bRSw>*5VXC9O{C}nZHCh2(91H`6TL9VH#4!r zzX`h15iH_(bI$5{JxNGyR1KMLI)&Z zUPOm4`9Bs&di8;6-*W83y zA<=NdV}_w2$w-U4Ee}D7uzH3AWMNoP&Y<4%9R?-|NlWZopLJxHp_yTD|KQe#eMtcy zzUT&Tj^b73rQhX;>C26VjYJTxY5l@#y)|W||3_O8yq4N3cDRX;Q6CJOVb53X7Qfd6 zk9aIFFv0C-x&1m&X!v{&6ta5XJ*c-)@N26jp(n$20=>PX|`v6 zDSI#kZsQ`mtE|3!=aq{4Q^E9zZo|)}$*)|71M-C{?`!hL79Pd*GE6H!<@U@{n|7(- z1*=_UtKtTdQxI;bCl9TJ2p8#qDLK>@BV7Af_@c)0`T3hrA8&HnIew-QYDI=ZereA< zX_JO?F@8z#D#^9?eZ)^VJcTLl>>ti$WM5SqX997%FGlo-f6&tx)ePMqZM^L zwelNoOW^UjLWXQ}ua^!4Xg|*{l@@kOP8ej8PtnDVg~+^6=XxQ)17;$pjmHmr=vXKX z(Fth?>Jb3Wfrv3lSD$3g?eucKR+RjDhq-E37#55FAx33?YD(J^8K%*SYU*P?CA%p9?S=aI*d!~N=svpr znb*&p%}ZHTo?`KcV(;@0!ShNNPc@c;+B0Z(6S|~$|1bceNW&iYDyhTqC#yWnv z{QOA^576(7CB=c|jLDGuD)e=1Q)LNwbYu%z)Ws%|9QklwZeH!yq+8WwcvekoAG?1F ziW*0js#&@MW*PrfaI%}1rHZH_3h$~B9bV#=$KIrxe0QEy&;gZ!anZp`vOT}tpSxq4 znVZeo0LG1(_7Ico?j!g=~Wri73xZIi1_ztUI14`z6G%CgkpRu?eUXnM*E-`Z zf0a`VmSp@vdN7d+R|f;~OyF%xh&4F|AHhGM;+@k%z(2y#k$!8dq>qdBLDNUs!HCUl z3yPN7RA!p8s!ghx;+f;8T9YGu0$oCoT{WfvfGs#+`8Sw4Ka93!^h$6+ClWGrHwIH&<_=hc9-*f0PPe!|w`N$6d03=VszjZZcr_)H93q6B(3uF(f+^CiUNE5kez+V}H_wQTVdjzJqe zEH3U4-1{P+spDMusc$u~z(-CRYpfq9VgGq!A9;TL$T%^aXF91E2ibH|C7kk1K&l`7 ze22MCLNA!z$VZA`VnjY4nD!vq{3`S|5U3?ly#HtY3wSvG2_NyA~ccipt$07!|!s~r*y zE;|Gqc80G_)~#`ev?6hnZg^V+VWP`J@>RwtFAQ1m)FdqAHEBa_ekX&DBDMF<3W=@4 zHb5`2{_7FwH1{&~i>ESrU!H`~m$CZvpZaetBB>JVtm`>lf}tBr_*Pw3ByB`GNrDOX z>xhTdMUmF{ex@W??tgD zTEHEOzVuR}&^wopZ}#N*&Q^WSu)!FrgNq*2_Vz9bXmCOCEnPp2T{uU~Ete+t-Drwt zyGKvwydO|7Q0Md(^>JMsKTeeYbqDigh6_$(5Jc;^?|BEpJK#3$vgS;Mh7!qN!1

-NwAPS@4M zb-9mpE;|`n6#2(wv;KeEOrtDLYmI>L6j3yXYMh@e!NQIUS zhZq}Oa# zlx6;1;ifY%dU52r6rC7-CSyU3+^&_;l1P_M7bb9Q(yf2s>3)^#kvkSnkE#;qnPt)} z1;8K~3}cub+;%0PrHPNSg^|Po>8aZ0fh}Meg;`+|f_SK27$YrKm=L=C8Q6WV7J}65 zqGiD4upGDgo>;~U9W#9LQ*JwSC6lYJ-=U?sX?tY`rOMLOgvOVVubCM0+J< z_b4+BC^URH&SsQa=Fo7Nw5=YCINm5BIZRP<}Zp_}mCq zqvwnt2b0q1GtHW>NAx=4)*&z;RhNGFNf9Wo8&~QBt=e>`_&3<{>NWJJWj;YFb%uq_ zxaK4{>^%AJy)=CW)Q<~W*tRiFnxofWRJ&-{3|>cHA+t2NerOL@AB<4 z)$c4t`BE2M@C3L*BAB$D_fB?B?YVQDx7{@A;Zf36pHc)c?Pgq%T0EXHGnDa%lpHB| z4C*;L)OL3zRIt%qMM37I-fhG^P&IKUT>~Sn5a|p>Yr+i-0V30 zKTUOe*THsbHSyA~&g_+}q;vf0y61J_CM@WFXfrtX8%z52>Cy09P%pQg#r59d4%Sb$ zp+okYB2KxY^J03?>C@nFp?Htd3YlTPQ_NEDy&(K}e!|4X6zZd0qkebp_MM^R;DvO=Pu&b$rx|3mGdoq>;N7{Il^A4PZ+5y&~unw?CZO@Fp8dCs+4ged@PF&mR}-fQgLlr0$(-ISnn%D}}0HbZXMLV@Wc92%Ef zsVEa(p*XoGY;m1cXp$f5!6(*>o_ce1Ci2`~&52&80YyD>THs+XR-m4@Y+Fvv2j_ai zV)n0y7U1G<1Y;^`4p~C3uRjH&kZUS0&>x+NfzioRPmzFqPWZg!TwW-?S$N95A)TV9 z9I_GE)`v>Ks@7$%Ba>3d=APBFwDTcZ!2hr-~dwpcd&eJj2^AXgl0**FXQeWqP)UB)b?4jJN4D5Gn@i?mC4F>H2zP>;A{EZC+W zeZUDY{MK|Zbk{3~_3X|Se0wK^E@mpiRNR*8FnLGTg?f4A-E z+ZEF{D3Q`$6|shqm={;NY)^Gsm^5@Bj}=4V1fG>C@Z z6sP-wIe0g+)ghxBu5Pj$;9$bo-0CnY`Y@VF=GNXzL|TX}Q75{8z^ZQ=KWuBTWDriU z*?$>N){&B5*{Rad+FBXa2ek~t+osX1XfVREK4v0)#ht~FT7aEc2O22sXFDdh!1vVL zmp&h5YkSVp%c=dnjko^Hdal%AY5=~TIF2(s+2pyW8cNDD+?D8d^C1Jkg1YV^=*j9p z7_?&C(-o?^?=PBiqjTi626LG{Vvm^?LAh_z5?;A;$%VVj5HJ$U7_J7Fv$!SH6~wwecyZxk3OW9D zy@na)hhfc{hQ7`>kx|oWd}ocSld4YaEXxyULn?b_&t=_}lshz%&AT>Ap|&y(mbHfF z8{&eO247x|?ryc*(J`H`;Oig9=?@P8@N&Eo&Ngtk`7^{;?_UP%&99d3#7FA_LgT3lBM3y8iI2JpGv_qRT51tFxxoj7en-49eP9gusF$5bRSlWwGtI^%??#50#c}5YPbb%jY1Heg+HzU9NWhkW$PudTg;ZBQ? z&~h;|VajIF>Yi?fGb^d%Km#uhh*ss_WZY%Rw{6n8spe2KWCJM@33)o1hY2-g&5c_> zJ~m^+ScGu)IwHdn!koo`j@!iYd$N)eF~;w>QO|=7 z(MEK8jBWUtSbc?>m>Ru1Yp*rqvseybCoS>0M@U8yV!iPJmmF?LZCl=1`nYJ_YtCOP z%afOchS1KPjk1danNf^uB;;34qBi6z<^NC|h83K8N>mdg51GZCeX3IO9J%t1h>zFV z$T8XFt+eJjCpTEeS8RSgCZWzSWlj2>S5cbUy=j>I)$eg|@QVPxd8JZ9Y+W}UOmT4Y z_vdu)jAb@sINXVB83N>v+_42M%5r~vQ}4e3X40y#+|czmv`$r4bmCvr${_VDwM|6Q zx#33P`?RNZfZXI*{+q^qA$xMapKx7Gn-`HN=Z^DyD6a!iY^-O(8QFTygup8dMa2)? zl@6^ZWW_X<+ZTxN964a-kaN&L)UTX(W&p7^VJ8nb?!X zo;EX>Wx@&;{VmgQD|(4*4UW-Z7;M_l_#H1xo-X9EVEcxc1^BNS9e zVTN`pQRmWajsg69`a|I*?9-Z*=apjFUv|E4PG3%eA1*>}j07 z-PHS~Pw+#TOpX+<`Z%LTRo?PK#nc%A2>X~@g-=a5PJz!zcNC(VcD!cR8=FQ<9b7D} zyvLpYf!OyPnNvf|$;WrjT5>$@PbA334}4Gk;g&g@DugeNI zo56kwmG9cLo?p(B0g8H0We81TKtywmp+gVRE)tAQr2H=23JEgI*=R?tjb?gx5mx}( z(@Ow1q$uVH3`LCUI9-^cY$NkyKQ&k}) z8fR=GDh7s*jR!bsJ)=b0jbYfRLSueGrj&xJF@6nX?Js2LG^QTyKX8Wi-$M_p@~OhF z!@E(;!#w5Sl0*V**%(xgRt^TWj)2VkgoWS7{7}xs&FjUV8y@AWu+V3P(C>6T&K!HLNkOA(BhNJ8+p6CrJ9aE%0LZK(o>ZC zIhWLRz6IpK(%{H^9O5u=f!y_(+3-@F*^wX7yNmSXzb-(d4l|^L*9!QSi!#{=oDapN zoK*FZ?b6q@J*IhQ$+G@Qbsl(&Zx<+S^>Ro@pXF><@#wz`t+EnzyY7!TLpg=DD>Fme zZj4IPNuy9KD1>G+g0_v`GVmkiJ{f=xL<->~e-Dx#R9y}8G$C=y=0M*xpX0AAyVf3> zZNymR*e~t|3^itpz_&~5kihSC|AtbqZ*OL7%!D3w56}tGF-yLtbZ_n+U3#3*ZPtNv z-_k;0EwmXZ71%Xll8da*6Fk2O1ng5;9j$@>zV znkFcI2i)5tZm^|B!#LpYBWW<{xdV{lPZLVly>}P!iHFkfYsMBGoV^X>mUI>f$P}1Z z=r#f?&y+xyg*wRx5Xp1DV*MKQ%VKra)@MzKmS^tm!qGabbUql$*@>qJCU0yuoLVx3 zmI*6At`ZKa)&*%Eucw(w)riZ9H@sM=S#<3l)hFJ1DjJB{E_?_qQd!Wk4MmKR$G4hH zy`x2QNE!aM)R_Dp2)x2!eBo;J-6-}1VQc1DJ zCQo#%Ao$T4W;hCpO{Y};0`9t0G8#rRv{N84lc-Q2xCsF|4bH+KG)VYhpkWXR)_ill zB4%b8658_+`%s9WP-y{gQBGqHnjRFPvL6)AC(NoW++??)&-cL4M$*k50*Rgx4eQ;y zTXZ3Z{-F+$j91l=^>A$t=-!|anghI$CRHfjf$r2g(iw1BPWge98Xbs>ews$>oJE(Y& zZEM|fwr0A98~o$sMo9_86CPqZox4=E9R&4g!|6YtCz4xkOB7XQ6&xqRPqKQIC-=+< zsf+u`4lTp<4aoU@whByyHAOz@h!XOQwjXr9Pr^$tngM12Mj;oxFcNYHfXQb^8yV2+ zB&xqe$L@JhvLf0)-ft1$gbsNJL6RG_vawxXuzZEARfTIG3DZbG9b9rgdNlpPY1S^? z1P`ql0!^4=nr_bun%LPH+f=cx&_G>S!*4Ua^xvziAIK)EjDN>pt=os^r~L!lO(;3y zp98uebiLv2oq#88R=Q%>pU*9=96@v7K3W%) zNwbSre4km0ILTl1x^p8qm?|~R+HJ}Q{qM?r#q!;XGVWR6B6W3h=u2)}c=wzOI?g=!np#AfSM)?dQms8-Ru;5T z2ZfcLkJps%n!!Y(86P_@Y2L)E42S#|JJRFnA}a4L(mHnAh4I)TIRl1Ck#Vprf?!J8 zm+P__apsPrhJgA{UQEMc9!M$CsA?st+6&v zk$yYw&dHhqewUA|#)hOGc|1IaHil2UyFkX)Ge$x647N_1LC@0pv#b%MoobSJgo!XY z=~x$#&D2;(NJUaF2+p8yghaD?S1UB2(7he#4dM(NAicYBV< z;{q#_pxYEruCL?r7^Rp{Tg)c@mK@})F?VYo1#Wj54A6=?=LmeBrrOAROy?4oX}@mu zSE}X$#)$-Z1}w*RI^Zh>e_%QZg&iRdR4#Xrrt~HRx^XC)9IO%}P2weEi5K=~nSfUB zY=3}#FwTmHT=iH!ikDh#z}-VPSDc8tPIti%$S<|)Hl}v>(g!?4%y<7GN3C3aN%~0% zo5p#^fn<7+O`rC~qkfEULhC9Tg}%}4l>`b=cZfq89U0O38R^I^IsM}}2AYQZ*Gi(O zKZ7kh5;}W%pbT87zlsLJlJvk4j`>%fJklxWSodB*%7nq-ukUp!U!fM4Qgq%qM8g=1 z=@||_n5>~zwX6CHgO(%SnP{@dSryW|K?>uZ2u;FO@Uf6p)UH$~@M<1+Xz3LX+%CPj z3KYIkI5hTC&2lJu-n7+b!G$M^M(wVrD`j^2_*Ej$9jtFa)E@KVR+;=V%P{9UFkQSO77Q>yIUj{1z?NQ zVom)OVkozQzeua~;rrQtahw(5!5_ezCH3WBLNf(9y2FSC zcAtM1MJ&Eok1V$RH#WqOCfh>@xdxb*-}C*mPV?S3|GGFQQRFJfm>?Y8eW!}M*mcZT z%n)|Q{GHtxSGzx!A4Oo>Ix#sJi<#|{E%r$`T6IuF+}<^|xIUb1$cbD-kOr7BVp@QW znmBcw@VHCn5!c6J%Cgp`uc5qbiKW>&BG~e?ruTfSOa9VnGkm%-f|}ZX)YU9$MK3df z3L|)S!z&uRlV2NxGaUr_quX+zFf)cZ;ni{jI^PQu@di~@(i9K{IEbF!N#M>p}5u*xr4pj4n<5wkPq=k;J52?elwE`ZS*4SOtH~w^Fj2e~2 z4d_mPIIvwO&5@uU$KayCxq|3VtZXcA9^1`t?PxlCcs|obm`fh_bL|hSxI;j>GPHxM z=4=K2$WBwk)kk-yu_`g)7(}OkhTr^Ut6xySFnXdHp?B>4xgSZw7mao7&=*VNGP{Uu z4cbINcRg6cUOocAC@0h-q-F`5V1juXUy?C)k-Ix0S@0MC;{^~!!Dn|iIZE$1?>SRx zdAaX7!}`hnj3FUtrL9DDd+{>yM4YMiGonND|*L=xOkG(N4?h&JZ zX9{QR<@@a1-SHs_o_nD-G^!zJ&YFHAx##}<^dK*{=YwZV4}PsA&8gr?T=>8gyVUw< zqVE}alcjdFXxdA^kbQC3jG9>c6bNwV@XB4mlLbXY6xJ3eSr`l+6Y&IC*G4QK5 zJ7@6aCT6BXoZ~-r{zKOq0`S#PDlh=(~~h# zc1eNu*R}eKLK<1S5-;u7HLZ!p$Wm$%rT9(!W@3EEr!PL;6xah3lc`V|@bR$(lx)oEz8SD>?hR;3p7q@MPoT5V{L@B#Ek@=4roqPnumWBAxCHoj-Z#Rptt6+!fC zX>G>^`#5ue%n!IQvO4XBBy}Czy{;DCMf>@eumzp?PLF(faL!hkCKP4SVjR*~Q+V?K z6~mOG{yjJFJZmMYFqEG6d8>OO$-F}>@i=jNGQ{__ZBraA$CIPv(OU$W7YGS+R$IEP zlwA_1@d2&kO0Q&jqCuZN;NYlS9k!+gt+UBmf!^Xm(53RJSUaL3xMm*Rtm$v( zE{S0bJBFnI9aOB%0rnOq`YF+;3gn*O-!2icr$I;~2S&wF?teOn>;_AHADY+bciLS~ zK#smTj6rJB92LXs~g**p?+261pv7CD$ zdU7RaO22(>GRpWPVB7CNK#usjk;dGL$vR!sE8v@elH2g*{kT=qvsLPl# zGXJHq1)hH&hkBFi$9_}-(e6P~rDhQyy0S<SwX=H_2uTN6u?IYlOlE|zknV_He{PfW9m_3L~dZ~1~z3J0ao zJhq2;&iy>Ro-y7(^ZBgk?lsb@tRxyF)QE!p45c1ipOO9 zRTH3UWYeAn>}wenG2KMZZkSUnw$3}Dk1YuqP#_1vh;8skg)bQ-8`3Yf1jlA6i7J>M z*!K>`P0}jG-LW4!!TuIecsqEzsCY?di`#!b$ESuV(!z7^=fQU0>sC5UJ4jXhRKO%WNx&NTjWJNH8FQB)#~3haZJXw5bY+WA^5;CB;#iQ??a*R>wsC8S?!Cc-q(_x zhix?c99h`mb3|HrC1H<&VvIzNvZI3ZaHn;PigzYmVo&lEgg zI|S9j8i?mfV~)K^_Be-i>TQ*Rb=IVj6KW`MDH6fKlD_qI>ZaqfE&dqG3sV>`Z z{kX-X!z7rZWBN8}9@S>#gz!F1Upkh%hY~Gj)X>RW9X0Gwr@8f!oL7CGpG7>aq!MQX z*H)qKz~4eUt!}*p^DX+&IV}^UlPu;xq3N`d5oGl-`lC604-mLKLdbs?mEJ|>1h59I zO={NR=~1kb4djAnfw-7y1z{s2e2r^v>Yx~g>U~u5iTd4Rw1aq$#LY0P=l+AT>R8Q# zOrm-)AdtVcvs^YeG{WcRPB!QU-o#Q?vFI-0nyRHj`OL1-v-9_*3G@kAVriKw}T zaHPz?lze|EB*4BwaadFHifI=262cc6ETCQ`qh#`DzcE5Z_#GhZUPEBq#3H|~Q44@n z7S;|O%x*-$RLOX)(59;2;~-4NwSNSWz~AU#7e(#rqfck(5Qr#DhFJ_6>R=GuDj*YL zjIH`g9%DA&;LbaM@eG<}C(3m_&M8$mkuN#C#qwG1pesk{ClCaSV2#MRiee&ieFs93|2p@%k)JIflX?;~pIkXPdX`oe&+8U#ffAJ+NuCO-mhcAU4 zJYYj~0=gB<2p@)!K*;gr;)4t)(7XP3 z&Dh#Wt{B+u-&6ZMb?VMMN=*8!9!T2c>&Ew3XQs)7dSi&9w}wwYVtU$!Fdu*>xMM{z z&sE%M68}#;|3_-8Oo!lhMam01{Isp+nNeyth}tTV)IJVQ#)K1QmXHtXG_J{Nziz8d z$_q~Je%ra%AMmstrCj!aRWKzGZ=Ddyzge@HAFg-RD`mC`m96@FDV0ef9nT-?993!% z#fDC8#y+zLO6}d3KE*e#@ItJ*F~1#Ybp|3a>cyatQqc0Z88Iupj4ZFBfsh5;(*QXM3D4)H;^G zqVLAF^0rKG0bCZtr3%}lH^2`Lt%oXh$4$lnY1$TYSrMh$u~LoXbz90v7y83MbEnU4 z7*?!(PnYq(;72fVU^iOMfNw8JyFld!dp#EnVJPXih4)2ja1`xzm_WKK-s` z2y$-U!XX0JkZqGyn7RZ+5q!ClYTSo$;&L<(rACk1i{zpR{^CPFNlxujL#{jWK1L1- z0uu7jK7?awhEt@ElPVZ%d{0`{zi63YnU$+L0W8Pvx%pPxWF87%mj~1@=qDRf4AxY7 ze#VBaw&8(d6_Y0Rm!#726atrFlyhPxlklD$PtyOL6wtr`+|e?`@m^ZBmRU`gUMSJ< zx83~wq%z=g$Xy8SxW{V25|FfSPrmf1Q9INLcnd1!FEdmhvULhxG+5``z6%|4#l6eXUJ-~#*- z$9M=1SBBx|#+cYb+?iB`K)8WNHea7HxsL|d!#HiL?a=Q8TI2_RP+F2nCTB2^4b(|~g@DwpzJ=4$ z=pn&Dk0-CdSp2X&PNS7}CoWZeS80SUX3h^!=1mL_ck<~c^OAug+Bn+~HTmKyWwgnW|G|$zA8U z_1cjA2Z}`oxaiy3RPfTRgF%rom2t@BLI=~|9ISV1u@zLVd^*liQFr+!l^6a-U2p@b(U-!eom==GVCv{0Y>D0ENfF#r#{T`C*E?~J~hgXj* zKtUSOQ@-rqdIi{tSYrV$+Gf``BEcN%Cwf?u3F1B&)@=|uu6&WYDl->Ofcrq;+Hu}E zkxzFpw*BKAwpC`z13}{J9$@6{7`alKI=F5CPkE?_(mO#{g zg;(vkI%D)mXJO!ydorB@!lpX}O^|zUJYDql%sVv%r6t)z?_)e29Ns*$<65iUUyohA zE!#|Zc&)FAnG&L*BzoGSw8cQNo)zdIn*4AgtZ1Whv#gc){FzWh9~u&Xkb5U4t)a<# zZI&_FZg^XhsAz*~v~hVTJgabZ{*4n9BJ#OHTjsUa0OfHyXb*Y@O?#k5JXB4z(au)@q zR8}v*>@+)nGesoJ(}--Y1L)JECQqrzhT37|Addm3J8e_0)P9G8(dnsaMByv?m@c16 z^BZkM^FqZ8&hSyqt|wCs89(XrUUvwU^xMl&&BPZKG$!F%|4v}~J{P~mkXiZQA)u{8 zTritu7;42e4kAWz4uqq<_>)a&FnU{%K6|pjpl?BUzoahsg>B0cPxJ}6DdjMTOQvIF zQ5BQYMc&l$V<0VQovWz|>EzzO^*K@#PxlN%SRJEtO0l4=_Byq3|6}uA^gt6a^n#mx zxJ(2!Kp=TrYL!VcD8_VoSHo?;802X%_-}j;}!utrTUv1 z))!^|b}kHBeM`7o9C|)I)@ycR{N5El2(T;Ln{L=b93;G8Nh(FYGI9rNr^H$2w+l2>JBf0@XCLT4!GcG(|55;?w55 z{Lp$Kl8=S~ZodfYJ&`f|s>?Y{H}u9ue9E6!9iFT%(I=mb)p;B*-)z8Y+hpSNJnc?b zH*}gf69!F#a%`hf@~EE_P&D0wWVIYP#vc_Z^{{ZQ)jrYorPD@J*LW9lL-SO{RzODGRHB$X=_xP%POz+d z5*^;XIkKEPH?jfm@;!bb6>U>5N3!%JGoyCvar@xFm6+|E89d{}H*tAFe|# zskxq61B}-5;9Q+(qyMop#e^qMZXeCO=a6*hx~Tt|Fw}5{Cf1IXo1_9G_K# zt}`9`O#P^!S2F3Aksj^covn`~IMMf^dG_e-Z zvx2A387-c##z5HdPg`2yJS=$J(9p&hJoSs!baJ75Qlw(UCYp;Sp~^g}xh}u(mH5KEvz5A|`GYeP zvgx5F@G$ppy#M#bxQ+3HKmVpQ5~ig;pQfLEqDJunypQ}9f9+equZeCe)`WDnHthD& ziKeu*o*Qp32mMpAD7M#&tO9Wr?iJr-2L|PIYUB6dvGhcIA+0pZS;(KLME3_sjkY^l zLnMp#40-i`6o!nJ*Z}deJC)eIsGr>dJGh^uQ(+^uM)$6E(o!q$q&GgM2FCWdZL6<1m|7*5)*87`L)>s`B?IX&@rKR%75S`&C4tYP{N&@{jqvAy%khZUV= zDHVK<`CTx7fiYyJXY?1Sck0*vps)y3D#4ukaMSnbQL}{;f3=#xX)?4hs=apg?R!eq z*NOPxTXhKqPkDlSZ27{);|dfHga32}|Cc=uAre#=QR2%(Ahx@^Ez;iRg;Xv46*>F> zfHJ!;?sQbEWvJ35^LZ96vF9o#l?OD^GjUGAm4FuIo~S94@^0n;D8$WYvYwoTRy_Kg z92xZHWR(V^e;jnsH`)xA5n^+h(}O#hfXBi9@D!e~v(Knrqz=bNQv&v1>b1Z6g1OlA z_pLN8ncq)=>GoOV5d5exaVLLaRQ8fLw-|w+R#ZY^m{ZX=eNSS)k$7JvVLO!3S;X7P zA+jw0yQKSHf96*t*eGTVD`k$QjP6+rWb|r#7YxS@2v!4P!^6q0Qox9Q*x?gP{g&Q% zjF<4|Eu+yR>9zAVR^P7Lner2&Z?K*Oz4oEwwH-cqQxnZMK#qFtFNi7#Fsw{qw5-{H3WIbf9uP?=@~bnI|rvc26oEeKvIR|K~Ck4eP(Vi;Wn+fPaiN z)P0bUpnCGfUcJ!^mGp2B5^aafD>HyUjEtJa5DqQDo4VRn%ddZPM~T`n+N?}8AL3~` ztoJW2GVj$f*TL`l$Qgj=bi*RfP;TiOVJc)U94BNzb;V?=Xrb}E!o~QU(D{b;h-W_x z7Pg%!&zF-b{Ls)F1-wciq9V9nCQv9<+E8x6tRIUCyK;H7e^CdbQTbjZz<{;j!Iso&TJqUB)+r1=5?cJm}n{iwfYKt;lmI zJS_8C8(Nq|N+3?gkG(x1|2USxe}!2Zfyz!^ykhVjXB^RQ4axVo&%>mxvG%ntEO#f1 zWUUwK`~cED(^th-_bBIt0(iceVLNOyV4`CX&Rm3j@^l7Zb;welV>n({{PB9R{M=(A zHYFx~ZuWT$15J=!?`elD_bpj3V5X#Y`ubR{q(ClM1*%2P&cg7YNO6n{$t~R5s!Q)L zIOWHM%eDI{dwxFPrMK7JLr(x~j9>>{73RGCX}|Q3wDMITReu2m7*3%FE0GgI5T}Om90hT&Zy_?^inZzEfrCeXn8yIYvtYK>pbQ|^fS%W&jIhS7M!zXS(;}4 zc$2$>9M!X#e(c2t5*(`)#`O7p&HV)eQqzpT{UqRy1Q79Xz+LaR`rk8zfR4bw=M+J} z7lhO_t@T$2*;}mbU$G)_E zKMoM<{b#+g{jU%%{7m{bR~UBNZ0a{xaxA)cm+Bqwh2;d|)P=KRS$V)OQbWa8<*G_E z<*INshJ+1&SbcB3+a}})@zb0wxET9W=-np-z19g`vL4Xa3lyz3E1oPa1BAU#0GV}< zSK#l=Z-2aFr?zv{XWw=M`>I-7DA2!k?<-Eo_v;Oj=jdwL?~7(x403<^T~Bi5*w|_0 zgJ15*YY+;U!m~G-xUfr72YbeWR}Ba}pyrObF8%%2n@K7ROuCe};$0Tm<^>N_lpWD~ zacFZZWHlE8)>aabIG-7aGoyA@(t|(_39xocZ-zfSM&V~+^@;~wH6NZ6OYv}>)44XfP(@__^ z#Yx%ubh@+eNBk=SXDEyy&VrND-kA3+>57A^E=w(Twvt;Hou7Psl-sX+Qhqh}qL%>a zDxHIDvI$EDefI-3Il&v5h?Xp~2Ogk!Gu0>IB-#$H60sv9k2S{6LxDJF7Z(PuiuvHI z&LU$=(k52BFyd0`SBXkY-Z1ZP`XERLU155PX!zEtg%mKBx{2!s&oh_FnAN$k@54p zL59n|?NOB^)^ZDl>k%bIW`QlKf@C?`2eiZOa%S0h-n}o&o|e9kKtn3C@cy#&+f9Ru zDWbO~VbAf?w3f=dJ6 z&(E_>1I?an(Cc8g;`d7$Cc>A{g+Nx1O{eoP>pZqodcXm7>EL8KTZ(+6LL{aL>Ss_%6 zA<24$R~&{$QlC!^V2*8f5euB!yXWg$JX;1z{!_|l-z4MH`gh^lB zFk87A9-=}j?<3C0yOA@)<4N2=;hg@zIii1YBoJuGHSZ;yqeu1tEP&k#ZWJzN13S&m z_-gHcKETG&ApZV-^AR@6l1y?z`eIkWwcT~i+8_S%;b)!2KxVKL1?aavyEz{X?SPtu z0t&ofh3`hocD~`rzE30B|Sv?61do1Tc#{4s1VC`}#^JskivzZ`K{bJO)+dz!3R}L-#yOB2TRJ(f@ccB17e^dDwMz??sEb z^r}`k_)@6(Xh>3HyDq79$-Y_Iq>4KuD?s}s%S3-W{j1)0p`yW$+jqsd*%)WxpJrJYR1Sa=9 z)9E~m7`r-)8+)_aih1v6=(zr)(8ZAVUb0$_Z~8@z_QZm|>e>ByZ$EB<`w5GNZJkx`_cbn>#xGv>c1{rxVXC&x8hJJ z5-1R)xNC8D2~gafLV;q%HMkcG?o!;{-QC@Gp8vb|(f6I^B-gdpPv#oqo@1ib5FZD{ z2x;753GMMk_+K~%(PB#H5m=zH`D_3TH!4=A1YQhr$hf6n2<&kPUFaX)?9epHY< zBLNLz9JH}h0{+N9D}sn6Py2tLkrM~lMwL6=nadJ@9#Yp}cNZ95ZuG|!=J7wRf^o0P zSBnJS{wMW7A0kzwC)!fArGFlh$^Mxy{z`e%uopAt=bX!>(88eV`4451l8O5JvjeK` zD$vkMBIKRV&(W}2{Pa6J(~SR-D6#8bv@Fp@H;ha0v|w$XbQ8KL<4!kftNy(B$oif5 ztqCb6{l^JeSGGzoUJJ+jyqavM%$zYpjMcI{gQnKgz zLpi?sSnRD6CJdB;7oHAPM~faTW>950lrVswfm>ow6niD037~Yz(Qn4fT-7F-%gF^3 zDElKgIqjItWsRD$vFYNv|L`-A^jv4JNDy@X6Se$30`6G(ux{gy#Bqhua51{*T%!oi+Y42b~Z%{gSypK2q|8hg~qc$x0!CZ8j1Tq|brb++x?`Z^dn*iut^! zZ{qN(V>&~hO3Y{Sl4oKDRTGE@YK)|}p6|(uu}1Q8HcV!G9Cg|PwD8W8Sg*`|RoKYf zMCrRQ_$(`DzWBQWz8}0*ETKkg6vX!;3iB{NL3{B+iArUj^Y;Im22s6ZSh0hlY>9^}FODL?%xM$v~|3D}1K{u6mid<$mSW+vzA*+FPpp zS4KJSp))@mIipl)XlL8r|26x! z=J{nQ;b4YgRz;}g_KT(k;Fsk#-9A{M&0x=RyPQOjG~H%3-KNda;r|}H=B?*}sFxCw zfz&!rY(?7(te5AIeA*Z9cHn-~v_baz+?#}M!+Bv#*lnJGl(5pUCdEuT8Q_AXx$ZrY z`^UfdnFe0i2)MKJS*Hqi0XyfJT2mMm>ods4M-+rl+(O;YEzMZ72i zU{BPxtC6;w{!oRCeKm_ZsgARwBl~ozR9#%4=ewPY_W1Qd@BDfZ*1nJ3`qgx@gNl|G zBL)Res0NospCsm?9Pe#Yh|v2JlGoPHlAB#5ykgg33kygruxt*LO=RF5!^tTZj{4kZ z^KxmSU*`1C%wN&9uD$5@;E4gv(iC*%-!}8wYF}ogZckul-dwb~CSuF`!rmqZpbxQ7 zAeWPJ<(tVu7&t5#_AB6B2g|kGeHwgn=FbwZ6!NA7UY(ecKrKm^ZSt7G$45|~$Ftu9 z#CPA|$!5?g?P*O9b5@0AyC$I@CEs z8|;UNa_xKGwzoCQn-?)LTfZ31I{u{R&BwQU?qAoozJPXs9cJ6bLgsAG-;%kuk~z|Q zNwvS;YJc6ogTo9IDYf?E_;&WzPGA~>?oQ;AXhTBahtKbYD+ytrx<6Yc(q?o}d{bg1C^`COa-_F-o` zCb#}?+O#5NJQfNXM$XxOCUtLHzITxmG{zk3cyR2a25hr;GM9N<7kxlAB8u?!v;!l6 zKElgUDEa0J?A&_y2%|h0SRfccONX)=m4;Piuo@eZgFj< zBPa;g`YHtnzrl{H0J9|(dtQ9h9fxsG6oyoYbWVjrK=!8P-k5~*Konh2FVfj!Uu)j&r5oXyROu!14?C+A0HLe>WOWTb%LMKAe-^ zOf;5Y1Q|Js6RM;+wC!uBJwKXIPmSW^_Pf|86CmCBOxt#y@@zFuZq!*b0B^c=F|O7e z%l}k<9tQfoUvu9_(#`&NF?@CQdtXw_17}WAjkEtC^||NbzK2MB|Npae>QaO{*TItO zJlWkiZWpYAv4vD|7)?Gqo2SLTEps=cQ-&FT3NH|Q>k%UTbHn=DCE-#$LN#}?{R%QY z`-8=$B$Sh#6^jTu5*5|XX7l1r|9Bamb(wLtW~lU;#_<2f6v%f$t*T@JVWN8xuVisADnY<8E_2T@-5tfVw(c!PjGMfX#6}kWU z3==u0Owjft^^qEKX-$)1cU1D)oH$+rLgl%Pm|r~BjP?(}go}Y`GBqnKbN^QaBp~ai zASR}tjAzP^G2L6pPAX)_=!aJk(qv+U2+#ipgm&`GtkNK^i#iLv{VDp@wyIoa#Lj#4 zf&DByk{wcceG{`K!rGNPA5=7tHicr~OgNDeK3!UqUicRVw$tk|2{_x_U*@ItwFA(8)CO6zK^fD2573dOs0R6K4{}bb1OlWJ&(A- z1c)#Zrv1!??Z#r35J{s6iikiEhlfebRY!n_M}QaiieJO~CjI73pSo{xJ=FfGa{bx& zc`a#iF_r6eBvXJ&XSBv+k$)(aRsiU+-gXp6FGs|tZ|Y`q?{&Wx>~hvk*PIw`>pj)z z$D{EQlYcoaRP+kEf=laqqaUTHkU+wsEOI>eEet{&HP3r_oHrOs=H*@0-2Cj?Vw_nz z1cF<0q(A0;FQEvYAsYM$J7KZbddkl43#gkrm8vs@l)y!MZ-%;FWQh1v5=*+`gMrY>9DCc?)~ z56#ZcTymGUU9KLBW({;E?1ko9VGiA0GJ5h%88`oZDfs&Le6WKmHWe8eCNtIQsmTxr z!6Wat&h5eK_=1~L?7+f(M}9_TmO3e;cGRDZPEth~gQ_(B1OZY5FuT(+&@saizHcn$ z*p1lzzKq|usOB)nD6V?C(f;J1*&cniG0dhlu0xFHEQiFwziir~hW;~Xsr}_;J0Tc_ zKaW+vDXRivp_RvvRlFRPlE3DD!T5T650l4Lv@fujFIsxaakxVp4EtCD3t?0&)AFno)4kk%Fc*@v8A@TpEJ2&*b_#qcu zJWymPRtNkZi#qjED@mz&gjOLr zYfz(6Ue&=82CJ0J4=P-AR!gE?j^@wFnP5K@X$bmR@65Yi8RyurX5O(D`}SGiw?w45 zR$S=s13;!+2#E)00``Sd zbdk)QfJ9m%A_=tpBH%M-p`$BWI=C@XL1)Dc&9?IMdqURXie!`ZmXrlw8c%77JeT0T zx&VP>NE2iFe6ji&rml}Y$7HL|tpliZM`$jnJsy>4L`hEbZv)WXMU~rJt-Ir%+-*nv zOjmq^aT$Yse5XcRM^G;v@%9kZ(q?)ujzf*_J{^ZE?&30 zTR@w6POJ`z0*L98TuRPXx2mKGFZy80eWAvQcAfp#!^gnxPp(YjFDVtQC=})rls(MghTs6~2{#dxCEf33)At~( z&kJTKTmKNB*djMXK7G_0`#eN_q0=L@2YDy6Kgfud> z(OsK2)MiDDXKi3AhvZ&4s)16&l+=Gc=htri=n)GSTp!hEXb3J(9%QoW4d`L+^hY&= zL&)9DpkQisXzbF^8#CW=E~5^-xlW2Fq@Lc1LCm9IEhe_iKczH~Q^-WJgiH!371P4m zE2QTl8-||wnJh>co+as=9o3G`SD~KPGxs8vx@W*5}cu(?Ql}F@I&P0!wZ5D zTcZl#eT7*L8LZUyVz zaep=Rbr(!*-rOr3`euq15E$B(ehNSRxx%7Uh@+D&mGaE;4@7Of1Q2*Qw@&4cO*>HS zdX*MAX?i>r;j2}~IT0n%BJKJD=?(P0RvtL`Rj}33mp?cZM z0qMfP-(6bQV%PZQnIdngv=;^02fXwRDjTbuM9UpMK;~H1qsVBgX}-OsMF)efJL4T zN0x0QkWstw%ez4~0Oa7(o3@UBSn(S7enWlHWx38g!tmQ8Fb$|4e^d>4n)w1(BGN8+ z^T<1Rr=9tTeqR;V?mX^iIkuBZ4v&gIQgkxC2EH%*`-CV%BYc&;wYg##V>wkyco_xEu)65pC@Ix|* zH525lg*2e_h8Imqn_u>eiV8Ud5%!x%m?VFhmgN4RLXii&w8jXy`AHj6d5e=S_U&wV&ut@ZC%ulL>VX@*qp!#;yH~dmZtXU85@-3I??GVy% zms6eAQ??YrG`?3fSG!g$JaRcrd*ypkTvCKRhT2b`K7p~!**3b^g+}3R^>tc|S#G`U z$T9^p<#lZDcU1vC&wdCFM`p1EWxUK7+fcxSd*yc2I+G;P^Swk|cg2-4sdA~4l*>ZR zg3rYVdZnbuD;gx${RXEaysbcoFjW!H6rrs;siCB`{P-#g(fQ{K|N9Zo!4xvQXrr$I)1Sn=A4+odk}<{8&UliOKef3IXeft3NG(4<>9{yMUj>tuaPrRORum9~s~U1>u=Ka=^t zg=($ZtBDfps<``f62Gp+_2ui*B1RNjcl<5*@W!)pKuv=4{SATJ`wUGjK&jq#{WSUn z38A-$t9A~KQQtCN)Zb|$o=kXL>-%N^%t-YhB${ujsxjf2MENVT``mE;$9utC2(j_;fW;TUn zh>K}y;$m3xwr!p9t)tMe{kCrFJ)`#QzvXYy(+>My ztOb>Jl|M5IcKf^TSG+bw*=Gn#ec?G)R@Eb;GiVog)P>H)h>O@!Sfl#6pdPByy;zR8 zvt$U8-G6eOGiuMR*mkE^vJlB5#vNoytv;gln^$y=0*B;DrRZ7wJs#L0z;)F1QyUlI z7W3xc3p~Pad18=oF*N$NY}gB)p0!OWSin`fZYwV4Zdhr?`S%i|%lFH)LsSIWszWAX z15^4k_a~$69Akk*L_s1=POylX89eptq-Youf)#JLnGh_3C(m0!cN{M+-eG>c`Cbz68ZK;e(-^eO=@}bxg{dG2 zF+;Ss5Z+yCLEafk%|s;{QcC`;uOc_t_U!tY2Hok%uB?o0n*4Xa4=(7 z6Cz_8MMAce*N>I91Ucn!$sW5k{IB<^V+sH4X|IT~51vbxMqEA>V2>^V9oHuqc@@FA z;`H^jDZl1mAtaa2cbs^>z{e!x#DVW?qdSlA+Ii-A-2Y;KtKa(Mt2HldXL?@jd48=q; zWibdkxq1AqNh&L*SNuHw8dOp7!|oenVwe*(ay}dkb@nt&TI)BP=@dHWqQao9p;@h` zXRz4KU4nKkmpLTr2unTN7!I8C0wTf)8dWEJZw!@;eh%2$RwEf!Z-<{l7%2Km8C9V9 z#vAU(%?i~vD_M-WQq!VZRKZy>s>xF6%QSAcHe{zy-ogrA?`Th^Qb(a(djsQ2<^Pz1 z>hQg)75b!>f#0-MEnJPy9baaYtYMbdj^< zu6VmP)Ab!5aCz1)cPkP>`h)lVZL)6cs~MWOrwCfjWQ+h1+F8saxAMCDX85 zx|1pU6~_IJ%7IiI80=#9`!o5dBcL?G)xJtRb9%#Re)zesV#`9|Hp8^!lwCtoAwa2J zTq=fc8FQQvPvP29Ng2-(BkexB52LKC#&4K+<#@{y863^V*-HJyHHt?X;M(?go4X%&N1f>wQJD{4eBW;GhdP)O&vUHt z5U0n>+3s-JA@@nGL&FQlv&~?V+Jh1y;?c`=Zm$okhKCes|J*>4giO+>I)HqXb3aLF z!cZog*1D6IodszG4e=q>jY*+bX!ZTi1k0l2<`YJ3>BlAG zLLTw`)8Xn(ZVbyUJPS8nJ70pWSeExrpH`HYeI2sUy*8POJngo;A=X_)cxnhJOVR5P z&tHd><9Rk9VUMBZ$_Z-#K}b!wUvyHhGriiEtbkKqEA&%g5AG=8CGx9D3WGU^Uhwy0 z!14T!AvQL&t6j7+20LI7W5GD|Uxj`mv)O#TgQ+tyPs%4yNo~Ay?Wwp*!>VX<$18uL zfE1C;<>e7@!8deu=O5ZWlkLwT7xON`mdbyB4U3R%7lPWGoxY|&{DK@S+CSmIR^5*l z^8Ln*W6=&1V)b@o=J3H^hL6c+S!d5L)^*4e9KX#A@mEs*$U?Q*b{vK^HT!d&k%s?$ zg7;uOOKGV!qGUN%!4Z8!ur{^gr78tA+Aj=xWDXZpcBnscDpeBA=F3cdSHZ6hq-UV` z@0g3}Bzeh_%xB%#y6LgeY1nipFh=hNpU=guQC!jhrz(gNJN5x-+%{C?@Aupa&sLvL z-=06rR`^lY?Q|U`E@+XwDlYA&1cqkx!RWM0c10g~X}Z3iByQq)g2RzZFOBF*he>07 zIIN3Fejsf{?`UkzoGFoITi$XTKfq!f3w7=|nm*kOOVm0JyOsVpKb>her}c8Sa}k)6LTGDgjPI8p?nF zFNw8c7(I6tsux?+kf54>BUrzEh$iZbn0uKwPN>WmKWd(8bpG_wR;Q}(dV+VY;LVqx z<%)Rcf)`Cw*+i8_shci-!1H=1H?Xo}8<*EqD1(2SrF0mN);G&D!SD4H%K+${6v}0h za^SOEb+SULWyGm~MvqG$L!OVkKn!WmiLqeMrq(MG&GIZIXWOP*>{rTAo!7d}LwGQD zg8f&nBBJxn<*S`~r4BPJi6@51joj)*mr@dI9JlG!vEbn!O`I*^OL*}?T#19lqNVAC z>PqEl%(f;)nKKPQv07qZ)gZz`=}+#`_v}?{UI#s?RVm`2J;B<&Hnxf@9eMey5L9|K z0O%<`H@D^jUKJy8&8kC%f-2>A=?I=*y5D+E#d-TRp@h&jY~#U8o6+TXX$FfFlkl=< z5H`zCs)`pGPl-3^CK7?}rI!yd3p$}9WsgsH!Uug#5d6OMP=n>>R2ozw8BA*7V7C48 zdktF^LCXIx3t(duLAL~sWOAP&CQlr&9GBn`{Aa6@u7&5d8^HY%nsmMEX{gdpP0QqP zo}5s2jC9YHI)JynGDn}vT`0=e|9Zz_UZ$PQqdUDs2=JBs{Q4n~gFZWN(V#)arCCz5 zL~`@1(^TrF;Mubs@SKrcqPJbAcwqaY^^x)N=sW*n87cYz#5$=b!C^x=!X(OCTK-nY zrbiYO;c2mBl$hLoy|*jm15d|c?HgTp3(+VaN#(1XYx_dx^cmTV5L&QX68@n+8mAO^ zFQoK;nk9T@F2_Nf`jm#W25hMXuzX5pFH}Y$*8z$jr}BB7?ef}ze-fT8zk931Sx?-B z1o*(gw3*He*Lob1r0afljOygd)K$ZGccQc$6}6fd)p%|GS>sfcFFA`Ut2=if$4)Qm zZ}EpR{V6=3GEir>F~0BWP%|Ea)1gp3jN?MOY~#K>Or^ex^&2GM55uoESIW?`g4Q<8=t&^;@|?MHRj;t|3B9(~ z_3b^AA*TJ=<`1gyP<2vRiq$;)jhJAnk&=i>-b;!E-bABAFXi=fpa+I3#jEJTJ3gL% zv>JAqtTK_F8J=4-K%9`3d^O=!~LrPvw@27HCxnzU5<97N*oAD*4v zK5sO+!tKe)05yJBI$r4hSl$V*Eriq(@xGxg-sRD$)ZN72K$4H@|J5*=Y+LWhMx5L0Mf?S*R&nF|Wyb>vgXrcgWbRtDZ-@>9 zC~OvjYI^_2|)+kv32Ut!td}H59?;Wx^L5g&}e2>7;V6FaD&w?d%86jUZpTc&duWr z9blav5>6g#3-{!j6G_BA?Rusj{gF!_`0l+>Nw}g{ z2%m1@6qW9W^-jfZIj;RJ#@*_h`SDNyFMp3~!0GffMsrq>A5ijo4qz?lxif7lr#ZeT~$+4>BlQZ(9Y!F3}2y z55Wm-O$n%dF~e%}$0?w$90UtXBSm<4)~PD8eiToH%V3QvLMt4xGOzB0ajTh&_8V#1 zb?gb`n8em3VF&xfcV{PHBmo*W_M{HBu>4Tx4KfI9Iwf6H;uieAq{qHvX=lxrGLrdh z$JrF%t?@c}j6L}H=t|Pzo#`w3 z&frmDl#TMJ#cBzb(v<7r!VKYSSv7})TL3TpOGGxAal&S+Yz}d{gm?Prj!zcnqn*!L zwr86Jn+S9|*Xj2`R48ry_@Zs70F z?{xj7$Ph7Mc@;B1ksT7wg|xT7Qx;W9U4(``hW*ZFEpW)aJ}&@J@7_pw9yB0-BKw-( zO{ksE7(bfK$ZD&RHNrS@m*IKRh$0%{#gD^2~n1*FskaEqynSv>66QCk$DIT0aSmB|4)CDaS`i0FQRae;ro3x<_#_x z)+;|G8-|0$c}5QSJ$Yw%+JfBL!mmhWOQpH6R`+WkI+c(^_|^Plwhx~p=1~UV`Jz!tAj>SsV72wvosMq4V+x()R9y$7Q5?Pp4&JztlvMsIr7-XuJzuT zvsCq+Gck_>a(r{hOJS3))N)qHZ(dv7?HVwz_MoAnkC_T`MDTyKYc2fhL6aHpdfOI$ zlw~yB)@Xm$HDPn{cevo;x76vIV*4ah9}j)J8bw_JQd1H**L{|JVWBK1Tco>xFS+dC zx#>ZX5aNiiBTcnJ3K63UJqQzlwXM(m^1fCZyvZ$z?J6H0XykVK2xBID4DM#{ffztT zp5Gi7VE-mfZhr4EE{rt011p~*a{R1>=Xr3VoBBBN-5~huM>6b%8iNKumVReirH?Cr z&y1%wdkvI|QQ&WUv#Z}9SI%eigPdZof^TNcDH@FO&BBduW4`;VwxSJm_yGkbP4(XG z_V~{Q)`;Le5zTIt4G^3{7&tx~%2dd4RGZ`L9eY2$C&sD$8BKT8*GbmQ@!{CyIU0H< zx4Oby>cO!2uCm6xd3DGR&pmP#u#$1P*uV@*sr3vRO=$b)w%%}QhDgtBsX$REU(<jUe9Im`%VXV+Wmu1Mf#!AS5wxJj3cRE4jJ1LGur_v zh=mMxd19f8BUENj4{C7LXF0nbR@4@<`w2yE-JfjT+ae+3;*N+&i69?{D2Y;r`2%cU z*i29HL7%_hB~9I+E(_XSO6%8M;@4&mLG+~{wHu=QnGLJ*c^;{b*=R{;->N`sj0=qD za&(y}APdtCa{1hCmlKbx5kCp~@}x@nzbJga1g_RS(Hx3>U?ECk)s^BBeoeaC8yqy7dY!qw zB(&kg{CC+7Hz>EBO^%`3q#m9dkuhf&{8ty7l|vK81n+RR`m|?A@-0t(D8LF41%Y%* zE5iTef$5g{<*p!mCCKBa2$y@m;UEcL+&!H;9d-;)2S|nhI;s3q0}Ql0*I*IK*i1T- zXb(6wU(lROZw_1u)}}14x~vcQ5h8;v{}d1+zO=E#IA6(dk>7*v1?DO6zTSgaX!q_p z7`W7Un!v`lOneO6Af z<{1*7ukuq*I@ZJv&=60BmD9GX3SlU&Vm^qxU(YUD6nd2dxl>HV)=i0?*&Ertl(xnl zDhQ@FfEmVLM5|OTVnSzetLTM7WIRC3KJ%x%B{F3V9<*~!h|BL2S6RZgtghR9CR0NM zNln_O@1B5;+x@XN!5r;q%q!X2?<)WvTlL7RYTq}W)|g?^ZEf3iGvPVPi%YkUM)pX1 z)go^d8?We8P~yu1wz=#t_?@p|@>GXAJ7WaG|Gp&c-nO7KqvK^!J|5m4h#RVyX9OtusWV;K2NE}5Ug@G zwps$kQQ)U3t2@dGPc4N9QM)L9Ul_$?{~L!O8!Ou*3pK~^ z2pEy7vzzuPfYySeP@w|N#$tfy;Js=2AhJS{lOPrEP^RFIg?(|gTvV)}s!y+#t6h-R zv5!S!Qqe3mIbMlZ027!8^|o^Za~6Ekgm5#mK>w73kyKjJRp*Rg5ljRJ*M$8jnK)ImymEZmCPuh?4+ei1K<{vpTuFDcx`9X!O z+){Ne8@p{_1^;-c*Ha?z81Rc&&?Lg^cN8kM2U5Dzx8-`25U+P&E;~MB}v%gv8b~A2bm9JaJKGP}dqUye_!7hiUK8u^j(5HKZwZE&Z#Mognga zk1yC4J*M^ZF9GRIZ9`UdYZ&i=rOyptI5>8zQU(Lv*hrZ%w6?ie_NnYe&CtmgdZ+`f zdnuI~Mbj}w`AHGt>7!CyfTMUOIwHTj)IZ2iP${J={NeV`uTRtZI5<329lnbVsH+Oo zv<_&MXuqu28nmT)b9v*9A4g^%dbmm?O(2ejDtSfJtAG2LATQNVK!+AJ`eweUsDCWo z0Z)I$?llkOg1-{E_k<=2L7ik-b7%uk^7otX{RQP?w%sh6FA_!ov=F{%q$d9xY1xIU zrd|E=VwfOudYRu&vCo=}kw?H30p~!vljKhPHNYUdP&QoDKBANcfxTK`YU|jis&VdAoQ=1D~MgA`>i>weFEpL zm$=5xOYlerj~v?nOv};M_LtS;D7cNJP~XP2j5z|cA9jiM=e1n zm|t|Y<7T%g?e7^wHSl5@?s+QQK4yAwZS~Njh!DRYkV##H)Fwit zI#TuTd`YI?-S4xE1b@3 zC|&Gl-)?@P_if*Amj!=$g z1CeR^Px7TG1a>JV=8|)Gx)o=Ite%Cac#BnW!RKmw5v|yJA8<3K`#RgFOZEM!N`p!o zC6<+jSZc!*;6D+|lh%)Zst|AKQ@|}odPQ#$<#E_T8)$(Apeau`x$BA(XgdEG__x)V zYQ&*4luOzKdyp#e3*+pk%%U#)|5Dy>^RTqUP}3Xh;>u2g%p`C)*PFgCL zDnp!huOr~M@r*VRG_7ibIH<~XiN}vaVkP0~zHrQ)k_A$zL{eSVh+f9til-F1RIxOnzc=jR+ME`YB(f1aKxfmm|&M@oEJ#n7n(dhd` zJ@pti7u2aO?t&NEvPsZKRFnPhqj~!|&;@gf!09K_NLFlS%iSVAWdsqMbYi1{XN+cI zrw^>W0rwuiE9$+>t7vvLFy5(d3gfgW;VDQ_+A9i|GnX_d3E0xUB?H#Q5k3Bae+@buAK>a9Q^~(5~%?rYc zGJ}ttS!qr!G1c<@XO*SR#6yI6p*I>j<5b|BRCG3rlrUo6R`3!n(q%WqLT~?daQi1w zn7pztLdso0YT_3)q3_>g~O5U1!&Xsi@7NUC#ULG-QEfRbn@>2&ycU^x$!AF zw+p{};&w()gBT40DH8w55utEJTMmP@)wUNr zNaDQk%lshyymYJ~n(HDcDMlEFjUV*(hJ}%uleb=!tZi!{E=$}qHLm2jT@M&Nnyj;( zx7$N=NtMWugb;Ek(U=Yjix)8r75%b>U$b~YB^;1b5}uSL_t>V)3LqJ&mwpFA=bR%I z)AF=G!AKtygGY+SVgPxEpa#;QaAJ_S#iyNYyKuUt38(kCx>6seilP0F2K1P)?Y?DU z2cia#A7DIT(-$ZUWA`h5O!HAwVxFywJqA8~^Y`Z4zhxKNi5fKUNm~+va$;vulrt)Q zHIBcMtCY+*ku3P0fcGEqa2-psf~$POzc2kQ3t(AE@uXDvH z6vTZQ7;XdS>$HCIooya1$@081p-TT5#5ZKikq9?wo&OnKn16kXlHNUAJK{Uz!9Knz zEWSxOQub~skA595WS`aI^hj8Bh5ufv1F!gf;2JxHS*xA-)yG#{g*WnXBT}%e8q32W54}(pgx5lp!bG&{^(gCm#RVU|zfMk+NSp`9X zGG11lN&8oUzHo25Z`3E6rUb~`xlyWFym#<2;mycf=yI=XeQe9KhvxLk4Un&N2{N%I z*o;!Yu)7dzIb(QSf|z>-7|3-2xl4l9@T$!p)5pZpU<3wo5`4++HwQoLuvsODDHGi& zd})}8W;k%FYQjY3E>MZ-M8}{{axk6#DeQAgIT#L@VWp~l?=LK-|Lviopz&UN$Ge8_ zn7Wv%XNe1I+&aklsnCn9nIluPupp%{yhNKfz8#xyZRcnu>Qw6e8`0evN`{D>Gy>Go zIft@hOXu+e!f+lzGD)&^Y7T`VvEQbN!w5k~a9jDGY;`*oPrJnW?Z|mg=_lT+Gb+>? zL}k%%GpBt)q_~Bcwenu?&QQM1gii55I_9H5W}nh)f-E7s{Jy6)>s6HYj}E6y#l2-G zI~ga3*_NSDRF$^?Hv?xXOP zVkB)_usXqJKX=QgTLjl-o(#@lwdZSt*_H2dwkz}@i&21PgiGxXPxoy$aHjhS|HNT~ z%9`l6V2`%lLR^5|5A0GMy-h^e8k14XAzM*8kJV4lvvoI zsI>ADVwb!X@bSEi^d)#OUAAuOj^)N#=~Z{o_O83E2V&ndr+#TYLE~9NJT8h&vd)ur zd>{4Rw?Fpsr5(>|zZ0I`8#?RvqM=6!qVUH=wH8>J`?1^eK=|6Cp^$UX(NY`WkYr-I zZqRCM0h`vvq6V7e^(`hmuXFq*wEHScwLlBw=9M#egayw%G)4(Z*OpsbX|nnlAH{K6 zCGc~kyl1BL1NGy2ySbQGLiiq{T~^*T z(PF4TI&0S&QW=q-$WKqzjm}L|5Xmk*DS*Qc_VMREy+^$*DRY`h5SoPw;N#P~dAw;gmHarjOAs)Jb8rpz-@qEN6MTC)Ru54i zuG<2-P*i9}5whtH(__Uml-KG$j_bkGTemzd_b`^ao%eEARIbhpvDYJRv*t#{=8>k;S}(`o=f=qCWar^5#Z$P z2vAz%>LN{|W!UvB7H zVD7^vK8zKr(FDl}EDb@Jv<*De`(I9_$rAgJj%3V5hI3Byp?jqK@Gj4P!IRDw zlP2^hwMMjj3s>Yk)=B!;giD$1Z*_K#DNNcAJtS*ZRRv4Rd_m#T*g0RTbEs^cO}iZs zD_N_9=l~GB>d;OLgRT!PBY9zW)tk$IPG`m3u7wbZJ{zc{>)ahLhVweDe2a1N6L7f+ z;p_@~xW@mP^#A#b|Fan{$YHrWVt&DbLJR2P`*|ph zFuoe zGb4K^AD5ZV-nK|168aT1wYqnLIb9`XhOjmYtApveHRSd8y=n5L&3C{Io-v_NtfZ7m zWv`mBBHObymyX4D!W6R4pdCpp#aih`mR57L-Lr@`8#07F7PFAygXu7m1v>h)7=l+7 z>boC9-!?{Xyfig%Ny&WA|4sq)`y+Xsj|`tSY3E`1t7u_OXsUrXegaaMpvi(%Wuj0E zCXmm1-sB9%Sjk;PM{0Y<{10#oPiFrj)3PWj(to~E@k=|lOA zxood!H@}Tjq!Tteu?hzTK+hR7Y$$HTg=|FCmNkH3QxzfL=-h9Yy=l&4ud#}t^MYfb z^luc;50R$69Ua0j-5KxqjLYS-{C!0ac<=hab>;g%VvctGCd^&GcXZa-|Cg9o&jzdg zSXRNk_uEw#`*u>60}G!1w&j#Zy~FWs;G zeZYXIE5vjouk-E7!>Qden{SAPb3;{k!hlKcG{LDii@OfqABqksc#GVrygSUcjFt{7 zziTv_!6*ky_|L~W<~BWdc5_~#QD!Msou8uX*()pLmZxMKs}pvUmhq|o z+FxNw8>BQt=U@2RM>I*Ekat^fQc;CaH}L~YFkQQZH$gRG!*zCpg$B5l1_cj5f8+XS zC=p99yf;EHNH0kBlfE77!@kQaD0OAgIDg4D*BtiAptUn?URzDi<9s~+@JObkEiBe& z@OMCd8F5@^6Co`Bk16IJ>L>w!(M3D-dUHsX7_s|ikhu>$(i)lkRnYDWY}kNdl_v?v zyq^u*=b+KZuio5!Gq?X6_Ruc%|55+csQ;t>ff|m59vN|WlM&&ctq^QSjgvlk4n{vtmlNSNOC<5SAVDh8@ptQvivGFUZI zs>D+i0M(pR;BaR>x&N_T%-b`k?_co~)cgLPML#E+^puPIe-7(~wLh(DR<>bOO!36uCB7it+D8zu4kXxCXxR%&Bmc%{9U9I z0~b!LT?`kV0_F3r4L#%Aj7(N4`^mPl#oNBz#h&f2^zS%i(whXOo?QlCtKG*_z*|aA zi<_&H5&o(QcgDu6P{Cr zWNAsFx7|;(KhZ-V)yu4)3RzhSQmalIx9v#a{nB z*pbK5tH@Q7T}}Uw-S&|J{1Pb?qq7xjtk&mcN*>SuS{l4n_sF!WgXy zv9n;9f!uyYA&*B>P)Hg zDNrvI9T5`g{UG@O~d-5x2QH^dy&`V%dVkBm@C2ccOx9K;9 zwQ}awo=jHY?f=8xTZTm$t?#4EFu))+fYLEEh;;YR4F;0ZNJxp&Idr!opfm!~D4o*X zAl;oxH=Gyu-oO9e=j`8gzMc=~%W%QPyepn(-D^G1eLw46$^%;lnj z*$;N=`w z=>Dq&;_M$K5M{-!HCe)=n*(dvh0PAHpykoU(=!`Z zt-hCE`X09)HSe@>JMxj7B}dKc!C*>f%EXg8vB7mNwbT9L+0MI~$f!f}m}-og>wYcA zdfWKbqmrid74>q4#0tUTuc#^8Gx2JUdomN1Hrb~A+b?#v(>{pIAH+vh*vJ5-ZVG@v zH-j)`X0)lHcDU?Rb*pdj(-!2)!FZiFYzYVNV8Ef#C(~NK*5UHumMVJ5;krMNeU)Xp z63a?W@j7z)nh|-`smohFWiNho2^a2Vl!PnVppq?B*Y0}kPAXZ*<39WSSfbWZW68IU z$Ny@yrp$g(qdRG-WMoUr@c9N1E99<3S7l(;CuQbBwxFtiUKe;DjWKnNx^h@HAOVX3l}ZI>6I5i#-xcn z`1Fz2<+xbnOg%+Ny?4ScqfpiBp9!p~J(@Wn3UWQjy z+q1SVLRWY>Yde*H*7L6^@j@lmNbzL3vg=ReQZZQ&UN9Y+bT5?JTyd+{LBD$ro|h9a~D1qYsc{S zfz~mV_nu5gAdn_gCe!J^oH`MuU~A6 zX|-3E3rQi7rjveA`c+|SINS0QXS)v)Hm4#nMmRh!!gvo_-JH`b*87e6c#=ZZ)6duX zMG2V`Atr)>NZdA!_b_v19#7G5H>;wzA@cMmZII-K{x*>fTWW?V5YG&>Gc6 zWoMs5wg_^|6<@cT^AqSjHL*kz>DVgIp|WcMuH)q|kIf9+n@OpV<|rrO(jSLPoa>~l z2e?E6ja@v=ekl?ibbZvQuQ^mOl6h5HZn(ogp{XS9wbP-Vm7>{V&ObG>nA02-Ic8Kg zqgVCF;&?P!m%`p~6(#M`w*TG5BlY{Qe0k-BlC0l82I|4lH_Ex9)sPMcqgN6v+Hj7K zPc*e{God2CI2pifA=n&P1Qe5Svp*YXeTnNfvk_z!zjsQr;)}l@jdC1|&18vqUS}^P zP)>a9m3WoMZBQOFy^DWZssHgnMfC+8q~oyEhLTNk&hc?ktAs!)#4Bsycky z=HTY`yA&S}A{)z-DPz@o+A{r7$zSlGZ%bo;wdD*Nu)%?dl@0rbLvP$lZmyC35!+-k zpX9LL&6_EjRjn#`N8z3u+Ys@%uTDYdA+kc#agZe)9NhUxo%|1QRp|#rj z66)x-LsZM_OwV9|5VJL;eef`QQ@Gx@8I_MY--sq`ApUa;9qq2j>F}J){F1=_HCot4 z3eu`oxJ6E?6!lXQtso4I4n2|Ut2^E^oBkjrz2!@%4NA(YUNwDok(V{Q50}5k&)oH> znJ3_@Uw-X5N0TPqP%WbP=%AkSjVr2In=0bqy))H6RxL*Wuxe@M_kShT+Lem36c}<| zv%W$t^m5Fl4zF=&Hl-Df~ zo%x0$X$Aqsl)NG{q@0=|C_J4REX>8_;L-4_;feZA4|W=s#+t)Zniw6P92D7}C?7K% z#TR!97?^?glp{JgNymb=gPVn%WNUx#srfJZ;8iaB+*g-PpLDElM+IIGw6+2-ZHt+N zTDo*rZAAyxljcE54kKyP83rM5l-xOHgGA{dI2ZtXQQnz`y zueN>=Uo#pJ!2ar*|0!>V_%6yK7+?Df*v>`jn5)0r{H;tis&KyGWB%3cw!UU-rQ5$` zy)X7gJnCyCGU1@;?rV)iVAluxhafh$j`A`*n7`d8+q9ntCF|L*>=|#89bT6c1!ZfBKVHTVq<}mO)J}YXn z_gwGRJRq9KXZo*A_#p>jB!?yw0Y#!g`qgijp5a0?kKPdIc2tQgKf1dsP4Pqp>mEkj zF{KPjSu)-G-H$Pat0g2&dTYxjx1Y~<{ko?wZyK71qgPgxj?DYTYgc|PTNrLm;hzEN z$p1QPP0}c)zm(SYLoBDWyH#*0I*v(3O(U_b!}A=Q*JO@Gq!-@aqvFkZ1wYqp_&JcDJS^Y~>UXZ}SiA6*B|?-cmD z;5FMfdeQ$f-~e_oNtUyvfk5$W%2)M`!L+;fBjD4Nm;DcZFGfctcjIfA%yYU$+65#Y zz8XEB8I8{x?O=MUBHiC!kYT)Qr2}q$2dNqxe|`wY42=}^3%K(L`-*B=IC$g`B(%v> z^KOmo?S9`uQJSjPjr+p>9zoBX6YHa{PcwI>v@{R@k&OFiPVS#SppPM(MD2+WOrLoxYLL2kx8l0%o=oQ+gctEz68?se0_Yg0l>Tg&z>xVzQNnG~pq1Zpsb%(l(nn#* z77tY#{1Ay+jd7UNLX+9XBJ=DnYTDY9QP0m_vJ&&#B;G`Pz+5;q<)9U`;gozH5Iy4g zVUf=Mj1;2H$Ncy|59cYA$F zJj4QpcPG&<^KKSL9g*bf?;O)FuxAD_E(Du98N`pZ4LYi@<(QR)?;Z8&04K$n<7$#d z2m=l|k+zl!aEB+T zo{&7g-E$zqr5gJpDMh@I^|r@}_ilx>d^$3D4;0@)VTb?UJk9r~B=V%qCX?^8ae+^VfwH3i}}0ZA18Aid)X_X67u%ljoL$wH=JcFj3+;=9$2 zyDy$?e|&pLe)xetR#F)JFFlh%S%s<4Bl~K4FuKeW=LOp_`s|HRrC_6S1fd)syUsP~ z@ZImW`UT=Fq&P}8Npb}ShkVT=w;gbf#TXc5Wdok=#DypHPo(xVZQw%%;|`5RC9tyDCkqyyInWKD@5gHW|kL+EK~6t{Y0YU1ea9ETtbCpHy(gC4;3ZhPhm06tWTdmwgm!5wQxnwSxk?uub=Fq!@`mW zV!x4Mt{u<;pfgHnB~NV@w_LrlZPyT<3n8`vqe1p_zBL# z5ek0{0qlWVkzBjA5=rCXBc3lz!V(C9|9Xq+dt9k!=PC8Y{QepS3-mCc>ee+)+-uu> zF!jb7SCi`QL2`f`><=b*qvTc1i1SFcqQubzVM+j^CyMax^=@qTw={+ji2Qv892uy< zDMyIl@6b}LT@E8+f->SczO?gZK>$KV%WCgM`OUT!t&{BCnMcIdhDHTTpPbfu9o7EwU`4S=4n z2DCb#=Y`NZmKAY-NWgr>M^Uy0|aRi zTg-LcB8PA$!9KETu_w;K4yOdH;D#rLki+Z`k<7T6Zp*4YQFl3?f5a#z3rowV-kWNZm50sYK3HL!94kHS-iD=c=C|5~oaHK^I1;ZQ3|5^?C;ofR7>=24_y{$uKgPvDFNz}kA zQLi=f5iUmMgeQ3Ck5LZwCE+lb0t+1%g{JaoVq^jV4VK5H^l}p%)&kl7Ww#ckW1yAi3)-L;v-ihyz57;X-3=)`aDjxXSa6YmSCHyVp|P>?(C)+g zBNGPD1b*V;kaPOiUBRD^pV}!Wqbn&X3f{|Ws9~iqERy{h_A*IhVNI3s76H822cpId zmqeKGZ}x>q7Q6eHf(^1;u4mW45hWH3#0;sezdno8Q4W&=Pv515J$fWt7;1E;1D%)A z8Fothkh0i?5|`5)8%2Nx%+?=VaEI{dwd}o9ck#i1^a%|PgL<{hPuhE*R;EX7l^pmY z^i3s+Tci4|aae?mQ>^Cq*S&J>;lxY>Okoz8vq*eGVgHQ!3Cb%B1$Wel}P1Gy_~=zmRHTYHHOMg1@dsVc_z6K1JyhyDE=) zUpguVuAt+_ZgaoDg34U2lk(c?$XDfZ+Zp~xjpPD$+3rDDF@vo^ICMt*W?e8P6%~~g z#x^^W-tdE2R?bpY(dB-f&6YqcgNxKUYROz!{G%@A)5G|>Tet28F3L#YCeV6MMi{=) z-jkkB9Mr}`0jI0i7e~%n&q|FXUlI+Zidmo4%szyGVIYf>>>n21gfBnL0RK?XNOI}1 zDUgh|r;{q`_Th7NwcwFyRq?Qo^k>)AF8mX!>F!qm`BFJz%^_q`%e!(_ zr|&n5)an^1s+w0~)2r0DAhJSYMnsYei~zPBUqv#FwD+_{UerBe)zqNu-kYf!-AMDX z^WH6)Sm)(6yt1xlcqJ~i7$vg*bPeA&G-0JPdUUqtz3mADx^m)MNH^NW*@DkF61w*5 z3AYh99qoj76tX~gWc+=!jQvo$WWEn@-|696Z~pAf_0iyV<)iPYUSDgScAobq^NrJs z8ni)1{LrvAe>dInyW4M3EI>fuLsBytz+v-jroQK^V=R*^q2-CB!vND@JIz`@8El*oIwU^&-FZ(} zx6&qQ`#7t)?rOj8oAchZGv+2>sBGHw3hMKwo#Hwhe}Y#)2`3fk2C&TIyYPoBnPeIP z2bD8+t*pnL)ILM10&_BWV#hHfrOh@^%-ttS4DGRq$#fmv`9Kk8Wc)S?Q$H*^S#0O) zlYSNIWDRg3npIe5sl`1LVG};L7xH2}cj=N7+P<7N6U8aCV_1(K%e>6j$awx$X}HX^ zbLQhEhN7F~tx(FHMNBp3zypj>arv(?v| zkP&%%xx?|v4C7LDQ$6EasLtDAhd^Y{Z0oIkhayZo3(R~kTZ!+f41 zU~kX@;eoz6?u~GIK>5!6Uv~lco=yzEb1bYynjrRPYcOvN3TyqHsTEwH4;c{R(xtkv zc;md1YZgQP5|Q@jH(+bFuw`X)2+cW0_fBD*>w?16MerZpx*L`C^9Dj zNKdd5(peT}P@?(_cP;CM5To;i6!3r)=WCs;1Ifv(+FH+T9W|Ub7RSE5$fxNvf=(}; z6P3HBd-XTGf(`zSKhy#I0OyMi#%%?KQ6(3C&wx2UOc9DhQG$EF7-$$Y!4g z_mi=+v;DaO-iorvAjp^7$}BZ<-jc|8_|V~eg*D(NK%xj)^>Cz06A}GY$Q3 zWA+CHX+>};g|o}k4o+~$`I(C>j=P6`HRFA-hc&{#WoRJ3N8RFHKHtR3Fbox^n`V20 zADo^R!jF&9Wzhxf-Pw$}i%i?-$Apr!b#&SlLEcO)qXk+Vva^WLMF1e4&u=#7pAt^6 zY8DQCI{Xa=sjKm4+T*aPshHEInJlB~{+WO1sU-vZ!vH3ckmej{uciQAG6k#8D>$I= zViP0sSqdZ`dtj0{@kP|_d%pj;va|Qo-*33$2J<+$k}RF-9~_b!58Gl~w`r3aw~>&n z*t1z;8F3Tl2+7``op|n#v81osCmi8O-?}LxUUU?A-7aQ163B^?R48+vqgfsik(P7LwAM- z+JUA=N{fVu6EhmoBqFM$oRKz&8Fb$cZkaRdp~ejLK<`hn5lC_s71FP&pZW7~WGGSC z#C9t(`VZ2*+IpbS?MRjvIeGT95ohxrCBx*(8-MV;UY3{DHX(1*3>>PgMj2(lV1aBg z)jreRyFt;KPUMOqLz zM_OXi&I?&_D?Dv?rC50w;o+2qbJOg!^XnP$uWEE^Ux)AaX(m^6-w4v|6P3ia379NO zFR5f>ODYdk04lr^C4&b^|7b)1jc@6}9#Kf+uUq8gDB0*+|FwA7tbn!Big>vR9De8M zI;8!En=F#WT)AQ$E5ZGZqy_$T5y`bB$Srt)F$5bb{N)^d8AU>JJHw$J-gchqy>N{; zqT@qIA4cR!EK`F|E;@=MhR*N)7PW7bocc!W!X*O9Uz4fGR^7HphN|^siTA+>hQAJg zB3Q6-UGfJPL|a9wKM*3ixip6#g^D6?x8KVX3S8DrQ{HCW1lEUL2M#0~Bq6U&cFQGs zKnGkd)0~)o-ATujv%Epl18BZ|GAQfz@Q%X}YcV4!H#CQ@JC-WXO{6JQ$WDJRENCt) zQ(E|e;)zy}n7R|S*gNkaF3e4dU)ml+*hn}V4m!vyOOr5EqIy!j4R6r^2S;gxJR~Up zMqxQ5SvT-tDnp9{KHtHOiz8NE^G6wLdSN(AenxXzQU`_66%KEFrlnE*N=H9SZOFE! zm~{;GR#Fe6sVweEL*|URYHF5xShwEngzn(x**0eRu{YC)`BCku&o&U;M4Pern7k6M zsSt6&%UAXdCkeLTRx0zi!QyxdT&m)&DXCvNWuT0r=?sxv;KR7oFTn{Sm*<;<#cjcK3IN_6bq9La7-IKgEE%1`J_CYw{)7!%HKt@tkNs7;=X4oA;5l5tX!c#pc!KE5B7a`A18Vf|6959Yhn_55l4tLmpI*Loia4(2~XAtK|t;=n8M~ zA~CS-(yg+(jbRb+SV~MX`$SU&U6{S(m0Wm({YuiW7Fj_!yhgGd?hSKU$ufwe&V{BE z28KRJS$`Yhi`7N%*mq2iS+3i7u}&^~X#$J9wp9Q8z>I5N_I8MD9G+ho(hy3%;=)Cf ziAE`pJ^NIR6DptDEVioXh)%OvzBB~*)s`acM&%X_T%gdypynP_JkfN`Pm|$TJ#^)y!0T`;N_g*%1HO7|sZ%ZMs%vjs zoD1M4A`BA$y45hA5HSRd11cAJDuYOpX_T+?^NxG;y>;@exvmy3L>w2F4itGy*27w^ zCP|Wcmy3yKgw7_SLe-2@{;DF=w~Q@Vq`cE|^x6Dyj`VR4|4I!{|I0akYgzD|mTdj- z;Hck~u-awn^^80MxWbVT|7|F6*@4rs8Qu0X+eTiWOph?7|rp zY=j+oiJ#n__t|(F{2KM-Rc93Ccs1$T(EOW)1)n?p$jGbXt*mVyOYYW)4CkGF*GFHF zI*Y_T9GAM2t(YqmA2=dDo(L{nN8M9`h>m+Xkm-5%43!jOP%RVHeoV>xPcz0xbQChI z+2CJae{GIS`pRLEky=Kj8wa505@B=lc5_+`P<-wNPgb-e&~S2+kR5eHGUMda>9H*+ zsAeDtm~1uE}IGHS30mfPIsTRq4Vy->|>fkh?k=zqs5vLuYV#z z+cWMlru5Hu8?t^k+qZ$7VKgD+f0XP!%bN~=lYp@KITd`-#e@YFv?c@PlIfyb66hgT zSx2np;GqoEDJ+qMp~kVCC+SHMq6JL&2zM)6)y@pKY|hl@YA)}IYw_|FNd#TA^dH-5 zmtR(>#u!`#pzWY*DA*f`(nB{5iow-(@&^$x-Z;8}b$ky46^XHt2y6ro!E_=#D_*# zfUb@F2CcyloIEM_8dF7y!=PTfkCO!C_;$a$TW%F?6*Z-*w1%Mr-Tgbh8%8jZNBq%r zJ@&k`VXvU1uV^~jx>30i!ARa($?x7K_#zf0{b}F<-Q^7s4nB{6oyQZa9-H=T!Q>q} zYdxod_sO`{#~PZs1@LeI=4+3o;2ho$qeFKs+1M&0>1k*VF4wk$J^_>~J^|GA;DE?qKf<# z^JBuu()knO+sp(1Q=GQ^W9Oyzi!RhD45{Z21}Il>hA~&4)wN{g-u| z{m6}4FIh2E0jLN>x7yJ2AW%4%+@>DwIeBu~P#2Ly6q1RF+&G-{<_{1Pe>;%Nl8gL+ z`Sn<3Agd|&%2QZQj?gm^(q0WEBz?Q_rU`E^_!vaC7Pr|hR-TQu1%+6h_K;9>03I6 z*xW|Fgs4xj%Ylm{UE~hfp>`fiL$}M+r?QcL=q`U2n>+MyZ-$pD3DPvEdIDcSIMdtU z-(|yz-lSuG$J&ajimT)bPZ%@QD!@_i~t3#z?!zZt!EUDdshBNCBqMtQ&q?~PRxV;a}e*~x@q zfuiygIkir?Fx7o5Yg7D7LXc zl4p;Q(RSD_BgOwGbz-$20g}C>wqf+H!@HG#*w819vWD;P5u-pdwu32iM9gUn#gS8$N=mEquuWZjf9AC}Pu-J> z8-9&J1kMp2rEo2&3}27dYjBoxwv-ss%&hlCk^8(8@@Hs&?qb5j8R=PO!H#o+)z`Qr z@uO)^O^~Ozh-DBWLnS$z@$nlIIUf-Ky7VO3YAM2%-E!HVQf=C@{ZhXS{l+EZRhWuM z5%;Q1<^>8YHAjr%2;7WY0x)~9K?+tyHb_NfOLwUi%IPZ}ivbNYzQc{73~8VUpAVH( z+_O4alvLQm z;mE+tP9Ie2HX1^lO}N^t(EFQ<&xyhC&^eki&Og4V4LYTw#Qp?OUKCH@V#q6KR%LIf zHU2$shp6#Qz{Ry#4oP%Ajrw5`wfj@pqjR2z#pCp8tVf0)iY9u^v<1p^+d9rBHx_lF z(P+Ws?nUS&!;hX?vAeZYK1jM#sG_??d?Y-?K|9HYg71w+KA!Z<`{T{wZ~Hw%l{~kyd!{3#QTfjK6<1GLz(D%>Xm(u zWdj)W5#*`dZad#l4=y?bCj%-Q$u1*h55W^FvK}jDhkN?uH_c)mK%c;ere)usuL&Q) zN|S+tvhbuG!r+CS_{bz7Nhl$kpr1~5Bj)WNY%E=@D-y<5iaMs=sI1eoI=$kNeadmqU}}N?>(Qb_j!U0g6iyT&&M4 zk0|EB1IfF|DYW*r3X*ue^GAuhWF+B1wkg8?(14?+*{IPK#Z&wc)o}cWP~=nP<<~7D zezg1qn36xG?-`qy2G7N5UUy1sd-UolzObiZTL8!$Un9g%wG^wUgz|868L8`O$<$}r z#v#&l!!CYE*?Ss|iKOSAq*9;;<4zYHyD{3oetb`bKK>$p^dz?F%g;KFl4Bj{YO5at z1s?NUjzBPvnFq!q(k%$tEy=d}$(RCp1#H}&NMs7I3?BTYBF`&Q?y1O-4NTVIDg_ip zFH@-1- zfT6*SFhLl5313~>V>j*#!jDKiRt2jmEeQUu`!EZooardo-7>IP=w`K{nW-9=_C%pU~@82hF7{F@NY@dFI zq5g)$mQn`0wCa%v5+AxlAQOlABv%PYg|di>iiQ(6vZno|ih1u7?aAc;;zv>pt$2Rf zpqqzTvWTE_f|oQ#cwx^KTbS>cqaQL5m{pCxvpQK3DC*B8$4eM_mjGm`DEb!Pkg_`&0e;Cf)LM-e(E5ET=9AKe*|g0S`yUN^O6Y}{un-%q;FSj?5m zQvj68NOwHb9s9FPnJplVzkcM5SoQEjHo@N>EVug*aTtiM0neRmGZyaXWDzg^mM0q|o|_NPhR z>ofhDE&Y45v(KkqlablSyb+88s3(t;ss-roKA_5mHQCaE5)Lq$BfoX1Ch5 zz@%-yr!kx4;3)tjulARbU*2)NXA=Dd)w7x2kiwDKjf!}Shf#1?3pD%3+ympbYo_hQ z1|m!bC*1gIPi8xf4o_Y3v7Y7eQZC4A)0iN(o&}2NOX=PhlJ>OPs9;v zj=G8$0NG8Ob<4~qoUhSwsnX%hCTU!W*Lh-kx|A-x1@h(2|1e8(&)bOnL2eFZOvz_m= z>$1iC;RL+UbefS!{9TdOc?uwC5jI>zH3xc6bgc`ULL? z{q2nF6UPG}Xk5G(6^~N9zjrIhf|J}m^WXf5`c->Z{QVLy9{RKaxHbxed&CaYPh1A& zW4*#3TI8{8C#;PJYV}1qnFmTM8IJtbg|dWAOBNabXn6`dcWn;zwKzgkOW@ikP(WIn<*NS>BlpP?9u;U zWM>K-U@5X3;+mhH>d^F`Dgu}lfN`K;acSUk?legbYFsHu3)VyCQX%{KdbhZ^xRDP< zrKP3W?>7cgB)l#iUq^|Qqsu@*>a_5S;rPw41wW)IBzsq=LoW4ywl!eHf916ZC@sn; zHW0YO?{&iS^wt13d$>1sba%QBsn5UcTRB%$P=Jbbu^VIPKt6P3GCvLrlX8%h_tLQ0 zLDwzdshpm+$eHeRDpc3oa1*Y0sXJ4PtxGYR2GV%=0n{uBdnPA|4~Bp-15mnC&RpCF zQ{rRR7+tz+&b;pikJjhEcNV}WT&|X*!!Y2JbPB8T5v)&#-d_XZ?*nZiA zQt^(*U1#`~sVFlBN3-WdU`hN$JHpHb`Fw?Rved`!CniY^6Vg#EP)B}0V`Fstn?D7c zLi9V=l&;YzU{xZ)_QniG@9Qg5d-!(cycAwqE*yz?0F6Pa`3APp=L8^eBhZMH#CbK2 zYc7wbne6fm#ii40z=hHMA6Gn}ZHzxFi9b^>$fbXUIC}G21DYM5RB^A%*grCpIJQ26 zZkzqv;{|#p`gNxgm3DKNVpp#Qlhe(L5d|$REgSNYWPHW;3ymLNzI^GNWtvd;-7;nP z^835Bo-Y7c_w#0cBr-UB^1eOsJtUbiJsi0K@VWa_nVFek(=&-)M7mAhUd;fYU3~%| z`ugW>u8G#D=N7py5k4F=N4p!ri-R_+YtL^uOrQP|wYKbbBql^hDHCPKl%S0C5m+2l zDMpQMkFP)jx?$Y*9Dk`*0bGu(H#5dJ_q*XN6*%Hi*sc_@;p)U_6kiX zuS84CzXoVnr-1`~m${cFn45N|8Nb+mXUe7AjNpSHQTRj7qrIm!XqD>wj(xemXzJDN z?CmSmf6cY;#^d<|dR?vn49b;MCi$5Z-?aH#BcFwbn z@OqaL;V1)l1=FYdSE(e@{_Cmt&sT^-#!+6Xj^$^N4Jje?4lc)88FK0j8sXGxA_Z-H|nW3LaH9xjZaj*8_Tkd8WNfY~W%ZCcd!kr)rh}?niCQA7KK2}L^ zB!h@eIej{zS(=}JQI)y*g8{7g2!zsux6e2kh{*|4nt_q;pHKFU*n|q-yd679J9+U<=xd-)ZO} zobRaail6RH{5Hr5%qMB(&m{eSH&xyXq?tUxM%v_~beH9X4NHJKAMnQbh*w(F0$#cZ zbQHrIpUd|@dTQyryL;(Fc0mv}R;_&Orx3X`;ZfEia8bxAjy+F1q~RMxFti*0IbZP( zT3(O1vCv_YuvG#5%Y^?j<~K&fq0o2&$tXFmuN=prwqLEVC1L+D(CQ6X=)|WDn4Lw! z&EZC`t2=_~adT+?EX8!*d3UnxX0h3Sp$gBjw9vui4aU$@E32(r z{W@oT+q9yRo0=PS$)AKqUdQyVvVUg`F6_HfRJG#S;J#s8RH!EsUfDRf?dbuqo%^TF zPs5*ww>GBM8&j6r&3MkaeXa#J*o-vv=*}OGTc^ryY;SLC;*1Z7x*z9xlW$fzZ~Q6n zA^FvPL-}^d;ASVb_64l#p>Gs_*7TXX>`aK%mDx?sK!s(ghNkA5ra6Y$(OBUB^+evx z^|a2pNuu}?5dkpOf`uH;4M3`iEBE&F>`MTzyScq-CpO&mnsvBc;20ik1>n;u=7idE zjzjC^OCewno2xTw_Ph9H{3EbK=e49{dL;m8w)nU5a3Qb~9o zw6z6&faGt{OG*0rsOsLS`JJQP+1XiVoEs{^z4FmR&Ei)Vr6;+N_h$M-rRDom?Gdv@ zdR5-8K;mcWRH#A_$2X1(JRMX!C4Nyss4iOOp{5pf^;v61PVm@GQ&CZ|`2)*~?Kvhq za{j)uqq0#OJ+INr3C{8qEB^}O&Tn~nU1c{11_iTDNjFzDvSuP<8xQ~C<9^JG`hk0T zPoMr7Rc5crZc`)Ln+BU53^DV1LpF1A}J=g6S^t? zw2>0SE|Je#vy1_6*#!p0Q|wGo=4f!tja*@j?9H5~Ht1BLq=X^T#X+|6woe!%sKnfl z0PPCZnXSu0qgMuSc;2&|t38nj==^HR$G```-?Z+#i(!%dYSXlr1ff@#-@R`xheGh_ z#D5$_lv&zl1dyZos^GsJ%(XGkRZHYt+5s|0onGWWR^M-iu@+B`7wgx#@l);HO)**Y zOt*W|0t<NaiAca`zJk!r z>TfeO75ZMkBX|awE0diZgn~yAxYXhl$-oY2V4c5bOn!@kRjDtnu#~9;ul?$imy*_W z`FY_whue-d97my>U%A%Hy0xh%GdeICwrG+?^ORfe=L{3^`r6vfN(3p7bLUZJ<#*`U^`1S&MQl>^EOdInS^c&-a5BX@@{#;$t?T|Fy%Qkn4KaC}( zVRM*AK|!_|)BrnY{RME75LLv6hSK3l1&M{~+v|}x9UpaPGUnHbM|TFjPk;C)Y-(G! zrW$3~YIWP)4|90AVn(zr=Wh1)=E~=K+Pb9u>#xu_8)ALOVX6WapER5+`tW(8{FHC+ zMIL~YXw$vkWxxBxXM1{F?{>7TS+g_IbnEmrMV+n8;fs0mg+@-{%RVLm8Dr$Y!cVs$ z&zQo5jjsm|I;or1Yc^7?PcgTE#Vqy9d3=Ju@VDNkeE6I7KUwWC=19Pf{7N!-&^CPp zCYTb#U3ar`i`1FooH=0y`RT;{_F^M?y7bqZ#3f^E3VH@?;N(F8Dc{RaLKXXITHT&> zkCEpq)OUjGr(^4bX(e@&;^oF_Rf5Br!GXzxG#Qpq7G1}vuUJIxmak*motzsR*MOtX zsq<97`#2?!z?|cY8itu$o7OS?=+-A1qsiuH*uHVkz|#`>@3aKMV~IBM-c6WBixItF zD+aQ-FLgc-rA=~<6WwWJiK%=zVv$b&yQgO-Am99O<+F5@a{oAPFkcU-;TY3GypQ?? zGJs=V3V%f16SWEw%VxIEL5NUY`$5 z4tu34e_=0Mht6B#&otSJ!$QDK+N4HY-OB^_n)lgCTfG-r@d|g&oqaTQ@7J4bv`3#f z%75K2)F6loe-wo176cWYDPglp7RnsBT%PV-NnjDP+c>Wk^I882xIeMC2RRkBvk=W+ ze5kZ|@Q+Q9g()31@eRva)sXFoz>Ks^<6O0a=_=$$nZVWAZ@iTE1415qp-p&`XX;f$ zY~Cqsc_3rx2S!Z5CbSRF)J?tB3&pBwfHqTc}FDC3Y#PGXFC>) z8NL8s7GaLxW?Yjd-&3k2T$|B|ciC83!=3p;kBJ+5Uq|9t1IJh-=2&B%&mNEf zZPKlH^U`c4^tAt1k)AXl%4MrGp0~?5`UWE6eQgf?i^INSL|tXwTDEJFL~r!A_d^Mp z_UFq?<+C~!#PZgYt&6AZ@$%hz8eMYQF^TlqboI=1p295i0Wf~izDeyniMBnv9~K@? zE{5wWwmVzHq=PN&s+WQeb3~EO?O~5->Rzn->|JT; zYz~m}?KSJO2O%VXLdw-owe~N43A&Y-<@J=5m0#9aN9)tqE(GySNfAdn9w=)#f6o51c|NimY;of)Mo)JthXzta#ky`;&qfz-q)BT?Xf&sm3#as~A zpkLjq5zp7gr^#&TUf&5i{GT{hHQy*P?-Va}ggtiA_t>kr+FSbGRQ224w!q_KT_GXb z-$=oCjQGzRSqh6E+_TG13LYJ7klWXn%`=_f?z=RJ6rQ3N8H(kZN64yURd~tvTS%w3 z(E||-A+@1gx(>M0#sv@@d!7Pn&~Img>D15jg#=voXAcKbL?o~FX|)6Ojd-&1O(1L5gU3sd}GHQ*DBweKGnCIxiH)nvCy@gIU%65sSAqDk?qyo z*m+Z+x$&E)sA#$Zi|C1EY0)$g?hlb#e95SLwO_HwP7J`z5E6OXT?>=wGYQ)#Y6LRc zy-^fG&llLI_Pl{OPcwkkCB=5(#TvWVWX%78rB6sT1t;VSwsm0N0W=(YMSy6vB=a=( z5a5bRwkJv!a(@lJCVJx%gE3A5W+(=X>9A&I1eIZU>QVREykdoD$M{7>H0u0awCG_M zjmPwr3Hc>&I3e5dQ(iF&)z(bljQb8!-r&~H${g#Np#RW)2>>FFlKlKb{t2>@ic0;I zGY4Y5cL^_H9dWjV8Rk4=o^rA6kujw$8At&rp9Q_IucsjKO7tD}IE-oO5u zjt~0nsf2095I0!1Fm6BVb9d8uU3^;kBNtHbDgebc;^p2PC`yUc5X>v6y+s5urPGx5Ho37v5WK`J`TplOH3v8hf3LG$HCOz zBxItlrBt*VwsrYvy_MEbQf(K{L>HliyI-{D*y0yuG z5J5qzbfhCq2SEYFOsJ7g=tvPn5Tr+>HxWi~P&!fsq$*tq!2}2jf}o*DH4FqCsz{M0 zQp|T46;Cxu-8K1>H(AMh&e><5eV+a7^SnE$%qvMqxryq9^f?$C8EerwB%+g6 zJ)o<*P(=~7q_64vrMS0+l{!onpac=al*@wpSoqxBe8Evc_3J}mt8wZn(8-At)ChL5 z!M_ZrG3cAqFr~7%2p=kAYR8G?#i)3L!c~pK90_y$ys8_h3JsMGo!dcq*|k%j;%c0S zFfO0-c*6^#a_Toe6dU_M?CUwK>~iQh>!l7!)^<68)%=L1C`kE64DaOQ?N6~hN(gYI zp6t-{>@|WRj>jn{2BE+P;wEivsGB!{0`Nd8^k@&mvGe!UUHxM#M3-u?q6Hz>7m|Ft zZfUkl#+pS)iqQ%g0ss()qvs04QP<7sVl8Lc-uCDzOeEr;K7GpVBC%pKN4Y8GV>a#U z(fGcIdS{ZX8?_L#+`2>a5#WsX*Iv@1E5`0+bn6^Nm<E;+N*MNv!sffzn z?#WQ_?y#W3syHsJE&=H>jB5tj{v-1MRxY#7t>h2{sO0h^j#qP=POMOl)Aa=xRgaE3(I(~r2qN-F+R@{TM;B|9tRhyV)jz2G{pKZlS5=Ib zN~>|mCuwo9#_9$tT%}F5-Y~9_QLHTV#Dj6mBER}FHk@xx5Z?KGK%1kp1Lr~Nu&alh zD8=~0)q4H%1qJfGI#Vc8%bIzGo;=kd=wpHG!>Qp1pr2T~eDW<7SpCb>7;wb2@v7tc z@zRYOsc0|t-{IqIuEIj-Q-*5n0>)gid|Ge8D6}CwvPp~j_Xi8Argkh{%H~7K9fj34{S**fj@Zv76ie*{FaanjRD6&20KHI1 zCY>}Ynnnn*3tAoe%FpN&hO4m>W<#Q>{3|ho&El|9Jm2b*c9Dzcs>;uR0eCWX4IflY zdFxZ~XijU0WAn;2fK!eG6N^_@q#AuEVKfyW3zuT>wuWxqr=D|2H`b(&Nx+$}+RR2H zHvQICbwYDZ!CG2eYrXDh5Z=x8PZDLXga5}>v%u(ZJLBt<>u#ac!Al8Q+QjyDoB+$A z%*zSyTCG>_(|?Di|G15~pzRl_^ATR@^$jjxN~idIN5fu<%YNce&JoJ)K~vWP>>Xuf zYI@-s*LdRTOtq_YyX@i~Ma8c|EqQcX*T-k=Fk$_rjB=-bc;W+BV1}xqM_F9~^UtQx zvzrbv9o-=@%KJIH-8*QChtg-Du>q} z_pyXOC?;BN!EB)wRV{?O;v};o6@p)h5%pZPSquJp`7@hP)?dQy9{UN=9}R{}W}i!4 zx;(D%3og;@B*f|@F}0**CvQ&+(j&Tq^81e;?z;lqPfwHIDT@HwQ6^CcPvh;kZ;;MO z2?^c~vV_+I`+m6hXHD}k59<=+_6|6w-dWi6idiiVm*A-PH6w$!s_D|thO-lc&dMY^ zlC#;+E7y@T2~=pqRAyx+saE-#oK@}h)MNjHW&3+RAFG+(};uh#hPho>ACl!dy zuPVzYELl!3%m)5!&JKhnvguoEe>pYLCI(@$4Wn|p+eYoF68t+fHlL7j>{d0YTLKZhcUs@bdN+%v~dVR@}gU zAqQl&Zel>CA?~*gvWDW)!Clwoe+M0Z6)(m6a$OUy5tb-W56U5N)Gx2>FfMy7e!CJ< z4c#(BnAvlQ5#^gFTd{jXhe~&0-EAC~#@jkPTY08{!m$5ceE@bDU`VBOc~`H7ttG9S zA6oD6Ezkp6E2w<(^uZ(C)6)Si{Ux3zC1zuQ=6tRIatClCWutaZYvEm0>*JD=l2cQn zS%FVwy~Z`M>1nhEvityHqpx;kaLg>+X1DeroN;JpDXggInrrZ1b#??o#`$%-%nrc! zDWD2R&kWIzw?6LEgP_``f0cW0ay(h}iSL2V zTTbx}N>@QRKG5W_ZJ)M@VJ#p&+&&xRsy~BGZyZDeY2{Siaz}AopfaF!6Oxi9s-2CO zA+taTir(z2z@cZ}iybc=FDHmWN`)#?0?W$Ee3Umo%1*tHk%{$FbR?#+B9&cma)gUD z#Uj;p1$tTY=eAdRzP62>-{&bXc78~vhn@PK`*ayXLD9?2n~ReI)R=#^=|3`oIk5`g zabXvBr>tQK3h8IueSpYsIs<6e;;aFEI#ahsa?uSAZt$Kh#i~c<@+O)|OG(}6rH_ne z8eSF79_IackLf+l5pEB=Ox-?zlkj&EK2i40Ecc>sz1oDVO+rRn!a2qA<4uJS!g z(F$D(r9!{B!?71S;C0kM@iQ)IU02&;E2@(=-hXcGW*~SrKn5T+6)rY|^lgA*P`}V8 z)I!3z40rZzMH0Tmg4()L6a$}+J*c*KJaBwyZh!U(^?A?ts@S1W$S)bz_5OY9mtZqs z0@QGiLz2u2QLd=Xn6&prRz@zCa^*<)K=N3~p3kWs0;?`hx=DRE*lPaXI8q;w2_QM4 zr6Ot3xQE@^;KeF#w#<9OaBGGGGNezOhz`_o)o!&{1e#PHj;ku}y@9mfQr>TQ@0Fbn6v+RD*y}(V z9O$lGgtEY4XosoXd&*wYxM=|mSCs+NEu@7oZd;8(<`UK|a|o$p(JSp-qoU(GJ~^o# z-^n>b!xuOzlHEs0k(3q}_jFO24ZHnS$j`d~)wC4t>~vvcI@e9gRd|TzFz0x1-uNb1 zopmGMixz)3&o2&2O$jgF{D@`X%Xq1-0-bcuT%*l5f*P^tmRKIH8EaFi$yZTp1$A|Q z0xlrHY1ARbYO>$fZK$TS2Y=H&q@dEcV6Dv z15*W5M+*{i4dqnT1VBH&+#b+Uoz04lKGZtPTRvkX%jevn(zML3gN>hdCv4XcWcT%j zQ8T)+QcoAz?!ltjLcTzSAVNwvEz z?ydv)fv{<+7k-uc=%60cUqOjb6s6?(*V`7>4GhbuEPv()W7xwmh7+`(^ebw{VWb6LSNDZVs;-FT^z z_@_?{6P@s*Yjh;Wa6?ctbwD;N^O(#3&K(mN@WJmT$bjX6z8ZToI5>D}kcPLOSN0%~PUSO)<2xgdRwJYav!&t&3pFqqmX_5R{RHkzNh3O3 zBVxGTqDwLWq_KP3?Cuqn1sv%h11W7!@qd-=d_iKSMC9xdvm(5+1_~+$XZkgiPWVVd z-oz6c?I-=-TrpdV0p%P=HdFpLW3H!@&t5^5DpC5z)sv{@vJM|Fx=n@FPtgu6(0XkO z4CiZtAQf+4ci)#l?i*d+-B43H3z(za`QnJvccv1}1GU9|EXCa}|7;lle)q4L{yPVG xNJ=_$-~jX(QuQ~Ze{K2S+4BFH{i=%^DUdsOqH|ET-q-=~LaOPimLY7z{{za@RGa_+ literal 0 HcmV?d00001 diff --git a/docs/source/api.rst b/docs/source/api.rst new file mode 100644 index 000000000..7ff7dc520 --- /dev/null +++ b/docs/source/api.rst @@ -0,0 +1,28 @@ +API +=== + +.. currentmodule:: dask_cuda + +**dask_cuda** + +.. autosummary:: + dask_cuda + + +Setup +------ + +.. autosummary:: + initialize + +.. autofunction:: initialize + + +Cluster +------- + +.. autosummary:: + LocalCUDACluster + +.. autoclass:: LocalCUDACluster + :members: diff --git a/docs/source/conf.py b/docs/source/conf.py new file mode 100644 index 000000000..70483769b --- /dev/null +++ b/docs/source/conf.py @@ -0,0 +1,182 @@ +# -*- coding: utf-8 -*- +# +# Configuration file for the Sphinx documentation builder. +# +# This file does only contain a selection of the most common options. For a +# full list see the documentation: +# http://www.sphinx-doc.org/en/master/config + +# -- Path setup -------------------------------------------------------------- + +# If extensions (or modules to document with autodoc) are in another directory, +# add these directories to sys.path here. If the directory is relative to the +# documentation root, use os.path.abspath to make it absolute, like shown here. +# +# import os +# import sys +# sys.path.insert(0, os.path.abspath('.')) + + +# -- Project information ----------------------------------------------------- + +project = "dask-cuda" +copyright = "2020, NVIDIA" +author = "NVIDIA" + +# The short X.Y version +version = "0.14" +# The full version, including alpha/beta/rc tags +release = "0.14.0" + + +# -- General configuration --------------------------------------------------- + +# If your documentation needs a minimal Sphinx version, state it here. +# +# needs_sphinx = '1.0' + +# Add any Sphinx extension module names here, as strings. They can be +# extensions coming with Sphinx (named 'sphinx.ext.*') or your custom +# ones. +extensions = [ + "sphinx.ext.autodoc", + "sphinx.ext.mathjax", + "sphinx.ext.viewcode", + "sphinx.ext.githubpages", + "sphinx.ext.autosummary", + "sphinx.ext.intersphinx", + "sphinx.ext.extlinks", +] + +# Add any paths that contain templates here, relative to this directory. +templates_path = ["_templates"] + +# The suffix(es) of source filenames. +# You can specify multiple suffix as a list of string: +# +# source_suffix = ['.rst', '.md'] +source_suffix = ".rst" + +# The master toctree document. +master_doc = "index" + +# The language for content autogenerated by Sphinx. Refer to documentation +# for a list of supported languages. +# +# This is also used if you do content translation via gettext catalogs. +# Usually you set "language" from the command line for these cases. +language = None + +# List of patterns, relative to source directory, that match files and +# directories to ignore when looking for source files. +# This pattern also affects html_static_path and html_extra_path. +exclude_patterns = [] + +# The name of the Pygments (syntax highlighting) style to use. +pygments_style = None + + +# -- Options for HTML output ------------------------------------------------- + +# The theme to use for HTML and HTML Help pages. See the documentation for +# a list of builtin themes. +# +html_theme = "sphinx_rtd_theme" + +# Theme options are theme-specific and customize the look and feel of a theme +# further. For a list of options available for each theme, see the +# documentation. +# +# html_theme_options = {} + +# Add any paths that contain custom static files (such as style sheets) here, +# relative to this directory. They are copied after the builtin static files, +# so a file named "default.css" will overwrite the builtin "default.css". +html_static_path = ["_static"] + +# Custom sidebar templates, must be a dictionary that maps document names +# to template names. +# +# The default sidebars (for documents that don't match any pattern) are +# defined by theme itself. Builtin themes are using these templates by +# default: ``['localtoc.html', 'relations.html', 'sourcelink.html', +# 'searchbox.html']``. +# +# html_sidebars = {} + + +# -- Options for HTMLHelp output --------------------------------------------- + +# Output file base name for HTML help builder. +htmlhelp_basename = "dask-cudadoc" + + +# -- Options for LaTeX output ------------------------------------------------ + +latex_elements = { + # The paper size ('letterpaper' or 'a4paper'). + # + # 'papersize': 'letterpaper', + # The font size ('10pt', '11pt' or '12pt'). + # + # 'pointsize': '10pt', + # Additional stuff for the LaTeX preamble. + # + # 'preamble': '', + # Latex figure (float) alignment + # + # 'figure_align': 'htbp', +} + +# Grouping the document tree into LaTeX files. List of tuples +# (source start file, target name, title, +# author, documentclass [howto, manual, or own class]). +latex_documents = [ + (master_doc, "dask-cuda.tex", "dask-cuda Documentation", "NVIDIA", "manual") +] + + +# -- Options for manual page output ------------------------------------------ + +# One entry per manual page. List of tuples +# (source start file, name, description, authors, manual section). +man_pages = [(master_doc, "dask-cuda", "dask-cuda Documentation", [author], 1)] + + +# -- Options for Texinfo output ---------------------------------------------- + +# Grouping the document tree into Texinfo files. List of tuples +# (source start file, target name, title, author, +# dir menu entry, description, category) +texinfo_documents = [ + ( + master_doc, + "dask-cuda", + "dask-cuda Documentation", + author, + "dask-cuda", + "One line description of project.", + "Miscellaneous", + ) +] + + +# -- Options for Epub output ------------------------------------------------- + +# Bibliographic Dublin Core info. +epub_title = project + +# The unique identifier of the text. This can be a ISBN number +# or the project homepage. +# +# epub_identifier = '' + +# A unique identification for the text. +# +# epub_uid = '' + +# A list of files that should not be packed into the epub file. +epub_exclude_files = ["search.html"] + + +# -- Extension configuration ------------------------------------------------- diff --git a/docs/source/index.rst b/docs/source/index.rst new file mode 100644 index 000000000..e11cf2b48 --- /dev/null +++ b/docs/source/index.rst @@ -0,0 +1,14 @@ +Dask-CUDA +====== + +Various utilities to improve deployment and management of Dask workers on CUDA-enabled systems. + + + +.. toctree:: + :maxdepth: 1 + :hidden: + + quickstart + worker + api diff --git a/docs/source/install.rst b/docs/source/install.rst new file mode 100644 index 000000000..3c23192d3 --- /dev/null +++ b/docs/source/install.rst @@ -0,0 +1,91 @@ +Install +======= + +Prerequisites +------------- + +UCX depends on the following system libraries being present: ``libibcm``, +``libibverbs``, ``librdmacm``, and ``libnuma`` (``numactl`` on Enterprise +Linux). Please install these with your Linux system's package manager. When +building from source you will also need the ``*-dev`` (``*-devel`` on +Enterprise Linux) packages as well. + +Conda +----- + +Some preliminary Conda packages can be installed as so. Replace +```` with either ``9.2``, ``10.0``, or ``10.1``. These are +available both on ``rapidsai`` and ``rapidsai-nightly``. + +With GPU support: + +:: + + conda create -n ucx -c conda-forge -c rapidsai \ + cudatoolkit= ucx-proc=*=gpu ucx ucx-py python=3.7 + +Without GPU support: + +:: + + conda create -n ucx -c conda-forge -c rapidsai \ + ucx-proc=*=cpu ucx ucx-py python=3.7 + +Note: These use UCX's ``v1.7.x`` branch. + +Source +------ + +The following instructions assume you'll be using ucx-py on a CUDA enabled system and is in a `Conda environment `_. + + +Build Dependencies +~~~~~~~~~~~~~~~~~~ + +:: + + conda create -n ucx -c conda-forge \ + automake make libtool pkg-config \ + libhwloc \ + python=3.7 setuptools cython>=0.29.14,<3.0.0a0 + +Test Dependencies +~~~~~~~~~~~~~~~~~ + +:: + + conda install -n ucx -c rapidsai -c nvidia -c conda-forge \ + pytest pytest-asyncio \ + cupy numba>=0.46 rmm \ + distributed + +UCX +~~~ + +:: + + conda activate ucx + git clone https://github.com/openucx/ucx + cd ucx + git checkout v1.7.x + ./autogen.sh + mkdir build + cd build + # Performance build + ../contrib/configure-release --prefix=$CONDA_PREFIX --with-cuda=$CUDA_HOME --enable-mt CPPFLAGS="-I/$CUDA_HOME/include" + # Debug build + ../contrib/configure-devel --prefix=$CONDA_PREFIX --with-cuda=$CUDA_HOME --enable-mt CPPFLAGS="-I/$CUDA_HOME/include" + make -j install + +UCX-Py +~~~~~~ + +:: + + conda activate ucx + git clone git@github.com:rapidsai/ucx-py.git + cd ucx-py + python setup.py build_ext --inplace + pip install . + # or for develop build + pip install -v -e . diff --git a/docs/source/quickstart.rst b/docs/source/quickstart.rst new file mode 100644 index 000000000..431065f7b --- /dev/null +++ b/docs/source/quickstart.rst @@ -0,0 +1,27 @@ +Quickstart +========== + + +Setup +----- + + +:: + + conda create -n dask-cuda -c rapidsai -c nvidia -c conda-forge \ + cudatoolkit= cudf dask-cuda distributed python=3.7 + +Creating a Dask-CUDA Cluster +---------------------------- + +Notebook +~~~~~~~~ + +.. code-block:: python + + from dask_cuda import LocalCUDACluster + from dask.distributed import Client + + # Create a Dask Cluster with one worker per GPU + cluster = LocalCUDACluster() + client = Client(cluster) \ No newline at end of file diff --git a/docs/source/worker.rst b/docs/source/worker.rst new file mode 100644 index 000000000..590f0ea78 --- /dev/null +++ b/docs/source/worker.rst @@ -0,0 +1,106 @@ +Worker +====== + +Dask-CUDA workers extend the standard Dask worker in two ways: + +1) Advanced networking configuration +2) GPU Memory Pool configuration + +These configurations can be defined in the single cluster use case with ``LocalCUDACluster`` or passed to workers on the cli with ``dask-cuda-worker`` + +Single Cluster configuration +---------------------------- + + +Command Line Tool +----------------- + +New configuration options:: + + --device-memory-limit + --rmm-pool-size + --enable-tcp-over-ucx / --disable-tcp-over-ucx + -enable-infiniband / --disable-infiniband + --enable-nvlink / --disable-nvlink + --net-devices + +Full details ``dask-cuda-worker`` options +:: + + $ dask-cuda-worker --help + Options: + --tls-ca-file PATH CA cert(s) file for TLS (in PEM format) + --tls-cert PATH certificate file for TLS (in PEM format) + --tls-key PATH private key file for TLS (in PEM format) + --dashboard-address TEXT dashboard address + --dashboard / --no-dashboard Launch dashboard [default: True] + --host TEXT Serving host. Should be an ip address that + is visible to the scheduler and other + workers. See --listen-address and --contact- + address if you need different listen and + contact addresses. See --interface. + + --interface TEXT The external interface used to connect to + the scheduler, usually an ethernet interface + is used for connection, and not an + InfiniBand interface (if one is available). + + --nthreads INTEGER Number of threads per process. + --name TEXT A unique name for this worker like + 'worker-1'. If used with --nprocs then the + process number will be appended like name-0, + name-1, name-2, ... + + --memory-limit TEXT Bytes of memory per process that the worker + can use. This can be an integer (bytes), + float (fraction of total system memory), + string (like 5GB or 5000M), 'auto', or zero + for no memory management + + --device-memory-limit TEXT Bytes of memory per CUDA device that the + worker can use. This can be an integer + (bytes), float (fraction of total device + memory), string (like 5GB or 5000M), 'auto', + or zero for no memory management (i.e., + allow full device memory usage). + + --rmm-pool-size TEXT If specified, initialize each worker with an + RMM pool of the given size, otherwise no RMM + pool is created. This can be an integer + (bytes) or string (like 5GB or 5000M). + + --reconnect / --no-reconnect Reconnect to scheduler if disconnected + --pid-file TEXT File to write the process PID + --local-directory TEXT Directory to place worker files + --resources TEXT Resources for task constraints like "GPU=2 + MEM=10e9". Resources are applied separately + to each worker process (only relevant when + starting multiple worker processes with '-- + nprocs'). + + --scheduler-file TEXT Filename to JSON encoded scheduler + information. Use with dask-scheduler + --scheduler-file + + --death-timeout TEXT Seconds to wait for a scheduler before + closing + + --dashboard-prefix TEXT Prefix for the Dashboard + --preload TEXT Module that should be loaded by each worker + process like "foo.bar" or "/path/to/foo.py" + + --enable-tcp-over-ucx / --disable-tcp-over-ucx + Enable TCP communication over UCX + --enable-infiniband / --disable-infiniband + Enable InfiniBand communication + --enable-nvlink / --disable-nvlink + Enable NVLink communication + --net-devices TEXT When None (default), 'UCX_NET_DEVICES' will + be left to its default. Otherwise, it must + be a non-empty string with the interface + name. Normally used only with --enable- + infiniband to specify the interface to be + used by the worker, such as 'mlx5_0:1' or + 'ib0'. + + --help Show this message and exit. \ No newline at end of file From be427c35f58a3b5f030b8934ce005faa191ffc8c Mon Sep 17 00:00:00 2001 From: Benjamin Zaitlen Date: Tue, 5 May 2020 15:28:24 -0400 Subject: [PATCH 38/67] remove install details --- docs/source/install.rst | 91 ----------------------------------------- 1 file changed, 91 deletions(-) delete mode 100644 docs/source/install.rst diff --git a/docs/source/install.rst b/docs/source/install.rst deleted file mode 100644 index 3c23192d3..000000000 --- a/docs/source/install.rst +++ /dev/null @@ -1,91 +0,0 @@ -Install -======= - -Prerequisites -------------- - -UCX depends on the following system libraries being present: ``libibcm``, -``libibverbs``, ``librdmacm``, and ``libnuma`` (``numactl`` on Enterprise -Linux). Please install these with your Linux system's package manager. When -building from source you will also need the ``*-dev`` (``*-devel`` on -Enterprise Linux) packages as well. - -Conda ------ - -Some preliminary Conda packages can be installed as so. Replace -```` with either ``9.2``, ``10.0``, or ``10.1``. These are -available both on ``rapidsai`` and ``rapidsai-nightly``. - -With GPU support: - -:: - - conda create -n ucx -c conda-forge -c rapidsai \ - cudatoolkit= ucx-proc=*=gpu ucx ucx-py python=3.7 - -Without GPU support: - -:: - - conda create -n ucx -c conda-forge -c rapidsai \ - ucx-proc=*=cpu ucx ucx-py python=3.7 - -Note: These use UCX's ``v1.7.x`` branch. - -Source ------- - -The following instructions assume you'll be using ucx-py on a CUDA enabled system and is in a `Conda environment `_. - - -Build Dependencies -~~~~~~~~~~~~~~~~~~ - -:: - - conda create -n ucx -c conda-forge \ - automake make libtool pkg-config \ - libhwloc \ - python=3.7 setuptools cython>=0.29.14,<3.0.0a0 - -Test Dependencies -~~~~~~~~~~~~~~~~~ - -:: - - conda install -n ucx -c rapidsai -c nvidia -c conda-forge \ - pytest pytest-asyncio \ - cupy numba>=0.46 rmm \ - distributed - -UCX -~~~ - -:: - - conda activate ucx - git clone https://github.com/openucx/ucx - cd ucx - git checkout v1.7.x - ./autogen.sh - mkdir build - cd build - # Performance build - ../contrib/configure-release --prefix=$CONDA_PREFIX --with-cuda=$CUDA_HOME --enable-mt CPPFLAGS="-I/$CUDA_HOME/include" - # Debug build - ../contrib/configure-devel --prefix=$CONDA_PREFIX --with-cuda=$CUDA_HOME --enable-mt CPPFLAGS="-I/$CUDA_HOME/include" - make -j install - -UCX-Py -~~~~~~ - -:: - - conda activate ucx - git clone git@github.com:rapidsai/ucx-py.git - cd ucx-py - python setup.py build_ext --inplace - pip install . - # or for develop build - pip install -v -e . From f968e8d8768e16b846907e7395cbdf199174737f Mon Sep 17 00:00:00 2001 From: Benjamin Zaitlen Date: Tue, 5 May 2020 15:30:16 -0400 Subject: [PATCH 39/67] RTD setup --- readthedocs.yml | 9 +++++++++ 1 file changed, 9 insertions(+) create mode 100644 readthedocs.yml diff --git a/readthedocs.yml b/readthedocs.yml new file mode 100644 index 000000000..b11377da6 --- /dev/null +++ b/readthedocs.yml @@ -0,0 +1,9 @@ +build: + image: latest + +python: + version: 3.7 + setup_py_install: true + +conda: + file: conda/environments/builddocs_py37.yml From 312ddd71270ec8adcbc43fe670fb312611d3f63f Mon Sep 17 00:00:00 2001 From: Benjamin Zaitlen Date: Tue, 5 May 2020 15:53:15 -0400 Subject: [PATCH 40/67] updates --- docs/source/api.rst | 4 -- docs/source/index.rst | 2 +- docs/source/worker.rst | 88 +++++++++++++++--------------------------- 3 files changed, 32 insertions(+), 62 deletions(-) diff --git a/docs/source/api.rst b/docs/source/api.rst index 7ff7dc520..ea8bfc903 100644 --- a/docs/source/api.rst +++ b/docs/source/api.rst @@ -5,10 +5,6 @@ API **dask_cuda** -.. autosummary:: - dask_cuda - - Setup ------ diff --git a/docs/source/index.rst b/docs/source/index.rst index e11cf2b48..ec472f93e 100644 --- a/docs/source/index.rst +++ b/docs/source/index.rst @@ -1,7 +1,7 @@ Dask-CUDA ====== -Various utilities to improve deployment and management of Dask workers on CUDA-enabled systems. +Dask-CUDA is tool for using `Dask `_ on GPUs. It extends Dask's `Single-Machine Cluster `_ and `Workers `_ for optimized distributed GPU workloads. diff --git a/docs/source/worker.rst b/docs/source/worker.rst index 590f0ea78..3f679a5ed 100644 --- a/docs/source/worker.rst +++ b/docs/source/worker.rst @@ -10,6 +10,37 @@ These configurations can be defined in the single cluster use case with ``LocalC Single Cluster configuration ---------------------------- +Dask-CUDA can be configured for single machine clusters with multiple GPUs such as as DGX1 or DGX2. Below is an example of configuring a single machine Dask cluster on a DGX2 with an RMM pool and NVLink enabled + +.. code-block:: python + + from dask.distributed import Client + from dask_cuda import LocalCUDACluster + from dask_cuda.initialize import initialize + + # Configurations + protocol = "ucx" + interface = "enp134s0f1" # DGX-2 + enable_tcp_over_ucx = True + enable_nvlink = True + enable_infiniband = False + + initialize( + create_cuda_context=True, + enable_tcp_over_ucx=enable_tcp_over_ucx, + enable_infiniband=enable_infiniband, + enable_nvlink=enable_nvlink, + ) + + cluster = LocalCUDACluster(local_directory="/tmp/USERNAME", + protocol=protocol, + interface=interface, + enable_tcp_over_ucx=enable_tcp_over_ucx, + enable_infiniband=enable_infiniband, + enable_nvlink=enable_nvlink, + rmm_pool="25GB", + ) + client = Client(cluster) Command Line Tool @@ -17,46 +48,11 @@ Command Line Tool New configuration options:: - --device-memory-limit - --rmm-pool-size - --enable-tcp-over-ucx / --disable-tcp-over-ucx - -enable-infiniband / --disable-infiniband - --enable-nvlink / --disable-nvlink - --net-devices - -Full details ``dask-cuda-worker`` options -:: - - $ dask-cuda-worker --help - Options: - --tls-ca-file PATH CA cert(s) file for TLS (in PEM format) - --tls-cert PATH certificate file for TLS (in PEM format) - --tls-key PATH private key file for TLS (in PEM format) - --dashboard-address TEXT dashboard address - --dashboard / --no-dashboard Launch dashboard [default: True] - --host TEXT Serving host. Should be an ip address that - is visible to the scheduler and other - workers. See --listen-address and --contact- - address if you need different listen and - contact addresses. See --interface. - --interface TEXT The external interface used to connect to the scheduler, usually an ethernet interface is used for connection, and not an InfiniBand interface (if one is available). - --nthreads INTEGER Number of threads per process. - --name TEXT A unique name for this worker like - 'worker-1'. If used with --nprocs then the - process number will be appended like name-0, - name-1, name-2, ... - - --memory-limit TEXT Bytes of memory per process that the worker - can use. This can be an integer (bytes), - float (fraction of total system memory), - string (like 5GB or 5000M), 'auto', or zero - for no memory management - --device-memory-limit TEXT Bytes of memory per CUDA device that the worker can use. This can be an integer (bytes), float (fraction of total device @@ -69,26 +65,6 @@ Full details ``dask-cuda-worker`` options pool is created. This can be an integer (bytes) or string (like 5GB or 5000M). - --reconnect / --no-reconnect Reconnect to scheduler if disconnected - --pid-file TEXT File to write the process PID - --local-directory TEXT Directory to place worker files - --resources TEXT Resources for task constraints like "GPU=2 - MEM=10e9". Resources are applied separately - to each worker process (only relevant when - starting multiple worker processes with '-- - nprocs'). - - --scheduler-file TEXT Filename to JSON encoded scheduler - information. Use with dask-scheduler - --scheduler-file - - --death-timeout TEXT Seconds to wait for a scheduler before - closing - - --dashboard-prefix TEXT Prefix for the Dashboard - --preload TEXT Module that should be loaded by each worker - process like "foo.bar" or "/path/to/foo.py" - --enable-tcp-over-ucx / --disable-tcp-over-ucx Enable TCP communication over UCX --enable-infiniband / --disable-infiniband @@ -102,5 +78,3 @@ Full details ``dask-cuda-worker`` options infiniband to specify the interface to be used by the worker, such as 'mlx5_0:1' or 'ib0'. - - --help Show this message and exit. \ No newline at end of file From a2de3e858223ccbc4075bd30fc8b9676e049f9a4 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Tue, 5 May 2020 14:02:00 -0700 Subject: [PATCH 41/67] Raise ValueError when ucx_net_devices="auto" IB is disabled It's currently dangerous to use ucx_net_devices="auto" for just any setup as it's impossible to predict all possible topologies, the topologically closest device may not be configured or disconnected. --- dask_cuda/utils.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/dask_cuda/utils.py b/dask_cuda/utils.py index e50d8aefc..cc567b1cc 100644 --- a/dask_cuda/utils.py +++ b/dask_cuda/utils.py @@ -202,6 +202,12 @@ def get_ucx_config( net_devices="", cuda_device_index=None, ): + if net_devices == "auto" and enable_infiniband is False: + raise ValueError( + "Using ucx_net_devices='auto' is currently only " + "supported when enable_infiniband=True." + ) + ucx_config = { "tcp": None, "infiniband": None, From 84e250a7dd2adc56ba2598debccdc9d8f4ae3a78 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Tue, 5 May 2020 14:09:11 -0700 Subject: [PATCH 42/67] Tests get_ucx_config raising exception --- dask_cuda/tests/test_utils.py | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/dask_cuda/tests/test_utils.py b/dask_cuda/tests/test_utils.py index 7093e7d94..222517d96 100644 --- a/dask_cuda/tests/test_utils.py +++ b/dask_cuda/tests/test_utils.py @@ -135,12 +135,18 @@ def test_get_ucx_net_devices_auto(): @pytest.mark.parametrize("enable_infiniband", [True, False]) @pytest.mark.parametrize("net_devices", ["eth0", "auto", ""]) def test_get_ucx_config(enable_tcp_over_ucx, enable_infiniband, net_devices): - ucx_config = get_ucx_config( - enable_tcp_over_ucx=enable_tcp_over_ucx, - enable_infiniband=enable_infiniband, - net_devices=net_devices, - cuda_device_index=0, - ) + kwargs = { + "enable_tcp_over_ucx": enable_tcp_over_ucx, + "enable_infiniband": enable_infiniband, + "net_devices": net_devices, + "cuda_device_index": 0, + } + if net_devices == "auto" and enable_infiniband is False: + with pytest.raises(ValueError): + get_ucx_config(**kwargs) + return + else: + ucx_config = get_ucx_config(**kwargs) if enable_tcp_over_ucx is True: assert ucx_config["tcp"] is True From 9e9cacc98532d95804ca2a0f2f120d1f233d25ab Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Tue, 5 May 2020 14:26:28 -0700 Subject: [PATCH 43/67] Update documentation for ucx_net_devices --- dask_cuda/dask_cuda_worker.py | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/dask_cuda/dask_cuda_worker.py b/dask_cuda/dask_cuda_worker.py index 869d85fdc..d2a13db05 100755 --- a/dask_cuda/dask_cuda_worker.py +++ b/dask_cuda/dask_cuda_worker.py @@ -192,9 +192,17 @@ type=str, default=None, help="When None (default), 'UCX_NET_DEVICES' will be left to its default. " - "Otherwise, it must be a non-empty string with the interface name. Normally " - "used only with --enable-infiniband to specify the interface to be used by " - "the worker, such as 'mlx5_0:1' or 'ib0'.", + "Otherwise, it must be a non-empty string with the interface name, such as " + "such as 'eth0' or 'auto' to allow for automatically choosing the closest " + "interface based on the system's topology. Normally used only with " + "--enable-infiniband to specify the interface to be used by the worker, " + "such as 'mlx5_0:1' or 'ib0'. " + "WARNING: 'auto' requires UCX-Py to be installed and compiled with hwloc " + "support. Additionally that will always use the closest interface, and " + "that may cause unexpected errors if that interface is not properly " + "configured or is disconnected, for that reason it's limited to " + "InfiniBand only and will still cause unpredictable errors if not _ALL_ " + "interfaces are connected and properly configured." ) def main( scheduler, From f868e55f7855e4e3019e0976a5f3c8f3a808099e Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Tue, 5 May 2020 14:51:39 -0700 Subject: [PATCH 44/67] Update LocalCUDACluster documentation for ucx_net_devices --- dask_cuda/local_cuda_cluster.py | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/dask_cuda/local_cuda_cluster.py b/dask_cuda/local_cuda_cluster.py index 7e40b6d6a..da29919c2 100644 --- a/dask_cuda/local_cuda_cluster.py +++ b/dask_cuda/local_cuda_cluster.py @@ -95,10 +95,12 @@ class LocalCUDACluster(LocalCluster): with the interface name, such as "eth0" or "auto" to allow for automatically choosing the closest interface based on the system's topology. - WARNING: "auto" requires UCX-Py to be installed and compiled with hwloc - support, and it will always use the closest interface which may lead to - unexpected errors if that interface is not properly configured or is - disconnected. + WARNING: 'auto' requires UCX-Py to be installed and compiled with hwloc + support. Additionally that will always use the closest interface, and + that may cause unexpected errors if that interface is not properly + configured or is disconnected, for that reason it's limited to + InfiniBand only and will still cause unpredictable errors if not _ALL_ + interfaces are connected and properly configured. rmm_pool: None, int or str When None (default), no RMM pool is initialized. If a different value is given, it can be an integer (bytes) or string (like 5GB or 5000M)." @@ -115,8 +117,9 @@ class LocalCUDACluster(LocalCluster): TypeError If enable_infiniband or enable_nvlink is True and protocol is not 'ucx' ValueError - If ucx_net_devices is an empty string or if it is "auto" and UCX-Py is - not installed or wasn't compiled with hwloc support. + If ucx_net_devices is an empty string, or if it is "auto" and UCX-Py is + not installed, or if it is "auto" and enable_infiniband=False, or UCX-Py + wasn't compiled with hwloc support. See Also -------- From 7d5bf845602a67325bd4faa4e3c07166f5c827c0 Mon Sep 17 00:00:00 2001 From: Benjamin Zaitlen Date: Wed, 6 May 2020 08:03:44 -0700 Subject: [PATCH 45/67] use rapids-dev-doc --- conda/environments/builddocs_py37.yml | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/conda/environments/builddocs_py37.yml b/conda/environments/builddocs_py37.yml index 9526681cb..32e6282f7 100644 --- a/conda/environments/builddocs_py37.yml +++ b/conda/environments/builddocs_py37.yml @@ -3,12 +3,4 @@ channels: - rapidsai-nightly - conda-forge dependencies: -# required for building docs -- sphinx -- sphinx_rtd_theme -- sphinxcontrib-websupport -- nbsphinx -- numpydoc -- recommonmark -- pandoc=<2.0.0 -- pip \ No newline at end of file +- rapids-doc-env From 8ffc9ab45e8cae69854ee7a47d3622703d98a7d4 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Thu, 7 May 2020 02:44:47 -0700 Subject: [PATCH 46/67] Add multi-node support to CuPy benchmark --- .../benchmarks/local_cupy_transpose_sum.py | 127 ++++++++++++++---- 1 file changed, 102 insertions(+), 25 deletions(-) diff --git a/dask_cuda/benchmarks/local_cupy_transpose_sum.py b/dask_cuda/benchmarks/local_cupy_transpose_sum.py index cb498d482..72357cf94 100644 --- a/dask_cuda/benchmarks/local_cupy_transpose_sum.py +++ b/dask_cuda/benchmarks/local_cupy_transpose_sum.py @@ -4,7 +4,7 @@ from time import perf_counter as clock import dask.array as da -from dask.distributed import Client, performance_report, wait +from dask.distributed import Client, SSHCluster, performance_report, wait from dask.utils import format_bytes, format_time, parse_bytes from dask_cuda.local_cuda_cluster import LocalCUDACluster @@ -12,20 +12,67 @@ import numpy as np +def get_scheduler_workers(dask_scheduler=None): + return dask_scheduler.workers + + async def run(args): + if args.multi_node is True: + Cluster = SSHCluster + cluster_args = [args.hosts.split(",")] + scheduler_addr = args.protocol + "://" + cluster_args[0][0] + ":8786" + + worker_options = {} + + # This looks counterintuitive but adding the variable name with + # an empty string is how we can enable CLI booleans currently, + # note that SSHCluster uses the dask-cuda-worker CLI. + if args.enable_tcp_over_ucx: + worker_options["enable_tcp_over_ucx"] = "" + if args.enable_nvlink: + worker_options["enable_nvlink"] = "" + if args.enable_infiniband: + worker_options["enable_infiniband"] = "" + + if args.ucx_net_devices: + worker_options["ucx_net_devices"] = args.ucx_net_devices + + cluster_kwargs = { + "connect_options": {"known_hosts": None}, + "scheduler_options": {"protocol": args.protocol}, + "worker_module": "dask_cuda.dask_cuda_worker", + "worker_options": worker_options, + "asynchronous": True, + # "n_workers": len(args.devs.split(",")), + # "CUDA_VISIBLE_DEVICES": args.devs, + } + else: + Cluster = LocalCUDACluster + cluster_args = [] + cluster_kwargs = { + "protocol": args.protocol, + "n_workers": len(args.devs.split(",")), + "CUDA_VISIBLE_DEVICES": args.devs, + "ucx_net_devices": args.ucx_net_devices, + "enable_tcp_over_ucx": args.enable_tcp_over_ucx, + "enable_infiniband": args.enable_infiniband, + "enable_nvlink": args.enable_nvlink, + "asynchronous": True, + } - # Set up workers on the local machine - async with LocalCUDACluster( - protocol=args.protocol, - n_workers=len(args.devs.split(",")), - CUDA_VISIBLE_DEVICES=args.devs, - ucx_net_devices=args.ucx_net_devices, - enable_tcp_over_ucx=args.enable_tcp_over_ucx, - enable_infiniband=args.enable_infiniband, - enable_nvlink=args.enable_nvlink, - asynchronous=True, - ) as cluster: - async with Client(cluster, asynchronous=True) as client: + async with Cluster(*cluster_args, **cluster_kwargs) as cluster: + # Use the scheduler address with an SSHCluster rather than the cluster + # object, otherwise we can't shut it down. + async with Client( + scheduler_addr if args.multi_node else cluster, asynchronous=True + ) as client: + print("Client connected", client) + + import time + + time.sleep(5) + scheduler_workers = await client.run_on_scheduler(get_scheduler_workers) + print("Client connected", client) def _worker_setup(size=None): import rmm @@ -69,20 +116,17 @@ def _worker_setup(size=None): if d["total"] >= args.ignore_size: bandwidths[k, d["who"]].append(d["bandwidth"]) total_nbytes[k, d["who"]].append(d["total"]) + bandwidths = { - ( - cluster.scheduler.workers[w1].name, - cluster.scheduler.workers[w2].name, - ): [ + (scheduler_workers[w1].name, scheduler_workers[w2].name,): [ "%s/s" % format_bytes(x) for x in np.quantile(v, [0.25, 0.50, 0.75]) ] for (w1, w2), v in bandwidths.items() } total_nbytes = { - ( - cluster.scheduler.workers[w1].name, - cluster.scheduler.workers[w2].name, - ): format_bytes(sum(nb)) + (scheduler_workers[w1].name, scheduler_workers[w2].name,): format_bytes( + sum(nb) + ) for (w1, w2), nb in total_nbytes.items() } @@ -100,10 +144,17 @@ def _worker_setup(size=None): print("(w1,w2) | 25% 50% 75% (total nbytes)") print("--------------------------") for (d1, d2), bw in sorted(bandwidths.items()): - print( - "(%02d,%02d) | %s %s %s (%s)" - % (d1, d2, bw[0], bw[1], bw[2], total_nbytes[(d1, d2)]) + fmt = ( + "(%s,%s) | %s %s %s (%s)" + if args.multi_node + else "(%02d,%02d) | %s %s %s (%s)" ) + print(fmt % (d1, d2, bw[0], bw[1], bw[2], total_nbytes[(d1, d2)])) + + # An SSHCluster will not automatically shut down, we have to + # ensure it does. + if args.multi_node: + await client.shutdown() def parse_args(): @@ -197,14 +248,40 @@ def parse_args(): help="The device to be used for UCX communication, or 'auto'. " "Ignored if protocol is 'tcp'", ) + parser.add_argument( + "--single-node", + action="store_true", + dest="multi_node", + help="Runs a single-node cluster on the current host.", + ) + parser.add_argument( + "--multi-node", + action="store_true", + dest="multi_node", + help="Runs a multi-node cluster on the hosts specified by --hosts.", + ) + parser.add_argument( + "--hosts", + default=None, + type=str, + help="Specifies a comma-separated list of IP addresses or hostnames. " + "The list begins with the host where the scheduler will be launched " + "followed by any number of workers, with a minimum of 1 worker. " + "Requires --multi-node, ignored otherwise.", + ) parser.set_defaults( enable_tcp_over_ucx=True, enable_infiniband=True, enable_nvlink=True ) args = parser.parse_args() + if args.protocol == "tcp": args.enable_tcp_over_ucx = False - args.enable_infinibank = False + args.enable_infiniband = False args.enable_nvlink = False + + if args.multi_node and len(args.hosts.split(",")) < 2: + raise ValueError("--multi-node requires at least 2 hosts") + return args From 8c33658a41e10f6728177efc0289dd6673fedc66 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Thu, 7 May 2020 05:35:23 -0700 Subject: [PATCH 47/67] Add benchmarks utils functions --- dask_cuda/benchmarks/__init__.py | 0 dask_cuda/benchmarks/utils.py | 185 +++++++++++++++++++++++++++++++ 2 files changed, 185 insertions(+) create mode 100644 dask_cuda/benchmarks/__init__.py create mode 100644 dask_cuda/benchmarks/utils.py diff --git a/dask_cuda/benchmarks/__init__.py b/dask_cuda/benchmarks/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/dask_cuda/benchmarks/utils.py b/dask_cuda/benchmarks/utils.py new file mode 100644 index 000000000..0401a3a01 --- /dev/null +++ b/dask_cuda/benchmarks/utils.py @@ -0,0 +1,185 @@ +import argparse + +from dask.distributed import SSHCluster +from dask_cuda.local_cuda_cluster import LocalCUDACluster + + +def parse_benchmark_args(description="Generic dask-cuda Benchmark", args_list=[]): + parser = argparse.ArgumentParser( + description=description + ) + parser.add_argument( + "-d", "--devs", default="0", type=str, help='GPU devices to use (default "0").' + ) + parser.add_argument( + "-p", + "--protocol", + choices=["tcp", "ucx"], + default="tcp", + type=str, + help="The communication protocol to use.", + ) + parser.add_argument( + "--profile", + metavar="PATH", + default=None, + type=str, + help="Write dask profile report (E.g. dask-report.html)", + ) + parser.add_argument( + "--no-rmm-pool", action="store_true", help="Disable the RMM memory pool" + ) + parser.add_argument( + "--enable-tcp-over-ucx", + action="store_true", + dest="enable_tcp_over_ucx", + help="Enable tcp over ucx.", + ) + parser.add_argument( + "--enable-infiniband", + action="store_true", + dest="enable_infiniband", + help="Enable infiniband over ucx.", + ) + parser.add_argument( + "--enable-nvlink", + action="store_true", + dest="enable_nvlink", + help="Enable NVLink over ucx.", + ) + parser.add_argument( + "--disable-tcp-over-ucx", + action="store_false", + dest="enable_tcp_over_ucx", + help="Disable tcp over ucx.", + ) + parser.add_argument( + "--disable-infiniband", + action="store_false", + dest="enable_infiniband", + help="Disable infiniband over ucx.", + ) + parser.add_argument( + "--disable-nvlink", + action="store_false", + dest="enable_nvlink", + help="Disable NVLink over ucx.", + ) + parser.add_argument( + "--ucx-net-devices", + default=None, + type=str, + help="The device to be used for UCX communication, or 'auto'. " + "Ignored if protocol is 'tcp'", + ) + parser.add_argument( + "--single-node", + action="store_true", + dest="multi_node", + help="Runs a single-node cluster on the current host.", + ) + parser.add_argument( + "--multi-node", + action="store_true", + dest="multi_node", + help="Runs a multi-node cluster on the hosts specified by --hosts.", + ) + parser.add_argument( + "--hosts", + default=None, + type=str, + help="Specifies a comma-separated list of IP addresses or hostnames. " + "The list begins with the host where the scheduler will be launched " + "followed by any number of workers, with a minimum of 1 worker. " + "Requires --multi-node, ignored otherwise.", + ) + + for args in args_list: + name = args.pop("name") + if not isinstance(name, list): + name = [name] + parser.add_argument(*name, **args) + + parser.set_defaults( + enable_tcp_over_ucx=True, enable_infiniband=True, enable_nvlink=True + ) + args = parser.parse_args() + + if args.protocol == "tcp": + args.enable_tcp_over_ucx = False + args.enable_infiniband = False + args.enable_nvlink = False + + if args.multi_node and len(args.hosts.split(",")) < 2: + raise ValueError("--multi-node requires at least 2 hosts") + + return args + + +def get_cluster_options(args): + if args.multi_node is True: + Cluster = SSHCluster + cluster_args = [args.hosts.split(",")] + scheduler_addr = args.protocol + "://" + cluster_args[0][0] + ":8786" + + worker_options = {} + + # This looks counterintuitive but adding the variable name with + # an empty string is how we can enable CLI booleans currently, + # note that SSHCluster uses the dask-cuda-worker CLI. + if args.enable_tcp_over_ucx: + worker_options["enable_tcp_over_ucx"] = "" + if args.enable_nvlink: + worker_options["enable_nvlink"] = "" + if args.enable_infiniband: + worker_options["enable_infiniband"] = "" + + if args.ucx_net_devices: + worker_options["ucx_net_devices"] = args.ucx_net_devices + + cluster_kwargs = { + "connect_options": {"known_hosts": None}, + "scheduler_options": {"protocol": args.protocol}, + "worker_module": "dask_cuda.dask_cuda_worker", + "worker_options": worker_options, + "asynchronous": True, + # "n_workers": len(args.devs.split(",")), + # "CUDA_VISIBLE_DEVICES": args.devs, + } + else: + Cluster = LocalCUDACluster + scheduler_addr = None + cluster_args = [] + cluster_kwargs = { + "protocol": args.protocol, + "n_workers": len(args.devs.split(",")), + "CUDA_VISIBLE_DEVICES": args.devs, + "ucx_net_devices": args.ucx_net_devices, + "enable_tcp_over_ucx": args.enable_tcp_over_ucx, + "enable_infiniband": args.enable_infiniband, + "enable_nvlink": args.enable_nvlink, + "asynchronous": True, + } + + return { + "class": Cluster, + "args": cluster_args, + "kwargs": cluster_kwargs, + "scheduler_addr": scheduler_addr, + } + + +def get_scheduler_workers(dask_scheduler=None): + return dask_scheduler.workers + + +def setup_memory_pool(pool_size=None, disable_pool=False): + import rmm + import cupy + + rmm.reinitialize( + pool_allocator=not disable_pool, + devices=0, + initial_pool_size=pool_size, + ) + cupy.cuda.set_allocator(rmm.rmm_cupy_allocator) From d9b1812bb2656fa3b85a63462bda2df98df05e87 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Thu, 7 May 2020 07:49:31 -0700 Subject: [PATCH 48/67] Use benchmarks.utils to simplify CuPy benchmark --- .../benchmarks/local_cupy_transpose_sum.py | 228 ++++-------------- 1 file changed, 41 insertions(+), 187 deletions(-) diff --git a/dask_cuda/benchmarks/local_cupy_transpose_sum.py b/dask_cuda/benchmarks/local_cupy_transpose_sum.py index 72357cf94..39ca4c24b 100644 --- a/dask_cuda/benchmarks/local_cupy_transpose_sum.py +++ b/dask_cuda/benchmarks/local_cupy_transpose_sum.py @@ -11,54 +11,20 @@ import cupy import numpy as np - -def get_scheduler_workers(dask_scheduler=None): - return dask_scheduler.workers +from dask_cuda.benchmarks.utils import ( + get_cluster_options, + get_scheduler_workers, + parse_benchmark_args, + setup_memory_pool, +) async def run(args): - if args.multi_node is True: - Cluster = SSHCluster - cluster_args = [args.hosts.split(",")] - scheduler_addr = args.protocol + "://" + cluster_args[0][0] + ":8786" - - worker_options = {} - - # This looks counterintuitive but adding the variable name with - # an empty string is how we can enable CLI booleans currently, - # note that SSHCluster uses the dask-cuda-worker CLI. - if args.enable_tcp_over_ucx: - worker_options["enable_tcp_over_ucx"] = "" - if args.enable_nvlink: - worker_options["enable_nvlink"] = "" - if args.enable_infiniband: - worker_options["enable_infiniband"] = "" - - if args.ucx_net_devices: - worker_options["ucx_net_devices"] = args.ucx_net_devices - - cluster_kwargs = { - "connect_options": {"known_hosts": None}, - "scheduler_options": {"protocol": args.protocol}, - "worker_module": "dask_cuda.dask_cuda_worker", - "worker_options": worker_options, - "asynchronous": True, - # "n_workers": len(args.devs.split(",")), - # "CUDA_VISIBLE_DEVICES": args.devs, - } - else: - Cluster = LocalCUDACluster - cluster_args = [] - cluster_kwargs = { - "protocol": args.protocol, - "n_workers": len(args.devs.split(",")), - "CUDA_VISIBLE_DEVICES": args.devs, - "ucx_net_devices": args.ucx_net_devices, - "enable_tcp_over_ucx": args.enable_tcp_over_ucx, - "enable_infiniband": args.enable_infiniband, - "enable_nvlink": args.enable_nvlink, - "asynchronous": True, - } + cluster_options = get_cluster_options(args) + Cluster = cluster_options["class"] + cluster_args = cluster_options["args"] + cluster_kwargs = cluster_options["kwargs"] + scheduler_addr = cluster_options["scheduler_addr"] async with Cluster(*cluster_args, **cluster_kwargs) as cluster: # Use the scheduler address with an SSHCluster rather than the cluster @@ -66,28 +32,14 @@ async def run(args): async with Client( scheduler_addr if args.multi_node else cluster, asynchronous=True ) as client: - print("Client connected", client) - - import time - - time.sleep(5) scheduler_workers = await client.run_on_scheduler(get_scheduler_workers) - print("Client connected", client) - - def _worker_setup(size=None): - import rmm - rmm.reinitialize( - pool_allocator=not args.no_rmm_pool, - devices=0, - initial_pool_size=size, - ) - cupy.cuda.set_allocator(rmm.rmm_cupy_allocator) - - await client.run(_worker_setup) + await client.run(setup_memory_pool, disable_pool=args.no_rmm_pool) # Create an RMM pool on the scheduler due to occasional deserialization # of CUDA objects. May cause issues with InfiniBand otherwise. - await client.run_on_scheduler(_worker_setup, 1e9) + await client.run_on_scheduler( + setup_memory_pool, 1e9, disable_pool=args.no_rmm_pool + ) # Create a simple random array rs = da.random.RandomState(RandomState=cupy.random.RandomState) @@ -158,131 +110,33 @@ def _worker_setup(size=None): def parse_args(): - parser = argparse.ArgumentParser( - description="Transpose on LocalCUDACluster benchmark" - ) - parser.add_argument( - "-d", "--devs", default="0", type=str, help='GPU devices to use (default "0").' - ) - parser.add_argument( - "-p", - "--protocol", - choices=["tcp", "ucx"], - default="tcp", - type=str, - help="The communication protocol to use.", - ) - parser.add_argument( - "-s", - "--size", - default="10000", - metavar="n", - type=int, - help="The size n in n^2 (default 10000)", - ) - parser.add_argument( - "-c", - "--chunk-size", - default="128 MiB", - metavar="nbytes", - type=str, - help='Chunk size (default "128 MiB")', - ) - parser.add_argument( - "--ignore-size", - default="1 MiB", - metavar="nbytes", - type=parse_bytes, - help='Ignore messages smaller than this (default "1 MB")', + special_args = [ + { + "name": ["-s", "--size",], + "default": "10000", + "metavar": "n", + "type": int, + "help": "The size n in n^2 (default 10000)", + }, + { + "name": ["-c", "--chunk-size",], + "default": "128 MiB", + "metavar": "nbytes", + "type": str, + "help": "Chunk size (default '128 MiB')", + }, + { + "name": "--ignore-size", + "default": "1 MiB", + "metavar": "nbytes", + "type": parse_bytes, + "help": "Ignore messages smaller than this (default '1 MB')", + }, + ] + + return parse_benchmark_args( + description="Transpose on LocalCUDACluster benchmark", args_list=special_args ) - parser.add_argument( - "--profile", - metavar="PATH", - default=None, - type=str, - help="Write dask profile report (E.g. dask-report.html)", - ) - parser.add_argument( - "--no-rmm-pool", action="store_true", help="Disable the RMM memory pool" - ) - parser.add_argument( - "--enable-tcp-over-ucx", - action="store_true", - dest="enable_tcp_over_ucx", - help="Enable tcp over ucx.", - ) - parser.add_argument( - "--enable-infiniband", - action="store_true", - dest="enable_infiniband", - help="Enable infiniband over ucx.", - ) - parser.add_argument( - "--enable-nvlink", - action="store_true", - dest="enable_nvlink", - help="Enable NVLink over ucx.", - ) - parser.add_argument( - "--disable-tcp-over-ucx", - action="store_false", - dest="enable_tcp_over_ucx", - help="Disable tcp over ucx.", - ) - parser.add_argument( - "--disable-infiniband", - action="store_false", - dest="enable_infiniband", - help="Disable infiniband over ucx.", - ) - parser.add_argument( - "--disable-nvlink", - action="store_false", - dest="enable_nvlink", - help="Disable NVLink over ucx.", - ) - parser.add_argument( - "--ucx-net-devices", - default=None, - type=str, - help="The device to be used for UCX communication, or 'auto'. " - "Ignored if protocol is 'tcp'", - ) - parser.add_argument( - "--single-node", - action="store_true", - dest="multi_node", - help="Runs a single-node cluster on the current host.", - ) - parser.add_argument( - "--multi-node", - action="store_true", - dest="multi_node", - help="Runs a multi-node cluster on the hosts specified by --hosts.", - ) - parser.add_argument( - "--hosts", - default=None, - type=str, - help="Specifies a comma-separated list of IP addresses or hostnames. " - "The list begins with the host where the scheduler will be launched " - "followed by any number of workers, with a minimum of 1 worker. " - "Requires --multi-node, ignored otherwise.", - ) - parser.set_defaults( - enable_tcp_over_ucx=True, enable_infiniband=True, enable_nvlink=True - ) - args = parser.parse_args() - - if args.protocol == "tcp": - args.enable_tcp_over_ucx = False - args.enable_infiniband = False - args.enable_nvlink = False - - if args.multi_node and len(args.hosts.split(",")) < 2: - raise ValueError("--multi-node requires at least 2 hosts") - - return args def main(): From af1e45fcb44d9b8258c22ee6f9f8f87028b510ac Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Thu, 7 May 2020 10:42:44 -0700 Subject: [PATCH 49/67] Remove asynchronous from benchmarks.utils.get_cluster_options --- dask_cuda/benchmarks/local_cupy_transpose_sum.py | 2 +- dask_cuda/benchmarks/utils.py | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/dask_cuda/benchmarks/local_cupy_transpose_sum.py b/dask_cuda/benchmarks/local_cupy_transpose_sum.py index 39ca4c24b..d7f3ff7ff 100644 --- a/dask_cuda/benchmarks/local_cupy_transpose_sum.py +++ b/dask_cuda/benchmarks/local_cupy_transpose_sum.py @@ -26,7 +26,7 @@ async def run(args): cluster_kwargs = cluster_options["kwargs"] scheduler_addr = cluster_options["scheduler_addr"] - async with Cluster(*cluster_args, **cluster_kwargs) as cluster: + async with Cluster(*cluster_args, **cluster_kwargs, asynchronous=True) as cluster: # Use the scheduler address with an SSHCluster rather than the cluster # object, otherwise we can't shut it down. async with Client( diff --git a/dask_cuda/benchmarks/utils.py b/dask_cuda/benchmarks/utils.py index 0401a3a01..6579959da 100644 --- a/dask_cuda/benchmarks/utils.py +++ b/dask_cuda/benchmarks/utils.py @@ -142,7 +142,6 @@ def get_cluster_options(args): "scheduler_options": {"protocol": args.protocol}, "worker_module": "dask_cuda.dask_cuda_worker", "worker_options": worker_options, - "asynchronous": True, # "n_workers": len(args.devs.split(",")), # "CUDA_VISIBLE_DEVICES": args.devs, } @@ -158,7 +157,6 @@ def get_cluster_options(args): "enable_tcp_over_ucx": args.enable_tcp_over_ucx, "enable_infiniband": args.enable_infiniband, "enable_nvlink": args.enable_nvlink, - "asynchronous": True, } return { From 12f39a5d78528cf3b180cd53585ecacfbef226d5 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Thu, 7 May 2020 11:09:12 -0700 Subject: [PATCH 50/67] Use benchmarks.utils to simplify cuDF benchmark --- dask_cuda/benchmarks/local_cudf_merge.py | 253 ++++++++--------------- 1 file changed, 84 insertions(+), 169 deletions(-) diff --git a/dask_cuda/benchmarks/local_cudf_merge.py b/dask_cuda/benchmarks/local_cudf_merge.py index 45a2b28fa..983e3a74c 100644 --- a/dask_cuda/benchmarks/local_cudf_merge.py +++ b/dask_cuda/benchmarks/local_cudf_merge.py @@ -7,7 +7,6 @@ from dask.dataframe.core import new_dd_object from dask.distributed import Client, performance_report, wait from dask.utils import format_bytes, format_time, parse_bytes -from dask_cuda import LocalCUDACluster from dask_cuda.initialize import initialize from dask_cuda import explicit_comms @@ -15,6 +14,13 @@ import cupy import numpy +from dask_cuda.benchmarks.utils import ( + get_cluster_options, + get_scheduler_workers, + parse_benchmark_args, + setup_memory_pool, +) + # Benchmarking cuDF merge operation based on # @@ -142,17 +148,17 @@ def merge_explicit_comms(args, ddf1, ddf2): return took -def run(client, args, write_profile=None): +def run(client, args, n_workers, write_profile=None): # Generate random Dask dataframes ddf_base = get_random_ddf( - args.chunk_size, args.n_workers, args.frac_match, "build", args + args.chunk_size, n_workers, args.frac_match, "build", args ).persist() ddf_other = get_random_ddf( - args.chunk_size, args.n_workers, args.frac_match, "other", args + args.chunk_size, n_workers, args.frac_match, "other", args ).persist() wait(ddf_base) wait(ddf_other) - client.wait_for_workers(args.n_workers) + client.wait_for_workers(n_workers) assert len(ddf_base.dtypes) == 2 assert len(ddf_other.dtypes) == 2 @@ -168,54 +174,28 @@ def run(client, args, write_profile=None): def main(args): - # Set up workers on the local machine - if args.protocol == "tcp": - cluster = LocalCUDACluster( - protocol=args.protocol, - n_workers=args.n_workers, - CUDA_VISIBLE_DEVICES=args.devs, - ) - else: - enable_infiniband = args.enable_infiniband - enable_nvlink = args.enable_nvlink - enable_tcp_over_ucx = args.enable_tcp_over_ucx - cluster = LocalCUDACluster( - protocol=args.protocol, - n_workers=args.n_workers, - CUDA_VISIBLE_DEVICES=args.devs, - ucx_net_devices=args.ucx_net_devices, - enable_tcp_over_ucx=enable_tcp_over_ucx, - enable_infiniband=enable_infiniband, - enable_nvlink=enable_nvlink, - ) - initialize( - create_cuda_context=True, - enable_tcp_over_ucx=enable_tcp_over_ucx, - enable_infiniband=enable_infiniband, - enable_nvlink=enable_nvlink, - ) - client = Client(cluster) - - def _worker_setup(initial_pool_size=None): - import rmm + cluster_options = get_cluster_options(args) + Cluster = cluster_options["class"] + cluster_args = cluster_options["args"] + cluster_kwargs = cluster_options["kwargs"] + scheduler_addr = cluster_options["scheduler_addr"] - rmm.reinitialize( - pool_allocator=not args.no_rmm_pool, - devices=0, - initial_pool_size=initial_pool_size, - ) - cupy.cuda.set_allocator(rmm.rmm_cupy_allocator) + cluster = Cluster(*cluster_args, **cluster_kwargs) + client = Client(scheduler_addr if args.multi_node else cluster) - client.run(_worker_setup) + client.run(setup_memory_pool, disable_pool=args.no_rmm_pool) # Create an RMM pool on the scheduler due to occasional deserialization # of CUDA objects. May cause issues with InfiniBand otherwise. - client.run_on_scheduler(_worker_setup, 1e9) + client.run_on_scheduler(setup_memory_pool, 1e9, disable_pool=args.no_rmm_pool) + + scheduler_workers = client.run_on_scheduler(get_scheduler_workers) + n_workers = len(scheduler_workers) took_list = [] for _ in range(args.runs - 1): - took_list.append(run(client, args, write_profile=None)) + took_list.append(run(client, args, n_workers, write_profile=None)) took_list.append( - run(client, args, write_profile=args.profile) + run(client, args, n_workers, write_profile=args.profile) ) # Only profiling the last run # Collect, aggregate, and print peer-to-peer bandwidths @@ -228,16 +208,13 @@ def _worker_setup(initial_pool_size=None): bandwidths[k, d["who"]].append(d["bandwidth"]) total_nbytes[k, d["who"]].append(d["total"]) bandwidths = { - (cluster.scheduler.workers[w1].name, cluster.scheduler.workers[w2].name): [ + (scheduler_workers[w1].name, scheduler_workers[w2].name): [ "%s/s" % format_bytes(x) for x in numpy.quantile(v, [0.25, 0.50, 0.75]) ] for (w1, w2), v in bandwidths.items() } total_nbytes = { - ( - cluster.scheduler.workers[w1].name, - cluster.scheduler.workers[w2].name, - ): format_bytes(sum(nb)) + (scheduler_workers[w1].name, scheduler_workers[w2].name,): format_bytes(sum(nb)) for (w1, w2), nb in total_nbytes.items() } @@ -274,132 +251,70 @@ def _worker_setup(initial_pool_size=None): print("(w1,w2) | 25% 50% 75% (total nbytes)") print("-------------------------------") for (d1, d2), bw in sorted(bandwidths.items()): - print( - "(%02d,%02d) | %s %s %s (%s)" - % (d1, d2, bw[0], bw[1], bw[2], total_nbytes[(d1, d2)]) + fmt = ( + "(%s,%s) | %s %s %s (%s)" + if args.multi_node + else "(%02d,%02d) | %s %s %s (%s)" ) + print(fmt % (d1, d2, bw[0], bw[1], bw[2], total_nbytes[(d1, d2)])) if args.markdown: print("```\n\n") + if args.multi_node: + client.shutdown() + client.close() + def parse_args(): - parser = argparse.ArgumentParser( - description="Merge (dask/cudf) on LocalCUDACluster benchmark" - ) - parser.add_argument( - "-d", "--devs", default="0", type=str, help='GPU devices to use (default "0").' - ) - parser.add_argument( - "-p", - "--protocol", - choices=["tcp", "ucx"], - default="tcp", - type=str, - help="The communication protocol to use.", - ) - parser.add_argument( - "-b", - "--backend", - choices=["dask", "explicit-comms"], - default="dask", - type=str, - help="The backend to use.", - ) - parser.add_argument( - "-c", - "--chunk-size", - default=1_000_000, - metavar="n", - type=int, - help="Chunk size (default 1_000_000)", - ) - parser.add_argument( - "--ignore-size", - default="1 MiB", - metavar="nbytes", - type=parse_bytes, - help='Ignore messages smaller than this (default "1 MB")', - ) - parser.add_argument( - "--frac-match", - default=0.3, - type=float, - help="Fraction of rows that matches (default 0.3)", - ) - parser.add_argument( - "--no-rmm-pool", action="store_true", help="Disable the RMM memory pool" - ) - parser.add_argument( - "--profile", - metavar="PATH", - default=None, - type=str, - help="Write dask profile report (E.g. dask-report.html)", - ) - parser.add_argument( - "--no-shuffle", - action="store_true", - help="Don't shuffle the keys of the left (base) dataframe.", - ) - parser.add_argument( - "--markdown", action="store_true", help="Write output as markdown" - ) - parser.add_argument( - "-s", - "--set-index", - action="store_true", - help="Call set_index on the key column to sort the joined dataframe.", - ) - parser.add_argument("--runs", default=3, type=int, help="Number of runs") - parser.add_argument( - "--enable-tcp-over-ucx", - action="store_true", - dest="enable_tcp_over_ucx", - help="Enable tcp over ucx.", - ) - parser.add_argument( - "--enable-infiniband", - action="store_true", - dest="enable_infiniband", - help="Enable infiniband over ucx.", - ) - parser.add_argument( - "--enable-nvlink", - action="store_true", - dest="enable_nvlink", - help="Enable NVLink over ucx.", - ) - parser.add_argument( - "--disable-tcp-over-ucx", - action="store_false", - dest="enable_tcp_over_ucx", - help="Disable tcp over ucx.", - ) - parser.add_argument( - "--disable-infiniband", - action="store_false", - dest="enable_infiniband", - help="Disable infiniband over ucx.", - ) - parser.add_argument( - "--disable-nvlink", - action="store_false", - dest="enable_nvlink", - help="Disable NVLink over ucx.", - ) - parser.add_argument( - "--ucx-net-devices", - default=None, - type=str, - help="The device to be used for UCX communication, or 'auto'. " - "Ignored if protocol is 'tcp'", - ) - parser.set_defaults( - enable_tcp_over_ucx=True, enable_infiniband=True, enable_nvlink=True + special_args = [ + { + "name": ["-b", "--backend",], + "choices": ["dask", "explicit-comms"], + "default": "dask", + "type": str, + "help": "The backend to use.", + }, + { + "name": ["-c", "--chunk-size",], + "default": 1_000_000, + "metavar": "n", + "type": int, + "help": "Chunk size (default 1_000_000)", + }, + { + "name": "--ignore-size", + "default": "1 MiB", + "metavar": "nbytes", + "type": parse_bytes, + "help": "Ignore messages smaller than this (default '1 MB')", + }, + { + "name": "--frac-match", + "default": 0.3, + "type": float, + "help": "Fraction of rows that matches (default 0.3)", + }, + { + "name": "--no-shuffle", + "action": "store_true", + "help": "Don't shuffle the keys of the left (base) dataframe.", + }, + { + "name": "--markdown", + "action": "store_true", + "help": "Write output as markdown", + }, + {"name": "--runs", "default": 3, "type": int, "help": "Number of runs",}, + { + "name": ["-s", "--set-index",], + "action": "store_true", + "help": "Call set_index on the key column to sort the joined dataframe.", + }, + ] + + return parse_benchmark_args( + description="Distributed merge (dask/cudf) benchmark", args_list=special_args ) - args = parser.parse_args() - args.n_workers = len(args.devs.split(",")) - return args if __name__ == "__main__": From 6d835236f0d4ebd3b1c796aaf73757fc969a3a03 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Thu, 7 May 2020 11:21:29 -0700 Subject: [PATCH 51/67] Fix black/isort formatting --- dask_cuda/benchmarks/local_cudf_merge.py | 11 ++++------- dask_cuda/benchmarks/local_cupy_transpose_sum.py | 8 +++----- dask_cuda/benchmarks/utils.py | 8 ++------ 3 files changed, 9 insertions(+), 18 deletions(-) diff --git a/dask_cuda/benchmarks/local_cudf_merge.py b/dask_cuda/benchmarks/local_cudf_merge.py index 983e3a74c..af4a64c51 100644 --- a/dask_cuda/benchmarks/local_cudf_merge.py +++ b/dask_cuda/benchmarks/local_cudf_merge.py @@ -1,4 +1,3 @@ -import argparse import math from collections import defaultdict from time import perf_counter as clock @@ -7,13 +6,7 @@ from dask.dataframe.core import new_dd_object from dask.distributed import Client, performance_report, wait from dask.utils import format_bytes, format_time, parse_bytes -from dask_cuda.initialize import initialize from dask_cuda import explicit_comms - -import cudf -import cupy -import numpy - from dask_cuda.benchmarks.utils import ( get_cluster_options, get_scheduler_workers, @@ -21,6 +14,10 @@ setup_memory_pool, ) +import cudf +import cupy +import numpy + # Benchmarking cuDF merge operation based on # diff --git a/dask_cuda/benchmarks/local_cupy_transpose_sum.py b/dask_cuda/benchmarks/local_cupy_transpose_sum.py index d7f3ff7ff..fac5a78bf 100644 --- a/dask_cuda/benchmarks/local_cupy_transpose_sum.py +++ b/dask_cuda/benchmarks/local_cupy_transpose_sum.py @@ -6,11 +6,6 @@ import dask.array as da from dask.distributed import Client, SSHCluster, performance_report, wait from dask.utils import format_bytes, format_time, parse_bytes -from dask_cuda.local_cuda_cluster import LocalCUDACluster - -import cupy -import numpy as np - from dask_cuda.benchmarks.utils import ( get_cluster_options, get_scheduler_workers, @@ -18,6 +13,9 @@ setup_memory_pool, ) +import cupy +import numpy as np + async def run(args): cluster_options = get_cluster_options(args) diff --git a/dask_cuda/benchmarks/utils.py b/dask_cuda/benchmarks/utils.py index 6579959da..a191a0385 100644 --- a/dask_cuda/benchmarks/utils.py +++ b/dask_cuda/benchmarks/utils.py @@ -5,9 +5,7 @@ def parse_benchmark_args(description="Generic dask-cuda Benchmark", args_list=[]): - parser = argparse.ArgumentParser( - description=description - ) + parser = argparse.ArgumentParser(description=description) parser.add_argument( "-d", "--devs", default="0", type=str, help='GPU devices to use (default "0").' ) @@ -176,8 +174,6 @@ def setup_memory_pool(pool_size=None, disable_pool=False): import cupy rmm.reinitialize( - pool_allocator=not disable_pool, - devices=0, - initial_pool_size=pool_size, + pool_allocator=not disable_pool, devices=0, initial_pool_size=pool_size, ) cupy.cuda.set_allocator(rmm.rmm_cupy_allocator) From c959ab06f220a1f4ce7b157ef48647c8fa45dc4f Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Thu, 7 May 2020 11:27:09 -0700 Subject: [PATCH 52/67] Sleep to give time for benchmark SSHCluster to spin --- dask_cuda/benchmarks/local_cudf_merge.py | 6 ++++++ dask_cuda/benchmarks/local_cupy_transpose_sum.py | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/dask_cuda/benchmarks/local_cudf_merge.py b/dask_cuda/benchmarks/local_cudf_merge.py index af4a64c51..01e785562 100644 --- a/dask_cuda/benchmarks/local_cudf_merge.py +++ b/dask_cuda/benchmarks/local_cudf_merge.py @@ -178,6 +178,12 @@ def main(args): scheduler_addr = cluster_options["scheduler_addr"] cluster = Cluster(*cluster_args, **cluster_kwargs) + if args.multi_node: + import time + # Allow some time for workers to start and connect to scheduler + # TODO: make this a command-line argument? + time.sleep(15) + client = Client(scheduler_addr if args.multi_node else cluster) client.run(setup_memory_pool, disable_pool=args.no_rmm_pool) diff --git a/dask_cuda/benchmarks/local_cupy_transpose_sum.py b/dask_cuda/benchmarks/local_cupy_transpose_sum.py index fac5a78bf..3dd484018 100644 --- a/dask_cuda/benchmarks/local_cupy_transpose_sum.py +++ b/dask_cuda/benchmarks/local_cupy_transpose_sum.py @@ -25,6 +25,12 @@ async def run(args): scheduler_addr = cluster_options["scheduler_addr"] async with Cluster(*cluster_args, **cluster_kwargs, asynchronous=True) as cluster: + if args.multi_node: + import time + # Allow some time for workers to start and connect to scheduler + # TODO: make this a command-line argument? + time.sleep(15) + # Use the scheduler address with an SSHCluster rather than the cluster # object, otherwise we can't shut it down. async with Client( From 189d93f439f1bce937aa7d772963371fc39d35fb Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Thu, 7 May 2020 14:20:59 -0700 Subject: [PATCH 53/67] Add example to benchmark --hosts option. --- dask_cuda/benchmarks/utils.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/dask_cuda/benchmarks/utils.py b/dask_cuda/benchmarks/utils.py index a191a0385..80fea4df3 100644 --- a/dask_cuda/benchmarks/utils.py +++ b/dask_cuda/benchmarks/utils.py @@ -89,7 +89,13 @@ def parse_benchmark_args(description="Generic dask-cuda Benchmark", args_list=[] help="Specifies a comma-separated list of IP addresses or hostnames. " "The list begins with the host where the scheduler will be launched " "followed by any number of workers, with a minimum of 1 worker. " - "Requires --multi-node, ignored otherwise.", + "Requires --multi-node, ignored otherwise. " + "Usage example: --multi-node --hosts 'dgx12,dgx12,10.10.10.10,dgx13' . " + "In the example, the benchmark is launched with scheduler on host " + "'dgx12' (first in the list), and workers on three hosts being 'dgx12', " + "'10.10.10.10', and 'dgx13'. " + "Note: --devs is currently ignored in multi-node mode and for each host " + "one worker per GPU will be launched.", ) for args in args_list: From 503e57360ca5ac8b9375354705582e94aee36545 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Thu, 7 May 2020 15:47:40 -0700 Subject: [PATCH 54/67] Add docs for UCX --- docs/source/index.rst | 1 + docs/source/ucx.rst | 122 ++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 123 insertions(+) create mode 100644 docs/source/ucx.rst diff --git a/docs/source/index.rst b/docs/source/index.rst index ec472f93e..5d9838796 100644 --- a/docs/source/index.rst +++ b/docs/source/index.rst @@ -11,4 +11,5 @@ Dask-CUDA is tool for using `Dask `_ on GPUs. It extends Dask quickstart worker + ucx api diff --git a/docs/source/ucx.rst b/docs/source/ucx.rst new file mode 100644 index 000000000..d6b475e84 --- /dev/null +++ b/docs/source/ucx.rst @@ -0,0 +1,122 @@ +UCX Integration +=============== + +Communication can be a major bottleneck in distributed systems, and that's no different in Dask and Dask-CUDA. To address that, Dask-CUDA provides integration with `UCX-Py `_, a Python interface for the `UCX `_ communication framework. UCX is a low-level library that provides high-performance networking, supporting several transports, including NVLink and InfiniBand for systems that have such specialized hardware, as well as TCP for those that do not. + + +Relevant Settings +----------------- + +In this list below we shall see all the Dask configurations relevant for using with Dask-CUDA currently. Note that these options can also be used with mainline Dask/Distributed if you can't use Dask-CUDA for some reason. Note that many improvments for running GPU code is readily available in Dask-CUDA -- but not in mainline Dask/Distributed -- thus, we recommend using Dask-CUDA when possible. + +- ``DASK_UCX__CUDA_COPY=True`` (default: ``False``): *Always required for UCX*, adds ``cuda_copy`` to ``UCX_TLS`` -- required for all CUDA transfers in UCX, both intra- and inter-node; +- ``DASK_UCX__TCP=True`` (default: ``False``): *Always required for UCX*, adds ``tcp`` to ``UCX_TLS`` -- required for all TCP transfers (e.g., where NVLink or IB is not available/disabled) in UCX, both intra- and inter-node; +- ``DASK_UCX__NVLINK=True`` (default: ``False``): Adds ``cuda_ipc`` to ``UCX_TLS`` -- required for all NVLink transfers in UCX, only affects intra-node. +- ``DASK_UCX__INFINIBAND=True`` (defalt: ``False``): Adds ``rc`` to ``UCX_TLS`` -- required for all InfiniBand transfers in UCX, only affects inter-node. +- ``DASK_UCX__RDMACM=True`` (default: ``False``): Replaces ``sockcm`` by ``rdmacm`` in ``UCX_TLS`` and ``UCX_SOCKADDR_TLS_PRIORITY``. ``rdmacm`` is the recommended method by UCX to use with IB and currently won't work if ``DASK_UCX__INFINIBAND=False``. +- ``DASK_UCX__NET_DEVICES=mlx5_0:1`` (default: ``None``, causes UCX to decide what device to use, possibly being suboptimal, implies ``UCX_NET_DEVICES=all``): this is very important when ``DASK_UCX__INFINIBAND=True`` to ensure the scheduler is connected over the InfiniBand interface. When ``DASK_UCX__INFINIBAND=False`` it's recommended to use the ethernet device instead, e.g., ``DASK_UCX__NET_DEVICES=enp1s0f0`` on a DGX-1. +- ``DASK_RMM__POOL_SIZE=1GB``: allocates an RMM pool for the process. In some circumstances, the Dask scheduler will deserialize CUDA data and cause a crash if there's no pool. + + +Important notes +--------------- + +* CUDA Memory Pool: With UCX, all NVLink and InfiniBand memory buffers have to be mapped from one process to another upon the first request for transfer of that buffer between a single pair of processes. This can be quite costly, consuming up to 100ms only for mapping, plus the transfer time itself. For this reason it is strongly recommended to use a `RAPIDS Memory Manager (RMM) `_ memory pool in such cases, incurring in a single mapping of the pool and all subsequent transfers will not be required to repeat that process. It is recommened to also keep the memory pool size to at least the minimum amount of memory used by the application, if possible one can map all GPU memory to a single pool and utilize that pool for the entire lifetime of the application. + +* Automatic detection of InfiniBand interfaces: it's especially important to note the usage of ``--net-devices="auto"`` in ``dask-cuda-worker``, which will automatically determine the InfiniBand interface that's closest to each GPU. For safety, this option can only be used if ``--enable-infiniband`` is specified. Be warned that this mode assumes all InfiniBand interfaces on the system are connected and properly configured, undefined behavior may occur otherwise. + + +Launching Scheduler, Workers and Clients Separately +--------------------------------------------------- + +The first way for starting a Dask cluster with UCX support is to start each process separately. The processes are ``dask-scheduler``, ``dask-cuda-worker`` and the client process utilizing ``distributed.Client`` that will connect to the cluster. Details follow for each of the processes. + +dask-scheduler +^^^^^^^^^^^^^^ + +The ``dask-scheduler`` has no parameters for UCX configuration -- different from what we will see for ``dask-cuda-worker`` on the next section -- for that reason we rely on Dask environment variables. Here's how to start the scheduler with all transports that are currently supported by Dask-CUDA: + +``DASK_RMM__POOL_SIZE=1GB DASK_UCX__CUDA_COPY=True DASK_UCX__TCP=True DASK_UCX__NVLINK=True DASK_UCX__INFINIBAND=True DASK_UCX__RDMACM=True DASK_UCX__NET_DEVICES=mlx5_0:1 dask-scheduler --protocol ucx --interface ib0`` + +Note above how we use ``DASK_UCX__NET_DEVICES=mlx5_0:1`` (the Mellanox name for ``ib0``) and the same interface with ``--interface ib0``. If the system doesn't have an InfiniBand interface available, you would normally use the main network interface, such as ``eth0``, as seen below: + +``DASK_RMM__POOL_SIZE=1GB DASK_UCX__CUDA_COPY=True DASK_UCX__TCP=True DASK_UCX__NVLINK=True dask-scheduler --protocol ucx --interface eth0`` + +Setting ``DASK_UCX__NET_DEVICES`` when using an interface that isn't an InfiniBand can generally be skipped. + + +dask-cuda-worker +^^^^^^^^^^^^^^^^ + +All ``DASK_*`` configurations described above have analogous parameters in ``dask-cuda-worker`` which are preferred over the regular configurations used for ``dask-scheduler`` due to some specializations, such as ``--net-devices="auto"`` which will correctly assign the topologically closest IB interface to the GPU of each worker, something that's not possible with ``DASK_UCX__NET_DEVICES``. + +- ``--disable-tcp-over-ucx`` (default) is analogous to ``DASK_UCX__TCP=False``, ``--enable-tcp-over-ucx`` is equivalent to ``DASK_UCX__TCP=True``; +- ``--disable-nvlink`` (default) is analogous to ``DASK_UCX__NVLINK=False``, ``--enable-nvlink`` is equivalent to ``DASK_UCX__NVLINK=True``; +- ``--disable-infiniband`` (default) is analogous to ``DASK_UCX__INFINIBAND=False``, ``--enable-infiniband`` is equivalent to ``DASK_UCX__INFINIBAND=True``; +- ``--net-devices`` (default ``None``, implies ``UCX_NET_DEVICES=all``) equivalent to ``DASK_UCX__NET_DEVICES``; +- ``--rmm-pool-size`` equivalent to ``DASK_RMM__POOL_SIZE``. + +Here's how to start workers with all transports that are currently relevant for us: + +``dask-cuda-worker ucx://SCHEDULER_IB0_IP:8786 --enable-tcp-over-ucx --enable-nvlink --enable-infiniband -- enable-rdmacm --net-devices="auto" --rmm-pool-size="30GB"`` + + +client +^^^^^^ + +The same configurations used for the scheduler should be used by the client. One possible exception is ``DASK_RMM__POOL_SIZE``, at this time it's unclear whether this is necessary or not, but using that should not cause any issues nevertheless. + +One can use ``os.environ`` inside the client script, it's important to set them at the very top before importing anything other than ``os``. See example below: + +.. code-block:: python + import os + + os.environ["DASK_RMM__POOL_SIZE"] = "1GB" + os.environ["DASK_UCX__CUDA_COPY"] = "True" # os.environ needs using strings, not Python True/False + os.environ["DASK_UCX__TCP"] = "True" + os.environ["DASK_UCX__NVLINK"] = "True" + os.environ["DASK_UCX__INFINIBAND"] = "True" + os.environ["DASK_UCX__NET_DEVICES"] = "mlx5_0:1" + + from distributed import Client + + client = Client("ucx://SCHEDULER_IB0_IP:8786") # SCHEDULER_IB0_IP must be the IP of ib0 on the node where scheduler runs + + # Client code goes here + + +Starting a local cluster (single-node only) +------------------------------------------- + +All options discussed previously are also available in ``LocalCUDACluster``. It is shown below how to start a local cluster with all UCX capabilities enabled: + +.. code-block:: python + + import os + + # The options here are to be used by the client only, + # inherent options for the Dask scheduler and workers + # have to be passed to LocalCUDACluster + os.environ["DASK_RMM__POOL_SIZE"] = "1GB" + os.environ["DASK_UCX__CUDA_COPY"] = "True" # os.environ needs using strings, not Python True/False + os.environ["DASK_UCX__TCP"] = "True" + os.environ["DASK_UCX__NVLINK"] = "True" + os.environ["DASK_UCX__INFINIBAND"] = "True" + os.environ["DASK_UCX__NET_DEVICES"] = "mlx5_0:1" + + from dask.distributed import Client + from dask_cuda import LocalCUDACluster + from dask_cuda.initialize import initialize + + cluster = LocalCUDACluster( + protocol = "ucx" + interface = "ib0" # Interface -- used for the scheduler + enable_tcp_over_ucx = True + enable_nvlink = True + enable_infiniband = True + ucx_net_devices="auto" + rmm_pool="24GB" + ) + client = Client(cluster) + + # Client code goes here From a1899d74c11ec77470fe11cf0a72c3b215f9d533 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Fri, 8 May 2020 05:01:38 -0700 Subject: [PATCH 55/67] Add Specializations for GPU Usage docs page --- docs/source/index.rst | 1 + docs/source/specializations.rst | 24 ++++++++++++++++++++++++ 2 files changed, 25 insertions(+) create mode 100644 docs/source/specializations.rst diff --git a/docs/source/index.rst b/docs/source/index.rst index 5d9838796..3d53e39cf 100644 --- a/docs/source/index.rst +++ b/docs/source/index.rst @@ -10,6 +10,7 @@ Dask-CUDA is tool for using `Dask `_ on GPUs. It extends Dask :hidden: quickstart + specializations worker ucx api diff --git a/docs/source/specializations.rst b/docs/source/specializations.rst new file mode 100644 index 000000000..5cbcc1389 --- /dev/null +++ b/docs/source/specializations.rst @@ -0,0 +1,24 @@ +Specializations for GPU Usage +============================= + +It is known that main line Dask and Distributed packages can be used to leverage GPU computing, utilizing libraries such as cuDF, CuPy and Numba. So why use Dask-CUDA instead? This section aims to answer this question. + +Automatic Instantiation of One-Worker-Per-GPU +--------------------------------------------- + +Using the ``dask-cuda-worker`` or ``LocalCUDACluster`` will automatically launch one worker for each GPU available on the node where that is running, avoiding the need for users to select GPUs in their application and thus reducing code complexity. + +Spilling From Device +-------------------- + +For applications that do not fit in GPU memory, Dask-CUDA supports spilling from device memory to host memory when the GPU can't fit more data. The spilling mechanism is automatically triggered once the user-defined limit is reached, such limit can be set via the ``--device-memory-limit`` and ``device_memory_limit`` arguments for ``dask-cuda-worker`` and ``LocalCUDACluster``, respectively. + +CPU Affinity +------------ + +To improve performance, setting CPU affinity for each GPU is automatically done, preventing memory transfers from taking sub-optimal paths. + +Automatic Selection of InfiniBand Device +---------------------------------------- + +When UCX is activated, Dask-CUDA is also capable of selecting the topologically closest InfiniBand device to each GPU, thus ensuring optimal path and improving performance even further by using Remote Direct Memory Access (RDMA) when available. From 0e8732e7066f218508999c32d7b10291d48cbe45 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Fri, 8 May 2020 05:17:31 -0700 Subject: [PATCH 56/67] Improve UCX docs text and formatting --- docs/source/ucx.rst | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/docs/source/ucx.rst b/docs/source/ucx.rst index d6b475e84..31170a609 100644 --- a/docs/source/ucx.rst +++ b/docs/source/ucx.rst @@ -1,13 +1,19 @@ UCX Integration =============== -Communication can be a major bottleneck in distributed systems, and that's no different in Dask and Dask-CUDA. To address that, Dask-CUDA provides integration with `UCX-Py `_, a Python interface for the `UCX `_ communication framework. UCX is a low-level library that provides high-performance networking, supporting several transports, including NVLink and InfiniBand for systems that have such specialized hardware, as well as TCP for those that do not. +Communication can be a major bottleneck in distributed systems, and that's no different in Dask and Dask-CUDA. To address that, Dask-CUDA provides integration with `UCX-Py `_, a Python interface for the `UCX `_ communication framework. UCX is a low-level library that provides high-performance networking and supports several transports, including NVLink and InfiniBand for systems that have such specialized hardware, as well as TCP for those that do not. + Relevant Settings ----------------- In this list below we shall see all the Dask configurations relevant for using with Dask-CUDA currently. Note that these options can also be used with mainline Dask/Distributed if you can't use Dask-CUDA for some reason. Note that many improvments for running GPU code is readily available in Dask-CUDA -- but not in mainline Dask/Distributed -- thus, we recommend using Dask-CUDA when possible. +In the list below we describe the relevant Dask configurations for using UCX with Dask-CUDA. + +.. note:: + These options can also be used with mainline Dask/Distributed outside of Dask-CUDA, however, we recommend using Dask-CUDA when possible. See the Specializations for GPU Usage section for details on the benefits of using Dask-CUDA. + - ``DASK_UCX__CUDA_COPY=True`` (default: ``False``): *Always required for UCX*, adds ``cuda_copy`` to ``UCX_TLS`` -- required for all CUDA transfers in UCX, both intra- and inter-node; - ``DASK_UCX__TCP=True`` (default: ``False``): *Always required for UCX*, adds ``tcp`` to ``UCX_TLS`` -- required for all TCP transfers (e.g., where NVLink or IB is not available/disabled) in UCX, both intra- and inter-node; @@ -36,11 +42,13 @@ dask-scheduler The ``dask-scheduler`` has no parameters for UCX configuration -- different from what we will see for ``dask-cuda-worker`` on the next section -- for that reason we rely on Dask environment variables. Here's how to start the scheduler with all transports that are currently supported by Dask-CUDA: -``DASK_RMM__POOL_SIZE=1GB DASK_UCX__CUDA_COPY=True DASK_UCX__TCP=True DASK_UCX__NVLINK=True DASK_UCX__INFINIBAND=True DASK_UCX__RDMACM=True DASK_UCX__NET_DEVICES=mlx5_0:1 dask-scheduler --protocol ucx --interface ib0`` +.. code-block:: bash + DASK_RMM__POOL_SIZE=1GB DASK_UCX__CUDA_COPY=True DASK_UCX__TCP=True DASK_UCX__NVLINK=True DASK_UCX__INFINIBAND=True DASK_UCX__RDMACM=True DASK_UCX__NET_DEVICES=mlx5_0:1 dask-scheduler --protocol ucx --interface ib0 Note above how we use ``DASK_UCX__NET_DEVICES=mlx5_0:1`` (the Mellanox name for ``ib0``) and the same interface with ``--interface ib0``. If the system doesn't have an InfiniBand interface available, you would normally use the main network interface, such as ``eth0``, as seen below: -``DASK_RMM__POOL_SIZE=1GB DASK_UCX__CUDA_COPY=True DASK_UCX__TCP=True DASK_UCX__NVLINK=True dask-scheduler --protocol ucx --interface eth0`` +.. code-block:: bash + DASK_RMM__POOL_SIZE=1GB DASK_UCX__CUDA_COPY=True DASK_UCX__TCP=True DASK_UCX__NVLINK=True dask-scheduler --protocol ucx --interface eth0 Setting ``DASK_UCX__NET_DEVICES`` when using an interface that isn't an InfiniBand can generally be skipped. @@ -58,7 +66,8 @@ All ``DASK_*`` configurations described above have analogous parameters in ``das Here's how to start workers with all transports that are currently relevant for us: -``dask-cuda-worker ucx://SCHEDULER_IB0_IP:8786 --enable-tcp-over-ucx --enable-nvlink --enable-infiniband -- enable-rdmacm --net-devices="auto" --rmm-pool-size="30GB"`` +.. code-block:: bash + dask-cuda-worker ucx://SCHEDULER_IB0_IP:8786 --enable-tcp-over-ucx --enable-nvlink --enable-infiniband -- enable-rdmacm --net-devices="auto" --rmm-pool-size="30GB" client From e4db5ae1535bded4e9248000cf6f6809f06ea305 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Fri, 8 May 2020 06:09:13 -0700 Subject: [PATCH 57/67] Add cross-reference links to docs pages --- docs/source/specializations.rst | 2 +- docs/source/ucx.rst | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/source/specializations.rst b/docs/source/specializations.rst index 5cbcc1389..0ed8cbba4 100644 --- a/docs/source/specializations.rst +++ b/docs/source/specializations.rst @@ -21,4 +21,4 @@ To improve performance, setting CPU affinity for each GPU is automatically done, Automatic Selection of InfiniBand Device ---------------------------------------- -When UCX is activated, Dask-CUDA is also capable of selecting the topologically closest InfiniBand device to each GPU, thus ensuring optimal path and improving performance even further by using Remote Direct Memory Access (RDMA) when available. +When UCX is activated, Dask-CUDA is also capable of selecting the topologically closest InfiniBand device to each GPU, thus ensuring optimal path and improving performance even further by using Remote Direct Memory Access (RDMA) when available. See the :doc:`UCX ` page for more details. diff --git a/docs/source/ucx.rst b/docs/source/ucx.rst index 31170a609..9df9f9fd7 100644 --- a/docs/source/ucx.rst +++ b/docs/source/ucx.rst @@ -12,7 +12,7 @@ In this list below we shall see all the Dask configurations relevant for using w In the list below we describe the relevant Dask configurations for using UCX with Dask-CUDA. .. note:: - These options can also be used with mainline Dask/Distributed outside of Dask-CUDA, however, we recommend using Dask-CUDA when possible. See the Specializations for GPU Usage section for details on the benefits of using Dask-CUDA. + These options can also be used with mainline Dask/Distributed outside of Dask-CUDA, however, we recommend using Dask-CUDA when possible. See the :doc:`Specializations for GPU Usage ` page for details on the benefits of using Dask-CUDA. - ``DASK_UCX__CUDA_COPY=True`` (default: ``False``): *Always required for UCX*, adds ``cuda_copy`` to ``UCX_TLS`` -- required for all CUDA transfers in UCX, both intra- and inter-node; From 8f2e71f65ef517c178563d351d3057c3651cdd38 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Fri, 8 May 2020 12:24:05 -0700 Subject: [PATCH 58/67] Improve specialization docs page text --- docs/source/specializations.rst | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/source/specializations.rst b/docs/source/specializations.rst index 0ed8cbba4..e044e312e 100644 --- a/docs/source/specializations.rst +++ b/docs/source/specializations.rst @@ -6,7 +6,7 @@ It is known that main line Dask and Distributed packages can be used to leverage Automatic Instantiation of One-Worker-Per-GPU --------------------------------------------- -Using the ``dask-cuda-worker`` or ``LocalCUDACluster`` will automatically launch one worker for each GPU available on the node where that is running, avoiding the need for users to select GPUs in their application and thus reducing code complexity. +Using the ``dask-cuda-worker`` or ``LocalCUDACluster`` will automatically launch one worker for each GPU available on the node from where it was executed, avoiding the need for users to select GPUs in their application and thus reducing code complexity. Spilling From Device -------------------- @@ -21,4 +21,4 @@ To improve performance, setting CPU affinity for each GPU is automatically done, Automatic Selection of InfiniBand Device ---------------------------------------- -When UCX is activated, Dask-CUDA is also capable of selecting the topologically closest InfiniBand device to each GPU, thus ensuring optimal path and improving performance even further by using Remote Direct Memory Access (RDMA) when available. See the :doc:`UCX ` page for more details. +When InfiniBand is activated, Dask-CUDA is also capable of selecting the topologically closest InfiniBand device to each GPU, thus ensuring optimal path and improving performance even further by using GPU Remote Direct Memory Access (RDMA) when available. See the :doc:`UCX ` page for more details. From cbd4cda86cae798c8f78eeed1c60848e32017482 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Fri, 8 May 2020 15:26:14 -0700 Subject: [PATCH 59/67] Add `--runs` argument to CuPy benchmark --- .../benchmarks/local_cupy_transpose_sum.py | 47 ++++++++++++------- 1 file changed, 30 insertions(+), 17 deletions(-) diff --git a/dask_cuda/benchmarks/local_cupy_transpose_sum.py b/dask_cuda/benchmarks/local_cupy_transpose_sum.py index 3dd484018..a681f22e0 100644 --- a/dask_cuda/benchmarks/local_cupy_transpose_sum.py +++ b/dask_cuda/benchmarks/local_cupy_transpose_sum.py @@ -17,6 +17,26 @@ import numpy as np +async def _run(client, args): + # Create a simple random array + rs = da.random.RandomState(RandomState=cupy.random.RandomState) + x = rs.random((args.size, args.size), chunks=args.chunk_size).persist() + await wait(x) + + # Execute the operations to benchmark + if args.profile is not None: + async with performance_report(filename=args.profile): + t1 = clock() + await client.compute((x + x.T).sum()) + took = clock() - t1 + else: + t1 = clock() + await client.compute((x + x.T).sum()) + took = clock() - t1 + + return (took, x.npartitions) + + async def run(args): cluster_options = get_cluster_options(args) Cluster = cluster_options["class"] @@ -45,21 +65,9 @@ async def run(args): setup_memory_pool, 1e9, disable_pool=args.no_rmm_pool ) - # Create a simple random array - rs = da.random.RandomState(RandomState=cupy.random.RandomState) - x = rs.random((args.size, args.size), chunks=args.chunk_size).persist() - await wait(x) - - # Execute the operations to benchmark - if args.profile is not None: - async with performance_report(filename=args.profile): - t1 = clock() - await client.compute((x + x.T).sum()) - took = clock() - t1 - else: - t1 = clock() - await client.compute((x + x.T).sum()) - took = clock() - t1 + took_list = [] + for i in range(args.runs): + took_list.append(await _run(client, args)) # Collect, aggregate, and print peer-to-peer bandwidths incoming_logs = await client.run( @@ -93,9 +101,13 @@ async def run(args): print(f"Ignore-size | {format_bytes(args.ignore_size)}") print(f"Protocol | {args.protocol}") print(f"Device(s) | {args.devs}") - print(f"npartitions | {x.npartitions}") print("==========================") - print(f"Total time | {format_time(took)}") + print("Wall-clock | npartitions") + print("--------------------------") + for (took, npartitions) in took_list: + t = format_time(took) + t += " " * (11 - len(t)) + print(f"{t} | {npartitions}") print("==========================") print("(w1,w2) | 25% 50% 75% (total nbytes)") print("--------------------------") @@ -136,6 +148,7 @@ def parse_args(): "type": parse_bytes, "help": "Ignore messages smaller than this (default '1 MB')", }, + {"name": "--runs", "default": 3, "type": int, "help": "Number of runs",}, ] return parse_benchmark_args( From 81faddba89430a9690f052f5d9c383c2207ad877 Mon Sep 17 00:00:00 2001 From: Randy Gelhausen Date: Wed, 13 May 2020 00:25:01 +0000 Subject: [PATCH 60/67] Fixing LocalCUDACluster example. Adding README link to docs --- README.md | 2 ++ docs/source/worker.rst | 4 ++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 0032028eb..872da00fa 100644 --- a/README.md +++ b/README.md @@ -18,6 +18,8 @@ cluster = LocalCUDACluster() client = Client(cluster) ``` +Documentation is available [here](https://dask-cuda.readthedocs.io/). + What this is not ---------------- diff --git a/docs/source/worker.rst b/docs/source/worker.rst index 3f679a5ed..2c2a8baa2 100644 --- a/docs/source/worker.rst +++ b/docs/source/worker.rst @@ -20,7 +20,7 @@ Dask-CUDA can be configured for single machine clusters with multiple GPUs such # Configurations protocol = "ucx" - interface = "enp134s0f1" # DGX-2 + interface = "enp6s0" # DGX-2 enable_tcp_over_ucx = True enable_nvlink = True enable_infiniband = False @@ -38,7 +38,7 @@ Dask-CUDA can be configured for single machine clusters with multiple GPUs such enable_tcp_over_ucx=enable_tcp_over_ucx, enable_infiniband=enable_infiniband, enable_nvlink=enable_nvlink, - rmm_pool="25GB", + rmm_pool_size="25GB", ) client = Client(cluster) From 2e09313e9ea95e6f8e9c8d9dd6110a6e61991417 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Thu, 28 May 2020 12:01:33 -0700 Subject: [PATCH 61/67] Add `nfinal` argument to shuffle_group, required in Dask >= 2.17 --- dask_cuda/explicit_comms/dataframe_merge.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/dask_cuda/explicit_comms/dataframe_merge.py b/dask_cuda/explicit_comms/dataframe_merge.py index 12a4ddeeb..99cb07f25 100644 --- a/dask_cuda/explicit_comms/dataframe_merge.py +++ b/dask_cuda/explicit_comms/dataframe_merge.py @@ -95,7 +95,11 @@ def partition_by_hash(df, columns, n_chunks, ignore_index=False): # Hashing `columns` in `df` and assing it to the "_partitions" column df["_partitions"] = partitioning_index(df[columns], n_chunks) # Split `df` based on the hash values in the "_partitions" column - ret = shuffle_group(df, "_partitions", 0, n_chunks, n_chunks, ignore_index) + try: + # For Dask < 2.17 compatibility + ret = shuffle_group(df, "_partitions", 0, n_chunks, n_chunks, ignore_index) + except TypeError: + ret = shuffle_group(df, "_partitions", 0, n_chunks, n_chunks, ignore_index, n_chunks) # Let's remove the partition column and return the partitions del df["_partitions"] From 2f0dda30412eb5a4612b64bce314a4fc19d94011 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Thu, 28 May 2020 15:33:57 -0700 Subject: [PATCH 62/67] Fix isort formatting --- dask_cuda/device_host_file.py | 3 +-- dask_cuda/explicit_comms/comms.py | 16 ++++++++++------ dask_cuda/explicit_comms/dataframe_merge.py | 9 +++++---- dask_cuda/explicit_comms/utils.py | 5 +++-- dask_cuda/tests/test_dask_cuda_worker.py | 2 +- dask_cuda/tests/test_dgx.py | 11 +++++------ dask_cuda/tests/test_explicit_comms.py | 12 +++++------- dask_cuda/tests/test_utils.py | 2 +- 8 files changed, 31 insertions(+), 29 deletions(-) diff --git a/dask_cuda/device_host_file.py b/dask_cuda/device_host_file.py index fc4f60bd8..2ca738cec 100644 --- a/dask_cuda/device_host_file.py +++ b/dask_cuda/device_host_file.py @@ -1,8 +1,6 @@ import functools import os -import numpy - import dask from distributed.protocol import ( dask_deserialize, @@ -15,6 +13,7 @@ from distributed.utils import nbytes from distributed.worker import weight +import numpy from zict import Buffer, File, Func from zict.common import ZictBase diff --git a/dask_cuda/explicit_comms/comms.py b/dask_cuda/explicit_comms/comms.py index 18c36fd57..4c7063591 100644 --- a/dask_cuda/explicit_comms/comms.py +++ b/dask_cuda/explicit_comms/comms.py @@ -1,15 +1,19 @@ -import time -import uuid import asyncio import concurrent.futures +import time +import uuid -import numpy as np -from distributed import get_worker, default_client, wait -from distributed.comm.addressing import parse_host_port, parse_address, unparse_address import distributed.comm +from distributed import default_client, get_worker, wait +from distributed.comm.addressing import ( + parse_address, + parse_host_port, + unparse_address, +) -from . import utils +import numpy as np +from . import utils _default_comms = None diff --git a/dask_cuda/explicit_comms/dataframe_merge.py b/dask_cuda/explicit_comms/dataframe_merge.py index 99cb07f25..5cd6b039a 100644 --- a/dask_cuda/explicit_comms/dataframe_merge.py +++ b/dask_cuda/explicit_comms/dataframe_merge.py @@ -1,11 +1,12 @@ import asyncio + +from dask.dataframe.shuffle import partitioning_index, shuffle_group +from distributed.protocol import to_serialize + +import cudf import numpy as np import pandas import rmm -import cudf - -from dask.dataframe.shuffle import shuffle_group, partitioning_index -from distributed.protocol import to_serialize from . import comms, utils diff --git a/dask_cuda/explicit_comms/utils.py b/dask_cuda/explicit_comms/utils.py index aa5fb7c28..b914b7923 100644 --- a/dask_cuda/explicit_comms/utils.py +++ b/dask_cuda/explicit_comms/utils.py @@ -1,10 +1,11 @@ from collections import OrderedDict -from tornado import gen -from toolz import first import dask.dataframe as dd from distributed import default_client, wait +from toolz import first +from tornado import gen + def workers_to_parts(futures): """ diff --git a/dask_cuda/tests/test_dask_cuda_worker.py b/dask_cuda/tests/test_dask_cuda_worker.py index 63cec983c..c02975197 100644 --- a/dask_cuda/tests/test_dask_cuda_worker.py +++ b/dask_cuda/tests/test_dask_cuda_worker.py @@ -3,12 +3,12 @@ import os from time import sleep +from dask_cuda.utils import get_gpu_count from distributed import Client from distributed.metrics import time from distributed.system import MEMORY_LIMIT from distributed.utils_test import loop # noqa: F401 from distributed.utils_test import popen -from dask_cuda.utils import get_gpu_count import pytest diff --git a/dask_cuda/tests/test_dgx.py b/dask_cuda/tests/test_dgx.py index 6cee0bb2a..9c2b3b604 100644 --- a/dask_cuda/tests/test_dgx.py +++ b/dask_cuda/tests/test_dgx.py @@ -1,19 +1,18 @@ import multiprocessing as mp import os import subprocess +from time import sleep import dask.array as da from dask_cuda import LocalCUDACluster +from dask_cuda.initialize import initialize +from dask_cuda.utils import get_gpu_count from distributed import Client +from distributed.metrics import time +from distributed.utils import get_ip_interface import numpy import pytest - -from time import sleep -from dask_cuda.utils import get_gpu_count -from dask_cuda.initialize import initialize -from distributed.metrics import time -from distributed.utils import get_ip_interface from tornado.ioloop import IOLoop mp = mp.get_context("spawn") diff --git a/dask_cuda/tests/test_explicit_comms.py b/dask_cuda/tests/test_explicit_comms.py index 4042bad89..e0ab13e09 100644 --- a/dask_cuda/tests/test_explicit_comms.py +++ b/dask_cuda/tests/test_explicit_comms.py @@ -1,17 +1,15 @@ import multiprocessing as mp -import pytest +import dask.dataframe as dd +from dask_cuda.explicit_comms import CommsContext, dataframe_merge from distributed import Client from distributed.deploy.local import LocalCluster -from dask_cuda.explicit_comms import CommsContext, dataframe_merge - -import pandas as pd -import dask.dataframe as dd -import numpy as np -import pytest import cudf import cupy +import numpy as np +import pandas as pd +import pytest mp = mp.get_context("spawn") ucp = pytest.importorskip("ucp") diff --git a/dask_cuda/tests/test_utils.py b/dask_cuda/tests/test_utils.py index 222517d96..a6b3f6279 100644 --- a/dask_cuda/tests/test_utils.py +++ b/dask_cuda/tests/test_utils.py @@ -5,8 +5,8 @@ get_device_total_memory, get_n_gpus, get_preload_options, - get_ucx_net_devices, get_ucx_config, + get_ucx_net_devices, unpack_bitmask, ) From ede89f481e3ee3e84a52b2bb139abc1de7310072 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Thu, 28 May 2020 15:35:22 -0700 Subject: [PATCH 63/67] Fix black formatting --- dask_cuda/benchmarks/local_cudf_merge.py | 1 + .../benchmarks/local_cupy_transpose_sum.py | 1 + dask_cuda/dask_cuda_worker.py | 2 +- dask_cuda/explicit_comms/dataframe_merge.py | 4 +++- dask_cuda/initialize.py | 3 +-- dask_cuda/tests/test_spill.py | 18 +++++++++++++----- 6 files changed, 20 insertions(+), 9 deletions(-) diff --git a/dask_cuda/benchmarks/local_cudf_merge.py b/dask_cuda/benchmarks/local_cudf_merge.py index 01e785562..abbe1191c 100644 --- a/dask_cuda/benchmarks/local_cudf_merge.py +++ b/dask_cuda/benchmarks/local_cudf_merge.py @@ -180,6 +180,7 @@ def main(args): cluster = Cluster(*cluster_args, **cluster_kwargs) if args.multi_node: import time + # Allow some time for workers to start and connect to scheduler # TODO: make this a command-line argument? time.sleep(15) diff --git a/dask_cuda/benchmarks/local_cupy_transpose_sum.py b/dask_cuda/benchmarks/local_cupy_transpose_sum.py index a681f22e0..7d5b8f118 100644 --- a/dask_cuda/benchmarks/local_cupy_transpose_sum.py +++ b/dask_cuda/benchmarks/local_cupy_transpose_sum.py @@ -47,6 +47,7 @@ async def run(args): async with Cluster(*cluster_args, **cluster_kwargs, asynchronous=True) as cluster: if args.multi_node: import time + # Allow some time for workers to start and connect to scheduler # TODO: make this a command-line argument? time.sleep(15) diff --git a/dask_cuda/dask_cuda_worker.py b/dask_cuda/dask_cuda_worker.py index d2a13db05..563fd6d09 100755 --- a/dask_cuda/dask_cuda_worker.py +++ b/dask_cuda/dask_cuda_worker.py @@ -202,7 +202,7 @@ "that may cause unexpected errors if that interface is not properly " "configured or is disconnected, for that reason it's limited to " "InfiniBand only and will still cause unpredictable errors if not _ALL_ " - "interfaces are connected and properly configured." + "interfaces are connected and properly configured.", ) def main( scheduler, diff --git a/dask_cuda/explicit_comms/dataframe_merge.py b/dask_cuda/explicit_comms/dataframe_merge.py index 5cd6b039a..d52429c34 100644 --- a/dask_cuda/explicit_comms/dataframe_merge.py +++ b/dask_cuda/explicit_comms/dataframe_merge.py @@ -100,7 +100,9 @@ def partition_by_hash(df, columns, n_chunks, ignore_index=False): # For Dask < 2.17 compatibility ret = shuffle_group(df, "_partitions", 0, n_chunks, n_chunks, ignore_index) except TypeError: - ret = shuffle_group(df, "_partitions", 0, n_chunks, n_chunks, ignore_index, n_chunks) + ret = shuffle_group( + df, "_partitions", 0, n_chunks, n_chunks, ignore_index, n_chunks + ) # Let's remove the partition column and return the partitions del df["_partitions"] diff --git a/dask_cuda/initialize.py b/dask_cuda/initialize.py index f6308f28f..a87c07980 100644 --- a/dask_cuda/initialize.py +++ b/dask_cuda/initialize.py @@ -85,8 +85,7 @@ def initialize( @click.option( "--enable-rdmacm/--disable-rdmacm", default=False, - help="Enable RDMA connection manager, " - "currently requires InfiniBand enabled." + help="Enable RDMA connection manager, currently requires InfiniBand enabled.", ) @click.option( "--net-devices", diff --git a/dask_cuda/tests/test_spill.py b/dask_cuda/tests/test_spill.py index bcd13161f..8f45c7297 100644 --- a/dask_cuda/tests/test_spill.py +++ b/dask_cuda/tests/test_spill.py @@ -31,14 +31,18 @@ def device_host_file_size_matches( # `dhf.disk` is only available when Worker's `memory_limit != 0` if dhf.disk is not None: - file_path = [os.path.join(dhf.disk.directory, safe_key(k)) for k in dhf.disk.keys()] + file_path = [ + os.path.join(dhf.disk.directory, safe_key(k)) for k in dhf.disk.keys() + ] file_size = [os.path.getsize(f) for f in file_path] byte_sum += sum(file_size) # Allow up to chunk_overhead bytes overhead per chunk device_overhead = len(dhf.device) * device_chunk_overhead host_overhead = len(dhf.host) * serialized_chunk_overhead - disk_overhead = len(dhf.disk) * serialized_chunk_overhead if dhf.disk is not None else 0 + disk_overhead = ( + len(dhf.disk) * serialized_chunk_overhead if dhf.disk is not None else 0 + ) return ( byte_sum >= total_bytes @@ -75,7 +79,7 @@ def delayed_worker_assert(total_size, device_chunk_overhead, serialized_chunk_ov ) -#@pytest.mark.xfail(reason="https://github.com/rapidsai/dask-cuda/issues/79") +# @pytest.mark.xfail(reason="https://github.com/rapidsai/dask-cuda/issues/79") @pytest.mark.parametrize( "params", [ @@ -220,7 +224,9 @@ async def test_cupy_cluster_device_spill(params): await client.run(worker_assert, x.nbytes, 1024, 1024) host_chunks = await client.run(lambda: len(get_worker().data.host)) - disk_chunks = await client.run(lambda: len(get_worker().data.disk or list())) + disk_chunks = await client.run( + lambda: len(get_worker().data.disk or list()) + ) for hc, dc in zip(host_chunks.values(), disk_chunks.values()): if params["spills_to_disk"]: assert dc > 0 @@ -388,7 +394,9 @@ async def test_cudf_cluster_device_spill(params): del cdf host_chunks = await client.run(lambda: len(get_worker().data.host)) - disk_chunks = await client.run(lambda: len(get_worker().data.disk or list())) + disk_chunks = await client.run( + lambda: len(get_worker().data.disk or list()) + ) for hc, dc in zip(host_chunks.values(), disk_chunks.values()): if params["spills_to_disk"]: assert dc > 0 From 743a05bd74dd804b06e1fc0fe31be7fcd9bbd429 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Thu, 28 May 2020 15:44:58 -0700 Subject: [PATCH 64/67] Fix flake8 errors --- dask_cuda/benchmarks/local_cupy_transpose_sum.py | 3 +-- dask_cuda/dask_cuda_worker.py | 3 +-- dask_cuda/explicit_comms/comms.py | 5 +---- dask_cuda/explicit_comms/dataframe_merge.py | 4 +--- dask_cuda/explicit_comms/utils.py | 4 +++- dask_cuda/local_cuda_cluster.py | 4 +--- dask_cuda/tests/test_explicit_comms.py | 4 +--- dask_cuda/tests/test_initialize.py | 6 +++--- dask_cuda/tests/test_utils.py | 2 +- dask_cuda/utils.py | 2 -- 10 files changed, 13 insertions(+), 24 deletions(-) diff --git a/dask_cuda/benchmarks/local_cupy_transpose_sum.py b/dask_cuda/benchmarks/local_cupy_transpose_sum.py index 7d5b8f118..86177dd28 100644 --- a/dask_cuda/benchmarks/local_cupy_transpose_sum.py +++ b/dask_cuda/benchmarks/local_cupy_transpose_sum.py @@ -1,10 +1,9 @@ -import argparse import asyncio from collections import defaultdict from time import perf_counter as clock import dask.array as da -from dask.distributed import Client, SSHCluster, performance_report, wait +from dask.distributed import Client, performance_report, wait from dask.utils import format_bytes, format_time, parse_bytes from dask_cuda.benchmarks.utils import ( get_cluster_options, diff --git a/dask_cuda/dask_cuda_worker.py b/dask_cuda/dask_cuda_worker.py index 563fd6d09..8b303f399 100755 --- a/dask_cuda/dask_cuda_worker.py +++ b/dask_cuda/dask_cuda_worker.py @@ -23,7 +23,6 @@ from tornado.ioloop import IOLoop, TimeoutError from .device_host_file import DeviceHostFile -from .initialize import initialize from .local_cuda_cluster import cuda_visible_devices from .utils import ( CPUAffinity, @@ -300,7 +299,7 @@ def del_pid_file(): if rmm_pool_size is not None: try: - import rmm + import rmm # noqa F401 except ImportError: raise ValueError( "RMM pool requested but module 'rmm' is not available. " diff --git a/dask_cuda/explicit_comms/comms.py b/dask_cuda/explicit_comms/comms.py index 4c7063591..44b2bde0f 100644 --- a/dask_cuda/explicit_comms/comms.py +++ b/dask_cuda/explicit_comms/comms.py @@ -4,15 +4,13 @@ import uuid import distributed.comm -from distributed import default_client, get_worker, wait +from distributed import default_client, get_worker from distributed.comm.addressing import ( parse_address, parse_host_port, unparse_address, ) -import numpy as np - from . import utils _default_comms = None @@ -211,7 +209,6 @@ def dataframe_operation(self, coroutine, df_list, extra_args=tuple()): dataframe: Dask.dataframe.Dataframe The resulting dataframe """ - key = uuid.uuid1() df_parts_list = [] for df in df_list: df_parts_list.append( diff --git a/dask_cuda/explicit_comms/dataframe_merge.py b/dask_cuda/explicit_comms/dataframe_merge.py index d52429c34..41e35d875 100644 --- a/dask_cuda/explicit_comms/dataframe_merge.py +++ b/dask_cuda/explicit_comms/dataframe_merge.py @@ -4,11 +4,9 @@ from distributed.protocol import to_serialize import cudf -import numpy as np import pandas -import rmm -from . import comms, utils +from . import comms async def send_df(ep, df): diff --git a/dask_cuda/explicit_comms/utils.py b/dask_cuda/explicit_comms/utils.py index b914b7923..be1c443e3 100644 --- a/dask_cuda/explicit_comms/utils.py +++ b/dask_cuda/explicit_comms/utils.py @@ -67,7 +67,9 @@ def get_meta(df): def dataframes_to_dask_dataframe(futures, client=None): """ - Convert a list of futures containing Dataframes (pandas or cudf) into a Dask.Dataframe + Convert a list of futures containing Dataframes (pandas or cudf) into a + Dask.Dataframe + :param futures: list of futures containing dataframes :param client: dask.distributed.Client Optional client to use :return: dask.Dataframe a dask.Dataframe diff --git a/dask_cuda/local_cuda_cluster.py b/dask_cuda/local_cuda_cluster.py index da29919c2..eeaab4ea4 100644 --- a/dask_cuda/local_cuda_cluster.py +++ b/dask_cuda/local_cuda_cluster.py @@ -3,12 +3,10 @@ import dask from dask.distributed import LocalCluster -from distributed.system import MEMORY_LIMIT from distributed.utils import parse_bytes from distributed.worker import parse_memory_limit from .device_host_file import DeviceHostFile -from .initialize import initialize from .utils import ( CPUAffinity, RMMPool, @@ -160,7 +158,7 @@ def __init__( self.rmm_pool_size = rmm_pool_size if rmm_pool_size is not None: try: - import rmm + import rmm # noqa F401 except ImportError: raise ValueError( "RMM pool requested but module 'rmm' is not available. " diff --git a/dask_cuda/tests/test_explicit_comms.py b/dask_cuda/tests/test_explicit_comms.py index e0ab13e09..2bacc8c50 100644 --- a/dask_cuda/tests/test_explicit_comms.py +++ b/dask_cuda/tests/test_explicit_comms.py @@ -6,7 +6,6 @@ from distributed.deploy.local import LocalCluster import cudf -import cupy import numpy as np import pandas as pd import pytest @@ -51,8 +50,7 @@ def _test_dataframe_merge(backend, protocol, n_workers): threads_per_worker=1, processes=True, ) as cluster: - with Client(cluster) as client: - comms = CommsContext(client) + with Client(cluster): nrows = n_workers * 10 # Let's make some dataframes that we can join on the "key" column diff --git a/dask_cuda/tests/test_initialize.py b/dask_cuda/tests/test_initialize.py index 455422f4b..7b450f418 100644 --- a/dask_cuda/tests/test_initialize.py +++ b/dask_cuda/tests/test_initialize.py @@ -44,7 +44,7 @@ def check_ucx_options(): assert "sockcm" in conf["SOCKADDR_TLS_PRIORITY"] return True - assert client.run_on_scheduler(check_ucx_options) == True + assert client.run_on_scheduler(check_ucx_options) is True assert all(client.run(check_ucx_options).values()) @@ -81,7 +81,7 @@ def check_ucx_options(): assert "sockcm" in conf["SOCKADDR_TLS_PRIORITY"] return True - assert client.run_on_scheduler(check_ucx_options) == True + assert client.run_on_scheduler(check_ucx_options) is True assert all(client.run(check_ucx_options).values()) @@ -119,7 +119,7 @@ def check_ucx_options(): assert conf["NET_DEVICES"] == "ib0" return True - assert client.run_on_scheduler(check_ucx_options) == True + assert client.run_on_scheduler(check_ucx_options) is True assert all(client.run(check_ucx_options).values()) diff --git a/dask_cuda/tests/test_utils.py b/dask_cuda/tests/test_utils.py index a6b3f6279..9780cd93e 100644 --- a/dask_cuda/tests/test_utils.py +++ b/dask_cuda/tests/test_utils.py @@ -128,7 +128,7 @@ def test_get_ucx_net_devices_auto(): # Since the actual device is system-dependent, we just check that # this function call doesn't fail. If any InfiniBand devices are # available, it will return that, otherwise an empty string. - dev = get_ucx_net_devices(idx, "auto") + get_ucx_net_devices(idx, "auto") @pytest.mark.parametrize("enable_tcp_over_ucx", [True, False]) diff --git a/dask_cuda/utils.py b/dask_cuda/utils.py index cc567b1cc..b52156fb1 100644 --- a/dask_cuda/utils.py +++ b/dask_cuda/utils.py @@ -3,8 +3,6 @@ import warnings from multiprocessing import cpu_count -from distributed.utils import get_ip_interface - import numpy as np import pynvml import toolz From 05ed330cfefa2288f5948a9d55cd1a6fd57e6bce Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Thu, 28 May 2020 15:54:11 -0700 Subject: [PATCH 65/67] Fix path to run flake8 on --- ci/checks/style.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ci/checks/style.sh b/ci/checks/style.sh index e8cb13210..c54580158 100755 --- a/ci/checks/style.sh +++ b/ci/checks/style.sh @@ -20,7 +20,7 @@ BLACK=`black --check .` BLACK_RETVAL=$? # Run flake8 and get results/return code -FLAKE=`flake8 python` +FLAKE=`flake8 dask-cuda` RETVAL=$? # Output results if failure otherwise show pass From 8a0eb982b9474f75120dfd0f1148159e39408b96 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Thu, 28 May 2020 15:55:29 -0700 Subject: [PATCH 66/67] Fix typo in flake8 directory --- ci/checks/style.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ci/checks/style.sh b/ci/checks/style.sh index c54580158..965f62926 100755 --- a/ci/checks/style.sh +++ b/ci/checks/style.sh @@ -20,7 +20,7 @@ BLACK=`black --check .` BLACK_RETVAL=$? # Run flake8 and get results/return code -FLAKE=`flake8 dask-cuda` +FLAKE=`flake8 dask_cuda` RETVAL=$? # Output results if failure otherwise show pass From 2f9d5dd9eae15f9eb50401dd1e64e1e043b3ee59 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Tue, 2 Jun 2020 16:10:59 -0700 Subject: [PATCH 67/67] Update changelog for 0.14 --- CHANGELOG.rst | 23 +++++++++++++++++++++-- 1 file changed, 21 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.rst b/CHANGELOG.rst index 96e84099b..6da596714 100644 --- a/CHANGELOG.rst +++ b/CHANGELOG.rst @@ -1,7 +1,23 @@ 0.14 ---- -- Publish branch-0.14 to conda (#262) -- Local gpuCI build script (#285) +- Publish branch-0.14 to conda (#262) `Paul Taylor`_ +- Fix behavior for `memory_limit=0` (#269) `Peter Andreas Entschev`_ +- Raise serialization errors when spilling (#272) `John Kirkham`_ +- Fix dask-cuda-worker memory_limit (#279) `Peter Andreas Entschev`_ +- Add NVTX annotations for spilling (#282) `Peter Andreas Entschev`_ +- Skip existing on conda uploads (#284) `Ray Douglass`_ +- Local gpuCI build script (#285) `Eli Fajardo`_ +- Remove deprecated DGX class (#286) `Peter Andreas Entschev`_ +- Add RDMACM support (#287) `Peter Andreas Entschev`_ +- Read the Docs Setup (#290) `Benjamin Zaitlen`_ +- Raise ValueError when ucx_net_devices="auto" and IB is disabled (#291) `Peter Andreas Entschev`_ +- Multi-node benchmarks (#293) `Peter Andreas Entschev`_ +- Add docs for UCX (#294) `Peter Andreas Entschev`_ +- Add `--runs` argument to CuPy benchmark (#295) `Peter Andreas Entschev`_ +- Fixing LocalCUDACluster example. Adding README links to docs (#297) `Randy Gelhausen`_ +- Add `nfinal` argument to shuffle_group, required in Dask >= 2.17 (#299) `Peter Andreas Entschev`_ +- Initialize parent process' UCX configuration (#301) `Peter Andreas Entschev`_ +- Add Read the Docs link (#302) `John Kirkham`_ 0.13 ---- @@ -124,3 +140,6 @@ .. _`Richard (Rick) Zamora`: https://github.com/rjzamora .. _`Benjamin Zaitlen`: https://github.com/quasiben .. _`Ray Douglass`: https://github.com/raydouglass +.. _`Paul Taylor`: https://github.com/trxcllnt +.. _`Eli Fajardo`: https://github.com/efajardo-nv +.. _`Randy Gelhausen`: https://github.com/randerzander