From 05b8d669d00b1a9a6b3e5233e222935445c37145 Mon Sep 17 00:00:00 2001 From: Florian Jetter Date: Mon, 10 Jun 2024 14:34:53 +0200 Subject: [PATCH 01/72] Add section about futures and variables (#11164) Co-authored-by: Patrick Hoefler <61934744+phofl@users.noreply.github.com> --- docs/source/futures.rst | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/docs/source/futures.rst b/docs/source/futures.rst index c624c9e9bfe..bbaadc33d88 100644 --- a/docs/source/futures.rst +++ b/docs/source/futures.rst @@ -340,6 +340,30 @@ part of a function: process(filename) +Submit task and retrieve results from a different process +--------------------------------------------------------- + +Sometimes we care about retrieving a result but not necessarily from the same process. + +.. code-block:: python + + from distributed import Variable + + var = Variable("my-result") + fut = client.submit(...) + var.set(fut) + +Using a ``Variable`` instructs dask to remember the result of this task under +the given name so that it can be retrieved later without having to keep the +Client alive in the meantime. + +.. code-block:: python + + var = Variable("my-result") + fut = var.get() + result = fut.result() + + Submit Tasks from Tasks ----------------------- From a21c5226d7e8f4656c24daa6673a94b966005a84 Mon Sep 17 00:00:00 2001 From: Florian Jetter Date: Tue, 11 Jun 2024 13:41:46 +0200 Subject: [PATCH 02/72] Estimate sizes of xarray collections (#11166) --- dask/sizeof.py | 32 ++++++++++++++++++++++++++++++++ dask/tests/test_sizeof.py | 22 ++++++++++++++++++++++ 2 files changed, 54 insertions(+) diff --git a/dask/sizeof.py b/dask/sizeof.py index fd997d2d900..b7e28d69270 100644 --- a/dask/sizeof.py +++ b/dask/sizeof.py @@ -258,6 +258,38 @@ def sizeof_pyarrow_chunked_array(data): return int(_get_col_size(data)) + 1000 +@sizeof.register_lazy("xarray") +def register_xarray(): + import sys + + import xarray as xr + + @sizeof.register(xr.DataArray) + @sizeof.register(xr.NamedArray) + @sizeof.register(xr.Dataset) + def xarray_sizeof_da(obj): + return obj.nbytes + + @sizeof.register(xr.core.indexes.Indexes) + def xarray_sizeof_indexes(obj): + return ( + sys.getsizeof(obj) + + sizeof(obj._index_type) + + sizeof(obj._indexes) + + sizeof(obj._variables) + + sizeof(obj._dims) + ) + + @sizeof.register(xr.core.indexes.PandasIndex) + def xarray_sizeof_pd_index(obj): + return ( + sys.getsizeof(obj) + + sizeof(obj.index) + + sizeof(obj.dim) + + sizeof(obj.coord_dtype) + ) + + def _register_entry_point_plugins(): """Register sizeof implementations exposed by the entry_point mechanism.""" for entry_point in importlib_metadata.entry_points(group="dask.sizeof"): diff --git a/dask/tests/test_sizeof.py b/dask/tests/test_sizeof.py index 40c479b94fb..8f9250fe160 100644 --- a/dask/tests/test_sizeof.py +++ b/dask/tests/test_sizeof.py @@ -250,3 +250,25 @@ def test_register_backend_entrypoint(tmp_path): pool.apply(_get_sizeof_on_path, args=(tmp_path, 3_14159265)) == 3_14159265 ) pool.join() + + +def test_xarray(): + xr = pytest.importorskip("xarray") + np = pytest.importorskip("numpy") + + ind = np.arange(-66, 67, 1).astype(float) + arr = np.random.random((len(ind),)) + + dataset = ( + xr.DataArray( + arr, + dims=["coord"], + coords={"coord": ind}, + ) + .rename("foo") + .to_dataset() + ) + assert sizeof(dataset) > sizeof(arr) + assert sizeof(dataset.foo) >= sizeof(arr) + assert sizeof(dataset["coord"]) >= sizeof(ind) + assert sizeof(dataset.indexes) >= sizeof(ind) From 1203d7f014e69f67815a4c74cff9cb398571b718 Mon Sep 17 00:00:00 2001 From: Hendrik Makait Date: Wed, 12 Jun 2024 04:23:43 +0200 Subject: [PATCH 03/72] Ensure compatibility for `xarray.NamedArray` (#11168) --- dask/sizeof.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/dask/sizeof.py b/dask/sizeof.py index b7e28d69270..ab98996052c 100644 --- a/dask/sizeof.py +++ b/dask/sizeof.py @@ -6,6 +6,7 @@ import sys from array import array +from packaging.version import Version from packaging.version import parse as parse_version from dask._compatibility import importlib_metadata @@ -264,12 +265,17 @@ def register_xarray(): import xarray as xr + XARRAY_VERSION = Version(xr.__version__) + XARRAY_GE_2024_02 = XARRAY_VERSION >= Version("2024.02.0") + @sizeof.register(xr.DataArray) - @sizeof.register(xr.NamedArray) @sizeof.register(xr.Dataset) def xarray_sizeof_da(obj): return obj.nbytes + if XARRAY_GE_2024_02: + xarray_sizeof_da = sizeof.register(xr.NamedArray)(xarray_sizeof_da) + @sizeof.register(xr.core.indexes.Indexes) def xarray_sizeof_indexes(obj): return ( From b4e84adadda520023d9d93272ebdb276695e021d Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Wed, 12 Jun 2024 11:06:36 -0500 Subject: [PATCH 04/72] Remove deprecated ``dask.compatibility`` module (#11172) --- dask/compatibility.py | 14 -------------- dask/tests/test_compatibility.py | 5 ----- 2 files changed, 19 deletions(-) delete mode 100644 dask/compatibility.py diff --git a/dask/compatibility.py b/dask/compatibility.py deleted file mode 100644 index 62e86c893e3..00000000000 --- a/dask/compatibility.py +++ /dev/null @@ -1,14 +0,0 @@ -from __future__ import annotations - -import warnings - -from dask._compatibility import EMSCRIPTEN as _EMSCRIPTEN # noqa -from dask._compatibility import PY_VERSION as _PY_VERSION # noqa -from dask._compatibility import entry_points, parse_version # noqa - -warnings.warn( - "`dask.compatibility` is not intended for external use and has been renamed to `dask._compatibility`. " - "This backward-compatible shim will be removed in a future release. Please find an alternative.", - DeprecationWarning, - stacklevel=2, -) diff --git a/dask/tests/test_compatibility.py b/dask/tests/test_compatibility.py index d845f12dd83..9a1a58e9824 100644 --- a/dask/tests/test_compatibility.py +++ b/dask/tests/test_compatibility.py @@ -5,11 +5,6 @@ from dask._compatibility import entry_points -def test_deprecation(): - with pytest.warns(DeprecationWarning): - from dask.compatibility import _EMSCRIPTEN # noqa - - def test_entry_points(): with pytest.warns(DeprecationWarning): assert "pytest" in [ep.name for ep in entry_points(group="console_scripts")] From da1d53af6fcbf535ad74b6edb16347468cafbf1f Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Wed, 12 Jun 2024 11:41:28 -0500 Subject: [PATCH 05/72] Use `packaging.version.Version` (#11171) --- dask/_compatibility.py | 4 ++-- dask/array/numpy_compat.py | 4 ++-- dask/array/tests/test_cupy_core.py | 10 ++++---- dask/array/tests/test_cupy_linalg.py | 6 ++--- dask/array/tests/test_cupy_overlap.py | 8 +++---- dask/array/tests/test_cupy_routines.py | 10 ++++---- dask/array/tests/test_linalg.py | 10 ++++---- dask/array/tests/test_sparse.py | 12 +++++----- dask/array/tests/test_stats.py | 7 +++--- dask/bytes/tests/test_http.py | 6 ++--- dask/dataframe/io/parquet/fastparquet.py | 6 ++--- dask/dataframe/io/tests/test_parquet.py | 30 +++++++++++------------- dask/dataframe/io/utils.py | 4 ++-- dask/dataframe/tests/test_reshape.py | 4 ++-- dask/diagnostics/profile_visualize.py | 4 ++-- dask/sizeof.py | 3 +-- 16 files changed, 62 insertions(+), 66 deletions(-) diff --git a/dask/_compatibility.py b/dask/_compatibility.py index b5ac9389f48..d628d3cbdb2 100644 --- a/dask/_compatibility.py +++ b/dask/_compatibility.py @@ -7,9 +7,9 @@ import importlib.metadata as importlib_metadata else: import importlib_metadata -from packaging.version import parse as parse_version +from packaging.version import Version -PY_VERSION = parse_version(".".join(map(str, sys.version_info[:3]))) +PY_VERSION = Version(".".join(map(str, sys.version_info[:3]))) EMSCRIPTEN = sys.platform == "emscripten" diff --git a/dask/array/numpy_compat.py b/dask/array/numpy_compat.py index 5489263ab06..8e0b67484f9 100644 --- a/dask/array/numpy_compat.py +++ b/dask/array/numpy_compat.py @@ -3,11 +3,11 @@ import warnings import numpy as np -from packaging.version import parse as parse_version +from packaging.version import Version from dask.utils import derived_from -_np_version = parse_version(np.__version__) +_np_version = Version(np.__version__) NUMPY_GE_122 = _np_version.release >= (1, 22) NUMPY_GE_123 = _np_version.release >= (1, 23) NUMPY_GE_124 = _np_version.release >= (1, 24) diff --git a/dask/array/tests/test_cupy_core.py b/dask/array/tests/test_cupy_core.py index efa30025f71..71c0a3b25e6 100644 --- a/dask/array/tests/test_cupy_core.py +++ b/dask/array/tests/test_cupy_core.py @@ -2,7 +2,7 @@ import numpy as np import pytest -from packaging.version import parse as parse_version +from packaging.version import Version pytestmark = pytest.mark.gpu @@ -12,7 +12,7 @@ from dask.sizeof import sizeof cupy = pytest.importorskip("cupy") -cupy_version = parse_version(cupy.__version__) +cupy_version = Version(cupy.__version__) functions = [ @@ -36,7 +36,7 @@ pytest.param( lambda x: x.mean(), marks=pytest.mark.skipif( - cupy_version < parse_version("6.4.0"), + cupy_version < Version("6.4.0"), reason="Requires CuPy 6.4.0+ " "(with https://github.com/cupy/cupy/pull/2418)", ), @@ -46,7 +46,7 @@ pytest.param( lambda x: x.std(), marks=pytest.mark.skipif( - cupy_version < parse_version("6.4.0"), + cupy_version < Version("6.4.0"), reason="Requires CuPy 6.4.0+ " "(with https://github.com/cupy/cupy/pull/2418)", ), @@ -54,7 +54,7 @@ pytest.param( lambda x: x.var(), marks=pytest.mark.skipif( - cupy_version < parse_version("6.4.0"), + cupy_version < Version("6.4.0"), reason="Requires CuPy 6.4.0+ " "(with https://github.com/cupy/cupy/pull/2418)", ), diff --git a/dask/array/tests/test_cupy_linalg.py b/dask/array/tests/test_cupy_linalg.py index c8bf63a140d..579714cfc3b 100644 --- a/dask/array/tests/test_cupy_linalg.py +++ b/dask/array/tests/test_cupy_linalg.py @@ -2,7 +2,7 @@ import numpy as np import pytest -from packaging.version import parse as parse_version +from packaging.version import Version pytestmark = pytest.mark.gpu @@ -10,11 +10,11 @@ from dask.array.utils import assert_eq cupy = pytest.importorskip("cupy") -cupy_version = parse_version(cupy.__version__) +cupy_version = Version(cupy.__version__) @pytest.mark.skipif( - cupy_version < parse_version("6.1.0"), + cupy_version < Version("6.1.0"), reason="Requires CuPy 6.1.0+ (with https://github.com/cupy/cupy/pull/2209)", ) @pytest.mark.parametrize( diff --git a/dask/array/tests/test_cupy_overlap.py b/dask/array/tests/test_cupy_overlap.py index 3de646f7956..7d896b3beb7 100644 --- a/dask/array/tests/test_cupy_overlap.py +++ b/dask/array/tests/test_cupy_overlap.py @@ -2,7 +2,7 @@ import numpy as np import pytest -from packaging.version import parse as parse_version +from packaging.version import Version pytestmark = pytest.mark.gpu @@ -10,7 +10,7 @@ from dask.array.utils import assert_eq, same_keys cupy = pytest.importorskip("cupy") -cupy_version = parse_version(cupy.__version__) +cupy_version = Version(cupy.__version__) def test_overlap_internal(): @@ -88,7 +88,7 @@ def test_nearest(): @pytest.mark.skipif( - cupy_version < parse_version("6.4.0"), + cupy_version < Version("6.4.0"), reason="Requires CuPy 6.4.0+ (with https://github.com/cupy/cupy/pull/2418)", ) def test_constant(): @@ -104,7 +104,7 @@ def test_constant(): @pytest.mark.skipif( - cupy_version < parse_version("6.4.0"), + cupy_version < Version("6.4.0"), reason="Requires CuPy 6.4.0+ (with https://github.com/cupy/cupy/pull/2418)", ) def test_boundaries(): diff --git a/dask/array/tests/test_cupy_routines.py b/dask/array/tests/test_cupy_routines.py index cdc84b68d91..b5f56ab1494 100644 --- a/dask/array/tests/test_cupy_routines.py +++ b/dask/array/tests/test_cupy_routines.py @@ -2,7 +2,7 @@ import numpy as np import pytest -from packaging.version import parse as parse_version +from packaging.version import Version pytestmark = pytest.mark.gpu @@ -10,11 +10,11 @@ from dask.array.utils import assert_eq, same_keys cupy = pytest.importorskip("cupy") -cupy_version = parse_version(cupy.__version__) +cupy_version = Version(cupy.__version__) @pytest.mark.skipif( - cupy_version < parse_version("6.4.0"), + cupy_version < Version("6.4.0"), reason="Requires CuPy 6.4.0+ (with https://github.com/cupy/cupy/pull/2418)", ) def test_bincount(): @@ -138,7 +138,7 @@ def test_digitize(bins_type): @pytest.mark.skipif( - cupy_version < parse_version("6.4.0"), + cupy_version < Version("6.4.0"), reason="Requires CuPy 6.4.0+ (with https://github.com/cupy/cupy/pull/2418)", ) def test_tril_triu(): @@ -155,7 +155,7 @@ def test_tril_triu(): @pytest.mark.skipif( - cupy_version < parse_version("6.4.0"), + cupy_version < Version("6.4.0"), reason="Requires CuPy 6.4.0+ (with https://github.com/cupy/cupy/pull/2418)", ) def test_tril_triu_non_square_arrays(): diff --git a/dask/array/tests/test_linalg.py b/dask/array/tests/test_linalg.py index 6ae933b105a..60eda63cb44 100644 --- a/dask/array/tests/test_linalg.py +++ b/dask/array/tests/test_linalg.py @@ -9,7 +9,7 @@ import numpy as np import scipy.linalg -from packaging.version import parse as parse_version +from packaging.version import Version import dask.array as da from dask.array.linalg import qr, sfqr, svd, svd_compressed, tsqr @@ -767,7 +767,7 @@ def _get_symmat(size): # `sym_pos` kwarg was deprecated in scipy 1.9.0 # ref: https://github.com/dask/dask/issues/9335 def _scipy_linalg_solve(a, b, assume_a): - if parse_version(scipy.__version__) >= parse_version("1.9.0"): + if Version(scipy.__version__) >= Version("1.9.0"): return scipy.linalg.solve(a=a, b=b, assume_a=assume_a) elif assume_a == "pos": return scipy.linalg.solve(a=a, b=b, sym_pos=True) @@ -991,7 +991,7 @@ def test_svd_incompatible_dimensions(ndim): @pytest.mark.xfail( - sys.platform == "darwin" and _np_version < parse_version("1.22"), + sys.platform == "darwin" and _np_version < Version("1.22"), reason="https://github.com/dask/dask/issues/7189", strict=False, ) @@ -1012,7 +1012,7 @@ def test_norm_any_ndim(shape, chunks, axis, norm, keepdims): @pytest.mark.xfail( - _np_version < parse_version("1.23"), + _np_version < Version("1.23"), reason="https://github.com/numpy/numpy/pull/17709", strict=False, ) @@ -1039,7 +1039,7 @@ def test_norm_any_prec(norm, keepdims, precision, isreal): @pytest.mark.slow @pytest.mark.xfail( - sys.platform == "darwin" and _np_version < parse_version("1.22"), + sys.platform == "darwin" and _np_version < Version("1.22"), reason="https://github.com/dask/dask/issues/7189", strict=False, ) diff --git a/dask/array/tests/test_sparse.py b/dask/array/tests/test_sparse.py index 82d42fe0227..0b996d7dcb0 100644 --- a/dask/array/tests/test_sparse.py +++ b/dask/array/tests/test_sparse.py @@ -2,7 +2,7 @@ import numpy as np import pytest -from packaging.version import parse as parse_version +from packaging.version import Version import dask import dask.array as da @@ -10,7 +10,7 @@ from dask.array.utils import assert_eq sparse = pytest.importorskip("sparse") -SPARSE_VERSION = parse_version(sparse.__version__) +SPARSE_VERSION = Version(sparse.__version__) if sparse: # Test failures on older versions of Numba. # Conda-Forge provides 0.35.0 on windows right now, causing failures like @@ -72,21 +72,21 @@ pytest.param( lambda x: np.zeros_like(x), marks=pytest.mark.xfail( - SPARSE_VERSION < parse_version("0.13.0"), + SPARSE_VERSION < Version("0.13.0"), reason="https://github.com/pydata/xarray/issues/5654", ), ), pytest.param( lambda x: np.ones_like(x), marks=pytest.mark.xfail( - SPARSE_VERSION < parse_version("0.13.0"), + SPARSE_VERSION < Version("0.13.0"), reason="https://github.com/pydata/xarray/issues/5654", ), ), pytest.param( lambda x: np.full_like(x, fill_value=2), marks=pytest.mark.xfail( - SPARSE_VERSION < parse_version("0.13.0"), + SPARSE_VERSION < Version("0.13.0"), reason="https://github.com/pydata/xarray/issues/5654", ), ), @@ -112,7 +112,7 @@ def test_basic(func): @pytest.mark.skipif( - SPARSE_VERSION < parse_version("0.7.0+10"), + SPARSE_VERSION < Version("0.7.0+10"), reason="fixed in https://github.com/pydata/sparse/pull/256", ) def test_tensordot(): diff --git a/dask/array/tests/test_stats.py b/dask/array/tests/test_stats.py index 0b8f47cca48..0e17accc035 100644 --- a/dask/array/tests/test_stats.py +++ b/dask/array/tests/test_stats.py @@ -3,7 +3,7 @@ import warnings import pytest -from packaging.version import parse as parse_version +from packaging.version import Version scipy = pytest.importorskip("scipy") import numpy as np @@ -74,9 +74,8 @@ def test_one(kind): "ttest_1samp", {}, marks=pytest.mark.xfail( - # NOTE: using nested `parse_version` calls here to handle night scipy releases - parse_version(parse_version(scipy.__version__).base_version) - >= parse_version("1.10.0"), + # NOTE: using nested `Version` calls here to handle night scipy releases + Version(Version(scipy.__version__).base_version) >= Version("1.10.0"), reason="https://github.com/dask/dask/issues/9499", ), ), diff --git a/dask/bytes/tests/test_http.py b/dask/bytes/tests/test_http.py index c63bab37cd7..9d14c1e0a85 100644 --- a/dask/bytes/tests/test_http.py +++ b/dask/bytes/tests/test_http.py @@ -8,7 +8,7 @@ import fsspec import pytest from fsspec.core import open_files -from packaging.version import parse as parse_version +from packaging.version import Version import dask.bag as db from dask.utils import tmpdir @@ -20,7 +20,7 @@ requests.exceptions.RequestException, FileNotFoundError, ) -if parse_version(fsspec.__version__) > parse_version("0.7.4"): +if Version(fsspec.__version__) > Version("0.7.4"): aiohttp = pytest.importorskip("aiohttp") errs = errs + (aiohttp.client_exceptions.ClientResponseError,) @@ -121,7 +121,7 @@ def test_ops_blocksize(dir_server): fn = files[1] f = open_files(root + fn, block_size=2)[0] with f as f: - if parse_version(fsspec.__version__) < parse_version("2021.11.1"): + if Version(fsspec.__version__) < Version("2021.11.1"): # fails because we want only 12 bytes with pytest.raises(ValueError): assert f.read(10) == expected[:10] diff --git a/dask/dataframe/io/parquet/fastparquet.py b/dask/dataframe/io/parquet/fastparquet.py index 0111309f557..3a5d1a48b41 100644 --- a/dask/dataframe/io/parquet/fastparquet.py +++ b/dask/dataframe/io/parquet/fastparquet.py @@ -10,7 +10,7 @@ import numpy as np import pandas as pd import tlz as toolz -from packaging.version import parse as parse_version +from packaging.version import Version from dask.core import flatten from dask.dataframe._compat import PANDAS_GE_201 @@ -1114,7 +1114,7 @@ def pf_to_pandas( size = sum(rg.num_rows for rg in rgs) df, views = pf.pre_allocate(size, columns, categories, index) if ( - parse_version(fastparquet.__version__) <= parse_version("2023.02.0") + Version(fastparquet.__version__) <= Version("2023.02.0") and PANDAS_GE_201 and df.columns.empty ): @@ -1327,7 +1327,7 @@ def write_partition( rgs = [] elif partition_on: mkdirs = lambda x: fs.mkdirs(x, exist_ok=True) - if parse_version(fastparquet.__version__) >= parse_version("0.1.4"): + if Version(fastparquet.__version__) >= Version("0.1.4"): rgs = partition_on_columns( df, partition_on, path, filename, fmd, compression, fs.open, mkdirs ) diff --git a/dask/dataframe/io/tests/test_parquet.py b/dask/dataframe/io/tests/test_parquet.py index c4ca859ba31..65add0c71af 100644 --- a/dask/dataframe/io/tests/test_parquet.py +++ b/dask/dataframe/io/tests/test_parquet.py @@ -13,7 +13,7 @@ import numpy as np import pandas as pd import pytest -from packaging.version import parse as parse_version +from packaging.version import Version import dask import dask.dataframe as dd @@ -36,18 +36,18 @@ import fastparquet except ImportError: fastparquet = False - fastparquet_version = parse_version("0") + fastparquet_version = Version("0") else: - fastparquet_version = parse_version(fastparquet.__version__) + fastparquet_version = Version(fastparquet.__version__) try: import pyarrow as pa - pyarrow_version = parse_version(pa.__version__) + pyarrow_version = Version(pa.__version__) except ImportError: pa = False - pyarrow_version = parse_version("0") + pyarrow_version = Version("0") try: import pyarrow.parquet as pq @@ -1150,7 +1150,7 @@ def test_roundtrip(tmpdir, df, write_kwargs, read_kwargs, engine): "x" in df and df.x.dtype == "M8[ns]" and engine == "fastparquet" - and fastparquet_version <= parse_version("0.6.3") + and fastparquet_version <= Version("0.6.3") ): pytest.xfail(reason="fastparquet doesn't support nanosecond precision yet") # non-ns times @@ -1161,9 +1161,7 @@ def test_roundtrip(tmpdir, df, write_kwargs, read_kwargs, engine): ): if engine == "pyarrow": pytest.xfail("https://github.com/apache/arrow/issues/15079") - elif engine == "fastparquet" and fastparquet_version <= parse_version( - "2022.12.0" - ): + elif engine == "fastparquet" and fastparquet_version <= Version("2022.12.0"): pytest.xfail(reason="https://github.com/dask/fastparquet/issues/837") tmp = str(tmpdir) @@ -1188,7 +1186,7 @@ def test_roundtrip(tmpdir, df, write_kwargs, read_kwargs, engine): @pytest.mark.xfail( - pyarrow_strings_enabled() and pyarrow_version < parse_version("12.0.0"), + pyarrow_strings_enabled() and pyarrow_version < Version("12.0.0"), reason="Known failure with pyarrow strings: https://github.com/apache/arrow/issues/33727", ) def test_categories(tmpdir, engine): @@ -2368,14 +2366,14 @@ def test_append_cat_fp(tmpdir, engine): pytest.param( pd.DataFrame({"x": [3, 2, 1]}).astype("M8[us]"), marks=pytest.mark.xfail( - PANDAS_GE_200 and pyarrow_version < parse_version("13.0.0.dev"), + PANDAS_GE_200 and pyarrow_version < Version("13.0.0.dev"), reason="https://github.com/apache/arrow/issues/15079", ), ), pytest.param( pd.DataFrame({"x": [3, 2, 1]}).astype("M8[ms]"), marks=pytest.mark.xfail( - PANDAS_GE_200 and pyarrow_version < parse_version("13.0.0.dev"), + PANDAS_GE_200 and pyarrow_version < Version("13.0.0.dev"), reason="https://github.com/apache/arrow/issues/15079", ), ), @@ -2839,7 +2837,7 @@ def test_split_row_groups_int_aggregate_files(tmpdir, engine, split_row_groups): ) @pytest.mark.parametrize("split_row_groups", [True, False]) def test_filter_nulls(tmpdir, filters, op, length, split_row_groups, engine): - if engine == "pyarrow" and parse_version(pa.__version__) < parse_version("8.0.0"): + if engine == "pyarrow" and Version(pa.__version__) < Version("8.0.0"): # See: https://issues.apache.org/jira/browse/ARROW-15312 pytest.skip("pyarrow>=8.0.0 needed for correct null filtering") path = tmpdir.join("test.parquet") @@ -2865,7 +2863,7 @@ def test_filter_nulls(tmpdir, filters, op, length, split_row_groups, engine): @PYARROW_MARK @pytest.mark.parametrize("split_row_groups", [True, False]) def test_filter_isna(tmpdir, split_row_groups): - if parse_version(pa.__version__) < parse_version("8.0.0"): + if Version(pa.__version__) < Version("8.0.0"): # See: https://issues.apache.org/jira/browse/ARROW-15312 pytest.skip("pyarrow>=8.0.0 needed for correct null filtering") path = tmpdir.join("test.parquet") @@ -3319,7 +3317,7 @@ def test_pandas_timestamp_overflow_pyarrow(tmpdir): table, f"{tmpdir}/file.parquet", use_deprecated_int96_timestamps=False ) - if pyarrow_version < parse_version("13.0.0.dev"): + if pyarrow_version < Version("13.0.0.dev"): # This will raise by default due to overflow with pytest.raises(pa.lib.ArrowInvalid) as e: dd.read_parquet(str(tmpdir)).compute() @@ -3428,7 +3426,7 @@ def test_partitioned_column_overlap(tmpdir, engine, write_cols): path = str(tmpdir) expect = pd.concat([_df1, _df2], ignore_index=True) - if engine == "fastparquet" and fastparquet_version > parse_version("0.8.3"): + if engine == "fastparquet" and fastparquet_version > Version("0.8.3"): # columns will change order and partitions will be categorical result = dd.read_parquet(path, engine=engine) assert result.compute().reset_index(drop=True).to_dict() == expect.to_dict() diff --git a/dask/dataframe/io/utils.py b/dask/dataframe/io/utils.py index f389f452169..9f1165388a9 100644 --- a/dask/dataframe/io/utils.py +++ b/dask/dataframe/io/utils.py @@ -7,7 +7,7 @@ import fsspec import pandas as pd from fsspec.implementations.local import LocalFileSystem -from packaging.version import parse as parse_version +from packaging.version import Version try: import fsspec.parquet as fsspec_parquet @@ -195,7 +195,7 @@ def _open_input_files( precache == "parquet" and fs is not None and not _is_local_fs(fs) - and parse_version(fsspec.__version__) > parse_version("2021.11.0") + and Version(fsspec.__version__) > Version("2021.11.0") ): kwargs.update(precache_options) row_groups = kwargs.pop("row_groups", None) or ([None] * len(paths)) diff --git a/dask/dataframe/tests/test_reshape.py b/dask/dataframe/tests/test_reshape.py index 4516216d5b3..8bdeffb2c94 100644 --- a/dask/dataframe/tests/test_reshape.py +++ b/dask/dataframe/tests/test_reshape.py @@ -6,7 +6,7 @@ import numpy as np import pandas as pd import pytest -from packaging.version import parse as parse_version +from packaging.version import Version import dask.dataframe as dd from dask.dataframe._compat import PANDAS_VERSION, tm @@ -102,7 +102,7 @@ def check_pandas_issue_45618_warning(test_func): def decorator(): with warnings.catch_warnings(record=True) as record: test_func() - if PANDAS_VERSION == parse_version("1.4.0"): + if PANDAS_VERSION == Version("1.4.0"): assert all( "In a future version, passing a SparseArray" in str(r.message) for r in record diff --git a/dask/diagnostics/profile_visualize.py b/dask/diagnostics/profile_visualize.py index d3292089386..ec9d1aca1f7 100644 --- a/dask/diagnostics/profile_visualize.py +++ b/dask/diagnostics/profile_visualize.py @@ -14,9 +14,9 @@ def BOKEH_VERSION(): import bokeh - from packaging.version import parse as parse_version + from packaging.version import Version - return parse_version(bokeh.__version__) + return Version(bokeh.__version__) _BOKEH_MISSING_MSG = "Diagnostics plots require `bokeh` to be installed" diff --git a/dask/sizeof.py b/dask/sizeof.py index ab98996052c..7c9945ef1df 100644 --- a/dask/sizeof.py +++ b/dask/sizeof.py @@ -7,7 +7,6 @@ from array import array from packaging.version import Version -from packaging.version import parse as parse_version from dask._compatibility import importlib_metadata from dask.utils import Dispatch @@ -222,7 +221,7 @@ def register_spmatrix(): import scipy from scipy import sparse - if parse_version(scipy.__version__) < parse_version("1.12.0.dev0"): + if Version(scipy.__version__) < Version("1.12.0.dev0"): @sizeof.register(sparse.dok_matrix) def sizeof_spmatrix_dok(s): From 91f791031d01ea6da8deef704bba5ea2e2f4bebf Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Fri, 14 Jun 2024 03:45:21 -0500 Subject: [PATCH 06/72] Fix ``test_dt_accessor`` with query planning disabled (#11177) --- dask/dataframe/tests/test_accessors.py | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/dask/dataframe/tests/test_accessors.py b/dask/dataframe/tests/test_accessors.py index 29ae9a06154..01a66f8999e 100644 --- a/dask/dataframe/tests/test_accessors.py +++ b/dask/dataframe/tests/test_accessors.py @@ -95,9 +95,7 @@ def df_ddf(): return df, ddf -@pytest.mark.skipif( - not PANDAS_GE_210 or PANDAS_GE_300, reason="warning is None|divisions are incorrect" -) +@pytest.mark.xfail(PANDAS_GE_300, reason="divisions are incorrect") def test_dt_accessor(df_ddf): df, ddf = df_ddf @@ -106,7 +104,10 @@ def test_dt_accessor(df_ddf): # pandas loses Series.name via datetime accessor # see https://github.com/pydata/pandas/issues/10712 assert_eq(ddf.dt_col.dt.date, df.dt_col.dt.date, check_names=False) - warning_ctx = pytest.warns(FutureWarning, match="will return a Series") + if PANDAS_GE_210: + warning_ctx = pytest.warns(FutureWarning, match="will return a Series") + else: + warning_ctx = contextlib.nullcontext() # to_pydatetime returns a numpy array in pandas, but a Series in dask # pandas will start returning a Series with 3.0 as well with warning_ctx: @@ -122,9 +123,9 @@ def test_dt_accessor(df_ddf): # The warnings is raised during construction of the expression, not the # materialization of the graph. Therefore, the singleton approach of # dask-expr avoids another warning - ctx = contextlib.nullcontext() + warning_ctx = contextlib.nullcontext() - with ctx: + with warning_ctx: assert set(ddf.dt_col.dt.to_pydatetime().dask) == set( ddf.dt_col.dt.to_pydatetime().dask ) From d1adae476b76926df9b51e5fb1cd135f17b42aeb Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Fri, 14 Jun 2024 15:42:14 -0500 Subject: [PATCH 07/72] bump version to 2024.6.0 --- docs/source/changelog.rst | 44 +++++++++++++++++++++++++++++++++++++++ pyproject.toml | 2 +- 2 files changed, 45 insertions(+), 1 deletion(-) diff --git a/docs/source/changelog.rst b/docs/source/changelog.rst index 8e8536002ca..9d5ccbb6cec 100644 --- a/docs/source/changelog.rst +++ b/docs/source/changelog.rst @@ -1,6 +1,49 @@ Changelog ========= +.. _v2024.6.0: + +2024.6.0 +-------- + +Highlights +^^^^^^^^^^ + +memmap array tokenization +""""""""""""""""""""""""" +Tokenizing ``memmap`` arrays will now avoid materializing the array into memory. + +See :pr:`11161` by `Florian Jetter`_ for more details. + + +.. dropdown:: Additional changes + + - Fix ``test_dt_accessor`` with query planning disabled (:pr:`11177`) `James Bourbeau`_ + - Use ``packaging.version.Version`` (:pr:`11171`) `James Bourbeau`_ + - Remove deprecated ``dask.compatibility`` module (:pr:`11172`) `James Bourbeau`_ + - Ensure compatibility for ``xarray.NamedArray`` (:pr:`11168`) `Hendrik Makait`_ + - Estimate sizes of ``xarray`` collections (:pr:`11166`) `Florian Jetter`_ + - Add section about futures and variables (:pr:`11164`) `Florian Jetter`_ + - Update docs for combined Dask community meeting info (:pr:`11159`) `Sarah Charlotte Johnson`_ + + - Avoid rounding error in ``test_prometheus_collect_count_total_by_cost_multipliers`` (:pr-distributed:`8687`) `Hendrik Makait`_ + - Log key collision count in ``update_graph`` log event (:pr-distributed:`8692`) `Hendrik Makait`_ + - Automate GitHub Releases when new tags are pushed (:pr-distributed:`8626`) `Jacob Tomlinson`_ + - Fix log event with multiple topics (:pr-distributed:`8691`) `Hendrik Makait`_ + - Rename ``safe`` to ``expected`` in ``Scheduler.remove_worker`` (:pr-distributed:`8686`) `Hendrik Makait`_ + - Log event during failure (:pr-distributed:`8663`) `Hendrik Makait`_ + - Eagerly update aggregate statistics for ``TaskPrefix`` instead of calculating them on-demand (:pr-distributed:`8681`) `Hendrik Makait`_ + - Improve graph submission time for P2P rechunking by avoiding unpack recursion into indices (:pr-distributed:`8672`) `Florian Jetter`_ + - Add safe keyword to ``remove-worker`` event (:pr-distributed:`8647`) `alex-rakowski`_ + - Improved errors and reduced logging for P2P RPC calls (:pr-distributed:`8666`) `Hendrik Makait`_ + - Adjust P2P tests for ``dask-expr`` (:pr-distributed:`8662`) `Hendrik Makait`_ + - Iterate over copy of ``Server.digests_total_since_heartbeat`` to avoid ``RuntimeError`` (:pr-distributed:`8670`) `Hendrik Makait`_ + - Log task state in Compute Failed (:pr-distributed:`8668`) `Hendrik Makait`_ + - Add Prometheus gauge for task groups (:pr-distributed:`8661`) `Hendrik Makait`_ + - Fix too strict assertion in shuffle code for ``pandas`` subclasses (:pr-distributed:`8667`) `Joris Van den Bossche`_ + - Reduce noise from erring tasks that are not supposed to be running (:pr-distributed:`8664`) `Hendrik Makait`_ + + .. _v2024.5.2: 2024.5.2 @@ -8221,3 +8264,4 @@ Other .. _`M Bussonnier`: https://github.com/Carreau .. _`Greg M. Fleishman`: https://github.com/GFleishman .. _`Victor Stinner`: https://github.com/vstinner +.. _`alex-rakowski`: https://github.com/alex-rakowski diff --git a/pyproject.toml b/pyproject.toml index a67c1795747..9e157bcbd20 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -57,7 +57,7 @@ dataframe = [ "pandas >= 1.3", "dask-expr >= 1.1, <1.2", # dask-expr pins the dask version ] -distributed = ["distributed == 2024.5.2"] +distributed = ["distributed == 2024.6.0"] diagnostics = [ "bokeh >= 2.4.2", "jinja2 >= 2.10.3", From f6e0690ec7b2a5b177470c85cef2193e5b97875e Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Sat, 15 Jun 2024 05:05:58 -0500 Subject: [PATCH 08/72] Fix ``test_map_freq_to_period_start`` for ``pandas=3`` (#11181) --- dask/dataframe/tests/test_dataframe.py | 20 ++++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/dask/dataframe/tests/test_dataframe.py b/dask/dataframe/tests/test_dataframe.py index 9971fd84409..639e0e0c5ea 100644 --- a/dask/dataframe/tests/test_dataframe.py +++ b/dask/dataframe/tests/test_dataframe.py @@ -2634,16 +2634,20 @@ def test_repartition_noop(type_ctor): ], ) def test_map_freq_to_period_start(freq, expected_freq): - if freq in ("A", "A-JUN", "BA", "2BA") and PANDAS_GE_300: - return - if PANDAS_GE_220 and freq not in ("ME", "MS", pd.Timedelta(seconds=1), "2QS-FEB"): - with pytest.warns( + # Handle warnnigs/errors from deprecation cycle in pandas + ctx = contextlib.nullcontext() + if PANDAS_GE_300 and ( + freq in ("A", "A-JUN", "BA", "2BA") + or freq not in ("ME", "MS", pd.Timedelta(seconds=1), "2QS-FEB") + ): + ctx = pytest.raises(ValueError, match="Invalid frequency") + elif PANDAS_GE_220 and freq not in ("ME", "MS", pd.Timedelta(seconds=1), "2QS-FEB"): + ctx = pytest.warns( FutureWarning, match="is deprecated and will be removed in a future version" - ): - new_freq = _map_freq_to_period_start(freq) - else: + ) + with ctx: new_freq = _map_freq_to_period_start(freq) - assert new_freq == expected_freq + assert new_freq == expected_freq def test_repartition_input_errors(): From 1755a718125ef2ea18e9c079ecf1509b7b6be730 Mon Sep 17 00:00:00 2001 From: Adam Williamson Date: Tue, 18 Jun 2024 07:08:01 -0700 Subject: [PATCH 09/72] Python 3.13 fixes (#11185) Signed-off-by: Adam Williamson --- dask/delayed.py | 21 +++++++++++---------- dask/tests/test_utils.py | 2 +- 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/dask/delayed.py b/dask/delayed.py index 1c63a39ef12..350e89fbfa8 100644 --- a/dask/delayed.py +++ b/dask/delayed.py @@ -141,16 +141,17 @@ def unpack_collections(expr): if hasattr(expr, f.name) } replace(expr, **_fields) - except TypeError as e: - raise TypeError( - f"Failed to unpack {typ} instance. " - "Note that using a custom __init__ is not supported." - ) from e - except ValueError as e: - raise ValueError( - f"Failed to unpack {typ} instance. " - "Note that using fields with `init=False` are not supported." - ) from e + except (TypeError, ValueError) as e: + if isinstance(e, ValueError) or "is declared with init=False" in str(e): + raise ValueError( + f"Failed to unpack {typ} instance. " + "Note that using fields with `init=False` are not supported." + ) from e + else: + raise TypeError( + f"Failed to unpack {typ} instance. " + "Note that using a custom __init__ is not supported." + ) from e return (apply, typ, (), (dict, args)), collections if is_namedtuple_instance(expr): diff --git a/dask/tests/test_utils.py b/dask/tests/test_utils.py index 328705f7f26..8aba2b3f380 100644 --- a/dask/tests/test_utils.py +++ b/dask/tests/test_utils.py @@ -600,7 +600,7 @@ def f(a, c): assert "not supported" in b_arg.lower() assert "dask" in b_arg.lower() - assert " extra docstring\n\n" in Zap.f.__doc__ + assert "extra docstring\n\n" in Zap.f.__doc__ @pytest.mark.parametrize( From d0bc9296b2c7ce5cb86971726ec27c7c92ab063e Mon Sep 17 00:00:00 2001 From: "Richard (Rick) Zamora" Date: Tue, 18 Jun 2024 10:55:09 -0500 Subject: [PATCH 10/72] Cache global query-planning config (#11183) --- dask/dataframe/__init__.py | 24 +++++++++++++++++++++--- dask/dataframe/tests/test_dataframe.py | 10 ++++++++++ 2 files changed, 31 insertions(+), 3 deletions(-) diff --git a/dask/dataframe/__init__.py b/dask/dataframe/__init__.py index 029eef5c54a..1a3e128f6ca 100644 --- a/dask/dataframe/__init__.py +++ b/dask/dataframe/__init__.py @@ -5,19 +5,37 @@ from packaging.version import Version +# The "dataframe.query-planning" config can only be processed once +DASK_EXPR_ENABLED: bool | None = None + def _dask_expr_enabled() -> bool: import pandas as pd import dask + global DASK_EXPR_ENABLED + use_dask_expr = dask.config.get("dataframe.query-planning") + if DASK_EXPR_ENABLED is not None: + if (use_dask_expr is True and DASK_EXPR_ENABLED is False) or ( + use_dask_expr is False and DASK_EXPR_ENABLED is True + ): + warnings.warn( + "The 'dataframe.query-planning' config is now set to " + f"{use_dask_expr}, but query planning is already " + f"{'enabled' if DASK_EXPR_ENABLED else 'disabled'}. " + "The query-planning config can only be changed before " + "`dask.dataframe` is first imported!" + ) + return DASK_EXPR_ENABLED + if ( use_dask_expr is False or use_dask_expr is None and Version(pd.__version__).major < 2 ): - return False + return (DASK_EXPR_ENABLED := False) try: import dask_expr # noqa: F401 except ImportError: @@ -29,10 +47,10 @@ def _dask_expr_enabled() -> bool: """ if use_dask_expr is None: warnings.warn(msg, FutureWarning) - return False + return (DASK_EXPR_ENABLED := False) else: raise ImportError(msg) - return True + return (DASK_EXPR_ENABLED := True) try: diff --git a/dask/dataframe/tests/test_dataframe.py b/dask/dataframe/tests/test_dataframe.py index 639e0e0c5ea..900a2068835 100644 --- a/dask/dataframe/tests/test_dataframe.py +++ b/dask/dataframe/tests/test_dataframe.py @@ -6476,3 +6476,13 @@ def test_enforce_runtime_divisions(): RuntimeError, match="`enforce_runtime_divisions` failed for partition 1" ): ddf.enforce_runtime_divisions().compute() + + +def test_query_planning_config_warns(): + # Make sure dd._dask_expr_enabled() warns if the current + # "dataframe.query-planning" config conflicts with the + # global dd.DASK_EXPR_ENABLED setting. + with dask.config.set({"dataframe.query-planning": not DASK_EXPR_ENABLED}): + expect = "enabled" if dd.DASK_EXPR_ENABLED else "disabled" + with pytest.warns(match=f"query planning is already {expect}"): + dd._dask_expr_enabled() From 92bb34eeb03304a23ba04403cfe521c72c164d5b Mon Sep 17 00:00:00 2001 From: fjetter Date: Wed, 19 Jun 2024 19:17:09 +0200 Subject: [PATCH 11/72] bump version to 2024.6.1 --- docs/source/changelog.rst | 23 +++++++++++++++++++++++ pyproject.toml | 2 +- 2 files changed, 24 insertions(+), 1 deletion(-) diff --git a/docs/source/changelog.rst b/docs/source/changelog.rst index 9d5ccbb6cec..41ed158218f 100644 --- a/docs/source/changelog.rst +++ b/docs/source/changelog.rst @@ -1,6 +1,28 @@ Changelog ========= +.. _v2024.6.1: + +2024.6.1 +-------- + +Highlights +^^^^^^^^^^ + +This release includes a critical fix that fixes a deadlock that can arise +when dependencies of root-ish tasks are rescheduled, e.g. due to a worker being +lost. + +See :pr-distributed:`8703` by `Hendrik Makait`_ for more details + +.. dropdown:: Additional changes + + - Cache global query-planning config (:pr:`11183`) `Richard (Rick) Zamora`_ + - Python 3.13 fixes (:pr:`11185`) `Adam Williamson`_ + - Fix ``test_map_freq_to_period_start`` for ``pandas=3`` (:pr:`11181`) `James Bourbeau`_ + - Bump release-drafter/release-drafter from 5 to 6 (:pr-distributed:`8699`) `dependabot[bot]`_ + + .. _v2024.6.0: 2024.6.0 @@ -8265,3 +8287,4 @@ Other .. _`Greg M. Fleishman`: https://github.com/GFleishman .. _`Victor Stinner`: https://github.com/vstinner .. _`alex-rakowski`: https://github.com/alex-rakowski +.. _`Adam Williamson`: https://github.com/AdamWill diff --git a/pyproject.toml b/pyproject.toml index 9e157bcbd20..6f65014c3c9 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -57,7 +57,7 @@ dataframe = [ "pandas >= 1.3", "dask-expr >= 1.1, <1.2", # dask-expr pins the dask version ] -distributed = ["distributed == 2024.6.0"] +distributed = ["distributed == 2024.6.1"] diagnostics = [ "bokeh >= 2.4.2", "jinja2 >= 2.10.3", From be42bf493c71937c0e7abef86e2af302821dc8cc Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Thu, 20 Jun 2024 14:11:09 -0500 Subject: [PATCH 12/72] Get docs build passing (#11184) --- .readthedocs.yaml | 5 +++-- docs/source/changelog.rst | 4 ++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/.readthedocs.yaml b/.readthedocs.yaml index efdbdeb4785..15efbdeed8e 100644 --- a/.readthedocs.yaml +++ b/.readthedocs.yaml @@ -3,11 +3,12 @@ version: 2 build: os: ubuntu-22.04 tools: - python: "3.11" + python: "3.12" sphinx: configuration: docs/source/conf.py - fail_on_warning: true + # Temporarily turning off to get docs build passing + # fail_on_warning: true python: install: diff --git a/docs/source/changelog.rst b/docs/source/changelog.rst index 41ed158218f..bb5b210852b 100644 --- a/docs/source/changelog.rst +++ b/docs/source/changelog.rst @@ -13,14 +13,14 @@ This release includes a critical fix that fixes a deadlock that can arise when dependencies of root-ish tasks are rescheduled, e.g. due to a worker being lost. -See :pr-distributed:`8703` by `Hendrik Makait`_ for more details +See :pr-distributed:`8703` by `Hendrik Makait`_ for more details. .. dropdown:: Additional changes - Cache global query-planning config (:pr:`11183`) `Richard (Rick) Zamora`_ - Python 3.13 fixes (:pr:`11185`) `Adam Williamson`_ - Fix ``test_map_freq_to_period_start`` for ``pandas=3`` (:pr:`11181`) `James Bourbeau`_ - - Bump release-drafter/release-drafter from 5 to 6 (:pr-distributed:`8699`) `dependabot[bot]`_ + - Bump release-drafter/release-drafter from 5 to 6 (:pr-distributed:`8699`) .. _v2024.6.0: From 450aa31a498d7147e7dd5709feff3f99778c6493 Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Thu, 20 Jun 2024 14:54:54 -0500 Subject: [PATCH 13/72] bump version to 2024.6.2 --- docs/source/changelog.rst | 13 +++++++++++++ pyproject.toml | 2 +- 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/docs/source/changelog.rst b/docs/source/changelog.rst index bb5b210852b..e14cbf6e2e9 100644 --- a/docs/source/changelog.rst +++ b/docs/source/changelog.rst @@ -1,6 +1,19 @@ Changelog ========= +.. _v2024.6.2: + +2024.6.2 +-------- + +This is a patch release to update an issue with ``dask`` and ``distributed`` +version pinning in the 2024.6.1 release. + +.. dropdown:: Additional changes + + - Get docs build passing (:pr:`11184`) `James Bourbeau`_ + - ``profile._f_lineno``: handle ``next_line`` being ``None`` in Python 3.13 (:pr:`8710`) `Adam Williamson`_ + .. _v2024.6.1: 2024.6.1 diff --git a/pyproject.toml b/pyproject.toml index 6f65014c3c9..1d036cbb954 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -57,7 +57,7 @@ dataframe = [ "pandas >= 1.3", "dask-expr >= 1.1, <1.2", # dask-expr pins the dask version ] -distributed = ["distributed == 2024.6.1"] +distributed = ["distributed == 2024.6.2"] diagnostics = [ "bokeh >= 2.4.2", "jinja2 >= 2.10.3", From 186852460f3838ccbd784cc0f710dd64015a53d7 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Fri, 21 Jun 2024 14:25:55 +0200 Subject: [PATCH 14/72] Add other IO connectors to docs (#11189) --- docs/source/dataframe-create.rst | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/source/dataframe-create.rst b/docs/source/dataframe-create.rst index c2cabf2cbb5..1b2c984620d 100644 --- a/docs/source/dataframe-create.rst +++ b/docs/source/dataframe-create.rst @@ -59,6 +59,12 @@ Pandas: from_pandas DataFrame.from_dict +Other File Formats: + +- `Snowflake `_ +- `Bigquery `_ +- `Delta Lake `_ + Creating -------- From 36e9d7c84c619abcfc498e3fdf7b37f7fb8fd400 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Fri, 21 Jun 2024 14:58:14 +0200 Subject: [PATCH 15/72] Remove from_delayed references (#11195) --- dask/dataframe/io/io.py | 37 +++++++++++++++++++++++++---------- docs/source/dataframe-sql.rst | 25 +++++++++-------------- 2 files changed, 36 insertions(+), 26 deletions(-) diff --git a/dask/dataframe/io/io.py b/dask/dataframe/io/io.py index 76ab6379386..173899ac1c0 100644 --- a/dask/dataframe/io/io.py +++ b/dask/dataframe/io/io.py @@ -590,7 +590,22 @@ def from_delayed( prefix: str = "from-delayed", verify_meta: bool = True, ) -> DataFrame | Series: - """Create Dask DataFrame from many Dask Delayed objects + """Create Dask DataFrame from many Dask Delayed objects. + + .. warning:: + ``from_delayed`` should only be used if the objects that create + the data are complex and cannot be easily represented as a single + function in an embarassingly parallel fashion. + + ``from_map`` is recommended if the query can be expressed as a single + function like: + + def read_xml(path): + return pd.read_xml(path) + + ddf = dd.from_map(read_xml, paths) + + ``from_delayed`` might be depreacted in the future. Parameters ---------- @@ -858,10 +873,13 @@ def from_map( enforce_metadata=True, **kwargs, ): - """Create a DataFrame collection from a custom function map + """Create a DataFrame collection from a custom function map. - WARNING: The ``from_map`` API is experimental, and stability is not - yet guaranteed. Use at your own risk! + ``from_map`` is the preferred option when reading from data sources + that are not natively supported by Dask or if the data source + requires custom handling before handing things of to Dask DataFrames. + Examples are things like binary files or other unstructured data that + doesn't have an IO connector. Parameters ---------- @@ -913,13 +931,13 @@ def from_map( dtype: object This API can also be used as an alternative to other file-based - IO functions, like ``read_parquet`` (which are already just + IO functions, like ``read_csv`` (which are already just ``from_map`` wrapper functions): >>> import pandas as pd >>> import dask.dataframe as dd - >>> paths = ["0.parquet", "1.parquet", "2.parquet"] - >>> dd.from_map(pd.read_parquet, paths).head() # doctest: +SKIP + >>> paths = ["0.csv", "1.csv", "2.csv"] + >>> dd.from_map(pd.read_csv, paths).head() # doctest: +SKIP name timestamp 2000-01-01 00:00:00 Laura @@ -939,11 +957,11 @@ def from_map( >>> import pandas as pd >>> import numpy as np >>> import dask.dataframe as dd - >>> paths = ["0.parquet", "1.parquet", "2.parquet"] + >>> paths = ["0.csv", "1.csv", "2.csv"] >>> file_sizes = [86400, 86400, 86400] >>> def func(path, row_offset): ... # Read parquet file and set RangeIndex offset - ... df = pd.read_parquet(path) + ... df = pd.read_csv(path) ... return df.set_index( ... pd.RangeIndex(row_offset, row_offset+len(df)) ... ) @@ -965,7 +983,6 @@ def from_map( See Also -------- - dask.dataframe.from_delayed dask.layers.DataFrameIOLayer """ diff --git a/docs/source/dataframe-sql.rst b/docs/source/dataframe-sql.rst index 4e7d9d67917..0cec1e60c44 100644 --- a/docs/source/dataframe-sql.rst +++ b/docs/source/dataframe-sql.rst @@ -204,34 +204,30 @@ Load from SQL, manual approaches If ``read_sql_table`` is not sufficient for your needs, you can try one of the following methods. -Delayed functions -^^^^^^^^^^^^^^^^^ +From Map functions +^^^^^^^^^^^^^^^^^^ Often you know more about your data and server than the generic approach above allows. Indeed, some database-like servers may simply not be supported by -SQLAlchemy, or provide an alternate API which is better optimised -(`snowflake example`_). - -.. _snowflake example: https://www.saturncloud.io/s/snowflake-and-dask/ +SQLAlchemy, or provide an alternate API which is better optimised. If you already have a way to fetch data from the database in partitions, -then you can wrap this function in :func:`dask.delayed` and construct a -dataframe this way. It might look something like +then you can use :func:`dask.dataframe.from_map` and construct a +dataframe this way. It might look something like. .. code-block:: python - from dask import delayed import dask.dataframe as dd - @delayed def fetch_partition(part): conn = establish_connection() df = fetch_query(base_query.format(part)) return df.astype(known_types) - ddf = dd.from_delayed([fetch_partition(part) for part in parts], - meta=known_types, - divisions=div_from_parts(parts)) + ddf = dd.from_map(fetch_partition, + parts, + meta=known_types, + divisions=div_from_parts(parts)) Where you must provide your own functions for setting up a connection to the server, your own query, and a way to format that query to be specific to each partition. @@ -239,9 +235,6 @@ For example, you might have ranges or specific unique values with a WHERE clause. The ``known_types`` here is used to transform the dataframe partition and provide a ``meta``, to help for consistency and avoid Dask having to analyse one partition up front to guess the columns/types; you may also want to explicitly set the index. -Furthermore, it is a good idea to provide -``divisions`` (the start/end of each partition in the index column), if possible, -since you likely know these from the subqueries you are constructing. Stream via client ^^^^^^^^^^^^^^^^^ From ff2488aec44d641696e0b7aa41ed9e995c710705 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Fri, 21 Jun 2024 15:58:14 +0200 Subject: [PATCH 16/72] Remove no longer supported ipython reference (#11196) --- docs/source/how-to/debug.rst | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/docs/source/how-to/debug.rst b/docs/source/how-to/debug.rst index 7bf4256068a..413e3e9c4f5 100644 --- a/docs/source/how-to/debug.rst +++ b/docs/source/how-to/debug.rst @@ -277,19 +277,3 @@ This can be very helpful if you want to use the Dask distributed API and still want to investigate what is going on directly within the workers. Information is not distilled for you like it is in the web diagnostics, but you have full low-level access. - - -Inspect state with IPython --------------------------- - -Sometimes you want to inspect the state of your cluster but you don't have the -luxury of operating on a single machine. In these cases you can launch an -IPython kernel on the scheduler and on every worker, which lets you inspect -state on the scheduler and workers as computations are completing. - -This does not give you the ability to run ``%pdb`` or ``%debug`` on remote -machines. The tasks are still running in separate threads, and so are not -easily accessible from an interactive IPython session. - -For more details, see the `Dask distributed IPython docs -`_. From cf2c5055057b549d13bc0f11a13b6b595effcec4 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Mon, 1 Jul 2024 15:42:27 +0200 Subject: [PATCH 17/72] Drop support for pandas<2 (#11199) --- .github/workflows/tests.yml | 7 + continuous_integration/environment-3.10.yaml | 10 +- continuous_integration/environment-3.11.yaml | 4 +- continuous_integration/environment-3.12.yaml | 2 +- continuous_integration/environment-3.9.yaml | 4 +- .../environment-mindeps-array.yaml | 2 +- .../environment-mindeps-dataframe.yaml | 6 +- .../environment-mindeps-distributed.yaml | 2 +- .../environment-mindeps-non-optional.yaml | 2 +- .../environment-mindeps-optional.yaml | 5 +- continuous_integration/recipe/meta.yaml | 2 +- dask/array/percentile.py | 7 +- dask/array/tests/test_array_core.py | 11 +- dask/dataframe/__init__.py | 11 +- dask/dataframe/_compat.py | 66 +--- dask/dataframe/_dtypes.py | 65 ++-- dask/dataframe/_pyarrow.py | 13 +- dask/dataframe/_pyarrow_compat.py | 17 - dask/dataframe/core.py | 347 +++--------------- dask/dataframe/groupby.py | 148 ++------ dask/dataframe/io/json.py | 9 +- dask/dataframe/io/tests/test_csv.py | 18 +- dask/dataframe/io/tests/test_demo.py | 6 +- dask/dataframe/io/tests/test_hdf.py | 1 + dask/dataframe/io/tests/test_io.py | 29 +- dask/dataframe/io/tests/test_json.py | 9 +- dask/dataframe/io/tests/test_parquet.py | 37 +- dask/dataframe/methods.py | 31 +- dask/dataframe/partitionquantiles.py | 30 +- dask/dataframe/reshape.py | 6 +- dask/dataframe/tests/test_accessors.py | 3 +- .../tests/test_arithmetics_reduction.py | 174 +++------ dask/dataframe/tests/test_categorical.py | 19 +- dask/dataframe/tests/test_dataframe.py | 254 +++---------- dask/dataframe/tests/test_groupby.py | 167 ++------- dask/dataframe/tests/test_methods.py | 4 +- dask/dataframe/tests/test_multi.py | 153 +------- dask/dataframe/tests/test_pyarrow.py | 50 +-- dask/dataframe/tests/test_pyarrow_compat.py | 18 +- dask/dataframe/tests/test_reshape.py | 33 +- dask/dataframe/tests/test_rolling.py | 6 +- dask/dataframe/tests/test_shuffle.py | 19 +- dask/dataframe/tests/test_utils_dataframe.py | 16 +- dask/dataframe/tseries/resample.py | 13 +- dask/dataframe/utils.py | 4 +- dask/tests/test_spark_compat.py | 5 +- docs/requirements-docs.txt | 2 +- docs/source/changelog.rst | 18 + docs/source/install.rst | 2 +- pyproject.toml | 6 +- 50 files changed, 387 insertions(+), 1486 deletions(-) diff --git a/.github/workflows/tests.yml b/.github/workflows/tests.yml index db02de98111..bfd08e39070 100644 --- a/.github/workflows/tests.yml +++ b/.github/workflows/tests.yml @@ -85,6 +85,13 @@ jobs: if: ${{ matrix.extra == 'pandas-nightly' }} run: python -m pip install --pre --extra-index https://pypi.anaconda.org/scientific-python-nightly-wheels/simple pandas -U + - name: Disable query planning for 3.9 builds + if: ${{ matrix.environment == '3.9' }} + run: | + export DASK_DATAFRAME__QUERY_PLANNING="False" + echo "DASK_DATAFRAME__QUERY_PLANNING: $DASK_DATAFRAME__QUERY_PLANNING" + echo "DASK_DATAFRAME__QUERY_PLANNING=$DASK_DATAFRAME__QUERY_PLANNING" >> $GITHUB_ENV + - name: Reconfigure pytest-timeout # No SIGALRM available on Windows if: ${{ matrix.os != 'windows-latest' }} diff --git a/continuous_integration/environment-3.10.yaml b/continuous_integration/environment-3.10.yaml index 77e73a66dbf..93abdbbc2bb 100644 --- a/continuous_integration/environment-3.10.yaml +++ b/continuous_integration/environment-3.10.yaml @@ -1,5 +1,5 @@ # This job includes coverage -name: test-environment +name: test-environment-310 channels: - conda-forge - nodefaults @@ -25,8 +25,8 @@ dependencies: # Optional dependencies - mimesis - numpy=1.23 - - pandas=1.5 - - numba<0.59.0 # Removed numba.generated_jit which pandas<2.1 uses + - pandas=2.1 + - numba - flask - fastparquet - h5py @@ -39,7 +39,8 @@ dependencies: - pyspark - tiledb - xarray - - sqlalchemy<2 # `pandas=1.5` doesn't support `sqlalchemy=2` + - numexpr>=2.8 # Something is pinning it to 2.7 + - sqlalchemy - pyarrow=10 - coverage - jsonschema @@ -75,3 +76,4 @@ dependencies: - pip - pip: - git+https://github.com/dask/distributed + - git+https://github.com/dask/dask-expr diff --git a/continuous_integration/environment-3.11.yaml b/continuous_integration/environment-3.11.yaml index c00d9eae0bb..5806843d99f 100644 --- a/continuous_integration/environment-3.11.yaml +++ b/continuous_integration/environment-3.11.yaml @@ -25,7 +25,7 @@ dependencies: # Optional dependencies - mimesis - numpy - - pandas=2.0 + - pandas=2 - numba - flask - fastparquet @@ -76,4 +76,4 @@ dependencies: - pip - pip: - git+https://github.com/dask/distributed - - git+https://github.com/dask-contrib/dask-expr + - git+https://github.com/dask/dask-expr diff --git a/continuous_integration/environment-3.12.yaml b/continuous_integration/environment-3.12.yaml index 7e112d4a379..425969ef5dc 100644 --- a/continuous_integration/environment-3.12.yaml +++ b/continuous_integration/environment-3.12.yaml @@ -75,4 +75,4 @@ dependencies: - pip - pip: - git+https://github.com/dask/distributed - - git+https://github.com/dask-contrib/dask-expr + - git+https://github.com/dask/dask-expr diff --git a/continuous_integration/environment-3.9.yaml b/continuous_integration/environment-3.9.yaml index d383afb090c..835626ca7ba 100644 --- a/continuous_integration/environment-3.9.yaml +++ b/continuous_integration/environment-3.9.yaml @@ -25,7 +25,7 @@ dependencies: # Optional dependencies - mimesis<12 - numpy=1.22 - - pandas=1.4 + - pandas=2.0 - numba<0.59.0 # Removed numba.generated_jit which pandas<2.1 uses - flask - fastparquet @@ -39,7 +39,7 @@ dependencies: - pyspark - tiledb - xarray - - sqlalchemy<2 # `pandas=1.4` doesn't support `sqlalchemy=2` + - sqlalchemy - pyarrow=9 - coverage - jsonschema diff --git a/continuous_integration/environment-mindeps-array.yaml b/continuous_integration/environment-mindeps-array.yaml index 8dbaf5f57b8..88e352bfb9c 100644 --- a/continuous_integration/environment-mindeps-array.yaml +++ b/continuous_integration/environment-mindeps-array.yaml @@ -8,7 +8,7 @@ dependencies: - pyyaml=5.3.1 - click=8.1 - cloudpickle=1.5.0 - - partd=1.2.0 + - partd=1.4.0 - fsspec=2021.09.0 - importlib-metadata=4.13.0 - toolz=0.10.0 diff --git a/continuous_integration/environment-mindeps-dataframe.yaml b/continuous_integration/environment-mindeps-dataframe.yaml index f86ad42f804..63d2d49c365 100644 --- a/continuous_integration/environment-mindeps-dataframe.yaml +++ b/continuous_integration/environment-mindeps-dataframe.yaml @@ -8,13 +8,13 @@ dependencies: - pyyaml=5.3.1 - click=8.1 - cloudpickle=1.5.0 - - partd=1.2.0 + - partd=1.4.0 - fsspec=2021.09.0 - importlib-metadata=4.13.0 - toolz=0.10.0 # optional dependencies pulled in by pip install dask[dataframe] - numpy=1.21 - - pandas=1.3 + - pandas=2.0 # test dependencies - pre-commit - pytest @@ -22,3 +22,5 @@ dependencies: - pytest-rerunfailures - pytest-timeout - pytest-xdist + - pip: + - git+https://github.com/dask/dask-expr diff --git a/continuous_integration/environment-mindeps-distributed.yaml b/continuous_integration/environment-mindeps-distributed.yaml index 2bb860291fe..1a1093969c9 100644 --- a/continuous_integration/environment-mindeps-distributed.yaml +++ b/continuous_integration/environment-mindeps-distributed.yaml @@ -8,7 +8,7 @@ dependencies: - pyyaml=5.3.1 - click=8.1 - cloudpickle=1.5.0 - - partd=1.2.0 + - partd=1.4.0 - fsspec=2021.09.0 - importlib-metadata=4.13.0 - toolz=0.10.0 diff --git a/continuous_integration/environment-mindeps-non-optional.yaml b/continuous_integration/environment-mindeps-non-optional.yaml index 01afb450dad..71f970bb682 100644 --- a/continuous_integration/environment-mindeps-non-optional.yaml +++ b/continuous_integration/environment-mindeps-non-optional.yaml @@ -8,7 +8,7 @@ dependencies: - pyyaml=5.3.1 - click=8.1 - cloudpickle=1.5.0 - - partd=1.2.0 + - partd=1.4.0 - fsspec=2021.09.0 - importlib-metadata=4.13.0 - toolz=0.10.0 diff --git a/continuous_integration/environment-mindeps-optional.yaml b/continuous_integration/environment-mindeps-optional.yaml index 3045a2232ab..8a28f807cf9 100644 --- a/continuous_integration/environment-mindeps-optional.yaml +++ b/continuous_integration/environment-mindeps-optional.yaml @@ -8,13 +8,13 @@ dependencies: - pyyaml=5.3.1 - click=8.1 - cloudpickle=1.5.0 - - partd=1.2.0 + - partd=1.4.0 - fsspec=2021.09.0 - importlib-metadata=4.13.0 - toolz=0.10.0 # optional dependencies pulled in by pip install dask[array,dataframe] - numpy=1.21 - - pandas=1.3 + - pandas=2.0 # optional dependencies pulled in by pip install dask[diagnostics] - bokeh=2.4.2 - jinja2=2.10.3 @@ -51,6 +51,7 @@ dependencies: - pip: # optional dependencies pulled in by pip install dask[distributed] - git+https://github.com/dask/distributed + - git+https://github.com/dask/dask-expr # test dependencies - pre-commit - pytest diff --git a/continuous_integration/recipe/meta.yaml b/continuous_integration/recipe/meta.yaml index f5834645f63..2def4eba349 100644 --- a/continuous_integration/recipe/meta.yaml +++ b/continuous_integration/recipe/meta.yaml @@ -31,7 +31,7 @@ requirements: - cloudpickle >=1.5.0 - fsspec >=2021.09.0 - packaging >=20.0 - - partd >=1.2.0 + - partd >=1.4.0 - pyyaml >=5.3.1 - toolz >=0.10.0 - importlib_metadata >=4.13.0 # [py<312] diff --git a/dask/array/percentile.py b/dask/array/percentile.py index 637ad979f09..9eb3c4199a4 100644 --- a/dask/array/percentile.py +++ b/dask/array/percentile.py @@ -36,12 +36,7 @@ def _percentile(a, q, method="linear"): if np.issubdtype(a.dtype, np.datetime64): values = a if type(a).__name__ in ("Series", "Index"): - from dask.dataframe._compat import PANDAS_GE_200 - - if PANDAS_GE_200: - a2 = values.astype("i8") - else: - a2 = values.view("i8") + a2 = values.astype("i8") else: a2 = values.view("i8") result = np_percentile(a2, q, method=method).astype(values.dtype) diff --git a/dask/array/tests/test_array_core.py b/dask/array/tests/test_array_core.py index 842090546c3..df9bf6449be 100644 --- a/dask/array/tests/test_array_core.py +++ b/dask/array/tests/test_array_core.py @@ -4557,17 +4557,10 @@ def test_normalize_chunks_nan(): def test_pandas_from_dask_array(): pd = pytest.importorskip("pandas") - from dask.dataframe._compat import PANDAS_GE_131 - a = da.ones((12,), chunks=4) s = pd.Series(a, index=range(12)) - - if not PANDAS_GE_131: - # https://github.com/pandas-dev/pandas/issues/38645 - assert s.dtype != a.dtype - else: - assert s.dtype == a.dtype - assert_eq(s.values, a) + assert s.dtype == a.dtype + assert_eq(s.values, a) def test_from_zarr_unique_name(): diff --git a/dask/dataframe/__init__.py b/dask/dataframe/__init__.py index 1a3e128f6ca..ca9bf46a891 100644 --- a/dask/dataframe/__init__.py +++ b/dask/dataframe/__init__.py @@ -3,20 +3,17 @@ import importlib import warnings -from packaging.version import Version - # The "dataframe.query-planning" config can only be processed once DASK_EXPR_ENABLED: bool | None = None def _dask_expr_enabled() -> bool: - import pandas as pd - import dask global DASK_EXPR_ENABLED use_dask_expr = dask.config.get("dataframe.query-planning") + if DASK_EXPR_ENABLED is not None: if (use_dask_expr is True and DASK_EXPR_ENABLED is False) or ( use_dask_expr is False and DASK_EXPR_ENABLED is True @@ -30,11 +27,7 @@ def _dask_expr_enabled() -> bool: ) return DASK_EXPR_ENABLED - if ( - use_dask_expr is False - or use_dask_expr is None - and Version(pd.__version__).major < 2 - ): + if use_dask_expr is False: return (DASK_EXPR_ENABLED := False) try: import dask_expr # noqa: F401 diff --git a/dask/dataframe/_compat.py b/dask/dataframe/_compat.py index 89f3518bda2..e163021effd 100644 --- a/dask/dataframe/_compat.py +++ b/dask/dataframe/_compat.py @@ -9,11 +9,7 @@ from packaging.version import Version PANDAS_VERSION = Version(pd.__version__) -PANDAS_GE_131 = PANDAS_VERSION >= Version("1.3.1") -PANDAS_GE_133 = PANDAS_VERSION >= Version("1.3.3") -PANDAS_GE_140 = PANDAS_VERSION >= Version("1.4.0") -PANDAS_GE_150 = PANDAS_VERSION >= Version("1.5.0") -PANDAS_GE_200 = PANDAS_VERSION.major >= 2 +PANDAS_GE_200 = True PANDAS_GE_201 = PANDAS_VERSION.release >= (2, 0, 1) PANDAS_GE_202 = PANDAS_VERSION.release >= (2, 0, 2) PANDAS_GE_210 = PANDAS_VERSION.release >= (2, 1, 0) @@ -87,43 +83,6 @@ def makeMixedDataFrame(): return df -@contextlib.contextmanager -def check_numeric_only_deprecation(name=None, show_nuisance_warning: bool = False): - supported_funcs = ["sum", "median", "prod", "min", "max", "std", "var", "quantile"] - if name not in supported_funcs and PANDAS_GE_150 and not PANDAS_GE_200: - with warnings.catch_warnings(): - warnings.filterwarnings( - "ignore", - message="The default value of numeric_only", - category=FutureWarning, - ) - yield - elif ( - not show_nuisance_warning and name not in supported_funcs and not PANDAS_GE_150 - ): - with warnings.catch_warnings(): - warnings.filterwarnings( - "ignore", - message="Dropping of nuisance columns in DataFrame", - category=FutureWarning, - ) - yield - else: - yield - - -@contextlib.contextmanager -def check_nuisance_columns_warning(): - if not PANDAS_GE_150: - with warnings.catch_warnings(record=True): - warnings.filterwarnings( - "ignore", "Dropping of nuisance columns", FutureWarning - ) - yield - else: - yield - - @contextlib.contextmanager def check_groupby_axis_deprecation(): if PANDAS_GE_210: @@ -224,7 +183,7 @@ def check_applymap_dataframe_deprecation(): @contextlib.contextmanager def check_reductions_runtime_warning(): - if PANDAS_GE_200 and not PANDAS_GE_201: + if not PANDAS_GE_201: with warnings.catch_warnings(): warnings.filterwarnings( "ignore", @@ -236,25 +195,11 @@ def check_reductions_runtime_warning(): yield -if PANDAS_GE_150: - IndexingError = pd.errors.IndexingError -else: - IndexingError = pd.core.indexing.IndexingError +IndexingError = pd.errors.IndexingError def is_any_real_numeric_dtype(arr_or_dtype) -> bool: - try: - # `is_any_real_numeric_dtype` was added in PANDAS_GE_200. - # We can remove this compatibility utility once we only support `pandas>=2.0` - return pd.api.types.is_any_real_numeric_dtype(arr_or_dtype) - except AttributeError: - from pandas.api.types import is_bool_dtype, is_complex_dtype, is_numeric_dtype - - return ( - is_numeric_dtype(arr_or_dtype) - and not is_complex_dtype(arr_or_dtype) - and not is_bool_dtype(arr_or_dtype) - ) + return pd.api.types.is_any_real_numeric_dtype(arr_or_dtype) def is_string_dtype(arr_or_dtype) -> bool: @@ -264,7 +209,4 @@ def is_string_dtype(arr_or_dtype) -> bool: dtype = arr_or_dtype.dtype else: dtype = arr_or_dtype - - if not PANDAS_GE_200: - return pd.api.types.is_dtype_equal(dtype, "string") return pd.api.types.is_string_dtype(dtype) diff --git a/dask/dataframe/_dtypes.py b/dask/dataframe/_dtypes.py index e40bfa99ced..d31eb053920 100644 --- a/dask/dataframe/_dtypes.py +++ b/dask/dataframe/_dtypes.py @@ -5,7 +5,6 @@ import pandas as pd -from dask.dataframe._compat import PANDAS_GE_150 from dask.dataframe.extensions import make_array_nonempty, make_scalar @@ -24,39 +23,37 @@ def _(dtype): return pd.array(["a", pd.NA], dtype=dtype) -if PANDAS_GE_150: - - @make_array_nonempty.register(pd.ArrowDtype) - def _make_array_nonempty_pyarrow_dtype(dtype): - import pyarrow as pa - - if pa.types.is_integer(dtype.pyarrow_dtype): - data = [1, 2] - elif pa.types.is_floating(dtype.pyarrow_dtype): - data = [1.5, 2.5] - elif pa.types.is_boolean(dtype.pyarrow_dtype): - data = [True, False] - elif pa.types.is_string(dtype.pyarrow_dtype) or pa.types.is_large_string( - dtype.pyarrow_dtype - ): - data = ["a", "b"] - elif pa.types.is_timestamp(dtype.pyarrow_dtype): - data = [pd.Timestamp("1970-01-01"), pd.Timestamp("1970-01-02")] - elif pa.types.is_date(dtype.pyarrow_dtype): - data = [date(1970, 1, 1), date(1970, 1, 2)] - elif pa.types.is_binary(dtype.pyarrow_dtype) or pa.types.is_large_binary( - dtype.pyarrow_dtype - ): - data = [b"a", b"b"] - elif pa.types.is_decimal(dtype.pyarrow_dtype): - data = [Decimal("1"), Decimal("0.0")] - elif pa.types.is_duration(dtype.pyarrow_dtype): - data = [pd.Timedelta("1 day"), pd.Timedelta("2 days")] - elif pa.types.is_time(dtype.pyarrow_dtype): - data = [time(12, 0), time(0, 12)] - else: - data = dtype.empty(2) - return pd.array(data, dtype=dtype) +@make_array_nonempty.register(pd.ArrowDtype) +def _make_array_nonempty_pyarrow_dtype(dtype): + import pyarrow as pa + + if pa.types.is_integer(dtype.pyarrow_dtype): + data = [1, 2] + elif pa.types.is_floating(dtype.pyarrow_dtype): + data = [1.5, 2.5] + elif pa.types.is_boolean(dtype.pyarrow_dtype): + data = [True, False] + elif pa.types.is_string(dtype.pyarrow_dtype) or pa.types.is_large_string( + dtype.pyarrow_dtype + ): + data = ["a", "b"] + elif pa.types.is_timestamp(dtype.pyarrow_dtype): + data = [pd.Timestamp("1970-01-01"), pd.Timestamp("1970-01-02")] + elif pa.types.is_date(dtype.pyarrow_dtype): + data = [date(1970, 1, 1), date(1970, 1, 2)] + elif pa.types.is_binary(dtype.pyarrow_dtype) or pa.types.is_large_binary( + dtype.pyarrow_dtype + ): + data = [b"a", b"b"] + elif pa.types.is_decimal(dtype.pyarrow_dtype): + data = [Decimal("1"), Decimal("0.0")] + elif pa.types.is_duration(dtype.pyarrow_dtype): + data = [pd.Timedelta("1 day"), pd.Timedelta("2 days")] + elif pa.types.is_time(dtype.pyarrow_dtype): + data = [time(12, 0), time(0, 12)] + else: + data = dtype.empty(2) + return pd.array(data, dtype=dtype) @make_scalar.register(str) diff --git a/dask/dataframe/_pyarrow.py b/dask/dataframe/_pyarrow.py index 2c44ee80e98..cdf956bc065 100644 --- a/dask/dataframe/_pyarrow.py +++ b/dask/dataframe/_pyarrow.py @@ -5,7 +5,6 @@ import pandas as pd from packaging.version import Version -from dask.dataframe._compat import PANDAS_GE_150, PANDAS_GE_200 from dask.dataframe.utils import is_dataframe_like, is_index_like, is_series_like try: @@ -18,12 +17,7 @@ def is_pyarrow_string_dtype(dtype): """Is the input dtype a pyarrow string?""" if pa is None: return False - - if PANDAS_GE_150: - pa_string_types = [pd.StringDtype("pyarrow"), pd.ArrowDtype(pa.string())] - else: - pa_string_types = [pd.StringDtype("pyarrow")] - return dtype in pa_string_types + return dtype in (pd.StringDtype("pyarrow"), pd.ArrowDtype(pa.string())) def is_object_string_dtype(dtype): @@ -113,11 +107,6 @@ def _to_string_dtype(df, dtype_check, index_check, string_dtype): def check_pyarrow_string_supported(): """Make sure we have all the required versions""" - if not PANDAS_GE_200: - raise RuntimeError( - "Using dask's `dataframe.convert-string` configuration " - "option requires `pandas>=2.0` to be installed." - ) if pa is None or Version(pa.__version__) < Version("12.0.0"): raise RuntimeError( "Using dask's `dataframe.convert-string` configuration " diff --git a/dask/dataframe/_pyarrow_compat.py b/dask/dataframe/_pyarrow_compat.py index 58839d182cd..b3a996ed8af 100644 --- a/dask/dataframe/_pyarrow_compat.py +++ b/dask/dataframe/_pyarrow_compat.py @@ -1,9 +1,7 @@ from __future__ import annotations -import copyreg import warnings -import pandas as pd from packaging.version import Version try: @@ -29,8 +27,6 @@ except ImportError: pa = None -from dask.dataframe._compat import PANDAS_GE_150, PANDAS_GE_200 - # Pickling of pyarrow arrays is effectively broken - pickling a slice of an # array ends up pickling the entire backing array. # @@ -54,16 +50,3 @@ def rebuild_arrowextensionarray(type_, chunks): def reduce_arrowextensionarray(x): return (rebuild_arrowextensionarray, (type(x), x._data.combine_chunks())) - - -# `pandas=2` includes efficient serialization of `pyarrow`-backed extension arrays. -# See https://github.com/pandas-dev/pandas/pull/49078 for details. -# We only need to backport efficient serialization for `pandas<2`. -if pa is not None and not PANDAS_GE_200: - if PANDAS_GE_150: - # Applies to all `pyarrow`-backed extension arrays (e.g. `string[pyarrow]`, `int64[pyarrow]`) - for type_ in [pd.arrays.ArrowExtensionArray, pd.arrays.ArrowStringArray]: - copyreg.dispatch_table[type_] = reduce_arrowextensionarray - else: - # Only `string[pyarrow]` is implemented, so just patch that - copyreg.dispatch_table[pd.arrays.ArrowStringArray] = reduce_arrowextensionarray diff --git a/dask/dataframe/core.py b/dask/dataframe/core.py index 8788b1553d9..1ba9d0eca6b 100644 --- a/dask/dataframe/core.py +++ b/dask/dataframe/core.py @@ -35,15 +35,10 @@ from dask.context import globalmethod from dask.dataframe import methods from dask.dataframe._compat import ( - PANDAS_GE_140, - PANDAS_GE_150, - PANDAS_GE_200, PANDAS_GE_210, PANDAS_GE_300, PANDAS_VERSION, check_convert_dtype_deprecation, - check_nuisance_columns_warning, - check_numeric_only_deprecation, check_reductions_runtime_warning, is_string_dtype, ) @@ -109,8 +104,6 @@ DEFAULT_GET = named_schedulers.get("threads", named_schedulers["sync"]) GROUP_KEYS_DEFAULT: bool | None = True -if PANDAS_GE_150 and not PANDAS_GE_200: - GROUP_KEYS_DEFAULT = None pd.set_option("compute.use_numexpr", False) @@ -152,12 +145,8 @@ def _numeric_only_maybe_warn(df, numeric_only, default=None): TODO: should move to numeric_only decorator. See https://github.com/dask/dask/pull/9952 """ if is_dataframe_like(df): - warn_numeric_only = False if numeric_only is no_default: - if PANDAS_GE_200: - numeric_only = False - else: - warn_numeric_only = True + numeric_only = False numerics = df._meta._get_numeric_data() has_non_numerics = len(numerics.columns) < len(df._meta.columns) @@ -166,12 +155,6 @@ def _numeric_only_maybe_warn(df, numeric_only, default=None): raise NotImplementedError( "'numeric_only=False' is not implemented in Dask." ) - elif warn_numeric_only: - warnings.warn( - "The default value of numeric_only in dask will be changed to False in " - "the future when using dask with pandas 2.0", - FutureWarning, - ) if numeric_only is no_default and default is not None: numeric_only = default return {} if numeric_only is no_default else {"numeric_only": numeric_only} @@ -478,7 +461,6 @@ def __init__(self, dsk, name, meta, divisions): from dask.dataframe._pyarrow import ( is_object_string_dataframe, - is_object_string_dtype, is_object_string_index, is_object_string_series, to_pyarrow_string, @@ -489,22 +471,6 @@ def __init__(self, dsk, name, meta, divisions): or is_object_string_series(meta) or is_object_string_index(meta) ): - # Prior to pandas=1.4, `pd.Index` couldn't contain extension dtypes. - # Here we don't cast objects to pyarrow strings where only the index - # contains non-pyarrow string data. - if not PANDAS_GE_140 and ( - ( - is_object_string_dataframe(meta) - and not any(is_object_string_dtype(d) for d in meta.dtypes) - ) - or ( - is_object_string_series(meta) - and not is_object_string_dtype(meta.dtype) - ) - or is_object_string_index(meta) - ): - return - # this is an internal call, and if we enforce metadata, # it may interfere when reading csv with enforce=False result = self.map_partitions( @@ -2008,7 +1974,6 @@ def sample(self, n=None, frac=None, replace=False, random_state=None): @derived_from(pd.DataFrame) def replace(self, to_replace=None, value=None, regex=False): - # In PANDAS_GE_140 pandas starts using no_default instead of None value_kwarg = {"value": value} if value is not None else {} return self.map_partitions( M.replace, @@ -2281,41 +2246,38 @@ def _reduction_agg( split_every=False, out=None, numeric_only=None, - none_is_zero=True, + none_is_zero: bool = True, ): - axis = self._validate_axis(axis, none_is_zero=none_is_zero) + axis = self._validate_axis(axis, none_is_zero=none_is_zero) # type: ignore if has_keyword(getattr(self._meta_nonempty, name), "numeric_only"): numeric_only_kwargs = {"numeric_only": numeric_only} else: numeric_only_kwargs = {} - with check_numeric_only_deprecation(name, True): - meta = getattr(self._meta_nonempty, name)( - axis=axis, skipna=skipna, **numeric_only_kwargs - ) + meta = getattr(self._meta_nonempty, name)( + axis=axis, skipna=skipna, **numeric_only_kwargs + ) - token = self._token_prefix + name + token = self._token_prefix + name # type: ignore if axis == 1: result = self.map_partitions( - _getattr_numeric_only, + getattr(M, name), meta=meta, token=token, skipna=skipna, axis=axis, - _dask_method_name=name, **numeric_only_kwargs, ) return handle_out(out, result) else: result = self.reduction( - _getattr_numeric_only, + getattr(M, name), meta=meta, token=token, skipna=skipna, axis=axis, split_every=split_every, - _dask_method_name=name, **numeric_only_kwargs, ) if isinstance(self, DataFrame) and isinstance(result, Series): @@ -2423,55 +2385,27 @@ def prod( @_dummy_numpy_dispatcher("out", deprecated=True) @derived_from(pd.DataFrame) def max(self, axis=0, skipna=True, split_every=False, out=None, numeric_only=None): - if ( - PANDAS_GE_140 - and not PANDAS_GE_200 - and axis is None - and isinstance(self, DataFrame) - ): - warnings.warn( - "In a future version, DataFrame.max(axis=None) will return a scalar max over the entire DataFrame. " - "To retain the old behavior, use 'frame.max(axis=0)' or just 'frame.max()'", - FutureWarning, - ) - axis = 0 - return self._reduction_agg( "max", axis=axis, skipna=skipna, split_every=split_every, out=out, - # Starting in pandas 2.0, `axis=None` does a full aggregation across both axes - none_is_zero=not PANDAS_GE_200, numeric_only=numeric_only, + none_is_zero=False, ) @_dummy_numpy_dispatcher("out", deprecated=True) @derived_from(pd.DataFrame) def min(self, axis=0, skipna=True, split_every=False, out=None, numeric_only=None): - if ( - PANDAS_GE_140 - and not PANDAS_GE_200 - and axis is None - and isinstance(self, DataFrame) - ): - warnings.warn( - "In a future version, DataFrame.min(axis=None) will return a scalar min over the entire DataFrame. " - "To retain the old behavior, use 'frame.min(axis=0)' or just 'frame.min()'", - FutureWarning, - ) - axis = 0 - return self._reduction_agg( "min", axis=axis, skipna=skipna, split_every=split_every, out=out, - # Starting in pandas 2.0, `axis=None` does a full aggregation across both axes - none_is_zero=not PANDAS_GE_200, numeric_only=numeric_only, + none_is_zero=False, ) @derived_from(pd.DataFrame) @@ -2614,24 +2548,11 @@ def mean( out=None, numeric_only=None, ): - if ( - PANDAS_GE_140 - and not PANDAS_GE_200 - and axis is None - and isinstance(self, DataFrame) - ): - warnings.warn( - "In a future version, DataFrame.mean(axis=None) will return a scalar mean over the entire DataFrame. " - "To retain the old behavior, use 'frame.mean(axis=0)' or just 'frame.mean()'", - FutureWarning, - ) - axis = self._validate_axis(axis, none_is_zero=not PANDAS_GE_200) + axis = self._validate_axis(axis, none_is_zero=False) _raise_if_object_series(self, "mean") - # NOTE: Do we want to warn here? - with check_numeric_only_deprecation(), check_nuisance_columns_warning(): - meta = self._meta_nonempty.mean( - axis=axis, skipna=skipna, numeric_only=numeric_only - ) + meta = self._meta_nonempty.mean( + axis=axis, skipna=skipna, numeric_only=numeric_only + ) if axis == 1: result = map_partitions( M.mean, @@ -2649,7 +2570,7 @@ def mean( s = num.sum(skipna=skipna, split_every=split_every) n = num.count(split_every=split_every) # Starting in pandas 2.0, `axis=None` does a full aggregation across both axes - if PANDAS_GE_200 and axis is None and isinstance(self, DataFrame): + if axis is None and isinstance(self, DataFrame): result = s.sum() / n.sum() else: name = self._token_prefix + "mean-%s" % tokenize(self, axis, skipna) @@ -2709,9 +2630,7 @@ def var( axis = self._validate_axis(axis) _raise_if_object_series(self, "var") numeric_only_kwargs = get_numeric_only_kwargs(numeric_only) - with check_numeric_only_deprecation( - "var", True - ), check_reductions_runtime_warning(): + with check_reductions_runtime_warning(): meta = self._meta_nonempty.var( axis=axis, skipna=skipna, @@ -2735,7 +2654,7 @@ def var( result = self._var_1d(self, skipna, ddof, split_every) return handle_out(out, result) - # pandas 1.0+ does not implement var on timedelta + # pandas does not implement var on timedelta result = self._var_numeric(skipna, ddof, split_every) if isinstance(self, DataFrame): @@ -2815,9 +2734,7 @@ def std( _raise_if_not_series_or_dataframe(self, "std") numeric_kwargs = get_numeric_only_kwargs(numeric_only) - with check_numeric_only_deprecation( - "std", True - ), check_reductions_runtime_warning(): + with check_reductions_runtime_warning(): meta = self._meta_nonempty.std(axis=axis, skipna=skipna, **numeric_kwargs) is_df_like = is_dataframe_like(self._meta) needs_time_conversion = False @@ -2937,11 +2854,8 @@ def skew( Further, this method currently does not support filtering out NaN values, which is again a difference to Pandas. """ - if PANDAS_GE_200 and axis is None: - raise ValueError( - "`axis=None` isn't currently supported for `skew` when using `pandas >=2` " - f"(pandas={str(PANDAS_VERSION)} is installed)." - ) + if axis is None: + raise ValueError("`axis=None` isn't currently supported for `skew`.") axis = self._validate_axis(axis) _raise_if_object_series(self, "skew") numeric_only_kwargs = get_numeric_only_kwargs(numeric_only) @@ -3058,11 +2972,8 @@ def kurtosis( Further, this method currently does not support filtering out NaN values, which is again a difference to Pandas. """ - if PANDAS_GE_200 and axis is None: - raise ValueError( - "`axis=None` isn't currently supported for `kurtosis` when using `pandas >=2` " - f"(pandas={str(PANDAS_VERSION)} is installed)." - ) + if axis is None: + raise ValueError("`axis=None` isn't currently supported for `kurtosis`.") axis = self._validate_axis(axis) _raise_if_object_series(self, "kurtosis") numeric_only_kwargs = get_numeric_only_kwargs(numeric_only) @@ -3173,10 +3084,9 @@ def _kurtosis_numeric(self, fisher=True, bias=True, nan_policy="propagate"): def sem(self, axis=None, skipna=True, ddof=1, split_every=False, numeric_only=None): axis = self._validate_axis(axis) _raise_if_object_series(self, "sem") - with check_numeric_only_deprecation(): - meta = self._meta_nonempty.sem( - axis=axis, skipna=skipna, ddof=ddof, numeric_only=numeric_only - ) + meta = self._meta_nonempty.sem( + axis=axis, skipna=skipna, ddof=ddof, numeric_only=numeric_only + ) if axis == 1: return map_partitions( M.sem, @@ -3224,9 +3134,7 @@ def quantile(self, q=0.5, axis=0, numeric_only=no_default, method="default"): axis = self._validate_axis(axis) keyname = "quantiles-concat--" + tokenize(self, q, axis) numeric_kwargs = get_numeric_only_kwargs(numeric_only) - - with check_numeric_only_deprecation("quantile", True): - meta = self._meta.quantile(q, axis=axis, **numeric_kwargs) + meta = self._meta.quantile(q, axis=axis, **numeric_kwargs) if axis == 1: if isinstance(q, list): @@ -3234,7 +3142,7 @@ def quantile(self, q=0.5, axis=0, numeric_only=no_default, method="default"): raise ValueError("'q' must be scalar when axis=1 is specified") return map_partitions( - _getattr_numeric_only, + M.quantile, self, q, axis, @@ -3243,16 +3151,10 @@ def quantile(self, q=0.5, axis=0, numeric_only=no_default, method="default"): **numeric_kwargs, meta=(q, "f8"), parent_meta=self._meta, - _dask_method_name="quantile", ) else: _raise_if_object_series(self, "quantile") - num = ( - self._get_numeric_data() - if numeric_only is True - or (not PANDAS_GE_200 and numeric_only is no_default) - else self - ) + num = self._get_numeric_data() if numeric_only is True else self quantiles = tuple( quantile(num.iloc[:, i], q, method) for i in range(len(num.columns)) ) @@ -3283,39 +3185,21 @@ def describe( percentiles_method="default", include=None, exclude=None, - datetime_is_numeric=no_default, ): - if PANDAS_GE_200: - if datetime_is_numeric is no_default: - datetime_is_numeric = True - datetime_is_numeric_kwarg = {} - else: - raise TypeError( - "datetime_is_numeric is removed in pandas>=2.0.0, datetime data will always be " - "summarized as numeric" - ) - else: - datetime_is_numeric = ( - False if datetime_is_numeric is no_default else datetime_is_numeric - ) - datetime_is_numeric_kwarg = {"datetime_is_numeric": datetime_is_numeric} - if self._meta.ndim == 1: meta = self._meta_nonempty.describe( percentiles=percentiles, include=include, exclude=exclude, - **datetime_is_numeric_kwarg, ) output = self._describe_1d( - self, split_every, percentiles, percentiles_method, datetime_is_numeric + self, split_every, percentiles, percentiles_method ) output._meta = meta return output elif (include is None) and (exclude is None): _include = [np.number, np.timedelta64] - if datetime_is_numeric: - _include.append(np.datetime64) + _include.append(np.datetime64) data = self._meta.select_dtypes(include=_include) # when some numerics/timedeltas are found, by default keep them @@ -3324,8 +3208,7 @@ def describe( else: # check if there are timedelta, boolean, or datetime columns _include = [np.timedelta64, bool] - if datetime_is_numeric: - _include.append(np.datetime64) + _include.append(np.datetime64) bools_and_times = self._meta.select_dtypes(include=_include) if len(bools_and_times.columns) == 0: return self._describe_numeric( @@ -3353,7 +3236,6 @@ def describe( split_every, percentiles, percentiles_method, - datetime_is_numeric, ) for col_idx in chosen_columns_indexes ] @@ -3362,9 +3244,7 @@ def describe( name = "describe--" + tokenize(self, split_every) layer = {(name, 0): (methods.describe_aggregate, stats_names)} graph = HighLevelGraph.from_collections(name, layer, dependencies=stats) - meta = self._meta_nonempty.describe( - include=include, exclude=exclude, **datetime_is_numeric_kwarg - ) + meta = self._meta_nonempty.describe(include=include, exclude=exclude) return new_dd_object(graph, name, meta, divisions=[None, None]) def _get_columns_indexes_based_on_dtypes(self, subset): @@ -3382,12 +3262,9 @@ def _describe_1d( split_every=False, percentiles=None, percentiles_method="default", - datetime_is_numeric=False, ): if is_bool_dtype(data._meta): - return self._describe_nonnumeric_1d( - data, split_every=split_every, datetime_is_numeric=datetime_is_numeric - ) + return self._describe_nonnumeric_1d(data, split_every=split_every) elif is_numeric_dtype(data._meta): return self._describe_numeric( data, @@ -3403,7 +3280,7 @@ def _describe_1d( percentiles_method=percentiles_method, is_timedelta_column=True, ) - elif is_datetime64_any_dtype(data._meta) and datetime_is_numeric: + elif is_datetime64_any_dtype(data._meta): return self._describe_numeric( data.dropna(), split_every=split_every, @@ -3412,9 +3289,7 @@ def _describe_1d( is_datetime_column=True, ) else: - return self._describe_nonnumeric_1d( - data, split_every=split_every, datetime_is_numeric=datetime_is_numeric - ) + return self._describe_nonnumeric_1d(data, split_every=split_every) def _describe_numeric( self, @@ -3471,11 +3346,7 @@ def _describe_numeric( meta = num._meta_nonempty.describe() return new_dd_object(graph, name, meta, divisions=[None, None]) - def _describe_nonnumeric_1d( - self, data, split_every=False, datetime_is_numeric=False - ): - from dask.dataframe.numeric import to_numeric - + def _describe_nonnumeric_1d(self, data, split_every=False): vcounts = data.value_counts(split_every=split_every) count_nonzero = vcounts[vcounts != 0] count_unique = count_nonzero.size @@ -3489,11 +3360,6 @@ def _describe_nonnumeric_1d( vcounts._head(1, npartitions=1, compute=False, safe=False), ] - if is_datetime64_any_dtype(data._meta) and not datetime_is_numeric: - min_ts = to_numeric(data.dropna()).min(split_every=split_every) - max_ts = to_numeric(data.dropna()).max(split_every=split_every) - stats.extend([min_ts, max_ts]) - stats_names = [(s._name, 0) for s in stats] colname = data._meta.name @@ -3502,12 +3368,7 @@ def _describe_nonnumeric_1d( (name, 0): (methods.describe_nonnumeric_aggregate, stats_names, colname) } graph = HighLevelGraph.from_collections(name, layer, dependencies=stats) - - if not PANDAS_GE_200: - datetime_is_numeric_kwarg = {"datetime_is_numeric": datetime_is_numeric} - else: - datetime_is_numeric_kwarg = {} - meta = data._meta_nonempty.describe(**datetime_is_numeric_kwarg) + meta = data._meta_nonempty.describe() return new_dd_object(graph, name, meta, divisions=[None, None]) def _cum_agg( @@ -3712,28 +3573,6 @@ def astype(self, dtype): M.astype, dtype=dtype, meta=meta, enforce_metadata=False ) - if not PANDAS_GE_200: - - @derived_from(pd.Series) - def append(self, other, interleave_partitions=False): - if PANDAS_GE_140: - warnings.warn( - "The frame.append method is deprecated and will be removed from" - "dask in a future version. Use dask.dataframe.concat instead.", - FutureWarning, - ) - # because DataFrame.append will override the method, - # wrap by pd.Series.append docstring - from dask.dataframe.multi import concat - - if isinstance(other, (list, dict)): - msg = "append doesn't support list or dict input" - raise NotImplementedError(msg) - - return concat( - [self, other], join="outer", interleave_partitions=interleave_partitions - ) - @derived_from(pd.Series) def dot(self, other, meta=no_default): if not isinstance(other, _Frame): @@ -4269,27 +4108,6 @@ def __getitem__(self, key): def _get_numeric_data(self, how="any", subset=None): return self - if not PANDAS_GE_200: - - @derived_from(pd.Series) - def iteritems(self): - if PANDAS_GE_150: - warnings.warn( - "iteritems is deprecated and will be removed in a future version. " - "Use .items instead.", - FutureWarning, - ) - # We use the `_` generator below to ensure the deprecation warning above - # is raised when `.iteritems()` is called, not when the first `next()` - # iteration happens - - def _(self): - for i in range(self.npartitions): - s = self.get_partition(i).compute() - yield from s.items() - - return _(self) - @derived_from(pd.Series) def __iter__(self): for i in range(self.npartitions): @@ -4686,19 +4504,6 @@ def __rdivmod__(self, other): res2 = other % self return res1, res2 - if not PANDAS_GE_200: - - @property - @derived_from(pd.Series) - def is_monotonic(self): - if PANDAS_GE_150: - warnings.warn( - "is_monotonic is deprecated and will be removed in a future version. " - "Use is_monotonic_increasing instead.", - FutureWarning, - ) - return self.is_monotonic_increasing - @property @derived_from(pd.Series) def is_monotonic_increasing(self): @@ -4921,19 +4726,6 @@ def map(self, arg, na_action=None, meta=no_default, is_monotonic=False): applied = applied.clear_divisions() return applied - if not PANDAS_GE_200: - - @property - @derived_from(pd.Index) - def is_monotonic(self): - if PANDAS_GE_150: - warnings.warn( - "is_monotonic is deprecated and will be removed in a future version. " - "Use is_monotonic_increasing instead.", - FutureWarning, - ) - return super().is_monotonic_increasing - @property @derived_from(pd.Index) def is_monotonic_increasing(self): @@ -6088,20 +5880,6 @@ def join( shuffle_method=shuffle_method, ) - if not PANDAS_GE_200: - - @derived_from(pd.DataFrame) - def append(self, other, interleave_partitions=False): - if isinstance(other, Series): - msg = ( - "Unable to appending dd.Series to dd.DataFrame." - "Use pd.Series to append as row." - ) - raise ValueError(msg) - elif is_series_like(other): - other = other.to_frame().T - return super().append(other, interleave_partitions=interleave_partitions) - @derived_from(pd.DataFrame) def iterrows(self): for i in range(self.npartitions): @@ -6399,9 +6177,7 @@ def info(self, buf=None, verbose=False, memory_usage=False): if len(self.columns) == 0: lines.append(f"{type(self.index._meta).__name__}: 0 entries") lines.append(f"Empty {type(self).__name__}") - if PANDAS_GE_150: - # pandas dataframe started adding a newline when info is called. - lines.append("") + lines.append("") put_lines(buf, lines) return @@ -7171,7 +6947,7 @@ def _emulate(func, *args, udf=False, **kwargs): Apply a function using args / kwargs. If arguments contain dd.DataFrame / dd.Series, using internal cache (``_meta``) for calculation """ - with raise_on_meta_error(funcname(func), udf=udf), check_numeric_only_deprecation(): + with raise_on_meta_error(funcname(func), udf=udf): return func(*_extract_meta(args, True), **_extract_meta(kwargs, True)) @@ -7642,24 +7418,10 @@ def _cov_corr( raise ValueError("split_every must be an integer >= 2") # Handle selecting numeric data and associated deprecation warning - maybe_warn = False if numeric_only is no_default: - if PANDAS_GE_200: - numeric_only = False - elif PANDAS_GE_150: - maybe_warn = True - numeric_only = True - else: - numeric_only = True + numeric_only = False all_numeric = df._get_numeric_data()._name == df._name - if maybe_warn and not all_numeric: - warnings.warn( - "The default value of numeric_only will be `False` " - "in a future version of Dask.", - FutureWarning, - ) - if numeric_only and not all_numeric: df = df._get_numeric_data() @@ -8319,9 +8081,7 @@ def _reduction_aggregate(x, aca_aggregate=None, **kwargs): def idxmaxmin_chunk(x, fn=None, skipna=True, numeric_only=False): - numeric_only_kwargs = ( - {} if not PANDAS_GE_150 or is_series_like(x) else {"numeric_only": numeric_only} - ) + numeric_only_kwargs = {} if is_series_like(x) else {"numeric_only": numeric_only} minmax = "max" if fn == "idxmax" else "min" if len(x) > 0: idx = getattr(x, fn)(skipna=skipna, **numeric_only_kwargs) @@ -8445,7 +8205,7 @@ def to_datetime(arg, meta=None, **kwargs): meta = meta_series_constructor(arg)([pd.Timestamp("2000", **tz_kwarg)]) meta.index = meta.index.astype(arg.index.dtype) meta.index.name = arg.index.name - if PANDAS_GE_200 and "infer_datetime_format" in kwargs: + if "infer_datetime_format" in kwargs: warnings.warn( "The argument 'infer_datetime_format' is deprecated and will be removed in a future version. " "A strict version of it is now the default, see " @@ -8814,19 +8574,11 @@ def series_map(base_series, map_series): def _convert_to_numeric(series, skipna): - if PANDAS_GE_200: - if skipna: - return series.dropna().astype("i8") - - # series.view("i8") with pd.NaT produces -9223372036854775808 is why we need to do this - return series.astype("i8").mask(series.isnull(), np.nan) - else: - view = "_view" if isinstance(series, Series) else "view" - if skipna: - return getattr(series.dropna(), view)("i8") + if skipna: + return series.dropna().astype("i8") - # series.view("i8") with pd.NaT produces -9223372036854775808 is why we need to do this - return getattr(series, view)("i8").mask(series.isnull(), np.nan) + # series.view("i8") with pd.NaT produces -9223372036854775808 is why we need to do this + return series.astype("i8").mask(series.isnull(), np.nan) def _sqrt_and_convert_to_timedelta(partition, axis, dtype=None, *args, **kwargs): @@ -8867,8 +8619,3 @@ def _raise_if_not_series_or_dataframe(x, funcname): "`%s` is only supported with objects that are Dataframes or Series" % funcname ) - - -def _getattr_numeric_only(*args, _dask_method_name, **kwargs): - with check_numeric_only_deprecation(): - return getattr(M, _dask_method_name)(*args, **kwargs) diff --git a/dask/dataframe/groupby.py b/dask/dataframe/groupby.py index 52cd73f3112..5163727954d 100644 --- a/dask/dataframe/groupby.py +++ b/dask/dataframe/groupby.py @@ -10,18 +10,15 @@ import numpy as np import pandas as pd +from pandas.core.apply import reconstruct_func, validate_func_kwargs from dask.base import is_dask_collection, tokenize from dask.core import flatten from dask.dataframe._compat import ( - PANDAS_GE_140, - PANDAS_GE_150, - PANDAS_GE_200, PANDAS_GE_210, PANDAS_GE_220, PANDAS_GE_300, check_groupby_axis_deprecation, - check_numeric_only_deprecation, check_observed_deprecation, ) from dask.dataframe.core import ( @@ -59,9 +56,6 @@ itemgetter, ) -if PANDAS_GE_140: - from pandas.core.apply import reconstruct_func, validate_func_kwargs - # ############################################# # # GroupBy implementation notes @@ -283,38 +277,6 @@ def _groupby_get_group(df, by_key, get_key, columns): return df.iloc[0:0] -def numeric_only_deprecate_default(func): - """Decorator for methods that should warn when numeric_only is default""" - - @wraps(func) - def wrapper(self, *args, **kwargs): - if isinstance(self, DataFrameGroupBy): - numeric_only = kwargs.get("numeric_only", no_default) - # Prior to `pandas=1.5`, `numeric_only` support wasn't uniformly supported - # in pandas. We don't support `numeric_only=False` in this case. - if not PANDAS_GE_150 and numeric_only is False: - raise NotImplementedError( - "'numeric_only=False' is not implemented in Dask." - ) - if PANDAS_GE_150 and not PANDAS_GE_200 and not self._all_numeric(): - if numeric_only is no_default: - warnings.warn( - "The default value of numeric_only will be changed to False in " - "the future when using dask with pandas 2.0", - FutureWarning, - ) - elif numeric_only is False and funcname(func) in ("sum", "prod"): - warnings.warn( - "Dropping invalid columns is deprecated. In a future version, a TypeError will be raised. " - f"Before calling .{funcname(func)}, select only columns which should be valid for the function", - FutureWarning, - ) - - return func(self, *args, **kwargs) - - return wrapper - - def numeric_only_not_implemented(func): """Decorator for methods that can't handle numeric_only=False""" @@ -328,29 +290,11 @@ def wrapper(self, *args, **kwargs): ) if maybe_raise: numeric_only = kwargs.get("numeric_only", no_default) - # Prior to `pandas=1.5`, `numeric_only` support wasn't uniformly supported - # in pandas. We don't support `numeric_only=False` in this case. - if not PANDAS_GE_150 and numeric_only is False: - raise NotImplementedError( - "'numeric_only=False' is not implemented in Dask." - ) if not self._all_numeric(): - if numeric_only is False or ( - PANDAS_GE_200 and numeric_only is no_default - ): + if numeric_only is False or numeric_only is no_default: raise NotImplementedError( "'numeric_only=False' is not implemented in Dask." ) - if ( - PANDAS_GE_150 - and not PANDAS_GE_200 - and numeric_only is no_default - ): - warnings.warn( - "The default value of numeric_only will be changed to False " - "in the future when using dask with pandas 2.0", - FutureWarning, - ) return func(self, *args, **kwargs) return wrapper @@ -429,11 +373,7 @@ def _groupby_aggregate( with check_observed_deprecation(): grouped = df.groupby(level=levels, sort=sort, **observed, **dropna) - - # we emit a warning earlier in stack about default numeric_only being deprecated, - # so there's no need to propagate the warning that pandas emits as well - with check_numeric_only_deprecation(): - return aggfunc(grouped, **kwargs) + return aggfunc(grouped, **kwargs) def _groupby_aggregate_spec( @@ -527,8 +467,7 @@ def _var_chunk(df, *by, numeric_only=no_default, observed=False, dropna=True): df = df.copy() g = _groupby_raise_unaligned(df, by=by, observed=observed, dropna=dropna) - with check_numeric_only_deprecation(): - x = g.sum(**numeric_only_kwargs) + x = g.sum(**numeric_only_kwargs) n = g[x.columns].count().rename(columns=lambda c: (c, "-count")) @@ -536,8 +475,7 @@ def _var_chunk(df, *by, numeric_only=no_default, observed=False, dropna=True): df[cols] = df[cols] ** 2 g2 = _groupby_raise_unaligned(df, by=by, observed=observed, dropna=dropna) - with check_numeric_only_deprecation(): - x2 = g2.sum(**numeric_only_kwargs).rename(columns=lambda c: (c, "-x2")) + x2 = g2.sum(**numeric_only_kwargs).rename(columns=lambda c: (c, "-x2")) return concat([x, x2, n], axis=1) @@ -755,11 +693,8 @@ def _nunique_df_chunk(df, *by, **kwargs): return df.drop_duplicates(subset=list(by) + [name]).set_index(list(by)) except Exception: pass - group_keys = {} - if PANDAS_GE_150: - group_keys["group_keys"] = True - g = _groupby_raise_unaligned(df, by=by, **group_keys) + g = _groupby_raise_unaligned(df, by=by, group_keys=True) if len(df) > 0: grouped = g[name].unique().explode().to_frame() else: @@ -1574,8 +1509,7 @@ def _single_agg( aggregate_kwargs = {} if meta is None: - with check_numeric_only_deprecation(): - meta = func(self._meta_nonempty, **chunk_kwargs) + meta = func(self._meta_nonempty, **chunk_kwargs) if columns is None: columns = meta.name if is_series_like(meta) else meta.columns @@ -1843,7 +1777,6 @@ def cumcount(self, axis=no_default): @_deprecated_kwarg("shuffle", "shuffle_method") @derived_from(pd.core.groupby.GroupBy) - @numeric_only_deprecate_default def sum( self, split_every=None, @@ -1869,7 +1802,6 @@ def sum( @_deprecated_kwarg("shuffle", "shuffle_method") @derived_from(pd.core.groupby.GroupBy) - @numeric_only_deprecate_default def prod( self, split_every=None, @@ -1935,7 +1867,6 @@ def max( @_deprecated_kwarg("shuffle", "shuffle_method") @derived_from(pd.DataFrame) - @numeric_only_deprecate_default def idxmin( self, split_every=None, @@ -1971,7 +1902,6 @@ def idxmin( @_deprecated_kwarg("shuffle", "shuffle_method") @derived_from(pd.DataFrame) - @numeric_only_deprecate_default def idxmax( self, split_every=None, @@ -2029,13 +1959,12 @@ def mean( numeric_only=no_default, ): # We sometimes emit this warning ourselves. We ignore it here so users only see it once. - with check_numeric_only_deprecation(): - s = self.sum( - split_every=split_every, - split_out=split_out, - shuffle_method=shuffle_method, - numeric_only=numeric_only, - ) + s = self.sum( + split_every=split_every, + split_out=split_out, + shuffle_method=shuffle_method, + numeric_only=numeric_only, + ) c = self.count( split_every=split_every, split_out=split_out, shuffle_method=shuffle_method ) @@ -2061,8 +1990,7 @@ def median( shuffle_method = shuffle_method or _determine_split_out_shuffle(True, split_out) numeric_only_kwargs = get_numeric_only_kwargs(numeric_only) - with check_numeric_only_deprecation(name="median"): - meta = self._meta_nonempty.median(**numeric_only_kwargs) + meta = self._meta_nonempty.median(**numeric_only_kwargs) columns = meta.name if is_series_like(meta) else meta.columns by = self.by if isinstance(self.by, list) else [self.by] return _shuffle_aggregate( @@ -2103,9 +2031,6 @@ def size(self, split_every=None, split_out=1, shuffle_method=None): @derived_from(pd.core.groupby.GroupBy) @numeric_only_not_implemented def var(self, ddof=1, split_every=None, split_out=1, numeric_only=no_default): - if not PANDAS_GE_150 and numeric_only is not no_default: - raise TypeError("numeric_only not supported for pandas < 1.5") - levels = _determine_levels(self.by) result = aca( [self.obj, self.by] @@ -2140,16 +2065,13 @@ def var(self, ddof=1, split_every=None, split_out=1, numeric_only=no_default): @derived_from(pd.core.groupby.GroupBy) @numeric_only_not_implemented def std(self, ddof=1, split_every=None, split_out=1, numeric_only=no_default): - if not PANDAS_GE_150 and numeric_only is not no_default: - raise TypeError("numeric_only not supported for pandas < 1.5") # We sometimes emit this warning ourselves. We ignore it here so users only see it once. - with check_numeric_only_deprecation(): - v = self.var( - ddof, - split_every=split_every, - split_out=split_out, - numeric_only=numeric_only, - ) + v = self.var( + ddof, + split_every=split_every, + split_out=split_out, + numeric_only=numeric_only, + ) result = map_partitions(np.sqrt, v, meta=v) return result @@ -2158,8 +2080,6 @@ def corr(self, ddof=1, split_every=None, split_out=1, numeric_only=no_default): """Groupby correlation: corr(X, Y) = cov(X, Y) / (std_x * std_y) """ - if not PANDAS_GE_150 and numeric_only is not no_default: - raise TypeError("numeric_only not supported for pandas < 1.5") return self.cov( split_every=split_every, split_out=split_out, @@ -2181,8 +2101,6 @@ def cov( When `std` is True calculate Correlation """ - if not PANDAS_GE_150 and numeric_only is not no_default: - raise TypeError("numeric_only not supported for pandas < 1.5") numeric_only_kwargs = get_numeric_only_kwargs(numeric_only) levels = _determine_levels(self.by) @@ -2298,15 +2216,14 @@ def aggregate( columns = None order = None column_projection = None - if PANDAS_GE_140: - if isinstance(self, DataFrameGroupBy): - if arg is None: - relabeling, arg, columns, order = reconstruct_func(arg, **kwargs) + if isinstance(self, DataFrameGroupBy): + if arg is None: + relabeling, arg, columns, order = reconstruct_func(arg, **kwargs) - elif isinstance(self, SeriesGroupBy): - relabeling = arg is None - if relabeling: - columns, arg = validate_func_kwargs(kwargs) + elif isinstance(self, SeriesGroupBy): + relabeling = arg is None + if relabeling: + columns, arg = validate_func_kwargs(kwargs) if isinstance(self.obj, DataFrame): if isinstance(self.by, tuple) or np.isscalar(self.by): @@ -2908,7 +2825,7 @@ def fillna(self, value=None, method=None, limit=None, axis=no_default): meta = self._meta_nonempty.apply(func, **kwargs) result = self.apply(func, meta=meta, **kwargs) - if PANDAS_GE_150 and self.group_keys: + if self.group_keys: return result.map_partitions(M.droplevel, self.by) return result @@ -2926,7 +2843,7 @@ def ffill(self, limit=None): meta = self._meta_nonempty.apply(func, **kwargs) result = self.apply(func, meta=meta, **kwargs) - if PANDAS_GE_150 and self.group_keys: + if self.group_keys: return result.map_partitions(M.droplevel, self.by) return result @@ -2943,7 +2860,7 @@ def bfill(self, limit=None): meta = self._meta_nonempty.apply(func, **kwargs) result = self.apply(func, meta=meta, **kwargs) - if PANDAS_GE_150 and self.group_keys: + if self.group_keys: return result.map_partitions(M.droplevel, self.by) return result @@ -3144,7 +3061,7 @@ def value_counts(self, split_every=None, split_out=1, shuffle_method=None): split_every=split_every, split_out=split_out, shuffle_method=shuffle_method, - # in pandas 2.0, Series returned from value_counts have a name + # Series returned from value_counts have a name # different from original object, but here, column name should # still reflect the original object name columns=self._meta.apply(pd.Series).name, @@ -3250,8 +3167,7 @@ def _head_aggregate(series_gb, **kwargs): def _median_aggregate(series_gb, **kwargs): - with check_numeric_only_deprecation(): - return series_gb.median(**kwargs) + return series_gb.median(**kwargs) def _shuffle_aggregate( diff --git a/dask/dataframe/io/json.py b/dask/dataframe/io/json.py index 79df964c6b5..44f30d05ba4 100644 --- a/dask/dataframe/io/json.py +++ b/dask/dataframe/io/json.py @@ -11,7 +11,6 @@ from dask.base import compute as dask_compute from dask.bytes import read_bytes from dask.core import flatten -from dask.dataframe._compat import PANDAS_GE_200, PANDAS_VERSION from dask.dataframe.backends import dataframe_creation_dispatch from dask.dataframe.io.io import from_delayed from dask.dataframe.utils import insert_meta_param_description, make_meta @@ -211,14 +210,8 @@ def read_json( if path_converter is None: path_converter = lambda x: x - # Handle engine string (Pandas>=2.0) + # Handle engine string if isinstance(engine, str): - if not PANDAS_GE_200: - raise ValueError( - f"Pandas>=2.0 is required to pass a string to the " - f"`engine` argument of `read_json` " - f"(pandas={str(PANDAS_VERSION)} is currently installed)." - ) engine = partial(pd.read_json, engine=engine) if blocksize: diff --git a/dask/dataframe/io/tests/test_csv.py b/dask/dataframe/io/tests/test_csv.py index 80bff226357..193d3122628 100644 --- a/dask/dataframe/io/tests/test_csv.py +++ b/dask/dataframe/io/tests/test_csv.py @@ -22,13 +22,7 @@ from dask.bytes.core import read_bytes from dask.bytes.utils import compress from dask.core import flatten -from dask.dataframe._compat import ( - PANDAS_GE_140, - PANDAS_GE_200, - PANDAS_GE_220, - PANDAS_GE_300, - tm, -) +from dask.dataframe._compat import PANDAS_GE_220, PANDAS_GE_300, tm from dask.dataframe.io.csv import ( _infer_block_size, auto_blocksize, @@ -383,11 +377,11 @@ def test_read_csv(dd_read, pd_read, text, sep): assert_eq(result, pd_read(fn, sep=sep)) -@pytest.mark.skipif( - not PANDAS_GE_200, reason="dataframe.convert-string requires pandas>=2.0" -) def test_read_csv_convert_string_config(): - pytest.importorskip("pyarrow", reason="Requires pyarrow strings") + pa = pytest.importorskip("pyarrow", reason="Requires pyarrow strings") + pyarrow_version = Version(pa.__version__) + if pyarrow_version.major < 12: + pytest.skip("requires arrow 12") with filetext(csv_text) as fn: df = pd.read_csv(fn) with dask.config.set({"dataframe.convert-string": True}): @@ -1266,7 +1260,6 @@ def test_read_csv_singleton_dtype(): assert_eq(pd.read_csv(fn, dtype=float), dd.read_csv(fn, dtype=float)) -@pytest.mark.skipif(not PANDAS_GE_140, reason="arrow engine available from 1.4") def test_read_csv_arrow_engine(): pytest.importorskip("pyarrow") sep_text = normalize_text( @@ -1826,6 +1819,7 @@ def test_csv_getitem_column_order(tmpdir): @pytest.mark.skip_with_pyarrow_strings # checks graph layers +@pytest.mark.skipif(dd._dask_expr_enabled(), reason="layers not supported") def test_getitem_optimization_after_filter(): with filetext(timeseries) as fn: expect = pd.read_csv(fn) diff --git a/dask/dataframe/io/tests/test_demo.py b/dask/dataframe/io/tests/test_demo.py index 11faac6bf9e..4481f65a489 100644 --- a/dask/dataframe/io/tests/test_demo.py +++ b/dask/dataframe/io/tests/test_demo.py @@ -6,7 +6,7 @@ import dask import dask.dataframe as dd from dask.blockwise import Blockwise, optimize_blockwise -from dask.dataframe._compat import PANDAS_GE_200, PANDAS_GE_220, tm +from dask.dataframe._compat import PANDAS_GE_220, tm from dask.dataframe.optimize import optimize_dataframe_getitem from dask.dataframe.utils import assert_eq, get_string_dtype @@ -94,6 +94,7 @@ def test_make_timeseries_no_args(): @pytest.mark.skip_with_pyarrow_strings # checks graph layers +@pytest.mark.skipif(dd._dask_expr_enabled(), reason="layers not supported") def test_make_timeseries_blockwise(): df = dd.demo.make_timeseries() df = df[["x", "y"]] @@ -229,8 +230,7 @@ def test_with_spec_non_default(seed): ddf = with_spec(spec, seed=seed) assert isinstance(ddf, dd.DataFrame) assert ddf.columns.tolist() == ["i1", "f1", "c1", "s1"] - if PANDAS_GE_200: - assert ddf.index.dtype == "int32" + assert ddf.index.dtype == "int32" assert ddf["i1"].dtype == "int32" assert ddf["f1"].dtype == "float32" assert ddf["c1"].dtype.name == "category" diff --git a/dask/dataframe/io/tests/test_hdf.py b/dask/dataframe/io/tests/test_hdf.py index 6fc3e15f7a6..929b5d82b4e 100644 --- a/dask/dataframe/io/tests/test_hdf.py +++ b/dask/dataframe/io/tests/test_hdf.py @@ -58,6 +58,7 @@ def test_to_hdf(): PY_VERSION >= Version("3.11"), reason="segfaults due to https://github.com/PyTables/PyTables/issues/977", ) +@pytest.mark.skipif(dd._dask_expr_enabled(), reason="layers not supported") def test_to_hdf_multiple_nodes(): pytest.importorskip("tables") df = pd.DataFrame( diff --git a/dask/dataframe/io/tests/test_io.py b/dask/dataframe/io/tests/test_io.py index 1cca2ee7928..e2dc628dbc2 100644 --- a/dask/dataframe/io/tests/test_io.py +++ b/dask/dataframe/io/tests/test_io.py @@ -5,13 +5,14 @@ import numpy as np import pandas as pd import pytest +from packaging.version import Version import dask import dask.array as da import dask.dataframe as dd from dask import config from dask.blockwise import Blockwise -from dask.dataframe._compat import PANDAS_GE_200, tm +from dask.dataframe._compat import tm from dask.dataframe.io.io import _meta_from_array from dask.dataframe.optimize import optimize from dask.dataframe.utils import assert_eq, get_string_dtype, pyarrow_strings_enabled @@ -283,11 +284,11 @@ def test_from_pandas_npartitions_duplicates(index): assert ddf.divisions == ("A", "B", "C", "C") -@pytest.mark.skipif( - not PANDAS_GE_200, reason="dataframe.convert-string requires pandas>=2.0" -) def test_from_pandas_convert_string_config(): - pytest.importorskip("pyarrow", reason="Requires pyarrow strings") + pa = pytest.importorskip("pyarrow", reason="Requires pyarrow strings") + pyarrow_version = Version(pa.__version__) + if pyarrow_version.major < 12: + pytest.skip("requires arrow 12") # With `dataframe.convert-string=False`, strings should remain objects with dask.config.set({"dataframe.convert-string": False}): @@ -321,24 +322,6 @@ def test_from_pandas_convert_string_config(): assert_eq(df_pyarrow, ddf) -@pytest.mark.skipif(PANDAS_GE_200, reason="Requires pandas<2.0") -def test_from_pandas_convert_string_config_raises(): - pytest.importorskip("pyarrow", reason="Different error without pyarrow") - df = pd.DataFrame( - { - "x": [1, 2, 3, 4], - "y": [5.0, 6.0, 7.0, 8.0], - "z": ["foo", "bar", "ricky", "bobby"], - }, - index=["a", "b", "c", "d"], - ) - with dask.config.set({"dataframe.convert-string": True}): - with pytest.raises( - RuntimeError, match="requires `pandas>=2.0` to be installed" - ): - dd.from_pandas(df, npartitions=2) - - @pytest.mark.parametrize("index", [[1, 2, 3], [3, 2, 1]]) @pytest.mark.parametrize("sort", [True, False]) def test_from_pandas_immutable(sort, index): diff --git a/dask/dataframe/io/tests/test_json.py b/dask/dataframe/io/tests/test_json.py index 8eeca100bb2..3ad1924d9c8 100644 --- a/dask/dataframe/io/tests/test_json.py +++ b/dask/dataframe/io/tests/test_json.py @@ -10,7 +10,6 @@ import dask import dask.dataframe as dd -from dask.dataframe._compat import PANDAS_GE_200 from dask.dataframe.utils import assert_eq from dask.utils import tmpdir, tmpfile @@ -129,12 +128,8 @@ def _my_json_reader(*args, **kwargs): def test_read_json_engine_str(engine): with tmpfile("json") as f: df.to_json(f, lines=False) - if isinstance(engine, str) and not PANDAS_GE_200: - with pytest.raises(ValueError, match="Pandas>=2.0 is required"): - dd.read_json(f, engine=engine, lines=False) - else: - got = dd.read_json(f, engine=engine, lines=False) - assert_eq(got, df) + got = dd.read_json(f, engine=engine, lines=False) + assert_eq(got, df) @pytest.mark.parametrize("orient", ["split", "records", "index", "columns", "values"]) diff --git a/dask/dataframe/io/tests/test_parquet.py b/dask/dataframe/io/tests/test_parquet.py index 65add0c71af..46b976c3a12 100644 --- a/dask/dataframe/io/tests/test_parquet.py +++ b/dask/dataframe/io/tests/test_parquet.py @@ -20,7 +20,7 @@ import dask.multiprocessing from dask.array.numpy_compat import NUMPY_GE_124 from dask.blockwise import Blockwise, optimize_blockwise -from dask.dataframe._compat import PANDAS_GE_150, PANDAS_GE_200, PANDAS_GE_202 +from dask.dataframe._compat import PANDAS_GE_202 from dask.dataframe.io.parquet.core import get_engine from dask.dataframe.io.parquet.utils import _parse_pandas_metadata from dask.dataframe.optimize import optimize_dataframe_getitem @@ -610,12 +610,7 @@ def test_roundtrip_nullable_dtypes(tmp_path): "dtype_backend", [ "pandas", - pytest.param( - "pyarrow", - marks=pytest.mark.skipif( - not PANDAS_GE_150, reason="Requires pyarrow-backed nullable dtypes" - ), - ), + "pyarrow", ], ) def test_use_nullable_dtypes(tmp_path, dtype_backend, engine): @@ -1154,11 +1149,7 @@ def test_roundtrip(tmpdir, df, write_kwargs, read_kwargs, engine): ): pytest.xfail(reason="fastparquet doesn't support nanosecond precision yet") # non-ns times - if ( - PANDAS_GE_200 - and "x" in df - and (df.x.dtype == "M8[ms]" or df.x.dtype == "M8[us]") - ): + if "x" in df and (df.x.dtype == "M8[ms]" or df.x.dtype == "M8[us]"): if engine == "pyarrow": pytest.xfail("https://github.com/apache/arrow/issues/15079") elif engine == "fastparquet" and fastparquet_version <= Version("2022.12.0"): @@ -2366,14 +2357,14 @@ def test_append_cat_fp(tmpdir, engine): pytest.param( pd.DataFrame({"x": [3, 2, 1]}).astype("M8[us]"), marks=pytest.mark.xfail( - PANDAS_GE_200 and pyarrow_version < Version("13.0.0.dev"), + pyarrow_version < Version("13.0.0.dev"), reason="https://github.com/apache/arrow/issues/15079", ), ), pytest.param( pd.DataFrame({"x": [3, 2, 1]}).astype("M8[ms]"), marks=pytest.mark.xfail( - PANDAS_GE_200 and pyarrow_version < Version("13.0.0.dev"), + pyarrow_version < Version("13.0.0.dev"), reason="https://github.com/apache/arrow/issues/15079", ), ), @@ -3516,7 +3507,7 @@ def test_partitioned_preserve_index(tmpdir, write_engine, read_engine): df1.to_parquet(tmp, partition_on="B", engine=write_engine) expect = data[data["B"] == 1] - if PANDAS_GE_200 and read_engine == "fastparquet": + if read_engine == "fastparquet": # fastparquet does not preserve dtype of cats expect = expect.copy() # SettingWithCopyWarning expect["B"] = expect["B"].astype( @@ -4437,7 +4428,7 @@ def test_custom_filename_with_partition(tmpdir, engine): def test_roundtrip_partitioned_pyarrow_dataset(tmpdir, engine): # See: https://github.com/dask/dask/issues/8650 - if engine == "fastparquet" and PANDAS_GE_200: + if engine == "fastparquet": # https://github.com/dask/dask/issues/9966 pytest.xfail("fastparquet reads as int64 while pyarrow does as int32") @@ -4674,6 +4665,7 @@ def test_pyarrow_filesystem_option(tmp_path, fs): @PYARROW_MARK @pytest.mark.network @pytest.mark.slow +@pytest.mark.skipif(pyarrow_version.major < 15, reason="Requires arrow 15") def test_pyarrow_filesystem_option_real_data(): # See: https://github.com/dask/dask/pull/10590 dd.read_parquet( @@ -4754,9 +4746,6 @@ def test_select_filtered_column_no_stats(tmp_path, engine): @PYARROW_MARK @pytest.mark.parametrize("convert_string", [True, False]) -@pytest.mark.skipif( - not PANDAS_GE_200, reason="dataframe.convert-string requires pandas>=2.0" -) def test_read_parquet_convert_string(tmp_path, convert_string, engine): df = pd.DataFrame( {"A": ["def", "abc", "ghi"], "B": [5, 2, 3], "C": ["x", "y", "z"]} @@ -4787,9 +4776,6 @@ def test_read_parquet_convert_string(tmp_path, convert_string, engine): @PYARROW_MARK -@pytest.mark.skipif( - not PANDAS_GE_200, reason="dataframe.convert-string requires pandas>=2.0" -) def test_read_parquet_convert_string_nullable_mapper(tmp_path, engine): """Make sure that when convert_string, dtype_backend and types_mapper are set, all three are used.""" @@ -4831,7 +4817,6 @@ def test_read_parquet_convert_string_nullable_mapper(tmp_path, engine): @PYARROW_MARK @pytest.mark.parametrize("dtype_backend", ["numpy_nullable", "pyarrow"]) -@pytest.mark.skipif(not PANDAS_GE_150, reason="Requires pyarrow-backed nullable dtypes") def test_dtype_backend(tmp_path, dtype_backend, engine): """ Test reading a parquet file without pandas metadata, @@ -4873,9 +4858,6 @@ def write_partition(df, i): @PYARROW_MARK -@pytest.mark.skipif( - not PANDAS_GE_200, reason="pd.Index does not support int32 before 2.0" -) def test_read_parquet_preserve_categorical_column_dtype(tmp_path): df = pd.DataFrame({"a": [1, 2], "b": ["x", "y"]}) @@ -4891,7 +4873,6 @@ def test_read_parquet_preserve_categorical_column_dtype(tmp_path): @PYARROW_MARK -@pytest.mark.skipif(not PANDAS_GE_200, reason="Requires pd.ArrowDtype") def test_dtype_backend_categoricals(tmp_path): df = pd.DataFrame({"a": pd.Series(["x", "y"], dtype="category"), "b": [1, 2]}) outdir = tmp_path / "out.parquet" @@ -4932,7 +4913,7 @@ def test_read_parquet_lists_not_converting(tmpdir): @PYARROW_MARK -@pytest.mark.skipif(not PANDAS_GE_200, reason="Requires pandas>=2.0") +@pytest.mark.skipif(pyarrow_version.major < 12, reason="Requires arrow >= 12") def test_parquet_string_roundtrip(tmpdir): pdf = pd.DataFrame({"a": ["a", "b", "c"]}, dtype="string[pyarrow]") pdf.to_parquet(tmpdir + "string.parquet") diff --git a/dask/dataframe/methods.py b/dask/dataframe/methods.py index 6b82ad9127c..06128b20d21 100644 --- a/dask/dataframe/methods.py +++ b/dask/dataframe/methods.py @@ -10,9 +10,6 @@ from tlz import partition from dask.dataframe._compat import ( - PANDAS_GE_131, - PANDAS_GE_140, - PANDAS_GE_200, check_apply_dataframe_deprecation, check_applymap_dataframe_deprecation, check_convert_dtype_deprecation, @@ -79,7 +76,7 @@ def try_loc(df, iindexer, cindexer=None): return df.head(0).loc[:, cindexer] -def boundary_slice(df, start, stop, right_boundary=True, left_boundary=True, kind=None): +def boundary_slice(df, start, stop, right_boundary=True, left_boundary=True): """Index slice start/stop. Can switch include/exclude boundaries. Examples @@ -114,20 +111,7 @@ def boundary_slice(df, start, stop, right_boundary=True, left_boundary=True, kin if len(df.index) == 0: return df - if PANDAS_GE_131: - if kind is not None: - warnings.warn( - "The `kind` argument is no longer used/supported. " - "It will be dropped in a future release.", - category=FutureWarning, - ) - kind_opts = {} - kind = "loc" - else: - kind = kind or "loc" - kind_opts = {"kind": kind} - - if kind == "loc" and not df.index.is_monotonic_increasing: + if not df.index.is_monotonic_increasing: # Pandas treats missing keys differently for label-slicing # on monotonic vs. non-monotonic indexes # If the index is monotonic, `df.loc[start:stop]` is fine. @@ -144,12 +128,12 @@ def boundary_slice(df, start, stop, right_boundary=True, left_boundary=True, kin df = df[df.index < stop] return df - result = getattr(df, kind)[start:stop] + result = df.loc[start:stop] if not right_boundary and stop is not None: - right_index = result.index.get_slice_bound(stop, "left", **kind_opts) + right_index = result.index.get_slice_bound(stop, "left") result = result.iloc[:right_index] if not left_boundary and start is not None: - left_index = result.index.get_slice_bound(start, "right", **kind_opts) + left_index = result.index.get_slice_bound(start, "right") result = result.iloc[left_index:] return result @@ -354,8 +338,7 @@ def assign(df, *pairs): # (to avoid modifying the original) # Setitem never modifies an array inplace with pandas 1.4 and up pairs = dict(partition(2, pairs)) - deep = bool(set(pairs) & set(df.columns)) and not PANDAS_GE_140 - df = df.copy(deep=bool(deep)) + df = df.copy(deep=False) with warnings.catch_warnings(): warnings.filterwarnings( "ignore", @@ -390,7 +373,7 @@ def value_counts_aggregate( out /= total_length if total_length is not None else out.sum() if sort: out = out.sort_values(ascending=ascending) - if PANDAS_GE_200 and normalize: + if normalize: out.name = "proportion" return out diff --git a/dask/dataframe/partitionquantiles.py b/dask/dataframe/partitionquantiles.py index 77ccf58ba2c..9ed891e57e8 100644 --- a/dask/dataframe/partitionquantiles.py +++ b/dask/dataframe/partitionquantiles.py @@ -78,7 +78,6 @@ from tlz import merge, merge_sorted, take from dask.base import tokenize -from dask.dataframe._compat import PANDAS_GE_150 from dask.dataframe.core import Series from dask.dataframe.dispatch import tolist_dispatch from dask.dataframe.utils import is_series_like @@ -415,9 +414,6 @@ def percentiles_summary(df, num_old, num_new, upsample, state): Scale factor to increase the number of percentiles calculated in each partition. Use to improve accuracy. """ - from dask.array.dispatch import percentile_lookup as _percentile - from dask.array.utils import array_safe - length = len(df) if length == 0: return () @@ -437,22 +433,18 @@ def percentiles_summary(df, num_old, num_new, upsample, state): vals = data.quantile(q=qs / 100, interpolation=interpolation) except (TypeError, NotImplementedError): # Series.quantile doesn't work with some data types (e.g. strings) - if PANDAS_GE_150: - # Fall back to DataFrame.quantile with "nearest" interpolation - interpolation = "nearest" - vals = ( - data.to_frame() - .quantile( - q=qs / 100, - interpolation=interpolation, - numeric_only=False, - method="table", - ) - .iloc[:, 0] + # Fall back to DataFrame.quantile with "nearest" interpolation + interpolation = "nearest" + vals = ( + data.to_frame() + .quantile( + q=qs / 100, + interpolation=interpolation, + numeric_only=False, + method="table", ) - else: - # Fall back to ndarray (Not supported for cuDF) - vals, _ = _percentile(array_safe(data, like=data.values), qs, interpolation) + .iloc[:, 0] + ) # Convert to array if necessary (and possible) if is_series_like(vals): diff --git a/dask/dataframe/reshape.py b/dask/dataframe/reshape.py index 58e067570bd..f06630d19aa 100644 --- a/dask/dataframe/reshape.py +++ b/dask/dataframe/reshape.py @@ -6,7 +6,6 @@ import dask from dask.dataframe import methods -from dask.dataframe._compat import PANDAS_GE_200 from dask.dataframe.core import DataFrame, Series, apply_concat_apply, map_partitions from dask.dataframe.utils import has_known_categories from dask.typing import no_default @@ -17,9 +16,6 @@ ############################################################### -_get_dummies_dtype_default = bool if PANDAS_GE_200 else np.uint8 - - def get_dummies( data, prefix=None, @@ -28,7 +24,7 @@ def get_dummies( columns=None, sparse=False, drop_first=False, - dtype=_get_dummies_dtype_default, + dtype=bool, **kwargs, ): """ diff --git a/dask/dataframe/tests/test_accessors.py b/dask/dataframe/tests/test_accessors.py index 01a66f8999e..e3be9519dbd 100644 --- a/dask/dataframe/tests/test_accessors.py +++ b/dask/dataframe/tests/test_accessors.py @@ -7,7 +7,7 @@ pd = pytest.importorskip("pandas") import dask.dataframe as dd -from dask.dataframe._compat import PANDAS_GE_140, PANDAS_GE_210, PANDAS_GE_300 +from dask.dataframe._compat import PANDAS_GE_210, PANDAS_GE_300 from dask.dataframe._pyarrow import to_pyarrow_string from dask.dataframe.utils import assert_eq, pyarrow_strings_enabled @@ -235,7 +235,6 @@ def test_str_accessor_extractall(df_ddf): ) -@pytest.mark.skipif(not PANDAS_GE_140, reason="requires pandas >= 1.4.0") @pytest.mark.parametrize("method", ["removeprefix", "removesuffix"]) def test_str_accessor_removeprefix_removesuffix(df_ddf, method): df, ddf = df_ddf diff --git a/dask/dataframe/tests/test_arithmetics_reduction.py b/dask/dataframe/tests/test_arithmetics_reduction.py index 7f36de7fe99..e0f0195a7f0 100644 --- a/dask/dataframe/tests/test_arithmetics_reduction.py +++ b/dask/dataframe/tests/test_arithmetics_reduction.py @@ -11,12 +11,6 @@ import dask.dataframe as dd from dask.array.numpy_compat import NUMPY_GE_125, NUMPY_GE_200 -from dask.dataframe._compat import ( - PANDAS_GE_140, - PANDAS_GE_150, - PANDAS_GE_200, - check_numeric_only_deprecation, -) from dask.dataframe.utils import ( assert_dask_graph, assert_eq, @@ -752,15 +746,12 @@ def test_reductions(split_every): bias_factor = (n * (n - 1)) ** 0.5 / (n - 2) assert_eq(dds.skew(), pds.skew() / bias_factor) - if PANDAS_GE_200: - # TODO: Remove this `if`-block once `axis=None` support is added. - # https://github.com/dask/dask/issues/9915 - with pytest.raises( - ValueError, match="`axis=None` isn't currently supported" - ): - dds.skew(axis=None) - else: - assert_eq(dds.skew(axis=None), pds.skew(axis=None) / bias_factor) + # TODO: Remove this `if`-block once `axis=None` support is added. + # https://github.com/dask/dask/issues/9915 + with pytest.raises( + ValueError, match="`axis=None` isn't currently supported" + ): + dds.skew(axis=None) if scipy: # pandas uses a bias factor for kurtosis, need to correct for that @@ -769,17 +760,12 @@ def test_reductions(split_every): offset = (6 * (n - 1)) / ((n - 2) * (n - 3)) assert_eq(factor * dds.kurtosis() + offset, pds.kurtosis()) - if PANDAS_GE_200: - # TODO: Remove this `if`-block once `axis=None` support is added. - # https://github.com/dask/dask/issues/9915 - with pytest.raises( - ValueError, match="`axis=None` isn't currently supported" - ): - dds.kurtosis(axis=None) - else: - assert_eq( - factor * dds.kurtosis(axis=None) + offset, pds.kurtosis(axis=None) - ) + # TODO: Remove this `if`-block once `axis=None` support is added. + # https://github.com/dask/dask/issues/9915 + with pytest.raises( + ValueError, match="`axis=None` isn't currently supported" + ): + dds.kurtosis(axis=None) with warnings.catch_warnings(): warnings.simplefilter("ignore", RuntimeWarning) @@ -967,30 +953,31 @@ def test_allany(split_every): pd.Series(np.random.choice([True, False], size=(100,))), 10 ) - with pytest.warns(FutureWarning, match="the 'out' keyword is deprecated"): - ddf.all(split_every=split_every, out=ddf_out_axis_default) - assert_eq(ddf_out_axis_default, df.all()) + if not DASK_EXPR_ENABLED: + with pytest.warns(FutureWarning, match="the 'out' keyword is deprecated"): + ddf.all(split_every=split_every, out=ddf_out_axis_default) + assert_eq(ddf_out_axis_default, df.all()) - with pytest.warns(FutureWarning, match="the 'out' keyword is deprecated"): - ddf.all(axis=1, split_every=split_every, out=ddf_out_axis1) - assert_eq(ddf_out_axis1, df.all(axis=1)) + with pytest.warns(FutureWarning, match="the 'out' keyword is deprecated"): + ddf.all(axis=1, split_every=split_every, out=ddf_out_axis1) + assert_eq(ddf_out_axis1, df.all(axis=1)) - with pytest.warns(FutureWarning, match="the 'out' keyword is deprecated"): - ddf.all(split_every=split_every, axis=0, out=ddf_out_axis_default) - assert_eq(ddf_out_axis_default, df.all(axis=0)) + with pytest.warns(FutureWarning, match="the 'out' keyword is deprecated"): + ddf.all(split_every=split_every, axis=0, out=ddf_out_axis_default) + assert_eq(ddf_out_axis_default, df.all(axis=0)) - # any - with pytest.warns(FutureWarning, match="the 'out' keyword is deprecated"): - ddf.any(split_every=split_every, out=ddf_out_axis_default) - assert_eq(ddf_out_axis_default, df.any()) + # any + with pytest.warns(FutureWarning, match="the 'out' keyword is deprecated"): + ddf.any(split_every=split_every, out=ddf_out_axis_default) + assert_eq(ddf_out_axis_default, df.any()) - with pytest.warns(FutureWarning, match="the 'out' keyword is deprecated"): - ddf.any(axis=1, split_every=split_every, out=ddf_out_axis1) - assert_eq(ddf_out_axis1, df.any(axis=1)) + with pytest.warns(FutureWarning, match="the 'out' keyword is deprecated"): + ddf.any(axis=1, split_every=split_every, out=ddf_out_axis1) + assert_eq(ddf_out_axis1, df.any(axis=1)) - with pytest.warns(FutureWarning, match="the 'out' keyword is deprecated"): - ddf.any(split_every=split_every, axis=0, out=ddf_out_axis_default) - assert_eq(ddf_out_axis_default, df.any(axis=0)) + with pytest.warns(FutureWarning, match="the 'out' keyword is deprecated"): + ddf.any(split_every=split_every, axis=0, out=ddf_out_axis_default) + assert_eq(ddf_out_axis_default, df.any(axis=0)) @pytest.mark.parametrize("split_every", [False, 2]) @@ -1199,28 +1186,17 @@ def test_reductions_frame(split_every): pytest.raises(ValueError, lambda: ddf1.sum(axis="incorrect").compute()) - # axis=None - if PANDAS_GE_140 and not PANDAS_GE_200: - ctx = pytest.warns(FutureWarning, match="axis=None") - else: - ctx = contextlib.nullcontext() # min - with ctx: - result = ddf1.min(axis=None, split_every=split_every) - with ctx: - expected = pdf1.min(axis=None) + result = ddf1.min(axis=None, split_every=split_every) + expected = pdf1.min(axis=None) assert_eq(result, expected) # max - with ctx: - result = ddf1.max(axis=None, split_every=split_every) - with ctx: - expected = pdf1.max(axis=None) + result = ddf1.max(axis=None, split_every=split_every) + expected = pdf1.max(axis=None) assert_eq(result, expected) # mean - with ctx: - result = ddf1.mean(axis=None, split_every=split_every) - with ctx: - expected = pdf1.mean(axis=None) + result = ddf1.mean(axis=None, split_every=split_every) + expected = pdf1.mean(axis=None) assert_eq(result, expected, check_dtype=not DASK_EXPR_ENABLED) if not DASK_EXPR_ENABLED: @@ -1330,10 +1306,9 @@ def test_reductions_frame_dtypes(func, kwargs, numeric_only): ddf = dd.from_pandas(df, 3) - with check_numeric_only_deprecation(): - expected = getattr(df, func)(**kwargs) - actual = getattr(ddf, func)(**kwargs) - assert_eq(expected, actual) + expected = getattr(df, func)(**kwargs) + actual = getattr(ddf, func)(**kwargs) + assert_eq(expected, actual) def test_count_numeric_only_axis_one(): @@ -1402,52 +1377,27 @@ def test_reductions_frame_dtypes_numeric_only_supported(func): ): getattr(ddf, func)(numeric_only=False) - warning = FutureWarning else: assert_eq( getattr(df, func)(numeric_only=False), getattr(ddf, func)(numeric_only=False), ) - warning = None # `numeric_only` default value - if PANDAS_GE_200: - if func in numeric_only_false_raises: - with pytest.raises( - errors, - match="'DatetimeArray' with dtype datetime64.*|" - "'DatetimeArray' does not implement reduction|could not convert|" - "'ArrowStringArray' with dtype string" - "|unsupported operand|no kernel|not supported", - ): - getattr(ddf, func)() - else: - assert_eq( - getattr(df, func)(), - getattr(ddf, func)(), - ) - elif PANDAS_GE_150: - if warning is None: - pd_result = getattr(df, func)() - dd_result = getattr(ddf, func)() - else: - with pytest.warns(warning, match="The default value of numeric_only"): - pd_result = getattr(df, func)() - with pytest.warns(warning, match="The default value of numeric_only"): - dd_result = getattr(ddf, func)() - assert_eq(pd_result, dd_result) + if func in numeric_only_false_raises: + with pytest.raises( + errors, + match="'DatetimeArray' with dtype datetime64.*|" + "'DatetimeArray' does not implement reduction|could not convert|" + "'ArrowStringArray' with dtype string" + "|unsupported operand|no kernel|not supported", + ): + getattr(ddf, func)() else: - if func in ["quantile"]: - warning = None - if warning is None: - pd_result = getattr(df, func)() - dd_result = getattr(ddf, func)() - else: - with pytest.warns(warning, match="Dropping of nuisance"): - pd_result = getattr(df, func)() - with pytest.warns(warning, match="Dropping of nuisance"): - dd_result = getattr(ddf, func)() - assert_eq(pd_result, dd_result) + assert_eq( + getattr(df, func)(), + getattr(ddf, func)(), + ) num_cols = ["int", "float"] if func != "quantile": @@ -1543,11 +1493,6 @@ def test_skew_kurt_numeric_only_false(func): with ctx: getattr(ddf, func)(numeric_only=False) - if PANDAS_GE_150 and not PANDAS_GE_200: - ctx = pytest.warns(FutureWarning, match="default value") - elif not PANDAS_GE_150: - ctx = pytest.warns(FutureWarning, match="nuisance columns") - with ctx: getattr(df, func)() with ctx: @@ -1861,7 +1806,7 @@ def test_datetime_std_across_axis1_null_results(skipna, numeric_only): ctx = contextlib.nullcontext() success = True - if numeric_only is False or (PANDAS_GE_200 and numeric_only is None): + if numeric_only is False or numeric_only is None: ctx = pytest.raises(TypeError) success = False elif numeric_only is None: @@ -1912,7 +1857,6 @@ def test_std_raises_on_index(): dd.from_pandas(pd.DataFrame({"test": [1, 2]}), npartitions=2).index.std() -@pytest.mark.skipif(not PANDAS_GE_200, reason="ArrowDtype not supported") def test_std_raises_with_arrow_string_ea(): pa = pytest.importorskip("pyarrow") ser = pd.Series(["a", "b", "c"], dtype=pd.ArrowDtype(pa.string())) @@ -1926,15 +1870,11 @@ def test_std_raises_with_arrow_string_ea(): [ pytest.param( "int64[pyarrow]", - marks=pytest.mark.skipif( - pa is None or not PANDAS_GE_150, reason="requires pyarrow installed" - ), + marks=pytest.mark.skipif(pa is None, reason="requires pyarrow installed"), ), pytest.param( "float64[pyarrow]", - marks=pytest.mark.skipif( - pa is None or not PANDAS_GE_150, reason="requires pyarrow installed" - ), + marks=pytest.mark.skipif(pa is None, reason="requires pyarrow installed"), ), "Int64", "Int32", diff --git a/dask/dataframe/tests/test_categorical.py b/dask/dataframe/tests/test_categorical.py index 33b0b82e1af..8dfea8386cc 100644 --- a/dask/dataframe/tests/test_categorical.py +++ b/dask/dataframe/tests/test_categorical.py @@ -11,13 +11,7 @@ import dask import dask.dataframe as dd from dask.dataframe import _compat -from dask.dataframe._compat import ( - PANDAS_GE_150, - PANDAS_GE_200, - PANDAS_GE_210, - PANDAS_GE_300, - tm, -) +from dask.dataframe._compat import PANDAS_GE_210, PANDAS_GE_300, tm from dask.dataframe._pyarrow import to_pyarrow_string from dask.dataframe.core import _concat from dask.dataframe.utils import ( @@ -148,19 +142,12 @@ def test_unknown_categories_cudf(): pytest.param( False, marks=[ - pytest.mark.xfail( - PANDAS_GE_200, reason="numeric_only=False not implemented" - ), - pytest.mark.xfail( - not PANDAS_GE_150, reason="`numeric_only` not implemented" - ), + pytest.mark.xfail(reason="numeric_only=False not implemented"), ], ), pytest.param( None, - marks=pytest.mark.xfail( - PANDAS_GE_200, reason="numeric_only=False not implemented" - ), + marks=pytest.mark.xfail(reason="numeric_only=False not implemented"), ), ], ) diff --git a/dask/dataframe/tests/test_dataframe.py b/dask/dataframe/tests/test_dataframe.py index 900a2068835..b2d0b2b418a 100644 --- a/dask/dataframe/tests/test_dataframe.py +++ b/dask/dataframe/tests/test_dataframe.py @@ -24,16 +24,7 @@ from dask.base import compute_as_if_collection from dask.blockwise import fuse_roots from dask.dataframe import _compat, methods -from dask.dataframe._compat import ( - PANDAS_GE_133, - PANDAS_GE_140, - PANDAS_GE_150, - PANDAS_GE_200, - PANDAS_GE_210, - PANDAS_GE_220, - PANDAS_GE_300, - tm, -) +from dask.dataframe._compat import PANDAS_GE_210, PANDAS_GE_220, PANDAS_GE_300, tm from dask.dataframe._pyarrow import to_pyarrow_string from dask.dataframe.core import ( Scalar, @@ -518,26 +509,19 @@ def test_describe(include, exclude, percentiles, subset): ddf = dd.from_pandas(df, 2) - if not PANDAS_GE_200: - datetime_is_numeric_kwarg = {"datetime_is_numeric": True} - else: - datetime_is_numeric_kwarg = {} - # Act actual = ddf.describe( include=include, exclude=exclude, percentiles=percentiles, - **datetime_is_numeric_kwarg, ) expected = df.describe( include=include, exclude=exclude, percentiles=percentiles, - **datetime_is_numeric_kwarg, ) - if "e" in expected and (datetime_is_numeric_kwarg or PANDAS_GE_200): + if "e" in expected: expected = _drop_mean(expected, "e") assert_eq(actual, expected) @@ -545,14 +529,10 @@ def test_describe(include, exclude, percentiles, subset): # Check series if subset is None: for col in ["a", "c", "e", "g"]: - expected = df[col].describe( - include=include, exclude=exclude, **datetime_is_numeric_kwarg - ) - if col == "e" and (datetime_is_numeric_kwarg or PANDAS_GE_200): + expected = df[col].describe(include=include, exclude=exclude) + if col == "e": expected = _drop_mean(expected) - actual = ddf[col].describe( - include=include, exclude=exclude, **datetime_is_numeric_kwarg - ) + actual = ddf[col].describe(include=include, exclude=exclude) assert_eq(expected, actual) @@ -578,8 +558,7 @@ def test_describe_without_datetime_is_numeric(): # Assert expected = df.describe() - if PANDAS_GE_200: - expected = _drop_mean(expected, "e") + expected = _drop_mean(expected, "e") assert_eq(ddf.describe(), expected) @@ -587,22 +566,8 @@ def test_describe_without_datetime_is_numeric(): for col in ["a", "c"]: assert_eq(df[col].describe(), ddf[col].describe()) - if PANDAS_GE_200: - expected = _drop_mean(df.e.describe()) - assert_eq(expected, ddf.e.describe()) - with pytest.raises( - TypeError, - match="datetime_is_numeric is removed in pandas>=2.0.0", - ): - ddf.e.describe(datetime_is_numeric=True) - else: - with pytest.warns( - FutureWarning, - match=( - "Treating datetime data as categorical rather than numeric in `.describe` is deprecated" - ), - ): - ddf.e.describe() + expected = _drop_mean(df.e.describe()) + assert_eq(expected, ddf.e.describe()) # Note: this warning is not always raised on Windows @@ -1611,19 +1576,6 @@ def test_empty_quantile(method): assert_eq(result, exp) -@contextlib.contextmanager -def assert_numeric_only_default_warning(numeric_only, func=None): - if func == "quantile" and not PANDAS_GE_150: - ctx = contextlib.nullcontext() - elif numeric_only is None and not PANDAS_GE_200: - ctx = pytest.warns(FutureWarning, match="default value of numeric_only") - else: - ctx = contextlib.nullcontext() - - with ctx: - yield - - # TODO: un-filter once https://github.com/dask/dask/issues/8960 is resolved. @pytest.mark.filterwarnings( "ignore:In future versions of pandas, numeric_only will be set to False:FutureWarning" @@ -1674,7 +1626,7 @@ def test_dataframe_quantile(method, expected, numeric_only): if numeric_only is not None: numeric_only_kwarg = {"numeric_only": numeric_only} - if numeric_only is False or (PANDAS_GE_200 and numeric_only is None): + if numeric_only is False or numeric_only is None: with pytest.raises(TypeError): df.quantile(**numeric_only_kwarg) with pytest.raises( @@ -1683,8 +1635,7 @@ def test_dataframe_quantile(method, expected, numeric_only): ): ddf.quantile(**numeric_only_kwarg) else: - with assert_numeric_only_default_warning(numeric_only, "quantile"): - result = ddf.quantile(method=method, **numeric_only_kwarg) + result = ddf.quantile(method=method, **numeric_only_kwarg) assert result.npartitions == 1 assert result.divisions == ("A", "X") @@ -1693,8 +1644,7 @@ def test_dataframe_quantile(method, expected, numeric_only): assert result.name == 0.5 assert_eq(result, expected[0], check_names=False) - with assert_numeric_only_default_warning(numeric_only, "quantile"): - result = ddf.quantile([0.25, 0.75], method=method, **numeric_only_kwarg) + result = ddf.quantile([0.25, 0.75], method=method, **numeric_only_kwarg) assert result.npartitions == 1 assert result.divisions == (0.25, 0.75) @@ -1710,14 +1660,11 @@ def test_dataframe_quantile(method, expected, numeric_only): # pandas issues a warning with 1.5, but not 1.3 expected = df.quantile(axis=1, **numeric_only_kwarg) - with assert_numeric_only_default_warning(numeric_only, "quantile"): - result = ddf.quantile(axis=1, method=method, **numeric_only_kwarg) + result = ddf.quantile(axis=1, method=method, **numeric_only_kwarg) assert_eq(result, expected) - with pytest.raises(ValueError), assert_numeric_only_default_warning( - numeric_only, "quantile" - ): + with pytest.raises(ValueError): ddf.quantile([0.25, 0.75], axis=1, method=method, **numeric_only_kwarg) @@ -3610,18 +3557,8 @@ def test_round(): "numeric_only", [ None, - pytest.param( - True, - marks=pytest.mark.skipif( - not PANDAS_GE_150, reason="numeric_only not yet implemented" - ), - ), - pytest.param( - False, - marks=pytest.mark.skipif( - not PANDAS_GE_150, reason="numeric_only not yet implemented" - ), - ), + True, + False, ], ) def test_cov_dataframe(numeric_only): @@ -3674,21 +3611,7 @@ def test_cov_series(): @pytest.mark.gpu @pytest.mark.parametrize( "numeric_only", - [ - None, - pytest.param( - True, - marks=pytest.mark.skipif( - not PANDAS_GE_150, reason="numeric_only not yet implemented" - ), - ), - pytest.param( - False, - marks=pytest.mark.skipif( - not PANDAS_GE_150, reason="numeric_only not yet implemented" - ), - ), - ], + [None, True, False], ) def test_cov_gpu(numeric_only): cudf = pytest.importorskip("cudf") @@ -3819,23 +3742,13 @@ def test_cov_corr_stable(): [ pytest.param( None, - marks=pytest.mark.xfail( - PANDAS_GE_200, reason="fails with non-numeric data" - ), - ), - pytest.param( - True, - marks=pytest.mark.skipif( - not PANDAS_GE_150, reason="numeric_only not yet implemented" - ), + marks=pytest.mark.xfail(reason="fails with non-numeric data"), ), + True, pytest.param( False, marks=[ - pytest.mark.skipif( - not PANDAS_GE_150, reason="numeric_only not yet implemented" - ), - pytest.mark.xfail(PANDAS_GE_150, reason="fails with non-numeric data"), + pytest.mark.xfail(reason="fails with non-numeric data"), ], ), ], @@ -3866,23 +3779,15 @@ def test_cov_corr_mixed(numeric_only): numeric_only_kwarg = {} if numeric_only is not None: numeric_only_kwarg = {"numeric_only": numeric_only} - if not numeric_only_kwarg and PANDAS_GE_150 and not PANDAS_GE_200: - ctx = pytest.warns(FutureWarning, match="default value of numeric_only") - else: - ctx = contextlib.nullcontext() # Corr - with ctx: - expected = df.corr(**numeric_only_kwarg) - with ctx: - result = ddf.corr(split_every=4, **numeric_only_kwarg) + expected = df.corr(**numeric_only_kwarg) + result = ddf.corr(split_every=4, **numeric_only_kwarg) assert_eq(result, expected, check_divisions=False) # Cov - with ctx: - expected = df.cov(**numeric_only_kwarg) - with ctx: - result = ddf.cov(split_every=4, **numeric_only_kwarg) + expected = df.cov(**numeric_only_kwarg) + result = ddf.cov(split_every=4, **numeric_only_kwarg) assert_eq(result, expected, check_divisions=False) @@ -4046,23 +3951,6 @@ def test_contains_series_raises_deprecated_warning_preserves_behavior(): assert not output -@pytest.mark.skipif(PANDAS_GE_200, reason="iteritems has been removed") -def test_series_iteritems(): - df = pd.DataFrame({"x": [1, 2, 3, 4]}) - ddf = dd.from_pandas(df, npartitions=2) - # `iteritems` was deprecated starting in `pandas=1.5.0` - with _check_warning( - PANDAS_GE_150, FutureWarning, message="iteritems is deprecated" - ): - pd_items = df["x"].iteritems() - with _check_warning( - PANDAS_GE_150, FutureWarning, message="iteritems is deprecated" - ): - dd_items = ddf["x"].iteritems() - for a, b in zip(pd_items, dd_items): - assert a == b - - def test_series_iter(): s = pd.DataFrame({"x": [1, 2, 3, 4]}) ds = dd.from_pandas(s, npartitions=2) @@ -4500,9 +4388,7 @@ def test_idxmaxmin(idx, skipna): ddf = dd.from_pandas(df, npartitions=3) # https://github.com/pandas-dev/pandas/issues/43587 - check_dtype = not all( - (PANDAS_GE_133, skipna is False, isinstance(idx, pd.DatetimeIndex)) - ) + check_dtype = not all((skipna is False, isinstance(idx, pd.DatetimeIndex))) ctx = contextlib.nullcontext() if PANDAS_GE_300 and not skipna: @@ -4580,29 +4466,19 @@ def test_idxmaxmin_numeric_only(func): } ) ddf = dd.from_pandas(df, npartitions=2) + assert_eq( + getattr(ddf, func)(numeric_only=False), + getattr(df, func)(numeric_only=False).sort_index(), + ) + assert_eq( + getattr(ddf, func)(numeric_only=True), + getattr(df, func)(numeric_only=True).sort_index(), + ) - if PANDAS_GE_150: - assert_eq( - getattr(ddf, func)(numeric_only=False), - getattr(df, func)(numeric_only=False).sort_index(), - ) - assert_eq( - getattr(ddf, func)(numeric_only=True), - getattr(df, func)(numeric_only=True).sort_index(), - ) - - assert_eq( - getattr(ddf.drop(columns="bool"), func)(numeric_only=True, axis=1), - getattr(df.drop(columns="bool"), func)( - numeric_only=True, axis=1 - ).sort_index(), - ) - - else: - with pytest.raises(TypeError, match="got an unexpected keyword"): - getattr(df, func)(numeric_only=False) - with pytest.raises(NotImplementedError, match="idxmax for pandas"): - getattr(ddf, func)(numeric_only=False) + assert_eq( + getattr(ddf.drop(columns="bool"), func)(numeric_only=True, axis=1), + getattr(df.drop(columns="bool"), func)(numeric_only=True, axis=1).sort_index(), + ) def test_idxmaxmin_empty_partitions(): @@ -4977,11 +4853,7 @@ def test_values_extension_dtypes(): result = ddf.y.values assert_eq(result, df.y.values.astype(object)) - # Prior to pandas=1.4, `pd.Index` couldn't hold extension dtypes - ctx = contextlib.nullcontext() - if PANDAS_GE_140: - ctx = pytest.warns(UserWarning, match="object dtype") - with ctx: + with pytest.warns(UserWarning, match="object dtype"): result = ddf.index.values assert_eq(result, df.index.values.astype(object)) @@ -5128,7 +5000,7 @@ def test_dataframe_mode(): assert_eq(ddf.mode(), df.mode()) # name is not preserved in older pandas - assert_eq(ddf.Name.mode(), df.Name.mode(), check_names=PANDAS_GE_140) + assert_eq(ddf.Name.mode(), df.Name.mode()) # test empty df = pd.DataFrame(columns=["a", "b"]) @@ -5212,12 +5084,7 @@ def test_to_datetime(gpu): # infer_datetime_format is not supported anymore in dask-expr if not DASK_EXPR_ENABLED: - if PANDAS_GE_200: - ctx = pytest.warns( - UserWarning, match="'infer_datetime_format' is deprecated" - ) - else: - ctx = contextlib.nullcontext() + ctx = pytest.warns(UserWarning, match="'infer_datetime_format' is deprecated") ctx_expected = contextlib.nullcontext() if gpu else ctx with ctx_expected: @@ -5544,7 +5411,8 @@ def test_meta_nonempty_uses_meta_value_if_provided(): offsets = pd.Series([pd.offsets.DateOffset(years=o) for o in range(3)]) dask_base = dd.from_pandas(base, npartitions=1) dask_offsets = dd.from_pandas(offsets, npartitions=1) - dask_offsets._meta = offsets.head() + if not DASK_EXPR_ENABLED: + dask_offsets._meta = offsets.head() with warnings.catch_warnings(): # not vectorized performance warning warnings.simplefilter("ignore", PerformanceWarning) @@ -6082,22 +5950,6 @@ def test_is_monotonic_numeric(series, reverse, cls): assert ds.is_monotonic_decreasing.compute() == pds.is_monotonic_decreasing -@pytest.mark.skipif(PANDAS_GE_200, reason="pandas removed is_monotonic") -def test_is_monotonic_deprecated(): - s = pd.Series(range(20)) - ds = dd.from_pandas(s, npartitions=5) - # `is_monotonic` was deprecated starting in `pandas=1.5.0` - with _check_warning( - PANDAS_GE_150, FutureWarning, message="is_monotonic is deprecated" - ): - expected = s.is_monotonic - with _check_warning( - PANDAS_GE_150, FutureWarning, message="is_monotonic is deprecated" - ): - result = ds.is_monotonic - assert_eq(expected, result) - - def test_is_monotonic_dt64(): s = pd.Series(pd.date_range("20130101", periods=10)) ds = dd.from_pandas(s, npartitions=5) @@ -6122,22 +5974,6 @@ def test_index_is_monotonic_dt64(): assert_eq(s_2.index.is_monotonic_decreasing, ds_2.index.is_monotonic_decreasing) -@pytest.mark.skipif(PANDAS_GE_200, reason="pandas removed is_monotonic") -def test_index_is_monotonic_deprecated(): - s = pd.Series(1, index=range(20)) - ds = dd.from_pandas(s, npartitions=5, sort=False) - # `is_monotonic` was deprecated starting in `pandas=1.5.0` - with _check_warning( - PANDAS_GE_150, FutureWarning, message="is_monotonic is deprecated" - ): - expected = s.index.is_monotonic - with _check_warning( - PANDAS_GE_150, FutureWarning, message="is_monotonic is deprecated" - ): - result = ds.index.is_monotonic - assert_eq(expected, result) - - def test_is_monotonic_empty_partitions(): df = pd.DataFrame({"a": [1, 2, 3, 4], "b": [4, 3, 2, 1]}) ddf = dd.from_pandas(df, npartitions=2) @@ -6226,9 +6062,6 @@ def test_repr_materialize(): assert all([not l.is_materialized() for l in s.dask.layers.values()]) -@pytest.mark.skipif( - not PANDAS_GE_150, reason="Requires native PyArrow-backed ExtensionArrays" -) @pytest.mark.parametrize( "dtype", [ @@ -6249,9 +6082,6 @@ def test_pyarrow_extension_dtype(dtype): assert_eq(expected, result) -@pytest.mark.skipif( - not PANDAS_GE_150, reason="Requires native PyArrow-backed ExtensionArrays" -) def test_pyarrow_decimal_extension_dtype(): # Similar to `test_pyarrow_extension_dtype` but for pyarrow decimal dtypes pa = pytest.importorskip("pyarrow") @@ -6346,7 +6176,7 @@ def test_mask_where_array_like(df, cond): assert_eq(expected, result) -@pytest.mark.xfail(DASK_EXPR_ENABLED, reason="duplicated columns not supported") +@pytest.mark.skipif(DASK_EXPR_ENABLED, reason="duplicated columns not supported") @pytest.mark.parametrize( "func, kwargs", [ diff --git a/dask/dataframe/tests/test_groupby.py b/dask/dataframe/tests/test_groupby.py index ad16b7abc75..a51fcf0c0ea 100644 --- a/dask/dataframe/tests/test_groupby.py +++ b/dask/dataframe/tests/test_groupby.py @@ -14,14 +14,9 @@ import dask.dataframe as dd from dask.dataframe import _compat from dask.dataframe._compat import ( - PANDAS_GE_140, - PANDAS_GE_150, - PANDAS_GE_200, PANDAS_GE_210, PANDAS_GE_220, PANDAS_GE_300, - check_nuisance_columns_warning, - check_numeric_only_deprecation, check_observed_deprecation, tm, ) @@ -30,7 +25,7 @@ from dask.dataframe.groupby import NUMERIC_ONLY_NOT_IMPLEMENTED from dask.dataframe.utils import assert_dask_graph, assert_eq, pyarrow_strings_enabled from dask.utils import M -from dask.utils_test import _check_warning, hlg_layer +from dask.utils_test import hlg_layer DASK_EXPR_ENABLED = dd._dask_expr_enabled() AGG_FUNCS = [ @@ -38,7 +33,6 @@ pytest.param( "mean", marks=pytest.mark.xfail( - condition=PANDAS_GE_200, reason="numeric_only=False not implemented", strict=False, ), @@ -51,7 +45,6 @@ pytest.param( "std", marks=pytest.mark.xfail( - condition=PANDAS_GE_200, reason="numeric_only=False not implemented", strict=False, ), @@ -59,7 +52,6 @@ pytest.param( "var", marks=pytest.mark.xfail( - condition=PANDAS_GE_200, reason="numeric_only=False not implemented", strict=False, ), @@ -67,7 +59,6 @@ pytest.param( "cov", marks=pytest.mark.xfail( - condition=PANDAS_GE_200, reason="numeric_only=False not implemented", strict=False, ), @@ -75,7 +66,6 @@ pytest.param( "corr", marks=pytest.mark.xfail( - condition=PANDAS_GE_200, reason="numeric_only=False not implemented", strict=False, ), @@ -1484,13 +1474,7 @@ def call(g, m, **kwargs): lambda df: df["a"] > 2, lambda df: [df["a"], df["b"]], lambda df: [df["a"] > 2], - pytest.param( - lambda df: [df["a"] > 2, df["b"] > 1], - marks=pytest.mark.xfail( - not PANDAS_GE_150, - reason="index dtype does not coincide: boolean != empty", - ), - ), + lambda df: [df["a"] > 2, df["b"] > 1], ], ) @pytest.mark.parametrize( @@ -2154,21 +2138,15 @@ def record_numeric_only_warnings(): [ pytest.param( "var", - marks=pytest.mark.xfail( - PANDAS_GE_200, reason="numeric_only=False not implemented" - ), + marks=pytest.mark.xfail(reason="numeric_only=False not implemented"), ), pytest.param( "std", - marks=pytest.mark.xfail( - PANDAS_GE_200, reason="numeric_only=False not implemented" - ), + marks=pytest.mark.xfail(reason="numeric_only=False not implemented"), ), pytest.param( "mean", - marks=pytest.mark.xfail( - PANDAS_GE_200, reason="numeric_only=False not implemented" - ), + marks=pytest.mark.xfail(reason="numeric_only=False not implemented"), ), pytest.param( "sum", @@ -2185,17 +2163,8 @@ def test_std_object_dtype(func): # DataFrame # TODO: add deprecation warnings to match pandas - ctx = contextlib.nullcontext() - if func != "sum": - ctx = check_nuisance_columns_warning() - with ctx, check_numeric_only_deprecation(): - expected = getattr(df, func)() - with _check_warning( - func in ["std", "var"] and not PANDAS_GE_200, - FutureWarning, - message="numeric_only", - ): - result = getattr(ddf, func)() + expected = getattr(df, func)() + result = getattr(ddf, func)() assert_eq(expected, result) # DataFrameGroupBy @@ -3001,13 +2970,7 @@ def test_groupby_grouper_dispatch(key): @pytest.mark.parametrize( "group_keys", [ - pytest.param( - True, - marks=pytest.mark.skipif( - not PANDAS_GE_150, - reason="cudf and pandas behave differently", - ), - ), + True, False, ], ) @@ -3045,15 +3008,7 @@ def test_groupby_dropna_with_agg(sort): df = pd.DataFrame( {"id1": ["a", None, "b"], "id2": [1, 2, None], "v1": [4.5, 5.5, None]} ) - if PANDAS_GE_200: - expected = df.groupby(["id1", "id2"], dropna=False, sort=sort).agg("sum") - else: - # before 2.0, sort=False appears to be disregarded, but only when - # grouping on index columns, which is what we do in dask groupby. - # So we should expect sorted index levels even with sort=False. - # Fixed in 2.0, possibly by https://github.com/pandas-dev/pandas/pull/49613 - expected = df.groupby(["id1", "id2"], dropna=False, sort=True).agg("sum") - + expected = df.groupby(["id1", "id2"], dropna=False, sort=sort).agg("sum") ddf = dd.from_pandas(df, 1) actual = ddf.groupby(["id1", "id2"], dropna=False, sort=sort).agg("sum") assert_eq(expected, actual) @@ -3154,13 +3109,13 @@ def test_groupby_large_ints_exception(backend): pytest.param( "mean", marks=pytest.mark.xfail( - PANDAS_GE_200, reason="numeric_only=False not implemented", strict=False + reason="numeric_only=False not implemented", strict=False ), ), pytest.param( "std", marks=pytest.mark.xfail( - PANDAS_GE_200, reason="numeric_only=False not implemented", strict=False + reason="numeric_only=False not implemented", strict=False ), ), ], @@ -3289,7 +3244,7 @@ def test_groupby_aggregate_categorical_observed( pytest.skip("Gives zeros rather than nans.") if agg_func in ["std", "var"] and observed: pytest.skip("Can't calculate observed with all nans") - if agg_func in ["sum", "prod"] and PANDAS_GE_200: + if agg_func in ["sum", "prod"]: pytest.xfail("Not implemented for category type with pandas 2.0") pdf = pd.DataFrame( @@ -3308,15 +3263,7 @@ def test_groupby_aggregate_categorical_observed( ddf["cat_2"] = ddf["cat_2"].cat.as_unknown() def agg(grp, **kwargs): - if isinstance(grp, pd.core.groupby.DataFrameGroupBy) or ( - PANDAS_GE_150 and not PANDAS_GE_200 - ): - # with pandas 1.5, dask also raises a warning on default numeric_only - ctx = check_numeric_only_deprecation - else: - ctx = contextlib.nullcontext - with ctx(): - return getattr(grp, agg_func)(**kwargs) + return getattr(grp, agg_func)(**kwargs) # only include numeric columns when passing to "min" or "max" # pandas default is numeric_only=False @@ -3350,7 +3297,6 @@ def test_groupby_cov_non_numeric_grouping_column(): assert_eq(ddf.groupby("b").cov(), pdf.groupby("b").cov()) -@pytest.mark.skipif(not PANDAS_GE_150, reason="requires pandas >= 1.5.0") def test_groupby_numeric_only_None_column_name(): df = pd.DataFrame({"a": [1, 2, 3], None: ["a", "b", "c"]}) ddf = dd.from_pandas(df, npartitions=1) @@ -3358,7 +3304,6 @@ def test_groupby_numeric_only_None_column_name(): ddf.groupby(lambda x: x).mean(numeric_only=False) -@pytest.mark.skipif(not PANDAS_GE_140, reason="requires pandas >= 1.4.0") @pytest.mark.parametrize("shuffle_method", [True, False]) def test_dataframe_named_agg(shuffle_method): df = pd.DataFrame( @@ -3382,7 +3327,6 @@ def test_dataframe_named_agg(shuffle_method): assert_eq(expected, actual) -@pytest.mark.skipif(not PANDAS_GE_140, reason="requires pandas >= 1.4.0") @pytest.mark.parametrize("shuffle_method", [True, False]) @pytest.mark.parametrize("agg", ["count", "mean", partial(np.var, ddof=1)]) def test_series_named_agg(shuffle_method, agg): @@ -3592,9 +3536,6 @@ def test_groupby_slice_getitem(by, slice_key): "numeric_only", [None, True, False], ) -@pytest.mark.skipif( - not PANDAS_GE_150, reason="numeric_only not implemented for pandas < 1.5" -) def test_groupby_numeric_only_supported(func, numeric_only): pdf = pd.DataFrame( { @@ -3608,22 +3549,9 @@ def test_groupby_numeric_only_supported(func, numeric_only): kwargs = {} if numeric_only is None else {"numeric_only": numeric_only} - # Some groupby methods will raise deprecation warnings or TypeErrors - # depending on the version of pandas being used. Here we check that - # dask and panadas have similar behavior ctx = contextlib.nullcontext() - if PANDAS_GE_150 and not PANDAS_GE_200: - if func in ("sum", "prod", "median"): - if numeric_only is None: - ctx = pytest.warns( - FutureWarning, match="The default value of numeric_only" - ) - elif numeric_only is False: - ctx = pytest.warns(FutureWarning, match="Dropping invalid columns") - try: - with ctx: - expected = getattr(pdf.groupby("ints"), func)(**kwargs) + expected = getattr(pdf.groupby("ints"), func)(**kwargs) successful_compute = True except TypeError: # Make sure dask and pandas raise the same error message @@ -3649,22 +3577,9 @@ def test_groupby_numeric_only_not_implemented(func, numeric_only): df = pd.DataFrame({"A": [1, 1, 2], "B": [3, 4, 3], "C": ["a", "b", "c"]}) ddf = dd.from_pandas(df, npartitions=3) - ctx = contextlib.nullcontext() - ctx_warn = pytest.warns(FutureWarning, match="The default value of numeric_only") - ctx_error = pytest.raises( + ctx = pytest.raises( NotImplementedError, match="'numeric_only=False' is not implemented in Dask" ) - if numeric_only is None: - if PANDAS_GE_150 and not PANDAS_GE_200: - # Start warning about upcoming change to `numeric_only` default value - ctx = ctx_warn - elif PANDAS_GE_200: - # Default was changed to `numeric_only=False` in pandas 2.0 - ctx = ctx_error - else: - # Always error when `numeric_only=False` - ctx = ctx_error - # Here `numeric_only=None` means "use default value for `numeric_only`" kwargs = {} if numeric_only is None else {"numeric_only": numeric_only} with ctx: @@ -3693,19 +3608,11 @@ def test_groupby_numeric_only_not_implemented(func, numeric_only): def test_groupby_numeric_only_true(func): df = pd.DataFrame({"A": [1, 1, 2, 2], "B": [3, 4, 3, 4], "C": ["a", "b", "c", "d"]}) ddf = dd.from_pandas(df, npartitions=2) - - if func in ["var", "std", "cov", "corr"] and not PANDAS_GE_150: - with pytest.raises(TypeError, match="numeric_only not supported"): - getattr(ddf.groupby("A"), func)(numeric_only=True) - with pytest.raises(TypeError, match="got an unexpected keyword"): - getattr(df.groupby("A"), func)(numeric_only=True) - else: - ddf_result = getattr(ddf.groupby("A"), func)(numeric_only=True) - pdf_result = getattr(df.groupby("A"), func)(numeric_only=True) - assert_eq(ddf_result, pdf_result) + ddf_result = getattr(ddf.groupby("A"), func)(numeric_only=True) + pdf_result = getattr(df.groupby("A"), func)(numeric_only=True) + assert_eq(ddf_result, pdf_result) -@pytest.mark.skipif(not PANDAS_GE_150, reason="numeric_only not supported for <1.5") @pytest.mark.parametrize("func", ["cov", "corr"]) def test_groupby_numeric_only_false_cov_corr(func): df = pd.DataFrame( @@ -3738,37 +3645,17 @@ def test_groupby_numeric_only_false(func): ) ddf = dd.from_pandas(df, npartitions=2) - if PANDAS_GE_200: - ctx = pytest.raises(TypeError, match="does not support") + ctx = pytest.raises(TypeError, match="does not support") - with ctx: - getattr(ddf.groupby("A"), func)(numeric_only=False) - with ctx: - getattr(df.groupby("A"), func)(numeric_only=False) - - with ctx: - getattr(ddf.groupby("A"), func)() - with ctx: - getattr(df.groupby("A"), func)() - else: - ctx = pytest.warns(FutureWarning, match="Dropping invalid columns") - - with ctx: - dd_result = getattr(ddf.groupby("A"), func)(numeric_only=False) - with ctx: - pd_result = getattr(df.groupby("A"), func)(numeric_only=False) - assert_eq(dd_result, pd_result) - - if PANDAS_GE_150: - ctx = pytest.warns(FutureWarning, match="default value of numeric_only") - else: - ctx = contextlib.nullcontext() + with ctx: + getattr(ddf.groupby("A"), func)(numeric_only=False) + with ctx: + getattr(df.groupby("A"), func)(numeric_only=False) - with ctx: - dd_result = getattr(ddf.groupby("A"), func)() - with ctx: - pd_result = getattr(df.groupby("A"), func)() - assert_eq(dd_result, pd_result) + with ctx: + getattr(ddf.groupby("A"), func)() + with ctx: + getattr(df.groupby("A"), func)() @pytest.mark.parametrize("func", ["var", "std"]) diff --git a/dask/dataframe/tests/test_methods.py b/dask/dataframe/tests/test_methods.py index fdb1cfd7598..e925b5d3754 100644 --- a/dask/dataframe/tests/test_methods.py +++ b/dask/dataframe/tests/test_methods.py @@ -4,12 +4,10 @@ import pandas as pd import dask.dataframe.methods as methods -from dask.dataframe._compat import PANDAS_GE_140 def test_assign_not_modifying_array_inplace(): df = pd.DataFrame({"a": [1, 2, 3], "b": 1.5}) result = methods.assign(df, "a", 5) assert not np.shares_memory(df["a"].values, result["a"].values) - if PANDAS_GE_140: - assert np.shares_memory(df["b"].values, result["b"].values) + assert np.shares_memory(df["b"].values, result["b"].values) diff --git a/dask/dataframe/tests/test_multi.py b/dask/dataframe/tests/test_multi.py index fb732202181..445f0adb8b7 100644 --- a/dask/dataframe/tests/test_multi.py +++ b/dask/dataframe/tests/test_multi.py @@ -11,14 +11,7 @@ import dask.dataframe as dd from dask._compatibility import PY_VERSION from dask.base import compute_as_if_collection -from dask.dataframe._compat import ( - PANDAS_GE_140, - PANDAS_GE_150, - PANDAS_GE_200, - PANDAS_GE_210, - PANDAS_GE_220, - tm, -) +from dask.dataframe._compat import PANDAS_GE_210, PANDAS_GE_220, tm from dask.dataframe.core import _Frame from dask.dataframe.methods import concat from dask.dataframe.multi import ( @@ -2437,12 +2430,7 @@ def test_concat_datetimeindex(): def check_append_with_warning(dask_obj, dask_append, pandas_obj, pandas_append): - if PANDAS_GE_140: - with pytest.warns(FutureWarning, match="append method is deprecated"): - expected = pandas_obj.append(pandas_append) - result = dask_obj.append(dask_append) - assert_eq(result, expected) - else: + with pytest.warns(FutureWarning, match="append method is deprecated"): expected = pandas_obj.append(pandas_append) result = dask_obj.append(dask_append) assert_eq(result, expected) @@ -2450,139 +2438,6 @@ def check_append_with_warning(dask_obj, dask_append, pandas_obj, pandas_append): return result -@pytest.mark.skipif(PANDAS_GE_200, reason="pandas removed append") -def test_append(): - df = pd.DataFrame({"a": [1, 2, 3, 4, 5, 6], "b": [1, 2, 3, 4, 5, 6]}) - df2 = pd.DataFrame( - {"a": [1, 2, 3, 4, 5, 6], "b": [1, 2, 3, 4, 5, 6]}, index=[6, 7, 8, 9, 10, 11] - ) - df3 = pd.DataFrame( - {"b": [1, 2, 3, 4, 5, 6], "c": [1, 2, 3, 4, 5, 6]}, index=[6, 7, 8, 9, 10, 11] - ) - - ddf = dd.from_pandas(df, 2) - ddf2 = dd.from_pandas(df2, 2) - ddf3 = dd.from_pandas(df3, 2) - - s = pd.Series([7, 8], name=6, index=["a", "b"]) - - check_append_with_warning(ddf, s, df, s) - check_append_with_warning(ddf, ddf2, df, df2) - check_append_with_warning(ddf.a, ddf2.a, df.a, df2.a) - - # different columns - check_append_with_warning(ddf, ddf3, df, df3) - check_append_with_warning(ddf.a, ddf3.b, df.a, df3.b) - - # dask + pandas - check_append_with_warning(ddf, df2, df, df2) - check_append_with_warning(ddf.a, df2.a, df.a, df2.a) - - check_append_with_warning(ddf, df3, df, df3) - check_append_with_warning(ddf.a, df3.b, df.a, df3.b) - - -@pytest.mark.skipif(PANDAS_GE_200, reason="pandas removed append") -def test_append2(): - dsk = { - ("x", 0): pd.DataFrame({"a": [1, 2, 3], "b": [4, 5, 6]}), - ("x", 1): pd.DataFrame({"a": [4, 5, 6], "b": [3, 2, 1]}), - ("x", 2): pd.DataFrame({"a": [7, 8, 9], "b": [0, 0, 0]}), - } - meta = make_meta({"a": "i8", "b": "i8"}, parent_meta=pd.DataFrame()) - ddf1 = dd.DataFrame(dsk, "x", meta, [None, None]) - df1 = ddf1.compute() - - dsk = { - ("y", 0): pd.DataFrame({"a": [10, 20, 30], "b": [40, 50, 60]}), - ("y", 1): pd.DataFrame({"a": [40, 50, 60], "b": [30, 20, 10]}), - ("y", 2): pd.DataFrame({"a": [70, 80, 90], "b": [0, 0, 0]}), - } - ddf2 = dd.DataFrame(dsk, "y", meta, [None, None]) - df2 = ddf2.compute() - - dsk = { - ("y", 0): pd.DataFrame({"b": [10, 20, 30], "c": [40, 50, 60]}), - ("y", 1): pd.DataFrame({"b": [40, 50, 60], "c": [30, 20, 10]}), - } - meta = make_meta({"b": "i8", "c": "i8"}, parent_meta=pd.DataFrame()) - ddf3 = dd.DataFrame(dsk, "y", meta, [None, None]) - df3 = ddf3.compute() - - check_append_with_warning(ddf1, ddf2, df1, df2) - check_append_with_warning(ddf2, ddf1, df2, df1) - - # different columns - check_append_with_warning(ddf1, ddf3, df1, df3) - check_append_with_warning(ddf3, ddf1, df3, df1) - - # Dask + pandas - check_append_with_warning(ddf1, df2, df1, df2) - check_append_with_warning(ddf2, df1, df2, df1) - - # different columns - check_append_with_warning(ddf1, df3, df1, df3) - check_append_with_warning(ddf3, df1, df3, df1) - - -@pytest.mark.skipif(PANDAS_GE_200, reason="pandas removed append") -def test_append_categorical(): - frames = [ - pd.DataFrame( - { - "x": np.arange(5, 10), - "y": list("abbba"), - "z": np.arange(5, 10, dtype="f8"), - } - ), - pd.DataFrame( - { - "x": np.arange(10, 15), - "y": list("bcbcc"), - "z": np.arange(10, 15, dtype="f8"), - } - ), - ] - frames2 = [] - for df in frames: - df.y = df.y.astype("category") - df2 = df.copy() - df2.y = df2.y.cat.set_categories(list("abc")) - df.index = df.y - frames2.append(df2.set_index(df2.y)) - - df1, df2 = frames2 - - for known in [True, False]: - dframes = [dd.from_pandas(p, npartitions=2, sort=False) for p in frames] - if not known: - dframes[0]._meta = clear_known_categories( - dframes[0]._meta, ["y"], index=True - ) - ddf1, ddf2 = dframes - - res = check_append_with_warning(ddf1, ddf2, df1, df2) - - assert has_known_categories(res.index) == known - assert has_known_categories(res.y) == known - - res = check_append_with_warning(ddf1.y, ddf2.y, df1.y, df2.y) - assert has_known_categories(res.index) == known - assert has_known_categories(res) == known - - res = check_append_with_warning(ddf1.index, ddf2.index, df1.index, df2.index) - assert has_known_categories(res) == known - - -@pytest.mark.skipif(PANDAS_GE_200, reason="pandas removed append") -def test_append_lose_divisions(): - df = pd.DataFrame({"x": [1, 2, 3, 4]}, index=[1, 2, 3, 4]) - ddf = dd.from_pandas(df, npartitions=2) - - res = check_append_with_warning(ddf, ddf, df, df) - assert res.known_divisions is False - - def test_singleton_divisions(): df = pd.DataFrame({"x": [1, 1, 1]}, index=[1, 2, 3]) ddf = dd.from_pandas(df, npartitions=2) @@ -2728,14 +2583,14 @@ def test_concat_ignore_order(ordered): pytest.param( "int64[pyarrow]", marks=pytest.mark.skipif( - pa is None or not PANDAS_GE_150, + pa is None, reason="Support for ArrowDtypes requires pyarrow and pandas>=1.5.0", ), ), pytest.param( "float64[pyarrow]", marks=pytest.mark.skipif( - pa is None or not PANDAS_GE_150, + pa is None, reason="Support for ArrowDtypes requires pyarrow and pandas>=1.5.0", ), ), diff --git a/dask/dataframe/tests/test_pyarrow.py b/dask/dataframe/tests/test_pyarrow.py index 821c0d6d33e..3c603b0c5f5 100644 --- a/dask/dataframe/tests/test_pyarrow.py +++ b/dask/dataframe/tests/test_pyarrow.py @@ -5,7 +5,6 @@ import pytest from pandas.tests.extension.decimal.array import DecimalDtype -from dask.dataframe._compat import PANDAS_GE_140, PANDAS_GE_150 from dask.dataframe._pyarrow import ( is_object_string_dataframe, is_object_string_dtype, @@ -26,22 +25,10 @@ (np.dtype(float), False), (pd.StringDtype("python"), False), (DecimalDtype(), False), - pytest.param( - pa.int64(), - False, - marks=pytest.mark.skipif(not PANDAS_GE_150, reason="Needs pd.ArrowDtype"), - ), - pytest.param( - pa.float64(), - False, - marks=pytest.mark.skipif(not PANDAS_GE_150, reason="Needs pd.ArrowDtype"), - ), + (pa.int64(), False), + (pa.float64(), False), (pd.StringDtype("pyarrow"), True), - pytest.param( - pa.string(), - True, - marks=pytest.mark.skipif(not PANDAS_GE_150, reason="Needs pd.ArrowDtype"), - ), + (pa.string(), True), ], ) def test_is_pyarrow_string_dtype(dtype, expected): @@ -59,22 +46,10 @@ def test_is_pyarrow_string_dtype(dtype, expected): (np.dtype(float), False), (pd.StringDtype("python"), True), (DecimalDtype(), False), - pytest.param( - pa.int64(), - False, - marks=pytest.mark.skipif(not PANDAS_GE_150, reason="Needs pd.ArrowDtype"), - ), - pytest.param( - pa.float64(), - False, - marks=pytest.mark.skipif(not PANDAS_GE_150, reason="Needs pd.ArrowDtype"), - ), + (pa.int64(), False), + (pa.float64(), False), + (pa.string(), False), (pd.StringDtype("pyarrow"), False), - pytest.param( - pa.string(), - False, - marks=pytest.mark.skipif(not PANDAS_GE_150, reason="Needs pd.ArrowDtype"), - ), ], ) def test_is_object_string_dtype(dtype, expected): @@ -89,10 +64,7 @@ def test_is_object_string_dtype(dtype, expected): (pd.Index(["a", "b"], dtype=object), True), (pd.Index(["a", "b"], dtype="string[python]"), True), # Prior to pandas=1.4, Index couldn't contain extension dtypes - ( - pd.Index(["a", "b"], dtype="string[pyarrow]"), - False if PANDAS_GE_140 else True, - ), + (pd.Index(["a", "b"], dtype="string[pyarrow]"), False), (pd.Index([1, 2], dtype=int), False), (pd.Index([1, 2], dtype=float), False), (pd.Series(["a", "b"], dtype=object), False), @@ -113,7 +85,7 @@ def test_is_object_string_dtype(dtype, expected): pd.Index(["a", "b"], dtype="string[pyarrow]"), ] ), - False if PANDAS_GE_140 else True, + False, ), ( pd.MultiIndex.from_arrays( @@ -149,8 +121,7 @@ def test_is_object_string_index(index, expected): pd.Series( [1, 2], dtype=float, index=pd.Index(["a", "b"], dtype="string[pyarrow]") ), - # Prior to pandas=1.4, Index couldn't contain extension dtypes - False if PANDAS_GE_140 else True, + False, ), (pd.Index(["a", "b"], dtype=object), False), ], @@ -179,8 +150,7 @@ def test_is_object_string_series(series, expected): dtype=float, index=pd.Index(["a", "b"], dtype="string[pyarrow]"), ), - # Prior to pandas=1.4, Index couldn't contain extension dtypes - False if PANDAS_GE_140 else True, + False, ), (pd.Series({"x": ["a", "b"]}, dtype=object), False), (pd.Index({"x": ["a", "b"]}, dtype=object), False), diff --git a/dask/dataframe/tests/test_pyarrow_compat.py b/dask/dataframe/tests/test_pyarrow_compat.py index 7378f5dba6a..5bb99c66df4 100644 --- a/dask/dataframe/tests/test_pyarrow_compat.py +++ b/dask/dataframe/tests/test_pyarrow_compat.py @@ -13,17 +13,13 @@ pa = pytest.importorskip("pyarrow") import dask.dataframe as dd -from dask.dataframe._compat import PANDAS_GE_150 # Tests are from https://github.com/pandas-dev/pandas/pull/49078 @pytest.fixture def data(dtype): - if PANDAS_GE_150: - pa_dtype = dtype.pyarrow_dtype - else: - pa_dtype = pa.string() + pa_dtype = dtype.pyarrow_dtype if pa.types.is_boolean(pa_dtype): data = [True, False] * 4 + [None] + [True, False] * 44 + [None] + [True, False] elif pa.types.is_floating(pa_dtype): @@ -81,15 +77,12 @@ def data(dtype): return pd.array(data * 100, dtype=dtype) -PYARROW_TYPES = tm.ALL_PYARROW_DTYPES if PANDAS_GE_150 else [pa.string()] +PYARROW_TYPES = tm.ALL_PYARROW_DTYPES @pytest.fixture(params=PYARROW_TYPES, ids=str) def dtype(request): - if PANDAS_GE_150: - return pd.ArrowDtype(pyarrow_dtype=request.param) - else: - return pd.StringDtype("pyarrow") + return pd.ArrowDtype(pyarrow_dtype=request.param) def test_pickle_roundtrip(data): @@ -112,10 +105,7 @@ def test_pickle_roundtrip(data): "string_dtype", [ "stringdtype", - pytest.param( - "arrowdtype", - marks=pytest.mark.skipif(not PANDAS_GE_150, reason="Requires ArrowDtype"), - ), + "arrowdtype", ], ) def test_pickle_roundtrip_pyarrow_string_implementations(string_dtype): diff --git a/dask/dataframe/tests/test_reshape.py b/dask/dataframe/tests/test_reshape.py index 8bdeffb2c94..04b53b7299e 100644 --- a/dask/dataframe/tests/test_reshape.py +++ b/dask/dataframe/tests/test_reshape.py @@ -6,11 +6,9 @@ import numpy as np import pandas as pd import pytest -from packaging.version import Version import dask.dataframe as dd -from dask.dataframe._compat import PANDAS_VERSION, tm -from dask.dataframe.reshape import _get_dummies_dtype_default +from dask.dataframe._compat import tm from dask.dataframe.utils import assert_eq @@ -94,25 +92,6 @@ def test_get_dummies_kwargs(): assert_eq(res, exp) -def check_pandas_issue_45618_warning(test_func): - # Check for FutureWarning raised in `pandas=1.4.0`-only. - # This can be removed when `pandas=1.4.0` is no longer supported (PANDAS_GE_140). - # See https://github.com/pandas-dev/pandas/issues/45618 for more details. - - def decorator(): - with warnings.catch_warnings(record=True) as record: - test_func() - if PANDAS_VERSION == Version("1.4.0"): - assert all( - "In a future version, passing a SparseArray" in str(r.message) - for r in record - ) - else: - assert not record - - return decorator - - @contextlib.contextmanager def ignore_numpy_bool8_deprecation(): # This warning comes from inside `pandas`. We can't do anything about it, so we ignore the warning. @@ -126,7 +105,6 @@ def ignore_numpy_bool8_deprecation(): yield -@check_pandas_issue_45618_warning def test_get_dummies_sparse(): s = pd.Series(pd.Categorical(["a", "b", "a"], categories=["a", "b", "c"])) ds = dd.from_pandas(s, 2) @@ -137,8 +115,8 @@ def test_get_dummies_sparse(): assert_eq(exp, res) dtype = res.compute().a.dtype - assert dtype.fill_value == _get_dummies_dtype_default(0) - assert dtype.subtype == _get_dummies_dtype_default + assert dtype.fill_value == bool(0) + assert dtype.subtype == bool assert isinstance(res.a.compute().dtype, pd.SparseDtype) exp = pd.get_dummies(s.to_frame(name="a"), sparse=True) @@ -148,7 +126,6 @@ def test_get_dummies_sparse(): assert isinstance(res.a_a.compute().dtype, pd.SparseDtype) -@check_pandas_issue_45618_warning def test_get_dummies_sparse_mix(): df = pd.DataFrame( { @@ -164,8 +141,8 @@ def test_get_dummies_sparse_mix(): assert_eq(exp, res) dtype = res.compute().A_a.dtype - assert dtype.fill_value == _get_dummies_dtype_default(0) - assert dtype.subtype == _get_dummies_dtype_default + assert dtype.fill_value == bool(0) + assert dtype.subtype == bool assert isinstance(res.A_a.compute().dtype, pd.SparseDtype) diff --git a/dask/dataframe/tests/test_rolling.py b/dask/dataframe/tests/test_rolling.py index 309ebaba008..122ec6c526e 100644 --- a/dask/dataframe/tests/test_rolling.py +++ b/dask/dataframe/tests/test_rolling.py @@ -8,7 +8,7 @@ import pytest import dask.dataframe as dd -from dask.dataframe._compat import PANDAS_GE_200, PANDAS_GE_210 +from dask.dataframe._compat import PANDAS_GE_210 from dask.dataframe.utils import assert_eq DASK_EXPR_ENABLED = dd._dask_expr_enabled() @@ -557,9 +557,7 @@ def test_rolling_agg_aggregate(): ) -@pytest.mark.skipif( - PANDAS_GE_200 and not PANDAS_GE_210, reason="buggy pandas implementation" -) +@pytest.mark.skipif(not PANDAS_GE_210, reason="buggy pandas implementation") def test_rolling_numba_engine(): pytest.importorskip("numba") df = pd.DataFrame({"A": range(5), "B": range(0, 10, 2)}) diff --git a/dask/dataframe/tests/test_shuffle.py b/dask/dataframe/tests/test_shuffle.py index 006bc7869b5..288ee79e75c 100644 --- a/dask/dataframe/tests/test_shuffle.py +++ b/dask/dataframe/tests/test_shuffle.py @@ -22,14 +22,7 @@ import dask import dask.dataframe as dd from dask.base import compute_as_if_collection -from dask.dataframe._compat import ( - PANDAS_GE_140, - PANDAS_GE_150, - PANDAS_GE_200, - PANDAS_GE_220, - assert_categorical_equal, - tm, -) +from dask.dataframe._compat import PANDAS_GE_220, assert_categorical_equal, tm from dask.dataframe.shuffle import ( _calculate_divisions, _noop, @@ -225,10 +218,7 @@ def test_set_index_general(npartitions, shuffle_method): index=np.random.random(100), ) # Ensure extension dtypes work - # NOTE: Older version of pandas have known issues with extension dtypes. - # We generally expect extension dtypes to work well when using `pandas>=1.4.0`. - if PANDAS_GE_140: - df = df.astype({"x": "Float64", "z": "string"}) + df = df.astype({"x": "Float64", "z": "string"}) ddf = dd.from_pandas(df, npartitions=npartitions) @@ -250,9 +240,6 @@ def test_set_index_general(npartitions, shuffle_method): assert_eq(df.set_index("x"), ddf.set_index("x", shuffle=shuffle_method)) -@pytest.mark.skipif( - not PANDAS_GE_150, reason="Only test `string[pyarrow]` on recent versions of pandas" -) @pytest.mark.parametrize( "string_dtype", ["string[python]", "string[pyarrow]", "object"] ) @@ -1196,7 +1183,6 @@ def test_set_index_timestamp(): assert_eq(df2, ddf.set_index("A"), check_freq=False) -@pytest.mark.skipif(not PANDAS_GE_140, reason="EA Indexes not supported before") def test_set_index_ea_dtype(): pdf = pd.DataFrame({"a": 1, "b": pd.Series([1, 2], dtype="Int64")}) ddf = dd.from_pandas(pdf, npartitions=2) @@ -1793,7 +1779,6 @@ def test_calculate_divisions(pdf, expected): @pytest.mark.skipif(pa is None, reason="Need pyarrow") -@pytest.mark.skipif(not PANDAS_GE_200, reason="dtype support not good before 2.0") @pytest.mark.parametrize( "data, dtype", [ diff --git a/dask/dataframe/tests/test_utils_dataframe.py b/dask/dataframe/tests/test_utils_dataframe.py index 3afa84a8536..700638b5741 100644 --- a/dask/dataframe/tests/test_utils_dataframe.py +++ b/dask/dataframe/tests/test_utils_dataframe.py @@ -11,7 +11,7 @@ import dask import dask.dataframe as dd -from dask.dataframe._compat import PANDAS_GE_200, PANDAS_GE_300, tm +from dask.dataframe._compat import PANDAS_GE_300, tm from dask.dataframe.core import apply_and_enforce from dask.dataframe.utils import ( UNKNOWN_CATEGORIES, @@ -273,11 +273,7 @@ def test_meta_nonempty_index(): idx = pd.Index([1], name="foo", dtype="int") res = meta_nonempty(idx) assert type(res) is type(idx) - if PANDAS_GE_200: - assert res.dtype == np.int_ - else: - # before pandas 2.0, index dtypes were only x64 - assert res.dtype == "int64" + assert res.dtype == np.int_ assert res.name == idx.name idx = pd.Index(["a"], name="foo") @@ -682,12 +678,8 @@ def test_pyarrow_strings_enabled(): except ImportError: pa = None - # If `pandas>=2` and `pyarrow>=12` are installed, then default to using pyarrow strings - if ( - PANDAS_GE_200 - and pa is not None - and Version(pa.__version__) >= Version("12.0.0") - ): + # If `pyarrow>=12` are installed, then default to using pyarrow strings + if pa is not None and Version(pa.__version__) >= Version("12.0.0"): assert pyarrow_strings_enabled() is True else: assert pyarrow_strings_enabled() is False diff --git a/dask/dataframe/tseries/resample.py b/dask/dataframe/tseries/resample.py index 3387c1ea1d2..bfad295a183 100644 --- a/dask/dataframe/tseries/resample.py +++ b/dask/dataframe/tseries/resample.py @@ -6,7 +6,6 @@ from dask.base import tokenize from dask.dataframe import methods -from dask.dataframe._compat import PANDAS_GE_140 from dask.dataframe.core import DataFrame, Series from dask.highlevelgraph import HighLevelGraph from dask.utils import derived_from @@ -27,15 +26,11 @@ def _resample_series( out = getattr(series.resample(rule, **resample_kwargs), how)( *how_args, **how_kwargs ) - - if PANDAS_GE_140: - if reindex_closed is None: - inclusive = "both" - else: - inclusive = reindex_closed - closed_kwargs = {"inclusive": inclusive} + if reindex_closed is None: + inclusive = "both" else: - closed_kwargs = {"closed": reindex_closed} + inclusive = reindex_closed + closed_kwargs = {"inclusive": inclusive} new_index = pd.date_range( start.tz_localize(None), diff --git a/dask/dataframe/utils.py b/dask/dataframe/utils.py index 38eca427597..cb7454782ff 100644 --- a/dask/dataframe/utils.py +++ b/dask/dataframe/utils.py @@ -21,7 +21,7 @@ _dtypes, methods, ) -from dask.dataframe._compat import PANDAS_GE_150, tm # noqa: F401 +from dask.dataframe._compat import tm # noqa: F401 from dask.dataframe.dispatch import ( # noqa : F401 is_categorical_dtype_dispatch, make_meta, @@ -849,7 +849,7 @@ def get_numeric_only_kwargs(numeric_only: bool | NoDefault) -> dict: def check_numeric_only_valid(numeric_only: bool | NoDefault, name: str) -> dict: - if PANDAS_GE_150 and numeric_only is not no_default: + if numeric_only is not no_default: return {"numeric_only": numeric_only} elif numeric_only is no_default: return {} diff --git a/dask/tests/test_spark_compat.py b/dask/tests/test_spark_compat.py index 061a440d11e..e3683de4825 100644 --- a/dask/tests/test_spark_compat.py +++ b/dask/tests/test_spark_compat.py @@ -18,7 +18,6 @@ import numpy as np import pandas as pd -from dask.dataframe._compat import PANDAS_GE_150, PANDAS_GE_200 from dask.dataframe.utils import assert_eq pytestmark = [ @@ -26,8 +25,7 @@ sys.platform != "linux", reason="Unnecessary, and hard to get spark working on non-linux platforms", ), - pytest.mark.skipif( - PANDAS_GE_200, + pytest.mark.skip( reason="pyspark doesn't yet have support for pandas 2.0", ), # we only test with pyarrow strings and pandas 2.0 @@ -175,7 +173,6 @@ def test_roundtrip_parquet_spark_to_dask_extension_dtypes(spark_session, tmpdir) assert_eq(ddf, pdf, check_index=False) -@pytest.mark.skipif(not PANDAS_GE_150, reason="Requires pyarrow-backed nullable dtypes") def test_read_decimal_dtype_pyarrow(spark_session, tmpdir): tmpdir = str(tmpdir) npartitions = 3 diff --git a/docs/requirements-docs.txt b/docs/requirements-docs.txt index de3d49b196c..f60d05511ed 100644 --- a/docs/requirements-docs.txt +++ b/docs/requirements-docs.txt @@ -17,7 +17,7 @@ sphinxcontrib-serializinghtml>=1.1.0,<1.1.10 sphinxcontrib-qthelp>=1.0.0,<1.0.7 toolz cloudpickle>=1.5.0 -pandas>=1.4.0 +pandas>=2.0.0 git+https://github.com/dask/distributed git+https://github.com/dask-contrib/dask-expr fsspec diff --git a/docs/source/changelog.rst b/docs/source/changelog.rst index e14cbf6e2e9..e7b69ad9996 100644 --- a/docs/source/changelog.rst +++ b/docs/source/changelog.rst @@ -1,6 +1,24 @@ Changelog ========= +.. _v2024.6.3: + +2024.6.3 +-------- + + +Highlights +^^^^^^^^^^ + +Drop pandas 1.x support +""""""""""""""""""""""" + +This release drops support for pandas versions of the 1.x series. pandas 2.0 +is now the required minimum version to run Dask DataFrame. + +The mimimum version of partd was also raised to 1.4.0. The versions before 1.4 +are not compatible with pandas 2. + .. _v2024.6.2: 2024.6.2 diff --git a/docs/source/install.rst b/docs/source/install.rst index c028a9988dd..feded0f3a86 100644 --- a/docs/source/install.rst +++ b/docs/source/install.rst @@ -142,7 +142,7 @@ These optional dependencies and their minimum supported versions are listed belo +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ | `numpy`_ | ``>=1.21`` | Required for ``dask.array`` | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ -| `pandas`_ | ``>=1.3`` | Required for ``dask.dataframe`` | +| `pandas`_ | ``>=2.0`` | Required for ``dask.dataframe`` | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ | `psutil`_ | ``>=5.7.2`` | Factor CPU affinity into CPU count, intelligently infer blocksize when reading CSV files | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ diff --git a/pyproject.toml b/pyproject.toml index 1d036cbb954..e8547661d0f 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -34,7 +34,7 @@ dependencies = [ "cloudpickle >= 1.5.0", "fsspec >= 2021.09.0", "packaging >= 20.0", - "partd >= 1.2.0", + "partd >= 1.4.0", "pyyaml >= 5.3.1", "toolz >= 0.10.0", # importlib.metadata has the following bugs fixed in 3.10.9 and 3.11.1 @@ -52,9 +52,7 @@ array = ["numpy >= 1.21"] bag = [] dataframe = [ "dask[array]", - # NOTE: dask-expr is pinning pandas>2. To use pandas<2, please install dask - # without using the dataframe extra - "pandas >= 1.3", + "pandas >= 2.0", "dask-expr >= 1.1, <1.2", # dask-expr pins the dask version ] distributed = ["distributed == 2024.6.2"] From e1bd6d13335fa04d55bf5844d734b1bdae6ea8c5 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Mon, 1 Jul 2024 18:00:13 +0200 Subject: [PATCH 18/72] Implement array expression switch (#11203) --- dask/array/__init__.py | 321 ++++++++++++++++++ dask/array/tests/test_array_core.py | 24 +- .../tests/test_array_expr_incompatible.py | 31 ++ dask/dask-schema.yaml | 4 + dask/dask.yaml | 1 + 5 files changed, 360 insertions(+), 21 deletions(-) create mode 100644 dask/array/tests/test_array_expr_incompatible.py diff --git a/dask/array/__init__.py b/dask/array/__init__.py index 3aea329c4db..6637e132c55 100644 --- a/dask/array/__init__.py +++ b/dask/array/__init__.py @@ -1,5 +1,35 @@ from __future__ import annotations +import importlib +import warnings + +# The "array.query-planning" config can only be processed once +ARRAY_EXPR_ENABLED: bool | None = None + + +def _array_expr_enabled() -> bool: + import dask + + global ARRAY_EXPR_ENABLED + + use_array_expr = dask.config.get("array.query-planning") + + if ARRAY_EXPR_ENABLED is not None: + if (use_array_expr is True and ARRAY_EXPR_ENABLED is False) or ( + use_array_expr is False and ARRAY_EXPR_ENABLED is True + ): + warnings.warn( + "The 'dataframe.query-planning' config is now set to " + f"{use_array_expr}, but query planning is already " + f"{'enabled' if ARRAY_EXPR_ENABLED else 'disabled'}. " + "The query-planning config can only be changed before " + "`dask.dataframe` is first imported!" + ) + return ARRAY_EXPR_ENABLED + + return use_array_expr if use_array_expr is not None else False + + try: from dask.array import backends, fft, lib, linalg, ma, overlap, random from dask.array.blockwise import atop, blockwise @@ -263,6 +293,11 @@ from dask.array.wrap import empty, full, ones, zeros from dask.base import compute + if _array_expr_enabled(): + import dask_expr.array as da + + da = importlib.reload(da) + except ImportError as e: msg = ( "Dask array requirements are not installed.\n\n" @@ -271,3 +306,289 @@ ' python -m pip install "dask[array]" --upgrade # or python -m pip install' ) raise ImportError(str(e) + "\n\n" + msg) from e + + +if _array_expr_enabled(): + + def raise_not_implemented_error(attr_name): + def inner_func(*args, **kwargs): + raise NotImplementedError( + f"Function {attr_name} is not implemented for dask-expr." + ) + + return inner_func + + try: + from dask_expr.array import ( # type: ignore + Array, + asarray, + from_array, + mean, + moment, + nanmean, + nanstd, + nansum, + nanvar, + prod, + random, + std, + sum, + var, + ) + + backends = raise_not_implemented_error("backends") + fft = raise_not_implemented_error("fft") + lib = raise_not_implemented_error("lib") + linalg = raise_not_implemented_error("linalg") + ma = raise_not_implemented_error("ma") + overlap = raise_not_implemented_error("overlap") + atop = raise_not_implemented_error("atop") + blockwise = raise_not_implemented_error("blockwise") + register_chunk_type = raise_not_implemented_error("register_chunk_type") + asanyarray = raise_not_implemented_error("asanyarray") + block = raise_not_implemented_error("block") + broadcast_arrays = raise_not_implemented_error("broadcast_arrays") + broadcast_to = raise_not_implemented_error("broadcast_to") + concatenate = raise_not_implemented_error("concatenate") + from_delayed = raise_not_implemented_error("from_delayed") + from_npy_stack = raise_not_implemented_error("from_npy_stack") + from_zarr = raise_not_implemented_error("from_zarr") + map_blocks = raise_not_implemented_error("map_blocks") + stack = raise_not_implemented_error("stack") + store = raise_not_implemented_error("store") + to_hdf5 = raise_not_implemented_error("to_hdf5") + to_npy_stack = raise_not_implemented_error("to_npy_stack") + to_zarr = raise_not_implemented_error("to_zarr") + unify_chunks = raise_not_implemented_error("unify_chunks") + arange = raise_not_implemented_error("arange") + diag = raise_not_implemented_error("diag") + diagonal = raise_not_implemented_error("diagonal") + empty_like = raise_not_implemented_error("empty_like") + eye = raise_not_implemented_error("eye") + fromfunction = raise_not_implemented_error("fromfunction") + full_like = raise_not_implemented_error("full_like") + indices = raise_not_implemented_error("indices") + linspace = raise_not_implemented_error("linspace") + meshgrid = raise_not_implemented_error("meshgrid") + ones_like = raise_not_implemented_error("ones_like") + pad = raise_not_implemented_error("pad") + repeat = raise_not_implemented_error("repeat") + tile = raise_not_implemented_error("tile") + tri = raise_not_implemented_error("tri") + zeros_like = raise_not_implemented_error("zeros_like") + apply_gufunc = raise_not_implemented_error("apply_gufunc") + as_gufunc = raise_not_implemented_error("as_gufunc") + gufunc = raise_not_implemented_error("gufunc") # type: ignore + moveaxis = raise_not_implemented_error("moveaxis") + rollaxis = raise_not_implemented_error("rollaxis") + optimize = raise_not_implemented_error("optimize") + map_overlap = raise_not_implemented_error("map_overlap") + percentile = raise_not_implemented_error("percentile") + rechunk = raise_not_implemented_error("rechunk") + all = raise_not_implemented_error("all") + any = raise_not_implemented_error("any") + argmax = raise_not_implemented_error("argmax") + argmin = raise_not_implemented_error("argmin") + argtopk = raise_not_implemented_error("argtopk") + cumprod = raise_not_implemented_error("cumprod") + cumsum = raise_not_implemented_error("cumsum") + max = raise_not_implemented_error("max") + median = raise_not_implemented_error("median") + min = raise_not_implemented_error("min") + nanargmax = raise_not_implemented_error("nanargmax") + nanargmin = raise_not_implemented_error("nanargmin") + nancumprod = raise_not_implemented_error("nancumprod") + nancumsum = raise_not_implemented_error("nancumsum") + nanmax = raise_not_implemented_error("nanmax") + nanmedian = raise_not_implemented_error("nanmedian") + nanmin = raise_not_implemented_error("nanmin") + nanprod = raise_not_implemented_error("nanprod") + reduction = raise_not_implemented_error("reduction") + topk = raise_not_implemented_error("topk") + trace = raise_not_implemented_error("trace") + reshape = raise_not_implemented_error("reshape") + allclose = raise_not_implemented_error("allclose") + append = raise_not_implemented_error("append") + apply_along_axis = raise_not_implemented_error("apply_along_axis") + apply_over_axes = raise_not_implemented_error("apply_over_axes") + argwhere = raise_not_implemented_error("argwhere") + around = raise_not_implemented_error("around") + array = raise_not_implemented_error("array") + atleast_1d = raise_not_implemented_error("atleast_1d") + atleast_2d = raise_not_implemented_error("atleast_2d") + atleast_3d = raise_not_implemented_error("atleast_3d") + average = raise_not_implemented_error("average") + bincount = raise_not_implemented_error("bincount") + choose = raise_not_implemented_error("choose") + coarsen = raise_not_implemented_error("coarsen") + compress = raise_not_implemented_error("compress") + corrcoef = raise_not_implemented_error("corrcoef") + count_nonzero = raise_not_implemented_error("count_nonzero") + cov = raise_not_implemented_error("cov") + delete = raise_not_implemented_error("delete") + diff = raise_not_implemented_error("diff") + digitize = raise_not_implemented_error("digitize") + dot = raise_not_implemented_error("dot") + dstack = raise_not_implemented_error("dstack") + ediff1d = raise_not_implemented_error("ediff1d") + einsum = raise_not_implemented_error("einsum") + expand_dims = raise_not_implemented_error("expand_dims") + extract = raise_not_implemented_error("extract") + flatnonzero = raise_not_implemented_error("flatnonzero") + flip = raise_not_implemented_error("flip") + fliplr = raise_not_implemented_error("fliplr") + flipud = raise_not_implemented_error("flipud") + gradient = raise_not_implemented_error("gradient") + histogram = raise_not_implemented_error("histogram") + histogram2d = raise_not_implemented_error("histogram2d") + histogramdd = raise_not_implemented_error("histogramdd") + hstack = raise_not_implemented_error("hstack") + insert = raise_not_implemented_error("insert") + isclose = raise_not_implemented_error("isclose") + isin = raise_not_implemented_error("isin") + isnull = raise_not_implemented_error("isnull") + matmul = raise_not_implemented_error("matmul") + ndim = raise_not_implemented_error("ndim") + nonzero = raise_not_implemented_error("nonzero") + notnull = raise_not_implemented_error("notnull") + outer = raise_not_implemented_error("outer") + piecewise = raise_not_implemented_error("piecewise") + ptp = raise_not_implemented_error("ptp") + ravel = raise_not_implemented_error("ravel") + ravel_multi_index = raise_not_implemented_error("ravel_multi_index") + result_type = raise_not_implemented_error("result_type") + roll = raise_not_implemented_error("roll") + rot90 = raise_not_implemented_error("rot90") + round = raise_not_implemented_error("round") + searchsorted = raise_not_implemented_error("searchsorted") + select = raise_not_implemented_error("select") + shape = raise_not_implemented_error("shape") + squeeze = raise_not_implemented_error("squeeze") + swapaxes = raise_not_implemented_error("swapaxes") + take = raise_not_implemented_error("take") + tensordot = raise_not_implemented_error("tensordot") + transpose = raise_not_implemented_error("transpose") + tril = raise_not_implemented_error("tril") + tril_indices = raise_not_implemented_error("tril_indices") + tril_indices_from = raise_not_implemented_error("tril_indices_from") + triu = raise_not_implemented_error("triu") + triu_indices = raise_not_implemented_error("triu_indices") + triu_indices_from = raise_not_implemented_error("triu_indices_from") + union1d = raise_not_implemented_error("union1d") + unique = raise_not_implemented_error("unique") + unravel_index = raise_not_implemented_error("unravel_index") + vdot = raise_not_implemented_error("vdot") + vstack = raise_not_implemented_error("vstack") + where = raise_not_implemented_error("where") + from_tiledb = raise_not_implemented_error("from_tiledb") + to_tiledb = raise_not_implemented_error("to_tiledb") + abs = raise_not_implemented_error("abs") + absolute = raise_not_implemented_error("absolute") + add = raise_not_implemented_error("add") + angle = raise_not_implemented_error("angle") + arccos = raise_not_implemented_error("arccos") + arccosh = raise_not_implemented_error("arccosh") + arcsin = raise_not_implemented_error("arcsin") + arcsinh = raise_not_implemented_error("arcsinh") + arctan = raise_not_implemented_error("arctan") + arctan2 = raise_not_implemented_error("arctan2") + arctanh = raise_not_implemented_error("arctanh") + bitwise_and = raise_not_implemented_error("bitwise_and") + bitwise_not = raise_not_implemented_error("bitwise_not") + bitwise_or = raise_not_implemented_error("bitwise_or") + bitwise_xor = raise_not_implemented_error("bitwise_xor") + cbrt = raise_not_implemented_error("cbrt") + ceil = raise_not_implemented_error("ceil") + clip = raise_not_implemented_error("clip") + conj = raise_not_implemented_error("conj") + copysign = raise_not_implemented_error("copysign") + cos = raise_not_implemented_error("cos") + cosh = raise_not_implemented_error("cosh") + deg2rad = raise_not_implemented_error("deg2rad") + degrees = raise_not_implemented_error("degrees") + divide = raise_not_implemented_error("divide") + divmod = raise_not_implemented_error("divmod") + equal = raise_not_implemented_error("equal") + exp = raise_not_implemented_error("exp") + exp2 = raise_not_implemented_error("exp2") + expm1 = raise_not_implemented_error("expm1") + fabs = raise_not_implemented_error("fabs") + fix = raise_not_implemented_error("fix") + float_power = raise_not_implemented_error("float_power") + floor = raise_not_implemented_error("floor") + floor_divide = raise_not_implemented_error("floor_divide") + fmax = raise_not_implemented_error("fmax") + fmin = raise_not_implemented_error("fmin") + fmod = raise_not_implemented_error("fmod") + frexp = raise_not_implemented_error("frexp") + frompyfunc = raise_not_implemented_error("frompyfunc") + greater = raise_not_implemented_error("greater") + greater_equal = raise_not_implemented_error("greater_equal") + hypot = raise_not_implemented_error("hypot") + i0 = raise_not_implemented_error("i0") + imag = raise_not_implemented_error("imag") + invert = raise_not_implemented_error("invert") + iscomplex = raise_not_implemented_error("iscomplex") + isfinite = raise_not_implemented_error("isfinite") + isinf = raise_not_implemented_error("isinf") + isnan = raise_not_implemented_error("isnan") + isneginf = raise_not_implemented_error("isneginf") + isposinf = raise_not_implemented_error("isposinf") + isreal = raise_not_implemented_error("isreal") + ldexp = raise_not_implemented_error("ldexp") + left_shift = raise_not_implemented_error("left_shift") + less = raise_not_implemented_error("less") + less_equal = raise_not_implemented_error("less_equal") + log = raise_not_implemented_error("log") + log1p = raise_not_implemented_error("log1p") + log2 = raise_not_implemented_error("log2") + log10 = raise_not_implemented_error("log10") + logaddexp = raise_not_implemented_error("logaddexp") + logaddexp2 = raise_not_implemented_error("logaddexp2") + logical_and = raise_not_implemented_error("logical_and") + logical_not = raise_not_implemented_error("logical_not") + logical_or = raise_not_implemented_error("logical_or") + logical_xor = raise_not_implemented_error("logical_xor") + maximum = raise_not_implemented_error("maximum") + minimum = raise_not_implemented_error("minimum") + mod = raise_not_implemented_error("mod") + modf = raise_not_implemented_error("modf") + multiply = raise_not_implemented_error("multiply") + nan_to_num = raise_not_implemented_error("nan_to_num") + negative = raise_not_implemented_error("negative") + nextafter = raise_not_implemented_error("nextafter") + not_equal = raise_not_implemented_error("not_equal") + positive = raise_not_implemented_error("positive") + power = raise_not_implemented_error("power") + rad2deg = raise_not_implemented_error("rad2deg") + radians = raise_not_implemented_error("radians") + real = raise_not_implemented_error("real") + reciprocal = raise_not_implemented_error("reciprocal") + remainder = raise_not_implemented_error("remainder") + right_shift = raise_not_implemented_error("right_shift") + rint = raise_not_implemented_error("rint") + sign = raise_not_implemented_error("sign") + signbit = raise_not_implemented_error("signbit") + sin = raise_not_implemented_error("sin") + sinc = raise_not_implemented_error("sinc") + sinh = raise_not_implemented_error("sinh") + spacing = raise_not_implemented_error("spacing") + sqrt = raise_not_implemented_error("sqrt") + square = raise_not_implemented_error("square") + subtract = raise_not_implemented_error("subtract") + tan = raise_not_implemented_error("tan") + tanh = raise_not_implemented_error("tanh") + true_divide = raise_not_implemented_error("true_divide") + trunc = raise_not_implemented_error("trunc") + empty = raise_not_implemented_error("empty") + full = raise_not_implemented_error("full") + ones = raise_not_implemented_error("ones") + zeros = raise_not_implemented_error("zeros") + + from dask.array.utils import assert_eq + from dask.base import compute + + except ImportError: + import dask.array as da + + da = importlib.reload(da) diff --git a/dask/array/tests/test_array_core.py b/dask/array/tests/test_array_core.py index df9bf6449be..0bf3fcc75aa 100644 --- a/dask/array/tests/test_array_core.py +++ b/dask/array/tests/test_array_core.py @@ -5,7 +5,6 @@ import pathlib import re import xml.etree.ElementTree -from unittest import mock import pytest @@ -53,7 +52,6 @@ ) from dask.array.numpy_compat import NUMPY_GE_200 from dask.array.reshape import _not_implemented_message -from dask.array.tests.test_dispatch import EncapsulateNDArray from dask.array.utils import assert_eq, same_keys from dask.base import compute_as_if_collection, tokenize from dask.blockwise import broadcast_dimensions @@ -2905,24 +2903,6 @@ def test_concatenate3_2(): ) -@pytest.mark.parametrize("one_d", [True, False]) -@mock.patch.object(da.core, "_concatenate2", wraps=da.core._concatenate2) -def test_concatenate3_nep18_dispatching(mock_concatenate2, one_d): - x = EncapsulateNDArray(np.arange(10)) - concat = [x, x] if one_d else [[x[None]], [x[None]]] - result = concatenate3(concat) - assert type(result) is type(x) - mock_concatenate2.assert_called() - mock_concatenate2.reset_mock() - - # When all the inputs are supported by plain `np.concatenate`, we should take the concatenate3 - # fastpath of allocating the full array up front and writing blocks into it. - concat = [x.arr, x.arr] if one_d else [[x.arr[None]], [x.arr[None]]] - plain_np_result = concatenate3(concat) - mock_concatenate2.assert_not_called() - assert type(plain_np_result) is np.ndarray - - def test_map_blocks3(): x = np.arange(10) y = np.arange(10) * 2 @@ -4701,8 +4681,10 @@ def test_zarr_group(): ], ) def test_regular_chunks(data): + from dask.array.core import _check_regular_chunks + chunkset, expected = data - assert da.core._check_regular_chunks(chunkset) == expected + assert _check_regular_chunks(chunkset) == expected def test_zarr_nocompute(): diff --git a/dask/array/tests/test_array_expr_incompatible.py b/dask/array/tests/test_array_expr_incompatible.py new file mode 100644 index 00000000000..454d331a880 --- /dev/null +++ b/dask/array/tests/test_array_expr_incompatible.py @@ -0,0 +1,31 @@ +from __future__ import annotations + +from unittest import mock + +import numpy as np +import pytest + +import dask.array as da +from dask.array.core import concatenate3 +from dask.array.tests.test_dispatch import EncapsulateNDArray + +if da._array_expr_enabled(): + pytest.skip("mock doesn't work") + + +@pytest.mark.parametrize("one_d", [True, False]) +@mock.patch.object(da.core, "_concatenate2", wraps=da.core._concatenate2) +def test_concatenate3_nep18_dispatching(mock_concatenate2, one_d): + x = EncapsulateNDArray(np.arange(10)) + concat = [x, x] if one_d else [[x[None]], [x[None]]] + result = concatenate3(concat) + assert type(result) is type(x) + mock_concatenate2.assert_called() + mock_concatenate2.reset_mock() + + # When all the inputs are supported by plain `np.concatenate`, we should take the concatenate3 + # fastpath of allocating the full array up front and writing blocks into it. + concat = [x.arr, x.arr] if one_d else [[x.arr[None]], [x.arr[None]]] + plain_np_result = concatenate3(concat) + mock_concatenate2.assert_not_called() + assert type(plain_np_result) is np.ndarray diff --git a/dask/dask-schema.yaml b/dask/dask-schema.yaml index 55a1aaacf26..12bf2ee0644 100644 --- a/dask/dask-schema.yaml +++ b/dask/dask-schema.yaml @@ -155,6 +155,10 @@ properties: warning is produced. Set to ``False`` to silence the warning and allow large output chunks. Set to ``True`` to silence the warning and avoid large output chunks. + query-planning: + type: [boolean, "null"] + description: | + Whether to use query planning for arrays. optimization: type: object diff --git a/dask/dask.yaml b/dask/dask.yaml index d312117425c..06509fe8764 100644 --- a/dask/dask.yaml +++ b/dask/dask.yaml @@ -28,6 +28,7 @@ array: size: 120 # pixels slicing: split-large-chunks: null # How to handle large output chunks in slicing. Warns by default. + query-planning: null # Whether to use dask-expr optimization: annotations: From 77b13b6beb2c6f763bbd1a3f1b0c5d929690eded Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Tue, 2 Jul 2024 11:41:53 -0400 Subject: [PATCH 19/72] Add utility to verify optional dependencies (#11205) --- dask/_compatibility.py | 115 ++++++++++++++++++++++++++ dask/array/numpy_compat.py | 3 + dask/dataframe/_compat.py | 6 +- dask/dataframe/_pyarrow.py | 3 +- dask/diagnostics/profile_visualize.py | 3 +- 5 files changed, 127 insertions(+), 3 deletions(-) diff --git a/dask/_compatibility.py b/dask/_compatibility.py index d628d3cbdb2..9267e7add21 100644 --- a/dask/_compatibility.py +++ b/dask/_compatibility.py @@ -1,7 +1,10 @@ from __future__ import annotations +import importlib import sys +import types import warnings +from typing import Literal if sys.version_info >= (3, 12): import importlib.metadata as importlib_metadata @@ -22,3 +25,115 @@ def entry_points(group=None): stacklevel=2, ) return importlib_metadata.entry_points(group=group) + + +VERSIONS = { + "numpy": "1.21.0", + "pandas": "2.0.0", + "bokeh": "2.4.2", + "jinja2": "2.10.3", + "pyarrow": "7.0.0", + "lz4": "4.3.2", +} + +# A mapping from import name to package name (on PyPI) for packages where +# these two names are different. + +INSTALL_MAPPING = { + "sqlalchemy": "SQLAlchemy", + "tables": "pytables", +} + + +def get_version(module: types.ModuleType) -> str: + version = getattr(module, "__version__", None) + + if version is None: + raise ImportError(f"Can't determine version for {module.__name__}") + if module.__name__ == "psycopg2": + # psycopg2 appends " (dt dec pq3 ext lo64)" to it's version + version = version.split()[0] + return version + + +def import_optional_dependency( + name: str, + extra: str = "", + min_version: str | None = None, + *, + errors: Literal["raise", "warn", "ignore"] = "raise", +) -> types.ModuleType | None: + """ + Import an optional dependency. + + By default, if a dependency is missing an ImportError with a nice + message will be raised. If a dependency is present, but too old, + we raise. + + Parameters + ---------- + name : str + The module name. + extra : str + Additional text to include in the ImportError message. + errors : str {'raise', 'warn', 'ignore'} + What to do when a dependency is not found or its version is too old. + + * raise : Raise an ImportError + * warn : Only applicable when a module's version is to old. + Warns that the version is too old and returns None + * ignore: If the module is not installed, return None, otherwise, + return the module, even if the version is too old. + It's expected that users validate the version locally when + using ``errors="ignore"`` (see. ``io/html.py``) + min_version : str, default None + Specify a minimum version that is different from the global pandas + minimum version required. + Returns + ------- + maybe_module : Optional[ModuleType] + The imported module, when found and the version is correct. + None is returned when the package is not found and `errors` + is False, or when the package's version is too old and `errors` + is ``'warn'`` or ``'ignore'``. + """ + assert errors in {"warn", "raise", "ignore"} + + package_name = INSTALL_MAPPING.get(name) + install_name = package_name if package_name is not None else name + + msg = ( + f"Missing optional dependency '{install_name}'. {extra} " + f"Use pip or conda to install {install_name}." + ) + try: + module = importlib.import_module(name) + except ImportError as err: + if errors == "raise": + raise ImportError(msg) from err + return None + + # Handle submodules: if we have submodule, grab parent module from sys.modules + parent = name.split(".")[0] + if parent != name: + install_name = parent + module_to_get = sys.modules[install_name] + else: + module_to_get = module + minimum_version = min_version if min_version is not None else VERSIONS.get(parent) + if minimum_version: + version = get_version(module_to_get) + if version and Version(version) < Version(minimum_version): + msg = ( + f"Dask requires version '{minimum_version}' or newer of '{parent}' " + f"(version '{version}' currently installed)." + ) + if errors == "warn": + warnings.warn(msg, UserWarning) + return None + elif errors == "raise": + raise ImportError(msg) + else: + return None + + return module diff --git a/dask/array/numpy_compat.py b/dask/array/numpy_compat.py index 8e0b67484f9..03160cbdde2 100644 --- a/dask/array/numpy_compat.py +++ b/dask/array/numpy_compat.py @@ -2,6 +2,9 @@ import warnings +from dask._compatibility import import_optional_dependency + +import_optional_dependency("numpy") import numpy as np from packaging.version import Version diff --git a/dask/dataframe/_compat.py b/dask/dataframe/_compat.py index e163021effd..adf49c29af9 100644 --- a/dask/dataframe/_compat.py +++ b/dask/dataframe/_compat.py @@ -4,12 +4,16 @@ import string import warnings +from dask._compatibility import import_optional_dependency + +import_optional_dependency("pandas") +import_optional_dependency("numpy") + import numpy as np import pandas as pd from packaging.version import Version PANDAS_VERSION = Version(pd.__version__) -PANDAS_GE_200 = True PANDAS_GE_201 = PANDAS_VERSION.release >= (2, 0, 1) PANDAS_GE_202 = PANDAS_VERSION.release >= (2, 0, 2) PANDAS_GE_210 = PANDAS_VERSION.release >= (2, 1, 0) diff --git a/dask/dataframe/_pyarrow.py b/dask/dataframe/_pyarrow.py index cdf956bc065..630f3bbff20 100644 --- a/dask/dataframe/_pyarrow.py +++ b/dask/dataframe/_pyarrow.py @@ -5,10 +5,11 @@ import pandas as pd from packaging.version import Version +from dask._compatibility import import_optional_dependency from dask.dataframe.utils import is_dataframe_like, is_index_like, is_series_like try: - import pyarrow as pa + pa = import_optional_dependency("pyarrow") except ImportError: pa = None diff --git a/dask/diagnostics/profile_visualize.py b/dask/diagnostics/profile_visualize.py index ec9d1aca1f7..d4b8ce9f6b8 100644 --- a/dask/diagnostics/profile_visualize.py +++ b/dask/diagnostics/profile_visualize.py @@ -8,12 +8,13 @@ from tlz import accumulate, groupby, pluck, unique +from dask._compatibility import import_optional_dependency from dask.core import istask from dask.utils import apply, funcname, import_required def BOKEH_VERSION(): - import bokeh + bokeh = import_optional_dependency("bokeh") from packaging.version import Version return Version(bokeh.__version__) From fc232aaf60217e1b26680febcc044b3b155bf88f Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Tue, 2 Jul 2024 11:58:56 -0400 Subject: [PATCH 20/72] Fix quantile for arrow dtypes (#11202) --- dask/dataframe/backends.py | 2 ++ dask/dataframe/tests/test_arithmetics_reduction.py | 8 ++++++++ 2 files changed, 10 insertions(+) diff --git a/dask/dataframe/backends.py b/dask/dataframe/backends.py index 48fd4ea109c..b2b4b63d927 100644 --- a/dask/dataframe/backends.py +++ b/dask/dataframe/backends.py @@ -734,6 +734,8 @@ def get_grouper_pandas(obj): @percentile_lookup.register((pd.Series, pd.Index)) def percentile(a, q, interpolation="linear"): + if isinstance(a.dtype, pd.ArrowDtype): + a = a.to_numpy() return _percentile(a, q, interpolation) diff --git a/dask/dataframe/tests/test_arithmetics_reduction.py b/dask/dataframe/tests/test_arithmetics_reduction.py index e0f0195a7f0..e40382d508a 100644 --- a/dask/dataframe/tests/test_arithmetics_reduction.py +++ b/dask/dataframe/tests/test_arithmetics_reduction.py @@ -1900,3 +1900,11 @@ def test_reductions_with_pandas_and_arrow_ea(dtype, func): dd_result = factor * dd_result + offset # _meta is wrongly NA assert_eq(dd_result, pd_result, check_dtype=False) + + +def test_quantile_arrow_dtype(): + pdf = pd.DataFrame({"foo": [1, 1, 1, 1, 1, 1]}).convert_dtypes( + dtype_backend="pyarrow" + ) + df = dd.from_pandas(pdf, npartitions=2) + assert_eq(df.quantile(0.25), pdf.quantile(0.25), check_dtype=False) From ecaa4c79ad65e090ed0d604523e21bc1b1ccbeff Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Tue, 2 Jul 2024 15:14:35 -0400 Subject: [PATCH 21/72] Add some array-expr methods (#11210) --- dask/array/__init__.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dask/array/__init__.py b/dask/array/__init__.py index 6637e132c55..58cac45bbd7 100644 --- a/dask/array/__init__.py +++ b/dask/array/__init__.py @@ -321,19 +321,24 @@ def inner_func(*args, **kwargs): try: from dask_expr.array import ( # type: ignore Array, + arange, asarray, + empty, from_array, + linspace, mean, moment, nanmean, nanstd, nansum, nanvar, + ones, prod, random, std, sum, var, + zeros, ) backends = raise_not_implemented_error("backends") @@ -360,7 +365,6 @@ def inner_func(*args, **kwargs): to_npy_stack = raise_not_implemented_error("to_npy_stack") to_zarr = raise_not_implemented_error("to_zarr") unify_chunks = raise_not_implemented_error("unify_chunks") - arange = raise_not_implemented_error("arange") diag = raise_not_implemented_error("diag") diagonal = raise_not_implemented_error("diagonal") empty_like = raise_not_implemented_error("empty_like") @@ -368,7 +372,6 @@ def inner_func(*args, **kwargs): fromfunction = raise_not_implemented_error("fromfunction") full_like = raise_not_implemented_error("full_like") indices = raise_not_implemented_error("indices") - linspace = raise_not_implemented_error("linspace") meshgrid = raise_not_implemented_error("meshgrid") ones_like = raise_not_implemented_error("ones_like") pad = raise_not_implemented_error("pad") @@ -580,10 +583,7 @@ def inner_func(*args, **kwargs): tanh = raise_not_implemented_error("tanh") true_divide = raise_not_implemented_error("true_divide") trunc = raise_not_implemented_error("trunc") - empty = raise_not_implemented_error("empty") full = raise_not_implemented_error("full") - ones = raise_not_implemented_error("ones") - zeros = raise_not_implemented_error("zeros") from dask.array.utils import assert_eq from dask.base import compute From 7820aa2d64113800a396d18140d0ac898d0c77db Mon Sep 17 00:00:00 2001 From: Sarah Charlotte Johnson Date: Tue, 2 Jul 2024 23:14:47 -0400 Subject: [PATCH 22/72] Update Coiled links in documentation (#11211) Co-authored-by: James Bourbeau --- docs/source/dataframe.rst | 6 +++--- docs/source/deployment-considerations.rst | 2 +- docs/source/ml.rst | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/source/dataframe.rst b/docs/source/dataframe.rst index 96a03f0774d..8bf16634a95 100644 --- a/docs/source/dataframe.rst +++ b/docs/source/dataframe.rst @@ -178,9 +178,9 @@ Examples Dask DataFrame is used across a wide variety of applications — anywhere where working with large tabular dataset. Here are a few large-scale examples: -- `Parquet ETL with Dask DataFrame `_ -- `XGBoost model training with Dask DataFrame `_ -- `Visualize 1,000,000,000 points `_ +- `Parquet ETL with Dask DataFrame `_ +- `XGBoost model training with Dask DataFrame `_ +- `Visualize 1,000,000,000 points `_ These examples all process larger-than-memory datasets on Dask clusters deployed with `Coiled `_, diff --git a/docs/source/deployment-considerations.rst b/docs/source/deployment-considerations.rst index 131b072682a..659d93e6d3e 100644 --- a/docs/source/deployment-considerations.rst +++ b/docs/source/deployment-considerations.rst @@ -22,7 +22,7 @@ Environment management options without additional infrastructure: * :class:`~distributed.diagnostics.plugin.PipInstall` plugin * :class:`~distributed.diagnostics.plugin.UploadDirectory` plugin -* Coiled's `package sync `_ automatically replicates a local environment onto a cluster, including local packages and Git dependencies. +* Coiled's `package sync `_ automatically replicates a local environment onto a cluster, including local packages and Git dependencies. Logging diff --git a/docs/source/ml.rst b/docs/source/ml.rst index c32365e1529..11fdbf901f0 100644 --- a/docs/source/ml.rst +++ b/docs/source/ml.rst @@ -71,7 +71,7 @@ Then we run many optimize methods in parallel. print(study.best_params) -For a more fully worked example see this :bdg-link-primary:`Optuna + XGBoost example `. +For a more fully worked example see this :bdg-link-primary:`Optuna + XGBoost example `. Dask Futures @@ -141,7 +141,7 @@ For example, using Dask DataFrame, XGBoost, and a local Dask cluster looks like ) predictions = xgb.dask.predict(client, model, X_test) -For a more fully worked example see this :bdg-link-primary:`XGBoost example `. +For a more fully worked example see this :bdg-link-primary:`XGBoost example `. .. _batch-prediction: From b1882a0d60afbbd8610ebc8ed4f72a6291e7694f Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Wed, 3 Jul 2024 11:09:00 -0400 Subject: [PATCH 23/72] Fix botocore re-raising error (#11209) --- dask/backends.py | 15 ++++++++++----- dask/dataframe/io/tests/test_parquet.py | 22 ++++++++++++++++++++++ 2 files changed, 32 insertions(+), 5 deletions(-) diff --git a/dask/backends.py b/dask/backends.py index 276429be799..4164bfc65b8 100644 --- a/dask/backends.py +++ b/dask/backends.py @@ -139,11 +139,16 @@ def wrapper(*args, **kwargs): try: return func(*args, **kwargs) except Exception as e: - raise type(e)( - f"An error occurred while calling the {funcname(func)} " - f"method registered to the {self.backend} backend.\n" - f"Original Message: {e}" - ) from e + try: + exc = type(e)( + f"An error occurred while calling the {funcname(func)} " + f"method registered to the {self.backend} backend.\n" + f"Original Message: {e}" + ) + except TypeError: + raise e + else: + raise exc from e wrapper.__name__ = dispatch_name return wrapper diff --git a/dask/dataframe/io/tests/test_parquet.py b/dask/dataframe/io/tests/test_parquet.py index 46b976c3a12..a79dd309028 100644 --- a/dask/dataframe/io/tests/test_parquet.py +++ b/dask/dataframe/io/tests/test_parquet.py @@ -4920,3 +4920,25 @@ def test_parquet_string_roundtrip(tmpdir): df = dd.read_parquet(tmpdir + "string.parquet") assert_eq(df, pdf) pd.testing.assert_frame_equal(df.compute(), pdf) + + +def test_parquet_botocore_exception(tmpdir): + pytest.importorskip("botocore") + + from unittest.mock import patch + + from botocore.exceptions import BotoCoreError + + pdf = pd.DataFrame({"a": ["a", "b", "c"]}, dtype="string[pyarrow]") + pdf.to_parquet(tmpdir + "string.parquet") + + def mock_get_engine(engine): + raise BotoCoreError + + with pytest.raises(BotoCoreError, match="An unspecified error occurred"): + if dd._dask_expr_enabled(): + with patch("dask_expr.io.parquet.get_engine", mock_get_engine): + dd.read_parquet(tmpdir + "string.parquet") + else: + with patch("dask.dataframe.io.parquet.core.get_engine", mock_get_engine): + dd.read_parquet(tmpdir + "string.parquet") From 8a39abe19b674dcf4f2b453ed6ede9f8549d655a Mon Sep 17 00:00:00 2001 From: Florian Jetter Date: Fri, 5 Jul 2024 21:03:10 +0200 Subject: [PATCH 24/72] Only count data that is in memory for ``xarray`` ``sizeof`` (#11206) --- dask/sizeof.py | 12 ++++++++++-- dask/tests/test_sizeof.py | 31 ++++++++++++++++++++++++++++++- 2 files changed, 40 insertions(+), 3 deletions(-) diff --git a/dask/sizeof.py b/dask/sizeof.py index 7c9945ef1df..6d9dccf51f8 100644 --- a/dask/sizeof.py +++ b/dask/sizeof.py @@ -267,10 +267,18 @@ def register_xarray(): XARRAY_VERSION = Version(xr.__version__) XARRAY_GE_2024_02 = XARRAY_VERSION >= Version("2024.02.0") + @sizeof.register(xr.core.utils.Frozen) + def xarray_sizeof_frozen(obj): + return sys.getsizeof(obj) + sizeof(obj.mapping) + @sizeof.register(xr.DataArray) - @sizeof.register(xr.Dataset) + @sizeof.register(xr.Variable) def xarray_sizeof_da(obj): - return obj.nbytes + return sys.getsizeof(obj) + sizeof(obj.data) + + @sizeof.register(xr.Dataset) + def xarray_sizeof_ds(obj): + return sys.getsizeof(obj) + sizeof(obj.variables) if XARRAY_GE_2024_02: xarray_sizeof_da = sizeof.register(xr.NamedArray)(xarray_sizeof_da) diff --git a/dask/tests/test_sizeof.py b/dask/tests/test_sizeof.py index 8f9250fe160..dd99d27d5db 100644 --- a/dask/tests/test_sizeof.py +++ b/dask/tests/test_sizeof.py @@ -8,7 +8,7 @@ from dask.multiprocessing import get_context from dask.sizeof import sizeof -from dask.utils import funcname +from dask.utils import funcname, tmpdir try: import pandas as pd @@ -272,3 +272,32 @@ def test_xarray(): assert sizeof(dataset.foo) >= sizeof(arr) assert sizeof(dataset["coord"]) >= sizeof(ind) assert sizeof(dataset.indexes) >= sizeof(ind) + + +def test_xarray_not_in_memory(): + xr = pytest.importorskip("xarray") + np = pytest.importorskip("numpy") + pytest.importorskip("zarr") + + ind = np.arange(-66, 67, 1).astype(float) + arr = np.random.random((len(ind),)) + + with tmpdir() as path: + xr.DataArray( + arr, + dims=["coord"], + coords={"coord": ind}, + ).rename( + "foo" + ).to_dataset().to_zarr(path) + dataset = xr.open_zarr(path, chunks={"foo": 10}) + assert not dataset.foo._in_memory + assert sizeof(ind) < sizeof(dataset) < sizeof(arr) + sizeof(ind) + assert sizeof(dataset.foo) < sizeof(arr) + assert sizeof(dataset["coord"]) >= sizeof(ind) + assert sizeof(dataset.indexes) >= sizeof(ind) + assert not dataset.foo._in_memory + + dataset.load() + assert dataset.foo._in_memory + assert sizeof(dataset) >= sizeof(arr) + sizeof(ind) From 1969a38112ec9f87e056a278478f072ad22e972e Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Fri, 5 Jul 2024 14:18:14 -0500 Subject: [PATCH 25/72] bump version to 2024.7.0 --- docs/source/changelog.rst | 66 ++++++++++++++++++++++++++++++++++----- pyproject.toml | 2 +- 2 files changed, 59 insertions(+), 9 deletions(-) diff --git a/docs/source/changelog.rst b/docs/source/changelog.rst index e7b69ad9996..03bd9e2c752 100644 --- a/docs/source/changelog.rst +++ b/docs/source/changelog.rst @@ -1,23 +1,73 @@ Changelog ========= -.. _v2024.6.3: +.. _v2024.7.0: -2024.6.3 +2024.7.0 -------- - Highlights ^^^^^^^^^^ -Drop pandas 1.x support -""""""""""""""""""""""" +Drop support for pandas 1.x +""""""""""""""""""""""""""" -This release drops support for pandas versions of the 1.x series. pandas 2.0 +This release drops support for ``pandas<2``. ``pandas`` 2.0 is now the required minimum version to run Dask DataFrame. -The mimimum version of partd was also raised to 1.4.0. The versions before 1.4 -are not compatible with pandas 2. +The mimimum version of ``partd`` was also raised to 1.4.0. Versions before 1.4 +are not compatible with ``pandas`` 2. + +See :pr:`11199` by `Patrick Hoefler`_ for more details. + +Publish-subscribe APIs deprecated +""""""""""""""""""""""""""""""""" + +:py:class:`distributed.Pub` and :py:class:`distributed.Sub` have been deprecated and will be removed +in a future release. Please switch to :py:func:`distributed.Client.log_event` and :py:func:`distributed.Worker.log_event` +instead. + +See :pr-distributed:`8724` by `Hendrik Makait`_ for more details. + +.. dropdown:: Additional changes + + - Only count data that is in memory for ``xarray`` ``sizeof`` (:pr:`11206`) `Florian Jetter`_ + - Fix ``botocore`` re-raising error (:pr:`11209`) `Patrick Hoefler`_ + - Update Coiled links in documentation (:pr:`11211`) `Sarah Charlotte Johnson`_ + - Add some array-expr methods (:pr:`11210`) `Patrick Hoefler`_ + - Fix ``quantile`` for arrow dtypes (:pr:`11202`) `Patrick Hoefler`_ + - Add utility to verify optional dependencies (:pr:`11205`) `Patrick Hoefler`_ + - Implement array expression switch (:pr:`11203`) `Patrick Hoefler`_ + - Remove no longer supported ``ipython`` reference (:pr:`11196`) `Patrick Hoefler`_ + - Remove ``from_delayed`` references (:pr:`11195`) `Patrick Hoefler`_ + - Add other IO connectors to docs (:pr:`11189`) `Patrick Hoefler`_ + + - Fix ``assert_eq`` import from ``cudf`` (:pr-distributed:`8747`) `James Bourbeau`_ + - Log traceback upon task error (:pr-distributed:`8746`) `Hendrik Makait`_ + - Update system monitor when polling Prometheus metrics (:pr-distributed:`8745`) `Hendrik Makait`_ + - Bump ``pandas`` to 2.0 in ``mindeps`` build (:pr-distributed:`8743`) `James Bourbeau`_ + - Refactor event logging functionality into broker (:pr-distributed:`8731`) `Hendrik Makait`_ + - Drop support for pandas 1.X (:pr-distributed:`8741`) `Hendrik Makait`_ + - Remove ``is_python_shutting_down`` (:pr-distributed:`8492`) `Hendrik Makait`_ + - Fix ``test_task_state_instance_are_garbage_collected`` (:pr-distributed:`8735`) `Hendrik Makait`_ + - Fix floating-point inaccuracy (:pr-distributed:`8736`) `Hendrik Makait`_ + - Fix ``pynvml`` handles (:pr-distributed:`8693`) `Benjamin Zaitlen`_ + - ``get_ip``: handle getting ``0.0.0.0`` (:pr-distributed:`8712`) `Adam Williamson`_ + - Remove ``FutureWarning`` in ``test_task_state_instance_are_garbage_collected`` (:pr-distributed:`8734`) `Hendrik Makait`_ + - Fix ``mindeps``-testing on CI (:pr-distributed:`8728`) `Hendrik Makait`_ + - Extract tests related to event-logging into separate file (:pr-distributed:`8733`) `Hendrik Makait`_ + - Use safer context for ``ProcessPoolExecutor`` (:pr-distributed:`8715`) `Elliott Sales de Andrade`_ + - Cache URL encoding of worker addresses in dashboard (:pr-distributed:`8725`) `Florian Jetter`_ + - More robust ``bokeh`` ``test_shuffling`` (:pr-distributed:`8727`) `Florian Jetter`_ + - Fix type in actor docs (:pr-distributed:`8711`) `Sultan Orazbayev`_ + - More useful warning if a plugin type is provided instead of instance (:pr-distributed:`8689`) `Florian Jetter`_ + - Improve error on cancelled tasks due to disconnect (:pr-distributed:`8705`) `Hendrik Makait`_ + - Fix wait condition on ``test_forget_errors`` (:pr-distributed:`8714`) `Elliott Sales de Andrade`_ + - Skip ``test_deadlock_dependency_of_queued_released`` (:pr-distributed:`8723`) `Hendrik Makait`_ + - Fix ``test_quiet_client_close`` (:pr-distributed:`8722`) `Hendrik Makait`_ + - Fix cleanup iteration in ``save_sys_modules`` (:pr-distributed:`8713`) `Elliott Sales de Andrade`_ + - Add quotes to missing ``bokeh`` installation commands (:pr-distributed:`8717`) `James Bourbeau`_ + .. _v2024.6.2: diff --git a/pyproject.toml b/pyproject.toml index e8547661d0f..065a21ebce4 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -55,7 +55,7 @@ dataframe = [ "pandas >= 2.0", "dask-expr >= 1.1, <1.2", # dask-expr pins the dask version ] -distributed = ["distributed == 2024.6.2"] +distributed = ["distributed == 2024.7.0"] diagnostics = [ "bokeh >= 2.4.2", "jinja2 >= 2.10.3", From 90c176dc6aca8cfade4b79e5e9b636c04b3f09e9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Jul 2024 21:52:32 -0400 Subject: [PATCH 26/72] Bump JamesIves/github-pages-deploy-action from 4.6.1 to 4.6.3 (#11222) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/test-report.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/test-report.yaml b/.github/workflows/test-report.yaml index 9b252929db4..1fb49605b52 100644 --- a/.github/workflows/test-report.yaml +++ b/.github/workflows/test-report.yaml @@ -61,7 +61,7 @@ jobs: mv test_report.html test_short_report.html deploy/ - name: Deploy 🚀 - uses: JamesIves/github-pages-deploy-action@v4.6.1 + uses: JamesIves/github-pages-deploy-action@v4.6.3 with: branch: gh-pages folder: deploy From b4b33caed8fc9cf77c9332442ab11cf00f90bb42 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Jul 2024 21:52:40 -0400 Subject: [PATCH 27/72] Bump xarray-contrib/issue-from-pytest-log from 1.2.8 to 1.3.0 (#11221) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/upstream.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/upstream.yml b/.github/workflows/upstream.yml index 5aecf51d60f..a3f60837c5e 100644 --- a/.github/workflows/upstream.yml +++ b/.github/workflows/upstream.yml @@ -80,7 +80,7 @@ jobs: && github.event_name != 'pull_request' && github.repository == 'dask/dask' && steps.run_tests.outcome == 'failure' - uses: xarray-contrib/issue-from-pytest-log@v1.2.8 + uses: xarray-contrib/issue-from-pytest-log@v1.3.0 with: log-path: output-log.jsonl issue-title: ⚠️ Upstream CI failed ⚠️ From 894a723f99ad215ced81cb04b64abb16772728f4 Mon Sep 17 00:00:00 2001 From: Florian Jetter Date: Fri, 12 Jul 2024 13:40:20 +0200 Subject: [PATCH 28/72] bump approx threshold for test_quantile (#10720) --- dask/dataframe/tests/test_dataframe.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dask/dataframe/tests/test_dataframe.py b/dask/dataframe/tests/test_dataframe.py index b2d0b2b418a..a9a77bdf1fe 100644 --- a/dask/dataframe/tests/test_dataframe.py +++ b/dask/dataframe/tests/test_dataframe.py @@ -1519,7 +1519,7 @@ def test_quantile(method, quantile): assert isinstance(result, dd.Series) result = result.compute() assert isinstance(result, pd.Series) - assert result.iloc[0] == pytest.approx(exp, rel=0.1) + assert result.iloc[0] == pytest.approx(exp, rel=0.15) # series / single result = df.x.quantile(quantile, method=method) @@ -1528,7 +1528,7 @@ def test_quantile(method, quantile): else: assert isinstance(result, dd.core.Scalar) result = result.compute() - assert result == pytest.approx(exp, rel=0.1) + assert result == pytest.approx(exp, rel=0.15) @pytest.mark.parametrize( From 6a04b4ed70934b548d25a793eed90f033a690531 Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Tue, 16 Jul 2024 11:00:19 -0500 Subject: [PATCH 29/72] Fixes for ``d`` freq deprecation in ``pandas=3`` (#11228) --- dask/dataframe/io/tests/test_parquet.py | 2 +- dask/dataframe/tests/test_utils_dataframe.py | 6 ++--- dask/dataframe/tseries/tests/test_resample.py | 26 ++++++++++++++++--- dask/datasets.py | 2 +- 4 files changed, 28 insertions(+), 8 deletions(-) diff --git a/dask/dataframe/io/tests/test_parquet.py b/dask/dataframe/io/tests/test_parquet.py index a79dd309028..51b1dc5a5ad 100644 --- a/dask/dataframe/io/tests/test_parquet.py +++ b/dask/dataframe/io/tests/test_parquet.py @@ -2397,7 +2397,7 @@ def test_roundtrip_arrow(tmpdir, df): def test_datasets_timeseries(tmpdir, engine): tmp_path = str(tmpdir) df = dask.datasets.timeseries( - start="2000-01-01", end="2000-01-10", freq="1d" + start="2000-01-01", end="2000-01-10", freq="1D" ).persist() df.to_parquet(tmp_path, engine=engine) diff --git a/dask/dataframe/tests/test_utils_dataframe.py b/dask/dataframe/tests/test_utils_dataframe.py index 700638b5741..82942d4ac1a 100644 --- a/dask/dataframe/tests/test_utils_dataframe.py +++ b/dask/dataframe/tests/test_utils_dataframe.py @@ -281,20 +281,20 @@ def test_meta_nonempty_index(): assert type(res) is pd.Index assert res.name == idx.name - idx = pd.DatetimeIndex(["1970-01-01"], freq="d", tz="America/New_York", name="foo") + idx = pd.DatetimeIndex(["1970-01-01"], freq="D", tz="America/New_York", name="foo") res = meta_nonempty(idx) assert type(res) is pd.DatetimeIndex assert res.tz == idx.tz assert res.freq == idx.freq assert res.name == idx.name - idx = pd.PeriodIndex(["1970-01-01"], freq="d", name="foo") + idx = pd.PeriodIndex(["1970-01-01"], freq="D", name="foo") res = meta_nonempty(idx) assert type(res) is pd.PeriodIndex assert res.freq == idx.freq assert res.name == idx.name - idx = pd.TimedeltaIndex([pd.Timedelta(1, "D")], freq="d", name="foo") + idx = pd.TimedeltaIndex([pd.Timedelta(1, "D")], freq="D", name="foo") res = meta_nonempty(idx) assert type(res) is pd.TimedeltaIndex assert res.freq == idx.freq diff --git a/dask/dataframe/tseries/tests/test_resample.py b/dask/dataframe/tseries/tests/test_resample.py index b9a22a83690..4839cb781ca 100644 --- a/dask/dataframe/tseries/tests/test_resample.py +++ b/dask/dataframe/tseries/tests/test_resample.py @@ -1,12 +1,13 @@ from __future__ import annotations +import contextlib from itertools import product import pandas as pd import pytest import dask.dataframe as dd -from dask.dataframe._compat import PANDAS_GE_220 +from dask.dataframe._compat import PANDAS_GE_220, PANDAS_GE_300 from dask.dataframe.utils import assert_eq @@ -212,7 +213,26 @@ def test_common_aggs(agg): f = lambda df: getattr(df, agg)() - res = f(ps.resample("1d")) - expected = f(ds.resample("1d")) + res = f(ps.resample("1D")) + expected = f(ds.resample("1D")) assert_eq(res, expected, check_dtype=False) + + +def test_rule_deprecated(): + index = pd.date_range("2000-01-01", "2000-02-15", freq="h") + s = pd.Series(range(len(index)), index=index) + ds = dd.from_pandas(s, npartitions=2) + + if PANDAS_GE_300: + ctx = pytest.warns(FutureWarning, match="'d' is deprecated") + else: + ctx = contextlib.nullcontext() + + with ctx: + res = s.resample("1d").count() + with ctx: + expected = ds.resample("1d").count() + + with ctx: + assert_eq(res, expected, check_dtype=False) diff --git a/dask/datasets.py b/dask/datasets.py index c2684515bda..b18e742ac94 100644 --- a/dask/datasets.py +++ b/dask/datasets.py @@ -11,7 +11,7 @@ def timeseries( start="2000-01-01", end="2000-01-31", freq="1s", - partition_freq="1d", + partition_freq="1D", dtypes=None, seed=None, **kwargs, From ca262bc2a2945114468a5b79272322a0ff71e6ec Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Wed, 17 Jul 2024 14:29:48 +0200 Subject: [PATCH 30/72] Ensure that dask expr DataFrames are optimized when put into delayed (#11231) --- dask/dataframe/tests/test_dataframe.py | 20 ++++++++++++++++++++ dask/delayed.py | 4 ++++ 2 files changed, 24 insertions(+) diff --git a/dask/dataframe/tests/test_dataframe.py b/dask/dataframe/tests/test_dataframe.py index a9a77bdf1fe..fb56de2d1b6 100644 --- a/dask/dataframe/tests/test_dataframe.py +++ b/dask/dataframe/tests/test_dataframe.py @@ -6316,3 +6316,23 @@ def test_query_planning_config_warns(): expect = "enabled" if dd.DASK_EXPR_ENABLED else "disabled" with pytest.warns(match=f"query planning is already {expect}"): dd._dask_expr_enabled() + + +def test_dataframe_into_delayed(): + if not DASK_EXPR_ENABLED: + pytest.skip("Only relevant for dask.expr") + + pdf = pd.DataFrame({"a": [1, 2, 3], "b": 1}) + df = dd.from_pandas(pdf, npartitions=2) + + def test_func(df): + return df.sum().sum() + + def delayed_func(i): + # sanity check + assert i.sum() == 6 + + df = df[["a"]].map_partitions(test_func, meta=(None, int)) + result = delayed(delayed_func)(df) + assert sum(map(len, result.dask.layers.values())) == 6 + result.compute() diff --git a/dask/delayed.py b/dask/delayed.py index 350e89fbfa8..1ff6a2dba56 100644 --- a/dask/delayed.py +++ b/dask/delayed.py @@ -95,6 +95,10 @@ def unpack_collections(expr): return expr._key, (expr,) if is_dask_collection(expr): + if hasattr(expr, "optimize"): + # Optimize dask-expr collections + expr = expr.optimize() + finalized = finalize(expr) return finalized._key, (finalized,) From 27ce2f8ff16f797aa41f64f2080d57bfbac9e677 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Wed, 17 Jul 2024 18:25:46 +0200 Subject: [PATCH 31/72] Preserve timestamp unit during meta creation (#11233) Co-authored-by: James Bourbeau --- dask/dataframe/backends.py | 2 +- dask/dataframe/tests/test_dataframe.py | 15 +++++++++++++++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/dask/dataframe/backends.py b/dask/dataframe/backends.py index b2b4b63d927..afdf46522ed 100644 --- a/dask/dataframe/backends.py +++ b/dask/dataframe/backends.py @@ -436,7 +436,7 @@ def _nonempty_series(s, idx=None): data = [s.iloc[0]] * 2 elif isinstance(dtype, pd.DatetimeTZDtype): entry = pd.Timestamp("1970-01-01", tz=dtype.tz) - data = [entry, entry] + data = pd.array([entry, entry], dtype=dtype) elif isinstance(dtype, pd.CategoricalDtype): if len(s.cat.categories): data = [s.cat.categories[0]] * 2 diff --git a/dask/dataframe/tests/test_dataframe.py b/dask/dataframe/tests/test_dataframe.py index fb56de2d1b6..d1fb8524ef2 100644 --- a/dask/dataframe/tests/test_dataframe.py +++ b/dask/dataframe/tests/test_dataframe.py @@ -36,6 +36,7 @@ repartition_divisions, total_mem_usage, ) +from dask.dataframe.dispatch import meta_nonempty from dask.dataframe.utils import ( assert_eq, assert_eq_dtypes, @@ -6308,6 +6309,20 @@ def test_enforce_runtime_divisions(): ddf.enforce_runtime_divisions().compute() +def test_preserve_ts_unit_in_meta_creation(): + pdf = pd.DataFrame( + { + "a": [1], + "timestamp": pd.Series( + [pd.Timestamp.utcnow()], dtype="datetime64[us, UTC]" + ), + } + ) + df = dd.from_pandas(pdf, npartitions=1) + assert_eq(meta_nonempty(df._meta).dtypes, pdf.dtypes) + assert_eq(df, pdf) + + def test_query_planning_config_warns(): # Make sure dd._dask_expr_enabled() warns if the current # "dataframe.query-planning" config conflicts with the From c9f3e39afab6761bc57cfb5c9ed031791212d442 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Fri, 19 Jul 2024 15:45:17 +0200 Subject: [PATCH 32/72] Remove and warn of persist usage (#11237) --- docs/source/best-practices.rst | 24 ------------------------ docs/source/user-interfaces.rst | 7 +++++++ 2 files changed, 7 insertions(+), 24 deletions(-) diff --git a/docs/source/best-practices.rst b/docs/source/best-practices.rst index e1e6ac03b06..37e4d6c5e64 100644 --- a/docs/source/best-practices.rst +++ b/docs/source/best-practices.rst @@ -197,30 +197,6 @@ using normal Python again. df = df.compute() # continue on with pandas/NumPy -Persist When You Can --------------------- - -Accessing data from RAM is often much faster than accessing it from disk. -Once you have your dataset in a clean state that both: - -1. Fits in memory -2. Is clean enough that you will want to try many different analyses - -Then it is a good time to *persist* your data in RAM - -.. code-block:: python - - df = dd.read_parquet("lots-of-data-*.parquet") - df = df.fillna(...) # clean up things lazily - df = df[df.name == 'Alice'] # get down to a more reasonable size - - df = df.persist() # trigger computation, persist in distributed RAM - -.. note:: This is only relevant if you are on a distributed machine. On a local - machine (using single-machine schedulers) ``persist`` just triggers immediate - computation like ``compute``. - - Store Data Efficiently ---------------------- diff --git a/docs/source/user-interfaces.rst b/docs/source/user-interfaces.rst index a85beca6d1f..75ce615b124 100644 --- a/docs/source/user-interfaces.rst +++ b/docs/source/user-interfaces.rst @@ -171,6 +171,13 @@ consider writing it to disk instead. Persist into Distributed Memory ------------------------------- +.. warning:: + + persist will store the full dataset in memory. This has the disadvantage that + the available memory must actually exceed the size of the dataset. Use persist + only when interactively iterating on the same dataset over and over again and + avoid it in productive use-cases as much as possible. + Alternatively, if you are on a cluster, then you may want to trigger a computation and store the results in distributed memory. In this case you do not want to call ``compute``, which would create a single Pandas, NumPy, or From 0a40ba288913ea780bbaf37ab8a04aedf8317659 Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Fri, 19 Jul 2024 19:45:15 -0500 Subject: [PATCH 33/72] bump version to 2024.7.1 --- docs/source/changelog.rst | 49 +++++++++++++++++++++++++++++++++++++++ pyproject.toml | 2 +- 2 files changed, 50 insertions(+), 1 deletion(-) diff --git a/docs/source/changelog.rst b/docs/source/changelog.rst index 03bd9e2c752..6d170fb4380 100644 --- a/docs/source/changelog.rst +++ b/docs/source/changelog.rst @@ -1,6 +1,54 @@ Changelog ========= +.. _v2024.7.1: + +2024.7.1 +-------- + +Highlights +^^^^^^^^^^ + +More resilient distributed lock +""""""""""""""""""""""""""""""" + +:py:class:`distributed.Lock` is now resilient to worker failures. +Previously deadlocks were possible in cases where a lock-holding worker +was lost and/or failed to release the lock due to an error. + +See :pr-distributed:`8770` by `Florian Jetter`_ for more details. + +.. dropdown:: Additional changes + + - Remove and warn of persist usage (:pr:`11237`) `Patrick Hoefler`_ + - Preserve ``timestamp`` unit during ``meta`` creation (:pr:`11233`) `Patrick Hoefler`_ + - Ensure that ``dask-expr`` ``DataFrames`` are optimized when put into ``delayed`` (:pr:`11231`) `Patrick Hoefler`_ + - Fixes for ``d`` freq deprecation in ``pandas=3`` (:pr:`11228`) `James Bourbeau`_ + - bump approx threshold for ``test_quantile`` (:pr:`10720`) `Florian Jetter`_ + - Bump ``xarray-contrib/issue-from-pytest-log`` from 1.2.8 to 1.3.0 (:pr:`11221`) + - Bump ``JamesIves/github-pages-deploy-action`` from 4.6.1 to 4.6.3 (:pr:`11222`) + + - Ensure ``Lock`` always register with scheduler (:pr-distributed:`8781`) `Florian Jetter`_ + - Temporarily pin ``setuptools < 71`` (:pr-distributed:`8785`) `James Bourbeau`_ + - Restore ``len()`` on ``TaskPrefix`` (:pr-distributed:`8783`) `Hendrik Makait`_ + - Avoid false positives for ``p2p-failed`` log event (:pr-distributed:`8777`) `Hendrik Makait`_ + - Expose paused and retired workers separately in prometheus (:pr-distributed:`8613`) `Patrick Hoefler`_ + - Creating transitions-failures log event (:pr-distributed:`8776`) `alex-rakowski`_ + - Implement HLG layer for P2P rechunking (:pr-distributed:`8751`) `Hendrik Makait`_ + - Add another test for a possible deadlock scenario caused by (:pr-distributed:`8703`) (:pr-distributed:`8769`) `Hendrik Makait`_ + - Raise an error if compute on persisted collection with released futures (:pr-distributed:`8764`) `Florian Jetter`_ + - Re-raise ``P2PConsistencyError`` from failed P2P tasks (:pr-distributed:`8748`) `Hendrik Makait`_ + - Robuster faster tests memory sampler (:pr-distributed:`8758`) `Florian Jetter`_ + - Fix ``scheduler_bokeh::test_shuffling`` (:pr-distributed:`8766`) `Florian Jetter`_ + - Increase timeouts for ``pubsub::test_client_worker`` (:pr-distributed:`8765`) `Florian Jetter`_ + - Factor out async taskgroup (:pr-distributed:`8756`) `Florian Jetter`_ + - Don't sort keys lexicographically in worker table (:pr-distributed:`8753`) `Florian Jetter`_ + - Use ``functools.cache`` instead of ``functools.lru_cache`` for extremely often called functions (:pr-distributed:`8762`) `Jonas Dedden`_ + - Robuster deeply nested structures (:pr-distributed:`8730`) `Florian Jetter`_ + - Adding HLG to MAP (:pr-distributed:`8740`) `alex-rakowski`_ + - Add close worker button to worker info page (:pr-distributed:`8742`) `James Bourbeau`_ + + .. _v2024.7.0: 2024.7.0 @@ -8369,3 +8417,4 @@ Other .. _`Victor Stinner`: https://github.com/vstinner .. _`alex-rakowski`: https://github.com/alex-rakowski .. _`Adam Williamson`: https://github.com/AdamWill +.. _`Jonas Dedden`: https://github.com/jonded94 diff --git a/pyproject.toml b/pyproject.toml index 065a21ebce4..4779ba14536 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -55,7 +55,7 @@ dataframe = [ "pandas >= 2.0", "dask-expr >= 1.1, <1.2", # dask-expr pins the dask version ] -distributed = ["distributed == 2024.7.0"] +distributed = ["distributed == 2024.7.1"] diagnostics = [ "bokeh >= 2.4.2", "jinja2 >= 2.10.3", From 7a69a853b6dba2cee7ac63beffd0c50493e95646 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Wed, 24 Jul 2024 13:49:20 +0200 Subject: [PATCH 34/72] Fix pandas nightly bugs (#11244) --- dask/dataframe/core.py | 16 ++++++++++--- dask/dataframe/indexing.py | 9 ++++++-- dask/dataframe/io/tests/test_csv.py | 16 ++++++++++++- dask/dataframe/io/tests/test_parquet.py | 5 ++-- dask/dataframe/io/tests/test_sql.py | 8 +++++-- dask/dataframe/methods.py | 23 +++++++++++-------- .../tests/test_arithmetics_reduction.py | 8 ++----- dask/dataframe/tests/test_dataframe.py | 2 +- dask/dataframe/tests/test_indexing.py | 12 ++++++++-- 9 files changed, 71 insertions(+), 28 deletions(-) diff --git a/dask/dataframe/core.py b/dask/dataframe/core.py index 1ba9d0eca6b..2c2b0c9000c 100644 --- a/dask/dataframe/core.py +++ b/dask/dataframe/core.py @@ -8596,14 +8596,24 @@ def _sqrt_and_convert_to_timedelta(partition, axis, dtype=None, *args, **kwargs) sqrt = np.sqrt(partition) if not is_df_like: - return pd.to_timedelta(sqrt) + result = pd.to_timedelta(sqrt, unit=kwargs.get("unit", None)) + if kwargs.get("unit", None) is not None: + result = result.as_unit(kwargs["unit"]) + return result time_col_mask = sqrt.index.isin(time_cols) matching_vals = sqrt[time_col_mask] if len(time_cols) > 0: sqrt = sqrt.astype(object) - for time_col, matching_val in zip(time_cols, matching_vals): - sqrt[time_col] = pd.to_timedelta(matching_val) + + units = kwargs.get("units", None) + if units is None: + units = [None] * len(time_cols) + for time_col, matching_val, unit in zip(time_cols, matching_vals, units): + result = pd.to_timedelta(matching_val, unit=kwargs.get("unit", None)) + if kwargs.get("unit", None) is not None: + result = result.as_unit(unit) + sqrt[time_col] = result if dtype is not None: sqrt = sqrt.astype(dtype) diff --git a/dask/dataframe/indexing.py b/dask/dataframe/indexing.py index 4c94425755c..76362b0d239 100644 --- a/dask/dataframe/indexing.py +++ b/dask/dataframe/indexing.py @@ -11,7 +11,7 @@ from dask.array.core import Array from dask.base import tokenize from dask.dataframe import methods -from dask.dataframe._compat import IndexingError +from dask.dataframe._compat import PANDAS_GE_300, IndexingError from dask.dataframe.core import Series, new_dd_object from dask.dataframe.utils import is_index_like, is_series_like, meta_nonempty from dask.highlevelgraph import HighLevelGraph @@ -363,13 +363,14 @@ def _coerce_loc_index(divisions, o): return o -def _maybe_partial_time_string(index, indexer): +def _maybe_partial_time_string(index, indexer, unit="ns"): """ Convert indexer for partial string selection if data has DatetimeIndex/PeriodIndex """ # do not pass dd.Index assert is_index_like(index) + unit = unit or "ns" if not isinstance(index, (pd.DatetimeIndex, pd.PeriodIndex)): return indexer @@ -384,11 +385,15 @@ def _maybe_partial_time_string(index, indexer): stop = index._maybe_cast_slice_bound(indexer.stop, "right") else: stop = indexer.stop + if PANDAS_GE_300: + start, stop = start.as_unit(unit), stop.as_unit(unit) return slice(start, stop) elif isinstance(indexer, str): start = index._maybe_cast_slice_bound(indexer, "left") stop = index._maybe_cast_slice_bound(indexer, "right") + if PANDAS_GE_300: + start, stop = start.as_unit(unit), stop.as_unit(unit) return slice(min(start, stop), max(start, stop)) return indexer diff --git a/dask/dataframe/io/tests/test_csv.py b/dask/dataframe/io/tests/test_csv.py index 193d3122628..2f7d6c53d5a 100644 --- a/dask/dataframe/io/tests/test_csv.py +++ b/dask/dataframe/io/tests/test_csv.py @@ -978,7 +978,10 @@ def test_late_dtypes(): ) with filetext(text) as fn: - sol = pd.read_csv(fn) + if PANDAS_GE_300: + sol = pd.read_csv(fn, parse_dates=["dates"]) + else: + sol = pd.read_csv(fn) msg = ( "Mismatched dtypes found in `pd.read_csv`/`pd.read_table`.\n" "\n" @@ -1037,6 +1040,17 @@ def test_late_dtypes(): dd.read_csv(fn, sample=50, dtype={"names": "O"}).compute(scheduler="sync") assert str(e.value) == msg + if PANDAS_GE_300: + # Specifying dtypes works + res = dd.read_csv( + fn, + sample=50, + dtype={"more_numbers": float, "names": object, "numbers": float}, + parse_dates=["dates"], + ) + assert_eq(res, sol) + return + with pytest.raises(ValueError) as e: dd.read_csv( fn, sample=50, parse_dates=["dates"], dtype={"names": "O"} diff --git a/dask/dataframe/io/tests/test_parquet.py b/dask/dataframe/io/tests/test_parquet.py index 51b1dc5a5ad..9f59cef3dbb 100644 --- a/dask/dataframe/io/tests/test_parquet.py +++ b/dask/dataframe/io/tests/test_parquet.py @@ -20,7 +20,7 @@ import dask.multiprocessing from dask.array.numpy_compat import NUMPY_GE_124 from dask.blockwise import Blockwise, optimize_blockwise -from dask.dataframe._compat import PANDAS_GE_202 +from dask.dataframe._compat import PANDAS_GE_202, PANDAS_GE_300 from dask.dataframe.io.parquet.core import get_engine from dask.dataframe.io.parquet.utils import _parse_pandas_metadata from dask.dataframe.optimize import optimize_dataframe_getitem @@ -1384,7 +1384,8 @@ def test_pyarrow_schema_inference(tmpdir, index, schema): "2017-01-02", "2017-01-06", "2017-01-09", - ] + ], + unit=None if not PANDAS_GE_300 else "ms", ), "amount": [100, 200, 300, 400, 500, 600, 700], }, diff --git a/dask/dataframe/io/tests/test_sql.py b/dask/dataframe/io/tests/test_sql.py index 705923870eb..a68ec58ea42 100644 --- a/dask/dataframe/io/tests/test_sql.py +++ b/dask/dataframe/io/tests/test_sql.py @@ -212,7 +212,7 @@ def test_needs_rational(db): part = data.get_partition(1).compute() assert part.dtypes.tolist() == [string_dtype, string_dtype] df2 = df.set_index("b") - assert_eq(data, df2) + assert_eq(data, df2, check_dtype=False) def test_simple(db): @@ -360,7 +360,11 @@ def test_datetimes(): assert data.index.dtype.kind == "M" assert data.divisions[0] == df.b.min() df2 = df.set_index("b") - assert_eq(data.map_partitions(lambda x: x.sort_index()), df2.sort_index()) + assert_eq( + data.map_partitions(lambda x: x.sort_index()), + df2.sort_index(), + check_dtype=False, + ) def test_extra_connection_engine_keywords(caplog, db): diff --git a/dask/dataframe/methods.py b/dask/dataframe/methods.py index 06128b20d21..3e07acc4885 100644 --- a/dask/dataframe/methods.py +++ b/dask/dataframe/methods.py @@ -194,8 +194,13 @@ def describe_aggregate(values): def describe_numeric_aggregate( - stats, name=None, is_timedelta_col=False, is_datetime_col=False + stats, + name=None, + is_timedelta_col=False, + is_datetime_col=False, + unit="ns", ): + unit = unit or "ns" assert len(stats) == 6 count, mean, std, min, q, max = stats @@ -205,17 +210,17 @@ def describe_numeric_aggregate( typ = type(q) if is_timedelta_col: - mean = pd.to_timedelta(mean) - std = pd.to_timedelta(std) - min = pd.to_timedelta(min) - max = pd.to_timedelta(max) - q = q.apply(lambda x: pd.to_timedelta(x)) + mean = pd.to_timedelta(mean, unit=unit).as_unit(unit) + std = pd.to_timedelta(std, unit=unit).as_unit(unit) + min = pd.to_timedelta(min, unit=unit).as_unit(unit) + max = pd.to_timedelta(max, unit=unit).as_unit(unit) + q = q.apply(lambda x: pd.to_timedelta(x, unit=unit).as_unit(unit)) if is_datetime_col: # mean is not implemented for datetime - min = pd.to_datetime(min) - max = pd.to_datetime(max) - q = q.apply(lambda x: pd.to_datetime(x)) + min = pd.to_datetime(min, unit=unit).as_unit(unit) + max = pd.to_datetime(max, unit=unit).as_unit(unit) + q = q.apply(lambda x: pd.to_datetime(x, unit=unit).as_unit(unit)) if is_datetime_col: part1 = typ([count, min], index=["count", "min"]) diff --git a/dask/dataframe/tests/test_arithmetics_reduction.py b/dask/dataframe/tests/test_arithmetics_reduction.py index e40382d508a..a2c06287ac1 100644 --- a/dask/dataframe/tests/test_arithmetics_reduction.py +++ b/dask/dataframe/tests/test_arithmetics_reduction.py @@ -1771,17 +1771,13 @@ def test_datetime_std_with_larger_dataset(axis, skipna, numeric_only): kwargs = {} if numeric_only is None else {"numeric_only": numeric_only} kwargs["skipna"] = skipna - expected = pdf[["dt1"]].std(axis=axis, **kwargs) - result = ddf[["dt1"]].std(axis=axis, **kwargs) - # assert_near_timedeltas(result.compute(), expected) - # Same thing but as Series. No axis, since axis=1 raises error assert_near_timedeltas(ddf["dt1"].std(**kwargs).compute(), pdf["dt1"].std(**kwargs)) # Computation on full dataset expected = pdf.std(axis=axis, **kwargs) - result = ddf.std(axis=axis, **kwargs) - assert_near_timedeltas(result.compute(), expected) + result = ddf.std(axis=axis, **kwargs).compute() + assert_near_timedeltas(result, expected) @pytest.mark.parametrize("skipna", [False, True]) diff --git a/dask/dataframe/tests/test_dataframe.py b/dask/dataframe/tests/test_dataframe.py index d1fb8524ef2..f0f65f72d89 100644 --- a/dask/dataframe/tests/test_dataframe.py +++ b/dask/dataframe/tests/test_dataframe.py @@ -6314,7 +6314,7 @@ def test_preserve_ts_unit_in_meta_creation(): { "a": [1], "timestamp": pd.Series( - [pd.Timestamp.utcnow()], dtype="datetime64[us, UTC]" + [pd.Timestamp.now("UTC")], dtype="datetime64[us, UTC]" ), } ) diff --git a/dask/dataframe/tests/test_indexing.py b/dask/dataframe/tests/test_indexing.py index 1a4d9b74390..2e68993f944 100644 --- a/dask/dataframe/tests/test_indexing.py +++ b/dask/dataframe/tests/test_indexing.py @@ -9,7 +9,13 @@ import dask import dask.dataframe as dd from dask.base import tokenize -from dask.dataframe._compat import PANDAS_GE_210, PANDAS_GE_220, IndexingError, tm +from dask.dataframe._compat import ( + PANDAS_GE_210, + PANDAS_GE_220, + PANDAS_GE_300, + IndexingError, + tm, +) from dask.dataframe.indexing import _coerce_loc_index from dask.dataframe.utils import assert_eq, make_meta, pyarrow_strings_enabled @@ -351,8 +357,10 @@ def test_getitem_integer_slice(): ddf = dd.from_pandas(df, 2) # except for float dtype indexes ctx = contextlib.nullcontext() - if PANDAS_GE_210: + if PANDAS_GE_210 and not PANDAS_GE_300: ctx = pytest.warns(FutureWarning, match="float-dtype index") + elif PANDAS_GE_300: + ctx = pytest.raises(NotImplementedError) with ctx: assert_eq(ddf[2:8], df[2:8]) with ctx: From 8f2d9c2318044ceefdb57fd4246dc88cf2dd1b0e Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Wed, 24 Jul 2024 16:53:06 +0200 Subject: [PATCH 35/72] Skip new warning from pandas (#11249) Co-authored-by: Hendrik Makait --- dask/dataframe/groupby.py | 16 ++++++++++------ dask/dataframe/indexing.py | 4 ++-- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/dask/dataframe/groupby.py b/dask/dataframe/groupby.py index 5163727954d..24bd5cc5c39 100644 --- a/dask/dataframe/groupby.py +++ b/dask/dataframe/groupby.py @@ -3123,12 +3123,16 @@ def _unique_aggregate(series_gb, name=None): def _value_counts(x, **kwargs): - if not x.groups or all( - pd.isna(key) for key in flatten(x.groups.keys(), container=tuple) - ): - return pd.Series(dtype=int) - else: - return x.value_counts(**kwargs) + with warnings.catch_warnings(): + warnings.filterwarnings( + "ignore", "`groups` by one element list returns", FutureWarning + ) + if not x.groups or all( + pd.isna(key) for key in flatten(x.groups.keys(), container=tuple) + ): + return pd.Series(dtype=int) + else: + return x.value_counts(**kwargs) def _value_counts_aggregate(series_gb): diff --git a/dask/dataframe/indexing.py b/dask/dataframe/indexing.py index 76362b0d239..b4fd45ffc19 100644 --- a/dask/dataframe/indexing.py +++ b/dask/dataframe/indexing.py @@ -385,14 +385,14 @@ def _maybe_partial_time_string(index, indexer, unit="ns"): stop = index._maybe_cast_slice_bound(indexer.stop, "right") else: stop = indexer.stop - if PANDAS_GE_300: + if PANDAS_GE_300 and hasattr(start, "as_unit"): start, stop = start.as_unit(unit), stop.as_unit(unit) return slice(start, stop) elif isinstance(indexer, str): start = index._maybe_cast_slice_bound(indexer, "left") stop = index._maybe_cast_slice_bound(indexer, "right") - if PANDAS_GE_300: + if PANDAS_GE_300 and hasattr(start, "as_unit"): start, stop = start.as_unit(unit), stop.as_unit(unit) return slice(min(start, stop), max(start, stop)) From 534cd7b7d546657339b260f3274b2d1016bc181a Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Thu, 25 Jul 2024 10:25:29 +0200 Subject: [PATCH 36/72] Fixup casting error in pandas 3 (#11250) --- dask/dataframe/backends.py | 7 ++++++- dask/dataframe/core.py | 12 +++++++++--- dask/dataframe/indexing.py | 6 ++++-- dask/dataframe/io/tests/test_parquet.py | 11 +++++++++-- dask/dataframe/tseries/resample.py | 1 + 5 files changed, 29 insertions(+), 8 deletions(-) diff --git a/dask/dataframe/backends.py b/dask/dataframe/backends.py index afdf46522ed..1ad06959c9a 100644 --- a/dask/dataframe/backends.py +++ b/dask/dataframe/backends.py @@ -371,7 +371,12 @@ def _nonempty_index(idx): # `self.monotonic_increasing` or `self.monotonic_decreasing` try: return pd.date_range( - start=start, periods=2, freq=idx.freq, tz=idx.tz, name=idx.name + start=start, + periods=2, + freq=idx.freq, + tz=idx.tz, + name=idx.name, + unit=idx.unit, ) except ValueError: # older pandas versions data = [start, "1970-01-02"] if idx.freq is None else None diff --git a/dask/dataframe/core.py b/dask/dataframe/core.py index 2c2b0c9000c..072f7af0bbd 100644 --- a/dask/dataframe/core.py +++ b/dask/dataframe/core.py @@ -8589,7 +8589,13 @@ def _sqrt_and_convert_to_timedelta(partition, axis, dtype=None, *args, **kwargs) category=RuntimeWarning, message="invalid value encountered in cast", ) - return pd.to_timedelta(M.std(partition, *args, axis=axis, **kwargs)) + unit = kwargs.pop("unit", None) + result = pd.to_timedelta( + M.std(partition, *args, axis=axis, **kwargs), unit=unit + ) + if unit is not None and dtype is not None: + result = result.astype(dtype) + return result is_df_like, time_cols = kwargs["is_df_like"], kwargs["time_cols"] @@ -8610,8 +8616,8 @@ def _sqrt_and_convert_to_timedelta(partition, axis, dtype=None, *args, **kwargs) if units is None: units = [None] * len(time_cols) for time_col, matching_val, unit in zip(time_cols, matching_vals, units): - result = pd.to_timedelta(matching_val, unit=kwargs.get("unit", None)) - if kwargs.get("unit", None) is not None: + result = pd.to_timedelta(matching_val, unit=unit) + if unit is not None: result = result.as_unit(unit) sqrt[time_col] = result diff --git a/dask/dataframe/indexing.py b/dask/dataframe/indexing.py index b4fd45ffc19..d03b15db705 100644 --- a/dask/dataframe/indexing.py +++ b/dask/dataframe/indexing.py @@ -386,14 +386,16 @@ def _maybe_partial_time_string(index, indexer, unit="ns"): else: stop = indexer.stop if PANDAS_GE_300 and hasattr(start, "as_unit"): - start, stop = start.as_unit(unit), stop.as_unit(unit) + start = None if start is None else start.as_unit(unit) + stop = None if stop is None else stop.as_unit(unit) return slice(start, stop) elif isinstance(indexer, str): start = index._maybe_cast_slice_bound(indexer, "left") stop = index._maybe_cast_slice_bound(indexer, "right") if PANDAS_GE_300 and hasattr(start, "as_unit"): - start, stop = start.as_unit(unit), stop.as_unit(unit) + start = None if start is None else start.as_unit(unit) + stop = None if stop is None else stop.as_unit(unit) return slice(min(start, stop), max(start, stop)) return indexer diff --git a/dask/dataframe/io/tests/test_parquet.py b/dask/dataframe/io/tests/test_parquet.py index 9f59cef3dbb..7b6baefaa91 100644 --- a/dask/dataframe/io/tests/test_parquet.py +++ b/dask/dataframe/io/tests/test_parquet.py @@ -4055,7 +4055,11 @@ def test_roundtrip_decimal_dtype(tmpdir): ddf1 = ddf1.astype({"col1": pd.ArrowDtype(pa.decimal128(5, 2))}) else: assert ddf1["col1"].dtype == ddf2["col1"].dtype - assert_eq(ddf1, ddf2, check_divisions=False) + + # seems to be a Pyarrow bug + assert_eq(ddf1, ddf2, check_divisions=False, check_dtype=not PANDAS_GE_300) + if PANDAS_GE_300: + assert ddf2["ts"].dtype != ddf1["ts"].dtype @PYARROW_MARK @@ -4080,7 +4084,10 @@ def test_roundtrip_date_dtype(tmpdir): ddf1 = ddf1.astype({"col1": pd.ArrowDtype(pa.date32())}) else: assert ddf1["col1"].dtype == ddf2["col1"].dtype - assert_eq(ddf1, ddf2, check_divisions=False) + # seems to be a Pyarrow bug + assert_eq(ddf1, ddf2, check_divisions=False, check_dtype=not PANDAS_GE_300) + if PANDAS_GE_300: + assert ddf2["ts"].dtype != ddf1["ts"].dtype def test_roundtrip_rename_columns(tmpdir, engine): diff --git a/dask/dataframe/tseries/resample.py b/dask/dataframe/tseries/resample.py index bfad295a183..bd6a25be3af 100644 --- a/dask/dataframe/tseries/resample.py +++ b/dask/dataframe/tseries/resample.py @@ -38,6 +38,7 @@ def _resample_series( freq=rule, **closed_kwargs, name=out.index.name, + unit=out.index.unit, ).tz_localize(start.tz, nonexistent="shift_forward") if not out.index.isin(new_index).all(): From e073ea45f9e8a6ea8aeee9695f8342015f5e887f Mon Sep 17 00:00:00 2001 From: Bernhard Raml Date: Fri, 26 Jul 2024 14:53:47 +0200 Subject: [PATCH 37/72] Match default chunksize in docstring to actual default set in code (#11254) Co-authored-by: Berhard Raml --- dask/multiprocessing.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dask/multiprocessing.py b/dask/multiprocessing.py index 54401115b98..1829698a372 100644 --- a/dask/multiprocessing.py +++ b/dask/multiprocessing.py @@ -179,7 +179,7 @@ def get( Function to initialize a worker process before running any tasks in it. chunksize: int, optional Size of chunks to use when dispatching work. - Defaults to 5 as some batching is helpful. + Defaults to 6 as some batching is helpful. If -1, will be computed to evenly divide ready work across workers. """ chunksize = chunksize or config.get("chunksize", 6) From f95007b1c3a569d3dd2de333bb7bd06b77d3a642 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Tue, 30 Jul 2024 12:30:16 +0200 Subject: [PATCH 38/72] Update data loading section of best practices (#11247) Co-authored-by: Florian Jetter --- docs/source/best-practices.rst | 173 ++++++++++++++++++--------------- 1 file changed, 97 insertions(+), 76 deletions(-) diff --git a/docs/source/best-practices.rst b/docs/source/best-practices.rst index 37e4d6c5e64..dfd439f270a 100644 --- a/docs/source/best-practices.rst +++ b/docs/source/best-practices.rst @@ -181,22 +181,6 @@ Fortunately, there are many options to support custom workloads: map_overlap reduction - -Stop Using Dask When No Longer Needed -------------------------------------- - -In many workloads it is common to use Dask to read in a large amount of data, -reduce it down, and then iterate on a much smaller amount of data. For this -latter stage on smaller data it may make sense to stop using Dask, and start -using normal Python again. - -.. code-block:: python - - df = dd.read_parquet("lots-of-data-*.parquet") - df = df.groupby('name').mean() # reduce data significantly - df = df.compute() # continue on with pandas/NumPy - - Store Data Efficiently ---------------------- @@ -245,103 +229,140 @@ For more information on threads, processes, and how to configure them in Dask, s Load Data with Dask ------------------- -If you need to work with large Python objects, then please let Dask create -them. A common anti-pattern we see is people creating large Python objects -outside of Dask, then giving those objects to Dask and asking it to manage them. -This works, but means that Dask needs to move around these very large objects -with its metadata, rather than as normal Dask-controlled results. +A common anti-pattern we se is people creating large Python objects like a DataFrame +or an Array on the client (i.e. their local machine) outside of Dask and then embedding +them into the computation. This means that Dask has to send these objects over the network +multiple times instead of just passing pointers to the data. + +This incurs a lot of overhead and slows down a computation quite significantly, especially +so if the network connection between the client and the scheduler is slow. It can +also overload the scheduler so that it errors with out of memory errors. Instead, you +should use Dask methods to load the data and use Dask to control the results. Here are some common patterns to avoid and nicer alternatives: -DataFrames -~~~~~~~~~~ -.. code-block:: python +.. tab-set:: - # Don't + .. tab-item:: DataFrames + :sync: dataframe - ddf = ... a dask dataframe ... - for fn in filenames: - df = pandas.read_csv(fn) # Read locally with pandas - ddf = ddf.append(df) # Give to Dask + We are using Dask to read a parquet dataset before appending a set of pandas + DataFrames to it. We are loading the csv files into memory before sending the + data to Dask. -.. code-block:: python + .. code-block:: python - # Do + ddf = dd.read_parquet(...) - ddf = dd.read_csv(filenames) + pandas_dfs = [] + for fn in filenames: + pandas_dfs(pandas.read_csv(fn)) # Read locally with pandas + ddf = dd.concat([ddf] + pandas_dfs) # Give to Dask -Arrays -~~~~~~ + Instead, we can use Dask to read the csv files directly, keeping all data + on the cluster. -.. code-block:: python + .. code-block:: python - # Don't + ddf = dd.read_parquet(...) + ddf2 = dd.read_csv(filenames) + ddf = dd.concat([ddf, ddf2]) - f = h5py.File(...) - x = np.asarray(f["x"]) # Get data as a NumPy array locally - x = da.from_array(x) # Hand NumPy array to Dask + .. tab-item:: Arrays + :sync: array -.. code-block:: python + We are using NumPy to create an in-memory array before handing it over to + Dask, forcing Dask to embed the array into the task graph instead of handling + pointers to the data. - # Do + .. code-block:: python - f = h5py.File(...) - x = da.from_array(f["x"]) # Let Dask do the reading + f = h5py.File(...) -Delayed -~~~~~~~ + x = np.asarray(f["x"]) # Get data as a NumPy array locally + x = da.from_array(x) # Hand NumPy array to Dask -.. code-block:: python + Instead, we can use Dask to read the file directly, keeping all data + on the cluster. - # Don't + .. code-block:: python - @dask.delayed - def process(a, b): - ... + f = h5py.File(...) + x = da.from_array(f["x"]) # Let Dask do the reading - df = pandas.read_csv("some-large-file.csv") # Create large object locally - results = [] - for item in L: - result = process(item, df) # include df in every delayed call - results.append(result) + .. tab-item:: Delayed + :sync: delayed -.. code-block:: python + We are using pandas to read a large CSV file before building a Graph + with delayed to parallelize a computation on the data. + + .. code-block:: python + + @dask.delayed + def process(a, b): + ... + + df = pandas.read_csv("some-large-file.csv") # Create large object locally + results = [] + for item in L: + result = process(item, df) # include df in every delayed call + results.append(result) + + Instead, we can use delayed to read the data as well. This avoid embedding the + large file into the graph, Dask can just pass a reference to the delayed + object around. - # Do + .. code-block:: python - @dask.delayed - def process(a, b): - ... + @dask.delayed + def process(a, b): + ... - df = dask.delayed(pandas.read_csv)("some-large-file.csv") # Let Dask build object - results = [] - for item in L: - result = process(item, df) # include pointer to df in every delayed call - results.append(result) + df = dask.delayed(pandas.read_csv)("some-large-file.csv") # Let Dask build object + results = [] + for item in L: + result = process(item, df) # include pointer to df in every delayed call + results.append(result) +Embedding large objects like pandas DataFrames or Arrays into the computation is a +frequent pain point for Dask users. It adds a significant delay until the scheduler +has received and is able to start the computation and stresses the scheduler during +the computation. + +Using Dask to load these objects instead avoids these issues and improves the performance +of a computation significantly. Avoid calling compute repeatedly -------------------------------- -Compute related results with shared computations in a single :func:`dask.compute` call +Calling ``compute`` will block the execution on the client until the Dask computation +completes. A pattern we regularly see is users calling ``compute`` in a loop or sequentially +on slightly different queries. + +This prohibits Dask from parallelizing different computations on the cluster and from +sharing intermediate results between different queries. .. code-block:: python - # Don't repeatedly call compute + foo = ... + results = [] + for i in range(...): + results.append(foo.select(...).compute()) - df = dd.read_csv("...") - xmin = df.x.min().compute() - xmax = df.x.max().compute() +This holds execution every time that the iteration arrives at the compute call computing +one query at a time. .. code-block:: python - # Do compute multiple results at the same time - - df = dd.read_csv("...") - - xmin, xmax = dask.compute(df.x.min(), df.x.max()) + foo = ... + results = [] + for i in range(...): + results.append(foo.select(...)) # no compute here + results = dask.compute(*results) This allows Dask to compute the shared parts of the computation (like the -``dd.read_csv`` call above) only once, rather than once per ``compute`` call. +``foo`` object above) only once, rather than once per ``compute`` call and allows +Dask to parallelize across the different selects as well instead of running +them sequentially. From 8dcf306e831ad91df32c0e45bd1670c93bc25b89 Mon Sep 17 00:00:00 2001 From: Sarah Charlotte Johnson Date: Wed, 31 Jul 2024 09:03:05 -0700 Subject: [PATCH 39/72] Update zoom link for monthly meeting (#11265) --- docs/source/support.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/source/support.rst b/docs/source/support.rst index 805e333e5c7..bfbfa23f0ae 100644 --- a/docs/source/support.rst +++ b/docs/source/support.rst @@ -17,7 +17,7 @@ Community Meeting We've combined the monthly Dask Demo Day and Dask Developer Meeting into a single, monthly Dask community meeting. Join us for the monthly community meetings on the first Thursday of the month at -10:00 US Central Time. `Join via Zoom `__. +10:00 US Central Time. `Join via Zoom `__. Have something you'd like to share? Let us know by dropping a comment on `this GitHub issue `__. From b57de0251ecdcf601d67b9e0bf7d447f2b457164 Mon Sep 17 00:00:00 2001 From: Florian Jetter Date: Fri, 2 Aug 2024 15:52:28 +0200 Subject: [PATCH 40/72] order: remove data task graph normalization (#11263) --- dask/order.py | 29 ----------------------------- 1 file changed, 29 deletions(-) diff --git a/dask/order.py b/dask/order.py index 6e427efa03c..d91e139a41e 100644 --- a/dask/order.py +++ b/dask/order.py @@ -46,7 +46,6 @@ arise, and the order we would like to be determined. """ -import copy from collections import defaultdict, deque, namedtuple from collections.abc import Iterable, Mapping, MutableMapping from typing import Any, Callable, Literal, NamedTuple, overload @@ -110,14 +109,7 @@ def order( expected_len = len(dsk) if dependencies is None: - dependencies_are_copy = True dependencies = {k: get_dependencies(dsk, k) for k in dsk} - else: - # Below we're removing items from the sets in this dict - # We need a deepcopy for that but we only want to do this if necessary - # since this only happens for special cases. - dependencies_are_copy = False - dependencies = dict(dependencies) dependents = reverse_dict(dependencies) @@ -137,7 +129,6 @@ def order( # way that is simpler to handle all_tasks = False n_removed_leaves = 0 - requires_data_task = defaultdict(set) while not all_tasks: all_tasks = True for leaf in list(leaf_nodes): @@ -165,23 +156,6 @@ def order( if not dependents[dep]: leaf_nodes.add(dep) - for root in list(root_nodes): - if root in leaf_nodes: - continue - if not istask(dsk[root]) and len(dependents[root]) > 1: - if not dependencies_are_copy: - dependencies_are_copy = True - dependencies = copy.deepcopy(dependencies) - root_nodes.remove(root) - for dep in dependents[root]: - requires_data_task[dep].add(root) - dependencies[dep].remove(root) - if not dependencies[dep]: - root_nodes.add(dep) - del dsk[root] - del dependencies[root] - del dependents[root] - num_needed, total_dependencies = ndependencies(dependencies, dependents) if len(total_dependencies) != len(dsk): cycle = getcycle(dsk, None) @@ -239,9 +213,6 @@ def add_to_result(item: Key) -> None: if item in result: continue - while requires_data_task[item]: - add_to_result(requires_data_task[item].pop()) - if return_stats: result[item] = Order(i, crit_path_counter - _crit_path_counter_offset) else: From 8fa8d2e86718db544c90165e115432468996901e Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Mon, 5 Aug 2024 17:48:08 +0200 Subject: [PATCH 41/72] Implement task-based array shuffle (#11262) Co-authored-by: Hendrik Makait --- dask/array/__init__.py | 1 + dask/array/_shuffle.py | 196 +++++++++++++++++++++++++++++++ dask/array/core.py | 17 +++ dask/array/tests/test_shuffle.py | 83 +++++++++++++ dask/dask-schema.yaml | 11 ++ dask/dask.yaml | 2 + docs/source/array-api.rst | 2 + 7 files changed, 312 insertions(+) create mode 100644 dask/array/_shuffle.py create mode 100644 dask/array/tests/test_shuffle.py diff --git a/dask/array/__init__.py b/dask/array/__init__.py index 58cac45bbd7..12849f6f1e1 100644 --- a/dask/array/__init__.py +++ b/dask/array/__init__.py @@ -32,6 +32,7 @@ def _array_expr_enabled() -> bool: try: from dask.array import backends, fft, lib, linalg, ma, overlap, random + from dask.array._shuffle import shuffle from dask.array.blockwise import atop, blockwise from dask.array.chunk_types import register_chunk_type from dask.array.core import ( diff --git a/dask/array/_shuffle.py b/dask/array/_shuffle.py new file mode 100644 index 00000000000..37aacb4bbdd --- /dev/null +++ b/dask/array/_shuffle.py @@ -0,0 +1,196 @@ +from __future__ import annotations + +import copy +from itertools import count, product + +import numpy as np + +from dask import config +from dask.array.chunk import getitem +from dask.array.core import Array, unknown_chunk_message +from dask.base import tokenize +from dask.highlevelgraph import HighLevelGraph + + +def shuffle(x, indexer: list[list[int]], axis): + """ + Reorders one dimensions of a Dask Array based on an indexer. + + The indexer defines a list of positional groups that will end up in the same chunk + together. A single group is in at most one chunk on this dimension, but a chunk + might contain multiple groups to avoid fragmentation of the array. + + The algorithm tries to balance the chunksizes as much as possible to ideally keep the + number of chunks consistent or at least manageable. + + Parameters + ---------- + x: dask array + Array to be shuffled. + indexer: list[list[int]] + The indexer that determines which elements along the dimension will end up in the + same chunk. Multiple groups can be in the same chunk to avoid fragmentation, but + each group will end up in exactly one chunk. + axis: int + The axis to shuffle along. + + Examples + -------- + >>> import dask.array as da + >>> import numpy as np + >>> arr = np.array([[1, 2, 3, 4, 5, 6, 7, 8], [9, 10, 11, 12, 13, 14, 15, 16]]) + >>> x = da.from_array(arr, chunks=(2, 4)) + + Separate the elements in different groups. + + >>> y = x.shuffle([[6, 5, 2], [4, 1], [3, 0, 7]], axis=1) + + The shuffle algorihthm will combine the first 2 groups into a single chunk to keep + the number of chunks small. + + The tolerance of increasing the chunk size is controlled by the configuration + "array.shuffle.chunksize-tolerance". The default value is 1.25. + + >>> y.chunks + ((2,), (5, 3)) + + The array was reordered along axis 1 according to the positional indexer that was given. + + >>> y.compute() + array([[ 7, 6, 3, 5, 2, 4, 1, 8], + [15, 14, 11, 13, 10, 12, 9, 16]]) + """ + if np.isnan(x.shape).any(): + raise ValueError( + f"Shuffling only allowed with known chunk sizes. {unknown_chunk_message}" + ) + assert isinstance(axis, int), "axis must be an integer" + + token = tokenize(x, indexer, axis) + out_name = f"shuffle-{token}" + + chunks, layer = _shuffle(x.chunks, indexer, axis, x.name, out_name, token) + graph = HighLevelGraph.from_collections(out_name, layer, dependencies=[x]) + + return Array(graph, out_name, chunks, meta=x) + + +def _shuffle(chunks, indexer, axis, in_name, out_name, token): + if not isinstance(indexer, list) or not all(isinstance(i, list) for i in indexer): + raise ValueError("indexer must be a list of lists of positional indices") + + if not axis <= len(chunks): + raise ValueError( + f"Axis {axis} is out of bounds for array with {len(chunks)} axes" + ) + + if max(map(max, indexer)) >= sum(chunks[axis]): + raise IndexError( + f"Indexer contains out of bounds index. Dimension only has {sum(chunks[axis])} elements." + ) + indexer = copy.deepcopy(indexer) + + chunksize_tolerance = config.get("array.shuffle.chunksize-tolerance") + chunk_size_limit = int(sum(chunks[axis]) / len(chunks[axis]) * chunksize_tolerance) + + # Figure out how many groups we can put into one chunk + current_chunk, new_chunks = [], [] + for idx in indexer: + if len(current_chunk) + len(idx) > chunk_size_limit and len(current_chunk) > 0: + new_chunks.append(current_chunk) + current_chunk = idx.copy() + else: + current_chunk.extend(idx) + if len(current_chunk) > chunk_size_limit / chunksize_tolerance: + new_chunks.append(current_chunk) + current_chunk = [] + if len(current_chunk) > 0: + new_chunks.append(current_chunk) + + chunk_boundaries = np.cumsum(chunks[axis]) + + # Get existing chunk tuple locations + chunk_tuples = list( + product(*(range(len(c)) for i, c in enumerate(chunks) if i != axis)) + ) + + intermediates = dict() + merges = dict() + split_name = f"shuffle-split-{token}" + slices = [slice(None)] * len(chunks) + split_name_suffixes = count() + + old_blocks = np.empty([len(c) for c in chunks], dtype="O") + for old_index in np.ndindex(old_blocks.shape): + old_blocks[old_index] = (in_name,) + old_index + + for new_chunk_idx, new_chunk_taker in enumerate(new_chunks): + new_chunk_taker = np.array(new_chunk_taker) + sorter = np.argsort(new_chunk_taker) + sorted_array = new_chunk_taker[sorter] + source_chunk_nr, taker_boundary = np.unique( + np.searchsorted(chunk_boundaries, sorted_array, side="right"), + return_index=True, + ) + taker_boundary = taker_boundary.tolist() + taker_boundary.append(len(new_chunk_taker)) + + taker_cache = {} + for chunk_tuple in chunk_tuples: + merge_keys = [] + + for c, b_start, b_end in zip( + source_chunk_nr, taker_boundary[:-1], taker_boundary[1:] + ): + # insert our axis chunk id into the chunk_tuple + chunk_key = convert_key(chunk_tuple, c, axis) + name = (split_name, next(split_name_suffixes)) + this_slice = slices.copy() + + # Cache the takers to allow de-duplication when serializing + # Ugly! + if c in taker_cache: + this_slice[axis] = taker_cache[c] + else: + this_slice[axis] = sorted_array[b_start:b_end] - ( + chunk_boundaries[c - 1] if c > 0 else 0 + ) + if len(source_chunk_nr) == 1: + this_slice[axis] = this_slice[axis][np.argsort(sorter)] + taker_cache[c] = this_slice[axis] + + intermediates[name] = getitem, old_blocks[chunk_key], tuple(this_slice) + merge_keys.append(name) + + merge_suffix = convert_key(chunk_tuple, new_chunk_idx, axis) + if len(merge_keys) > 1: + merges[(out_name,) + merge_suffix] = ( + concatenate_arrays, + merge_keys, + sorter, + axis, + ) + elif len(merge_keys) == 1: + merges[(out_name,) + merge_suffix] = intermediates.pop(merge_keys[0]) + else: + raise NotImplementedError + + output_chunks = [] + for i, c in enumerate(chunks): + if i == axis: + output_chunks.append(tuple(map(len, new_chunks))) + else: + output_chunks.append(c) + + layer = {**merges, **intermediates} + return tuple(output_chunks), layer + + +def concatenate_arrays(arrs, sorter, axis): + return np.take(np.concatenate(arrs, axis=axis), np.argsort(sorter), axis=axis) + + +def convert_key(key, chunk, axis): + key = list(key) + key.insert(axis, chunk) + return tuple(key) diff --git a/dask/array/core.py b/dask/array/core.py index 59f10fecc87..b710fe491f8 100644 --- a/dask/array/core.py +++ b/dask/array/core.py @@ -2762,6 +2762,23 @@ def rechunk( return rechunk(self, chunks, threshold, block_size_limit, balance, method) + def shuffle( + self, + indexer: list[list[int]], + axis, + ): + """Reorders one dimensions of a Dask Array based on an indexer. + + Refer to :func:`dask.array.shuffle` for full documentation. + + See Also + -------- + dask.array.shuffle : equivalent function + """ + from dask.array._shuffle import shuffle + + return shuffle(self, indexer, axis) + @property def real(self): from dask.array.ufunc import real diff --git a/dask/array/tests/test_shuffle.py b/dask/array/tests/test_shuffle.py new file mode 100644 index 00000000000..469f22be040 --- /dev/null +++ b/dask/array/tests/test_shuffle.py @@ -0,0 +1,83 @@ +from __future__ import annotations + +import numpy as np +import pytest + +import dask +import dask.array as da +from dask.array import assert_eq, shuffle +from dask.core import flatten + + +@pytest.fixture() +def arr(): + return np.arange(0, 24).reshape(8, 3).T.copy() + + +@pytest.fixture() +def darr(arr): + return da.from_array(arr, chunks=((2, 1), (4, 4))) + + +@pytest.mark.parametrize( + "indexer, chunks", + [ + ([[1, 5, 6], [0, 2, 3, 4, 7]], (3, 5)), + ([[1, 5, 6], [0, 3], [4, 2, 7]], (5, 3)), + ([[1], [0, 6, 5, 3, 2, 4], [7]], (1, 6, 1)), + ([[1, 5, 1, 5, 1, 5], [1, 6, 4, 2, 7]], (6, 5)), + ], +) +def test_shuffle(arr, darr, indexer, chunks): + result = darr.shuffle(indexer, axis=1) + expected = arr[:, list(flatten(indexer))] + assert_eq(result, expected) + assert result.chunks[0] == darr.chunks[0] + assert result.chunks[1] == chunks + + +@pytest.mark.parametrize("tol, chunks", ((1, (3, 2, 3)), (1.4, (5, 3)))) +def test_shuffle_config_tolerance(arr, darr, tol, chunks): + indexer = [[1, 5, 6], [0, 3], [4, 2, 7]] + with dask.config.set({"array.shuffle.chunksize-tolerance": tol}): + result = darr.shuffle(indexer, axis=1) + expected = arr[:, [1, 5, 6, 0, 3, 4, 2, 7]] + assert_eq(result, expected) + assert result.chunks[0] == darr.chunks[0] + assert result.chunks[1] == chunks + + +def test_shuffle_larger_array(): + arr = da.random.random((15, 15, 15), chunks=(5, 5, 5)) + indexer = np.arange(0, 15) + np.random.shuffle(indexer) + indexer = [indexer[0:6], indexer[6:8], indexer[8:9], indexer[9:]] + indexer = list(map(list, indexer)) + take_indexer = list(flatten(indexer)) + assert_eq(shuffle(arr, indexer, axis=1), arr[..., take_indexer, :]) + + +def test_incompatible_indexer(darr): + with pytest.raises(ValueError, match="indexer must be a list of lists"): + darr.shuffle("s", axis=1) + + with pytest.raises(ValueError, match="indexer must be a list of lists"): + darr.shuffle([1], axis=1) + + +def test_unknown_chunk_sizes(darr): + darr._chunks = ((np.nan, 1), (4, 4)) + with pytest.raises( + ValueError, match="Shuffling only allowed with known chunk sizes" + ): + darr.shuffle([[1]], axis=1) + + +def test_oob_axis(darr): + with pytest.raises(ValueError, match="is out of bounds"): + darr.shuffle([[1]], axis=5) + + +def test_oob_indexer(darr): + with pytest.raises(IndexError, match="Indexer contains out of bounds index"): + darr.shuffle([[16]], axis=1) diff --git a/dask/dask-schema.yaml b/dask/dask-schema.yaml index 12bf2ee0644..e05aac03835 100644 --- a/dask/dask-schema.yaml +++ b/dask/dask-schema.yaml @@ -135,6 +135,17 @@ properties: The graph growth factor above which task-based shuffling introduces an intermediate step. + shuffle: + type: object + properties: + + chunksize-tolerance: + type: number + description: | + Upper tolerance for the shuffle algorithm when creating output chunks. + Default is 1.25. This means that the shuffle algorithm can exceed + the average input chunk size along this dimension by 25%. + svg: type: object properties: diff --git a/dask/dask.yaml b/dask/dask.yaml index 06509fe8764..6e35c10e444 100644 --- a/dask/dask.yaml +++ b/dask/dask.yaml @@ -24,6 +24,8 @@ array: rechunk: method: "tasks" # Rechunking method to use threshold: 4 + shuffle: + chunksize-tolerance: 1.25 # Tolerance for the shuffle algorithm when creating output chunks. svg: size: 120 # pixels slicing: diff --git a/docs/source/array-api.rst b/docs/source/array-api.rst index b7964261392..4765378e5d7 100644 --- a/docs/source/array-api.rst +++ b/docs/source/array-api.rst @@ -207,6 +207,7 @@ Top level functions searchsorted select shape + shuffle sign signbit sin @@ -300,6 +301,7 @@ Array Array.reshape Array.round Array.shape + Array.shuffle Array.size Array.squeeze Array.std From 1587b3bd22ec42c4a8c61ba4d0f25ec2aa59b1e4 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Mon, 5 Aug 2024 20:35:07 +0200 Subject: [PATCH 42/72] Use the shuffle algorithm for take (#11267) --- dask/array/slicing.py | 188 ++++++----------------------- dask/array/tests/test_slicing.py | 195 +++++++++++++++++-------------- 2 files changed, 141 insertions(+), 242 deletions(-) diff --git a/dask/array/slicing.py b/dask/array/slicing.py index 1ed630a2558..f9add043aa7 100644 --- a/dask/array/slicing.py +++ b/dask/array/slicing.py @@ -11,7 +11,7 @@ import numpy as np from tlz import concat, memoize, merge, pluck -from dask import config, core, utils +from dask import core from dask.array.chunk import getitem from dask.base import is_dask_collection, tokenize from dask.highlevelgraph import HighLevelGraph @@ -173,13 +173,13 @@ def slice_array(out_name, in_name, blockdims, index, itemsize): index += (slice(None, None, None),) * missing # Pass down to next function - dsk_out, bd_out = slice_with_newaxes(out_name, in_name, blockdims, index, itemsize) + dsk_out, bd_out = slice_with_newaxes(out_name, in_name, blockdims, index) bd_out = tuple(map(tuple, bd_out)) return dsk_out, bd_out -def slice_with_newaxes(out_name, in_name, blockdims, index, itemsize): +def slice_with_newaxes(out_name, in_name, blockdims, index): """ Handle indexing with Nones @@ -195,7 +195,7 @@ def slice_with_newaxes(out_name, in_name, blockdims, index, itemsize): where_none[i] -= n # Pass down and do work - dsk, blockdims2 = slice_wrap_lists(out_name, in_name, blockdims, index2, itemsize) + dsk, blockdims2 = slice_wrap_lists(out_name, in_name, blockdims, index2) if where_none: expand = expander(where_none) @@ -220,7 +220,7 @@ def slice_with_newaxes(out_name, in_name, blockdims, index, itemsize): return dsk, blockdims2 -def slice_wrap_lists(out_name, in_name, blockdims, index, itemsize): +def slice_wrap_lists(out_name, in_name, blockdims, index): """ Fancy indexing along blocked array dasks @@ -262,7 +262,7 @@ def slice_wrap_lists(out_name, in_name, blockdims, index, itemsize): if all(is_arraylike(i) or i == slice(None, None, None) for i in index): axis = where_list[0] blockdims2, dsk3 = take( - out_name, in_name, blockdims, index[where_list[0]], itemsize, axis=axis + out_name, in_name, blockdims, index[where_list[0]], axis=axis ) # Mixed case. Both slices/integers and lists. slice/integer then take else: @@ -279,7 +279,7 @@ def slice_wrap_lists(out_name, in_name, blockdims, index, itemsize): ) # Do work - blockdims2, dsk2 = take(out_name, tmp, blockdims2, index[axis], 8, axis=axis2) + blockdims2, dsk2 = take(out_name, tmp, blockdims2, index[axis], axis=axis2) dsk3 = merge(dsk, dsk2) return dsk3, blockdims2 @@ -540,56 +540,7 @@ def issorted(seq): return np.all(seq[:-1] <= seq[1:]) -def slicing_plan(chunks, index): - """Construct a plan to slice chunks with the given index - - Parameters - ---------- - chunks : Tuple[int] - One dimensions worth of chunking information - index : np.ndarray[int] - The index passed to slice on that dimension - - Returns - ------- - out : List[Tuple[int, np.ndarray]] - A list of chunk/sub-index pairs corresponding to each output chunk - """ - from dask.array.utils import asarray_safe - - if not is_arraylike(index): - index = np.asanyarray(index) - - cum_chunks_tup = cached_cumsum(chunks) - cum_chunks = asarray_safe(cum_chunks_tup, like=index) - if cum_chunks.dtype.kind != "f": # Don't cast NaN chunks to int - # This is important when index.dtype=uint64 (or uint32 on 32-bit hosts) to - # prevent accidental automatic casting during `index - cum_chunks` below - cum_chunks = cum_chunks.astype(index.dtype) - - # this dispactches to the array library - chunk_locations = np.searchsorted(cum_chunks, index, side="right") - - # but we need chunk_locations as python ints for getitem calls downstream - chunk_locations = chunk_locations.tolist() - where = np.where(np.diff(chunk_locations))[0] + 1 - - extra = asarray_safe([0], like=where) - c_loc = asarray_safe([len(chunk_locations)], like=where) - where = np.concatenate([extra, where, c_loc]) - - out = [] - for i in range(len(where) - 1): - sub_index = index[where[i] : where[i + 1]] - chunk = chunk_locations[where[i]] - if chunk > 0: - sub_index = sub_index - cum_chunks[chunk - 1] - out.append((chunk, sub_index)) - - return out - - -def take(outname, inname, chunks, index, itemsize, axis=0): +def take(outname, inname, chunks, index, axis=0): """Index array with an iterable of index Handles a single index by a single list @@ -597,115 +548,44 @@ def take(outname, inname, chunks, index, itemsize, axis=0): Mimics ``np.take`` >>> from pprint import pprint - >>> chunks, dsk = take('y', 'x', [(20, 20, 20, 20)], [5, 1, 47, 3], 8, axis=0) + >>> chunks, dsk = take('y', 'x', [(20, 20, 20, 20)], [5, 1, 47, 3], axis=0) >>> chunks - ((2, 1, 1),) - >>> pprint(dsk) # doctest: +ELLIPSIS - {('y', 0): (, ('x', 0), (array([5, 1]),)), - ('y', 1): (, ('x', 2), (array([7]),)), - ('y', 2): (, ('x', 0), (array([3]),))} + ((4,),) - When list is sorted we retain original block structure + When list is sorted we still try to preserve properly sized chunks. - >>> chunks, dsk = take('y', 'x', [(20, 20, 20, 20)], [1, 3, 5, 47], 8, axis=0) + >>> chunks, dsk = take('y', 'x', [(20, 20, 20, 20)], [1, 3, 5, 47], axis=0) >>> chunks - ((3, 1),) - >>> pprint(dsk) # doctest: +ELLIPSIS +NORMALIZE_WHITESPACE - {('y', 0): (, - ('x', 0), - (array([1, 3, 5]),)), - ('y', 1): (, ('x', 2), (array([7]),))} + ((4,),) When any indexed blocks would otherwise grow larger than - dask.config.array.chunk-size, we might split them, - depending on the value of ``dask.config.slicing.split-large-chunks``. - - >>> import dask - >>> with dask.config.set({"array.slicing.split-large-chunks": True}): - ... chunks, dsk = take('y', 'x', [(1, 1, 1), (2000, 2000), (2000, 2000)], - ... [0] + [1] * 6 + [2], axis=0, itemsize=8) - >>> chunks - ((1, 3, 3, 1), (2000, 2000), (2000, 2000)) + dask.config.array.chunk-size, we will split them to avoid + growing chunksizes. """ - from dask.array.core import PerformanceWarning - plan = slicing_plan(chunks[axis], index) - if len(plan) >= len(chunks[axis]) * 10: - factor = math.ceil(len(plan) / len(chunks[axis])) + if not np.isnan(chunks[axis]).any(): + from dask.array._shuffle import _shuffle - warnings.warn( - "Slicing with an out-of-order index is generating %d " - "times more chunks" % factor, - PerformanceWarning, - stacklevel=6, - ) + average_chunk_size = int(sum(chunks[axis]) / len(chunks[axis])) - # Check for chunks from the plan that would violate the user's - # configured chunk size. - nbytes = utils.parse_bytes(config.get("array.chunk-size")) - other_chunks = [chunks[i] for i in range(len(chunks)) if i != axis] - other_numel = math.prod(max(x) for x in other_chunks) + indexer = [] + index = np.asarray(index) + for i in range(0, len(index), average_chunk_size): + indexer.append(index[i : i + average_chunk_size].tolist()) - if math.isnan(other_numel) or other_numel == 0: - warnsize = maxsize = math.inf + token = ( + outname.split("-")[-1] + if "-" in outname + else tokenize(outname, chunks, index, axis) + ) + chunks, graph = _shuffle(chunks, indexer, axis, inname, outname, token) + return chunks, graph else: - maxsize = math.ceil(nbytes / (other_numel * itemsize)) - warnsize = maxsize * 5 - - split = config.get("array.slicing.split-large-chunks", None) - - # Warn only when the default is not specified. - warned = split is not None - - for _, index_list in plan: - if not warned and len(index_list) > warnsize: - msg = ( - "Slicing is producing a large chunk. To accept the large\n" - "chunk and silence this warning, set the option\n" - " >>> with dask.config.set(**{'array.slicing.split_large_chunks': False}):\n" - " ... array[indexer]\n\n" - "To avoid creating the large chunks, set the option\n" - " >>> with dask.config.set(**{'array.slicing.split_large_chunks': True}):\n" - " ... array[indexer]" - ) - warnings.warn(msg, PerformanceWarning, stacklevel=6) - warned = True - - where_index = [] - index_lists = [] - for where_idx, index_list in plan: - index_length = len(index_list) - if split and index_length > maxsize: - index_sublist = np.array_split( - index_list, math.ceil(index_length / maxsize) - ) - index_lists.extend(index_sublist) - where_index.extend([where_idx] * len(index_sublist)) - else: - if not is_arraylike(index_list): - index_list = np.array(index_list) - index_lists.append(index_list) - where_index.append(where_idx) - - dims = [range(len(bd)) for bd in chunks] - - indims = list(dims) - indims[axis] = list(range(len(where_index))) - keys = list(product([outname], *indims)) - - outdims = list(dims) - outdims[axis] = where_index - slices = [[colon] * len(bd) for bd in chunks] - slices[axis] = index_lists - slices = list(product(*slices)) - inkeys = list(product([inname], *outdims)) - values = [(getitem, inkey, slc) for inkey, slc in zip(inkeys, slices)] - - chunks2 = list(chunks) - chunks2[axis] = tuple(map(len, index_lists)) - dsk = dict(zip(keys, values)) - - return tuple(chunks2), dsk + from dask.array.core import unknown_chunk_message + + raise ValueError( + f"Array chunk size or shape is unknown. {unknown_chunk_message}" + ) def posify_index(shape, ind): diff --git a/dask/array/tests/test_slicing.py b/dask/array/tests/test_slicing.py index 8c72fc25c6a..ed8a99e0c70 100644 --- a/dask/array/tests/test_slicing.py +++ b/dask/array/tests/test_slicing.py @@ -4,7 +4,8 @@ import warnings import pytest -from tlz import merge + +from dask.array._shuffle import concatenate_arrays np = pytest.importorskip("numpy") @@ -21,7 +22,6 @@ sanitize_index, shuffle_slice, slice_array, - slicing_plan, take, ) from dask.array.utils import assert_eq, same_keys @@ -315,62 +315,111 @@ def test_slicing_with_newaxis(): def test_take(): - chunks, dsk = take("y", "x", [(20, 20, 20, 20)], [5, 1, 47, 3], itemsize=8, axis=0) + chunks, dsk = take("y-y", "x", [(20, 20, 20, 20)], [5, 1, 47, 3], axis=0) expected = { - ("y", 0): (getitem, ("x", 0), (np.array([5, 1]),)), - ("y", 1): (getitem, ("x", 2), (np.array([7]),)), - ("y", 2): (getitem, ("x", 0), (np.array([3]),)), + ("y-y", 0): ( + concatenate_arrays, + [ + ("shuffle-split-y", 0), + ("shuffle-split-y", 1), + ], + np.array([1, 3, 0, 2]), + 0, + ), + ("shuffle-split-y", 0): ( + getitem, + ("x", 0), + (np.array([1, 3, 5]),), + ), + ("shuffle-split-y", 1): ( + getitem, + ("x", 2), + (np.array([7]),), + ), } np.testing.assert_equal(sorted(dsk.items()), sorted(expected.items())) - assert chunks == ((2, 1, 1),) + assert chunks == ((4,),) - chunks, dsk = take( - "y", "x", [(20, 20, 20, 20), (20, 20)], [5, 1, 47, 3], itemsize=8, axis=0 - ) + chunks, dsk = take("y-y", "x", [(20, 20, 20, 20), (20, 20)], [5, 1, 47, 3], axis=0) expected = { - ("y", 0, 0): ( + ("y-y", 0, 0): ( + concatenate_arrays, + [ + ("shuffle-split-y", 0), + ("shuffle-split-y", 1), + ], + np.array([1, 3, 0, 2]), + 0, + ), + ("y-y", 0, 1): ( + concatenate_arrays, + [ + ("shuffle-split-y", 2), + ("shuffle-split-y", 3), + ], + np.array([1, 3, 0, 2]), + 0, + ), + ("shuffle-split-y", 0): ( getitem, ("x", 0, 0), - (np.array([5, 1]), slice(None, None, None)), + (np.array([1, 3, 5]), slice(None)), ), - ("y", 0, 1): ( + ("shuffle-split-y", 1): ( + getitem, + ("x", 2, 0), + (np.array([7]), slice(None)), + ), + ("shuffle-split-y", 2): ( getitem, ("x", 0, 1), - (np.array([5, 1]), slice(None, None, None)), + (np.array([1, 3, 5]), slice(None)), + ), + ("shuffle-split-y", 3): ( + getitem, + ("x", 2, 1), + (np.array([7]), slice(None)), ), - ("y", 1, 0): (getitem, ("x", 2, 0), (np.array([7]), slice(None, None, None))), - ("y", 1, 1): (getitem, ("x", 2, 1), (np.array([7]), slice(None, None, None))), - ("y", 2, 0): (getitem, ("x", 0, 0), (np.array([3]), slice(None, None, None))), - ("y", 2, 1): (getitem, ("x", 0, 1), (np.array([3]), slice(None, None, None))), } np.testing.assert_equal(sorted(dsk.items()), sorted(expected.items())) - assert chunks == ((2, 1, 1), (20, 20)) + assert chunks == ((4,), (20, 20)) def test_take_sorted(): - chunks, dsk = take("y", "x", [(20, 20, 20, 20)], [1, 3, 5, 47], itemsize=8, axis=0) + chunks, dsk = take("y-y", "x", [(20, 20, 20, 20)], [1, 3, 5, 47], axis=0) expected = { - ("y", 0): (getitem, ("x", 0), ([1, 3, 5],)), - ("y", 1): (getitem, ("x", 2), ([7],)), + ("y-y", 0): ( + concatenate_arrays, + [ + ("shuffle-split-y", 0), + ("shuffle-split-y", 1), + ], + np.array([0, 1, 2, 3]), + 0, + ), + ("shuffle-split-y", 0): ( + getitem, + ("x", 0), + (np.array([1, 3, 5]),), + ), + ("shuffle-split-y", 1): ( + getitem, + ("x", 2), + (np.array([7]),), + ), } np.testing.assert_equal(dsk, expected) - assert chunks == ((3, 1),) + assert chunks == ((4,),) - chunks, dsk = take( - "y", "x", [(20, 20, 20, 20), (20, 20)], [1, 3, 5, 37], itemsize=8, axis=1 - ) - expected = merge( - { - ("y", i, 0): (getitem, ("x", i, 0), (slice(None, None, None), [1, 3, 5])) - for i in range(4) - }, - { - ("y", i, 1): (getitem, ("x", i, 1), (slice(None, None, None), [17])) - for i in range(4) - }, - ) + chunks, dsk = take("y", "x", [(20, 20, 20, 20)], np.arange(0, 80), axis=0) + expected = { + ("y", 0): (getitem, ("x", 0), (np.arange(0, 20),)), + ("y", 1): (getitem, ("x", 1), (np.arange(0, 20),)), + ("y", 2): (getitem, ("x", 2), (np.arange(0, 20),)), + ("y", 3): (getitem, ("x", 3), (np.arange(0, 20),)), + } np.testing.assert_equal(dsk, expected) - assert chunks == ((20, 20, 20, 20), (3, 1)) + assert chunks == ((20, 20, 20, 20),) def test_slicing_chunks(): @@ -392,14 +441,14 @@ def test_slicing_chunks(): def test_slicing_with_numpy_arrays(): a, bd1 = slice_array( - "y", + "y-y", "x", ((3, 3, 3, 1), (3, 3, 3, 1)), (np.array([1, 2, 9]), slice(None, None, None)), itemsize=8, ) b, bd2 = slice_array( - "y", + "y-y", "x", ((3, 3, 3, 1), (3, 3, 3, 1)), (np.array([1, 2, 9]), slice(None, None, None)), @@ -412,7 +461,7 @@ def test_slicing_with_numpy_arrays(): i = [False, True, True, False, False, False, False, False, False, True] index = (i, slice(None, None, None)) index = normalize_index(index, (10, 10)) - c, bd3 = slice_array("y", "x", ((3, 3, 3, 1), (3, 3, 3, 1)), index, itemsize=8) + c, bd3 = slice_array("y-y", "x", ((3, 3, 3, 1), (3, 3, 3, 1)), index, itemsize=8) assert bd1 == bd3 np.testing.assert_equal(a, c) @@ -860,27 +909,6 @@ def test_take_semi_sorted(): assert y.chunks == ((5, 5, 5),) -@pytest.mark.parametrize( - "chunks,index,expected", - [ - ((5, 5, 5), np.arange(5, 15) % 10, [(1, np.arange(5)), (0, np.arange(5))]), - ( - (5, 5, 5, 5), - np.arange(20) // 2, - [(0, np.arange(10) // 2), (1, np.arange(10) // 2)], - ), - ((10, 10), [15, 2, 3, 15], [(1, [5]), (0, [2, 3]), (1, [5])]), - ], -) -def test_slicing_plan(chunks, index, expected): - plan = slicing_plan(chunks, index=index) - assert len(plan) == len(expected) - for (i, x), (j, y) in zip(plan, expected): - assert i == j - assert len(x) == len(y) - assert (x == y).all() - - def test_getitem_avoids_large_chunks(): with dask.config.set({"array.chunk-size": "0.1Mb"}): a = np.arange(2 * 128 * 128, dtype="int64").reshape(2, 128, 128) @@ -895,12 +923,9 @@ def test_getitem_avoids_large_chunks(): arr = da.from_array(a, chunks=(1, 128, 128)) # large chunks expected = a[indexer] - # By default, we warn - with pytest.warns(da.PerformanceWarning): - result = arr[indexer] - + result = arr[indexer] assert_eq(result, expected) - assert result.chunks == ((1, 11), (128,), (128,)) + assert result.chunks == ((1,) * 12, (128,), (128,)) # Users can silence the warning with dask.config.set({"array.slicing.split-large-chunks": False}): @@ -936,38 +961,36 @@ def test_take_avoids_large_chunks(): # unit test for https://github.com/dask/dask/issues/6270 with dask.config.set({"array.slicing.split-large-chunks": True}): chunks = ((1, 1, 1, 1), (500,), (500,)) - itemsize = 8 index = np.array([0, 1] + [2] * 101 + [3]) - chunks2, dsk = take("a", "b", chunks, index, itemsize) - assert chunks2 == ((1, 1, 51, 50, 1), (500,), (500,)) - assert len(dsk) == 5 + chunks2, dsk = take("a", "b", chunks, index) + assert chunks2 == ((1,) * 104, (500,), (500,)) + assert len(dsk) == 104 index = np.array([0] * 101 + [1, 2, 3]) - chunks2, dsk = take("a", "b", chunks, index, itemsize) - assert chunks2 == ((51, 50, 1, 1, 1), (500,), (500,)) - assert len(dsk) == 5 + chunks2, dsk = take("a", "b", chunks, index) + assert chunks2 == ((1,) * 104, (500,), (500,)) + assert len(dsk) == 104 index = np.array([0, 1, 2] + [3] * 101) - chunks2, dsk = take("a", "b", chunks, index, itemsize) - assert chunks2 == ((1, 1, 1, 51, 50), (500,), (500,)) - assert len(dsk) == 5 + chunks2, dsk = take("a", "b", chunks, index) + assert chunks2 == ((1,) * 104, (500,), (500,)) + assert len(dsk) == 104 chunks = ((500,), (1, 1, 1, 1), (500,)) index = np.array([0, 1, 2] + [3] * 101) - chunks2, dsk = take("a", "b", chunks, index, itemsize, axis=1) - assert chunks2 == ((500,), (1, 1, 1, 51, 50), (500,)) - assert len(dsk) == 5 + chunks2, dsk = take("a", "b", chunks, index, axis=1) + assert chunks2 == ((500,), (1,) * 104, (500,)) + assert len(dsk) == 104 def test_take_uses_config(): with dask.config.set({"array.slicing.split-large-chunks": True}): chunks = ((1, 1, 1, 1), (500,), (500,)) index = np.array([0, 1] + [2] * 101 + [3]) - itemsize = 8 with config.set({"array.chunk-size": "10GB"}): - chunks2, dsk = take("a", "b", chunks, index, itemsize) - assert chunks2 == ((1, 1, 101, 1), (500,), (500,)) - assert len(dsk) == 4 + chunks2, dsk = take("a", "b", chunks, index) + assert chunks2 == ((1,) * 104, (500,), (500,)) + assert len(dsk) == 104 def test_pathological_unsorted_slicing(): @@ -976,11 +999,7 @@ def test_pathological_unsorted_slicing(): # [0, 10, 20, ... 90, 1, 11, 21, ... 91, ...] index = np.arange(100).reshape(10, 10).ravel(order="F") - with pytest.warns(da.PerformanceWarning) as info: - x[index] - - assert "10" in str(info.list[0]) - assert "out-of-order" in str(info.list[0]) + assert_eq(x[index], x.compute()[index]) @pytest.mark.parametrize("params", [(2, 2, 1), (5, 3, 2)]) From 98a43d8c9b291329b2bcc143a935e48a027afb8e Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Tue, 6 Aug 2024 12:10:50 +0200 Subject: [PATCH 43/72] Revert "order: remove data task graph normalization" (#11276) --- dask/order.py | 29 +++++++++++++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/dask/order.py b/dask/order.py index d91e139a41e..6e427efa03c 100644 --- a/dask/order.py +++ b/dask/order.py @@ -46,6 +46,7 @@ arise, and the order we would like to be determined. """ +import copy from collections import defaultdict, deque, namedtuple from collections.abc import Iterable, Mapping, MutableMapping from typing import Any, Callable, Literal, NamedTuple, overload @@ -109,7 +110,14 @@ def order( expected_len = len(dsk) if dependencies is None: + dependencies_are_copy = True dependencies = {k: get_dependencies(dsk, k) for k in dsk} + else: + # Below we're removing items from the sets in this dict + # We need a deepcopy for that but we only want to do this if necessary + # since this only happens for special cases. + dependencies_are_copy = False + dependencies = dict(dependencies) dependents = reverse_dict(dependencies) @@ -129,6 +137,7 @@ def order( # way that is simpler to handle all_tasks = False n_removed_leaves = 0 + requires_data_task = defaultdict(set) while not all_tasks: all_tasks = True for leaf in list(leaf_nodes): @@ -156,6 +165,23 @@ def order( if not dependents[dep]: leaf_nodes.add(dep) + for root in list(root_nodes): + if root in leaf_nodes: + continue + if not istask(dsk[root]) and len(dependents[root]) > 1: + if not dependencies_are_copy: + dependencies_are_copy = True + dependencies = copy.deepcopy(dependencies) + root_nodes.remove(root) + for dep in dependents[root]: + requires_data_task[dep].add(root) + dependencies[dep].remove(root) + if not dependencies[dep]: + root_nodes.add(dep) + del dsk[root] + del dependencies[root] + del dependents[root] + num_needed, total_dependencies = ndependencies(dependencies, dependents) if len(total_dependencies) != len(dsk): cycle = getcycle(dsk, None) @@ -213,6 +239,9 @@ def add_to_result(item: Key) -> None: if item in result: continue + while requires_data_task[item]: + add_to_result(requires_data_task[item].pop()) + if return_stats: result[item] = Order(i, crit_path_counter - _crit_path_counter_offset) else: From 62628cb90ad19cfa915c33b3d194a3a3f96b5b22 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Tue, 6 Aug 2024 14:14:28 +0200 Subject: [PATCH 44/72] Add changelog entry for take (#11274) Co-authored-by: Hendrik Makait --- docs/source/changelog.rst | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/docs/source/changelog.rst b/docs/source/changelog.rst index 6d170fb4380..e64e8e8f2a6 100644 --- a/docs/source/changelog.rst +++ b/docs/source/changelog.rst @@ -1,6 +1,34 @@ Changelog ========= +.. _v2024.8.0: + +2024.8.0 +-------- + +Highlights +^^^^^^^^^^ + +Improve efficiency and performance of slicing with positional indexers +"""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""" + +Performance improvement for slicing a Dask Array with a positional indexer. +Random access patterns are now more stable and produce easier-to-use results. + +.. code-block:: python + + x[slice(None), [1, 1, 3, 6, 3, 4, 5]] + +Using a positional indexer was previously prone to drastically increasing the +number of output chunks and generating a very large task graph. This has been +fixed with a more efficient algorithm. + +The new algorithm will keep the chunk-sizes along the axis that is indexed +the same to avoid fragmentation of chunks or a large increase in chunk-size. + +See :pr:`11262` and :pr:`11267` by `Patrick Hoefler`_ for more details and performance +benchmarks. + .. _v2024.7.1: 2024.7.1 From b61e6975becc2ca47369c3c0792869704cd3d2cb Mon Sep 17 00:00:00 2001 From: Florian Jetter Date: Tue, 6 Aug 2024 15:39:26 +0200 Subject: [PATCH 45/72] order: add regression test for xarray map reduce (#11277) --- dask/tests/test_order.py | 79 +++++++++++++++++++++++++++++++++++++++- 1 file changed, 77 insertions(+), 2 deletions(-) diff --git a/dask/tests/test_order.py b/dask/tests/test_order.py index 8980176a779..70a1c6923ec 100644 --- a/dask/tests/test_order.py +++ b/dask/tests/test_order.py @@ -26,12 +26,12 @@ def f(*args): pass -def visualize(dsk, **kwargs): +def visualize(dsk, suffix="", **kwargs): """Utility to visualize the raw low level graphs in this tests suite. This automatically generates a set of visualizations with different metrics and writes them out to a file prefixed by the test name and suffixed by the measure used.""" - funcname = inspect.stack()[1][3] + funcname = inspect.stack()[1][3] + suffix if hasattr(dsk, "__dask_graph__"): dsk = collections_to_dsk([dsk], optimize_graph=True) @@ -2022,6 +2022,81 @@ def test_order_flox_reduction_2(abcde): assert o[final_nodes[ix]] - o[final_nodes[ix - 1]] == 5 +def test_xarray_map_reduce_with_slicing(): + # See https://github.com/dask/distributed/pull/8818 + dsk = { + ("transpose", 0, 0, 0): (f, ("groupby-agg", 0, 0, 0)), + ("transpose", 0, 0, 1): (f, ("groupby-agg", 0, 0, 1)), + ("transpose", 0, 1, 0): (f, ("groupby-agg", 0, 1, 0)), + ("transpose", 0, 1, 1): (f, ("groupby-agg", 0, 1, 1)), + ("groupby-chunk", 0, 0, 0): (f, ("array", 0), ("getitem", 0, 0, 0)), + ("groupby-chunk", 0, 0, 1): (f, ("array", 1), ("getitem", 1, 0, 0)), + ("groupby-chunk", 0, 1, 0): (f, ("array", 0), ("getitem", 0, 0, 1)), + ("groupby-chunk", 0, 1, 1): (f, ("array", 1), ("getitem", 1, 0, 1)), + ("groupby-chunk", 1, 0, 0): (f, ("array", 0), ("getitem", 0, 1, 0)), + ("groupby-chunk", 1, 0, 1): (f, ("array", 1), ("getitem", 1, 1, 0)), + ("groupby-chunk", 1, 1, 0): (f, ("array", 0), ("getitem", 0, 1, 1)), + ("groupby-chunk", 1, 1, 1): (f, ("getitem", 1, 1, 1), ("array", 1)), + ("getitem", 0, 0, 0): (f, ("open_data", 0, 0, 0)), + ("getitem", 0, 0, 1): (f, ("open_data", 0, 0, 1)), + ("getitem", 0, 1, 0): (f, ("open_data", 0, 1, 0)), + ("getitem", 0, 1, 1): (f, ("open_data", 0, 1, 1)), + ("getitem", 1, 0, 0): (f, ("open_data", 1, 0, 0)), + ("getitem", 1, 0, 1): (f, ("open_data", 1, 0, 1)), + ("getitem", 1, 1, 0): (f, ("open_data", 1, 1, 0)), + ("getitem", 1, 1, 1): (f, ("open_data", 1, 1, 1)), + "data": "a", + ("array", 0): "b", + ("array", 1): "c", + ("open_data", 0, 1, 1): (f, "data"), + ("groupby-agg", 0, 1, 0): ( + f, + ("groupby-chunk", 1, 0, 1), + ("groupby-chunk", 1, 0, 0), + ), + ("groupby-agg", 0, 1, 1): ( + f, + ("groupby-chunk", 1, 1, 0), + ("groupby-chunk", 1, 1, 1), + ), + ("open_data", 1, 0, 0): (f, "data"), + ("open_data", 0, 1, 0): (f, "data"), + ("open_data", 1, 0, 1): (f, "data"), + ("open_data", 1, 1, 0): (f, "data"), + ("groupby-agg", 0, 0, 1): ( + f, + ("groupby-chunk", 0, 1, 0), + ("groupby-chunk", 0, 1, 1), + ), + ("open_data", 0, 0, 1): (f, "data"), + ("open_data", 1, 1, 1): (f, "data"), + ("groupby-agg", 0, 0, 0): ( + f, + ("groupby-chunk", 0, 0, 0), + ("groupby-chunk", 0, 0, 1), + ), + ("open_data", 0, 0, 0): (f, "data"), + } + o = order(dsk) + + assert_topological_sort(dsk, o) + final_nodes = sorted( + [("transpose", 0, ix, jx) for ix in range(2) for jx in range(2)], + key=o.__getitem__, + ) + all_diffs = [] + for ix in range(1, len(final_nodes)): + # This assumes that all the data tasks are scheduled first. + # Then, there are exactly four dependencies to load for every final + # task. + all_diffs.append(o[final_nodes[ix]] - o[final_nodes[ix - 1]]) + + assert set(all_diffs) == {8} + + _, pressure = diagnostics(dsk, o=o) + assert max(pressure) <= 5 + + def test_xarray_8414(): # https://github.com/pydata/xarray/issues/8414#issuecomment-1793860552 np = pytest.importorskip("numpy") From 44b23ac09ea5d7826ad59ee3a535a0b6835d85b9 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Tue, 6 Aug 2024 18:57:56 +0200 Subject: [PATCH 46/72] Add changelog for dask order patch (#11278) --- docs/source/changelog.rst | 37 ++++++++++++++++++ .../changelog/dask-order-constant-memory.png | Bin 0 -> 295225 bytes .../changelog/dask-order-growing-memory.png | Bin 0 -> 247604 bytes 3 files changed, 37 insertions(+) create mode 100644 docs/source/images/changelog/dask-order-constant-memory.png create mode 100644 docs/source/images/changelog/dask-order-growing-memory.png diff --git a/docs/source/changelog.rst b/docs/source/changelog.rst index e64e8e8f2a6..d5ebe823d41 100644 --- a/docs/source/changelog.rst +++ b/docs/source/changelog.rst @@ -29,6 +29,43 @@ the same to avoid fragmentation of chunks or a large increase in chunk-size. See :pr:`11262` and :pr:`11267` by `Patrick Hoefler`_ for more details and performance benchmarks. + +Improve scheduling efficiency for Xarray GroupBy-Reduce patterns +"""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""" + +The scheduler previously created an inefficient execution graph for Xarray GroupBy-Reduction +patterns like: + +.. code-block:: python + + import xarray as xr + + arr = xr.open_zarr(...) + arr.groupby("time.month").mean() + +An issue in the algorithm that creates the execution order of the task graph +lead to an inefficient execution strategy that accumulates a lot of unneceessary memory on +the cluster. + +.. image:: images/changelog/dask-order-growing-memory.png + :width: 75% + :align: center + :alt: Memory keeps accumulating on the cluster when running an embarassingly parallel + operation. + +The operation itself is embarassingly parallel. Using the proper execution strategy +the scheduler can now execute the operation with constant memory, avoiding spilling +and allowing us to scale to larger datasets. + +.. image:: images/changelog/dask-order-constant-memory.png + :width: 75% + :align: center + :alt: Same operation is running with constant memory usage for the whole computation + and can scale for bigger datasets. + +See :pr-distributed:`8818` by `Patrick Hoefler`_ for more details and examples. + + .. _v2024.7.1: 2024.7.1 diff --git a/docs/source/images/changelog/dask-order-constant-memory.png b/docs/source/images/changelog/dask-order-constant-memory.png new file mode 100644 index 0000000000000000000000000000000000000000..23787f3d56f24b4411ad2120d0822ab686612bb9 GIT binary patch literal 295225 zcmeFYXIK;6+CLgXfY4+@4V{E8H6RE`4WUaDMG%z|nhzbM_s{|a0Tl#AKu|#i=|!c5 zt{{koE>%IPL0af1?&7o0Isf1q3oo&bmiuVfB+U<|N!lja~-9 zF8S!JjvCeDcmZu4wNa>1j0GABp^{2z#7U*3K&@{+V|YoEQ76RdnHodG_*y6x_ga<{ zqxb!q1N6P?SM{4aH z$%-KDTZK_yIk=0AS`rmZ3Cei?b&($P&0dinGZd{N2OG~yY&8Q2qL3tfi!P=tX`efH zGUN=)7@G?t!dwW$9Z4%IDfvwO?$GNtXd3L1dTs&t!9Dej=ss?#_itxAO+v%utk5L( z$Emkf$4(X($7h`3;R`S8V&^jUuMt8#0#u8E5hfR)n`p+!75(Tnp*^gS;}fc*Xn=ajtL( zNU#m$SjtR8jG;k-ctgMyOi{z%xM>8Q6?s+(X(Hl?XdmtZBB%fRMTIiyGEY!WQ_UxX zIgQ9AS@affO-q`4{HN{Y{3|F!?%_7K#n2)OG=9oobguu#^tV8;jW*an zhW28t*asT@HYOXYb2E_17%mcZtb41a1NRILk^$~?ps}Ix*U^-KOti^b(4LCvDCFUz z7T0D|VU)$am=XH{*=@sqpra*9$%8d&)|H_l8d$m*!*Lg3PBR^6r59G zh#}Od@X`m`Qs8vryFy(H<}JmWw6CGpl3o`kS!kVMYL07ZgPu>kQAz6w8jQ=8XWU8B zoqgh~o=3ab7D}>iWPcfx-kwI%Ba20_ti{?_D&0E`ANMeS4~}xA#Y{ zO0V>7SZ~=K$t+db0Mn=T3yQXxp6DGRIfO~C-P=pwtbghL3TH@g()KIHSSd)@Nh;kn zxm$AAy!xVTfNj0Bc~H6ME6-xjh814bqP!M96Yt!sx2`5n8oiqEulSz)i}shmFTOpc zU*CVZeGC6)z){MO%@E5V$zaF+7spE}CdoMtX7*r2vR~rfJ@GCS zcy#^IsgJ&Aba@_R8%jOqFAn^mf0sSW^KN@q_1$+_1CrZAu-8-%P0gQj4Viyf|I{ks zAyIR8@7;r}<*YYZJr@^B8=sLbF7gKRZ)wXYGvL<0%a{1jw<(Mwk?@WnFX0My9Lrrcq){B-uJ$reE*!% zpvNf>%et$zkSpivWZj#cB~Ug5XMM?e1uVwy+QenMqDaFxy5U!QEX2>dHJil^uwAK{8lgN`Ip;i`LAx{N?hkg9AEl+cTEZ? zR&)1Be<~X&o6hqVaA=uodVBa@y*YGfvwqN{VE1eLFmh}T--R{NGtuhX&X&nuFFRk> zX#L$fxLk60YS^RvLHYD}&N#_yXF|R5$+&FgSLX@WgZBedW}cUQ&U#k331VAET9S;g zw5SI83RMUGN&aEAyg>4b`(n;)Nc)DlxMGP&$(52vdx@KSo9X?6zQcYCDuVuN0gjEG zjeIwM27C$n-e}#Z8Sp7^{?(<|%?{i(thNTWK9e$&U27#@d(`2@*kwH#Ah9cL{ITXZrnrx|DBE~D_S({jp-qRrnM=e~#OI9f zh&qIz?pNJZv|tiHZ;`}g*H@gwo+dLjDk23vXqsaN?!6^&(#fiHY%>8H|P5(KWT z)P;=95k>1we%+h2|Euwg$Ioq-zxr<9duS)ykXLi)%%m)ae5$-4C7xkLNjc86mWQpJPR3#za2jUX3@!=@hb9}i|wp>-1hkS zk)r$<{)e!k~Eb5rHD zDC)N7-7UA*&e?8JS70r3&e`Z}`DN>Rqn(8nF4ZYjt@6&YZ!bOVo;05hzI5`}@RF`a zp;DE{`0&!p_jczR2i&d&%1WM<+#T8GTuvQ{9tEl%iJA-hT$^gsFGvgAtKOSC%r`ZA zaOyB-Plu%258xgO2FXkZHg`V#*5>`u`|VoG$C!@`8e!q}t9w_nErKd0 zo0}Xf_NU&5_v~!!1g-~9*hg#6YtP2Y(b3Qe^8Hf%Y~yHQ=?@Q9*(7ZW&l-$JI~`xN zdS-2FvsZ3x^W`%C)yP-fEl;0XH`kY*dU@wFv*0!V+RdEBs)ewceO>0Q#P<1_#&6`l z0&G-Ebo&Wr!f4^`zSVxoPX5z;S@(FJ8eW6@toQft7vFy~Kz?|`!|yNEqtrdQOKx?8 z*#%Z=mv83Z)LcL8Ovqy_E-xA$;hL8_RV(8Y9wGVbPx$Wi41OIY^Smd1<*JS^vsv|8M6FrPlM?G(W4V`uwEJ8&=6{!|NVGf(uxlcaXR z@M(Cn>UPOQL4!WhfOa}%+N^2f*Y)klFLV!Byoz;-&9$CJ@CizhAMZcgSKJIZxQCaM z>%JV(6)kh*dRYGb&B^Xm^4(;5m_dIo4-If}wowH%b%{zl85}cC54y3jbzVdC!&Y<~ zd+bT)r-0qNqg1|p4|Twn5BcVohN%jt8DjXJ-@#gz^2U^xKT%UWF?+vsrzrVYYGsh7 z_h7i>FXfp61&yAQC+pEygx>F;J&>DerbkbX5r`BGHpRup$ko&obed9zf*@2pAR0=E zigIdF@%}06Q%QiR|4|PHfs$^3Aph!PM!EhzGAQToG5@?$-%bS4QU0+~&X7XzzkAab z6;l7Z4DqM5fpn~Njf^N)E3ChZi+6y#&rSOWA#IcjdS64^01${%^!G_+bU}nfnSb=g zMVp&8rsvhLK3=kp&OS~qvLRl+l-?kadWagO=;dp*dL<9Mc9%_{G?`k=u z&_BA|^w2=sm|6(w`uMvDDa*>s$|E%ygoK3D{heLaE@1TkHJtKK1L=PArmvcuTySu( zY_OuNkH4E7N>x==PF_JyK|zMnLna{9`=(=vjCX+WKPUP3JQ$Y%tp5$)n>T#Cg?`WL z=;RZ4Qv-?oz0jZIpZDn!a^vrnyaWFASd<5p``sdkl9iYHGdE?Z`tMpbiyI*>p0=19 zUX(qftf7fQDXOUdW5EBb>F*`~W2nu43{_VBzYYD5rvK;AO93wax;|c%MQ>{UO|XBh z`=5>f8mKP!``Q0vFaEjF|EQ(xv?hbP+#jlGG8owz5h#S@yMZykNV!sA_WJ;$bAac-kRrgx6)E#smOAKkcbx%=(sl#Af@`N8If=Rpsja}QLQ ze<{57N+cB`B1A=Rs10V#i1{CiuXEdk5=y4HCoKQ#z5SkV3rdwxDnMM)75#5d!d|rX-&Pcg6+jdMUxaNUOaF)M_344l%unk7`UDziZNzhf z%6*If9iEi=5+ox3+lq$&Kj{8Xc>ezsblORfpq0_gdans<&#$ioZyfD!-?rGd9O%>+nY}VxEbgkpja|!%aKSs6n=Fg8# zWpQL~kFUoEtM_&$qE8CSQ~59PcqnYfh5kC)oxT?uD`=jlQWYuV=vDQfe5tU40_p8- zs7>piP5!Y6L7&lyzf$XNi2@gR43`;c9g=im#pn#=4CYSY+~mnkR0escYRuql$*DQ?S8H{Oy6pFS6A{d=!)9*XRC>6k8C+Wx8>TQ*OjIa zKOTh;Lmma^Uh}zkI&?}VXWx8IKRX?v=x86c=klf^jYsLUqk%xEv)W=_z{gjM`F2aC z;-dn``(G~GKRG)UQmlTCr_r91>ejinMkhL#<5BX){CrP~ap441+O}GxnKZ}vvy1Dx z^qgY*A|i?JwwdPMHz#QMHv2Gk<$+I&QAP94l`3vUS9ZY#rNf- zb~yU;mKPSVj!q3{kJtOi5zVTzfC9~fm71-+#R75MGsq4KQ+fr>PBk34`?UdrfjxrW z5(xZksdpFIYtE^CmqIrcjja#5KdX8KQX4+5jMaSle$PH%wt4I0&hGv1jMZG65hR@} zq}im^F)TD_i@WCW7O8`!B>LoF`sdf+X_+wr`Hm7m)m-i8jFsn-AdyDgpT25YS+$4&tY{gRsf9%NKN$A8-uK~^CZ z^NfHm&oBMdI@vRS1L{(?h+v3b9C#?Im{7h;hD1ZDsx<69N4P4Q%$3;-4k;+=Fv(zd zJZ=1f-qJq$C@^d`A&+BYF4e_Dwti7w+jxY5=vKX-Mb7nOcF|O{n)EnLsDpX zO(gQxA53SoX;bQyGW7fE|FHPj>&Tc7TU}iC?DK596|Ki%W6J5~q#7k2yVWXWu_XJFLZ^esY~cQ+7M8+cIA)L-)SlV@m`Qd}_Yi^V&v_ z^^RKMm+<*o%d!+P^i}WU;(xrMOd^yz>KnD5FaA8t4@WR$f6Uc~^5*?~7(698$A*M0 zW>l7sZTuA~uyR{6a!O0MW%|9e zCUY%LA1<72l^nxX(b$%kZx!)R_E<-FzDX1qBev1J+bAiZMcT(J6iUaUepe1 z9Z$(Y9M?7xLPkRg~6;8hE{sSE<`kQPN)y*6H6Z`kwD*{62 z;#Wim5|H5#UCi87@p@fhoAgY2jzx6fXP@|I0CmgB@j<6`81y;`FN>&xk{|J@5_u;>L# zavhRm@GAZwHm=XRcYjC!ASA7xMsWO0gbH|$)Sl}0*{S;=Q=>Bg#wT^qQ4_~(Z*c9Z z?lpo@<#*e(yFd{s4en8&;2dt4&m9rH`(~QM+kh_=^KS35)^g8IKh|pD{n0oEPCI`41WRt5_M?8`7IyRxoQ4I!!Z}}l+F8r3}H|!UzOp3nQqAoYyI^JYD zWOz{)z!i0GnXp`Bb{*T)cn{Sb(wg6{-c_2qj~bo(RQssrNkx3A199|INl^4??vZ!T zTA*yV5?rr!maBz0(j++2hKRWB$2gH$@Wx|tZM(oDM0vL|n|WYxM%9K)ZEtnDz;3E^ zKWWE+Uu!c?BkHm(o!7$Nq%Lnq&_ad5+6LXrXP3}6B`@EFk@H(F4j`5Y$1}Xdl--WM zzRZUIEkLp;RzuxK_le7Y^ubClFy11MzU!5`PxH1l3fJJ4AL@$2?bf)yH@+ioZWc$& zcJa`t_SyE2cQfL-mT;YZdNwik)bJ&aqEThQk+2v+Q4)S`4r<9!Y1g|wKwg8aZ~oYJn-!Fp@7+EQt;u~yMB^e`9yir=-n(6aWp zdPH?+Pyp0$>z>eGpNYyT&TOXl&pd-t7-5J5cM?>kCtyBn1DmQ(@Kj!!@;N#BSep4J zZoks;pzC|zJub5dbACJlpXM2wv~j)oNjA_m<_srZTY9`ERneqyEz>J#$7G+I%4-Rm zR}_hlg0EM}h~!?ZXxaY@wXfB13XxLvG@TF$eeuG&{Pb}zyfcK<1jIX7H?B6=A4siM z$dWm_12#Mk!@~yl+qE{90rR)ko$zUBWzuv2E${Bkpl>1n*1fWBQIqW21KPoq_NyYo z1m(G!Rpe%dbj#8ybJ^-l7xw;W~zu5v|jal6;DOt(3exnl!uU-`S_7W5AR(Y-Cs zQBD+f&|^2S8+Y6}2&9k&7>Nn{WBa!QMgg5DqmvTOc+kdM@dEbD+w~Y;6}I%i#Mf$e z`^Nb8vm4h4D6B@YPgOoK#RJ{|Tym1ZzIV{XdUm`!$nuaDbii-8X*wW8F4*@oVh}b+ zu_SZfZo4ww*&6;Tm!=zM11o7~7|TM*{Q7T7siTv(arN?ARXJ{lvRKpdLv74wUd@5! zcPkTpJ7r4!=k|?we{gNGoILsZx#wzo&wLzbt?;3G{_5ZI#QhRQ#FVT3+Ohk?o<9p| zgAOTbs7&`O3%JgzM@4fz0LL-zcKp0+WhSYw%stKa`0*4);#R0jK@!NF>Qp9;PL^}IQ0B|_k%1^ogT#hW zBu1iB2E6NBQ}vZ^Fx^3ZP((XjqC3P%JU_$p1YIg5+XHt;{u)-sdWoKt1ptpC{a(iDF>t`5NIt^eR2vt`rp$`}*h82gdeJd_}VG zh>b3;pM7$z=jB?+PGJ<$Z8D=$U78;}9zxjeE8Ss&XfM3fmR7eG z#(0=0{C={h1s8S$EQBdWf>_#wpfw~Ae2BG+6Cp{BQC!A1U)?_*IZF&g?pS%t!0DKY2J z>~l=Sk&=R_*LWs89AZ9yD6=LSWBk=p%WKBgS&*byzuM6olB}HZ9n6R;hGbRs0tgcC z$c8j-y*7PgKG!+Pp*CG;iE8XQb=%}gL+g>u(b0OeoQ_R(KqYxgOG|u|pbfSM*VGm) zvN|k`Av%W=^^Ii^Dd)uquW_Q_ua_=g%ld8J?08m`(%FX|tTvdq3KIRgSsyuhb3TkZ zn3(1~A*MTYhi64uL)CXNObk1W!+8ki3=?A^~PEvXc61zjEy5p3c$h=sc zX&jZ)+b#98z`5Qlvpfr2-G|sCv_(20|H>R_9)9G&-?Tdwo?6KpW$yaJ*6rb?1?}3p zp^+BWBKU#LppCfAAn*w7JFCvYCO*ZttNp8o9^sE3NsWC zloT+=@w~&=;Ym5VM_*`W=@4mdHBnB8)3bN*FV)Xsf*1B4zrFMEo6(&DU8 zj=#F49qw@3+B(E!@Dy>B!4(fBI%W5*-liVR#uwBK2roI2g`?hvf6=^dTVikWZzoQy zwGhShU?}z!{NvJVyAA~+I-oc<(D6v?@g-asWCyCv+Uder+w1(CLt@f|ga5cui^4^D z3c}Z3IYE%HtI&mR@x8bHND>~_)(uVZ>DRFw5Jl?7fuz6?Fcr}*UpM0+0RmxFxP+AN zQ-b`clJLEy2AyG>e0+UeKhjfXmtrMl3Hdtz)fIH{s)2>TP;%SpSVs(7ycC9IT+bdz z0|Wtt1VVtMqzOZ_dx;mO3jw*zRU91a^SrfOnWq(Qx-%X=zffMTi}^i@ zv%LM>Qs9rD<60f$d!Pixd-Aw-IMtFwUAzQ%6Dmjg9wl8J;9%;I=cfkQ_kU%Es1VCdEJQ=gW`j`Nsk*G4QX~#FBMJ! zyuIV-&j=fH`$~x{G<%?e0U$d!Mi%PcMMabJBJ#t2avfo#s=6ewXnl4M_$IJchpB)6 z;H-X3n;3{I$>#>?ea+qay4SK~D|H^xng0V%8iVzT`K)5G6Zb8TI|+q{)? z7or!l0}p_svPOX6S)JJEGB>CKS*#zc5hthF7EWedW}HtN8n78W-J?4GcHcCSXHIR7 zt-Fz=z9iU5KE?A{)6K>}x8IpBRBV%~8$N&O8VM1>5#`ggp^sWB^ddyoN*&^rSTylo zvq?<`6Qq$B)p8dZEy+9y2kkvDzg)Zi8eo=mZurdj+Zy@to^fOTDzmw>&S!4Z8@_Rp zetxj9`?vPV@mu?BmzXg4Pwf+eM}Z?Y7YF5Xp|BNE_vo&y%a8kHn?2>se;%eE+yV6D zAE1SHPL9`4QZ}B2;BG;zh3DfzvP4htBN#yhWCq;PmIH(Bln@ty!|vXXLY1Z;vdc$C zacOQ&5U0VtRn?z)HG?wxn0!9@3i=|{9Ma}!dJR#J;)IOCdUG27;S}&^ znu5IRs36}zr~>*)-+P%tH{YM_YgQMpFF9OQ9b*aTh8J1i1;l%!=xH+@(H8kK18~Nr zSf#+3b+W)j8-jtD8I)OvYek&O6z$%%xoeC_c|~(B;!cK4W^9=^4eJ*jmh03z%-%5grX$i397;Qg?I+dcgcGEW)3o_E zSC^V@{l*~oE>d0Nm^|M_@*57^hb^0sRhF;x>+Gi&1o%K(@u~O%`CI49-)Yil+5cgJ z+ENsM{a*=|_33}(k{TRA+ayEx1KI}r(6U-T@$qD#d>I)q`&qT*sfldMW9Ooqq#W6! zQZ0H*yfj3F{>l`E{pj5legK-KXPl+hgnXg@;jf_&x)&o(df`=!^mM8^t?e z^%vsKw$sI8_jNM(i`VC2w|4MgNT4nrS=zS=Z9%FD^};}6+G#|p)Ivr30GVCVKa3ZS zi5J8;PcbGPDjmEhs(zxQsI@k?Us4*^TbaC)+$%En0O$(Qv*5K%j!}#P(Sc{>JBv=p zV4?>TRpJUmNJp$k?-KGL0VNI{JfN_|PKQp#@$-@L23H>sk_9wV1`~h;!}ya~76cq9 zOJB(Wp3Tq+Qy2=@qFbgH>pZ1RBF2|9p#}KR6O6{AndWmp!#14`1?)@gKgas9L?!jc zL1SkZeP>#Lzl06#B{8UbKg3(!!jV@qHxhE<(-or~bP8w})q-6RDRoR8Nuy3ImdI`` zh$5zKsXfrA^^PLPbW{mcsy0_M5tLFeWUmrfzf})(@pTJ2zOzn(N9mp9wxYcIKh40G zxmQs6uw`x!Zfi-t_Ov=kE0zy_FdKT+#(=b|pu873EW{McXwN*vGMgUAIsiRO2&_$d zE$NJ?orFS(;1#(@M4TFC}4hM=ohl_&{=-sEI z4pzTgZ9>9fN}~(Og&R$wo2S>7rsRIJzKqNnO?BY=j84pL^Ex&QLO>KWKgj~3+x`SX z6y{Be5suwJgoA4!*m*aI{+RVCM0}yLzBO|$7amk+3<((fE5&>oi_jLjfDnOgVVn^^ zj0*3DyJK%mzkH}_B;VjFO_mC$Ap$oalr+{=ifuY(HI4@rGsGA5T zesj290qAK!p2CWuudz_2%po?U;})+sq=Vb1-mDY8sTW@Vp_t{jg#)8P8&d2Z6el98 zng)S(fkuAr+I*JkDM1j@pz;O(ggS3!d7=i0#Ks0l`$n=x3xmPF{z30;v`FJ@BJan7 z`3A_*T+_BBKzwsG{6*wN)iG=)^<%;SFH3w$#Qc#B`N?JS>G8@^s2R)pveD_~yP48j|XPh|LZ$7iYs?&x1WyEYq&OWNd)#ZX$&2%(lmjl)@%1}N2e50o49qh^13fE} zesCb8@gB22l}^{X-2k~~>4ej38sctT*4|l$=Vbnd9tS1PQa4vR;r2%gTj<0HU2&EL z`dj92oilEeQ}Be^ur`XuOjtdku)K(01%;>Q_>k=CPzBae@fs0ugJ%FIjy86vSFDt& z>6k|jcbwOe0OuHVc!L73;w||(1{`NSCAv>z=x{53x46(ek){1M_{GdBZ855%c3l>Y ziv%GBZ0ZL?Ux8GwK2jP`1g-3@q35)}x0*BQ3Vs1`ee>qKULWnRW00y@w_#vTY%H3;1A)1Wh zCl#-SP9YI}$OER#`%1X&Ig;?HlU!}s zc4vr?Q^GXjx}V^xu!A{oaU@0W^I${LIB}pw5~GDB$EMms&wXuQpw&}dT6S%h#>+r{>4 z)z3Ri3sd_|g>HUKXnjL+%MBlrUP@KYP<XGgcPIZFo2@Hsoa-!Ix%ttX7Vo}(e=gl-P-XJlE*S;<6 zV8$`2JV*2n4+Y*Pc&9qt6UfcC`a*s2t6V5tNhXSI;wH@V$qX+71ACmVn23%?7}BFkcNWjlaCmaI)loog?U zkC#}GPc+2vzS4qpU)jA~o=|DjjOT!SJh{19@bTo$_Y)2TqZTrxb*6*n29b`@m>=>J zg`w`$I!s!f0@nZfqNC5&tuhlpisx;YjSZ`p7hV=QEWLntVhRx(y9Nz`JiV8JBqpK%(;|6vgOHC#|W* z#hbUVk(hN#6||>05GnN^#5BJP>RiG-gq$A>ds&Qbg@W16xIB;|IG~6@;ExbH6!A%!5{M`lT1v;6k71d5>#wPX>l|!7_h__p*hcQ$Nc?*9lj-+yN}RMAGHT!)`{N#Bf5qU;nRFJr;`aqwt(BM$*i*B? zWH!zK8bO0HG*tgkCj=QzH^lhjGT@Yo&etmdq{cWiRyZ+67^Vf8c1BzEerkvmMxxrs z;g)Zkiq+YPF}(v^xZ8E^5?~%sa;F|V1$#P0$i4L9@{=qn>uf)$mKgP^jm#07l2HNn z9jHnfuk$YARMnL5RCi!iHo|cB0PgkdOvEGd2LZnnRRc#^it&Ej4jd>UPUrZ^c}RP} zy&gqH#H`qjcivAV3t~Ie#Xq_K3^w_c>z0>0pfXRGI|sc`b-k9T$sdgWOl;O4h!S$A!^W9uOSwD^@$2Kdx1>hZXsot6&q` zM0GqdX-`b=$3}H7#`h*l1XAG=q*6-L_aCfey011UeWh;o9@TYhO zHKM1&`5kNMv6JT={kEJK`Pd89OwcAQ6v7_GNeR7MgFB%p{B1}REHLc(O%Y7y7fd$9 z))clS+Qx_G;pTQmZd@mD+4aJh^g*_{)Qd0tXVk%D-^W^>k#xHI^q+di8;@>HeDZ3T zvObL2Oon5-mAe~+X)`0chQn)$4nWiH6`~n{5=xpXEv&n&FY;8P6EE0~z2QJL7yzyjC8chS~ zc0?64gqw+>Z;qX#U*>Vuev0SS6Cy9e7q?7I5B$&P`Ph<_>$CT_7;0 zPo6+1f2YQCbl;D{QnrN`yz@R;Hvn0Ya8$M`A5L|SCyElfXFTuNs$kEH7Is5a<Ww_tkat*}bxL{L4dGq7TV0UO$2_y8cgOJJy<#Bljhx*t_6w zBatH(5C@7D76~*8Fth~%Fv+H$NZ!8q(+ACdBx1v3H=Sv!?F&m~dNAtGK$Sj&Of-bq z>iVrQ#~OmGvZxkbLM(;OYQ+HV8A z&C@wCFVbw_v+6~RYC6`d@*6YaZKV0b&1mKegS3y$VHX>q-W=QdDW)gO6 z2>p%7i0bAg*tPw77=q%s*Fl4~A!0CUC@GVQ)%Gex{skkESf_QIW4$QT2*^PpY$2i~ z+g$pNlF{OL8{IZ8M{1qZ7^Z43Xui|N&!|z17|3nf!4sudSgT-TT*%w)NzJKauc?$C zqHV8sI^tvgPJmqC9d1m&`{9`e@dHC8xu+6Twq7` zTB^1Q=3e^kDL7r-2;4zCzYyJ_sOj92)QThCKn?(3V1}c4ZnG#`k^3dxP?7qLE zdiXF5hRWH+^K7MhvE|O|_{f0PCU9uumZ)`Uemn4&SYj(yroWexcx zE=(;TyAi0cdOE>msqi9$NM=H3-mO<{YVBd|YTV1**mP*mYo2B9b~O*~s6h?A-H+v{0=br7Cm?HDpVTIwiZXP**+6uwV~k@ zdiAX0;3G-ZiHc&JO5dw6s3p`E??>4dg%aS)XyWZ@6uc#=5M#$9Guii=kkd0-`B12} zLi%rQ`Tt;$QKA}eoe>O~)P9SZ1_A@I#5xxbMI4TActDjKB1msG%nN<_eBp9y`DHxv zAj!NpdiQJa_cFF|V>`eHdVy`rnNboq@+Dz~6Wk9`fcZgb+44jSle@A#v(Ojmofkc` z-|tPS*okz~?raPkY% z0rybQ4LU@884`n}L>N33hDsTU3>zYlhx$FQ_rvNzOs%>ZL_ydK81q|)WxV6}nw<1J zM+_(H#DF98Oj#Vkfb3Sw%bA%=M?I_2f;iOGq#H!QMY z=ylWeKY_-rU`lBGC^Fu&;ExjhLk}EL^w2y3>wE%)h=UAV+p;cSw^cX~{UEIW<=xBD zFQ0)|PN@nn5pXUYx0VVDRD9owkJCr#%ijwEBAS1P$L;9b5;NLsiUIfUE$=)}<7)Wn zE&Y7TFJpwj4R9()+*imr{U}?nvxoG`x2A0_Dko`U3cQW+;c(*DFohzOB$`LKHXpnb zo(Inf@t@Tpy_(u^8qcij-DtDgWx zWS+Ryw6jw-3cf(}M!^+6q#}JQ3xe`Bqm52CEC85JQ+i~rf6*;o!o~{Nn;M*qq>|-%k*=naA&%egPkkohrinP>Zk0b!sFfF@q|-cTO3sdb#?iBefx)coC8x6 z0WUc`ls!NmP&5#g{DBMod1}9HACEd~(`WN(^kv32)2tXJ3ax=nsO$^ur;iutlo%FX z>an>~+L~=)qA}TXQ`BHIFRTOPiY6a@8Zg$pqzIO9Ib*4JYAQlw=fp1rONN*Ges ztc@8&qT8D&YVZf+9rlqSO@eZ!SKmJ0vayU{#mj>AJ&h#P;Ktbn!{&L*<It2Y+rf+5CI<{VeS0SjEQ3V$CR#Pam&ySad30f~g8T-E~&&bS0Bz zMwvt>+b}3!r`7oBwd-ykX$Iws3UczD$8fn}_^{$Q_p+^XN~UG^0o>NqnU#1~iD`Zj z@7SNU)PCjs15~GM)JR4o{)ZLC2Tu2PcUHq>sSyPHb+~aCT|mMDg{yc0wVRqd=Mh0)Vi+&7qEL&e8P zP8h->Fo)U>$jzb)4ixWN@l{@;F+`wR5jq!i@dq980+UYajAX25y_s{kT=fO{rI#fb zi>wjj`KeeTjGi?Qn%Zs)p&;b%u|Zxa>>TGPm7A6^+F5dFU(r*UQ%fUDQ1H z9Vt%JL1)BL@>hxrkiVP9eE6bFRGDKfzs{*xwpakgIaETi%A)Z^{JpY0ye^^|p`b`N zHdXc7b1>j0b|7MImpXDLjuC2)*BMm`EidCN>IEHd)tG-eSZ^*EeabSw%YQVx%ZQCj zD6k%}J&LSc+~IN$%`URA{a-V_5Ai*AXcU0xYCoRrm4aHY_VGK>z9DYK?q7 zQMdyq^T=He$x=|-^TPlVwRcfNU%TcC9tq_ucr3ojpzEAc{uRhc{O-mwFnCKF`&maz zWyEc|yRAdJM5?#BVAA5;|zC+K*hph=V@`p4X=SY-PK z>E&hoY2cTZ^A#dD6$-+QVaLGh{o(|p*yg))Xx4$3?&AH=3P2iO#Sbm_-vAuwQxnSw z3!#g`!$RLOGSNY7I1tzvDpW?*FIo-c`k*v@ou?i~6KQ^aQ@4l7jJ}guunK0htfgKn zGeA(XlWI9EaelLITQn#xoupT(c(ncNnhdXa68H{aW~+skPoU<$g?Bk^Xd((93~ zsFSgxbk($~fp~eg7C5#|!*7c9$W^0w{WGzyYJikS$tOACc3XmOBoQPi0>uuaG;BYt z-g!mgVXqk0JEIC>z7Lz{!M0uD%eCkiuWolYuLHW&9R)U8OA5kXe~!&DvJc@5tl~#F zMMFub^~S^U$hgtX{Pzuj^=ayT%=Sa;Jh3MTO~GCKL& zURHclzp_qD51ZT7Z=y{t{q%RH=!er3H{KPO$EziOU{)Fj(%FROY#N1EdqWvEjr_og z5J41}kk=Qh2g-9%zD_ zVYQD>?I*(6d!QkS+TDska|rE(jp;zau$IDzkr=R=77iV_05xV45ha;@2>G&fX)mHQ z)8#z(W!!t{X^4g7rN43vwQ(k-+jP zY1^}-f2AJ)HfX4GxkIEwnUh9w!DPq2AdX{EcXk<{K~@|5zLyu=!;@-WV^$q>$EEwq z`E&9%zhdK5N>bdCKEw%<&LyaJimZXzxlsqum=Nl#>Tfq2yxCyMK`Fy0ZNxV+XZYsWcCF4by?RNI zY7bCuS@Vuwdaf1rAKwqjP`F^T-gwxbP;46rsVy%@0jm&OhKROlC8sdn7DrN5QAN-J z^ho4=UqO9q4*d^zg*0Qe!LyJudNWqvGIm#rAOs_*qBP3~N`~pXFR)O(?|mmEVHA^9 z&-x7$GP8)-j~CL`dY&M$Mz0emB&{!h7W@#=bSVY1p}XsIL!$1hAi@OdAa&(}Cf#>5w#03lPr~!bBGe+&L z^z_DedOr4@Crdb#QeGVqydU8;k!o{3)K=^<$8zc-N>seDP0)fxNh?XR)>2XF&v;p~vBLj-;Yl8MZC%-$=;XrUy z$+trJ%XA&UbQ}VdZl*z3RHoyi6cgx@Isu4eP#a&G+X5QrTvbmkPepJ$b&DSp*@)Rt zV%j0Ck3}%ir0fX80vRW;fzwag5$ZyrUCPDqyezntFq1N?nTQwV0uA8Y{C3WQA0;&P zv=zgL)@9(G2+auZH(KIg5;ySm(Kt*jHUDu1>Uzj2-rOSmX`*MFHjsDXju?>>Y8XG5 z5G<87{k}-?#Tx6}?AkVq|LWXYYGQd`1w5IB7j1MihWU#6b>p{@=`tL3F}IfDFSRod z68v=ZvOq#VExg*mHORWdhWKu~1v80++9JM4L0a+q;N-Bc2;aEk1u=20XlP&|B^pA8 zA6wLWaZktTpx37OLwr29zEN5`KYpH^@q&$iimrv>7vrjLGE-9I@D*iAK|)&@-}$L) z6aFZK?<+|$QC@2Ey%0(Mi&i+zd+EaHUlx=ljTCY;S+o6a6Et>_f^F=6Q;?m@W^$T) zSu>-3VC{wk;hStCLTScy@o`7qgTz1~rnBriJBzM4w?QXgVXS&QW;PoleW>2*CX+3h z=Zkkvde!s(Y!WHV8(zj!4KELlkPjq8yxPH-^uLlO_jku%M{Q8glQ_lg=9`80rd=a8 z+bZKK3!YQni8!X&r425)Sivx&sXOm*WL{vk6JGv3u@|L+5=KJgN@VY?huMk~&rH#+ zyB3f%sGR~A5=o6#F0ew07g!dQ7f79@*a}w>W=r5JTZordm8H>J4~DE&4X2U(A1;}o(s#SkqaJ3r7<3B<8(VE>nbn+?~w3svknS|x10E*EE zM`BcTZ6Lgtl-mDaay1fQVPj1Uymgppd_H%gMlPYyv5NEV^iMdler|H!hW0tsq+uoh{Y z2;*;R-5+AAZQ&55H$46`%cKIWtGwD4sx!Bjz1>aRkHGeRw;SVqW` z7TLnQMn4u7V+xtHfLbs=SKf)q;G+yWeL>mMf8;FT=cPNyop#?OK7*}AZ^C)Aa+ET_ z99M;5GWa`vMtCU+yrca1o)_$>yZo_CMVzmI9fK}tU+_LUOQ9DCa{q)S${JN0Ae zf>^1&H^=fL&~y2&=3=N(e!!6!E23XJ4WsSA-MIIFx6jIq)`X^Y>;`vW1a3p3+1gxj?Vp={GHoJFe{U;dL*m_g0}Se{3yMcbwqt67 zKi(}-ZM)&O(&jElTbdTsg}MIXg&&Iz!a^99H%D5Mb1q6Z$Bf`U^PR=Bxbj_PLM8Zy z4ezeC{gvxTn0vpgx3Njg_rO`+!>_WF4?A^dkD4!aOlALS=tx#j)jWClI~TwRjSbr} zp*GdX?DJbcr49EY9Y2EQhZd8__JgPh(hpaT)TMZ2P+^?X?47xFMt^NNsQ@KOk{@mQ zv*lz@FDV)23vUq;*TRe#pdNsFo5s<-mJTHto4+6KEI$6V#Be9)BWE$cZY9%TdYHcD zcf~^%Tqy!#vWgo5R$Ka8U2h1UCeRM3G#pRJY=A5xqr2e*@XI`az*OT z9b5nCv4IqVa=|Ow&`|hYl@dOjl_?ot$~n$7GJRfX;@P;Wl|7P5K5uF5-dOE|*qA7XX{MsBpd*2VaPJanBzoQP=YlFRFT4$OIr%EKf) zo^;aW3qhquU$2NJd~bskbq&m)BnAw1*0pU%uSvg2a>7q9a63~qYYg{B$X#~|-@hgE zJp0yG=JZpuPxs^_?`a$EO!I8DeE6<-5|SR`dqqDj1j%i~4ySh6^@yr_yG$y{BVKw{ z^V$2N(d|FGRQzOMu578l(&5iqG7UcD>Fuw<)R1)CWsOBBBMEO{vmx#x{v$IeIb(_9 z&TC)3ZRqO);^b=vj2FUX=T0D{^Nl^Dc~rb1&1_YcHdp-}#wXGW+xB7KAoMqtX@0Q# zDf=%~yZ|HV_nzLZZNLb~Jj-xdj+*v!^{=EpkM!j-!*vPSk)w?}&xx*I?@3ZI@7$!E z!q2#%C#YcM?Rfeq~HNdq_FpmYFehJw{~IvU+1U{NO!-|nq6(eFNYIb znp#_=5h1C$(^~n>xdL@jCBYh^B^n23A58>UNZ5rZ-A^c>FF zOzDg1DEr>Rbp47xrOw)^Poxul^V8h&>LXKgg9F2pk4_6#l{mYi#VDJ5JJBCLM)Q&) zYe&R}bkwBwo`Vc0oChj>!yk8?7d6A{Q~y9-Z1ZdK4poQPSZmDoccga|@-eA76zA4v z#4*0{S(f%2^R1mvx)`&~NB5(JtqW&)sq)y@vZB`9AG2e!H(}4(Z)W|hVIDD?xPg&z zHox^uY?pMwYUuNJ7*MqDE}po?BuJF^R}Y?#R$s{t(lAzHusr@5-nU)8G7<3B3T?kD zLGF10qmc!Lwe}Of{X&H2Sw_UhIn&e&0sXVOo$trvt{g@G0fHW11Q42&r^^-jM{()5 z#;N3c2tM>;W(GcQ!=p_9VklFZ_sMGPz2KLAhAzWet99!|%Gd>{ddU&Cg3C%0;TMwI zVYti)`at$Jl{PvfnT-iA+GT@E3RB|#HL#JcjNMx9&M;e-jo3XHi;dYp{Do9S5(D&n z?M27PKbS%qCFyaA5X@yq5O~+n97QI#P`;}pr@k>&l z==VLBzH8wj8ljTabvH~U5Av>yC%mBUZgp5e30U$i$5*%I>^VkSkm&u9F8JcQr-23^ zhQI3aAJZ)VIm8Mc4(L{Xrbm>@nx@cn#s{S=w%3|qR|BDM?>eycrNU{}GCT#5i7#xi!y zOVr%v?-V9}M)b{;_B6ZNo)#EUz(=qOHVswqA6O`uqA0WEBH2C;VmD?2^So|vz3Paj zG4>uIx?d5d3K^?&v4JL1avP;!N59SR!yBTH0xyKdK&hD8SaDg}IvAqv!at|nD+e3h zbo8o@k^fH*b6tu?(3WLcN(^lIV>jYi^ebS(CKlo_rKfUVQtmm0!Q`F`YH|~+QstvK zg!s@#9-EbAjqu66(tY^28cZSP{EqX=m)P%B+fAppH^$7P?|0JqctX*)w?*!4ioBSO zWf~4zdj99q;!DW@kUQMKU0VFZ4gY5z<%~UP8?Ov2qi#J4viG*4=#OvLsoni+jcf2M@HIU`08J!x*~`J zHD{;W>!UlkCFuG5hM2qa4PLWA-YWa7jeyJ7u-&ISdR1CDK+TL_T1cACdL0@927}^( zSSDp9Ir8)gl$v#$4XI$e{2y7>e_yzX)h>bD!echJcCn%Rhl!8OaHn$BbY{V}+3;>^f- zt2I>bF&RDuwcH^Mwfn$1DKW@LambrG>v)OF?+r-mM*gchHv8W8g~;9}cNCnrL#Yc*J8RUfpu-0edCip? zBAv!LuRW#!;k1x%;6MY%0|pv{DI~IjLtr)7$Vlz(T%a>xPtLbPymPA(jHE&hCQ)K*BiL|CN>4(&W zW;NWS><;X;116yXuyY(|56E%$29&vTz(%#8q0?u9)t53?6|q{Agp`?m&=XEn&m+?F$g21h&T}_zCVHzK#a6h!=iO_;AKV>wPrG z#1K}%@)wu+r!cV=w4;n8i|V11iCqT^uo;vYsh_$l{(iRCxX9f?P~b5!}Gh# z{6AMC!H@yj=fCOA;0HLsl;puv{{Ll`GX%drEZ{veodV90>}0SRhkFA{K*Yo6@b`%S z9}8~AfOWoe;*eQ|46sr)0A-~Akh`auoc!OJus65HpbD(oG!?Ccj|peMNkt z`Qw-Vb}RhAS|nQMg#HlD@89&_PUQD+D*%4oaAA4+vmyvC_d17-6f;b58krDCQcInm z#>cBHuQy-U5f4ZtZ`o zVA4#sq6lLPd<#j3WPE&%ohG{@cLB!Mme*4(lW4v1#EA2sk0I!T{0CpcWCp%u%D7FI zm_E_{>HgFkQ+T2`BCt{OcNezD3ob_d)`j=m3E0P8EE!`V*c10lgP^4?CaUc8CjceR z1mK97w~2_rn$i~mKpoN=Ldq>@J(L#+Q(g2rCC{o09;yE$L;d4AQ?WcqthO^l->}-{ z>ARVRyTaxQUyXg@nA2S+O+wzK?=3gi*KK=o)PQ1>XJf5REe6JKXN- z_cmi0oJu-+N0c%;@tx}-aph^)QNxJBBmDSR1nncH7WO5n)NoCFWSF{`V1;;1D1IPy zJ6$D84b@=l(QSij_*4uKT{bdHzFy3jEV9w?&oboyX``Q?6aWM!5w8k241wM>_D)VP zesLghPJmGOJ@fyzK3c4X1-TcU0&gRj{lDp2AYv#i!H29dH{X;QYKJ_RK}{sh&c@9$ z&{$8^ILbzt&9MF$!Hlr~^J)y4z}0B^&&>Wgr7tX%Ap$6Y+PSA))<<$GZARlq=*1(c zld7-MlFoifpaMm3Qq6NQ8!Smjpv|7le-SW-xl zltN1rpvpIN7c6a9o|C=M%?odpDI;V?fR9TGW0)j&M0Wm!R9k;_Ux}De=&VAl-3;Q<8bI5tzyQ;iK;mo5m#Gi*8EMH zpuC^!L{+w5AtppBRm-hom&weVUi{e$=MvLF@~iFsEwSpiesC)P=4 z^SdaIm8k#1WR)Ge6~a-Kd|o5vp^Qn2WUb%#i_37q8oFzeM~kEe5w*d*orz!$R%Ulp zCdJ|W={MWhzdz$0cJTR+Xdj*qW5I-B=5ph;$8fBhyH4!@e_P7#g&m8>gfw6N?|6W{ z5N|n2*qYVdGdY|fU+LFTAW)FwG~PxmZV>YcenJg=XZ+%V&1&iGXJ_I^3i%oG*AZU+ zfB$}k3>#p?bsd=cH?OeHE$V$^tYj;$yDTcK@FPfsZqYfsl*D%4D)Wxg{`sG`u|lEs zu-W>6bu!yyV|)aAY~YEE+U+c_tx#gUq(kSx>lhDMMfUGHD6u9R@%U$;hdq8g5zTWyzb(5RDyDlUK% zP*Pjev-r7#^9{>%+ng}?73x>#;t^4jFEnJcYUF0W(|8-N({xL~TGPkYwYowPmaorq z0{KV8maN9QYQfjyd7oP3Y{bFu*Y z=QOeU2CVULzZuPPVMpw<7}5PRG5S5=XX1g6t!$>BlqDF#dZb%QMuT(&wqqqQFv4cI z3XriXNF7!I*9^PsMrmjKC_t14osZm<-vD9ZWs&~f*8mDJ8u0Y?GOGc}C~!K83b6&F_$sHMBt1)n^oQmH6K8g0Zu}9_C@e-dPl8*i@Jn#T^ddG08_4cK%d@F z47dAfqSoM&4h~A-MECC%xGLd9V)ZEHVfk$7i`}JXOOWNO=YTwM6%-CQtVlr;!AbkH z`QklepD)Ef9qU{-Cn^9HX9b`<{2{;b@+@Q{(eqJ1;BtJ`5pn$$*lW-rhGNM*p8#)h zOs0DrJ)0`hdtDx2b7cm(2n90c`zo()tFkTAM&7)~XF6=@AUy=!;_h{&NoBEcDsENf zcn(DnmesHvSjnJhaGYFsWG+Iu*{oft7xxpS3E7st@p0l#=Ya%dN+LDPJyEnGN^~pA zdv^a}7R@p0dpv(UBm%3_d&Nyu1} zxs_i16aCWtHbuAFIeSiIe68 zLtq6%i%ePhLnvwwq(hXa^WoeB3C7+@kW7%LCU`q_IF#YX{W@f4J)QO3z3;x9>NX%x z=k+@HDs3_(9^xG(`lAcJ;`yVSnc(#2>2`Q?rTugrJdZw~ek9^En!+CgC^ZZtJiwIn z>>bDXHsgrynLw66ZiGN!#d4Ylm)NhfT>{6ojdF}CmUq&4SIjP(hw3P+w6Ktq)1Zb& z3=oH|7FN$F{l+4CzBf1s2o4899VR@FGE(X(nil_Qy7$SG)k(lwqk~|bk$LG?c z%UveDA4QYY5pgqF?kdfL7jKM5vEWu!-Duwtgn$bty?5*Nj5LEJ`gT^5n5Iu>}0y@yuU>bXc55LZVi0)WC?)!FqZ;&~AhhYO` zb869I2fFTnVk$f%FwgmDF61rP1}*|4Rw95))fHOXRV7_8o;qIMh|gz144e1_oYq*x zb^f07kekmGx^GHfAcES0{g~_W7A}RlrSRXK5tsBp<*3mcEYT@e59GtpUg|YxsePPf}PoV+g5qOp)0cmigX-$sZ9^#+wI$!91PlNY*;H)8e#iUc(H+yXK-Y8= zVi^^8P@EkE9IiTiF9DO}Lq4CYA{!;KNZHF@ur*%dU(Hlw@lmI&F@_aS26KOiOUPUh z+5N;n$oe+rM|j~LV0DvLF>5lxvYJ9iC^>Z1t5#ZnU9pm&ORC-`@(XW@Ny5`Pxvb3B z2pNyMx0D!r+%6FLI?=)Y^C-SR@!trxza-;MRBfy1S8pq@kJ>91dsV?3kB`GDM6F~B zCu=xKx-W)kM9oN~rfq&X@5p_38o_3!X%oAqS4{CwcyNM1O)!XWcF}c?$6)Hlvt;f} z9cb5HoC}p8E{!D_YBeLjE``r*aV-bm<^_;q&2>trt$8ptDa0FjG_d?bG;jjxJ&&8}0#@$u?>A7I)rB8IIc+Y~8c z=}PXIhix9zBn%{g35IJCU0d#;$P~U4{580L7BUN2;XyP4h9_<>P=MK-+>bGJnG2%I zb;5{M@4b!j0hlIpvGQ*LM(<7YlHKHBFsI#oRoUCm&KB1f*PjUv z_3X`1vG0bkS5w{1k|$B_yeZC_`dpV1eU&=>>ZFfBa%?%8hQ5tAGd8Sqzh*o3bO+~d>`-GYdtA+sjSrEzH!e}K`}&1Kzh=J{WB;tPkDSf^8gZTLzsgFVk zt+(zCVMe0IwbQnSA+YQiSdgUboX5;;jib3+SDhZQ~e==q2~qC z_E2G!9qg`f^Q=(j%Pn7#TO+yDI`~Ik8!M(+&A`0;x{x6KH38zk+MIKnBLP85Pma>gmNC{swDaQ zzph&3JQfEH7Bg&XgU)LY*-d$}K=&)LWc9`SIu_J@+qq6UE9!Agi>}QcajDF!YqEjv zAf)tqlv`oAtXNCqra{;54VEx>W*f*(g{$wQ`$D8pkG6kFCMQTi?C+-vS5_4NpgU=N zvYT7+Xg_(c`4TDlBnU-e>Tc~6I@zJeNvGeuVSU5@u6TM07wB+j;sSWigbqJC_$W9m zapVR4diYAaU_3?pyoU9c(DBxw(I7{E_X9_alWAEbeWeOgVrJWca;gG!Z(=P zm=SreRE6JBTPY)f8&7^In?>a`LX~7}TiKbUW2@Yy?P1$XrBA7SLg+tPRb>a{^7%=N zKVCazWh|Lg?$iqDpfuCN-8Y@zP(3pG@97ur#Xg^ji7E!W?t>Yj_k--|9BOo&dand+ zt$XQS+p_;6>8rH;EM#-?Eu8>)MMV=u7!}#Th7geMwb93BFz+KO&R+u>ITpkwp1Q=5 zqcI>vlEGJ}wt_YOJXv3TU)-ehThaS?=?$5c1j%(i#%=b-THBn%m8@t=sYf3J2|}G- z_W`79hA)4{E0;TXh0aqQL7f&*(w?L~g~8NZ_QPB7Acn3VEkvtfx&O?<#UrdVPE=4m z=U962!B=YHNnICYbQ##g%HQ)OYEms-4{GYSucA*8u+f$-IGx`1PPp2KzSVKyxljIY zXv=Sesl=QdTDbFbeexS11*hoZ##fTlxbFI>c{SQT)WpkQ)NszTP43<6NSdnQ=^5>Y zm;7i|v1cEgPmrfMUiTQ-970B4St0PufBNbHg8l9zQ<^%^9!KLp7&Y>+o3~}?vxNtQ ztp7%Q?uljcriuyTxQ$@!3zp7QpWD`rz$`aX-`6v~08u5CU+u z?ZETH@n?sD)wk(sV*^nGWYXm0x?_MMF7dY-KWJda@uy(>%CT`lJaQdm+YE9sP+8+H za+#;CkSy=QFY+g0j`8GHb}_56=?9>oO*^fBtP+%`bi?}}g~(yts&D)m@kQgh8|THN z*=|l}mi=z)cPZHtrs?9pu?OZo3{h3;d<3)Z@LL#+(dcAHT+Ag4dVZO1%@C07Cj%hG z`}57Oczt`;`pl=Q2Ovuy_$nplGbPQOyv=#-&zE1Ae1)#q!3d=pLu@CT7&uQIop^uV zEjtsw*cUjT@#Wn;0dGypoXb|Fms@h*DqC#RA2q~Q&zxde4RMpflixM9l?hQt=wm6# z%u-+iZd+5a1TI~$5c1L>yY=}E7QRd#Tq?f&E}VIrJ~`}*6QG4-`P*2U`$?Q1wzxEi zV*b9nS-K2>lhC--OcsnY)Q16LVOChSq3gJ%>>YSLTc4#FSIKAkV2BIgx^!bfz~o9P zU_Or@bUl^6!@y9GB`rq1aTVY^?E4eqRq{&BcaM#eq{IMb1qnODe^+<`W_?Mr-#(b!_G*afXM>4i+ zRUTb({z@_xtC7;WEPh8*;tyK7qvk!7ui%y~-LcroNFflo8!moacczBvRmTsOu-v9c zOaqX)`tEHS+e1YQ=HFT*uR_%J=pRgjt`5`tWq_Pm)AywIlI}X1ot&%}hnMUEGDkoY zp+hO-3$G7VyX<(Cyu)d1@qEWu#%6unf&M$A1iISJWBY-g`m7X_ef9;=zd}R=TsG7DUos> z2=11avY>aN?)R%OB`!4`D=}>scDKdtctJV&+H@+UnI~8!8YZKZ(TI-}Wf|?I6FYOX z7#F!%W67C6ipYBmT?lL3?Gste47sJm-LNCwZ6z;+^$w^T6MffXo7>?X6w9w|AL*zrq(+Zn&nIQY9xy7{wc%TD;I}bXA^sjR~{1nT(OZ6}}TmEy(B?JT&4t zU6L6v*#9sv*{iF~*|`rT%;$h{`;vXq$9-!oyY2&6MV!VAF<?OTCl9i;6NX&+RWQ=>NW%dcL~FJE7HniYxc2sr{9S8#I+SOzxBF2Y~0u1A2 zi-uX!u!j@xo|3e`^u{4qP2&8H7w`*!7uAoKBlj&T$)k!UyK_a3LYQ+|lCGQG7cV6x0c1GPu>xjCgjbNEs=6@}3 z3AH|eon~i|PB4G&x{kbZUmLX+Ap{~-aPr|D5dfPX(V3ny`w0{5xk1MB~f^C{6|&vwc&;_Ou>E&h8m9@}vq*f}jM<8$n0kgn`G{)4B4F(jFh zn>gJkX8PiNdGjw?GCYx;fR)xo^2fEm)mk|fOkVT_>NQx)ocuU-Sxqh0&LRC^kQyyJ zmL&I0B@f9EoZJ6`U7Q$d{ke|w_=C=_kDC?DdgM-E9>Ej2zA#~3SiVMD);}9lpccjb zZluY6yv*`9X8Vty^c@iOZUXmP!&t%dst=C&O0 zX!RDdvT|PzXMK#@Xxq|*h^Q#dCq>H&gc%V1x6WwhfX;5QF6)+V-g;*rZ72Csuhgs^(C@V|%ydG=#|C}?Mdr4{0-I#Ost?T?%YA8we~&0kLQcjPosO~t zQ?;~i2Hgj2#C!?v<)Aq^G2P4kw@Ru)2H0jB^8Zz!n9lK&*DLkIPuKA}xb*#-67c)f z{Gudu!r5cV`}zZA0&jX;z4*)U1)L_@!1W>JEj3X27bcn$2$3YsKL+Bly&FCmJ9(xr z*d}4r@zLNk?%2##Uh96e>|z6RRpEjtQ=cC^3#GyWxDy=lzO(cIs__SX8t_(>2Tvbz zK7yqv(ryFt7~%_Q_$lGBJV!g~VtCFDWkk(*z?x^8jTkAB)3Ijv95>n?vaSz4`(efO zP|4BFwzJgsp(ay1`~b(7Q4%ntGz$5DtM+l*VIhRKv_R;6TWSXuFC-=jZEp(gE~S-Y zW|&L5%qRCkCRp|dR49;7o;>lUDmgp_)!6ekInn`w`74&>2fh#_OI&bB2<>UuXA?WS zLK|D#J+|U^VJs+)s_urTbN-@edLk8j?MDV`5t)sJu>?xsMPo@kNsFj<93p4z!T&9blKxVE;p@9I_w zn#Y-P2GN>oPgvOvk*=}5s*z5I`_$*#6Yjx==A9Wd*WWx%L$Q%x&k1GgLFIML$jBjN zgWrg;*FkyUY;~Ao+Ef@s5JwIXL^2%fO`lCf>Gl$dU8wfLFym98$huexj-qbfnLhbR43zbEu z+q~YJ_vBE#?mOr-Wc%8wrgrjN7$2G8_Ls!<>o=Y5ufbgE#Sn6Zb>9tsv$fopb2cBb? z^PLb0XP>NE^r8!Mq-IX{s$U}(a zUXFi8cTdl=XnU(fzQZ%Z{zbd_`exgEI@L|su)F^zh%$*X=_AMb@+ zI$ObE?lmNM4P=_;e0i&&u!zWcsg!}~`jva_>!z1h6q%uFELrpo*T~2W?d^-eGOhZH z%pNmbgn{p*b9KvcVxl_P#LtT#=w2G#yiyF!N>5m=c~tg!)d;i2{}%=OL)~<1@TY1$ z)NluDn3MXn-Rc%7IE(U|toU4A?0b`m4+lm>tF;3)3%lOdRbA96p6ocdC?qO8H$tgE zz;umJmV6LsT3;oQ=G-Gpm_1gSTe~*$K$?+QY06Y)({-H&{*nuppZdPg**JX5;`7*8 zqR#o3?a&$Zmx_BX{sx4X!x-_|$+ajbC}vKs%04hQPG?r9e?nhQ|6EB)NmNnh7t@dq zaTi4w=f=;HhSr}^ErR1zU7TIF@i}fcCe_?mz}I>zH6a9 z{XS8f_#}QM5MZbri5pVVzK1=A<4XeJ2!93tv0C=npZ}S$cj)2t|LJktCa}TKAiCMfx=MdQD1l?Uucw(r;s%Wp{i6)R`4_}n zL`HOYnv&P>4%|h9JNW{{)gZQc$@IPI7QEwv77un#M5Dd%tES&~9)x*=fUn zPn=xYq*X9+gSq&y#67|;m_GtVc!kL$0%n3wjT7wq*FTgm`TaC~IM5pw{cvz<`$qQj zwO8jxb?!2jkL_P^&i>pVUppt??xArv2zqp%D4OOqd@V{pHGO_W8HxvO=jh{uv&Fha z{hMK4Q2gNwx$C}{=ke?Ou+e1#b7H5b(&QA))zy*-kr}5FR-uLs|_BJ61oG2=SyCNJ&=`>w&PObhqHYY z3Wp-0I96)m)>t!G8??Cs%m916bLN)OcW@~Qf$!yf&2M^Y`VYwkH=n_H+?R`JZoPTR zDb~8x+au_s zxCuftQF6SM*bgrd9E@niVyx)z3I)`QM-Bgu8)<#r!9^X;H;uj3b%wHb2`A(tR8%wz zWrGXTnV)O7x<(8VVk!pOm0&^sgxB&Ln&c6h>F+ht^DeTYm448pW5uAAm3OC6dH9&O z|F5p-Y1TW8VZ-n$+YNVv08Qk5y8te4%ZOf;NT>F3Mf>d&jzj-P52gsIcF&yb72>_p zPy-9^_owvK5Ro2m>&h;cA$5J7Oq5dchXB2{Vq3%zT>M4Vx>GQCe5cEWc4H7-1K<`a zYx)v43t<<)6^)<%;imrt2YxxB7sB4wA8xa9HE>_cpY(dyq`p`=U8F`xk}bm-Q7_+w+s2c5ccYOwF@(8kOaWjKoIhJ&%df+Zurn zlG#70IJYMe*vo$6`w$I5=DlUJKMX{%?1!$=-Ee9LXIZ{o`?>vz{To!?0LUt2pFM~~ zh2aW#s8%olJ*h}H6#?d~0+gAMjKw1nR^kg>rS-*B;z^2s!3b$E9Pmm!gVE-^^$ogR z#87vSwp)4WsMzx1T+<}mwY5dNWBX|PSM%{6`c0N)i{{VaN{pI|j8LsXayU8+A)1Mv zz=3MTZfVm%7r!U3O2{ETlk<91KvO=i{i8H}HG#FDYtnK3%uL)9sGvO0tYCs=Y1 z&cnf^X*cPcKDJ?4xcQ8eR)&j%S~kDQBCU`Cs`KDnz2QU4VM#UUHGFfD&IB` zwdSG~=~a|li`id2xdhch6F@5)@3aC8C18lT33fvR2uJNtb%k5S9o$TevDzWF$K z*%y5|3|*uswiue1YviWkg+965VKB!Ct<-Y$o~dZ7V5fV#aP=#|dsOmzF%M ztS|mmCwr4Nfp)gr#;&N=rfYxY(8F!}bVl7~S7T}kt*K^F+@o|hQbdwpP0l#v5`!Gg zLDjMHvC*ToEt%^I*GMZ35hFJk&`m*xcYVN(+IDJZTLI1d3}_DC zpNae?K7zQ9qAnoFi41a~Nd8Vt+Ft`kvnRus8KHSS0QwClGv2?f91+;JxlZ$e-J|K0 zi@`D$HIcF~KxQs>J*YCC1YDBt1is01Xb|&gv|3R`jk(SnL#x?58fa*vG^OX{$fG(j zn{tCEs%Z8e73YU#hbhAJXmx~?w3J-GJ|XuHX_3-E1VyHZefdlL z^$YToj5{gIP^!sf*XtgQ;+(2}*?KLE0r((q6 z;o&L4_t*6L;`_Ux4tu}4}Y2p+4)w^wV(_YDPPqIT%6Xmep z#uC^h#UsWVgMCj$&1#D9ZM+w`RXm*V-Vi7%YRJ=;|nD5ow8gCo$AFpi(V}aYx*5=SrE4{0+E$ zdVuv;Ij$X>&q@3`>3Q@K}Gz7oN&lyXvMk8hMvB{TcuKbWEA&VRBdAP@wLt&K9z9ksh5-kIwGdL%i;WwLm?d-C7Y0KI$n!R}anO$nD-mE9d@L&EhDD`LU6rmR^{^NXj14lNlG4j)g^Z$@@kg>hd9{&~1~PwAGjYkI3BfJxY5~*) zausJUzFu>Z>018ia3x92?U)J02ND!YpDXkTZ)?7OsSv$JLSah>&nJW{1yWj!?1!N8 zaAJ@w=!1eFloCOjSv5a4mS6X^BsJv220{zfO4>Xaj~Gt5ECR1sIyNOuptX!p=H0~> zJXnwz9ho!{AK|3^fiL&(TmVg1TD$VDsE?5bH&;V4IJ5&WY~gz1YMxbca&l*!>8wNQ zkVXOmUDFO98?x|oQuDIX7HKmJ+mDl~ZQa{buaI@FFKbHw%dJbxnLKv>SZujAab@~D z04#b=3XMh`l^s%?m7PBjd*x!a@J^ba&C^> z7@4|@T|im%iW5m$dY6ugR$lg}{0&64F!Ola$_O?9{E-V0k@(TS@YJzMX5XLQlD?l% z-#@mlS07^Narlxss-n;KyLSi$+y^r4KzMCcKDdHBh%;^OgZR2|Rlwah% zkz&itRFg?MnXlJj7cL5L*{E@dtLaqMM{^6*g7ocncAAC#YyZ`p!WZ_VYsAI{@^eNJ z1>$r<^k{2~Ubk{B#OopTI5=9^1Az!V>l>W93a#Yowc~-WIYLxV0V2o^>fACwV=W1x z)<9bYby-~?~kbS@jb^ToBVPYt`U|IENYl^U%9?Wz%pGu<`Qfm zUK2b8UA=s)S*RoyD>$lE=vEKD6ZO{Y{t45DdWn{HinuxPCc%G&83CNJjhG$U5mAld zIaLD(CxJ*WdkatLMsa3?WK?cS#oYvE95x9tskOJyb&3LaWLa-b$Z9hw@27IauHnU~ zbF+6hAV>6n*O+3aR|jct?JMflxP)rgA7}-Kk)=*@4Xi|_;*|~dK65PFj|kT?wr>xL zrb$CxBhb18&9S+P(3HoNR39z42g0rC5U<^b#{ZWAE%~G7gT}e?S(*X^#F1w=A;^ml zDMnN$0{7!u-entM!$SXT zSFP3k9yGTGMeDs)M>u+5iioDni+`rw$+_z7pW-PY2Tm#amipLt0cYaco=GwBxyccM z?B^^NrImWt0r%ezOyw6K5^DH5mI%G17*b+-rN}}@PO#q*V+>i1mMJSd7dipmt zB={`$0oQ*TO)!9nO0W5yl8~)r(cIlEj9!LcZ{XSjJTs4ci>_%wym4Y9Dxtb&DN4#g z-ccv6y}K+~p`X#w$wIlPI=|HIzj7KPyo=i;nMZFm@a|Aqq`RzYS65G&i}CZ^*@}Pf z*85`-Q*T%{B~noNsjk67$Wceu6P~APVB^#|^eaa-ewoLV`X2YE@_Q>#=plH*et(P{*m9UTMb}UemCDf3Vz#jKmcAL zhRVCBhA3^k`i=*NF2Qe4c9=b}t?nmBLQ@e8Xt@99H1b7YJ`{`0V^=W{d@8?MM)a+j zD@Impo=4uNK(Ci%EiR2SF|;~k{lPji)bhkxNiRe^{o6- zW}Row9mS^Qq+xVGTHuC=JmVa*Zd=2Msb$rzL)oHlNU_u$)CQ( z73osyj+tPy5D-_2Adj^!-VayEyTSCLI4@eccByNOc06v@iQs$h^rJ%49;E@0NyV~v zt3F#*EG@f3%8C}R9rz|LI=hnp`sUThLxAp zU-<7VHP!Xc^=VeuUmkarz7e_Z{vbAeW5n$HyO~=3a2dh$xs3{D^u(ngt?2rE8_oKn zhe;45yvMC;o)m8VYCnhQN}q#R2P)4&%FKrcHwB73xUIW~kGpo`1QHn;dHnxl>n+2g zj=H{4K~O@0p*u!81q76i8M+1-xsL#L1I&JN7ujWkj9F?nRQJiCX-w&+gsCj2BOwK69*>(!)8@}UG0k7TLSSy@ptJr3^Rw}1T1&1vk$r~R5N^0u|5L- zMP{F9kcB5~c58RC)QqBfVc1aQVCt#s3z(JkXErPeu!0Dumsq+HmE;GhfSWWXd)C)= z$D5)`M3TIRt4n5!Xt8*C?DiSah7iULm(vaAz}dyb3pfdDX&)vR()uF}_fX`9VHDBQ z`9*Nywc7mVEyj8Ku5WhYKM!VYz%x#QDbGu1om?+^8``#A%C7Q_Nus=C%fogcsb=oxXZzBW3a(I)x61hJ$dQU4SVYMjV8hC_7 zo!{X66#HOz8m9fv-J{eB4}G!{EHn|PldmiOx?cSvBOhGvq_WW?0o-rWUGNShZc-lc`M*)4&qV^6{q@<1jCFlzLKozyM z@+l8kM~Bj(oRtk6PC8Nt_lhY73Jl> zU7Vf2A08gQL?PNx7B@!7Dag3c(b3sXmcZG0cqV~a&ck?j_27hizP0k+5Hvbkfsr)m zW0t0;r8qCmG z6;W=ae)vRxNVWINw;J^Ltx4wx1C_)e5RxO~kH;*S z>6aflBwMyDC}@3hxrl5&YzHN$){U7l4eI3V;e?r#RmMS>#l*{%0~BKHALIu&quF_z zWkE3fwJ(cJdkvMC=Wz|Kk6ExOIq)y^Rc=&Zp@{L7G>wn(Irs@I5d{++#7W~WWb zid6c*m|)H{zqd*4{0oNp7g0#Dc{)SRu4mrD{y&J?3rNDIB2$k3LW)gzk$tCkuL(kz z&O;WX?JJNda2C7-$VwZAkJt>v#JUtdkeO z9roGvnl~y8Iu1)pN_2gE+8DklV~Kq-`}ncByIZ``@6q=X&>??EAdrH-Q_vMK1zi@$ z{U*?U`MQ|nSJfu{O}#X<9)8d1lt}w0(KrB^{0TC^SwyqfBO%e~_|PZqM<{mx^``%| zkyb?vBi)CoHn4dpUjvaR9X@aGjB5z@%*Qr-*f}`2Vq4qkT0Pn7fZUM=Y?(jbBng=a zA#c>0X8qsP33;LvaST6>Kv5EP2DP)Y1Sd?~h_n9{^Lq7&1T1I{&Db0;Tw&#dAozZm zFMgI}Ea^{A&FxY*q%bBS&o{3)Ed0B%uFa#(3k6(oZS&t9avv$g!=|zHsW97Ji%Y2c zsT-)jQ#VozvI1TNDw0C5XjGwjUl^cd{m#96ElUqU^Jeg4w{&>=HsZ*L*k4EhfA z-4tl?d3hU6ho{R{7-@Wi9w=e!M_G?Uy9oTt8%0mZ-)qY?cR1<~#%z

Iw4G`;>mp3baK$B)IY}qW5Dr~{E$XbNB~8oQYa7o{6>9kP0a#er?}flp``!*aSw$I z#G@lU=laxt6rg1VA%eOo*YcQwoVSVMO7Y4#_9yPI!kQkV7T2dExFN%6nocKN0Phm6 z=RIAnVs`Qbn8lI07lu~J5*k63KmPX&Ov<7xTkqh3Cb#gKCSSKXj`TooQL2SUH{_Z7ugx-a9Q6Jubi?!O+`8_pj7vkv*%H+lu=y5F@<65jB zp{jyCuh#lL8~pQDD71#oyWHN2C6)FeZ+zU;a;V1DdboL+J0#d{4_Wt6rU2aFM! zGQw#wOpIPLN8bs)GqPs5BgwVT^HWOOlE0}?B%Jp5YG2Dst}F1!XC2NY`Xlz8*)Eh( zn(Ex~@5658!%v^b+vQ7UrU4%5 z(phoBQ;XFIMcc%_aLdBRi;2s)`kxqdZ4q_)*&=#hL-P*kcJzPuf)5;q z>Pl4r+CeW~r4<%t9M@CkF-1`DJ+kK9#C8l4X551V3@FF~@5_LdDswzLev(Kpg??(f znaf(Ba0b_;2%87|Wea~pmF#W|@p}LJ)~DgZfF|xnBC3Vu<}M+k*9O09Y>a!OMf<<$VVr z(rRL#4hb#VlmRn82g~Mt5aM}4S}6ct`~Q!$N*;`g(Tiy zQkxgmnXhd89Mi|RI`npw?@$^7`B3IiiWh@=(plIdcOyRMlNt|pYDGqkQdC7XHC#%B zC*RuJg)2i4{`|go-=8!_LMKleQBOxEKECNo599gPk8W;_x66*S)lbO0W}`SJ56h!OY?-`xYUc zg4FgJy?BFB!o|Vcfn)g?&;g>4#=2ja@l4vdLidPJ5+4jka^=+V)Yd_!Ae2$;Y=b;} zczZQ}K{^KMrDfC*=|;^{E5A#M-cI z866~W8{uJz{-LtdF2+{Haa}6(3Csbisp%W>mU3jA0-jGuCMk*}^nL6^d3^pc!|s8z zE*=?oc!-J`W33+g;-U%>e(t|N()Pu&Yf6oRZ+5)ldldWORyr?q1Ir$2#@2dn{N zD}f-U-U;0c2S7{KzLu-lN5D`KOinoG^_f8R%kA{iHs<`;f@S+9Pb~o-YZ4O8@8*Km z?0Fswp+Fyf?SHb3vLyrcDzdg5gU2YB@+Q<3S}`t-@MrE#4MTdZvwu_TKN`=IFjZ%S z^4Bp<+kjkoKSqhh1h)FzHasa??rARpNdBm)aoKl%i#EAZRN>R~$=6(~Va>89Xial? z0e9IhK>dvqkK`mwE3Vc+$uHCX9#GOQ zDEZx}DzT5Ig~^C8wZFZZWpK%Vzj9u6V+F`nGav7^4v9Lhf{%F4fxq?SvIhL(QRo(Z z>s70_M~@G7m+_f*y&{JzU4ah{nx~x4y+rq~ZfX3eAR{D_DPdKM=p<=$k|FNkCyTQ;+5DPEp?4?PSwy)L*T8%Ruq%X8MWX^*eck_^koGc$AZ45f-6a8Bt zDhDcv&%Idv!sC{m1D6XK7Eg7FARxZx2R~iuFtgUF3!}$WISCGosbod}4ZQ2cd5Q$A zN+yY0OSbv{D@`|2rUtyu{g)v(ca{+XT1#MD*>IQg9>+T2Y>q{;R7NRfEgaP!pq9Mvj25JQb|gjsg_LQQ(enXeefOd1A$-1cBb9>zyRh7M59tXD1cRS4a&wc+TO-0H+Whzy}0J6Fk{2sRa zZl?3K1H*)9`f-c;3xt<_0iA4sdEm22-!_aA&wSz%JR{ZO-d?s@MOY#+QH#9(K{Y$3 zfgLI$tVAW~n{1W}Or*d6=aecssO+};94oUlcP;gK^}7NO)x>slJ6CrDM?|TlXRD!(TO(pq)0$Mmr$J8_6y61V} zCHzzj4bdRs&(OVYMn^Ia>!gn{`S`s#ael^v={gN&^;h(lla4xu+qgE+9bFRE8cm_P z`G{PWqAqw@d{)jI4Et^D^R+yJcHwaT5Cuh|&x zH+ONoO8a)VBDdmU{o)WJu zjCn4r%Q&MW1aqfZt#*k)Fq*_+*+vP6o(M8WQ@gJ@Chkk|edEP=g*_JRN9*K6w5 zzh8GOMW_fRgU5@#r z8R=1|cFV^LmAr5P?RD{|*`p^(-!qv2L@0G>bN(=#H#Ah`2oE8KuUrjgrv6s7-Sb?? zt%eyCjNjn$ZY!U8^kwv^8!?8h%Y8ylu~gn15Bz3q;L%M0K#53{B0_W$%#Zq z7%S5vH8D3nK5pkY`P|b~=Z%Nd(QM>XK=t4)6ILsyP@x@O#2MRcBCCjMl0R_Gs)zlR zqo}YO;1#>Mr`;>j}Z(>@Xntl{EX>-(T#HchrfK5 zDd#kgHE4?U)mnJe1MKlbuh?oC+G^}5dI!DeSK?BHka3n?&8aUi3CzzL*$jSTF-o ziJwDAXc13R73uE>sc1?Uz#<*%r=X^O=H&@Ai1((j5Enxim-<`4Y_mZ1cqL_b{<2A8 zxiCwEAZT(T0C$(K<2v$4q_15fM@)S5C8i4VH>493&dXzz+Ty_bS87H$%4fAXQUtey z@1PTEz58wwhQ2r{F~2d@*=YG&^&;?LuWvrQJJ!dB)8Ueji7<)ixw_OiFsZ~5>@|1x zr1wG%W5q_gz178tr^yhdF9ZjnCE2?O-I9_$7*2%oo`(i=$0<=_l?M#nSegi#l{u1Q z+uxjaZIo?e}L&$P1Mx5k2xR|lmf!o~mf&q`moJ+Hj6YVK*;kz8r&)h2H<+Yk-c|OC3|onnY|W;TmS5bjd;^rPtFtnn zeVjV>pEz(p@T$xXFDk^ds!an&@1A*;Oze1QN0h^3e=XH#`zEhmIPa1ZpQAG2Q>3>} z9sXe{R^*~^EY+MY1t=4*QW4y}i1X$?9-U;X4*FO3-pldMjF->r<bl$?Vqv%T2d>s!rE&(!52N|3z#(mfU{K~_og#%^-d%Au*_Vo>C0AVIW^M^?D@nLo zgWj)QQwG^{Q3(HA16BN;6p!RHX|H9N6H)>9c96OyC(V8zmN{)f(I#c8_Ek7~U!q-2 zd)1enf$u`zR+)t-LbnHX4G~)J1P6!V6ZIN3R>(vzXqa9$<PAO_6!b*=qt@3cc z{J3~O-*KO2Wnpm=p*BF}$(@bQ%*J-UOC!xRp^77)n3$;lH;8`+2)y+Dcf|w#>IgZm ztFH$D83XT=;Kt$glc$Z=M-Qu14_&1JGrqdI3ylC}*)4uFQ*Bbw)HFMfl+*1AI5+Qu zW|}-s3tC(cQW=`4%c#2^{u(^=-2)%_%S*Y^jEC>NF3L1u>HIFc0rx8x2lMys5+8%P zmBsa1ok#WBooDr$y|o&Xb>Z;IH<_|=AzeiDD3lw3h%Gwk=ffwoUu4K?jB)$^oyC9x zfuSY`E(V?k;euXeI#1ffsJ&j~S9)hQ@=jCCFo4-!#qa?TQUBXTL>h4|_H_A6PO& z@8PNzRy&;V9uss<6DJIj4)zVPH*Rlk8ELC+8L(lMK1U&dU;USMWE^4d{1UT9le^m_@*XusZh7^D-H$Y6($7Be{y|o1Jw}?)G2m$YwF|^zW&7zS6^DM`*d1Soz{x`J`Qhn+`6Qrjx>%)QMNM1)0W# z8yY;t(RKI%fV9Z(uHWwr*f56SDJ%4f;KDCPIR+M^3$RB^hesR_YaD{0zU1K^U?<|V z8WuulDG}&yP*0F1dqL6cm2*0m!D*YwNp|l?Y~xQeogG z1}EIJY2&Q0*r&+Kqu$J9pED6ts-5eXBtX=*j7&cJeF*;+%%zNLq$FiVK7|*lg6qQU z-U$wr34W}>aaw=gVMJ$EY?ueeiR%$j>`y|vekUV4`Xy;e;vofYluYfS);fVK&x|rl zZaQWC3c-w+bUhd0_sP{mQfjf%PL{+GRr2gooI= zNg$E56`+MyQ!SMK?DR=J>(;h@o==t-E#di?t{&**`B7}-JoX!mi(2dmO5b%`TH@O) z`|H0wbj4r2=62`7vGCqmncuwUaS~!x;w3+`Dv^WUPh+(|$C7(}e)9*pQ6{WtrTOxA z`JCPVw9$g3NEI}+YfTy6TC=;t@s2yG3G)&vp2n0L*kp~X1CzCm|N3XEgzts{7sdtN-`*XHg`pGbAlYwZQ~5*{jb30TeW6d7(4d@NF@;y? z{CP+E3Ch_^mAj86?#x$T4*>B;ldlgSwlq^;ONTUMggM<{py(igybB9w)W1x3Tk^0317=2{ zd`cYHpp0$*1qHVXBzvH{{cTE&V~n}#OU(D}j9-)UzjfQ zYOj|+ZNm-u51&IW%$I+VyBVrB98DwB478&LZaCWLYzu-?5=$MUA+MP$b<<0Uct5rT zdJ5lm%)EdP<2a-2<2akY`_SlqIF6F~Vrb<#D46PQ597h+3BKS1QG+BipO;xvI}MlC zG)iSM#VO+x8qW5cW28|}qO~WJ5JX(tj={W6uS(9GJ6$HF9w!u*?)CYP8H#!P05^7?v9Rcdz~FOzAAUp4?&xO ztyJpBG`F)sp#Aa3N6G=C#bh=a+uLNMrxV}~R36U*jLq6+m8k^Ek-SQ0Zj=bj#Gx9?M%(U>j6744wqBgb<5B>ms{!)T3+Xm2Po zLjTF~?msqr-;HJwM|MMRJ{=|nuMX*){#7voEO5kt1PL-Q;|-2DRJzoKN*L4_JDA`vk-4qxQN!2Py&! zK5SL zXzMpMIM@^#iI89=rFP^U{K3A_NO^=;5d(Y8pEM7i(LPUMr!(<_O2@txBM==KVAC97 z3I5piwU~wA_%p({?Dklpa4bo7A~AllfiaIde;`*LE@PoeN@Gocz3Z8qc`j8coO0JB zgG7CStGUl(`It(uQG+exNr5~bh+&OA+UF0ngOsv$pA$@{xOZpMjiigx!nth+!m`A%P`=|RPsFm})~U(n+>_NTM%*0c zWgR#Kf>}J}IV-nsufnGq*|hg?-Huy@$Z8g><}vQJp5unP9Lj6d_d2!ORi#KIhO*h^ z>oX7shp$)NN-j$5!^+WRfO`KDYA-OK8z$#tD940lwIe^hmIgc0zyqj1fXqQa6 zW5xP+iv(1kFx*aYyAJTOyAA_0%S7*xDJUooMIP2g?#L-9T$-Aimh`cC@VU9UPjzmi zg_eNWj8>=tfR8i|1>)R{M*D6&#ThiWwA|-?9)x3>sv@mK(oVmBCQ$D;l2OfK7we3T-nwU~N?@>W#& zupB}ZRsgNr9p~o8#&?_f29MjWnwpwoY`^`Z{pS!>QCSB8m*YlJp&uuA(|-5K8pdCN zzD?!A>m7Ax4%uzAKI+-u8s#q4y9Pv_BK#v3GtLjMOkwC%$avv^#x7gTb1FQLi>|Ax z+Q1~Uauzm{>8~k33t5O951(_hzCfe&gf58Mqcq}tD9XLk$;53yxi88?`+@dsebU^2 zmyaIaQVe!ieyJQ5?qlGY!%XawR%H5eGKhr|W9sf4noNjb$lOSmzrq@l{OHbgA}5N$ zgxf5DRd>l7yVO(n}&J&1gZMGXGsDn*sz zt{M4fVeu>PslpFcXjW=mAxj1tW*wth5dNfy9{wf681I019gT#|B^8mhBgSe}n(g8A zSQwJLYfCRk6y~mniaI@+_f6lCj|UdJIc5jh;D_MjIhN?Ci}|Q%*(R{z&GxY5vs@gTj9*;06{|)*7L&C$U_qlOrn{684h^6Q$;yz zp;I;j>gM0;wcpVH?(cQ%b$-;5|fb=Kw9|E8HI@-P9D6aDbBRV}b-UB3_Mt z`@f(-K7-I>1m1--6XCKRYnO+8~>}2iD`ws8^)9YNEzhBfwVqlGR z!`@R>>$(1zdxx(M1TGDfX~-P#_UzWuWAFlnrI;bvhICq-gWlLt==-5=!lPRcDfE3o zF(bY5x*Axe_R+(_6%>r%Hx#-(a}e?Ru78TF;?Xpv+RH5s(dp8|vod0C0d`};^1dB1C@;(m8 zjFEXfYJ1#Tx<71b?;ue3r6noELMvLn+vGb0zSS9!B|a$>GeUDJCxNKuLSgT@Wh$t! z&>hO56sYeG2=Wse?B62;oxhd6aH^vg<=#-Fqp2H5t80f!x`j5~}zE*t@3oW0MP91&of>~3vFEN>L>sG`A8i&k=|4dU|taUPC z#W07pJ10j%3~TTi<Uac=?nv0-4UCT2hP75sS(VK{>!^&5l4q>f_yZfmD_+ z(X>(aPUCX{WlphFzgEuu)cs=yVoq=FQV2Yz%+0x@%EO~2;a9APYOIEK^J5UmpG`5+ zlIlE5y=iTwLlkJv>#+8ejt7Y)GAr(|IcSHKiKVmPD&S#=cTkE;JXXSL1R)TDD<>41 z_c{$>1|3QFrcd(8qFpFDR0LHG*2qJGvv`JBxfu-lVmEqTsfS`8R%SD$G&e6QnH`eu zd7$;}(~LggB<|FGBWgz;nnPc1^+!%YG>&YKBd+g6yPsUS`-+EPFsOoshUZ%T5Nk89yxnSf?Mjt{eOdYqGYb3~U&A>VNp&j4+t~VcD&)7!_~oRh zuJA`r+Pqq8ezUx&lp*>q#pK*NQ3rfTcRk_~FYe`)Ex9o;-#eQ$Aw+ zLb0*-f(Lv2Jfm!^hhF4ZtUpREQ;IRLEj!n+P|H&YKGZjCclZSeYNrh_q7pMea*2Jd zIh5~dp18b@oXg$Y-F18l#94g28-Dx_SXY&BMu~QT9O~0N+`Q3=03ph@=xtE2!t(jv zbQ{ld^uyuNf>mYvuB0TbTT(eCU^*pga1Q9U1Om}3ii!rZr7>RgzJ5G+zWl!Y_;BeP z6&dNFBPyK>B!G^xBr6NLxVknsH}fkoS1&V`JxE%X9!a+aFH+(mM|8PXkED$V`tGOP z{^ju7A%HPvA(~wF)RkIMZ@I`CkwB=`ZfD!DOCYrTcA5uZ+(+tdSW!^2w5p6dC4)`r z5`B5Vj<$>=B~!hJ5Gs*O|1TQI0kqv8;l^WMlG`OTi*bmD-4JnxQKGxt@cv1rLH@jg zdX&?a;xCiB$;oRr+LR3zD-W>(cuQU#^JL%Z5HzBED$1$a?@`i2$WbXHSZ+gjURc(OiAc=9w(;xkknFI(xG$+&Ucyxt zGcHDGOV9&QP~6|^Wcko~8+U?qUj`VSKvBa>MFWP3(q zdijSD^u@(I0$0^W7HfRmxGf=w77@s7GEv#6b`s%8qcn{(r6=btl@`C!ol&HJhJ5YW9HR|4%4%N`d->Xbz~-VU|c7Yc4XWD2>D11SnBj!DIa z;mEY17r`;2*q6jw!?D(dM_W?Kq>Dr~s$}Cw2H1MR=YG{uR&5w4y!b);H6PUX$KU%- zZ45hB(c@LFB{KVHM>rlbp{#qM8l#@SRzxZryjoh_>J;+tfUWfT-acj=R7jzW!#_xl zSb?U|Jd+ysZt?rkJQM%I1w;%{0?liYtd_Q_2|QsWp1ZU0d)0J z;=y%*^jV2s><(C7(FHjE1^p*DBokSKOmF%)d{-F=z0!DX9D#LjCqm!0w}R%-YhJxRn>7<`>!9^ zr^URcKiV-R!*(BiR31`d=}eZ96QU&K#f6$NSCwkO0`)*4=$+4JMDT;R z4np+o?mEX>9{)RE9hT=pw;zX9&!(Pd7WqYG2i=YBlKufG1tWRvwD7!VSX?E@d6gw;~h{RgL^}kh;Tm zSNI^}08%=mW95*M;SvY#6j;#tBUkPz&S^Iw`zmc94I^tOBG@Jjs|ye2IYS%fIZg1F zwk6rr#L?8Mg5mi|YqU?e?125{B&9hT>kE`^2?yPhfcR9+qw~-@Nj9wlD@2&=PBs6%FCc88hwh*FB zV`Pw!xYDpUS1Y$imbnmx^A02A&kz^pJS#rrcTMd$?$7eXI6P#b?cBPQoB7uRA=u~M zk+=sjcnt6Q>xCaT-T%;;fY`XQH7)Pj_PE;mNO?_x@H!U|mX#hxFtxC^pVooE9xUoF zK-SrjNju=&UPcxp*wr5J<;8NU8`6$6*V5jL-0&?q8R%QN{0&dMyDT0Ehl4X4_FC}A z3Lr-gy0`^zqR>i1JI*>W6zJHAv622(-w()o-C!#}7a8Wl)V=M1w!?zcdk!m`!c| zEGVN>ZiUfS&uq4+VBNWutvsFBqw{Y;u4c0*3QzdL;(~`dhu_(Ec<0;tFJ%E4h?wL7 z=xbC7-Yc75Pgm<>)K{T5H&1CFH^UBUlr)5%IMDuRs>}Oi>OkTcW%*D_e959%Wg^b% zzkwrS1iA-HkVu+ zIypAC=6hdi(DJmkkDZO}H-(`4$iaN$@cebQcX(vvd9;m{74XRRs)&mt=3Cq<8M>z` z4UQV+444{*+QsWQXOoxj58V#e`XU~_Q_m@zJ9tO$2KGHj@dM6${HARmO0nl9BDUzM z?tZ%;ME2m`T(PZ_X-hE+S`(%1_G%bQ{`#~Jca9^v4G)h(@k|neE^KbUEe=rpE zXkYa}W|dU+M=jr*NDKr`#CwsKkfhbxc1gI;nVY9di*BIu*8Pf7%Nr# zWNP&c4LE9E;`AQ#;;5@J44PVbw85S-0W83{&5RF!#dF=T z-fZ~OAGQjOcM()m7+Yw6HBOVGA~>@ju9)~b7JH(*)zoXS6T!X8HmuaCX5BK zi<(})#{V}9;M2j`Tmy0=01Jm^0dZI0nSB{p6p-lz^F`#IA4^JxKw06Qj_9RAYKjL3 z2kj#G%x^rIjD}yXKHmuf?*PFQOO_QiHDmKzX6EJs_;Q-t+3vr7c1QYW73M zf|6Y;P8cbhTq|jq6Nq)uYCM!E*5hJrx}6SPS<*7B_Z3+N(T&Hd%33);J9=bgG5y``U*1o55yrnB6@=i(EU%HdCtx@#II7l|zm#DP5veb`(XDVP@%W1Kx=Yik z$=8DJ0dY`T8lh#4Lhb=8Q(ggC>r_w)3zg#c z_&MJ8M|b&U_9oja7jlCzoO};1-iVoNoNc0#?mCFr2N?`fK_{#$fephXWLN7~=;BVQ z>pqSVn~S_a?93NYd!f>)dWybnb0V|TyuDEh1?CFuU4CCk3Cyn?*CWa?+?Qo5@$Sp{ zbfKATeH>g@VtE0!jhES6_D2WTk||S8z*WMKlNzUNkZ~>BZ#Y+Oj68`wyXv935r)UWe{Cg5oAU6zeY*l|um+w{Ct!idNq_$-kSY6c*>S|c>r3kn`}kkA^p_|bBO^%7Hw-A@*^J*cY7?UU zS}RE^gr#5XnT%cbdQD2!SSc)kwk;8jGQLPnjYkj(JHi2Y%)L@PFuq)r;93vQv1T3* ztKvNS%0=}C?G=rYuUo;={fT^F<;EM58r`kUL_O4YhXYGGp>8@O1SEXzJ=NLrWIYBK z)-U7^dn)&#p2vo_x!M&v0T@B?D`U4Sdv1v34jIE4Dr84iia46%AdbY)8PtNEt&wtM zP1I^y^CDl`s}T8A+p3OMtgQoV@R~A|$8veb9C=V;{@;R?EUk%>GpCM9*u?MxKElHP`GxrTNimbpslS}xp-{%s^n9FaD@y_Q29GalhSP7}a*AG^9 zGVp66uW*J+DJ8`qHnU#`Gq0bnaTR?_IidhNvFs5Nj{r?E3~KM~t0Lj;rXzz^&!%as zDR#1yx7BqzG-9(Di#ye|!%T3;wNkzEzPQ5%ji-bo%uwA}g(0Jn;hBPX{Id! zNQ#1OUuA>@>MP&KGQT@!twvV6uw|2a-V2A%Q$Nc5b~;ik5oIp?N%gxYKko%2wubwU z_pDJ-TRTL?=Mr2l;1t~bgCAxqz(pD~bfzZBc|5#FR_!7^Bs2uyZn(q0yc$ED`K*2YdljYu2YgkyxUE4Blqiw?O;~5Oq^!F2GD@MFkpC_kYF#S4aN&Ux5h&R%Z_!r9@ZdZvC>O#OkXpL+)5o zR5CWJktC$h>;!xkNnFU9Nl~GB9R+#x1PlEMhT>gv0bR^=<_1LQ=M58ax(rttr7)_n z=(7})~N@X^TdYmG&uo!-yD~O zRez5x!X){NF)dWxMk!a9c&z!TA?_fCk0>#vo$%8gBtYW1i>t0lm`71)x2s3{& z4E6mvX6Uh6v4KI(6xS^H*_ey`*&8#qS%gj9lx$fnLO4H7ZfwCe7B=(YRC6<4I^ydZ z?u}xs-otUqg_!zXciPcw=_1#I_#?NH%V%LXkl8~qv><%`?DnEF$@6>IeYxj|BG!dd zeg+;@DNofBk&GvM-pk#j^3pNyZ+(};);O=atp02y9ref1o#|Dc^?0nz_cRVu!^p{J0TayFV4$P@WJtL85LS20+;|1aMI^$jA4kdfOy!Al7Lp;30?JR#Q`dErnOuWlfFSCWjqOgWG z&}HsHDIRXK#0M<$=p+R3q*idRH#d|`-v$ofM8&;|ooCT&sNk-xmR$9A17b_%r@Sdt z+(JSbmkP@Qk-_0kw*Nc4IQJjg^D&gQyoQP?c!GwzaUmdy&|{YmKGk&y=#Q(X&in z#*hE42D=yoQq_~2noy#mPnPrCWh$Qe7>yUr7EPI*Wuz!(+BFwJ<%tn1=wB43O1(Wv zhw;gb`!G$E@Bd-zt>W75n`mF$B{&pKa425fAy{#DDFi7LcMs4~iU%$3?!k*wic2Z( zE$;4mc;5XvS9@Rnb3d8i%v$qZ9ymU2F?t;mJzMFa@wmNAcllETcAbN`P8%dAQ3TrOa3>+;9R4qWJX&d5lX5k z5iCIrCE}KO$D`B&xFqN%U^5uoz+z+LG8 z4(JaNiV@}#1aKA-m~52gcCyG^MyBHzc-g0nVUwh>V)aeUTwuNgmS^0VSWIZ{t+mPe zo#5Ty9NrIQ6V1EF`+zWCkHigzF>*OCuXRCAlKO534TLHt?v@@$J#HtsjN^3qXMRRqXpRiS??0u7vagLx&|Hl5>XA+#Tp) zpBY(5r`*BxFJ8L5T@sHhOG=og=c~3Epbd~Z(U@kbysO#ciFb|xRX-XMb^c<*GBzjQ zf2b$TkXYbR&qs}^!?prkAGacv%{4-48Nz5~wc;Bwpe4_47TZcA|9WQTi5m z*?xZ(Q)F5J(n7fta&2vN{tD|7eiB>zcQuNws04CU6n%1)6<1xsC(`nDoyA;E88&X~ z@0~qc<$tl49$!cJ_!3fFzcWw|gEZVp;A(uu9zrO2SV)Dj|{n%_mz_M;N zj)r*ftyEgrY}X{W&rb^c!5D{t(RwFC6ie(*$TR2YhWS4;RU!{u;jQ>97tzRXM{GZq z+!Fn~?3L0#ss2r=i)FBIa#FJL9*^IK6&!-HIp<*1|)6|D3ucV(DwDuE0kjR z=2LG_T(%}aptMRuYiq|rI+sR${(Nh1M~C%$i}Suk2EYKI_KLkHBU!TPO1q3%m%!ls zbWb5Gc4SPI^DI> zx(3EY-SfFn=SC!n(9a80Z#yw-({%mDDnPM5NACzPl-P;m8CrC))S~!^I@qu#?2-K6 zyvE*1K`&ii*hr7wZUR}H6jj}f0)03u%A;vSWkGEikF_tkma(~C3nd>kiUxQc-W?dB zCG`c2ZyoR}yAiIap)#~te-$v_MAIYuY2_`B2}ZgiV$=|oTa;EIVhOM{uY1UjU>T2j z-h7eB*onBo2w`pUs^BFFO_0jEkkVzqVonl{|N5wQx{b^*KKAK2qDk&I_-cdfy}%q> z+CL)X#hN2{NnS9a>NiZwoMPeIwUjUKZ*2U4c_~58p_n6u-hYGjy12y zUAyyt>|hqA3JLx?C?Wb1!zokjS`G4f7V=_9QAH86EDVmKNfOY@qemv%l z?Mo*A(A&^*QRPwSs2+$(2l!OTAxWY5{AymurE%9HbrK{_tu3Y{C`+uCDQF+{(Jy~6 z%zDdwDLhL;0 z-56JPMV+|ztp@1}aUt>257CdTlw5Ga8`MD-zs&V+DABI~oY}d+#4pq(4f0PikV%d# z_Scw~;7+KZF*?Z1TJNpPe?|oy* z%6RjEXg>#&fJCcDdi4JW&TW)R_U7dQYxH{A@MW;t#1ymY>UcQLnF4|t{oa0<)LOY+Um~?o-=ow5+%7eYSi)`?QD~Y1+Pg5Xt4oK;_|b^8JSq}c84~DB z>!KK>&GK$IhdkNB9o{KrlI(X9GD_NMsQuji)S>X|yi}$@Bn*ES&zuLwkHZ>c2o%(C zfx4YCG8WWUgHrn%qff%E-iKyQdlnj}6mQ2nE{4$MuqPD-pxv$h4L$VTyE$h*C@#WwBQLT#(XZ*=+JlEnegD<=?|y#ECk+viDQW{PD+G znMz~yM$!`U2$bg%&?FFDV-`S0GwmRa_j@UP!}rbAOufjIKA0+V!-zu;OAWOJP}{UX zw}Y6pYVg55)?SYMl$=l}?23lwp{fBlb-I^=ygzoWxRt&RZw zvuNlE<}@$0kpmHa2>UNwRxQcAP|K=td+aMxEGA0|rodkLcfJ$8QIaq9itnnfG!#F$ z$5bGo;54}eaD4!dJS31iWE`gMy$I7H_g1P7ELK}dFlRJPfg~uw{RB~YMvGmnOC8l_ zmF;t-hZbwLytmI^G`5y6OmQ(63QwFctgv3N>WLcg^mtOnS--?h;jYVJy{aq+2Fx;% zPlY>NC)n9-3Ro2#S_IEpd)g^g?tXU4iqu(op^8b92H6u= ze>-~lzhEMccorF+_}(9#2dx!ZiXKY%2HzgczWrqY(ClMsF%4j(qWj@c2#(fOL8oEK z*YK8s{y&RVW-!BIwL=t`fDwEQ`1i91JsHIEEp#Y9D<|xM29;3hC)8s-#nJ8I)IxN$ z`#CNru$E`6Jo*UE(U~-FR_pjo6Kx^d;}sT|Y5>8^^1D(jpqia>Bypb zVfy3}J@TGo;K2z4ir(PbpYi65Nuz>DAzO((HdRQGdZR062u7q)N{}e>GM#^1Y1#R|AuN@0?y^G0-^J_c;89%ErGVhgUGWG{ zT-@Os`vjENYI$poQtSJpYyuOkQ`D(ke&st9pZ_h0-oP;5%@R$Ykx;{SIw=Ee%%`Mb z`T+brrzHOCDy&t>y~)uo^;{TkC5V#EkHj#o#G5nYho=<@5fo3xW2qbd+#n@Q@GTOJ z5siVkJo4cE;_U9C>_qX~K-z-cvpFhOOqK+{#h&{QR0*<}R#>oS2LEj~0^y*iqNv3v zfF_%~3ESt-t_0 zppJBt$;UA9lhvlQ8P_g3;p7`iKWYoy7 zUW337sb(3sA)rctCILLqDrVH)Q2lIa>(0Q$s%^H+Q}iUuI6WrP-3aUv6TPZy&Ot#EDMV9{-aGPw;aedCmFU|rU3zkN;kekj{GqS2c?l& zOLr;_$e#i#ZL=Fzm{-(QzBKZn0DqNUj+ci=7(Me7T+J2|9w;rrAG-caB3kS3dwz|s zcm?nMhw48452w+vh5?`&GwMTU7?oq(y(fM=>+C+~RG#_p3RcAu`?C^+3M}Qean}g( zmYyjjbN2MJ4Eb9l8&aY_t)drj7dY9QL5C{oc`v_Z{~@~|P+2=($_=Q8DAY&NN19^8 zl0lbuPh#?FI?3Cq{a&(0GJue`t*eg(Qqqv>YrKyB%(lgD{H3R~St5@R*Zf!za`br} zg9H$?G2sSi`SSQ%VcGR)Nq=rpM@e|wA4MbUt~`SGump?bv)lun9q5~@86t0_U^y$M z)LcARFYa^9a{0F_5yr4>|I<#!kdXH^?ZD}Z=wY+U>)kUcOJquzXE{QUt9H{;7w1Dr zZ(Mi0y#*o2Kskk-wcD0aTblLcfh=hg^dST{4_DTLm0pSf858VWSMb?7tQy;Qb)R=K z@jZiGojxw1AVX;#vf+mctVjLMMzJih_#x`eJd-Y`u9QIacuR>h(Vv{r(Gu35M}qNH;_CAVog))^#snO z>g^7Np?(xM=+>TnPP$yB(_F*8omyVjyrKZpFvmvfo)jMr=$$e!Ut;1}7b9t&)Iqyh zA%?}PD)(*fody~g-gV$(rxeJrOc5mnTrF3(rpNqhgcwip1iH-2e3d1~;C^^Fw*Vn7 z>-sE}nV3sYVMk1vvPD>^o#;wJ&ESJIse`JKEWsXJ^NCMdcV}86MqG8(QytO@12eB< zmttO79di*WYP@N4*ruk~$mBr!$HEpJ_41oGZEy9>QjEu%U~8&hvgVg35ap- z++Cf5?-)NDQ3*Fzr%dEb|6s>sP>NX?ddq64<*N|xV;Z)Xtd38%M>Gz!UG1RZ0#8>G zda~Y6^5NKS%au{SqnXa=^Q>D$l&Q;;ApX3>8x7^^k)Hrp_>F>6T# zjdF3#4d493ECR+QZpid;W*`Df2$nVxV#o0@E9xgorSoihf0XnR2S1C=OR20LH#+~G z5o0ZFK2LsB$~*56+L3Mjg(<0RQTOLng0vmArEQen_T~qlp>8n>|0@s0=+-)pBSB7n z-60fjStk7C9(5~hXAHaJEU#Z^_pFW4(7l^4?R1jwIaR~Q``YQ30vVdUiBzuq;B8); z;`8u;*doLQ)cNI%o9|42nY_d)-@BwqRT>HZvq17dp(B|=YD~98A1O3S4f=i3yc)1_ z;u*5v+Kp!=de2ty-X(tG%GPT(1o}M4d8HmxaslnjHmaLCX=&tVM@hX4g{EiS>uk8I zr=HEsq|!e}Q?nxS54U3vN%REUBHsrm9VXsblAX6!kL{d#>y&Zd3Jclchaqs{vrwP)8H$}U>yH)9&5NCv3Q`AdP3}#9U%W_sBw6(@2!)}wLQNxZG32y=a=z70 z?Cwctg8}cDPU!PxR)ZTqyG7exqT=xE&(A+VL$B%HK#|jxY6^*#tX$M z2tI@eyZ_%f0=zik3TL;shxp9SX?E;{TD=2l zA#%3=Yb&5)_A~#tfB>dB7oou`K7R8I8uD-AwXA<}L!YA%?2<>PZmQ%AUqYF|!*i1d zSM|sy2;vamcmreiENtOQ8er~X5EcfqmMBDCpwPHW_o8txq!|5DN>1oekJtFQ^G@TS zaST1U`QyN^fkf1+ax0h{b%Q&+gIP3j+H?KUj zU-6TcD>CfqAh1jM?;8N%-BKc+8^$!oPZnb ze;CN05ulE+#8tP!j#OUm{(ieQ-UrATgcIUQ*O$`Gf0YXv2e!nM&6QlUQW`_2dZ!ai zg*KW92g4aQ{2TbpIggkj#9G8aMxnhJ0IR1LW$;FJwnPip=eRd59S?v1OuUKgs@FyJ z!&F z@F-eaz1P3fL`=zgg@u`JVyr(d6NU~^ZK-6(di+N!SYr(3mDC=wq01YznvVVVg?uLe zt+U#@taPn<I9-X8OvPx!b-f${{Z>}cs?xJ!fTc&RN4s5q zZS#t_(R1;CIm0gM{Cypr+`E*x4q#8I+@AT_+Aq^V_xT?gFlSnwO)~?L5uEfVJSR;V zW#j2vDMOWKV+Mu-YE8^a2}S+Hbq?!fQKm8;4F#!YI~M!eB(li#L?!3<(Ah|~uB`PQ z%6f_TK8S3;wYWrwz23ncF+h*GD*NW@MBu+7QWmcTLQjnmojX}3mtea}C^zB@$WTFE zdRuGr-K-UeW7epVNLn|x1!>t-A~3+=BOL=*nf4!LXvs6QvTk}}tW7}tRAD#72o2Zu z?9aegW6A^qU3C4G_zQPS=#kP8gL$I(de6MqtZ;7uS{v6*?4b<=@$Yrje5L(Jyk4R( zJ>N$;$QBJI6t6TVQw1kYZ)H^!aO}*SYpu}j?dBD#ZI2Wk6Qxr>hua)*xpiN8(>3QM zz-#l7U(xF{usep1jRW{eHl9NMpcCOBOy1K@}> z3$7YiGl&zvOa*XvS&Mtsr^RiuPRJGYJfGWaN;-fd&0`|& zN798~dr7yN^J^@Dm&t`g!4LE|AbTBr=LP z)&Xt_`E)Q+J*Jo*+}lpp{!?+p>m22*!U4-9@A^sXV0tOg#*M`w?Knn)wRL{Y>~_Kt z3f*5JJ-MI>vK__ilu_34+|jC4i4SH4kB^r^aa3=NHsIFaGmR)$noFxNsoNcANef7=3C>ciWO1}=t`Y$4?(UNwke^uOtR!{Arn_smyeE8#T6@xou9^h97lXy_wgCe^=#H! z5GZ)LXkr?&^GDr@gsZL%oav$=MT07YoS@vb;d?r1#on#{_|l0Yp}V!&@v6&pzEuB- zmx`y=v8UH^8K?1KmAYVWy@t>tV7+`7_xA+ul#qem|Ao32I!SH^z<$vUuP*wzFV#VG zv$=B5_{&~ybn=5=(ytmBpcz+s3R`nsYXOdaxqBh7C1f}w3dxabM~=P>X-6Wz8Ik6! zpo?z9i(g~1OhEkkm2>yu^SEXcdhgKlncsD^zJ#z2L*;zNW2q->Bv1em6hYh=27#E^ z+u}9Nu_<-WA0UpEp=V-RRB?`r*SVtp8+_K0EpF!EnfLcPxM3$O23eZLiC;`W$^&U0 zRA0`7X(Bi-0~osXm@|svl{!L4E^92_QK;o?dQZbyuaYe(bB4a0%8Jd`T=_m5eS!pq zJzTfEyGlL3@e$pc8J^hOX#l~3`CdP|?gX2McHc})RrjnW+^{%R0V`Ah%y}6LE*(Mx;9@9g6#3b__VVHap*KGf_ zzB!gFgv0#f6U>%j#$N zdy9@$fF|-m^LDem_y0y5nziktbQ-(1*Geiss=ojEV&VAsLe6ax6U{}aq-UEZormdGp?|J)V8TcU7=#FvJ4x>C;`vL$Q~Z%rt07k7XrheBae#t= zpww)34)M9G<5c#tS2$c3oQv@w=h`1$A4rW}^rprcSt|3je|{8k4BwrLcer6D)F=Jz zVr!3UOFMmoghm~?QRYSXt&;=BO57nRK`=-}RQ2QC`MWqKHVO(){&aqum3V6D6jyjN zgPS(f+QO(l!=F)ry1Q)j@S+I2VQBk z#2+R0e8V1^>cx`r6n0f!1&adkTcr!?!R)Y0*DCe+S@#7&#Z4ebbysz;r2}1ulEBf% zJdtQ3W-N;Z(P+At1FvrRCz3xNgNgq}^FQ8*AI^Mw!-nB3A3B=-BzI^S7^~>V9U{!j zr8mG-j*%DtzC&X9xe~TRKQbD;koYVa?`p$URx))#zqRw&x;XVjgItlI_n=J}y`0Fg z)wvyZmPjC4HxGktU=zSuyt>{J^R5!Pd5X3<-oTt?`*dD6lL@rL{y>^y)gP^j3d1SE zbUi-fbb0c=%Rpx&Vr%lQtTSQQj|L=UCF1IROq|Mt1GV>(o&|;lkfs@y)g`_ZVos*7 zyE5{yP}`Bjc|Nn9;M=msjlY6&*_tnR)bJlcW?d^}gzkYs3bT5mF&y3K9r1_sK{--= z6MRf^SQ75`N}Kg_17wUhrbtG9LnuubcSnyn#^DoHc&Cy8%~ptr785J)GH}G&j{YTH zgP(P7+_w3GkDoveh|2Sx6V8-h8RdEQ)4V0vDJ|+K!0dwWW%YG_xjK28e5~3-e9eQ@ z922wq@Bi=5e`AdoQ)pdmhcUwrF{9j%q#r6gP>Ks8KL%y!t}SwA&P11MCn0#zhz8kQ zXL<_p#-6#FGM0c@igK|V5Y3>Az=VAHhTUL3 zeHkXB?0yEqQ=tgL69fP8`$9W8su(VmZdLKmvs?1Z{v!bQKmulkHL8i7hxP?+XV%4+ zia+?!@PD87QGht!TOi3sK{cVh9sZw6@se;q?fqpw{JjkM)YW4SmT3=+k2fU9wYH{O5>Y<69)wC! z_-)`wWh^S`?W`A>zy;p*ohBl-69YQg-C%+VbeOv{4aMmF@A2bzJ@gFpBNp}D14%64 z(711c!D0(-!D=>%PiviPljEVlc>xDdj!^DNhEE=$1=vpO8L{$_UqXN+9u%117 zz~8=rKEOE#p>18FL zy=(0uz53uAr*Y(}=hF1QKdyg72K?Z((ou#l?wsJ4^4egUB9keU_9aRzqg(1v90mn{ z%a2JE(jbzq38N%Orp!g~JUPySveHe}o8I5+2S@%ja&$b*xLT&+~+Azjx73Wm@Ag4=9 zS7aENz0GNJ&n$v?o{!z;mtiCkS&0RAtl^eUv7T=9T9-op#xI1UOh2QuO@#M4e)-*Y zpDTzMLDf#6g^7W@t)xSIp36@()r^rnXP&Zg?wZUj+;nyZG>CcxTw31$w6zL-MN zWT+-0bTp>YX?9nTkwEjUopn;4ovS`#a;K9$xJ*_q8MA9^+gN{Mu^OnZ3cfUVo@84XI#k_6G~|Zz z8wwd>6S;{4mQK$GFB&sqCLijTp7@p;O3C{qk_kcw(}d1vwz;4_{1!MM)~5jj}E+a!lv+;x|(~KXJ)I=J_?g104R2M*51c7c=LHH-eKl8q{icc*^H5~Wk_)g&8qoT z)1CY~vj>+&ND@@2SA~a!bQIK1zmJx>?qi$V#E}o))>_{-Jhqcfoj7murj9L%S_VG~ zGyD$jyV+hdO;tJ!Y+HzSbyRj7exL_Rs?(Yf1*7u3NIfQZRcC4Qj{Wdmt2;M*Z6SK{ zKYS|q-hG((l#dTj#%H=)>pXpt-o2;rx9quQ_`euPOSbNp-Du{GH=$(IB?G2P4`ATkP< zjcOL}y}R3Nv2!Ah^4TZ^@t zyQgAwc^P~YmD;|&!w0FihWHt7S%a%bhb^qR+plpV2&BG@{+{NOo3k*CU2mkIKmsX! z`J?8$c1FYus9Hqn-H-^$=HLML8?%8#Qkgo>Q-QaX2CBBF6ShnJ@La3Ws43uF^) zZp*otsCWFQja*A9S*_Yot-?OY=*ies;HwIIl}4~`p^Nd#fD3o)(x9|8=e>ey9QLf# zsXLF|NqY9G4q;H3A7W|QbR2EEB#VN;U5EgCTY+nK2Q? z`9!I9`>JesDaBjB3G2>B#onRjHG_yYCxWbP6vDp1h4Yg_{LT@BMP@)d$ONBmY7!=sbTZWh~weeAp|=L2_Y& z-_Gb8zwEp`=FIlTP#V9y;s|xCQL#x;Haj{^90M&bWa*pG!GPnKI*#)5dgyK2dY@Z> zq7}>Ky+QsefPf5mEab+30920w&^O*EY4BR+nCeZ*!l_m-G8gz3ee=ec@1>WK$5_5 z@c4bom(az1#bwy8ni?Yo9U+;6_EbgIjx4w%(>vfOk-iGqunoaya^=Q75?SZ%cnh+6 zf9Tr{ZagdZ3W8N+C?z5ZL|HO=^l$(%15vvXTvCry+Df-j_oRvBsoK4Qm>awGD4aIY z&iAmlUOB~h7a8q#bwfelUXeaP8A=1d=t)`hZchmF!0tiVUg`y;9xYPIp4!nL)1yiq zVSj+W1AsT9lU(y>nLaK!g*{!P^neH7TqQwNkf%oi;qlfh^(5Guu+i^n+D;$Eb@yvW zKdq#Ueoz10(m;>u8rXiLQ+DUW4R=wHrSl;(9LIJiF|1YeDRU6f8O!#*z5F5r_ia9E zg@lpGaQyiYhx#t)t)XG4iM-JR-X7sP=Gpm2_)t?Zm%2?Q6UwfB#z#ktcL~GSE9{NK z*Xn^e@tK<&kqril(ytSDN5YiytoB6C9sXQBLDbmuiI=}4J$%REG|RDHigGw`jp|+S zsD4^2rCT87i)@wg=J5LkM{iZm9)P<{ZRrOA9aRfjt-;%NVnTb_$1d7+%6%y75JpOJ zSaf_37ki1ZL(}!bw1OX_XQ=f?0MUyw?e{rP+EJ{@sFr7mJ|ce~*~5P)@S5-F3|qAN zb36_{+@49d1CZ4lCgr%twb?^Tj86;h+Oh(YSrfrd$vBM=CQn#lgJian&7A&%_9_;^ zjoBlGz%#Dl&#C%Uu?+un(K~Nh9@;T!hs4XLVw9`1GJxinaL_6pkKfAi4qY=@wly%5d+X+ChYhG>OobA$#AMsQ3-nNd(hfNBycYm`^? zpZPu4AM86hk3F->DRja#U1eex ztg7Ev+wrc;nt>fch93r7nAdlsk~R*#M}P-4v>~RO3Qc{k%+f69IVSz^c+Sfa61`4# z1cPRer3e3%iYIYd6ZpS%p~OEvtdkihj3QddG;gs0mjT=3IrPTf)R;p;(JQen$2Iui zl!Q%plD$l@b;Z#;K7wR-P)=BcQI?36enL*!31W*s{l)3nk%IAq44t`R0}JndXQlS9 zN?3g1PsQSQRj;eGS>)@@zf{)a!xl1^Wbxbwe+<2h$6;FZRX@?aopg8&g{P40k16j)zJSjK=^0E(^9;x^L`)4w{zZ{P_n=;D6f4eB`@X)4y(J?uR zEG7E4Ohh_|fg2_Fcx^Op2<8epk?&Mm>CB7Qyr;ji9D)ztd9G*BT6*gp9PR*6Uv>5U z3gm{G@kN=(+JYaRY9oqz-rG^re;(giHPDlB8&Wf0Lv1bQK!p(G(x2y2>P=yXM23#3 zHaWGT(RXdIdwN#%2Kj(QGLv+kE@m%str%)FZrbv29D2&@tvYAzi7l{$GCUH+Y^WtW zBM>`}i88a2%HNKGzXy46sbj$kwf9JcE>nUymybhWbU&W+bt21coXX_rg76v!!;edW zba?wx4KfliGUXQxn+iwIz9~h!?WETyPk^Kl-^- zgji>>=Y`+lSWMZPbrwc%1}X1m>$GuFa)Dmr^TDjq47%Sxoqf$R$|jybDNeIMw!nJs zNPO>-l%>(pxQ}Vf`@%qn)twBR_8M8jt+kxNOlC@F^ST_I!dOC?-A_n3)7iWB_P}%O=u;8HUV^UEM^9|L0H6kjASw< zH$58ur=&S6B)%zWrldpM;WQb(iND=19N^>2@N1oAxD${y@R!zZL#X_suP%rqcET9O zAVm4Ka(vvcNg>w43+GG*#dJ>&{hqEPC{FwGqJ~5fQ3ytI9h5(1f2vgYXaxgT3<<>| zYIv4r;0PpOW)Ky5^TPQ`kPi{m^-%f@)81D3BGPTEw^o5E@UQBlKTn{$AkskxgVm1; zy@`|-HYS=#`#%H=ENGCB%NAE8EK*eOTz@iN?{if8XMHZN08dfNZka z#P2Q?mj*h13Z^L+Hlt!>9t2nXnlIdi@?}^+7Nzq|7a4)|7zTnI_-W^V-Kz6d zND8?&x$rM&{jbvA#W`I#sp5P5M*dRzs>9;>eX>bh(>h7Gze?PW>pb}z~lk!-HFs!e9s}g}GRrKfmk1lsB z2YeszzX#YwPT4sbf=^A;?^`=OooB-j-ph^5SksX-iWbhMh%Ln`=9aheKF?ATX7T_- z(5J95-kJ0O0y`Wooasx@*}9KYA%&Ujh8yEFhUAOWnLYX2N-~k73W_*i1G(M&cGhxG zS~wPSR5;o$er63Mn!Y&P<$VM??*HH;g40H~?Jc5Q;fU3i-DDyDIKJ~$JTVDKRut5{RSD;D`lB29~YC9sFz%uGmIQrsxj zA=8f2NE136J&OUG4mc1`zLvu#EJZ79FYGbuyxlb4)&`Fc29--}uN~(Y?^Emg?+@mg zjiP&9T|;Cr%YpolRtTr9b>^gyHM8s|Ke~ zihV3d@4?Jf`{J;_Ec_u@KHh=RQ-iy}A{p=)4lBR=ItQcHdmA-#XxoG+;OL{&-*KVM zC9BQjxWu499Mk^eUWA(iS#H>Ct|Kb(JQ2EP1J~Ehr3+x)x|EtaG5H=&xR0sidR0fklb^GffyOCGa-omk2)hZ zYixu{VV)bFUl$**n{4*h6PUz5_5yfHJXl;79NZ74ZAIgw={OPm^I~!r)kSjGMkXn- z?8iJA)m@Z>%sKWg@zxFs)ZD`&MMg$XY@PdPPL~gLI@w&zXnat`YRH(oNosUgfIT2J zG-ULkAZVu|LF3{Z$XtGteTK@|C%+M3FoDNr%wBdY9)wH*$0i`SLfTEG?y0SZ%XLhM zT@GZrgozL}XSF7oHba8QjMRHIV89RswYnpEPDRtypF#KXtYRmcw5sPfzm-zTo`S3j zeVPtMU~j(gvevNwQS>y$K(N47xw-KjyLr_B2iGN1pB|$QkDNcF%zrPvdl{ptQjHYb z-~W}Ud>#6f6)4~9Q`sa7^o0rXCMs@zgV!jvv10mY&28<`{85t}(%YAJSJO)q9QrZT zakHhh6YfFn(p*mKp$h9g@3_F6CVc!1{vr7Qs}#7ZfYcBDuHkE&->RVxCM^6_O*&TO zg=5Hl^>O~qr&=ch`@k!?9 zz`tKF$UG+M8!5W)b+sES~;A3>V4j zU|=Zwnx5onX;@o~(XGjYiCDu6 zY&n^h4rJg8<(3x=2;x(ln1Upj?aatpCan2^rSj^Z^D9Z!>Yq~=Kq9W1uIU5_u9-wU zl1aR8uiB0xAjAk@N2E#LHRez)kCS_$aE1>MTUhJ9!{fmOd5pRD7XwD%ofGdqRE}(l zix0`E0$+B*BnKh?YrVOmU`O$;+a}Y^KU7(>yRL`Asu!Lv}Q zt4`N&kjq<#VlV;vTWv|H;$*vpVt#S%KYj>M!O@SO zR177Cyf|8grZF&B$_MU0{eEx7;&5m$a2GV;0lerlf2aEu;J3%EMDK0TYuz;M(LCAh zpX2762k9CKCF7nzrTZ4+KY!FPeI@r3Ixu5`)oY*`HuHWP4ovj@DKW3G|MrGN(bINZ zS!)7PWQuSzR(BR_&Bf~Z8Oco&V0@c~BDF%qOUZ+F`y$X+;JTJE;p?w}6gvFcQKp;aAEy=hS6gn;ovbW)3#<$tcgWE080>pKpwP>Dc$U54rN$HF!k+R<^zPo6Ht zF5QK7W+Mec4R>CFMtA62P{Nl>DrlGKlATbtuh8+C@^#z~dHlW0xdiJwxozhUWEX46 zXJgYlrpfiH^0cp^Xa_dh&UFdnF&>t)xMju*ihMGqBiy8?O`H=vI!z5w_P|{EleRN) z^ElJ4hwg-axSKTgl@zqX_|#v3c`j#J2~N3S10KFygV zJ4d`OH}#+1v>$8F7)2}(!9VJcvC75kemR`jrjW%z`yDSQKk_`@F1q3s3zV`d zUAh)3;$l8e4gg}Js!e7NYKl<-zPfWOja83+brSj@`^35O7yz4XSYC^|@ZdDGGe`hYrhp{$=Ik@W9`5Zm7CIM%Va_L20dp@6jLAKH`pm$7*0v*PzV=Yfzxx|RBr`ITsTbTvzhH&WU;(}+cp$lRDQYwd zs*8D{>zYnSEX7_HB5Whu3)Gi_B(%sWz1~~x_;XRjqg$RJ^;+*NCw2@zheQw#=97+o z1F*chc|rpoOH0C|ROlBTAU)YC*Ux-I5i%yYjh}5-ng;_ZG;6jJT*tihP8tCZK#t4* zkEXK>Yr_B9_(mh$(u|f69Ss7L?ha|x0ZNB}Fj^#}Ho8GXI!8$lB}Rv|l%%vcI-mXi z*Y&*IwYS?j=X;;~{v5~L&(o^ubpGG(M?kw?6h9(%`Mclx&$^AXA%oSmN!Z|*h08nx z^>oB69t6XQr1Oh-FUp?Fre(>ezvPRKA;m=Iyp^Si`{Q!;bYr-1SVf}Bg~dDl8LU}! zY1lO>W|zBsqG!dxNJk>YC^@$@ug8ygTt!-2@~-F zShMPk?TB7t{C_>E?&;`=rG%VEFWc(7o2jYj)>jH-vlOMz^{LeOV#(WGa_WH!yS_R( zihU9TkrqlUtD_)#eEIs4QJN11L3{?$P=e*eAHo$;5t&Dx-v*8m;;T1wnLt$sw+4mC zzgto}PMVL#1I4JBz^U|OjJ8CADvRko5c*#Eae{FWJ_z&1l7&10yIj=-Jm}qGS0~UI zaEkE`5?`40%(91jQlokH6Rg4;z0?4@jjie?9AS}M0Y;IF3X<$D9n6y-f9z(ZzuDt6 z9eyL1L}asCB{v#vDi6o!(Lq$lg3xn^kb@c%$lHoa zLdZUI3)%!&j7}Xy!e5F%rRqiERMU1_1O6cUI)(B6WrGU$Yj&%(hpqy+^S=_w==;4w zuYbl_yMk?R!LSdeDU1{~qxy5;gCTb<;nb{XYHl%UjG0_=BQSwIA%0xA)(acLxYsjG zeux@r_Lk^k%-+Q0n49pPi4b+#TK293j);+7O?A z?za7r_?#jG=ETPN?1qEiYCTs*M1vW>YcBC*(qmpxkQ~B~4;^!7p^$l|zCg9`a9_eA zRT-psXzy~LgRCjpG2#}zH%pdbuKHK|P#ru9$)4cMaeT;p_G1yj=5%;-& z(V!nRAvgZW!3CdnDYwO0cUi9fYQfdg)>Xi{&&*kMrb9^4hdTC337G(8BQvRG{#DiV z`4rf8S6}%W`F8`Ylt@qgV8YTRbYEnndb*l(z;Tr5a98!V5_6obpofYhujA!2VakuX1e5Yr0%X^9Uu9hN>CdxW(Ey|4CXYl^4sYtG zFqOX+R-BbhCIhF;vu1+-b9!F~P2X(H{pX%wl6}3@{`M#C?HSe>e)>!9$DIyDLT#f% zdEt;kLy76gD$aAmi1h$JXY0?b?oN?ZPNBz9jxQ``j=IrjM7(OId4SR&N<%nac;Hjg zQ%+>u-N_Z@!`(mVavZ=3!QKGdqbvbrL&7){!{5N?IVUQd4Qr*blT2|xE_`w(m*kYk zs^QPtaG9QXFkXWjXBJZzKu^zVOy6-oqUX-JvIw2h^S-5=CmOvg`sMB-M?a4$!O_Rj zj}&+?vmn-22)nhJ!wRP~I!aGH z^dPRNCZAdkvt~f`odhl_KH4;NMuFPkLNiA?Ng=~}qC)R<%5AmXR9s-8BI;%3KrxHD zar1DJE4M`bW1#L>MKu322cosZA^{w1qrh0@|F?j{EGOiR#vBkD@XHDL4 zH*o~DrnP*04b5ZJ_kQgJeaBq8b0!>C)=r81{%X6y2JP|^EbH}a??=a>2VHp^g*`39qq^$Gp0F)PAta!3OQ`LBvEDP6!r%bmFbkY#cv=Xxw3m${A*OA`H;c58+l!TuLrs{DazHp$*pIFnVWM4DL;wWTPzi|xu;X+gvHLnNuoUD?hduxb!U*J8Q zqmtK=BR7+?`V+p!dw*sZdZ_j#nB!qW$I35;vK7C|z*M>~X@xbkLJ3&53S2%@+TP0f zrAz-biD+kL^6sOB7nRp;4M_Sm?_l~ule5BBb)@Y4UJ<5pyh12W_=9=Qx0S82=!$@y;AOh)tALHp`&xOb+?f4Rz|> z#6B?~QB6k*MqqsVgwZT7OK1P9VE_GV{Np`_NOg7rYB;O4{RXHoq1i^8W9e*J*?1Ir z|EG-VYsh^{=h002p6Y%4$r!fnA>)EKCmd9IMZ(j-mj@D-S&Q z43jIP)rC+Gxn4wY^&1o?6B&VBH<}&Xp^TLC=VRM`=e?eH0;;F(kL)2Z zrqGVZkz#+D5S(ju%?Lr96dDM%iypf07V+5=ds*Yd`;Rg#Pm^jkHqK@%3O@zO&(m{k zE-ywU<7z8&O!YddG}^}df^K~UsABrpQevoK3h95I-S+qsy!Q%+F8$dX0+xL6_bo`+ z`*~ZUsLbl*s(s>2M>glQL`S$eR;KgU81;DF&j|V@a*dGybE9+ubxAq zw>?4TDv6(*2yhVVdX-74ac-rj@#To~ThMvU?dp7b_=9U;iZZ4+~`hgd% zBupZ@?ZnJ57RsESIEX^Py!-9jv$nesa?NEEt#v4a*s2A#7qBy`AC$JTU*D za+OSK&(NLtT+So71(3IMP6PTC_VGHAu%Tu1%5W+`GgQ~$H_uP+T1ltit}kp5avZRA zjpG<+ezS<9I)ZNGGEKu+{-|~ToO8{c{B6^T*o@+H_19zn(Ks_BwYA(z z@zqjio6uHA(-Egz9_K1v&Vvy%r%_{Xn_C8?!G?2gTk~R0mQh*kH-XT1sVbh%F|D1G z+qZ}=Til-=D=!OHy^qEo3WpHr9uDxxs`lKa6Bm}nlsbcF-Ox56N<@F7s5$=plv4rb^a7@JD>!)nmhs9G{`WLPMM0QKFR5!`22eL+v%n@ebTX53i?o}{XHiQN!@TE^a1b}M z{K{6j8fFIDz?)~@FlN_sxHm#ba(_T?eKVwYmWxtdH=DBTR~T4i={O{vE2 zG5VFKwPB-P&_5OVao%J?>tY^%G->_gJh_Pno=wD&YAg3O$E2l@;TPqb$1PaKegGwD zyMp>)?zI{XJOzm~kY#0MLCDh4V7o=`3Qa1Xg;zm5LXW|+fw3RgfB4Z4#JSPY9`rz{m zZt_8eB&x=@zv}YTZmSxlMrTQ!x0~`v=1g*GI#lQn7TY#zio9yoWL zfBHvZ_FEy35tR6ZH^1d+Li=~j?ht|xzXuVJ+%n(r`v7RJwUtCFPW88D(@;c*0}qD# z&IVU?)A0HyUS6%waRP`SpDF2Y1r;A}8|) zqb6Zg><)JvO^J~aa(N_wTloR?=Uk+=| zn7k8%8f1MZI@P6aW9@b_XLl+I>)92u7ha(>IXXoPn1f#YdI!e)D-(_9y=cXn;PO$B ztn~I_^d4v3>D%fWu2U0n=Psw_eqSqUX1xx&v*p5_buFFZtLqP!0&Nfu2F~667Wg30 zk6~NBB3ZaW2>8tVuoU9sqbW$dQ9X=x+% zPp-p5EA1Iq6b-lIpiyLITrI9O3oMqa`e+hJOO=no89#M|eb-=~%5HU+DOg)H9V$p1 z`s*QhxkdAP_Q~q&fvfa+wcFL=y&C3mmh}%ACF*JSF|5~`(Bd+Z3^JpCfqIU)&)Bcu z7^3NcCR{Js&SxE!F6Fx2)m6j9(Nl>^VitSh66#N9#Yy@dpUDuvX`K}1635qfO_Uos z0-e713nZOKMdy{#7>Wope*f{bzeC|y@KK$&eS*8rU%G4KWS@XXpoV9dj%FQ$U=Pas zmu>F}%30a@&<8)_x?t->4a-4a zlawO+DO9|Rr;s_({GD58Hp)}C(WMtzMSm_c!rA6bb)0Mdi?MVBI|VzroMMOP_^M;K z<))pmq5mdn>n7*wK}j@Ddy=|4bn3UdC^~Y@Zn>hson^-pu>$vhuJ|E`=)+~UR6prl z?4{qtHE?ZX^TOO`*fpJ?PV{1@C-a!KC*@I2Mt2$tB!d4cFoX*ucnWX_|Q zck+U7))qT#F%(>_ zr!+v3kIBnA=jnEkigGuXZhrnWx_@hRdpj?ROJyDUI}wLA7<=$(?E)IosZ&_QjPP%3@lL0#3qCuftTvnlJ_ zl5#^+mXcz7@`uaYaN6Z{vP8@c`r4BlSG}ng9;6&;b^lFzo`dn$jedQ~V74q>2GINR z^hY(pmwCDpQ8&PqS1~kBon_FtE(#}q@bc#@%Ia%KJw=Av;}158*Ny&juW)^^K+*<8 z0&*MuPxc}6fT-1iDSoAK7Zp{o+LfDEbh_@=`q%tzHXFpr-b9XQ_{XVJ%2MhVwNn(p zwPdc;Mr-R!??mOY8r1#+^(if-pOyaRhde*|dq$QieI2Ql4aQi*j9ax(#vq+J7*^Dc z1ynD+VXo&R+E7%M{(%2--gah5?rU6>_2heV^y{`mHwgm^E~(BUVlta4c%-m?s%6BkVq2pyY$f^rSSjxSa(KL!hAu1L zV`!TR(2qK5f$N880I=W>P=PF?T_W%Hdp=W6iCo@>G)+wgT`p=_nvNsHC(nA6W$fbp@`lVU8;_*dMeYtG^<&A!>OSX z4y2f&-zk)eprJ{nSAtW)xO<|}Pmzj*uN!_&KJIhvOn-SB!jv00idm4F^XvJR`Ut1s zTSLs?I#F3Sm!`925$>Psip{5730$EJXmG|K52_H(Zgly?u|i(-JA|9o`W&vi6hrKq z{<%tu^;NslvEL|TzgjT~-|zS;wFi{XA53JmDBND?_w{(|>U6!+GA#~fIEQEkpRbQ0 zwq$<>2#@_WKqWaR?etN-U{b>_ihwpGfOKi_yT69F#$0=Xf4@NwSq~8JNdd*FGOuIw zA7{c4^i%`VQ)C5P5863j=ruOk+jt9vNhZ_iSa6%*{29)gQh4%i$2*H~LtN#NyWZLA z9^&Tu$!MX4JJCchFz3$eesKna^7FO>U{F>yg>xdb8V~37+i=@c`LCt!jOw=+my8-L zo*zr^{PK-0u)UIBx@vP&%{q`yBgN0m_FoPH@qL=IRFCNeN+76rzrBrLU#!aO`nL^8 zBIj8@wHp<|A8+2`l>CrlqELb}TqA79tx;AFN>jH61py{QMF85iI=oewLrmN$-{m4} z3)>9x^*>FEi%I7l{ zr;US2YMg%i4{%eHYpfLx-O-XWg#Z)bu6<%Sw=r{|W|k zv^p0u}1sEb};XUf<3(dJAw4 zULraIx#t003V(=hm9SI5c-ULEz4HIf0Mm!=Zp$Yq^bSx}9s5n(F4)1m>D z<;A)1a?nthi*PZOjuHbdf@sb043bzt%2X|n?N-(ic|b$5hKc7Z54l3L;*OGIGQO@W zJOXrgcovv(UoFCx8SRk2GF5ZsMb>uW(Y)=sF5I`4pQN`}4mBc~$lT32LyL8ZBL(FM z3zj-MNyZ3Y+3}`0 z*(M=M8|jrca(oW_WWVCJ;#ZvW6zSw(&)%UR{&WT(fUk<=s-DR@x~)FN;0mdX<&V6u zZ%y$DHbQBx=JMaU)%f=aXoxA(68F>8C2JF4rtUwD&>`(@w(2hV+n8TCjj@ozUwo+V zd!vNf@q3+!PP;FQie_Tqbh4UBAvIk+>8UJPr@90wfp>L^hCJ z?_eD^SBG4;NBeR@95IfV7X&v0@V4lO%IGz`9pWiVXb7=}Oa`g-x?cF=a-sZ`7D!26 zk21;Mg=J=oC9O;~GPeu8%$#ssm%ee&gPFJKKbJ|a>|hVB+y#k@F-h=Er_wwW&j>|h zlgPoI)wOLaOFqve&VSF0&Sic5inWquJXe;3%YEJdj9c5%VcX%@*&({2rw%b!-@HMe z9}f9nYpX+^f0_Bn5#AYi+L_n&Go<1ickDIWjU1zaC5^VeWYIMVv9BPb!Djm#mz`Gk zh06s>gM$-8jOgpC|15%)@$WI6IuusLgbzVj#!==dV_%2_d0hxi%&E~{*K49q+8-y% zIp1w@ofaPw-W@;u8e$_?AufXOAT>%5aXhh#{*>&Qq%xS~3`~e#HCE6JdX%7WNrHNI zaB;0@CzLH$CncU9>w+^F4M25;Nk9tG(}|o~fLi^% zPbVwo%9$~dgqmwqF0rsZk zeif{a9Nm3ZFh!?)Vh@Zf>XX9PpJ5+KaByaCag;$j;D;RVzpWf61}dDU$tMiQX+dy9 zX`&Uu-odD0jnHE&qKKqvS~%eoS9h)t=lU~3oSI|}mN?bSU9AWe+;&dg^GLW%hMd1D zYFHrKDbW0NMg5G7D^d?Q`Pg9m$k8TuFsu556kFm6595>y=yjNCKoosimZ(Bqu^{^f=itO3{i-0QQU!-Vo&b>0-t+5tvSzHc^TGXG#*?Bvr@*YQj zAOP{yA}z@Ct$KkX<)vzRmJ9nCfb#()ke4B;_t{esINR=uwvNKcr&}cd^>`Lu6&%hq zeD^`95?5q->~=cg<*yw=ycetmYtnOsl4{=>e=E0E4FA4qsW#~?7BI8~l~97u{tnC! z9pvI3ls))%+X%&D;Cx`<=^H_Q)_hZnJ=3kN$NT%`a$0iLskQ zZf2971C3WwgA{{IX0xzKvX@Wl(bEXAzZ|un$oyKaR@TaB_=F@>Z$+b++x%o~&9aA1#QDSiZoL5qoXUQ5Ip?-uS!*Ge z>(Icy>vWMTDK+1|y+W>X+9G*>WIOp0KOo{DV4X54^s0@wcUfC?YDMA3kJHHqRJe7+ zh7$02S>Rp0bjPTpFpGVrycNs#+xZ{hsW||7nS*_%tp4Yd@`va*4oROnD00M20L+OV>-Fi(k1})O25W&+q+{wf)r`xa*-n9Oe&SnjE99- z5t7jnoY1?O^Fa5o48Tq#94VFSMddGvzew3w{jK)RcsJn2Nx1aG>2rue#JnG9O1-=) zmZj%5W-FyP{0~zcL4c$x{R=gw8iJiJP2=qWKuY~OU{p()ZHHCGd0^;WeI^MXZZ)o| z2C{=e4M93R0>TA03{lfvybGP^=R{ynAoUw}#QpG`eA^Pj0;jzAdKV${JTbBe2L4CC z^r_*nCLhtnosC`ql2XO4HV8^L@9ea!>Blu$^Lo~j#9 zd}@{e5Ru^MPIG)?!`T($&1CN~9rFGO467*tHq86`s@=eb@XKEjQPPLpkJVrXF((Z~^@xew{cv>i)xI8IREE$uaRiMD*y4I8+sXJb- zrWv@9GovdX>v9&tgO$0cUt=%SNSc+H=PDiF^bPS0;T+dJ#gMJwDXVD8GfsBrsozz! z4{XNelCzT^kMLfxn1e!j)8&(d6Hfm5VXS$VB8Hbf?nq)o(r?Ek##NIHQ6C{;RUiuA zt;#xgOVKR9noftQe~z%TuAq?@Hui1$?5v;`0p?h{?Nm&qoX>rs0V*{itRJ|k-YsT!w zq6ydDr~m>{r&V>Icl~&D9^r06&%T_$R^#|g!S}QD#P+IZTgESLrOKZH97w@y*~3!L ze$o9{r^bo$*M-O{?Z26!bzZ)XWJdawLR$e(pq=LjLyuLM^2lq)`tB*~n3!)Waz6Kt zK7q<5?-rlXWLrVzf{4kL=*gjRcssR;r5r@Mb7`@z&=Df-_d-R>>jO{P7Mg;Ym*ia( z)`S*Vy~YD-&=c+tqyu0`meXvRj>M7`$_NX&bXk8$r%oza1^r}9QtyRHxNP};ue0Tz zXg_`%Vz7hBdy(zs;kR-Q1pA=OhPU)~)%sq|7n~=5gHF9}O{)A2eQcZ6={W;pg&fb4 z>5ldChv#3(m1YY3Y%qeMI*qwKF$vJ;wznJV1Fhv}`%vr6_6y5huz`qYbxP=`A-R2y z`z0#F*$3WfcVl0bjn$v-GVh6Q% zy}kM(R-R{?kLv?&%p8@l5NAi%mLF8*>D>xZ>0jMYML=xnuV1g;D`fMxbZuoZQihQn zGRnRswKZ_7>A46NQI#rLs2?MLajZ}J_wq+JRmf zLgl$bE9szP9+Yx}w^6ct=HsC4m zzfK{YPoKzhu25+oZIqw;UT@w4!EEYco~cwmNJAnaAi;JGE}NGF5W%4FSdNqVAM;Mq&kakD>{nJ;IBal< zMhEgPhJ#Gfu;dO?oDH8Z2zt@x({zq}5=Jh@ZM4X_G&c-{S4OI^J$}R3oT|-I^zae% zpGB_n7&a>H&hslxtU^ZqS}BQz^f%YfotbQRexTx^uwlojSF&(#p7BvhoPQt9`xo{n zR_T>5Gxa6uz=JsiZ?Y|Ynu1D#PJK|Xz!UpeI>#1znf}`U<@eh83yUVu*CrPrk2fjq`8n4H+saoBzEhVP^LT7!H& z`(Vp?pn3$1Cc^YA_ND}9*0Q&$*)6WOnzB5f{iYZ%ulC3q+$Es#*Eu+yPfV49Pb!r4 z=Z@xDB1fEkF7ecSYOHIK2m>_e*?Yg;s=fqqUY&{nfQBbG@HsNpzTGuO`3jM|t=05J zyq#B`@cMrFf*qHKkWvn(<$^I>R{qnoO~k%!^lMo6LZ!xaTK$ES*qL$=`9HO;fz7}i zIvh(6M*jfkZ^^d}OXsEM)|szvrmE`p?2F3g+r@fvvT8;ZyTR4VckgA-Yx!owM8Gci zoyI<;<>D`P^?UnO$7lDNbH`Z!j2U;kb)^<0++VKR?ftyS_ObjJ{ooEY<`apSV)-Or zb0oQdb;nM9uYMuBmEFW*W`yOqR=Km@&fvywzK zyiJWbVFwjHT;);xXE3v? zAQX#x^(F^2+Wo?n=x061P4DA7x`A`u>iSE}Y34c_;fT}YQGho0b+&puJG(1R3l`Gd z+y2d$Rnr=jr{xMIR0w0t6@IyfK<9%O%ljGED3-;~tt8Ll{HcqjEvpf{Lq(zEh0g`A~qC zv$sMB?0O}+H0dQJ*bDkDJm^lr>|iRjz%ejy&;X^)PwHAx*)eI!J?fnnVN^hnd=ku> zfm5+>4nnkXC0(_GG_4PwjtY3{V+w3*?6~ykgIK3uzpAljf6_SBwsUEJO*8&WCER=l zi^S9?;Q_oZh}Y^uN2rw|RDU|+{J^4uDjC2tEfQipwH0r6qyvp zd%2cA$9p)L+F2qFPG-H}CC|aKyVwnj7wzRMGA;qi)tQvKcdFIeA#!;b^?2|peU!hdRZXWN&d-Z>ZpB0ImcC709w8~Uv|LK_Okja&c6yxRew>=xB=7$Fb zTBGx~y@C-c@6hC%vb%_@oR>!T=TWwe@*_DFzsmQ=pg7Y#NLB0Hi<{H!xapezkT+_b z{zrHJA#bksa<^bQ6mg%~>_WxF3#;!!V|8g#H4@0?Zf2Ues@!oLa7EEfCdyOo)MPUW zF9+=y26mkH=WxawuG4a?ptSJ&-aoIqvSL4cf5=PoLV?7t8{_Hq%Qdz8+H#_pFCY0d z^5}jmm<388KfYBL8r&GD$ycD$Ha0#m=Kt7-f9m%_@Tv>&gZc(=J5;*xe3~5L-%SNA z@cVEAC})ZOaA5+F1V^Hn6W+w=QrIc8>Cisn3$B##ru|zyGwcj}B@v-5%?N2r=t)z# zn5CWPQJb&w+^MJ$77s05n`BP@?I}OH`w{RWRMDIRLT@J%h-fBXj_1_nuuY3V%_%WLESb&^ z7CKQ(W=-)T%2D)adLNh*lu?mGA=W=?#~M}ET(BV$RgjekHI6*sM}eeL4#`d`am=?TEsqbK;2z#V# zVP_1i=nrcAu0Auvs2GKGpI(oO2nPH1)Sr(ComK*NI9mvRmY0PUJ(~=W28y+bN`RHA zd zl`n{Mfn|@`ivnxx2U8m7Fmx|Lkb!`*bN7w%VFsYM7KRb$!T#^ZK(f>Jwn=YOOgWt* zMth*VHZ76+$?D!u1`8uE`T<&{ya#x-4)Vmum!4PAa&N0seb|z42L#)E3*vJLD({}I zgG#?YJQ#TR)WW0ow|cy53HO+6;5Or?GLp!nG8EkaFv;=4*R!PLwWUJ7!u>=)K_g}8*?yR zd;|8OpO#GB^gpvQsqhQfH}Q(YS(;K|sW?~q+7QWtZJJ_inW*$aL=YH}jC%NwTd$Y@R0ymS`tpG3 z_wa&+qa|V^L|J~bqrGHo^SyAM!qcn`ZwozDLoMjTFIYTQw+9*Nljt8?Nxdo$T(ois z62AReYg>4>QsLQjSVVKSyVN6)qL)^HQXZr#b#6QNswj>FN26i&iMBf4Y&RZ9 zN^68Pm$+lI5O_XehS@H}cOMWDW+54!c9(@&v>`^}$qZ`OrU#~CRD@;oDPAPqzL;)c zw&m8F=PXGQHUWIwm&At3NlBCha1!k!q%b-6*lKeHBV4ICw{Y|-+pKRJv_gR1@tQs5 z{^yO(SHtb8`0S+ycH{l{f-6~ae5-TkyI06|To%YSl7Ffu{LLYS`2#UAdM7$Jaw-$` zc6bI@VJ5OC@X35;I9~CllV3=aWW6k<#*xEVh~!3h*gnjOt)o0Vj6zIY%RNULIoBVNAgrU>za>pMEs2bhm@`8q>|KIr zkeI-kk()|?ps4!LG0?1k+lBJ>{`oF~mBfJOn&9u1NBH}xf6UrjAM1x@18lt}L$8xS zDM67{XLCXI<)oj7e+6`&i?g5c!TRSpuirbOlSRBbYt6;y{|HVhgk!DkLy^0}%nP&I zWYb+H+&pVn%~-Gi3*E;tOncw=#TU%fwyOByCDjP^>g>S6M!iE<7{ph(Jd#!@j7NIt zoK1FRfeuDPTykVZv4O#}GCxnv{S};>QOFIyJ{VrCU4d6~AMa^@lf5nz5Om0el+Y|Q^DGOu6f zWH-A6tbUS1U0`kDg1$fD_W%@79c8fbu05Y50|?1{Ncaui#+_wl+NM4pyDg~ zTlinr8KZ}vf-i3~aa9AE94#VW=;WEpn$XVZ+m^S(mos9oaH7|FVm0A<;G}%*LgLx3e(X8ydgE|Jx0rxs7>|O?4wZM{m z8U#2x_>awj=-3nsS&Ap2`Ohr^ne^>o)wt#qgg^OK@N-%k;>Yq@IL7MvUjsJQ#&Hj` zCB)?5RUOchz&yDjt4EmvmoiVGJUdS~wm%H;2S}m(Ssw!5+&|`iV|}ERi|3;&c70er z_25Ld&*S?yw6cTEJy#|@QU z+QySz^7tWMMF{2H6b)jlJ;qu`1&c;Duc|neCMD}kZ1kGaAOxrRlzCvuLk-pPOKwV3 zz4Z040%OPS)zU*^f%Es=+{ggcp?W=v29qLdqT1o@X3ldL5oyXvlfN;dyLEAh)?)6{ zE6Zl&fM%`I6>;6!%UzY*ByYnuu5PkDB{W>HVy}wa!G`R_`qrlL)TAk@5%qW}RkZF9 z2HU0qshRaW=I;j0%f`!r(bAoKzMEl`6kADgB|9MZX~)zs0WR(P%=Obii{#UjcgQQ~ z8xE`amIDVBG&uQD^xq2^SlO@>EILU0FRCyPg7`hBMDTO_$8Fo`!+PQPfrLQe%$?)j zDYJUpWDRj%qA$Lp!h7d%@kbnglPQx3k!%!>?uas@{bG1T^h@P`Im7XF3wXft{Eyml zEGug-_FwmOXm_@~rV+}`mCgy?6+^|1y`orVfLSzg6b`jMsYX)I|8y!Li_Bq1XPl;O zRm=nnDo>>tvM28<62HgNHxC!s(V}rp|50`)Bnn5a-+lSd+REH97pVFZ_e%kxR26pr zd`g&9-DQ0HBZhW~)-lNYF}2X-3D7^WzbufqK!WxD_nWPcLRBFVLIICZ?4kL|LuDaS zb{~oCz|RZEfcpnx`Hy56P@J5YO+E5viT;Ug()&lV|CHVD#C>W5C6s1)0#2QpvtH;6 z*_0IhLJeRnC(PJ^!kwtp)N_E9Y?;j#ceYc{;P%Kh*i zXw9bPSt)@b9(RzxWu;1O3E&Qjzyp^kKgZowk2Mw-BVeOjF=NMlfEERIH#xciK1dLt z241H&aU|@Dg1qwv0;@wiMt4701?Z`CQGLrNkdjKddfNZbj|1Gofx1}oObBw7)XWwo zea|JQp-BcQeViLdd;2juAo#Qrr#9`+2!KPKAd{01}y|^NR>> zMvlidy5YmS(zS^`Hcq!nFb{@wQb@NxL5Xqfj}8mZbB8}c zySYY_Zck9cQ}WDHj;pY9L$vtok1@<9g8(Kn{J zgk-vnb{Iwp?3w&cVTuPgoy0WO$#Bztr+!g=DLEhM{;@%k+cv~?^oJt#h!Eo$_=!`} z!kFJ=9L1xz^DmB+2ltoY2I>cf-aO802&fKcTXo9IRqd3rM{P9zNNH&8*o<1y-i%6~ zQ_wQWPBiUdWYz2d?Cfs-l>_v4LDqRk!V%o{(m!-RN>S)B#RM14$PM55+mJg>$BUTy zfw&>2^g2m#u)kjJC{@YnQvd4YS*wY9nLjHO)X_7FOxjnAEnf4 zwqe)E*NC~>rRrk09DL?znfJ`QE{9MHNj`lKm8{>4;5ibgHv-$n%H zq4dwHgLFc}NS^=HsP9j~$Uo=nOPkuutc=}}{^_{Yo|LOO*|sbzd%53zBgDBIG8H|T zQ@d>sds{V}cb*f+k6l9XhwSZ|1$E`>zv}Jq!>!MQM=tD+|JjREROABJ{YalBAVRS6ds8H8s%=_?9I@<;2N9O14V)&xs;BXBZMKR}}FVi^5Lry1?6mG}!WaTRyE~_Ej2!8%sDAXsT1rU!Ie?Y|cA(B8`>P7rI$9iG6%AWh-X7Q6)2km z`Jax!Iu(yk1uCk*H2CI&pJQY@wVac3f}C~4rAWzly9HQO)ZNH(_tQJD^2a~deq3RH zGNvwRoZ8)Cv;kOC9b_qBsU)lI4WA7_@raH*H5YU%AN9uMP!8>y=n4}QM& zhP*;GyCA;|^eAq$#9T{}9Fuk?i6ygsuZU9TL=Az2vf{e7U6m1L!~(zYL$v*&FKhrr&2}OpFTADe~aB>uc-jz)I7@^+8g;uGObhXL$MP`x{)^my0_B=2y98v0b)xQ zs*DA8!L)o9DZN!8*VhId1myQO@9VwO?+PyRF316yTl?}J_E7U-Mm`6yEwlCB?BPfU z`{?Dy)Mp{JWNFCY`#5t~My(7Rr0Mz)AtWaPiE@SL)ukaf;FpY7j} zy(?p*3T#05s&&^c8PnfA2m8HEd0%!ZqKBL}yL54Ef%`Kz7KiX*_mD>1l0X4{qrP4p zurXK0aE~2XXH=6(eJeySDYy%<*Du6QSd`CO6JY$FNcSPaXJ`JkOB|7Hd^6*YNG*1A zo>9HIwE?(gb_z0ZHn;nEgYBKs-93KbFcOwKq&l=q~9 z(hyPYk4i&Cwcl6GPmY})Xo#q>GuF+UHa*a94*-K2BC6jm8kTQzd=Dlnjp4h6vk~w? zBrbqeT?R5UkAwgcbvs-Q81-5*JT`1QQe<32sxwa40Ctx|a-|2jYS&TmKI9;fZt5+G zpxvH#CHHq@BwFrSFex>nW|!Iy!}X4x+=sNudoj?}KRE%=%SF@s?tmL4MmYUIoWVq2 z*UtkqBAEd|R;5vs>sskBM%592eA2%F9U4=iCuAAft}_AKuMb| zTO$rEfXvk{SuG;Ak_S1TV)Lh4J8}W2>^bB?Wcodt{sS@>(ja}R78b|c{Y8i({`9rA zJvqO#JYXXEqqzaAfEeR%xxRq4?Sdnh>l^Dq_Dofa>qzUJ>`T8!DoY`GzyO(=*|{fz z%rgNLt<6aR=G7p1JedQYscGHqfg0r2->)XHfLz-CfGPlZ8KWx$UH}{s=D>5HV_&Ug z0cBpv{w%V7nO{|Xsq0H1#$5qS5ltfcx8}?sdxrc(n#vY_DugrXvu-psn!d_e>^VLO zc)KH)Ifk6QE6-45=~BO;0@4RZzVs&JN>)jL!7tMf6=3^Q$#?`fx>qf`rLSct-I&Pa zQZq1p>g2e;JY0CQi7?~aj8`68SJPHVv{jHGHA)(av zIT6y~RQ4dxAtUFutZl>!TVP=TGX&L8W{9vI>iTd%_gw+}OF%(B|JFR`gvnbA3eUN*dHnysoX3j_%Js+me)Ho{hGLy-p{Mz~aZvG6}HYy4`V}qQMag=L*_RAF^%@DE1 zP)4>nxjCeeI*`|-kI6eOkG=3f*1pUn8U1xuA9AbyJi#V&1G=k=BU{e=bzMnJQ0k!HEUvT{7U}a_gNQrX5Cs__Vqeja5(!O5v6wXCpq)k z^y(3=$o^n{txJ}%_e4fDYqkARX^5!y`>OfLvC{($5jA$kx_Q&42m0*+PH96#_1i_m z@=Y>RYBDfNfx`Inmt=++nL^*i z$s?V(D(l6%(N8n7j@l(?+U-w(q52&sGp9vQi^3MI%25@GZJeGC$o6EwIH=oYaL0CU zSE9YYtbxBc#p-bYrE6)ewMC9InWK{}=$=d7xedQfEy7YU>=(2~?7K zVEqCx$VKf6m?A?cBRb{$0@Z4`uZ1mni1i^wRQf%1EjCbxId+*HHJ7NbE z5m1m`ML@8NK*{Dzr+NFu&|O8-6-Qj1j*%Tyq-{g|~bx{i}%6;nR zE(-hSW1;&+)-J`|{=i-HqJ&>u_D}Li8S||vy54v2$G@q(5O*$nChDv+nNRfdsk#46 znIow9?FHH?+TdJl{@DF_CVN-woBg*s(dRn7$96t=)9w+E*WD@4YJ^{x7Ll^|>~m`Ece@R;@@{igk6`lt0%1#lNwS6j7I3a#6jhNAv$( z8LRhZtXlt&;gqudm3j`a=03muXxA&-kA*OIw)?g8i~AE~u{m> zi0ZeBM&y?c1tLq44kY_PW_DfxxW$Q720S=04h6~4?gIf}052{|gRcX+|BwT}XvVu^ zdjouO_-S{u42U?M=R9#XzPy71L-8E}wI0dgbW@_N$70b&ESrjLQ$V>d z0-8!2Ju3iQO%`sYhY6P73%%J%JqA zoJ0Lk5vxFG5%*L~z}TJnEcuXT1dInli@297pj`_(c}%tk#QG%f14uj)ph}c8tH0&B z*yhwZA?kWFh8<3+H-S}%pJF4H_X3_fq+c6F#odXbq_RL(i*^Ry4YpXL00L~qb_FnH zBlt-U@Yhl#1RPux&~1Ijojg!)b_2I(^4fAQ0al-g%JSJ*XNk-Q!Vd>fKGUOiJbJdzw^a);`0TXf~0%r8X z!It0EHX&KnU!}JBw)^do&De=A=Z7KJVidg9R)=F z-t|}cr;sd2#y7HVzL&ODcj|g-$d=05Fp8Qy(~Lfu>Cc1j+dtdG{Cc8dGT@?QG6Iw_T()~Xa+ty-;}cO9PB*n3b`P`BBqAI&_$ zGMci@*rueTCPZFSS}MifF@0(6vXB^=J6V94ON?D-5XxhLCj0U|8^x)qM}Yv#Gv1LN z>$9&y#P?OSZLREkIgScc7Q1tg<_*8QH)<%sA;sn}Bqx={P|q8E-P@J+^Qtdq50p9Q z)dYIH5k;o67;;+Rz|vS>QtBFWa-dN%^;hb4ITph5A0Hm>dNIT>`$cUd{}F|!wy|#; z?L27xJ;zA=PVj+Xj{R9jDO0!QJf;IvYcH5}W2{FPl;Fw9`FAIUD~MdD-V@nNm8zZD zF9r1{xx&=?6^>G&N9az~-kRqDH?nIMh4%?Lll{iB<^B%_2;7!?-<@5q=KPO|7!7IHeW*i3 z)qeLw`OG2)zwAyXPgx|PJjN{h!m z;7=s~uW65)V;#9NfTX`)8Bm<15pc0w?c7|C*8olc0p_5(p7$k+e-d&c&%SJ4i2BsA zIr7EWGrk`1_CyZ%0<8ZMkovKJzkiK&mV4+=iO}OCn{V5!0R#mV0z#IOXx}j@w6iI_E@~$C-)t_;Q%sv}Xs(#*0 zu@rkK7cjr55%W8*^*3K-p8F>A973jhSZ4fANIs%iD&@W?j{#iP?3zFZQ2U+yj#LeR z%cAVcUZa&In!huqJ--}QAB6xG7-6ha=(2{Ill6{T!uWF`5|AUvp`j=~Tbm%@5m6LE z@*1(Ha?g`953rO|7f4EdY+h72f)A*20O#$_iLLGz$x#9hvTsobJJLp;y;vJ2GM*xj z+6HM;Ea*~SuLUrECG*h!tP6GDUKiqTbLMAtjzpT4`))6O zdyhw>$U}k-wx0EGfksEtkKPTjCI{Hr>~D-WpXGDdy~-biI79N*YB}qz;17GDntV1^ zjZ14$rI@djQ2vf$%Q-_IePR2tuIB=Nv+<*v6x6Ujdp;5OYw|gO?@5_ATGdOdwyA%1 zA?qNO*MX=Db{_n?*uWuPZ(rLysyu`8h>v5ATLIG*{4&7nEiesfKN)C4S2 zxn2{3N>EX&IyH8wZ;but(k`!r95$a~#r2sdvR~3R$Y*oTKo*QwV|~}pM1{}7S`Y}r zoh_m@Q9y@v_DO*@cP}bA*mUl8k9^LOg~ZQZ=R^p>Z}M+;nRg|aYF+jh&MS+ukH4G% zuKF1|`D*X=uMm^0`8=oV&O$`ZFXwV|vpKr2eOxD>HC|rZ5K-f$rJLvMqy`!ys*~Dk zUfa~bRa^rN5j8d%m;9dxHT#?W0eVL0crW^u z94LTr5xfp|#m2DY?wVGj^pPsS@S!Y-DBuqZFXZv(Ayp5DEMa3Kp|=QC_IAM8p(p@R zU^I|&YwW__4^athyCH`|VC$ZcrZ#GaY z34!W>m%9SOumb#NfR&d5tgy{=pX`1fih|(jfGhgI4N)POSDP|4H6XX)-i$e*x%_3H zXI}sDrw}$FPN;KOV%;AAUOxiT^wsutl(xB*(vGU(Xx2oQq|}Yn!LACtPX4bD04p=T z5T*hm%!fk-+7wHg$gvQ$#fmL?dd^4fL6JH!eTOPwMLyGK-(JE>A zzSN_6-uOW-Q~DV{6g`Lo3N&+o3n9(Q_uz?(B`24l2&J+Holt@#L#;fJ(*Qu0n~+d({7_+eMwQC;Gg*tv}D4cR;XA);^@!g0wHiHqxD)lYNTZ zG(%Bxh>_o zkysvR3jme_O%BhcsRQ+-&fdsrshCyy4@pVt` z*Rvt<5f2ng-$(UzG0$Wlhltvm>*t^UU7p)s?x7H6l^K zp2`{1pK?v#q9l~}M&21Yq6F#G^UQr|@4rQ*EV!FjmQzplGNx|R zjNg{ghl|=$U3Dw9?Qg85NbaZ8hyOd*)DZBgzCQOiy}N>Wmz6O)aq8M{bGD@XuJdzQ zuZ+!}L14Y#V>tf(R@T<%x``-rhW*wZh^Pb;AvvkQCnfMkUujBvHP8@I?bTEBs_|6= z4G}fI+Pit$rUv?~frg0cvuZ~47eFsw?=0&odyQ`k_;3r5Emuii8qiCgjgcxMkpVC6 zOVlNh;_Dnz3po}rPU2z-RRO{66dPNhrS~eiX`*UiBLcXepXg=Lx4;5`#8@nVK!+0G z;VU^nyp>2#NyP2R9Rwf0XHnN*6G`Y0QQHL#_Bli>v*CC>pK&OF55h}+u6qI?$a@4- zpBGEOSpj>;djWa?ujQ)Hw^+=56+nAw#*MzoE(%~w88wu}=N5t_V+0U?FdzY2vPFp+ z1{nI>JC`QuaDd05C;$W&I}Wi3B+3Be5g1H;@_2V7YsrNaz^SjJqB$Qh2)KGF0Fi!7 zX~K3Bxw9d_AVPRfEdLya+2-^vXRI8bb)z?JE}sTQK1IKXQl`eU0EQd5ZG)_frt$Ag%q)pX`dR z#G)Qth*xv7s|KRgcTu*mi35m?zys<-C!)}+#~va8K?v+b4lGGj@7jRxh}d=sbJ54f zFi;twyf@^QwoWsql>d|`n-LI*eHg{dlGx1Y^T42?EWlWS7*t9EMUF>hhtvVKQvp%o zZHzLC6;chbw)^?@_@O34pvx73KtGj#vvOM$z?>ol!K(gxTFW72kR$G?kXQkD3z3|k z8^hLFbCW)xEzA$BZ>b06{$ZIZ;@Vj3ge(G3pUl|%{P=~UjQV8X*9GJ@&ZcFqDAC*j z{M83FlDseZ__o2hzUodpASlgOA0$#+{xCq`H39CBbx1*bB*886(6Aqq!vhgs=LTvw zSfgW6-B|}$wEFui`DfITYzaRZo ze42ewJ^t(G(zmY59s!}bDE0AR)?>E8lcJ!ccmGz#I zh8mqR^S#vTfy~9)#yY5;tqnry^>-B8cx(=-!#>;_>dF}E}_L1gr<9A-> z?WaOQX`8_)6YI~oTD}%FZq8!VV+izDqqzS*1h>U?de*#W5K-$F7xkFB?m3_LRR{Jm z0!d!Td&$qm))Wyauk5A}iT27AyfZ@NBYsA*ShFaikk4fwk-fwnS!?83wdM#;pa5^@ zIY;uo$MR=2MATSro#x&~s|Fe(YP4#;`P`-k##jwFpfuKequ(g3Fjy|gxgxpvi5x5^ z#~R_*000HJ2+%@()LWe}oOkXpSx3tPf!J?JzAaD2(p(fD4*2e`ox&1=;tP1TH-LfY zZ@~S60B(z81%qt6JMV`y7>fk}j39E5V*&PBq6tq0h?UF4A@0F|-Eubo`PdhyW%5a;F_XmpwxW1a-&8l_C^aM9F}ZxjBUJ^0Wg&#(M(%ty!JeE}47kNP2q}sx<@{ zb&xp-=nv$+BXjMV5b*jUi>Ov$X#Fhhd^NAwFYM0T{&~omFLSZwl*>rlo9{dq0%u$7 zN?9A;n0X6m_Fzb8(cBb0gDuEW`xfd~_p(2ixp-lob#aK7n&jwd|Mzh$%>+guMG&;d z1NtJa>~mIT9|wTHJIXZeqx4jsZ4JP@<+D?Th@yCv3k-?H>U9;%(-tJwV4ux%u>8{3 zYxVn~^mn$0z)?X*h<^9HGJOc)XYP@EOkEfkdvdXEqY9T_@Sy^x&Z`tT>4QDjp*Pdd zsQ`9oy`}>AE^|4%;fg4^$ z@@s+G<^g2sd$A~Ht3_>Q9Z<4_`fu`!-B=V&@;`D{kUNdFBT~0#8(BB4HO?f~(zmkD zu#Rez*%{N1ubp4WsHsuR^kv;f(a$~L%!TAt%l)ugJ(jsnz!t^kec4|;6~fY7(OFA- zmhoAv8G@0tjk$%Truk5=DC1~;<|3rLeP^AokIJTZ{R*VBPFUOC>CSuPH56~=w8y#? zZ-~a;e>>i4UY$Gl3557p0-Rp{=xDLIwEvls_2=;rvd<&{OhBN(j)C$;*83**tKRR$ zT;F7U@flR8_TJX3dDE{igios=u;FY2P9N+V5m5pkF9id*J;6SYF=~jYeyeCie#u%xJ^%-dliBo&N@dS;dk%~M zJ>U`Yp{Iq$aNii>2ml0dC?HG@Jph%tCF1#}qF`E=&8b6vI|Z@zyLuw;NkQ^e0PH0J z$k-h@KzXlQ1GJ3AVywVI9R_XIf!@agGS!Fvjfv_N*}HsRfW913F2`1Z`s%qv;ISDy zk^_8gyR<1$tPclVyCwa>`y&g+dvRD44Xuy5Smd#0(@#Z_BRJRPFa>xS%Hr?{Gz7x4 z$fAw|Ca=zzrPf##@NOt`Q+ZAx^u>Vn&uwEtcdY9~#v`-5M z!y-`0*xnh?PW0oJfKA<4RoAiroloU(&0_Mpzw4(-MQkhE4hZRj3v-c$4_OQ+Y8(}!AYf1<+y zkr>u?oqV?O*M zfbWYT%U{c7j?(6GR;-KaX5-4rQjQv`U6QEo&Cj4058G z2j$!8MBP)rPk&#{!AJ-S5Mq2}4$ZNL%$&DN8BJ0Q- z0ej`h@{9oEjUl)I`m-ii#A#3Rwg$6mbsVC0e^j#wGy4d6@>u&KnW%;qWF7?c-yhP> zdciIY0Bz#vq>EH-9u-{HZ2J^SY3ftCN)W@>g^Va5bH*jL1>yohSAAByHS_1!A^WMu z_JjMWccU_Jd{QEX5b=NSRB0B^xE#jYxI*XDq#a(Ec~gDrsDvTq3yDPLom zgR&7=nsputhCo3Y_Ei1-VXs#yboPainv#AxBSiS}>~GjrA~9={4$*3!v!=)`^>PAT zzQ|fu^@-oQ?hKhvrAjr9s6p8DBo7ArZ2D4BS|-RvzCa2mO04slYc8bUvjZ&moW19! z-17tJN0hJX!|SN`kL8+{emWOK#t*^DKujf3IdGV9nldH5*?#WnM)=k<8MxjjZsVwa4y{@#MpUXODjxzq=%-Z)} z*56+;zw|D*r0=jOye+D{#l}p=jR{ z6!%cq_lYQh|M@{Tag;erP@sOiIpd%ZN8Njzfrukx$Xu@t{`Z5!T`#M3nAjSMXYBbN=6~nTiinD(XXcUH5-9LH5tZ?ajOuiL?)e$x?fZs^8gDJ$ zJa0QS&=67W)Kc@BrUu4A4cKTkMAYavG_uDboehu# zYXFx#G9u?#>9K7&oWm|}?70C85M*qAioHe~Hv@bVbqmCj%jakgLn60t4oF1#1-KXS zJJh*++PJm0UQ&VezZEioHJ?QGKq<+fkx}wDvCR>&4Djv=Ottj^-7hjGPYR(ss>p=RKehM4|LIK5*rbAf( zhx!2GuJ;2Bii(y`;+BxXh^QwMtz4@a1_C@CtR1kaW8~_hmN=Dz zHt%}}R-mz*C_pf6U|jh5K!;iL1qB8LqVe`-a;_i(04o!6zxT)9=h0jgH*BB~5p00i zVto-(?9BPf66@B?pLgUEC0wf~dR5Dq{U>6^T!5G<7FwxeDjJFeN=fP?m-+=#19Pco zW&~6k&RkeX8l4dDj&u*CE^RyMy~nGU%F;by{?dDl*^*F%?&jH((5hT`X{?BY6*FhD24XMe(p?E8RgmYpWC}_ z(5PtUWe;P#XSn~~cI|wKXVUjC1@J2hq0|pzrc#al+_gNQ7#N4B~TVFdDwH>vqsC2fgM^e9jXP#Ua5(+@9ua&6uyiad{ zx&G`bE^Bu_?Ls3EP#5!4SN*06w?2=O|_xE+?*t4l0>SH$2f{D}x zf}vF(6Kn)L?#Q@YpFOJfp`f)!=wH2yv8PY4BYSas z6=Y!C2{b??A;}*MNj^X0n!>mh0bi7p*|*vgaC!JR?V_*99mq}-@mN&K*(*1t^BOS! z)(TDe`s9%_X4t#Qo4GM0l)wW4I>-@oeqFSQGo86ojNr~vGwM59a3-AL|-wzNWx|xDRbTg}?l>s7dkChML z{+78@O1_Vwe2$@fFOUro(uV!rT>C>n0O7t3SnfH1!Jb1dhh-rSJ1xdz{$EsA|NFtA zuICbkcRXzXpyjXsa3bcYRU8-rutTi^==6Ta!s`(0HP$>D-LJQmwl z7H;MoKm`JEdh)6)4Zw|PqeP)9Ew-K7Zx|kym6wYW5lQoY`Vr6*u(@R3q}cl;PfXf} z8m1HC!`xT;dA`|^zy~0v{A-5;E>r*gSKeQrT#{#eDv`rThHKOI*MxZJ6ucg&S}IbE zN+mhUGS?$S_C;w|t3?*)9@t#5;iR^ZN2gtKz`BZ11p0f%&vNP0NX2ETKZ+u1FJq+i z?G}Ks`>)5|4^bfMD;073OV@?;4fO#qvQE_beI}^C?m8Uk`TP1kfU3?@^?sBN)^zOy z{PsI}WaPIp4^cSF`RHEq|F-L=R>uAC%cYS0A()acku9fy57%7*vhNGn$|_DU1~43{ z4k(l#P9O1{)=SCc1i#}dWYq4gJ*Uq8kYIuD6NnTgeEuHEwmfxX4r}$def38_ckhBA7(R#vNk#xq!&8Weoh3`9Ps`p~Pd)iR{~$aig!Alg-Vo=BuhdRH_YQ z@2x071$79tVF6rn!G(aAmya5;r)_9mw-;i&XdfUz$X)>PNC7uHWJfvIuw%@nxS9=0|9wGH^_C+5h_{O|w&YPOCdQS-B4LLgsj?m|k*LBe*&YLbdH=XJ94|yG} zXU6fmn`gEiJW{{!)%^9V)O86yOir!SnU}vy|3=aa5ZH9ofZ4_zdfF*A;YG2K&8|E?4!3e2 ztxlw-TrU7rsxO4wSS~D;wuO}29!2C`>|4Mv4B1(ke@qkar75#M2fwSEdCfc3j_pYkENSBGJK`V>*3 z=&@z$U}vIxku_|isMM$<=4Vb>lH4pnT=`ts*Nyd3t3lWVB;q z<_A|HW4Z|$z<#ITeyjKQvwDp(@3M0%xk>_NB8kt3~|J>Y49dtf{DsP0u*%C0E-D(BS{lBjJGS5*I3Hqprng7@I!uDfjE|x>82GcjE ze>@v-k9{6OZ!qg6i!mx5WF!z$a(M*NoyqaEHh?Wa-oB-$&vrI5O7(h^OVEQjo;H0QdStT~R~$wZ3PHBIFs1RZ1H+ ztAZ2|X!ad!rKq$K3wfL}@7^4yQ(XW6KmbWZK~$alqmn`Ts0VXQCDb}5LlOIW5Ir{)Mx#o)7GZ&f<&C^*^8dLPdDY~oy1RSq{O6}O|7xcO5Rc{;?ZOUuORjHH=ZFJA zMfP#dXUJDM45>A>xAsFo?Z{rrTyrRM>amPvB*@Kqtu~dn%Y1pcma6SM$CdIL)}KWc z9fkIZ)@DX%SKKyxO~WvhEL6 zJtCrB2$8)bxfW}y<<*F&?AzI!Q@6V}b8A!DtAU1yYOkJ}SB0Dg(U*#+H^13sI$`&YFG@B=z3Q0ZjJEkN}pr%16f3t$DfUlbAw={A-Nkgq;u z114EVQ7zRui5${d^sUK(uG0d5_WmN;{~5C3g+#alYaAjGQBQP7%mGyZvdw`Yz-Ry$ z))}V*gd$A!Q&vdDw(R-|v2d#TsfhBSD1aaBB(kyO3JdV#Fz-G9rR<=dN+dqZI6m(~ zsagZD0yJMt4iULT4g~1@A$EYjgdAO$ad`jg*#(?6U$g?<+~a@RZJz4_+)fJsJ}VYp zz(Dy0HVr~O1{^^~vC>00o6C^24z!XUi<(~(A_#a4xGjKKKMex#)SgKC{Q+j>y#eIT zj@{zCT)_VgD`uDPjSs&A;!+_Pmw*=0zFyBtlr@vTS)aLPtd|192e3dwd7dw##G%YM zoeOYbE)bEcZv9)o)8CPF+K>v6Z5fbosOM9C@Q-?~a^UPwT~AG)lcS;#DIt9yOg}1E zbpA6C=-H}0TIu zKWBb27Lb+c76s5!N1eV`{hqzQ*3aQH5G@om`e-44^E}jP`sU2^Ja$m?Q zR-=Gb>LXXDebE>M_Oi=kCx?JxWp#H5HYD4kfcz|2dlLVJ49q-D^(K;D4zTYtPyd`K zbNQ4uW}bc^{bf$Zi+S3-S*xy2B&Y)6r7Sj2W*o1HVjPLyiTz`b>TK)V`ktwxrjs;A$0#fZGy*Gm5aMnyYM;C;c zSdl(*YY1%id{nEg#!r6_VXa_kxHUkroH;E2kY@!b?xEi4Nf!3^epk}(j6>rdVJetS zo~Vl%KgB*QV}){0|6h`Nw&t?sYiA6|kEg9%tYGCT-|9clfB>InB zMqjd#&E;>+mA7P{Z+-IjV?F4s_j8Svf7gL3r!V{vVmB)ATypiXb!VqGCqXFla+cCQ zh!Q!5OOQks1nbuaA%g|%oXp%NAF-gvRUsHxEeye!_NfT|kqdepEA`sfedy2DYXMeA zvZobb#fe~b=4Hg4v#TJ1f$Z23wDvXDg)N_Dj~ue!dVO8?HQMX(oE@Hyd@~H$$rQNP^7g3G<%)FMDz~!32l(SPUM#Z zspQmbygto=r;dE*Isaq;_sy}~YNsHgYI_6uO&pdlUHYx-o~XVg;ugIJG#x0atewvq z&eu2u7GN^ZfROq$2kwIb%K6^|-{h`9&hgalw8%U3W9sQ#4lMe^D*@L|2M7Y(srRSX zFYfxE4T}qqKa}8Nr}sIQUJrG|LJ=X-VXjOv4D$67(oBB0%lDq(f7ctnsD>^Y@-h5?*N_bt@a*7kYw%**b5YY zAam7FDdYZh!$b4~6oN=+mo{QG`B~;)d6UdNfPShI3N~{n^(<8!#do>F%HD|BNh=$E_z>*y{eS)SqcP_F=$scA00hZ@(DA zP{7!IA#GU`Q!~gh+iGrV_1r*n^Bwz8`H19bv);@}JqiSo^O5}zyTRCeYx^GY0lpvuH;FP9KscJxA5X)yy7~DLNe$D`L z70MH6&2|Z-?5`LXCo-3QdFn#cn_qQp%XqaO2_RaSy^~x_f;;UaMx#(bzL<4%N6z`y z+hV(lh)U3hb3wr&vi5a4Kaa+TJ*^xu8 z2v}g~9RZe}iA@}(32bK|a7i8nQOmDH`6a6C$AB#!e|-+g4<({pL>W+dAfIhDHkt*P zU)yB?%RY)N)#o8VCS`*zl1<(S_GX&`mUW1E6}va~4M0x{xz`g(Esug|Mxf8r@elv+ z^@|GNOJ~q2s5YAQHe8==4!aN_5AgCyfHRgCAB50F3IWUHiFhV<7f&UBl0I6l%eWbl zf*0cv51cNP}{vVfcIK&qlqsmIE(_Ya3%^@S+D$(D`Sra7!r2z2q_Ke51 zvCjbLiF}s7p?A@D%YmA$BoZOxP(eO72GBf_ z&tccaW@<&i%8da?Hbj|J=R4_X8&(0?mD&QKCCZ&$sb`!TAeItyZ7$YdK$ANEfzNix z7ab4ql)GkUfG~hA@b}itr|i7g`58Dv+bV;pt^fQR#LKroT!UrN z$M?0J8@tbsGNUV-dUd?)p zgwpO*O%Fr?M3p6QhRQ_W>|M-7RV00$qg&}@uDh89UFPXRIA#uGQ7j+N+vyjSBbO3w zFR*1p`h*~zxyE!@Mksj)cLschbmb9V<2?-b~?A1lhUj48mncDgD1@fVWiGF#pFMXkT(P?hCnRe`3y}Of#QY&neaH-P_IYVjp@SYY(eN>ZYAJ z1F)b*!kT~e&4&`$VZYJ4&GntFQAEpMbUgju-`NYvD@zr|k+!3tdM%c4+S8sHIldxF#`Pg=y-pC5 ztJ5(!_@@~Ae@UV1Y(dc}FydhLW#6Uls`Y4cIrFW_Im3RmzrtNaeke-fZE1&3GPfF& z)3San${hG$RI)!m1hzR{}oTK}veqFUEW^V9KN z0}T;1zT3Zf`lbecuK~bPLqz>v9V7lt4ZHw{|1UX#-U$#9N0}0Jx-#I+6L&8t8&e>M zgXBnV0Nq;w;6fN}J8-7!L`3~9q=-n^fdGE}%zOPa9`>aF@@&+} zb?*<6g}4#%E4R($0H$NP07pfiirRKKd1v>ju3d?gtD_Jdwx3)ZrC2~Jr4!&w9U=a{ z3J~;a4vSm5ft71wbp%X*GK9+CVl}iVM8jAIDMzXe^(}t+4&@Hxa)rE`x>N7KkCizf z{xz0{PbXibmq?Mc zceglzw6w&)&G)y~^X`7u{Riw9d+mLl*L5A|d3+8po&x^Aqfl0PptkBN2>;R_>XU6I z627h`OT?Zg{cat#X2eczs0A!Y$^lg)b6}kY`42DL>G6aPdx*TeHqs*0nYu2Q9*mId z$K$=!y|UcJp=F18lP!o36@rk3o~;&PYG%NN zq^j&%c2`HLW?}0W*#dLVu%gF;`=_3OeCmum1JfLF%JN69nhf@50Bh5p5C({(lOy;l zvIEc& z2SWm-|D@mD=;6MMB1^x}7V7`e$UQvVb||<5UAi{2C6)x2I356u6N5dKnQ_V|xR_lw zN&c+1GZ$##AV5 zg#`)~hFJwzqK+sq)*8$p_*ExtH9CrIGyeQ-kd0s1T_L+a?=6eS>w(?TJdO-a_DXt#Z=v}#Ny0M{72q>}i-klpXz!dZX3pMJ+tTq~jqb6aj`a02Qa%3?+as9Iq`kWJW`+J+qfqZrk zc*V~EcJ1SjlC=Y+Vj{%@BQdF3XN|EaX#Bb3B20g;$Cylo<)7cBk-#b(PXYr`O;#JSA{8pX+FR0ZkRy==xrsqjDFkVfIP*MdQw@dTy&vJrZk*>VRrP6a6Rvx<)X&Gdp$-mjCrEFmOaxI(kdL_ zD7RS?kqiG>p=6LEd~g&J`6rmKuD`9pTWGYrD_Wi1oBwAl9RK(GSH5vgM1$4M&#lP| znwo_xnoTGLENrgH+GPaqoNZ~lmhPdj7Httl&JB~q!X_F=eBU}%KEke zjh|VM$C4mr@_YT$snuk^Sk@m4Z$6f~EpvN!JLp|?FJ^b9cw>F7tj+#j)u(OGPc30P zJ{84ot|WlM2mGFM6f+;TC%BZWeGMyzZe&8U?4RBxD)M4`(`fS>WJqTR+r zOSj$%tc_;@=+1r-9=^;&^vLgnM+tEzdXW0LR_#1}5 zZr}Y64^YRQb8SCZZvt-i4Q+W{xC!>Uy=s}RO5E}XP20q7E%v#EI|VLPLRBq?|H-6F z{%Yq6`x^T8|dYi+~Ia;jhdoiB?EL6Wchw#+R6 zf2!TH^FcWk=g+jD?NzoYdVMjy`ce9+DSUZr?5B zRBmbyHZ*+$_iPSq*t;zUU0*ZiI>LK1kTu^5Booy6xQ9z?bsVA6mX?qr=s41_T{QKW zrG)C~kUEnNoobWHK>t>i8#cb~cq|vZ{F5s`jVu1kY1gQpT+z_OC^E(nYj`vRyd=`u zvpf$LNGz?o-dmwK`qS*{_CUQsSte6>`*&UdAxx>AQUigzfJueEp{bC3-5wJbsfwf1 zEJLIMaB}c#USy0i3BX*RmUZt_e4I}%e!;KntrI^!YvRpUDPR7aM*8K~o3C750q5Zk z%NUHElsaitIV-r7;w_vZhtrY;7{e}}6{|iJqdd-ev)Efhw?F_FYgwVO=LsxW8mB*m zHU6h3n&@XI3zVTkZjj~9>ObG7NJRP4=HWKrZ^|dI=Cq2YyhDPymVL9594)9$3xUuCiUk{PXVH*L zjxnBDt71ZqUBSmh66e;$*#nhJ)K&M(THRT6Ge7?$Ac)CRvNMI(*;cERZ)|i*zY9HA z$bj%7VqPt@X{#$nfh(BFx}?Y>EoSsHZQTafnUN!s`ud$aIr$)P8yAvv#|#ucYp<4f z7}*4jhe8C;v3vLDV`q#mFyF{geHz*2nkKGy#RH7rlq?*UFs3|h#Ga`^oI1RRyR@Ds zeBjF%p{<%+2-7W4GY;ZRZ?n4=mf7fi2c$?a!cFQ;QsU!IhaRL7@0jLC!s?Ym`jnI?h%jC|tA*fV06ZT^!` zL%5%oouqPpj?A`dfg`%AU$*awhPIgea*_(0Mdir6=lel}lfQ>&QQl6xK1SO&s zWP*+cpZKJm-8P61tv5+&6W<@z=5%|_N#!)f3|XOBoKZ5Bho7rgJvsE0u8gWUPLBq6 zFYLExK02|5-6wH{b1S$scr@ZdL%ebAsN3}i^dULaN8`qh*?X=%QO&ca^Xt&~o%P4@ zwhwOT?Zw;wO2%HLLsq`HsL+^m4|elo_qTLXxgL0B-n99r2vAG4F)`~6Ll)h4y-{O% zL61+$@KHwpQCeMnoD?}aW@1TPi+lO2#cjWbYS9bBhiUBBRR7|v^G8#Rmpf2qw4-V$ zjiOR+T&<*#tqp5%epKiBgOW*~=g@i!>lbX;!}k+kV)qmVKL0XZ>FGIbt(5pvsf$gW z2!z6f1&vPgsw25Pf7VP$sqXUMz55c>5@61CjKxx`<`=7Ecv07@O__a+i!mi0=XmkQ z>pRn-k}^iZl(q@iV9Oju&%yaUzRuHcfDFS^3yBHGgpt8F`KyF3g%^?;AhYlyW+c6x za(O$vnQw91{tj`YqO?je@vEkccuyDUaOTTbaDUR9n9!?|bX7E@ZQJF?eDQI^oc3?B zoXo_LXVi_5JN?)j^v8dr!y=y2w)gsQP36Wm#2?dN%nD0Cc8(WDB0Y>+5M7;;Uoq<4 zUoWRhYu8uq@d{dK$_|m)clq_SFQl%McVC%ZF-(i8&_YvJ{52;RjR=Mk+Fkfs-maj$mUj zr=Qc~=Yj0Abc|?v26Zs?Y!;b2Dn6|PzW6DVMDMEYC_}pF(%JTut%9t@QAB0ftk2(B zgL35OwABNz9iH$f^^HW#IA&Q=KIHH|2%c`WcWCZd#8}~q=JeuM5&CSAAmT|2yr^xo zsY<`k5kd!^4@fJLbzpC*$iekh=iWIBAP0M!2!KF0NVezo3KH)Pg-n!r=H8WcIvEPl&6 zG`RyR>&_$S{l!LuqRm^a`???8Aea5eQn=p~*rLxtgPb4>J2Sl*ri1ta5&I`^EERuz z&{pThS3TD3g_nu1VNo_N8O@H%b5AqvsdZECq>Bg~V+KMzSX#W31~da&axYuD?tFR6j_rB3km+d7L$12(n}b-OUG$bW*%x08 z6SOrM*!q_P*Y9Pa9d}x*X@LBI60HDPB_`f_|945ju}dSe-}9W+`>?3c_|9qVv0pah>$o5 zLB~JGv^pjSr#&q-Y;hX?tS@T4ZVtjHU)iX-m)|>9P039e5%=arS82#xtyJ&1#fy>w z^Fstz>M@HW_X*&6boaj%wK#Y zc{kZ--98=tPcEITLT`V`e)>dFbfa$aF|2W#x#!oAI*(*VgQPu2lMLMo`5Q)sSHzri zzCAP%p)Z|3upb*f=q!+-c{eWR;M`3Y)1$y;l+XvENnJpY7+sU5;W!MlUx##XT{kJn zUG5TgYQy|P4-$x$8ONhPB$(K}4zm4q`xLO2_~&xR2Jg)0QG=4CN^md%6sv&tq7yKD zDzgnQruHOJOJD_t*FQQj#(%adB@Yo{e5HZh=&1U;4n`;CED$sePu_sn6G z`#gRjq!2p85rF~)^fe^`@@4mE@9A}dNUa^AE(FN3&?Q3;l0RZ^wg&VKm*?Mz1S%&m zt9;m;Z?y&JQL-aSvJl8{OiEovXGJCQq<6vhb0JI!QK`VSFKS(nrW*B1n>J7q$ZSN^ z9i?M&%8aQBcs=AIf8DoliTR+)_;1yWWvhJ|Byp%K-s9F27GULw%(J$|EOu}VUr1aI z$%=eh(IA5Fv5vugD*N+s+}HmC2OM#y>#fi00aLpvy#je)Y&F$$UHZr$)!Npdx56~D z1Jb0WVW+gJp7_LRDOaVI97{g=q&0b0^A*)KKk`%GQ+nLr#&D!1oB8DO_ywbgL2v>_ zZbOgVQrzxO^?G)6H&DJ+`1R?N&fIQWKem=M#9q@q=RC6GGXAoY+@-Y5k)2_7PzFtf|{JSN(f7v83n>p-t% zB2q3TwY|hurY7_-i^js~VpD^5m4S=FguKh_5~<<5!G+;o9p}y3-GNen&q<91ZT5i4 z8N<(T%|$(k727zsSLYf%cg7FD!lBB~GC{oiSHJNlZd_JCWu6zAK39E427ScS-xo0I zls>;KR!~W+v-Al{V1t^>1ywlT&I^7C*Nf53a`_l*(%!1Ol59-{h4AVIX1Lu(eK2!0 z*1|oF*Im>7$TYy1ycrW3Gy5Xj5NJJD(lxes>2RONv!?_S(<nDd z!GF~t#QE*Npkq@0@rmo>V0F~PB7o{A1&jY@`fs&r!ak9$48%&2ORGy8aj$?~0iJ*M zQCO057g}N)$ZuPL#z~rI7%jsWQ$&A%T1JD)e1UK&_WEVk+hCEsLMX5dxB);so9BQ1 zVhx#;{aFxPdIgKS>>dWnhyx{H+=h;3vl_%Z#JdC-H>We(w3fiNLkgy(?C2w`Dga8B zX!6rWR;J>82Yib7{%R|b;U@cZ=P5wpr794X*g>odby!LGh+N?}=o}qo`KL(t(32}C;o`H~>!i4m9>gs3Xcf|e<_?FM_lsXx=) zrp4Jc!a#JT?pKo1sN#Y*Z0iM5j(=^RTR3MIs_PStR3A1sA4#rWtR3m~i4j}xemMs6 z?Z}c@yUKm$EN}ZRL^Vs}r14{DekTZ##ohezjFXDDTdMut_MYQDdN$xwF-#O~t-<SF0;yyuVZQgk$Xs^YCvfBgDas)3Bu{4*hP=ETXnABkMCISI=PLHWP zW6Mkj#Hv7nPiQc?uDaHen@Ey6uFyoyYyC^>c#P$#kK~&3`p{(>@Sdw`o4Z1@&MK9t z#x}q?O-h8iJGOK5lQ9?Ti?Xt{(G~sE5_@`#LUxd&QD23#>5ZKAzPf;?`VfZ$|1wRe zckoT2BTkSdp*&Vc_7jl9HORT7y#6`iVAlPY7if!Iw?+Q(>FcDOutSz%2-Kzigb%tu zlNN?UWuR$IWB+Ce>$=LFwf#Z7thpk)UcVdjX4TLSwKrKJ;&yUytnGNRtlQ-sv?I*Q+^I0amO^|X*!uHpU_`0LT z5F<}TnckulWbI}ZHNv4MoIcQ7PK4!5>-(6_T>t*!V94P)``6^{*|s2(<}oZviSN3f zhSM+G{G?%SK(g)H*!s=12%kVo`=+I)S;xQt9md+}dP(%l+YP2@x&L}4k+iAq?dWGjTK4*tCSMlPXZS?BfI4M{k zS~LO>Pd@tqR|5SdCRZg702o{r)Y}E!L46}gAF|ZB|M+%r4OxuC_p$})7~kT8>_7>uiv5ujCt_!8D0TM_jd?9`=^2seLOlf2XJKT zyG_(=)IVK8PY+z4;eND|&b7q!wmdd8#tqCqLe`|4=bo4wmz#tt5K`^(zkDirYmp*S zi#3v(4d*EM6C80nyE^%Rptf#YaZKTK=K#Ti`+LNJHQfXVL8jH*R`}$w@G;+TNM|=P z7La?)A^dM2Q06G|Qwx7LP(}3#dAy z8yBPrdC#m(;#s*7`?e(D_w7yR(D?9pg=`Bypf3(>l`g8$32HDDu;CT2Hggg!NjTne zs6Bl+z-8H4f`(-s+RUFcI%;BnSyBujcnav4pNl*Wk0}Ake~VI5X;3A9uEEp>r((%!e&{YcNY7D=-8Ur2Qjt&{EgY8~VQsniF5A&Z-eX)2g?Kwx?Pf`7|6YJX; zzbZo&blUN1lUPPZt9ouDESQq+Afj91eIcghpyZxAw@BGY0xX!oWRj<#e72Jy>P8l= zwA@`MgTFw^boybJkzlIjo_gwcuuUQ2#o3qp?Qz57^}DJHV2Jqfr-icAX8m@6mtpH1 z3fVCFL{Z@j^6pgMmsvG*tTAGPdaifeL+%eJc5O<*7v5?Tonk|m>BS_ub5+o*gNuYd zRSHx?CoahRo=cMJSkr^J{db5by@iwabNwrqdy#pa=C0Hxu%)%5cV9_772AMqd0+5oqxbzetBK@IV4xwaK0SY4x0S0$mq`BQ0^VWEr)+ zt>@m-gbmXHAs?dC5d(l9$kulSi}=77WNOv8=>$L~rD;Zj4bzZ~(rD2{uGw9Pd|f?E zcwF)rzcCx*A1FiiF}t{DwpDuPJVD9DW)g5fwaPhEZLx{r@zZdm&nK2A%->Fa&;R{e z#07@JC;wvZ;2I!21)(hIIu_<7nSE&;%(0n3t^^xsVGAC%CP3=c&=em?+8Nb2IY9>t zc%%6xKe+KU$9zP`+}{)LzL5^362_1xT)CI9qlD{w@hZ&uFU07`Y_7=*+I@8RO(z_P|coL23~#-><*3@v^_W}>e&WJAJZ^8aGB zlQlTh3z>?DL3XRjAM3u?t&i`<4ILaaE>0tkR?-}m7jj2d6d#rQ0$Omkf{npiH0xWg zsDS`5{}wGE55Ir@*gvo&{Dx)R>pAMFXjQZ+K|M)~y}a17GiCw?_L)%9QV|ekQ)N_L z9>#dNcH;XKNITTxPC(4|n%-KxU2aeP4DU34HeYtDBIPB^sCqlPaQm-2kiyC#pk8L$ zSILmg@@VWS097lWGE8fxI7!WR!HA>Rje~JIIDd(!u-CIl4z{5eAnH}3y0sqIu-y$- znPwW*Y*&ao81B%*BLvu`{sUW{=~zaG0c@IBON($@OpN$DhBVH()2a8b@1EzR@9UJZKNs$vs7qS!wp7SDavv7A3!#rV9>-v$_U9daUV^fEft+Q*M+hgLFL<7`rA>Dm>~#=<&C zY>FDBst{c7q$f=Em!Fv?TU~#>MSI4l;1k#?4?#A%6$4TLhdJ)+wW`$v#5`wDufNKLLk)}rP|P`=8Q5FjmJ(0&}CG?T0Ga zhg2`CndnPD0mG6xPCr7bVl^9^p$-uic|noVex$^Y(7=6kVw({(C9p~+ zFKG1A&+&5Nlp~vNO9RG(&SJ&ieb?pG6vqa&8k4?wKL>(!4c)kzzHbu>!h|+hGw~yF z$T+O4Sdk6lqg^Ly2kBTjFXbRTk1N@+^O0Nlva)u2_IJ^Lq>g|_=vj&;5#eJ80HPUJ z2CQ^4wzRf2OCbWP$q|7+jR15tqXI)!(Nd2LH`fGfpGE?YokQbo9V-Ak@a~HzX*J+} zS6Qt$>vuokS0@1}eZlvB!h$dS>3<#u{l-id7i8IcZyC-ii?=!-KEyoG3Ec)IxoLoh z&u;aX$iD`=AP1s$d;#{(@jSJ|JtQBipnn`&FZ-T5hr3q{T+{#K_+XX%Ju2+0CxhtL zxF`UT;z0xKor}e3xBFI^OsCZNj6eeLu*LPp7)&+6n;5$#MElp?2YDop65hf zIJYC$2Hrp=1~#7lcx=)>ZxAndA-wk%H+D(*XdBSyDYr@oMwM_0S`n%GNx#cf;fK$D zT&*C(oYJI7mg9zd1eX}TIUpp33KcB=GRIxzfN~3QrvK4XI?>rWQ4nfXx~p_?#y>-4 zE;K=Y1mxDLYB-aow_4dXw4KCsX=&#_cVEftQpfU!9;F5{f{QHvn&*CSho4NsSC>Hp z#{(-~D=R6{7fc%ZVKY-R7<7_7K}5py6%kp+wy-V78+=mQ#UR;nT|{L7HtR`Y6+~4V zzbEOIjd|n)m+Nww6e*wktHl#HkSOKf>$?OVdkc+DV>ZCn>3#i)G%ZL$w&fa6U?Tsx z)7}tRhyPcW^fU;|ph`AZK=Awqs}8Qv z`++|7)5G@@w6bkiin*$0g_N&ucgHcFX*_ z2w8J*OsWB!cC227O+{So@NVksd=#~_r4+rCML72AFJb(#tdvU}i~jPN#jP9-ztE)3 zdn%O2wZUDxuC}p32Pp~63p5%>zryw$=*6=*Yrw9bYocirKO^P{)-#<4HyPblO9k66;8>0;N)C>cEf3HAKW%S=Qrdu9ow#oX_QiQX^ zBPVdkBUauUXF-PCnue6OdoJ(%{gnomu+$m&-ElKNyjC*Q2H#0xuu(GN?k9#ShI$l( zKGN@&x9T{6c*?^a5h{2pGXeOkaM35GcU}?jDwn$F;|>`**2AWjhsh*Y5x#`S{B!BY zRo9Ko)aiQt?ycyC$W-qeM(Zej+xWKvO0St~$TF8U`{ByD)Pa-}>`V*kEN7?!qG}R zs=9f_o7FPy-?lQ3F~hw9uR#`XX{X#qd>T*5u8D3sIv4&tvKuP2@PU<05M@fe+-KvI z+WOXDDQ9Z@Lx(AW=UfIQmo+iczape&q(YgYkFz2sHGyFJ>tLp0qUt41cF|Lj%fyE5 zM#GYTR|r1%_1q)hv00gW=ssPYa&i=OQqEPojCIY0AIpH5l*0%sGR=QV9_9R)wA{kb z16noylD^pp8E-lG{z#r}8;J%%Nh3oWjlbh@x~lI#Eo(OD z_GU>&$t@>-se%vXe_5*~R{L%MCRUIS#VzsAJ@GwweMsBt$XqWw_*Lkbg~)|`IOzQI z{tain>H6%%eTsL1`7C80&89!Hn;9v#^0E>WJcJ`52mm z0VrGG9LgU+PZj^qRo^B4-(Y0GfdxS4J`aEMxAgBLz(kL(bId1T__Z@^tWfva`TgCm zJ8@+?n^1c^A=EJZH8C?mJ1}UYOSOeIdFjg(bfL6HSG#A}mgQKw21uyi$;`&${z$}-iLdVv={{x&%`nJ#$IN?6;zjoWa12@x8J1T+5xpIK?j29a(4&Rs2SmmPXiOxVR;fr&X- z#4^C`!AVT|)q`VZ_3c_wkjuf+cX$2^`r*!mZ*wHD)n`a!rGHK&>_yN2IbFVieKA6^ z<|KCpmE=?mEc-y=vD+e6<@xW>_f4?QwdCt)nS9ClrE%o3?j`3C5iJ{U!##5s0WeQ8 zw$EShrWOdytmVXwbykIw!DKGIKUEEyz z_O8RP`i^XLWN%MZ0CL9~%kk7vQ3X96#SC0{nb9UoP;lmlpo9FF!ShVf2>^Q4D78x( zDlCz_RqksP$5aj1kIYJ&cM9z3zx5o&2drtmdGn(b1|tSNQ=ra?$)zTs^b2;2em5Uw z460VmwTfjnkh&0=a6w&Ir*45*8xpQ-b28!IG`NHDpG$nPoube&1fyb1aBg>+8FrV* zYp?L19IDT3G=jXMzILZ6Tf%F7fI}g9<+H~j8^kSA(>wRqy@@sAzVx@b5*{7_70X$* z>OKz-QGGLMO3}k7*pxVFUjUtKV)4G)a1*m$BFFBE5SZE&U^}2JZ9@kns7O3-OP>saCUd|EFF6JcMRGe=l_ z>RSwaywdPUR+XuI#$@k$YQVP54vpKWe1JW8%ttdZ3dmEbx?MKMb9~q=FU$BGUO+QZ zSbf9WzZQvU^7iZsgETnDnt=WD`-p!9a{=t~gczOL!TFPSHI-$7STC4Kt)gvgT;`7b ztFvi7o#;WVANVnm=sI{+!}|D|OR%c>7#_iwF=DJL~r7*!KK{N>uhH1E7Tn) zVo;*vx$VIvmmlihMAlt*zG1}WdkKOmvy>rIn=C;Mh*k^DJ}BCHP5q}%41F_JW#lac zk=e;uj#4;Gk3#s%%DcK+R?CYSTyZk*xkEx5S~@uCFo}VOnCgT-r~SoR3kb+kcgUro zP+avLyVnQcV$L(;q@Bs{_jl%Cb?orLwC)2Cu8(;7TcsYA+ z4AV!s+My2{9!qj5@Z|wvO{4B2agY(5%8%V=8y-@Z@l+`Cn_`l8GY1m2#0{?SRisvR z4&o^u%A`LieWmFQ+w&QnaLfqm>O4s13aS2W*vmm3C8BYsyUPP4kM<;Pz*}c|v>?GT0YSuB! zIr(RNq$nGnjWliLhBKOt7$rXm(jd<4;}qzTaB23Iyc<&4gzui-{NlNg$<}CA$^Fed zUbFY1DxfS?se#QoDwHd1OjF!aZyUFDGQoa3-6w<`ew=A6wo78R@cC2N?aTG-FA3my zPXF6orT28&RK1(s7&Od#Cs0UO9??dcWS!v$>LlUcj20NUxfx}K`53P|3#Cd^@BC6t z$HqOq*N0~kAh-cxN`9I_;*=%emDXPy%of`lMZmjqK9Uka`ujbby;k*}L+_q*ISZ#~ zxu7|8AUFxnkDq+Ve7wTmX^uy|>BJOORz8p#%~Fi)f4z>avx`Ado2j$t``MlA20Qdu z*TO&P9B+9YV8e&6uF^kEvvpqO@(z{cx5*9%saOyFN+9QBo27M%ae_L`jlpbj*nM;T z)F_)zCanqqch6}iE?P$-Pf1wbjD9B&87qHxFQt?s*&{7N8WM)8oyEi{I+b_|p1;NI zDrEl3){7JR3EMyaChzbHlWEW%>o0|?f_>#k6j3R2WcwEqVU@-r?27d(^&8W1HBk0F zt$mM%(=(rZRU&yD5|z$#<2n8)dla9E;3D(?#{w{T8~h2Bb;0q8VOYY@Lp=OP^3VNk z%tM_{*%TRLBK)f>%hBVT$01R5IKszPzfxy8KVHJwYc$WU+9_3?0SRsns57mv~-H;73bzgLKF-ZXbT{4}U0 z{HlRd2h4VSc(e^=Zu404X4c%i{ERCJpEKpiJc)evU7-KEnpPAEbi#Jljmx=o^KqaM0K&B5SW^=5Pr`Z<6yzk0PV* z%>;i7&ZlpstqGXkvJIM*4!$y^-mzs}TKM2Ib7K3`ql1wT_WUiKDCWo;Zt!UZfbWrN zHno;8lPIAIx+ueEm%}DBup8*s=&HcM@RXyhgPHbVv;Vx0k=&@7r`rFjUYnpgami*LP~pzU<=qmw{GAjwQ&DN^=b`in-(hW7V^G8`W^!2MO5m$gI#ns_1{ z^XD-SnLs5+cGdug>-)Bqho&uBQV776oijCNT(F|3L#_QvSGIBVp^Sw2E5dtn&x$<* zEWid}L*m!kVAyap-*h|Wey#LA_jX`4#k*J+Va0W^akc%nLeRR?uW$tGg30oe z)ghkAP#3vQS`uYB%ZHtNt_;SpEcOj+|9csoIB7|i@_~y&?S{GIF@M(exse1@1IAjD z!QY<{d)?Kw-*_0ORD>-TdGP5Im-n|0BoClySJTVVQ(N5+dBlb0s*lqCWaD#!TL;zb(s6O#q3e+lbMCT<^$~9|ST&QvJ${NG^SevcB{$1U)0xHX#n0zf z7?&#`yd1|ZnI?gO_id~^i|e-^%oX-JUqm~EZCo#`)L+2ZU}1rNB-iq}45SdxHhkRu zHPBTs#`k|_N-1Arzf8C0$Sq0;j90DT3XnGAG`H2(@=Rx?^@vpF>2GtWEKh;bP&P}by%*{9355b`S}T#MPJ+5 z+6Z^mI)G!~`{QOA5v(-q>vT&)-itoHEaF(t`V|r5&+c5-z5Taa7;@H>Y7;z8O(@Cb ziPe8HiKA2(B+W>=S$W>V4b%jdPbX@v&Z9=&>(iftubqR6e88;QF*n2JLiELYl{;m# zZ`YOZo|OTF40vAvPzTltHd~d)7*h`?$Cm3mSKCbQ-sEBAv{d3ddAl3kjKc%h*)Bkz zznB;GNRd8z^Nvf+y~#o>cVc;3F(}L@+~5jmMx$vXXKJWuDi+nJoS^>FE_TPX9NL2@ zfIlTA#>I1JBl@5#VdWuai;i_~hx7=S$LD7}E^vcy`5LMqz>xi}i zs{O~|%;2R1$EnPo-I>}mWiM$0lV;B&Bl*|kv8*BAJc_r&w2u7BtTd=@bydse$=R9r z*4jiR_hug1hk6$0JN$^(p_Or!edQe{A?#xK{=@Fi1oA@m!Z2GZbSWAs_ zpBq$B$vB=7`26tFe<{``R6PpxF7Wbcim0a&!%)z}Tii#yA58rc4L|Y zMU_|8L413c0b)Vq`Dq9Iy>OmiGF@jne z#>iO)sYAVoBH;y|3II3v;qXqSM9{6fwID6bw?gX3={F*fediGw?2XJ zrDqsEa&!;w>YQ?PxQ#YOKNPA_<7z+3m5xqzC8}o9JYDbi6tya0PsinoLgn#65{r%j zPiDz^$?eFH8^2Ik{?c$1a>nUdNgTE;CAonTidabrEQmJNil1dcGQM7;v9)F=fPI~( zc+-0Eg|r~Dt>X!6Tp!t{?wY7lO0@SAjRap%3eCcc2a~+kt#U3*R1Loe+ zsd2S*hU~^)?N2>z9?YSY*k14@eL>s^G>{#YOHSMM!n>7qe#*EJFu7AgRPi!)ipv`| zYQJk&zu)(B)GpS((R28+C-}cg2{T~O6xjcKBCNy2%O>>>l6+Jf2ES)H z8ysJB%%osKsOxtz<135x4CLQOUHm>=eHIPM@n)Xg9%@`&g#=^FrYr5r=dN(S*C$I) zoi!1qvzNBFtoz&^Q(D=nlaTJ$Xjb@!v6IE+ypu%)^94WQf)gVHmjqr&WLQP&jg`Yg zaW#!yx4mrFvtFtFDa%;U&M7QK^vBxJ(v{ELp&s6dJZr_W6bQ*CI^tJFPF77xwNn&i znFk$_0X8hA4SsqnD4p>1Mr7U*qpuOM&&=Ix%Em#Run5MzDc1&hG>TuywUS?5eqmd zgBRmIwf^}w{fndrh(t-Sef>}ke-uAvb|OcqR`yA1WDZTW=P`o$cP#&LVvY)!0dp>k z-h7@fn-N?(^xTtUX%v{cV5#=)zSelOj<4{@1JogwxDt=^Cs>R|xE@q|+k$vA%ccA8 zunY;mki*WgO6aEI;M;68G>dWMt-9}5fRt_hReJZ`M&bO|HGq}k*{ecoVMPIq){vNT zt-rELloJ}~7e@vGZ94DjGD59e(^Vtbvch8Z5SQFQdeExSLM1b zGX-#7=2U63;i9Af>8t*uIz#>@f|MD+NJ&6{ce8=gOmCxAb;3eM>)tQ;)V6WKIXEpC zmn)cH!teQIIP3LRcY>oDaYOsmB|v{q>)Lf`6igf1j-DXYSDyo)}z)A&<9? zOzCO%)pMDsQVVd|tIfZ)ILgT`*~92fxsVGvdCKgL9fEk+zZhMVQ~X*V6-4*)|1L;1 zSw?aB|F!5&rsqbJA9-f(k2oUz?pcwl?nm;MtUC@kNZk4$L}{PR3C`Jq1McF5U8sR^%cKtb1i3@9Mb7 z-)vQ|4ZONFk-EM!^#i`XHPWXM3`e|7RK(c?l{L4bT8$UHp~PHOg4q7Aa8h^9WX+Im zWzhRNv0uG{=_J{rehyh5!TW~7p9R=A_3Vx+Q+OfixoRkZ%}>wnDOfD;sZ*bB;j>m) zBv6pm6973%)qtH{+>qE)CcdF3CDtj=-W3-;tz@4mRXh!%V!e$>C|LJhO&K_y1qQlB z-+&4MS&31SU5lN-EC3C=bIjjTs9-f( zI0fc#JzH*yXkT(6rM1tcDA68u=7T8L>)P7ljqI~qg9E9i(9;X4oTP=QR)l&=;KJ%4 zRNlb+SRD0!zqW&V^FDQi4o2=}ZFPwzNyq>78cp<~_&zTep5-&2a4+l9fsX4toWoS^ zXcs?wJI4=OUr!ZXe868Z5!?C9g($tc1g=!LJhZ>VaRFEiz;#HXCUSE{%35bdu)L=hL-=Xz%jx7QFH3GD}s)5*NS8c2&QK<3l?CV2ci5=?@r1a}SY1oz-B!QI{deg8VA z&dsUs?z!69+Uo6@>F#GHAR1fD?piD$<;sP|%-lVUv6nt6*bPnX#e-e2W(^&5dc9D89iq?u&-o*gDzCV7`bS9@bEa_Kc*gLv{VO=b87Z8C z=nO>n%uiN1Y_UKzz|RsodGe*YmZl<^rpnoH$+w$884iu*i%!yR2iW;~X)W`hkR&bf z6@KwR*_9e6J_KxoW6Zjr0CCYuX~A<3FuMV~xQ@QiGoqTDD4>r=$jj?`z^T2lUj#ZS z!-nCjJO^x$SOx)-I9zV=Yb)$y9idj3T5I4EXEve<8VV(1BZ7g_dVA@~1>_ix>GXEH zf$5KHdudS%^dEN3C__@muBRUoNGwylk`lnqT@?HgXR4{eYt`=ffU~8ei#D3~fcuR< zt>ugcvqtvLI3OxfGUv&^uy$Ey9#h$^oEk2U^YRLd!5_9ZHrL}93u_KmwRLrMBYalD zq8K0`1>DZ&4bxFeD+dmS%%RMI-uhU=G@rCjfbB)&;`7FZpV4`N4pHb$XmZA=5Leuz zn(B=cUvDdw+WO}jpI+ynkjKYUm1F7q4C6U&s;|(agKc6~lMQ{|$rwMzLpS8J6=YsK zpo3w0736ZE!!Y2x#S-K#|ELOU9jPrlDVb$^3+W8hG>@RfxbXDko^pOFiGCmicwaDA zj5?drEK^+R;O)&kAG?{RmThx2C$nTaqCISVMnv3@AeamKw0DqXeRlXdn_6%v*kk`3 zX_RnqymoqXqw}xeA!+1yRV5D#dpBM33BLt7AXHW)(x({YIvi|QGgAI+s$|bU_`FAQ z>PG#p3HYLeRgZ>d)D)A8F1vQ0|G2w}{R*^LG$Jcwj9hy$ad3+y0LuXyE-wh-rWXw{ z(Nq*i52Wu&L)_R9MEehNr_2vLa!1%nn{fwXKC07O6+)moO_S+_Ba~w7#B!IhD|al! z3_|986Kg{5PB;*SH%#B!pvf3;t(CuBGd;Ks#c~)v6~e1Gwj#U!c=4j zBx2LW`6Hrf;%}^1!r6}oX!YWFCmy`4R>C0-$jCM8MbX>d;Du-nJ(WXutNMckIHFS; zaF}6P4?AmHHR$5d-E_zzSkM}g!`1vYlIGP7IT6WMTSZr18Nj97z;IGZkOUdHv;~<0 zgdRu~2*~$7|H5X%BHHj|?5AkQv)Pd=@5!pRA3l+}H6Vdeue2J~9mYvV^`JmharQwx zRONpQknfjP)~UpukshP9qEzFQ2`Q}0B-umiTGPvH>Z4FD?h*ESg)jdzRH9`}a|p+t z!Z5O7msEk$azJ;J56efxSgH`jYDJU`<(#M@3)AQ+YGNZ;-l0 z_6enW$V8&;D?LsP43Vi6q>rHA6nVJnb-H;#68{7qr1^H)lw(kv-_ZTc5P!lIV2ujaeh$O5L&}Q0A$bO$}Ku?c0 zql%e~7bTF!F19@9$k%lFu*f@*Bym02{Fe}l5Lkmv*31OUb4=4R3T!${T$-CQ&Pg+T zI%U7hSvhWrXH%9NaI!T;$wc@@XI9DPp!5}_Z(3<#e}rUw_a+;oVVOKrxVJXDu@mI& zFqg{}xB(@O7Em8ax#iN64~-x8;&pC~;y7c8W}KA6QxE#g$#azRq2;OUe&v*OJ*2}_ zK)a_Xh7tk8B=J2b_s^R{9T?E*a@RsD7->+`LgVg$5&Rt0mw-85HTi)WzwL|9rxe&9 z;u`})eh~@?kmdIwc8Lr4)^`Yn3vhT*m?HL@^mvz-J0X@M9fRdVJuw1&XXP-99(SuxyK8TSLKuh2_vM1j!;P81d_+UuiisdC~+7ZTJy}C<>D@apT7}uea-;j0P}9 zo|#T_odb7Y^pRS+g#PdwTx{AbG|g(4uo|P1<8ggT{+Z+O0-aB{H7tr%Sw$1kU(Pp3 z4N#G@#ffO*Kb~KiJy1gq5CNB(;)U@3G>sGms0?sZI0>Q$l0-nBg|Ix|E0g>x9C&*D z`KB!OXMKR-8dO{gb{WAM?|atC?X3ThZ^f=Xt6*Pd%!)IoxsL3q)8rFHDk~>{`(f5O zd{#J4`^&C#iJ#!6cd0kbzgMeS98cz%9XzSKglA!AY=KBWG^>}n!Tu~ej3a=AXPkU! zz`Yk~?0e*uhZ`O47C6u}!K)+m99P(y`J&{$P{<6><;FB^SO)lt`Zj2xuyfQ|Q*f@- zP;R|ZMvj|b8X2e1yLbYde%)kU`_`~#Ckbl{({Ji=ge1`%$@ug<`l7}?3rTgAEXmvK zR^(=KWPDvN@}w)zT~8Wq#0?xuhUYdSmz}rfET! zRpKp5j~3b}afykWF{)j2^fb+QjA|jaWbC>b)(lvM+<+evsHJy}g@igfqggNny6pw#X%;I<$SbwUdTxj34y6P2YfH#NrbY!p+ zAp3gbBt*$+r4lYAT6#`+@Re%HJ(H>=_eZPl*vcq(HlOZho)_Tq#^i@E!tN#CHXgsD z?`XyzXU+gEFbC%M+dBUO)5X;*8~x(->mljGGbi$%g*NBQamgp$YduP^uR&3>)HXMaLPY9>orLqd8$u*_C!fGy|LsQ>W`}< zS%@94CAc}AwO9xRST*oz9n>0axkOgS-lDvR6()s&W(F|4oJBo~VRz&qa6bPN06&7+0Qj_GVl<=ht5- zPVf*~Pg+;^?(PXoP_X-Zh+#f-SGCaYh<;#_blMh|qS zL=#nS-#%I}8eMtx^{`)oFEx`a%{cBh>(;*{RG~FOlgleE$s93-T+&+!k|T7_o!Sr} zSwzFIyQqRNWwGdLK)6EpwK|PUN0)m8U}_6do&l5p9-j(f!%#Q5@ptGao2$B`35|-u zeK_uYIj*eo_8bD)2s$N>(1CVYbWG!V3;p3{`(QeHdMmBxvxe-(BWXOV9xt4^BEF7~ zhvAXVuN}{rT4sdUMfL?t*Dkxn3qJ6 zvzm|Nr9(dIZ3WOG8?iC;}vM;S1}60R-_clV7%icY#Y$qV0K5vp5%s2kX1*?mz$ zNoZ5D3H%T`|-x#1BD~~Jw2*4`(W03AC*?+;P0K_4-twHI!{)@4j$Yo*9*GeA4dw6 zlonSc#bfG13QnQ6rGipXiMBLFYqvKxH6y-iKmSg(CKMH(_o$L&M~Xue8aV0KSMo4i6))IA9&>)h+ut$!`~W1 zXO)}-h7I$`jyUfxGCdWwC5}Jk8C}0r`ItPMpIaw4xaQO|9#zFwoa#RO3=*j4wyNjx zB{+PiP+aZ;Q0_csi}?8?dJ)c0pMDh;nG{yjc!!b9xXZ~YJGfhq9N zXxLg`B*-Y@{=@2&K0M91W4$MK-c!(k9WnL;jj2IPS43=oW+bh7c~=y88**i`ZBNb- zxI}o`Ex?_ z#J%_@FG@%93&ABEn(hrrk2M8fSX_#l>&)ihs}AWtiMWTa&wLE-cMq4ke)vzn7vr@s ze9=F95)7aCWgN%|c6W(-J3*Nhh&AVs8*Wy_r>ZAROcYA3v1W9B`=VrB&|vr7;+-f@ z&z_H9A>`V9S0jG-3BNTfbh%wjHtuOw_PVYvOMI_|iDhrsyN!?4d^*(Zj$mrWLVw3_ ze+sJYxwXWYZY4Gcx6*bLIg}mj3UYG+>>7#*tA$U~a;3Xz29i?hJTr4JiS&8S4NVE7 z7)8yyFxLNdrOu^>=R;N&hJe*p$8rt|_?_jSuR^ooV&@azBB9g*pGquCJK)v{Ol9$z zmThW8j+?xiXD@N70u)ArpG>hK+V~$WJeiQy4>7y||3-&{6eeImWUeE>q4WYrk3kvB{yQ~PZkc&+Y) z=Td@L`9Z>zOq$9#tA1xvh@8ywE>(O4;ZM!WNQ@I&y#)FLA({N1d+me!sx2 z_T(;EQ~5_&Y^{!(Z-ph9qDk`7-Q53vxZ$(eV1xi8^^!4*DM$}<$ibyCvFZR7;#iH2 zdRGbvK98c@vaBWY)*+~!ckpaGOC5ZJCA<)EtdY&7ATN#$By1IfdZr}Rgn6-`n3txU zsbx}|?_P~5C?Gw(y8h5dc(RIl;Pe(6vPwi zMy#o7aAVFf0Ch6P2kHMf-LvC{9E zg6lYuEBX#ZF0=x-BiXVijat*+D%^8hqTXNMKAS%FdcIMzg(=q4TN`)T8YlHQ`h;>WGa zPRS5uUg(f@1gbgvv65F9i-1?93$a0*=_i*w*~X@cgP&8|mTeC&Jfk!>OjEX4IYW+d z#_H~}ag+(i_63Vqxqc2usT5{sXrhDh87b`L(7YMosnKhEvss(LBU!WCN24c{vNcCM z+CVy;a^aW##I<=dji$f>V6ufcRG7Emx2da6wTzn#Tg^9n=6gP5pFXai2+tj|TQ{9X zKUGC}lKP^!JSB7__JlB*DW+E`m6tH*pyud4-e4rtC%{|h2vYUM`rYOfFDvC8P3}s@ z=u14KO|y4&$to9J4$DmYyohA|InA&iFl6zK3|aL`sF@@wlmj{F08;&YiBD~@jbDu3j5{?V>v^i$TE{H&0INed^j z2UlOkL2>8eOQWY2b*=z6qQMIT<1V7clI9}`rISGa=7~Aq&STwb~yH)Sg5a&^-DhgOmeiow79I6$B>Pr#t6@^o~*{4eULUWVP< zq{7)6+gD7%c9IVyZ{kNCg6}8z*VOAL8|9lgQ)T6vQ)Th3&nkd>N$x$NJBc!@*;0wW z+zhNWT$|#M+e8)b%N4)2R>{x~I&Kh|`tNG{k}!#)W8P-9O<1pe+nybteA^{p%GojG zRX_YM>IR8O&8YO)f5@)sO^U?+*zS4&g+ozm-)^sj{~qm4)Z&10Wh2q#Z24(;;OB@o z7WnpTG16GtadC_uH5Cc($^rt(a*h8Dnrf#HT1I;G{ZuTEkNo$qD-yclkCAe9hgBM5 z)&e<%e-kpX_CU|BFgs_%P?hy}ep#uCbqgL5VXfb9S&yx6@bI_U7JgKTT1FV@9nUs6 zY2`s7w)82G5%PrH`!&&TP_M-$4Z1w4yuDKCZcl&gkp=XKS3ygNV7Qq5w~YoQjBCGH zxA91@kSAI*axV~Nz3uU)B2hbCVAtRCn?U)}EkZ~gCJYDt5DA^Gl2JrF-*BNar>>j@ zzN`9IskP9J61I;>FSX7cVOl}gPa)5Aw?pFdWHG_7nm0tc(y2gQc4rv^4QPy4IKXZ5 zb}KDfQE*;VDFe7f@^ta;ERzLofTSDG8KZNLACauQIZB=AATkQl!Lo9!Ts2dGB#D5y z>OhIkB_K!Z8zA~yU}&s~wr@DCG9}FzO|wEwS7hA<>2$g#M@0Z5G6tMnP_((2L%kSxV3%~}Jyf-_wZ*kzY!r-o6~o@cem;U< ze*KcoG$)njX{fAjMV%jx<=i(N9xCyDHH9XJ^G7CxB&hqP0BE?+7hT9;W66~VsveXC zLAC4Y6BFRpyujCrYs>e#=7Huj3DL`MNb@s6oF%0l?ggd3KNr_k%d0G)$~@jbqarfy zCR^8!`a=;J=!jfERm|70f^RFl+GFFJ>Ce$arMp|&hTg(r=o2wC)?FHl2Km94f6s%-|= z>k3mtcG-r(cMQX8e^!9HJpD}Btj8)545Bg|dd{$v2)7GARZOh=BH$P7- zj3WK0F#4nY64ZEzj!`l?aI{&EoDvv<6L|?D{B)y%V0du%%>WlzS4vDvIQvjpXI{|S z)h4G!-dil!m&PF~y3ozYQL3H3NNC^zeI<(?m=g_tuOtMTT1*_5xJJHJ`vfHeYDR_H z1GJoagn}jQ2#ry_V@u_>T)r{8B)u3dD_dX%yUo=!2~6p3b`1s6b;i)DqCpmn+W?P& zv-BHjB>!af=Lh3sqU?OT;IQ_x~u2DjBV)whJZ{Qh?*kkB2(XfrrA zr`^+c|F)M1)beKzpT9%Dcvn&v=*9tmxer$5tf!4>5F-^!UX?LrCsks4t6i9^dY$im z+<@iga1SiCrYVV zaTD_D`L!;;tzhn^lZX&i;KTf?2Heuk|Hc>9fHD$`A_wzc08HT}=(`UMp7SeqFX~3_ zNIGDLX_TA7eHsFxbBDHlO$~>4<2I4LeA;c<{O2i>KehFmoPb$-JT}2M<_|A52kYC_ zgjUeUtC0Jd!DrrYSpRdLh$uy|-JonZ`NbIVxQ=%>!k2mSz;C@^!kZCrnxVMOD{Dv{ z9y0|RI$aDruT^^!MpiT0Hk`J?-FP^T`{8mO2~Lyft3KHLU|=W{FamS#@+RVbiM*-M z4Pq^qkXHwlQto0q=51|k__YhD{jGO6 zQTL}nT7d{Y@Z~E4J;y0>OUuqjjMOVejz1Y-W6mSpKxiB*~!?ympk*XlL? zXhhTFx=!IEJ0e9u@}*BwE^WQ-W&xf2k+K+Fu7c1ybbixVnCXRXES#&E8WLI~_^o0l zBS<+&g;n?ISIN$`;D1`wLAYT@bBz=Pj$QoqPKM}#ivK+;!d4;~vcOx+J5XZh86^n5 z|D3p;-xpYZ^nPRm)8VVQMyI%dTJ4((kz%%WM83cd@_^;!a^|;^3$jLkz{SQ15@u3Q zSt^KLN9`>5NE;6v|FZ+jSzMu~setVN9D-E5Pbz&tY&Vrd3yBw;*7hM;Ge|Ew{kHNx z@H%-}ucm4&X}2tyF+sQr1$OQyn(j)p6osPL=q{Z9iZtZUHIih> zA4C2eGN^|ynjxmKNdyRP1)#aeNQ@%d)X?9|QTTZX*l!bT47M=b(862HoGMhC3^isH zM)KHZc3Aq5u2XlrB%b)1g6bY^tO8_!y}rB716ia2I4_WN zLGFDg){j#y4ix1IBF!7Twhi$UI(T+++_Pm$GVM}x8_JWSDy&Gmy;fZ~fzg9mwrBn& zP&I0@sNE{`ueD}6>({*;q(|-?C21X2?4|AsdWY*L8Z?bdZ9m_rRe!_RS37iGDW{(Z zw)lTajx*Fgz0u~RH<3qeo1e#*VYO>NHYC-R7K15uCGu({dcJ&pOJX7toVxk4aHcda zd(*Ok3tXc!IpDhJxTQCKl3F-;UgGmet`lhrJENpp9HJPbtu0WMQQ~>XU8V0*&QKCz z^oya5B22;gu834VcCE_;E}e{41*l|4YIU@I{%`cLj&jMekk~4N^p(eDz+S#l> zegP?`Yaj&~b+&K!9`Fh_pXyi~)kd49Z?Xh!`y{|>6697%Uq_h-ZO!#*jx;P|N?STs zAX4#FGBk}{k3e=1x8`ih?5g6_v-@R_{?m$6IYOE!lt}?VwJf1-(fC-R_DG6R*pu{R zG9@*a4NkWFGs_D>xFBF?vRxXdBW-$^OLk4DDOj#ddv(SnOO(4g|3(}MW^XjrLYoU7 z;fp#UEY-~deP_$Ior6xr_E(Z~6?(_5Cif``p7u&G7f}5D7)_@{FT?xuN+o6n(Tg#_ zC*NxwA!BW?>(%8xvqBaZ5C*wPws1WG2EH|XPJYNq-uG^3mG%GIqqALVxT}@xG$`&R z^mfJ#l=AXhsxa=aD)HW6CxU@e>1M(=)m&t}x#g=@@&;s?RdbG5s+#aU{LApdPFGRs zeLK>}{eP&=(`eIDxZ{UOwLfw;?=gb8<7qrH?>dJK@S@APq;UT%x^w;+Hk0!D za1ooFIVW0q!`QN0q};N2r*lQVXaCI+0^ukPr|_h=!)$gj!<}$y^TJp8)NXU(j-UjJ0mYniD~Hz*hpus>Td>SGywt51vX7z6(*1?{VODhOkX zvpL=HX~J`8WAr+YQ!jG8SPNxJ4x#JKMYOa=+oRlYE2EugJ`Pn~_!lEF1Lnuo!L;M+ zh{K0fARRRiOuo671z7||`_t)S@_xgi4XjV8v4op19ViSE{Qs9+gbDGG^6;LI^%Q2*_Cg#?f`&Viy zXx7I{(H48jPMe0a6dF^ap`7;zBa=fFi*WR2#8$-o0Fn%do~s$d?5JzHBeotI?`jzi z&7$ZW<@QFwRVP{?^yUpfcFdfxq1lyw$}&8V#w%cOnfgU|T%kvY`@u}cS7U}Noa?C( zG!puMF(@R@gt};`Fw}Cch%u#$LA%pZGU#GINhMaFX6sJA!x3zryx;2pyV^8XyQbJM zA_x;lUF!BI9|=9ZR8RioG8bgT*_=Ogu2Ca!?W7ZG_!K*JbRe5Ayh}=o{^Lu-7q8!W z2t%Ds^Hi-#7~+FRyp_eDzC8yZL1|pRP6}a+>!Y$_q6Qj@Q42Af%chyP$R*qoVifA7 zim^w5Lpjj+TZilY)N-BRYo*o~jlC7yOzjyd^kc ze51#`)+d&^nj!;7e8I%y8?}v9>xlE)_WTKd%{astK6+^}8*h+r%SH;JzDr))39d^X zI2KE(39vu&hTn-vscs9UHoVnYEQwiY7`DW z)rnkZ`YkvjWL%f|@_3BEeC@`OUksFLol=|h=d_(_fAAkYhCUbHFyc!O3L9yCEUQ(5 zysf1T-Atwo(4!!J#f3R&a%#B5gbQ4Bh?Q2PqpGmd6q22q4Oj5>Gn%aYV- z&KkI-AhotZhT2wlFkJR1K?#}+4N|F>6fqNiMb7(2H3c9+c-ma}e!)<}fRt2vUS&JnBwxzIOxY-6v@QgMxBK>_YNHs^o2xgzXLjL+y zi^uRZhl4{1Hdw1~x_8ps-o8WG#)Lj;a11ucf%2-RWZ#^VPGTU#FC%(w-rU=Y%028* zT7V*3Kx_55BvK{iT{L#oaV*n)GuG!;(pw$Vzk1Y=o4@g>Z*fAuO_xca+LdpHwDJj>YM>xejVs>6v7=rHT{`8e$>WuoW|rUGyAS@Mezs9Qr0In zK2|6W#P_M4E)E(U_h9pe990@zXp&f`v*F@)>6r#QUc1ToA2a;tCY>7`jQKJc@F7ky zHRyq1qLHC-PRea(5Myx2t6F%stEnc#D1Afpd4g}6Dz6jZdI;B&deB_C&sm#vmr|c% z&TpHkqe0nFlZ4QEg*Pq@nQG1XbQTh-X?gmv9>1`KA2cj$W7gL&XQ#clc+0`QabVuD zH#sAP-2sdwUgc!Z{Rdo>U|Xl|Q$5#DIv!B*`%UffZ*Ce;XVf<6<4Spo9!9h(1-eR* zQ0k#4VMxZ3>I#>}4+;?x3g~}^|6kO#NM4n0e8b_@>G+-|6eO%tpS?DuBzCWZw(!=E zIvlI}{rFv(v?F{C)3Nb+tc{c@W#4e*i`s(#PWr;^P=i*+Dxw7TQZxi*zWx8Rmi~^o z6ik+K*mAEsTvZzAG!ChE+bR2PAe}GKW2%p?iN;X&FMriPOQH(t>TA}RIYx^b7!`$ZednDzHT@8 zC4hc8z~$(;YHLT+%j`=#kVaRWI_OlB~+mtdG2V`Ok^T2{$ZcPgaJD&*9|`iAa-rUrSQ|&=Kll zX;iW@_+NQYfI8_x!m$c*NYkWd%+9GVD_wSzEx%LllJ`$`JaLeQaUnxlt!A*#g@E)N zr2fS!gN_zWOhAKS{^0AWB&qX0v*cp0$;M{?WZKA)U7nVsxzY~axL2GpxUs}f>Dvs_|p zDn88YBlbD_1G`*QY!TRu_oVn6dSDj@@GYV9{_dgiyO^Ul4n!MFV@l4Q6 zGGt}wjSdH5?t<)VweolV{9$mZL1ipzU1-(b+A%L_G9PsQwZHTxk2j=3c1J+{uMX|^ z@jbFU1vRs|4N*R5(b81EN%=k-y_Qan>{umIKK+!qyrlAT*Km+q-_mwj#K4^|2Ddl< z1dWcF#Z_Pfn~LYz^3O~*G>4Khs$L5Vx*aRY`pA{8g-V9R^!j9m7Q*JzHOYv-1FxgY zJM~N}!GD>t^GAJ`&5chf6*nAChaK^?Iiq99~7bHqB6EXj@oMT|oHc~t`ioUs?zv3MMuiM=Rk$^jxD&lMNv-p)uQ3u-7jK*l;VxNzBbF0DDjiEEJB?#pMXOPyHIzQ@ zyXq zh(=g|VwxEJ@m|?rCD=D}MwZ7DWfQtXI}Gc9A!i_?Mk()6RLTw*mQdD}d=N~McIUe( z6VfiMij;POONq&o4I->q^T`0N64!ah=h83jZitJ@EvyW!+RiwYUm|_HBVLDJQ0nbv z37U&WCOMfzC51K(ONg)*Roz{-_qgbeRPYQ>Uu=dWLwkdT=7BJ|#TiR zop1n22xk3+VW3+2n!Kkx5UgrOXPVZR;qqytux2I_A+A(wy}6I zF%OJkti^j!0l?ey3}7Bil)3!U`C10R3;7h5T-07#%&7^{{3AbT0I~Qi@8LT{-bT?H z9x!P zLj<#K2~0dP<}uriW37XCy5&K=+Azb~cw^^+x%a}WF6&bBdnj!Q=`<=e-*ZB^<4?}^ zys)i50h$wX??Gs~o(e`E448MkIjkFIp(Zj!77b^<#a7vdcAC?Afi?FZG@_-XvfqWb zHhg8~JtSch7-z<)YVQ^j_;{EYI$oZsF%K5UDD$XhJg~@#ex$GWH$1LHw_CG3JgNeo zp3-Wx)E;*-#`2*vD`gK}NMk;mA5B&ONW=j=xAA(4xqY_D1W4(12iJdBUBGhw8s_8?Fl<#w76!?X^#;+aO z8|dVBt_!&GAHJx|&HHY0mfKeWEFS>K{Q4;DW? znQB0-GBYF|>Jq*XLlVXE@eqpnuh>9pB$MJ*`OJSguOhAurzZ5k&HtQ;Zu|^~mynS& zGbZf@Ry3sUM42^udJ2QhQC}UTG4?0s3zJqzq*4^2*5wg5+rA?nckxAWC;Lw;{O_n= ztW+Zn?hY2!QjO)>vY+=}jJa4s61&L3+^;TEtr(K6?J{>nrPVU-prr$Q&hBZ5p*ZdnU z%w2R)ae@q06NMYT&p)#*-XWXC+96-Yo#4|}>BFoqm`*3-+0c&Mt22?RDTcbZv_h5$ zPo(n&ni%F+eX2Hj&lT5y`feM18d=hI;bJBXycV5AZI78)#atCeAI={vFx^3@0$dz# zk&#)~_X&<5t7bdPH-~bWQF^Y0S>&aBq+3p+xJrx0sP2^ja=#eqIh78)3#jLl)V3(A z6GFDk+X1FOdZkh9bW!meOgJv{e?7=NEP9nFPrmlX#tZ&g94E{@IO4oDbovAICX(C+O!dFcNuf*X`Xn7s-D_LJvHT1rHcaRFYjY|j61^%;XFQP26WF@v*A(Q5uOUOm~hHsAf5c#r% z`caN+pq%NN#3NQB1(R}v#N6C<2QZ*mWIEc>`ReTF4TvycS-)KdI%`*79*|TwYzV(| z&K`=&8y?no?FD$9octgFW3dy&p~1i0c0rP}ra z(^H!jQ)=3*jb}G0?w+zJBuRqKlx9jsoDCgT=uBSv%TEfdDAJ() ze|G@@%cR)~l0?;uFQAR*%GwCaVEIUwt!TPVM{hrgtBOx6TW(TRn}{Lw&Ov!f2=!0> z+g1|U4&~;kqLpnT`S_vIu|TDx^n*X6+ozM*n9?_DqR?d&rdpt4_-wZ2=Z1l1GUGs% z;zDj^K)43om}|K-iTcm1aH!sa6F{_hRg1KuMe#eUU6W(t;T+{YTi%AKnPl#W)N2(r z=IrY5r@MfoBAH6JckU<;31~x4X!-Jyh1$ad(3FLcVDTl9gKT9ZTQoC&T)cz87^j$0 znZgT;CB2AFg9UibHtl3l#g`0IwX813vZw1()P09^ZuR@09Hevu$6ui^q>&~_9dhFa zv`4Jw<4B@Er_7OL^+R<%>)@1X7_pLg6$@0*J-tv!Dv%f&`W<*k+PS#@}Sa`L`AO7>8I{OCc|trk|JfY%tm^PlUuGwAnzFdc4807Yy`+dr2D z6FG$dfpU6zv)CHfPGsM}dsC1bQ{1}C&qjz|08@f{1X5{${^jgsh(x9#Kcs)U2+6Gs z2!C<%6+-_neob+F#X|{_#DhE=zUFDicatjR{b0JEq_8cajZ2{5gl3SzV-@9)s?G07 z^SY9LTn$j!mxK}x_Y=XcT$R};5tkO@+9eeuZ?~KNs*uFH-`AJZ!lS+VJ z2ng($#*vq|Dkw7dNn~>p<#h#a!k|efZ>1I|JBbY0aPD4sg`W*5{F^+}ox(6ypQUP; zr2V5k^?se%9?N5=Zj7#iy|QNsqKGyDx?+t@52_T~_{W@*16OZ=+@M>h6&&bY-FH zpB_x{5tP$&8kWOq0*}gm|8nAVG>D5h&i)%#5Cebv2H!F`6L3X0WwuD(SlL`ocbvv; zB9TI75>*#NYy^DMRwgs{tX95sX+u~(X!pt*WY=WWF{=(yna8!r zZD%<)c#sXDAWeS@QXM7t5&5*2&ag_ElW2f@@>mG*p^WVQ4%%&I#Ue=7 zY-@MGvPfuSD}VD@-z<{#YRsoHi;gVt>SEE>>ZZ%IGenallbA)ZLcDhwpy$TF(oONi zy45C>T)mz=hQXlCUuh38ex-$!$mB`wp>!$em1sy|Jo{Oq=WksvEB zeI8Q=v4cd;rUF-wbK5V(yvvUiF!#XJgQ}9uurt&Z2Hn^GwW2;?QmQ(FVU<$2ugRzL zoV*|OMe>!1njpn{ZO(T|%@h_%l;OomP~M#Kda(R&t5~qAC=A;X130)+n6aW074>fo zo5Qo(Bf+xDs0F{sF{QaQiNg{1qu7!+oUMF`dM6sZAd&PDT?DTu0)25vfN)O?NN0w zA|r9-Ywrw5h`h@wN#+#|pIISh9?#MWa^6$IoanAqkA~a%SUjsksla+W?+r64Ty$v0 zg)kReg-4+KEH7G-jrM&4eAJx^SY21DGaHkfY&}8RrekL$vU0WU2;5W|XiykKd|=FM z6ucla1TI3%UkpBcglukEmZbOoN~YnYUP9Mj8;ayTQAOP;&R~9%KnJPNSXRfwv*8=> z&GEdP%m_~o6n2gWg#c7&4i%t%#qu8*F}i~6<*Jtq2ZMD-uq!l%$ktWjN=}~X9@c0P zXXvMBZgru#FgR`ZffCyn zYGOh9UK!sEFoAoE88V(+!CX^^I^*;iPe4CghQM6_x;ygg#Vvs zNeSxyuc_`vlx_a_5h)}j=iczGGq`^pyA8{KSt3#=Pmy|?FPh=igi0u)D*bXHPTMl@ z9f=T_;ZEF|WSDhNrJ(}gz`sMU1&H=7TcG{c@k3TU6kK>HOA%l>1l=14# zpnjYpr@atz6N#<70KP3IwK)FN-$%TTz^SPDd6=?J0Xx;MCAF_iwhi(->pnC+{;6Vjv%VaBq?KQ@__kAN+0*TErLV{QG)CK6%_M z)&pVg)}|Pr)xNacIHImlXyAt^)B8fG7;x- z9@kl_|8$hgejTBZq&LsHCgdQMq#Pi=9#w`i>oWjUwCKtB#$9xaGk3g?zi@^qqRvnm z@`Th~^U?aaapb8(Tos5JG`)_#v4S3U7S0>Q zfO`&%EGyj(7ENMO3#5ynxZ{}=#k#&6RU6yYg*YZ8voY_q5QSTc!B9)?za}?HN??+U zVp$i;fNSmZ(O6{f4^EDEFJ{FQG6GRVVu)0cQck_`|kYBb#OVNQxme{SX zneuB~QP=R$R4zA-UyHM|cL>9dl6$ri;UP>wHju4&0jj5?`8Q(ir#$;hdd zZstU`Aa}~Et%!w2^eyLo-9fT!cHRJ;i>8WJj`liyaN6kUD!AIFCo_o%*zk=?cH9fh zSHnt_YnaW`WsADcCp@80Q#_J0gkSy30K5iQW|tLK7+)_nzw;C5K*=c$V@c0*z-pl; z4EMM$DiV#GG++J@Zt%_p4Z8vJ}(BBkFITTZ-j^Ub~e?W#ce^`2LB4mNEqlfIVS>rjx*>ox3Fg%Qo-t_@H9 zal>`}ajbIFtDYU|D=wX`O&-_^u^iwc>wPTE5lMBJ#SmDUQh1K7JuY`m@57X+V>AJW z#s5Ttly&>2+t0ajVEF%`0R~V+dhNeo4>|a8X`ra<(@tS^3W|RqR;CM@Gz$NJ9;Z)ZC7W?Z*ose%Qev3zA zM{2Hh&7dYv;$_QtXivC2YwCK2=UTfJ4{|Qmz2l2ydQYFH-P`R5*;Z!_%tg}#>cjFt z+`@KQ$t;A#=&Ww`Z*A_lWw6LuCX$_>;P2399OO9D4ml8tSqBclDj9!9`@g$f6Zi{X zY$=#HOUmY%NW#mzYm9qeyVP&9E#7FH|M4z7JC-rD3HB$sXo4+o$OIH-JgxIG;N9g- zv&zVpVC(M;%po3-!+QtrRveInty!TY)L>_8iZ;weeEsrR{KQk=_FsLFaLj()(Jl*O z^;qMn`klzAccU<>bfX`X2XYkWOF}~gWeQL<*3;p#Cs9hNp+tSNL8xbdlKAz#eNNql z3LXXgd=ycMvtuTTzPtz<4?Hg)t~@YIHR6sE(GHej1Wwr=DEqq_vTh~8I0>2dO@ejC z;@1SlRT+_bZ)#y&ZF2AF$sGsZ*ZkWKTN=?n_-t#IvoJNju_uQRg&4vDdF_;WS=kKG zaFd3z(T|aa#qf1KrGBrDtP&izZiu9swNs8dsi@B(M^BF@-p4-nQ1_gqV~Rosiy#Sl zi_Ac}{RJ4nD-t~#9){0f6jT+6R*a(2Q5*c9n~LzsJT`4KrT^4R91Na!JD4k141gv+s~CX= z=1A7Xp#szBd^-Vj!Pc8bVne?ISNP8^1}C-4e3kqQwVonYz{xGJ;S<3eXz&gA)JKn3 zoT=Zc#C0KpF{E_l1|a+5{cPX)cQi{iLuzN$w3FmI{O;d}IeaqGl6eY-J?KZqD&sFN z&|nnyqMN#n9=^yY_%9Uy&KNlW-^4*gVXqq*k5uLmCIae*Enaf>%7wi_p;ac^J-Iv-^Y zUZQb);NVe*%!c3Pib=cd+$ZUQMXE8yG~8r$2SW1p68{du0C)OSxh|6wLFkZAPI8A3 zzW3FRBiBLdaJQXg&ILVd0K8gN&oecIHlX>z6gH77p{BTY%P*%Uz*vF8n7U{C zq|$EF?-nBm{KmoP%%Au3I?Hhu$ynwhT=a^xj9sL16H_$l<_iCrH*E-qaRQ|7{B98Q z@QwWb6Bf6<_<}cF7n9Ct>_J1l-%avvrf!a9+-HqR-q+WNfYTj#N{}BpXwoEIR^8>3 ztblfgC!qDKh!x*#D$aT)hz=t}PEHrO;ZkjjtJB+TElWPS{?PSI_V zWzH<5uB5L@z$n%75eE#lbjfbg|3lbYM#a@F>%u@FL4!NN-Q6t&_r~3V1b3&Akl^m_ z?(Ux80TSHZ-Ce##-o1}~_uM;1kN!nBJ!Z{0>yfHj@1uBi1{>6pf6j(ojUot(?|A2_ z$Py4#Qk)HLKR^Zhi{Sg`O+cZgP;!`%@g~Nv(E!~T02bFpjj}FNsrWTL=GNyOUYI{- zrkNAVio!vI-Nz21C+o_V2!ObPzGcOqe>Ep}Cd(gi_ z&5;jv%A9?79F&Md^xmRyU>dA6yZxEOa>k-*MF6AlA^b}_2UJ9{9&>+` zKEL$&1y1l7o7YLxJQV~=WdWg}d1HhNfh&!Kun_mo3A!XAI1G71 zT%#YE=GhQl65YL`V_|=$=tNOKc#=l1F>*bb%q(SGQe*G4=GZTC$VL~!F{_Yoap(~M z*y9fEpD;(u8XbH`jlwPN*F|ll&*C8&@no^R#!;3G4?tX#E%e6U__(#DzCj)d^e|Ou zbv0eU2zZbOsrYM63n|pn2fvF%+XCe)F4ipXD4s*=vYZDD(s>bYWEpl3&L_@XMT*$sZRLwM3Cr1lUvcaI@J7sN(&h1rWRP(}qg8u+;NiqYCI?$js@g!^d19 zB5D{b(24SKut&Z~CBce@f1@aXyjOP%u@+ z9g8Kg5~n*T`pbuo*??z7Vl?h4Z9~c z4V^O9;&xYz^#L-dB_hAJj&*M?sU*$*svP>TJ2<8;_)S%P37eB83e~-#@YZBh=K%LL zX5j%i@&x0pupN$}ADZ1A9%^EY;=9kejOCq*q4fJa>x^_3$bxi%Ns+n|5hTwOhPiMM z9h|oO9eNDJSuxG`kv7vl7!YS~_0lR5)gUz5`OS3RX6xve(4+iE1~fLEh7jMYu6qEg zN`2iA2|WcCWH6Yh>jjfKLYfcXgb>BynJ6cA7d_^X-i?`GAJ!8^V zL7~)>16A4KYCzR2yn0)c=quC-Ms!Lu66+LvQ;#(4^tWQPVZKzyNg)LK29s$Uc9x8 zp{ufB)a{J5fELx5Nedv+^n16u8DeP8DLh%T_gRWC0*QEE^IcZ@4G*$DL>OAod^dkv z=G7+3rtPpeqU=P}H?2x~($DNwm_cYw<0qCA9ihz-(v%$s`BPQX)7{)C-(q@c7x-T* z7J+N~7}i*v;bN91gKa@W;aG55^3R(8qnU}wUSwI_jdW5vmJwqzEsCnw$j)YTzi$y6 z3~|W1QgXLPfyZHY(Ejc0|3%px_BKy58K9nNh_xtYk_SuZrrl*52Jb) zOjfo(uR~N`f&BWF_RToQF-QH=4e5{kv=GMxL~4P#1U8}ru62Sm&zZ-)+qhTc{*Kfx z!mJXlIn{vhAq=xk@!5JRO&W&_vlvd?m27N#ez=jlMc1g&o2X3uHPgiKt5 z-&cVOp$N1nqWkdA3!~jA>9Ixl{$Y>5?fm$$i|-POPIbE*8zMyis3U4l@oUJC1RQOJ zKDVH1O#^))%Fx|^2Bj$>PL*!rnGYVmkp$nO+h4BWNpd^)&o9uhS-9Ahv4~a>h7U`F zep`isXg!9p&e(%{{++2j4jc*x=Xzsg&Y_mIacCK4z5+Oa1xauaJ@ z>N128qAeeuN(Py8C{683X6F>&_U#wz7UIQ@t5MqRGpz(%T~V_k$l@)Eku8w>O&RPz zsjuPZJB2LMeU`WelkGjYfe3`RcZfaK?*F8b{BeNmvOx>IFTMF& z{cYVC&35Q~tWkq0+|ZgQgkuV2w*2p23KDje=mLY;dXYGqSJ#a0 z-zQ)3JzC)llN4M`aN{(|(AX(iOqxltG_Vb*5{Xf)QkSiM_5~H*Ss3maht!&q#QG%GDx+le`-4ASKioNAl+X-L=g|N zdPzp?;$NZ?f`)!xCYXvCF!e3JgVY{{lAhvR{I#kq#9j?E;|&$(@zXsr3(tx)c!{Q5 z);}JnJ|dg9tdh$FC!VQ}$OPmQ2II|OUjpr$A{^sMidcQ-guXh2f85I_+m|-gK7>3z3FQ$djG=l9_2Wlz0kV1FI$kYzN>Ep= z_$q0Kh>+qf*@pP`wTUp52{*R9m<8IjLrw&@`d6&JAx1Iu*j?%Q=Tfl zWqM~X#qR55FSHq6c+JH`ry3J9pT*hL2WB z94xv&*Z;CfO9ANE)IdWQW%+37D5`2oXA zC#oHZc`Ea3^4ye(xuC&BJ)&s$FUYJGFS*WPK69*>4mS_?w>fbKX>UWDxtOOs7rqQq zHhshnQleS)+(cJJ*;QbPJv>8Jlb84TDdY^HNB78fP>kOR0lJ}Cv`1F{7;~cwQHs2a zh0Y!MaQ`2XSk)VAUw{e05&T}!Tc~^pG$W zZLWM5hDUa|6p?!Wm$3ndiHs^tYaf5ldQly{iK=3~q1KNt#~0TZ z?b!fk$W5b<9qLc z?o}ru-`AU+Qks0_L~uQ8H@ZYyEA3k}xDk+V58?bS>V)sBLO5I!{Bt|}+c^tzMEwkb z3gF_pzpqExQBOlXz3aD5L4P~@@z3SAp_dW;(2JstqSu5oD!Pi+6{Bb8Npn4LXx)YA zx4Ty+6}J+LI_2aaA-c%Xv2V*0xqpSp)?eTrD^WNI@gdfY5|I9ri(C)uYr0I((@Jbd zt-jhCpf~Bc@K%tzKm6I>zvL<{G`@nEQNgi>g*Dq<8bcikJ|?>u&(P7=ZKeAym1or0x}unTNZkS&Oc60Se@#Z9svH&syI- zX{d?-?DZS*L+<4yc?kJ10#0g6W;)c1z8tu{Elo__d)L;BA9&134Ix*eOIL^fP8iHv z+vT2}_puv8frhu?Gd-I%?1d%tBNS(BF(p$S%ZuN3!C7!EdLZpnL2jpg2RwR&hzipt zRR`9L2t}|2ZBJMh7u?D_4|VH=tyn*y=qn{Z9*(&Bcw&{X zsR>e|&PagHw2(r5RfJkcI=m8GK_Y`^UdnlkoBGCJW=v#RHF<{T5WJ)Ks3>OmiG|az zXoDX=5~SksZD$MaI?n@3!H6&e=;_XI`abUE`vmm=0n~*TtI$o=TnuR28OJ-}UI^;4 zeQn`cL6l2;7U|3aXw$f@$!}9+-=F4Ko+f1P@*R8uZ%I9ROM6hHvawDaYksGQUQi6G zTYAG^^8Ioae3^AA0U!TJ=C~3ZXKe3b*cq8jkB^v(G8R( z_g~0Ce!@Fh^o(l9Ga=F+6P|2$dLw?bmB#{w1kl=;e+d*a{ zE8#`@2#=s=$*k4tlC(bYx)WSTSG}BZi7hss5%%z}J;E_8mPO8+i9B2G4o|KEQ-a5` zYP_&GFg};kK!60PXy&W99k?@!S##p1$9khTUz?`>9~jC#6J8-)_(*MQAKU3Y=N}xb zJl4;X5qfNmqzpe0>l)c(`RXAo)j)ggRoKV@rX?W@4RE`knYP;!s!%dX2fosOq@g!% zZ#<^v^UuE4^DScfgS6s}16bq63B-Bx3-P&ud5%cFaXR?^@%9GYm4r8>U9Dm|*jq`q z692H}e{Xh_FT5icSBPS44`WK~KQ3Tj-1DiI_SE%Y`RT&SJ+8nEqi9Zhuhi6dvEyJZRH!mq|Qr(y~^vvR(P= zaekIf*en$xF-8Yatok&jCJO|@TjXt;S704Z8L&JqsZlh)|L8cyhWZZGWvjIN3h zV=sCASDO=T&c2~T3N`jxcauTpmhM(MPBt_HA*v@r6(-DA9Ik$9>TC(&8^(i-YX@ zw9OB&Z!CsxQTKGaN{|f|kl=xqQI<&T*WaQ`1ewVuaVjGSALWQ@yDHuWr`o5%U!RXs zYcz>+?f51XYXEhVOM=94_UqinW%j^nJp_oXM&*&Vs|dJAypI6O;D0Md8TNKZ78Y4D zp$pe3BHVf4I;+!wCeN^h`hMd){lu49D$a? z`tWj-Q|OIoeE)2vpuaLudSn+vlKd5h$E1FHY{Q4nI2T3s&u9H85CtSSmDroN#v=7~ znG=>WrQ?B;J%h4@um`5Z5E8H#&Vtt;z=2b@nJ2|^`XsHc=)t0zAgsMIU1q`d4m%M9 zvXHzAxMc>3{fSXBq=Bm|GSIw`Trx9hdwU*jy9_|ya#15^UOgjSTt83AknNpc^$YG9 zEnUmAMxvMs^sFDc^=u7F**(>tmn!b*%iqXPH$s7K@rW#8awxlGN5YCUH_ipU-P}Z- zxiB@ii1p{ZDO zEif?a^D$=Yosm;wi6y$e2RUQ5IqZIZFn4Wm)Yw?!*25UO4{!^WJjXTf|^Uo6h zYc~g)9bg*A(5KBo`BrZ!eqw|kmt1~Efsnr2l#uLvI?j(v<~)GRiSq!Iu#BhROWUw% zFlte|1D$o(&0Z=PTT`7hRx*1+a(1@*7MlYpP?VsmEPUM(FJFRpp(pc{p@?;d}Fjy?j|GU0-&L}+Ty*YO{zC1=U?D`#kiN+1m z+<@2hyeRE7iNnx1@oD7vnYgqiXyj_+P`rj8di@iorafIKM z>qD8$$~u!%#?Z-1Jtd6py8|iXgPBNClCdqWL^x4ZvpM67fZi1TLofD_ix1e;zvi8? z7_AGqstcUb^SsVN<}fi>!iOy>{E9q6(p}P<(yrHJ_D_bR)YJM}?l_W_o#rh6$+=yj z5xy77*Eo4aA2H)RF}QE7aNk-g^YDEvdhA^PnO$?ZM}rH(7~0Qu!%&+-mgXN&niJsml9#sc=+f>Du$=xPZ^~`20$*#q-AIb@EZ`PEm6e zxXTyPd{c53=JNT;=SX2==hof3Vr9YSey^}C=oI<#6I9O)5*|A>67JW!+12j5mBlK- z!LFvRZ>LF5i@^6io;M37>7)T8oHP0CR=XaXy&HrYt)g0mG>L$EhX9+G$go7+LZ(82 z!pgi@75pjEPY&gGdMSR$zX^Rlrn26TO0XN9UxByyIdc)}!pnWmsfib8`& z-EpMw$8~oE*0FF5bKURVoNfj$b>FUjOSeer+|VUz@E!3uQ+zNI5FnUaXEE`gKn!V~ zR@x6UOcvD{^@c1yJ)doFZ=Y`s-#IerwI$)<;l*Uqxv4;e z2OtL~GaHGfaN8TDa9V4nu$wF4@j88Z8wp)J&nZ+3!`+%Vn$DNlnt8C&?EU2NoZgA; zcJ9^}h8tTTpPo{nkeSiV>PHjV+Nuo!TKce-Fs6=Y%BPyfr@&dDoa8rMyIQ;M$Vof` z#Sd#rrcZ@0{Z`i2+~b1@4b=k_oLxdfccg4*&QoC)zqn$BcF0t_W zlcQBlEAlWWgtFrY5C;S=p5(pr4N~;i>Pk&_%64*r(8HrC}{ig z{IoY){kd7c)rFxp$!ny^e#1e(&COn`)>KZb#mO8`z{81_j?RFcfT(j-@HHK zgZabt(H$EwgDf4A$!F)y(AeoLboQf8Yri@~vp3Ign|n&%A{%(C)W|6XbGnx6Yc|D| z^4qrWZ=bz5=R(f{QUvT(3Oa^;_ZUVj`w2#oMmNUzM)Sv9Ci6Ug$v-vrfVY}|kp-VV zs7~#r_o`}M7j&LOchWHsOOqY9n^ok!k^GQm*_1r0-vasjX;uA`04}owl4?6stRgO# zgIx&@%mbS4+5ZUKcaC7;zCxg~wvf+q9L@{>>260&N)Qki#8_*r#1ZkM^mKi9P-1#& z4HiU!HUoB@&e(<@I4d8!M`5O;k;uj388T$5r+a9&%EOA{z#S23oe2?a-dnkE3xms& z&9bShps<=@_`j z`%{Ym`b!a-seEo-Qj<&)dfYR-ha1s{EZ*PWM@=C8E{7xC9s;{5P zsM8b$Sqpx+&IeBI*$HnJ|1WOvMFcDrlwkrhVyN+2fA=!0l*#%MlyIVJ73yhZ!lH=u8 z&|^zL>oA>UdpoC(SBjsM=j$~|X1T7*+-DM=+3Limo8{Kei9TpyQc6;4((p8udNW}^ z4Jx#(EVQr6rc_5%sMX+N_?&J|w2F}XipXC7b0O)fAV70ssLq0F?9Q2Wi7ggC3s0s1 zh!%6=^{v9qKKONDIvVb7hf7N28<0c%?Fzl_U1#e7mP>!tYc8PnQ%nGkH_GL0L3dUe9Z*V?uvqto-|B@XgZYvgb+u(9qD~ z!DW-^3J@CB*8SCP|Gl5V=*`34=Z^8$bdqo3)!Y%Kp=c*|RHw-c0HKzN@#+{^Yw82C?*5Q7gtc zhHp`~n?}S30q$@*{FwY@*%rCpMi#9biB-ThBo|6$wR~i^%y%q{jrY!YOJ3ZO+sXD8obsMcYx{EG-{{Mx?(RTZ#%+0iBmX6x z#FRp>P`KZKHnGJ*rX@Ts@8uNPsnU@6c2)nRHOZo=par}`Vlep$tLo{3)CE;|(sP(H z{|ce+=C=BJ5>ltzP20Vr_NN&aX6FOTLWlf%zICb}%l!hta`3Bi0@hSn5WzOxZU6Ne z<=2Pq$j2GT$@)Cds)^zhs|S%25M_WLIl1tY<@|zOl#kk{rf1 z0Tcm0wBzH?@1|V^2Ybu@nMxcniubX*bhJrsZz!A3`|2T7(G5QpQ!<;47qMs7PkbFT zsn*+{GYsn_1TyaBIP9b@Oc1G%;h)WAz&XmEklQS|&##68{i?SdPzA7gb2o62I-Z57 zCyKv%@8R~xj%IUML1;gtr!KjCF1T=;%#nl8!eYJ~-*|Gd&p-wWJ{;Cw-tA--78Dd9 zs)ZBqu(m&*4>w1C;Je0d?z|5On1t|@GVikOeB$(1W6e2aExL48dS>zzT6?+NKo-gq zdbu0hd)MKiQ&X@#nx5L8@73NbB`s{rvGyXRq; zD!(zBMI~m;28!~(65KItKcWcEJO)W}BA6g0(mPrKeC_k<9|ptT*-e1||qnI+h2X+}G~8r~f?clRPK2(Jz#44~l|8_ic9AlD@ZnIqJ-RITL!8c>Dd* zvu0RBD};Z!2;{>-<9)gcsdbJ#KBMoGuHR`goInp(M8Dee9btk%xPWHhd8yM!=%%vc zp7v?x0I`R5+(^5CYm?tL?F{%irs3sBvOkL~7a zAO!YlqG)z(=-{DpEz|lC+zG-3lQ_`SWb0)-8TYNnl1(>0tBR*#cH%kscvcBoXLISc zK9$C%eJT^wQcYjhR*FZb)g$w;_^%{k^^UY^!}#;OOB*WjzJhIF1mgR)mS9)|9OY zgieX8MsKQqZQ2~HlwQ0d`Yj|N;;N%q44DvX@aJ{i1zx<5=IhM?K%*(5pf+A*_~X2~ zN$lzF4_V$Mp^3rX%VXwCEYO2CUkgwSZU+4het+4o(k_0~A1O5DVfKWdvW?{j>uA{{ zKQWJdzRP^M@G*@V{>(rS+7<7UYe)b6#(A?e^`N2s=z;KE4yVmx60pPWnD+XKd}Mv@ zCo&x}1;O`B=f2h}_(DPgP}^KM-zR?s+`sm6ICd4?4>uZ&KFQ{TI_tcF?wOv%UGfX{-ac97A9ZDUyUnd>L~kkmNt_AxQ3)rK=~ajouzOuPlt7 zxksu8<7(PVUs?`XNt*&U%k5^{M07&mu~!JY6OsS zv+?}`PzMDGdD;S)t1!eCn?WDVDYF8UY#mDt=1;*UOczIo<)0U1tEV+_u7>~zxoL;%jr3&BK;fkG zRfLlNs&s8WQDBlV;neXSM&p(9d7Pdawtx@RTI|+D-ewz%4)Wgb=0x`0{vn5u4fgr zvn*wz6%($JcB2iDHmtKU&UPf+3B2XiJfBVa?=J5m@Mw9oY3<%Y^0t3RTzGd&CHK+W zm|5$WboEt(3wD$Vm10%;843he5$%K)aOIp{2R}3vtd0N^sOj;Z)Cb5coxUiS}U$_Jm-FiU3 zU;pH^s-v9X!>8;@mLBM*43Q%qPLQ_UU8O0R$z!NuIvcjNlH=?HO+U#qHV>L>U-DJ- z)KK88j-Y=-Kp^0txWDt2`ib#DoTk_M@%#@`E@Lp;F(HkjfeME4CHH z@p}std+YlZ>6Qz}Ax^MBfMRJ9bl<^V?YEr)Wg(mF^uW+|I^j54tzvP@k_)I$waJxL3G$ zd$xf)dFT3aypj(GD)J-vEWOwX)b?t$jbN{~QC^Ap}EKVqKK6x06Zulm<;wv7UO#LOi)smBVO8DGSAY zl)0?S!t+G9`tj2td<|(0*a3!4wXB~dNl@E|Y8lSD%(?KwlV4`JLJKyK#4`KlP2g2mw+0X-0N%OSW2rLO&PyKZn(;-JFM^}T zt0g^PV}ZlswhK!f!(b9Toq8X&?8`vm?}ZsNR>43;yT;}51rSL7GqjL?MOt2A3QB3! zp*(IvVbp%q2#_T}bPEeP1aLqfh#mp%LRji%viHhe#&Hh>oeS=1KJdBFyUxiD&FKO< zz#Dl0*pBV&=T%PXDl01|9H9d`i4;T$#Qv@AOy#So_VU#E!u90ya{7|bQ@fM*S6T32 z8x%YjfReLb=m0;A7s#d7Y-acjGooApWM-@KUM-!^l~Lw~#@Tg$viR}@_=d4|JY*Un zWN~Ebw%ALewE= zE2}o2Dzi67Lrj(0`Y}%3#+M5m+PKA5=r<*(L?h*HZq@?hQBNwmRLHr*IIc^NzC9J% zlDaG6L$;6(W0u=$#zK|wCD-vFdN(QO7u0_Je?)XJ6;xu9Cs4l#lC@yz64WSuI-?;J zFtSSlrW@1hjX{CP0v}r>TTeu@EysYW#)@z=FRhiJS9j|(fy^;IbP}}c;XN~4LlQra z=h44s0jx4WNPMOFs``}|N_=RYCVt2AD)fG#^J!rbrrayMYxw2C=Y`2%sHWg~r1Oqz z?+3Eb<99iV+7fF%w^M`pHh0JQW=Atqd>ZT~fYs0+Gl79Ej6xqzoAx* zecteSjNB7?419_5%U_9^N5gs8j8$;BVG}3>NbcplWo5p)KA7V{r(Pi8+B*TV-fS$R zPU3B{{@z!&BMwL!lY3)*ApPj&@jmm-i8LB?vy+mYQ*;QY?y74ZK1Ik~F=IIrl~u^U zSgAUvuov`39yV4)M!fW5zeuCM*_WPy5d1(|T#Fq$QJW;MUETs~VDjEz@^UB|*UIxu zLS4t_?7g=Z{#np{h@L;u*S>|DB51;)`qir{U$^#pHzU16?3w8PKmYdRM6+Bag!i%? z6KyDyL7vgRd_O(%Aw@0vuvazG#K#c=6|sq{u6Nd)P1iq zjAtGdGxdgH)eDbTQmcmv-_&fmi_-F+VtO*I?XYyS#e##!Jh5em)>Y`XlZBzJm7mLUdL@& z(^~=_2XeMNW4^g7m-F_Tf=heg(G^b%J`c7N5)dF5rs8X0WK^rmRws~kpsUo%bdu#W zxTF23O^z>v(byjpUiO5ZphTaheSvu&TaVDA8Q4Lrf_+&i?>dJ_Ef>NxyFW<>B!@As z;0voPND3^BHO|p;70Zx{JXJXI?x5z|8V4Fv<7qga-{mjK zia;206}>>Dd>X5NOrBR115`mULOGYoF~(S-2eHmIL=M@`bTz=kf1Qr`qOCxbI79D0 z!JIVor$q?rhKRJeUrSSHY9#PrqOmM~tU7_%S?#+KciB%IUlUP3)i+GU9SP*hbmp!$$>&}`eh&#h|4Dq_Lcne$8J zR%n&6`ZWi$!3YiAIQXd%dqJea?KjJ}?JH#A#5nA^e|#C1PNX6vQh06vQ;IB)@5XWE z#^f?ZbTU_;?&u@5sKFlUDpGAY7OJWx7X(3JP)%nz28bphqoGS_N=ZaOIi3!cBQifR zDCHeJ(iI|PF!dPy)i|%Bw?-cMnH04&24_L*=Nl3AtrcBq zwGNC=3s@P&z%+l>A+!l8Dm=6~v{t(jmV*`*{`(?!g62Im!e?rlV!vLslzDMEcUUM3 z8fwr==bPDT!3!^TIb)#V9LgU-T3Y6q_yJp$Kzks{BK^2}N9koc_r@lLK2O7l({Z_I z%VxSJ5*&N2Y3FeMzM*Wi&+*`*YLs*koj!;EkJ2r}G=2+VpV(NaFKkfP260zssGKfk z4m}5v*km#;YP>9JJgzM%?aUH&aZpj0`X^FaJ2kw$4^U{JWlc*3!eWIA#Xe@*Js{7$ z1XgxFj|r5E4fn5*jQ9$AR6-0!B^cG%bF;TZm$^eRK|#RJt66&!2hhSYXG9obeW<5W{( ze4mR|5Zv?7(wYZr|9UD{7y>Ov`!ntDJ(qJwfQ&If?~n<9F4%S=i91S(E-BD9ojl0$ zvYZH9FcMXV681A}4OD@DM*xEdTDIzZT)=`EjTy0M_>1qo4YXN!LyGU+qKR3JlIfgE z_*709(N69KOkfNYR(89bW%OqaXpl77=ou*9YF@Kd&5KUD$RO3tSr8Qk1FQG<^({j9 zWEIEbuq}*(0a?@d9CRRi)0bfB3KnW4A!5=}V28sr`=bcL3eUaXg*#o0=Gw3FacO{i zC@z6H^ZsK3;u2f$WLy;0g$pk{! zj<3qq)|Z31`+}$s9>FPAxO^I9`?wF_R!1b|i((t2tWri)Olq3&3pC)^FqJpFi=4V4 z`F_4tTG`={6KqseRrUL-riQglqq;{-@Ck^}TTd}D7;k_ouIDG}i8wgal*ay~E`7ky z4WPD)AQBS9_p3Mj@wV->2SKCLyIrQ&_2JqYxNd)n87(#0zv;Z~Wzs2x*hoW0)4>2? z7#(7KDnsr)3tOFUuuer+zUjpFS-H3c(|bkU*#2P0-e?dyzUcCMbGvlny+aOM2v2!m zA37G4Dp@cQEve6b9>-m=&<*{n^Gc>9bg3Bm!6?JfqXjLKeYO*f?d+!8UxVDzs_AUF zdOQ@NAiE|4ZUjdEeC{2$O!fR@RbOAjwjO0>?x%*GZ*&oY*jg3i?MeowcwM>ISCs*j zY$885Z-k`_8SDXvxB0mL4n4mD&47GTgi@oVGgUU}50q{@f_e#g-<^oC5WL3JB*CW! zNTQ6QevVR}AycNg`$aH#)cyq1_)Qcw$@EHM4|bw5B$`jUps z&?TjuB$*s>Z-`<<{R}D!bEC>}Ne+_^i4B5BDIVtjd0<&4)oHsTIOBf7TUlM5C!wN( zL7Gc?q!m2M24zxWADZbaI=Qa@>BC0|Xli}xu%zP%ntLTBB^|&a>2Gm9NLzH@K|WcH z^tmOYE?jooK%$D%_!N@vLwsyosF2zB{PeKYc8n}(OirDX0lwBO(`}ueGPJO;=vn&_ z*zKX9UZG>}WA}We|AI+DK@k)Y(SR%)eWMV1d~EZ0^7336YlI;grx<6FO24#XS}M?c zLuhpY{^I%j+W6-e>gVjo!`iV$A}MM1=V)Up?2eUp?K|&A3U{6}AS|h-ToKpRw1wog z*2Ze5?+jCWg&fC?&NlaT_-p6)Q8k<+n5jvv`f zY8uXdEfd995UU#M!6;*rlTbC>cjcO9`&S5wp+2Ioju3xNpz!ta7ZP0Z^`CYp{55dY zb12$o%)zJMFe*ts-TynB{bO#%Oh5zl1lP?R_QM$aYGwa{-IYsXtfEflyZS;djMe&b zu~~y1SsgkT4pbCKR208E^wlf)Y1e`vqx;(lj-z9LubBMVu}0uN(tj)yoowr(CAH9N zVxJ|0CePuo`K>1?>uqOfCM_tA3le1+qNbwi&tJ>sH)?5p0YsW^zLU0%3^SF92?e2} zoPs2ACd%y!g42=yfs&gQpA|8ryY0{97`X{a%aygY6IWwGF9atY4;7`Ymw*=VHQgwf z=0+oPXeNm>vc0{1DIdic@8b|{2hnKgch1R*vBh6%>Or=wavPVyX) zk1ie&*UN^nif@_>Yt|N0)0-5Bk*8gw4F4#)@7cQgtNYG>#E`4*JZ3{rJAY zEKM>xh^N?oRwu)+d@Gr;yvn?&GxR6iQtNM7WWZZ8n?uOd02d)EOs<726|wQe?$5DL zT~bma79NN*{9scf+Mi+&JKp}_IWO>%<$jg+_ag0DgiZ^pQ8|0G^0<`7mro8`|BxSy z1sQLfDVN+XMrvUbaMMYP>MVo`W`er5?3KSJ^zw|}4eCN8?ji!L8**_tAeR27qsT{FIQ8@YT7LP>hT&xJUws#wZqFajT*0vR*BQ753q$E0fkAu|zqw z6YGj!s9dxUzxFqT$mVH(i<=*(>hiIr9}qBb?$MHyT(ZG;a?RSBAN!P{3!NJ!TkW+t zC=IT<^uclZ_Rl$pjE8u6yYTYI(mBUwL5}#)il0d)BGhTeL=3puItv#SJ-wh8zr#I@ z`8kF&YK4iAPzs6Rt>`D*)du6E1A_ zQUqq?KsCYD>8SRYE|e9orr<`!1keC-tDHCuhyE%Z6DlS*UHIjDAitcHUB&P!g=DR* z8O|g#e&#jN`37c&`!p*{=o(@84b=pN4+euQNj{x-OKBH`0}`!fTlzp2^?X?<#54HQ z$xT*F9RL8l`ZpuP%}ur;31xQ#aZTQgoIXtDTXM-Men|0P8`_F zhtNqg9tWOermu&X-vL(e;=`t=2}5b3zBg{G%aKF07Eg0?P&IY2#2@k;LYS#m$McGX zls>|P_9mSb|AT~xaH6aW`uqH!NmFefeO--fDPfqt)ocFLnd!n| zMaxYO3yTaE&U^9_o&S#YUycGgPHIwJXZOZ#ZRd|mxncTpZQyaPa#bRS!7R}(-h}sH zi@q*cf|lQ53S;ao_>BQaeIH!RkgXlPDMLdL8z=YH+)Q;DuT`*(!l$NH(1Fthi3*z` zoH$jNfeA{~wjUS8xbpeBlof5uWyVZu(Kxj=5gy~mmJkG=7BB7*=qnoMI_)9#VQ+Vr z+GeHW$ivNzTZzGenulgEzveS9pENMaR+Fnz>W5?cDW5*s?fQ9;v z>5A}x^aHnDml!zF&ex|KmU5yIn@%hsnQ~w*b@zt| zb-0K)L~WgLwfpIe0vwQZ(#lAPhIxxP!k89!+DCacr3CuLS?sC7Ck43;gV@z6t|K`UA4!h>S7_HTsORjubaL?>6EPpfQ zJ0`94qA4|P<$s_zu^SWysEy}b?EoTz8+D48A;*X;I?)gnM46ot8`?s|5^2JM3gAYcSY+u5>Ip(CX=s>+BY$Rb+X&}6 zjm>=gDJB3qtx6XJ$6&f{fK?8Ls zmr_AW~5DXHJi+l*D_($g%< zsrn_BmL&Dvze2nmoz27hm^Ja$m_1*7dCvEQDt@o@FJ1ygy88Tfg=^dalNxVdkTUXN zp^|}NaxIjhNoOXUPzHO-3|kURO4}o9*tV2C)sD+_wTg57~eQMy!Qf@m$6xNlHbp~i3(Oy)ZHjKarxu*)z#WS zG@1OVA~#5|x3`xEax)YsC(LLQ!;-O@O(cXgP-JhfFw=(i$m~*z4vjba~d~MK4xw<&eG3g%dmkKhb?#j5}6V;D-1n_Fn?&8GxVRbDAB&{7MK12s zqlp5lX!j36U7?}lb>?CfEv@~WtlNQ6SlWoKclhF(ENH>wL8SZZy`ih#9v&WPS!*~h zD_$5a46T zIJ)JOJ8ClWTR|Z1OG2>+3}9|^qLR+`x!{6Pk~)|7q!cV~OX=Z5_C$v+=visGaD!cP z3Mu~$&;NpSJqo*h@plo^bv&q3^8#TJ$0V+1g6^W476+4gIC)h429{!!8705E;fI;C z{`n*dzh6doH7@FuEolj6KVu`1tjvlP9Va%@f;dkUscDz{Bcm~=?{ zpxrp4p|g_+Q2L#n6LAJ7mzUp*qk;06;X@pPl>$LZdwY8oiHV7P&7bT==YGiFZCGNk zSf!n;=Z7|V?bXQaxNh_zF++tq*0Ml_IM#G1J3Dg#rA#_;m}KS0>0?L@FSizAAO{vX zV&xZm`QySla=a~EzDBce7`^Dq*Y?K%if~2jxL4`mA?nQqB#|r+*U?fn^bn$=vDtT) zH-0T!t#RB!k6V zEC@K`%~pfsHK~Uag3kBHLL#)GMTyy8H`PNSS&l~GNzzsEAfH++Q;(RafEk)JsfBkL zMb37=Om6!}V2SV#+BPF%YJzD__UYpZ$r7mDaddy`U&irR_*>n>5?Q_Q=O5N83)vz#Lc`A_8ozKKI1tW*qlko}H92F0K`_ovnwAbV1D`!p3tb0t2^wJyQFd&lS54HsWndgUP|+*Cf0v zc`6;?qKJAYGK>Fu$&jw9LM0x(t+;4x?ZR)iNPPg!EOOBSGD5g>$5LA_e$%wJso#E$ z6UN-kzdqNEzWc#Qbg~E?6v)J$8f3wN%CRM?(x6-D=KTp$&&Wcs)Ug~32Qs**i`zV| z@%3J|Y-pucFSO7OA8-B=keA(c zJgxp8_$X61a~CfO3)JR!T_h$ZCo4-r`Ty~C6;M&NT~~Nf2@!`Dq=p(05RniBRG2|& zh7ynt>5@_e6$G83k&qCOp@x)3Q9?Qd1cy`%l#ozDlra7?esB5x`mM!s&2lMo?{lB$ zoW1wi=UjyYX{h)(CVA;Nqc8ynR znmni+7p)T(wI!6JS{mBp8Jlh^{)lR6x%~$h73+&6S&T4+T~&`M|5;Q{Zq+&m4;z=p zLt@Bz&IIfmrUd^Sw=g{M=)>|sB%wqTy|drsCryp3kovLu38agTB3)p7H`{>v|~=CTJ=TQ*GkJ zh_Wyt#puqA@vS_$ELJRyf4{k9Txe3bh)GhGo6=7C!ivbbxsS#eI5DqooVxR}vex%3 z+R&nN$KDLjj?p7X(Fv(q`}D;E+=PJzimb%~3i?Njt=VfxH6z4}kde!>wh}y(OmWOF z=MSFSB@m9>cydkOxX~V5+}l67u(oXIAj?>~d_HvgVJ!769BpzcF)B6d!|kfTRjv`A5o#x(hb*}qVl(lt9`6^=NNfph7& zH>t=u&4eA^IR<+1p=L(xT1(y|Za#SY@!=yj5m%3cV)>1F=jSaFGls-<``(L?rc7e1 zoWK5TfAq*w%3%|v3|49 z7_^{p$DN-3?9p5KIgjVjN#_LXnMXBWmHSbX|L<}G!kTM=yO_ynRM`nxL~}|^>E&1o z9(9nb8AoK3Q%Cm&c*CaD)Oyo!0zxfaafuys$PRLX+Ton3%1$F=6R*1{yy(%-ppE9L zA_X-XJM-N6>?Yb)IML_fH;Ov_^=LV3^zGJChE>cn$!h!q>)jBdYCPvpZt4*B=%ac{qiSI34 zZdWuK-#vGxmf806gd!VFyA#o&H|XxJ8clbBpK^R;8n7LacXOQH$oXPTh;fB1OvZLe z;=@DB?qUONK#G~|+xU-*Te7LKt-CDrfk^#jAN3vugiK6eF~aXEvPbH^BvN$9#l-PG zj;Qj`77KCvH-QSm6o%;|oq-VHx|*=0wA5=CbnuotzmAo&4_3FggqdW(fLUvVefeR6VJu|pe&q*CcZT}$#kqjl%Kd<`KpgW>s;54dL_dK;P)5jAJSiQ$nU7vk^st$E`m@+CxBp) z6s7;T4@fkFcp}6&0_lP`FkQ{7p|pu}u|k1U_in#8 zqiC!Cx?mexB1v`Bo4jk9KdV?j$u(%g6L&@8wjN)ez3eoNIXUrgFD&a(h8 zFO3HE%ged^6nPWaXZxifU&R6oJ+88LB^b!6L8R~5<33ePR+dhWFq(sGEG#ij3ky9` z%eHN80$a$=ebz#3!UTogkF+G1v8~Kle_xsL_6Wx0=&8>Yai!Ly0mXEkt`lCRWt=q0oyv!mXZbVMZD z%|1-?VguFTxi~UJdR2k2{`mCj`mSO`JC1lrOuR@J?O-C_*kJr@OJ*SgM;oLrK%jKY zi+@3mp&2zJ@e$3p$tkpri4^%ph;ZQ)HfJ61h1ZtZZV%~?NcqDYg(IWM`L9JZd+Qu9 z=k(TG`F~DFE>s)$+rEA1&tb^)Y+;*CwY^M$EX8cZ!y2-J6!JjhV%z(H`wlj7#(6h)VGo(leB$!EQoAcm3~pW*Hg?FKaKRM*^t7*1@0hs* zTd~IY(3rgRK+dU7;lnmqeUZnadWqL-{JFWf5a-y`Z}GXMq7yHBkeAV#kIE!H_TAnPQI@m4Rpl>~PZ$5!6+Rb~RFn;gNewK1_{i1Pg?7-GK~z|9 z@BKn=Jpq!1@sPkhX28C~ElU>DvxM6hGqls?{(0`{gqw*5J>NU}`GS1w zDYDKQGV>Fxo6bEIPmg563grwj64)A##6JErW%oxy)q@;tsKqQ`xw5U!W@>~Sewm+AA+?w-r9pCLETzkH7uA(2+6A?W z1BSVIZzH3qXV=)sMuj`d=pY|EFH1RU|ASMu+WmgO`T#?jMn>h|z;+rY+W#)pcE z1E>Tqs+-uX77n2elQ5W4Ns&K;(T{}i_m{$l*z_G3(%c4fTK%q-*M++dcp+U#FFK?m zaS@=%oZ}~mX=xw|znwXa>^aUKz06J4c1!Qam{4(_K7pJMF=72E0HxvdjH3q6@C!nO z`DeSHJUt#-`Nh7Z+Bc|pY-yqL0G_xrWTfNY-}(3FlL+>#6w0S-GNvLS3_2q(`3YA^ zsT^1xw4hpgSmpO_^yah}Vwx>}*M&I9EW6n(ZirDbKt5dO6{WirgPb>X?<~Q&IPskD z{EkXCIAJOca84sKO0-YMd@QEA=BJ?e0}he@hU%pC^ZAwN)TlI8tl!3C0%6y#{RkOx zg5?Y2u+chJe(}Q{!}Di_n5J)4q7$vq9IsBoj^j4><*KAs#KAVaMig7TZxcb}TpBs% z-HIe|9NRT0CmC^Gv-6B;n-h^a@oZY-O))$%Jbq;4?>qhXhe!KTmhnUnHT3Z7rwO0& zKf>2AUPBkBW-=y~fThK+cA7%i;%j)1fp4}n_v5OfR=f(|{Kp-!<-*q$G-SYPFyblw znNVm){nh^r7K0TniZG~vth=Df;wU8Mr8{B%FT4Vw>93y|T_{3gv3){>Z-ImtXv42B zA!X*_Dl6$3JTDZsS|cxjDq#~5Q(Q#db-xsrEI_BxDSn&*1 zc6N@|BG}YPxv;%ER&S%39RRB8nx2jLQSAJCwg2;hO4Aym6H`I}gr60Kpz~R{iK=#C z0s(7~l2kE+Q3`@*E5lXorxf!aCINpgt6HC*-;2sgTS{xU(k0n6VK9?t z`CSH*RUV#`66MDuur?B+Jp+lES76oGj#E-D?|@}zxJoEMH%CGwRB4)5g+SwI^2IVqsHc)u;!3v*Y`=%T}VUDyG9TXmfjm5ijB#% zk<=&x5H6d76RiWU3h`m}y`77g^k052?t~XrK8o=E`C)_%=-qpH&H_Qn6KJ0j1YX>3 zZCsGwJEDv7QHgERv%E80RzTM?x->4IM8}R=yHO5n@upwzke6^hm|cFLFM@ zDb;ld7_P7GVZy4TdbCb>6ro9>Hm;PvTKJHQOes<{nEk`Has!P4i?vMHuh()sxFd*c z+A5o9L&Gq%A@5ZHXxX`kEn$mI*RYJPL^P`cm{g*AOqrRF+gd$O*VqWxUc&<(6tzyF z3J*+22~M=SI!K-oOLwP4o6J)G-%qGu^TNrQY;!cp3PmW78N0|&`R5k4|khkJ+v|p$8>Z2`O9-SHS1l}9)9R(b_q7isf z?-6Wfdf1oh3-$$V*DueeVSn+}a4}!@y;|qpK5izCoS(F-TaM40**=DFV08JAPib%J zYbcFL#9#{hv|*2XCn;x;s;*SWhQ2Y5U7QY*447P zrcNwnNj0%SMyzL7mTVOKM+^Rc$4t?Ka^}tnrgl=$75P9;9_}W0q)U`8vVC4f$x4Zz zusDHfh0skr5Q!t;BI0qYbP!1oA?cTS$t+f)ey=GNHySqCNki68 z2QZ{>M}!EC?ve-@T>K!7*xA}W&*0X5jU;+1nJlCYHG)Vj{<*)takMoB)r1!MFmnD4 z6F8yNB-IftJkb`MD8gz6E+RGt9W{*C|L=%4iPhEC9-pLtn$g5NE-WnUnDvIYhsN@# zBLf|s>V*sU7NB5hJ^R@$$im(G$7bBZqJx6~lV&=f$iWi_E7Zeoq~gpHu4TBM$7;Y~ z78TJ!ihlg~U+D7BjCbM=sOc~2s)Kk2&}i)mnO3wgVRbpQn|M3%s6ZQ}$QA>n=spdO zH!|ehes<+jOh8c30e*h|vd`=5A;A|eu;=9Duyb;zv*X1peHT&e`FLt60nf*PwTGtlV;?p@6(1vb?=&MHp15%*2JZgFB zn-@7RI##{4%J!Y-nLB0W$zx__X8vJmsU55lyLRt(S` zf+~)^a)f3bzYZ0;Hwgx(%k?RbX{)SoT!+MjS>U*SJfEyazVkc8NUkCm)z#JIR8%0= z*Vl=qrR~tnZb4<}=m@zusBj$>ml7Hqt4z_Zp#Np({I+0Eb+EEXQe52CmKGIeW@hWn z)oGIo+nvzTYyQ#D&27<*UL@Aql5|4^?qfO!civ`6CJu=5yn}Uh|M(9T0kdGYH@mklTFh9?73SHNf(N}xo9j}mj_4z z^hu*OpO>D^Zo$L^Xy5g;`S7CQ{e-K%p+HM=_-2+CKUq18!dvNg# z(q&LA>BXF#0N`hKo8#{-)<1rHwfwsEsP9}{q`hq}BY#4QMy%42BTw@NG_|!eqGN2) zPg^+-l~h%oW6sGoIUBKkCvH`}Fgcl(O%Su$rZd>ip+7!8&X8_A@cjf}({K;I{O_eZ z8a|`Z@(70qcW{9o27H#amWlggi8!wXhVKDv^#OUQ{~wl;|K`;r$PLO;i1qQ*UA|^$ z(%e|?LE+_GjOvn^9a2<@TVw@WHnM)r@ky03gfC^rK<1DK@NnK&nw+=7vRX9y(y)~t zndWxb=PI{3f8ww>&(pOP87n2?Hh=c}8&e6`FX0Nx%F1Vtxm>s~HXLKCB&vqSpwajG ztAi3PgABKA$k}ajOH1D7k|UXX@@w{uzHmTbSs7Ve1NJI4-QRcDLsmWgB!cBS&I``a z;HcKWY3tvg*91=>)+;co+L1(?s3@ICB9~A$FrfqbOa#F5I}RdPb_md~9Drje`0%Zr zot=}1hku6i=b-g8?qNOyt>~DT`uD|rESzjNq$BTXXlMi)J{W$;2eh6A87JFEO3bUM z?s}B@==CR_vpVsHqa&y^|GzK_oei%8>q3q~d9q*M^1GyfA6nTiDd}(oaoEP)G>b-}WTO zOeAFK1sTHuuPXgllK5Z-X4ilQ5SpD%3wR+&s-I9 zzoYunH~kMvLHW&=*^ifHKG*CF9JA!vyd1nuvRj5-6zUl8JsD(g_8LMaGnS|PNx}Lh z3@H<|)0Gk1TWPW1g<4B~e+sW6UB1^bW3{1PBB>}PPRLxwqavZ?5v8W@M*6(Y%GP&T zkigof|1Mft4VE+(@yYf%OAX`GCHk|I1@Z9+J>&8tEj?%KBG&U-T3Q6sS>I5;T+0+B zybq~pKGZ3kFDa#rFZ+fKx-dZ-Z02;y@pJiCPKqh9ut{OaR{gWwHN2LuUOT~jnZHQK zKOaH_bpQ@CMP669FoGm3a&(kq(3n%Ep3Bf$N<@ zgYrxxmtQ^hz3f0+5OxV|_$>;~RHjVAueGMfSyw^L%vR1qU{wL_h!ZMT3>{*6=WoIx zE5M`ku82|^5B?Ixc|2J(C~|q z>-jV1gmZ%Y?%ce2GcGk154le`7nNtSJ7P^zLBo9o*bePvTC|xoDQ;3;nrp;E*jtED zA9%w;oJe)d=2!l#(xT@|BpO3dQv{TzEa49)pn}lz^b;s(LuWxOCubx*zo-; zcW@fu8hK#Ih`o$(zPg*N7uXMq#BFHCQO?iwrI!T+1lW0b?Sm74T1(4U>G*LTu#n$z zst?+flCY=`yMBEamT88WfqZs3nP3=N?a=|D-sS_r^+P? zTNB|f&P$2EeARVy;*xmDq|Ve{)63KE8*ReUbG56PYH!stPsv0OrQyk7e-ox2G0x5L zJaSA<`>^KopB@&&#JrhjpyH@6VD{5=X@S%gb{ABLMClG!I3S9ZQ&E=*Uf>;V8O%9l zarf@s0IW`HYpX;(ULUXG3?oqhb+m(lex+MD>lNTwCwF(hXIowsYRvov=g*%ncO2ke zoP01!Bc`T?wJj7U$MEd4RiM}HNx``+R3zNk z?8<(fGD3x)s3ROk5A99;;lenoc)bgE31j#Wl7H@)AAQYzcnE!cr|t$OfOd3hBts^X zgOk&HVMP4)@hjZ3{TV)kdL^cBL0Kwk@3FMF2G)E*s3r!ylLJ)gxi4;O6A??sH!44#W zz*f%Y=_ZPpj`kbD_CwK=^yPy1#2~r9BIJ+VUSBr6C0Te}F<8mNTX;si-yQbDm=y>! zvmNCghAJwQ#&<2F0oL0M7a8fOK$Cz4&unfK$Nv4^!M(}i7p9ZWl-qU2V0ycIJGikP;j6EP9O72L&qw-(c9TF2& zsQ@J9d*)zKRTUH4A{nGg;{{NMdc@r7EN>@Wg#yD>YOv0Psb5nBDGO>?{Z2&a!sQg!x!nxY z_ADHw4J|z+^h>yzU6twg?0Ho+pxnyyed83{(#O@_Lv}z?B*b~zjnvdS#7Z+WxxuCh``Yn0&;*snRd9%vflMz-jhcpU*q*+WkZ~{AnQ=Z{lNrl| zgC;Vw^Lza74z$v7(mtm#pZiqw>r;*9JxZyJSf6e`CmoQ6t9&f+8bU$oV{kZ#CF-F~ z&)t#+{b=XEDj%m~##dXAS zr_!;&Q%|gs^Eh>)-L12SrBx6t_&vP^4_z&|Gp(}z_j5@ZpY}O;=oMJx{In597AFx& z&PY+jeP<|h@{$AWpSt!b<1aa}$}M*FL{ zMD$F)ZjCnizd;p$TjBXW7_F(H0n*Kv#iMF^^z`)E+1V6L-@ksaD8OxfzVbcb%iO8s z|K3os;dkpk?a67h+d*y$eVZg36{XD!!79JIfNb!p>qf!8JN&y`#{^1FC>UTK97Q-F z^dXM4swL>1)mA$!V|6+3A>baU@={*S<;x^X?eT(e4Fli5=nAKpbmE~wlbsnmI2iE3 z9)3xm=j9^@3FNbzn4An+;5LRihro^yMAS%M#sR3Ui28xh&&XGpTQ- zg9Jx{X`v8d9JK9FB z-&{m9=N>`Mzo%b%^xn15Qy5W0I%Cqw25r7XYod6IJBk3aP`+=savsh%oW%#Ei&|M=8J;C%69|NJB0-zew=R614Df#c zT>jz4RAkxg>?}VY-y@9f=B}9NXw$xQM0IubtGide9(-}-d$8MNy;0$2&{fvgIS}Dl zqlG=A{fGQ}(IlD#y_c4rof~b;`iz8lyUoNb$WJv2yHm*v@B5pDm!`1nwKi+yAK?^7 z7NsMwvd^xwq>dS8hDfm?Sm2cJu)z~|@H*pWf8^rsVB|b7=L<8bj^+bhT{{E~(dag@ zaB?HajlEng>kFe5?$2aLt12rS6Iup7)73y`NIY{p0J=mNu~h$CEq#hC@@(s?xs2H* zn;IqBt8?O{T4L4wX=KE5Ga(@Z_#{KtWMgeD>+C?*DFXv0GRRPXx1Si=+rM4B*WIli6&sra z&Tw+(3#Vhd{d}wL%C_9Yt@+}rHy_)lpjyXLUo&5xP_q>i;*9WX_+!??*V<)%@6b)V zI7j3>-e~;7kbNRN!j#`6=~pyxZXu8KiaV9#h-ELF5j7t2P%A=D3lQ{x6M}{D?^;Gz z=#|xO;SK1<^gN(=VPBM2Pk+CfmewBNRpnw)A+=9S9~5(zmX^*fEU1D6%PxbVX(twz zFF#2B7Gq(Um6p~rH%H`rIWlq`rjtxreVewrJ^s0`ptBt+uaFOhJ&^PM5Aeif<=cfz zZ5P1C*4xIBN5T(1PI$sX{2hpRT)=#VZPkTBZ_ z)PrtgcMrhCK5K);FMbdnzvEu7cD%Sa%N6u#+|k&{YybP`(0Vos2dbS+oSmCn4R-Ip zr@a{u`0yh7%2vUZ7P>3K9C#!s@-Mdf`PZS`oy@RBm^> z&P!>)tvvqCwjwuCkn5T=8k`)9GZQ+IxU|ZJFxTFpia~qQ`AD2-tt*FVQNR-0zw7^| zA2n}Dyq+7(k%;DINDUPsl~z_V!c>6(=kF>?bu*Vf!m(HBOhFda*6_%sL2W0j-nY1y zW#h3{Bqlqf+Fg9W7bG5R03tl5{@19JIvJc8qflvv-JUaBhMTaKa=DK|C)uk*(csKK zMFVcr6#(nipb{4p%*JzhYiLAhPIsRAJD*P+*`uav0p86Q0;K6@LIkaE=L*{85jA;8 zre|7e>ge#8-@sn#%pYVm2IQ6o47%o74ZRw=R7AV;dZlzpR}Cr16yIeGj)<;*@jtIY zzb@o5FFaKS0^@TZaV1q96${t-)}0A@XXfh>9yMdA$+Adb0=8EhourR89jEx2?Z}NwGYfnW08@5O7~olqDaL(_xG=$x^>EtS*b!fQgO*}Or`DJe_chC0+n8?ca{n3 zKG}K)S0t+4s=$Pe!hw|p9RoK#^ZT3DwooNoq@bD5yzIjx87L>&bk%L?DLOy9r^q$z zz=(Y`IQ+AF_^%9;E%fBb(d#)frAPg1CSk?LYockGu*D(-jeLW(BrH0wBRp_a(?c1A zw7Oh{JNrO8r=@7)nP>iz*O{gVS~>J*tDQeD#mLBLUYh1$W5a%c-Rmy>m>r5^_FOQr zt*uQ@t3aOn2pO?bcrJ2C(+o!*q*p4l@|?pY;)yDny=Pq7l}z9IgzhutDUtH;j--%#5`6@83rqJg91EX?c3)sX*+ls}LS9$ocM7pzs?< zlZ+Ddt>=se~@(>ySlo1UK7FkdTY4&%;W;l7rtygOKePE zx-6Na@ulh}C|Z)q{F})qhjE(glOy8dS$T60oRx$`DKY*n9k+WFLvnB$_%qS}OO1df zgg!?+apbZdIvR^+8Nh*pW@ck*$qZ$y@_))!niu*~Z+?KG7Tyg+n496hqyo4X(8Ip{ z`|(c;bl*Vhc>4-d=e7LuJ-z4ZCV^>LUUojLni;jg36z7SrTkNr4e0`znDq75}?t8r!^K#VXK5EKuM zb6%}3HTDWTm;sXwkuT8@qCTs7PzQSw= zy8LptaVg>YA2TIj8mZNCP}-{%vM9~9R{x3OHrVGPg>s-l#rr`XgLg$dN(Nz+nH463 z@n<}f5@8RU`1kvBjw>w&z-U7&o_OxyF-cije3DRhQOOyyRc3+XqVvkh53L6|HP9Cw z^89YELC}!lF^hC_v$Eoe;Z~Yga8*5Qi+5gQr*FT!x$5wFB2V*{%6WSjRWb%6ZlQ-u zMJh|^;=>;ye!CV=^r@@&dSp-POTJ7R&p^`%5qiX;8VJ1mi?vmZu;_O1dHRxmlYGh{ z`RJHPlS*)vm4ATr>*6;U1wBO^o8JM{CZZox0=%{@Al-{mrxVr^xwQb>dbn zg#ZlXz+9Fxo+-FfDY>(lPCpJT<_dGaCfdAC5C(_bswWz zDn&6}pV16P=d9A&f9Ti0bQfwRiU^dNYDVKAKD{*9$;}Z>qUsSLEMA|Wg&HX_|@0`F)Joo^l0eg%3E=HK3W$4DPw13 zgs^8+lt!g*@PB*F|HSyq!AxG{u7-Xf-K7g^-~`z(gJ^f%Z)|D3Wp*8u!39_lwO{FxyD54EX!kE|jRY zZy7AQM1nvC8eSAKVyE)Jh(Dpwhgv-2+G+KulX-Yz(G+t@mwHFUa{)MBe7_0*biBZR zCd%xBTr);xI7%KjOG5dgs#hZco2XYT_CV9-y8hZ`E9e!mah9KvWMff@!W9aAl|NZ; zbaXVRj^ujnc^@A~5%Y#XmI9p&6>V*LAz@*Qf;;z`u;>Z{TlcZ@M}8WmU0tWqX!P^r z<}VLX{!mbJ7y5#aUM}s+cqkJ z>{q_6WvvXKvEq}LAIA&kzodi8`lMW(t*f(Myz5DAqoi{B6e?K_?qo?}!q ze2_(Kb!WwqF7^i;$T-Upi!FvoG6l85zs+c37u?;&L9d(De_!#9=@M`=|D}m=kmF5h zw~v2gZyK*-fLTuH%a@gDX=%;Fo_-biT>s#~$@?-=Qd7vHp59&y-(fHwRGgpg8Q=Vy zl@&QJw#xy+3U2sgNhn+4l3H9mN2H&_I}=B#B7q|m?Ls+IIY}K-|1)0qo0&EG)B$NO z96%c$-?enV@}WRtI<5#1JqyNPTlK$bO_SGa%9sQ4l*($SPVox~?Si45El?nt%wcHl zWX3ja(l;Lcva?9e_?oSVlG0@KtbgOk;NWXQ(Z*b4vaKNr)?iY$(L2yw@y9bTKErWo zd41aWwiIG`@K{vY3xs`MyUo|tJ#LbScF9jF$Gl;7(C>j!$mIy)Ss{Xl-<$fR1NSCv zp()`&-lh;?HJ_hQ6rqnL%Zxt}9p8~4@}yd?Pm88(kJ{?>SXfxB1q^I#UW@I_!fnAE zn&lzQL-+f;h+$z=Z5~k)S z$I;@Td~+g2y2}n0O0n`)$=8opzJI=Q)MsW-t^2bX1qWt9gLCI*#KNI8Kt-My<~`L$ z^X0AZ@khtMv>v|$x)->raFLPeT%#i|u5Z-`K8L|o(EYu$eHaF{F2k+bettjsFNzON zpjBHs9dlPzaE1n!&Zb-_k{D?2sND0mpfj6SoVx-*;5J0c}96()_zrJR;(hNfT@^hbo`<7;A5zru>o16Pse>pm*JaDOI z)I5xdpX~9C#?8Nb=Pg64YEU83U&^n#B#_Vzo`TsX&VW;=Y_Aocswp)P#V)okAnv@&*%wNfpbiB^-t-;u4OMU$*7=!VCa{?x^yh9ei9>1Y< z>C>xrHY;DO!O9A2gSW0rKd!IeCk4T*}6V3>c?kY-Ihu^jq$_&t0sl!W|cMtaz78lboYio^na&xhS zmoIyTg@v&ZzpCAdf>Y-%)&q3Y6#IeAWh57<-wN2-Fx zh`e1A!=Fvih77P=^ihO3WKTU4Ht!t@dJ}qoy2>r6^DTdR=J|jWqdBQ9TnA zTuDjEMFS^SSD*FlGK&YhMv`nDT^h-ig0eEwLt)acm&eFdT>R1aS&NOwXwqP>I6HT$ zS3?)=rS~Qo(eBfxU8f_|0tv7aaKG6J;m}1&2(xu*l>Os1kJUjJ{@S|yXq-XV@(nu&86UpU+fhdgm&wu;54FR)m&7_ z%52O)@=gm(Z za#eD+?tQk;TsV&edApSMrB3Js1qGU+qf^5;IJ3}&cyPVW*mf3)<-#4r>r+#3k^!p!;FkF7la^P!jXiTUCwtk{-&PO@tCHI zYeJ|T_D63Q01-CBRxWqwWo6?_N=ytR5>diAx8u@iVhC5$(r&lFJ3u0X<`))P$%zU` z{ezD|%)0PGr=9#I8z#oaGl(M11xYw}b(saO$Cd{RcLfD!?}WhzWrq%(e$-!c@j|5( zoq7edd|}81`QJ&SX&{ZVXm)};C$kPi2#!#M16?B2i~Vu1+kN>Wmco12?uYK9&j6057LaVaUdu`v_j+#n~^o$EJl48D9B3nl_NT{}B|6JZx$ zU*-YDwLSjtpW|V|&9yb)*Bi3Kf7Kcl$ny*`OFBo{(w8JsL`Kgr=i82MDJmMJMz@Wf z9_4Rk0d7728*Xg^*+&LER6j{rn9cv9e~YVyBP?qjNhaVdO9skwCi;9#7${EbkaeH-gZA_PViCt9}XJIgZ8*7D)LdDm@s1- zZVSYMzlV&rN?9t0yec(&q{?a5Cj$l8K z(QFins~>qDsp)4dTm;>3AE?hCnTq$!K)S5aMiAeIFLnKKRug^EI^GYrMjm6~X&>qH zhsG{{+Pskw<55=^7xPIn1PCqMM&+V1Oqe3R;G6r?XmN8x3fI$P-QXx{0QE^;G zM)d952LK&B@Kz3)er6dPc@1nLEu-CEiz}`i6D|vY$&;RM7S{SwrkWZ=tR2)vEyql* zW*Euiq9aa#sIXj~s^UiXvgX_`6_b4-cWCs`7FA9};Kuiy2KCOAfRUg_y#N<+1d;R} zZHUQ)Vk8Llf7{B(igd?WR#qzBNe4sg#+BEKW#)~cTU&u_Tr$?q8>r%xo`F$zE~pgw z28V}-m*ta0ne<$>&=nHQR(6gN$7|Xlmb$!lJ$2ppUMKW6-GyKBl4|U8kOz=i%c6W- z4~E?5!!I7F2mkDe%(Y+H;hICq7hUIs%EHxaA%>Mg-4gURg>u-TG@-}@*NO#=$1xZ2 z;)k{*GaG*1Dy0FU{@!Vn+Ud=q z6%odsQ*PeM8ffvLNjNKpLBd+-22bQX5nA0;2Gxv`^XvxlRTj+>B8fOj^r^7)heq&7 z3V)jH^0zD%63)nOi%!I-PCBK=;Oc!JaB@wyvLaY+m@h17JgqQ-qTJ(u73Ic4xA)!} z?f3v=Ot~I7DRZ@Z%lYxRo`jJ|9D;^gT904mz3_RrwTRrRbg+4n=gU*r{uPC&DPMon zB$8iauY|IKK6aGVS!mh%FoRTlAj=m2+6E`RpGDT-MxUHo>@SPb$mL*2PPWpd24qpg zCF*fj`e`&c+;VVcsdqs#8#(olNNZ`nzWczM@005oO>Hgf$3v?tZ25P=8+`TM=7!UZ zGZt3>qWGD$h?=_)Nz5JX&$zf0Etr~`TF?4?mOVdL*ClDyR|4J`Ht`PX!)iAxusw$r zq6V#RUz4dqEpR7iJ`vtZc^^0ExOn^42X8O0C-@Vzd-@SJGS=-Z10ifo@A9p%k&%24WiWxrGLqp3A10SXXI&B zsBDkwj}JJoY@U80NYR6RXhRNu!cciBQWGh<1LiSNoEX|SK<@i`esdtWnHjm5$k^C1 zCp>b+OiSht*&(++Z!~(yqMqs#D5!}^N!bq{9)O(XCsT$B4(bzr126RSn(kpSf8bAF z=P`LcW2ozlT(58yXxn5NsKosKP+#Bu+a+t-N7v@;6r`oYAoJhrzj!~ZwzhWM5Fd0U zH8r)Xx7TOQ!=#D(T{307RK(4BtQ_~Occ=8C=o$CYG2S{(*WMaS}D3>s3zR1 z`=@bP^TOc%N2(z${WuOZA{dhbYvUf-`5)ag6HB#RQa0H zGMiHDgZ%tmP#kH4cktcbxPM`V5NTahQsPVL0;PRxU3GPD&+tkoXjIbD>~hzI5uUX< zC>O9o4s0U{U58~QSbKYuk7c*ZgA56nc#eHLW-=h^_AS@u)2K$_er=lJ;o)gQ=|g9^-!Pg7I2_OtfDn|UZ`ap(q*SOtFB)+wH{i0|9svRzUp9AMry{6(a}*HW3GGL^VEY4 zYpw%~JdLVP?mX!)gLV>X*4zT#rYE%$3JMd?zfH`3tBLKDhkTg^<|$dmokP1zH@!TL zvUM>Fq;j)hof>w_hf&^5m-EyxGD?b9u}RZV=~dLsH(mPX6+9Np{B)%BY@g4RO!@cM zA5DCGs=$Jq7REC223|s@27QA(6WgY-A4Zd}#u>#KU(k7;m6R&7Am#y zO(_%FvsGf7c)q9^sYFNX_V}@2J3VE4)*pxxcr!jeapNa2I#_u`Z3b7Mm){ITe7?DZ zChq_LG=HH|i^I9_#F?>R^ZbYRaIE7%F<&r5RpR5DVM-EM_(C-`CXFzZs9Z z7d*k)PccLzHQSSP@ccYUB;kZg=x!QGYC+sRsP%nUL1(*jzco@<$1}6A06iTVLc;5l z6WJ^;oCJTe*0vJ0(DT@$yLaWBMX1B!g*}=tE^oW@Z{omB2jBHs zaTH2Y9aa6}T1m^roM4_d_A*C!v*~IFr)L*1sRB%5MBs=$ zI`S8bu^!J|GZ46a*w?G`GgKE33y<7qdxaG^LHUcHlo_r2YrQzs={b|qZC+DeRGe(!9T{cWa{}GC+Hy|p#Xmx>2{2gFg=g_$ zp#-UwZNW9s`}%W1jrP~vBc=J5rPNWbyO_?Q`R<2R_S26>texO&IK_lL`TIU-|Fnlq9=*d9!5+1fZ6n2$mrXsV$6{ zvXTwr+j01e3pkS1_JqpZ%aJ6y|n|)EMQogPtb6^l#4#Q217*DQRqbD zvf7zv$q04CI@Op6A#(>KkL|G^3wbj~n@eArI6v>gZxY8YbtJ~OA2HE%^6{OW2ilma zSj_rPp61K8s~K(O(arUd{N9}=Km7Emod82719I2Slav(KX2VaMP&_Z$0OEz95i z@j(2J?bkPW9tZJvN#~Y_hF;4hJZ!0MZ{2+TJkBG~1bwpk)!3Eq+b{FQ;&<89!0Vz7 ztgp?sojZK>?KV_c`wocart3Ln-)Li*K!oluDlgYl{`%1%A)@l0by?!NiI>{Rln`M? zgVPl!Enjo+-s2^d>!3>do{pg)2eW_*{4r!tj82y}3-&m(T(>w2wydnU&1h8an4qEM zixl2VRd%eORDQd7saFLF^9s_Ci^SW96+&}-6c8-jZv`)Qp9Ug#PSFwFcjHH(c=$2~ z@^$7LBc5RrjK6j4iwIlWJTtD6$ym-k1YG*!yT;4)ng}8~33n!Jt2$BKw zgUrmi%(PN;H+QcJ96NRv?Ddb=E~$Db30@4HZk1pE^odqq5B%1$J0I2Bb?$vi`2p%2 z`O{owoTHqaoW~9lns*?J8yYN>x90daH{QfXM?JgallnxKp^|!O&!gSx?R&q(Wo|Wn z-##uM@c#CFWAj6_qcJ0wUI81ieaoKtc_>fw{Htb)@un+HC(tK-m2NX%+tK=Fka6EY zq}>ypeT=k)V$>4l;bLEEIQu!D<~*duXO8cG6Jn5AhEgiyOLlPlo4xmA-x)BF^ZvZgve1?akeI z{#<`vlPCRqhp1wM&6eJJWkMA%Kf?lp30G&b@n_1V(bsjNqlV*n>D3Am*RpwXEL)tr zje&Ny!SiL~_#3_}ymKY=jI?6B$&DLRbo{Qob$a`-DGOmEmrk38$XIS$ec!sJuJiR% zT~m_W%OC@N1B1T%RJ(SW>AHrokL++=A9u`3WoEHtt7-tU9>sjEIaKnC3@0;5H?V)? z;@P@b<=ti7TXj3m@m&6xZQkgcN**Nb^F_H3S#;%s`IZrSQXk=$pKBU%P{G$YEwb>y z*O@1e)>kUikuG~fpKJdzs+Le&&&UWE1{XvS(>z3q-X|u<@X=stSXhZ!x`nujT3P){ z6k)~!89}^=SfP*=fcCfBhZNpY_PAK>-V8HjzNzmtdLwI)cJEEva9@Ny)i-P9`jbL@ z>tUfzDqA#1?(kaZ8Al$^y&;AVpDk}2$r-3KX}Xn|eERjciioIan)V`Zd)TJV(Nom} z+n^~%dj%SzF9rtAnHN*=SH-Dp!F!{$R;erJKuMCu3*_G9o#dYMcz%x${Oyv?2a7ZK z!UjR2iHcKEp?fD(JwtwN;9E%NtKYUiq=!ClIUK8eXoPl^c4~fY;JBRY4FP6|7*xo- zo!^?ywfd+izkYeuXyfZ3eQXsYHqJ5bJiG!T8yCn-I?$wwUN6MWb8^Tb=R1eaiIgsq zt|XLAWhdo$j_qY`>n$rOkUQiaWAMgm%Fq~KLuIoe)H>p=HAA>Ak*#d5if><99}Wr z|0C?p!=Yf`_VE@H6~k`lOL-CWJqGP}Xha|g_E+=ux`kcokf{5*#X+S*jN z*3y64zoLvXx~FNk)Ynw|){0uC06<+?D}fisA6@F92z1kfqx9K%+I89aiHP?bJE`i4 z9f4~>`r*Vxwn+KtsmORH;q)FQqF?wTXB{p&l;&@alEwFi(mqflBlC|S^NnxK-8j{m6hWa+PEYd@(755-m00?7Nr{m3%P{s{V&lpfL%h+38xwaR}R`p;|7BUV3l*2BGJR@K7Lr$65&KkMo`MGRE-H0% zL1KeH*X~Nba|X_cx5qOz77hAPj=}+&lmZTSmJAk-Cj*|;=eZ9PZg6DW-iokLc6M>G zSrAUG3utt^E<-;fZ+YWJvR~~>H&o6=Has*|b%u!PPoIQyag7V+`hC6BmNiXP2a?bO}OnPbDRYi>Ge0c&(@X|f=; zP6GRF`b|su(O_<}&B~D9eZ`LqG$&INF3{wwSYji+pM87Ngly9?UdzL|gCut-sG&%= zTevps#Rcif%@^`te#{%ghm&k7} zQJ%hdl}6#(wQGK;C^BQU*CPt-=>P*M3;cehEo@W<8A@_ib%jf=i*iaQVA zMrlx86Oo;CjL=t(OO^J}-veHEcSpl@NNYObK0fEfInMBAeE6Ys0nslIzf`7@IJWKX zflGhHiN~TBTZ1AC=MdrP_dQg8SK=yvWI)`<*(%x9r(J^=ZE#1~eE2@ewi~#AK}}}w z2!DjqGFu8ldHc<1nck;tcXyZbn4Ye#ISFl#X2L{Faxx!`B_)3Sg2j-m&YetF}^vS1L_l&GNDAuH0TdWMcd?>=<%wZZ1b-Mh9vMj8sS@$v1@mrmC#*VDF~ zYRP`_LXItzCG5#J-_54+(b1bgVLNwg>DJ`j1kc{yUJgqr@LD*O^}K-ep2LzyY$=5u zD7qPGXpl=wOARcpAm7}A+&v8?t&*oaFKh;Gg>)Wv($3mC0%Ehc&t9##R?1rX-a41t z&)x4FJUp#g<)_KdW>o^k=jTbDLsfxs{vdCN3}X~)gPo-8+}xi}%nFTDDs%40uKnD& zafy-1{{o%q8q887V3sO(HXm({Y(o^9E&1bY=NKXf#)#KtY&jx$n~6S?iG~n3;8`=E z*gjMG)r_e_`ZtX!bgN&FlFOk*6wo$EsA`R>f`R8su1#)<#@Z%)=?acZVK0ZeBoP8F z+p8aounMa!yrAB%c!-85B6NbS7i@0Z+uK&Jdo$w?ZCyBb?&^mb`BIiZk}dx#t+BDO z61O!gHxSQgBvBN8)`md>&`gS3(xg2|g7Q9pwq99ZPw}#qq7pq!0PSRAA2s_)iqV47cI?j6USY9&2B)SpNHjk4*)_ zy$H{3py0AH`|LbZk%L_zx-ck3QY$Gd%U`+n{KJU+hZITIuF^GsYK80Mq=+)pS%vJJ z910Q!BXWG_nI8QFjty8zn&@{WI5c#pSkz;{Ckxy?rT~8v(K$LxVOS2U-c;1ogag0% zX_*7Ue6pXM9;0E>bjqALs;8Ri%={3m-myBG&&xdWMnl_S8L7N@QQMeQ#^3ie{9Ffm zk+#*n?VY?D=uwAwcVd~StVo{~(3YH*emk8`t1) zhuK1*e+&(!ICd;{CCG8>bJL?K5F-lBag99myNt?+WftWt3z(vrk+~L5lQwRf9G@C3 zkZ32&ah?bBrumIVfn-+5*NeCUuOHtM2fUQk<*&#;Fl6ka(kZN}8d5%wJKvJ;Pe#=S zI8T_Ag3-bRU98KO>mxd7flwqcO=cr5e7@Q}%!|LJe^=SXJe`E}o{IAQ+$X(awvq!% zt%tk-;`m*T`cE3xPxTnhVI#Xmz*SQKK;@uBNkRxUdW4SYs_)XEFwxAdESPSJ955MRi-%#HCCzMJ)X^ZG z4$npbD~vp6M!zW#O;t2AcGE%%LhiV_Bq!=Xi=Q*iVx`7v7%1usou6J{Ntbz%-QByg ziZRcpavu5P@|RD>9E!zMxQO-@wrS?s%twEe6D03PA#IJ7vYu$=!0obO`faE7^gYQN zCiVOPKinzs08=#_F2${`yv4@+03kw9q!jdL?{v|fAvmh1tUxJdJiD!0Z_yIly(8qI0FO2dD54qpoxnv zcXD+VbIL0!(u96w3hqTuPtWxzOagh}EL3a7Dz?}3V?)||gBB;Rz?9eW`t`;Z$yX0p zyQvwKSYHM@D?0w*@C_o<)yYLtxLi`iMpAxza&t+Sd|Kd|ylKhV^XT$?9Pi~O2g=k1a60+}7@Sies21w!=y<7vC~tG0RIJa2-~Fv4J*I5&ppn0mfg;vXBYtZ}+d85G zP2$0bDC3a!b9jIJLE@$8{|hI>S0)9RWYG&~?+=0rYS3bl8k;$|vSo^9DYcA^oNUd- zCB@79gD96w;D_g;(y{0XS!y8J=S)L}{T_M|wMst6)+^yKbNmZSp|-EDZ%!w((W2p6 z)>EoxsT3o?mxZFp{qn%wh%O34t-*bmlP*g6o~gY`(i2p6c6OquH;jixe}sh!$qMZK zgzEUDE{>2?zCUlEdiX6{w#aDDV+H9ZKQDG2I}@F!_L{w07N_T{&b|FT&UW=3`DJAO zbDutT{Mk{v=u5 zPUPpsiSr3&foOUXx*j9zBPSVYg@@;tCuTiQ$?8I2;{{vc;9T3nnqzfNHJUfP%P5u6 zxJT%~O;!EtPwjh7dQd9%HOj47|F)G?7;9El(V9w_=rqj+)`w`(W;rFG_@)$h?+^+-PsBKs-me97c)f%#Y3Vrm#~ehA!z8- zK&1rM49SKX34JQEY>&UD+`M_S z2dcXFpUz)oDm=hi^|pj6aQxji)#8t|Z%Qz@Y}}mDwp44#^Weq5O)JLqVj?r@X$m3d zp65_6anYv}``w}Jr@5^8t3Uuk;=?O)$oywoFrD6V2jQ%$BG!~&v5%Jj6#OAKsbEj+ z?(eGYwI6wK|30KIX%iUZM6yQ+5}+DqQ+(^!m2`vTWp6Gy^T_h_w50|4t-^GlE#7V` zs=cpl^nqwv3#tHyqWwbTIus4E)fv)hE{hSDxTUBrUVAB|55`3R+`vbG0cB(FfF| ztKo=RKdT-*n-M?p zh%W%jipJwm$JQ$iEZhB@5~9a^FH4d<#33=FHHlSXMd&^Dig^w|`k|!XMHO+ac8le>2TOSEj(ORuux+d7ySFyI2(fu5lu zn-6QO2oOBg)=Ci}Xjn3DSx$O}oz99Y!|RO2@x2##JrgUdoF}GrAKqXMma^eV+AqH0 z6%~`Vq-ee}xe7g0K-_yiKRNjzt3V@Lh)zO0pWpCRN_M)gyHtgQZCdHR9|!cbCTm zfS4uQaM;J_Jp3<@mt(0h-?4Iu<(|Bmym~}aceN*~gjdQw^x7|@Y9!DDZ!i#gEPGP12A{LfX4I0`Uc9$DbU83 zxGd-|aNMi$BTjSepRwSVF08khluq|L9M!W4ssFZgEy#d8mTp?mjujt;VwV=Z{rB@% z*JL0Egs=#iZ_+ioX9ys$HIi3BF@_U=?q|~QC&<#$X;jN)U~k6lQB+f9Tl$HQpFakV zNBRcHWfYFVaI7OGSFnB)MeLysYwi}-?tU9CA;R)#x+pBQA+gf}=0la!bnw*7OdHT= zIXu37d3L@()?~$XK9x51xB>*XoTkmD?KPWgY?T!T$H|#1*!ZPU1E?s)4f=ZQg)6u@T zL|n{z`r_@6vnM9B_#P17YMd=r5o=>oq5lX=z~$S_#dnP%R6@xkEa)+IP-a#KTvZAC zdtC`OkEi0t7!Z1g5!q&yfw+^cc=bxR6a@T#^o z8W2JTMOjRE@ljZGwyk9S(bQwH*^65`SXD>1-(7DO-)Z#6{WKU}{Pu(9X(zXu6ahW> zpyh!9tG5jhAmma|LI(=w_L|1cq7Y^L@qt_$p~U<0iS%Bapby9%qVJ*X>WBju@3rLd z1T6OI)yFI=>lYCs=g+f3O3jfTzme``6`(IK?+XTyK9L5dZqtUQ$IXUS03FiwOBT1k z3=KSiQO+Txupo$#5vJpmUuIy zMoh41#kPrK%3wv_+5E40;g`yJ;lZPbjqeRVM>|~IVOi!Bl%--w-isG6!wOy|K0FXN zH9%`$K9H3G&k`r$GzG*VzV&ff0MK}x5;?E05kHc@6s1a!IkxEEBe@Sj&ps!Nq{t(P zMD88&VEui2A$K4|9Z+^33)Hrrl1cUWSy|XvI5O&e97AmFXA~fOFtd(1T!It1du1ch z?#-24D%qX4r85e_bAa~7!4_s^Wn~=rkxJ>KH@<&YQ&S7U>glxt&^7gAxdFx~b*|&F z2g#^DB+R$f-GR|{;V3Sckldf@^gLfT|BehcGni2S^1O=vR!qiP21`bd`2kCjzgAMP zfOLN>2JB9X*$$6jQ4EMOQ61B<2o|`mnNaekBbcwr65&1LwIRN8E3`??T@6_6dKS#g zqCve76cR$K97I&cEj$seb7 zu1y2&%;>#QSg35o0&m7G@CPTrBF8xcL@Xe}1Dqskhm6+qaH_#yjo%+x`E3md<&Fi! z>j>>QoJ3|2H;&uD-~`m_;G@`SqyFEs<6n2d6B;aW@CE`>uL5}OsKfSI)myiUNJvOd zlDn*b#(>T|5-gvLzkM0XJV+t^s<&4ckV=vR3!pQxN~5zh<7K5ho5%LSGx z@lbdURl5|xGI@Q@Tu#f?msy}nP2SRmCFN1Ev6j4`ax487Vi5HD<}?IO8gJ(g@-1cos zU?$!hY?C>BYA3_-C;g@0)GjUKjxLA=4HQvEj)<^ESKimEp(xOpZ*lODrXEvN?Vg%cqQg|;!SPoRdF6gl(mw+L^D#y%uP`pw5@ zW6D~oDo#xoAu!kLOY3=rRZWLexM3{wf9oqPn`MK(mKL0xTt^$cqy1$#YWIJ5|)w`towu_H^y+3L-I^Aiy6veTX6On`eafyLazs7#aD%M{&j& zVLkhBvfK&MD^%Sq_^#2LA`S!dA4(T38_J)ZPLD~6Imh!>HH@-vOS_(ieyZ;2iOJue zDEreE==qdb@tmBwihZ3*RluX)ZE?ut#G8&saS!*fo!E*^Z^(t-W^W+Ip!7EJH@vZV zMY;)=JKMBB9Pr$9le5zqUXj;%Jk2yCd(DcN)or>TrdLTb|0<}%6{s<;5T}7V(l30D z^V9Z>fS8E7Xx4*dH1wkg5x{5Pcn=%P(Q`aY^qqlS`0gB4$zc?YM+k-4HLz?)Ejq^kUHq7h1s6 zQ)7B|wjDatska#llNsrG)gG13wXJI?b7a~B=qeqWyMo&;vvn(Y&Zv6K3lh2~v z%8Hr-@eQRsRN|VtV-vD-`2+tkUi_g4&7=;K6j(HCdD`h|hYzcP&{!S)Q)6}KfbIu` zjpy~Y7VeH0GTxb(TLyPK>M52URu-C=#}pM6g~Y`HqqJ@72=I%!e-+i!8m0XH zVe`cKxGRn3FkWA$bx(2k@KA$@l~-ALQAJrf0&YtB@Wu)wZTo6EIk6A1)K=`Hdfn+% zU8!XHSt`A4WaYO()|J2`%^`cybsn>#cI<=%Q7HbzPQ)L$iM)I^Yf26?N=ptff@q~{aUFM z!+M^8#lRQeJqMIU2<4im{1GJY`))NqPwBsM8TZMY@C$H%oIXcyYgcbRIG;Vm3KO!L zy4q?UOu4O{43ww6iZ+p2lj3d?!Gu{0RyT$aRX8(@W6=|?CQm{L470(|H9JpbI7mcP z)!LfC&c&q$zW)gI2zBLi6S7JXY6=PpIbYucggE!*OH4w-N!?J9QxRX-d-WWnIc6Jk9E!D1R2|7B0tf%8 zBEbCT%k;JCtv^rb^rlMJm`$WZu?PWYo9@McBg97nbT*ws)7BX)JaJ<}Y875Co7^TJ zIXCfE{S?0MnNJ{`)6>`Is5uGP>z4guBx-NM`2hxD<>kY@k-v>Y`2=FWp-9> zZu^HaJJPks5%>HmCA_vvQ^cL$DD$-txdCn~uTNNlt*6r6p_4yKesw$oogZ5QPW$7k zhge{Xn2@MU-p^sj$ig8^374di{-w?23o#91@RF4O03N@FhaSkuiZAW5g&oT5_%@&8 z7*Ymm%u0!(VoZ(qm~Y7&H~w!`Pu6GQrOolRtofXRRG4rI$2mL>#zyfPbKukWh8VM=g0|fN#LK9tjwEL-%+XG+qR3o z)yU6P+GSklUS`vNgCjZOR!RNn)p~}EJZ|>P55vP6e6pn<`2+++ba?sr)QIGx9xQM` zbEM529iLrk@E$Gw>M@(@N)mX7?q82TjB-`V+e8sN7$J4<9LLg9QHg?LUwz7D4fIw} zLq*Ld3^f0eBjz)vxMH;cvqB>dH=v3U zHQgoh?V6c=?9uN)N^Jq|ww#DSaSTjc9&z{s@`s&BgfV3w zG3J5w3a9u6@P^1Kf_QGVKO#avu&-#|F09Ze7icDFZA?r>tEgL{wLo-I0k-Ue)s;Jc zX-4l+kbE7Yj>6oyRnanNu#ok6ue)$-wU=!=A{ZLdiOD>(+Kt(t*z}Z)vF@ z%2o#>44!ThtE}|7V7E8XTv~18Klfa#+&A)7gjhQRzPSSrNQ)iMk_0Q402`eec4!b$ zA;|8nJX!_+3>7V{M|lvox8@6qvw*KiboD;%tu<(?uG{0VYlH^Ydd;%c-as%t`|b0}T(^(oN$A@4B6> zZl5?JPwmBv`*}oO|7^O?*@R0Qxw5;h#2qu{>B+7#jKtdX(^xTbUXA#<+n}ZQ^l#C9 z-J7;^Sv-Q7D`>)dvarEpuHr#u+W6+wS_7Q<>!rC;sGx?kZoM&*3i&s1@J#K zJX}>l!5@edE#TKP4N0EJ{urus%-jKzRq>Y(WopnxsV~SBh?s#^ruN5DO^%4!ojavr zd4l6=xj3g=H#>|kcx4XAO0k*|0wk=^eNV5-zUoc$yC|MTikQxG=#eUWB=o0w&MlHB z-4}CcF&mr+%f)uJkcfi|pDc831MYe_(!+Z&cmzA(L~EPt;SPPN`9|c7M_V-(!uFrM zjt+IU7?zxKTokR`Vij}x*EiXMhqRRy z1M(4XA9&a5P|6G?Y_CGR<(?flG*WsbJ(%FQMN_y{)Tego`%^a|2SKr++}-H&9m1?5 zgUZ~lXJ%Ue#6jTmWxipB9_?Yk99GG20xWg^E`Bkm@Alej;}&kt|MUCw^yv7qS37=s zpd#pU_pa)dD=AY`Q_4n0NhD-sAh(f6Wt>|hZw9p!Fa;9$V$;(FiM0su3!XE72ec`% zlrCJpU8Wdfm}hcnY-3?xOZAP8Ts6d6UMt&@|^$ml4CjLaB&(K*-u zDNd$GlBC}LlqRMMn7Y*06#?nX|Bf^}p3+`DVo;RhBw{-Mr}1R+h1rX1+`pT6`ew7< z{-#;rSzuPKXeZ;j|3vClC!MCL?46542{fGbrc0A*2$9+AKx+wHA zWZ|yhqFA{1c8RLl+1MN2T#9itS+uTD4l@77Z5SS8rqo^wBIr$iTI3|tL)De@U=(AD zx5lY|a7i{N{^4#N^F^tlV_g%@o{Nr_RS`vFO`q8JfvAYiB8Ja+P5cV=g76@C#fZDF z6L>-|5EUT|CVt&oqtYWTZ;m&MT_kBq?dE{XHtS@V{skXlkD7SIW20n_NWVJkRLCOs6kanLD>LG1cyuI-<9><%;rGkG? zK1w_5+gMZRbTo*Y9Lq#Pr>68tL-gu zA~^`|zWhITzi#m734=#>Jz*n>kc`YN3Rwm1s|ry~y}i9zNl%Q+hklSY(%1MS+p4k| zdoT2TeSdHci*UgM3Rn<2KgTHY{BbBwNElB8>OE({mzlp+ktttoJ)BZA$U6?|LDtVC z!g)wuHN`$4oCveEnQ_E4RFEn5o#sO!M95Aqm5V@qZ7kUEDMajpjgpMGcfgV!PoSvz zNE-96NXPME^;npu^24^==+_#nnQa87Se>G?XD62zSjj|RwHJU9%cjKl1AqF(;v!zS zsiF12DTs_Wvm{EvQj5}P?H7IEK9yqM6*c$}Cm5D0gyBO-7VmYGDaf&q2g*iSci!lH z(762Gq0_A-3ggV$R7(KLoUoW}QS6dgKF$4kP60F^hGXs|Ue0>G}Gx(4gdD zUh=OSF}u8Y(|5rcI48<;kC4BcTwT^VFuPpV>MC|BOIfk6(tH02B(L)^G&a-*3S^pK zr0o(%1}-W2*zz+$_+&qdcE7|bTnGydO@K`?U_pO;?(^peM66CzHM5VD0E37}gg-MZ z&=6P{D2xW(AlY}PK_|eIrbj4iLSbJX7aUOn~xV@L>9bKIx_sIgc;M3ICZ4!D+ zO76_*YfQhVXU346l|zrd3MI^;`vRbRA&gREKHvqlC11!XIV#0`(u1$q5$Lg~0RzA{ z3l1Dnk$i|!kk+a`i7cN~#{3Ui9EyrPj`1;yey z=6u(CSByL!Osm_3qzuk8pQ5F5H6YhPY+RR5Ic-!7R`-x94*w}+vfs{lq7goB9b*iM zLk+#y_5epAg5*`UqkRjyxK@W9`hZI9f9=N3y zp`ot|l@aa;EF2;SPUq97=Z31Bb8B~2+e?7F$Ea&k6|YIWw%25^6g@45Xp(Y~Mom|nsSjIT5qbjd|>%*k2srk7!NiQg_a zo~eSJ4)Z9p>Bkp<0oscAiU&;$4Q+I^VTWNHSSjTd7ScnQhUuDc546E8BUJCeWPk&P zR92>RdkLuPz#&L{1?KWuu#%<~tmdWyPnbsR?0Ek3(T@@C%%3_Lo3nIH+%;G4?C8z@ zAzRwXMW{Nnk-YlfX5XmN(PFyx&-`8}tfu<0XZIU1BxQo$C62}1rKKC$HmJ*znSgVu z<)_1hSKThI`-Q~p7MnGk9ryWJM;xuy*47ePJ4!%m6GbL~`{5 z6v7!1`0BfhNr=nOo;~yPd*TWKDI!x1-|0LS#-r|ViUuRwR#c(V zn`x~3_ABLV2m<+m=xPaukMylAEfKJwhAq0}`Ew2Ayo1k=Z1)lX0V8g9X$P#M<32|gJiqg()mL(ChM6cht(ewa(L)4^+0@G{ z(d4Y!RD9DK1I}F}WY)hNxlR_sf$gYe6BYV4Z% zuRdny|8IZkW+XErtb!js62r1&$ZUQ*Xb4^X&Wu7&i&z=>_XSer?6-fUb}cZqtB^dmOrshyc{c0`@yx`8cqz%f`16i z*NKJX2@w^HW$s;ze#?e2E|_o@bc)?79rqc~n55pA5^4K-mRsNp+6~Q0)^#AaOrUbiAp1>wgGK z;uUwL70i1+2qQPn_&XVTaF25@M>sB~pMG2y#YRM?>o4hI5We=EvwsQMEO>A3Tl$Fp zYj3EI7KHlYKBVO#RgKr=QGt1(*X?AQ3sCmwT^u#E?5*{_7aMX!dC_|HX`bz99M?yhiJ{2<~6zK>I2;|gb<$HksTlzf9X z=QU#B8+v-p|JCc)iKm#E#faPpeIiyw1Q81ISfE#5_rjXvJpU`*TnIm#v&Ma7dR|U~_9N2N9L0@9xG&dzPrw=7 z<=eI3LL)XTQkcB3gD(gd?x4`eT8=HIhVHdS^WyzBjv1B)bJwa2=LL#ckNly0tD>!> zK$Y&c5?)PQ>MDu85V zeKm^NdsALBRe}S?-jg#koX{{vrKH$12}?=w!$AAUcUtgcHv2pS}=&2fy=7EW)Ay}+h{fA7EVq8N>4VRvaHJ-eTzxA*3c8% z5BNn#3`BDkPcj6`r~T#5gAX5WI2Rg;SMsLr>0_Mc{&uu0}#y8MzX-TB_e+hd~!6WKRa21qB6x zBcN`?6fE)jlsP&)l2TIWb|PP$CdkFARlb(Ny110tjJy8a^I|KMkd*A~;5IWeV@q{{ zp&$%E#JkjH)f|o8el^<9%f}FG8N9~Z*;o!hb3)pAgO095LP5@`hrX!_T)R6g$9!507+g@&c(WML0(RS1eBeD$tm?ae5id0{^+k18LXt`YvD< zozq8(79xcNfS44d10J^%T#PzRmgcIJ0dOP95`N#J^7HsI>ZLmAgJvU~2e>4tjG@Q!wR7V-U(EkkbZ093`Bhvie^-!^58*OzDOnC-TkEY(AlQlduEx2We%4SJKwux8& zK0aK{BOmFD^9*z~%DxY`GchnN_OQ3_ z0OOwP{#Q^n(hd#|Y>6G3yRtxX^!w&%-8uOFJvS~EFJdMqcRwH?KpBjJcuqIpoQ9^! z>DD4_3(I=)OlzqC$7}aFy6uhW%5B%ki#OXVIC62LL|l>`2DFdJrro>0ir4<-rKLD96=X{U*hl^>>*&!fSEAKe#gwoO*m9=JJNL#mJs_vyBVEeg?j>Nw%hy$T3QB#tWqlC}Q) zrC@R06ZJNZm{EwsbYKkn$!2kvZun>U-2wTOP?`INEpcGp($Zq-=p_|o1|fKUaHWK4 z^sP9%E2Dd|P>ldnToi6SR3mbJL4 z^uWRzVyA!x_>hD}oMK`UE&QRGE+dfcxf(=w{CH9yEsd`AEa25QbaWwu&|*zZ)9C2v zlwt7jPg6fHNj7~~8jXE~75T1K7*`Qr!wz2~j6A|6h&8H^erjRjua#5$8I(Jb%Ki5N=aU+s=7KU5d(Dl64+8}2vePZfBHk& zSw+T6gwTAOMbL@uUs<^u-@koGK*M~&ko;wdCLj19i@*58+f*T^lShxjHaXr5cL@F7 zstH>q0_@lW|F`8KR!2lXo+@Y`VR6~pz9bPlc7A?!Kq0d5_{Kh%G^=~mgK6#%Xyka5 zLje1+0mU;Y5R?6IL=G^KkSCf`?eq(kC@auK=sj_G3zg^nzKUZw7;%X3X7qeI05S(< zHN_a%3D|}BQ+uqk4>^B5F_^n<@ry1Zo1U4TWs4^R)^KalG!b7a1eAZhR9VN#03sQ^ z$h3~PVXMS4%{P`VojbZt`I7J@Ci%^f+#Gr65X|{c+p7%RYke z`78eEk8l3C|MA#|#cHNJUGo07qTJr~OHnk8>`7=`VCezN<0Tnyj#=CZ*T(3pr< z+!5hTe=Av4T5dc)OqQ+EZZ`Zi8-=nt#d{FND@wd2wSTxJLF9A`H%`X`&-6@P=Jg<= zS((HU$;-XkNkpqRXFWJ@bd#jLIuTq8`8n!3-a5YbO$-DxKNc%UW1$oR;k*!lSIX)c zIXOA(yu584;ccCru|Q&Rbasvg-}t=Z;`nZvAL_q6uyEV$W_!ru+STBHu6aIK13j6K zq1i3hHgAr&U{g>pC#$8ROJ1fQt!)E0M^A~=y5`SgRq_~s2h%mtF6-gS@%ky8c&EXD z`O{yh*)pQAN^aFPXTPhtan4>sY^Xn7Q;Zjf;>ogJPHSzQ*bn^NY?dVnB7m*jOW6PG zmxv(UBEbZLgiu>tT)YGvB>@>aANa$=!$V`Z9t06OL><2}ur5^4XJkA#ZPG;3fA9ct z)``+y&#qxa8!o=%MoyQ!&D?1rBrhB5`;LdI{e~DDqrJZZ6p!JRpN%g|792&PT~KJn zO|@_Y=4DitEEg+65ixz+1cJ=<3C-kRK}PmIaXLLK`_l%Eh4I~E7CglISN;8ZU_WAI zLHDh0y+^HtD0U4E4iArmWzs}m3t(St1uTVcla}Epj0I8je~9idhAh%i?1M(LSFEba zPT5EgO-g#N6>hVk5zo8*u)`qbQ~Mt>YMdMoy3Y&LR5r`$me5mrZOvEzrJr?1ts+9(4sQMx z8u$wDzEU#&I`~NgJ$+S-9dh?6y z+gA0O*I}u5f7+sx+ANy2C|vAD#>=r8W0W{ZDaG8%lm@&K|B5wJV+@!f#B3X0x>H%a z)mX@ZkIFpoKFJ{*(EFr=M_q@(M?5Ah-skphZQy`s0d~JMQe{rj0MyA4_$QG*&`ld` zA)?aK&Vowxc`4Yo&`|L3vgc~Uq7Z-+&6#9>$n8D7{;LAReox+>x=z%f_e`Xk)Me5X zAt??@Ig6B}5#DW}V=?tr$rjD5hrxg5$GJKP#Wvcgts4%PY z009Kk|t3E|3yCV(wLatlD5i>4MZ3?p;qdK1U;%J7<5EvM0`+{{^?z z4f=eWXJrLedJxbPgci48f~+B7`^{HaR_^jKDyn1#ouZ;5ztyi_&k1MUZvQ0=aJtpG zB=W6o3+9dridfg5gmr7h2cY2(CP-;0zLrPNEz79?t@(mZaba7$`KAV1q#(dnmw^-! zn|P8xaYa6ghW?@eUJAhP(sKZW5N@ZB62rx=*^7=11=Lml`=|-m(pmqjFcRs% z$&Ag)yB*SO_^2$aulv!I7GEG-9A`e4;tT_d)bUI2Os6C$Ma7iy5i#>IQ+msGE!$4#k>+z*h$RFIv zKM#JKj=rx5IT5+CaBD>%<9S7{B6j&>!a7xW^8Pf&p2qB}8VRom0Vp?*+e&@KZ0*07 z97r8Gu-UV?kxyBbiOfeJRr!SaE`ds?-ADtS{f0ypWW_0J3jc*pLr0vKpC1LORxyJb zuoHL_z<#gc?r!EXG$PI_SKjRF7Usnlfp^DVh9kq<;CG4IjJ-{kb>*`ml;0w`1suH9 zTK-ZqUOl>Bi5l8Wd7DH8h5>|GG`t#?FwqE)hXeQ*!4(|Yrd|cn5><;27r%Q8BpqWE zZQ0xXVF?&o!#8aiZw6>ioNgg*72qUHdhF4CFP;W-7giqj8H!+7ygExV9l4ta5AU(;y$uf}a!7VLLC zf!F8+E1VTxzT@W}*)M->bdrjPVlCrMb91K@544=}bzd8g{BnPVtkdyn%SZn>{Gzi1 z1*JvCnt8bD{2&4>DPhn7b?&ravM`beETBo;RM3xuxm_v0-uVlJV@ZwaH%Y?H#~q5* z?mXj9lVMKQLyJYf27$zTmb*TQ=EOeppN}iZ!xIVZf~uqAiaC#*_G4JNW(fw}SOw$w_hwf?mpnaE9rHQ4ggICb9b4A=wk6=G-Rn!FEdynd zB`UM%K(c#@t66`I#+DjhP6h7OB4+-oFj}~MMlvGDhH#V~Gxy9lN*=f!N&m&|Xn?K~ zW=YxA)!WbLSh(Tth29~KfT1CH=}BCpz^SKO|EKKJ&CAPs{y{vVWvuK#XptFeaDv}tl^sB{D-7Nc^0e+e73 zEL}kqD{C>-K3XfpQtwY7o(^EWiXtXNgA9`&k1ynfQuJ;O=0#}d9MA{>5BcWye+A*N zK_7d2bt0{`LaT(8K2SD`0G|Ime0&0{o4SB3r9^L^j>bNiC4dYq(6z}Uyn;pb7`*1Ig za(QP-si_o~ln_HMfFqa;y(#?^8?(lhyo&?^}UB-ze>lI6Happv|s^!oxSgM$O%%gzj3JBi2DM@Ago%Q=IIFH z1AZ~yH<8+0B-WBGStzt}wGBtG@DeWPlT<1%4#Sd9pWQ#kG}#5$?EW86wkR3`RPR>? zZyK%jms2VOnc{H;J0pUgU_-dts(KY^Tcik;O32ag!-P9B8?(gCO%g zulAbO1O?Yc?eGsb_q?tyO?eNQ(iusY`ItPw*xSL-j|&h@_q4P9qJwZBM|8{#5F4HS z?i%PaXg;-!bffYeTE5M^X$$XAUReVQ2dx;K?!G_J9ymC(-}ML*O}BE9`PG(k^F+;1 zAqEk9oc7nj4M5@YUsG+5-PC$0@P6HNF!XK#a{e?B8%i8MDg844nn8>2zLNTy_^YSd zEo-CPI)zlF#YErs30~i~?~tSlS{|$WGri7Z(Ps7N!HeGS*dD)i{6KhkfL`oPUii(W zFt%IH?Qv%|gFO#KSJ@AJjODaYZokKRnKW4EEvxlKWi*dK&5}oH!uzX~P$JZMcs$4lTn|}znI4$;wZ(UL7xr=Q88 zxE^yx7f2w=ILSu^=QAD3xp zM4p(I78e{mX%>~N9C?f$3^`W63<(<=8I}A%1M40q-obISm(P?~o*kgJ-l6n{Wo_b^L<}r)}BHO30J4pGG zah+$ct7efcZq*K#zg|s_ad?z=1NC2L4M*?$6T`xYUlJ&4{bHVq>-lu%7^bk(gH^Sg zq*o2cTX`UW;CEqU;_7}>jQ@$BK$38RojX-vb)?6AJX1aET(7pEX>CMdVc`e{=pdh> z$%z8ko_Ft9;a^eeOIJChQoB@^Jx?bO(=?YrSfctqr#4q{Z9XlOuixFQy|PFd+`T~4 z&@6Bk&;^;D|2|;ic~A3QMfJsKDvb7;55Ouf<&UKgxKnY}gdga{!?5Plf1=NGYcPU3 zc0ABHRX`5s^mV)W`gvm7w7y_;MgCzn@B05o*qg^g+4uebC9+k_*ph5xY!fM4DEnaS z##Z(gWp7c+E`NhI8wxEbvXmBUlr>T!OO~>QEb)6EUDx%yuk*gs_pj@GJg)P6 zbac#d9G~~{dOcsY?ooRA9YdF&eseSc^35eLFI^|6{7?-+m$PTfJBY6`$$rhmtE?6) zh0Z|_{nS%`-FbZ}hRU;ta`sjPwOZfliF(Z{7VGeX_SNL&h@LO#o~>rrdk04L{P#2a zuSZzHD7ZqD$MaBy$c1!L+Cf1*QHndswK#TBW-Ct?25zK2o`fU3;N8V3fLHXM&CkmN zDUvxUSSflA zTx^u~@u;bsg--uHl)wm)(^CtqURlJN4Oy%M{M=v1UVrIMfM1U{MF;c0Z|SoKj4jGO z#jBd*d_)nvvUM`OpJ3sLG8^R*Mo*|ySea)B8*w?A^xt<>G+sBN{);rMZOjE%>EJ;M ztR3J321+}~e@`Vanv5PWGqp7;-Xd7|qfEw9JlaG&nn+ZZ8z^~@gU?U$G=7^aIB~sM zaJg_P>LX8+f#zvdoeKKdZU5(m5-!R7;oo=f@Tmn?QK1h7Q7_T^=4{;6PhMHlG{54Pz2;rr#-I z@(W?E|NUKJRl74q$0wos7&Y-}sItFjw_Ls%u`J);bG?jpA@;1k8mXxkvclH(_fNRP z+w!O<^Gb}G(N)&OY=s&6Qw!3GNy#5vb`p(8mS(lMW2UcDGZsab#~+bxZ`>|xdnMZ$ zoO9%VznfcWoA95r7>Ro_Tkr`$=iX^5CxWlG5#qXpnJ|4~zfu<+N~M|9BH^~YcykkZ zsC<`w<{74#St*N>yH>=r1zq)pI=k=ka`&(<)W4*xaRi+-u63fA#X!*EQm4PKP=^V9 zu&3((`5_XbpS>6{tSHGJmN__4^PNNPDHo+xF1=h3e!PY8a#{oYh=a6-c_7zWukvTA zTefW3*sv0Q@!&s5nW^;aLFQv+#at767Ct>yt|kZ}DE$(kH(>&_cZ$7#K+SctR3)w=vbnEcw>T z$a$~p<^x|j_%r45y2oBN@!aY-ZLIt7BXM@+k6@sxsAnH*F1SqXPU7Je^33)+cJ?ug zeu|Kq=@(7xVR2h|$NDDiBC>$0)YFG!7c1xI=SO8A1V?^;d5_Ae4VqEcO3!_|}d6I0bvvvjPMl;J{ z3XdGtmY+yIuNktYJa|d@`(821k+Ztq8hmQdS#RSE&R3(!K>qQU#WC9|(Nnmr*Lmi6 zIhF=M(H%Xv5Rq9DQ-J0m(#z*Iejmavu{0_F@jelrZDT0?Jy(sUlJ>-D8)~WJb6;xU z!*nb3s6Zt!ItoG(@jWX47J&KND=|tSKIq%8jey@VzL57QKb^t%_Ng!o|zAu_nPd3OuC^>vOWbRtaBMj`v^tvt0cMNsR|@ z-a~DLWqns9zLTs_y3^rdB=z3Q^SwEsGHzN9f&Rnkh0~Ni{M0pW(_KeZl$8;VckJls zDE(vb5@=5u7#TIy)rC~3kn(0`&YMC0RU8;pK#ZaN1B=D(RzDqlU_vh!wVe_CTzTxXYwX4; zwFJP@r+gWD|Ab|nPIuuR)V)iLAX|=ZYnTZD|8u#1@#H<~)h()wOvli~Rr#Ezn;P)1 z6nGl{$waZ$Sm159s^DWmiUG#eSFwz^!Iq7jP)Kk0jRPqhr>>vAi=MB>3 ztnK_D%t*ovZ4yGja1HV!FBtt!<7a$7=Ax#r-xgHa9dvP%K;Y3$L7vMBo9vnV= z*bv7Q!c4l! zLwtJ8Ct)w8eK;`_t%Po0-oE9DaQ0q%m`JEteAoRGT=z30Z67^fBz5x2b#weqz59Ck zIlHer3Oi%;ggZyez^`w_3935A-74;G(6p4>2ToUegEjBY*?}65zOPwfd}tlA9zoa& z!>}>)MUg#}HISvw3DT|}UK}%ukJ31Nq_k5L&E!>GY*cJpf`_p3$_WL1BqS2Ok=xm06$buy@GnptG(hQ?hLYz_p2 z)73w}AXteIY0e&HF3zv^WhZ`T3nNYN-%1L82N5CHJj-irCSXxQznrStq6f5ZNyAtV zI>zt(34VXhRJuyf3bgMDKDmn&SNM8569EuaWmuI-DtwFY9QA_cJ_%j~Zf=#chzIks zeSlQWWo&}AwY9;Z6JZa781^;>gW+p25AoKhW5h6c%z_ubWhx&y)A@7!8?*d*hWvd~ zgI7xIt5@y-bcr2g3CV!Ml)Y$MWdx;#uG~j}yG?Bip1zJ$N;YV=`O#JIUg_d! z6twm#3Qnc;FCQH8=yP=B4CS=mYX9WCI*i8_1$r0in57yZk3k5QUk@u@%c-ceNpQeQ zYJEdsRPK3q3A*r{qjm8|-h_)d${usViSzlIr`gd&R-)9X8NR|e@y|EJn+8UEqZtoY z;nmpl;}Hvq8tQC_cX6v>g2;9&t~Pu^{43}jX~J3lvljs1N6vWxx4OC)*dr4C1ooV3 z0=7qdUZ{RWpp&3=y*Pgn=hU5Ijtbp%1~<6pJ}f;DBd6{Gby{B4z%RX-K3>C{B2JRZo3fnMNK>F-Z1>7ku5F^C2TY_H4P<`ZH+HRi&6UX z6>IN{Rl^EiCoe~PWB1{Kx2I}&+0lCivT)g;Ti>snROB=J*C*%eZgW&u5k_we-}RM1 zLlTR^T!kGMgNzH63$}B`>{M+ua!v0W83@8q++hK6;QCml9$P=V&j@q?J3c4T;XcjJ$kpfybs@=>x` z%5o_*=4J%L!T0Z9dlh%2MB;L8BRKB(n{^NFeL=DU2I?=Eh%R_-%q7>R{5rVG7}g^R z>h1Z$^PgCqoEzyY!sV7hl58M&;y0vszoBuKmH-oG(#joPDl>SgxHF`~@SU%2RePG9 zn#j7jg|S1!t;TLLo;E096W-d^bC|aW6~wQ|qP(8E;fv*1^nTf;-{Ac$I{WT&UXh#^J4JFH2hQMr1|fn_ zPVD_bZ_D;^7=_P`=VvxS!0>4R6Xr?0qvZ}uHBYfYWXiu|O6;$I-9Du5;JfcECmZ_& zx&+>kqw0Kyu8@m0_AnJdXqtrCv0Kwm2df?w)z}5@J#y#}eZ?VBgxZOK;1TH#XlpYd zxTtRzWvWIU@3qrVcj;|TC-rI}c%$xm9jnuz!P>e7SVmRlU|er#cUAn+c1gZ^2=z6T zG+}}2QY?J*fIi2*9!0zTr=EXj9I1}$++%oc6>ZES#he^}KKVBhB6bIY5O$HGe@F8i z#lz#)_cn{e?&L{IK|=#Uhziw1Rd|xHEtvpW!w54%D5_>{P9H%^{wCp&4-*cbP$luR z55EOjt>fzAw-7u!j!1~K;%PstX6=|{k;cdHG9MesKn-PW0hr17*C@egRP{F-u7 zWoyZ|24o^cw(=BrU*O{8q}|A>TgFs`v2);$TUn)wH|#@LMGO1Re$-I^*paHu>iqd+ zM5#{Daz?BP>q+uu{(v8`u}x`cH1I4!1Jlcu{zCdSZP@k7^Hub&`Wp& z|4Q8#p+8ER_zbsXUg`hF3r5$wZ?>6UalpUQfzaF;pcrz)&R$iq` z4W!*`LX69_iq|fCk;PL^hvu*k?|9jG&G18`&k(z~zSaB^nV{qvT_vMPaW z%X9Bp&`hSg=60CwnlFNJdVG8DDI7&>s|r z_&RQWN0Q0D@x`4^`WZqBc~&QB%?AzAZz22C(}30THTbU%ofO^nZOg)udlhEQ73z_{ zlOfKZC7&jo7IX=#^Yzc%xiS0V$bHB`pr_-zdr1G-o4cj1RX!aAu~J50I2pR?;>(td z2_K$?+Cl=}$mLquyCCajiTgSuj4CTSpe{cGC`i4Sm_Ne8y`|xp?5+_;S$xci5Hr+T zUR~|yBA&>T9JsLzH3M?<{!=r^+Xs-iOYdo_D;(=8YNr_I(#0u~F=SI#X;vaDrkR-$ z=a!TZ6Vn!>!RE=>6GGeG7%yr#n3g1Zb><=@%If-Xu(PMwU~2>%1WpUwUtcH*;$FTs zhJ=SS!$}ww^~mwV#ErW}MRFxx8Y>{~| zTY)T^?Z6dh2b39Y)g?j<3mbvzLHB>}nZD8;ULLeM75d`%;pdVscMY?fY7xGxDP(E! zGo7TBTos@IT>ni!rUh_4tVhL43vIUb&P8!+b_=`5f8 ze1=&g0!(e{*6QdJpV)xbLO`gV4Q?gXXGYHmax6X>MxQZE2Myp`3X6*bJD#1kTiYZ1 z%4)ZyIujEJ9B(U}XeVy0LjIi^UhzeUoBF<5C@ZuDLZ5N9#17T<;~1V#Ka%8v6dtN* zgc!CF04fH%F@;*S+@Y0V^iGSUK3%D|iqfOLZx9fol)K`qkF2g1b;Ua*w*9#@efCK7 zt@06Jk0gj(Isn_4KoztH-ke%RMZ5Z`V?eiN#S|$fdK^Zfzjraq(}R7}Z7dJ*Ojwcy zfk}|#bXUVyQj0-bio7>rUQL#co?ai&0^$v5l6XI+Y}<*GX9SeL_?WV~5U_()=Unv_ z*#pb17YCJ~0fScKslETKmKp0A@_s?_$C{;G7W?B|A|L$~<`NobC!Khc&n*E8dh9MnPY0bd8HE_G!jQ)$=6_zX%l5xS zZ4a9sw0x?VlYQ2uk{(!j@2SvROJ-WlPwep6)kv^yv2yj$7yHLAvHMSa;Fzq;n2 zlKk_<-ABaqse*L0>S3#djXUSR6SieV4;Tl|v6(shUH^4nxtWbPpo7nn<2SYPmc&xX zIkF-y5!2)#us-m}aLR7XTF~jj*B3{E7XGMt}v=2V4pJ&dmDas(%&oh0&zGRa-+gd< z*7jo7qX=W4&Sx<|9y{>;W3m_LzW(A*-#^-z@46UHXT3M@$2%Qu3V1C<%HS9HVWUn3 z271r*F1@Fb1qz8Ky4G6U#D_i}RS943iEV;<`D{q*3tcm`y>5LOZ9PuWctI2gS0M&R zyv#oSc0*Ii=>$m|Y$sR2eYW8Dg5_lv0kuhn>yVIqWCUX_^TK*RC_85sVQ+Jb_)Y z_U$4ij^rznv!yGSm!*aj8)Ro_@}YKpMv$Gn{hVnpU;WAD3sW=mQ)?mI?<2C(Av;r2 zvpuV51$|NjtN2vtze}ygh5hwbM2*BR#q`=q<|o~PTgu8LqfH1Xq-61iZ<V%A`O9f=^p=I3q7Yn>nw7R z?nIcCX?OVK%knjEf<=~|g__4ejU5v+b99bob1g83(k`)~t!m*Q9XVRNl8B&IzzkJWC`wpiU6Bmu6L@sQ`=l5m{u8DOS-+h~W#^cS=0-(&e z!h;C^YifZ%q0-rU63ToxJ=93l4*vhZ2Y$IZbSAl9T(Wt%bJwmt!8GqeLoU^wA`?@B zLupy!RMpkhL%`-OPF4`(|Hg4x_0YLdVL`^wkWRG_5fPEsB;E#IY*b*>Hz-Z_;n`aQ zkXGm;ul0|Bwb}YJ`_a||>LMP!54FfQH3Scaz0r$=@FSIgJ_aFd#=T#V$bxiLIrjv{ zAwH76Ae1u@>8Pt_U-RVs;_u@KxluN#YDx|wAI%J0DTqBuHgax+$jme6l=WkEP^XJR zCqHm=5TjuhZZ-)?BRpx56BGQ&Iqyw=oiQP?7hc7+MI=&WBgJ}TE9n1t9EhU6@}wsj z=|fOfp*85#@#bQ9b5i+;eq1s%+WWcx(P%%p)Igte9<(ryp01|WHr6A+)z6Ezh1p|; z7TOhNkEIowaMgDpfQ zSnRvpqQ4OK>p5N2y-T~>wNf6_UMREqu$c9>khSRhv(~;m3k^df0|IOlV^6Z4f|)Op zpL_A4V-cF|lbn|V9vLM_~)0~a|_oOjZXYm`YkP^KUb$^5F$?UMX-TCIfqB8IklHsotp zhg&0zz5UR$@0-UrZGVB;P{iI5hK=b;V-XiC&AX_u7tPOqj>(0OfbQ4PSIQwB;Ab** z^Ymh9{~!6$^1Vc`;G6*ej_7#<=6Lu-Nh zV%rJ~(8|bQ2M}r(cqee1e7w%ffBwy(yk}b?q-@znrL)MkA$RJ6E*14?)iJu{!|8zd zKMk^e3FPcWpX{E@gm^9_Rw+aC#AVJLj?fG_2PVm7GUnQoZ8B3aW3hKas_K>%ZXI3F z^ch+bhlvjZDlEx=AuY7B_n^e%KAoJS8*2JQ#>80GJQ}?ZvXBr&h`zLOr$2s^#_6h> zD!!8y-TP z$o|q480sij=XckCJ=@v?fY2|mcbXBRK0@`U977Fu>|) zb<)A^=XTJ3uCp)?Gov*LyVQpS z25Aen6~x8Ho?t4P6rijL8k(;i(G5RVw4RrrAKkwU@l(;Hm2u-BoERaSzcR~0;N`AX z-jeG%CZSDO{hNb7$*v0Y47^~#WAbET%(m-`liIhh`8G@O2Uo3QW@bX*D$_m=9%?-z zF%|?a`_=J+4?;`1QU;Ixv@31<0J3sy;r+Q0{&PL_xv6^ zm-U>wJFp1A_pb=T?NW}u^}(`*jsBj1|C-^u+F0PrcBJCXFZNJ>uo&uA>QSe1&I?hx zo$!s7j(^=_9N2=AJ8~M|xy-{#tcEay>$HT2<)gqJNq(0W6G5%t&!yFQ>cNcqzw8M- zTOQICjR@!>p)KlP)#C1O@P@Jv?>c)>h*6jSZuIR~jL~~_=g<7M*la}NZL05_b=2D0 zbyRY&Eb!}gm+)>cFzFl|-&e(H-=QZsxOOjutYP(y?%hyp+6Frh*FM3UtU}kwLkRgX zC1H0Cxoq%QlK-PKs)GuI8|v*@ulSj6-_IQw>K(T!EQgP`0Pdf(b z7P5vX)34eoQRItT;Urh?gubVm?e7&aS72B1mEV~gm*p8}fp^ib zIhoE@mV{{vp#J%-eI~@PBkB#I?Y;=bw;pJR4>`+$%PRgraISsDu~!o{atS6Xw<&{c z|3)(D+^q!6L@K*cO{+xH(ZOI&|CD4}!39lu!cKR>jx1hYT4p>amYueR<)Z-Q{C82~ z4B4U|WRZ#S@q^!fye(&`+a7_*Njxvf>c*->K0#0zRCdBj#@j)X(q>r)_Lx7e&quxiuU0p59 zV_+`n>XVyJ)kGNHd;4BiFD@VzR>%lEE|nbk_?^hTH#JEj(WIj9rj!FXY0Jl&Ee^ig zgilu$@$9n#J|PjLYx+3KjIIM!}S_?n7g@K;^ zj3%$LZztg6cbNCp^RUkMa`?1)U?AFX0YS4eT8xVX4ec-lKl$01-duf~BDxZCH#!3Aw zYzVls@*4AHl`VmQp^MM*;I|*U5a%(Q(Hd!8<%E3r#Y&l8`mpTWnaa-P$Z<3HB27@~ z)Qblf2GRsir;k0V%q4kdO5i)Yo^JE}Rzq^_%4lRI_TsX-bN2%Sp5lo{4uVY<(no@~ z;LWES5J`-w|*XP|eU@J#y#OGqJDl2v&2$Ja5)Ve+Bh(pfWBB2X7Y&OhQp6bd~&m*R~Z@SlCf9{ zJo)=hi-SUwR!%E{?STLL-)8nFYplDd08nXg*jPCVd4fsX=yx^L1grwB;RQ*oH1*3F zM6>(7WT|ti%GQ$6Yq>my4%;00U&=mau<-Vv2bWL>+lC)p+QDC z7O71wEjgrz<9;Eap$NN9Dl}#E8n?`p^@`C;j-)<_u(HZW7m97e;50nrF<7CSjT(R1 zW*@khdEv(-LkK6Y*7-`Xt7+gR`{R$tldRFi5^4Ct75w81rwU3JM2jziLTJU*{(f6n z$0dUnfhYdK7&sOyYQM7*6Qcu63owONTINg7a-xh|^#Zj^*`2f-l{kos5rc7j`hVRk zYfGSVvl*fwv@*$ugLqRwRTmbjeMj@YC~MFh`GcHfuwE(5t*1B@Ne_Z{^)yaO${2c% zDDujK)*B_OUl6}S=4xk4m8}V**X)^+ej*YIFV1E*Z+Zt^h}~!1J>gHC3O%peQPa2y zf8yCKhN!Q}9`yC4a^QN6_pL0nLEMn^p|VXz>iC~LxAv=W;zmenT`=6p)^Xt&{H_|fsBhaz$;(2 z5~uLp2xIBHjo7gUwjdE=;HKzED|!6*s0l$kOAZO6iPZzH4UOQ) zoSd8^E{9acvS1V0@35Ohy?DrP*k7}x{~LEa6l@(U$5424qkAY~3QHo|@b5ou{;1>E zVA4d~JT7}wfI>5U1OASyz=GP(@sB198bC!Pr5GfBVvo;$ly1xibqG*`5&!}p5gjeZ zD2mNCG44ZVCSVaawY6Q#`S|npbp$97bON0A@;@4Sd6e7Rl|{Be)X5pH9Pqu4R6z98 zB^7`KDI3>x_!T58`9+kI%L^7~;bk_VJiH2Knq^gXIPk zmH)NefSfhOz_ll~Ro+BkUKn7QB!2?e_vPUE6g<22w~AXNn5D!+43I7$rJ@Ubr+>dC z5QUTPWir+qe%582JNfxLmsBFkf`WF>|G&DkaM3fl7EbtpqaSiHh5DnT zoY-e1vFlZ)88~`)7>oaTKat~Q=CittsH?BFDj_rV9X`RCS;e6vvQ1CDA`Nx=O$2CJrK*mfD*S|`lYKAw_!9In+VqzJ@)K>?Jk?K0D2`5yEV}^yW=RJP>{#yKg#ai7 zks6R6L45?j!2?mnDyb+vJr1_0%Owyv+CQD>&e$1>*+ zqwlo&9Q3 zdwm#Z<`WfuRekA-E|_~Qf~C+Xe|A@%W(URGIYV!4RTZ!`^9Swp-j4l@`u+ZGUsJzq zyTNcR%GC@dC$G5g#*L*Mf4eI8Hx6jz3Xo6h=1u)f1%2OqmfXMtH&$Z0bdf{T!qPlU z$}{z-J!KG3&-WA@asPkqHhON0Q!zio%F9P@#4yCA91!_y!KCy&%J7XV2G(im-EFGS zgGG^;k^*=-MxtRo@sDB&2%2og0T(V1VK!qRDIu|YcKFK|AyC8R&B#0s`q!x}>MP9O zb&i&|w)+IWGbd1=p|TvKs$R~8)NWZ6O>W|qzZ}T%+IY#b(HCRg4%wgTaM@m+BQRcJ-H-wcvUetg4ZR)>! zZqIj@!w*tZxm{dbVgY>O0%<_d{EtJk5%Wb+A1_G9v3s`|O#WadBL&OZxTGWz_&@Vz zo@nQbWmo-BM&v+sgE0>@yKt1k54oE3SB%;O4zI8wXRV1_yOX?#N4o(RC81SXoCL#jdX|EdPPX9f98033_Zu#n0Nl;>u^6^}_r zg)N&5K?Nc135b&QORp!xXDTK_ze#C~WRVjD)*A>Jy$FV}9Do7cqSYiEA2$YQrY z<`iwxf2FI9*D45Qx&pWUZ=xkA?fd9OhVKWWmO`7Ty1yi7GXq}Yw>NKv>-0DSx7PX} zw^kM6aN~?&wWX~B&TA&VTj}W>EnylfjHlmR(~xFj6hCq;wECbl9LEDJcl`b-TD-2d!YW@1NVf{t6*D^YcK;k(-sJ4m~^;CKaXq zH-IGYHN|;$$!6t1z}d4#;PEib3LAjTY_NLZVeD-P6%`{>ZQGkwnD158o`E2)nMD?JoG|6nC-5q{ zBTaPd;Fo`MdUSYd5LV+zbMooq$Dgr}Dkml-A%Pe#U%q5;WVfC@@dytv1W-cQKE3>= zC8;hDaAOdrhzzo(KkUBSgTMk;_32RyUAdOai*b=+zb?OBTcfN`zdU-G8ro+D`!8X*b8(3RZvqjirlzK%_wIFXIvUUS{%!F&h`cd}F>FxcXGPkK4j`I5EVS6=s*abG=#Rg;#i6$VJ(2j3y%F}O zvI#UyObKES>gGhkk>7xTK4i)&i<_|?oR^E3=7Y6+E*R8XulcGqvd0!pz!$w4V&raG_S z4Yu59;}%8(^GkSgRj~{83OpeD#s5(vg4rz&iGn1FHDn)k2(aibwsXU>_kB}#d2x>l z_BeO34Y8Mr!yMfcS70f;NYHU|0{b(44vzW%VRxi z0FS95Vvi=6nQmTktKr|YNJ4fEqm0n%lRQcksr?4&=8m`?c^gO_`mi7<`pN?#l8ngy zlwp(r)7!(@!-JqRoU2z|e@CJs!HJ)ICnY=5ST1IP5FwLr29R$p36*|J#s2HdV|fCU zbV1?Z&LJ0kqk*azk{&utL@Ys-^jjU{h*4S zxbOVTF2>#PC(8#437{69)AKigP1Epj_4F+gkVd7Em;Ss2i8}D>`xOJc&$u5vO$&9w zC`XW$Pi;eWv7mBo?ze{aWuqs`3|+MGbU&B%u4cweUh`!6qp})3BS3SJ`G)ji_&V5~ zjnD4OIOl&dm1He1)TATKZbsb41pyAiET4q7LQD5>cJ(vpe&UT)e?eF~Vq*pnR%B#k z&Ye4I(lRocguL8bE#z}oTxZ0sun1BRZ-hg43c`8!Zd zkK9<5p{!ho-R@CyjcWT?NTkHC1q8e_zOhM2wl^D`$2kbeImk1giSwu!ko~=UDq{3u3dwg!=?2NC2)(XqN2E&8Au9D zfXtZy?_5M=q;!~hqVNOvo!$CU8OYP1t})1?e(j4)p3dCkK2e`%dQesMT~~d1>9{C9 zL8=B7>aVm(n#T-=W`1S4eyvVxW}EqMC4>&Cp)*txQ>WC>%FKL> zg_}OAjEP}7?kaNmug_frB!|qGaj>6?a9I4NL=&_H1v!AK0C>lt>FIM~gZ%FRV8p$9 z_W?#~O^u4xv14ROoAQ*Vwzg#OyTjl2!9j~@B+R47?fN;|H_kDRBGVkS7Nxq+GUVbcaSo!SC)G8V)R&*l_!hN2LM%#H z4Hx-A6CMs0s~J)BeK=gLu^LTh)Hn7t6B4xZnbs z@g0<09YTs`rl9-X6uhX`;BHk?56NfOEaP+222OX7Cj8& zGru|sJQ!4061tVmXj#Im)#-|vLt9gHlpA?bfZ_yLa`;mq2?@ zV(gsrh;}(NDg^SsGkNBqFx6ETWDKhhDNA=SIsSRhVX)0!1A4p#jW?}Q4V^6^KkkxN z$}8<|+KrQ+$BGNx)(F`~`$A~|6q$V|LW?JbHvN7`(A)t0V`m#}ENP|c;j5O0K+t(z z^U^Ml6!U0%jHUP+;F=y4f38>Nfvzf9)#0NI;{nu2>g1el8XUgbBeugU{Up+7uKkb*tECc=my9ddG?MtgRsUo`6Wt0_vwv_K0H7ld0{CstT*& z^P9&n&>l9}gHu%OZHWT`G;2c4F45u9YN$_aOL+U{3=P_JOPcawBw}PcNft~rty%fR z!pukx*xhXvE7+Ju8q)rg*F%lw6JbjxYJ4O4`=IW!fmWxwN86GKp006UU)r=6aAFD) zn11AwtgBzM6YoaMs6d}Ht9*xchM!Ac{*Cj~n(=X$=aO-zVnz~yeZS?M89VLIgmKloX+H_#cyXaqGHIMYV$X9BNqMzd0}L=T zcoX;M`{^wboEt;qrJd0FYP~1yX$*;44G01Po^EA1vNmvk-8Y1;dR^it8{;wB_i1gI z6Z0Q(&nflG3^8sT(@;}4%Ig34wvr{J3EX=h9R>sWAPKc;;iLgXv=TXp$}7=#PRXTv z7Ut<|%cb2^eFFMoxwM)e(RXshL@2f$o4iXIdRsEt7Ive$Fauw2Qy>TZfN9}>U0Z5OMzSD>@$PP)xs^E5p3K`_oQXI}ojF*k z6Sv{A(Eyj)8NbO7D!Vlv2?McqG}mGKyl(;phXJ4e`h$aASO87SpdzV>%>-0g9Q}0c z7WjPj{tVdCD668fACwqK9uU~ez(3;b$jX7Ymq+ILkX-o% zN|bS1AC$C-DtYA3~Xpl@Rgn&Ny zC>tJAseuj5e(QNf8gYuHl;FQVk1)dzWKmOE`0;}PF}L8$(F1bwkA(%szIj7qV^!y# z>%j2u>%M0t1TwfdhxNJShgYWUk3OCEVL>WuU*c0r6?_>I#B^TNm8 z^i}WONGZiEU5EkG5CrSx+%B4isgmB9wqnQ{R=bw2^r@_sy}02!)tA77GTiLW6&%kl3E={;kkGihu?T2xH${zxUU_K>ApFG76#x?goW|hP$DH zr`ojltq@EWHTJHf?_yk!>OP!$_qR`t{8s<%A^6V679KcC-s!Cv9MZHpNj3+U!uQcGp{p|T+>~V$z5VCQo4q*NxQbEf%Agoba0923j05;tZToLr%dja;m zyB@<_p15vf3(f1~>Ae{#i5Y{sE@mh%doA;>H}B$FluD;ZwYi=FF3Q^t)rEXT1 z%D^u!10-BLbkQ_~8ucP=wd=wlGUd-}_jItc0= z`4Xd=baAY;5vnFb!FP~xy${ZDk(~Kv)&L0*--ix5&)6to?@; z8>gm1Ubk<<+G};*q+nr!p$kXNqs=~th5!N8Pc$LmcM74(&Kql)U#$v#N1GO@2SEaJ z@yFBIUj6r1T8turmakF<&gX9u|6m=R;`z^4fmQ>n3Qz3*{YF3;{oKwcAGS{8(YHuC1L!10=jCl4QzHFefJj^B|g3@3_A)JrZCg9=;^;Jfodmh*HP9t^p7|Zn#x*pOw*w z+nlGC1eT$xkp)GER0ToMN{0fXKJ$x z3tswqVb`zggHQW&(yb+=1gt(2u%dJHM)`;vq~3fyB2b2-Jd>1{K4yASb2zNZ+XwJ( zHVd0t-XL35y{5a|Uk4cm$}yuj_6mHs5${+*Vq!e$Rx>hb@XsF=Nu%G|1rcnBQ>g6z z{o{DFBYG5{1w_JVl4s&2_>~U=tlrGb!=TF#q4qBH3C7)AHZlB|?OEhwQar$Ypq8Br z1kmMbc$Zi3B@A!eqG~k)MhtcX}+FPYb z6aM>mXZW@4S9@2lFbtbLhh8$8!3>&amX@0X;`v}ZI|QM~mY6~cMFW(eqCkKCpWd)X z`N-*ZlgCfBb1j9eZ(I>VM@)XuUUjnPT! z=~<&fx+~X@`ueB1H$}@yGX;fi{&?O8Qs&6}uFK>6+89^+5E?m@MFLi;$XGT)%zryN z@7u~hiU+L(t-A2Xu*CD6=oFXXadmbkAQNR8oQgdg@4I)}+Uj~0u8mF+RIkbIMu<1U zs;Ulz%T{m1As@*TJ#TYlK{nH+i18}YY^RRUqd0glsW?iL1zuengiB{Odsc6$siKMS ztiJDpObh~E5^khc#9W*R#q^X$i<1Dng|EPh@VaZrlN#dQnM2BxBfr|72FyEi=cBd@mU`L^Uk4sQ-a-Q-iRq#Xa~2$)9yb0WhuEvBHX&t6Jc z*lOtH7jv^hlCMZf%QasN^6=elS2-ZK^BdXZMLSiaP&dpRCc-(e=F-oWEzCvWPM=c#bi>O2s5KN|#9^yOCa1cL~DMOcF>rKW(n&~mUNGgYd_jTGGzmGgr zYliZ^%%JrO^P@*^mHWfu3XT&YD69|}^eO4ZDrCeo`KeEJe~(Rg4wQOkJ=A^+ZjQaf zQdYVr;^Nr_CuJx(SRrc+6waGBN{9?rF%oZXI2K1rON-xPi1%nUCVTW{NZC1^sn>$* zvd90s!&DH0`Cc%1NSY$pL~~#^Iy5vC<&W5K**Q2;9TSQM-_Hyor?9X7z?07M9eUuu z0&An|H$D!X`?b27MTQY%$`3K|?x91$e<}2i!?(>k|F#JzZDhqk=oDCEniWCyupYl4 zi00>Oks2S&p$z1U@wp_Z}8@d#>snO(b#LIK9rNLBR5 zLCl0!B60jeRvarMisNfz@@#)wG-jegXpGr+D?D~m$8wy!A_t>XXX@^KF8{(Pv;^D3M~<7si`RTm*|&l z;1xJOX=I}^Wkw_lLgYX)0-JwrUcL0Y;lY$+Fvt0G5b2co zT|GRX+|!2)$G>##LKJh2^zI4JQf9tuV+zRGi@{CTkRmS=WZ8)OlHxF~bh|zYZqZf|SN}yl14;PosS>2%&MoLhM^amZ8OJTTIk@?- zVBa=P;S;Wi7}VAd+x|c<(=Bb2b1R;42L?XA7LfaT+*BJ5uAOdz6lg+K4C3Ta9K7Jy zY(|b(#hvR0H2S|Q6(hC7HQ>DMh?{mfPP%a;9*8B7MB=^Y`j>+lNEm9*vvXIXo3y}w zcge^+Ia2J0&dr+)abWB$5^Ds;Wc+RscoMBPc_G?F4p4{2Ls7(!qZi4j>gg|!w3~Wf zH=i-oI;{Z?A6)o9dT$s-M$a$=`*{y%EZ6SKuu)-geZ^Kae@-2l{z!kC*c;0X!~lyA z^Io8xA9$Pdl1jq0M#i25PCny_xNMHPzaHqvFcStJVXjppMIn>=?n&ch1kFt;ny^bL zf)W!aP^C`;ag~whu5r+gj*U8=xokQ5rs(u@@Es%uieRx9v=G(mnj8~C|!?c77hOC4yQrD9*h*BT0dSn5+1} zG>%)6`$AS9zoNMY%)d6DiWxv)L@d~d50q4I z{omaP^DOMYoZF3Iz83wDit3n} zn!3N~viZGDYmzHvX$CJ`^0|fP&tFS-xCm28eRl?Eab~E*!XW?L0)!{Od1D+bsE?QX zL2%j#U6!DAvZbV-kRNvv)W*LG*V_wU70&*oSHqeJ)!J~QNH}_ zM;v&N{v&?z{RW7o^lrimc#ubJ)L!h-H8!6uCBl?Y@UXV2IX>%<=lLx*^A&lUm0+px zd~+w7jFl>Zo5R^@w>vA5M~(vwGOa+G-*@z%mHr)_u#S%qYjgH&5;(;Wc!`9B1c5*x zWzmQOjSCC%mN|eYY1sRWB+`T5KB%O!vM+}!P(-F^c)CI>btZ`C)IuK;{aJl!!9&w? z2$^GZ--<>3x+WpMA1_~YBiWOK*!XT&5i19==C7S8d@ZV4o6kUKSjZ?{07VB_29Yf{#DWaJ&C>MzIddZZ(#mpM z(0;$gDIai2WC#=p+$Be93@#xaEhM4$KYCO9rnnkV3USH2RYvFq9;K2Te=kEhHfG3p{Vhcmj@t zcq$%8DK-7_!cjl4%DpKfKD6Put_e5CFi@uca+ay#)8TUn)yc|r{z$5HB2f{7fWBzL zR6z2-KH10?xg+1WS#@LCIG7Z2RYP3~5K1Yj#t3re4ronQNhC<;lf9Vp{ZsWmdKSKf zlG=4^dH44MNM4cS#`>@SdE8xXwvVJlQuTgqbN6>O5;4c;y6pHBwnp2Vz>^>3O~=j6 z0b1WZB1-~)GzvJ=XtK#@YuG*M74qmyD+zMeGlr{+D5T45ONo^(vb-fPdyGncJ_YX<~Bw(Xf5EppDgm5pZ8Zyj1PRuW7paq0s<1(``$vpnAyeQOYqD8fWXcC|w|p zGpzkJe^Ncf{9d$-a_qr7@h~gY-=xm&rGywRHhv6V=ih@`h|gTZoAmbfqB%H(2}hK5@vt&!e*2bO zT3XuRlo@cQu&T=Tli#$mQ0Y`lmrV)M7dP2GJuHTZEMJB5rQS}BFBtb}gq2u1he*?? z*99~WC-&fAF#2nKxBp+NZYcKc>)C8OT_<^x?UA4Y4yZ10IC>0D&;adazV-2SxGj7S zKO19@qmDyf#EBOl&ei_v#%91{JCc6o)4{}o2x@ZScB?{xRJz)gEgBqFnI_dpM6i}2^5%UXG!SMbZy4NnFRG+OA1M%R9 znWSl7Ndw^44S8x{eU}P*#(N|X{l(AB%+Swgo&8uTJl2{nLx)

+q{-@8thS*>}fN z-S_`DC?h#mN`+%(reTz%Q;n{@%Cl>%M<~6#dcX@%g;RYd&Ak=b?mCjO+U-#gK<)**s%kqg>1M`ovBb&=d3Z zd4Tnt#6CB`FU+S%FjN7oPpSOD-=Asjos0q}WKW8)7m8V?8E1jI;8A?!>Z-EPr_ku| z&j)K;#fMR7er3#dd)U7)+A;?u1)7we5BL;NCFP?itigOgEe*{DB*6#BrRqV>Ruq8| z&G-Jlxp%=Di=M$0Cw=hlOGn;4T z(^oc3uBytg1>+cgnVfsBNP|1|?+G)$b1&`J|;yeBi!vFtKrP2$R5rQpO5adtg-L zU@%O~83ptW|NSy{Mh*F_4(Z1~%N>tCyRioW^B1P66T<4z4y&}xQ#0+rNLv42te{aQ zCGEZ1eENhC)JW;ze8a$K*H4<3L_i%3QYw!>-+J=x_@kG=1TOYExnB&FH}-w zzb}dR3=(}-*YvZVz3vs8rwUrlTc&6|BGMZlkQ)3!K|*`Zc9y`asOn_hB4pGaw@Cm^4uTT zu{HsqC$Uie6AX~Q=xKHPw*JSD3*GyAdT!7CR6h%}0MG+a12H^B4em+*`;i)2T9Lrl zM!|tl@1#ubbgiC8Cqq>e22j*)-ptD`^BAv&xh#t^;f(`dW1=XqcGFkKLQKaO7;{bP z^sPOUgGq`Cw{HApffe*Q)-Has)ESjuXyDM-mB;>d3C666ZU?eqaCYbvBn9 zjJJtsOTpRNva++3v8Pj+00z5|t*s9YH>o$UR=rF(v+Eb% z1?9-x#^!Gl!>T_|{|7izC)9YJJj_QMUf(KZ!iZ!SIHvLBbyBprGV9lyyuWqsw!5Nn<;@b$vsYEpQW%3s z?r&r7zgtj?(qjrwk79n0CeH;Bai%&C6e8X*?-U@c=sZz=(eLiqOBtV8?GnQ;w8)))K0hCleH#iL z(Kz8xkDqQFZnb~EyXh;LqY56y@I1B?!jf9dOSKW$B zqzFO?QJaqI^YH5$U@{>5j*j?eX(HH(YTkWKPFopyC}5#wVWF_BS^+O8!f?M#PfZ!Q z>VH<>_`G8OS-|i^oosQXNNI+);Q4_CUoNAPLP<;OkV~(6v%`+3rP38#9V`mwXBy1q zsq1T3^;&ryQ1kw0%5V4ELdlAiC(~U~*$^nkI3t$8MnF~22lV2}G_@v&DAHc)|EDWi z#kg;$8Vzf+ZuGtTvH6@eIxDN{hiz2P>s)?J87;WC7=ZgQkw?K1|DCxd3Ru1yDJnA; z%__3RdzYT*&C|y+wH~uHF4nM45FkuB1)P03r=nawVUfh=Bb@w8ftk6FbF0iHzc4bniG-B#3*aLZDckFWEOI`OhN|ELg=7YIUkvUBNzyd=>+W zZ#Fip{!|~R65kJLpn*p(b`?`P_=l6=d~NgnRLS#6kzq-uwB0hVtH{DeF3=lsP!=Zm z^D-${-alO zAUxlDG9lp-)^p2fnEjaJE#{hAnn#u$wWcd+X!*z@%*>OreI}+T=G|)C`PmnjpA`_A zPMpRsUJQ98?wEL2Qd-)~&d%-vA-DUzq4Oh>*~W7r17Zh&D&7^PI4)lD*$6zQh|}=W z`q})0DRX(ix_b&a#&s^!Vg0Qg(za7JfOOllPywy2htv3FEF1hUiW+|iY=j>6Tbi=y zo@Bvz3dc|S>!OWTT?nj%Az}2YAsH2>7vdeSjN`biMT=27v$DsJTM-vL8}$4WxwG>S z!2obQMkXeKG_Ae8oSY`lnu)J*VT9I((r9r@M`+C@WnU_rv(f$0?(-akEk5j3InFMN zG9Ot)4BGMd=KTnrJo6K&(XKonBWi-{8bR8wO2~c4_X;}X&bm$Q;bWYl=W_K4hN6=E zS{-s>_h`8eBZfWuifN+Qh-IQv5Xz921@nMW3~8IUXPIe)g&SSHqV@Hsfw_4?0xxl>_pOJR6PxSPTyG29>ln2`=U@dVh<0)JZzduZ`*RA4 z$8T0{^~NHKPcWs~p@q=5ly+=e!*KL-C1hQC@{)1!O0$=PaR#Dw5NT`bSgI2lJ!mGA z^`~tPPg+FPc$cO`LsNG?o-p- zQxIMMTtP$tNr&s4!n7T?2-^GA`in{VW~+=07SYzLktlN51_y>wJRZNo;I@6Q5eEZz zc_TABmy#TduU1J|4wfw;2sR?<)nYdrKQUgk`Q2-@nD(bI@S%mSUOlq1(4sr}NV+#K zw`xuJ121)|b$xUv@XXuyzYCDUHuKfj)p6Xy1s7PKnmC6Pp2fVnJ(xMF@JdEjWm2~D zM$$yiOSu{j(FP?$Q(FxhjDo3jJtq&Ef6q_MHc5(uF}v2*pHy-t??D+lRJg35$rBl8 zrP^~@iax{)8sPBe_|wOiryad-pvXPpgWfd-(CYcII?~gBeG3?OCxJ(@h>|lwI(@iU zIpjbmP`$l3jEo-9Y*Z_Wq!qfm65+P$hTul?%WK5Qp^NS=;@gya2a#hfl|{wvN`@Y$ z8Xd-;#fsD08w&TvoMMW)@Z>iqP?io(paCw^b~cD4b#BlAb9D1A-=M)Aha4wOFA&^5 zi?m&N$&Y4W@AsrCRWW?g=tp74Tnbe9^|oqj)6;V(?)n5U02c=b(}^5EUJO9E6SQgL zN{<#eFfTu9-A_0MuZN?J6Qq9p361S2+4i^PE0o;1i0eG zaRW=q=jAT(js+b{%&Xz@{WkPDjs*p9GK)~_fwFP1EH&oH%(Q|U(=W>%Qw9x7)m5hF zzPAV0eNrpZ>WlWodE9Q;SK)gSDg2T8|w%?M$Gkj*DY->6H~`!XCSuTz>L@+**IY&v$?b)paOEy<`(H_Sa^?lp7lYq5uDV zr?MaSB|Ss+acj1(14$9u(PvLKNHb#Cs+T^D9dhlv;?u}?;CII?Y*&I$rM+1F-LEAn zRT6zc1slk^_bo@lFmxCZZ=PU;v#!a%rwUYK#a5vr#I9mMpf+69~h2%A&o!XPzBhKtx5w4Vjf^ zlLYf?OtecHLTE5S)W0*rDfdw1odKjV<8xyym}9akMXL(rgtG|3oE)0%A~g#}X}>tq zmS%4sH-d@=b5foOa;V9XrW9isN&tbPwzjqdISi5xT@oTKyx=?M zuBSVY!r{37_W9vPGf&TogsiNB$?oC+J7x1Kug=@I?}&;^i&OENiwMif^+0jj@hno( zd(Ams+9?stAn!x`58oJCOi)-d%Fw%;y02Ubw>G7haP(}CeS{oZ`%B$SkRT&QgxF0C zNDJz_xBb5E@Vh1W9N{ve#Y*U!i0yM4A9*>-nOvt?;1FlYW!w4;6|V_K+4*0L^4Fdo zhlGR#V>2^53Zb9(+Gu*er#AHLtOG_#aA6Uk_nCIkrJ$R~ve{|lAw%)C+VFU}N0#e; zpvc3_bufT*C%eaZxZ47FCT^GZJ~oy#TT0zmKkHDowYUAf2(gn$4VUWvZ^DHi7;ZXD zpq%2-S*&0HDcV&VZQU)J6l^*Kh>neL_+QDYF$hWWgK5MQSk{lj=b;@7W#sixhaynj)Z^Xb&~6=;&zI7A)yTZ52qmbw}vRV?nAn_~qD2j|<9h;A%R;sJnU4 ztYmUtoirUr?w-vwu1|b+*}X~#Em!D;cZjtN|BwD-cpYwYGHc4?MC?4m> zO{fywbzVt2j6Boemf$kZsX?AeGW%LSyeh;P*$HqkeQwZV-Yc(g?u;G^Sv)%fY-e5Z zPF079$pCh;fvMIaU{nr#0;ZZXhnCHp-Ifojp}9JW*AKq++PB0O8@0ODHm8f7N+zKqj;OVCe^#QGSf2&sovh^zn1q zW+h*u#mo+%$S>wD4&Bk?#rvjzy7(Nsj-r+VXO%=kU;wsg90L zX>oK;M%37J(g?95WvjiX)xpEkf&G_U(dh7KJKM%WV%NgO#oo2dW%6byADzHjXNgBB zBI<$~GBY!|CG+q(hu4;eZDRsS2Akr3bn~aOKb2U^Gr3F;oH8?bi~dgs?Pk?*_|e56 z(p9)MM>EIhF$Un|T7*xILoBda_-eM)^X}wn7Qz&Vn3L>dIMzQ_zW0o(Zq2-BoCNuV z6kxfm=JcL8>ACP!RjYnnJ4wDLIAlY-wTQ1TVe?7Ff=57lW@ctyLe{zKj~E`cefxIg z{!_wJf(G2Z+t8-82Ul&3!7x?XfNeVAQec&ulstX|hT6NJE|pCre&pcb(DIGF(IQ#6 z&};fpFXCF4v*laOX@y^|tz4yJJ2jcIii_JQY|`mE3Hg-hT}WlLofQu#%6G(CiK8su z31@Cy=^zLd^-uez_puO|=`CJ4R|_AOL50eE-+EiLj*c*DD-NVF3QuA~k>x__>=&OA zJ~;jMK%HUL!|D6W4EZwN{*<1)Kf&>(KoSnGFVRULO^K3gNB=5 z?63sdA{}7eoB*Y`ze$DVjKa@&VzTRG7hCaB_QG5(Uv)giJ@<%c zdt&G8R_6wdFI+f?3L~aH>|?#Q;Gr!7+Uln-DUHMMtn0!^6C%yS(^ES>n%gkX;CuOS zqNhFR5=+^&y+EX}Rz2)nsKR^zQG=T__sZpCu0%?0$ZnJmbVLdnyweW?(9?b>zT-=q zi9uNvw|LR~#*;?CGL%bL#$t*AeT~KVIJ9-iNL&jZhP1k zstLB>N0CESQT*mDM;c5v`->ND;o8dwi@bD_vy)6vI1X2EQtuU44(w#71SJwHAeS)# zjRD$|*P*UhY}0`E3rcQoZssb~QdL#Gpg%L7`??rfGl-je#B0+t2sdaeZd|N{?uLj+ z{FT!-pYteD6?d9d>M%p%DgDntMY_*Gj}Ftt8pL|9aI`E~IT+K#8u1Qv5aYxwUY&VW zLX2IIb!Zy?Ixl%K4NNlhGF|6>)*>Txn(h?sdoC4lTw4hnt~s5peNv-jr0lB8bFkt3 zYy_RQqHZBZ%wBJ7m3Nf3v3Jhp4BXF3Nd-K?aYcDzkY^8i-n1|H}lSbN>()95{(!6*!?|LKWI#7Q(K(}nS zfs(0(V{JIU6p#dR6>7tr%qWLRmqcYMO=|720Dgz2aL-`9V0qQYM`$DT*~KmPwVQeT zJu=^)d)P<&n6~k#B+P`7_uMYXPu4&N~N+~U~>9Yv%m zc0ia;1wRn{E?PcJpseuO^WM9Iby1g5n(YD2T*mk}&BTLg%U>IPf!+2!QH``5+I#oK zepZN=uT7`byS{thLYs8I+2hl?Nc?1|YDP0EfTWH@J6zDaA|k8y+e}kg5eab1wR23` zkc&#X;eK%^<2K9l-Q*lVO*4d2!z89ucHLH622*TPvB^Q;Vm^oOw#Nl9Fs1{W~DntW^x zB${2~SVJitkwESBtj28qUxfHwWEed^9R1TEvRMjX?uZyxJOKfywlYL1YZT#Tn)O!U z=l=JM3P*Cd_!*T?dLO7R2vgWRV_)8D$Qnnm-jLW}4G`4U6k;9YF!IRRWm@P(kS~zW zO*|JZ4X#}~KyYK6Ya=iQ6}g!*{zAOfWQccUtz7|TGN;Fp;*cr*hb)o~6RZarZtMOP zg)a^_M(MWB+|Aso{6uGoclz;5pBK#__X zpE3}o^sV0!xtB&#!XA#x)Si#OlDN5}nfbo&dUxMMc1iCkFJ_(#24eE zy-wu+C2>A&aIc7n$RscT-0A?F2fIM(!mpRFz`ocCqmQ-*oVAlhCt!<|HpTDRj-7J| zwv04jAG#!y^|j3%XDXA${TJDzbJ_ifD*ewsTkD8y=9aJDtob*xKD<~+Ta{W9J{dwB z4trl&;mH+;a9Or@AD$Ia#Bsc?nR={H2Il~$|GcU2Np!G*bMvX252Tf2Mm$ud*OT$7 z&CZSa*7c4{g}&pt*gDM!)$-_;KrWvf6ul^l6HN4+y6CyoIZUZ4Tn-1EhoF|2&_^!_ zy_cY=*a7jxpw+>GSu!|2-Pz?u2Z~xnUxrDgfb&&2#2GQ-mSAssi zR4fr7{FG z6Obzpv4@qzIKxhu-q+Ra!5SzwwVRnA7G87F-X*uO&{B+TA#&}@5{NGJPeeT-dVwsO zLV%qDMLX!U1O8^Sb9nS(%VKJw(7RWgYvafZ%JuyWiO+&vkDJ!%w)=Q&Z7fexnyYTW zY?GrMcZ#1%G(c240zjt${5!Wm1g0AV(5{9Y5X+~aA@*CWb#`6Lj_);TQ4dx%jh?0; zn^rO#pKEKb3atsLJo-s|le*mOdeO26y&TxL@Cvl!&deQ3ipOM+G3s7?3X{LV2} z!d$@%@>R0%R$y8Df#|ugJAp^9p~xo?RmyLuW8uhQpQo)#U<;i3s?O=-P?0%FscA#3 z8a{&3)Kcbl>g|`-+R{Dcmofy!lvd`Gw!Tr?$`1#JoV|t2%{0!d#}hwD1f&L?l$%!A z{Gni&xiiKQ>E}3bm}9D>L6VRyKK8p9v@x{4`p1zk4QLK7I2! znP_K#6;kpZx<-TPDoNBddC{KITwPy9)jyb3_2P+Tk-P9Gd_ew1>qO#5qycQ{N3cQ{Y-h`B5)l?us^m};JL zouI*tLbo1?g5x#wj-EKU%IaFkmw3e#=7WGM;#liaMc>AkT%*NHvHtr0!S{Q= zFSny98b*gvhCn@W!Wh@}%KZ+F{(~gXly$swOJh6Kiw;!@f8j4zjxZ=sj7rQMx8sK! z%m`j;U3tBf;V_n89_?s%W3qU@ zhuxu#d?YJ2{4k0pR3M~pk~uPgUwfskFIIqqQc(`j{X)xh>K2b$)!-NQ$)i& z*C$IRA%efW`ie5%ObPN3$r5KJe-YJrD~A9OL)T(bE@0&0QNA$(eCG5^d3xm+ z#$gw-6Dr%92AS$XirV+}9JWlVQCh6F#NODiAxQ(L;jA-za9aU4R2MRV7KY*6020e1 z<8CfO@cb!<)^ArT+K&+^%?+pk{=K=uI(%NAPC`;Lj_4t=vk<1J^MGH;KPVemYfd~Z zgoNP4Am2T+0KrNX10CM|n9TwdXs6rsU~qGdn^l&c8k!4Oo^e-Hq(GrglDq@JcXnIa z$(5SlonK?l;T_zm!D0|>6l`0IAb1>Ko%4{KtomRPpD?>+B{LkMtv$*9?9Fw;!%(FF z&ekmjZZC}d$glbpM<}x?_ zVW#bZ9eNc_J{};*Xb`}e96axZttwM?rR3f+1>Cett`n|)2uauIR|a!XSPCD$kbmp} zm4sjFXp)@glKa`l@Nt*tcIZjjwu30c^Nzl_xwO@(7Ifs6lDT6$g!t{ErGbio1T^G( z+kQ*@`3}ed9m+W)u0%!;o8?)-WmNF}Zs)!#s;Q~DEY|QIWqIo>7eY)->@uY_H__<3 zSuojK*$o<0)p=OFFi`G_K9W9?nVl_0C7?|F`0?XRy(wz?Weq2NcT|>-Hz9K@kK6;A zF5)f33?hZiY6%|murb+ALpjNMX5CJ`ih_k)YLM>hD8#ndkC8Koh_sC30Yy)HS>ql$ zy|>I#OQT6UnC89ugKO&`tl2|E>$v88!h$*}f-~6`U-6}Z9*@-6h%i?j&ISF!(cju0K-9(Yi6$iy?Dt91lzj2{lx$!Ga z*TMw?3x!mmMQsKxU!0u6T2t7ISFdFK`NUpI|B`aUYqTf=j-NANU5ggQRcx!`5)sS$x` z5))+~o>kj92Ftcja|p-SS@jhf_Ijx{2OJ}oaYCWbv{ciN2Cy!44T%-;b@7|3f_9-(q&78T(3yKJi-p@|p7%X@p25+jMxfDpDa-PCO{PS;H; zrl<%ZMH}Ky*8uoAsS_9&IOSuadqU)6dQE(-%*&mdVrYMz(z@;q>O_~%zHZGj2~-#K z=B%u_&>MahJ<6dwXO9)JI%N$w(a`4v)K(=a4~4?*Ne`(UY1sPteXa zuo{EPF$ks2RCzc;7qQfbyqi(h3J`pS7Yl3x8q=3KIja}EwB3b*<$le)FLoZ)kC*dw z?MNRf2(50vwjbjYt7U*;Hx0sJ|G@>|4Mh)EWyap0(I-S!A2cz0k79)`%V=EINHL&I z@{L#?j`@wT=gL=vY%uOYk33V+1_@;tIB0X-l$F4F3671=zP}gyx5f>&>WAaE97Hu} ze(j)U4J~9%DhA^C4q$V%hw2t`MQ}~9?Z!pffK;F{jhA)59jn!xb3>>YZN^&6>P?9y ztK&~uYjhMw_Clln-EFt>;z;5%3Axs$rpq7~RRppt$LxW)hFA*rM!(vG@fd$6Nkq<49Zo=d%3(2Id>O7pKgqeK1i0# zg+7V4f#Fqm%^`$jw9YH`?QR~icV~d0G6_M@@xaL#JGJ_3Ov+j!Rl4!+{7YX?p~y3L zz_(xWpJw7l-`XE)u(MjgeD4?~zHJsnC7it@;LoK^4<=fH>aey7mPft+7#`wWeYN2G z&EJKDs!87R7Z*E+lRB3iy7yj*>79m3!b_tpJErt@%t~)2 z3mp`?j}=m^=Gi!+eHQCpMG=riiek?ycngSemHKR~<&Ca-Xf;XL>09M0WsMAt#=Y>8xLTgB)jDx(k4Q(FZXGceP*^5 zwhFlykC60#;Dn%o*3&H>DEw-pykX$w@;SGe=kPi}$91zIt%lQ6RJ0-eQwo;UFC9RF zW^PMrI8PMnY4M0F%)+<05DUqDIXxBj%Q-y|1p}H)LF>Zc&=9^Dm=tGZH-5p~L*6J} z4+ahzwUf?cus3(K47Wr^Nhu4MpVkN)^)AAe$$Jw)kMu-;5e!TqDvn!BEY3tah`*Xa zbrh1VB%0W~hhDD0v7}e--MuYCN2u+GFx50nuc)D?c-%0>L9Usda2tiI%PYFf1FmG^ zrFqfySK~~CkLyf7=ld9T0%SvPP;R>XonH9}l&zU>($KbdN$i|tZ>s=}fP=xpTiQ90J93uhOMSTwo{$%zo= zNfIPn(zKh@O6WINbygO$oEfq};-EWODexU@O!RyU@eFZPR9koS^bL3U$p`X30QUX9 zC=-;@Lb^2nZN`o~EX46-HVV{!ApsNGwA9#+uWgI(|k-1sfzE zkPP&-f!Uxxr=>gErT;!XdV#~-KqDR%89mos;$(@t=wTTkG5NqE-TNz%e`nE==etbe za!k@Pgcyyzu_9|9uFgjxB&8m|qThZ!pX`HNci6Nd`->78Kwo_#=+*>mzGkR+um})X(^T#l$eLDqxO5DCNw6!h zX@EGC!G}>f4#bN6`4?Cp6<$gbNCm={caqPXkERQvuUKDFFaVMSpXo~8J3S}_XMJW z>(>Spw2J_v5N69IA(j@z?_g@HTR+E7ry3(GCFwrXTo_JF>FVlwcgl-Sole7pODsLT z-F8*hp*(kR$D&=3qesI1uwKpj>CsA;N)c^!A;-~n38u79Y1*3m#=MI}txBVfJn(5+ zYU&GZcYhs3jy;Mq-f>GgI*!I`j$23yuT+Jbe#&-J<-d16t*KKnV z`bBKeKbI@EEziynA#f zjTAOO+FtR+nacIFo%;V=dnDT3-*z^GlwEUpa(tu6_1!VTsbj>oXvcAHo-x`0QkiH2 zd)z3X9{+*Oye070n@NXhpuU2c zsXo^Q+ZgxP4@4s~7rlHzp~bF-fQZUL7}@G($Eo9j->kicxAL12+jU$KHy{BmLIBGS zp#L2rN+~|!s3-s>)Ag94b%Fxg+RZ;$fRG3|VOyg}ki|O_XrSKW;Jyi|%$NrY=XvgE zs61WZbamzUjhhp^nW2C#a#`$JRy{m1?=9&%;T&SUlJ6Ca!Q(tUCbgi73Mb-kl~g$R zd;ZJ8H#BQk?qem4P0m+Jx{{R)%T-CC3z0V;0osRQJ`NZBc>(P7KdsYW9x!mgu8f$; zp334uwFlk@P=*C*>1iIJfI(<{!HZV5Xfi=T!fQg61BNc=;!Q@s3-r)c_g?9A-)?xU za8RJ@Vb6Ylm3$xxw`Q;Vhyt;~Vud(UWYJ>B3%Ii5t3hp065t-UUlD-9BOrf49V#k_g#Pw(+gAFiDIzoH??WPYf@!gK zs=)Pwl3}JQsgO*2nP;qq4va=h%uFkwDq!6&i7VpDd;L`rxHJI1LZBdFEVCVk zd+623zuu-quBmA3Kr&)7I0>vNQlfEJ8kq?^Ieu{bchXI9pMVp~@%;P#U9$(Ehrhm3 zu)OU;-Fsu@hakwJTvpMBG?z6{>c?syt63750W!*))e0!rP>1~dpGO1^W42~5!|?Zm$r9g z5C=t{BU&e!GAowzz3-5|$dl3JAOQpaW<#=vC4PGKkrrr#wRoe*H<_wb--IHnFaEdF zVGex;U9@-UeCSa<9R9m+#>7^^W%W3@5-jLFVykeXU{)CVzaOT$zXKV}vUBI3KestM zZstP_&@eo9t&mdZVbhp!C0OY?Fq~e>SLRiaT zTKq+vs|pfKdNLPSS_UuFvcqp2&Ru)_49%{WtgxALc?B)oH&YP>mo!nc9Yx+thpAyB ztT~PYaBgZc3De?>DZ(@aszGGzEn9pfRUU59;bl96me!+~EEXxV-8#+kWl8+w4N3&C9Q_d#?k2 z&9i9oFe?H2!yZ0Y%X$j($ogI!!u$b2g6_-&M_1!nyIXAVrHh{m`e^n6ypv&f|8E}2 z{Sm@Rc5zi?=9kTvm32!;E2Nkpc$s*3(dQ84sJzXZg=8=D@(iowZGys zIDxlhx+X-ndzhPdUwCFhHx<>LK} z80)2!MNR_j4ptt}qLOcO+1Qp3BZa4~?^sV#g8>7zfEolY86N&{Z-V)6Y?UK7)Jw#g zmaF~lAkmX&O%-M)BtzuT+40u#vgkBVe+}K%W-ot16#0$NyZ`kL-)wzdc6%^b89CN7 zon^0vGc~kne368WhjRNuA!|K1npz%3{$7ClM|Aw({7)8)53mw?S2q`)Ac{rn;@y;teKtbeM{;OJ;bd z#doJXW&BCZVR{+SPZO^Qi8iQ0%U5(@C)H=THPnCQWgk#4EGy$fn&3iriv%z3U@2ztl-NNqc|bS>BVR@J*E)uAefqmooPdykBh6jtTkgNAQUvw?Ap4 zXlZFVs#p`L!ML&|h?XmIedzGal7q+4Mgnx4{s)Ey)qC*M##*!Id1;V#VDk$i!Tlzd*BKyGT08?(|hP+R=7D_ zf(g~w4J9jo_*5p;ti<+D(7WXQVe`*DgoHebn z@>j7{-RElDjBZmtY<(sFE=}kK_}DALiMZ{L`=FtS9IJEy>@IR_&YBSeg9LN#)k`B9 zVhtt}Na3D*Z(j6CCbq1;gA)@t%KH%qf@t_k;Qh+N#65raemQSU0>5;w??#Sgn$H37 z&+(;P{%yQ2n$6R{F7KHrD$*+sSKd<)+Rk49Pl*dVzW(ryAQ;Y+Hv<{lw?HwJs%k$n zv&;3U-OSh9{Qe|sX#CYe9q!qSeSoca1_xtAdQXMS709!28E!ngYQ-S0pZZbXi`*XQ zmx#sPbbO9tR5-o&cBCP4qKtJ8*YF-vE#7K`~gOBk% z`3A@1r+n4K(d}3BW(9bgt#3^w5=kg>HN*fo4>e?6%t-w_B1$?gNN^QA*Rtpjms(Ny zxS^E#D&z?|_+Y_2H{tCML-6`Ju@PJoU-p1mbDFjuUqlpvPvA6_Z)jU%_Fj!RwiZ3r zj?)Nwsxe^BkLGvnZ*qeV9`OFv>G@|6wLR!W9ORpsd@8eUrD};An}xS#*gt58% z0Q9@;FBmAX2}5G=+QaD4wcvPQ$Yae?!zcr4vP9R?%SG4@YUmGh6+q&* zE{4IWU^#Tu%Ex9e!0h*De5{W!gcnQQ8{LDlDx|zv@T}KwUW^l7ENKFUC3PgaR;NPR zTD@zP467V0PE{K>&LX1P*$K@C7abo5k=%YIT8XOy_KYKey-|f53)V{A^G4^%?a%!F zn5y`w2`+Qi4lQ{hU6ph^3^UJ4IIygZ#&TrxiPzI$;0>FCK88dX!y6U`MjutOa;ChY z`>g%;r*R*LvQpO%7EA(X@|U|{(&L<#-EgLZn)6X>XhR{S@U#6eoiTWqdi^X1WdRG_ zj_*>ZT;zcV^*O&=K<>PTT!zKbUJe7?*vJ?I(l!T0CNys%qO7e~wa`Ja2$#LbIK;JK zHw>FOOtV_jXg}wvImet+T5$!SqV}?=v7ImD%43ZMSYY6D8{W- z^S3^aRlv+~h$V(7GE+?7%J+P`?WXw6Q5ySYbyA}odQ!gW+ELEr!iI#W*AGK1w}6nW zRw76KQuh~>7C`sB4f?40zpmUlnDT4)fcxSaK>A{lg#!REDqjCYEs>el!s3CP1ozmt6R+ zRttUhByw!2@w7$UEBMx*a>WyecBMc}_b52z81HsR_L8Y%FhbAr2&t%Yvq+f*!%495 z#F3Sd2L8!%Z z_`fyZez$h`?S#B3m$8kqbl-g5ntI2x!LT|Y%cTk~uu!Zi`3?5{|CTqDK7BJ)C$KI6 zQ>pAP(iG6yaVGZlY*3_V;Q1qAc=UybGmNr}UsRL4Rjpt&%@(oe?)E#i6P)Cd8*Xh) zFJW>KEC{8lc~H_$BPSaIl{e|h`3}Y~WxfmQ8_%S^owrhi#iEoa1|TpAUY@-^z5R)P zhiNA=?2DcDClpks_My!nfl+|<2VS+kX(fq?d@oF?5F5N@0&9mI= zQQhwP9l7(CoeRA0mi8Va;9tDuh0`0FZ->l$CDBcT@pkhk9e#$EJ@RLT`_H2H_Jd1v ziKQbQ;R3}`=wW&)Jm*{`r)@DxH7RQ$*T^ zpd!VaO!}G~FgVkb!=^aAfeMb9n*}2+`)DQLAQ-eOUNfC8RD0kz%53g$R*Rdt!?KlG zfUp#RXtjRCnL=EZOg6s3V{79lgld>gTZ_f;0opN&c40g{+r#5 zhuXWhZ(%PKNCD)Mtbi6vGB_ALWZ<6(ndx}}g5zzTSx~$5L$z{ajS(KsIA}t6shq7X!@p-Dnz-iT@c9RNan9p^2 zY1$R+V@b+KJA@hv9?HyV=t{eN?GEntdjPMd>~*uk(pdVTV1*WN5M2QwN>gE4Sbe4+3FgwH`( zm_{Sn7&BD=*@`DPi1c%7;|=Fy_-ILw(-wY;`jjWn<>eb}H|1}R(jU>s$@EGy8Nmc$ z%rLYELN?a)G;N0BQRH74?*4)?;Id{%KOW|zR5557g)98uOhP}(QT0*#zvY;ef7XiM z9MQ^hm}bOWhZdKJ2%c4da9~}MFvLRGqq7S|-a&mAaS) z-)^YVVx+_SXU|ab8?S}VyHtB1RN8a-Cj0hhDq1OiC!xq05OA00v?v({vJGGFypJHP z@sO{)*2N8QV0Is4fW7^PuNwH%eQhuJAEpO(1@H!~I-dSA1`whSjd$sBK6g$a@mFba zLBj9yqX1_a^glYA$3kc$sXz#TJ^eG6vOiG>2o9_iU$fbX%(M(iTmApp)5opF-JIHJ zqY~3J6uHb*q^>^SeK%6LFZzox(Dm(v)shKull=l)d4|h->8Rg%`0f0`Z~s0GLU9oX z5qh|;tRT|Zfh^hf%SN08IyrPhLi9KbOf9Vze@-Xl3lJGz3FO>`7*lR;Vz~jViF8YU zI|4_UrB~S-Z~mL7svvJ(LXJ5X9EH0O;q$ugs^Mh|qLDu$ik%20R{XVJ8CNFXU1a69cwfuxMGei=-?*F|{;N|G5e z>BQ{%iwGYXKQ|LX^FbASVA9ow#NXVo6s9xD2%|Iv;OitEjG-XMefLT%4zD2A7#+pl zbsJ^KZH=vJdUjD(`ZzqU^ayr|#m|glMs|J>|4jsUk_}d`kBguK94qN|U1EJoz9D(tC16Qx72Ha26E#c9@>XbrwQu9hyAyMJ(|sBN!9e$EQ7m-iSedRKhm^J??*Z(~6?hI)G_~ zmQ4b75z%UsE~_$Vty?&UAl!BcP$fkbgv8?J3LYTX>ic*m7Q`rCV#h!X6D|by^|&g2 z1HC}Lj6-BJAm%QoA&!|wv!GZ;}-GQ~F@BRmE?)~MOk&wcSm(La#FjmBSuBmzjaEG<_? zPrIelVjf7AhAF}rdGP7KY`;8JOhdt-=6e{^B)?7^VZ@w<2zHG!yV=U-FW5;bB%GH$ za2M9^TjP!;E{F^}V zW4fZX(sdd=Bv_EnTNI{|)0+7XIhMRYTa+DV_PN>1$Bh>AWB$mnL7zD|tDeQd9uOei zWb}Z^<_-YP0OM^Ps6QOu70wgR6UGxY@Fkc{gaUe$y83aSh>#Gi-?63|y0T9X^eB`} zGA?x#p5N*0Wur6t-Mq1a^Z>$7%6o0$giG3JRs~dVdmJwJ{wC%cc=)jD&US)(_5VKDnHlLe^e2 zK+G6;dsj+INOZh?u-5_LjK~8*qJV~sFls7{Af^-w?T=bp2*1)Ky!HNC(Ow~dCs{A- z5l>5BmW9uY&C@^8{o2`IVl_4-sk0 z?S&Ci)W-sj1#nlENQq^o+ZhNMZmiTO?2jjAvt}16m!MgUJrXzhKL7{EyP`DKSB2ui zhR1Xh_cduwzXV1AM6;Arul-O=68`&^{=iz6p0dK|7XY(NeGW7EjW4Mc*6Se!kzCJ> zA;+v}Fk(_?MS6%)3zTY>ZI7$~!TA?0rgp93-bOhK9M*)9*{-eAemCx`8)(s?DB}Tm zwfb&sca?Vq%$a&3g&*;uVREg{YyOcpz#A_M$(tY6csUMi!bLE6EiO9D4*k5WbnPaM zc%LN-pFyMC@}*LXNS{r<*S#%9N%s2Rnw(pKfXNLXnohaIMual408+wV8_ao+VaCG^=Zhz8CD zbvjHn@E!4zJy3>cI-q27X1Ftil2bj(J~qAN#Qb&6#q^_&=1(ks^j!dbKZg!pr+KNG z-LiuMs2*s{Mn&!l&FNc!N%DtoX@ZiSO8TG#M;XBdEII78%?u9RIUT0@JV47^1q|ra-2wn5LN>R(dag`rmL$PS~|55km@ldyY z|8PkQk!*!R8oMlIEo+RiFJp_B*Yn@=$N73WX})uOkK_34pZEI%iIas%G^bquH4O!f-ooU0J<;TA zo>{-YRjhBXI&%?Tzs0ZsZP<>N6T=K-waH@X3yC|cRDc;)p4!=}-T@NmmG625jG1c$ zy&jL-BMD|xKrBYja}`1KTWB%sK*!&noPr(ZH)2?fN&I2qNw*wH6k*dO9>;xev(eyn-6rd_6ltAqYv@GkPrwgtX)hV$i zNT2S{V!4GEo`iJcn>&XyB~Nb1D_JpYBN{g)pS7Uc#t-1*n_`_09$Xg+6!pD+;G;UY zsgF^d#aqW+@E!4ifS?|{ts7}btAYRl;N)dam1al000zhqp&Yy+j|R*>DSHwao4*AE zERV^A694M6RzMKHHJK4P&_>^|GCSZ_{bxjc`e^lf} zmpwI6J?iN9+=RSqUAd-%9Oms#>|c7u1D$$r=+-6D+cD(x#uzS(E?{aUzxV{($?mC%RRZeGlHojah8C(9o_fSq4c{9qlc z4-x)3@l-{9Y77U#>sd}>;t~0PjS?4}R-Z!k_0GTOmXx=BlVXJh1rO+S&<`wIy}KED zWffs^q{T4v>mJA$Z`a_ae~8^MW9Y5g-nhC67j+6c;#c|xwq?+qqW#%?Gr%_zR_(f` z;8wYu=wi%*;8K#xIxas#XzeDgJ&X63M5pr?mjJjtTuXF%bq*Sq0l3@xqtegkTRGx! zv;xUd{wGQ=>rdSxlUDw4?A(;OYs$2I*R}nM=qti#R$l-y!r=KTy@NUzlUogFR89N? zZ3?qdP~VRE5$P0*3VskK82Y5X9wYBfHZgac1$=Res71-=?aAux#?;&79CueiJ$XJ_ zjNj@Eed7#(lN5ECT#_{%$_y0d(QCP@FJHba0kSH1#{gij2NXULn05{quoQ+3l)M-~ zfVEk~3h*JJH~Ay6IYqWlarFoD(6_msA5(r-e#oo=#3z|EU2RQO^y}BzXQ1zLOXbtq z$_tDTzH%OA_?NI@7gZ4>(PifKLvJnuuJ@YT0jyr)lLwBONGCn?NCDcNb_`-onQGB* z?O2G6Fu|XU8{%p-)nMugK-o_=164d{S*Fh8FutYKTcc0l1w}I+Cnw+TZ@HRN3G@J8 zSqLTX(_}&gUoBU37fg8l`F6S!{UEU%p-X}G^sn2iK|8V^1x0s!o~Q6X{s}P0!YLbI zk-^(poHW$gb`cR4u4va`R{GiBe?{JBhSpd=*{g%M2WHAKDaqb846?;FHZ~rF*JJ%M z$NH~E2%iRuStyj)3+;+}H)|ZAWg<6$!%X#&{iC5o{@1`YG7zvDa}Ina9UW@n z5cS5rsq#KzgV?S!ffa#|QhmnTG72B1odl{Co}&yDV$-6-MAx@R2rFP2ua!m6V%KT0 z-?<210mtw~J1(@L`DXlKZ%NKT(R?I^AjEP`y#M-v4P1w>uDu)haR2>0n1qfIV=hFhE94~>@;*MDYsvq00?VynV-U6HGNuOnp{b_*Ut$V3>;cl!R7s5?=m58 zB>>~~I{s3XuG@dsSB9d*dl8~Kh^p*#45|ZP`aZUCn|K(f4A7@C`ni`DrC+%Mmav^> zqM_N-K!sL>kZay-Dqvqmv{J}3puGxzlOX6mvG`|eHi!YM%d8DFY0`>8Kvz))%qtKK zvaQ`hRX+bL<$rA3VyoihqrlOIDjURF8OhkaFVa$Kv`YAe<&XKIPtCULm$i8Nz0JMs& z1nB?DH;1^30L}jf_2zu$Hr_{64-<{M^Bhj+@}E0zS>^v0c>T;_SoXv@BY@SW za>BH*KzP@beC5ovj2_*mavr||3dpGq>7%2Le!%CU#?q+<(`NwcB9vEJ^&4T-=i*xI z`*nC3)cAKD@k*AH96Y$DoLg%D>et&|q9K*uv@~@XK$BXSW1v}G^b-rg5n}kPj2TOX z(9HEeL?%LgS59;ImMpyEwqDzxq@V&s1M%M}d69beK_bC#y0qtINePK7aLw)=xWje4 z@mUXme&ZJUa_bfG+wbpY_Z__-!xbUvio3DJTaP%`hzyJy~$NoOn(FIalXj zvl9NMj|vkm%O@YJ!ShYy(p9-ZC-$XT}Ndv6;0-AhQ@hfSdaI6Gs;Mit+&8#siTU0P$KA%l&hJAEsU zI~F-PugVu28@K(y_fw5F`qTdVL1GUQg2L@T1=wi6Z1d+mmzJlE8FZ5g9UmX2<{c=t z{}d@fpBW*bXX(;;#wYR;sHJEB6}rB@to25p1e9U>+E@T#sD66yY1c?o3-WB$CSX(m zF8hODFIj3Xx#+zbofD(637tLT z&dh|9G*u_v_5O=WVOaMBH4lSYGH+y=^U;#asfmfkyhfkf%Y5gDg4Z6#LzI}9oQwm% zZ7^9EEcb<^`yJqByBpoND8Fv?M$FhtQ9!Z0vQkkO?ee7(0QO0V!+yVl_v;306>0W( zca$bB?T=Hs2N&^+P{2Od+o>{p^Ts{L16V&IJObQs{@f&1UsP2j6$NE>e52_~dR{YYLQZH}a%uWC@5*06q_WOV}S@jYmLL}8QmzI{? zWB5ZkMO#-Pm0E$b|isNXI^sgge#oeOp!l*RY$txZ<|pO8l**Tv@GjwA0kvby5;%`U>sTlHPQTwcT8qx+o7Eao=KrVgfQL2Rlt6uzRXF zq_bClK}%Hc>cq0oFa}?7Ecd;@zp71%ZJ-Bgk@1Eu+fZUZ7r_YO4?Ov`fYG}^ym7Ur zBPT>qnG`~t52`BO?;s8}G(Y2?^8+W+C#9^WDxED_@|&|CS|q2`Ju+H)UViPnJvc5m z52({BRa`e#R}Xa&QqVgxk!;Y*5`DuH$dEg#0ycH~VE$p<1!H3eAXz=cgX=AE8N7u! ze68Ps=N15$^~m`Hf_UrHwPb>kNn}>`(k&qIWJw$`zHjugBTGv_v^<+{3zV1YVfkA_ zA+C#KClYfo3C&XExVGR%|(dTbbD9@eHqUPuo2f|r50ibkC2`5?s zd*TRSMFxBD9a1bjxwE#+g!tbtoMh{v(?vS#>HF{1Ix3xp?M$~I*e?MnY=}_i_}Eo@ zEi7C~oe{)Ap*<}G!2FMbdxaF0+lLF9m7ZrkJ;bHJCQ})wWMFOmvu06VUj9{1&KG2i z6tj76iT?n?d=1Fh0G_WlKIPX#i0}9sHz{})!K@v&*~)0I&IMnfe&w+RZum2& z3bM5XU7uWhqEz&I$IVcp!>P^51&IFAuxZ7BO4KzL!l~=53EdRZ6}mRzIk91aAom82 zvarzIYIRm_Sbzikw}~JQRzmVSK&thZZAg*yohE-S8RN_MPzTzxW$D?%Z(&%I z71NKxiK1MDn_M}vR-YAo7SJisJ>qgvN!4Tk*vMmAJ)ar`_<(oq)AQIz!}bS65%tX( zL-#1$+?@--S+QL5W6aoWvLXVK;iSa|Ha>Z3*BMnj{b8r|=dHB{)2Ew$yx-%0_{rL) z=sRbHwaN1bhaA?tu6n(xa<#f5Ek%w-NcqHG)dMt@M~&6!MeoG;)tM$Hgn8E^FH!!;oU-P{QokV|97$(e^3F9}ihd$+zGm`AMZt z;v{^e?o-Bu7lT2(?$o5Qg7Dps5L{^fy8_iI$B%cg&F%W%pNtnvJ$ zyjeHhRm(I5JAZSw7-AlQCX`XS*)Lqn_wBA-D!a7M=S|QqXnI~=84 zv|x}l)3}ZbDp#C2?STppJe2iyucFKUV+;OzvbzZt8#X1Lu~eP|_Ce%3K2nl!q$!mx zU&#;qMjs{YLBgNQed@2a8+k}!hvF3)lasIq5s|H0|Hpp*W0}_mij}Wz=c1bNx4BTl zi8C{5c1#%3ORYh&v#M24%gM*1QAX|fh1HY6Rd75TXXe>ySmN(sOnCD8A13~NI{$bY zoS6n+;T{yuNn_2wb7t;`-?6&JSvO4#>Ej9)ip&vB^pqy-%3<_2IlCT4#ngruw$1eW z%0xuNyECz-J*4zNrf)^rlGpk_MGHm~=?62-D;T(FnI#BA9sWHZzZ=RrqxmwN(d&K> zRBWg?umMljqN!=M;Boi6X0&uDymIh!rZt>f9OI{GqSMd*!`87F6Ei6Se_7nY#o-Qr zHms~YS`Vj4uHx9D=fK8T*QE})A8^p{I*x73b#Q9NVTjyD6!Vv)!u0oO^Mc~+K__aY}<*5?_wYEpWZ$-SDEfc>>Ab&`bCSV@9)v?D1vp9_Wi*%UEYF%ng$SLBR7i^}D= z?a3XZ0UJ0^CP~KE6e2HwFoipw($UZx*dE1@2HCL&8qDNSpKbwE?yFM1rj)Zn9N4JZ z$i>)Be5AI!OI6s;S>L;LC*VctoCDERF`Dpr#;aYP6d&54zYB$HNNt%@f3HC@ZV$P= zJ-wWZlU8nyB1<^N5FL6_{G1iw$Yad1Qy)-=sjHxH=HLF@r=UBSoPDX@ z$ydn`Qlt4vi|x_mDFrr*hVqcN9xheK2#nr~>n(WI0~}cT9nU%H@bmB>wzfZbYK{kR z&r#jEW4!-QWrQ40?6X}Ml9``;U2?NuEwfv}!Pe6#VGitYgKYR)NeM#i!GS6j6m16w zwjp13wapiVhoeAW5k*7y_i_E73hu8XZ=0iD;YE>E%!{x1_ztB_))lu$_YQn#wP+{} zIopn}nE&%BJcd{n=HyYr41&(VVHlOwxVj0XzrflqaQQZuSYj7oJQg5d1C})1cN0013lzNm?*hwb1B)aF?DXuae|>D+y5AP~JjWb}Z=03(-XJ zI!So%qiM%_?lMaA{9tiX3mCgQTo?VN1>f7qLmC3(+9UiMeWE3AxG%y~&YU-{G^ zDyrv#VZo3@|GzA#ucj!2s-*5n->PLmZ zAH`%tgQ_AKcC21APm{hRLI1$OyKn5+T_il&3{z!j3j(<}-`~KBDidy_w#%9u<=d!h z8ze8#-W+XG||M${i?>s2PB^}E>FKXHQ@7{QQGr{<+i5X4i3#)Mpm5Pjp(6W z4Uc&ZdEqP6b6s0aDJeO+)SB)5dl?QpALd5N6bB#l^n^7$XDoGU?ZL3vi|FpH+j+-yBOHg5>?va3s_&(E80+6u!I~H=;3)Ktj+blW7|h!hyvC&%r|Ez zqPeew{q3!5hI{KeF!hA7>!VZ-mD_*qK_z04wg~fkX;?t!O^V2U#R-(O=smu+?_Hec zsp9O|f>$(g8j`rx)UAnO$*fPA>qD>ozvJyIt4>;CIk5Lk6J|x25P9z^8xs!xdrK(z zHY&k1$7^6*525=LJC;EMMK0=zl_FfP#}9E~je7|+tUG450@|qD8f6xP|NT1Mf7$xj zULbAw#u2NlD)<$88}R3p{AY*w?7DwDqt%&nKQ}Idk95Jbsz4KtKhmTxHII59qW5eLDX8My+MKeJ;+#A@;9G8*u>>$J^#Q<*XuO@DDmX8#n2_YQ1d zyGtDjnm*TjdHTCHCF}lysRnNkOMD|WpazGcY%f%)LDDewcV1qy0U=54_#@i(gZO%u^&v2$5Iqa6{9=x zv{2eQkIKymq)Kf%4Zg`a%$>kXR57ZsL0DYJ;(vS3|L`ZkDl zieK(OE$#0gZ4XetW-PU9sG6%0NekDo-aa|iTdX{7cl!+CNcij}cmJfiXrf0m2Ue_E zmfCy0`y>Pr!m6Tvu}o};>wjKlLGFr*8 z#rB_1gMEl$okt#NR>K{Ut#AS!w0nMyKpm+D+bQQg{eYA*NmiDQ)Z*0$8dm^@|u)Zx$ECxtk_YO|^8|NE8I z;BOyBkrTvg&E8264(*L5=CxX}T#v-CNP_bpqq7>pF@G&>bH0RXJA!C8k(7d5&;xYRt z$_}bYBpIcC0UX;qH+-BkS{f~Vo5>Wh;#bmV^r@D$VZqi{TZSpdXV2Akp~&LkR$pFD zE_5){kn^20OgfVzed$(9Z5_mE5;+0>Np0k4B6JLe0K;15Yyb6y!W$!B3nBa6*4xWb zUmnZQtk?Y9*d;KE8g~a&8NJEf@Dsrnpyy2wmi{_T!LL0L+{6#~iCeXAV9Oz_WE(_zVn_$}++6iW0y(j~=6i`M%Wxj1X*qt6 zC>=r!8V;cVgA#6u>@B()G)^WTdiaFVd!^Rh3f$B4@EfkYd4VkwiU`)Vz z@Gw^NRUl68F_e$B-Ensu&1Oa!-@nizzC0l)A0w8A5kRb#=D` z+M7%6n*;h;>4{6sPH53+NkSIJ0T++Nj2P{ZnQ}4M&8{n@x7wJ9N}%V+lHPF?73PBj z*J6v)xnlXjkbbX^COV__q9xPC>hO`F?AT5S!UG7A72e+7q54i}*3u2d*+96~%X{jx z`3$8HH!aVj)r^w4po;7rbR;0I|(yF6jKE}hd*OT4xoP*x*lLirVuWwOA3Ut zKeW7uG0Db|4ttGQ9!moO815-@5)3|%coHo90~BpJtqzFYtlLX^MU06C+(XvhK9FX& zQwJ15=A6-`p+d|M?jn`kT+=+Gi&g%&%n^p_B71dbm{c`v*$_UOHCjvl%q)%s`9ocA zolXTu!bKCdpVy+KLQVwNQI}mtkz>R=VeIY}d<)+|5_f)dfnl_Y?S z8AdL^)LsySN69M+eC{l;#~QnTH&Q&RUvO00^P`UkBg(7jQp#7^u&%E}ccaJ)aw6n6 zawuH)qcRC^`>S?ZaJe8UQCK<~v5F)ml+P%M2j4|}2lG(B3QCW*#Ur;MhM&X?=XTr) zJv}{6%Z5Fgrc}^_PgayhBFGzXbfqJG#}GHnp{Kw}Aox7U7>AVbfFLDPM5_fyR}_># z`3wSy0oVtt5JUxV%K91A;FGf$!9)FQgBR%MhJ)Guy!M`&76U7_Wu2Lzr~!DGzP-*v zDQq9XkSYpYHh;=Nyzk68z$A9g)%wj;^sRP$^6^nywB8Mus{cU+tG$@*<1?&=b0e3} zDBPei0lj8o>QNy{=|Z%*QRg!Q-YA{>lXl0?CN%gP!H)@v53aSXZOK5k#M+SS@&$y)_|Co zr-Q;37W&)RA%-+7R}o7?rwh)dtx}aRrFBL^R*9??O)MWqwAgg7--+m}E{BdjJtupK z3C@x`hPd7-@K*wCT$#1CIvU=Y{V=z7IuUj^(qb{SoT5}54}SG-C0t#?(PebA7v>T0Iq4f!<()tz%&0vzi#ir=2-!Kr--?oQWJ!T7 zHJ*LRV9@Z|t}nIls*jU%ge0LwKC=ODaYh!*(CO38=v#3_`yC^N%^M(QBWV9q!FQ2# zs`DqmUW!t2pE{m#d1hmd4nyh&`C8q!Mv;e%NB2b&_p)J)|7X>6-e}d8i5LW5wrmrg z8cdyw1R26?{SlP*#o3&yk1+ocGOPpN4|g}PV{Pov(z2l=0T|+!J@u6sZyXq+*X-Rm-5OUq_W0oXwR=`oXu(m{(Df zgtFOQ?&OmXJPlGg#SjJjOqTCdGJ6a$N31y;(hzBp0yc=opu;op8JfV-)nDCewWM@2 z=wjh!c!IF;g5$ou^ubeqqZe<{1i>1J?SFQ~YHm8BE2~c{fRtS^_*Pbo+Wi48?u)Z_ z7oReg`mVf?&_F?Y;2X$7)&Eu&c1IJ(2#m|Br>>Sy9qOwhpC+^@F`{rc&k$l$6wfw2 z8SU_gcuh`kJ>MRQ*D#8HiA-|BpfsPKT%f0PK zqZ~|(KcY*oGp0|o=%h=hH$RGP{*!HMX=Y|NJ~icqG5J95@8nFngCW)U`K>F3uE!g&?2%#+fMkltbTIO7v-k^X*fewH|Oq5}j3Hkhp*v z&fMlgHg^X;w=;6FP~#DN3x3FZjETsN2#2v5R}LkSU^__Fw>rfjj$tNe2D#E*1>-L& zzY@CjY=2$t*@v|rp8Vm)1RC#+p^3o|IgwoNOP<1_?a?@w*esLe<6We=`bQX&xnSq) zOK9uGGrEwju zB{?gI#x^cuW8*@<8!bLR+a*)Qmw>7A`On|-+q-HJ|JQa-YjVkkJ3mWA{Es-PyZ$JX z_4+s_f9_pvhMJ;4HwS(8SxNqox9$EG5}x+vhn=*VpLKI(>0hHa9&RFPO}7`;mPGE| zllQ<;%LUz)^G0!s5ekRW1mFI*)Z}n51zV&94T@%^{%%e1y?(pCnJqiNQ&%Bdm!>Wm z>8I`*xfJ03ebF}i1d=Lb{dMWe-`DdKuDe;A)< zc1Y%8R><(s>wq81IE(jjCX;X8ytnaps61u_@&DGbxv4O4y>Pxi{yd`Qh=tv#GtCj8 z=;Y>V-$_{g%!9o-BY*hj)Qqm=i?S*50ZX5oAA@|trr+Im)Sbp>ky_+DaWPSXLk4Tq257hnW0DJyts3thpsl$$Jcv+!tzH20yMF*S%gh?$~-! z|EnZ-C4mELsx}e(-)>RxfBQ-lBvDW-IQ60tkAG>oQy6JfPM+NO%CD}%S#!DR5&3}p zbM{hkvG3tLUAKPOp2;`QpN-~|th-obixG`#To%Qz#^#Rcp>k8NwmNVNf#9RmySKE{ zTfpV%x)^%v4MnAiIq>_U(I(vM*p_>CZc-^X8~oITn|o82J#s?fypDktW~%~ z$-6Y%PknPHYBdJA_e-W4YhHEvCf#m*%rc+&{(RQ3^6Gu>{w2q=j&Si|ZPOQ5yU101 z?mdg;!+a7*n5Z^9@{X?!i9a~XaEhlXJzv6+`rObN|EBxBrL#4IPq3S?*mJdI&iWo{ zIS=1zknXzJVWcu3k#lg`<89ULpHKhe9%${+U^Tr;!8NSD3y-G3s@yzi2l9sDo z{fq84aFs>7dVc&c+3(b~1I?|=t@UP+NX7n*_v{m$UkBhVG=9wmQ_}qRA&WOYb&f4b z$b0TrM%xi5mC`u3Lol8uC%Vlnw)eU;0YhJEW1@j#D_jy}* zaFtQqUX^B~h2%-@oC*$!ROn5^!cPVlL0BuDxU_Yh^1?8XtLi#XI)2s!i2+L^DwJtR zLhya(Udb^~^t?3F9TB0^HT69qjWjES3D>DTs4+*ZqRvJj#o*w+3JMm63$a)_o(r|M z++K8l8Z{Z*1(RnNo`#E33Q~9tC1FExS;7kUIgqXmwS>V8P1{p&lrek()&FK;jIc0i zRngmrx{wT@D`Ccf${Wc5>Wj40QOp1`5!sP5H)iG1qz8qXoAGtdkXIrRzPUcWYJl>> z+77^>BEPR{*8(d^ZgXQlx%>>+|G9 z`g}gbF9a|VPX(oRSe=rPd2xUIS2^HgKUN{Yk&bNO96?aDe^woL#+&Qfc>@!uTZNhMQ=OKnoOElS_$WSr3}44B-Un6Ov}dsbCjRl3tIQGgF)2bVCM8 z)`igz!pKuvWC}$xO{wjNyDCo*Oc4w-VN(wDw|E`5I`gwq-pkAD#YMxOOP+dfHpqFM zetpBQ1G-Y#A>G1!X8r^PQwwQPGC2JvoMx%vO7?IaFm$iv6Kft&OFS}u^^#gRO zR%Za^{wWruNrdrgzo!xhyytVq1#cRYh+V}g$ZXL$z&Y^m!EJ3U+rd1S!PkvqIv5hb zE%%HXA$ac+7h_=w-3Yn6*iHX9fS!@Zpb`P7T!W zNhTswFu;F=6_$F!YwLmnSz}L6Pg{278TRGcDDjZ}+kaHf zKr6~J$^ENPb_h(gl|N-ur7-s!w%f~1DN@HRufIH4!a{sV*5Qci|uHuDva*-w4?vL|$SWZRTQ zmg3Stlcpop;VNB!3~UKbg$kv+>)!rh7w94#?5W=NeghM~Yd=OXKV0|vd21{)bEQRY zIJ)GN&3eZBpJ{>tJU{#wn+J3A@@WCGtDjj}S=l=1zcv_d z2Qw(Y9JY9O(=}ZJMK0^Reh`t&v{kd)R1PkHjIF!^_k6cnUPq#nRRAC|2N4V@jF0QW z=Td(iqb#kwhKNtGS5V2+7uFa07*adQyNd6xO!pmy0g#O)KR6o~?Ued&Prmx1Tl>hO z(DuCZ)vJ@9f@%H>TD~yfYLCuR08HBsZeYXu{P56#DTJ;2g%v*dRBx^HX|SxAx;R{5 zIhF=!Ap;B5w#;?m2McHa&1o%xE$ez5(F)}NZF)`H5E@WEAploj;lRR8aD&`g!jS|L zLM55KR*o@IH&GIzX!lP9D}lP&I5AH{Lk-+jCxRLI4Iz;@^Jci#r+TyhIS01sj(x=Y z8z^$P@}I$UCv^Lo-KnA`>+N>{-c@pnnfT}@geWgWmDG);1?4k#g+DT&l$=6@jb=Syvz3hA#F!lI7Nc>sMlkHw=|1TlV`Z%E$}u9`5|R+eqWy zuKGj!7OfXb%2` z@`ZU>6%Cyuk8Qa0!2}k9kV@FqhA4$Ee%Q}V-IU&A6Qpy=YHI)Q0;DfDU3IHUyF1?|#v9ZP1xRE~+>DxdI5O&nsmxtX9 z?vVW%g7bTQ<*lV$E+GsFkV2hXFBc%24bbsFzxFF1C?5#)liOY!?z~3y7zXcv_UALo z-hmX|Zbx)cF5_xc_0IOD@mqiec1Eh*!mu3m17<|s1izE_knX4hnpNslT_^6WhZLdL z!A0pb3P{p>Y<2S@^t+rgKMbBEi^OffY;%-d`yM|-zYH_Y_s+0>GV*7kBO>PZQ`Awj z;lkUi9~Oc+IXMwG5ix28X4mE(XYP6uD(z4o-PX}zkNU3D6!dSD!Y@s#-`_s312j+a zJxu;a{4?#NgdLcjO)uN;W$f(cX0RIBgFhFa636k6x|x|8wZtt1M{Hspw6$FG+K=L|>9pHbyyu@9-nn(UV%8q4oEKq+wRRmyp1VuV~BY63`sz)G8 z(i!RG>LyhfIcGQYW&IYPwke+}@mc!^w2jx9_M@Q#XEu_-hpzh*OKX}V?f&|r*(I&V z-k+LG?2qcaV=Bg5Epw$^-Q}Heh-We~DZx85tc^7@=WJQ{eAX9ouHkFB(T92!=SZA~ zyt%cv%s1XEJoZ+9|Mm1_Qgst2OV}=sucxnqLoMWZI!o1Tsb~R5U!Pa2Ne~vW$jyTw zh0m%89T01_G#F!Mdftlo>=8M#%hh@RMOXewVet=+?}ZiCj~8@_u`>_Cr|kCknN z5|r@`4yJlQGjy&B)pPWn5QfG@57v+2+<{4n{UG409iB;gyd0j{ybN!W*3k{2o087} zi=I=6<|=;3=4S}Yc^n~BSHKMXJ*%ZW0sO6pcQk24N`t{`xK`E+5NHpa^1 z6RVab?=MmpP`FwjxNOzl0FL?N_u@=r_c=dgFpw%&XuBa_^pcs+hgJqbC1(Z~fV^ns4`Sw)sjkEnW3V8^0UxaOh}a z@z-{b2F{+ovHTtd%W0$4&$uVx*@*VUHgp$KcjsFjgQ=ZCe3H}-zn}U1)LIQ4iex`0 z2Aa@dBwDgX(@guR=gN1x>ry%ry19zXxhyxX^S(V9?Jy{A7J_v?K3N+4`~5ivudqGO z_EW9?*qC+;ItKF(-%9Z%yR`rC6iIUuV%QN4{=k&(0H82-MD3Y($De_D+3s+xp?%li ziB5>v?3Mi%u%SeA*~-qHkOQ9^|w z%j_lYPS&_XJAZCohSBoYMYE4D_jCE4n20=hx}BwW$lkE`yzU-Z8oIflz={{`@SQq6syZI`xjDD6ffZmS$a`@* zT2SgHko*V)l$K%C%HLKv7Pv{gC^;KJKk?4J|WX?c_WKwX(ni zw_eoov;};VYv5HeW9hx^rGDO+F21YFef-F(-05s z&!n4PD$d7WWKdi(B$o$4rl4{@GAgpGt1YQ}>m9>b&*$QiCrQ7%+s?O4E{}dEZ(mk~ zYy%|lNx$UQ#h_b2MdtQGP2>kwdgCgKyCH`|_eF~E8HJC`y=H#iZa6ahxt`?g?_|3B z@pSa!=a*@|LlH=va2dTd<)nO2#4Mjq0Qw2epZbLC+I^>dTXcpv?LGP>?*>kvj{_1L zkrDlC?`}S@*_I8ER$~P$rU4iD@upoi^36sq|JC6Kyx4@gxsld6ou=nj*?wgEM$;s!sJ)WWRzPHFhqk}Z9{6{MFtA*}z3@)FFf*^yjneax|=y0Ija z(yp!#5_2jxek)U28@C~YPzQ<5_$sltzvHa~pe$=z8oLf_0#*|2O?Sk&z}Qf!`IwLvBwDJv_> z%`3`%dT!g=D`9gE$wK1xKjG7TPtZ-0IB(VEu@&2u;lmwSK_;^wLtGgxGf2MbB->w; zyYiLATlzwRal?R4OYr=Ok?y(f ztAYWusKBvr_IRw>NB&+kZP8a>)BjQ#;u{~N5KU<;uX!dm@Ce_N6wziRl)G@~`!I12 z)kU+C@Y(8}K;O-0&tNX$W?tuNOdV$_KwRWaGmmS&qZhyY zGBJ}jvGcwmn0YolPI+5yx$m@`r~K#KedJty;m$hf)5SpJ2nnCz-2j^*%4oYwXY78< z2_5mmmkq$!U^Ix+J{OuyU25X6?0w2Ps+MlglKKY;6$g-n>8{5dc?GvQC!aT0oGf~G zy}#mxNc`Jz9AsX|xR)YC{c7dO$eX7dMD+Q}Bi<@XS$DbYjcd1iOpX+(X+OfzSn+i< ztAgV>ZEk0QXjd29uZQ^(a>e=1%#4g-R8&D&}LpO*KW&K+^iM$YGy26Az5ul zQYy6B{1MxlYae4 z`ZpPhzyE1X4cO{^qr-Xsv-hcmALIO5Dczq3UMQyvNfm5sws5y+kmtHkXpMn7RxPq@l0%&?9G1J!FeI&~u zE%hs8t2>9gySw+D_$9QYvCDM|;&OkOA3h?X8l$m z>uSVLK`7e<&nkG08*V6UBK4Fcgpd((FdaM=mv^q0YTUVvFoX(2K19#ryf|qRnZ*oU zQ;|aN-MU|1v|E|_Zbv5NK%R>xqp~ufhM7D*KHkM+!Q*J!(Fl}h53TX0&Y8-U?sV72 zU4aE9r_dlarFQZW6xT8xhS%D7LWf;lT(qv#+*|pS%Q)U;`f~GGcpvZ$?|Yq3Y`kHG zH5)l5T^-O@Rw#PJ{@8L%*1le%+YsaQulKhWYk`8L?uFLF4S(Joa^cO_`D^rZoF5gW zK6wyCR!nn6hF(;V?|GrqKD%-(r%ehr2$OS=y56N#(plGb!(LPsDv059*eoIF)3&GO zlTEXaN2^c``Pu?$4$RdNFHgrltDWVkPF?=AePc@1$GJ(Sgy}n3vB51>HDK{w_Vs-o ztknTh34O0F8H{L<7$^Pvx?c(1%s|n4E478)^t6e5qyA^B@Xf|QxBzlZb}q_J6_}W< z%P&lhN2*#j&bK^0i>fc4C*#t!i5*wR59lo?D!x3&iu3h%Xi0P>x!; zKh`KqR%pmM{#+Z<`uD=xCcHMz7yWGC)yixj?0QlB&!INv%~p>{hY@eDq+yl~0sV}; zABUKN`Z1|09j2s?)PBW)t5Sv>1q+Oqt*}nck>?t>e3l-_ZkR3=+-$2pcRX8r`N_cJ zoza_xrLUx$Ql;gW3-F=|WkyybMK0Z})4~PMkCn7~I9i=fG%-uSZ#<>uHH-pL!WCUm z;D-cpF7^1e{ENl>36Qr`Fof_mCH@PYKAZFJ-D67pneHnOhiYi1xvIh!E%SX* zyaw#2kl=Ulsb>p~&SNpXd-}RL+Lqmo-AVpx;m(I&`LB;W{(@E!pAw%Oc{3cVbNlSb zBjb2$yj22|INgPIcK&)Kn?c)9ffPRA;x2I%NPbQ|>*;t#4({(|Yp&lH&N-&#iR~sw zqK{NqR0jsEO%=Dsm)#gTHAyR)E>u!_?fYvJgy?g^Q(|hi-)De+_Wh!M2eZYM>OHP! zGtS@mO^6cHzJk!lD8+k+-u&tvTK0N(a9|tjUcL5wG4sN7S)m@M2Bp&{soRXM-;;6e zWf9~ZY-727@;qiCa!s}0(_^nI_IjLVQzHK_ZH6-qVk-tyUjsy7ey4~%Lqz5AcjGo0 zd%e_u5MJdy!mC@dFa=z67(W{DAgQt7AxYf^soQK`893|md9@At2B&WRYTRxKHX+;DEo6%&yFJ_@BO^IKv-J=K~w>VocFl$ z#I>tUU{chZ8n^P^yKP*WdMNj)s_k&VtzKdBaZbb8k}D&Y*Y`7gl3Si-b7*I--d}>| z+0Kw%itjtWacwF^N!h}_I$_9W-DiF68%^6-^K$CqZfAlrT=ho%=n9LQ_HH|sQrveE z@x~;-8Jh%h@gKYneAIoZs7-CXJ1>uTvA5I6YQ9g!?4!`iV{h*cEdjntrkxwrmGO)= z+g5+oDE_(OAzE#5{55HPn>MOoJivR_DuPY-J8W& z`S|aYjNEd6((W23vK-OF45k5a8Ufc#6+yfEukspNd4xnq@P6D zefJ*4DQyUTdIUsk;(WTN8HqNu9wWdr#~}N|UF{5g;;D=A@eCu`YA26MvQ$cp`%f?T z)E+CSez0`c@teSHp4cmu_nY4Rx_hv8eJm}lX-Xo{PDD7D6VSpJQ^C*I2ts%F199Wne; z#Gb9fKIr>5`=Ig2A(?a3#GqcDz)cbZ-?6ceiT-=N^N+?~TnZNaNKO0Y+rpaaDfGdq zd$Z|Vyhl1_7~g3o&2!vtEQ(4}w+!b==sU+$H1g%!OL3;KJhKA*qNpIxsXs?v*Dp+| z#J7EX6(v?B*pz%$WJ2WA$M*A=KD(`tDepY-GTrEFtsu-4=M()()E7AQ>j8LfiyqJz!=FLEC9I|;fQQ^Gy8a})21t0i0Hp3B~B4-y#w`Q>DM*LEa~!}X+x z%w*)&m3yf22!&CF4z~>d2BmzC-Q;6>0dpVr=WItQ%L88KpiQdG|<;jjpRb%U)ExTBqc zD90?CA#RPr5G+g7&hLyz3^&i08CgDUZLjV^sk}pPhUDMix}(HeLC9agybC_!yc(Zx zZ}oqXbnWp>|KC56>%`1u?#A4j%SIRX`!yn@k^2Z;5J~R$F>?#K>bdTq2iThT-@2{r$7Y`{A$I`@GKUoagg-o~HzQplIRg#!-LM7vJ2soKwKrk#3yFz78#2o6&4;K)&`p zrg3_?*lQ%wMUPI3%c~qGw?qZM7BicfyAdm;%j;ghyOgXdltR(3d}-GAs9OU5&_r;O z0-k?yff?je^5FW~!MQ>nsi5h(lEY#%DUZwY?@ud<^D>$SCs;x2FziSP zchA_^Uvi*O=vL58^W*$+Qu}#u#0`F^0&(N1Z_Yfivfl; zGU@2fBk5uhJ#Fz>b6?%l@)SKYp>GHG7_nE$&^d&Q zEdL$(lBejD-GJ0hzHMZ)=3J3JgcLMmT_D|%H*PUGon-LSj^ePF1di<(u`hiv=>RJ!y_A9vKqQcn#pFbPyK z^xG{f5p~7EQB@*uo59BP$`YxNks)x7FXaA;WS8}dWR0nlsrjmhg;r^3ll3oT2}OXI zAo6xxR%k!~cTbC?njyFvltyY)9oAAU2K*Ma1;Aucp|>humXD$U+5 zl;Q;rkkNCJl2qP}bTu-qKGM*$Et7@AF4!{Qy9-z*2gqstw{Kn{dyRmo&PJUIYt zT#J}_4_zG5A!<4z#;4&%IPl(bWVZJrmZXDSNV(vws1B9FcX@r~Fp*1e63Zz!RxV&B zDP+HORF0Gl_vn1G-K|g)^d4D2j$LrT1@LBM9cDZ0C-7#7*JqpQ|KMewL|m^kQ*Gb8 zPk$gRcp6R&`eArrC<+k(3h+zqk{g=Wr0_IDOfKpBi6-Xp)&Bmm(U-8Q3fRZsEfF7; zr*OxmA}aBlNzdKwneR*hl#r?3TOm({ttvk;adI@2Eei^24I_UtciK~bV9(t8kxM*k zYh-{Js5~i=f%~sGN5LLbDIf_pT*yDxx@Y1*1s+ z)ip&99@UF6cq>*QsQm2&ihzMr3)&GqrtQZO6&P$>kwM{$+PJpcbaelZ?QNqbOYwb= z>wGEG7D{b!a`om*cX5(pNB%C)r4*leVIgC*FY}cO#r=KftBLB5ei&sGwvx#VcNcLw zuy)Cj+J=w)*Vj-R3om^QJox9nTec_{q_>rQdNZFI_uNp4mEat`H$S5}wP)3imbKF=L!-d+6b3BEdOo`%8_k)nxx6V!1TkF?{c|Dd=p6Y#)(h z_CU5TTBk4721t`>muOl^@`s2mqqJ+*Y)iPKQZRb@S)_mBluv*s+1;H+%!%mZlWK;tKVooNZ9L$~d@#!Au!;{b7qprvmtgN!*3 z&rOA4hg+2i)&wb8rJ&ISVLx4gMjcs?;wy^GgnZZVTjv(>T`Y_|u0E%no;;t?-1GOI zLp^FZFI79bL7QCU?!-xUS;5SE>!heG+eT4MbA5n_3Mv*u-_RGd0pSYkV_l)a81|;& zx_oJiysfjqE%2m2d}hi~85AoR8YChjbw>MmR{Yp(Kx97g?7nCDU^)LP!Egv`1WWPF z#)7)dcQCkYoJ^ak{1t=Dv#Zod4)Y&%2r5@UYw#n_Q^4mo#gL2hJ5EyHlo0P=ib+pA z*M4p)Xp}Ja=-dSs3U6P^*%3Zw)03_%1=EuIfE?!KGLMKv?TqkAFR?=;Z&Uo{SSe){ z<+42GV+>8F{jw+@lNg#($FuX5sHpAW6Z#2)^I9j3hurLm)q%XM$GJfD;Thk3fE4`0 z_YYDp#h!Od54Lw%v+rmrd4BH3SLg;LJx~D-8WHqn*r&b-pkBYDutuJh_6KXe>@_1b zxp|wGqxSuLf7G`#HP3)I%y{gP#DMPV0Y6E*|c47f~^$ zdmFS+8k6rOob;G>r2M*xY(2_#-p}(qFY+X|a0c3=K3)F}TEwU@&t1sgS;^PoZ;DeR zrv(57QU@0b-wiG8zmo~A% zagyY{=dMpII@vS_x=vL8K6*JAYg)F5??zQ3_O~6;i`?~;YM_Y$I-!yWrx4N_x@7B%cDyH9_NHs?K>;fgW)ea8GEdmzZKB)WI0;#r%|k|{OAN`e6QTa<`&k9A^w)k{eStN2W5o400Ly_OYEBn&JX| z>C5gHXd~24a2aa<*ykLrr6Z;jH~EI}9#h%)Q$=SeRQTcCd(K;K$gt-Y)HB$ffsLqrH0c6xJ_c++QytiYR8!Dp^))_|X2p{JbJiW zQ-fEG%=1PZy(BCu5<<<3U5Q$*U~~tH+vBP}%cCJ**ZYNtti@Q2TxDN7CJktW!s7l&A)h%nx0$ezXbZ~dZ`{Z(6c zU*7=a*j2m7fDq#fdMrzCRu$l-#@oMeG4OC69}tg)qj%pCvnF zgy4z-Iq0P8oZr422564$?v^XvKgM^cp;?<^ z!+&4cdYsNaVK8BPamPH3;64fYq4s<7Y|vmNS5@4PaY%Uw<+SRzAWgHwa>N9p*9&U6 zCUTLmnI?x}ZUm|ve&Hs$J~@oB*riS9XzfF(qvQ?AKs-tgs26{a>_j1k)}4Y{W5{R3 z?uz*M9|NzHDE$eu^LeL2F(U_RzZ7kMi{;4lq-}tnv<wkc@VHxtlZnE&Yb@g$I zZ0X{a5Q;wB(e@Un4^LO{VyjRF6RW6BT7uT8hTz$qW%2hBB3DiWzhueR!GDYk42Yv}#t%UnX01rZ%mc2gmm9;q{_9ylb|X1jZxqKd z_FrUv4Iibva606M)`7`D_cN1x9z0T1E2O`i~{XrO0SrX zrWzbBkXT;E#Ty{hC`y&EC|&iE8ME7vL@N!#N-2GWvm@*=DV)BsD5*gF z_~U%!O>!!5UfjB7VzWl9$l`cn4uXOBKe_mgucK}b2c z3yRMni5Y??ok?);w*MJAoBsaK7u8e!!Z*^buI*cat}ojVYItmKrpZ7|CBUJDo(c^?idjB|y`EzG|_Dz~1_Kd7HPkrd17?wJdX zTRl0CY7jl%4)RsEv5c4Iid9z%9x8t_cb4t<=+i5QV`iQrOHF`MLDqK|IX(ix-b|nf zW5b_Olz6@k=`5_YlCDRDTU&+e1u_nBkV)%zHlhs|@i!X)NKoqBnkEFm6%|d;ye);F zkmt#Rb9=9A1-};@JtOvjc+=#lU~K5)wRyuI$%5%kdQN?V^1lot@_nqbo_}gW2s0G( z-f*AAaCplh|46)f;QUy4sTvAM**^Q%z&o%A4hgQrY)}nyQD)b0j~HaFy-2 zc{tJteAk~|g%?W)0F-C98ZnqUDKmQPlXRt~VY4oOq|$LEKoT#4Uu7dk1jk?8%+B1p zkBm2)%t;&8C@(R1*b+xlWuMihUR%QIw8fC4Q*3;(>1xG zM@{HvumO#^rNaJAXoLR=6@2?-C`<7xz7LpE2Nj(viE7}+^fHvFrpX;)n6DxUi`>^$ zatqFM23;S7yPvvT{itU18QSXXK_}h)3XjB|RXkky>X;@YJ2M+d2I^2Qe~~$lx508i z_4=k8&ujQbYkWQeF35{?XahtopH*U^bZ-Ds z^3N0R$iREy*WdBnr-K=%-^Ea6&1v&m6k|94aU%AVuP~;@1Sy1lCIi#DY) z-ShvG?f=m#_4^V@>2FW=cHbET_yk4VTfQ&$HO^&`E#1a9^JU0DNYPH8!nxqyXbP{! zkdWb&zo((ZH~A=q^TwNb^FR+6u8)3PC$N82J!FJsA^HjssU2QFDFT(QOA#Ynl^uH) zlakRlA)>SA-=z3*xr}_BznBI6R@Sf?g`x2G&(_sW%518?$%0+gm`ymSCop=y-q6A> zP}q0-^*vh@$2eqU4;Lcckd}T_-W&T}T+xsA=UQ}0^83#7!fUgx-oDsqA&ts4!(*V` zE0^`ARY*@5q=jyg?$+9t?>ARF zM#lCl-|$VB3i%de(Gio4ORo|}ib*)Q75LjYOO(RJ<+AadS;q{U$P6eTY>_oV||0|U0}PYk^; z<|$ls?GsyO&~1f{EqY&QZ}tY+(_!W)a&Z)32QemJTzSxqQVk^soNT)bZhzyiIrR?Dwmm4hCZ05{AUJ zBBf>euvwRjz5F8C13&xSWd-%&ByCJ!HpwOw_Wnrrey-BWN4^!@3ls@9jRuA4A@gNI zewD+Ks816`W>hEpX{F0yCaW%E7y^K@U>ebd~8nVjVtO6 z(4_Rc89whREVJ(yxf$6@{+l2F5+?CFtSQqv2GOUqe+OzjStJXT+X=3VJ^xC4IaVtW z7#=^#_U<%a{}chi3)>3Pj`*Yl5NOj zJ)!@d*7qlx?ceu+VRM-Z0u82wqpq4QE~MVU*_{L64avF4 z@sjOl5CiyzK_$=<-UAG2&8^YKcG^I=N~{V$Gq@omrahumDZMjrS9t5G$5mH|)IVqf z&lRvh>tZ~Ib(@3iAN>CB)7K`_>JXdEgS>99gB=4j&58`Qd7PU0Q&=D}y&N)Bvg@Au z{wPVl#R#ateZke^0k=P6if;(?<*lSf0RsKQy|>4I{@&}^zO=P?u0y%6-&P~-49@KEVH`rcR?t4(=(qq(>QhmKW zlZBiN5v=36V5>K4<6_ZUJ>t)57cC2NA+^VSAYW9z93WSC>h5h-qH;mdv>4X@A%UM+ z2`<}u`s{stZMZzZ&E%BNevDy01OMyTknru_;j>Nq_GfbxK}I$Zu>DHGYBc)o1oU8Y@k!SwJ8*x?{xoV`=D*WtC>b#dGmo3DLL3dovPgHEe z8QmSVicH_h@IuLen=-l;+$St}q~U1FOy7u>9{ODwTy*tZzjKSRK^rlB4O1h<{?et{ zCUB20KY2j;t(J@EF$#D1@kdmihhO*Ngq z<1lP1Vac$opi0cQda;`O4TX98LtXY=P%~VR7!iHEH6lN$n|#E_EQ`YH@C%GzUCX^L zH?8W+Ur#AB)!q7pv*9f@v`9Nc;Q_XlYKON2fH$MxyL`cG1)Oc7aBrT<#eZ+pwHPo! z=H5%6BOj!Zlf(xyH^f#{q__gtr1E7Et9Ny(?pgdSU5DsI=Nguc^ zXam3B8lD4Z155!|S}u7>ofBK$DF~I?Gg*-YTaRXtoUBTJBSOrpuHp>CWKO4C*dl>6 zvhRJaggJ6bu|vdNvJ_zE@%{ZO7l6C{l%WV?c6WAa%N_W>`EQkf3H#NLm2Ns;0SF}H zY|i>XRNqO7O*K2GGQf}?NLLTJWt85~@%)>uONC+{@!U82XBVGI?ex4R&o?bbytH}P zl5i3_Q3@D=M#vkv!BWY%frCwLml247?IY)gDzLGE#-DS2qa)9p(gG@fZ5m^@+S-y4 zt1O3s>_Sq}9ui27?C!gAc@d;XgNNaHdxIvtGQ4O0-F|Mu+fJ~Kt~239vDg0&xhi!# zcstG2cb3BcTdkdHz`S++1>y*Rd(+v^xB#Dp;gpNSZ^t$LhY!R4^uN+`Soxit%# zdqrm7p}=UX2}!`&0H3cnBa-eId4vLO;*@^zC{D;FYR?b2!>3PVkEOY5fFgD|wi6|6 z6WD3#lOddR+vtlvR(uwQ(RhZKp-v|X6L;<=Q|$u+Yx2}u-DkBO(I8g`Z|0XNU8fTu z8g#gwpe2%Z5Ab_?%=~^3FuGfY{a8HK_Lf?%eux$S73kyqOKJ$Gfd`Wx&e!%hCY=+` z2fQ?@n=6EDxFwnlU?G8g%EA^a7sQVqUi^8#J0nDS_})w@6R)=h3KANq(0AL&gBRH& zA$vJUC4c@HsMAlWir*S;l`L-6?}B@^X)Tu}D!%NyMcn)fY%N0onDYxpo+GH<*rom`_9l*(IjKEzaJu=*LSabQ z4%_Zcp$ssYth1P+hP{p&sP!`%;l~t@@M}oj_6ok%fxiB3QG@dNQukoPQ-H62y-7|U z1FJqnalTY1M{`k#c+b94CHrB^7n^}n`}%KM>Pd3F$zvB*5#azG;ETRM_;_dFM6WCb zU^%zvCblQxSM^SVTzA&iaCXz0qqeFi_+yk&|1Qwm+IFDs0VeA6*CIg-e;?jMp4C9s zo3R6%y$+X>mh1jl^_z2fKe{&z`ubG%dw2NOAZcJg`X1SN?|OLV^GH5XRRZ7|uU<0| zqvZhbxAKhc+47FczSUb|HVTHH?nqsHNRo?~)ms(xkICwF#}_ z%ohJpSBDbjZ3%$0`ePHn!w5B#G56uv0z;3l-D?1nd4jp?e7Z&zW1d_W%K7)yq1BXk zw|6?~7vP#1eCJ=(pcRbGG>S)7&iS5xbJ=&M!E2T7dL9ywhX&#PHWY?c5p0AJAiEhG z`3ip9yytj1c23FS6i-i(Q*z*=H89r0CR|y(qo~!mAN>S9p=_>`{psY3>?|G^CEh8m zA+2ZxV{42`ny#TxXIv_{mS5Y(`2J^l#Z3WVbgu%i`z&2_&{ZO2L)#Z`@YcNOL*4^& z6GiO$Xzngf=&s+j<0w*PBpCWK{Jy^R;B^o4^M8qk?|D+?wgM;uhd#?um}1dAUuXaG zQ&M=SeR!taFS`y8R*_XP8&XYVrrcJdIhp4+TC~HIS0M7*jVF(#nco9H^~Ok)Gaw^n znTo5g2k8v0pAiYvfFt`oxf){|~zrU*t_l{Q?@-^*eXN z`?Yh2sU+~;tpb}&@HU5w_@+~<;sTKQ2%|tSxNk6^>k0q@>M5Ss)sSy&6aeV92c$g5 zJUQFI#~)Y;eQVmBAmt|iFmlCcDD%>5#i_+d&00d2l%<>WvkT6iyQA(YbM1!7(@1e0 zf9KPCmz|kqoxdk_Ad4JcA7M3bVj&h<1y!;)?C1hyr=&8`w*L3ZM zcjh7Z4Mk8uumXjV4CE-W-?v-}+*@1b?sOk){}_Wxa1g^gp0dbSe{{&UJj!jePn8-AvA3I(?0s~#w;bMFlWc3|T*lq@hKrnJ6jj|>qecgBGHBlf6&u%2uw|q7HX8%okpma zvT73V4R3kdQu{7>L&8sd0<(vi&;{lhv9Hf9;b-Z`t0gH#*T{TtQlmKI%elX#)niJ4XE)2(E{>W_ z&@}@b^>-P=>0VCp2$pptyVb7pBy%Sw24*3Un%ak_n5gw-9ky3x+f$}249rN&@#bqB zIkzp_J!rfM?99Y0`%a#-tOb|m^_Mrorn^edys~z;6-%&XeyQL{f-HhE1dEZZg=&G+ z5r!Ssy}WI98`jA>)bQt7Em;34z7e(<*jMCLQJqK!v6Z6$a|P}hWlIs&t2YTuZe2bF zb3JYj%LysGM!_VG(hlR6a9}CwN0qF)2g9e@HE>D^o5la$cphYAFZDg;-0rR}4EEGZ zrRbGto#l#a;3W_a1r(jfl#IYJnE1rjq^W5;r~z-ZF^JTNq8FIl!pSngnrQmwLI?R!INJWxJzP+KWLNdN zA_=!%y9lZ6pjj-h`J631#dV1rffHo?O=6Roy(N`a(4(zBP#+kB1xw*{xMI5dHl#su_~=LXFYAsI-CR$AmZ$M($fhp3%igV^eN!i?nF6L<=2Cck`q09eHZlrooHfo&w=0S($Rg z1wQ|`@dX#wsF*5N2Hg()o3kNL<=wrz6+{04VBL@ORC%uAZueeqvI43$eXzUxAg2(4 zhmo79W<0nE?N8ANzl>ur(MmBgImKv4O1#t1R#gCLV)>$=cs#R?kimrQr!goAN@@h4yVC5pS>f$s1eQtpf{yn~!1Eq0v zke`~<_%<@lGkr?Ks(v=A2@t!fa?!a1zp{4!bv7bDmfwv;&|mBq(&`%1r8}>USBb|c zSUao22AXY2V6oS*1l+~x4xCP!4yVn#%Da}gjVHTlT8ARNiGp1F(=4Qhtm;hql+oP` zCzMC;raq06Yu{+4TL9-WCe>WO=$7I_f;R&a7FPSQmruyCr<(6CWN0VI46MNRpQtC9y#)OjrxwZr#C6k>vh1g?1*XVPks10ArJ=>oFMvuJ(<< z@b=o?+4A(F&sc0>wU9}=fsL8U9WJB~p&;cX-K;F3LkS(cZlR!v;)qBi%m^05 z;{$y=a>+I9OYYJOMB{!8l^l0l1ckp_y{No2;2X2=G*RY&&74MiC{V zu-isN79WIQ0LG}fEug1SBYbg3Z+lz&WQaEzA8RmHA-Vbh$ft%T5~X^cl6mG_Dpw?3 zO8VKFnOnPRwW9|i7m9~ZC8jv&^h_826ZA!vZ>wprP1|MN>Vps!+_1ht9`tOMF1;PO z($vR8J)I*+KkHc@x3T3X0)orkw~AINVPKEfFU~9r(y`DFZ8;kkj(yIZJH9JHKLaaf zYk5Ju=4rK&zoM%3AcM$Veftm_=dAF&o#K>~KT0Sml&=cRsBzk+W>( zh88x3)fz;4L2&rf@*&ISGmtC|9Q3%WS2q|E&w!34nZdK`;7AS_dl{K!Lmour)McZt zsK35YX6*(P81%u&ZZkwOBnuT=cI{wd-LM%VCoqMH z2P7s?eesVz26q?l{?sN0G8y3TpvxE`Dwu2Q(fe+=_Nu89&;7;PRyWzd5(*@6VEVjh zWz4J&TT^UnuTR0`g2Z15=smIC2pVLTgj3H_Oa@+25;tBKhCc~Q944~r9NxGjQacjx zXy^4*7mMu_=f?iGY=~ufA-(%;(i4_Fn@r<4>ObUbN{m2Id7H9Y4f(x`lLMw@NiFNO zp?YvSe)t_Q_DXJC%uyGwwA7hpY+<=AIt6U!JSz*VTJJHRx&uf7T`s%N5U{s?7pV@| z!Y0iylf@fjLOHyX6qNgKwo9W_!K*;wa++}sMbqP+PLUo=8CUXa=&@BxurAFb7a%?N z81sf_YhD(3;bbSMS~7P9Csjs*J;;%Ot6 zqXvR=qcEFE`YH*z%X)kNo!TTEi!($ryRi`^8c1=p!ys#wKK=b5cHf=u7aA-Y_ub0kCoMLGa~BUpQtsi?#j4rabbVY2h?T-6E0{VTi% zD`Gx2VI@~1nXb~=Q#B~nR5a~EGO+k*mgI{<&0xP!BLXk%z>#99qWjpU0;Q` z#W&FWKxMJrSrX4tMvwJte^2Ey#ajRow(3VRHrW~|vX=du+*gYBYQ z0w&#T`2ECm;>#$zJk~h%y70IeDU!TckhF;hiDzh`oyN!YC{Wz>4?0lXlqeK^0q|e3 zMdziZ)ibT5>3G@#?1bGBWRo+x;92V+3p~S^LYvdgoxWc2PT@=w)+x>QeALY;#3;b5 zGf7AN=Pp@TJuLV*sPaRb%v++XGe1Q?ZJYZ2A$!bC z?!Y^*UpBiV;RjU-xT>Ih+rk1?&r_FwY(Bjs!piLR7&${`Qr=*klK9JQ>iaM)<&{cT zHcNfL>OTRQ5TF(@3O%h-8c%Xl6F+7>?r~fHPE~&M>xRrI?k2RGF{!M124#`c+vWa* zGflajMi{GI_|=Xy`|Yd{ncXk46Il`2kcs-Rv#4>Ug zMq}N`2tU3JJT&>#&>S*4aYh_9!*fy`vJ~Px#W^R$;~-^{XR&U9XQQ!Dah3<2jRmw+ zjp+dz?*``-`OR~~$D+~XJ>a&81?pJQf;8h64})|zrP5Y!75>x^Pf5kI(Z6@!sn}eP z%6yhYXp~Aqw&5Ek^tj)0zh<6m-LTu4#UXGdlbIts;I{=(nSqp0s7^1_^q=p2d2=1= zXJt-pD9el!8fXefug)LLZB9-FmDJc(^NUO6#QnK9RmIK6E4+Jol9kvE#Qa3n*GAl> z8Nxe{!k9Znxc;pED24~R%xXR4BW(?aO-UbG8r9);^ z10P8P^D9Q2L_!@Em^MvXQxTI0fw>#4j5fw&W?j*9f#c1SEHGEqEUU=ahiz^jk9c<7 za1^aMJ%FBns0B@GHA|ZgC@VwlTLH;FS%+;l3d76w+sG+fI(cg5TO_ko9=Lj)3&VS4 zR0OFGy_X!#o!arFJt%5O_GreMwTi|QgLs^&3nV1Dqr@_v%5}fY1}VMQ@R)Bcy3uDI zPKY(VMUxx3-vB)Vds+7z?pHmHi<&5Urp`jLNF&b1p+yRnet`Hp+qzqmZbxJB%Zwpw z8PWLWliX03y5gpQ7pN0${2d(A(+KUcRq73GiPSxO@u&D>HXx%)kSynZaFurARYw_Q z>Nq%h+Y&_nsPdnCm=35SHV$r0LitW7J&sdO5cK~IW%BSG63(gmsZ)=$xoIG+3!XQ+ ztNLa~xmGww2y`APa~I4(WiBlWJq6ftn9Kq6gJdGr z359cyFRPw};#AbqHw#FWT3;D`tkVjhV++xaZ$rLRLlxfN(MPt7XRyjJda&50*0 zm3}{_N@QUF8yW`T+GK7@k!BO13I6oQDNhPmO|)iHTvP~9iDcvN9T!Va>0n_-DuVF= ze>d+sGr#UpG0)r}??ZT;Pno$5ArhG;zAE!)$FbA00+SPJFW`T8I@^H+BNbZ_H_ zch91brR!zcsz15=H{>JBD~SRBtF;K);0m0!))(=@2ups*n^x-uXDS-EMM6x7OXZCy z-}j%)0q3-e*Y5Li;S=4AS1<~(ZX?!qTp(g^b>X|-#`QaSHc}yh9noJNL|aFUn$Q>y zg}E3^L=&xwS`|e;!pZBK5Q)EoKf75A`AH>P){nGlX7R zU96R%a2jW+kl*Dc%4oq{1d>Ef6cvT~uM4_2gBW`wK+c6@TupKMoYCD8pmO4;oiMRj zqsk5>;xxyMGJOy(ZV(O>XTmhzHFDQe zp$VcBHGv)^3SJl^-)dn$UQ!!kZmO%UzH#!qK= z>uE&Fh+>*`>G*!`NcQ7!z>)Rx-5z3rNQzF{#f9hTcj!7|L+;|7Ah+pV;I}2fWdeaz zg$6<{^&g6kC-Oyd5WBk@F)Eid(Y8tW>wll1^PXY$tYv1}#3hy1%NhOjbrS;IMMY^m zg_2*+3RRyvr4V6mq<@Xo6vZF_acWWDy-q`0;$T)lEC!`_`1cKoF7-uO)MGQ#1`Ab5 zN;0A9%8@e5|4ANm)_n*}l^x$eMV1BaX+e_5;)abBelM(}B2&kWOVkM+Hm@7gVVyjw z3KbvTKcMlg;@tvud85C8W2lg5<99$GB8FgM{@ah7$PId-!xP9cqBT+0YPdq(Z{p%E ztKCKRuOmsjQw$O8XfS+Ghl!%FX$xlE|^ z<*nB&PpJf+AyO~U@ySA=&(m0LtqZ1Yw)Pp+;T_P5mvn@qu^;$$(F$iShpk4xYE+60 z{5vvxbWWv|8~86I`6VZl-BA4GMw8z(foNk<^%qn1^zP#m|4m<>)9f}^M!By8XpXNI zgZ~(mY5f{c&RZyiq*X|YearyrO&7JE3{>rNhs-XS`7aj?E-re%+IaPf)T1BUrSKh5 z_anm%>3fsFM|7X=)Kg3R03BDsc}aAAo|~krE_GkgT?iuiiLI4DP&#?h+>iGk%|iR)o>tO_^#x?rMJ25?4_$2{#y zk$diLAt1;-%a7}#G^NL>Nbv|+WJ(W-187|$?_QKOBRqKfH@E+b!&+JO>kUMnZ0f9D zG&W$wmB!x<%QhPnI#=%jdC_vBKI5G@=yADZXbKJL*i8h408z^JYZ6OVYU&L`vG=ZA{8Y!5Z ziE~{eOk~2vuGH!QWHj;lTJt8DB_5elY-?iIRDO~-NlZCpQ-fs3MBx)8+SKiJ(2(ttbQ z4%LFByRY|AB7-ju{{ugG(_)#VAy?}yXaDE>6o-Z@RjJkY&A z>g&=n>PQ)kpq5G;|55ieG-k$pdR;)qWkZ0unj}fbR`>5YKX4R`Y}pxqFKhi`b>iT0 zm;$Ps36$dxox%nQhW0&WoyUBUowzfwJ(VuoNiqi7EDNMyAL;sg79Had1_Q2DQ}HA> zOiP&M<7o@acH~CWL&8(*(!mdqH({H3!+nXaUYrll7wWrg2-B;V(UyOY1c)EkHgs-Y zRpO5(ds#kqS$p{JU4m3+G(=$u=k{R7J*Z3#>z0^lFZ&5&o1v4@N`7z8YO_^F6OMA! z`t-?2up<_T?2#$^fzm|O&1<2nWC9nH-{-6)` z*I&4S-?pyJ?n8HvNQ#NLbwoq8+k@B7G-GsP{#d?)*tN=1XOO+Op@DHaY^f196XpJr zQA~0Z>EpHluC9gC2RdW~h8G|oup5F@-Y8x_tGaG3n)a$|#8vMUyEdyaz8J|rFSZiC zm*HaGS&IO18xi^Ow^#%17P0u%w@By<60W3xy%69^U%=6TKT51a|Dd-uYdFaIitfkvy797W4}+&%w-NhxM`F)aqsdVb z+iMbZyvhc`Xq<{}TRGGQELZO;0v>UXI+bDN+}jq<_XL5a_!;ZHStum%ol4t4Ih?sf z%aL?N?h@cPgz`6OOCf1xIG?7SO0Y?j<~1mu+of^l0A>TTQZrmAcVno#MBG|%@aN&r zA(sB`kJE?ApYMV$61145L2}S7TmG9N*M4RFV6Xr8$@fhGuSa%THE*F>?LudMfmT<+ z`gD!)g!Wlx@{BT#y|BSMx(YF!1SCPy;nKgGTmuFIG-tUH|i^kD>XA1_O zS^oN`cAn|>As$o{o(bHxc3U1aX?_SCh2jrP-yvhD@hUidn#}4@gebXUUD$O)p6Pr8 zI6{(i5UFES{T%XbdUxOcy<)`rLY7B7)ZFR8PWexqonf=}BQ_PJHRW*a`|A5=u%`i& z--gLi{7oFXJ?k(kFbmGgvQ;ti?ZLI`br=RiQlj$fn1~ClC9m;7D(^;e{Q2-uvptV^ zaFZ$2AwI?S=twhya-G~5^e$!GSg-{VWk*svAx1Z}=qa=~vbLs<-e7lYfN~S!O-y~6 z$)0m%y`NLV?k^n_{kGH5&(fa}2ZQT~<=&}Nz1v&0L$8Auh8NaED`EqMVvcD51HP+I zckygtF%j>1uf{KVexquqGxPRZqa09%HREh(SGUkK{gw#7Fw%SNJrD6xIH7?qx_&vW zyIM}bt+L^I-|3C^Lvg>-j z-(1^Wxr3f4Qo|CdP7Z0pHGU-dNc_(KwRh(6P`B?MZ_(3;^0h`P%}54gSF~6fL(8O^ z$})C?vbIvml4WR&sc0c(Yf2%-OsOa;jV+au7z!m4M#?f-l5>5ZbDrmEI?wMs=il=> zKYz{Z%Z%^b-}}C<>%Ok*{=BOS@DeIn-w!_!Ua*~H{xJHk*|FpMqM*Ju?UBBFhuR@# z?!D&rT4tmjOyW(n#?zS0as3C$u&fG5)D#__=@|Ymhyf64|Nap#^2OxX=Zn1aQ~p7% z%x51&C)#b1c5eUqM(7Ot<1<8G8fih2b!VVbpW46ri~o9`V_GC}M}l$I>%RwO{-x{o zY$LiqaH>4=_bTOo-bR#yI+SWV=K75MgGSrGhw*=!KK~zvF-Rk-_1>YYF82;*0~+ZH zXll~JCxkY@03nwi8PRO|+<4t7rSYPY1c=dxmcM)aZJ@0*v=>ku9_4k%*Su3{`rv4K zWEfvLq9@8l${In}U4-)Z+&@+RP~v-(n}s_FgZmDBeAaVePuT`QyJ)Qdp=%=$%?s*? zS69G_d35@$23$cZ;0lu7#poTIt75h$eQ&vV4~mho+!w$v#DcVfz3I0f|!rGpBc)Z zEy$?Iv0jX-80~p5dGI_4=PxkuTE7Yk!LLAS?fi4I2|>{FmEC6o@~vI#xT0fXxy1t8 z`@--WQ)Dt)%$FJcd~ZPJHk7q+OF2&$?(2aoYOH{q;sf2w#Lo2S^-hGYQ!0owiwU=7tXNz0ijmo1lbgJeqI$zT5J+`1!ecOiz zfu0*l(`XuI0?=C_qvRgxxElb?p+5@r*;zgqa(s=$`IFvbg;og)@*fr$w}&0jPo5>I zpO@5BL=Z8#?_Nl6WvmLyFFhqI>-Doh29;)KteE0X~gH49dg+g}CtCEtzIr z?Z?{;{Xt6B_wo3MZ@9u`lo3UGlaw9|ZCxp9GPCC`bl~!<{f7QB+~`m`o(3!LfBQm_ zC|N~lndOT0tIII8Vh~v^#RqmtTU3{=9X4r6j`55y2d#_CD0Jk$j!VfcfmCp<_MN4K zb4IHyB|kge4$!E@*Y7sm`4ms3Zojf=gS}gL@T#86r|ZAGe@x!_CgS`XH&pNRuiLQM z4F01=xw==>f26BII0WD$7=&%_TQ^cvhXzuG|(uPS)8#P9X%=E$ukM(8g$-Imu}PWHJD0 z5@`NSH?+w!aBXQ|Y}ID^!j7;_$$zd>7Ua z`+cj9l9j_cNqj=g!0_;^Emr{=zg_!D$Jv1VLv0~f!*of@1;DM+?AJds{HZth2SFr!x`n`O zQt%^O6Xo~@ZE}a^%pO>y+P4pX?Vw|a6O&$YHAVPT~Al_uHHwsr-RF&bx9LB5L`-CNFxDvIkZ9$~L*4COJFz4W z##Tk|IG;F#PD$C_de2*@mHeB0yUO$dc&0Wd*9@+gJ_-(}BZ5u+C|SPTx_u~wz4rI&>VO3kS;A*RK9 zj(&Ns=E(@i1HHgY{3tq}q+)OoxnI2>^@?m(^)_a}lH^=1@?=-O8eNT*za-*h#TdrA zsCFdDYI(?DCyf*2zC0lzKfkLsB}G7ps6=WV_Dj8&#X4G#9DSj9$p>PH6fz7uuZr#CCf#ZeqUS|*I1;@>bLn>t=8)U!6AE6(vG>8sq zx*old(+|qwS_%W}oW7I**a_!{-78PglsJmAN8lLyo*FiDz;2RdE@}p!fxpcgdsDqY zbRn8sMtZ+p(KX4e-Z+O6-!;td6PL!k#CYbGm=H~Hm%6i4TsMN3dNSRR?eKY^|4zG3 zWrmd^xe{x{VJ*VV9dY2$tZLGTg|M>R1-7xH(dogB)QTMC;7VxuEge+iC2MTIB4??G zi-{kvTDwA_3)jMH+ri;2NhM>pVThcxC2U(ShSfpo+7D-0xz1PTP6+Q4y+T1^{7HQn z8jp;(4n(A&;3F=tJ@S73J68>jv)SK}Nu&j+U0vnJkL=YU+IyLg^t@5;=a;3dW9%nv zS+Bfz@&p}kkmY_1BR>&_C7J@fFFU~O6=j)i;n!d0$!3@*&zf}_H;T-YPh!&zjv~=~ z74axB&*BpPeDNePPG{%oExUEkW=|j+5Hr`XuNAnwg?QCoAb}8YsN|t7U7gEDqL|+L ze)czJHtmWpIr?KL6AiDd_IhM4O@1ulA^w$VdI4eMROfs}BLFR8%dGj%Tazmi0yMdo z@$)~Vp=OuhG{610+JK&zE*;NJuO6hCYm#nPchEG}CRbQuaI3{9K$x;N1VcgHpb(nn z%NXWg*aX777zm^^XtGZW)ja<|dONT(MjG%PvZfuU_qC)wV*LK;p?O_fP1;kAq zhJqwgBN9x^*O;+g!QVA>as+pOtBS#e>f6LMo*`?Jr&L2FUe4isYTlKpP5#bFX#kv= zs~Fy(0J&YQSWbNB2zI#!2_1T7oB+)owhU*CIj3E&TxteFdkc@mnZBBI%TGauEKx3R z=>y2%?P!dvY5lP&a{5e}Ov{BlB#&>*96FpCkPnlEq%USf&p|3amrZ;QnwtEUqBg5$ zxO~-kz^&YZpIvyO#SjEueq}2-BAy+ZE)fel(IIaRMPNhU8}m?hHcpai0&U%6VV}DV z*hB`c<}Ng+7htTsQ|~=#+wN>E+(N{ZSKF}#D11ik+A%>^?_iMEDy$fmhV67mmB*U3 zn033KQmy)Ka>CjGOG!02jskCRKMFft1a)%l+O8T>91M2ObV*7AD<2S23Kx6Ki*INR znr&z7ON-&iwV~XQ!3ewXmXxkbo9OuD7mtxQ4#blKYhCn&OrraT39S29SB7WTMB|zH zBb{f}epIJ1y=P=bjQ0cq3FVCUjOI7Yfnsy%YmO&=wI^0!Pl{U}xBN6`;N|QMS=Z=z zv1_ndO6eMND4XXI&&qD;>Z{hH)u{^-bj$GMkt7ap7e7>k%gReN#aLjh1wFh8{+avs z4X^Fp(gA5@XK=@1p41~r%fgj_VN2LIiA4Clg7{C4)`KzrA^pTvvhUc4Zk9^IwyWd)SIzj!}PM zOO#7pmDDp_W&T6o6dY_12qqFGtvZQppn&Umi;(AS1S}!g-z>?xRR1PB+E&7PF4qse(h;FF@Hnfiqh^4v~*m zpvgCH7i9BXxAA|F@K0`hPLEZL;~1%olmSvMAuD#P-7^NPne>C>qqt3(e@ItTi zAgV2$$G4o5AbBk>M>d$qC^tbE$N|3Aoaqv48Qx+wH*jwfyoGwnA17XAPMuliyy_yu z+4eo*iDhC(D9yhye*>JR3g=Xpp)l0uElksF{)0Il%m9M0x16@}kzX_6>m6WH;Fy!B zf}i4jzP$vNX@J~bHJmsr16 zC2^!=mDVyeM=+E?FB0tY0`TAo*~Aw_dbvZ}34vxlz@NL?&DCFgQ4*f&2;*TTvGLhr z((GtuB??fR_cjAbRIo>*v^DhLRnvRl&SX!)eXrGgG6CAnvhB)A3E|m=sN#7mkYIY? zFb|$B({$Br-TZeuQL5-SD%gS)b)P zL*N2$0zi|IS9Kx{<-{mXEUk!|7PHp?q}p*|UzA9S$#3sLz@Fe9Gs*!HB<&cPLECT- zIgCaDgRRx;OF+&dA;&UVcEXC%G;@)ciU)Q*cE%*k(hB5g1}LBo!zw>(n)(@FhEv|y zu1%Lu@G`4U&@>(aY)<~oIm4`orxR-+9m7h%}jBg2B05g~q5WAh(`<(ejwW zTFqXTPwYM&GIDCNVtf7{%Be{lj10 z(EuzQQaguoHEsV!>4e4tH#Z&|;_OvLU94OHpIuj+FvAK8gIc3bRG;fu9sqeGwM*hj z{U{#sq^Z!M#u4|5Nw|y*cHAuLy7{R+gTSn4qX<-D`}rJI;BZ}UIISiGB7iZ0u3ZX%4z4m4TZbxpgLc>jnT`bqAvbmFj%M(Z&|UyTlNs9-Y09Z?un7z_ zuTv3y3jfkpJUq2qv3WVTa7@fy1E6jORVwW|qt*tSvZOMgVBUfiOIm^8p2!2n$oW7$yEDZ zO(fnz>9|aXSohe`$J{WCHN%Nu!x%M$Z7dB-N?JS1n3W0TnjmOzPZntKbR<85tcnW< zvOza7F@&yTg+_rbx8$hKGiT2W*ki-Z`dDUctnE}n2z**Q;eo0Ymb{NsuBYxwe;tuw zF6y7Jqi(yXrMBhQZGijdx5eU4_Ydz2ZG0tf@0{e2F#tQ;J?;K@F3UtpnyEHrF$20qQZ; zvbDq5@+6m7Ofc=E{u-x33y1TQLV?l5_2&L^q??oW9+*&6ZU#OG_F+8`_Uc%>6wXy% zvDDz{U7fBZY{hcVlMrNbi+yJDEJ$m6YX>8tora;vP>hR=9gRD#WVr13%5QJCH3@aQ zr2e#+XgqE~Mgz{=k6fX#P5LaN-Q03Y4dLG2)ZbtU^ z>S|vhE62eP(f4^2J*&d5z%^aZ;7m(P3v(LDPNs5y7$;zH=tdVn^Lm@-f{9y3fH1Me z^XM`bOP%#?PubYVpuX#OZc9)EW4toQ_OKxvYJFmWIuCD@7gIUIB2NB79Kt@w~rGoMNS!?d#IZ5jQ5oMGHwjI_NI! z_2NcrkzJU{`WfCerP}7%udmr%MN^^@7wLp_gpB5YxKc8-O=-tQAC-~3J|teZjH7r> zONyu8wTiP%TNg*^k6I$9L#S^vbc9;%!iEc~Hy#DEBfig*#w3)euHP5PDj6)r>(cY^cQi4aP49tHYj?PH z@8h}Rs@IrxLo1(Gj7BI1pXmCQvzoeDpz+!;^Np6owy=51qt~K*EvBYhWzG2w;|_pD zLZ#WZq3$}}XnPn(YQrLrTC~IC&-|#2j}4liY2MxFC~`T`;xQ)CSDxJY?3});DI@@_ zylCd?!g!*Er@6L}@7MWAhM5^Iq>h7eB7anBwzZ6t+yFi)96Zafrr-jad%lxd^h#SdDLvkmlSvfryp$b6<-Z$$Aq~Hj6P)**lI^^(xc;9`OCS*)jO0|9}Lhp zxVPpP53&s0drSYWei5%fUEGLnL713N?&1hs316$W(FI?5^%2L{AZ-2t2~}CmWU|)= zA|d&Uf9G<@JN`T%Cb!UPgZ7?+Uzki;x#B!C2f-vS!%=a<)&sTqKCQ-qEobbmJK9Pn zC6wlXR;h~CCT!ltyP_qaW3h17%Hz>zt6y(XJqn_OY{=ztcPu84^w%^N9Gz+?aC^`I zrJOjsBNM`Yv{og)HV+d2xqBdzLqo<@$II+2wRVeCND?ht;ih}7yW)TTCO?;174-x(IVCYoJQ$DNykN-O;4rbvlpO72>_;vn0lp^cluc`CL m!}#}5{`WEYKRGL-rX;25E5mQFwu!Y)FFr-B}9xCJ%}=lPV_QH4<<%LFgghdf~X;S4++B% zy$6ZT=)-8g$^CwB?tR|(_uTtD@89ov&T;naeRf-W?X^B@?FFf;t$yw5?W-gtB-b<^ zsXiegp#YJPkh@S)5OZ#m3-6PVTrG1{QPI^**pZMtLMEqN(bZqQ6?y@8 zm!T5kyutrYN=b(FeI(lhC7E$bjZj@>N^(+xm$l&nFJDp`d4IfGdiixV$1RuG(96{I z9I@f`!naf({qY?;m0gE19O1GT@>?eAFY~`yyZxO2CPos zNYZ&H(|!{vPV(T@+u)g-z+8>yC^0RRWQ-^7PPfk9w>#b1U<&b@fZH2kl~+l~#iF@i zT?s9TCIHi>{qNCE0PLtabvRUk2nu0-{*Q7=gY~VHarE!x(sF<&DRMg@y+DD!@AI9S z0f8d=%G)>I$G(xCxL963ntSs`Fzust=v`^04QABS&mt>IPl{TSUQ|$DvNvhXc$u?6 zj&vWV@fwoPwNLHd*-t6edd4K$;X&qJb5Dmt);sEW^%|$MD4+`6d{&{ryfZjh~~Cp6Ba(**P2B2L<*~kH%ncuXy^pzTRgB zd7g&Dn8-=kTS=@7uU&2nz05_z>`zuk6FmA0c$w{Eu1f`n0%t^?;W7UxQf;?hPSb3khsyWUfa_9hwL=KwfgwT`l&XBwo$vOb)4laX9Ca-()EMOex ze)*7@JRtK7r7#Es4O~`hr7ceQ2`47+KGAWi@1GbH7ZiL|by+{^ zW7j0nD9>Gr4NS>ip8Dv$7&#+<)V$i$V>#8B#v3DZD9~wKKSL{<(K1XFuC= zY-~Hvg3&_F&%^oK&p*`)Pm{Rai0>|5(Aw!gdLVZtCk)$-Yh(Xv+3L4=WhaMHj9e)E zW?KfG79GPV;4x)&iI>0i7;3^gEOlFZ?s^KVsqKe@&y&|6cbA z7k`Su<04)Q&Vx6zz2CmAe7pZm@D2U9l0^PwX<;v|e9I+q(*zggLkNg9oqOXQs<-cln!|7(J01f}b&1*NBZUnpx zco}_Dg#WrOeUb1;M9&Aew`}j8y}SP7`8}1}8R$m>?^*MF2Go*nB)cTFCs!u*B=_?l z4AR$=W@+ixqUNyLFu=O!W&#o}=Bj}@> zvFj5=Zj_$5p4s~vW!IL;o7H1p4qll{&zEmtq`}JxG9_-gY$eT4&dLcw2YT1EuCZTB z=wgf0yj>=-n(mg4OqZ3wI$w9zt2X~k{#3SF*rCam4{U<6RpVb2gQ}WYOc|TyA|5rU zz|FgLc_H>q+7bN#i#NjhQrdn6ZW&uK+2~oaCc{9V!1#S60dhuuHbdAXKq;aq=qWO2 z(rKb-3~45!q@#SIn$&J^8|U52yPDU~bsw7V@n%P1Q5!JcIeMq<{q(OIeitYEyP`Ap zlM;-wW7eh5-M&tc8g570trTnVJ4*66p9-zWaj*-__D8K34RQzy4w52xhHKTHaj&)81EPWt+1wOx$e zrCiGF*`Y0C&CwcA3h)=vGHEN;Y1UDhOrN6-hvn3H|MndnUh#rE1y2j!9Y^gQ@5N)- zpO1Ph-DmgQ^0Kb&tYz_D^&0X0U29aU;5Fp4SZ7$@WCg5ZFjY5opB9?_x>YdKEyp5n z1pge!m93C{Ih*y_%V*Zl?4Ic~`7|yztv9L%xgd-Pp9waIGsKsZ!*YpnbKh(5PVB6E z^_*QuWk^g&+C>#fB8kU?UNZl%xo}qMK*lwZ8^SJE$-_2US;KU~X(E1v2b)Hk-S}J> zG`MHg`_r&(lXLj>Z_a8?c9j{GSY`HT;0ulR$@bsvDUoB5@elpgZm91+?2k2P4G@1U z^q6jhB`*GYd?|{}Vx!uBY@v;(Msqi1+R~}^p7ZK~om20dlq@shFPT-RwltC;u6)TQ z0p8d8ZNNCY?yWf1`M~FXUY0x4{;~lbLH5Dmi-(zB<{s0!jb5j(j^@e7BHu)gkC_O8!_^GqxSE-BH zho-xJh8IqwYbwrfB`TaJN7qXG%w%i(?JazS`K9>rV+Xg^W5+_q=_=mw=y1DR%(klK z#Q7Xo9xt3`YiVa(KMg%r+E(d?t@pttLqvYwoaXz?C&+(V|Dxe{S3Ft~eSW_}!p&Jd z!D8^-wbr%W75}BdtxVX6ub-c7yuwO-QM;=@Jd52|XwIjpGyhks+Yh(zTg^X0e=Nxd z!fQ5dk=$H;1%es7`4teJAJ>3J$aLvxOZZZ&xv^r!%Az((}?A`j4_4ojshS z&ti{74DGA2=p20+WAAKl1>9-pi%jagPr0LGjEf@IKMT3TgZOuU%=E?z`ID}4yng7%d(ob{Le7gmg4d=e5IJ(CuU{NiZ{y&WH)4PyHCxzZF#S_tftTTG*4BZ zEU;o_jd7K@*~Ty1&&JNmPiOg;cd`~gMhpBJPS z=WY`=2>l1}!@0m)yBj+sem_TWt1Tg{7`S%8vt8x&hkdU{1}3a=^ZhBYme-pZnr??= zT|~FD!{?Zqqz?+Fa=xf>sZ-3ooYQWc+I@C_7`c)`>zb#Or=yr3#KJCc^q%mMAin2y zk`f~#(q$a^{D1gD@Vsihe@c%0MSC{`*)#Tr*N=V)N_p1L^^#81C=w;`r;_gc9 z)LY?S3--U2|Gn^kE6R!dnft%d;(rMJk6a?96|TyO{9QGLtFNioiDnfEi=(QJ0r5>_ zvp*M^B=Ha5|GbH5GOA*#ySleXNFI=As45xwlWxo$M`S-tJ3JGR)fmS=5M5 z?#3-6lY3nlC9bfzvey_gf}xFQ$m6b!eR-Aa?$4{aZ!2*pZ(bt;*7d&w-QhA>TY^~) z8H+y>8-@@jP5_*E95qi_uP>W zDF5?~xgADqFiM8@>VIo62ArTkN5=V|Z&7Drx0u^&a(w?~CdwRLs)YYGxX9KL`WP_S zxp438f0->MaTcqW{>$L#$g0#hx;jhV^+)_yN)w4k`=2RCEdBq9_kUmI|BuA;mjH~{ zdMuVZ4&_^KPF8pHinPd;JZlOZFSqU}4?ZKDp9z`O(AczvUuGT3eHa_{HI|W96D^tqzeXW}f?$S~d;+1`BZ&uu$ z-bJTwK5`RK()r_q=CXzszVUWh<8DvC!n>SRce5{Un`e*bWOh0MaF)Q8cSC{SmijVI zhO0qzeU(f)KQpdDGlCG=z3$NhK3nVU2R1xVG;?xMfYEDY=GY$DwJbe?{3vuKHj*A>9wYOKpyezVWp zE;G{DXL_jN@+LXeDK}`zzRU)$ZztxQzK8?WCW93K!uwW22&)Bfxuc&E z9rE~T@zb675SDECRvjB8Hhoar${v#X!7Q9P43Zg4SnV*i&yvUtSW2_-=WRBIg)RrJ zo6X&BKJIhBj-T<;&O-L?hS``-Afh zb7yYxif5}jmlD72)O=}j^*@=KQ)ryO=Du>aUs8o-t69E)_>7$jlEburhMX^lM2Q6N z#CFX1&B?QPbkcVa*GJmiVC(Y40H|dCK3KDMGAhAq&YyBH7r^m)y_unWb4V#Yc*H zU6CK0QlZXcpIFg?&wf;Cme#Ko&8x5z2CVX>6P^7uW$MoTT23tDr`#G`=}=b@2A!$? zH~nf%O9N3zQ;O{F7k`C+YcK~L)JE|^pODI5{YOHY-GN_bscP46#Am)ppTN6QF!@uXZu+QANh;6tb#zP#HRaFx7H4#U?50 z+c=4$T9EPB6|g*hEEUp|denMFVc@H8uSMTTo(O7~@IWZR;N%tNvqD~1XRJu`c%1NA zk=S!vG$jy_940;?UZf*7Avo`Sb})vU7XP}dy(MG=fvCA}1?hwo7b+}!vBWb07RT@M z>;G$KVNefH$h4iFiGQ(-C=Y4-GpcHI)A3+@@Nf$A-FMh05Xtv7*IPcMMW&rgR@S_$ z?vQ7?c6iC^IbDc)M~NTvwW+rUg?X>m#yV5Sv%%o8I4W116$NE9Lf!Yvg^Hp_dx%;b z6{72hjTrN^j%ase@Ozz?->)NL>22jh)CP=O2qW3^UPtrM_;bfgp7c-~u|Th7I3e42 z6c@;DI!yQ;{sL-BRPoRWL4JdO8+#0K?7;b+@BcP-3DVqIXuSOB;w#y$6>*Gv?R?by z=OOKNkJS$;*WKNji<=KVxzF(5$GI7KGJedkI^uoDckXbrj;G(?IzuP8avxUqI*Jqs=lP2YJP9f>6SvaVkCX-`d2Kp=mX2k(nzGc#*(4tm-q*QnS+P?1+@8J`g^>Pt2SuCpCU z7b)~`u=Hmz9Z{ay)q!o)LvW0c9d_&|v0-`nDFnKPX%nhH4g0~)$_LFQ&^;rl9z@%$ zEc+N_A#BR}R1YlYiaTU0?Dh^1%iwF&K9Ec^y)Ao z3p=g{v-82d*i194IX%c$&$e9rxUB74n^R~%>^1T!*l5fU0Xp1S9sRhKi2Vy7>r{w) zdfKr#>E1szu+?=sj@r}1FD(9!svFNdvKKq@bb;DZUwU3L-Xe%Rm(FG-<2Ah zam^=$Ckmag6BcI70csG8@ATWL(bMM@rdi@ye^c& zujF>W1?ya_DmwsTh2T804~F&HbHwD1e_c<~P3zjJnqSwW6DC3b)$a z^K8=;U#fF3%9u35#+fT-H$0O6L)i!=c_`;|D>d*H;t?zm0)zNx!>>}L$a9IJ9d~su z_H~-r3{wK1xn_PkojX7L0yp=au!|VT2__8XTvt4qz}8P39>VIsQ~XHjciI$9GykYq z$lE+FM}$*NTZB7+_py7M@;}_#&jr3-qqZzFud96dWe*^Qr#o!X4cQEwbSyY{?jBa9 zaCEwd#Rc8kF>GP1Oy9S6Ezx=r?40Pc`F@(9&e~Hb-xO~$iT_ag)w5Eir9<0v#F8)m zBv54d;asso!3DdoKfKF28m~~`?w_B7TLX%jH|7!cR%Go~?_*1sOU>p`1wQKyBypc| zoViQZ5XQIEk|06|4`XHTkz~0Gb9m{pz?J5npR4Z4WO<(yzo)?5P^>tN z@cU|Pxkqz0aU$fjF(@@{>EMhX^(7S5kIF8w-a|hlqN%ewK+bQ4aDHc(15_Yxy1?Yn zucSi8p&R~QTjCtpz0Nvrkp)sv47vVKpvV+L1_pJt#64;M%iz+gp`o_NAf<2BU&xjA zI@r&*x8Kns9BNl`W9uC>YP})GF8m1vs%E(&Y@0%u4P-PZ@lZM4Cnr@wd=SF3)P z)h@ggzDJ)&*$SYwD`$Cqq*XjMHrX{4=@ixbJH>ML@?8o_&f4qYW|lO&5#_c-S19VWIo5nw-MlJctwo1>euMm;5~e636ja`_R#aS1X{p#z zg}K6YR|Q#UaS~W7Wvw+clVYO&8T1U`G^3y7Z?vh0$FFARoJ=nV)+lZMZpG;8a7{Pi zXJv`FOy1&G=VLm&-5A}O?Y&yJ{BGN>Nc+2A%JEbkk)-<~7rP=I-HPWs!W{AaM|g2cdwfbg*GLxO;P1&5zab$F` zDdHqr@zfeOMKIdzUYLyx!M?G}6A3_bu&v`4wA=&raoEYa$Y(y?vGfJ*l!*2=4Bz1K zwUlP-+KnH*ffpw2g3jTmw(3pYJ-xO7+O6?l*`@|SNglfO6B=5N`4^0b2&I_mgMP>% z4d{d4!zn|{z2IyjYHixc{)iQSTF!}HXEZqxr=PHIx`YeB!U+vtTXoB~Ye5oONqPE? zE=<|UBfn;>MZtGwckm&Kjd?(mJlu-y@0>OGm7dgr?$LW;=Yz1v!~XemXlp|oS#C9} zr;BMG?eMs;L%D@-u%JzoyS=j3I$U8HlF%Rr=6CE3Uv|ElEX@=u6llc(AmO)9K+Mr< z0c#i89`53}^X-#4+7|aVL0!ds)3l+*B@BSR@zKhQ&>zL5KDVqB{hIn5 z2=Y@s(x62q$#MnT33=Caxx#>*0nIzaSj3KkL4MX&G%;tGO*zE^sPPdMDD#NDUBY;AM8K1YD}c*}?Po)MuLrQxjK;a2b^%c{V|s}Gu}(A4 zi{L)F0mXf4T#T-7_exds>2DFd-|$)QuaA2(v7}tXmUN8=T48Yt8qe{s0>)6ZG)6%L zrcuJ93LNd+PEMd-%-|t+-QlG3$w`INNRjZ{iyOeisnZB z^$;Ib{mxY8zV;uPESgp`&tfzdb>h^!zB^^~q=t^mzmVL{u21n$raO##I2(tI_FRK1 zhd?34SMuh+7i7D*9;fgWtj%JsfZpOmsY+T_7~8$h4=gXLGt%=G9|HCJ{oE<;sOX$E zN@bVDTfN9oY5A($(s`9T?FEDdRIY2k9Cx^b^ugr(3%TDSf=aLArYDnsr6-u@f$Dxs z9JAClOQ&%8!EA?z5YC#QU)=1g?IJth@Raf=yG+K=@Rid=_KRA3JGzL=^T-E}RRxq% z#tjyjJ&fqaC>z6Hiv;C-V8Glz=8dfV zlz^su{Lil8fo2J>Mii^7N$)vb9`Xm~C9-{@5Ugh-u``E2w)NZ%(#rI*#M*PwT?B1< z{X^dEirJI59o4Uwt{)W9P=>eYHp`?kiqTESdvMu$QF1};${}_Tx~b}{J@=-AQ8w~t zc&_GK?yjY*m+2F&%(||DhapaC$-W>=?84?OAdV_zbS#xdf_mEia-wowOrZo6VF(zo zyGdtPEWfWcj~(G(A2TZ_@B#=}k`3&d2R{zvNhdbepB6$X@{h z*iT!ZuQChI1Z=MC1%90j*`~`jGc}(?0o78w$upXOmuL;TI)%PWi%t74#?3&Ab|CN- zD5mPRbIcYY_S^yi$q_qWx15~tnZ&e37Gi)KO@OX~K*r~{w#;xSkN9nz^uf75Z`{Ld z@Xqr&F|UIrqG9^!q7D-CR80G#P^D!Ez4r7#SJF;Y8{!1nII{2`%ktaYSUhk88q29G zLeZ;>Gywcwv$);9*D_FmXm9QOYU(IXH3aSf|LGJJ^&`r))CSj-H_`tzZc+54;oO5U z4qkl{WBZ~F=xy~#!p|#;;}Ngx6|XNCAVbM5ugtqw%7Z8sVZk^o3h6d{4}@Ml#$BA_ z=80Bdk2Y!2BNsK9gf~n&bUc#DKaIrekRK#Em{sN*8G3C4DHr&aUX!Y_FlOsrxn8;_ z7|ukhCrefJ-9x;i8FLj|Vg8EabzPU9vb0KMc?kS6 zKO2_Wgv2pWA+F;6`U~}R4<=`pZM(?dm7y)IvnMdO-Ca2;*VeS;N)-D1={g_I@iK|| zkq^G9@VX_8_ z3P{D%HHL8xzdLWlm-3v=>#@OroFN9V7Y%l~COy{$T-##AvK>zrdSk@1!DUWz z;-!TrRG+rG<5Bi&(oPUjj_D~`UBp<<3y*B;JE>1GG&qg3Q}*+Dc0wvnzRNGkL%suU zyD?^YF#q&!DM|rnYj+9Ao`pNx&KBiGQTMFWBfC_AVm;qqkGk)BUyzM1;;ufv+ficIx}xFVHc&N0S5Fg5FO zDEAw{@7a@Cyzh=v$PJ%q6xDC~$b4+b`QD|3B&Nry+_L~TDg!${t0!Zr69O?OJUxKe z%Sx#GOUr0lo4a&jTrlu-?k7jB72ZFBVnkpkMuFHl?G~fyNMzbB=N|iFOded5H zw5IuInyUYa+j#Lq#o=d_z*3^aSC}E?KEy>xFU{&?X1!%9<+GUT<@|>D#0C?!>YFXnIa;0x+|R&nz&IaUG$3ydX-+& zt#xTsmGn**+_+EnbzjcHY-d;2*PKxMEI$563DMEfY(a(02GzXC-2kcmcgU_tf|$$K zIH9UY(t@BoTte_KKQ4fJ$k*ypeRf#twgh_Zea^|uyh0wdxboLuhiq!YAK&y$jpp~i zTC*#LgfsrNrvK-`9ebj=C;Sv}5i>1`z6A^2ijSxj*9HP2K}t95hEub2l5VKV*<_pU zcGfWP3H0Wwg6DV#3gyc$~qUCmm~i^qIPV_Vf?iZPePJ=JOP3B1p&p-9(GYqH$FT4!`M z&RAMqQNE=|7oXYxM~HdMRNAl~uc$|{pKYrnBpZA-sj|qod&&oO(#XP@1TrJ&-<7NC zu;fD_bi93;Uj%O?SbUS&ZltXNUI?T=G6h~;n|-jqLAFqcd6{#*Uy$V`M5S`$e7sv> z6z?{8IL<$(3T@9pzE#z#I=rl(r+T#J75Q|NZ<_<<<><%SS+it>rU&Ppx~P)WBK2AUBt;NWWPzT{N`vON<{@97wZf=%b$_ z_H($`XO3lo+k<$EG{}+z~{cg-&B!K~q?1e2z@_d@0)n|{CShzEoWMbA8 zY3GVD;0@heridsjMR~HQj@ym8fy<7W;&Pc8ODrulVQd!0kbcvK4N#1qBm#@zng4&2=2s1Jj>*?G- zIPGDxp}U*n=esc9y!QUQvd$%Q%c95+d$PQKb|C9*6R^Admrc%eh-`9hpUL}I;Wb_* z>Og-pMp>jG`7*;G<@ZR@@Ml83wvq~*I^!*2bnEtLN$I245gGgy@(tESZj0w|Cy2{T zjZf|T6{&qYHH>79D!tMt)qn*@J4uKUMO8R5irlg){xYX79riBw`%<50c93C)NB!Kx z3>!{M+ur1u`ekngKYPgV8-7e$`U^`K6>0WQ4W@msHX076-df#@5oaS>zw^U>{y3z< zVoUW%maEX|d$1E3J3>wuOwau2$R6rzr*^zM`+pP0Vq{`qB zrPUMnkUR)R1yyPRj{=MLS)t{g%*9Up1>XtbGH}pPrDPpm4m*k_njrqhAYGWqU*|{{n7|JkcMv^ep2;=if5{ z=%di*k4U&CztM>nqaUd>a+Zvrw7QHM_de0CPD1iL*wERIa}S1&mj|oZccMfivci;5 zLP|BGXHv|@Nzb2#PZZXEG44_Yv^tqGv;z~gPA0BIn!atymyyeefqlHq#ZFp_UbMbX zN1%-GLEQ#)Y9hiP1wUK|;GBBln`KbHuE=wZ*D0JFh(o1I$G zPp)eWW7b!uD>$%jbiNhfx}^2%;!DWMmw74EPt#xjAA@FPk>#4~B5am9C)07dt5aU*iv8#)ER0RJMdG5rh&@va4!Gy$CY)0_AGvLEnMax)=16eFo zx2Qo^tz{W?g-D7M6O$+nNjgC9tdO0PB(T{B>alEGG)eO;uJXRwA8j*@IpoeV&)QG6bL1`aEBR26^btVg3 zQGxuTs=o~0Xk_g4UkddVvPRscAU9PmEW}W0w(&(WW@08$cID=gXVpWVlv zyuF?%WdA$O?hxEjZt!f3nnkc%g`+GFMI}B)^{f5ZTv*}6c&{< zn2&*tP@4+T+`&1-!6$(o-%(ksNzeu90Xgy2$k!YHHnI=VP)<8>}LllbdR$q$Wow88}~&PDYbjv&iU(18|S?Ft=sDgzLtrr zzuMy7$lN2?tNiV*XnrH{0DglADU3#SfovGY4co_}>$VK@JVZN`iDr&y+z^^+X(ZL6 z36OgrTUu!y%LEUeb(H|%muyqeUgu%vkv)uQbj=8ttta( zUKb5xC`N9LeNLMX7gZtWaT%!;Z^(#iDHstIKx9g`lm^m`#n|Mcd)i*mEyI&PJ!s_G zGCl0TKy9jky~)T21GjIw;zv_CCZFDo{PSUp0D(!`7!{{sR5d(rOJohpJw0?fxf0H(DQ{UBD-*tsM ze0^E*;#jzF(QMN+zuUm4)O5aQ{fwcK5wz83b^M88(AkoFSxmnZxHbxV4S31RhN5a@ z0z~?}Lscu)nQqT6en*?y+@~w9_DXx+X>Ur0p+Y>J14$Z7s-S6cxoIB>pha>efo}kaW!4q>kwBK0OMFL4Rdn z{9^J$b9r4@AolF(TzwBmz;4DjB3< zKI^RIiD-p$e{MNpQg@OYVvOSI?0o&J6Q(p4}1=OXt$4{ z>&$`&4tE>}7an%hS5HZmNM)nbx?Y!a*83$@JO}Vb_((6$w zuG^(&;B9Rpe=sp&1Wr(VDeI|bv!oRb%@$bu+f>jOeyR?ZK*&(v;Yavy@qc1atN)9? znvQhge)XvxsS#Uy<7FSM@4Kw|Gg8XAQUpvJK#%F7mLXR5jH*QEBr}PgsX< zpTat{<%S10%&aGFw-=-xy1R>CAgwzO8TQtmw!@H+PAMCyiZx)Ds#$(yp-7BJ|4nj; z^+Yj)A~GU3a=u9d6w{zwLo$zgWGbFfJqEdMVh_3s% zcX+V8g+c~q?@=D3t5(Au*K_<=xRnp1)5cXtW}83JDo#aeEd1IM1C8S7nEEKBdM%S4 zIZ`O@OfExcH5*JMHBFvGjjv~=J3zjT77E6v&nPB?!yWlOO4Z*@zQcX*M=JQq_D4lU31GU!{Z=2tc^)||!hEz=lM5G_>MH=aP} zng+;=QUL<`_1VA^0U-yWR3TR8K(odaLNtB`ZjQ@S*qAZ)?Zs1a$&=;;MCMeK(X=2W zw&=H;TEujS4wl}ea{FFy`(b9kea1y=*YQ5{!t?gmvYY!uQF-wM&8Rq!9{;Y?;vLiq z&ce!R*xv{W>@WT)I(a0L=#Xg7#a{rd;gwj2kw z6)uH|qrJ7_vaQ^SN>DEAFnK;A07HbdFMm#V-s_62WJti|!nBjX;0cO06jhb`WoUbe z-58+V*A9|Y{xL}Pp3PGrDY-)0Ri8u}^n8c~-Nb}qV1#o{CIq6i_ zWcVYOpn)7-;p5K&JI!do8zF0rlBXf;n!VE|Juyrc-KtRK3-5l_OL_e{8$+@<)>~+M zE^WZz0X0`69DIp#sal6BF`TJPQ0$(nltnt9{0wj}G1+b(Ud;M6rMzqNA|}!`St)qWie#kE>Hn zoKsGJ=Ij*_pfkzt(@)$)QHA(gE@tUQr?K?o;i3Eym%S>2uV8*DNK5^d9CCk7y0E?) zAOC8WB|g%2L)UXhbnUeVNO8$n39nQR)s@P7>iVfsmxoGt^*ouw%8@o^llFb-Lhsx` zU8+bJ<}#VQUaY@E7Z@7OWKTipjCdsNa@37^C7!{0Y-Q&)rU#2t-x3)>N z3a^>?8opzVe9utPo61p{R%fnRFi!iR7G``Uff(1rRVfC-lzjP&Nd;%xr7Wo#VX}dc{JCdl#7rhW%gn}pd9DrG<)dx?aF`=r7qZ(Q!V#3tZ~$Lv1_)x{;!-ZctPSI zCJ>D^u2zBbbBkBnWq;&%6cx028;%0nV#+Xd2Wbuv{|mRqoi4V+MqMdH`i_w8gTh|h zgEX3*@E6okJh5Ly7I~c`Iak@F|Fz;*h=)vjd93=ef29GK^TGW)Xe!Nmx&SJ5v|I(v zWzuiEEHEqs_HO>&V5CZ8^Tp?0N+u{^BTpUh!g2(XU^dtpG)pYlqB? zv==8(<#)okI1VLxo7bc)pt%{2pXO3WEOPn6^ya=!urbpVh$-dwGOs&vTT+XUbx)*j z_tO_Wv@4|BpqeKSR<1Q&=^R5w`*ohov&|Yuv8a4xmhGFa@Qyb<%!}fhMJMewBCH|) z_$KGBs;6%rE7q9S2fv#epU(K{T9RlDEddPO%;g=08;3}4`G=&*qNy1$kT${ zt9mgZ6nbf@3KsgkTWM(_o%GRP{H(r~zT7dMlLC$SAKEw1AMww> zhQtS3Ce>MmP(#Nv(!rb!k04#x@!vAPVZUx;4=ye4tUS@KJW7=!gdwE&sZ4bsf@jO0 z`i29|;8%E=apt|x_2PTPyhNad>%6xCc}vWB`p5e_VyHAI>gD(B17lIYQ>Bw+DjZ@= zf$;P{`L?Velyo?|N0j-IX-$zI(8XtC&TMO=P*e1ZX<(+rwe~D_p6+-^G-{rWwj( z79;ERI@*q54uD?FH0T+9LEr4H2651Emi1c&qOt%W|&&?_H zFI=jE)bUhIeo*C_nC7qh_0>YF-p+uO12CC1*sz5T_Q+VC7&Y>_Yx5gr2F#q=rr;17 zJhI`vDV62-?#NhLt!%K2$N{>|=^B<*i)s8_WO~eb;tL=)3F^6n>W$YtgP&tjWQ z*)bN~&61$hD2%dA6<(Vu+V+uyV$~YE3s6VZ54UF5$0yxP^|PBbWZJTc*08zh_})3) zVw@tHkAc7d4S!oUclFHHI2ugol5DW7As_nc55-=$Ir$U#(Oh8$@PFyDESt3p7{-dSqi00z3cm%rD z-H~;^875=qS=1TatNzJ-8SoJkeiPOTcuVOXSx(gLo`Cl;1+c)_rxYS3s>p5}?PoKx7$OBDVIFu4X_$bE)0aqB-kNS;(RuKBoybRCU5Y2n%8IJ~LqU zN_+D_w@?H~+|Cei*aS~LYkfW(66xHiS;{F$t(Ly+#Z}Xno~BmGv++gm4+lMdf4DVP zsWp2IR~-@h=^EzMpAe%csn6i80mBQ%-8UWM{gM`rI;j8hGey^kAQJa zgf|1x!!J1BDh6I0oM~l9T`ShMs2(D>iy2KdUo?%gJ2ziUJCW6C**tKW@#r9ILN^fj znXzs$V(#j4SITW{qdg%MJYoW}{RGYsJpP4$QMl7B8(Ot1Zf~pMK#NQCL#0{Eu}Of2 zXLhW_n3tJy`amf<$iJ=xKwj-9J!SDF$9dzBHOJh0q&wb?~1$uls#=C9Ij!6A^ z_cP-l16Bs+1flaj(6p}OdlzCQ=oMaA>bg@FY#bGNULZn=9qB(>DjrNTp&Yux z##gQ){ioU4OCaBVK;sL)=Gi>4%OgYhgQ2Iv5N5D%W0}u}Gt|aQTtl+jqsvpfi$=-B zn9cR~#N;b!2j9kee|Plf@mXcV%BG9`*;Q)Q08o5u>rpOy7UFP1UKCuj*4_0;E&d$3 zAr=L^G}8fJKeNL;_0_5Es@01_#Zv zmVD?;y_X^7y0!EPJXdAA?vM5qwXyvG1}Pj?#BZ`Nzf4x}fQtW$!8-5nN+;B?YDCc2|vylj1x^kq27v?1I?^*vX%p)^%TXCuQtCWgoJW6myr zuO$%hNt;KIewWf~t*k_uHReSwZVQlbG97_T{M_eM&|Q158ac(yxoN-RWtG_F`zY~+RE*b zPSkM~v7H&NRKRc}!aj~1OL)|hJE~nLHA;GoDHPZ%q`A6jc2+w1t$Oky`yf81a+Wl= zF>P5@E85Gch8PHZw&^xZFJpH@v^b`HvOAa>KpTZpxZ?z?Re%PFR9$GY7}g-8@lnTx zSv%Vj38#NtsI+5rn=?8W9^>pvlF55%e&m)mGS)$c;&qMO#3<)3QpGX|72*ji<>*?x z3gaN7M3`Tj@2%`V?ozzB+_>1a-~n>`+^Py>t2>HAk~&SRN7Z1?GJ4t344PVV3qyv> z$B$K22H>6KMNWWJ3uhb&VqK&6NdYfDh6t;R&b!ib) zBG@-Pxooo^dW3g^{b>(`aBPY1P)wLpV9Q!ZupgL*7|E|#{QP3_%KP9J(GVtK3^$g3 zHdYMuIfrjiqpqoHmTJDfhhe{p<^OQ^7G6=cYvaEX!bpdN(v8wcm!#4lDxD*sG}0|O zlpumIG=h{O-3>!YmvjwCI&=&$#Ba}e-^b@Xp7(tJfZtlfTASK?-*sKreZ}YgTk)l$ z72mu+r{8bEMFoWQ5?ke7QHJp86Dw4bT1HrPB$Kv3i>Q^aWNIY~4wu8?YsYtngez|_ zAw+SN#|EGql$F=SgZ<-Gu0%Ig&Tw~x;y+TouoP5H^nAAHV!u7~DEjIFI2)u{bwm#C zdc%n&wd_AX)U=f${o}b*#JlIF7JS+<8>T)sJtj^|IsFR4LEp;t5(}|juA6^p%us#U zF>tHRYG%bI;!A!?p;jLy2?xljqvgIW=$-UOo$*ijgqcFKmkB4)u+%tN$ztZNj*}~8 zCzgYW{(vCiHr*v|mABun0^tg4hBfxI8e(S&3+qP>=VxhrY|jdgDKW~f4E^+EUU*zq z>YW}f;LSCrp09mj8zzlQmI<-t*T0@K?ZV=pMXAjdS04>}>nbJ(_tv!#uB7wBM4wsi z?RZ1@zeIC^AAuGx(;auE65EfVcx++LVN(wH=C?P33EjT(&yFf@4N&nVg4UmJL^adV z_Sr~+-HpaGoY$tg!AqaF*&QMP>km@LCpwtg-V=MD%BA3a)SPGwt_^_XG{cbU#%|4 zzIS|KJ0hvU6|$rw=B4$qP$T|g!^YS4Zi+)N2a5yA%0g1~lA1US*TbsuOO(*TKnhz3 z;7ObiBKi2KPGE&jLYipAKPUcP^&1wdSjd%*Fws_c5n2b zQ<2RMq>_pqdxfQYz#V)H9=a4+`P#O4Q>r}dx~I9Vw;R;lW=48m0Op}0Ys%V|={VG# zXcmw>@5wRNs{SOYo$kCzJHo5Tzeo6D{#Zy5l}x5EBTpnzX_+SYhB64z6Eo5&?bHSMA9p-}R%2)qqlJBf@ z0{PdX4DEKG&vw69dAr90=ozsY<tW|v1xW^&p~CVqPl3`O)QDy!f=peJM8= z-hQkdd_u%aMcft~sB!IgCV{p4s{hYzQ8XxFPr;SjC5S@;Q1e58D7d>!4bJY+TNa7a zoOrAM#dyj_#zO;i98x1diMvH8BN#kE%BmI`fm1f`5@CaNf0+bc(7WnusY^IS`87q1 z35p;6cGjWAzIN$iTQC>o+spAqdQ0`_KI70ErVfsvkL!u=A15%0f>b}(LcF~xoWBfy z)wFaWk{xDMA)Qg`lsksyD#&5n8;S8Wz$qIP8)U`0y==%WwS1*et8G4ZiTkSm6X|ex zN04WpaWCHiO-0x?yV>`GUeC5(F8i_;LZk44AgaOuxXq<6$_+8_i58prt8f(?-9(Yg z&`3-??GJ|zWu|f;;t$Dpux0?)2uVnl|0nuHlAl*4_Su`8++Q?n9h09gV1HU*{po{) zJEX7~0a;&Ge8}p6H&mSzg?}Tc;Dj!n+61}BQ}B3z5GizXI;>sJl*&)ncv3xOVmHIK zE9DSdr@i5q{{p>s7Z#@6bgaGei(lR4tQY#!>4MYFhXW!+BGqnJ$JTBy@@7Ijb?pVG z{n-394ZmxUhIoI%uPREYW;7V;>UZC@Jf_^`iRt3A9B+x{7Zo3>wNdMkojzh+iQ=#7 zxcc5}G_;r2>)KY?m2&*+o%}Bp)zhMAlJg=v2%wlA^F274-l{3q?y9HHd32IuoiJix zSyHAqdyoDddv7Ph0o`3)+_Uq?O~J+Ko*TA1j+jo<_*2yDjq+U=cLnF^nIBu`Qa|`; zY>Z%MV)Nmhdt4B_ewwbkmg>ZA~An34Kwx<$s?^J0;sZTwQ=@T3S^f^NC_ScAV~e7>o7irHpIW$MVN zu~Ka0XnG=(qPp73Dm;Q(BInp(QLnKQe9~C`f_)bsuD3yiM|pXCMf77gp{BTS9c;L( z`D5|&%f%_F!(|=SyTtrvHVnMHs9(pE0#y66V=$=)Izc3@dQa)5pI8msQX=Vv)=!S~ z8yX0^Eq8-;x$M1fqdc87{OcxU+UTw#c; zn7r%rH(5n;*)?jGuOO5|;+><;%y$z77L7!_m)7GT`}Gh00)qE*o}{k-b$h^Jm`n;hTgO67<}aV}Up2C8{5S7KJI>aBi<&e}W7FF0 zVJb%qsf)~y!-{;VX-=BzmLnf{Lw`u$e=`JDbNONuo!c8(5Y3d#`q4{UpY%R!pKOo# zhX`}fSA*)lH}*-U{$7jg9VSfeRex^_DbdfS{u9!)RjJ8q*=J)tD+3Oe1L$oLrakn zQoJ_1w|x}GuLyzchX9Gj%WX&cL5bZ)|8LPzT(d$M_8CwAx7xS07Cr7e+zAJNm;e4x zVs|SSDc{$HQFY;(D>zZW6f)lISx>f^B$siut6sIMzUgSUG-l$PF9b`>Plqhz2bQ#F ziiII9+tssSh7NVP)(Wq5*%U0lpB726*(md@!3y&;W{{+XtEn2|o1dE_MCF<7rGvwI z3V$2W5CL|DW&4ClqM!-)L`$3b?jGea8zk zmqnTWZR7opd;GqbC!_i?LBYjPtY|zdRaNJ#^goW#RT?IjtKQ54*FDY%TN z3@r$&z>#Qaku1xOzdv?p*Dm`*Q>^!T|X;_Hm!-!;jPf;z`D7tPMmwu;VSsa4?$bZ~luLXdqoP5*`>!uIzJ-t|$ zk@4Me$-I~0y;Uf!$75Jslq7nqlS&$Dx zv^NLeVZ9$8G%N@UV{6D-sR7F(I&L+HcuMGIaIR%q27my8S7cry_L7Hu^Q=|U01w>GbFt4<>IINwgbEZ}9{u8<~H{1p0&B^(Irz<_Z|Zkkf~YTqC4{P(w~c!49)5sm?!+f=2_Fh{L!n|7MkE*wEI@;jUL6Gg z{gi*RjnF@w;s&Ct4T=wWHGp>Trtp8HqC{6j=dwQiukxqv3GgCDc2&tqVqG2m$*YmX zQ>^HlD8$qMr-J><@Nht@VD4-$XSwJIuaW4&j#^^ifjr!CZ~oi=^edth4K2P7H1-3= z%pjL;NGlKeCaftJf7x(?9+SnkSZ)zK05v(?7}&2Rbn*aH5 z;3rn-7}a!^?@}fSKsl+ARpE=p?GNK`i{6vD7STe=?DE}TBk6s{igohfrcy_HC|(2O zh0zf%<|Fc!$(P++L0!Z{rdbOWS)zkYnZta&`Q`aBER5h@7*HYV9&d3)rcNMlc1>#b zh|9RXvb)@}FD#f)ya1Bll{VFaa6$g(?TMji;L7)c{+8WNC^W!6gRjS-1zeF(GM92MosIvzrx;q;(l|6ylS_4V z;G3a^#w(zQd`9y2l_sZj=)rw8Ord$I;r9>YfdJ$HTBB1R^j91g1@A_FmSL!d7nl{ zgwE;Dm=KAzuOlz|M8<*e;Lt-(&0Ml!5w;zgpPc8d!-a(DaHWgVN4Z3$KE|!ONTTITZ z2Fr#zzc&R4sP8&9`9Nsi^f>Lc;UJL%1o0m$4U5DW|J*zQGyMbvta%jH%G4rDSI3ly z_HEH5T!bn(=+U`KU!v#PY7)wtn_IAc9AB73ZT%^cvOa(;ntkfOaqYOPPfb?!GKDJ= zMkrw?dNeGL`k=8;)@Z#l`sQz!!*8Ra)t4Pu0L?XfC+D3|Pq#`J8!=U^lTdy}U5nY6 zfp2}#_x$FslUE-k65(*qHs5MywMI)61ntdRS*)W_QvBPyDVkj&X1D^Z2$_F3$p2ox zX4h!!(=~N03=kU8@((`aABMF`^RuOB8L0LXD+8hK(bucJAN^ia%W>+oUy-6WcqcJ8 zz!oT+Wgu4kTSattccm|q|sRFz-c}}qbqRGR)lFfuWEqef7SEf_xnpNAENO}UgJ=mXRKWv z5*{h#yDh@b5j}{!HO0w8Fd;=)E z^(QW{-(CFo#nc0!^(VI%)9OTl5qxy zL32lYLh%;ObinJ{_T`!U=j*zB1YWmp=8fP3kN+iF@uAha&hmGnZYc08ZWyo$2>bC> zmk3SzbLwG)9EomZ1&1j{Q5;jPtf`Lty`p~WXR`(LYdNO8>5T+T6H-loDhxqppl_`j z-<+HRzXE2KuQynQ^iRF#68kpVw>xN?{%qF-__H#~y|rN=I=iL?&MjC$S1+B`Xsd{d z0k6l;#egp3=kLLrJnoC@L_qP?J|Jg2=Amcr7`i|%xxPAwUr!$&#|f|;^WZ!?m11FLym0| zxebNKGeBgV0H7cZ2%%ptoFo6mHv}F61K=(qG#+!#Qi$#){cOSt#VuY^G*yuS@xuvG z)=l`B-tQ=js~t%*xOjDCx79eVYByE-3F4FN926{$J0x|n2lKh~HSKqOp#r4oQ}kK? z*|Uw}xO^nS4m09*b8}Y&+l$2#6J{117jCB_c{JhJ1Flozb~B?EhZE+PdEBrtI>VZP zYLuPbBISc5ZaPB4GFMU8&JX5_DDTeE&h5o7-!q`Ehwe)SPV8?x2KdyjrPzi6z9g4Z zfTSblyQPzP7wyt9-yF>~MnN~5T&~cQhF{t~?n6H&FkWl2%sa6A8$_|v=dWUc4sxo! zzCuB)GrX)k0a&4U<0`k~W_EM8I$Ivu&w#_!Mf{el;}V>(TRDMo&cObW2tY4(mDVTI z1~eiU;J~VJq1;pLu+|_x(=9fdhVlLq9FnXs^+3ZaAJcig6v1M9;R2K+z5CP&K!G6e zL_>$o0LI#jgFYPR-tX*0I3n?HK~EuX3S@NFfGjF!BCB?^|H#O7M6rqHF+e69EVrAV zP)idkab?D-0Me{tn;fEvZ{ahCMYLUP9KUkYciSq&9K#}7L?)50b*RMvvf|T6m;Z1W z?T}5);fEznIMA4wG!&+S)Np`EM3e#7+1E^b)LacbC)U%~{uMbmte zi_4)x_fV6(Rj8ubQZKz4=R+q1ybtocn@%9@O@A-E_T(4@DWOW_%9w7^S(k=4mB<3_ zDr-^7oxdcZH7?B)aQ3|m38O19OvEGd${}}XI*!W;2!RyF4E&BJei0E#?zQfr@j+6! zTOmiiceTE3Z`axbw#W9~{H}iu1c8+;!+70}$4kT0e7c-bx8x6y>q@*&+7E|h7E4BO zfUI`PX#B~4?QMYqOYc)@!I$I2$-Pj~Uqu!5ZF>h>KVaM&spG1z@L3#9kd8M`o~wThR%>$j|hz$FcHECit`?@VNBl zARm2<*cvp$%Xv_2-8BajI3=ZAS*b$!3zv>x7Ze();ZO0WWIBXA8bUZ;;~FjEhp?OI ziAbKc&zKwc`I zzDc`(v2_fqKTd^IZKsjNig}8zCoqRiru&{cO@JT-*6E&8OVVOiXxMJQ=fIB&uv-6* zMAf?P53PZGY!?=+#S^?!Wmtkrp*G$Z!you-^1XYx1r}+Wr`S4)C7qB-o*<5YS`>Z<1nw|qJM}dXe{y`n!-_>|8bvkj31@5CE zbq-TK?X|y(8@&0fdtK3yh%k@l!*sv&?-pX|LsVpApAuaD)V5|jWM)iy?VEAdrqur4 zLpm^fq~v@B%@?aY9tTt7Ea}%c@*?bZc1@O1ui`C1dY*M@`7Z@s`v(JpH_V5@v*cVi!26nJ`&)FOES#JLg4c#gg+ z_aK0`yTn?y$e_xGT_C%0HTETek;lB76gbG)8F2Oxb|7aSraaQ@_h{z2*~jtjE?E`) z2<;hh5J%(7=JC(E11`EVjLpPYtdwYh`MEbZyh4oTJ@<(MnD$_vf zqA{7Yr53qajkUj)Zl&TSG2~ z#R~HA{m0*^7M$S`um&FK*P=z6#5;^{)M5cEVbeL~7honse{n+(V($b3L<@@QcW(*C zY0hP&t`3|~nHkhO{->d3%lt8+Vwb1#VSR&=R<*Bw#9i`Yj|w1lCqi0k?d_sws>4?DjSly&kb$i2Yb{8B7=+%xmUo1up}lKF;FdHHR3lWw4r&;lgp#Qd zSpo5ElRI0>yjp&BIf>8Vb(5w6ej@DuLe!=*2?*hy3XpDzYITJa;rKXB=?1|W03K+q zY*_e#L6Ay8g?FLIn9>vk?buUZj9O-qX9 z^nUp~RM>6AYO0+1!Z3rZO`|~Vwp9n?z$;PL)pGdzL|W&K$lRt;BAF&D&!YKMC9q~> z&g=(`-Nv7v;q*LS15VHSvI?5+ccl)WCh!2ZV`b+RmvVH(rLqXp{|5>5IoJZ=5%i6+ zsn=YeCBqGCWF+VOk?|$sYl7eQ>g?Zj2Q(rQ;OX{jK`mTd3$a}tucrNx4U5MFIhGp4 zb=bj6Xn2GO-WIO}V25lh8JT})q9!`gJVj$JL}C`Ml0IQy@;*t$-I7pnYpJXm+L+2v z)ETN4JDCK`5r=R-ImxkgY)YeHNOqIOUHx@kQSbHJGv0Tn2jy+ER)CX!yKQvhi#d2uvh2$29- z&dtxp&?bS1TQx5>Suo!&W#wvy-52$qBsViL=02JqquK`Tb>+Nh-e^W(TQtL*DPUxw zQQ4z_%_;%7!kCpNU_<^V73s;u(b#r~Q2u=_b{{8P)GNM+rBw;>THW9nDJ#+Qf)>Q= zzbb$LUzG~;-(~Q?t{VMm^(19D@qn0p_PnB%ieFjiHo5D6xa0!crS~@XEJ!Agm9DbV zn?4B;?RqmbASt@MWC?@gI1as?>VXn(FD|AUo?z(&m6bLoQ$Z@ZBIIY?9uOHw()bR4 zFrNnuw4q)5_??*pE#8*$vdacv)mdAM%{>wO*od0UfD5P5xCf}Fiph>6!v|d*NBRS+ zf`CF>olu?bjsYY{neB(`=UJGRYJ5yw`Y%f$0Cv17mE1brZDOwk@SIDL0owB2S4(BU zO}$9%P`8@c%o1lA0R@%g@H8OGn-k(JAykBeRY!4t6m}s4MWr+(!Px>FHeO6}(j!-Z zfrdbdOsxdTYIg%8MhApw;6R|+WRbk_q*F`gxEg-k5zMd6XdA=`i#DIr+funS89T`p zy$T^?GZEj+1goh8#aPWTpi;WqiUqgQ|De z!{FU5J{GlS#q43$CPtIO;|p_wd^}=@8}N75v=T)&NXh-4J5mi6MOiV<=72-Nc?AK> z?u1T3_*}jc1FJt9G=0nF7v;j3(RHSf^_9nVl5I>OD9w5K6KvI*UczWJ(bZtwN*6IC z#ipcckz=#gCQl?a9(uYPaEKK%_ctS53Nqy6P7kLZpIq6ET@r01`Sa zLY}dr{f%ZcVP5bh+mVdyREfNBDC-AAx|uSEId~J;A2wR6k9&>rv&{vYA9V8iJEO}Z z`FsJm?G8$Oig^Xig}&!Bd6^)e{L&X4h}mhg^90O0nMTz7E}e#5R~v{uW2%C-8Tj^O z6`J95RR^!L&isW^p6d7U=;J@WTHNXd9+0{)E}w`7*Gf$B`HDXVw2)S>*q?5I(p?bW zJ*pdtYBR%XZwUr&Ox1d+_C1{fjtjL(X#zH! z+DWc`4(4C34j^az)75tcKyeGdcaG;a#F+qs6{6|4r2|v8ms{Hu?APAXFVF z0c`kz*gxqzs5?>J8(BA@&afHu1qPgES~%TXT{TCt*j}M(nC%5~Kr-~aw+Z#yl1I;b z6Ai1ZKQsHz1el;HHl`t_fSCb9qmFsT`#zv|n=^uJFTNT4z(&q6%vg;i@_n#0W43Qe z0G;{s<4se_S%v!DPYVFpyaYSUI_+Jf_>tlOHEBut&@xS*sS_p#QAD(R)J|EPBBo8 z`2CWXY}=9KN)AqS+1GJZ6lnV89Fhx>zCIg5q9YP}Ya%UPrA4ItCVV7RWFqsfV@^Mw z>mC<{pvlwLEL=cf_8#WROS6Z3VlVl7DBj<6J^*&?kcZk=OTn}Fu~tSP{X6A?cJ%{| zqGgtxdOJNXK-JiklK~%F^Ij z<&(x?1wU0rsmp`<=EhN%uaiVyC+&eO$JvLC<^E$aLsn{)C6iUG9}+8zLhsl+LRMx; z)f}a+Lz*El!1_Mnf}DF?y^8Q6tt1<2I3v;|T)bRmhB>ANGTZ~J_O*+aSgJODn_oY+ zrje>00{&rp98U?2>icS_zw7@q6utv7rMjAWQCWMvUrqG#YrH^)>rrH)kn_0Hrfn?U zp!j}ip+kJ;JM>s#fawCL!4`BP4qt>(SXQ%8E%_F+h_C+N&-?j>Zn`TBCG24{`fiOO zr!{M?6<^tkkTwI37a}=|(Bvz8>^>rp+-0o)QBHF}NbU#egSgnE)wGv4Y9W_YxMwFt z9I^`wWXq>3vAcb;Bf{}7I`pi>WS#_Q1Pgi%5J@62Ci zBxeNhfz3Uv)O*`q&;p2KNF=(KH;(017ik)5yw*Wgejfe6iU-Ltw1O>G59BVsaX{5< znBx!t&M<018=psNlD#*ahYWYUY)-FA=-7r=IxWW^JlN_Kx*#Xi;+Caf*)@$1b7TOOErK!uA9^K zu5^C^s<=_yK)S2rYfidOpFWclLp135uP_DFY6dt3D|~)T(#N`?4K_Kq)B9HOyt&OrG7*fQ&-O3&MWxei2t<2a;HvJ)8E3#P6| ziu*ung54>=5*r`D_j%Ey^q!MHJ{klh`8PKeS_@z38qV$;E?!ufhHaq{nkNMLvu~cV zN85?oGdtvkNC`Dh58e^D50nIIN}v5Sqb`*2ckbTXV2&k>zC9GreUO1{%%9BNdidP1 zz$NrB50%se_@%W6hW^}1m@Sg*Ia+j+@fM-?5(9gW$PRYxsc!{%A9XgT=~(g~S$>@@ z9Q62pBI9^H)3SQ$vs=n`ZtB0|Y`iB^XvwBGv-y5i_5AYjHTXJGh=L=PlOd5AKnJ#vuGul&q5$IjTnEo_5ki50 zL5FZ_a|zaGE^W5Q^+Va$5CFd1$BWECe2!qd)W?DphyjMN*7c9NCetuFQbK)PAb>Dg zFD47HS@#982#=gc#&_`O+XwNhpl}yy}Q&S%(5utuowWEH@9?sg3yWM z_UZ9kUtkxjQBu?N-%YifU6d0oNq=a+!r7&;D{!Xyr?(hEiP0Zg-74SbIVIlMSZpsk zfAZe*v*nYis6es(CsExQ`eW+hirn|ItraT$SP-mmTdP2@{47|`4na+Bn6lCGvi7t; z2Suu!PwUaMmqNSN(V;R`dtO)l{kXzDqCn#6$Zj#~{K_hX%ecJO(aLZ7R}AeH{J4aA zVL0)P&2<9e0hd~1_s&(3!$~)>$Hitce63-&1w!vtW_jdT(tR(5f4?~Kqq$gqr@g?Z zfqJlpnz~R|$GGoWpC({#Cnc&$bp@f#?N?!CJXkO|kWECr13=U>sQ3F!ugdWbg~ zHp%org(Co*1G0Ja&>6E$ChMk7S58-6>quK8h&uO=kYKY$zDLk|y=J_GYauO5>PTKM zfZPH7kEn5J0IuC4Q3bc{T%u$7_{gPZF&&+QQU`b-&2y{M4fbH5z%%#K?zLgq=F+Mv zBv_a0buy7EM1D43{hPh(w&zLiF|2~uk;=qw`MQsLMg|pDG21n{UOn)2d){dcaALC4 z)h!1+x9}4br1QD@NLNnn|D!+Rv7y+DVm)!1(dE&K80rP*HV`cIYm6P)+=bymz`RAx zqE`1zM}uSOd}7r3CFam}PcKzi)XW?0T{~(rzq*1KL#sZeM&k~6{6pw$m7%+?ls|y1 z8$7-a@m*ieC!iN z`eVVI`+G1VBtbRW&F9Vx(I*9G8$8IomhoAK)9ZR!UuHgkoI`yKoDH&IZ*v1P@IB(g z0Q>hHYeTcX#XIHGEUEPk(|dWt7#(j3ip%Z3PH#0KcaUk;zCXBvqpXzfrsKutzgkP& zrJWvi*%5n;rSRwp zxj(h`vtGCK^5PSd#7$i#@s9Vdx)YbUw|jzjwD21>+7Kx9NWlzB-4aabA5 zmt~dh=O_2+C*g2&k*Vv}!ElcSukJdmjkj4}1LG0o_e9~QH*0Rtk{baRIyzvXNn0ZY zQezU=yFb_08E5 z7pJD%FUzwPHHU)?rdGh~;@0lZ@(E&H{2B_1B#`9kdS7OB? z&Xn^HqqBd3YFViE6F?+P_pnDo^$TC5ZP)lF7~})MeOMAzEQz&?C!-=r>5-5tp&=L* zPQaMKDe|E6w!o!3)jWYFqPJqca}!C!m40{95Qjgkh|rIMfC}bhM80O`L*=SMvlxfh ztvDmK>o?54(a{p@_nFG2xOj4*Y1Ba%J_uE)>0}9sqs>irzw4~=_qJGD+662c!FiCMpPz$PR#s_Sgs;Zdm||;aN$#UEKa{&jt;Ud$EDi<^a}wQi zXjXw7Qr^%o)y0=@CPEHlo`#$%s4a_(w70J>{a5fMRkXDQ+{3|PN2*;xMJ1AqzbC|U z$yN5@!-t~XI6mxDV+tbJvB8MiJt>-Kk7aRc_Rf`0PsXehM z2gNfppV=~FaaO_vOMNesji6Sa7kh6tYZqTQt5GsO@%3%$1Ca$2YjOl7WMi%nD}cl$ z?mK06C*zMO_FaOWKx<`E_0A(hBJ{*l#S;t2-t&y&c3za>M^q7cfVG%HrnV2#pP-jY36uhV@nLV z)wH7bmPBGO}L%J;s-#zom( zh3OC@#Y$bBglU26!&W($2`mm&MMQ4N6KHsDMIkss{3eJ@TLwGT0!OrNFqCX3TFwFo zlfCQg%sdl|{c!r@(nl%5GtVTmRW{;_0$6@+AO#n{m9pghJ`HxOdmD1prn3yGRzZX}ttRsD)@-7QiK z*ZLAfr42&f!23!KPv}~<#ijTT6pScfJF1dPZZt2M)g9sJN)!?PB0Scys{n^PXY)DJ z_>s6WId#ZpEwwvD$(a7CjOD9mAdPO7jU#RuApEo&NTTua?B;_3P}b}++&}5)#QxY` zem&ecl*1WwL>p24s%_l%bW!Y$=m7<{Hg!4w8a(yInpc z#`mTzt-jnpD7vg#C|L()C0!aY#yHlwx{?2s?BT687#Hc0hg39};Wk^eVK91U<;)yg zIB6OtnUki|hvdPNXXN@W2&Hw5c>*7mQEoDAHCJW6=1qS*aHduIti?@!<@uBDsBi8B zamtADCDBU3Zj%tcwl}X6Epdi5Wk)b1ETv#NC*6JaCkwgm_gXJG$(>QejBYlL(kdl~ z=^Sx5+Cib?zsmUDe4&$~s@f>8C3McCz)*SfwDG|VkG#=%{M=54FJAe$VV@sbtnj8l z&x1lDpyKJ`&{t@&vMrdk9n}gWdC#}+)Och=LVWWdA3Y5-y7pEUHq>Tf7K3(dIIB2z zO2gxCP1gOL>ujtY`mLE*laE^6#1d8#>9aX?4_VlQS`C$3+e|og&@I?t&z{!zXFRXc2Se^OX92e;bp8 zO93wGy~XB~h|Gz(R3aU&Afo*hf<_VXti-r*SN`Egv9=J6xb?Dm%3+8sV@Tk0k9V}c z$~K_}kbgA~DMbY$Ln`L*hPT3icJA->9}$v-vjHn}rPDdj^lNh(8$8rW&ojzDVzt7t5EjuczzEb+ z6m^(LnAnIpsh*N9bCWE8_vV64DakD7aE$s!UtZt?haw6dI$V00?Fm$)-hp!mS><%P zVXiF2-B4UvQ2R{xMhW9$|3R z<#DtvKq%DO1=}~+RElj4M;-)@T8lGqi2iJi)56GqB-Rns6z((=d$9gYft!^r>+`uo z<}!@NMCkn9AS2tC`q!=XBh({!j}@dND?H4b8b}CMZ|6k`8npo=Nph1Gk6f5^Sf7QB zH=Pt@R!eESCTZt3nSLNk4ZSG^>oFhd@p=!Hx9~>FUbRF_cY;YdLr|KSfnAB&ETYnk zQi&g`1i-GlMyk%|?|ui6;xtGCEM|0?R*TRB0#3>A>%$ z#YDZ9s~>@dEWtUs(TI=_4WHO92z4f24F>xPr-84&@7U$u zgd0AjH`g!>A zoo?F-Xh!N`@`wl zi((dTwDK>4HwwgY$xZ59y`+0{wYdhK{&MbkGSWGMyT6QavVHp`P7W8GJLR@dgB2u5 zj62Z~G82`yMV(SE!yvt7jGyQ*^oYZ}=m#faQD>?}00Hw(bZ{Jw?PuOi42gKGR8zRX zVQ!(5&a;hB0>rOHPA21aCTyV)R&8+Ut3?lNjTbR%r&VXtRx?8#r5YH?EJB(R{D`Sz>KrAkT(o<}*y zfk1E#I7v|#Ag$< zKVr8?*XwA)jCmNGvfjf&1ZNKV7vr>D%h!hUM2l$YF@Bj%^&>2}YFosWfX#&)P@&Z-G8 zt9C3&qlSQ1bXI%;s`b#8qkIWLR4X%DBR`osV*7_WCJv-xUiv7^n>FuE2);d{h6&aQ zS4eR#4o-#YIkrE)W+rByN-{gOjMTt&SNU20{e)Zs7w5)GoO_mOkl$@*u6h$|jSJcs zT0$w}Cs*MXn@HZ81||mz#|ru7sbV9ECW#>shsbPLnvOa7PH5S_{#P!IniUBb1jF3z zXs!EQPvo$kiBgD5aI85Q+21Ygt2zIMxv#^us3G5Y2N{gf%ZTXnRxk=(p~|k)Kr34M zI(t=UIE}6SYdm7y8a|T6L3`7`SOEC9xW&Y#mseKir!bzlg)RQuhJnKSjnA z7M5I@`lk)H6Omz(wutMUPs(!^dlV!Z@~1No+2)$0L5W7zFlz0vQ#J}Qhy!L~)cS;A z^J^iwD>{+4eis|wN$H&%8=0ky3Fn2)i*AI_EX5fn@b^-Bj?Jpu%93z3lceB|n^ zHo4Z=fUNFg45otivi|-7iLjp@pUQ@$n?}IuPFh+$`>frw&b6#2a)|M=FKG=-gDXhK zkQLGFDs)qs=%=?;py6DO_S{_{1v%&w6&1Nc9l+9}y5`#4W0G$rV>@k<5EFZylanLN z&u>nCQ_NR7Go-aA?Y6z*KfE>y3@TV_@b^%qz=!O6j>u>OT5oEHPva?MSH_rR95op( zeq@J-7X*>8VfHwiT(6s7A7x_Yq`J4W#)Um$4|poq6Ix2BIO!mdJOkqnz8kuCj=)e7 zJgLK^CFO$iz^@A!&c3whIgzOwl>oTslxl=OcH?>J6S~gIvl_9Dwt_hId0od zD2+VW=e%9I*fMG^0#6Iaq@+?toM!m?9D5d~&m`MM5I-heq4v0w;Rov85A{h022b4+ zTx8nb(IrYW6Gg>XYSNB%jbSCt#Qx;5lG0qrYX;)d@$qNt&?jnYuFDG1W@F~zkH&F_ zi)qfA)7=)W!C>%?FgJnBGq5KitHJxKKb$DxkGM#nK~(UM%1AI%3q4P8;)1iS(n}pf z8MfocdG@%2SoQQDhC5peSTxpL$g?-hly;^PmQVze7|Te@U8lKHb(7q6+VbG1B!O@CW&G)8H|e z<-t=JRx}+?rX<4C>Tj)MWv+{2$u28~v#>j-ZmCv2D(3JN+5^SA9* z6&7-;YHFI2vvH^Za$0iGyXK#Ay0wdONs{Q)IJ``qt71nX?#TX#UhDk^gqS~s5Tvou zfXkvFQZ^^f47IC+9(h<`S)@`0Cxmk3tGTji72qzo(bs1G!ll{5GNP#|ZK-K^j4(LN zdcH_k%2Uhs)EAT(qg{cMGdr!^nC1oHY1o4o_c8Lp^CFDq7V~iq zJ$F=~Q}6Q?C93p6MG(F=uxV|@4d>;UR2Et~I(H0gW=l~Hl|5ljF%auh^K(kqii{iE zHMo*2H?K;cFb09Wy=UeJ2L}zrX~~G$PcPT}O9>}6zY^88c-Q^2i-0b@ee+@~m^g3v z$9Wo;^|%&}MM?_e6Q`gS3dNCQj@q@3GfAF=pQO8NyEXcRz2Bmi!a0WUW*o?$xO~oK z5u22)?agV~qN0{kl+DGvq@?7B2hhb|t8X)Fyjm)8-0^hoFV4D`PAILQmM-T@N=w#V zZQbzWar<-|L3+t<6|%&A$`Y|Y)lsD0I-uRg9iI0R} zD{e6_B%()fk^&5E^#+lpajgc#laPKbqwLM&p?v$d@i3OL&X}=fpBR)FCC0uD$}SaU8zhx5cG-6`h(btXX(W}h zBpGBIV@tLcOSVBm_FW;*<-Whq{d?~F^ZEUr`}a>?^+zw)d7an$Jl@CgK8_Re=m!`E zm)KR$(nevMwfn*v*D4y?&+8=p`#Ln;kVLVGJM+p#uf4dc{HPavV8#Ml;veIeCEwVD z%14?+)-`d>>X_B`wjyT4kIRU95<*9Vcf#;GBsTGDYkKN=PH45cWwiC6i>Up=TUE4F zru_bvoTk&`q9UX%)2MKJpTgScA6+$k^y2wHDS^KN8gl`-Z0$Y5mGD8#PNsE^f@ZOF zj=1{l@>DnvQ(@8D*dwHSJRhKWZKu^2q)%Y5^|9b(xnjd&ZysfRa7tgTn|*BhY#u&A z_<{b=KJ&S>!{-Jp#Y!jW7tESEId|tXw7#!~Bm%I+g&1_flpgVkKfO(pudR5kBJ{#S z`0=;8|8Zclz_7Bg>C|9DWPdm6d#!-Yov{SMI@DzOvrhBSi8e8XS{Y_tYThUI5!*C6 zh3E=X%?w+qYurh|Zix*RBn3jV1}C*=sg9`m!}cVPD-4G8tt=^hKFp01t+t-eCe~@c z(a}Rsr2=M#MTD#*vKDZp(Eh#!M6u1Mv9DSw>$t`xCdXytj<_@@nG4*=_ZtW&$_eO5 zH9%18?f;xZm|!*Hz{{#-wCyR^pswcDQ;pA1&EifJr+!hy>O+vaAt<85saZWi>i1em z=K^jG@p+fX`|poP2OUZV@uKuZxrKONF6#J)X0zh40Ihktx-78uXaNPm%oUW<&6a=YVC<# zd@AN?WPHRJzh1j=AHdT+l__U*K?Angt4K2$iy&+N4V6EP-4WLXX1=RPz8_%`ehIwY z$C<|(1B2)|jDKWBh~8DWX1XV|0=2e4Ry{i}HoUz2L5b zQ~X{r=m(}wBxwl~iwxk=)|1a?+jO3|B6=Yk|MhIibRuii7B4agLPXCvb|mLaX6HSs zxSHa6ch^$I#^VymIr12z1}k@-^n2nK;r``XouRB?X!U1> zsERU0m_5wLL{>|R8=3hTKeh*K(4Enr9VBfrSqdm62C(7dX*4DD91&+7K3xnRzYV2> z2I1)UC%*~$8RAfg?j+z!TH4RdAY5K>(SKB-wi8XHg|=nOy0}i9!)r%|c8oHKI8=8( z1Cojs&-YGiK&gFAoV+g`<${o!t6apsVYK4**50qbH7z{&Q-f7sa;s*U-9WNxC%Go* zWP@Df)IyJ9$9^yrAI?njS=riV*TP3`I_8I3>)KUzt5 ztGTL`>aYDfM$-l})XsuS%SCEE0`*fTv)1gE^v@1L0?>#lOWIhtn5E1AxuVOlC4jc%%|$e8S!v8 z??zq40H9;TOyp*0u_|S+yKm1ucXf!v$yp28dTKrTM@&+*J-7GcW<#SsR? zSzspyMpamy5W^>-r-7IChKL66!93L{+r}B%wF7jl>%(6w^oiOMv0#x5zOnCM7*xx` z!vdtFLiZ=~gX`83QcHHJ;oo4Zg!+?GzB#MQu@6*o?Mxycw7dszY|gj24vM7HG|VJ z6pT*J5+(z1<$z%&ca5`cOeK^=f&Q&~<7HtY_fcm`U)KE}JLm|cHzB1^%8gi;Y^>WY zf2mQURf!2IRM?}jO8r#xVD0tR)l|#vH_+ck z7|1!N1U(EtV+VSC_fZ`FECSpQ{f%QM{yY?&8u3D{*?#lXPR}Y@9;-NxA*sN z1^>Qb{y`K_K~eL=cNcWHFe@ET?Y#ti>Oo_3=Ctps?csJ;Bi^2Df4Lsx82p~W1+S#* z_xW^&eMGcPVQ;^JLMkD+4LwtM?04%4$!%5xrVpj_4cPDfHyrqFTfdZCz{~rYzHO0- zE+)EARzSIMsh_L6g%zVe<55u?cMCoGj!TbeHNdWHu0 z-+=;D8e*!WO6Nb1dVrrF`T`27I)sxF;@jL_321;#>jC&E@-BWGDk${mO{7+TG+tb_ z_C$N$w{)*PqqDMfVUT41Yb7v}3zm`-V)$AFM$7Wty4wAkPD&djj>-7`?3*uh;v%~c zOg8!lt5UUWh2PKlG=3&MpYLgqMvhWKP#`h6Br)@tUep|bygl^L#VrAQr6|krL4`>b zRS6T>T;uIInQWnNAyX8gCB(905N}im2hpsx+o7Y6c>|Ud^t^0dC z+mmH0?$DbB^BGw8Ln-H_woNg$wiK7Z#}v>gH^8TPYsT}&5#H=aCi(bKFs|>`h`0Z8 zGyu?rC2HWtza7i}6e_VW7)vYHn0MS4@dO4t?m3a?23i!3WC64^8I>mecC>j25;vu+neiKPmpJ>|F-w*le2y&^iy$F(qq{ZochtA zu?VG84PuTX+4A3tk~r1H=5HcYieZkACGZqmQmCv#3VYfV%*VWI!ZwOWoEMY7ht+=` zNx<)`#1|z1SK_*7K0+H}uN5oH-phtnAOuD7zWe22~<^p}OVra6p0V z=MTONS5gQbwRG>}$_np$)2*3`4*ID)aAAq-YvGr~bErn9B-R%_dMCVd1R}Cetkl<3l)Y3bXZI$0n!$skFM zu2Qwtpn!np8&Jcy98gSY_VJ%op<39F*%g~`vzM23FRj{fF?Usd^wsFI@n&{zqt75z zD%UOLb}d~Q8%kYHRGDoU0$WafSyty`xSajSISyx0UZ`ul3szKdavmbUs_;IyW3?Z& zHE^*Qt0n4EsFr|kVipUmGunAO3XqO8s#MEwLV$w7@$=iWzbP1){77T_XcTG%MDJUg z@QV@88YC3FoKjKn$P=8YSPB2d`&%rhw*3|3&itM82Jh3j>=|SNneAZ=(oG`AA1P&8 z&+1-!ZnH?}sJfJ?I(=>^cHs)Pkv@(PC~kt{PV8ahY|{=~(vGTA)nZuaJa4bYbn-Xc z#fvK^xGgyvWDgU9BZK6OCxeI0oajl^ikCtv$jn@zckIGaSM9+tg-{>=n-7kP$NI-A z{l6(5cqKXm>E=orCQ<;;RS6M&rXmLah+X}|PK3Cm`zoDLaXEm94{0>*BHK>iGyO>3 zx#9NIM#LZ>r`P{Hu3p)e`L&4afVYL%J_M2IYbAaL{%wPQ{*T0LXu&Qj*56|Fe_BN< z2t8iOQPhhv5YijoGB!+Hltwm%5g=M(k*_Wjg*{R#$YJK$&-JT8N!@}t2WgWGi)`%- z*ci82w)Uf%ObBq=*iNSd8iG5~P1yCQX~aWrdKmoOmOn-}xDa`O9p!FisTF@_VW$CB zaOU1mPO#G-3KYQnUG^a(D>KGAbeB{6`qHLxpj z)c?+2H53pB_)Y}q+G7B8{)gxaiIRpD6lg0%7rNj4q{`GLjy4h4JAr)mgO--f6?Ra1 zvzcE?>LP8y&y2lrKNuLg`jdg_VnNz!I_Cs29+Z;&y4WRJI3C<|vFSBDmm9$AI#38X zM?US~5#eK=r%@#}Vf4o8G~Y5StKdBdf}hL(_1&i1km)aMPY}WjleMWU00wBFA4ynu zPZR|BDNOvaI6^Ha3k+vukzKGS8R=^qI;h}%C*+;5TSj^;p%n%o9;m*_!~+!cbFnou z->stuGqoFm{mp~TL9h{2>ZW=d(cWtD-VS1%fO0xra_Ic#vX5TA8xm=}z z^t!DVU-%y{&Exv!iRZ3MI{PtSh^s=J^MSe5Hl6b$&C=z!ns^bIRb^&2#B1;5nI#MT zcT17?EeSnfgR__Q?mzbwdCO!Ht?ZxjO`4R;DrN_t*MGHv}yJ!r~WJ)nP_ptO#CPRZ!I(I(TObo-=D~MX*#|UkY$u-yeC=`{XRQS2`+4| z^E}&mk^4}uk)ckAKbNZGrZ=a=sbI_x5ir`If5#Aw;Q_^8G`Zp{(Kdb0W=MZcK=p*3 zqSXZ1R|mJ}8g$yYd?(AQpyQ1qS+3{n*0;OKS4?7e*v*E7MtLr`&U=o97@xhPHOg&Q zo!P-%fC8kxDa{juwtal`PKYhwY`%QV$=NU^Hv?5M^%QuTs1r9tfp2Kos)Qkl3HZ4B z3raoy2FhuFXFQR4Vga2bO$s5IfY$qmPLrJuOggVSX3`1WU~ zVQZ$c9Y3BboydLjIcNN^N`NJnn@GZ_8f8^sGPO^|foHS%0}$yj zWS^8}_EI2)3dY>YjYL5$+No0nK#Ig8o~-~6Gyl0HCP@)^YY&j6OaGy79RYCdU4UKA z8G{PRo^807HjiX)L;7!~sOca?oMR`lu#8v3Pw7_com{rMe?DHj*#R$6E46xDRXQ{% zWtqhQ#VF4EbUITm@oN`5D4ve0V#%)JIN_EL^nkYH_3@p56HInhkm(Hzxf`4E^+NGT*gY1N$thVuOJj zX!`E@WGFg$?r9<3!ub*`-^~y9Qu2;q9g^2eFAN4=%*D?hpa-=oerrpkO!SD^dAZe< z%C$@Nr-TE9NAeOZC}lh+@@@evPm#_j_UR>;z55H9G*#bnd3n*(71pRT44n}DcPdit zKRxEWgTQaM^s*yeq8WzejXqTIH zJFoXq1x&FMg#S$iEQc0|HKQd}d2@nkX-!z!UXSE(KcgTG?qR&^6vp?eXVZ4Y&EsZF zW{RIBn+Mf@zTp>VgihIorH0`-3GUT<9(SC1HTceELNP+{cgEHFe8{E{CZw2OEKzU~ z6*P&fs84<_ReMI=^1J1Z{z`}za>2W4T78h;>TeP=8hEs|+I*zy2{IXW`NZ9Ttm`(HHQAnpWhQAdCvvY@Fig=hf{1bTyzz%6woHq@RFw+$5t#UpTeFQ= zqUf|$r4r%!i(}hTY^_5xLs`Y!KSn-Fw{wbJVa?;!%u>8j@E<&b_UP z(h(;_U(^rJfg#QGi1%Hb=g$Hs&p+3_qPbVjIDNMpt)SMq-{!f#1PNRVx-IZC~@J}hengv1_9fi##0yAEbndA;gK zvvd_Zm2;t zu=9^#*uzzN@FP0!kF{xbBc^U`QCNz((aP`miiQEir}j?6c8t43Tz}PjuZg@M@nV6$ z;x^fyzqPfO)1~pGm`LuoTsL#zWAipZr=r!=+yP~2dwaxd#MZ0c)w;}~N_jk}WgHlR zJ0&Og?&(tr(cle5;#B-}R`_J^3f^DETz z`}glQ;o%K4KW468L*2Y}Yu6B(e%QrlHs`A`Ha6yos~57evNAKF4FW&9S9a?BA7*5b zZ9Pv&O17H*k#`hb2|+Y(Zoc*R^CLGO{0J~GG~{DsZP0?S*cceZd)G@AN%B&ssnn`N z-N|5$$=C&VclXNDQaM3D9*S{3#+L}0M-RN+^gAUjjf=SOAV%s7MwfUWHE9<2=RZLd z@a;vJ8aMISpE!97WO<>qoS&1iO?ivtNQwyJu7`p%nXGlq3=$vPl$0 zL8?R)l$%^SDiVq6U(wXN3rDUFKe3$xT5 zPUMqhLF532FX}%*@&?pRv0TOX+BfU7LDbgbW4uH9#1O4?>QWd1yx7X~M7K}!gHYr## zct;n_)jWc~`?R>ZY!`BoLQPCR;pEJik{t%rR#$9c6l{Ihl(V?6?OPcH18;Gjf*h00Et5*{gZe)LJP!93p>V zIyhi?biNJvUvgJdd$o2Q?a13m7}qk+?`aK9U}f%kBZ(oGh4TDXX+U&rhGWpK*ORC4w5 zR>MP7#Wn7j7ydGme_+`_==*_+re+){@s{k?=Rv!)QrF)6BmFLjWegQEt2g7ujE z`_*VcKd1#uL3ZDlFW=j$+y{w!kkk~v9edK4m671;O^ZTm=T$ibJ@k+y=26(Rvs;lIPyma?5qj(e`du~fS;wR9pJAK@O&69623)t!1Gw>yE#s%8uBOzf zh4$05?QNZEF7wEq$2S5VibVqkfodjHF<)wLdhf~g=xxx4&iQQMz@c8DP`Q(A8l;Z) zw|y=B(2FKQ=y)-4_Obs^rqeVeGg3S6JiP2dCs9Qyo`$>37Enknir&wSY7(XH@{brT1rzRDI&1?VXi zS#kpqa?;V0t2R5z;iuAftJ(kT6+#Ll=koNgjoGnnfjvyc%=gapZRLm!<$xa_AMr#p z51KxiIXq}QbUTXS$+e*z(B9>T|GuRhWM@Y%t#s?HZT9!EO)mW1rrh8JT$1!MDRs)* zJJua|8FFu~mHq9@I5sPi9j}O1^3cE;j4q2*_1+FyWJ|t<=ds&wz?4K9J4-Ru32uwa zy^v7*8g}|q1Uptst8B%;kP$X+54i|vH1u|YLPtoWq}^xIC3Kc%N9-s^9D$0K@_X3gp;t|{f=C^)oUZ?3wK&29RQkIKcX`imXi9yZ52jJo zrN-wIX}Z=i*7py9^oL;}%x>B_^~n7!*uFegXD02otU5fHIP@QR8OaviZ|-2i z!+2cIFU{iSB<=S+jpkd{_@~xZSs1P!o+vYUO~NEL>D9nrCjWu5zF>}}^W}QNQmhuh zropWY9ii?b$Gzr{`^$kc?Ps6v-EqA8O!#HT8)p`7#yRE=6Z@YVc83~uhsO=2-0Z)= zr=NMg%Ab1mP+eVOf~{6h+mD&t7r@=nV~#P9`XJkY7eJckg{e+!9* z?>;&BJ=ho^$J|@gUfl zbXob98RFmu8vRu|YjIQUJG9kZIiko&$xB(5irBS_|KD#Qa_zj8bX^|tQQ(?s{^;Vn zZ<*qPkBJp)gsaB-CRV(`$Birl!kpbob1B$r}vse9ucF)OT}Y;pFE0=brv@w_cIi+pEZ>dVin#WVfl`(&)Zn?TV;e$YKn(Q!$PyWoJC3 z7Z_$?xT!s6fT(zjfW_q<2m(K!0 z_x)DewG}ontkSbFVzW?GfSHk~JQ4bN$q+Gm{KAl9qS3X#vIfAv0A9-O5ZZ6^L~eWH z_jd9pMQm2zss2Q1DF?w1gSa>O-fFUS-m3?}ZF8uA`Twf5a{*443oBM#P?>wkdPYv<`P+Xthr8h&<(FB5-~IP z=%4JVh{0)R(%!BAvPYek+=OrGQjXnDwHf0wB zp|SxL``>&;AxX+~y?7)mL$WWa>3)9x3I^P!Wf`<;R1Lg1tr4&A4Z%U2hH-=i@JwOoN0qwbzy@9~fIiah!vB_Q?~p1Nz>H74dIU@&d*#g1K1cP9jxM$V zmrvMD=ouQelWYqL3Yzp~`Yp8+=!GGE$iA@E^)7iqdytBXO44%7jEzmkL9y0BrwVG$ zjE464j!&{DWjD=9q@-&783H4ey~f`jwO`?@F{1p8&~4SA()gK$nFbUhF-DhBpFvfU zpRV=H9bOw>d$`RUr9hj~Ayc|%3;2iEFOT`=Q>Z`>Hv?j=Ma{84nbVnN*_n6ZGshLo z(KzxFz9!9P7tF1fI39c;$oUllD{N|%`ZweX_5h6gl(!+ZvHLK0iXkG<4W*QK%`PZV zL_WKf_H?ZnudAA%fIj0fjmKSU+WSP`nHkj(UpVI;blW^{Nq_R3R66W_WVy%4)Jsx$ z!rq7J?-AZZ3MqYm?p6bLnxAfDNFKedmtl1x*tnqp71i8(@dipDg;1bTFF)ZQ4g6BS zp96pVQyd3Yy9?~i3TbJPcGeA~KWbX`3^UixC!?kcTq0rL!y6}n`!!^ggYr2+ZBhzT zc;{n<_x@~q9S*c2`b~A*7z*>Q$ODnB-U_Bb)^W`UuYPQyVe0WZ5orqxh*Q7f7#NPS zQ9n2eCN<)8%vUOPC61`_4#WekFj#`l2H2qZC(%ycFm3)NF+# zXHt~t;{2=j%0zKUObX+;k7G?f965APXKmUkD=LbvJ!SuWvu{Q4q$BZr%Le73=sa~D z@O6}2n6LO7*z!s%YHeO+Q}5%qRci7Ur}@>1xeQ-k0CJJy1kK(yt*iv4c%L`Ld){6W zsJyjlQqD4x1^0RwcMI=jRGrIThf1tG@cg8u>B;bdyNQq6S@#9s6HoImMvzmCiCgU@ z=QQb~L|9=s6Gf8ALDFxBIv?xo4jcv3lxL9q8IVzBPfNlt|6zHIuc-#O#2TyGA9TQW zhjq1urfF~zKe{!aP0F3PJ1acs+C;#GxA;t6rb{-T!l;k2%fA0nV9{K%D{yaf0)P`{ z5G*@P>enzMlm#?uW2{|(YRUs|pEEHrAvbsav33mm3T9l4Ea2BMWY{}RzISv>RAt1k z!HB9e8P`IJ21{P7+l+_rc>rlu;j^kIrIY)Ho1NV`!*fX;K(_Q!+dV-*HQB9e+*6Z! z_R3*my?xYi+q&F68fvC~g%!5gS|GMj&eqE3Lk?K_Hg!6w##;V+Z!1hn+M--dk3zkD zFdR99xKADf&5=9mJ$y#*Mt9oAoc$LgRVLflst?^;b7dPwl_pY-&TIpV=TESDa$I$&mmfx$S2BgJ$UhOigJq73DJm)H z;;6kMVe$%D)c!a9e)%6z4lVTNT&!3W$teYyEQ9Av_GHyp>p#~la=~u2vwND0l^ZS; z4H0{T;|aOo*_5)>QIS64E8Cray@48JjJaCD3k`zP`z^oaiai=_l5+vnB^MGrzsil4 zxeH7~0m5c8fc)4E*e8G=1T#5(W&;sRVu4b%Wxe2=mR~eL zJnyzbz3nswz%t*mFdL`nmBN@wZ@|J28?9!BBFP%O?pkwkIZ&U<7v5Jl%bvat-_UL2*!u!{!n>K$VAq?bepG$ltUtqUuuyEg>!^EbS29leL zh0xJO^TFcV(xi2C5OgFx^)rXY8!>XVw+tCJQyW|d?(pl3@0tS<#9LPbc>Q1E6 zLAfy=O`v zpbER3QYOw!+l_j-+|WEt1^H)9^ss;HoJB@~yE6dRto2naCDus9xolcE>cg#m1@C-2 zJ^2rvFyxT0rHu_4@qwVh;b-z?V4zk9h*%JY$^LqV(y^o@`H)a+(w1f^8v4wrO;Y7< z%g3K_v+=VrW|>FvYNfN$Bk8D$oD7lwa8LeY2Ix1GrABml$Sr&Kz}G+FvNFfe9QucG z-?`>@j0TFWr%#n`UuR_hk=A>z({Dl$exs zT1Mtpcj;(0J#?BGV1Hn_So7goD!NgjKFwY8N0`Am%Vew+!=A|8a~ejSxzS0+Y_;kr zfO-)KP`&rKz|E1N4l#G70EKH@UjXvFCTzmHcYJZV@{Rj%>WU!E_~$@65p~PK;wkpm zIjQNZ;ieTXr<-Fav9sgY2Z~)~0teq8I)3J$@XLsBF*CBV!W`t=vT{{fCAzxG=C2h1 z_rXLlq+SX)-QG9${qq}r`4aHP)q+v^|O#}LGzHQIYu*QYj?0WV#PC2i@6P$43JN=sY@TqKPK_)n56 zNgpqr>N2g`XK(zKKCQq#>Mj!2yJ-eRxvX4}BC5$YYB~$-%+6

*i$IRxOv0 zUS=|*9@3b#$`cb=i?O9u4vEt6^Wg^zj%pP#B+$+Vl>iG^|9)XmO=pL>6`Y7zEXNkN zY8`H9jZ`;my$;#-{RM+x1VY6`YtAfxE;b)|VO2~->Fj`Z<}GjUcsRNt4TE-!ml zhn=jIJpoi@-qRXVf49!&k-jG?@$NbLsnq;3|8Ju5?}qP<9FUvt^9kfE$@ad5srlNO z!1fUDgPwuN(&|qTP-n(`^XBtLO^lgvIn?*8p#6=?uY=l7q$CPv=Xq|4J<{aqn_h{p z4NR86pvebwLM-8-y7Sfpz!juk35VSDCAmlIf|v`F(UbX)AG;@`r?3~4T=ItWbPO4p z>!jbe!qlZ%_Ll=2D}fuUzWDg~5F>S%>5X%Doa#03r=2lyd)8zEsV>Tl-9XOrW&sR{ z<{YOq+Re#g-CE=$ei>U|G@Xog9raV}8*XYYO$OeSU@N$EFNRAKr^_M?o7n%MoQu~T z;eDNLM{e{xg^0ikGtLkUiLQ+*?S|E~ndUic@Ebz``k7&KP6-Sz+lAq=A^p<~MyjZ2x|d_y>ai0vzc8$UTg1M;2%5 zA~0N}5OiBcB8Y+!gWxD2soydZ7RlJ1X7h;OR#N<{Dz&hon5IcE%f}n<^$eMqav^>; zxU_uzPoZN^^b!{5`U^cj<+k<-`N`}9O;BJDtLkxSk-jHQ<1DPl$ERGW{{H?dJ52iA ziG0AazwIB>PM~9EWNL`-w@?>JJfRl2{1!+u3vq8Vw_L=|E1oEjzK+YEf4f`K`1{ms zIS|vJq3d>@%nOE1B1*RYMq_XK272Z;Yj0;Jij3?PN6uPNRn7&b*qj;V5|Ro4?(YF) zRk2vGc``N`rq6N{mQ+nV{S5s?A2g622}8!?)=ra%nsL&TvEVymWRq;PuZ5*q*CSwB z)9ggRiZLKE%x20RGb8&_Mh;@;6LWMAgVq)J$wX%OrJy@-g`vSFyfm+?Z$wS0eUL_K(!x$G6YtFlvUiJcpuK(W*TF%x z9cp#sfyYsfMJ}H5{N&0m#rr2!Fdom=~w9A$COZf zTB;@CfipnM4FM!)0MBXrtjHJyP(A{W&rex(T7 z@$E#x4&V^<5j~B_;FmmyLSbdAZ)|c8ni?`O(k03)-YW>ATy#%7A+*?75Q+QoQ9ueC z(NIbDPKno#TbUbN^wm*w*&Enklosy4i z;jKBFu27o@dIp!z17{Uot3wQy(goqj+FTU$d=!eV9putBHwR32P#ykHBHuv7o0J>V zZ$!K~`&3qlAakg~fcs-4m5094RK!M;)(owz(ru*}nQM8_WX(j_%*@R4Rw&m_>M~oc zHu?1SwZ86us4RfUm%B=9z2rRxxG?216Qh|Y44E0@_)D|)>PHnOJ+kf$c{DHOsrrAY z(ILxEIslCEp9e%Zv8S!A|i*(ry{MNckbX7{OrOIOF z?)!$pYa#8gLLxW#1Mh?YUK_cda*d+-chwRtiIo=zgX{8OQRUqHPf-f&PV^}Rb1|b( zGbiU}*{>={F%#{KAYdRi(HD^S{-yF=s}&f^YMQ!bHg~r4CSVc-Gwvg2C_ztWfJ*q~ zEGMCdH9)6)R{YedF2b?U*L=VHHh!4$eWu#73JUpY2C)tHKFG?NMeYJ;o`-zl6BMz8 zUQ3$lk+k8kJkS+(JrFbo6B^B|7FmhQYl-G$O=L^HL24=o(yUI&$au@!{?TLJDXFUZ z3YJX1NlJ>LkE?Dfn;he)=hJ#uz8Brz9X6w?a`EEDEAQomxPXQG+wT)n0+K$D%fkkE zHn%yIdPPp{L z8z98s3SBkEGiSwkfFw`Q4q~y$o)ZRxVQ!PztFd4$A(+0(i*0%gA&aw1p@OJ6%U!>0 z`|SDqtpQs(@gG1qYWWcd>fg6cy6$l7wd)38vj?blVPT*xL#t=eNgWQ?YfCXOg4;y; z!8&W=unGg>VzF5Bg}Vl4d7O8}=!24^G;v{B@{R#=3E#e5o8|}9dJ40naROuOJ-q}@ zm3F%6Y;w3N*Zb#^Ub*jQ9FO~5!Q~6^w4!B7(?@Q3c?CL7Q#A&4lo{-W9E4ylnR7%N zV|AOzWc9Ne)7wdFR>>sN=a>{uR^IE)aS`Gb*B{61eekcejRD3y_`F61DXT!{3rhRz z$-k0a&WXonC3$i8#Pta(XbEW}laC*Vfzjx;w~Pu-k9kbqy@l^<=0j%N*4g+q`13|c zE;z~`zZYZUjia719UJ*=wfhhyv*pf8DFvf944J8=gyb(4>#;lGV~N@lEURb744p`B zAA1~D-fZAZWNCt8Joy{CEHF1t9^gRsM*!V3J$E6Oq{*FCm~jF~O#odp0nAnW7UTE8 z@XdB(AVrDquyq2hinQ*JsRU#}@YlooAI@r~WKbgqv4i>SK-yy{8 zbxhrxj`m zAj>I+WQA8QoV`VQLqWX52fc+h=&46ABn`Mu5LclJ3^=p?iwYOq|GqBxAmy0Zwl94iY{E}62yrYhAGVyQ ztz#WI3!$BsT2L9t$(a4k&(_+!2UjC&ls-;M!c(4xpBVtaU}4s%?@-&dg^kinG*xn= zU71t?uCWF;X2+Z`yGg69Ngds(8d#`LU77j)-akbB{lQE~{q#&j@ZQJSm1>7-tA$-) ztNF*ixct{KJSv<{{@pF-;PG$b$iDfe(Ss5D!hyIw7~Us`m{&>k%jcN|y+UZ+Wd3k= zNzA~9QqOnZto!2#$YCPcfe;APxS*mjWCf?33gm=?s)KWx$f*ih7U52Og0olL|6m$8 zEr*u9%b!B<<@YmrDslEhrC;qN+%zbUAC<=Anz6M~# z)Mo>*gCI4j_i>zTZCYc|9*D1#A^gV^aJc%jSM5 z2{RAOG^Fd7g|grN?t!WuHNXoYxwr>8Y1q~pI_s1Bh{QN6=p#K-#Kl{zf@`5S^GSx0 z6zZ{B^PFJAC6|o15lro%I077##M7j(hQJW4Q!)THJ0KBYZ(^V=fs!WF)y}U^^z(qx z#v@<%Ukj2F6P>CAf3=CFvxq`3G1S!!BWMSG4Usr5CZ-s`Zp#Nsl)<1(b&|Dr*GcKk zA!UJz!#LCTIMA;>GD%%+uJV$}_cLjow%01g+e)D&EvhAtnHG`pTxFe@MhxJC${V-F z_AZ~csE%beskRi?cH>gJ#6PnMwbRJ|>Ng$YG9Gq?jg}omI6`$N-5gal{@WqvC=Ok$ zh$r*5rb&0tRe92*;CK}6c7a}WA>YnkLa~K~#sU$#NMvaOKS-H-CgFD;^@FfX5Wr}8 zKSe~(Q-RSOdP_4&oH^hKvo;z3B;#H0+UNO}<0=D{@GJ#t9c-VyRJ{~;E5O;XG?%*C z90l|{B>ko&A@_KfDLph&Q=j-*7SSb5;-OXk<_4A>qJ{(}O4yqLpnOS6GuL}snO4;-Q6jKQ_*Z)K@h#Tgw zIc1L@eY7#P@BaGHLT=F~yOXA)5G^3*M>co5Pql>5G74--;J7cQ_^Zx$K1(!c63V0y z@ts*2G^2X4cE{s>wkt7Dwn?92|Mq8HcEmQvX@u?5=y_d%cR&&1rE3GOP+J`X!@lNj zgQ;LC6WX5m1Q^!3{j=iYkrPGiUxoVr;zWt+08HG?oP?`rF;#KRw>sx(*yeMh8?0+k zlQuI%9sz1DTh(J^T=D)D{8Z$^6G*k6*7avd`OH7>ro)773_bSBzgBS$i&5hhsp_sE z$o$(=gLyh`)bX@riwEbv6U>3Rkb{z?fTiuoq{y5m&}z@RiCWFsK~7pG$L2rZTcZf$JzMcgY_3>xtD*spwPPY=|2&-7hUn>0Bspu-@iHBIxcp$d2EVU*0s>FUml1ISOmBu7| zj$y`^C}EZD#Xt0F7azG^rOS8K1#XbL35~i5k6J!a{S|jWPW+P_8hWCRClK?6^J3 zmKCjj_Gj=C;Hq_x^`$?~a#*bg5z324Px>1G#wP`{6$j7iZ$IcTMw9-#(?facKmcGz zt7Z{W>e@^B=GR)zk#EJJHu_lhQt>w8Ijd~)H>Zf2QzjCsP82W41Y3Y$Gr9@LF6;QO zhJYV>>*G%{NmrYKxm506_*XgsiqHnmz2As${fyoX4TEx6cD-cU#%F$CHZ~=$2jXYD z-R?e;DHV+_`YK!2bKdk#q&zf?8w`=4#WqDECy6}C@uazCZ{FyZ5bJMC2A>OHIz#6w zDrH*$k)vu%y6T3dQu|#Neo+K(F;sHI&jLp@82wz*ZoJ6iyMgiH@RXu z%JD#9Kt_491o)Y}Mf_OQP(`AitrDe7-qTx7Rl$l%y(Dw?Y972h?c{tS_ulGKQY>M$ z;%iQ|*pbI;_36U>|2!nc+Q1fm*r+X`HV!hZlys-|UD(YEb7#TC=K{vCO{uE_82lML zgb{*VKYLM@+1qsf1tBw~%3np}t69sN}w2%j-)Kc9I0GVq!MzahX)Y!f>FmB%?+TlUlMz_>H^~k7u2(kC0NR7Rs-QY zy90zXKxsD1j3v0DoOeqB3R<&9^=-$IGy3k2Z=}C|^Fb`=LwY-QqK(IbX!Vuk1qitG z$=(wN>j$d}QIr@Bq6-Vh^;bTHT~jF0oa-iqBsM}|L#={1)@d~V57 zMbcavEg+5d;k?ch;6v{;o#IS3vsE7D_MIz|)~8zO>p*6Pt_G0LbbY8WzlOn{fY5fh z*vbWFHsxxqG^5c9V4c){H~1F^4H+lrt-HVsSkC!7K*IukJbT%YM-nhvkU$QPdH!Ow zunD9uwWfC$jy;AG2;Nf5;-g;K_ORx2dBhSdD zt6En{r$EH#WPU)~bw=!XZgNb(63)1bf>Pnhx5#KDs=4c6C~odq{g7r?MW7ShnPQ#9 zatB18zDo?IP}Q=mSz$GJ^$;FULZhloJ9XVx^1p@T5MKKCpB~fe%r=q3`un`Hj_$!zQ(hPzhcorb~u=IDNOx3gap$6B~Qb1cD3 zP?t`LEoMHYfL~X^nRBmckcrlVeyh=*;v{;q-^((QvP4vLw8_3%gjg*H1h-^8ZTBV7 z%0Rimp#+E&a$X_*Xf5m&fymdTM&uQSs4L-XvswWw~{m}JW01Nj+{6%N+s#Dc%v8R%XMg6a8 zhr$aV2gesLp2etyT^Zr=_4Z4K&y&pUu?fA7=`Wr?v?hh5;B z@Thp}Fb0d-I)OTOmdlA=7&dqF*XJUj(`8h`yQ}OjhEHFrsh0yV2JZinHbQRuWBbL# zoDj#qRGI#@CEr3c-IQqSK~Y{j^Y1GK5FlMNW}+{{R~E!@!7M{s?>e{fvsAG*nX&v> z(n652k&UB5WX*oHxB>2xa-pB#-X}sho0rr7V(h)6n%tsqQNbWYl#(EzNGAcLh9*dd zh!8^e2na|BqSBibX-X49F9AapkYb_t-V7ojN|P>CM5HTC5q&%7zIWg6c;0yTjFCTN zFoxvY-`Z=<)#jWH6JdIbsR3)QYfJoHsWic1;Zp=AgX+k3Egnf(Mb|Ywo8X_^5;YUF zz%W2Yw;@W)2`^J8`f@Iykt$8sH2_d`iZuWPcYiQt{<(z_T+TEn;4u4|rBR@8ZKy=$ z5ETE?6|Xq4eqcCX5a~1-N>}s&aA`l0?DxhS(#^e}jVUyLXLs&!D1KqRbx>H*bj4~Sb9uZ+9RgMhl)r=Nc@kyUdk1hj$AfI4d$Gvp-|cL4+%d&-pO~g( z;ycd&>{+?Tkxge zw-jRrwM}l@fXq08;i(BnCPXU;nOzYex&LqL|8qukz9A4nJ^z6Z12j}2A!vW@6G)@O z`E+*nwCaZMim7Rzs+?kumeRcQFHD`eQDwq+W})3r5z%aG1Kk7Y_mb?}mkryf^YAg| z6&V!;+eLU4D|$zd0q`sswH|%UR`Pc+XoVx2_gAk$D*Jy`e)d|lqJ~rO-)&X@cHeq zF^lY)JIB9QBYl2?qvl13D0g8zj*A;d;d_iuaa(dyf(kRqy^|bL!x%HQDN)^t8g#K; z9Cl%=G7LKEUy6f{3yof6a^_F;-3m;U+@33z`z8WPcAu(>0%zSQc3y9i$N49bEfw=I zsz!dx{8MQZ83#I$&aN%Hsa7q1?`M1$ zPy2uCzgSXVKc1A7B+AP>Z)6ZXQ%6PF@H!@_-o&H`9*g|8VI+|nPokz6pu>D8(8n+i(}DK`s?;bvHO&x{h`i0E(66gRB3d_;BZPqv!1C3jQk5+aEq=1hb0~?|cx!a| zJTy~2wpt=spCL|oO~Q(sf80lvoGi0;k-i?juqaBk5LGHS`+&PstX22twA9-kDBQYw zT=7LXA=z$k%;!m_P_d3>vMv5aKp!xG3%9R#pJk?OsD_wx{F{8hSB*Ney@<}8p7S(? z^3R~mXF?{ER??_GeDI1pdm0m`2d@^(y_VB7>VlX7S>`Q725ngBBSO0rST_Bf zo+G)fcoJ?cs(_T@{S~;#5(YgLb${c~=u0}7Y7Pv-e+TIE|G>-Beh#uBKR+KQ%U}7d z!g1uwWW?W^Ipd@EZX8_^=xEvJf<0?_gxEaOt&^UwlTm%vIjQ!(X|jy-WTA7bQ@4Ll zt&=08aXz^PjDZ}yz1Q)=P{z`VlSMwY3Cv~#xkQzdUQF5fGz4IDo=`S8P&IiQ@DNX%r4zKbrN-CjAkjMIdMog0Y2C>UXz8!^~`?<19@ z2j_2=u@IoUr11fI$q-W{in&AZN>l9-oi^5~&9hLM^$+wOe1~6+cQDrAwx^}zS&f&g z>W-Tk<(G-c^F?)z2}$yeBHeqbx32U4z?2VqY4D6vr?bS?>Z*6csUWO_1K}PaW!)G<@5$8I4i+zIZZ$wABrSZ5|RHkkKSHN9li;2p;n? zJ)fz*`J}`A#T5f+oP{=K`W5^6ED!t0V-RwW^7p+rxqOOOv+Z=C?y2XwUvR-5=RQ+J z<80~5i_a~gFZMn?+DN`8hdB|E==8Q=Mz?BZ1L#P`b?Sr+9t zv?Fz6L%kaS_x=gNlKVG@z{<54BO-I$biobu|7KGh73qg8ZCBMqyU4e%6)BxU7} zMAG=x=Iaf!z#tCRV^)3ttO(#%!NmHoQ;(lkVKE#^nELR<8>&}U#xL73x##}W;&E2u z%_}cF*C0tSgo1Es9s0z%8x$_TaZBo%)#QTApinYdBmoZa%=qUdy9KZB1&e)nKK?=2 zK!D1Fk+4}d0`$Hm6%77M{@tZsN#+C+9sK``Yo?t!CwWUH(JJBw%+%t<`nAKzn%oI5 zK`rVZSMoK~0FBmoPr4F?L%Y%hCW=74m2GC#0z1h4f^#sq&@(O6wqQ&GNwN=ZO<*%n z!>Z|jR5uo=gtm~wVDOClI$C5_0VLmo%pJkJaA1_xnL3;i|BTHK=$sg-or)FP+AHpG z&u&!hyY87;t(J?duw-VwB;O7>b5n3ioal1tGAg*E=N;#T*=l9|@;xaBx*bDoW&}wM zkB=ch%`O8xkGSzL^R+cAd`%jd<9~g4l2+AfaE1NFQW?fZn7){@tz3lG_&4@5Ot8b= zDkp25hwGx)I;(Jd)(8*O7q1d}ZAho?gM!hZZ|>18U)?##8B1^tq2~}{FF)A@juwWnS#Z5B_G}zi~V3b zN_W;M8m3=2`LKjh&)YUJGLkT16Y;`xz2}I=yIg5X%6LHxZkOfT2Ix|Qw_rJ>gLRL1 zXcV;izB_*5yDba6fC;XXN@Su(F>}mtMz@#fV9|3r6Jb_Kq;bg8>jn3rz#8|bS6~0j z8e1S{u0hb>7UVYqx~T2s;fe&B3KVKpU#k4>uu}HBYV!L}yiKE0H%kH+M`f9+bzVos zj6nLdqw8$%cn5kLw5|*H+GmByapetk1-Xc3KDq%*|+WU$K||LJ%4JOSFgB|QXN z`)*sXXBQKKjfC!m>8L%EZIb{dT2C#2{7Dffu;JrDF(U^iWBz3%Ce%bG-98OGYeA~~ zc=IJ@coO5#p^d|>nGkl)Q1!$5+*y@pn^iVgZ+-0B>y}P2f3X%!yv;i`Wg2CaYFD7p z1CkS^2S3LPBWtGkm=YmMsO@EgsZ3H1DWBxKafi?XK^f7Z5;JoOca-kI;LP`+t_S)$ zSSNYol7V+1%>4}5^IH4UpSs!UmEFvNPtE{Q#~|>{h|oBg^`jcrIW=JP;a5VB~yK`$1BO`o0$>)GxP*PKeBlD zFbV|aGW|M+Yk!smM)h#%FueDSR^pvG2%=jaBTiGaY?YKNDbk~d{ZuG;U7W1pfS;3I z^|q%MluzXmcvX_?Ir2r}l2*wxPclA{I7hx^;~katw5P^|34ZU+(+sMW4g>t(uILhV zHpzE)*qbBxs->A_^LGD{&9>` zG40xJqa=^d=TBbe%(&_kGcL68ghZFKsreTbM&eANg_qHt@|mAs!h25c<DqmHtbPdE$b3^&v`!~t6Lz}{Nn@Qkp^{#Wl!PU zn@l}5m7RF~vyN}71kon@B-#C#Cggi*ZDYgpET0O^bo8Uv#rmp4m&}Dah#^|_?xkl- z$BO+JmL8KT=jz>Vx4)hJQlqGiee*_aTy%)N>s|s!*Gg@Zz{|ZywE)6_?52{?eA+4|g~#SNk2|5N2EzQ#2|xTbX#ZlBz4kiM#bNVtxwv*1Y9w_wnCa!0Pon`dFh5h% z)ch$G0puR)LLc4DSFyD=O$7IdXVIl7j;L?-??~*d|gR zEK7>b7l>WkENcZ$m`)=2Mt-$ka^)?>*f!n?wLbO4jx@11T0 zUet-x-&x@#@Sf;-o%g(d4obAMG2yP;Fgy;*CTYN^g$XKj(4b3={&$8NuJAwwNY-Ck1)#vb-pM*X|chRmC0a)qJK}*Q1Ymq&?~Rx%^u(2Rs@DY z8O%1f@=x0+(%s5Ab+z)$lT#zPy-tePQWpKN!K4XF#sAJ?&!N&S4F!+YkQ=$n;yjN@{hD>ICe9T#;5!^-e*!Pjaf27GEWo~C< z&LOH@?JOJ+$>XTZx$WEQ;uC*8u=}YOA1_KlIpV~aXP;e0swh)rO=sp!kq~Wd@)!%l zu&;~bY`xd6UVWHm0M9Gg3YA8YG5xE{jRr=7Cl;LU;3rj{uGW03X>n;G@{0bxnk|HQ189a}iZvPB!5}O2f3S)-l!f zOZUe`A%U)gkC_?1`AU&5anA}We_j;{O%QsG^@j1b8|!Mz#pIpc+r73E*N*dCo39Wy zJ$He=C&DR40DD!+sw2r@&15io$WVtsMGVB04>ey^BwX#Xk0hLu8M0fn+)JgzASQEw zb8iF_ebwFlP_+R`Fj-F1G2f^&%(t@)EjCfH- z5&17;Q+XXX&iVujXuT@>kX_Lr+b35s0^^w85bmpf`I*Wn_SWbG)5(cvP}WctrkJaz zG;dlupM99gUWqI&I!&N2?cR-OR}HQ=i|%d|Lfzs)gO5idQK(*G*#a8KV{hT!u zC9WdM`;dZyYapdxW-F;Dt5`>eVlEeCWX-b=tboJQ5~5-73rj3%rdzKzQ^@33((lg& zV%6}S-GbH;;{Oa&Oev6Y&5(i&d94H|_VbS`f&wNR*jfbqw2)Szj&coL=Q1i*((VbK z7UGxU;Uamzeuwo~iG&FtF{rOnFso;*qlraeK}2x7nL$AOQbdT*iVWUsoqu$K8@&lF zW>kIB?iO3Tf*M#$_qcgm=OJg<=kaKZy_BcubNmd@Q^x8!|v78RGdah#z>J4;py_$s2H`(OTck4I9mZO%_ECdUb0Kwd935=1m6x_#E{4DNiO6CQS8IDtGF zX)IG$suq$1BE}6upiq^d-Abtx_a?_T4vC~Y?cUdURVH~g#|0jJ4S5FL zOTisqw-x$u_ZC33TEc>*8HeSso}0R2W$aCThp8q{zRd#abY+M=Er#&Ufxu+q5>2mrd3Qef zeaR7AT-hBCjY1I<@KXF-_R(2P%|$Ta3kpjXu2(&^{mGm~MMZ zl$5?J{?wf8+aXsSg1h&gTB7oC1$9)%jC4TT)*&AN6C&UOa8sx}k>KnpA^Ts>6$v*( zKXH_6+ThNpr1v?oX1w^o>lCDfR)@ed1ZO#YejvcGIgggDl$aEzNV8q8)=)p`5hW83OdR zUawXZL0YJ1Pv#>e`7Z?!&d?Yv`qyqVv%e6kf|;d;f!PckcCN1aKS3LyHwZljgyi(z zm>srlL@&&Ip=>8d#!{6& z5{i^fPaMEmM;keL;vHsE9G_p%_G%Fs7q}=UJdv? zdLVspao&5M1X_2M4{L44ic@u>uI97sUAs9!mCxThU=R{0;31j&`qd~j0VY~Po*8xA zO+MhO&iyr^D;K}M1%K}prQ9E^0=FC*{FqLNIo3s+en+~7f`@s&LJhIa48k2`^luMf z!uBMR8$N^pCEnMKaNGH>FP{kqMaVT}XIWJH$viG~=B|DC7UihvkxQ@4y}-+vG$@^S zOqlTUF9mOw&?Vy60Rz1Bmrnmc!B1?dP7M8J!eZl}(=}sDFV0gBT)2#bQ@#6? z@koMB1{&W9!sUj9Taxr6yvEYR$WK3b*-P!~HDt6v*6H=WT0v0ISvSaK_yeG~cSD2_ z4+U?<+GUpJuIvEWh_RjYr3|E-wKzsLPkcwMBwx`mwu)+5y2Ii#0SVW=glPAVtE z+paxX?awLpifaD^XUE~G@e9sTP@>1lJWGx)vhCr?HJyB9ZvyFc_{XcW?8&6@pM5C4 zT5z?Qi_{gF#V85G684MyKe7ci$WQb!&tIe+>q;G>{ui|8>f4SSs}N#+OyQW-=k2*6 z=)r9^G#MbdDbrN@6c&~{zMK&|A%E+i9Pm-Iz_Hloz>>ZnhQ0PMJYc(AkRg-d>`OxQ zj~?-&)Z;EJ?OZlBx%<-+&v)tVlQ=#NiZQ(=47mJhOzp>+NLki*5t;JtzVH@-FOye_ zX^s$+MyDV3RtB?TfgG~un4Gi$Y`MTJ0 zrRkLxnmze0^o{3B0K&Z53$mtU>olN{KoPrniB`k&VJUioe^gw@)u`145Xi#i%*cbS zYmWo3&)xWYdlyXC>l>MvlycaU=gf=?+w z3MFF)$Z$(t`-P*RF<+Zm`rc_*VhK=`ABfhPt;5~4qgZ< zBI$}BJs^|)bXMH|$Z+|}Yal`bpV(NP8L0DE3o=S;PF)Axc`pV?2^&9CWJNyt0j11U z0jM{fOvRC>;bMxi9mZ~L`Q*`TRc9+mEk|(hw!J?Ge&NIFhtcEAmzXm>H>FwNxA#9~ z2qG)DxEB9g{WbWCfSNE$p=qvHHp}&M6{Cayx_5V^-jm|towa5=oQEXs;~S--*s7{b zd9rp)yk=C04M4J2$Yn**4>sveX<(aagtup6#aGW}R=e+%LCLSi31{$mJkmV^$dR9u z5)upqXj`T|=WR1!lVLPfasWsI7QO%RVbsGxb)Y|IUk#e%o$fDDAd#MG4m$=#rKw*0 z*>+wd@5EAuLzGb`w7>KP{jk`BR+#9VfW-E|!7iG4ON=UtP?trbLm*#;RI$>c+yU=% zR&eIi*9KJ@;B5p7f#CIs@PBAc1V;$8!}O!0bw@8=vSZ_*T_4J#T4g_#cXmBrPvcs7 z7<{#rTYUwo&$=PfUm5f9L4jbm#i<2YFGfkgS6L4dpx;g)a47Hnigj6ib=ZGEj_GOy z4HB|T6yd7K8;tkTLO>j*zbw#$3fzv5=vB#=n&5UQWm4?vpZx$|GiqDF75Wf%!TC7c zo7s5bYnD1k7sSTRfk8I?jmI`&>E|1?qGgJ$;B!ys+f+#~lG3RH1ef;BD_&HC1%C#O{15oYdWbOL|`4mp{^>alMa^QKOZ0KEl%N#qWTtr78WXgccLWpR7T{|%>fUKQ>4sU45fnvpL71c?_ zKThR73^z-c>UzC>eRtHRX+$39wf@A#qnr&MCOXIx57Vca5Z*foruKKR&doTF)s4FH za@Fj<#;_3un+XB)*cU=N@FGuy$aRv#;P3|T=SXfTnfFoKTmq=vPh%W@^Ov`JK|T85 zIIbWYK{J1`*JSO>X=k0ryS|2;&J*6OkYy{4mB`?HyM+oyOOGCKm0@33_j}4x0 z+vsA=`r|S(xmGoN_0ll{!OLbS6paf64s5hrC;rcVe6Smg{mSpiPk_Wq3tBX}XVdgv z4zGMb{@S^~hmjXE_X0#^6KT;}V`v_0 zS8q)btQtTO3m$uWQA`BU5tww3Gh;xL5)^S(5YnU{qdkG~3uwx)<3nb?eJ)i}G*4b7 zqwV#h$dEf2Z1HN+wRx05yMBE* z!CflBIHbQg`fEbmYu%a_ZOocPU!{itV%&re#S{(IL?C_!9eoDvJAPoHplhV=EP+v= zac1u%_AwJZE}6%Y(KhKRI#_um49=iA@|z#Uyjx^O%dQWQ?|goV^S`o=NyxzBxxOT? z(+1aRF@-XBV#R&qz**(2Fc?$GO$leMjlLr61 zAHjDJ<#Zyb_*Uj`OiW&!{N62IufMoY&2U$X6WuV_$eQYpz9MA7WJYz&`DeYg0WxiG zgP-DF{#{b!D?ba#Bt0M=FcoA9As=dK5(qDa_z1z1l9m z^SD+FTn}geFV&0x(|v<~8)@Kd+b0=U^YJRjGr zxKPeSb9>VQ1@k3F*upp~ucnx(0JaoeZC zen9*pgBc@|%Dht;rSg`658+N7duv|F$Y&lW>HkSG&7ORHMY}0cQ$Z;2owV%)uR9_3qXWUa#T@JB=YUpe(v&VGFe+2 zdma!&;e?B#K+VVyuhCz2@em)Au&evOFeI85y!OxX@t02iVR`GiIGod@xa?{6VOUAa z;=xUiflmXTdF+qT;-KvEGDX4reDZ^zH>I*5eTw#!`HSFwBtTEh`M?hcvcOGhBgy1o z!19bi84+)#+_S&-Kt=vPRM+J9lKG1@JmAp}zQq#U(OYl}zsB>q6IZhG&k42VMN@I}8qyKvm2?vPPGJ zVgsK+w6{YO3><#xZT&;Be?S$!@()nA5LX-b!AmDX_eOVJ{5P+&VdVQKKkj4}2y{}qCSLK&qwMP8O9$%p}1VI%^y!<0$W)y^Gw<0zb_J7Xqii)tH}b?7p*dFC}rBfWDvq zBXM%#pzc3_e9bCFz1Tcoli{p6I&jIUNBhdnLWJ2FhdR}Yvi58h!#mJmGntlGzds^s z`y1ZJMXAX!u|wZZ3f3J;p5Q^gD!Cnr$9I6NJ%WsOZ(Q&6Pl3T<@IkkhvSoG<>os-a znS;H6;h6~vDP#XET*#5y`$4RqRJuv25k~2>F~qFBL5rwYtvX;2vc^*!1z&u#JX>^n zruy370*UY%;u&WpVni^E4Ks5dnyk?ex_`_;$oS%xfn(sjO5t7u{e$V8TZ%nWN@XZi01LN zugzDZ|8gaWDOyjK(;cK8>w*-JFKdsm3N$v-hRR9T;Y7M195}YyMF(e-6}>6om(uJ0 z|4mVk1&4PbE>?8+D1+3Z%**^ZMxj49S2UxSD}?BazaPx0aeCi!C;hdK?$u#OD+O>` z78*DrKF=Lw0iR}2t6XNRaHgWeM{u~TkfJFc1!P)ygg?Ch1mK6;rD9qr;7x_VN`lNE z-ZYY5jS3ZamwYginH5U%rN!q#`^#FEWYVqfuiPPUK6_J}e;|KKruEvGBV)V^DBk_u z7J0-YnC}0oq>n(jH>HO%%C`5%NW~HeN#_wrB8hG>ydgXTpeT)l$-6sX+#|Bp^Ev+? zfHKmO+NA2+XhaY!@U!5yyJlNf*=kK`D>ORdK_nx2a^g76-a;bm5h+p=S~Nb+KE>P+ z@kANWMnvMNft-!FnENw(COrPlrNcy0yZ^WgF>}gP3zazk1FMaF+KpyuJGG%8Lm>FO zedb{Hl>s&u|H|OqzviH5dE4{RaEF?UwwA7(l^p${cl?x{?S;2f6Vq`;`yZ!o{CaIrN5n0_-p$eca?63|@f2J|zyHxZ#u7 ze1SmFR#BX+bG`y5L#DE%*2^j0@c;d^%ONFYP;eN``Q_-i>@Bb{DMuIrc}34PVzN`W z_tNA_dcc%(F&b|SYNaV68iuWzUUrXbISqj&} z^U-!e&zA}XT{e%8&QQ`Qe~`oEOi!nr!Q&nLX1o=I)fJe*#9f0!P|2sUsb3z}V<5FH zrcdS)pkCz%g%1^dKjQy5U&>jx(TR$R=iWL<>K7B8T>F{V51QS4 z*XMqymkXl#&Y25#b?2sefpBhsw#3Ht+bVmS8E`mMw*}P<&ooThsqbWvGAycNWOh== zEpYL+EVl*CZ)hbI2-5tdvAhE8g2^57ibQ$<_U4ps#tJEp&0KWR$Cj_iek7}$>LpZ8 zk3>%GyKN?r(e(t^{ymU>K5VKzS3tp5JLE6rf20^(g!VUJ_2iBCb+=n@9}cqNeet8W zc-e&5HEerWSyP>YP&>-+(E*k*GT9-&V znA5sA+U6imt@HnI0kkuPFWDg`h9jcaFUL4337#?Lqk1>^H!8NEG>Ny!vslNn+I|Q|)usN{T0c5B(K6!Nq%r{2<@+{{6V`IfkHQ4aLrstJW z>CN-W9-52Nqv0w{AJ~VPphS3;wgKoY3tax}-QoN|ikFHnPf6mvY?-*0=4y4m@1vkO z$Sl)GEt5@B%l)cum1pZ-_R(gMF$>Ks{wuy|fxGCmf}CQw2`h^I?;wv8fo=t^Ug&~o z*^WJ}5sxNheI_iDA$|*M5t(!T+5f5T5F9C3UU}Dq9+N7w7h|!E8_?omc^E1!4upRV z=o8?%76JX6z#ZWE7?OC!y(4rnA9Jptg?(p9{t> zrZCexXu6OVdUIvL>t!?Qosm-_Z4mPloP5MVDE?3l!}kK2C9>Nz-wm6+#_AKi0NFS37a8&{X%y zzlbjJ>u>9%{lAJ)y-Oy#)$hcE20SZhDb$}J^t#h-hL^CwKgXJc?-x4|J@D~_n} z^e|Yb?XGi${a|PXOw%kf0yj$;@x%NPVh-VMl~gXgbi#v+y>fcF^Q#Z6TBUJTM;%QD zH_!2j%Z=jQb?>P`=L=y+-u-qV_yE5J?@( zE>jy?A<|izy@;1c@zk=Kg%_%2cIfp+F_YU}J<~I7_t)_nYGMbu;8>1d}?v}y3 zw@oZsXH?t+o=Q!nXik_XxHO#{E*B>dWDSxwz@?#61Q`*Az`!}MD4 z8w{JRHMOt3#6?A?tyJ57=XF_F7av}wSDk6dH+a6vY>DHR@%9k);|k& zyc*%g#pD)|KaME%%88j@xQX_|G*`Hy#kEef-m2uyKF$|x{Vw;^RaIlA<4=Z16{YNs zJ|j>J>K$!I2g~_B{*U0l!j#ITySX&K7lxiOzZU6XK$F&^e|I`Crj=5n3>FiR42mbQ zSKT6d=8>KXVKkwQr$C-%PixfsOiI<6eqTyTu3w?Gp+O-dE2~EJ*LFOhl|On@YSna$ z^255I>URQ=$H^%(RO&8qXZWiP9tmD(9eQK)!OD}U2CffkiMy{s6cUWbJwM% zoxOO_$1XViT?4DHUrQvi^!4}Gbara=_4QSSg+VShLw~3Es{nee_4W19_wScR`*%J4 z_buufTtc5zSC2e@@#4e!_4s8Dx_#$6cV0I(HI>xWjcI9VP0z_4cy9POI6UY0{{8!g zUU*%wT6=8GUfR1S38d_7@5BDUWx3Y^>_2q{cE1@dYz!~=;U&2 zy#XH2FXX`a2Tqh#|!ojtVa;# zAr%);xm@t6{R{WYl^>COC2!c=-x(PpzJLF|VQ_NNfs>PSbU4%?LzMoc{)lH~7B!!s zV7CrI*?(_*{$)gLoxQ!iOR*i(>gwuxvVlrP-y;bNUEOGo>6s(eQhr6nDd**pXC`}{ z^b+GM!$qp!+)hF_*eFa~Rt+9qE;G&c&d$y*hCGJihDwHL78aKAUq^@QfaY=|bs-cV z++E{0PgZpnj{iMWpzAW5k&%&=pVPk0x*=$XzI-{HQ#EKhj#Kf|o`03|b8?u^iMxLY zH_Tn6z7aKl>#&^C$OSAXj+<9(zbt|Nz##E)mQ&FH>BrhkfFJ^-z@ofNj&I&X7;n^f zjom%Za`xLnlLyOErdC2K&LfXzDF={rl`vW};;>KRFE@*9vN*=Ci#k%^t!j#BNTr0PU%}r#+LPwy#8BH)wK3{ zuhmpIT}@5RE8AS++`E>1+RIc0C-=VrF4kJtS(QhfLLwqnWHNd`dZl;QvjuBS(MX!_ zu+2%xG0ZLBVl932Zuu9Nf(JQ{OW9#*X-PlK@Ve6|AF9IqW(}xR{?8Ugur^XM{-XvH zG(smbM5I5|yTZNwT+=Hjh;W2a{`sMV8@S;wQ6Vy*mg^Y*{-e1Ez8z**yAu6F%`K}V zO@X7caq>jSXdD*JVcgbd-?@vjwnZ3{W}xB*hs+do2W!TUIb`Dq_nb0IEN6stP`}fR#2f{s@|CQ^mSbV$7vlKjF6Hk z<$gA4h(aaO>m1S?PC@V_{3Pv$>VEj4PDlH^PJCV-pQ7idm@H-g(aOrIEuFJ_Q3rhz z$+5Aq;t2PnA6}*pqo6PhOX;H(de`}ia_pHUy8Ev>uAT@#nJkb|1L_N`<|U^8jB-3l zPzWz$Q#+TJQq4;3 z8Dh(d20o_4{cWH2yk()G^*4uDT1Rn$u>~i)2I16jn9)xWkEPk9#5ekffM<7(5x{pf z%t&plJZdn~Lth`ul5cT_Or|2bCz0r^=mA#n)dOXZgL}6${ucV65S%0UC^AsJ@QQc{ ztqClkqO!XasAhe@^#~Nw3Bf4Lx0AEK)@K_D_f)*?bOa5smi2BO2nNz6oWnd&h4EU7 zAJtgwIgNxhSA2EHYPeYgQcY$5-$(oFKv_y($?%$_O-IL&9#+S&&@f7px%2B)773OM zfc?L2ZEE&cZ8DeAi8w<6-=6xivgWT*7IkB{Z8v9H_Q-I9`m14e5-j}7Z?mdin3m0? z2#c!j0%3xw#dy_?AoUM1*2V~AvviKdKP{I^m1(Q@;^{?`wwiT*?-#cBI#1hS0G{V# z9eVI}o-XNB{HBCjS^-v&VcdElI*h!$oe{h>N4g?$@=;tn?WE;#=e?)xay68sQ7xYt z7|7wxbucdhMkOkx1OzVdI!zq>0HUkwppaLP$+y1nA^!;MdI*$r)Lbv?hUoGdoix-c zXJIy>zVQ|0LLLi`TUh1g&l>1-BwYbALiZPuKhj)qZBUhNi)&F(mB*ni!rAwK-BY~x zHGjXnsqTib(2TL0o7>~&@Q|C#E&LA?j1K@eCyv5TY$ZXrA+n#LEFfS1J3w4;dvm&B z@kIP4V3`-M*^YG$cqxA&adJ*YI(J?tl2P^NK+t~EY{OTR8!#{|HrEka3uHrFO)bqY zl*WgIJUhw2MbXLldQY!ov-BfxYXZra?v7v7Ka;ukS_IRDs<3LXg<=XC=jYB@?Jul$ z4BFe?E6R)kRM6PU&;x-jkV@Y964homXd2Rj;97c#x7=6qGZdZMv~i#-?UQ|Xi&_(r zS-ZZQ&&dXw`)Vj7+IzYRgAcfre0bcHL2JZ^1%Aaa$wd&x^1o}s%=61>dU+F%f{)G~ zM6;Tl#Htw5x%sxecGt+XdrT_#-T&aTKGycD#;V2pee3sF*Ga&7tvBWwQV=)p*+Dtj zz1wpiRS!`V#TrMsnor-F*SAMGW`1ZSfAs`1fVU}R}U zCaU*-X3!V*6zcjj~FxZQ} z(BrNek3GmDRUdh8E`-3smHC6d_gP;;b!0PJfA2N5gbMZ^X8;blGcY?XP3!D4dqD;v zX;P#wP=NZdXnyVa$$}Rrd9VI3OpodeMdXo4Iqf*Y1qQhvw_a}S?6ftzn|v&9gY;cL z0=)la$!DE{^jIZY5Ar~SYUo&;e0>?9Fnb59KTuereooDOYGeGHFWWt{tJ=ELoj5!% zSO3#*Qpr}DUu+dUDTVoyRY{`C3Mxa4I){JPN;5P-6UsOQvFTD1Kk1y0B$0&FlZvT5 zIuU)(Jp_ixiPR`woOLouEBiT-k{|C*1W1OGNqLO)(H* z;%QR@7jk^TU~ZvEvxi=D{JX_(6N@3CFrI8>5y|Fo!U?%77khhm!s$Xep-as~>fv+e ziRUa2ppR$-ls{E$vM^8>+hcm5sg`(;(4{mXK_O`&_UjfFR?s2}du7V~A03QAxY^&o z_O0M92ARHxoe;XqP=8ayBsaI;!iemEuR2fipFVx+W|!Bz>_jdhleG*-F8xv1sHG2f zyyFoFdR1Qx)AL(gI~#c~41q-=MgGBF)Py$#RK6socm#}L+qoBIngQtwIOre&HPn1Q zy8CHU%jsOjkP&m88_)gQlp?Dkvl1y_D0!=I6)YfP-pUA))0pOb57+CgFgUyCIgO2z zI#iewt_^_GUnCX{&h5Vbud2*sQ)I6QM;DO>eqwEFjvmsr#lx38|Lvp`ci@1MnMYy=#->55(=#mK9rvIDQ-^a2$j??pyEO4S5 zW8~T9we`#JXb9n>i~Jc>5*`w1e>yd5!CP3+%mK_Ii(PLt_$Lm+DZ=KgHE+a6`0yZR z>WA#TnBc=z-{h2~H-BKxe@JmW9JI(g{}@t=Ccxwh|8G+S5u>BhC!6&2ib9bMn9M}> zel^J?;Z_v)7Pk-cjP?KCAIqCl`o^Y_YJ{JBfJijM5Ok=5F3hf?I(mYTyw`$^+nzjm z*>kE8(M<4(iUv96MZ)H8@!SuR87{9=B#kU^JmaQxaJ!dBxsx%-X(&0mBWQHM?NjjeBpKiuE+(Fj?}8&B}JvyN&qxV}hlFjRgsb$|D|2jA6R z;gMsRQeiD%r)Bim^QJ%Vuo%@ZvFh^ro6+@wM-D|t-|-RepVUD%EzHa+ZT-KkPiS5y z-R!G;Lxh|XSROlgFj-5J!%3O`4vqk0Y=Z-rQK&Dyv$8pm874SfJ1s>+|GXA{K|a5f zvIeD0UaAp$4AKBGgDGq8^!`F&gxTch=>|`I{qpm0!D(UsaqS<^in1GvR%-U0dg)bM zakzFJdgM7FB%orjY{;EsNw=pJENN(%=sA{Zj6WG_e+_r=4fA+QgjD`!$c#k!<97An)18FG&bVD)u6bpRXM{c7v;CrPm21t2aKMp&q zbPGe+1VF}G2>ENZ?N{F~Adxmzpk80v;=kW`R786H`n8#(kBTQZqW|;fr&Faqx=^5c zQcuzp7K}=!s8SVZnI&Ne6y=kaKd{X;rW~x9W$MXq33NoLdxfIqvjW5C!Q4WJKb03f zZ+&8&_WgGKQH%Gpmq5;CQ73mfTfyv(1gD5svJ>G>NLFMk%n`AydSB$nv%garwqPDO zoC#jn^s1P3Gi{dz{$%(Xu}KtZxEF`d)1quAN0sXSZoR^fa<`V5TBOA)yX9pQ9&!kI zkz1LaEccT*MhLJo?il-&Y!Z4XWoc)zez6Vr+ix=yNoJ@29(wWnsOzrjqT#j3e= z9ywYcrt5N)i^6I9RgYmwdMWh5~_b7(Dc+l<+Fw+%E>SC+$q>n(y*a zY>ian+{Q~)-_IYq{jH@}BxRfC^2zMv&hd|#c15o&ZusSasl%on$Wln%OpUk8xp&;` zk87!jJl{WqDO$5J+pHG25cIqwl-&Rn-@&Tk#)`7mfb2rK!9x6YMUfm&9Y8txfFuCUM-mlqsrS1?`{ zvJyLHeg8v>RGxs%H^(SwFLPW?knqvLV!YKiM+|ErP_ae6%~ZCPeJ?8x?L@x@`nc1$ zON+Fs2f=~A4`+S)>o|T81~dgzX=mSgH8`pQdo}aY{#YLt3``DBvV>oSCvu8v_)C@_cNu3rg=%`1}MH`H4E?GGv)e+NYXf1oPG9H2_`V816& zKcFfAcE}}Z)!rec3rHxElNZAvm)oyP;aVbI_jspcQty84z;MWmh)2OKK z9`}_Ygl%KUN&@%M=PP?auqXai_pEDf`@#2j_g#JiIl51Nw+#!+H)#&Bk_w6G>3(fR zGoHU?;+$dA=BnDhLPoI7il?MUUUPsvL8482=>a5=~>%@3@4VJho5HL zFPg;>0=e+pbvN)D#fYwMn=MuX&5mbeH*M#_BHrny$0(TLcVb$3_^aaVYByw@j{Hgp z(#_H<@6LH(0+ysroNv1~{>4@}C4O-zbE|&#SFBBo-uR=!Z1<*_OE)hL9{Al1c;z9* zBP*4(2AG}w@vgz=>Sq{pHb5NQLIm^7_@t%Yu>>=kQYe&^9Zf=KS1HjXF6((BX|5|5 z3=ZGmgTbdhEBVKkrhyX2daqqg76Fs3G-&030_N+~yu3F{hXhYG8u(tO8q%kN)@dlZo$ zeJqgQ4qXUcEy~~eaY@pAn*oD%=lH$F_!~!o6(yCGL%UsDqkUc`=gO~WN$f4p-ruHw zl>UD)_7*@<_HFyHG)uF)3QE@kf`lU7EWIG10!l7ONH@|AOCz-s(qPa?H!Ll!bR#8= z5`w^Y;eEgFbKn2@=ACDTVHlVJ=8E5WoacFjacc03BznK!jwiH4=~bTg`@cR??XG-p zyD)0FExK~kXJan)z}-!IGFSZ z8MKKxr6xq_retNIjo%cE^xp14>CprVQ-@|zcl*fX!OY9&HPqg*aeY<|Nmr_z%fIcG z2S_+`(h0SxPo8s#y%bZ}zIB}640)l<2a9iC#N6qRr=5GVC3*Bb;OOOFZajp*ASEl& zavLavAAokwm>P*~g<{SdCtGPcd93z!M-yO0wX^fy0a%9)IdeaSU~N+?0nsHQlhy$& zz*Jt%Tl zDkmHn$Vpo85P)vX`RD%+bQ_oR2OAUS=G(WI2$D}%tudu3A_M(0%OW!;97JPQ(~|pz zJGZ4fth}Xg0%Wjc%K3v}6O9>Zra!ju60?pZEvwAlY$_+Gy|Mg?CK0P-U!Yf<4nVH> z5__rYcQrfsrN*ajv^G0t-i?f_Z58&P2NR5(*0Y}T`sL2q8`K+JzmdOzrZ)$ql|#9g z1|2}8X(GO9BN!h#;^wEPqor?)xA6XU;FEIrEFyTEcLDTIYbp&_r`=L+441343{nD3 zn$DTN>rGF(0p{G3ZxeL}mleHKxf2_y1KDqq7lXyl>G(AFzn1!bWn&W-X}KR?<^=hu zD)K5}TfgQUpg%5psYsTOO&#vN=N`eJe2+ivgCSE>Q^%hl3{6f|T4YdXNiVWWE%0$r z_?D+@WeJ-^1jWu&S*Lh$YZJ}hs*&L5U&2p&;XH?cd#`dCIR>={*$Z$hcPfn0La7l? z%hDnJa*ky!@-i|3Ug_?iJ32Z_*zpr?{mBGUX4_?X(UsEjd$obIn$tnh36PFV#^w$+ zSpkiolxug9mO72H-qycaGFKEEvb>BCJ}{Ej%t96yf7eeWrp zfXSTz1fY%ql)Z2A06}?)Jr#0+P|+ZoZG1I$lx%5Z<1$dl$t29V_pn}FMdg07{ox%0 z=R>Jb(!%(?RiF8b-<=7j9%hqgd;x=((A&3f+iCC&3sxvUtQlwnt3#R6ze)LBHfqLb z+;9AS*srdxLF53$NO@8(o0}GA=qxdvWZB19MN(1OII-7yu(It|MFYH zquKKNCNfKa=9s$$!-|`x?98WmkhbuseIYd?is0I7d+UcLgtqU{_G->zN*&nlV?OtOpQt1F?TO=IdTxGxXD^uw zK?Dw2A6Xx%WZ2A_vGb1gL`mJKCPJ>P2u!2ocRsUFO=alOc4U<$zF7UH*%*{xGw<0? zS|EycS#-<1&RjqI@xx#?BA&YxCB)FZHtj9B+6KhP-K&wZNv0##l#OryLYM%Ka>x*K z9|S>FpnV|0ssfMwTTaNc<+G!$>~$WVNz@9vetj@$doPlWYwuO8Cor*mhkNAx@0$~l zg!_HCKOnQH>8oVa^tYAq(6et)Qch(oR=VYM4JChnv^=W7D$zDorVf^DkE|ftj$VaGQZosfKXMJhRVPiJ&G`fl7=s>ln=U&w4N0i ziZVVhY?!sOH`x4CzN#>7rMvfD0*?bH$-Wb9%+-UaaIsPRvq1`%4!_X}f6xhE5XNBQ z=A^A8BT$Xp5hzx%z)Nv}UlD}v`6+RK5a&ap_D>^xz}dw&M&$j~v~_7+Tdx}o3r%b6 z2h2~7ZH-$9U^kxfrhh6dTg-A;T;ZTQZC}In1X8I~=^dK95r&3u6G)0ix=>yPbi%c2 z1p?^l#S2=5RpaS$q&lI480}jMlPPu>BQW)RmNjYoJuv?b2PJxNafS1a-9qz@@1V~z zzk}3d)>++pX60D@~ZsNUJ!IL*A!0>G9 z;{%JqTJt$~hT1kGV6VidQLO$ih*fG&at=gXj}9{Ey5GA z&s-ZT4#3{-`>Tr@cILPdVDvImxz?Xdj010BKXd{v+JL2t}pu{1~(5*+zIb5Gw42V zV22Uyzx6ezk$ih`NBGsC%&Yy~hP`@a-^;_#oeirKP;5h@KyUzOL`B7Yp~=z!(waKM zJ;%4wS!Tu+IaDjR@(60=u*#j5OzZBIqy#PdZ++WO^=L33_n7Tl45>3Tz zz;EOSu*)|3EV|W6B-^O~f*)4ad#4Zk#mXOHW`yduFRbST`5#)3Js?(k$~w8}W9i#e z+8V*cImr?@+VR{2fk5t$HT#7odeFQF!31h;lbK%uns8oIGgfVt;OZ3#otkSJP_ z7dUYvf~sEM;R=zHlMC9SYqRBdZ3E`Lvyur^Z;ef^P~8lEvq#yBo^N&+hje(owUH4) z%3v7#(_rz!#OkLo)FH#t=%IQ7l^2C!e;gKE}-L z)p^z0lU>lIC;G>srR!M9CRVVJ{u1hY~S zfVVg;;lm7Ap+`fkqdDXc?_w}s9IFE4dTpjAx8Ief->odo;CAk&T9*_*Ha`E!cwX9l zou1h0#=NkoqF`uPn1MaPQ6^wNXdt67&Vj#{0z#q81ssB_$~mx@9P4bflFW%Gm@z2-D(`!a zi_m040OZVCZSF^hAtBS^UIAOmOAk8iEP7xpgBz3kPUaF5APcw>uJ(zPZ-Z6g5_K@NArPHad7bB8JNJ62f@v%#F=)cb?b z!l7E~7ALnj_yYjHTjpsa((IK4bol;{&nUYB9AOSpdb1^rB!nEn=#Z27I)J&2YeGOm zBIKv+a6W4-o%-1ii}}SLzasM{^MY=BsrWN^9(AbxPccF1XF(_V!QdVyt^HZMX>pR;u#_a3|&k|g_~WcZnvdumHZZ0Bp_&N!e|I@Rtg{vj}ausi}5 z(~l3qRcO_UZ}%8(ajsneOau8ti|Po}s^SCDL2%A{N|i)Pq1S>SAQKfJ;5M+oUd`kX zuyc=|enl9~W%+hMOVg&zAmC7Is8XtP2~2CfVc*;E=nK_f&yWJY4Ce@uh$HDhY3W0< zXXKa9U#sy4%!@0!uTHN}K+c)ME}E}d{r|VuB?UhAK8`xK1`pqDezJpMDZv-EJ~Bde z82nLy7%Ldcilyq};NXyGt=DS{F7x-PGOd>6aBRk;U(*11B`2>mv8A8URNwu<5P93tho!ec69 z>3ZYLKPnb~!dX@Sy-nK-nJ5r@NU0#1uF$-WGb5W5kX1^8Tt+qhqJZd~NREUpC?3@hcOYuqGoQ-R9TtmI68XMk z|IXZDa9hj_v|)vj-I{(t5D~SS0wA`LIMLk)itS*H5*satuEDRzljlO7MRcnP=tg7N zrCzztV{j^h0nW*OoE$DceIb+L__NuP0qu}rmcm3WEF-g`(x5cM@G5or zqHEK5$RRT~efYIlD(r&Pt^oRN@_D7FXkA*;7=k<(p?X8F-eij2M+o&k0OX`&Wp)j zWKQxmklRcqAl{f!*1gv(&Ud82-5lRo`d&mJ|5b)Coro=MNRs_cVSg!!rDs;V3D&;rXYk- z*c;AM)!A;x_^1KeN;zk3LE(FSDwm)6$#a9Xao%=UD7z|gmR;d=Do9pqqchCd`Chqt zy1#w0TIjP@x7B2dwff$CUG9Q!vU|BomClWZw7QOO&XFiT-B6l03eb_4FyPbJ_+fJ% zV7a;CyVms^;T&w9-SE$*P#i zsnaTQ7P5~NhP^oZ{a+l7Wo@{(Zc&Z!GC!{FP12OTpM#elty{ezh<~p1ndL<*94i5A zrKFr_%Is@ll6o^YT@!z|9}b{OTvm3zK+EKT^@=)yRk-uXy8+{i#M}9GoNg`vfXA>t z7HKwWBFTDxd~_>OEf8w;oxbwvPdDTCY?6UWpqKOm*w``5u9~R{6=(0aF5i+i8IY6y z-0(dY#b8#+8or8d$2i;Bjl@WMi=!Unv-eDr2ld8~+Z|2M#8Cq)Vh9DBOi!9m9JtVJ zJ@=;)tWP^%cm$Xq?2cs-0C?&&{Kgf{m<(VMD?kf@WSs6HU}Fw|PO1=)*4|_SbdFS^ z|CcXB04Z$IlPBxv=nbtK(Hj{K1hW2YLTHl3aG6*<6_iUJ^9M#2w_61b%BRe%)aXPd zxKSvsV4;E^Ua~W?uFyQu7um5*6Gu`NN0JMvbXRo~uV!EHxnF%}ORwo?j?Gx161#rw zm_6M)srjJS%G=L14PIM$Jxx@*`x)`<;)|Xb{ z2IeV%zV$ZLBsNCvH&9!t>3jk*RcnX5}dVjPHSd_IXK>N!Mc6p1gbrEBQHQx8} z9`cFUE8N)mZ%jb~NSPNO7SMMVUU7n-F@o)wMKmHuq{KJ-?qs_@)S|#rRS?Mqv(oC? zt+2x4gh4}zaPXOGkzpXmFU&l2j$`R3r{qm9EYgCh1SU@YPxKS5XG);6`&o~0VTU^E ztL{r8iFgcICvsA$HsrAby5eU#v~cIm4zWsG9hozqbM2z}*dy;Duo}bHTsS1=4KnnC z`|fZHQfd?oWfCiwIL3Go_eSXOe0nay8mX$!XzO?wyg1J4hK3+hCsedIR{n_P|A2ArwsrgvQUa|H7)yIEYX~sjT-Qo-9tP)I!(d zgb%fL95ITcAJa7+qC`^mSo4)(1Xm%(mnc2BFws~RdDMoLd3u_3r+2=EDk)W zS6PKA5X%v_ZE{a>Foayi#!J!=kdD0-+HnmAl&ImukleX{O+o>geHO0eR%V&l#!4&d zS72OT>M1PX8a1EsNy<5^$1oTztQ-HLH;$$kRBlH;k}#bwco}8>#oic^!}^8xO!b=? ztH_%J61cQvvs-jElbRqhJL-LqjqxKA!CAZZ9@$4I>>g_RJv)Kz`Jnq+>+xqEd!98_ z*+vT7lW{`{VUgQh-Tkxd-o3IHY=X;Cme<7n5v5iXNXfo$uU}1Rl!d{!FB-FTr$Q~CuSD97d1*v})TCT=}z!4AmM&kTOmD4%y$~a|_lU{09Q8II3=D*}(+z9oY0z zVLsM@tgrBZ=~z_VFFb+wf?U$F%X*0jPs-KI^hD`rL7TzRy3@m|xi^(j0Qvn|8C<3G zh&7iR(jQq?eqyHtgnhlSW8PVL?xXY=5&W7pj+0yYXmmu2RqDQ!_Uo4{$7)-DUK2|u z>1%);eLsME^(UtpZztT7s1pu|Cm+O(b4UL-((C|5#@0)62{iS0`nOIA2S-ZxU64+` zBy1U7{OH#`{VEk&grMa6AKH3k0{w$l(;5{fGi!)T2ylY2?e(xX0m7XfQ+!(mg7*AF z#ST=GuAzmy|6wwZ%o`5%XV<>Oz%1co#_{D>o}LK#iOOtYsjtm0*OS4g5k7>qqmK~W zHki@aX0QZH0)>F78+{Xu=j#azWqC($n10>^H(sGxbLZ`k(LsXY^w4?u^cx?%E|fsQ z-7k7bmch_#T@6JsuI_TI7GN`$x!6=q$1F;R6!p(z!B zr3dS^g!NjW|1Vy=8r$N-dh@+7&qOZdBTzDLJt;qbFl?;E9{ZBh)El|$VsV3q6tnRuywD~jgX_bguPSNN?A#+Ig8n&4* zC({WJ>yAISA;kbYQh-=-^3liV9N6&u5p)(s0*ak_zD=2b;ixh;+{tiH>VZ5lO_h~f zfB=wJvPslmnK>D;jM0E~sfS~$pzikgw|@pqa#X76pb+x937zM|`t^>0DjXxKf&ZRn zGb(5H@r1so5C+3UNHh**z%(Bue1gK#%j!w8DW&B$p#Z}=a&-Gw2jEC@Kyj0tOL<&0 zcyqPlDU=-XvN3E_d%L{t6G56E72`mg`HWAT(46ILpVnhGC5J>)!1SctlzMeT{@U#h zaGzPc#09Jt`aGG~d6<2Um3=ZHkoQnXFUKlw?>V3n3`eD(mtxqZu6;~V$^Tb&!afD= zvth;T+eeLiS+9QF%VXWmDs2D!IWPeXLi_V}z`b6`0Om9DzjBa>E_VFkek><{tB5cI zjyt=8DP3-9Q~GxR(w9(90V^F_#^DYLEU%ss@$9V2Aa&*CkF2jb3vw@=S|8>VaKCc$ z%a#ZM1i$h9*r~c@W{-*UdOvR4I)?QQ+gb$uI#> zzh->W-*xaDkPmKzEbhpQG)BgK-Ja^e;4YYs@u@Gj;0(zkXhXJ-{tKVKYd7O5Sd55w zy&qVDZBi$Yw1h?jh8oD90j7nw%15gz}!sW(F;;d6zB4F~R=KpqW01Qe*vyzda5CteKS&NgP_t%K zH_HkZAce>ul#e{Rr{Z9Zy^&~40?vRK&25xB*Qy8-Eo0-wYhmP@%Ik)kYw{;WM0jdQ zL``z=W~N6Tn7^7rF;NlVCPasOA&*ayNrt@QIvnbJ3g0J>?8F?Kx;-iFABOL|4bGCA zz^yqMl(3M0rjl+OB3;oWIs+2!)KfObr~G@p(kchiKod$QZiJXFa|2w!gPJTnKdRSY z*%AR!81)pp36N1L_lP<3f{{5GHQdSI?s!44dmz=gvg|;-w;=Xz_b%G&QlGapp=N%6 zs(fOX8w*ItRK4-!S-r4~|$e}mr-hR3f0{ShcYO*XCjDr6@e?#Kf4uVj8 zn3{oXM&_h5uaJZjngk|vj*X4=kY!XXp4LVa1no+;bY~uzt?O33Op;er?C7I01qcbZ z7#QqPD3slPGlXLw-RTo~=QjJZyzTs8l#&A91>!_`8yLv*C9MA}T5CyJ6S<~12+tvB^i6zK32QxH6cl2$R zFf$WajcZ@WWrXZZDvChmIzedZb5D?Cuj7oqvAuCxrPi|Mbi~lGLyF~JTn`UnAb)s% z0S4qHwPlQN(?a-@p|Ko=59@Xpz80SfNsUqgImM3n@L)cE5fLkyaLU}=TtJht-qq6+ z9xa!L27F1EF|)Jj5VqxIK+gvVL`ENQOGwOBR8-6=sdhYhpsA_em?aII^7PjMY(eif zy#8IMKH=(PD87VJ5`v2z;ALC^>4RLYr0BsLQp!ZCrq0uv%QyDOtf*1twjNDpebGFO@l4p?~g|ja1Cf4}zjBRmbd5dt{KuR#Ym5$UzJz%cMi zu7Nwb7;Ju4-wX_cu6~Psa5;0*?%s}TKmnx@<`8$#2&Sf|cQbWx7!C~$ojuvi0``t7 z`)bS~1odo4)A*g_^zBe~^?gB3&P|%25+cbkIb>%|rcL}lA#f)g602hWdHuKoZm^#h zG6^{3glQ`0Qm1ccbByeEp63bNCzNh*!=Y65rKUK|i6|~ax zbFYx)baZ4Cm)(K;g8^BtZ6;3m3Fhl|_(hfOU(U-LG_bh zHVeoHtF9U8yI)s%#?PSOU=%02db%xPtz;%lcooc)%s%goXhd-o+9pkzG#5T<`Q?NV zm*{ZN4cZF^5s^zsNQ}?T*#dfEJ6vom8ft2Xc3gQvb$V?jV;4wRq-Fa=w)p6968CSs z#jK#fK)8xZ#OnI`bAVz3D4TVytll>ag|m~D07s+}dRPS5s{P}mU#>OM7U`mGy`cLJ zf(7}GgWg}y1ET*ff+)B);=&`NomiwE^hQ)9M7 z6WTZTWnWSt^a6dhRcg4*k7_Yp@RbKniAjv=-v^52k9};Yh}*3ehdVv#+2;N%pn?FG zkv)bCmzB@Ol48EaGaLwwtO9E!d+>(P!QO{pkO++48~@i{Lo)^JG&DL8N72Hf@S&jK zGhl~j(BeB6T~fkBwSeJZucsAm+$9`3YPAEY-yM^eslj7VEyU{tMUZ)W0~%KvK&x79 z(t-cH$+IfIprBr+FofOVbWdP}UV;ZG1hJ|rvH!A631DaUkM+$4dClII`=Ut&JDXx~ zEVdmk$)?GMkfGkc7fDNf>&mrm@ky7E?7eciUD=|i1ath|LoIHakO94L&1ZK{+JcT_ zDCw!}PoB^1U|H|dmyhrArvilZi&*3b$3q`HVK>q zt3L4_*vSun0CGfPUr-Tfj$TZKxLHd(3fE)GXES&$^WzThoUzTfCN(*ZsPb;?fBj1# zSPRMGJP981jY$l4^(t~n!oCzs+LP#fX3nOm_97r<5IVD&)aW`MWjv*dt3 zMhoRR4NY7QVpkdlwe`S=h~7E}GQ%yAugshM+MjoS{`}edGh{-BKwbfVe)#F*+W3qN z+wJVE23gf_U?97;uG`QHu&4Ro$F9r;cXuBtP)i_u_GTy=m0{?j=tAa@HF-xph&~(| zW!o;d&2yO&*pur5)vrt!VW-29g=49t5ih0ogfh0}jF{vrT6d}B2Ry$8PwWePJb^kGt@n154^foHUsPTj~_o)l1rYp0*X=#d3kv|!xZPH zXYDEeyuqL@A)n(Z3p>7Q6gy8RKCdULrmn6|;lRHg>zu`7`H#wX)f`+S5sXTF z_^1)NbcE5%leVil;-G$0*q4BS64Z}qA2%#bdWqVhd3;|5pLq|!C?rG>7leRpRskR7qi#&9IN=Bwrli)65gMXv@m-}cSsr^N zRrN^n7pAMLOMe}Qj&wVagY#Z_d!Q`4mIiu1CtX+fqKqwZ_4aqPaUNuj3d$}jDmrVq zXIT?-Oh$|!RPB4=DMbC_@zbYkhU+F2BT7h%~CVu>JO;x`K0 z{&HKQDHQ-=<&*35-5=fjNYlh2xl=7j$Zh!e^~s`cID3?PUtqr}RY{hQmBHI5xj{&g zl!T2oVc+6zc{7p0$z5i?Vp)^ZJK?NRm}f?>%kxIG`s-Re?_CcLbzcJEj;HQ6#x{4_ z;iYu)vE(73%L}J=+?@1{xEZEyps?gi-qz_k^I<}0L|t=&Go<|H4PLxXN_wI9=1ubJ z}ON#@d2RU+XCU_Z$RqdS=4QWg$Ovg<~yaXq<7r6@>$ChV}k5+JJ75V+}qv4QSb8P zQbddEOeZ_Ze>`|s!#B#pFo=80)xP+1HJt-{;wr0`lwtA=SGYs{TrzP>Yis8qS*#3jl-;urA-~KGPjEYl$hzc4;^W`GZGO!A@hJ4V zAuw@>`-J+>amI+vm3m`+JtLCNeVWqwON6X-J!=Uvr?L6Pjd-;UvDR;7E1}e{;8bva z_FiJGq5{T&rzTCaO$}CN!b2GTT5sQ}BAF~KoK0CFc5v2{)4~Ceg@9MEr ztUx0#lwA$TXciHZ8p%sJ?~>;WF+L!4Nl-+4gtEJ)hQ>7PGViSC_VTbo&M) zS7(iylCMt#?uP)jg|dY*s;A2)j-{kK1C0Oj|}!W_EZr1D)7EH~9-AEJL7k;?=kRY{iwdD-qjnSNlcR#f&c!2^7T?L{J(f|($!F#Edv)rx+hVACeXiU}17N8M5o8ePt`&h_ zi#H-6*ad+(?VQg<*jl&_B3oQ0iyyVaC+RBoKDAxrhP6QmkuG)DyMlb17wcPfLWv$i zy+%;7W*#pt`mY8D88LBhp-+L(-r%oGqg~giWrSH?g+i`scyl26?n&_D?^RcdrxDLTa9E0RHd7lrO|6SAQP6 zfgs|;U|9R5i(e7lehA-zuGbkgc}{H`8aD5_jDr_5IjPI;qT#(rGm~_>cFgbgh+GdJ z$g54psxXxOGp+h_c*9#KaM3LW3nDf`luQ`kAWxE8`d$#BH&GOUsyWmd#2+cAKHaek z6d(jQ21=1Z#Fi@3?Z4MNAjP6efG_1+Fj~4Bq$lXzq6IZ*1on+u*F!dv7ccLfETcHS zsCr~(hGZ*lJUDP-AZ)tsGPqMv%e?h#yE7osoR&?y&Fd^0s|`3Bf5)HbD#Y&vwxN zxslZ{#bQT7+qYtF%z>R1WSEHq;d+JDq- z?MHEa(6b&a8Op%i($XPOF|m6Tgteu=BqG4CUcE|AN?JW*;dt!fsOF^Z0e9V^jN@C1 z{}#YRZJyNlOM{<^m<@oMxWc)#xfJhIx4dc<3B&v0m3Q zBR+nK-pIhfmUeSLexvqxK~RGA;Xk*b%|7Pz!z5F@T009faMvxR?duy8qxo z#<|JPbXE1MALi`Bv=W|Q0Muu$rlvMNHf9Qi)$T{TdV4oQcsEi#j=t*H0`4PE_4UOW z_I|=Y9T_h`fIELKS48*EiIv9>Q$~#armgo00x8zTP=pCV=`xg#|1&sB>2BZ9Zb{31t z!jY6=!m+G;T(aGmmt*pucPW{x;)U=gJ(RG(wR5wCidjm+mjS~De-D;h&)A1$ z{OwRp4&;p}%Uoe$K5FutC3E2UX0MSaE^42SqxHB{ns58uBuD|#F>McXd?q}y^cWiX zi}ojd3eMuA4CYu8fi?R-*vA)zJ+15McycTbWYEv)z2xoT&*mun-#0_Q-laE&Rs>o2 zD6@Kg-cZL(EjzoQpbL;D6jxP!C3RER@X1|8x8o_=+7|E3H_eD}iksvBJ`2T>vIGnr z%UghaV4B=_AIEDcgz7C}HT||9P``frMOB6U6S*;AyK8*;T(Q>L6Ko*k_#w?i4ghh zc@2Z6`YFxHTL6!htpaNukt<3Ty;i1#SJxW5{bmNi9 zNlPdlhvDPLk85QJFrlP0MSuj)afQa5=r)vXqtVjdzNWfba6S37_mM9Wa4G7mIruun z5(R)aLb1hZga3fHaB1wu+DJtGM$ z?BCXx@8`1mbvsc#j$=M&`&q~Yxv=KXKaC8bHa7IfZ7<>7>)O!9?^s;tce!wGKym1& zWP*$qg#jRIQ~5GDZ3nmx`%4|i=L=W!gXtosGu5`0z=JN9oec;ab-g9--(SY>FCnKD zHr6RMth$!iBexz}kN?ix4F_ENsD8kiRsc^ox8J9ZHSfXC5pgIVY{LYkLhxfdpPt{*&7Mw8b zGV8B@P15fl#7ws$Q-T5jjhNnowa|2Ek-J0t8O7G)K*GX1;b3bDf=;D(&x$(i6ON#- z#0$(h>ee_zuMUo7(oaqQq+T#$FDd%BV}wIMQ7aqh)8E*^sDd~?8GcWy;NJ|pcPrm9ym&npME&n@2KZu8 z5nDaZ{}Ix2Ev7ur)5W{1q?&a4>o`+GG+}`x)Q~)jWw{Pd=hx7{oGXO5Dru@Z7d8We zbn2l>Ca1Q-_Jhgl@na13%6Us&M7l9b6r};>_1qoRSvTsdS?Btkw2DaBCg+_mdi2i! zh>w{Ujj+|x=(nyC2q;W4mv3Y;wk2cjvYNkd(+33E#f>v<2i}h9X;v1mxzG@+#l>;7uD2p@T-{)oj%a!8d+A!b%0jyptS(0+&$A#-N#qUimeOU}AHLfRgW?QNcxH);I!7E5cghS-t{NiwF56Z;_t%&1< z(=^2t#;Y}heA7U`3BI>RMmJ1yz8LB!P#KC<<38syL4vYMTA!S0p-+KVUs`ru#TC4t9a7bl1Jp91}- z0$4-EKy$keQ-L|&w0IXT^37~t0_r7n#?rIa(13M zZ0rD*z<|~)gt8k`@y1ag{^STlR7Bw8nV^p{Q$myk7V-kYQC78V_}v9IH&!OA=*8Ts z`b9^cP|`K^A#-xKJ4t^XudOjp@KVxL_AT#A{C!X`Q^eq=^I>TQB7c`#&mE&6uro%L z;Ik0w)t@Y)u<K)%f!@|Yjk&Y5tcZ&U}a#j=s8G7Ap2!v7m?{T=O4BG#o7C|XPNFui~CK&Y&rhwS@ z+I53LM+9gwLreZy;lomaGx5q)I~b3`Pl>|M2^2fdDiEru!jDYTX9|Vek9}ed-jTN| zo7|$i`J(ex@3AO6R7i+Q^aQ69)a>hJG%SiA=yr=Iin!*ih@2-1PDravA#8|Y9iT}g zvVAyvYdblMOYSe#oIhbO3)bbpw*?6zh~g6Z^qY5@LFa;`MM~?HgvoMeRoxGyyzuTi z(xx&P)tOa$II#pQJf$vIq_#hnHaqczUpprO7s{fByHPrszJ}@9rrTo3zz~ASIw)WYi-B1jOKR!+mrx991a(`-Ln7w+nrWq2pgLif z;7y=fDujd6`X1cFx&S&S+Isx%9lnU7N5*A*X%G^T+o);_(}0~2VrDM3`O&4E#WQ<1 zvIfT{+a9|tgjo1a50GPT6J=oSWII&v{qeJ^UI1&A^51Xn{wdH`BHFSyfnYQY5m%nm zQv#NovUlqLNg^l%gEUBtUBGkX(Hs?atsC-2!Rv3)a+F8h!MO7kIDs`$o~Z$?tsCs> zMP;AragkUnZ|?rkE@l>*y1Va6QqPjZFM1Z__U&a2T{m^maBEK?l#nv*V}4V;#EW6 zUoXyU{LY1vj1;@}dLTQ%s+_$&pWvT^dTti1TY9F03v~?BOtcRAHs5PRqv&TXhN^Vj zb_64Jtmjjl1~}M=$Z69)RN7GW6-_|%pS#_qB@1^sEZO~5KXu!2*|P>O6lO~?ND^@? zN&fxQXx)P8*Dg^c><+Yw>7Bi2hU_tA2r+yE@W;q8192!z`ZzRS&+b|K{nw+aekAW6wT5GOYke}rj-)E$VbfA_E>V|ZT3++f3lP#v)%TN1a}I%;)G z%?lgulE|(gzOM3d>LMq~iox$ce7K6iuV-3bS((eqUB2u-AZ=`@AjAfi zk1-R5UIW(#h0`a^CryfDUmvl=00&LD*tcnu|6C*816*+`IH1d~!O0D`zf*OMzbzTu zd??4l-IiasF25v+AW^2E*F_FAho4S*;hZEukoUMAlI@0j^dn*_Nutce*}uO{Xb)-w z?-b0(ooY^q8|yG?k!16UQEr(FbrNqKZohy(#Vlf1JmRgMi3c)|Lv|ykrW%q-4+8Ti zyo8{_?D$4uGdX`vIDnLoH^ACPfq9UTIldhuHCXK{hn|!%0d}$gra#!b%UhAm$8SqO zM(aS8JpmY$5qF;K%&hb9S)}Z@PhL7m3NgFR00R)dmz$MCk`LUJxWBfCa~$hz13q5w z4Q01TwSj`%{1)dDxJ52Furf3i0Dtq5=AVEBeiPPR2hg!8N+ z;Ee)vLa+qRO7KAB$OK4sf(hU^Ok{LL7H0S9y_Cr zW?r_=@emkz|JioIB5w3wYf!-b$^roD{)AElj>|n?dBhwFNZa9izj-v#9`1_DN@(Lo z=F8)I2){Re#<;Wbe3V!Ow;MLZOqlPJr5!uv1GAWNt=OpQKhycYa0U>_lgqru_X}Y7 zZi-?!3=}5hRC)Em8wu2OX>K~kP8GR?%+jw6KNKr)9)*q~qDa(chyf;pKe>{7iOGf*0W+Sy`6YzOm|i%wiz# zVgSSF;qnWRfW7M?=Z;%g`1i^AX9d7bgtea7np#iLT};e2anC_Tz$Hf4bfSzjMty33 z68Z((xm_CHleupx&KTu-dpUvVK|;E7;978D2BAq};>@jCkq~97E|EEZz7YZ}B!w(o zeEH3!Y1;(A-JTRyz|YCkjWNJs$1{Cv=yH1|!r@sWM2_ok>e3pnY|aI?$+F8w0|4p9 zTkLIfR8i@FS-MN*e)`ETHuIOdUpHx->9SZLab=-0ny?b5V8CDsXnu>fS8|L0!DWzw zw1%M7NsQ~(R>x(8+3*=vNS<3<)w-S#L?5U)J7{T8yz}SE{=i6zQ$qneq*(vvs=z(Y zO4tcH*4GbVbgX;92%Y-!@TKh>%Q5X2@~#I5RkdnSRUDr~yl`eJ3OEJ?lP#v#4!Pxp zfBg#n+I)TWq=NxiFV8!=PsmJm<(B#X(e>6*QLbOuurvco4J9Qx4ALMlbcZye(yb!W zjg&MDX%GVpA=r}A4dYOPgh6*BEfPw7_xL+{&U)YXTkHJcKS$T|%>C?p?`vNnQtmn_ zZ)fkDoS=Uzb=Lkwk3#ATC;%IAAvGI&(ITQ!QjYWU^KNBHz+q(&5UnPJg&zR=1@B~k zKm(%=z(a#BOx>dnj|R^s@a*Z)Ki$ypskY~;ZB`S72U}vn_^gVtqDG#CDk&p{e%;}V zhdLZgB~fY?#^^g>LsN|4?~WJwMB|X*V!S$5n-rSxD;eZ^f;}S#Y9gG9+Pu-C;?3jp z7B=a#)o-&{X@9<^TrV|!cj0UNKPAc*0@)Uxvce*E@FU)Xaen-@y9}(l_+v%Qpy&>M z0eWN=%Yi#nIsw`WyEl$hP^he|cge{V&H%7kSh&RxhnKCdyEG!L7By^a@{aDaQ<>7v zSo#}zK3ZH_5c=?|lIDyGpu2evujT*u`~POidvJ+}DQpyE!wDypp8Ck6$>ig_vobR) zRx@R9re@!|9p`qrzulvzTmPX}cqAvq^Upd}Ke$xhJn;GC*4E^?Zl03UH07|~n*W}_ z_{y$u!_&*-ejLpO^!Vy{oH=BMT}@;RS^ubasPI~-^5JbQ4`NqQh_Chx3~JKScJ@KY z1o0n$1b89~?~JnPJJj5Y6^=sMirmc*wyCbKm*LgIc_ZGxUpFXEO-Fb8EQwjl{Kk#14hpK22_WhV#<{5FtoU!yw_T*#4&(_m^a*)=zqO9WJdipw z>fKuTOeOz7q6!N(=7Yfa%^X)}>a7^SRch>e0EF0S556#-!EqMTyJQ59H?i@aIwe{foHvTE?a zODM*F>V78@x}G_M2tBSn`k7%9`ty1)_;~ws<@e_=osA%&230EZqyD(3+xKkdn}Mwqqo zG790#oQ!M@uAIGzy%_4w9F4S0h=o(9w4~4gl<{!m*rrwyEPsLT{>E;NLQ zq0QaiDfeX3wjY}YaJC`IyLyvcL!adDfk5%}4`GK@e6_&j!}=&tAMR z2b4gZS9K`FPuIuazah`x*H@72Zn|?uMuvr*-E2XT0sjsq?y02H$DseVBc|PiBcq_! zM2fmHZ0%`Jbf}#TjHD*9WB6pDi-TrYZ$8SiU(=aIwdS)wibEzds`1Jv+;;KmkonbF z?7)6^coRa~hP>FL*)5S!G7Qct!367i%R~5%K=MOt$`6HxMtUYsiw*~{!y9+*`2v_C%{ ziYL#=z;xtt9z)m;t>=!P-GujPSNqiR{sE)F?gS{H_onr0#-g*Egf_Q@&=0fuKtv;0 zM6a05wG-077Es%}Tj5a<#Y#d-dYx*qLh`E#d0lp=*k4Bk^z484RpwcxL;zWCWo_m!1 z!=+Q`?A!ORbM6m;4_9uDD}VU6q16BC#Wt+W z<9q0Su2S2$G%4Lw|BG3<6_xuwRy|=>%Ad6gwo%jOoCPC^;j zL^SC^6jG{&c2r1B+9*})vB=Y8h~gPY+y*PcTB(NM&k9nG>$yPmisxrD zxO0G^nxv%UQZvH`@7=_~UCQj1H*h8^IzFe@D`i^mneSXqf4lTxf3MJ2_!Ce)#QhLg zI{FV_F`d+kfUQb1dS8)Kxc$EE;&PTLT-y-f0-{M*Pg=L1O{&fM8h3iYn<2v4WSz?D z%5QFPRXgb!Wfco*`|CsKu0E)7C~*9zH*CE|efS9qqw4HZAgbq)QhpfVt@CsPD>#`# z3*)>|WYUSfmUExqxv3cngKK$s1=!nT0)v7k0YFy{5= zlK1#My?HL5OObWc*iea;v!%@yEQH?K`TyPZr09f250o%6QERe#sQS#(zt-zBnvuT1 zxa8AW$Ga&*?Lg|Ouj=r`LsiWp*fN#BVe0y$MKmBj*~`)JN-ih{OcdsK{&|w`REOWq zYgqFEWePjdgzC`yI=WBH0tHl(R2Irk!qXTXlws=U@90PUOE*XIy=4PPb`N*v_kmSk zyFMB*J3-~679d1wkT~PO=Q9w}hIj?;j!{7VEN*O^AR{Ado^9P8dOO|)2y~q9Qd;)^ z<2%|+jz>oI(w%6XyCkc~o%!8Vo97yHmWO`q@+Oj;mNC~NTEKcr-Pjbca;0)K#-hVGydfYnU0ov6&8EJ-qA;2`+e9=h1Pn6n$ z58|ckVZHx z70)I@fyLVx$Y2r#+ohzasHpp%n4lS8X1#ZCFy7mnXhB(sUTHRM32nM@^Jd`xd&43J z!RxOI+4g!F;`#cs8%9)AAzW+?X=ZXqFPwvHGxjd6g>hm&dBk*ra;7h*1KUHzrWc6G zp|ocW1#3P(bZ>EYs(b-b0E2gefkl4~MeBDeL}1gnf;n6_U-ceA6mnS72-W3Rpr)-- zRDcFUK|kbYqIEmiA%T8#9b`lKZH!4U3}X8rWDuMI>IG)WUWZd)0lWdssw-7vDa|(O z&3R>>0r^xMte&een&N!DFpw)GDlUU0D8Mxs1IS-f?CMQI7I0I_Q2%}X;16sWt*i(h zQ$Ohh(pUpwzY#|UnIdXTRD%}{l-4QE@q?O)npg=cF$O`sd%?~=4a!5qp?iI?aGUA* z@cmT?JvoWGOJ)5w+X3T>=ReoRwg4$hp@^r@0t(eOeQcJvcY~u7OEK|+yTcT)-!pD; zbz&>?iNNL2+rfX6`47pZ_2xoMm1R zmWC;ZOU9_$uL-H?vKpTDAve;!bzNc|I4=~a-78>$#(SC9EpOYqWNt+v9XgXo!79tK zAz-IqEz*`(&DXaI$Qo*CTqv2MAxlpNt#)vMZlM6I$ITeepFK0ZcJ1+j$lB^=_0o8a z)s-)xPfLu%5MFBW-9Q5eSFa5}&b*jE15@M;`@@)7FrE zkiRDKE9VO*Ix-kHLQcp`{*S5MP08?iT3DYkige^kG^CL?U!>gM;injL=N@Y*4c?7FMEJKsh@J0Ca1W#H`@q2@{ zq6D%P?sL^g6@B;Ky?3gy6m{jdd+3iqq=B7#(_Z?42#`c@vD>NeM*KPZ;Pz~trykK7bY9_+^umKGcvgC2Ho z_Hi)XgCqX|5O&YWnyj_2qdwx$Q z+ADpqk-IFfU&E<{TfVaWcljzO6h1psDM6yK;kSjdy7#T7=i`Z_n563}pazaK3H4jQ z{#f&&kv39n)*j8~KN*Pr^AcJ=@F>~ML6=c0kq-70(Cw8jKx-t6hWO|jrfR*nO?BY@ z4eY-<(g*vaCkE>vfhQ?P+Pfq4N8zPWu;y_9#o*JudJYg|MA(k(=+GFZ12i0=p6+hX z6B9G2>$Juslq#X z2Bc)1hK>9}2nw4+Mwmoiyl@m0=IGb=0%fe}YIM3n#d=!re3^v-2Vetx) z5%`k~8I+mKMK%huuW5vi(`dQ=@3WYWuqqoxy!X~EQ^Z&V1_q)>L=cF(fVJ&bb|?db z5kL1bntGp@41DgAk&0k`b1SO>?VcRxq?rnI>Yqmp-{$lvXzo;ypB5F?-Nr=u($T`cyMife z0l^D+n5b$35{lZ~8Gr&D%|ud8pk1i1x!XH%OsHwb4u=m9frT3Q#8VW0btxQgtKu@H zpgb#CK;PhXLPK7NZdRV9wn$ZquTTS+FmA!iN~pR-rn@y}(O2Acua$=o`(N!V&riQ8WQ1%qn` z`h~DMY#!3v3na!3ZIwYLYeQa4cbv%aG7 z-<+uG*fcs~G|^$h&ZIr2m3{Xay57F`b6-rm)O#ki3!@cptlOp%`gg@Rt3R`#M+SWg z2Z?x4Mq1eZ%$8yy8m@cmDoW`fCJO1HIDDz<1oYRSNwdQ_)1u=HEiGMGa4Ijrtt|Am zryETGY5z;R^Hq`snUMiIwq3+DaI;XKeR3 zYW@@*Ftqe4tIg!A+T&9|^x4i6zjUDziVZL4TZ@GcD$3kehZhTlzYou$GD91Nz1q$< zV72W3(#zWLRl+5Y-R74_2@0LvFmFjqFL5-vjmXs$z=?_#r`Z=c*cnj4aV+ZrGyQ#gih|#QJ_a zNOchq6{U9HyyWs0mRTpgfF+LX3Sg~q&`cQv;Y=nq`Wh<5U}&m#nY5R!-YNrm4t00#4lixwY7)@Br;R+ow&GomtW31 zPg0tuaBK1(#FnX_!GpViEF4^b-*{6{4U`vHUKi!(+n(0dbK!G?o?1!x#KAZgUR=u^{4kn~B>$7zW$MBj{&kVey$G zCQ2S0G&)FV?QF)iEq$_ErdO{bgM@}(suR-Vwp9;fHvir&aY~2Owl1G(iCHfIWUeH? zo_sT|H6Vkq!&b{S~B=CSefg<2eTbQAsAZxYBu)8wUm{ zsxLLuSUV_l_kycx`XhRGX?Sx9pLWY~b=OG@YPKonzcssc)dKIRKnRfkq8#ieAkfw7 zAgf$g^;BaDOO!MYx{Nw|ecB}tmd$O9Fnx-HgM)?=BxuDSA$kiN>wVtNZ&k4RR!;3>?qM#pMF)3GFIP@9t;hWtBhKtztZam*USuH^AJy^|JmrY?)GrXmuc;A_NOqpGa}9>1 zgg_@x<|SjUMvJDJ?}{>C7IPIRj3JA=z>?YRxhvb(O?pSC!l8gfeGJdXK zSWv&B%0sWHtvCvK{}%W>|97P}b9&Ts*v4)`pg;*A@`g{h=wpRoT4p-AQ3PPvSF$2J zTKBi6-FNn zO-MXl;@!D^l*y9ddUihJBQ)A2EiKKB+9{drj~9T;RO6`M`h;E8+P%sCLvIKTo9N^e zkWpe!)P7X`TR4j3@CUfcS|`I8Vf8G(I$>6~poQ6{E|YPn=Xj)aiB8`OvI2IlsRUOp zDkXNQFVN&%prt-MS#ynKR2?GEP`^ZYp+kD`#1Np#%tp8BUdw$Z6NfG<190%RjrB%# zpqwKkrgLSavN-x-8$b833L4(8SN#rst7h{jFPQHB3={68Xuy5S;BR4TSHA#+CzvtI zWpr<$NcaKsZ|HyN*@M>P!<(r5qwptK76I%Fi-O|fkL}{3L&#(@8D2JS+#v0##{!^5 zp^n%2?3XTECK}Hj&H_rBTpZZdlNb&DBVol|AYe;{QWnM%7G%Jvny_rhbDUZA95A1@ zs(}m3s#^mK!7k7s4#vsUTpGtr#>876>=+1P6?wS0mr(LZ4lrk{q zlJ_7QvRxM>JcR5cHHe!Dk>P`Ddu%t?4B%r)c`2rrZ954`S~@9-`#CUI-$iNa)*Jvt z+f$J`MDU*z3C<;dLMx$KWcc7!<}jV7%?@2@hW_l$PJ~bJSp=$cb8}R8eq7ekj`J+n zOCx%H(+YL3eI+~}@hI99wy%S2SF1z4q#$Ktje7tFk7XDmF}n}8(3q1AP);Q2N0_t8 zi;33&F{`43mtkz<^RDaAh=Pl^&ut}v`=P4gP05|b!rf(Gf=b`^v+}P^3=aMC^S9bl zBLD|byGyj~;F1I54O-aHfaoJ+0xgYTUNXdzn;Mw_q>PAf&2%s}zeAOhjfOrxWuv*! zPQrFl4H@O=hv#-g;(nc~;;eDtOecS@YbC;5KD_t~pW!dqlzgr+2XTuq)oT<%3MosF zPNej-JSxwJ2Gh~B&Svo3Tlal=I5XdH2$UJmg${=GCE)oO{cu3n4m#M_`Rpgxg)H*M zPcnbssM3qx{JEYOK(R}nXA|XsUr4kWf8*^yQ0&%LIDBVl$bK1>zd}tp_?HVA7p91l z_C4=V@V6thg5L1Z+cbYNFY{*QgFqPK_k#KK_yXz14J-sY;TjM1H}v{g;C$I*3bPsa z>J{7~`sv#za+m8vXh3PpjY?Wmou7vzJt-5XTJ9y;CP4%f{HWKyS+A zCH|y{oqrHXMYIwPbi9nb=_b7IyQyON8CFxb)6HsGJIsc0Ik4l7@ z?IYK7=3y5vdl^qG;PWyjTdh^k2e?RKA2F(_w|5l1GlA>6@V%3OHz&uKQYYn-{UN)u z zwA_6TVl!l%w1v>a!opHP3yRy zSPr-G=lH!U`RrC8%ZU(mn1@!}w90a;dVcX{gf9>K6d&G>exvsHrb@BSau)%fZadEFFbf`Ngpk5ir&cRe*XT-pMpFyC!>$|RM>o*SFT25CVg9V2Wp1N+gkk?4j z^ejea94;T&lh0q@t9G51tI#y~goZeW?jP4HP$`e~!Z6qj4+Coz#jgYKppqA8k@CVA zBYi5E*yAFqK0OGKaULIM&yd^M2Lz1Zgm0QDY*oO@AKc(MU%z}oct+qyuO$Y2DD0wS zetLEJ;@bLpCAj4MnORv~S)!x6l8Q^J3knKe2J^_vqrvA(x?K3u_3t8Un;FkfJ5jIC z)Wd5Chm!&G5z4Y=|zc8Gw=iaPiZG4*tR|x1M)fWj;AtZlimdbjbcyeW9!?%FYMhX}Z z;qXLW-R-I>HJo|LqoDPw<-cc+KQ=7LfbzNRc~5QJY)7<=U#i;}H<)lFfL?M86H7HW zS$J7*e=ps_yNrOdBSb&od)!bm@Vj9bdc!t{m9FI#a!@N&ji5G4bkU9_;JQGOXzgO$ zKNF>HCwrNC@_HmP4iFYBQb_sgCI7BYNe^+F>_VW(-jXRrgNvAO-1(hVDwGwi-Y2^A zgHL2u>oHPdDtJAdfW+nO^yH?AiQ}c9=C+yKikRP{I=o0UrV*D{B>wqns)3 zSoHE`=ONHxr1J3cHnhd@kkK~kUA^iV`N%*nT)GhfOSkj3&*a2{U=Aopt&ofuTm^XO ztgWqir#vltl$yGLC9M}PP?1U@%<`IGa(U8i?hf4>ea&j7+Cs#ab zC&X+s+$qg1Y`@6hD`zViRjv7k@!N;ok8xFLyLH<-{xl13j^(6{rBO;WYoFLdpdtSp^j*haP5v;VdaJZO9Kmqt)qM)2|q7n9* z{{A&pAic1y7T(W+13%|yoDE-TX${cn8+hUW+9(H~rk`4s;@?v=t7Uu7ukXbG|KJz# zIeVI9n^oMn361lKk5PJ<8DlMzXaz@2G{J^H4B#n~w7==>b_pbWXr?A^MCG)?R&5~U zGxC&tLo#UtrT**-Yi&hcC`$BvJ<|K0BZtO4+3MM!U!`j=i$T;gAn93DM{yk%hzcIU=IxjO-cb)XePOxR&;nC)IjH3?{SucAkpeJ+V6ir6huaB%XwYmeMrh zBO593T)DXkhbJ1nEm)eQ&Sh*Hk4}d4ODOkHZSAJNPq@#iB_oWBD6+f`_6awxUq?KR z^fXEsblfS6e7;p042tQ^Q_)ZKe;oi0GtN!zJJdO|#yQ(R`&3*)4Wu3GOdTwQ1n~sx zL9aSU&c3#<-t-o3=9A@_s$*)58{X1hq4-d}yvr&L7)ujc{>)jfoU(6bti6HDT@L|9(fU(MzF?CWq!IMy7D^A@PhGhBlQ8aOmif?bl1aHZ4?k0A;t^sp%8`S$jm#rt+UHejidd$;NjUmYAXJZK>3?} z_MDUI6ZgnPM?Z)PmM3_|aac8I8s*D@;bY<%^3kDPD;Pyx=0QM(_%cTYAV0QgGV=r#4Qy{tL4Za@va^Lpg^W7#e{swYSy)tI^GycP?gL zOn;0n76 z$qTUKfl2FL*Wr5HSn>P6^juIj=cQCwp+Ia!9LrlwyG34oQyHa{YJAo2 zUD^lH*(Em~U4fIIheg(HU{S*>23(rvzC}mslabM^Kg@T)Kw*s|unrl~cx3cQ;%)o0 zsO31@N0O{G>lxee6?Ef2iuA|Rj_Sd|-Yg$X|4VH>d~(i;4846R2UV`Gfw(}pq20sY z5w?fz&-)nXQM|=2K_7$F9^ueG_>jLmtOAQ!vLmqV#^Ez7YNajF!-S@{ve_1b& zk;-ohk(JFREB*8%QhPisMK_k6oq#r;Ye;vSKKu2{{Libky$!APVa8zhtPwB|JP9>q z_$-9KJ?M$x4YNK`;KuI4<8l8xXS|jr!lWmmTiT!B4%6a@TOoAHUq>q~b$?W*jXUa@gIzGrd7)yE3`0Jsz(#^oe#b>-D zFXDB_uRXeb`!K&>C|0eOKK)+idkrt<)gc@;sJU^>HEE>*6=n)-4xL9cAvYhU-Fgd} zJk!_TSa|aVzV6>``ulJ5Cx9af?r>46!0CtsQCgU!$(>Q+RGZxsGEK`w7SOa76?yxM z*1+teA;RCg8+fksY8XCj-EQ{t|Di}7po^;`&{VeH1_A&3OQ(U2-|?0bIv+kLkJy}h zY|f!p`E(`=g9X_Q7utB*R$g3;!{_TRude5w9J0GU|2|aWU3SHwICV7nL;402``OI^ ze6pH7;J4&9_moXP8Lw8RHmUtS-%aF(8NmTD?) zIVba|?YzqFRh&;;``75fILD-S$3<%6q#0QHTAuHpp@Nis$kZs5`-4ICV89GiZb)oS zh$*37(^yugBupcVqe=8EFuj(1HI7VIA%Ht1LEqaPtdP}erkX_eH$qxLcc5q1!&qC74`Y1# zt+PhxoQ1iIaBY!m@#95*sHAG_UZjR7#C<46tpc!{OsR!l<$r#HA)HL5qvFN2U)#Ix z(f_}MKLsA8muenf9d+D{u|7?{r1CT#I_LGq4!`ssTl4Z_&9$$AHD8B0ljgOe#rJ|n zsEhG#AM{jz`kF^Tg_rUw6a5+0+uag9*ZbME>topsZvlGW9c?tv)3EGG8rmEGb_Hq_ zc+EA^M0N!f7(xfOqnRxDGZCZQi!@-Ra$Or$w(5jy3yJ}-yIlA-#KI4Jg6XVv)_>j4 zT3-_4$2osfk22<}zt|_SZ)WA7_>0T%9`JTrNv%eqZV06#uc1*fTFNgq1*-XoxyB$g{qo{cu?7aL8^7JCz~EgJ03=oV z(`4$><&l>~&e?JDj!GizI9VU1yatrj>!D3^1`97IlmNL0;_({fxc0FE?>09;UL6B1 zzA?Epo`O^-26;yUW?J-#oJ9JkuH={(7w~3fiO` z?dIbvxo6i>fXuyd9ip`N7L6N8DJlMIdL4e9VeRjS*5tRG+ z9M+(JI8(o6gCYrLh#*$KQ1TW8(ib<>)C?owf!mI>Y+_{%l_Ge9v7 zUH|gxtG`v#1$!R&r#ZS3GWLAIs(5rETrN7?hCNTz8KByzEfQTym5*%#&sV>aFS25=~Mw-_vLry zrGM(G5;s*oyQs9d_$(Eb%mP+hQygrgXv%e*nF;9k(Pw)?Gs=P`16 za#UPW(ud2c287i-Iy$<#pG_nmS%1x+MmUKpQzNpX|d3QxStWVhg9?dtKl z4&_H}3AP2M5A*WCON)oH_>couqZKI^7gyiKK_J9l?$7g&EY)ml1=Ka0&k25Wfqbwc zD?pNW9lHf67j8Mh=mWclXLkqgx9wd+%_y??=owPY%+xozE(J=%xl=PTWM6jH`>rKM zdR`Bhk%}j4XlR(MuT>y}1GkjRbv7Q`)tTc}cRn)JG|}>{aL+1Q|6gw~Y#K7)D=U+H z@W=yLl@e~q$?$y3`Xv~Kac=fsFf~cH%l=SN^g`royA&N)NB2?Yx5AUqn7e=6MOqaJ zOR6r6X2yVfSmvmMLzfUZJWQ<>6^a-=jKR^bqaO7AB-udXmI`Ld(++Zj%Xtsn@|bhN6g@q^*0MKb;pU;(M49E*emMR0t;xp9V4)vSJu^6~m&d3;YU)4=c@WAw z-Lg4&R9dl?Em_c_nz1GWs2cx|qX!5QhN-P-l0%X2)@0O}EHz$YH*$@f-!tZ+!etR- zbY@%T*?^_n<2^`x_0XD=cuFL6UWH!K`_pfY7>*YHKZEItBr2F2r8%z}a8Z7T zYv$`gEYi9UMY#j8K&JyGx5 zQ%8mvJ=@I#yjMUmtY6{TSN6En6kTwx@FASj@R=fo zPoG!(HJGx#H*DiSsFN7xm>nOntJ=ON>T&@40kPt%M4^$vvrO z9F~#QR>&zJ+jZ)LgQ|?z*1(xm5{U~-FCoC_eRs%7G_+SF3c74x{y=oZB%VuP@=i-T zU{%f%1I<9d>hlbAm{MDmTvxhKjN-}9{{0rl`%^ z=rZHncrX>T|NheHfUEc<-Goj5YtQu&(IfGpo++|;E+1`?H^@iV)|)^OwQVnyq#aVv z3+%|Y0vdNZnDxKpleTY|_C_CF5f{IZakX?q^K>%yQs@errMIPUMkROQgTw2;-6XDA zP_wml`9Q4kvZy?W@X{v&KfFs{cwgS}4DUui=k6Rc57x3txl2$=D#p*#SI13%#QEf` zf9n3>u#vSz(D=6||L(^wJxwMq#xAWRN6Az=pLbeY4c__F!xTV6Q7gxL2`Y9yl$RGprLJzcT$~E}+2ay<- zI9#YWT}wYyq%o0%4BxWWnvB9Hhy@&()9qMAgnw7Ae2`@}Jf53YqLzZ}>g434TlN_+ z%Ngl@!dsdkLn5855iLWV@`22x4Y79pBUD2dWp?#?4*tEe|f{AgG{Jg zsD)4Wu{BDoH~nQlQ|aPfaV|ZK$%FOF`Qf|t?sOtNe*?b5O%3PDetgYzgK{)8l03BE*9BTX=EfoQb>_9o_+ z@Au1p(lz-l$jXyr==%e$Y(Ti12<^tz*A~gF`6#9{ zRq#!t>ujKPz%0>tm2(EvDS(AfX|j6`DLuudo;I4!*NH_z)!`H;oBl1{oGC2Oios9b ze+Sx7mArRTmI1)vg7n(%t46S>m}^5)Jj{u{Wf) z^UC}6b7HXhA1qy|3|gyehYuAMvfOw_97m&v$_Iu8d=oV1TV9Bi)AE3uSgkB?t!69~ z$U9lemd^)Ps};zGMb6OjAi8?PZ&BD*j9}tiQ{ego)U9o9)%5nn_}%OS8p&9Lr@c*=a3Dnms+{CL&y^^C>P_?fGbP;Yc^+NO?|$7EwiUDcfCeAjUBHh^b6&PP;yICfdTE#5Rwz-$srEFe~wBA$%@(n*T3Kp%> zw{N}iKAw-ilfX$&(wi5wDb0JUH*Gg2a+cQG)JJJRLRqvO3Zuu&J_-valObH8u^SXs zmcrX*wGP?4!H}v~L+H(ly6rhX7j5o6*}Ifa)z%hTeb3(WsYS%J4&MC0Q`Hi6+OwC+ z!-V6o@;f~yhx-lg8%-@u%U-ZU#f;;KV2g^tBnMqb;-3|O61*{n*g-V@T*}r***3h) z7WBCv;QD+dFk$26%4YtyvKOFS|spAos9vqkRFZpVrY9FPB z{=2l44OhOU<+D#&sIlJA6)~Pnwpm)RHZ$)UX&GcF+%)w{db zmr?JP9ElvK`=g`94_0mSTU+PODX8YHCT^QcmrFn`;G*mgm50C=rBERVJy@$6w)R%% z#ZyO)q6t}02c|~rvuEsw3h`G2P7NYCDepf3e}4(i>+@_^@oYW{BEtmN?xnvLfV++X zjT|lV=+PrP;QHDp2t>M)%ZM4tz~i4?8)sijLl-9)#mDt01LWZO19jV-il>9GU&;9i z#1qL)RX{|fK=a`Jpw8W8ZBhQ6>rfJT3X!y7SH5@DG6j(^1S<9T)7F2AO{+7EjAqe8 zt1QG=`p$)$l0VFCY`+w6Vnpu4dxwNIj$oA@=IAG3E=ahJvIXzIJ6=vIGT_u@^x5h5 z-9;?ASR)R45m*#|f8vq}g|X|;ETg(!mvB8T>-Vpq1fF9tH@BOI!QyVPP;&zaU0!B8 z^((v6le`?QM{XpA?q*LTbf`!cS6Kp&9@8 zzJZrd!rO*@poUlSS1>cq)prZ1-beR)KR9HoRenuY5Xe^I-C}qoE5*FeqH$ z@M5)9fp{c_gq2rV?yONhWt~`tbVd8A$al~})ux;I@E*6E9h|K%X4o|IT#hj_##=0( z^Q#!?K-Mw1Y^b#@0s)_!1z)`(_fwQgEIBK^lttH~!Gm8;e^Yq=T7p4Np2dLn}&A(-$f@OGv077&9KZcgQbJ$t6%yV zuh+N{uCfFk+LdFX2%fQ^7!a~D0)v&B;p3M)0m^UF%-aSfUDfuL-dUyldj9byTYveA zr1$`5>br*WNJfV8>OOmkHHBEx=x}I>oXZAfz8D=b<8#`Uy#iK)nKVAcXaq<_qM_v( z`4DSH&0^Y;CemiMgU;!{X~2ST^i;|oB@Q~+JZYQR8YWpKz3b-YS8+I65P#GaQjida zH%Uv(sJ|gi;081=t~9F8wzN@U@EtiU5*WLdO_B_rt~P8nHwcl*11qJU-xu{Q3lN?! zF@)k=BSQ(x;U+%n6Saned*G5bG^K6j)}7GOZu##ki6?fLFZ*hsJ0{}Afx3X#)H+wd zC7>WPQD-krz2L{Geo2h`SFA?yi>m2l8>bp$x(#~U66|(uyRE$XD6$&PJ9tw-t2&J| ztXDhUQxJW}dy<%0)%7nl5Jvj2m9AC;7==ZNcvktgGL8Kbl$Y+HuZV^cBI)j~a^qf> z%1LQEfR1wr&vPv0pWr&HagKVf6;8&B7wevjv<*wk=0y(T`1O`b;v~PM2YOpqG@CqN zij#ycqV38a)#$nA@{hD6*o*o(z^8f);b{TDGeeiBC{J;_2$q%feCLAKue+sTBW&>n z*d{WloMi9wZIFD!2imkZ^4Sg%n@Kn~!xG^A>K@2=$6NQ9>Nh)Nqm+TAPjl%ic!6lU zouH=t?KFN9ZB#Awe`+VG3_dLnikw<$??SS<@MSrd=G#1)7BFmNnUl%;4Z^4wo51mv z$;TDlcing7KB1z%$`9My&pJk4@MKogr6pzW3S5_qqFiD9m%*aOehY^Nn~dLPDD|@t zR6`Q$jsU2dhl=;6tru}S(0whSNHd>=rnBwGV{v3m-WXhot}tpXKs=# z$$C5$X`47d#9$LTi~sVcS@jG1LD5GKsZ#pePdist>@d!Q*qes8-tT>uR%weJZ?`16 zqg%E}_r9w8+k>-%2UZkg@PvADI<{!5h)QunfR=6AIol zXACWe(QhrPyb1~zQnnf$wsakJ4Y}Dbm1HkUUp;kjbPvFItMmrnVvb;Dh^LVwUeFOu zym9Rsr`?=};jh0Vr)?k9LolN(?R_D%k7G~~amR|_{VVD?8V7xJ+IvVfM**{Pln&?Wov}8 zEzg0zZD~Lx*VdQ- zWHcRKUS5_)2wRUUHQi8dlz1DOgyMLYQG{XMZ+2L2pO$oG)7F{4+QYouKHBJ+ZF+l- zfN|5qmtI_rd5%YDK=wh;9$ce(A|lGnC(ru_OBKn01gPL&vUqql@Jarbo`>`mc7}p- zE|JL(P7?*xT`1tDAS^LNe}`ubvreev%klN@>J`--Tjp1TJCHBWa+KCyJV|4{iDdR# z()Kx^Xyc=Hls2Vw?--_kIreJ~c>CMY16NA@|B#$GskFhAlZp{+yQwcBNDKQVaDn2B zx`)nyXq#U;|KQNIWbl!wtTwiI+5l&YUYAAe=!t*~!Kcrk-N!y435L*m(_bfVU)Z}) zqE0i;SdJ1^_HXm2?QXx1XQ6*zd6<^F_Qa*zXMV0uu$ChJ z{8p=8tB7O0mi7HbI;l8)AzEHHOW^()L14wrva%eM7x;~Dd0^&fbPgE-nWQWN=+=0) zd-E!O?drsg;|dy8XE<{Rn1{b-qDNes=ccs3$7 z5tpHahIe26m0LDjtoaAi1qeClQZ9Sw`Q{d*zrBn?)~74zJQsPCjo{IpslQ8yMKU6^ zPHM{>APvoB{E*#H>FQ3m%V5>%Js-ZFi%GQ7j~Jq9Bm_U72mR^MR?8OgOr}Yz{h`Dq zY16(ohuj$-mg99t#R{tiKP&feEWN(mj<8kz0`a8uw!U2i=ehCo@84o1fp-KiMCB1d3hg|HsyK$5Yvd{c!Bj;Yeh!gRCRjWF1tFm9h>pGO{-r z;bd~-uiqwHiWE0L|NjCz0Pd8()9ec#W=$H#xpeeV1IUDx%!zJqS$UycSF zJTz~}eXJq+8?Ce3h3eL!!OLmLrQ8}W=KOXUmWcP>EW~k9QQuObXK#~NB;)cCc+r%J z*reD~K6SKz+sm@P@x$k()-5ZOq-&;HX=X~}-0$h;EWf&`%FGyuqH3vQ-#U}Kyh=Ar zjy53qkt)FkCkRUW7n6|pBTS)(mtMmzS2!G-Mv^l>zcPeF*j>dMg|D7(fLrl`C8gh1 z!2tQvCH`N5d`$VDkyIu2#TR|su$PC&T`1bextl;HJCXcmNuqjiD3e{ zXpEF<qrgl#m9GY#{K|{!stS;f>4<9 z6c}jm;H8o}z$ZH!q1*+z+6J(mJg*Nkb)vw3NIE1KpzCS+gOigh7r3P__Zfd<$=sWy zaZk!ICTGdrHzhn4nj&=%$uK4rvW2-RWvjfpXKsr#w@QVm<_o0io+6_naTY~kKja43$jB3Hnu`k^)*=3%*qShYo+(HoHyoyPn)5**`3L!d)*NWdu zeEFl@MtLx33kahptrHbKA*q0sJlwv@>O~u9ULAuYh%iJy=!NC?iCCf2+A;8HsTgXf z+78@x_Mn|)Id##dEsWzyhZ6|}^OY-CHjdKwmBCBiUSZuey{`yN6GgtV#o40omiI~f{?@&)=zx%d#NWI+`RjyEoyakwDIw}M0kJRx z%@VIG9QygC3KnPjnhoM{THJolUS4m_bS^yce`en@@WrzoDvX6C$7e=`L(h)VYt{38 zdi)gj-Mch)^Uc(HM~Y+=3->7L_B|_gNtG(kVQXh178vu{0-c&rLSj+&^*q9%wBOvP zC~zd}s#y|#qw5z7^+~VM#zs|gnQDkxTb(g2EaF8KrVOjfva^qj0XFLFtS>m;fxk&Z z`d+59&Ri(b2NsT(V1Diqoh=UdL{sh!B^@^oRcfJ*@l5*QXxw3-L1k}?&L?dyU!?1# zwY>tmNnjbIsR{E;KK`x<8A>%p(P+&*lVwucstl%2n(n zElrf6qAGc&8m2ba!u+w|vgenNSGuAJ+q9)M-nIyz5js%tQbrgZ0WS+L6{6s07OlM$ zwsN{koKi2+?i~Q{JVi`bs0@Q>^UnR|<1alfIsCxiP-9}byYMK7_A4)bg~xYmi%~Z& zceb2lzRSv~v3AR->irqCn_iri9%{bf^M@^?zE9`3N4HhkctJnza2WUNG-`Rwfk+69 zbpI%~lj}Tuz42KHWu0xeL}zsNU0GiBXT7xJN)5~&!wW*EtuRtUo%Mo~vmr{I9br1n zq-ABuBZB@fl#eqAODQ7k#J3KGVmQaGtzoH|P-nr|Yi8r#mbOi2Y~V%|lPY?zLsI|s zGEjo3q=o>VtcoC?XuF`y_DoyH%4df3zqQSk!H{t=JftUNYjl;_b|^S-If>HkQ;P&? zpA2*(8p={^%gl4>ZE{N-Va(agngJFf9FLWoB$`qUiL7h=P8#Hac=D*~L}oPy$Wl+y z(_$xD(tm)iBc7$@rTbj2oEm|FG)~IOBzNnBp_$n_$!2pCMZ?v)OR}8&9OV*pV_LOj zoteNq#_{%TceUh*J1o;Al@@pI9Zz3!5zk@-Q4S7UG62Jty5o`Ihgv?BcN98Lo5kc6 zSpJ?awvDLsjE?=Bk3OSJn05EA{zWT2tR-5+R^N}(cw50KV39c8Gsll=;&4S-7?7&y zjY|xFEq!M1(#}FyMcVCqE)s_3GELChxV$p2mB{V8#yw3(a7)Fl^(kZN*ES)#&ZO0p zVQn%K;n}=$+bG5Y4oTC(6}Jt_^9)VT%`w^Cf>3)_0!uDJCOX!jc7qtF=pOzEtEw+m z^XsoiWcGGeOf|7CpXtw(GHBB@egCCY1H4dt6R7;KF3dpthFjriewO})FZIBXj$c`` zSHcZd+zJ8S>mc#y+v!#?U7|_u2q#Nh^bz<(fG6$r`%`$JPlj}~S=py7p4N<>g~Lvg zue|xvBv*_XqyA zUeLr$nL_NyWBVy^DgH^qbv!6P4W)lOF3;$YB{xRp)u-R{#8&O3}?=olC_x@7=Q#c{18RxWb44n|ojz0gEUlKVh4?Q61lkh02-sH90bFAKR=f9oU-}LT)Z3#EZM)Y{^A!twZx>s0zR45Q zyXJ+oSzoTWm(J7+%TrR_#)u@5f^D>)?yVa0-#tJ8=WJ~=uT+{wQF}?2H$8nEJT9^jad_3b%u!W0 zc9(iqj{oAIjdC&|YawLPOTJ-4;NDP>H4$F&_Da~iwOH<(-248$}0h|;oV=RbP_ z`SXM$B_|>WkH+-Q)Fu3t&H$&V;z#i`7^FGIzWs=8kzU5zV*!^Ym;5h4z=@YaxLApI z3CgG!@%rKA57_ms@^BdG$!EyyK&ItDsSbb8P^2BLSREri1}EQK#{EIZ_qwavTNxzDvI zhEvC|d1;p)uma4-`m^w~IR#5ErTbKzxBkQ2q^}%53V1EsmG(e+ zEs3D4%O>`bn)w8!;r?~p4(_U^kt?bR5HBKMvu?Ll#_JTO)OXx8J{ELMaZPiG56ZF* zR0t2F3{}}P&+ZPdHl7;m=qxfDrI@p(cg_c%GX?OkN^ZGZA*30RKZ;w{UP^S{*2Z=h z5E0G%Suc)}xe;KWU29gl*$GL^z71T)7<9q@I-#b{z^Zeup*NAT>x_)ZYcaq^%L^a@ zQqfrpdD3puIY!3Tf#WPbg|q$dtQB>C$8bY+g zQuxxCf%wPZXO-EOOYgbqZ%c3KY=@wvu>&UsRiX2cSjDPvx|!`DR>pLlg13klX zQtm7uqtbMeg&8hAYZXprx#xqbNIe%hs=3{5)gE5~V=P3VL`{R)S?HQoAFEr=%ZEi5 z9h7QUN3Fc_oy82uShQE%rC<{qn!~ZJXW(*!hOMGluT&(bcClEC7=<6K$%r5NxuJ1Atr>asLvYX}VdeqNW2_hVN1@*?#&sz!- zWiq`pileU6?L2?$s#lBfoGJZdSn4T0B%SRiJ)!Q)2{;tQRDAkT0y&9!SNk0FtU{xl zkEFDh^~L;nnBA#Voqf$uzVWk-R>_)xLF(E`=4G0i=G61$0c~NQyx6C!y*SB#0U-{; zJk5|X`?R_Hxzeqx&(24|qUqso&DCTjnfzCaZ(>Ugv>g`d8$j124sJ}ZCRn^(y!1V( z(-Wy1XmQ6GL&sPeM@8{TSGREXsA>Ou(&msICI?8OXB8tRf4Lu#`4BFY?k-BCal7Z| zzt?UR7z!P|YSx{`$j0>YMZh;VA$r?RRmA0TVkUP3EbZo`L`9lddl8emg1FB7A*t(^ zcIc?WMOl!TN$K64p|q>QV#(VUU-3+O9u+d>PL-V`xH}B%|64r3@r2xM{_-7| z8hI3KSHXbXoh!5rQrgA9dFtU6u^IVP z{J#LUtwHkgd$&bBxcxauuIHt^-437B~Z!J39PkkV{y#@}jTyBTzha&Y*o2z)V< zfG+3s`aW-ICu~Z7+tcJ&4ewVJQ=BmV%L^c(cF*8vqxp%BK!>p~ghMxa-kR9OZ=&kp z+%VggtB2LYl-!iJZ;yF;R94|Q*Q;uKs)Q!O70VV|rQVx9Gv)pf4XqMlmS)}M%j(pR zQ((ljvWVsT-N++&0|G4ka%l5|YGv3B)ZG+)|7rxrLg3Mwi=fhvawr_>OEL$SbxLT z-#K(Ox{%^wSn z>gD%c(IqTMDy{;qlxIz{va;5{a59?rO_NbTP5F(eXlP^!@N`}iG`i&F%bDi1J9JKq z$YhLENMP0tBco*&nB!=1g<32_{k(t8K`b!(3ubj)|0kg@gsk@rVXyCUm};o=!~S<( zlQP$>16l}Gtqb*fv=eIsA?PZ*?eo?8CP2iW4CUc!FR0}C^yaR;+;6QCvgI^nMV6#E z|M}^JXts|Yc_}bcm^&2i;ogix@U@b|GB|)Q7-P*n($kYb=2;8^3-JzWNBf(#O-=Gr zcT7P*#Swnng!lQwr&o)Z)>$0a1AZ%zc`Ui*ezkmtQ0BzkFgJIx>(8t> zJoFb75%IwgiSuIllp{Ow;gBWy2ga9P;-RV+L5)QpKYpCOBv-xb3!Jw)%(#b1e}Ci+ zA_APY)iWYbUJ-8jRf2@z#Y`3*P z^ZOoxKYW!!!1&@?nKG;vMF#}!xh3%MXe*#p7uJ>nJ{=B=UZhLc5P+E01__VyTfF6)r6k3uAG#q_>-ahON>VJ@gruab zUOq@|JP>!BsI=L8r>|455yP4$V&>5wBWPGEEr!pA5i>9_7;Tq{IZTxS6|y5!hJF(I zvttK#FCchSqU6K}*D&1wI9&h7ES8!oflHrjn&QHOv$8ElSoAXPOySx?qgWM6z6d?X ze+xL^vt5L&h((8K8_L4pAnKp6;7RnE7p38YZ*C9IQ$0q|mc@a(;D51TS@oL%sC4EB z5>;@()n1P&jRcBwP0Z??fhKU`?@1~wDXGDGg9Gvm4BhBc(!4k6sGVu?9kMDX&Bs_8 zw7YWS2(S-qz{^=C%*VH6P&Deu3ETjYWNg>h@Bb|Ky?TV*4LH!?`dJ4sokZ&x`WPAC zAkDT3hF4KVL08>WdlEvS>pfd%@y=~M7>kUjAPjL5FxkzpyX#@mAU~52H7Mr3Jji*p z{5a9+p0 zncW%YC;XiY#OB@XLI?aZxJbmbbdQJa=9}ry3>LBqx>msqCre68W=JBtoc;WID~A_^ zMY%s*^?*5hTb@7P$<^L2L@1ouhV?1=le#&ZOlSWDF*Ip+#(vR@t*wKX0;V?y)QJyM zmMdF%fR*0%Wu3Q?TZ;hIUB>zyt=HnY)A)*FsF^C{{*4ag*D|oNZ8&^rdd>{|xJsEw zvBplr*MZg@5avZb9(E^li(H(NfoBJe65rn6*%=3rw+ #ppcyij;Tk;uS2CyQ!O zCkwbhD;icR=a`@0aTsIh>(`Ew(W~#wKWYH+(>YiB0S?1_`jH9okI2YxmS3IR0nt!3 zi>2xk&5uBM zpJL5bH?V_tZ3ZX-&iD|lvp#un@FU9^Et&y?9$o46oVTM3Eq`9f>CNONkmgK8L{!Ru z8LVMs`H;W}v;yZeVWs%(wGKX3OS^6sY;yy_L8jetfcG~QvO!HBex&R)y2Fc z>u##+Q%Szpr+&d~{~vndBMa zOrt+g4<*Pan$TCJAtTa2#c|9Kus0fjZ|j@cKMp(YhqC_heaMstBuT>{a{V7 z)57yX7F7|s{%&rABX3d!*Q5Ee-JH`IF?~2~T3XtKt?_YePrQW>gs4iu$+XU`6&Ng; zIfGzL;1p<$_hYjhqiVY3yOD|4c1%uAu7SYf8oX0oWo6~Y58B~#z$S_u=6SefzwomR zaTJn6)L%J6JIy<8n%h@wzAGs|G!pPNUo%VAbEvt8qNjHanb7jGf65$J| z7KLi+Mry+k5qcybKDrIUmrma%)_Mi*h2m+UUG1{*0Jh3*55;I#x;}skE6%}a(N|1{ zGnEC|6rPfU3a#cgURTRM4diSl)cSt$Op>{9;cYad*Y!bxR+DmzP@BJ#-Qw^jadx=ooraQgB|M`y|X?BC`BJgr#cVmLtC7)IALh`V} zUem+pIZ?M{x`2 zOwn+hZT>w>zKWq?8ujVZjbngOE25)^xkKlLEN2DIbzYYdy^lJV1eAdJ{wf!gjPYgr zkx&+$kSI#dzMsT-SpjcCH9%HF_zT%1fVS{HH)~fkJK+XQh0!;umm1%WJEjz7f1i6k zww?VnTmWaK9Fqbt)!r!IPRHwIoewO2Z<;{I)2jgZVk?=}m^NgAnUb1%Z|y}s6c7H` zTZi&}#3or=ZDcj?yT;=A3K6l+Y2$aF*VgX)yX&UU6RJf%dBOvXn{hnXFWj}I0b$Q| ztl;&o<$f~un~Av*b}FlG3@V~GMqB>aaB@vLlWtu>bu`p!)*I4R%H-D^bR$8b{`}Nc z4>~X|Oj`|&UipuTcp5&a(uE8-1=ICkrwWh$OWSlt8GAXVg4hk^bTiuqXqM|3j!93m zRpSkJk7ogkb^^$;@D}aff6XzLrq1%tq5-IywVa!~5Sl17Af{YEWJW7AAk#XkWrCdb z0f`UOAco`jkx~aE1X%wlK`l}ulkRDanGeAk#&e#X79E+@(mk4qRU_fgaSi;P9)tY? z7v!lVQqq0+X7cvI7y0`K^pL_>r)xT0kjx6Zzhu{ry)C)UwIP8$g;$N9V1b6G%cWk| z-4cW{EoME#aiSYw;i)qAVo%Qv5Gra{A!=n2o0fbWu7ej%)Q{z5NvYnm=w+Pz2> z#Yc>m+t!N(O$;cyaOxAGU&BA39`Vb@dHRfZNZ5TYD0xYkkJL15nUp1Bpxh34r`JWPsxu@i~nIyrXSxn(9r(@t_BmlT{*-u-<^pa>@6ZNr50>>r^ z(X-M3CO0@#$L7K?ZA7YN#$b8i1Da!C!PxMf@w|bwVX=`VKFRw=k-_^FXaBxP z@bDdMv>`!thv;CtEBOpQ)%j$#*81+h=w!*c@v5WQGjr7u z-!E$bE6nnZ4HvvHR;?Bp9QPx@4Q}ydgCcr;=qp?YeByqa#jK^^`FMtea9(TS<^l8lG)Lj;{zc9*?+} zR_{$xfhoml3gl^hVxa>eMDeUL9v$^OTwGi%W1z>k20_Ix` zB0|q!POK%Xc8Nh!{z-A>AZ*pheNPU%*DvOd0TT%<8)f?%rN6HD#;) zjrYKw`BNjNL@M$UI7st?<7y&*zR6o0WVx$_84_L0!w<-moK{$>4aSe!Ih5$zI4clx ztp;bGPCjb8-h;XPC(@7P@FTC@r$)m7G6s_ib*d1rBYA>Irf0YL#^l~Fy#4NcjmNy- z+Rv?vfr*I&-qYdaXuom#rZ!>;=^@;z*#aoqfR~x6IS**h>UZtOAY|A42t3H5>~L#xJ9d9}?fk}RYWU492ZpOVFBnVRJ8oJN<%0e;0}<88gSP1YBboge$?^F_L}v@b=P^sOV^S2bSW=#UJo}aGX|b72whwgSr5AT6^M*tI^L^& z>D|@=#^3b<#gpfFxi*MhH3L$b2@nQR3}}3d*Qltdr1j?U0AD|Oa3*}RM-Di|%xhoy z9Q3q7Y=aXyTJ7JCKYCa80V#=lqp+GJkyud1tpJ4B>*#*<@9e7;;0tmOSv9~hmVi43qc z%U)L&Ta)FI@7-m8@nCmF;rB&@KRh3tD*IvL&yU)f8u}tB7k_~a77aol-!7~Ppspq( z#GY-f*55h&P+Rf^BrI&_srt+Uu~u7$&fPDtr5KBDiSS4z4hwu#1x#f8)93KB=H*@N z!P9`ku6hlY94r#DY0Hk>h1nE3opQu`T!Ae%be-R(*6~9o}j6c-D zPiW+o+!x#yyR1Y|xc~Y^Oqu9)ZzC$OFaVN@wa*gudux-X(?#>_sHY1~vj6+lcyP@m zfTX0s`zi>6V`_t>LT;fxKJAjToWCt=y9nAkEL+JuXl&-aE1np@~E>Uh8XL(O8d|%xf1fcF!@EQajelGZ1 zC|P6azukE@z7wZK_2Svq7CxWoOz;d~qk_#%9lwx}S1KnsVk9K~_6DB$@!dB~uOj%z z-P-Ia?oqj8BEV<7sOkywI=Gv${k9lw zs-`7jiF}Uo6)a;4g7#d&ad80tG62T2Hy6o13Cl6f}U%yZ%p8XL?`akV!Ge{=kz}ZNJG{7*bp#@ zb-2IdffsjZ$$>;=`9_0;=pj8o8()aa2@(Zk*hNW65E}}B3=mn5Mxo*;8*J;mLG+K~ zNQaLWh#vJRjRAUw)mDs`j$1i?=&$EruFHL3C5DX0U@#NUSB!tUoR_iwS}!+pAs}{0 zUyFBs`iD1+V}m^NI!!ZdlUzpD4Zr^Y?@&Dcb;SFV{c?V&+ z2(kIFXl{D|WL%4d!L0;PPdX=tp}g5<83hK~bns6bR|45_y#S^#JAL9NPX!^ABw>b< zCW2(!EFvy~lFS3S)%niYS2!KJfoyLPV97l)j^%w>?+288D56c&p;CzEC>&EdJ+%mRQmp2d_h=xg_)3SdLv7ma8i4Zo6 zheyXk;kq!`m=Iic+5>v`tSs*vt5&vyW!^n1M5=`tA(K6KIV>W#tb#6b594G{9mzJ?xxBD5p zJY*z`;-aClpY0~j{2+r3my#uh@(K0QpIv@3du8AY3j+|7(^z7x!qRlazKVZ1+~NOU zK@Ke?WOd|XeyV0iAwdE4eF-VGCIOq}U{Q(ox{ghu=b8F)dj|Gc#asd>Juw%`3^p)6XiX4~4hl#jsypcL&?sd@F3_NPDLNq14ed`$! zhBBN4XFdP8woWMK`(1110vt}#ke(Bdk@iEXc(D?#1Gi~ICqS9U#>Zs;mh5ejp|s7b z$fddeGC5rP`quVnCGY`pM3*x(9VP(f92M5C?c@ENnmDLxU{!1q>Awc9^!L-7r;?l4 z*3lB>*?%`WF8Xod_VyJiJzmPOkOA$bc-|f{%QslEl&bHF`|aS}LyJ>WQPnYMA(h;@ z;J}m727J_=ot?TtM_aAy1_t<3#c|;w@V^HZ%J_Aoq0g+k+#hD-9J|xr^yxi}7C;H=sc88AskA}0-Fc%i>4?Wb)dqdAjdCVkP}_)CICXye&Z}wmoewVn_kuU^s_8_@Yu3L~3c}#FAd;hPsuPvb3@5 zhRWqZ@mq6M_aAR=#)FM8SbR2mEXL@$<#s3Czi$PmtI`8G(g%Qy)}JY3jXwrk$AW@_ zO2E^;a*SmaRQL8(Q@s5m&Dp5wGy;nn) zZ8>mj`t(NfOVy{qRFN$r?n0f*1WZ~$vW?D)OGqTF!&4U#0=f(sQw(q zfe}nGB)~zdQ~?BBH7hb~CXyLGGcL^1pJe~{!CJnD#ziKOlfiC+<_5uwLRwynfx#s| zPMMR#z`fd5JOd_Odge<%jy>_gVt7*&kX3R4Yzk9*9#}lXt7Wqkmd7isOU+3TEX#_2HJVv8w1nK;8OAHH;^J0fNz7)AUm-HUE2@F%SHSL6@PMC zD@eep077EoXdo%txI+O;hQj^y(?QyV3~l{$_tY#FYvaPbOT|PtV!h45;e7Qv;iM#Q zn^^@UZ%n5&(_s-QYnt{SkY1wEs;#;9Jutq#*3Oo{E$6#Y4WvX9`D(G5XpHcqQuAg9ka1BO4>JR5mTKUYHC1U7 zyZUVkv?-W!IXSr!zlVxN{M%0ztXwcTEeIEi01^$0GWMFU zP?|ToS|9|P=~{flry7qhY-Y>_jy`?69#+hJ^em^h#gZ8k4YZv;cl1WxdbhP)M{Rjx zcG5ojj<$R>LAeJNTf;)esQLAQ6Id#T-L$Jq)oswkh!cX zKp)dkbk^xTvUjC1_zNo_Ulkk~zhHCZ;9f9Qi$VRt0Oi@e> zMNw&~&jwk)OE56EnGSWaw|7$>_cAXBo5uB;F(-hWaC&mtYi!`toA)ChkN#?{67MUq zU*LGuKvGiSIhbHrJ$1Vp?XXFKa((;0fw44?*B;x6j6_pi;0El8IZQ$vLGMFm7M7~~ zCL|inEIqSWBfXZZcOEvDZT=320>#y@1vG1jQ;jsR8#HN>6SUQy=Q<&&WOXj$`c}>Z zuZ{tIV1>Z-^|a~9N1o}|)*h(dzQ>+FaBl#V0#e3I(ZC^AdmKMD9)VnO2YNWjamwTd zf@~pKZo<%zo;1AVJ+lfU>6aq zvj7i6^@1s>8n4q6Pb1hM*Y}OIG+&P7t-sysu7jPmDj+6_p-P6bJi`?A*(f7UV}!FJ zmSb9ujyPqhls;mWiufWdBgMoCjYl&X#iRG*76oM~ifX*r5-krc!V=F5Ulik!#GTk#Yb;2&qT zoXs>@#SZ4Qj?-<=PXF4DW^%NoI}(sl+z<}(p=y1{H}{ZiHX^x`W91^m$c{$?eNLr# zSPzUY453Ty?Rsv?TXwk66=cS<{_nKXB7~!H#%?EAYN=>R9VY@}W5TwuIXlmR?QdFw zm22d;G=+LJUo|`yQsDI0-WpW@jqZa#`f#GZ@Ab!Asx~$9~oZb+Awfc9!^AxB16+ z5VYC-`gO(hw5cFJzuOrmrq2*k3K+0E^KKn)^jWV23>!nh4j2=MUpm>*x7Hg!z%ytD z6!tR}`{~vfS@A2ruEL#PLoF5tfphT0C1^D?MT)+8v|dEnXD~dy)Db>6=dP*l_>}$~ zUA6Hf&9;Rc5&|SuQL~UVv@pG#k(chf>Rq(+YRCvj+5GvtY|1?a9d(yABNT%5uQWIt z-?zN;+`ZN$%lvmAik}*2!8VY@t>jANx}6&;kl5Xd?4>^TawKB=_!A)+jFJas(N+h< z65Ur6dt3k^pI=b0%Jt>RXn(eXBmKF zVq;@HJC)Y{`s+NtZ_5Iz`x5K-4c;P30bsdM#WA9P7orP&dd&d4gqJ9&UJ&_HN%A>9J#{eOWbQ4Ug38tGa!X8@zg7n)j{IxPtTlLzft!3uW#wCun=hR3h8`m`Gn+2_0w{zdrUvlY z0>GK=1T&El+mQw0_P>gXvKZX~qYW&8E|h{XVoDm+V_`Cy3dwCt(3)){Zm%|ie~$l| zjf7deBKq!~P6zT>M!f2o9OLb0KK2>JMqyo<0oKv;mP+vT6%X;zYy51}q5Q4Fo-?5u zxR`THIam+dS$7Bx^@~@`1BhFw4z)Z9dL%^ztHYmUFtQ+UPYK;F118nlgR1yF-k7o6 zuL3BZcCiYZ&}QIs!35*{UI2{a^MS3Y?BrAlO@Gv55FiL{S}{mT&YGs@h>3|Y#pej( zv2sA>slPi8E(=d#1g&Z26efWZ7YKAT4kLf9ZVsR3`cJm4k{dKP=q}m%!*Q$6NVUc+ z^$mI`dZ#O5e@Pgh$Gtrmprf;0Z&Wl{y}d4{!PY!f0yPSXZfCAm#APXgLuP2#woP8FE~%_M$P7n}!Kov7ZSuA&cy208IwLnb@U|L3;LdeM+gwqEr550khZ-M( z+aQSoF6!?UAMEHNUbspM0(=WmMnnn5a|4(qMn*t4eVWj zY}5*(7lg&cw(k~euaa?swWP|jg@fTgA4etOR2E%k+J#VZKK411E!cG}{hau@S8joU zZ$~+n{lRg71`-FE=M{kE#Oy&;t@>x3PRwrM4+o&_IWR{rdo{DZ^8S1(jJ*GmHZUE~ zfb!lwHcDoJE#LConpKtq&JDnrck-vyDm5)_$>eKMM&Dz_v`rgI@R9K*r@*z=1<+O- z`W!t;a?C}!rNJjUJXN+ zI(w^u!-BN++|zK2?eV{yrfT40z>++fp84~6?4HilZ{uSyGeB7&>S6*BV^8AE2#%Jo z-#h6chjEV|+(~`JvVH+xDhD6`%q#7)`oj6CiOhScho1gw;N8J5A@N&^AiQ@L>u@t$ zGJali1WqpNAgDONEaeir6}a1B;{+oc|8#M^#)MAYcWR~Vt?LL_kfJj!Thb~d%xmJ; ztye=<7kdU*?Ih2PC+cd>VJ(iHxh-DpomeQN{@eI0b7_{bz|cC zt1uXxE`M5Lvl&%RqYe;H%r@{7lf*X#?(<vpY|BJ+I~mh9*!^%SRJ`{U94<^-8qopB zf6-K)Oi=o>E;R^?@&F5Z39Fl?_P8=V=Q^f1P&)LCx=Ovj;e1tJ^rQjcV13lLX!EV( zpU(=Hn>b-pDl2P-7oIzi12hTHTgl~DUKHerwyA0z zba=qqgij~?*j%t+VyiA==JUIKuz_Fw{J8?q@*SO>Yx43q@f!eM%~U*9HH*K(WAV*| z!Q-Vs6j-w)o_^u;k5kU<3z)@auoaMC?%Fjr{1H`TPI9At8zFtc<=gG;LJ$QUTOGCa zt=3z5hkLf+sa>~81L=ArKq1`g`x=fSgc;`5OV8NxTG*cZUSHGiKC^Eq z_*-cQ8k#Lqu*>k$y68ha+4{iy2PYUky+cp@H1nRqk=BF>VdOTbuun7L08o1N3=Y^3 zAW`cTnoSMRgnS`tAclgPn%V%3UcDig#SFEHniE32ujNj_oE~|E?^g?o0WpUcPdir* zCgTOGVCDMrlnERM_g!4#s?5LW61d_zQHWoOrn}WuY+y}*mheg@!s2NYmA7w zx!lq1Wa~C|jVZu}3=Vz?padp%*BZYzh|b-z6#bx}W_<3C9yiz;@V)@vJU-e*1|5KJ2DVbK#oYPqZfrGof$_VUC3O<4V6BYP&FVcmYrL%-3l@9Q zra)x0GvJ2rS2_eqLh$0d)_Z@M{Zq^jlp)I=xlu}yZQC(Ea{3cuyqZcG7E@2Lg>JF& z6snxugo+y63!HRq5TXLP!wB8>(IL4TPPG?hu8_XEUQdk)(K}TO!U_RQ0@&3Q=XX=5 z-biXMIX12xgE7V>)adQor4Kigm|WHDXR4n`_l-WTyl?GN&({10$P3=teH9DB2l-t? zmDS`WnU|#X_SxGyKJ?MU8I-V5ub!~|u*m^eDQai96==}b;hb>hD{rYvCdiYo{JJ!N z21^UitH)!(Cu!^B6$_F%qdj*)s7jlwhUw8i6mbC49CFDD2<^lh-Owz{Qtx>}DPgAX z+8LXNF^x2BKML>BBc`7lK^W&A-p_)6Gds*vwrWu7-s;FdHlkh4m z%_|Yk>hB$HFH?=be&ZuYocx45Z>*1%pq4fE>w{_8)J5JYk77Xuu|tERq5pLn=|JF! z&k?23wzqQhum=Yn`)FWfbs&1(-|C?!^Vf3|I^xf^A_%%5tetdln!4BXKN&b^bMtJ+T6Q+JyEhXe4Zhfe zr9mCFFNuY3ZjipWRy;m~We@n%4ofle)8turvlhE z0GxSz&^nFTzl0^xP`9Srhb&T@i={uxEsE-EvUn@><@2O9$1yYLv5*?si~jv>b*DGR z1ct<>4838?5wOQ+C(eNHxs>`kIm>CCvc4+2v5zX_Eq0tPTSMsO+1!zQiuDYd9M{S+ z(Th{3JOoLgPqGMoJ4mF3vBJrer0ODHcmZPGl(?ez6TJAR$Jvl0zzIB+R_^Lh1UwP1 zrValCZ;xwn>k~V!mRww!rBd znO~kNfkV~mRxe|!R)qzGP>s$+pd#xU$%nLzbJj9$7dkjtCS>v^=romKF?o}Dg>^se;uaYP*0jlPaM^1g8dW%7lbrE3UCvF6vTmFG_Ur_OJ}xcSpsndqR%=b}p6qA<<8vr@)Hp?c^ji_bR1uTx1wxze@C9(tm36&~zo^`=I3 z?surGN#}9M_`H#5O}Fw=y{?QsfFwkg^-IWfwU3WKS-jOTLGw{lK0Gs;{+9NWd|Q{* zK>(U7@6?5*UdvH>cyKPqrhw1DQk!m?;SSEuEPKjuqin|Icw;$tMeCnO$@e59KuvlTIesNAi?*-Vt6aP-_cQOu&KGMcl4`e6Brm{oQl@ znN=zIDbBr)$mC>NW;Qm-h?_u3lUvdfCT!O5E>#%)sjVzsymh7~ss$fKW&3t;;Gh5K znG(`S+xAK?DOcZ;u}$eze|XE*c=bKYHU)UwP?D16sSOfYMt=CTi~HEMx}HPEgoptS zj_cfLE3#F#&i!Jwgm&9^^J_>xHeIBfq)!vOr=pwZt90v6CKttK6A1}i+qgI{`%Tb} zd2;njp)KrtZa1xt5BA@jTi_?8cDCnM$(AFqRD3IHQL z^G0Uw9<4ivle76`9|MVK>47*9P4>(1E~oMvU$g2|>r8#{U?H&z8E~fy)QxGe^UeR9 zR(Z&=w7>i*;fTsZkYDN%VCJ*@X#M^aenBK z`J|Ixi_hteb4U_?O4A>a1*=(da}J&_VM@#Seoi-V>!vBznj9q4&j-akPmb$xB{K=8 z4mF#)S=|}S>YZK4$(Ve4G*FpRa{3Fs(RS-6f045vxZbQPk^y)jQU}xNJtYj`ER+rB z8QME5fLiLyx4n#wi~zkKe?mUc&?98X0}qqIjh&OnhU=Xcp%D3|GSA}7sy&3o|JWIiIFI8it@%5+2hGJs>!`E9!MfvV+-_i^XGtw<7 zHHdV#3Jf4hBPk&@G)l(+Ln9$bgMx}6C>=7CfV7IVq=dAz)O(Kme(t@0`+3*1)-0EQ zvEZ8Py3TJL$LA2GWn*&(^2l1`yb8Exwej&}Siyf}WaR%{0r&ghdaJ|lHIjbci5d(Q zXe9bGY>Hm!;Jd+M@4VPKBwFx1;z&Jfx7Nvn@cN9!HBqCU z;kHFMZDY-A3%_BL-1_*&PqQNpbqsLE#1un#6}{LH_kG5l?%8T$p6hwvhh1~9@jw)n znnAQ!<7n$h(>zxK3=*63-Kg_ku85V5rD`|nJF zLD>RuMq&4r0wN-;E@QqJesedN|0 z)O3x|h!QYbz^=J3?3mevVqg=mLY&5oSH6#`Rfm*ao6DxfU_Ea$!2Q5nptj}h$0_0O zK4t{J1s)AwkDdQ6z2+g@WOJ=Eb--~gEtO{7up_(a^YVj8`=hCc#L$Q1m!BJ1wNRv7 zV%(n~(5Vc2My>y|8~sW?2t2I!}y)3WP)6#8ld)CT->A`;s^dQYr#fNu4UgJ({fE|YB}-o#kPBEpcev~9E&r%aPv2~4UI8+IHU8KnH#C<6THBZ(Uc z`3$xFZ3y!C)B;}QI(u!djW*EIvAFFLm*kZX2~8Z48uFI+Wys;+xpeG} zgm!lhbibWvYjT!3J3EgDHFaGrru^qZ%i)kuO?>vuLytZtJ{)Q&CR7yaNtsGz$KE-! zvQJ+YpN2BGS3OvJ^R9<1YwaR$ka8yIqZi`0-h(R5zD~)RC3$t(a_H!bAJH#X7B)1mc4-a2DFFg(oi6puRVV!1ho#v}a3CQMHcG@cmLY%|seExh6j5jyA_1@FV zM|?!-@!<&Fo=139_V(9cRvsBA0{w@Kg%4wQPkx?A_2~_%5w>wP6>|nRhsAL2Udv(z zaoI+6wvir?ZZ&G6?*6m#ag?*Ii0u{Y1xon=H6?NPyhO|FP)O$gelSq1l-c4BH;7Z~ z$7L3JlLFsmZ10?k&sCFt*sx_dr1o4wQyQpcOI0#&b-&&hti#12g__L^9@A@M@U_`< z@t2;w>Xaz&CU=>Om3z*A9?GX5EBc9(V3_6_*PRW$T`_@o-Vq;Io^|@#WIP=tGAH*m zOFKTh@iXAzovOZic2l&x%q_L2GCYlD>qSjMbmUpjEyEr(n&%gC%4sSRrbE z^VoE=Zcju%y09>Gwxs2Gqds|hn*jXfucZd^USO}Ei!|=%qH_^RQ7?fy+u#Q6{Lb=v zjkW#G>4BvK&h42l^};+Nd5lQ4Q71|w_JHfwxA~i|xIf>XsPVjSzUVb8w%@=#P3#jx zbzV>MVAVA?2xe5)+=yZ-8OZKTc3%yl=ayN!F+*uYjT@{c+I!8;`@@jYVJ)|9@ul_N z3fs~FV-SNPyYJf{CUMrEYc74Zo@i6+a<4=$6kddzEhIc^EMk*X)4(OO>FRw~x(HJh zMbMZVdiZXv7*mK6HZm1QFRm9Bw`c#yjSvdCml|4;KPxXrty;=!!h8=g{n?YzxoW_S zsHqrg%^4?VJXY;AxdHBiiCTC{;@*adAvzR=z8-eyhP@Lo_`%9b8SaoPvtM!xL_gsj zi^^R66xWAlF?S>JxwTrt6WxPkf6O<_*d+U_SlWn5XZozi1mksHkKjewBz-Juvc~l& z?okc7YIQ`D@NLY5jmWp|lU9(>$Vlc%m*{HOw=d=N43_a;W3n3snH(_jMj+V zO-gx9A021TPUA#DuIDd5oJ9Qq;r^dD29KT&DEBU+%s`S2O`_F7GdBovbaB5S zll}g7_Y{T9)R$_L|Hs9kWj5?{zCBT$L*ps)YOb-qlJSF9ooyIj=?huh6(uo~n#Y`J zBtj~<)vb0%qqjR3F?s$f1GTZlzjtrieW*!Ys*pOXA+`=R;s-zf3AAR-nD4@clfxoXo z$Ltn#&&`VAK;i%U3F}s#bFf~ByOZ`+rmE~futHkc8kY{~;Kk!vpDm;<^?A+Xti`ov zyAobCIvA6*@|wrWdf~M~kb*ztNl2Uh<-KWhp=5+P&ns0uF=h={c8fat_VNGzf&8yC~6 z01>=5AV{tA(hfC2(yb7ay{KgA%o%zsG9~kO+e9zDMo#97OO^g=mTMo5a#d&46vTPd ze>gQwoXR(!H~r`Rp&JEUIY(67QApfpF?uspJc;Kp-RIBXZ8ZT=U26Z2CxyKWe_!uG z(s%bC5902B@*^gui?L|D<()cBA<5@!*mm$?yyd=UuKR!nT{tFhFshSfC$Ny8AC*C0 zcw90tCB#hcH4Pg`|Gc}qsvRgHqh;bP{iwI*zC*`u1j_E2&%7f{$EfZ7Ppzd-frY8v3|)K>k#wo3AsMdnQYw7xk}>zlm0n?eeD1D^X9gd zYUZ(CJH9^R2zj)O)V>#qR(X}J6BjF}Jua$0(AxKCv#(vP=2n2>Yj!JZ&a0&psW-vR z6?VxKMvkK-s&T#Ua>V@F#-Pf6_U zg|YTd7IRt}c|+vwGiccSY&=5_{S+0)qt(|^&)@#I9%+-cbVHANR=y~OE2^hqmm}qN zDc{SA<|AA05nGR(9oA&VgdqVnz3VpC;f2X;rt%P*>kx z%)AwePM5u%WSZ!_Y-_IAI_Ps01>cpTJv{sF;Zk$?!PcfMH-*9Ozf0h@qgq&z$_Yc^dtdFNN+oq1jnJ840sh-}j>SIM$wZCI1j3vI?g7efs zcAd+U!;;*$o3-zLHEx>p zwrvhgH#?<%{c!kW&HnVt=r>W9v(G)7+fx}%B5p&q2@|`tQ0hljDTP}N24a$%cMV!w zLP{nQ^PC&bSqc1qlr?#7*5pdlveLNB7q-zdd)KwppZ}JAm32VHd$9B9k5pht0tux_?~MOO zZ4h+q+SjjNCB(%aVhwWHwq;;;jI~rt*KTaMkajSe-HK_gm-FTQh2!T4fW<^yub#zW zuzt$rw&DLFeNrY|;his}8{auj^HB-T5=y&OYp~aqq`ES-(StH{#w=ZSF2(B%^iK}{_6~T%Ne=AO zFS|62ud9X5SMf73X4m!&oF97J2j=~6ZfzXOv^30iT~s^|CbFF0-Lr3+RSlOkrW#Ep zVz&G8Mb^H&rPz?%df}$p^S*7A?p_w)N0c%EZ_dl$!uba<`uu=!bOL zJI{kYv91{1&h*b2N<4HO0G6n#WkiFJ>d&F*Z6aoeZThX>j|(WnJ~K=>CFm1zIXuC4dtwKI!%H=MNe zHetgHwTX(S_ofm)<6a~G$F|9s92W$;cTwb`Y;k8x8>nI^W*utnCrObycH9~!aydgI zo`Aq^oItrs(pvi0Cc(qw82!9W-Lt-o-8?6Nu3MnrIQxhi46PMpCS>kmf4DzB{wfLP z(v=tik)d((CZBrz_M+9lm`(*}tK)?n?`rV~0=m`Q&)Lw>;ohqm0yRA{hktvyQ!Ghk; z2FT%AhIRnOL1lMOXLSOGpzTQh?&u>+a~HqzH;z5*srxS+`)WOYKo6P&iZ6b&bi9Z^;3&a4pin=61klupM6A-qswtL@+V1)~0>Cr=_l6V3i+o@aL$T=v<%WWh7ct-E{l!z+yc= zbnK%OVcFSw%C6q5sNI+axtYNVT-elp2Bjs)bGeuDQZEf1ZV|)-cWv{K@d$ z%z|;G8B*z#kAwo=*sAS``Ln1*#x>AF`FvIz zp{a}3l(CY`;z;kn(EGrR=1z|9p;|p}%V8YZPHPDB1IbV}y@ne^N2tPa7j<>#=BR(% z4sQu@YE^?S%|Ebw;$&(}1>x4zHmB@&hJE@E-@)(AQ|v?3%g$0-qq?F8ZweyBwl07WBlNWS>NycRf(N5%u!R%-8_|2De11d z`-p0ST3)!xdioK#QAmxH{4VPQa+JUEuS)iXuBg;=uU@e-dud$jt%5?y9k-JrHtLFk zzMdi1w8owY#m;~6PrA{}X5=B?Bn@s5pRbQ$W(uOGMNrpVlDS)ay|G?G`4THUjX~y+ z`01f_WO86EX4CeW4DJ@O6qo&6$+*%kVNfpL`|0qW)F{%XnV+wzJiZ9xo!w{~a;JMN z;6YKX^B=E|uv4IkU%&%Y3S2IruI^u&kuR z6~)U5eX=&*SVB;24RGgjBoaYo4-r7}go^0yQ0L5(g;Wx)_?6tAd zp_=0g0F25%QM}~@si)oo!`yA8v@+a$rgQwlnuW|9V`ijJ-{NOM!V19ITPfwjoGKxL#~6J1P|CF zuHg#3yZN7!A~h>-nun;*n|jSZ%bj zvzplp$lIJEEl&Dy)tfh<^w(R+mB(Z!o)V{SLh6kRyH2es8YSx$ZBxJcqUOKQO`6x= z?1Cx)H1yKT&n^KTW*N?6J~W#pjFs;?kOJ`ux>Mi3+zTkW*#&`0*B5-&pgOpd;2T4| zLQVW&bl8C$se98eahzB!eF|@O=6F+x^7O3zP>{DoZ81E_4PvkF_^$!w|NM)2LoiQKKQ)x;PB8?S2rmk zKAv731jj%YqjS5aA+a=Xu`_O6UEtVsUwc7JPWqv8SdYN|ba6J}ROSwF7oc=R{wf13 zY37y`jMVhK3$((8QYVLH(Cv|4x2z7nUD;w&QEo+ML3Zo;_%prO0J1uw!eF6Yi1TKr zq0LXlj+dn=9v)YPo>N#l)(n+R?w$%xa?N-%(CE6~sVkKznKJU3@g*FMl0a1F>t0*5 zJ}M!bitd&K?57~@#02B2y?#=eW$kaZ=8LweH$MU9F61x27 zp`XQ1M1U*9d_r?Rj8o0PAhite+N7((>zb2rCcITD6QJrbU==GMAkbvtasYB^e%|1N zWuCuM$aCHf;?jQG;#DG#^-F|}a$>%#s4g13a&rE~SQV!eLk`TQNI_rZ(14-jfB zeEj^FLLA~sdwXxc#m(`+1qxvOS5~)tZEO}4?*|4pS79gMU&C!srgG&CLzk+v)(|Kq z%WH|hx{vj!P&$kp$uqUfp0qyqRTZ`g;cC}km`H^g8kSf3#8+KVo!eJODqXtSU1T7f z_#(GF?-r$uQ8{!u-i8hIUhJg1tE+h|XpB{|lx(?MuuTAH~m&F~7M1H%IpdZ6+LFLu+`*=Fow=HZTfw;zM9}81|D)N_P9Vh*SyRFhWTw)3GKAD4|HxRx^n;2`5vLY>FSG0V%~i85ZqZ z5e#z6&pc|u%WCIdV{lJ>W#04Wp6W;Vaho|&(Ib=$>V-u$HKUVLQvkscm=jJYQaSMh zZ31g(XfV{%vjQOmTMG8~;L+i#boCSw?w87PfL`Ilpp+~ zrYM{3%THWC{O|#ckS{tJwafp8JP;2r(Ktp1;uvv13bW-v9XK%RAW$bM;FC64@gG$I z<_S3Z!i5ZS;Q;hFEp2ThVp^nN0AwJvJTK=?Yq8)PQ}_HJ)+F~xv}~^2gR`&PZAE-- z%(1}tIJW#6c!JQd{uc$=?WDbc2df{hUIl8rhz)2${V$)%`J^myObp7l8*g8mI?9l-6!bix(+q4(A zv~=U&drI*f7LkAd@xJ;Luh8h7%+fyWil{C(#zI05VBb%MYr-kwa*FKwJqqlGzY$2U417`)Nx@oD_a>xP#T&l$7h zi}1^;szx5_RT8Ap#cx8Wnhz%BMwHvMV}ZRePB*45@)-ZTugdqbrG8uf8Qf!bbzcQr zvlu_a=c#-e;_os^*j@6LdMkzphC&N#?;jOxQi;eJdz*h+u(>87Lq-8<Ls{0w;I~Dm_(kuyz}TJ}G?+ z0-9@(kDQ&Ggs^N6nIrre7+I@5NGa+F4<|H}(l5j+BUkbQ1`oGp)|p{@Q?dggl;_PK z!A8n>cEw2ZZ}D(+CzJxM6xCUu+YuRj6}LzhXr9NP&_I`kxfw6xRWdCmbJ(l+McP39 zXi6KLr2I+k=5K1lCpMRE-Cc{TYV?+}!Wkc%Cm&jldYkyoD&Amj9@DfAF__oxQQG7O z4N0GY5L5Xp*3kkb5B#!$`h^WfbCzpVD1Q$lV&x)O+t|+uNG-o)+x5AeVeobR`x7?G z10JL2MHCYuf+wQR7Rjwt^z0p`k>{MM`1RIg{me8l<=eq-Ik7w8E z&3_TY2kR35Qo@&^b4O&}0OVil$0L8wFpjwa<4wMvf95Br7<*n`7+o(?nPMD?MPqUNnX0C&GDBM&fU9n$qY+A zr!T9JT1)*kxo-qXq@b&3{`VqQ=5Bhz2hG<#Hs8~;_iV%^Jic8ktvK@2V##`xA>iQA ze^pITP&Ezhuw;{FYOrfZ%2;y_>mznS+i|F|C)MJ;i&KyXsI>`f6txjC7l)Y4?&=-3ASjs_(De7u1 zPX-F9OtX8o%Ni=?FVZo&dK7*tT{&VL(h}_f{$1|9hm7xKf3@YgmX0fwUB&(E8^a|V zd)NGjI;Ze28jgbhE-V4uVi)U-$g#EQuL=9Bq~iGk9iJSvoVDcEetXEfH`3gS zCF!sM21G6rV@8VtWObX}mBB#-6C0afI9`MSeNlT6UeT2kYO>~%TWt%AE{mO;=8fLH z01@VYnsL+u@U|7J1^&Zr=^@?@{uJa}*HBPI=!avGUxKmcC2PZ0Tk3XpFPufLGBW3Q8^smSw>M!MJY3sVi zyaGl>MrfhX#b8Pj5|yEiF1(J%Ie|h+xu-*L&n?NgB#+_V_F7ZH+UW(~-tZ%Lj4zuy z6Bz`$rzrg|zS5a`DD>zW^s5BEW@lGPaMSeoN+hK2QHT|8$a?tnuTnmIjc#jCIo8z0 z9E=0I7s9ieyF)rhJ2BTu-km!hH_x=2B7K@m zjG0^4Y-o&|RnMFq5ns9Iw1$G?Siiz8uaZ#Z>+QTySK2Q;F6Q_Tb1^$^nGP@|vh?O# zbe;i{Xnn>UX9+L6wjAnnH*Sfy6wg_0CGc-O3c0~Ha0+HEj*gC%fW2M+mRN@A$&)8; zx-MQcxv68`GRXE^Vf0BC+TZ;~Uh1kKB5%)<3h}G1x|)8E_v6`bE!n{1N-t|T%f*$u z3;ZsiHN1E%RLWPda}LUtnI$lhlGY#UeT>+5pRDc$jwtg_>Hwf5n}^ySS`qt}J;5;T zPI4p2QB&P|5fKr{=l|lxi@&(EyiYc+Z={@*g#(k;$KFjPFiGaDsxE`5nQmh{ddiB| z;}1MMs?yWa>VsbDRlS-E+kB!zhT?LY@~R*HsWK7~flHEK=NUNbD*59cF<1N3ZYeohaK;K{zaP01V}s8^ zQ{f;O{;s$k4}tF9CEhI^uWY`u33s31eKxtlC!ANIoJp_z>WJ?%)0$&yb(!GkVOmL} z?>%8&JO+j&Pd#SF+)UAB8@P8Mah=mHbq1lNrZyL?CjPZbrkDTyZ57aMTCQbEJ$XGN zGxwhL(~(iIQBe3}S1(|?8>Qh%6Rn!eA5HfU!D0b|4$}W>Tu)Z=o6zS#{wb-i!x6rj zfwiCkUJu|QEMEtZ&+_FJwFa}cp{qx6!=;Xx^iyv_W*P6E;PZ7$Gk~)=KU-1sOHWPx z&{MXxDI7}y!7$ zB6+pR=KqAXcUJtq7eKRs&jR&6zXM`ysu=T!(t~xumj(g}JlqHo_5o#o8W!@QOXe0s z8Cnm5@x{3OTgTiSU*A8>i77=Yt>8~dv&j1P>-=s#UWog&;LGZbJoSbQ&*%ZT6>ubM z$*O+ z0E>mx=9?Vu#dr#(v;Ri6J~oGu@F75VYGUH_I$fE$sGwamU-slTy-AHvjiFb~Cy;#3 zv|71?Pjq;g_sO@HM?j#jL6c9)BU_NQ`wk@pDxhOLg=Ol5bti?6eNKK%$m`s+xnJXk zKUSDa6-koVS&!6;GT>Ac`c9%Jjno>x4gNGSuJA(B{>!GLW0rZv!#~oB`r<#()jCYXO}!T+L_iGbJLRi_Gjhdp&(LwRHXvj1C@MhQHc z&>B~GsfCKypIq5`!Nc|B&%w1PQb)+a^u2%_#XzvM*_cxX9c1*zVht7{vNDt}iRkpI zk>`DDl4Dm&8oGcpLwLzoQ9((mT{O%63#N?~E$}`Z$ z^{EuOg4ug``p?#gM5WvhKal6g^RMC#A{z~AU>vK21r zxbjFTj{ymF${Wr1pTekCO$(lPr=JKm;)DA=;DMAR?^3|&6IXA$Z<~@w=K!vWkY0*B z!(MCQDRzB&e~`msKBD{{iF_XhJ(pwEeX(Os5-5CWtNeW_aH)qHL}e^(lEvSZ;@~5L zXNPaAauneRWgeT85ck*51y>%>OsS1HEugPXOYr_#>nxD@yf74))BVw5h}Gza^b6TV zAQt}g-NMC7{eJqwK;6d!p@8#)v0Q;qUhc>#0Z}l-!N2cnfE|q(Mui?pGS8r09@&5A zDRo~+aFbrYx%t)mMrFol>xH}lcBNm5?()?XH9PnTJj9RV-)J5Cd_WvpU)wc`knFo4Jef}iQ`GF0n+&tA4F{CTIT zM}(XP=1b4V3%*%rTIec_dNpq+ z2S=HSwb=)!PP6+;Tssj8A5kAbLAh-Im#CS#dvsk}4_43@+Cbqg&_kvmqt`$tQx%!EQef+fP3$xI5n*D zr@+8D5`c9fQar~;DM>#cc20gU6WcUd67E!wGBJmlG07jdqtZ)19Lw%x{?XuhOG&3! z;~L>hQeHM_xn4!jY(!R{Zr3$s!1Q&~X}`n5*)6#%$@u6$`Qb5`&w$+Ik=EXMb*ay0 zcL>;9`dc?6&1@t)A+_NHh{SOC?wo~*;T9k3MFzLFTwqDkBJJ2ac$k1g-BY&IDc;KP z%k7ZMCbPn!f`_y$(Ei&yJT+|HsUMK}aJnlsUFTvuR2QlT3W@8j7wfGX2GwsG*+9fD z(F{v}iT}ZTRCXGXoQIM!&BSHw; zeqtGiW33mX4=v8XZ&7v*GAVo)h=v}!xNO{w-d9QjyI%IdCx2ala(8iI-1a;2)+7k@ zdTvJrpWzFzw|*4(g1dm+Km}$>@&4x*7D9p;1#>xM(Z!8ObTJh-V8<)Rmrfxg(wBM0 zvGRaX7U@1;tLgc2*6b$UQU_ODAM*Jh&7m?V_6?YwFACpIP&%2QE2C2Fo&eWa3!@gbnY+YaTcVp>ou#5x||4m)gnjBot8%A+&} zGQscfU}-}S*@mxT7CMhip**yw7(HZzr8P(dMVxuv$Q8z~tuKm*93+ei2@ra{`oO9J zKbMTdUywfOaU!IVJF~3q&+%f?CN_e_%RLYzy-Uz`l=92;yHktf;aJmz6SFY20wsp| zZcskRpr2|9sDo0iMp*^C1&+~Y_uCfjJOZ$RCmS_m54=wAq=cP*ZVO^yW%WH(DU3R8 z|BiSdz}qNEzcVl$7r+vh{?7dT&rl6gcA6+w-h%2@@Ijg7#FTsMX5vbD-4D<3JA9LR zsI~*q)~ny!U04174g$i`xs{^v)zM*9{Lb45 za)hzPX~Ae6C6PG~Qs;$oL664i8|JisE&yXNU-0WOYkVlQK<;-9aE`kTHr&6P$)}SA zoh~meE`fX^K~v6wpE1HW9Yg!KawWzV8S6^`dNEd9Nk|U z@c9z2qUgt{1srt>_Z>xH!CLR%IaDxJ(6CIOx=hs$)Pxk3g@;YLv~{_DC1~sZ5xA%0 zN70baQOHqQPhe`l8|W3nR}F&410voPdM!4raVP3JOAvgRokVymY3sv zJ1re{%(RUDy?69nMT6Xwag^PC{t>BLHql$0R^E1xl&4hIOkU7U{%Y~bxKTuW=mQs~ zy}CNu;1c?dcgEg2WJ$VHV119`OuYsZrf$?4zqo!wuF}y;h&k!-pUyMGxGOVbn)50O ziH|zMOM_*2D#g;dUmt=(bKkgbHJaj~BKcseM`DtfOMi~M|4xA!G8oJjCA&W%r1}dE zXa!Eu15aeWgVWxhH@0l$^0f7G3Ife}VGr1erca(Udg1sFnZ?zC17lbpBS=gu#=3f| zKnVl67*5t($tYOG+ztWeV#9v#0<*Aw*lKPa zvyqT+Vj;Vm0Ekr#g|6Dn2aJffSvhYBeBcC*{3UaGvYHp+yM|{oOKietxsHK zd+x1N3&f0oCFC#kG#gm`BGCsY+h{OGK{$r&v=Jdp7GK=#BSkFR9hu4u4J=;LW+9m{-q$H#)ow1XwM8WyCWKyjfew`IvI~}isS$m??^ynZCqRuiW8)x7}8e` z9;>kM;vD$V62Pr3i2rgHyCa#Sh8Bb>mEs$nh*dXGbeW@~!F;fQteVFW)Y2G3*Ts8%6J}3{p{Xpf`OF}R8)u)9@Z7vWJeHia`0fF2-+t*{lhxAF?{aIoa z<23PI=Vp)emaJPqpGd=#f&23BMt_c)s(?>aL419Zf~9rRiFc=~^|kZ3#@thSwo*Ld z;fs-}tHGAzHKGGGUe_b)lM}ydlXc}O9?$wO3k8B2m7>^<-PtQ6FNgF z*M4-}#tX;2N#%i%mf-&+y@4_EjwIL!nlGtY7rk`o{FA)+sfxU)Q#b}tNtRxM{ujKq zF9H^TDd_jlK161LK`3;N?2$e34Kt{Rdg}_UmF11bE6x4kvbn#3X{x(~{2Qq7$?~?I zd#O}jO`-GFA#@q)G*p9$^u&aBoN+c`XQfvycwsa>{Q-l$sl4u_P{qrWp-l(HQ&3K~ zLN#?WL3ZikEk7s^$c_=+1(5FJyhuo%LoE1(rr&|Rw^~Pz*b<=|lYX2ukJRm-qWkqd zq8_EXp<91j*>qkdWgEr?8?C1wd^xqyx5=}=A1*A|#@^7Bm7 z{MQ-+bZzi_kWL?n?lKVa)+S4AT1S^t@%R?S&VgKFTbnk{*C$iPNs0otHu8C+@7Pk^ zLAVw}5niYXvH8f*StI~tj48w*uEdh(WUi>4+PFq97cT-0SOddG^W@?K6yzdKrxZUf zb(L)1?lFSK-6j`7Y_D6n8fhgUxGX>X8nd9LXaV&$@rUjtWy5!yX3qyvmWpH`5GISS zzml(L=McdFZEk({Zb$(z%ASd~xsZbL=&+UlNeDQ%6ln^S;1c3^yH=kV*%{%w3!Ffk zO~p4x8Fy11zt)vR8R>O%@@7|oKN~^ZdNwMQF=f&mXT6P@v{qT#E=)lpymy{q`w-XiP-EN=pF^cVWdECxkK zWZ_{h&1SXbd|kdUUe(N)4`<&T>+7zqw7Se(3R>IDxAK_piU5{#mmha zO%6AYVIm%}PnMW^xZB`OJ1o@pxv#HDC|EGiy;kK<>*c8;c=FFJR}|(R7mQcA&nk33 zOZxh=eNR(AQm6rV=z4X7Z2;@rSva^|VQ_6hRIN~sxa88x68+ebtGyE9raznGjiop0KYWIxCPPf{&=B|p1)b;ky zUNkOj%v8D=FHI`F44)^cY^;XWkBGvz-i}!oiMS}bj1?9t(=xEyh=AVHx`_BYI5AT? z*Z(QX!A`*x!h!)p-ccOzeoy@50ruZeFn$upqeu#x2ocP93ir@s0HboL^(6-|eeu47 zokEtMIh?Z75`DO8Oy*}Va1D632jwfjo%~!n?h|QsY4Y2C@XIM_sVbd@$9&dB`@L7u zxdHwygG?z9;DwEDxXOEyHKfeZ1`RltvcUD$gP;8Y+2~*$K36ZybExNk+-G@VgDntd z>=dS)t7>S?*yK-^QY&&{Q%KR{=O$3W_aS=&lA8x(mX=RNj!ts@epKlyoOEt(C$h)_ zNl;kPt5=S_;C;*fSkRC(Use_n-Z$S5|5-C?{gd8uemI372HkXWTJYZ&$LTc;SBx1I z^XfhN&@yYsK$7;3_ymBlfujaX+8E88kgO6`h2uC!PzBbZdXlUd&?R`FjO;y>I#M%MRSSQdO84FTUQ-XX)T zqR@9>Q@zWjUG@D<2U_#%nmpZ|V_9#z1Av$e$*pL#{2}*i1X-=z!rY4!?DV7V{iSm4 z5&S{Z-S*D2AluNJ5ic0{251p5F1aHdR5Tzt1GbG%l9pv@j`UtVlij334dy{CWQ zg4{nkngXw3&C?FHi#bn*dZ9-6-&q4F%`wigP#4~=y<66iQ{Z81Z9#fn?w_Vy(~NJE z@?OnM>kW-gcL{?6nqx{L^1fA#XfZ-g%UVb zW#knYA~s&Ns}JYc_6-jD#tk&|nA>jBfaIYz3DvV5PwvYIidd+*M{5EDG*iuTv+6EL zw+kMxi1{ha1Mbb`_TrT$%2lLQ9_|OeV>uFXFwCqAIe8XR#dfh1Jt&l%mbMgLE^j!^ z!@D!FwTkUKq4KN7)^Kq%ArLO>^p$5!Bi4;977Am;R)1TUSjy_`*|Ma&tc9E;;pjbk z<;rtdE;b&Rjz~;&6T{%P9=nEin>bKnOc7gS#IK8SPkPSTStXk@ zOjEriu3cV)$^Dv)-uN1~>AP^JR0r{uK+@T%SHG4?rNfPwK8uR+28RNu-Ga)y8w5-P zG%eOkE0e7RLPs0vr>4S(H_Bgqt0$h#lVqv?4-E-+Bk%>B6gi=M9yAq@TF+mDltzbx zE4u;@@!KHN|JUnM^T%6wi2bC5go)MFS5V4hxkjc30h{8_@RUW_B_+o|o{4+04Iy4v z1PI)#v-rpZWU7B>Eh>e5;o|c0o1Jk?l5XRcmw$}L4$)!YFXxlc+p!Wxt$ZrrQ=%h` z6Q-0A#X`D44#&}E!0c&WdU|?8rB_~4I?xCi#=fluP?|Xh+&rx!QU(g_IEs8h8EOa> zpN~7;Sxo~ltHDkkDO%hiRL|Ubycr2aAr*NMr;2&mOCf^WwJesuC(6VjuFq(SJHp0Z zvl-5cx^ySgwUI{P)Wf5Og|8ECW__j(wFoOp)=xatC3}6k;TaudG849~__(!&%NtyVK-Q&&dW%>P{kR^pDr?=~Ha&dlGm2`*%?~>AO zoqy31VTg0XlVMmTo4W|;ZZmU>R>6&H6|;?io}Ntv=M${Mv&kNX+9(#@sCyHa>V1~&(&(5Cl%@R=Y2rr6S`BCtdTSDS{QITOqc~m51 zJN300wvYFRCMnuHD?MBw=1Pm1iivps>k zViYqsG5Hf8`4dkli5p9hmE>I_N8ZxI>hlT9dDN#Qh?u>Dmcf*fAF>O-`QG0x7NS%6 zuU77rd3FHY$@zk&K7F=jfwxi0@K*EAlg;51 z6Y<{6jKkaxK$N^&o>S20v+`hDyE#`b*_@%?Z|M)Pauy>D$p8?+bIdPOIs4pbnt~wg zQ;7yCDfx%AG}y|;PB_M4UM3TB7#$V8@#NfR?BY#h<0Q62?~KHj^Tyu7{zA`tl6`)C z<1dbgI8~_%?b0T)*U&52ISfOfCnrsfrVrBvjLc6#st@DBIzE`q<4ZdWiuNO46zxI59bKDNt>3S=FT4~r2=dp~!_5Q07@BjZEBfLDgK2bM*#bcR~&rXEp zj*`l0_!c;7(bI5-oHqUR0g0YASjn2x1xbRPE{S4urapz+j4C-kEu@ z=QyI~(LNOoXa-|jXii)d?RU`RZxSgUhk4oVF#C5cE{PiK9M45iPD`vE&Yddk^@~;T zB_ok+Cl`Y6i|UnE2&V2-_1M0Q21Yubx0?V4ag4g6j&{;M)K!8>p+wgDz3P7`*vV5- z5b6o1Gr||pT9QldrdIFQoT|1-YK7{bgl+1!G%j(uC-xKs@1!# z&oxE@Ubw;(&GsKxt9DxUoBnySgtkJ+3dDt`q=Hb+c^ZN}MPfPaY6Llx-cO#{e`<1( zv!6L}0$>a9EG59_b#^|Wl8sypI(^m6orV^o^CG}`J}+&;#>TdtgX6-nbk%}5=bJw) z!UlrErM0+17QXzVRg!*HCnYRN->!fUo9Ajy)ql6(TKH*hR@NGR3?K7X(Q4B_3vnVj z>~l2|zo*IUFugvp1m4%ZeSKwE16g@b_yhH>%uDq4lIAU6RM*yCq(ZT!*`QDR(Q>f= z5U-cpvv53h0ypQr453Tpj-PWyzds%VWqnhhj%Pd+aT)Lo z2ROpor~h}W*hMU59Zp)MM^42f6EITo9m?|?r0I4wY{&pNTKc~aqwpepBHnswlfm8~ zuN0KTZ`J{N7U4uM=D;|mD;-t=W-V(WOABB1mv;Y*rXrMFgiKJNJM-?NT7T~nfyrTX^*SH zz4`v3Gbd)mVE~!`h5wlgj(RkHT#^@ZwnakE#i48Q<(#s9WS6b!8i%#bONjF+7<{c~ zUAZ*XRiV$!%u}vBcy(WVf8u9}f^ERdyf6q;G`j~Wg5)BFuNPgHK!pqTw;XrW%y{rg zNN~cwYt!~*WI_qe2O|T>RB+r@E)z_8QaL6K9NEMbq~plYhNrO2=Z>%KGrDeW=@zkr zF|;H{-f+u^;>~$;CWs?<*ZD*%e@9+a8E(jD+z?*6k70k?rGE$zrNP{DHKRbaxzLL^ zu4p+D5<_G58_J>GDQnG1Iwl*;=m`MgSU1?*uMouZqkZ=N-)E2Oe&-eTn9mPE+}%BM z>++=LbAu6%UYs9+LnWN6sf>xKC9^?ubC@T?tWpZ20J0JOQ$)_CyLlQXXfHx9a= zE_+=_K&5`JV|39|AJRIuv{l7FAYs}d9br51&Y$3~7 zVhoafi6La)DU_W-(SjknM6&No*2WUDq!7sxB1@<&*?!M-zMtRcoX+RE&ULQ(ulKy4 z_x;@W>wYZ*+Zrz2g!q`bhuFAS>KMlF8L&?+8<*LndEMeeE_!pB%%eA>X(@CMvb{qm z1&z0!h5)}2Y$q0d3nRZ71J*u5<5Ioe8XE^*9@7fUq^If{EGf^t%=#(3r2>*-vdE@> z6IQk;Hd?>GV|Sb5o&r)gWy=+oN-F8wC5;u+vH}Pm1_-Hpx#K*Fz}spAZ+SX0VG*MA z^<6*=X~pR;?v>`fA%`~X&j-)?wp|={8=H)Bp|cs|uXluId20aWP9T<2MT;%sLGipW zi0K0|z}U4j7}KNnmDK+t&H#SmA!!kuiNHBe`2mBQSEHZKf8~I)16$worMG%{K!$@k zF;H%0^EwPn?fsg7A|{D(I!{P_G?6Z_!$J1xOrU#ZM#8NZn<9=I4j3nIudGhLE2Q|< zK_&K_B(C(;FN*Ow6EhyxeF#jCghROk++4pRA0Bz4RskEwZ-2Qw3iMlj2sy;AqP!cI z&~**{@SvqM>lcEQ@C;Tf&z|rPP#X_zPKArO9_Py1R1==V}tcmsOW}qQ+zY^ml*2 z0(tbTRuDAN%p_KskSUn4*{>3|(2=oAQ6)6ukL8AK-ya@^&?yT*?C)J4tEPMkYv!+S8RUg)~)x(k{{ZV=_~%Y1J4>7AM^5u$)rK1eV~?s zIPm?(4bPrUyxnkDHMveY)M2<;9{`eUmA_Tm1NT>Z@LQ@kn%b0|0}dhrw0+Kc$|3OQA7n5%iXPX{6pvhb*LyP~ zbDT7K0f0;$9mJ>*^>FEmH=}5u%jG)smZUBcsDcyRU%m#FZWZK($TBbpjX#We^ihlw zF41~Wxdu9tlbFM={y>#|a6~b*3H}iusZJifxj|KM^x%r~I%@?Yh@v&df}+;_D|cqw z^n6HLdZ^TuRL7KSP@7!p*^&J6b0fU%lo{R8s z&*t?&lAS*Gc1MXCs#G9YLVNEgvvJ>i?&0}Q+U#EkA)tf6WpFn4KVAT)F>vW6wr*eg zNZdI}xG>R}*Y5M}9$i&nWH$he>?jt`9n7A-TW(N*Y3M&G(UY1B$$(pth1NA@!Ug-y zt--2H#xsmW&DB+$gV~rDy;@1c1GYP;Bz8X==pXY z%$rowzlu3ao{YEPY({*vP%1&=BpMxlGO#6eP32_NIWxJ)dZ}P&RaaPO5(@+O(u}pG zy7O~gfO5YU4ss9&Mj)uZX9Z_YFu}*p*iJQGhsw zZ1XuHNC(nkWRRg@DyF8X3-gO6W=<0i9TIXzTs{L@wL{jOU(a654vSm#PA*P#MTVic~Y zx|Xf?)Xv4re3_YPasKbw2EX=yzWW(Iar?R(+L0xd5Sg410&(`%Co(cJi&4VQm9e$Mo$0#Ppz$I;G?<+_!T@_>Cv`xxA)6y+H@4xfrb&NH2K{Gf!e)( z(&47K;yU<-u^q(e9x%0@I=2&~&>#@-tT`XTemSIKuzF{M}QYmpjdvk+&T+PBn7>t4Fw;H!5vTju1x5A%1BcZvcILRZI z@_H$bgBh0{6BxdpaY;&tijUWM*P9eNE!C()MZ|@pY=wBW`7ZKyWES<5J4og4pG|a| z3gYqQ`Nz2}*8)J&hS}$Xdm)m;@y;}qa6vkLsYS>~7z4t~;4zY@(*D6~*t(BUg9oSB4_Z&skA24Q@&!y@de&IQ#^g|#)teMNp1x)FIU z&>$*}6~w5MEd4LKf#6f6I5G{ez+*;sGt?easQK;HzrkcJIsImFUq>!qK> z;Og%CoM)Y}A`#Thir$OyL@U8W+wMKv6aD1%RM9VfTO?nj=8 zM|T(|F%@`ZyPnQ$0b6|4vPYfzq+Zyy=b=IOh=pkUZoQuyH^U;|} zUUR+WI{*J=u7u^0g0;K^#dDv|m1J~oI8(v_X?lHUXlQ5>&Q;n;=#SUY)n(6s5wlm) z(%8^x{cJ*+A9|PBr+1@3AaOT$7L0SDlBn;II->vx6xJDZGt+3Fi(V!7EKDxkZ4dTg zm^nTh*WR?C%6Vw($7u8RV#%q|2#G0|xC<8*oFCBhGTrNs%E*6S+jOCm(()OqzAwqd z^;&-#KC=NgvtpVOHyq+ZT}3k>{(wT6k8FRy&w#SoYVCaf&Oc!0&>A)1oOi1_zS$!W z3{q(`@19BOqin)*BfQbTdLvBuG7Sa%NM;hin_p*d%iNs}^H%s+@A*oWdv0!ys0**G zsuBl4*6t-zDq2c9RaMneVx$(pPFTWp>eK+g-t|LeXRYg`eSUua(eC3V;S?R6K#-r~g5 z`DQNk_^gLJR-?2k-`t`Yh@_)u!_wQdzrcK5{D|zIks%uz#SKv0mw}2@6b>a?+a7f~ z)ufsEDoy--TmXL#-RA=v4Hea_7q=~hF1A}L#;%;9gy(~$6KEv`(D-wR^%eeYlhz)f zXLffl)3+{s@uKSV<%6O3Ag}{203;wFD)|mofd>ZIR>>ps;P1su(rzJj=c%}viIX&qWRF|@_m**yMQH<{D$_;Mh( zwT&w%EELw((yHw3)fzhz;pcxPD~81$+{iN{&)Q+U3;{#^eG_*u9Bi=AXTx)A-1{ef zsn15R0Q5Y(C$8;KZ1ASi4&lPb7zVMZ8L1_;X9OMyb*)=0jyhRi;1%2(#cq1ws{ zv_LTdt?~PNHX3nN$O&T!yOc{=l73d|hdH2!UG(dxfSY<6g?&_@gg^N%faZ`7(CZ+` z9UOp}4~k~eWw#{Cs|F~5|HZ1=h2bkeW~Qbb7ETf=WHiL8ACj!waO>89Ay_V`kdjiB z{Tz`yuvki5e*|m}_8|o+KrUH)g~Ue%;?mtbdZN#hnLt5iO1>pXA`6$7Jiro7`WJqYc-v>;5iw8a31QJe~+y7gvUMVj(PhI}(cj~BJEQ-$V9D^OfOrOk%RdF_0~LP=|5t?n5Ku2_uP1;&e4_1sW)PO(9MeXO5)o5Z+adnu(j*6gy?QHG zWVmT9f%bW@uEDAh{^-0Ej)S7s5gYkNmRlI5_Y~SOJy@<+52JVFWaM&+bez2g@#GT~ z>8*erVdLNoMSsjso5E*}?mXSDZ zonsI55Kb{3B*cK6_Q&qM+97~>0w`P{-isJ3)BJPPJUJu7)yBrgsj9W>SZy;&Qdwk< z32{|e9j&X|l#KDw>gP**1n>2Si@fxHpi0t6{wUYJue-vmbLV!7^}k96|<%Ja8CHZ#gntV9QScO<#bR6-eg#G zw(~Us4hENB0+!h3J6Io!KYrslB4?S^jlxLWOx99kfIwGTwZ*H&VBa-iFNdIf2r{(~ zF3kT%H&Y}}Du;2qP|_29qwkIM^u#zXT#%JrUjK5l^@-Ah$B*r(J24o{c6R|vqrn9n z>*RY{x*&nX2WV~Pw}_29 z&jOTl!g$AKd(y1AB45IscTeS5{^J@Xrh1TI@Q&+sqJxN9u{Q5sMHd2xvFGedPfEQ_J1H|UeV&G^2FML8L<~Dox;P`4@dHE0!YG6Sts;!`t zu$KrTOt>?BI4hyMi6C`I9ITi>p2uE)?>pJh$i=R##O@93`(Ld#l$dV6-4^Que(X0? za>Gww7Lc?1TOlW2SxvBPAViqF47It=T*Yu^_Y__`eiohMWKoIS@`dTQxNc{y1K%#oPLbr;6E1%Ue~78Ec69S$^L~gC81wUnuOaRvDR^Hk>H9^YaoY4R?+qOS@`q+z_@V^;HHl znT(Qe727TPD@x`Cnh~ZccHQQ(DrKxe%@ilcuOMfSPUPnOJy3^udx!4x07Z2!N(s+W zEq5RvBj3sq1KV3hdY}bZBXQVyo!$R`7(D?qFc657;_Dtfcu=~vwN+eP+`skxyEw?m zGX$1=l>@g`RKoI~Ju?Ho7?qi2LVMBRL>{L*b3TXerk*gD3HEE4!dR_U-U+(Nr6dS8 zPE028b&QgD@h-~Q^q4rx@8=Pmf37Z|Vpn*0(9{#}WxG+F9PmTOjFkx71t@$ES9@yA z^taG(p8V!L#U;Sn>v_PxJvn_eO3b+eEixu?{N=5xirBa~alWt)6l6kAOc!sG_r?TPJw=H3eKS)oJ0Hq2Ud3T8?s} zS0k}g2i@wM<*(t@zrhgVGHrFHJ=_nhP50YR-W{gH)AH(vdu<7RAhq(ZO=x_CK_BAz*v z6y*#m%1pXkzg9E2*?dE-0k{2ifMZ|%MlD=`k4Nj{>svXG`2Ha_CO*=}X`9?yaUS$L zrF4%D2nuH*gcg_>m}3A|%iV4T8i3TTpD1A3U(Ra^=d( z=F6{mU^N_pGbAai1MN?}P6!FfzIZ#wtkJXw>A=u<^-tB1CGOP9f%eVGFv8II<9oUT zCW>Mmq)=ng^cO)vC#__J-KgU+4UOaD?pan&RaNgTO>$s%Ifa!kQ9@cT%f+e0bEHh5 z6Dp*xCNBlwb-1qlrGfZcHcwNQ1h@Ly}yMp(i z8d$)a+&zv0n3N2tFGMA--||~Q;H&A`=>lT0KzQQHx8kocM)!!1qFN>m`bRv!(#ey@ zb*y$8K5g{F!D-A(Ms=U4Ztx zwd>IyZ0*q2EimQDzU9Wxg_UL|R(h;HeNHWX2SJ-oaJ&~62ASm7O4(&opZeFAQU_Si zg|6%>q#ST#&yx4jGB&9q-y?7j8AJ_5g&s+I?%P(m-2vPqeFFnqP*a_qEsrZUHay1S zios<#MMj=C1#xPa9b2_wqNfjSTix*et~?Yf&jCs~7p;aXXzph|LG@y+w@*34N|L)~ z!e*F15-noZIBV>oa2-qj(J1ppWwb6igd&!OqoOA~CXTjl>m?-6-XKaLlFnHIGTC<> ztJryOz@{lwN^dpT%r|fBt0y}*yBn^8-l%=dZNR6jxvqJv>hF#^#G$_}3z9B5tZ3(( zDLMSky%}5sQp656GsHIcg6JpX5I1bj)Q=z4IyyQZK#djZCdeu5gKE&&)Kt>Y@WBvx zHB<5 zp~98xdC2spc46Acu=Oab2#cm{=W0(j6?TN2;Eb(@yBay9byeR3s>%g<0g;9K$gh@Z z)o!VwrtI&ap6KbMjTQzU&&Mr<%I4wf*zLFO<3n>u;|y_b+}@I{xn< z_@A?B@(>s&%2C1#MZujpKmlj%IR=&_ZAg8D4KS`2ynMy^yE53*gub4;xdJA@ganQX ze(t+2pfzIxDZ40&MKqxPW;@Th4d4zAf+j&FcP6co(rwzloezq)iAW{UgQqP6I`p&6 z5A90FH}R{hq=kPYvX*wPeU}j5mxxboxOvh&;th^YxpZpe!g^L{cvu&OqE#ZJ!=Rhm zB}6@9ZZ_AN&d)@%JkYr|Sk7SD)?v8iIqjjY`4CePxohs$IolFw?c6=Oi?xeF4RmXl z)?WPPUH|6^|9a+K@=*Qysj_zQ*$9ei&m#5A>)+ZUZahVqG7)gDksj#BPqpweg&Qk( zgPcYOb-^nIYoy}V)@fNm#l3mBEzRlEO-5)R4572Lfs%&CGT9b8S?jvpe}0hYD=;=F zq^z-}N4%|sS!(FPOBICbDP<3})#~)eW%K9y+mQgDmP{1+gmlf$Ju~Jz1${Pt+sRY~ z_i>a}CQuq2(y=)rErMq_k~)?vXqhuB-H%V*>FVRP5S;*_Gvu|i7T2h^_+^B#98D89 zuV3>wQ!x7egAWzRVl#0A+_0oTCzUI}KS1i(#Jt4)N8b(pstxn#K<)FCRz$6x`v~*V z@i=KhX0dGQU6Q`tmU@;rLkM2hbpAnn&tN7V9U8JWF0-m3?lyBJ>0z)r3qac=Y=HpC zWsBRvwEzvjxeqk?D+Jsn1>>E4FcVYU&GhH2fy!%<_-IP1VyS1}uhH?=S1!s~z%uAq z8}(n=j!4N~3{UfwrB`le7dxo}X??f&jL=5QaYNSRma2YU7(>+u>uR2T=u zpyI|xHc?FnaRj34hIV=6qp_~naPD=VUwgajQ%#^pwyF{WqGOEi$e|(#K@kBQ*PT z<0=lWLE)>0v23VC;PIi@6d}=a4_f1?z|#AZld!E{veZAGS9z zU5cOcE~t?BSf=pgE&c5bn|zQS8BgcZi2N5E3u2%^5(b1h?N$e2fs7JvONYo6KXu=) z)@#B%4SeGz>FY-}X5oXNSZ@My&Ib+dU^`;eazV_TikVs3!s4Qtx_Xp1Ccq8XL;?S( z3ddc3>vEZHb1)D?1FBd3s ztSNKVWJq2-`php*8m2;qis~0Q;9#CjKMnk3nV4DaJsp=(-2|El)z(`@CWUoFvz;M zZ~ZvrY+3(FtCR0MC43SP!X!SfvOieK}pHFD|_mG^wP$V(kFuaa0o3`ESMLLeSUZ~aW;LlS!>Rdx^ya#tU7WE zSdN``O*;qAbz$0X|H2&|;t4xB9zLG6QBD^Z`hAYHUeM}!f318qF1JuQ6!kCE9yPNL z=FX2H!phS7YI2AfyKiBtnH?xOr$GHm1reJL+M=Y9smo~R4b~QV#HTy^u4BE+M3r)d zuom$KSHb3*D>RY~X87$P1$b=ckn$oEf<%x(Sl%L>u03SMd9CN6(quk@LAAiN|6xo5 zg?BD+b}1GMx^-hfm2W+v?K3?c-BR*KI*jYSsf>7i@E!W2A6J_=o8Q^!?4FbF(pyP2 zos3A5r8cwef^?F}oga9&V`3OAVWy~KZsCI$oUL#eT3?pEMjQ4aduzqLh}@*^kMqpu z!3;2pHeZudkreaV7U^)r)p8{<&ba<`N&QTFMlW<>(uk#^w)3V9DDB*L4=N5p-S$2? zF46qsVKdOw8=06m0@Kpu#2E?EcV-8PrTM|hk`>7V$=ikyuInTWG{Us;{A087#V`9q zvJG-MZN?OU=LWBGJ1-Xrimj#CzJs%UWBFXQo0eJ&>RLd#+T3OnO>InCl#J(rQ%W#} z5Gr^w(Hkp?x+HOal=A#sd;^0GZ@Y&o-cb6q31=*y+L>wQ0qTj^ECTZab&%)i|7cGX z!9+%5$$!#@WDo4c*YXfHsp3D;d20&G&tjL;*jvEwjy* zRRo!w5$31k@&Qty!W4mHsP@uQZ$Vw?C{R@m6Jxjq>tprNk?Hd=$a-JhmZW?V;+pqx zLk)j&!bcsYJa64ss9rpGbD6_)a~kS-DQ z%&$IK#8B^Z%CWl)h=-mE$x$HZc=_J)@7LFOVB7^J9FUm07AMhC{`IR8AE@nXP?eRc@M_)CCLPqbf3)*zIqu&sd}I)IdRUl zBgLRJJx4X{1FbBOFC`HIq66Qcb1#MOG2S!D_s(yaK06cVA>Kk4{mNI_SpAw^4DdKG z1L}sp+H6~X#JT?7Em$$1bo-j>Ex(t*8Mmx%B>VF}V$g_K1*4#(BBKX-BW;y0ed{IH zN{E5t3_hmK_c|pAh^?g=`MwHIJa>PAXa3k z3w4T3$a@aux1VsS4NeR7c$mcGtZVqR$pv=}sDYHWz0QfR`*Z&UW63|eXOjd_NiQfr zcKpg!Lj|(o*%CWYu+oZgGk*=wScQAeWp7{zhC9u)$E2W-P%a>R$-3(X60560!n3mr zaNXA~r)qh|Gs_rBF?g}!UBjo!8*ys@HmP-=Zh1HI1OHdj@s%;&xVBTUkh9hdYaoJY|jtVdZwby&c~P*9vSAV@VsJ6z>LW=T%2ul*2Pbib5oTbS1&gNTtOQ(<@wT=Q8d};$TqaoVeUDvB z7dHC%TzcnVALy5FO0@oXr~trJGr6&eF%9B7)CM z|F~z$?1c5Yy)Mp^*d|L){opCfTu)!phiwu9#kwlWq`c|aW;L!B-25BIRsLM-o=FbB z?&zoLbN;j{KHIQ!e_7!1i7QezRUcsEt4D@`mhBPmNGn?fl%P4RlIuF9XKLREew%9j zvNw-ZVd0;AD8C@foTb6`CVg)p_&%Y(VUXwu|F^b^?kq$Yk*3ev9^dXgCW5$KLX$;m z5Of`fixxrXA-OSeX0nmppJ=I&KpgHr9Xv*R_ZA>qZ9%A0A}eiX$NVRL27M%U)WCDB z?r?u+dO7EpDGMwS4}FzUs>2kzF93wm2960AKuylFa;Y1Xs5Uz(ZCn(GwC3TJ@#vrrePF~Vmn>Rzd!o8^SpGSfOW>i$7Us85p^*zUZyCTu2p_Vm?my`PYNwXoX;N^WEQ>Z8ofkok3v6 zewv7OLd zaqCo^BrteRl|+poUwJyje(#v=no{|Vk#`ri1b76Zh8u6P%)Zqr#8~e?yGj2M?_ z%|A?e4sLN`XB4(*`Sh$1qD1>oPDw}XiB1%FfUUy;tCbpqo`XERW-h*MJl3csp}4Q& zo?`(!ChiQP!`3~z1dmTkt0XSp@%Z|SED;%C1-7}l>6k@7OSTZ9{d8ubWYTN9j$xup{s%?rt8)rWp~t5ynn$$6z40skt2D-MJa@GUINO0z!sYN4TuM&LH6MTa z%eABwyQEvMjgmi`KY#An{EwOd4j1b{dml@EtA8ARPEHv^_d>Ngy~uYh8=nL5>fn>gat@tQ@jSKO-cJ@}O)n3W-YF z%8?m!*xy|AzHXQ-sH!Po6OhMeOapttkD1j$PP3ecG?5x|Dg@-cq(9Wz>#7IA*_RbQ zZRvI9{J}5)*dV|p0Y`8!(p6d%ff@BR_5LDBK=q3wHwy_7oY%nI$Z37@sJNUCe&nZ9 zoapH6?44Io(0w~F&~h8{0=j~icKI7`h5p2B6rOp#B_meaAS3({yb$;yQ z6*wcgb#+dR(fFG{cOdg+!|6Ynv%vW4!HG_Ygqd@Uw+a0-_HmU`~|)9>apJWOx{fEr^0zmvY>BiD==22q26g zFO1HRD656!h8n_N&Mm+y`Ae&5O~%^l+4*~t@F?>Fk(Ybv>dCLD$y7_>^p0<&Gn^}7 z#WJG9l;>f^Cm1RElVVVeq~xL^M$S1FteJS^fWL=JFRGxfui@g5VK6z|Ns`>MrJWe2N~7#-JY;Oxmt^cbuKpElD?8fd-YA!I`jO zu4yKlL!h_QxXHZZe7v5{+^N;n3ONyjyN@yighDN`PyVHX zjO%tRQl}aU9$adtq%hJ#p-|&r-@A)trndaC*QW~|;!#2>i?nYyxGea*eAbtisEvn; zv=>p>A1JgS$E5P!yUU0d;h_o5oJ`ocj}6}2%LQ2z&kYJJk|3gjqh;6K0q?7P<;wQr z6ZZ*o_vVF4!8==zvc@0jD&EpYmmeAcZy`%yes`Zs;cFMLIK7t;IENGJ_$)kZbT&1F zKzOLX9Lh)c5W>P1Ou;vIPMt=bGd%B7)R#I&hpoNEbD|dqyhi3qE=)SykX0OG!!^LJ z&@st>xIA64ob#kEG?d~2EBcF363Tj&_KnIvZxtRCbhi`rGu~_S(;?c1eG~@cbB9e1 zo`sdw)0CkWk*lAz3@i!_QJ7QW%lqc%>K5#3IzcJbM#|D}HLy*8%um_r9N-G0U-RuWY@j z`>%0>7BpWO;h&`>5VzOHlQ7OG@HA&FwBr*pnc+{oT;m849c3w>L#WDqL9wx}ArP$q zgb>@SA8f(dss;0Z-{7EsQ+RMM(UrDopUD`FMrTXN6d}etmO)xw+~MBfT|e=xJDYLW z$ARN4yLFSn$gLqzZMc4L4Xwnrh4V^=O6jP6{?0pm-DD0a;`0LRBv)JxFpw#Hh3Yh6YsdvE(JMO=i(hKiy~=abhNEN92rl3>)0 zqyOH?(tQG5f8!yRWOhtL7%{D+L`uLEv6|@KhMO#%Saj&&2Dr zqJJ`G0M(7|WGsai&dm;@;mT#Nm5>Z2ssgI9Kv%LPSQWGbqkX@osKd@Y+J&Ol z<4oUn>S+a)nt%V&F()YW&_8Cp7hd9*Y9%Ccu(`fNb3B4`ws|h(XAlC2Pk>58cj!w& zh1^_7xjsR3EV$(R^|ABpwX0XLAn~S#hF!`eDCl5}Kmh00AaLuokT@@II}8rbcr4>J zA4A;mM_QkUKEcj4e95W`4%~@cvdMYVXbhVAzuLkXe#vmXm0#PuFTSxiYBA=OGbQv8H6M`8k8Fv$}>( zSL!6Bq{ywZDPHeV)*|FMb>MK~4Yef#!D?2Q^^E1B*CBsJIW9DgNwS+T!b2PD)`P8V zco6|JT)>uwvrXQV0qje|N~K*V?D|s!h$b<>JFzl1myg-;O)O>E*b14bAo0dHg9uX0 zdc^od+PyY-f9J!qdr#IgUS_gB1jR|ktIjD8HRBAyiz36NufPqe{Kyz#UrKjA7(#KQ zne;1b-2M%)7D$=lqR`}>7@Q?kG)^5VrGbh_lL@AI*77f)P7XOHtDIg^*@V8ZH4o69|NZi_ZPQSbcH*)NU!v>w29^=sb`OX~{>Ji~sxO@F{@{ zBA6fG3=(GtjL~{%!tLSS=jTRnxV8Kuyj~}2>2Lsi%tZEeON*kjqoc>dWn#$YCd3Q0 z*qvQnW+0P_9YEiuPGLJ$H{jiX#I1uy*C*=|eH)10XSMck*qGDb`01LV0J{^1i@flo z1{#4lsfFWu{K1bpev-L9Ei!aWVVCUzoGG#>rlc@&JW@JgfsR2|#4|avnBsr^tIP3i zFY$OEueku%gkaB#sT!}?O?5MB0s#|Ld+=#A=n-B|w4k#aBOJsHx=CzkSWZ*Z>V63e3q7$Jd)@^L?1DX&E*3j%X33#fA{A7&I_hD28_lP4 zZv2l63}uZBy=jb8Mj~%^>)+Ewde+>c`@+GCkf(zirfvu0C?c;*D0-;O?}J*zKrL^B zR&F0fGtifml$2_P)md6vV$38>IF0$GE>+L)WQMQ_z%hi=z~{E zPbw_H5l@bLI?7KHsu5x{R@IYYJ^r%-cnta0bl)ZyDn89Gcc$aqRjjzs#ZR?41i!$W zX@FlIwsYQGCGOAi2`ZBqINZaR+_9AIxaCg3|Xb}SVT%+M|(%L*wyMg*| za~OUw9UY0^_=boj@1>xnB*Hmt--X62af}S;z%TF1AJg_{&txYGi;7kOii42A_^&G} z9lF=*;)wStBKMoxOgl`;U+`M%n-3SrKW)5G64N`2;{Ddn&bdrx%Z)$uDEL8cq#cj$ z9|!k+XTqUfNGR|CwMJ}4vXgv%*_L(Z#}6B^HqN*b_(dwhcCUbd+Lo3+&Am z&so`RCk20e?N{GcwZAIn#mnXOfsVTHs5bIk-g5^+8ogIAsd)Op^=~zD)gE)_caI}p z#s^Hsikj3WXJ6&JrDz;h_CJPbB}mJ?6P<(;XAp~>sJdXkmZ@}n$m5M{EY`u0iGXTZ z6YJZpHH?@)l-KB^>M3~k%)+XifYMyIrxHcZPDe5@SOhd}I3Ew*T1h$m(sK7_NB5F5 zy$Krhp-B2*36j(I+YY7!yszd`sO04PGYD21tm|;UU>#i)G-o9#I!uB;D9qihjeO7K zzz#IgIj29sPK}PsVlx!-N?(q~NyjlpzTV#LvIq>YOuogpNe0kZzV-4J_dw36-x&}8 zs6=#%pbH{%zk^YTEjk!HtAxJr^GP(8Bpf^UVGRZcAk*F@dLwL@+#FdvyROl2qYefm z_r2@U>;9ZwsVfx$V^wKkd+?X<|JImU|D-6d429Xi#FKAF3rQkMysbY1!VI06!eWtN zmp*bf?)!LJKZIE?BZbL*k7W~D_Dp-D&v77gg)J+yW_DBLlIgTkLeG5Fc3K~0J3mAsmya+@QGoB-=pe8 zJyU4PQ(}ZWEif!aZz;Tycu9anK(s@JdDS&EEH4$z&d!>Lh3$B6QpLLSgKNU+!-o&0 zog;M$>O0XwplWV>{5J8mjkm3zyw;6PoeAYS$pel8Q%=J_#e6D#2BFq0KI_$5WImpIMHgvB1&Z3aaWW@fVCu>RzknVsVF8IxH2%A1O~jh4q5J|Am?vfLJskb{4U5N zdO#KETdBe)(r!DsL6+dYCCmu3RY<0TN7_+}?G`X!4Rg(AYgAwDf6sO4Zpm1QsGM8} z_5p82-no%&p7>$`(c!CbteEVl&vz+)EB*h{fd2pgw{e2bi&|N0qiU7jf_mqs?uJTD0{i1z}3jgN{muk`TDgI^oG&^rMkCM27v*D0Jqt7X%?U$glF9dfryzj~&Kr%`Ws z$hYc-f%7+f&31(rNl7}7usW~6GyN;b@LKed7TMxu6yJ}D60IcomO3?Ee7FFpiLaMjFK0qw{9;AiINLFSxNh8QKB=sYtAZ&7cu*G3i=Pwqn$6ca4hFW4KPtp#|pKt2(J+6TqScqxqe(0~YMZkR%P zi-Ka{&-$gLe~*%3;U_MqR5EVTMO{a~o9U2=#T9B^1MSbMSoM_C!qZsXjLg$-PRZa| z0q1fJx>W?r*387G`G&p=cN}~d9^zc<>ZE)h6vUa^v7Dq_*%GBciKwwV{P>*A!@xPvA~v{AKJWLt39@#zEpqG<^e8y z^{4Yy7gd|sz=5=QNOef8l_BkI%AVf^?SWkgyj?PzY9iYz+RgG3{3~1B|Tp@<_tO}BH4kc$V}ex zMDhIa5dmto*cNZ~P*dF`UL|)(iW7mYm5(e;M}glcS_`+rsd_g1_K6QL z+W=K*3eWah9mRPK0cG`h-b=Y0v+p`hDWq`w1&M0yA7h-V9UEggD38`0oC|!eVIJEF zO|VsaZ%Je%a`^TEZWtEJyd}7+j)Ywe zFMIKCkR9;VqT0mHoKoLN^2nDMi0x5KYjuK-D>OcA|3D+9N$@s{wrY zd$$PW+y}3S_=xQS;plLGZ8n;_#Zde~QqqE%Y(>V%RpJiPCa2`(e}_bOn03#O3uic< zd6Hh4-gsKWJJd+CmNKE=Idsx=NTDZL=aH1mxn+{)d%D<{-Ia8xjL+)D3rZLCFm~BV z24{+d?D3q)z*8$MRicH-g^}Zq5i{0mkZ%d$KpwGIBo0?2%kN}xi=#_?(;HWGARr{y z>m9Uy%>Ed*1)KhOw^8Gts2y>&FaXjxG)0iH@~zquVSj5qZdp^Lg6x#% z3p!2ZQLkRgTN}PQ_)M-x(f;$oPJCl8Zlo}i1rWB@m|7s_GHZps)xpAf`RUn9n%zVH z28e)ML#qvkdwrAL8=KpwC_gfHj#flaFFRpl1;i4Mu}vC0kHrBlOW;D1@hYGl*1UWv z@1wm@ANS^4Qgi0c3~`4bv>8~~! zuzwGMqI0ycBqgZ6zY@u3~(e2pzi#)2p1;Wa7Bq*1*lHg^^c!^G`mTIBO zLi$xHoavg;E_c~)W4=Eb5@!Hjb4QW0+hOJvz?Zlk%+}u4SY6W|4~xZJtTo1VfZ68u zHPXS)9V#F@+Oy;6sGZ#J=-}2^2LN~hi^mAyA0TZ?@m6ObK&kNfs4cU=&19yl{q96C*X|B z=XjXhx2b;xcqx2-RAm(IhjCfHmf%?u++#L8`npws5IE&5sTAmZg-u<~OtsVG!dXk) z+=A%;8vbUuKx3f?`ifIjhUfu)Ld7xAMyB#hr4m5Yo9rT6jspk4mL`C67PZ&cp3HONb1_)AClYO zuaIuW;i@0CHPvSci%wLWWduItk%#TrFPFfZpgpYYh)wAzdu8hn2E--MDlIN9n&&-# z{u*mbAC>mN2IMd@F-LRFZ*JDj6Z~-$ib6s{-9Q8W>YrFYh&DQs$5FgwbzWivD;!15 zFhygLb|;+aHJN}!e)^Srl^k{&O}S7Vof{8AL0%wbunPLxrZrsre`96*XZ`OlFhhV;GVbktj{`#A z_T@4{#H_9OtFr48RbAUx(1qB!`x;k0jwHeDaX=*a1NB|P<)iU?6%|M!Az^{51_t7S zf}Ms+sBGM%s>{4zbiU&oHewTuEC3_7m;ev5A~C9kDKsCR&BOAHhb7C4cP#7rYWew^ zDXH^)wL0E~i81G{hjklDx(I7k-GkPj4awGwiq}+GLHJ*!_*5f4tP$jjeash6Sm#Wu zj#{^YxbkopIWA-#CxpmX~pMjC6qPC{0qM#a?R8rCeJS`BSSYk^D+DT z%l6mMKl)R&XDW*EyS!%*+lJ4vDy;B0+Dy7#De#m)0r~Ui&xTkW(UciPV42HR2tFG8 z;vFuYW+Lu+30bi*F-gNNJrNqsUHj&VbaWeUwvGt~^ag!b3|1(~Nf=;3{9SD;D~?k> zoO`szX7v#kIFJq-r-Ffe=XjgAqK8j)RO@*l+#geSahW1HZK+1)nx2?3=_}}#=xrV$ zuykB})jjn2{7F;BTZR2SS8z?Hxb~fBs^bJY+Q2K5zk{g%F=Lj)AiK8$-bw&|APzgH zqCkN332tzoM_5f5M8x9ILZTfDBZX$N|BtP&0E+s1_f}kKiCs!MmyQLIl2QZ-0qKq< zr9+TLN?a5K=|&Kg?vP$sx&);gC6rX8Bz({Q?!Eu_-k;9wFgUX_emUQBp7X>fN-?8U zKAXfu2!JZH1TFRM64aeY5+JM*xQ2n;4e$BvUYapA{Lpe3KqC@=TYqW|Jl(=}}zy!e3G$ockh51rvj#VEslnI+k>$# zpm9sF_W<@I|AFj6Po)v z^Ti+j1W7=5r+0E`T{i)G37b&xk5hf@^dUv{TtNeh<^JQZ{8A3U8c$jI z!NwT+!hn&%edOaL?B84=?7iKJU-A4>K8E58$jcVh0ba;fDh~0T)ktF7j+Zk4pCoyr z12m?+46Y8v1AtA&E#EPj@=GBq&MI08h?(%jq* z>EBI^#o3YAImq73;>?Uc=aL7*-g<+T#oV_UywcXd-deZDnVa@`y=hB}RvM4ObtwyG zx^pRb`wk7|e+6vR@=)(&Hf3@D5fsS8LDORD?vw54^)7|#OeN0CzpFkMq*(BBs-T|P ze4|-156)RoQGm=Q-Z#ypy{N6=f=Q1y8+~i$~o#aLXXzhs!fvqZE zrK>no&$p=Eq6&ir)BM`2SHMGCOsS?>F_T${LKdHl%ijy|1yJ_aI<=48#(q7Jkbg)X zcBtIy^K!d0y+TRsivi1XP+ zj-aF`h;iFsWJb8RDe1Z00y!|GnB$)@85k5Gm_qqexVV4-K0olR+%C*lK`l*{rmcpt zG9$j{^lo~Ep)TbsjO>LUkZ2h3^RQz`65?o=-c3H3@;SQoF_OY8iRSTg4jR9LkW zzrhU7WiT{OS?*@qSADx$no)P!1>u3Set#_vWiVlfn|V>EiHM|ZpVUGGLqLxC4_5rY zb-|{!HKc7cq|a*-DX5Ks-3|u_$JO11kAqiYKnc;l4ZY9x1b>f*>~M%a6P(ORf75)- zL5$_*23_q1n&A2K=sKSlRn6?P(PE~vsvr4T;Dl^OiO|b6Orn&3#)DNBaA7qP4zBF~ zV5#x&J32-p;%sE6-gh5rsVU?>zho>C5t6Feh512m9@3fw9+bEx2=w&!$01 z6R1b!TZu%^<-o2azvq9)li&u-s;gEy}K8yu2Ci%G7g%qdcq%6gz z#HJXWuz7l!u{dEMVdaJhgpAA%m_KoG5E2m$Z5}j|F9QZ@y|48tXRu7yw=H;!N(C9% zl}Xb1zGYOGTl;90L@~#oUWgvNm;tuL#SgDnPzB7#ptCu<{rM=DC(^-4fpQvp_2KuV z@Y8_Gtnsu0vpHvsY`Ha0e?;O=Gv`Q-NP)>GJXLy z_Gj#l_>yS;A?UJK_IIJc{hA5GoWQF);;I0dPKLz$Tg0iTe@3N&j<1@Bef zxJO7@4nfli{H@tURs^D_51O&(Ew$v9U=MAZ9+--JpYwi(>L=b`aPs2A4$XMlNI?U} zfcrZDwa$X}SqVrOY3b<5XA_x4e?KlC7H3})sxxFg)!JCg1#Ld;^ zceN1N%VygEpHi_|utTn+`eMpPM@7ecv)ZvC+S*m^StPxVQVG9WrB_6sUIO|B3PSD8 za0aQl`oAOQfB(KW4_WI9pXMw>=8g_e1ORQ?TH%2#8vyHJ^3RJy(?k<1MwlC$nPvIj zV$9Hg_RFQ+PoW(OE?Tq-D){U(lQ(cTp#zc@VgHPr%;_#12-=<;*R_Db_f3BwTkizA zD~pF@^GpR`bkp&u$FaIi-h%cv66rVbl-$QOuw`Ue?9lMedWF}|7{lKm#An2O`MJ@K zUDK$#Lh*53l{ZVL36PgW%@^H-UXgzp;e<1Tj`T6b@?=Sh0T;YZ$9AT=Bt?#wQ}7LB z^`GN)dvy}vJs}Yosomz!I2Grs59UJZA(%VIegxl(E_!wsT4sax!kDH&zH@t39IP=6 ztg#JF;W$M4vaJemFf$f30(u9PovqEEEHh)FA4RIU9*7R8zpQU2)=~Y!Ia@dOX!c1F zLF@aK#c832W44W>@|)n^|JTRD7k|%$gIZ5*PF?L*bu^N-6Uofc%!oqints|oBsV*rG%+afY|(^%5ETIBsqGo9p?CF7$6I& zU|FgG9nPE-`zn!IV5=JZ)>%Fub7Y7J(NjO;8+siD4w&Z2DtP6HTeqBm98dS`ssT+# z=BrmLIMvBkfksLj*n^)LSjhk0AE&w3%FlaZRmEhRrYau4QRZ>!q#rZ)FClvdFyQCs z7j#j1zw z&_R$(;st_u`C;00=9L__5Z>^=w{OgilK>@d-eF3u0h8mV@F$y}8~Q)r!eDMx!BZs2 znKmo8YQzVW=-!qBL97<``LRH==g)k&_{xqAxDSoW|JYh`d$638^7DbXQW)w^pXgmo-_ODQ$ow>PLucEl6-MlSsgG)zKdKL z-U$8n7F;m3O*f|h|DzJ%T!^V@S9|KsO4oGyNVX?BDCeU@7-7Y z|Brr8yHlXstANXNMC!aGvB2=J5p4|GTyF8$>^^{aKQCKizcquVI!0G zWaPn9(p;WgkUkqRvjOn#?DcT7YNJr1s~GqfWudJ`FkSHgm{<|+?pkj@e0|^W3ok~? zKzsYoe_sAc_m-+vDPZ=i3g8g+CBV}641X2!gWhZ-xu$yfo%PGQYJA^w2!2E=t3i%Q ztarV$wz5)F>rg1=7X;}U85pK_hL2@~A4ssWvhuN9KA%A}o0Wt3`#dVD@$-bcw8gGe|2deqrIU=-GoXrNdH0YUCzXxhHJXTJzc!$F<;qPH9<5 z$r*9b-k)zgk(|SgndSGiFL`_=IQG3vS-R(8d2)GA#K=K;vgc}s#N*)umNO?&^(QK7 zb1u7ITQxa9h>E)$YR0KFesgj{{(E&}UiOKcOo5 zx@oTE^YiCV1E8}|TUXcU<=Jv;c_e<*gi8V-g!qB@AOOW-9py8()(L_Mqq>MDm6heD zM**Y#TJ*Jv>CuPt^Yb@_gxpm9_D+B{h)WDe7WIsbzOAXezFk@2bD}al*x=k2blxa> z^QIA?LV#w0ggkmZR^Gv(h%Q59a!6LEIs3D>u+4YJ`Q0N>*{N!dkSgL}Ck68#?5Jr^ zwFzr@^Izmxd<|cxzCE){gL{DoDBiVOp`$m!oB%wha@{3$5L?4k|cqI0+(J?R8-H@)LFmG zq;g@wjFakVWMrn=lJfc6EWiF}ug!_+G*QPFp)1}*8b3YX`yCWz$^^~;$jMgE1UX95t5+VzA#&XEaUK8I#sj6Eor!@-@EY7Z!erO? zJv}`$x2Eb{v#}(up~P`m$s>d23L~V0NB_{~SiUMYZ@|L3;$alL3EU=Jq7CyA5fOD> zn|fk)!;3@d_xkNRE^kQVA&XRK`=+-2@~4O-^%fe`y9BO*`yL^a zj^utVH)Z=?eQqHOM>i&>)&CL90a%XyGACSJlwjge@*IRwlJr2x1ds8gB6j-r| zI0njOkTC*l|EE39~kKV^%A}HUhs@VDQ;v z*Ibu+#=ocrG2<+&FFhXsEzj6_@hb>4$ZA1-=$(@oPpZX=m|#PMcscduj~C~n=8`}5!x|A~o>_?Ec6D?lsbeiXZ?AMj5Y?N7l_qlGPZ9QK|NgMBuoE=w zoXclH0^y}+3_f$81tcr}xaCDrC&L3QdJuSe%>^_zHUcMsHZRKA`|iti4~#Tx+*~r) zia7xo_;V4bL1E?Ca|}iWHu399#3iEg0`~*l@|Aar1sEWIJ%dQd{Bim<)sT1aaQeb) zx~`t*;~2Ds2C)`>VN28?H_bToC#hTx!pI&XDDZe@Q4@g%NtB7SAbf1=`+eVfOt_#r zxTJrpsNL#i^ExpzMj}rP%6ikzbjeE;EcC7zi+@*;)~Kp39ZMCl+X5bZqIKd%09xQV zO^5=z5w!h+j3o5p7T9^LVenwF_rf#yvyYe?sQs z*s9(oSOB+}894M=s&eQ_?Nw3>n%z*rb)lYBINytuedN)x=!cRsJa7L-;=+bkpFI{r zNeA%J-kpN<;<7H(cuYO(9x)@bzni6x<~-GPG~bk1*M?#+QD?OF`IqlmkhJZC`Qy!~ z*iBH?Z|8ZIdP7}vS@-J0f@S-OUTwRZjJsd*U}dzNByNtm-TK$8fh@!onv>#_jl`jZ z?c}z7H8hSKR1B(H2?p5`#cRoPU)Y& z^u>E`d|)XqFKe8hwF8_&vFEHM#|p0LB%S}Sm!~%TMK!zFO6t+)Yd$=v5vW|%G+)-G}j5X;@1;|vNb+&?RF;`>x(>oU+Y1s%U``QA;p z<6hG0^K(>bzA=>vJ&0Go)cgf=93Ic-wE8nr_6!{a5Z+BH(RNwB-HiTQh$91;tPZP7 ztofCNqYHlf{QdyvIzM6-@ck~90+Df?bGG*!cQ_FmCLGbRRvvs_?5Kb6V?dA&Z(aT? znkOpJ2=INPN+#&+A+{g7zxqv9h(3y$>RJnu3T_!GV{v4??3w`moa-D&-lb5sT|8)Q z|3XO;NKM)pw^?>Elu{0jz032fhBa9C;&^S^nH3pS8ZwE+v8Se`JFbp%Yu4` z>zh?!{qkQLga-~2^+Eo6uV?Fg7$o;W1TstO)xScJD<3svTdcY;X9W8XAFnJ3!hK$4 zIVS-#X&}%=(JlT*LCyjtPnN6<3enqNm{|4n&VIg?6;&Zt zp&-#iy1njQx+UKGIr#Tcu<2dg_2q0;Z}qzTxA}`{16pkcT#;wyi#zYXs3~_+>9$=Q zl>dXcAEXEhlrm}*Od(KXQC+#Moo(v$tv)SDzG*_nnn5}*DUQ~0y3oC@OsLCS*Lp@T z-289lBi3>-M3DL!O8Fi^KdSav?VPKTTl`Z!cM7PkBWZU~B?9+h`zI!M;uqc2)r zkg(U&xKX-<6qo6Vqbv_`cTqLVX57QBvk-VEVwc&^1wl8 zD(yP?E%Fg>4X1jH=Jr3HgIHm9kd`?Nmh2IcmL&(9h#RzgpD=cn2$6f=zUQpxs_EzE z)=K=zRux{B9+pV^`_T3Gng`pN<;=w({&4bGSlFxZak>Pal8&V%6M*xW^Y&zXTz$OW zyq8p7@AI9A`G-X!0p4mf27}c4B{M~^cYh24@n9(5tSWZCGche@8^#u22cNXdmyj_w zjzw-Fc=lg~25O699cusmF>nWI8&ig5m%_{Y>O&R*9Pm0eE7oaUP?vF8A-hPsbp1iX zt9z$KSg=M1(4eD$dsgyxJI_@;m?rHRFo>_*4czrtTy96lpFdCj_*P?9_Cvu}2;^#K zp2XB4YCD08j7v(K>++OQQC;0cqF!Zl5$JQ>GO+vPJTY($s# zln8IV*~fAmw+p{Mnv!w1`81}~yxt5zD!BrgY9L%+$OJ7v`@}y2sRg!ILMs*N?DMF< z<)M*&hC{j#SesHV5j5eLewmlihufbpf0j%N7UUspqE4%y9PBomoN=@OM+KJ>R(RO^ z)MVBy&9;xgO!Yw*p!85Ph@sC)F$zMv(u0A8$k6R*fJ@u{b+CDOIO{q)8|*jB!^>l8 z(+TV+11F(iZO)wIsE`tmcQT~V8i=KG{3#&538ST;03$GUAFO;208fAVmE3lge|iT^ zOc+HRtlQafAS}D`KO~@=S;r&2y>1?%{F&lhS^_2wOFTlr`VC+>*JXdd$9Cw6^bexT z4NcJwR~M=pfj%@6`T6NJ5TsB|rUsN9?K=ep1;%0Mx#!e+I-2Txb{AwWb^Tgh|7LNw9Xk%d){t$e$zXxX&eG@_l$(4n*cvi73f%9LaYl*RR&mY8dynsk?)^8%xCRI zJ;T7-2Z72}f_OUUy0IR0ICqj`{ z8aL&F5`N+wKSlLbBIC=;=Pw^`CvlNe2_{_04m?cFl=SLXR8*V;WDq*8D4Zz#H92}| zzfU$&dtc6j&nJT=BpzrQ9GAYXrywoiSz7W-rA7;Ff@xucyu16_;B`*+GtQF}zu#vg zzqj7>uJ{U^WsTDjPpmZ;7ZsUITgd9@-hn2BE_hr#Pmb8mB_htG>NTmwOF97dfxwU^? zNWKZ3_AfKef6$L5IG`IJoZ&1Fj$rg1EcOpRPU6&C=gdI{Lz!`MCLq2SIey&vE;lwe zm}wd@pO~vIj6X>^iJTiya;eJpgFHUU*U7H2!|WgRs^1Ub8>KD z9a6nj)y|GU2yRZ1^Dl5UK0}7i0(m#=w<^(xn)Jr?&b7foz7LV1{KW$#(6SP_BVlRbBO|HG}g6a!+FMLhnHIM5XV)9=Fz#OawG?dx@6|K>Ad+eG86ZZ%0rWNYF_AgUJ+ zO9bZx zfZr^8d~Nn?v9$MvL%Dx-0vWY4xBcwTUXbWG-HBPv?>5BTA$=V^dk5WE`1ksa7Q8{y z%#@8vgpS&a&mD46Sp5ilZn< zc>%(FG*M~_-6prs7y4OWs2c(ML$6hs}uvvcHWj> z`B4OB%Zrnf>qcZ)iSX_R02=s%*5_3=dJL|iK7pw#{CDoC4Zv0j`6lroX~`6(7N>+%ps@P9{L|35_Jd(dDU=GTdeX17*6x#}5Hgmd5*IYQCBWc->l)kmK?`yB z_Q#CBEYFfWnYiApGgIarFn?cTWXM+R!n{bNhTl=Mp4HP)dvl89yK)B$hL)$#=KY=q z!hpON<>Y%E#IoyM1ysAHJxNy1@3+CfII&D^e*SA5mNdjMhRs4-C%eJFn;3w_%6Z01 z%hk(wGJUJsN=t`~jEzk{+*t4Ie`HZ;RQ_?3ZTEMpn#6v#xF1>;$}I zTnHcsf}NMRXch?lO!Eju&(J3jn;a1V!cp`AfAi=v6;#z}9fAOCb!%g--T75Y1JjD13p07cn50Fdz@sTdxW}~^wHY!+!UdYr zuoqLBDL4|#)niWCC$VC${|2&Duy_cG84u)ucQeDgyVtugt+hlL1+5oAeK9DBZ3I>o zAvsc6dG*#vB_R6qS|J}sYJIq+f@HP#9^d9{1bIvp1Xoy_wW5n`mfptBPKx_GzzH;| zMBqSld%AbH-$SF}iZoBLENu51SUxhq3O{JW>Yp=n35fri|Yz{QIRD1trzh1Lvb}=t0co zo-x{;hwiD0`ibhDHz4e8J}*BprU<$9>>hEQnUzd<+ZDQOF%hXq_fCPE2eUovvs=fj z>dE|wkk;h)*9q$Kggd3`@kmxd4fWzuen$Q&4er!~^NYeiVqXFdHnO zGgo{_n3+SI9S*7to;G$cIB4Ze=7FCHKjD8Gw4c5xE)kcKV&CMwU9~-1FNJQ09Sx=~ zb-?1prmFD6IzlL^oW>Op?fq8>Au(fPT2?X;yspa;c)WC3frp04!1PgmISv?kP7DOw zOL42r%+Ah#<|Z>J{SJ;cZ84oeA6j0d|HP~4X%Lt<3A=}E)`z7fGp=fT-wLCPYv3S#5%>sdN}GgkVotvb>~q{wp1N5m z)^hT6(LrRGmvgDW!E?I!$BgcHkTKltu$NvKhln2XxF8e3AZ>! zO1nr8Ncxtl{n6l;gWOI51fd=ZnA7Two7*!IiInWK3$=!2iQ2eT^dcN^6a7~JEjl?K zvHcmK6+mvyJoWgZ|7zBypQOKQ7K48Wo{pCXt&@+<)%`x{TCPkFD*C60*qZM3Z_ z`)5_ybwf11Xwz!qRNRH)X&j1>hL-moYS3o~UZkEcW?ziZ1Py&7e5$RjsR;#D-BldC zVhV$Exe&A(UfD~}*BMIz;&?G}apcL5M?cLu?VcT!AA$AY^Ab?u}L=(eNvgJ zt*SaunxsKN_;7OR?CUhNbyFT)yM1SH5>1YjxiA-Ig44g{_`Z+!FE_(;Y{ zDzmtrqhfB5w!XwH#=frP78@3oJ2P#k%TIIR6a=_SvhGBrhI3za&%}B|kADTDo|*k$ z28E#gOdA3_0E9~NC<5Xrlr$O47^DwZx#Lpli=$Bbx`@geHl95u4$!ALeHQveIF$42 z`F4uWCOEg5rKwZ4U*nV*-Seud#xg5L`ujc5A4|9_1R!{`Y?W0Z*NICEEN0mjM(Wf` z^ek+y%7-eZ^T0qwl4sYse<8SK>ih%`NiauU2eDm$+w6X_l|rK$n^n*IHoxI-O92M_f6W2 z3e}-=@2avM2f97oPa1LHzm&n3qaY4Ev*?JUmzPk%bq|oWIJ{&jcJgJ#Y(qj5^fe~#OMoszU|* z%ww7c#PMZeHWL?!h7+;{%x=1qbg^uBmpU!Ja2HF zJem*{V68}D3pWBbK8Y-YlCt~@X?}{Oh5lIExozNjQTvG4SQw(ngX!{nhB|KuwX*Z= zj=Biq&nR#G3(_}4lNBuY*Arg^LJff+M8suVH@TB9C9UY(Lx~R*q{tDiX`1_7yjs6Q z8N@K-Z-iuRKN95S0DUt4Xh3ZtNoN>$D+l8Q#&Zn6)Qso+FOfwd5wRvPonBah%OzSn9CzA>*eu@1nQvFj`u&j zj!nvvJDH}2Wc7krI-;UGI&SjSPwaRNiIH?Sb2yF&yRFdNF7qUYned1-DAFCIUD1$T_oMXUw7r~s zZDJ2NvSl#yS);BqLPp{{)?glU^1s(>Ixrt z)QpP=c}@~}4hZ@mZ{{Icr8tH?*x`V}+^3|oB&&(tujCs2+G=hU!q}dr*Z39H)SDU@ zZI!(@vRxQ3_M1ofWo3NnC<3ZFSzC4-IAOWm04dfUkw{DrW;88cud;*m!}mbmZv z7@u8@dfM9?$ww#w6-cHP4wYer{uVM1MNbj;7r3Q#dlryoY?BVG?J_fUq9@b49KUZ4 z*<7z{xi!z@@rsXhJF7|?+2+~m_a83g|Nf<2j3A?V#wy!1#dQ@EMbJ;6r?!=w!lhD6 z{|x-&rIEyY5}xLiU*7ASZOVE_5FYGwdR|0qy$g17L5Nq!jU0eyy)f_ZFdzPsV6CdG zj;MS1qh4plvk;E^V7{FPqVf_b42Xexs=N;tck%6t$`zUHdI~)Hda1sVbvHMLk_6B` z!NbLKvi8A5Snsb>z@2Ip-lO4@U7HOwEM29jTGk z`ZF(i9Zb(%#G_E8i=R=qiO?Fx>i_Sgb9As`w z@Zby!haXw)yj1uH$6nMZ>N;Imb58c2AL~%0*LGol&AIM$1Cni`_Swp>zewYNfHNlz zjHdSZY&m|JjOsB z{xR#xJsP0qVPZd%=O1lq)4EoDRKNE@An4YE$7WX`zw?}Re7(m zh)Dm0BAidu<}dHT=L3Lb%CrG6PTdC3Cww+?>c|D}2T~?A;=R}jBL~oQdfn($E{Ldf z;;LmO7fODNkd}@hejy;`vzG(Rd>!@l^lZ=aPR3LP4R3_{-YTZRl%A8PLDo{fKcTum zn2^H6J&adbEk*Lmcg*>`Q$i11MC&olqc3wV5)xCkdbmmj9=BJ6cPLPzf;iY@+u@sY@hFXpvk~kEJ@0@`<`VFO8 z!b!+BR{IpG`qpj3)wh1@r_W!O8PHMf%8q8Jv4t!SH9kp>KRS&*+quaQu`EE6km5%D zdkv@n;!*!ry!-WbUz0>P&UPgf7E-5lBRdBUP<)@71eZ8Ra*mXHaq1izKh3#jn{HIZ z*UsBFg-@v3Z+^MEdfQ0r#cIy(m$E3eS?$z^hWJgkn+5nUnoPNF8)TsguLTA2+B2G0 zJHx8E=lGf4{>^1>LXeos|%&iBZFS)eY}AlhuV>cdXc#5Ns=o_N6kg{6&k7@ zb{Q0nIO<;Iu4@WKvd-tSF;%_m#kZoV`}-x)4tFcFY(=h|9mwRRY|zoyhi+kfv3OGo zm}4wkp`JRxkxVK57_;LX??}4U0G{J`3`^4l;>Cwe za>q@^TJn1YDnx#fRn6ALy&AWPII|O19HK<8qxRoJkFSd`|2?olZiaYX3oJvf+!9dS zVmww+bK<^Ve~k=uyd!89-u1RG#M-3H&&_23U*10fo)386+O*RN6@yThV)_~tjsqT2 zbLx3WYw2l9s;prBnEixA~T|{a=;P*5=qUUekU+41^+<$ zR$Cwx$n<8{CKf9Iv6Do($h`rzp~0D{FC_!<_Fu2pt}Q%hicvuoZ-;^79~!$!r>?@z z{7DjKIVth*dtIwg*G&s+c$Yy5ClBAkoA5nAY=iwV%f!EfGv1qU%Sa^n0TSR#Nfl)T z9Y0fiBD>-Hmo>o`!C7a4 z$GkSi;pJ@n#Rc!?qb^Moqk6j%im~P${$Z%BhcqyG@h%FrCw|QFe~_jmmC2UI1z)EP z)6-HtFHhd@jkVumtW&*zMepOBYiBHUFzVSepWaun=A>5}wPSPav!P;~eCsdv6%8{u z$0AH4X=t5e{&KFuuZ9pch>zB~Kh43iMqGMP8IK+*bEQ$kIC>MT%dB4*Nf`bYiXNai z37=cB>4<3|S#^#|>NZ0FeAzI*t*<;Bv`tDSb!CmUN%(>|$?l9xKgSuHfM?o}G}}as3t?)ZyB-0zJmNtr&<=jz2Vx?R z6{}8=2oN^2(QuH$2?cx&xvi%;sX%43fQ6$QZf8QOoaAhFCtpFN2=4c*CnMG(z2HTH z!GMNFg(!X_&s~kau?z*(+Xo61MeMA)9OO4njbXsy?cYQC5LY(MkCOIwWe2&Xp0WZa z^mR8g=eh?mX%BrgtF#7;%bfl^lK5_AB;+6%-)<}KuAkzaH zLxYmsqc4e3SaP?)ed-0D7r-YK_<)VvzK40O!a7mwx{^q~`d((g%Wn`5Xp7!|1Rc!! zD_xA0Yz{Itk&2$AjuV>(Dw+J5o!Hq=zBrR+NM$Q?SA+#XF(X+UUj8h%6-0ceBnsrf zIl3>0TxlhLI!^es`)vuG1l0Lm%rO+;!ZDck5A0GRhBrD&qDEz_h*>5npp*^#eJbHE zqfqasWLJy5msGNDPG#_Sc8j5Lw`gkI9L{}jLH%QID z*2H_wPXNQAz_55KhJ)DrT%CCSj6)rpnbsn{fsUh+glC#r3QOB+L-+pDcuopw_ri)6 z-|SN07eF>YU4DLKz|zQ>kek<$^y(iJ<@chL3ebGs_la$}PWfw`ZNSa`>J|@xz>K1hK%Ul$!Tow&wmAp-_t4MyXm#}$^1pjeChqIv*Ba@^ zZiSZVa#S{+Dwo6mU}`es6f5VvcC>EI!wJVw)_4|0=G;^0wl=fH!6@W9^C~}PlY#%# z=q^tD=)mYpx>F0pOX>YvJn*rrg3fFlOur!_wJyvt6UU2gjRTs#2K=F=E`#$zD$j8e z?01TKYqo8^CiV5B-q1Y%9)6J?W7AzS4m_6s-GPwPB)9)|y;8EqwJ0Q|;nOEqFBt4nz1DAYPw`m6FMYs!o3f%>0Y;^s|jn9{`nP0oq`elvi?erx^ zCkIf;uau> zR=k1$#nMtn37D7tX+Z1iMF+BQhRx_Y!2jUwzWYLIKm_Q{hhoA3C;AIE+e!LA+0L<9 zy_>hoaDko66u!~T%y+wW@dwfq_$;H+LN1+o+DWOjb`ms+fI~vLno36)Kgx^f$Yo#920t0_-X6xdJkp=iPUQyhgO2hm zzwOOth(X`ETlpn4gaaFRp#>EIA3hCc02+jWEIX!W+}RsHFYhsN6>=hl=>Z9OB>-pf zc*d8|=iy7Uf4}0-#rN=G-hYieFJ%3X&z?bYc>*f)EXko1d;)BDt>*LDC2={Ivz-o9eE|!dX z?sDbGHwDHDzc{*WfoEE$d3i!oqTR!WC^ssi2!|%@I>HEcm8RWF z#ooQqv}oZh-dgLjt54o|znu?(m8AH5GOv8+&xls|be{l^VgLsMtGwm^a(zlVA@(5I zzD_O|O{?#Fr8r$rq^i=&U;$rh-r>%gZT?1~GlyUIPS889Ud=|)CiU72sMG%}!G9K` zLI%=^+p`)7cbPf}h3Plxx{BR^V|utpTue&%`akVUsg!Wqq*fUsMpCt)7VEJG*N!UEr5k1Vp(1c8+ayqUJ$r!ZOH4ysVQ6}6{ zQ~W6u_2omDWDa){dkhr{2Ns#`bK zO@4XkubcI)ze-0xEzn;X-ok;jek!5%f8WMsh@Mi;W1rh*!+mrX!H)}7IRzhIl^X7^ zu*em_L>8^ZRs!rx)l@t1|Fy%X1K}e<+?q$og-Q29B zT|B?N^!QdrHB_C`nR{)uyOl{#;T|2YN0?2DhBLoj*8Jr-$6zORPOXlwqcy%%|Jj=V z{az}0Ke)DVH{o)YTt2L5ZJjEFB}y>Y1iEUwo_S_3bGa+Deo^I1$} zY2tfHhu4^p`w=SDSjJHe7*L){H8ItXl+3^0Qp^UN?Q3XyM%V1TR5CA5Ot{Yfh+19NWj)U5FjJ_?Id zKL)IBrZq;2-hjp>yYW=d#4a>AINHPsmdj0yJlOE>txPZdNku>IG0{m*74 ztHrsaN7y)0Mt5W8AZ%pBYeR~Nv!K8(A??+>S7(n}>e{qir}i9(EEhKrRmO&1``3BB44Eiz-#bn)RekoZ&`u%yX3eG}U0mUH4o(gT8ncPVbFhZy2}M0N zK#(T|b3kC0smjA_JRoM|B5AWWG`U-{#_?pa?|+-^k2M!<&jaDILMbu)1M1C@^HJ+&A5 z?q-b(;yU^D&R=Z8f79ad!n3w^SBzRD(HJA;eaX$+Pb`q8YWr6-#_kX-mU$6>Z4hmg z@odp&m^f}M4(2-!QZ%aiQLX)RZdkabp;~9PjzVNqMSN7Bb!j>H|AZ~DDhBm9R(gS! zm<|*b5&S}Du}F|FgC;wLL`9Z*t>EiW=lHL}=mR%THgHal?c{_iW!6%wP1wNJ6b+5( zNdEqG@&XU@?tIkky_Y5@Uy6Ohbn$W}ceFuf@8C~s-9+n$dC}5zKf>Z%c{xg|V1XRD zZZZ<=|30k~S{AIrY^0pvzC1vGr3&;e(6y=y?NGf<4T45e1XNb)#~mSZJ87QDA4?O* zwIFiSoDsCpqcO11b(>CZ5Wt3gP5TXfJ2y!-*?_xn*>nDLNv~4<mL=FRrRv=d@ZaUpOBjL=1utV)UAgx zw0~??JzWb2KRt3H&Y2SSpZ<$tn`}g!ii(_Jlf46#TI#Y8*wu;RG5rS^m>L&i)aBUv zxgmH#DVdTaw`i||J?pN=xQjRevKi;!g`5U>HOzKF$y?7CyHPm~yTe9Kxt3puNz-vI z9lWNex>Yo+?V;8Ds+8P2bkhAV0p3e+X_C62bmex_LE+gv3La>`jG0G=i?tmx!Xl;% zC(4CoK21^>ASz`pB*@fCUiz-_P2K|=HaaJ7!zLgv1- z#2a8}<7^XCr@QLl81XVArzgCJ+|tbB`rVJ6umAg^|NnslhEuHp11MpE+||6fJ%^TCR*GT9u~yd#^?r)x*8mQt&qi z7yQj2up_@cN*rEn93wYZlb-%8SA5)ri@>SYv3O&7m40#cT71lShzOSE-I_ zvyi{5h^x24c9K9vwcx^1?fwnC|CIy%vo!6r_=Ag=rX|GPIa6qUT6E}XSsR;!c&`5~ z`ka!dG6?pS8%3qgUV?rgotn&?;v>A{cwPj7Ir0y0Mi@P@VEy#bfBnm?guPa%O!Mf0 zShA$oH=~zN71_X1$*uW>$thQ`c~C$yPWf$bEjI_3og-t1><7Ug>%qqi_?cyke%x3TMrOSW+uwU)ImV+=FGE@#61EOBLlB;1pSi8R)6$D54+_Q#O zN!%)!;-?|OTeH(E%ehx!oMqnN%1Z&FyZwIZ6zgvtmF~BC8|7_A>vi-gB}Gt97~HOD zGK;=$!SwIw8m`?QhFUcKyh-ZhBZRP-q!s=cf6$TTac=MWzQ0uMk)DxOg#&8_PL)u1 zdAS-l6tw}kIv;`4L@|);`G&PASnyqE(ACl5l8`V2S=u=rc*NQLo>8S#a1|!p@3oT5 zC`_Hs3D?d`bZ-1GSMImT>pqVY5HdP;LuKYux&?N1Ne~c^nTVBH%Fd5dQl|O; zy=0r}IsW#ycMZ9|zt!gdMn|iUad|RWu}L_xEigIIG@#=mc4y5oO-gp9 z1w&E((#M1$lmy^-JrlIex+NiIVQ-_ zZ(Ecpklnxn{q#Qv=|30pEj6%IvRMujN^gzF9g8{PtRmPfVr1%Ctl$ohu1(O>A6qkF zi;(LR(H5*FBL2I+-7yYUz!5-J&vx(oYEx z>DoM5W^8y_Mg&IY7&sZC3SXNpw*zF{RyGZq6`^P)3;J*}(gLK_#Ke{+unB zKq8Pb`d_rvW8RE=(^rg>XHPNVXHSDB91*RVzbmS9R^{i5^Egf8w9?!Roz+>eIRCx{ z$>b~waayR<0s;;Xhhj6mhb%EJPt|H#6C*(g0JTgJcfXFZi;?E=!@|=SwLO(c)(P9K z(U@e5mC-pLDjXt^wDR>r(_H-s-o~N0eOr27eltGs4NUiQHq0+k(JmVI4|r&vzc6L~ z`dU}2%6)a6qWRJGzw8G-U64C#WoJ50Qr|qY?FAu?gB|nr$wAevs(h}7HO;K#*JmYr z3&hGom7f-qrh@3v3Qi+TN=!DOnt5=Or8q!5%!=G=O1M91;X$B(p*CkpPk6Gsin6+9 zOlgddyGE+nwD#-%&_^5`7~WA_x__SteC7(B5E&v?8QI0M^5MttS{|!iq0;f0?GH?U z{>aDEQd6?A9Xw4#RzDbb&`&~>Z$Y(CF) z*u1Wrt?Rt!tl|8cWWrGB!K-`+-gO-XAxn-j=%lr}Ph1Zhs2qGurZ*)iL0NT#5d$%* zy8hRr|Hs!?Kt;K>T^k@EDlq~oDM$z+AR-MCf|4qoGm_GcG|~(y(p}OqG|~(y-Q6Hv zLpS_4@4vqFfA4|wyN+wQ);ULJp69-=*w^0s%FU22z+tPLhW!DbDzdQs~sFN^p6(sf~ig%%d0RT*zj3~iv|Oi$Hf zg7>v>c3G;q7--0-j32H?!6mEBt}{7_nQJTRp)=v@_%J_Aiq*>;MyeA)-L_3F7EAVe)%nH#Q4YB3VC09 zdyKVV`;a=ZmG(%9KBbC!&eLpSq_Cpy{^Sm2#P1Jx4P=AQVbXpJnb_gNd^GUW z%+xH2F*npuh6MM~=a&bZ7^XB-`=e*lw_}FM=qOSdUUv=3Zrx7CBdzF7fhgGb_p_0_ zu8oU|>)DTTSfBin)}hUrS!|m0g2}P0A|SFEBGJ$;o3r#5i$=boIT-J^!gtuLdLXmI zd}71j{%o4mRJT``p0s@j){Aw!yN0ICg2iVnsNH>h(qeom^pWQRkJCbaxiiObEnbaE z1}~U72=hNCail78!NONcI6s7QpMzFrQV znz&zl2vRQaD!L^zbaWU0IzK#2FFnVH#uzA6O%$6}Agp3aWEWRLs|f5S>)@|do(F}+ zBs91mq6ADut^_31F#73nea0#;msPR?JGW>5DDM+l!}P-5o-rfC98^%USk^S|zXaw_ zM}L82*h-Nfi~K&J1WDlle)zg(L5|P+5?qZ%&s$aHU9NCaYBBFXY?1NxeoFc-FE_y} z{rnQaqbGq&MIQKj?Dkewj*1=foh@x2?48?k2B$##fkT&x&6<4hOx{Kd4Z?ut7nBBG z$RA;|i2O@jfq17VCfN_P@&?vkb!-L>j`_B4iL$$dEt>P#s-!-t%ln{1Z0)S=GajhK z!)>si&Ay27R_Vk>DgN_5;=Y~6a7o;Ho;BP}JWjGaMEPX>NyE}`$QyaBtj<;rqf%3i zv{1{;=adoeC4M#QzPPA=8sR-ZGB=zo1&W)MRINsg*PLw0tn4R-yDR!*2~pYAnuwe@ zx_pr#f;ZN(^UFUx@GEk#2(F7p_%2_tX`Za5{;`leLt1%96H47v5n8)HIAr`%O6W+5>V!<2eE7ZpThe1DN4fCdw&|f`C$#?xDBPqc2S;2XS-8s3uoU3Z5 zllvGfcFO(rSBv3h$ge{iJ6z$D&9)fQ1ompO(mOI!sw@4a;pLWX@>09^q`tPFzaN&h zwa1!&qas+=u_2$%*b%TVGkf^YzU3`ND>~T9?)q`oRn7=7?$cM_c#6=<$^A_6Pf-Fk zp1kt%cEOIM|8`vVv~Z|(jQkWyLFO3S)pN{JE*E{x$4UCSVNx^e*@Hm>y~2r{k9rKW zt=qlHD~FtquDm60-kUfyl_9&0VK>!EKF(%3l01r&)&7vGZy4I}l3}jML`ERpyoV)DENclIVT%b}5p9k1RQ^_}%xcO0qGl{r?%l@;Q`kAh-ienX!O ztj|sZJ{uKrKin?-92y-&;-=g3^q;W8_qVUH{FIr^T5e%M8S&g9gr9eDB-1PDgyaSl zG{Qs8xD-{=xhg)r`nSNf(xO@NN~2Z~F11R#b#(N=(J{Av<6{Ko6Vv#e9TKF1YDhgK~4Ebu6fpQo~}^! zUiph&LId}Ie+_?~u&GClJcMPsyaS-igDq_nPl_k;TRvd5mJHw_))3u+2MU=y{1bMb z4d^cKWMgk&xcwuC+_=R;JIyYpUg$YAl+!hDi%tFV^?p``vjnyE8L}SEON?!4X^~%| z+|`z648V=p5&JG*5H@kD1!3<%7H63C&7}P{9DOc#A~fwYl?ZdjJS4qa$ZFBrBK8Ts z7i=>q##xnDDACB7;j?q$>zi4>2)lX~@1XNU{2t*X@w2dIOq$U#`KVmZ%*6PcEPv=& zoToTNag4!26SWp*{RfnB7Cw8Tpcy#6XMXyL8UzlxzWgM&vH`a2K6bl-?%}l5T)g7p z_xS_(>U}-YlBDU3c~JrH2BDOV8d%pQkojA4L3$Gf2Dp+FR9yC1(tFlMNy)mh=w*8J zt&+t}LmqZIDsXFAJc(yCpXKf6Ys{pLanK;ropA~e5OK|i!^TqFpL#~pp zsASQ99L5_iU-3{du#Q$Xrz$zKW+W|Gp<+Ovfv&R1GS2_Z%ERu9Xr%QJF7T|QRCY2T zU+2K2A?xtheVUg5Qn~93mK{lOk@8_c{v@Afyuaio!^q{7SnqmQZ?MisQod%ZQBAu< zt3a{r@$=Svn&B4s7K?=w3Eh>VPPnx8Z@(W{5z=p5#Z=X>3ulW1+cczAlQ-HyjLQXV)b_n2PW<#N2kx{rsqqdJVfU{w)+k?t*{;Se)B z`_++@o3$f|@*mX(?!>L2<;PYdlEG?jb6k4t5>EE{;-zcK??jY8Ge0nT-W#|t|Mf!T z5+#0ju(_=v3Q|l2F25-3G59cTwRVCyd*+hzp)E}@r!U>TLycC5V`al9;N6q_xost% zXT?^(A2a@nD5}-4(~&BhgtazFWlbGsvqf-y^-mrC=h1z8!)1WbTT#x7EbDJKH&5aw zVsFWSfyFUd`=ee49LZ5stccjm>UNQwsd|I-fe)PL_;jA(NeC}in#!`!gX=p1Sxj_i zFNrHi#ZxcOn9zS?t@vohiug=(u~reaFs1vA#^&o6oYK#dW}_d&az+bCj57Z8?6;B( zgz1PLz|AeMltH+`x73ZxyE3^BIllNKYIvp+S37Qt<^Sw`F5feghYm3zN$*d@oqGL& z885h|yp8XxD_lEx3k{77v;$kPXxcLf4z?r zgI%xzRC2~eSObV+gC+VWM=41&8Ty-kz93lql|jKU{~`VXBJfI#efkg;{TxG->Peni z`xB_MQ?8@}@kwY*2(Dkfe9KISTPT;R!o#}48b>iSNMh`$y_6qpmh3_GTi9i%#IR!@ zqHFIx9a-P4`0T6lx+CT3#w};6gCEE_OdRS{mh;eXoo&-PVzts;zs%xqvQ;I58r>Vh zj{wSSr~f(c^($JccSnS8A1?gTF9e<}s&P4#*wh7Ui;qhjeqKepaTSeR^eW<5Uh`=W zai+eASn8$SGb-;L)@7G&VqkfyF8Tr}WK4?#lp$;Xk`Yw)gLy4cC}bNen^NJ*9sA_! zajtZh)%!D=qqi_|UamC*#LfLtVd;tZp#_ikqta&Ek<{m_Y%@1SJ~@?(ZTHH_g@C?O zDf4Bktdowl}&(p=-4BHwIQPA)oLI#osW=teUd^ z4Avn#H+B;o($qA^6!-hsPZt#A56j_WDlxM;s=| zO|I}?!VIfgUxXc0$4BER5Lf%w%3X#P}Ic^?*O`x52Cnd%_=T|V(57fuH`1-4# z5R>QS&6=>8Di&Hw*5eh8G(L`j?42QdUbj37L9WUwS^gbZT@>uZ%7=MU7~rM9ohp(H zYS17(Y~&=8egN0VUPrT=Xp}NgoZw{6y`wT{(;mj5UO7TU(y)%Jac;Aq@S)w=Xmh^g zGv%0|*IK)l(Tk4W+$aa5tUvd~-pr32d^SRVhbuBX=S_7!;oo&7*i3I>*))22heq!c zzpSYoPCTeFk*_b_AXjIW@^Ye^|BDnB{D?)#XF0?OQq$C?!t3=&7-f`>n9=Qo;nbaUB1~GG*|c@OIOVNlS^C2woSfj~&epTzP3QGZ#*58lD6IRg^YAv1)R~x)?pt*(eOf2cftKmc&6Pv|hzp zKd9>p)7Y8{V&-v$QQj;3!zt{4c{imRZM2k|4LRpP8BpHC_bZ^9<$j<@QN}A#m27Ck z!nPqcwRsUz@a`rAFT`v+Ns2$jA}M2Uw3FzX`{#?@?uDz~y8Nv<+H%wluB7teOBUV! zuo!&kKw;?*K&x*F+nFa-(hEntafLH?EVy-fz-EzePSISn!Si?#ZTL?5QFP28(c;)% z%7nml7WEww8n(xg1D%pbqoo-iqm5F8{XOsrj+PIq0%#$~i>U+0YT!@ zX`D=WX{5&PN3srDXn@!;VdZh$Q+<~-m!Y7bzf@et9)!Vz7BH=eR*c{2;vvG;=Bb7=}v8>@Jzbu%y@Qk{*K=*Gw~U zy{TOgFRfQ_rc{`}3{a+~>AK|?mb_)!%;lATE;I3 z9<-}?kvEx($xEx1K-X60+2 zQOt(iSN5I6^tmRzfv>;$^4qdoU#S}$3!8=}=ht3||B;C_>WN z_Z2chv}h@Dr8G3@4oqz-ymB8X!q5Vg`d0a4^#&phcmMc0OW0L8UTPL&-deKefw=HPC?XD7e>QLXQ-tJEBz4GtaM1Qos4irAMfmCS zXIJwzDHjFCh{G)x`q0G`hR$f!lZ)#Bbe3nP<6<0+4*S;3Ec)zU1N^14a=8?`44HZW z+D6~`LGF`O)K4(Y7AkB_>;&h&T^xnyB8Ll(9(n`yT9S#)XU3}J@&UG_O@!5|U)3jR z<=93EgG1S`Cap}$IgJYYAz2NRF<8`LH(G_Re)$!&>e5hrFhyT(-y^#Z&OcSMOlD4d zPmd9q#NVuE>}n}xgc6Fy-poeX5tx03e21zalpEGd<0_VbGg#0~?G%uQC#Q zyx)W^F8vv#Tp#&}E9*noCXLgo$iJ+q)-gmIp(xq0G(m(E-tD|EO6c~Vqo>!wo6I#e zjsSF%^1(Oh#BY8=AhPP9TA^0r`l#|%GU2Y=$tiVXJ9?L7pT7XKq`Ie*hoHH$G}-F z(R%1_Mkce&fORkpQusyZ%Vr5?00!yN|Ve&cAz_ZFvn28Nk{ys z)v0;7NcK@M`*-@n3bY4?FZ*32(M!W$E$GrQQky-tUrfBx4SavQi+Q*gXu<@79RXGO zar62@I}3wmY|6yo;V8p9!=fJ=!S`L^OOdPrTFc-ai%QkHGK0|ZTfR`xpKMAQRssJ;UF)!=r7#Twos@J61~yBRc80pSmMcFUSLW6vbmuVC zScRo04P-8>5juwUZmCtK{A7BCVtJONL3)v21sas83!!dRjBlBdCb$0Q(I2e0Du_#T zb?WQr3?7T`uABvTaOvDn(9z}1{;{%l|0tzu-p<$_FxK2<J`|ag)R8_NP)J|lehdUGBz`ANeUfi(sJSv(oWcYJja=#X!6gByohkOGUemn=c zh?)Hn#D0(OIJV`8u*;-3+)yN?(w7$4S+o)oY7Q)E3!fSlR-mg-C?pbDhX9{_hsTUdVQ;z{EHjj+Tp=PG5uI92E(l-W?-A!pq4vsZUdWbf zkBEvY*q*=x<6V2ZgNbiez(nQ@40wBi5qmP=z*E!GGO)0yE|NlOLb7arph2>{TXI2M zv~j4p-6vm{p$StZ9bC&u%$J^0Tj?b@iO|uG?v;TQ8)&*X?Fv1o<-ZI4aNSh3A+vU1 z|LMTg$^bexjj`3etTGG5BTl_c#!`H#lFu5tR=L(nAJU3KOw|%-BfJNuhUbB$Si^Ru z&Ffn@V2ETx*X85H!N!}VGErZy{SD)g*t}|j#Wc(4J?G#OS69CNroyNje1*^rQSx>BbqF~U{A4&O+inb%qN5584zfpOdHZf?R@lCS27mFqi zC9$;~cu9Yw)91l(!NA0o;JKkgP)!;c*OL+^pjLA1gA_HmF<@Lw;G12d5$PYxX3JGs zrfAQY0J}l~yosivHtm`7<2MXo0Lbc5E}N4(9qkVXLUb}ZWh($9v zUu4l6C=O6(vlcOc6oOGx&YVKIWqsHy>wSL%o7YKP*w1)EhIQ`8mkdX5i(u1V*P|*D zeJv8hQ|Fv5K*uMF#<1)V(XlI__kC5=(#mN|XO=$binUtxIMtu*+IsJ295XwuzogWv zFZTNJwu6^LaYHTY7o|+$XmLvz<7hOBP^C^$EhNL|)Q@^S7~{G){GxlDmD1$CvSkE%kt#EJU9BvT1A?tO0U4e%ACuJMO0TA%#||Du<0GR ze-|rCn=r{6>6M6?_sT}ZK5>Y@_$R;(s$UJIgp_ZxlQ&Mp#(s1v>8*2JZfWxsQMuugQ8&(v){+ zS_m2DUhCJ~E_}0yWoM_5p+2dWdtV;P!cO&WL~o{C9Jcpvgz8F~Z*YC9G8IvIbH8|B zV6|c-4SUd@m74QYW0_uN{^vo_b)61fLoLrWWlRPdjhRx5`R4b``R4Tzqi$a4jEsy2 ze!021UUBuO?=E6uNJu$M-(uj9g*ZDMEIbvEmJS5IOl~tLyI`@26Rq%{_V$oK5S5I^ zf3qA$#Kn=JVUc|tuQ}pdbPH)}5&<3z8HI(cHnB(~k~`7M3(tuMEUrU3IZlx!Jfo$> z_|kZG;ur&kNMu)UgrMV29+r2Bu{(avqUI}+gYcedR(lI(Sclj8p3v`>-)EW&F*fPU zXmLKTC^{BxB*XjPgy#Q5T$g5sZsQ^T%B}`BhDFO`yTHz=r9uO>E&3uKJn-t50V&xU zWGj&0H-+;KnlIyrqGIy5G~kWS1-w0^;`vA&cDoR&W!v4i_pwi%wrVOXRh{<~oKr4N z<1Z{wN2kFH3NOchn^6rfjuh%z9c|CUfEfZB1MMlxyLa!9!&UaBaZbm2z-36<*4DPQ zqeG4PE$XAjpuFn6`#7r&aLDob$)?qEFZI!?&F|SPmQ>x;(C}y?5@|_E$vo)PHzlRW z1);#BIaS|45ZDoGk_fNnw0>TW#K6vym7QbjJsf|GESp!o!MmeK`YOkC7&-90_;u(7ja5bg8Y0LQi?Ry?Mlzca$X3lmt*Ho^`UtB*Vm!Iop&8)jRx z?||na{o&Ho!Hi3TLnI9jzJtx%Piks8_le(p1-@LP)sFUgbx-LYgRL?P>{DroA1K@; zAnkut{kiVI_Q2`PVhTY%2aG6yA=WotSvfiMifDe%BS-$+%ZI?UN*!bOBIIlFyf0L3 ztv?@|^vUDsf)rmYFx%+~1mFqr34>f%xbYW{jnnF!^PoJ@tBD1+j&@4ZLCB!%Snmpl zD3n_@)nU~Q;3Ja%pyPy3wZ=&e_-tdlK7hh!w8MEQP&4zP4)d-U zw@ZyD?m8_rPZOc4viq_S)tN(3XEk7qFA(otA4Ntn~jP&IIO0;vwq&s4t zP9r6@;MGq#Ej9vZ9y@H4<X_*@(S5NmC(2?v+;n2C*-&7PHIzA({6D5 zsqm%gnA^a;gZOx8|G`Y63nCpgb3e_I`0?YOt+{5m838Y^ht-)g9z^QhJ6-(byFq_i zTkn!N0}t~f)DI;2V)M2y^px@uvEg2W!*a?I0Unu{1Mr7C-l*8z$ZkdD&4E>W17P*6 z+Z;*`-{}FFx6SvSu`%M^1eE%tOGn!burP!glZmG`f`8$p%Y>~)r?Ej_e{xcxxmNL` z_?AA1_H6shpEaFPZi3j<^w|!5GGp02!2_}{+Dl;90TY<;?+0URdFE23<(_*TDN?bO zmKD;5si7q=nOeKWD_ZxGdFRm}Pa`+?te*giw_(#Jcq9}k-{Z-I<4b4L^I4{IO2x7~ z+!hCp8UH-T!=XWYMK^ilWw1&&8LX&tqrgR>>fF4+8s$6nZEtLCrUMT`@AFmq^%FrI zzyugk1$ZNe^q8C2Gt@~7;(U%L>!ruQ;V=sr`6)LxHVS;5)sC-ZZyd}~aZ6|jOsS~g zR)o)Fy{RPfz?q82U}UN}HSeyU6>xk($Tbw|5D3$tw!`KhGI$_xXwf*$C6&P#YlmEa2oYs+Zm0s@QAyNDfaO6F6 z1gs4Vvd)jjz--BKj+HHOX+~Fk}1g8XRUV1~FWhNz@CkC}Juya#D zU0vN2In_pXUnu5uw1d2I{s#nTW?O1hUji1c(qV7H=Fp+`uNeclO~-` zjE+zQkm`b@ygXFrsm(MCsB=KZr+ButsHLlm@x_zeceAhF??|(3r=b~zg=|!irS=+vG1zlfHoZM)4DU6~pY(g!SDvKv8E>G^ z-90L^cC@b+lIddw1UuObbO>T2t2+=!_Bo&hVs+p$Rp5idI&`UV+5+6GCr_TZ#^HdH z1lf4Ly}g}~QS6038H{wp#G!v zPoDB{MacfP53O_9#N7O_1zZeYE|Jc-cmb-r&OjH>j-U0$_+zwt>TxY?yk|8R2N$k) zhA-1SH96TUnNCG z#&>|F<#T#(l`SauwY`kaH7!RSC0Lu6a#Mw^M+@1Ic7OeN zvE!Pj1pvwO>JLu9^62HWgQ+2~4sNih~xKugomyKI(y1>g8yD}*jWo5p6y{`HND(&{A_NKPm^ zR4ZOSNv(#VRlJYbD|0tkXdrR50!)j#p}yUxKVX!vWy*>mTTb`FnV=6m5h1A05>5hB zLf%Jmm}Fcwde4}){?@zQVCwtp;tu`GZ@Kgl_ujqx4>DVBq2b(iuvw;o4oqj!oNn>) z*!*_pw%^ab+!|qvz^?hRS?`8r;<{e-94w}DnA?`JZJ(}}6G z*z=)_V$g09M)v%x0dK z)`VyfUo4NcMKX@VyIADav&hYj1Lw6J0axdZ84oW4GKxb~f!O2@QbG1~9(u0L4Az*s zHXpWnPMt2WBY_*C*ZHdR8A70M`6xj%@y5(}nS}r-7HJ&Ny>M`7CLMM5osAkp6)lB}W7htz94bRKSFnNOVEM(DQ%10Gz=f%Tmo9MrrLNs$l*L4AFahk&DM3JCl=>u*1w- z7hW4+@24H}sR0hdY2CR$F}v#w!;8wl!H782050DtWexw}!923T#Q8+UmU(OHe9XB5 zB5}w^4qiEr?JFBnXGZYk>{b~`iv}JIF-D+L$3eZ_w;1mXVEc<5=kqRS92&A|K5~gu zuyPdB_2K68PpYcfMCvwQ8jA^T*0%j4Mco@5;9(xv{!mlaeLFCTF;M%c7%+(p4Cku{57`$2V1H_# zVOe84DqeNM_GZ&yl|Y>^v%i4MiK)&!;8NDsLLKLIQNJ$-;XBQ}I5HRBsUmY=KX<%n z-pr!8V?JdJ34(qhg$n8deWi z5!PeG0c<*@KP4~0zf`twc6z!>rr|dd8z>V_A6|;}lf`P+jHM!d-?dAkgZ_=x7c89vFF9{YQA`T}uozo7SF4Xhz>+Qut zLPOEBsCh{KbagrZE!(XoQg>b1n3>Pnm|Aed`nD5)f+8T!j`$5-nSpjGGWW1+I7%7D z06~PW1u2&ab?nbzNXXBa^obIU*=8V3tlW#Zu0O4#n;sHPU8k9|R$4 zMw7Y&QROFRZ(niQrv;vuAqdNycPzF()+xxufIj#USqX@nm)EB%Ty$vKz1*lDV5;my z+*Zpv@x5TM85gC66s}gEP)XD+3FUDRJ$-oP$l!W^y2)g9u*x42Mc-)5zb)`-7FXGRa<9C5Kd)Mf1v*gRVe5H6u9oz1s4f$Qff%U}#(HgHQ ziVXjJ=NUHB{>DTW*saSstgE6DU1`}1{`|NS7jjV_RZC4)@gW+4@QYj>U~9-s@28}VAsY~Z z6TK(R(Xam^-PL}!7H>-;6xrbHIIZMmt~G6SmAJXnYH%F&+DDL!x&Xuzavjn z+D*s=4}WvtL)F&11y59b1E!E_$loWNJ6&ok%|`^!&dHmO^1Cf~BT!8|V3FQ&Ma@YC z%*GqERUSoTr@V;7;c?c?usFQf!nd>i#bgs-AInO~`)nK8&gR}f0=1@)l?Tow*0m{1 zwHEQgRt^iwvQ?cE@u;X^$O3RkxLMyZZ6dlap_8AN#xKGt&j6eC&=QeV4*+h3zN+2e zsuRIVgeoc&6Y97p_pz>}jpGRY!f`OKyxSEH`j8qNSFFwFW`F+#!941~a=X9vHH*;* zMpVH^r%xAP?Ga|(y{Y(vtl=^X-8_CA+De{~aDgYJqyx0^4){Evxv$24qG^wF*VzF` z+I@U4&ZKOi$mx{m%AO*OXJjFVE8d_RX8Ag}-n>c$esP3}9+4lQI`}8gj?PY^O1^8g zy263={~0JhjhIhkfS5Jh?r>;wcv)vC&v=Am>UBclpl(vW(R=iL5fv>>1x3)ya_Kvc)*;TNpLpp-M`R zLnO9AzYDdGq;sx3&_kI^kZBxd!tWF78wL)djqUU53wVZ`=^GnY_i5H0?0`K!04h?y z`vhiR1MlzepXMN2ycmBu|1kf@FY(Rllic$KlXDtO6-=72G%^saffn5VshZ*AZd-=M z{!z~C*;C6X z`T}fMT569CEd#yHJ)pIDq+_;a0j6l?0n+}ulJ)ANDgp?s_r zat{j&gV_}7PT{O%-BQ=oknC`32x_@GoU(msK8vqPh?Jo5sNc1^S zOgmCJ#H&ZOvfn5AD^~i}Q7O z(N}-ItS`&KzQ)ctUgCSs<6X|@JJU92WqhYw|LNyQXa4N^q2s%ik8>r66H2J&ca2qu z%F2gi&=YfKT398NeB{pBmS034!M*~iQQ)TBnjjQ8MGC_CWA#79=frtSbkOrMe(Ktw z?`s=$E9Bu9DjN*$Q;>B30<^XN`xL2DwlE#q3mM>0W@cr%5umFt`~gJ{dHkcHw-?`e zexS_engH7ZRXa;mbNG(A^7T*X+JPL=WczXdvWz~*Si;U?7lyVU&wtjdxf0->b3oZ< zPPV6EPR?W+Hk+8_P9$Vs(Pq4TRo1+%vpo0#Bu9vGm`j_>*$HQUq8EcSJG zC(O*uBs}b&srPc5bC?fRa2?vmZ3^5oGXXd(6%QvHWH*ch5D)$%+I*9~QEes{rll{W zS5l%dua5wvPWi2+`yC34@2`70C>ubpKb!+~&oIz96uTm_BXDl>y!G}C zl##NQ>K-1ARbPWLHs7Zczi)Oa$l27+WDz-7 zHDEJUCxojze7MBy?>@l6U0{bMngLW^toBi1}@-duBP=5O%=4W&)0Tzr3Nraf%jm_ANk%-l7=n5k!nP)e3iC!LOg zjc)Pbd|JV;day0+XN1PN02+ja-=mG&B{4BE;ak0KydxnRdbO*DgTtxR(KF}>GnDfA zQx*ZR^Tgo})pv!9-saREJs9ORq}6b2ESDl9cnoEMeS?g*#! zoSgfu9JX&~5su%!PoL~|BkS+s@Hp7W=rOU*C$>(U?0sw$=e5PTYw}~gnTs4YER{pt zIBEz^$Q%z|WY58nz2&h;U@qVm$%D}qGfxJW%JZ4ThCwlGS^(BJ{mdS)71(#Z%8L=s zcpJ!8d7fS^86RsTqj2$Cq?t;rVAWc*glKhSJteaMaP(*y!E;DS%ge5(j8G(0Xy=Q7 zQR<{^^gI|BLfagf<|Q+pC{Nkg*qENJ0YiqFFK!6a_tMe|0s;ceJi;gf%<~?Fw}1cs z9q&URZ{L~%Aiftq(MwfKJ3bqD3YbtOY`R~JAB4xplcQm%`?Lt5O)2DVcDZ_-uwMlJ zKun@E_g{USjIKLaM$Ba~@t~oa5BVzjJ;2@~lb`F*6V7(T%5UeXz*Pfqbc0AsVY9=b zFJpg?<`56ws7N-Epwh%MDo!1$$ae1w^C1}k!*u;s_Uj{T!jSRc>90h|c=n>KzTBXD zfg)?8s+A9n83+5ueB?BTM8z!nLUiY2>i(-;5ZuGWAwR9FW{=z0+zdxVfG_MMFd=jU znm^4z?11iy&_MZ~fNik?%g-*Bh>jf(S3E1&18MFUI!}6!dLb;A*4OXLzd3Z`t&Et8 z_qtQ9uTf1B60TeB>|AZ)9B|?FNBn~=ZqMK#;hF^uMxp4>a$&(w21U+L$2ipRof>Q^ z%q$j!5d*mJYoJO_1A@tVWlUVHe1tPQ4i-5ZVr~4Qdr}g8R0CXwhq{?{`g8@?%!t1b zE)~LN`z*kSxpEJ-PTJS)J*7&Lb_S%hTH^U?R4oN}hOL9m2$8}O&Ny|}mtvsko_C$i zObcC#m@GzLT%fxLGgCW8=W`Cyo=1UXPY%WIlJ88Z04E11D8UrJPVJ;bEgZ1b)Odv| z-wkLFJn96)$kN-sN1FSZ+m2I?G3R9_=Y_@Nm^AXD_J-!l@ng~J3X9wW!)zoWW7z>5SJT)ACfoHKhW;qbb}1bl z60bNR3{4saAYxvqSyW_X-#OzuyVkJhgUZaG!oqWtepJrzxw~GaMb&ZFD zHc2`=U&fcMnk+h^fSR{Aah=-ESQP#yR13qT~AjpHS{a34QTq14eat8$75}3 zNGp6MTV4YK5o@KERY&>l^1)+MyOve_RaQJte*(yth{nb)7np__6`4!zPhf-*Cb8Fz z5uk{>b>2egXjA^qh)}}jaoXV_2XAqq1k*ZwA+=AWfKXb@?q9DBFY;@U==_(=u#Q~b z5Dur5HhOWtbl^Rc{vzK|sCwVHT_IL33>D2Gz5~Sij=4j9TVN=iHnN~fe)*+ay%+6$ z?g7AJD*kIP25mGTmOylMbaH}hUE{`NQWXxuFeGCO`mzP=^?O^@f9PJ3`Q5nn=_UB$ zTUiraUy}1J6k{DVeI&ZRssMWk;1FeDE0w2TM-21&qS^TWY9?<{iaafl2dQNk58rOq z^XXo&^SnuclAH9O5K39@1)%$n$b81%>#e?9m0u&s)~F;VEhVdOUgvr;6MYpVeXEFe zC+A?{V5r<`TRG5JnR$lY3_z_#DO}RrhK4}S61Ht9_gB|_HUx`Ckj39O4)_Ws3G>`= zNx5VlPv|F#cD8rw=~&C(PL?^IfQvYSQn`Hn<&CM&fbH3X6sxkz6B~FJDON)85QwU< z*F86{x!#F*j|yP30D_CH?#YTK6&lplkev(e8&!?WGLkDB)kikGIjJlF3fR}d)eofH zVH%_ouhM>7142fhBj@3MyA zlAF{!*IlD2a904X5q<3Tawc;kCwY63j|L+`k&{}~K-;|VwZa2`j4>hoW&1sJU|w|n z*SDi%(SbX6p?>k@H4Vkb7|UHGphF1XY5~h1&r8dn&!powr-Ou5fz6L#fCJ9elTXFW z!;-R&)cSQ>Ej z$44|*Y7O`I;1PJH^2M604GokCsIam8`eNX%CKHt-sXa3K1=+B1|7i3pFXw-wP&R0{ z#4jn|uLfLkNFpOXPyPADb=S^4m*gf>RE5`B%2Zq{eNGM|YP8?F`I8MX(W7r~BXo`8jSukiOF zaN~Jl+(3n^SO9L|&kRC`gdvHwCs7{_Hq)!iyEEp1FBu=7(HS+q+>hAI@Pn|`V#iSW z4oMZ%kptw))5pb(1_HY97a_mc=18NMpnsPZ-xtHpvw(6|QOP;Dc>b!}pLzm|#=INhCD0T;$T!}hw579&z zBYqoZJbJ#3go6LqIwm)$yyKqtNRSZuA}l@x{ncJW9nm&bo~1zVVCVVBkrTAAqN z!$8uu5cg?f*b+CHT-H>I{5UbPkIi5qBm@`XlXTCB zu4V9i)HOjz!-)pugI~5+OZf($+TVLqhHGS4B5pZ=5jdG;&EOGhl9KCd-dog_8_rWv zCCr_q{huFASXtA+i3lAYM;Mgo&!!GYO90aoNxiT) zO1IN^*?_u(1zPqPgu>v3kt^{Xm*Ey9=rP-#UwYX$S>2qqJ-yYI5gmbdxpE>!MQj^>m<@>HSuYC3FP87j$bGE>ER-1Bu6xX^w}SgvSJucC z7N>tpDLKP8Us`3hfFJ#f_Bkt~l2^&YuE*fMeQoQro zT~NZwzrhO6Sq6?(+*Ymn)YdqDS>IrS9=3X;nvT@gDl0XK9OA^jtf4{Re+~VxV|azk zjSiV`gRDG>>i$rnv%TIZHBn?hw`@J9BccpEs2Y!ZChw9C3p3*}ShSk;YPwDV zdr7&=HyuNyd}ntam;^W%tDMzCo{+@JnmLK&J?oLVXsx>^N1z21$Y$D14Taal?niju z5?z@?xxt4VX2qgV=u)?Tm_W+tUQm%N5RMS8?l}?wuYyNZy1S@)t;U9W$GUA9c?oxI z8qUDlna40fTONhJ5k*WITVrcR;g{QLp0b%05(3X&E~DUR+9X1bgf1H-1YY zf(DE}2l!uQ@DbCl>2xx6S}iGC)O`L4F?u#^6^ws&_%`K%5|AqWYxwS(Tzcz<=~%)r$D{46)CS)sfQE>`fwu~=B;2VHamUsAghM~ zGOB*bfGK^5_Hqk_MH7bESdXjd11{a?kn>ZP!u>nk1~Ux+A=o{ENFHyfR_N#wIGBh_ z#;K1%GcNi2l88PT{55|-1?eI7g;R1u!ECXcnmGu})IyeuA?qD%gTWX4Les=oLjL#K z+h~=o4I!ZwaRtQOHI?0o{0>vuo>>$7OJg@1o&s^7`EM>omYMn3%3k3YOi_SAQp+Ux zr&^L3uK;V_GAV-CHPm~*W*JZZI+yJ3_~V--ueqk>t%iaF#;%rh=% zi4EKSws)?)kcQmvQ~k@KG^s|Hoo)^P-lMLrPt~xRH_1rxIhIXnR;4Jvd5yd9l%k-}%8#zzM!oau-^PJ}ZfCX!lSdY?=@WUd&$}TPfZ(<_!eLRU0F7Ji=2YPf0)Ced zi7%UB|J$X}P^RCvudS?9YZ;^tTft;y25@p1)C8DkW*R0z_dN&tf@t56sF?KE6>);h zc-v5K+!!*XFeXiqjQ6E(V7{xdeJ#8E4sAd0>%`+|$K$~$4z;51l)Q`|mNVd^+PpOC zWB-{BzHnaTxqUV7V!C|y!moDx!b=0Du~308;c`XIx8Otj3QKv8?e9O_6M4a2tJ5fq z>GUPs^e#{ny`6oE`QkDIWPH9j|E*^*-apB2tSNEd-;v9yOk5%AI5adUJ9}4aeNqwk zb@l$d_f@av@3nRWDAQSnio~4*TEj;7P~^S=Ju9olsG35$!hYDQsajCM=im@Mc6&R> z+P^<+)7D906(IlV2F{%_y9%43XVp4GJcP2bsFi$(TDR`F#{Fk-# zMS}kLm6)^-`Bt=9<(0ez9wKjk0!lS`p+t6s_eh4ypbNO?zP=b#C{ys7y-r^2_JLeI zH0ouUqxqgF`ots|!GVT{^^sLrez-=syq9*zejDWy8f~eWt+%Ha~uuJ8BLgFeU6CGp;CIb7+2RoS@w~$y-|R|1Aemy{ynseD7GErXAw_@T&tjRJ7?f^lUOz3kMkG6)UGXK;YatzO?VBYks+%9ejG^oL&t z9_LG1>rvq2m2%ghis@J5xm(efapYRknL{ZZ3W&jUDent+-|kLv)}EWP#|(Uo6C{1} z@sc47d{pOWa;LvAm=B86vq4@R7hB8bsdXzk{abtedNnJ@QRWuAQ#D&nls&Z`da`Ai z6PeC%8nTc-yi#omkLC7XG?r4YH`O9s(Uny$=oSwF)l^>h*HC!pkeU$ZO1(;@pjSNUXx>sApB;B?Ua7>i-ZF`F#F5x#{xTotoIGr5$ryQFz=oE{ zfE8A{HBJ^K^`S+JlWRwt1?DH{1e1jvw372g z{!FQQ*-M@BfCD{?69=ft{#p<$PaB zFP&uJIb=gPv5&VR$KNXow?n^(RFZ)wmDyec59>E_ryQZPboEnwkL7Cmh*n3TOvxt<=`~bNYQ&oU=ESb?n;rT2o=2 zH1U)5n67l`*Y88GC-p5)>I5f&qKKPuV3Hl z2P@__A1n>J*kuGjOlGpBE1jQH&{y49Uikw+0AiRb4m`W|H!0I;l2T0yLYqVQ+N3-QEk>eNtF^Y_w^>*OaT}G z+jUT_sL=8fT@g1jO|3Z5;=eV$MLIq`!U45*GO4$IE_;c}o%{q2()Hg^v8x{Ov%0Ag zU69j9{bx=Ssl<~XrG>4g*Roo0klKGpCMWwZ`Dbk|a>kDp;rY|2bIX_u#i@~?vnYQw zy`HO+)3Z{0rv%v|Gt4k0K%oibST@Q|`^8^Sh$fW+Pm`{fuE4)No&(Q?X#lOTOg;Iq zd}ociv1RMB2c1xkD8E?HJFYDa^ewO%FLB<`HJYB;{X8D3CM z0T@Z9%mXk%+%r9rL67j%4hyPKJUqjYX4%)y)PB7>Pf98-WS6pSFEV zE|!-`(KWSlzr404M|HQwcytxlR=s*bx(?8z&?o5Lo_rDgpML|D54ylZYarSi2f-21 zYjp$CeIz#lNI6%YJ=k6LvH@#Oy&AXKnp9N$b$PN9g$9(|}MFjs|Uf=D?(Xd_vtO_ zTE#=Jj?9D_lkK4DQserj%!yb!yKFC0|K9GJEnd4)7k;)(V8LIPGylw|6?J$r)RoT$ zBmiVMP5gmGl$ z`7RP|wWwt|Q!9MSFO_xo^dEZ8U&G`kSLq_Rgqn$WP%GE8vc7zJFz{z!HJSqMZZtLS z)}of(-Wi~=6v!GEMO#O1CXN-;HsTZy8gsaYX1@il48UEmW$@A+ z8+1JT#jGs4{~}}@QNPrN=|OWk=*iWWx@@~V%;w#TSt)k{Z*)bE=$ej<9p7LmHvA$Y zN+gc^n|;R*2x+**oP0sbfA4wf@Agi5liFIfY@Z@|xG(x~hEEaG4OKgxYZ1`{zNvRB z?YcV$6dtzw!?#eQin&siHaPJO^+M2_~2{}~LG$k-eJj%8aDtPq(6$khI7pjMkaL_=76Y9yOzK4cWy zg4=^T96WJ(l06Tbt0=g-mm(;4&*}3x&sj!{e4o_lKaZPXRvLwBSOpG0XDy5NwXe1k z-SIO6K?CGlo=jzKDRQ`vUhP9_HS`J6MWv=KhhXdg;*YKhaFH(^00H_yUprmli0B&( zr)p$!{&<$Do8MMLC*UAZY=1vBoRnkw9>aC`(Q~SU30ua1$tQ)oHwM!RCC-}-MYFt@ zC%qGunWPpW=WA{2Tc8kY4U~2@q7yGoaNFfavQ;;q9eSB(i@E~< z&YmS&>A3U_UIbpOEvCa*6073IocHbPFuwXW zN>rI5WHt#nA;Uj|jAlSI=7yW6wVyjHou0vM&aOp!jB%pPo7`ReFC*zE_s!ou{%AA| zVxIWhet*G(E}#XU{0jqBrd=}J| zPF{8E49CFx^Nmq`z(m&gv>4C%=KL5xTh%wkC)R+Sfm!t$1608ZaOmb1a5Ckeyma8= z6TIGCt3&-q#{-)#_bZmxTfVw}8vA`2?z<5`1E;30%t{ljYy`$Mgtf;-w;U~We!EO~ z85kLUgC^G{Re1=CXkg;MI}Hf{>y~U$Hz~9X!v2nUf&U_&(|^1dy!6FiR;+Kydc)ae z&mI>0(>`Fg{vsgS;XMZSHsp&$jD>2~+Ina{^?JKehnZi;_5U1ad4OAjxg z5C9Wn;B^R@_G>Ha5SnGDztSM}UmEQFhX#OBp;-;k`7^(Uw?3i+?3bJ14Rija_D*y4 z{a6|)AG{39H3r=Kv$_R-8UT7K*8j45@DBy3MkUz6{lC$MUIU;F5Crhg1Ohew+`Zrj zb8~H3G@BK5{1AvFWyYs&^{om#YVaUK=aYvg6FL(Rjwmtv3t-Tq+gCGkD!=qzc zRUiMuKSeVB-9Lb1DuC~OS(I71%Y=`TfQV%*=>OtHh`h%y`+Hx*xJJH6B~#-0O=7An z;5(#)!r6Z@qJhN$ckj6RUwRQQ5dTAr{~^Zzfnfh58UL*+{zo$Y-;xXgZ8>0C$yN(? z+sV)(&=b3=rZ_nuy^?_ex-AXx7 zU_67%mY~I{CKlDBFjm%#ne+lW1mMMG4Mi@f3aeEcla_$f1?jJBYvur!qd*!#w1lL3 zFbaC0taazr$kSA*jgqR#D22lu%z!r#+SL@$#jB~fgTJb&US1Iyn*}EetTFaCzv4t$ z<8}j}hc*)dKh%mzyujlHdExJLQT+(=D!#FDi`g0M%rRWDW$*1ntD9nc@tgO5)JFZq z9gG8qWHx%ix6p9O3+HiD4;^efQw!3yeks{Xg|3S4XR6eCVQyp)d`}*j2PlS*o2HR1 z?Qp-kLd?LM1q`0>8k9e-)(cpm-&*9v`Ivlrq0u-O3Yr8ucVvHlG=e)%>^2giTxR|j zW&|uM<<@_6WxY4;WxckYf!37+5fkf;vC=J0+#V+yKd7NbY(Ofv)VRh`Kv$dO9l3lj zu=2z9!zn5h+oTK2%hp@;6)+j=Xtu_1ucjh(-<#LPrV@Z=}3l3|X3-8M|- zPC!8*7Cp;xS;}0A$`%xTTI`>n{;+Nfj-K4Aot+sgc`o+!tDfvu_53*_R$7?lVP6P` zK4Ct|;K$oS0&*;sRx-EQkpVI9)DG`0^)Ur2IasN2_IC2yZ)P)A^J+Ejn0zy#2ECFx zjiHgUCcY|kxOCNPYwG@Lu-mSz%scTBmD9*uUhhs9Xr$tZONa)RC0!=P=~**@96;Db zxmlx^DpAxP&9@C2W)R_ZR{0@P!Dm^ofcYTQm#t2xVUzEAuON0KC7Za7Q-8;1d-$`23=_`o zPVWSH?}9G3POT5QtiHTGi72q@qo#ip-0b8%<=&(1d>CWcWL<#+#_2%2Ikl0uSDrqw z)Fn(3n+-gk4rA<81=_pk=cvsOKQr~)b4wp&ob30J-+XLbhr7f^u_+HY`+8IksW6#A zl#Cz$u29s`t^wMMyvCl}Q4+l@E_Vq<>P&wH0!p4Ka{K{O6K92Y0zcI=ixBZ|CH{}MNba1*-&@WN*u|3oU z6Gx^!3~1AloCd@sP2~rtQBWk~Nf;!r+lVhyiPG61wNP$;Ar{Q>dbze~Nk~lkAfSe8 zEYTljZ?CGB{5T7eN*@#ictLQiL2ZCos3jN?_T7IHZ+R^LHV zA9&-nz;W|VpjbRF`{PBgrPp#%>8(^%QGQV)HVF!o3VuL-ZheMYia0F zO=;O1_F9T6XG@kCNz<{%K=}FVD`yJrxwb2c%FoHscA6(}Noo;K}M}pykFWrxVzJqVKBEQ!DGPAJ<*h66I3d1IicL*??3Es6%>|jxZMQNX1^# zEfZ7GhOEcwSS`>H^t&<&0Z#{9$IvaeQquyi6cz3FP1_;7!#ZT%z;?pT2Y7hgSJS73 zjPZ`uq*m=5Dd3BT-Wp_zY+c|1-dt#Mm0o|S2a~Y@k{E~1RXtrQuxkAE23P6HhkAf} z`EXF&-_#VqQ6tlFK{fvqeBr!sbw<*i9r&zCM~tMb1u_1X#7wuN>hRvO5zWujL=`3AO_f*gyMNO> zNoD>3df!17;*NJx>nw8EW*^WN4xoTAc8pJpohkW8#(z?o#|1ww8Vlf;#;_JJSchM? zX}Na1Qw}~BjqE%KT*UvJd-J_TZ`0e_akKY(X(L*V=nC_&^HH3x0AS%}T6QDA)s(Ed zJJc;f(8lxqKJ@-lUsROuQV7Dv_w@DSsCP;Z^Lss&K@D03-Xsrh^1MP@oWVclHIi?OmvNZIr2qkE8VH=#$}5^dVr?gX4CuSx3M3U9F(K(3Iza5VFeB> zD)%@0@ahpL7O-t9s8g>9J3YCZ$XBy~KU5k@9YCoN_2E^TG zw~Z7~e`T-mn$FksP4oLLXY9Q8y_abpdRp^K^qJvTdi4<5{WsbljEq0V5&$l|${90?Te6lq^@uwwb&( zg^dr$1BlQCBpN=|BIJhBD@quN-Ws=ml?8 zo#Kryd&{lKny?Gj@12-6E)z0s8GuJ#3JXen`z;8ayKkHd0ewHc9uo7`KF9i2@iinN zPUJWFH24Ld8SST;aUjlfJh}4o;ao8J=6-aR&o94_3=N^tA$hHg9|ZLMq>O?g_EweV z^=!vEe2)cQ7-U&q zH50Ppvjd`#BM*WVQKKNwYD>QNpG~VpgbWo|q%~pHkNad)K)I0p9RQymz5nj?mr&n- zI~uK4c+{~{%K-l+vGAXu93FibqMQDfv-)6oBX_b)g9fBqpiQ(8ah zDNJzqZ&v2WpRMNVpc_1$(3Li7XkZY9N&V}WCY9DIvfFiwlN$V)zjWpHI=z1BtJN43 zKdX7!m%QaYdul@_$BLIzM-#a7jmgCXMFkX}$;)g#`J<~OGlO9$;)`pugTSoV@ z16OmRBA1P_0kb$~bk?D_WgPIHR-?b1`vdc2hephw9)ONl8o*V&URw0d7Jzrzi$obr zr%V)K4~Nt^=da0*C4j4+`KLiWWl{g|5i^Pi@( zNwX2HEo8`RxcN(ksl`Si*9|`_w;oRD`5Zrr%|g`wU}7RDL_WpB5({zvDcbY1^Ii}J zD}50PM$z*Rh1ue+-x^{yJ8?cGX(>5WrSJ1IxrbFcICx6`N>NDm$XKFAQNUT-rqM3L zUDbcvBU^TAP))G|?7SXii(1J|IE1=6CU_&16veqPcMz`@Ch5@{PI@fj20?AJc5X{d zY#vMG0hE+)Hr97AT15FmmHKOr!Y=)nS?6+@#Hhh|*K9Pl`Bxy~wW)8IzxTAwkX1pt zX-Hu7hxJQ&SqzK>ZgEy5rM>wwJJ+~^^Vqq(!@IQ8Ph_0z9`*0+QAA79B3-qJWpv&)(o zP{f^k0jGD@)J(t6MA2WRx@AJfCQKNLqcGYA)Lgg1grny#3OMTPm1V$Wcg5R#lh5rk z?=uJhGe9L*tjH_QNGAc`Z``$MuRCYj3+=LnLA!Crco=NbJayQ-YzKjz@pCRn*^yzE zPMBq8w({^rU>d0n7SKZ?Y#YF2zG*I(&b-*xAG(L#28lrpdQq84rEVWfV z5}EgK0VvN=C9m6QG)$1#i{(iM7rxbbHuV|^U{!kz%6qQ{ydE4?m5oCcz3sMFI`3ez z-(C`*3(_N!d2In`eJU1Y+LCPinhdHD`?Vi3Bcs5AL?M>tP>SrUK1@&DVunc+98;ip;b33=4e)}~QfDrv0=*%*pzM6|Qfr=;P(JZ4$_A;6!x1~{8!S1| z_10tLhSMsk=v62)#E3~%)5^GRczz{{p8=E zivk8PRQCxtmNc*FxcM2oms#IBy=s!=|Iq?9b$-z z+8r%Z50^+L0cW%c1I!BXdGlckQ8Tn)=n#@Q9O^G2v`ZckjfG#FvNhHyO88?Y>;~Jd zqebJRacv>))-w4&Le_Q{nxt+FXeGj`o?Tf!4lU=QQ2TvP_dJC{^NvIFd^hW)=eW5f z=HO=BIshnf?7Elrgc;J zJDrFIArG?*oI&1vKQZMRdEMNbX|(FrjDqL8iJrkAOjQo~0*`tO*Y)tVS9DL;xSpu{ z;ypTsCp-%=fm*hlkf1C_!p(hPZb}ytMJF#Xc3YEdHA+7(@Hb~;k$#2qX!E4JiTi{3 zJ_*P2AYmbO5&S?bs@Zw0_~ac9#4fg7E_93dyx!5n<L1!b` zRP3Y%MmbJ??~IV41J7lUt6O;=-fAjJ8#8}jR#bt>oL&zs?)a$FFRZ-^d7Z%6xGFNt z(`S(PjwjXxW$7?yh&UUy#L>Oz6{W1OVS!Z+UqKfT5Q(D@x)W^}e`^K5oidxxrTe7S z8$N1wHtN3}g5l3&j5jDmV)d=C z{lu~T<#wuP({MOuIHd)GB&Ia1v@X*vlfg9UauP^9_d4ryHAHOp%QsOp3(6e->K2V?x)Cqjy$=<}tFiwk7-An-(9K4k%*+0thLu5vAIml4N#^x6TMk zEI6ItO}RZi$&XFxdFuQkTgEY_Pf~wYi+i3VOA>}Sf44Lz@TuF2xBjue#-=1fp ziI)P#u(e@AGvJlPcyhy7s*FkKa|FQ_3dbK2U5R)k-AW_%nwS{pdOnGdDPVw!)-TAK zAPnToO|NgnK!0URdQtwyM0! zU0Z#RnwcTOo-*)lmdiv1=)Mlvsu`yuNSce^;z?T^ zWo7=ik#8yA1ief$Wn`>L_7lUuDely2SlcU$B|dA>+KnK=(`3}Q{BKTn zWls$BwQW`;yzbZzG2ETnn9kNB3Lv0Fxj6)1ILTN$C>8667^uUDSlq7}y99)&nuv1^ z6cMAnQ#krA1L5MW7>h~SdWpkFQxblzHMpk8)=M%Z36=?bu^Xkxb{_S<&~~blzeU-9 zwAj4H(5l@ZLKR&L`7sbjNJ>K85pPTX)T5Q??!~c9^(%k3Zm{ zL+H$A$_(Wk48;E3n#YA%lmqeDsNmAGT7)}`3C+s&RAsH+miZ&)#v6mj$luKpl?>Pd zrp$`-(0C(E`9$_$6?Pv)F)t45sJ70vIonD0!g3ygUf*C5vHxm)3!6*8-nFErY4SRG z9~8s630-{7>aSfbOUgneg-i1z+MTR~gv{0g+QN;RpnPa*JKQf{eh2A7-f(AkqX~vTnfB&d{ zHBi6u2;|A#bACS}S`?uq-9#gmNKAam7MiD$eE5_;g!Dc z%H+qu@Lxrql^B=g5gHou1!Al8A#FO&%tAc(lQyOxLEU$JJZ28}@Ocg72dGmbN3^0@ zsa#QcPt$(kyj(zexSDlt^QsEiaVKC!7X1lZLY;YalKHfPeWBwBc4X z5Oh{1QmyV``eo3+5kg~GK;i9-GnqXbn^Wvm-0YRlytp&gZ;&c-FPRTjh81&R*4b#L z=-FO^D4U44!L^udONYY=Rl3wYaT5MvfT!$BVz#T6FDdf{bS5Of&b&B#A|Isu!R-+f zsWs(?3v;-k7#}JxUKhiiHHN}&zMb}^_AmZy+@*Sct#J&p`%T3cjI}FX@WUwnruU!e zfwM;iRzlsG6dw}h28DdFZw(TZxYs@f94oNwQ_Ye&5LF3n0fI3;9hSKL=owbom4=I| zK`})=QUZA4qIuSNJ~;5}1u*%r$5O0@IC1Ij{bSctArr&8WPSk&L*@3_iB}UPh!n=o zJLe&6P1_S!*M|spNV7T`o;7zl4EG5k*Y}&BP#D$%NlhXTdQYW z0hM{jEKp>SQ&Bx91uV9_#x3aJlbA7*mhbdg>Z@xk7Zdz%lNi~=gjOFdL+x|oefP|o z+4gdH9*t_UUl2drh+J)vXy?k3M56{s)3ZqMi_j(&hd~7@t6fk&~@n$35F5%s8`wQo?MPL z=ZhOM95jFlmp~|3+oSDL=Vx};6(2Ckq(fwq!0w71WCZ#C3S#4_5A z=AcTWrLC0@^CMD=$t_@!l~iX{P6#pQ!~TJQ&zeVv%35~m zrhurOMJ92q=e31uAQfhuii}=U_z~Ypvpu+wS1)lH#@Li1bUNWRdvSckX65bN8C3zz z%!522S^#4T#OlroOOe3?R3v6gmI#4P`Eh30h*xAi^?zGI0Lg zIDsn$`+AyG`PB{yGOTCZck7&rep0V8q$=lSta<# z%DUlwWt+9uJc9IcsEZK95DUYG(qPeq1CGWi`*VLdBZ^WOoxBN+)MTkEtSbv<3=6wH0DZa1$K+t!}GcZ zL*Ye59vl|H*C2OoK8wrO*vn(YEIzB5GwSgyPAZT6 zt%AaF{U9ZaXwhT(7ejAakqY-UY8S%e33KP}_w3=*N6kw@ZZ`_TZHV*(dMW#&j3&VA*Q+$H6BUuo&&tT1QXCt&REy|SOTSzU2#e>cBsVi!j}(^38jW=fjAS!g z_4y3SED+DBe4bvIHV$cCW=s#xI|9bP<3<(6U3{Th(v`j%jp64{vd?}Ye`Xcdyu8jLu59QM zDw1t=tYG6Sj|wJZFT7}HiO`1z?;C4U>-9y zWjGCrli>Kt@jw0hIuzGrJmZ{4A(0J|8^386oeWYR7v{3+xVAES5+_*{rmD3yNbKYT z9GWlIU|kF8r3o_%>?dD)>6$5zu(zStKa`918>r$ z4jF2HBqaMY*)cMM>N)yxFlvOnnEuJ$g968kmQ4$OuHaOr+EZa5v^w2IcM0eq)w0DxPqk4^#9rQzD4O^&;{Y?h&KvW>p)D?_!7kQhbL7&#t=UY$igh$J z?K5CwPz?;ET|c>_HFn& zjoK3mN&O0e5k2oIUABvEpw%a=GQQ)OgTIgYVxelxHj8#k!zGM&zS89 zan9Kpi+uZ9ZFw6p>9zgsT3eY_jD%1Ckcr9|3Z>PpZuQavsVdzdf0>tbgY}RrCx@gI zJ{-CO7LvNu7h=9~zbO06+5DlU1i`V>CmwjB2mA+>=iyVecl!R-Tth;ot9AJV(GUk% z48e6t-x@H>E?d$*>1NW^%_{LZdDTeljMZ<~@wVIY6XxSvv#VW`Q`r0L#`y$6T?)xWRG3&*I@RQz@3%FL7Q(g~3LFhmk@brTCAYPADPss0{2Lzs%#9tw zO3J_a@Uz`=Cp~aDLYcJ&lGdh2eJs^m)odT?R-xC@W+{Np%lzgJh#Rup8aA-74+xs0-I%{otIuQHIijdK>bPoK55L4{g?gPQZ22U@jvL#r}yGHv;hC(%W$Z2_&LE_g8@^OM57KL zJlDyUYsKFoju!|i3I+XEI2%f!(s{>sS`oxsV0k-?#p<44MMl(ZzeHB@cVAMio>JH) z{cq-ziHf^f{p&4{AuONF>g|u(W74-}XnI(`xq`;K{0{Yddzme+1+7u$JR~yiB*tG% zSmigk%~c;hA3T2u)QTW^RBH}(D{myG!HebVPkVESloCISJzFRrC8H0qhSQ{i2_!C2 zUs|fWI})y8`UiLypa+srF`iaJ-%M2*69?tbrpE6|1kCM|N`6x9Pn7b?G8FJC<&!+A zxPA=T943(8rLCT8#+@YKP{73aO9%BY4pvWX@*~*M-q(5?s6?OioaX2YrEUSy+N|hS zOTB5`3Hax*XVjNTg@z%pDqs{MHEo}_cOBGT#+QTh!k@i~tr z>XySLa1HpPCnYle3Jdat4WpM*Gb%cg%#(SQ=U_8Cd!D*_y}ER8={~V7eOBGl1cW_TJ`WvQN}h+22j!kWSSzizosCZsjteg!Z&BuF(m`qf0dFB~ zW~Blt%REVwq&^nP?aQ4O3l)*DNQh&4>74vbF|7Y`Iu;0HQO}Hk?8`U2?(h{5V2ewg z4W{&nT0UJIfMV#xxe9D?=>tYAuu>qa`SEJeoYm|S69efIS^9u6ru-IsKYYliDE@Z& zh+6A6b!*Z}@BVn(gZo%*Q|m7ovz0oh!JkcpavqW9R?RbfaA*@_*90xlwkSdMy43Yz zUs~=QJJ*W{2_!y^%Y&(cTB`Pmd$d8dn9pKD-#?vlOUN6)5)}GQsR^oX=R|R`k93$` zJR7kiU-a(=Y8iHFdf!L1(%yP}>#2|vOvL?Y)KK!4jXWf!igwlGT1M}tj`i+`?903h zAZ(gJ{*8g}X@d?E&$@Y|74EE7&B^yKN9r8skuN2P7Siw6t( z(N&$WFYh&fyuBmpVj#jN(70Yy9Le{NzV0pGy&4Pl-kQt$pX#b?ZMjdLZ-@a# zSs?AZ4A8D;Dof=fR_Do%@`*Z_*&-3@)BU)$Rx%zhTnKf5rO!_Eg&03X{+D{;(9R-J z$zD-aZ8euPjN~$mYPC)Mw()L@V0qJPAP}2(T|UP`VfeNQ_L+#}n@uU1g*OWsNu=pF8HZqJFADU@L%cc}h>*y12u@uz6S1v&?1 zdI@t5IP{+ohr&E+FI^GVT)lp$L?qV$vJPL-G2owARk-*Bw^yp-lmM^U;($p5Rnbc; zH@RA@4k+AX=+aN->zk!s>FzL;(&jrG>MJR`dRet!VBT*t{d5))e8YD_o<7IAnIOFU zgEH<#KKWZ=7tHaOzS&>G(2=E%dybGC5oLPY?0pU34}H;p*1it(6(5g;4_ITxht2Fn zb(m{D2@#)MAt9iizHg&`ji=>?kdMKNZSyx?t?mkL(YS6eO1h$(qO+E*9N6)%e(llx z)%1E&?yFOoI@56um>sDf@o(=kvEJtUXd%+7K`hJp&M$KSuRcZmR9reTWa)#Hp;;^!`ycqj**s5X0NTsxR{jchd}2mED<@c%R1J^ z5tm0aCu_t{QjvR}UDplcx=xq4zQp2nyJ`@bBeBxxsm!xr@aKRjR1x=>W@Q>TU`>b>f#wT;6ic+}| zLqq;Y&4~=r_oo35>1uT=6iFbDydvb2{UVH!=RQOvg%Kh!5~7-0&Go#1NT*HMge;D( zFrOUtNte5SRY`@k+?riNlPEo&e=p|!w}hkj^9dL#qg6`) zI0T+LaFgig6yW)I^)4;%)z>&LxpZAi7NoQi5sOYGPIFAnThAo9G>71pjf0Y!vQURu zimR$(HdGHxaLelYGAd6hZwxcG<~GJ>a*yU=&sqxwy)`tb4aU2ZL+(zmW53-45jAWx zwt8oQTul`21dF#xN-3nnB-4?dX5#bA$4gA6`dYt{;R6flM+Njnwmy1027_r#1YSjt zH#vR;^&{PJcIEe(I|iMrtZ%Vb%NR_2y-p%OK8jkwj8rudw3>@$Tf04$N zi|7ZnH3Lv8ziAPWCzgC6yBzZtbp)JN$UUW%Ja)UkY53U^dCR!}^`}3;0I!5O7UA{) z_qaZA(}vS+b{)RNQM%Sgi><2_ylD?<#V}-E?-R#-eZ!|}STVxDXYcS7WGjyh^(m+p zF%;EN4rM&N-s=%qFiLVlbXjQ4qr#vN4F$Ya+r1%E77w{3{x^H}ZKp~2!ZA0XKumOI zoL`LzpAif0{eCCD^!YYlB{QVbuyF8h|)i)QM zxJ4+?T^cBwqteF^uq0@4LnfNUk5Dc8U1-!5ArrFAgy7GeV;m<{yzG)P4<}$Rlx+$U zLxyg+IHxi~Ufj4*T-gjgiGniAz#JJyO;uzRf*Iq!Cr#O_vbE$o%?n-lYl0}~>FDv1 zjXXPxPgDdlNFL32wUxW<_KEmPWlg1(54K}rv?$eQpOm?+O7%l#EHQWT5H7XlYy+&?RMBSq&SJIB&J zKQ2$h>*Fqh@j;d(Y2c7kdQApl54KIt!*22la>wLPhE`ix*kexZ!si-b> znF7G5M3Q$({xqmq)`HXxM^J&&VI;mGyr0f9g;z@fc;r?Ex(;T$3x*={UpPEi`7pE~ zp7RQDJg>o#A?48#3Ly~JxJwK(sM`l#jl0e&4e(#*1$L6nrb~5!XXO@05$e0>OArC(?0rJO zi#K5?QP%4?lZEe=aeiY&?5R3DOpMvA$Q+uSRZ9(?2%;*MCcP}K@(MTiMARF`nf>J2 zpTB>4IJ5<30aa6s3?IC)xu8bGWTqI_^-}4sDhP)^ysOP^bHj1i&7o~?DD<KgeNckJJ^68l}!UGN(EoP zh-uorvwe}(9zM%@TC`R-bo%0>gU)FIL6>8;FFb-=LD}j7|7MPWHu4JP`co|{>oURY zS#H=tQnQn))5h-8U!JosWH~a*ay6o!zEuj80Qx$FQA?K;^fGA-Acywgf({_I-e8#V zvHekXaBzDVwto4toG81W)za?vr*xGb;hm?gt19G~UmzV!MlpKy-rouC;v^eIa+HA5 zJk!j|1?VYRHV`KK{AmEyp`Yp#Sc@CSB)2~j55D4mp@>_-T@(9R6l{CS6Sl&@f44c$ zAfU7RI-j_)J9I@5RdR!D=GYlTzyE%?-AYi~c{L(>e4C>c*mNH!?uh8U76s`B8mXJa z%XhTZzmy&q+4LfK8L;W1@7p_R>zz-B18IDK1wE;Idsz{##>;kaTbo%wUxa&J3ZM=F zY#^M0596M5upv#Z-U?a9(x7L6l-sLt!wyC>2~J$6iMfE2#aZ zt9q!N7y*<2bb~&*`Rfh(0TnH)&m~;tWXv#b#(#nfvzza5twKs?PaD%HR82-gqexhN z@)pP5xcdz$7q^NUr&{752EOe9K%uuoH!)U`A%dLcRn~o*P`x=;08fQ?e&M2;fqL2t zMAY*UXCs@c48is@?&*($~JLxGKc%{r4BJZKWB*3EZ71z^vuB$ zI+!YgE)<6es1eCjvz1eo)d#3oe&0Bx6prP%Elp7Lhjv1x5u4XJHYNgShQGekJC%1;UD5IP3u>q4+zWmSY7aJ z_V8TG%2!*LQnQe^X|Ip&0DQ5$53v>^xdJ9+KU1zibEI-^FZ&jjr+uMlg>aeQb2Kfm zJ0&3rK2@*@y%_%~&tjRoiO3RFCQ7ce!o5x`_RF#)XH`K2cB$GDKg*hv7q4(16yw{> zhW0WNCa*T0trthVZ2$c}eS$8AU^}HsGJY#onJ?Ywre&;)WfvG)`7$4I8 zICA4TH~5b<|Eg;Y&?MtoG0Y^&L&x$_M+dRmlA^)q1q{f}QL=XcVMy>&Jp5p7T8 zhxl!ixKOgJ%D`@86pMN$C0z`LObZAp1^!oi-x&{QxAw~vz4rt`)DSKD5Jo4u=!8L{ z3xbH~gXkrqh3HWt(V~Pz87&ggMf4iI*HK0}w`cG7dF*Gu`@HAV`ErgA^Sd!K_qx}* z>VI9=e+2>`2-*S`QW1`w{cFfJ?$=a}s5Jf0kqo(!qLP&n%RT8e7n64BOzO5CVYg`U z@9Jzo5=E2<5M4b9mVBP`yXi#GHxGlu_|YlpY#>C*$HVfP5$0FdKmlQ}q@A3OA76tv z|AS}#FZ_0)ux|{7);%wGrOf28p^#)SmK(}Z{n!x+MOnozVA_JVX6!#91+pZS}z!;LEXSBKu?Vrbp*+U2A}d~Au4K%c=I8m)V@Y14#_Fn>ksnN5(uXk6H75S93N*-OXmmg1^sDTiQPo@;|hJ>FN+=$`WEyS~x9Lih4p z#Vz&J;?=!TIYbVA*Czo>k6B8;kc75Vi}MAGu}g%#`RnxArq`!W=>92@7KIs$bX>wi zA{!N9wzKIKMl+8b{L<*R6lD4@H7#4dNdxJ7;;k~Ju)g6z^cPOS0ol{KUcb__@ul;U zyNxS5vOY^0UQ=WW1Z?a)qjK4JDg`mp&x^-Dl5nAIl4EaH#xg-4u_iz+9ubV=`AHIKXb$S*+D?lTnSJ%PPQsJa+{ZKNP{q~1;VS6ybKqzM$#xzuB>wP zk!Q0;->}oe4N}6TJ?PCy8*Kvm4ktu2B5prBZ(k>i#B|kx#LF-*fXR zai5~I(@?<-B1@?$8ONcHee4!7U9$=pW`7Ls_Nu(ld60}lMXLs;_rVT5<8dMdBv!>T zw^YHL+?BM9$%`VqtedRv0#0gA0qf+`uP7OCX&cqQ5ske15d&HaVTmD)XlV7Ip z5~U%gE0yy<%P5WHhY=O>XpRy4cBQyMIT|yCXc+JSgWzQ|kR4z8`Ly0ilbna%iu81%KL3Pm{7qdunFnU?q6&iS1 zOV`@^fhS9!ZcwbXz}Dpy*L0|BSLf{v{m6U(#^u5}{6g$*Js{ z(XRNR_fpNCh3k1X_#EZ+h~5GyaQ_RI5@%`p%j9ZJhzw=ftaKX zz}C7BM!pR`8!yx}XDyIEbjhNJIma{(olYhQCo>h5*zn%1S*bve1{|-Gsl%i(uORs6 zRdvEfm~U$y#=Ci(_LpW_zaMI@%;^sC+BE z0Gm@>0u^I!}4q1LgKgMf;GSYv2 zvI{4mknqqs4bdT@7I#rJm#)eESzs-Z11OhmEbwR^?rSgcivT1I=f#N3U0_*`DY$$N zis2}m(YK?o(~iEMWdZ+~o;gimG{4m+ni^s?vSgZh2)e^!2f8l+Tee3ZFD|iO_OP)A5I=#i z2qZ(q<25F4;diQ&imNP8zu>5#y^X(1eVvV6NppxpW$)Tkm&gw<89`DplWSO_C$0VvSDB+((lJh%o$oB- zRb8(tF9y}e5Z&P`AhqpR>A?fU+GUK_8>T+A6ipuq_ zc!Uc91Qt5+h5gr%&EWfoX^aEdnz~u<4o+8q%D}1)fBV3@K3TEWCqLOHg%KW) zo!2ddtoq%K-pn__s77SE*UMV#&`HB9EGQZtG?|*+x_1Em#ZK!v>OK^gQET%(?09su zuA%i?mvAV7H@4~i1m&UgXR;rC;k8XQ&HNtI+161djs$eV06b*7YD7J-k>b+#4k$_S z{lWiJ#e-2NKMF?E(Z$&ZW$bS^^S;Mudbwj36qz@|+r%8Jqe9lPHuA6%2BZGgnK_CX zJeB*j^txx{u6O9!chZrXvx7k;*u*-{Dy&UMI~=wRG>U8w!Ktz0kk z8MFl`^U586KtDP@7ch#T5f)4^Xb+5h>HGDA?$(maa$rGy3e8A~^&k;C4;p*{?592y zf8U{!SNXPGujjkFz&%2%nyLswWsy-0LhLZRyy1JcDcjlcKL z86~5iZFDHn5_nBH>V>MAm7v_G{UGGZZ#%JH}obt=KWYf*_lNIu&;Wm6Hrjxlnw_r0d~|1ZXZ;b z5^^3)FJQ+FmLMl>`Cs#zKIBa6fB&e^H?&|`N$h(1F6x0=W`DTBq0jeH^3l_vM_6oA zN}yz%^>mr?rW12k>zL=%LOamB_Jsc7qP67!&Vi?b9uW58+C*^W94b>R;i(Mi5{iy-u*#HvbXCG8**X*Si z-B!AQzneCI#Mbm}{%Zb)i?@%5mo8f0WLw%q0QJ_2Z7JT__Y!4o9Oh7_0V%%;L`@H< zr-jrO1Q}Y&?e@&7obnnY9+u|I>!Bp>MpSaVD_X#hJyKo3;)o3#y({^AjyKh+62yh? z;H(s3#AK%XA^WR8H8vYLEC9mI<0DzDB{tMw=B_N4i+@DTNi`Y$nQWc#4a6mZUN#9H zE;=IpPws@9@o=>6H^hdJBFM`EC-s#OJfDudtK8kIJbkZZQxw&_`N&RAIdjj9T=dR414 zZ{H0dNx1OCK`Qn^8uzwGSsDAlyg%LgkQ@=&&JMz_l zBEWq$pd`=^Nn;Nb4Oo26&T(QvnYRgCm(=?y z0Aj*?l8Tk3U(YJ}PHqQR=E5wwI0QH);he?-f(~>2VW9RN<~-U&GQUfx^lA_bs0&-^ z6ohfY2Is}sr+s)tDSiXM?bG6!+nGhFO6DkY>WG8E4_guywm)xi(F-&;HW;q+-AK^- zNhEP-m2M@!5l6g>D~cIvLrncR>;rw_<)wS3XOhiplAS&Pl94AQ;LeA3pc<)f*adA) zT$EwXJU32c%wt>M6&dfk;k$p>#hoL!@%G$e>}c3qr)E5%4< zK!RT5PU|fc-=m=B*&dvrlS^dkP}VgqZBV4VM*bypQ!rFBeN~ z(0K6+)TMf5LPT!D!+^14Xm!gw!yF`bfVdd<6AkharXQS0kD;PSq^tZU}j^x-E@ldA80Lz0ejb@w6{~K9z<|s46?P@Krde_FL1OqyAUnV@LQV zMRKBd#c@o^F9why0NKkJ+fvm`&VGI{cfn7eU04QWB8s=9u&(J#uVF_V-CF`mr$l0h z%`JjQ9*F(y1tF>u5&wK)6s?`F<}rB8>J!EKE)W)B#N>-kWmoRamWlcY;o)!qvx;54Lgh30Xf+ ze(Cnli?IHjdBpKqz%_zH_@@-?c1T zzVg)G8FpCoSuf(d3=A9ny|MjzWbdw34OeDOeI};x)Szpxua&g=%Vxdf;x$Se9AB>P`XO!A|y$OCOi!{p<>11)eS)u<>xf8pd% zv2lTFP~ZbK#K@iWVUI?F&eI<0(a#9=J(sO(j3sPbiP@4gn4i%EiEV1h^@c)wt&K6i zJiF3D+7iTlp^Dw&3@IRUI%+cX&5_O=R+dz}eRh&EOItcmq>qoKP0y z0^AuulBQFg>7B53GtCe}_4W2$#Lrup63qO%;Z6+ZHinztst~3|jeCTn!4iyfL|)U& z1e84!mwR<%Mr2GMU0gp)lG)D8|8a8W*DG|3fyV+4j=Y(Vjaf*yi^5?@KO4Z2xqn>% znT*3wS<>Tj#BzZj6wG#+&c?ETCUftH0QpjnzjBo89DQ2(4!(zuntVjH4F+0c4$k#> z`dz}87O<<0!6k#~XAZu}wXsunNKw1|CY=C1pmN3O+nq;g82?e7_G_U<>yeC+IY(HB z6F}2qymQP+D0fd?R@o-Hsh~q?sJRojLI+dHk1hk(p0?OcSaf99YJx>RzQH*nMe}c3 zUBsGBJs9I|$`)4v6O7j!h32LWo9LkYOl_MuO34T-C~NX7jL{_#aa6;rb#R+oTm==@ zqV{{4os=Wo7AXF0sN=5>n_J@#d}}OVT+XAoUrMBcq;1x8CKL(LoJt=9=y^v1>~l_| z-AA|IWC~?9TBhpA653|?89Z4G8aujd(AsHIl4!XEvBb&6yfsklM{{%@+V(J%I9pE? zP=4=bo4^W*HVl2D~%)( z;nprp>=Dkhq&8kR{CG3sQx^Aj#*kzx^hqQCgaZcI7;V-7PY*_hx$yQ`iTSjxRukp?JFPb3QX|;+;f-2-^Qcx*b7gJ5VXguUK>#9b23M{EVo4dfQ#)YUCG*m zg*F@CEn&gYu4URO&tLXbo>uFz_eU?+v(INAFrwbLS#VmkD(&fY;%#E9PbU5v(QkJG zE-}9H!RIzNCGYqIS-4*OI)Xq&4)0e$(?#et_;?zF zM=Xz_m|X1k;9WxWu-8$Qlx(9MIB76^YyA}t(c|2goiZab0s8(yr6Wymq%If0b^75a z0+UGdq0!#QXX}TsqxL%%8UT{Ub%2<9UJ!vVp!qew|sPB|sEwYe9TGcTJiL1z@!W|kBUH7~e3*|- zdx^p-2~dFq(x8-;oJ`Yx)>@YBcUYQt^1$EmC{-oCh!Go5e!!pTa^T{{+q7|!^g6&T z+Z>UA*hl)r8Y-vdhUnB%Kh)Q@gM2x2P=fyYb~uXZ?j6(!DtYb;FY zG#u0jNu_@l22NM`d~XE$q|94nfu@B#>`@-ZB$~xt?1XD$2cz6i>8~jg62t&dF+P5R zX3+pr;RVfQBgQ@W$6_F2jc664)8{tm_kh?t6)IU~4MjE5`^X}hw#f!w{mK+(>#v?3 z2nS)O_o~d`C2&1?JGT zzkOuO`LT;wpP>!>gC@s0rf}k~(ZpxL4Xf}4%FJQOM`9jd?$ow4_*TsTvNKt!|Fo0Sj(}2FJEH=vs?0Bb z0d@ZUn%>qbum5Z)=~ffmG>LOLoSJsW_T2m!7qeu`Et)H~Bw%>1gXvxKphWD=vBh+s zzax$P(y){hNui(0#rVy61=XNGcq}t)V1g*jGs%wO9_W{ASHYpJFs?9Mz{f#$P2D53 zG1kk3TbyFc+&7siv}-0B0J>K9aNQS}Dw{ZE;&3k|?KfPi&(Ub*l5EH%>-iAu)s&s>2TE+phY1=X8n^Wc2bWa zJ_sYWE3eH-x7z@^p>ZdB@0+a}p*uT+bl!2jf@UxmebPZ}`_gU}UWT4;n!ang6!Vq8e0;U*=j&FQ$-_~t$@lElo3ra#8TOT8 z5r+kvxOQ-&wsVo1veC;nwo@$A{LquwQOutEXsz{IOAH^Tu;j#N^kT58y5oFl;MYG0 zCVcVeUURu^uB6 z&uZs%*^*IivGs{xl)obz8#6h5zS?LpmN1-X;W}~Pg_(cPkagyOUSpV}wXw)QHG{Ot zZuWbiY$mNo1=5F&RWzZiS1l>tbr;tQKZ_MgM~?=Mo#fVfnBv%33r=%1`R|Lq0p|4-$AKZAdg2mQY> zm56%$^^uA^dw{;jQA_&x;rPc#YNnMo8j=7%(0C*@&sYMx+6&PsTes!7#{dbdMyt)r zKt4D)iN}yc9bjjBs?y|&L2tBrS-MO&x&RbCtzLCF|H{*kcylS`J%*ee|0?CyDhPlq zP26fv~#_Ot-#{K*afWXY5Fw6g>Pu9Oka;}4{)jZ5&t>jb9B|8j2!ei}xusY-n zFmeHGD4LFI!(}cV01=t_R)A>Jv-y|GV*sh+&p()6xI6$J9iX09s-i^Z;OA3^3+Z+M zKKU#yBRr3YeT>%o88lTO&LSTEh$r0%A8Xmu-&P4;I*4cjB!Aws4 z-rv>Ij6q{n&N8R{q~oNG7r{X>a%}*e-85hQc1zAJO@_`(ni-cr=yQQD8UI!KVEk7J z=3-;=@KxeaP1p{xmHl@eZMs`T%*G!fP_}R)iEt!yvP- zxYBmLr_`{aRNC-AUV;j!1XAtpX>8>Of(W>>|9C-B--eICm*GAfW7y<;uFKuIAm-Bh zg_`+;g<4BmYc7+YMJiU;)&|=mDDTJJxOMjPQ~z?#%p-Ph&VOhd@BjNT*}iGAL*c8&fVX0%x~JUPy6$f2P>>Q zbcahH?9E|ig57=%xst0WIKBsHs`9CM4A{<{Pmgx&TSD;PJ;0z>l5^WBgv8T!VLoW5 zlasZwy|HRXLo4zd8d=hJyHf>O0gfF{3FglxQ(*;F2j+i`B*g-SnA@etK>nOWY-lN7 zkk^#=CAZ-1eva_KAQ7v!a1Vf7rciRLzH(p4tp1r`1)YC4i&RvwE6La6xBh}N%nx`e zNqVwmvfir~2}eW%2k4ryCOBr9EH7kuQ5K)NIn|>pjpL{i2%!G;-fqf_ooa6g1XsU-!((VSyQ<#?p>G7 zP>9MknREDTXI*VSSWsVNmHj+Kn;C@sA+p5M!UEiaorc#Wv5|jKbAM}EhIBmk8tnpa zZE3+_dzSv6(wbcZCE{?voVj`ZwHgFw20<47$LtwW-gn9XD&l-n=_6)<)yeDl6Z<3fKi}PV01TV`*S0$MC5kn`u5|8krK&bN8Z(S! zNw{-dNM@ZE?#`vnO_u^^^YtUcF?)VQGX)PQFwwsh^#~Ls-vK02g#pygQ6r8-t2S9l z*s2WkDWBk0E)$rRP{Mr7j#M#2?N)cY6W%Ytgk>2%zE6$UZn*jV4X%w^`7>!Bn~{Tf z7Wsj`9C`YTYI31lN0G#YV4B0W7m7Z&GrN10P^jY5x{;nV5y#!VI0gybC%=y`8Nstm zT$7w9Ww`&Lsedf<2M`Wrcjs0s^*>emQh&^`ZCC-3;Vfg6eEL>P=1ll(yOmUeY$i~e zWa%{%xgkI}wad9D!4@|hOE1Pp>bnW(PNq<^<2w!nBg~`d=VrLpJ}I35Gt>DXC_=Fh zj(~HK;cojfMHwh`#4+`*%`^wsJ)p3}+v64gwlz{=U8=ndoWNzT?G=?=_w=_`*b(F! z!^zN`lCWo zvhd{OqzyBB#e%_0@afclj!Dk>UzO5atg!iXF_)f_aP;P6x)H-^M*uiUbA z0FarKU~KN&zo?UFE?MiI13XXeNmymGgXdkQh>eLlq3MgfIO%(NN-_8Hl;e8z$pMZ@ zV=J9%)Z8^e51;vOFC*G#rNWNZg^)x@0+JcyE}d*o-L0vHz4*sCo?u>iB0rLZh34u{M31!{B+GpBkVFQg2r@i!#ipu@n^1pMW$ zOFeV_e1;w$1YDX90SDe4JjQ2?Wupx~UYTmL?a(NJAnImFv!)(`%$4l4ymLD2NMzPL z2l2$DeqD zs3~rfj7YrYXUd((k^jT#E_Ox^y*FGkkPDzyceQH-IKb+2DUd#G)M2}}y_wxP-y$eE zpY*}G`U*7CsY>GJF!YWDc@^gR{XaET%q_9MFy%ut(=R{L(l-82wC|A90 zv1)xmEWwuZh{|WCaXok@h>Vr{&NJR2dRC?AZ$5~uB-&Zh9AXwyfNZ_eCcoo`&uv2! zg;d>R*hT3d?(UBlxH}**!YTE=`v35Ymo_IG2SWF&D@bCGCRHEa2WOW}uW{BL7j+&) z1%dZ+@352>NzYK80LKA*TiZh7kIB1{_2xzfdQ(-=MJ8*n2L=1c1>*_e@rlQClFgQ9 zDlWhu88y$?o3|=ra&eh_1t#_N#L82>$Z3bJ`%cqZ_`VD#DrS2o?VLZN1ra=5*V8An z@;yIZ@>V>+rs?vjtuT@~7i0?Vi~@M?g~}w{7vCQ5EiDz3gTj6(sudOKp`t}~KL9-I zoH#L*ZQ6bw0%7nat*fCN$N8fxngOFb2*5*Z#J>uq^iv+uqv`8S9 zvd^}V5kREK7DmD3e`HAl#)IT)Q0&n?lje=Z1J5Bcw2s|VHF*k4AADJK2nAw;C3vuF zVDS0HEp85j3-SU1c-l3h+ne8W9awCX>oJ>Oyy3#UIk^w0HnQ*}@!(I`6?Ce%<#kGS zwyeH>rB#H=g|O*a^(jTT2CkCdPHk=WB`K!7bX3W=jT15fmdJgl{vLudJ6mcSh>Rc) z>l=LL0`=Dk99o%g4kINnh~C>Rl*O`kYi)=ffG;@3M84hcxbqFs;v=$zwuX1f&rVhf z-G7u!j1?Ds4Ud0&T77Q(rz3&~Es_CY6NzWmyDK6TgV(kWZdCe);@tkdok^qA8l|;A zt)oZX)*j@`(8&+*4ksY-U$bdhaUol&|YR4=Etut+ME5nCG+q&Qm-{*OYtUJ{et9x^>J^H z@iqVq<#ieZtZ`BawgTG?3;^q+o_Pox1g}alcNZC-2iTtcUux4-K6lNkGH<4aX9p34 z0-7a=OPK^N#OJUI7X!B%Lr92np!h}EZ z=6dRwRmDeQPvDZ?=qky;qZLQkx~cRA`5j|RHeD5Kqhygo69qqNwm`)QUo9hle;<*vegQDrhp!+k8$?Z}Pfu=!S6Jole(G-G`7;U> zE(BE3eHG($<-r_OW}e_sUIskP3VQ0UC<|{z0aMoTXbfAn-g^&=cmW}zyTXe*~oIFD>5GG5c85%dC{1usxdOzpkf+G zA{T13*F&+VyIyk^9dn(YdmgyD%`DdxF$P zM&gqk;KP9N&ulGV%UQcXr(1!rgwQC23osQ#cw*cfnLuJMb($7526)3YtJe0Pt`#h} z76R$u z5~d>)Ul32|>DE++seumTov?Vz?@Yh4mMe{LRVR;ke6oKMcb*_YYI&4*5TEN=0Av)L791mExW)p3pe;{^tD$@ zg09Rx*kVKAD&kx}?JB|Ip-6mm*8~nI1UMGp(jhJx6M=C_)WJAJDgd3m8gPtGH*Z$a zE`pqdj7lI1=^98)qsOPfQ%G0U0QjRSL^HDfYb;~B9gB!vU$%#-VAvem>V_hOvgzX7 z=bZ>;z`0h12~DP+qpO9P@u?WlPU+Hg_B59wSVrsh9d?aODG72iK}fnoT)k2mAn}d!Ul((h48&MDCxvKB$p%8B-7OnLb*E9OlvK} zv&ngrii)xqDD??EKmE%8jP~^A5@q??=!bdotlV|B#h2g2_+V|A-*ao~?!j_@9+f&- zY+LzSQ4LCnn`eZwMo(?q4$+D4_66*G~v$3Lk_9&?XAPC<*C7JqUhp%s8 zk#%WJp95fCeUqN_gDWwY5`>83y6jHYm7K2jI+RYc0iYoF9l4=TE@+Kh4NlcZHN$eC zS$&0Pl<#SR@sUBbbdB97>y^E`eGZEc$(YGB0ZQ@wpPwTUz*>!xLIiHxz{qgO2>NbB zMurxJh`Y|fGb<~A_n5KBe(SHa?PbslaQlkLhpK2)RzSuRv9myOV7{W~mA}j^)hy}9Wu7b;w#W1C4x|d7A+l}70{s9QEtl~@aqc5o}?V2^`dc3gzZ++-msMI1M=DzRsLru6QIapO7rL&cVYcF zQD}8D2xCVq;}0Hm%X$=DF=NS_q84U4OT}v3^R5#Fxo)`&T1}w&#yg`NeQHa>9K82h zS<>$S5ck-xlYaZ+%Ip`h02bi-bD@dID;A*p)+KOAORnxTQ7g!Aa=pUNr7cIo@o0P2 z9ypHSxVOrGjuKOCE=az26A!mY0YT0^Pw-;ScCm>PM?ed?3I+J+C;=ocy)3+W z+L&BjTc&y zI7a7+*Le-G(%uKa5%(XgB;&_#Wukzr4qrM)i;#^bY|oc;<7KL1bB=f=btx~&uqid} zsV_2%IS~A;g=A4M8bDU~xyiW8fc!q?g_w|aP$bb9jiTf`JaDH)M`hk!%1Sn)t{6HX zwO<7wloeBcQ8RU7T6zk70${N_do2TQXR2{E_tF8GJyx&HH(?pds7|Pj#_4+_-t$_a zX3#8-bJ}J2Kub*wz~yQolte!YZU+0E8A)Nf;$)J!MW2ZVK^rZBwX8I*O6^R#t_W4q zD-FTMu=gER)H1nq6{RstrNKNM%tN@>GnRb*XTh8~v}$Y#hy;6({5amfi;HG>s;tIh zf?H8UH#3tizbnQ3IrX(`$A;JZC?UfOWz(C;Sn+3^WQd(x{x-tf0HjdXZi&%qx0J-) z6zo9CpLNO~?+#ZeqfwQ)2NXi|tE;O|7G!;0x~!Rf!?~g6t#k1*#j;jg5QtQIySZ@) zl-#V%&ztwqdU}h-xx}E<>~#Z&=9|>Zgzj7I>BGDHw+Iroy4rc(>8;mz_v6 z0p|3bVnfyyWg{?40ui;?phUnW?{c#nDCLWciMk(mMzM?$(b%@fZ?2VEj~rBMoPQ{o z_;jfImqKD;2jd5j{`KAec9rb_m{k2dwCvT1_{-w{afJ|=-MYHGe}Ajqr4Ts$lver6 z`TW}z5yvUxUx4esT?G{bc#>Ys815CN`r9b71Divdz<_wP!7yK0=f6L% zKR^aVXD8uH5$T!-Nk3ez#a@Blz$4zVgs0M9cKQP0^M2 z7o`F6G*sjle^mtjc27F>Bgwzt`|7CmDFHz0dNt`4Y52G21%w0l%5ATV@Q=6tf8qF- g8T-E^95WYK3^cfpLjt6KV*x*^ikb?gatP%A0`*J;&j0`b literal 0 HcmV?d00001 From 3c4bbaf7a59f1ff02e1c8ffb31f6d914c216f4a9 Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Tue, 6 Aug 2024 15:21:11 -0500 Subject: [PATCH 47/72] bump version to 2024.8.0 --- docs/source/changelog.rst | 36 ++++++++++++++++++++++++++++++++++++ pyproject.toml | 2 +- 2 files changed, 37 insertions(+), 1 deletion(-) diff --git a/docs/source/changelog.rst b/docs/source/changelog.rst index d5ebe823d41..0f57a879b78 100644 --- a/docs/source/changelog.rst +++ b/docs/source/changelog.rst @@ -65,6 +65,41 @@ and allowing us to scale to larger datasets. See :pr-distributed:`8818` by `Patrick Hoefler`_ for more details and examples. +.. dropdown:: Additional changes + + - Add changelog for dask order patch (:pr:`11278`) `Patrick Hoefler`_ + - Add regression test for ``xarray`` map reduce (:pr:`11277`) `Florian Jetter`_ + - Add changelog entry for ``take`` (:pr:`11274`) `Patrick Hoefler`_ + - Revert "order: remove data task graph normalization" (:pr:`11276`) `Patrick Hoefler`_ + - Use the shuffle algorithm for ``take`` (:pr:`11267`) `Patrick Hoefler`_ + - Implement task-based array shuffle (:pr:`11262`) `Patrick Hoefler`_ + - Remove data task graph normalization (:pr:`11263`) `Florian Jetter`_ + - Update zoom link for monthly meeting (:pr:`11265`) `Sarah Charlotte Johnson`_ + - Update data loading section of best practices (:pr:`11247`) `Patrick Hoefler`_ + - Match default ``chunksize`` in docstring to actual default set in code (:pr:`11254`) `Bernhard Raml`_ + - Fixup casting error in ``pandas`` 3 (:pr:`11250`) `Patrick Hoefler`_ + - Skip new warning from ``pandas`` (:pr:`11249`) `Patrick Hoefler`_ + - Fix ``pandas`` nightly bugs (:pr:`11244`) `Patrick Hoefler`_ + + - Run graph normalisation after dask order (:pr-distributed:`8818`) `Patrick Hoefler`_ + - Update large graph size warning to remove scatter recommendation (:pr-distributed:`8815`) `Patrick Hoefler`_ + - Fail tasks exceeding ``no-workers-timeout`` (:pr-distributed:`8806`) `Hendrik Makait`_ + - Fix exception handling for ``NannyPlugin.setup`` and ``NannyPlugin.teardown`` (:pr-distributed:`8811`) `Hendrik Makait`_ + - Fix exception handling for ``WorkerPlugin.setup`` and ``WorkerPlugin.teardown`` (:pr-distributed:`8810`) `Hendrik Makait`_ + - typo fix (:pr-distributed:`8812`) `alex-rakowski`_ + - Fix ``if`` / ``else`` for ``send_recv_from_rpc`` (:pr-distributed:`8809`) `Patrick Hoefler`_ + - Ensure that adaptive only stops once (:pr-distributed:`8807`) `Hendrik Makait`_ + - Reduce noise from GC-related logging (:pr-distributed:`8804`) `Hendrik Makait`_ + - Remove unused ``delete_interval`` and ``synchronize_worker_interval`` from ``Scheduler`` (:pr-distributed:`8801`) `Hendrik Makait`_ + - Change log level for Compute Failed log message (:pr-distributed:`8802`) `Patrick Hoefler`_ + - Add Prometheus metric for time spent on GC (:pr-distributed:`8803`) `Hendrik Makait`_ + - Add Prometheus metrics for ``dask_worker_{added|removed}_total`` (:pr-distributed:`8798`) `Hendrik Makait`_ + - Add log event for ``worker-ttl-timed-out`` (:pr-distributed:`8800`) `Hendrik Makait`_ + - Add Prometheus metrics for ``dask_client_connections_{added|removed}_total`` (:pr-distributed:`8799`) `Hendrik Makait`_ + - Fix ``PackageInstall`` plugin (:pr-distributed:`8794`) `Hendrik Makait`_ + - Make stealing more robust (:pr-distributed:`8788`) `Hendrik Makait`_ + - Leave a warning about future instantiation (:pr-distributed:`8782`) `Florian Jetter`_ + .. _v2024.7.1: @@ -8483,3 +8518,4 @@ Other .. _`alex-rakowski`: https://github.com/alex-rakowski .. _`Adam Williamson`: https://github.com/AdamWill .. _`Jonas Dedden`: https://github.com/jonded94 +.. _`Bernhard Raml`: https://github.com/SwamyDev diff --git a/pyproject.toml b/pyproject.toml index 4779ba14536..c2f42539258 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -55,7 +55,7 @@ dataframe = [ "pandas >= 2.0", "dask-expr >= 1.1, <1.2", # dask-expr pins the dask version ] -distributed = ["distributed == 2024.7.1"] +distributed = ["distributed == 2024.8.0"] diagnostics = [ "bokeh >= 2.4.2", "jinja2 >= 2.10.3", From 1c8df0cd54d79f33767bc6384e1a2b46653cdc94 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Thu, 8 Aug 2024 05:16:08 +0200 Subject: [PATCH 48/72] Drop support for Python 3.9 (#11245) Co-authored-by: James Bourbeau --- .github/workflows/additional.yml | 4 +- .github/workflows/pre-commit.yml | 2 +- .github/workflows/tests.yml | 8 +- .pre-commit-config.yaml | 4 +- continuous_integration/environment-3.9.yaml | 77 ------------------- .../environment-mindeps-array.yaml | 6 +- .../environment-mindeps-dataframe.yaml | 6 +- .../environment-mindeps-distributed.yaml | 6 +- .../environment-mindeps-non-optional.yaml | 6 +- .../environment-mindeps-optional.yaml | 32 ++++---- continuous_integration/gpuci/axis.yaml | 1 - continuous_integration/recipe/meta.yaml | 4 +- dask/_version.py | 3 +- dask/array/numpy_compat.py | 9 ++- dask/dataframe/io/demo.py | 3 +- dask/order.py | 4 +- dask/typing.py | 6 +- dask/utils.py | 4 +- docs/source/changelog.rst | 14 ++++ docs/source/install.rst | 20 ++--- pyproject.toml | 7 +- 21 files changed, 82 insertions(+), 144 deletions(-) delete mode 100644 continuous_integration/environment-3.9.yaml diff --git a/.github/workflows/additional.yml b/.github/workflows/additional.yml index 1ed697fdee4..448e3ec7fc9 100644 --- a/.github/workflows/additional.yml +++ b/.github/workflows/additional.yml @@ -41,7 +41,7 @@ jobs: strategy: fail-fast: false matrix: - python-version: ["3.9", "3.10", "3.11", "3.12"] + python-version: ["3.10", "3.11", "3.12"] steps: - name: Checkout source uses: actions/checkout@v4.1.5 @@ -53,7 +53,7 @@ jobs: miniforge-version: latest use-mamba: true channel-priority: strict - python-version: "3.9" + python-version: "3.10" activate-environment: test-environment auto-activate-base: false diff --git a/.github/workflows/pre-commit.yml b/.github/workflows/pre-commit.yml index da93c37b572..90153e3fa74 100644 --- a/.github/workflows/pre-commit.yml +++ b/.github/workflows/pre-commit.yml @@ -14,5 +14,5 @@ jobs: - uses: actions/checkout@v4.1.5 - uses: actions/setup-python@v5 with: - python-version: '3.9' + python-version: '3.10' - uses: pre-commit/action@v3.0.1 diff --git a/.github/workflows/tests.yml b/.github/workflows/tests.yml index bfd08e39070..aa2e66e3d6d 100644 --- a/.github/workflows/tests.yml +++ b/.github/workflows/tests.yml @@ -26,11 +26,9 @@ jobs: fail-fast: false matrix: os: ["windows-latest", "ubuntu-latest", "macos-14"] - environment: ["3.9", "3.10", "3.11", "3.12"] + environment: ["3.10", "3.11", "3.12"] extra: [null] exclude: - - os: "macos-14" - environment: "3.9" - os: "macos-14" environment: "3.10" - os: "macos-14" @@ -85,8 +83,8 @@ jobs: if: ${{ matrix.extra == 'pandas-nightly' }} run: python -m pip install --pre --extra-index https://pypi.anaconda.org/scientific-python-nightly-wheels/simple pandas -U - - name: Disable query planning for 3.9 builds - if: ${{ matrix.environment == '3.9' }} + - name: Disable query planning for 3.10 builds + if: ${{ matrix.environment == '3.10' }} run: | export DASK_DATAFRAME__QUERY_PLANNING="False" echo "DASK_DATAFRAME__QUERY_PLANNING: $DASK_DATAFRAME__QUERY_PLANNING" diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index 77dd53560ea..f749e95094e 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -19,14 +19,14 @@ repos: hooks: - id: pyupgrade args: - - --py39-plus + - --py310-plus - repo: https://github.com/psf/black rev: 23.12.1 hooks: - id: black language_version: python3 args: - - --target-version=py39 + - --target-version=py310 - repo: https://github.com/pycqa/flake8 rev: 7.0.0 hooks: diff --git a/continuous_integration/environment-3.9.yaml b/continuous_integration/environment-3.9.yaml deleted file mode 100644 index 835626ca7ba..00000000000 --- a/continuous_integration/environment-3.9.yaml +++ /dev/null @@ -1,77 +0,0 @@ -# This job includes coverage -name: test-environment -channels: - - conda-forge - - nodefaults -dependencies: - # required dependencies - - python=3.9 - - packaging - - pyyaml - - click - - cloudpickle - - partd - - fsspec - - importlib_metadata - - toolz - # test dependencies - - pre-commit - - pytest - - pytest-cov - - pytest-rerunfailures - - pytest-timeout - - pytest-xdist - - moto<5 - # Optional dependencies - - mimesis<12 - - numpy=1.22 - - pandas=2.0 - - numba<0.59.0 # Removed numba.generated_jit which pandas<2.1 uses - - flask - - fastparquet - - h5py - - pytables - - zarr - # `tiledb-py=0.17.5` lead to strange seg faults in CI. - # We should unpin when possible. - # https://github.com/dask/dask/pull/9569 - - tiledb-py<0.17.4 - - pyspark - - tiledb - - xarray - - sqlalchemy - - pyarrow=9 - - coverage - - jsonschema - # other -- IO - - boto3 - - botocore - - bokeh - - httpretty - - aiohttp - - s3fs>=2021.9.0 - - crick - - cytoolz - - distributed - - ipython - - ipycytoscape - # until https://github.com/jupyter-widgets/ipywidgets/issues/3731 is fixed - - ipywidgets<8.0.5 - - ipykernel<6.22.0 - - lz4 - - psutil - - requests - - scikit-image<0.20 - - scikit-learn - - scipy - - python-snappy - - sparse - - cachey - - python-graphviz - - python-cityhash - - python-xxhash - - mmh3 - - jinja2 - - pip - - pip: - - git+https://github.com/dask/distributed diff --git a/continuous_integration/environment-mindeps-array.yaml b/continuous_integration/environment-mindeps-array.yaml index 88e352bfb9c..28239098764 100644 --- a/continuous_integration/environment-mindeps-array.yaml +++ b/continuous_integration/environment-mindeps-array.yaml @@ -4,10 +4,10 @@ channels: dependencies: # required dependencies - packaging=20.0 - - python=3.9 - - pyyaml=5.3.1 + - python=3.10 + - pyyaml=5.4.1 - click=8.1 - - cloudpickle=1.5.0 + - cloudpickle=2.0.0 - partd=1.4.0 - fsspec=2021.09.0 - importlib-metadata=4.13.0 diff --git a/continuous_integration/environment-mindeps-dataframe.yaml b/continuous_integration/environment-mindeps-dataframe.yaml index 63d2d49c365..ddf40e26452 100644 --- a/continuous_integration/environment-mindeps-dataframe.yaml +++ b/continuous_integration/environment-mindeps-dataframe.yaml @@ -4,10 +4,10 @@ channels: dependencies: # required dependencies - packaging=20.0 - - python=3.9 - - pyyaml=5.3.1 + - python=3.10 + - pyyaml=5.4.1 - click=8.1 - - cloudpickle=1.5.0 + - cloudpickle=2.0.0 - partd=1.4.0 - fsspec=2021.09.0 - importlib-metadata=4.13.0 diff --git a/continuous_integration/environment-mindeps-distributed.yaml b/continuous_integration/environment-mindeps-distributed.yaml index 1a1093969c9..c6b287ec4e8 100644 --- a/continuous_integration/environment-mindeps-distributed.yaml +++ b/continuous_integration/environment-mindeps-distributed.yaml @@ -4,10 +4,10 @@ channels: dependencies: # required dependencies - packaging=20.0 - - python=3.9 - - pyyaml=5.3.1 + - python=3.10 + - pyyaml=5.4.1 - click=8.1 - - cloudpickle=1.5.0 + - cloudpickle=2.0.0 - partd=1.4.0 - fsspec=2021.09.0 - importlib-metadata=4.13.0 diff --git a/continuous_integration/environment-mindeps-non-optional.yaml b/continuous_integration/environment-mindeps-non-optional.yaml index 71f970bb682..1f653c6a0b4 100644 --- a/continuous_integration/environment-mindeps-non-optional.yaml +++ b/continuous_integration/environment-mindeps-non-optional.yaml @@ -4,10 +4,10 @@ channels: dependencies: # required dependencies - packaging=20.0 - - python=3.9 - - pyyaml=5.3.1 + - python=3.10 + - pyyaml=5.4.1 - click=8.1 - - cloudpickle=1.5.0 + - cloudpickle=2.0.0 - partd=1.4.0 - fsspec=2021.09.0 - importlib-metadata=4.13.0 diff --git a/continuous_integration/environment-mindeps-optional.yaml b/continuous_integration/environment-mindeps-optional.yaml index 8a28f807cf9..5fa8cd11f2c 100644 --- a/continuous_integration/environment-mindeps-optional.yaml +++ b/continuous_integration/environment-mindeps-optional.yaml @@ -4,10 +4,10 @@ channels: dependencies: # required dependencies - packaging=20.0 - - python=3.9 - - pyyaml=5.3.1 + - python=3.10 + - pyyaml=5.4.1 - click=8.1 - - cloudpickle=1.5.0 + - cloudpickle=2.0.0 - partd=1.4.0 - fsspec=2021.09.0 - importlib-metadata=4.13.0 @@ -24,28 +24,28 @@ dependencies: # optional dependencies used by dask - cachey=0.1.1 - crick=0.0.3 - - cytoolz=0.11.0 + - cytoolz=0.11.2 - dask-ml=1.4.0 - - fastavro=1.1.0 + - fastavro=1.4.7 - fastparquet=0.8.2 - - h5py=2.10.0 + - h5py=3.4.0 - ipycytoscape=1.0.1 - - IPython=7.16.1 - - matplotlib=3.4.1 + - IPython=7.30.1 + - matplotlib=3.5.0 - mimesis=5.3.0 - - mmh3=2.5.1 - - psutil=5.7.2 + - mmh3=3.0.0 + - psutil=5.8.0 - python-cityhash=0.4.6 - python-graphviz=0.8.4 - - python-snappy=0.5.4 - - python-xxhash=2.0.0 + - python-snappy=0.6.0 + - python-xxhash=2.0.2 - s3fs=2021.9.0 - - scikit-image=0.17.2 - - scipy=1.5.2 + - scikit-image=0.19.0 + - scipy=1.7.2 - sparse=0.12.0 - - sqlalchemy=1.4.16 + - sqlalchemy=1.4.26 - tblib=1.6.0 - - tiledb-py=0.8.1 + - tiledb-py=0.12 - zarr=2.12.0 - pip - pip: diff --git a/continuous_integration/gpuci/axis.yaml b/continuous_integration/gpuci/axis.yaml index 77e9b9b45c8..25b4c78ef9f 100644 --- a/continuous_integration/gpuci/axis.yaml +++ b/continuous_integration/gpuci/axis.yaml @@ -1,5 +1,4 @@ PYTHON_VER: -- "3.9" - "3.10" - "3.11" diff --git a/continuous_integration/recipe/meta.yaml b/continuous_integration/recipe/meta.yaml index 2def4eba349..3a62ec08c51 100644 --- a/continuous_integration/recipe/meta.yaml +++ b/continuous_integration/recipe/meta.yaml @@ -20,13 +20,13 @@ build: requirements: host: - - python >=3.9 + - python >=3.10 - pip - versioneer =0.29 - tomli # [py<311] run: - - python >=3.9 + - python >=3.10 - click >=8.1 - cloudpickle >=1.5.0 - fsspec >=2021.09.0 diff --git a/dask/_version.py b/dask/_version.py index 88f145403ff..09281d8ae23 100644 --- a/dask/_version.py +++ b/dask/_version.py @@ -17,7 +17,8 @@ import re import subprocess import sys -from typing import Any, Callable +from collections.abc import Callable +from typing import Any def get_keywords() -> dict[str, str]: diff --git a/dask/array/numpy_compat.py b/dask/array/numpy_compat.py index 03160cbdde2..dcb32ef8c32 100644 --- a/dask/array/numpy_compat.py +++ b/dask/array/numpy_compat.py @@ -22,9 +22,12 @@ from numpy.exceptions import AxisError, ComplexWarning # noqa: F401 from numpy.lib.array_utils import normalize_axis_index, normalize_axis_tuple else: - from numpy import AxisError, ComplexWarning # noqa: F401 - from numpy.core.numeric import normalize_axis_index # type: ignore[attr-defined] - from numpy.core.numeric import normalize_axis_tuple # type: ignore[attr-defined] + from numpy import ( # type: ignore[no-redef, attr-defined] # noqa: F401 + AxisError, + ComplexWarning, + ) + from numpy.core.numeric import normalize_axis_index # type: ignore[no-redef] + from numpy.core.numeric import normalize_axis_tuple # type: ignore[no-redef] # Taken from scikit-learn: diff --git a/dask/dataframe/io/demo.py b/dask/dataframe/io/demo.py index 91757b61a76..416cdab185f 100644 --- a/dask/dataframe/io/demo.py +++ b/dask/dataframe/io/demo.py @@ -2,8 +2,9 @@ import re import string +from collections.abc import Callable from dataclasses import asdict, dataclass, field -from typing import Any, Callable, cast +from typing import Any, cast import numpy as np import pandas as pd diff --git a/dask/order.py b/dask/order.py index 6e427efa03c..90b7c223bf3 100644 --- a/dask/order.py +++ b/dask/order.py @@ -48,8 +48,8 @@ """ import copy from collections import defaultdict, deque, namedtuple -from collections.abc import Iterable, Mapping, MutableMapping -from typing import Any, Callable, Literal, NamedTuple, overload +from collections.abc import Callable, Iterable, Mapping, MutableMapping +from typing import Any, Literal, NamedTuple, overload from dask.core import get_dependencies, get_deps, getcycle, istask, reverse_dict from dask.typing import Key diff --git a/dask/typing.py b/dask/typing.py index b33c5299047..3a65d2986c0 100644 --- a/dask/typing.py +++ b/dask/typing.py @@ -15,10 +15,10 @@ if TYPE_CHECKING: # IPython import is relatively slow. Avoid if not necessary - from IPython.display import DisplayObject - # TODO import from typing (requires Python >=3.10) - from typing_extensions import TypeAlias + from typing import TypeAlias + + from IPython.display import DisplayObject CollType = TypeVar("CollType", bound="DaskCollection") CollType_co = TypeVar("CollType_co", bound="DaskCollection", covariant=True) diff --git a/dask/utils.py b/dask/utils.py index d192fe02936..9441233016f 100644 --- a/dask/utils.py +++ b/dask/utils.py @@ -11,7 +11,7 @@ import types import uuid import warnings -from collections.abc import Hashable, Iterable, Iterator, Mapping, Set +from collections.abc import Callable, Hashable, Iterable, Iterator, Mapping, Set from contextlib import contextmanager, nullcontext, suppress from datetime import datetime, timedelta from errno import ENOENT @@ -20,7 +20,7 @@ from numbers import Integral, Number from operator import add from threading import Lock -from typing import Any, Callable, ClassVar, Literal, TypeVar, cast, overload +from typing import Any, ClassVar, Literal, TypeVar, cast, overload from weakref import WeakValueDictionary import tlz as toolz diff --git a/docs/source/changelog.rst b/docs/source/changelog.rst index 0f57a879b78..63fcf15bab3 100644 --- a/docs/source/changelog.rst +++ b/docs/source/changelog.rst @@ -1,6 +1,20 @@ Changelog ========= +.. _v2024.8.1: + +2024.8.1 +-------- + +Highlights +^^^^^^^^^^ + +Drop support for Python 3.9 +""""""""""""""""""""""""""" + +This release drops support for Python 3.9 in accordance with NEP 29. Python 3.10 +is now the required minimum version to run Dask. + .. _v2024.8.0: 2024.8.0 diff --git a/docs/source/install.rst b/docs/source/install.rst index feded0f3a86..c759348afe9 100644 --- a/docs/source/install.rst +++ b/docs/source/install.rst @@ -112,7 +112,7 @@ These optional dependencies and their minimum supported versions are listed belo +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ | `crick`_ | ``>=0.0.3`` | Use ``tdigest`` internal method for dataframe statistics computation | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ -| `cytoolz`_ | ``>=0.11.0`` | Faster cythonized implementation of internal iterators, functions, and dictionaries | +| `cytoolz`_ | ``>=0.11.2`` | Faster cythonized implementation of internal iterators, functions, and dictionaries | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ | `dask-expr`_ | | Required for ``dask.dataframe``; Pins to a specific Dask version | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ @@ -124,43 +124,43 @@ These optional dependencies and their minimum supported versions are listed belo +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ | `graphviz`_ | ``>=0.8.4`` | Graph visualization using the graphviz engine | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ -| `h5py`_ | ``>=2.10.0`` | Storing array data in hdf5 files | +| `h5py`_ | ``>=3.4.0`` | Storing array data in hdf5 files | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ | `ipycytoscape`_ | ``>=1.0.1`` | Graph visualization using the cytoscape engine | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ -| `IPython`_ | ``>=7.16.1`` | Write graph visualizations made with graphviz engine to file | +| `IPython`_ | ``>=7.30.1`` | Write graph visualizations made with graphviz engine to file | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ | `jinja2`_ | ``>=2.10.3`` | HTML representations of Dask objects in Jupyter notebooks (required for ``dask.diagnostics``) | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ | `lz4`_ | ``>=4.3.2`` | Transparent use of lz4 compression algorithm | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ -| `matplotlib`_ | ``>=3.4.1`` | Color map support for graph visualization | +| `matplotlib`_ | ``>=3.5.0`` | Color map support for graph visualization | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ | `mimesis`_ | ``>=5.3.0`` | Random bag data generation with :func:`dask.datasets.make_people` | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ -| `mmh3`_ | ``>=2.5.1`` | Use MurmurHash hash functions for array hashing (~8x faster than SHA1) | +| `mmh3`_ | ``>=3.0.0`` | Use MurmurHash hash functions for array hashing (~8x faster than SHA1) | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ | `numpy`_ | ``>=1.21`` | Required for ``dask.array`` | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ | `pandas`_ | ``>=2.0`` | Required for ``dask.dataframe`` | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ -| `psutil`_ | ``>=5.7.2`` | Factor CPU affinity into CPU count, intelligently infer blocksize when reading CSV files | +| `psutil`_ | ``>=5.8.0`` | Factor CPU affinity into CPU count, intelligently infer blocksize when reading CSV files | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ | `pyarrow`_ | ``>=7.0`` | Support for Apache Arrow datatypes & engine when storing/reading Apache ORC or Parquet files | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ -| `python-snappy`_ | ``>=0.5.4`` | Snappy compression to bs used when storing/reading Avro or Parquet files | +| `python-snappy`_ | ``>=0.6.0`` | Snappy compression to bs used when storing/reading Avro or Parquet files | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ | `s3fs`_ | ``>=2021.9.0`` | Storing and reading data located in Amazon S3 | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ -| `scipy`_ | ``>=1.5.2`` | Required for ``dask.array.stats``, ``dask.array.fft``, and :func:`dask.array.linalg.lu` | +| `scipy`_ | ``>=1.7.2`` | Required for ``dask.array.stats``, ``dask.array.fft``, and :func:`dask.array.linalg.lu` | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ | `sparse`_ | ``>=0.12.0`` | Use sparse arrays as backend for dask arrays | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ -| `sqlalchemy`_ | ``>=1.4.16`` | Writing and reading from SQL databases | +| `sqlalchemy`_ | ``>=1.4.26`` | Writing and reading from SQL databases | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ | `tblib`_ | ``>=1.6.0`` | Serialization of worker traceback objects | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ -| `tiledb`_ | ``>=0.8.1`` | Storing and reading data from TileDB files | +| `tiledb`_ | ``>=0.12.0`` | Storing and reading data from TileDB files | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ | `xxhash`_ | ``>=2.0.0`` | Use xxHash hash functions for array hashing (~2x faster than MurmurHash, slightly slower than CityHash) | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ diff --git a/pyproject.toml b/pyproject.toml index c2f42539258..8fb3dbc4e9d 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -17,7 +17,6 @@ classifiers = [ "Programming Language :: Python", "Programming Language :: Python :: 3", "Programming Language :: Python :: 3 :: Only", - "Programming Language :: Python :: 3.9", "Programming Language :: Python :: 3.10", "Programming Language :: Python :: 3.11", "Programming Language :: Python :: 3.12", @@ -26,12 +25,12 @@ classifiers = [ ] readme = "README.rst" urls = {Homepage = "https://github.com/dask/dask/"} -requires-python = ">=3.9" +requires-python = ">=3.10" dependencies = [ # NOTE: These are tested in `continuous_integration/test_imports.sh` If # you modify these, make sure to change the corresponding line there. "click >= 8.1", - "cloudpickle >= 1.5.0", + "cloudpickle >= 2.0.0", "fsspec >= 2021.09.0", "packaging >= 20.0", "partd >= 1.4.0", @@ -159,7 +158,7 @@ timeout = 300 [tool.mypy] # Silence errors about Python 3.9-style delayed type annotations on Python 3.8 -python_version = "3.9" +python_version = "3.10" # See https://github.com/python/mypy/issues/12286 for automatic multi-platform support platform = "linux" # platform = win32 From e4f83a377eb8f38420b85cce7fefaa0ec321db99 Mon Sep 17 00:00:00 2001 From: Lucas Colley Date: Thu, 8 Aug 2024 10:52:54 +0100 Subject: [PATCH 49/72] API: expose `np` dtypes in `dask.array` namespace (#11178) --- dask/array/__init__.py | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/dask/array/__init__.py b/dask/array/__init__.py index 12849f6f1e1..b880d2a4de1 100644 --- a/dask/array/__init__.py +++ b/dask/array/__init__.py @@ -31,6 +31,22 @@ def _array_expr_enabled() -> bool: try: + from numpy import bool_ as bool + from numpy import ( + complex64, + complex128, + float32, + float64, + int8, + int16, + int32, + int64, + uint8, + uint16, + uint32, + uint64, + ) + from dask.array import backends, fft, lib, linalg, ma, overlap, random from dask.array._shuffle import shuffle from dask.array.blockwise import atop, blockwise From 65270980adab9c4f908ee81daa596dae16d56188 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Thu, 8 Aug 2024 13:13:26 +0200 Subject: [PATCH 50/72] Remove automatic resizing in reshape (#11269) --- dask/array/reshape.py | 50 +--------------------- dask/array/tests/test_array_core.py | 64 +---------------------------- 2 files changed, 3 insertions(+), 111 deletions(-) diff --git a/dask/array/reshape.py b/dask/array/reshape.py index 9a0318dfce7..a655f60c990 100644 --- a/dask/array/reshape.py +++ b/dask/array/reshape.py @@ -1,21 +1,18 @@ from __future__ import annotations import math -import warnings -from collections import Counter from functools import reduce from itertools import product from operator import mul import numpy as np -from dask import config -from dask.array.core import Array, normalize_chunks +from dask.array.core import Array from dask.array.utils import meta_from_array from dask.base import tokenize from dask.core import flatten from dask.highlevelgraph import HighLevelGraph -from dask.utils import M, parse_bytes +from dask.utils import M _not_implemented_message = """ Dask's reshape only supports operations that merge or split existing dimensions @@ -203,7 +200,6 @@ def reshape(x, shape, merge_chunks=True, limit=None): numpy.reshape """ # Sanitize inputs, look for -1 in shape - from dask.array.core import PerformanceWarning from dask.array.slicing import sanitize_index shape = tuple(map(sanitize_index, shape)) @@ -248,48 +244,6 @@ def reshape(x, shape, merge_chunks=True, limit=None): x = x.rechunk({i: 1 for i in range(din - dout)}) inchunks, outchunks = reshape_rechunk(x.shape, shape, x.chunks) - # Check output chunks are not too large - max_chunksize_in_bytes = reduce(mul, [max(i) for i in outchunks]) * x.dtype.itemsize - - if limit is None: - limit = parse_bytes(config.get("array.chunk-size")) - split = config.get("array.slicing.split-large-chunks", None) - else: - limit = parse_bytes(limit) - split = True - - if max_chunksize_in_bytes > limit: - if split is None: - msg = ( - "Reshaping is producing a large chunk. To accept the large\n" - "chunk and silence this warning, set the option\n" - " >>> with dask.config.set(**{'array.slicing.split_large_chunks': False}):\n" - " ... array.reshape(shape)\n\n" - "To avoid creating the large chunks, set the option\n" - " >>> with dask.config.set(**{'array.slicing.split_large_chunks': True}):\n" - " ... array.reshape(shape)" - "Explicitly passing ``limit`` to ``reshape`` will also silence this warning\n" - " >>> array.reshape(shape, limit='128 MiB')" - ) - warnings.warn(msg, PerformanceWarning, stacklevel=6) - elif split: - # Leave chunk sizes unaltered where possible - matching_chunks = Counter(inchunks) & Counter(outchunks) - chunk_plan = [] - for out in outchunks: - if matching_chunks[out] > 0: - chunk_plan.append(out) - matching_chunks[out] -= 1 - else: - chunk_plan.append("auto") - outchunks = normalize_chunks( - chunk_plan, - shape=shape, - limit=limit, - dtype=x.dtype, - previous_chunks=inchunks, - ) - x2 = x.rechunk(inchunks) # Construct graph diff --git a/dask/array/tests/test_array_core.py b/dask/array/tests/test_array_core.py index 0bf3fcc75aa..70eeb0a9fad 100644 --- a/dask/array/tests/test_array_core.py +++ b/dask/array/tests/test_array_core.py @@ -15,7 +15,6 @@ import os import time import warnings -from functools import reduce from io import StringIO from operator import add, sub from threading import Lock @@ -29,7 +28,6 @@ from dask.array.core import ( Array, BlockView, - PerformanceWarning, blockdims_from_blockshape, broadcast_chunks, broadcast_shapes, @@ -60,7 +58,7 @@ from dask.delayed import Delayed, delayed from dask.highlevelgraph import HighLevelGraph, MaterializedLayer from dask.layers import Blockwise -from dask.utils import SerializableLock, key_split, parse_bytes, tmpdir, tmpfile +from dask.utils import SerializableLock, key_split, tmpdir, tmpfile from dask.utils_test import dec, hlg_layer_topological, inc @@ -1237,66 +1235,6 @@ def test_reshape_unknown_dimensions(): pytest.raises(ValueError, lambda: da.reshape(a, (-1, -1))) -@pytest.mark.parametrize( - "limit", # in bytes - [ - None, # Default value: dask.config.get("array.chunk-size") - 134217728, # 128 MiB (default value size on a typical laptop) - 67108864, # 64 MiB (half the typical default value size) - ], -) -@pytest.mark.parametrize( - "shape, chunks, reshape_size", - [ - # Test reshape where output chunks would otherwise be too large - ((300, 180, 4, 18483), (-1, -1, 1, 183), (300, 180, -1)), - # Test reshape where multiple chunks match between input and output - ((300, 300, 4, 18483), (-1, -1, 1, 183), (300, 300, -1)), - ], -) -def test_reshape_avoids_large_chunks(limit, shape, chunks, reshape_size): - array = da.random.default_rng().random(shape, chunks=chunks) - if limit is None: - with dask.config.set(**{"array.slicing.split_large_chunks": True}): - result = array.reshape(*reshape_size, limit=limit) - else: - result = array.reshape(*reshape_size, limit=limit) - nbytes = array.dtype.itemsize - max_chunksize_in_bytes = reduce(operator.mul, result.chunksize) * nbytes - if limit is None: - limit = parse_bytes(dask.config.get("array.chunk-size")) - assert max_chunksize_in_bytes < limit - - -def test_reshape_warns_by_default_if_it_is_producing_large_chunks(): - # Test reshape where output chunks would otherwise be too large - shape, chunks, reshape_size = (300, 180, 4, 18483), (-1, -1, 1, 183), (300, 180, -1) - array = da.random.default_rng().random(shape, chunks=chunks) - - with pytest.warns(PerformanceWarning) as record: - result = array.reshape(*reshape_size) - nbytes = array.dtype.itemsize - max_chunksize_in_bytes = reduce(operator.mul, result.chunksize) * nbytes - limit = parse_bytes(dask.config.get("array.chunk-size")) - assert max_chunksize_in_bytes > limit - - assert len(record) == 1 - - with dask.config.set(**{"array.slicing.split_large_chunks": False}): - result = array.reshape(*reshape_size) - nbytes = array.dtype.itemsize - max_chunksize_in_bytes = reduce(operator.mul, result.chunksize) * nbytes - limit = parse_bytes(dask.config.get("array.chunk-size")) - assert max_chunksize_in_bytes > limit - - with dask.config.set(**{"array.slicing.split_large_chunks": True}): - result = array.reshape(*reshape_size) - nbytes = array.dtype.itemsize - max_chunksize_in_bytes = reduce(operator.mul, result.chunksize) * nbytes - limit = parse_bytes(dask.config.get("array.chunk-size")) - assert max_chunksize_in_bytes < limit - - def test_full(): d = da.full((3, 4), 2, chunks=((2, 1), (2, 2))) assert d.chunks == ((2, 1), (2, 2)) From ed5f68897b3a097f7c5ec1a9ec13ce49c112a544 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Thu, 8 Aug 2024 16:03:06 +0200 Subject: [PATCH 51/72] Ignore typing of return value (#11286) --- dask/array/__init__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dask/array/__init__.py b/dask/array/__init__.py index b880d2a4de1..bcd85873376 100644 --- a/dask/array/__init__.py +++ b/dask/array/__init__.py @@ -25,7 +25,7 @@ def _array_expr_enabled() -> bool: "The query-planning config can only be changed before " "`dask.dataframe` is first imported!" ) - return ARRAY_EXPR_ENABLED + return ARRAY_EXPR_ENABLED # type: ignore[return-value] return use_array_expr if use_array_expr is not None else False From 1ba9c5bf9d0b005b89a04804f29c16e5f0f2dadf Mon Sep 17 00:00:00 2001 From: Lucas Colley Date: Fri, 9 Aug 2024 12:58:28 +0100 Subject: [PATCH 52/72] Add numpy constants to array api (#11287) --- dask/array/__init__.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/dask/array/__init__.py b/dask/array/__init__.py index bcd85873376..0874199cc7c 100644 --- a/dask/array/__init__.py +++ b/dask/array/__init__.py @@ -35,12 +35,18 @@ def _array_expr_enabled() -> bool: from numpy import ( complex64, complex128, + e, + euler_gamma, float32, float64, + inf, int8, int16, int32, int64, + nan, + newaxis, + pi, uint8, uint16, uint32, From e83454339bbdafadd973cf82a44b30f9ac4d9918 Mon Sep 17 00:00:00 2001 From: Lucas Colley Date: Mon, 12 Aug 2024 15:46:20 +0100 Subject: [PATCH 53/72] array: fix `asarray` for array input with `dtype` (#11288) --- dask/array/core.py | 33 ++++++++++++++++++++++------- dask/array/tests/test_array_core.py | 12 +++++++++++ 2 files changed, 37 insertions(+), 8 deletions(-) diff --git a/dask/array/core.py b/dask/array/core.py index b710fe491f8..3009dcb11df 100644 --- a/dask/array/core.py +++ b/dask/array/core.py @@ -4524,6 +4524,13 @@ def retrieve_from_ooc( return load_dsk +def _as_dtype(a, dtype): + if dtype is None: + return a + else: + return a.astype(dtype) + + def asarray( a, allow_unknown_chunksizes=False, dtype=None, order=None, *, like=None, **kwargs ): @@ -4573,16 +4580,22 @@ def asarray( >>> y = [[1, 2, 3], [4, 5, 6]] >>> da.asarray(y) dask.array + + .. warning:: + `order` is ignored if `a` is an `Array`, has the attribute ``to_dask_array``, + or is a list or tuple of `Array`'s. """ if like is None: if isinstance(a, Array): - return a + return _as_dtype(a, dtype) elif hasattr(a, "to_dask_array"): - return a.to_dask_array() + return _as_dtype(a.to_dask_array(), dtype) elif type(a).__module__.split(".")[0] == "xarray" and hasattr(a, "data"): - return asarray(a.data) + return _as_dtype(asarray(a.data, order=order), dtype) elif isinstance(a, (list, tuple)) and any(isinstance(i, Array) for i in a): - return stack(a, allow_unknown_chunksizes=allow_unknown_chunksizes) + return _as_dtype( + stack(a, allow_unknown_chunksizes=allow_unknown_chunksizes), dtype + ) elif not isinstance(getattr(a, "shape", None), Iterable): a = np.asarray(a, dtype=dtype, order=order) else: @@ -4641,16 +4654,20 @@ def asanyarray(a, dtype=None, order=None, *, like=None, inline_array=False): >>> y = [[1, 2, 3], [4, 5, 6]] >>> da.asanyarray(y) dask.array + + .. warning:: + `order` is ignored if `a` is an `Array`, has the attribute ``to_dask_array``, + or is a list or tuple of `Array`'s. """ if like is None: if isinstance(a, Array): - return a + return _as_dtype(a, dtype) elif hasattr(a, "to_dask_array"): - return a.to_dask_array() + return _as_dtype(a.to_dask_array(), dtype) elif type(a).__module__.split(".")[0] == "xarray" and hasattr(a, "data"): - return asanyarray(a.data) + return _as_dtype(asarray(a.data, order=order), dtype) elif isinstance(a, (list, tuple)) and any(isinstance(i, Array) for i in a): - return stack(a) + return _as_dtype(stack(a), dtype) elif not isinstance(getattr(a, "shape", None), Iterable): a = np.asanyarray(a, dtype=dtype, order=order) else: diff --git a/dask/array/tests/test_array_core.py b/dask/array/tests/test_array_core.py index 70eeb0a9fad..d975cfced3d 100644 --- a/dask/array/tests/test_array_core.py +++ b/dask/array/tests/test_array_core.py @@ -2704,6 +2704,18 @@ def test_asarray(asarray): assert_eq(asarray(y), x) +@pytest.mark.parametrize("asarray", [da.asarray, da.asanyarray]) +def test_asarray_array_dtype(asarray): + # test array input + x = asarray([1, 2]) + assert_eq(asarray(x, dtype=da.float32), np.asarray(x, dtype=np.float32)) + + x = asarray(x, dtype=da.float64) + assert x.dtype == da.float64 + x = asarray(x, dtype=da.int32) + assert x.dtype == da.int32 + + @pytest.mark.parametrize("asarray", [da.asarray, da.asanyarray]) def test_asarray_dask_dataframe(asarray): # https://github.com/dask/dask/issues/3885 From a6c6fb142e78f85570a2c65e61e4933c03cac602 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Mon, 12 Aug 2024 17:03:29 +0200 Subject: [PATCH 54/72] Fix slicing for masked arrays (#11300) --- dask/array/_shuffle.py | 4 +++- dask/array/tests/test_slicing.py | 6 ++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/dask/array/_shuffle.py b/dask/array/_shuffle.py index 37aacb4bbdd..89f435d7b02 100644 --- a/dask/array/_shuffle.py +++ b/dask/array/_shuffle.py @@ -8,6 +8,7 @@ from dask import config from dask.array.chunk import getitem from dask.array.core import Array, unknown_chunk_message +from dask.array.dispatch import concatenate_lookup from dask.base import tokenize from dask.highlevelgraph import HighLevelGraph @@ -187,7 +188,8 @@ def _shuffle(chunks, indexer, axis, in_name, out_name, token): def concatenate_arrays(arrs, sorter, axis): - return np.take(np.concatenate(arrs, axis=axis), np.argsort(sorter), axis=axis) + concatenate = concatenate_lookup.dispatch(type(arrs[0])) + return np.take(concatenate(arrs, axis=axis), np.argsort(sorter), axis=axis) def convert_key(key, chunk, axis): diff --git a/dask/array/tests/test_slicing.py b/dask/array/tests/test_slicing.py index ed8a99e0c70..1bde2a01e98 100644 --- a/dask/array/tests/test_slicing.py +++ b/dask/array/tests/test_slicing.py @@ -1069,6 +1069,12 @@ def test_slice_array_3d_with_bool_numpy_array(): assert_eq(actual, expected) +def test_slice_masked_arrays(): + arr = np.ma.array(range(8), mask=[0, 0, 1, 0, 0, 1, 0, 1]) + darr = da.from_array(arr, chunks=(4, 4)) + assert_eq(darr[[2, 6]], arr[[2, 6]]) + + def test_slice_array_null_dimension(): array = da.from_array(np.zeros((3, 0))) expected = np.zeros((3, 0))[[0]] From d32fe8eb43b1d189b5e87a520803f7e7916b3d40 Mon Sep 17 00:00:00 2001 From: Sarah Charlotte Johnson Date: Mon, 12 Aug 2024 08:40:43 -0700 Subject: [PATCH 55/72] Link to dask vs spark benchmarks on dask docs (#11289) --- docs/source/spark.rst | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/source/spark.rst b/docs/source/spark.rst index 2eabde8ad83..c6ee3e240c8 100644 --- a/docs/source/spark.rst +++ b/docs/source/spark.rst @@ -19,6 +19,7 @@ Generally Dask is smaller and lighter weight than Spark. This means that it has fewer features and, instead, is used in conjunction with other libraries, particularly those in the numeric Python ecosystem. It couples with libraries like Pandas or Scikit-Learn to achieve high-level functionality. +Additionally, Dask is often faster and more robustly performant on `standard benchmarks `_ than Spark. Language ~~~~~~~~ From 25695ad280b749ddda7b9e6bdbc01b6f2b06d0aa Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Mon, 12 Aug 2024 19:00:00 +0200 Subject: [PATCH 56/72] Enable slicing with only one unknonw chunk (#11301) Co-authored-by: Hendrik Makait --- dask/array/slicing.py | 9 +++++++++ dask/array/tests/test_slicing.py | 23 +++++++++++++++++++++++ 2 files changed, 32 insertions(+) diff --git a/dask/array/slicing.py b/dask/array/slicing.py index f9add043aa7..3f345cfb82a 100644 --- a/dask/array/slicing.py +++ b/dask/array/slicing.py @@ -580,6 +580,15 @@ def take(outname, inname, chunks, index, axis=0): ) chunks, graph = _shuffle(chunks, indexer, axis, inname, outname, token) return chunks, graph + elif len(chunks[axis]) == 1: + slices = [slice(None)] * len(chunks) + slices[axis] = list(index) + slices = tuple(slices) + chunk_tuples = list(product(*(range(len(c)) for i, c in enumerate(chunks)))) + dsk = { + (outname,) + ct: (getitem, (inname,) + ct, slices) for ct in chunk_tuples + } + return chunks, dsk else: from dask.array.core import unknown_chunk_message diff --git a/dask/array/tests/test_slicing.py b/dask/array/tests/test_slicing.py index 1bde2a01e98..0e9cfce2819 100644 --- a/dask/array/tests/test_slicing.py +++ b/dask/array/tests/test_slicing.py @@ -1049,6 +1049,29 @@ def test_shuffle_slice(size, chunks): assert_eq(a, b) +def test_unknown_chunks_length_one(): + a = np.arange(256, dtype=int) + arr = da.from_array(a, chunks=(256,)) + # np.flatnonzero dispatches + result = np.flatnonzero(arr) + assert_eq(result[[0, -1]], np.flatnonzero(a)[[0, -1]]) + + result = da.flatnonzero(arr) + assert_eq(result[[0, -1]], np.flatnonzero(a)[[0, -1]]) + + a = a.reshape(16, 16) + arr = da.from_array(a, chunks=(8, 16)) + arr._chunks = ((8, 8), (np.nan,)) + result = arr[:, [0, -1]] + expected = a[:, [0, -1]] + assert_eq(result, expected) + + arr = da.from_array(a, chunks=(8, 8)) + arr._chunks = ((8, 8), (np.nan, np.nan)) + with pytest.raises(ValueError, match="Array chunk size or shape"): + arr[:, [0, -1]] + + @pytest.mark.parametrize("lock", [True, False]) @pytest.mark.parametrize("asarray", [True, False]) @pytest.mark.parametrize("fancy", [True, False]) From 33f07ecf403a8aa452bf6a9829a8fa72cec142a5 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Mon, 12 Aug 2024 20:13:38 +0200 Subject: [PATCH 57/72] Keep chunksize consistent in reshape (#11273) Co-authored-by: Hendrik Makait --- dask/array/reshape.py | 118 ++++++++++++++++++++++++++++- dask/array/tests/test_reshape.py | 126 +++++++++++++++++++++++++++++-- dask/array/tests/test_xarray.py | 7 ++ 3 files changed, 242 insertions(+), 9 deletions(-) diff --git a/dask/array/reshape.py b/dask/array/reshape.py index a655f60c990..56e1f338366 100644 --- a/dask/array/reshape.py +++ b/dask/array/reshape.py @@ -75,10 +75,14 @@ def reshape_rechunk(inshape, outshape, inchunks): chunk_reduction = reduce(mul, map(len, inchunks[ileft + 1 : ii + 1])) result_inchunks[ileft] = expand_tuple(inchunks[ileft], chunk_reduction) - prod = reduce(mul, inshape[ileft + 1 : ii + 1]) # 16 - result_outchunks[oi] = tuple( - prod * c for c in result_inchunks[ileft] - ) # (1, 1, 1, 1) .* 16 + max_in_chunk = _cal_max_chunk_size(inchunks, ileft, ii) + result_inchunks = _smooth_chunks( + ileft, ii, max_in_chunk, result_inchunks + ) + # Build cross product of result_inchunks[ileft:ii+1] + result_outchunks[oi] = _calc_lower_dimension_chunks( + result_inchunks, ileft, ii + ) oi -= 1 ii = ileft - 1 @@ -98,12 +102,118 @@ def reshape_rechunk(inshape, outshape, inchunks): result_outchunks[oleft] = tuple(c // cs for c in result_inchunks[ii]) + max_in_chunk = _cal_max_chunk_size(inchunks, ii, ii) + result_outchunks = _smooth_chunks(oleft, oi, max_in_chunk, result_outchunks) + # Build cross product of result_outchunks[oleft:oi+1] + result_inchunks[ii] = _calc_lower_dimension_chunks( + result_outchunks, oleft, oi + ) oi = oleft - 1 ii -= 1 return tuple(result_inchunks), tuple(result_outchunks) +def _calc_lower_dimension_chunks(chunks, start, stop): + # We need the lower dimension chunks to match what the higher dimension chunks + # can be combined to, i.e. multiply the different dimensions + return tuple( + map( + lambda x: reduce(mul, x), + product(*chunks[start : stop + 1]), + ) + ) + + +def _smooth_chunks(ileft, ii, max_in_chunk, result_inchunks): + # The previous step squashed the whole dimension into a single + # chunk for ileft + 1 (and potentially combined too many elements + # into a single chunk for ileft as well). We split up the single + # chunk into multiple chunks to match the max_in_chunk to keep + # chunksizes consistent: + # ((1, 1), (200)) -> ((1, 1), (20, ) * 10) for max_in_chunk = 20 + # It's important to ensure that all dimensions before the dimension + # we adjust have all-1 chunks to respect C contiguous arrays + # during the reshaping + # Example: + # Assume arr = da.from_array(np.arange(0, 12).reshape(4, 3), chunks=(2, 3)) + # Reshaping to arr.reshape(-1, ) will return + # [ 0 1 2 3 4 5 6 7 8 9 10 11] + # The first dimension of the reshaped axis are the chunks with length 2 + # Assume we split the second dimension into (2, 1), i.e. setting the chunks to + # ((2, 2), (2, 1)) and the output chunks to ((4, 2, 4, 2), ) + # In this case, the individual chunks do not hold a contiguous sequence. + # For example, the first chunk is [[0, 1], [3, 4]]. + # Then, the result will be different because we first reshape the individual, + # non-contiguous chunks before concatenating them: + # [ 0 1 3 4 2 5 6 7 9 10 8 11] + # This is equivalent to + # arr = np.arange(0, 12).reshape(4, 3) + # np.concatenate(list(map(lambda x: x.reshape(-1), [arr[:2, :2], arr[:2, 2:], arr[2:, :2], arr[2:, 2:]]))) + + ileft_orig = ileft + max_result_in_chunk = _cal_max_chunk_size(result_inchunks, ileft, ii) + if max_in_chunk == max_result_in_chunk: + # reshaping doesn't mess up + return result_inchunks + + while all(x == 1 for x in result_inchunks[ileft]): + # Find the first dimension where we can split chunks + ileft += 1 + + if ileft < ii + 1: + factor = math.ceil(max_result_in_chunk / max_in_chunk) + result_in_chunk = result_inchunks[ileft] + + if len(result_in_chunk) == 1: + # This is a trivial case, when we arrive here is the chunk we are + # splitting the same length as the whole dimension and all previous + # chunks that are reshaped into the same dimension are all-one. + # So we can split this dimension. + elem = result_in_chunk[0] + factor = min(factor, elem) + ceil_elem = math.ceil(elem / factor) + new_inchunk = [ceil_elem] * factor + for i in range(ceil_elem * factor - elem): + new_inchunk[i] -= 1 + result_inchunks[ileft] = tuple(new_inchunk) + + if all(x == 1 for x in new_inchunk) and ileft < ii: + # might have to do another round + return _smooth_chunks(ileft_orig, ii, max_in_chunk, result_inchunks) + else: + # We are now in the more complicated case. The first dimension in the set + # of dimensions to squash has non-ones and our max chunk is bigger than + # what we want. We need to split the non-ones into multiple chunks along + # this axis. + other_max_chunk = max_result_in_chunk // max(result_inchunks[ileft]) + result_in = [] + + for elem_in in result_in_chunk: + if elem_in * other_max_chunk <= max_in_chunk: + result_in.append(elem_in) + continue + + factor = math.ceil(elem_in * other_max_chunk / max_in_chunk) + ceil_elem = math.ceil(elem_in / factor) + new_in_chunk = [ceil_elem] * math.ceil(factor) + for i in range(ceil_elem * factor - elem_in): + new_in_chunk[i] -= 1 + result_in.extend(new_in_chunk) + + result_inchunks[ileft] = tuple(result_in) + return result_inchunks + + +def _cal_max_chunk_size(chunks, start, stop): + return int( + reduce( + mul, + [max(chunks[axis]) for axis in range(start, stop + 1)], + ) + ) + + def expand_tuple(chunks, factor): """ diff --git a/dask/array/tests/test_reshape.py b/dask/array/tests/test_reshape.py index cf6e99b5b90..ebd2eae39f3 100644 --- a/dask/array/tests/test_reshape.py +++ b/dask/array/tests/test_reshape.py @@ -4,7 +4,12 @@ import pytest import dask.array as da -from dask.array.reshape import contract_tuple, expand_tuple, reshape_rechunk +from dask.array.reshape import ( + _smooth_chunks, + contract_tuple, + expand_tuple, + reshape_rechunk, +) from dask.array.utils import assert_eq @@ -32,12 +37,12 @@ (64,), (4, 4, 4), ((8, 8, 8, 8, 8, 8, 8, 8),), - ((16, 16, 16, 16),), - ((1, 1, 1, 1), (4,), (4,)), + ((8, 8, 8, 8, 8, 8, 8, 8),), + ((1, 1, 1, 1), (2, 2), (4,)), ), ((64,), (4, 4, 4), ((32, 32),), ((32, 32),), ((2, 2), (4,), (4,))), ((64,), (4, 4, 4), ((16, 48),), ((16, 48),), ((1, 3), (4,), (4,))), - ((64,), (4, 4, 4), ((20, 44),), ((16, 48),), ((1, 3), (4,), (4,))), + ((64,), (4, 4, 4), ((20, 44),), ((16, 16, 32),), ((1, 1, 2), (4,), (4,))), ( (64, 4), (8, 8, 4), @@ -136,7 +141,21 @@ def test_reshape_all_chunked_no_merge(inshape, inchunks, outshape, outchunks): "inshape, inchunks, expected_inchunks, outshape, outchunks", [ # (2, 3, 4) -> (24,). This does merge, since the second dim isn't fully chunked! - ((2, 3, 4), ((1, 1), (1, 2), (2, 2)), ((1, 1), (3,), (4,)), (24,), ((12, 12),)), + ( + (2, 3, 4), + ((1, 1), (1, 2), (2, 2)), + ( + (1, 1), + ( + 1, + 1, + 1, + ), + (4,), + ), + (24,), + ((4, 4, 4, 4, 4, 4),), + ), ], ) def test_reshape_all_not_chunked_merge( @@ -207,3 +226,100 @@ def test_reshape_merge_chunks(inshape, inchunks, outshape, outchunks): assert_eq(result, base.reshape(outshape)) assert result.chunks != a.reshape(outshape).chunks + + +def test_smooth_chunks(): + ii = 2 + ileft = 1 + result_inchunks = [None, (1, 1, 1, 1, 1, 1, 1, 1, 1, 1), (20,)] + result_in = _smooth_chunks(ileft, ii, 2, result_inchunks) + expected_in = [None, (1, 1, 1, 1, 1, 1, 1, 1, 1, 1), (2, 2, 2, 2, 2, 2, 2, 2, 2, 2)] + assert result_in == expected_in + + +def test_smooth_chunks_not_divisible(): + ii = 2 + ileft = 1 + result_inchunks = [None, (1,) * 36, (9,)] + result_in = _smooth_chunks(ileft, ii, 6, result_inchunks) + expected_in = [None, (1,) * 36, (4, 5)] + assert result_in == expected_in + + +def test_smooth_chunks_first_dimension(): + ii = 1 + ileft = 0 + result_inchunks = [ + (1, 1, 3, 1, 1, 3), + (6,), + ( + 3, + 3, + ), + ] + result_in = _smooth_chunks(ileft, ii, 12, result_inchunks) + expected_in = [(1, 1, 1, 2, 1, 1, 1, 2), (6,), (3, 3)] + assert result_in == expected_in + result_in = _smooth_chunks(ileft, ii, 6, result_inchunks) + expected_in = [(1, 1, 1, 1, 1, 1, 1, 1, 1, 1), (6,), (3, 3)] + assert result_in == expected_in + + +def test_smooth_chunks_large_dimension(): + # Ensure that we properly smooth 2 dimensions + ii = 3 + ileft = 1 + result_inchunks = [None, (1, 1), (2,), (400,)] + result_in = _smooth_chunks(ileft, ii, 4, result_inchunks) + expected_in = [None, (1, 1), (1, 1), (4,) * 100] + assert result_in == expected_in + + +def test_reshape_not_squashing_chunks(): + arr = np.arange(0, 30 * 10 * 20).reshape(30, 10, 20) + darr = da.from_array(arr, chunks=(30, 2, 1)) + + result = darr.reshape(30, 200) + + assert result.chunks == ((30,), (2,) * 100) + assert np.prod(result.shape) == np.prod(darr.shape) + assert_eq(result, arr.reshape(30, 200)) + + +@pytest.mark.parametrize( + "reshaper, chunks", + [ + ((2, 2, 100), ((1, 1), (1, 1), (20,) * 5)), + ((2, 200), ((1, 1), (20,) * 10)), + ((10, 10, 4), ((1,) * 10, (5, 5), (4,))), + ((2, 25, 8), ((1,) * 2, (2,) * 5 + (3,) * 5, (8,))), + ], +) +def test_reshape_to_higher_dimension(reshaper, chunks): + arr = np.arange(0, 4_000).reshape(10, 400) + darr = da.from_array(arr, chunks=(5, (20,) * 19 + (19, 1))) + + result = darr.reshape(10, *reshaper) + + assert result.chunks == ((5, 5),) + chunks + assert np.prod(result.shape) == np.prod(darr.shape) + assert_eq(result, arr.reshape(10, *reshaper)) + + +def test_reshape_lower_dimension(): + arr = da.ones((20, 20, 5), chunks=(10, 10, 5)) + result = arr.reshape(400, 5) + assert result.chunks == ((100,) * 4, (5,)) + + +def test_reshape_split_out_chunks(): + # too large to assert the actual result + arr = da.ones(shape=(2, 1000, 100, 4800), chunks=(2, 1000, 100, 83)) + result = arr.reshape(2, 1000, 100, 4, 1200) + assert result.chunks == ( + (2,), + (1000,), + (100,), + (1, 1, 1, 1), + (80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80), + ) diff --git a/dask/array/tests/test_xarray.py b/dask/array/tests/test_xarray.py index 078755577f6..4bb0739d91d 100644 --- a/dask/array/tests/test_xarray.py +++ b/dask/array/tests/test_xarray.py @@ -47,3 +47,10 @@ def test_fft(): result = da.fft.fft(x) expected = da.fft.fft(x.data) assert_eq(result, expected) + + +def test_polyfit_reshaping(): + # Regression test for https://github.com/pydata/xarray/issues/4554 + arr = xr.DataArray(da.ones((10, 20, 30), chunks=(1, 5, 30)), dims=["z", "y", "x"]) + result = arr.polyfit("x", 1) + assert result.polyfit_coefficients.chunks == ((2,), (1,) * 10, (5,) * 4) From 83272708f94e4b08e53f877de21931d3dde31f05 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Tue, 13 Aug 2024 12:51:32 +0200 Subject: [PATCH 58/72] Make shuffle a no-op if possible (#11291) --- dask/array/_shuffle.py | 14 ++++++++++++++ dask/array/slicing.py | 8 ++++++++ dask/array/tests/test_shuffle.py | 13 +++++++++++++ dask/array/tests/test_slicing.py | 25 +++++++++++++++++++++---- 4 files changed, 56 insertions(+), 4 deletions(-) diff --git a/dask/array/_shuffle.py b/dask/array/_shuffle.py index 89f435d7b02..6e3dd2e4ab2 100644 --- a/dask/array/_shuffle.py +++ b/dask/array/_shuffle.py @@ -71,6 +71,9 @@ def shuffle(x, indexer: list[list[int]], axis): out_name = f"shuffle-{token}" chunks, layer = _shuffle(x.chunks, indexer, axis, x.name, out_name, token) + if len(layer) == 0: + return Array(x.dask, x.name, x.chunks, meta=x) + graph = HighLevelGraph.from_collections(out_name, layer, dependencies=[x]) return Array(graph, out_name, chunks, meta=x) @@ -89,6 +92,17 @@ def _shuffle(chunks, indexer, axis, in_name, out_name, token): raise IndexError( f"Indexer contains out of bounds index. Dimension only has {sum(chunks[axis])} elements." ) + + if len(indexer) == len(chunks[axis]): + # check if the array is already shuffled the way we want + ctr = 0 + for idx, c in zip(indexer, chunks[axis]): + if idx != list(range(ctr, ctr + c)): + break + ctr += c + else: + return chunks, {} + indexer = copy.deepcopy(indexer) chunksize_tolerance = config.get("array.shuffle.chunksize-tolerance") diff --git a/dask/array/slicing.py b/dask/array/slicing.py index 3f345cfb82a..0a0421fd79f 100644 --- a/dask/array/slicing.py +++ b/dask/array/slicing.py @@ -566,6 +566,14 @@ def take(outname, inname, chunks, index, axis=0): if not np.isnan(chunks[axis]).any(): from dask.array._shuffle import _shuffle + arange = np.arange(np.sum(chunks[axis])) + if len(index) == len(arange) and np.abs(index - arange).sum() == 0: + # TODO: This should be a real no-op, but the call stack is + # too deep to do this efficiently for now + chunk_tuples = list(product(*(range(len(c)) for i, c in enumerate(chunks)))) + graph = {(outname,) + c: (inname,) + c for c in chunk_tuples} + return tuple(chunks), graph + average_chunk_size = int(sum(chunks[axis]) / len(chunks[axis])) indexer = [] diff --git a/dask/array/tests/test_shuffle.py b/dask/array/tests/test_shuffle.py index 469f22be040..2882cbc8752 100644 --- a/dask/array/tests/test_shuffle.py +++ b/dask/array/tests/test_shuffle.py @@ -81,3 +81,16 @@ def test_oob_axis(darr): def test_oob_indexer(darr): with pytest.raises(IndexError, match="Indexer contains out of bounds index"): darr.shuffle([[16]], axis=1) + + +def test_shuffle_no_op_with_correct_indexer(): + arr = da.ones((250, 100), chunks=((50, 100, 33, 67), 100)) + indexer = [ + list(range(0, 50)), + list(range(50, 150)), + list(range(150, 183)), + list(range(183, 250)), + ] + result = arr.shuffle(indexer, axis=0) + assert result.dask == arr.dask + assert_eq(arr, result) diff --git a/dask/array/tests/test_slicing.py b/dask/array/tests/test_slicing.py index 0e9cfce2819..f503c5feef4 100644 --- a/dask/array/tests/test_slicing.py +++ b/dask/array/tests/test_slicing.py @@ -413,10 +413,10 @@ def test_take_sorted(): chunks, dsk = take("y", "x", [(20, 20, 20, 20)], np.arange(0, 80), axis=0) expected = { - ("y", 0): (getitem, ("x", 0), (np.arange(0, 20),)), - ("y", 1): (getitem, ("x", 1), (np.arange(0, 20),)), - ("y", 2): (getitem, ("x", 2), (np.arange(0, 20),)), - ("y", 3): (getitem, ("x", 3), (np.arange(0, 20),)), + ("y", 0): ("x", 0), + ("y", 1): ("x", 1), + ("y", 2): ("x", 2), + ("y", 3): ("x", 3), } np.testing.assert_equal(dsk, expected) assert chunks == ((20, 20, 20, 20),) @@ -1102,3 +1102,20 @@ def test_slice_array_null_dimension(): array = da.from_array(np.zeros((3, 0))) expected = np.zeros((3, 0))[[0]] assert_eq(array[[0]], expected) + + +def test_take_sorted_indexer(): + arr = da.ones((250, 100), chunks=((50, 100, 33, 67), 100)) + indexer = list(range(0, 250)) + result = arr[indexer, :] + assert_eq(arr, result) + assert { + **dict(arr.dask), + **{ + k: k2 + for k, k2 in zip( + [k for k in dict(result.dask) if "getitem" in k[0]], + dict(arr.dask).keys(), + ) + }, + } == dict(result.dask) From a969dee09d0c63d991cdf34df46a94ebc66c95fb Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Tue, 13 Aug 2024 13:52:36 +0200 Subject: [PATCH 59/72] Fix upstream numpy build (#11304) --- dask/array/numpy_compat.py | 1 + dask/array/tests/test_creation.py | 4 +++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/dask/array/numpy_compat.py b/dask/array/numpy_compat.py index dcb32ef8c32..7137f034fda 100644 --- a/dask/array/numpy_compat.py +++ b/dask/array/numpy_compat.py @@ -16,6 +16,7 @@ NUMPY_GE_124 = _np_version.release >= (1, 24) NUMPY_GE_125 = _np_version.release >= (1, 25) NUMPY_GE_200 = _np_version.release >= (2, 0) +NUMPY_GE_210 = _np_version.release >= (2, 1) if NUMPY_GE_200: diff --git a/dask/array/tests/test_creation.py b/dask/array/tests/test_creation.py index db11190dd74..bbf9f88cb61 100644 --- a/dask/array/tests/test_creation.py +++ b/dask/array/tests/test_creation.py @@ -11,7 +11,7 @@ import dask import dask.array as da from dask.array.core import normalize_chunks -from dask.array.numpy_compat import AxisError +from dask.array.numpy_compat import NUMPY_GE_210, AxisError from dask.array.utils import assert_eq, same_keys @@ -975,6 +975,8 @@ def test_nan_empty_like(shape_chunks, dtype): @pytest.mark.parametrize("shape_chunks", [((50, 4), (10, 2)), ((50,), (10,))]) @pytest.mark.parametrize("dtype", ["u4", np.float32, None, np.int64]) def test_nan_full_like(val, shape_chunks, dtype): + if NUMPY_GE_210 and val == -1 and dtype == "u4": + pytest.xfail("can't insert negative numbers into unsigned integer") shape, chunks = shape_chunks x1 = da.random.standard_normal(size=shape, chunks=chunks) y1 = x1[x1 < 0.5] From 67b2852d5f48d46527e3ac9c78122b483aef4057 Mon Sep 17 00:00:00 2001 From: Florian Jetter Date: Tue, 13 Aug 2024 14:45:17 +0200 Subject: [PATCH 60/72] order: ensure runnable tasks are certainly runnable (#11305) --- dask/order.py | 23 +++++++++++++---------- dask/tests/test_order.py | 3 +++ 2 files changed, 16 insertions(+), 10 deletions(-) diff --git a/dask/order.py b/dask/order.py index 90b7c223bf3..4de05c2b49b 100644 --- a/dask/order.py +++ b/dask/order.py @@ -315,6 +315,8 @@ def process_runnables() -> None: while branches: path = branches.popleft() while True: + # Loop invariant. Too expensive to compute at runtime + # assert not set(known_runnable_paths).intersection(runnable_hull) current = path[-1] runnable_hull.add(current) deps_downstream = dependents[current] @@ -326,6 +328,8 @@ def process_runnables() -> None: if num_needed[current] <= 1: for k in path: add_to_result(k) + else: + runnable_hull.discard(current) elif len(path) == 1 or len(deps_upstream) == 1: if len(deps_downstream) > 1: for d in sorted(deps_downstream, key=sort_key): @@ -337,11 +341,11 @@ def process_runnables() -> None: branch.append(d) branches.append(branch) break - runnable_hull.update(deps_downstream) - path.extend(sorted(deps_downstream, key=sort_key)) + path.extend(deps_downstream) continue elif current in known_runnable_paths: known_runnable_paths[current].append(path) + runnable_hull.discard(current) if ( len(known_runnable_paths[current]) >= num_needed[current] @@ -371,6 +375,7 @@ def process_runnables() -> None: add_to_result(k) else: known_runnable_paths[current] = [path] + runnable_hull.discard(current) break # Pick strategy @@ -434,10 +439,11 @@ def get_target() -> Key: candidates = leaf_nodes skey: Callable = sort_key - if runnable_hull: - skey = lambda k: (num_needed[k], sort_key(k)) - candidates = runnable_hull & candidates - elif reachable_hull: + # We're not considering runnable_hull because if there was a + # runnable leaf node, process_runnables should've taken care of + # it already, i.e. the intersection of runnable_hull and + # candidates is always empty + if reachable_hull: skey = lambda k: (num_needed[k], sort_key(k)) candidates = reachable_hull & candidates @@ -445,6 +451,7 @@ def get_target() -> Key: if seed := pick_seed(): candidates = leafs_connected[seed] else: + # FIXME: This seems to be dead code (at least untested) candidates = runnable_hull or reachable_hull # FIXME: This can be very expensive return min(candidates, key=skey) @@ -573,10 +580,6 @@ def path_pop() -> Key: if item in result: continue if num_needed[item]: - if item in known_runnable_paths: - for path in known_runnable_paths_pop(item): - path_extend(reversed(path)) - continue path_append(item) deps = dependencies[item].difference(result) unknown: list[Key] = [] diff --git a/dask/tests/test_order.py b/dask/tests/test_order.py index 70a1c6923ec..0a456cbcccd 100644 --- a/dask/tests/test_order.py +++ b/dask/tests/test_order.py @@ -339,6 +339,9 @@ def test_favor_longest_critical_path(abcde): assert o[e] > o[b] +@pytest.mark.xfail( + reason="Second target pick does not include already computed nodes properly" +) def test_run_smaller_sections(abcde): r""" aa From 182392329e982ce2e946b59405554c0468160027 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Wed, 14 Aug 2024 14:46:55 +0200 Subject: [PATCH 61/72] order: Choose better target for branches with multiple leaf nodes (#11303) --- dask/order.py | 108 +++++++++----------- dask/tests/test_order.py | 210 +++++++++++++++++++++++++++++++++++++-- 2 files changed, 250 insertions(+), 68 deletions(-) diff --git a/dask/order.py b/dask/order.py index 4de05c2b49b..62fb678b2fb 100644 --- a/dask/order.py +++ b/dask/order.py @@ -208,6 +208,9 @@ def order( _sort_keys_cache: dict[Key, tuple[int, int, int, int, str]] = {} + leafs_connected_to_loaded_roots: set[Key] = set() + processed_roots = set() + def sort_key(x: Key) -> tuple[int, int, int, int, str]: try: return _sort_keys_cache[x] @@ -246,6 +249,10 @@ def add_to_result(item: Key) -> None: result[item] = Order(i, crit_path_counter - _crit_path_counter_offset) else: result[item] = i + + if item in root_nodes: + processed_roots.add(item) + i += 1 # Note: This is a `set` and therefore this introduces a certain # randomness. However, this randomness should not have any impact on @@ -386,6 +393,43 @@ def process_runnables() -> None: # writing, the most expensive part of ordering is the prep work (mostly # connected roots + sort_key) which can be reused for multiple orderings. + def get_target(longest_path: bool = False) -> Key: + # Some topologies benefit if the node with the most dependencies + # is used as first choice, others benefit from the opposite. + candidates = leaf_nodes + skey: Callable = sort_key + if reachable_hull: + skey = lambda k: (num_needed[k], sort_key(k)) + + all_leafs_accessible = len(leafs_connected_to_loaded_roots) >= len(candidates) + if reachable_hull and not all_leafs_accessible: + # Avoid this branch if we can already access all leafs. Just pick + # one of them without the expensive selection process in this case. + + candidates = reachable_hull & candidates + if not candidates: + # We can't reach a leaf node directly, but we still have nodes + # with results in memory, these notes can inform our path towards + # a new preferred leaf node. + for r in processed_roots: + leafs_connected_to_loaded_roots.update(leafs_connected[r]) + processed_roots.clear() + + leafs_connected_to_loaded_roots.intersection_update(leaf_nodes) + candidates = leafs_connected_to_loaded_roots + else: + leafs_connected_to_loaded_roots.update(candidates) + + elif not reachable_hull: + # We reach a new and independent branch, so throw away previous branch + leafs_connected_to_loaded_roots.clear() + + if longest_path and (not reachable_hull or all_leafs_accessible): + return leaf_nodes_sorted.pop() + else: + # FIXME: This can be very expensive + return min(candidates, key=skey) + def use_longest_path() -> bool: size = 0 # Heavy reducer / splitter topologies often benefit from a very @@ -404,65 +448,9 @@ def use_longest_path() -> bool: return True longest_path = use_longest_path() - - def get_target() -> Key: - raise NotImplementedError() - - if not longest_path: - - def _build_get_target() -> Callable[[], Key]: - occurrences: defaultdict[Key, int] = defaultdict(int) - for t in leaf_nodes: - for r in roots_connected[t]: - occurrences[r] += 1 - occurences_grouped = defaultdict(set) - for root, occ in occurrences.items(): - occurences_grouped[occ].add(root) - occurences_grouped_sorted = {} - for k, v in occurences_grouped.items(): - occurences_grouped_sorted[k] = sorted(v, key=sort_key, reverse=True) - del occurences_grouped, occurrences - - def pick_seed() -> Key | None: - while occurences_grouped_sorted: - key = max(occurences_grouped_sorted) - picked_root = occurences_grouped_sorted[key][-1] - if picked_root in result: - occurences_grouped_sorted[key].pop() - if not occurences_grouped_sorted[key]: - del occurences_grouped_sorted[key] - continue - return picked_root - return None - - def get_target() -> Key: - candidates = leaf_nodes - skey: Callable = sort_key - - # We're not considering runnable_hull because if there was a - # runnable leaf node, process_runnables should've taken care of - # it already, i.e. the intersection of runnable_hull and - # candidates is always empty - if reachable_hull: - skey = lambda k: (num_needed[k], sort_key(k)) - candidates = reachable_hull & candidates - - if not candidates: - if seed := pick_seed(): - candidates = leafs_connected[seed] - else: - # FIXME: This seems to be dead code (at least untested) - candidates = runnable_hull or reachable_hull - # FIXME: This can be very expensive - return min(candidates, key=skey) - - return get_target - - get_target = _build_get_target() - else: + leaf_nodes_sorted = [] + if longest_path: leaf_nodes_sorted = sorted(leaf_nodes, key=sort_key, reverse=False) - get_target = leaf_nodes_sorted.pop - del leaf_nodes_sorted # ************************************************************************* # CORE ALGORITHM STARTS HERE @@ -497,7 +485,7 @@ def get_target() -> Key: # can define any route through the graph that should be considered as top # priority. # - # 1. Determine the target node by calling `get_target`` and append the + # 1. Determine the target node by calling ``get_target`` and append the # target to the critical path stack # 2. Take the _most valuable_ (max given a `sort_key`) of its dependents # and append it to the critical path stack. This key is the new target. @@ -562,7 +550,7 @@ def path_pop() -> Key: assert not scrit_path # A. Build the critical path - target = get_target() + target = get_target(longest_path=longest_path) next_deps = dependencies[target] path_append(target) diff --git a/dask/tests/test_order.py b/dask/tests/test_order.py index 0a456cbcccd..bea8caf7383 100644 --- a/dask/tests/test_order.py +++ b/dask/tests/test_order.py @@ -256,7 +256,7 @@ def test_prefer_deep(abcde): | | d a - Prefer longer chains first so we should start with c + Prefer longer chains first so we should start with d """ a, b, c, d, e = abcde dsk = {a: 1, b: (f, a), c: (f, b), d: 1, e: (f, d)} @@ -339,9 +339,6 @@ def test_favor_longest_critical_path(abcde): assert o[e] > o[b] -@pytest.mark.xfail( - reason="Second target pick does not include already computed nodes properly" -) def test_run_smaller_sections(abcde): r""" aa @@ -578,7 +575,7 @@ def test_map_overlap(abcde): |/ | \ | / | \| d1 d2 d3 d4 d5 | | | - e1 e2 e5 + e1 e3 e5 Want to finish b1 before we start on e5 """ @@ -1198,8 +1195,6 @@ def test_xarray_like_reduction(): def test_array_vs_dataframe(optimize): xr = pytest.importorskip("xarray") dd = pytest.importorskip("dask.dataframe") - if dd._dask_expr_enabled(): - pytest.xfail("doesn't work yet") import dask.array as da @@ -1224,7 +1219,8 @@ def test_array_vs_dataframe(optimize): diag_df = diagnostics( collections_to_dsk([mean.to_dask_dataframe()], optimize_graph=optimize) ) - assert max(diag_df[1]) == max(diag_array[1]) + assert max(diag_df[1]) == (15 if dd._dask_expr_enabled() else 38) + assert max(diag_array[1]) == 38 assert max(diag_array[1]) < 50 @@ -2100,6 +2096,204 @@ def test_xarray_map_reduce_with_slicing(): assert max(pressure) <= 5 +@pytest.mark.parametrize("use_longest_path", [True, False]) +def test_xarray_rechunk_map_reduce_cohorts(use_longest_path): + dsk = { + ("transpose", 0, 0, 0): (f, ("concat-groupby", 0, 0, 0)), + ("transpose", 0, 1, 0): (f, ("concat-groupby", 0, 1, 0)), + ("transpose", 1, 0, 0): (f, ("concat-groupby", 1, 0, 0)), + ("transpose", 1, 1, 0): (f, ("concat-groupby", 1, 1, 0)), + ("groupby-cohort", 0, 0, 0): (f, ("groupby-chunk", 0, 0, 0)), + ("groupby-cohort", 0, 0, 1): (f, ("groupby-chunk", 0, 0, 1)), + ("groupby-cohort", 1, 0, 0): (f, ("groupby-chunk", 1, 0, 0)), + ("groupby-cohort", 1, 0, 1): (f, ("groupby-chunk", 1, 0, 1)), + ("groupby-cohort-2", 0, 0, 0): (f, ("groupby-chunk-2", 0, 0, 0)), + ("groupby-cohort-2", 0, 0, 1): (f, ("groupby-chunk-2", 0, 0, 1)), + ("groupby-cohort-2", 1, 0, 0): (f, ("groupby-chunk-2", 1, 0, 0)), + ("groupby-cohort-2", 1, 0, 1): (f, ("groupby-chunk-2", 1, 0, 1)), + ("rechunk-merge", 3, 0, 0): ( + f, + ("concat-shuffle", 4, 0, 0), + ("rechunk-split", 12), + ), + ("rechunk-merge", 0, 0, 0): ( + f, + ("rechunk-split", 1), + ("concat-shuffle", 0, 0, 0), + ), + ("rechunk-merge", 3, 1, 0): ( + f, + ("rechunk-split", 14), + ("concat-shuffle", 4, 1, 0), + ), + ("rechunk-merge", 2, 1, 0): (f, ("rechunk-split", 10), ("rechunk-split", 11)), + ("rechunk-split", 12): (f, ("concat-shuffle", 3, 0, 0)), + ("rechunk-merge", 0, 1, 0): ( + f, + ("rechunk-split", 3), + ("concat-shuffle", 0, 1, 0), + ), + ("rechunk-merge", 1, 0, 0): (f, ("rechunk-split", 4), ("rechunk-split", 5)), + ("rechunk-merge", 1, 1, 0): (f, ("rechunk-split", 7), ("rechunk-split", 6)), + ("rechunk-split", 5): (f, ("concat-shuffle", 2, 0, 0)), + ("rechunk-split", 11): (f, ("concat-shuffle", 3, 1, 0)), + ("rechunk-merge", 2, 0, 0): (f, ("rechunk-split", 8), ("rechunk-split", 9)), + ("rechunk-split", 1): (f, ("concat-shuffle", 1, 0, 0)), + ("rechunk-split", 14): (f, ("concat-shuffle", 3, 1, 0)), + ("rechunk-split", 4): (f, ("concat-shuffle", 1, 0, 0)), + ("rechunk-split", 7): (f, ("concat-shuffle", 2, 1, 0)), + ("rechunk-split", 10): (f, ("concat-shuffle", 2, 1, 0)), + ("rechunk-split", 6): (f, ("concat-shuffle", 1, 1, 0)), + ("rechunk-split", 3): (f, ("concat-shuffle", 1, 1, 0)), + ("rechunk-split", 9): (f, ("concat-shuffle", 3, 0, 0)), + ("rechunk-split", 8): (f, ("concat-shuffle", 2, 0, 0)), + ("concat-shuffle", 0, 0, 0): (f, ("shuffle-split", 0), ("shuffle-split", 1)), + ("concat-shuffle", 0, 1, 0): ( + f, + ("shuffle-split", 106), + ("shuffle-split", 107), + ), + ("concat-shuffle", 1, 0, 0): ( + f, + ("shuffle-split", 4665), + ("shuffle-split", 4664), + ), + ("concat-shuffle", 1, 1, 0): ( + f, + ("shuffle-split", 4770), + ("shuffle-split", 4771), + ), + ("concat-shuffle", 2, 0, 0): ( + f, + ("shuffle-split", 9328), + ("shuffle-split", 9329), + ("shuffle-split", 9330), + ), + ("concat-shuffle", 2, 1, 0): ( + f, + ("shuffle-split", 9487), + ("shuffle-split", 9488), + ("shuffle-split", 9489), + ), + ("concat-shuffle", 3, 0, 0): ( + f, + ("shuffle-split", 16324), + ("shuffle-split", 16325), + ), + ("concat-shuffle", 3, 1, 0): ( + f, + ("shuffle-split", 16430), + ("shuffle-split", 16431), + ), + ("concat-shuffle", 4, 0, 0): ( + f, + ("shuffle-split", 20989), + ("shuffle-split", 20988), + ), + ("concat-shuffle", 4, 1, 0): ( + f, + ("shuffle-split", 21094), + ("shuffle-split", 21095), + ), + ("shuffle-split", 9487): (f, ("getitem-2", 2, 1, 0)), + ("shuffle-split", 9489): (f, ("getitem-2", 14, 1, 0)), + ("shuffle-split", 106): (f, ("getitem-open", 106)), + ("shuffle-split", 4664): (f, ("getitem-2", 1, 0, 0)), + ("shuffle-split", 16431): (f, ("getitem-2", 15, 1, 0)), + ("shuffle-split", 16324): (f, ("getitem-2", 14, 0, 0)), + ("shuffle-split", 107): (f, ("getitem-2", 1, 1, 0)), + ("shuffle-split", 4665): (f, ("getitem-2", 2, 0, 0)), + ("shuffle-split", 4770): (f, ("getitem-2", 1, 1, 0)), + ("shuffle-split", 0): (f, ("getitem-open", 0)), + ("shuffle-split", 9328): (f, ("getitem-2", 2, 0, 0)), + ("shuffle-split", 9488): (f, ("getitem-open", 9488)), + ("shuffle-split", 16325): (f, ("getitem-2", 15, 0, 0)), + ("shuffle-split", 16430): (f, ("getitem-2", 14, 1, 0)), + ("shuffle-split", 20988): (f, ("getitem-2", 15, 0, 0)), + ("shuffle-split", 9329): (f, ("getitem-open", 9329)), + ("shuffle-split", 4771): (f, ("getitem-2", 2, 1, 0)), + ("shuffle-split", 1): (f, ("getitem-2", 1, 0, 0)), + ("shuffle-split", 20989): (f, ("getitem-open", 20989)), + ("shuffle-split", 9330): (f, ("getitem-2", 14, 0, 0)), + ("shuffle-split", 21094): (f, ("getitem-2", 15, 1, 0)), + ("shuffle-split", 21095): (f, ("getitem-open", 21095)), + ("getitem-2", 1, 0, 0): (f, ("open_dataset", 1, 0, 0)), + ("getitem-2", 14, 0, 0): (f, ("open_dataset", 14, 0, 0)), + ("getitem-2", 2, 1, 0): (f, ("open_dataset", 2, 1, 0)), + ("getitem-2", 15, 0, 0): (f, ("open_dataset", 15, 0, 0)), + ("getitem-2", 15, 1, 0): (f, ("open_dataset", 15, 1, 0)), + ("getitem-2", 2, 0, 0): (f, ("open_dataset", 2, 0, 0)), + ("getitem-2", 1, 1, 0): (f, ("open_dataset", 1, 1, 0)), + ("getitem-2", 14, 1, 0): (f, ("open_dataset", 14, 1, 0)), + ("groupby-chunk-2", 0, 0, 1): (f, ("rechunk-merge", 2, 0, 0)), + ("groupby-chunk-2", 0, 0, 0): (f, ("rechunk-merge", 0, 0, 0)), + ("concat-groupby", 0, 0, 0): ( + f, + ("groupby-cohort-2", 0, 0, 0), + ("groupby-cohort-2", 0, 0, 1), + ), + ("groupby-chunk", 0, 0, 1): (f, ("rechunk-merge", 3, 0, 0)), + ("groupby-chunk", 0, 0, 0): (f, ("rechunk-merge", 1, 0, 0)), + ("concat-groupby", 1, 0, 0): ( + f, + ("groupby-cohort", 0, 0, 0), + ("groupby-cohort", 0, 0, 1), + ), + ("groupby-chunk", 1, 0, 1): (f, ("rechunk-merge", 3, 1, 0)), + ("groupby-chunk", 1, 0, 0): (f, ("rechunk-merge", 1, 1, 0)), + ("concat-groupby", 1, 1, 0): ( + f, + ("groupby-cohort", 1, 0, 0), + ("groupby-cohort", 1, 0, 1), + ), + ("open_dataset", 14, 1, 0): (f,), + ("groupby-chunk-2", 1, 0, 0): (f, ("rechunk-merge", 0, 1, 0)), + ("groupby-chunk-2", 1, 0, 1): (f, ("rechunk-merge", 2, 1, 0)), + ("concat-groupby", 0, 1, 0): ( + f, + ("groupby-cohort-2", 1, 0, 1), + ("groupby-cohort-2", 1, 0, 0), + ), + ("getitem-open", 9329): (f,), + ("open_dataset", 2, 1, 0): (f,), + ("open_dataset", 15, 1, 0): (f), + ("getitem-open", 20989): (f,), + ("getitem-open", 0): (f,), + ("open_dataset", 1, 0, 0): (f,), + ("getitem-open", 9488): (f,), + ("getitem-open", 21095): (f,), + ("open_dataset", 2, 0, 0): (f,), + ("getitem-open", 106): (f,), + ("open_dataset", 1, 1, 0): (f,), + ("open_dataset", 14, 0, 0): (f), + ("open_dataset", 15, 0, 0): (f), + } + if use_longest_path: + # ensure that we run through longes path True and False + keys = [("open-dataset", i, 0, 0) for i in range(20, 35)] + dsk.update({("dummy", 0): (f, keys)}) + dsk.update({k: (f,) for k in keys}) + + o = order(dsk) + + assert_topological_sort(dsk, o) + _, pressure = diagnostics(dsk, o=o) + # cut the dummy tasks in the end + assert max(pressure[:99]) <= 7 + + final_nodes = sorted( + [("transpose", ix, jx, 0) for ix in range(2) for jx in range(2)], + key=o.__getitem__, + ) + all_diffs = [] + for ix in range(1, len(final_nodes)): + all_diffs.append(o[final_nodes[ix]] - o[final_nodes[ix - 1]]) + + # We process a big chunk first and then a small side-branch + # before we repeat this for the next independent branch + assert all_diffs == [10, 39, 10] + + def test_xarray_8414(): # https://github.com/pydata/xarray/issues/8414#issuecomment-1793860552 np = pytest.importorskip("numpy") From 0260c47b553a7ad8a5044e43eb075c44048b2b1b Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Wed, 14 Aug 2024 22:51:54 +0200 Subject: [PATCH 62/72] Add tests to cover more cases of new reshape implementation (#11313) --- dask/array/tests/test_rechunk.py | 19 +++++++++++++++++++ dask/array/tests/test_reshape.py | 7 +++++++ 2 files changed, 26 insertions(+) diff --git a/dask/array/tests/test_rechunk.py b/dask/array/tests/test_rechunk.py index dc1b646b826..02eea953f08 100644 --- a/dask/array/tests/test_rechunk.py +++ b/dask/array/tests/test_rechunk.py @@ -1172,3 +1172,22 @@ def test_old_to_new_with_zero(): [[(0, slice(0, 4))], [(2, slice(0, 0))], [(2, slice(0, 2))], [(2, slice(2, 4))]] ] assert result == expected + + +def test_rechunk_non_perfect_slicing_of_dimensions(): + # GH#7859 + # this matters -- 1060 and 1058 work + shape = (200, 100, 1059) + final_chunks = (64, 64, 64) + + arr = da.coarsen( + da.mean, + da.zeros(shape, chunks=(1, -1, -1)), + {0: 2, 1: 2, 2: 2}, + trim_excess=True, + ) + result = arr.rechunk(*final_chunks) + assert_eq(arr, result) + result_b = arr.rechunk(final_chunks) + assert_eq(arr, result) + assert result.chunks == result_b.chunks diff --git a/dask/array/tests/test_reshape.py b/dask/array/tests/test_reshape.py index ebd2eae39f3..bc9dfa461d8 100644 --- a/dask/array/tests/test_reshape.py +++ b/dask/array/tests/test_reshape.py @@ -323,3 +323,10 @@ def test_reshape_split_out_chunks(): (1, 1, 1, 1), (80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, 80), ) + + +def test_argwhere_reshaping_not_concating_chunks(): + # GH#10080 + arr = da.random.random((500, 500, 500), chunks=(100, 100, 100)) < 0 + result = da.argwhere(arr) + assert result.chunks == ((np.nan,) * 125, (1, 1, 1)) From b597ebc2177e37fff0930cf67ec86565192d00d7 Mon Sep 17 00:00:00 2001 From: Florian Jetter Date: Thu, 15 Aug 2024 17:43:24 +0200 Subject: [PATCH 63/72] Test ordering on distributed scheduler (#11310) Co-authored-by: Patrick Hoefler <61934744+phofl@users.noreply.github.com> --- dask/order.py | 24 ++++++------- dask/tests/test_order.py | 75 ++++++++++++++++++++++++++++++++++++---- 2 files changed, 78 insertions(+), 21 deletions(-) diff --git a/dask/order.py b/dask/order.py index 62fb678b2fb..06cff4a8eb0 100644 --- a/dask/order.py +++ b/dask/order.py @@ -137,7 +137,7 @@ def order( # way that is simpler to handle all_tasks = False n_removed_leaves = 0 - requires_data_task = defaultdict(set) + requires_data_task = defaultdict(list) while not all_tasks: all_tasks = True for leaf in list(leaf_nodes): @@ -174,7 +174,7 @@ def order( dependencies = copy.deepcopy(dependencies) root_nodes.remove(root) for dep in dependents[root]: - requires_data_task[dep].add(root) + requires_data_task[dep].append(root) dependencies[dep].remove(root) if not dependencies[dep]: root_nodes.add(dep) @@ -254,12 +254,7 @@ def add_to_result(item: Key) -> None: processed_roots.add(item) i += 1 - # Note: This is a `set` and therefore this introduces a certain - # randomness. However, this randomness should not have any impact on - # the final result since the `process_runnable` should produce - # equivalent results regardless of the order in which runnable is - # populated (not identical but equivalent) - for dep in dependents.get(item, ()): + for dep in sorted(dependents.get(item, ()), key=sort_key): num_needed[dep] -= 1 reachable_hull.add(dep) if not num_needed[dep]: @@ -551,14 +546,15 @@ def path_pop() -> Key: # A. Build the critical path target = get_target(longest_path=longest_path) - next_deps = dependencies[target] path_append(target) - while next_deps: - item = max(next_deps, key=sort_key) - path_append(item) - next_deps = dependencies[item] - path_extend(next_deps) + if deps_target := dependencies[target]: + next_deps = [max(deps_target, key=sort_key)] + while next_deps: + item = next_deps[-1] + path_append(item) + next_deps = sorted(dependencies[item], key=sort_key) + path_extend(next_deps) # B. Walk the critical path diff --git a/dask/tests/test_order.py b/dask/tests/test_order.py index bea8caf7383..50176d49aaf 100644 --- a/dask/tests/test_order.py +++ b/dask/tests/test_order.py @@ -1,5 +1,6 @@ from __future__ import annotations +import asyncio import inspect import pytest @@ -8,9 +9,19 @@ from dask import delayed from dask.base import collections_to_dsk, key_split, visualize_dsk from dask.core import get_deps -from dask.order import _connecting_to_roots, diagnostics, ndependencies, order +from dask.order import _connecting_to_roots, diagnostics, ndependencies from dask.utils_test import add, inc +try: + import distributed # noqa: F401 + + HAS_DISTRIBUTED = True +except ImportError: + HAS_DISTRIBUTED = False + + +GRAPHS_FOR_DISTRIBUTED = [] + @pytest.fixture( params=[ @@ -26,6 +37,16 @@ def f(*args): pass +def order(dsk, *args, **kwargs): + from dask.order import order as _order + + GRAPHS_FOR_DISTRIBUTED.append((inspect.stack()[1][3], dsk)) + + assert _order(dsk, *args, **kwargs) == _order(dsk, *args, **kwargs) + + return _order(dsk, *args, **kwargs) + + def visualize(dsk, suffix="", **kwargs): """Utility to visualize the raw low level graphs in this tests suite. This automatically generates a set of visualizations with different metrics and @@ -711,18 +732,24 @@ def test_many_branches_use_ndependencies(abcde): def test_order_cycle(): + # Note: We're overriding `order` in this module to run some additional tests + # and to queue up the graphs for the distributed scheduler test below. We + # don't want to run these broken graphs on the scheduler since the test + # below is not robust to broken graphs + from dask.order import order as _order + with pytest.raises(RuntimeError, match="Cycle detected"): dask.get({"a": (f, "a")}, "a") # we encounter this in `get` with pytest.raises(RuntimeError, match="Cycle detected"): - order({"a": (f, "a")}) # trivial self-loop + _order({"a": (f, "a")}) # trivial self-loop with pytest.raises(RuntimeError, match="Cycle detected"): - order({("a", 0): (f, ("a", 0))}) # non-string + _order({("a", 0): (f, ("a", 0))}) # non-string with pytest.raises(RuntimeError, match="Cycle detected"): - order({"a": (f, "b"), "b": (f, "c"), "c": (f, "a")}) # non-trivial loop + _order({"a": (f, "b"), "b": (f, "c"), "c": (f, "a")}) # non-trivial loop with pytest.raises(RuntimeError, match="Cycle detected"): - order({"a": (f, "b"), "b": (f, "c"), "c": (f, "a", "d"), "d": 1}) + _order({"a": (f, "b"), "b": (f, "c"), "c": (f, "a", "d"), "d": 1}) with pytest.raises(RuntimeError, match="Cycle detected"): - order({"a": (f, "b"), "b": (f, "c"), "c": (f, "a", "d"), "d": (f, "b")}) + _order({"a": (f, "b"), "b": (f, "c"), "c": (f, "a", "d"), "d": (f, "b")}) def test_order_empty(): @@ -1963,7 +1990,7 @@ def test_gh_3055_explicit(abcde): (c, 0, 0): (f, (b, 0), (a, 2), (a, 1)), (d, 0, 0): (f, (c, 0, 1), (c, 0, 0)), (d, 0, 1): (f, (c, 0, 1), (c, 0, 0)), - (f, 1, 1): (f, (d, 0, 1)), + ("f", 1, 1): (f, (d, 0, 1)), (c, 1, 0): (f, (b, 1, 0), (b, 1, 2)), (c, 0, 2): (f, (b, 0, 0), (b, 0, 1)), (e, 0): (f, (c, 1, 0), (c, 0, 2)), @@ -2472,3 +2499,37 @@ def test_do_not_mutate_input(): assert_topological_sort(dsk, o) assert dsk == dsk_copy assert dependencies == dependencies_copy + + +if HAS_DISTRIBUTED: + from dask.tests.test_distributed import gen_cluster + + @pytest.mark.slow + @gen_cluster(client=True, nthreads=[]) + async def test_order_on_distributed_cluster(c, s): + # Note: For the GRAPHS_FOR_DISTRIBUTED to be populated, the above tests + # have to run first + for testname, dsk in GRAPHS_FOR_DISTRIBUTED: + if not dsk: + continue + + while s.tasks: + await asyncio.sleep(0.01) + _, dependents = get_deps(dsk) + output_keys = [k for k, v in dependents.items() if not v] + futs = c.get(dsk, output_keys, sync=False) # noqa: F841 + while not s.tasks: + await asyncio.sleep(0.01) + actual = {ts.key: ts.priority for ts in s.tasks.values()} + actual_keys = sorted(actual, key=actual.__getitem__) + + expected = dask.order.order(dsk) + expected_keys = sorted(expected, key=expected.__getitem__) + try: + assert actual_keys == expected_keys, ( + testname, + actual_keys, + expected_keys, + ) + finally: + del futs From 26a4c0ffee77c80db6473536dc86c8a65385de5a Mon Sep 17 00:00:00 2001 From: Florian Jetter Date: Thu, 15 Aug 2024 22:41:56 +0200 Subject: [PATCH 64/72] Revert "Test ordering on distributed scheduler (#11310)" (#11321) --- dask/order.py | 24 +++++++------ dask/tests/test_order.py | 75 ++++------------------------------------ 2 files changed, 21 insertions(+), 78 deletions(-) diff --git a/dask/order.py b/dask/order.py index 06cff4a8eb0..62fb678b2fb 100644 --- a/dask/order.py +++ b/dask/order.py @@ -137,7 +137,7 @@ def order( # way that is simpler to handle all_tasks = False n_removed_leaves = 0 - requires_data_task = defaultdict(list) + requires_data_task = defaultdict(set) while not all_tasks: all_tasks = True for leaf in list(leaf_nodes): @@ -174,7 +174,7 @@ def order( dependencies = copy.deepcopy(dependencies) root_nodes.remove(root) for dep in dependents[root]: - requires_data_task[dep].append(root) + requires_data_task[dep].add(root) dependencies[dep].remove(root) if not dependencies[dep]: root_nodes.add(dep) @@ -254,7 +254,12 @@ def add_to_result(item: Key) -> None: processed_roots.add(item) i += 1 - for dep in sorted(dependents.get(item, ()), key=sort_key): + # Note: This is a `set` and therefore this introduces a certain + # randomness. However, this randomness should not have any impact on + # the final result since the `process_runnable` should produce + # equivalent results regardless of the order in which runnable is + # populated (not identical but equivalent) + for dep in dependents.get(item, ()): num_needed[dep] -= 1 reachable_hull.add(dep) if not num_needed[dep]: @@ -546,15 +551,14 @@ def path_pop() -> Key: # A. Build the critical path target = get_target(longest_path=longest_path) + next_deps = dependencies[target] path_append(target) - if deps_target := dependencies[target]: - next_deps = [max(deps_target, key=sort_key)] - while next_deps: - item = next_deps[-1] - path_append(item) - next_deps = sorted(dependencies[item], key=sort_key) - path_extend(next_deps) + while next_deps: + item = max(next_deps, key=sort_key) + path_append(item) + next_deps = dependencies[item] + path_extend(next_deps) # B. Walk the critical path diff --git a/dask/tests/test_order.py b/dask/tests/test_order.py index 50176d49aaf..bea8caf7383 100644 --- a/dask/tests/test_order.py +++ b/dask/tests/test_order.py @@ -1,6 +1,5 @@ from __future__ import annotations -import asyncio import inspect import pytest @@ -9,19 +8,9 @@ from dask import delayed from dask.base import collections_to_dsk, key_split, visualize_dsk from dask.core import get_deps -from dask.order import _connecting_to_roots, diagnostics, ndependencies +from dask.order import _connecting_to_roots, diagnostics, ndependencies, order from dask.utils_test import add, inc -try: - import distributed # noqa: F401 - - HAS_DISTRIBUTED = True -except ImportError: - HAS_DISTRIBUTED = False - - -GRAPHS_FOR_DISTRIBUTED = [] - @pytest.fixture( params=[ @@ -37,16 +26,6 @@ def f(*args): pass -def order(dsk, *args, **kwargs): - from dask.order import order as _order - - GRAPHS_FOR_DISTRIBUTED.append((inspect.stack()[1][3], dsk)) - - assert _order(dsk, *args, **kwargs) == _order(dsk, *args, **kwargs) - - return _order(dsk, *args, **kwargs) - - def visualize(dsk, suffix="", **kwargs): """Utility to visualize the raw low level graphs in this tests suite. This automatically generates a set of visualizations with different metrics and @@ -732,24 +711,18 @@ def test_many_branches_use_ndependencies(abcde): def test_order_cycle(): - # Note: We're overriding `order` in this module to run some additional tests - # and to queue up the graphs for the distributed scheduler test below. We - # don't want to run these broken graphs on the scheduler since the test - # below is not robust to broken graphs - from dask.order import order as _order - with pytest.raises(RuntimeError, match="Cycle detected"): dask.get({"a": (f, "a")}, "a") # we encounter this in `get` with pytest.raises(RuntimeError, match="Cycle detected"): - _order({"a": (f, "a")}) # trivial self-loop + order({"a": (f, "a")}) # trivial self-loop with pytest.raises(RuntimeError, match="Cycle detected"): - _order({("a", 0): (f, ("a", 0))}) # non-string + order({("a", 0): (f, ("a", 0))}) # non-string with pytest.raises(RuntimeError, match="Cycle detected"): - _order({"a": (f, "b"), "b": (f, "c"), "c": (f, "a")}) # non-trivial loop + order({"a": (f, "b"), "b": (f, "c"), "c": (f, "a")}) # non-trivial loop with pytest.raises(RuntimeError, match="Cycle detected"): - _order({"a": (f, "b"), "b": (f, "c"), "c": (f, "a", "d"), "d": 1}) + order({"a": (f, "b"), "b": (f, "c"), "c": (f, "a", "d"), "d": 1}) with pytest.raises(RuntimeError, match="Cycle detected"): - _order({"a": (f, "b"), "b": (f, "c"), "c": (f, "a", "d"), "d": (f, "b")}) + order({"a": (f, "b"), "b": (f, "c"), "c": (f, "a", "d"), "d": (f, "b")}) def test_order_empty(): @@ -1990,7 +1963,7 @@ def test_gh_3055_explicit(abcde): (c, 0, 0): (f, (b, 0), (a, 2), (a, 1)), (d, 0, 0): (f, (c, 0, 1), (c, 0, 0)), (d, 0, 1): (f, (c, 0, 1), (c, 0, 0)), - ("f", 1, 1): (f, (d, 0, 1)), + (f, 1, 1): (f, (d, 0, 1)), (c, 1, 0): (f, (b, 1, 0), (b, 1, 2)), (c, 0, 2): (f, (b, 0, 0), (b, 0, 1)), (e, 0): (f, (c, 1, 0), (c, 0, 2)), @@ -2499,37 +2472,3 @@ def test_do_not_mutate_input(): assert_topological_sort(dsk, o) assert dsk == dsk_copy assert dependencies == dependencies_copy - - -if HAS_DISTRIBUTED: - from dask.tests.test_distributed import gen_cluster - - @pytest.mark.slow - @gen_cluster(client=True, nthreads=[]) - async def test_order_on_distributed_cluster(c, s): - # Note: For the GRAPHS_FOR_DISTRIBUTED to be populated, the above tests - # have to run first - for testname, dsk in GRAPHS_FOR_DISTRIBUTED: - if not dsk: - continue - - while s.tasks: - await asyncio.sleep(0.01) - _, dependents = get_deps(dsk) - output_keys = [k for k, v in dependents.items() if not v] - futs = c.get(dsk, output_keys, sync=False) # noqa: F841 - while not s.tasks: - await asyncio.sleep(0.01) - actual = {ts.key: ts.priority for ts in s.tasks.values()} - actual_keys = sorted(actual, key=actual.__getitem__) - - expected = dask.order.order(dsk) - expected_keys = sorted(expected, key=expected.__getitem__) - try: - assert actual_keys == expected_keys, ( - testname, - actual_keys, - expected_keys, - ) - finally: - del futs From 33a3f4b35cec5247e0320f42461b9b96aae0ebd2 Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Fri, 16 Aug 2024 03:35:39 -0500 Subject: [PATCH 65/72] Ensure we test against numpy 2 in CI (#11182) Co-authored-by: Patrick Hoefler <61934744+phofl@users.noreply.github.com> --- continuous_integration/environment-3.12.yaml | 5 +++-- dask/array/blockwise.py | 2 +- dask/array/overlap.py | 6 +++--- dask/array/routines.py | 8 ++++---- dask/array/slicing.py | 4 ++-- dask/base.py | 8 ++++---- dask/dataframe/backends.py | 2 +- dask/dataframe/core.py | 2 +- dask/dataframe/partitionquantiles.py | 4 ++-- dask/tests/test_tokenize.py | 1 + docs/source/develop.rst | 2 +- 11 files changed, 23 insertions(+), 21 deletions(-) diff --git a/continuous_integration/environment-3.12.yaml b/continuous_integration/environment-3.12.yaml index 425969ef5dc..341fa3722e0 100644 --- a/continuous_integration/environment-3.12.yaml +++ b/continuous_integration/environment-3.12.yaml @@ -23,13 +23,14 @@ dependencies: - moto<5 # Optional dependencies - mimesis - - numpy + - numpy >=2 # only tested here - pandas - numba - flask - fastparquet - h5py - - pytables + # Temporarily removing to allow `numpy >=2` to be installed + # - pytables - zarr # `tiledb-py=0.17.5` lead to strange seg faults in CI, However 0.18 is needed for 3.11 # https://github.com/dask/dask/pull/9569 diff --git a/dask/array/blockwise.py b/dask/array/blockwise.py index 4a2a7aed779..300120e8b12 100644 --- a/dask/array/blockwise.py +++ b/dask/array/blockwise.py @@ -122,7 +122,7 @@ def blockwise( >>> z = blockwise(sequence_dot, '', a, 'i', b, 'i', dtype='f8') >>> z.compute() - 250 + np.int64(250) Add new single-chunk dimensions with the ``new_axes=`` keyword, including the length of the new dimension. New dimensions will always be in a single diff --git a/dask/array/overlap.py b/dask/array/overlap.py index 76e8680a384..ff4e2de03c6 100644 --- a/dask/array/overlap.py +++ b/dask/array/overlap.py @@ -613,11 +613,11 @@ def map_overlap( >>> x = da.ones(10, dtype='int') >>> block_args = dict(chunks=(), drop_axis=0) >>> da.map_blocks(func, x, **block_args).compute() - 10 + np.int64(10) >>> da.map_overlap(func, x, **block_args, boundary='reflect').compute() - 10 + np.int64(10) >>> da.map_overlap(func, x, **block_args, depth=1, boundary='reflect').compute() - 12 + np.int64(12) For functions that may not handle 0-d arrays, it's also possible to specify ``meta`` with an empty array matching the type of the expected result. In diff --git a/dask/array/routines.py b/dask/array/routines.py index 38fafa6da6b..4780f71d72d 100644 --- a/dask/array/routines.py +++ b/dask/array/routines.py @@ -1312,7 +1312,7 @@ def histogramdd(sample, bins, range=None, normed=None, weights=None, density=Non >>> w = da.random.uniform(0, 1, size=(1000,), chunks=x.chunksize[0]) >>> h, edges = da.histogramdd(x, bins=bins, range=ranges, weights=w) >>> np.isclose(h.sum().compute(), w.sum().compute()) - True + np.True_ Using a sequence of 1D arrays as the input: @@ -2270,11 +2270,11 @@ def aligned_coarsen_chunks(chunks: list[int], multiple: int) -> tuple[int, ...]: Examples -------- >>> aligned_coarsen_chunks(chunks=(1, 2, 3), multiple=4) - (4, 2) + (np.int64(4), np.int64(2)) >>> aligned_coarsen_chunks(chunks=(1, 20, 3, 4), multiple=4) - (4, 20, 4) + (np.int64(4), np.int64(20), np.int64(4)) >>> aligned_coarsen_chunks(chunks=(20, 10, 15, 23, 24), multiple=10) - (20, 10, 20, 20, 20, 2) + (np.int64(20), np.int64(10), np.int64(20), np.int64(20), np.int64(20), np.int64(2)) """ overflow = np.array(chunks) % multiple excess = overflow.sum() diff --git a/dask/array/slicing.py b/dask/array/slicing.py index 0a0421fd79f..a1bf99e8586 100644 --- a/dask/array/slicing.py +++ b/dask/array/slicing.py @@ -531,9 +531,9 @@ def issorted(seq): """Is sequence sorted? >>> issorted([1, 2, 3]) - True + np.True_ >>> issorted([3, 1, 2]) - False + np.False_ """ if len(seq) == 0: return True diff --git a/dask/base.py b/dask/base.py index 7343152c8ba..ecd8a98a5d1 100644 --- a/dask/base.py +++ b/dask/base.py @@ -579,9 +579,9 @@ def optimize(*args, traverse=True, **kwargs): >>> a2, b2 = dask.optimize(a, b) >>> a2.compute() == a.compute() - True + np.True_ >>> b2.compute() == b.compute() - True + np.True_ """ collections, repack = unpack_collections(*args, traverse=traverse) if not collections: @@ -634,12 +634,12 @@ def compute( >>> a = da.arange(10, chunks=2).sum() >>> b = da.arange(10, chunks=2).mean() >>> dask.compute(a, b) - (45, 4.5) + (np.int64(45), np.float64(4.5)) By default, dask objects inside python collections will also be computed: >>> dask.compute({'a': a, 'b': b, 'c': 1}) - ({'a': 45, 'b': 4.5, 'c': 1},) + ({'a': np.int64(45), 'b': np.float64(4.5), 'c': 1},) """ collections, repack = unpack_collections(*args, traverse=traverse) diff --git a/dask/dataframe/backends.py b/dask/dataframe/backends.py index 1ad06959c9a..90ea18868f1 100644 --- a/dask/dataframe/backends.py +++ b/dask/dataframe/backends.py @@ -282,7 +282,7 @@ def make_meta_object(x, index=None): >>> make_meta_object(('a', 'f8')) Series([], Name: a, dtype: float64) >>> make_meta_object('i8') - 1 + np.int64(1) """ if is_arraylike(x) and x.shape: diff --git a/dask/dataframe/core.py b/dask/dataframe/core.py index 072f7af0bbd..39fab085865 100644 --- a/dask/dataframe/core.py +++ b/dask/dataframe/core.py @@ -1408,7 +1408,7 @@ def reduction( >>> res = ddf.x.reduction(count_greater, aggregate=lambda x: x.sum(), ... chunk_kwargs={'value': 25}) >>> res.compute() - 25 + np.int64(25) Aggregate both the sum and count of a Series at the same time: diff --git a/dask/dataframe/partitionquantiles.py b/dask/dataframe/partitionquantiles.py index 9ed891e57e8..ca82a0d42d6 100644 --- a/dask/dataframe/partitionquantiles.py +++ b/dask/dataframe/partitionquantiles.py @@ -252,13 +252,13 @@ def percentiles_to_weights(qs, vals, length): between the first and second percentiles, and then scaled by length: >>> 0.5 * length * (percentiles[1] - percentiles[0]) - 125.0 + np.float64(125.0) The second weight uses the difference of percentiles on both sides, so it will be twice the first weight if the percentiles are equally spaced: >>> 0.5 * length * (percentiles[2] - percentiles[0]) - 250.0 + np.float64(250.0) """ if length == 0: return () diff --git a/dask/tests/test_tokenize.py b/dask/tests/test_tokenize.py index 49461e541f8..fc422d99224 100644 --- a/dask/tests/test_tokenize.py +++ b/dask/tests/test_tokenize.py @@ -1271,6 +1271,7 @@ def test_tokenize_random_functions(module): """random.random() and other methods of the global random state do not compare as equal to themselves after a pickle roundtrip""" module = eval(module) + module.seed(2) a = module.random b = pickle.loads(pickle.dumps(a)) diff --git a/docs/source/develop.rst b/docs/source/develop.rst index bf48d4d414a..57880f998c5 100644 --- a/docs/source/develop.rst +++ b/docs/source/develop.rst @@ -254,7 +254,7 @@ after the line. .. _numpydoc: https://numpydoc.readthedocs.io/en/latest/format.html#docstring-standard -Docstrings are tested under Python 3.11 on GitHub Actions. You can test +Docstrings are tested under Python 3.12 on GitHub Actions. You can test docstrings with pytest as follows:: py.test dask --doctest-modules From 8bc8c496575fe9714fd6f17120269db28f16b793 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Fri, 16 Aug 2024 16:20:18 +0200 Subject: [PATCH 66/72] Implement automatic rechunking for shuffle (#11311) Co-authored-by: Deepak Cherian --- dask/array/_shuffle.py | 66 +++++++++++++++++++++++++++++++- dask/array/core.py | 7 ++-- dask/array/tests/test_shuffle.py | 36 +++++++++++++---- 3 files changed, 97 insertions(+), 12 deletions(-) diff --git a/dask/array/_shuffle.py b/dask/array/_shuffle.py index 6e3dd2e4ab2..fa1c6ccd659 100644 --- a/dask/array/_shuffle.py +++ b/dask/array/_shuffle.py @@ -1,7 +1,9 @@ from __future__ import annotations import copy +import math from itertools import count, product +from typing import Literal import numpy as np @@ -13,7 +15,7 @@ from dask.highlevelgraph import HighLevelGraph -def shuffle(x, indexer: list[list[int]], axis): +def shuffle(x, indexer: list[list[int]], axis: int, chunks: Literal["auto"] = "auto"): """ Reorders one dimensions of a Dask Array based on an indexer. @@ -34,6 +36,12 @@ def shuffle(x, indexer: list[list[int]], axis): each group will end up in exactly one chunk. axis: int The axis to shuffle along. + chunks: "auto" + Hint on how to rechunk if single groups are becoming too large. The default is + to split chunks along the other dimensions evenly to keep the chunksize + consistent. The rechunking is done in a way that ensures that non all-to-all + network communication is necessary, chunks are only split and not combined with + other chunks. Examples -------- @@ -66,6 +74,9 @@ def shuffle(x, indexer: list[list[int]], axis): f"Shuffling only allowed with known chunk sizes. {unknown_chunk_message}" ) assert isinstance(axis, int), "axis must be an integer" + _validate_indexer(x.chunks, indexer, axis) + + x = _rechunk_other_dimensions(x, max(map(len, indexer)), axis, chunks) token = tokenize(x, indexer, axis) out_name = f"shuffle-{token}" @@ -79,7 +90,54 @@ def shuffle(x, indexer: list[list[int]], axis): return Array(graph, out_name, chunks, meta=x) -def _shuffle(chunks, indexer, axis, in_name, out_name, token): +def _rechunk_other_dimensions( + x: Array, longest_group: int, axis: int, chunks: Literal["auto"] +) -> Array: + assert chunks == "auto", "Only auto is supported for now" + chunksize_tolerance = config.get("array.shuffle.chunksize-tolerance") + + if longest_group <= max(x.chunks[axis]) * chunksize_tolerance: + # We are staying below our threshold, so don't rechunk + return x + + # How much the chunksizes on our shuffle axis will increase + chunksize_inc_factor = longest_group / max(x.chunks[axis]) + + new_chunks = [] + for i in range(len(x.chunks)): + if i == axis: + new_chunks.append(x.chunks[i]) + continue + + new_chunksizes = [] + # calculate what the max chunk size in this dimension is and split every + # chunk that is larger than that. We split the increase factor evenly + # between all dimensions that are not shuffled. + up_chunksize_limit_for_dim = max(x.chunks[i]) / ( + chunksize_inc_factor ** (1 / (len(x.chunks) - 1)) + ) + for c in x.chunks[i]: + if c > chunksize_tolerance * up_chunksize_limit_for_dim: + factor = math.ceil(c / up_chunksize_limit_for_dim) + + # Ensure that we end up at least with chunksize 1 + factor = min(factor, c) + + chunksize, remainder = divmod(c, factor) + nc = [chunksize] * factor + for ii in range(remainder): + # Add remainder parts to the first few chunks + nc[ii] += 1 + new_chunksizes.extend(nc) + + else: + new_chunksizes.append(c) + new_chunks.append(tuple(new_chunksizes)) + + return x.rechunk(tuple(new_chunks)) + + +def _validate_indexer(chunks, indexer, axis): if not isinstance(indexer, list) or not all(isinstance(i, list) for i in indexer): raise ValueError("indexer must be a list of lists of positional indices") @@ -93,6 +151,10 @@ def _shuffle(chunks, indexer, axis, in_name, out_name, token): f"Indexer contains out of bounds index. Dimension only has {sum(chunks[axis])} elements." ) + +def _shuffle(chunks, indexer, axis, in_name, out_name, token): + _validate_indexer(chunks, indexer, axis) + if len(indexer) == len(chunks[axis]): # check if the array is already shuffled the way we want ctr = 0 diff --git a/dask/array/core.py b/dask/array/core.py index 3009dcb11df..a35c05f76bc 100644 --- a/dask/array/core.py +++ b/dask/array/core.py @@ -17,7 +17,7 @@ from numbers import Integral, Number from operator import add, mul from threading import Lock -from typing import Any, TypeVar, Union, cast +from typing import Any, Literal, TypeVar, Union, cast import numpy as np from numpy.typing import ArrayLike @@ -2765,7 +2765,8 @@ def rechunk( def shuffle( self, indexer: list[list[int]], - axis, + axis: int, + chunks: Literal["auto"] = "auto", ): """Reorders one dimensions of a Dask Array based on an indexer. @@ -2777,7 +2778,7 @@ def shuffle( """ from dask.array._shuffle import shuffle - return shuffle(self, indexer, axis) + return shuffle(self, indexer, axis, chunks) @property def real(self): diff --git a/dask/array/tests/test_shuffle.py b/dask/array/tests/test_shuffle.py index 2882cbc8752..86cadcfc2e6 100644 --- a/dask/array/tests/test_shuffle.py +++ b/dask/array/tests/test_shuffle.py @@ -6,6 +6,7 @@ import dask import dask.array as da from dask.array import assert_eq, shuffle +from dask.array._shuffle import _rechunk_other_dimensions from dask.core import flatten @@ -20,19 +21,19 @@ def darr(arr): @pytest.mark.parametrize( - "indexer, chunks", + "indexer, chunks, other_chunks", [ - ([[1, 5, 6], [0, 2, 3, 4, 7]], (3, 5)), - ([[1, 5, 6], [0, 3], [4, 2, 7]], (5, 3)), - ([[1], [0, 6, 5, 3, 2, 4], [7]], (1, 6, 1)), - ([[1, 5, 1, 5, 1, 5], [1, 6, 4, 2, 7]], (6, 5)), + ([[1, 5, 6], [0, 2, 3, 4, 7]], (3, 5), (2, 1)), + ([[1, 5, 6], [0, 3], [4, 2, 7]], (5, 3), (2, 1)), + ([[1], [0, 6, 5, 3, 2, 4], [7]], (1, 6, 1), (1, 1, 1)), + ([[1, 5, 1, 5, 1, 5], [1, 6, 4, 2, 7]], (6, 5), (1, 1, 1)), ], ) -def test_shuffle(arr, darr, indexer, chunks): +def test_shuffle(arr, darr, indexer, chunks, other_chunks): result = darr.shuffle(indexer, axis=1) expected = arr[:, list(flatten(indexer))] assert_eq(result, expected) - assert result.chunks[0] == darr.chunks[0] + assert result.chunks[0] == other_chunks assert result.chunks[1] == chunks @@ -94,3 +95,24 @@ def test_shuffle_no_op_with_correct_indexer(): result = arr.shuffle(indexer, axis=0) assert result.dask == arr.dask assert_eq(arr, result) + + +def test_resize_other_dimensions(): + arr = da.random.random((250, 50), chunks=((45, 100, 38, 67), 10)) + result = _rechunk_other_dimensions(arr, 20, 1, "auto") + assert result.chunks == ((45, 50, 50, 38, 34, 33), (10,) * 5) + assert_eq(arr, result) + + arr = da.random.random((250, 50, 20), chunks=((45, 100, 38, 67), 10, 10)) + result = _rechunk_other_dimensions(arr, 20, 1, "auto") + assert result.chunks == ((45, 50, 50, 38, 67), (10,) * 5, (5,) * 4) + assert_eq(arr, result) + + result = _rechunk_other_dimensions(arr, 40, 1, "auto") + assert result.chunks == ((45, 50, 50, 38, 34, 33), (10,) * 5, (5,) * 4) + assert_eq(arr, result) + + arr = da.random.random((250, 50, 5), chunks=((45, 100, 38, 67), 10, 1)) + result = _rechunk_other_dimensions(arr, 40, 1, "auto") + assert result.chunks == ((45, 50, 50, 38, 34, 33), (10,) * 5, (1,) * 5) + assert_eq(arr, result) From 058f1b69ee3115d63046acf60e537d358462a466 Mon Sep 17 00:00:00 2001 From: "Richard (Rick) Zamora" Date: Fri, 16 Aug 2024 07:52:35 -0700 Subject: [PATCH 67/72] Upgrade gpuCI and fix Dask Array failures with "cupy" backend (#11309) --- continuous_integration/gpuci/axis.yaml | 2 +- dask/array/slicing.py | 5 +++-- dask/dataframe/tests/test_dataframe.py | 2 ++ 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/continuous_integration/gpuci/axis.yaml b/continuous_integration/gpuci/axis.yaml index 25b4c78ef9f..7b30d50651a 100644 --- a/continuous_integration/gpuci/axis.yaml +++ b/continuous_integration/gpuci/axis.yaml @@ -9,6 +9,6 @@ LINUX_VER: - ubuntu20.04 RAPIDS_VER: -- "24.08" +- "24.10" excludes: diff --git a/dask/array/slicing.py b/dask/array/slicing.py index a1bf99e8586..c18e529392c 100644 --- a/dask/array/slicing.py +++ b/dask/array/slicing.py @@ -565,8 +565,9 @@ def take(outname, inname, chunks, index, axis=0): if not np.isnan(chunks[axis]).any(): from dask.array._shuffle import _shuffle + from dask.array.utils import arange_safe, asarray_safe - arange = np.arange(np.sum(chunks[axis])) + arange = arange_safe(np.sum(chunks[axis]), like=index) if len(index) == len(arange) and np.abs(index - arange).sum() == 0: # TODO: This should be a real no-op, but the call stack is # too deep to do this efficiently for now @@ -577,7 +578,7 @@ def take(outname, inname, chunks, index, axis=0): average_chunk_size = int(sum(chunks[axis]) / len(chunks[axis])) indexer = [] - index = np.asarray(index) + index = asarray_safe(index, like=index) for i in range(0, len(index), average_chunk_size): indexer.append(index[i : i + average_chunk_size].tolist()) diff --git a/dask/dataframe/tests/test_dataframe.py b/dask/dataframe/tests/test_dataframe.py index f0f65f72d89..a2cc707c2ed 100644 --- a/dask/dataframe/tests/test_dataframe.py +++ b/dask/dataframe/tests/test_dataframe.py @@ -3610,6 +3610,7 @@ def test_cov_series(): @pytest.mark.gpu +@pytest.mark.skip(reason="https://github.com/rapidsai/cudf/issues/16560") @pytest.mark.parametrize( "numeric_only", [None, True, False], @@ -3682,6 +3683,7 @@ def test_corr(): @pytest.mark.gpu +@pytest.mark.skip(reason="https://github.com/rapidsai/cudf/issues/16560") def test_corr_gpu(): cudf = pytest.importorskip("cudf") From d39cd9968c33cfaed613c379242fe4523da89b03 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Fri, 16 Aug 2024 18:31:37 +0200 Subject: [PATCH 68/72] Rename chunksize-tolerance option (#11317) --- dask/array/_shuffle.py | 6 +++--- dask/array/tests/test_shuffle.py | 2 +- dask/config.py | 1 + dask/dask-schema.yaml | 18 +++++++----------- dask/dask.yaml | 3 +-- 5 files changed, 13 insertions(+), 17 deletions(-) diff --git a/dask/array/_shuffle.py b/dask/array/_shuffle.py index fa1c6ccd659..78d2faa4292 100644 --- a/dask/array/_shuffle.py +++ b/dask/array/_shuffle.py @@ -58,7 +58,7 @@ def shuffle(x, indexer: list[list[int]], axis: int, chunks: Literal["auto"] = "a the number of chunks small. The tolerance of increasing the chunk size is controlled by the configuration - "array.shuffle.chunksize-tolerance". The default value is 1.25. + "array.chunk-size-tolerance". The default value is 1.25. >>> y.chunks ((2,), (5, 3)) @@ -94,7 +94,7 @@ def _rechunk_other_dimensions( x: Array, longest_group: int, axis: int, chunks: Literal["auto"] ) -> Array: assert chunks == "auto", "Only auto is supported for now" - chunksize_tolerance = config.get("array.shuffle.chunksize-tolerance") + chunksize_tolerance = config.get("array.chunk-size-tolerance") if longest_group <= max(x.chunks[axis]) * chunksize_tolerance: # We are staying below our threshold, so don't rechunk @@ -167,7 +167,7 @@ def _shuffle(chunks, indexer, axis, in_name, out_name, token): indexer = copy.deepcopy(indexer) - chunksize_tolerance = config.get("array.shuffle.chunksize-tolerance") + chunksize_tolerance = config.get("array.chunk-size-tolerance") chunk_size_limit = int(sum(chunks[axis]) / len(chunks[axis]) * chunksize_tolerance) # Figure out how many groups we can put into one chunk diff --git a/dask/array/tests/test_shuffle.py b/dask/array/tests/test_shuffle.py index 86cadcfc2e6..f0f717de10e 100644 --- a/dask/array/tests/test_shuffle.py +++ b/dask/array/tests/test_shuffle.py @@ -40,7 +40,7 @@ def test_shuffle(arr, darr, indexer, chunks, other_chunks): @pytest.mark.parametrize("tol, chunks", ((1, (3, 2, 3)), (1.4, (5, 3)))) def test_shuffle_config_tolerance(arr, darr, tol, chunks): indexer = [[1, 5, 6], [0, 3], [4, 2, 7]] - with dask.config.set({"array.shuffle.chunksize-tolerance": tol}): + with dask.config.set({"array.chunk-size-tolerance": tol}): result = darr.shuffle(indexer, axis=1) expected = arr[:, [1, 5, 6, 0, 3, 4, 2, 7]] assert_eq(result, expected) diff --git a/dask/config.py b/dask/config.py index 1376ae50f65..ecb0f37d031 100644 --- a/dask/config.py +++ b/dask/config.py @@ -721,6 +721,7 @@ def expand_environment_variables(config: Any) -> Any: "dataframe.shuffle.algorithm": "dataframe.shuffle.method", "dataframe.shuffle-compression": "dataframe.shuffle.compression", "admin.traceback.shorten.what": "admin.traceback.shorten", # changed in 2023.9.0 + "array.shuffle.chunksize-tolerance": "array.chunk-size-tolerance", } diff --git a/dask/dask-schema.yaml b/dask/dask-schema.yaml index e05aac03835..548c8e96475 100644 --- a/dask/dask-schema.yaml +++ b/dask/dask-schema.yaml @@ -119,6 +119,13 @@ properties: description: | The default chunk size to target. Default is "128MiB". + chunk-size-tolerance: + type: number + description: | + Upper tolerance for different algorithms when creating output chunks. + Default is 1.25. This means that the algorithms can exceed + the average input chunk size along this dimension by 25%. + rechunk: type: object properties: @@ -135,17 +142,6 @@ properties: The graph growth factor above which task-based shuffling introduces an intermediate step. - shuffle: - type: object - properties: - - chunksize-tolerance: - type: number - description: | - Upper tolerance for the shuffle algorithm when creating output chunks. - Default is 1.25. This means that the shuffle algorithm can exceed - the average input chunk size along this dimension by 25%. - svg: type: object properties: diff --git a/dask/dask.yaml b/dask/dask.yaml index 6e35c10e444..e8d00aa14b6 100644 --- a/dask/dask.yaml +++ b/dask/dask.yaml @@ -21,11 +21,10 @@ dataframe: array: backend: "numpy" # Backend array library for input IO and data creation chunk-size: "128MiB" + chunk-size-tolerance: 1.25 # Tolerance for different algorithms when creating output chunks. rechunk: method: "tasks" # Rechunking method to use threshold: 4 - shuffle: - chunksize-tolerance: 1.25 # Tolerance for the shuffle algorithm when creating output chunks. svg: size: 120 # pixels slicing: From 651464d9069d0a183a47f0d0a43189eb8f293fa3 Mon Sep 17 00:00:00 2001 From: Patrick Hoefler <61934744+phofl@users.noreply.github.com> Date: Fri, 16 Aug 2024 22:46:19 +0200 Subject: [PATCH 69/72] Add changelog entry for reshape and ordering improvements (#11324) --- docs/source/changelog.rst | 46 ++++++++++++++++++ .../changelog/reshape-constant-memory.png | Bin 0 -> 61297 bytes .../changelog/reshape-memory-increase.png | Bin 0 -> 74624 bytes 3 files changed, 46 insertions(+) create mode 100644 docs/source/images/changelog/reshape-constant-memory.png create mode 100644 docs/source/images/changelog/reshape-memory-increase.png diff --git a/docs/source/changelog.rst b/docs/source/changelog.rst index 63fcf15bab3..4d05c7aa3e8 100644 --- a/docs/source/changelog.rst +++ b/docs/source/changelog.rst @@ -9,6 +9,51 @@ Changelog Highlights ^^^^^^^^^^ +Improve output chunksizes for reshaping Dask Arrays +""""""""""""""""""""""""""""""""""""""""""""""""""" + +Reshaping a Dask Array oftentimes squashed the dimensions to reshape into a single +chunk. This caused very large output chunks and subsequently a lot of out of memory +errors and performance issues. + +.. code-block:: + + arr = da.ones(shape=(1000, 100, 48_000), chunks=(1000, 100, 83)) + arr.reshape(1000, 100, 4, 12_000) + +Previously, this put the last dimension into a single chunk of size 12_000. + +.. image:: images/changelog/reshape-memory-increase.png + :width: 75% + :align: center + :alt: Size of each individual chunk increases to over 1GB + +The new algorithm will ensure that the chunk-size between in- and output is kept +the same. This will avoid large increases in chunk-size and fragmentation of chunks. + +.. image:: images/changelog/reshape-constant-memory.png + :width: 75% + :align: center + :alt: Size of each individual chunk stays the same + +Improve scheduling efficiency for Xarray Rechunk-GroupBy-Reduce patterns +"""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""" + +The scheduler previously created an inefficient execution graph for Xarray GroupBy-Reduction +patterns that use the cohorts strategy: + +.. code-block:: python + + import xarray as xr + + arr = xr.open_zarr(...) + arr.chunk(time=TimeResampler("ME")).groupby("time.month").mean() + +An issue in the algorithm that creates the execution order of the task graph +lead to an inefficient execution strategy that accumulates a lot of unnecessary memory on +the cluster. The improvement is very similar to +:ref:`the previous ordering improvement in 2024.08.0 `. + Drop support for Python 3.9 """"""""""""""""""""""""""" @@ -43,6 +88,7 @@ the same to avoid fragmentation of chunks or a large increase in chunk-size. See :pr:`11262` and :pr:`11267` by `Patrick Hoefler`_ for more details and performance benchmarks. +.. _label.xarray_groupby_ordering: Improve scheduling efficiency for Xarray GroupBy-Reduce patterns """""""""""""""""""""""""""""""""""""""""""""""""""""""""""""""" diff --git a/docs/source/images/changelog/reshape-constant-memory.png b/docs/source/images/changelog/reshape-constant-memory.png new file mode 100644 index 0000000000000000000000000000000000000000..18e5517b1c7c2a393539a0f73c1f0451034fbdd3 GIT binary patch literal 61297 zcmeFZby!q=+ct`TbR!~NBGL`g4FZxP4HD8YBi$g~A>9pv(%oIs4bmYELwD^p-tl<9 z-+MpDxA)(Bju~dns$X4kUgvc!f)wSY&`}6cU|?X-rC*CH!@xYIgMoRZii7~{0l{VC z!@!`Fn2CugN{fk+E82jK%`A;zU|t8s#v>}e-G2J%!4=Gl%t-NsHj`U~_fd{7sjvv| z43hLGMNuU9NA$t9-t@u2NUC-PD8-K>tH_^PhJ1RAT2CJ0UC;DXTw+~E52@j1qlx=q zAcbd1;No^Og9-U7AbjmurFuxT&e3BCq#q;A~bMta3 zDvBYoK>rgRkH{WTsAeVemWcA#mKeV>R5%m$*5u&yRrnzA44Un@K$b~D&(Udov7GKR zmJVxJa5aYl0-v4#)wV7LJA*uW+*B~GYpHPpRaz0fG)nX}?xm&KeTwf$Ai>ZRU*?pR z(;TdaAI~rg-ZLP_oe$$**A7pe#X$t9@0f4!N^>pPaYWs7T{j1SL*>S{c~Ua2Yexc2 z#)j=VM_Fi&W{gTUksN91dsJs%O#EL@8)qQeI9NrV61}jw^)`M64@1@pqo0rdxb4$p zN*E$1*b+34shI(;2h5g!+QX^~!`gg8Q1N;(@>sGJ zP5lwyGW`4}{6jb|(-tK?f@N>`NZ3xj$Lf!5LL&$TisB4B*#$RH$!qY=h^h05gp zZTZC({6(wb2qHooJqxU$w>6UYA=cw8(XWwDb1)M{Rk3hwyc#oLUOo%`+6pm@^>hb*4;`IUHRp$JMOP7Z8= zlxC#EPmHmEMoB@#kI(4*y;wV5>ASJ&KVy1OR4j)pFU`4Q8-hI_-oj|wSRN)@xxDuq>>8gfMbF8ZDP()KzQ6madMtMAisEM=Y@IFsmhGh$EoY2u z%$FF2aup3*jcNu3ha$^5%WTVCtFc@z9?%J#h|p4i3O$ro~Xi{>n7z)0Yd%AkEax z#2EX?Ax4-S_nJP3B-?&OGUiFFWlVc)c}!pI5bfD0cKxGNIfXp@QH7C%@s^hsFDqiM z`jTUJV;f?7R5tQ!3l86*s1T|^3fz>8@?8pE6oAyMl-;xZmDrWEa;inGT4wR8rfp4a zQ#L+sJ{jO)-Tcg32+ATYY*xN2y=FX9LYG4)L;u`O8YW9v!nvIUN(xHi;~ciYuu!Vf zu7ua*t70;3`tXufJ@kVl?HZe*xR%bGnpT$k>jp7b?H)y%HzrN;K0~-VQA}^SC87DFy+ixq&*N|J-z52jN-dis z8zsA;`;}pyb<~mInmq1I=hXAIocW*Cw3g;3bSw)dvz+*R)B439L0$8t?BxXg4C94E zg-aMo;wmm*2JRQ%ppzQ&drqi7kkQ zh>`^m_(`|Vwy(Fx`%e2tNH|G8kvf$a3ehHUW|v`M{3QgQnMjq3$v9@hpvP2}4)f-s{VNEpeA%ST7s_eb+i ze9j$iCLXL05-GOY*7J&uwzuJzEAZ34QNGi#m?DYGR$1Uxug#(7I}nUo z=*s2Eyo@*hvhpPZ?QOkZ!+Li_oKW2Tt1?cIh16%A(R-^}t9Gl4ng&n_lj_SMTGE*^ zJ*hSM?ji3~G6%*b`=-vkpRJ%V(054l*r%}#0T%~FWp7Wz~ z`vb?h_nyLQ!Yf|Ph>sD;h>v+D)b$mWY@Rv3IypR{T9KOdG`LrJTcE0;epMu+KB-2c z?Oxa2oR_ECRGp7e96f32GBRc@{%36 ziNHS=dGEgMmwe&d<0S;oErB3*TbA2NOZLrkRX3|7!+dH7ccnL&E^()) z0?Pu=ToP^%_p=A2)*`0Q3|vvudWC46zus4_a<45BUhiEGow=Sbxja4IJA!dso;=uY z@gx~=m3P)X7TuK?jGP?#p?|P46d&>)qZy}(&@=TRpq{jQp*%NYGWyG$>w;_c6Wg2kKxXD{HMcHL#yjKNqVCKe!W}6?9L(H~hHwX6lPM*K#_WK+i+&fu|<< z;5GLR^Kuii==_h+HUPl}8>veh%gMn2m=*>J2L2Ht%wu5d5%3XwMD+Kz|bX;rpti)Yx|KZa12J|t(del z@cq`%#>fa{YYMhQUY{Qab|8OztzioT^OPF;c_gj;{1CYRw3&*!ox0o`UPG`Ill}*= zff19F)ko-kVECPQflVtTJAHB|D@%|quaf}fuQPaoZRl=hO7dT)*jWfrs>>;oi-B#7 z$hnwUm{=$UQOL>3`E5QJ^D2u={&_j@OMueU&h8^GGqaDl3m70X^awL66ASa-_XaNIhwkN7G;=bt)DSnb0_F^8Ly(<=jsMsA|HqO4 zwD{YV>i@Ztm7VP$m;QF>e_r~|*2qQ-Yz4GwC-|TB`m^2N5B_-}KQnabzfHv-^ZaWs zFwuf2{LFtZnji|iQ1lY8j>Kl-3M#-iAZ5Q^-+;d_fe-W>_;_?t5C`wUzzD-gi;Ji@ zJ=#k}Y$P$f?s+dAlP~f*($rI@kQ`nl2gklhH|wHW{o6&k%X`%WBYTTUaY7MM@dH(F z@<1Kcr?jh{&9(y37Xzs_Ljo7`Zy^U)ZjjXDIOk2i^?}7f1eC2uVCCrMU5k|421vE&g=XMp8vMC z*04Bq^d3kPF#kH~Siqd~BL0^}qmvgFHR{6jBK$Ay0K&2PUl#5E7mj~u&Ho4C$cKX! zx;u@&Ih%gJ(0DSWHL|GfwBiQ|AoD;vNwXW1E_S~;U$Lng4kZ&>6u3R?&}Vl&+gayY zay$8`!T!4#S4jb@xlh7gr4H5fufeS(j`J&mw})aEqf%sbtcKmS0_QUw`*oXXAU{0S z{I3pQSt%94@Okp7JEZQXqr!!*2Td3A5BC=>dia~kV8NPhLW^3f>L1-!4g1wWt84YU z#k;J^Aoxv1r4 z;b7i=T4$5jerMukh;N_d>q^nz@7xa6=1UmU6X-N3_QR(+ts*;Kt^{z;`bpsHa2ys} z&Q)7wa})uF;bmAWRRsTPIL}6cH~orB2UiHc{H$6ea5%sFuNa`vM=}|$=^NmPLvulKbWTOtxw|_U~$4wsYehsHDQt;v|lnrD7r_gqh7qE;X%VZCi0esh6*(Y4F zT*?=KEN;7;?Zn1E4012oxrw)CQLes7wp>kfKmXBqd*2Q~nKhU5J%VC$P$rwj4gPz# zvJTr>2DyA} z{L=VAcu3=kyyL9d7e6`v<6f!-B`K2iJcV?Ks)kYQM*;TEOe#nU*jki&we$V`F@DVu z?{2LhzV6KEAf7&@Jb4b2c=sbX1o(xDBVTQUAnN(fAlwDORuF@NFJ08$yW2l&V574n zpJ3U-JfG46*8!6Mf}wu^k<1OtK}U-Ghhc<3yIqTAtui~Mq9iO$%17@8U^&|v#k)>f zGFub#6~A-7zWY{OZ;til*c+AUqgVh2x0Sn!6UFS&|R z6TclzM}IKzq6dcG*bcpi2sgw<8|Ma8v*1usm=rAh1DU-2iZllBpiWpdWd%gxD6#R{ zHbHZshW+-PL*;OPA*Z{u=>=F+2CScEpvcLOvPk`pc8cfudCaTI@7zs0*)172cD6mi9p-14tq@h9@%<(@Uo)?3c_=w~dR z@fe9*X3=h3ZY55J#9ok2r5k+!ckr!zW>f_6w~}U;TF(KjzLoX(_ZZW7R4T{VMico; zphf3R^!|UMih5{J0$IzR3i^dAuv^8($O`i!5!jG=4*&`QIh;SQwCX>0Jeug^7 z7P^#lPK!7y2Dt3mWUH3DbN%_rd?XF9DD{o@*XaW5ah2ZWh2$l}`7VCOkc!0l-Fgp>H#2j=;juvlP3$h5z{)afeVW0Ehndi(Zw5&ow$ zeJ8gJh>{&(!<}`BFa=IW8jYS}soaiwYe-$l)f*VBTnG zZm9CZe$CQG)z5As@IXkL=2yiamaA4cOy9W-l1g;U9c0`WwQCVVf=@0P-}3nN0p5E! z6_^Aj-evr`7sY#wHS<|s_ItvnbpJ7m?_>Z69ZUL*Qx+;eU&C6H!Oh#%3l4LQ##e9_ zKdFB{Wop=CbDVsq=JyR-0@8G`s8^>gN}YwgDm`)dGc=ehjz&KJ&DQ|7P|C(-Z&Xcd zIf;xeWMb=1YzP3ZnavxV5Ki>lYLxx@=D;lrsn*>D9wLB=2d2t%S=mtF8Vd@;5Q@em-t?0<9T&&<$1aUE|+~CTdn%y%765g zNH7fNIX;f3Q@Oufi-1~eTL8F9>NZt>QsZoIe)yb`>nk&hMYGjyA6;{9{@avMDmp_0 zUQe_MSZJT~kzoN7m1(GA#omcouS`P#XJ&V>C{VJy38(}v5|Q1vJl84$svve#xsbb}pDqU2I}IkzAJA9^Djt;QLjtax)J=1$s=h#+zXS4f8r6;;pn1zXU; z;!z2|^zR{cWeAK;Lm*W!Y(}`xLQ^6A8Oi%0}uup_p|lCqD|pMB=Vnbfbh2Ub~ssA zTTB-(Gg6d8XVV$(^N9N`5aQVazCoew>}57mCH$jC@1Xum0FKpaR2V-NT|Vw-KDe}? zP3E!9yV=T)m=7gjpJ(f5$Ua13e5^Z%{dctD#Y+VU#1KZlgVZlGm$3~mc=Z#P+idK4 z!1UDJ#ZtQ3XJ?;{qKW^LJQA(|sF)v?FW-Isqr*kGuaAXy=j&`os_cvFHol(esNDSS ze||v&bErw570dRAn)FXdl^kBLGHp`rr>p^cRHtQLkgHkGGn@A7hQCe#&Vimr2grri zsHA9pI!dVgC=VIs@K(ge}9jl0Znni ziNE^=DgJTj@6!|sfdDo4=&jjr%KP`C{juR?4232HGM76am%hX3`X40C9# zlO&eu|F82dybHC~XQZzCb^p40D(FBg$^F_$)g)N8>(tLN{lhyaqc9)U`i2ULC8fCt4fhMUp zKstOSO?Y0s6o-)b@p;$xgEsrKRkjxkj_FW9zVl(Yk_f6^d811IRL)8$ed*^@?2u`5 zeYy=vvM9D*a=%DT2IA0#vK~^c%OP5m3M`h)sJ`c_T&T+LC}Hs^^bCvu#%E#a;oh;P z1A|W`hh)~YA8Ltg3ahC9G49vT^+eaxT=!!WzHrjad(2s_K-4M}uf}AV>)Lz}8!!bg z$xXpuI_>HMSrl&6+wX2jhCW+wPP&avwkWRCd3fZm-6MgI zPQ>lr9Xew-g-OCY;Lf)`m!aKqpIqO?>o7ncA9p`KEYv`g?lcSdpGn5r5m8XZ;B~|K z$w0K4V6l$$ewnxQw)AdMik#W>xPs8y{mtEl^L`&)dQ4j7VzpYqY2m)+&5obc)O{R% z(}W8T#zEsvG10~G<#J?#I_)1!A1>il=l?+e z!GQOABlK$WC3K#53cl2bFx9^t4zcM#=eA>RJQOVz!=W3Gu&7gZ&y2EqlfX8vW8X90 zw&6D1ho`35W>xB;aIu+MbN`p#CAFCYB=Ij1nB9q2jCX;Jo6<$97E8BH+JFR@rp$P2R7SdSiyo|vu za9R6Y)v_+bT($VBENb@_U^8OX&G3~{-%zVeefOFG{caj3xBz3Z-BU8_Cxb9jcicy> z6M#W755!73>4NUwBvFcq9@_n}gfuGUMGR%f`;Lp&C84gIrc>tMN_JcdD;#uj?@p_x zbjOQ66U89PYJ&a7$5659?~XkhqNaMRY7%T>6maSLs}}O6lB-%Ywl(I8xJfl`w2Dua zcY#is*tjbIow_)hT>JyN{L!@uBxt7)zY_ZOdETF-|KdmhPAPrt|I``@$UKz zH4dP_tS=_j-usy)nGwKB17_x;t8X&ia#vN;tvEwz3jMYtLeR!g*Nta7)ibWb-s77z z8J}@Vax9T$kHSSiYx(7@MbSFGt=YC;V64Yg&i?hT)tVfm)D=e=Tfw4o&D?}bMRhVT zqdI-lf6KzP!TQ|$H@0MM0`!}Y83F6jva1hR57O|gbCKWWVdIxP_yY1EpTN?_*m&6P zg`c~0jZGz)zgvTu1AAg7NxuwSEEgbx9S)D%ik~Z!+B@A+P_|8~j6*VxWly3e;N5F>S3DlvG zwVOYoi`Ha_=zGq_SR*K#pF^!-u>LcJX75lxdmbRiwd0Fr>lT~19jt79_7kXcZzwlx z`k8mRcB=HVdI{v1#S7oeg2t{3&y8cV-ICN3-wDzRn7OOE_NS&z7XK}sx`6Sbo>)>5 zHM%2tyZ;B?{R2%2qXVct(PISY%ZZY?pqHqxjLe6i*tgAhQv&REzR7~_NxKtuZ;NF; z$)=0Ze zu4RE*KY4(Pc{b5@Wn?ZS5{6{$hWy}WINtr-8~@^{4yG_L8gq!A((-|-tQJq6nSqh& zRnBS7S@iH_0%mzr*LfZ9V%DOh4&YShit2a1g2FhbK3c3x27RbhbnOkV zz61zn2dEFUv(GkypdymkDQ2II;^FJqDScE3Dpt@WWzqXI=d>0^MD2O}od(*zE#Q5; zp=6-XDLU6oYf;4zFDTvp92tQa2xJ5CkfJ;M(NQ}?9GBdu<79E(s4o# zS~~|qJw$C<79izV7q9C=+sN$!5SEbD5M~E}9=6{tZUB8;I!&&cF$k#x?AQV{$_0I= zjz5j&c;5s6N(xlF;=g=-%Qbxl9+kki7jQZKm@*FKUo3X*??O68CU*IXnLJ6R?<Qyv!T?QOL{bw*CQH%69e7J*8^Ke!v@omKVHYm0#_sF4P0n$5xp zX)#}#)Fu6dU|(2@>&<@6rsUM!BtW~^fB?XV9?}jjX*x%V67X=(?G*Q{R>QNV_4Xv-j3!q~;@SJq9F}b{ThipWwg--=OqGfd zkNN3Dr2m(Kv=4PCTdd-}%NDv6-64}e^iCW|=J%7C1CyE0B;DNsB;KGQf~|`t&$rK^ zCVx;MI+y20skje+Xu!K$SkWGVZoR(%;A?OEBj=68FG|Nij$lvM<7!=UH=7=d37YWT z+L7!Jv_)d|o3-vC*~AdMTyETp;6D!J+ek1xp1MMv4=HZBzgA>89Y;x&f|uyj^PS`n zvs^J860JjYUmEl6`%ERZf&AQ~ZtE6Xig@MzvPWn~7yt2Fss7vIRFCWJf@zxRfmk)& zZD{(|bXc~jndseCQSHP1%>qDD9Ta^}Ur*XJY{QXd_?#Z8m8AnvEtcdeTr#&-k8iA2 z^_0`LT#9QfN}9OT!?vyf@s+B0&Xn%5Q$Vq~bO&(^zX>ED9dS}NP|6%z%3D`RYWKMj zWlZqxJI#Kc$(91?;=+^Cik$&KfDRyU@ zwc5Ujh&>j7PPEdTdTgaz&)m*Xy6jzt`Rz`WHHz!<>{w(HU*s1UcvYYdz=1grnp~Qq z8k;xcoqc}-#Re!)1T`J?&Mw*H#%q>z$}}jOL&3OY#CulJ+kn@{)=gm}+ z`uAjW-MAJZjMX0jb$$Gp*ULp3^+~nkoZeaSlL}uU>Mde?tDep~S;- zQOLQMXhJ;nw0|QlVvE4Q`B>ICBzD}^dovI_dtNQNrO)qiA|DOPxiyLHg>FTn2xcU9 z7~mO-4#N~H@oVPmF=V%u|9-?Of+&!E8C+^0juju~!8dOn zHc`%+n}MvhHw*5B#C)CS?HXB_uhtxfH6&c{daJbIhMi{^8P}dsVV*-azmjc!ZHq93 zbt%TlLp45cQW^%Vf3g#z>pHbsP^WGXxakCMgE4dV#-(7o#)^t}eYWizR!DGm9Dc+w zI`=$_dZ-6hYe&K%!!!FMBEl-6hpXXC;UIkL%$A~NQ&W4KB-l*u@3pEyxSOtmTwj0` zyTH##vG(FEaPyUMHYRagiDn+uJNy3Wov~|Fm@!cMG#H|q@>R1?1h?vs)6WUcT{9i%4eL27*FS4R) z5})OVqQrj4+Qk()b7wE{X}Lcu@#6N&9X&&R#A<#ICuj!EJRjP`;q3;+N1NXA>xN>{ zMXfGlKS_tgSZ{=IS#MY3T4wW@2xK>L}KE*`$(Z;<`{lRR!`T?SfvBydXTNa#S+Y5ya}MvU8SN z$;j^nqzj3j+kyj$yogH-5Zv1V>xy^kEd*>q&wbe!HfF7F)bMveut z8Clt=Zc#^}!j}k3z}c1BDclGhZGqbs7vgbuR?em4T5;8W^9gP`Y` zQQEHrLk9&g^#p85r+r6Xmot<_*)WF_>pB%0I%{3_$j6YEzMF1+f=Z#aHtBO6fZPH7XF zrHxCG*sCmGy%9`PK*GevqjO*3A3fZBbHToB#=7 zX?|vAbZ4^*E(N{xzBZty@&m^adGX5^Rvu4m*3g@|+ck8>_2X>z{cr5b4wPN7-iJn2 zLuGvyWpU+$m~vr(D77^*0amYp3fl;hXk1}4(#Oy%_m0}?OMwpdk%P1W`+!n^8lh|z zd_zC&rLGAR=d1nA*`LjyC@!(Tl_y}rd(<4|b?5(FY(jFRIy-CFB z_VQyMX6(GR1S%{*SM##_Yu|zBQ|a`s2W%@lD4YL=hVf6h4AhqW>~6SOa9l)&TqJJ@ z240tWo>=sg>IK|_H+Ni&=D&%IPO842JX1H>Z#0h@lk*J6rGh=#rE7zN2Pqa z^3@4RhR1uC^+h z5P*;45y5}@Ma4ixg3i|du5T#`AzDE-ZY|n-^OcJ@eom?$2@Tv4&PVIo(+~|%SFGws zqQa#^k!ek->lxNHo8f0IKXXljSUj4VP1W<&xBuXl!QlD(53zuT{$WRl&B1l!6|?XD zBIG9M%f^bmchY6@3-1k|%(WZ%%G4{L*bJlk-08h7I1tb$q9~?@ThUXB=r^^j74_|V z713J@a}&z3gUVrtuSJI+n8Z{P2x}Zz+!@=!&Ox`RP6|{r#37M;SPMv@4~sQMaSho^ z1QU9vI%`MyXSAO+0d)xo-j7$2b%a@cJ>g+Qxj4GyU#<9lR4mTtralWBwoha`p#7FE z;A*6wrlBh%gvPRj+0qBmnPkZ(}7B$BuZQGzGqGnTkMs>_blat;<3Jp6YpDd7ZkU^;8VNT%^_7 zS0a0jaThxZPObdVesf{h($BqF2foTcu_-CBc!oj{#S~lVpa=sTIeuI^hb(-_DrU*@>@ndAWYwGE}vI3 z$>;xk{5ZP-S65LfWLGp8F>C28=~+Y|niJO5Lg&$d=<~xOtjsYo+L^vta{TH6j#;@U z`nggOq(c!3>zkYFk@zIOSoN{2*EMh8%|F9TW9t)^0(GQ3wGP+>gRT+y&ioN>`il`@ ze40fHcou_KK9>4{2+)|5<^4cQP+h=Ev-=H#L{vR5Fpb3rKy_ICXYxa!n4~pUls%P&que`ugV`*eGRY;nah*Wwo z4;EI*{}9QA5E4wdc?s0A;1n9?ST?4w71B;SY71EJ2Eu&@GaTgvQS~D}YQT6Efc=)w zVegq52be^FcEC1f7a!bXfS3nFDb2$DO@Tt}f=-A0-Ws)WK&g!nrr@jLjw^uJLKW>l zd^7>cou`@MK548xSD6EB-Zl2;|L~GSJe!ENy}iTR-nCPmyAO01dLqr z8V`h*G@Fpoj)1VfGtuOgdOIDui=px|i)4WE4?P8V;lYKTFPi|{G@#qW)k`mRU6Gvz zRsm-k7%2k`#m#=$Tza96#;W!ogsjFsqoa#z!%o;Axx*M2YSr0yicN2kV<%lLm9?JY z(Pd|Q02N&Vhu0Z%`QS$NE4V$Z#(SVl6Sd;7yi0Z(C>o8M9^`{a0CQVDB4XE$aLcuZ z2^-0|$}NGVtiZD#qkd3fx}f>g%bccFn4{CL5(~+RFR@o@j$p`)sMz- zVk3ImKwx&t2e*N>Y+crN zK6@jbCF!|-&0$0og!UCNgs14SIBJWY2+M9Sm`pO^wl2>k{A9)gUEKk3Lty=ett3ZY z*=7Z;!96sTDG)Up)8lpfkU`>h+lj?-XOgC86nh7aK!ty!&rKm~gRLnwISUcTwhhZ2 z!;utX^EGvgcPq0gDz;iE4;jMyaMtin;rpFD|8KNEQ{@}7X{PC}c?Kx$ zA55#3{|D2ou|G%`PyJ`MG<|Z>=8H+yx4+2rf@ysE_g8jFH>XP#uq7sGI!|PViX;1g z&_p%dU0Q1hhpy-tsCHpMdp^wTsiGTq#H;SQ|3f5gzpG7`u-B(62-6QKRtejuew9Dc zjhXs*d0_={nZ_TFSy3bFlMoi>S+978D)O@2n|V%OtOll#1^iWpRpi$Kw_8sQ8!xrA z=bnRQc0yd_z9928J3faE@;s0>C-QNmfY+B^w6ZoLC=9ugQ4y!>%x06$uiPiM8}VRZ zit$??uRKRO9395Qi*yjgjh+;I(Fm}BLCnq8PmH|W8@uKbpq^%)2&H>) zs({5g2vXMahU$27s*M2U_2#PL3%?%n>cjCmuSq8{2)DdGJP`_-korA!V0E@ zIYHw!WJX}Pk=z@Ak__rb`Y&!zYn>ZAvG7@(M&k-!*)~6GzZ*m#%?jVZqH|~F;GbDt z4J>EW*vCHXAQ4AQ2!Wt4BwmGexnZs-mmF+4SQ03hw~KuoevSLYy<{W@6BZ~|wg#T@ z(Ng5JR>R(06QrE2uE+yQ(2X2`!rSFUpI4vVEGr|v>~+kP8Fs`!*r3LqqGTrXbhC>m zZDD=*x-0|)c!z%mc#m93%bvwLDdR9@%NBE^WHK=bw5W*n;&;mMFJI00s3%SsrIag` z9ZwE>1gRDnVPkm_Qhe%oGE80mXN{ZOqD!XeI%^Mc^rl~L#IYRrLQKs6h}3=oT04MD~01X0J(?MA0sd1l47`@A?S=;js>Yhsb>~4 zIKxWtlA!kaV>s1r-Mff+5Ex^nn>9=H&+n zN6(qnT%dg2`%qE=UL=!N=lD}^t1OE3H4iZMPA!dIVD!b3rzeY(PH9%qgJm+;PR+YD zq4y(C5qj2q3^|M)>ilFvdIrl3BAcH(OhIJifrS{4|I^j%X>w=UEE^}+>DS%Rkcq?( zI!B#kjvoJvw?rEEQkO;wDr8!V;Vfg*{8bKa9t|_>TiypPpyD~Fw&I~vVM8emB7@d? zF1LL$?UfA!TQgtX!psTtd*~*Uahj`G67%a$sH*Ef?tl6w$n!fYQuYcZodMG0&|{14 z5NzwkLI&0?4F1DbmSusft(~S`$`E*;K&;zEG)#O1zETLG-D9ax&JYKynlqrznG^fa zuPMJ443wM$|8|GIF=yb}I^pnh2GA!04|N5;{mRk-AaucYl8#R2Xe`(o>pbItrUqkP zvs0-Sp{I2Td;pS8l{qupKoK&I(~HR0S_H(s4;ig45Gvr&49to|QQ64vH$5==Zil5J ztYebsUv9p-KbD9gxAY@>|5YUuCxVZE$Sj0e(RpyvG7R_7Qw>AlR8cB)I^SCEC_qB$ zJJZ?q{Y&oET_7p(R68B0qxo2C_Rx`HpNoO(ncTLEqrcc*Al`(y-Pa7T(Egjuy8eM7 zxtdXmL7O=G&n~xm!7WuGpv{L1&j)-^b&BXZpw8PEz@saATG|hGJSE0Z4`cTaj?=cJ ziFgQ`%9c#lIYNVTx@w$^pQFW2ERdm7rryWIp5TjR4}?$R+ekTJzn}?TjUR9{ndRlT z$&F|EiW;vaoB|G87jtFfC4bz3lsqW2&9xk9J(~vCV%tI#CvNebd=(L3gB-?RU>fTT z6k#lld&kqQ_Fi^-jD(>bQO<@5Y_AV819^&kjz^xKp3T~v|1d$!wqdTCP;Sd_@_k?W zi1l^LS~wqUX1MdfL0TUh+8MO%2IWsIMcQ_3Nn_Prvtc&)%IKi^}tGR6QPXfS@~>oWOc*5>bVW zH_bB?YdKJeQeTjVgh=Z+>5{{=b(f@LGQ0?$<>D-Ihiz=6JpTbE+60Qe_ON`($>HHz zf~MHLmZ;jR#P3E$K&9@Wj3>gnCML_%Dw=9)_H~Z_Mww8j?(P{#M zlqJiXLF(=e8Tz6RlblJ)C4M@lG0!wUKUaww#JAT7_i)nJ0nmXR2N;6v+q(>hyf|k( zpM8D4ZxQm22go~nsQ(2JfO=@+&I8g^RB9irGF2v3p@VnQ-L?ZG^DumVb_5rGea^Ey zj>CR8++WYn=ZFOP59&-hguO=5ew9D`Dtw?f8P9Q(-tB+K^lh>5G_RWb+oJL~{Unps zbhvg0M_54a=aVl474y;HyOj6pvW(D@@{mj_547 zSi2ETVi2EBx})C~coM=|Lp_*6g|d}z z>=KVXBK%D2TT))AT*DbC`LkefDaKDKeN{9&%aL2FPL69)HUpyK?*PZKnLXhK>wtMG zK*xyalMyy}#?t1pK}@cy>ue-z&e28G7|FDTLuZoBKKeynWXjys{3 z^T%{o#QF9C4+OB=J>8xth&DVNb74V_<#c&+*G3=Z@JNBqK!8$pS)vRGPhS|Juga|o znCZ|nnvT+G0sp4L5Bk3VsM2&a9-SY>t$LRhXT?Z7BjK`4c!&F@` zNK@*Vo~69Fegwl@0(~}Y@f|AT|HIi^M^(MGZKDWC2uevwx6)lMLL{V9x>G>9J48AJ zX^>9o?k=TELb^e^8_rzrz2C?EeDC?r7-y`1)G=J^x7NJpo!52Um2$)nbv6ULtya^& zM7or{bDC(Bz{EL#tiFcJ5E-wHSmcp#I#xhsGUrbd?oe0#5?F8VCVTbV;Dg&@kmxs5 zHiRv`)X%6noi~)72y?@ipMP*Y=yYn&9tPCb8*R*&91Q%Xip5rYaPH%o@1w-n$f@Sy z>JSR)s;bbeM0!O&3XX<=s)9J7I;=MTqjS*pE=|kF&x`Z^J*EwgJSr4JwGoH!TPwou zy~9jazZMifTMhLuF0*j`u{vNor1o;-g@}uDCvvaL!>@*~K;`5EU^KrG9ix5{wydoi z1uH#>Bo3UtT?MPev4-F}#mVs`r3CR~(h|g1auQe7woDj@s}>=}G&UR-zD83HJtuM9 z82=!iR~V2vg?ioE_rxa4-GKSAKy_AXEPI69s~R5BACl#%o>$+O^x7mzZFcI{MduAp zmW4vEEEb9MRc%?ik&`}-XG6&4q~U9+Yo8qI(vOFX{FIH8^LvaioK8oh#kspQdvu|Q z(4Ntt_|xIn29qPPjSsh*eNn~??Gp7xny!{=ND&jGz&Y$MlTT!458DE-eIQ>~-kEB> zfJb6Gg0fQp1_p9bv@F6YofbNyTsFytmi#ePO1e<37|!#x>1R*0mCeZsUMS8@@}Dk& zNji6gMBn$GWv2xf8fVdI_VY3}M&a1CfnThQ>7#GzW}V@sx0fwB*sI(X`(o^rW?gN;ua3PFKHr-c%@e5%T-~JOi-wTC(^D2RF z?jD^en+6Je%)l7bMvkqDA>VmwID{wtA!x}u=C*M*$T1b-37Zt@?W3I*>a|1-7_ z`Y0iRp#Zw-DKuEsz5xM#u9q+khTZ2unm- z$S686);tOZ&Ki{%!hK=;=95dk3s!Eq)j~5zoq$>8e_-6>tkBOEll}XYpNr;YtwKo} zzbj0|9C#aF3_4s@U7s5XFOA@B)_IF#ijKzP)o#>=jrQ9Pqa+rVEY$}knrr1h-xlw3 zR?XX1{f-@d0J^w4>KG>Xw>wk+D6#jFU<2XK6-=2w5UJaKr7y(saOdJ$_za6`Ly6s3f(Oj$Mjkxx~*bbPVqXMv( z=LOm3U6`u4(*gH!Lnw*e^okx1attPI$7eDw19D*I2()X<@)AwTu{g?^ay<5%ay2e{ z+U({Ng)qxd(E*UQ3Ua-lVg~utQsb!CztH38KE~9f>8wO~{|q7lL}C{}gboyjkQPkA zS2v-`Fp6z8)#^Uyy{s;k_<#G{B)Q9_0FrtDwn99o_*nx8uC;&(mN2bL%8W8s0|w>| zP|~uh&i4KIX~X~b87zwjKwN;+)cjj&I&6q!5b7LQZHPEM)Zfi=o-q{}?8~ z5{|?EwoUQBZIhV{ioiz2M6j$*Ac!THPTKp~$5^Au{QTqEcP1V;QwdAmFX?p;93NqD zgC>(?Uh;sv9l4w&{8#Zj$bRGTtT@e=r^~P7X8|F!V{NG;paBr+ANDT0+F;Shr=`%D*3KI^bnAz0o zSzF2T)z7jBVH^A$01h`r%-NanL#)F zL)YS#W(}9Mm}8V4dL!gyaNe)kM#O}S;66`uXX2bs{_MvnKuAA=lJlT=5NlYLi^=%e z+pnOB`uT#kKf%gV-T ziLEcvZ8`a{JOH7I+0+7JX5&>fKv*V$62sJV616>DVdDIX2*@q$WrQ$3io3-}j^Tei z@Lyacb~>oFasT6m>C>ZTp~@O==U<#OK{7v^Qy1CVV80cmBi^qN_51Va|Ci4LEdpC8 z1mF49Z}21Vc4%?82>?no2*oIKd8%HkIoD(G0td!34@> zco`6s%Knd_q>z`)>+n@W*XNpj42{u(WRAd*>4Xie)u$OL)+v=3JqUHyXic13SY zV)5>?w!bd{6^KnD&MvrSC5=bH6#i7z7t0|HfIQ4aFEKl36stz9S)2_lrpmr@T=@^z z>{CyrORVeA@L{0R07;k?;~V(_7!t$7DbUZjJg6C(*skT7sdCtzsc}j|d-imu%CUIE_>f2OvANLe%2hB|RjXyr zcC9R)K01ZNy9U?i{Q%B)r*rC-egj_{O_?+OsD}ibEr(bJefPtEe!)sd*gzaWxApl+ zaRAt&b)Z8qSTt<3Bv&Zx-dVxASD(UZk?3}npGyd*l(lv&9^YCw`Mu_20hN+O9_n9V z6Qs9!(IH|lPSw@apM$8x{r0HEuUBGpvieUV+1C`ojOX61Rg>D=Pk2_*YkJE=KI*S!)cH6dtRoES!F%#|g<<%<6iJ=@Cr+71jq z0$3SHl49j}u3_Rg$JG6JW{$INWztE=&0`edK0k;4B zd7!I;N!thW_HD4>L!|yh4!S@U@mYx$f(t;1*Z?XME2?F-Ku5z;)zVN)^g^bOCf)Yz z0mpy&<r)r-fI!4XGJI{taYX1;)Hn#e6>mWA|GSa-cYp$$Yjq`i>*PBwk;N3ZS|I0110H;*4CNThNqNgqvDt|gTYUtqyb!3SK3%0>~-d&n- zy&ERwvR(ZqSU77nLtJc;#A#VnHR)(xSSI?O4~tsLIy2R65XvYkyQEYa>^9Db0Lc|~ zQJ5IRUP)B~1vU6`Rhs!c;ONOqcK;&j{$*0*@W388Y~sh?U_1|op>R9jo*Zfd$TrIX z?3WFE)v9#pbZ&pgFN1AY(D$92VlnWoe8&$SKJvdks(4BRhcg{=SPwc8^Q)ghhBxfGo5SQNVJog zEb;Q1th3t?J;lsDs?%)j4XUe+v9KeKU)`?0jMQuXyV(@b$7!JdrVLiQuGgrl;5SHP z&CE) zbFEuXEcMqQc0;=esDzu)fRv}**^2=J^Y4(vMq@^7vqYgsj2z?ikUkjO@Nrs@XjiP2 zS)D>n(%WVQP>}3?WB=<$Q7Nt46?8@cgO{7+{FlM|Lk*rs$t({kTG*R$4z;vH@FlhK z;7oJV%=_a3DB0`Tk6TbV)&c>IZVwKZ@|Wx7JstM{$-7m{+6#V+j2}reOg^hIFPw5h zj&Vd#vIIa*+np*fh4`NW1Bl@PPlWI!D)Bk&fUqvm+pru?d-Yqqs;l?mda3~W@6tUj)hP~uen0Cidl6)@dC7%Ehx&q9ts7ma%VpTXy7OH z4b4DY%d$86h2;Rpq&<{e5kTykcinP--SX~id<{rM4FTOZgI5h8f!$Qra`=&E0hB}> z2`nn*&D5xZ1$mIIwV~1hmcWmst4`~se}pY#-ScF&_{m}m{0iaJW{bTyt zMc+Lz{&f^1=5a&+G!_pjzWPyQ?sscUUN<^8njk15&kmo0V>=w`xt{5uW@ee%&~M_u}U7o-b{B(i9ERs+iSRo5K_2m_q2oKz4py_f{X2R#bWBN&oU-uhT## zl8l>;#vq=TC3y-Jd@>f<>>vP{@Ln+pjp3FvlbUwY%=KkP_0Wbq%Xq9s@HvH6KcFz1 z75LVJ3hzVO%mu(EYy(nJif1?VJm>?)vOT z&cxeqm(j{j`D4gdyCHaP7Y&zwpi#UOcsu~7s8pR0W>3FO{k zYqX2(i9TSd6gF&TgtXfm@ z&hi()EBUSrm2EuC`Gi}h(6=IDGhN#s&hm3PXr)iWo#MLji_9pQUB2_^qQ|wfV#v`J*azx-D|mcy*DpTQGCzMv z>;aM_|CLL>YxCH{S)pmJd1ngr4fJn7KmZ3x7jhpjsC#N1ZUr>H+LnP9nC7rYTW^ju z9hs7COC3N+bQmu+by=~@)d7vh30;q5u{GfNH^T+)E=WDVz{T2Gn0&SypQL5!60|N5 zzRzw9a<7Ck(+#D*pG3vr-b(lkfwP6XKCnStSsBcX;D6SCYz4B;@U!?bCLkh~%fOik z6<~u6EcQU zlfLf)b*eCs*}V75Dx`C66MOSolOHh3W&!JC@||0toU_0UAOfL{Id4r_)XcRTdHKh& zH$IIDWmJTUUuDK|^y!#gf+C+4-Ne0Z=9h7>ZgB4FbnRx(>B>uR)qafJf>z<+ql#0I zEG`*gm}FOD$$!F-95AP>f86*3i0cg+yKRL{>@%$(Lt&Qw1$bX#EVyd0?HI|YMs~M= zK&jMw(NT{;*nze<%ivJXB*RXF*Gf|5Ioc@puY{4-2=Q1A-@NtMd4--9LS^wae$j17jp8i1 z@7`QDj0Y^RD-=eFa^DdZg!Oll?FPSxDh35mq=Zii)yan)RJ3~5G%nTJV7s>cL1OB6qVVd0FeDna zPw+4Z*~&1>QX>GL!h{6r|56-W!d8cne%O#p?^fXtN!2%^RC*Q^o3MPTi?-za1qcH0 zTXpQW+&4o7AYYeIOG;tVZnbwy`r@Y`LXei;2E8+2@2Db1wke<9UT^pzUwU#1!Go zrB7H>i~C*SayiD}oJ*MJXT?r=Up+MWKj3`AnR4CJ=a%ebn?DK5MD6aq_(3TBgQ99u zFUMMAk6q~Xt5Lp1aJ3vuYgP;_#6j|X%RPOE{gUC#_4%ngZIyf3waH~jQ;m9^ey$(6 zpcEUyr{nUD&rxz1vuryl61*AYhsc?ZK<(|}e09pktzD9ta(Xm#G?X<7wdJk z@itF!7M@(8(p)8f z4TIvayEWt5@BH~({J0>85Gkb}VhMvJ(Pfjj+ zMa*m8VVnko$^-zI9S5Y#coh29O~?9=7S?fSL-=sIq z?k%LNUqvXd^4hENizy)C++V3Ubz%ViZ7m5oDy}}rQargcxSRM7N+~-7KAw=uFtv2t z*}VCBe$au2*0gYW&~0^6X*%W`;pLpAH!D`Om*uKD)HDeZ$A6G~9W4bP3Nu8Utb??h zESuaTg)ELW+0oh?!zX)qg zf*fs~X8zV+{Vy5-yF>Xi)&S zz)dX(05z`)$(A2qI?OoUzL-a8gpjHqz(EWC25eL-=%}lO(CGw zcD^DHQx&n_T6&d;=dLDtE~0ITZi?|8-S)ZUY`YorJ(Yk=B^w%IY*;4$p3wa-inKa9pCpgovCI4Jve`1QxDO<*s9ylYa>udn_N4ye zG$$#^1zWp%7EZNWpE) zPa?d6g$Ac+faZtM_N8Fp`HN~O@eh`4ZX?JiWQoBzL#mW~jid$t>@e6lYt{`lsDszGq?pLyHHrkKU;j#^1u(Q+)J9B-z>3 zQ8zHA#EYA!P+ch7ulK<4+W_b>&%)+eP~_kJHEAj7+Eq`Q{gv#Ghk!kt;=%>+5*^ z)#0%*k+zJpDs$K~Z9~W#;g8%$hoc=RVL_xLWjL`&3wzX9FVm zhVh56+j1=)1R|ecbB%r|>%h0<3*m|bxxeduCoEE_C*SQMT-SqL%lO zTF-uz0|7NEe9MB(x>tPqhPOcJlOY04^PA!%E~m;Tjkokq?33lhLq+%OR8J@@OUu7` zRgo(lUAI|1?QZ&v_Ut3xkPVUm-Z?MA(2Cl)kk*0Jh5(e(IcbB^<|Ei2U>eRsLd4=H zR=6rbf z+u71K@(F|_;vUc!?M~N2^oQrMkr#_1z2O%4Fb$;NccL_Q6 z-PbPnm{>e%?j{?w8@8GOY*P{S{+7AT5-C=}O@J45x;P2faVA)6_{PPfer<9w8QeDTR*E~nz`5Crze z0tW|;-5$wkkvH>tkW}nm5D^4W5mQzY0<@Vs6Dl#9w|v^{q|?`XzOOos@{c}M*S`Ff z1hY@;yt91R=hDZc-{kXt$fbo$Q=SJkGHOE^-em#W`{|2#yPsup#qIJWs%Ua z0Qw9pgczTY#texQVogyvti`+CHcvd7d3vCaPQ!mxmBe&l?5uFOVkyeA+=4DSgfLNB z`EF7NYmdI^lc^GVh-E4-`s*#_;DES&hTabxXlpU(Xo0fgxLBwqlmv1#s*wjwl6jpW zF}R5Fxh}L<^Tzfi<#rk4N6mSW>#sK53J%FJPYb1|ZZni~ff!=-XGF>RXajj!NK0h% z1~nYHniIExtm4L!Tknfp_%4qjW30`@nM~H7gN1f1i30UIaS`2lpmE@MlW?WoG2$36 zMw-&<+25d0@^B`6|7~Ajs-IO}^rarjayx5hRbPv8264=(ee&{IQ@q37B_VEX=J;{` z_)*u6q_TEHbyT~VArLp)$c%B>IP~#Z5N=FFFFGLJ=;#K3=?}|UHoG`x&R7&@iQfJf zA5GpSC-eyTzObO}G84(mqVaX1!yNSkT%P#-!*7B{gfw2H1pLoZK`8vFgza3*GL?wW zQiTyS(D2!0fF?DWsBh?qN;;@Ta*8z6QM99iGG^>*Ld=_)!M*l1Jr{)x<5Fa=sXKe_ z2G(5L07%x6LEGT>28J7!&wrhp@77o@B3dA*DV5*z(qWV$tiq0Cy(2#SRO}%Sh%M1% zo&-#&5Wk~QSto0@`pt_z`};EP&(0@g@aoC9Ed^g=Ht2F(WIB;!3STekwURXFi19tg zAY%|sCBYi}b#5>;!dM?O!q-nvYyb1gR@MUprS1E^XKLkcVVs_J^J2o9$a)A3TU%f4 zM5qDWkaI6O-6X%JH9uy=M4?i%C+g#*oYm;rvOV*=De{u*Skj`WEeJBPq7N zDG0Abxk9R0Yx!qxcIq=B(vl4|I2tPK0-5~BZh@1A$u?hX2Yw0~+qkIq@Ok@a9b6f~ zzZb~re}G812xJi6^awK9tl#rrkQ^-^85e=O7lo$b9H%#%>j?B^3bjy1kp~QV+7XJD zTn5@v4t*GRUq`lJh#J?JU`!l`3Tx~nL@e;~+v1o(l9s8$4V`6iw_w%iYWTFj@B`2?}<}B5eteCNj(Fom8Y~EqfiM~>z zmpYAMGl+gh1WQ=WffK6vYdnmEB=piqOI?x0#&}F*f#w4irc7hm)nn(zp{@xc=N7)Bc11K!oT+JX9;yxj$^{*e(uD>l#yA%5iy>TQu z(GmXjr4R;=j}L9!g0=$!Gl_th)X#GUgQ_Nbpa;uuADl0*4?I3b&*R(h-RoKD( zT24^#K;V7n+6qNDNqg(jK~hMmD4+Pc*SC--M=?s&rXBzRh)T=;eyUWLD;UGn*|RhS z79w`wgDLx#U}^+mN_=^xLv-`a_QH#`1h!8uU3NyS3%4C$XsA*A% zVDg*qX@2hg@5DhbeIM2Y73Fw5YG$GuAk-2Ayl>~(Wi7sD=Y#P)XN)FF48?Okp{Het z74-~D4B}4D3B3KNf1=cLaQ)PZWSpIAaH?6=ea5Bwkod8)$ncjy5cAPmK*PNrgMSF;9iycO#yemZ2$k=?b41+dF&D147w5M=s+fue2w9@HKDsId| zDZT+0(eympm0{}aW^ZJGg*)KKO!SKns|V@+pRoJ_pLYU1AXi?QkEL(=6nD?pE30vn zVjZm*su=}Z7EY5+Z^FqK57VPekcF8m07!x6WVOQKF4{R3$JgHkVR__G({RZg=wEAy z5!q3GD#Uk+_qy;5E_JqC*L0%t7vgU(tw z`h+AX`9e&D2!v$951YK1`AZLWOZ7u{y^C{&%ChKhLuY{oYJ} zMe}w1y>!pg#C3(9KuZ&xt@^J2{ss)8ts!%(y+6;aFd?#(W{e#U1G0Z=#Ot zYI)t*+CWwB^a+^gFTggtrwIi{s<^fr#w+)H^591@V}qN~|A zdZv=QP!+b9gao}sE+elryHi|$y?|~M(dEAy+Gp*}OJCQ*A5iHn|3YB{tCzoP*ot_} z+WqoVTp9O@dg(=ydpgaT+KcW!uCk6P{=*vd&?40VgNdeZ#S zv95-3UjDFn+`KKEjpSXw0gsGowL1e#m2+}nZR00w{pNQoUgK%+|2KyiwV*>kx3MeViA7h#2RCTZ`cCCSyk-}v$5nc6Aa8LB~3hPzP$g#|*ryFz&`^_DiNAP;_SZH_YsLM(KprLY>=Aqmc9lH(HbEJGvUc+aIb6^xbJ{~n)LO@r;*xg`CKEU_j7Lvc`at5KJ4?5lNdxyuI+>eARrVX! z%e%Hzb!@Yg6lArk`WTE0ARG=WD2%mvF81l#8mXsCG#3zue2~} zLcPv_MY|cf$W-QHNj)eI3%A!ab7@I3e|8NWH$4rIBr0q@d%-L1mS7p~sD9fCIp$Nm zab)i@pI{>Sa&BNQIyM>`G}DV>Xs13=w`tXf>wn4Yn_KDe`~a-D%|5DLN0gOrXV*o& z=n6>5Q3oCn@JXKsp#&vt#)G4-d4D7y;84^3dTv?iI>SK0<2S%~+}f2)cidvG^&Dsi zGDl03Wl&tVE=J6{v813Bxw)1K-K#S^n3u;>3OXLx@h7lES}FX=o0I$jQee9u&L%WT zYK^n zikLf=MmkT}Fs&i)KEr#nJ&niM*<|_yJlZ5wABO57TD=MwcGdQyQariVTD;^n(+W9a zjdf|m)r8Wyn04KM8dqrhyRa}v6y*iP{{U3$DdyrOu(GK`RUs=*Jp~sYMKuCX$h{;_ zh2}aeFxToOk~mYR)H_ie_Yzb8kP7;Uf zknBwLy&DFN5~#W*4CLOZS1lx=Oa}-ZXU%D#6j3A&2{DxN5A@C7*Dj*O+*LgMaHJL( zwCSi?jpopQZDXp|y$RcU3loCH`oRzO{ztIf_dRA9{GBrBA}kNNC(5FCBMvySY^a9; z@m0oXNj^;BTxi3_ z`jmWgkD(5{h<>36yAQK1W8mVv)tWg}_!Ovu@J!74U!iAgItB@T5d0Vz7~pFK;@c%q z;^e3S<4$;;YYgfA`fbLR+8NtEhS#rr8J~^pmD&$<(b=WAtRpvHQ2e}Tn@bw?pf7nK zt$EPJ3O3cNe*W6;iZAN7X4C^Lz)TYM4Q{gqk$@uloZW-6#7y@ z^|Fky@<9>W)wSs3$MT78Fmo?IFUN`!$kFNutP{<2uOz#9CoE7AEQJ?utv?n{S+B*( z+W;&J`^HKGabL$zQ?U0)EIt89%6Yk}0Pe`tldp86Hu?H@4_9(dpv8Tn>Bo@9W~0D2 z9Gi+d0Wmi}bae!lzU2Z@fl9SESk(znDW=itqS!yBc%ajY4$J-=Qpcq23B~_>e&aX% z&B{6XaHPZY{w}rYrE!%y@-mDkw&X~6dqi5;7t+`X92`#=0{Oxl931j$PIY8YSPeQ! zf--arVR(yiWt5~Iru_k<`NY*#N*bJ4p_fe>rXiJj%%O6WsdNN;kIi2BvQ9%G;qD`_(SmbtF`=Dp5R*f74d8$+TlPF z7h)<24Y_oS{g%SbrOL#<1BAB4{nD27Y4y_-B^+Ces5bT#bgi3M3!_~+-1`I_+PEc` zgo^7fmRYHpgS>H1Lmc8>{2DWs!a-C*m;$LQ7pw!}B_jg0k%rca{53YF%#ZT`m6C{@ z(S_hW1slyxeJGDGd{FxRnS^tF!qf1Ww;@|ECbtqL7QMxgn4TmI$`ddXF216mBJFR8 z(niwG;^_cOl$&Opt&FWQbJFkcK40Poirq)9agdPIH<{5tDiLcy=hbE2TygEdHgEKo z8nGIWYx!ul9mJ68chmApn;w5xtM}o<(JnB=maE%7H_hMh zNy!b*eX+LjE7$X+Nu#H^10Uao9Wj8zQVpRS1(%UsT+Z)(uTtvgwHRH)a2)F|sPIvT zm(6bLhZX*DRV!Mv!y_$>q>9t&qX;>V8D5^d2qu^erEvTZT}~>0d|cCi{5TJO_DcKH zLW>udUGPQt%tkW{e9STt-Lxjs2l^GYoLd;VhS&?4WC5>c7Hgx-s3TN<7tg-5@V=Af zt`K<-HcmW50rv~*+M(LW;oROkGnm#enk#VP0TKs<(Ul6_8P7}UpQFN@Q6YU_)Ps0l zo#gWnS-|dZr~6OMip~{3%lrL`mxaU5OO}nRm^36G3}L1A@OT{W4Ev$ik46zfTh)^p zVq%nZ@E%smsI?R=58hH(=u`ETVUS@EQ})RYeMHZK7D^0v^VR~P>(96w>S6p?DB7t1t;^igM>pucodDfjV0ADy7S*}W}zfAlvrWQZQW=!$x8C0 zov=GV!WaZh1Wq&e=XlqM2EFw)&-hAyiqS2rMWNFnCi5V?U#u8cw z6H=v(HZW)4&3JAeiy6HBf#}W#-5I=!1kpx4 znkK~^7g2ibjmOJ)a`PDt458m(@%Cd)Mvi|VM2YZi-yxWg%93kpsg?KGEnIvzs;^cF z>tZc?tHM`y4>FrTV4T=!Rxb08=`Idg(oSSd9uEZwkX zAkH5>&`xpsh<{DzDQV9z&gW941`JO|A7+KKtZnS%ns2(*$an;-BpM2O010-T1xwwj z7&?_FdkZc1lgXV99}m)C4VME3L@DJsaQ{Xd!_=nJ{gL^^MNmo|2S?m z+y-}c?+zj3nD$9#iig(G$<|nhFa&jw>eIs8;=Pi;Z{buZ0GgHm{+-(5$SXsV z2+o?z?L>Mcm}ggvN@g-kM6iOxUTN`iX7h-zjm%u|AyxMPM6vFvc7p%GqLpx;YY};? z50C%k=OmBSm+hUeaVparL#b$YH%|5{x7Ae)Pqm03bk3M;;V_K;z40bK=~P0L44X}E)eJaBZHkRS=0CHqD+f08`ki|X zOuigRp4#m>aSvm@>%Q2U;XpW9IbEIZ88hFjkia|CZg!uBiT3WzXI)Rst)71C`DW88 zn#0QNWK(|St?|o{I>P-jLd=wi8Vfjkyv=?QiKUNypT>;2nTAfeR>(_vx#^C1+J{}f z^)#}O6wF*_K1iQVrYNV$eol=1q~hRg;OX;utYnD4@zJ_Zaz|5B!B?fnf4$yA{oicL zUW7-Vhv1scuDc=LL&`+ZSFtFwQ>4RLf{89?vO$<4Y za0X2mwRExuKUo2)BA#} z^OA#;6EJ+BdI03L|K~eEiv3Vm^p*DqW4>}(0WdmXMdhf1DJf<8^n2dUH@b{jQzCyzrKq$f;se6q1>fILF zX-YC5AnT&v2dFNhtuX|%He40#3%xDYX_=RlX(_nXaF~NMsOCyDQ?Q+WWo>Xy&YcUv zWqQJ>)9h{y5_*w_gNwtJ?qw@mkF%X=k(v42Zg1ko3Yiqhw6*V(r4220fp&X*5j?Ro zV2yDbdikDvM=z7*eF7RTyT_##F4c+!jV7Q`TU7=QLTS@lgIsK>wuC%yueQ=r78ZqL47kg(;G2a|d1);-n-XD|EDulXtl^5~_#`2aEqORfMca)HJCR_?2od8&OeP#-vjl1<)H6ORiy?SiuLOX@2SIKeCJ;-Tn^d;+T7&JIzX7KI8bGBwp|CJnKByh z%S2oOkTtR3*N`MbmD1HwTuU%nh- zJ&!|%Qc?kS6%^0=@8%<3@0bWA@t>7NWJ8f?r&sH!zSP9FGw&;;?P5}Uf?!f2_ra5I zAKc|tC&{@?sF{R)(>MZHy!@fmJg%H93_;bMo_7mivHG0)Nt7hx_l+OW_nBZV)Bo_8 z^V9IH`zYqasA9F^AD1ErSa0%+ zskwZ3b>y30DlHNKk?W}40z`98sOIIMn;&FKTz?dd0wra+$*{$Xl=DWKTY!@tN{Iq? zF4<&Rn-Jl?O0ktma=U5Wr6~XlJJJrRGK7W13LxY@Qa4? zp$=&`(i|qQCX2Y=9!7W(7L0}H+$?vV3x94U#RNv(sa-Mfr%{UnqaKxOpg=y6hAE>F zvim-L-p$oqwowiJrN;oRFS(;OlXR&m^%86mNAo=RPzhA5bD#?wmDjewd_InE8A^A! zxzt#=QXceC#StQ8WzrQlJURB$T*+1qb_@K36sC8w@`$bwoy6`rzupzz(uKg)NCeo> zQKVE7f$s;o;(ia%!F?G9?#98JPSDj7B22C$A+|E&eg~s(Pz3p)U-)<#Y*?T{`8d#+ zkQ@PBO<;!)u||Fnahz9Z1w7BzL9Z|~4Z|x_utxq(0O|ZXhJ@;M!~AKn>loODNxd;v zh|5jk^MEW(j)#=Uh>Z&BAOEsx<0rt(6FTOaB+_twB7iXW7QjzhGqM5&((Kf`lUx_U z$aPL6$oIjJS8P8;!9C$74VWR<)L%`Ztb7S! z%8V*;U!&vcZ`d*lNb5ivOR}g|d%DZpwf&@oPt?Erq7f{#vS-+Cze9=^q9TVPJX1Co z@Y#&Bav^Ik9H_G8)YfUW0BPFpougK1A8Mv|+^ulDOJL+$-ANmoCo`GZSvhOmnx@@A z4B{d3t4|{}Er(%Ns{=Z>zf??^hO`|p6trD6v^`FLRBeXqhO34!0ItjH!8PM1YvDA* z1Da4dTcKXy#W0~>>=(lf?f;xOCG!^4ixJhmj26^^Kgb{ypW;|*^uzsZcYxb+Awt~s z($Oz~h^!Ne0;K=N1O2hEKGf=>w!TIx;dF&XdhaLu`4|hW3!TOdImQ~SL0(983Xa>J zHc91s4+l`g)%B=#63{7#J&*v(2AcGZ+IP_2p&QEtN_{^mJUP;+W$3UaLT4L}w)2ub zVBBq7T7i2{M7H6g7w5HF5tWax|)Z2E2}}n#MH-2RzFe|Fu>wfS9*CYosj1F@TAM z(u7drEoQjlusA)zLe{k|0NXbww^t<~cp#L2_@2HfPVN%y3O0TP-E%tx*LU*hI%hCv zV;Lj~(d%GewLJg~=QuU45V_Q>=2L3clXPet0eX3_2taKv3Nwl$?6)*zunf6wer~3*sd2i43ed^)d(NQ z6W##wL;Xn@#e{J2p_RmaXtHUjla}FEd^Z#KbZw5S&7qNV5xl?h{H8*ij=$_ntul0p zj>ZM9ASL1HE%y>ROxF*uCNG>P?FMBK5bl^BD-xK zd93K1hl=hiv-7P9|6F-vJ4bak3>@wETI&KVfQ%ARPBO^L^8Q%_lDC>ZJFPp!_sIbho((fJYXzNn>a3I3CPmKc&3{y}no(X%zT@9D${aPQlcKj%tp zv{yuePhY0Vwm4|Wz+y@3w)^!4S=OFP{>O>_9bvAhtU>aY18`yxT#_O#s(DjDNYZPP z6uT*wm-^PP3ZVrq)nrinJglLkrS~NJ~ixNDU3r(gRY`9e!Th_w^j_z2Ex} ze1AEQ^EhYNv*)#6d%f0ruIF<-gL_94h>M0_QjlBaQyDL0jlfwXp33vb=Jup;H_FPa#gpnL?oUdB-) z5~!)hQL&W!fecYg#pvNw8!3r#x(W>v?FW;H;}(t%w0X_mC4(STmeR?==)-y+Y)?gJ z-`9VNus^Hn6U=lkzoB)@4XSg>YgTQ%R>KMnE~wDoF&U;0q=|x@q4s|`pCgJ+)0dRO zr3S)Adxf9_w#3a;pImb1y(%-l;>r~}l=M}MExSIKg|nU9L367L8f5OsqRqzz?vD|m zmI(QVH#&D-{PZoba9X5YL&WvH%i4r?=0)~zj7~qaY+UV<88miZ1Ghqb=hTS|7%(~;?|AQdtf-R(!hOh z<-EzWi6oux@jO6%6f3b6-#vHmSWB1r1QchTTS)~;<&7iIc+y?Kxiqi_>3&X4P~ceC z4p|%<6crRzhp+w{75;Oy$MKwlrmKuTd84v)$X)d#jYDei(hWX9Px0Q3J_4nfLBR-c zGo})A1PI^T5zm2>m2>&^nEcfz+umN5|qV}jUP#=i%4iaY@DGp6ik#!=W4ho zdO61zlO7bCW@cD%*O#XXq&68~E;3UA=o~rqCr)f8a2zKu=iZ(n0f2>?e ze6$`YnFJL;IP|Xv#%j>1t*AQk@PFPp6}b->4031se&9564=8d)Y=BD@t;;Et;X#py z{H3~8=#4H%n@z%Pc1>QYtG+{icNizaAZwR?5O;^Q&8R5?zc}zTWC25*!{jk~6r-ov zvyY#&s(?%9^5$H{@f3k%oTO9OcaXAMMao`(KO|&9H5Yi_vk<(hLU%36(QJ1&5ppad zDcm4&ECoJOcgMa7Ta<$xD^w)mCYK)RUObXth;U5dt=SK9?;@xv~gHh6#q6D$g)XnMI+dLJGCjISW)l%9z! z8D{gVRbC=W0+bZZOK>lccOw?$k=u6F%+vdyHTaF;w5FNtF^s@D+UuMB=WdS1{rS(9cb~ z1gQx>@&Kh;QWt!y!@Y#qa|&+}3@*;CxvBf5UMwh>^fnA{a^R-8h=%*)V#^O7%M?>i zRxHFucRm$(aTbL`ZZ>H4YLLRLmW7=Zy^L%$+DaIj<2^QGS_c|o3Hri7LjZqR0R!iP zE!|?8`z0LYNRUM2l7oo7B>v>+e?@EqqN&w@!YvR_Yo65*s(}jAqYpTJ4ZGA_x)XSh zJ6v&-J4}E=63!xW5QV+5vPgt+Jyd~2L%bo>u4M$zXFOdq6!Zr?R zx?004fnt1*Bf#Ff@4=lNj>g{u<7pjR=^JB8Uf^dzn8y22zA~l%d7?1!gpgZaI=RjC z9Gn9%^0$5I#d$vfw_EiXQ>kp`RUWafGi1}wGIKe<@s@~+$Jp3J3_;(bkmlRMU2y2I zgLzmkowmv<0~R0=Dp`K#dm^RLb%_cTL$i^a4;ryIc^W1Qll z8lf$9@Z{pF@2+UDgt#gyYYSlI*tr9Vw7Q+sQ25Wt;;ur` zP@3EQ@}CR;j~Gs653)VH2_HeD|MiRi6XpATKz8l?LpJ>W|M3C5hv03g7weh--zyv4ba}=tr(3!t?k^yepNWUrznw80&I@Vt3>_qKtf~*X z-3LqhKsA&A6zDisRs&$H#AyX&1i5436}!H4$L~&8l%L~I{5svs2Gaj>E>8fFy4ru# z2JnQ{y(!?d`b-pT^Fz_7au7SM!2Vy=gU&z|2qf+Q4KXbn{Sp=5Xd<@)uB8f46*Fq= zd!-NHVYlH9TpONpzN+w1gPs!&@%Pv1xU~ENX%VhYY9yMboqi01Yi;s++Tqbux`1!y_z)@Fy`)&e0)|r zRNEn3;ACE?ZgKtLDFDGE5PMZaF4o-y@VJ8s6V!BXGPpc?5ukg9hXCh~jon(iIl14Q z;ZMe{IXEm2m;ND?{C0as39%VPYA2a+Wqpn*?N-xfEGQCwqdt0I7x#nOA+Ea!MT&;7 ze1I#P@PCeX9RqgKpSgBk{B>5yJutrIr`P~=h!)8-4Ja_1pbWMIGQ)(4-Lmpe`6+qV zz^Y5@{aN*W@XQm|c^NZ~0>M!bbh8Gm74v45sb2)6X3Q=C78xQJt}$-MoM6E0lkiLs zc?+(`SxQqQyn*(5FpbP)E+}EHKhaVnC&{v5R$nspXElG5#C9rird!RjA?-pNl-xjo z;(QKK7vtugr>v}+?$`egBU8`uFG%Gu)tOyR+XQ6D`LkL&u0TIWyx0OETZ?q$A&J(H`o6UXrlW{MZ3?vh?l(`JHTj%5lh^m% z8giUIebWjFZ+vJ)4%38uKFPP>9dC$Vy9?KX=Uta?2C`NKFs;bw^cX6CHYfwi2TH&} zE`#_C<<#iI8Ol18!YNO+8GvgfagrB;bxebqH#fM!32Fs`(d(rpt1)k3^9z?Nz$CIw zS8mjoOH1&lJI$(Hr9_xgjOR0&J6;Mf!6s{!bH2(`;L_F&{SGHPL#jpx?b>qAEC!SN zu&MWQ3c{(=-&a8JFotYrH`yARAkK9a)yrOx($pTuM;OTTXVHs-3_}td^4{X(r@^P! z7nk`~Yrxo+J}7XM4m?uY>1Y{J$$9dEE`Pz`^Tu%!$E+T9=Pjj$fl%Vmsbvew&s!3h z@5Gae$fj(BTF87O7p(!nNIw)$J-z5wjk!QPwqa}KwI{1alKTj+uDEc!>)Hj~HZKI{vub(EtOWxxb0pLz1m z!u_#>!Ll`-iB&O*<#wx57m%5@!3uFz8_5<%;6&9C7S9cjGu3ZjsQ6 zEr4Axr*UW^rCs*bSFYRrRPYUFmNn+hW>Wb~eBF*WBUzDuk;TZLr)wXS*ds+8qxF^V z0I)}-iLg8v&xbS0ds2C-&VF8>CVpPwasY`Ih_!WUVzz029Av&;-Vr}Su!7I$mH z9c}uTWWROR8=9@SEfjDqu4511RSL1$6)G6iux6%fufU}pyx!5u@nWrxT(gk zYK&tEOt_d8d!@#`{3CV=*AWW!7if==-vI{tmHJ!I9PQXfs-kQZ))u@Y(t}Xq$Cpc! z;$n!~sU*`jrvX7mI)BYzY}zn5NP21c(-fy2T4|yi^iL+#ISvz!6qsLjP9?pUS*TK> zxjAHX_$oiMz*v6y%P#NHS#I>QHt9Z9pT~H!=Qr7#;M$N0k#G%jN zq+X^yxzWAN+ujwy)?KU_rfAH?eUXpk8>_`v$SvF?7 zvrtAYM|Q`WY%RGE0%IRbCeNBJN87Y<2L@gl#HMe8#J%;PfkB=UH@Ok!mI3vfI<7uL zX=_T3F0XgYUqCTyN_IFstOR?GKMR6lp69_dAMNzOJ`_`$JJsV=XXVoU8qs*;^_Upx za!^JM4`LV(=j}Y1$_$QGNsMd^RU7CV%UDzo&eq)_8P$?y*meN`la=AxMIvS7IkBG= zW{03VkRU&3{(=Yoa=pp)l{P=jux14mFE+dT^X^T186pi%%nEb&?OTP~T+VgRZyuF_ zkrKGxhQy(sc?q+^l8T4?L*-uc!U~;V2zzV-gjj~c*3V=cpnPipimyCp5IP>%DGJ%& zY7$Y=hM4I)x^E73rgeZWY9hglVBRlp1gB-gguH~HF&vMWk6rSfZsmA_4vgczC(o_5 z1u#7)p&VCHG1((&BPaa4j2D%A%81Eidl410PMgT^r!CZ~oK^~r~K4`>?b9#UL_MXXw zh#Bv|^C`!vSjh?pBhD@Fm>q2?gRWVficaFGwJ>S+kieQ^Th%IT~3nVH|a}1~~ z$J%m-RCrxx`=p6A)zO^Wxn35WzzgT=->~0Sh&t>dAmF%jejr6;QbFZjt)|Lf+ii)A z$}*KKFVy%bSk*JlmxsQcl;P{GbD2HO;mgoS+(7)Vji%P)o@%mFbM%~{_s^mtDnSxd zk`ru6C8xlQcW7)Q#cUD=SBNF99*}RfV5O3I+i8WXd_xDPrb@RuuD3hdm{iKK9&B7d z1PS8_A%DLWQ|%@%@~)`PDWk^!2kjOcAW{=JdT2D&a^lU zqRSLM66vW!;E6qRkIfP!&(NJb!O0QKQSj~1*(-g`GG;lVt z3$t`DX&P8oZ{gQ;lAuZYt8kUpV;3A=Ewj#?-v!C1M9rd#onl7Vz1CYRNu{(@4Bb8i zu(*W-N(Lp4d(ps;=~D>__HHA4s4K>#uV_`ruRSO68Mz3Y^d)*x>gH_15!8GQX;TM{ z=Lfn7#jhdsO@cBFg^=wxt@r-)0b?FXJ~%ypb|frYt>3q#@06Dm1ZFrc_C_<|>lO`G z>}w^5wQX>2{FX-H0%BkA8Ta5c6A@Rcbd!9lG2Lei|yh{s5t@S$9O-!FI)@e_L2>9lFFZ}-n-ifgUPXjp ztk9b;m+0!8e)nbXdMmB_)Gi!eYa!Oor#DFXAr?lQZd3`2wqlja={TSJ+~9KsjO>-_ zB)#?KOy*{8r>t=c*61+$*(692buZP??bj2KnU>xDt9#~8q%3QvDwUK=hutyK{~=@p zV07%VW6>3Q9V!z=Kklz|({g^5{-j(~V>}<>W~SeOt9X=4>HEt=xsDw3g*;L%`yb%% zmTa)=EM?4|p*BP|YYQI@iFUpTPi%S$sK{Vf#R{q+2F>O&P>srKPy zLDw+?`h*1h3z26g`MNgdcM1+@>HVb=NpBbu#Y@*}s}mgMG$qp~Y9~M55%fF1$J?O# zKD_GilOj*<@`9*=clUeWWn%+k;Zwzd4E}sDvhW(b5UoPXj%^?ZnyXQ?jF6xCM3wolHTMx8Od`pTB|8iC&}sepsL@ zA>q`S`EXR@#-ex{=eJ8$^e5^yp&y2X8k@210f_~Evo!NcDY1*%Kg{xRjGy*xd2kYA z{0)_}6>*NLmuHjJ^BMgWh0?G7{Dx^2Oy4|1^XN&q_-@WJeW+xft|Y5QLi1XH@P*Mw zr&%o0b>$#?qp!On9B8U9)h=)9Fpin}UZl$$x?qN*xwU%FYJ7G!p{bHuE+HwWU~KP| z_v;)p8ZFnX8H#zc{}^Vkw2q##e-zC*@FdXwEQ+GOwh$9ozq9CU+;%Re+=_KMFO(>~&S zO*C@y;5_Y}PDau2xV777oUP{wh?MW2BfO5Q46WD9EN$f`wyoZ1rhcsF7q){TFd9mq z($hr~yGC;^%TDFM+XAii(IVf*ZpCdEKbl)wa91$Q464UQ*aqbI-TPvM{us3DM!udC z7)!36#cJBLY5P*Kl)fEg?gpO|8ra5Syap*G6;sK4SRG}Hxe@s;C9V3w!UHjkrDLu& zF*}T!47XkR5E;D)nY3O#+=kwsioLlatm~1?O7`i~7iCpkO^brkJ+HkmM=bs_o5JCP z6)AEj7&mJ}6-r-sbeWlbW&kkTKG1EEb_%N)=^tno^V}&R~EbCb8 zqz`JbJ)}45{Bx@4>kAC$>=XTURB1E))kby=Gpr91)3v<^iV+KcA#ho@eSC;dZV!|n zI*d)1DC)qb(V3(-P6u8ORVy(QQ1&MUzjn~oD?BDjYyr0oP3{OidEiu5hXu-doGFeQCrD{#`&CO zWRNj<73?27z7Y7>7Pl@x8ZIDR*M=uAQPD@p$#~2cdTcZ2(9gu9fX^1mXryGxw;*gX zriyOY5WVUXVIqaTn=kHX6Vtr+VojQA=(ZG3%5G^kZhe}N*4&G4rF8RNZeuV}AhS$q zC*cTF3bz&!r%S@?vQ3T!QR!hcGylMJMooL{GK2M}RlTn}!Ekq~q0n}`!U=ep&1bDg zJnj4%&t)pC3e<;(K(C4>Mkxh@W@Fq=biJGA;y13%o%poNg)t?G-a0-BLuJJwJfTR~ z?xhn@BB@(l@IW$k5)&8O9ck`=R|;N9#P0B^vZ|na?#XQq*sM3U=5E6;T`i*O1L1hO zF7NGhx^E&u9u6G+UrPPGow%8emh77ee_-wYjUHq%DL{-R^Xk1sk*F5GEvDhvmg+ zJ4tATMN0nM@>6xPX=pzo-84kU%;S*zLSwSSLI(^7Fk_5mVvN3RCA>3mi1en$RkAey6IJ3!ze{CQ0b3X5LQZ@(S@H26{n>p6* z)Q@VnpVagD7nT&)BDR=`;qeJJR<wLQl{1sicRj4<=ZRf1aBVWcMS5g)a z7E&hF>*$zx2=UjoW<*@?^1vf!J?i&9x$BaDSYmYn6HIWql*But<})4$ zoc*%S#4!nd2U6IwMEM}zEX0fV<*ou}*(&bJb!5U&ypF{6aK(ISo9pnzmTNFf(XTky zE%Vn^i```&*I?Gb`OgbBwv!M*sjvUTtziA6LU7haGP$|(2F%f({)&QRa}B-l1i;rm zCTdEV8s>MVM@?R-xlC*(RXr<4dSvtP17($cGx16Gz<$#K15z9?m78 z+f+1BcKo*gqje?6?$D~3-_SEuw_K%dzvFWxzv97nw#HhZ2+OY`I-Xe6vM4~VUk=&I z9CC{+M#;K;Ah%lqB}@OPOwHd<=RYquTXUg zRo#?{EKV*Kzlbr6thB#=`-RT6$ynZ%=rt7%d6XqzQ#;F0M=IqxCX@CWj?KX#UN;Z=rr6WRk1&!}y&o&tD4R>3_AZlx@!8 zacXL%N=SPLMHPzYb)5eit_kDMT`!_$(p#90KMw1pD=xBeQq+Y@=GtTFVIc!w8kLSC z)^2OM9`%lQJR2Oh*mCOkh@K7hk{jtOVGFb2mtx+DA$~IO__b6L9S!ssNhVs1(|zJd3K2bLMZ*@wz5?n*wM`pwGj%W+4D@xffyi?rPMYU@<3}c#7vt| zww&DqLqOpPJ)$`H6yoLn4KF!bo0h(h47*dKs0MydBGyIy;Tj`x2l zj1MUtPdgHLZzBjc}HD!;9 z!p^L-g`PB(#_nm9GH2oN9$JSFKhVD`6SjY!ZaF@t<@@C}E^#U|N#(>N)2ZaFyS!GF zDF`WMTmsQgrC#PG)|%H8Y>7=La6_W#3DeB%CtIiuiIBB+0P!0XaD9!hxcZGfB%V#x zxW-E(r|A;MlI^9}%pH8~>nE#2?W_XmKL^BsuxD-i+M{+(el92Nar>5giIWf|M(L8u^R?ukKmA#&d z1wNejP$}<9qHCyT4u?OgR^xl#wv;tF+YzmD_VHEKGBq?1cn^4ey)( zL*C_CN8KiNSlezVk|ElqKe#tGi9CPeNr~O;MAtsgJYyI(IKi=O@fU_2uL>c)urAU8 zR7Cy-_>Xn2V7g}IMB-?b)Jp}i>zcQIJ%K-oagm9=^%H|VxbF?vXe~<|#xcui`&V?E zDyh8HLF;MLUp`eD>e|)#a`B;n9?3$NU~8kk$DnJWmk`ql=nqu1Wr|fPbjk@wHy-Br zk$MH)p(6I!=%LmWa5@{HP+U=~I!e+lmu-!#GBgMH5>Z z9WXP~GK82+s*X#UeI*}m1Tpt?-OI@2c%J>%Rj;NxDfs9yUtSqUe^U67tOiZm+O(8p z`G}>XW?z8=w<*OIDtK^Qo^@BCaVHUp?FKK6r`J^mL?n4+tUh5>W)I~;hk}3mqt-P+ zQ5S)@@yjx;(EISjEB@;3FrQe$q>v)7196tI$yI~QuuTty?aiHX6|?6!w;!N8Y4ttK zJlSSx8fkCcQaW}TRBTMUVD9Htj(!vK{uzltqTC58pua$6_k<8*-FV;)hVDAsJVS-{ zPYBj_3LfbGRoc&!Pz3=J6SY~+;YK|0vrgg%$i=s>)bTN-PZOJiqV7f^ooLgeOfiHJ z@`E-pW~JXN=M&f;OKyq@LK{{?php%lNVDT{fQwZP4Gn&L&e(nDgV9@1U8fsp6W{WB zovh@MOK=G1HMcf;NX>cudML*YtzciP#w$V{bRP&Xn&Jjcx#Q-e6P;9@rRd|D;hh^h zpY=~Vk)ap6iu?uSPXc=7ZrV_M*eWGME|Kc!PUTF}qKC-ZtO0>mC)*%6V;NQ!01Sn%v3SB zmKe*{Is%>I9OQ~T3tM4dt94Yrm>T5|zqbl3fOe#j@?U1~h!c)m=Lv)cmB1Ty%^Z_k ze$+A89B%2YvuB2hRhW_YW|b?Zm25``eS|yq0g8&Ywt!x0vC5CvuSj{ND~xrati)9T zKHwig=F$|sqjEdbURuPz{T75D zG|^ee$im92k!gyHJtbEWd4(q$DTf2|^Wu`)zJbh$SDMpbKH^Ap&?7ux9_|pbDwqRW zBza`<}fDWru;|5&`C|I>7at}uRZ;hH89Fce&e9ZVeS28)r*AiK>hhQww*_rAk zb*D(tRShD>TZm|?i6u=|AatO%l@2c2^rhTP5_g8+yQ2x&!>)It1EkM+vaK)okRKrx33a2*^a2XyMH;Hx zc}un5)W0LUB`$;eI=tiF=;Eg~gpgA(+bK3}b&FXe9Aa;vYa<@UvV~{Emz)o3UZF2I zQY#HJ9k60adU9gR?mn_BU%nSTtv6<_o<$}SNJ0PP?Xt}l?B2eU$QcDE`*#&TKlv9I zrxW4@JhO7Ra+@0=#4a7zwsW&tpqQ*uG~bHm<`Ml$`Q^QE8+Vlv$+7M>>J>MI9VN&6mhjX>I({Opc9*uC3JcUsu%RN866i^3w-Z5Kq|YosZUz0kqEQ+B9Z z8$6va&M6;r3z|Gbnooz@_mWLs^4`@(8xO6{i@}qihBWo2*ozZKME%E`iQcs#E3uQ# z7=pEkHPMz95ewV)Y4?_$f4XbNPf6mUU>Ne$WvvPgX8iexp9}B>5n8+DqwhwTOYDJ{ zp`(e27$#MJr{D59wd2FMBY@D4_TpE^tXBoEvuPy*#-xZ}Y5EkQl9QYj>j{&QTohfISDpy~l&3oIO@E$QU;TUOfBalyWpqqc`^97?sC(Tz%sO z^CaHz0vO_d7xwL;0`0csF;REs$2VRD)ZM->#Zqf9EJBxfPx0`i!Gp!qaBI02HXoL;8VI-C9o}7L9@jE*5;;-FQFOY5&!G}_ z_WwTZf!vN)3OAQd`%9dxSs@N>g~?XVt-iA9I{?O5W2E~CYc5I?=E?2RV0do2gqn{k z<XaOFM8MAQPn7nO~w;R^?2C?-Un~Rt?D&r?mD2SWDZ$D0^ zVHHt9qkp&5Pe6OI8|HQYT}LMk_O_!aJLv1KHR~tdf4Z*sHLhh#|IJ?nyLuf!0}8`T zG1;l+Fa^(hFxO{4V0xWB4*2izHqPV8pA>8s7hT;Tordu!*{zjCA8V@DpF63(B>1>e zt=RC|07okm{j6A4dGZwH`JaC=xQyK)WmAtgvo>Y=n<(jg#nF^-Q(zkL0FofWdL4n( z=H_0KD&u|GvD`0qODefO_lozFYU-)~lPA}+5Jrp@-r&FU*q1mH03*!Vg3}W%bPS__ z5(9sw2XryjG8xz9u=yA4Zs_}Kpq7DjkT1}K&s0pLi0EkWzpWqw(6L%VGIsvvF#Mqa z|9(vA5QXxovn|WNfchV{?SIxPloC+XHW;kL|9ScT+usBIWe&8CH{Ra-TkHAXJ~Jx| z7+ci}dg1>PZT@-=;9q&3f(M;0w-E4ueg_Jou5m{V{%3Ffe|#)70no;BdMi`^&+qvE z&r-zkq;3KcB5i@-e_vAb8$K35VasPT`BUege5gYaYypeRiCvw)Kr=aI@I;zg$Nzmy zp|7CzE3AzCcm3+~q`;@N7Olen=oNk9Rlt8S-+OTs`|ksc>w_qN&U?ecf6BZpQm|#L z2I66_{<_#cpo-Ew&l>!9qg8=tx+3=#`S;UAMZJ*9%zpXL-^I~@o$;n|q~@;;e3=DY z{_Lu7;vXSseLb{0D!59%{B@}}55TSRT7BmJb64g_aH$f67`S!|Y1ZSvZe3XttYCkEsyM@+k7eBk*M2|3U8eZYep4`h1@SK#XifP3d|})! zv|3)29xwh{c{7NNd)?g@czyoYvpocYu@OmjrQdO+o`P|`l`%X6h~6Z=1DmYHU#Dxo z`jm_g`$_>!yr6{und4&q=bwtd`HdHDTJwC>lPI1C469!cpaappT_c&d_zEYd&&&F5 z@7(xBe!4RU5(RJjfaYZ#H|tWGiMlKkU9OdE@|0xz|Aa3kt89CfnqU^sCwCOO|kA6 z$L%DB?rqw+(cM;It16JX2}#N)kLQJ)JV;*w8S=s>-qW9Mb%_%DUx>@{7f^NU{@fWL zG5GXJx$@=F)tPy4QEl!C@J~qbss7y6{1A!(UUcl5Vaadx|2ZyvBB56<;5u;wYJtR2 z2(jBlu6h|=aolO={U-TvAUkY`E8yb-VE37Nx4wP}?^y!{;6^W)y$W7Al{3>en8#Xe zHl7eU{vkxG8tQp}TNAo5C5Rwqy9FZUTLfT>x*^Ktlxr?HTZ~3?#I#T(V7u#H{u}b+Xuu>u3tqktLD}F z0;`nZxUCJj#wOq0#raQ6uU8=K=#ye5_J|CD&IT1o=+IC6e7_OW1z$Bm0pfNh=+#)a zI+lF~1V=I;cmiC12))UK6pJ-e=eZR^IEoeI55Qd*iL8VX^nxcWPP+_=zmRtCKtAo% zD$RQSfj8ZeiG2`Ik;2!^pqyr_gHKi7utuWiVFNM5jFPc58HTuBrQqAyxQyBOc?Y2L!yFjN2tx?xB^21lC9wm=wuoQ??G>+c6T(0P)nl7GU$hBJF5v$9{Ljq8k=t13Ta|?@*XqWs*r=-h))yfXpR%-IVoppj1PBBWhpz2atp}>TMw)r-6}We*&FQ z0ya3(othYbF8uOrpoGaNVZ6=Yyvo4GV}Q%W)&h0CoBm_&*?ma(kpXe;c6iNOKTtQ^ z%e!Yk{`AAg8M9>Gb-bBHi1Zvuyc){ar2v!(2-8u19a>K!%hsTFt$}LcG#Dy*E5D#~S^Onzh`Y~Z-7zyU za&|AyEbIZ$z0)m!vu?RWonNv+7K(vHQ=>p*xa`f3 zn_o!>l<HV{d*N3{8@OY+xo5sT`B z^akv`3;1bawq<~QbZ0ofe8PPZ=q(FEqK{;)aWuy7GFruI08^DYESwT~xmf046VVdF z$-oE^2df^MlVn3J+Un3%6DQ!+k^w0Ed0g^!wX?Kq3INx0Y7u`W=r|6If;a@UWxZH# z3zADhqn7E_L1ygIhgioBr}5}7#vTAw&*)c`tYMt-J)+%f+&pf-2|rka#Jz`r!~3?J zGxkl$HeiEVbZLwV#$9b#1I%AslN-kcyR@s`vDF|-s{&whDq^e1nr}g5%(3h10m*x! zL(Y~{xzVtVp9G^lofK@J)g9vh-80XjJ>#e7%{1!+X`}|5fGO_RV?6lv^?Mp&$v3cx z8Zb80E$KX(_i7KsNX}vjrp-?_s7Wydh9^q*_6i zK?44Yuz7C~Jmh=n`)!FGP}ZCFnq_9K0pHFY5_BEgX7cZy~-3Il*@VF9~e|b zpn&tBAlijHl}we>?`Ad4Sm^QyIDx;>KU9xdp0dh*J6NWBojM#)080n;p5b^Q&6In9 zg?z1I{=wXsj3>K1vYC&T_z@hxk*~&O;p@dFhNV;qzuTLrC_6L<(D}pCXl7`AuYWR3 zB)y8R^lP!uDL4({4Qi~;d-f_G=S`YLvROsFhy3Xug!vvS7<~|DBI8A6$|R*9l+w9M z9hyS}cp1bbDx}hh0X_SG&c?uPek&6}dM3ynP`b3aloXyP?msvy^TnL0;bdk|JLH1R$Gu9WhEi*}3llZHsOhwNDjV!cT3)uClk(tokVR z-r(I2Y9l_4#Q&Zx4In{rC1p!J0Y^GBBnMwQ(~ZYTjP^7B2sCxH%UU2Up_%a`(ROE0 z3nCc(O{_Cn1w=n|sU7@=qO>I=|E2EpaL>X%Jo@%cl91So$=5prS$DtBpOBA{GqjZj zW#M+=&dr3SC{G?oTc>*j?Z<#W{sU#nE&m-!7ZejIl+UUuchg-3U12e4vVGP|{RVN1 z-{A?^maAZU5Q>A0Mhq5;T}=W>>V=du^p!zM294|XjX+#K!;_!t_u15@feJC2H>(#! zUugebDhQe*LvD4yq2vUd?M`4OjIX1}DKucm3hV-G3hC@RnE_DqI{gsF`+ngJdsp(0#_@O@m7Bf%3X%EW2~`|U8u7whr>9}79XGXpt-H#HdVOC;P=-`JB>g5O z-0&u{(ApaA*FH?o{fVT-`2HceOeLyvuw!j}kWQVlf7-8|1&ftops288;1V>VA)Dz4 zk@y+wS2OERrF9jdeGyXxyB~FQ`TYY0n}$o3t!66E?K`Lz6oY0u&0bf{NVHbU>OALW z{|axPrd3R!@}`5V?YbAV2um+F`PI|zf)KKh6Rt2qw!pu)1E3E<5A&EbPmLTn!07wr z!K@Jj+Lni(V*#kM4QRsjA!*ZZ4?(&j22Ip18cSN^Z+d*)F^s;r&)gbPFsgct_8)Qy zCAfyVNzMhh9b~JGoPQPaC%2BM#|PPyH7OgqkX5?IqB1$~3GALCJ6)2g2z@B-M?UIGQ=_m8NwT5)>vf);sMu;I!8MUpivc@FtKzyx zJ#+L7Fh|{qaXJwuQvM>;J@%Uoyf9}Jib zw~QrL`((=g5X3TEk;|jwl1DDObbI+mtRn&n2b60_aafOG0*Q4WPk0em{&YIg$<=PT zC8PpJ2p@@I#i86AyogFIcZd&{ocuicgupOm-Dn`AM_o#iNcn-$e!igJ&d zq<(f0qe36XJE!jFG=2$tBw%`X?^fEK1XEUY+(!G?AoF^&qF9U+bHnW&OzJNgs#{TH z6P;c$M>aZ(pF)D#ieg$+gT!7s9-CWU#Q3#Of|O17hx`Dik^82H?*M)w+j-RPcWXZG z>uBAcQg7_*GTPY@;?A-b3;*_8)BkeOlsQ1bbUn4$bNCh4kLypbEw{5G0^uuu0(66i z*VxRC%2;0_;u}A7rl*hn?#AsEg?r^OW-}#II9Q`-F!auy1Swf+!IUJhaHOdh>=3Ia z3bSu`L!?gTAecAyqil|po#6J232(KOsaQ6^X0{o5uBS$AQ%oP|)Ecm|V$$eMX&6)` zwcOJf`RR@QL4$baO3NA1Oww~IL89E{U~0{AhPVl#&$yW)qfF8+!@Ecs{GV|^W0@P6L7O9n_{b#ylbx%g;8qpRta%(z}u8UZ78vUiLljLJ0NZmLaeP0F| zNa2kTKPT0qT1_NOjk4cFRRBEy-PufwE6Y?m3>U6UIcemq?2ZH>9y5C;ya2!Y-~cO z2TvN`3|}PCcALG{E3`_aSQ%&d76?4*ko0AX4gR9-gG#GFgR{^q4KAtKod`qt{kV+& zsgGnW-gvl)KL(bGOneaEBiO21hugfGd@K~@8fqaKZNsu~jZNuB*FV^TPcS2>*nzv0 zZ(?9rUpw6byp+bi{;cjj4yY9mg{XrcW3%bR7&6l)*F=PC!w1 z#qLTqs@*fGlO|pzwfLatp`lJmGU^`V+BIS!#-c2OZiTW>d7vB>#+PH)j5)`Y?)%0x zy%Y7I(d8rS=`DA9*Nx5Gp-+|Q&YKT&n6?Es>H2QVl{aGx5;3fD`L%~j{Gec&H#KL;f5@f+9K#&-2DqP|yC9+9$n_*=ZWDR4o)Aw=>Kf&GeS z^XIzDzqlG}H(W1AmbT>{8@xi>c&X;AZ>E_7m)3`jRc4KE`~7f?OD^PW8$C?UH|eWm zF?IfjR(viy`eDWCRs>_OwI$WGg~0Cl8{jfN1|Uuu@UnS46-=fjVJ{$^oUT0PTq=d6 zP9D~BtMp{>Q(S8C@KdYeX_${ObR9V~iNnp|vp1IN4pOiYeKG*SQ}>%mXocRzRMLk! z^X?&6s$S=|OKWpL2!LeMXO=P-fLANio#O+qYYnyUZM>ieH#eYY#=>^#5n-A+)u1MF z;;PSyr8Ko^c8R+0SBR=?^gMV;&@Fml$x3f7q6y4qxyubEH@FDfm3SVAz!%5xrQ5rn zN-?J~pzjr}&vQukQb*jmfNJ;GJKbTNhgM((*>l4h{D}BQzJx%9hmzfV-EdkwQI!le zkx|#4`HiJO;nR7|)ZzzC|LUWRP5+bLVBrsaYLq9!h46?knLrir);liLdy+KyErYAvmS z@NGHX*Joeuhx%KX!%2SGS+ytMw>cU4TaL&y?A2Yv|I5jI(4#;t%h?}g2rUB5pY14doN-JNK+?khRgiDV^8uR) z#mUhu?^7sWSHr*FR?MMM60_8 zPW*1#?Jam~UfJl=PdZ9nJk`pK+Y!7vaQo9WjrL?}sD=v+Bu2ztP9P5;AV82(rGFqt z(8)b4LaDt%ZW9o1nOx#5ReM{)9wP$ap)5L|BH)pIHqzIWO^`YTa#lU^qm)5%XgN`^7 zSyCN2R)fmIo-!~qFx#S07CA!hj#-^%8LXO*Xue!iC2wFtvxop+ zX|dc!F@e#qXRe_zF=b#F$CCo((Ll==J=DZI(b7-Cjtz0HP$=yWsRv?^5U#46uDe0T zTR`Vj)#w!YjAM`BLdchyQT>aUaRSt|#*K7G2sh})D=(~O)to@hzJ_GPt%+0dkY{vk z;=;k0U7a}nOVc11)CZx0mbvvYKvQs{Zu$fv87@baZpT0r*|j`+`jv3L)PL8MFzvhC zXWPNMNfwN_F{!%BiRUxqDXotgj%5*FhvlieB>4OkM|(RY+jtM3fUak*k#UR^?eg7* z&y|5@)iQ8gw8o~Ac-N;szQ{?E%kI+rXAe>qK;>S2rO~;^1P38Xoxtm|KftZ7kq#t9SM6NoR>NF6N$VIhVy~cCUoOxLJ#V^MJ=%V@IY9~s z4I%^6BHvc4=HseSx@NBh`pA?xdg2uUHX2A`F}58|0M8%V!IdK&`LMsbcp*gVjanag0aHqJrDf4Tnm2G;-v zEB&*iT4z=WzYJ)2T{F}ce6t?)^`g0rPQXYlU#&7^6 z1^B5&mP8W7+omyn+j#Lf;E5YK8#ZGNNv;S|*>|kirsITX`SDvK%?pu)nBQ9bz$QpJ zHC&zSO3tR+CjFS&yptF9lnY36lK9?qXF^~>J37a<Cgyl7Qwvz zSOwx78I>|L%SpskhSZTrsKAQ(N>eVdGp}mwUk3yv-Rtk#9dU4ObOZJ?c=5R8ZgZ8J z(2d_|SofZFe^84>40d3KKUi1VniFAy6b!Jwsw21OAFxZwob}@6^J_1!-T5e8eT)@! zRJ{~gWB}XhjArYkw2Nl+q_KN+*(7QGv=D^30S7Sl0vZ0jEH*wvwGwc4$o5Wv2ugqR zqa(2mUo}6Y(4Gtt#>`57G|L&skgJA6?&j@vvEFu9$BRf4Jvb&E7*w1j=M4^JrG>6QiWV7!*MOVO>-N9yf(AmD6=zx1IHB{-{zO*(|vA;0SB&slR_xT_nOO5wROu8KWEu)Y-9(vlZX>P{M>sw2K`=-jJ= z$%bafof_7!h4lS+=7&ZUGvLQsX{FIHVw_T5uiM7}J3w#x4AH*gXwZhU$x8Ods=M;9 z8hZ+gE~X$Zg}`T6ggMB)*tK6{;ph_K>#&B=FnY%~7%@0F+w{+Wx0S*H6gxub4t^Am z+y#1uJLd69AM)x%i1ALRWOZQAPv#y!8=U>kp1uE~P&c0)rP~EkD=#B$7HBHW{4&*0 zns#N2Oyx-kD~;Kk_xRFNF}|5pAW`}xDU?WQ*Fv*+37&RbM? zVJ~pF%i?!PI)ex9H?z}P-V=@YK7)2fy+HG+QH!#~xM{|WFw+9|+R+iJn5&2P%>r@i(~09n7OTz*Fw ze6|(a>Ws_4IrOJn-h}bPCS@BnK}SV%m38?}f~R7k8CJlcUG5iuLkKqiy38CnHu3WA zemTVS!h{17pv4`63uYraeRe>RZ=9!+;5CH<(`L{#aOCFuwplQ<)aC=-@Wi;hA4wIn z3(yq}R;QQ2YIuVSKqs*8`kk`}o{A_Tm-+}EhbjhwENVV0=5!p3f}0LE0A{}}P?O*Jy00r> zc^7sv5==n=$bqYN=K%ZPsEdK1Gxf{VK`G_tdbz1M6tIC-lQ@)4??n`wD6Tm0MFp4x zZGT@~jFcjgU2(txw4`IjyGQK zT5$UK1KoIebG$8L-36)!x^(OnM-mc~ZVS&9L7 zQHC!D%p>7(kITnixRoVl**;bJ69=2mUip+WvPB?`D&a3_#%N>gTe~DWM4f D01Iy1 literal 0 HcmV?d00001 diff --git a/docs/source/images/changelog/reshape-memory-increase.png b/docs/source/images/changelog/reshape-memory-increase.png new file mode 100644 index 0000000000000000000000000000000000000000..7e2fba25281ff144959549023c101ed8c84c1448 GIT binary patch literal 74624 zcmeFZcQ{;K_XaG22tkSxQPb2Q!KfKEh(wJR1QF4DXGD((qCALRMjI`N-U$*Ug3-%p ziD2|L+Aw^3^0d6q`+J}3`}?~-*EQpqGiRT@*Iwmb_gY7Ynu-Di*%dMZ0s;!f`}ZCZ z5RkwK2u{_Il7LU1)r-C(ARsHTl$BLel$B*wbFw$Hv@s+2_l92@(KPYB?wu_m626V$P) z8Dgz8Ap!(a5pbVL8v0DdrdRyRZ-ibut>5TS-R`{6p>l^rfacPb75_3a0wVt4>k()C z-UVan6DK?bsm3mul3!J2y+`j&!pq5-e>b+T?gMG~g^at2>GX&3cQ<{z=()Q;fBX2* z6UL_@vq7C57A-nW~J(h?j* zs1LkqB$R$_@GFd`*`+Rtkn;u<&ak10_CA;(&ycim-zJm@a3l@bI8U4@QxHX)O_eC4 zeg2Y@e*>I=<8o-gGq}fnQbo8Hk|ggu*Q+mZSv3lbfD1_-sX=fZJXlRq@LXeH(+5(e zSC;QdYzTS-6K|962FrfSak!g8vhcxkL%W8$*e|j*d_!)Z?FH3ZpixT}snqlHVwvFy zSCu}VK6@|oH<5d9-bs^1UCbuTSI~=c`$5?s+#oMu;&7SkmH(YK5#twkj4$)DQIwOE z6KlN6Zx?*VYqItfd*^z>`eQO}ChJhwk7cF5je4w**+^6o>4!kW8v`1HnLhe_{N~`2 zaYi=atW29KQ5`wDRTlOS@L`RuT2IJXMz)Bo{qD+C00kNm?4@kt}kbk)1Q@ zoMKP5+M{cn@tO|`*bT1kL{D{fcA`5)I&Vc^=zJH$`Bs$ISs7t8FQEIIjm$19-_?hm z`k%FaXm81Gd65M@lXA?ur@_ym&k2cr7@HfbTBfP%tXp+U)vd^;)+WoQdWAtWGo^{~ zpi`DIZJy)+V;~ZCUJ> zw>_LYeHZFZr7Ei;==xNV>jTXk)*R)r2VaukF2AjR+o3t1SChY?MW%U0b1nacx@jIP zpDo``$M%tT<|}mpb^Yur8QbP@n#vJp3+I%1hXv}dqIVX46MJWu$^5SA5$-*fXGfhv znSzDlw|3_6hgV7Dmfz%_z5z-{ zKhyhuy{y-h4mEZq^G21xo=XPNyc)tP?zwi!Yp>JdrU)CgV6L!8ObAv9M}(Wiu3}9{ zUXffNNj{f!?$tT1b7QYWU%9?&l&8L~lO>o%mQ~*_VuEmt-jtYCxitE5_(n_i#E&XY z8!L0J+mq(w5IXS@<6;N9PZP`nW%OOQ2Hy3&n@+J~e%dtE@cF3wZlh=4c2%!+`rc&g z@U^k;uRobQlzS-IwG+n^xBgD)U5)mfwtErh@YJw%QF77rc*6LG?Jnx>`F2-WtgfL%+INFz>TG7L2|HZ(ORaaf+Bfl}oFIn4QL2m3!+$z}i5y0^#7E6*Lq_Dk@`1@;HQ-!e ze}IqfOMU8!ycd1jPrGKcT2`+PM$KKVyviawDH|rk5=^<^pSL=t z{OI{PA+~D>p?PlhD2*2SaMO;paHelC2X|+q%?S^2&$bulK6g&!Qk6x-U7PD2rnTDOe1g zDz+0AO6OtaSXm0ptJc|>njiaJ=_Wo+zO5(sP>S=Pu^QOi-84iRuf@wNB@Jj#i?$ume6V^UB`1bPX{bmoyiN^`nnLDdbs2nZqM=vwV~lJoa8MIJf#I zyTW+gpKLZetrqMJJzRaaOFoiPlG?cDf%{4^AC5?N6K1d-_*2~tX8mc!8M{w^CsbFy!40Hy18J2Lgv6}T&VK#DP=b0CI zTBjtZXTfpBcj4LTE7%om&yLsbG>mp@Ws|^tW@vq}*_Y|7mx`z1mdvtTchnGa%y|7< zPkfjWWz)sRE54~G!L2M_)0Z1XcXCnb)$-RANTx%lRT@xRhCALvXOgLGv!t_BB@r(e zS-AJJvH4hmZRf-I*L-~KIxjx?^5D#listH1_NGE&L(X1M_?mcy_`K$~8WDmity5AV zgnr{^2`o1;O3<4~jPD0(|C5gc&U>+=ryLklr3uwj8E2PKzIv?PSm8synZ~m+6Ju{$0D=uQeXMAv ztW3ZKj!6lKPF*1&21lpBOX3v6U&r#NI0#Pv{G5=0AlQap<6MJ+| z{*TMSU(jn7E-nsYe0=Wi?!4{-y!KA!e0M}eMfq;?^YQcZfN$_Pd)m1ed+^vfU;llO zKgYRe>TKd<>EL2%Z^w!s*Z7&es|)nnHT;eK`uTmHrXH4m-^tGTk7a=c^5M_$-Qm5> z_t)6q(!2Oi#ndc4Ol@@US=s`bfqO^@3f{i^^ZWmG=I=ZHa^>T{ue>7!5&8AfUrzn= zQY~jwCs})2a8DPBzxV5pi+?@&$Ax$K@Js(i6u-;)=cho>5@dJz{@OGNvfhBQ0kDsZ zmiJUO!Ec~tKYu2`I|q2-e}h-@`&t_g3IYNt0>yjMnjWWCCeJqLI3IUxs^-b(ROIJn zb3}*Ino_?ldr5ElImz}&hMuus193^G-BOv@=Z_-qfr~lVE{6$j!Walx|=%Cs6 z|2m%lW*?G&Sr&mmAq^{<6G`4-eEnZ0Lr4TYI`?n0B_!sOLTbS2BHHBtrwG9bQrJHI zze$gXOw$jZE5*kc9R9y0Lr8iavmx+rvn8d@Bal}0Hx!{{_+R^ppCv2$oB6-Z_B@B$ zsem-XMh8Zr|G7=zgcP!^_uppAss;!0BZhqwvH3T>0E(0TZ?lzx|Nn~fzvcJ;E6)G6 ztN%|Zj#Liq{w~UXwdD-Uxm1(w`R>)%*CR-k3(sBGT71IYw)36w7dO`#1ni%edW_^( zyNwO|@J_dUWk|=UQZ9^kOvlLEv@4FM9J{lOqSd!CULCJ2_dk7R+zn+L?=~%9-cu1s zDaKKu6imsq5=3YC|MFI%)R)EG-le(@>z3-jSj>sH31)J8o!W5^XfB-pY{TxjwcFup zb#g#h|AUSQrqxjH;`ga;Gxpud)cF5y>qSx^JEoaW@z4CeQH-Msz0^Ssx3 z<6b}W;GGUelXydz<+v$-;z76Y^y8hSp+=Q%0o^Ghwt*e(caM!vup`Y4!DZb^kZqlX zOJLmBG>>bYa#kxx3=v*R6)Vjr$8IgebdLuR%7MBCr#yGZ%u&lF&CE*MXT;psYsxKGG5v;mD(CQPgd>UKZs|U2KEbWa+UM;p4s>`q8&K^Z z_2;QyPv$ob&6f<-oCBh*($zWNpZlC$FO-z5%LOsvX60tg6$I5(=BaXeL2{KSI7QywNZF>jZL zi^|orrG>_k@0^tQf#mqB}-Xq*a#yYfWW;beGXUY zte$PVHy@@pV)`Fu%pG0WDcn6iTtCd3THvXkxX75ymR+}3kl0gLT=&J$bB@No`B=Vc z!s($n7M-=K>WhuBt>}|Ke!ea+SzAY55TZ62xzL@wpdyqSW)*=TkCEZN4JP|Hs-ocq zpygY<-|63d=Zud3;aYufaT9jwd@%C{C#mO;-xOp9SbX-4QQmu#vI7&pO*MLQg(03tAy^z;&{B$y3r>j zj?ai$Z-N3oK@WRhNO0eNx?YaTib`Vtsr$t?U|LdyK;R9Hz;H9J7aKJ<3EP%;3y~|k zEBG93DxhquCmY6j*Ib5PXj0sNtNAT>trYl{g#31;C+bBVXINe!QB9I3z&I(_jE)6~ z*S8mX_Y`E)9U+h`<*rYYUPj(KT73K1jW#M-l(Pc0yrsUFN^`cDPW5hd99~&hL)g-GGh=}s;#s?(B94tm3;yy_C37H`) zdy!0#+9sRJk%|IUWvl$clpxgJ_n(jy*Un$z+YePSXZ+QjMDs`?V^)^i`mW-Ersf6z zAaXjrW*?l4IVxX+odNiGompUGyoh7&qf7*HFg{X@J(Al|T6MxHN&DxL3xcx8@Ao|_ zF@%;#Qb1-AYyp|3kZ=YiW^q%_m*rz>A&RAMe{;5x zO-KS+17NnbMuqVv-U6g<((QK{9Y2G?7vsu%#JJJ*^Me|y zJnHpx#r11tr%sd96cV-N2Qcr7}CNRhal{Ma)R{Z%Ha! zVM)ie+dblqGY=}G2XCOzeOV6$C3E#etm0I<(EeFi_RuEX5+6|#iYq_t#}73H9%|&O zP$+i|fgP5omnQIUIGdNw*qjJ)&t2M+uRIj0_!z`;EJX7?uQ&A-1w)d|`5`OaMp&gJ z>Ts}ev&0uyD4fy@&o%E&GvvrR?gmasp#Den!)MlJ5e&jTRUr7S?|<(ZqJj-JAh1lK zY3bNQ^=WhJ{}6=|UK9;>jF8_&aY@aW6!v<7*0pJ}*g)LAf4f)G-aN(el{kuolF_1i z3TADQBzci;l;g4bOyBdKr&Wi79>`SiU~@zL=auB0W=IW3x=ZX}hG{u@OQ__tDnB=75TyZTQUp?S@})_GwZI93+1 zELtAD(T^dQ0xczvHUeFjhk2+SZ0Z&fQ-K{de%E>5i|&q^*be5Yw+PLWe5o^>sLC9` zv^Q97)W|L5Icp_CtmAOssC*sF^#e&L8k{wYSZEq6C2U6DBl!z5vgao_fTO4(jFiAy zgd7gcC)?Yxyx5RdQQ_h}LcRi6!@JO^*~fx~3{mgC&YXn!jb6YkAxW`I2bpz6sKD|P zj{>9POU@haVc2V1$<1%XM^>v)J>+i$dwIy_z(6f2FD=*H=zq;P{f_`(`Z1E+_|t4z z)nsPsmW#{l*PFxzp}esnUN!LrpUY}Kuo-CBANO?Cr8OTnAMfy$tg{B`;Gs{^6$nRYie1XRgz|QHC zUL4HDqtq3c=9I(4N06;Ooxvc@e+^YZ93WJ^)(JQHx!OEI!sU??;UI3S zhz=gBG#dc0hcYQb`r_yjm&FH!a-@V!NHbePg8;>C}cCox$@+$zu-- zm~LR2@9}mzVqD@sP0$8!f=xb*(4QvgPo6i%>iWiDxa>1qvIoQh6IZMMoiJIGibB{0 z3Zc(N_x1g}Ntc^;t;Ufn9^5l50`6jlg52ubh$~%v__sNH8+BleLW!?9rKA%dM3_p)Q zB~D!oLftn*@ra)>0e+G632ZTE7lclGWa1M5-)>I5?A1YD8flJFK>)OB$U+Buoo5z5 zSR(i2M5+(2u*~c+<1gg3YeNB(qp}`)5tGDK2Yl4g=cxtg-h{@r%VazsNWolR`UPCs zkwqMLPVieGS?j?DO)8eQEu^}qLGkA9Y4t@3Vxaj&96e@{6>b9{B%;?secEIx@GFm1 zz$4u3{>@}BH(qOu#R+?gLx6$Dx1i&U4LqFwYbnHdz44<9m;GID7zttT$FdZt^s8*f zNK&JiY2^=#KF?=l)Nk92Xq<=$PrAg^pDjsGZ9bagGwhgNJIAyzV&uE3QdR?3Q10Vw za0129Rp1Y>o+I!EWD-tAf~$3%b}`QP)W6isz$JE_n@C;G@l(PPag7 z7U@B}&>6vZ+mO=y{{h;FuY(ocjt@O8hxc%KHiSz=M+qhooQ`@R(@Em|_?o+5Y-_8F zFW796Vhi~0t|URri%Pvh?>BY6x(KY@t`WW2KD*7JZ#T#NnxhXgCP0594la zGK&=JSMxqTe&0q}?t3)L0$}>m)A5;>z|~g_cBJiqyz+0QMdI*j7`Nj^TU1pgdzwe< z(nyKXv#Bp{xmPg*s=_8Sh^DC7+dIbp#YrNc|1Q^FPf+>w?JM zg!8Js@1Z{0#T{iU?1kx50QmGm5K94Zw(#gA-dP1RN0~8|Ub2!=sF9S1nJ!34OZ-T=#EsVB%RPrSEn1d;ssS!p`e-3`Ch+J&^z?F8V5g4p4Q z$ub5z2DCgp`c)MIJtA$ugRvCqmLuG$G(DWVAtN5&j9W?7uV@ z4^qCJ1svxTWcKyXoQ})?L$Bn?%8o+mvrnPOu{@S!;Esg^dBiS`(8=Yhq}r^EmbPS$ zkq8}!jur29UXP4L3oD?k`Z9y|-fJjVE@l+zU+S2Sm(U=Fs!AMW8#uHOuVrQ!H;$Dg z-CJr723q!3;pw;5psHz~!?7fY^)uB6-EZ>SXMrQTEN*mZXaY5Ta_o+B>|hEsbz;Ii z@x|@9f$b$mxt{?EvT5_t*256Qj|o#QesrJX(W0s^86;?t1@%_PaFKZRSS0;#s{~Q; zGT6`{$5fFAcwplHfDNv9yMW>n4?f$T?@m@=4{kT|T7}>(nWJGFOH?G#^LX zs%ylVw7=O*Vs{?tc13Oj7A0X#His!oCKsS(O%E2RT{v^?3t*>)t9cfyNM+&4L2I^h zfCqS4Kx*=rV`PnT0Dk+W5i2Ft&&1S;ECsfdzuW(iFG!`>px(`pN zAWPv)R{ZG*$>V*bqvgE}WO=WI^-!S>-<`;_|FHr*_%(Kk^25Nh;<%em{rt>#L1Dfuj9J&){oCsL)!X52swo>byKmN8VyaH5=R?t zYV6bF)0VlZW&i+{0mi5kpWcqofwa&TI6w;JZ3T?>Nuqh$VkAj=_!S8Ms;d1L4Up1r zk-gaN7DgODAB@dDB9hhWT<*B``)We|d}VC-N?muCi

t1n#LuQ$$IyajTxO59ANzpEbpT-+csnDRl{BYs9esWPJWaE1kl zf}rp5QEay$1@m2zsyNlMo|jp_f@8cEfJm~%4iY{|>)n@sL{CeiwYp{A`eW>}25(=3 zr?c1b!Ggwa%WqC{X1|!jpJIFa81OY0jh?7b{2b0-=Hh)i%u3_ll-s21i0)*aEBRuq zi1r_A`h}^5PXJkeNljq-V@xSFLOUd$x=Xg&0BLYL2pN0~z@*udpVI#ySAPNiQM2}8 z&)MILH&IIdZ@G2G@ewyhS*VmvISJ2n#dyLQX?}qOe;7j)E3gA1gL9PsTKMXe_k*)D z@WlNLw!cjLm+ayc(Zw4;cJQ#4jQ>M2f1BPIsM|cx`c(G!C;xpU6@cG=mpdZ=0T%yx zUseV%b(@OEw14c~uaWYviQSI^7%qU91n<}X`$m6$_8g#zm~h&zUk%{TbAPSo>_t|z zBJ$IW+W&c@KW67|1EyZq_e}S{_vx=WNwG=;w|}1&A)E+$2_CL!PJHHZiW5V zOnzC&jr(9{r0D`>{|{yUd!A(%Ska&5!%Y9%T>h4wKM_!)8|Y^aAY1!Wq5u0xN{G;t z9LA=U5Y_Eidh0($3W|}yS5`|!ruR*;j zKxldiB+khU*1xBde+lP?=F_i$~9yhl0*4B3pmlzp&szwVG3K9ZRce^LY z5jVc;8|YR^;%cpxDf4#@9T|$c$Cf z8pv&h0l0AcBx3lNgnzEwzk<*QIgo9sVLe`*k**U`E2N69+dxK03fN+@pZ| zU`as*Qb|}SFnd0~-6Qplyhb{*hwaMlV6g$JYSIqXDyiNBG@2{u!}m{NtH~wJn>|7A zb00nFcM?4tkmh7;n)Q!8q`3*^oGRvC){`oxGia=d#Yte0vMr+8ui)kwbf4Q=?ZUY4 z$>H=&gWv|pu~2;qEKEu5_l?Q-rB#=Uu8aow`Yxm@39eW+$eEz*r*UM5_e5bJ;s_G5 zD4l#T^A_n6XK77);H>KmyG8^RoqKNb&k5kkymW3g-GH=wjok4u^K@H)vqFQeqeDzN z(IGy_a`)b<-7damo1;P1#_g*soanAEE!Wi+iAlY z^(np&wgx1DraDg?<@9a`q*dG{79F5~?XL9=8^X3tTGIj=q8SAVwN(Ay=Pns+EA=M8 z*)=%hWdHUG_#+Ndu=__(5Hggo(9qD8cjY~0y5&B$BdtmOKMg$%6EEW8R_O#-)$Jw6@f~FUB$~FL)mHzq# z%yT8J+vM%UP}yM0$_sQ6Y%81zg`NJ_)GEP>3wC!R#k~U3L}(EIP#HKG+D#%b$B(8x=o@EY{Lh|U{fCar;5}aN+k$h_y#N{7f^tNbD;n7%dmVh*t0K3a zQvp}#0F&#}3Bg_WlR^OTB|gwvn(vdF#^C|Wdk3ar7 zEAePWWL6CON`8eTVajE~w<=(J`wjZvslJA@Tk~uGBWB$%`>87`Ky+=vRMns|q6=zC z52Y?(Vp;QJFY$xT-FgXqyQSb4BLja#c79j$OIOmSB7Ubn1^h#ugFwqcOnRYa9#9OX zfHW7ex8{aIz($ka=Wadm>LK#~!YehZF;?QRw0#||Xxqm44+#+hHt=(=91uvPfY;|n zsVeEn2*3C2dn5il*ZiaoNYC%aMq2k*KS0J{xIb?GLkFJU1UIMANu*;#a|$qVYrhAn zqkzMVlrUmCoLv>ULh{r9{Po5v3BurkBio-dc!Ja@ z)aF^1gg4t7+HY5g-}uKc%=lrbI#&q)8sVq$u=?VMDg9-btj572oe`3Am!DQz52@5^ zv;u4VdXE4f;dViUi`~uup+Vqwvci8FUuOw^cN)&Zwr0N?I`Ktuq5IOV2w7QxoU}R# zVuhHzW2~yjYzCwQ@QD||Cto(z0B*Ywulz^X%!d*}P-;x0p`jUL_F6Hx9AWbls-3aFp+orzE;_E{BV`>F*RGnOp9< zKP@b(Dg(8hx3JVn*URz*{#zIJ!3E81O zbfTXQ(0U*z>JdWOl&7TXUlhKFHzFBi42$`0wAZNdE)leL=N+{^C~+nOI^@=@v*qafj8(uH2?0Y8UwPlg zZ)h(|`=r_~by2d~reU*vZYyeF@(f6%)$R}injp-;ZOn8RRB?WWz0p85H77urp-!D) zJP&w=EPvTA#HsYnhm-3K4xbZaoi=)XkIU&ov!AebpM$>fS}v-77O(cfxs5`26f`l= zfJ?MA%s?R32KA}fuIB=>RAlzGbF_FS=~2=lib2pK%E0jx4I(($QLe)Wu!|(+5n@Xm z_;fnZF?A0gZr|iGsIk{#QwKo4%y8$~%f%^GP?oG-~5p+v=qIbT|4r=ED+5@!wGRoE-G}j(=uPPw8_z3gQw& z_cf~QH7JF)W`YcU64drnK7Z$o_XMdr)Oo1lb48w8ACvpUx5m#b1`3s~Bvj0dN)f~8 z&ur+kLXYmk9hQMw3P3dxfV(;%$0ztxjo$#4>a;L7#TnN4;9%=bd!RaOR>889{)qNK(9p?{qmB^dqA{KCO(^5k9u@b2x-Mz9;JO0@&beB%jl=ilh^-UKTkHRNl)G>pF@f;ufu z)>q?pN_=^At~E9u?9WC9@7{6xaUV1W95y2bPmG!0Z#rDFCa1kcs*-IzQfydm zWaVb|DU`-=K-%{ZHLc_yWkiihar(k}Y_W0hM>oexc%gPDSUAJihY&b?0aRQbLc|5Z z^BvP?6KRQAttge{?6sRgQaknjM3Pv8tQLdRkn3c%y?K{j* znizl_0c&rV&N%*DxTrDN?Apln@G2gq07UeA46cJKcqI#ZM)N#*e|yyNkO`$pHf}Va*3ro`~aK$f_fp=pjkP40-I~yIAQ(ggpd9_ewM*FsHnhl#ICfUZ8hKrVm-1>5j zG_BP;mg$H^x9)4X^lE*`>_;dwtlPSuP>pIX-^=B;=tu4QsITVEMu_*YC?DiE#SFec z7!BT?5?x4{jf}%4DkHPP7kvtSz74+9KmCMTxkdMGHra&RS(1(5D~tAJYl%xSN|gbcP8TqmgE+wfc!zl3_mjn`{ONQqv2}oGV3ml5v&6@e z_*6&=yor>@Pmu3V#|#idmxoKrGc&L0mw6~U5Ez8&)1jlJo6TNAPtl(y#naNQfR={e zqKt&IlfNq{vp`5ku4k}aov7x77$}n;P;u;R@Qqi?AMUQM4(r?R2w%VsCLcI@uX5ad zl-s}s^MgBl;(G9)J~iga7vT9H(!Ag z#MF`I0G*Xer*A^(m|1dSQf$BeZ!mI0`)WM7YeQH-k8Za02GxNvzf-3l*`e{Ms>%D1 zWINq^R~O~EkQ(Snk>{y^D@AMp@;ougzVLOX-!srhb}k93p?nj!K)Yw0y2cLb9WTb` zKhyFVG791lKK+^9J2jxQNPC(AcDvqgQp~<8$!LJ#D3R8+u6XyWR%SOcpI)TVvcz^d z(rR)RVDbw#Vko!*cb!B*W44|WVAVJz{eQC5Jt>Q;GawZ0T5V^el9TiX+;7kj_hFgi z75lu$s2l~xR4WkbPdZs9OJp2Rf^K-5zrZf$HtNL`e9J1!kXDtOQl0cNsk;Jg3Xy>X zS^KDKzyb5Sp+Js}0ohzU*agX+m79BniA2!x!lVX99l@SB9s^p&dy#3-cLpf8|0Ax) zu7bG!WnMdf(S@iOs9yV*`!8eD!8L@n}D*l*IpdyWJCAqkAph9--vl6 zdDb{TXh%yGR`k7Jxc#KLJ*@djLG`v41IPH)T4r9fe+z&cb_3MMcd`!$1D^${*o|fh zEA4!chIsJvz%(?wQH!hIfB=n6_e0%EmbvI!#>7*Qvo#lQ1fwd&AZm|4Xbea$tGYM$#y&e!_%hk_Iq$>ym@f zg!=2^+clt#tW;dHDCv@l$kl}0!_u{`92yAuyQZ$AXvOqMZtZusC(DmA^~Uf-g{0nh z4L-_)N}6iWfXLU-e}LYTDh@~sZAdp`z`S9Gj@#9*3ialLWCvj;%l(gV57PDBV_C;e z4b7vnTvXZp^d|3jreSe8%#6mW`S)%CqVenn5bryKIA)DO#(JTcza-8Qnbl-F7^_z? zFbo0&-+o}9dw+5O%mm4Uz5dwOre1C+b4WAha87Z%lFF5GO^D}PF&)!zywK3)t;2dG zL!M)5AJ77CS zu)$Yy(0`S166b6&PZBW!x+n345KsMlosu-2WOlv!KHJdy%4m-LKK*5~au@qB+8vXO z!z^t3=7~i(uf}*yKr|?t59{9=fjoDj$K)-lK%5s&RQduH8==h^(mIwBUlrg5CXfxV9(*xfv^h?`0N&$xu_a540XwX*%aUS({_&5zb<0Y zA#1LS{W*pLcGkYf)6UJ6`uSsB^gkxuaQbrtDs}=LVlPkCbfl^;nEt9nvwJpTtVQo<~)*^r1P7HG<_~iVOUNpB;3FGGAK!_k(_E= zEi!A;q$XX?NDYVtnZ4%#<_wBLc;e|&`Axlv_tKS52k$9A`LJ9}H{u>4E{QT`3be}P zsL@uW$>Z?bI=rJ1_0B6n#_WVS!KxV+W1!r6;9G;x;}tVNX-z*nZtK;A@MbNWimi=X zr@AOtTQ*I|Gu!&)wTn6*`e0nkZsnTlC2vA6g7u3x{3k;KTA_K~A@y)snl$ zK<23_`^XgZ8|)=5?$C8=AvOB;G=$N8&_~__2XfLWN%w|un#!-Rkii}C*;aBs40aR~ zUvw@V@JY3ycy7v!ua<73nXg_t@;HL8D01@CLt45K5}vi~^8K`eM~(8X3`k$8{AEks zR%T967lNM$>;Ty}uD5sP1Ja_|w0s^Ez}cDJOv$qV))fz^5h@?DOOVW_Mv_G8Yp*XSQd=Uo@}o4BHJP?e6Jqgl>nsy zfm5ytMH`S}^l?^CC91;mk}!(gizeY-nANOpGj7*#JG$Uz=6MuP{u<93e# zD>KToKU=ZX`)z(R>ChwfzDC)W*G9|teP1m36pTwPncv@IsD9q>SzF)mN-IVIg$ZMh zUNW|P(ucbP%XLj~Fx1IXWbW$o~FkdE+R)OdA(fljMV!r86P3pat3 zXMDAKnpv=xv05oZ1<&ig{gcb|JAn>Sz%YBTlkZo=l1C1AOp%&ixbIzjD?4XP&XoHa z*h_7~b^0N0r#TnL@eq0a26Ry4lGs9uvr688EsL6M<+8F(HJV4MT2!b} zI%_^;;`{F2tZFYC+L7fQcDVz>u+Tez+Ae_1(7GxW-tMv;nM^O?bA{EsYx2U#H=G3Z zrT4NElKxO(H9%%1IWvj7VWu8vUyAC>e1Fe4%4m=-k{K)4#uC)F#I2+(urH^~S8$L) z!MQ|!-#&L)V-8;nxNu$mqunFiZUgAl&hV`3C_qG_7HeqLY(Hg1h;dP?$*mVn-Oxc0?+SskZHtWwai(X$#O&3;!QpgcUk6cjnzz$5NHKT zt5eWUw&4)2JKicC^KpQ)pz)UM7DE5#Z^)`P(1oQms&})E&w&R6hR68;a6Lbn)tQsY zH4^Cf!%3KHSh5KNU}!ScRB3i!)LA5YAM8Ya;_(;62zLS5 zv8rVda%&gA*W%Xk?AS<$Slcd5$RSkU0$e?f?`*J7IJqec3JLThSJf4M<3c;4p6XCA zxmv6Vr_ZMOHO=Hzg-=DhhcPoQeaq3u`0e2HGqy#F`^0i~wn>`yrhsk9<0Nw(Wa7zB zQg>##wZBXaT9zt_sKs);KIGrNPnA^fwrT|)4*>1_`>2`Uf!iKavvw*o|B^qPxo5H4 z<|HGFkYETKvbzjF9luwe8l!i$bw*t?Gn13%Z8qdY{tF6kN_~$e9-y87;-J;46 zo`>9awjDNsX`qVmcC?{5NTvv%&GQlMH~UiialY1QX8Z~CPeBw5&F<{1Bl)mt!?n)l z>&F?#u7fTp&}qUl5mu4H95)FP5#2`8ZP0?_CH=;iYIiT!tNOV6a8FEm1gCx#3Hqz% zc@X0vee2!@G(Yw@^Jj-}TFu-O(S`8b%=KJ5WY)A}-}oBVAb&ciF2ZT|&@3j%Dykr| z<_ZM-s|R9nC4BK@!ADZB6c%pa{wjnayZU61FI=BG6TM2iN1?Kln_;^w?PsLP zAdj;~M`9Mzym|(w!}WVaX3vR(2bv}oryKw33r4`tk;`_iyjG`BtOQS zqkIErKb>7rbswM5XU>OXNZ==Km68U&g2$ktcTb^FbuavZ@2jZ~gIF?G&}!`ITeg?) zXO?VRbyj)czL0b;XxB5p#IACyG~9-*MmoMV4gV-p+Ka8v@Uv08`8YD{@&r*nUOHbVX27Jt2=efBakl{%VjdZ0#H=cUgZ_%rzcwDAwC_CeprCBpne zMh2!FKOxO=Z^W*)t`+8oLPVoxh2M$zXWP0W4e)4`bNkmBWKy+-9S*xo-mXFMH9<5> zY1z`tZZyW~5U}nb8N4-9^)>d#>^rg!d87tSu~Xbaf-N$uyWUq5TR(tWf}sX@$m}%R z+Ie6DYF(DicIB3^r|2e%rQ#T6t{l8E}-bgPd* zzbt}na*#20>PGmml~erC26st=jE0>$b^C?RQa^^g3d=VY(#3!NFMxseLt(vPgFMUj7GJZ;gGUva(fD z1041)U=*hiS_1OQacF-lgxsDzM1PTv*wp7z;@ky{h2+|Ln&i``54N+?X*?xe*TWKzn)*tY8)yO! zOl1viHmZMnsfy>mi2vliKwmQ1Qttmz74a`C&RvD9ytrS`%?~p6^YNqAl~PT4gl#{lmEca$b(Ug(%71lN6|z8(LyXSah+MZ&kxv8yv?gO^xow4=o7v z-ZjCuII`hujZ+|-eCp;&3>7nM=F7rAfY7}|EN%)k)rk%671cDI$3dXe9F{{vEMRxZ zGD{(na#G`=zU07x75^h|39Ro^m4w?@Es`|&#@<}Mb8SF5=SgwYTCV9?SZ1>`=-WRv z@K7bl5MP`k{Sz@mKB433@dxRHeRuV>SoyoD86mb*Wmx+ z>#8r?;e7JaR$)aN>gq2RI1^hxkgatr^lA2#MS$t~D`5f$fz z#LJ^O^X?>5BVSyla6vGMmHd6%qGj6yi{=Y@MyOEQ>d%zEhNf2A6O!x_7dTW;;TW_e zZ*f`8E44_2M&W_Ae(#MEhx^Le2Xd9?cI0axR8XDqrQh2I^5L9V69|A(-@j;eC) zzJOr^38fq97D1&EVM`+^DM)vh2+|=U(j9^U2nYxQ(jX;DE8Q)Ul9F3M;9a-p_&n!4 z@ArPce;8+s;~0DI`?{|cbIm!ItX=(~KJkCmM=u&kymhthu`=3iea zQYWx+^Cw3U(EPD4eO2!*&R-ybnfCo_KqVEzIC}g&HT|OZGCjM2+V%Cl-Pe9iJHB~k z(2D>md&>0aV3)Hz^m^r9#DS5E-Mpq|k47xIzXSvIemD)3eT}a@f%L9?&6Law8Lg1H z`cL<}>iK;cRdmGYJBJ$kl+_+@Kl8}QpWdeSC~B=hZ`kpol3quNA^(z%YEe>W+E3N7 zBW^_dn~QxKjjrEoSZxItF5g+sZ>l3!5kL^YRk)Lu0Hlvo(hjfl<#BfdUInN0X7tF+ zC_>A!HWN%w_mbw~wY5%rG@hrw3<_IovBs;v#h3fd!)!o-4E^zsOAaO=`_u-Lua^Ll zT&kgh8XPJ#pytYRn;+bNcjN37h&&~IiEIq&c5{QDIjXh2cBl^W^U}wALUhSMyxeJE zF2zV8e=vj17wGqC={RyI2F;%{6hhd&4F&?r*=!x(I^tLrVE~PP!S>YmnJhZ>_cG`@ z1XuTseSQAn1Xhe~i&FV<-)`ruqY;|CLCT(X1EhsC%Ym&1S_Cq8O8YiGF&XqfUH9rW z5s(30&=2F1GuZ&^j6gyqR|?0^Z5Iq0U%ozh?R-0_=KgEZN2xifUL9S4Qu|X(=^a|V ze4xeM1rtbUb;e%I0VRPPW7FAN2a1L7H8In-GU&3uH&dVE@soc*fXI>7qD0nzkoO;{ zqiqF-$+ZU_zc9?&Y^eO6`c;tV?S|uf6)SaD`oVB%KQ)&Je!b{ErH#qs8)OfVKBMcw%x4$}{~qCV0q`7P zKMcGW)^=Bf!u9ab+o)BGDv+*@YY(KCK}&(P*p&8^>wa%*NAbt3z<>8^=n+7ofyMfn=6#fZN z2(^ieqXU4t#C&&aorj_Q?nfPe;d2ss2D9kIO#E6h z{97jm%QKlDtxqEF7ZkxJ%s$NV zPE%{R3!USG(6QcS&RYVPj>g<>JO+leNG?&aI2E&?I;gVkksAi#{{?Scv^IKA9z7Ig&xoxnE8DD5RfgVsXR0H635LgyR9OB}DWhUFH&W(Tz+(cdTGulCO2v(=qNM=?;x zZad$}2MK%pGG*}d_FZ8Ff6Iyo#<{|zs+ZEe1J5`%EKLI>u=W-%&J^nIwa!Z{?i;P| zuOV86?2TYQjy1Zx^u~hNH$P+rI4h_9yq!+oqnA>bDT3-i(!@sdS+|3E=}idIg|tZ? zJN9mn1lg=V*jZe8&3aBOFF=>-H&mN(_#qgYkcCxNq2HsYctbbJzei82Rkyn7h6){O zj71%dJG<>_V=l$-VtakVNO=N>Oo|-xPdMaKaGvn!*2&nzK&cV6T#z-E0(ot0woup6 zL)Jn06$01Y0z&PbTiJBj30ebhob=N~kG-G|w(;u8we-zU&fEWdON?wQ_dCnro^$xK z{Ql>^4sIFhsx{DgUCIb9^h#HPfy3 dU(n*(oqQP}d%kx&=|Z!Q@Sc;hHs=L4^wyNdW_0A%^ay6@4aH-uj$Lryn;JEY4h!Xes%mM!Eu~1O@O38|7oN|S* z;&r;XM$!hY8Z?Y(|Ase`s9kHue+2IuC{E*O_!jLvFwcG`im!l>DBD+q)u4 z=pP+BzKQ9r4e#df| z$8ddt9ZCtm9|NbV0sY`^BF9qiHmN`_xi%bwX0jIHndRUXhiA-#5-Z`cQ(c-?1qUk1 zc1xdc>$^BOkKYry=)SlTtyI^TCa6 zp4z$_NT`-s`wHM?1a~09nth|=jL*Q;`m&N`aD?~9Guv9N=XK@nTM?8*wl^-nr2s8dC0@^Lc~%*|t2i4+hlMZK*|n zfm24OT}5Ys8rfv-@rS|UYUOd6r)nfhd3afszaAJ39%&z??aLgcS4*Ubp$^_ue!25p z@dUtfzs1~aBB++OK#Uh_+nw9qQMDm{gBS`szNoxdN(!0;H-g=6+35`+BPZS$dyn

G6Q6b0Ac8m65K)s`v4{hFOOs9-H);Qb~yF)Jx;vAJq1&J;MO0{sI z{|OT6))#eivL8LQoXT6tSi&YfEs-zeyQWy_T7|24Flwtd@D@Vp)3 zrSp9ysZ#+g=aIwpNn~A8OB4P6tbu(={+kMOYXaqIfy%6Oc3nz1R3Y^PmNvBE9vfdk=&xId-+H$n(ce*T#HC!vh8@;&#R8N!h zOMY*k1`gyTuZNzm*d>nnxY(y4h*q7<98`M!k+L9%jSD!ED?wzh*Vt{gtJ4-X!jK0y z=KEC&I9e`yanu3!a3AhW2MtDykEIPCdl5EYKOXsU>SGUnRO4fKUGB0{_z6VT|Ikbj zf`43SSVDId5_xlcB1UcZu@eNAdV75~=PC9M9uEhe)gb9Q3O~%Bs5PuJ)}0W~+Z|hV zHnny>Si*5e{8(^)C)juAQA=w689*Y=EU5z1h3k-AUE^G@lU&p(Od z4CPRe+dkLta{m3gswX)lR(^o7^Nosg7syBE+T#O1z161K+PUV~Qs7oF)>u{*P_=uz zYNxm-1K8X6z;**E&iaNgtrjB|%C-X#?K~UnN)W@|+*{5sdLLg~c&nq&K-`+ylk$z$ zgz!<(uJZOfJ~y@UbL!_%2a>KZ%_OUI5J|T^>%8~R#}PvUH2xRdN$-i!0R$2%Z}hoh z7y_!&bN=o)sOaa}w{I4^XV;Q&O}|rupRdS?LonH5n3+V82L$E=u7();*F z$oAK_o0A4`oIT>TV-xavYFwtcz#_*0_}t6`puFC2Uw*L}+jZ5C`PY$6|D*Zm{0_Rm zi0%BxbH`-i$(?pbff|4j)Bxhd=O(5PZ`4cTd0j zYoXBJkKmsd6#3WZ5ym~i%|CPGZ{Vv!UkmW7vDJUb7q9V@Qh1v|`b)u;oxSY6w@MX} zbJ2QaKci=F z{|_9=|K}`yf%U|7y*jT4?>^o1J-qvw{7T?vA10e}s`&*9E?jcE1~c!Kp>jQLIw8O- zaz63{T?ivU^1K(orSc*Gok_htFi7|Fyy3P=sl>j7bBXeQR1V_lrik2RR^`tFu1dE8 zGpq{N@9NZ0h7BFp?hU^r7sH_8PHzyf8{?V{q!cr$y>f70?VV3f*UPsqInbZE!h88D zIyILVNy_l?Ynh86k=lA@-;bKQVq2)(Sy3I7-a2SZ`F}iRxTs_O-&n&PCaRc9cUS6Y zwTsjyF?Z;VL5AD0BtBGNv{-Pm8^_wXW$@mu9A9LeGNR_Jvqbl=P=e}kWQLE;BKhH{ zuITLLgo~&^EP@J{7hE;>7%>S>vE7%xAfcl9*!kbe3J^shqCBx5>u1x*@`p!Q-6_b# zs84ohJp7^_)-k9G2EYqf@w54<(Z~vkA%URbZ_$JCLkGi&XcaaKC`#Yv=C&a`DXyu% z-%9Rv0>Yi2#30ZW@B4lCh9t4Q9D&^}0(G-g85ow8rtn$m0-7MzFCf{Y@#yit#LkFf zO%uT`{3a%`ZUPd_V|A^X=+c#?UPI{EjN{o=@ZIhfU7!1IG!dbZ9@H``yU?9xWLk1yjhNeT z`@;M0HWNgFIV(?o`c-u52Nzak>AUb#RcM_tP(7*hXpfScW%S64-`gzCPgY9sJm}L!O`q{jQdW|`vs*^zj&Zmx&3id zuw*~f`7kgH#&R|PDC`am5M#L7tS3zj#m`y zu8#6iUUM*Rd~{CGb8F7H7|c6Hah~2JkFbMV+0+P*D~M=zaX4H8T1&AlJ8E`PG!#A zS8j0IY_aH78Jlod|DEa^J#5aVeK~BJO4+*oZY0Ib5ci1yMAdJ`7>HmRc`k6R?7{2F zBCckoutwpnJ4BYCgWuVn78EqBN%>1S{F7R}+7Cge_OAn*)BNC+#F0a7Va9RpYTb^y z?N-)+i7fFsM0dY3R-S*wuyzDFf1<6;+0w-? z#poR6ZKymrl*5q%n>0WY$4jtj1#zc@8FP@}CWt129;Nnt!HE3-qZFRMmTozMNs5jw zy+7YUY@(8{RbUT}XL6o|2zS?cHf(i;Rq>I{WkzB)qmdiaXNb-XB8YMZj$N0TZq@&%(r8oZ{YU#O8hKG(d+*#F#?QfURq3VR;GXd2L652J`?ZYO z!@raw*eq)Bvgl~DIth4z4Z?#N;fwzY^69luLUbr5twBUCJl7IMYX`;;TY}Vh-fq?n zfe$u@Q1e%{3k)q}ngzTOzdtTSQUgwp58&E}f9jg?qr>)6AVCI>EHdyx9NOQ8XF**) zwENW>Duw^1w17t-Q0Dll*oMQP`Wk}{JP4_8xaPWd^}fq7%ZaLRS$wJC^-Gmzi!_qj zYW+f&@4;Si(iPoo{nnQfJ%3Tqf)G{y+kDWSyqs|kJPKyni-w0===#N|8 z=}Uj>!9aq%KymoZ_(m8F8H+{D59lqrH5V6_t|WA z18J1k`QCR{Fu0X?=ke-w{R+i&jH6d z!+3ib{im5Kms0UrjeMcv3*7J2W%^WxR|F55sMTk%o@F=-I zK%kzxGV=rlz~ip%_W?+1&104tIm+eLE(@63l{R&eyX%7@lz-Q=0gsX2aCn9Y{v6zM z{a3CMZvS4)j2;(eHa7}S`KlkExAVce?1-rqMrce)jdNYE}luQpHksRcyv}d)b_S;RPz2w zH4T5K8vaT05Tq`Whmaw^N5%wyc4wCUC!8Qo233kJr$M}ec?N_|d$xOAIwehdV|d;b zw)Kf?gHAnD&)&7iJLHu%fCM5 zG)4z#lQl5;wE*RNM5xMVzg%iH-;p@A`5w}NgYGC@9a_$l?my|^dZc8W`vFK>OMJCU1wXH~OSQ3}OZXWE@_ zE>J})tz6T4|K&M|-yVfPIUIx7xIZ>bKA|Q5zn*)^Z~JC&zrA$#^M50HG6n73a<4n! zL*%?SyG8$|m!ug93dBf@HKI663kv=JUvInkt8*yuHLHZ_(1^>FdA#ylFh!zzehsp7 zOEkUHa?LDEU3Q1SjN#1#@F4P@1^wm$l@q69BYN6@ddh3%${~aj2WX=a_SU%9N#SWaZ?kFyk{Q}W;dw( z@otafdEanbwia=p5{BRN(`27OOLHP({9S%3ey4iBL-gczrvIEUedp07SIWd2`)+59x~jn@KbkkU{|}KnrM%3MV0rG(NBf9XR0#28kIaxFbo`EIhMY z1$AXa%Henxu<9ETr6^ec3_6kUQF?(q(;jrIhS>48n}`o3Vxt5AxLcl#X-<(|WfxN0 z)Bx9g4GbK^4P;5c^Gv1NYT6gAT#d^|2*}ub{|L%WJ?>dZCa$B+Zp-g96f(e-Rkx<@ zHh!mek4x_+HQl!-f*nx#F{t9hi#z}D{psgm=5R_>>wg?=E^(-0OGKY}1|G>{6h=No zHv*I%L2L!4{~hZ(SLUs%nJ3itKYz@8C*NKGcfl#JP%{C^x8a-52N!mh2S3vRPr|N0 zJ5r2eei6Q|9b)C%`BFq^p0}6kEEawlg%PehCB4BDXN(CLeCFwchboJl?e$wAs;W)~hyhs2Pnr-%*bKMKbY+o+ogA_r~ zhVsCY{Fm38_)u2r3s6VZsVI>$yvSUYROYDb8BIH2wB2Jz zoCl7zjkQP-01osHaKrt%JL#_j$9NTBR`jhv6CF-sDIE0>zl0zG3nh0uxq9~sQmr}b%eE3dFA=ho|Cd2@y_ETO2FXfzy*~B zJ+{>L|4~cCk?wqXCh!?>oaAB?Fza}w%rE{O(a6rypS(jlI>!@`;g=jA9X@{+?(#WO zXv!K!`$;1013P0NX{3AR*1?f80_x9;&}~Z}p4(jXg>OAn;h^U~Fy%3ex1yo{oIy9$ z@4!Z6F6Q0|!#XeGj#g~e3c%`^TwbCyq=$hsh<)|<>8D1&-dV*jKPR0g!5o1)kJO&?`j_2{d!5{&&_krt_PnMKxu+Uy8`BQs{$N^&)Vc9uygx|LDWBnSj&2i#~@Zu znq?mlX96$P>=JlxdSD{Yc@Wc4>%U%Xk!>aeqxFCf;6dg9<_Ot@mT(EX8p$=BnB;*X zEh%#Rb5@b&t?jqmhL?!y5pkep{ zx9Xk=_Zs544vy5kl*1noBh@^VhAvp~g&;opubKyZ29kNq9@zxbdF7(OTCTAQah-mq zy14l&<%C*o+r?q542L$~lH>XqFmAtTnOuaXSj8A-#(Q97iACo(QfgD6Tz9JKcwx8B zb6^Q41c0}H6)FiUz26lN5${+MHl=s_+l%9f2^eJF-`?|}^+01LPaX7)JNc1dIAiex zxUZ3u>R1gA=uOwgBht1XU;*p^(QT@Cd99AKu!?Cdw)>^XuwAR(1yHr23mP>!mlqu; zs+_aM!ggCS{1$Y{D>dGVF<=qU*&lqm@jPy2?FtkV-gcXSpl^>|8*acV*Pa6I)kq~n zsqj$A)#n(CEbpsQZRsq)@^77GPsT<28ix8U#3es#yY!H`!m@-*{r%=P=q2cbZu!98~5L8&F z8ou~pl!4tDLZv32mELy6|DZo@67QdzRjZTLGpT?R367yXW%S95lg7*q_%em|!CI6`so)-f8Mza~ z!}uW`@axc5lMWrG6}S2(zz?(s2}?gD=t_WR{k;FSNN1e-tE1x29S>cooN;EpY0- z?v!_TADHCcE-m!6j zoTB2hC{8nTH$5Bs(wTCiyYF4!_)Zh5aj=asV`M%DcGpMYP(vaUpjn}+<0y~^MX6p{qV83t&Y)) z&!=TkfxnnSuLMq7>gwFPN_~~_OQ@3c@?&|&VO@!xdtJbn_!9Txc;Ai)g}cM>3h~rM z#|5rfg8>a8Fd0qJ8ejHIkF+*r>EPb%4>ufJ2u|FSasd~UXOPvk0<&D_ZKjJOZ)!~3 zqdHW+!8GK;ur_ZX80NBi7&2+7RyBvZ)+#i{ZsC{KvOaxxjqwf}uSKm81NSo%3l^_? z*Ka_C9iMQ#n1Q8Z#^lrtj`5a&Y%@H6lO)|MjvtCNfPNa&P?{#T(TzS$4DnnoZv6su zO7`IKc}Stwt-gSr^9}2}D4R|LH_wtegd|8n@Se3W_&AUV{5+b$ew~e-fODGWO$_e@ zu;4DAAQtnbU=B&{mU2mla5OM3EI9@hlP&g4R3mInJv0@A=KU);MOI1i&{X0FvR~B` zmT(^S#fD>_h=DT+U##dOEOP?7Z6EIHZ0YE5*+YG+Gh|1`{H-b1o4pD5v8KU5)Pbbu z36r5M5vom|@X8$9tLEpVp4TOZVr!o9m_2D6mtO^)8^8Bzaj-pGQt|j~y?)AjkhM~~ zcBU9FK8wk}SDx6bX%)90ZuHCa+j~=ogma4}GsGXC=CZS#K+8QZSo2(YrxvcrldLcG zU&8d|1TwIk9<(W?n&DPI0gs>71?uUaPa-pB(c}BXNa}I2zK43^oX0V)JvFCRDDA6Rq zpK4{nvRv}cbM@aFJKE@kh;O&0%J*-`z0mmoIx4mB9|`97UrsRB z#|A$3A)YHW{?UrTY-m`|DAhFg+D>7o%DSw37i#T>jN&?Y{gW?TtzvLymY#Xx+9w}s z{!R%y<^wqn(OdG~KrBXLD$_EVdc0w|ih9^%lMKwJadI^H%1%HJAi>LO>8gnX5aVzysu-LCxdaP)#lzUf6u9iOxVo1OtIl%3{%P99ueup9Gk@h?4 z*gy)zEJ?R|!dANbRWO~c%a&@aUH!K(POdk1h3^fX#~{xVt@dt;k)3m2&|UQln>73~ zeHk6OSSBa(O$U!-vg7poaNnd|uZ_E79otHYboJ{O=ZzsqSVFs+?5e6wv3#qIyk<2yv8*17o)_z*UF#jX}TYQ zr-1`A=jUlb^OE*a_9wGs(`R%dd1zyeOy_=?DOHZy=Cd-Tg+6v;O@8fHz#8B0w=R|O zD@Xac)(ZDz4t5!CfG&&564N~p20m|KE#PfAFTEENejB&MYUK!TRnbD%F<6puShx@7ikxVpPLXk~OYQ!GZc=2yIk<{&h2QvY~kVtT(zYAX9SbxtP<58%qWjdoCh6 zJT1>mBm^ya^wCu}Y9=NUET)VnO)(xMR+l*xOOe;R6S~l+xeCY1*!07&CCdDhyPH;~ zN~|gw^EJZR(*iQ-)))+Gu^FypEPaL~=0ekR1tmS+AH*t&Q8$_17eENLkbn?cvLpTT z287VrK1Rh&X(`hWRd*&N^vDj?r;Oj6vl0h1V8(m;FjZf{v^}f#wNuxxy7_bKa2z zmsJDuG8#GtUtZ{v-hv?h8!28bw);oe9HN18sS2(N-g4ylvR^}ZOd_Ags$|VrctFRh zH+tzF3goz4jS`O)Q5lPfsX0{QxgXpiw^N5?*gm=STH4$m^tOp|%x~yxQ2d`k`6EB> zv*dH%Fo~ttqQv0sRGnO1*=ODveKvi`pAvqAWKS^8v^Yhz(S^iVlAs8OrRz;Rx|k|U z;I$ib*F2RT{u{Ipk3ltyVi-R{w~I&kcfk2`ilDP z1n%lHr5A-SPi~T}9W<#A36|G05t5vqd(p1lEY{*!E7ngbd^gn9K9Q$F*6;l@iJgh- zvm>@O<@!Xb7^DFo2|c`e`$Ju>85H;ZTJFe-m(O_QSVnr9)l2uhdog?99E0vBfeaN} zJSGM`A17XxsR`+w!H`tov~?==!xp1kF1<1o<)I+}%SY3QUTD^sMs*p{i}}^Q zoY>8@X1F#HfZaOd{G|2vrL^F(3o~aM=cY1O&2ToQvrZAa??yDu%=vAG#(TAv-xo7) zrm}irqSF3>WR!QBfgvVF(yz0#H{l5|6{wU~8F zK{kp;XFq%9IX=gD@=$CD9ZSbAXVNGml-Euh2342c_0Xyov_LNgAecp@h&^lN{=wyQ z9NJM(;R(iWOV1w?yIil6IBy}4F*Vu7Ogyn&PzaS0+37ilpq&&^&-*`k8TIvd!;D|E zk#31J>9CsY`)ktF9DygkK-(eUwtso zR7uFHOD611AQE11plkbXN(sf(XX$2`|U11Q;b9%~D3AJHPr+R$Dp)T&p*Dh)*>8i3FDNhX-V@@Or8_bE1*}eEVN=HHkiZz$iT_SYNqqM1io4C zQM(S*gT!gZ*f1u=(eoL>rAE ziZh4KmHqOWN9JmOt=YDyhQ&suB?>EY$A<2O+4QZ|y^D=fQSE`@WxY!!cNQIY|H`;3 z2N%9uvOMO_*p=zf+FLY$2;AE3dsa~H?SWa4S8@=OS4f!i_O=|gXr)eM1+sah$;4-T z)?-08v-*Lm#E*j5m7Vb(2buIy?8h0C*R9l8q9?B5$L3o6g!(f&#F6t%n79s+YDSJe zDpOje`XKX-c3>*6q4wPMA9x)6wFk+X(z;g^>0T1{&Q=;ZP;UjcJ8Xs+RoJHll*M9S z5RQ^)md($Q=rLo@x!vX?$soP+I4WODHj6Njf#+Tz_U7x=5tS%<%!jKNV#Sqi*2Ee0 zSyCk_X(`rUFlS~k>w_nTRn5V&#a;VjC8&w-?wl|yJqZ}V?u3=mjPr~m@NcAt8{_w2 zdwG-#^K6@N5Q}MP!Ql0$v=mIs=EQ-Q__N3#%Q>9u)hK8%BSCqPzb6tU{A#@y{*SvN zT|sc>C64JTl)R<>4Xd&^C!e>Ylz*2z5D}YJm-(-^)(2Jw`%Q)-baJW~ z3c;tQPkotu@clq6d+}7HIF}+8N3TXws&(%JaYOO91H7*9Lwy*d@p-iz`DK=>Y^4d} zzbdui(}|8OKaAI7A|2aU5UvK~v=fXp)2`!PBk1^%9EkBysj*ZoyB~GwMD(h5DD($w+0mp^k-HI% zB`ifjxWBg3PV_foCJyx?U38ooFB5%~Vv~$%+H#q2#mERgG|!OD!0J6=uOw^AKu*fi zyZHA>t#{z@s5OgJMkJwWpIesHKw-JM$e@)r{ONZ)ag7|`Ph|Jr=1S{C)Pr@Vshr|N zAvbSFv1;{X>6R_pt3)M^b+W@E#JwbO+LAZoH(I%#{3=Q#h@0@0)2^v@3VEl%EVPzJ z{q0;1CC>}fkhdGaiX|OoRoj!^vJ%jw{OKD8rXYTpIViYm5!huEajnf!zuJ#q1lAZ0 zll=xtE}Vg-bs84+Wn9Bq!zK~LsFy!DjKASf@uaD~!8(_PN1{;ZO4#~1gzn}+JNpdv z)IA{pDl(D@xMRB&R&$Fbu9C$?zF)8l4-QF<(H3<82|2sl8LDp4;SZsqW|*TaIytF! zN(V*dL%NQ!1_UxE+|DbvFv^ABHJ@8Ql@^i(Yx&svPc_h;K3%oD&B5~71`ketRnR2Te@{ciboTJoT;bE7NTyzxZvS>aRmt=+gDKkM% zG}f1NVoap>Q_T}OIroRR9$991`-J~p@$YWXs-M8ln6l=<`{Dboq{T1)eI!Hw?;~+u z&-l-g5UsJ`D<{lmNt>&1t;k6eOz+KMx#F=Gza@Gt$GdMvNlL(AsQ>86#{CvrLIE9I zo$GTmO)*Jqt(QtB3i_{@LuS8`8;G5hX^%zzphI6Yu`=z0r~88 z>a%)^AB%2beN@lU z-3gzojrHORRWg+?Q1DnF|& zj~~`--<^1#B}GJjawR2yyBk9m#nI%o_Ih;M_QmIr3nU9mpFh`}*EwtS1cPfFPD12- zw_&mBK-?-E10?-@y+rRMR^pUjs78-r#VDLnqa=$dcl&4U>m_JCzO=x*M=vn{D$wrU zty;@!Y%ZX8LA6we`yfEq$eO$n&*nx)NUS^I_xZa>ebz=yPhGDDO>z>>)p{G>)(m7p zmEe2KVDjOg)Nw5m9VpBc>#VG1mf>#LXRH=VYH7qwBqc|)0c)K4$_$vaQ2^jer#N1M z;1%<9PN&|dnvp4Cv$%*$eW&jG{_i`2RNlHsOMT(x?UD_|Y#Chm5)wNkuZBMmJBhq- z&iq_f(A|(F#;w*}!i4v7X8WFOKG2B>&qnZZ5zXUkW<}a6GFq7Xhb%d7pO#1IWP~fK zI}OWa_>NjU9Sw+m&fZcG@Ah(T2KPyaIUzNRLvxI4TVCM>4r6n|%Xtr>9io<+&^Nj?gp9+lY zMgm4`LVP-u6d~0K%G+S0_mp<3y2;UkmR~4EHUqVt+5n7P-7`JbIy!WJiJ6QMFL$lM zuy_35NyX{^om6TUuO9rBR2KKhq@$(1#Kj3`Xg)({be8RUEet-g&Jd`rY&ocNye*ZK z#*O7MCFG81_?b!r-cO|CtVDG{QYkczJy9TYNDmdZ5AIUt}-cV5->`YUgxlkHf@M zkh1Mdokmaj3=hHRC*!Xm3w3ZIV`-E%6z5{=4HOjcbV7IpHaB;9@p?Z!9zo`c_9Vm$ zAauCBN-b!g9B@i9l{24TD&y51>KFN`IWK3aTDZnJr>Fb&z?LKUyx*B`EiD2 z)c%UukCg@0Ef?x$`weu%NnPZr*p=#&<3r`No4)3JRni$}_VID4J_)-lV5~HA*+aPc zG!K@&MQtKXgP6xL_KbE#P_Wn)YI9#alzub7L}&KrIO^H7&)}b<40L3w?fAcQJans{ z{*=ZY-6t9=B05p+C9Brb1ZE6kIqbJ3fvkP|oMOlqiNZuD1M{(;Ew5CXN{HoOtsqM| z-)_^!kH|6CJmE_e$#y{p4FRXZAhgQr1e4m{*bt(J>YXbZ80RI@FSZ9}1j`-2r;wd8 zt#6>($l+ljaKz8xvUHE(}nx-<2<2sqRqdy zogKHF5m#$A<_)kdOQdvhxvR>K*DIApXt>2_mLACU(yop6$7ZlA4M|ZwqYOFkJ7!)R zZ~`w{wY=AaN@)M_aUAFntE@!5lEhP7NpKdy`Wc4c_*)Y;p{3YKveiB0YLt;A;y%%XN; zKfj(GClD2qi#M*VgiXBMHB)a-F1}2bz8s8>mM)giPoIhYrEuj>INNppI{A5nY*y%t z`5me&8zO?8e0TQz>=s-I^Sk@M(j4A|G0TMul`r|Ls&>r^g}l~FYX9yvi;x}LKLtr< zy{{}_Sl$A>aD-_w+YW45%6d;6b;=*mq{fK3M7F7DVsnd)1@pczXyeiCrp)IyjB~HX^DcX5nvEn~;j8a(sim^yy^vbMUiOBlg86A1LdRqVe&qkK@{C+|) z-_~pRT$sK7z9$cvdRU;|?HVV@+2!Jlrx{DQu{(6*(uMl3*9T?LcYv^a#$oH-<2 zfGB&OoP%oqc!K5r|k z0`(SfopGELY{|h=+>U-iLZz1TjIy_x%~=@|^Qg|svuMAA9cZmX?kg?!(Z|xMDt;8d zWPhEm^>*Bng_`tsQd4HFRu?QBY@_~fF5HnIb zFNVK5^-o`OaQBB{QUmB8T~%cs^%ZIFxFN~KDsS~@=nIrxC)t0YxYx+Z9Yt*JIj^+p zJHc)`uRqBJk>BG8TW*~|B`6baXz(-R#LE}GgYY%TWnZ$-|)U4BZ5lnAN- zRON^$ShHAyLP$B``x7Qrj5F{UTSy7Zq-z9hfij={;5MuOvuA$=Gr$Ab5IjJ`iGN>L zzsN(FtQV)eEKQ0f>RLQWUcX#aDlkeWxKBAdz+4riiFp(%!u)sGt$Z5SaO#C6brdv&e+t}8$W*$ z&TGlPCR9&&p=SE1@C9!%2IrPo9ZVQ1L`iu_YEWn$ruM5-9|j|QL2P7dQAzKhIZEXD z8XEy=Z?SrL-1X*+&gmr}M&bs!%|CsmP_1SfLwU-mTpjDT|-9 zSg*9aiB){Shxga0vH0D+arwY-J+TF^A(y9Xe>#_4nD1!L)dI&zZQm{N$nz;}>{7t> z&H1jMZn760Km&#=Dd;pMDpja5oDvz<7IV-G#a^wm%+}qDQuiZ8GY2Dl;;(O1wZi6 z!zZ5EE=u9lZ45@?!?%55gIziE|4BJYa7Lk_mhIzAL^(yuikj!8o1x*w?Ikxw*1@py z^c;Y{7m|OK(Sfye`EZqV6~+<0f%!;-xKTf7n>? zbMRS-0Y;a_$N6ji`{DawfUIs3?pPs5DA^}@&#ZP@y5-6pDLl;%UY>m*U_OUMr`mqr z9vG6>tzMG;iGvSnFgTRWZMmPuxUVEwpt?l_*M0}WAqa%g{ono$gakJ1SFWmyE%1J> z{(SHkj-J^N!XOMA9qor>osd1-VWW{LC)qM6&z6=x*_|G)C~GDT&Kwh^4?__O~Gi^>GKhRZ3Ui)y5!}z>sE3}=Tq#X zHY0h?o83@TGv;ce#5E1)dFCWH|Gvxg*l$t`LpMO2JTBOia4M70O#eKUV*QkA{SeT# zx+18Bx?(<$2rOXgvx~c@6#^#oSL~EwNS9V2%B66v#9U3 z5&~g7u~zULW;_WAHp%3TDL#4y((UQl*{EzKY*q?9Jn)Tn*_^J+bPXN_d z4C2Ta@1iCAO^U}jiJvlc<2^wJ*2_lgB;k{E4oux#!dfw3zm2nHTRbof@mzoFV8Q71 zlMZ)&J;5dJLhvT}bTs3vo;CQ~xcQKbb)zrbwl0qllr*~$H8b|J5SywM7cd?!!`ka* z<9=xf*Q(K1JtXkl^nKef@mOqgM$Z_5gxa!W0S&({;YF5=>0dYO<5j}8N)bFh2~=T(knG4lXX~YD}87q)JKT&7?GEjEpTJB ztil&#yb{Q}CGlA9o$ZoH?5If~exf-D->q{wpFh4C7K{|L zMzCI#aXjDt!mGsv$7FRla!}>|@o$9uexzK5yreZ-f3kI9B3f}hKi47^NskKa`Kw3d zv%=OxH7pmpc-Ey3A}k{I!kS*Uxl{(DZ6b~%k7}#dhZF{VwD`e!`k1`(km)i9JJero@M#$aOyF=di5I^tZ_MJM^X7e% zy0DcZZFWkot;7jx$qdUFQgAq8|JAQq0nA1ltPHT}C`u@fd$Posas@rZPV#`-l$fmK zZ0O^d88sZnW(+A5^F^F!A-L|CqmOy0AKb5c_l; z^;fLy0GE(_KKa+ z;j`7KlGQe8f>T7N{ov&vrqQkv-`kj;egSpW8`=PdIEIA2i}%YTHYqs=`L@tBup8dt z{>H$KC?0y=?mt)L9vh9I{ax0usXE6DpO7bArl3nJVT?|)%f(Pi6&=0O_u$WQq;n&M z2=0NYic&o(TYv25t#ZprPUey`!sXzC*1~-qhidB#>^BlO9R^<=M`T~HG!=E56a&f3 zMRgoPc2nHzn-AFz0wroGr&gsOieuF6^eN(Sa4)YyE3SpFwCukqGA%D{qT0nmxn-k7 zJmuQ2oY;8fkb&~Yo(ZAGn&F#ZX9H`;X`rbHc6TqnJ~{D6wyaqT4PWfrKRS8VT*Nr@ z@C#*i&^y?_G-?k-vo}{GtU@HNA}XFkrNJ99OO$fN84r^0ZX>JN7GNuUbzgT z8}4K%uo94vSV~VPjv~QQRO7hrY58*sJpjk+QQoO zNOySIE&i-uk@m8FbY1*p+!fMClo#13ssIj02y!I<^B(D;*Xz4_}NW1>)%S(~U zWs|u>6VZkItD2|}qm9XtAwK&x%d2^B<7!hkIP3cdNduUO;2Pe08R6_Xyx-#Sn8%8_ z1zlzLYp>~a`<3DU9y&I1=x26`U&iN>){Zk?rOf?0c`NITu*bpfidD}FzOrrQGI>=t zxH5fGAah3PVBQ)Gp&M&V_Q`Z$J0D`sc)+w$C#k@A?$#7ttsrA;^7X4C)wAd{&Xiz# zEH#x*ezv2~RmN%JZhfGo6}W3&c~hY~Oj3$snyeCRlV)o0U=!43<*Nka$>#z?cQcnQ zxZ8q%eWJ8xcDY-Yc$y+(2b?8sUl|)?1gtFlynH5;ckd1t%|&0o{Fh>G@bu(aNIfgf z=Gets3L>hzeMI$+qORyQ?0fKU(Ijjd^tJi^yjwLMS*Fc$RVH=3xj<)ycGG>%wAno@ z21_Yb@Dp<#iU@UGbo0u!mTAg*l~T&->p)j|E5VfHMAvc_u{`;I82b-+Ec^F=9B+t= zlpRv`C}d{u%O0U+=zQf{S{G@rhBVV+x2l zII^G9Qmp;xe90SWEV>)W93m)G6i?-kLYcF9papIl;P+%G)C=7TCm8jvolbg|sPpu# z)12c!f#nr={q3PiAM8a?11z4y-4SKu=+QK4_tVT0STC;fae?8lXAKDDrN6nku6-?= zr|a}s)lgc144@(&WWS#>)s4%njj2WtM?BmIWyE>r5j|esYg3jXez|pK>t^~bZmN4V z9b@R|N=lEV-8Pq{6ArqjhZur*>o$RKWhlsNnqyVYk)8L}YhVf?mOvXkey4$p%H?8Y zRfTA?qj#b}6_S4eX^$rJQ8P94GEoAzp6LhZs@7~YO$Xh_Q_j3Q_Rj`#2ANAl9#Z!2 z0HNWaYi1R(6d+H91M|IY?+5g3(S^@cwT$hSfvnk@cHVoEjP{~HPONckV+y)P+m`*1 za`9=lS7yojl%ZB6NA}`Y`b3D4mPKiapwR3YpH^&BdH8aSw;Sawj@j|-+9$`p`PyfU zj+}&TKTaj`Uwmtym1K46dZ_HI(p_(Hg{k1nEFK@DJ|XmR-ph+IX?MaM*J-wWL%8o8 z%k~C&fMb}m;XeTaF%;N|0*tgi*&xGlpHJWxv<)x0bgM0H)v@rR)Er;4V8SQRce7*x zFd4m6xR{fWYK$^?ou~4yj+6X{_iP<_PY?^7`na_^Mfc%KuNzea!?Wd&EhSTJFJ7sy zjb5nD2}2IsR{7%fas^FJm35#|cdM}={iW4tL=F?Od8YV3sVG1>8{uIVs_{MKq0K%D2nWI1yjFnB*vE z-vDO_fyJ}(4+2Q9>+Q9~`cLiwM@|>Ov-tCl)a0&8gm{_%oJ^HxT#G&wRe#|`qTAtA zb`F**+SHqB8|RPe*WM6Yf&?)ZYBqrY zRRj?Q{7vgbkXU8}-c#B5`ssq=UBXSc4-Uu3Fy6Y4n&j+6E;;uO=r4MspfJB*5n_1T zN8#%6eyUXWtIWq9Ur7DX5>_ZEukgZ>eD^r5KyG*DQrdxLlW!brG++2w zVgbx%i_&i4b+^G$gp=2WXnyFS5r8m!<7W6&7Tu$Lim&t}hH_KVvL|i4oCG3#?32G* zxG#;C3flv^Aw543l(rB`#YNtlgF;6duGuh+<^M$yhM6HOU48Zc9`--a4F6$dfoRek zPva>vMm%^Atmye}$Yvj7pGwBq5BP4@j3qzZo#wDTUJsI{F2HGQJ40yhX;cFsG@|gM z&0hhJ-@n(qTz^OLKtUa%*^=Czjtv1l$2W-HFT_>RL6zlM)!4I6&l6P*u>X!2y~|b% z+qJkil3$Sq(h<3E zP>jh904CwfB%~1zVK%2cDew`esuKQQZ3{rCyfphNe<*Wn4AuAg~?{obBM&pU-&pske1Eb#M<-&M$u!t15_nX78X!ohg@AEs?$Wd8r9I*n(5nP#?^y8V z_9Ig=q)*$qcKPOOLQj=8!Wm>f8$hY>{qd*Qd2&tjC#5z&nFTvo543r~+-JTMf=gD4 z*!@VV$Ruy!2qMYb6tq-N=s2fD%D^N+n9Mt?S|65%FudP68>ynrr$|xZAI$Wx5<@Jq z2elu#y$=8IK=$tRB0|KC7JiZQL7{43H4&N?zS@2p{9wN92L5WtyW>>JL34 z5XPDzy7kg__Ed8*jrj^ve?-?vE?NI$(SANdNMS=3djYI?-MKH>wQaA~u6>e*SPu`|HWtNt-2w@lZO3d+`t9{2+Y*Qh zq&*c{GroNwfhbOe>wAD;L3~lGj_f9Nj2_EXQr1FyDF4RB`yfI}TVJj_w9(Y47u^uf zj=q<5Er8m+EIG`=7bGktssLdLM&>gAcDLf@&)+qC#MUNa*r#zgr5@Ye_*Wm%p3IyqI&G>!zAaY zjn&MUvMmA4ynjY;@VGdR(FuxVdgM*#62CssJ&EZ7DGk@<1@b#sk^1cm*@aLH!oOq% z?vNUBo%mvp9tb!;13f7gFcy9~n_~a#HV74L9Rx!h9S^+ZUpPbveNc%jJ;)TGkgmo4 zW1K*S@8xID+u71@5imq6t^|oT@Y)%^MQ_eR-ft#k5}~xfcJl2p#moSvmHGn*4(#s} zUKpSJ+k1NlPMc*f8b86GLf14g{ z!HiZpQ=K0kqCqQvzka#eEq#A@e!&O3UN^hAo6(!+whRFC?dLy6cghBh>c2|fj;`Mm z);atQf~&#Tdd@^-qz8Zh@*id|UU<>@!IjF{myFb(lLcv>nGC`T#Uf9z4nB`?_~#2t zHCac;+yM^j0&jp@Agd9R5i9ovI6Vc>ra=UYE;#Qu6U-9H2XtVOXV+hTi`AJYk>1K2 zg8qX#BX%>8?Fu0_>>~p&&?uX_Z7jBY&`AAhH@P~uiKf^LrBe&7rNTF)6=HN5K!d5r zPm3$86;^+FVzKn8D!QyP;^%3xJ(C!1$opy|W(Zv3ZHIxCaQmdNjTOieNZXrAtAH`C z(BBvVa)=~I-^5}hp=qoW-1OsqyV_MHCvX9IRG~KnJ5)CSt6gyZ(W`EXNN-P5yaRom z{15Lbk#cB(@;C@8t_}!uailxPh%MKyGh0t68N3qyc;m1jFsS&RRw!mZA>?Mx z%6qb0&trtAAp6^wnx-sTt3CROEWFWEs50n{3wf|uOet7?RIpS*S_scyT&1)emy+dwkug6r5L^^GYvhn?+f#!xft z27OTBgMg0|`miBAQz`4qKO(b&f0H7P|M6`nj7( zNAadtO?43c6r#1jCt$I3s>l?abJZ!XeyVd2v;SOMHah{)S{XBi2eKial9gJf+12PO(oI+ybnzkQa}XpI?Zoy+xpEoEd~|$d?G@2sBU7e6a>m!s`}4w z9@6}2Z9}C<%9S?5yn&f$N6`fb$ArC!2dCj1*GU?)0~?vMv9Rf-s=pN``7CtZ{*?=> zA8h=EnI4PfgC`qFzu%|UIn%SX)8x{4$_P&QX!F@_1*!zNF>k zZ3WDuf#c7DW#i#VbRLNqC7%0zlo=1rFf(T58JJ%#LIp#RM0z>tHYRf2z!r)4F%za4 z50S`bp?-hQI*6H(Ec*#5OYW(x8Sfext9R(??VWO?bv$Ix4oG?l0Q%!}NzGtF`In9h zp!iD@PgpUZs6Ahh-k_GIe6q0}sme>;ZWKq1$9hPpC}SN4!NSw+=nL@VRl7a{t7C_CZLiN7HOnDu8$3Cub(nDB>-HOzPll%3bgRl&Cn zmZc(tCsY)hF!ntlx*y_wdL&##L<2r<5g}cBc zT)$h-GCiKX1@Rd{0!Fd8@B4cBB40Nz7&%HGdv;X?g}Q<)Gh<0Un!tn9@_SXgK$b^2$39{jO;347{uRSy=HPmM(nvu zV*_Ht8f|6-LDPBOhwuV*DgEydNnu3iK|wJ5wYn;IleJnPyfYO|^s;g@78z5Sqk4dnd z0%+p=#PZh|bsm09cx=dB6b(z3vX(z8`0nptvZ{nCxU*+9sn`!OYs=$JvgghY*kweU zK9Oq`yBAxkctWok?ur*bib!viQJW`=!2jl>_$z7wcZ;W*<bP^{^r3Sud}w zaz_aT4L-0}ruWFL`ts!IZf@EKLM@duqQK@95=0z^nOHOIkxhEYBRZs9!AM8LW) zQ;u~Eyc%qJAc|B;5mw05hkGjcAn|dJZAm4Nq@W#XXWNSI#s*lT7SDI5^Q3I6(+NVzchai6e+_|G+o#`9d>N zcDV8FJcPYmR+XJ}HE7#qO(XzPFpg4Qa z<8mEj%jcIdD(tqpr~EG+MXU;iLG9~1?9vi)x64t!QqU4AEEyg!Yey}5Izfg1*kc)4 zvDmwZj|a!`JLCGws=S$L$%Pw913?yFN-x#LKMkGY-gPO_*-mrJltpfRhYLPlqlY3K zewG?cQN}Fg-alUQKR?RGQ&d96eqpf4kcARQIR+pLv-P$O(mxj2Gc2Hd65XT}PGh1e zx!Mf#jPs%F*}x3><6_N7XlzZLetuTtOy+sp;Q?)hpo%-(fpFAPngSSFaOd z8zB1?L0GP==Mp1;cl|*U)IbG=u)n@QlVlgzXSK%E^IE5iJxkEDNN!s~U><`z?QGHK zVZSUV&oAk-eNS|4@Lxy5f8qe`Aw0x+MDWJzyDDsRDtYp+v|)D$pq26Uo?ZdBs3edw z3dTH+sFusrMfbKvvzP_eLH^fe;P>Ne318!iABtXPf7i+8vqngwp=+|-snv*CM}PtZ z;WY8$tb$$;*vF}MYPPf0lngU$5CP~UkdWD8u(Gc$QjTdy>$F>EdBCfy-_yp@IEKNev%I#7iCKV*WSzzn8i{Fu-?S4C zeH5#L<`{rjC*OhB528d)fd(6oT$wVNbFpTSo(m$t< zg~z;|xYnjQbulJy&#XMJ-&5lfIXeN6l`zEiG2hljDU+rV34_!%rL%y^@2{1uLBQuV|v zc;QI+W+ymBZeP(7L8rX7k~;A+0O_ZPXm&x#1q=SZrW1lWoANEQ~t zQ6aF)3!NSTPpSo{1#mmC58JsB`6IW~Fr8|?fiv4_9I`eV6? zbf$lNvoj&O%;7K@(UpniH6*@4+)GMtU&nira2J#DiFaA0hy#X85NxUoAPpilLZ>@|sV3FQAv7oFBq38zOB0Zr9k1XHf5V_}*?WOtQ z{}oU4qac0&Us|N&v&IN@qOAK%3N6XEq(!$N*N^DTE)&;eOzo2wtq06@j7Sp08CHtU zIMz&!IbjK10z{zKb=Zi;Iiyu|DWpBaq}HsOU|Wsg2IXVMswpD?Ws4y}4i=tyqjbbY zK||6PsQkCD=v%YFgzEkHmCuZZyl1S}1@l?PZXdX;x0@;_Mg^e(Ay;UX&)!;YtftyJ z@!IBXL%b*PFrRL{TCcYIAM?py;SjXtdrB+qU0DTEgKl`ktee%*o+`Z z$og$E24v!V-c)?S5=EOLP7Ir5)&*wnYMv!oeex3kK|iy8|0I4J_}OdXSB0Znse~aY zK)~jj?zF_`wQHe6-s|0^SQ&7=c!#dJDPaW#Jbs9_>?>@Bc@JGyhm7z2XHUb*-?NKf z=Q1R_oT@m-uTo%eVM|jA9e;+{-tfE}v6FE6pl32#4bO{dnZxHFH!2dET z2{&v3pt5UZCM^EYrW?8{-q=>sb@c?pn7fd^wPVA|H0!XmG4I@2UKj`0n zwo;t-a+RLh;1jSUO_ErGwbb4>PEdH-k=AYlYq`S6fls}H)Oic<%QS$5`a`OI}#{cud|M^C{8`-SZ zM+eLQAFuhZ{Bpn=M&!%JqR!0!+7JGL1fL`$i>+>_+V%hEQ;A=IS$1Ts4hg1^!zcQr zuyud)Dd#>p2(aTG-d$4Mrky87?pFKOA*7>}2TU8=5{S+P01`O!_3W)@wbc%`>)38V zn7PmOif{N2LAQ4eAb8)7NYpg&!UMNUb7+nua^3&i;z;VF1pOyA@dfQp@r%s5#edFh zO4-aFLMi9UePFq?dltY1sLj%`1fL)wSk~?zcH@&KkV6WvF}K`XNFOTzd~dK`pqt^y z^t$NVA|3$v&nVl<{PX%0aPVila}f5mB0azdSgeI!IRa%gX`(4;yBv)P^zBv_b-vG-oIT^n<= zx0qo*=;7f+Ik*RbyGK@#gvmff#@Opt_ca~aTo`(%k;KuX;|H;$AHm_F8;V5zoNQSf$5d4*zTE3u2hN+GBf;|ws82k- z^2#HL%L-tT0xzWafY=r3KWAy&6-ayshptW|CG9FsE}!{SaiPP1UJMaOHp{>6W&`iZ zCEP(4h8>6@&q=){PdMY4h2ab1R@=;0xhrmfr*I>S)d5c5WSF|2jOf-=_%0uU{TEGVgb2t%+4&83TlY8cWq8>Hut8oPnRbE8+6M)cbs z%3cWN74j(BRmGwHT+e41`~%TAaL2jO1q&lBdjlZt;g9oLlB*aKyjdvGJsh6Ba>Cf# zQKBP`q|v5DWdKP=f3|8x!rr+@i9N0=3wv<#b|N`_g|st|IaeNeb0BQFLtEcxf(!QH zNoYDOKeIM|Vbd$v3Yr19Vi^n1B=B{9P(+=(M4?jRMSAC}@v2lo%Sb)PH&hqEaf-S1 zwmh;;o)A9tv-k&sV3VzSN;(ixc>uGb3vn`qcXEyR<_=31hg(_Ku`;-B-f?#zYFq%X z@k=8UL{1x(V%{qX=)O2A%}n+;i7Ap-Cls7>|J*a;GG3#ZHhva(pgs`(NYDwZvE!)4 z;cbQ3(8Vs^pR;-AugK8>n}($YG}$C&<7<5N4CaIda2NPs6jG56YZtV>H{mWSMbv?l z5*8*-xoI?)C2SZP@$!b~PDO)p8muA?8oWGvYK)KWN7$4ot(YjpZ?5H$A3)|vHN%QT zF?`6Tw7U!OqMCaD5EeCdF7II^r5BW%XvCF^K{eWffTYnsYL=>>BIE}oUs@#2^`4yO zt@vnqUyL3^T~)e!rrUzN2PV_(_n|HFvwUZ$iY39tTD<`@NjSYpX@a0Vd_kw~1nZdu zQtdV^cs}-8Z9K~RQTr|enx7tou=`z+p@f8epjpgt0TnW9@~80Jml28;7OWm$T)p?Q z8F7My@=VeTloT=14c|nh&VStfGoACmOV|bXuEVxz7Sa;JR6B^=oi_=86G1%!BX=x+8`O;bhPCr<577|u3}xV|MrAPMC=jts!Ty0!n-?jFbDslCeGSK*VLP#uUzb6n{uw=V`GJ-dl7>t zj_EpmFUpn!%MX@F&uqLzbwsB{R|Bh8>88OL)7RRsb@#Zn1eIXrnI(gQqw81JZXUjU z&2;zS#L*z5l7h?lLEF9E{si<^1ScK|?AiwkPXq0$8+0-@1TxPmhHoWsF^kvNi*jL; z&kWj26=kR)2g5I4`FhBveOObv3-{X1G(uiumXM*Nle;urT>!JmNZbuQeROi-bR^8K z;9eubg#`lp${he!N@MrPbEH$>nkaE##=on(n)=r}9h<8;^IvoRUg zhoe}qJk>y{rzm|pgqSL>L%EJtoZTCcna%}+W4VDPE2?I?+!-1WqBEtEP3e&Bi!vrwH5LLfW1Lpk}Z4zmn}jGZH;K=f<7R>FoPG{kq-5grDKY zRl=41;bM~}o39@B6LMjq+PAWTHv5s9Vqk3Uz9pFs&4TQ#6sDe05R)AtHTRg+qF5#Vulqv+xgg(4q+f*opx^=z=9GY?nw?bmJRRjOpK{5})*# zZFwc%cspm5{A7BoGhfYvo5bi!m}NTm4lp+(=avMx-i`_nLi(PVRK8C?9rJuOvKA)s zqDfg(4}jjfWRJ|FInSE}%CQxlYhk33v=^jZcA>sk4A6z^1NFzdsyi7z-}H72T09b* zh;6Bsco$Wgslw$1ArpH!ze^~_q5LKGY)2%B!F%jxbpwboidqEz@phOvNHL1rn(@07<; z5B829*R`tgAN>)O4xI9TLhAY211cjl;_m1iGjqU1RoUK6L{FcN>!2(|OgXSmo)(4X zvZwRLoc%a{c+}@wDVM5&LyktfJb@ZcOkSK}*%@!z%=z&uZz@yx#K9&jm9ZCuQsLUJ zM(6h&3njAhba$sP^x}*wlpPk%n77t*T6L$~c#)cW*muAb1Wu@meY%P*dI05@U|BG z=G-Ba3g#=L7NA`=khJm;*~S{vEhcH>8lF=1vWN%rE98JK^PJEFe>duT=q(=Q1FyZB zM3VZ2vniw)I+bx}?<-2XbR;CsrV3-s?&gQI>enfGnE`~O?s13XbUi}8I;iX(eq#tj zzw{MSo{TGL6rTB;h?%TZ`#XU*tb7L$*fw-wH82C=m1OMO*q&}iDKu9;aIf2n>GZJ} zCo%fCEp2ar@#Dc~-9~Hi+pP3W?3317^X*05y%57h+Mm$8vbQICL4Lkp zJkh*6f)))$1dhHB*dh*DIGaKUw`kIJ^>y$l;H%%?&@}S%agt93NS}&O6*uAk6l9xL z`I?}N!V|aUXksk-fp+38?o?+hN=x~Sxa2LOavQYP<2zMXdJKv7(l|7sh2@?DLxz4G zAB`JtqnUv1diZML&s)qaq7VfFUg1@aK{*bg(_K6^I#5S&U_4AO@0>)wMD+F^*{XXc z`SJgM z0Ety!i>}szMiQ;UVTo~UeA@C-15L8t*_XA1O>Q_E61fhjUtZ`6!4-#ncdgSv7Oi-O zPI?YmU zO(q8>@@Li149M4zOnqjN+%mcClhqpEaU!+nH{bPHCM{1;B=IWUZ%FY&&x!K0cE$@SN8cL^pC3cOmjT=mq6Jof= zqfMct5xag(`xibP#nghtGdC1?53g80uVnR239w612(4asbLqSEdv=@o4W%2=ci2?= zq#^EjP?|P(xKayB!m;59btNqQN=k8qEq8VdvEv4bI4*PGlMqj-40FuK8I!g$z`j04 z`*1xgi;-yV+orr|ryA_v4-i#Pm%usyDS$xR;0Nz59LALHFMEUY8yg z8pg=&Ln=HoF+mQNbsQ&8O=4X_k2p`q*5*&u0~4 z+0t!1ca&_JFpxfyBTe4`B{$|2XqMa*4QxJRZ>Dr@7gS4@BliTBZ`cH7YBJhn)@ZwL zkkzNdO}D@AuTf91_neuVPHBXczsr!|-nqq_mKZ_S!Yq4t!8PLTMO%CiG5Z2-T-Jg{ zufm1h4?n&GviI8#lj(|rbgfC{GWE%(iN^sMVjg@-Z}~&tizt!2YnY^4CnxP_5|JL? zZfgu~I_=vhI2m&>()SDvT7|TJRHV_rilj=^(cj1W!48Y!=&Hd{hA2Dpa`oPuszE7q z9$(fdG9pOSjnlGw!d+*~M9K^>^h(<6WDEk23NNkZ6|!cvK4;nH!$@Oi2p55oCWQF6 zUx=)sUSRx0c=}%5+k(d2wz}1<<;fpAbNUEY?_Z5RK!%YcUNR&TaZ@e8&|}lh_Qb+9 zHSn4F=DgA_@`?+LN}j1TTQPm@&5yAi+exq;+wx1@uP@-5Rq& zlN=&MtHzKq4-)3Oe`Ym2Y}C?Y#?CEB>G{$1kiEMRe)q~k<%3}N6M=_=yjR&`Ifxr6 z78Qn5XcjFbeb@N^Gun7GMZJW%suLsLtDOw5-?%vZ=t&Kd)BLn7+O$qkXzZ_N)5Mzw_R!2 zzR;!X244s-?z)MwN~@GDw4=Vo&P}&^c5HO_t%b4YRt(9(+)Innh})gI(CCU#ub z6nP=d-Ro!X=mR8715WiS3_ZFxkpA^jF6s#M7S?!mdo`a;H$!lLnADS|>~5R9rSSbh zZX=q#8^_$Z)f?qj?V47;Pkd%JjqhIrUX95`mGoE)eFvJR>GsLz;VZPgt~~pCOv^}# zxb_{)fVzO+4cdL` z@xc3mXsrs%YdGZre>_$4waSDFa(?Ax+wwbyA32WKuxYATzuVHAgP&=@B+@noc}a^)wPn{ACZUL>M9+W?Tph z(Ph)y6IJ6Ut(&SVxIdri5g?S}F)!|uG;1%U|{ z$CHZHtad(PXO;zhXG9ig$D zATv^=kkUu;?kpkBZy62)8C9CB3k=IW|So(2>2BUQM5b^n^A){3m^3$Uaf*|ysK z0ry$V(jfEU8O5PoN+eQOx-F=slLidfVPA)xo!#gZ$4zFPS%P%CiC2kzQ}5naBVju8 z>omIH{cAKIy6PigO}fA7fm+y!?i??rpa?%u&&YuU*J^@{6_(0bm%uilsp(`(cE4{f$l-GQ@| zv4av91>ddHU!~7bwL5U}<6DNrhv%C$2D7y2f*%<5G;02^NfV76>w!8A@uCDKr0?)Z z{9=O1PS+-GVs9t=fkcb(wa>@LwY+iu>vgZGi$c)Eon(#ZAV0t4F?EIX4b(PY#bqj7PpD!@->#pvF%q7k^i*rOe>>aO z9h+=9+__G0gskFNLl|ESN&saW+|uJs*7s4o?5ML(XG-V#>pt2icN?tt^8lWE65F|=aW{RXwH*m{2R!MD(w zsHJO?=K%vKs%@-#e>p4YxEEHlp^0F9L?lwVUm#8S!GpU{g;DFULtXByN+Y<)k0x0Z zSc0x9Kg@eu*Izp|e9|ntNywlsQ32HY`yMWKQ0PnTwO@WP{xSshRwchh0~;jBG6tke z;=>QpF5iPj(!vH5Ok;pU9_7l|nixrvZ~bAcC_&j$w{7x6rI%8z|8$VZiO-Us)Mwe^ zm-ktUxLK^gT8+{E+jRnjG zpc{Om_q10~0;)x+BojFR{pdw89bTsvh{`I}2+(M2GEh@K<%sFD8*vZ3Z?`VLIyvEk zFv`(S9%-&^64|yJU@{;Ty?(GyM%ZBU$nkGXl;1PB;~6*NLL+Ki{TyOyS}_5L2Erx* z`0LHSUX{DtP(9N7sWXV#ig^FrSf)IEUabTj<(F~k$A=#R|K(&j;h7Z_^#ll>J(_4b zW(#Nj9X@F_BzsdiA+3?DD|BA^8*Ry4D(^;DvS2I1R@k#E{Wg}}5BMvP`;R6o-B*oe zAJPnlK>Y^<1~2xjo##xI@5NxlNp3fqGSbYL0iI;DP)>e5&B4{#hAi}6RamfCPmqb# z8jaz-?Pj-^#n!hDk%TB64 z8{$gz^iq$yc-(5yozLu;r>1)i2G1^GGJrorQ%l1)EV0{ZZlYX0-`49KUjN)_wg)Lm z3tJ<+pa3V7?TKN~kK70p6Jem*A-h3D^|N31Lyr!V;>ew2C8tw*S!0z+TH#p#OlllG&pZzSHE>G1xd8 zyZ=z2`)j=V_M5#APGUi8f3S(;5|Q#oUzQ*guPDz-CvD$g@>az?nh*f)!2l6+(Nr+W z9qBF~d75!~yL6Wis%(hf)xe?Qh&4KhcY|-4rL@62@V!M!f5$Ao5@3; zm4J@>go~&A6EtcgPB+@7nY;vbK2wq`p4){b_YwL|7vSC}Q-hjKF&UyK0@w0*9ML3m zAe=5f?GRq^CN`%V%rU(69&*~f)idyi#P_smJAiO(Z2 z%I?aHOEqpcK3VznXh};dAxci+XYb4UnpB~>A|@|-bugEd_}3=Ba{>v;@)FAb?!c?+z8Jp?n~A~^g3S_3enlrd zh2CKmh_lWGqhzgw8V@uPnDo3fg?#X&d5>JzL(4wihhKp^B`qYwHUgktW3l6fEQf}Y zVA-yzDf|OShhJ=AX^-jE4o*4TSt>fWi$_mz7iS+kB*=WU{@4;fVk$8-7S}n?Ykm72 zw#Djt)B8=|we>Z+H`j@#ww;_*u`C+{1NNiOUw-)GXuB1JjQ(_3WMn2ED;+~VFqf-9 zG-%KK=3LcujIc8ugNge%aEb+AjM;9XwP-5d%uKKxJD3f{y$>t|6CA{eEe5ihtV9yk zXMzN94i5KQyWxab6xQhkCY?mL{Mm9rlQ+vK)wUS5$7XE|{vDAp+LvIgRna6nclZ_R zK|l291@;c_rR061Xo%9$>?%h|Lv z5f7)Q`Iy~1qi5Pj53xhBFg)ir;e-t6x2MoemIXS^{XTC3n=yK}eAYY(VU>6J6h&zp<+hYs z4>GI=t5u!M8p0(VZR!tf6?8i`dsLzno=a5u>~<@m6y`Lca?kz27U6|0Qe!CG>C`w? zQ5SUiJx9(xD>(Nu^-*z`a9IK5TJ(NPlAmKy$J`&+u61yc^{R>|lQAbb_2>cM%GbrQ zOjK*=ezkA4!|PB9!E=Joc|ErPNoCnXXvW>NXrJ9Gi1Q=*c4d zxq6bGKRCYNU3dvY|3zb@_u)Jt6GoW)v!?sbEHuC%2+dyY^6~c5Y=!PI6oiDcIQxiN2&z5hRoH%8&vvy{9=rY?zpTFG40M^xj5Kr2hvr&x4?X`ddXp`ZfnFx?q z&Rx>uJk#iFnw1xlXwkvjS#o!%s2kf24Ts_sRZxu$Gq=L|ZuXlhk@qC=v)rL!NhCAF zxre2`%lkWkL^MdQ)s>o&ZEt?q2Rz_x)*zW(ZU1UiDZsWNO0Ew&2VDyPOGHu*IV@CB z{=ml(U{O*as@19h-ib;b`~#&obb-A2b?J3jWA}b5e>vNRgI)7|q@5ZfxwjcE&IOO- zfUll!o=yX$zV{=xgy6@IAOWBnCOBrWg1YxgOrQKKn#@M2G^ooOcYk!jQ*4%^ZWI#M zNl+Npb+4RYQn+Tsp+EUPfIy^|5S=BnX$C>%`(#na?76|`#tL}Ho~ojdzLVTk$02x= z*DFgkz||CUzl><5q&G(uWkC=rpRU%xUMq|rxxF}pv>b$d_<+E$dLpW|_@8<7E` z9-YXjZK-)K`{V^HXJS$!p7YlNU5>4@+Ok~nfqcbPWPwx;q(4FJB|a=0cob-}3G;zm z7f?K64$;ol#fSQ(^)=<|qkG;GwmMBKy}0AMbYwG;#94+i>gjdq&3k4zSth$0K_yr6 z@4bPi?#v+r8vMFZ;Wa~nru!N3kT4+-Vrq`Bp1es-i%~rN@!B#5lo}&}~w_S+^%89(R_?SR0!9+HvLzj>`2>+c| zOV@@YQP3NTiNIP$@pc@s=F^gj$le8oKeN=#Pc(QJY^F5drh)z_^SN87h|y2`lH-wHqOPU~K&%y}jWX22uT#OqUm z$FCMKXeo^m{j6MW&gVhC`%7t@)PI->=2b>Yvw{i{+K1^RSzSQXIy!zdk{dInzKr=}pkghYO|YLNLvqm3Nn&EBh zP>1~JU=Vz|%qvEG=~dI01uGy z^7#_Ypkhs|zXY*&_OsY$JZU17o}{+vZ?EKP;LEOtuM`fvna$M}jtsc|-RQPzK#9TU zT((2wAgWueO%HZNDejFw$((~rFOY_BMk&|AYkrSDyaS;^s0+}W5C(RS^DpH?Ei)FuGn79fSSC=0OYjyxAiz1ST|V50-<+hOAiHh7B^b+l2+2aIQ?atmLcWK+5-o~(TMl< zAVhSqRsHfczzY0XA=R45w6KsONCG_9K?{2Wf&yW9Tq^f|Ff9b!^e`c{eVrxXd; zB?Z|E_aL$zLM$*o#~~*gVw-z(`B=w#$BJoQx`x*a;K(6L*2!27`J4Z|_rE5<84|eN znTcxccr|x2_OzRI^roF07tYjb8Sn9U!J2zeEAKMl>GihLn!5m`fdFG zm=1sam>G#kkd^Jzp8PMp*)IYva?cIoKR)Ka{3v^x4W@qNXVJ@$_x{%p{pFXe8#vnE z7S12*=&v9B#noUMoqpx`gedJfrEB~9%KdLQ z@c))5{qvAt&cL|2Y_t>%|E~}I<@f&&jp8p)HE_KGOP6=+4>HVu8OysEda1*|-1P6L{NJyVSzDQKAgOO1G03>6@L$GARvItC z&~?cB1=sW6=cz0i+zaJFt)M?ev3F6qKsk@?Q8@eCr)%@T>*!Ft0*=3bT9x6# zMJ6{rh%ekD9uk_oe!gFfpA(E(fFSX)-U90(K$n{I; z9)J1Q{X9Y#^K))0Uw^&+wIq}ZPA_TcagpMmH^g}trXx`$;gfs6JecfzL>esgQlI+Y z@_A%q?RNN}_sjU>DI9?N-#GQ}fs8{Yd(5P+$#2i~AR2xf?p~(z=glZE8pDOw*dFix z_V?PNFuwy!-$noS+n=+5A6`VW?8TU0ZvHMRE(ESG;D5Q~&%f7x27j(QpFZgF>l7To zhs&4C8O~DvF^l~Jli)&uTi7rCzdkKzB>XnChV99N-#!(de!v!P|I4=@I*Y%qnlC4i z8vvOjM~4LRM(m+pP}X9*y4jAfmM0gY4 zA!qA<`+G{nL1kj$-vr}eSz>q-{MvD>zpdw+i06dHjmw3cp0>crM${RuJ77%1hseDQ zmw`T21%aB*_KAx4BS%jpIZm}CTfcjp93}m|%^MUck0Cy=$cYdZKqx|;;INQ8-nj=? z>jsy(6Jm7$hdqY><)>!+Eb6Be`T9${iv0L1`t2HNq9v4Xo`g%nX48H+;l*#WQ8pST zsrdnSU_2(7G~K`1MmDNA+Y0w(_3e|B*1IYv%TfSQlm+7)!+JRPwxM>>qJ#u>UmcNxDnZIY=3 zQkV;1c|e@AKL~cg*W0VHZ=JxgB?XDkJ|J#tz1FXZ770GTucGnh0nHr{&QlFuNj za=Bc90)PcV?sfttX^nb_As8*iV!;nfozO`ATzTBw+ngXomksir3?w3jHciT1Ap3bB z?hF1}d@z^t8=OR`;Tgw%=HZ|gh)*y?Yn>!#i$himxj%O{1d58U-(~$i1RTf^+>J-$ zvbGK$#!rG;xuvMiY7M8JMgdF$EhDzSj+g<(L+-B-|D-EE;5{LPXqOgla(AJ7m+SZK z9=91Z7E7vv1+bV1O2!gG{}jj&WIPKc6kAYm<#bBr~(SM&g_lP50PbO3!nVj z`sziPrj&utSqao}X9<<4K%Vo;~xZ2I>gK zFd+VRXTpm4hQhaPkhaxuI~+(yvs9p5i;pn5`kif#L-(dXyZ0#ndW92Z^+Qc0HyAE_ z(ZOjE!j^C*pN2irLNfs5KyMKjt#8IvQk6tc@{g1MJPlOH#tnPY&VPhD^Q{$3NNJ!R zu7M)St*jK}fX=tJFj~s_8<{zj)dg@Ywx=Mscz$bB91+d zvNDdD?2&zpLqwv|P$V;zm4stt9wW1oaA$N#jOrj2j(N-8aUxqr8PDfi-97dE z0nbnO>($Ha#P@v1bzR@j^%?K?`?E0=&f18H+HKDkjDw0>f8!1=Yc}y-Hw`_?0e`R` z%uYhPZ4vUqDVL|kjff@`LM!iOck*4IZAOTOh+!bK*n-Cr9Ct1N~&3t-wVD$Yj!M8_vw3IjFrpr>~!5aUKMRvroAd|I^5B zwA*gKQn1dq)PT?%eGpr{gtcnk2%~3r3LS%}b0XaHqR;E2ByR(L0#@T4%4L*~B!E64 zcuqaJ2GiF49^o#GgT6!Y6w?!-j>m-vJs)Ak_$Ubia9$@yDtji zG==eQ`X){B0(YR_cF5@A+Rh6~bEcQvik_x2-tglo0O7DY;C#KwLw6?3m1|KKUBeGR z&cM5umUyNVY0wR+&z{P4qE@Zk4Bbj_n$zuqMIkqkZc&M|u}-Zlo66ap=}9)ebMxCf zuY4eSca=EegX!%knV|@Fl`Y_rN!=59Hu_mv=lrl!;<2L(upy>_a%=s7RAt2;uK9c* zNp)ICbA^TkG#3ZKla)+2c`KIgY8G>=0$565u-A9r_v9Z{S1A2U-Kq(RSZfuy5}j`tDAI@(PEnOoI$>(ihjpBVM3v-xF94 z5c9mVAmZV}Mzz>l%X4qr>;n+Llv>Vsqp(}^V#l7EyEGc4(YO_18@1}UVIh|Q0sO`( z@je2AxH3D?|4>2A5#A`%Vx2t5K2-3l-+dboyr?=RLG8ZVP}UT-1dYIh;!DsS^)7SV zYksD@yUe?MS%(t$yuSM2&^X_;NJ2duV|wY0too-)E#V0P&PBe6!b!!ovJaraRoW1l zlbx}P#U4XV?NOG71Ht{(sIQTW6vP?!m?nZ zB*Jmk;y^$5yfK||S9#r+%l*VJ;8OU=uYQWl&0+yogIH{t@!rb)^A#gwTS}=BRN}tb$+*n3!ARJ3@2bLVd*p zu?J&E<90sjvrWVi1h%m8s4(tfTCqTYPb{Cg=DdAYk5<>UsMS!MlUMOLr#Wq9YTq{` zw>A#8xg_e=`|SEz<(!R>s9oO8j(CL#{`rp_{&PLVLhL&>6SgihjZgQBLM4Y4U?m4! zG?OfCez^%Gz6J&!Far;I`6~=)hSU4ZnObQ=<5akPU>`9uFt|h6(A)FTm*B+UN8z`x z>J6k~n!V8TJ}u69{)UmjH9Yo4)&xC9#h7!kK{RVIABH1lt*Lx*b!A)PzD&J|wAlqw zdTjZa+{1)@o*w=C#AU>mL~k7VGhucs6I=r(Q^uYCb4#|zoX#U0-Kv0yo)rn$5GVNs)7*XJfp+In_TMjezM|-NR~B z&9R-??!9W+{P6C`zq3bQHAQerEUh{@TGfYNL^!OjfvzG8wcfI9C4?6F&t4g)A5m}F z5G`Vlya9<7#C-IK=sK&|Wy))5`DNMT&$o2X#I1Fvrx>av3RGT3q!N5{PkA-%bK?u~ z<;+M&nkP(iOT^t$8Zh5fxUV!Ae>Mt9B;Kz1mZ?7W5V&}GWg`sXt6^)i-j!GA+ar#g zeNRFiAAhMc|Il-QtGYCPF%mpm-=`_p7&p!_t_=#NA`J57esXys&q&H*+4=T|@#S#F zqtpVhg37+KO}YZ?R+-TAvFgz3UfQ^pPw|)&?EcBVC54yivA3GN5OqmDjf7&UxBdQC#$}`Y4I><}(Nf zeD2;r&&;)WVwmY?qe}hKU$c3ViqC-Tf=P|nUro3~fd!Gv!x3*hJf>gvRuyHWsm7K( zE(DbVKGN5(Dj=ySb?b$4z@nyY0;9%Nlgn6NH9_^Cqe>DP~M@bAqzkFB^=sE@@e(m$QC7n$eql zr%3v~`*3Z`t!a(B>x(U(>ed27HBOBwL>3AD6`O~cIcd2Sm}b(a0qSjtX0b+L z=v7nVMBYtb>RB%>k977-L_x>Bog6<>>Bn}B;_5r=mhuT|`z2oV<>lK(HG0_Ta+OLz zc_tNMLGufo;t(c#Q$!eCAuJgO{%gU=0a1s1y;Y^k+e+84)f1c_wxM;bEdhe9(O&*+ zrfT|e!U02++=PPZgevvH?@XR5y?_wEfwLtye6Y&IO+UY>apn=OMEXoA$CE6sX&lkGEJy7m$!&|aJo;ylztqI{%U16Q@jMR;$izzJ2 zcXb!UbXx}?5(-GCaL>}_NPPxpwRoz}Bgg~)w2nuhlPUl5fRQK8 zTr$Y3`cn9TH%jA`<0F&F9-H=4mNNl-zVShS5d^X#{{-LMtwciqA*;BS(I}HkIl8ZG zm)x`w*>oNahQMBkI~yT5n~?Y@wq(H4m=E1El{oEufh2ZGx35n#C;f5^GtVe_e_sHu z5da(~A>^_!hFdu5%#dOAS>#HzM;f7B;B zv_p@gu$$lB&#IZf(9qI8GodldGJsj*rDn6X5|%AGo`*BH){IEFSb2;GKf-61&#i`w zsw$kT`rO828d}N`X(TI9NQ^oJ;ALi&E}^Ev?tiX5XclI+jRd(zu5eaM!Y4IRbHz^AWees+&n%PHvv#)AC>rUC68zG|x-9s-ClxuzR z!+obcOOCv&q;(ZSzt_M?CK6^za6HxITk;thdR^-{5HE@YTDp;mQ93vD z`w@+2Hk_-5GPaMhj9`rp#8i`m5JzO(et8Qfmz;6pwGak8X0`d@1LkK%9gZp6sN|tT z+A(uw?Btmx82yLjTV-Z)^?$Kp{_Z%DHeV4M+XJ>BNt0vhFGRTR4iH zY$~wxyZpyThS^xQzlx1y`KmUGC7llA3yH7_@ZnbpE>ks4CjJiyUoUTo0M9&P zeLQgZ4KvI0ou;cv9-H~8xJgTL0AHtE-|ZGON9&&@qg#oHbK?rm4N9{UQ1QyMtR&yn z0d+#Fkbt&evt90UrhlH#uk}ZprpTRDcyMdN$NRwPaVu3)wR}%!L_X14oAWFL9Gdma zVF`Dp-ex2jXYrhxwn}eVg#PK83QrOSKTW*%%nqWb!o97ukCRtZ-Ql&uX_1+*^Ly4{ zPkP8!aF94gyL~q{C`#F`i%ngTCo`An>m`J`{TD>L{cgPDyNF(2p?6aW|LkT+hbT!}~h`h!AHW|Ne^EGEi;AWyhRt z;~70EjM%l!fxzY^QDRQlV7gqNV*qV=N#LTFX@U3sxXb-k@5zoF{)(-wze?smo=yhw z2=L$$R6<03ZnkR0Op3=d3_r|`mPJqXW$jU!4k6G?IE~DbjZ7uKlyCUgKAo^o_0>w+ z5Dr-z7tdR+NiExm+k5Uxfd>!(GJFTpR3x8k(z2f$dwavZU6#l0ptBHK9o_ztTV%;f zZSbhV$4{0&{6iA5!iTrKafj@pc|DUFzU>n|mXbAr$ozq>Y~d+L>YYoqlFRbZ0|e0U zAZleS&R0r6Bu$$0F)Cv}N+!<5)2n2tE^Or}L}CuHNi7bhk0LSw0ms{8md0c*HGci_ zc40djdUZ-;7Ii)ApkbE^ZS~v}H1ABBbEX2su8%dPr=A1(U@M>k@)pb^j^g@P3<7IM z0DZVVzOd7Mq3LXl#^sHVFG_D^yaQu!FZ=2b|K#V-!>1vctT-oqj6$zz-X}Tpaw`ZC z=;=Nr-tkdiU3sLt{Q`+j8O?y!*?>M>v5rzFC@_K|$+6vg3@AvJsp*Ij|4zyHHLB=R zvx#s)UN=N~KCXjV1~BABQrR}6(o7Ee0w1pJ@;O>Is^7df#Y}?1#$uCDNs#p9x0n4P zQULCf{#{O7Ga{P_60S-CRFzpQV3*>!!ksUI_N;|pYFP%`-Npc%(N_eRzasPet*{D# z*DI+-HW#vxZs-=-SB%uN#~Q1$Kfl@9TwU~Ji)@O>TUgAWi5jujY_F93T#qA$tl>dT zRkecEL!9=e?Ul#Q{)4Sl=G-S-zYAP%uSs>5SRxl|a%_&ZU$EJmD1iI4HWFPCbxgrk z%5lM3vbd!XeDBj9`83B=x><+!lfa@lcY;oE2d3!h+{4!meXN6#6T57*xNY`*u=YAM znQDDvVHoGvfgxwp*+o6ihoq)K&_NZ2RU|e0fV2in|51?FnpXHRxvcdEk3}YAK2$yU zY${HaK=V)kUcqAvZG(b&Qm8WouJfPh6NkkA_O5HcF|2j0D=G(td7s|;D7>CwxGHc_ zQ{37g+0Sw;)XCG6$wlC1k4%mWnoCe;>>~Vq$&ik%V6ukoqOnsQ*vpUUK0O!m-ga=< z(aK5BWO>Nau7~Yl{7iW7&MECz<_vQEM-3Y16A|ta6h3u7a_os{X@6>^x(JYp?pVa2 zHbt|GDB8%|-@Qd>J}piaCk%8!opsCfw>&4JzACArcQIg#22JMgG)(Z^ z7g!sUJI=2zcOoI++u+TlEr=%0Wn=$b{}++<&(?I3k|u8#hFPM|#_CVC=Ok{t62_)8 zr4)%i<0Mr?2+ouaGL2iAIw$n#7-^X)=WHq8hp8N1H(%Qq(-cF;2MSbjg(VQ~~ z*pxxO#~5FB4egIo$K|F0fZAvQU0s_s)H8iwKY^?nmmhHD{o1&O-nw!pPcpMh<28NM z@l9(cpPbE~8Eyu|Wr04fdCb>pcObIAk#hRE#)?}R$gZ7}@TKY!Wo%jhafFliWXC&1 z&PGK&udr86>P`yXfEv;TQpN?*MZ$|ZE7IN{svB+0a>$is#4;_+Rb7=Z&vZrUa5)oV zu=BNOiwZyk6LG`o#*a)kJtF+>qwHe$xpv8S7WQR*pk=+-V~C$;D{%uXzfzHRQrG&d z@`YG{BQq@%-4cdB8RB1wrC2LY2Z|zc=04iesv#1b{_BIr3NOn834D{u=9@?Vxv^;f zv8;4UMt$uWA=6Q%N94OF{cmFZ0hH||s9V*Dbj`aBRWxcw39Oc+{PINO(q@@8?yDeR z7x%4;-k~pvl`3p?*(G4Hv4ws=(`5yCAhn2YFT1^nTPP)WTZjhbs!zh!aqoD)4H!5p}765PwK@ z2$ixL&2?F3_gzJ#SO}qItpK0+QKL!?d0Gp$WoD7M493xO=;!P2a@w_kBZ2wUQ|ywm z@WyQ6G<_ZQbZx z|9pgWf2ld9evv7+VPyBo-lZVAn`EGWoJTD<^4YE=8 z4Pbs|s=v91)0w8v->5Uv5yA4rpPKZl&uBMqD0YXiwlX+%r%iPICR79r+BTS9W;dEU zV~r{Fi(+o=-F4?poUhm_jHJ9Px;iS*K3qprU#lAoMJYGBYgu%gC=&wV#_vA)BHxDtwLbUP+Y9qWKwDWk-5SSZhUyWykwS<&1PmxhuCOMiJ z#bj7X!8RNV!#7@6aAG8s$8YMN_go~(?;)%M+|5B$O27cS#WJPhJJWYeW55(Y&uNc{ zy?l`!vZS>cQ<6&zt`5{_0+97BKWH$|EnZze$59O`^NmOargW`Fh@}#LhTEnz`zFA; zYrVVxfSiWp#`m_3r$gCbr(b%#`U;PL`@jKT8TU_7T3#2V*+tIeYU^A@x@I9lzV2Da zPn@cYiJe&=lLBD?k8AQ3O3q}{jLbp4utanzsPc2^Urks0s2KV0p3)(P>?zp}uM`IZ zrUb3tLOK+e`{~b#+!KemrF}6w{n?H~`K7Xm1#R{&N-{$$!ZN)@zY%YE`UMbCKQ6p} z8|D6}ncr7dM<(mVT7AJx;yI?3Rw_eFO;Td*sq~gY(1?FX>Zs)(ZKKE!Z7RF>b%E|R z(yHYpv_ypF(L~>8Q|SWn%CQI|5EGBKA9;vKapyC1`>=TDdRKL7TUd zu)a(^YoZlf#xj|>Z6u&d0P2#F@>e2Gi!osBP$zq+qInyyh93@1jp{1wIC^mZj0KQc z@KOMd{IGnxWyW?JgFWMyb8Imlcw*}=Q^groQi=yY2fgEXU>z@W7Ue$NP5u_aCuv-&^a za=M#HY-W2Rt|i==ML=cq{Nb%*_$YbFy~FQrx&r96=pIhZ*A!-f=igdoQcR&pr4LfP8n&5^wsV`^Rgl&sc#jCGj z%*s?24nv%?l7?XP$7VwR&m8A_%;CMnC@FuaPBV?{7|v7dMoO49r5L7&H?>pv4M+b{ zwwPoVx0Sp^wErG1R-W;r^zptaf)~ltMo-j0;wp}@VZ3ehb{&4GY+>&a3ela=DN<`A z?l7Z$R`d}kk*?bU{ffv@#DpUVQf@uJ3W z_(TYN8NR2GRCOASB3)H>^V*25yMTZ!v(#&A?d~}CDEyNao<$Ipd-z;Qn0=O&AY{mZ zw&*W4RcIT;k1~5f{O}hvGU{cKcyG@sJF;UyPU=a1n9Sqd6U$TFrI{}MBnNupG5|>l z?@BCUuB#hEe2(-{QVHwz;0^s5ICg0G%Fb{JqcfOgIb(Y3Z(hfC z+zCh>^BkM_2^iIF;s6~uzN5qHNi^t-W6cie$fYG0QwdG4j|viJS8ZM?or4un;MIo& zu{zSvwLuoRmp+l}4G6nid_%eQfZ+MmBmJwXqqHJdcrhKgZ?F?1JwTX<7QmO9?r_KX zM+|7YOW1&MY(EjLq-BM;);F1~ZOIw~_1yOor{WSdJsv94c3g_Lgw`@`JqI@n8jl-D zXh!CAfFnCPTQXIY!fjo%JD`to(1nJTXVr77r_{q^rS%O-Bqmgyn>dZ21uD#PwR*6| zVGPZ?7X|DHs}CC0__PD@7Y{RLO6Pm+ilN1yVn3a*#sfXOrk{6ZPa$`5E%8W-1(2V2Uvv5Dw4VKNovP;@HW?>e)Ms~ zDe^Jg1BV(dI8PUVvyrJs&hQ9IkSs~KH4Hq0PMy5#-dpN>#_;@qx*NjRkya|@g?nAd zh+b0+4@wbe5yeEk7%YZ=7mDYK%FwBsj^N_8m4hekUGoh!FUF)v`XY2r0jKm1?DwjD z2Nl&dGR^pYVd-nJ^O?uarLqiy)7O=7?}0^vI-Y{MQHH+R_C5YTMi)95ywzWg#+S}* zin;RI=`Q-coC~_=#q~OR&Z3bAty*s;Yakh}Vb>vcXMP|<`|67gBg&qE!e|=&Uar$o zt`kpcH}EGsS>^L|ECRAiCIuO=(dWo>WReKoktiS5>6_Lx0=`;{1{iC5lnhE13Np8y za~@6zkF7oAH{^>e*=NP`>$7iS9$p!YlGH{fe_%v8@+|*yeTc1oiUf7S3@T{H&xNw z)rl?lQ-BmPM{%a&BIY+I?=U~ux?}ZM^DIspQ`kv?^7p#pam#YsJOu115>3Hl3nmm# z(V|}fGs`a2Eq%&seMF!9VAGdy3H0##!u$cNE;Tvn4;FkYCveTAg(Zr{@42qY?J%p} z!?MRu@vC+_qWLb%etJ&F%iGg+1>5rLEX<@q!6APk|#=Gt@>i-)EP!Rzxz zG0-^{gpV&;NqHygx(!&t*=GHY#Q%;mo0LVau`(c2w9g#Gjy|Uzc1OGevk!1~3Dft# z0A`2j=z$}KNL6JNw$!&k2}bGC(5tZV%K3&?|7nigCS;6!W~<1*%S{|UgN)IKjXd

a0qvTKX^Zy~QFxzp>n5c!c?|=SzoZ5a}iu%W+)k(j;Mhjnyl=7$k_iM-> zY(#~YDkNFKG=SxZauVZ@QdZn|oUw*${DkBQ(b(EU^*PUuH zpsU;IF+euS|JK3(%NVRu+n%eM5s10u3oWL%8SghU$ivx&{SSGoj(q?C literal 0 HcmV?d00001 From 1d771959509d09c34195fa19d9ae8446ae3a8726 Mon Sep 17 00:00:00 2001 From: Florian Jetter Date: Fri, 16 Aug 2024 23:00:50 +0200 Subject: [PATCH 70/72] Ensure pickle does not change tokens (#11320) --- .../environment-mindeps-array.yaml | 2 +- .../environment-mindeps-dataframe.yaml | 2 +- .../environment-mindeps-distributed.yaml | 2 +- .../environment-mindeps-non-optional.yaml | 2 +- .../environment-mindeps-optional.yaml | 2 +- dask/_compatibility.py | 4 + dask/base.py | 118 +++++++++++++----- dask/dataframe/io/tests/test_io.py | 41 +++--- dask/dataframe/tests/test_dataframe.py | 6 + dask/dataframe/tests/test_groupby.py | 22 ++++ dask/tests/test_tokenize.py | 31 ++++- pyproject.toml | 2 +- 12 files changed, 170 insertions(+), 64 deletions(-) diff --git a/continuous_integration/environment-mindeps-array.yaml b/continuous_integration/environment-mindeps-array.yaml index 28239098764..0a2ca790dab 100644 --- a/continuous_integration/environment-mindeps-array.yaml +++ b/continuous_integration/environment-mindeps-array.yaml @@ -7,7 +7,7 @@ dependencies: - python=3.10 - pyyaml=5.4.1 - click=8.1 - - cloudpickle=2.0.0 + - cloudpickle=3.0.0 - partd=1.4.0 - fsspec=2021.09.0 - importlib-metadata=4.13.0 diff --git a/continuous_integration/environment-mindeps-dataframe.yaml b/continuous_integration/environment-mindeps-dataframe.yaml index ddf40e26452..18130f85421 100644 --- a/continuous_integration/environment-mindeps-dataframe.yaml +++ b/continuous_integration/environment-mindeps-dataframe.yaml @@ -7,7 +7,7 @@ dependencies: - python=3.10 - pyyaml=5.4.1 - click=8.1 - - cloudpickle=2.0.0 + - cloudpickle=3.0.0 - partd=1.4.0 - fsspec=2021.09.0 - importlib-metadata=4.13.0 diff --git a/continuous_integration/environment-mindeps-distributed.yaml b/continuous_integration/environment-mindeps-distributed.yaml index c6b287ec4e8..f94ef731747 100644 --- a/continuous_integration/environment-mindeps-distributed.yaml +++ b/continuous_integration/environment-mindeps-distributed.yaml @@ -7,7 +7,7 @@ dependencies: - python=3.10 - pyyaml=5.4.1 - click=8.1 - - cloudpickle=2.0.0 + - cloudpickle=3.0.0 - partd=1.4.0 - fsspec=2021.09.0 - importlib-metadata=4.13.0 diff --git a/continuous_integration/environment-mindeps-non-optional.yaml b/continuous_integration/environment-mindeps-non-optional.yaml index 1f653c6a0b4..806f45bdf4e 100644 --- a/continuous_integration/environment-mindeps-non-optional.yaml +++ b/continuous_integration/environment-mindeps-non-optional.yaml @@ -7,7 +7,7 @@ dependencies: - python=3.10 - pyyaml=5.4.1 - click=8.1 - - cloudpickle=2.0.0 + - cloudpickle=3.0.0 - partd=1.4.0 - fsspec=2021.09.0 - importlib-metadata=4.13.0 diff --git a/continuous_integration/environment-mindeps-optional.yaml b/continuous_integration/environment-mindeps-optional.yaml index 5fa8cd11f2c..ef5c262a616 100644 --- a/continuous_integration/environment-mindeps-optional.yaml +++ b/continuous_integration/environment-mindeps-optional.yaml @@ -7,7 +7,7 @@ dependencies: - python=3.10 - pyyaml=5.4.1 - click=8.1 - - cloudpickle=2.0.0 + - cloudpickle=3.0.0 - partd=1.4.0 - fsspec=2021.09.0 - importlib-metadata=4.13.0 diff --git a/dask/_compatibility.py b/dask/_compatibility.py index 9267e7add21..87b94c1a6a1 100644 --- a/dask/_compatibility.py +++ b/dask/_compatibility.py @@ -16,6 +16,10 @@ EMSCRIPTEN = sys.platform == "emscripten" +LINUX = sys.platform == "linux" +MACOS = sys.platform == "darwin" +WINDOWS = sys.platform == "win32" + def entry_points(group=None): warnings.warn( diff --git a/dask/base.py b/dask/base.py index ecd8a98a5d1..25b8b0c2055 100644 --- a/dask/base.py +++ b/dask/base.py @@ -1,11 +1,11 @@ from __future__ import annotations +import copyreg import dataclasses import datetime import decimal import hashlib import inspect -import os import pathlib import pickle import types @@ -1206,6 +1206,9 @@ def normalize_object(o): if type(o) is object: return _normalize_pure_object(o) + if isinstance(o, type): + copyreg._slotnames(o) + if dataclasses.is_dataclass(o) and not isinstance(o, type): return _normalize_dataclass(o) @@ -1236,19 +1239,28 @@ def _normalize_pure_object(o: object) -> tuple[str, int]: def _normalize_pickle(o: object) -> tuple: buffers: list[pickle.PickleBuffer] = [] - pik: bytes | None - try: - pik = pickle.dumps(o, protocol=5, buffer_callback=buffers.append) - if b"__main__" in pik: - pik = None - except Exception: - pik = None + pik: int | None = None + pik2: int + for mod in [pickle, cloudpickle]: + for _ in range(3): + buffers.clear() + try: + out = mod.dumps(o, protocol=5, buffer_callback=buffers.append) + mod.loads(out, buffers=buffers) + pik2 = hash_buffer_hex(out) + except Exception: + break + if pik == pik2: + break + pik = pik2 + else: + _maybe_raise_nondeterministic("Failed to tokenize deterministically") + break if pik is None: - buffers.clear() - pik = cloudpickle.dumps(o, protocol=5, buffer_callback=buffers.append) - - return hash_buffer_hex(pik), [hash_buffer_hex(buf) for buf in buffers] + _maybe_raise_nondeterministic("Failed to tokenize deterministically") + pik = int(uuid.uuid4()) + return pik, [hash_buffer_hex(buf) for buf in buffers] def _normalize_dataclass(obj): @@ -1313,6 +1325,13 @@ def normalize_dataframe(df): data = list(mgr.arrays) + [df.columns, df.index] return list(map(normalize_token, data)) + @normalize_token.register(pd.arrays.ArrowExtensionArray) + def normalize_extension_array(arr): + try: + return (type(arr), normalize_token(arr._pa_array)) + except AttributeError: + return (type(arr), normalize_token(arr._data)) + @normalize_token.register(pd.api.extensions.ExtensionArray) def normalize_extension_array(arr): import numpy as np @@ -1345,34 +1364,67 @@ def register_pyarrow(): def normalize_datatype(dt): return pickle.dumps(dt, protocol=4) + @normalize_token.register(pa.Table) + def normalize_table(dt): + return ( + "pa.Table", + normalize_token(dt.schema), + normalize_token(dt.columns), + ) + + @normalize_token.register(pa.ChunkedArray) + def normalize_chunked_array(arr): + return ( + "pa.ChunkedArray", + normalize_token(arr.type), + normalize_token(arr.chunks), + ) + + @normalize_token.register(pa.Array) + def normalize_chunked_array(arr): + return ( + "pa.Array", + normalize_token(arr.type), + normalize_token(arr.buffers()), + ) + + @normalize_token.register(pa.Buffer) + def normalize_chunked_array(buf): + return ("pa.Buffer", hash_buffer_hex(buf)) + @normalize_token.register_lazy("numpy") def register_numpy(): import numpy as np + @normalize_token.register(np.ndarray) + def normalize_array(x): + if not x.shape: + return (x.item(), x.dtype) + if x.dtype.hasobject: + try: + try: + # string fast-path + data = hash_buffer_hex( + "-".join(x.flat).encode( + encoding="utf-8", errors="surrogatepass" + ) + ) + except UnicodeDecodeError: + # bytes fast-path + data = hash_buffer_hex(b"-".join(x.flat)) + except (TypeError, UnicodeDecodeError): + return normalize_object(x) + else: + try: + data = hash_buffer_hex(x.ravel(order="K").view("i1")) + except (BufferError, AttributeError, ValueError): + data = hash_buffer_hex(x.copy().ravel(order="K").view("i1")) + return (data, x.dtype, x.shape) + @normalize_token.register(np.memmap) def normalize_mmap(mm): - if hasattr(mm, "mode") and getattr(mm, "filename", None): - if hasattr(mm.base, "ctypes"): - offset = ( - mm.ctypes._as_parameter_.value - mm.base.ctypes._as_parameter_.value - ) - else: - offset = 0 # root memmap's have mmap object as base - if hasattr( - mm, "offset" - ): # offset numpy used while opening, and not the offset to the beginning of file - offset += mm.offset - return ( - mm.filename, - os.path.getmtime(mm.filename), - mm.dtype, - mm.shape, - mm.strides, - offset, - ) - else: - return normalize_object(mm) + return hash_buffer_hex(np.ascontiguousarray(mm)) @normalize_token.register(np.ufunc) def normalize_ufunc(func): diff --git a/dask/dataframe/io/tests/test_io.py b/dask/dataframe/io/tests/test_io.py index e2dc628dbc2..85b8f818960 100644 --- a/dask/dataframe/io/tests/test_io.py +++ b/dask/dataframe/io/tests/test_io.py @@ -1034,29 +1034,30 @@ def func(t): assert_eq(ddf.compute(), expect) -@pytest.mark.xfail(DASK_EXPR_ENABLED, reason="hashing not deterministic") -def test_from_map_column_projection(): - # Test that column projection works - # as expected with from_map when - # enforce_metadata=True +class MyFunc: + projected: list[str] = [] - projected = [] + def __init__(self, columns=None): + self.columns = columns - class MyFunc: - def __init__(self, columns=None): - self.columns = columns + def project_columns(self, columns): + return MyFunc(columns) - def project_columns(self, columns): - return MyFunc(columns) + def __call__(self, t, columns=None): + cols = self.columns or columns + size = t[0] + 1 + x = t[1] + df = pd.DataFrame({"A": [x] * size, "B": [10] * size}) + if cols is None: + return df + MyFunc.projected.extend(cols) + return df[cols] - def __call__(self, t): - size = t[0] + 1 - x = t[1] - df = pd.DataFrame({"A": [x] * size, "B": [10] * size}) - if self.columns is None: - return df - projected.extend(self.columns) - return df[self.columns] + +def test_from_map_column_projection(): + # Test that column projection works + # as expected with from_map when + # enforce_metadata=True ddf = dd.from_map( MyFunc(), @@ -1072,7 +1073,7 @@ def __call__(self, t): index=[0, 0, 1, 0, 1, 2], ) assert_eq(ddf["A"], expect["A"]) - assert set(projected) == {"A"} + assert set(MyFunc.projected) == {"A"} assert_eq(ddf, expect) diff --git a/dask/dataframe/tests/test_dataframe.py b/dask/dataframe/tests/test_dataframe.py index a2cc707c2ed..e05252c62d5 100644 --- a/dask/dataframe/tests/test_dataframe.py +++ b/dask/dataframe/tests/test_dataframe.py @@ -2,6 +2,7 @@ import contextlib import decimal +import sys import warnings import weakref import xml.etree.ElementTree @@ -21,6 +22,7 @@ import dask.dataframe as dd import dask.dataframe.groupby from dask import delayed +from dask._compatibility import WINDOWS from dask.base import compute_as_if_collection from dask.blockwise import fuse_roots from dask.dataframe import _compat, methods @@ -635,6 +637,10 @@ def test_describe_for_possibly_unsorted_q(): assert_eq(r["75%"], 75.0) +@pytest.mark.skipif( + WINDOWS and sys.version_info < (3, 11), + reason="https://github.com/dask/dask/pull/11320#issuecomment-2293798597", +) def test_cumulative(): index = [f"row{i:03d}" for i in range(100)] df = pd.DataFrame(np.random.randn(100, 5), columns=list("abcde"), index=index) diff --git a/dask/dataframe/tests/test_groupby.py b/dask/dataframe/tests/test_groupby.py index a51fcf0c0ea..0da0ac57344 100644 --- a/dask/dataframe/tests/test_groupby.py +++ b/dask/dataframe/tests/test_groupby.py @@ -2,6 +2,7 @@ import contextlib import operator +import sys import warnings from datetime import datetime from functools import partial @@ -12,6 +13,7 @@ import dask import dask.dataframe as dd +from dask._compatibility import WINDOWS from dask.dataframe import _compat from dask.dataframe._compat import ( PANDAS_GE_210, @@ -1641,6 +1643,10 @@ def test_groupby_numeric_column(): assert_eq(ddf.groupby(ddf.A)[0].sum(), df.groupby(df.A)[0].sum()) +@pytest.mark.skipif( + WINDOWS and sys.version_info < (3, 11), + reason="https://github.com/dask/dask/pull/11320#issuecomment-2293798597", +) @pytest.mark.parametrize("sel", ["a", "c", "d", ["a", "b"], ["c", "d"]]) @pytest.mark.parametrize("key", ["a", ["a", "b"]]) @pytest.mark.parametrize("func", ["cumsum", "cumprod", "cumcount"]) @@ -1667,6 +1673,10 @@ def test_series_groupby_multi_character_column_name(): assert_eq(df.groupby("aa").aa.cumsum(), ddf.groupby("aa").aa.cumsum()) +@pytest.mark.skipif( + WINDOWS and sys.version_info < (3, 11), + reason="https://github.com/dask/dask/pull/11320#issuecomment-2293798597", +) @pytest.mark.skipif(DASK_EXPR_ENABLED, reason="axis doesn't exist in dask-expr") @pytest.mark.parametrize("func", ["cumsum", "cumprod"]) def test_cumulative_axis(func): @@ -1769,6 +1779,10 @@ def test_groupby_string_label(): tm.assert_frame_equal(result, expected) +@pytest.mark.skipif( + WINDOWS and sys.version_info < (3, 11), + reason="https://github.com/dask/dask/pull/11320#issuecomment-2293798597", +) @pytest.mark.parametrize("op", ["cumsum", "cumprod"]) def test_groupby_dataframe_cum_caching(op): """Test caching behavior of cumulative operations on grouped dataframes. @@ -1850,6 +1864,10 @@ def test_groupby_agg_grouper_multiple(slice_): assert_eq(result, expected) +@pytest.mark.skipif( + WINDOWS and sys.version_info < (3, 11), + reason="https://github.com/dask/dask/pull/11320#issuecomment-2293798597", +) @pytest.mark.parametrize( "agg_func", [ @@ -3586,6 +3604,10 @@ def test_groupby_numeric_only_not_implemented(func, numeric_only): getattr(ddf.groupby("A"), func)(**kwargs) +@pytest.mark.skipif( + WINDOWS and sys.version_info < (3, 11), + reason="https://github.com/dask/dask/pull/11320#issuecomment-2293798597", +) @pytest.mark.parametrize( "func", [ diff --git a/dask/tests/test_tokenize.py b/dask/tests/test_tokenize.py index fc422d99224..f93e8626ef1 100644 --- a/dask/tests/test_tokenize.py +++ b/dask/tests/test_tokenize.py @@ -19,7 +19,7 @@ import dask from dask.base import TokenizationError, normalize_token, tokenize -from dask.core import literal +from dask.core import flatten, literal from dask.utils import tmpfile from dask.utils_test import import_or_none @@ -57,7 +57,7 @@ def check_tokenize(*args, **kwargs): # Test idempotency (the same object tokenizes to the same value) after = tokenize(*args, **kwargs) - assert before == after + assert before == after, (args, kwargs) # Test same-interpreter determinism (two identical objects tokenize to the # same value as long as you do it on the same interpreter) We are not @@ -71,8 +71,10 @@ def check_tokenize(*args, **kwargs): args3, kwargs3 = cloudpickle.loads(cloudpickle.dumps((args3, kwargs3))) tok2 = tokenize(*args2, **kwargs2) + assert tok2 == before, (args, kwargs) + tok3 = tokenize(*args3, **kwargs3) - assert tok2 == tok3 + assert tok2 == tok3, (args, kwargs) # Skip: different interpreter determinism @@ -138,7 +140,8 @@ def test_tokenize_numpy_array_supports_uneven_sizes(): @pytest.mark.skipif("not np") def test_tokenize_discontiguous_numpy_array(): - check_tokenize(np.random.random(8)[::2]) + arr = np.random.random(8) + assert check_tokenize(arr[::2]) != check_tokenize(arr[::3]) @pytest.mark.skipif("not np") @@ -229,7 +232,7 @@ def test_tokenize_numpy_memmap(): z = check_tokenize(np.load(fn, mmap_mode="r")) assert check_tokenize(x1) == check_tokenize(x2) - assert y != z + assert y == z with tmpfile(".npy") as fn: x = np.random.normal(size=(10, 10)) @@ -1316,6 +1319,15 @@ def test_tokenize_pyarrow_datatypes_complex(): assert check_tokenize(a) != check_tokenize(b) +@pytest.mark.skipif("not pa") +def test_pyarrow_table(): + a = pa.table({"x": [1, 2, 3], "y": ["a", "b", "c"]}) + b = pa.table({"x": [1, 2, 3], "y": ["a", "b", "c"]}) + c = pa.table({"x": [1, 2, 3], "y": ["a", "b", "d"]}) + assert check_tokenize(a) == check_tokenize(b) + assert check_tokenize(a) != check_tokenize(c) + + @pytest.mark.skipif("not np") def test_tokenize_opaque_object_with_buffers(): # pickle will extract PickleBuffer objects out of this @@ -1421,3 +1433,12 @@ def local_guvectorize(x, y, out): ] tokens = [check_tokenize(func) for func in all_funcs] assert len(tokens) == len(set(tokens)) + + +@pytest.mark.skipif("not pd") +def test_tokenize_pandas_arrow_strings(): + ser = pd.Series(["a", "b"], dtype="string[pyarrow]") + check_tokenize(ser) + tokens = normalize_token(ser) + # Maybe a little brittle but will do for now + assert any(str(tok) == "string" for tok in flatten(tokens)) diff --git a/pyproject.toml b/pyproject.toml index 8fb3dbc4e9d..9e3d841b8a8 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -30,7 +30,7 @@ dependencies = [ # NOTE: These are tested in `continuous_integration/test_imports.sh` If # you modify these, make sure to change the corresponding line there. "click >= 8.1", - "cloudpickle >= 2.0.0", + "cloudpickle >= 3.0.0", "fsspec >= 2021.09.0", "packaging >= 20.0", "partd >= 1.4.0", From b7d9bf49f682de8d2ef51f4617e3da782400c290 Mon Sep 17 00:00:00 2001 From: James Bourbeau Date: Fri, 16 Aug 2024 17:10:50 -0500 Subject: [PATCH 71/72] bump version to 2024.8.1 --- docs/source/changelog.rst | 38 ++++++++++++++++++++++++++++++++++++++ docs/source/install.rst | 2 +- pyproject.toml | 2 +- 3 files changed, 40 insertions(+), 2 deletions(-) diff --git a/docs/source/changelog.rst b/docs/source/changelog.rst index 4d05c7aa3e8..4af188f5666 100644 --- a/docs/source/changelog.rst +++ b/docs/source/changelog.rst @@ -60,6 +60,43 @@ Drop support for Python 3.9 This release drops support for Python 3.9 in accordance with NEP 29. Python 3.10 is now the required minimum version to run Dask. +See :pr:`11245` and :pr-distributed:`8793` by `Patrick Hoefler`_ for more details. + + +.. dropdown:: Additional changes + + - Ensure ``pickle`` does not change tokens (:pr:`11320`) `Florian Jetter`_ + - Add changelog entry for ``reshape`` and ordering improvements (:pr:`11324`) `Patrick Hoefler`_ + - Rename ``chunksize-tolerance`` option (:pr:`11317`) `Patrick Hoefler`_ + - Upgrade gpuCI and fix Dask Array failures with "cupy" backend (:pr:`11309`) `Richard (Rick) Zamora`_ + - Implement automatic rechunking for ``shuffle`` (:pr:`11311`) `Patrick Hoefler`_ + - Ensure we test against ``numpy`` 2 in CI (:pr:`11182`) `James Bourbeau`_ + - Revert "Test ordering on distributed scheduler (:pr:`11310`)" (:pr:`11321`) `Florian Jetter`_ + - Test ordering on distributed scheduler (:pr:`11310`) `Florian Jetter`_ + - Add tests to cover more cases of new ``reshape`` implementation (:pr:`11313`) `Patrick Hoefler`_ + - Order: Choose better target for branches with multiple leaf nodes (:pr:`11303`) `Patrick Hoefler`_ + - Order: Ensure runnable tasks are certainly runnable (:pr:`11305`) `Florian Jetter`_ + - Fix upstream ``numpy`` build (:pr:`11304`) `Patrick Hoefler`_ + - Make ``shuffle`` a no-op if possible (:pr:`11291`) `Patrick Hoefler`_ + - Keep ``chunksize`` consistent in ``reshape`` (:pr:`11273`) `Patrick Hoefler`_ + - Enable slicing with only one unknown chunk (:pr:`11301`) `Patrick Hoefler`_ + - Link to ``dask`` vs ``spark`` benchmarks on Dask docs (:pr:`11289`) `Sarah Charlotte Johnson`_ + - Fix slicing for masked arrays (:pr:`11300`) `Patrick Hoefler`_ + - Array: fix ``asarray`` for array input with ``dtype`` (:pr:`11288`) `Lucas Colley`_ + - Add ``numpy`` constants to array api (:pr:`11287`) `Lucas Colley`_ + - Ignore typing of return value (:pr:`11286`) `Patrick Hoefler`_ + - Remove automatic resizing in reshape (:pr:`11269`) `Patrick Hoefler`_ + - API: expose ``np`` dtypes in ``dask.array`` namespace (:pr:`11178`) `Lucas Colley`_ + + - Reduce frequency of unmanaged memory use warning (:pr-distributed:`8834`) `Patrick Hoefler`_ + - Update gpuCI ``RAPIDS_VER`` to ``24.10`` (:pr-distributed:`8786`) + - Avoid ``RuntimeError: dictionary changed size during iteration`` in ``Server._shift_counters()`` (:pr-distributed:`8828`) `Hendrik Makait`_ + - Improve concurrent close for scheduler (:pr-distributed:`8829`) `Hendrik Makait`_ + - MINOR: Extract truncation logic out of partial concatenation in P2P rechunking (:pr-distributed:`8826`) `Hendrik Makait`_ + - avoid excessive attribute access overhead for ``remove_from_task_prefix_count`` (:pr-distributed:`8821`) `Florian Jetter`_ + - Avoid key validation if validation is disabled (:pr-distributed:`8822`) `Florian Jetter`_ + - Log ``worker_client`` event (:pr-distributed:`8819`) `James Bourbeau`_ + .. _v2024.8.0: 2024.8.0 @@ -8579,3 +8616,4 @@ Other .. _`Adam Williamson`: https://github.com/AdamWill .. _`Jonas Dedden`: https://github.com/jonded94 .. _`Bernhard Raml`: https://github.com/SwamyDev +.. _`Lucas Colley`: https://github.com/lucascolley diff --git a/docs/source/install.rst b/docs/source/install.rst index c759348afe9..f5ef8dd1ed0 100644 --- a/docs/source/install.rst +++ b/docs/source/install.rst @@ -124,7 +124,7 @@ These optional dependencies and their minimum supported versions are listed belo +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ | `graphviz`_ | ``>=0.8.4`` | Graph visualization using the graphviz engine | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ -| `h5py`_ | ``>=3.4.0`` | Storing array data in hdf5 files | +| `h5py`_ | ``>=3.4.0`` | Storing array data in hdf5 files | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ | `ipycytoscape`_ | ``>=1.0.1`` | Graph visualization using the cytoscape engine | +------------------+-----------------+---------------------------------------------------------------------------------------------------------+ diff --git a/pyproject.toml b/pyproject.toml index 9e3d841b8a8..416993ee14b 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -54,7 +54,7 @@ dataframe = [ "pandas >= 2.0", "dask-expr >= 1.1, <1.2", # dask-expr pins the dask version ] -distributed = ["distributed == 2024.8.0"] +distributed = ["distributed == 2024.8.1"] diagnostics = [ "bokeh >= 2.4.2", "jinja2 >= 2.10.3", From 7373f4b80a6b1764fc6ab2a9726373207a92b2f4 Mon Sep 17 00:00:00 2001 From: David Stansby Date: Mon, 19 Aug 2024 14:13:55 +0100 Subject: [PATCH 72/72] Fix map_overlap with new_axis (#11128) --- dask/array/overlap.py | 21 +++++++++++++++++++++ dask/array/tests/test_overlap.py | 14 ++++++++++++++ 2 files changed, 35 insertions(+) diff --git a/dask/array/overlap.py b/dask/array/overlap.py index ff4e2de03c6..c996831a6b3 100644 --- a/dask/array/overlap.py +++ b/dask/array/overlap.py @@ -741,6 +741,27 @@ def assert_int_chunksize(xs): # note that keys are relabeled to match values in range(x.ndim) depth = {n: depth[ax] for n, ax in enumerate(kept_axes)} boundary = {n: boundary[ax] for n, ax in enumerate(kept_axes)} + + # add any new axes to depth and boundary variables + new_axis = kwargs.pop("new_axis", None) + if new_axis is not None: + if isinstance(new_axis, Number): + new_axis = [new_axis] + + # convert negative new_axis to equivalent positive value + ndim_out = max(a.ndim for a in args if isinstance(a, Array)) + new_axis = [d % ndim_out for d in new_axis] + + for axis in new_axis: + for existing_axis in list(depth.keys()): + if existing_axis >= axis: + # Shuffle existing axis forward to give room to insert new_axis + depth[existing_axis + 1] = depth[existing_axis] + boundary[existing_axis + 1] = boundary[existing_axis] + + depth[axis] = 0 + boundary[axis] = "none" + return trim_internal(x, depth, boundary) else: return x diff --git a/dask/array/tests/test_overlap.py b/dask/array/tests/test_overlap.py index 65908592320..42a5c87e85c 100644 --- a/dask/array/tests/test_overlap.py +++ b/dask/array/tests/test_overlap.py @@ -826,3 +826,17 @@ def test_sliding_window_errors(window_shape, axis): arr = da.zeros((4, 3)) with pytest.raises(ValueError): sliding_window_view(arr, window_shape, axis) + + +def test_map_overlap_new_axis(): + arr = da.arange(6, chunks=2) + assert arr.shape == (6,) + assert arr.chunks == ((2, 2, 2),) + + actual = arr.map_overlap(lambda x: np.stack([x, x + 0.5]), depth=1, new_axis=[0]) + expected = np.stack([np.arange(6), np.arange(6) + 0.5]) + + assert actual.chunks == ((1,), (2, 2, 2)) + # Shape and chunks aren't known until array is computed, + # so don't expclitly check shape or chunks in assert_eq + assert_eq(expected, actual, check_shape=False, check_chunks=False)