From 68e4bbc4222dc4a3cd0f383e0b59d2e541b1b00c Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 1 Aug 2022 17:31:41 -0700 Subject: [PATCH] [docs] Reorganize the tensor data support docs; general editing (#26952) Why are these changes needed? Editing pass over the tensor support docs for clarity: Make heavy use of tabbed guides to condense the content Rewrite examples to be more organized around creating vs reading tensors Use doc_code for testing --- doc/source/data/consuming-datasets.rst | 1 - doc/source/data/creating-datasets.rst | 34 +- doc/source/data/dataset-internals.rst | 3 +- doc/source/data/dataset-tensor-support.rst | 575 +++++------------- doc/source/data/doc_code/tensor.py | 498 +++++++++++++++ doc/source/data/key-concepts.rst | 10 +- doc/source/data/performance-tips.rst | 6 +- doc/source/data/pipelining-compute.rst | 2 +- doc/source/data/transforming-datasets.rst | 2 +- doc/source/ray-air/predictors.rst | 2 +- .../ray/air/util/tensor_extensions/pandas.py | 11 +- python/ray/data/_internal/arrow_block.py | 3 + python/ray/data/dataset.py | 2 +- .../data}/image-folder/cat/123.png | Bin .../data}/image-folder/cat/foo.jpg | Bin .../data}/image-folder/cat/not-an-image | 0 .../data}/image-folder/dog/xxx.PNG | Bin .../data}/image-folder/not-an-image | 0 ...7cc55e94e3e8d91776eec17774a_000000.parquet | Bin 0 -> 51975 bytes ...7cc55e94e3e8d91776eec17774a_000001.parquet | Bin 0 -> 52031 bytes ...7cc55e94e3e8d91776eec17774a_000002.parquet | Bin 0 -> 43004 bytes python/ray/data/tests/test_dataset_formats.py | 4 +- .../ray/data/tests/test_dataset_pipeline.py | 4 +- 23 files changed, 709 insertions(+), 448 deletions(-) create mode 100644 doc/source/data/doc_code/tensor.py rename python/ray/data/{tests => examples/data}/image-folder/cat/123.png (100%) rename python/ray/data/{tests => examples/data}/image-folder/cat/foo.jpg (100%) rename python/ray/data/{tests => examples/data}/image-folder/cat/not-an-image (100%) rename python/ray/data/{tests => examples/data}/image-folder/dog/xxx.PNG (100%) rename python/ray/data/{tests => examples/data}/image-folder/not-an-image (100%) create mode 100644 python/ray/data/examples/data/parquet_images_mini/652597cc55e94e3e8d91776eec17774a_000000.parquet create mode 100644 python/ray/data/examples/data/parquet_images_mini/652597cc55e94e3e8d91776eec17774a_000001.parquet create mode 100644 python/ray/data/examples/data/parquet_images_mini/652597cc55e94e3e8d91776eec17774a_000002.parquet diff --git a/doc/source/data/consuming-datasets.rst b/doc/source/data/consuming-datasets.rst index 0b6fb1337be07..401fda1a5ff17 100644 --- a/doc/source/data/consuming-datasets.rst +++ b/doc/source/data/consuming-datasets.rst @@ -80,7 +80,6 @@ This is a common pattern useful for loading and sharding data between distribute .. _saving_datasets: -=============== Saving Datasets =============== diff --git a/doc/source/data/creating-datasets.rst b/doc/source/data/creating-datasets.rst index 650be332a11b3..9dfc871794bd1 100644 --- a/doc/source/data/creating-datasets.rst +++ b/doc/source/data/creating-datasets.rst @@ -388,11 +388,8 @@ futures. ``Dataset`` backed by the distributed Pandas DataFrame partitions that underly the Dask DataFrame. - .. note:: - - This conversion should have near-zero overhead: it involves zero data copying and - zero data movement. Datasets simply reinterprets the existing Dask DataFrame partitions - as Ray Datasets partitions without touching the underlying data. + This conversion has near-zero overhead, since Datasets simply reinterprets existing + Dask-in-Ray partition objects as Dataset blocks. .. literalinclude:: ./doc_code/creating_datasets.py :language: python @@ -418,11 +415,8 @@ futures. Create a ``Dataset`` from a Modin DataFrame. This constructs a ``Dataset`` backed by the distributed Pandas DataFrame partitions that underly the Modin DataFrame. - .. note:: - - This conversion should have near-zero overhead: it involves zero data copying and - zero data movement. Datasets simply reinterprets the existing Modin DataFrame partitions - as Ray Datasets partitions without touching the underlying data. + This conversion has near-zero overhead, since Datasets simply reinterprets existing + Modin partition objects as Dataset blocks. .. literalinclude:: ./doc_code/creating_datasets.py :language: python @@ -434,11 +428,8 @@ futures. Create a ``Dataset`` from a Mars DataFrame. This constructs a ``Dataset`` backed by the distributed Pandas DataFrame partitions that underly the Mars DataFrame. - .. note:: - - This conversion should have near-zero overhead: it involves zero data copying and - zero data movement. Datasets simply reinterprets the existing Mars DataFrame partitions - as Ray Datasets partitions without touching the underlying data. + This conversion has near-zero overhead, since Datasets simply reinterprets existing + Mars partition objects as Dataset blocks. .. literalinclude:: ./doc_code/creating_datasets.py :language: python @@ -527,6 +518,19 @@ converts it into a Ray Dataset directly. ray_datasets["train"].take(2) # [{'text': ''}, {'text': ' = Valkyria Chronicles III = \n'}] +.. _datasets_from_images: + +------------------------------- +From Image Files (experimental) +------------------------------- + +Load image data stored as individual files using :py:class:`~ray.data.datasource.ImageFolderDatasource`: + +.. literalinclude:: ./doc_code/tensor.py + :language: python + :start-after: __create_images_begin__ + :end-before: __create_images_end__ + .. _datasets_custom_datasource: ------------------ diff --git a/doc/source/data/dataset-internals.rst b/doc/source/data/dataset-internals.rst index e6d4158e494f5..b8c86ba34bdcf 100644 --- a/doc/source/data/dataset-internals.rst +++ b/doc/source/data/dataset-internals.rst @@ -135,9 +135,10 @@ as either `Arrow Tables `__. Different ways of creating Datasets leads to a different starting internal format: + * Reading tabular files (Parquet, CSV, JSON) creates Arrow blocks initially. * Converting from Pandas, Dask, Modin, and Mars creates Pandas blocks initially. -* Reading NumPy files or converting from NumPy ndarrays creaates Arrow blocks. +* Reading NumPy files or converting from NumPy ndarrays creates Arrow blocks. However, this internal format is not exposed to the user. Datasets converts between formats as needed internally depending on the specified ``batch_format`` of transformations. diff --git a/doc/source/data/dataset-tensor-support.rst b/doc/source/data/dataset-tensor-support.rst index d2172e24caa8b..0e5fa00d21e7b 100644 --- a/doc/source/data/dataset-tensor-support.rst +++ b/doc/source/data/dataset-tensor-support.rst @@ -3,446 +3,197 @@ ML Tensor Support ================= -Tables with tensor columns -~~~~~~~~~~~~~~~~~~~~~~~~~~ +Tensor (multi-dimensional array) data is ubiquitous in ML workloads. However, popular data formats such as Pandas, Parquet, and Arrow don't natively support tensor data types. To bridge this gap, Datasets provides a unified tensor data type that can be used to represent and store tensor data: -Datasets supports tables with fixed-shape tensor columns, where each element in the column is a tensor (n-dimensional array) with the same shape. As an example, this allows you to use Pandas and Ray Datasets to read, write, and manipulate e.g., images. All conversions between Pandas, Arrow, and Parquet, and all application of aggregations/operations to the underlying image ndarrays are taken care of by Ray Datasets. +* For Pandas, Datasets will transparently convert ``List[np.ndarray]`` columns to and from the :class:`TensorDtype ` extension type. +* For Parquet, the Datasets Arrow extension :class:`ArrowTensorType ` allows Tensors to be loaded and stored in Parquet format. +* In addition, single-column Tensor datasets can be created from NumPy (.npy) files. -With our Pandas extension type, :class:`TensorDtype `, and extension array, :class:`TensorArray `, you can do familiar aggregations and arithmetic, comparison, and logical operations on a DataFrame containing a tensor column and the operations will be applied to the underlying tensors as expected. With our Arrow extension type, :class:`ArrowTensorType `, and extension array, :class:`ArrowTensorArray `, you'll be able to import that DataFrame into Ray Datasets and read/write the data from/to the Parquet format. +Datasets automatically converts between the extension types/arrays above. This means you can just think of "Tensors" as a single first-class data type in Datasets. -Automatic conversion between the Pandas and Arrow extension types/arrays keeps the details under-the-hood, so you only have to worry about casting the column to a tensor column using our Pandas extension type when first ingesting the table into a ``Dataset``, whether from storage or in-memory. All table operations downstream from that cast should work automatically. +Creating Tensor Datasets +------------------------ -Single-column tensor datasets -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +This section shows how to create single and multi-column Tensor datasets. -The most basic case is when a dataset only has a single column, which is of tensor -type. This kind of dataset can be: +.. tabbed:: Synthetic Data -* created with :func:`range_tensor() ` - or :func:`from_numpy() `, -* transformed with NumPy UDFs via - :meth:`ds.map_batches() `, -* consumed with :meth:`ds.iter_rows() ` and - :meth:`ds.iter_batches() `, and -* can be read from and written to ``.npy`` files. + Create a synthetic tensor dataset from a range of integers. -Here is an end-to-end example: - -.. code-block:: python - - # Create a synthetic pure-tensor Dataset. - ds = ray.data.range_tensor(10, shape=(3, 5)) - # -> Dataset(num_blocks=10, num_rows=10, - # schema={__value__: }) - - # Create a pure-tensor Dataset from an existing NumPy ndarray. - arr = np.arange(10 * 3 * 5).reshape((10, 3, 5)) - ds = ray.data.from_numpy(arr) - # -> Dataset(num_blocks=1, num_rows=10, - # schema={__value__: }) - - # Transform the tensors. Datasets will automatically unpack the single-column Arrow - # table into a NumPy ndarray, provide that ndarray to your UDF, and then repack it - # into a single-column Arrow table; this will be a zero-copy conversion in both - # cases. - ds = ds.map_batches(lambda arr: arr / arr.max()) - # -> Dataset(num_blocks=1, num_rows=10, - # schema={__value__: }) - - # Consume the tensor. This will yield the underlying (3, 5) ndarrays. - for arr in ds.iter_rows(): - assert isinstance(arr, np.ndarray) - assert arr.shape == (3, 5) - - # Consume the tensor in batches. - for arr in ds.iter_batches(batch_size=2): - assert isinstance(arr, np.ndarray) - assert arr.shape == (2, 3, 5) - - # Save to storage. This will write out the blocks of the tensor column as NPY files. - ds.write_numpy("/tmp/tensor_out") - - # Read back from storage. - ray.data.read_numpy("/tmp/tensor_out") - # -> Dataset(num_blocks=1, num_rows=?, - # schema={__value__: }) - -Reading existing serialized tensor columns -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -If you already have a Parquet dataset with columns containing serialized tensors, you can have these tensor columns cast to our tensor extension type at read-time by giving a simple schema for the tensor columns. Note that these tensors must have been serialized as their raw NumPy ndarray bytes in C-contiguous order (e.g. serialized via ``ndarray.tobytes()``). - -.. code-block:: python - - import ray - import numpy as np - import pandas as pd - - path = "/tmp/some_path" - - # Create a DataFrame with a list of serialized ndarrays as a column. - # Note that we do not cast it to a tensor array, so each element in the - # column is an opaque blob of bytes. - arr = np.arange(24).reshape((3, 2, 2, 2)) - df = pd.DataFrame({ - "one": [1, 2, 3], - "two": [tensor.tobytes() for tensor in arr]}) - - # Write the dataset to Parquet. The tensor column will be written as an - # array of opaque byte blobs. - ds = ray.data.from_pandas([df]) - ds.write_parquet(path) - - # Read the Parquet files into a new Dataset, with the serialized tensors - # automatically cast to our tensor column extension type. - ds = ray.data.read_parquet( - path, tensor_column_schema={"two": (np.int, (2, 2, 2))}) - - # Internally, this column is represented with our Arrow tensor extension - # type. - print(ds.schema()) - # -> one: int64 - # two: extension> - -If your serialized tensors don't fit the above constraints (e.g. they're stored in Fortran-contiguous order, or they're pickled), you can manually cast this tensor column to our tensor extension type via a read-time user-defined function. This UDF will be pushed down to Ray Datasets' IO layer and executed on each block in parallel, as it's read from storage. - -.. code-block:: python - - import pickle - import pyarrow as pa - from ray.data.extensions import TensorArray - - # Create a DataFrame with a list of pickled ndarrays as a column. - arr = np.arange(24).reshape((3, 2, 2, 2)) - df = pd.DataFrame({ - "one": [1, 2, 3], - "two": [pickle.dumps(tensor) for tensor in arr]}) - - # Write the dataset to Parquet. The tensor column will be written as an - # array of opaque byte blobs. - ds = ray.data.from_pandas([df]) - ds.write_parquet(path) - - # Manually deserialize the tensor pickle bytes and cast to our tensor - # extension type. For the sake of efficiency, we directly construct a - # TensorArray rather than .astype() casting on the mutated column with - # TensorDtype. - def cast_udf(block: pa.Table) -> pa.Table: - block = block.to_pandas() - block["two"] = TensorArray([pickle.loads(a) for a in block["two"]]) - return pa.Table.from_pandas(block) - - # Read the Parquet files into a new Dataset, applying the casting UDF - # on-the-fly within the underlying read tasks. - ds = ray.data.read_parquet(path, _block_udf=cast_udf) - - # Internally, this column is represented with our Arrow tensor extension - # type. - print(ds.schema()) - # -> one: int64 - # two: extension> + **Single-column only**: -.. note:: + .. literalinclude:: ./doc_code/tensor.py + :language: python + :start-after: __create_range_begin__ + :end-before: __create_range_end__ + +.. tabbed:: Pandas UDF + + Create tensor datasets by returning ``List[np.ndarray]`` columns from a Pandas UDF. + + **Single-column**: + + .. literalinclude:: ./doc_code/tensor.py + :language: python + :start-after: __create_pandas_begin__ + :end-before: __create_pandas_end__ + + **Multi-column**: + + .. literalinclude:: ./doc_code/tensor.py + :language: python + :start-after: __create_pandas_2_begin__ + :end-before: __create_pandas_2_end__ + +.. tabbed:: NumPy + + Create from in-memory numpy data or from previously saved NumPy (.npy) files. + + **Single-column only**: - The ``tensor_column_schema`` and ``_block_udf`` parameters are both experimental developer APIs and may break in future versions. + .. literalinclude:: ./doc_code/tensor.py + :language: python + :start-after: __create_numpy_begin__ + :end-before: __create_numpy_end__ -Working with tensor column datasets -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +.. tabbed:: Parquet -Now that the tensor column is properly typed and in a ``Dataset``, we can perform operations on the dataset as if it was a normal table: + There are two ways to construct a parquet Tensor dataset: (1) loading a previously-saved Tensor + dataset, or (2) casting non-Tensor parquet columns to Tensor type. When casting data, a tensor + schema or deserialization UDF must be provided. The following are examples for each method. -.. code-block:: python + **Previously-saved Tensor datasets**: - # Arrow and Pandas is now aware of this tensor column, so we can do the - # typical DataFrame operations on this column. - ds = ds.map_batches(lambda x: 2 * (x + 1), batch_format="pandas") - # -> Map Progress: 100%|████████████████████| 200/200 [00:00<00:00, 1123.54it/s] - print(ds) - # -> Dataset( - # num_blocks=1, num_rows=3, - # schema=) - print([row["two"] for row in ds.take(5)]) - # -> [2, 4, 6, 8, 10] + .. literalinclude:: ./doc_code/tensor.py + :language: python + :start-after: __create_parquet_1_begin__ + :end-before: __create_parquet_1_end__ -Writing and reading tensor columns -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + **Cast from data stored in C-contiguous format**: -This dataset can then be written to Parquet files. The tensor column schema will be preserved via the Pandas and Arrow extension types and associated metadata, allowing us to later read the Parquet files into a Dataset without needing to specify a column casting schema. This Pandas --> Arrow --> Parquet --> Arrow --> Pandas conversion support makes working with tensor columns extremely easy when using Ray Datasets to both write and read data. + For tensors stored as raw NumPy ndarray bytes in C-contiguous order (e.g., via ``ndarray.tobytes()``), all you need to specify is the tensor column schema. The following is an end-to-end example: -.. code-block:: python + .. literalinclude:: ./doc_code/tensor.py + :language: python + :start-after: __create_parquet_2_begin__ + :end-before: __create_parquet_2_end__ - # You can write the dataset to Parquet. - ds.write_parquet("/some/path") - # And you can read it back. - read_ds = ray.data.read_parquet("/some/path") - print(read_ds.schema()) - # -> one: int64 - # two: extension> + **Cast from data stored in custom formats**: -.. _datasets_tensor_ml_exchange: + For tensors stored in other formats (e.g., pickled), you can specify a deserializer UDF that returns TensorArray columns: -Converting to a Torch/TensorFlow Dataset -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + .. literalinclude:: ./doc_code/tensor.py + :language: python + :start-after: __create_parquet_3_begin__ + :end-before: __create_parquet_3_end__ -This dataset can also be converted to a Torch or TensorFlow dataset via the standard -:meth:`ds.to_torch() ` and -:meth:`ds.to_tf() ` APIs for ingestion into those respective ML -training frameworks. The tensor column will be automatically converted to a -Torch/TensorFlow tensor without incurring any copies. +.. tabbed:: Images (experimental) + + Load image data stored as individual files using :py:class:`~ray.data.datasource.ImageFolderDatasource`: + + **Image and label columns**: + + .. literalinclude:: ./doc_code/tensor.py + :language: python + :start-after: __create_images_begin__ + :end-before: __create_images_end__ .. note:: - When converting to a TensorFlow Dataset, you will need to give the full tensor spec - for the tensor columns, including the shape of each underlying tensor element in said - column. - - -.. tabbed:: Torch - - Convert a ``Dataset`` containing a single tensor feature column to a Torch ``IterableDataset``. - - .. code-block:: python - - import ray - import numpy as np - import pandas as pd - import torch + By convention, single-column Tensor datasets are represented with a single ``__value__`` column. + This kind of dataset will be converted automatically to/from NumPy array format in all transformation and consumption APIs. + +Transforming / Consuming Tensor Data +------------------------------------ + +Like any other Dataset, Datasets with tensor columns can be consumed / transformed in batches via the :meth:`ds.iter_batches(batch_format=\) ` and :meth:`ds.map_batches(fn, batch_format=\) ` APIs. This section shows the available batch formats and their behavior: + +.. tabbed:: "native" (default) + + **Single-column**: + + .. literalinclude:: ./doc_code/tensor.py + :language: python + :start-after: __consume_native_begin__ + :end-before: __consume_native_end__ + + **Multi-column**: + + .. literalinclude:: ./doc_code/tensor.py + :language: python + :start-after: __consume_native_2_begin__ + :end-before: __consume_native_2_end__ + +.. tabbed:: "pandas" + + **Single-column**: + + .. literalinclude:: ./doc_code/tensor.py + :language: python + :start-after: __consume_pandas_begin__ + :end-before: __consume_pandas_end__ + + **Multi-column**: + + .. literalinclude:: ./doc_code/tensor.py + :language: python + :start-after: __consume_pandas_2_begin__ + :end-before: __consume_pandas_2_end__ + +.. tabbed:: "pyarrow" + + **Single-column**: + + .. literalinclude:: ./doc_code/tensor.py + :language: python + :start-after: __consume_pyarrow_begin__ + :end-before: __consume_pyarrow_end__ + + **Multi-column**: + + .. literalinclude:: ./doc_code/tensor.py + :language: python + :start-after: __consume_pyarrow_2_begin__ + :end-before: __consume_pyarrow_2_end__ + +.. tabbed:: "numpy" + + **Single-column**: + + .. literalinclude:: ./doc_code/tensor.py + :language: python + :start-after: __consume_numpy_begin__ + :end-before: __consume_numpy_end__ + + **Multi-column**: + + .. literalinclude:: ./doc_code/tensor.py + :language: python + :start-after: __consume_numpy_2_begin__ + :end-before: __consume_numpy_2_end__ + +Saving Tensor Datasets +---------------------- + +Because Tensor datasets rely on Datasets-specific extension types, they can only be saved in formats that preserve Arrow metadata (currently only Parquet). In addition, single-column Tensor datasets can be saved in NumPy format. - df = pd.DataFrame({ - "feature": TensorArray(np.arange(4096).reshape((4, 32, 32))), - "label": [1, 2, 3, 4], - }) - ds = ray.data.from_pandas(df) +.. tabbed:: Parquet - # Convert the dataset to a Torch IterableDataset. - torch_ds = ds.to_torch( - label_column="label", - batch_size=2, - unsqueeze_label_tensor=False, - unsqueeze_feature_tensors=False, - ) + .. literalinclude:: ./doc_code/tensor.py + :language: python + :start-after: __write_1_begin_ + :end-before: __write_1_end__ - # A feature tensor and label tensor is yielded per batch. - for X, y in torch_ds: - # Train model(X, y) +.. tabbed:: NumPy -.. tabbed:: TensorFlow - - Convert a ``Dataset`` containing a single tensor feature column to a TensorFlow ``tf.data.Dataset``. - - .. code-block:: python - - import ray - import numpy as np - import pandas as pd - import tensorflow as tf - - tensor_element_shape = (32, 32) - - df = pd.DataFrame({ - "feature": TensorArray(np.arange(4096).reshape((4,) + tensor_element_shape)), - "label": [1, 2, 3, 4], - }) - ds = ray.data.from_pandas(df) - - # Convert the dataset to a TensorFlow Dataset. - tf_ds = ds.to_tf( - label_column="label", - output_signature=( - tf.TensorSpec(shape=(None, 1) + tensor_element_shape, dtype=tf.float32), - tf.TensorSpec(shape=(None,), dtype=tf.float32), - ), - batch_size=2, - ) - - # A feature tensor and label tensor is yielded per batch. - for X, y in tf_ds: - # Train model(X, y) - -If your (tensor) columns have different shapes, -these columns are incompatible and you will not be able to stack the column tensors -into a single tensor. Instead, you will need to group the columns by compatibility in -the ``feature_columns`` argument. - -E.g., if columns ``"feature_1"`` and ``"feature_2"`` are incompatible, you should give -``to_torch()`` a ``feature_columns=[["feature_1"], ["feature_2"]]`` argument in order to -instruct it to return separate tensors for ``"feature_1"`` and ``"feature_2"``. For -``to_torch()``, if isolating single columns as in the ``"feature_1"`` + ``"feature_2"`` -example, you may also want to provide ``unsqueeze_feature_tensors=False`` in order to -remove the redundant column dimension for each of the unit column tensors. - -.. tabbed:: Torch - - Convert a ``Dataset`` containing a tensor feature column and a scalar feature column - to a Torch ``IterableDataset``. - - .. code-block:: python - - import ray - import numpy as np - import pandas as pd - import torch - - df = pd.DataFrame({ - "feature_1": TensorArray(np.arange(4096).reshape((4, 32, 32))), - "feature_2": [5, 6, 7, 8], - "label": [1, 2, 3, 4], - }) - ds = ray.data.from_pandas(df) - - # Convert the dataset to a Torch IterableDataset. - torch_ds = ds.to_torch( - label_column="label", - feature_columns=[["feature_1"], ["feature_2"]], - batch_size=2, - unsqueeze_label_tensor=False, - unsqueeze_feature_tensors=False, - ) - - # Two feature tensors and one label tensor is yielded per batch. - for (feature_1, feature_2), y in torch_ds: - # Train model((feature_1, feature_2), y) - -.. tabbed:: TensorFlow - - Convert a ``Dataset`` containing a tensor feature column and a scalar feature column - to a TensorFlow ``tf.data.Dataset``. - - .. code-block:: python - - import ray - import numpy as np - import pandas as pd - import torch - - tensor_element_shape = (32, 32) - - df = pd.DataFrame({ - "feature_1": TensorArray(np.arange(4096).reshape((4,) + tensor_element_shape)), - "feature_2": [5, 6, 7, 8], - "label": [1, 2, 3, 4], - }) - ds = ray.data.from_pandas(df) - - # Convert the dataset to a TensorFlow Dataset. - tf_ds = ds.to_tf( - label_column="label", - feature_columns=[["feature_1"], ["feature_2"]], - output_signature=( - ( - tf.TensorSpec(shape=(None, 1) + tensor_element_shape, dtype=tf.float32), - tf.TensorSpec(shape=(None, 1), dtype=tf.int64), - ), - tf.TensorSpec(shape=(None,), dtype=tf.float32), - ), - batch_size=2, - ) - - # Two feature tensors and one label tensor is yielded per batch. - for (feature_1, feature_2), y in tf_ds: - # Train model((feature_1, feature_2), y) - -End-to-end workflow with our Pandas extension type -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -If working with in-memory Pandas DataFrames that you want to analyze, manipulate, store, and eventually read, the Pandas/Arrow extension types/arrays make it easy to extend this end-to-end workflow to tensor columns. - -.. code-block:: python - - from ray.data.extensions import TensorDtype - - # Create a DataFrame with a list of ndarrays as a column. - df = pd.DataFrame({ - "one": [1, 2, 3], - "two": list(np.arange(24).reshape((3, 2, 2, 2)))}) - # Note the opaque np.object dtype for this column. - print(df.dtypes) - # -> one int64 - # two object - # dtype: object - - # Cast column to our TensorDtype Pandas extension type. - df["two"] = df["two"].astype(TensorDtype()) - - # Note that the column dtype is now TensorDtype instead of - # np.object. - print(df.dtypes) - # -> one int64 - # two TensorDtype - # dtype: object - - # Pandas is now aware of this tensor column, and we can do the - # typical DataFrame operations on this column. - col = 2 * df["two"] - # The ndarrays underlying the tensor column will be manipulated, - # but the column itself will continue to be a Pandas type. - print(type(col)) - # -> pandas.core.series.Series - print(col) - # -> 0 [[[ 2 4] - # [ 6 8]] - # [[10 12] - # [14 16]]] - # 1 [[[18 20] - # [22 24]] - # [[26 28] - # [30 32]]] - # 2 [[[34 36] - # [38 40]] - # [[42 44] - # [46 48]]] - # Name: two, dtype: TensorDtype - - # Once you do an aggregation on that column that returns a single - # row's value, you get back our TensorArrayElement type. - tensor = col.mean() - print(type(tensor)) - # -> ray.data.extensions.tensor_extension.TensorArrayElement - print(tensor) - # -> array([[[18., 20.], - # [22., 24.]], - # [[26., 28.], - # [30., 32.]]]) - - # This is a light wrapper around a NumPy ndarray, and can easily - # be converted to an ndarray. - type(tensor.to_numpy()) - # -> numpy.ndarray - - # In addition to doing Pandas operations on the tensor column, - # you can now put the DataFrame directly into a Dataset. - ds = ray.data.from_pandas([df]) - # Internally, this column is represented with the corresponding - # Arrow tensor extension type. - print(ds.schema()) - # -> one: int64 - # two: extension> - - # You can write the dataset to Parquet. - ds.write_parquet("/some/path") - # And you can read it back. - read_ds = ray.data.read_parquet("/some/path") - print(read_ds.schema()) - # -> one: int64 - # two: extension> - - read_df = read_ds.to_pandas() - print(read_df.dtypes) - # -> one int64 - # two TensorDtype - # dtype: object - - # The tensor extension type is preserved along the - # Pandas --> Arrow --> Parquet --> Arrow --> Pandas - # conversion chain. - print(read_df.equals(df)) - # -> True + .. literalinclude:: ./doc_code/tensor.py + :language: python + :start-after: __write_2_begin_ + :end-before: __write_2_end__ Limitations -~~~~~~~~~~~ +----------- -This feature currently comes with a few known limitations that we are either actively working on addressing or have already implemented workarounds for. +The following are current limitations of Tensor datasets. - * All tensors in a tensor column currently must be the same shape. Please let us know if you require heterogeneous tensor shape for your tensor column! Tracking issue is `here `__. - * Automatic casting via specifying an override Arrow schema when reading Parquet is blocked by Arrow supporting custom ExtensionType casting kernels. See `issue `__. An explicit ``tensor_column_schema`` parameter has been added for :func:`read_parquet() ` as a stopgap solution. +* All tensors in a tensor column must have the same shape; see GitHub issue `#18316 `__. An error will be raised in the ragged tensor case. Automatic casting can be disabled with ``ray.data.context.DatasetContext.get_current().enable_tensor_extension_cast = False`` in the ragged tensor scenario. diff --git a/doc/source/data/doc_code/tensor.py b/doc/source/data/doc_code/tensor.py new file mode 100644 index 0000000000000..9d3ba794ce3ee --- /dev/null +++ b/doc/source/data/doc_code/tensor.py @@ -0,0 +1,498 @@ +# flake8: noqa + +from typing import Dict, Any + +# fmt: off +# __create_range_begin__ +import ray + +# Create a Dataset of tensors. +ds = ray.data.range_tensor(10000, shape=(64, 64)) +# -> Dataset(num_blocks=200, num_rows=10000, +# schema={__value__: ArrowTensorType(shape=(64, 64), dtype=int64)}) + +ds.take(2) +# -> [array([[0, 0, 0, ..., 0, 0, 0], +# [0, 0, 0, ..., 0, 0, 0], +# [0, 0, 0, ..., 0, 0, 0], +# ..., +# [0, 0, 0, ..., 0, 0, 0], +# [0, 0, 0, ..., 0, 0, 0], +# [0, 0, 0, ..., 0, 0, 0]]), +# array([[1, 1, 1, ..., 1, 1, 1], +# [1, 1, 1, ..., 1, 1, 1], +# [1, 1, 1, ..., 1, 1, 1], +# ..., +# [1, 1, 1, ..., 1, 1, 1], +# [1, 1, 1, ..., 1, 1, 1], +# [1, 1, 1, ..., 1, 1, 1]])] +# __create_range_end__ + +# __create_pandas_begin__ +import ray + +import pandas as pd +import numpy as np + +# Start with a tabular base dataset. +ds = ray.data.range_table(1000) + +# Create a single TensorArray column. +def single_col_udf(batch: pd.DataFrame) -> pd.DataFrame: + bs = len(batch) + + # Lists of ndarrays are automatically cast to TensorArray. + arr = [np.zeros((128, 128, 3)) for _ in range(bs)] + return pd.DataFrame({"__value__": arr}) + + ## Alternatively, manually construct a TensorArray from a single ndarray. + # from ray.data.extensions.tensor_extension import TensorArray + # arr = TensorArray(np.zeros((bs, 128, 128, 3), dtype=np.int64)) + # return pd.DataFrame({"__value__": arr}) + + +ds.map_batches(single_col_udf) +# -> Dataset(num_blocks=17, num_rows=1000, +# schema={__value__: TensorDtype(shape=(128, 128, 3), dtype=int64)}) +# __create_pandas_end__ + +# __create_pandas_2_begin__ +# Create multiple TensorArray columns. +def multi_col_udf(batch: pd.DataFrame) -> pd.DataFrame: + bs = len(batch) + + # Lists of ndarrays are automatically cast to TensorArray. + image = [np.zeros((128, 128, 3), dtype=np.int64) for _ in range(bs)] + embed = [np.zeros((256,), dtype=np.uint8) for _ in range(bs)] + return pd.DataFrame({"image": image, "embed": embed}) + + ## Alternatively, manually construct TensorArrays from ndarray batches. + # image = TensorArray(np.zeros((bs, 128, 128, 3), dtype=np.int64)) + # embed = TensorArray(np.zeros((bs, 256,), dtype=np.uint8)) + # return pd.DataFrame({"image": image, "embed": embed}) + + +ds.map_batches(multi_col_udf) +# -> Dataset(num_blocks=17, num_rows=1000, +# schema={image: TensorDtype(shape=(128, 128, 3), dtype=int64), +# embed: TensorDtype(shape=(256,), dtype=uint8)}) +# __create_pandas_2_end__ + +# __create_numpy_begin__ +import ray + +# From in-memory numpy data. +ray.data.from_numpy(np.zeros((1000, 128, 128, 3), dtype=np.int64)) +# -> Dataset(num_blocks=1, num_rows=1000, +# schema={__value__: ArrowTensorType(shape=(128, 128, 3), dtype=int64)}) + +# From saved numpy files. +ray.data.read_numpy("example://mnist_subset.npy") +# -> Dataset(num_blocks=1, num_rows=3, +# schema={__value__: ArrowTensorType(shape=(28, 28), dtype=uint8)}) +# __create_numpy_end__ + +# __create_parquet_1_begin__ +import ray + +# Reading previously saved Tensor data works out of the box. +ds = ray.data.read_parquet("example://parquet_images_mini") +# -> Dataset(num_blocks=3, num_rows=3, +# schema={image: ArrowTensorType(shape=(128, 128, 3), dtype=uint8), +# label: string}) + +ds.take(1) +# -> [{'image': +# array([[[ 92, 71, 57], +# [107, 87, 72], +# ..., +# [141, 161, 185], +# [139, 158, 184]], +# +# ..., +# +# [[135, 135, 109], +# [135, 135, 108], +# ..., +# [167, 150, 89], +# [165, 146, 90]]], dtype=uint8), +# 'label': 'cat', +# }] +# __create_parquet_1_end__ + +import shutil +shutil.rmtree("/tmp/some_path", ignore_errors=True) + +# __create_parquet_2_begin__ +import ray +import numpy as np +import pandas as pd + +path = "/tmp/some_path" + +# Create a DataFrame with a list of serialized ndarrays as a column. +# Note that we do not cast it to a tensor array, so each element in the +# column is an opaque blob of bytes. +arr = np.arange(24).reshape((3, 2, 2, 2)) +df = pd.DataFrame({ + "one": [1, 2, 3], + "two": [tensor.tobytes() for tensor in arr]}) + +# Write the dataset to Parquet. The tensor column will be written as an +# array of opaque byte blobs. +ds = ray.data.from_pandas([df]) +ds.write_parquet(path) + +# Read the Parquet files into a new Dataset, with the serialized tensors +# automatically cast to our tensor column extension type. +ds = ray.data.read_parquet( + path, tensor_column_schema={"two": (np.int, (2, 2, 2))}) + +# The new column is represented with as a Tensor extension type. +print(ds.schema()) +# -> one: int64 +# two: extension> +# __create_parquet_2_end__ + +ds.fully_executed() +shutil.rmtree(path) + +# __create_parquet_3_begin__ +import pickle +import pyarrow as pa +from ray.data.extensions import TensorArray + +path = "/tmp/some_path" + +# Create a DataFrame with a list of pickled ndarrays as a column. +arr = np.arange(24).reshape((3, 2, 2, 2)) +df = pd.DataFrame({ + "one": [1, 2, 3], + "two": [pickle.dumps(tensor) for tensor in arr]}) + +# Write the dataset to Parquet. The tensor column will be written as an +# array of opaque byte blobs. +ds = ray.data.from_pandas([df]) +ds.write_parquet(path) + +# Manually deserialize the tensor pickle bytes and cast to our tensor +# extension type. +def cast_udf(block: pa.Table) -> pa.Table: + block = block.to_pandas() + block["two"] = TensorArray([pickle.loads(a) for a in block["two"]]) + return pa.Table.from_pandas(block) + +# Read the Parquet files into a new Dataset, applying the casting UDF +# on-the-fly within the underlying read tasks. +ds = ray.data.read_parquet(path, _block_udf=cast_udf) + +# The new column is represented with as a Tensor extension type. +print(ds.schema()) +# -> one: int64 +# two: extension> +# __create_parquet_3_end__ +ds.fully_executed() + +# __create_images_begin__ +from ray.data.datasource import ImageFolderDatasource + +ds = ray.data.read_datasource( + ImageFolderDatasource(), root="example://image-folder", size=(128, 128)) +# -> Dataset(num_blocks=3, num_rows=3, +# schema={image: TensorDtype(shape=(128, 128, 3), dtype=uint8), +# label: object}) + +ds.take(1) +# -> [{'image': +# array([[[ 92, 71, 57], +# [107, 87, 72], +# ..., +# [141, 161, 185], +# [139, 158, 184]], +# +# ..., +# +# [[135, 135, 109], +# [135, 135, 108], +# ..., +# [167, 150, 89], +# [165, 146, 90]]], dtype=uint8), +# 'label': 'cat', +# }] +# __create_images_end__ + + +# __consume_native_begin__ +import ray + +# Read a single-column example dataset. +ds = ray.data.read_numpy("example://mnist_subset.npy") +# -> Dataset(num_blocks=1, num_rows=3, +# schema={__value__: ArrowTensorType(shape=(28, 28), dtype=uint8)}) + +def add_one(batch: np.ndarray) -> np.ndarray: + return batch + 1 + +# This processes batches in numpy.ndarray format. +ds = ds.map_batches(add_one) + +# This returns batches in numpy.ndarray format. +next(ds.iter_batches()) +# -> array([[[1, 1, 1, ..., 1, 1, 1], +# [1, 1, 1, ..., 1, 1, 1], +# ..., +# [1, 1, 1, ..., 1, 1, 1], +# [1, 1, 1, ..., 1, 1, 1]], +# +# ..., +# +# [[1, 1, 1, ..., 1, 1, 1], +# [1, 1, 1, ..., 1, 1, 1], +# ..., +# [1, 1, 1, ..., 1, 1, 1], +# [1, 1, 1, ..., 1, 1, 1]]], dtype=uint8) +# __consume_native_end__ + +# __consume_native_2_begin__ +import ray + +# Read a multi-column example dataset. +ds = ray.data.read_parquet("example://parquet_images_mini") +# -> Dataset(num_blocks=3, num_rows=3, +# schema={image: ArrowTensorType(shape=(128, 128, 3), dtype=uint8), +# label: string}) + +def add_one(batch: pd.DataFrame) -> pd.DataFrame: + batch["image"] += 1 + return batch + +# This processes batches in pd.DataFrame format. +ds = ds.map_batches(add_one) + +# This returns pandas batches with List[np.ndarray] columns. +next(ds.iter_batches()) +# -> image label +# 0 [[[ 96, 76, 61], [ 92, 72, 57], [ 92, 72,... cat +# 1 [[[ 38, 38, 39], [ 39, 39, 40], [ 39, 39,... cat +# 2 [[[ 47, 39, 33], [ 43, 36, 29], [ 43, 36,... dog +# __consume_native_2_end__ + +# __consume_pandas_begin__ +import ray + +# Read a single-column example dataset. +ds = ray.data.read_numpy("example://mnist_subset.npy") +# -> Dataset(num_blocks=1, num_rows=3, +# schema={__value__: ArrowTensorType(shape=(28, 28), dtype=uint8)}) + +def add_one(batch: pd.DataFrame) -> pd.DataFrame: + batch["__value__"] += 1 + return batch + +# This processes batches in pd.DataFrame format. +ds = ds.map_batches(add_one, batch_format="pandas") + +# This returns pandas batches with List[np.ndarray] columns. +next(ds.iter_batches(batch_format="pandas")) +# -> __value__ +# 0 [[ 1, 1, 1, 1, 1, 1, 1, 1, 1,... +# 1 [[ 1, 1, 1, 1, 1, 1, 1, 1, 1,... +# 2 [[ 1, 1, 1, 1, 1, 1, 1, 1, 1,... +# __consume_pandas_end__ + +# __consume_pandas_2_begin__ +import ray + +# Read a multi-column example dataset. +ds = ray.data.read_parquet("example://parquet_images_mini") +# -> Dataset(num_blocks=3, num_rows=3, +# schema={image: ArrowTensorType(shape=(128, 128, 3), dtype=uint8), +# label: string}) + +def add_one(batch: pd.DataFrame) -> pd.DataFrame: + batch["image"] += 1 + return batch + +# This processes batches in pd.DataFrame format. +ds = ds.map_batches(add_one, batch_format="pandas") + +# This returns pandas batches with List[np.ndarray] columns. +next(ds.iter_batches(batch_format="pandas")) +# -> image label +# 0 [[[ 96, 76, 61], [ 92, 72, 57], [ 92, 72,... cat +# 1 [[[ 38, 38, 39], [ 39, 39, 40], [ 39, 39,... cat +# 2 [[[ 47, 39, 33], [ 43, 36, 29], [ 43, 36,... dog +# __consume_pandas_2_end__ + +# __consume_pyarrow_begin__ +import ray +from ray.data.extensions.tensor_extension import ArrowTensorArray + +import pyarrow + +# Read a single-column example dataset. +ds = ray.data.read_numpy("example://mnist_subset.npy") +# -> Dataset(num_blocks=1, num_rows=3, +# schema={__value__: ArrowTensorType(shape=(28, 28), dtype=uint8)}) + +def add_one(batch: pyarrow.Table) -> pyarrow.Table: + np_col = np.array( + [ + np.ndarray((28, 28), buffer=buf, dtype=np.uint8) + for buf in batch.column("__value__") + ] + ) + np_col += 1 + + return batch.set_column( + batch._ensure_integer_index("__value__"), + "__value__", + ArrowTensorArray.from_numpy(np_col), + ) + +# This processes batches in pyarrow.Table format. +ds = ds.map_batches(add_one, batch_format="pyarrow") + +# This returns batches in pyarrow.Table format. +next(ds.iter_batches(batch_format="pyarrow")) +# pyarrow.Table +# __value__: extension> +# ---- +# __value__: [[[1,1,1,1,1,1,1,1,1,1,...],...,[1,1,1,1,1,1,1,1,1,1,...]]] +# __consume_pyarrow_end__ + +# __consume_pyarrow_2_begin__ +# Read a multi-column example dataset. +ds = ray.data.read_parquet("example://parquet_images_mini") +# -> Dataset(num_blocks=3, num_rows=3, +# schema={image: TensorDtype(shape=(128, 128, 3), dtype=uint8), label: object}) + +def add_one(batch: pyarrow.Table) -> pyarrow.Table: + np_col = np.array( + [ + np.ndarray((128, 128, 3), buffer=buf, dtype=np.uint8) + for buf in batch.column("image") + ] + ) + np_col += 1 + + return batch.set_column( + batch._ensure_integer_index("image"), + "image", + ArrowTensorArray.from_numpy(np_col), + ) + +# This processes batches in pyarrow.Table format. +ds = ds.map_batches(add_one, batch_format="pyarrow") + +# This returns batches in pyarrow.Table format. +next(ds.iter_batches(batch_format="pyarrow")) +# pyarrow.Table +# image: extension> +# label: string +# ---- +# image: [[[92,71,57,107,87,72,113,97,85,122,...,85,170,152,88,167,150,89,165,146,90]]] +# label: [["cat"]] +# __consume_pyarrow_2_end__ + +# __consume_numpy_begin__ +import ray + +# Read a single-column example dataset. +ds = ray.data.read_numpy("example://mnist_subset.npy") +# -> Dataset(num_blocks=1, num_rows=3, +# schema={__value__: ArrowTensorType(shape=(28, 28), dtype=uint8)}) + +def add_one(batch: np.ndarray) -> np.ndarray: + batch += 1 + return batch + +# This processes batches in np.ndarray format. +ds = ds.map_batches(add_one, batch_format="numpy") + +# This returns batches in np.ndarray format. +next(ds.iter_batches(batch_format="numpy")) +# -> array([[[1, 1, 1, ..., 1, 1, 1], +# [1, 1, 1, ..., 1, 1, 1], +# ..., +# [1, 1, 1, ..., 1, 1, 1], +# [1, 1, 1, ..., 1, 1, 1]], +# +# ..., +# +# [[1, 1, 1, ..., 1, 1, 1], +# [1, 1, 1, ..., 1, 1, 1], +# ..., +# [1, 1, 1, ..., 1, 1, 1], +# [1, 1, 1, ..., 1, 1, 1]]], dtype=uint8) +# __consume_numpy_end__ + +# __consume_numpy_2_begin__ +# Read a multi-column example dataset. +ds = ray.data.read_parquet("example://parquet_images_mini") +# -> Dataset(num_blocks=3, num_rows=3, +# schema={image: TensorDtype(shape=(128, 128, 3), dtype=uint8), label: object}) + +def add_one(batch: Dict[str, Any]) -> Dict[str, Any]: + assert isinstance(batch, dict) + batch["image"] += 1 + return batch + +# This processes batches in np.ndarray format. +ds = ds.map_batches(add_one, batch_format="numpy") + +# This returns batches in Dict[str, np.ndarray] format. +next(ds.iter_batches(batch_format="numpy")) +# -> {'image': array([[[[ 92, 71, 57], +# [107, 87, 72], +# ..., +# [141, 161, 185], +# [139, 158, 184]], +# +# ..., +# +# [[135, 135, 109], +# [135, 135, 108], +# ..., +# [167, 150, 89], +# [165, 146, 90]]]], dtype=uint8), +# 'label': array(['cat'], dtype=object)} +# __consume_numpy_2_end__ + + +ds.fully_executed() +shutil.rmtree("/tmp/some_path") + +# __write_1_begin__ +# Read a multi-column example dataset. +ds = ray.data.read_parquet("example://parquet_images_mini") +# -> Dataset(num_blocks=3, num_rows=3, +# schema={image: TensorDtype(shape=(128, 128, 3), dtype=uint8), label: object}) + +# You can write the dataset to Parquet. +ds.write_parquet("/tmp/some_path") + +# And you can read it back. +read_ds = ray.data.read_parquet("/tmp/some_path") +print(read_ds.schema()) +# -> image: extension> +# label: string +# __write_1_end__ + +read_ds.fully_executed() +shutil.rmtree("/tmp/some_path") + +# __write_2_begin__ +# Read a single-column example dataset. +ds = ray.data.read_numpy("example://mnist_subset.npy") +# -> Dataset(num_blocks=1, num_rows=3, +# schema={__value__: ArrowTensorType(shape=(28, 28), dtype=uint8)}) + +# You can write the dataset to Parquet. +ds.write_numpy("/tmp/some_path") + +# And you can read it back. +read_ds = ray.data.read_numpy("/tmp/some_path") +print(read_ds.schema()) +# -> __value__: extension> +# __write_2_end__ diff --git a/doc/source/data/key-concepts.rst b/doc/source/data/key-concepts.rst index d1c95c3b47459..83df11be0557c 100644 --- a/doc/source/data/key-concepts.rst +++ b/doc/source/data/key-concepts.rst @@ -13,10 +13,16 @@ Datasets A Dataset consists of a list of Ray object references to *blocks*. Each block holds a set of items in either `Arrow table format `__ or a Python list (for non-tabular data). -For ML use cases, Datasets also natively supports :ref:`Tensor data `. +For ML use cases, Datasets also natively supports mixing :ref:`Tensor ` and tabular data. Having multiple blocks in a dataset allows for parallel transformation and ingest. -The following figure visualizes a Dataset that has three Arrow table blocks, each block holding 1000 rows each: +Informally, we refer to: + +* A Dataset with Arrow blocks as a *Tabular Dataset*, +* A Dataset with Python list blocks as a *Simple Dataset*, and +* A Tabular Dataset with one or more tensor-type columns as a *Tensor Dataset*. + +The following figure visualizes a tabular dataset with three blocks, each block holding 1000 rows each: .. image:: images/dataset-arch.svg diff --git a/doc/source/data/performance-tips.rst b/doc/source/data/performance-tips.rst index 748055790edcd..5620306b20c51 100644 --- a/doc/source/data/performance-tips.rst +++ b/doc/source/data/performance-tips.rst @@ -119,8 +119,8 @@ Some Dataset operations require a *shuffle* operation, meaning that data is shuf These operations include ``Dataset.random_shuffle``, ``Dataset.sort`` and ``Dataset.groupby``. Shuffle can be challenging to scale to large data sizes and clusters, especially when the total dataset size cannot fit into memory. -Starting in Ray v1.13, Datasets provides an alternative shuffle implementation known as push-based shuffle for improving large-scale performance. -We recommend trying this out if your dataset has more than 1k partitions (input files) or 1TB of data. +Datasets provides an alternative shuffle implementation known as push-based shuffle for improving large-scale performance. +We recommend trying this out if your dataset has more than 1000 blocks or is larger than 1 TB in size. To try this out locally or on a cluster, you can start with the `nightly release test `_ that Ray runs for ``Dataset.random_shuffle`` and ``Dataset.sort``. To get an idea of the performance you can expect, here are some run time results for ``Dataset.random_shuffle`` on 1-10TB of data on 20 machines (m5.4xlarge instances on AWS EC2, each with 16 vCPUs, 64GB RAM). @@ -155,5 +155,3 @@ setting the ``DatasetContext.use_push_based_shuffle`` flag: ds = ray.data.range(n, parallelism=parallelism) print(ds.random_shuffle().take(10)) # [954, 405, 434, 501, 956, 762, 488, 920, 657, 834] - -Push-based shuffle is available as **alpha** in Ray 1.13+. Expect some rough edges, and please file any feature requests and bug reports on GitHub Issues. diff --git a/doc/source/data/pipelining-compute.rst b/doc/source/data/pipelining-compute.rst index c436c5abe3aab..75d0a6f00d32a 100644 --- a/doc/source/data/pipelining-compute.rst +++ b/doc/source/data/pipelining-compute.rst @@ -203,7 +203,7 @@ You can also specify the size of each window using ``bytes_per_window``. In this # -> INFO -- Created DatasetPipeline with 73 windows: 9120MiB min, 9431MiB max, 9287MiB mean # -> INFO -- Blocks per window: 10 min, 16 max, 14 mean # -> INFO -- ✔️ This pipeline's per-window parallelism is high enough to fully utilize the cluster. - # -> INFO -- ✔️ This pipeline's windows can each fit in object store memory without spilling. + # -> INFO -- ✔️ This pipeline's windows likely fit in object store memory without spilling. Datasets will warn you if the windows are too large or each window has insufficient parallelism (too few blocks). Check out the reported statistics for window size and blocks per window to ensure efficient pipeline execution. diff --git a/doc/source/data/transforming-datasets.rst b/doc/source/data/transforming-datasets.rst index b920f2bccda82..294e207ba4a5e 100644 --- a/doc/source/data/transforming-datasets.rst +++ b/doc/source/data/transforming-datasets.rst @@ -176,7 +176,7 @@ Here is an overview of the available batch formats: * **Tensor Datasets** (single-column): Each batch will be a single `numpy.ndarray `__ - containing the single-tensor-column for this batch. + containing the single tensor column for this batch. * **Simple Datasets**: Each batch will be a single NumPy ndarray, where Datasets will attempt to convert each list-batch to an ndarray. diff --git a/doc/source/ray-air/predictors.rst b/doc/source/ray-air/predictors.rst index fc4c73a454718..afaedd882f98d 100644 --- a/doc/source/ray-air/predictors.rst +++ b/doc/source/ray-air/predictors.rst @@ -35,7 +35,7 @@ Predictors expose a ``predict`` method that accepts an input batch of type ``Dat **Life of a prediction:** Underneath the hood, when the ``Predictor.predict`` method is called the following occurs: -- The input batch is converted into a Pandas DataFrame. Tensor input (like a ``np.ndarray``) will be converted into a single column Pandas Dataframe. +- The input batch is converted into a Pandas DataFrame. Tensor input (like a ``np.ndarray``) will be converted into a single-column Pandas Dataframe. - If there is a :ref:`Preprocessor ` saved in the provided :ref:`Checkpoint `, the preprocessor will be used to transform the DataFrame. - The transformed DataFrame will be passed to the model for inference. - The predictions will be outputted by ``predict`` in the same type as the original input. diff --git a/python/ray/air/util/tensor_extensions/pandas.py b/python/ray/air/util/tensor_extensions/pandas.py index 0e4a455140f63..51c97d9fc4ab4 100644 --- a/python/ray/air/util/tensor_extensions/pandas.py +++ b/python/ray/air/util/tensor_extensions/pandas.py @@ -435,7 +435,7 @@ def _is_boolean(self): return is_bool_dtype(self._dtype) -class TensorOpsMixin(pd.api.extensions.ExtensionScalarOpsMixin): +class _TensorOpsMixin(pd.api.extensions.ExtensionScalarOpsMixin): """ Mixin for TensorArray operator support, applying operations on the underlying ndarrays. @@ -489,7 +489,7 @@ def _create_logical_method(cls, op): return cls._create_method(op) -class TensorScalarCastMixin: +class _TensorScalarCastMixin: """ Mixin for casting scalar tensors to a particular numeric type. """ @@ -513,7 +513,8 @@ def __oct__(self): return self._scalarfunc(oct) -class TensorArrayElement(TensorOpsMixin, TensorScalarCastMixin): +@PublicAPI(stability="beta") +class TensorArrayElement(_TensorOpsMixin, _TensorScalarCastMixin): """ Single element of a TensorArray, wrapping an underlying ndarray. """ @@ -578,8 +579,8 @@ def __array__(self, dtype: np.dtype = None, **kwargs) -> np.ndarray: @PublicAPI(stability="beta") class TensorArray( pd.api.extensions.ExtensionArray, - TensorOpsMixin, - TensorScalarCastMixin, + _TensorOpsMixin, + _TensorScalarCastMixin, ): """ Pandas `ExtensionArray` representing a tensor column, i.e. a column diff --git a/python/ray/data/_internal/arrow_block.py b/python/ray/data/_internal/arrow_block.py index 692f31227256f..fe0c05c21de27 100644 --- a/python/ray/data/_internal/arrow_block.py +++ b/python/ray/data/_internal/arrow_block.py @@ -41,6 +41,9 @@ try: import pyarrow + + # This import is necessary to load the tensor extension type. + from ray.data.extensions.tensor_extension import ArrowTensorType # noqa except ImportError: pyarrow = None diff --git a/python/ray/data/dataset.py b/python/ray/data/dataset.py index 0477fa55cfe37..04ea6eeade78a 100644 --- a/python/ray/data/dataset.py +++ b/python/ray/data/dataset.py @@ -3280,7 +3280,7 @@ def fmt(size_bytes): ) else: logger.info( - f"{OK_PREFIX} This pipeline's windows can each fit in " + f"{OK_PREFIX} This pipeline's windows likely fit in " "object store memory without spilling." ) except Exception as e: diff --git a/python/ray/data/tests/image-folder/cat/123.png b/python/ray/data/examples/data/image-folder/cat/123.png similarity index 100% rename from python/ray/data/tests/image-folder/cat/123.png rename to python/ray/data/examples/data/image-folder/cat/123.png diff --git a/python/ray/data/tests/image-folder/cat/foo.jpg b/python/ray/data/examples/data/image-folder/cat/foo.jpg similarity index 100% rename from python/ray/data/tests/image-folder/cat/foo.jpg rename to python/ray/data/examples/data/image-folder/cat/foo.jpg diff --git a/python/ray/data/tests/image-folder/cat/not-an-image b/python/ray/data/examples/data/image-folder/cat/not-an-image similarity index 100% rename from python/ray/data/tests/image-folder/cat/not-an-image rename to python/ray/data/examples/data/image-folder/cat/not-an-image diff --git a/python/ray/data/tests/image-folder/dog/xxx.PNG b/python/ray/data/examples/data/image-folder/dog/xxx.PNG similarity index 100% rename from python/ray/data/tests/image-folder/dog/xxx.PNG rename to python/ray/data/examples/data/image-folder/dog/xxx.PNG diff --git a/python/ray/data/tests/image-folder/not-an-image b/python/ray/data/examples/data/image-folder/not-an-image similarity index 100% rename from python/ray/data/tests/image-folder/not-an-image rename to python/ray/data/examples/data/image-folder/not-an-image diff --git a/python/ray/data/examples/data/parquet_images_mini/652597cc55e94e3e8d91776eec17774a_000000.parquet b/python/ray/data/examples/data/parquet_images_mini/652597cc55e94e3e8d91776eec17774a_000000.parquet new file mode 100644 index 0000000000000000000000000000000000000000..f23198520b7ce342e2b5a267bcab211679fb6ff3 GIT binary patch literal 51975 zcmZU*`B$6Sy5}bl7D#BG=U4MQ&jQ&hVJtKQlE7HTmL)WUFt%lZ0SSXlS}N!Gf_^S;;yOeVA_s97;ugd`hz!I7h$6%y#6yTBh)swJ#3P6*#7&4Dhz3L+q6@JP zu>nzs=s~m~9z$$F96{_s3?L36-iJ7W_yFP*;x`cI5YHf9Kzs)A5yV#zzk~QQh}RH* z0r4@!pCJBIh~Gl|R}f!7{5izGf%qEYA0a-0_#;FI;yJ{9h%<;b#1n`Wh&V(NA_cJk z@i{~v;(Lg{hWJ+y-$DFqh;JeObBJ#s{xgXG0^&=EzlV4V@hQY#LL5W9ggAg$gV=`H zg(yR$A%2AT5aI`j{}ST&5dR6pzl8V~5N{y<0ipnL1tJPD0Wl9T1rdgL1@Q-n{~BTz z;%^|(z2;s3ofGy)^|cKp~LZfF=katVL|VRVd08YfdmsI(zEW0Z+w zu{m5GfiDo!L}G~)k;$n9g;J%~P_;UZ-XJ%c^k$3IW_LJUZo9{8b@==flL7B#x9>`D z^1aa1RnPP_V|XSKosMba;zUwH)Qr+nvn<*ii8fE7jizZ_JO_uw$U1c*>5^MB+4zFq zy=b1kZeP0L%-wV@-*Qf4EZ=nIZa9~2JD09H7f@^%Ft9J&w)<{7b63pyJB8v(sq8b^ zLzSc}648p-2?4dLLxei9Oh@30IO$r7Bqgxwjn#;7H6mt3h@^<>XW+|K1DRim5wbd_ zNggTbE#_#^UR?7pUvIA8^>5sD0lvFY-&NPbZP&sL*V0Xt+*_i{w=k|mm#nPs)b|Cn`8NB9fU32Ci+JF#7{$YD=ld<&(fI(w>N0r{K^0%mxxeNVcsQ&8xpLjz= zyp)WS(}fjwRV#=k>$Or=TQE9pR)@2hahAPy$AZ1nb-5QE-fM2(t>DaFsC&2Ry;cfL z)tz3Yx-8_G-s(@1fc|^)Y1)38mc|S~bZVa7ABl$44qq~JU|GKBT)1spx^2tduq~rF zmqB7o{iUnu`#-aA8}-0<)wy)T=Dm)3Fma$SZ)9x~hOAv$tLWG~U9T!or33_;fRh%b zNHx@e?TC0gLS)I2w7kldS1IT#IV~rX4AedtpceG9Qe85eEI1vuz(Uk}4O9Gfbm1ys zh1aDU(WRTw+^wkjAsLlj{ZMtG_&vM#`X#*Tt>pILHTjOM1&ZIXh3=Sv4E$l}&O6or z>0szz9Sn`kr~hex=zrNC8riGCJHQ0I_C|iqXT1^n9%^EJjQ`fbOZb16e<+)MEIE7v zz#z4ML}VS4DTh?rES<3*!?83RZi2wqHTpHkAJrg2<`{a7e?$0b{}SC))0Gqoi>YKQ zSe?$e-Q|v!mz%}4K=7%1<9cxAZfNqVCwHwdF=OyAYmyG4Ac=?k>8EB_X>%;vK8d!S z?o;sXHo~mxDgskfZgpqlOGfvJbNaqDcgwzf#Xfz{>Amjs-g8c0aW3DqFWt6d$>RjS ze0QB2cfmMF9=i|xy>4+Ymg2soHKZ;z4FL;b%w%)nmZ$rEb8XMR{vMh_8&_SBLcZJn zg&S{Ky5V2C@pB@%z*aPuAd&8&{E|fvPy<>3WD!Q|+TipNZTA@fgVg+{cy))&J*4($ zmH7RbjKu@}QxsHxAx%nT)Dan2Ag4AeN<^E8>7}BALSZ(QoQ0^>9&j~Vp0KYIJjrZ4 z@Sa@FuiP##J>0XnXB3`gUdf@-VFvIyv)lbSR)3$=2Lt$wHV0u+1z>>HV4u*IbA^c& zyZ1pfhfPPw`xg2iZTH@@FW-Wy1_q#_gY@g~p=_WASbaC0)7J{&DZQ(eHI@|$V?lq7^ARGq!W5cn38U6zwt1}a zX@BdWynZ!4^PBS22hA?(0aP;o#?=d&zS}4ZDB!`*U}oF`CtSI=He3`ltdgO{4E_JY z{@3tZmnh@U0T{IBhK1W>SPeU+OXit_RZTn1<4^=peH*lz7%wFb>MC3vJm?84u`238 zO;<3y^T66^#)E;(ntSO*aPn4Y@@9DnGImB?&It+ua!oHocr03Ko&{5%#^Or2douX#4SDfLb$7>0%Sz9k$P%?sdqWR0#&6Ufjrqy@Dx^Ww2 z0ZajF?%VPQ#&S+;4k*-?h)7;z#`Xj%A*inzkPe3r+8f_}bn0|!Vf2j1I!Cx8)&3A2Kgt}i*tiQ1`nrZh zBxclzvYM)0uM*^RVue}FYLrP`q9m7FvPoOX8MQjf0dF`s6E16dAK2VoZF#w7 z>Co$?R8_!dG)NQ%r9YtI2aJI%)eu(cF-^@7F`2cta>?(uIhLdDu0OQm&rG`9Gp>m# zl+21Nzvv2rF?U?c*Ic;=uBGeI+%TwMQe-H3?e35KvwY@GWHEn@8J@N+uJtWI*O!_SHsLp9l4y#W6mFYk=qaxIxV?2BnF@8Fizou48@(+`}WwfCAW_z-}> z>hY`D%})Rr^besy-6V|e2v>KB=`9syh}_@r;a8P|bga#1L!J^Oyapp79temskQ^qt zl!%j)Q&8#mv>K*fB(SK}QCl<~$P}l$#ibKZtLvQb>EqJ|TR34TRGGC@N`)Ih|C}NU z8&yQiR91BYo?I^1CX+^s(`NTu{f=hY?Qw)WaUkP?F}yljp3bzVxa67ec{1ze&{VlK z*$hoZi$15rYiYWZ=4M4M9k(eVlqBc!ea90n3P#Ie@;W&Pe?%F)dX!j(I4gU z_j!~#Uiwr+I@GM5Ygp%r^r3)%s*?;!Q~U~(nlx0*wzAbR?F!8_a}O}JKZMeb&Fl-c zuXKmlFr!@yw}$ZY?${OAl6*(s)gOfZpRwutPWi#uHJex7l|TOJ;gI*?{Mko`k3R)q zz#V_hm>uTKj}b=C8>@$6(y59&NA8bOB`o@Y%f#_x?S3pJVK?xa27?E)yCKmLXf+`t zf}|8ZVm8(*)pWL`-tY1R+}=RT*K{x0!_&riFl+BfvlezDLPmP3hM1~SDXEAC8W)~2 zA{AsS@={W3wiN76rz`6BG~;DY0Hqm-Hyv?LCtfbb9hrC_9}j!u#pU?KQat1HOe{qm ztv#E+Sa6n2NsB=m5h;6Y0#3=`DkZB*{yaT>s!VS&xwA3KoQA)z>5pnC`)uw!J3SX8 z&Bj(oV%y`f{T=q`j)t|HpqvV5^U{>0!dB|lCR?-v4*FY*o~8TosrMX{AI4|iFL(Dm z>vugHSKomxo-nU{H&GUDy#qAIKx;SJ!&sr5EZ?`o6i1OC{PeHK%-%bI&HM0l^W&XM zp8+sn9Y2w5y~GWjwdZ$I`{Rx6Low@=n4Uu@`&HU%4^Qe1xO5y}$)Pbhk{HZ-PM^o= zLvLLZ!otcikcHV8-59AmOU9-@4lcMHZhPEqE@w*7fLd>>M=A+6Poq={*veXhS=Yr3 zbzW9c(`V%teb!mfTWuD<-RW_;;{Im533wcUr|c*@9Py6BQFH{{j!r1vX@M1r#D%go{%dv1(mDpyAq$;0|Oj0AHX2K_Fj4YF0cn} z4aV*A#%<5WjpoKp6d!O4z^u692Olm>$bkcx3CjVV(F^}z`=DR2}`o1fs& zL1)o7QIBUJjf64x`y_GMOrox}v75)OA_8&ZN-k)it9w zVo4foCAGC+a+&o_XQ3Rm#a(uX-|Fy315Lmm@H9K+awqNx!~>`kama{?rl(`IHq91i z$yCb9vFplUE>{2qUl%AKRKPOIC8)zRJR=pl7`q_@2VP)A3oqleY~-Rk@}b?&$~ zG)A92QqGM*Hn92gJVshXOeu6Vz0qp#Ec?9&o~3K$^_z~Fy#Qu_W9ofJ_dUlNa2Mlh zd=0o=UI%Ufp7ooa4FI(Oa{;qp=^nP+U$^G&+j9@>%h#OC4{ghL02u6C`te}o?yW^5hFPS0?sJDj2O)ckH^e?+)FD`w3RxubikJ`0g>$qY)B!9^27mS)P* z&@1o;T)}`V<=_=nv56Ne80xm9$!@V&t)IjJz~L3FvZB)1z4 zcB8^-thfqlzgg?CCFAyz!)gpfEdiIg<9BwN_OQnq_PD}je;CS=2Xev_Zfne zs9VGIp)nSott8`9Waj{lL^-p!*)i z-9Q((3OL|8#2J?0GUrshW)qo+dFyqLR9?2}ZIG+B~HlYv3%F-S^0 zD<)ah^r3m+)`hZ4Pn8sa1yWm5ZM7BrO&6%ugeBB!_S;G>3ruva%bazY)h=_zWdY?Y zE~COA@&M{m zjKAUV(zg6zS2I_ssrNPoMGNsnc~ zttOfQSNy2-F!%38b6B^#{BG`}wdb;V`UL=kOCNpp>8D=yKw4iVKH1!Na)jWtHj8Y4%-ks~6i|55dHvj+(MHP@~}gX5WcA70C_0Ib0R0LA~d2iUj)GabslAFKJUg)32zd+DAPEBLUr zLzi6WD!tdt!8_*QWlQTtq4m@V`@Q^PDRctB;Q1%N{o?a)0T^t)`f}^ho6*fztfMEC zu}id_$N2MM#`w|Ta7;FWc`$$4Se+B6w^e)=T8h8}6jqhL+M|rp)AKR@wm?Ry3mFw^ zD#;^T&l zZNRDz`wJ5uo4f3Emr(;IJl0~<8t~g;f$&7_e!DGdHrh)jo2gQgi{NZls^Vw{TqR|` zN1BDoj~W2dZ=t#iQQgl7BS(b8Bf{<`;c!zta-=#vf;!Y29cM!O*_Ob0X(W-znk>d< z$QnHGZ(Q+j+;FX5b*;UJ0$bO0I6JuQN4p;31JY0bVK};kN^Z@;KKi;7)%(89`@oFq zea+H(SjgWo<}a5*FG`uG`plxsq1p(3rnd z3f(DXF6%QdlJ2Lbi34qMF&sI4cQ{S?!3H0xo0NZz=lC08H$T%h-wP1J-2RsfDT!nxo``$o-DeInX)Yuw`U*+pbj;iVZSrrvX!IerWJOc1(!`9wV15NtT|gZ>6CE3 zC*bsBv~4=9!mtDnoul<0rvBj(D*gCy`}`#iIDQNR$N}Q)8F_w(PCA7aLO^UwnN&l) zt}QuB{*XDd0_6h}_rAq@y#NFs6hi;&Pu#EW6!Mn~`4`5}Q)A|7$$g-oSWFgAOkrR} z+nLM;Ce@A^wPUIh?<&e`hVr`HvmtLTM4C$x|8m_mU5|R})|S+s2Vn617vKKjyFUOh zc=OqhkKTMS_WZMvC!fqe`iM0AjB@mvfA*+9_L#OaEIB_Lj2z=;4^#8AgzY)P>U@JV z+DLB;x!dCOC^5aSBF$I1^Gw)J%2?7AS3zdzc`BPi(<~_Bc0E*`PP72iKkR}ez#MRu zpv*wEkDIk+DE^oO7q4-PI^e_%@R%|6Z4*sf(Qkvl>xf#))`H(|bU8~=b24htT8(O3 zR+gQ1or+VBQ4R3yKx@>R<4MnQGSD>zrnG^nEGp=XI^I>6*VSd8+Otq;E>-+Fg=<+6 zoi;?LipHGvzOki4nOC@T5$B5a`uRS_PD>ZDH$IdjGW_U zN80nJoY5i9_I!%9pF(XI6{h#W12QZGl5LHQ1y7yiD!fTUG|Sokk|FL)I#3VL*0O_^ zAhZf$Sx|zR?y#9)%7X_e#tINGs5?$mFeF z6wv7%x@26CEDKDYgw~VDdU)9;PuO)k**i_I~*Bhd+M(I{*gXfARZozWnaVH{YK<|8nH@H={2~M9H2GF!cl`Y68$!c_?I!7SUu8jasFt5COJ=$TkpTa$+n(WCC@f z5|Col|Ki6V{_^*~17Pso_rL%AtFJ%8a$xt#r~5BICP7cS`3$vS6D)W{85;%*XyZGQ z!*l%UE@NnGus^4<`JD;)Pq*QrVt__7!#!pnlY%h!zZ1 zqZ|w<)njT6Qm=A!4LlpirPH7IAe zvyL#ODkTQJrlQw~JrprOtqG~3236Of3NR9lD!^-y0Zjw>g}?glhrj;%cK{5&{q47q z1D|{e3&B_CPd?j!`35rp%K_AYOWd=^kOq{oP5$v&fA^TSwTmAb0S|DibE))xgEFe3 zt^RC4P1vs}2_;d_t`-D(f(cl_qp)a0 z{v^w=eXwQy=>82AKWdamOA)75WQ@eJ39`J`5D^C&4xf%+B~ymPt0Rs1bI$1)Zg-e* z{D^k;Z0{EXc87_hyG+_>0-kjdado;_&ZEl(nwmh6NYrAyI-Mu)@eDnlg341=0gar@ zu9IUma!iCUb%;`+q9;^zo~nmJ1gLKjR}F8DZ=ouG^(B`Ey25IwA`sgwD_=tbFLpvSE&yUEq zXE~(#27i=DTdm?*$^lD*BZ-84MK7fnRGo^1D;a?%2a>>HhjsTC1E8Xy2Ea-TIRM*V zo5^9;g8BgqW`JEk5j9T0W(-vyYnjlEL$+zF_$>-oLGCg}T%|~~REJ*IC}(SRR0C67 zBeyjT8ht=Q`14}c7GZy!GdqSm++>`C0Z-oP|Ga&;iL7q1@hnJ~8p5e7RiJ-Fz!vc| z8n%YcN+OvFh%Z2t;&<$cRm!&z zm0}b+hVqZ!{qVP6qrl)Ve*E^^AHV(TtFNDZ^8MJeFXvu0rN10<_e2Ao{O12B@V!8i2U$piw+a8z4s>J4ab z=`3}u1ptx&RUZWm_>~7{Q^2AHI<{o-!hk4DfB0KTht(9fYs)ru)2#4YR(92I)M<0`pF#XR! z|0k@MApZ0DCm+whe!e~Ynl$#3yL;9@J!b5W;8;Th%BWbfN(TL9Bz7ChDox&N@0oZi zON41v3tY*%$Ea|CzD_-?yC^X6e+`-h$$-riFlz&56BrPNt(dh0tx(ts-w)ZNEj!g7 zv!aQrU-uiO{!-1aulbWTt5#=LX!KHLR#274ggP>gP3Th@t1AASkTpWsKX0F&4d8fd z=MsPO75CAb{^qmP=?;}U4~MvQXn*KLIdnie^mtqVv!7nAl2s~%s6tdKG6Gt^13|xp zKOz%}2p|T;fC?aDsKhE2{I$1W-*oYux+edF$XCINyUUA2u6MC}x^40#OkB44;GW6`t{^M7xN0-v)!~Bsk{M@)~ z^)$tu7t_E15*^27!yObNUQSPGd8$HP>8$XgS&=_kFB=um;lnl;1wHUV&wL!Hwy-+9 zEkgf4{q->W19tdMIHyy1Eb^wY?$?X_dYwP{7ME7wtY|E9rd~?VCJ?!j0A2lnoubeg zuoBF{NJc6gvhFH~*P#Q#))!QNF_A39^lyL#LR9|? z#CyWGsetJY$^&nIU3`I=f!c*&s89niVDL9T{_s~n{_r*QLErrN^4S+7FF)Hq`(%6U zW7f{A^f45iC$KM=d-BcJ$DeP#`F!ZvH~Wu2VnJzwv+L6_#{O=bG}P#W0cm=wuOTu- z)V7@200ZJ6geGNqSXLa~L*USg138Lh1x&R_Q;{YV5nZG%&xaXHvH($-a{^azdFSkDa64n3F$E!!L zxNvU0b+pHrA8)f}8~stTjH_%&5@ei|PB5^=SpfnD6e4tUMCp_>t%g_>wSni?!ps$U zO470s^T4Kd*kF#M)Bl2Z20#*k0a?@lvmsunm-Ql4Zcty%Zf4mYlct>HmBE8#0#X40 z8{l755LJ^JqM`>LNHj81k3WyBjtJ+`JQ(b5wrAnq-Y74{00YzsCfr*@%a1fs6KUXa zDP2iMKwlL>g)|UE_;z`P0V0HVi10*PMGz8GLL#RjhU>RLiXcp-gz!qhXb^+|fzSX_ z?;tc9*c<5Y=pSDIqW=*C1~B{ofL5Sy!2n2r!{N{Nciyayzv7NuqRkG|wuaNkm$rvr z%{}>K=+!q+ir~m}_2?PA*&jK=^gk8$N2^f(#W4F*@|d{7Q)zW%y_8z0_bic~Ez)yF zdUiS8s$fPdY!J$C(lsFyPz&Jr^WD;5$)ZKbriH=-;)kuKuw5H=s>3!#$0BzaYh}H_ z15+Qw*RuEC8ldNuOM-GqS2l_~dL8_u?TVPWPA>__Y9%3}8VFL97=F9T9TJZof(NO& zb1;D2mPpA$IkPH_QAO-tf=y?~dc7V3LN|n*6roKZv>OeM5D=oT4eT8P0hdbQI4K#2 zGiYDXmf_w4cV__f7kUvI{U!Sq*c)&YUIuV5+EDXBcK`;U{_lSL-H&i22L?R(;&Ah` z{o^;J!)N{3mxIx@ZjE?QmEiT za+6J>YDr2p*1?0SMM^f;5StWnMyPfJJz8P9U>>6$)Ug!-D!dw9q2XBow5wo%6AXaO zFrY2kvqgKQXft#yQU~a4(s)elGLY1i^_qBz=P(K!1yR5(4Ok$lpy$=OOai-tU8qq_ zXf;WdjLiz>o|WG=PBzBn-&~c!mUEVBql*2_TWsAn1}k>`NL$@{ZqOl4Gi) z!vFB&?|%Dh?ax0>pT3stZ?@Cp9MUdhZil}+%sRTX{qo5+Yz8);q1x}jU;$d;x+p#E4k|)g+=q=PH52y%{8L4PPWT?R#bkTAFV=iPeDR+%_fhzKE``G7zT@5RTs{}2 zkHP=$i?6@NfC0wuKmYQ(=O2B)`RL1$vrk43Us2{>4p=*c{#K)Zire0mobGUshe=0| zNoS8x#x7CLU*c!S+S`Xh%3Kw;UP4KY%Bg4&ldhM%&>!Z$G(}Hspd=^@LL%w$uW~ghQ`!lrW5% zK!MjW>xxe4gf-%h8r)IEM6@yyRfp{gSTLYFf-HbF85RQ_p--ocQu*`n(4O6<2^zSB zIHhcG$WUieP&d#y50e(u@AvsMRBlZ7edS9F2Vii)n~y(#^!n4C%{Q~>uUNB>Xxn3L{vmNNuNsV|xU+-# z-TvVi<@ktlenc5R;*W2F8}028aetqROUHx~0T~xj3e`NaNk>h>>L()WYw$o=RT7Cy zx<*NmGV0z+Z~K+>Xcu%LJseWNGqA#i^x3czDvs72HDz3;klz$?K?W#_wtCni>J$Wl zLLy*6>6is!o2Y2NkY0|e@2a?6>IrM5W0QmW(5-^{kO2xhWPl`*f{HUh4;`rD8kMLa z3#mkcSlCXbWSIJR23Aj?s>m2L976)YK!)!R012N#<1^qEe4mTwb0rjjBH_LTO@cHE z%x!Fvr}^A8g#w_7K!INuzjOH%6w32gZ(d=*;Q6ahoT~nvUQ9fBdGW_^80E3s0fM=V- zPsVp1%^tqo9@^xu9?SZt;`VBf*jI{ah>X;r?928~8MB9jA$SHbf}7tWkmkq(Zj3mn zF@^F3SpmI<06s1QH*UcL#FQYK5{+bnkQBgCT&kqwKmuIoZ?2>ITd~>$`p4fUz=i(M z=!5=n3jJ39kU!Zfms(|gzNrOm-PQ=GjZrJwVd{hUwFxWeFLg)jnW#MEQsn*Wd~+`w zYHA^MGJaS^vhhMBsuh^!F{4yjlEzAPwjQqH)!0&AFQ$ZM9~#PbLx>|ZWQ5d!kYY4a z10i7`ZpZ)_G|+eO1db$yqj4}N+KfI9-^WS%G8)RD-@h;ebplHfNk5H3<4R~0iG(ka zpeqVgHmGeVwEp?=+4=Dp#@X=>a5R2?b~>`NKXgnQ9j9@(+Kf?Ric3fEOoUP8@Wm1q zhc%BI9cpjSrB>$&a4V(HqH`pPYAV7*Dx#iRM};F8ZGx=jAzB{M#1oo$gd{JOOk5a{ zGKvU=nz$fUS?X0Aw4oK)Lg~u-IvDV44;+O`z@qM$)kS9(Dp07IY{g5hxIXUzJ=HM9 z!2@)}1JsMKsRBzl(J&}s-K9w(XL!1ULP2&EdqC>0i@#6leb1I%)|9X0`40qlk2umeCb zV9PLUvlI>sH2amX4N49=iDCNx+5nj57?F(Y0(r=92>BKHCP=Rida~ZK$y-iN$4k== z<8;8{4Om)^La1DDdyK`VvEwhm<&3hcV6`QarbxZ6iLuGlp0G-7RI3CsIR#X!sa};p zr#9$RLa&;lk~vkRO(xH>w1j{TV>$wAqF;45fdtua1WiMW^%M_j!6T@&>X!kb)mko zE-uvJP>x)vQ~$XEuneQ1y{J(RemDw-fW-iNt_iF9!hm30-|}dowm^l!>O7QsL*||C zd5^l~$p*_>RQfnbuU~eQmIKCI$C3-1b46#)ZOvu;%X!aqz8nmd^O>^S4cC05X1KDa zmI`!AW(-j>iFBoyt`ySgLZwm|Q;OM4B3mhr(S^+3g%cbq(ypp-WTFfa%2bU3k-z}N zDZH427q%shpW#Rv9EmU`5fkuY_%`@v6x^@ra~ge)5Tn6w6Zq{Ezs;ew+Y(rQ+A>)R zK5hcJh^8P_)vgK$m#)x;PS3$=O6W@sZQv6JIz3bx@(p zvP_&wgG(mbsQ%K1F(NL=RfPyrh*V)yV3EVoC?r4v&UE5NI32BfN*AC;K*0_aB~gdz2&u~@YH&KsvBHi4=!v3mwl~Fu;?#4^~Q>{uHo^h8eWyHL1G#eTcgrwkOU7A z@W=uUIl)$GVnjAm7^63oJpvt)gZ$QsDp5uj4-g>(X)shUNeOr&0gnyd>@lK~jug@{C`k_d_Th7eC^FbF9bd^iI3-C_=+;#9!{B91D?^@K9I z5XU5P*a$&W6(*=+T~Az7it9`$=lHNdRx>HyVqthMLJZuAeMW=7VzTvj6xBOGN&g6P%ab@{rabf0UVQq53 zH?t62XgP`=v%REHOKWwFNXphpV>&5QCsm51OsSMvm&SmaNGa0M1p;bK>30q1_{LMK^h5TC}IpDsyh)wj7Y?&`Vyg7 z(rDlbDJWQj6nqc}+Jd%<*oJ=~*v709*i<2}Cr;3b0=f#?H=Pm@(TO#>P}*zMp$nzr z>eWGnj0XcM^g%X;gC#%@?PpDFgu){y7Yx*boNA0XoJQcwANzNKl^t2Lk6G*Wb+s5cZ|zLbIxsF%0w43QD4Tjf#P2exi07i zJu^$46)<3O`Q&6_W(5rJ`DT1x-*hL`v3jh@f~jIMC`<-}7CwX%F=*=wZ3L(Q*}6hq zlPg3KgH9?>)apFcCs9nGVom1$0Uk)RcT_cCr$LAcdKc7 zH8!=TRMmRIT9s2H%cSH%os5r=C2}ghq8wz|ZBv4vhdr27WsHFOJrsG*80o=Ly|Lc2 zDVWwQ+n?llN&=4&@<3EB)POjgGn7Pd(i1R>!sfc$4(ls)m7qR4{WDfW#)|g&&=a*B z$zaDgJz>jbtP8ECFIe6P#@B=KwV(s&2IJjex!dwgdCQYaVbp+`>64YEw+2jZOs_9Z z1Vhbm-0ZUJoff^bkaQMIPNT_Lf=>tndb2)hPMQj)tWm4ht1C(Pd=uOjlFQZb`4VZ} zAd<>;wMarF<>_h~0elQ!7vl+-faYfsN*+dnM+bPCUyu-qcp`Y9f`LRVrU`|#MuSF3 z!JKa3h9nrDtJi3&L)Tj*|A`=nuH5L5m z0!f`rlT#%XrYx)BBn3jfNL7-OVbT{2N)$QW2xpU+{$_*Hu4ej^Y)^^jF(%4JLAfA` z14doEB#7%1U_sa-g@cC-R2|peh5Cw&{ha|&mjWek(Xx!GzcC$H&lT5l#cpn5D(9Yp z*X8icbYR9CpImmVEEG>r1H31bxtU4GfDPZ|`f}Ho$xR3H8CN*qjF$mh++#**n#+LS z>~Wc!(SqMGeACI36O2YVfGiM-OgOz<@3f-@fR}Ph0xSOeguLq9`Mf=pac11Mi3xMDXbFc4olYUpF$MxghocmampmT5 z->;8GlUBRQ?$kPMTC*9&X~W~-$13nqa1;q3GDK1!WP%*DQR_psV`F~+iLj9BCrD%pbsER5v9=( zD=0J;b;6R>0fh?BXkf$pr3f9$ky!yrK(}h&=0VxWL)9+9cLsA zM?e_hLB>^qw#W-huiLhicP)6!8_S*bCHI;y-}SYoHi9!7(~}$Cm5tWQM(AW?0%iTe zfc(nLG#D_m*qwTM(!Km*Oble7`@(@6Y!8vl7}Y13xri3=vY> zRWgmINy+O~wLw{l&=pd+I!To$kO*5CQF7|NL4*oZzpZ8y&=UxF_*Ltb>L5NF1TUxs zmaNEZl3KJi7#kIu6664;zr_FxG2EA zcka0_U!QpKptyLaGx;>oU2&|>lzmg4g|0uh2EYKsWiT*cAQ{3-hCp#i|CGib>GKZ( zTK^DFJ7h=>Ik+K&m}V;nQk_syCmU*rL4+7|h#^5VXc~rC3aXBRi9^*-5R*E1@}-2^ zfdzxcQbFgv*;KP7>o&dIrjNY!z`!;dn6M;U6wr#ocHn0N*guQ!F~vOakjt2eTSN|< zG+>W(tnx4vp{OF$RJUNmRW!fT-?5PkUnBrJ``4!@*K#Y}+(~zOu^T+-<`24|#S8J* zLMv0jm6e6bgQ>~O2a^vTOkTM;d3$eW?@u!y?ES;c2mdhj{ySj&!9PxYh|>MwAJ^Xh z$MwB`Sby&yH{SafzPo?FaQ9D3cmHhp>TjlRf9Spaf%nGy!JEKd5L`ixc`tbHz1F?E zt^2n_4{v2&T%TBcSX{Xro_rCQdEw|j1z<41Z^;-aFhG%PF=!+B{-LBlPUD}`_{S3d zF&^k27bwlXv_vke+`{6+& zW`H1W7CD^t0H_}|6kUo5S0#hCpHTnJOBcN-RG^CnXu&<@%g?NPQT3;Ci&M*sQ{IEA z*1=RBHDE2XvYwe-%TLbaW>(f_4o;>nU+!MJ-o1BY?dIP4?GI6IU|fu^ez1lC?{D1r z(0B7g->nZ9u6(d`<^9~1z2$p*)A!!<-n;9)el>XgHp+dB2e(@fZUFgfH$o3@W?tNL zA3T^?xl^3H9GmF^~Z#= z5o%+fP3G$85-D>K(XPE>lSy%d4_;!*hM zqb=*Qn_Ny4jNwh2No30kY}tfUt+A`wuosAGVLPCK5`=;pfIaC=@}MDdm}}_tj~c?D zzF+Nz?EvhF1IFpF4UTt~^8SUE2b!P@52k#f$@SpkRPJDA>0oC0U}pMY2EJ^ewFlJS zaL;UnX1dE$GaFMYGu?xO^%r+G9^PMgaBJcI4V3G@;JbHw1H*Uy_QL(EOAoFtUAvll zczgNbjp>Isy?20H!8=!ickcbd<$J-)S6Y{^QUo!|snBMc)d^>|W;aEwdU+I8Ug5H-{dR3Ls)wG<<0`=z-n7CM zTGJkUpU$ooShERhmgh3TQ(<0H4_h)gnz%3k76KkLrD*-JMmjD<(O;QpW-}gBKCTZ2 z3f{17dBVDsaV_PW7apuHcc!50S!*54+WHE?>{RxW4@K-txh{ z>BZ~b#rwgN2N>5{C)Zkw4}l=cHK4WlAiww^w0J*raNWIlq5hq4_XSX#x-&6zZDIx9 zyblAy;Ow8l7)vqEIQXNsWEe+#DeFHTP#!V3uNkB#GVUb;?MO8k?{Q|?!ciSLU02d7 zJe*cXNJ_<^enCMstCcn_&uJ1_^>9TQt}DZp@roZV&)T(~XtL}t#XXjIvjAgw(*^g) z;Y>a$jV47=6MAXE5BGTV2~P5s0zQF#~>-|aDl;Hga97=#1nj%~VV z2McoUXf6aU_+cBomg|7RCu=b2r=k1+CsS|5pYcshErq+ey@~EZYi(nBeQjf7YHDd_ za(VK^yYe)6@&aY?sTX7V;OX+giyX@3<;Ba>CwHb-?szBf1ZOS>XI`|XfTyjg1B}Ji z)M9Jq0OM(E=4pQBMP~A{dlKZnTY*E^n=4iEg?0r=?yS%1L*SIGf?sqQaHo28nkB*Km& zuq{C8?Jl4UTVc3g;;cjQM{5r}j43DcjuIT~g8{*SIT)~cJ5F!d=`A|F6ZYv$bP26S zfwdfnoSA~T4s5`yySr5EE_c?vj`iuzdM>lR5Zu^U@~y2eb1%E~0H z(q5F4>64Rbc%5AI&MXF}P+HxS{Mt%>eKNE@6I!3b=;qfin98H?%w*PQ+-s8)Yb(X| z$?(EdAh+(A_Qkz>zWDTpV|mSy>!K{LA)F&5g~2%@V<4POq&=*{y;R8_69m43rr)&w73l5jQ3>C=&pM;k;3vdes1#TIZ3-vf00a;)ljBbM8ZNpFl zI?j4As({L$DWkQg6+qQ*q0d(=t+*wBfx{AVSn?fnuxOpm_?PnWg)=T zK-}(Al*=*1!R@Y)$SvJSw<@>a+bz9UpT6Jut#h1IRrlNa^hzser#ezgYwhRRdq2}j z^lHUGCLN`7=S1o_Tbs2r{T5wp0#u@zIc~;d%^J?^NFy6>w8%!I-AuOHi$->yB&G{; znk1+2P7P|>#4n2^ji<$&#UhzWg7jvs*{ZeLwRSe%CgN?f)~4e?_6t&3I+Z261cMbx zM>4g3sykbykJFJDl0T71u*d)lsqV*rLP7VvRP#IOZbPk^=ZleqHJrt-TD?`P}e zuAr;R__rm&m0|M$0qThJyAb5J`3IY5Sr+t`&I8)>&^Q+L9ES(TVa5P{wSibU0y8)+ z7hw`GkB$eszaQ`vQRNx7r^ADE)RT_p)1gt??~1TCzyyMNrBwh)cXRxBkm=*%>1HB7 zT|bfY;}nP=qgWbci>P8`>eSIB5bNa8uF*&F=y^Fed+UuCLsTwE#%I|~qFtLI8`g{? zSp6(Os?nqyjcha7ZY`SaIW%Yxa7^`q8m(G0TWb+Dmik(j z0^-?BocIF7R)H=9YDLu|@oqnto<$O|;_SRX3npgK83QaF#-ANF7FZa6RzAb_qmuu< zlK(rU?t2ISM+g5~neC0tc;^sQmBYP~P~qA<7{q$BlshPF>&j(WtMQtRf#)N@T?%`( z;lMcP&jz-D6rIm_CQi(w-$y>Q*E7t-I zO7ikZ*F&PkE)F;g{<1REbA@^%Kj1b5%7y^q4|sQhCMUuXC4XHRvOA)<`8yrxs_3J- zGwfMGfS`F5W&~J;P}J#4chnJY31PTg)DVw(GRXEu5c%n~5qgVxx9_I0Rsv|1?`XZKc-@CT2IE6N_m(Ic+x9 zjn;YrwAYL5dO`dOTvn&#Gp19L!5pDedXc1)4LaGR8ZD~ProJk{ngkpY3Ae<8fC@Rf zhM8=|Zn~c<&Q`_abn!gG0E?0M{b<8N{NBJ~EPl`8O@8wxFMgkw-sKBVdB;Uwd7D=% z#wF3XWEv}%>)=+Ur#J>}vNwUAB~b zW`{3daE`@3!^VvY(L9pD!l*@jj;dN*^4TZW3ite=As0r_O}h%YUxRKV$C^`xVvBQt_nG-wRCpR^Ee4D6fyKQMWlg z^HZr=o9Zu!2$K_Aq|W9woU_w4F+XiBPMeL>Wb<_1V$nXG16c;s?CCmtdWy4|!TK3% zVmc*Q8OV8(Tre`wIM1QP7n@+6QwzTga7Mrea#TDW&lOYYem6bKrH@w;0E@iz^Di(L zKAQ^%9u}6uXJ8<^8#rzTr5(F#Z=l_`XjK+1*P~sU)$*}pYp|m9Q)LjU7&CBinD>}L z=gSLx=1tim^;rt_ywoxhn_ZhR5@-(FsG!qYc_S<8)GKxeuUwjRP?$T|#1Qesjt`F0 zVT9U7t1u2+7x+ho=PJMy8>WKe+RQ>1>%^*@?e<#nK(iJ?0xS~B`w^3sX-c-YTp{O( zcUTqOzj?U$d3$unQ|%bH*D^lWQyRklP<)bk79g8GNg^c^M=S!@o#|EI9u+FkF5c37eI5JiSMuQgBYG?glDzJk*tPt@$T%l_CEU;iT$VEpflo=*0RHRO>d_>os zO?6svKNTq_I%vffjyD`>MPcJ!F8A*qE`C()d~Z5Bm^|zVIM+&}WZ)7a0+>z&D0m8t>Z2F=zpa93h3L}X?KBdB$m^ElN_baMa}_0prA1r~KR=w{$^?Ur3P zfLr_I9_{qT_r3ge&%Ez8t9*IAGrzTuwjQNuELCWS*IL1~VWTih^#g}#P?CE}le|(l zR@&534X77X9(dd%&f%qLzbd)j)$bi_Z|^jRHyiFnNxnC>DeQ%9Z#0S4g|VJ=<}DNh zjP9m8 zQtkX?I{K^;?Fi*ZrJ>0_5=MJtygq1LfbaxpMOKl@k94XV`O$ms<@bj-e^XU|Fde8l&wzDvw;jyJ(hJ{R#L z=gtVwTAe1l@c-*6HJ=j6X}h^@HrGk!0A{=nh-Rp7p3T4DbPmgaXJYY1gXa1aPv_@5RD|+Y|t%e(4?9Go@b3(vl(x{{Y(EOuuR``izx|;z(7QdNx>n(FVC!FMhnI%Ub>4dD zDd^3GtwG9hS>?(NZ^JiDBv<<{RDK z_u_{Gb>-45JS?j@0kbA-Hy-EK;sLhjng%Y(+1dQ1!zvEt*-)P7btG| z#R&cn0kX_VyrMHm{Db#mjR6*&{L?Q8ALpFF3nQl+EY+iWFpwWxWE}nd1|T7DyV2ohur<0>)mg6zWwIu z`@in~>3{D2_V?Xy{*F#p)#tx*p1$E<|7hjzx-_>ItElW+2J(~Oz&0yyA{}Yk?|=ZQ z;3$3!|L@5}26V1YcYSo%O?SOyH_)9qLv^J`rh)(GKU_={umQB^zgg}d*d&)yjY^BW z+(_U@!5u2PR)IF+bh!K@5QOXJ0*_g$)u6Ktl57xUqfI7T5kvEddm-{zwF24Pt`u$(N2=KT6efiUW@BjQiqR;;! zuYO~^dQ&h}4Yqx|RPR=ELU~QBF6j5f>41z8;M`D~nKRG+zw0DZWir(xyB)ggN5~=S zbYTpEY@!!%ulblkpt|7U1}eG`^Au`SMXGH*OCgA&F4WWII<3rkgG?mb>3JhQPcrT~ z+oI?;m1)y>L{TjQH+h;>f><=O(`K7_7r-;&a{8+=d~rq3 zaqp$Qx9QGx+P|L;U8SRkeg9$1y&tk)d-Hqc(H=~pUb?b74lOdS*|8ieb%xSViAwv5 z#B{iK_4N7j&7a`>Zx{=}^t1n``osU;`|0od-~YG%^MCR#e;TR|l=92G;HqBWcC_-~ zfS+KDpa22lv2iBhp^5`~RVTYXGTtLo9Wv#oQy5y=3ztXsl2EBL3AdP6dS&Du*)W%a zqfp{o1P*@LEDfQ5HEQlAoJ6slehBFNN ztN`(#z4{Bz?bUPpPOQFjT;G{FFZ#}lqVFMv`LL!NZ3UMu3_8fa1w z02Z)XG-?PWwA(eBh{vgJm+GbfIu)l=H6osAXQ(DkCJAEE&d%HIX{$ADLIxI$K`^KF z^eg5`mi#h``!OsefuBVr=N63pRL@gYjSNK zKKL9azhQE27bM(LG<>1Wi0a$Pv@_dvw=3pIR9~UAoXVYJ<{UVIpMJwk?q>NEBf$#(Te=ANBxmqTYsuh7akcrf& zx9JR-qq1=t_5m3hB#|Vti){_0Yf?t4}giOvb(;SrCJQwI1vJN&3LPo z%Hq2R6yyABsbAQCHiF~&Q;Y=`$4;O+b5;}1orLd;0R8eMoDh&G-_ObyvGUHjuNtmD z1uQqc@vhUwyxA;;8?*IkEZiAy_B@5dfpS~d@+@i%Zlf|IXNZO#$)QQH_prBnbo2Ax z=Re)Q`R4xTzuaT+*0+C=Jbgd8KG^7YN|o!eQB{|2(YqA(7-v2!T215UW`rN`{B6g* zO*)h$f(_yr#Xk)qP1O3`S@gK$>sa#c@yKf#2kPcf*%HL0Ic8{tMci>G40B0j5SiWxdh5nZYsr`s92MUku!%u(IUvS1v{ z*IoX+*RvF8XP9TBa_lb?sD{YRW~$Z75ttcIF^=cU{Xfn$B(Rt{pJvXRne#3FV8tNe zt0sIq%=Rw>mwn%5!go3IT^>6x&g<2%{U$Vc40sNDjjp2Oh)k+m6*PEybA9*Y-kU$&W6}!7<=y-2&> z_bk-G;Luh7Je5AKtzw9M(%m4Lg0DhiCfLrLH{t)=ktuS)Or8b7n{J!z)A3-f8>;0P z#2G|usaUO+sDUmrmm#{DY@C8mYEvv|DoZ{ilO;1*nn9cVf)*2idj^vrK==V$zlHys z)`}n^6oqSlBHm5JQ!KCsQeOr5E&snt06&DY;JAK)%TREDOVVz@eCHj%FOR;htADmQ ze$GpOG;DqzA#!SbujYT&+II7s`$0kBQ*+DOrCmKiz0lY(MD@DZCg;obOZ`6g>hfXl z>9BgRk0#g0w`jJ%)9hD;qCI?4Myq@9Y$Q&H9KB0HyBL)~-5^|09ReWqvmnl=t;Dny zLDB8BSxaW)1fpA!S>!xeJP%`ncP&>WGr268%hG8C-P)OB+&tm0TGVli#uEVkXSH6W zPS(`fv^H->8e}XR58zRUX9Ym5%2cjLrSM{o(o~G5Q61W+3FNWTbT&s5K$j*`0L3Ch zq*%lu5>RW`SY+aDfJ(K=E}p;bb}pMulXNjtW9UxhX2{Bo)W16qyP~AZ!!GGrtSW`g}Ue5ycJI(zJ*&IBD}?8pw>mR(F(V{W;@j+(k;4= z^gJ$}$k5~R-^fOiE!_Xn71KRgr%tACSEtS6IT1+KoXwP_nesfNg>|jo%#@q7w@C(? zWT;6*n?$fhoVSQrn>cQhvn&gOoRMS#U_~lY*D>?)qkrHF2qhxRY zOoWQ}Gu>Hyb(~tU8a&GYC31}C|3}2^PYebOf|jSK`6e{p4UBest~;Odyj@X$BQtQ}d z)hahiu}x~^4SAfQT(Xf%q!o#SCsAu8Xe%&_mjhX)tkwA~!WgO%L!%VBFVJZe4<*TG z_miF@x7$iVfHs{YGwDpK2;hy`%65~D+M*H1MF-cNCfG-l23xJWXm#g})OvvkMDF!^ z^?I5(X@t%Q-)UyNj=R=1K+PcIT2rGnonModHDR9;^(o<-66HD3TM*s_*-6r!1|4Y9 zffns=)BY?SBIqbdhDicdpRo*aTq9<2qMss*T{4{`S1U4?rgO#2Dp5<%Qjz2CTl`5M zBNW=r0azT{pN{QS7A)_$4Go^oEl)wuP1ti2&F}gRk6zcEyL8}mydOw^&ToFw3ce}v z|16vQRV@FHq3OSj+y4}7|2*9OAl!a0=Dd+r?$nwmgYD8I+^@^HNLB{SL%&_-ud5Lf z?zlau(zbiF7(%6$iA!QZNyt|VG-`eYBWevpdEQz^8oo18k-A23H%)*W52zrGm3*7A@$%J>-)6BfNS`djw29-jn7scB*hj$l_-qR`e z{_6F*|8XAt*l@qhnqN}Nml^5HjP)f2h+opymt^53>3B(~UlOj5#OO6)cunL_2=f`~ zIi)RYdN8N#3#y)^>P^boqI_+-oTYmN*&)dQMTTl*I8L6Y=va|2+4cVPj*&od7*5BB|2D1Q?$?0U72<Xkh1+kf%L7McSFNdzjkg2o1DeA8W@EVi zRvL7>&I4u?#08^e6wa6J`9a=gb~(lltHCPJ4s}|=1kFu(E9MEI&JMY?VH7or9akDv z=xEF9hEREqR0`fvC~!z-bB!z`0Cs&FzisUO_#6Swb>?(hn>L;WIB#Td^Tfp`h13Zm z|7Yv?$vXXdJ^MHhy=*w&wGE%>!Y4}ji5kA3hc8IM3v&2^5WXP9FS61XS=pzo;{&1m zK&anA3{v}=)}GL#Gs>{0##3s%$mEk5bA$4<=s}ydXK5!%mub3}p}cV_kjnVGnNY44 zPS=9Pc(9*3pLOwJmDntIJj*fP{a}zwga`LQ7Fa|Fmr=_lqHl-=ArjXyR1p&`t9R?l z$3fxFZ2deEeycYADAWC9<^3RB{>^6kvxfWEiT*Dd?l(H_PkhObg6*3P=Rhjob?~Y# z{+@YrSa)o_qe;j!3_E4#m=_afisPbzL8ojk+YR=-%j1&e(cUCBYK1z3ScCZnX4zz* z=DYJkzf%@Ni%ihdP2g}0A(`7CBTX{hBvvePO(NG~(afT{q&rXIg4ta!P@5ZJ3;>0S z)Y}vvh{ zt8C#FQFuYfJ`s)&r1Bl9d`XvH)1?!tbVjMylzN(R%_;4I8Z{_GlNz@&=4{48WGrND zK-KD*xHBI2rBdZwx0lX&i>p9!JhRuO%_}*drR;u}N!~30J`6r#}2Ojrt75z6n?w=;y?=_Nd`P=sb z)7^%%E0tH3lY4_;p9!&*IGwRBY8S`Q{1x^jg7#Qw5banPZ&Wwb^Cgd3YLE^`Vw*wA z9~X4!GO(y^zWgRocLXtM3F94R<=QGlps|-Ow&+NU#u2+>Kz;%AK!j(}Oivp?WZo*` z#RcczU}Dc*1U=Vht<%@-vy;r(+3M_cb}|ipOgi7S#-G4Hy?I3nzas^|CyajtvVuQm zg@4R${+Ja5zbCA~cZBs7DSbgoKhV;5wDct{{YXhqsKOalIL*k`8QC=Bm}iv9nz9iu zHRI}5>a8oA(h}Vf-5qCg`S{AyUD;P@XFB3bN6HL}ZU&KZz_RZjTme{gJo{eHQO|r` z9^d*#_jT>%Kz-qHRFMxhSa)5*N2TDP!2c}Pyct%0wk?00Y=0-`{%NWINzVPg!u^>i zd83hhwr%eOrrizasZhC4^RM$lMZKW!xt1Y|F@{k-;c+BpNdyP6=pgL32YR0JU^Fnm z4~a)ofuUeD$ZTT=-=j7<(M*FWV;DdgwV;S4l@8JIV}dGX8AP%au9@i;K&P8@nk~IT zNiMFLiA4)N0;q}WH%Qc7rEw%9>2iuTjF0HYqs~rNXJ-(=|3v`8@ro3GM+$#W2>wXe z{*bl(Th{n*S;4<&hyR`x{u@9D|3C=;NZ=npB-jZY)$kKNd`AyoQp1nb@O5T*QX8Jt zgr{-gIxd{1g!ArZ(cLt1)@DxHUKNP6gHEe6kx{&u?-tG7zK4aS+Xp;ei>fnVfJM)9 z==Lbe`77u6s6M(z3fNM*M<(7-*wI>_O2QkNU{}h&+h`61mCyX;n~CXXx#asL_j@@P zi#HX?=LyQIb=ybV_Qtq<+HfvplRd5N$ShPiW!#Qo60Qpqfl@JSNJR6;QO|j35bAh( z^^rZV%$KYN+0Y;pj4|ffVe??dL|*ExtD_i28MdRGE0vB?wHZN0SfsO=^e-ATsWj@V zaMmV?#3FmVXdfpbK(>gtaFx=VRmaW%KkkeR`5RJxiQTqpy?Ymo4)LqJ(|@ z9f6%b`~v~%83F#C5rEZz82%d}0RF(pBltaB6lsK`dO;gM(Y6ni?Hy%%nX!GW*`TG)<=iAaGeI%<7&yzh$lweiHS@DB7HI~TyBXA!wQ z4?eTXIab&WR|D;lM}3`F?ipnFTIpqJvm+Z;rM9PyZdbs2mP0KTCFAx9l$o!v_)?6uu)je58D%_UDdotmr z)OfMs?+7MO{K~Foc~>#LUv448XZiM^vVE*#x>@ScZt^7|i($Srv}nbsVez|}J5zBM$fbjL!;_#M zPcNd2#(ZlQJ^kZv!LQhdE zd%BuVXQ#=~Nz46_u)HHjpGf5^()xRH_(#(C2L{Zhf5sme!x%EVpT&0sYXIag{J-VD z%={|ANs12v&T_VO&Ng2e8)-o+GR#JVWKl>Jg_-`Q1|)=;-=Jrgd+R&^3-@4Iws4#t zQ#~)SkM)+(gITK@s}D!YE0=7)WId8?uBC##jqP5bxv=r78u@O8b0;?)$W8ctl$&s#VS&2V#}8B;ufsc-VTN=ujHf$ArNMq1VfA(#m6%x)MIPO+0`!}iX8};@hC6=|Pu^rR z1@;@Lo0<|3U~$r#zi!V@NTy!-v^$>`*U9tKR^Wv2eIz~aXzdGH_8l$!BW(mAz!xxE zAM7)L0PGFP_=+&T_#y!R9rX+Uy4P_X_&@3L|A~L1I3$Y#s&CBnZM8l>li*Vc9X-n@#q-#A48Uw4$+EHBuhBWQvk>U$!}t3T`*Hy`k>yrH!|v zk>6A};QcH6rrqThLxr9=`l?B^!+F?|i>f@)jYhO9kUU8%moD42Sqxu+%)lAY2OyXD*$npi#hH}vvn(aO7u=h z`%4EC2GEWS ziK2ik+UO#m>T5Fn3DxJ(2_8AC09at_cbpS|wSkIzATL{%KE?*N_A%Eo(&t?d2IZj^ z0$BGQi~-zk_>2MYc~6?db&ZXb5GsvXne6X)TyvJvt-H2qSFp$l zletZEC2XaK?KA{15)nQT(GU?HQLHfNSF#BPSOlD31i+#LYp_?m1G(F>EPG5$>6L-& zaf$Lu)#!@=w^HN%Mt33LJ@Mtc6V8Er`yQ=j9LZ-B_s+z9WO^$fp17*2NU$ zhp#iNjx*GX+5Sy>fz<1|IiEKd$rjU4(abU#D5fTsDHCU6aLL*n?Gp1^vk}j>Q#9G7 zPz#7^G-Sh3me#@v@{De)wQ9DDjqD8hg(=xNqwODQ?I%k5Dl>e=2*Cf2(tSs1zN2;D z(fsdd+be)&{ulmVP&$@>&C9svV_b8R(wwGr)2?yR9X4{}W=`B%ZQ3hg7W}7;*)%_! zo@CRNY($iPQEQ=EWZs&u8;f~^=_g7y z8;urY3Jrp-?Wa&-l3Y-WIn`Le@qm4#8q+CcTv2y}L?E4}(INp5$c(JAM7mA%Tg0qE z#1@2qO*&uGhIdrqMMn6761<}LufRSv`Ht4SqIIum{;weUFK9m4e?e(JWi%gZ6A1A7 zs{r$^F_{~(?2Frs05AYSJ8f&Hb?x+|m6o?6%XU%Th5-Ev3kXp0v%n%ymv`(-?||d> zn9Am@Gtae;^d8s4Sg9Bl_SNER$8gVTyAW!s#>%d4c{eeA#{Hke{ldRVzq{4L{V)cg zXAEFRb@X(2yQ{c|4cu*y9(jkmHqnikbD`87jfD?(1gc$?ff`-L6PADexMUowhk4g# zkbldb0Z})~w?QI>J`?0x7wzP{(E!j)lmwcMWUG;EHJym$sR`9w`VWO(mtyqC2J1UO4+)?NNQXG^XGtnXpmPb)2Grj5-sznRuF z)01YT(u~MkMR}_aR9MIZ^`+k_2e9baIe>Rya(g7cJl8qa4@M6j_2JloUVx+0=2j-S zwDNa^lc#3^c>f^4Bjf)efQa$``ojy+)y2b6RdxHMxZgz*QgXB_SKR0xo`hRGkB_th zm8Y;BD3=~|dMkMbrDmkmjg&Tn(uiUf4E`wB4VVRk7Lr`9AI~5Wg~PPnTr^vY28(2C zk!&(@G|^VknB#vUT1gtNo@jTGOU!j=#bPj#i1z#8VlfNt4yzL#VB>wyFKe^!9%Rfzp#>h?x*^u$!`tEvLYg|u?5wjGYeTojS!Whim4j2siKL#IV^ zg~Nt&uDna^v}glf`|-Iy5}9>V=?uoXwX$p#0;)ff?FH&^n5q!eb2b<>Tc|E<&_t_7 zk-2WXkG`wpnEyQ3iD9}=tTQ|ABxZqr-ybPr#0*3Y$K#l6a!!)5b~c>VxsKnuV>*k<@AC*LvPy`p(9sLBgU{wX8>KmkkOWk&w7RssKv|IhIkD-dpL6^%eM zVrxYBi?nW$)+{2EMTD0uR)7Y;G7s{x0O6gwsZ)Q;Q12Zu?3Wq#MGX5Rl}D`@EB8l* zYqfaKF}#%8b~c(PL1mY}ywjKtDw6kXGxr;l{<8`GUk?kYKHq>C3wRdbsH(WdS%BM6 zm7}QgR8&n`aHpC53N+IRXjHofuIEHivWw{=>m1l?FB)$G8EZGc}XN4J3bCYqrH&d7Y&ueQOt&M$Ig4l z=nkKC`e@Khhq~QxEfb`X5zdAiL~l+E&dAYgq+v)aqHLd70bbC&SD>CjLK`t4IEYV`&E--2aPLy{iSeuEz>DnLy^Jy zb~{29DFjeXv`Pq|aY;35@nlpm4p7-+H-tO&S+Ijvs$4DsM?{e|DofEVrj8S(gQ%u$ zt{chIWMkSQnpB4Fc2n%$KMsYXy^h!E#m7|fInO{HV|l`1-}yOa^bao)N+ndUU} z6*jwW5z!@!8emb>%p;R&ng`4y6$VB5q9}&|$^Npz0t;szq;CN%5PKLHL`?X>r5cwG z4N8T!fbhe$L%1guT#ES@!%5X9ztNZ;d6GLhHofS*MD;EA@LP=V(5nvgsyk+|%Hgg` zadUX}q}T_lSHMyAaGyDnw~7lnK3$~0A99YQ8bwJU(uyToDfpLz{}K2vOk84~T3FGF zb(YemY%Yb$W_(~ff$v)N!)RrQqoxvdCLF2j47;W3`S#ybQ1DX>48uQaTy(aehz7QAVM!8%<5rV;rZ zC@vQ)!22)2L(4t}^~{UQGZLA#4|%o9P&(8)6t2R)62TwgtyFN&u&>#%Rh~2)1lk^# z@cE+8Th$w|uUC94QoIoITBA44&#G*!*xkbS_n{@j7kuIr`}Xb#-(eBoTR`vDg^AVU=N+%&OSbITZ|%u zjaONXM%7vkl~UOx(L`t6X%h!9MR#%Ii3J1xj?e41mz^H_z+kr+ob_?pXYTbpoz5T> zf(Fi5tUOEfBI!Ub6^&Ep6dBBtod)5Y66RCFbwU~waGd@nRY?`BpG2aFY=~E-nv*h1tM|=Y56+6Tt}AE$P)a|i*GrLB4^R( zBw1i#*6KYYP|w98?-ChGU*bOM!WF|lA`inovF(1SyD;*qy5-XZ;Wq9gM|7}-+c|v0 zuz&UK!_}LItM@9!=fkT5#ns)_{^S19?h%8Vqy48N7{L|ek@R1T;9e)W70CCkxjc{O0SXq|O~yPWrWJ${VGiPnQb-|@^{jC5Ar zXex6~(LrR>o1}A2cuq+z1bEFPS|EV@3+U%X1^fPGhGGA8O>-94tW%n4S2NE|7HP1L zBbGNWR@O!NX;FSwT%JXirvTVbbJh{gG{Tt|8G!TO@^4Usb}oR$s3g)lR4&q59r3%mxilCnJqAa9gpcnyEM|4xt}Xk6{;@K!%DEO#P2@PMS|i*9l$tND06-PFV&NgO?P1e&r+Mc$i1m zZ`ju?RuF))&Bd#{m+X1gR7&v z{i8=__h0W`|AO1yqrJ!d`^T%xJJsH!h|1;o)DpAOVk{!BO8jdYOd50ppsaRGtU zJW@Mqu>lhtwQr@Kyu)r$`|w3ocfQl{g#3eGxE_o7`u$#d73ij-nM{bFdQH0iRRH9A zjkw;wt8w0CSUlf0GnLoyJPi8?_~6O|>NUwU%YGW{FX!wwUq?)*5z|>@i{EG9zf2=w ze+w-743f`CFpE+_2O~xINI25hq^uEdoUYC7IaSvxn=9tD_iBa`=Ps+{e#h@kJ)0RC_bIGPH;3(naDdBPqN2N{PzzjBZ*0nwNxZ^eZ{g!al9y@np zc3~va>7pVu?k6(tCS{+~tN~Egj~Uy$n*2k}^bW{yUS>ESGfP}Hnb#J`XD2Q2zcy-a`Mf=y{Bpv%sBEip?hQw@YkIibstzQFpY@8* z%mI9Lpo0J3zf7Mb+&{g61`5RHj&>2Lo^UVWhorVcwU~>Z z`_TrUP8N-9Dg*y8Z#(oD4+q9W=TKD^>%9eu-@ysHcrmjrQ5WMoF({g521$fQk*Hx7 zaQW*6r$I25@w8UCRytu`6EfSVG&GN_7^>;3t9#{9$2$%M%;AtFhS?#ra-`_Z<@`0& zh1ULAfDeoS+g~(z7T_#CVcouNk;B$!*vI2G8Idor_0!vRS^}J=xo0d`{v}`em&}U{ z`Cs^F$!GXaNJ=FP|KtH!C>=|g4D(pGQY+Ug){BJ?o1sc*R0wQWM%_Mta-^%=PL}sP z)1`d7vy@ahT!j61x6D|%UA^k&;qXRv`1a;daih4pxxx(y-X9uZ@82I_``=!EzPcZ3eF1PSj8ddUUaYEoL{y z>cf~*7`DG12F$kdNHZu^JPtWzFgrA6CEubp)(uRzzSpTl?m5KF4ndNCd>r=lkL~Gx zxx4b#QeL9w`yv2S`-Y+)+l$QdRc8AlV`4O*;Ds0g)~U*(%S+~Xja8+wk~h-JWSX-` zZyEMi+|!l*SA13C9|hnh``kvK@jXozSh&W*ky#92Q5p%9TANJGFDNxq2hUoNi=~{6 z)g;_(heEDks5cr#Hol6lIn+$9CY61j{HU_Lm2+-6rd!i?Pa@gFq)}$fs0v@(sNxxR zb-lX}0a*5r?hp3w@2(J;zI}W^pZ3wNWdDYvc;Y=&`P?0WX=k&%kjgJ*yi3$_sde{S z-JXGu{_Q=p@s`!#%DFi#OCNd;eMh|wjX2S_7jGEQPV%0brpuTh-%@(lB&rjf@N>$S|?WD&m5CV_gcxrlD{v z5O9q~y^a6C*Qs<+g2yOPk?${;M;y-4)^shwv{%ukNOhr7;Q4!_xPH9aI{*~7cZb)H zs-xY9{Tuz&4fpB>7O*_L!MDheC^K@OhTA7Grz*t^w8=$DbD`Ev@2=dLl$$_tIz5;Yp0lj>BO(2OybJwBfKM3?!agr+%d_}$ z+Fdqs%jSyHSeY8DZE_`9u;i~9+8@ps{6>Pb{+a)2geAYokk1+b!+(nf79OA4;?@FK zX?WNm5o#`6my0`lVJFEYcz6gHjz#Dz{oH;obVKt z%9UIW8CYxJUNU1}G4u8C=Hcq`;p$GLc+@L)xra9r)y-CQ!%^KVA8sn5n~DC0&%H59 zZic3t&GN}AN7}gR(CjF6sG+#j+ECeen-?Bg#0tAr<&^2&rL9-H^p7hbJv>1FhVk4f zIQI>M^&#Gb1dyNIjJ+ zQ}J>(<7|+FY1VVr9(^PV@0g-1IkWe139WHn#+R>C%hT?1o&)s^`^nrknd2^U`gu+? z&572zhi3s;1NdqKUj={xFuVV^0`$KM(3jLZW1rs)U@_>8JY`qjS^8oR9;-UPDKYjS zD7b9WlFsVTSY;Ejw6d|vg&WRrxD^<=MyT*?4>|=r6OVUPkz=^@J;!umVonES2bl4% z$de&e&(S|E^{7|@ZYtcH33pc~*|lwVQN}pr>1=@+6&99lD(o|I%f=9}REw zn|+T}WtZul%C}p$cG(&8LLNT4>Cb(_m|HySNE00yrb`_63Nd#fSe8be){uP@7@!_r z7%p35UPmIN?gzESb8Rs;DjplsiF~f;Nv-U)l#iHfY;f=J3_$5-5B;i z#+N7EA_?&HHB?0^Mu4vkQ0M^}y)12yb1u!!F>V*Ce0e^WChq2%6B)F8!_*xps6}+7bU^HR(L%YGkvdirNrr4x(&RJhaajBgJp5dTS)ko z#jrXO)JDSEVpLlUX%l|+amR7)mPUQz@bkY3DI@2u^zmplGprJZRezj|Qkz9>hQFC z_*H;s4gT*e;Hv=Mb9>;}31Hz4Tb?zr|DPI|t;+GnF%rTWWNOI3m%&+rCbBtF8GE0NYJ%99(8Bq+_51S z!#GjAlC^!m^0-q7y2U~F28%?L5di;eH5;w^hFsCmP3KcNPc1db)aoSVY?Jn90Zvl+aPvQjGh2V2UgDm=NShX{&jAUulfHMN5F3kz#ANN zVs!uuci2-7nVo?#_#faYx-cM#3^4G`I`8s$#{`O|QDGnC&5 zD^HuslbBbvYIbD&OQrGFH9Rti6`q2KsUkul*H{x8*n&Rc%xf)%Wa(h38*{}Iqgo$h zRM5nv#a~xp3>zxUyyEjRtYjGTijM<~2F0K{5_6?z+Ess)ix^TXbG&QG#34W(0yG#4 zu$;EWr&;A0DLly_-XLGIZa9s<<8qd`w=DZ9)#4xSHYOTmR366=Z_qaj?8tc1JjL@KLTrAx8Z zBo<3H8$Bz5N-$Iy1^YJRHJ_2-Uc955u`pJ41oG-o{v?!x{Hl1e zW7S+pQFG_8-KqsgS|KLFJ`7|MpPJ(}@cbTrur8SO#1VfX7gVN>OSOcHj*I}R@5;fpU1RUh^+!9yKW@OupNRll3>#pcD^}vD=;0*4E@j(Aq0{rF(;IbRVKGeIb!P;Z#G7Q z&BIW5C>UNb8rZJ+x;@?GQpdaG^Dg*cTz+8$Dwjqca1RK02t`~BD?7r<&W3j()?7+; z_cH#T(ue^>3d2T)af~Rbc55qvye>E}&dS0_K-vvDYR4t2uO=g|Y;KgzX^DuM?mMXC zLOdv5`GpBD4rt6B(m{f-TNvz#&pXoNkgOk6Mvm31zP1|~$8+XPd_ZRG=qqYu!M}al ze(RZ%>Ul;yPa)cXw-%?F-nJs#Rz#m%!SgWepTz;ySqfkTI0O4%&ftITg8qdCuxiPTY% z%9hpw-p#BAViBXk=1?GH?SR#QuerA&oCnfDJ;Od?s|fHh5KI^ihP(?Q%f3~&SK!|| zj7Mq=K-;LsR!jlh;&Fw~9fmx%m~)8fDXUN+9#c|@5?NGd)2?<-+s?VNkrJ7ZQ88;g zEL{09WQ8B|X~K0>!thrc` zLAn+hYqHD7TNfApR(jix=v!$K_{Xse@=qA3PQX9NXZTkE|9Awjr!aE|(yt@>e?9_O z+P@e8_z#V}kRbdvkCoeRC zw0xObQz-F%)j+xxOSxjJeq)6UD#jE9S4P{B5#c=Dtw3`tU>FD3wdPvD;C49K8&2+p zntQQsPiniC8IP31E7wM4km}8{?V!X|ZgtH3s9k_DTK$f76_q^;K=;)IEFi7TMqFfH zNgo$7=i*dUxat`DZe7ed3EI&OqKnpTQKukWW?a$CFD-^;>Elut4HD@wBR~xT)SHAe z*>*14g9WKgW~>cX0LB1Tl4e@pSUt>haQnXq@Bjf$x~dbF{j-${I86f&EJUYi(He+| zra)0YFX|V4eG*9M8wq_An6XIc!8w@!4f!so*H!Pi>~7bfthSWZ9$(4qRDO}bQW!ly z(%9DlB#^njNKjD9VFpr|L1AkxNUYB)Y#xL{)li@q8n1>-Q1xi2yMBg1cf{iD*Z-us z-RN$uw(Ej%-!Z%@Z4NPz6vG%jrDc7@bLVw|0sOh}7)LLhf0z|uR-zMXvhN~_U|&to z9GP=zEiCT(hiR{^U)CJgdFS>Dqk&x)vNIZ-yM?okxF3?H&mFmhD;_aqx*qh=;T6y# zoQ<|G*|sNXZKEb_cKJ<2d+_ec;m$8w(=!=p-Lev1B*`MjLa0f;`Xy7+sQSP{WUbVBQuJ_b-w;BR48u&`E0Z(0N zMz~@iGr$HsR_39cXZ_H~Ce!Mk%>Wt{980N;!=PZ2LJ2EZEY@!}Y)Dn05E_}-wh%J# zAqH^8f)NJ)al=2d8utrBg=2H5#xz)mB(LTSMm%R;*Bco9PPlRFEKrC7k|0h3O-BqWe%fw>0L?Y?FA6rQRnZ<4C4v(9SP zsjRZfFUW65PSc;;a`q0=8U|m?E8*l zU#will`5+jocB0|1vqBT-dE1@^|OiEdM3aj_iBXBxM0A4UziK9*2&&?Bn<8z1bB_v zFT|aI+#~MdP90Eo4$Jo=1>rD_a5ew|)>;_^D1^kI3>xgKP^j#=7+DC&LOmq-0TrPO zU<4}Qpx}jIb-L)cYF%EXeC}OK(7P&VW`}2sR+nk_x;5Ch3a3_w&FYw1;hHv0@K&V3 z{Id4!-m>bPne8(}gB?%;V!{r90#aq?mzFK~{;g`AtvD>sAit?O9(LU}zt*qpE{k<@ zqB2Xu>24ScP?`(C4qTwY#UOZ*an+y_1PISK5eZOc`h&VH*_}qZwipGc&G1CdMkKJ4 z3H8#VC7$59qyYT0-0mdhKR_n}OzVB9m4_=r9)tfg4z9axkEBK8ge!-Rxybcjl7Iil z$iFKa@gmL2N4!)>C2r?ksP3xFw&WFV~82cCaL`Zj%*HQ&r> z>8;GnRnP#*@C>u99{^w7-*>NCH;bpN6IW5jWfZL>yEqhpwnxGQ_CVyOc zpO@QM0LVax+2s+OfJxiBw1}=WNKQZjQjX;8K-R&D=cVhfGgd%1V5RE(Zk_b{eZ{Utqh{)E z&+Rn_i#slD+2vUmRFmL76@oT^!)0>_Q-_0~>1`wdxk&oHtdW*`-|C8`SA{RxkPPru z8&(3u0N=;%54Ql=1?2ws9vbe4-}ZY>D{z4mvNzK!$+Pr68ISU}NB}qlz$nxK1JN2# zp9#V=Rid*oUP@RvbO=~3hx+Nj6bay;Cc4uwWeYt4XMh>Oa> zaVVePYv&n=fJ2rXWjBTrT>E^ulQwfm05sSHyuJmXv=5Gtd>>_$eoUs~Xi)cemW$ zonUbzsI3P*CYR^brP^F>n+wYzxC{**to$pU+$4SX;G@eik)hetsc;dQ(L?Np}L!}Xn5rI(Rs%077%MaSPP;@?_(CP6BR`HukqJ}$g^ zB<_?$CWRp~aIn-+&>NO zk0PL*K|m3($IE+G02D%dWJN|`LI{Bh21C{X0n#7vb?ZK=ZrQDa2D4(fr%`6mK=m%W zy&LXg->sS4p3@*eJq@}~kqk(KZJK_d`9&+a#)Afy6>??=8L$#|_|h3buA(~tfPDe8 z@1Zw$UGO^IX~k<2yBmqXRtfU<-Ygy_Zxg%}zo`=j|7Nf z0c1mf?MR(S_^12b>Av5TU^XJMIkRt3V%B)Z&6VL&M4hcwE-noI_ul8>nzRaM0dTKX zF1wwRUge}UJkCe_7mn<&;|K*n8GuvHx6A|ifG^cy8VP{eM*^GyfPE}L`%wTWgk%zU zM}UIt3(1yB2;gH7NCs)(3oyPqAmH-{EZu+wYcTWrR}Bq3)Rt#=$K&1bc-K9}zN%q( z`fk;PWN=LdfpE|d4(lS#Z(7NFtIx?>FlJ7E=CoRzw8aU>UBP$nw!)!z5wlM)#nU}% zJz8ImRE&{}7EC26t0AT(t!1-5%W)%da0+OJD z0Q?6SOC1of%q0jw7=Q#221tb>WiC=%+U~OE{Q`d*S`opct*dI^?LG|#Z7zWj91jBh z%3OB1z0$>dc0osH6_k%SM009CZv>FP{FrY98*5FZwiiH6%v$+P0#n1S<0Uw1lm}esh z25-0L?V7#RmUkDhQ}pf>i_4m};qmlU)#MJsZ5KpP*mgPK{+C6UbrJ4=5qN~4^_AQD zN~Phq_#SqH1SB2?GXHLWACbS4lwruSTHc@CMlDi|Ohw2Pcz@{y|5?QU7kIBR&BaJA z3NLwuw_Gl#L;Xx(TB=VI^;1YbvVUC;Y>BcZ8m40rcXCiv;@u;Ok!pMFWbYW<|Db-B zr$z_ra6;QKyV14 z|9NKGur0V!ON7aF_zP_y_LL=E8&8(Bi88P)*njQuiYzwv}k|6#9ewLUZI^qP{Nt>NP)i#sOgtoH? z)lM{lC6>1rV7l<#cMAml@4o??%!WPj5;6pGoVhT=ehis7}c!`yfxUQJ*&ip!THD8_J6!NuFZh8bgf z7kJ~!e6ugN($y4QzIa(@m~}C^8KLf|>!H@Ulno262cw&(hD)ihbSB8Mfie+gaU#Ov zKU2yO7T1P7wC-7*3m;Oq&y0C#1%*8tZQ_L?U#q{dvaanQc5tgqUp^HT`(44JU`8Z!AW9-ZEhLoCOMd^zYmBRQ-)je;PlyH>}N-B2F%^ zFlD+XU5QsWh_TU)cy)F85p_RRuEyHs5Y-;+NB5%?SV_%0f7TY;PwXeDTf%8(;qiuM z)(9!*fBa&mSx|;ANrJ8Hp%9CEXT4v_N0gFa=91nA5kT&GjA`xJ2B>AinoF$ z`LoEcA`#TU!piU|ph>5|UAGI|XQ+GwtFy`tKavJjUEwl)DgG%YHB+BU{mWi6G1kPm ze%(GPrAsIL&3Q8gztUJ5L;3o=8Pm}+(y%d>PCoWZ!_(+_i;tqZ%6wi*u-aRV{Un^Q zS|?G}J7_z3F4L;P@`KZ<1ds7bddBFx(0J-o8gLWnCtWI}Vp8EF>Te)68{;;Y`2+pt z;5+=P+KFYN3cQ0J^u+Lo1$ArREvpLKB-YwH-I?deotnsHQq#D{7bG{7>!XY1CLB_P>=&J#TLm8rPk8 z=B8T2>s)r?stW67%K0}pZ-Mtpy_AG~{#*u*+r}~(xv5{v`QFIbW=xk0+=ug4RejR0 z7#)|2W9^Ce6MlT&bw>SK=^M5TjKaIN$u<3;FcLoj6C3Ayp z1)j%Z6a26Rqi3vhE(PTqDbc|?%7eyDC$3K=4f$&+A@ten$_byr{x|peR-&zUhHyt} zJc;?P)RLct{yVK}L>>;68VV8|L?>)y8i~M6E?32s9W*sU3C}}(vaD3#w083`9?zfJ z-}KmX6rT{sMV7_w@7UZs7V3@9yN}S$vK1WqxDwCfuJi9uH+cJVpFxXa`x>ZZ*msCh z2d=~u_6)Z)_*y;INw-^Sv<t2&ZSQrJ!@!FQkA+__1qogqJ!Q?*JyYS{be{|yw&?OI)bbIsP0Ie|?o zXMJ|^QEATiYw&4wK7#dzC{7yP_8@c9igVyU+-IGC3(GcQ5Z}?ae2vq&@|ilS*7TDk z+B2jdS6ot8SBpPfFOIvzci2d56x;0AQ_t=Yy9i22Hornn1NZ+X70tBR2KvX!`U<24=1fxMwVvK#0fE`U@ zP$3~CkUj*EX1Uhh`+VQ|p8MneSkH6M&wJ0kV^#r+wfDIn&v=WOS+gpsG2Z!B)vQ@> z=H3H=%gvU5a#%j#kh)xLy4=jg$mo&fe?BzzFfuaogn7ZX!nVLR!#2S-!rWmSVC!M) zU~aHAu(hz&uvM^?uobXnFlQL{ZR7~EhdIFPV9Q~yFdLXH%o=6|vxHf|%wc9QQ?T2gZig!5U#buv@S}*nQXo*jrdD>^rOj_7FA( zdk%X6`!`q(>>TVG>^kf#>=8@`dj`|OcrZTfChRus66`#z3C4nPVC}GBm=4wr`v7|a zy9c`qy9&DjJhV0h@;X3Y&t>z-D1TV86rWU_W8M!TtvOJM7S%32W+{~@}A2U%m45H%iMQ)bF&3Huf=i;w@@Q5Gtz3685ZPkw%PRmu)u%Q|N7(q zH8&=;Xl!Z)uQ4_@HUUsDH8nCbGc`9iv#>C?u&}VS1T3wr2&^sYt*x!Bt*vcrtZi*= z?Ck99?HwE)?VX$)0cU3?Yg2P$BS&lFb}LreY4Ys&DN{jJRDs-O)b5=eYW}f`3LxJ4-D85 z6dV#7wlh3*S48-3ugE=Kk-H_9~1!K`cm1!w&EMJ#i6adBC*#@N)_*a;O*P8(BJIe-p3 zJ6mI8BO7z$-Qd^;G}zkO+HJBcu(dT_>D`%y^`ILYfWNhCU6(IkzIE%?73+-M@edzd zUDucwE^|Z^dwb&|C+yJPexurY_p;^7*P_MNcYh?Uc5_>|e!bg<4IA7yY}n|wVbi(| zn^uDs!px@C8#k@q1Z-Tr(S4=+hLs!KR&7|f!foY>bt|^6-nw=53KUkjtyr~w)#?qa zykGqhyK&3PO@~%(I<#WbFTk=*zqoEZXSkm=+aZ($wjlauS`ofn_JJxm@hM30NlZ=Mk(#3e(=4#)cfDEwEQv7Mc}UDJR4oADwBP%by4@u1wO>Axz>fVm2@ zyXol(2?@Rc3Vwd+NlAWbX-O$5-abC5MMXZv#YHA2IYu($*R{ti;N-Tp*3be@2sd?c zaayy+1>~+>d+U(d+U3rsZXh33E-sh((NGVo;L|I?)4G-G*4>IYw+@A^TbH2;_F!XU zWo~YYz2lowQqu61)~1g;eHD>p1# z{rvyj4Qb%xZksl*-@JLl=FRTLrl&T!Z{Fy>Y2$`X?tnY6d4v1r_3oS9+&8atfBnDy z7M;)xJs`k%2z9SJpAu`e{A|f9^Au1{|G&CwECM+{E*wZrr^dC9m2NrPr zIXOi*@qrpED+|zVZ*S-1WDl;vCS3VHqL(dOwQB3imH*ghwtDqH_LYI&Wy{urQnUhG zU0vYwt5>gDzn&-fgT3C}ef|3Na8Ho#3_>j}%}q>-@kRJXPtVMNfIuKKGcZ0r;K-5m z)YKHL#aBZc>t=TI0S5=uzAZbn>YWmar2=Ko3^;QZ-G=RHf-9uVbih=8`o~w=t{6@&DLwjY(cz( zQ(%r!uCHHdOlD0?CWzl299R$&yz$zysIbDwsJ&aS{c(GbSHw}Th&_?vdxK))0s_j@ z(+kql%6xqMi;GK)jQw+R%DugJ#m8ra1)ZqMs;((LxijumMh1YwwrwRPCH?>kwYA%F zbL+CQDkHqIp#bQ|(I+Itr==b7@ks&krbZ?f7G{XOc6K&Rj*brSKkx<~!2H%_YeD0x zt=F8u?z(k;m(5&VV=f#25$m=Z3~$)r2CCiM5D%8SI6Fc(GczM}prj+|_zrwiP*7-S zXgIz#EG!D5`}%sKtNEmQo8%OsWwFB_5A19an%3AmEw^`B;k0(O>sB}T`l@vs5r@`q z+~U6J(55ZFY&rDHmS283^vk~;BFjVg1gf_j+PG;8{CxHLjVsoW4h}32im3qM;ZY-RibtrTc2PP3zr3{gzFe55cKv7vH>P)27Yt8#k@r;J(sr{nnN1)~;B&X6s6q z6)RmyX^ZoEcXQX}-qyCk#U>GHDcMPBRY#8Gru)_V`ucl% zZV$}d5fl>=7E}=y78e#)7#6xSD6Aqdusq%~5kBtcmq>sUK>E&rfQ-n<6LobLl9SKI z#$Mcisj;c4`SN7|g^-ZAii({83g!DMc9e%C7wjml* zt(wuv;pqqmh%(kz7I1l6XkcrNOAiup938yx`ofO^(g$3eye}KAdj9qJ3a||z0zm`> zB{auAjEoV)Q^GHMg@%VmdU<(e=Vw>>n;QM`|KO|f3B4^WD%>|SGu7{isZWu$iK(5L zg`=hQ8e50u_D(C-xUO5ab;HWl8{O6;U~k#D2~-1{HX?!}dTm&}dfnDlD_xgub#h&9 zzhf zxY2m?X3MQxjN7dM3VWmXMDIQfps;K2?$}+?aXVu{_x9w(JpY>9+HE-Cps*-J)+2ri z-sM)y(7Qp7v8k1jiK)41rkSZRI=i`ATGypYx(B?%%CsJtW63f%Y|J z$@o?91OJ+`{L0GA(9jg$1Y_?MGmj#x9Ai6EGbc-HR|oqQE^8?ES8jB3BkXSi@i>bj zl&xRAdiB;7D{wXCdMqE7rQLTd{HD>Me&>ZQktc=4Q2Q ztBJF-x2;{er6um3!itL`QVIEWAm7uoCLpuMGoU8kv&Pr6CLz8i!M7$op~f#g**Cr- zIA(8nWJXrifjs}}kcw+ZkDfYlqV2+kmOFQtcki*adkH=cT z4*|e00re@~M?6x!!K#r-F$A(SHwDjhZ-6>9Y^*)` zY3DwD2B6US{MPeZF8~y(A3Z-){pkG5GpA2H?6`Dn@9v}fDhg}17v$E}heu?`-~<;O z2IowO&-C>SL^&ul67~66xs|mz3;XAlZZ9n>Kzj;_rDe(Wc{RD)O0p_zK`D3*j0wYr zehKkOe!kuiDD4QA>G6o%U_CNFs}eieUa+HLXKeJIqfJ*j_8)t2;Kal8ubOV2zj@~6 z>+07R9-XP~Ttw6B3#T8xx^nD6?3F9~_C^;J7M6unts)@94q zyDi(W-g)&(+pWvZU0rezIjt=d%#4DIJ)%<5@{`hY6L78#EDemQ35+QNf`bZT!pcL# z<08Fcy|RwhZaZC;e6wQb^TS8qJUIRKHw)gR=_jfP8ZN*P*y?5usyXR+5Rv&+H^Yi8NpB`O3bK!K;%}ZAg9^QW=_Grew z*uC5LRpf0is4el&&aL&zs`ARFkYAObUzMGm1!QGsuqJ8U}O^Pks9TFB;U_BH$Jl@FsL*rv@9$tIW)W= zDso>$er#6mp1P8A{$(c;_g&b#=g$5c_g=khfAqQU?8*MSkD0ygg2o2c(>`I>7tug7 zfWouxf$o-m0ENbT<>ZB`0j4!tvj!7UfBQO#NItuK+U$inylLT?A&eHm38@* zwUzh`It#J#+_3{!(elyr<7Yp-dw1_TQM9MMr~TXe_U|C>-QA1VKHSDiG&%6_e8#04 z*rp;jx?o>iX>wv|@{YuixVYG8D1Gk60qp1V@w0DQAD`;I|Gn*5U-w|kka$Qe83v^6 zekmItzcdeZ4KzRP`qJ1p^7Yxs_orR$Py2e?p1u9nc(MKa-JT~OE}lC6;r65BXJ3B$ ze6{KI{sRx8>aIOU@hugFdqd)4(XIFGiz_d$*s)_@Vq$q&&Gv1%+wvpx!!l!h)6!Fo zi_L7U9G0(fTCvP-nX8qnv$?ZmWi2Pd#w00n$R<%Q3;?tHv`58O9C!-j)HVyQ%?RH=Cyt&XFYaab~en!)3+ zmAo;DvR^D|8DqCV5~*ZRDj67Kj|_|Znui(&`oDDz^n88#xcB>;$8Ya^xcm9VNqp(6 zo7E>?-8}HH>Gaj}4-Oo>ar)tn8z+t(JH3DZgDY3g?JC?`Ua+sUzBD(#GAb}E9x@me z7h9Nmm>Z{>nRuJ#_?dWk`lJOVB}66!RAt801x5KsO*4^{H?eY8ekFD1)pM7)Y^YMo_tKpuDIC!%fu6pa*n-6!dqgl_l25izd zF#M%oik;B=92`<~wG1_k41RCuqV{(G-Q%}s-`vGk;N;WgJ7+p?VJA(U$FHFZE3o={ z@Avk`zNg(S-OXdz%Me?pPzoeEjYP+x!IlXWN|lre4P--XWxs;aB4!LqnFC{zFGJ8k z(bX@0HaPUPYp}g- zcGV;vE(p045_@3Ro>zxEE?jzW>%{pFH>0I*!_(Ur2j4yse{WDe?N*Jnum*>P z!wRlcB9byiV&=G*0iZB2#Qrj@08nTdlQfH&SauT`hT2A2zCZ1zkK)ba_aE$5Kn-OXLt+b|@=FJbg6l-L{g2}l`$3J@y= z0CbTm1w#^EvqCl~Weki#0)R0fWqcWubT#99$J$2vzdajhZyf3Q{`6FD{)R98PX~tjTE<5D#b1WmgJVp1Fj@{vc=&1p6ePT1i4K+cJs=mr zZ@|~}vmq@HyerTpR4jb8RLSXQ^9H3#P(LV@H7j)eY!%R~z|V#-%7Gz9cfVw$MbY;~ z+Vy3uy9IHP-8{q`mMWzR9b2MeGB`|$N-AaaH;V_l`=2#*-+$70>T%EQkN3{rI{Ez5 zt?HMbnocx5yz%hFr2_{tZag@=|McPgS2G^Gx^&{@v4fAUzCL^A^1b69pS-*H^i=ym zdjpOZeBl=e`q`RciF%018e;%@DMKq|7{rVwIa|hO@c00p>AS3f$g0)>7Yo=n5$ zu_P)jlj4-Yp$OHfaj2jhOEeHuZ5SLOPtWHF$2IZ^jeL?L1jbcbky0&(252+HR5i18 zpbz?W53{i(G*MIz@>6rp8z6JUL*6 z5MP09r84!fM9?Az-GurfCP%8oPlYgmTFOw5F*QRH&agt&FXj!4c~Z7O!O$>eESXBH z;>a~>zFs@7*N<{Iavo2wWN;K>S#yhcq_3s%Ti=uS->%>5x&7hZwc{U-ciy>t;a2C( z&QGtZfzGB!mtQ}BapBg7=O<5neEjWVd-wfkL(c|rgm5DKGIAtLjuyv8ekf0`0Awt& z3`>KACua!wG7W$N90<%4^4UBbvVtNXBrpZo0bv6*V@l01Qw4Q!_$Uk!Av8FAh6WOd zREE(7jPnTpTx^T2#4|*RV!4YLQAH*XgtjhFYQE_4iA@bPqjk82I+Ruf4bJ{i*NQ@Aclhed_Lq zi?>hSdv~W5XuWpt@rR4o-<|4x{G_e7t-G;%Xk=K@LSh03KU%3a=vZ2rTFcPrnH-ix z0FQ>RsTDjx5ATNaQXYo_4xu09GT~Sh8<33fJ`V-$WK2N|Tp7No&;iYC9@^vZ+1Qgt zE@Sa^dJ(ASX-9PsfW>8MglzaRJR4z&#Ya6r>N!IEI&e;?LzGf#uoUwk0s0&G=j!yN zu|PNps0@&E6niF$QagmG^oB!9&M-qT%!HbRcBvBmP0Q8^nJPJh!&h=dSi;Ac0xq1b z2|g!g=*J{#K+04r7%a9@&y;DEJfTj&RcSDQ5whJoghVQGtkF{b3PRUC#!En{g_ z27}6=zILV*ph*9e&!xdcqXi>XyZ z3=S9{z=>dt@de>TsssiRY?%V00og>(P;-e4^8)bk5&OXsB&tPIhKV1-i61Dagp*1Z zA3LAd6)^@qL8itRp%SZjhH(yenl(D3AD`8a&Y-{&O=VOSQp?mClq$JSzz38F z3NiyzsZ~HGsj7Kc`DIYiJs|!vARZV{3=S$lTDQBq)NXrgKMafD!V zi2w#MPd`MuhyaWah-Cv}-GBmJK}QZA+!5If5)Kgn^vU!h00l|_ISK;#1gMnH{mPyC zRX_d{aFXIO1f!rEyz^MoD*ZI20g*w;8E2|QY#mpu6pEDwF({_MRvMT(A%n-o zX;20Mum>G95K0(YG0vM}LI0Snc}UqmBpV*njY)Y@sY;;`u-P05Tg_mylnlL2rseSr z0+o;>kZU+xmWHp_h*$)CK+WN*QO^@dJoLx+K0T!tEi^OY{Ba(Fb@oaJ!mcn0(U z5hVbLV-?56$~nw zfhW`Ilq?xTtz>Ed6u{${4BZ+UXqRe$gIxrD)NCB={Na;T3=MdvNhcK1SR|j;5+6Yz z&oH49PJmL1d;}770J%biI0Wqx=K!MgQc#sxtEI8{!|lIhB>Wp3QG-ZQpFMJcyc8b#3VSyFx3*KnxWLAMh87qavCZWctWL2i*pTI z!U0eK^+QSo15%(g01+S>6w^)(7aRw4+DQN&3^o@q$)hSk2O20@h!AjY@QXM>^Sih! zvNy_o7d?1UOCkmZqK8zc#V_e+YMLdS7B=EMBA*J&VVr1n=x~T?qdeh+hCid7{7pOl zn}$2X(@!%6qu^7*gQFwhgYZR=ngyW2d>gtMLjal3PY|dOv8Gk5Ie~skpe5jF=KvLJ z0w;2X(lD&RB^0ilggnhS%P?i&{uJ_m$VI<#M}J45Nv)mZaYoT^6zDW;&6t8SEL9|Y`PzE>@pHB9_&nEYMM|Ebc> zF*xHA-XiJ$n*vS;<4gfM1_9VtauBPC06O&?k2R$t0?cp>GaTWpMmQ@lOd*URdW|u0 z1Eocr!9+z2At6wY_77I|u>4u4pfwK&*4?w;K zH(nwHKQTiKMFAARl7hk_U4$IsuXKR(Pw`wIdBR;m!zzQ#}0~NCcG>CK-Z% z2E7VxNV7j%&Tqn)-w>;&Wg1$EIi7w>r6qdE)DuKqoJ0_=G~@&ig`mwMIEE>xqU4N9 zR0wk9EQ2#Wb;pYv5Zs2~euRsY6xK4yRWYA5$5WlCGC*g+z3iFNW_Gq)v3=QxVWao?g(@Nt9&|A@aqn3+}Y5$ma4r;?D z;u`ZCOzQL#Iu-&5SeSwr10eCAK-H55qz!2XTG4Dy$(aWGOu-~mwFJnzPULo?1q5P}SFZ~{IXM637;6<5srSIEUYxl~7h0*Nscttc!W z_M!&FK^F<4*`$_)cd9)$S%i|J1$%>o3b<1Mf);>}sDu(^{~AgI4IkZr^n?X6>8(hR zBc46KExi@_B1$>5r#mZ_RDZyAz^Dw-TRSVz{#5B_Rr(o92tqOqd_*Nyp#f<_HRI`L zuvV#=V+tlDs&O`NRH36GLV?Xgpdu=ufezamaP9mXNB$dn{7)HcilLd72qxKrafM2x zAQaI5`L|E&OElNJDLY0xsDlHMWVkv~Ig@D~B-PO#X?a2Z8yLTiRiG@@Dc=ua$BR&l3L zPk$G_Wl;dIj}WC$;W~E|?kE69h99&B3%o;#KY7PmjqtZW+puP2lmHCQv_v%tfOG{w z$$&DOs>E6dAQ1l0i2kM#{i+iFQ0iwSoC$?$T+9=V@%V&H++PSpY?^=loD!!4z%Ye^ zgf$`NjE<@J!#wV=PTmj5gjf#o_)?Is83$1KZ~3R0rHeoCU5RtUz$ zs?jl?Xb8?tyji5cIaoaj#sMZS%Jg#*{j@?o35v%Aq9JHNWZ?Hh3CN}rjR{7joN)jJ z@B=pmPyo+3v_yB`hroqxwX-^Cqn`oe41^}m1nC4g9pGf4o#n}Y@Z`VA46_Ufpq^H6CdGpBG1Vx1 z85mQEq=Hc~hj_GDGYQ@m>IpHyq!a>h#)mkgLxRy^0ug{R0jVEH;a>$<46{oGSjrI{ z9%z1P0l+>TmazoCG_#961T~AIqCg9<$UgQ1(&6zmXmu&_i9;-9eo-^5rGC#0M~=&p zpK8Moj{c`kGsjR(Dr6XSz?j4w++8dByKwv;Ske+e5$FOJ7t2Hd3OW|9H-A%ef7Qr; zN01i?e}lU>$yhT?%`^m1pdV`{oJG5j#l0`UKa4gfvi{cw7mait8dREbN%NeBSyS^U3gxW94`X5>TwnV#aGoM-3x zR~lvkhIWSV&ze)Hr{U;`AH=OG3Femzl!9T3@X0at#8M2YCs02OaK`&7pa2r+Tihm; zzz&X(T=G#@D2v`Y&lG(Jyl@+U%}@_Ia8(GWCL1nX*I++HwuSRCxnjo&mA)FHwo*a> z>f}u}T-cVTo7{;}7ytC#@E~r5sT<-J38xvu{&>DZ?$S@S8X#g3B7=$Ydg!~062=g=+?#GC~&R=?yQ3n9z6B;QdX)@<;rWSUHHn?qOjs7%D z|0vbz-UULO0%mZX&rnY@K{^2n(}@2J7A|LI5d4?+00=;WJYo!j4K!E^ zR1&YB=Uu9wT7tPTJ(QRmV$m8vh6#ux=prY}M^lSInw~?XmUBmbL z`T-R15tVHN;-@Wa=mLSD0`4<;>Isl6Hq@U;&s7HYZGD*}C4zsn3 z7-Jfm*g8Bvz+e_`c`5cW<{%hls}O`Ip#k~bu8d{zWu$#pk zrVhhg6N^x3ra7!RwRT2t_z69Dnh6F^BxVT5BwFw;1sE8UmSeyU!weMvQ@EIAa&RNS z2P6o~7#h;f@DP(I$b^6C020>pVjqy=T(GEudP2&Y7PIESyh01iDhveTc@Qn2Leqi< zU>|V_-EQ8mQNS=SJ^W6Mz|({9WE`Fv#iY-F^KHO82KhlQq6)_OsYfPwYBGgNjErIv zD!kC+nj8$Jb&o0G+HmBawtzP|`UVgsz>eGv>XD%PO`T0Rp z=Z*89_8+X?fAGSkhi7P@eC$;<>QBGyJn{PZ!wa|2{=%&{XW#Wae*g4a!*JIZ$(I&c z%dl!ls!>R^3>{yk!GQ4$%P`B)&p`u>I+L+#^2lI*1R4W47{$X`Tws{NUI%R&Is2PPISHpwNY6fp_x>*o~uDYkZoZJ3>h za8?1#D?w?1K0t*ckdFdIH5tiA0n~w5JbA*B8eoQtV-?&TPEVe#ERMs&n?XAsGg1$_ z{wAFM-7xVR9(Un67PbZd7+1#gs=j{7{k~=p4&QCM)B67T$9or!zd8Tt4u}S`hi@F* z)o~#1@UeZ-9R-C)%PMx)gzWV%-&K-WSYNQSz94Q}@;vY{Z%3hjd2H#vy=a!aa}V0@ zJbL5Oi8D8=uiZI$>h9yOZ+kGKvgK)?bf5*ZP1H&rU#*|h$`OlZG$2$zEznOx42)cA z0K=3QyM5(6AI`7=;zqlDT#JQVwhzLXC?Fa$rjE6lm?_{h_&>2VNR+C$Rk=jgauH* zlR!K$1W=&s@pup#(0zb@T8O70csz(_j38Y!`}fJ;{^!K+|33cXAN-lWLkwtuZP6Z& z*WmQM4Fd=s7mk0pdgeKt_~?UIyE;zqi$1qKZa3)7$}5PhEe*@64~)o-56wyl^70Fg zOv(&T3kdc03{0iHcjQxEXK8wC#L_Hd_xe-eQ11Xz;U3e@9T?IZ;FjX(3Z@t^-O z`V-W{{bx8_+_>}cq!Lpnp0$X3+D6XaegE?F*^UEEJC7dQvFi%_ds}ivR$g*cWnN%J zCD;VZJ_&(2NuEZj>E<3sEQ*pWbG)rhQf!P<7vO^eYEW-cl!i8@K7K|i2`1jY-tj>} z5w&%BcyHk7(TuAHu0n>hFU~%GeDUkowwC5$nNG!JY0(E}g#4fK(I4FL-?-zy^MT3V zQIJzAXhlCTyf4sV{1i`g@L-e6(@&_mv%-mA<?O=($0+|DakI!>H;c;V6KldX53G`0;6 zNqKlk$Wf2#4byV|EEgJ#{*aIUl#6}}Md%N+TK=q_i_2Av8cpC~2_BqaPK!=GDlkm5 z__Hk0Pdsf?84#txKE`n;ng0r`Lf_B=a}4woAuuD8&jOSTLAqwXR z(qRbjAdSrAS;VM{%-#`U9*&MM&y|7pMG*c)0%{H%IEP;JbREp_!CaNEBh9D2Kl^a8 zx9RiQ+vlrq?Z0|68qsFgmD2LUO8-y3FvbgqX;rfG~=BUsInW7CGKlCg9zJ z&~2XMV3A{QVd7{B7&}@SI}sS6U~g$+Z;=BrY|ZACurx_AD@rQ%PVn~2Oo$H)3d^p_ z^)K6wmpYCfyYZ^?^R>qp?{|$1i4_8wPK!xbLhck_G%FX)3i&ez?v$20r(8kg9(B{K*_;VuWWKh`Q8M?? zfLj0x1h8|E9fH+0`V)&Bh2?N=9`cbt9|x4XkXF)piQJ7^E`stpLuPY4cA@(fB% z&n!;!HA?j}FZQ-F_OUf9rq~A6M#wVYY-2{l#m3CV7BF7~skb(DvNnP)4i+YM<~i8b z#G^Aqh=>ttK?!3Eyzptf5pFfQZGy^a6E1TQ<_K`2O_E?Nd)W?|eA%^3$G6r+4h!U0YfZ zK9Oy=4+DbXIhkIVUlWXT4ZaUV{d8fWNqd=&$5M!t%Zx-pIBpO zvBu7Fjf17D16krD8lW*Yfm#lh#&+f=c2L5~$j;gfZ)RIrnidzQ9P#rEjtQ^ZRvI0h z@$%-Q*4DRQzm9zQG9;DCFz1t_!Mp^mQpaLqMkRwIQIgrAOdY1jVTKk9lSRdtQz{`C z!(1bjQ6poqAqvJkpbqYRXi#@RxCtV|Gy2IU2!H`Bo&lru7(Acj$sxmn5<~z32!PH3 z0h9*vz<`7->ZFc>QlS55aI`XwL8lh-SaOw~%Q1-5@=+FdQqP}c$;UOqQPiLT=89w5 z`PaSy%q_ol^7gAUok#Z{C@jExM8bq`80X}Wt*+%Lj3>|2=uc<0jC!W{3#8k-vBn3xn57yG0qcve-`Rm8;}IB@V2UI4lCv9N48p8E}WkpQm~v~@KP z3^Bz@4MSjH2oRZs2*^}SMu;K;=y30Z!2mo@LKve*D$x^2XP`m41c^)}MBy}nrz0d{ zk+TMgUXv)Wl`4gj2cRI4sU$i9gQvlpKmwLfZQyIUEQ)cTRS8 z-n#kv)76JBuRd&QdRcwp)AQ$-&)zxv=Hr`tx9`2Z__qDx+xxALdp_L0e)hxNH?60j zMBl)Wl&NAMb_oE3jANieG{QB`m$+0J|<+b=45B%43@#TlY^_X{hx4hfD$fvbKAidv^wHN zI2%i_N z)`k=o?mdcEJRjgi!`DqWt6w#pLE-Abm#;5$zPWwzTU*zdLMapIbppI1sO51CJiHE} zma8!7o5fcV@9`C{94R{Bf-3? z;^czdGWK@JK;Qtbk&^~443M&wosE?(s0W)!Bw=X|&8#iWP(~%zViQNC@j*v%a==cU zk>11J4mH>u3ZXrcM^FMF(Ux%z`hc;ixdqaJAQ6ZY@@aXeq=bb9mDJZKCKl}5R~{Ev z5gS_=5)y)xF35V|8}I2yr9p@Z3d6_B+*+g@38^T=E6yF~jsY1It{!Z zVuYT@!OJvSmBt`Y3k7PqfW;-y^T9r69<%@=0ZXKs$Eb?+-(iVDB`3fYA^~5c7Xd^D7Qhz(YQAb7BHUuo z5U8Qe=sfVK3V=K+4oAZRP{7a$gCS!}7*eTXczA5ErN3+BOJm#9-Y54TUp#f~U&rkh#Xl4CEUlH3%d|iY-THYa6>% zkK(}0_^Pb@lKQ%m`r7(!1ljo!p7FlLIVQ+swq}hhx(c$PAw3)}o&5a#f`Y)ecqJf6R=didO zHCF&=7Qx3mP>=)yZ~)dQpuwXm{WuCez3orhzXK?|ecyZi`rCVtueY||d-LY@$&YVv zUh3?8{`ySy!BfH1%*+ zlDNj)^iLURG5)$lW70(spB9n|fpn6tY^eYww+2bG2MKYmH8-~~Gq*G~vobZeHpiI+ zrxZI!GYjjq6pxrd&&n(>iv8Seb(M9sRry}w0iM1-9>vx~C?{Jx1mgWB^r>g)~ZIO$TEwEh$Ti8Gw+Av66g*a}G$sa%RS7FKo^Ru0q> z1v?WX3%{h)u$X|{DzB1lx%G8bb+uWwx!GP3QJDe0DJdS%486q48i~!UEi9~zjZ9O0 zkXOYoFfcPbJgO?Is-(UId2sgZIsEBUXHU-)F=m*nwJfl&7wQ46T(9M73HX2>&=auq zTo#L~R&zCKxkf`k%Rntr$;WtU^!yJi%kco5YPae1!&|DM$O{0SR%X(i&tW`+DQNfoPCh;i$W-fM%Fdl!UP5e zBLf=}+#&B6GJ}ClPaM{Gj}_>+G`5L+u72TL1!kZ)>X zYhr3uWMbi+YT}uY78Mkbn;pI_H@~(rqB1Ko3)xtZ%q+&!69<%(>YY;5l#*JU>Qn6F zk%JS4cdAEHT8iJ1w19Zu@X)Zzs;ttyyxn`FKUIH%0Bkl>t0n9k41io{;8NfV4I&DB zXkb8xfCel%1Yl7D5E(cI4acD7Ko8WTfAIAzkybyd)s7nslK=_<0Z1vh6i6?aiTsA~ z@t#O{M>6Oj2h$Nc!(>`&YKo6fDmX1F^7t!!P`MOvd^CU_Bo`%dek{h0HC&m1 z$_`4R7bKy9G9?DB1W2C;-5714A9Wf)@V7#)u3}?Lj~w&VBBQialk_By%=o0xnE3q2 zn5_KJY_G8Vh%m3npos98NFXXEJSsRmG&m|OFf=GIEG9E3Fd#THKGW0J(>FaZAU-lG zG`A|tKd&Tu*RD?&s(X8SB@(7VE5u-`&>)u!xfJ+vAs`|kDo_&Wxmp%i|5yGY0r+Q8 z{OehKE#93o=tm9OaiL)nKmi#%aS1^ErPG-`eSJxC&qbh;PV|YER0ERz)7Lki`t!se zq@r|6U*hr@7(nt{{F&0A*oWiOUJudhMR}Ag3;Z zM{$m6st4iyh<7ol_w`HlOz;j4I1(0^5FX|k8JY=%Q-A!0xD=n$> z$0bs2RTVO9h6drZ4KX6ZDFI6A_r}InU%2q3rd%)KG>%O(5oR2U#myN8kHNyxxz^Rg_IQdH2Fip+dC}@B*NdxAJ`_DRw=d#-RKp8 zIQzy>X(5p|4>vFo5nf)%4;q0mhE-^Ttvx(EaBU8bh+o?fra%<_nke>3u3;*jE#e6Y zEOd_B1qCQxkw%!lyrDM3CtN7vk5UEuz#mB$;^+8 z%B{-ItKYUOc4zg4Gf#SZ8B8WuDCY{fd^rU!pU)Lh80E`Hi43#`h(n|kAlMKA2x#^> zLcK<=r35hOMMCW;*D%f(P68-^V~TD+6hJdr4nTU-z~EpojVojPAq{6J(n3ao@2sq< z+Pb>Dygc0Emz8bbo}8RqfV7omwY7Cvi;bX~hmQ}uALPTgp#cdM$o`CL4J+J+K^0so z*kzLl0O>v3uGq%`G{I_4`q~^T9lCP9UPb* z1z(TK^ok73_ln7nh=~XfL`_(XZ%}YT6aIz8#D@lXA`toK<;H~+WE|alWB-+7=Z^00 zID9T+&z|VmkR8d8Alob4D-!OXT~e}b`}VTKhxc@LcHX~#pTSU)utR*G%jNR9T+sqX zx!h5?d{izR6$(cU1`)X?!kHMSVH!B~5}*dbpTr-;f1!3+D6vXY&JjVo;Kcu^NM1w180qM(O3_T0l6i2Ml<1 za#XFw6+XEmYof0B$xRnYLP1~jYw-wfLT{MR3a7Pl%oM=;t<#D7DiX^>2#^UhqyobW zA#sH}WA|c^2bl;lT!@@`C>QSCjl#KOH?Crssv5T^mmfX9)d^fa-gNWKo+}+l&KDZ# z<>%{}>YZdvpK-05ni7vwrdKG^mL?~bl!w&B6(;VA4vF2nZ(jw* z6!J>^F;bG1TbEr0F>)hn>#LG?)Wk-|ojZ2r#sPRvN5_6V!ap3nXKzJeNb-*4lA4md zveNRnkiAC_UpjsI=BG#R9$#!2>1OgU@WWNh`7F6;0i$a9_yQ)?@=3LD0?-I2HHJwJ zpq&&DVA3!J!!!rLqi2FCwH&~FBw(sz|APz)C=eWba0$69uO7Vm>Lms`&%eI#u<7-I zgH0GGJn^at$>3g`MEO)8}qHJa*~yrHo@}d3f*9=!(6Ci97ezloupd zU<$#}OD9g8ul{uFOWU;;mxIgX?X z0A8&EW&o9(#;ghxV7PzcLg(d0oI$3_N6+yQ&sT1p{do4n?Q3sZk6-)n{Kd6T&yQnC zxPS7!t*;wt+@5t0J{=io==y@Je8?tO7=75Qsx~1YI4wQi!~2MFQL1T<2MusV1C;>E z79iH#h#vn(0Z&yxKI%<8Qi@a4uz68R5?Y!Y7u%Q{(frdAU_e9%u2$X zFw9Uo@$mIQEFT;^z5m3Gj0cwv@9)^#akTL0?&xzF$6g(5>g>Gy@x$%6J?+oBzDN}k z43o+EDj^`EfOo`!ah@EQ|LoAVW4-Ve@y%9hGA6qZJMVlwz+CVz86(VOLp2r{`ACy1?$cE=J$fW2Q z9G31G2o2B;XdXc&I0a=O7Zig&&u#HEY} z8Rrh1JAMAZ`Reo4m#ZJW==^l+(S_%q&U~)E`RKy=>N5u~+`Rhw<*QE@KHYh7r?vHN zPj7GA)2A){%@PJvt>+23JOeVg0GMhFjOv8IxK23P1j9S~3(p_%6q0%>Pae!ulOvXB z!29ns0D~ig{QwGZ=f-Cv0K6MD1N~zFNeekLfJ&@ImdGImlYV+8l_-&05zXGWf2;m{ zEBa^$(pLtCg+l}1%oxA;%%lWQC;_tYd<>j|IIs`W$x}pQ1SAid^hMr5{UT5awTJ*_ zh*f5EU%`Vi1R?B-u%BI-SC)({Zl@2P?tJm-`PN6I>^u4K?i(5>Ti={)eRr+(-B~1Gy8Z6tVd0rUQM3~15J!9ipaGm3q{E{Dcs)7aE!+Id;&w&fIB@`u+?%TVMdoEso^7ifhp7-CM^n7dY{rde$ z?avy%cl9+44vq{Df%;*lQmR8zAGK0%kZI)A0|iwLOK(5aaz^rLsO zq@KjfP6-EeR+_jb1K70rJpda4-&t2gIUDEYS@7qXW

}0X*U>}heuA}&{L>Q9Jpr50TA)3;4YtJ|ny?KX z`_OaBPOc>1N`D^grFv_ENk z@}#Z3y$y*Gzu#~8`t4a;W7pG$?yhG8-F+>CUH#47W5WYtX|qH!#88TLBWX3LyZk!IIO#l70Yk;n4sJ@M$DM z1yF#egL>41bO?iwXh5rC;E$soy#(I>^=TJQ%(zfO-dscl#2$nP0N#%!PRWZM06vZI zgD3)T2BC0wtU?>Qdw_9pjV0YHpbfUg+zMJzl;QW-1LT3ORdppuwR`m3rI#1Zd~E&r zy`g=i`PrBLu7RQMmazelPc2cvJfJTl&0T$i&l$ZW?P~n??CbZZAiuG(tD&L0 z@7cgeSIa>6aQ}dGteLGCW-y18N~uhz;Hl>MM=CLvmccSK!5Azd1v!JoW#|FE5)=0c z$own{^BUl-W-|XoIISeWd_{W7B0V?jFH1xikPFbE1`!^9O=|$g!9ERWfC6|&x6pF& zNX>v|I8z}0<0@(Y>8sm!guw9;^Fb(l6yyRZ;A2vHJZMG0M>xUs2gDMDA`l9Zz$Ti( z{Xs5bJ~TiX>_eE)h-{qH^U4ZJYLfFS>!8H{*WI~4Rdr^4-q)n+=B6MAI8hMs0%#yAh$3L3$jwVs5Y)&`1XK`F=to$q(8^<4JztfiMFbSmyAVEq?i;R)X# zxp?+i0nX;?wK@a>u5?kj4D`kla>N*JT)lD)i0jVfpzDo=b=X5zTVr8icXicexVc={ zwD;Ky=J+qO< z5v^AIKcJxgP89}6px0^`%>{IvIde0(glwri$jTv+Q5@W14XBxVl|OPX9}YyGnmLI&Gtsxb;U$1V6H%9&Zo@%W??LKa z^+SC|^}&B&P^#W?WVi*+y03O!Ds7{iaO(S{p!pkU0DU>*Qviello+oL!U{fR#sA-a z`wG*04E{`+`}2Z%)8GAY)r{G5mYPXKudJnGwS4iX3&5GHWrH#O?iG8s6T^G=&Trxu z27MKkjPr?#TY~y?OTvU7-M4SYB~SO>b6vencpjiXceNU!aP^vS!2Yf>K_42LdYdj^ zy8;1)-Bn)yV-e8R*-ON|gM`j2?w7EJ2TbU&YvBq9(X9OMfD<8V-Rvrp??3b)IoUAdP4-evjBlFRRt$#LSJXeAYo^z}{_`=L$!AAfmJ0xR!sgx32~p$nI>$`H1xP5k^vs01k{4mo+295{0uI87X6C zI8h}1rZ81NG!h}52R;Qaae(u%qqrU&O-ZmOg0*iYY|8J@QYJnPyg5KrVA2`4NWYQ0Sg}3-T@=Xv#{g-|@W7%TRpUua& z=mIPf`Q)ud^q+A3%&cW@E)#Y`W_$|X?@Zsw_U$=;Tg*$TcRq-WNsfO{(C4xj3XYW) zHlh$+z4=63Conr4q%d18dkgX%?VqE^nQzo>X%cHX{83+U#Jp`q){g{o;5=_@xf(2If+ z?zWHMoWs4%$Y5{zNG~x}MNC8iYbmvGaFBQ~hJ#KP6I12nsQuo9sF=HSoo zWee1NR3Sn<0x1gqm9Fb%K6D!jnsod``ihf5ABD{fji0wF5)A2~{_S%@y?=!O?}B-s zqvFfoUVzVeb?sk&|JzrX(+n5j&%qp!{q6@VIfvtL{IlX0ezx_SQ{P`aaiQQ$Nn7_` z;&;om?Q>VV;M<*gmiyVVFPGTup3=fL2*Cg2aM1UL_6ARdC$C}0vHC+rp8Z8Eo2);U z*Wfu*T3_GMV1(XRMME2X0Yat20ZpOqb2)pj+`g96S=N(=ONgt{(($6(~X2qu)v7lYRgVfO~Go?0)v0}3d+DmVr3MY~CsAc+^ct^>3! zeAGSAgHUd~(v;PBxg@JvX&~tW85r;!Y?+Kt8U_l7zsvr3_s-DZ_i!z6K(!0>s6S0m zycAvs_rp2@zKx}J{dhw{$|>}l%ncQ7RTvp}#}KhR2;fGr0^e+RTq>-?@Efh~WpoC$6&vySEDHZASH_78dtoc_1(T-YVc98bG$SC`p$l z4L<@+Rd5R6&QAqdgu+8!tbyYx!vuc9&WiyqV25$Q4LXaqVjT0s!f`E$X|cE& za0h*A1Pas4--pIyH`vggi^p+m4egm!G3Ws|QkTe+a7~;L(DSWH&(Tzu(^Z$l zVOP5M*5Q7t>R3ln*k09ksiR?kUeST`85egQ$=-hQYl-%s{_WwNThqhSPkp^@`<^7m z+EX)5A1lhMXy~Y-hQ53*M+n&4+sWV$1mJ$85lZXFh+PLO`q~WYE!Qf*5MVch#Qm5m zz|;aS@<2`socIGKzZyK9qAWBrTDq6a7&kaxofF8Qng|7@gKs&Vi+~%Paw2fyPA~)N zh-|v9_!$rfcdeM!3qe5gl0c65=Id>${$S0?dM)n972%`3ip&><2n9YGRp4<^l*?6h z)j*H>w^e7gRb@4_R}}2nzc2U1_eZuwocem};hi7v{wVPMg$Ow?zCD}}v2DfoM=!cB zIRhMfd}AXcu1j>A`151=YQX)8MF?uR~S;l0>$BFC#e02c+s*-qM-{v9;}IIsFD z2Dfh-o^!w-Cqz{p6Su@R0NgtuvzCj*FfbjFsaD9k|D0V{GzEA6iTtHMM*bQ(4G`g`dFj!nXznElf-1jvp`^YkK6yO|f+H0>1cY z{^1qhUEH@%Y#@sfI+{UEd(H536B(I|K*Vh*sG%2Dd~jg!-~f^NHJ-l-DldZ&qsAd@ z@U<6^ZFK{DJpt_u&>t8jHS})B6FLP5jU##i5Tlw7=(ey2^p!TsHV0S$=%coF>Ln`9 zT^Z-MZ8@S1CF}rKMD0u03l1ni33UYRCR#vL-qf-%XNxMnq7MyJ#)zRL(^FAWsu`XS}iiut=Y#09MTVB@MU|2*T93S;KSbtgHQ3wGQZPf?`DB-es z4W;|@JoLm4oX^cTpNrecyF}NaT?sh4Z`XI(+dlg?EG}Tlhbvbto3Z@w%l;CwXvU%i za~FKby78#@KAs=3E!%zJ!j6W9lFTXypboy4e-rhG10~=Z3dipp7`anSq;P~NF_Ibq zVI0~>Yd#@NNM>M$Lah-98H8|-LyUvFw|`8S-r};M0ffSx=3$oo!i261HcYVki8W|F zELIgJcoI@1>4IVw5u0_yHwRceJIv|Oa9wgG<&$=b6k>(vuC`Y}0j@if1XZN&I(p(l zcGB_ilx<(k4`Z0~!;Npg^LZpA4^g4P0SjFpGxS4tHHMao$Xe8qSJ7^e$Lmx9JJ%^K zJZDm06_WR==u8D!*-|BU;rV^J2Tokb$Ua}hN==qCme!Y4oGvP2RqCFkr0|GzRwaEE z74s>HJ>UNGqPcU)wYuQL4>qR71itS&{N1VJ8M#Gg>PuL2wYLfXkI%nxr-mw^39m~N zgPtfV#vRQt{QZyC>DkZQ&KoKdkmRq^$tRYPkX`~L4tcvatiN{#KP42Nv zZPiq*gin>xyKKXLGHkXg0n?=%-5(u3XUB+aNz>u zq~B$qoF9e_1bmRVajwk9Wm+yh_30}&rNw{oet61vNA{g4$U9S!m3c0QOjb1q%Sm9v zvUsuzkwk}%SU;(c0N=a?CQ$)GC?yf|DpIzrk9WQb)0Z)_WjI4c@6OrjE|4k)TG&69j?)uowr?TuI3S=5f4+shl@9TjaC4%P2U-GAgnD(NAQxK9A| zt^-V$IS=l6mzc`I^o_g=1^Z8=k|*(KczVQom+Q^sWIX@8l^fsrd=&&RY_(+J7vxMN zTjF`vpfX<9bd9dpP2PAhAjW79PIaiCq)~Tp(Syu`kSH=5O-Z+AHUtwQ2=$5ZKHh#M zFK;8w%P@Hv^K$U{mYiJV^Fk<)rHf=<#N7aY;zfWeIQLiH1NOmN4-kX@J$4eOBb$F5 zq0noAEvHG`j)Q$<3xfmVM<9~M?&@u_w}D`3)#bdB!UJb_>^XWO{Ogk&5=eoz0x~EQ zU}DSh3^!d=Pl=~A^HN(Qa07i-caAN3`sp1F9QH$oWI#OXK5%p&)0Ix3^5^##73EPC z*YDrK)S6>_+;t%L)SmBmhbIIF1a7FC7_(N_1TER;A z+vt7`BVxRAq_Klj1$`C*W8(Z$qLPS63^A1&nTK+p*(gT53$U$e5f%r(gaEK=DjPy5 zXg)T*FG!-(8W4r7Y&P_OjBW5>2t08xTX|0LMywboXV@StazPk{1%#A>5Ns0xq)?Dr zbmr)R)7y9D(jB5ecoj%Fn^ytyfDGV;bxjDYspv<}&2Gp#$NBrul%79SpIdP3#D&wl z4rJi)(f8cbXUS-L_Rx+3mbdNj6y=@aBMhDTe#_4E2-k)%+)nJe_{f;pH)ChZ_MN|?ru7|mA_VhQxD~p)PSj+`d(YYwH)!8Y( z&(A630yeIT}0aD@xijiDAKvfkS5+cyC1O ziZ`db!_#@$!x$~E)V}?O*`MDZ*_Q#4WDUjb_h}Fwi9|(BN6@=a80F&s|vI|>OWXb6WSm-rbApwYF zeU!;0=p{fM6yaRNJAtD~n@FggVm`7{sTGI~NwVMUgk2&(VPr;F)L+X1Fb7X{xUa4< za$7d$T&FgH5==z+>nfv#MN9SUArG&mGcFV`WtiC+OlD=4Kz&6!(*?4s>&OD#&|Y|^ zq@5AgA5P@*7H7lG!we46A4%D^g|R<(Y5@%{GQjdMYU`j+&DT$Z!+|@{+hRD7AoD6t z4k+6kyvPQGgO+;;1q}qJTSAi|rfhTIEztEqv&FlRFVjlnNJ>2`i)4*tku+%s32`Rz z%?K+;C>M5eiYB3?OvE$3^&TMK+B*r4=1o5-1&+z)OvolvC7KD%k_(keh18m5L+m6) zBZYH&Z`0E_Q_)V&=c2qzgrcD*K*2*2;`Yoe#w!}@+B45FYJI6Ai!8^Cvr=s{V9vlu z5@SWVXD{r}qf?Q^P~44t)S{WpePh+1;D!F^s}a1I186YOn#_DGot5R-wmda6g=E5( zB4Xhpy$mJ5qQTqoGu=Ovj*z61l2B8#ftsx^@1w+5=(UkStUxLnzQx;-UQaS7_TMJKq)0glBPpZHR@j;(&B$9o~RV!i^U!fG{`y|G~J{rY3nthH<( z4`=cu5TLjA-qF6DeJ_kvs%q+^8wdAx^3q2~~f}@{0%(Ey*&yLmajZ zh!*B`WVSc!|e5r1ESqjX1vsG_@K9v z&m0?OxbzcptdD3KLnL z!c)!OI2L@}V^c7m0SN5bqiY4in|le65hXaVXprgdk~-F^6pSf2DhR&vT8xRZPBK9q zA+Z*!DbRid(0Au&n6I=&{a`={usARc3;C(4Ow-j5oBzD%<(;sT3Vk}-hWwgFjB!CuT%XSllN+TtQNL zij7*mbCZ0~OMt8xk?~qR*zZVRjV5agB{ir#j|?e#AH&iQ|GRLDe)1OJzllKXkU@z? zER2{aRIS;DhRUJxo}1-8H3KX_VFeKDeeM>sM%wm2_nzZ|yv|4bkEs9s=fk(2^^;GF zCo`#vd|Tf-NPiB-5HJdEC-hqO8$mK3_4n)Wst*MxB0AXTCC?;OwV;QJYqTe zz~KFI0!f-!ZG>ejCp_37D|^T^^K_*4vH1VH4{<`3_kIF)MIP{5BsuZJ;?^gmy+Nyd z$lFAWeR#9S#y`7R)c=L(H{W`Kz`=L~$nQk#A3tuvd7=%9-ri0vW)r~6IzxlJCDN-o zPj{B}Tl--9c)zo_GWY5y@VOB1q_XvyExwrOqc|Wa-d%$}RR&J$krKeU62boN!=C%d zb8tsS?mis6^>DZq?SC@BXpbzkwzBs63jc_Jm*B79>(=?(|2N8tyrQAtXDj+_`K$yu zUX}XOFqT>`^v3lGZN5|VEBh3%7$D$$j9o{uarJF;e-2)Fe+@ zj2QGEYjXEu2pF^pfCB`lBw}*_4a1d>i;>#LfR0sxJ3>GQTlaCcKHc|PgTmwlg0hXp zSOWZ@*N=Xu|Fa)`w{v8FR2uk-Cq!R?)jMB9CT`d$`xTtWnAL8~u~D(7dyFdFDJC#5 zjZkXb2XNq%76tzS+E2<7!C$iVU<0FMKzMo7FBU@128o>k;DY(cBSZo|8tJK+*FWa&Ps0M`| zq@u;y0J=|xN5~ll0S^oTa6qCYf%}DGLYOG@qtLek(OVphOTd37{!YxZogc=c&mIm? z0&o|8%I^2^9}?i5a?h-LpO6=Bh-!5_0wBY1;H3d3T15IsoFck!slPD-rv+GcfGLOw z0Z#O_tVZCcpgvee8vxCnaczLp$Gr~weV6jS^%=WoT;!bwz{Rlvb@hI*+aF(ozY}mR zXY*f~$q`EaJ_sNgn6ZE-gDo$Xo+rbjwtZr@SweTVW{Loj}b!+n2@`Chb_>c5F z8R;=8!z63iL$5p=tQ1j33=5CRnnq?=}Sc^$_}Q^v-`VTdB>Rz+KZ2OPOtY@@SbEep1>^ws6Z5Se z^P;U@(7uH=R{#oLAlcRz)+%!r7n*6_tTlv*;#+0dM=`IB@0HWoX+hZ^4w5a{O9LMf z(C?{z4*He1$dvb^iqZfFJ_354kUI|nZiD`_VeF%oe(*y$Aa$FJYK=#L6YjBPOZgmF z`wxPeubc$fZ;Ayb^VZV5(3Dp=eXR^yyi}|=J^H@KoT98a&a&PH>HAQ9D?Q${nNg3V zK}K|TrM1-gzOMLW@%hjo@$ViR{GWR%5F!-(mdfYgui)3-|5?RmK`F2*0`M2`pCQ=> z{pTaMej2&;91hG_gmB=Lv2c&w0|7?qoUi1^DeKG#d+X!C``TDhdpY_wtxqg){b-P3 zS7>{tr|U6Ft$ic=))efWT5ukzhv{S{om9`F|C$jxP7na%&y4}78H^^NRkYur{~v5C z?;<~92Gk2$3#j&s0SFDM{fhrjVgsZvq}CgazkVExxxL5Ie&1w>agf)k68tXxQQ0;S zHVWQXmX=b#gaaq~dKTN7N^_#cfyK~ROzrzpaN;gk@X|T48uK%p-ld_u-%%F4k=C8^DS0`baQKO>;u z0{XQ-v47a)749E=KrUe_auIJvVVsbI0ON!h6BQZ5f$449imwH&o3-VEGk3s!?`Ds` z@bfK$Hg49*fsLjUWg8O;&i#Jq3(#VJUNbN=VaB4b9M}}t`0HJoNjC2l-Qy&J6phWa z91Z?H@WFo+1$9`)fo`SBd`*H6iK%p-u z@%-I~TDCY);tjP<0tENLC;h{Z5ejMpgDua+1}eqtC;=mtp#Pj&13MwBc60HgX7ZS; zKmgEZ@{`Uqpb-L9?mOYm86pljb26=QFARe+%#t z@bE8Tu5??HcD*sl-N+5!tU1$tA=$%+5K{pX6mm;w3eZ-Kv1&@6Q#?e>TG zbGr?se#nVGFKNbcLB%7_)c@2B+#edaE7E@-!3ZFLrjS+vgSUPfR1R)EFTVBA$g|?{ zIIz9YC<3IDW2^t&{1HTOcZvdIe^@$s;^bHVFk!-Pr%Hms*yuFA|HI@dGi3MVNs}kZbzWijRM~LE zzrH?^ElJcSPI%*C9?$f9eWt0uaYx6*yLpm{Q?^an{V#9q&YSec{anBE$9am0uYDdk zVd|vKu?g|X(UV`D5IAwF{yd&w($e|dEb!H-=BLWTPyExwspcpDY0A>N|MEK?e8S}4 zvrq5-rwL2{$et+^U;E>0GyX3?_N)BjDgB2gO_}({@1~s1o%COYn5mOI(J67!o2Pca zx_w$=N?iQbFJn{P>B%XZr@jCBXWOSGv5(TUP0=YE;-}4ceOgL%ay&ay(%o)8ZBC2c zl*Y$drkv`bINKECO>yTsKIrwnG#8JzE<5@v>l5SMabJ$!r;qhzTBb*CH!VFSF*Y?$ zpCCQilWBjE8nZDzHcg%^IX*2qE;=om!?#b1PvP*ylnrw3s+GZSy*F*A<6%ZW#WbVQ za*0t7;@j=_EWTT&jCz!n@hO{AH!YM$p1wID+7nOuyny#+ylysg{xakBIQiY3%N5Q)`SXd(HP^LZgDcoYApM{XX5WGZqrT6Oot$HRza*UWDCQd0QF5t; zt0a?{eBt6`{w$VF?Bd|`sKvqC*tv+)UGbTro|xnacYN@?jWGdRXUD7w+!nGib;FuD zn-il0-04w4A@76)g=R*s2~3Gv5we+bdgPkfZaF3cy+mV*}=8Ml3Psq{ap$ByjG$ zPZlo93|RT80*n>Vd8VLHGXJcGmwB1;tl+#k zdJTHGHZ;)ilCmI>a%Rk&Si@sn^6Jdk0QZ)dMA6rfIZHRX(w7B=&NKRF>HOL`QSR83 z<(`;;xd!iLE9P!nx-mR)y;`0zl@($n#uS5GZ*O_~Z9Xhm5Mg%4%2rLue8xUqu9(Ha ziBW63_84h@zi5LiQtT5uUgC0@@55aS!d>jwKj9gC?{Y0OW@~nu&*s((!{s*S`j*bf zq2~EkOG%&aJ#igor`fnxnQvByu2?xc_MND;3A3WG<>)o@(!-LC4w^HA7YSo-av?i)FJYc*Uxud27Rx9p`TJq(z}u_?B6b z22Xvc4Xzcdi=r7<){@a-HXy)E9Nd(>Y5wBYNIPHBYutbyOOthB0Zd!B6hNReN^(i z%(Vf*=uhxAcW4;LuAYs3&k74&5$q2BWL1JEVud?suKY6L55H<2*zm=#;3vll9^Nmd z#33Q(Ajp*uW-A+`AB)}mrdfQ76wG>Ond%y!h}y<5wrgnVbtL$ literal 0 HcmV?d00001 diff --git a/python/ray/data/examples/data/parquet_images_mini/652597cc55e94e3e8d91776eec17774a_000002.parquet b/python/ray/data/examples/data/parquet_images_mini/652597cc55e94e3e8d91776eec17774a_000002.parquet new file mode 100644 index 0000000000000000000000000000000000000000..a698d140c8987ec2a86945b83a4b70695b16c26f GIT binary patch literal 43004 zcmZs@d05-$nJz3@G7%ttfB+$c5ZX5+Aqh!Nmfdr%^T(I#xs?}zv32X&pXb%P z=Iz=bYEZv#(1dSJ&@U-LR0fH3|D8YFqePNOBq_iLA^;H(2Iv4WU<71944?&K0VQAp zq5u^T2gCyjKoXz_)Ib`L4I~0vfK=c;paj?sdjfNr1*=mhG3cAxukLb^w#WPGAO@2Bv`Bz+PYP_F90m>m zM}ZT-3UCBi1&#w}fn&gF;1F;DxB{F5ZUC2n+rTy89&ih|4qOJ#16P4Fz$xG)um*er zJO&;C4}dR#Pl4ya4}hn@_kkzCXTU??4saK^3H%6n2K*E70{A)bGvHr=uYg|wuYsQe zKL&mXyaGN4J_cR_KLP#)_zmzi@LS;Dfqw)37w|3cOW;?)e*nJ&eh;hz-vIv%+y{OQ z{BPh7z(wFsz#oDC0=@(O6ZjI~0Bj%}2mykDARrJ30DJ)^;0O2v48R8<06O3ecmbY3 z5Gm;2TpmFueoa*d_4j+g$UTBQI7X6;>i*kVRQJ!|zwi1f>*qaip!{w9?;FWfobb~W znv+D{K%s7g;NU@dheq=B@}|%|AU=ezAH?6+kMTwVeEkA^eF8k(5*Wk`ru(u&yx5@u z99C!;H=M^82t{FHiS+-9j3-w_C{+MN5tu9KhHW)?t8GI{R7XuNh!|*dQ ziCm_YDkO3hPbuU@M!%IXk1?S{2~ilOhUkb`h1R5ri#G^OhUf%MT%tHGNtT=_CVQsg zXjU0zdUdiolBY}&#iWK&cxmd$G@deDA}3SUui_A)OpAz2m1l6%xtUp+*{p0LIABw# zf6!*aBZuxC=;ghc?vYLS1`|QKL=Ya`LiY{z^4Lt5CtJBV>>D^Y4DJ&1XCQUi=&I56jCTz<5kno;oF88e%{)B?GnP@Y;KO)HS5XF=rY1etg zw!BMal0Wo$=OYLXe{+$4?jlpwDSzuHzv?HGoOl9~h{ply|0~M_RJs#y-u6N2dj2LR z+56oOAUF^nlo03u-D5N1o9!PI>c1(JK?`Qkav7WcA^w|kAw*C~z_y}n2Df-~YF_SJ ztAOe!MN2H=I7pc$x=cf+w8o3$ObUI1%9xo%QdB00t4xOK_^9f5bF~E$3#lnfvQ{M4mM7p) zo~SVygi%7RCXNRYC%Xj)10-4!B_vb+@RAJc(!{C7$s$;uI!wUlgy-i47v+ZJg)&&1 z>3GLVL$(zMdX)syGjoW42+o0Y_yD}-(l;+2rjotQ`>nO$4Gall20-98+y_zsOo+B6 zgytQx?H>|ABf-1CPTa-=qw=Oe1-6l3OP)Bu8u6F1HWQiI{&}GR;lUwAxq#oKEtnyuw_YekE-thEMWX@XS8LmI38|D~zg(prO`FSpm%HQTD% z8f)5HYdh>VJmQ&#Dhu9&UEndvzI} zZ>@{R6K@%AtFP;9tm$gCc5I}qZ|`ia>1?WLZ>sKYtmzZtGS`4t*O`12&SE2zOlN@Ru^AiX{z_9+_zPl8tP1V z&+urgOsFqUs;!8vOAsf@j0&w(qD+^jaKi-bjI7L5b}1)3Siq;yas_$*luj@JVTI4i)7Zu7g(WFjnS5?uOnx{d?5%La;WlFMWL6#yaM|h<;=gF~2$YDP@EDb5 z1=xP8tMz3`Z&lZ=)$wNcBbx@=4SRClDm~b4uBqs*uWN6p?rhvhQG(#c>h6ZBHc%^X z<3PWm+|`zZxciCq<%uAup)MY(+5|W%2#WNrk4h3!n0$tv9bYm={Q+85) zMN(~9Y_-WyVHPGTjD}=2Vz^$K!3z^0#6$|%VVt5EZa6G;~nG(>a6bcAlDpfHldj?6C#gK)#MCp|r=`~vY3 ziOt^^m{wE@nTnu?=cX31WD4DhM43#f0E=ctMUoi2L2$5`703q?G|@qn9Fqnag{mS}Q3<6_6odcj z1T*b?ZG04H#5;q(1#c0Y35)`M9`ClsD8j4K%QLi*>98h9kO#?_J}T%W0^KQSMm~xO zuG$q{0Xr-Zr5EVZ3n9`pwiNb>>@q(uLpyXH9D-6oUUp%yKLm%EEEaxR62^fB#!1tK zq70RquZRef@gn6`NwuF$tAo*kHE3O_gG{PO|C8E5e*aI||MOU7C9~k_?66R|D+b=q zRtJV0__tS=wSM(DE%8lsH-J#QMkrWiEa)kZl~n{$s7Yc^${|pKN;ImstS}G z{?N13ylhBDVKIl5QgC=9F+^%dxlNcoZJTx9x6cvaLL%9LbLm;$Pt9FeFZ z9CXG(`iwSFC;=1TqdG|jZfq8_6~7XsLlMPs2|}$|uV%hrsK2FTLH{9mKiyT~o5B#u z@jTvUsaOjWfuTeycwveNm=YiQ5UoqW>rR$K={V|i{@{8%zpxad3eSmngs5azR2qa9g+39fI&|MC!V7@MwTK`5+z%ERFXMkqH7R^3 z)C2{xsC<<%Q6)(f;x)pxsLjbJxZ!E!8c8HbfW8QIDMmMYh06R)2oCwhA$fr`2o4$S zTu`UwbCZ>+DqV^=A|^_!H0yOq_*HVc!Wb|6^N0Kfl9~7q8nq5a#Qc`#>2c~8>i@mL z&`oM!Fz9Lz5@DqCkn&ZUT9AA^_7D%!z?Qt9)m&X>uZwrehx{6lke7ud)9X>8Fh2BJ zNBlIypF_>ymtcgVurvX?q%?=fW^5|SW)v1@FWYINcVxk41F(#0r5 zVP>PqB-DZ?kX2>&p~&iBs-Q+?Oo}r*0^=2W#3k!<8G=0G63%6s2Vc|5+}?4$5)M#< z(kwY%hDuCpQb`ho5sA_2q&QKmUKeGQ8?;K$7N<;=WMt_C;Se11iv#ikH$iZKx{1=* zAXO+$2VrKTJTVS$WJJ6|TPDWe99JGCtQ5EXHjD8GZ>rVgLov@Du+ZQ>44v8EGLQSP zJ?S)L%5L&pWMmCh7XGMsg6gR8p=iKabCm_OL0UFan4kKj*^qx(lB!JR%0vSggPx5$ z*AwL_`30=d9J;UnCQ`QlCRR41pd>q$T@uD&tC6QGGoqz2aq^7h$aH;JTAVUPgAys3 z2a-S)#M9$F&{v^}c3NN#DkCUE5U+}rdH#+DuPGl=x#Xi-7u6!>yTB(5oW~zp&xunf zCF^4K&|EoQc{B_YE=&*(!6CmiATuNgf&dh;ji|=1=kTwQQ)|DF28KGMeQRG{nP= z6k4~n+>=6Wt8vXKo2x63q%~9|)WpXc`qr}%;$mfbc=DLk!o1uN^s+WFJsF!wsNFL| z1F~45>4nG}*%2umEgx?$hlj37I#O&@BcZgbPh{#YPIh*DpzosOHZh*YLui7DmvLS_Coj#r@X7^Ehq*9}=ih|=*Srf#y@rHVf*;Ymvyt1ze?0%#cMu#6Lp|;kU zSF%^-RjxW3iYX+>;we(=@SwX>gaQmwk=TlKwbBIDaa5Oqp!V9zj{3STTUA#>b%(90 zv%adkwhnAHl$&Zy=89OcJ0B27BLt#}mK+h8!VNFYp$GYq(Vn7_-VLB_$qn&B{mlhs z>=I=@TZbN!pis(T%kzuWsReP7=?0!OUT7>ANB=|6(a=Ogr{MML%nD@HwWcVPi0}yr z)QC0+Gw`()S{KRkH6erj!B(IUWitK{3q)8{ym0Uzpv2>>1wFyuvNF$S2fp^ z+bR<35)DY5VpV!LdE#KmdUjldR>w~*UWYf_ zg)FzQSuvoI8!k!VsH9LewJBN)tFtJQ%Vcqg%TcN-aUTs~lQ{3mi1uee|KTrc5?tTz zp~lM&|CMBsJM0_i@xlmG+Xj-cYlF&lpU+Lp++38i4T6JI5Du56;YC6d%Vb*cQkx*N z#Ty#S%uVGE2Fixy~&C%+Zw7J^>zFi#=m>>U6o#G&@_8hd|ORLdwnGothdn$zuMJg?QODl zHCj6wYC3Jz-St&1RTb9wSc^dw6(><>bt>fKJUN<(>abJ+yBG=vO+yP5kN!bSc(UAp zZK$h@V5p^=IVIVGlF(FEFeIa>M4r!fN-_k|az4TU=u3#!g1d5!ur3O!#!dBS{0rmp z*J_G~ypQoiFF#{~N^3$BLy~a(ud}`DTVstJ62?!4G zSSnoxqCmV#QWq5m0vpPrnkr&jD--Qirsiskqoxcwro*ao@Q{;t!nPV~JSkat=@6XJ z+`yA~><~B4ceU2{+HHN!jXiCx&UQNlhu-GKF1xL>skY5lg=(m!D*k~7sgCcU!F0ua z>oaw|jWsa6hdV_g$X;9w7A%Z)7AcYN_wkb?M)Q)ua6}UG ze`qv#zOTD^;8%?7UCn<$T91eQrm>Y=76b=pSV1v2mjS^61`IE067tG5$u&s|aNJs% zWUoqSu8wcBmUY*bcc=en_P@y;Jhw`<*7mj3!{yO&>LOEOJJ)abx7%GEj)A`RL1z~P zhoKG!=mob>Tt|K7o34`y+`fIw+*;aLTYa^o zwyLo*zQQasB=fZL3|$5XBF)H0rwA=MZe}R@J7{ErRWvWr*K~J=Y2KSb?=}nzd~@lW ziv71`hS0%qVQ64}acEi|3_njQ$O5-wp3bB~a8*>rifa<0m~nxD;1_90_J1%d{F%dE z%n#TiB&f*pV&fvr#7TlA5|HS0q#+sxi7rC`6{3QOOy#8z176>NKM0Z%Wqux~BoKmw zJ~9=#u0>`Z!9kbGPDAf5FBFGpX$sQf3Qb&flA=C7%GKpp zjb@vLA4rGF*7IGn?eG7Op^pdsf(*4aktv^dw@!C7j|_E<54(0wP0Z{X-@Rwo-q}6- z=H?-LXZP>kzp%Klv~XZ@dAetyqZysi?u{Rp`xoz17v-+SA?zdrJj`5{F2;)+-!DsU|vo@Ven>tL6IaTT`W;2sFE#Wp?ru} zg=Ae5?Jl8O{a9KZYPbXyJJ46|s@y;<`&Dib57AMOYWYL=QJ%^o!c?Ldp)^B}Xvz5iR*4_9csZH)8L;+nJsC_M}KGgc(;FS+_ z+QH#Of9Ju8fyG@1_Ra4Q5Ujtu30}FqskYfxZHMW@2ejHc>}Vz1?6p<4 z%6JS_&^}3&C1YBm;z{+o7+zSKTU|Nm8RsE^6oeP&XXbM9O4HeB8?ogAt~51Y$;k@K zFNn!vrxj%ul!j&o287aSr3`vLt5lwvA1#ubleKO(kz2-!G33D*2qmzJ(H%h2F#f_C zA4}*b@pcXKN9a&bgLv>8F~*}2kxqp=OsL~0E7N&!M*?f-rJm#EyAy_x8>2SvW8@*4O52Yj8SjZv+MBv9<;<2^xFa z8arC*(JXGQt!u5XazG#LjlFGlNTgb- zA!rj6ab*S-_(bOe1fV^IZXa50P)(3g#x3B{pstlv`Ows~c!(!LCrRucPZN)VMd)gx zGmK7qjJWkZ{11&S^j9G`;O8uw=xWCAz1)3ufz1n465r97EneUDQ#%@61DpEE>sz~_ zmMw!_&BMKIU~_(9%6VvQ_x)psZ#=mD@bZ<9A6@_Z6%f;FVNVG@_ z(nKj~;FFyjnoZwE^TYVfr!>G{5T2>#rwG+L{FpfolqM&_i)ck?U4b76+E{46nG_E^ zY#TOVpc)ayR0c7qGUmHMJbVwWKs4FqZLN3&`ZtBmCHPeeY164)ay&P`7JLInm#-e`)02-q{nUR{8zwkFVc&{PEN8 zfAPr=zWn|dpMUo0vqv94ee(R`c_{7PnO#f6{e!*D;IY?U+ul&u-B8)lS_3MFI$H<2 zny35Prh7UE`@4o+JxJDQU?%r7%J&)1tG*(-_s!qA<+k(RUW{jr0uke;<1gBsx`SeE;txv#b9xo$O&4 zve5h_QnXjC8=2n=tq0}MO=u(p2Y3kdAQ98h0%&4L!{7f)#t_*Ld1h~`%7reKWXx;* zxAyuzd;MUSV_|go#ku_#R*qk}b^q#<=bt@$`NJQ4|3^Ri;Saz3;>%Ay{p7QcpI*Om z|JdE5=k_m*PER_!huV=XRkzf{w^qkv`qtZA@9Jn7cD5rDjZX~hnwr>i@WA5E9lQ7M zM4VdNy|{1p416z~!rcCYd&WoJGC0&}cXqeB)z)rnuCK!A!=w=^w4!8DWSm+a9U&D) z=s=4)CM8n9O=E`_mWJlOG3YJtVs`JDO>E18gG?8I+hkp2vd<16r^J~vP|Lhrf`|(fy z>7Rc3v!DL(%b$Ju$!CuqKDc!3I4D?}Sax-QIw+j2BDS{7Y_Bcrve!cAmIrzdE)Om) zOr4$IbLQ;sLofEig{~Ywb8Pj<$-75h-#v2S@R3u8UL4sw3j$w_PwW_(MpiI7*pH;k z)z#)`sHu*RHLLV0xJjN4)WxZ#T4fB#Nhv4-ORUWyIbNG!xXkU?QP_@iVR3L|YCbQL zFNx%9l^I55hQ!TVepo&`oz3DGU|)oTAp~+uba&7}#KD}bs+8AJqx6hR>`Lm_`@aY! zMKOLBLWeOM8DDUy(K=#Y$t>h*@C0%uWd7(#G%~_MsLeI|d$#wJtz_6Ozn|nJ96o*+ z#x_(FPs13S!~45h>PM%0=3gz{y|;G$#Iega9(?@r>X%=B@zWpu>_JeiPe1+i zG4${A&p!VA@k?0jrBka%kuxq2On2G4ZI#w|gS*?SkXXyinClMqwH+K8*!^k=(dWYH zL*NPV{EZwtf8p4j6UUrycsqIL^$Nay`pCW`YrBumEka(Ln}IeXA%&A{sYmZ$6)h2I zb&;Z&l(4k?w5+0n%+fcc(L=Tc5kXtFdHV(VZT0eIh6d0JOM)}<3nJ6ksVS>b7H z;&U&WAg`26k!9v)6lT6vJIe;T+hF$(X8)Zj4R7119wO7=&}N63UkGVXOS#Ma*oIC9 z$+~}lcrSodPl&?hE)6Pc6n>~ukV3ndGQV{c%qadae#M9f6EMuF(7}P*SsrxmKRC5- zZuao;dzVg}ym8~^!)s5jK78`@;q{jfuV21+{?zfKur&l|1cs(6Q<))7f$UVPtV@(( z*6?KIqo-2lubsxpeXNgf&F2*66_o@RXNPQNc=>PgpnH($zWA!r$DFr;}X7OWfBzXM|qYdHpn zm=-ovCLre>=-v=k5AMGPVKWCuC;L00 zTs83qb996RQ%-rh#SqtAUq0F2KC=T6dhx`{p(hv5Ke~S9+LZ^-AKbop`vge4b^kSf z0|dN56K|h9ed*ZYOUGctr=gO_DZ$&W@#RiOtI3Q2D_3Tu3JQwyibI0~{X+;EJ!nf1 ziQ>!Ly3LzOZ1N7FZwd_{Siw1knWd=(;i(0AP&?LUMkui-aEmk-KkgFO6@i^zC7iU$p(%KLeEgbkxc6r7aMz>cAV`B!f`tkUTj z&;bu07{T8ZyoM_7NoGP`G2e{XxGB{5E`R`S!h?9ANF z0SvDoFJG*XQb=33Z=sNB-hKgIL7^c;NoZhraVYr9XS1@xi!*}*LmAr`Sj!3`l`^)m zg9EXJfwdm23nDo&G3X4YcMyMwF%>Z2BC|j#qNtWpB6<)1$5&W8guG=MwMGM@nT4Rh zI3;Ja`w{s}QiINeM1$dG|4oFfc(Sh>Bn}Xf0jx%-CN?Ns1M9!7NW{1v*=u8UMN@5E zx4qHj>^iVKxo6MLxpTYsot?XPbZzy>i=%tzcTG(q+ik5b!)RNh7ipDgk(_XlsL|`J zmZ)BP{nVHfrtL&PeSYiy)5|xX+`I*XZr#6tcK`wdeq+P0!Qah0mu}p;efj2{Yd0=l zy?*5-((yZ&UhJJ88FMw&SDVGi*~0P*^Rh#TZEsnmkSXL3nD4%WFLM}PtlZ6DnVne* zA`9|Li;A;LbNtaOr+fP_e3``|UTE}Urhr@+?hFnNY08^{Q?!M#lPMo!o)*xcgVqg? zJs*b#;`=Btn4WZb8#P%@Y@y4iZ|)ab&&rI}Z|&MofQziWJ~ zzY7aJ6=sFlsMAYRuo8pK53J2VAF*JF6)S`fWZbJS=5F494NZJ&$zbpP>r=Nc9D^W; zKp$_OcPrqN8yBBmzX8Yk_|e19pFO?y{N~*wu;mll8^B;itjw&?M3hCvu8-v&^XD#R%kS1$=I1YEdpfD-&xni7G9&Px_i0kUXN4 zKL2{<@`Kx83VeaSOD9(#I6z?3S8m+~eUGkPe#70P?|=I2hhKaO4o@6Ay?f@>n9FIa zsZ+&i1za|4*>CGRTge}isUJ}{Y}~s2L$6KVrJ;c!HzhTTQxFbCWaZ`r2NEGrK@UGH z7!%%kTQMWNm}Fm41@}9tdXUVl^-{NzgXqj%eu2cP<{Oj zLd+PLPCt?p)Jc9u7#l-k5JNx{tQV{uaP=aLSnxI6U~fw;q710RtHvNisMYDh(nU(X zB3f@TB{tgX+q>I(dON0F&XM7PX=fLj0+_~0BGa+x=3b~_7h`b-%Qce7RD)iGeJ5=B zP4;)q?_4@|oOlti?n0QjaO&>K)2nFgoLsqk;nZQUbNj?e^dp`;zxndvlTSZ>`o(9@ ze(=SoKLtL13iUj@dvRoJ0P3%ZGv*hBZwo?d_t85azW2dL|GeS7pMyb~FDW-W1l*+y zva;B$%+l=8H^ljG+T!KyZj{4<(H+B?*Mm*9bIHn{R5Bt*i=eV z==JKzu=M=Mj8tK=wk$rOrM0QAx5L%nGdw*sIynup8f&mgr^W7ED&vQ$Z=7lDQbDpZ}4(rXH$O8uJ!G?1#Tg^m;lrn}*F0v}YhVpzn=wpsB)-)Wi4kZ#-p28uD>LR-BPnVKKEi>_h#|!GWQPv4I1C6EjW= zt^j0k^RPIGr8ih-N}@zH@v8&DL`>JLhKifQq$qvq$&s zJ#_EIVU%a54xhh&;l{-~51%~$_|d~ppS}G4=b!!T%P+toINiOtXVB%eR+Vcbq^#ml ziYN7Zzx4dd3}a9< zbTl6RZ)OdX>TD*e0_1(nKZY^e0yt1Tq3$UvNYvr=! zX>9BXYL)5H>KL<7Qd1V&-daCA)H}Y6j>^Qs(&(#QON)CJK9x;3e)PqskAL|6PtXy2eeCq?%&UR^zN&JINE#WO9rDfxzxbD5ef{fy{f%4x zbZ7HGJA2AUiF6q(qq;!f0F8iWF{vDOW;&-p6=y_n80znt8eJYgu)K6&`QX&@ z*iavIPo+gqyAgM0<+u!tL2Gs6mK2i~K;cG&rts|PkXIx)7e zV+v%fojrH_(3z9RPTx9z>H704kFH)r&4=pm%P&61`PAvvR|^OGy4o$KBxP8}=0N`s zKH3PvzWVhy&^@=!ez=k8=jpZ0BQ!ezbo$dbd2FG)Oa5>x`GZZ~J~_muywH#|HcOp> z4zUzXP3C_qu)|X}M3zrrYsYZhy}U`h?ryTwP6l{$U#F$5cgs|Vc6@N2DDIGs>_fW!_;81MqL+p4j25Sdz-{+IP*ney~P5QY*%FNLJ&br~v&JV~a;pr6Q_GY4U+n^OFx>kmUSGa(^U=d6&mLVxzw47vA78uv{P>YG zc#9k@O=gWOBV7;}Ku|w?@0VY>Ep}w@{ z?P6io0s7X-_*%&Py|=iN_^UVC#$*rVB~~U4lOduloa7P@kz9Po`gbTWS|LzP%*$Kr zEZFG=jj<{tx3pC(9Y&?*)5(@1NiI-2S1*79WN`EjKt2 z!9tK<7{*TvLoLT;hi8^T4-s(-(LTZUR!mx)DAJ&hz{XwOnfqLki7GK2dl`*<7rn2= z<`lc=7&-P6f8p^vY2cCU`wb?LEw;AWIt+BH6QU}jaJwQdN@!H*BNC$&j@E_)Bg04c zon1M+x_acyy%%ROhI_T+Kz|3u`iTU`w0=G^4csB<8j?j(aU!u%s^Uc=lz>wqCNj7a zCDcL`$-Fr9OVn~Cn1zHaB`waoJ#4D8U`YsbsxmXS%@pVj zRacmLI@)&bm;z}Rj;*Y$9y)$z-_gChQDmUc-c*B2EP9x&82fWrq*F+eRJdy-k{RXE zyf@mY4oikaL>fh5JSkrknIh+>@pNHPvgqncOLsGNnOvw^j^BH6>-LE!&#yeZ{`~2a z%Sibi+`cq7JLBrR0M6*V$ zic8i;sHKsp>hKn6qP6G=VziGDbXB4OGYJ$6;Lojwu`>1FXy3Lmr6Sa0Lvoj@pBkky z^^?=kNWhLJ1PA;q_LrCgoP-ftHHtG2Mj#ekx|(XOWUNN?*c-5m*5AzU6%Als<7mRN zhMf@^aUBb3HI^i2O<7`fMM8_MW^kx)|H6*dBS(?iVp@4`-(IxTp?TO}Xl|&kAbu)b zKc>ZP$0UU|QHDo}Ia;kr)+vl~+>L}r;!a*fWSS@}^^K(IBGV+dVrx&~K}*Gbo}Xbjq~+f|A_q%;M0P6n8dS zZb@jUt!}KZX|>h1G}JfMTWhPzD=nt@q$rbFQM-w(;=h;uE5c7BZWbKV2YHg>8Iit%Fq3?g{S&cbqo)#~Lxx)dSka7@5HwG} zcr(Uk>@#taNhW`Y)CoK7B}U=-dhCr@68f$l>et(fxh~dO>|-5mgPjhXI~!~5);j#v zSPsBIyRW?&NeZ$wu(tYU5^>}Olx}8z&#qllBa<+y?$*}YO7K``!d|H=L8DKSNs<)2 zSOw3l!i_Gi#{=MIie^mI2j*3?y(nc~d`beK(MLvy1I)SX-0e{b#VzH_^A zSlqer>fqAC_|%TkW&FFA2M5NSeM6mXz4k`;aIOk5F%}tFnT9c~XZS=JdIkSOkesKU z`Jqi*=J9OeZlh#FiDCOzZLfkRNLMNllm*#E__;8ti3{o30De$XCM4W8>$Gwi$Av`(McVKcF+J_h(sWz){gP8|fHE}wvG%YMOJS%f^ z0V^jxs{~hVVbgkb7`6ye=AiE`l&EXUO#Pkc$sD-1clW77XKo)m{rt{-aC&HM?$!9{ zKyPPzYlGcpZLF!XR#vpw>W5uJ^Lt($ySsY+!t0ZV@7}$4=IGwJv-8+2S)7}h!8*ju zu7d}cCr1XH108K$EjD{id95WDnG5VQ``65a(Q*eViQ&m&-jAUsMI*_RCX62lgx7PM z@t+egBFjNqn2fD|40W|Sw6@?58!CuCZ5!1v_=zVm=f#0BL%N#yz3bm~*TY3(=+WSW zVS~!y-u40LW?M^FOG9sSldGd`7(5OSAb)`lf=w{EFushW(AC%3;b=v!*IHWze`u{V z)y13R%__4(E7MC=T3kogMF}I4WVq8R;wdrKVTI>1GDC^{@X#3Sl|-gOHzkEV@ zGgR6LQ0Z)M-g#i;#qPz!XZBq>y?W*Tg*&fLfy(_mFl*|AZemlRqq(Ki-a6FV1qFmk z-Z^pM#;rTIU!Od+di=<~eJ|$bUd)|)aSpP3@67)FJI0qL2Pd6f&bC%8MAlo9Kvkm9 z{LAcb<@JJ{0;VECJ}S5#CtvgQ2!wjNTP-so%!v>h>R%?E#l!!Q=U_KhqDdfJTEbsS zcfejP@3@BDy9F(N?ya;2eqW8l3QetsdJeUSu|S#VYg)y-qLU>qRLlNhCZZLJS_& zVH^Rw6gPc?1A^GaA!%79Qa%^A=6T8#CwP=7`7$G~rYxbS&3VSSBWv@s(8MPlp5mHB7QzMGaH{N5VDO_vYDPA9*n?) zCD#Z9X=R319wXLEYAgx8&8_1I(yw;io0~s&=*%6AHcuS}jq^JXP7O~_xSR)urw=Yq zyjq%?Tbx-rbmZpk`_HaE{PfYICpT}NzkvDOq2ou7-aT{X^pPW{jvqgK=ID$0+1Xu7 z2PX!HdpbLsYHekSFi_|m$`L03L6g+oCyhxaX^gsL!;o65V9tcQb<&?t!-!WMAY%G) zOI>7Aw`JquKV<*?4;#tNHQ1lTO_y@GMfk+&P6r4BZNnhX*|j|2Jh(iJowc*``{4nh zekg0ZTN<>7mJ? zp7F`Co#O}QcP>D)E}TAn{lSAzKK}TNXV0!afBxXorDLnBhwt56Iez@)-Mc4FpFX*I z{LtAs{I*vICPw05QZP0-%mq+`0WVEG3*>3z%sA%+qU(5a0m4Rm92HS1C48Z*yqE1uF2Jn zWwy4lzRpS5@R;k>(&*gG&LjKg5KrI=5lP?#YwOAq4X9V7{M7vP`~qoAx*oec(eAPX zC5UVN7+1&V8g&^m+|rR5afe@;!7T)lG!RM24WSjVHfQ9AL;vt04rZaT%q(uGtLSsI zfGw!Y(cO#CIS_f_?x9mhj>4)DAP`v)cP3rV9V3%72Y0~Ry*_;R@`GESzkCTIpI*Lv z@%HW4hYznDI^Y0$d9b&AWTQVi zwT!r&d&Woi&F){_xA*YTHE;&Aee3s@2QXV{MKpGGb&d{BzZxIkH$Q)Jb@lnJJOAM9 z`t>X4&x5LCcU67$w@#e6eeBffGcR7u&MfXYZ~#f8y#ah7M#vJ{PCdevUPS~uBgH)+ z$E1jYTCw7URXpt8vAIQ=F=_b`@)(s- z4W8n7a#tV+9{{2ksa#ZXvW>jH10UFckLn0zY+@G&;-fuyk?DvgHmpl^+Yb!7K+oFZ zp1Uv3f*2?h7&?u*aQEWEfsyInPJ4Y#Q%zN`!#+9W+%-CSc5xAY>&Bh)aB0x0M_4wz zbLalCW3X8~-nww%%I({?F1$W==$`Z5x!L($Hb$! zBWD+19lp2k_Ujc4s9|}?KoC7qq*;=pqP6OnG|n5dn;O9E{Z}8X+h9jXF0gAK7ZHgM zL&2w_zz)Mw3&V3m7(q6^du@lepG~&0MGsvbeqbcOF4Q;Lw@Zt4F{MRObv{ z*usv{L07M%rM|MPw%pR*+Bn$LcW`-m@661pLx--MSGmr=eDdV^ty||`zrJ(|yDt|m z-oO9+&Yhc=F1=nobYyP#xrH4|V?(IfaqlTHTI*_IWCmpa2_3zKeliUsT6w?wQ9VG# zJ>)@IN59q-> z1`9Z#2Wc97d|RunraT^%lRiR+xG2Pyn@Vqrl38NK2?il^hVD(I7Od~c%w}Y8!}X#V43DsJhmHMR%fkqU&=!~%ID#BHdk*Tfc+j0?xw_gN z=sv|K;GH*kFfLF9SAXODc{u+ox9?xLyNX{m zw{yo1B*<;8O_lMu+JO93BauSH5~5K{k`e!<`@&QGdB$UkV8DwS8;8LTRV;B0khF{? zEmf3mA*<)qxKZNnnP~eHd&wYxjI2N7iC9X_)L zrMj@PD(^ppgaYooskRCUMUqB=oC2#N6)2+1@PP^m)%a`@HGcX5}4UAh^cw1a*i7L$=cLw(=6aJ)Qtq?5<*!QNiX&&=$YLTUk%>uha+ zv#*FXR3^qY+iItKyLOE(ug%VawVStZKf8MM^Ea#^qrjJ-3_c2O{@RtB&+nYSbo%b< z+T7f$g9j(4UF~*zRT-`aq6uQwGin5kPBF6?tTDVj1u(8YW{m1b9?SthSfOdy>qiuW zi!lxF#A1Xd@Evj&3oaYdg*%Sc0H&8`ma83W@O=kI#uj&tWAW(jnY9z2P}YlH%*~<+ zg1Fby=|C2N=s=X1h~u?&LQ3O)gd z@bJse%f-qeX0;vm2834F_`cb%I@ax=2Yv}W!AC*ac3^1K)!yD@Ypg1-GbJU6aZ}#V zQd@%@77l7{ejW}Aq3+{{4?lhR^5wN_&TCh$z)686*ds{0`uqmyy1#Px?!LWyQAbY< zB0WO47yU}CvZHuUie~?vVE&r~6MiO5swR1W9~KtKkc$Z!NVdFHesJmdz<_!6SLlZ@ z`u)b8DhG4S3!4-53|bJq0mYMi^cl3QtJQ{7VHBQxd3b(d>Ba6D=+=pqRqU6*DA9Ry zS4}+~P84=+E#R)LmH3@uJ)@(!skg1Que$}0z3vZ5X|mT>p;HXn`1ytG((K$2dcY=c z%sxRLwzo9NJhTk3iUt+NUxUO7igoxXJ`JG~rnz&}?Z}(!YmmP}s}f~GO)|Pu!iKua z{`T%2BXFgQ$VZ_`kDfgF?9n4wJ2NF8vwo*gqzhJItZ3Zm z$sZDr{?;W&vTz^y6a<8!P6B;MR*|VrP519SFtPj9!K3FE;l<#>5dH39-1q9>IHNCM z{WfMWgZ;g5_z)b1U42;98JTuYU^X+t62ZU9}L z&OuU&PyvsKzvbMkgWwBp9giRjz6(x&tfvcnL9YyP_`sP6_ZFu-QlaHlnG!neEeFO1 zckkJ=dh{q<7AhoIB;w#3$}WPnw<`AN`sHgkFWxzMY6S_y;+|Kdlf%8;Xk36Y=1YH1 z4?`h?okYiN?+E4zKSU{9(BQi~(X~mAK*TB#lmJ8+q7M= zFNEn8BpLR~{+kkPp(uiYLdF5+vj$X}lRX_vhydfub9;83nVZF`8|Z^C!!9dweN^nY z5ahlSG(ItgdTD8Sm?#shpC4b|v*Q2+B^{QlcZ>|8x~i?f^)8(N)0G@XZU7PLzYY7U zMY)?%3{{z8(FZ_E52v+v{{kW+*aLNli%2JsSm9PnIRbHn4x5?MlvFIhMy7JWVSJRP z*;YU48d_Y~bz}{0_7tr1<7?NRUcP?q#tmp21kWJtfBEG4!z(v$VXkrH=$ZMM{b(!o zb>gDqTb+w%S2Ib;Y9ff5C?x(&FqQ)NE_js_UkE`d%%KADuhrE|e}_gzjB^e)-xJu!jw*o6tk#`mo`vH?BN5@fvRP$hq0Y zrO^pzN4u>W?5V#F_QU5qVEE6;^27ak3>t*#+EMf<6YU#FChpcPNXC?yi0N_dJQ5#R z5vKDF@bKgsW5FjX0gWe*4@amaOsAD!diu&WFxG?qVUrony_vnp#FA zOj1Pi^9zb_v8gl^yV7WUhY7f`syO5oQ)2`33kOjKo}JkP29cXk3`UJ)vXDu7Vox?z6X!``{4SW3{Cm6Q9p)J(H9@7q z6o)C1((q$UutjDu<9wM+2FtyoCrhpt_^_zeWwV1^n6?+|%70Ub+W39+LDZCtp%71x zUrzLNj1La&9G#p+oSofs_vl%KjGO0AV2Kb*m~e$Cm~d4JkJ!cpsfZ>paHK5!e{(Re6F2A7+LINH~-hcO-JWXrcaIATo6yyMeaL4=z5x z_29;xTX2D}VCdk-uDSP~-?)AM{>t5zqibhz6=9$U*J^RkZ@7YnPrD#fh@dQzg80*R z`6ym#d-k8?-Aq2${h2z%R7ea@9*s{elJZfB;?rxA87t9r{6C&C6{7YDW%_cN(*jZ# z;ln32ociN>yt^pxbbv#&E8!?0U=cEkgk^Gi&(z3nRGYhJR*t-IV$JLN%{$oF#C9=O z3qdJ6M!$ah1lHtE{(rTdd0dodw&$y$fI{IFU*Q!L5s*SaK~@z6v_Mf1K|vO`U;)Ki z?E6l!3JT(mJMQ~#)NC#>n_2oMdat*;$6aG$dU7w_lU}Afoy_+w(z)F|^Ur)fGv-NA z6h-mYsq>uYEWh(RAYT#m8}Hn_eCzz>x88o^{G~TYGAUR*;X}4k`Z6Na+vubW3@PGZrZ3hzFc?qrJPmYprRfPU?5I<20T#RoLldI+h*% z3Ey#k3Cjgbo1W^|zw4=1vv0j?hkxHidZ@LDh2V(Yc6F`8401+tlpH&C8CSn~_j+Ls?0V zK0h-&YpymzOWC# z*IA~piydndy>h}lzrgIdq4kxe``Y_=4D2MoL7;W(66+ER{}^p#6C=HQ*Dh1lxCk)v zz>$OJb`w?9l;&rL_?nVN6M7^yn^T&x!2P$Aq=SM=3RN~+%tOOe%HV`E=i3v2y4<3^He>-YK^)2|abFWWuw61<|@7<4Xe?q(LyZ5eq@X5O$Ub*`5o41ag8rnLrrH^WA zO?6F0X<0!Y3dSbperHSbrhUEWA*ONDz71?&ay~?$Uct~{5WFs4EG%1=U`KC>dFoob zq|^j!=r!Sia9Ff(jo+l%pP6 zWd#Yi%Ns_SGUvfI(x>CpxY5$iY3^2zE(iPsrHfy^D)xKt=8y(*mS`JmeP`RoeH+lZ z%^TO_{b9x~-n{VM$6tJK`^twbq$saz^+yO`U(dX}h^!n@$DZ_j3fBxP_x8M8Z z+86I!IClNe_RaTqkq{$$(NbMiUqY*+p{2fILn{fWb@(s1D*eqhlo8mfTAx`{V-`$a zGSO*S+_D6T?xv*~kCkgJ5=Vr&G)Zu~z0r=y?>6`lH-qa(ciy8T z4-BiVIQ%8z1GMzfoqzf0!+-hU&Zn#oKfm+IyO%GL-Z_5#^!78`Xw2P9)`Os)q6`rQ zP7JvQlF6;L4OJxt1^Udn8d5wk20|?wH`;uuof4wgwW{UGET!4*D#hzaX4_14DA{&L`}eD|f(QTzliaH!fa$aDuH% z#D?Q5V?!c?kQ$A8ngG9`d2nYz3x!_;;g=7G497Iy&Oa^5ergQo&KgH?foMbKPA@w8 zNTc1<7(*BHD8bUl{da0-ju1|ceXZKRcsHxr&kp8_P2R2F3Cf;s$|YyF?WA)Vgf$=s zkcRX(esBla(&vDIxgBZx;g)YfZvNu4kN)-3kN)+ukGS&-I{0WNy-HWhTkjlwa2Ox~ z8SqU#)PXj1G!wdX(dgLLg6EAiTXJ-fLB8bB>1A3NKXw!-I@M!~REuP@hfmf%db&0_ zMHXttV{@wv2ZimD_;KzlCQO^^9T^l-Rax2B)l0SJ=!p|(=B=BTUVrEOHB&9;_PbX< zc<&Q-2hg++KDqkAyVpK`>+KUX32*8lY@qgBqSxs|{X^%>^q)B`h#+xFaD)#`OGp&< zHHa2NAU~jb_|p=jeS~k;h})-ynVBmhq>PCQPoZpq9hi)wOgE#{ry<)SsiM@y0H$)C zPyE@t*$zwD*JiA?N2>cf+Z@c@jPd(AxAg798{R=}>d;w^+ArR`B?xm}!7*{6AKrf7 z*C-sG=QiMlZhrOwI7FHum6nV6`sw?q>7Dnku-6gk`3u4)vG!Yc-zSqS)J6}VCl+k2 zgY*DOiGr+ftq;-jY-diBcrfdBR%@4Aq`b69c{Z|^KY58@RgWJc($Tg{7mphY63S!x zjJdcvr6qeeY&f)a>jQF}$7skpiiTdk$bt99y{njUlRSa^fvCT7^<6@~p+o03t|w7K z@L!_OY{}C4BT6q%KmVDY+Bpz?%?a1Y0xb-MwGSi<%(n6Z3?w~C7hbG^ei8W3gq~4i zMyYHsIe`>2g&5I}P&;R?;2iGWF%C4J0Cbf6q{ z`ph|iB!Ch@*MW%ziY5K0o3(gkiOD!{Nk}{2XxZx;iE?{UnH|EEQgH` zpFT&B=w^n+BU`1F{n$}R5v`2S29d!-ig&l}IDDOG^d!yc=Z{{v`1VZ%X$qm=mG}9T ztDj%JcKhQmzIgj6$r*|gRJ_ac3-ieG_{^evciz#k_QuEcjOOfL;I#Z$1v`Se~(mi+38bk3i<{PN|iXYT)hzQD{zDB@SO|d_E8&CYz z+S0heu%Vyi7V+=S{XHOiXpI1Jeetcgzj*!D)oX4Y9-rU)gx&rjCbP5M5uoCy?|=9& zn%Zn+hcH?1e@I6BT_kwz&0By`4jiQ2=ww%0b6Zmt-L){^LLf?te#QcL2PTdi6=!Le zXbl|n$;*`IUq4|zd-dqqD_NFLpCieKDbJLtYsZaJBSTM*K+=!(q#k-VY~OM2_^Bg@ z51e{%^1#s($KSs2;LUev?06Gp{QSz*dsnY<=3D^-bKx)%B|IF+Z+37aycT^S04rdN^cGMp%-x+x;XtSAaU zYdq{J@d8)#HY)0U%od101_u6M7SbgBNloI~t!r>y8~LX_y;SnL!8Zbr0{sfS^VT~^ z@Z!54Uq%PP5Z}?X_iS>c8~oGHKKNAAgyMA5y=cvP{T1G#4MZ}6gp)d-S@QKPo3{h4?exnL33tCcwrU)OK^~ufFmgOIglYE zM}ezkiPruhMq$uTF-KT9RT}hDg9VZ%%$1N@$1J;w_L_#ys0OQ$T16l9Ft(QQ{jGiN z9ewRaVp>}Fc5mEIN2IXcFZjfPw~s!+wt+0Z^6_OMJiz9YtDoP~bnC1fKY#zDJMVw= zAxXaY5AMKHz~-fAfHsR;6zqZaZQgmmrwi>vgyltrpuV-f0fBI;IgTHzwv?78C1oss zoct7dJ^SjLmtX(JBt=<}nkKi+!&Rw|AEJ8EE-QTI1?TItnyc%<1#CYzc>Un%Q#TGB zIemBN(B10?Z#)2Yb?L$vx8A`3(>T+D&vohefrDpuP>^gYuPBTNMeBm6&X~HueZk7f z3nonVj)o-yjH@3~jP#j<`T^=2v4fym+)rjm`D@_YgQ!6IwcSc^jiqhXP`>Azd6%p4 zvBS(2-?=Kg+giHYTKDyL^lWIqzrOF>?)!(%?zn#B#)A_l-?@1dnZa#9NIE-=b>$7kIWT;~6XpjJ zdh(jZB8@5L=1CbD7Ejo!&%XNaCaEE~Z=SyT&65{jJ$m*vI~TPaZ$HjEW{`3}eUaSIpQLrpN-dC{~-`*Dqdv!#g%#MLw`@7fmlXEG_rRR+z6GajL+N&3j6XFtRc(i() zjQpN``QPOQ1^4=IpML$#ljmPOeEQTnPLbe3xSV`ydciWnRDCNoFGWwx_R{V zx6Y3&(8aW_Al|FkZcwuvg!eva`B0c<^EG=MRlSW=iFU4Bew`%63AW$4fh~KwI$LY% zsm_K61D?(0pxo;mQPAlW>_SHmg)NT`IrCk{5P!seEHS4kDh(?_{p=><&WYN z=IN^zYUa#rC^q!Ab?n-^59tljOLO-Oem=)vPgiH(hIaH|=l(sgN9^83fv3HLa$X7a zpRg=?r>}Axryi4#m?SAxwpJNLO571F6-%7PE?T%6ZFQXy}VXxC#CRwMNCBJ<1{HsSV zzI^iHH&0%Dll=V4wNIW|J${s^dMw6`rh7h8;|qWQ8w92du^qX4=E(M~s9p+}1j7@|ATa-rlB`v!7EdxBrr^*&e)1BBo|u!4^VRX=Fgd_{ zDTjd@=PtcMIOfMYOnimy>kso34j+MX6bx?`Hm z>}$JPh-7pYs_qW$_p(H#c8SyaYYN36SlWREk?kR9+}BBoc#u^78N7>wL^pCLv#E=e zCcXzJuG0xgA0O8B&9|N&w1E)kL!C-EVk=yv74xMgD;W1Z%&m^^J#6a_O-_~Fd;2$`^dmYN_@I#Ns{5&50a*6A&J zEv*btJq3{eu@Y@49#?m^N}Y)+I%k^FMGI@cdP!)K=-(RUC~T9|BpXG^k>h@N9xX!6 z)$3a7y2xhqwo`7QOhBLV@gq0r!v&X2|M!EFryz@<0*Ih-(K$kIUO4vVt@HG~a3BdE zC#vt<`$&X0?jHW*`k{ko2O%Wc*VYV$LP0j%EMy=3&{hvOci=8i6vG^=RwOcwMq!-+ z^{14ymOJ#6GS!Ag@yX-uU1vmPg@!aXRFGIB%S*2Ot2D99zlL|%^b zI?~H`4jA(V?k;I-mf*pUQP?J>Vue$lf0ZJnS`Yq)BM)QEIhbf=W@)$7-X@NweyX{? zo`G}_gn2>BOHN4qeYQ=gSMFS$%ARu;e$~% zbOOR6?4D5N6#F~xCSkxy)l}_ZZbLP&U2ybwtY`DXVJAsJ@pK@;+d%4@%q`ZC@>a`n z(S7FX(`V0tjvk`3^5CHp2Tsur$YF)UDU7uSArwy%?jF2OphHh0)6JapaGf-f$j*wS zK|RzrfL5>IKsP7@=dWA?ZR%oZc^Rc-Ny1j12tLN+TS^o1Mh66XWGeJvBAQ|aa~hA z1cCi@-PCR{HVa;BUAxv%$)b97T^#a~c}E_c9HOQ`VH%(j{wHe7zN4Yz!r_ysC#U^~ zx8M8lbHUHSyS1-vKKSJQdr?c6*bpJj=#gSs_3+^-uG484hPTdn@@ve0dCiHs0rMZ! z1+UwrxK>8s_BUeYWkEmh&P zxleEmm^vFu$hq|0n40?0L7EOHhr=msYIG1?q&_Dk`Ymn3`DQsrrEiB@b3h4-SKnz1 zRHU;>aCDgsa<{*(imeSWfjYCWFL*e*r)%r(jYkdv&N}t>u>;4h-?)DI4E$gSlu98T zsAn1)|5qUJA1m% z?A%D1qubb0Q&~*VEIbjy{iecq6}4glz9eNxVT|y2RHb$5V=KC=EuJ7Lgvc#{z*>$W zizjO>Qk2$fA3jaABJv!wDD5?9g*mWbCnIC2*Dm-AM9M8R(}Cjc9VchQojed!x#(;Sm%z^Qa7GRFj#N;>SgKV4ocyyxTXR?*5!Zy=!u9ST6QNr3fG zH{aX4AKX%PLj_GuAwGC8u9ISBr>}5x3^w$_wa{`5{m9&0E(KXDP=`E}#>FqI_H80RDc&c*Za2?Yz zsH(#{1d$I)*jitKtp@4>%(%KZKQAm4HoBSKv!_S9xKCaK@!v#2J=P(CgVe~rdyhZviyb*kaN(($|j_Q^h( zG;!$eg#$N^A3S?%`?=EyZD7~HrarWi*auJqnupaRbAjevz5M!n@5+=9RJJH#LtDu! zK==idDl9T|xJQ6Pods1~AZj-Y#!Pw)r^yYRv>bW`c(*{?!Dt;8&(JU`&viP?31OOf z$c+qkgCPf^RbY)ktGKNMt`+1(HrsXf0$3?s){I}{B)|YXYP6lDErcc+2#s4Lrm$5) z(g8IBd?!xgvazEa66}^*VH8tdJbw7pA;BC<>lyQ1AZiYpw{ADxEVO`VkFAPLFey@mz+ z_}$BI5)-oXcgkl+BV@{Gk3Lev1R{8f{OQ!3HbWl5GK~Qh(~%#K(8Q^oGcAz4h?aKrs3XCN@uEEzk0y1iwGacCgqYZ}X5zR>(TkWZ!mJDoAlfH~ z*}71|6&8*-rwQ#N)cL6t$o&ttjuiZ3Mmtc_*I6j~MWsh79mh4H##@R7KiV5>%UL>$ z&K}pj?X3gwcW&MS82qmY^1z`TP!1tNC}ts(YpgDBt*PqkT{k#zpMFiM1-H)MB=Cm1 zg`?xr;RB}un*-`FHa9nvl;&nK76s!M#4I>jgp6jK?E{fBa}Z#4mZe3`z%$H?B2#b} z%Zsycrp|v}meyC^P^7N1!q7&DsVii9`EWvDcuuKK1x$*PMF?GvxNMw*%(pSd&Kg!7 zyQL}!Vvrxr>S_mtD8$k1Pd7GA>}rPu+SeaGdiK~Nh4>H&&URlB%;=_Izc5qwa72P6 zDp&|HH}YwZ+^xitwvOuQmux30O5`H{s4?PtXGMi%>7nYcbuxFWX_83Sw&Us4`Wkf> z!_jQ)ruy!-M(A%24Q@e(mk!)GHnjcZp&dug44xU-1Vtp~5>D9aisDXV3lSsz&?in_ zC!m_a-Yr{aVPzUp29SKzvttol`z?n0mU_pcr9Tx9#V-%wj zY#r>DCYUFY*BEUlOOR%E2mnw&y0Oz1E?NEBVw+K8%$KSlQ;>zINL(E&x;d|eD_p~{ z5fe+SK__f+l4e({Vdt0n?W7PlnUGrU_;Pf#;Y*=7Wr1`oUE25>R1+lA{A!hob)qRz zt5Zg_>XlyAd;43^z|)(z9MiN8#~(g;_Q2h92X_qYxKI7v*ilR z=j%f^=o%)pCf1`U2$vcVP^j;y4-o7zrw!)?z#Y4_MWZn;WdaO)7u_R@LhGPy%Lt+Ua|cUVbWY<&(X+mA&BWKn+mBLPCjPVNCX9$q#@mn7;pXMeqs|lKEUE^yl2tM5ZI;)F}T}X!&f9&jIrVPtz zLmTu~H+BphJ978v_0z|1Y#lnYd;7MH10+d}&43RL`fx*GW>bB6_lEYtfh`A5Z-?9k z5}|8QTv1Rxbpw3%rtWp^jn#O5xX&<*VdO%BIV0e=m8U0hpKXiiz-pmlg-qLs@Iq#= z2Qp)#K*uOVQ_jz5H(bOhskta#9=jTZ*xJ%>vz@fZR<*84=&n0!+U_7$-*9Ba<3X~1ZI`}R@^E~`#3fl_L%LIX*)3x(F&^;87Xr2!@FG7Rp47#AA zOr}N7En(i6X-PBd?KngXnL>SoUA~SfgWK8+Q{gs-e$z32?AXz^mdqthGD}Q-ti;nZ zv+(Z7W831d#92x(V{v>SLA%5xn4XeTp8K}Wo9KkagEWDUfJ!uDbEK~#rC+N-o#%wK zOFsnCaHNL1+}P#^wjZ(_ay747r7m0VGDV9ib`u7rz%EJAsB}S@|zwi zd=)rd!-i#wQ!TubCBlo1VWI;TI}AYuJ)aK0LZRCy<_Le(>snHIE@Z+{O;JYXIJ++!ZDa| z*};$wglWCNvjfrIwP!C(Rj@Syw!?XTo5aobb1j1#DHpVM)K)O@5~{^Wp8$*=TUaLk zHqG|}kd^5_2ldMjljp*WC|)JABBZV5igMvnts+x+zjrrPpuw51X4jIC&AnrVRD>s-#jTb$!t041(FH4 zgXF`~#>Y>e?*b<;X#}#`?Ah8PVa;Skv-m{sh3=t&P$!n9Pqgh2Gw0hd;W1pyU8u;E{!*f5 zvVj<0uV(t4FtdI-ei7$6GR@DN>)kq>L@tt~8NBaI)L>IvuT8eHc+!%QO!D;^BBcre z)JL+mN=UR$kmboqlVq!8Vlml_3cXV#Efs3F1i-Dl?LnKOvoK_UIYe*R@Yx zS}m8Q37OmIMC;LsN(4%Bn;Jo4su^;`LF5ixzT_nWqujkbJvg%{qarDp1Qp)ch7vfi z$YDd~Sz9cVIXESF%c#u#`)3(tc$3*xr!jwYnKryU>^Xfno@<(rWV zJMf)vgQnYU%Wx8j*^VZTAgt)Qje9VmmrfqMb@Vvv;w9FJqbCm%*leWs!O2`Y01Bt2(J>2!eXG@rw2Mi5EwbAC2+Mj6w}*go3#rp7ok2Tc6q1jSnK z=HSO9vtPb=obpU%@hC9^Xa*2Sw~nEu%(+%7lvcK?jHPC4la^c9S*h%o#VuO%8VMgl zNKiY5%t%=q)10J#pP)vX6Mm%^H<#M5r?W&i5BGlO5Mw-&LnHUeM1kNvK`(Z!Unf|4 zc(fM#W|S>4X8t5_*{BgT6+6iNrJ;PGa@yfP4t4P-Ls(s8vO=C7AFwF2xs??g3RhY zP555D;z!|E;ZjP7dsV8K6{)mztuM}H+D=1J4i?8ySWzI@L{2vphSRA>FC&F*H={oX z#h3^VJskNFn*d@8EbW=8=X%MS6O1rNm% zKUK@MX5rT>ydfBq2GgZ^E2Z7;ihLImh3I6JlImWZ(i|62O;*+2!%+xw8P?bB>u5qW zxFYD&(3wHf3g?eqM?g1E96_0`pC+Bz)7{$Ms5_!h{AfGDu zF|o;}7$$U0yQ(U=_82__{S9G0;~mK902e9A3IoX9A}krDC>2*u z?rWFL-K!3h(L3O7k{XRtimJ{l_E<%@0l!dCf$Hgp|0L4Pd?xISjxL>Tl_ZPS_x9sR zpFX$q#L#v`_u%^J!$%R-*ntt9jq)(Ouy6CF-{wf4H+y$`aP17lB5PPBg}1I_ zE*AEsIF}s=K8L-|kagx(X47f^G6=%f&ML!JWrlz- za**JYTb;i$E!cZDkU3%)CS`-o7C#XvYOupzwF`GM#qZi3Ibd7lKzD=WZa?++9h|G1 zbZj)eg&lV@C?~VW72eVc==&Xac&?2}XiO3uxI-91%htm-36`aF`ST zgDgob<`#1>NPit`N8HO`&7!e#VClXxuBQ>GnUJ*_Xs=Az71*H!7t|nZtySss1fSHW z-LnH|abM}ka6Sr9NsphDsa&6DJbI~ooXq?-rCOO_Eu$#ijEfMK%8bBwwf)$rMXPX+ znFH{hq{i=&e!&}cMNx1@uua0sB86U9)+sx7nf{*ZC29LKGJ9bvYjs~+v!=N?{40LNun0X)&u!lLFXZ2F$jHIZ)T!L z^_2#aRHR60!ow;ZIJljO6cAz~7W`-^br}m!a)iA{P$ASyj;=FE@~4!v>a=`98F5FCcH>Rnc(tdaYw>AwpKwqN$1)m_Z1AVA@Cx&gQBla{1<^# zj9WKFUinJYV`OGE=q82dNy2(sO#CGZ8ZaVq9QHOAsT6y)e_8PZIg=JXI#`-ijWy88 zw$^pG89P@q0RrQ6Zp(Ve=kd`X4JRbs&`~G-YQ}Ihl!tKl(Y=WZ)ispDHAI1!!qAb^ zXHO2@rP^`n#1Z^FJQczSn8``MDf_M;8jiB6DgdMXzAM?XSHnX78Ynhlt>2t9V=dy5#hg)>eJXZi5z+eDb*N8N67U*QgNU6it2Dhe2)cE-G zl|c$4Gg-r3b_Ph7USpU9OhV$eY|n06MR#Z8E8~Q9qiD5m9Ob@gl%lb7>+oyT9yuZV zTlQ>dLqxmwc5=(02(}#h1+WF{Zko29#Vx@r>O}&NvO4`@bjMOu1CaILz`>g*PMth` z2H*w}gG|Az35HFajVJt|cAV$I8JHU(+;nvR?P!5b%mi0j?Jd}z774mH|{xfW-}#nMi^Z{ZJ>1k)lvGs1gqmcfbWBpg9rG15=i5FFxI~Kv4L|3r` z^}NagX|(kDOzYyfAx$E6Y+^f?0T? zuuiqHw1sH`pVxbV3*iKyj3VjBdZtpIu(@TFEJurNb|#dm^F(`6B69zijia=2mpnLD z+TlgbOwlH6^|N8-P<=?NpS&008D~ zy}&KC}BJ8+DGMGh;aPXvXHO(2y?>S`!^mMwP$B0DPR_EA@?|f*?5feAz1uM2O>kL%X4f zXt9HcRMV~6&=?5dzF(L$RaMm=(BO)musG#}u|_^t$#Q=-lN2F!>}qde2GEY(8|hvp zX-0D!AbGo5VZB!)Kn1$O%Zl`o6b?MYnBmG$-%#ch`4?wlK6QRCp#xCVi7RIR+xOBh zyS*4(N2P($Rn1jS=Jwb-)X}>=k3$vTlJq_ZAIuOQQ=FuJcB})kpr>%ov`JcDPe}h-0A7_AhB2(3u)WG4Y_x_cs$P7sL24ak+As$-@)wjv znlgrT#lC9Cwra+oRrS)it8j$>t$kF!SqS`6^dAV$9`+PT>l%!j23Pb1X%HiO8=tV@ z`3M=#)7#ce(G)$Q?s@pY5P&5J@ev7$2ORIy=LSf7*8p;#D+>f$?Az>>J(uJ{RRLL$ zTp@Rq_^p6Qn#2da(w7FA!P#n*D}xtV(0VgQv%V6vn!|uSib7Z-4cPv2!J;%yG%-%H zOL~l6AP$wqOO?elYqKW_OEYXN5y}`iS!TP_$2L0x^G|{@dv2*H?N%WVL9MDU@@@5Q z@?x)W%Cn)seO=foo$3VZ<>?tDH&{*dSCtkD$!ckgqoPIJy!fZ9RJU zI)enRz5WilR=8)F&~SF!ebTl}`9WcDi>Q7On=n18l9{S`SYF*3qOJU~pi3LYKO5XT zN82+xgXP^|&y*KFjRj*B71mpZYP=bO`fh(LwcV!E9x{&>Y4_l=;#vI z1cz2=s~7sh1%p0czfb$aFSnl?xPEZ^rDF#G2_qB|s5n2&TflDuKvY|qkGT!^pGR(m z{Bj9XW*IS?pWX6r$`jevzmfi%Z(pmTTyLgm(3?R4*|OkArrWOAsN3e8@FkK1GEQWrpl>LqM~l z*tY?VtP=O_7F)OZM-mkRrDSo3Pea-l%yjo;K^pI7L)&tbusK^%B2!TcV;6^QkCi65 zxgVKK2y)Cz(eL8mq3WpBG%v|hRQfjSm9H$WVNz)S*ww>Hv+Ks$&3G_WZkf+P7cxWg z;1wW7BZlZD?vgQ-LgS&>A6zSP!W!~3*)rmq4a%=n^#2J}{L?MzdWy-enfapFM81FsA$SbW%S&XZ%D)1uNBPMW`J+6?y=2xy@V3=f)9 z80K$4z=rIql0q`qj9;o|S({tzik$!nJ?4{0Pny5L6!0090yzArZpkZ(>kxLkYbf?C zwxLnDO&pB(t`&atRMa}zSUXUr9Xr*L!fZ!%86Hgj%bCr}-(q&urVsaZ3nA0Rs4m_QznRSL-9odHV*;nf^qPRPH3xmSYLVUKLa% zZB*q&fGdhyW*U{s{bwe6cNo`8E~6D?!W4&_;W^GqWOm zuuG+R;knINoavnvdj{fvn|3{SaWP53@hxl-e8Xhy9mCR9y*E?Tr*8<}~hqzaUHe$lss@=9ongk@`m!L9MYZ^T2 zD*$$|gBz=~EAyYI>KX*wUK{2c)`_Z77!Wt3GgDpwPux8x+)t4^{DI?vbi8@^B*Y!` zAP|D}bdvim=ffh;!(QRGv@oKaCXxa{@2qGLf8{9r$0S9r(7az7Bi^wVjka#xcFe!^A$WjwHnxKJ@=k{7vA$jlfMn^P?WrYQ=f!%1#^Hf1F(w=hhrlqU`VT2-EiC!JN2OX`s@7Z0EyI}~{-%_(aevO-I9 z!^(@Y&@tayMOi?zsxr^J*}GPE;K-k2C&?+-0S)|lV?*s}Dxh~-IsyYifN*}EOyC-{ zPij2m{@cTJyM%jp^U=PM%~&Rsbas|*yfDk4DsB|#C4hBOntMQ=#%hB2Mf%+sJ7Swk zgYdm5HrX?jPDK`=^|WYt%0L|&Gz4Yl&0F@PO~Brvj3eIz!fvOgxsoWNxr#BRxl9=% zpppxGnW1E1Ih7ERlsMozHMsKA^`6Y(&NqlKJ7JLb5R#uTxPoL@i11d5=#`-Iay53@0ENwRQ63kPW~%6<>h^*` zj&vFna217u^IDnCON`D@<<|K2y7iHJaWZ#=7LrwK@wK8}GIw(6m@phk9+dPIku<6V z5{d8%$`*i0=XP#FP!Q(~-rvvhg#7}t$mdV48JrE2IEaT>iCvpD7d2uZD$UVwIg?Q8 zO;53)^po4?gra0IPOYlQF>VsayV(P;$to`(DP=>#xYb`^xZD} z&z&|k75R0x=88OHkw;}NyA0tpmkR|O)r@U(Ygbp~*U-#d;@c!Pxe4WyPdEq*_M);< zKQYlXX_T+L*QSW!D`m1g;gf0_9>=#jA}WN}BGn~m`lNpk4yGLg*bVrbNK&>X7(O*) z+=b5)G4HzZQp#Ez2u|w9nhTAS7(>@MNlocug85{1UUr4XT2ZCx9suo26$;!DxIu6> z(Cfj2xu+8yj%kuTbLbyf|x!_0Vp!{&grw17{-PDIei!z1Z`bZt~W${RL zGo-raXTLvSv4_G5RUIA$;A3Y*>4sG@SfVvC|ovq#aO*<(Z6x=KHNHf9I!sji}c zQcP1>KF_WoIxCZ>$1+N>Irax{&U3eBJM!V>`NH1gOis?B7`}tbEgD=9%FHb znS~6PlZMVoZkv1dO4-Z3`!pL=zFETW;%&Tg=K713bC&`Qn2V6uhYTdZY_ZC6g`2sm zvOEWK|7>7BTqg9^T(^6d=R7L7)N#qhp(W@_U4?KsP|+^l4p4)j5#Ag(o1#1u+u5T? zh+41kwzmiT>yiG2bG-`Z#tJG;uCVEZ%3n4}~b-lcNW5q%%W%1*?p@*)nz3x^kq z&rz47JWl=Te^@_4=ce(AKo}=TcD)KVR^0Ui@L_4w2T2-XCHg~Nq(lZ~n zwj@XTR;{Wn5voV^RI3^a1odfEMedUSkv|9m+O2`MF=1zbYoXQ)WnyTE^qtP1FYxFw zzm?@FzMXDjd5Byl-A)XvU!xgN13S_=+0ZJUE94KVk(!nSdUn2(Dy&4Pm?0ESv?J6@EvBlp zK4i)0P(L{vOb7HuyjBZ=bzYb-?U3|vp_&Vl0uJRzLXjADtJ;pbJT@bA!rfGwAv9VGDf!@LBGbl42YH*nIvYYm1`bHGo!Q6}xuTOU`1 zN||RS1v$)BGg#a0i1kG4MbLf%UPS-1HD?uieT3WgNx$s*qkj>V^vuu_LZz(8N}+P9 zDy&SXnX1;6=L+ksRouQ*Z!1#qN&QZ8Nsy*2vq2ZbVM_=WeOIcPry%6M^@CCQ_v?3j z+opQdrcy-z(5_-dwyUZ(@K9Hq%HI&JD53zs1%k?vn}$Abh&z!A-|$h zm0zhZaxjxxTP>wuZ?cqz{-E-_JjPS)JX7JN`jbMTa8y}yt8kj`pyEF>g@eNMMInC? z9K?7BGY6HZP*^#d^Q`QWTz(EU<#hRB2TKP_`E|^B*ipXW8P=|5+?pwsW(v>G|HL)f zj9inWg+Z68H*h6pQoYos^!)yhD$ig1QT30%|9`j&GpE%d3P;O|tUP^@&OxOJF>@Sw zdt8BKd^mqBMCE9DtMbL0Y0Ml=Z&@S7pB|^?!W9lSJT~&zHHvs=9+AwPoSmlse-PRK zlXoor>t;*I%u_8L{G-ZV{$Ly}OLfKBx(dhlR1NO=#o7AWj4UWaii#`T!(3N2xEJym z_uR{M#kqR->8|d@xKBJ#TxBruT}7p?ypnJIP4`Pmxf@`*t2fEVixicIft0a5m3aW}q>wf-0NoIjQt5UwQB7LPUTUV*$ z=?(7sVxFE~oGbrs%JLZB+3rn$cQOBX749aDmS5r@F66%+{-8{!Gk0Pq_iz{b)|I-!A@@$&Uh5sZ4caui{@7MJ6 z!~b+m|9A2+{Wa5PQx0EoNx4a6w6B%x%Ijo_mDcIX%S&qHKVPi}7Fkj(KgwtiKYw?w z(*37D{&zovj>Iq6y-DF{qfJUmObZ*?QjUKbJwyH%q+OuZYM1f9{BN;V8?24W)y8PG zx!UMl(_>Ll|M2q~`9Xf?r=Krc7CrJkx29K-Z|0}WSUO{d`~xk&%)f>5P1eGgs+9|4 z>UnShzpmBS#g%3jB^mTF9KV6J0hwtb^|1vdxoJT-wt;?W258hR-}a#uS|}u z;C|JLv;c$roXq0Xip=Pm+}Pp-{}t(R{_e)>K$NSmLSudR)b*1|}YWXO_VXMRj* zdNR*1N~q3UlvB_X)!t}t9yv(!|Sr+-aw1B*ofg{)Tx6dIxC@Cb1Yn6YV85mc# zGA+S>;`%Outi-6_}UD?}jd(zo0G=xi48B8_avIUyO`b zjyxwcJIzp)JwLZ};Fym zu|>6cD~eJpOs`k`|6Kl6*=e;EIjLMzbX-AJQ7WHHXq|km$hkt7h90KJg_u5<;;0br z*JTD}nLfwtqSU&qKtpwAzO1jYLGcCJswL5Jp(g$NN$1mpRvNO36G}4!gH7^Yk{nzg zU$87cXJqG@wo0`$#Wb74&W%Cx`j#}z;S zd_k^ug}gu6$BVRD)6bV_qn2rTeB|G<8h)(WB25{pbaX!^|<`BE0f)MXWg z2E`-8^uW~MY?EFgJHyC6lCOLI-|UO&w@lYpDBoQu-z=F6$i_uHs7RBZ1lSA@!>9JMW=}D!T(IKYm%nXXl`TJ|mstd`> zDvqnp4oO}z-+$zK|EB+`@?Kt$r3>dMQi k^p(C@rKP^6^ThS11H;vHk~mGrUy@y>Zk)uW+N6O01-KwhZvX%Q literal 0 HcmV?d00001 diff --git a/python/ray/data/tests/test_dataset_formats.py b/python/ray/data/tests/test_dataset_formats.py index 551b75ad31117..0287699e3c4cf 100644 --- a/python/ray/data/tests/test_dataset_formats.py +++ b/python/ray/data/tests/test_dataset_formats.py @@ -2825,7 +2825,7 @@ def test_torch_datasource_value_error(ray_start_regular_shared, local_path): def test_image_folder_datasource( ray_start_regular_shared, enable_automatic_tensor_extension_cast ): - root = os.path.join(os.path.dirname(__file__), "image-folder") + root = "example://image-folder" ds = ray.data.read_datasource(ImageFolderDatasource(), root=root, size=(64, 64)) assert ds.count() == 3 @@ -2851,7 +2851,7 @@ def test_image_folder_datasource_e2e(ray_start_regular_shared): from torchvision import transforms from torchvision.models import resnet18 - root = os.path.join(os.path.dirname(__file__), "image-folder") + root = "example://image-folder" dataset = ray.data.read_datasource( ImageFolderDatasource(), root=root, size=(32, 32) ) diff --git a/python/ray/data/tests/test_dataset_pipeline.py b/python/ray/data/tests/test_dataset_pipeline.py index 10c54932281a3..c33c1602a6913 100644 --- a/python/ray/data/tests/test_dataset_pipeline.py +++ b/python/ray/data/tests/test_dataset_pipeline.py @@ -52,7 +52,7 @@ def test_warnings(shutdown_only): assert dataset.logger.infos == [ "Created DatasetPipeline with 10 windows: 8b min, 8b max, 8b mean", "Blocks per window: 1 min, 1 max, 1 mean", - f"{OK_PREFIX} This pipeline's windows can each fit in object store memory " + f"{OK_PREFIX} This pipeline's windows likely fit in object store memory " "without spilling.", ] @@ -117,7 +117,7 @@ def test_warnings(shutdown_only): "Blocks per window: 10 min, 10 max, 10 mean", f"{OK_PREFIX} This pipeline's per-window parallelism is high enough to fully " "utilize the cluster.", - f"{OK_PREFIX} This pipeline's windows can each fit in object store memory " + f"{OK_PREFIX} This pipeline's windows likely fit in object store memory " "without spilling.", ]