From 7f2d3b14468d15bd58ada51d3f7e5791ec84f0ad Mon Sep 17 00:00:00 2001 From: Andrew Gazelka Date: Wed, 18 Dec 2024 18:22:09 -0800 Subject: [PATCH 01/11] test(connect): add more tests for `createDataFrame` (#3607) --- tests/connect/test_create_df.py | 66 +++++++++++++++++++++++++++++++-- 1 file changed, 63 insertions(+), 3 deletions(-) diff --git a/tests/connect/test_create_df.py b/tests/connect/test_create_df.py index 187f4fbc5a..426fe57df9 100644 --- a/tests/connect/test_create_df.py +++ b/tests/connect/test_create_df.py @@ -1,7 +1,7 @@ from __future__ import annotations -def test_create_df(spark_session): +def test_create_simple_df(spark_session): # Create simple DataFrame with single column data = [(1,), (2,), (3,)] df = spark_session.createDataFrame(data, ["id"]) @@ -11,6 +11,8 @@ def test_create_df(spark_session): assert len(df_pandas) == 3, "DataFrame should have 3 rows" assert list(df_pandas["id"]) == [1, 2, 3], "DataFrame should contain expected values" + +def test_create_float_df(spark_session): # Create DataFrame with float column float_data = [(1.1,), (2.2,), (3.3,)] df_float = spark_session.createDataFrame(float_data, ["value"]) @@ -18,6 +20,8 @@ def test_create_df(spark_session): assert len(df_float_pandas) == 3, "Float DataFrame should have 3 rows" assert list(df_float_pandas["value"]) == [1.1, 2.2, 3.3], "Float DataFrame should contain expected values" + +def test_create_two_column_df(spark_session): # Create DataFrame with two numeric columns two_col_data = [(1, 10), (2, 20), (3, 30)] df_two = spark_session.createDataFrame(two_col_data, ["num1", "num2"]) @@ -26,10 +30,66 @@ def test_create_df(spark_session): assert list(df_two_pandas["num1"]) == [1, 2, 3], "First number column should contain expected values" assert list(df_two_pandas["num2"]) == [10, 20, 30], "Second number column should contain expected values" - # now do boolean - print("now testing boolean") + +def test_create_boolean_df(spark_session): boolean_data = [(True,), (False,), (True,)] df_boolean = spark_session.createDataFrame(boolean_data, ["value"]) df_boolean_pandas = df_boolean.toPandas() assert len(df_boolean_pandas) == 3, "Boolean DataFrame should have 3 rows" assert list(df_boolean_pandas["value"]) == [True, False, True], "Boolean DataFrame should contain expected values" + + +def test_create_string_df(spark_session): + # Create DataFrame with string column + string_data = [("hello",), ("world",), ("test",)] + df_string = spark_session.createDataFrame(string_data, ["text"]) + df_string_pandas = df_string.toPandas() + assert len(df_string_pandas) == 3, "String DataFrame should have 3 rows" + assert list(df_string_pandas["text"]) == [ + "hello", + "world", + "test", + ], "String DataFrame should contain expected values" + + +def test_create_mixed_type_df(spark_session): + # Create DataFrame with mixed types + mixed_data = [(1, "one", True), (2, "two", False), (3, "three", True)] + df_mixed = spark_session.createDataFrame(mixed_data, ["id", "name", "active"]) + df_mixed_pandas = df_mixed.toPandas() + assert len(df_mixed_pandas) == 3, "Mixed-type DataFrame should have 3 rows" + assert list(df_mixed_pandas["id"]) == [1, 2, 3], "ID column should contain expected values" + assert list(df_mixed_pandas["name"]) == ["one", "two", "three"], "Name column should contain expected values" + assert list(df_mixed_pandas["active"]) == [True, False, True], "Active column should contain expected values" + + +def test_create_null_df(spark_session): + # Create DataFrame with null values + null_data = [(1, None), (None, "test"), (3, "data")] + df_null = spark_session.createDataFrame(null_data, ["id", "value"]) + df_null_pandas = df_null.toPandas() + assert len(df_null_pandas) == 3, "Null-containing DataFrame should have 3 rows" + assert df_null_pandas["id"].isna().sum() == 1, "ID column should have one null value" + assert df_null_pandas["value"].isna().sum() == 1, "Value column should have one null value" + + +def test_create_decimal_df(spark_session): + # Create DataFrame with decimal/double precision numbers + decimal_data = [(1.23456789,), (9.87654321,), (5.55555555,)] + df_decimal = spark_session.createDataFrame(decimal_data, ["amount"]) + df_decimal_pandas = df_decimal.toPandas() + assert len(df_decimal_pandas) == 3, "Decimal DataFrame should have 3 rows" + assert abs(df_decimal_pandas["amount"][0] - 1.23456789) < 1e-8, "Decimal values should maintain precision" + + +def test_create_special_chars_df(spark_session): + # Create DataFrame with empty strings and special characters + special_data = [("",), (" ",), ("!@#$%^&*",)] + df_special = spark_session.createDataFrame(special_data, ["special"]) + df_special_pandas = df_special.toPandas() + assert len(df_special_pandas) == 3, "Special character DataFrame should have 3 rows" + assert list(df_special_pandas["special"]) == [ + "", + " ", + "!@#$%^&*", + ], "Special character DataFrame should contain expected values" From c30f6a868731ad21f259b5c3419bfe3e0f42dcea Mon Sep 17 00:00:00 2001 From: Kev Wang Date: Wed, 18 Dec 2024 18:31:42 -0800 Subject: [PATCH 02/11] fix(udf): udf call with empty table and batch size (#3604) --- daft/udf.py | 2 +- tests/expressions/test_udf.py | 16 ++++++++++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/daft/udf.py b/daft/udf.py index 36e841c683..3efad69dc2 100644 --- a/daft/udf.py +++ b/daft/udf.py @@ -125,7 +125,7 @@ def get_args_for_slice(start: int, end: int): return args, kwargs - if batch_size is None: + if batch_size is None or len(evaluated_expressions[0]) <= batch_size: args, kwargs = get_args_for_slice(0, len(evaluated_expressions[0])) try: results = [func(*args, **kwargs)] diff --git a/tests/expressions/test_udf.py b/tests/expressions/test_udf.py index 0fad641939..ff43df639e 100644 --- a/tests/expressions/test_udf.py +++ b/tests/expressions/test_udf.py @@ -423,3 +423,19 @@ def noop(data): with pytest.raises(OverflowError): table.eval_expression_list([noop.override_options(batch_size=-1)(col("a"))]) + + +@pytest.mark.parametrize("batch_size", [None, 1, 2]) +@pytest.mark.parametrize("use_actor_pool", [False, True]) +def test_udf_empty(batch_size, use_actor_pool): + df = daft.from_pydict({"a": []}) + + @udf(return_dtype=DataType.int64(), batch_size=batch_size) + def identity(data): + return data + + if use_actor_pool: + identity = identity.with_concurrency(2) + + result = df.select(identity(col("a"))) + assert result.to_pydict() == {"a": []} From 8f8e2102a69a927b96a6f1cc2eebd3aae9acbafc Mon Sep 17 00:00:00 2001 From: Kev Wang Date: Wed, 18 Dec 2024 19:50:23 -0800 Subject: [PATCH 03/11] chore!: drop support for Python 3.8 (#3592) BREAKING CHANGE: Python 3.8 has reached EOL, set our minimum version to 3.9 --- .github/workflows/build-artifact-s3.yml | 2 +- .github/workflows/nightlies-tests.yml | 2 +- .github/workflows/notebook-checker.yml | 2 +- .github/workflows/property-based-tests.yml | 2 +- .github/workflows/python-package.yml | 22 ++++---- .github/workflows/python-publish.yml | 2 +- .ruff.toml | 1 + Cargo.toml | 2 +- daft/dataframe/dataframe.py | 4 -- daft/pickle/cloudpickle.py | 15 +---- daft/pickle/compat.py | 20 ++----- pyproject.toml | 4 +- requirements-dev.txt | 55 ++++++++----------- tests/io/delta_lake/test_table_read.py | 6 +- .../delta_lake/test_table_read_pushdowns.py | 5 +- tests/io/delta_lake/test_table_write.py | 12 +--- tests/io/lancedb/test_lancedb_writes.py | 11 ++-- tests/series/test_temporal_ops.py | 14 ----- tests/test_datatypes.py | 35 +++++------- 19 files changed, 76 insertions(+), 140 deletions(-) diff --git a/.github/workflows/build-artifact-s3.yml b/.github/workflows/build-artifact-s3.yml index 227705d5c4..8cd536cd10 100644 --- a/.github/workflows/build-artifact-s3.yml +++ b/.github/workflows/build-artifact-s3.yml @@ -14,7 +14,7 @@ on: env: PACKAGE_NAME: getdaft - PYTHON_VERSION: 3.8 + PYTHON_VERSION: 3.9 jobs: build-and-push: diff --git a/.github/workflows/nightlies-tests.yml b/.github/workflows/nightlies-tests.yml index 065bf017e1..75ad22798d 100644 --- a/.github/workflows/nightlies-tests.yml +++ b/.github/workflows/nightlies-tests.yml @@ -12,7 +12,7 @@ on: env: DAFT_ANALYTICS_ENABLED: '0' UV_SYSTEM_PYTHON: 1 - PYTHON_VERSION: '3.8' + PYTHON_VERSION: '3.9' jobs: integration-test-tpch: diff --git a/.github/workflows/notebook-checker.yml b/.github/workflows/notebook-checker.yml index 116af74797..07864a4e0c 100644 --- a/.github/workflows/notebook-checker.yml +++ b/.github/workflows/notebook-checker.yml @@ -15,7 +15,7 @@ jobs: strategy: fail-fast: false matrix: - python-version: ['3.8'] + python-version: ['3.9'] steps: - uses: actions/checkout@v4 - name: Set up Python ${{ matrix.python-version }} diff --git a/.github/workflows/property-based-tests.yml b/.github/workflows/property-based-tests.yml index 540a255aaa..51e1e90310 100644 --- a/.github/workflows/property-based-tests.yml +++ b/.github/workflows/property-based-tests.yml @@ -15,7 +15,7 @@ jobs: strategy: fail-fast: false matrix: - python-version: ['3.8'] + python-version: ['3.9'] daft_runner: [py] steps: - uses: actions/checkout@v4 diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index 2425787734..c08f570fd2 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -24,7 +24,7 @@ jobs: strategy: fail-fast: false matrix: - python-version: ['3.8', '3.10'] + python-version: ['3.9', '3.10'] daft-runner: [py, ray, native] pyarrow-version: [7.0.0, 16.0.0] os: [ubuntu-20.04, windows-latest] @@ -40,10 +40,10 @@ jobs: python-version: '3.10' pyarrow-version: 7.0.0 os: ubuntu-20.04 - - python-version: '3.8' + - python-version: '3.9' pyarrow-version: 16.0.0 - os: windows-latest - python-version: '3.8' + python-version: '3.9' - os: windows-latest pyarrow-version: 7.0.0 steps: @@ -181,7 +181,7 @@ jobs: package-name: getdaft strategy: matrix: - python-version: ['3.8'] + python-version: ['3.9'] steps: - uses: actions/checkout@v4 with: @@ -219,7 +219,7 @@ jobs: strategy: fail-fast: false matrix: - python-version: ['3.8'] + python-version: ['3.9'] daft-runner: [py, ray, native] steps: - uses: actions/checkout@v4 @@ -294,7 +294,7 @@ jobs: strategy: fail-fast: false matrix: - python-version: ['3.8'] # can't use 3.7 due to requiring anon mode for adlfs + python-version: ['3.9'] # can't use 3.7 due to requiring anon mode for adlfs daft-runner: [py, ray, native] steps: - uses: actions/checkout@v4 @@ -372,7 +372,7 @@ jobs: strategy: fail-fast: false matrix: - python-version: ['3.8'] # can't use 3.7 due to requiring anon mode for adlfs + python-version: ['3.9'] # can't use 3.7 due to requiring anon mode for adlfs daft-runner: [py, ray, native] # These permissions are needed to interact with GitHub's OIDC Token endpoint. # This is used in the step "Assume GitHub Actions AWS Credentials" @@ -466,7 +466,7 @@ jobs: strategy: fail-fast: false matrix: - python-version: ['3.8'] # can't use 3.7 due to requiring anon mode for adlfs + python-version: ['3.9'] # can't use 3.7 due to requiring anon mode for adlfs daft-runner: [py, ray, native] steps: - uses: actions/checkout@v4 @@ -543,7 +543,7 @@ jobs: strategy: fail-fast: false matrix: - python-version: ['3.8'] # can't use 3.7 due to requiring anon mode for adlfs + python-version: ['3.9'] # can't use 3.7 due to requiring anon mode for adlfs daft-runner: [py, ray, native] steps: - uses: actions/checkout@v4 @@ -854,7 +854,7 @@ jobs: fail-fast: false matrix: os: [ubuntu, Windows] - python-version: ['3.8'] + python-version: ['3.9'] steps: - uses: actions/checkout@v4 - uses: moonrepo/setup-rust@v1 @@ -946,7 +946,7 @@ jobs: runs-on: ubuntu-latest timeout-minutes: 15 env: - python-version: '3.8' + python-version: '3.9' steps: - uses: actions/checkout@v4 - name: Set up Python ${{ env.python-version }} diff --git a/.github/workflows/python-publish.yml b/.github/workflows/python-publish.yml index 1f5950e55c..e21010a939 100644 --- a/.github/workflows/python-publish.yml +++ b/.github/workflows/python-publish.yml @@ -162,7 +162,7 @@ jobs: with: # Really doesn't matter what version we upload with # just the version we test with - python-version: '3.8' + python-version: '3.9' channels: conda-forge channel-priority: true diff --git a/.ruff.toml b/.ruff.toml index 87bcabe0cf..96b4dc7531 100644 --- a/.ruff.toml +++ b/.ruff.toml @@ -1,6 +1,7 @@ fix = true indent-width = 4 line-length = 120 +# TODO: clean up typing code and update to py39 target-version = "py38" [format] diff --git a/Cargo.toml b/Cargo.toml index b371fd37be..dd88281828 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -276,7 +276,7 @@ features = ['async'] path = "src/parquet2" [workspace.dependencies.pyo3] -features = ["extension-module", "multiple-pymethods", "abi3-py38", "indexmap"] +features = ["extension-module", "multiple-pymethods", "abi3-py39", "indexmap"] version = "0.21.0" [workspace.dependencies.pyo3-log] diff --git a/daft/dataframe/dataframe.py b/daft/dataframe/dataframe.py index 220c7787d8..4264883c28 100644 --- a/daft/dataframe/dataframe.py +++ b/daft/dataframe/dataframe.py @@ -1094,13 +1094,9 @@ def write_lance( (Showing first 1 of 1 rows) """ - import sys - from daft import from_pydict from daft.io.object_store_options import io_config_to_storage_options - if sys.version_info < (3, 9): - raise ValueError("'write_lance' requires python 3.9 or higher") try: import lance import pyarrow as pa diff --git a/daft/pickle/cloudpickle.py b/daft/pickle/cloudpickle.py index 1cc1dc6cec..7b9ff0edbd 100644 --- a/daft/pickle/cloudpickle.py +++ b/daft/pickle/cloudpickle.py @@ -70,20 +70,7 @@ except ImportError: _typing_extensions = Literal = Final = None -if sys.version_info >= (3, 8): - from types import CellType -else: - - def f(): - a = 1 - - def g(): - return a - - return g - - CellType = type(f().__closure__[0]) - +from types import CellType # cloudpickle is meant for inter process communication: we expect all # communicating processes to run the same Python version hence we favor diff --git a/daft/pickle/compat.py b/daft/pickle/compat.py index 7f51f653b0..cd343fee01 100644 --- a/daft/pickle/compat.py +++ b/daft/pickle/compat.py @@ -4,20 +4,8 @@ from __future__ import annotations -import sys +import pickle # noqa: F401 -if sys.version_info < (3, 8): - try: - import pickle5 as pickle - from pickle5 import Pickler - except ImportError: - import pickle - - # Use the Python pickler for old CPython versions - from pickle import _Pickler as Pickler -else: - import pickle # noqa: F401 - - # Pickler will the C implementation in CPython and the Python - # implementation in PyPy - from pickle import Pickler # noqa: F401 +# Pickler will the C implementation in CPython and the Python +# implementation in PyPy +from pickle import Pickler # noqa: F401 diff --git a/pyproject.toml b/pyproject.toml index 6448275771..ff692da981 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -19,7 +19,7 @@ maintainers = [ ] name = "getdaft" readme = "README.rst" -requires-python = ">=3.8" +requires-python = ">=3.9" [project.optional-dependencies] all = ["getdaft[aws, azure, gcp, ray, pandas, numpy, iceberg, deltalake, sql, unity]"] @@ -64,7 +64,7 @@ features = ["python"] [tool.mypy] exclude = ['daft/pickle/*.py$'] files = ["daft/**/*.py", "daft/**/*.pyx", "tests/**/*.py"] -python_version = "3.8" +python_version = "3.9" warn_return_any = true warn_unused_configs = true diff --git a/requirements-dev.txt b/requirements-dev.txt index 0950a00cf6..633fe0e5c3 100644 --- a/requirements-dev.txt +++ b/requirements-dev.txt @@ -26,12 +26,9 @@ pytest-codspeed==2.2.1 # Testing dependencies lxml==5.3.0 -dask==2023.5.0; python_version == '3.8' -dask[dataframe]==2024.4.1; python_version >= '3.9' -numpy; python_version < '3.9' -numpy==1.26.2; python_version >= '3.9' -pandas==2.0.3; python_version == '3.8' -pandas==2.1.3; python_version >= '3.9' +dask[dataframe]==2024.4.1 +numpy==1.26.2 +pandas==2.1.3 xxhash>=3.0.0 Pillow==10.4.0 opencv-python==4.10.0.84 @@ -39,60 +36,56 @@ tiktoken==0.7.0 duckdb==1.1.2 # Pyarrow -pyarrow==16.0.0; python_version >= '3.9' -pyarrow==15.0.0; python_version < '3.9' +pyarrow==16.0.0 # Ray -ray[data, client]==2.10.0; python_version == '3.8' -ray[data, client]==2.34.0; python_version >= '3.9' +ray[data, client]==2.34.0 # Lance -lancedb>=0.6.10; python_version >= '3.8' +lancedb>=0.6.10 #Iceberg -pyiceberg==0.7.0; python_version >= '3.8' -tenacity==8.2.3; python_version >= '3.8' +pyiceberg==0.7.0 +tenacity==8.2.3 # Delta Lake deltalake==0.5.8; platform_system == "Windows" -deltalake==0.18.2; platform_system != "Windows" and python_version < '3.9' -deltalake==0.19.2; platform_system != "Windows" and python_version >= '3.9' +deltalake==0.19.2; platform_system != "Windows" # Databricks databricks-sdk==0.12.0 unitycatalog==0.1.1 #SQL -sqlalchemy==2.0.36; python_version >= '3.8' -connectorx==0.2.3; platform_system == "Linux" and platform_machine == "aarch64" and python_version >= '3.8' -connectorx==0.3.2; (platform_system != "Linux" or platform_machine != "aarch64") and python_version <= '3.8' -connectorx==0.3.3; (platform_system != "Linux" or platform_machine != "aarch64") and python_version > '3.8' -trino[sqlalchemy]==0.328.0; python_version >= '3.8' -PyMySQL==1.1.0; python_version >= '3.8' -psycopg2-binary==2.9.10; python_version >= '3.8' -sqlglot==23.3.0; python_version >= '3.8' -pyodbc==5.1.0; python_version >= '3.8' +sqlalchemy==2.0.36 +connectorx==0.2.3; platform_system == "Linux" and platform_machine == "aarch64" +connectorx==0.3.3; platform_system != "Linux" or platform_machine != "aarch64" +trino[sqlalchemy]==0.328.0 +PyMySQL==1.1.0 +psycopg2-binary==2.9.10 +sqlglot==23.3.0 +pyodbc==5.1.0 # AWS -s3fs==2023.12.0; python_version >= '3.8' +s3fs==2023.12.0 # on old versions of s3fs's pinned botocore, they neglected to pin urllib3<2 which leads to: # "ImportError: cannot import name 'DEFAULT_CIPHERS' from 'urllib3.util.ssl_'" -boto3==1.34.51; python_version >= '3.8' -moto[s3,server]==5.0.21; python_version >= '3.8' +boto3==1.34.51 +moto[s3,server]==5.0.21 # Azure -adlfs==2024.7.0; python_version >= '3.8' +adlfs==2024.7.0 azure-storage-blob==12.24.0 # GCS -gcsfs==2023.12.0; python_version >= '3.8' +gcsfs==2023.12.0 # Documentation myst-nb>=0.16.0 Sphinx==5.3.0 -sphinx-book-theme==1.1.0; python_version >= "3.9" +sphinx-book-theme==1.1.0 sphinx-reredirects>=0.1.1 sphinx-copybutton>=0.5.2 -sphinx-autosummary-accessors==2023.4.0; python_version >= "3.9" +sphinx-autosummary-accessors==2023.4.0 sphinx-tabs==3.4.5 # Daft connect testing diff --git a/tests/io/delta_lake/test_table_read.py b/tests/io/delta_lake/test_table_read.py index 273006659f..bf1d0fe2d9 100644 --- a/tests/io/delta_lake/test_table_read.py +++ b/tests/io/delta_lake/test_table_read.py @@ -1,7 +1,5 @@ from __future__ import annotations -import sys - import pyarrow as pa import pytest @@ -11,9 +9,9 @@ from tests.utils import assert_pyarrow_tables_equal PYARROW_LE_8_0_0 = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < (8, 0, 0) -PYTHON_LT_3_8 = sys.version_info[:2] < (3, 8) pytestmark = pytest.mark.skipif( - PYARROW_LE_8_0_0 or PYTHON_LT_3_8, reason="deltalake only supported if pyarrow >= 8.0.0 and python >= 3.8" + PYARROW_LE_8_0_0, + reason="deltalake only supported if pyarrow >= 8.0.0", ) diff --git a/tests/io/delta_lake/test_table_read_pushdowns.py b/tests/io/delta_lake/test_table_read_pushdowns.py index 0a147dff72..95c57ecc8a 100644 --- a/tests/io/delta_lake/test_table_read_pushdowns.py +++ b/tests/io/delta_lake/test_table_read_pushdowns.py @@ -7,7 +7,6 @@ from daft.io.object_store_options import io_config_to_storage_options deltalake = pytest.importorskip("deltalake") -import sys import pyarrow as pa import pyarrow.compute as pc @@ -18,9 +17,9 @@ from tests.utils import assert_pyarrow_tables_equal PYARROW_LE_8_0_0 = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < (8, 0, 0) -PYTHON_LT_3_8 = sys.version_info[:2] < (3, 8) pytestmark = pytest.mark.skipif( - PYARROW_LE_8_0_0 or PYTHON_LT_3_8, reason="deltalake only supported if pyarrow >= 8.0.0 and python >= 3.8" + PYARROW_LE_8_0_0, + reason="deltalake only supported if pyarrow >= 8.0.0", ) diff --git a/tests/io/delta_lake/test_table_write.py b/tests/io/delta_lake/test_table_write.py index 098caac34a..ef6772740f 100644 --- a/tests/io/delta_lake/test_table_write.py +++ b/tests/io/delta_lake/test_table_write.py @@ -2,7 +2,6 @@ import datetime import decimal -import sys from pathlib import Path import pyarrow as pa @@ -13,15 +12,10 @@ from daft.logical.schema import Schema from tests.conftest import get_tests_daft_runner_name -PYARROW_LE_8_0_0 = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < ( - 8, - 0, - 0, -) -PYTHON_LT_3_8 = sys.version_info[:2] < (3, 8) +PYARROW_LE_8_0_0 = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < (8, 0, 0) pytestmark = pytest.mark.skipif( - PYARROW_LE_8_0_0 or PYTHON_LT_3_8, - reason="deltalake only supported if pyarrow >= 8.0.0 and python >= 3.8", + PYARROW_LE_8_0_0, + reason="deltalake only supported if pyarrow >= 8.0.0", ) diff --git a/tests/io/lancedb/test_lancedb_writes.py b/tests/io/lancedb/test_lancedb_writes.py index 274c69ff74..e45c3b60d6 100644 --- a/tests/io/lancedb/test_lancedb_writes.py +++ b/tests/io/lancedb/test_lancedb_writes.py @@ -1,9 +1,8 @@ -import sys - import pyarrow as pa import pytest import daft +from tests.conftest import get_tests_daft_runner_name from tests.integration.io.conftest import minio_create_bucket TABLE_NAME = "my_table" @@ -14,10 +13,8 @@ } PYARROW_LE_8_0_0 = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < (8, 0, 0) -PY_LE_3_9_0 = sys.version_info < (3, 9) -pytestmark = pytest.mark.skipif( - PYARROW_LE_8_0_0 or PY_LE_3_9_0, reason="lance only supported if pyarrow >= 8.0.0 and python >= 3.9.0" -) + +pytestmark = pytest.mark.skipif(PYARROW_LE_8_0_0, reason="lance only supported if pyarrow >= 8.0.0") @pytest.fixture(scope="function") @@ -33,6 +30,8 @@ def test_lancedb_roundtrip(lance_dataset_path): assert df.to_pydict() == data +# TODO: re-enable test on Ray when fixed +@pytest.mark.skipif(get_tests_daft_runner_name() == "ray", reason="Lance fails to load credentials on Ray") @pytest.mark.integration() def test_lancedb_minio(minio_io_config): df = daft.from_pydict(data) diff --git a/tests/series/test_temporal_ops.py b/tests/series/test_temporal_ops.py index d98a174f27..5cf5679299 100644 --- a/tests/series/test_temporal_ops.py +++ b/tests/series/test_temporal_ops.py @@ -1,7 +1,5 @@ from __future__ import annotations -import sys - import pytest from daft.datatype import DataType, TimeUnit @@ -279,10 +277,6 @@ def ts_with_tz_maker(y, m, d, h, mi, s, us, tz): return datetime(y, m, d, h, mi, s, us, pytz.timezone(tz)) -@pytest.mark.skipif( - sys.version_info < (3, 8), - reason="Timezone conversions via PyArrow are supported in Python 3.8+", -) @pytest.mark.parametrize( ["input", "interval", "expected"], [ @@ -319,10 +313,6 @@ def test_series_timestamp_truncate_operation(input, interval, expected, tz) -> N assert expected_series.to_pylist() == truncated -@pytest.mark.skipif( - sys.version_info < (3, 8), - reason="Timezone conversions via PyArrow are supported in Python 3.8+", -) @pytest.mark.parametrize("tz", [None, "UTC", "+09:00", "-13:00"]) @pytest.mark.parametrize( ["input", "interval", "expected", "relative_to"], @@ -360,10 +350,6 @@ def test_series_timestamp_truncate_operation_with_relative_to(tz, input, interva assert expected_series.to_pylist() == truncated -@pytest.mark.skipif( - sys.version_info < (3, 8), - reason="Timezone conversions via PyArrow are supported in Python 3.8+", -) @pytest.mark.parametrize("tz", [None, "UTC", "+09:00", "-13:00"]) @pytest.mark.parametrize( ["input", "interval", "expected", "relative_to"], diff --git a/tests/test_datatypes.py b/tests/test_datatypes.py index 27270be427..195b603af4 100644 --- a/tests/test_datatypes.py +++ b/tests/test_datatypes.py @@ -1,7 +1,6 @@ from __future__ import annotations import copy -import sys from typing import Dict, List import pytest @@ -52,25 +51,21 @@ def test_datatype_parsing(source, expected): assert DataType._infer_type(source) == expected -# These tests are only valid for more modern versions of Python, but can't be skipped in the conventional -# way either because we cannot even run the subscripting during import-time -if sys.version_info >= (3, 9): - - @pytest.mark.parametrize( - ["source", "expected"], - [ - # These tests must be run in later version of Python that allow for subscripting of types - (list[str], DataType.list(DataType.string())), - (dict[str, int], DataType.map(DataType.string(), DataType.int64())), - ( - {"foo": list[str], "bar": int}, - DataType.struct({"foo": DataType.list(DataType.string()), "bar": DataType.int64()}), - ), - (list[list[str]], DataType.list(DataType.list(DataType.string()))), - ], - ) - def test_subscripted_datatype_parsing(source, expected): - assert DataType._infer_type(source) == expected +@pytest.mark.parametrize( + ["source", "expected"], + [ + # These tests must be run in later version of Python that allow for subscripting of types + (list[str], DataType.list(DataType.string())), + (dict[str, int], DataType.map(DataType.string(), DataType.int64())), + ( + {"foo": list[str], "bar": int}, + DataType.struct({"foo": DataType.list(DataType.string()), "bar": DataType.int64()}), + ), + (list[list[str]], DataType.list(DataType.list(DataType.string()))), + ], +) +def test_subscripted_datatype_parsing(source, expected): + assert DataType._infer_type(source) == expected @pytest.mark.parametrize( From e706caa4b47153d8b8986120a2654398f2c948b9 Mon Sep 17 00:00:00 2001 From: Colin Ho Date: Wed, 18 Dec 2024 21:08:42 -0800 Subject: [PATCH 04/11] feat(swordfish): Progress Bar (#3571) Uses the https://docs.rs/indicatif/latest/indicatif/index.html library to make a progress bar. Tracks rows received + rows emitted for each operator. The counting is attached to the input / output receivers for each operator, so there shouldn't be much contention. By default, the progress bar is printed to stderr. It is also cleared upon finish. Tested piping it out to stderr, e.g. `python script.py 2 > errors.log`, and the log was empty. https://github.com/user-attachments/assets/c494bc2e-afe0-4bfe-9f87-96d1b0dbc577 --------- Co-authored-by: Colin Ho Co-authored-by: Colin Ho --- Cargo.lock | 61 +++- daft/runners/progress_bar.py | 94 ++++-- src/daft-local-execution/Cargo.toml | 1 + .../src/intermediate_ops/intermediate_op.rs | 12 +- src/daft-local-execution/src/lib.rs | 40 ++- src/daft-local-execution/src/progress_bar.rs | 295 ++++++++++++++++++ src/daft-local-execution/src/run.rs | 14 +- src/daft-local-execution/src/runtime_stats.rs | 42 ++- .../src/sinks/blocking_sink.rs | 17 +- .../src/sinks/streaming_sink.rs | 11 +- .../src/sources/source.rs | 10 +- 11 files changed, 553 insertions(+), 44 deletions(-) create mode 100644 src/daft-local-execution/src/progress_bar.rs diff --git a/Cargo.lock b/Cargo.lock index 653204e89f..3b93f43998 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1420,7 +1420,7 @@ checksum = "7e959d788268e3bf9d35ace83e81b124190378e4c91c9067524675e33394b8ba" dependencies = [ "strum 0.24.1", "strum_macros 0.24.3", - "unicode-width", + "unicode-width 0.1.13", ] [[package]] @@ -1432,7 +1432,7 @@ dependencies = [ "crossterm", "strum 0.26.3", "strum_macros 0.26.4", - "unicode-width", + "unicode-width 0.1.13", ] [[package]] @@ -1607,6 +1607,19 @@ dependencies = [ "crossbeam-utils", ] +[[package]] +name = "console" +version = "0.15.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0e1f83fc076bd6dd27517eacdf25fef6c4dfe5f1d7448bafaaf3a26f13b5e4eb" +dependencies = [ + "encode_unicode", + "lazy_static", + "libc", + "unicode-width 0.1.13", + "windows-sys 0.52.0", +] + [[package]] name = "const-oid" version = "0.9.6" @@ -2286,6 +2299,7 @@ dependencies = [ "daft-writers", "futures", "indexmap 2.5.0", + "indicatif", "lazy_static", "log", "loole", @@ -2797,6 +2811,12 @@ version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "60b1af1c220855b6ceac025d3f6ecdd2b7c4894bfe9cd9bda4fbb4bc7c0d4cf0" +[[package]] +name = "encode_unicode" +version = "0.3.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a357d28ed41a50f9c765dbfe56cbc04a64e53e5fc58ba79fbc34c10ef3df831f" + [[package]] name = "encoding_rs" version = "0.8.34" @@ -3743,6 +3763,19 @@ dependencies = [ "serde", ] +[[package]] +name = "indicatif" +version = "0.17.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cbf675b85ed934d3c67b5c5469701eec7db22689d0a2139d856e0925fa28b281" +dependencies = [ + "console", + "number_prefix", + "portable-atomic", + "unicode-width 0.2.0", + "web-time", +] + [[package]] name = "indoc" version = "2.0.5" @@ -4428,6 +4461,12 @@ dependencies = [ "libc", ] +[[package]] +name = "number_prefix" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "830b246a0e5f20af87141b25c173cd1b609bd7779a4617d6ec582abaf90870f3" + [[package]] name = "numpy" version = "0.21.0" @@ -6308,7 +6347,7 @@ checksum = "23d434d3f8967a09480fb04132ebe0a3e088c173e6d0ee7897abbdf4eab0f8b9" dependencies = [ "smawk", "unicode-linebreak", - "unicode-width", + "unicode-width 0.1.13", ] [[package]] @@ -6800,6 +6839,12 @@ version = "0.1.13" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0336d538f7abc86d282a4189614dfaa90810dfc2c6f6427eaf88e16311dd225d" +[[package]] +name = "unicode-width" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1fc81956842c57dac11422a97c3b8195a1ff727f06e85c84ed2e8aa277c9a0fd" + [[package]] name = "unicode-xid" version = "0.2.4" @@ -7060,6 +7105,16 @@ dependencies = [ "wasm-bindgen", ] +[[package]] +name = "web-time" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a6580f308b1fad9207618087a65c04e7a10bc77e02c8e84e9b00dd4b12fa0bb" +dependencies = [ + "js-sys", + "wasm-bindgen", +] + [[package]] name = "weezl" version = "0.1.8" diff --git a/daft/runners/progress_bar.py b/daft/runners/progress_bar.py index e5f7172317..ed4c91c9a3 100644 --- a/daft/runners/progress_bar.py +++ b/daft/runners/progress_bar.py @@ -8,44 +8,48 @@ from daft.execution.execution_step import PartitionTask -class ProgressBar: - def __init__(self, use_ray_tqdm: bool, show_tasks_bar: bool = False, disable: bool = False) -> None: - self.show_tasks_bar = show_tasks_bar - self._maxinterval = 5.0 - - # Choose the appropriate tqdm module depending on whether we need to use Ray's tqdm - self.use_ray_tqdm = use_ray_tqdm - if use_ray_tqdm: - from ray.experimental.tqdm_ray import tqdm - else: - from tqdm.auto import tqdm as _tqdm +def get_tqdm(use_ray_tqdm: bool) -> Any: + # Choose the appropriate tqdm module depending on whether we need to use Ray's tqdm + if use_ray_tqdm: + from ray.experimental.tqdm_ray import tqdm + else: + from tqdm.auto import tqdm as _tqdm - try: - import sys + try: + import sys - from IPython import get_ipython + from IPython import get_ipython - ipython = get_ipython() + ipython = get_ipython() - # write to sys.stdout if in jupyter notebook - # source: https://github.com/tqdm/tqdm/blob/74722959a8626fd2057be03e14dcf899c25a3fd5/tqdm/autonotebook.py#L14 - if ipython is not None and "IPKernelApp" in ipython.config: + # write to sys.stdout if in jupyter notebook + # source: https://github.com/tqdm/tqdm/blob/74722959a8626fd2057be03e14dcf899c25a3fd5/tqdm/autonotebook.py#L14 + if ipython is not None and "IPKernelApp" in ipython.config: - class tqdm(_tqdm): # type: ignore[no-redef] - def __init__(self, *args, **kwargs): - kwargs = kwargs.copy() - if "file" not in kwargs: - kwargs["file"] = sys.stdout # avoid the red block in IPython + class tqdm(_tqdm): # type: ignore[no-redef] + def __init__(self, *args, **kwargs): + kwargs = kwargs.copy() + if "file" not in kwargs: + kwargs["file"] = sys.stdout # avoid the red block in IPython - super().__init__(*args, **kwargs) - else: - tqdm = _tqdm - except ImportError: + super().__init__(*args, **kwargs) + else: tqdm = _tqdm + except ImportError: + tqdm = _tqdm + + return tqdm - self.tqdm_mod = tqdm - self.pbars: dict[int, tqdm] = dict() +class ProgressBar: + def __init__(self, use_ray_tqdm: bool, show_tasks_bar: bool = False, disable: bool = False) -> None: + self.show_tasks_bar = show_tasks_bar + self._maxinterval = 5.0 + + self.use_ray_tqdm = use_ray_tqdm + self.tqdm_mod = get_tqdm(use_ray_tqdm) + + self.pbars: dict[int, Any] = dict() self.disable = ( disable or not bool(int(os.environ.get("RAY_TQDM", "1"))) @@ -102,3 +106,35 @@ def close(self) -> None: p.clear() p.close() del p + + +# Progress Bar for local execution, should only be used in the native executor +class SwordfishProgressBar: + def __init__(self) -> None: + self._maxinterval = 5.0 + self.tqdm_mod = get_tqdm(False) + self.pbars: dict[int, Any] = dict() + + def make_new_bar(self, bar_format: str, initial_message: str) -> int: + pbar_id = len(self.pbars) + self.pbars[pbar_id] = self.tqdm_mod( + bar_format=bar_format, + desc=initial_message, + position=pbar_id, + leave=False, + mininterval=1.0, + maxinterval=self._maxinterval, + ) + return pbar_id + + def update_bar(self, pbar_id: int, message: str) -> None: + self.pbars[pbar_id].set_description_str(message) + + def close_bar(self, pbar_id: int) -> None: + self.pbars[pbar_id].close() + del self.pbars[pbar_id] + + def close(self) -> None: + for p in self.pbars.values(): + p.close() + del p diff --git a/src/daft-local-execution/Cargo.toml b/src/daft-local-execution/Cargo.toml index 06ddd3efee..a30a51e342 100644 --- a/src/daft-local-execution/Cargo.toml +++ b/src/daft-local-execution/Cargo.toml @@ -23,6 +23,7 @@ daft-table = {path = "../daft-table", default-features = false} daft-writers = {path = "../daft-writers", default-features = false} futures = {workspace = true} indexmap = {workspace = true} +indicatif = "0.17.9" lazy_static = {workspace = true} log = {workspace = true} loole = "0.4.0" diff --git a/src/daft-local-execution/src/intermediate_ops/intermediate_op.rs b/src/daft-local-execution/src/intermediate_ops/intermediate_op.rs index 9b0782c782..693c0bad07 100644 --- a/src/daft-local-execution/src/intermediate_ops/intermediate_op.rs +++ b/src/daft-local-execution/src/intermediate_ops/intermediate_op.rs @@ -14,6 +14,7 @@ use crate::{ }, dispatcher::{DispatchSpawner, RoundRobinDispatcher, UnorderedDispatcher}, pipeline::PipelineNode, + progress_bar::ProgressBarColor, runtime_stats::{CountingReceiver, CountingSender, RuntimeStatsContext}, ExecutionRuntimeContext, OperatorOutput, PipelineExecutionSnafu, NUM_CPUS, }; @@ -201,11 +202,18 @@ impl PipelineNode for IntermediateNode { runtime_handle: &mut ExecutionRuntimeContext, ) -> crate::Result>> { let mut child_result_receivers = Vec::with_capacity(self.children.len()); + let progress_bar = runtime_handle.make_progress_bar( + self.name(), + ProgressBarColor::Magenta, + true, + self.runtime_stats.clone(), + ); for child in &self.children { let child_result_receiver = child.start(maintain_order, runtime_handle)?; child_result_receivers.push(CountingReceiver::new( child_result_receiver, self.runtime_stats.clone(), + progress_bar.clone(), )); } let op = self.intermediate_op.clone(); @@ -213,7 +221,8 @@ impl PipelineNode for IntermediateNode { node_name: self.name(), })?; let (destination_sender, destination_receiver) = create_channel(1); - let counting_sender = CountingSender::new(destination_sender, self.runtime_stats.clone()); + let counting_sender = + CountingSender::new(destination_sender, self.runtime_stats.clone(), progress_bar); let dispatch_spawner = self .intermediate_op @@ -246,6 +255,7 @@ impl PipelineNode for IntermediateNode { ); Ok(destination_receiver) } + fn as_tree_display(&self) -> &dyn TreeDisplay { self } diff --git a/src/daft-local-execution/src/lib.rs b/src/daft-local-execution/src/lib.rs index b1b5d99bb0..654deea901 100644 --- a/src/daft-local-execution/src/lib.rs +++ b/src/daft-local-execution/src/lib.rs @@ -6,6 +6,7 @@ mod channel; mod dispatcher; mod intermediate_ops; mod pipeline; +mod progress_bar; mod run; mod runtime_stats; mod sinks; @@ -15,13 +16,16 @@ mod state_bridge; use std::{ future::Future, pin::Pin, + sync::Arc, task::{Context, Poll}, }; use common_error::{DaftError, DaftResult}; use common_runtime::RuntimeTask; use lazy_static::lazy_static; +use progress_bar::{OperatorProgressBar, ProgressBarColor, ProgressBarManager}; pub use run::{run_local, ExecutionEngineResult, NativeExecutor}; +use runtime_stats::RuntimeStatsContext; use snafu::{futures::TryFutureExt, ResultExt, Snafu}; lazy_static! { @@ -116,14 +120,19 @@ impl RuntimeHandle { pub struct ExecutionRuntimeContext { worker_set: TaskSet>, default_morsel_size: usize, + progress_bar_manager: Option>, } impl ExecutionRuntimeContext { #[must_use] - pub fn new(default_morsel_size: usize) -> Self { + pub fn new( + default_morsel_size: usize, + progress_bar_manager: Option>, + ) -> Self { Self { worker_set: TaskSet::new(), default_morsel_size, + progress_bar_manager, } } pub fn spawn( @@ -149,11 +158,40 @@ impl ExecutionRuntimeContext { self.default_morsel_size } + pub fn make_progress_bar( + &self, + prefix: &str, + color: ProgressBarColor, + show_received: bool, + runtime_stats: Arc, + ) -> Option> { + if let Some(ref pb_manager) = self.progress_bar_manager { + let pb = pb_manager + .make_new_bar(color, prefix, show_received) + .unwrap(); + Some(Arc::new(OperatorProgressBar::new( + pb, + runtime_stats, + show_received, + ))) + } else { + None + } + } + pub(crate) fn handle(&self) -> RuntimeHandle { RuntimeHandle(tokio::runtime::Handle::current()) } } +impl Drop for ExecutionRuntimeContext { + fn drop(&mut self) { + if let Some(pbm) = self.progress_bar_manager.take() { + let _ = pbm.close_all(); + } + } +} + #[cfg(feature = "python")] use pyo3::prelude::*; diff --git a/src/daft-local-execution/src/progress_bar.rs b/src/daft-local-execution/src/progress_bar.rs new file mode 100644 index 0000000000..cf7fa17e3a --- /dev/null +++ b/src/daft-local-execution/src/progress_bar.rs @@ -0,0 +1,295 @@ +use std::{ + sync::{ + atomic::{AtomicU64, Ordering}, + Arc, + }, + time::Instant, +}; + +use common_error::DaftResult; +use indicatif::{HumanCount, ProgressStyle}; + +use crate::runtime_stats::RuntimeStatsContext; + +pub trait ProgressBar: Send + Sync { + fn set_message(&self, message: String) -> DaftResult<()>; + fn close(&self) -> DaftResult<()>; +} + +pub trait ProgressBarManager { + fn make_new_bar( + &self, + color: ProgressBarColor, + prefix: &str, + show_received: bool, + ) -> DaftResult>; + + fn close_all(&self) -> DaftResult<()>; +} + +pub enum ProgressBarColor { + Blue, + Magenta, + Cyan, +} + +impl ProgressBarColor { + fn to_str(&self) -> &'static str { + match self { + Self::Blue => "blue", + Self::Magenta => "magenta", + Self::Cyan => "cyan", + } + } +} + +pub struct OperatorProgressBar { + inner_progress_bar: Box, + runtime_stats: Arc, + show_received: bool, + start_time: Instant, + last_update: AtomicU64, +} + +impl OperatorProgressBar { + // 100ms = 100_000_000ns + const UPDATE_INTERVAL: u64 = 100_000_000; + + pub fn new( + progress_bar: Box, + runtime_stats: Arc, + show_received: bool, + ) -> Self { + Self { + inner_progress_bar: progress_bar, + runtime_stats, + show_received, + start_time: Instant::now(), + last_update: AtomicU64::new(0), + } + } + + fn should_update_progress_bar(&self, now: Instant) -> bool { + if now < self.start_time { + return false; + } + + let prev = self.last_update.load(Ordering::Acquire); + let elapsed = (now - self.start_time).as_nanos() as u64; + let diff = elapsed.saturating_sub(prev); + + // Fast path - check if enough time has passed + if diff < Self::UPDATE_INTERVAL { + return false; + } + + // Only calculate remainder if we're actually going to update + let remainder = diff % Self::UPDATE_INTERVAL; + self.last_update + .store(elapsed - remainder, Ordering::Release); + true + } + + pub fn render(&self) { + let now = std::time::Instant::now(); + if self.should_update_progress_bar(now) { + let rows_received = self.runtime_stats.get_rows_received(); + let rows_emitted = self.runtime_stats.get_rows_emitted(); + let msg = if self.show_received { + format!( + "{} rows received, {} rows emitted", + HumanCount(rows_received), + HumanCount(rows_emitted) + ) + } else { + format!("{} rows emitted", HumanCount(rows_emitted)) + }; + let _ = self.inner_progress_bar.set_message(msg); + } + } +} + +impl Drop for OperatorProgressBar { + fn drop(&mut self) { + let _ = self.inner_progress_bar.close(); + } +} + +struct IndicatifProgressBar(indicatif::ProgressBar); + +impl ProgressBar for IndicatifProgressBar { + fn set_message(&self, message: String) -> DaftResult<()> { + self.0.set_message(message); + Ok(()) + } + + fn close(&self) -> DaftResult<()> { + self.0.finish_and_clear(); + Ok(()) + } +} + +struct IndicatifProgressBarManager { + multi_progress: indicatif::MultiProgress, +} + +impl IndicatifProgressBarManager { + fn new() -> Self { + Self { + multi_progress: indicatif::MultiProgress::new(), + } + } +} + +impl ProgressBarManager for IndicatifProgressBarManager { + fn make_new_bar( + &self, + color: ProgressBarColor, + prefix: &str, + show_received: bool, + ) -> DaftResult> { + let template_str = format!( + "๐Ÿ—ก๏ธ ๐ŸŸ {{spinner:.green}} {{prefix:.{color}/bold}} | [{{elapsed_precise}}] {{msg}}", + color = color.to_str(), + ); + + let initial_message = if show_received { + "0 rows received, 0 rows emitted".to_string() + } else { + "0 rows emitted".to_string() + }; + + let pb = indicatif::ProgressBar::new_spinner() + .with_style( + ProgressStyle::default_spinner() + .template(template_str.as_str()) + .unwrap(), + ) + .with_prefix(prefix.to_string()) + .with_message(initial_message); + + self.multi_progress.add(pb.clone()); + DaftResult::Ok(Box::new(IndicatifProgressBar(pb))) + } + + fn close_all(&self) -> DaftResult<()> { + Ok(self.multi_progress.clear()?) + } +} + +pub fn make_progress_bar_manager() -> Box { + #[cfg(feature = "python")] + { + if python::in_notebook() { + Box::new(python::TqdmProgressBarManager::new()) + } else { + Box::new(IndicatifProgressBarManager::new()) + } + } + + #[cfg(not(feature = "python"))] + { + Box::new(IndicatifProgressBarManager::new()) + } +} + +#[cfg(feature = "python")] +mod python { + use pyo3::{types::PyAnyMethods, PyObject, Python}; + + use super::*; + + pub fn in_notebook() -> bool { + pyo3::Python::with_gil(|py| { + py.import_bound(pyo3::intern!(py, "daft.utils")) + .and_then(|m| m.getattr(pyo3::intern!(py, "in_notebook"))) + .and_then(|m| m.call0()) + .and_then(|m| m.extract()) + .expect("Failed to determine if running in notebook") + }) + } + + struct TqdmProgressBar { + pb_id: usize, + manager: TqdmProgressBarManager, + } + + impl ProgressBar for TqdmProgressBar { + fn set_message(&self, message: String) -> DaftResult<()> { + self.manager.update_bar(self.pb_id, message.as_str()) + } + + fn close(&self) -> DaftResult<()> { + self.manager.close_bar(self.pb_id) + } + } + + #[derive(Clone)] + pub struct TqdmProgressBarManager { + inner: PyObject, + } + + impl TqdmProgressBarManager { + pub fn new() -> Self { + Python::with_gil(|py| { + let module = py.import_bound("daft.runners.progress_bar")?; + let progress_bar_class = module.getattr("SwordfishProgressBar")?; + let pb_object = progress_bar_class.call0()?; + DaftResult::Ok(Self { + inner: pb_object.into(), + }) + }) + .expect("Failed to create progress bar") + } + + fn update_bar(&self, pb_id: usize, message: &str) -> DaftResult<()> { + Python::with_gil(|py| { + self.inner + .call_method1(py, "update_bar", (pb_id, message))?; + DaftResult::Ok(()) + }) + } + + fn close_bar(&self, pb_id: usize) -> DaftResult<()> { + Python::with_gil(|py| { + self.inner.call_method1(py, "close_bar", (pb_id,))?; + DaftResult::Ok(()) + }) + } + } + + impl ProgressBarManager for TqdmProgressBarManager { + fn make_new_bar( + &self, + _color: ProgressBarColor, + prefix: &str, + show_received: bool, + ) -> DaftResult> { + let bar_format = format!("๐Ÿ—ก๏ธ ๐ŸŸ {prefix}: {{elapsed}} {{desc}}", prefix = prefix); + let initial_message = if show_received { + "0 rows received, 0 rows emitted".to_string() + } else { + "0 rows emitted".to_string() + }; + let pb_id = Python::with_gil(|py| { + let pb_id = + self.inner + .call_method1(py, "make_new_bar", (bar_format, initial_message))?; + let pb_id = pb_id.extract::(py)?; + DaftResult::Ok(pb_id) + })?; + + DaftResult::Ok(Box::new(TqdmProgressBar { + pb_id, + manager: self.clone(), + })) + } + + fn close_all(&self) -> DaftResult<()> { + Python::with_gil(|py| { + self.inner.call_method0(py, "close")?; + DaftResult::Ok(()) + }) + } + } +} diff --git a/src/daft-local-execution/src/run.rs b/src/daft-local-execution/src/run.rs index 32b92c9015..db630b3d11 100644 --- a/src/daft-local-execution/src/run.rs +++ b/src/daft-local-execution/src/run.rs @@ -29,6 +29,7 @@ use { use crate::{ channel::{create_channel, Receiver}, pipeline::{physical_plan_to_pipeline, viz_pipeline}, + progress_bar::make_progress_bar_manager, Error, ExecutionRuntimeContext, }; @@ -160,6 +161,15 @@ fn should_enable_explain_analyze() -> bool { } } +fn should_enable_progress_bar() -> bool { + let progress_var_name = "DAFT_PROGRESS_BAR"; + if let Ok(val) = std::env::var(progress_var_name) { + matches!(val.trim().to_lowercase().as_str(), "1" | "true") + } else { + true // Return true when env var is not set + } +} + pub struct ExecutionEngineReceiverIterator { receiver: Receiver>, handle: Option>>, @@ -263,12 +273,14 @@ pub fn run_local( let pipeline = physical_plan_to_pipeline(physical_plan, psets, &cfg)?; let (tx, rx) = create_channel(results_buffer_size.unwrap_or(1)); let handle = std::thread::spawn(move || { + let pb_manager = should_enable_progress_bar().then(make_progress_bar_manager); let runtime = tokio::runtime::Builder::new_current_thread() .enable_all() .build() .expect("Failed to create tokio runtime"); let execution_task = async { - let mut runtime_handle = ExecutionRuntimeContext::new(cfg.default_morsel_size); + let mut runtime_handle = + ExecutionRuntimeContext::new(cfg.default_morsel_size, pb_manager); let receiver = pipeline.start(true, &mut runtime_handle)?; while let Some(val) = receiver.recv().await { diff --git a/src/daft-local-execution/src/runtime_stats.rs b/src/daft-local-execution/src/runtime_stats.rs index 3ef944ba9b..b35fd2d864 100644 --- a/src/daft-local-execution/src/runtime_stats.rs +++ b/src/daft-local-execution/src/runtime_stats.rs @@ -8,7 +8,10 @@ use std::{ use daft_micropartition::MicroPartition; use loole::SendError; -use crate::channel::{Receiver, Sender}; +use crate::{ + channel::{Receiver, Sender}, + progress_bar::OperatorProgressBar, +}; #[derive(Default)] pub struct RuntimeStatsContext { @@ -86,6 +89,16 @@ impl RuntimeStatsContext { self.rows_emitted .fetch_add(rows, std::sync::atomic::Ordering::Relaxed); } + + pub(crate) fn get_rows_received(&self) -> u64 { + self.rows_received + .load(std::sync::atomic::Ordering::Relaxed) + } + + pub(crate) fn get_rows_emitted(&self) -> u64 { + self.rows_emitted.load(std::sync::atomic::Ordering::Relaxed) + } + #[allow(unused)] pub(crate) fn reset(&self) { self.rows_received @@ -109,11 +122,20 @@ impl RuntimeStatsContext { pub struct CountingSender { sender: Sender>, rt: Arc, + progress_bar: Option>, } impl CountingSender { - pub(crate) fn new(sender: Sender>, rt: Arc) -> Self { - Self { sender, rt } + pub(crate) fn new( + sender: Sender>, + rt: Arc, + progress_bar: Option>, + ) -> Self { + Self { + sender, + rt, + progress_bar, + } } #[inline] pub(crate) async fn send( @@ -121,6 +143,9 @@ impl CountingSender { v: Arc, ) -> Result<(), SendError>> { self.rt.mark_rows_emitted(v.len() as u64); + if let Some(ref pb) = self.progress_bar { + pb.render(); + } self.sender.send(v).await?; Ok(()) } @@ -129,20 +154,29 @@ impl CountingSender { pub struct CountingReceiver { receiver: Receiver>, rt: Arc, + progress_bar: Option>, } impl CountingReceiver { pub(crate) fn new( receiver: Receiver>, rt: Arc, + progress_bar: Option>, ) -> Self { - Self { receiver, rt } + Self { + receiver, + rt, + progress_bar, + } } #[inline] pub(crate) async fn recv(&self) -> Option> { let v = self.receiver.recv().await; if let Some(ref v) = v { self.rt.mark_rows_received(v.len() as u64); + if let Some(ref pb) = self.progress_bar { + pb.render(); + } } v } diff --git a/src/daft-local-execution/src/sinks/blocking_sink.rs b/src/daft-local-execution/src/sinks/blocking_sink.rs index 03660d72f3..3344f5ddf7 100644 --- a/src/daft-local-execution/src/sinks/blocking_sink.rs +++ b/src/daft-local-execution/src/sinks/blocking_sink.rs @@ -11,6 +11,7 @@ use crate::{ channel::{create_channel, Receiver}, dispatcher::{DispatchSpawner, UnorderedDispatcher}, pipeline::PipelineNode, + progress_bar::ProgressBarColor, runtime_stats::{CountingReceiver, CountingSender, RuntimeStatsContext}, ExecutionRuntimeContext, JoinSnafu, OperatorOutput, TaskSet, }; @@ -144,12 +145,22 @@ impl PipelineNode for BlockingSinkNode { _maintain_order: bool, runtime_handle: &mut ExecutionRuntimeContext, ) -> crate::Result>> { + let progress_bar = runtime_handle.make_progress_bar( + self.name(), + ProgressBarColor::Cyan, + true, + self.runtime_stats.clone(), + ); let child_results_receiver = self.child.start(false, runtime_handle)?; - let counting_receiver = - CountingReceiver::new(child_results_receiver, self.runtime_stats.clone()); + let counting_receiver = CountingReceiver::new( + child_results_receiver, + self.runtime_stats.clone(), + progress_bar.clone(), + ); let (destination_sender, destination_receiver) = create_channel(1); - let counting_sender = CountingSender::new(destination_sender, self.runtime_stats.clone()); + let counting_sender = + CountingSender::new(destination_sender, self.runtime_stats.clone(), progress_bar); let op = self.op.clone(); let runtime_stats = self.runtime_stats.clone(); diff --git a/src/daft-local-execution/src/sinks/streaming_sink.rs b/src/daft-local-execution/src/sinks/streaming_sink.rs index 9d5452cf4b..a54af796cd 100644 --- a/src/daft-local-execution/src/sinks/streaming_sink.rs +++ b/src/daft-local-execution/src/sinks/streaming_sink.rs @@ -14,6 +14,7 @@ use crate::{ }, dispatcher::DispatchSpawner, pipeline::PipelineNode, + progress_bar::ProgressBarColor, runtime_stats::{CountingReceiver, CountingSender, RuntimeStatsContext}, ExecutionRuntimeContext, JoinSnafu, OperatorOutput, TaskSet, NUM_CPUS, }; @@ -195,17 +196,25 @@ impl PipelineNode for StreamingSinkNode { maintain_order: bool, runtime_handle: &mut ExecutionRuntimeContext, ) -> crate::Result>> { + let progress_bar = runtime_handle.make_progress_bar( + self.name(), + ProgressBarColor::Cyan, + true, + self.runtime_stats.clone(), + ); let mut child_result_receivers = Vec::with_capacity(self.children.len()); for child in &self.children { let child_result_receiver = child.start(maintain_order, runtime_handle)?; child_result_receivers.push(CountingReceiver::new( child_result_receiver, self.runtime_stats.clone(), + progress_bar.clone(), )); } let (destination_sender, destination_receiver) = create_channel(1); - let counting_sender = CountingSender::new(destination_sender, self.runtime_stats.clone()); + let counting_sender = + CountingSender::new(destination_sender, self.runtime_stats.clone(), progress_bar); let op = self.op.clone(); let runtime_stats = self.runtime_stats.clone(); diff --git a/src/daft-local-execution/src/sources/source.rs b/src/daft-local-execution/src/sources/source.rs index 8a2dd8c45c..2f0f9a2e23 100644 --- a/src/daft-local-execution/src/sources/source.rs +++ b/src/daft-local-execution/src/sources/source.rs @@ -11,6 +11,7 @@ use futures::{stream::BoxStream, StreamExt}; use crate::{ channel::{create_channel, Receiver}, pipeline::PipelineNode, + progress_bar::ProgressBarColor, runtime_stats::{CountingSender, RuntimeStatsContext}, ExecutionRuntimeContext, }; @@ -76,10 +77,17 @@ impl PipelineNode for SourceNode { maintain_order: bool, runtime_handle: &mut ExecutionRuntimeContext, ) -> crate::Result>> { + let progress_bar = runtime_handle.make_progress_bar( + self.name(), + ProgressBarColor::Blue, + false, + self.runtime_stats.clone(), + ); let source = self.source.clone(); let io_stats = self.io_stats.clone(); let (destination_sender, destination_receiver) = create_channel(1); - let counting_sender = CountingSender::new(destination_sender, self.runtime_stats.clone()); + let counting_sender = + CountingSender::new(destination_sender, self.runtime_stats.clone(), progress_bar); runtime_handle.spawn( async move { let mut has_data = false; From 5e40837ad9c7b860371fa8ee916c840e54a82233 Mon Sep 17 00:00:00 2001 From: Colin Ho Date: Wed, 18 Dec 2024 22:09:47 -0800 Subject: [PATCH 05/11] feat: Default native runner (#3608) Set swordfish as default Note: - Disables printing of physical plan explains for now --------- Co-authored-by: Colin Ho --- daft/context.py | 9 +-------- daft/dataframe/dataframe.py | 7 ++++--- daft/runners/pyrunner.py | 4 ++-- src/daft-local-plan/src/translate.rs | 2 +- 4 files changed, 8 insertions(+), 14 deletions(-) diff --git a/daft/context.py b/daft/context.py index dbec115099..6c18ca2998 100644 --- a/daft/context.py +++ b/daft/context.py @@ -7,7 +7,6 @@ import warnings from typing import TYPE_CHECKING, ClassVar, Literal -from daft import get_build_type from daft.daft import IOConfig, PyDaftExecutionConfig, PyDaftPlanningConfig if TYPE_CHECKING: @@ -109,14 +108,8 @@ def _get_runner_config_from_env() -> _RunnerConfig: max_task_backlog=task_backlog, force_client_mode=ray_force_client_mode, ) - - # Use native runner if in dev mode - elif get_build_type() == "dev": - return _NativeRunnerConfig() - - # Fall back on PyRunner else: - return _PyRunnerConfig(use_thread_pool=use_thread_pool) + return _NativeRunnerConfig() @dataclasses.dataclass diff --git a/daft/dataframe/dataframe.py b/daft/dataframe/dataframe.py index 4264883c28..ca23c73cbb 100644 --- a/daft/dataframe/dataframe.py +++ b/daft/dataframe/dataframe.py @@ -208,9 +208,10 @@ def explain( print_to_file("\n== Optimized Logical Plan ==\n") builder = builder.optimize() print_to_file(builder.pretty_print(simple)) - print_to_file("\n== Physical Plan ==\n") - physical_plan_scheduler = builder.to_physical_plan_scheduler(get_context().daft_execution_config) - print_to_file(physical_plan_scheduler.pretty_print(simple, format=format)) + if get_context().get_or_create_runner().name != "native": + print_to_file("\n== Physical Plan ==\n") + physical_plan_scheduler = builder.to_physical_plan_scheduler(get_context().daft_execution_config) + print_to_file(physical_plan_scheduler.pretty_print(simple, format=format)) else: print_to_file( "\n \nSet `show_all=True` to also see the Optimized and Physical plans. This will run the query optimizer.", diff --git a/daft/runners/pyrunner.py b/daft/runners/pyrunner.py index 3fb6b61277..450bc4eb57 100644 --- a/daft/runners/pyrunner.py +++ b/daft/runners/pyrunner.py @@ -340,8 +340,8 @@ def run_iter( results_buffer_size: int | None = None, ) -> Iterator[LocalMaterializedResult]: warnings.warn( - "PyRunner will be deprecated in v0.4.0 and the new NativeRunner will become the default for local execution." - "We recommend switching to the NativeRunner now via `daft.context.set_runner_native()` or by setting the env variable `DAFT_RUNNER=native`. " + "PyRunner is deprecated and the new NativeRunner is now the default for local execution." + "Please switch to the NativeRunner now via `daft.context.set_runner_native()` or by setting the env variable `DAFT_RUNNER=native`. " "Please report any issues at github.com/Eventual-Inc/Daft/issues", ) # NOTE: Freeze and use this same execution config for the entire execution diff --git a/src/daft-local-plan/src/translate.rs b/src/daft-local-plan/src/translate.rs index e876c350a8..0547e9e737 100644 --- a/src/daft-local-plan/src/translate.rs +++ b/src/daft-local-plan/src/translate.rs @@ -197,7 +197,7 @@ pub fn translate(plan: &LogicalPlanRef) -> DaftResult { )) } LogicalPlan::Repartition(repartition) => { - log::warn!("Repartition Not supported for Local Executor!; This will be a No-Op"); + log::warn!("Repartition not supported on the NativeRunner. This will be a no-op. Please use the RayRunner instead if you need to repartition"); translate(&repartition.input) } LogicalPlan::MonotonicallyIncreasingId(monotonically_increasing_id) => { From 063de4d78479863eea2c6fa1751f24e2e90f152d Mon Sep 17 00:00:00 2001 From: Jay Chia <17691182+jaychia@users.noreply.github.com> Date: Thu, 19 Dec 2024 14:48:45 +0800 Subject: [PATCH 06/11] chore: add warning for native runner (#3613) Co-authored-by: Jay Chia --- daft/context.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/daft/context.py b/daft/context.py index 6c18ca2998..7f975ae83e 100644 --- a/daft/context.py +++ b/daft/context.py @@ -165,6 +165,12 @@ def get_or_create_runner(self) -> Runner: elif runner_config.name == "native": from daft.runners.native_runner import NativeRunner + warnings.warn( + "Daft is configured to use the new NativeRunner by default as of v0.4.0. " + "If you are encountering any regressions, please switch back to the legacy PyRunner via `daft.context.set_runner_py()` or by setting the env variable `DAFT_RUNNER=py`. " + "We appreciate you filing issues and helping make the NativeRunner better: https://github.com/Eventual-Inc/Daft/issues", + ) + assert isinstance(runner_config, _NativeRunnerConfig) self._runner = NativeRunner() From a76f8003b770fe84101ef0493fbd0fe4660fe313 Mon Sep 17 00:00:00 2001 From: Jay Chia <17691182+jaychia@users.noreply.github.com> Date: Thu, 19 Dec 2024 16:31:07 +0800 Subject: [PATCH 07/11] chore!: upgrade Ray pins and pyarrow pins (#3612) Updates the lower bound of pyarrow to `pyarrow>=8.0.0`. This allows us to flatten some code checks. However, it turns out that our tests aren't being properly skipped -- I had to update the tests to just skip based on our lower bound (skip if version < 9.0.0) which very loose, but otherwise searching for the individual versions for each suite of tests was quite difficult. --------- Co-authored-by: Jay Chia --- .github/workflows/python-package.yml | 12 ++++++------ daft/table/table_io.py | 13 ++++--------- pyproject.toml | 2 +- tests/integration/iceberg/conftest.py | 6 ++++-- tests/io/delta_lake/test_table_read.py | 6 +++--- tests/io/delta_lake/test_table_read_pushdowns.py | 6 +++--- tests/io/delta_lake/test_table_write.py | 6 +++--- tests/io/hudi/test_table_read.py | 4 ++-- tests/io/iceberg/test_iceberg_writes.py | 4 ++-- tests/io/lancedb/test_lancedb_reads.py | 4 ++-- tests/io/lancedb/test_lancedb_writes.py | 5 ++--- 11 files changed, 32 insertions(+), 36 deletions(-) diff --git a/.github/workflows/python-package.yml b/.github/workflows/python-package.yml index c08f570fd2..17a7dd793c 100644 --- a/.github/workflows/python-package.yml +++ b/.github/workflows/python-package.yml @@ -26,26 +26,26 @@ jobs: matrix: python-version: ['3.9', '3.10'] daft-runner: [py, ray, native] - pyarrow-version: [7.0.0, 16.0.0] + pyarrow-version: [8.0.0, 16.0.0] os: [ubuntu-20.04, windows-latest] exclude: - daft-runner: ray - pyarrow-version: 7.0.0 + pyarrow-version: 8.0.0 os: ubuntu-20.04 - daft-runner: py python-version: '3.10' - pyarrow-version: 7.0.0 + pyarrow-version: 8.0.0 os: ubuntu-20.04 - daft-runner: native python-version: '3.10' - pyarrow-version: 7.0.0 + pyarrow-version: 8.0.0 os: ubuntu-20.04 - python-version: '3.9' pyarrow-version: 16.0.0 - os: windows-latest python-version: '3.9' - os: windows-latest - pyarrow-version: 7.0.0 + pyarrow-version: 8.0.0 steps: - uses: actions/checkout@v4 - uses: moonrepo/setup-rust@v1 @@ -93,7 +93,7 @@ jobs: run: uv pip install pyarrow==${{ matrix.pyarrow-version }} - name: Override deltalake for pyarrow - if: ${{ (matrix.pyarrow-version == '7.0.0') }} + if: ${{ (matrix.pyarrow-version == '8.0.0') }} run: uv pip install deltalake==0.10.0 - name: Build library and Test with pytest (unix) diff --git a/daft/table/table_io.py b/daft/table/table_io.py index 30744d57d3..66f27d53bc 100644 --- a/daft/table/table_io.py +++ b/daft/table/table_io.py @@ -554,16 +554,11 @@ def _write_tabular_arrow_table( ): kwargs = dict() - from daft.utils import get_arrow_version + kwargs["max_rows_per_file"] = rows_per_file + kwargs["min_rows_per_group"] = rows_per_row_group + kwargs["max_rows_per_group"] = rows_per_row_group - arrow_version = get_arrow_version() - - if arrow_version >= (7, 0, 0): - kwargs["max_rows_per_file"] = rows_per_file - kwargs["min_rows_per_group"] = rows_per_row_group - kwargs["max_rows_per_group"] = rows_per_row_group - - if arrow_version >= (8, 0, 0) and not create_dir: + if not create_dir: kwargs["create_dir"] = False basename_template = _generate_basename_template(format.default_extname, version) diff --git a/pyproject.toml b/pyproject.toml index ff692da981..225bb8fc9a 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -5,7 +5,7 @@ requires = ["maturin>=1.5.0,<2.0.0"] [project] authors = [{name = "Eventual Inc", email = "daft@eventualcomputing.com"}] dependencies = [ - "pyarrow >= 7.0.0", + "pyarrow >= 8.0.0", "fsspec", "tqdm", "typing-extensions >= 4.0.0; python_version < '3.10'" diff --git a/tests/integration/iceberg/conftest.py b/tests/integration/iceberg/conftest.py index 6550571260..b03e419ca4 100644 --- a/tests/integration/iceberg/conftest.py +++ b/tests/integration/iceberg/conftest.py @@ -10,8 +10,10 @@ pyiceberg = pytest.importorskip("pyiceberg") -PYARROW_LE_8_0_0 = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < (8, 0, 0) -pytestmark = pytest.mark.skipif(PYARROW_LE_8_0_0, reason="iceberg writes only supported if pyarrow >= 8.0.0") +PYARROW_LOWER_BOUND_SKIP = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < (9, 0, 0) +pytestmark = pytest.mark.skipif( + PYARROW_LOWER_BOUND_SKIP, reason="iceberg writes not supported on old versions of pyarrow" +) import tenacity from pyiceberg.catalog import Catalog, load_catalog diff --git a/tests/io/delta_lake/test_table_read.py b/tests/io/delta_lake/test_table_read.py index bf1d0fe2d9..304086bfe0 100644 --- a/tests/io/delta_lake/test_table_read.py +++ b/tests/io/delta_lake/test_table_read.py @@ -8,10 +8,10 @@ from daft.logical.schema import Schema from tests.utils import assert_pyarrow_tables_equal -PYARROW_LE_8_0_0 = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < (8, 0, 0) +PYARROW_LOWER_BOUND_SKIP = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < (9, 0, 0) pytestmark = pytest.mark.skipif( - PYARROW_LE_8_0_0, - reason="deltalake only supported if pyarrow >= 8.0.0", + PYARROW_LOWER_BOUND_SKIP, + reason="deltalake not supported on older versions of pyarrow", ) diff --git a/tests/io/delta_lake/test_table_read_pushdowns.py b/tests/io/delta_lake/test_table_read_pushdowns.py index 95c57ecc8a..3144009e2b 100644 --- a/tests/io/delta_lake/test_table_read_pushdowns.py +++ b/tests/io/delta_lake/test_table_read_pushdowns.py @@ -16,10 +16,10 @@ from daft.logical.schema import Schema from tests.utils import assert_pyarrow_tables_equal -PYARROW_LE_8_0_0 = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < (8, 0, 0) +PYARROW_LOWER_BOUND_SKIP = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < (9, 0, 0) pytestmark = pytest.mark.skipif( - PYARROW_LE_8_0_0, - reason="deltalake only supported if pyarrow >= 8.0.0", + PYARROW_LOWER_BOUND_SKIP, + reason="deltalake not supported on older versions of pyarrow", ) diff --git a/tests/io/delta_lake/test_table_write.py b/tests/io/delta_lake/test_table_write.py index ef6772740f..461c3acd35 100644 --- a/tests/io/delta_lake/test_table_write.py +++ b/tests/io/delta_lake/test_table_write.py @@ -12,10 +12,10 @@ from daft.logical.schema import Schema from tests.conftest import get_tests_daft_runner_name -PYARROW_LE_8_0_0 = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < (8, 0, 0) +PYARROW_LOWER_BOUND_SKIP = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < (9, 0, 0) pytestmark = pytest.mark.skipif( - PYARROW_LE_8_0_0, - reason="deltalake only supported if pyarrow >= 8.0.0", + PYARROW_LOWER_BOUND_SKIP, + reason="deltalake not supported on older versions of pyarrow", ) diff --git a/tests/io/hudi/test_table_read.py b/tests/io/hudi/test_table_read.py index f0207e9383..509535ae02 100644 --- a/tests/io/hudi/test_table_read.py +++ b/tests/io/hudi/test_table_read.py @@ -7,8 +7,8 @@ import daft -PYARROW_LE_8_0_0 = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < (8, 0, 0) -pytestmark = pytest.mark.skipif(PYARROW_LE_8_0_0, reason="hudi only supported if pyarrow >= 8.0.0") +PYARROW_LOWER_BOUND_SKIP = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < (9, 0, 0) +pytestmark = pytest.mark.skipif(PYARROW_LOWER_BOUND_SKIP, reason="hudi not supported on old versions of pyarrow") def test_read_table(get_testing_table_for_supported_cases): diff --git a/tests/io/iceberg/test_iceberg_writes.py b/tests/io/iceberg/test_iceberg_writes.py index 8282ed1626..989abb7149 100644 --- a/tests/io/iceberg/test_iceberg_writes.py +++ b/tests/io/iceberg/test_iceberg_writes.py @@ -10,8 +10,8 @@ pyiceberg = pytest.importorskip("pyiceberg") -PYARROW_LE_8_0_0 = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < (8, 0, 0) -pytestmark = pytest.mark.skipif(PYARROW_LE_8_0_0, reason="iceberg only supported if pyarrow >= 8.0.0") +PYARROW_LOWER_BOUND_SKIP = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < (9, 0, 0) +pytestmark = pytest.mark.skipif(PYARROW_LOWER_BOUND_SKIP, reason="iceberg not supported on old versions of pyarrow") from pyiceberg.catalog.sql import SqlCatalog diff --git a/tests/io/lancedb/test_lancedb_reads.py b/tests/io/lancedb/test_lancedb_reads.py index ad3062ee19..2e5581e5cc 100644 --- a/tests/io/lancedb/test_lancedb_reads.py +++ b/tests/io/lancedb/test_lancedb_reads.py @@ -11,8 +11,8 @@ "long": [-122.7, -74.1], } -PYARROW_LE_8_0_0 = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < (8, 0, 0) -pytestmark = pytest.mark.skipif(PYARROW_LE_8_0_0, reason="lance only supported if pyarrow >= 8.0.0") +PYARROW_LOWER_BOUND_SKIP = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < (9, 0, 0) +pytestmark = pytest.mark.skipif(PYARROW_LOWER_BOUND_SKIP, reason="lance not supported on old versions of pyarrow") @pytest.fixture(scope="function") diff --git a/tests/io/lancedb/test_lancedb_writes.py b/tests/io/lancedb/test_lancedb_writes.py index e45c3b60d6..daf75e092d 100644 --- a/tests/io/lancedb/test_lancedb_writes.py +++ b/tests/io/lancedb/test_lancedb_writes.py @@ -12,9 +12,8 @@ "long": [-122.7, -74.1], } -PYARROW_LE_8_0_0 = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < (8, 0, 0) - -pytestmark = pytest.mark.skipif(PYARROW_LE_8_0_0, reason="lance only supported if pyarrow >= 8.0.0") +PYARROW_LOWER_BOUND_SKIP = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) < (9, 0, 0) +pytestmark = pytest.mark.skipif(PYARROW_LOWER_BOUND_SKIP, reason="lance not supported on old versions of pyarrow") @pytest.fixture(scope="function") From fb66ed1af9796b214b9085d3edc127ea01d938de Mon Sep 17 00:00:00 2001 From: ccmao1130 Date: Thu, 19 Dec 2024 00:44:35 -0800 Subject: [PATCH 08/11] docs: daft documentation v2 (#3595) Revamping Daft Documentation with MkDocs - [ ] Complete FAQ section on Overview page - [ ] Delete `core_concepts` folder if we decide to keep mega `core_concepts.md` page (or vice versa) - [ ] Complete all `todo(docs)` - [ ] **!! Update methods to link to API docs (not URL)** After MVP: - [ ] **!! Coming from Spark Guide** - [ ] Incorporate Daft Launcher docs - [ ] Can/Should we combine ^ and distributed computing page? - [ ] SQL docs, 1-1 mapping to expressions - [ ] Update wording for Integrations/SQL page (not clear the difference between this and daft-sql) - [ ] Add more `See Also` or `What's Next` across all pages - [ ] More diagrams in `core_concepts.md` (and other pages) where applicable (See Integrations/SQL pages as example) --------- Co-authored-by: Sammy Sidhu --- .pre-commit-config.yaml | 2 + docs-v2/10min.ipynb | 1797 +++++++++++++ docs-v2/advanced/distributed.md | 72 + docs-v2/advanced/memory.md | 66 + docs-v2/advanced/partitioning.md | 113 + docs-v2/core_concepts.md | 2335 +++++++++++++++++ docs-v2/core_concepts/aggregations.md | 111 + docs-v2/core_concepts/dataframe.md | 654 +++++ docs-v2/core_concepts/datatypes.md | 96 + docs-v2/core_concepts/expressions.md | 744 ++++++ docs-v2/core_concepts/read_write.md | 142 + docs-v2/core_concepts/sql.md | 224 ++ docs-v2/core_concepts/udf.md | 213 ++ docs-v2/img/architecture.png | Bin 0 -> 32822 bytes docs-v2/img/daft.png | Bin 0 -> 58145 bytes docs-v2/img/daft_diagram.png | Bin 0 -> 42148 bytes docs-v2/img/execution_model.png | Bin 0 -> 25200 bytes docs-v2/img/favicon.png | Bin 0 -> 13989 bytes docs-v2/img/in_memory_data_representation.png | Bin 0 -> 18177 bytes docs-v2/img/sql_distributed_read.png | Bin 0 -> 234381 bytes docs-v2/index.md | 173 ++ docs-v2/install.md | 47 + docs-v2/integrations/aws.md | 53 + docs-v2/integrations/azure.md | 80 + docs-v2/integrations/delta_lake.md | 128 + docs-v2/integrations/hudi.md | 76 + docs-v2/integrations/huggingface.md | 69 + docs-v2/integrations/iceberg.md | 110 + docs-v2/integrations/ray.md | 93 + docs-v2/integrations/sql.md | 159 ++ docs-v2/integrations/unity_catalog.md | 69 + docs-v2/migration/dask_migration.md | 132 + docs-v2/quickstart.ipynb | 807 ++++++ docs-v2/quickstart.md | 418 +++ docs-v2/resources/architecture.md | 94 + docs-v2/resources/benchmarks/tpch-1000sf.html | 2 + docs-v2/resources/benchmarks/tpch-100sf.html | 2 + .../tpch-nodes-count-daft-1000-sf.html | 2 + docs-v2/resources/benchmarks/tpch.md | 153 ++ docs-v2/resources/dataframe_comparison.md | 76 + docs-v2/resources/telemetry.md | 22 + docs-v2/resources/tutorials.md | 45 + docs-v2/terms.md | 91 + mkdocs.yml | 141 + 44 files changed, 9611 insertions(+) create mode 100644 docs-v2/10min.ipynb create mode 100644 docs-v2/advanced/distributed.md create mode 100644 docs-v2/advanced/memory.md create mode 100644 docs-v2/advanced/partitioning.md create mode 100644 docs-v2/core_concepts.md create mode 100644 docs-v2/core_concepts/aggregations.md create mode 100644 docs-v2/core_concepts/dataframe.md create mode 100644 docs-v2/core_concepts/datatypes.md create mode 100644 docs-v2/core_concepts/expressions.md create mode 100644 docs-v2/core_concepts/read_write.md create mode 100644 docs-v2/core_concepts/sql.md create mode 100644 docs-v2/core_concepts/udf.md create mode 100644 docs-v2/img/architecture.png create mode 100644 docs-v2/img/daft.png create mode 100644 docs-v2/img/daft_diagram.png create mode 100644 docs-v2/img/execution_model.png create mode 100644 docs-v2/img/favicon.png create mode 100644 docs-v2/img/in_memory_data_representation.png create mode 100644 docs-v2/img/sql_distributed_read.png create mode 100644 docs-v2/index.md create mode 100644 docs-v2/install.md create mode 100644 docs-v2/integrations/aws.md create mode 100644 docs-v2/integrations/azure.md create mode 100644 docs-v2/integrations/delta_lake.md create mode 100644 docs-v2/integrations/hudi.md create mode 100644 docs-v2/integrations/huggingface.md create mode 100644 docs-v2/integrations/iceberg.md create mode 100644 docs-v2/integrations/ray.md create mode 100644 docs-v2/integrations/sql.md create mode 100644 docs-v2/integrations/unity_catalog.md create mode 100644 docs-v2/migration/dask_migration.md create mode 100644 docs-v2/quickstart.ipynb create mode 100644 docs-v2/quickstart.md create mode 100644 docs-v2/resources/architecture.md create mode 100644 docs-v2/resources/benchmarks/tpch-1000sf.html create mode 100644 docs-v2/resources/benchmarks/tpch-100sf.html create mode 100644 docs-v2/resources/benchmarks/tpch-nodes-count-daft-1000-sf.html create mode 100644 docs-v2/resources/benchmarks/tpch.md create mode 100644 docs-v2/resources/dataframe_comparison.md create mode 100644 docs-v2/resources/telemetry.md create mode 100644 docs-v2/resources/tutorials.md create mode 100644 docs-v2/terms.md create mode 100644 mkdocs.yml diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index 4362e615db..16021912bd 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -23,6 +23,8 @@ repos: )$ - id: check-yaml exclude: kubernetes-ops + args: + - --unsafe - id: pretty-format-json exclude: | (?x)^( diff --git a/docs-v2/10min.ipynb b/docs-v2/10min.ipynb new file mode 100644 index 0000000000..5e3eef1816 --- /dev/null +++ b/docs-v2/10min.ipynb @@ -0,0 +1,1797 @@ +{ + "cells": [ + { + "attachments": {}, + "cell_type": "markdown", + "metadata": {}, + "source": [ + "> \ud83d\udca1 **Hint** \n", + ">\n", + "> \u2728\u2728 Run this notebook on Google Colab \u2728\u2728 \n", + "> \n", + "> You can [run this notebook yourself with Google Colab](#)." + ] + }, + { + "attachments": {}, + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# 10 minutes Quickstart\n", + "\n", + "This is a short introduction to all the main functionality in Daft, geared towards new users.\n", + "\n", + "## What is Daft?\n", + "Daft is a distributed query engine built for running ETL, analytics, and ML/AI workloads at scale. Daft is implemented in Rust (fast!) and exposes a familiar Python dataframe API (friendly!). \n", + "\n", + "In this Quickstart you will learn the basics of Daft\u2019s DataFrame API and the features that set it apart from frameworks like pandas, pySpark, Dask and Ray. You will build a database of dog owners and their fluffy companions and see how you can use Daft to download images from URLs, run an ML classifier and call custom UDFs, all within an interactive DataFrame interface. Woof! \ud83d\udc36\n", + "\n", + "## When Should I use Daft?\n", + "\n", + "Daft is the right tool for you if you are working with any of the following:\n", + "- **Large datasets** that don't fit into memory or would benefit from parallelization\n", + "- **Multimodal data types** such as images, JSON, vector embeddings, and tensors\n", + "- **Formats that support data skipping** through automatic partition pruning and stats-based file pruning for filter predicates\n", + "- **ML workloads** that would benefit from interactive computation within DataFrame (via UDFs)\n", + "\n", + "Read more about how Daft compares to other DataFrames in our [FAQ](/faq/dataframe_comparison.rst).\n", + "\n", + "Let's jump in! \ud83e\ude82" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Install and Import Daft\n", + "\n", + "You can install Daft using `pip`:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "scrolled": true + }, + "outputs": [], + "source": [ + "%pip install getdaft" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "And then import Daft and some of its classes which we'll need later on:" + ] + }, + { + "cell_type": "code", + "execution_count": 1, + "metadata": { + "tags": [] + }, + "outputs": [], + "source": [ + "import daft\n", + "from daft import DataType, udf" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Create your first Daft DataFrame\n", + "\n", + "See also: [API Reference: DataFrame Construction](df-input-output)\n", + "\n", + "To begin, let's create a DataFrame from a dictionary of columns:" + ] + }, + { + "cell_type": "code", + "execution_count": 2, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
integers
Int64
floats
Float64
bools
Boolean
strings
Utf8
bytes
Binary
dates
Date
lists
List[Int64]
nulls
Null
1
1.5
true
a
b\"a\"
1994-01-01
[1, 1, 1]
None
2
2.5
true
b
b\"b\"
1994-01-02
[2, 2, 2]
None
3
3.5
false
c
b\"c\"
1994-01-03
[3, 3, 3]
None
4
4.5
false
d
b\"d\"
1994-01-04
[4, 4, 4]
None
\n", + "(Showing first 4 of 4 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 integers \u2506 floats \u2506 bools \u2506 \u2026 \u2506 dates \u2506 lists \u2506 nulls \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Int64 \u2506 Float64 \u2506 Boolean \u2506 (2 hidden) \u2506 Date \u2506 List[Int64] \u2506 Null \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 1 \u2506 1.5 \u2506 true \u2506 \u2026 \u2506 1994-01-01 \u2506 [1, 1, 1] \u2506 None \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 2 \u2506 2.5 \u2506 true \u2506 \u2026 \u2506 1994-01-02 \u2506 [2, 2, 2] \u2506 None \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 3 \u2506 3.5 \u2506 false \u2506 \u2026 \u2506 1994-01-03 \u2506 [3, 3, 3] \u2506 None \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 4 \u2506 4.5 \u2506 false \u2506 \u2026 \u2506 1994-01-04 \u2506 [4, 4, 4] \u2506 None \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 4 of 4 rows)" + ] + }, + "execution_count": 2, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "import datetime\n", + "\n", + "df = daft.from_pydict(\n", + " {\n", + " \"integers\": [1, 2, 3, 4],\n", + " \"floats\": [1.5, 2.5, 3.5, 4.5],\n", + " \"bools\": [True, True, False, False],\n", + " \"strings\": [\"a\", \"b\", \"c\", \"d\"],\n", + " \"bytes\": [b\"a\", b\"b\", b\"c\", b\"d\"],\n", + " \"dates\": [\n", + " datetime.date(1994, 1, 1),\n", + " datetime.date(1994, 1, 2),\n", + " datetime.date(1994, 1, 3),\n", + " datetime.date(1994, 1, 4),\n", + " ],\n", + " \"lists\": [[1, 1, 1], [2, 2, 2], [3, 3, 3], [4, 4, 4]],\n", + " \"nulls\": [None, None, None, None],\n", + " }\n", + ")\n", + "\n", + "df" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Nice. If you've worked with DataFrame libraries like pandas, Dask or Spark this should look familiar." + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "### Multimodal Data Types" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Daft is built for multimodal data type support. Daft DataFrames can contain more data types than other DataFrame APIs like pandas, Spark or Dask. Daft columns can contain URLs, images, tensors and Python classes. You'll get to work with some of these data types in a moment.\n", + "\n", + "For a complete list of supported data types see: [API Reference: DataTypes](../api_docs/datatype)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "### Data Sources\n", + "\n", + "You can also load DataFrames from other sources, such as:\n", + "\n", + "1. CSV files: {func}`daft.read_csv(\"s3://bucket/*.csv\") `\n", + "2. Parquet files: {func}`daft.read_parquet(\"/path/*.parquet\") `\n", + "3. JSON line-delimited files: {func}`daft.read_json(\"/path/*.parquet\") `\n", + "4. Files on disk: {func}`daft.from_glob_path(\"/path/*.jpeg\") `\n", + "\n", + "Daft automatically supports local paths as well as paths to object storage such as AWS S3:\n", + "\n", + "```\n", + "df = daft.read_json(\"s3://path/to/bucket/file.jsonl\")\n", + "```\n", + "\n", + "See [User Guide: Integrations](/user_guide/integrations) to learn more about working with other formats like Delta Lake and Iceberg." + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Let's read in a Parquet file from a public S3 bucket. Note that this Parquet file is partitioned on the `country` column. This will be important later on." + ] + }, + { + "cell_type": "code", + "execution_count": 3, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "
first_name
Utf8
last_name
Utf8
age
Int64
DoB
Date
country
Utf8
has_dog
Boolean
\n", + "(No data to display: Dataframe not materialized)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 first_name \u2506 last_name \u2506 age \u2506 DoB \u2506 country \u2506 has_dog \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Utf8 \u2506 Int64 \u2506 Date \u2506 Utf8 \u2506 Boolean \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(No data to display: Dataframe not materialized)" + ] + }, + "execution_count": 3, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "# Set IO Configurations to use anonymous data access mode\n", + "daft.set_planning_config(default_io_config=daft.io.IOConfig(s3=daft.io.S3Config(anonymous=True)))\n", + "\n", + "df = daft.read_parquet(\"s3://daft-public-data/tutorials/10-min/sample-data-dog-owners-partitioned.pq/**\")\n", + "df" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Executing and Displaying Data\n", + "\n", + "Daft DataFrames are lazy by default. This means that the contents will not be computed (\"materialized\") unless you explicitly tell Daft to do so. This is best practice for working with larger-than-memory datasets and parallel/distributed architectures.\n", + "\n", + "The file we have just loaded only has 5 rows. You can materialize the whole DataFrame in memory easily using the {meth}`df.collect() ` method:" + ] + }, + { + "cell_type": "code", + "execution_count": 4, + "metadata": {}, + "outputs": [ + { + "data": { + "application/vnd.jupyter.widget-view+json": { + "model_id": "38e35f25fe264d678fb1d733fa634dc9", + "version_major": 2, + "version_minor": 0 + }, + "text/plain": [ + "ScanWithTask [Stage:1]: 0%| | 0/1 [00:00\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
first_name
Utf8
last_name
Utf8
age
Int64
DoB
Date
country
Utf8
has_dog
Boolean
Shandra
Shamas
57
1967-01-02
United Kingdom
true
Zaya
Zaphora
40
1984-04-07
United Kingdom
true
Wolfgang
Winter
23
2001-02-12
Germany
None
Ernesto
Evergreen
34
1990-04-03
Canada
true
James
Jale
62
1962-03-24
Canada
true
\n", + "(Showing first 5 of 5 rows)\n", + "" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 first_name \u2506 last_name \u2506 age \u2506 DoB \u2506 country \u2506 has_dog \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Utf8 \u2506 Int64 \u2506 Date \u2506 Utf8 \u2506 Boolean \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Shandra \u2506 Shamas \u2506 57 \u2506 1967-01-02 \u2506 United Kingdom \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya \u2506 Zaphora \u2506 40 \u2506 1984-04-07 \u2506 United Kingdom \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Wolfgang \u2506 Winter \u2506 23 \u2506 2001-02-12 \u2506 Germany \u2506 None \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Ernesto \u2506 Evergreen \u2506 34 \u2506 1990-04-03 \u2506 Canada \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 James \u2506 Jale \u2506 62 \u2506 1962-03-24 \u2506 Canada \u2506 true \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 5 of 5 rows)" + ] + }, + "execution_count": 4, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "df.collect()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "You can also take a look at just the first few rows with the {meth}`df.show() ` method:" + ] + }, + { + "cell_type": "code", + "execution_count": 5, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
first_name
Utf8
last_name
Utf8
age
Int64
DoB
Date
country
Utf8
has_dog
Boolean
Shandra
Shamas
57
1967-01-02
United Kingdom
true
Zaya
Zaphora
40
1984-04-07
United Kingdom
true
Wolfgang
Winter
23
2001-02-12
Germany
None
\n", + "(Showing first 3 of 5 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 first_name \u2506 last_name \u2506 age \u2506 DoB \u2506 country \u2506 has_dog \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Utf8 \u2506 Int64 \u2506 Date \u2506 Utf8 \u2506 Boolean \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Shandra \u2506 Shamas \u2506 57 \u2506 1967-01-02 \u2506 United Kingdom \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya \u2506 Zaphora \u2506 40 \u2506 1984-04-07 \u2506 United Kingdom \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Wolfgang \u2506 Winter \u2506 23 \u2506 2001-02-12 \u2506 Germany \u2506 None \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 3 of 5 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df.show(3)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Use `.show` for quick visualisation in an interactive notebook." + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Basic DataFrame Operations\n", + "\n", + "Let's take a look at some of the most common DataFrame operations." + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Selecting Columns" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "You can **select** specific columns from your DataFrame with the {meth}`df.select() ` method:" + ] + }, + { + "cell_type": "code", + "execution_count": 6, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
first_name
Utf8
has_dog
Boolean
Shandra
true
Zaya
true
Wolfgang
None
Ernesto
true
James
true
\n", + "(Showing first 5 of 5 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 first_name \u2506 has_dog \u2502\n", + "\u2502 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Boolean \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Shandra \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Wolfgang \u2506 None \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Ernesto \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 James \u2506 true \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 5 of 5 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df.select(\"first_name\", \"has_dog\").show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "### Excluding Data\n", + "\n", + "You can **limit** the number of rows in a dataframe by calling {meth}`df.limit() `. Use `limit` and not `show` when you want to return a limited number of rows for further transformation." + ] + }, + { + "cell_type": "code", + "execution_count": 7, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
first_name
Utf8
last_name
Utf8
age
Int64
DoB
Date
country
Utf8
has_dog
Boolean
Shandra
Shamas
57
1967-01-02
United Kingdom
true
\n", + "(Showing first 1 of 1 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 first_name \u2506 last_name \u2506 age \u2506 DoB \u2506 country \u2506 has_dog \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Utf8 \u2506 Int64 \u2506 Date \u2506 Utf8 \u2506 Boolean \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Shandra \u2506 Shamas \u2506 57 \u2506 1967-01-02 \u2506 United Kingdom \u2506 true \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 1 of 1 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df.limit(1).show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "To **drop** columns from the dataframe, call {meth}`df.exclude() `:" + ] + }, + { + "cell_type": "code", + "execution_count": 8, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
first_name
Utf8
last_name
Utf8
age
Int64
country
Utf8
has_dog
Boolean
Shandra
Shamas
57
United Kingdom
true
Zaya
Zaphora
40
United Kingdom
true
Wolfgang
Winter
23
Germany
None
Ernesto
Evergreen
34
Canada
true
James
Jale
62
Canada
true
\n", + "(Showing first 5 of 5 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 first_name \u2506 last_name \u2506 age \u2506 country \u2506 has_dog \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Utf8 \u2506 Int64 \u2506 Utf8 \u2506 Boolean \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Shandra \u2506 Shamas \u2506 57 \u2506 United Kingdom \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya \u2506 Zaphora \u2506 40 \u2506 United Kingdom \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Wolfgang \u2506 Winter \u2506 23 \u2506 Germany \u2506 None \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Ernesto \u2506 Evergreen \u2506 34 \u2506 Canada \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 James \u2506 Jale \u2506 62 \u2506 Canada \u2506 true \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 5 of 5 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df.exclude(\"DoB\").show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "### Transforming Columns with Expressions" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "See: [Expressions](user_guide/expressions.rst)\n", + "\n", + "Expressions are an API for defining computation that needs to happen over your columns.\n", + "\n", + "For example, use the {meth}`daft.col() ` expression together with the `with_column` method to create a new column `full_name`, joining the contents of the `last_name` column to the `first_name` column:" + ] + }, + { + "cell_type": "code", + "execution_count": 9, + "metadata": { + "scrolled": true + }, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
full_name
Utf8
age
Int64
country
Utf8
has_dog
Boolean
Shandra Shamas
57
United Kingdom
true
Zaya Zaphora
40
United Kingdom
true
Wolfgang Winter
23
Germany
None
Ernesto Evergreen
34
Canada
true
James Jale
62
Canada
true
\n", + "(Showing first 5 of 5 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 full_name \u2506 age \u2506 country \u2506 has_dog \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Int64 \u2506 Utf8 \u2506 Boolean \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Shandra Shamas \u2506 57 \u2506 United Kingdom \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya Zaphora \u2506 40 \u2506 United Kingdom \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Wolfgang Winter \u2506 23 \u2506 Germany \u2506 None \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Ernesto Evergreen \u2506 34 \u2506 Canada \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 James Jale \u2506 62 \u2506 Canada \u2506 true \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 5 of 5 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df = df.with_column(\"full_name\", daft.col(\"first_name\") + \" \" + daft.col(\"last_name\"))\n", + "df.select(\"full_name\", \"age\", \"country\", \"has_dog\").show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Alternatively, you can also run your column transforms using Expressions directly inside your `select` call:" + ] + }, + { + "cell_type": "code", + "execution_count": 17, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
first_name
Utf8
age
Int64
country
Utf8
Shandra Shamas
57
United Kingdom
Zaya Zaphora
40
United Kingdom
Wolfgang Winter
23
Germany
Ernesto Evergreen
34
Canada
James Jale
62
Canada
\n", + "(Showing first 5 of 5 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 first_name \u2506 age \u2506 country \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Int64 \u2506 Utf8 \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Shandra Shamas \u2506 57 \u2506 United Kingdom \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya Zaphora \u2506 40 \u2506 United Kingdom \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Wolfgang Winter \u2506 23 \u2506 Germany \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Ernesto Evergreen \u2506 34 \u2506 Canada \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 James Jale \u2506 62 \u2506 Canada \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 5 of 5 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df.select((daft.col(\"first_name\") + \" \" + daft.col(\"last_name\")), \"age\", \"country\").show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Some Expression methods are only allowed on certain types and are accessible through \"method accessors\" (see: [Expression Accessor Properties](expression-accessor-properties)).\n", + "\n", + "For example, the `.dt.year()` expression is only valid when run on a `datetime` column.\n", + "\n", + "Below we use an Expression to extract the year from a `datetime` column:" + ] + }, + { + "cell_type": "code", + "execution_count": 10, + "metadata": { + "scrolled": true + }, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
first_name
Utf8
last_name
Utf8
age
Int64
DoB
Date
country
Utf8
has_dog
Boolean
full_name
Utf8
DoB_year
Int32
Shandra
Shamas
57
1967-01-02
United Kingdom
true
Shandra Shamas
1967
Zaya
Zaphora
40
1984-04-07
United Kingdom
true
Zaya Zaphora
1984
Wolfgang
Winter
23
2001-02-12
Germany
None
Wolfgang Winter
2001
Ernesto
Evergreen
34
1990-04-03
Canada
true
Ernesto Evergreen
1990
James
Jale
62
1962-03-24
Canada
true
James Jale
1962
\n", + "(Showing first 5 of 5 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 first_name \u2506 last_name \u2506 age \u2506 \u2026 \u2506 has_dog \u2506 full_name \u2506 DoB_year \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Utf8 \u2506 Int64 \u2506 (2 hidden) \u2506 Boolean \u2506 Utf8 \u2506 Int32 \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Shandra \u2506 Shamas \u2506 57 \u2506 \u2026 \u2506 true \u2506 Shandra Shamas \u2506 1967 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya \u2506 Zaphora \u2506 40 \u2506 \u2026 \u2506 true \u2506 Zaya Zaphora \u2506 1984 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Wolfgang \u2506 Winter \u2506 23 \u2506 \u2026 \u2506 None \u2506 Wolfgang Winter \u2506 2001 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Ernesto \u2506 Evergreen \u2506 34 \u2506 \u2026 \u2506 true \u2506 Ernesto Evergreen \u2506 1990 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 James \u2506 Jale \u2506 62 \u2506 \u2026 \u2506 true \u2506 James Jale \u2506 1962 \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 5 of 5 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df_year = df.with_column(\"DoB_year\", df[\"DoB\"].dt.year())\n", + "df_year.show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Other DataFrame Operations" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "### Sorting Data\n", + "\n", + "You can **sort** a dataframe with {meth}`df.sort() `, which we do so here in ascending order:" + ] + }, + { + "cell_type": "code", + "execution_count": 11, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
first_name
Utf8
last_name
Utf8
age
Int64
DoB
Date
country
Utf8
has_dog
Boolean
full_name
Utf8
Wolfgang
Winter
23
2001-02-12
Germany
None
Wolfgang Winter
Ernesto
Evergreen
34
1990-04-03
Canada
true
Ernesto Evergreen
Zaya
Zaphora
40
1984-04-07
United Kingdom
true
Zaya Zaphora
Shandra
Shamas
57
1967-01-02
United Kingdom
true
Shandra Shamas
James
Jale
62
1962-03-24
Canada
true
James Jale
\n", + "(Showing first 5 of 5 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 first_name \u2506 last_name \u2506 age \u2506 \u2026 \u2506 country \u2506 has_dog \u2506 full_name \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Utf8 \u2506 Int64 \u2506 (1 hidden) \u2506 Utf8 \u2506 Boolean \u2506 Utf8 \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Wolfgang \u2506 Winter \u2506 23 \u2506 \u2026 \u2506 Germany \u2506 None \u2506 Wolfgang Winter \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Ernesto \u2506 Evergreen \u2506 34 \u2506 \u2026 \u2506 Canada \u2506 true \u2506 Ernesto Evergreen \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya \u2506 Zaphora \u2506 40 \u2506 \u2026 \u2506 United Kingdom \u2506 true \u2506 Zaya Zaphora \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Shandra \u2506 Shamas \u2506 57 \u2506 \u2026 \u2506 United Kingdom \u2506 true \u2506 Shandra Shamas \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 James \u2506 Jale \u2506 62 \u2506 \u2026 \u2506 Canada \u2506 true \u2506 James Jale \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 5 of 5 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df.sort(df[\"age\"], desc=False).show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "### Grouping and Aggregating Data\n", + "\n", + "You can **group** and **aggregate** your data using the {meth}`df.groupby() ` method:\n", + "\n", + "Groupby aggregation operations over a dataset happens in 2 phases:\n", + "\n", + "1. Splitting the data into groups based on some criteria using {meth}`df.groupby() `\n", + "2. Specifying how to aggregate the data for each group using {meth}`GroupedDataFrame.agg() `\n", + "\n", + "For example:" + ] + }, + { + "cell_type": "code", + "execution_count": 12, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
country
Utf8
counts
UInt64
Canada
2
Germany
1
United Kingdom
2
\n", + "(Showing first 3 of 3 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 country \u2506 counts \u2502\n", + "\u2502 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 UInt64 \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Canada \u2506 2 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Germany \u2506 1 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 United Kingdom \u2506 2 \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 3 of 3 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "# select only columns for grouping\n", + "grouping_df = df.select(df[\"country\"], df[\"first_name\"].alias(\"counts\"))\n", + "\n", + "# groupby country column and count the number of countries\n", + "grouping_df.groupby(df[\"country\"]).count().show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Note that we can use {meth}`.alias() ` to quickly rename columns." + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "### Missing Data\n", + "\n", + "All columns in Daft are \"nullable\" by default. Unlike other frameworks such as Pandas, Daft differentiates between \"null\" (missing) and \"nan\" (stands for not a number - a special value indicating an invalid float)." + ] + }, + { + "cell_type": "code", + "execution_count": 13, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
floats
Float64
floats_is_null
Boolean
floats_is_nan
Boolean
1.5
false
false
None
true
None
NaN
false
true
\n", + "(Showing first 3 of 3 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 floats \u2506 floats_is_null \u2506 floats_is_nan \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Float64 \u2506 Boolean \u2506 Boolean \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 1.5 \u2506 false \u2506 false \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 None \u2506 true \u2506 None \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 NaN \u2506 false \u2506 true \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 3 of 3 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "missing_data_df = daft.from_pydict(\n", + " {\n", + " \"floats\": [1.5, None, float(\"nan\")],\n", + " }\n", + ")\n", + "missing_data_df = missing_data_df.with_column(\"floats_is_null\", missing_data_df[\"floats\"].is_null()).with_column(\n", + " \"floats_is_nan\", missing_data_df[\"floats\"].float.is_nan()\n", + ")\n", + "\n", + "missing_data_df.show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Let's correct the one missing value in our dataset:" + ] + }, + { + "cell_type": "code", + "execution_count": 14, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
first_name
Utf8
last_name
Utf8
age
Int64
DoB
Date
country
Utf8
has_dog
Boolean
full_name
Utf8
Shandra
Shamas
57
1967-01-02
United Kingdom
true
Shandra Shamas
Zaya
Zaphora
40
1984-04-07
United Kingdom
true
Zaya Zaphora
Wolfgang
Winter
23
2001-02-12
Germany
true
Wolfgang Winter
Ernesto
Evergreen
34
1990-04-03
Canada
true
Ernesto Evergreen
James
Jale
62
1962-03-24
Canada
true
James Jale
\n", + "(Showing first 5 of 5 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 first_name \u2506 last_name \u2506 age \u2506 \u2026 \u2506 country \u2506 has_dog \u2506 full_name \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Utf8 \u2506 Int64 \u2506 (1 hidden) \u2506 Utf8 \u2506 Boolean \u2506 Utf8 \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Shandra \u2506 Shamas \u2506 57 \u2506 \u2026 \u2506 United Kingdom \u2506 true \u2506 Shandra Shamas \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya \u2506 Zaphora \u2506 40 \u2506 \u2026 \u2506 United Kingdom \u2506 true \u2506 Zaya Zaphora \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Wolfgang \u2506 Winter \u2506 23 \u2506 \u2026 \u2506 Germany \u2506 true \u2506 Wolfgang Winter \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Ernesto \u2506 Evergreen \u2506 34 \u2506 \u2026 \u2506 Canada \u2506 true \u2506 Ernesto Evergreen \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 James \u2506 Jale \u2506 62 \u2506 \u2026 \u2506 Canada \u2506 true \u2506 James Jale \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 5 of 5 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df = df.with_column(\"has_dog\", df[\"has_dog\"].is_null().if_else(True, df[\"has_dog\"]))\n", + "df.show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "### Filtering Data\n", + "\n", + "You can **filter** rows in your DataFrame with a predicate using the {meth}`df.where() ` method:" + ] + }, + { + "cell_type": "code", + "execution_count": 16, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
first_name
Utf8
last_name
Utf8
age
Int64
DoB
Date
country
Utf8
has_dog
Boolean
full_name
Utf8
James
Jale
62
1962-03-24
Canada
true
James Jale
Shandra
Shamas
57
1967-01-02
United Kingdom
true
Shandra Shamas
Zaya
Zaphora
40
1984-04-07
United Kingdom
true
Zaya Zaphora
\n", + "(Showing first 3 of 3 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 first_name \u2506 last_name \u2506 age \u2506 \u2026 \u2506 country \u2506 has_dog \u2506 full_name \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Utf8 \u2506 Int64 \u2506 (1 hidden) \u2506 Utf8 \u2506 Boolean \u2506 Utf8 \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 James \u2506 Jale \u2506 62 \u2506 \u2026 \u2506 Canada \u2506 true \u2506 James Jale \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Shandra \u2506 Shamas \u2506 57 \u2506 \u2026 \u2506 United Kingdom \u2506 true \u2506 Shandra Shamas \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya \u2506 Zaphora \u2506 40 \u2506 \u2026 \u2506 United Kingdom \u2506 true \u2506 Zaya Zaphora \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 3 of 3 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df.where(df[\"age\"] > 35).show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Filtering can give you powerful optimization when you are working with partitioned files or tables. Daft will use the predicate to read only the necessary partitions, skipping any data that is not relevant.\n", + "\n", + "For example, our Parquet file is partitioned on the `country` column. This means that queries with a `country` predicate will benefit from query optimization:" + ] + }, + { + "cell_type": "code", + "execution_count": 17, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
first_name
Utf8
last_name
Utf8
age
Int64
DoB
Date
country
Utf8
has_dog
Boolean
full_name
Utf8
Ernesto
Evergreen
34
1990-04-03
Canada
true
Ernesto Evergreen
James
Jale
62
1962-03-24
Canada
true
James Jale
\n", + "(Showing first 2 of 2 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 first_name \u2506 last_name \u2506 age \u2506 \u2026 \u2506 country \u2506 has_dog \u2506 full_name \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Utf8 \u2506 Int64 \u2506 (1 hidden) \u2506 Utf8 \u2506 Boolean \u2506 Utf8 \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Ernesto \u2506 Evergreen \u2506 34 \u2506 \u2026 \u2506 Canada \u2506 true \u2506 Ernesto Evergreen \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 James \u2506 Jale \u2506 62 \u2506 \u2026 \u2506 Canada \u2506 true \u2506 James Jale \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 2 of 2 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df.where(df[\"country\"] == \"Canada\").show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Daft only needs to read in 1 file for this query, instead of 3." + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Query Planning\n", + "\n", + "Daft is lazy: computations on your DataFrame are not executed immediately. Instead, Daft creates a `LogicalPlan` which defines the operations that need to happen to materialize the requested result. Think of this LogicalPlan as a recipe. \n", + "\n", + "You can examine this logical plan using {meth}`df.explain() `:" + ] + }, + { + "cell_type": "code", + "execution_count": 18, + "metadata": {}, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "== Unoptimized Logical Plan ==\n", + "\n", + "* Filter: col(country) == lit(\"Canada\")\n", + "|\n", + "* GlobScanOperator\n", + "| Glob paths = [s3://daft-public-data/tutorials/10-min/sample-data-dog-owners-\n", + "| partitioned.pq/**]\n", + "| Coerce int96 timestamp unit = Nanoseconds\n", + "| IO config = S3 config = { Max connections = 8, Retry initial backoff ms = 1000,\n", + "| Connect timeout ms = 30000, Read timeout ms = 30000, Max retries = 25, Retry\n", + "| mode = adaptive, Anonymous = false, Use SSL = true, Verify SSL = true, Check\n", + "| hostname SSL = true, Requester pays = false, Force Virtual Addressing = false },\n", + "| Azure config = { Anonymous = false, Use SSL = true }, GCS config = { Anonymous =\n", + "| false }, HTTP config = { user_agent = daft/0.0.1 }\n", + "| Use multithreading = true\n", + "| File schema = first_name#Utf8, last_name#Utf8, age#Int64, DoB#Date,\n", + "| country#Utf8, has_dog#Boolean\n", + "| Partitioning keys = []\n", + "| Output schema = first_name#Utf8, last_name#Utf8, age#Int64, DoB#Date,\n", + "| country#Utf8, has_dog#Boolean\n", + "\n", + "\n", + "== Optimized Logical Plan ==\n", + "\n", + "* GlobScanOperator\n", + "| Glob paths = [s3://daft-public-data/tutorials/10-min/sample-data-dog-owners-\n", + "| partitioned.pq/**]\n", + "| Coerce int96 timestamp unit = Nanoseconds\n", + "| IO config = S3 config = { Max connections = 8, Retry initial backoff ms = 1000,\n", + "| Connect timeout ms = 30000, Read timeout ms = 30000, Max retries = 25, Retry\n", + "| mode = adaptive, Anonymous = false, Use SSL = true, Verify SSL = true, Check\n", + "| hostname SSL = true, Requester pays = false, Force Virtual Addressing = false },\n", + "| Azure config = { Anonymous = false, Use SSL = true }, GCS config = { Anonymous =\n", + "| false }, HTTP config = { user_agent = daft/0.0.1 }\n", + "| Use multithreading = true\n", + "| File schema = first_name#Utf8, last_name#Utf8, age#Int64, DoB#Date,\n", + "| country#Utf8, has_dog#Boolean\n", + "| Partitioning keys = []\n", + "| Filter pushdown = col(country) == lit(\"Canada\")\n", + "| Output schema = first_name#Utf8, last_name#Utf8, age#Int64, DoB#Date,\n", + "| country#Utf8, has_dog#Boolean\n", + "\n", + "\n", + "== Physical Plan ==\n", + "\n", + "* TabularScan:\n", + "| Num Scan Tasks = 1\n", + "| Estimated Scan Bytes = 6336\n", + "| Clustering spec = { Num partitions = 1 }\n", + "\n" + ] + } + ], + "source": [ + "df2 = daft.read_parquet(\"s3://daft-public-data/tutorials/10-min/sample-data-dog-owners-partitioned.pq/**\")\n", + "df2.where(df[\"country\"] == \"Canada\").explain(show_all=True)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Because we are filtering our DataFrame on the partition column `country`, Daft can optimize the Logical Plan and save us time and computing resources by only reading a single partition from disk. " + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## More Advanced Operations\n", + "\n", + "You've made it half-way! Time to bring in some fluffy beings \ud83d\udc36\n", + "\n", + "Let's bring all of the elements together to see how you can use Daft to:\n", + "- perform more advanced operations like **joins**\n", + "- work with **multimodal data** like Python classes, URLs, and Images,\n", + "- apply **custom User-Defined Functions** to your columns,\n", + "- and **run ML workloads** within your DataFrame," + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "### Merging DataFrames\n", + "\n", + "DataFrames can be joined with {meth}`df.join() `.\n", + "\n", + "Let's use a join to reunite our `owners` with their sweet fluffy companions. We'll create a `dogs` DataFrame from a Python dictionary and then join this to our existing dataframe with the owners data." + ] + }, + { + "cell_type": "code", + "execution_count": 19, + "metadata": {}, + "outputs": [], + "source": [ + "df_dogs = daft.from_pydict(\n", + " {\n", + " \"urls\": [\n", + " \"https://live.staticflickr.com/65535/53671838774_03ba68d203_o.jpg\",\n", + " \"https://live.staticflickr.com/65535/53671700073_2c9441422e_o.jpg\",\n", + " \"https://live.staticflickr.com/65535/53670606332_1ea5f2ce68_o.jpg\",\n", + " \"https://live.staticflickr.com/65535/53671838039_b97411a441_o.jpg\",\n", + " \"https://live.staticflickr.com/65535/53671698613_0230f8af3c_o.jpg\",\n", + " ],\n", + " \"full_name\": [\n", + " \"Ernesto Evergreen\",\n", + " \"James Jale\",\n", + " \"Wolfgang Winter\",\n", + " \"Shandra Shamas\",\n", + " \"Zaya Zaphora\",\n", + " ],\n", + " \"dog_name\": [\"Ernie\", \"Jackie\", \"Wolfie\", \"Shaggie\", \"Zadie\"],\n", + " }\n", + ")" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Let's join and drop some columns to keep the output easy to read:" + ] + }, + { + "cell_type": "code", + "execution_count": 20, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
has_dog
Boolean
full_name
Utf8
urls
Utf8
dog_name
Utf8
true
Ernesto Evergreen
https://live.staticflickr.com/65535/53671838774_03ba68d203_o.jpg
Ernie
true
James Jale
https://live.staticflickr.com/65535/53671700073_2c9441422e_o.jpg
Jackie
true
Wolfgang Winter
https://live.staticflickr.com/65535/53670606332_1ea5f2ce68_o.jpg
Wolfie
true
Shandra Shamas
https://live.staticflickr.com/65535/53671838039_b97411a441_o.jpg
Shaggie
true
Zaya Zaphora
https://live.staticflickr.com/65535/53671698613_0230f8af3c_o.jpg
Zadie
\n", + "(Showing first 5 of 5 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 has_dog \u2506 full_name \u2506 urls \u2506 dog_name \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Boolean \u2506 Utf8 \u2506 Utf8 \u2506 Utf8 \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 true \u2506 Ernesto Evergreen \u2506 https://live.staticflickr.com\u2026 \u2506 Ernie \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 true \u2506 James Jale \u2506 https://live.staticflickr.com\u2026 \u2506 Jackie \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 true \u2506 Wolfgang Winter \u2506 https://live.staticflickr.com\u2026 \u2506 Wolfie \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 true \u2506 Shandra Shamas \u2506 https://live.staticflickr.com\u2026 \u2506 Shaggie \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 true \u2506 Zaya Zaphora \u2506 https://live.staticflickr.com\u2026 \u2506 Zadie \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 5 of 5 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df_family = df.join(df_dogs, on=\"full_name\").exclude(\"first_name\", \"last_name\", \"DoB\", \"country\", \"age\")\n", + "df_family.show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Let's just quickly re-order the columns for easier reading:" + ] + }, + { + "cell_type": "code", + "execution_count": 21, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
full_name
Utf8
has_dog
Boolean
dog_name
Utf8
urls
Utf8
Ernesto Evergreen
true
Ernie
https://live.staticflickr.com/65535/53671838774_03ba68d203_o.jpg
James Jale
true
Jackie
https://live.staticflickr.com/65535/53671700073_2c9441422e_o.jpg
Wolfgang Winter
true
Wolfie
https://live.staticflickr.com/65535/53670606332_1ea5f2ce68_o.jpg
Shandra Shamas
true
Shaggie
https://live.staticflickr.com/65535/53671838039_b97411a441_o.jpg
Zaya Zaphora
true
Zadie
https://live.staticflickr.com/65535/53671698613_0230f8af3c_o.jpg
\n", + "(Showing first 5 of 5 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 full_name \u2506 has_dog \u2506 dog_name \u2506 urls \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Boolean \u2506 Utf8 \u2506 Utf8 \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Ernesto Evergreen \u2506 true \u2506 Ernie \u2506 https://live.staticflickr.com\u2026 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 James Jale \u2506 true \u2506 Jackie \u2506 https://live.staticflickr.com\u2026 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Wolfgang Winter \u2506 true \u2506 Wolfie \u2506 https://live.staticflickr.com\u2026 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Shandra Shamas \u2506 true \u2506 Shaggie \u2506 https://live.staticflickr.com\u2026 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya Zaphora \u2506 true \u2506 Zadie \u2506 https://live.staticflickr.com\u2026 \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 5 of 5 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df_family = df_family.select(\"full_name\", \"has_dog\", \"dog_name\", \"urls\")\n", + "df_family.show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "### Working with Multimodal Data" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Daft is built to work comfortably with multimodal data types, including URLs and Images.\n", + "\n", + "You can use the {meth}`url.download() ` expression to download the bytes from a URL. Let's store them in a new column using the `with_column` method:" + ] + }, + { + "cell_type": "code", + "execution_count": 22, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
full_name
Utf8
has_dog
Boolean
dog_name
Utf8
urls
Utf8
image_bytes
Binary
Ernesto Evergreen
true
Ernie
https://live.staticflickr.com/65535/53671838774_03ba68d203_o.jpg
b\"\\xff\\xd8\\xff\\xe0\\x00\\x10JFIF\\x00\\x01\"...
James Jale
true
Jackie
https://live.staticflickr.com/65535/53671700073_2c9441422e_o.jpg
b\"\\xff\\xd8\\xff\\xe0\\x00\\x10JFIF\\x00\\x01\"...
Wolfgang Winter
true
Wolfie
https://live.staticflickr.com/65535/53670606332_1ea5f2ce68_o.jpg
b\"\\xff\\xd8\\xff\\xe0\\x00\\x10JFIF\\x00\\x01\"...
Shandra Shamas
true
Shaggie
https://live.staticflickr.com/65535/53671838039_b97411a441_o.jpg
b\"\\xff\\xd8\\xff\\xe0\\x00\\x10JFIF\\x00\\x01\"...
Zaya Zaphora
true
Zadie
https://live.staticflickr.com/65535/53671698613_0230f8af3c_o.jpg
b\"\\xff\\xd8\\xff\\xe0\\x00\\x10JFIF\\x00\\x01\"...
\n", + "(Showing first 5 of 5 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 full_name \u2506 has_dog \u2506 dog_name \u2506 urls \u2506 image_bytes \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Boolean \u2506 Utf8 \u2506 Utf8 \u2506 Binary \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Ernesto Evergreen \u2506 true \u2506 Ernie \u2506 https://live.staticflickr.com\u2026 \u2506 b\"\\xff\\xd8\\xff\\xe0\\x00\\x10JFI\u2026 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 James Jale \u2506 true \u2506 Jackie \u2506 https://live.staticflickr.com\u2026 \u2506 b\"\\xff\\xd8\\xff\\xe0\\x00\\x10JFI\u2026 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Wolfgang Winter \u2506 true \u2506 Wolfie \u2506 https://live.staticflickr.com\u2026 \u2506 b\"\\xff\\xd8\\xff\\xe0\\x00\\x10JFI\u2026 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Shandra Shamas \u2506 true \u2506 Shaggie \u2506 https://live.staticflickr.com\u2026 \u2506 b\"\\xff\\xd8\\xff\\xe0\\x00\\x10JFI\u2026 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya Zaphora \u2506 true \u2506 Zadie \u2506 https://live.staticflickr.com\u2026 \u2506 b\"\\xff\\xd8\\xff\\xe0\\x00\\x10JFI\u2026 \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 5 of 5 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df_family = df_family.with_column(\"image_bytes\", df_dogs[\"urls\"].url.download(on_error=\"null\"))\n", + "df_family.show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Great! But where's the fluffiness? \ud83d\ude41\n", + "\n", + "Let's turn the bytes into human-readable images using `image.decode`:" + ] + }, + { + "cell_type": "code", + "execution_count": 23, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
full_name
Utf8
has_dog
Boolean
dog_name
Utf8
urls
Utf8
image_bytes
Binary
image
Image[MIXED]
Ernesto Evergreen
true
Ernie
https://live.staticflickr.com/65535/53671838774_03ba68d203_o.jpg
b\"\\xff\\xd8\\xff\\xe0\\x00\\x10JFIF\\x00\\x01\"...
\"<Image\" />
James Jale
true
Jackie
https://live.staticflickr.com/65535/53671700073_2c9441422e_o.jpg
b\"\\xff\\xd8\\xff\\xe0\\x00\\x10JFIF\\x00\\x01\"...
\"<Image\" />
Wolfgang Winter
true
Wolfie
https://live.staticflickr.com/65535/53670606332_1ea5f2ce68_o.jpg
b\"\\xff\\xd8\\xff\\xe0\\x00\\x10JFIF\\x00\\x01\"...
\"<Image\" />
Shandra Shamas
true
Shaggie
https://live.staticflickr.com/65535/53671838039_b97411a441_o.jpg
b\"\\xff\\xd8\\xff\\xe0\\x00\\x10JFIF\\x00\\x01\"...
\"<Image\" />
Zaya Zaphora
true
Zadie
https://live.staticflickr.com/65535/53671698613_0230f8af3c_o.jpg
b\"\\xff\\xd8\\xff\\xe0\\x00\\x10JFIF\\x00\\x01\"...
\"<Image\" />
\n", + "(Showing first 5 of 5 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 full_name \u2506 has_dog \u2506 dog_name \u2506 urls \u2506 image_bytes \u2506 image \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Boolean \u2506 Utf8 \u2506 Utf8 \u2506 Binary \u2506 Image[MIXED] \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Ernesto Evergreen \u2506 true \u2506 Ernie \u2506 https://live.staticflickr.com \u2506 b\"\\xff\\xd8\\xff\\xe0\\x00\\x10JF \u2506 \u2502\n", + "\u2502 \u2506 \u2506 \u2506 \u2026 \u2506 I\u2026 \u2506 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 James Jale \u2506 true \u2506 Jackie \u2506 https://live.staticflickr.com \u2506 b\"\\xff\\xd8\\xff\\xe0\\x00\\x10JF \u2506 \u2502\n", + "\u2502 \u2506 \u2506 \u2506 \u2026 \u2506 I\u2026 \u2506 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Wolfgang Winter \u2506 true \u2506 Wolfie \u2506 https://live.staticflickr.com \u2506 b\"\\xff\\xd8\\xff\\xe0\\x00\\x10JF \u2506 \u2502\n", + "\u2502 \u2506 \u2506 \u2506 \u2026 \u2506 I\u2026 \u2506 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Shandra Shamas \u2506 true \u2506 Shaggie \u2506 https://live.staticflickr.com \u2506 b\"\\xff\\xd8\\xff\\xe0\\x00\\x10JF \u2506 \u2502\n", + "\u2502 \u2506 \u2506 \u2506 \u2026 \u2506 I\u2026 \u2506 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya Zaphora \u2506 true \u2506 Zadie \u2506 https://live.staticflickr.com \u2506 b\"\\xff\\xd8\\xff\\xe0\\x00\\x10JF \u2506 \u2502\n", + "\u2502 \u2506 \u2506 \u2506 \u2026 \u2506 I\u2026 \u2506 \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 5 of 5 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df_family = df_family.with_column(\"image\", daft.col(\"image_bytes\").image.decode())\n", + "df_family.show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Woof! \ud83d\udc36" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "### User-Defined Functions\n", + "\n", + "See: [UDF User Guide](user_guide/udf)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "You can use User-Defined Functions (UDFs) to run computations over multiple rows or columns." + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "As the final part of this Quickstart, you'll build a Machine Learning model to classify our new fluffy friends by breed. \n", + "\n", + "Daft enables you to do all this right within our DataFrame, using UDFs. \n", + "\n", + "### ML Workloads\n", + "\n", + "We'll define a function that uses a pre-trained PyTorch model: [ResNet50](https://pytorch.org/vision/main/models/generated/torchvision.models.resnet50.html) to classify the dog pictures. We'll pass the contents of the image `urls` column and send the classification predictions to a new column `classify_breed`." + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Working with PyTorch adds some complexity but you can just run the cells below to perform the classification.\n", + "\n", + "First, make sure to install and import some extra dependencies:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "scrolled": true + }, + "outputs": [], + "source": [ + "%pip install validators matplotlib Pillow torch torchvision" + ] + }, + { + "cell_type": "code", + "execution_count": 24, + "metadata": {}, + "outputs": [], + "source": [ + "# import additional libraries, these are necessary for PyTorch\n", + "\n", + "import torch" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Then, go ahead and define your `ClassifyImages` UDF. \n", + "\n", + "Models are expensive to initialize and load, so we want to do this as few times as possible, and share a model across multiple invocations." + ] + }, + { + "cell_type": "code", + "execution_count": 25, + "metadata": {}, + "outputs": [], + "source": [ + "@udf(return_dtype=DataType.fixed_size_list(dtype=DataType.string(), size=2))\n", + "class ClassifyImages:\n", + " def __init__(self):\n", + " # Perform expensive initializations - create and load the pre-trained model\n", + " self.model = torch.hub.load(\"NVIDIA/DeepLearningExamples:torchhub\", \"nvidia_resnet50\", pretrained=True)\n", + " self.utils = torch.hub.load(\"NVIDIA/DeepLearningExamples:torchhub\", \"nvidia_convnets_processing_utils\")\n", + " self.model.eval().to(torch.device(\"cpu\"))\n", + "\n", + " def __call__(self, images_urls):\n", + " uris = images_urls.to_pylist()\n", + " batch = torch.cat([self.utils.prepare_input_from_uri(uri) for uri in uris]).to(torch.device(\"cpu\"))\n", + "\n", + " with torch.no_grad():\n", + " output = torch.nn.functional.softmax(self.model(batch), dim=1)\n", + "\n", + " results = self.utils.pick_n_best(predictions=output, n=1)\n", + " return [result[0] for result in results]" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Nice, now you're all set to call this function on the `urls` column and store the outputs in a new column we'll call `classify breeds`:" + ] + }, + { + "cell_type": "code", + "execution_count": 26, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
dog_name
Utf8
image
Image[MIXED]
classify_breed
FixedSizeList[Utf8; 2]
Ernie
\"<Image\" />
[boxer, 52.3%]
Jackie
\"<Image\" />
[American Staffordshire terrier, Staffordshire terrier, American pit bull terrier, pit bull terrier, 42.4%]
Wolfie
\"<Image\" />
[collie, 49.6%]
Shaggie
\"<Image\" />
[standard schnauzer, 29.6%]
Zadie
\"<Image\" />
[Rottweiler, 78.6%]
\n", + "(Showing first 5 of 5 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 dog_name \u2506 image \u2506 classify_breed \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Image[MIXED] \u2506 FixedSizeList[Utf8; 2] \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Ernie \u2506 \u2506 [boxer, 52.3%] \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Jackie \u2506 \u2506 [American Staffordshire terri\u2026 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Wolfie \u2506 \u2506 [collie, 49.6%] \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Shaggie \u2506 \u2506 [standard schnauzer, 29.6%] \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zadie \u2506 \u2506 [Rottweiler, 78.6%] \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 5 of 5 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "classified_images_df = df_family.with_column(\"classify_breed\", ClassifyImages(daft.col(\"urls\")))\n", + "\n", + "classified_images_df.select(\"dog_name\", \"image\", \"classify_breed\").show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Nice work!\n", + "\n", + "It looks like our pre-trained model is more familiar with some specific breeds. You could do further work to fine-tune this model to improve performance." + ] + }, + { + "attachments": {}, + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Writing Data\n", + "\n", + "See: [Writing Data](df-writing-data)\n", + "\n", + "Writing data will execute your DataFrame and write the results out to the specified backend. For example, to write data out to Parquet with {meth}`df.write_parquet() `:\n" + ] + }, + { + "cell_type": "code", + "execution_count": 39, + "metadata": { + "tags": [] + }, + "outputs": [ + { + "name": "stderr", + "output_type": "stream", + "text": [ + " \r" + ] + } + ], + "source": [ + "written_df = df.write_parquet(\"my-dataframe.parquet\")" + ] + }, + { + "attachments": {}, + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Note that writing your dataframe is a **blocking** operation that executes your DataFrame. It will return a new `DataFrame` that contains the filepaths to the written data:" + ] + }, + { + "cell_type": "code", + "execution_count": 40, + "metadata": { + "tags": [] + }, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
path
Utf8
my-dataframe.parquet/36bdcc36-9fec-4be8-b22e-a792cc5c6c4c-0.parquet
\n", + "(Showing first 1 of 1 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 path \u2502\n", + "\u2502 --- \u2502\n", + "\u2502 Utf8 \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 my-dataframe.parquet/36bdcc36\u2026 \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 1 of 1 rows)" + ] + }, + "execution_count": 40, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "written_df" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## What's Next?\n", + "\n", + "Now that you have a basic sense of Daft's functionality and features, take a look at some of the other resources to help you get the most out of Daft:\n", + "\n", + "- [The Daft User Guide](/user_guide/index.rst) for more information on specific topics\n", + "- Hands-on Tutorials in Google Colab on:\n", + " - Image Classification\n", + " - NLP Similarity Search / Vector Embedding\n", + " - Querying Images\n", + " - Image Generation with GPUs\n", + "\n", + "\n", + "### Contributing\n", + "Excited about Daft and want to contribute? Join us [on Github](https://github.com/Eventual-Inc/Daft) \ud83d\ude80" + ] + } + ], + "metadata": { + "kernelspec": { + "display_name": "venv", + "language": "python", + "name": "python3" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.11.8" + } + }, + "nbformat": 4, + "nbformat_minor": 4 +} diff --git a/docs-v2/advanced/distributed.md b/docs-v2/advanced/distributed.md new file mode 100644 index 0000000000..9f78b4e0f9 --- /dev/null +++ b/docs-v2/advanced/distributed.md @@ -0,0 +1,72 @@ +# Distributed Computing + +By default, Daft runs using your local machine's resources and your operations are thus limited by the CPUs, memory and GPUs available to you in your single local development machine. + +However, Daft has strong integrations with [Ray](https://www.ray.io) which is a distributed computing framework for distributing computations across a cluster of machines. Here is a snippet showing how you can connect Daft to a Ray cluster: + +=== "๐Ÿ Python" + + ```python + import daft + + daft.context.set_runner_ray() + ``` + +By default, if no address is specified Daft will spin up a Ray cluster locally on your machine. If you are running Daft on a powerful machine (such as an AWS P3 machine which is equipped with multiple GPUs) this is already very useful because Daft can parallelize its execution of computation across your CPUs and GPUs. However, if instead you already have your own Ray cluster running remotely, you can connect Daft to it by supplying an address: + +=== "๐Ÿ Python" + + ```python + daft.context.set_runner_ray(address="ray://url-to-mycluster") + ``` + +For more information about the `address` keyword argument, please see the [Ray documentation on initialization](https://docs.ray.io/en/latest/ray-core/api/doc/ray.init.html). + + +If you want to start a single node ray cluster on your local machine, you can do the following: + +```bash +> pip install ray[default] +> ray start --head --port=6379 +``` + +This should output something like: + +``` +Usage stats collection is enabled. To disable this, add `--disable-usage-stats` to the command that starts the cluster, or run the following command: `ray disable-usage-stats` before starting the cluster. See https://docs.ray.io/en/master/cluster/usage-stats.html for more details. + +Local node IP: 127.0.0.1 + +-------------------- +Ray runtime started. +-------------------- + +... +``` + +You can take the IP address and port and pass it to Daft: + +=== "๐Ÿ Python" + + ```python + >>> import daft + >>> daft.context.set_runner_ray("127.0.0.1:6379") + DaftContext(_daft_execution_config=, _daft_planning_config=, _runner_config=_RayRunnerConfig(address='127.0.0.1:6379', max_task_backlog=None), _disallow_set_runner=True, _runner=None) + >>> df = daft.from_pydict({ + ... 'text': ['hello', 'world'] + ... }) + 2024-07-29 15:49:26,610 INFO worker.py:1567 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379... + 2024-07-29 15:49:26,622 INFO worker.py:1752 -- Connected to Ray cluster. + >>> print(df) + โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ + โ”‚ text โ”‚ + โ”‚ --- โ”‚ + โ”‚ Utf8 โ”‚ + โ•žโ•โ•โ•โ•โ•โ•โ•โ•ก + โ”‚ hello โ”‚ + โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค + โ”‚ world โ”‚ + โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + + (Showing first 2 of 2 rows) + ``` diff --git a/docs-v2/advanced/memory.md b/docs-v2/advanced/memory.md new file mode 100644 index 0000000000..c6706ae590 --- /dev/null +++ b/docs-v2/advanced/memory.md @@ -0,0 +1,66 @@ +# Managing Memory Usage + +Managing memory usage and avoiding out-of-memory (OOM) issues while still maintaining efficient throughput is one of the biggest challenges when building resilient big data processing system! + +This page is a walkthrough on how Daft handles such situations and possible remedies available to users when you encounter such situations. + +## Out-of-core Processing + +Daft supports [out-of-core data processing](https://en.wikipedia.org/wiki/External_memory_algorithm) when running on the Ray runner by leveraging Ray's object spilling capabilities. + +This means that when the total amount of data in Daft gets too large, Daft will spill data onto disk. This slows down the overall workload (because data now needs to be written to and read from disk) but frees up space in working memory for Daft to continue executing work without causing an OOM. + +You will be alerted when spilling is occurring by log messages that look like this: + +``` +(raylet, ip=xx.xx.xx.xx) Spilled 16920 MiB, 9 objects, write throughput 576 MiB/s. +... +``` + +**Troubleshooting** + +Spilling to disk is a mechanism that Daft uses to ensure workload completion in an environment where there is insufficient memory, but in some cases this can cause issues. + +1. If your cluster is extremely aggressive with spilling (e.g. spilling hundreds of gigabytes of data) it can be possible that your machine may eventually run out of disk space and be killed by your cloud provider + +2. Overly aggressive spilling can also cause your overall workload to be much slower + +There are some things you can do that will help with this. + +1. Use machines with more available memory per-CPU to increase each Ray worker's available memory (e.g. [AWS EC2 r5 instances](https://aws.amazon.com/ec2/instance-types/r5/) + +2. Use more machines in your cluster to increase overall cluster memory size + +3. Use machines with attached local nvme SSD drives for higher throughput when spilling (e.g. AWS EC2 r5d instances) + +For more troubleshooting, you may also wish to consult the [Ray documentation's recommendations for object spilling](https://docs.ray.io/en/latest/ray-core/objects/object-spilling.html). + +## Dealing with out-of-memory (OOM) errors + +While Daft is built to be extremely memory-efficient, there will inevitably be situations in which it has poorly estimated the amount of memory that it will require for a certain operation, or simply cannot do so (for example when running arbitrary user-defined Python functions). + +Even with object spilling enabled, you may still sometimes see errors indicating OOMKill behavior on various levels such as your operating system, Ray or a higher-level cluster orchestrator such as Kubernetes: + +1. On the local PyRunner, you may see that your operating system kills the process with an error message `OOMKilled`. + +2. On the RayRunner, you may notice Ray logs indicating that workers are aggressively being killed by the Raylet with log messages such as: `Workers (tasks / actors) killed due to memory pressure (OOM)` + +3. If you are running in an environment such as Kubernetes, you may notice that your pods are being killed or restarted with an `OOMKill` reason + +These OOMKills are often recoverable (Daft-on-Ray will take care of retrying work after reviving the workers), however they may often significantly affect the runtime of your workload or if we simply cannot recover, fail the workload entirely. + +**Troubleshooting** + +There are some options available to you. + +1. Use machines with more available memory per-CPU to increase each Ray worker's available memory (e.g. AWS EC2 r5 instances) + +2. Use more machines in your cluster to increase overall cluster memory size + +3. Aggressively filter your data so that Daft can avoid reading data that it does not have to (e.g. `df.where(...)`) + +4. Request more memory for your UDFs (see [Resource Requests](../core_concepts/udf.md#resource-requests) if your UDFs are memory intensive (e.g. decompression of data, running large matrix computations etc) + +5. Increase the number of partitions in your dataframe (hence making each partition smaller) using something like: `df.into_partitions(df.num_partitions() * 2)` + +If your workload continues to experience OOM issues, perhaps Daft could be better estimating the required memory to run certain steps in your workload. Please contact Daft developers on our forums! diff --git a/docs-v2/advanced/partitioning.md b/docs-v2/advanced/partitioning.md new file mode 100644 index 0000000000..cbf6edca09 --- /dev/null +++ b/docs-v2/advanced/partitioning.md @@ -0,0 +1,113 @@ +# Partitioning + +Daft is a **distributed** dataframe. This means internally, data is represented as partitions which are then spread out across your system. + +## Why do we need partitions? + +When running in a distributed settings (a cluster of machines), Daft spreads your dataframe's data across these machines. This means that your workload is able to efficiently utilize all the resources in your cluster because each machine is able to work on its assigned partition(s) independently. + +Additionally, certain global operations in a distributed setting requires data to be partitioned in a specific way for the operation to be correct, because all the data matching a certain criteria needs to be on the same machine and in the same partition. For example, in a groupby-aggregation Daft needs to bring together all the data for a given key into the same partition before it can perform a definitive local groupby-aggregation which is then globally correct. Daft refers to this as a "clustering specification", and you are able to see this in the plans that it constructs as well. + +!!! note "Note" + + When running locally on just a single machine, Daft is currently still using partitioning as well. This is still useful for controlling parallelism and how much data is being materialized at a time. + + However, Daft's new experimental execution engine will remove the concept of partitioning entirely for local execution. You may enable it with `DAFT_RUNNER=native`. Instead of using partitioning to control parallelism, this new execution engine performs a streaming-based execution on small "morsels" of data, which provides much more stable memory utilization while improving the user experience with not having to worry about partitioning. + +This user guide helps you think about how to correctly partition your data to improve performance as well as memory stability in Daft. + +General rule of thumb: + +1. **Have Enough Partitions**: our general recommendation for high throughput and maximal resource utilization is to have *at least* `2 x TOTAL_NUM_CPUS` partitions, which allows Daft to fully saturate your CPUs. + +2. **More Partitions**: if you are observing memory issues (excessive spilling or out-of-memory (OOM) issues) then you may choose to increase the number of partitions. This increases the amount of overhead in your system, but improves overall memory stability (since each partition will be smaller). + +3. **Fewer Partitions**: if you are observing a large amount of overhead (e.g. if you observe that shuffle operations such as joins and sorts are taking too much time), then you may choose to decrease the number of partitions. This decreases the amount of overhead in the system, at the cost of using more memory (since each partition will be larger). + +!!! tip "See Also" + + [Managing Memory Usage](memory.md) - a guide for dealing with memory issues when using Daft + +## How is my data partitioned? + +Daft will automatically use certain heuristics to determine the number of partitions for you when you create a DataFrame. When reading data from files (e.g. Parquet, CSV or JSON), Daft will group small files/split large files appropriately +into nicely-sized partitions based on their estimated in-memory data sizes. + +To interrogate the partitioning of your current DataFrame, you may use the [`df.explain(show_all=True)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.explain.html#daft.DataFrame.explain) method. Here is an example output from a simple `df = daft.read_parquet(...)` call on a fairly large number of Parquet files. + +=== "๐Ÿ Python" + + ```python + df = daft.read_parquet("s3://bucket/path_to_100_parquet_files/**") + df.explain(show_all=True) + ``` + +``` {title="Output"} + + == Unoptimized Logical Plan == + + * GlobScanOperator + | Glob paths = [s3://bucket/path_to_100_parquet_files/**] + | ... + + + ... + + + == Physical Plan == + + * TabularScan: + | Num Scan Tasks = 3 + | Estimated Scan Bytes = 72000000 + | Clustering spec = { Num partitions = 3 } + | ... +``` + +In the above example, the call to [`df.read_parquet`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.read_parquet.html) read 100 Parquet files, but the Physical Plan indicates that Daft will only create 3 partitions. This is because these files are quite small (in this example, totalling about 72MB of data) and Daft recognizes that it should be able to read them as just 3 partitions, each with about 33 files each! + +## How can I change the way my data is partitioned? + +You can change the way your data is partitioned by leveraging certain DataFrame methods: + +1. [`daft.DataFrame.repartition`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.repartition.html#daft.DataFrame.repartition): repartitions your data into `N` partitions by performing a hash-bucketing that ensure that all data with the same values for the specified columns ends up in the same partition. Expensive, requires data movement between partitions and machines. + +2. [`daft.DataFrame.into_partitions`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.into_partitions.html#daft.DataFrame.into_partitions): splits or coalesces adjacent partitions to meet the specified target number of total partitions. This is less expensive than a call to `df.repartition` because it does not require shuffling or moving data between partitions. + +3. Many global dataframe operations such as [`daft.DataFrame.join`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.join.html#daft.DataFrame.join), [`daft.DataFrame.sort`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.sort.html#daft.DataFrame.sort) and [`daft.GroupedDataframe.agg`](https://www.getdaft.io/projects/docs/en/stable/api_docs/groupby.html#daft.dataframe.GroupedDataFrame.agg) will change the partitioning of your data. This is because they require shuffling data between partitions to be globally correct. + +Note that many of these methods will change both the *number of partitions* as well as the *clustering specification* of the new partitioning. For example, when calling `df.repartition(8, col("x"))`, the resultant dataframe will now have 8 partitions in total with the additional guarantee that all rows with the same value of `col("x")` are in the same partition! This is called "hash partitioning". + +=== "๐Ÿ Python" + + ```python + df = df.repartition(8, daft.col("x")) + df.explain(show_all=True) + ``` + +``` {title="Output"} + + == Unoptimized Logical Plan == + + * Repartition: Scheme = Hash + | Num partitions = Some(8) + | By = col(x) + | + * GlobScanOperator + | Glob paths = [s3://bucket/path_to_1000_parquet_files/**] + | ... + + ... + + == Physical Plan == + + * ReduceMerge + | + * FanoutByHash: 8 + | Partition by = col(x) + | + * TabularScan: + | Num Scan Tasks = 3 + | Estimated Scan Bytes = 72000000 + | Clustering spec = { Num partitions = 3 } + | ... +``` diff --git a/docs-v2/core_concepts.md b/docs-v2/core_concepts.md new file mode 100644 index 0000000000..e748cfc4cf --- /dev/null +++ b/docs-v2/core_concepts.md @@ -0,0 +1,2335 @@ +# Core Concepts + +!!! failure "todo(docs): Created a mega Core Concepts page to improve user journey. Right now `toc_depth=3` for visibility into subheadings, but are there now too many subheadings? Can we combine/condense some sections? My concern is that `toc_depth=2` is not informative unless we turn off `toc.integrate` and have a right-hand menu for all the subheadings." + +Learn about the core concepts that Daft is built on! + +## DataFrame + +!!! failure "todo(docs): Check that this page makes sense. Can we have a 1-1 mapping of "Common data operations that you would perform on DataFrames are: ..." to its respective section?" + +!!! failure "todo(docs): I reused some of these sections in the Quickstart (create df, execute df and view data, select rows, select columns) but the examples in the quickstart are different. Should we still keep those sections on this page?" + + +If you are coming from other DataFrame libraries such as Pandas or Polars, here are some key differences about Daft DataFrames: + +1. **Distributed:** When running in a distributed cluster, Daft splits your data into smaller "chunks" called *Partitions*. This allows Daft to process your data in parallel across multiple machines, leveraging more resources to work with large datasets. + +2. **Lazy:** When you write operations on a DataFrame, Daft doesn't execute them immediately. Instead, it creates a plan (called a query plan) of what needs to be done. This plan is optimized and only executed when you specifically request the results, which can lead to more efficient computations. + +3. **Multimodal:** Unlike traditional tables that usually contain simple data types like numbers and text, Daft DataFrames can handle complex data types in its columns. This includes things like images, audio files, or even custom Python objects. + +For a full comparison between Daft and other DataFrame Libraries, see [DataFrame Comparison](resources/dataframe_comparison.md). + +Common data operations that you would perform on DataFrames are: + +1. [**Filtering rows:**](core_concepts.md#selecting-rows) Use [`df.where(...)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.where.html#daft.DataFrame.where) to keep only the rows that meet certain conditions. +2. **Creating new columns:** Use [`df.with_column(...)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.with_column.html#daft.DataFrame.with_column) to add a new column based on calculations from existing ones. +3. [**Joining DataFrames:**](core_concepts.md#combining-dataframes) Use [`df.join(other_df, ...)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.join.html#daft.DataFrame.join) to combine two DataFrames based on common columns. +4. [**Sorting:**](core_concepts.md#reordering-rows) Use [`df.sort(...)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.sort.html#daft.DataFrame.sort) to arrange your data based on values in one or more columns. +5. [**Grouping and aggregating:**](core_concepts.md#aggregations-and-grouping) Use [`df.groupby(...).agg(...)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.groupby.html#daft.DataFrame.groupby) to summarize your data by groups. + +### Creating a Dataframe + +!!! tip "See Also" + + [Reading Data](core_concepts.md#reading-data) and [Writing Data](core_concepts.md#writing-data) - a more in-depth guide on various options for reading and writing data to and from Daft DataFrames from in-memory data (Python, Arrow), files (Parquet, CSV, JSON), SQL Databases and Data Catalogs + +Let's create our first Dataframe from a Python dictionary of columns. + +=== "๐Ÿ Python" + + ```python + import daft + + df = daft.from_pydict({ + "A": [1, 2, 3, 4], + "B": [1.5, 2.5, 3.5, 4.5], + "C": [True, True, False, False], + "D": [None, None, None, None], + }) + ``` + +Examine your Dataframe by printing it: + +``` +df +``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”† B โ”† C โ”† D โ”‚ +โ”‚ --- โ”† --- โ”† --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Float64 โ”† Boolean โ”† Null โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 1 โ”† 1.5 โ”† true โ”† None โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2 โ”† 2.5 โ”† true โ”† None โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 3 โ”† 3.5 โ”† false โ”† None โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 4 โ”† 4.5 โ”† false โ”† None โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 4 of 4 rows) +``` + +Congratulations - you just created your first DataFrame! It has 4 columns, "A", "B", "C", and "D". Let's try to select only the "A", "B", and "C" columns: + +=== "๐Ÿ Python" + ``` python + df = df.select("A", "B", "C") + df + ``` + +=== "โš™๏ธ SQL" + ```python + df = daft.sql("SELECT A, B, C FROM df") + df + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”† B โ”† C โ”‚ +โ”‚ --- โ”† --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Float64 โ”† Boolean โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(No data to display: Dataframe not materialized) +``` + +But wait - why is it printing the message `(No data to display: Dataframe not materialized)` and where are the rows of each column? + +### Executing DataFrame and Viewing Data + +The reason that our DataFrame currently does not display its rows is that Daft DataFrames are **lazy**. This just means that Daft DataFrames will defer all its work until you tell it to execute. + +In this case, Daft is just deferring the work required to read the data and select columns, however in practice this laziness can be very useful for helping Daft optimize your queries before execution! + +!!! info "Info" + + When you call methods on a Daft Dataframe, it defers the work by adding to an internal "plan". You can examine the current plan of a DataFrame by calling [`df.explain()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.explain.html#daft.DataFrame.explain)! + + Passing the `show_all=True` argument will show you the plan after Daft applies its query optimizations and the physical (lower-level) plan. + + ``` + Plan Output + + == Unoptimized Logical Plan == + + * Project: col(A), col(B), col(C) + | + * Source: + | Number of partitions = 1 + | Output schema = A#Int64, B#Float64, C#Boolean, D#Null + + + == Optimized Logical Plan == + + * Project: col(A), col(B), col(C) + | + * Source: + | Number of partitions = 1 + | Output schema = A#Int64, B#Float64, C#Boolean, D#Null + + + == Physical Plan == + + * Project: col(A), col(B), col(C) + | Clustering spec = { Num partitions = 1 } + | + * InMemoryScan: + | Schema = A#Int64, B#Float64, C#Boolean, D#Null, + | Size bytes = 65, + | Clustering spec = { Num partitions = 1 } + ``` + +We can tell Daft to execute our DataFrame and store the results in-memory using [`df.collect()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.collect.html#daft.DataFrame.collect): + +=== "๐Ÿ Python" + ``` python + df.collect() + df + ``` + +``` {title="Output"} +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”† B โ”† C โ”† D โ”‚ +โ”‚ --- โ”† --- โ”† --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Float64 โ”† Boolean โ”† Null โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 1 โ”† 1.5 โ”† true โ”† None โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2 โ”† 2.5 โ”† true โ”† None โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 3 โ”† 3.5 โ”† false โ”† None โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 4 โ”† 4.5 โ”† false โ”† None โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 4 of 4 rows) +``` + +Now your DataFrame object `df` is **materialized** - Daft has executed all the steps required to compute the results, and has cached the results in memory so that it can display this preview. + +Any subsequent operations on `df` will avoid recomputations, and just use this materialized result! + +### When should I materialize my DataFrame? + +If you "eagerly" call [`df.collect()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.collect.html#daft.DataFrame.collect) immediately on every DataFrame, you may run into issues: + +1. If data is too large at any step, materializing all of it may cause memory issues +2. Optimizations are not possible since we cannot "predict future operations" + +However, data science is all about experimentation and trying different things on the same data. This means that materialization is crucial when working interactively with DataFrames, since it speeds up all subsequent experimentation on that DataFrame. + +We suggest materializing DataFrames using [`df.collect()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.collect.html#daft.DataFrame.collect) when they contain expensive operations (e.g. sorts or expensive function calls) and have to be called multiple times by downstream code: + +=== "๐Ÿ Python" + ``` python + df = df.sort("A") # expensive sort + df.collect() # materialize the DataFrame + + # All subsequent work on df avoids recomputing previous steps + df.sum("B").show() + df.mean("B").show() + df.with_column("try_this", df["A"] + 1).show(5) + ``` + +=== "โš™๏ธ SQL" + ```python + df = daft.sql("SELECT * FROM df ORDER BY A") + df.collect() + + # All subsequent work on df avoids recomputing previous steps + daft.sql("SELECT sum(B) FROM df").show() + daft.sql("SELECT mean(B) FROM df").show() + daft.sql("SELECT *, (A + 1) AS try_this FROM df").show(5) + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ B โ”‚ +โ”‚ --- โ”‚ +โ”‚ Float64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 12 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 1 of 1 rows) + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ B โ”‚ +โ”‚ --- โ”‚ +โ”‚ Float64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 3 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 1 of 1 rows) + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”† B โ”† C โ”† try_this โ”‚ +โ”‚ --- โ”† --- โ”† --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Float64 โ”† Boolean โ”† Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 1 โ”† 1.5 โ”† true โ”† 2 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2 โ”† 2.5 โ”† true โ”† 3 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 3 โ”† 3.5 โ”† false โ”† 4 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 4 โ”† 4.5 โ”† false โ”† 5 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 4 of 4 rows) +``` + +In many other cases however, there are better options than materializing your entire DataFrame with [`df.collect()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.collect.html#daft.DataFrame.collect): + +1. **Peeking with df.show(N)**: If you only want to "peek" at the first few rows of your data for visualization purposes, you can use [`df.show(N)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.show.html#daft.DataFrame.show), which processes and shows only the first `N` rows. +2. **Writing to disk**: The `df.write_*` methods will process and write your data to disk per-partition, avoiding materializing it all in memory at once. +3. **Pruning data**: You can materialize your DataFrame after performing a [`df.limit()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.limit.html#daft.DataFrame.limit), [`df.where()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.where.html#daft.DataFrame.where) or [`df.select()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.select.html#daft.DataFrame.select) operation which processes your data or prune it down to a smaller size. + +### Schemas and Types + +Notice also that when we printed our DataFrame, Daft displayed its **schema**. Each column of your DataFrame has a **name** and a **type**, and all data in that column will adhere to that type! + +Daft can display your DataFrame's schema without materializing it. Under the hood, it performs intelligent sampling of your data to determine the appropriate schema, and if you make any modifications to your DataFrame it can infer the resulting types based on the operation. + +!!! note "Note" + + Under the hood, Daft represents data in the [Apache Arrow](https://arrow.apache.org/) format, which allows it to efficiently represent and work on data using high-performance kernels which are written in Rust. + +### Running Computation with Expressions + +To run computations on data in our DataFrame, we use Expressions. + +The following statement will [`df.show()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.show.html#daft.DataFrame.show) a DataFrame that has only one column - the column `A` from our original DataFrame but with every row incremented by 1. + +=== "๐Ÿ Python" + ``` python + df.select(df["A"] + 1).show() + ``` + +=== "โš™๏ธ SQL" + ```python + daft.sql("SELECT A + 1 FROM df").show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”‚ +โ”‚ --- โ”‚ +โ”‚ Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 2 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 3 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 4 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 5 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 4 of 4 rows) +``` + +!!! info "Info" + + A common pattern is to create a new columns using [`DataFrame.with_column`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.with_column.html): + + === "๐Ÿ Python" + ``` python + # Creates a new column named "foo" which takes on values + # of column "A" incremented by 1 + df = df.with_column("foo", df["A"] + 1) + df.show() + ``` + + === "โš™๏ธ SQL" + ```python + # Creates a new column named "foo" which takes on values + # of column "A" incremented by 1 + df = daft.sql("SELECT *, A + 1 AS foo FROM df") + df.show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”† B โ”† C โ”† foo โ”‚ +โ”‚ --- โ”† --- โ”† --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Float64 โ”† Boolean โ”† Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 1 โ”† 1.5 โ”† true โ”† 2 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2 โ”† 2.5 โ”† true โ”† 3 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 3 โ”† 3.5 โ”† false โ”† 4 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 4 โ”† 4.5 โ”† false โ”† 5 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 4 of 4 rows) +``` + +Congratulations, you have just written your first **Expression**: `df["A"] + 1`! Expressions are a powerful way of describing computation on columns. For more details, check out the next section on [Expressions](core_concepts.md#expressions). + + + +### Selecting Rows + +We can limit the rows to the first ``N`` rows using [`df.limit(N)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.limit.html#daft.DataFrame.limit): + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({ + "A": [1, 2, 3, 4, 5], + "B": [6, 7, 8, 9, 10], + }) + + df.limit(3).show() + ``` + +``` {title="Output"} + ++---------+---------+ +| A | B | +| Int64 | Int64 | ++=========+=========+ +| 1 | 6 | ++---------+---------+ +| 2 | 7 | ++---------+---------+ +| 3 | 8 | ++---------+---------+ +(Showing first 3 rows) +``` + +We can also filter rows using [`df.where()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.where.html#daft.DataFrame.where), which takes an input a Logical Expression predicate: + +=== "๐Ÿ Python" + ``` python + df.where(df["A"] > 3).show() + ``` + +``` {title="Output"} + ++---------+---------+ +| A | B | +| Int64 | Int64 | ++=========+=========+ +| 4 | 9 | ++---------+---------+ +| 5 | 10 | ++---------+---------+ +(Showing first 2 rows) +``` + +### Selecting Columns + +Select specific columns in a DataFrame using [`df.select()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.select.html#daft.DataFrame.select), which also takes Expressions as an input. + +=== "๐Ÿ Python" + ``` python + import daft + + df = daft.from_pydict({"A": [1, 2, 3], "B": [4, 5, 6]}) + + df.select("A").show() + ``` + +``` {title="Output"} + ++---------+ +| A | +| Int64 | ++=========+ +| 1 | ++---------+ +| 2 | ++---------+ +| 3 | ++---------+ +(Showing first 3 rows) +``` + +A useful alias for [`df.select()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.select.html#daft.DataFrame.select) is indexing a DataFrame with a list of column names or Expressions: + +=== "๐Ÿ Python" + ``` python + df[["A", "B"]].show() + ``` + +``` {title="Output"} + ++---------+---------+ +| A | B | +| Int64 | Int64 | ++=========+=========+ +| 1 | 4 | ++---------+---------+ +| 2 | 5 | ++---------+---------+ +| 3 | 6 | ++---------+---------+ +(Showing first 3 rows) +``` + +Sometimes, it may be useful to exclude certain columns from a DataFrame. This can be done with [`df.exclude()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.exclude.html#daft.DataFrame.exclude): + +=== "๐Ÿ Python" + ``` python + df.exclude("A").show() + ``` + +```{title="Output"} + ++---------+ +| B | +| Int64 | ++=========+ +| 4 | ++---------+ +| 5 | ++---------+ +| 6 | ++---------+ +(Showing first 3 rows) +``` + +Adding a new column can be achieved with [`df.with_column()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.with_column.html#daft.DataFrame.with_column): + +=== "๐Ÿ Python" + ``` python + df.with_column("C", df["A"] + df["B"]).show() + ``` + +``` {title="Output"} + ++---------+---------+---------+ +| A | B | C | +| Int64 | Int64 | Int64 | ++=========+=========+=========+ +| 1 | 4 | 5 | ++---------+---------+---------+ +| 2 | 5 | 7 | ++---------+---------+---------+ +| 3 | 6 | 9 | ++---------+---------+---------+ +(Showing first 3 rows) +``` + +#### Selecting Columns Using Wildcards + +We can select multiple columns at once using wildcards. The expression [`.col(*)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.col.html#daft.col) selects every column in a DataFrame, and you can operate on this expression in the same way as a single column: + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({"A": [1, 2, 3], "B": [4, 5, 6]}) + df.select(col("*") * 3).show() + ``` + +``` {title="Output"} +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”† B โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 3 โ”† 12 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 6 โ”† 15 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 9 โ”† 18 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ +``` + +We can also select multiple columns within structs using `col("struct.*")`: + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({ + "A": [ + {"B": 1, "C": 2}, + {"B": 3, "C": 4} + ] + }) + df.select(col("A.*")).show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ B โ”† C โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 1 โ”† 2 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 3 โ”† 4 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ +``` + +Under the hood, wildcards work by finding all of the columns that match, then copying the expression several times and replacing the wildcard. This means that there are some caveats: + +* Only one wildcard is allowed per expression tree. This means that `col("*") + col("*")` and similar expressions do not work. +* Be conscious about duplicated column names. Any code like `df.select(col("*"), col("*") + 3)` will not work because the wildcards expand into the same column names. + + For the same reason, `col("A") + col("*")` will not work because the name on the left-hand side is inherited, meaning all the output columns are named `A`, causing an error if there is more than one. + However, `col("*") + col("A")` will work fine. + +### Combining DataFrames + +Two DataFrames can be column-wise joined using [`df.join()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.join.html#daft.DataFrame.join). + +This requires a "join key", which can be supplied as the `on` argument if both DataFrames have the same name for their key columns, or the `left_on` and `right_on` argument if the key column has different names in each DataFrame. + +Daft also supports multi-column joins if you have a join key comprising of multiple columns! + +=== "๐Ÿ Python" + ``` python + df1 = daft.from_pydict({"A": [1, 2, 3], "B": [4, 5, 6]}) + df2 = daft.from_pydict({"A": [1, 2, 3], "C": [7, 8, 9]}) + + df1.join(df2, on="A").show() + ``` + +``` {title="Output"} + ++---------+---------+---------+ +| A | B | C | +| Int64 | Int64 | Int64 | ++=========+=========+=========+ +| 1 | 4 | 7 | ++---------+---------+---------+ +| 2 | 5 | 8 | ++---------+---------+---------+ +| 3 | 6 | 9 | ++---------+---------+---------+ +(Showing first 3 rows) +``` + +### Reordering Rows + +Rows in a DataFrame can be reordered based on some column using [`df.sort()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.sort.html#daft.DataFrame.sort). Daft also supports multi-column sorts for sorting on multiple columns at once. + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({ + "A": [1, 2, 3], + "B": [6, 7, 8], + }) + + df.sort("A", desc=True).show() + ``` + +```{title="Output"} + ++---------+---------+ +| A | B | +| Int64 | Int64 | ++=========+=========+ +| 3 | 8 | ++---------+---------+ +| 2 | 7 | ++---------+---------+ +| 1 | 6 | ++---------+---------+ +(Showing first 3 rows) +``` + +### Exploding Columns + +The [`df.explode()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.explode.html#daft.DataFrame.explode) method can be used to explode a column containing a list of values into multiple rows. All other rows will be **duplicated**. + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({ + "A": [1, 2, 3], + "B": [[1, 2, 3], [4, 5, 6], [7, 8, 9]], + }) + + df.explode("B").show() + ``` + +``` {title="Output"} + ++---------+---------+ +| A | B | +| Int64 | Int64 | ++=========+=========+ +| 1 | 1 | ++---------+---------+ +| 1 | 2 | ++---------+---------+ +| 1 | 3 | ++---------+---------+ +| 2 | 4 | ++---------+---------+ +| 2 | 5 | ++---------+---------+ +| 2 | 6 | ++---------+---------+ +| 3 | 7 | ++---------+---------+ +| 3 | 8 | ++---------+---------+ +(Showing first 8 rows) +``` + + + + +## Expressions + +Expressions are how you can express computations that should be run over columns of data. + +### Creating Expressions + +#### Referring to a column in a DataFrame + +Most commonly you will be creating expressions by using the [`daft.col`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.col.html#daft.col) function. + +=== "๐Ÿ Python" + ``` python + # Refers to column "A" + daft.col("A") + ``` + +=== "โš™๏ธ SQL" + ```python + daft.sql_expr("A") + ``` + +``` {title="Output"} + +col(A) +``` + +The above code creates an Expression that refers to a column named `"A"`. + +### Using SQL + +Daft can also parse valid SQL as expressions. + +=== "โš™๏ธ SQL" + ```python + daft.sql_expr("A + 1") + ``` +``` {title="Output"} + +col(A) + lit(1) +``` + +The above code will create an expression representing "the column named 'x' incremented by 1". For many APIs, [`sql_expr`](https://www.getdaft.io/projects/docs/en/stable/api_docs/sql.html#daft.sql_expr) will actually be applied for you as syntactic sugar! + +#### Literals + +You may find yourself needing to hardcode a "single value" oftentimes as an expression. Daft provides a [`lit()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.lit.html) helper to do so: + +=== "๐Ÿ Python" + ``` python + from daft import lit + + # Refers to an expression which always evaluates to 42 + lit(42) + ``` + +=== "โš™๏ธ SQL" + ```python + # Refers to an expression which always evaluates to 42 + daft.sql_expr("42") + ``` + +```{title="Output"} + +lit(42) +``` +This special :func:`~daft.expressions.lit` expression we just created evaluates always to the value ``42``. + +#### Wildcard Expressions + +You can create expressions on multiple columns at once using a wildcard. The expression [`col("*")`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.col.html#daft.col)) selects every column in a DataFrame, and you can operate on this expression in the same way as a single column: + +=== "๐Ÿ Python" + ``` python + import daft + from daft import col + + df = daft.from_pydict({"A": [1, 2, 3], "B": [4, 5, 6]}) + df.select(col("*") * 3).show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”† B โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 3 โ”† 12 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 6 โ”† 15 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 9 โ”† 18 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ +``` + +Wildcards also work very well for accessing all members of a struct column: + +=== "๐Ÿ Python" + ``` python + + import daft + from daft import col + + df = daft.from_pydict({ + "person": [ + {"name": "Alice", "age": 30}, + {"name": "Bob", "age": 25}, + {"name": "Charlie", "age": 35} + ] + }) + + # Access all fields of the 'person' struct + df.select(col("person.*")).show() + ``` + +=== "โš™๏ธ SQL" + ```python + import daft + + df = daft.from_pydict({ + "person": [ + {"name": "Alice", "age": 30}, + {"name": "Bob", "age": 25}, + {"name": "Charlie", "age": 35} + ] + }) + + # Access all fields of the 'person' struct using SQL + daft.sql("SELECT person.* FROM df").show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ name โ”† age โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ String โ”† Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ Alice โ”† 30 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ Bob โ”† 25 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ Charlie โ”† 35 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ +``` + +In this example, we use the wildcard `*` to access all fields of the `person` struct column. This is equivalent to selecting each field individually (`person.name`, `person.age`), but is more concise and flexible, especially when dealing with structs that have many fields. + + + +### Composing Expressions + +#### Numeric Expressions + +Since column "A" is an integer, we can run numeric computation such as addition, division and checking its value. Here are some examples where we create new columns using the results of such computations: + +=== "๐Ÿ Python" + ``` python + # Add 1 to each element in column "A" + df = df.with_column("A_add_one", df["A"] + 1) + + # Divide each element in column A by 2 + df = df.with_column("A_divide_two", df["A"] / 2.) + + # Check if each element in column A is more than 1 + df = df.with_column("A_gt_1", df["A"] > 1) + + df.collect() + ``` + +=== "โš™๏ธ SQL" + ```python + df = daft.sql(""" + SELECT + *, + A + 1 AS A_add_one, + A / 2.0 AS A_divide_two, + A > 1 AS A_gt_1 + FROM df + """) + df.collect() + ``` + +```{title="Output"} + ++---------+-------------+----------------+-----------+ +| A | A_add_one | A_divide_two | A_gt_1 | +| Int64 | Int64 | Float64 | Boolean | ++=========+=============+================+===========+ +| 1 | 2 | 0.5 | false | ++---------+-------------+----------------+-----------+ +| 2 | 3 | 1 | true | ++---------+-------------+----------------+-----------+ +| 3 | 4 | 1.5 | true | ++---------+-------------+----------------+-----------+ +(Showing first 3 of 3 rows) +``` + +Notice that the returned types of these operations are also well-typed according to their input types. For example, calling ``df["A"] > 1`` returns a column of type [`Boolean`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.bool). + +Both the [`Float`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.float32) and [`Int`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.int16) types are numeric types, and inherit many of the same arithmetic Expression operations. You may find the full list of numeric operations in the [Expressions API Reference](https://www.getdaft.io/projects/docs/en/stable/api_docs/expressions.html#numeric). + +#### String Expressions + +Daft also lets you have columns of strings in a DataFrame. Let's take a look! + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({"B": ["foo", "bar", "baz"]}) + df.show() + ``` + +``` {title="Output"} + ++--------+ +| B | +| Utf8 | ++========+ +| foo | ++--------+ +| bar | ++--------+ +| baz | ++--------+ +(Showing first 3 rows) +``` + +Unlike the numeric types, the string type does not support arithmetic operations such as `*` and `/`. The one exception to this is the `+` operator, which is overridden to concatenate two string expressions as is commonly done in Python. Let's try that! + +=== "๐Ÿ Python" + ``` python + df = df.with_column("B2", df["B"] + "foo") + df.show() + ``` + +=== "โš™๏ธ SQL" + ```python + df = daft.sql("SELECT *, B + 'foo' AS B2 FROM df") + df.show() + ``` + +``` {title="Output"} + ++--------+--------+ +| B | B2 | +| Utf8 | Utf8 | ++========+========+ +| foo | foofoo | ++--------+--------+ +| bar | barfoo | ++--------+--------+ +| baz | bazfoo | ++--------+--------+ +(Showing first 3 rows) +``` + +There are also many string operators that are accessed through a separate [`.str.*`](https://www.getdaft.io/projects/docs/en/stable/api_docs/expressions.html#strings) "method namespace". + +For example, to check if each element in column "B" contains the substring "a", we can use the [`.str.contains`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.Expression.str.contains.html#daft.Expression.str.contains) method: + +=== "๐Ÿ Python" + ``` python + df = df.with_column("B2_contains_B", df["B2"].str.contains(df["B"])) + df.show() + ``` + +=== "โš™๏ธ SQL" + ```python + df = daft.sql("SELECT *, contains(B2, B) AS B2_contains_B FROM df") + df.show() + ``` + +``` {title="Output"} + ++--------+--------+-----------------+ +| B | B2 | B2_contains_B | +| Utf8 | Utf8 | Boolean | ++========+========+=================+ +| foo | foofoo | true | ++--------+--------+-----------------+ +| bar | barfoo | true | ++--------+--------+-----------------+ +| baz | bazfoo | true | ++--------+--------+-----------------+ +(Showing first 3 rows) +``` + +You may find a full list of string operations in the [Expressions API Reference](https://www.getdaft.io/projects/docs/en/stable/api_docs/expressions.html). + +#### URL Expressions + +One special case of a String column you may find yourself working with is a column of URL strings. + +Daft provides the [`.url.*`](https://www.getdaft.io/projects/docs/en/stable/api_docs/expressions.html) method namespace with functionality for working with URL strings. For example, to download data from URLs: + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({ + "urls": [ + "https://www.google.com", + "s3://daft-public-data/open-images/validation-images/0001eeaf4aed83f9.jpg", + ], + }) + df = df.with_column("data", df["urls"].url.download()) + df.collect() + ``` + +=== "โš™๏ธ SQL" + ```python + df = daft.from_pydict({ + "urls": [ + "https://www.google.com", + "s3://daft-public-data/open-images/validation-images/0001eeaf4aed83f9.jpg", + ], + }) + df = daft.sql(""" + SELECT + urls, + url_download(urls) AS data + FROM df + """) + df.collect() + ``` + +``` {title="Output"} + ++----------------------+----------------------+ +| urls | data | +| Utf8 | Binary | ++======================+======================+ +| https://www.google.c | b' df["B"]).if_else(df["A"], df["B"]), + ) + + df.collect() + ``` + +=== "โš™๏ธ SQL" + ```python + df = daft.from_pydict({"A": [1, 2, 3], "B": [0, 2, 4]}) + + df = daft.sql(""" + SELECT + A, + B, + CASE + WHEN A > B THEN A + ELSE B + END AS A_if_bigger_else_B + FROM df + """) + + df.collect() + ``` + +```{title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”† B โ”† A_if_bigger_else_B โ”‚ +โ”‚ --- โ”† --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Int64 โ”† Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 1 โ”† 0 โ”† 1 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2 โ”† 2 โ”† 2 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 3 โ”† 4 โ”† 4 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 3 of 3 rows) +``` + +This is a useful expression for cleaning your data! + + +#### Temporal Expressions + +Daft provides rich support for working with temporal data types like Timestamp and Duration. Let's explore some common temporal operations: + +##### Basic Temporal Operations + +You can perform arithmetic operations with timestamps and durations, such as adding a duration to a timestamp or calculating the duration between two timestamps: + +=== "๐Ÿ Python" + ``` python + import datetime + + df = daft.from_pydict({ + "timestamp": [ + datetime.datetime(2021, 1, 1, 0, 1, 1), + datetime.datetime(2021, 1, 1, 0, 1, 59), + datetime.datetime(2021, 1, 1, 0, 2, 0), + ] + }) + + # Add 10 seconds to each timestamp + df = df.with_column( + "plus_10_seconds", + df["timestamp"] + datetime.timedelta(seconds=10) + ) + + df.show() + ``` + +=== "โš™๏ธ SQL" + ```python + import datetime + + df = daft.from_pydict({ + "timestamp": [ + datetime.datetime(2021, 1, 1, 0, 1, 1), + datetime.datetime(2021, 1, 1, 0, 1, 59), + datetime.datetime(2021, 1, 1, 0, 2, 0), + ] + }) + + # Add 10 seconds to each timestamp and calculate duration between timestamps + df = daft.sql(""" + SELECT + timestamp, + timestamp + INTERVAL '10 seconds' as plus_10_seconds, + FROM df + """) + + df.show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ timestamp โ”† plus_10_seconds โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ Timestamp(Microseconds, None) โ”† Timestamp(Microseconds, None) โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 2021-01-01 00:01:01 โ”† 2021-01-01 00:01:11 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2021-01-01 00:01:59 โ”† 2021-01-01 00:02:09 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2021-01-01 00:02:00 โ”† 2021-01-01 00:02:10 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ +``` + +##### Temporal Component Extraction + +The [`.dt.*`](https://www.getdaft.io/projects/docs/en/stable/api_docs/expressions.html#temporal) method namespace provides extraction methods for the components of a timestamp, such as year, month, day, hour, minute, and second: + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({ + "timestamp": [ + datetime.datetime(2021, 1, 1, 0, 1, 1), + datetime.datetime(2021, 1, 1, 0, 1, 59), + datetime.datetime(2021, 1, 1, 0, 2, 0), + ] + }) + + # Extract year, month, day, hour, minute, and second from the timestamp + df = df.with_columns({ + "year": df["timestamp"].dt.year(), + "month": df["timestamp"].dt.month(), + "day": df["timestamp"].dt.day(), + "hour": df["timestamp"].dt.hour(), + "minute": df["timestamp"].dt.minute(), + "second": df["timestamp"].dt.second() + }) + + df.show() + ``` + +=== "โš™๏ธ SQL" + ```python + df = daft.from_pydict({ + "timestamp": [ + datetime.datetime(2021, 1, 1, 0, 1, 1), + datetime.datetime(2021, 1, 1, 0, 1, 59), + datetime.datetime(2021, 1, 1, 0, 2, 0), + ] + }) + + # Extract year, month, day, hour, minute, and second from the timestamp + df = daft.sql(""" + SELECT + timestamp, + year(timestamp) as year, + month(timestamp) as month, + day(timestamp) as day, + hour(timestamp) as hour, + minute(timestamp) as minute, + second(timestamp) as second + FROM df + """) + + df.show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ timestamp โ”† year โ”† month โ”† day โ”† hour โ”† minute โ”† second โ”‚ +โ”‚ --- โ”† --- โ”† --- โ”† --- โ”† --- โ”† --- โ”† --- โ”‚ +โ”‚ Timestamp(Microseconds, None) โ”† Int32 โ”† UInt32 โ”† UInt32 โ”† UInt32 โ”† UInt32 โ”† UInt32 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 2021-01-01 00:01:01 โ”† 2021 โ”† 1 โ”† 1 โ”† 0 โ”† 1 โ”† 1 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2021-01-01 00:01:59 โ”† 2021 โ”† 1 โ”† 1 โ”† 0 โ”† 1 โ”† 59 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2021-01-01 00:02:00 โ”† 2021 โ”† 1 โ”† 1 โ”† 0 โ”† 2 โ”† 0 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ +``` + +##### Time Zone Operations + +You can parse strings as timestamps with time zones and convert between different time zones: + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({ + "timestamp_str": [ + "2021-01-01 00:00:00.123 +0800", + "2021-01-02 12:30:00.456 +0800" + ] + }) + + # Parse the timestamp string with time zone and convert to New York time + df = df.with_column( + "ny_time", + df["timestamp_str"].str.to_datetime( + "%Y-%m-%d %H:%M:%S%.3f %z", + timezone="America/New_York" + ) + ) + + df.show() + ``` + +=== "โš™๏ธ SQL" + ```python + df = daft.from_pydict({ + "timestamp_str": [ + "2021-01-01 00:00:00.123 +0800", + "2021-01-02 12:30:00.456 +0800" + ] + }) + + # Parse the timestamp string with time zone and convert to New York time + df = daft.sql(""" + SELECT + timestamp_str, + to_datetime(timestamp_str, '%Y-%m-%d %H:%M:%S%.3f %z', 'America/New_York') as ny_time + FROM df + """) + + df.show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ timestamp_str โ”† ny_time โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ Utf8 โ”† Timestamp(Milliseconds, Some("America/New_York")) โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 2021-01-01 00:00:00.123 +0800 โ”† 2020-12-31 11:00:00.123 EST โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2021-01-02 12:30:00.456 +0800 โ”† 2021-01-01 23:30:00.456 EST โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ +``` + +##### Temporal Truncation + +The [`.dt.truncate()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.Expression.dt.truncate.html#daft.Expression.dt.truncate) method allows you to truncate timestamps to specific time units. This can be useful for grouping data by time periods. For example, to truncate timestamps to the nearest hour: + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({ + "timestamp": [ + datetime.datetime(2021, 1, 7, 0, 1, 1), + datetime.datetime(2021, 1, 8, 0, 1, 59), + datetime.datetime(2021, 1, 9, 0, 30, 0), + datetime.datetime(2021, 1, 10, 1, 59, 59), + ] + }) + + # Truncate timestamps to the nearest hour + df = df.with_column( + "hour_start", + df["timestamp"].dt.truncate("1 hour") + ) + + df.show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ timestamp โ”† hour_start โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ Timestamp(Microseconds, None) โ”† Timestamp(Microseconds, None) โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 2021-01-07 00:01:01 โ”† 2021-01-07 00:00:00 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2021-01-08 00:01:59 โ”† 2021-01-08 00:00:00 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2021-01-09 00:30:00 โ”† 2021-01-09 00:00:00 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2021-01-10 01:59:59 โ”† 2021-01-10 01:00:00 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ +``` + +!!! failure "todo(docs): Should this section also have sql examples?" + +Daft can read data from a variety of sources, and write data to many destinations. + +## Reading Data + +### From Files + +DataFrames can be loaded from file(s) on some filesystem, commonly your local filesystem or a remote cloud object store such as AWS S3. + +Additionally, Daft can read data from a variety of container file formats, including CSV, line-delimited JSON and Parquet. + +Daft supports file paths to a single file, a directory of files, and wildcards. It also supports paths to remote object storage such as AWS S3. +=== "๐Ÿ Python" + ```python + import daft + + # You can read a single CSV file from your local filesystem + df = daft.read_csv("path/to/file.csv") + + # You can also read folders of CSV files, or include wildcards to select for patterns of file paths + df = daft.read_csv("path/to/*.csv") + + # Other formats such as parquet and line-delimited JSON are also supported + df = daft.read_parquet("path/to/*.parquet") + df = daft.read_json("path/to/*.json") + + # Remote filesystems such as AWS S3 are also supported, and can be specified with their protocols + df = daft.read_csv("s3://mybucket/path/to/*.csv") + ``` + +To learn more about each of these constructors, as well as the options that they support, consult the API documentation on [`creating DataFrames from files`](https://www.getdaft.io/projects/docs/en/stable/api_docs/creation.html#df-io-files). + +### From Data Catalogs + +If you use catalogs such as Apache Iceberg or Hive, you may wish to consult our user guide on integrations with Data Catalogs: [`Daft integration with Data Catalogs`](https://www.getdaft.io/projects/docs/en/stable/user_guide/integrations.html). + +### From File Paths + +Daft also provides an easy utility to create a DataFrame from globbing a path. You can use the [`daft.from_glob_path`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.from_glob_path.html#daft.from_glob_path) method which will read a DataFrame of globbed filepaths. + +=== "๐Ÿ Python" + ``` python + df = daft.from_glob_path("s3://mybucket/path/to/images/*.jpeg") + + # +----------+------+-----+ + # | name | size | ... | + # +----------+------+-----+ + # ... + ``` + +This is especially useful for reading things such as a folder of images or documents into Daft. A common pattern is to then download data from these files into your DataFrame as bytes, using the [`.url.download()`](https://getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.Expression.url.download.html#daft.Expression.url.download) method. + + +### From Memory + +For testing, or small datasets that fit in memory, you may also create DataFrames using Python lists and dictionaries. + +=== "๐Ÿ Python" + ``` python + # Create DataFrame using a dictionary of {column_name: list_of_values} + df = daft.from_pydict({"A": [1, 2, 3], "B": ["foo", "bar", "baz"]}) + + # Create DataFrame using a list of rows, where each row is a dictionary of {column_name: value} + df = daft.from_pylist([{"A": 1, "B": "foo"}, {"A": 2, "B": "bar"}, {"A": 3, "B": "baz"}]) + ``` + +To learn more, consult the API documentation on [`creating DataFrames from in-memory data structures`](https://www.getdaft.io/projects/docs/en/stable/api_docs/creation.html#df-io-in-memory). + +### From Databases + +Daft can also read data from a variety of databases, including PostgreSQL, MySQL, Trino, and SQLite using the [`daft.read_sql`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.read_sql.html#daft.read_sql) method. In order to partition the data, you can specify a partition column, which will allow Daft to read the data in parallel. + +=== "๐Ÿ Python" + ``` python + # Read from a PostgreSQL database + uri = "postgresql://user:password@host:port/database" + df = daft.read_sql("SELECT * FROM my_table", uri) + + # Read with a partition column + df = daft.read_sql("SELECT * FROM my_table", partition_col="date", uri) + ``` + +To learn more, consult the [`SQL User Guide`](https://www.getdaft.io/projects/docs/en/stable/user_guide/integrations/sql.html) or the API documentation on [`daft.read_sql`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.read_sql.html#daft.read_sql). + +### Reading a column of URLs + +Daft provides a convenient way to read data from a column of URLs using the [`.url.download()`](https://getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.Expression.url.download.html#daft.Expression.url.download) method. This is particularly useful when you have a DataFrame with a column containing URLs pointing to external resources that you want to fetch and incorporate into your DataFrame. + +Here's an example of how to use this feature: + +=== "๐Ÿ Python" + ```python + # Assume we have a DataFrame with a column named 'image_urls' + df = daft.from_pydict({ + "image_urls": [ + "https://example.com/image1.jpg", + "https://example.com/image2.jpg", + "https://example.com/image3.jpg" + ] + }) + + # Download the content from the URLs and create a new column 'image_data' + df = df.with_column("image_data", df["image_urls"].url.download()) + df.show() + ``` + +``` {title="Output"} + ++------------------------------------+------------------------------------+ +| image_urls | image_data | +| Utf8 | Binary | ++====================================+====================================+ +| https://example.com/image1.jpg | b'\xff\xd8\xff\xe0\x00\x10JFIF...' | ++------------------------------------+------------------------------------+ +| https://example.com/image2.jpg | b'\xff\xd8\xff\xe0\x00\x10JFIF...' | ++------------------------------------+------------------------------------+ +| https://example.com/image3.jpg | b'\xff\xd8\xff\xe0\x00\x10JFIF...' | ++------------------------------------+------------------------------------+ + +(Showing first 3 of 3 rows) +``` + +This approach allows you to efficiently download and process data from a large number of URLs in parallel, leveraging Daft's distributed computing capabilities. + +## Writing Data + +Writing data will execute your DataFrame and write the results out to the specified backend. The [`df.write_*(...)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/dataframe.html#df-write-data) methods are used to write DataFrames to files or other destinations. + +=== "๐Ÿ Python" + ``` python + # Write to various file formats in a local folder + df.write_csv("path/to/folder/") + df.write_parquet("path/to/folder/") + + # Write DataFrame to a remote filesystem such as AWS S3 + df.write_csv("s3://mybucket/path/") + ``` + +!!! note "Note" + + Because Daft is a distributed DataFrame library, by default it will produce multiple files (one per partition) at your specified destination. Writing your dataframe is a **blocking** operation that executes your DataFrame. It will return a new `DataFrame` that contains the filepaths to the written data. + +## DataTypes + +All columns in a Daft DataFrame have a DataType (also often abbreviated as `dtype`). + +All elements of a column are of the same dtype, or they can be the special Null value (indicating a missing value). + +Daft provides simple DataTypes that are ubiquituous in many DataFrames such as numbers, strings and dates - all the way up to more complex types like tensors and images. + +!!! tip "Tip" + + For a full overview on all the DataTypes that Daft supports, see the [DataType API Reference](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html). + + +### Numeric DataTypes + +Numeric DataTypes allows Daft to represent numbers. These numbers can differ in terms of the number of bits used to represent them (8, 16, 32 or 64 bits) and the semantic meaning of those bits +(float vs integer vs unsigned integers). + +Examples: + +1. [`DataType.int8()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.int8): represents an 8-bit signed integer (-128 to 127) +2. [`DataType.float32()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.float32): represents a 32-bit float (a float number with about 7 decimal digits of precision) + +Columns/expressions with these datatypes can be operated on with many numeric expressions such as `+` and `*`. + +See also: [Numeric Expressions](https://www.getdaft.io/projects/docs/en/stable/user_guide/expressions.html#userguide-numeric-expressions) + +### Logical DataTypes + +The [`Boolean`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.bool) DataType represents values which are boolean values: `True`, `False` or `Null`. + +Columns/expressions with this dtype can be operated on using logical expressions such as ``&`` and [`.if_else()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.Expression.if_else.html#daft.Expression.if_else). + +See also: [Logical Expressions](https://www.getdaft.io/projects/docs/en/stable/user_guide/expressions.html#userguide-logical-expressions) + +### String Types + +Daft has string types, which represent a variable-length string of characters. + +As a convenience method, string types also support the `+` Expression, which has been overloaded to support concatenation of elements between two [`DataType.string()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.string) columns. + +1. [`DataType.string()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.string): represents a string of UTF-8 characters +2. [`DataType.binary()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.binary): represents a string of bytes + +See also: [String Expressions](https://www.getdaft.io/projects/docs/en/stable/user_guide/expressions.html#userguide-string-expressions) + +### Temporal DataTypes + +Temporal DataTypes represent data that have to do with time. + +Examples: + +1. [`DataType.date()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.date): represents a Date (year, month and day) +2. [`DataType.timestamp()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.timestamp): represents a Timestamp (particular instance in time) + +See also: [Temporal Expressions](https://www.getdaft.io/projects/docs/en/stable/api_docs/expressions.html#api-expressions-temporal) + +### Nested DataTypes + +Nested DataTypes wrap other DataTypes, allowing you to compose types into complex data structures. + +Examples: + +1. [`DataType.list(child_dtype)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.list): represents a list where each element is of the child `dtype` +2. [`DataType.struct({"field_name": child_dtype})`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.struct): represents a structure that has children `dtype`s, each mapped to a field name + +### Python DataType + +The [`DataType.python()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.python) DataType represent items that are Python objects. + +!!! warning "Warning" + + Daft does not impose any invariants about what *Python types* these objects are. To Daft, these are just generic Python objects! + +Python is AWESOME because it's so flexible, but it's also slow and memory inefficient! Thus we recommend: + +1. **Cast early!**: Casting your Python data into native Daft DataTypes if possible - this results in much more efficient downstream data serialization and computation. +2. **Use Python UDFs**: If there is no suitable Daft representation for your Python objects, use Python UDFs to process your Python data and extract the relevant data to be returned as native Daft DataTypes! + +!!! note "Note" + + If you work with Python classes for a generalizable use-case (e.g. documents, protobufs), it may be that these types are good candidates for "promotion" into a native Daft type! Please get in touch with the Daft team and we would love to work together on building your type into canonical Daft types. + +### Complex DataTypes + +Daft supports many more interesting complex DataTypes, for example: + +* [`DataType.tensor()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.tensor): Multi-dimensional (potentially uniformly-shaped) tensors of data +* [`DataType.embedding()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.embedding): Lower-dimensional vector representation of data (e.g. words) +* [`DataType.image()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.image): NHWC images + +Daft abstracts away the in-memory representation of your data and provides kernels for many common operations on top of these data types. For supported image operations see the [image expressions API reference](https://www.getdaft.io/projects/docs/en/stable/api_docs/expressions.html#api-expressions-images). For more complex algorithms, you can also drop into a Python UDF to process this data using your custom Python libraries. + +Please add suggestions for new DataTypes to our Github Discussions page! + +## SQL + +Daft supports Structured Query Language (SQL) as a way of constructing query plans (represented in Python as a [`daft.DataFrame`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.html#daft.DataFrame)) and expressions ([`daft.Expression`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.html#daft.DataFrame)). + +SQL is a human-readable way of constructing these query plans, and can often be more ergonomic than using DataFrames for writing queries. + +!!! tip "Daft's SQL support is new and is constantly being improved on!" + + Please give us feedback or submit an [issue](https://github.com/Eventual-Inc/Daft/issues) and we'd love to hear more about what you would like. + + +### Running SQL on DataFrames + +Daft's [`daft.sql`](https://www.getdaft.io/projects/docs/en/stable/api_docs/sql.html#daft.sql) function will automatically detect any [`daft.DataFrame`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.html#daft.DataFrame) objects in your current Python environment to let you query them easily by name. + +=== "โš™๏ธ SQL" + ```python + # Note the variable name `my_special_df` + my_special_df = daft.from_pydict({"A": [1, 2, 3], "B": [1, 2, 3]}) + + # Use the SQL table name "my_special_df" to refer to the above DataFrame! + sql_df = daft.sql("SELECT A, B FROM my_special_df") + + sql_df.show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”† B โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 1 โ”† 1 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2 โ”† 2 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 3 โ”† 3 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 3 of 3 rows) +``` + +In the above example, we query the DataFrame called `"my_special_df"` by simply referring to it in the SQL command. This produces a new DataFrame `sql_df` which can +natively integrate with the rest of your Daft query. + +### Reading data from SQL + +!!! warning "Warning" + + This feature is a WIP and will be coming soon! We will support reading common datasources directly from SQL: + + === "๐Ÿ Python" + + ```python + daft.sql("SELECT * FROM read_parquet('s3://...')") + daft.sql("SELECT * FROM read_delta_lake('s3://...')") + ``` + + Today, a workaround for this is to construct your dataframe in Python first and use it from SQL instead: + + === "๐Ÿ Python" + + ```python + df = daft.read_parquet("s3://...") + daft.sql("SELECT * FROM df") + ``` + + We appreciate your patience with us and hope to deliver this crucial feature soon! + +### SQL Expressions + +SQL has the concept of expressions as well. Here is an example of a simple addition expression, adding columns "a" and "b" in SQL to produce a new column C. + +We also present here the equivalent query for SQL and DataFrame. Notice how similar the concepts are! + +=== "โš™๏ธ SQL" + ```python + df = daft.from_pydict({"A": [1, 2, 3], "B": [1, 2, 3]}) + df = daft.sql("SELECT A + B as C FROM df") + df.show() + ``` + +=== "๐Ÿ Python" + ``` python + expr = (daft.col("A") + daft.col("B")).alias("C") + + df = daft.from_pydict({"A": [1, 2, 3], "B": [1, 2, 3]}) + df = df.select(expr) + df.show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ C โ”‚ +โ”‚ --- โ”‚ +โ”‚ Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 2 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 4 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 6 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 3 of 3 rows) +``` + +In the above query, both the SQL version of the query and the DataFrame version of the query produce the same result. + +Under the hood, they run the same Expression `col("A") + col("B")`! + +One really cool trick you can do is to use the [`daft.sql_expr`](https://www.getdaft.io/projects/docs/en/stable/api_docs/sql.html#daft.sql_expr) function as a helper to easily create Expressions. The following are equivalent: + +=== "โš™๏ธ SQL" + ```python + sql_expr = daft.sql_expr("A + B as C") + print("SQL expression:", sql_expr) + ``` + +=== "๐Ÿ Python" + ``` python + py_expr = (daft.col("A") + daft.col("B")).alias("C") + print("Python expression:", py_expr) + ``` + +``` {title="Output"} + +SQL expression: col(A) + col(B) as C +Python expression: col(A) + col(B) as C +``` + +This means that you can pretty much use SQL anywhere you use Python expressions, making Daft extremely versatile at mixing workflows which leverage both SQL and Python. + +As an example, consider the filter query below and compare the two equivalent Python and SQL queries: + +=== "โš™๏ธ SQL" + ```python + df = daft.from_pydict({"A": [1, 2, 3], "B": [1, 2, 3]}) + + # Daft automatically converts this string using `daft.sql_expr` + df = df.where("A < 2") + + df.show() + ``` + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({"A": [1, 2, 3], "B": [1, 2, 3]}) + + # Using Daft's Python Expression API + df = df.where(df["A"] < 2) + + df.show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”† B โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 1 โ”† 1 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 1 of 1 rows) +``` + +Pretty sweet! Of course, this support for running Expressions on your columns extends well beyond arithmetic as we'll see in the next section on SQL Functions. + +### SQL Functions + +SQL also has access to all of Daft's powerful [`daft.Expression`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.html#daft.DataFrame) functionality through SQL functions. + +However, unlike the Python Expression API which encourages method-chaining (e.g. `col("a").url.download().image.decode()`), in SQL you have to do function nesting instead (e.g. `"image_decode(url_download(a))"`). + +!!! note "Note" + + A full catalog of the available SQL Functions in Daft is available in the [`../api_docs/sql`](https://www.getdaft.io/projects/docs/en/stable/api_docs/sql.html). + + Note that it closely mirrors the Python API, with some function naming differences vs the available Python methods. + We also have some aliased functions for ANSI SQL-compliance or familiarity to users coming from other common SQL dialects such as PostgreSQL and SparkSQL to easily find their functionality. + +Here is an example of an equivalent function call in SQL vs Python: + +=== "โš™๏ธ SQL" + ```python + df = daft.from_pydict({"urls": [ + "https://user-images.githubusercontent.com/17691182/190476440-28f29e87-8e3b-41c4-9c28-e112e595f558.png", + "https://user-images.githubusercontent.com/17691182/190476440-28f29e87-8e3b-41c4-9c28-e112e595f558.png", + "https://user-images.githubusercontent.com/17691182/190476440-28f29e87-8e3b-41c4-9c28-e112e595f558.png", + ]}) + df = daft.sql("SELECT image_decode(url_download(urls)) FROM df") + df.show() + ``` + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({"urls": [ + "https://user-images.githubusercontent.com/17691182/190476440-28f29e87-8e3b-41c4-9c28-e112e595f558.png", + "https://user-images.githubusercontent.com/17691182/190476440-28f29e87-8e3b-41c4-9c28-e112e595f558.png", + "https://user-images.githubusercontent.com/17691182/190476440-28f29e87-8e3b-41c4-9c28-e112e595f558.png", + ]}) + df = df.select(daft.col("urls").url.download().image.decode()) + df.show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ urls โ”‚ +โ”‚ --- โ”‚ +โ”‚ Image[MIXED] โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 3 of 3 rows) +``` + +## Aggregations and Grouping + +Some operations such as the sum or the average of a column are called **aggregations**. Aggregations are operations that reduce the number of rows in a column. + +### Global Aggregations + +An aggregation can be applied on an entire DataFrame, for example to get the mean on a specific column: + +=== "๐Ÿ Python" + ``` python + import daft + + df = daft.from_pydict({ + "class": ["a", "a", "b", "b"], + "score": [10, 20., 30., 40], + }) + + df.mean("score").show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ score โ”‚ +โ”‚ --- โ”‚ +โ”‚ Float64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 25 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 1 of 1 rows) +``` + +For a full list of available Dataframe aggregations, see [Aggregations](https://www.getdaft.io/projects/docs/en/stable/api_docs/dataframe.html#df-aggregations). + +Aggregations can also be mixed and matched across columns, via the `agg` method: + +=== "๐Ÿ Python" + ``` python + df.agg( + df["score"].mean().alias("mean_score"), + df["score"].max().alias("max_score"), + df["class"].count().alias("class_count"), + ).show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ mean_score โ”† max_score โ”† class_count โ”‚ +โ”‚ --- โ”† --- โ”† --- โ”‚ +โ”‚ Float64 โ”† Float64 โ”† UInt64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 25 โ”† 40 โ”† 4 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 1 of 1 rows) +``` + +For a full list of available aggregation expressions, see [Aggregation Expressions](https://www.getdaft.io/projects/docs/en/stable/api_docs/expressions.html#api-aggregation-expression) + +### Grouped Aggregations + +Aggregations can also be called on a "Grouped DataFrame". For the above example, perhaps we want to get the mean "score" not for the entire DataFrame, but for each "class". + +Let's run the mean of column "score" again, but this time grouped by "class": + +=== "๐Ÿ Python" + ``` python + df.groupby("class").mean("score").show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ class โ”† score โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ Utf8 โ”† Float64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ a โ”† 15 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ b โ”† 35 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 2 of 2 rows) +``` + +To run multiple aggregations on a Grouped DataFrame, you can use the `agg` method: + +=== "๐Ÿ Python" + ``` python + df.groupby("class").agg( + df["score"].mean().alias("mean_score"), + df["score"].max().alias("max_score"), + ).show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ class โ”† mean_score โ”† max_score โ”‚ +โ”‚ --- โ”† --- โ”† --- โ”‚ +โ”‚ Utf8 โ”† Float64 โ”† Float64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ a โ”† 15 โ”† 20 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ b โ”† 35 โ”† 40 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 2 of 2 rows) +``` + +## User-Defined Functions (UDF) + +A key piece of functionality in Daft is the ability to flexibly define custom functions that can run computations on any data in your dataframe. This section walks you through the different types of UDFs that Daft allows you to run. + +Let's first create a dataframe that will be used as a running example throughout this tutorial! + +=== "๐Ÿ Python" + ``` python + import daft + import numpy as np + + df = daft.from_pydict({ + # the `image` column contains images represented as 2D numpy arrays + "image": [np.ones((128, 128)) for i in range(16)], + # the `crop` column contains a box to crop from our image, represented as a list of integers: [x1, x2, y1, y2] + "crop": [[0, 1, 0, 1] for i in range(16)], + }) + ``` + + +### Per-column per-row functions using [`.apply`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.Expression.apply.html) + +You can use [`.apply`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.Expression.apply.html) to run a Python function on every row in a column. + +For example, the following example creates a new `flattened_image` column by calling `.flatten()` on every object in the `image` column. + +=== "๐Ÿ Python" + ``` python + df.with_column( + "flattened_image", + df["image"].apply(lambda img: img.flatten(), return_dtype=daft.DataType.python()) + ).show(2) + ``` + +``` {title="Output"} + ++----------------------+---------------+---------------------+ +| image | crop | flattened_image | +| Python | List[Int64] | Python | ++======================+===============+=====================+ +| [[1. 1. 1. ... 1. 1. | [0, 1, 0, 1] | [1. 1. 1. ... 1. 1. | +| 1.] [1. 1. 1. ... | | 1.] | +| 1. 1. 1.] [1. 1.... | | | ++----------------------+---------------+---------------------+ +| [[1. 1. 1. ... 1. 1. | [0, 1, 0, 1] | [1. 1. 1. ... 1. 1. | +| 1.] [1. 1. 1. ... | | 1.] | +| 1. 1. 1.] [1. 1.... | | | ++----------------------+---------------+---------------------+ +(Showing first 2 rows) +``` + +Note here that we use the `return_dtype` keyword argument to specify that our returned column type is a Python column! + +### Multi-column per-partition functions using [`@udf`](https://www.getdaft.io/projects/docs/en/stable/api_docs/udf.html#creating-udfs) + +[`.apply`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.Expression.apply.html) is great for convenience, but has two main limitations: + +1. It can only run on single columns +2. It can only run on single items at a time + +Daft provides the [`@udf`](https://www.getdaft.io/projects/docs/en/stable/api_docs/udf.html#creating-udfs) decorator for defining your own UDFs that process multiple columns or multiple rows at a time. + +For example, let's try writing a function that will crop all our images in the `image` column by its corresponding value in the `crop` column: + +=== "๐Ÿ Python" + ``` python + @daft.udf(return_dtype=daft.DataType.python()) + def crop_images(images, crops, padding=0): + cropped = [] + for img, crop in zip(images.to_pylist(), crops.to_pylist()): + x1, x2, y1, y2 = crop + cropped_img = img[x1:x2 + padding, y1:y2 + padding] + cropped.append(cropped_img) + return cropped + + df = df.with_column( + "cropped", + crop_images(df["image"], df["crop"], padding=1), + ) + df.show(2) + ``` + +``` {title="Output"} + ++----------------------+---------------+--------------------+ +| image | crop | cropped | +| Python | List[Int64] | Python | ++======================+===============+====================+ +| [[1. 1. 1. ... 1. 1. | [0, 1, 0, 1] | [[1. 1.] [1. 1.]] | +| 1.] [1. 1. 1. ... | | | +| 1. 1. 1.] [1. 1.... | | | ++----------------------+---------------+--------------------+ +| [[1. 1. 1. ... 1. 1. | [0, 1, 0, 1] | [[1. 1.] [1. 1.]] | +| 1.] [1. 1. 1. ... | | | +| 1. 1. 1.] [1. 1.... | | | ++----------------------+---------------+--------------------+ +(Showing first 2 rows) +``` + +There's a few things happening here, let's break it down: + +1. `crop_images` is a normal Python function. It takes as input: + + a. A list of images: `images` + + b. A list of cropping boxes: `crops` + + c. An integer indicating how much padding to apply to the right and bottom of the cropping: `padding` + +2. To allow Daft to pass column data into the `images` and `crops` arguments, we decorate the function with [`@udf`](https://www.getdaft.io/projects/docs/en/stable/api_docs/udf.html#creating-udfs) + + a. `return_dtype` defines the returned data type. In this case, we return a column containing Python objects of numpy arrays + + b. At runtime, because we call the UDF on the `image` and `crop` columns, the UDF will receive a [`daft.Series`](https://www.getdaft.io/projects/docs/en/stable/api_docs/series.html#daft.Series) object for each argument. + +3. We can create a new column in our DataFrame by applying our UDF on the `"image"` and `"crop"` columns inside of a [`df.with_column()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.with_column.html#daft.DataFrame.with_column) call. + +#### UDF Inputs + +When you specify an Expression as an input to a UDF, Daft will calculate the result of that Expression and pass it into your function as a [`daft.Series`](https://www.getdaft.io/projects/docs/en/stable/api_docs/series.html#daft.Series) object. + +The Daft [`daft.Series`](https://www.getdaft.io/projects/docs/en/stable/api_docs/series.html#daft.Series) is just an abstraction on a "column" of data! You can obtain several different data representations from a [`daft.Series`](https://www.getdaft.io/projects/docs/en/stable/api_docs/series.html#daft.Series): + +1. PyArrow Arrays (`pa.Array`): [`s.to_arrow()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/series.html#daft.Series.to_arrow) +2. Python lists (`list`): [`s.to_pylist()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/series.html#daft.Series.to_pylist) + +Depending on your application, you may choose a different data representation that is more performant or more convenient! + +!!! info "Info" + + Certain array formats have some restrictions around the type of data that they can handle: + + 1. **Null Handling**: In Pandas and Numpy, nulls are represented as NaNs for numeric types, and Nones for non-numeric types. Additionally, the existence of nulls will trigger a type casting from integer to float arrays. If null handling is important to your use-case, we recommend using one of the other available options. + + 2. **Python Objects**: PyArrow array formats cannot support Python columns. + + We recommend using Python lists if performance is not a major consideration, and using the arrow-native formats such as PyArrow arrays and numpy arrays if performance is important. + +#### Return Types + +The `return_dtype` argument specifies what type of column your UDF will return. Types can be specified using the [`daft.DataType`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType) class. + +Your UDF function itself needs to return a batch of columnar data, and can do so as any one of the following array types: + +1. Numpy Arrays (`np.ndarray`) +2. PyArrow Arrays (`pa.Array`) +3. Python lists (`list`) + +Note that if the data you have returned is not castable to the return_dtype that you specify (e.g. if you return a list of floats when you've specified a `return_dtype=DataType.bool()`), Daft will throw a runtime error! + +### Class UDFs + +UDFs can also be created on Classes, which allow for initialization on some expensive state that can be shared between invocations of the class, for example downloading data or creating a model. + +=== "๐Ÿ Python" + ``` python + @daft.udf(return_dtype=daft.DataType.int64()) + class RunModel: + + def __init__(self): + # Perform expensive initializations + self._model = create_model() + + def __call__(self, features_col): + return self._model(features_col) + ``` + +Running Class UDFs are exactly the same as running their functional cousins. + +=== "๐Ÿ Python" + ``` python + df = df.with_column("image_classifications", RunModel(df["images"])) + ``` + +### Resource Requests + +Sometimes, you may want to request for specific resources for your UDF. For example, some UDFs need one GPU to run as they will load a model onto the GPU. + +To do so, you can create your UDF and assign it a resource request: + +=== "๐Ÿ Python" + ``` python + @daft.udf(return_dtype=daft.DataType.int64(), num_gpus=1) + class RunModelWithOneGPU: + + def __init__(self): + # Perform expensive initializations + self._model = create_model() + + def __call__(self, features_col): + return self._model(features_col) + ``` + + ``` python + df = df.with_column( + "image_classifications", + RunModelWithOneGPU(df["images"]), + ) + ``` + +In the above example, if Daft ran on a Ray cluster consisting of 8 GPUs and 64 CPUs, Daft would be able to run 8 replicas of your UDF in parallel, thus massively increasing the throughput of your UDF! + +UDFs can also be parametrized with new resource requests after being initialized. + +=== "๐Ÿ Python" + ``` python + RunModelWithTwoGPUs = RunModelWithOneGPU.override_options(num_gpus=2) + df = df.with_column( + "image_classifications", + RunModelWithTwoGPUs(df["images"]), + ) + ``` + +### Example: UDFs in ML Workloads + +Weโ€™ll define a function that uses a pre-trained PyTorch model: [ResNet50](https://pytorch.org/vision/main/models/generated/torchvision.models.resnet50.html) to classify the dog pictures. Weโ€™ll pass the contents of the image `urls` column and send the classification predictions to a new column `classify_breed`. + +Working with PyTorch adds some complexity but you can just run the cells below to perform the classification. + +First, make sure to install and import some extra dependencies: + +```bash + +%pip install validators matplotlib Pillow torch torchvision + +``` + +=== "๐Ÿ Python" + + ```python + # import additional libraries, these are necessary for PyTorch + import torch + ``` + +Define your `ClassifyImages` UDF. Models are expensive to initialize and load, so we want to do this as few times as possible, and share a model across multiple invocations. + +=== "๐Ÿ Python" + + ```python + @udf(return_dtype=DataType.fixed_size_list(dtype=DataType.string(), size=2)) + class ClassifyImages: + def __init__(self): + # Perform expensive initializations - create and load the pre-trained model + self.model = torch.hub.load("NVIDIA/DeepLearningExamples:torchhub", "nvidia_resnet50", pretrained=True) + self.utils = torch.hub.load("NVIDIA/DeepLearningExamples:torchhub", "nvidia_convnets_processing_utils") + self.model.eval().to(torch.device("cpu")) + + def __call__(self, images_urls): + uris = images_urls.to_pylist() + batch = torch.cat([self.utils.prepare_input_from_uri(uri) for uri in uris]).to(torch.device("cpu")) + + with torch.no_grad(): + output = torch.nn.functional.softmax(self.model(batch), dim=1) + + results = self.utils.pick_n_best(predictions=output, n=1) + return [result[0] for result in results] + ``` + +Now you're ready to call this function on the `urls` column and store the outputs in a new column we'll call `classify_breed`: + +=== "๐Ÿ Python" + + ```python + classified_images_df = df_family.with_column("classify_breed", ClassifyImages(daft.col("urls"))) + classified_images_df.select("dog_name", "image", "classify_breed").show() + ``` + +!!! failure "todo(docs): Insert table of dog urls? or new UDF example?" + +## Multimodal Data + +Daft is built to work comfortably with multimodal data types, including URLs and images. You can use the [`url.download()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.Expression.url.download.html#daft.Expression.url.download) expression to download the bytes from a URL. Letโ€™s store them in a new column using the `with_column` method: + +!!! failure "todo(docs): This example is originally from 10min quickstart" + +=== "๐Ÿ Python" + + ```python + df_family = df_family.with_column("image_bytes", df_dogs["urls"].url.download(on_error="null")) + df_family.show() + ``` + +```{title="Output"} ++-------------------+---------+----------+------------------------------------------------------------------+--------------------------------------------+ +| full_name | has_dog | dog_name | urls | image_bytes | +| Utf8 | Boolean | Utf8 | Utf8 | Binary | ++-------------------+---------+----------+------------------------------------------------------------------+--------------------------------------------+ +| Ernesto Evergreen | true | Ernie | https://live.staticflickr.com/65535/53671838774_03ba68d203_o.jpg | b"\xff\xd8\xff\xe0\x00\x10JFIF\x00\x01"... | +| James Jale | true | Jackie | https://live.staticflickr.com/65535/53671700073_2c9441422e_o.jpg | b"\xff\xd8\xff\xe0\x00\x10JFIF\x00\x01"... | +| Wolfgang Winter | true | Wolfie | https://live.staticflickr.com/65535/53670606332_1ea5f2ce68_o.jpg | b"\xff\xd8\xff\xe0\x00\x10JFIF\x00\x01"... | +| Shandra Shamas | true | Shaggie | https://live.staticflickr.com/65535/53671838039_b97411a441_o.jpg | b"\xff\xd8\xff\xe0\x00\x10JFIF\x00\x01"... | +| Zaya Zaphora | true | Zadie | https://live.staticflickr.com/65535/53671698613_0230f8af3c_o.jpg | b"\xff\xd8\xff\xe0\x00\x10JFIF\x00\x01"... | ++-------------------+---------+----------+------------------------------------------------------------------+--------------------------------------------+ +(Showing first 5 of 5 rows) +``` + +Letโ€™s turn the bytes into human-readable images using [`image.decode()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.Expression.image.decode.html): + +=== "๐Ÿ Python" + + ```python + df_family = df_family.with_column("image", daft.col("image_bytes").image.decode()) + df_family.show() + ``` + +!!! failure "todo(docs): This example is originally from 10min quickstart. Insert dog images or create new example?" + +## What's Next? + +### Integrations + +
+ +- [:simple-ray: **Ray**](integrations/ray.md) +- [**Unity Catalog**](integrations/unity_catalog.md) +- [**Apache Iceberg**](integrations/iceberg.md) +- [**Delta Lake**](integrations/delta_lake.md) +- [:material-microsoft-azure: **Microsoft Azure**](integrations/azure.md) +- [:fontawesome-brands-aws: **Amazon Web Services (AWS)**](integrations/aws.md) +- [**SQL**](integrations/sql.md) +- [:simple-huggingface: **Hugging Face Datasets**](integrations/huggingface.md) + +
+ +### Migration Guide + +
+ + +- [:simple-dask: **Coming from Dask**](migration/dask_migration.md) + +
+ +### Tutorials + +
+ +- [:material-image-edit: **MNIST Digit Classification**](https://colab.research.google.com/github/Eventual-Inc/Daft/blob/main/tutorials/mnist.ipynb) +- [:octicons-search-16: **Running LLMs on the Red Pajamas Dataset**](https://colab.research.google.com/github/Eventual-Inc/Daft/blob/main/tutorials/embeddings/daft_tutorial_embeddings_stackexchange.ipynb) +- [:material-image-search: **Querying Images with UDFs**](https://colab.research.google.com/github/Eventual-Inc/Daft/blob/main/tutorials/image_querying/top_n_red_color.ipynb) +- [:material-image-sync: **Image Generation on GPUs**](https://colab.research.google.com/github/Eventual-Inc/Daft/blob/main/tutorials/text_to_image/text_to_image_generation.ipynb) + +
+ +### Advanced + +
+ +- [:material-memory: **Managing Memory Usage**](advanced/memory.md) +- [:fontawesome-solid-equals: **Partitioning**](advanced/partitioning.md) +- [:material-distribute-vertical-center: **Distributed Computing**](advanced/distributed.md) + +
diff --git a/docs-v2/core_concepts/aggregations.md b/docs-v2/core_concepts/aggregations.md new file mode 100644 index 0000000000..4bb835f59d --- /dev/null +++ b/docs-v2/core_concepts/aggregations.md @@ -0,0 +1,111 @@ +# Aggregations and Grouping + +Some operations such as the sum or the average of a column are called **aggregations**. Aggregations are operations that reduce the number of rows in a column. + +## Global Aggregations + +An aggregation can be applied on an entire DataFrame, for example to get the mean on a specific column: + +=== "๐Ÿ Python" + ``` python + import daft + + df = daft.from_pydict({ + "class": ["a", "a", "b", "b"], + "score": [10, 20., 30., 40], + }) + + df.mean("score").show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ score โ”‚ +โ”‚ --- โ”‚ +โ”‚ Float64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 25 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 1 of 1 rows) +``` + +For a full list of available Dataframe aggregations, see [Aggregations](https://www.getdaft.io/projects/docs/en/stable/api_docs/dataframe.html#df-aggregations). + +Aggregations can also be mixed and matched across columns, via the `agg` method: + +=== "๐Ÿ Python" + ``` python + df.agg( + df["score"].mean().alias("mean_score"), + df["score"].max().alias("max_score"), + df["class"].count().alias("class_count"), + ).show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ mean_score โ”† max_score โ”† class_count โ”‚ +โ”‚ --- โ”† --- โ”† --- โ”‚ +โ”‚ Float64 โ”† Float64 โ”† UInt64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 25 โ”† 40 โ”† 4 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 1 of 1 rows) +``` + +For a full list of available aggregation expressions, see [Aggregation Expressions](https://www.getdaft.io/projects/docs/en/stable/api_docs/expressions.html#api-aggregation-expression) + +## Grouped Aggregations + +Aggregations can also be called on a "Grouped DataFrame". For the above example, perhaps we want to get the mean "score" not for the entire DataFrame, but for each "class". + +Let's run the mean of column "score" again, but this time grouped by "class": + +=== "๐Ÿ Python" + ``` python + df.groupby("class").mean("score").show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ class โ”† score โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ Utf8 โ”† Float64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ a โ”† 15 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ b โ”† 35 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 2 of 2 rows) +``` + +To run multiple aggregations on a Grouped DataFrame, you can use the `agg` method: + +=== "๐Ÿ Python" + ``` python + df.groupby("class").agg( + df["score"].mean().alias("mean_score"), + df["score"].max().alias("max_score"), + ).show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ class โ”† mean_score โ”† max_score โ”‚ +โ”‚ --- โ”† --- โ”† --- โ”‚ +โ”‚ Utf8 โ”† Float64 โ”† Float64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ a โ”† 15 โ”† 20 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ b โ”† 35 โ”† 40 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 2 of 2 rows) +``` diff --git a/docs-v2/core_concepts/dataframe.md b/docs-v2/core_concepts/dataframe.md new file mode 100644 index 0000000000..9ee1bb3320 --- /dev/null +++ b/docs-v2/core_concepts/dataframe.md @@ -0,0 +1,654 @@ +# DataFrame + +!!! failure "todo(docs): Check that this page makes sense. Can we have a 1-1 mapping of "Common data operations that you would perform on DataFrames are: ..." to its respective section?" + +!!! failure "todo(docs): I reused some of these sections in the Quickstart (create df, execute df and view data, select rows, select columns) but the examples in the quickstart are different. Should we still keep those sections on this page?" + + +If you are coming from other DataFrame libraries such as Pandas or Polars, here are some key differences about Daft DataFrames: + +1. **Distributed:** When running in a distributed cluster, Daft splits your data into smaller "chunks" called *Partitions*. This allows Daft to process your data in parallel across multiple machines, leveraging more resources to work with large datasets. + +2. **Lazy:** When you write operations on a DataFrame, Daft doesn't execute them immediately. Instead, it creates a plan (called a query plan) of what needs to be done. This plan is optimized and only executed when you specifically request the results, which can lead to more efficient computations. + +3. **Multimodal:** Unlike traditional tables that usually contain simple data types like numbers and text, Daft DataFrames can handle complex data types in its columns. This includes things like images, audio files, or even custom Python objects. + +For a full comparison between Daft and other DataFrame Libraries, see [DataFrame Comparison](../resources/dataframe_comparison.md). + +Common data operations that you would perform on DataFrames are: + +1. [**Filtering rows:**](dataframe.md/#selecting-rows) Use [`df.where(...)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.where.html#daft.DataFrame.where) to keep only the rows that meet certain conditions. +2. **Creating new columns:** Use [`df.with_column(...)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.with_column.html#daft.DataFrame.with_column) to add a new column based on calculations from existing ones. +3. [**Joining DataFrames:**](dataframe.md/#combining-dataframes) Use [`df.join(other_df, ...)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.join.html#daft.DataFrame.join) to combine two DataFrames based on common columns. +4. [**Sorting:**](dataframe.md#reordering-rows) Use [`df.sort(...)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.sort.html#daft.DataFrame.sort) to arrange your data based on values in one or more columns. +5. **Grouping and aggregating:** Use [`df.groupby(...).agg(...)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.groupby.html#daft.DataFrame.groupby) to summarize your data by groups. + +## Creating a DataFrame + +!!! tip "See Also" + + [Reading/Writing Data](read_write.md) - a more in-depth guide on various options for reading/writing data to/from Daft DataFrames from in-memory data (Python, Arrow), files (Parquet, CSV, JSON), SQL Databases and Data Catalogs + +Let's create our first Dataframe from a Python dictionary of columns. + +=== "๐Ÿ Python" + + ```python + import daft + + df = daft.from_pydict({ + "A": [1, 2, 3, 4], + "B": [1.5, 2.5, 3.5, 4.5], + "C": [True, True, False, False], + "D": [None, None, None, None], + }) + ``` + +Examine your Dataframe by printing it: + +``` +df +``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”† B โ”† C โ”† D โ”‚ +โ”‚ --- โ”† --- โ”† --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Float64 โ”† Boolean โ”† Null โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 1 โ”† 1.5 โ”† true โ”† None โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2 โ”† 2.5 โ”† true โ”† None โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 3 โ”† 3.5 โ”† false โ”† None โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 4 โ”† 4.5 โ”† false โ”† None โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 4 of 4 rows) +``` + +Congratulations - you just created your first DataFrame! It has 4 columns, "A", "B", "C", and "D". Let's try to select only the "A", "B", and "C" columns: + +=== "๐Ÿ Python" + ``` python + df = df.select("A", "B", "C") + df + ``` + +=== "โš™๏ธ SQL" + ```python + df = daft.sql("SELECT A, B, C FROM df") + df + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”† B โ”† C โ”‚ +โ”‚ --- โ”† --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Float64 โ”† Boolean โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(No data to display: Dataframe not materialized) +``` + +But wait - why is it printing the message `(No data to display: Dataframe not materialized)` and where are the rows of each column? + +## Executing DataFrame and Viewing Data + +The reason that our DataFrame currently does not display its rows is that Daft DataFrames are **lazy**. This just means that Daft DataFrames will defer all its work until you tell it to execute. + +In this case, Daft is just deferring the work required to read the data and select columns, however in practice this laziness can be very useful for helping Daft optimize your queries before execution! + +!!! info "Info" + + When you call methods on a Daft Dataframe, it defers the work by adding to an internal "plan". You can examine the current plan of a DataFrame by calling [`df.explain()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.explain.html#daft.DataFrame.explain)! + + Passing the `show_all=True` argument will show you the plan after Daft applies its query optimizations and the physical (lower-level) plan. + + ``` + Plan Output + + == Unoptimized Logical Plan == + + * Project: col(A), col(B), col(C) + | + * Source: + | Number of partitions = 1 + | Output schema = A#Int64, B#Float64, C#Boolean, D#Null + + + == Optimized Logical Plan == + + * Project: col(A), col(B), col(C) + | + * Source: + | Number of partitions = 1 + | Output schema = A#Int64, B#Float64, C#Boolean, D#Null + + + == Physical Plan == + + * Project: col(A), col(B), col(C) + | Clustering spec = { Num partitions = 1 } + | + * InMemoryScan: + | Schema = A#Int64, B#Float64, C#Boolean, D#Null, + | Size bytes = 65, + | Clustering spec = { Num partitions = 1 } + ``` + +We can tell Daft to execute our DataFrame and store the results in-memory using [`df.collect()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.collect.html#daft.DataFrame.collect): + +=== "๐Ÿ Python" + ``` python + df.collect() + df + ``` + +``` {title="Output"} +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”† B โ”† C โ”† D โ”‚ +โ”‚ --- โ”† --- โ”† --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Float64 โ”† Boolean โ”† Null โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 1 โ”† 1.5 โ”† true โ”† None โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2 โ”† 2.5 โ”† true โ”† None โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 3 โ”† 3.5 โ”† false โ”† None โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 4 โ”† 4.5 โ”† false โ”† None โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 4 of 4 rows) +``` + +Now your DataFrame object `df` is **materialized** - Daft has executed all the steps required to compute the results, and has cached the results in memory so that it can display this preview. + +Any subsequent operations on `df` will avoid recomputations, and just use this materialized result! + +### When should I materialize my DataFrame? + +If you "eagerly" call [`df.collect()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.collect.html#daft.DataFrame.collect) immediately on every DataFrame, you may run into issues: + +1. If data is too large at any step, materializing all of it may cause memory issues +2. Optimizations are not possible since we cannot "predict future operations" + +However, data science is all about experimentation and trying different things on the same data. This means that materialization is crucial when working interactively with DataFrames, since it speeds up all subsequent experimentation on that DataFrame. + +We suggest materializing DataFrames using [`df.collect()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.collect.html#daft.DataFrame.collect) when they contain expensive operations (e.g. sorts or expensive function calls) and have to be called multiple times by downstream code: + +=== "๐Ÿ Python" + ``` python + df = df.sort("A") # expensive sort + df.collect() # materialize the DataFrame + + # All subsequent work on df avoids recomputing previous steps + df.sum("B").show() + df.mean("B").show() + df.with_column("try_this", df["A"] + 1).show(5) + ``` + +=== "โš™๏ธ SQL" + ```python + df = daft.sql("SELECT * FROM df ORDER BY A") + df.collect() + + # All subsequent work on df avoids recomputing previous steps + daft.sql("SELECT sum(B) FROM df").show() + daft.sql("SELECT mean(B) FROM df").show() + daft.sql("SELECT *, (A + 1) AS try_this FROM df").show(5) + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ B โ”‚ +โ”‚ --- โ”‚ +โ”‚ Float64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 12 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 1 of 1 rows) + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ B โ”‚ +โ”‚ --- โ”‚ +โ”‚ Float64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 3 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 1 of 1 rows) + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”† B โ”† C โ”† try_this โ”‚ +โ”‚ --- โ”† --- โ”† --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Float64 โ”† Boolean โ”† Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 1 โ”† 1.5 โ”† true โ”† 2 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2 โ”† 2.5 โ”† true โ”† 3 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 3 โ”† 3.5 โ”† false โ”† 4 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 4 โ”† 4.5 โ”† false โ”† 5 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 4 of 4 rows) +``` + +In many other cases however, there are better options than materializing your entire DataFrame with [`df.collect()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.collect.html#daft.DataFrame.collect): + +1. **Peeking with df.show(N)**: If you only want to "peek" at the first few rows of your data for visualization purposes, you can use [`df.show(N)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.show.html#daft.DataFrame.show), which processes and shows only the first `N` rows. +2. **Writing to disk**: The `df.write_*` methods will process and write your data to disk per-partition, avoiding materializing it all in memory at once. +3. **Pruning data**: You can materialize your DataFrame after performing a [`df.limit()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.limit.html#daft.DataFrame.limit), [`df.where()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.where.html#daft.DataFrame.where) or [`df.select()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.select.html#daft.DataFrame.select) operation which processes your data or prune it down to a smaller size. + +## Schemas and Types + +Notice also that when we printed our DataFrame, Daft displayed its **schema**. Each column of your DataFrame has a **name** and a **type**, and all data in that column will adhere to that type! + +Daft can display your DataFrame's schema without materializing it. Under the hood, it performs intelligent sampling of your data to determine the appropriate schema, and if you make any modifications to your DataFrame it can infer the resulting types based on the operation. + +!!! note "Note" + + Under the hood, Daft represents data in the [Apache Arrow](https://arrow.apache.org/) format, which allows it to efficiently represent and work on data using high-performance kernels which are written in Rust. + +## Running Computation with Expressions + +To run computations on data in our DataFrame, we use Expressions. + +The following statement will [`df.show()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.show.html#daft.DataFrame.show) a DataFrame that has only one column - the column `A` from our original DataFrame but with every row incremented by 1. + +=== "๐Ÿ Python" + ``` python + df.select(df["A"] + 1).show() + ``` + +=== "โš™๏ธ SQL" + ```python + daft.sql("SELECT A + 1 FROM df").show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”‚ +โ”‚ --- โ”‚ +โ”‚ Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 2 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 3 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 4 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 5 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 4 of 4 rows) +``` + +!!! info "Info" + + A common pattern is to create a new columns using [`DataFrame.with_column`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.with_column.html): + + === "๐Ÿ Python" + ``` python + # Creates a new column named "foo" which takes on values + # of column "A" incremented by 1 + df = df.with_column("foo", df["A"] + 1) + df.show() + ``` + + === "โš™๏ธ SQL" + ```python + # Creates a new column named "foo" which takes on values + # of column "A" incremented by 1 + df = daft.sql("SELECT *, A + 1 AS foo FROM df") + df.show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”† B โ”† C โ”† foo โ”‚ +โ”‚ --- โ”† --- โ”† --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Float64 โ”† Boolean โ”† Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 1 โ”† 1.5 โ”† true โ”† 2 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2 โ”† 2.5 โ”† true โ”† 3 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 3 โ”† 3.5 โ”† false โ”† 4 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 4 โ”† 4.5 โ”† false โ”† 5 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 4 of 4 rows) +``` + +Congratulations, you have just written your first **Expression**: `df["A"] + 1`! Expressions are a powerful way of describing computation on columns. For more details, check out the next section on [Expressions](expressions.md). + + + +## Selecting Rows + +We can limit the rows to the first ``N`` rows using [`df.limit(N)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.limit.html#daft.DataFrame.limit): + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({ + "A": [1, 2, 3, 4, 5], + "B": [6, 7, 8, 9, 10], + }) + + df.limit(3).show() + ``` + +``` {title="Output"} + ++---------+---------+ +| A | B | +| Int64 | Int64 | ++=========+=========+ +| 1 | 6 | ++---------+---------+ +| 2 | 7 | ++---------+---------+ +| 3 | 8 | ++---------+---------+ +(Showing first 3 rows) +``` + +We can also filter rows using [`df.where()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.where.html#daft.DataFrame.where), which takes an input a Logical Expression predicate: + +=== "๐Ÿ Python" + ``` python + df.where(df["A"] > 3).show() + ``` + +``` {title="Output"} + ++---------+---------+ +| A | B | +| Int64 | Int64 | ++=========+=========+ +| 4 | 9 | ++---------+---------+ +| 5 | 10 | ++---------+---------+ +(Showing first 2 rows) +``` + +## Selecting Columns + +Select specific columns in a DataFrame using [`df.select()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.select.html#daft.DataFrame.select), which also takes Expressions as an input. + +=== "๐Ÿ Python" + ``` python + import daft + + df = daft.from_pydict({"A": [1, 2, 3], "B": [4, 5, 6]}) + + df.select("A").show() + ``` + +``` {title="Output"} + ++---------+ +| A | +| Int64 | ++=========+ +| 1 | ++---------+ +| 2 | ++---------+ +| 3 | ++---------+ +(Showing first 3 rows) +``` + +A useful alias for [`df.select()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.select.html#daft.DataFrame.select) is indexing a DataFrame with a list of column names or Expressions: + +=== "๐Ÿ Python" + ``` python + df[["A", "B"]].show() + ``` + +``` {title="Output"} + ++---------+---------+ +| A | B | +| Int64 | Int64 | ++=========+=========+ +| 1 | 4 | ++---------+---------+ +| 2 | 5 | ++---------+---------+ +| 3 | 6 | ++---------+---------+ +(Showing first 3 rows) +``` + +Sometimes, it may be useful to exclude certain columns from a DataFrame. This can be done with [`df.exclude()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.exclude.html#daft.DataFrame.exclude): + +=== "๐Ÿ Python" + ``` python + df.exclude("A").show() + ``` + +```{title="Output"} + ++---------+ +| B | +| Int64 | ++=========+ +| 4 | ++---------+ +| 5 | ++---------+ +| 6 | ++---------+ +(Showing first 3 rows) +``` + +Adding a new column can be achieved with [`df.with_column()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.with_column.html#daft.DataFrame.with_column): + +=== "๐Ÿ Python" + ``` python + df.with_column("C", df["A"] + df["B"]).show() + ``` + +``` {title="Output"} + ++---------+---------+---------+ +| A | B | C | +| Int64 | Int64 | Int64 | ++=========+=========+=========+ +| 1 | 4 | 5 | ++---------+---------+---------+ +| 2 | 5 | 7 | ++---------+---------+---------+ +| 3 | 6 | 9 | ++---------+---------+---------+ +(Showing first 3 rows) +``` + +### Selecting Columns Using Wildcards + +We can select multiple columns at once using wildcards. The expression [`.col(*)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.col.html#daft.col) selects every column in a DataFrame, and you can operate on this expression in the same way as a single column: + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({"A": [1, 2, 3], "B": [4, 5, 6]}) + df.select(col("*") * 3).show() + ``` + +``` {title="Output"} +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”† B โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 3 โ”† 12 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 6 โ”† 15 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 9 โ”† 18 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ +``` + +We can also select multiple columns within structs using `col("struct.*")`: + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({ + "A": [ + {"B": 1, "C": 2}, + {"B": 3, "C": 4} + ] + }) + df.select(col("A.*")).show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ B โ”† C โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 1 โ”† 2 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 3 โ”† 4 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ +``` + +Under the hood, wildcards work by finding all of the columns that match, then copying the expression several times and replacing the wildcard. This means that there are some caveats: + +* Only one wildcard is allowed per expression tree. This means that `col("*") + col("*")` and similar expressions do not work. +* Be conscious about duplicated column names. Any code like `df.select(col("*"), col("*") + 3)` will not work because the wildcards expand into the same column names. + + For the same reason, `col("A") + col("*")` will not work because the name on the left-hand side is inherited, meaning all the output columns are named `A`, causing an error if there is more than one. + However, `col("*") + col("A")` will work fine. + +## Combining DataFrames + +Two DataFrames can be column-wise joined using [`df.join()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.join.html#daft.DataFrame.join). + +This requires a "join key", which can be supplied as the `on` argument if both DataFrames have the same name for their key columns, or the `left_on` and `right_on` argument if the key column has different names in each DataFrame. + +Daft also supports multi-column joins if you have a join key comprising of multiple columns! + +=== "๐Ÿ Python" + ``` python + df1 = daft.from_pydict({"A": [1, 2, 3], "B": [4, 5, 6]}) + df2 = daft.from_pydict({"A": [1, 2, 3], "C": [7, 8, 9]}) + + df1.join(df2, on="A").show() + ``` + +``` {title="Output"} + ++---------+---------+---------+ +| A | B | C | +| Int64 | Int64 | Int64 | ++=========+=========+=========+ +| 1 | 4 | 7 | ++---------+---------+---------+ +| 2 | 5 | 8 | ++---------+---------+---------+ +| 3 | 6 | 9 | ++---------+---------+---------+ +(Showing first 3 rows) +``` + +## Reordering Rows + +Rows in a DataFrame can be reordered based on some column using [`df.sort()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.sort.html#daft.DataFrame.sort). Daft also supports multi-column sorts for sorting on multiple columns at once. + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({ + "A": [1, 2, 3], + "B": [6, 7, 8], + }) + + df.sort("A", desc=True).show() + ``` + +```{title="Output"} + ++---------+---------+ +| A | B | +| Int64 | Int64 | ++=========+=========+ +| 3 | 8 | ++---------+---------+ +| 2 | 7 | ++---------+---------+ +| 1 | 6 | ++---------+---------+ +(Showing first 3 rows) +``` + +## Exploding Columns + +The [`df.explode()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.explode.html#daft.DataFrame.explode) method can be used to explode a column containing a list of values into multiple rows. All other rows will be **duplicated**. + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({ + "A": [1, 2, 3], + "B": [[1, 2, 3], [4, 5, 6], [7, 8, 9]], + }) + + df.explode("B").show() + ``` + +``` {title="Output"} + ++---------+---------+ +| A | B | +| Int64 | Int64 | ++=========+=========+ +| 1 | 1 | ++---------+---------+ +| 1 | 2 | ++---------+---------+ +| 1 | 3 | ++---------+---------+ +| 2 | 4 | ++---------+---------+ +| 2 | 5 | ++---------+---------+ +| 2 | 6 | ++---------+---------+ +| 3 | 7 | ++---------+---------+ +| 3 | 8 | ++---------+---------+ +(Showing first 8 rows) +``` + + + diff --git a/docs-v2/core_concepts/datatypes.md b/docs-v2/core_concepts/datatypes.md new file mode 100644 index 0000000000..f932623806 --- /dev/null +++ b/docs-v2/core_concepts/datatypes.md @@ -0,0 +1,96 @@ +# DataTypes + +All columns in a Daft DataFrame have a DataType (also often abbreviated as `dtype`). + +All elements of a column are of the same dtype, or they can be the special Null value (indicating a missing value). + +Daft provides simple DataTypes that are ubiquituous in many DataFrames such as numbers, strings and dates - all the way up to more complex types like tensors and images. + +!!! tip "Tip" + + For a full overview on all the DataTypes that Daft supports, see the [DataType API Reference](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html). + + +## Numeric DataTypes + +Numeric DataTypes allows Daft to represent numbers. These numbers can differ in terms of the number of bits used to represent them (8, 16, 32 or 64 bits) and the semantic meaning of those bits +(float vs integer vs unsigned integers). + +Examples: + +1. [`DataType.int8()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.int8): represents an 8-bit signed integer (-128 to 127) +2. [`DataType.float32()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.float32): represents a 32-bit float (a float number with about 7 decimal digits of precision) + +Columns/expressions with these datatypes can be operated on with many numeric expressions such as `+` and `*`. + +See also: [Numeric Expressions](https://www.getdaft.io/projects/docs/en/stable/user_guide/expressions.html#userguide-numeric-expressions) + +## Logical DataTypes + +The [`Boolean`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.bool) DataType represents values which are boolean values: `True`, `False` or `Null`. + +Columns/expressions with this dtype can be operated on using logical expressions such as ``&`` and [`.if_else()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.Expression.if_else.html#daft.Expression.if_else). + +See also: [Logical Expressions](https://www.getdaft.io/projects/docs/en/stable/user_guide/expressions.html#userguide-logical-expressions) + +## String Types + +Daft has string types, which represent a variable-length string of characters. + +As a convenience method, string types also support the `+` Expression, which has been overloaded to support concatenation of elements between two [`DataType.string()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.string) columns. + +1. [`DataType.string()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.string): represents a string of UTF-8 characters +2. [`DataType.binary()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.binary): represents a string of bytes + +See also: [String Expressions](https://www.getdaft.io/projects/docs/en/stable/user_guide/expressions.html#userguide-string-expressions) + +## Temporal + +Temporal dtypes represent data that have to do with time. + +Examples: + +1. [`DataType.date()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.date): represents a Date (year, month and day) +2. [`DataType.timestamp()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.timestamp): represents a Timestamp (particular instance in time) + +See also: [Temporal Expressions](https://www.getdaft.io/projects/docs/en/stable/api_docs/expressions.html#api-expressions-temporal) + +## Nested + +Nested DataTypes wrap other DataTypes, allowing you to compose types into complex data structures. + +Examples: + +1. [`DataType.list(child_dtype)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.list): represents a list where each element is of the child `dtype` +2. [`DataType.struct({"field_name": child_dtype})`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.struct): represents a structure that has children `dtype`s, each mapped to a field name + +## Python + +The [`DataType.python()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.python) dtype represent items that are Python objects. + +!!! warning "Warning" + + Daft does not impose any invariants about what *Python types* these objects are. To Daft, these are just generic Python objects! + +Python is AWESOME because it's so flexible, but it's also slow and memory inefficient! Thus we recommend: + +1. **Cast early!**: Casting your Python data into native Daft DataTypes if possible - this results in much more efficient downstream data serialization and computation. +2. **Use Python UDFs**: If there is no suitable Daft representation for your Python objects, use Python UDFs to process your Python data and extract the relevant data to be returned as native Daft DataTypes! + +!!! note "Note" + + If you work with Python classes for a generalizable use-case (e.g. documents, protobufs), it may be that these types are good candidates for "promotion" into a native Daft type! Please get in touch with the Daft team and we would love to work together on building your type into canonical Daft types. + +## Complex Types + +Daft supports many more interesting complex DataTypes, for example: + +* [`DataType.tensor()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.tensor): Multi-dimensional (potentially uniformly-shaped) tensors of data +* [`DataType.embedding()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.embedding): Lower-dimensional vector representation of data (e.g. words) +* [`DataType.image()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.image): NHWC images + +Daft abstracts away the in-memory representation of your data and provides kernels for many common operations on top of these data types. For supported image operations see the [image expressions API reference](https://www.getdaft.io/projects/docs/en/stable/api_docs/expressions.html#api-expressions-images). + +For more complex algorithms, you can also drop into a Python UDF to process this data using your custom Python libraries. + +Please add suggestions for new DataTypes to our [Github Discussions page](https://github.com/Eventual-Inc/Daft/discussions)! diff --git a/docs-v2/core_concepts/expressions.md b/docs-v2/core_concepts/expressions.md new file mode 100644 index 0000000000..81ba19bfc6 --- /dev/null +++ b/docs-v2/core_concepts/expressions.md @@ -0,0 +1,744 @@ +# Expressions + +Expressions are how you can express computations that should be run over columns of data. + +## Creating Expressions + +### Referring to a column in a DataFrame + +Most commonly you will be creating expressions by using the [`daft.col`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.col.html#daft.col) function. + +=== "๐Ÿ Python" + ``` python + # Refers to column "A" + daft.col("A") + ``` + +=== "โš™๏ธ SQL" + ```python + daft.sql_expr("A") + ``` + +``` {title="Output"} + +col(A) +``` + +The above code creates an Expression that refers to a column named `"A"`. + +### Using SQL + +Daft can also parse valid SQL as expressions. + +=== "โš™๏ธ SQL" + ```python + daft.sql_expr("A + 1") + ``` +``` {title="Output"} + +col(A) + lit(1) +``` + +The above code will create an expression representing "the column named 'x' incremented by 1". For many APIs, [`sql_expr`](https://www.getdaft.io/projects/docs/en/stable/api_docs/sql.html#daft.sql_expr) will actually be applied for you as syntactic sugar! + +### Literals + +You may find yourself needing to hardcode a "single value" oftentimes as an expression. Daft provides a [`lit()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.lit.html) helper to do so: + +=== "๐Ÿ Python" + ``` python + from daft import lit + + # Refers to an expression which always evaluates to 42 + lit(42) + ``` + +=== "โš™๏ธ SQL" + ```python + # Refers to an expression which always evaluates to 42 + daft.sql_expr("42") + ``` + +```{title="Output"} + +lit(42) +``` +This special :func:`~daft.expressions.lit` expression we just created evaluates always to the value ``42``. + +### Wildcard Expressions + +You can create expressions on multiple columns at once using a wildcard. The expression [`col("*")`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.col.html#daft.col)) selects every column in a DataFrame, and you can operate on this expression in the same way as a single column: + +=== "๐Ÿ Python" + ``` python + import daft + from daft import col + + df = daft.from_pydict({"A": [1, 2, 3], "B": [4, 5, 6]}) + df.select(col("*") * 3).show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”† B โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 3 โ”† 12 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 6 โ”† 15 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 9 โ”† 18 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ +``` + +Wildcards also work very well for accessing all members of a struct column: + +=== "๐Ÿ Python" + ``` python + + import daft + from daft import col + + df = daft.from_pydict({ + "person": [ + {"name": "Alice", "age": 30}, + {"name": "Bob", "age": 25}, + {"name": "Charlie", "age": 35} + ] + }) + + # Access all fields of the 'person' struct + df.select(col("person.*")).show() + ``` + +=== "โš™๏ธ SQL" + ```python + import daft + + df = daft.from_pydict({ + "person": [ + {"name": "Alice", "age": 30}, + {"name": "Bob", "age": 25}, + {"name": "Charlie", "age": 35} + ] + }) + + # Access all fields of the 'person' struct using SQL + daft.sql("SELECT person.* FROM df").show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ name โ”† age โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ String โ”† Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ Alice โ”† 30 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ Bob โ”† 25 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ Charlie โ”† 35 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ +``` + +In this example, we use the wildcard `*` to access all fields of the `person` struct column. This is equivalent to selecting each field individually (`person.name`, `person.age`), but is more concise and flexible, especially when dealing with structs that have many fields. + + + +## Composing Expressions + +### Numeric Expressions + +Since column "A" is an integer, we can run numeric computation such as addition, division and checking its value. Here are some examples where we create new columns using the results of such computations: + +=== "๐Ÿ Python" + ``` python + # Add 1 to each element in column "A" + df = df.with_column("A_add_one", df["A"] + 1) + + # Divide each element in column A by 2 + df = df.with_column("A_divide_two", df["A"] / 2.) + + # Check if each element in column A is more than 1 + df = df.with_column("A_gt_1", df["A"] > 1) + + df.collect() + ``` + +=== "โš™๏ธ SQL" + ```python + df = daft.sql(""" + SELECT + *, + A + 1 AS A_add_one, + A / 2.0 AS A_divide_two, + A > 1 AS A_gt_1 + FROM df + """) + df.collect() + ``` + +```{title="Output"} + ++---------+-------------+----------------+-----------+ +| A | A_add_one | A_divide_two | A_gt_1 | +| Int64 | Int64 | Float64 | Boolean | ++=========+=============+================+===========+ +| 1 | 2 | 0.5 | false | ++---------+-------------+----------------+-----------+ +| 2 | 3 | 1 | true | ++---------+-------------+----------------+-----------+ +| 3 | 4 | 1.5 | true | ++---------+-------------+----------------+-----------+ +(Showing first 3 of 3 rows) +``` + +Notice that the returned types of these operations are also well-typed according to their input types. For example, calling ``df["A"] > 1`` returns a column of type [`Boolean`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.bool). + +Both the [`Float`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.float32) and [`Int`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.int16) types are numeric types, and inherit many of the same arithmetic Expression operations. You may find the full list of numeric operations in the [Expressions API Reference](https://www.getdaft.io/projects/docs/en/stable/api_docs/expressions.html#numeric). + +### String Expressions + +Daft also lets you have columns of strings in a DataFrame. Let's take a look! + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({"B": ["foo", "bar", "baz"]}) + df.show() + ``` + +``` {title="Output"} + ++--------+ +| B | +| Utf8 | ++========+ +| foo | ++--------+ +| bar | ++--------+ +| baz | ++--------+ +(Showing first 3 rows) +``` + +Unlike the numeric types, the string type does not support arithmetic operations such as `*` and `/`. The one exception to this is the `+` operator, which is overridden to concatenate two string expressions as is commonly done in Python. Let's try that! + +=== "๐Ÿ Python" + ``` python + df = df.with_column("B2", df["B"] + "foo") + df.show() + ``` + +=== "โš™๏ธ SQL" + ```python + df = daft.sql("SELECT *, B + 'foo' AS B2 FROM df") + df.show() + ``` + +``` {title="Output"} + ++--------+--------+ +| B | B2 | +| Utf8 | Utf8 | ++========+========+ +| foo | foofoo | ++--------+--------+ +| bar | barfoo | ++--------+--------+ +| baz | bazfoo | ++--------+--------+ +(Showing first 3 rows) +``` + +There are also many string operators that are accessed through a separate [`.str.*`](https://www.getdaft.io/projects/docs/en/stable/api_docs/expressions.html#strings) "method namespace". + +For example, to check if each element in column "B" contains the substring "a", we can use the [`.str.contains`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.Expression.str.contains.html#daft.Expression.str.contains) method: + +=== "๐Ÿ Python" + ``` python + df = df.with_column("B2_contains_B", df["B2"].str.contains(df["B"])) + df.show() + ``` + +=== "โš™๏ธ SQL" + ```python + df = daft.sql("SELECT *, contains(B2, B) AS B2_contains_B FROM df") + df.show() + ``` + +``` {title="Output"} + ++--------+--------+-----------------+ +| B | B2 | B2_contains_B | +| Utf8 | Utf8 | Boolean | ++========+========+=================+ +| foo | foofoo | true | ++--------+--------+-----------------+ +| bar | barfoo | true | ++--------+--------+-----------------+ +| baz | bazfoo | true | ++--------+--------+-----------------+ +(Showing first 3 rows) +``` + +You may find a full list of string operations in the [Expressions API Reference](https://www.getdaft.io/projects/docs/en/stable/api_docs/expressions.html). + +### URL Expressions + +One special case of a String column you may find yourself working with is a column of URL strings. + +Daft provides the [`.url.*`](https://www.getdaft.io/projects/docs/en/stable/api_docs/expressions.html) method namespace with functionality for working with URL strings. For example, to download data from URLs: + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({ + "urls": [ + "https://www.google.com", + "s3://daft-public-data/open-images/validation-images/0001eeaf4aed83f9.jpg", + ], + }) + df = df.with_column("data", df["urls"].url.download()) + df.collect() + ``` + +=== "โš™๏ธ SQL" + ```python + df = daft.from_pydict({ + "urls": [ + "https://www.google.com", + "s3://daft-public-data/open-images/validation-images/0001eeaf4aed83f9.jpg", + ], + }) + df = daft.sql(""" + SELECT + urls, + url_download(urls) AS data + FROM df + """) + df.collect() + ``` + +``` {title="Output"} + ++----------------------+----------------------+ +| urls | data | +| Utf8 | Binary | ++======================+======================+ +| https://www.google.c | b' df["B"]).if_else(df["A"], df["B"]), + ) + + df.collect() + ``` + +=== "โš™๏ธ SQL" + ```python + df = daft.from_pydict({"A": [1, 2, 3], "B": [0, 2, 4]}) + + df = daft.sql(""" + SELECT + A, + B, + CASE + WHEN A > B THEN A + ELSE B + END AS A_if_bigger_else_B + FROM df + """) + + df.collect() + ``` + +```{title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”† B โ”† A_if_bigger_else_B โ”‚ +โ”‚ --- โ”† --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Int64 โ”† Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 1 โ”† 0 โ”† 1 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2 โ”† 2 โ”† 2 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 3 โ”† 4 โ”† 4 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 3 of 3 rows) +``` + +This is a useful expression for cleaning your data! + + +### Temporal Expressions + +Daft provides rich support for working with temporal data types like Timestamp and Duration. Let's explore some common temporal operations: + +#### Basic Temporal Operations + +You can perform arithmetic operations with timestamps and durations, such as adding a duration to a timestamp or calculating the duration between two timestamps: + +=== "๐Ÿ Python" + ``` python + import datetime + + df = daft.from_pydict({ + "timestamp": [ + datetime.datetime(2021, 1, 1, 0, 1, 1), + datetime.datetime(2021, 1, 1, 0, 1, 59), + datetime.datetime(2021, 1, 1, 0, 2, 0), + ] + }) + + # Add 10 seconds to each timestamp + df = df.with_column( + "plus_10_seconds", + df["timestamp"] + datetime.timedelta(seconds=10) + ) + + df.show() + ``` + +=== "โš™๏ธ SQL" + ```python + import datetime + + df = daft.from_pydict({ + "timestamp": [ + datetime.datetime(2021, 1, 1, 0, 1, 1), + datetime.datetime(2021, 1, 1, 0, 1, 59), + datetime.datetime(2021, 1, 1, 0, 2, 0), + ] + }) + + # Add 10 seconds to each timestamp and calculate duration between timestamps + df = daft.sql(""" + SELECT + timestamp, + timestamp + INTERVAL '10 seconds' as plus_10_seconds, + FROM df + """) + + df.show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ timestamp โ”† plus_10_seconds โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ Timestamp(Microseconds, None) โ”† Timestamp(Microseconds, None) โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 2021-01-01 00:01:01 โ”† 2021-01-01 00:01:11 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2021-01-01 00:01:59 โ”† 2021-01-01 00:02:09 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2021-01-01 00:02:00 โ”† 2021-01-01 00:02:10 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ +``` + +#### Temporal Component Extraction + +The [`.dt.*`](https://www.getdaft.io/projects/docs/en/stable/api_docs/expressions.html#temporal) method namespace provides extraction methods for the components of a timestamp, such as year, month, day, hour, minute, and second: + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({ + "timestamp": [ + datetime.datetime(2021, 1, 1, 0, 1, 1), + datetime.datetime(2021, 1, 1, 0, 1, 59), + datetime.datetime(2021, 1, 1, 0, 2, 0), + ] + }) + + # Extract year, month, day, hour, minute, and second from the timestamp + df = df.with_columns({ + "year": df["timestamp"].dt.year(), + "month": df["timestamp"].dt.month(), + "day": df["timestamp"].dt.day(), + "hour": df["timestamp"].dt.hour(), + "minute": df["timestamp"].dt.minute(), + "second": df["timestamp"].dt.second() + }) + + df.show() + ``` + +=== "โš™๏ธ SQL" + ```python + df = daft.from_pydict({ + "timestamp": [ + datetime.datetime(2021, 1, 1, 0, 1, 1), + datetime.datetime(2021, 1, 1, 0, 1, 59), + datetime.datetime(2021, 1, 1, 0, 2, 0), + ] + }) + + # Extract year, month, day, hour, minute, and second from the timestamp + df = daft.sql(""" + SELECT + timestamp, + year(timestamp) as year, + month(timestamp) as month, + day(timestamp) as day, + hour(timestamp) as hour, + minute(timestamp) as minute, + second(timestamp) as second + FROM df + """) + + df.show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ timestamp โ”† year โ”† month โ”† day โ”† hour โ”† minute โ”† second โ”‚ +โ”‚ --- โ”† --- โ”† --- โ”† --- โ”† --- โ”† --- โ”† --- โ”‚ +โ”‚ Timestamp(Microseconds, None) โ”† Int32 โ”† UInt32 โ”† UInt32 โ”† UInt32 โ”† UInt32 โ”† UInt32 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 2021-01-01 00:01:01 โ”† 2021 โ”† 1 โ”† 1 โ”† 0 โ”† 1 โ”† 1 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2021-01-01 00:01:59 โ”† 2021 โ”† 1 โ”† 1 โ”† 0 โ”† 1 โ”† 59 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2021-01-01 00:02:00 โ”† 2021 โ”† 1 โ”† 1 โ”† 0 โ”† 2 โ”† 0 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ +``` + +#### Time Zone Operations + +You can parse strings as timestamps with time zones and convert between different time zones: + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({ + "timestamp_str": [ + "2021-01-01 00:00:00.123 +0800", + "2021-01-02 12:30:00.456 +0800" + ] + }) + + # Parse the timestamp string with time zone and convert to New York time + df = df.with_column( + "ny_time", + df["timestamp_str"].str.to_datetime( + "%Y-%m-%d %H:%M:%S%.3f %z", + timezone="America/New_York" + ) + ) + + df.show() + ``` + +=== "โš™๏ธ SQL" + ```python + df = daft.from_pydict({ + "timestamp_str": [ + "2021-01-01 00:00:00.123 +0800", + "2021-01-02 12:30:00.456 +0800" + ] + }) + + # Parse the timestamp string with time zone and convert to New York time + df = daft.sql(""" + SELECT + timestamp_str, + to_datetime(timestamp_str, '%Y-%m-%d %H:%M:%S%.3f %z', 'America/New_York') as ny_time + FROM df + """) + + df.show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ timestamp_str โ”† ny_time โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ Utf8 โ”† Timestamp(Milliseconds, Some("America/New_York")) โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 2021-01-01 00:00:00.123 +0800 โ”† 2020-12-31 11:00:00.123 EST โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2021-01-02 12:30:00.456 +0800 โ”† 2021-01-01 23:30:00.456 EST โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ +``` + +#### Temporal Truncation + +The [`.dt.truncate()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.Expression.dt.truncate.html#daft.Expression.dt.truncate) method allows you to truncate timestamps to specific time units. This can be useful for grouping data by time periods. For example, to truncate timestamps to the nearest hour: + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({ + "timestamp": [ + datetime.datetime(2021, 1, 7, 0, 1, 1), + datetime.datetime(2021, 1, 8, 0, 1, 59), + datetime.datetime(2021, 1, 9, 0, 30, 0), + datetime.datetime(2021, 1, 10, 1, 59, 59), + ] + }) + + # Truncate timestamps to the nearest hour + df = df.with_column( + "hour_start", + df["timestamp"].dt.truncate("1 hour") + ) + + df.show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ timestamp โ”† hour_start โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ Timestamp(Microseconds, None) โ”† Timestamp(Microseconds, None) โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 2021-01-07 00:01:01 โ”† 2021-01-07 00:00:00 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2021-01-08 00:01:59 โ”† 2021-01-08 00:00:00 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2021-01-09 00:30:00 โ”† 2021-01-09 00:00:00 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2021-01-10 01:59:59 โ”† 2021-01-10 01:00:00 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ +``` diff --git a/docs-v2/core_concepts/read_write.md b/docs-v2/core_concepts/read_write.md new file mode 100644 index 0000000000..fa3aeef066 --- /dev/null +++ b/docs-v2/core_concepts/read_write.md @@ -0,0 +1,142 @@ +# Reading/Writing Data + +!!! failure "todo(docs): Should this page also have sql examples?" + +Daft can read data from a variety of sources, and write data to many destinations. + +## Reading Data + +### From Files + +DataFrames can be loaded from file(s) on some filesystem, commonly your local filesystem or a remote cloud object store such as AWS S3. Additionally, Daft can read data from a variety of container file formats, including CSV, line-delimited JSON and Parquet. + +Daft supports file paths to a single file, a directory of files, and wildcards. It also supports paths to remote object storage such as AWS S3. +=== "๐Ÿ Python" + ```python + import daft + + # You can read a single CSV file from your local filesystem + df = daft.read_csv("path/to/file.csv") + + # You can also read folders of CSV files, or include wildcards to select for patterns of file paths + df = daft.read_csv("path/to/*.csv") + + # Other formats such as parquet and line-delimited JSON are also supported + df = daft.read_parquet("path/to/*.parquet") + df = daft.read_json("path/to/*.json") + + # Remote filesystems such as AWS S3 are also supported, and can be specified with their protocols + df = daft.read_csv("s3://mybucket/path/to/*.csv") + ``` + +To learn more about each of these constructors, as well as the options that they support, consult the API documentation on [`creating DataFrames from files`](https://www.getdaft.io/projects/docs/en/stable/api_docs/creation.html#df-io-files). + +### From Data Catalogs + +If you use catalogs such as Apache Iceberg or Hive, you may wish to consult our user guide on integrations with Data Catalogs: [`Daft integration with Data Catalogs`](https://www.getdaft.io/projects/docs/en/stable/user_guide/integrations.html). + +### From File Paths + +Daft also provides an easy utility to create a DataFrame from globbing a path. You can use the [`daft.from_glob_path`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.from_glob_path.html#daft.from_glob_path) method which will read a DataFrame of globbed filepaths. + +=== "๐Ÿ Python" + ``` python + df = daft.from_glob_path("s3://mybucket/path/to/images/*.jpeg") + + # +----------+------+-----+ + # | name | size | ... | + # +----------+------+-----+ + # ... + ``` + +This is especially useful for reading things such as a folder of images or documents into Daft. A common pattern is to then download data from these files into your DataFrame as bytes, using the [`.url.download()`](https://getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.Expression.url.download.html#daft.Expression.url.download) method. + + +### From Memory + +For testing, or small datasets that fit in memory, you may also create DataFrames using Python lists and dictionaries. + +=== "๐Ÿ Python" + ``` python + # Create DataFrame using a dictionary of {column_name: list_of_values} + df = daft.from_pydict({"A": [1, 2, 3], "B": ["foo", "bar", "baz"]}) + + # Create DataFrame using a list of rows, where each row is a dictionary of {column_name: value} + df = daft.from_pylist([{"A": 1, "B": "foo"}, {"A": 2, "B": "bar"}, {"A": 3, "B": "baz"}]) + ``` + +To learn more, consult the API documentation on [`creating DataFrames from in-memory data structures`](https://www.getdaft.io/projects/docs/en/stable/api_docs/creation.html#df-io-in-memory). + +### From Databases + +Daft can also read data from a variety of databases, including PostgreSQL, MySQL, Trino, and SQLite using the [`daft.read_sql`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.read_sql.html#daft.read_sql) method. In order to partition the data, you can specify a partition column, which will allow Daft to read the data in parallel. + +=== "๐Ÿ Python" + ``` python + # Read from a PostgreSQL database + uri = "postgresql://user:password@host:port/database" + df = daft.read_sql("SELECT * FROM my_table", uri) + + # Read with a partition column + df = daft.read_sql("SELECT * FROM my_table", partition_col="date", uri) + ``` + +To learn more, consult the [`SQL User Guide`](https://www.getdaft.io/projects/docs/en/stable/user_guide/integrations/sql.html) or the API documentation on [`daft.read_sql`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.read_sql.html#daft.read_sql). + +## Reading a column of URLs + +Daft provides a convenient way to read data from a column of URLs using the [`.url.download()`](https://getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.Expression.url.download.html#daft.Expression.url.download) method. This is particularly useful when you have a DataFrame with a column containing URLs pointing to external resources that you want to fetch and incorporate into your DataFrame. + +Here's an example of how to use this feature: + +=== "๐Ÿ Python" + ```python + # Assume we have a DataFrame with a column named 'image_urls' + df = daft.from_pydict({ + "image_urls": [ + "https://example.com/image1.jpg", + "https://example.com/image2.jpg", + "https://example.com/image3.jpg" + ] + }) + + # Download the content from the URLs and create a new column 'image_data' + df = df.with_column("image_data", df["image_urls"].url.download()) + df.show() + ``` + +``` {title="Output"} + ++------------------------------------+------------------------------------+ +| image_urls | image_data | +| Utf8 | Binary | ++====================================+====================================+ +| https://example.com/image1.jpg | b'\xff\xd8\xff\xe0\x00\x10JFIF...' | ++------------------------------------+------------------------------------+ +| https://example.com/image2.jpg | b'\xff\xd8\xff\xe0\x00\x10JFIF...' | ++------------------------------------+------------------------------------+ +| https://example.com/image3.jpg | b'\xff\xd8\xff\xe0\x00\x10JFIF...' | ++------------------------------------+------------------------------------+ + +(Showing first 3 of 3 rows) +``` + +This approach allows you to efficiently download and process data from a large number of URLs in parallel, leveraging Daft's distributed computing capabilities. + +## Writing Data + +Writing data will execute your DataFrame and write the results out to the specified backend. The [`df.write_*(...)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/dataframe.html#df-write-data) methods are used to write DataFrames to files or other destinations. + +=== "๐Ÿ Python" + ``` python + # Write to various file formats in a local folder + df.write_csv("path/to/folder/") + df.write_parquet("path/to/folder/") + + # Write DataFrame to a remote filesystem such as AWS S3 + df.write_csv("s3://mybucket/path/") + ``` + +!!! note "Note" + + Because Daft is a distributed DataFrame library, by default it will produce multiple files (one per partition) at your specified destination. Writing your dataframe is a **blocking** operation that executes your DataFrame. It will return a new `DataFrame` that contains the filepaths to the written data. diff --git a/docs-v2/core_concepts/sql.md b/docs-v2/core_concepts/sql.md new file mode 100644 index 0000000000..55ebde486e --- /dev/null +++ b/docs-v2/core_concepts/sql.md @@ -0,0 +1,224 @@ +# SQL + +Daft supports Structured Query Language (SQL) as a way of constructing query plans (represented in Python as a [`daft.DataFrame`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.html#daft.DataFrame)) and expressions ([`daft.Expression`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.html#daft.DataFrame)). + +SQL is a human-readable way of constructing these query plans, and can often be more ergonomic than using DataFrames for writing queries. + +!!! tip "Daft's SQL support is new and is constantly being improved on!" + + Please give us feedback or submit an [issue](https://github.com/Eventual-Inc/Daft/issues) and we'd love to hear more about what you would like. + + +## Running SQL on DataFrames + +Daft's [`daft.sql`](https://www.getdaft.io/projects/docs/en/stable/api_docs/sql.html#daft.sql) function will automatically detect any [`daft.DataFrame`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.html#daft.DataFrame) objects in your current Python environment to let you query them easily by name. + +=== "โš™๏ธ SQL" + ```python + # Note the variable name `my_special_df` + my_special_df = daft.from_pydict({"A": [1, 2, 3], "B": [1, 2, 3]}) + + # Use the SQL table name "my_special_df" to refer to the above DataFrame! + sql_df = daft.sql("SELECT A, B FROM my_special_df") + + sql_df.show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”† B โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 1 โ”† 1 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 2 โ”† 2 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 3 โ”† 3 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 3 of 3 rows) +``` + +In the above example, we query the DataFrame called `"my_special_df"` by simply referring to it in the SQL command. This produces a new DataFrame `sql_df` which can natively integrate with the rest of your Daft query. + +## Reading data from SQL + +!!! warning "Warning" + + This feature is a WIP and will be coming soon! We will support reading common datasources directly from SQL: + + === "๐Ÿ Python" + + ```python + daft.sql("SELECT * FROM read_parquet('s3://...')") + daft.sql("SELECT * FROM read_delta_lake('s3://...')") + ``` + + Today, a workaround for this is to construct your dataframe in Python first and use it from SQL instead: + + === "๐Ÿ Python" + + ```python + df = daft.read_parquet("s3://...") + daft.sql("SELECT * FROM df") + ``` + + We appreciate your patience with us and hope to deliver this crucial feature soon! + +## SQL Expressions + +SQL has the concept of expressions as well. Here is an example of a simple addition expression, adding columns "a" and "b" in SQL to produce a new column C. + +We also present here the equivalent query for SQL and DataFrame. Notice how similar the concepts are! + +=== "โš™๏ธ SQL" + ```python + df = daft.from_pydict({"A": [1, 2, 3], "B": [1, 2, 3]}) + df = daft.sql("SELECT A + B as C FROM df") + df.show() + ``` + +=== "๐Ÿ Python" + ``` python + expr = (daft.col("A") + daft.col("B")).alias("C") + + df = daft.from_pydict({"A": [1, 2, 3], "B": [1, 2, 3]}) + df = df.select(expr) + df.show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ C โ”‚ +โ”‚ --- โ”‚ +โ”‚ Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 2 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 4 โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 6 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 3 of 3 rows) +``` + +In the above query, both the SQL version of the query and the DataFrame version of the query produce the same result. + +Under the hood, they run the same Expression `col("A") + col("B")`! + +One really cool trick you can do is to use the [`daft.sql_expr`](https://www.getdaft.io/projects/docs/en/stable/api_docs/sql.html#daft.sql_expr) function as a helper to easily create Expressions. The following are equivalent: + +=== "โš™๏ธ SQL" + ```python + sql_expr = daft.sql_expr("A + B as C") + print("SQL expression:", sql_expr) + ``` + +=== "๐Ÿ Python" + ``` python + py_expr = (daft.col("A") + daft.col("B")).alias("C") + print("Python expression:", py_expr) + ``` + +``` {title="Output"} + +SQL expression: col(A) + col(B) as C +Python expression: col(A) + col(B) as C +``` + +This means that you can pretty much use SQL anywhere you use Python expressions, making Daft extremely versatile at mixing workflows which leverage both SQL and Python. + +As an example, consider the filter query below and compare the two equivalent Python and SQL queries: + +=== "โš™๏ธ SQL" + ```python + df = daft.from_pydict({"A": [1, 2, 3], "B": [1, 2, 3]}) + + # Daft automatically converts this string using `daft.sql_expr` + df = df.where("A < 2") + + df.show() + ``` + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({"A": [1, 2, 3], "B": [1, 2, 3]}) + + # Using Daft's Python Expression API + df = df.where(df["A"] < 2) + + df.show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ A โ”† B โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Int64 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 1 โ”† 1 โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 1 of 1 rows) +``` + +Pretty sweet! Of course, this support for running Expressions on your columns extends well beyond arithmetic as we'll see in the next section on SQL Functions. + +## SQL Functions + +SQL also has access to all of Daft's powerful [`daft.Expression`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.html#daft.DataFrame) functionality through SQL functions. + +However, unlike the Python Expression API which encourages method-chaining (e.g. `col("a").url.download().image.decode()`), in SQL you have to do function nesting instead (e.g. `"image_decode(url_download(a))"`). + +!!! note "Note" + + A full catalog of the available SQL Functions in Daft is available in the [`../api_docs/sql`](https://www.getdaft.io/projects/docs/en/stable/api_docs/sql.html). + + Note that it closely mirrors the Python API, with some function naming differences vs the available Python methods. + We also have some aliased functions for ANSI SQL-compliance or familiarity to users coming from other common SQL dialects such as PostgreSQL and SparkSQL to easily find their functionality. + +Here is an example of an equivalent function call in SQL vs Python: + +=== "โš™๏ธ SQL" + ```python + df = daft.from_pydict({"urls": [ + "https://user-images.githubusercontent.com/17691182/190476440-28f29e87-8e3b-41c4-9c28-e112e595f558.png", + "https://user-images.githubusercontent.com/17691182/190476440-28f29e87-8e3b-41c4-9c28-e112e595f558.png", + "https://user-images.githubusercontent.com/17691182/190476440-28f29e87-8e3b-41c4-9c28-e112e595f558.png", + ]}) + df = daft.sql("SELECT image_decode(url_download(urls)) FROM df") + df.show() + ``` + +=== "๐Ÿ Python" + ``` python + df = daft.from_pydict({"urls": [ + "https://user-images.githubusercontent.com/17691182/190476440-28f29e87-8e3b-41c4-9c28-e112e595f558.png", + "https://user-images.githubusercontent.com/17691182/190476440-28f29e87-8e3b-41c4-9c28-e112e595f558.png", + "https://user-images.githubusercontent.com/17691182/190476440-28f29e87-8e3b-41c4-9c28-e112e595f558.png", + ]}) + df = df.select(daft.col("urls").url.download().image.decode()) + df.show() + ``` + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ urls โ”‚ +โ”‚ --- โ”‚ +โ”‚ Image[MIXED] โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 3 of 3 rows) +``` diff --git a/docs-v2/core_concepts/udf.md b/docs-v2/core_concepts/udf.md new file mode 100644 index 0000000000..a57913a05f --- /dev/null +++ b/docs-v2/core_concepts/udf.md @@ -0,0 +1,213 @@ +# User-Defined Functions (UDF) + +A key piece of functionality in Daft is the ability to flexibly define custom functions that can run on any data in your dataframe. This guide walks you through the different types of UDFs that Daft allows you to run. + +Let's first create a dataframe that will be used as a running example throughout this tutorial! + +=== "๐Ÿ Python" + ``` python + import daft + import numpy as np + + df = daft.from_pydict({ + # the `image` column contains images represented as 2D numpy arrays + "image": [np.ones((128, 128)) for i in range(16)], + # the `crop` column contains a box to crop from our image, represented as a list of integers: [x1, x2, y1, y2] + "crop": [[0, 1, 0, 1] for i in range(16)], + }) + ``` + + +## Per-column per-row functions using [`.apply`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.Expression.apply.html) + +You can use [`.apply`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.Expression.apply.html) to run a Python function on every row in a column. + +For example, the following example creates a new `flattened_image` column by calling `.flatten()` on every object in the `image` column. + +=== "๐Ÿ Python" + ``` python + df.with_column( + "flattened_image", + df["image"].apply(lambda img: img.flatten(), return_dtype=daft.DataType.python()) + ).show(2) + ``` + +``` {title="Output"} + ++----------------------+---------------+---------------------+ +| image | crop | flattened_image | +| Python | List[Int64] | Python | ++======================+===============+=====================+ +| [[1. 1. 1. ... 1. 1. | [0, 1, 0, 1] | [1. 1. 1. ... 1. 1. | +| 1.] [1. 1. 1. ... | | 1.] | +| 1. 1. 1.] [1. 1.... | | | ++----------------------+---------------+---------------------+ +| [[1. 1. 1. ... 1. 1. | [0, 1, 0, 1] | [1. 1. 1. ... 1. 1. | +| 1.] [1. 1. 1. ... | | 1.] | +| 1. 1. 1.] [1. 1.... | | | ++----------------------+---------------+---------------------+ +(Showing first 2 rows) +``` + +Note here that we use the `return_dtype` keyword argument to specify that our returned column type is a Python column! + +## Multi-column per-partition functions using [`@udf`](https://www.getdaft.io/projects/docs/en/stable/api_docs/udf.html#creating-udfs) + +[`.apply`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.Expression.apply.html) is great for convenience, but has two main limitations: + +1. It can only run on single columns +2. It can only run on single items at a time + +Daft provides the [`@udf`](https://www.getdaft.io/projects/docs/en/stable/api_docs/udf.html#creating-udfs) decorator for defining your own UDFs that process multiple columns or multiple rows at a time. + +For example, let's try writing a function that will crop all our images in the `image` column by its corresponding value in the `crop` column: + +=== "๐Ÿ Python" + ``` python + @daft.udf(return_dtype=daft.DataType.python()) + def crop_images(images, crops, padding=0): + cropped = [] + for img, crop in zip(images.to_pylist(), crops.to_pylist()): + x1, x2, y1, y2 = crop + cropped_img = img[x1:x2 + padding, y1:y2 + padding] + cropped.append(cropped_img) + return cropped + + df = df.with_column( + "cropped", + crop_images(df["image"], df["crop"], padding=1), + ) + df.show(2) + ``` + +``` {title="Output"} + ++----------------------+---------------+--------------------+ +| image | crop | cropped | +| Python | List[Int64] | Python | ++======================+===============+====================+ +| [[1. 1. 1. ... 1. 1. | [0, 1, 0, 1] | [[1. 1.] [1. 1.]] | +| 1.] [1. 1. 1. ... | | | +| 1. 1. 1.] [1. 1.... | | | ++----------------------+---------------+--------------------+ +| [[1. 1. 1. ... 1. 1. | [0, 1, 0, 1] | [[1. 1.] [1. 1.]] | +| 1.] [1. 1. 1. ... | | | +| 1. 1. 1.] [1. 1.... | | | ++----------------------+---------------+--------------------+ +(Showing first 2 rows) +``` + +There's a few things happening here, let's break it down: + +1. `crop_images` is a normal Python function. It takes as input: + + a. A list of images: `images` + + b. A list of cropping boxes: `crops` + + c. An integer indicating how much padding to apply to the right and bottom of the cropping: `padding` + +2. To allow Daft to pass column data into the `images` and `crops` arguments, we decorate the function with [`@udf`](https://www.getdaft.io/projects/docs/en/stable/api_docs/udf.html#creating-udfs) + + a. `return_dtype` defines the returned data type. In this case, we return a column containing Python objects of numpy arrays + + b. At runtime, because we call the UDF on the `image` and `crop` columns, the UDF will receive a [`daft.Series`](https://www.getdaft.io/projects/docs/en/stable/api_docs/series.html#daft.Series) object for each argument. + +3. We can create a new column in our DataFrame by applying our UDF on the `"image"` and `"crop"` columns inside of a [`df.with_column()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.with_column.html#daft.DataFrame.with_column) call. + +### UDF Inputs + + +When you specify an Expression as an input to a UDF, Daft will calculate the result of that Expression and pass it into your function as a [`daft.Series`](https://www.getdaft.io/projects/docs/en/stable/api_docs/series.html#daft.Series) object. + +The Daft [`daft.Series`](https://www.getdaft.io/projects/docs/en/stable/api_docs/series.html#daft.Series) is just an abstraction on a "column" of data! You can obtain several different data representations from a [`daft.Series`](https://www.getdaft.io/projects/docs/en/stable/api_docs/series.html#daft.Series): + +1. PyArrow Arrays (`pa.Array`): [`s.to_arrow()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/series.html#daft.Series.to_arrow) +2. Python lists (`list`): [`s.to_pylist()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/series.html#daft.Series.to_pylist) + +Depending on your application, you may choose a different data representation that is more performant or more convenient! + +!!! info "Info" + + Certain array formats have some restrictions around the type of data that they can handle: + + 1. **Null Handling**: In Pandas and Numpy, nulls are represented as NaNs for numeric types, and Nones for non-numeric types. Additionally, the existence of nulls will trigger a type casting from integer to float arrays. If null handling is important to your use-case, we recommend using one of the other available options. + + 2. **Python Objects**: PyArrow array formats cannot support Python columns. + + We recommend using Python lists if performance is not a major consideration, and using the arrow-native formats such as PyArrow arrays and numpy arrays if performance is important. + +### Return Types + +The `return_dtype` argument specifies what type of column your UDF will return. Types can be specified using the [`daft.DataType`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType) class. + +Your UDF function itself needs to return a batch of columnar data, and can do so as any one of the following array types: + +1. Numpy Arrays (`np.ndarray`) +2. PyArrow Arrays (`pa.Array`) +3. Python lists (`list`) + +Note that if the data you have returned is not castable to the return_dtype that you specify (e.g. if you return a list of floats when you've specified a `return_dtype=DataType.bool()`), Daft will throw a runtime error! + +## Class UDFs + +UDFs can also be created on Classes, which allow for initialization on some expensive state that can be shared between invocations of the class, for example downloading data or creating a model. + +=== "๐Ÿ Python" + ``` python + @daft.udf(return_dtype=daft.DataType.int64()) + class RunModel: + + def __init__(self): + # Perform expensive initializations + self._model = create_model() + + def __call__(self, features_col): + return self._model(features_col) + ``` + +Running Class UDFs are exactly the same as running their functional cousins. + +=== "๐Ÿ Python" + ``` python + df = df.with_column("image_classifications", RunModel(df["images"])) + ``` + +## Resource Requests + +Sometimes, you may want to request for specific resources for your UDF. For example, some UDFs need one GPU to run as they will load a model onto the GPU. + +To do so, you can create your UDF and assign it a resource request: + +=== "๐Ÿ Python" + ``` python + @daft.udf(return_dtype=daft.DataType.int64(), num_gpus=1) + class RunModelWithOneGPU: + + def __init__(self): + # Perform expensive initializations + self._model = create_model() + + def __call__(self, features_col): + return self._model(features_col) + ``` + + ``` python + df = df.with_column( + "image_classifications", + RunModelWithOneGPU(df["images"]), + ) + ``` + +In the above example, if Daft ran on a Ray cluster consisting of 8 GPUs and 64 CPUs, Daft would be able to run 8 replicas of your UDF in parallel, thus massively increasing the throughput of your UDF! + +UDFs can also be parametrized with new resource requests after being initialized. + +=== "๐Ÿ Python" + ``` python + RunModelWithTwoGPUs = RunModelWithOneGPU.override_options(num_gpus=2) + df = df.with_column( + "image_classifications", + RunModelWithTwoGPUs(df["images"]), + ) + ``` diff --git a/docs-v2/img/architecture.png b/docs-v2/img/architecture.png new file mode 100644 index 0000000000000000000000000000000000000000..f5133b2736e5b2b365e342bd9ae4ec86c860913e GIT binary patch literal 32822 zcmdqIc|6qL`#+4{RLW9Fk*JiSsEClAP?n;|UdkR*jAaZ)R6;#4v*~Gr!mL`F_63{rLWK-+$bX$9?s-(0xt?bduiKc5 z2+Iib@$rdRTA14L@$FFH;}f{OO9=Q+PFUPSK0X-R($wf?g!|Ie&Mb*h&bO6Q3#hyN z4`8}u0bii&?u)lRVO1331P+{ecq`}KZL(Q++->qMPig)=XRguCc=5kKuK5qY*$tzv ze$_pFFoLB`7~{V5-8WX=Oco@c`M57nG2{Ngz^2#LWBY(1|9|_pK|d1l>#_ZE2baSg zMS5_X@Ifts*HQkXCGIhsa~}C21KI|+%I&;5)6I`$gM$Bj*$E1zO=sOKyM8wp#&Pn2 zqgIGq4pXfe_EP3z&B9of?nXXznSyJ4G_%tM3NrJv{SCG%I^Kufz9~>TI16M%*VSXrGfZ*n*#mlyx*M(x-;KihP*xF^y>T1 zH!5&q4D0N^DaN29bjp^AoguHRHAY71rRLKLxZCqctXue0kxuBs-bZ^fNL4Nk`Z#w7 z-q1%wXl=~=h*HES(>v z2WZz_cCNO!(LOQ%Og1QscEG|-7fk+~zaQvN;h(S;)^ZE2Wx09L`$+iYBlHfj%R^0^ ztwnfge&o;W%aEGd81@z-PC>3{Zg|yC8`bK(446MZ1;$txXawTwejqF}7#}@O0iDJ2 zjUYW>G$>$VdaqhX6#8hw;hMP*wFAMpI=>mK0)0Kdck(5^o9jAjk)nEQLseMUkp7x0 zFJ(}N&fPJDu=IE{{FpoyPK=iJfKpKym0J~c>tFY%ft$gUX?XlyvDs@TYt3t0YdY(Z zmA5K28*WuPU?*t1d1H56uX08)${mq}TAzYwMsugAZl52z$gKRVPExWv(!If*+reei z)qP_C8o1BWYXXYM9P2*sBLb0wO7qDo?au_7Skpc zX)x(|e^D-cGX3{~V+rFlIo^iHn^!|J?^9F7HmZKYLx@kUN7aubSbQyc0JZP#}PTAo@BML$%4d74# z8A8K|Sv0Ghi2`C5pQGV7JQYLx+qBrdEpjK%AM{iMPK3|T*QYJxeO9q%G$dhO7Sy_O z&BksgAD>~ieMby?QN^)3&%PxraGpi$u$4KN(V-t%upUzY9=3!43sVsQI|*27SbnqA z2Hv9r3}0nipl=y}zh<$R*#-=3Q|sN|j4KcU8eOlL4DLP47S&z)7?`1CYa??oW5kTc z{hc7c3r2!p#Tmt7&03@%&sEG8KyTawxa!cB=KZ9_GvX zM?PA0D+)*exq+}Spn+l8$Oos8cdFd2b*zVyNR;K4X+f%fsgQ`wi;o5-Km!Fye{A&> z=TD~?vECP5gy2`R>>!8eOg}#k4*fu2`0zp$focw2%y%g?epUzt4T$#`TZ3q%y{%rTNv5jXE1^b3cJW}<)i=u8J9dg*{?2*8Ta-E=+hDC)w-zel zFJ-k(iUe71nHS{hcK{I^+RmQfdB*R3TdqKykikpiE6S`|Akg^imKQ3K1C3k`D?_Lnv1VET0wT+MynvTC(t>-AK$o6b zW}G}B;%rGt)F(>VUOw*+V*u|AE94r1bQ|c>5feT9?&8#inC%Fk+3)ECdDz<+Snq z#CJO6+aaGbyj1y0sFHF}IL$yhdH_i41b-+mxA57O1ke2>QYM9=A9GSp`|+=iI(lfY zFJhUhW|fe+-pud{N8~b)Qrm{2rPbV*CL#0i0p1Ql*|S`NnodxnUa25khqXyr_wuAY zqR>utzUd(|FST@b9)4&zT!U~~(=LeFqAcz~^5Wl6Sng5RzK(%hEj2&haF+b5GpPnT zN_h|Cv;v9I%yr>}HYP#$#`6zuEn@*%-hBsYihg|cCqQTAad#Q6*ZvzAE8sOh~_#*_^w3hLsU@J@Ev~U7M>Bh}0 zY}G>c&~KLBsTE8F$Cr_u)vh)$n8pK3*~FN_ihqSnakg>iHM8z2~K{u-=1RGUR@ zDB!`N4)CG1^ILGTJNL^@`PNbH<_=(}(sR&=={Y#cl)XF0r#5 zeEYR1d~0JS6>bzSq>c4}MHlMHlJu2TtD8tU#`FGVuRDUAZ|@P09=KI;bMq^6g|~j?bE4JIXLFfDQc_~gQ9+9ChU41n zBld7m`{qe>yZ>lKBE%;jQW9>H(9~KjTIt*n1jpK#_3`+cHI@b|gEvJXr-?KB|2hKL z9a`U6zj)@yx~4f6)^ZU(@WXYv5fDf!u_@5rTUR zq*dFP8%m*>_2lr&Jl1y;F^+cMhoZ|TNioR0K#pFQRoM9-NyhGt@-1g|zhMNzNaeI1 zdLBBPZqB*?uk7{njhY81`BAtMy3yw`IJZd_T=(Z3kbEdRxyf#E=~Xv=lgDk&tBj)B7@E# z!niMcGu3QmhF<+KEg%Bi`E~Iih7-$MqPiNukO|!TBK*v8hJDLIYn|EYQSoyiRxZ`*GN99(DMf$)gTE^04h^ z08kdkv%dg#@ZB9{lZ#W``{y7Sm95J*msh$901l1Ca6SVsV?Qo}KlAV0J_qA$pJf%QVdEc+4Ozd*`*ko4Y0JgYz7__uP?^h@wZ8?EE7nx z9_|sppN=vjEDWFss_fRnNykD^9J0_>AyTp@h?w9Hclsv z2CX#Qq;;?#0}c7!Pw59m(i(^{MBBVeD~{05R-9(5>S<-6R-~KinEevue0#ksIr&^d=48xqIs4XU~ zBM88jQTJ5r3{8zUYOh%fwH_Y_Ko6%DAYyD!v5Ft^K8&Ra?b5v9pR2sF3JJC>5#C&` z-`3W8Ic0pG3UCT-*tZvl3?||qygJ-)2fv1du@}`F{%jc0Jhr3%0os0y2iJi2G>C6< z)-bjO1#jN07weHvc4!AZX5ax&489Ng{mFsmxE;;2^|~>%#gBC07*Q^CNP8WAmIBz{d@Er`!wxr2eSD zcSyjZBjJ&4yitZf2L*9vNDwWZ(=06D&zPWiA(4_)nTrdH0EbW#S;HK4uKskdry2sB zf#+{lyF&8cy7bD!GH zJIOlC=%dPvz6W8b(4^x5x~O{+K(~l1@2|dj>)TWpwK6xd>i?HtKbhnYQhnJKbG$eq z_jFKa)Wx5;vM#b@f z@+mO%$Ey=ARyAwFO2vt^A2Nf0QGW;oCIj&nK3$fb9%6H9vkcnsdMGc;pcOc zt+W>nM)S<;O(uQ^p*|Ww7Xz@XX?wWW*NgoY!89XHCu$G06iDLQ%%13D~qE{Ks z*`kqFpoTB*Va8uyyJK{>A0V*+V=wV>XmNoKlIy|@86gI#u8bJM5=cpR&)9w40&e5; zG`GB`1FfU#c^~HP#}r3z{ALka^IbgpmsdMKPpEM^-F@PX`HjCG5fgJ?WKDZT}#>$T~iP>=3Z+5^xwT0uH{Vp{;5R*<2^xHyDXXHh1<=hpz9qH z|2b6!A@6Xu?^?}KOWBjgU(*1WvM29F=I&ft*O5A9XfbZWFTl5}tFpHF!&vHG+}}2U z>+3@c2r)0(+)D)nckPCrGUVfXOjfU@mpmo*+u+Xa+yAW3MIA#q-!>%-ZL<*+7T?df z9t+eM(IJs|z?;7o92N7>&3L^^xAy^+NGK6%d;H?J8JY#)g12I>t9yP|Y zWFCjoU^PP$ESH)WY~Sq}VgsLn`w)W`qZplW2-dQ11;Mc!`hlw#?K0dd*Drf7 zQt50iJfV3i18lH5Z^Hkt2`Wg%gcloV?pQjcZ(!~jNZ<`=nLb`~`d$^}S&|B-r0h(1 zNS@zPj!c0*QLD;ow>^I_qpIhUG%>n4Gi33WszxL1&qSmpa|*pZD)=1|p7;8;4Ggy) zkpDkN`RW>bc-}hz@h)wAnq&cszXapIaK`QW)g3E59)iNMZNI6v10m2_&`^YS#N?wbpzy$_I%NbrkZ?O#Jck~;VfU6^}`*u?kx z5C_Bc-LzHwn)5*ntdk`sIhi3q(RX%$<{$C-QZZqPVWXp_V80idEa}7A|LOAIgOUoY zmZl&5Zb20-+GT&1p0FofyMv5St17CDhV;i zb9H=1bt>)*c2Sigp-1~5$0ZKjKS|mQab)=uqD087z0fb^grW*MQzqdg$z$x-nNqCz zptN&}+*w1Rl!f0Y?OM`4_unhG2Y2QvH*6PRAis(U+t{A9;2_A_RZu&ml{*>Tj9H&z z4AjUUIC*4#q5&645AuyHRHr9f4(P%CbxKDs_2rf@O$Lj@)WjgTra#s8t&~U+>lVTr za3hXo?;c)pu|s1es??!!_H5_J?D$u24dnhSTCO_Bj)d2q*Pl{cC@IbjUeUxA58?Vx zR>Je`G5tq&ZtCbGEfvGuujJ=?MAMU3%@o55-lmUl%4~6`jn8`4JI3ZwbFr87tI7ms ze{9s1^1Ss?TH$A>nwCW!nveof@zK8JFPDAlz(RdHp{W`TNz@brmoHAyQian}OFA+R zrg;!&1!#{{+n6s*TQY?VvTr|oC8?k+)E9j-*2-Pu=Oz?o(u!9uS&MSqrWGV=Qau9G zBM#fg-VF69fMr|V9I2z=&-gX3`hUBY*aPvh$m|Mn4<0-;5NPYVm2IJs8oFIUQKv1i@OXXwiB~PY-^>C!RJu zxj_OU`I;2${*3oDYOBY%X!l@kfQh@#=Y~V0WCSfvpE9h={91?37qec}EVG~Hp2V>V zH314!mY&_-X7X_@PeMAyYI4jy8K2)c5CHoEFI8k*1%fR=7QQ(00Ag0ydZ8{EJ0Pjx zEMx{%1rtAPtd@x%(q?vc(J3o5nq5zetyF`r6IoP7V@gut=R{~qh`a2>RCdk%4u^Iu8#=c-B(N`} zB;b_czcL`g7^unYoN0hr)jijq=yzJ-792>F+UZ}aCmEw`s^K>O;IM3gx3KtJu|@E{ zyNeQusQy)J)){@O7g1wHT0uv0sWGCCIfK~%OW7uT%aL}99eL3;AluZfRW+h4BKTa0 zSknK_5-F8a+9oV-1?|P-)3u?#MR0B|n<}M{Bsw9Z3(8_%QkJa4e!iTdZ9bD)9q09y zH%)t_LH?R+lJMh4boJJG`;|YWas5P=ulC3{kJ!E6qfR{XkW%yBf>VQ(*BWuM2+F#% z2W=`Q_}4u`AeHiQ&?tMh9b~2#qw`GQN!qKY0{<8)Z|_eOsYmTBKBZ%IHNm00zjDP- z=0z?)J3?!H^00V;PhF~{c<43*K?p%dxX=ngThj6RR)O11bYKTbN7cF zd@FpsR{F}epv5Ef(Yse}2wG&@$q1*F5+aVFz1NiFp%}1nvo`Zf7tH->(t!}}w6Hi_ zrLgTJw=5d`DYdw5m;a)gVLo~E5=G1K%0MD@^pCW{3`xW`@~rY1&aKivQtjJ=ndy3$ z&;wTV0@rGjp*}Em{f|NYfERkrMZN{sGWDw*N1$RV`LRjq0 zb5N0z3H51FU>d^(Nwyg6BkX956xC?xu0u~Z-WQap_gzgH*@e#)=+Z~GokATGWMVr&@KiD39)&;umj<`3)SxRN8nX=&05%KO3JGf0aU+O~dw4L`YNdw3e*X$ajfWDDuk z{f)momLjP@b$=;W^IiafXTe*0(DxP(@1tKblTy$P7Jcps1W2hVCiM0Q22{724;n)7 z59}Bmxi;zJBb;_b{<1xJCE#*YwidA5%&_+EmZ}iLC6jN$a{8KxhetDl{td)w!XPux z_(1AhnQwo_R(jl*$d_Mhk11X8RD!D|O4d0^B)Y9vWP);pGa22c^LM@jJLypRtnt@@ zA=AZau`kLCuiA|H_pb^dZqdFaZt5Iz!^r4I2RwMk-njS_wH&btmj5!? z4Sk6CQ+TbX#Yw85)FhvgXltOkUv~wMv_brO1deNjsddAmsk7&_xQUXkXRn3T@u49y zo9Val4aYxSV{R1_8AGj;7GF+;G=%Suypg1A*ti!Y)9n#U?(&rzK!9fPjOenVH&x(r zXS%H1)(vB8%+cEu9=7xnimOP-YUAqd)SdeW?+(1&uw{2ZQ;Cm0A1C}`tv=)aDZ$1p zRdl_F_Vml=pSmq+|9x9Z@Hs=%cWeFSpsl2ojuU6=Vw9PX^tLNOlHd6LHO)dnY%)%(fP4kPB7OX2QV}`Mt74u%(K)Q@>nd^3;C-NL$#G4gbJlyF4!atWemJK|f*KrKjm@9H{UX6tDP z+oL1>QRMc*4bqe4jv_wv+@t=>atyD5yXZv>C@X>+p1cpm21c2qt{`9=L= ze_fAVLIC9NKy^+AIJho`WenX@S8B3jt0-oZRrM*lRi1*`6lr!{Hkn#pX$R^vde0d7 zS@jv2FIJA&nk6&hG@>f%!=;qn1RQs;E13Bdk?*%!$rqY%vcrYk&H=M<_<}|xBaC() zynePes?)nYv+nXqnhm+Y*GHa#Slj+H(S>YT6#qEZmK$M~HDH9U@l#8~QEJ*rkG4Bp ztL42lFf5;TSMsOOM0e{3v5OdYxi%7g0sc!5NlbwOtwm#-rA3Xnt2L-hNj=vrRp_d`w5&Y>ElNv z^*}oUnhQi>cCvF00r^C^^tiAsR2Vh%d;`^_Cu!AGam)c4u~)V8n#FTtemimd*t!lz zEW!l&bK&T1#KzwvkkQ{WW?Ed90HPDM^F%-0hp=-{Cz@t}rLaCj2AUy)^JcTioEg1m zr0DINavzouPXbxgB~?uIkh+R;2sB3 zgY_3&@emaHY2qe?=&g$(`cn7$uFB6(xTJ)+H`6tkO+g_s8{pf+!PP$_u`c=23wgg# z(QTVv1EnS{mo_WsPvXehbuZMWR>G0i^30~v)~$~cZ0~1U+!iMjPvmHSgA+|MavQ0% z_aOmf-w)ZW#cNYSJ#$t$m(CL-wqO zwGvd5r-#papAkVrgnZFDb!s_h$173FLKy4Ez_&rAlSf)GB2sM>iO7qg&680GCX8JP z(Xr&b+451;dp69dVCR{2wwyRJpZqBrK9)&?kJubyPe;4w7D&GY=eJUZ5Sv#!cB=is zNrvwl%aWEoIn^^;tOT;IW2r)YJ-5akm&5bRntkk86lBBII_zToyh<~1xNkWzj3WNu zP3J9n^EN0EL}mYAz`38iNi6rmLWa|hf>85&Q`htYT<%GtzlA`k-i`@z>S%Y=umU08!=Veuz1h2@0a9bztTR~SC_IXY5I=b{{`ZH^-O!-8LI`P3GY8?R8V|5MpzvWiJ)72(3PF-1x z@^9J1PY?ozG^mHrb;S0pS*bAP)-{7<~2rQ!WgB4US1+X-KKMXIXJGxr3c(2zRp7gfWX<5?0|RKwSpJgp-saC}jS_RSa6 z2mI5i)miT=6SMVHaV?$=yD$8$zFm(^`BO2N53YXwiLCBDMA!4+C=fqf8JH~+)gm%7Rt1r)>QtdMY$pISmbvCn>Gxbn-y zG9UYK*Gxe6+ZPc{L-)wn&rW${_Ki|+1bxz=qIz1dv2rwZj3%LSx0q|9*qS}!s2>rG zfi4aBnf3{{FsYNxId8OW%`)e2;o+wZ!Ee;$j2Ui>G-E1QKM_X>?&MI604nldE}TuZ zF?L+bJ>(xS;1N-QyG!`&gwY^l>wNr=z%OazB^7eUNI#?7^iGtdojdQCXP2a6_}Z&D z@`P$tWe*0F)VlfoO=9IEO1S>bT_3Wxyr)0iQY`;71UXJRSJ_t`4(%`~GQQRjdV8$J zcJf!Zq#5bwO@IYQ2D*VN^Rft(XTpRxO?`n>D*A{NYARdcesD(uAbd}QFLb!*5i37d@Ww z4pK4Ow$O9iy$Iv4?f$PtF*SI~>Q|YOT$LM?1i?^F0@oA_a+>ne%%q>K;JX2iYf_b8 zG6*+?8r=Mc-Fhy}!^=lloNAdpiG1oPzL{z6c*y>dXNUCvbfBr=lj!enSltKz(ywM6 z<#z`i>_KEq8QEfZq0vGyahhvar0#~Br7yP00QJF=;R`|-RP@>3)8#(Y&4)`zI( z_TQ^^VS!5mPhRK$cbIpR6;2zL{ci|S696!f$iGF^|4XfptldVf_cAzRB{KrlPkpDT zK;1kUsE)?zM&N6_)+Z50az_iFw=9$Pf?7xaRXg13TWaCA>-@*5>w@s(Ax1qKJo~QBgxZ>VN*s1L!PE>FF zVyLpIxrWPYZZ{e6SnlZE=VMdk?+G0>*Z@AhuqOVA!FGeliL}43zW+G^f=%dEPO1DY zXdcgD^s(+6R_pQl`@y>ZPC}0TU6xmQ|E!8tublEi4nmF&9vKIww3fR$lC+G$-9%v* zZ$9_dsFsn#Ecih0m|GltrQ2?uGIkf*Cd79qQ}%MBLz6yGd)El)euDTSca)NMNaY)n zmOdKP#A{iPh2>{!G#*!k7l&#p)o$wiN5%i@TFzMc@@H+eSyfhg<=ym$ zri=a^*|x|f{VKP!&}f_Yd3Rq1%%udfRUA8y6(+V+U+F>p=s(1{E(eVk;!}9PHL!G9 z7pTr}G7MAGz+{CYy_qCe@%>4&{mZo%eMXKKj4z%CE_h%A zuOtY!`GB2uIlXsXs9%m8rMw389=D0(l~Y(}AL{z0T^%Vw?nix0YcE3f5GVY_7`rS> zXU|4BCb+Z9LdV=w`GG=vuRShrBp@>3;-PEAl;BGTEE{~`+joA)4&I9Lbwgo^3Ul5X zdOuW*YoO-Zhm#R*lc5*XS_a?drT4>zn*EU-fP!+BsLaekKl0};`C$8<&*!%tRXlrP zCU=F%gl~?lE;|MGyzk8y%WY6ntaWl&9(RzvbRLelR;Nlz31~a*b3b`vX%i5|QAeBt z1DIKa7_SZ|HwUjX(NHtSPs_yvAASHg;P%(3Ojidq)jHU?ve89Y1BxE~hZ+iNvWRHQ zFDcIZ{nCGN;WqkH)mUIYh8xcQ<-*b0_SCVouB-Ng<1dFBx-VxrbTpc+FrAgC>ym)P zj_A!@&^oYY0-CDD4FuDxWE0XPO&4)zKvN;uFYi=QE_ThW}X7)^ej}XHl`d3m{UIUj>`i zPUiI;SPU^Y&@@n@7y5)G7(}2jg`LX`ot)~C@S;O6E3foiX1IgbidBlU&uZe3>1l3n z$j=jkgD>PFlT#k#Z$hsWU%88C@D)nKw_IDuWpB07yXEp)n~>U@9xuXHJF*uZIMH_> zZMnRu;&W-US4k&VS|lvEv?EJa4y_^6|6pt$i|$t$U-WPOA%`cy&^N@mGIGQimFNV7 z&L(jH%o@1wLmy7fYn?y_C8h+@YhAjKiQbnt8r4bZ?4|XK@Y%f}yB^u?mjRjP#G^)C z9n?f`J+8e*@efeY-PNlH>-7xF=K7W);DL?$fMN-p?q-3{$xs{PI1A2CiVdM}SzG(~ z5$6`a@g298)-~mPcSy<6$7{$NVJ;M1MffZ3=y(+{q0FWo<)D~BUvNDjz*$Et`P`&0 zkvBA(*5qEi7T?n*kHvU32SHZ&(0wEjbW?#RB~yMbwoO&yR! zFEv;nvL2}6ZK{`$jS+I0dCYrS8hC(YOT3+zf!^q8eetanxA^dSuzF7Z>x_wQR_hXB z6Bwp?qLs|{2)?4RsSV!}y{>FJ-yZeCG3rc&R{6}nS2h!^_scTWR|^YbWG80Dp?acm zOe*zz0u~c)aj*vR`&|RnH~`h&ZQ0O>&-W-kWSv=J zLgc^tGH-VvHj+pRC=&_ zu6qC=^@bS}UW;li+O>r3Jg!KsjjR?YZmtQ_w0R*Y@S)-{SVro0Meig`}SQMZ@N4c&`2i*6m_}u^u@ou-ykNY_;0oV zSUbL3=gA3}V28%FuZKY57mI!<1HknZNL?`{fRcHVi$7$eQS6t(v%)3zLl{eR9+r0f z+oEo^Sl}lBb#;MEht{k%Ha3<8?yu}rfBazAS}UN6;>u`^Fz(-R+dET5U;P0-c_|0r z=0}N15u@PwV~VC|9iFL%fbc06osjnG1R$EemU*y?r)xfo1&jyQ=;(=SfF^qC;KJ^F);)rQO$pN((F(;P@V@g zRKxNw&td`PHE3plVbz2P=#@NC_Pvm-03hX}j=lpN zkkD>o@nb)rxn32~{D-I1n*2XC*ygs}GO6)cjbl_hQ9B6NV^7GDt}6!0;bCzX#dH(@ z7fYUug0&($mmWM)Kek`x$_t_PP40YdzNqSR$us!TvCB(}|BFQu4#45%WWX`OZXrR7 zL;u^Xzy6d|NIoW*-~pd4Y09lyfgpQwN<#VLI>i5%Y429CkB#yid>DN=v7go7zno5$ zR#0lD{PITRS_WM>_UV{6%tIFPwck)nt7$@Qe?W14vn@y~K}usCS5~!ryrJ=sWFks# znl!lqMnmNNEA#dhghg+M;Wwx zmdO@NQ5me6hHFxq|8)BRtT<1tr(+>1Q%}*4;E^pwf_THy0MVGfShJgO@050L?JH6y z>F=_=zHD&zVESzjDg<^?_HD|w%R$RY*_L$6-$UVHmlsJrV-rZubSNYB6@9&~OQ&sx zo@hzuR%~6hXvJ5B{rqO}C0H_a*I4liejPo$IHz**oJR?OmZbdKm(jXzYE_3qD7m;JlQ!zF4jrY(? zAXMu~-i)goj?nZ5p;e$W8qvOhUd8-Ukk4deqElsb(Q_DyK#}j+C0*UzQ2K?u7d|v_^Qi6wq=bYX*wb(;>{(djrkYcN#zPs_&mHdV{gI_Zt)Hgk ztv(=mm9?n`M7C~O$n+!Fw#IQk{nL3hgz(785Qkx+<9k>}a^6z74p4z3DdJKf<%JFI zsOASqM9;(4rQWMjNZ!S-Ye@@t+?z;KTBHbY7})Q51Z>jI&)|x zxUdH@^IN60CxSA7d6i$R0c`)ju{$?;=Eb1Pz5)kV{@|os^*A_Zy5e=7}4=3e99fBnkkaK~w}yJUA?42 zd1gwqkI#jfeodQP72ujQ^0h-Iw_hcg%387-Q#+Iq(V7jaKdX@{pr~gz#u~tkrdl{N z*=XYZ+L*Va0{1bF_OA%ix-*dnqVgpbP9>c-Y)_VS{l>5)g1Zk3+;ZhVrhNN>PM>99 z=0*unVw=x&UCeb?(sygRpx+OOcJvj$U|+>L=Q{u8p!-aOt3rZruh#CagF~~g~>-w86O2~BIq*!g}2`Cm?A)!iL(1? zgXXq`1=M>!%QjwI$V9gUjAagR^}2ZVMZ(ra#v9fik)I{4TMvh$!!__KhL`?P;ms|@ z3A^ryVR6ZB4_`?sIHpif1}OWO&IkC4b!)AVuUB-Z-*%DGdUo1yJHh#f!3^fj!ipaU zkkT?|OE8kt3K9M%1k8=ydaL{AB59?hX+UHdkUx7o#oB-HG*&RH|8!u z-c)Y+d=s$~e^zF#zYJGKjZ4iB=T&8eu5A_T(Bge0s0T!7Bx@D_Gz92<-#sfTvF1Bn zOB1Gx-;A;X`54o@@qtWpL5s}XRIBE}fPU2#5~`)SH9QL^FQtHM%$kFbG(bG>{T|Gm z*@7S)=MiKEGem#wtrIg_y)1(nxx_-402qR}N1RCi;BjHh1yonvdVT&M(9m*02c`E1 zA=ZlS;_)dpe5s|RlIY96-Z2!ot}S)S*`SUy7o7`_c!4U2T3R=+wGj<+>2E<&GAn&V zukO9y7p*}?Q*U92^q|Fq!Bv$&G5Z_SwhpsGQGTc4#;xGyKn(h1^CMhEChG=r@Mhhd zD^Bya{jytD(?g$l#nUa>HTGm`D{FxJN1IkJ^i-zq*-91@*vnS@tKPkp!q!;gi0Z|Z zHPX8O!+NcLRYT#b1W-eG5wY)2Nx+oWqT>z0c*d|_J}%p@7r6V#xResT2_q0)xrCON z1-4f&69)H~pFBH1{aY;(Dv3G_)aPD1o*N%5*XR8BRVuLwvkeIGbDUpKtRKl$vXh;P`R=Dot84+L)$;u+2xj2ay<&wj)KWKB>}M& zOytb^r#Up36>Y_Qz20u`%3nmL6_fnL3WHdC($u~I`Aw{aXx}Y+xl$~vy;v=JJC$OP zclGH9TnBv|Wx1HQY_Oe;&sd&O%R(*d+0)AqEdS^c=>Y%Y_V*#Xtw&UyB`4UHN36^O8)h2ccDz-+utgCZ`h zt*VA;UCH^A?T7PvfLm~pS8rp7?u<;hAo3uryus)^>>vOGQO`wR_syoe)hgmtxNs^xPD&w8*pQX#j_oHE8$Hv z14_^()}`YfjY^*$_bo++7ge1$?0a@g_COnqL;pwP94hk!8Q>ijJ?*u}(TNR&c^6hA z0>UAC0{1@z+#0O;d9qIH^i3vKCy?WAR9uUw@0cK z1)nDg((|^y+G2oe@S54PEyk&ERcQ=`F?tNs-HV?=Rim^ zG1}X+nM2x;*M0ssLJ)$_g@w>_zWsHP0650x4$fe(=EB0y!DkICx!YqCsc|`=#B7?wno8qjihU0Y!V(MM|2K zoL%weR_^z~>C@^T1H};K3^_~m4mV**+fn9!YE3Umw3+8dr z0xNCKXeJXw$~uT7u}agX;5-icu?UYQKj0X zs;lD-}{;!-*(zCW|tS~^5N9SA8^~;~1 zOAt`O36)1b5`3;I1@a(ZRqj;d1_O|tgMAwGtib9?EFO=bM8dZ8BHFdw64JKhmHsZS zKTtMzoA|l6D>P!X+2X;kjuEXY7e$7yj82a+-5MHl4BC7^L5F7I_~Qa~9k0|Zd^;ug zO*_R^P~{zOl|dJ*b?H<@B4<2%+BCH^uCq>VluAJKTu=%FuAHIe!={3wHwl(Kg9oIj zN!U&Kt+%()^0!6awHgexgT4V|>9ne{mDN#T2X-AU4ti@sOAwZ(p&Dh+VSpDq{mAP* z4=6qR2j82#tZN%;y2a^H{g0Ka4eTf>Dl%$)YYpRumVMkxhFFDUjRV(>y1P3L1HhH! zo-F!F1Ak1)lN3E1T>i3y??(hF#hA(DPJK+SxCvZjmT=GoK)Av1fT^(1%vGwzlK2U( zf*U$@13pWd-=H$mvw$6FYVcyi8Sw}7nB29M9Nlo*P~d0E0fnu2f1FfkT%Go76C#c* zE`k}0nwjC@E|dq@QXZbgCv^EM<;4d^20UWN=ZCDodo3}R$}G=L7>xDP!MwW^@7S|{3VBtreks@t@s6)aRsUu0pOQ-eaq`7B z`uV^D5HrsGcdu?8FpJ-Zae#F#^<0>L1>9>p-_pzV6?-mpg|y*H4LqL!@qOnN=0vmN zw;Rz`2m@TzrwWrg!vVdpTDih+7yNfw_UJ2L0$auhL)(i6C+f0SyaPahVR}Je#3?oKvP|AG-~q42LJ_?nIO=0X{o6wd92P zK`1fQV*ikr%R#KW8pQezm<<=g7G%Ttix}RlRdcAJ6>P|m2z#R`r~KeZ2#{V_htrC9 z1VhC!yXTMmw@PZu=H63=*AJADvkoV^ZT8wi%{1IpMv>`pAtqN*;0f`K+G4@y;c=nc zVy(>h8}0WRLPg^td*v{demZp~4Y%%RRj!{>cJCVn0ik@Re$}>~5MJYMIY0 z;VF54aS5Ee@$hUxQ&EEUCr3cFZ#xhC9@$eULFddjEA^i5xmlf8>&{=?VJMsi&j1da z=0)q*d-9+K9lGP_-w=|lRX-;3L3{dWVk7BxFpR$_!<;QxB`SBGz7~-~QZwv3PxE%W z<^`O-4tsxgn0!;w_iHH6lvYqWZE2RQ-Sa|V7Wkcfd#%bIaWm)zjj8SG=fZaakM~7M zozy9ugvm>t@@5bJehff5DebPD@uI<*8~BUSze`N2M@muQX4JaUuiV3&9u1H0n|s7V z$^+j6_^Y}SSdy*(0ZZey6k>fCapbf!*2R2Aehw;je*)%pO1<9YHXMHnD`e-JF50Q` z&S)Ee++H!d{7{!P8xV2PI9WRLUm~EFaN|rNM@X-V~jl) z_|+KTXy6t=lK;9{V0fK0@k|1zeKA=JMnoUI?mW2e6)TF=!1JAr7ou+&57v!J-^bOf z{)qA_-6L*)n9^%R6&W4<1U~fyTzjKk){ zVBnPJ+9mEv(px(>lfuduC&fctl&}?5Ha5%#%;)QvaDXTVZw^wggPqJpik4=QFMhvy z)Uza4CGtk>QXa3AAOYW1sd9G{DnrCt=S`w4~3ArI<<{{&on#g^%nk0IZFd<-S*2XE;OK+`2zoVbstI}hp7Dg^px~^AEoB#AM^jy-g^f%*?r-nK}A170V$#a z(gZ}Bf`Aa|y$c9PSBi8Z9fSlFkfL;{0V5DVy7W#Il+Z%&p@{U}K@vjF8~lCeoIg&P zd(X_7xidF^LbBVt_S$Q&=XutPE|{2kI`P1tT5~^4Be2_RIT5BiBHe>!-N+rukvF)L z(weAwEnDhh?$jadGGQ&E)bD4vN`{kvujx=Jo6`iqv^w$*c6x4e`rNTKOQG8HOOmUT zf(nqzwF+>$e1BIR#W=Q)(W75e6P89Dmky<-@k5U*>CG? zV84bnsuGQ9YO<8C!B-QLb1*qN3@o&^xo|1bL!nH7o8!^&B}IyjTk^5;)|*55)jL*I zJBaY26ifM|q`Z`bKdte=(YAlQ<}oxQ?f)qQNHn~MZU$4Z_7r^a9e#|Ty!$gC4k-^n zrBarGc?T)=k{_!r>*W)PjJ-8S8q2GeI4B>X`W-Iw@LmcE&wY=7NO{lT4}9jgE?FsA zbG7y2w)~XQQphsoO*O;4KLmp2)|;jiB_*hFOr}fO;g1vz)3t%7Rik^(L%mfSgN-YY z%EO65;&EUD8gPyyh9sVaGvgq^k@_cG|hEFKEGohycNJW?Y9 zt94fcs!~Qt<1c;Lav9tpcFzK1A?o4hs3&=rwvQ?=XMfWFf+~MtSYqF$s9C8$v9jCM z8LE)O>R!I3F#rZYfll?A2LLq{nz^-}w%jGSp(MhS$#?ikigpvYdIafZS3>?d>EARZ zfKtcV{$$hbc&f%jS>sOuFyKiNE*cMRyRMnCX~-)H7`2fhe+k#K%*wM&firTIg+v$bN=7T}I)9N^7mH-`oJvEx~h4 z=yKp0ZlOCNegV1v|8?x+{~=ftm(XvIO&az{kYqe#lGHQ36(C`BuPp$m$EC4((6^9; zM`3+-0l2kNcc#N$lkD8dO#t9SEtRIEO#;Z8T;(Q!WE%tcU4Cpr|2_}!L#N73Wf4{! zKuOX6E+A{z(2IZw(Xqc!J&ok&kt{!cc?M1aFgIS1_`Y9_uy@L~izse{!DzY70AThfTy7s%YM(Bps%3nL^^6`FC-GD4SxAUw-`=8)m?eORzj!|~ zzWgOj{KABf@r| zpS;Y%GqENTMBAC#cZhEshbBlev9F{wu1lM`D@#8Fq-(-0` zw6ob`g|SREz6BbP>xvWEodul6ovCcEhmr*gxlsv{Ps>N=*nFWt+XkENZg;P%t<495 zTLBV5A_83EV(l+=ahqpBX9E^z6tavp^M0S;`MyEBer>Uu#EBr`?>40 z7-s-Ntkf*pdIT^l#22C+xO#YTImPh&`T^rN zOGSTq_qG~>_EN|zcLB9zUmi>}q(Q=Ns1=b*;=?5INiLS=Hp!|kI@d7#lz?LO0tSk~ z_ZOT%{tVm2r`ekiO6GY9NtT#f^t4}>Q4wKnZne-E)b(7SJ6fSo0N`-aRvYFW@@cD? zxdkw%Aa!$C@v6~5Pk?TK)b42ygMEm0VggtRY`+f>y(*G^9yl-n{V&vFGkX70AWF`S zg?i@yn0(;k>M3rKP2!U@d6^x7%}*b=z;fXU^@{sdA{ga2Efe;U3AJ0-437i$e$h)5o6h-S}a0pjtZwNJw% z0OuxG#SrTPkg%Q&dnluMr}C=4ys2Fv zc-;6aH}?1htrW8Uo8NP>_|vS70PEZplRohYr`PDoHy-te%lGqJZy&w}>S6`Ou8CzU zgu*K7`2(OmlztV4ns=>Usj50=xA^omT4)2LE7;AGl_1J>uj@dU?883FVfy$UUw3HR zfKR%ujG*SNe`slK+5(Tu1|f_3rqU+3MV7;QN>a=HpJJLt5GP(O$*u7hVIfG3+(3~L zx6wftRsmR3bQ2X&4}wkcVmHh?^)O)Uy!5})@g^Po39y%e${sZ5N---F+Xr=1)$U^q zmo+>e84}0MVe>yc&zjHi_F87(Nl*B?(B#)teb5Wbg*R!F8jU*0m_BygAbh>#6N2;y zN`wI`I=>BOCrv>3mG|gucJfuVTa$o4zLGZT1dUm7x&4(y4Fn&!7mCwKw|GDd-H&%fGpkbKC01nL%uaNk?c_Pld=ILemHta``f7C6bf zpfM0!fAfkcbWlU!-|@#bKU}`Oq0Hn2))hCOT1VJ7Sr1J=ZfhNBG+(JxU6&9a7xC_U z6Zduj)-F!)o!JKiKQ1nR5~N8C$QlH6GfImK>^k5IRF8T+^<^EBTwK*sfs;JgX+z@1 z{L6Q6%#H)!N*5;n5 z9}^odFcmCwi~Fi_Zz>BZVp($Vh`B`~An^8YkZ9Gi^;eeUbs}$1B!aQ|;xcB# z62)yVruXlO#=PPdWKQ+8j6WqY@!6yt*qkp;_L}ZRW`94x$Gw%h6G=7vT9lS4=)HN+ zazNx3bTFx2LTq~E3A>22UZ1k@Rxb2T9ZlUgOpB^q11m?FLN)+YRTibaL@)w$koqhS z6=dx>VlYcfo z@;Q;KqCO;3VDR=haL-I3%WsU+p*#VJPkK0RspZucfGZP$EEFOAyrsRc02#x&cTXiO zKC1opZrFxq5=^Eyb>|p~T*(3Cyukj*3+5yb4LRLd$aMsCMp<`NYK?F5-jH2Vq+sMg z)Q|J%)q+wN@F*Dt4)UIuACVWeEH@Ut!~y!s`PYG_3|c639v=>s^a2k5x#N(}FAMYB ztAaGk`257G`#4R%`-3t$6`CW81JZ@)>fI@rojg&||LPxt z3)5o*EF9+78h9snYA`=ny!hw)BMF7v##yt2Zz$R$nYqQ*4A(6m^-3s7qfDrfX4;yZ zK5SDYLoV;g;9tmPF0CK;d-McuN=x;(@vi~GZK0tO^a%i{VIa2h%TdgMzt+sek>N=_x+{?*mf*O#+Os2lNnCVP)eE{IQ%?E zS!;pe&pZM>@7KH1F51ZL%UhBsU60!W=LVmqoCNwGk-b#%I&}-!)!B-OXrqrx`$0>m zjOztlwAYYuEmw8K6p#mY{^=Z*-DkD z@K_RTDC+Ey@<$<-j-}S6JY40L)-#e*J(O>2eD5FIiR7xguwM6MrjuXx&hAWM^c>-+ zwtq(&UP--;?;K|ttZw72J5d)LrC=1w1;xojKEWeS=hRim%C*gPx6GQfD6aXPC{f0) zUOl~LHoy>FnObo@7j%ry$jhN;;MS8(8FeMdXMZNV;~pMr%~)H_l1FV`MHgg>;4VP5 z$abHjO@~)RVI_gNz;*z+HFRiw(syTLj%L2l^}U|{mV2ARgtEU5xabubM>2P4Y8P@o z$V^Z-KKRw35!H0|e2{syn%tc|?MMPTp(P_=Q7^YsPZ)aU4z}5zj3cgsbudZi$=bAWzR-&B2ItGB&-k=+io2tx+Cx9z zrSIIhpX1**U5P$lzx5$8;cW$HxmB_s>T1L2mxhVqz7579DW#@rr$kr47Lm$x?}tC< zOAWm}s8d6r?vbnk1a3EbN0gj4yRs*AxcKc^k3|al07FN`DS7Emk)J)BUerq=rmNFL z+%Dv{%qlX{nkrneI!s^>>v4k4K0)JyZ*$GBNBOZ(#Hwrh6+Txxg`xAm8%GM61fA3^ z3K>i@dAvQ8aLT(_I_NLYGj&~1HtS~}YE#m@W@aWc2{u`Dq<|}j|HDAOkUL)din+jWsmTF9f^-0rl9jzRZeO z;p;Pht|J_1l{uFY8f%c&VpGNE@ti+drsK_?Ah9ex>Yvi>eLA$M1rSiQC2dptGN4nDSraEdnYLp+PTwnBgKUpPhbk2n0Nj;94!vX{su{4D4K6( zX5f_#kD~9Z* zaFb8pbb)2?XLG}3;f?biuP9ixAw=01B^w z1|sNbUbgJda)iD!PBQ9ws~ShUVLdRoWz-KC$>)?{?+D{~Blq?l3RVSI$y&Bbsr1{K z2wN{9c5#_2LJZF3W0>te>pHTmG3FH>XtOgJM?nMcF19r;tNjE>;YUaS)TBBLs(Ymy z9&H3@*OtX3#>5E=ZK2tVR|Z1c4X7V2aVd+Az)V7(v=rNG4-R!ybyqTfW3qp{^Bmn*+YOhZ@x?9Sa)e z%$s5|(#!r9B+q@`|%NBRC5iRp&ayTKlnRJeVQdgXLj&JiQ z`<4gRhQBx9re~`du_;W8G0&xI@SY{- zx}wYO$ri3Y?fVw>rTlSxy@|&+W=GAdT-3JfIo#BgYEF{d!#Bj7%{HJyWq;yv{e1jA@ zfxZHfaGBKELR|-9-s-dNz5&tgkF&B%pl3jMJVq*P1p>YQKW!9xz2^8Ns;M-WdFYyz zg3ZiE#^~Y`Pt@OdAB7~KOE*oD+@Gt2UyXy-=vZLQ`w6#zejOZTzl(q=rqd8-|KlBd zK$I;)QE^#H~!LkvC1 zi%k8X?HsXQ2xXMnfo+BAq|;3yfaQDshB4V^L(Il_w0C67BGvt?imIg2VZWRPxM{}8 z4UW#n1Nt5izi@!kk()x2&vC#4){plRGz1AupV_Q$B|+M6qh9gI68O?H`~8CHV*HZK z!T8XqN}*SS5t121ORHjI`4L*a6@M5p{T@4gZ3;r_wJdpbU)rO)K$;J?@s^SW9-pe($Q*QkPI^f6u z?>{s#j;`7fVkV1f;;0BQQq@6KnYpQ2ff5&0DPmb0{3bDv z#LrKer8XKj4DD1$d3hwyj1=2cc;~{GShl?CYb7KqH;!Cy)g9-;F9CnJc)alU1gh{itmr-!kF_yUUG8cZ+H4QvHNzeOyUgF*U_*(88_}=7@he#VnG7kyz z>&HZ#l57Pupy@~64np%uw={3QjF@BDC;q_TU3*klXEP%1?gpM#&oOY$iroJFRJ@RY z$fIgM-dj1#ZYBiLlgsbRIdKUkqyHBGnfTV(8?Po%Y%@LW7;IY5R*8PnvDL zEZI!ejJ%{DWcK@ISpx1BE$_W~VLo12L&^Mk{)luhiD?YlwK=KL*qRFP44~%`yA(jA zcE1x7tH)s{;p(%e*Vmv}3#$8$0c)@BX=##iD5-y>=US?jz+e{J#ezwm2J}@Cn1wJJ z9PIYUu3`MQFS$xeHw%E3PTJYz!66@=wRJ7Xb_LK4%xgv+@Ty{5?tirH&zx?`>c0T^o3E##&e3EuNu`>Wwu$n@EsvX zP;E6b_f$CRzfVdpsQqya0B`2bEV-gh9n_P4CUSZ1Wz36Em1}NwMf-5y8O1P*FM&JA z!|Az#r{*Q5-JAB9w{{D+aM&0i;Hu!Cnp*d6Jc@HMj;{G_NQu&khR+7 z-ORfp6N3q^C-2O>ZIix9we3*A?@A>`L62oSws|a`ZML=U}RKLWU8=`5P zI7uERI>CS22pne~s4-ia3#aB^sapNUq+i1bG2hhN(K74` zNH&LbvhZx4yA!7$=&B1VmFf?ck{@_*X*6mI(cUv580z6^c@gwkk95G$*^$Mkq#G2z zdRZZ5=B?zcu4{3`62W`muY6<&0^K7g9anA)Y~@3)9C_cPe~Y_EW?%9RL3%{iEakVX zUu1;|AI#339Stfh|M{*vBBmu3Q@0=aEVV_cck|?71B|c?17xPUD!_@TV@4is*5dx^ zr)(0I%W&~meOVao_*HJ@BOpr^ElFBzD?ZtBrV;^|hjJ&Na1}%Pu9+paJM3$dPLN+; zfZB}JJk95j{lv%`Ax(*|oIl>tqY{PoWTm4vfvS|uUZh3-VHl*D9~eGd8eZjgG^OZG z=>6}8_~1HQ{w1%FXY}J?*?2F5QGYiq^}Nr;{Ir#fQE#7pqD0WcJLkn-%c_KH#X%p{ zTN+&?jY_x-Gz%_ueIf%CNsjyWTME=n+{E#qZ6kG#h3RRrV@raUROV%|x?xh&r!fFAv6{EN-DFfzG z55DA^ES&=K)xv^^H9Sa`JZDYFT0Z*Wf9)t<`le(>spJL@!L>k&P&Y1$sjZlr)JHj# zB1G8m$DcQ<4Trt3$pSXc>StZwH4nT==Z^fZ6NudhikZ1WJTg|r^WO{JE&1Z}$(2d0 z!cv`Re^<8g??Mb5BtKwRNvmys^*8cSLjXlo4k)AER$unx^unIk88wVK{8L(gywjBs z8DXbfIpCsTx34>V`;qBU)5&*_rv$sLKCBj*u)j-m=c;>NeeUVZ(AOpww}z<-uKUYfTUbQASF#Wf)-6F)|UTe>|5UUh-2Lk$RPC4<9PwF>fi$*qSE zq67!9PSUR<9KoN5(1N*{Wp2j>8a%N{g}iJAUa9AIc0$f_tjkne^=M32{e?mU;yc_9aK+!BGflJ1 zVeMK&_zmtj0+(i-+bMi+sLo5$S^h!Do=)*SX@=11YD2@no4g!h*BLJ}?a6 zUa0YkMhURlOS&Ie0E@l_c2hq(cPG8ViJlasd(*$Q343N_-d9K?Wi zG>%VJXensE(K0Ju!_tk)(OZX&eX4EKP9JCjISdeBRram?8TN>d<7_Vk&n20HUW*|I zW2rGiLo%zysCMHk`a$gjje(6jj0F?WK<(Gmb@Upz_C{*`AWhR#^{<2$|5H`6w-oq7 zxZ}b+kM^)|JBCJFG@I$yEL$Yd4^`F;FtC9{y?=k)BvStom_Gd z|4N?<45oAW;;0-N?=QtL1){#B3v}W|^K;BNKeBqMkcCp^i&n*@f(#Ai!t)RYGEa zA`hkKn!A2wu-D(=P__y-TJRiB&oL0Vf$e%*4WuJh8!A<1z(DbZ_)nT-gCE;4!YaOhM)T%X_) z6FSlBCen>ri}tC05e*NU4B#9>|7n6tyIOcPHXpm1)E>)y(P2dU$5>>Fu>)xiu2H;; z_2jx|BP!X<)mFOZK^LS!Xoy?eW3U%vg2pos=$uZ^Q24lOc{VOtt}jx%TD+>5-ubkb z1xF-$dbxTw9&=a8yEq6XP-m$Qs{*?!Rnoh<8~WhxL*<%!ZRnpiW`mPM_o~9@jh!3D zI#sB{8&xk%wmvS!?Q5Lb!F9IMEejSiZi5;FFY{PMJPk#xrx!pxV_rsW_(D}eCh&L{ z|EZG}h|{md*qPH*I5h#m^?ov7W{ao9W6#0f{}I#FN#|DbTFt_7%vC>~LDZ+2YN?@@ z@!K^K1B*fJA^RLCsk8-c_kJ!r17uSeTt}Xu$^FrD*h3dttp7z|!**4UK3_6ojqRR3 zT6;QnQI4$KYdZGhMvnPygGx?I0cPN3Q<>A`xGTyvSRDf)8MlOqDrb2;lyTGAnsf#J zVc(vHyT4S?C)8eyEqSMfslA81){2}xgbJg>{Y+O?oRD_*mk>e&os`G6j60Vu7E<9YBAT=EzPutEb zkwDWU%4MFxHyN<$3;45E+f20Ri8S$5W0t}M^3emk2S}8}v*ZdE`N;P%1Tl#($s%`z zXOMYwO(pbKF(=JHitBXK*Nzvt0Y#g3tZ@Xw!>_aPfT#V{u(yF1I!(_Q7Oo1x~c1=XTp2`Hn8j?|ZtIRaK%=01_N}&tX2Janb zhGe=BdpH@|#_Gm4IwRtpWu~65xyd;Jo0_U5>kwmPL!LYdLgwvqXU^yk7$j(XMw(MK z*8Gv_dQ<;k0)ky%B+74OC8D%?d{5f7n22Tbh)me;0P&ftQYAp@u`}&Ft$^&OJys35 z9mc^9p5&*gE~o=6Fh6qiWf;5+ zy&7~dPPi0E>8)$~ETfBsh@VWPGAmp)oH6nKwO~0T|fh9rSG2sYwHM8<_Ig5%bo-U60QKO5xq-A5;SmWhquw^BVhcGMt&X zRE$ft1*eg>>k)TWKdVH%&xju_-2hU^HbE=%4a1P+^lRfU6E&s6IShD!qt|wK%nuCt zxazfK%xgz`XKL$_IV2h8Z>V{tN3}gQVcSVH^EpN8%_-JC7XU~ox-rO}A6N;BTdHZ8 z;Cq@YiR^rP-SsZdye_u3Bq!)mmk~DFFW!l#ulbNRHxmsHnT#?drxy}QqT`31I0&g zoSm9>Ij{L3NWt64<$oC2bqr;c5xmCIxzIkbq7K{d(8rpe>Q5gvX$jWEkBX41j+*MI z`IXv}l|yH@f0cDpMl^c^nadTI-pPF@LO|UR-^oq<R{EXi@dLTnzzU7lQII}t-^1q@unTwy3yZ&*7nq-(f%>5%f0RPJ=j zw74&S;WH(?ihkTV#l@%;bh)EpWgw7vynWn?{bJ{Y_<|sMM$PHtY~+DO2MKHVCG>HX z7nNwCadk{C?CqJyP~)mMKg691FsXhKv?)CSfBAR1BjZ0z1G|H>fHOL>qa8( z)hsEzc0Pecn4OsRBbT7jK?^oZ`3sQxf zz2=GmPj72f5vx|@W{AZ-BMBH=;ri5Hr|NhU$DO{U)nBGvZ($xd(ScqFbo@wfT8QrN z1CFi)`Argy1siY)uv``He0Yv080X?XCw*)z=soRr!9%M`KqK5dif`|-Mj zA_h1v87@k=cc^&!KXiX$gX>zA?cYGI194npzO~&G!;anBu=kUebHA`%UV%5o-GhWS zSRj8AZ(-}--I|!!Yr0Tc8Jhg97=Y#yCb5=yktyyy?fFwyPPipqQ5Khy&TKW;YlhK^ZDRKJWY>+j-*@Vzm{a}PkEVU?R%(A;=`dp8Rj^IFM(obABC$&o5i?BlGVYa8nsRh#38DbqCi;qde?CYuG1%@ z!ZWH&Ns~IqPVBrl+I>{&^Ts0W;Uo%3Z&@e^-duUz)qxXL_wUuz{%@BVP|y{$+*5Zd zqo8)wi2C;z)g4!dT9V!8ANw2iiJJv|fcRuPUR$-}@OdRh=(;D{o$-mQSdgIo3R7?6 zS-AFe$LOTAU5vXV_--2&a7~K>USRNUj_saA7@zDNJsY5vyw{M8r-WyMd%njJa8FBK z;rVz02Wmz(+SK*>7q3~NmLYboCtiJm$yutzRtC0Ncr9iH`y}wT@prqOuZD8E70W+g z@&<-!YW%k$yWQ*Vz{#FYBO|%N8rLPy#ctYsQ$~Fhmat`D%dwrQcaS66W;baR0{G-| z(kS=AOqKInN57StgX#yxrK2vu&}_(A9TFlc`i~NqI>qX+9-gMTN{6${F!P)LUV(np z7Eg5ifZqB_i~5j2LQWH$(N#_CkKVHll!i0itxCHkV;-?#;oV#6Py_>A3q2scX=17<>SH1#AF{M+q^~+DrJXc6W6c`KGCWaYR8ZM9M%J26dm zPI1qKYM{APGgsHI+()M2Fm1hfAP@0BoV!QuIUDsntQUMzNg^QT4f%QaMS`6)(UJs9 z+brP`r(+D?#Zo^CZp@yuL1!iJXTWXC>x{`XHg7bM4pylJ>5IWk7D%B> z3GhNCB2_Ao_xYc8s3@2tyfNj9Z~7F*o@~{j*EpM448*JC;ryw5$;KVM?js5mmj_FM zqLM2d3KpiDD=#Lr=6m|}>^%$p9k)%~SCTZP1MAy_-i`)9T(mo9*avAx8L9rgOy8(6 z0H@aKoHzWP>_)PC)p=(XIA1fXbMkL1!q6wjmGJ7E>ufHiCz%#$;*VdLCG{pImLY5J zm+~chs>xa5RLj^)`LOe}LcHEBN5t6C8zcL|k<>Ck_UQh9QWPB{ZvMZD&o$AgKVof^ z=F73n<)J%r_@-WcDjdxhA@F{GJYuTLQq3_i5bByr{wBcKITfEJcsC2HX%jK+vx{EU z_?m^KHQgIB_2Uc{;q2c?P51s>(B9JLbvk6&Lpvquq?JjsYsrj%^@@)*64Q ze`X0bq?DE>kbZly4kR!~$08Dn=SJw+$^#nsBRTxa`cYo+bgr1gQv2@I6Wi=hn*W>M z%^F;@z5ODL$XqyKc6{ET(IeDExUou5xkbD(X3zuA_lw^+TBvJa-CIp`U zb=%&Tlj`z|i=yvB0Ssf2+CHnM^alPTjGqpRQS=u4e(p&=?f8L=Or?iXCmZ*V$Xa{} zKGfqbDX-~g3G6bl*XuUW>rCW4jaG-V6sa$}Cf*dN1t*`%yBH{-}s^V8Jplt0%r9f zze(MkYWf{e5Hjg*+VJb$H;D>*7ay>T&41e^+p4vEBn4!OZB=c=m!Yo~ezh#yar}7K zzC~zR(F#*OS}Q2I$H0cSglCfSD^0W`Dojf&9rL8H^_8C?a1$tq!<3B3pppoUX@`?d zxFVH?!ayW~b3^#1xrGzk$ajk+8{*qeLF_~#n9sh}e?qF^M zNI!)J-_~Et_0(4a?;bEyk2h}(JG{nE59{w+r^Yrrj07P^CnO=^ovhQ!F6rYsLOV=^ zD`Eg#{x#&5(b`(*ARP9j(ao__mitbI{z=RIh5C}InM%GBS4S$pgQG_6bqkEn}S5`{lsR?B04wl|O$ zvHlKhgNWKhyY*S~tfIXtv=I`nx?-EMAx+tn9vO~e(~aa({1JmwFSjP(eWiJL8h;z zuaqmMfcMa~60GuBrgR450(um+TQfpMOdI`Q9e<0t0Y(bbeaK)4j->D~ZCtwkv%&c$ zI5G1+&)RtjRW5^v-e1ac4JO3tLmI+h$5iSXaPj0xVo?3M^dk2^;*u!oG@y13r_3|M~nau^>m!v9M&o zzgSWxb`MOUw{48fP1Q|}Ox(dE+$8Kjdy{bfS&yfbg!|8DoX?oOr;H?30SalB8d^?T zN{S-Jb~fBbnDe;ZZ0s=$u*BR%z(*TXCnKnvjkT?#h@1F@qZK0HGv+bR1?bTdr-$Mf zw3Jk#(smA}P(f~9Zr%$LL{KPH%)!J=L|sPq_jd4;_yr3mCwmbd9#>aaZdZP8I|p-~ ztJkkz=i%k!;p5{1E4UopZJmtVxNIF+ejW19IWne>#txSDPL_7IP|UeT5A2+s#4lXH zT=egMzu+`=v;1?Dt>bUB0D?T2Z+Nb9^YZ-vZ3D>u3*U)dZbtV1$4yR_X8(t~FyH*z zc0`O{Fp6Oy5K*;sGqu*1v9vL@bp-fG2=NMw9l`dWZ~kC+Y^TJ+-WMHC9{uCDg4mISx&FJ`x8Tr{Wi6Q2wC;o}fMeX;&;{#6?QDWpSvQ3i0p zaPiJaHhY>`>*oy@e~o&@DKzU+)9S_Lb5(ze!SSH~p(9+ZmHVDe(9VB=CM(5Cw){F9 zG{)i!yZtRa*)r?J%~vd!#X|G*F9wTy^0hj}{}!3=Y7^rD?T*0PNc<|p>OR&Ps=k6PIY3;&1w8W&&|I)p3%7}lgqE=Sp& z*;1&VeANzY>j3ieC3xk}9u))1p$N$PZ2NkG} z@DBh-)wV&V^fNrujgg@(n?XUl+wE;SMm&FSARF|trVv1KE7X3A3o+qY=eWLx{T3p~!RSD$lfj#(r$+r-c7dL^T6Y zB);N90NBY%7EF_|^HKFl(8?@3vEZGj|B-ZEDyYEd8sWE90%&lauYMj6zcT(j5wFAF zM2NWzvlwn&zmX5nWI!NWg}p!G%cMDeqPct$T-Z7ufl{nb;2bj!!H4@Wv$*AXd>Qh% zl06xIv6E*aEFc6*DcA2hoyCR>b0#s7ZCv5!85WbR_xlS-7$+dt`?$+HFR*Y>)QyTk zo^d6I0{l1rI&OxTEJpB&J&fL?d8}j?-@=q9LQBRHB=yXER2dAhJ}!a)HAGq zFH2m}=stfWx+jjU?FMKl+)*P3XAG9Xwsr$)toJhM$^JSL@fi@1(g&N@3qod5;`?>n zX{F15TffNj_R3#^OlA=YI9=J!o+5>Xpdxzh$p4LRpDIDi?=LQQY5`p86KF*+gcZh) zwy9XpHdbCesTJUCL>mEFjBR~%3P#QriO{6L6fFvVIyiv^hBvEb;6NcdHCzC?g1dtvY30#w~YZLur{zJmnLck5c~w?c>OxVG_k=>vRJGlC1X)zH1&z^VNgB*b))k6 z(VJl7>vloZtRCxMz>vX`7{MZs`<58Z&b$j}*BD+Ii^ zI%EBBA_-tj>qmC~JZ3;lO+u6UC9xge@k%Sv>AzP(?H$3Lu_N!<0q_NP5n@6sHpRH~ z^e3UmLPP)}l%PNJaRQ>(z-&+RD~(t40fQ^b^B|8--2EH!BDP!)gpykvqqra4@9-~wTXS<{@dSHd%6spv;GbR4dSzl#* zJJD=bOHaVL0ym-j)#*;p;K$*@h)pLIz_TRoc0B?Mr-{hkpvOc-9QW2qE@s{aWMk8c z!SG1%VsJpYjL)5Zdi;~LfRccN1lsviWPscgzYf@b^vb`X^KV`bmI13>-*g=cu2#*( zVmfsUB!83S|5YLRn!xiG@|P7ypa|%<*5frx+iV}9-_M_GDGa7tPvPXT;juHh4o9?# z`K)o$)k~7e0_XA`*x!2isd*6EA2YpVAsDOg>NT=tVqC<5Jr|gov|_cj-nwK~cp$G? zzhm=5df~ir4cC~mB^t-zmL$$8a8xcC2%x@Lnvfko$rn}xIkW37RC)1ZW-C#+J(``u z$;H_(vrT$BHmuyA6xJk#OVDw&LkYA0lXep6$^D;krhPTc0~g+@(Xh^9+&#NGxggK1`HGN%2ylicGd5{PdWN$(bnEiAvP-tK>+qiT0|ekA2lCVO!sPsM z`3WwmeIz;V-ggy$&0U#P<^j76i7Zn*78B07(aR}y1RwN3{Dv}*`f)taf-T}Pj&_hD zBcszCR2ms1zGiJ?REFcW@~z+Hz^_jN!07d^C+@b&itYGf#Rn!CS8Ka6>Ryu~8zUdVTlA_eU3bzy2F|Eils7KXyxNaMgM^*!8A-4O{Uex_5!`*^1T3@}Pe zdhK>^j`FyDVdY9HQZNW;gW;svc9We(_Te~GzIatyZ#K9f#U~7++j~irHXZXP>nxo& zu&Z6OTDiS1OD~q#yPYOP+<+aZeM7rN>7B*n#88%m!Z6+PiSj8uOz8U$ga{`*jQy!G z)sOnRl%7-D7)o(Iw&8X_Gc zKum-h3F=h6PzUP_cLBjhB=OP5F2ROSQ2qP~hrgMfd3Ha$DwG#jT1ClMJ3TA@QT;2J z6VCkvrXz7X#{bS!h)Fnd{Yq%dkAV4NBeB=mh;xuA(+V7_yW}uKQ2^Lzpy;g=(*G1s zVt%$Vf6Botlst-Uk5A5FuKX&sT|iqMD}F~3$nCBGvCLGajz6($5D$Hj7Rj)#Uf&hq z&5340+{K#`JZnHtTJ-AfR*D4ROMxZWr}WD4M8QMu-nUiw*z7@CQ^LU%euDYTUQ^MM zusZ=35Pu@D{*p3fyv#8YAizDii6*C?3^qs|l9y%G<|-8V^mq@E0*VfPz+!?FTRsZ> zWeI=B`b~t5r|*y`>^70aVYBd(yY0>MiYC4*#28}%UEk>RHVH}enLmughFqYA@&@cd zy7!Cm$~b*&ZqS09kqc7;z>b$bvVgzI3!}w>q+7kMgBR@tO z-Sy|r9hXKKZkQqMHeu>+IDr#i(V%}bbKu= zsSm)ND^Y#gRvYm(PMpSMx8+_2BLKBi2HPw~@W;DO_T%fHD}U?S--C&a2IWmUoktMi zt+mG0gs*>8IrpafEJ)8lYU-r0N>6p1gu^UQe#PCj%MLWqf4 z_9EPw5%bC>dKr}j8f?(gG`F288ib{3k^m5l0E~HS%}QeVa}y|_gx2N+UT<ayAq>WU0ZBQ3Mv&lHf&IdPobq zNFP@yGz7U4g?L>NJJv|^s2PVMO2XUIdYLRLeea6l5Tj8y19=b zPw?SJtwF^2!!L&CAvmw_{NmqlKWYB_g<}acCf$9E=GyV~oXoYR+X_&%i}DCD z{#k8Xq}xMlWgwn6!=5Dd09wHUz$MqdcMBZfB*P6hg(Si%m>>?u0_Vj;M-;tG>t~kM zjRc8C+>R&|1qz?>Ldhp%W!@l()~&U#=p-0a9T5Uan`JOYwu{c}IZE_i?(UH*1(sMB zW9{F}mi^%;P!X+Pwp)lj00m{xl{I8d#F5lVf6(eY#kWfwExwZ*$Q-J$>>RIB9kl(r z+ZFhfmmp-i(+=7H<2QlHs3)jeh{By$EWOI`mPHvRIjYALy*(`9x>xaz?5qws^2qLJ zu!7zKIcwX$x`iQvC<1-xOFrsuQsNBURVMISFHIW8 z-m>xO`yJ_5Ck`-FRB&|5%I0I_F)cbQeXbRznA3fbfAf!~nMv(8-ms7$@~QJXYcTZiT{$q}XkGD$tb z@J^EB+`%}1hMw_uM2h6XWNE|98H6QzHU#^K{hVaLAmN;Sd4jkM*wI|uFf)ZUt$R=* z^Z2h~drndDJe9;7zg9g5!KXk8zpT#*9QY7*yY2?GBU$f0QVFF-es4*886-((qM|FN z==ZCk;S2gs)6Y)OcAjWz2DeV2M=0SC=w>N*n?#YqxZF)V4ETr$;E7A7WcW@WM@CDj z7LKK1=8=O6*SCo_+Ci`L6#35w#E~DR4yl~A_)=+Jn>3=w*g;4<#jp;Yn5cO)=W!jW zF?_g3gP5m#pLqmHrwH(bCCg!a&=a@F!t$EOb<_1@!+A0i!15Cj+UX*NSUw~^!t<&VxuTDo20*DNW z*kvce<8E}}T%+TkUK8jc8j1vG4O3Er5UU=bsBv{F;rHh+eK zd6Y~2%ovC!V}?Fo`Gs{8A*hhLHq-DPBZ4LIT-vt7WMTY=bLqD_E^hZQzWb#b{}h1# zwIgpR&d<(_rk<+$Y$zH%H2Y@nVp8T&H8uD(Fgbh~F^J<{&zGo?^A2}gBK)%0!BWeT zQ^oUZzX;h4LVKm%J${d4bbUkdGc|Fg*|+jG$xg+`nVBUJ0sgGGV*SfYL2m-}JKG^* ze9Qqs)eu$29xu*#+YWj#-3g7|FDE(s+iv+X3rycUAx%vLuSvG}N!WGAPD{Y~T4jym z&3|7WMesF&^C%GfWAr}g`tC;WtaC(8`v^8Blt$=gY9#|5O`$ zm04NrQXCrV|EDSP{65fzHA=Y?b#uepi1+hc+C`{7zr-^|>m{Ube&>S)fgt}D@mP9w z++O@3nEKk$oABX0P1y@0R~9~7va~j?nl=-k4Xp!+pi)X z`D>?_U(HA)E$XW8{HGI3PR!dQ@ICa_z;T5)1aZ-6pThfXoBhoqGp{xS14DiZ6lgUh zTNWpLIMKxr!^dl_tpwF_Nz9 z9D21l5+omlOjzf7E~n%l0YXAnHtH+Ud1QC~Rm&s^#G~QJnU?+gHQA(*5#$!>wXdX* z986fQ-K}GNBOEka>*He`ejAL>`pGS70Mff1S6Pl_Fi>X%Q^x5|n%JJOhK|9Wvu-x6 zf%kPr6dchri!u#u1(=|>Wu0R>_3CdaJyJo7I9S5+^fuq*()S)Cm$5R(8~(qLumJ5G z%Q_XZ(GxM<24pHBp>na9Y-KFkD8jPJJSgFaLe_VH_{8qe)BTGMkP|EF;BWRBn@ywo*l^{ECo0kI&Y6T`s4EOjs?sosJKLARhSw(mIN~G{?hy4Yyd~LdBUS( zoCx-<{pnIsq_}~0knDlzPo2i}cwtGHewU7xB`6C0;X@1*%FopkZyCei@_{g4YUfD9 z$S^JYxY+>xV`uu-QsAu3|7aTP(2&4d^dh{F{abfyF>V^~(%BO-BMM;^*jE#kE!?>@ z)i%bXIsQug_nk0+n>)iFy$4?VAzq=_Xv_Ng8Ga?*-A`tQM>0?h3YY!Lt=NA+1)o4V zSS6UkMIZhImmBAHr1Nj!r5Brkf8KwpEumEf&}o z^w6?&MwU?N;i+`@W1J!z46L2F<=*kS)X_TdD8M1hs1@dCUn@$XHeZ={4hN3xu2nbY z$bZe`pHE?UvW06^_sQz`k}Xf0zmPcYG_Y_u2uz}{8zhT>(genSWvW1hQrMp?l_7C^ zd-#|jcY`=`Pj1kq{PrZNM79HQv_S`q#|*!rY~nq^tbfHUi576liry0M-zS(${`GMG z+x;&o{tyMI>;I^BoE-m%jLrr7%8QeR=Lw1F;bB|-3sa49bd6dTw=3)H8 z`f${!vj=kxef;>ytobJqzvDh0wghzLed0%|jYBXUQ&oW)n=>;milr+u_K%Ept>eWs zLXySK2G)IQOEhSTTddo4kU0!a!_2>Z@i1$jF7vr!K<(ZTPWj;!LjuqM#14cNG8#_i zbn75yxmgX}(m4JlG?ADa>MY4)5_=%0wZ%tkGr2;-CP`^>H^@D`V<(tvXMIA&S$iR2 zzG&~pwKqx5sV!p1(S^c+&iw2!(qO^FXfvgXLLD}jc3246T&LqLG*5MTr?LFNp6k`U z>z+Z&nD#|gqos)6{MJq%aX?FjUdjHN_i3pdNck+qn*N}khA;h3C`Q!_xG0+``!N-7+^VB0aZidG6yTP&)hPVZ?9xNU+kVz<=N|Bm4zO6 zVjlA+g*%Ihi@tn+owlXYBECFZD*Lq$>j<)sw+~o#c4@N1FI!8PV=u-c{dFwFSZ zv?N^z@U&g0?%I^WRVO5LomTp$;^jNY;h(ZO`rt!~ZvJF{-7_3dK9YI4$9L9$?0?nU zNsP|tC%o(9X4zhrxZu&g1ZLLSC_6o0$-{<2gH@%wU8l`I=j~Z{4y9g}toYt}&Nt!0 z)1j;f4s6r332+Wcw)V1C1828mL@Ce%Y((wca(CMOs9@aF6RdG6341>9%x5II^4{nw zAG^l$PicqZUdc$`yW_k*Tq|!oq5>*jvpNGia2rSDl?I#8G{k4ZjlJ45mOoMGQXQ#=Cp`)ncA9_cCyh!imQwDrU$kc9xZA z(W$Aablwb3(-z**FNRB*go_hLu1~ z!^`q!cK)3$`MqPVGGrAlELq(if&J@KpAz<1}ApxIC^Np zS?L?`73Vp7?_6g$*msf?9eq$Le7ydYiy6}Eo4;?v>oK%%+TaDXLvEU=?AsHuM7~(J zK*BWJhY(&r`w{x}JQ=UCinnK9GhQGIuYH>&kd2+%H^e|=YXIj)6RP%z{CNGRT$`j1 z*yG_%H{cPac13CgH~M<&1YJ-WUOJ8jO|xV9>{2l4o0q$(-JXyV=5IXJKF|hkvoPBf zCVEFMgoCfhn+Ro7!3F9yDfNn0OeybQ$-f7BmwMP7{n=l{<@wiP60MezO{wn_nuInh zIj#jxJxL8bGtHRE#@{hc8{0E!6rv)#$L#RWbP?itXwxYISMf@VsDoZaO4&~2a(OMkx*?hQDBr8*++d=<++D? z^d0BD#b#|$3C-fFX#rzf42~BfR}-5Tzg;a-8Mv2*4bfp)U6&dQ6{290OX?qxdekTN zP#N_J(YW0Q^McOq%5YB%3}pmGdshU%JU(5C3HlztWgs@Xm-Tu^?s1{5sO*n31eL4|TSWsi zW*x9O_$#eLuo3T-HN~gcfvC(2Tr!iGlID@QbtT+zY0k5}wHx150d*@4lzf)%M7w*i z=7;dWFKtOocke%Kmy}kzS391A2H=QGb#}hS$8jiSOeqTRU2YQUbe8Y0QmQY}9kYsCHj-N&+LN((%s><;q>A&^Xi zMsJlwiiDC?PX5#>JT%3gE?)}2xo zctNiSQ4ex8!R0W{0cOI=9X_Qg+;T__`{CnyTtqnDlyeGydmyWrcdfUI=8Q2ugq8z3 zp=mJ2Di$m`>9uNL+@a1Q+8?fO>E>~vgEm3J#)M}cG#@YqriIukGGAT^NS>iS$Zp0` zyDkcHl5C~0aM_^CU#2-qh&gYGS0|EgcV=p})9=d|>cHyMFLw<+)`N>Fjue!<@jZAF z12+uQ|MID|CwwjwRPq7c&tg}9PrvM)%!J__NJCg>ZLe?!q;@#*%Ol`pS3TD}CJXm3 zsW8PWE7kgx-7%=o#n#kBxKLXfmt^_-#GBR^JYcUNi=PG^4BouWwA(u7su~FG3u)VOO`) zexy}xp5m)3_teuX`IS-?*Z@WQqS#}T2nh^=5o6L57Ya$xDHP;nh>c|9ul92Wqmq|9 z!;MJKY&Gy)YgGyLRbwgeMi`u4t5(+<9)$|wT`0ssDj+)G1EK=L zJ?+XysB{!EJOm{SC!0?jbZ$8wxl*pk3X@P}%;yYapXaS;gs|ile7|#}Sv+DS{JKR?-CY>p^O?0R+C^ ze78PgLwz<#vb82esE&P&TGUQ(?2_y=6hR}$2Xte=yIC{#mbRvb-F$W!*u(=R^9}dgOBVU_3Sv1?Xpf%8xshs2(FqrE1$nHtcjp zcp2+|rS&1c_qSJ}Z?v;EHY+aJg0YIgiQ<7FM+ z&d)_aOxX0dR8DO&^`Mz(%U)Y6%p&Uguh^&NKZ?Jtx2R_Edjgp}*yr713#Fu^+cglW zn1UEDT&-Q`tXZUd+-H38wslpC5Z?fqmh!o`$X$5}+0kn=Yq8^thwD#n-n}dhIu(+Y z8j=JBkrjB8cBQ_^dS|w4k&C6fVn^l1BtjgAo3r>zbNJqXADf`!-S5P`F_zZKol6*L zCImI4?CgHjyl7M|&?0a)e6zDDV@=AOy=Lp9q>p>&VaDDoyCJ`66TB$Bn58Tg?{A_9 z4&DdMTpOA=lbF=RilURWC#;mIIwmFqOeDx@fNI-!N}sPCt`f+^nqaeXSIaLY0;QJ` z^N?QIl$&br3zGPC_^k(KAC$X_9tNzy^A9^bh+WDXXVCQ?73Hou@R#yk@+A$GS0S`_aHNlU} z&ys?4#p7P3($Tiv4wW|SOtFzQTTF-ZkJcMxNy#(s!B%E3Np%r#4<#KY)xTS`a<}A? zKp?>=h|t$r0VL(EAdKv$6aCb!U1hAwVt)>2CVLMI_f2*c?gmLg3Zkai2V-QKCi46v zWS9ICCD5Eee%%t54W6s%&A000?G6^E)EsCYyl&aZnAdvv-uHkazevIK`QC-`@9V9d z_b7wM!M$r=*tg|#^S-Ox090WenAFM^$n6AT`@}UMR5Ws z3c51)1;@EGImf-zZp+X%J28!BE|#cw$=TLdx2J8(E<0A}o-Oo+5#xRG7jNpU;G$=g zzG0|`$lH7JN#SviGtCf2;pKFFJ)_gT4`{@3oI(94kNI8X#0MCS|Pufp0c?x0zF2FImX2oqgt&^&mk^BoYJ2>P1_;0 zs`5(_Hg+mZ#-WWnLpv<=tAg4I#V#-OPoyRhVRFI)6LMrR-0o&f-cc)hF08!M2Wbi~ z)wuMrnH-c!pr2C;ZEPJc5%b>~4V<=u%Vt@3s^>B-L=0bByF6L5^Rea0__n{^0M0nM z1=6?1XCCI^(YrU+Ee&}~Hb)sJ`zT?K75LJEYuNkOKZc9zPR`a6u!3rL#piiAaBab_ zXP@nTX+8b+VpS$ZsIOmCsDA+OtG5q~)RVP{lFcs>Ol0J>tbNi>=jLU0_*j3*#G{iOH1m@B5|58eua!6nes zSUX=ZNkAr;cSDnWUPGwhzHs>YWuHXZI?&+T!gB%0TlKnoR|Za-iom&e>>Sx_R|?u} zNKL=!TowyQJI(Udcx_ku%!@w0bE|Rd6w#dbhpB_Kao(0>1iL?X@y&0wHCJt0+|D1p zvR4~}b0CraO1;X6p44+q6*WnvIOl-;aqrY+CUeAl7x4O}dOD!Rvi_e_k+6 z1&s^dF-HBI`=#QSt_5dJ2@v1m1E|zTX6%6}6tnn74-z&`%E;ulZTq_VD!}m{i#SzM;Jg z4rXR*R-#sZ)@IP0zo*NldWP58VtS)s=|L+8zHN@u0HJ4liTT5GMT%(=iE0bO{nZ~7 z2R@`545ZzMi#Hd2TtTYXFTDGSRKoVjS~ZOI{x(R!dWCm2(u7UK>nKV~EX!+V&RN{M3k%Pp`izO;cmM&gM#Q)F7>MR^tH(_TI3sIf#Phg)!n(m{vRFi5j5u= zgW|!uJ}$k$mElg*aF~ts;+%AU;d@BNq3(Fz8VP2!D(YyoN|hqm=Z5EPdmS7jOiC{Y zgB^|y)P{LBvfCHZT)-g#jp7aVnbnl4TZY4(nwLkjwV+Q!l84qjw9fpn&?0r-FXB4m zd|$hKh85j=aY#MJRD>aYLBB$8#NN3Mc~hLJKCsAZbztn!U`HnuA1C%~a4lLZP-t-K z>UB`8ZY3O4;CHU?`bza{c2y z)0@O2bw#3O)}a-~{O})|{VWFEnL72FB^=Ex{-RuqhQ_q$$-QXO1=Tc| z!NKdz9P8kcqfnuQ1I#NOJ_z<L*5g@% zj_Oh-afSX_Et%5NaltI>#flj9x@F2uV&~#(rSv8K&aN&du}c~2il|2yh1+r^yxIL!eIoL>%f2*C)dcAw@~XScuA~Y7>!_=IJ$DY^nQb19DGDWHQQ4` z6xmv$Wlf4I+y7Leu61QMpbjZoTDWx6XXRq#@>9xB>%+jhFuWTjBLR^k3TN%q?la_2 z-ad@}sE7snWBb(#b?TOTnGv|1z+XWN(rZ!Iwt{S3aeC&IX&CafYU|%@j(FAOX@Sr( z^!bOXC)^Ji4wqsHnKN~{21?1ZM%jBGp>{-u!^D%kc<{4^gWrwRe$Zh5u(CB&9lrl9 zEv$O7c{{MS4Ua(4&bA=|%Y9 z(__Z&-BNsOP8+P;-jCPlA1|-4`}%?ir2MrJcyB;|%dlcvlh)tvDJ-wLU_5U=Fh4aU z)$^&jUXSVa+DEgls9e6GIutds())(ng4Gbi{X5A@x_YI4Ht?ZRRA3qhUw-32AD^UX zF;C?Dq58{?uYyRHf-2uJYBip9JvgZq$Sdn0%2ORbhN}0Ms*LvEih`?QYdAZkJ$!c- zakupzs_>Sj$*<*uQPt-P&=;NPP^7oC*z?8SPF+Yg*($b4mxjAIr+<`szqFc~0hM($ zDX+JCdCZ8e!d_%N;X;IWlJ>Advva7ITvmDk1}UCHM;;e(Gyv7Vm1)Vv9fL>d<=n z&4IE0)C);9x=rIv;@hejTPCfLw5&L{`WG+-JRPPOt0UVz&11tzz{+OW)AXQ~R(*J{ zD?-tZ&H3u7s3_nxWpVrdh?eT2vtRrgC0XiD6q#>NOSvO7YZLw|C{*qCV zr#$ARU4#o^yx5pq_Fz!Rq|NLd3#$VV1|{AorQ#Yu9A@ z@}nI_&W(!LyhwI@ju7YJkhNAZModI`hZIJK7)2uV(Ly?KO`1xc80P z>K|?_a%_6rq?d*bm<$~-HGe7+ytq4bcYmh&Rm75C!Az-wcINb^^Fx;*#?O64#V_~M zT9;=__9uPTBvA)BqKED0CM{fl9ImSzd|dWfr9R}RnIdgNjqH@HlbJ zlqBS@Y=%^9>KR)~rg4;}EDmSqn`@3L>*y@4H0zBHt%TYXPSM6(8Sa;#EiV7*bbb~k z$=bQpocMIP!ewo@TY26!ovFn^6OKFJvVpt((m!n7BCs}SUwk*VC*`@v8tTx{MgLIb z?U|mwVaQTY@vzSLgSv)&ui3i|Rlzl7go>qZjKb&=$bA!^fdi#+RgY~y(6L+1xA4`^!;;De?Kf7TAPVNq`R}w#9#+ zb>`MpOdmG|ViVY@W^Vigz}a3gw)8Lx)>6u^Q4nuOxR3RC?d1WZdB7({9}O`aJX+ zMMhw0ORox%=vth!lXl5`VK2?1#H4QFScoZC&$_2>b5-%XMM3Y^4G!(^Y9tmP_lB$~ zHg6=GZWc4gqB)y5U~WlvJCc3*o8mYp}ea(w(Z(xT<(!pU85r7ELbu!bk8rHHVW>ez}?iLm1blpP>ln+`D|vh6sr?j4a~a zk_BC&wTw@518*Cw%incS4UK*%sFJ#%T&cqyo25=~o35PIHVJcG$fw7zpwjbSi*g0^8c z)-_I>uXS4ryDmD*afeRd5_k;&0G#EEviyOuxuAeprwAwlD2*%v&i#ppN<1@ zv`tCtvlLqecg`Nq3a(dM12*fsks~qs`kjReCK);Jf6X(2_seytV;ILeHriIls5q}r zQQ=KBzg@Ss=IQGl??EOk$TuJf6R%Xbm|A<>8F5_C%WLUV5`F*d$6CGK`j>}{g5t=&BxWrO@^IEe8=Aw>*;Bl7cyr1p-S|0M^@~l)m{`WWk0ODmhJh}S$l25 z@)8I4c;dWq7c&*vDh|clxr}m$k%*b93*4z38Ea|G^V&<;ku^ax+TZ$Uvetx#6ZTLW z2{Jg9$#Ju71kZP2fljXcwc%J&FOW`H#g8CdqXc0ZCw`7$Py)vt9laEW*n=( z-eL#8pF@W-W>|%i!Li9yAiru+ROF42=mXX$!%0x+b;g-98+0oWqP#G zHVBHFW~HrpCa#rlrm8vk4P8y?MPnC7r{ria?}XE9x&|gPsqwJUAy}<6dC9p zvvhSl$$EVvEhlYUPt-_!X|g1&WV=MK!TIKcQ@fp}SzIW;!><*#kv!wKMnMPVCMFlg ze^sJ(ud?(JXC zt~FNkJ^$e5kYMTcuyQRvr-J)!sN9POE&fs5wWp^mW=%YWRR@Jy`M9`qO7R^wTAXjd zvJn=SN^LA*R?_Muil(o|8HbD8evlyE_&#ORb1yzEf0NTz7Iw(ECod6fUKU;VuR`4O?-kCzkT0*8id=RVB}07!xXcS);&J{rdESHJP-G|U=DR~a zVXOQPV$$-8i7nmc%)<0et9T>FwDg24a28#a{Qhv-(f z#)+|5(405Yqz;SB9=w^wNBzc+q{ta{)hwv!^VOFwW}DS(!e7oDy~_q#K$S3b@;rF6 zkzjgxz34$6jKOY3bgK!rL{i;C(gmro%DHMDcCs_m2&4 z>FE5lf0ROU-X`2ZTzfNzIV{fRfqLFKy=m2Z@6Ji3-_us@h*ZK+5r5emy7rb!kGtPF zCr=^HFsGuzg~?#KWMn1OBTsOgoY%)Ln5i)*9sec@hvoVwYKn*j_K1h}&Bl1xc4j`@ zjSowBD#6Q`puP5vCuu8GJj85s!0sg#+4H+F+@_(QcLs{9_Ovw8N6Z(MR_^FKI_Kvw zS@GOq@13RrIRpsxPy{jzF%lJa6`2N9m-Dyg*LDbqSH2;1};B~p>?Yu<6q)%wvHwiV;_+gT^=ei z8R>%c7GEtlG3cKynP*Ja*XlR6ivD2q{{8aX{-(KVqGM` zs`&PoNBJ+gD}~jNBhR8cyKKVh<{tN7CR$2n$Ii}+D1GKKHMGpH?TVUn)%3Bg%(o*o zU$k(`_UA55UR*Bp8W@`rrhPLUGclAm#P-wA_gt{%7>vs0Tjote)?A8+{u_(5T+cEZ z)Za#;)^%J2gm$m%0Yj8X%H7(Awv3Y|j zA4Yi`9DCd;<&P==w3nfp-Cwpkccb)OT3dd4j4dz*O=)KMgW(A~@J<_QcQwBcH_y}l zyYSwgwu=gz+E|FOctn1eRYO=~qr?qs_b+25!%hsP8E{gYIi%p*-m^Uop&)8VOx80P zUslgfDMoRs+I;Hk%FV034YjK4?sReMr%Bb(JlxuCU!E1(@!)m~Kgaljc#c!teF#lq zo@xA(x_i!edwPPwEVBQj=yVA*wp$FM?zUd zG3}xAu-v04SFt~1BsQ$**L&g6OTSBL3|!D2Ac^d6HN;Zy~eT^aCO}>pkKp%A%o8YQER^OYA3* z{UFXM&gF_r7V)7UuzkS(c;aZTREC4BPw|cqQ%{OT@krMr5(TE+p&-W9rzQu23)1fIgZg>ICiwj z^0mm8(7V(8h3QgVxRP0Ph0U^X;eD%-;wwg3HTm-so zyE^Nn*LKUiMaTCu*^xStMNvi_uzx9UYr70oDo}K@P0CRu{GN&cXLZUJ$3$R2XPNh>&DA9j z=8??mg?%a`kqZKyqGZz}>lMWf%v$yZ zHA7q0Upa=&;V>O&G`*+^zQ=gb4|mZt69~Lw86s;%*{Pl3kvE0)bfkr2Q9Ot=B41ev z@_?!nR!A@J7gcmWon{tYLH9n4>ynGqq66D{Wk)3E07daHHVg*B&L+=eJmYD>JgVq=XDME;KlD3@Vq_TUr+37^RI2*${*7 z2)RQuciQyZ1RmnWWYb(aiNV_=iA1+h9`k(rZ}oC#Z*-|jLUL@yMA%T#i}9jV9QeXZVI$;;L|MA-gOg=B{n~>3tzC^}y`8|;`(pVY?$VA#`9dQr(p{xo=Hbu4)5q<2v4L)Z!o0x_aGRs<)9WQB4~YhDpK)Q8ypDyoq!~HOOzR3iUldvA;wj#tH{geO{^Ln1DfV9mf8gt1?um5IDPo@GV3A{6c((EpI+zq zTG@F0CuvK_LHpVYPzvZJ#pteVQ#3?sci=)cF16n><1+rO?7)rihEOSYTxAr}1TPP1 z7Eps^l}LS_siP>YUW66GCOG!;R6Q$1bhO4-Ef0cU%+0GLnl~;#>)TyyJydscovX(B zLlYHMK%D{&tHV+aH{;!bNO{B|T|;&XhD5SyP2~DVoRFJ3?aOW*NlN~7@HJ(j@-ZAu?Zo1r%F%S`}lA8BqOd`40(x3v) zD8QPf)(ZQZV!8x8_U$d{M&f3AX~&BtpSoT0vd}gCC6>!L74ssfUWe>4Lj2{W>Vk(- zu^p?+kWvHv;0lxk5JS)H{zjmR&bkchBYwxGL~A!9k1F(2TVxew6KmudVXZ0FjgW_|;= zv`DZRL8>d3t-%WiRCKz1x_D`nM1t|?SL?XX5ubFYm{slbyYbg-r-S;xMC+gw-Y(S= z@_PskkP77hlDcP!Uq`bY%vBueTs=QI%!H}NVM~kl7Bd^`%*L59gX@#|gX{MTmfF}D z&%f;4WX5>*Ut2X{BW2zZG%KX`jMR_`*4?#UPJ@~qrOn*oVi9KUjQbENQ-^mW+)Pkz zW26js?uq9h?z1g&EXS|x=a&DIhHp&2aSMT@ItJL65V*VXQOtqSw7TnNNzA0_>~$ZRm;Htbr)@BlfXTOBfiiWtXy*u0H(WTbzLv_yp@ z=~UZcV#6#4mW&xJumdG)jygppMJkC^+V*X;l3I(%dCKv(6rw7R-p!Ni$d%XmO`!Io zKUA@N%s=ccG1|9y(zMBJ0i>Py0QTgZX_I?Ue#fMPY^mrh_p)KJxN#acPqFxL_vKTL zcOw+!6qGz*U;Y*~6f8RXV>b7*idgw$3!C^QX7buVFoUhrdc%Nm-tE9`k!2+LOX4<+ ztB#y1t=?R@Z;b#rb(CrFH9r6o(b!m}XM#aGZy~A*B5*!-;+@xdB*UCCizSf`!HMvt z&YX=y+F-NbNM{wR#z`Y5%Yw{d(rw%7IKmxFbAW27?liyW!T4?cDrHkbK{-oLVMvTM z4M}5q`TX=!e6@0lJ(fFOd31PcXw-JjPKdrY4r}x0Rc1p1>>yU{XU?!$-=;Iv!Es5& zQY^f92Ktl>6gc0sXBc@iw4RITS4e0ADoPo_p<0%IkXz_H>VzudT^YDZZy(yLn+9eS z2_CqZ|1fz}V7P}Vd+64*r-k(TK-|9dux8Y0-ON``Mrs%&H)U1i z)^<5#JNw3KMP-?+)NPeQ?(4k*=OZ}tE~tT%dHbGenT!e44Gb!>u+?bzO1Y}Z>J zhfK}`Qv8qn^uedQ0=$#cnIG2<7kSkXRi-81Ff4ezNM#+kmf3UI?ALG@<4%KcG#;tVn-{$^P^*W zhes@%o~a!*na*t8w0a1>x6qrLzD8qugmKxkU?+hUTugj|^^>H3N^^g1<4E}2=00-J zU#npIHu~_j6Dv3RP)>9%-oK%@tg0n@6GSWUsSWU9NdSfSYCz%rDhPlMuKrmiS=OGO z{;e>J>-cK9xUn@6tcp-A>`EZGn zw>90dVxJ~{G2YY2_1fXH>ykAKf)Kt6ib$$K@6$T$jC_zOx%y^HI$qwmUWU=$P3?h@ z19SJN{rE5LUw+_b0*@4r_dYz2*{4$qJzj7p_JPL!vUvwWbT95{5kl#h#{wwo|LQc_Oqd5~<~E{Hbi26P)LF!@}||Tc?04 zB`e)`Auayuy5cyF2(O1vIr^r}`Z)r*I&D)L9c_Y(yBA9kcQ=a*;QD_GgL++IMPcw? zZ#vd6{t;jK_GtHDnkN_!#aNnswo;{urk(6SEB5mH%Sj*nWZwDPCFGOS+J{D?BsJSx zw%pIrE#0@Dx0jfc>vd-w?ZM9h(kV#|$9q_~W6NGNqd}LQB{f)(sc+ z#LeB}N#?M4%7E>B=5lv`gzj}a&4LQ$A4`D5QAkj-K7D)I*0JQ*ohqca_glvcD9x~8 z??Qt1i7MU|i=)-l@>QNEhMziKQ$Ry)G55TMR7$UXQXREva!kNI((=02I52n)dJC`8 zckd7^nV-GChiu*Y;qFhHLgUfmuT{+}Pf&G}1vaT<)3ew?UM1ciHd(dzA$Mx&7@8g^ zihbi>5w^*?KV?Ut<<`cx8&vyB9#VV_RDM3koT15V*zBs9P{yt>T-c&kpl&>tdiZy$ zZ&33feidC(`)pJdFNGVy338C6uos~ko7Bq~k3*N^zSMO;KW{^%|D>#Dgb=tClI|qK zsVX0ItqNzoUiQAypI@xuJp~dxUyFR(i(Yn+8!YASsY=zV_QN$+wX#{m@LEur&s$sG z_1v>+str20zN5FbWk~tGK2>;uL%X}q=_-L0kZ{8CJZHL8z%0pv_ie+gYgyB4rN_`h z=OQyrHiJW#KCAmUclpyeV)y_10+XS>47nPV6XW8Fn?&BDOvnbm@VylN_N(uS)BjTG*&|--CzeiQwFy>MCTjfh-EZ_+o2HkTPnh zTgU{^o$AS|crrfX$OAVI#V=c=&12PkkG1I|kd#3xtVDHWEu_AW1>ht4thu6M6T`Kw@U3c9Q z|3Gb8dyZjk1|s5d;mf%2d2juWV^mZvh!);W-9!B@DRES|ZmFOfzf69`#J-)EcPy}q z=1=vMqlqQgZ{Zis93h^}|S$lZd_vRhs!% z@)h5JxQQRe{zLX+M=MqDchMg5=7 z2`edn_en@K>6Ab(N>Pwb+S#>L72VHwu3oekor_yUn(tXGSo9b{DkVWYM4HShmYjDR zyn$+N0Fyg4`{2>{LWOq*6n1Zd@Z#mQ#IPkeEmTMj%Gw!O5T_nrSBWcA9>q5Iu?dra4?-=%|W-VRCgl@gd!$v1&z!^nR;_=vzzZ zcxbxk7D84Hqy^@c+W1D0_V!vQ=os%^hY3BM~%4C5@WCM~ReAW;&! z;qeeIH6OT?*+cid9pflk;ISa(oLrda8#fmUv}15=GV@(BI--zZErka?&(oX^`^gAV zFz+!X*W4`gTO=UIv!`D9cz_rT7OZ#MEmyD`S$XncJC3S%!drHG7F^k{Wpf03$KAT zvUor~wPnxq+OTAr)70J8yH4KPt#=|(qDW&?zCbh64#Du|FZegUbARgj_Hk6LUwH*m z-T5;+(-9u$@U!4kBAy3e*21{rX@vc(orM4RJM)?qWOY-mIi-(0=NOcdN4oC>pW>}T1*mqtO8;XAw2E8biLRX;O+dKuUx@j~P)U*I{?Ek33?)Nxp3@3y;2m5urezkg!NQ!{~KARNeVk=FN=>SQ&8rX7fq;D^PS-eB9y4IiW<^4Ppw%?}H zeNqwrQj*>KUD8=j=aS^q@P2ym9#2$mn(C;DRZQO8h4hcEu-;<;2@!R+rLW(Vx>31n zeed(Y^qNoY>+0avGYKcre$9`zc;*6x?Re*uMeXBnvqJ7Yt=x2CnV+wWilG$J zl9A1gF{^6glYX$Qv>htsx%%N^rBIb7iqPz)s}DL9uf2{aJcjia+XSe~9dWN#u!%uG zbb4hWXd!^4x#Z|<@Y^-&X?z&x6$6sD${D(sONvm z<79hUJ`LbX4$vhr0ayl)J4Y`@)_H^5yRI=G4Ai4!qY5Z@>W)VPcc2o&dBXfW&oqhd z9bVElTmtFVEyw|81r2bjrS>fb^fhxxatkN%=Wuiik4f!InR^nk)ls%05Gzk+Z}}(2 zN8H5o^jD%SD0+lFOD{6b3K=R3t-SBejm7`vV80u~J5toVt^8IEoR@fXG(G({ug2p~>bn+L7`W z9sR4X*~KmH=S|c8bjI)M#RHkrx=v0z_U6w|I`p@8sx=9j)t3yWJMmTAF_o%gn)_c$ z1Oo^qeT@r%0hC^D??IV|s8>~SG(>R#c(myj`yAnC+NEkjn`%@YqBn zqbmp>M_iWO5>I3PVx=0#Ue1nt4t<5`24)f6K3X`~; zOL9;u7|Gsj*4Q~p-~qDQPD!pTz|LUJ0tRt8{&A-P2fJr7PMMIgJLF0H&G7B}S?PJ- z+bVFjFOT$ZuCyI5Hr#z1!5p^#4#}c-p_jrB$*mw6ZSO4%Bu#sWTsIW9 zSbEkkKdq`lreRNT?6I5J#jok5!X+W z0nX#mv1qB<`8<_g?uh!7dNI6Y7(AQc&)ea$#gXN;N%*sxcYD!MhyDdJNU&23U0J$d zlWhO(r?esbBLuMTG?^jPFfqIUUupVw3?sxBu#(E#qSI451ybc7`a6KCY3dcK2d|L< zxiTU&33$10Zyg10EeSt(Ze_PY>%v%yhng@%0Mjcyxh)z>m)_pGd`|#%>l~p%kuJ zUpUwEgqHKyfctjk(AY0W$ib3LkHl+L?^BGbj8stNb9lRmxyxA4we%Ul}KY>9AP z&#NCKUW~f*khxEN6xk-01;>7(NTZo{dbMS(yzt`zPEw2YqLu)a9==pbTieDTs@0Ef zp1%FniS?xV^#D5{X+Z|Nm1`kE#^m=E9E0`zZs#HiN9(x&bg{Z45DoWlCBC$bv!q&@|80pEsm^IpZK@tSTyYjvnQPmT6y zgiJ1*`uNfP57`S=33e7DW8C;~#a%Z~+bPn_^Mp5UxE^fODh@}Km>W`vK{FBHo`DF_ z${2V+MZPaITXnPKYbrcli@=T;Sa|Ql^9}P-+S>|@OOFbXTN73qPt7-jM*YjL;<8(& z^sDD(cPE=x4oxo&&VGt9Ya$nWYprRDHRQ|F+(@*CvRzq1C2N;GRAihna( z?NCUq1-JJ~Bk@#}{ReFU{1t}8=Oj^YVtOo2r(CP=6^u1$(t!V{IH+Qh7#V%h9Pwkg zID*?FZ@g|k>C1ddsP4w7kS0BZSo{=Ll9~n)c*bw+DF;Su8ER1%H z`uIM(bd8-*2c?XRms0XrA|q|I;G){!-Tb zGQ?r9Kh`6#Q~E228-kU(>q6~NTdM{o|2C1b{d|#_E!OB~&DCr4bTMFt;TNb~^~_HZ znL2jNPA7ZYyOr@en_;V{=bs)?e?Bx>FC&w!R_id0FU+kl@h(UqnTdM1oO~e(eq?gO z0TH6srztMKi&*~mN77|r8Tu(Fq-OkZ9ot|>4eV*+ee;vxbSW|3$fzHX-J?~?+ zspsZDY66oN2l^agrP)29k|-YT9!k+)Ch!?VDItst|7bRF42#0;-W8Wm{|9XI^o;FH zCPvuQJ(_4Vx};|i5XbN5v(()Lbd2D`czZ^cJaY zD)m)rRzVZ!{FO|(GbQcql_l9DT<*2?4;bw4Px#Z+7})J~*9_5&mG%V9Y&Cp^lS_5x z%@kXm)VcFS+wR_0rG>NnKbSbCxl$JOsF@wL9(e&+pYFqJphA*^pS3egkiu+q^<$D# zRd<$cE&XBhA#k&v4T;@hU*>6onIcP|q1b!C9lXQ{so;o58bw6(|NnO8st&7^Lp8X4 zh@B1>lx#~+#-s#)x{fzUO_;5Y8BJL{JHzRj17IG5Wa>?|_bq))>Utrcdzfv*1tdvZ zp5PU;T4t-4zfUmBCKf&m9slRt3@#gq3rg~q85)e}k`vDudzOlPt37%|9w`I{97fF? zd+Kk?rT_1~e7H+J?b`EqZ7qa?r@}2~xJgtgYO0!nRClp5HzZ>sgUyI(+*-{QUx;7v;O+|tJ= z`=9x(KWl7;z>@Q5x!1p^CY97N-SLJ!1C)@COa>(^t5kR6H8x@)JoV&PC24j5`Qejf zjoq8yL$qBw?NRc3r5SA?hY+B&#H@9a(0u~flf@tvsuTPx= zV0-N}CTz4BBs$i^x)x(3PryYMDOnhtdOl?3{v`>7S{Tm!IqDem=W@`T($avZ4o?xW|*aLBlS!%CNw?;V_%Hji} z^3A!t*^IXwW}SXuLDd3 zz=d<{0dU~}uD8ie8PSn~7I~}FVDX9I^kFI0vD@2Jj}Vx?rL`U(vc$Rn_~`jHU?#O7 z-3|I|8$}?`_GHr5481eG2>{>{t`In}nxar;b}X8v?K@sUEJ37#mn<+y-0dn350pyH z!mzlInS0XW?~hoNq%`6Y>?&K{`fQ!(+|8u6HY|Y80buB(Nq%}ubqRoP1fNK?p^TtU zeUrLiq>^oV_(1sb+{m(A!`+C5&kQ`tn)Rk#90_MIVKb;fg(2{eV%vi8iwS0WPl7H0 zu>KkoQO7?=Tbd5y$I|lgTe>QrNdnO87I=h;`|dfo+h>H5xn*Uq;LOIs&ZlcX&3Xh) zR$es!!g~dYo1W@gOg|Zf0S481KERQm88lDC&6w19m$#~xU;wm>?J2|kxCYo?rPp8| zuSD!Kx!8;^0H_Kh+yJeAWh-L;(wYKcVTtSA*s-a~rN4hiIpb@2OM`L%9?;0o(O}}s z>gq;fsQ=v)rlKXHh%~>^@=wdo8Z}_79t<~I_R6QS*zz5uBvK?*-C!c>Xyaq}e9|r7 zr*>CC5yTXc!tGt%XJp%KeKSyhyF`ffj<_j~aC5TYHGT)E0j;`o7@?xn1V)zVXy9~Z z34-DOozsa41e=ya$$BAvaT^`$h}z^p!j9E@D+h^O!Kgt-0lIJi%Jg~mDO3HO%l?Xy zm>}9iqWYVfha`$y+$Zo&kG=SHJ>W%ANX$@-JUfH8(z) z*T!>MlrfSM&*Of6NcOaJ`{l^0*^K^^PM`l1fnwaHdUihBU5|1A$|uUCNuCD*9Z9Z~ zO56PW{+{Q{EQO~jCPf=?rG4P1&ayh(Wim&Sj z$}zYoaiHa2ZvsF+tAPc=Kg!_uIANAjGdrJ7!s4fnYqrDO-Rl6v*GodGkO)D*)PT|9 zQAIjSqBj*h{QF`Gj*B(Ys~u8Y(xr;h0-p&@=>D;uPcM$6nq_S-)T&F>rEOWmy%>@- z_Apakl9t`Q?jF6N?80tG(2^mGD<_M;vc8F^n4YCrI-%kWRp*(*43Y`Mk0Vy6mLD2* zOnl`?3{)}Y+@^587;D~QM+tgKGQ_b(EMJZ8G<3wv?~1?B|2bK6{V(N!pI{?+z<7_g z9UcvTv}lz?xi}PHQ<)@?F!A?K%z(_SkpaR~Y()X`7HlCorXSb?86qxA{0nz#T%~7M z&74~wV_>dfW`^$CzYF86hfQ$w6iIrI*LLve^ zcj>=<0fOg5#FxsxA0a-xwM5xn2DoHMNI4yXmw)tiTXJzg;SCu*_MZ5^2A$ZAQb)Uy zOx}`lYK=-(HB zRzq`aWzVyJRM=!z7r8novcr2g?~?!i4PXqdGG7 z81~8x=wm0F2N3#52B$M9(C1*h05lP=uiz7nLvY+HcS1)MOa7-H6`n^4R|amqZuY?Y zgxhv8wk+_es!_ms2oC?n>pQuPHEAas34rXAv6(TfMU^Oa4W2GAg?eW);Lc!=35jZT zPqFpM8{y1BG~YTQ;ks@0GfsN#O_ShRR}WcTB`^fMX|ov;MjOShDO>( z~#_i~7t%Z<-%rHN<&(~!3qidMzyd8S zUDx>-ikgH>uWL2PTvGMu>ZcU(EEtQY`)3e<>;+8oj#0uo2lqygO3u;>B)#2p@aZ8u z(@%}i(Ilu8ic$W67d&E;h$Lpni&z>>R1{r|6FEV+z_gfP9uw|4&RUhV$ zPT)LU#MBl4T|N_l3lJk!+!91lqtl-d#yWmFCS6;R{FE0=kk7d?M(Fl!Q}tF zIG~U1$!~H*uc!Tz_k23$9`{>~Y{*TZ4j<*{u5P2T52bz7`2JmV0DwibwZHSbi}^X2 zD&CJ-9qLI1&3Mrd|27z(Jwc+!M6#bE(IXxN_2tI$>-@`|y(+N#T&C*Tr!ye5C&Kny zyVyHvu7LYsGf+a)(5P= zxw4p$oqtri-d?j8qAy!idmzuyMxHz!SA zHUv_ii$*rP8Pvb?p@&%91yVb@M9bsvg3P=&E&;tp|L({x?^Lz%p(|58J);Z&vpXGc z?U}Um2JWfFfFTI=`mvFD<~a;w5yjhrGvk0{-1c6b3Ff{oC(vd2`ni zWqHjBl$la1=smhU6nPjV)#}g{|8!ZKyL`eJ=*o3rbfH7tn(EVJva3!2O6VBTfos!{ z;fl~4xDCK%0NpMY_errnVNF~?tQON*`w+?F>R~7L)h00@dv-r3M%C0`B;%MchkHe} z>8aOeU;TDP#8?#v|0s&-%jQzZi0LiYU&!pe-X|w~$}>;U_4hmhp-M_<1|%#W3yL53 zl)~a})^vD%Y2B*3qJdUj>FvD%JeNNNflwcifhBEt4a;H}#>GR`bLY;Gf|C|7aYmDW zDte9g5=E&Euc|8~uL;Po=)*p)TIxNnH_Y|%o3oF1KBy>q%gQvUFTDxBnGrY{MyzCO zkB%1(lv_9R5-hTM;Be+}W?un^3NaKjk(XYx@&cnE)E?RdAVNLvXxeesyG28St8-Kd z&K&%QYF$6qI9@Ur3MI99C}L!+uAEsI1wsQbek;Dhjk~*qHad2_3bXS7C5O<-D3!Li zBc#&}#1riA1)&}W`h4$73kBF6UB^5>vyds3;5ypp%w+F#+79BkN|7nViz5LvVD27W zGOcqC#9uta=0r!c%#6?MJzd21J0IlqLI)b-wyRq`#Tr2US0gnx&wth^O5Zeme%gcL zkJyxH2dRd@<(C5buH}A@Q5cE4o0U&#>-ntt6)3xJ%wX%Si>SP>e03GgF|_Gw{^It? z$WdB_FFZhFb2B%B$=^w&`u`Y+8A2x?q`S5@Q-srQAY~jhm8@h7Y8C^Dqz5#lx;Y<- zL5r}r%-Wz$9;C}pgIn19^O@3Tf2rw4ut?55jmMpI!28cuf|^<8_v&3Nj}+$~Z8&j% z5HJfovVrguU+;b1T%IzYC_I_cEG1~)2ha_qj`O`@0}^GqvAEw6baX~h^@3s3vM89W z?jgKA=YykX`L)1yR-k8Q^b46ZTE08|f-%9`R(Fpqn})PEw^}RCpHKC8#HK9G%+5V+ zl0z?gZsGD-cBi8@;uZ8S=+$ewdB-|2;Gv#q3t-1Xi0pDWbo1~=A%wq<-H@8kNu zn0$=@8)1cd0Wd zw)+F3@aBmULQ57G2F!VYXd*SJ-UT%0Z`WUM2NA1Rcl0Fw^}NVr@PSmr;K;ZcN|C%R zJ`x~q6d%a6QBDd9p8AL=C9&oAFp$ZNh!C(YxX%*Fd3fY3m7nQT@UNX^!c__dUPc=) zeeT&iX%;%}e0Y*zV0_bqHK^GKB(Gini5WUr?$BZ&{hiObFf|-ncx1^q`-XtIzA9O> zuq5gKSdv>L+{{Tf$ZJ$@uk?OpaK6~;a2}?I=YEv%wmlPpWSsn^xm{3PJ+nE?v9o;- zziFlfgak0W{u0KJ7#L4Rm1J*3{PxP#Nr0)6X?Joh>cxNfL)P0ei)m&RBO?Sx3kr{< z(1ofDYMYg+Mk9II8Wv~IY(fk~=UVoC;{dhbTKi^{4+#8Jk6oip;T<`YQzySdTRNY3 zYwST%H9SZmk*pGMgkv^dCl3p$QcI2Rp+^sBU=Y1YH+k6dd!w`;f0srGl?6N?P?tB% z`EBJI8^~uGR2MXeii2V}DQe<^>e6tP_Lft6_*5H|3FgZaLp=+l7T7L}U&vfc@WTG! zp4^?l8%z7I&oH?pHQ=uBkiyS|_@&O$?iv1&0Bo<``ewU|>Rfu6h@|nmzVCroq5vbN zA)O2pn8Ccn7Z50t4y}oTo@TQBkHZoAmb(r~CU@5sUN%NKRxQ6(Fkjdh1!gK$hRrnY z+UU(PU)SBnyK0Txx+B?-+Yv@?ST52HQNVOTd)qawri&a@X@6hK8?u(UF>vp05A}|c zSJi=96e)4{6BIM;o+VkVY`^yNU|0$_5J%)tSL_{#~!Gi?U6B(s&RxLw-y(u@CF zlT1bnh$}*{E8Y4oxZ^DcAp`9amA}Hq$}1U5HqgDmilG<24dQ4-m0N@F?Bm64kLydAPGiU;}5Ks5ZxT<#o8oDxXygfX=hVHy~5@n2K$&=317! z2QIc82n*;}#OuF2F>p{3sc2nPYiO*_Sga)CD>gvB9B(l?y%0sl{Obl1ju~cu8QM#D z=6k{{89gn9>cJAeWg|4By#6V+C0=!58)UEwJW(O(qA6{Chfk2^mV&u?dr6vvv0g%5 zAhrlzWII$*nGjd54E^#~__Fk7oOq2_!(-snQ*UX9w?^IWaPwIKx_|Bpu(CZz5##cJ ztE&VEF0ykl9JS2*Vlx?x^UUQ3v4uK7F(P$Z4%lEVx#A8ncMp+!Z>Wb%v3(Ni{WjYkCDEJIj@@fKY9V3)hUg*1W!2C z{&BCigU_&x-1LWX5#B1gA?iArjCfF+h`R!t=gQY7_D8t{n|<3}>)&oiY92l1f| zg@n#1zUanglT66$py1(>5JooQ-N>F#^rB$3;?{GSK)i*ETJj%BG03N%Hwp4Ebwefd zTgtp)#A?fC)l5~2mf-UZ7q3B(yiC}??-xL7E%xfsyMs}>e6Z^?REN4ZB;+$LpLaf| zPh{mD3vhgE*aMM(k%p%{4ipH(W@k{V_1!rkxaWTi3fnjdOVt+RhJ%x7hlNF+i_HVU zn_@n#?{xu;1JOxA8ePn}gYX9SgA@tcm>qIWxGy9i@if`)dRf7cqQ>S5CuRV^Ce#Fi#7-1hB&T<1<}+q4eT;* z0AVY-xrDg6M}s8cwX9Tp_;>|ZzNSxy3lv4(txQtL7qUYwYTLDYsEGi$c!6-tB;m#X z_5ws|d0j_)H`Co zx8g&?2R6KJ$_XF%Oh)NnKPzyq7-IEw>25Ngd{6m2A=xP&g=BNsXM4Ojir&h%78Y4o z+LI4$-`b7oo=n=3wGYvlGqE%szf`tL zoU8>j`_dpfSnC;Qv@w>Fs!YgZq?|~4X-oF=fkfXrwz>!wv;|bb;ir*a+#VUdU=o-ZG{&YFU$l%BJLI={2N@^x zK^~4OS3@e!M1^6$@QN9eXi~BQ`}*j{{IjoGRJ@r6`rdzN<~u$Zo9Yov1wd|Y8@^3lU3eD_8Tirw#BkAxWPOwX^yt&L@tQo{ zb?VqqK855#?lHejV&dxLafc+|fpkE@CjH99feSz7@3xG2q>!Dt)vhp^o0+(}DK@!I&ag#y4gx|nl0!Lm+Ycv=pgn$GRP=Cd`sB6ClPr)BU{;yjtp)- zN&voBypcT$N=X`Dno^wqYwZ-!o+K;SxV<`RB~TQ0xihlyh$oyybqVwr8HjzWv;0dL*bL723MnJ@nk%F^p2wh_5crh0fBuf7-IAH z9ai<{@?oxghim9~UXv5wlHP7oM@ z-kf_(JU{Xh`%TcMg|u%|jYwU@@6u4izMM$5nm6kivH@3VIx~M!F>~T0{=StPm1y_Y zJ$30vD?^Gf+tx0B-|CfYB_N9_N@G-9NmNogEp8bz#I8%Cc8Jh5ERJ&JSz6aEww;Si zGyjI7VOB^Qqdf7a#~gi?oKy0e1!AH9O%+Qra5;%AvYedL1+vY_;GPt)O~xr{qA@YP zJALe|`4IOtg0r$z@OjQ^QZCJ#J>AWz?6Csbd$cLtEvV@&#e6|Ls9<%!-8UQ4%nbXg zVVY7eJi%`1DV_WEWT)onY(bEeS5rT_v5yZgi3)KOFd(u}WF!jxW*#h?3SS_5hO8eN zpMXjZvB#FC4{?$rI)Z7>EQpn-zUngPq={Kezgy2%m&+Krm$~Grq!^A`Kb$;t^NJFX zbO=6|t%h}dXUF#6cADY%_C)I7ytRe5>R!dv`!j-v1`N08j8mI4t}b8^+9K`)3gIJx z!u>!13rT-}_(>7wF7x{l?7LWU>rY5g6?A-IvH!qxnG2xui(4OgDQdKWwBfYjq^^EXBS__CLCKuRTbU_1GYlp{boz#k5XE(p3wzNy|e zoYUA|r(b?_c>^#xKW-i&J&bVb6jzrIS{}NkEY}4Ih61xy_T{otkY?IlQk%(EFY+_X zYkL~ZE=`OkKpa5>YIJFP`=%7M#FRv?L6Y`VE6W$H-a%~1c^H4I>lzuEbm=_W_j3n7 zdbYmNiUmgFEEi9*wt>VgHOYj5vDah^&7Ypzij*BTxZOwe!lsmIHHanrlLFzEx+P8x z{WEz}^NV>kKe+5W3(drp4CqGbbNC7e^A^n5oy&VOrIy95)&Hy`zorbjt46wOrx zjq%hnXyrxmr=A)NmOys=c@mOKzbK4!GX{GHDsiH9O*UAcYGd=qShMj8uds%7#(+b^ zqrS#6?T9Rc0Z1OI?YN@%m=@I6?@ppNB^D^DE1d;p${Cr}){}O|sNfY>Og{ItI zX*qoK-b=ZG_U<3ERE8OEgC2z-Kg>CKRAdAyZfZDJK>Eg6R~=^E{ZT@_LpRi6t0fXl zw8Jt@hFYrIvRz|xP;u@~^GQvjWbQL{<``}!SkGE;uGPG>lw{ZMCrQQCAGs@EfR63GDQ;Q#l_pt5v=?k^JeUGm|B5c4=JUH zY~7ZxD6Mp6C%Wy2`wxG=WB?CTO&H!EhTTw=fvqAeB;nPBdVH4O9^Ac>t#0UUuu-o6 z{j1c}7cH(2+M?&r*6v^ymAynFSda}&c zF22lo2@fzaor%x!2+xu+K1~W>q7E$^SL2*j@dH*0k2@OsJDf;_tk~y3y%f*(SO(|M1EZ&{qSZ6h5<_S4*3a?dqhn-= zcDB{HG;`7yhr~S!bXYRY^SEFg=FBe>DCuN^ESdx$Q+~jfO~dJl$eP~z1z(bK51Opd zvc~g%CkCkxPX;A?i@?zNXkgy(eClr-wMO}f|A#(s8uKKAbV*1Xva-?5Wb8Z-kLc)p z<+)T%g9+)Kqsc~4zuP52a1id38nfqK&fQK^iaFVytM2x@eiaZxX}QE2iMZ{}&w>!^^MM0IMiz08NIHr^Jnq z$teLjuK+0uXhjPe+qVV~^T?%&yvhnHz%|0%U>AUl8FdNBOC6q)`UtU&w4N7VV{Y7P zif|!o+!U9Zz4Q@Ia>t)tj9iA=A9*|_MjNqLI#9I2H9dKu_v)+Ah>GuH1q)n*^fRiN z1@sNb#A-pIa3YvKp1Dabzcr5@GiPy}gJ9jyp0Q=5nO94wWT*>@x& zvei1G+`#$fBo^S*Tpx`EHs>iOhXp}>3)PH?3x(@%TUJb>9mAXPL1H||nPVQ+d`4Z5 zPWjGfeBI&+?ah=yUNFV1xWA4PytnC_D)Px!>JBc4a><_Qnn7MOFteR|7zL~ zT$q37H@c(Tl*WaCMhUw-Ar@Zj^N$_e?R}!{N$F#v`2OHgo&0cVXGo@-c~r6a@ppnU zV(XcX_fqf&_(u|boh-0>fR3!@B9$wEh!AXHPL=e60ON3T!^0HB`~RLb4!Ub?GyG{P6a6QAEdG(2bATo2H3HmaK#NOP!hO;_$c0)C zUiQIRhr+GDE1`heqP>cUeZDx7&APSB6mp%z09i;;Rdwmgv_kTT4_ps_rx__Kuoui4 zxM<lax(ym7<;91A96|F37ir&Ke|p4z30I>wdG_jaunzJB?_M~<+dd& zE$vTf%~0z?!dWr+KC!xDp`+aI%LK?aPS*icKz!Yy!7kA{;7zD({&FQ0NJh-54^EZ^ z`@b*djscvPh9YHc%i$ynDQ5ZdFl#*L3FZ)V!nhKFRVi+ztVQ84kjNIj-|y-9|Jr-&xTu>iepFUL1Qn1HkWd~K zB~?mdSwTQ)B&0(`I;Cq_5qKm;DQTp;TS5>}kX&G4DG`xamRxG7JA~f{fA{`*?|;`n zd@cL=?97=nXU@!=c%NZ6=C^qJb<3jQ7a!hH{V3SENk$5&oBdqUf2J`wu_eb!((Fi{ z@VAo2X)bOLty0!9+3W67GKoC+AwwX0lOI~Uq3rdRddOk&!Yu4GhoC3_SEg~#_pw0^ zKWwc_Nh|hS_C>i?V(ATIY&s+!TZB!8Urh~4&VXBEEIVIyNYag;(`I@lA4_^QlYJ&( zSUb$>x$(T=okj=cfwSogyq?%5PO_ z7QMPj%yG+hTVOA-_8RjH?TSm)S!M^VLd}a}y4_}u0M;PCB<<>%vR)cXzcuwvn?=<2 z59F&S`%|ZTHg$}DI^7gk&U0?3-CkO2nU-~W_l$v}yC>HAdF#0aRT`&&J)Y0$J*lq= z8;L0($k(A1@`q>2M^6kQpXbn9hc`}Aq{uT!nKSTC)s0A0cn=)o1aytvXEqA zPdY=f&7?JTu?=Q)p@hFYp8v=;11fj!?)<{-*q*vlOjQvm_r>?xk0 zs!lV`{P-LY_3E@Yze(yF#)_)C?A7U>&1e0=IQ`C*vIv?Wv=2N8Gm@fhGQ6dPjEF~J zq7sQ!WHBDKf!JsF;duB+J@(#r3vIE+{xUzCSrU&A)=2)|;yVJb(f_(*oXinf;!eBD zvL^5Ugq$?UKnfOqzfON3=L7bZ>fRjxMo*eYw|S@1mDWB{O{4Au0|ct%thrT91AM1q z&(dazV_Tfy7f;BqQXYFIpUx(^wNlPdt?zrMLAGysFZXQ{jMYCm7P9s>W&_IU@4OYx#zZ9C#So4?E4{_-8+e9T zj{BVv>nWudNzWW@X z;-ZA9^Q#@TnA>MTJr*`yFPhYGS=z`~F#R)E* z0DG65#fJkHpMTh0RD>6k)XL;OH)9LV4$%e)<;n2wx_Mdb^lxf6SHM&2ix)I3IID3g z=Q*&N(V>V8Q85ZBv^w6Np%uGR?0WyH7r1wDwtac)z?5CxmfKlbpfmu?5RU8WQ>bl5 zZ3FXt?fc9L=oH{Th~xo$)!*t8h4Q0u6IB)(gTf&2sDF0%KZC1Mig%}r(9f4MbU%`z zJ}Vz4&y$h@jQ?+b>G`gN8M6XQ;*5UE89JC!f_ZuD&xOsL37g=0uiV{XvM)T>XLAygM3F~@?tJqMv5!s^jaBWcAkDpM*eiwpp}knksj`B zRI7+5lbg)iTJ0X_@2XHG#2KzV&bAu1t#JY|3byd>L0jR$T_Qpdw6$I8`6tdC)(pX4 zFw(M%(9N`#P873tPG-ds9IUD%=i6AHpnUyYikE_?y%S2vbNj*=N7sZI_T%v6GmID` zyu>-~f>)|SkcYI#$YtR)qk?aB+xPd$+rBqF*4Djk6q{*q{+90S7G5Kin;EsGoMiZc zFsW$AgV|0u7bRAWZQ9$(eBwS_>{u7#TAN>&I^L_py%}jN#5YIozeAnaR&))m?aF zPuT&o#X4o%l+5&Hi@|aDN{c`NG-_WJ)zrSonT=8|x9pYYH|m-uZR{-^p~hi6Z0ROt zNjZ$@a-Q80x0x}i}%!i5mH5FR_42xX%35mQx@(2oziBFi_t zDyYdu2l-mbo9jSkt;;6N{S?h_IcXjj66%&F(INGEH8%%@X@#}OBoJFl}XNRpVGmeA!5eXz9 z(Se<8M}mviF^z)I?Vs*|!^ndRE0z_~t=EI|e-Ay{7}yGvN?4#wNVj{$W#P-1GVUdj z;dIg3wDbwI*rkW?y;pi^7R_tpK=PM$Y?j7^GWsN}o^OG64jMVTTfCxUviFBft%==eEmVO1N2qb#grXb{d2K7)WqZT z3Zq8k^N{8bei|C)pUW9pcfE!8`xNpDy?f5ac#WM6NpqzY(R%PyP=uqr@SXMTh*d;p z!4Mo(uC-V}vi5qn+So2$+j#uvQxEo)E+d{fOi@_IRTj?0;eHHiOVoULYj!s|PvjqV zpm?7$)7O5);?skmpJK8duRG}YlQcVSh0p6@=oNjAJO`##-d-hwrmnOxrqGNzf#mPd zceMXLmRz||MpUsv3^1Fhy8!S#v5CXEw>w*>94Oj+u>Fhvf#ZFh-k)v7I3#stEAGSV(F_6(G!!;$gj^0qMzVVQ;JCLDISarm3$1>F(HL5CXZ zutW(mHK%gREvBfF9}e}cyBQc_jo$A3*_M0xHZctp+OIP}yww>rGUvufM3t`1y35-k zwzOsYc;~FtfEK0R^4{XD=d0>BMEs^!ZSD1}?@%^%Tuy44&VQ{oHCKVRyp0tgO0iZO zNrdZIrc^0)`ev2yX7!{mpfi%kGzNS#y!<_E5Fi(f+wpC=gAWUXU3HnN8Oe2agI1+_ z7msvjVTD%3b+#U!qYLvn9<)Q1k7#*`(LBe8sQ?ucg!Q>n5QYW3$)U5G+Gwf7jEW< zPL&}@wCJ>-t?uQ0ZR=>)I2X(nFIM5WirrbM9J8WDX;MaOT}@I+c|mXGHW<)iT^R-< zhM>8(1xLqhRR(uSAVFNZ1c4uq&KSIGAm@(V$i^(tA6}_Hn3m$lZqq?TT&kJ`avwCs z;Bj`I$S-;h__VQwQfKWgRH)bX*=Ab)yS-A6ZVY_?6>F5dI&4E#(ZQ4O%BDO=Ku5dk z7BnIEvwxV~P?pK+sk@dKCUH7aZr$!ItLZ73wMt`}H|8k^I}JDsdx)0Hy| zju(`?E7FOp@I>ErXDH#$X+jm;l;fo6rr1usE-#<_K_cHSrqNI%{{la~L~CGATDzU{ zQS>`Pp7@I8vka@T3?h-B#!O@_|AtRFYhf_)YEmT#w?t)dMUV9_!&%2t2!>w%et@pjE^cU;=?aLFg zzc;q!C*=~2zC-3nqt{n!{AP706>+pqcx*&h<+sZ>_;VIdC;0Yg=& z((fR*zV>;pXbfA98KHGN_FXru5eiD~qx~fM4H6%pQ!J2wOqfM7B3EYEUXG~^T{#`x zT~n~K5jpB-A4}QI{Z5!(vMS~6QgTSFZr2j*^TvA{vG#O+H3Iq)tnHl|7?95uE> z!QwJnkMCpY&&O@nLb^qgA-;P^x8d10{dONhujhr!tE+JQu5b$6t*Cqxf%H)R(z||{ z7B4hk({E&GlKeqq%*Jqh(rfLqk5a#h+VAxlt!IiJL%D_Z8-u+|BWk8o*!wD^gwUt4 z&RT;D@&a24(A2tA_=OsL_xS8am%N02IlRU?Ype^ay5k-dVZ?ZJYXJGMO9(Av6MvpO zZl1n$nuM6ihl%oH#HU#Y3c@%k9z9IDcg_b=>G>gq(k_k8f|!j#ST)i)Q$F#0NJ*3P z&8m(Hr{b{GwsomCV*MvKrSSA~ImzFiDPlGEw{LZR5#$wdCQHtxE`5l!nPnRF(D)r? zQ9Jy5FsfHbKcz0JojQQbGFV%kD`%^)b6xGZUA!n-C}$+9XzQALy17SH^NPxpYRepM zjmc#Hr`QI2c8WierDp9%lLvYr0V6Sxn1$80vfB3>_DqG_71Wgvk4@Ga)|WXkH$Ac1 ziL7(M=BGukOW~az1L052m{W|koDVV)Luo>zZVgx6<=nUIu9|${|Bi}StPA^+O~n^n z^%j&2k!MMi$Tlavd<#|kD`6Pv7^yooclv-$#>&@v*p4m~dx^NUGX&tp!`i4`YcKGZ z@u(@nb5E@-ZDkA@nzKF`CGL$K;t}gC@BbOq_dA&g7Q(?lGXES-sT?>$HEm^??)!Gx zRfCavII4TyS(J`7 z54-4gbr6{%uY1pG+M2yqC^u2acicbU`p!7|rU87a9#!|^X3tN~U3Qn;A6sEei>seY z$BZsDJ*jP_j2w`a#~!K3J}!gASR9@CN5)`_am`Str18G_4ZNqP9vW3P#tGZiNmSv4 ziOy4;28$hLBCbN(>yWXotv`!?i$7b{k1&#;m4i*BgiL>ZCKP<>tAkONYqWV_>E0Xn zE6-KjuZVPnWHTktNwH;GPra!@MzxC$rrR4oK?^I z@tT$7gAt1w2bcGwSj=-x9-DQc`#F>&A7nSWryw_T6{O-kCtHcy4CFa?!WUoZ`kJBoI zPyDlXKacOh_IPy?^$G&R7@5m9mOx^`=~$DwDDUf(@iFt^GbX2!w*r~ljYqOJ%7f=@^Ejw-tE+#S?Wl)#V0b-uH)!Dt1#_-s85iST zWsoUv3)xRHG1UUqoJ8j8jJxWfE8px%i1(P&84t z!r_Op94$S0ch@$zcqOV~Y_Rzj;&N(d=}s&DyK(Yd7^Gh_BGDL2_Jc&izud;Nyo3ei zHNCGOvVb}s(PNfa zALsqsR{2ruF0;IOpj8#K)ttFqiY%vK|Gm7I!xpvi35hkf>N(L`x01-Xnb{!B6~)W5 zd~VW_!XzIT+t-xM;hz;lG0y`VGkc37>+%Z>J^Y0=15A#Do~4_bV3*%8DgeDG1J@b| z<@6k&Z{{Cf^jJ5RFv4l_b0wG!aX>ZKi*yS5GrF2h(zlp8a}5a~ngdnceHZ5rz%Qzq ze&@f~xc`+Hr`zBga8q1qyU0MyuXE4E7I+R6MO9$4MA1S@Rt(v;WHfr_VGN;j;tK3k z7?cooaZgshR0Se0kh(Hr)p421E630aUkglT7HdPH!n=ckhS}QFK5D}uS@95(^^5zq z_0eo1BWzde zXzD6dU`GEgR+LOVu0Sh@7BAtX@x}(@Qpz_*(L@X;53=N1@OU|>)3gpl7ms=2x%oed zeZ3T$|HP+vUjl`?A0(9|XH6VjH>AlZV@<@9^dU6=aNS@QY?;1^YEIj`^ZNb5n+W!( z;N<6Kt{PaTHpCP#p2;DjoULqoG7`Gl{YV<^$2+@;^vz6t3Ruf~&>+sw^Km`bLTl87 zyEfrGk(AI!WO=#RvXi!pQ2CN7s~fYnMzo8&g;~O@UaPg;a=I@`s>0|jV-wXsC@Xms zN`lXP@X+)i>xvObL72;Dtnvd54T$f07-V zbxp=aIpT%Rryo3mO_hiJj|d!lqP{k77WUH{9TZqRASCn)HWa* z`j()loNdf^i%>t_YX*pqgzxNbi-oSPX5OU{+L{T7=b)^>J;=ab1yK#rj+zQ zmES&7ys9!U;X)s^{gK~|_g8YvGO!{IU^@pavVQAYXS>oif;d5%x6suTg?oYG4zVmx zea2flFzEF0x}vo?`0m+#^u(w@U|9e6J}&}>sm$($Iyd`*-C9si!_RzcuulA6^ZiJ& zHATmCWBoac^r?HEZw0f3l*(YLYZ!g^^0Wdh&whYt_61gV4n?gN)%u2qHJ4HK4C_G8 z0A&LZBkRBpF#B=_W8Q3A!fCh&065GLW3SVRZX zE1>ZG%X^lZr>f_kw#sXbiZZrK1Xm7OIo>oCInv|oBr_)C59?j!o{j|l1Sv6h5M#dS zPCGzk)^Dc^%Mxo0Rz`E!mhSKJCcYt+3FV3rSrdVlb@!GB>l^Yz!zKD%t9r|m3FO!~D-n>WK{_K|aQ?B1p!|#^T zmhN$_PT?$>lFYw(jFm49e_31ZSsafFHpaE^ZOxGF2kX{{Eb7vBFPHa#+>OPXyAmvS zKn8Jo4gXK!BBvgZt$z^bKu_UfC6H(?{a(tcMS&c~@)$AZm+r(%tnl7LzBdznh9g`@ zj>|iaJEn_+WH3(K^7^Eky9l2c8MMiMqj2zBD zwDH<3SnR|Yr>{bQv&M17Yg$<5EX5<~ACH*HZPYw{3LL7XWserChJv@)829wUTW6|o zcVob=fBUo9Q++hB&k{tLMyf|7wz|?PX3Kf#-L@{Vw&R#YMaXgu48{$7X+S1a8SoC6 zXb-u$FSJ*$tlE1GS9a2VCxfpbp9u^!7{Sjt=w?Z~Ny=>_jobnN(2avuT`vRW_VNiyhlF4&|Jf!+Um#W9z%Vl|!<;mGAFDnL6D!;ZSNX0zeEbN!=W&|WCV_PA$ zm-N&W@9*sjj^ONHLTVg==SgsOmS7ml3BVWfn1lh?+S3%zQC7e&_=&0e% zG|BJRx*rfQi*pI7dofMZQp|qTG3%rj)@-pc8#iK9A8VU&H7m^`(=gIa{xtkgxZ|(| zl*~pE`c~9q;^z2>+d~8T3Ae4U`gnKO!C8kANR3m)j~CA0#*TRYNFP?8ztp;oBctSe zm}C$7RY|=nSPM75qewB63;W96g-z6+7FX0}l#mx&ES9f`i0DPVQCY;AX+XIrU+8Gk zrfPX6%XutdiWUvKuJY`zE7r*y&~LfzU6yJPox6W}R66M*0R$xazVgJ}o1m!>6|}&z;z~ z)LPF?k82>0&ssB6ET4O&X}Yc|s{tpe@GV4Da2 zZSp-Av03x|8CnQ&95oBCsKw7nHo^GkKJ$@YcW&lZHT>H=N+Br5RJPX~t#V!bRP~Ib z4vlPlUbBr=`y}dPE;nOSZ=uzB*-?d+Z}6@0{dR83Xs$0Gw24Q$<7?M~Hl_*By(G?M zhZ$7oUpdNt!6_8CFqoY>#jN}I;R96|K^vo^+dxeS(hvk6z)an~A>d1f&|15BuRDpl zPlD8dag60mMDiNfkAvkEhZJDf)PvhY6YQCv47?{Z<7#z& z#gVuXQs)R?Rk{c>^8c;;TjW%*#R#P0+c%anoxpK)oDXn& z(p&Afg)~13N|J+Wn}iX8-$Y}N)Q1k+FQ2ZhS4u7Fa~9A5-o88IH>iY{e^eBmD*GBR zTEDRkx7+w)p1@-J23CpONRbUF1n1XI67uK-KEq@930czDT}kiL1d>>Qo_yln?cm## zj)_~iJvPBD!yc_%dI7@K!@*a&BhuOWFE5mvx2#dKSZ$O0%%dfa_q!B- z%SJyikR;gUsY&7as(>;9Xh^GsBaYrp@$fG0&&N9>xyEBy{qm|5NgGl^Th%#B?_&8b)0;4E>YI4irbs8pwcg6#UsBkeii5^`>so~@1C?5W`5CXg z5U5}KnKHhAzcHUOT0~Q=fvXKJA>j8@0+VeYzUbqM8CGs75*K`OebD&VU8f=j&0?zP4;+yL+M5(mOIis@PPP9G%!BLY*7yjDTXi}@`N zM9zyl=)QdY82JDYf+Dn~2{nk5`n{zEIg?>gRHWPrcDXV!Hu7kv9T-ydfT4)sb2BFM z-}eNwh?d+tT0$VgQefi~M_N=LUPr3lUidbBVI?Ngvd#>lLXbh2xGHIP1Daroxh7aP z1@wU;l5807FC6N%m^gZ!M|AdU35JwBPH+4zz0h@@p4mC*m=nauWC@SB00a`PBTcM9ixLt z`J7a_roS`R1(o^qp0}VG42Toz!dokY5O(qjxXqr*btE3-l^PANRPv@;dT1$&h!6t+ zeM8;h>jBD}&+blSc8$JrgsM*i(HRB5I4-96!!^#uPKO#cimUjjWt1Wx=z+Ji-T65l zzcb$#dH1SP#;MO{@zs6tl?`E?Uao40TktrU9axXZA@3H>%}g&otVLt>x@IUKyHo(H zX0uXkQ*fMNFcZlUtER)(f3^E9?EX@zZe(KX*5$A{R~=RBcx`o(C#ygA;Ct?3O>464 zQr}TrFk3C*JHn6T8uInC$(*<`;X6KGVx(RkZ45KPfC!t5e2iEGK!Le%c5C#p*XE~J zD!VILe_}C{7@~Uyw)!vmp4l+ z*LWfE#VqfeW4O?6MFLU z2xOCVd)K6gS?R26-q%&1#NXpH4m-5z(|IrGAoAxlY(b^f0(1EL^!0CE5J#t1-dNf{}FW-gNpZW zC_Pfn_A=vkf;+G5W3Yv*2?&Mva0gwWhJtJjHxQ*O?e&>&od>VjWn8K3>%l@)b~ zjWDCcLhVZnIEGUIAtoB6b3~uITa)>qp*1-$z=+3O_Gk6xjO;A!<}8b_*Ov25Zn>RH zb*yhAy&Kb{-Qv-iDv;gMix=`Vb$`_5>*9;O=wkEgZv9DHgQ-BS3YN8RF!Glvt{)%< z0>mNp$x2mD{y9~F05@gdMeZ^JU-Czfemob^vW^y(*GF-6EbJ`&s?sLA!Tr6>;)65` zA3@FI5Uu* zs5uf1HEK26>L^AN?iZ5$dD#kc50aP=@0GIxrKbZML6UU zuzqt`{-x^bgS9dhR!G@7k>0ZvQ6L{ZqYR4(I8K~O;wRvuRb$|!W>pFV(#4tk^eNup zT^G~*^g8KwzLcQqBHi*OGW6xY;m)>%d-Er$s2-DZ7e9jvNn{!2G% z8jJuZxpdT65el2m(6o2lyn!b{jZXM3g(~0ofdXGn3Ru!PFflj6xrh9v;uZ7?l{4as z;cqVG@UxRkh5Jbjnj%HI7T_x9_4eYw$xF>zt^X+8AUX5%>{A6)!KwGR=g2DyP&0u+ zTzNi`LPuQ;26@M?fpUdjiRkZF4GM@kuER{LsZ?rx=cSaF!##Is-2f4Om`Q>2#p?rn}DRuj7~S9 zImwLKe;}L(s9H1LL$|gU;khc3XRAAzP{Q&vU%RQ!eJ_6Ch55V(6L9;f(DCV7^(}H} z?ChV``H2!58^ z>JEG&f#(3$*!=J3&huI|j#zIu3e!MXyIo)cH#)_tV(Wt<0@JT56k$*l)%Mn z`OfTlmYz|~nRH#jI9##}0Rw?4D}mZ{YMWbDP9g*hpzLLnqwG1k?X9~>sH6GNxorUd zgu3oNzdD^?7sAE)T?Qb%NP>-uX26X-cp|Ij_bq{-Y}%U1GJoXKwBaoQ*SSrC40qI} z!g@Tr$Y@z%yLf?xT1!hU6`OyWA=OK6#u?tD`z$dBcT!iP6I~*0S+l1nMFN+i^y4%6 zoDH*Es2d|XgiakQ*xersvCWTI&*-CeTwdlGc3E8l(@_5jo`hUtzLQ-lFC9{z&O+Xp zpA~;MjjV9n9LeMB)Z^zsi1QNmOxJfHmO+Y29cS$#!U-Gsx;CK0+)T4%n-%@)TnrH_ z1SwYW6fC^hhUD|ke2HU>v>PQ4_|;j~BfTCB&%sbL;%kt5YGwj1LM$A4Oi}dP*U2G; zw2rmNA~!EZeF+e-q(oB~c9q$-qxbGXyzZe91f(5iEji@6afmR2i6uB=exTS1997i( zRB|>-_eGtNnc^vmtCpo%>xJVEXQ)z{fBDS&kQZhkd9+@^&YqbMzvXBo?t(`6Xv)*h zmsVYl?99%3r**6fPa%9^#Gq!tH`eX**$-3-FwS*7AI|`sQzc)Ztf&Ld4H%+C)k<#A zdkS$7S^t_1Ng!Km?qf!LsK&qH8s%|?|6FJoOiVzkoZ)97Z$g5kR43l*B~ECi^$!+H z+bE$if4uM1X~u1zrNHk*+D-X?lp;4|`Cg{N==ea^HvdYW*T(pq>^{J3bp&8k16i@K zQA6A;Bi~f#0YY}bwe1A2qTwEHr%!Kg*m{_oH$D_Nw=JTxTAr?}1Xi@pKS?!lOs1m` zDu2&nVwm++Bd79c1)q>z#F4h$)F~)kYkha|*imebu)Uw8>-bC{j$lo+W3jYag_j9& z0YJoAZ=T|<+IE@DthKY7#ylubS;D!KD7sQwt+4c!ax=D{1{LHJr?t}f&Z%$$AnPv- zQnLkn8`RmsYCeRSVB*SjV2f2jx2?ABq;4I}8%Pc1#XGb^{pXw|8ASNz>=S1hAdX-| z8%U0EiJu^XQS!a~c*>ETDANtv8#}B1vm?V*Iq7RU|3@6i6Hka0lB8myc%%q}uO()w zxvv{%EMPnb8ddfTuD_0yzgM;a%NQRHj<-zsA42#eEf$@-Ghy^wkKkV`PW3~5(8PUV zb(^F5#B#(sH^9~Cg96z=#A#KGwTQ2N39ILYwdU}*-g`n{|LuJpa#sFrfnpA026#vGm1fuIF5S>z=jXVgJ;cq)U`_U+)v~>NWXKiG5#8&%vVN>K;0jLoeI)- zBm614RW9wE%+qTrJdjzEkplq!^ypBem0>U^!}AIzk7z}Uk<$QO4<>!al4?$cFas}9 z6z_=-@H$&Y?wG?ms$Io!H@fYI5`)*R_Y!+9b$1O-_`w?do}63HIw3L5#$NoicqI5? z9z0`rxJgboVGi>K`a`iP_!I6RnGDYY^R6yR@kkOC>+%12+LJB8DwlfxGPqdNvhZH# z&D)Wb^d-hUgaL0>;_SSNuhj6>M~vB+_5Kf!cZ+Rtj~Ma2;=K%-{H!e4?-O~ctZu

B8-`eS*on!)wY;;sO6x_R!siE5Ps)7?i0 zi(ie!fH`2TPB>z5dStB*qLd9|F(eRI0jRY43A`5$Vq%bZfAdeCo=+3A{P334i5ALt zkzwc;DUxgK74ADP-TP2s`6^}rIgGeJ+g8(>k4k0e{6pTTsx2y75EMUG zpyX(Zp;FaKsy1FVEJp5wdxm{HTT*-8>r+m3N)Iw386y)Cqt%GrQVI1VX_&H@p6W5n zgveNNQXBqpj~wk5rs`my*0fnaid6@Qx`w}gbMRWm=$tkqKi-3n6>Xt6QTU%w`$J3q zU;jM>Csel2|7A`8^^k=H2>oBEgU2f8@GaP5K*H>Jv!%W*OfPweV=JwYo zHgHvqN+6!5>d$?<#+PY!FS2G3Qo3kviGEaz!9*~;BP+u z2a3qTJH_`O^?!c~2a+Xw`MC1Q-;WOHe4YM@=wwePhK+a!#NwRKaT3t~?Io~GQ2*x; z{_s=j5(_V13;vV^r0k9TBbE*4o;1&Feeb@ zb?^Uq8EHrWTiFF({-ZYkO!Fn0=<6f=F~^#L9f+>BU9{;xf5hr=#tTznVI7eikMI1k zBmw&r4sx011b_>8FOTZ$IeBf@5!1tCJ*ZIhI{h)@Mf(2`0}k5K(f-kj%14kK$@WYY zkoPYAQx420mtpnwx6?z06>bl0j@~lW1FFOI<@(=P*?l*ZL|A$AE<_#O*WFlcO9VvI zQvO=%uvWr=Bb$8%z;}kD6bsz*UCY>AS4CDh^{O*Uz@iWzc_oz^wB5!X>w&DZO7>T1ojUYD~cvB7_bWgJX6hLjV=4mUvZ@GSOMq%l`3Ahhs zw8H<1@i=U9l%h~Mo%uI%!QtP8BVlNqM%?=QtLj~$a(1zajJe;7vk%0e@kGOv3hEBV~ zx$6N(B-vg7Em{010(iwI1iBlnSYv5WAjdgo;F9hWNX77*bR_(t?nwU1 z&ditY|og&NsZg zL&|pZdgbfzx0gl;EE|TdyNiRcTm%v!V=J_DH*hc@{7R%P2Zt_ZxoF~Ys=G6( zISb674kC`+>eQP)Lk#}%K1BdILmVWLe5XMYLr%QNzV(<0JC&J%2>>%nAd5oUSSlx-% z$Xz6p&dJfXYPf~WvcvY|4x4bJ0g&eBEnNXc|0AuG`VmEr|FsCKu9qSQq>{6A2Lo-# zAf5BSSHOoPc7!5kTGgQ}8eBvVMuk`hBUpdY`DcU&U&8bu5ML2y6 zh>voun=CwQ!AD;=QIW@7e94u4?RfG)a|YFrX6)5S%r$KK@W;->o*F6%M-s#qwz3lb zqYe>9%v&*;@ozu=J~itt%@2p8FenG37z)vwJi+wml$p~vT{ zABY&@774L@hM=103C_J$glWWdp%o5EV0jNf!zoLxPE0~{2yFFL231j&Pzwsk7jjna z<`K0pFz;)kCr0Vr@CM?jQJvM)%e+RKz^}LDa4n|kh{?o+uhUtt#JoFqOebjJ>&>N0 z;bQ~H5bxAr>jLgR*$K)S%unof670`yk8wZeKkTfEQdWyzyJKuoxu zU~B$9m3Mxy)-D@(%m85Agy$#fV6aX_q*t$KJV}O z58lrUm(R}5ojWt<%!%(gGnYVFX)!buJQNrh7&P%W!tyXM@D(sHur>fhXpOX9bQkmk z!RWQrYZ#dFaMT-}N6_!YdT->VU|?J*VPL#`U|_DGRbCq~Fb<3`Fk9L%Fx>GlFgVu9 z^>QHS4IcwlaYHF77#e690P_eI3kDuqf`$IVz~aF?^ad@#yoJU8QF~eHur?2m!JSYLLO+nL->BKaz@TD3 z{K3K`Ccl98re>n3YOg9K$*pH)!JwmWrE9h88Uh>~)AC7Uq_A+z?*U zpDnnd<%en@De=!H_GY}Ks#3DVLRPj0#Ow@=42-0FD8$6XAX|MyZh2wRe;$W^<0Un= zx3}g70-cdOk7-CKt^UDGc!H31-+fKrM(V>-qMcjS0}&w5jL>Xvo*1{ zH?guLe&|<6*UG`3mz4Bjpg%vq=4o$Y_;(~ryMM@n5(s>_17u=g1pet8`Y7n3mRrWw z!~i<-L;rkCpr0-OueyK60RbNd|JP)GQTnqMN-7@;2>6F>d?>i!MI;y)0T^-N*NPC> zofPC*C8eoaoKIXbTQ6B*dA)NxV>h*56tyZo~M3;>5!QIGV$0|+44 ziUZgiw$x~Ze%5$Dp*;8y zxLcYGs@S!HkNF>Agnms&gh~f0QEq7OE683y3t{|4|A7H0Uieubk}vPGXmem}KUT zT^9MjDqu=XO!5w%j^f!LYJL}t`JoDXR_p!tf3+2kR2Ck33s3u1{(mQNja)Q)NBVzr zB1KSpxj6ZqLul%B0}^?d^QOsQ$`I0H(m3lI^Bk2xsYwxRUr#)PXJ{`>+Gm z#vnt8d7C}NbnGz?J5$ph;vmVy7_;yW37N`@U))JdI+!->TP_$BBDgr% z3($`Irn@uH_}yOtv6X~@z0~+IK&IgkMX#a=cKN?58J+BqV7aV%vq%E7@;8w6+4!CkbnVHykl@*`Tkwg6+ak3AOkt3#Qo{%Gk2+} z`LT1O6!r}JYK%O}inqRFiCz>p6OTWdTrYqbI2YE#{dJ?Nvenk&vhzDwuJig_&d)>7 zheZ09o-vFAIbW&LT}iEAZpl#_H+D)Yx$x4vZT{?;4s$$Fr&6>rgr5q2ft_Ci6zKjbLU_)RH zH)o5E7X8;EOT#dIAV#L2=g0mUXpJJ`pCcA+4cb{Z?W!mf{=|zTsrCVa5_*KrQ(fC# z$y)Y&onY8C#`_L;O3 zjj_S-^&y**;0W~4TaEDJrE}~q$5T4(=lsBap>0yxFsS};adSg;^|w(V5GFnW$QOrx zXs!f2;WLE1L|$I~FgftJ+;1KII*%xRW4^A#U0J-3kFRLPaaOM@T{Jszf4`@$xkcuJ zIf*0{L((vS23s+}MU=Bs@1;vTS6CNjo5GvRgIvE(M>qZ@zQ)*7hYtAO>i#!It)U=u zZ7QB8sAtE>LzT0I(Y55xYu=#jUJT5^qOvg9PxH0u1Dyh`7xYn!wzBoh#Wp9 z#x^65t$LpxQSvVnq!v10e)97Y3@*&bX8l72=`bvU;G&R&@;f_`91L;5Q(@%eN~~5> zKS;T(_8DfXBq!}+?ssKAgh^r2<9e{?>miS833UYP4e=0EeTkXw(W`KE!jo0u2s!rL z$_(?Og~dLL_dyh!{bR-ws&e5Mpc>Ss?kc-oZn{tchaAg{1e_!@yJ8A(StN&Gm@JsZ z$D)qWY%q`S(7XP}+qnxyQ#(j<1(u)&;MMtLaNJNyI3x{JFV?P5BO79C;qxg5o#D{6 z^=Q!-u%6WMh^>fj_NC5XQyzw1R*pe6)btcmbbHm-E{+H54N(Y4ta%u?7<@ zi@mxt(Pgk5j%lc>!+EjHT;?G<}e>I6# ztEOdfiEeeHI?7fp^HTwc(-6!R7t2l{(+80xUk;WIlQ8ZHkjIQnn&i1IrSyJ~gQAO; zdnC7=!rZjGkV>fy7P0S|bH5g7P&>-s(l8*?LLsACf3168E7nNz~ zxbVnGV15!75Ujx>A&?=*2omxOGU9paOL+s3TH<`38WmkbrQ~$KijQAub!-nSnei>d zL{f6;l=Wi|^A#OV`l1Jdavj$g_tU6?XJi;b8`e!=*nw~YurYRn@2c%oD=q zt;YCQLIIqNt>9RnFwk}BE0=S9RU4&l3bqj4g1B;5k`1@%$PMEs!d zkXwv8k|gybOICV#>$z>Az&^DLkQ_rwSfIcgD7w!6xO^%bJG@paaOs2`2liJ=Q?*2~cqEh>DM0Bu@7J>kxvm`hpofXCq{5JtE`DkU;eV&@ggo7C zhz|1ma8*V?sK*oRr~0^M9hYJ_M%#Vz~zkj<)H0M|lQ~K8Y zOKz((xVvT)5~zLtm}KjaucDYPxMaKz%2}4#8cGnZLAbwrjkDdGPYhb}Zj4PGTZ`!} zb7JYl@8j-)ip$SLZ?KNc1t)!g*OC`+PaGGTvhjiis4#!H4%9o!0`8TG#IN-DSURhP2)-aYfS=Fl)Kb^u=I|ldY^^{>i*J^!0JYzeTWK@;qSPeOW z9?>Ln-VYX1#gg(!Yd8I!))a@Tb>U54t?sZ|xI~7@N98XyMfGq@&w^GA$VTNj| zqJjY?z{b|cRqZz=Hsy0gwyMoOSo?U^z8vpUjkJ0zqSjj@(G9|bJ^REYFsa3$D)%yl zJNyv+vf^G$B;;t-5Ud#ZjoW*R71J0S_RgQSNpb0QhiHrNPD!QQ5IiQSYU&(2TQUPQ zgp`eh8KMSsT_{rRh<#6-e6Hg^%({$=I3g!~pf)Wp(T?WM)`LU_P(K(1^@FdNpPiA((yzn~-uPS}iF65BT}+e;vUAhH zkw)0T-L%BH!U%078cq6?I6F8iFknM*wYMR*fFfvJNcN&ca5%UvHK&~J6Oy@y2ApjCx^4P<@Ka76A$r0x(8gwr;emPcDPB<^V9Vr7Ljxp zxRw`SclDedH@N2}f>H_1xzPs}J_wU#&SSJ9&8)U5*>=0GI2nQ1na>LqE%n6acWo_I zZ!_0b?jI*TtF*0-l2TT+7v%C_EWfReh&Z*tKv8ZUUQx5S56#C45U9m2^CyW7sWCx^ zIa+eOP8mtQlIUjlFSLD#NxY>gA|8> zH4utHpD(S9#{a}@>3nbmwED}LG16DXPc-vO#qDDsEJ`$`HXO%%9w40lHnZQg@L*;1 zv4ik$a7BENz%AN6O^gUpqfkHg)GI)QdeCajvNut~uEC#I+oF;1E#}6~YicxN)PFxC zfKVY$Tq_v+aS&qNM*B4n7ami;vPY5T!3BH@^9p>IL_69ViK{e*VClg1D&{X{)ejc1 z^r3@i=B*NFWqVf}{+0|XG*obhW%~GUw%Tr81tZ&D;U6giW0Zamm3|G-SMOeOxLqe0 zylq2OJ+<0xPEufd4vT5b0P91a(fvHk+}%83KipbB;a}s#2(0-O1y1_fJ!427PZC|l zg2mjBgcELVWbWb;{77NSQyHp6`%jRv@f0orqar}%s&3vUs@*JaR<*z%5FpTuOddGx zeSHp?b(@sT!TB8?ZXm!}JgWeFuN!j}HmCZcNbpm3u#tT-V>jI$@ityAb}|2revEOv zYXEAk6KmjQq3>=lk|ck*bbbW$Y)Et4;qcqxwY*)$-=XyeIUN17ilZ-%qs3YE(1o*S zfdvbLopgpIdPvA_URHZFtN)A0svvmzx9(MC-H1;YPS0dVc}hVE}9=k2YSKkSKTU7GCH|ww=gdkz)34 zj)`K86(p@;^5Fg%cO40O;lwIXt>jeE1^}yO_t|NixJ6x|9mL4s*PK$s zwRwr8`<-|fF8?hh?B21wwv8=mc4n3vvn=Oa=X;TcDiW6uq`ix@)K!n-nOQ?@^<#O*I2;ajA$ z{M8@*{aN*Tvg@${PiD?9*2N-s!zZmy$W8sT5b3UfDv@qHFm~1ch3vA(ywJ5DP8A9d zJGc5nK1YBZFuWhbR~DceDx4O|F8Av9Q!~g_O5J!jN^SwuoinHAizyuvv~p(lRZkxb z+61;h<57jp02(>?os5>sL*38olQ1#1AUmD@MWIS$g>5VuIp3-0&FaM_AbapUzbWtYd~q@h<7CT{zLG1 zM91HkGT3acb8%`z{OD}(^Kva7tUC!tkn%NOJ3Q^bA0i(v59jj5;NA~qEVZDvTyQt@ z9ad7crJA2EhyvJC!Zy!rn0l*rjM0otC+rrf|H)8CQw^63vgc1{d`1gj&S(1=d_FNj zVJ-Al9cs0VDZD>xi!?S51*@QSf9q8quKuHJ6>rlEea!Pkzhrrj>`ye-nkS`;Y$PI0 zp4n*pGz_6w#N_e�Xoxg=cgWs`xnt$7U3a4)VoD)J;$+C`NelLU|kaEf9)I9s*=s z?i|qPWGojO%sNSQVSbI+qm@|i0wL020|db#nr$EW(sudzafSwLMa@48!wSvkOo+3b zEOk~2bo;eVKtY2Yh5d4S6+PS~g_qbfon9w6!STj|ovyxKzJJ-i0D*dhlYkh8A!-gn z4d+j6zJG(}tFi|CFTRVuYio zcMER-4rzSuwhQfjvGjfJ1V{2eapqI>R<|-Xjvckmdnk&Dg*d9!?povWtIR-!q6YyZ zFa8!i5ho_c_ohPs-MlYdQ>iHPfbMbukyPv20LXCfuZh zq&jr5**6%h>o@aAo+!U7`230990?AxX^YO-%ArwuhWC~VE(Q>PzzU?ES#(*53F;^i zHlfJm{-qbg`3TQvT|eM+=oqgRaM-rn*UogE%T#Nq$n?tu?8W)5^OJLrcbdr=VpY(? zrF8VE8dt^?M@~+TPK|;A%<3$Lx^WJz=}cz{=aZwA{2H~#Q@}nxSnSB?Z+j%zX*(=B zz4gu_YUoWl=1Hmsn*Zhr6iGbffqi2~pJOs&>h)!QaIo~z<$U^8ABV7V_q_3(vx;2q zWzZTU{q|~@+BAgA+><3X;|*P$=c~1h^v6Tt>t^6VMyYrskUNA9B)0uWFN>+lE)8cI z_FvjLZv$U8EVcKwEbqB$E1Q?Hj}b?RgxgPmhOjD3W=(p@*x)iafYp*QAAg%>s9(kK zR=UAov#6QQ*=19=3q73%BR)%Kg#|MoROn=(U0)9GlUR1`o&`)uqYJ61;!}< zh9p0IBNj59$FjlsMbvryd1;I9jDD5{)|j!86DDNku;FFerIU0xP4oV`jJ4L3XVtb( z90pd!oCC11<_`tX0>K;YPNG5kmZl?e{({3Kxn0CvzJ7w!ajk}Nm3IyBW`#84655?o zM#7nARAg2?O-?mdO&Ip6f9ZfahX8?VA{#a;mS?m$V~Fh# zQaukxj8Du!7tZX+Q=r<^-an=k>LQ0v5ex?slDmIQv}kNqFfJJBE>=|*BLFx$gVT(n zA_nZX^LPF|Urib9P1k*LW{A@!kL9L3KTf730v4eEM5hx7**6DE39^^c=h^;pS`Slu zxTwnJ{Q07~b!I#o>SI*E6o;y#gK^J!HwjNuxz*M&wO4czU8Z%eU;FlAs=FP1{>o5F z(uq~!oBQo&Ed09{z)JWRu&Z0!Zetr*6P@*XPpk85q|G!BgElU?SU4T^po?^1|d!@<|7FlWQ^cQ;?O zA?8Bf_ypY-M95Pi91%fOza7<205XoD-J9aFTdQ(k#>n{X5{GE}YX8DFEZ(dNOqk*^ zZ=;pA(acld;;MJq-O-Zd@m6sa6sYF=-04cQ;OdWU|GCDva89!(;(mtK?2n=(Eg$Xp#Y}qF#g7sP5mpeNg?yj(yo; zlLn!v`IkrB5Qf9iu+=Cv)x7VjoW+)*->Gg`NSz)?jUa%8q4=oy*7@`FR~LvjDAU#YMMKivml{4#dRAw#WZh6qnuVb-PH8ygwO zV(Yb@(P7oR#6GQU=9ES>Y%FkptJzZQ82I*lW7(;}88sItP$@O(6BY2PL8{#_yJw5R z&&Paj!#)#G6(>-=6vK$Q-8~f7;A8GOBc<{W+V@t*RK1#)ul;`Rq-nH^h7EKcugv1%2;j;S5_K{H}8jDi0MW(UDJPl+S@or6`B^0Lypo+jS zq_76Fkz2MT$)OwL8x>KE6&)EU@EmX;60Gv( zAuILUb}UyE%4s&3a+)V2Oxz(NLMn4~&3(VIvRx{LYoh6R@ZDY+Je$8tQc{*dT&YMS zsU#9QD02%#XH>dz$LA9R3eZw5YD@4R12X&^PKN(HqQUtrsB965FmZeD^E!9!cGp>0 zFN?v|Mq;1CaWPJP6C%NG&BYfnGrrm>s(vlTxq*YMCo`PH5VD~o1$5E_jKYS?J>{Kc z+<OBC?loCaMX#ylIV{pMQT0Am zIi=*MfP+UuEohC|YfMtT>{nPNa*ckM2dl+UZU+3}rX6w!mtG4vhR11?W7dZWy`I0H zo8P8B1x(fMNawumaGoU5oY1ogX|A|8t_@F%`PyotHi{FwUL;$_Y((>ZXjJXl@ia&5 zdRHaG3&Zh^`Wzo~(ZuB8@ZcNre!F!|Icm(*DdangQJ{>b?DxEw8|)tvF||g3NyDH? z&U{CI9PGVT#tWZY)rA-cZZ-XH+c^9J*25~z%Kl zJ5Jr6>*7Dt8c!Xqu(z@4^%cT2dnEqWsOzYzea75wQCY#VwKyewc$+*b!otpsRM{$l zWlGa!{8M`bj3sw-1mZql(E!ExDn7|D!y#U5UCLsI+tl5l4Eqx#4}=!xm1t^KZo0JJ zJQ6^_BRhZ^DAAVM9NWc&3OOjRJR7uJhxSYhOGb>9rMUP@=f~A`hLPge3rs^{OSuQ2 z8`0l00~A3eBlwZ3T*_Mw74NR*8%E|OO1XPZ7dC?}n;bYob4Cqgkq;hyK(dX!AE7=K zX$x@fH>k<;6gm?p;-2lV^;wxKIao5gg+b`_qR?Tyi9gaK+c%Hh*Nu_-l_HQqD5*co z^Ie!RjJeJck*PPc1D)D9GR{pRdtAl4#gO zTn>2y$Mp+e)+`eH++ZkFR5#*%3DO|_t0E@VA9>-_uuE+OcLBgPkpW457@^sNe!}Bh z3)20fQ05mhS)syfJ&vI(^Mj;t*}fn+OA>+JU(cR{?uX`vR*3v4T`wNt2|Il zQjfAQSmMVt&*z6}IRyRuU11xpy*>no5p1q+=?cF)p0>UFPISwLH}IJ{gI!(vOFQji zxp-TO0x@=}58CP{VieH?P?^} zHuLadkHKv9IG5_keRQElt-OoiOb;&Se$e5c6tN}PrVm>6q zZX8qD(*dE=W4GX>GQVNIt(olzejyofh0c#e-@I_;o$3+NPkitrkwWsxV5 zu6|gbcg{F^PK>UD|Dn;E&3mfBXvZX zHavl&1Y0HWOwH{zy+328O??I=IidkCJD31KKPxX)~_=DKM&;tOks(DEgbJl!rzvRv|w4uMm4po1*zq)}V@u?!z5p;NjJqxqo zUxrs|*)+Mca$FB}3A+!hN2I8XgMEA;Omtux$`K??4Bgn!jcly+p>kTr?=ro7mGU>D zut$Z)z=-egbkt0wzh}6Ho2yzcbr_hYYCW>+1G}t>TV7-o6ZwX0DhBinCHX(L5#g5v zl}>Q5=3ineJZ=@aupqo@q#Vx}<=cAUhtozDTE8S--CPpw{mNkrDu;t7@UIh`aQ1iF z`=kS|5HN#^@?Ew_3hn-6B#s1rNVHwAWW-z0XZe`JUzjCe*Le3ml_uB_in2;}0PD0* z;k$ogg6AJeXiyRj$U}Z6jQSpapD?39jWuv`v>3||=e{TSTGW@0q{HV?k`lhW`R1}j zJA1~(tX3nayt(p3rzYP8l0f=5>jF&-^FdkbP{lP*+e&4D62*uL$`iM1^)ta!quRIr zLDibz>Adw`w4!KJqcI_a#PE`s`-OK+f?cHFBzO=DuSxm$~PhU`k{U@d2((4g4AQZ`J0a7A|< z>!_3`{g$W8d}Xpbxy)q33+dv(JC)BDBAR3YX+Q9ZVL?^1Q2BoNm?-G&g~=Ov|dAEwVKiVN5<`B}e_*ai^WABm!$L#VL%eT2Q6kyXQ(Ps~M$w-&t z7A;{0u;1EuG~3vuA(bWY7H!7a^|XhEcl8t!;L^SK^pdB1{C$3*Gox>KuTM6|TID&` z<)tE&2VNN?tcmJ`zIZVVnQX53{^Pc zOx*WKumoX@eM~-`$KY>Ov^$U_dZ!xF=doC&kDfLuLtu67dMEI~AxNOduPBEM15{Cl z`j8qcPm)#*2J!Ev^_m?< z=w5ZQ&j8+0X7hyz|Dfpoj4VVtWId}lvvWl4`B!718BZ)(V;kgocB-8ml~CKFuHtzv zo#ji`irYfFCysVaMib`!;dfhn6}x%nEaL_*xP!DiXk+!Wg{mk|iA3Kl<&z<>GPpz? z6*v*y#u+iBbv=bGg!S%?GAD23B((IHmGgjMs_y|PVk((*0=(~^lO2I;NA2RGe>6c80o`U$4B=bD$`sw?EHuT?%n>|*`m6r zS=VNupm9I)^}ynK191PGtQMMu^7U75z8eYO*(X;!!d^1mnP`h19 z{lgPAUTzC^vMNr%bduE6G4ss=OF2rD1a5}TFmS`p*SAu-MGv73-a zcWAF&(N`iyY7%;}KDNQ|82VtT&OT_H-&t-G9y?hg(R~r!dnLop9Plxw zGAd5zn)HH!&s9!oJV&eOw6vq`$-3!&Z~=p?Sf>2E+8z@rtbQ>W@sQBA>h{~;@jxiT zH!(-&LKdnWrzWRON`4MAP&5RGKLYFaaO2vDtioW`@Ur!XUeMLLbc3b2&eQ3tUgLSG zqqc43U{w2s)h{)cX)ys| z`+44#992aHs*diii|Epyi_|sZb9FX#-kWatt{b3`sV(zLFW;GFyK$@hD_ccO0H;Z$ zF8PA#1J~`HpH@AE)iZRX!Q`^8E5;?L7ulXrb8WK^$V*1mX+H@TF~>q2SYK61vrD^c zk5j0_LccpSQQATqE27wWfxd8+>e*Bii&IJYC0~fzd~2XGYG^stx!d1eZ`=pRpQNLT z)?Kes*C&=Gh$GYS?!6gGsxv7==EvIG;X9tI*I)GqDAF>uh+&Lyb4Y|B6isTqbPiUXom7FsSCDiI-d}{UPRt6xm zx7o${O@@MKLLEb7Nt37AKgDEB8}I{5%ujSttxC>jrNzaXkgGK}gc=U3Vk=>9RV3Bg zxKpYTY|JMB}6`ES;9XfdJ?XJ z4VL6L6iw|k_Lo~oB{cjv<)DfQ9y}sKAEIsi@*Y`1)uM5c-ST*#X2|^VSe4QD`Fi=d zqTxUGv>V;K1fv3?nUL%E%3%W=pQF`Z?CRi6qx?X8J!Rv3WbN|?T#nFiN0Quow;6-I z#jZX3B#SGZ^Se_%49a3>_d9vBIX< zFOLTAh0$eT8#?ZSY@KNnpDx^pHxAlJ_4gqMQ_`?T)h-sUzR8%cff!~suzSq~tEVyJH0*{I&K>g4e${kYeRIAP93CUC zc7=id18Il1BEqX)1+OtLf>X6TLf0|DiIsD9RHWex;p87`zb>?A9vg-nq4X@lGp z5}_P5?GmWek1G0=zMvjsp^VFG!zJdIUM9ZuM_Sm|iaL z?)u23?>3`so1ZI;HtEy-+{640#*O*C!=6^GuK9FeD_Baaiazx?9mly@cdji$-W|)^ zf6F&HYI{~(yI?(5a_xJ4m!D=GmFG6p)NqoLDBv3Wlj!-m|~ljX$dy3C6Uh>})-*sFB897PRxwPFzY`5Dj8cYzA5R{5a8`88JJY^niG>){WJ~IbXcoU-#`G!^gX&XtC>GmDBnzyP}qu<8e1`G8t+o=5A!iadQ!RYlMA1K;$-j zyS8L?c62nwbxNo5&H-(*R>okb^y;p%-=YuD<$0jfwcmKHJRaA!?Djpb>*;9Q(kQJA zjk$`tNoO=qOo>{bO*<0d>s;m6Uc}ZyLS@s&U65zUJnX1VAvHi$2{0iBUjIhx<1KxO z;}8Y0#?dzS?lRnr@r({%RY=HbgwRdAh5vh;a-RD^v!JVvrKXitfjr#PrAI0oCTqTM zVC&rXL9SOk#dihae}kEoPywQSZ;e>TjGvgg_i_%>SN9Pk-Acfh4#BY?O4 zx9fS$>Zi^v-0MG#g_U`9LeiAG5fLlg#U*^Aof*W!;`e;}g1AJZMvZicmY$8^xIBsK z^_3OEFzms|*0~>pOuZ=USrZh|AhjKMKpTtAey~8>{!dh=9T>WhLZ&?pyFoG;Fu8q&&3+pb1Gvp`W92GOsTSTZCnoCXA6N{PPVadHng&GIw5T55 zcNh6AOQep415QPz(%*wUIV!RPawsUBx>FTpdSp(BP1ZpYP$m5Ym4{l`!2_(8k; zvqI=YrCV-@H?O;snCK;vKN3_@2o4L=wNjvy##WGhr`v%nG=%|Cu&e9Kg?ADdQO|2l zZ%4ZJ!$hCgL@4digUXdD5omRpc4~VXi=**{R1-N3WgO-JTm>zEgCb(MV)9`o!C0gV?*;d{}U808Pr?yqJp_P{d`Gb^owYw)$%LV3}T*M+wl>x2;FF#OXo*gd0`E8zJK zDtg?&d+RO6_W0rLe3q^lUaDkpPUMeQNpeX~NCg2c^v@h6%LNZ7gLD~ANS~Gm!Eigz zrW;ooprWR(3#ILL!0Hr3Za>Fiu4Yi9wKcU3gZsL+FwahfeU5bDU=1hJTUanrRztSc zTXZG-%S!B1b9SRCENsGXsv3uxK9HsE9;|*K7?b}OAWL_6ICJ&>F*^Cuz)oyk6pBLF z_vQ=RudS(@l1tZ7HTzBX;o=Quvy)xL_0x%Lsbf5saQvsewH{@~JlC0}ai+ zN9)wyb&$+vh%ug!`@fI8J<;oElkfmIA8#bW2t~_#o~c~hk|yT{xK%L!P+^&&d1VDn zYkgLw-ZPPzt{Df4MD~7i|GPlLp$Ob>yGeDI(Py#Uj1B6n)kTp@Chl&J#>Oe>lPt%T zl!2~|u^&9Ijz#un_-99M*CMUzRkU2oC-Z%tFXvQZ=&3RW*+lxHuL6h&mw3MRx0yH~ zi)8GY*#`zx#lh%+GrUq)o3a7~8{Hn$Jq=^-=MpDF^S96@DLy<{vcdKf{(z3SsWB~x*C8-6TP^Pq#E%)SSoA`hJy$eqJ{nLb?RUf7l8Q+)2$rXP=h z!=EE4-FOcc@ypzpzkna%n^u#DRD<}QS@GpvWY-dsV7TOsD*a4hag8H=4{fgd^=`*r zCx6NqOY33L!U^7De4POj`b*_*qrFDj*<@!0l$j!8aI1IZ5)dH>Yu%cR;ZAU5{*8l6 zt1l9cC~hvgLA;aObTWir;W$y=FQO&p6(~#)_B)R>a@3#HCHP$J3X>ffqE!;7>nP(z#=tZ--0T{&sv(( zHi_H=EV-@^keuxFJVB)zkpZ-18X+D1chSN5r*Rhh}FNm%W*ormZ- zn_$-P%zI8pj8ryEvEE8It9x!`yTz#FJo5%&4WH{0zLPQ-WcCg_?#A0`YXRkU!v_Y? zuKMD8Y$S+$fK?-`5zbDAnA=d75EL5KovGtb*r^Y!|DgC8ZlinhG2*4*GKH5hCms5; zkfIZ#FR%G#l{hEZc5TV+GKU^44HpInuvL&8INZ&TG)P`6d*+iF-PCYO zB+nTj`(&TXhdR?Q4r!t#ahK68KI*VF@w_G?L2L14l1gl2l1XF#54~8?_=;5}%bq4A zs&+bs>6H;kD&#B48ZsXQm|Q>Bj_?Cg&u zNp$>+7#+~&nfrYV=}9r=f(}^G&z3l}7)bRpPtky?n8iNHB|X~j_*fyi=Rb5k6M5>o zSqAyAD=F8;p_@qpW)3tOCXW_avt{z8RgCF-Faa$xR~sb7S-) zpsfC1gH8;$iMatGk@$1~%Ln=%GOUaE-ZZai-`%Y0_=uu(&IG#J1;&Ys9X`vQk0!VK zI?N#M$KNW#&+RF#+gNv1!~LI*0ojY=aD6OUJ}XC3uhvf>#k;imn|SO{z_%t^FBe%C zR9m#-kQeV*X_#@vfPZavX(2$uR%@Be(sK5jl7JDo^z?+AjcyfQWm!VOXa~dYqIJ%w zC9fd5&R&!&5a=$bT4XE(R4fN#!KI18Bzx+ckBV|72i``BLYQ>Qp7Zs8kz6?Xz)J#y zJYvA_=f$Mc$qp}=&=s@mh|m9C=M7V;;rDEV6$}0chC`{lVf=TH$x^MB8zljXhvUaQ-jB}be?a!S6GLAz z@t%GaEKTh$yoZ8czw;ik*E~hh1p6Oznj#TB5+Rq!6zj3YrJ-?kad)@EN_i#C)7(lq zW(Jv!fxe>O+dEd=?jy65QQLykj^77IsDi=+UQ|Le&u51f#7}EtOEmi+Z)~OsoqZbB zE+WZ2=H3TrrgAY=U=tRpl~K?8j=HmZOmW69=Q@MKK;C(0AUe zjjVx&c-w!Det0tw4?iPRo5qA?vabCwJA27Hw$XUgTRGxIC+4}EteBV(Hb-^8RJo@B zE_&(s<1;l~x?-n9Y@_;??7OorUVn>M!ok3|p4zAuB5r<5ukxuWKh+$Wm8CE~r}kYLqb+l3F6YQoRY zIBMKiV-MC{b}K%=$!me0Vh;J@vR%B6Q5B%tzC&L-OFC5a^}&_wz~!0z{$Bv;lyLSb z*Mm%PllIaqSS-t4yfSQL_m+b!!GxPd&Z!Ne-ACcpD;Y+@=sf|dA^uyy$K3VCdqW!n z1fEwXo7-Ub-A5d^S7URHJh^qi{d+x*4Hhlhj|w#}7sYimg}?F`a(vkT9#HrAyP`CV zOBQVO-^D2Vc%zkX*>winuik0_4a<#E@AHzZ?=bgfsNBEJ*_=BxRk++=C7n)B zmJp1D&MhkJk(YN|8xtxBo7WvI9mG2u_OLt#J{cp|1WH_ExU#=x(goe585?pf&%Z4p zu+1WP5iro(*<|uQ8V5P6>B0~Xzd73I0yC^-yRFXRn=y7zvMV%qNh)K_vW+#ARKZc% zVsqm%JxB$MV7B2aGE8c|U%y@Aru-%cDu>-KVLig{?!x&S*1-^RI|BvpX(2Bo6~rGa zmm4n@(4Oe@xsM`GC<(?Q&e)ynh!Cn0`aA$K8VlSsqO*J>BI~@A@^E~+wR?wqx0SP2 zHf6B$c&kf;iaOKklY0S3;lU?^p4Pgzue4sVF!>&!ZcomJxHmr#ff z4Edc}e^w_u`mI`-wWSOyP?`oO;{{GyQ+JB%6`4pz#dB~!S3m(HIs2Lc;;PM7=(0O_uYzl=T3W&B|m9i@^A2NLQhq z>sB_G^>EOUP0BR1*w9YDnKUMV6{U+cs&Qy^pupTfggfVj6edBYkoR_-=IIpUsB?oE zc$&+KpJ6h>f%q36cQ+W-Z2Qomy-9mNgwEGF&5S}Xt!Fu8CVjw+Jk?cVIGreyKaa6Ox zK4S-h6c(@Xn_mWKwd6g$qJ?Whj>f92G#uDUQFVRTV@!whRJ`bsaJWh#rqE;J$4mDxJ1nreq_Jj9aP&HJCu|*p zGU{Zj#w)V~Hyaj)wbyqE)hNlsg=UxfHsTk>%~B)@p{j{h5)4&l9#Z?lpk#0UeTG z+mMo2O(OLa^(uAR{I~+uvK~$&K;PS_xgIvcin5roQmH-aEhHzDn7$m%q(X&%MM*H( ze1r8vEnvO;;4;mqjaBQQHc?tdss-Dh$NI}x%fZH19xLBAM=~CgtkAPugZwMuVN}JY zuDjLjpJ+6nW95+O1IDVpI@7I0)i0v4;QwfQEd=}COPgn&D#i_9C6FB9$0!edeUeb2 zk7XbSnOqqAzCW7w21({veYZCl2i2|LxX@7*LiEkWz1eNK^VZ0aq=|CEqJ=)6Ws(TG z+i$hNYwje}$Cw>|R9(y44D-E~B=)B<;KFS zjMzZQ|KaN`!=n7YxL-w7LL{X_x*bqT=`QK+?(P{;2!I0$QsmjP)O^HCinnUB-TiNB zHH%gly?*-(Ivm=29j#jsYk1}rFEU>C^7ClX%;n>F7`kcK>_tkj&Q2_H#I$X`|GC_* z0?z8;w@Xy+n65N|={G`4K4Nut}D3rx>-Zh5p)A(UMES zEBXJ?;QD$?fUH;Wf=%+c)F%C~HOQ@+r!Nk^HHpp`3%=l8r>x>&1mw=|1!OV_)jk_5 zRmHWd3wP>R?Li;Qn4f$_G>FKXQ%cdz{7^0sNwEL*T1NH4wJliTgLJ|4am8%8 z>#lF2`Pi-?V=9ShA@E04Ol{d$Hc0KGs|pCdDe{evx~(|Q_}EQ^rV3uGEPbV1Gm_uX zV9nt!UOiK0%C#$WQNodhww)c=hEY$>$DR!9wQXM9wgs{Vs?y!e7J8}T;_lo9hlhH) zIlSh3S@Mf-YCim@0KjS@q3W~Vq?_R~uVbp5^#ZJk<^czsqJGMJtD6EV{5RNY%{u!N zFAMbBkPrhZH!*8vTxEVCqO^rb9g#gl!?Xif;l2&}R{apQ795eyjc_fzhVeawI*tqE z5bc)`hfyx5_JtOVn}Q%9vhK(GN`d2qE$nweg=KW4bJq)tb=3a;a;7YHLia#N;A43L zd-C*dbMV{oWB-aNZi1$!Go$Ei$!}#d6oPx5FLmU<2-p{vo1YX?g=!%mtA)>dT+R!p zsGK0?Zst?VB>(YZRio9drX(CdYPFO(U<%IQxp6#$xn&(Z6_{plh>S0;klbV3U`na~ z)vIcg%c4nD0cdRm${m)gJ(afS_ruhB(UG367y<{ibqXtK?=rKjQhnA=Q9{z#g`j1?{#QSRV(d>N~@JLiQ z9>*e~#gryV*G8c*!Vld9#M!|Mt@kv?2Akj4%zUORSdVnBZ$Da*&CBvgR$hNiy`z_2 z2h-l76U06Tu#h$e%yHpf2F-J(%VzHZllLD?{c^F=2`|_;*7<7hhS!%DdJOO=^-rhF z=_(N%iM=hXuZ-l_DGEsyL{hxA5oY)5C?O(ToVr}Em%oDMd3l{?Z8#00j=&!09YzNj zyeRs(+p*^&aiVOd>Ex)~?em+0I_TxN0!zq0!on8D;d>=YN)+3VHYh)dA3sYme!#;E z#amq_CLKmv-N#;_Bfc9f?9)i$zE31^8|#dFtq0z)TO~nU_SOByP1?D}uWd}FIJE%N z3tG8Utx4f;k}-YV!=j|Ae2&xSKedXl*{D?1hgFy(7zsR2pYuU!XEl6hEYt)P=lk-h z8GIbn-l*&xD@q?b2XUXa_ll2;-K`*k=HjKud#)9=Foz(x#Rl>3-su+uAW7j#}l z1&K6OcPKHSi$H7H0n44~Kwkq|LGZ!WrcCx@xrcaroOjg~T=xK{WQz$n**)E4LIAy= zd&E+PRv;l9?@>~1!(e^eguuVd%JJO;N{q5yV5v4*!WiR9czolz_BPoGT^ zWcr%8$E>(@e;AC!z{Fk;P&qa=1uxi2DAbTu0);-PF4ShABTj#;hoBU$&ffU`N z7=gYdy8ul3X;rdI$Xqi^_1D!XJl>`+(S%uxx-bLCe;rX+@~&ZjVWS`W@YBL>sfWeA zSSxj)Y6pGtIpi4d8aCh`rUQ8|c73ccp+wDCOTw*}BA0+)sCDyBA6z^R>^!5(jDZr% zL`X&lkfM9Q`M9>%Dc>`6PT-NQiNlmc$42^uq6m5qz}v!O>L_b{x$;<@>#~jfU_%2E zet1xQpB29MB*1swhn&16+XG&d^OM-gGIf%*Y{UFfl0-o53R70&p$OQhxhl$Q$Ov&r zfuj%}t$2-#QF>8f2QyNP{^^@tE9C>*vG8MCk11~hAy2rzIc~Lot3L?O zwSorWn3QnvF{L;8CAsr8QNa^hSnN`5RCfFGEK+u8n$Thbs<8b_=KTkKJan~8O!m~q zlJ{`)65mWHHhi=D=Wuv$880KPc$Iusije)^S;fuPo%~9zgxnD|!S3*wU2{*st!E43PA4d~nB7Pdf}g$cQ!RPfB>q z{`kqXk{QmCyM-JCD3v|RVqG5!n#rlePSvJlaELNp3Pz{|&H>me%&cnnl&k1nu4( zkTW^vp{`#NtGz+zw16`Biz@#ysSUojod_92T5bNCa>9Li`pBMZo1o?ZaK7XPTIe;7 z_U#5g?w2%dA- z43ZxPMv2#BesFFa0mAK`Wb#*Vx*jgP1$Olj6E2G$h1sm_ZtmYwU$|^7kh`5P3x~v$ z4pq35i>Ci-4nd&WNa#e%6xS9e0L85Wmk?6B(8~4V#i}a9c-==5Y?u36MHEWN3mc^Q z!7CtNVVa%Qgo#y9=w@V<{9RN65Tf~>Qo z)4x@p4o(})Cf7tqFpgxf)$WxJac4!Wj224U-@DxC14c{{mDP-#G^}pyVFkU$QrGh2 zGnpRVV&4Gy2nM%N0a^EvBU+e})9SVf$zIIHVsMHx z&J{@SPvf+^VE(t{CFf!Ng2$b~qtZStqh#D!QuNtua&(E2@7H4iu0kdp>uT&!{xQ8s zBaDi7>W{F-dEHB78=5zFEPu}$Dg&ZMxnq$JHJmv!#ygT+9k(DET$?0SfC(^aK%IL% zOsf_T#==tYsRtp{chNi=7cVMr`)rQXzPC2>!`b8qckM#rxbF=TYPvVBC{b1tf7f$B z0~e-Y2P?M^GCzNG&&odeZ(*N#!6lfoBBbGUOo1J4xeA?B_Wh>xjm@RS;k1_31nZM2 z=osv(V&U8AU_@&oCa=vl^mVixd;l|D4cjN$)HHFnUh=bsN2A;slCM^HtTJxaOP1m* zp*6m%D7r2iL?n<#R;0shPTR}bc+LgT>&)K4Jjmr4_#{cRjmkGaddB9j*XcYj(jI&r zRX^{KqOdWXX?b1QaT;{#K+QWW&&gG)rzTN+DuQ_dG}!m@uG&bZRLuQ#TdDYW^`rAxILOlMv2+7M?d1 zt;qgh`~}aBdOUPAaXoQoP04=(ETre2?Ky*lvr#KYAytsg=J_a#ZL)1N-MycGA8h=U zCsI3#m%}mhfz@GXC;9A~_F3E^M2$h#D>y;J#PYDR^{&BS*@}u<*Uz62=)-bUr*(JV z2|!)4N?Ba(sDmeg8t?bwzq)#$^1e0!fgQY8Y9dw|yCw zY5C;x5WV%rZL0X>tl%f*Afm>8Z$SR=Q8aTDUfYk8u{dVPLPeRiSqb@-6+X$y{BQrO zwvY&&oLg^lvlYF8iJ(dDCeh?L#N*SkB0K)hVaByfTi7))iL8xwIY<@RqXDDfNALkklB$kLyE_6J{7w zQhl>W5E3Br%Sbo33Z>L0|Fd3UQ*4tJzv}yn0R=xaw*}s`)3hNe$kAFdzOyD{A(Iwv z7>Xb=rEg5rb_NL?X)A*E=;8r)c(%6j`)`~|V81z4|91-K?;k*zmY0N=_&Py;pW}_DP}lGlOU*duE81-6|+r z4d)A}RRVDuSQs~*1EKTS@N_sXHEr?EXal3J0yEXKlwXb(cFYI(6?{)YxZ-uel5#Bd&F ze`2lSAZ-O-b8MTWmbj%+pX519PvHm0#S?mWy*^g|u!lRs+2cmqNV~^@bve$<((x5n zn~>Nb`2p5x5nWKA*Wh2O%cK(i!N2^wYb@}XtTksR{m@ML#$2I9{I}uCNI%IV@B9BX zDZmUb&C?&W*(7U2FULJRkC>g_h5b(Ide|yGDD+e(g14d{m&CQE;)^@!uPWkOtI0%bg0+wn0Q_Dy z;#~{OZEb*0a^vKaGONslwCSrXW=9{8>k_IU!#$B~-$#4kgDGvh+8r(fFj)VKihw;* z1OXfpqjg&TtG66e5Y#VOhHrh)!;kek5JXL?z36(xIi z1@x6k{nAp=o|y`M^J-sS%&sypH?Ad6)t+nh&AE(Bc#%DfNc3dx@pea#5Uu%r@nx@- z+4^@ILC`DDHjUY>q%suj!h|&3k*_nRzXK%N zrjMZ?E9^csRl-~e8>*|#aYm_pH2SprTO*m3I;JtQPwiECiZYmfq9GWf9jza>ZFt^H z6>>WonqCc~UP$KhH&Ecf96toEFyXDsldN+15is~!kAOkFJ-53(Ov*8WxsAM3uUk(? z3O+{YYkRH4AYLmWgn@Lv%3o@to!c+oYJ;%bfcgz%G|un2UQAycx|!$n;$*Q$`hQ$- zj+ovtHzh^1D-HR-l_o3TvD7uOWt?97AAF+QF}X=L_~7FHMG7}zOLsUEs7m95MiSzD z$Gx~%92Qw&z#+Aiqa^T(&F$>C5oy}aO6>ws zti!k^R8io+&B~PZIml4qy)+GTW~p-nsVUNd*O=C3mBH7Z8BP?*M-O_l)b5@h zBHuV199g8ws&HVKT{N%!whf$6K#f%#e2*~mb)tQ)i^_I)T>!!13L|qoTJn6O^mLT^ zJ$fRK!aB`=p2uVXKH}++DvE`#Z4NHeFqbJTYp#Fb^c-2<9&AWpM>)h^s4QzCHwx#f zT|jOKO{K?zsB?0sMMHevIBY{du1iB_(|`7vD4a7zl?STKgyN7BBKulbpTlbJqx4x* zVumiX=(965+t@ZT7Wq5of0L6ziud7`pSGb9ms-d+j=LvwbxJVDpZzaZO+%(B*n(mzb(0^d-qUQzd4SpJxZ~%WpL6MUr;pd6G z97b;c-AJlUTA|G02ml@oMh^$PaUW_pa4#b=F5s^oRQ(mH{w}Jx@ z1Shxs2ZM=y%-;68CVq6Y5tj@-_GR_ADCB8Y>cnDw$a?qRa`+Qxaxh_WW;=jh z=UHha-a-aRGO)=lm1N88cN2RZ%&l9;d4GcPC`-&P^10^7;FVw7SP!2ZOvMwWR;oZj z1%96%PVQkOXrsop7tikzPnIjUioX^bxoAzY$g4h?dWX&g;C_&=rXJ`()wF_{NNC(;W8%hpr*FHdE2Qz1?yAuJi}`9k-^vN zUgqQdWlu@(!e_9)@A7L$(slDj~fCnQEyLhYhk<@oEkMXv+S7poJ4mT zjOyF#=hi$IUOb!1pv9eoCyzEIOmP<-8+y6m#*S18UOm7&xMaT@WjrD$Yq^+_wEl5v zBZt=8ByB$SR){kr(c29TKX}nSn??vLJ$yKs)ulqiVeoujou>)$-aKdk`I*qiF3?f& zx_jfRX1*9LPSk$qqjVy(V4|ipt*fQIHe_Ph58-?o9X_alVq9%~IM=Zm zPU9cxtu%pz#lpIuL9g5^?*9M{VJ*-hXW$k&8gHWjV(+{KmC1fp)>*n znyR|c;=2=i2zCxuI&Q$Id54;+AvnSPEvIdf0jwFBF09CeVIM`hbEWAzK_dzTEYP)u zSFY*3X%92Gi8R~1zVUf+tw`WJ@HvZBvND{a4S%8bwqlvNra4|Scz3i#1*#5r7EZLL zqnN~HZM*C_l2l%7^`olP__-jqvE?k%C+Wnhlm9X^@4KhXJJiZ~*Q&+I=kb?^(WAng zvc{fiA}e6>(nF88rhRkH%j2R>ENztCx%{%Z`EI8&5R?!>1;#tRJt>3;3;c4g&<$|y zX!0dm-Od^?M(Bc>g)r{}sy9M~%e?fCn4QfB`-fM>QpQRMHkU3?7hCY;2=Hfto)#Xo z=o^u>=Mktyv*-Ho8ym^%o_Oc`+aSSd=5?!aa*OwC39K-@`5)CEY?AxHa36yBjM}MO z>$fi<=z5)^BzDr%W%@z>Q<@ED^FiBanLf2-3uxFNoN5H}9Sq;Si){t8JSK4@PUA-# zkNDESfy@?X>A@fOwlJ|ogeBMfn+D(VMQA3`11f0|Q;l~W+k$CgkAihKr$@p&Qi~)! z1SEcX3li@t9|qQlcq<*woxXnuUZtmeN})eqtfHTMSORn;<~Deq+xS`gEB9qv`v`5K zO+)gmN$Ki*%Hy?DZC~GEr6opO03T1toCF7h_Q18v&F|opI(p_NUy?`b8(O(5X6+G} zJbauDprI-Tk3`bu)9*{9po4sA)JHUw-^KA-?iD#lYjZR@oKz$FFR6#K#blsbIB>~fl$B}rjLmF&>>H1%Vf3l#B6qYtE ziQJzlynL>~p?59O!pOTJ;`JErgfi!o7rIf9%$}W@h^ST8_cKr4((u2F>RAkbwDc7{JU8_Wq6dZZMuM?5b~x}#}vr@bMNet zyDTe!@;*~$ZgnW(Y|+yYT6nLVn#CY-@JVL>`E5uLI1cD;bD1*Z#pSNC^7%|Wna#oS zjDP%BE*sLN6jvrIC+K;rf4(#BXAZ)pK>HPDlw#~sdh(T-Z>m7u)fLu(xGMJ^67LYv zT6G!Qc$wl|WRE69vz(-l=v$6(=_VIftMAT-xA%+xag6@s*R$C1lD(G6n|?cs5_F?7 zQ^SbEV;kJ~=sjJdX8-wd$Sg8#zjl3ya!m2dw`YRGzw(;;<7{C3CE-juf6#k#n}+qd zEkUn2tyq5&a`WxO*|&l$C!~AvW-e&J=-0=+uA>O;$M!2u!b^!jUrELbaR3z}s0*kz zVo4gTO1v++-B|~f5m~?}-VH}^{P?`12ysTbBzNcE!Smle3{spGy33)$KVI(hETwf= zbT-ul2F4zOPg4jq$pzR32#zm6yk!`v!k*E(Li?HkbMzm_{9zp7EI|J2_xR-KSI75S zk3=m~*m9EB{v;oHPKHKR&Mg+zTP$@{MiXs|*PKv3Qx>BhEi8VIIG@ZR9As(BNHkid zdJ|ely5}-w9WJ)Cx*Yp|_sOU8S0fCQSCjq{_c=QYI6?ekrVv22FFkj8R*6b=VB<)= zpo~1<`O`8*ovTkgBbEh*v6bWiUXp{+=(yvRGL5QE2X`d`>^^%>=1oqN#8n9){Nzn< z_Q5^PpD0X>w#oOX8btQ|m9Tl|G@8 z(kh=Gr_KZ272!o?xH}B8;5!tnv8UTh_o4RU2M_D)QO#RITidg8*J0`jc$vGKtZh6} zkn38$l*YJ7#&6vf^|$n_maNoV4bFuZ{M|TcnmC+NpjdCm|fR(D~$2gg{D+ zfhgu<$e3~AC9NeZ-x^2Ho^jp$+61S=BEM?ztRc1`XJ2$GVc0;eFd`Xv?Sizak#fAA z-wa?_AxyVac(^0+nn(9Mv)D6Sw1IX|nwb#@q@dp>vG=&y_k(OxFSWHlwFQ2D4RDF7 zL(v(qeY^gf9`!Y{_=acX|NFhE!=mmOnVJz#?LdNNGzO{(?d{at)Zh+2O`47|uR8;` z#(kc%?W==Wt4TU@_L($MTqOm*)141zE9cGtq{=0yVnyqm4Q4wthY={*n)anWCd7ob z26&tyFcbRS0j)gt^$Q(RhQTQa;qQP^c@H2+PgmrsB2athN7}j(oMg?a2!S2M?~?J} zvOU7E=6{aaPt|__4E5mJJ7Jf?y)>H-1AAnDSE&d$@SIlY+TXtA_l5h2~oISs(S#u)6D;9FR{?u;V9*G4bdTu--|C zK_Hq7M5>z+>sk6$Dq6AT_62EUCdpRs8|J>(cEM`*Jvhq}ct624dz%K6e(2Y&h^351 zdBS_5IFlr>Q}H+gEcU7^_ldzoTkzh-gQZ$4_Vsh~xiYDmv78K58|K^i(MerjL(Sqeq)W*lF@pS7$2+<}~+oJs_dY~vV z&*_0PA9j_ zzD2UW$s#0msTl?l+G>8Eg&ST3lKj~x;-)(N8F484&4U~Qvw#OkN6W{}5d~lkqLu5l zDD=*f7@6?=-%MN{HYp?{&DtJ-s&@G!L921vF8iT#%df*6{5`i}8RsU+g~CkH4%?ir z#hKm*h9Y)tQ?TuXt5*Jd0Yp20ld`0|K8%p6oMloW;GEC9j0-W|WSKb)jr;|lFaED2 zF)Oe=35ijCO}GKb9R8@2%)#JlnK`Ya8TCgk&Y`G^BSan7;fsxGqx@|z0sa5Vseeug z2b8eeax!YV`;At-C4Z$AuAk7KYfXFTj)jlgOU#aS?cR4K0#uSFC%`43x1WW-Wiz9o zI@JG~GIU&#B~+fRizcsPRne&}6Tj`)M+a~WqJ^8qgRHEU=y!Z{Ynp48A4P!N7pKgn zR$I@FZ%F#$luF%IQJ*e#`Qv9HhY91&*Y=<(iOLA)6PR(;HA^U(`;zUaE86C1QFvTR z0yV<0QxSd1xJ;FBr}>W2Q3FN;eh?+5(J?Z)<5AtOr9cwMs&nSpk>;8xpax9`eviMu z{Re?j`PHCGpD}un#7rihpGtxZ%>a50bQ0=geK#{F3FG&bGU{WaG8wVj5%9|VtBO8V zeVpJxJLE~e&8;xjQA1|?k@^R55{UFCOuUH;Ct;WtuXC6Bz5=t!9DyEFlVOkmuZD8~ zmynkDW4~DY*SH^cMdMp-dfM601lMj6A}T2w#h+XydZx#03%k>Fl@W%k@QJc>WIGmk zVsx&wD~AA(mBz&PhOoc?MI|EmbO__J=FxCaS;LA^bS>>Gyt+7_izc$^X*On%ziV5N zhU{0-XRp6oz);K+_$%L!WQUM<;~jb0GRD9q;^$Q!F*(cLc5n5Ev?{NVV{B(qwvwU+g7;mk8R|VLL_DyoI8r>~SS>+L}wE5YrqC?ea zTM8Db*?*2-GQc!xazHEW_8`zz%TX=)+&$_;eMW?^>HJ3*d6pQY&6tlj#?B3s)r#}e zHUe+tj247_qNlk4bc1Md8-d`h6HH5&DxrX`eOSS?{5bTDf zv?qc0tFyodG*wmif<*3n)3(ZuuR&o6Ou=mO4-Uf*+{9{$?X37;`*mh>)(`=rEyk5~ zJ=OPW>NEXtedYkTA}_1r7vdu(>cmow@np1gyGh>QSS0(#c{ye1T($u@Mifs!@1)8UG z?uXnXjWyy1b>Bb8?7M!mx={ZmyOd7SUK#)TP)l(N`_VgCs@-6@Q?JMRvs{9+Tw%TP zm9mrnzDmGRRlmHXr@~#UiVrgC$)#G8hlbf}o{j2kMghD2IQM$*tRgTIOKGt$g-aW% z_XO;QlfP9~%%#iyCwStaju&!mugL6L<3~CV1q7ySQoz1l(%L-dY%Q%%w3&D7+_eJI zZjm8n(N)T=jq{T$)-6!A4|v&@s2a^Gg3m^7sI2_!SvsL{$X1dPvM_@_BR$-{^P^EP zyW3ga?*qd}#HOzEiY)6YvE}9I@$FTji=4h(ipN9hL6cEGM)>s5^WSIudNesillCv4-;l=}B`9$W zE8y-}8rgJ50aU@+#sT|>VFx7l9MmW~hTA)8WZLNEK6>Qlh_@nZqj9(+!6w|+oR;h~ z&658@;;{Xb9Nc1jpB{?}4Brex%xY+3YcY|)1DjXYMN^~%Y66S91uvh}YISWe_!Sm( zS8ExiZyYao!mT0E^ByqwGLCq2!Q*m23IubUIPfs6r$lvsZWV0do>HLVsW5~~7malz zyNeX($vke#b$#wUv>h&#YA$TH6*(x82c$iu=N$r?jA2)jg1|Mltrnd29aY~ap$BfG z7qbOUr+a=zV-5d#zkQy#wuIK$P@DVyY-_cbn?j`J$_-(yUrCJT22ApeU&KUMku3vN zk6~U!R>$gF;-qy?+rE+mK&_6N1J7Yo?YI`RM^)pR3why}OJ?3=@z1NIyk{Ti!F4Q* zm_o5D&99AT&BtP);q~h1lKgFe3ah(T5w$Y=^eHgcZ%z$~J+{I-Xk4r(W%ZGjGAO$A zzsGvgYYQP9e(>mK)Vi@=Vcy^S0LN9!G_0^y)LQ@DAnl!WtnqMcgw?Bwb&#K4=Nwo1 zQQ>)o*Zu57v&@3YyoF~$Qf`VzSq=~CA@Gnsof1ag{8<3Fg@_*H#j~BU7=Jdb(HB;a zcu$xWfPCP}lQ@7Aa?HckEgX&D`P)lg<*&qVHGNeWN^KnIlv#Y? zkIjHE%E|fv#bj}E&g ztr0APcuFGN;5=?i)}$X6q^x~N%4~ZUcx~I?K4f!l;-EH4u`kr@G@#Er?n|xC-+M0Q zeJ)pHFBqZ_MFgo?x9oBy9uI_?RA_X0=-EoTv1L!kN0Qp;D+|d5tgV4O*( zr)sh9zPZYWtNnnj90`fX;r7!gXPk?st5Jgke_oO;?%K@Om`PG=#{AGj$lSco)*45z5Xc)b;JSvv8%`7Eza0L1?i}r*XGvliue2x{>!cwy&nP@`F(6bnOrsRmJmlePiK$N29PiV#jODGbZ$(3;&j%oj8(@md94q?}Ve` z63UeON!24rLHf^&D81-zKgGsNfmZw_&-FTLzr+K1*oA99-*4_)t9Zw$Hddfp~E zZ1OJFw#;wc$g0M5-Z#j3e182Lf$td{nrTd{a2ydkOu^;JGZ@OdWN<*ldC0w@nSeUC zllt;3DSK0nJFc%U@0sXKsw?!$uq@>veD_)uQtnDzxyh-;Ybp-O-N&ErYeNXOe9K)9 z?479Aq*S~x9VrH$|9U+iCw<$VL#CWSe>C=kc5=~T(`#1h{^sfzO}CryEz*8z6UYL6KghENWDlPQVCl*ot15`KPNeL^`$-$ zR8lxFSu+R!ek$jd*7)7IS{c?wCXoTk-$CV(d4E<({fW@*IRB1*y{$i7hNPsUS$kL# zoZ>LElGW_G(XgOlcJ7{TKxOG9|2S5?^;yawM0|quTSVxbitE)|EnF5q$FnVFDM`@Y z@$LNhBga%R=p;R7a^_{f%A9dxU}8rtfIFv##K2&1#((|>MTTvV4h#F92_<|^Do4jo zfJ~oKd4j55Jui8~m6QgfGQG|UsZR$@x}7{O?qk+Rkrk8{LH#WDoSWQCFD85MYv?q6 zl!2M&PB>T_&F5Y?Etds!LHZ0fnzSHztflRO-Jwzqr=|{+$H$4LGDGS;kkpDI#1{Y7 zyqD{BJD;d?m{T>vdqeU|74;sXsqY%cQMa3oUOWa~muH{CYm^fGRjl83Ira|x#E_eS z4l-)q_0cEJc}U{0HC3389f|F2iK>w)8BmY%4(qv%r`|AwTal>|X##0)mlb%-eowqn z=0Jg1@f?24Dnl_sH%%uf#hvCQTFv%2_l39q(o09cxlPhkud|z~uUuc=Pr>_VW_{zU zg=f1qsn@Xq7Fe@YZlr8OZ=R)xM#1qmeXPnKq4Z6yEH70?=bqiVmWIIJJTeM?fAFH; zzLwb(kfCo3|2H60lu;b2s#c?SPJ`!*C#PU(M?iBO>8@kD0ctULUaHZw;#TJnBTB{9 z@_aN$O;VBOW2HRxN{QKXbii=ktGQA=b|6}u}z;}iUVuH@zW0Sp_W*$1@}@QtTY{j=?bv-{AStQd#%vJc3a$~%Ls zXy2Z}zh|0)-?%a9z-x~LjZr|0w3bUWfOgHuQiv*Lk0o0x}RxF5HHS4Ap(F%I;j`cT~9IV3v%E`;fT(nF z{DybC$d|lv_X=az>bpwElLXKW9v5T!?|l0Y6E_pp<~8&O=Iq9nTatO@1=PtOKiC}) z)@7{=@($nWK6LjtPc5UyH=ogU5BZKECA3z5ZwMVHY|zKyMv*ZMjp3`Tje$Zlpn6S; z{!A+ls?-yo*UUEcAwRIy_bY{}WBT#o{)|%=8_v~p1YE{`x43@b?Qe}IS8RqOLu^;H z;7v00dEN?mu{N4OXH+pY*z)mtGtfy*K>7E0mPR(JcY+k-Zs>m+}ITYKOYxfJMN@ulxW6~B1 zPrC5T3_6d@bh)QGxgq(5@eD@&T+98JT>=v6g^(nt2Y*bt#^veU`Inz~_BS`aeA4f| zHlma)%%EkL*mmBP#M}+893yZk4(zhsGgl=E^@?Df(DYlUV?eg%5RDfo*NCzxQ=2$@ zfg}COBfB<=9yY|A=Wm2{8t#>)o19Gfqo*W-4_q(T?YOYqx+uc!XL>(jO*WK9226k^ zX0;hi7v%?PBS>?N&&yBL1Yyx0Hgki>j;?;KA2bkXI}vC%+!?#v&hr*4nQZJSn9mkl zYt|m=*Y>@pW^^lPaH=M1{zz;VuNMi4+8<}Xs79C;6B@Jb)T_C%-Tpx(ce@REs38fr zl>;`M&Gr-D3D19lTDUZ>lxRQ4K-s|mxZR^5s!3IzL5YRmASR%$V_BJ%pAtK2vq`}q zg^*wBDD$ZtKWN065k|$u3N2SvyPZ)ZAemQ)<&#M#u#pEm;+e)iw1pc;Q+vD`XiMRl z9P+%H9^EWoTv)4V|8DU&I^x4OJ1^*j_VKsQ1!@#a zT@6g`a2vI}B_+uBSAX}3fV~+vzhl9u(td*53Au3cz_okVczn&LBc*62-L0IV96MbZ zPSAkZlf|P)>SUFo%16#h@Hp~;Y; z!%+c>Q8`P_tw||&7rWKT@R`B!hFSl6*}))VimW!$D1{xk^^rpZ=>?^4q;L2|tOlc=hk!+9 zpiaN&E(1DN(1fEKg(*ceWy(8_w0T88GoXOCGB!GWSo%xF+s6G9?P7c?#AAlSBAy@f zVZx}pn4UTK$^k$B#DZtua0uYD6J+V%jJuTFFU-Ie?Azb*QjcTjHyeAuyOEWPZH^eyCkLn4}*OYxW zZa_II@|)r)k14Ll4eMl$>#bz%tgzV(kY1x`IPq#%)27V)3)DsV#5l!G0|bgd;g>2O znKRtqSPAR7``VXK^Z?v?u%(A`l^wEdla#~vDE`R$=?)AgoNLSLG9|({8tn%w|%?3Qflh< zaZp`*bpg`RGO@ByJB&ZEj4#a;&XxG4H%8}BoE3}k^a$0XgxW|>jP3bgnI0r^S_SON z^2YIIXJp4lFEVu4pMC!39-%&(GdL!Efi>g1UL&hXiH}gGnT3eqaF=({aRHsoL6bdw ztS+(F(wbqUMtO5t>_CjFLx``ygk}t-a$bDoof9c)CfJiz(fsR{l-YB z5g=gL#NCYyUA=R+y&0I=6u39_jH=r>Q> zV;R&c)tyj^jZ(_f?^vak#q~vb(=e#|8W-Zm;EU>yb{1A2^hC(4Sy;no(j1V$S$A%X ziWqhWjK0Fufm6~#+QjF|sXnxXvxLMpHN`l1%l#q7larhz4LY?3L?v+~-G3s^GdK_$ z@G8z_98Py5(n~pIa*<=c>723ABzZ+O9H9qOs+?~xft^7aQs=cA%Hr?#L#Bnw$1;xF zZ7d@}YeP14_7HY|kcJ|QW$j#$U~s-qx-cUf`x2w|fv)IxrQmwdLgU#hoelKBv=|yC z5QafTfyA1h_gew_qM$HJ`!A+l*tL1h7!YIM=pN}wr(goI15F=*?N?l&k8|J|RM*`oPA%r<;+!DGk-+OTilAxG;-{kf=}KAX$&&|>Nncdk4+(!%asy)pVG(Eu{ukE2H#Ds)RDXGURJdhgZnc$?(e$%#-|NWp{h>5i)UG{Tiz|3mdsb^a3^P_66V*Lt z&D5|7FjLv-+xXYi<^4#uy(_Ml74zvW78tQ&?2QrVX7yB79{QMTn`DXWNO$BE}D zvtY|OVEC6iog{22VRHI)E0+OrvP% zUu2@<*F6mdSYa(OF9BG3N*HD027ZsG110re3#IlekWS5wI3Z%-#baz$3rTvX5!Rb# zRUQA!y|Wyx_W2Z>&EEXY&~(0k4@4)tM6MvBRM0G}h)tUK!U>VCa=hf!y3QUiCjMu( z`Dz!A15G)*+6Rq?;0}x%z9r?NdV|VQZO@2$%D_?ANZ`Y~rOE>6GR5UK~6nT=}}Z=6_NyEMcq@w$tTnDNZd;WX&D;B(g}p48*~ zePv8N@M!iw=5C9tiAZPhbHhsUcZ*~Er8%h7)}(3CspJ~3QVQv*%_kpk#Cdb~&Gc3dTkz4^ucC5oA_^IChZjZ=imfKzt5x2-6!O8Cs-v`U)*u{MXU| zW7bd@(H2{gN?q@>VOCz};$!c)`cRPaTu-vfK_|8xVE>h7f7XznDs2)l{tSapnOBmz zI55@(-J)7aptmbvM_3e>iQYL`z#E$!qjbj#dtKFuQJuq2zr|FJD46#8-^7)*h7%MN zTrbh>HTRET%oC*0HXIx$lcQ%&LntxAsRn?{(nY?W*0D(h`B?}klT}%;FUe(MDAOqj zP*;t!6^OnY5E>6oo@2r|CaWg#UXSr+MM+ahb67 z;eMmxqoPsp7qIbJJmUrDJb#aqgf-KlU{7q?BBeQ{VU?ozctLZhyuIhtce==&n*8Eq zs`@3mhW+23)>9db&dD|D3m(f@9K9@Dv=)v{w=>+t>IMd?Rjj0qh)2%5qgyxc!WXt% zP(yX1SsGeq9rP0Q>vdc-s1N?j@-3St*tEDM`O)C+g}#9dC83E_=B(^HFHK6tHpF7y zj|0a(QVu2;iVcQF&0ru6xy3CEZ=K|9MlT*~FZqJln)1nTdORlPs`iaUh?T;L0&8`; zXmi#4zqf2{^=>zmqy8Q3Q%qPQ#Gc0RLi#!`(L*;=4ku)Gm$gyIdO4g?(6ZPZT3oIS zs5koMw7X!0j8t8rmfY9inR;kKVDW2`g=={V!epzQ`}G8);${ygM&Qz7;szI$+;=-1>bk&=Ml{g3z!FsEkGp<4myjsliDmsEq2BA z9)9Q`1Ci=|VFEX;u7~FeiTX6gM26cajG@d7v$Iz0Y{u^_K%HpQ)dMqgz>o1ai0HFa zTp#}X!(cOoo9vr#jcA$yHo06Q#OVrr~C_gV)3j*=#EN1~LqT*&@Xa?=+SxC^XJioO74Uxo)f& zBk2kYfi!hLFUG{>iW{OZ@tn%9t*+_pm84o)QwKJf(YJacEd_sl# zFATkZ=FI^MjCLHcRB9jC4YWjmN#e*}8Mb+$!R>nxm;(hB+%u zrd+tF53#$uE2M*dgpql6-@O8(OyS{zOHjV&%m35fmH$K8y?;FVFzuOA*|&+x^5`)X zSt5CigfMm~OSU1)j0lkk<*|-kNY*ifVP+U>A!IGaHijZ)8hsI@ANan$ z^V9vBx#yhgT=#X}*LA(GbDtw~XJzJzCf~K(f1Qs;PA@?HY?W%wL))%m#}pyX$|neP z@(GNZ3&S{OpUmhSJl2@h)(VH4eLbXY1aII8!Nc9KklrLeiSfKAv{023mG~B3nLDiy zTl4%Vrdm*6w^nhlcq*# zm2b#=VYcQr*wR(<$@u#ar#9h>Bl@atnw?kLy3#X9aRw_zB;kfYDZQ*Y3VQ07>~?V= z5eKdsMi5oY2=l=(J#YQ9)m9}1eL`i=y$;^YYC?>d!@$mWxo z^~3$^tYM&igMFY#?q7QI=Va7OHn6Gur3yH0y}g0^4)%=Pb)^ zC=2bcvb{QKIswmr7?V+e!#AK+0!OYADT&bEPn%6MxmQDw1D}~zWJ3x@UeLZydMyFa zEu;-faX?Ztj{;r_Rt@7+^gaz&34jg~`rm}hR&rOlM}Pcoe#-H$NvU5=NgZI9#yieN z@eJ879QH&4uBl+2FQR{0;`Lh{vW(sw7sgFBIX4oW@0A`aBJ;IA!*2=gb}8OpDe(ZU z476&p)ZgOu%is+U11`pUwOM1}g?s;Ul|2$2?!olZk>DurO8!)OSlCmugx5n-*%Ydxpp%{d78sUN#hF<5^ zt9O_xP~k*CBWur{t-6lck@)po*h0O*8tuuo4>$2B`GSu|k4NBNkWdV!<-BZOxB5*d zg8n0)L${ury9w80oCuUm%&f%Mj5?ELV_u5{DFqBxn&?PCMQE;PjU&_dJS|HFfLqUt z2#mIA74D7965m(@v)2^5?mqn{H%dF+*?Xq zvrg!Tr0d}~>!m|qt^%j-RoJ_kZogZdQI8_jak)$tb%YIAj_%*{avEhIyLdX(>#d&k z16p|;I^RH3th05f6sL%)6jH7^XOG0{NM=t3SlG!OEae1Qfh@tKTy#Iq=oUtuV19F= zW52YQRm-aSZB+=w>(KDd! zB1T&`J1J zg;&^7?|qVCnZv6WdGIe^iN_t5IAM96|6Sh~i0U9QO%H!{HLCEL9-cw~NLHIXv5Q0h zVYi+c4KaPB=OrV1Mba?5L9(wKIC!ohk5&a1r7~`X6&-33<3AypM$po*KYQZUce}=9 zQ++XF+Oo6cLmc)DJLqQR8({EGW(Tbh`ZJ@KtTG(ct40+M03F+}ZHWJxVxcd%O?2B+ zUVVEOCo`7_tMNz>R@&%@eSQGMtpE{;FLs<1%{IV6QyliW!mi>-r3Kh6a-Vg=+Uu*j za!|>QvZQ>%ZDL%Nq`RFZm6jf>77T$Rcq`kKJnfndbWI z>#vgs1VaTMVrsW)_;;a!4{RRTOIa*kQ6KyMf}`ho;173og@s5CSV8-hGjUJjXlph- zG^Xp1q${8GsG@A4LdVKAZqCj=?1ET``O%Hf&kz3Rsk@DWO!n|L!?Gdiz`yq3tdaAK zcNTB)+~*j+ab9(G5Qkbk+Ze|}y5m)Y>LM;d1i2yJ!`s z9-1{lEZll6KhfEka|UvqIes!hR83!Y%J&NAYG$6=`>w8RRlWpv|GEq zq(xnMxF+08g|w3vG3~>{!lz#Dfddlx@}!qNz$n4kpuV#fEWGe=p7gbG<0pPeC5PB{jWe5KJi13uw89_1*VD5bCw!?%{4o#~6JdoW zek+cic8Vu>OE^q4JoRDvsj}xh1G%xV&s1``faDQuwBgivX4j@PzH)xvUT7Tid-&Z^ znDRaR<%Oy3)d=RY-^9H#{*}$i^E!uM+8SEe;|dE^iC@e{FJxqBD=uIK>!>5QA5M=o za48o?zWFIu(HZvbhW>hHP8FhSBr@Skx`#ICl#tcKmbVu9wQ;<6O6~xYc@Nnop zC7Xni;qb=lu|tOrIX<5f;l!hPciOI|FE-C)r$6&m%Z9Ps`JKrph`}Dt352z(!QBRp=liBl(1cU@fD#)WS=BN z3guj9shJKQp0W|Xr}SeIe@tnyf!ctG8fFok{EC;SQ>}tH-@X-2_5D=r@tQsR@6oSe zBZcHNpBs8as4d)hRe(`Ei*~cm)l__$&voX3v*tjA!!XNz$H}qRo0u4?wYW88b_lZi zw&gs0X{LX1kos_8k<7SD5F^r1I*iudo>DpqFTa%TL;H4uR8tn{fBSrjkG?iYfUZqX za8w$Qu3Xu^?`q*Lx#8kpk^~cX&~nZ#LU&d?rccFQWF+1447^mA@RJI1P6;W+eU_DG z*E`yP0(q;l6zY9!*cF4N4J6|9IWDViwrqQ}Db=;=U%VF$EOx1oF*C7;e6fAtaiL(n zK>IIR=$5|ghOKO=5W9t)!D|2VR=eV4B*(dJZ_{AhGq+o1TA&8;FgdaGF{FtAOoLb-Mhy9bW zP_-R~nz%8}u?3*ve1Xm7{K3_dsFC*vM)*cP9;zHX#}@UztS%LugC-0Tm9js|u}1Pr zBIjfXzIt&+8-wLa+hM4Ek_ngd0dzGa}JA&%h;Jp4dd7ezN;JNqkauUz_TM7Mv3TM?Yn_VY$ke?^;(uEgpe|q-BAyRK64{(; z?@bi&&PSpn32ML=;}-^Rwwm~hW<=n)s;el9$4NXyrN{e%gYwR)W z!hTm}_OHCvS@b5!KCcfqx4Cp#Fh)8t?983cu~_T?cImM@JWtDu|9MIdIu+dtAGD%g z`w)@Dvf2P$5gBucEySAe6K~i!@9@N_%$g2m-BQx)RwB2^Jy^Qbun2Wj@tOys4F%h% zn;_qqtt5)+mT@oNQbb_*#4IfeV%Lo|$F;&N9;sb=m3@=v6t}2VY@N_yw8JZcJZZES z{~^7d;?Ig?LOnpKNTiCKAtA<&o-D=KExc)@Hx-yE)_nPKe@LlrM}OA1W2dy{E}3A! zys(lfX!4=Zr;*%(fV#S=m~0Z>kYRPOoZuhB6&~>?Nqo%WZO<$q z)#Qd_%RX3_ zj`>rAlvi8QeRo*!pnWWLo)#f&fsEZ6yDQ6Y6b^MWtDfhzy*PP)v#CTfnVfB_)UCgI zC9WhvL9!%aEF5B&y~Y7|gZKy&w?EklZ_8uwqvJ~;QL59CC-AXVEdG-kKerW_?9{D2 zQxH{gBy^4^ZlCFE6An{NN@}4f%a4K}^y5unoYoWxL8FQ_`^2`B`h5;JM{WhX;XcLWrDpvN>PoteW&4Kq)4R3)YIEflF^(|AF~D=_XzJ*jmcmKy61ds`WjtGQbq+UMi+w z04$(@+J6}29^paNSfH$mEu{jqLizXKnzpyUf27X%0 zh3FsK_%q!C!nS)o4~11m2LlwH`L|9nZ}*8)I6p8{}vGBPnxZpUClw z|46VZ=lO)vZpD)x=hu>J2`V4JSZm8Px1)IUY{^12Q9M9~E6UQsPk$gnY>2ab(BkRz zD?Sby&!5NAZ8Jplg>{lpdYN0hk(b?3Zs*KTtOBa@+`k1>Ppcn|;yr`_4P1@iSP*(E zY=T({_nh58ni6ot12Fp#s+S6xI( zW%E2lmN+^?!cq>{g$&N)%Ow*$n#75Ksu%S5@SalNx(ST4 z-#@lraORV3nc2 zNkLZsSM3V?&IET{qKpGxw~ps**}#w6JzNyzY-; zwOtct<^CZ}c3@Tot@_l|Eo$!l?U;U=cr2~IN5+~T@T}kj0`xp$zO5zr{efYYo9W8U zK9xwwp1tb9mDc$UujE=~^dhC2}^rC566i z$odqI)2n&S%6g0Be+$pz7(zqOgv1HXEEazKFL79xB|2)rF6DyCh{Y@$^0zTVUpR@uyx&+l^D)ieocR84>OZIH^2m_$nA-?s-wR< z&pE=);>0~_5q=9{$$gyMz6O9Cs#@MGt?W@Xz$IBC#0`$w(=fmb(774|leUHh2Y(YBKSvXJ%`QplBUtD>-;E0_xBt6gK$Ud>i_o3%=f>XZ zlwUt;m;j_z(Z!B~zoli_4jbn;K==Ruhdn#_Kdz-^=D^Z&ipq_By3~$+yiBzZvI`(Nli|c$I#ZAQQyecz?jj^+U_F(7@r#tC}?f$q)*~z zZDr%gk?cR2 z{C7TIjU5dg%8-k`7YeZ=x8IG7uQO8&@SfQ9d$l>dtRuX6a9J_`R|mHAIg|A_@PRRDpH>3_OS z00CEL3lH4oc(+|FYqlsQ!f%eyI3tKkML;wa(S#Z1s8ESS*^rk}+AR6KWMgpittUVTi$) zOpV+*xse8q%jYz|%*+*USJ&9ZiCn=MS9K^s#CGpO55pf~Hkhsg4vgChDy9~yK-O*~Z_`U%tey~@q1JVj2T2e)AlP!O)XJgNu z4^C0R=k3=~sxy^Efy*U@C~KlzZ(!yQ9M?IO9lO9lsER1?k-L~US*Qu4$>Z@SpYPl1 zf_Fi4v(zyAkyr3#&)MnwP|B^PtlNx>c5e^aljT8>SZDT1qtX0quH0;q;BU$$hh3D& zmBBIm1*7q*M7hxjtbCpjIH(*Mdrj zF`}P0bHlWo+x~TvX)-m=`R!m)0fj$kRv!%NPZ-5yt^6l$?*IXy?e$F*sWGiLy|xEx zO0QHxdZ1^6ZG5f-3i~s%ruQcwl%9&rhUemmB#IM)kmCib`=KWl#>}3J?LNd`TD`th z6!NQ0S3{=AMQaP~6ymb;1$kfejy|iuED$Gr@fLT+K|*^i_!uHMe28TV8mp!jgL11S zwc`TQc=|vaNd>mS)?wBJ3i-qWCHgxmEXbPGTvhy@D{tx$h!8hvbU(`P_gCft#GA89`~1 z1oXjsAN@zu8Snybx(3PXs*W!-#>YLoHG*HwelPv5K0e_uh!yxnm~Z||r7S0UW|RWI z&_eEj%6LvimvL@=GXTc0s1j@GWgfQ}P<8&><{2BlQ zuv#JeBJKIUc}RdA4e-l00RRKB)5=MBwXG0Un6{sTaBLpoB+enzU~SvtmUo26Qax}6JP);0*m2i)=c!$CX%pkQGULdz7etXjCkfztmo=?m$ zEXzii4@lK}JeMRC2Vc!A_qYDY`k+o`d>+}-cp`?O)@q9H=wj@?!Ek>3ZH?FHt=!z< zZWdlDo;I4@>M_}9LGsg@fCZi%_h0pn9Qc_LHVba@h`0%ws4}gZ#G&$T0@x24XHwt_TsB?&vVVGbWf%< z4sDdvjlde5iJ)<@K8ruHZSzuX2m9|83AbgbLzg`-jp)7;b>pa3_*6Dt_*7SaUwdL} z^~YG7op)Nw)%Z}?5X;hnAs*XKe_kmyD znpd1y`36k|E4}Ydeqqum$^YfF#kbM!_M(s#{AYXnj0Re-K2P<)1sB>G?TUm2n|K;- zrP9LIOj>?&2-O4P8>z#bIE4rVak&Ki6cvhuHQq%TUGpS^uhB-H@~@^xaVdXB?(tEj z#^@E{)2u%eD#sXZ`*}beW zv-@Fr?|pEMw^0uGW(yNluRS4hu7&Bd#$4TEJYc)K6$m^1FymknSA9x;a`EKuD;uwi zT^SF1e>Sw?*mnF$fLdjFpPBjO`Rr7Vbh*XldH=S_CS$gIdf`9qP5woKIk{^-o&?@dX9p1f-M_;^9{XvBpIS=>l&-E&k zQ$uMKt+KD?L94AJs!5I3D>s78p3lYg=Tj31(-j}dmHPWddJ?98cej6r}D!yIR<9sQTb8=N!Ilpo_rLQUHSt z8k3iRYEKGqw*2UT`$T>Q(;?7J9@&ie>oIiAtndISv*EFbb}q)g-gENWkg@lj?MkqF zK{AyGD3u;XgV9KWCSES{8nr(O2AA($k}MAQVAlC4g*^m!-=5LU>6;)=i)dCQi-TyzT8I=QpLzVS~gc?D$Zp%%N>7> zPuL^Zd4bv=zFT{K_}hU&Wy4qKoi?ei&+wLlY#F3o9boY6YFlUT5n%J9EcMO0MYV=v zXQrjfu{Jhwhgd*7l_JMdeWkKp_87jTaWCnQF`c&oA*-g#L`KbnZvv$@%yoKT`I3m^ z&hjTm-3c5P@9Zk#e>5SbaL_JPd0qcVm%2*DdtLn}3mS$$ilt^WxctTqvIJ~U+IbD> zT|)$Ud=5eiY%rvG>hWi5&4%03Y1B1KFgFL61)Ahe}Spx2+h53N4 z058SpwcKgrQ7hhTONOMXr(c+iI{jE!lkx{x&c&<}{3Zc*}%t$!W)y zm$~VVWEg~wciWU^drhZQoaS4s7$-S7Gt8KGL_}jLTY0OG$-u-T+9+x!(`T*r*$(;CQ`7Ht3F{-XJ z;65u^(<*ZEviBt&eRxE#GF!35fa7TXK6K&ix)`i7LC)0wi~Ak=N)xJ2<&R3muzn?G z6pjyjLFhaAg@?1s_rgs+_wk6r6eJ<~LWNK>`c7ZezR(0=0y@1P=JH(vr<~`})lUh3 zPHjbO!jn z^tnm*8JbM`dXB$|;*!!48W}kI8|1W#q!|BkaVmq#3I;FjjcGmLxuNX2%eMW1)UC-hs%G)OP9HbJY6vA-+u_#r*0 z&;#T|>H9;MI!Y=(VG(Vv*V)f8V$H_K?>@R7-*K8U;k{%$q_i_Ct~DBMUrmP@y@QQT zA#V|RbrW9TaCakho&NaaKbDez9)_O2)R;S5y{IzH`-wKxU68a0{flF6ZR70&Gia%X zZupkp575NB6Hc-2kKrKdM!2?prhvE>O(*TQ?2Fbmw7;Gi)bZB(^!l_3d@*1%8gt)PldTV zGq{T{p57SjIIR=XF>Ms?s;0?1&kz?>)@7Ksr#ulaz40X8g%0VPr{TI$6snm<*=Lw< zsfmUXD1+}R^eox|I0BBtZswD_S?T3?=* z*vd9x^N3L7Ty;Pf zg!dc1rr4Kn7P|$t1_lVIC!mFh4m`027Z|NW)YIRH@9Lv}VDqjw1@IbFg=B~#DIh2N z-HG(&`2Z%qeS?sf^7|Ivk<1&*aC)~Ezew!+5QJ-jFy~lmoXhRZM-UqCfXhZ@Z`Fyq zm)Ag_U_7+Y!$fEmt z+>P*9tI1@(k^Qdo04qwme*EYtmGU{cq7~Ef8TTN#u4Ih6>+xH=ylCdPUUSv+)n;SH zbAG2RT)VNkI;!YLVXTQviXSYc?uQe=6%+aG02hn@nB@R=2peHuxd{hlh@&_groqT$E6>|X-JSB4&q=TkiM`$*)oni4uEgm!Hz@Y8DGv&NlnD-++w zPaKcB)w$hos#n|WO*lk^_~W1Z{1@3>>_|4j_PC<#AMtc~GyqNFa%JouY*D8#i`G5M!=QG{)o(gs+Uoo^*YP2qyP|oKq}N>57Z9~5FwQ0iqnly-hb@%!H_yki)mCK z#zll$$+<@or=4j3&7a^%Km-L!XhHMe@!+PUc_@K(d^B<;{{~rMU?7n770ju<=-=^n zSfF^W{|^g@E!8Ym$n6-_Y`#8`i=)2$R=8#S{o4x^JTbX!f)UzPHp9;x|8>>>h4IG4 zJA0PU%a5F+ghJ@mVfotL^E3+@82BPm7@3BY3mxb`Q?>sc+L3TL9o}2COl7L%x(oH` z%N20ff>B>OD_oi~`iF!32LO_+!Zii2WZ#IRiIeDe7gF%?@vDr#xVB47@6wRb6UOiL zTc*D6%(Yt=SSsZ3jmNOBMbaNlJ8^yen8QzSgr7^Wn+g2Aj&@iSo?6cw6jay>p+4vM zxkyDkm;&}`Bop|S2ESwjf0%ZKIt^VVuL)8sg=p9E1aW({`~RAPKpMjff0h1G+A-N!4O9rj*4QQ)$Mm zPU&=brBfKy9R6?6Xy^SwG2hryu$gq4^1tvS?6M!KJKbLZxR7`b1rG;rWdDa^l0=|S zDjQ*XUu0NQ0QTVYW{1PQJ&RnLMbKX_*$QSn?B8Ph zG`l97sKI{~f#As$A@`!zk0Z~P>L)JcDF_o(%wjr>-Z&k>o{&l4x}@m%irtt>a9mjo zl@-|e_s@vYTL0X%;UJDX+Uc0!!jrsX%)#$p-yy~&g<6fccD~K9gGLKr1QGd`#<(wl5UYt_vMIGTumW55w~T{(Wd zhKdk@|B4BY8hhxPE9i@La-PS7gm&9|(*fEkg1EtBsdSn`c@XtcG;9I;P;Xh>KM;}_ zse+}GP_8|Xkh^2RvZ1xDSWg}-Mr|qmDkCp9yylNh5AzNd(#<4s09?NNNhfPH97qQ9 z)E@>AWh^l%2oXUF2**-`Q;|;o*>P1al!T{}s-XM^YKrPdQ_A?KBr|faO}Cg<W1q z1c*}Q=F_Gi`^Ls<7FORNTaAw!bOs< z2zc6bV+i;U&zni%+nk?^-P-kzBL~y#sEikKPD#TVJ1q|wvM63*?&3a`y>T4G?T8_X zl<<2!2odpnnB5UR))_5{xIf+evWK(X=-9KxL?Y&?1-{%382QHXYUh1P;a($~fuaLf zCEZ+P`c<~8bB#vgFM&!t9PDTsv{F|hpLX2@^uG4scF?P;IzC%(w(eFS6x^nJSzqI& ze7OpC?C6Gkx;R<-w)Drk?quc98fHrQbqq1L*>u)~y7fJbCE)$SByN*59*3o?v zh)&}F`dk%JU>S$>lAo&8F+E{Jqaw@|6gGL_}e1~fg7i4%WZzI~_HCOH_-PGQn?jW-fN zl(1R3%*_J2&1a232woqTt~I%@wNWwZHk0>=-`;a(5R2a>8W5)>uoeki-8a=izwz4^ z*HNg?l!1V8BtS8=ON_|K%yb9B=Wqb|l*+rcA^zG_emMWNwMexjrMR=h_TB||a)<65 zZ_>Nf=xQ#|KKQL^NxXyU{!Xa=p7 z&RK6XVOgN&4kp?6fHoW7pz+?I_`uz+z-QJ%RqRaB%kB{wanzR>$RUVmT~u2CP4qA{ zvXJjpcj-!N6L>Ns#kxbpbtOwbm;QKWR*P+(wevPGBatUotZ)w1CP#YD<<$N0tN4V^ zSyeZ8vclLszSi0p{O5A1Za;3v&My~s6Rb`=C)lb#7#jDqfcY6I^XYiv>Uwvoim(pi zkXkG7T8?Uqry*sn_x)me&v&Hx+O(Xcj^At7k6yDzvT~!z1X0juyL4$InrqQxvvCFc zyyjuUJ%PqhC6Rc4V|`J>@#+Si+h;p0i^qG_#q9h|n^M4YAZi?MN6^kL_LT1olUD6J z-%rmp6`cK0^BPVbk_mR19&LHeyYI6Zg2=M?}*-6xy%y zxtEpfTLS8846M7>HBGU%=#^USMjfaseAtXRYz#D@#U4#QzQtlp**!=aRl<5DnFMM8pH~1{V0`xz0EaojgyI)}^ z&B+oInyHo&FxY12)V02H<1&iP-iFe0&NcWQfW=`YK#S|ot72A4Xuw{Rb zN(Q*7$8wDQ!DF3iw~bPxT~@F&tHJu>D=kPNW;HWox>z=^)KC4H)6P5OFS;I85!N4t zZ=fMmr42N=>gjH2U-QT4mxf(`e|v2$?i@eUVKMAa3Xa@Q^SpR*##bSI-RK#@gC@GS zR+?1bY=1_56R*^1|LkPKh{=pl{jkh5 zJU?Kf*Y$laih9xi={0luV*zJ=_FPCmGx({ob}wmPZaMv0kX~6CukoT|9%@tJ%7}(y zK(o2XZHhI4$rk+5HVP8)=+zTBH3%LfeFU9C`m^u&>kqtHOB*pV$fqcF?dTAgNwH8W z6UkiMVe(HSs>w+-%9$)kEC#(vMB2~kZfcVoAu0(Eml)E&znd5NWEWGmO+FJ#x5~@y z_Hq%k2KX2Dmw3sV3 z{YkBs$Bb}LVLW;<)U<$$`|^q-74`CDYUHxHsq;&*A)>0seSO@tMWiM<3`OevZO!Ou z7pUkL%c!;CHx3y1HMuX$(K&yGeD#%E*m5kJMG;WaCy4Yi^F0Y83qAIHWHf;D`S z>$ZS0#EsxKnVm&=TNw~UV7Hl3;!~>5;}?sjzUTe&tmT9AUiM4Ou<6G&F+Oy_u_}`c zNGHXmcePn#U-5e*m(BXR$UtM9;k2I=ib^DT!evkHT}*$qwpml$?=qXGlR$0GN)whY z(xw>bxqV0)9=X)D-5ZUsSdT~42T#D~MmV>mgQB$QxBQ6s&c0A9v;w@h8%p1?*5B}Q zA?5LXD`(IdwX}xWL=nq%Z{Tsui(D0O+A&Nen7=z1OWbf_L@WtMr)=;>`HN!N=9~K% zdFs^Qbn#4#cZY@0kU}9$x9;uq+8SXaCZV)5q2V~9wpa|A1ojEa3Cpbf`-L6qk^+3| ztybG^t4v&Lt1F~i#O^N$C#-U(cJq<>kX<)yme-UiVeW`C`T8vGjhX;&YfzV3ACG@6 z36xJVe%zoVf45#DsnB$a8(&+&3GKqTl(?3<&jOmHUE({TXK=hJd)ytF5rkyS9pjcv zMo@8S#^{&6xW8^{{XF=`WD*G$8eG51Pk@THdB#cmS`G!ORC!vBtbc5LGOdz)@kE-DMNgg4xCz0cLfHbxy}k@ocBJ#Ms`QsA z;nLS%uKqSp=NM!5AvK(5W~~>*fMEZu6~G zeBaB}*8>clB$gZgTD3x-4?)j6u_WoAF)lU(aPu}yaj z;W87U2TEN15nG1kh6G(t>YB%hX7dtUK`~T!iqM&syBx!Ph@7RY9IIv05+^?y!Tyq? zZ=yxArL>Dv>&`>aYH5W)G=jf|Fb~}7F4c`p#6|#P$B>7X| zr87Pcp5y))U_K9a7Yvi}JJMR!{?O>s%%1h`dv4J+#X0Z!ki;i}yHBci0=+%n3wo1Z z&J`GRA5pVPS`Q<>tG3*kSt9H#)$Yatv0g}xWqgfVF576{xN3HWg`;fse7MNaTh$*q z8UkjkX1=y;5jouqhC}{ZvZg6T5k1Bt99v9o$>r=PIZW(_x0=l>YMwLYz#lw7{w)bS z50Un0E9fv&))q(3G>WlLDbwAuszqGf8HlA&`nlQ2sQ$Qx12eEZQjP<{%OKm@F3K;{ zF!kt%zL4L0b^fE$L#dla8i5*!|f1k$3C_=)XP7Q+?y6Pxs zFBin&kPO#(Kik!|`6z2QCS76uMqE6^zq~HiXfBsSSu~ki%??vhxSNkoU(WJFI-s6b z9hsS{YM9FF&Ur1n)#LUzb^E)H=YQ^f7HyF&{*tMkKX0o+XRkP-y&H&bx6Ivplx*Lh zZ@J)QBGAp*s^|Z?SO%@+%OB3WUk6cgODqim_g*&-`4xHA#dLk%iBC~v+flNr%&n8H zhbny?kL*CJ7oUCZlZfOmaxjVF$6TPOb0_hjYl~BnuBU40GHUn&`^sY(1iva8V4@UuYvJ^2Nxe9 z4ijg@yv;mdB*Tai_zMA7Sk45|Tww>8ziOq3+>hU_nTc?kL^HI(kE-)9O)^S7$s?Xq zZBM>Jc$;!#7U&d01qgRe-3Q&5*6OWFDj+*UWIdWu)f3U81UJ2YG1B7xy3IwXLD}r3U>`3w$ zkhZ{)s)IpsPym*pnot5W{s)KvSV43nC@Pms)i)^eEwETLS2zn< z3Y4cmGl|~3K3t$%HmNj9`F`GZH9N}wT7zL-GqkD#O1Ql+pSMK^)vcQ4koe?8lSm)q8YPzMMJj&*b0BlltR6rH$|AspXHW$ z{#G><{nc1A*dnI$9foh4+vQ&9miHJQ1|@{=+oU?qm*0Xayd5+p^0{&Q)_*fTRWDf|m4uU#XJlrc zgpy6S3S!q_(@QoI+~biX7Ds*B9}s*#jUD>1BI?$IeFvH;AxffO?RUT9@^B8A=AHNm&%vyjMBI3a>ijToQQ&6u1pJXd1Os4lJB9NZW6W>wVBV-t@&;Lt?u^) zBxdom7jJKnqsQDW{(_`zIpf&!t517K`bEGwc0A*nrcAKI54px1<-U>h;2D()b=u}? z)>~5!V0HFFjSS*SfGOqV*~E;gLXmL$egpJoQwytVs0m^(=h{;yo6P4k@iEKgep+(r zG&P?3V}0z%^R*bYe2;_5isIYN#NU64EcO(+Rr%XLQ=-rT8MvdA9Js&o2=Ap_(Lk@or)uG?F=5ot)>2|mviKTWMolbe|J}jp z;CNsqfl`VxH0C4~gDN&XIdo}QFFjPksoDLyv4#lD+A+0V>+yn52ST9Yv27`2faDH7g@aLf3O=QdPTV-y(}2}y2gb^ zH5L80tU-5$(fwt&rZ+}xV`JqIlOHATPdCp%dLuy(h_|r`3d^;k+v$9)32Sw9(WRNf z(0Q=>u9+l#FE4NHo}a{#PyXmX`{ENd98I%nHxSrL&+kl8_H=8Of&cX+(G_ihOx% zlwahTJd8}~s5ETMvEc_M0r%fYm+JNs9(~_w`XW=XToH!W~wuVTFRnzdsO3{W;rn-Cp1^6>&*|8W%=Be3h)A76;bR3^l)$MgDzqwATnC z3_=hNec{eQ)tAe^nN$gziCp)GVC%?QMBztWhE4V>ezck}z{lKy6E3bWQItp7;vTc- zAE0+r^$cca#cs_KWRCGU&jJ^7B2OS21^T}skaYQyywJ{eopJGM``X+uEw@Ky`|-MK znsPVrhmSmo_$OCSZ*i45Q;DSJ4Vo~yX18POWf-jbem-O53h5eVZr#;tanfNm9;MAf z4rf3*nlWY%RbTPF{cD^bC}c?2o!#0n zSyAZsk)$54Fym%_JtEw0EJD*-FW=O{1jZQVCFetBA-cMr^Uzez*}(_qB4X20GJ$wH zP7cclE9$OP49B3j6ch={;r1L(dGnL^xI4jx@vdUuVLBXj^YDOi`aKgm@gFVXaXM4w zb0ffnouX0@?9w&moE6ybO5nTlsFSgCK;m2C_yGS*n5+5-j~E&IvbN{xtgrT!D8e4* zX&j;N+3dS?r!GWH2Ni*P&4sZ8!nOv)h_`YN^INyp&Eu%=;&Ytqfkv*7l|bOb)glaG zD6L2%XsbO(sb9PtKqrlId}0y6YN&lWiS0^RfbFx`EDQn^Ps2H`#PB)=&4)f_XPv`T zo-t425?e>7j`2N`s>sA$%3_bUQi!D7&iZdPE5NJ@>O4x)kZ1}oM)m53v0M-(1tf3`9u zf~&zF_+Cois}V^7$VS&tJ&&&MZJ_LRWD_Yr%ERdD=C;IE5lJL1jvR;5)uH#A!bw4p zM!XcbGvHBEG&~)&FeCkr=P;|h(>)fWQ@+5>0q!GVD(r;yAlM1 zG##!FE7?RpYplAjD_m7R!E`?eJPn_?zi;tI4yZ8>!hH&MW76O;``p{qyk`si&@NHX zHwBIDqTl0=^HLR|!=P^q;@V$PiPh4WtgzM4pD**{UG%YQ1Q88L!1 zKM$T4cP-d}B|!~W6Lkt1_FoQpITA``T`-N~L`0uh7@-#%*O05Huy@|${5&6^#n!4h z0C;>)@+M>x`mi(heCZpef`A7-7s4nUO`@ZTj)y4c{{X3leq|{;;v8n*@0TJv8hZUL(D${T;DTb(wt@z=^E59{{(gIxZGy2C%Zd*a7}ykgL-r0{DFW(^`0&5 z2E3QAQ_zDB7jkxBgs=|-8-$u!Ny`&~RwRS(vy8|LNe9NGz?LAD?*y#qOZz%lUm%Qd zPoDRrwi(_H)^~7PNOcu#UaLJW=*u-X-0ju~duGPBezvou$@jcPkV7%3p8Mq52$Yt+s!HJ(1*_|eiJG0yLa!s^nN!%zlFR=IsDb!@-K;fN|vr) z58wr^#HMjPU>N+Cg}t_VLm#HAPq19nCb?mfgDgalEWxZp(1oz`A{8A6(KG`iEw~)= zBXtsS9=F*%juL?fi_5;8UIE!YJ;cJChYbqy;|9E#!{BrNC_*aWS?{7<`qdBzsR1e? zhNL`QkNGnILimvfveD}Y-)q1yl)JDjHlayzKL75538~)#tGv(sqhKO*?b|(;{yWqx z*3X@BEX-nk-dXgBqAPKAi~tW0$?>{%sd?KMq_1D_ zhv6=@yYFG&@6Ej?`q(`?-kIqOVyB-8G(t~t4t1%ad+*^kB%%_bJAc#!X2zffI&eiO!L%uS8_Vz@HZ}K?*5;gNOly?RSbv*vr}E9uw(}K(y%Y z*!PbG1RG~o_XNl|(}96D0D+K)2`k)*{2Ilf)Ra9lW_ps=a7g;ND~;k%XrE=W^JuBa ziOMCT&JA}(Y%_*lPi6-7`~(4w=u`PYH2jFjC*e!N=0)hLt?NhFS^^j>*~d1mv9r&j zsiep6J%rCM@vAVesI(C38&Wxj@9+Mx{u#H~F@beBx{PLQHxS$2^t`czA@Ll(i;up8 z?^WA?o_$eEQlNO~<~K9MuM&YK#8s|gu4CHhUerfqJlHaMlIBx2U%KkjruNL~6&{JP z7(CxrfeSwhWm_m|BQQ3F9G6GTY!XV;>}j<7-b&a;Hwi>Xy?ifTjnuJDs;hO7{z6V2 zVqV-&16>z|gE;Td=+5X@VhP{ouL#|VBVeZxL@4t4CfUPq58P8`0<;q;ZOrFAhmfDW zfRCmstHhgnQeo#K=74O352{N*0+mX(|{$A<%f6uRWSU`p2e>;)Sg z=tg8oOTg0KrxK$}dc3zU$yez#CTSEBY|N^2AdP#Si2dfvegS_RWzB1bRge7}#W?T* zcum@Sv8jQe)w#^&Gk_xWq8|lw`A$tzK&A-xgok%lH(}Y9cc0fLV1VZ;qQYPZbE5g~ z`KMYJV*5Vh9bZtCpOi=!O3cC&^E8-VO+YuN)7^~-^2#Yy>rr_G@1-rJs#xK|ZY_j~Oj!(}@fZevt;~LJ11Zw0wj0>L8lLvYPAVuG z=e~+8UI}<9)#S0k*EUvN=)Fp7BjbK{{hI^&WdV%fEd zc%f99y~JF?-mwkR8m$~82H!;}QkEvYg6DeAmtXR(!e*dMeGz_nL*0VK$!2;>t1Y%6 z>3t9N007UOwwo1JpkEQJgWcry5AkLZGr%t2j~#OvCW_cD3Gl!;Lf`z{s^$RMt^#ZV zt@?m>AV0}lRbzmS2%0|6j^tOx1y6Blh?sGoZ7R0Di*(>+bDq(p(jiQ*D}*oEMxcq% zb)GG$XJ$Am@gl5VE^y#6KNYYcM=Wee`Zj*;J+$Gq9(lj@9DJVi6|kX1EUJ*FOM>-+ zD0H141n?br>@e^^6j>wbSULR89Cc3L0dx@ZqH+y+3;w(e(n6*bNFp{iK;S1caJ6=` z#cYzrx6;&2-K<$~00L3kFVG36!_NoxYaNd-kFGj zo3V3@hdyBLqak*0ZbYb?O)#+8es2#j*~vLa+fW1ya1VjOz^!AF`6nQxcJTDG7h|-w zk{Rp`m@*;}xVI^EY-$eSFcg8e0vE_MbGVXosyt+7Hq4o<0_Zm6g4E+ui}3i%Z;B3K zf5a`MNeZEE1~?0$Zaq>-H_e1e0AN;^MkB^1hk$x0gNlxf9Vo5s>_0|bR=xv4ec|}J z2$+nNml)M(k_@cqb~pTJ=>#whvek-Xj2M+_mQxsy1`E+G088u7-RS*cpUSF2TRkga z@6Ssj!l1tP@~^F~>hwmZZaS>GUB0zBZE4vSZs~k26g|hK1Qj#DDLZIZs1)N@0PRl= zOKZS(f_`3Z&^$Bv>goDQfy*mTa5QZh_PuIma$^S%r{<%D)u;|eHQ17aM}rAz%#s6O zy+f3%f~0rQ2c}HaZS;nqVqBA|G{|hS`z!eCI{Fw3wrq7KawA(M9(+!9+}y=@9qerX z3YI9GG}-JWU`b(6m3h59=znP}`)~KP4>c#xF^*XYePu3{NjvW2C<#~vO!_L7AU$+t zm8a|o4H5qcp33<6q{ya`azUDf7ezi*40%IMfdQWcb}CotQ9VA z!VRYjP24}OPgNT@j7Kmu7?Q3D@8y1VbVXHWXVUG!q}5<|(Scv&nrjG^xbai4>mmbb z!4OnO0ePQ=Q@BF%3D_Z*;hclIZ@4k(m9LJSaM_B&{*FU%$%NMuOKofkdaeQ;9a#kW z4bwEzF!CJBKX{)d8g(wCc8f)*i}tW2g>=$G>?m%XODs|@DnU~QdZ{{Z3NKRp09;Q~ z_Lsj=>a{}yf4Bz_O>VgyCb>|UblT#E#T@X*GKNvGs;uUKh}Hv)x_q`M^tyZ#K^$eT z@qA6b3L*E$q{bT~pC8#MlDch+;)s0@Eh9IVlVz$Zfp+ryFJIIxk{C(OjXTv4vNrOM zOdZb82v;6go1?BhTx}?vJk}?&)UHfrNkBJC;+^Td6n=7SW};A@DM130%Tk##nh;E+ zJ|9fjQJc)xU7X^h-#)-3?WA%sqiloK{sO>?*41smQ_r4)y!hqt>#eFPqy&s)lFG=5 zQz`+5OWoXrm!FTZ|8{bS0HGGAZXxR7w#)T1^F#m9$>b$B7i;h@RbN`0JhywLWW*fr zE6<;72?xGremaGkP{o6qg_0j`>nC)sL zwuju-AGHv41*zL+TxC`C*@vMl^~$t=WwBd`YV!aMvdA)*l=9-R%X|8#{HJEeq$XTczmF-Bb;Vm74&qkODfy01y66{}Y-OLbBCao7#fwYTD*m>~^J=BC&V zZ4Vg^nLe1k{{;nL`~K5YkZY-85oJpJ>^JVn@A%x^5u0jEd(aZQ&KGy^c*+e50c71zX@^jgU} z7i|i4g!Aqv(4|Y=$mbsAJ!8;qJ_^rDSK0d=PZkXvVV1quB+#FeBDu!% zLAQ6uOdBldU#>ZI?LG7QjD}Z>ij#j@6)E(xiOcE^6b!?jH}sY!tPEyP%t#Wcb{HW~ z(jQ_|f0qV%C_8%2bP})9N#mMYh*9ZZ^KGjltcJxI2!0OtX;9b-iF4a{b^=4A_#X^bhf)aQx+D|eC z`&-&lq$ErORR1uIOQOaYoH~$ey(S=-W`zQDb7y+z8BO^QCBCkR$)2K6V`HyWd!(2` z?v-g@WAba5AX62+6zj2;0Bt#Y)IHERu^%aZKlexNFX#5yzKHA6HFrKi(f#s->uj(2 zA|)4@Doutfw6IX}n$rqWb}&5HV51s>)U;%9B#}9SY#Ng97-$@y4Kr_&nTP-416Qo( z<>Q#+d+)Cp+tEJdGs)>%cP-zAe7pnBP=jN22+TM0yQE zW~^3x`nQ1*)b`eN$0IN`?G#zRHRAEJ8)u;YbkPs*t-KQ}ZikVRz4#IPMj4xGS3++E zGi1wZrO*-Rd%M)iNu#i}B2qXq;`b2nluQr17>BqkNG6Bf-j^so7SkTLq;1?f?TgYB~8#`e&M(c>^LRg zF{Nie62gXMBcvq^we)F0KV9j|C*Cq#a_lf{Ra&F0C>$<1skj^dxs6#?)_w)H0Bwo&S8+K#x)mejWpK2R^aB*63eeQdo zqH$R8;+i~f*7%SvAw~iR4QX#t@3M&thV-eWI=|Y=lLT$Vm;bh1N*r&lh3#=$2wD=- zok?Rq7n`rK?D`0ap6!izP@(U#FcNggk45bjsGt4RWHhxx-R1l>y)%4H!}F?NM@X8p zLau4=S4790Yo;@l>$Hdp+Wl*Yh>o6uUozEJUF8S_ehI*ho6`z;{gZ2Nx(s4_@|8Cr zmf@w-;)S1I_yu#2;a%Zo`bwvLz*-xIG&z`5@)5p9N?O+{wUIE~wpT3oO$$kT)4F4UK5p`pZs%yVnl ze$Xn)fI3&)!iO>!z9X%ydoAZhPx!2?q+7gCtdn6TGjJZ3oG2M-SgwXYVI%L zVA6Ns;*yQAY@yy0kF7Br!UKYOS{cvo+@QUF>lXi=_9^4-nU1FApB@?_t`=+sv%EIB zF((ugFUKcGM=QZ>ZxJhbGevkoYn}Ca6-lpdSqpxR=ZH?JtepIg5Z&9qKwwa)7~t8W z10gJ*XC1(=iu=`q2gZ8Ckf^?K0U~u?K?IQ-Tex0H`gb!(^Ec1*J@GPl;q*OXq+8+f zN@#-K0v+K&ShwCUFfSB3Xzd2j_rgO}J(IcJQE;91i50+8eA}C4I4WpA$`E$b2Jn*( z?$l~=T_(YCup>Ryl z&yt5>z0yU2ecjj?SCD)`=fXyZ8H-%o^ja*UgQr+*p}`2ciEhCI?xBN~4j!{#9GWz9 zZi=7G`x`7Fm$g&- z&?FW1g63i@VwzYYe-&sumTJJ=%8*k9pBlnXFY1LV5=U9MhXAO)_w$S2T8@$%+_jR5 zyXOR@Lw=T2b;_r()sUi8@44E<>wpUs@!Xz_wKp9#!1H{4yB2Zlj49ue*GjM0tDXq`J?A7~(;U2OXj&jm7mm^6?z&9dDl zM8ef`1q}_WuI8cZ6J!_Q1%`<)W?|sgz}xlsemZaN``%nh z!$@`B1QQ?r2KF1_s;-StP`I;MgIOmP{|tHT0Xv!swlHeV#<=6HX)RyK2ZV$280yFe zF0*AqUcTg`+~c)OMR14Jmq_SmE+En{L@?4uV%e&eK(k4Yu8r` z-b$QiUw<~fTeM(B2|gn1d$4m@hCI!AE>rp5FqV3z@i)9Ytq9;+~?^JTqW z+G$epL6h>|UJ1>tUIfh2gXQad2f9+l96Z5Tg2l%rNFQ4C(bou)0Fq+?e4HuJdEWBn z#zI|RDi?g|E9R{ZuZ5Jo-5tASb3GZ4?#qR<$md%CvIFVja^aWMYpxRS-;K_=xd3a6 z$MRAht+6B%;daW{d0&o%jnQinUId@-akL@^QVhr2?Vr}=Kf*KpT)~J^A=1GHrU%p@ z03L0(f3_-~OHykCBG=t4`AhP29`w#st6VFe>u7BdR9x$FXTlFIg%2I|ZH&&Dt6#)@ z23W-F*xWXEW$IZM-pvvHwY8YzCA!bi?={4&4&O%(lqTO9ol+oq@II zB7#(X^dWevQRk`n$6)m=8H?f)sMqlZhYtO|wc`(!J24{}Ze5aqTP0wULuAG zN(Jz&FwM{BE?pv!0ZNLBIOh4p{UG|gq0OxyPj#swwqo=ks|X_7X<4r_mV7$epK9+Q z@0JVGK2Zuy1&jX7T%b71Ry)WIaORF|1(!qS?OIL?^XbUpn11*Q=%-9tk)5!IF~Lrq zq~6%AzHUqg-zY?fNAtLnYNOd-Z8?`e=QH@$=t;Z&^x`%?8X=y*CDMRdthpaEbGvkY zj9h*$A1lBG2{Vo{snBisFtC2t>7|M5)o7%wDdh%+t;R!f;tRC z#%)&6*|n;U6AcTe78Z2k>fJJJ0CkIspI8kOC#}i}9HM0|t0I6zh z^|D#o{etH<53Li`a9M_0|uw95Vj3_q*mh z{$JTevQ9DHLi++u_GQgxpO)jp53Y{OKaQyPBE6`ZT2bQo1;yc=fQleuA0cW(cZ3Rw z$&q*YOx~mskzg)o^llfr)Q0z@P4^b$^Kk8xJ??HFjJZ8cI^&OhQPRXm+jHbXhhJ=! z!f0$q-GfYCmq(`5q1j@(^~H3OR5OnV_M_!Pa9g~Pi8q3`Y*Z45D)(3$4{nte-;ls} z!gcm2`M_18k7}skXy>7pKH+4nyyEZkyrbB&CP{QB-#xwpBn-pDunY2#t+3>dYrm{4 z6J>lV#O{?BT&quDk*8z2CONI$8MC|O+<)!bL1#7Uq?66=WHf8WnkqAhZ zy&0et0t}-IYqYTy6Q7&G;8XmPgJi_fuENBeQ>(=!l%V0I+;5kvEdNsxCV6u4L4Z zI~~x1vV)F0TeP`Q$h8($YVr%?UnKylpz4f<6IU0cD?h8;wRTm#IwEE~)$B)3(?Ud> zJ?SXIAFzJr+#wb*N4#|+o}0Ag{zzPGspcpkW!O;4&%(y%FKcr7kFCr%RxIRxIue(h*w_B$7v{>H80)+WpsXK)HA02tJL)D*VkZ|RUT7s7L$xVqLkFZn2WfnAyx^2Et-!V+B7D^QAdb|sFN+0pqB{ODR%Wpw)Wu3Cp{(cD%M~S?1r6Oc z(b0_gH=44&!i=cp}Bvl|i77BMVi-+o8;uz!^MnZ6cudlTV<`kK5spwC>iq7Ke9# znefnrVi=<0tbf?4ochp-{0$-{ZPQq8_koIP+oS@ibcZgD zFDZSjSB}Q8a2Ma9Bd7F-Bt|CRg7y18Sz5_d;$=*zF*c${^=HzJE-b35L^xP+p__74 z1?7qPkYVl>o=MwV=dWW@TD};(fR9}(h@D3a_N*EU+9G|o(OZA<-h|91*DTe}e0pug zKV(=78$(@^?NfNmUe_B?fLUDTmUnd9l%+{0j;JS7CIQ!N@MxkQ@5kdrN&pvaw8mG! ztXiSD(sPWK8@%XPKtbcQdg&b#ba52L(Q80m%HkN=t5fvdziD}PPxl`*jGj>p-kyJB zb?L1dnB&kbRWxQY4WCT(`2lqYAo$+T585^sl7y^~aT<5Gle6?7*FVI!vqK}I9TKYe z-bGx;avor%V)NZJ$kr0PAz89KC*0izbIv{02&Sw$4 zYGu%6x5?p}>u(3+O!8u-(Oe(7JQatcGH7#D2Rsc@5}VoGlZ4EM3m?;v`C6+zmW?P? z&l+@_EvE}I#4jyR2wBr;F&$xrQm&F_{Ds*|{?Z&2=mT zw@xgUEBJ8_%B_bVbW$Rw!nBD_vL;h)UUFZz8!8AHMg)H|K0<^m&`#jdm>?wMZspvO1 zXq($>EGWe?Ibs$1Da3ix`Vfo4%q($6tE#(wGq1lEX$}iXsq*K?BRBruc##%AMB0z#dRna(2QVS`pj#bZ)jkyC{f>Va zVjulCPgwB2zz0y}#ab6f9U|w7W*BZ!bSkObr#8w>BHFS5{qV89S7}xV(inl z{mbo2|EOL-<;{@pXdv-df*Fr(%VWL;*ya!FrMN^uf2&gWTbzg5J!RuP)$l5mR#*?1 zx%^*Qqx9bQ|F04>&&lsmScZt?s=miK4JB$H*D!K@Xk*w>3&PhgyIFuY!AO#Hgo=Pp zLl}m9J>mPXBXK>-3S$?@K|H_4@$EHotV%g|q7j(A+DYiBxQRJU)b#o=|2y84{+M`5 zZZZXt6&3igmVNrwO<|jd2jg=$bhBVQuyP=#2v$eEXDGTI?IQrDwj)Xf=X(auPWgk} zl{Qt9o;{NMVfH8Ryq(H(cTVH7)WpToNgG45UNsbL-WvAK=ARH6e^S6jYL8RtzI`(k zhrAeU;;Kt6Q+V3OQboF5L^zLrej{PQYc9d(hk&Fx8aTe(A=!-PVkqcxS8hfuGt!asGL*eCJbXgI zbn7$7+|cU_lo`&*TIlK<7uvR_)b#O1o1@0Jg5GZ}cfys<(M?|WpiChDa`o7O0GcUI zcwvL*qL2>WPnW*Go0>VOTGflM5zsuP-Cj#;V$dK}0QS$nlrgiF;1QtM8|NHGYz^~y zp{~p|bEO<}qiNfkGY8f$$j*U9WDdR3bR7k@ks$-8;k}lwtk^!K{-0J$;WA8Rde?4m ze;vvUa)^1If0^ppzJ_S4eZbs~N!(_$^gG~zBB>Y%cAL6;z0qpWx<7yrH(G&68t(NW z6l*LYBICDHhBc-?5g@SDpYA?ak0GMs(_592@^`m9D73X2IX%+t(j;By&WyUe)GN~^ z$O2GHDyb@vhyM*t`jhW>X~!Sh=nbtWQpKN|NIs&1_ag$TkBx`;e~x^f4Q5A!->c>) z(hXJ>7sq8M(u4R1XRxc^qdlE-PxvxrtT)Crb`1wrR@yU4eFxMX^YPJMKkL)3R%bD8 z35=`Vy1TDCU>!v2`NpsslleTc#&Own$8fHV5umCt;ZrYNc>o8!BYAmB}D=jN3jTy3sOhm3< z8de)~9!xK5Tn*kznT2La5Nc{Tw*+sdwzPX!e!O>w?}`Pj0VGjCQ{&GJo!a$;#&lra zulbe=O||CSiI!0FiWMZL+CM3a@(}>gUi4rZr+6W%`auT}Tk7w&GUmWneR*Rf zDc7pN635rxzOhl)2iVtAmuu@Yeqq!SFvi}GOvg(_o(~h4XGr*MC>dS;%BR;Raq0^} z!WIw(kWan`E3AN8p0ZH4w$Mrtpp0yUFt^6W`ga@(VqU=~JDgXS?={4?llkB%C1lGZ zo8XNXC zT|huH#}`L7ma!yknrowSu#a(WJBe- z75p^%=AzuH)B0p_exjl49f!#SG5@Rv)`SAFmokI?(IxJ4(Ba!^{tiA1skB!ae2bPO z+cKt_F$q@|Qp=Fdp%lI8-<(A?C!=ZA%+i5A$&8c_d+MI9vn7!D+S~`U0DsQ^$&>xG zSLf0i1lKJ4mH(?&6$+%UMC-=i&u$rjQHwWcXHr{?PNcr8JpnzSH!99nu=VEccq(Kg z!#twi{>1gURd|Ix{(U7TE>Z|(x27mkNutY&6v1KxFJN*sPtw~F+M$}S9|(ogOW
vO+0_(;yd5+E@6W@>81u3Zk0<4S$UuX^FVawCo#CaXr~gs}k1EY*Vz9l9MuE@|AiHXx&K9KS39dDGHeH z9!mKsFY}xTI8sF!^qR#aCd)-(rGFi_D6xObE;n2IG4vbR1kfjSKDjH5;Y4F&;t!np z0My#SI~sj;sA`o1IQ6z9mnWOK9vt@`E7*=7c#_|_C5NMEjvj%!P!79MXMfHD!^OxS zIR9dwkLHK(-bfF_b}Wv9#5XJffvl*E#j+<7>EAE0t*hi)+qfd;4M18{Iu zct&h!LEeUrgJ4?tDB!sEY(A^BQw4_2_->)G!3NJ;+}#q_v~Vy9_emF=U3%&0&ryCn*Y` zC(h9*3#b$4nTOabosxc7L;9Uc7xCch+{Bms`DeuWZ=z4cy%o$a5eOt*|M#`~tL=e~ zR`Y;#py%B=uO#?_d(0@udIm-oG}8t#ofqFsdiK{8yJ4wzYM>0bmb)y~yf7 zQI^M-*k&!ktAe^`V#j6bdiN*)-ye?v0-dQqVHh3y9Pimtw6Ht0JG%d$_x`^x(*L?* zhj+f)Mh}38RdUxEx9=1ANBETdR^8!f5C30z65&Bl(ELDg8`&R`tRr0y`mSFWa$3|T2cxO%==Y8OLGlw9W_%Qbw!?OW{Qjm@6>zc9e zH|*{t{@xz*=8cz!3y*Uw*T^YUJV8qN_nyRpsMVe3&tu^SBGug@9{C#P39mmI*@S#x zob$7>psl1{?_;nx3f=?hGQ&KX=&1HGt5KjvyxIqYt2hD zt5>CdpeP)G`26iC*iiUU?czf7m0wrqRA8k$A4M$#<05L$Yz<6vj2)&W$PK{NXHX3y5_N_eVJ94i(!|yNH z>hZajiNSIK*lNsVM94-}uQ8XW*9?FX74>Z{$AOO=1P0$5T{}+HTvPNBLvB7Ee94F9 z1#%_`O*9txY=Z!-$4kbE`0zpiZ07~NNYhve35>$OYv4K-0J6ccX(CmS3xL0`&Tj!T zN0G-~PtpN36sh`r&m3Zu?uRRo4vieFe+kW)B99vneFpzktsId6`J5I)S8rUy5>=3i zcCZ`S0)M0QfBefa+yoTfTgLY-N!Z?tCFauNZ1&vG|t&rf{E#`=< zAdM5V55rl;eJ^&4VdL zeYRp=hpZy*Hjgt}itY3>-(`^I&lA!&@rREnZu8yQ&&YG;13>Mt-!i`yEAB?4rRCU+ z6Eg2)nbtwd$+wD_!gy_lJ^fd`isYD7-!Bz!(U5+{b}&{%47-@RC+ z=B-Wlkp&KK=iIjWEHw5k#ob591<2YP2nV^kicQ6aQ<`{rL6mWHutXm$EgZT}eo)?vipo)G)Um>*h7g|+>%o#v_z^%x`EV(0SepoW#B(4rmd zXpzH22eLQ8$(W)6#agZQFg#*jK&)wj;s{^IdYoC()PKViJ3OT*eLC}U)3Ag5fWbAS zIUJ-$Bi-|uEzEbF-`z&rcPA%G>jy;~StjvOrr9hW3E}v6n0mG5f%!=@t8*uV=Q%uX zw1gRnXNe^ZrPT-4AIr?7K8-#7u#B@&f6n7N^!|nE30@zn?-b|Injy3q9i+lsjlI69 zjJJsG0>ttQ$xKD(CS6IS$59dbGwvzQ9sWwEJaL9dyHY8p4?lXH|3nHn`+sWCHXHNR z-Ry*fS*wZj`#4e+_SUnWW?4-gquFu{MpVt+A065!*EA0#y!2>5M>qZeaC3IYyQ+b9 zG^yH-?TBGd?LY&N>OMR}I+f1Yy;Q)(VT(~?6?jcM>8;Oac%)oBk3zWgiCr|qtg|cu z)~oFt4p%EWJ1t?uYSeo>GOsu#H72Fz`E==fHKz2Y24Y7+sXs9Y@Yivuy@gHG}g~9BKXikDb2R1Z|c*M4mmcrLEv?*c?j=|FbbdQ)dP^ z_7qpH#sg}utWEFwkK^MOr*FfUa-J73hs83_CWuMOZQcod#F4QJxclGF8~6;(L(1DZ zQmhXG;_5t(!+827Kd1wGQ5-HUr{?M669c4T&hUJ)6QIFzQq6zmF|{Bj;-WaattU+g z%2q>x{~oN`l!6%Ae?)M&%&Gn0;|_#mE$2p~F^AN)O@~^~;uPUX&bBn#2JRv^99M_5 z4K5)6|F2tj^405KFHUK9<*^Qzwe2Xvk86IlH5uEVjNh%4p_PnJSNJr#zG0@dXb7l8 zE+B4|=Z9JKA*9%%1J-XeK4R|*qzqKHcIf^{Y4vD+sJ+#$8FgP=TrS6YEgKDo@B)GN zZp*=K&i+3KJ>DIp9TJ?A>PxO#6i6-2uR7J;C&wu0j#?+D;JYSg?D^ZDLJAcslHVJ zP#1W?PeS6P=KhJN)+N?H54T`WU#SAWxd0A}g0~_YQ|#h_imNT?HD;G#hXcl zM3@?oREKdhob`VDuAbJ!3goEce>sVpKQlikBby3&})Xlnu@Q3+Y+nE7 z2jsH8on-}x1}bQvJuTc*6Pkd+OZxG!v6~m3a_U)ayQwc+X8cQSny#3n}%E}?(uU|%^J8scMf~U%8b9=wbc}nv`X2{X~IbCyrJ)E z+y%-tFf(ids5y$S&la#7D6!@!$>SoM-86o8h4RpU^(`ea!1IzHAiFV-#?w+YJXTZa zQr`yz^{e150*=bY)9eeBg>}m4WmJCH7l=@?E7;VrZVH<3$xbW7(1ezTT9_Qu`bixK_d5wwtsv)w7~fF{{b44vr|26XB}s z_>#h?JeSXk@W{I}F?!(+usOQ{2oCcqqfc3ny%^@{bmeE#S5&sNu7#f_`ds?(WpQvQ zvGP-`Jg^jhDFj-#AIe$VaWu?}lr(?ao!v(z^gv`+)N{&vD=~!HkM+c(f8C=i%9g3`d%qT&mSYaO?WL42kt1j6w}J4ndvtL05vG!{{!x+rPx>{%>!f(x?f zrH}n)zrq6eMwBo>-e2NkJQCJ)P0W`qtrXHo$+ky@5vaCe8ebBTb54R|jHJ(g3&a?8BYFN-JiHU!k{n z_c)EGU|$P9`i$;wH;$-$B7xqkN2y%BAN@zm;vWXBns|dV#0Q_+itw|OCZ;5REAlJB zN7K;FBBpZqgLAqukFrqMoeKFM>Fp znea+Xx5P2>?#5P$LsPF`&xK5Fb201Cu58y3zw;$c+|#VF_+3!cNUFH!F%i$qd9Oms z^&g5I>EfA3S2i9L+K>076gT{K%ikMUPchjWc#S4>ca#2&bh9NrZ@nP#{igg-aKV1< zHr52AK-6fwu~4!3mR9p}rLIP@Way#XL&ErmN8_im@-p;=2NFdiDp!AqQI^l>D;n{! zpDhd|XY{`;{!ylII`!Zfo_sf|4rErPtX4`jAv^Pi4zt-&Y@&dzM^ohCyAe&;LkmW^ z)=BXjh2YLvWEtxZ|CkD{XmrQNu|h7PcI*dJ$DA6`jw8}f#F_H@wd-pQ?%>>0}V+F$-n z`#LIlu!u4@Ln%XJ|TE82<7N8V!~O(2nWdNF8#7@ z$(EUjYpZ!Cu*5u&(3}!`yw8cb?^S)(`2^O^EgR2!SUabuzkep&T-Bt{;4gUW0`T$H zpga25h5dKKu2122IL9!$`{N|;h&rFM2jn8g{b|;%-VrYcE67@JP*XPnoxWIenwuvA z7`xu7t_geQiuhl|B8RU;?!OJvikOo<42(BDws*DTHS=gV+fDb1Pa3a?+j5%!A1D1r zNj}U{_&q~)f%x=#0U*xqDagh!(?biqt+W$niKZq(AFvB>kmvNH@k@!UZGEef6IyiC zDU$vd^bIq@S7TrfR2}9CZz?KN@C9)*_V625C_d{2N59|ROyXls2-5>wmrSI1_j(yf zO%%*(ZY0c6Wmg2&7oWo}4XlD9rXMQ8wVQU2yKW#$*N+#ZgC@1CKE z?ot4m+ZIl>m0u9^KPgE6{h%W6nK?K5($;@>gJ!zvjlUEYv^o|ML8E_Ij5oBDo|TMo z4~_H3W|Y9M5!2wMmYe)&EzJ7uYI3a%DMcfkYfE~<6~r$EZe!9UKTa#2K~v6n&|}}R z`X9FKH;&(WzFrM`(sog057>`%E?Kp6B?H-T&)(w^TeH$disS-OL!;bi$9d}P@l}wL zcisuSaQ-U|4{u|RZfV(kDEsG%gg|<86f5!I$J^V~cQH)UI$9oj(_6}BErRZ%{PLe} zyH1bgOCPGC9O>?k+F;&N97>ah-bYR&N1{!gdjYkeupO~Lok?6+oCx3$*C`8^b6 z{#-deSbFX1&NIxE;@LcgZ%|B!>{1$Ly}pa|b@J)X(7fI*5}9X9&{IG*c^%Bux(1ck z->f=R1;X#VJh>B1cJt0*sNBzDnPQk#nWJI%gEZ`S1?D{}e>YG>acNPk_=+d>3^XBj z@(U_xkbnVZbqR~(E)d;|TYcRg#Z?exgeK3f9qssYsn`(1p5@m<759#^4YqFF*t%iQ z(CyBie46G)f6goVHKka8nY~g(+3X_i`82DU%#7SgnZt+?N`u5MLxp4+^>K0~-zp>Ofp*1+&%3s)Ex zzcaa|7IN4PgUL6F*zKoC&u7EX{|*r1Gb-44=c9yw>Tr#elksppP?zu+x||!(P9pa5 zEMX_x)A6ZQ#_s8P z*EDuTfLjXlPF{`z%B}{(ABj?yF$aV;Qek^z4#lKcV|KIM%UrVZgbB-BgzKqt3)3`# zHIhPeN}IF@ZtB>Nj!*zDfcS0(I;6=TV~49HI_@6+1xnt~DR#Lut!Z6q+0%UY*jA!kSf=r0fCI!9Z>>|Xz!B}>erN~Jacxf}T%r1PVVtCf#S zn8_J^Gl5qB0#Q$SvbsVk@9{}{L#5rB)2B!%<}~)0E3fq`>;>ZvQ(8lAPp3rL3av|C z8q}?;MFE?2|9q;s*Ucz3_EhO`a>l@HuvOd%@?E(6J$|x)$$xBabdb^Um62Y& zA%2y z!;d{HV9|yu&t2|u3T;tjetIjefEulv>K>RJihIdjJIiny`BFFUM!LU0N)0{ucP6E3 zK*>LJ&`vm*RsVbE7F19%Q#p~ff=o-^WTV@BhdM;vDV@dE$+N;{;^R`dYlEI#>UTN9 z18DXyF#hSois6*|p~L9L(12e9i~G0E|E@Siq9gXVV@0L2^$GtIr;K?F77#YN$AQbA znP3AbNLrq~q|czqF7ffy?|&=+@9AMlZECE_u7!4dYq63nKOlgjC)1qBlNN@xHQ^Lr z6c6nn;$AMcf1-4mqXH+{g)=+uX7S*e~!QpQcuD;?pc3Xfy462aEdZ2j`!x z^{p8RpW3hp%6jxhI%m=%i0rLw#roHg`>GueRIOH{@F9jix>B_#*yFC4qC+|*YaL{P zZ!SPW?h5@@RD0!I;6TU#qiLnJA%!;_Mz(BXRspN{8a?)xgpV9fAofdTN~xevWFOin z&(%}69|ys+1#GdIW&aMwJgF*TGAqf}%dee;IGN^gjgaH8GJ?h#>5Pj*61m*9@m*R| zw!ex{fJcpOYx=GqJ9IJ?gp?!zu zvE<~8EjrSNcNZl>`3E%=lBbRaJDG%zuQ}cMUC2(e7|~%KPofPZF{j53 z<0*aeO_6MyfI)>q8{99P4<$*S|fXo<2I55GyYgG;z6`y-$tx%+-xP6!$2=!X=sGp;h{2O1iE} zwa)GVFB+|)NuL?4jO6b2?F3vG0fO}x8{3oFD)L?JkG$5C2G~_>U9DS-J+aYYL5VD1 z29#NipKQnA7!QwscH0b}zboPVM zqvMuEzFhjlztb}H3}b70Zummrz;Zd2|L%fpSe)G3cd6dx6AnYgPJ1f3STDx8TO0>Z z1v}z~H>w_VEa{Z1CJuWCWFi@Nq6 zji=wyUwGG;fgVWYGxW}8`mNtt=`AcwN!(({rx=PI;z=IoY4(a){Wg7Puc@MH*l!4A zkpxEP*f%D5@&f)m3}wx1JY1VbY_ch7Zw5>D&lyeGu@oI!3BR56Q$Ne!-c7!_DBQc) zLs>(ggyo7yMd7JUcZWr%^XtW)9IgF1>9ihXQGSa)lj21J+2mt_aI^zEp6R@EZsQw! zAzHq`f@ze%0+}+BXxMoh>(YxkMyI$foi#H#GDC{(!h=+rAW%Qa&TF2N9QLVtV%~4w zq6ln0+hC|gUF@yh%*%2J$0)mP&AsEwI-0C@VGF44!ppO9I+BKyuOGfC|+$3nm|{S3hR1ZiXYTR;a>iMsK+0(E#D6WFJv2ZitV-ikQ?{oc(iQak(1*RIzaYd0l9ylN7Z@49M1S2QbO4|}ic z{YD}HFNA=n9orpQ{P_s-XHw(_cmYuWkrA&d4`zx6_vUIRe7;O&03~%az@5OJ1Uklsy(fT^sqVn@7#oS3oE!<(TD##4;=dOmK=mvM z*UVHTG4YKjomK>|3jwPM2o^K|s;ux&ZW-e&U4I^U_X;5P-z_^#SNxff9J-`ZY5-2Y zvQqFD?52Oa1O%TI`CNFW&Vq8avV%cV7rS7dK4qQ~O8yM;X8ruflk8bLy zUomwW-U8u_+iy^9Oc9BRuPY}sIjD&U570RPDFJv9arnP`mBFt3-DoO{B-Oa)#(q$S z$kAy72xbwGtV`QHCkn*ABONUR=TY!t^%Iyw{|F8LDu%8f7%han(Vc&1>Nh(|Pk>(% zJfKW1ROPhHf&u`U0uGPCYi8jQ=cHQQ{(klIVnAsHAn;B z*L|DOyXKK&%UCvARa_eo_;eiHOJ~3dBv5g(^V`!q|CAB%))fKdcX<}jzRPvU5xPA2 zKqoiC4E}EciG%E#K}>{Gtu~LP4yE~pEB{>ebz}-GYu%cy!&sx+1TH*>5iRmXnhX;VlYcugtK2_&8`6_{Wx)G<4IuIgBECD9 zAIhzZfsgA$gw(61TG;IXj%dist9T-drtX|HBdBPU81M)L2otL5Kf5DKl)%a7$nHIl ziT}`{in(`iFNs!>*WYQVRignBpZfyhx6@!dKZt$fglC?+&I?-7(7<7ev%jNmO|zG7 z&S5}HX9Cpu?KUAEUcmz^Wfbt+1q5);0{Q7e zuq${m<*r%5u>Z>jCtYs=5&HuO#Tkn?%<+`rel*U;0yJGQs{rRTN}XFF^0t3nY(x)^ zzuI_+**OC6`4`;0!aVOJfS7+MbA|T5?V`wI=*$BxcM8jXuOWyn6#+9y3PJ3OR^HcX zTZbyT&|ZBI>x!I2gs87V!P{?$xzQV{@5*B5PH57;h1nWE2RBTeExq*%b z`ko!UgVXTX%!YTRw%le9+Z+jd;hdhOSQoiMLVqPes_%x!Tvh#6^WY?&-X< zQc18kFvQEP^hg6H1m3IB#eV&i*0KWvquD9+_36gEyxB+$>&n8~Sv&Al9RTN2IdD&f z`>&Ez&+XgH#Cb7Ig>~k{R(EJfAACmky3Is}RqF0NuaX2s%Ca0|E8NP(t?ORmY0c^9 zJcf5C+s}LBzuJLC`cFByW#OC&lcS<}8lha>t$KSrrbP_T{mnATZ-5FMW>hMGv zgv9FN-D6Tt>PVm;tMR(*2KO&8TSzSV@eaj<-F9mo3Zj43LL#=HYh-uaM zmoldgdQH9}T-psxy^0ztNLqUAi+`mDht=VZI#fWmbWEHo*G@4&RS##Gd_H)w(j<|% zVii2=PP!B_{Heoin!cDodNUPwT>@*5S%~d{>QDgphk$=(y>F1ue11q_3W^!WSI%iV z`G$tf8YS6PCeZvJGp%=Fr3KOw(VsL#;DbfWEwy9D`#r_ViHM87gxCb8^VU+b(Ifvd z=8u=`WHnYxG523NUbqg!x|*hs{%C*4_Nk>XxH$VIB?QN-lic#UE_jig*UlQ&W;Y1I zS#w7d&=9WmHc7PWg0^C^RKS}ca4(eQjzp5i#!79!bgUYA#jKSXu`8}ikjpfh%&+sW zyuTvEMOE#gA8);4mIQe|v-Lzk_fr#@`!rX z5Y{(L4N|7*${5Sfmr%e#7Cl-0j&K{JH>P2`*{3|6_>^=&jo#u;m$rjVaF!(pMUaG% zU&XU2CfWO9_~lnVUUG*G`fvUiaQgfyaCKQUz`&p&L?dLNAEnnOziwp2PuAsHDnvV;GF0IT|UZ(nprE-y+f1EZ7*$%C`|5g?CDV3G__$y7I8IY%{_JMnT5!b z{JV;{&vNlnMN-#~kDO2pxeFpIW(C>nQ3&3jBAiAujNtJ)QxH(+vGKv1zusHug@C~O z?hI}eq3(eYkn7w^-ocyS)YHHI#oe>0A#`udU|OW4jkmZ**7by9c$cK+jVh8r_NhZ;IoM5qYJU)z_< zEdne*K4b1YtHP~tYbNwT2u*JPlQqB+Z6H3}JKs)qy{#fIee~9&Qmy zdPKKiOo!aD@)^`&pxTp}p%f`}fagS+x~J@gyt?ZJ{Su!q1do0@dY9~XGqxm`T zgD?R#RYyq&dF@k{O_B#fb?b~B&La3_o)NP{@UL90xu?48x3hTtYWN*B*(N-)MtP>L^Pl`UJ+901{QDe)Buu^V zep7nPwt=r!Y%kxPdlyl$YYq2R95t1k_)vkzFOxeS&j(>h%`J>>32OZq!BL6dGV@?e z>13cR=*mg6Y=M zLZ*r@&zj2*if{X1!7K3G4jh-@d!N8rTqiL6ePe_mZwN^}smB?G`ENb~z#_7C&-074!ozqA#uLYhxK5O61eLfj zMxYlFLL!|)Z;|JO&_K20`+FrJUEc0rd;Pu=xAEpXV;%H{ZP9GoYAvZ1$sbyqIqHPV z?Rg=2P7>tKBB~3*Rz$UiE(PftmurZC6FLzx;Np2c(uqHz&$e-+E_ora7Pq;1w3~14 zoAhv(&b5I%6-C)~{RrDoYCA+PJ`G!BdLXf0=VlaL* z`hNz-dXXKNRjS3djX?XsrmQ`@DDebKE?)AT*al$YfVL6eJk`tAkheZ6^ic5SlMgx6 z$N%O79a{g^K(vya+WCLVQ$EafWy^gzLRtII=c+R!%DF3*}RY&%$ zUGnTn`F04m{)O$?-tN5CC-4chVtqS!bzaVWOLY~=YFdC7jH+panCoW-w}0Ul zO4ZFMYNC0jBDtO^AG2p)o6@e=?Ri$om#;x`Agb(|=#t2+_^7oU_kDaZhf+3{7t-hM zKwMjkD&}5tItVYlQ~<{ec+a2Jstf0KeA3+0Mk-6nD*VE-mLM;?P`tt-^56EX7BkJO z{TS!w8(@vkVAjN#(I}uFO|p*&Mfsw*h$!cq&!XL|g?;Mb;=5ax_Ul9M6S+QE6siO} za7|OPd!%t~T^0sg^qNO4rsbrHx#xUQQ5M20dV(UbbQJw~+FdvngS<(a-Hf*gePJ@s znp-QBsG=jUBNyLQf01-uBli76O)s#4?l^e5ic+h#QV3iEF_T zd0+GD__h=t2x;E{$g5?ZEVQtjxd1giK0IkGeo6{BrtXgr7MSF{ywS3*>B0adD%+J6ci&ez|#Siy}>>iqDU_ zLiz)3?u*Y=2gNjIDK?05pN^|gp13jJaZ8=S`M9~!>+&{o85H*R|B@OmJ~=4@dzt4h zySqW6_e1M~AS3>!q*+FUIswT=6_rLg{h?>Pn@2FS6)_6)tz^>R0~GH5BI6k^%JRZF z3GB4Amsws7S4c5;P*y$RX1iuq4qWL#M8GM*c?z3bfCZw?MhMn%L8_+*+I2^YsNW?h zQG;ZH1Tkppu74$)CE5E+kc*h`%XjB?UMhL1nEQR@RM)GNIPTg`iO!6JO|7lm9hCXX z`HE;+9;(U9YscSnL&8!Ew~0BSJt)_Fb;73p|9CI&TUaNrwcx)7q08Zo!Mf~dfU_A5 zdj7zekSd*oJ=R0?+>PfSRIWTBDY!a^in;qI%y5y}n}+ z#|=%nL7O!p-m6VXitD!i3j+|Wmbv-K<=>g=?7c7+><-uA`RjalbBs{uB$ms-9~%90 zA!wW@x%4|H>*w4X^X@CZ9zU9KdkE+<fNK{FN3Hd%31>J??Bwo-xQ)>!5-+SMzQA&y{ATt2bNbFZ&D!ZXl;m))BM78 z3KesJ?iH3iPFi0LPF~&YnGc=PE$sx^wO;|@xgVs?hrP7m71r0`m%s{OISkxdvvRC? z$!+O+4g=mzq0WmEI?ma_<;X_8#?A{)X8-k9EL+zt%?H^H-2nUk6<6pzW$1g+FBFoW z6?CV1c~h~(YqHL1@Gz3bMf=YxyM8f;C8|&F$=ARkVEf1!ixK=lQQ`96o>#X+f7ABx zQWZ{ff#;svH(OlLB3(fagKY6d$H3{o+>{1xG*Q+a1)opdT~miCQQyNKLyKDb=WOV$J)%*zO!3NIWPFzFT_{N~{Lm zwEDO1+EH+@sn~Snfx}!vP_u^rDYmjM3Z8u<1kw@(_0*LbUG1pK5nf@Woyq0vugczE z_hV)QpPB<}UvryZSSj5kJ4VXz-p>%2+xIWzQck7J?dbCKN#<3vaYc|X0})P?t6S#P z-97d~y3@5)_Pgr6GQvlX0~KW@3E;x$rYIpfKglOmdR9WuKk zK_?2L>SSH68V%CO7UM;S;k0nJb(AQh)7i$PIvN6gI&240&$|yvVL;@yKwuPal+UreqfCh0W_<3g7Y)Q~DHEid!<%7XAnHk$w*y5TZ z*aZWs$RteYHa3@;8-MiRbox1Y2a=4jz{RbdLB6MCei9_ znZebrG2!P+w4mT^uF=W^U#V#wI=_hYroVzhkA#xcPDE|jy;lR>~ku$ zP80#o09E@#?+wm(4|?Oua0n@97GZ{IThEpqhSt3$*@amodlr}fq`Iy zcxLKTB3w{a>Sp_l?YdaG%4kl(MW(omhGBPqg zJLAthN}}Tb>JEJ3r}*OFV8a6jJ3BiwIkPcY+nIt{xVgE(%&cHmRz{!&qrIz@gMka9 zl|AL(LH<3CsENIiotce;nY9(!%eV%H){YMR6cjHL{patmoDOE6|2vbF{lAt4ED-$i z4VZ zQJVnr8{JhzC@3K)DN$h+7wE$bQ2MK`+2`V@Sskl-zTj(U{P(9mEAR(DR_e3FM@2o~ zlSqY27%9F&2blK$c?n2~J$_ zMM-5r;JS&*uBIGwK5WH**EXp(C2 z6H%G?<~<)l+OyWx$ouc_xxb$pw8rpz?U~Q5HWdHT;PG@+?Zz61xgwk`CS|()X{I}L zIPB?tmwL7(2Z@4{B)fsLDxf>e=XJ8bzIBs)CP5uumMg7zPPCBiPE6bEcLc{eEG9oz z^?~X7M(j(ARo}`IZ}JookYxHYi8t=QEBkY!uYO-i`(4F}l`D4iSShtJYQGMnql^rp z!64O0XPRHnUUk`)wxFxjr1WZY{*O~fZoOJ9vHz*uekm`(<=rJ$w77c^LPK8Rvtt4W z5i2Mzg9Ac5g#XR1-~;HFu-~JDYQ!DEE+j@s4XzuNCFo1T70yDrNOT*?O`qcfL222t zY^sG|itLXDGop~YwGr&;h%R*Xce)b@cQc{*d;HFPBqf%=N-yAl?mg2t=m_Rskcl{3 z!8OL&))9K4-865S@9m=ZHwaE&=WWJcI=$Z?=@r>cq`q_*Msr~NjH+Ck>cewLkH0U! zT(4jm|B!0$W}kU@PzydI^eKI?O)Xv87p&aYoZT)%6Zz>;QM}N9Fs}a*o6Hvp2Q%mN z%g|(|ko-Xe_x6gp=O~+d#Y4E;q^-m*Z|N1IpvN_aD(2pPeD69!@zRdQZHh}>Lqok4 zN@@ls*TtB80KxIfbd_96rWd(F*t=>$Xv+hI^Jy3TbQIWgZDVkEeVyD*!g@<4kJXoh z5c^@pKo{{j?xRD9UeMvyw%(8o{`PucY7c(uBh5@mHUrQc`vIYSI89?ab-tSr#W`s4DXvDx9pWjhAKvQwxZKa;ruXru8LJ8n&Oo;S9|bfvoi5CrQR=?Sb$#(Zi;A?maFTTn-sRM(pG?tl z^74t3`%WIO+E1hSgimP9j%+rOxPrRsCV4;iI7~44u7%mlY-57LosHdOCdrF(Tk_p) z-NbjO8(@1pPu#Wa7xnJ9i5zZk@1Yldgid?;jfNne_S&jWD((#zQNPP!1!Xb4Dinfa zHd$AjVVkC@(hDRxUd|4>;h*6}T!U$R^}8!(Ceyj?-ML`0f~}TsCxgrH^#P+G-comL zxIf)>64Z#GKS#nhQje9;l!lyUsrTq^v-e*b1YjI5Tvx8|z}toLc7;BL)1HQfm5)?I z=$~}*a(sE|VFq;%IU(C#)hyH6?%{Wr{NUnWL)1!`u*55Z-LGOK#^Bt_y-lK8vsk@x zy;vlsb14M71-(ypn=$Oz@7ekEiP!c@L-O;lwdXy|XRGwQc7k?!i+aEcDTZ#L+gCfh zl|67eW)_KZdiLK*^PUSxdVZ`Cht`L9vV0{_Pi>+4+?C&QesK3WD98qy;lqOi_hYWg zaz9gZJZ_^BpS0!u(h#vDts0o$$nEidTzwh*gcy0?>A%}R%hli>yd}h=(`?p#S$I5r zvxmYBmUF0byuVLQ#L@fIycOvuzi2|*!#>=FIs9j*D=zHVEBE?VQBb8KLhs^^i6Fx0 zG&w|YJLEdQY~D)aK1-1waW}F^G-`^o&5Yk6xwhJ2h&_RW{m5j{n@R`<1e5xS-L_N^ z52D72_SSuLjN#4Pgym2@rdEn>Hukt-y0Xp?H{3O@TXXd@-H?Z!lB~|JbUbIz`ng4$ zLDD`i&1Oe=9aOeXv~$D)=P62(@FvR~_ecX>PN(2G4bs}YtZ%yeI~CHK8t%y~MsElT zX}i%vbV3mx;G|@mF<8rc8x96@PuOO#Wp6+2EG`^y z?M*Uf9JI>2%1z!I3|!o(TtKAeN2^+@2Bm69FGp-;V0bB*OPg|A_j%UiR(0Rb8)rmc zMRO8PKauJus~g>Y3Ms@^e)qs`O_bMSpjlD%dhd-+2mj`t%vbs+6`E--w+uweA%j@8 zS-$<>@9>+w8jJ!vOFLmdfMuPRGc=3@sh^AjyGy(M>@j9YCyQ%4mv@)CZkP@tjcf!H zSG-EP`FT4~)D^$+jBq19-M2lsc*^wLz&<^$$*w9EeHgEfq8*M9M$<%2c`|yU=S-M)Dj%nS*cHz36aCi8 z^9SXhCNAZt{LR0@_qacOtU8BCI$UCivm2J!HVMI*v-*wfhSR&lv^#h#hTm+`~Z(a10nzI} zCoN2`iw{o8`gbMh3Uf@DxKPKh4Zb{2Dh^*m!Dd4ye8Rbg%B_w$i+ZA#zW=^5IG&lL z(ci~_KK2dFLlz!|6yBrrnOrZ4GdwXcE%=6%>q_5j>n`ah*Ai>b8f~~wTIj{tr|?WO z?Eab`!~KEexNd^K?cIK)Yshu%I-D8hP_!Hxgq9Bz==5RB|HkFCt-jgXL#ybbe9S~Z)yVvr#JNoQRfEzO2!>Calv6 zt5jsPX^@(_=(JUs8+qzV~I!o9WZy4(oa+6-K<0OCg93$BH=z-%YIUPW@Hf;OJ=nDxi0Pafr9b?(Xe_cS zaK-yZ3TB}~HwL(}D^l#xsp5v+@$&T+ORux@X{{_Qn&x_7rP)nh&zNLG@quaEmQN@5 zgjGle{hszZo{I~9=F`f051t=Pvn)=;1g{}%A6qYYxnO=6 zLd8~wA+GE8fkrA-zmoGJT-WocxTF?SW<7)XBYx%s>BoH>DY`{=+Nd&x<@wQp_401S zb+^*kj{<_yi`j`^*&s1_4s;~p6Hk$iQ*4m;o7#SaD?$_RwHlo{o{<_?B}3>r;r;JU zlkL{s`$yf)BhLwM+I8daI-BdpajRgiJq&p&O9RPlUXvtmuR|N?%?NRle_ea(jI>Nk z3g;f>p?dTrbh(#15~*($m7SmjssEtw_>FUqWOFdHu6lieXFv;9C^u0W8P{#A5Q#Yc z^Wgz@W1qq<)Ni=oW{9^N)wV2|Z{#WLX0~786$Xy#%p#Xp!7XyKl;=Fiw?`kHN1Tg+ zJIE(;f;Q9lh~IyrxHbE18P?S$f0HzCWg+nQ>QtZoFD*~`YU0)|chPB=C?agNCnCq! zdo$Fkj#JF>w+Ocp#Cy%A4n;?MrW945(Hw+OBRj^|1U<}{5l)D$^~Q5wE&YLa;MT3i z_xZpF-D7{!p-}DAdch^bmEH=Uc5OmxMa`7>&L)KdaRAY?0^yVgcKjohK#ya1$7uK3 znw?;$8K3QHS58s9Em)B+sQ*6e$@Jo#{4=q~r)!bN^C#S9W$(5p-J9kpzUBLiKKWV- z!MBOq`VvHOVRl;^kN;)rvA4uwxr!D^4O*|k1U+euc@5i z9Na#Pee511uV81-?123P3pvy#`YKkpPu6%6wxn99hBng@u(gY=!XdHOz=PFuenS7}*OLTGc*SyWwp?t`mB3AN5SL60 z>XF{%hzgnPVuX8hf9Q0X)a}p1ECCnHAn;GuMVi%t!L1;X;{Jq??((@VUVdF7b^frc zU8e)ERPL(oj9Hki^(VIwogqSUE)oON>{3h-c9zFZN3c+8<5xUgJ|Z*>?96+7K=EoN=L_qc;Z~%#hV06`3ic)Ual84}{g4|&B<;y8fu`3CxcyS`eO`%` z{)|mpPKc$f)?Kp}Cn9Mj#$Ca_J4SIqrPoww63&?xxn+4nUEEU3)2isZl4LkAB5}Tn z@|oi&R`yt(l750#)BYVOtEMY=PsEY>>BZpFM#Rez(u&1!J?rp`k8zo<2ipt6!D+VV z9w_tq(KlZOVJC!>MUJKAj|uB=p1vJUZPc?hWts}=8-~5*-NT!C1aElZuc}kYdy7Q% z44V_N+xjO_UAJXU#K0JOwX46qsPm3|H0fK<8SQv{i@NHqKm4>gG{hT}t>gQUdlYLn zHnNiU>%>d?QSipmpZE&XttNkJL znYAVOI+~-X*4k46ZB?5kvdRkbgjGv#7vIBIv7UQP0o$$@zFvE(mBe6wAl>|9%Z<0+ ztip>02m0ITa{HO#PJ$fQ;vqHf&&11fXP(mKXaOJkhHEdN-$4E?f5wbwiB^;=qsw$hVxj8NxY9e+ zEkuLixHdWNX-NJm*=c9enW{4Sp9u^m5x`ATS>=4P5@mBiCVKKU;FSy zhmpm~wYX`m@lNdCNNum5kI~LC)yQ@CiA??UKonw#m-U4QN|!gC|1te`^IM^QZoMmm z4NaZ7KWD50?>F>NNZY(>H&>{h6>B;5rQqFlZ++36ZXMeQ0KBU5KnCcNjP0I zFTu+g%`-YSNtsMN4VH_ZJsve?wmXw4RQEgFrE}g1Ay{06wbcXNBzhdKW8WGuHnqnb+dDb~G!)lZxVr}`jwy&k zEZAo%92tKztuC7z#i{p=%htRJU5^8rtb-nC4Ue3A zX!xTL_L`({wl&vdHD#ejT7~9OuuL)C>Jj`do3=Xf?SJeAN<~b&ZL>R0=Mu=Ef^5UA z>D0rR?UllP1upwVMJyIKGZcsOLk8Ls0yRq!+3_5xqOp7zW3v-YUlI7~6Mp;{`1q0Y zSI&{kDBIZUQd|7pBbfAk|DYMdiiF5o?Z(Ku2Khga>uJT{ zEn#M?&)@zqVT`0=v5m$U{9Jy{u<_%ov5LF&EX*%Aob=U*-)sd5FT&!?(>xooOi+f* z_fHc342VIrWOqCIF0OV@iMwawh-=I}F!hmHk?mALu$%PsTQ=i$)r^9Sz``WYyxw6E zat7U{$WT&Dhx3UB)4|r-b|dS7N8xNYBm`xzIS4wu^Kkq7MQv)Ou`1of{GLotr8rd{ zf$QZY1^$)}`SY#DzS5W%#D+LrK}e~D!{~kI_m+J<^8_N5gyNpjHuk*(Nj}WBLG@3h zpz(CJv~z~!?>P;3t`?<7{DsYIHjj5L))!n7#S;*yqQ|UzH>g|ZZ@$%uk=~RyifN09 zlgM84$iwAgyuMvlrnEybUb`b`-= z#~}U_Y9U6K*tBfBp;a+oF4e2i1y(*6ZK|g=8?4?fi-(2|D$GE3NB|NsC>N;?Pyf{c z40rq<>!!Bj{swmlckuV4!l@+m_rt`+t0e7wAK|o)HFn;}FE|G8SUC&kunkp7!|OG)+|%Ki zBi?l#lpP+zl6fvp;PIR>n>ZsaQyR~PNz|Av_ds+yWSn8r$y=Dob+y-F+;I8Rq68Fi z20?9dH+DrpajOqp)2>2jsEn{nk85}(_VX)Wv<}sMrpSIIBD%wke!;S;l7t%q&lN9K zU}19@MWGcdIwuu`2inZuxknzgk+`8^)8u}Q=LigKz_xHqxNXO0X7%uXfp%n-J^ymLM_))#2xAh-)V%7}xO>kgh? zIrTFV`TgnidWDmH_j%S=j{q;6FqSKJ!p7a3X_9>dKk&k^&!#WJsopxB^J>}2pB8>b zJ#)6XojJVfp}HBfloDKSs6KxkNKXC8lCa6B-lKuYV>y^y@A0+%iaigDKc}VAbm6gd z`ORvVD1WZAXfq{ZX3h5#%LhRPgof2GesM5i@yG69@s_1?%>qFpR01e7EFp(KljoEe zwf@qOm}XpJw}nx*waXGBAEf2 zPS5rY&Q*Ig1ih|YV>99Lc*1gTu<&D$r^6RqT-S>yZad|O($J%;PDOn8zXi~iUSo)* zv!Y{OS(WvsQBehbxlvk8<(36w2fu^n3vMh#wB=K1jVi?OZf`ovhYK!|Q>sNq%e^pe zESPvG-W22Af}XohzS<($R9??WNl&8Zi_mc{#J>_D zXT>|8y9!Zh`pILgMf7WG{8OQ)u~qG|QO%*pzWdn^+PynGENOzH5q;Ewf*t5A7vpz% zAwePC!{$XvgUUVW@@6ZjxzHk6uKC}c0u%o*D5|m3tJ%h}xb4sTL21HECq#LFU!)EnL(?A!%j zln))#9?*d0W#c^392<@foFpPt{R*X*yT0@q!|jD#tx?3!i#awAsu~3~2zJnI;&>TP zc$Z7?UuPdrI}qZ-dM>j$&BBjMmC7R5bw?hbmC5QAjP8sQpbNT^?;(9$6rw)jMr$58 zi;s;FAy|i2*Jp@G;*s#a+29$;>b8sQ(Z}D^dqZ7`<7jdL>9Ae)ue0U&(`)>X!B+cI2?F^UR0ReFfIm2$|Oe?Qpb0? z73a35XdSmhE*8&7&%ZoJY>rK=_Rq}T`Z}EjtvB>9K z=Z;n1_AT96CC_^@1U76RfBY7GsUEEoem7H)9eg$mws_x^c4ZzRYudzjn^a* zru$^C_0~MXqT_Qj9?x`l1QrKT$bjcsprg)99e&_YvEbJUO)Bu@8r-r>&m?BS) zyoGb0N-CIi5v{dhfXWvccYk0{wNkfMEN(Pq&x;`$7i<1;i1M8?`f8-Yaw@^HMmGsZ zgM9nV*UUfhjYe2cjZn{?L`Ljl?gw`UKg;r(tExKg#FF4p_KKME8!ETUm3q=qxbLWp zM`^&`R`u*2PXTwAm3!}7Y)enZJ{KjyH|xj-GBmx)r4{4dAW~J*E&VvL<;kxn%Xk@K zf2<+zev4?4?&?*4yWJPj_$9zP-Mw$_PSNX3!;QrhL!#lcV3-4rd{L_V%$TX#^o<~` zl}9-3(ki~FE$>0a@6qu>FiI^@@%x2ovncwQU0u^+%apzEo>F-K2BVi(&l{hbpPHB@ z&!mxZ`d1j|W=_}1-EUP)9(}VmDK9tG8;|#?7a@J?2n;e@^MXl-PKL4_N)y)B(jNE| z&s9~sE98HSkgdw@yIm<5O?|$r!HS1GK5qO*r6&(RedMz$wYNUQr$5cyXv}XQtRF5Pvy|8of(zVJcD_o+I-x)NP}x@v}R4RNy!x-hHZa>Fn6D zjEKmQXs>|({k$7zbE#C|@+v~}s-aC7Z>LlE2*J+?MeLcZo4LZvm?o&ux117&n}pU4 zxlQi=*FL2kJ32oO$g0dVb?f`zsYxE&NWJ2z)@Wt_1x4-QUZ=PN-Q$QvJ3>?^-8?Wh z@A&-tqW8V?IyVwA-qC(vOz+~o>eU@#plhZ`uJw=nM?1LJu)4_Sp(4Y(cl*xY7s_$( z2~Tlq!DG)0Me4*Y2YZ=*8z#~NHrh}LkM!IV!N_SDKqG4M z_IMIcfJIv*j0A2xd!1R-Lb;JE^+MU3A0TP=I9Jc{jwEJSNdo7NmdlF#-T{H8d|v7z z*Cf67uHiiT8T$JK8_S5l#y~Q61xDjPY?FYiSgFCMI`g>+=l|cMWc9WDBC36*P{lp$W2nX*dDpFh@&!MV9nN&}Fa_E#(uUuj1T{q!g2#foKrML`_rZx`2URC2|Wkyh^4&4=b` z{3x$!qo<7OGSw-32`jFyEc*7~lWMQ5HLZP^`icOvIia$AFwu*D zZ*n~Y99q|Z3s|$hhuq@V=ZE2^}5pjc;5bLV5GV}~$gE6MF zNwXx;J{FlQYV~G&^%>crNlsLk5_5|nep{K&PqpA&f|jj{sOPEnV7~OxenPV#a&o^Q zZHsoqx>hRT)X8*iUPPJR=2W}F;co7@xTq78#%(BHIJ}2nvD}|5{+8{ktOx!Ij{Tg; z{NP$4O-6_>_eH^${5_lgTke{fhA>aI=i%xoaE*HX|7PY%=f;6PkGO_j|rH^Uz& z2B^r{^&|ErW$$U-ERQ-0MHGB>eJn{qh@RiyF1~8jzUm^6=qdNDrOJ-%aPq%slg-3? zhV4H!C$eroR)O=t?5;-*BK3!$pmmVYG}~5|5d6?Z^_C)^G&&{*s#(U8LV4EllbV}# zzrCaA#+cITvaY&tlNy{ohue@TU^l51h>Am)Ct{~t@nbZUR5E-UR^bl9GJd}*PE5Bk zl{L%`iA*aAkX#eFp=7ZX{2JNi>9CP;zFxN0?k)YWdeV}4$Mk!-)0$$eL5**_Q_~^ZXogKB*45WCEKG)obRVC;qBXp#35ku!jk}VeK9^pJehG|k6#W?dX=en_pAn{@0mzQSaf6P7?;i;(&F*_ zHKA_4TU}Fi7jfX_zePZ4BV}E!eK@%}-Wr=p5U|VCGlSMF*FKIqB{Z!TU)NpYEw6XE zX^`)D@7`=B@QdC#%zs|{)0OSEi%!4+Z{oe)EXQ`kr=Rp3glM{4#R6MsVRx(M7UugC z+~%Usfh4s3Zhu^gJ_X?6eD-9vJfxLSz1pqQv`&r+6t?0ixiSP-t_LQelQ1rwk4G~a z^UHFgdh}~OQ!VOoI%o>sf=swC-+J>+G?l46!+<;cJA~o4v<*Wj59v2_syy!IF2eX; z2k=s1s=}N$+(pUAm>g>J)$E_*9@}&4QDZ#5KRqx1I`jAiuGKH-Np@#BgR%HkZxIwf z58<<~ch4A@`P(#aOOu>Tu0?&w61G%_Gmf5$MK{{-mWZ;&i~wgbo|)47qp8TgpU{p? z4@V6;C%%4@UM{GfWp%4u`wINHyeO20J0I z#=0|+32FzPoZ#i!%+_On^=GcgtIdq_gKNk=Vg~`KgyNFIxk^vjijJ_DQgg*0BI0>2)o$FN}oF#_{nx15BQCCQ~GF~?F7t5;zNxrR2C8F0CO zcAHzE_01rUt|LACzLMHlm^+9cZj$TlYgze*bikF`;Hy?dY)1_*v2yLz;@o6gaH|t@ z-mmvD?!k_QJMvv*-h_P z$qBSn6L+VmLanGdbn&8<)E0)w`wNC%Jwxs-)1hgMkm`Mc5~-im6=62j*`?0AOG>rv zdChn|*M>`1(18}jkydB?WM~NBMT~)XRps~UlhlT^Px9$TjQ+OE-utDW;p>PI8nG?n z8;|2lEA%A2>ldub_1by4br->hLT&_ky0v3UA@&7q%2yh8=Ic0 zHkY<(bB52xleO0;hk^{B9>^xh%f%LT6fMfw7jEoZ)6;H#M;K@!wjt=C>EQl2a{CiC zzzB+ldf7JOh3rYq`pppFB|mPp>#869YEt!CNTCLaMhW5X$0SN)W#E{s%SkpVPblHv z#r@OgZ%^PAF5G8I%~jWBHDFPW?^M*);U8Ew=~fLZW{$k(HFLMsv=SX{OiWD7r4!2tUvhg(+UNvd^qeMB zzN6-NN}R=YW)gHf#jU{$ zfp%#-Sz;7wo`Ag^^O#VN({%1vc@V# zG~i-)6Q-3uDWC^cb3$KK6Z?wWP@M6jwi3z<+-^wNEbNG7UL}Uy#z29nklGZ8!-tD8 zMvKFf)_!bedS1;51O5t%d z4`R}3ctD?>*^7Acf_(cnR zoX?v$AyK4W6?FWq^;3%L+hmKy9T9mf;pmuD{c6Im!4d)L;#l~>K;F#;6NU$lG{KF{ z22h183&pRRWYqg;nUbjXMKf6tr-xg+CMiYlIg)g zTpdAkJ=WXwkIZ>mii#C)!`&2Vv3qHK#RnLEn6BU(U(vNF&0y%gt!C?U>Hn6t8D!c_X|3h8&tZPB`z?6p*>ggG>i5q(X^HxhsE2)@`2`%8A)=xdv({{$ zdP2MJE){tNZ)=S&YDbz7esyFs8-da!vQEYI& zl~pIE&#(S;bo-j@p^%dmGCF^yaif|SNQLPIqw#cz)xlh3EQZHtV~y6@JyE^`s)J-9 zrQY)oa1$O|7nYiSpj zEjAR@!7WLd6)wxc(0^OW|L%3i6I_o^O0eD&7Mj&WM}hRE9J08gwR7*ZkX5}B_NG7v zd4RDW2kkIIC;>SD#N#`G=Eq%ZdPYe*gg4#5xcY<+e`(RO5Vq(%`a_J8`JI~bft$ip zFeY6r$wiWyHzfspto?fn`%DTPDql<-$u~QY(|4#L$Hay@g#;kcP@&4{cS=0UvxYPZ zRJd*|luTI%I}9^rKC^5V*mOKFZ()hSHbh9Uo|>a(LcUS}KB;`7{0~Ogq%z5Tp|ETW zzT59f`XKxN0I={XUZ`KFClsUc>ujai)~_+ZdHj8#)Z}*j5s#k(gWED%+fa~HD zVSwLgz$ry=>aG+ZC4Pwy1U@6encGJr#F6LI)qBmd=Vbx8}F-2b*A)ki(&>@ zTLO*~46Yw;qL1l!L~qQb0fx_>DHETG&s~n@K1~lTs1{%2z`bhyAdld|&E=Pyy@;MD2x@7HJ zYZfn4Rk9;!@_+39e=gapfR2tj29JA2Ga9etBYWqIejU$E&g^?doh9#K;&(JmIhAkMJ7_z}$MK0BfCGQt|`SE}52O&TUUdfK|kcieliA0e+yk^`wR z0@MUe7bW^N)r$uo>0xv@Mp~Pt(VnMDHT&?V&TyJpZED{5gH$mE!UKx9=5KJ z&?Gs(*`PJQ#QcS|5K=DT4%G)BUlMr1n1>00QUuXX7tPz;fZ`cV%&FL)y0;WDU`bci3%0UX9e|N)F1mdhtqGmW9Y?15>im4R7z-WK2 zfKW$IKg?=It2ce*^Zy50X*eGq3$Q_=sV~rSIK@jkGt3$Xh6e(e3jiwz3Ua^`Nnc`3 z3oBZ_c&6_#&D->V;<`aY4U|N;%?K!qj;3sr_P@T8f+K|ChQ+YfZi34O9;G3l)qHV5 zsjT-fy9jQw7SOK$etzRj+^4v11C0>}&y`2KJ(LPBnk zuq+gQs9ZW2KqEJxH+;+R3*U?u4nO>Q{n6W3ovTkWj7&%|)9#FxXA$+|svpBt63C9**fA+Q68*&92U3pxcS zRJ@Xwps)E$yu^(m1maivZJEmU$#rUaMwC4u9*BVOF^N?R;^-D?w#zPJ#i{vz=n>>I8V%-DvF(M3Jw5}w?Gpd z$Fh3ZNchmDekA2E*<8RN0WU;>@)qtK07WkbG@r|H()ak3cc6`ApFv^p18OkwJL&<3 zF%z*Yj$(~)Q5Y<0p#3R=y{qUO1Xv;rxRKMVyJssy0 zVx=O%roN$dJBC&EL0TL<;p@7=^ z_>bRVLcwTs!SAmk3Sox+!zTf>fYkMahN~!k*wI2HNE2J+vcR<5v_56MXVjzV$|m0c zfx4v0OtSEbVP10TEE6CGWt(vD;V{FLkA&$bh;^CZR<&4}S4H*(Q(+4KbszEai$c~H zw}lIYO4L*-q7i+=}reqn~wy1=qxirk^OEU69ad>q!)*jNKoH*eR zOu`5^_**oy@+1$0+!0%qBx@x@z~)PJ(7=8hJ}n83axHmGt{ zh26Hj>`#FrkdtPoob~v7+^sW^#zHJGyN3`ch!ADt#@_v z)+v=MTSZu&5_r;2PJQa()64(~%x$vvo6#6}XZ0hf1@$j4R+`);s8UPl#L5CpL5b$2 zaakqcA+@`B-DABhl+q*R7A zHspVCKy`*sZK#Y}+{6`C3J^yCN~ZT!c?thvHApWf2+~=vs(r5s+cECfRW zWNvaKz|JZqps9BCyzI#W$tmG&)-X*y-x|#xV^Qp2GN8?{2`;6^-QScAMg)jgl&1pr z`oXpNWCsjixZsP!Auq`3V5qz4Dhm^|rmn_7_csgPrxVgU& z#5}$HmI&ZVUU0^na6H&Z0=^U}YuFyqf8wJv;Quh-Nx#+x@-;s#j(FhQ%?FM`6iQjU zdWBP}qqWAuzn?0>=}#F{40u{O|z)V2~(jkuXpz7Y8UZ_s@z|Ip~!X z;y>9EYtax74P~ql+)KL6RE7U%EWiIncbKnU5^`y&4KZ{l+zoho95fa{Pw2$k{u=T{ z;xI8(fIcZS54wuklQbEYDswh~PTbTl&c|**BM$UOdf{8lE+KNKk5Kjyvn=4MeM}&L z;s2WqMhVUsy#d)PuG0ipnfiAX?*YS%rXeb+8O&~tfo!gng6rRy0E2zPZ~?P5Qu31b zka4|OBuNuhE?WT`yO-Q|jwr7ySCY&dt_jE$Qga_RJK|s3Wg#Jzb1;1S@FSQinh{{X zXh!aC3ul;2Uy=vM7w?@?7q^^Zc@qOiAqnUS2mnZl0;(_M?%RgB5)=xf{K^&6KLg&? zr=Lnh7HAV~`t~)*lOyvyc#|mxAff_iv>D5l>BH@S;?T_XI0a0g9X! z#e#DDn`R)vK&SX+E>Xb?)Aaw1J>=0vXn^zv$UgP4LYbh6EpAU5d6;_2HgL#RB(?I( z(?#Y$IB71Z$vOcnGlad;4;nJeHpK+>5?xRvuLHzh)dH~Ue*%vua0(oxrmpTBP6k`_ zH{t=AhFB%KTQCtyx~bTlg#Zcwa>P9;hN)XD0oeYxz!EkRp-38dWz#r*$yh0y zm{UcQ!j#&Ldh5bjhiplB=1IS}VFZvAIZQh~M=Tpe44=JN&aG1st+8d28~R@ z_yRUZS3vy=c;JKqSO98~)p%c0IT$cpnWX?@v}1CSh~Krh8xLtq>C=KUH~L>?-p(YFanZ1*4Xwzzyxs6CB-D07#Z zXT77q{ZnhEpHcGe>vY??SY#O4N`768?w}~-EsB+s1dx79L4KgIxzDP+{!Cn`cdC=B z^qCzJR<=NAdx|4OcuTW1-GG!cZ2h{GW8%eM3gDFNA)e4IcshZd#~EyDJBQvBdj;BrElGa3YF;|Qc*)cv(O2601R^Ca))plSqNqshgV zp@~3=4ZD}$_Yl>g54gKarh+?+l)I1LCQlOPmcqw1v%`Q(JMh9gwu~>C2_%{UMdC%; zWBqZ0+qg(j{QRo3_hC@ASk1AQ9lZkxh;)-BzLB~Ys)U?Y4^st-IULy7z|^*wLL*CG zP+PwT)mPAmi5N-)i2$Y(OhH^}P-K*i3iyCp_R&r@tIKsOMphi@y-JaK1{8xV-jisD-viU z#V2sqZ0AoS{4C$M`zrO!Dx)(%I9a4D%k)5}(@__`Ckmc&H%{&$5zOdaCwzkfa#Udc z4BB!LSe@9;i$quaA3QJ#qf5Un?G5fSwg>U4qS;^~4~v>>JqVZAZ83V9&Y_d%>zPeM z*vNs7;h;9C3?HXc(i`Gf+uUfx&QL}_NXj*a(^f!JR-x_^sG1A0*=!j9#~rrU85ZS; zL09mTA(6*Q%bGjzfE86VEllM4yW67Vyqx0&$VAZ?#)OGT8EHcPK{5kir9e3S+~qAq zF?qMQY7D#N{5>stD!;2tM~;j%r?~h^(j%6m=-<67DcF(Ye<^3OazN5PM-o=Bz58)~ zHP5HudHaW8(sFvZs6@tWT%YeH&5RZ5n5vs=$kw%7)X3lCm@HnrIxggKU_4ivH@8VS zXQ*H$0BuA-&2=8@8}GllAR7^*soDs7OH$)HQ0@cDiL+AQ(Gv8yIk}`j5T7pV_>A(^ zuj3s(y*i!(`)660Y%;P8J4?IPH1A^fi&9$poK6Lk4C$t7HSe>UxpH&mcZ-?sPRff; z`)kByw+_vpo;QC_61qleyU4P83x2&ZuQD$PbH7f&-wLd^6rVdVQJ$RaS)SUu=X8v- z3j&;z7EZs;seR{cmU$wm-ACu!(#3O}zd`kYouBhv%F08=liOaG3;LONNaor736x!9 z8P3avg?dWJ7J?U0c3lMWux+^?PuJ z8lX_Qf7021XapF;;H8S>85)KdHah5cvyAnb$SMi&|*4P;#Z+57`u z0%JVYFR2lFqw$M^3yv2=TLvu<4X&a zEq%ch$ddnS0&2hnIVy7#3IA(?6Tk$BEx~kT|7(JHzywiBX3FaSYl1c$zyvoGKYaez z1d0>@Zlr(8>bxXb|K4Ea0j#Wq1;f0gHvis4LjbJMC2%-W{~IVTH9{e#=$12|=oVtD2YKRg0S r@&CH_{~+Q&VdVdb9pB$rH_zx8^^yfvF1jy2C?h2%CtCVJ|J(lsJZ`07 literal 0 HcmV?d00001 diff --git a/docs-v2/img/sql_distributed_read.png b/docs-v2/img/sql_distributed_read.png new file mode 100644 index 0000000000000000000000000000000000000000..4172d83fa95bddbb2ff65fa1fd1e8d94d3136586 GIT binary patch literal 234381 zcmZ6z1yqz_(>}h^ozjhzgmehf-3^ipBHi8HNOvO$(jc`|=@DIx$qg;rilT+8dl z-Xgk*fu%qH!}4kFtHBc^`3*Xi$zq2bkc%R_HJOa!y76xv83STXzJqW3KtAGc9rMJg zp0%5vq4f^!T}WhSi%ae-->T=V6iqnZRBOzMD^8(HtE3R}TjMd-We>#L?Rt!z-W4pr zO=c7F*jw zy-AYuVi|3s476XLS1G*_P`Jw2S+^~yz1Y1{-R;QOIq#dXdgwpwUrrCj>#@_ql+{{# z6Sp{>mX-}LF%}E@KaXDFA;!kPDh!?L66ZA(d)V-C-^%y#@j+1CIYkJhoiNJsI^?`O ztq;tTJK}qH4ZK%aY>)FaGI9iQg3T;$o8$XamQFoAE_}u_RQkgp3r604=XJtM)n4z6 zyzQM87*e)87FZd}Jr?gTPkFVc`%AqpHLJFf`^xK=AfhRM2_~$RPIp&lC=zaK+459! zc2$Hg^zqkKQ}_;9Yu*&_ZQbE2=Oy||liBEYiw1c*h(t-$}&UektWa>OvfGX7gL>s!LE8|`Z*5lXzw z&v-8f+}wRq?)^S7KD4#+sWVGA2HdOExo+PKRn952EGSG&;NtrG>*b9f@{d!U9>|ya zmZyxkFyb!Hm>zr?@=x7*yqt7jO%u%FPr&s26CW=k{#WHug>-)8PxV5KdTEQV*HzH( zj@xfr^R|k}QtJdPJhi^Z(nTyG7PD1f(4)M_9&YZ)SDmJPJMIJr+iOHl-8;+!90!dF z;=a>CC=Z)TeLHHugL^I7c6^6E7FFD|G6Y~(fRc&gIhHGV0+-tX(z)&P<&9sPAN{v} zhF-eL{WyQ0`Q_x4L>E)U%~v$=;re9GMmoBVqwlvUlIWRQIh=S(4-0?;^Umjg+YE43 zv5yp=ElDP701=cuK*q&;H<(8a5r(^lqBpyUu~Og<4uyFWL}Eqp#Vw9)dc=HX$alMx zM24PU`$@U!NFcS3>_yYL+He+w<9*@HKz#37i*GgGRZhJ|)u0r4W zDs=Ea*P$UB<8&{yU-Q@6$2Y!LHa7tA{PsvOB6@Gm@N1L@G_M4*y9hzb&h~a$nsiCK z0A^4z3kbgm`R>3frjDs-MH6pT;<|O~FXgP(qb}R{Ek04}t&v+oDmkO>^v=N1YT>Kp z$G~xla(^x?iFfIT_@=IlzfyB>BvyuS_V;&Ls)e{(Q|65lmzhInuB*8jBFTN2A3-)V z3E`ADElv9YgM`}L@!y$#0uQOwm9W>fVBdN}2nvzgEplr;N7<;gD@<<``C9Cf8`-@1o?0JK$Ze)uGOjY&SLXDr@ih^F+ z6gX}{-l+dSL+pl4KcN1RxPsz`5!YrW{#+F72IDWLs?bR@X2~ymDJjj=(`4Aqu`B56un=-gk7_fk*hXSPTd!vMEEAyXr=vspv*Nk7Sq$JQu& zC9S*PKrZ+p|4z=-&21TeIq*(Sa5Vg#PdWP?!zgODO=VJr6^{$)l-DpDjua^t!=dNx9 zVPuWZXm7>GzMzg)8e>M=}U_kCCk79M?OF<XQMcfK)GJo6D;D8Jf4votFl26IxNT1G>8-6~d8gNu_v+ zS98VwtVT_@F2B2yz~x9T;5HOnB*WH@{9*|w(y7@W6?kg@AY4|x z*43OG`l$G86e5eylIj&4i2xC4EDLJD`oDe`z;BL$E5H@u6V(>h%yk1ZOuSk<{{7pZ zTxF$Uq~ijsyt>c)qLPh5@Z`25sAm-7ZSUGVYGZ>EX7)mU6x2AE_N%hwtG2{|c0*03 z8Y;ck9M^B&G~6o?4%D5ISSAtLioOwvSWyUn*kc`;{aKvXgo9E|;2d_^Yxu-gbtEv& zUPLQ!gs>1;NvD0HM3>0@h*HG&dj*;DP64}J#acMvac>Dl*NuZ{Qg-Crqhuk#b=JVC zFX^#)*LU7KMXp&=)j@Z4en;E+kJ1lbdY?OO)OaL!TlGQCD^uFeFDI3shA^FwR%$8! z&n9qr2xE*>4!Oak&ZN?pEA8(^8>E|nf#W7$V1623rXuJSEt9g_E|npqqL7ogLHhHc z(m^fPc;j6ChV&L{jwOlOx6-oo6=Zw_#00RkByVij%w+ycDHIfT#BOlq3j}L3K?j;O z>d*>Iekbf0my*d)AxgKEpuXb>N-5qE>h|5a&yvxwSgMn4E*<%5tEr>w1@eqGduCvR z@2QG6@C|V8a^F$3H6sx6o-P-ETO+cnmo*Cc2ZgE=N!d`QI>08B% z$UU>Th-w~T!dz{Fp#9?m#{*nL1mxSlW#U9D7$}_rA7iN~rE=QUX(y@~MPvUt*aU8$ z%0xSKwWzdImaetUT_SV6DlAZk4;&!JkkXfu6JQc-}ehVymbl?3d zsk6UPm%*G&-NQi2ytRlnd&nv#LD#lyO)TY_Jz?Ysy@v2Z_Z7MldSKjhhHCO+!WgbF zayThjH0XzI3vlP=Tu8d<Ya1e@cv7(5xXt3Ymq~mz%$vIS_jlf_AMK< zj}zYso#n(EbK0^Kj!XJ0m{4BBLFXIFpWv|)PfbkV1>q;NaB>TyL2*s_gmCi4-NpR3 z3r%8ei>Gc4*%;)~6LsWNtx7P`M8z-W?=U z?`+{KFtgX{=N0M;rBv@b~t)?xIW9 z&tRQH&Z2i$_x+HIDK(~|USC(A(YRX8eIeJm*q+y-16|D$@L2i(OxInJk2-2kg`moO zLFZ+ps;nIj%hd19s@}{=8Mp|Ia!v=Y)*XIPi)U*-TffH_QAR8vqZ@U zI9-jhu$Qc^OhnnQpY}pmHATe>#9>lwHbB>}3eyOiuI|lr0&*{=(V`GZ`USP}g|2<3 zD_r^pW98GRI+IZEK9R_?i%{7M$H%+v0-?#-S;eHzG#|^UfWuEZl9*eyu{94JB$dhK z(j~cHYwldk3@z%qf$smTaM-?1N@=H9Y8Fz-CWXw!#Vtsilb*g|J_FgjF)2Gxq1h<0 zXqZM#_{H-vz73oj6J9HFISHR2IRD9Eogk8zO3PA+HfEU8C8}@|jEg+-Ec6@TsF>O_ zK+Y>|H#19M1LT|51*cAQj&@2lw5bylA0I^=uodLBotLA_I~`ez`upUA^oJ(;KIth? zKpDt)1r1H&J&&=G>6{0)@j1fZ$d`LTbX*B*w!<=rh(S|@`rW~jSSs>yx{jGN3)IZw zNc-BhWm@)`+TX+-*fa7NBhkP6V?S6{fPE~B*@X!0OpJ^cmrm*MiYxcU-WqO}R5zBT zMg)grh<^BIf$=clP-BipIptqnLjh`}0w7K8oZHx>xviimTew%hUA~<4J@o!w^fASM zR(Q6gMdsYkp>%%2Ir6QQipI%9^GW0t&OcE>*L|YN$Miq8!tOpxOe(z$FtIawGedWB zEpnWcU^u@(j8Wx)7XAh#aS9dOiSuTUOpR1pgJ7p{Z$UF931^v#GE!)f!7x~5TJ-!eE)B^zbXGq!QNbV zcSWO^a8$`cC;d*Pk6H z0f9{@8oAnvxloh`HUK@C%I=7TH}#XvR{F~nef1-*+8h_nY9Mt9SV%!Qi;T@7@Q7Wr z!+NL0$*bhx&u?}v5{-p`5b*xq9(2K{C*!EX@`l2>Ip*sn5P@{-wI8gk0lT{VUR?$8 zm?EuB9%n|Kxeo|S!cTU3C2V5w{I2bIh&xR5IC|4ygr0{Zydk)$Dti1U@S%FXXhCDp zeP&z1!FS3h&#QR&Tn1r61@2kin20IEp3O~Pif3*g&7qdAf{Z3f-rpU91ti*nWyk6M zKlp)e_d8!(k7R_H*twAT$V#iLRw4;@EGjC3KEnaDl(G28D?jvx*3QeYiAQo?n_Bq+t3$K6Fyaqjgm+R@tr=s+;#>UEma5MEqF;lcl3^<~5Nh zVcN7gx|!|lzA?#hgYNTmZJ-N$2GIYuluhU=KoF6<)a}xh-J!$Q>D#iNEa&!MaO}<= z9k_ve3^g#W^V72^r&eud7Ozh{e5JkB)U6$Z&0geJ zipp{91ihX6<);3VkTcom3w?FvU%Fm24~oc_QB|LdP2_0H008ba6ZlAyA?X=(ekI-N zRRIZ1<6BhOT}JJ0@3ocgk&DAe$aef$c!N(bQqe`;uDf9NgmZ(bV{*Tf50m)2^bKw3 z2Bfm=CrAHk%D;lbR`^br5pAI~GxXJ$iOh|3+dN*RR*PK*(0k9Bas%^t%qVm2&62(>K1?D z9A%>^f+xx1^9_S0! z|D0MM2HbC-gXARyM@|0CY#}IBV%V(S?1FR0>$pyjtIQ$Q=T|T=HcM*%F>+NUcti5~ zkG3vONUfwroUnJw+_#V!G}@?daDZHIsG4LGVz>bpk>yY!VLv*Ll@Z-xHkzy`$-Zlv zk4Dz;YIje{^3C0}aMG?|JL7x{cWI?PV}&+fKUGv+UJIZ=>{%ZU%j<3cTXTY~;8>`~ zM{}-MT*Qs6?o5tAPqVwwbc#E=ycac^t+#cNkjK}0Zv>|Qd~WzkUwkl2s4letuy58#B(6qid%lNsy*TN#h?!@?(B> zV3*-0=7zJXZY`?~&#SlXVmvmrqL%B)7irs%;q?2Brf{vvcm zLgK2Zp42G9`_T1*?RjAz17EfTKmH)Tg{9NSIv={SSwStjq zey-S@-%HkFIXqqvRvSiEiwz8#$T4vW>c`idqC+ZbJp1pf>z9x#kz z%oXZ=l5S9QtS>Xr%RaB$Arf#fX>M9EyDF=x^O`=1&6sckZSuHC~@c zV64*v)je>Y1+{w<8%+_ay2`crFh&wK4@t<%N-eY>&uy8rz}2Z4T`|1wlW|pYo+!2? z#b!mRD3#x*Rx^45AbjN@tp3Jn9(Bf1R6X;}&(Fj96MtY!sXj;F5>-t-xw#d+nln>< zFu6==t+r1AV@8rB4;oXHX3A+enpas?afLqaIVr5Bpg$|Ih2Z*aAmt$y=D6;6+}~t9 z?S5ONW#@yjHWj$23cJ|bw?@H8p_o7pSuQp=Lg%PLHS&(rtbWKP=Df9YF3m87)$Es% z&tKNU0z&Zo5)DYoo-52j-^=kvRB*npy*vIM9Bz!O{N3%Nqz=eas_!k>&ai`Z!KH!X zn**VSi%FsP-f9yY&DXMTYM89QY5wZ!q0qwkM3+EcSH|Btm0wy`1FBFL36}F>O%DiA zt%Nb9K`DoQgLj1d0(AUhtGQ^j`!xy>6 zFuCySFY0F#u`2ZxbglIQM7~CQBflVjFIN)9XZQZEww~AU=nNn7PA?IE7(72Zjbr`- zVGp^WmpJ0zNWze<*&h|WN?5E%Fu1n(VTy#sm3ngUi-0r=Ev6S^u5wH*+bQ9zKe^Qa zQ6BQcL*=FLM6+-BSfg|YqkIXMASGi(?#No&#;+x-#Lq)&I1-qQfG<|h>NutU9TF=L z9q5PmgVlg>G2(w0{cY5er8I1Zl=dVvt*8e%%Z+d-y@rR1`vF#R0ZG5L#ot$KPyS z9*^57@H(ocT%4aJ)Q08t(wb8N6mdUzB1xK0uoUTjFuQTZVvgd8Jhe&yfR(*fN&do) zb-V+3vS_yfVeCh+U2IL_Q$+I|B&)HCVr?16K+l(S&3yaW-`vS$*`9shtI!d+8YINi z_{0Bi3FD$b7QAl7*CXf8PkkG}H;$`*j-^$M>+WzV+SjcxN++0j42RIn*+N|3h8R&)MV zax?i%YVX`i_QI-sNAn2>yzBZt+he(ul_NiAg)Tk@l#D6pA&DYHIu}{5q+mbk0T_!> z@YYWHOqmqlF9ihZ@bOlILn1HVGbe%r3t1kz#W;=_Ee2|`wxW|Yulmr(-`#yL3=Wy< z>~k*|Pg{Ux|7%N87HTQWoLDN)P`X9JJ;9@6TgW)^s(F%BH$jmhTA(p;c>iopa=~Tz zymCI9G!TlnmW^772pN)4lf&ghkJD@lM~Pr#Z|-_3FbVkPCNqGG%LLXaD+POFL95PkEe>wGTqxB#|LsjI?ne>lm5t}-APJ=d$R@<7fJBMcA zSqKNaRg&CUX5N*4fzgtGWKrT}J`Wzx97~N9Z0) zcoM89eYu{n`{ua+CcVmYq2Te|T4R&6Hc$4%;K$l6YIw&;m*D4>xW@B4Cc|*^r-WeM znZs$_@Xdlk(tfkbLVI14_hxp)<($q6DQC-M$Y+_?(wQu8enk+%xhL->p4D8#&fh7v zt|e1ktJWAxgP)drM2O3|TnSewH%Mof$PkOWUGU^yHDFcIw|6yQ@z)t2g|fMu<%g;d zj0yBvk1;l$Mbg5?VgIS2=Hfyt(4~ZFjjc(N#t(_ebRH#EpoOvS*S3zWogxco1g@!* z66kpW;qrZf1>lSU_^P-$QP#xFG>LWY7T{V36 zY07jr_^MvQ`Y3)I{Oq>I?kcyn3a=gW-!z&yK%pY9kqKa7t4zIU2oBH%_l11FDKqmc zv@g{Zsrpw+d?n=o3#cFqZbozPaIN=nKkM)VzJAbirY`FJr)J`__M3gTMXP%U=qyN{Uc+E?f=&0s ze|m;1N~a0x2*9cLKeIoCtQp$VFpqppCWleytI-2bf*1RB%6YSvGaMr*4=QEE*{&v@STYpZ1YjKTMnC z=r8KCjVu9|ar8+FC3M&b2aAe2Oj0_Qx7UgpwIYv>;q=RL=$X5MkFA!D%l+|Z5I3d< z+rDVy#m`(>u)f(3H!c!gi@j82W!#U?q6D}RkcnVT%@Jn>{l0Z)_NT-AlHxkbtH0lI z$qIQtPN&O0;GgXnzk%CV2C!AnAM|Da>x113u zoO|B`7I|D3_Ft`XsL(P$3c-lp@xqNfuu_kuT1|QR{8SZuqqj>(1~NxMCIWZuyaLS< zGUj6)PueJ=hW%?osR%pr z``U?w^+Kbiu-)C0MmC$J`vQ(dMNb}i`9VDGP!K#n6cn}cGbHsTo&?XUM%eTF*NM|e z$gADeTX^K~^_706#P7cQ7C3%%3A!JCD70^k%g(;AeMHDKHqso_ydnrAw9r=dt9TAs zUDn<7dcGr%lP~f8$?EH?tC=!dsxK-fC+EBt$(CP-2Dc$ZTj(1`D3tW%i7E)+X}2`4 zyr1T>H2JLT?sir#k6iS;ekZgeXH=kTQ@ESp7MrI<(Fa9`(t+()Ynp*Rr$0=n-5L_l z2rKMyY0A_UaKB~~;(Dm5QQN6E&q`RqP&ghvl0@N44BFdB!%~Ys{Z%>&+sxV|xkIh~ z`hOW4T2>*~8$jV!e)6%Me>qAf9-x0D?j`4=_i2-a9QolO@P2BBtoD83 z?M;2FnSqqMMy-k$W6!ta@%C`Fv={$rSqPZl?YN`XnoC()Y84-j4~+bYDt5=bNFy@# zFt+`;NdzOKc!`rcmXWu&%f~0%O|fia?plW<0e`{lH77N~3{J@xQ}h3iP8iMjWVQnjb_A6?3&HOy}6M`wrC zhFUDgN#64>c1+c0o*@hfA#|?lb1+ z!j}uH*@q=>=k~UU;UzGhRp&lQkj0C3P$*5Ig-s6_V%B_lMH_IRzN+`o-c9!{wX*uwNc3VmH*g^Xpk*Epco+_@=Cj$k$ z`$0)hL~Mi2NxYfSO@)K2ZpI9Fsi@~P!i1{^(BC%nZX!u8@hB8sy-s9?uKrr2?poDx z={F9FpOs4vigh$$ww683!iO)?Fq|GiozgOAV$$6nv0S!t5BGq=Uw?QAhN#}{;x8EY z#J(!`diP_-SUN(ghfMQ?uf!YSzgvSOBq4aMQO}}PP@0D{C2J%qRwHzAI^bdQPj89}Q&B4ue13LvXXwO|2yV`@BmyLdF{RQLGg1i2z8BpXHNEfej)~@xy zqHJBMecatX5MpE2%z5W))*Wt>gF-_(;*zNA+!Su|rC2Rg*#k!tHk`w8sa469Doc>@ zp{%9`ltElm^D`!Ygh&B+HKXc(*#Cr)Hu|-iuN~Ocg*hWe3!Lfwn4kGWSM*6dV0;Jv#Ud`8k_SbVZJ#f zKqhG>l?)glv1KFxm8VBg2@)mb0 zfA+07lAqu|#hoX6Vkxpi^TYT7&8EEymG)&!i}5-J{^Y5OT&$!!`TEYzKGf#`{maSXEO_)4p6yP%q3u6qJR45gecOBNrQu;wR0Nr0 zUR&pPv_qL`jLD>J?%JmjXvV1k!;Yaz=h_9HEFwwc8|Dn8a)tplk6iYJ%R}GCSCf~S zu=o!w)v!JdQfaUTAK;QBC^2P!!7+KZPtdw)JPjT#WrBA<9&DUt_SehCkVv-50C~-o z`bvKP1yCpzy@2&IUsI)Q{Z!UX?iVl6VmN5eg`8_-RD7p^k?a1`0*5H_xlI2}&0Cvj zKuDcOy}UIs9#sU(+SULznQ$;;jE@l}LPtlz-HhW<*I^3)1cevrCk5A5jtEQ)gi(hU zG0LKsu;1P59{PS56_eYy)33co$Qt`ttD*+J=nZzQrG=MieOA~+CN0`!m1_Jf3WiWz z88&#zMsGir&2(BVrVY>Ox&=t1c(ZlCUbndvjRXpx9hVyWRIuyQWkmQaD$O%&>Y7DW zUjKqT*8o-o;wS>0f&{ZRb{zF)VdFdym(#Dj_KLfQ+2w0+A+R|A?WePE^aM?R<%k;Z z+kwwU_hOG?7oQ3q_TYW>^(89Gtve2Ob~IA|6o%>kZLo3Bx|90P5S4`_9iD#ilzh_Q zS&Y*asjZXKdr$>@@QmTZ*I_bR`%6^uZJb<}?6dst(0j0h>f^?y3<)bXn6JW5Wbdnn zK}S@_bO0~XbJTs$(fIaZ#VnC>?Ff!4RO$<-_d2gGs7BcH8aX`m@R(213AS^)xTO=j zIYVW7YZPB;p>j3X_?R$ppYUUgocvjm15VqrO+cYWs9bbjcK@Al$lP@s<&@MoAyKuL zly0f*ggT&a?C&~Uony<*-wKp}sz+UrkuJ*Y=P%k&U4hE8 zq6j2U?l8$enUl79Z0$GvP*ho|i-D17XKTRn_$Tyuxqo_1w1L0es-O=i@PAtEv~A zg1JqxxbRV>pT(TUQhr*onV8}>cBh$~p`jWM)V!OO6+M}|`MP3c!DmL`XwIOuQ%s+p zEuG`|>Pc0nMi+7QfPv#0mUuL-jmS@)_v4VO2hUpcTPQKCZFjJ@!O7sjvYJQt>JO`y z`G1G!7&wr*jBE8&^j&ACwU{@GW~ zQNpK|3RWU{EXvD{mZiZxY0IH#5t+eqtoeH$n|JthHS=? zIbTZzIA`HaQO#`84Pj1L-!B-g?>g3q{|ZA1bZ#vrE0#i|^OWHG*@VUW5~@Sg$cF6x z%!35{?}y7(BbHon|FRXQEiD#yx-!|~Ryiz;zK@pZdRc|BQ15GYRwK~Y^(jzeV5rHM z`rLhnuJQGCKX+TfTR%GL;R&nIp^p{v*Q3T?dKF@VpR5cR>pYHeG}9)UHh#7Am_O6O zuVvsa{*x=2b|hhi4|kQ2Y@Zf(m&yw)F7mhUv%gk{jin8rl|P<(4k9BVk^X|EQ|^3Z z>H)kFVHZ0gKI>7XhaX8|O`i7yt51`;fv>b1HNzz+k=}f)Q}AN$G@aGpHu0^K*l^U_ z5psdfWRl;W^HV~UyP9p0xzmCMo4UZA_>9|icqIvf=dPZBV&lqSJmthICh=5-iTBWO zVCzA|^!e`ied@P9<@S;+>~b(`ip}RZC#*2Yh z=#LjOqVqyO>t%U)_h*b=YeaMZ)PVb!4@Apse8CJ&h4n>-M5lC z2DI5Di?GS@;Ey0ne*dtFFS>L?1w_^KowaUQ2o4sm`0}bqP{_92la|{KMe}zSJ8tC{XUSZJ@H+&P1UhJtei{4& z_ttFD%Gg_icOBZV+2K5n9C3{^b@e3*TiFeL&F)kx2T@yV z%=vi{t2;ZZHg-t;zLL@I!JZ6oVsLiKX4d{{W>LT+F3d$5C%>e6zj)dH(4idH9k_NB zf}>@r_kli=sCh@ipH6IaI245)_Gh^kcY4DGEtt4DSKPn-toYx9-aCjCgx--*=uIAV zAuo9@M$XUi4Z6j$LSi&De4uIVAhny+!1wD@R8LPenK*{g*jd%TUK+(yADf!u2Da+a zsvuVpt~nWOw+1vDO$evyFzO*Wdw=Nn*D~SUTq~u)Cu%{-pq7;Vyk9N)d7r6JRo`DD z)8H9f3BsV_kC=?{ZTzQKM26oE)+f^)FAg${t!R3NVJuMdT}!`0iAH^C zq*O`q^Rl~Mb9M&4r8FLZ%`yo4B5m+Gc2>Evwu^mq9Y(=ocWSYLxbgNpp{J|3f91Zm zRK1$BeYW{-b@*2$o{w|Z}K1^59O2G>#v`OA& zc3NpZ>TNS|(@a5wv;6HzywFpRHw1CmJ(vA(HluTQdtAgfYVLyyPquoNQf-Wk+0Dc4 zdJI^c_a8nWh6S$CQ~4&66I0JlgLEDqD8LXJEux#^mM^nPVZXAX6U(+h^HwA2Ve(Z|nY&x%asr%uxL-qM!0;g&1KgX}G-TUXs zS!Jm(N*A-UG+ak9g*AZ63{3L^2W(($`__UG8f?u9u(>;leT;wO>gurn0nAZA%l^v<5=fi z2DKeqJKjhH)3o8BM4nEkv$0f5(K6sgQ>xqZN<0heAFXX)2`4JztlgOx621F$SRO%C zTs&>#1!I?*>yvtpNR(m$tWYjixbEkuvQB>`xz*OO!{M9 z{!ds99d5j@-+z+bAjFWA(Rast^m+JF<*6j^Vl6&0aMKQ(6Eyqn6DZD&y?w0I7_dbp z6t0E_Qu>@j6GSg3HT}QSwn$2DfS#()H((e^a0&dElXfZHHLzS0(?O+^S=jd$6{idQ zuC~Bo5`}d`=+GlLU89w0T$qv^q6Cpfq#{}uj=v&^Prj?(oj+EG5&QUR$COPftX&zT z({{n`f^iHJt6EGkK??jVB`w%olkxbhsWw{sR>dy6qgA|L_&f@ZSorkW&8q|MApnj;ZQr5u%OiPYa62rX2HeAYblqf9PbdLo$(Y}Uw*kGjR@t#IU*`U+au_Q{7rz}iOF6+H zc;*IOy7or>{nSQ=-lo(H-1h3%7X~x=4L(L^#xx&={wX!{iv}t0_IeoJ@&1y{07yb? zrPk50ksQw=IHvPo3NT{(@R^3)*(-RmL+7h~b?jtWUZGFhE^Le0&vswpcVQUzU!OFH zdT*G$U>3rB>7GwMPzC}SL(^!X%8O~K82>6nU8iW-?L+ADMFSC43$rg9mppgL#-eL% zVW)vK{H3Dz`i&fHlJ_|(vhp<1AZ_kQ>Q8-}u6WEMx@bT5UG}9F;`lZn%a#N~Fu|-? z7E!?4)8EN-&o;|U7+LD1M1OOnr`p?#Y-ba#H)>As-uHoxg1_@(hEm$=g~c$eKLb3T z%L!~QTCYFF%1A&EG#Hp`^Njx95zDWs5uau%7WNbkO*nJDiT(!DJZ0eKN8ilZ^cK&= zvDh%nZfut;;s_t`sQ zPBsv~m-xp=P4YQ~Zk9+?bB$LT^0@}fP;#OzX6&0Hp}}1xGj|HQr}ZGVtX#$DB3^C!pNqX~pkjy(SA~>B2n8lcQrmAl^v_ zR6VMzD|?6~^4|O?1IYZ-riG2fG_QfyvJF1$epbUfJ>bm8Q)`{cdZ}L+?-d%H)(;=H z><=A=Fn#SE+L)Lp{}LxKTGLn@-~^mmpc^G(N(OYcV)g{#^^2w1{y}!y^?QCeO8oJKyLIl7yXz8MkxYFY-DbebDD=Z5miM7WCcJDlj@Xnd=svr@}Cg(-*5H z1GsQ&yWAPLOKDm5XG@pLL$f?=n9)4T2#qa1N#j4H&t$0#6kCQtEW_<`Xk0HrQ!hBL z%NAGj_pT8(UqKzDDb;cS*$XVW9Fs)8XjhPSxmD0Bg;jIabEABT{$ap30j*wi)EbDG zF_}=#NS&MolbQe7_|m=68zQzahtzb#q1EuMO>tP7`IL4MqHkKU_kB*~BT9oRkaNOn zY1a+d5cc92EX*XuybTWqU-uBap=EHUk0C1CzYV&Dh{b*un`T2nJTM>uoOaMI>C)Z? zLhJnY%@b(b0j1fERp6%DHn-hubao-f=5sM>;@IE4V4gWx^JpXR)7@b*wgB&kXYPVd zs> z@)Zyw5BPR;Y#Hd^=SX3W>R8Z~u3>zhL*;kh7Lkp{HN{7JJK}|RsOXhP7q1@j0<-MN zdw{KGDO%I`t7r|QoRRnkm5J$TCf(XMzpo`f0zDVmA(BXvo>@i@7H|TXBgY$Wljti! z4pv^bi~;TEtF^!3eTT5I39Q{2D&F={OR<9`w(x@cSs#&~1fOORqV5#dDAj)UQ+kPo zIcyNPJ~JIg)&m*7)K5QRV3xon3@2rS%S>=-5x?nOeuS?3jD!;z%j~kZ=JFXrrT6!3 zQDJ9w-9Kn8N^vtJc@huTfgZO4{oz1419-Y(Ci+{D*nwpymPA#7&PR&2`-{9LHXTjZ z)1zT8^=S_`*1NrXJuO@RVSDWJ=7BF%AM^vQ2ErMTVUE143Md6CGHG3g+d8&i=WYj_ z{4}b}p^evLyt%1&ct@$(!$M~lFX%2&s7?cu)&=+REAd*6S^FbnQQpT*F7Fl#qm9um zuaZT>oe>d}FR39r2ArFKQzB$ykZ_t{ex%axhw1qxMbA5jOFe9^v&+M;)d}z999#E= zfBpIet2|n2tWSzHrAY}NzRFxxZe2McI1IRRzWsFVr?v?jB@i6dCibboep^w)s4w@t zo)LL+;}ssRXswFK%lYd2+t*_7UoLEGS`OOL**o0Pd(fS}&Uzeu(e1pbUT&|4ppV_4k+RuPn?^YM|gWMCFN!Byw~MZ3;#)X6}Mw(!GK;We!v zg-!iWK{&$HwEf+9?<~7v{(}+&$G0@7fFSO;OGK}N zE`tje9`)w2fc~@@V<;Ymz77mYTx=)9z~6rf%i~5OJS`y8xA|QqKP)BNq+}VQ*Uza3 zasM=NY+t<@Cd+;W+U`I6!)YZCHsl5=5I$_j>~Uu0pK2P6E9D!e zvo5(`S7rOD`9$kXjMwj5^Yg>xO53D;KFfj{c?_y`2K58RB5Z9c*b&&sphPCCE!$L- zS#{YxV-({PGo3eYeRdcPtOFn4I2x^~E?ZK9MxwaQVEg|{TeGT%C-NwJ;8Xu&qHu8A{D6tH`?IoMf7&)H;5#TOLQB1H< z=iCeS+SiC4*FqHE=*j#td7{N`u~o2<6IlAo9xM4IivwuARJxA1)K5lEo}Qjow{&Vd z;{zocv6)vA47+*|qxec{;=CS%PX58l=7dYw*lX1mc>V4LfK|sZTwKACKctd3ivn@t zY}&K^c!#g8+aVUqR|xCaOUVf_VlY9P^_i)y*iHG_;HzGEXfmfIuO7Xqbao1dbwQf8 zdWOer;~JMOk&xKwVI5sq3cwC*dq`I^>a|zh^Qt;*cbYcx4Nx!E*pT_cS>5uI8ddpO z>BCI3my1xS83W0O#;TWu*TGd*4)*PFhulKmaenq#2JHbefKL>W#7l0szJGws-Qo7C z_Ic2CfxVy0kaUGPdgmEZJPGdnupa z+wzQS`*$#&w5LFj8E^fBJl^qN+^Af%0JhL;dT@AdQNnX2L);n|IDan?cNXp;6p-ud zCti2pCJjnSzspY@q-$wPkp7oqtgv!&w*{o}oB~zI9cNtV&d z?op&WDuz>6in!#9F`WNq=R|5wZQ{6cTl~_r6N^_L0nkb2;N_|uM6;hc!jzl~K@?e&8<{ZM{9bT0D4N5NY3Rw}Zw6UUE55-xWD+K^{3>4ES{uc)hdz zHz|^;C{HL~*kTtQ%i6BFVmGtalKGwY_;Lo@@>QvIK|#Ug^!&0Ucc#R!PwzE@hhh)?)=JL)=iMhda$tSyhEdbj1PayKhV zIdh3;B7!=t$h;viLftzD4ed^BhoVtlSAHm;wYR3RH8Ei+(b#Y@pPSRci;{k_0U#G~ zZkWv}x{;E-y~}@j?qA2z0C8vUDLkmmV+EuFwn$*T>W*_p>_viic4>-=hMp!S;GMnL zmb|=<);6K;O=7j@S7%J(PZ{x={iT%pK72f;ldNP$4A?iCaR-5AAHUmuFaW)$FaAt# zD}nG0=ZNmM8tfLkxOB3gp=$M^Ww({y41fCFTVZI}9RBapq=$<#mpdaESq1xlz**U^W`!tXKdK_0&>6mc;_alwCZ9c za` zGq!=}$*r>A*w{EDXHL7V1pOflF_jAGl;TeWKj^K3mWuygQ5j0c39TrPUHBa7 zM_1c*?T358fm^rWpU28RaI8oRcr?0MC>Di-U{$}Qr`BeRuM~7ak zvvoFrXi(d43((5;22cxA@m_I`c4(tpXf^OxlQ*1a+T-1W)wxe^MyR~q}srQ>FC zkj%NI3o8t7?^;_O+82G7p@**nt4s6e1TRk4W~P5FC^!c8%c7VR*6kvs$+Tcp*G%bS zu*dNLp}%=1h2-e+Kr-UqSlZrfo?eH&WGw$-7%Q<8J$p5_;;312R#+SZ?^zjgdi2_~ zd4ChNZ((MDS@m3~GS(S`yhre+$qeRojOvqW+TtWJ!bN1mBGa2mcHE=+P9d;=CV(Q` zB1H{z1Ym^X*h~NHy!xdZ6O)xY{N|^~k6rb;-kxX`>nv@s*ENF%y^>o6Njf2cm#OpZ z_jy!><9-Lb4cT{(Pw86S_U%aq-dWbT2#0A--{V7|+59vT@xxJqsL(DNo?%vSC~m)> zFl0S24(!-DiLQ9W1vbbWEf&%xS{Xu1DF}qOKY>HMGc33se&oEk0sE~Rv+sQ~%?4pF zRIJ0YC!xyA%Phzpd@V1+`ffQKM!75{slk?*+_|q{-h*{O&7DXE@KeX5XKTj=LCW#% z+97h_qd;a)a}fdrl=Ywt7~v02lrB*}G)$-N$Cv0{2{&hC%63;FFDx6_^zf-03?pG$ zBP?!@?Rf5O1aSv3+O*h9VbhO@z-;2!(%-syXIllmI1pRf8+L4eh=a`FMcavcy&y!R z`I9?vqHW;lx42I=q)E3j4W3&)!egP_h4s;juS@e2)3|Sc&ONn1O;lp=RoP`~JJrH^ z$--)ZofRdp@qKkQo_n*DX~q2v))UlIa5_V{$g1t`2E{6V$b z_J`!ytRr7Bk&xkt-wLKuRzv%# z*M7x#CVwb}x)J|V@1GhI8qVc&R2kAD8uWVSu>bC5BdIPnvX)7;|6&~Efe1FYUp)IY z3qY8j{@Lhq@ChczlHK61?_$JYFW$xW53B_YEy#SY#VX@G!Gsxt!xQBpkW2p8G#_=O zM?2l{rl~q>i^luL$>pwuBDCm5fgrQL%ivEcI`+dTkdtl)!JIc$s3d!St8WF*8RtA>sGF z6?wSK5gnE5Oy~~};#M;};o>uNMOM3x|1v=sFtPmxLhB}n`7;`n`=-$qq_v^OH0u4R zMmE#7nnUaOwZ#(B!5+fx1FosoXIvCqM6 zJ1IcyO#(EIqM%k|TTp>wHjCa2k0a(d8hgy>BMp42>Wg$2$FjV4YZjhDx+yu+MX959 z6)5boc#4HDc|A7|MR>DgkVWilXhZsrOn55FE>B?4B)In(oVb%HQ{6Iiep-mLd<`DT z04V{QnH)Ntu%&-XcA_)w>je|&cy6O;l8y*tt4z#b)-v=QiAVpQc_xHdr_DDJt5P$$ z<4`8YrleJgqILCII}&83u}~4-+9H##%T^K^$3%tAA*#!!NZAj33Jf?gZ^GiI_rp+* z_}(wJ{!rfuv2u?erj{LFko4^YIFLIZySO)UQER=(^2og+(y5Nef)!thc5r2 zR$I)O+J|T5o26Q~k&VE_8DtGG8Es8>m=F{1BOoSVvuP-fux)t|TE4KJSKFU`^ON{j z7`(zQmZ2|#r0Qoh?Q2*>PW5?02b);Xd{sFkee^mA@8({;LfVYF2S)>9E8l<@9oo?dbyZU*sD5rVVe4i>KaZ$<1qTDO%|MLBV;lUU?^G z?^0=ckwgXMYz6)XsYJJ>`(5&LAP+4Kov5?XLPg}v$*rX^P@@r(AhAI3U7MW5O>q!X zP<JyCF^ZaAJ)(b@Ulws$hmoPIFVh4XaBJTM`*eP(s9y>aKtRE!TfAfq_> zem1b_q4+w?W<><4fAohcTjb64Kvz85 z-sAmOyDUhA?xLMso2K+7Z*cYH*JTZE8Po|Z_%o#JXdgQu?fkaQ*mCIz&Q$;?Uy9#$_%(u%~9t&O;yZrUvu$c z+aoWJNbE!^9l|-UTo2u%4zJ5dHf4VWfqaSp>ewJ2ry3hnu?vVs))?*T!97P#6KS>{ zm`g^j6bcWC+-ov_v zk2DST;&18y(sOXJIjiKk+eCSaYyomelIEmXRb%P?83@vtTWd4*_8Q#@4koVi%fh@P zTXS8W?c9H%{IT@a)(fJ7h@9|0QYo^Rmc^<@l)&e~@EX<3-OWSze)n*HK3a}6SEKqg z6OH`wtA-)_0vM{w^zobcCq!Qqm+2bKWWm0nZ%0pSS1TMmb3<2W#=P}%(7ux$Fm%-k z$^=`x8uU`bb?oKa#lx2GJO1zjgLikbc}tx;>?)PHJdpN^SmxKIZO5lZ-cJoe-NRZn ztZn3#E{?2jK?F#Dyh2WDFV>s1 zlxj$74kF#G;`)AUqGiMVV>L0Zs_99dpNDD+(mnd?eAsXK&~#BT-?bW7d*W>8UVsjN z$nnB+2p-0WcN9mf)pAd+tlsBOo6qgBhB}Bfl>YX+>-4%96M1xzaU4V)`?2mHLm(Z+ zv|grrO-i~TtaG)J;p5N7~atvtDXyKI9HV8HdQ0sxO? z%V=K}-*q}`EP9$fyi;i}v#WI_N7asP#h9B)sCsr<(gG+-FFBv@*}h}s(g=B-F)aUG zkOp;~xYj0teo)IYuZ1f>MeH|j)^uG#T@>rtx_|~^VeMJa*6+j>Am1Cdbn^?qU&PPe zJvh2X&anCRg7Vu7bOFm^2B#&rJEEIMcQtZg8+e^9cSNg6GL`F*FP8=9xzaYKJD`!um$jm$eBNk#!kA^c1pz^W|)( zDo?~>x%%qle_ujMznmhk*tsv`k*Bti$a5+XE^hqSC;5#7U|;tVQ$&5Ybxz` zV!hzGLQgt$)AFUy^{y4GQO7gkR0KH9i`lA8*e?7|$rPWLhS&UsRBl$)}@c(Pd9i3sff?&II< zs91}^xwk?anG{dW$u`U=fnd=g7!*Q(o|dW;tu5kd9OKaSUf%>(xZ3Bm`>d?KCFJmWsi*|B=(6CklZJUC z8Uo-Bc|aBt2ZY)g<&dU?iXRm8=ux+95M4wvq^sPd6&(t&b+^lbYw=8d{pFM&g z+Z;_<*y7Nw6$^cw5E0AFZvikQ=l z$SID5>I9o%H~m=orM_K@`FWsWIFIitP6AAy7lUUURhO&Lqj@su8{*>c7IN{yZSWljP>x$=0t2RGHFwb?WjL zd((ow^3P8JhhLYSj$2d>3xKS*&K~^Uy?I5$On7@7vT~YSACzNC6=X9vX9#F+@ zr*9X6jBOhWUO3bCLH_f*SUsK53IH4wB{!;Y;Tg${sY-w6?%~~5JB1e&)P?3_zI^7? zpbh8qFc-{ledDHDFn{ZBfQdw9>HJi2^%X(JG+^Z_>1j3A!UqG$JWu_ps=Y=cyKs=% z+r_>uX`C~$H<$=n|90&c8^_Xrn(=wa*z0wU*id2$D$0d;ZMQ8kmtSXplcj4B*{kH;y;U%x$XksM z=Ba<&^Skg~x_$mm;Zs`ib9I;P=+2?^7;dfZd#j+q6*&h3`!v?|xK$^r-50w~96utT6FGpDPvi1~D^^3;~P{P~xu7?Nt zM=LB0+j8_k&0O(9cL6!9SW-ui2mFpg580DmJ_hgbf+xD|rCmiHkFP{%lkn5WpNMzcunA+xG_ zG&V>g;Ncxs6vq?!cyUvF?N!cIab^6yVm>El*SHN+rDgb=R~d|>UhS>FgGJzi{7Dn= zBuw3pww0RXLy4V}`pd%SRT_S)k9S8Jf`kz;|02mr-U}>ukh8szdhAB<`usRNee)LhyCu?DQ znL2}8j8l`NmuliQX=VLYvWeYvta^d;$R7Mh_uz5sxFkgQaneU*O2eAU^}_N`O=ubr zliQwr_BI<+euh<{KO&I`QG+To23m;Gn_t3Km5zpp+z;Tup_>Ma&ccp$%6uJz3EEZq z3KTsJf3EGKb%pfnUm(<%h7|8bW3YqK!oYJxRg3q#^h>HttCnY!X~0ckV6uYoQwu~S zqGvchcLo2=|L{AtN@|OSyoc0g43%yElt{5Fst6`5u78$A=hJe}f+K%oz-^P^I z2ScqJ7@^ z^3E0X+0JW`3l2=qnwymKCa@&9re*L~%z?M@{WdQ1@Ib;K_A(Vh>hQQN$7Qjx4X5#m zaEQMm5>aA8~YZW zIrr*C2o@k!oPt6J6DY-2a{~y$rvU5DWRSybu5Ov#l#op%zC&B znN+x$CBi2(X2ScN6kZ9pRdir~Lc-CW?ynH|8+-)W!4GnND4+Z}{ko8AgE7|NpQ=k~ z&1bw$=SkLFr;}3Q#EAreA~v6!c##}O>?rB{mJdzn-@q*0*D3XBLE)>MsGSef59yBw z8jVpGMy={>Ur(LAd#MTnAB>+rpx3#k9@HM6>-fm;W00+0w7&cMLyja3Fn{8pROceq zR3rz;9hM<57yx30lO84>J@-*QRF;m3L2mkNfIf>}_WKYVBJQ%V%OtI`?}m ziK~Z|PR%=n^5Dvoc^KH)=Q=FpGM_?!&VJ6FYt#@4^TYo6yAS!wj46^a(B&{hgV()( zpCEJM=4tJ&5QT&CRmE0=310cfuPpU>rE<2@Cf$5-X{v7ESCzA*eA@)Nis9k9etuV2 z{dw)t1YHYBeqAVH1QAI+cI`Oz*nfhAt% z($#HFR(-w2VQhIWi4E|5?SI~+W{mQ8TD@A0KZti&hfYQl=37rkE+E{G%k`~a3z>g- zu1&*4U}T03)owiW4U+$Kca#ML&(rhc+?_yvH{j=e*HnIuRAzYm4c@WeQ6Z`2605Wv zTb6gWB5FQr#?DjEcbIP`XF4=_m5=PpIg&|5f0ydxRsnIo?vn?F&=>ft);7@Sq3~4$ z`L}!NSeJ8tc>13E_z??h+l4Pf7j}@l#!Nvcjj`H(NK*x9@AsP2{5~1VO2u!Vr@!!! z=IKo+?~!1RuB~|B9*Ou*Uj7YX)kZ=$QSIHH#DiA#MlS2QNmbrI$OG7_Gm)dfqHGt_ z;QgcgC@HsxET`RgI;Tu^yzdlEL{0~y;p3`YLEo8F($FYyTaUd{`d3QEr?)$;QkEB% ze*@?Ybr!T!41`DL zccT;`J^ol`+S$I1=PF@3(z>lD8c()ERohhzCj0sCMI(HAIQ1xw;ba!&u?}dSM>{Hy z_;Nq*@zJ}=@#XS?V()hsisJl)Kr1>nRERB&4qZW@mmaWbf+9usZa6a;lRlJ`y$-)= zIvPBmWl1v3+E8U8JHCA5HX6*lk@FA3V4+@Q6HCrm4GgsdItXc~p66BN|rcyZcH zR3UqIsN;u82^yl=(+ZiFIP1=zME5}Cn6mq`hstk6?B89!%OmbS5zOa_fh`E8EnCrS zc6r!C+#|N#TLC%(twKskT!@#-*>cReih@FOvH$w<8s2@avb9B9wwn}+Y~$=0%q1NS zgL|^PLAJ!)pc)CXt4>=jBAS5@-hh0cW8PE|7N*>Zi-mIAq zM<|oYUk+Xn^Snx<1=WCA7K5oD%SSb7k;*&4D?ebm-afW$>p6~odiZr=wAWUjCms-> zQlG&a+tXh3OZ3Oz;=^@yl0lcVLg(EaSXI@6b`32)vC57x58( z1g4%4rm%bS4D#WGi|j3#R2|756Bc$K{g~PRjTsDDIzEbYuM;)vvlhsrN?W@(=T^yM zC!Bkr-19(grRD(o5e1*sR!S*daSHZq7VYIzl!5)2+zMp(zJ$Nu1P_^<^k?o^q->rt zN9^{AH#GhD`8|B?cbqC*m$oC37m6otQUs*j!fy=w(cUEUz2xkOeg7<3HC#=89{VqL zEjRmHUcPfeGRd76-WK_JW2Ftg!P#Obyx0CJzIi zr}c4~d0|GcN$pvhm!ly-s1t&Ujuyw&wl_3Z$4UZ%l)s;Xg5421KxX-@kz9Vz#o=6`B z#5;BD?@(qIk(+%qYL7ZYz;s72uycR*;83_I>_7KhxPht60uLawp{%^$ar*0RxNXvm zecYp+JF3nSYs+@;8)^=&a0Yd%i(+1JELcOPd6InV(G{7WLz1R6ZwtBIW`ifUo?S%x z_o5-=G=Fd%aezJu8OJJ5Q!mVb+x!0NUO|xF)3I&*&2m5`Krp7NKp*%XJ^S`^N7d5I zsdz|Dp?>O;v#>`s6?RsWX#{YIp)E71y#uv!^caY=314{2ay?AyDVTIWh*Dv9eN!(4m0N=jn(I;X8uz#?{45ld+0qls*}h z&DzA?>mR3z>T>pNKLmtTL!fY^-%PaH9M@P+-jesDFV43;#3rF_1ugOSPXr%JFF7(d zhJ`(ODsDX&|M9W1*G}#jGrW~Ux|oL+#S#k%DN&Jj#E-i_Fz7OA!WPIha`j6oSw!%E zM^=(n@nX(r1g_dF9xHu+^E#@&6c;Z(x8*~r?VLQB!a}IF{G4)jOfJc5Q)@d)2LBGc z+BG&fk7rr{OyxToW9n6a+U;=sY&)q4s&=Vk+nouX#xGueWi`N}Nk!RgX-aj$444Z$ z>nGKAyRwT8AqYp18?klVfqEweBNk)^v|f}^S$?HFuABwSgyoHGzkSOM*%>zTzHhNZ zpSzP-Q0!fu<~-pASR2~(*kX>n;Yg>?A*-R%D0RxB>`%=_mJr6YgMzv*!ZKCj^x^cp zVw>}5Zy5T&tpyQw-cW_~yC)InidI?gM*}Nz5U)JfVqLLu{!)rL=@jq}N3GwGL6tE~ zC_OL3t=VMaw9RdHd<)eL4F6%J?Ql>v{VYH`aM+OhUVr`^vcdLJ=MHn@fVraKAI-#2 z6ncqeH9`jBY`!yb?R+5kRr(Y4<$EnTAVv;iEa8GtMZvb}ym)z~=lDPZa0XY8e8P9p z0NcN$Yg+(F_PCV%pJ1sV&%+rZoxaCQ;^p-K)Ua^7-giwB3jRbzR}&n=QvCCdavhSw z4BD<@lhWbhqh)Q(69-fZ2)i`TJ814#cF^j36XmEI$n=kRnJ#S$PGovi`LRd&B0{%B z`47CZ>rv4=E#_=`dKuIS(qGm3Viq+9x69R2D~g1IewCIqVp$=r`R_Devu)vBU#v*|B;k^OUz z(VIDwsI;|{ALk|P=IQv))U^m-19zs3!@SE&mnuivrPFX z6%J#L%Bx7yFvaGxOUh-PTwomw7fo}jF?#|CwFaG%p_9Gf> zlxm8FH93aM3zMN!wgrGi9<>BeFAu*SS?g8pD^YEe`Xg-IrokB$z>lmY!~eCS_BQ*r6G(bg>Ou<3n!t025!%!f z6sO?!RRSMc-c@%6Zf~JlHEK4~0{+U&&3nKB*?u1?%-3 zhs7^OF{V?qpIR>0)s69(%LAEHPQ<*n%Rng_f!c`TTnu?!8!hs02jO-na`q&Tp}Zb| zWvKPL13}c^AX1Y4^JmHrRn|%=Ic%)!~j9f=GPfri-~#1vfOFMN=j0m znawKCAZJx)cw4fQ7CXEG&tc$fjeP`yYrfsVjioldkInin79T725K!E_x&>EG6XMt+ zwW`aP|=^00dDdd+Zw?=e~tp&MYT;aGynGEKs=f&0{;7 z`xZvz;C`jQc<5^o^M6i~@N?V*<)5?E0WFdY;c_DihV_k>4#grF?Og{pEgL$Ju03pmPY+oK0(8lYkdAj(uhl;=lL0 zR`WqQCf)Zc8Izf;-;Y;r12}|k+}!7F8ko9V+thM&R)|k+9W|j!efgwKPGGOB~6oJ320f|LJ{A;p| zpT6e_t);cB2S^gx@1KZ~of*Z^J@}rzi|9A)JPwDvRLV|xxX^ovzSSVlKsb^LpS7c@ znYT8gZl6B=Y46NRS$rzeu+J9v@p64kUJWYVs^dF<>yuKj1S&BSzKS|vTe`YJ00}BC zeEFlgrua+9MiZsmCw;w;snc)Sk#i*{uPf7gv8BD*GNZZJPZxWH~Hwdq>-xxVbV(U_>GWY<`ZzF4gpaTwnf>V}s+ZQn{u-jb>-+l`e(#P|;KAsu zSUa^@<`!fk=R~|&3?7;uRKc7!63tu{63T^#`)&BVZ%pc)wiN0fw+L(se%S}Jylvy( ze{S&mqv4;_T9X0jn9?}tH#I|0_d3>-QjruxnGvXBWI{IHv>Gs5Ztjh}zChuZ^LO4D zXe17p`(?SY(7FG?Q3r*yWHlqXFM+V3HqgW<6^I=S`2mGH`#NCMRrG$rc0e3zS2EQi zI)IVNQmJztvrt|UOodvVX?X`$Jl$4MG(;^L{uGqXiw!7|NogvW+?@(p`bfUx5_psK z5$!MN#;d zctaaBfMvd>2l^)ba@D_`;}oG_fp^|pPGXpFp3PH@LmIyciFoxKcMm#}(p)jKm+CK3 zO=j~G^LX&B((OgkApmV$8jV|Cq4QPLLh7(pRkf3XR@A9ANnKhvX0s6^c2im7u%8*9 zzDyWwxQ?W{V5yR#d5KqzCIV_qh(6ZBMrclm2my%*BUTIs2{r5de zN3?Ba;%_s^6QtUcB?~-CTUd?xT*lZe-c~y86pK*7_*Und%vEt4AahS6vFVdkA>}f! zx5MkzipQwPyANTF+W=uzRx|R!H@3S&~&P(FCZHVC*6RCYP z23zuobbaZ~nX>x<^=XI4S=Xw^O?AR>V!yBi&%(!t&0Bx8;ez4e;a#`z5C2H!@PdDa-Zo3Wb2glWs zuLwl(o1D_o4N>G5VlSUbEA7-OWes3Ah1d^O8@T~!o7Llf?t?UHl2fktn%Af0McN&wA?`M%N$t!zcqZ*dqUmT&@RAg}ZKgIjVv`hS%XC zL!zGF+P4OTGChUR&u{mZTf!KSK9~-!u>5S;1A*1ZF<2pUy23Kv$*Vs^aDcal?gLe{ zK?_v%OYrb#lwc{|Ud7&!!;R%7LB?D>yhg4|CsD-e@4yf7<1BO}WKr3U%}l&`?;Dk= zu8mS(7prA71d0D#WS%a!Iltr{6??|n4G;gIP=3F^A-ysCdAg37dFqfq@7YLIpQ5D(E5?s5 zwD##jWBjKgIIoO>+b6#bbQMgNzf$%Ph|OSjXQi=}PfEvWIOo1PWQz=L{h@oqQjx~i z3tl(u6JW-B-k!-vQEwaCGZQw?e!g=YW;dzU;>Qh@y#{aEe?l(>?g-F?_0|<`&uB1_ z2!zGFV?L3MLDFjsK0XpP+}m z5$bDaOnhLAz6DwWkrML^U?36Qw?ZXAqrQ~SiPzf=|5M?vO&@S}YM3=pjAdapL#U4_ zTsHe1#QiTmG@}T?brTEeLet|C=|bh4a~32?Te9mn{vwguV7%i{O;7#`ps1mlRw**N zion98n}knR#&n|cman%dHRgwsmd6Vk)6dF?4W11&ZaM@s-upEQnzIE=Kt|wiGClU5 z-TmedV-AbQ1(^_xgc<_?JYfiM{4)KtO&P$zkT+1C>nPZmtE>0tpdk+)XnDo-0`&ai zCZxL1k<#6*X-^C`)gWdenkp|X{`j!%+PoLbl;P9M<%s{NCg~D;y_l?m@6a5vL+H!D z^k$AqkDU~vN7R13>~D&#Vo9v3T4&I*Lo)sui9| z>0m-yiWp)$eOh)z`%#}P)2+MwQ6*${I`)akjqf?tBmL!f>RqgEh+H%dDP-p+H%9gf zqj!{y5dFTW#P-!qjp&{-p!|DZDp2sI3k2HX0pA>MnHJjRao=-v!-}kl%lFu7&DkuoK;{KZu=Jh*YTu5b^A@ul}0K2NEk zUpq^f$;iUN{hB|tJ*I8P2QZ|mX0WCCK2uijy@7+_9pFH6ePSOqboWUkoxva4i0R$J zOl`|xY?SOmn`lhB9|CF^^58`aX%XTI`B{j1S5gS~p(tX3(o#Ipz8Y}C6~ZCx?fyN;ck zG;F2f-c)27+)1^VmTNZIX0$DuKWZenZ~S_QQuzXShvB%-pUwpy?(`QGaJgRG=w5f4-Dc?NhG=0y`}(IM6pt3;UmjWhHncOuNU?mI|1)+#Xab_f1pmDZk*Ef{ z8PZ27A#uc_9^zp7xo~j7o)%Gf0#6OF{ZQE=bsfU_qxvw0ujEs7ooH;}5xqxr=8b@> zQ?t~@$CG<>wo1X5j^2J%sw`JuJjekWCV=D4M;2uLMe^H;ey;{~r@HV&tc|HHq(h}j$_jxu)chJp2kfnVn(9Qf=&6n(YJ=oK?2ayWC zQTP9ZN^0DT8FIuLIqfCiIF9z)5yh7C4c8!!xM%R*dNH~uo%NV39k3SDrFCsO(1s2e zzf1w8d$%;?f>I>}!UO?vN#MYcbVPD~}@$CE3jr>WM)h#bb;JiA#=7lmuZS$JSfviX@d z>KS_j$AR$`D&L9i+cf`8?Bm3cs?^6#5C99;zYa(Nk$rKVd2CWtw8+-0w`Oi$o(_27 z;hd~1O|QP)Z}26MFh8kK*L9{;$BgYs()-nOi-%#RXPn-iB<%AW=Jfn5pg#p^9ZnNz0v`gX67d$`)`ru!`|LYhpN}yy|5Qq+8hV!b!MJ};Vu^B zTbRa&+s-?EJ!anMYl`jj7<&z^`0aBp*T)?`vnSQ`t%u352m+M-^FQOkgRicrRuK>A zTGDRMH@ib>Bwr1(h&ZC?>+O~W4KHceu^$M#!~}<`sP`Q0>4*}!X5@I(k+}<#g@nj@fa2wNVO!W zvrz~0v@*&&1tY@VD>5MAvybHtobdEtF|nqUC4dQH}exYDN|hrn?;tMPD{argm?@}06Y zcrA}>7-T>#T1`hb()a-%=lxxTeqo1mB^u!Qg!8xnUVwK*s2A`j?IxNw5|?u$+h-qv zTyYxBL(w9=&yoAG=pKCSh?XbUZEakwb=2hKox8oeODY3T-44754EWaG;Ap;&(6+W6 z)>n_K?Ral<6t3+OH{*jTWKb3k`?&|zY71Wr=+ z&x;Aa03|y}NCLF|oAq@s8mnTMh`^xP?<_6y3mu72|2<8S%@JY@Rsc4d6tlv6li@ie zEWz+d0C7Q#AoH3$nEe)BqUA)dfrdX2IQFR zHRABKb;F(wqx1Z4D#1$Dn{3X^>QN#`9S`B*2Qpe18#US|M4;Xr zQNpwyl5P}&-~Xl8Q$AAlJ({jfHn#S)u07Xz#xG@?Y<0KNVwJ@3jZ%gL3^zPNH^Jlu zNy>KMp)lYt&I(Ol6^(#BkB)ED74=4o6y>jmgth0}>|KEKmH6a{l17~BjDPBqI)}#@ zi=t$_4S{z*&~}g8_-uE3a);U0^dy{j!O&q((r-yT7=>A0c97hN+I1`C7c^J%ows1q z-SuKM&^g)T+D>v4dg$@p$8_&$;>VdMRvkmKUzp%ru_SE4wsw;$=_qY^YvuX)hgx^X zn$u_^p%?`iA2#{&&QXcy_e{LTBP1pXxfsvWTIE7`AWKJB7W+)6n~=?p{WxP8f%4t= z&xQ0(6@e)frC91y@H`I=_cv`IOpZQTall{&TZLKz?Vd*AeDGzFSyi>gc}?yFYFVlA zWomr=l zx&Khm)s&_4MONA>8-DV#-W=@;$6H%~D63q+8nS`)wx-Ayd`|mb&-rpt-K9h!1f%fKh#<3xETG8~uIsFm0>@3Aa2>PoiATj=qm z#ScBcmSTjXaZ;50BtXlW5x1vi98)*fRr(DVS>|Zo`s1@4&E5(bgMn8<;@78NYHpTC zo183+PKs7t|I3l4)Jrcq*kE)QqW%10M}9Z9ECC@R&z z6?Y499Y64}@UD%L9MkId(r|}Hii@e<_D29MK&YDiPJ{f-(SrkB;C}4Z)YazZ*;sJK z{IGGjFmy3I8a_~Xm}}h9J3VXyuOa$tk<5yCe;92dDrN)+1uJHV%>JGtX*dwiXdK#e zy^G7JuafmTXerlQsrFE!r_!v;6~p z)9G{{Z!ZBOqX5>u_p7$$1W&&&!Y%Ui9OMI~=T$Df>&nBjEO&I)f*-^RluM-E&|VMc zV3aYxlDusHZk9R!loA1Q*Mrbj-Lt}1SCyEadf!Oz$oVR~J$XoF?!lyQy(`21wL)~7 zIP+b=%lFf-@DoM&5c1$E6(quIpkLthW#_AV&xzG=tpCEA9nEX^<%bqo>WD z)?t3f74A)|vO>s;NSkAzJiLg@S+kf?MbgTO$dlJ<;Z-kHl$Aa~*1+3Hq<8sW3>1V9 zN2B!3<|}&qVlf}=A7*E6)7nTgj=gV$$^>Sfhz%M9qG7tw5s?fYZqG)9zQ=d6Vrd~{ ztBKC~`7W`>p6O6Hc29P(delpylI1{ z`)ykFKb|pP1B`@9-ycI!mV>(WI3{-%4Z?;lvHtKA0_w?r^bLbiIVvcenwy$4JV{wG z#%EeBQ`6Xm!+_CpnN-(;LgA1IEiL4tcwBF0nmD#M>;gtR!z|AqNl@d@M*`fJ($1Mq zPo-@BH(Pq&Y>zmS%uwV*lUnlvr4>PCEhb@Bm@x#OM-ilQmZzEqv(1z_aJgp2_ji~! zEVHGwZ*R7q?hHmmo~o9{Hnt!*thA;s;5MIpZ|Bbsns@+kBRhE}erL!dS%y{KP8sIS zesOlz{bkDU#e{|hP!XPS$2sR*t%NZEsFCa@n!quhWuI9nf1b>@*HHPdb&Q-GK7nKh zG@>+qB=276y5z~VUFSn14XSTj$shdK0nEzWlE~_vF}+UKk-*hu!uvssn&*)HL$`piCU>K9VjXDNGcjFaE9YDih#!8;DuQ?I zSXHvg16q-Jw)ww*txe^5pWdlp;tcBBB#TNGqrndlNDb(jVJLw(a15MH`y()e!%Zoa zC8_o;OTvnZIcKRv2HSyNRJ`20mC(c*0?opgaZ`mg!J|4!$pKRm z9Eh?8{P{FDD(ZtZF>!CWvW23PvomUW@4(f6mgT+_p=4ixNVDlsvqv<>ov5sG|2 zPgzIW;4bI7a^RF8Zk>2ISk@0@c*wC#0@_-zIw^f0HIOw9Zg0+>gyP9&ogXDM9uxuz zO@G^v6|2EFpmwRkBI+At(5zfk`8*Bl&j{@K*ld(O6%Aik*iigzKH9GAP+djv{!X^p zh*Z5c($j%z%c!&zRdHi~bw4zifaq5B6onZSeM|gY!B0qvZu9D1pAiJ~51kPKuM*yq?}0Aco=86!at0ALIQKkVW#9gU`L(`9L7L7t`;(AQ{v?6G zRq)IQgF<*#L7RXTt5-l8PpTP#-f%0RpE0ON|cT=nm_iOO3BL#OoS6MyHe5oCMH3FZ1FPENiV{GY{W6gyN5O= zTRBQPTu(iC_?%zA6>*r`m}zS+a_<8HUGoEwr(j-asJNY;;<2B{dxa$= znNUFIa^<8(XUeev(Ej_;)D#2{N3UAd1T9It$rO^W24G*>x_5}a@{S%6IvCc}%OTdE z{ke3}Y6j>P@gr6jvyWOE&UWr1Ugv7bFr9V9NN}3!Rt7O4$00E7`Qv~ie&y&CqfI4= z@o$kTtxr0~q)Cx)CSq-3&*=A}$|nHErQ$U(O1x-+Q9VZmPN_)?LqXrrbT=Et!KZG2 zN;qyVG8nbx8fvGA&}CGMIb+7v8?$%&{oF`*0p{jt%g&9lvxmQ~WnAMyFc2z}$r&i=m^KAliv@ z^wG+Fvn>LZaNY%^?vfPNvSyKxek2eRYtD@-d9RE#m@PF|tA|B7GbM%)s5&BYw=JT- z=O{sMv1S3QG4WL9L^+dlwvL`dl5&A>tr(&V*0#ETW< z1sE-`IHeOUvI~_|vJjE}8Ym@ks(pgOz>2u!A4F}qb)wz3TGppAK@|6VZ#*7Qe*=N2=*CcM5chPBSP{%#iU7@hFW9(U|pj@AH-dx%INL=NiK$xk;%;N6enVOYUt%IXp9L6J3k9??XK>d%^P|(vP zRqu+G&LtAKr)D|Rw{Ch=nkC$?JdwRL>DnM#k1(COngOvwwAQuJYSMMlWlbp2D+{bi zct4&XR^)Ws$10OpI_t9fOT!0_l{QM9garzIXH}9u>hDW`Ske(Z&xA@UU09b{h)8NV z6-~K7fDq-<5bCYDzA}a6USgiK#^TKZz z%c=F*2UMRBP8z-r#Zs0o3pJGtzhYqcP1tlZ{G3YWur|mZPi6fOZTUKY(>E}6 zUylN%>HrR;<2-5&O~rLpIJ9qa6q<$D2pgdUyfun!2>R~~R|ik#l~bsGotkTfFkj$v zq{wGqJp9)+w_s!)6`g_8Wp{hmR~tC)Cw_ZOuSjm5sJGCXvP1(xy-?9hDVmggEH6vd z*+i!=E4NlvI#zKWOB%Hd5ef5f>8A6WkN5J8Mym*6adWa^qyd@e#Hou9*vckEB0wQp zA+kci6%yAueBO<6W_gEFp%>_`YeVv)ykC)9%_CrU#t3vi-1-39wyftB&Q=;or(P=9 zlBd!b*WJu~Kt>os@ObmI8Lc&x%>%KAw(l>Kv{iJJ<^$C5QS}${vPGJ+tq(U851Fx@ zqgh1jtcZi}f=b{ubb3`rS(!}IoPYW2e5AFP#K%k`PQRp8qL1OrN>w-51**VaRyiv4 z*eJGUT?%2xz~YBtk@S}2v`zd{)Z+DG<8J7)PzMoDBY%L205gkDMNFb&?D{VH1QIcv z{vL`H4{$W2v)QHcV40=v5NP0*9loq4UY7Pgq8sa%?83>u-h&lHT1uG`TCJ9 zijW&>dK14u2U^^aawIcC*l#Y-$$vwxQh4Hr1S=oRqG_cDawg}|U1x8q@&;>%WjKAY zVwzOh_=A*C)- zgB6)8A$G=It0`UjfQ={){NT7ca;-CNCivn1UBnb7c!s5+VsGy!Gq&Wgxui`2T3S z#(+B9w;h*lW7$}?xoq2~TDDsY%R1S%%~i{`w`?2l>9_ye$Ld2p=fQnn_k~}J&fA!$ z97Vxwozzm+0$xV`n*!|yXRo!46?ysU%5ozJmGFI7(3PXmd`4L6%1L(AT4U`J#Tm^b z4Fx-U$rkGq0JKJ)>2eIXYqM~uC+ClwgPaT$)K#p-U{Ei_EAPqM70a4o0NSf-uA(Ja zpbJ{(dM8UPi9M-xK7@xi&L9bNbHLVhg!jeE4Wwl|G~R}olg)vF6{BcsC?R_qLoeRy zillbi4pmh3ELwbt6+@6B$+u-K(W{Nr73HS1#eqO@)@|t`{dAXPZEYF|Vtw-Bziuy~ z5dZg@f@y)FZ$O#$y%e+Kxi{M_Ndofu8VFzyO}a#6@V=PxMK=*1!dth?V(YKgCc(p+ zYd7h-joR!_6nA%<87ZDWEpfchR_IDb>H$Kb>pG)yTu=dppI9~PH*H^i z8q%FfV#ea#4j#YuJ*Z;1k{VajCYEzW5v(V?6adL8W)6)^PXjpPjGUOB$0`%=wR0Ax z^%QP(N>N2VVQWT!x)#s6%6Dzc&yRy&u6dN&PVp>@?)-;|fpWFoQD@iJ0dHc78-ySB z;K*Squ>wYJe#a4Q2Ud0JopehUzl%y5u$snx$PbU1r-{+zChpR4aNitlWD8#R#~6BC zes^Ew`qG_+%~}4%>fdvpn-=g-$WG~v3biuxIH<*O7rdQ(oQ~_aKx~qxQ>qEoW$E4v70$$) zQmQ-Ham{5?V+gNdKst_RFYB975m znUNq1{*b+Zo5wpy`bhv?JbR;JgQ?XuN!2GB-SDM1(O(S_L2 zwKVBBjPjnZ)|j8a+qqGb8gB z>F|tfWF1cvuu3Nw>!KeC7PQ>_glbz<$c1_2>I(&WZ@MhLN`c(vTcKEV08;5x+sYy& zdXIUm3jL8d7`;WofFpPWMN`wy=Jggr`I|3%HdZ%+6fF>tC};mB3SbDbl)sFUy~N$! z^#B_g1@acnJOw&(z5P6u5U#x9O>|41<}#&0m@||^u7yl=x=SP9LT$gByF3-gY?A1= z+UrUUQTo?t>B^p*IUj!TAiras1}(TfVBFd*j?RW5a0@#cbzf24F|Qg;C%iKb5f43d zl_C;T>1>=aH&^m^5tonJHjxnW3t$m*Hz*@9tNH%jlyeyhJL=dP7+hR#s9(J7a!4T2 zEO!mh34Xof*Z;jJgmO2BT(LBZDO=sYa@$F``_B1z+N{Ebsm)q)uPXdb>!U5>ru@%n&tcbV90Y*-DH5q-HnQ^=Se!5zprLE!B>$#RrHO#@F^TbC){LrUgv zc};-!U|q@70jU32hBXJ237vZ5(S6%Udkf`rOVTycYGOzV%Ny$YT&*f%UrT!D{SHIx zxbS!kDvhywzvt^JU$_G1KOysyu7PG<^WQNaH0$v-9Esl29ZzzC9B5b%r4cHH6KfFU(7|n0|Ej3W9JndKOfF4^+^n3)a*3#VsUp3zE849Tb+3*Jd@S z?49nyD&E!$f0XBxM1)M!Tr&PUV$MQ%U>bas17V~>DXbnSpveE0P13$ocl?Jj?m*WH zZM#OKu`(D(d}&Lu$2l}~Sp5@~ckUuy`ds(k%4L5vButV2TJ-6}QWIgS7Rfa|{@Bh3 zE4-3=skTfk19?r>1Q6!giL<(Kf7oP*Fys5~{^!!2W2*N-;L3DL^3NPDG4IW~WwzXF z0^SvamI@Z5IgZ{THiRrD%>gzMyEJ*EMsC+MEsm7b=zx-Q=>vM}<6h45X^I!m)-Mpt z6Tu1xg!O~Mem{NXg^#gsAplsf<;rIf*2o3`n2UuhM~oIzp+NW|gmlXlWx-nrauEm3ohi zqz%aiM!mG!?TPWhP)bQ$DLKonIg;?uM8$o_QmGS9e)p9fMXOF9r(-I z>BXT;#hy1Rwc=_m(``?_N*Itd_Dx-QxoMYnC7e86?dfso{WNc)TuLXU z;k^tqFAF>Tqf3c;#5kXi?K#tig5rK*v0?(LA77L}alf#H5haysJt6_;zN%B8F?q4? z*q|+-?XBoir3nO?$m?!zyF5wOiN9}#ySe9hc4Hhs?PW+H5)YH}rn5h4IAdXRMkr0y#EmmS_OB4ri`dqr(RKkVMX9 zP#ps#Y|8>5xExbU*TtYG366*7F+EP_L_&}9sXk%pNf9cI5R0orr-oAouN0C(f0u1r znkJX^W%l;3`G0LJTCV52=COVKXL(*?#WdIUq(+-odwiP1y{6rx9xKV!CY3z1=o1EA z=t9RN)Ap)UuR|wf;Q5K~5y3bkV3p~l;|T*ayULf?N=jL;1rwXM)BWN&AGmUaaH55U zy9AP3no->!50(k8J=uy>_w^y{wMHY-v0IhV5@OG-(H&kPErg695T+i4cEshyS1;5&yyzOWrY`ngt@%+BX6+-q_r#3iLO{`RFFR`)4YVawPpkIM* zOuK*L^5>CdnQ+b2biaK)seiujeu+2NmKgGh40S}%A;z~NDka8s$oO3r>y7 zUM{jF(!yDt$|ceV6>Ev!-94l%lgh*`5dD@^+RGKc!w51Ae~_N*V%VS_`LL80A*$ifdH(h{yeiP*CK zHim}6+A?d+jsLx0(4-qg?qgMo{7F;(K`QFM(KQ$YK1kG9ZpKyxa*K|W6_Ar=9qDa+ZV8zMwV+eR{%bMDteF=zL7v-dxmoDewYIIlujbjX1| z>v_wMF}7UI%Wa~B`YZEt6!PZMLf#AV@W^@KJL-_}8w7>8Hd=aQWVuQN&xX=V^FOw{ zXYu&dE_Z}^zflQc%Y)`)H{R_RCaZeUuN^NNMtWpIZKt89LQhu*R!sDvOl{^P355~y z5T5K2s)*9cs8tNnX>!36$=5$!Cvp&zb;Ara7#Ydtr_q!Z1+X;z$~Ng^O+waOVboK+MK`I40$jB@^6n?3v2&-wXu zT+&yHD~)4rrrrD3Hg2pV zVI4v>AfZ+ebGSIfKNl4v*3!F{#k}@*U21dzI%4YeO{q58DP!atwl-HsajanDdOa1N z5-VifB*agcaEEOQ+Q3?bdq>Xt{fnma*Llj6E_-J=xf{lYfNxs?)LTQVQYG49P*g&9 zTW`xmF4y8hVQ5V4Gj~k$!*8XI=@Hq2r&Aa)Df{B-q33?((zbjQjE1zGi@vYpYoGGD z-GsWis)~BfkJD^}E!s^{f-dj9rK-iY2=GVvFY2=elN$;cKww^;T4Hl>-IkTP-;=!v zow6f9-zihHl*uTG-OU-;6vHicE7JH>h;=4;G#?IlE>{fu51{G6d^sBLn378F_mhP09VLMIp*$i z_MAs<8lHNaLc(C0qa5>~jie%sS&bcZ>33$Y*6$Bwr%m_d;epeoZ4Puy<&nD7Y#qdj z(imZdf@X{XO!Lcbp3+U+8k@+LIg|xWfXEVYa*$8XICEFYC!i6sS9R-)>G-Ml_^$7N zz~dhJ#kSmuU^}c{Q5~P2Wf9@C zRdsTpyW#S&>z@Ibj@2XQLm@|fL23lfWpiyNZtu)CCritjuT%+o98mw$y4Ru`sIBX{ zunmXh3mE(ztEy^rp=p+HUKXP((61Dutdu;))Q+apO}0hY*6AbcYZHZ&JIFrwdh13Z z+ISub&8&Wf5=AT?3VeGcQsKr%rg1!mOU?R=X}G-noG?j@$J;O&wTC&*ZPMGQMpk0p z8$}Ls(NDCe*}3{$7^|85#UhCGi%uW<2JsT{A~0iH^2(SQ0W?Q{luF6e4Dsi9_J12C z)bo@kD!!Pco1+L zib98bn7vTw&<1Wq--PM4g}q_4Qm=MeM9yR6yxUuoNEU_*+`}|^F#dsRU@DMtnrj67&RSX#Mm8;ftN6qiE_zGlLGe#oj8Ur zx8-uljorZ@11B?-^LN?vYoRCX<|vlnFp1n?Pj)YL*cA)XVsU76jc=(V1K3h2^&kxu zIx;sn9SANh(72CJKiZAUEpSI87x`E`#}2P?b20PQIGQMU#2S`Ms7pJl@N!1_7i_G* z#xPY!N&x+BbvM+!2?Ty6^DAA={eJ7d+o(qy<;t5a7}6JYKiIrF{*47uSe`EK()EFY zRn^A+R3;k7y!p=x#k4wfQDHDI5QrR`CY6I+S0tL&lR38M*}h=Le^Lazl5i_JB&&Uc zyH7Obw#?P!!MO3mC>ISjXn6TrArcM$f3X5;AvL!dFU%}KYN2tnyoeRIIDWmUN4>1b zbY-9(B?P6qiyfAI;tFl-4EUi=w8jioqwljKu@1hq_fb$}wWJ(e0LB&wHuUusdOqn1 zTxXUYrHj*$VCPR+Y}1+&#dKMwYXe=_ZF)Aap>nU@>ZOHQYv{LarE1*-F`E_fo!@pf z^`&$E>*agC%~&)6uJO=EHFRO(17yZZ^ zr?S6|MGIjZdb zK-QArG&a~gSV(NyMD-a9anMCFy$Gzob`10ArTM#{P3FTdjPrYgV;7OYRu4yYIP|FX zxwb86kTdnw3UkyZR~LZ8qlepP%kcRaRTzxBK9LUCiBxqmC!H%S|IWWu%twEp`%h=o z(-2Y=Qe?$nN-JTn&mbBo7B;`t_UUWzXZsoh8hETmw}Kh+k^&d~p%Xt{icY4aFckAU zM54F5Z2gxsP%F+j-HZxGIO}+J;V*E1vfy7=z#w~7&X5||7Q_;)7(~y)t)NV!&(<7? zlxxJZ^=Sl#vQ3Wy(J~2p9`vZi0+TqVx|B-o3pi z*|y;dq$cgQU8PLoCTX185}O(#b%xa{#)uItaLqI^yR%X)H<`k0@0U%cmWv3`ki&U{ zHnfBqEL9jZkg(cHs_tm{3k-vtp=+~(5LX{e zA#E`$qMQ#eS9o}}Te@(@^cn9TYYL1I5@o}B93x-vr*S=wiPiC`I?>|~XN50XZejfr zx}&1g<2ZywW2=1=wu?lK(2BoClr?96l&|-FkO~#EeQ!+jKTG*Zg9(5BEEZGf|8y|) zslNJberJdA`k5SZw?k>-M?)!U!odIi9TclGQ)M0FpGW>5k zY4oXa_aNckL1WrOUX{A_i@1Gl=o%WIl0Dsu1bkwoz4E?{PQK2_bdJR5Am$lRW4JFR z6+`H^2b6kqv!b~M17|P^1HC&V^^*l%sgoo;W8DNg!^&*k>?-PhmWjB^5?suUG%eeX zasESoZRd`jB|tr7)WWtVy1eJs@2o)|iwjmP!hfb|bYVW6Zua{_?a@QqhlNY-LX)tl zHY|UI%MgZ(YV`4YV5tIMErkAyOPzdCxVz&m*G!Q6@+YcE+g#mHbBymZblB)d%4oh0 z_H~{7qml6bc$=~e8#AH$f$y2r`_rGuNk0&Wz^HcH|JvIP@V*nB{c09TL0zqP_-YSJ zE7_1QIqW34u0m=!Q%5S>o5w+l&}-xibo=NPWKUn0>noR$9>!K zMCgfyyRy1EYz-e_1GEHvGXPXSU;7hu?iNnhd~lv2`3VhBzklfFKp%o09f0zOZa-Cm zfC9+(t<~K@vauo7rjfv<ZIU3E+5*7C65<0K3zpFl%x&>dJ? zf)J{FxChxsJRi+&gTNz;{WBK*v(Me*RBo`h-%UQFJF+>*ZW6m`du{t%@HYSbMe9z< zB4(;Eh!P@SrzL-lPBeWJ0g%3lwu;Yaql}A}7%JLBi!@@?dXrx44RS-x^MF1lb~urmE|=^w+vbIDji5w7f1IbgkW8#zrSCvxgs^8q4{t+BsN1?jN5=44 zpZ#|6wnRxFrIY**B?QBUExbM0+J-dT?0Ow`0Qw(;0y-pil>!;mVIzclDfizk&mFe@ zb{z3-Kl@MIva!$no>@vZ8Lg}2X8u|bvcwuv1I3mr90-uJ;_F&n$Hu(#9Px8Q2-)|` za-(e^c0F`^_JI!(yr1#*RyEP_I$x*<1WcI+HUKFaa^dU=gA!w*B*Fp9wG1(4Kli)@ z_WAeq*gwl%rQ3}*QM`8zTUB|^TUqcsR~^4py>GhT3L)+eD=o1h>SHgH6V2wHJy?yJ z*3tVnkEEcCn&9O^g9Rj=^$W1KhbvbTbA9!D6u!fN0vk;(9ae^&ph6c(FfYj1?G#c+(nI902}&+ z>3R!Ip7F?!&s#eM8tcBHa}ydD}; zuwo#|S7QoqfT&#tc(+La=paJ>?+n>}vUXvZjmqv{^!ccY7uA1Vbgi3sq?StE`eNO> zxj+QapKadE309@9D9Cm_?FIt|>hsmqBLKw~t??pfqeqMLIS;vkP{3?uqa^TvM;Xc$KfV8j%=|F&6q&-6(rb z_BLGstLiahtSP5VvpU{#QS8*R_3mPPbHrSgj>o_(PwYF%T%S*%qo{5tWdx3%;o-_V z_N6Tn!8Cwg2ykftvX8A>Gk3&DVF#ZVLF}nlJw&WHX7K$V4A+Csf;DlJ{uE)e+LS>^ zGFbxHXt`o%|K+1gWNFe)KYCb+j`eJ{R1f8;XyWbOrKN<8>I=_p5xlb*b#_!SS zm=wH%T$Z)bcF~~oO9cu6!M;T}+kcCw$1s~b(h&Q@Mx2(qSg*NEmNKumSA_ivv^77@ zdjMs@Sl6GzcohkJGqJhinNY%M5rGSZxU#QJG|lEZbYXHsU*YIuRUFCkwMc+6I_cr1 zf2&E>G#~FQ9X$KHX&RahufZX-jEjl}Nxa^|Ht7!;H^KKAjG~@%l?+Cxgpzmo)~+OS zM7L`}xwN5wX(RX`H9L+v8J?b<^VE1N?oR0s{pBbJwBo>w@QlkUs3(q-VCwGN(h9Ed zsz*yd1Mk&eUR|2NAP;4>iflTvnX+HKOrDM-JI?Te@uQX&DTXa49j#$hPhOv7SVW=? zEQY0EckbBJA z#WMB#(f8xc3!8loT|X?)TT2-$JCEG))VGh)dG~kUZU|mnUQlz)lJgoM_n;CX9D7BC z2R&mR7gcXO4v3*8#Q1DX^Yr{|R{F(GRJiCO$RrBPsI^ucXtKedM9m8#`5Of<%E`J= zP~ZG|6|P=vx4gMAzr&l?6+dv8e6_4RsQGmhHufd}Qzy|T{ZxN`?2X)b$}_5dMsfgT zI_h?IM>qSuaz9&e80CCOjG@PUy`fGK@iQUe>G%NB((3&(ptTCFFm-W01o!6nWocO_ z6s&?@tSBx-ka#wX@L%``L#5pn`>t8yanbyPM>nx|>Z>v4-~SLDFu4i&F1kkDBw6TE zY6}Sm2M0TCq3Bmx$_RG0jPMiTk_Jo50-TGMJxArwgSNl4e?paiQ`u1TyIK?KdgxQl zMk5!L1RK>N46E+szIjz7;=lM)?}a`TDQ2J=#cxtK7*cUaKbJUe%ccJm)}IZ};(X}heUpZDLRyoQ z`lmkgLh8oIr*ru}Jo!r$w2bD!uk!*V8(*%C(4$*JQaK0=rv7q2{aS{ADa2a9d;OuNci2maruGzUk{rqFij9$rdKh~xRT>|=p1q~J;_Y1LQT)8wc zQAqXiWqt0@l{Sq~<*$0Y9f0B$uc{)EGMFa$7jP<}t0*(XAUSAAR)Ns_bqgii^H0L! zrpGFCW2N>pJ5T5*iO_s+m;A)bScO1Hmnx(6xE1*5t52sXI#N{r_grsP^tm}LcSMv zMf10(z|h6ngqhV_q6K;m>Dr|QIQsavfai#;tv3&qxV-$CZ#?B+GbV(?j&c?by%XGE zyRRlPmf&V&C7 zFm!1zY1CaO@oD4E4222vpFc(a*KO#Dj_=egV;V3e&?w3LJgF#YN-MW$(<$wDQzNBQ zQCvHqZ=z)qrPbFhONHA#fYT?F7*CHEdfCSCezN!b>tVlMWJnocp2zm>p}b$n>&hYg zx@Q4~>c#>fN0N}w<=||n2nGE_M5tvA-wI3at#9iSwD3zHQ$S@?Z|B-monb#A8BM(1 zQd14wh$q4P^?X(S$}JVFuTSRqiTH}#AsH)Nk>wA{;lQb7>eBcm6RqXEhLIKbW3zh! z-%Tk!+c3a@{Z;wdGY{C|M?|pCDd?Rf*aQuPdxSx;I0w47BhUBNH^$JXwHg|B=ldlL zp5kY(-viR{=1KO0BY_V>4tjn+3tZi2s(PXDejud|m39kt)tPlOHz|I1x_M=~vydIG zW*NL54%gT>cpW(-3&2vWRk6moy}hL_J~`A-Pg1YbOZnKNOj$A#ZEB@aO72y<$|uUD zE>|iIu^&GGYYCC}AX~2NP)Je}v0qJ2m^~KFrAGOawu&y%XlG@q<7+L!A=0PHq5K(b zl4aO}VZ!!F!6vFe=*!JqKXd2JWVWarT+7Uu_eEjKo&Qm2?zxbij$~>Rb!{R?X^J|7 zEhn{NNEQzJVQnITcq2^(QFC4}OpG{}LNoC7K0X<3APUaELBWAq%P&MV z%bCu9WLHC=BWJ%$@}5xZx>0g89C>7=xFhR{{%Xf{vTjj`;ivzs>T}CmiSKuXF8hRt z+UmpptQ}!B&i%v1eD||cd5=JI%aNz+E@`#u)Z8%6{Jp38W>zf~z~5e!`!NPjdgyBd zuM4H)#wXUb*@sNYjhC)x&hZYmno?e2oKdwTZtKgywAOd3)V_V}3cA-<-!Gg+I8Ndq z#yyqT$Fkvkm8M^YO=JftC9fXe_P&#S|3^3hOVP?t67M^)0`Y?Zxk`C)aqx|eKc~Nh zj_}!yP7v#cCU=RJR8Njamy0e}G2Rx`l>R2=`ONzEWXn95`wl#=>VExfPv^@Hu4iGq zCGyt% z7&G(LPu6`1x88h@4v+x~NGvz#4182pj+=W&W;0KL^{dzhRoz!-9K8j56!j)Y$1=xI zLyh9`X}Riujxh|nP3`8pgM6jJ52}nzhQ=vOh}*--XbDH-R87j|K4e)e2diF?(Py$P9#Cu{{ z7$|D0Tk+V8G(w?^n}p4$nC6p~@~MrDM{tN1`LbDzO;5MYzupW(a4kyU^)1YzSAg_q zR@1JCRtHr=Zfk1vK=|N93~2E1(rk~s;{BGB6QAO;Fr0We@(>)OT;;O!;q_^5K=ioy zk&Odm%@xgU>YMwEAC;nNU|?-~g8#Zar`(1AR}#UAL_z;(Gl;bCji=XIw#@#XA`Hj| znmkk1MHAUurXxW-g3hJz2T9$Z_vz+^ON$5a{6&&2q}m#_jeqShUc4n0o1m>b6i8(4 zrPMZ@t;Tz-9r8(Hy3Pc~&?ZyEeI&=!jA>>mXFOgYpWLyBqt6%pH=ol*KGp3UfUf1l z+uzm~K0%0f3RXv7P8B8;jwYFkgz6LFM8GPD#l(J=ArwIbClr?9po4aF&j7E!GPP>p z_O#S}8ld?9J4i~*NQrtF-qHO?C#w?0pHX(IrB(ZLh>h`?_9+@lNvx{m03Nml!kHQ){z}z%CsqX-?k&ZD#O8> z&gMF?n93>u-raEZBzV2wrjj2o4rEg5WDB$typx$pIC24!%=Q!=-FdkN?li7C8_dI{ zF^U{@Kib6zz+oM9-_!q$Ab9kwzNegd&^_aVnUy4mQ=m{MwF+1^5f;i4c{N~QI@{3d z^a?V`hjL7s&Bn4>tm^#o<%|6Ky7&gJa${kc5Qg3d8?Hr*+?-X;eAp%Sf?k|!>NfGz zM~P!4@_#1fJy-=10|j3j*cd4XhYZz$!R|EA7G!@9l^W^D1SaI~VTXZ$6&ae*Me0;c zi~PKSB*Mo_q4)J)$1qv%rEh#f=Gu2&5LzapzlIFF-VQgjx!rD1o3%?4;TSH2OP8pt z_-X$J4AB~DvAL)KW6Q`#TrB?_9^&x`yyhL8pC`P&_Lr%#(m-cm$VVo;xx5L*r9`GUR!U3PT$k(dqCNlVE@mrsrjEDmu3_GJb6C$@bp2-U8O4Y znx7BX{I)ZX8Zfr;zHT&7n};dfBsiJ7AKaa?3a^y{m7c#FlBHREzJuu{3gQzGIqUH`{D({<4`GMLLUhg9&EeEt{|d8oV^ z-;rfVaX`Nc`f>~#urvgMvAiG16^hI+E=qyCaJ~rmM)Bk&@yC3;cT74nd9rgU!N0$9 zMkmWsGw=WF05s!)Q|hv;EA7ps*PijO{S-}}t*LFie@PpuFh!_xxZy!w;(iP^ockPM zk!fZls_}~jJ6Y#nzw+0g$1y7?M86|}GX#$O+O8J7AImq)sLBGFj#?)BZAuJ#$55XO z$jtnWf86t>4I)(v-%7!8 z5}+#FM>pACRnxW5VqeGgBJ40AZ^uP7uc%?G6@CM=hJ#t^t>FA1dd<6cE~x1N&NK!p zc^@1n$IhO;%CL$a#O7d`Q#J7RTpzA-uVF)2IG*v5ZoE2O1aWT{Vy6B6&^a?#s!)xo z(%f*2lt#q~5Sj3UM9W}uMZ^&m&9SMtBO-)9FLFu?L9768A5%Xo6!Vm_p{T%~+tzXn3{0m&udb6=YX@D;3 z5AgTt$S>wDZYrHW7ifX@CCOq-5foF6) zw&TnP880iV4)1fTrZlGYwWe-COfCpkS?&2r)MK$wC2UD^Joi9DOYSvw!;)jv;OUQz z$KSgosO>LqKx*(}q%Dixo>waCBz?EGvnv+7D~T*i+|~Z@2m~_^AVcf_D-j|`pg3bq9G55}rS?D9G8S|JJ!`|a+ z`Zes)`v{xa;eDh{=Bc@7zwv=linhOsE9hC(+z~g{ejkO zM^*52Yu&&%SH5Xp@5#U@zkfxvWXnM*0{ndB>Z%A5l(UR-`bkM9hIj^A#jQL_MhbME zG~7u_@Zrl#zlF8vkpBU1Y^v4CO}fDFEhpek`ep`NR!U^&;%c8!gebO@NCkDgDHDJC zau)IYWNeh>7zH~hkLg%uzjWjwusHi=fx1|c@SPK!LDbdj51)|7mH%5<>F_1wostei1oXy+m$~ww_>bx#_qa-O4;^ZtFlFx2A*b)`XcamOa zINrR@4Mf_v|E%1t(8&?u$xmND@`)iXPmo{iR=A(8M@=AcMe5_srbu!;F0X$z(~)*@ zLmt=g`XBUb1Y}LZ9g`FjK(o6uTMQdAduIthbj>}mqeja_%|)CEEUfb)=P3i)AR;R+ zre=ZUyEd!|pbm=5d+L1i{ULaW8rfH!PW^MhEX(=Nk;{I=*KKx1AMK<2D9n-1Lti%( z$097qdoA?J3oQzLvqkBxMk`QCK1Y<6v!5N3vGT7`|M)(Cwtfqe=hu9Zyxqn?m>9(n zMI8MO%vh8fhdWw^+Z6}{o%aULz%Vb4ApY^Dbfa>A2Upe5?E>#Zcr6yz_iq@Vv~GS9 zT{iNlQwaJu(IF0-`F+-lZ9vyvaU6O5Qv5ZX+;uWIOs9JOycvdxvxOx^Pkj(#4=4jDr`<7(fq5nU+ zR&B=CqP13ervhp+QxZNKQ#RQgCnTc}^|Kh!a;3ucrEO&u=e1ZMXxHlrJdT|LQ(1k- zUbb_KiT%DEJZGbSAVtop_rK_wg`S_qKohb4z0N+_hF*9N|*KH$jiZrxHP)2H8=UY6h>qyV^k$ zsn;94a^^Wml!!eT?1Bg0N+%htPi4)M(sxIO@&w=IjLvr2-a7nTj8t8>T(CZU^6E73 zU49*(MJSWDv#dO0N1&HMbSXi0CJ@+uA3Fiox^7w%52(F@muw8EadF=|l)&KJ5sqy* zm(me^AoBgVwoOu{@11+CGybO6yu+KeI@&jC_s2uCh=u&N_HtmZooXno{L=Jaek)bO za{c=L{rl}Vu6b&lvYn!TSY~ohkeV4Tv2hV7(5>O$sI4ZWw1Bi;rQ#QS{2fx6&s)t^ z^j+z3Rk@N+Lz0TEZXv-?SrNIlGt*yEbk5IY;WFvp&{(PH68^#%c)Vr5{dHYDeK6vk ztxMP{!};`fEkyhjHmPaLKE$I+CbO3?>VqwnWqKKaaqo);GiyLs!)G)7P!jd9B5w!a zB^Nl?sIBI&x;tH^m!nj5&6NgA`3+NI5Z~@T;R&g>iSbNarfCyEKhS`anr>inMQ_>7 z>#J`l;D3dV zAMOjmfl&@MSMB z2sD~W@Z5=HbXf zsx^^pBXl{KmZLn+L%!*;pUMz2@uQgqZS|xnRRJQsi=;g^zQs)nQH8t7bSlVzrKAE` z`1O@?Q;xiw&!}3QJ`fh0s|@b}gk#(K(W3QV=4r8AGx(E{JQR$C2k@*|5NdY%e8pdb zJcU6U5enFZPW6MO6nC}$q6vj~Y$M;nJ==4AFJoQAxhyxMB6doH{}htt^vm8x7Ng(f z&FG^6t2%X;PtGmm-nqV#`p#Up<}`JD1ghq+Hy+au&q>VEw@eOUdTD=uk=y%w%F$)c zPDx`bJhqRD(s7#2OWs)5Itk#6ZLh^vT-*3D*VsbWZn1k~A;q?s=t!&Tv{ zl&<718hC8=fY88(R*WB74>RQ16eu%Qf8iotf25vEV$!{fr%vqVmFD}cb-LAi*gr|s z0(2FUWsA(kS!D(=dUj{aQDSBe<-WzJ_)adc<&s18T!Zud{odg+CNz ziZ-=nYUNi34Moqb`vd*`l(S`O=>NQCQxG8#9S>;-;O%fomLC;1IIu4A zPif0X?HlWfssu2U>H_0T;^JwkD@~G`u^(y8W+;<9xvU`>~MU0W#pio zB!uT*Hk*r%`n42;x{PkV56d~k2?{4dPK2Vs06O+pVj||~ z@W^zqkk4MYqJjTkK=)je#pxt!h0c6w_&Mlo3o*>w%wmDK6}aCF9a8S_SPgk=nG0bF zlkYk)5eSohswi+)RU-YlNu(a^Q~*vU__*8~NXR%Eb3tf37N-7;fmB1GO$p*uM-4L%+9TFhB{6J3M0KmEHu;z%d17xW9^nE@=x1uG*EU9ZAf>T zBp&P)yGdPC^qqFTQN6Lop7t;8VM|lPpR}zwHMUba2~kN_nNee}3Et#fE=vE0e(JlM z+fD7!-Ks~h-ZE#OKNp%|-`@_?GY%QKGN7?7vo=()f5lz>#s&|y7dGKVt-=c75X!5v z%>z2h-?%i%9+4?C2^5X&=FwrhFP@{$<7|zriru32zdl`j1JxT^K(E{FYqfVK_Icd9 zDYo1&?;?o)lf#U+$?^fZSxGQQeCJ@;@go?Sq+koMK#s|L$)wZoJ2_J0PO@%`wMwL&3 z>2L*3kX(rQoVLFFuYXcA3~(YQoXQ!dYf4*4!y`h7`iy~igir?;^0;Jhlnk&GsWz~3=#o~S;L-wvEvB}1CYN`5 zhiUIsHO92A;SnZzXNPB%_dRk_W<05PmU$+WEL&Dy3=8oWm7STCzHkvwBT9aOK%-`! z1_eK00mtvNYgK;P9rS7zL5P38iG?CoLw8EesJzIDQQifw!WPvTW1G+1`@R3H-I@VK zgZJ-31EVJ_cbJ*79hXp_vF#4kr@R_uKRTGMv}sE+lbTnt<<;XL9xK6~pacewLr2Ou zNga9uDD{hW9*(O!al_s$?K>`iX4eq9)AvSTebhRh*6;709c*q|-2y&TfK4YNA&tI$ zug`GJ!}sKTj+vx8%DPtdEKPrb+-JN@C8rv-m7rp$Ea;0vB`CetuI&4+ZQ_x4MVc6M zrX5*rmLds!DYV z^go88t2sC70ZY%8Q6b7nl$}E}r(bwn-JEzc$N6*|HzU0fo$sy|j~!WQi8&PIhWFIL zP+6f79GJ27!uAA|un(Vq9HJZRSxmS92NVHu#>h;Z(F4U(Zu{11|UJ%2x?29XZ*L+|bs8WObv%VvkV zUr*kKx=d)XXa*owO}dO`EcG0q;9Ae=&K6H}N=&fpFzj(*BZAyCUF3;U3|7A>GYJ)K z3bkX|p2!AWKvaikw&u8@Vbl8GY4gN$9kM7kB1^puEmj);$JJLx zMb*7;qX;tAtfcZYSt@aSRtrLdUPRihI2BfgpZbWO99)BajMK>1xh5llrSie z&{fNNmAiBAa&*RTPjU2X=}&P+B|)&I3X^8KfY0w@_B*e&wf&bLfnq# zp{s?aUnpd_vuD##w?^EF9Dc>XdpWVKB))vF6mM&MXkbeHP~6uCjH3k1u%4wqkv%E9 zzWjK=!IGxgtX&FZ5ee7W28Ls~jIth?MyM()ffB=nO5k83C-7sY^^C9xND3$vO$Jd_ zr}}>U>~XK|?4W*%a)$7FXmeyj5RHeW?Ir8U!J5y)iEr%rAH0;^nqVr{a3M$RKXiIeJ+C&sF%KkNF8NcXD) zmo?jWC6a4~Sml%XpFWT5Li^#t`+HMO7V5u~)0|PF_qyp%f4r}G3YIqr>oW2=n=Js* zRdX*2P7S8Xg~ubX)k0elC&e zV^w4Vx6aSnGh~y?VsV}*N{7-!PZWk?p@p}ulyeQfXDc~r8eN$PLE+CUNCxUcz}7J+)(wZD+H~zO;RisH>6Yisq(HYOAnh z!LJV+wWpFB7#uWz9(X!}o7Hlk|CPdiw9K2Nvx=LW`>ZuJ@OIo7rlg`3QT>s2i#l(7 z@R(KRnU7G|xGXX+J3f>QKR<)_xB-2GaU8e%#whvgOu=>FSSVqn9dW(MBYH#U^GIfE zzF`kyP4u#KhCI+cS3Ns}Y;hLNe<<4vQ9w|jMPcM&V$4vHrGm7J5UZ}oq+qlJRH+I6 z;PJSzR^czAmCe?U#E+6z41y(?=<)PaW`f@_JYfsE%KlWX?ayKQ$toRfIAPK4+2eBA z6xPGQFL`K&-4HR(P%&co@aKM6RN1Oc7OrxT*iZBaE9JN%vXI5?ZAOwJO`2{uk(`no zH#AaS>B#&Gzchi;r?H$R=oM5VTLt|kwugrcKG}y;zVx@f*$FEzX+N+qLb!yI1{mmU z1?y@;@r;}$4%aRQ2mwp0A6L8c7=2Z_JVsx}e)q6;OL?S>zbIwWdllY$&#hyLr! zYhwfa*-C1Ds_-vgRBKDga_iu#x(;lg;l#bacoR-!%cGXKKoarsTvqa~Fn%&!MGTRP zpT-VLe>A#9(w#0}=*b)C=3DkG1f|~Z7nisGmW(=i>okEzqan&ovZFqQIq;`(+;2Ei zLUgp7VdtG=EiHzv;;CY-{2sk{H_%lps*g5_2Eu}sl|i@7;pcis`lk;3<15AWRc&i5 z!|>m@0NOBW$<6zl^_y|x2UXYR2+Tvl1Y(izme#M6{WJr44h!IQe=_Cjg3rOCF*+W* znAbrTx}W=Ph+plV`N#oX&Ob)g*#=o2=>a3-hl1eNhv<6v1g>!@H)2L*^Vz$xgK9#K zw3$Rut~#uI|K-D0WBG$1h0NyNmzAuXSiuXyqCnANv;ClxMzMyXWEi;t!!)X8U%1cL zAjrBK9rrtz^kLG;BKi@#7cE`3hO(-rbqQU;@m*a^sA$VV8EoGu%|v5{P@H)=#Yq`U z?e1pak{F}%7}sk#qmB0Okx3QTJ8#qTu56Lt8;zNoHkc}JA>UzYKado>ye{G&EnPAR zetB)Ty|#nxPbA%otW$F6n6pz*RibJ`xA5X0)2A4*Sle_$?D}T(d`47|OFereDf1@( zkfW{un#4Wj%0m23zTt}z1z&m$i$b|mLQPm^Ff|n>x!3*TlHwLO|Eha`0k&BK*M@-C zLEg@VzoVV2Uo`g?bN18LgK_)9lO(Js=8R%_4Mzf6Jh)Mm$;#5nL)ep=Le%+RlYaej zO7^`VuxO=XDPlwD$m^zVGUntZA)3JLF0AY42cNQM%VYh{>Sw3TeY#vkel5WAI%xp17o8qv<*dw498T&*H zRw7627M=HT?*LZC!}rd;+cO7fLfm06(f&7>*E!jwWc8KHg`;#xNQh6a1x~WIzds4c zx4HO@_cy6uOvz=ig&`(_-UK4TSP<7Y*`=w33}{A4OTh0N72(^;WzTJ~B!}TU^iE5S zucG${)(=;oMOMtdm`v3t9 znG-CN&oivTwgC{p2yBp`mqzTIlj!W=mq}J;n?_9^MH0?x`tK0aRfOwyI?~D~aVBx8 zd`~^+v{ihmHtRX_{#eirKwg=ojO@4XkcSenk+q`uB?&B-XI=P?d$s~#qhBdBQPuU_ zNA%Lh^a(LB>n)N_0)g%KX~X-BRYv}|>mB)a{h5-B&y_y}Zu<{`QB-H>S3!4oS0$N( zfJ(WZaJluuA>T!63q{qK=UwcoI08N7VvX?hM^N7%CQhI@e4T)bo-~Y&bX|hpso58b zZ?AoxR)o5eKyvC63M+~iJxEs30ehyql~D=a4~_ktNuNA;I{0bq(%+L;A^E zU=K>0@8=z^HG>u8fxuHZ;HOjgV=W-17m?P|1ot8=(*Mrxdc?FX8f3yKR$=+3!hm2N z9ASmw!Z|D9>ILrk{oDN7gN;w&?NbB;y_5m^{>YfUgH+>*bOEvQ9bQqM2{lHefz^G* zbz95JTN;GxXL@v+2NWj=0z+$OzHd3FH~8DwE>>FVWIy~ z7K(b4V0|46-{3CH`NP?JO-6-v=c?Yd8K|r(ON?w}%v!Z?8R#6U#bP)f;QA1r5K?lV zy&oQp?*p98DFElk1Yk?)Ii?RUG53kHzK;S?LkDd^y^#M3DZOuv&?Gs|m?Jktv2+YUX$o=2 z`-2=nWg?^Iz@72QX_6f81Gm$PRUG%U`eLvf;z!IfpkQDve}93PMCg`o<@LiL?TAql zC15U+NRGqo1Yy#jCA_uQMt_aA^u#+Z2MxLbyCYiSk=X-ESKA_!Xa zCr5(lR+2AImej!}Ar8f-z#BLb>IZjc*u4Z6U-odES1i{%`Bi8|?6AM%uRe8osj;`KrhafBn8$B7Xs?hC0L>Cs5?MAReycXE z;3PB{HUCsEZ;c$9nUn;$e>2u)-|Oz?rywa`pDEA*6o;(9!@=>TYn3W7T(*_CVQ=o(34;z++7?5n%3wRM}VwZ#?UL;}vs5=tMQ*gCUc zFhh7Ay-j!4b8hbAwsmv%Ztw#4tMNy1PHzg0K`D2mwTnAzyXVew3oc(mPqHmk0;1AU zP64U|7$rd6XDXv|^(T5B{Sjm-5I9+_M(omR0jmcMqg?$R&R< zfozLH%PJ>JbHq4(Z_b%4pg8;mfLd6#g2fEaFz<|RCw?WH`vO@0@_W&HdG+Lw1dg#* z9=V+svJyK>y}Xgb%4W~x!uIy8cklWVc7wkp@V(Y?Q5^g*9I2>@gsZi?wc@9hB~sF{ zXDP4%FJjugkSQ_K!cWk~1|fW;)2OA29RN9kUjjJ{-^hDu(?LEF4DOR@v|`@E#SrR7 zju{g`O)%yhSG{TydL}gz3sgKfpD+dEGbn&ys*$ct=&W;#f;{@Xc>kE|vUN(y^6~_> z@%S*6voq{jJD*{#@-}ji?Yx+q5zoa^4KZxJcWNn z)5huO7NJu*Jwy1qHru-WLL2J!jh_m+D~F~bp39d{moZBY*Fu%5-Y$EyO4knwn_T`i0LuQDB4A01m~+yvkR|5P`dM9Dp3}vsP|d%CwuIaqhyayu2BE zIWvx{rMEaAie8A5te1H?=zNWLMu>Fd3eRkBe;~Gjdvex5pMTKjiD(9SiwJ<|inC8Ep017p+yRj!Kf0d=;l~;m! z))8d3Oj3}pDN*PW03jHBrgyjuBAdm`OO^8w4=e*kCFBfb@!?mVWl&dp;(i*k zC|FhtKt9pRkBwdRnkEr*(B+dr-=T^PnK!yVZW~u3qoB}`NA_8+Ts2|vul^A)2j|AW zQC>pz!x3LE9*8MrkWBW{V8vQp((kQv*tA<~GgOCrTw3{JA$%N`Q$fevEl*he8Co&q z#y*{;WXv)9Y_Ak~jJ+d%MYQhFJl3nPhH<@g%uyuf0i}uMr-eBtM-C-d;#_W2D{XZc ztS~;5a4#9OLpOT1ET`klFH`sbf~p2G1Un~Cwpdqd2<>Z5Fh z9#F`7bf5vF%TWe9Sb@aAW+Kp>8-dH)s55zioK~Ob(4ww2EI`oG2^exAD=5Dc0<%7g zGr~2a{HVzUAuyA)w~y$J2ijRcWrj4`oNTy z6bTtZXQ!N~F!1_0J2mtSUtL)_kl7o6)8IwX{^K5e zR8ABimt*ct4x~=9Z6zRfRuE7}=>+4+8`H6HdjI;}Ng;C6Ltk$g_#ba(hv%^nL}YXgH=j!2?LyN9Bc@jra%3n{=2=uW8}ik4Z&j?i{bOFGL$lPZ z4Nvx{iAMOyEa}A3QoqV|!((}+lCpNhk@y3R{^xk_%38-Z?gi%tCgZz{{hQ0{tJQua zQ+sz$Z}H^NYPl}bQGl`KdroTCdZajScxk|?fR{cDaQx0mZOKHs9U`y2cWpu?{t*6Y z$sD#PLGyo*=UtXkg@A9nG7(SojSO7vj(=cx3(H)bf%fT%oU&TqrfHGKSnl5j6JFP5 z*!zvLUAH1sOo1=FvQ;T@O?2wZy6EU4>MW~4BIGy*0i!8cKGbCw(ZKvCPpp_dRC@Me z3WYz?_L>_Jnz<1wX{}i(2quv9m%Lu?CEiB4cT!puJ1qKEjncYlcPk3P`tMo~LUW)# zHaIpM&$U-;I3}?A&l&WT4TvEG>C(J8CSQklzscl3MO0&K{HM2ppi@}+b@*3Fu0^9T z*nMfWu{uXN!x^YRfn-v1OKRAQE8~GV-@PeDR7q2w4ph*I{h*YRF~vcO``@2M>R0}- zTP+E_2@G0~j#VMO-#Zx>JFEB6qA?jdZe;eu{&J*49r5`_5oHZW?X4}UehFoo@X{*{ zZEIlk^A^~+B1*N`Ui;Mij*h!#~ ziuJepsZO&Us#dU5X6iFS*_TEs>jMUnioJ-_r(U0p-Us~nftXL1US1sV;K4F~LDA96 zX}bLsAlpp$^t?;XCDlB0ukSSkT{fW;71SI^$(e|&jD4@EpzX0mM05Ky>{iSS*1|G1 zQAra$eTGnhI#QSQdruQjfPPu2mkh9TD=C6Sm~$+5(`NL=JQ-9ISmSjWSe0iIq=LD0 zr_#<>JbR{KRP^nI(0pu@$6`Bx0Vu4eVjzg(;hCYAdB-#}Ck+yEWQ1Fsy)l&TWlb4@ zHlCi*aX~iZom?~&Q9B(J7{pyKKds#qpOKV*uXk8x&3dcw`6@IbE++bR_heH>CNQV| z?X?f@B0;j!h^*k0`O}= z-4h|vNt?wLo}my_bom2bR<-|D+)cfAYiVRWudMW z?UKKj9afD*(A6OUMny1*z23>;h0pGO6Ksum8v>=7`4+0nC(gC}*>`ZVUa7xu%$d>R zR9tLXP^1O?W^XKDm{&iCy5sK#M~Ld!yNM@=N@AyTptZ4!1o`?>jF!BfeIjc!Th1bw`g(T>pqmEKWaRMwWXeYdvmz@pV|ftk%k2&}X(y(x3wI;S@kT8QeJLMf zb4wFmB+5LS@}7t@e{YVB8(t9!A;6K=WWb7vR`vI!;@Gi%!A82{^66^=ziwTFt!Z)? zG(XrZzbQCxkCAV#aD$sfCn8MqvxMm9j;9D6SAvG$I_spzTr_=ix1@4;2&)UpEjTE8 zCE`Ae>wKk({Y7{h&hAHUT;UU(&81G2=|F$6^|HvVAGdnB{+|HJN9m%}#pSPf=9IcD z=diCr%uLBmYNHF?KHLF?OK;TGk+RKFaHf706cOUhGyBPBr6_WZ^-fej(tqw|Vvonl zSlfq)-)9#`fQtBBe$|$@oo;XkI&q?gS%_{9y{3Blu1=;dkN>5cScILad~K+jNtoKt z{#<$^oc%cmyt_L=U8aMSt)GB&8<4tIb>fkLzY8Tzo=dN?I0a)8zCkH-`p}*!+djGy zmkA1#r8xSg5LB5MsKUFdq74CXu)ZT|BG4|+{Zw6w{hbV*`$r1CTzq!GR|Np8+$R7v z|2Dt>ox5qE0yVW2`f52h?9J7Z^jJWZ^hq1xO@M$Qd>t^aZM3%t1i?{8rb+4-&<=yq_cGn3kVVOqTCre5!g=60^k2p&QLf6?NBHBM{;qDYqN>a=4 za1t=>z2;)2*~9-9@lO>AVW$+p&_+tIjUaC)So6-%C=9x9b0 zlY-trZLAkd?|{9#(}!7L)iqaF$8g>SX0peO5=MoCYqH$ z6dX}<%weISwjTYu0zeUXgU6Q_--h(DCVLuA{>bm~(fDdw_q~U>h+C(wVVX3plx|~U zDUPFi146ALTvYL8iDv&7@ZT^0R4ouPQn!D4SK3@>5aQ*ECl_=N?eA+O9zX@Mw8)I~ z#%jWbVjJ3oQ_39VS|jOcT(aqfXb3BX`CNX!{r%YJ{ILx*I&BS4`1E>j^&HmXj4ulw zs`*3=IUQcxJKoT>d^7t4cJLeM&fX zm!a#M0G-EquR}5(g|%W@TlcO03xtnb;;7p|vOZoq6U;zlyl@ZMv) z%1Apt%Z!YsAiA@K+<-RStbYOq&f=$5w-%{0-vM_nvM9>ZP0w8E79lhDN^EYQ9FT1w z%$Q}Uvx_`=mB=G`ts_{{M|9D$-g9hye|2;ZRrkK5(#w04%2>KMeYk!dXk{17y`_77 zM9=n~#_LZ87GW(z@2OkzvEknD5E$=|ONL(C@u>E*(w8o}_BvE8%bI%BsAI6>&nHvMZ0$NAnLJsk>BTQ z2;aHF(M&_op&KCzWPm`3Ns2$jf7F0XNEFd}sSLk55^I3ibo>fR4<#HaAuzAI|FJjT zm(*5d&cp^p23%%|_tAG#r^ly>8vWcV+A0m~Q~;2n!VfZTbE4}YT3v8zU9attnab%D zz?%W%#);5MdPOX5gAf%i3v4w9Ms*ouFHnl#;*|OIvR!$CV&& zEK{C3u#@Hcr<~8>3icy&l;6*ywkGLOyMxs0$BFrKqid>a9U{gjhX0& zjJJZd6Pd0d5i2%RG0!|kd#0!HCmUJmZM}5H=YR2O5{?)?LNtbz{in2lj#g#P+%4re z?MnH|?#kU=lb>0+wkr~f>;3R&ajj2pg~O@QPxspCV|@jWoAFU_(Pf!PcI65>Prg z&%&q5NBq(0F0(0i`tf1D`*CV2MHF^lKdj}mJ5`%~Ty63hv%?_FIR-~=n-sb56up{I zpB}faRBJSEMv+^ox1}p@xO}EmpM0TMW$}mIn@zvq$P&bT1T%6o+Voy@8@h|xrc0j{ zIvvCSWLZYxVGbN4C-zM2yE~2KGo0ctJD;PNXijJ8)iCSo0ck391ipwwJ&p#qO9kmU zTb5f63aP+qzR{PR_ER_=3Zy~fCSN^8FqsAGe+_c0`VMMY^^{U}+R$4AyM1_-Ogy6f z)-QhuUCkuWxaSNNQ=2)e*~g=Jm5+TZbjU+QlWn{%{@;)j1U4-$g-%7<@cVmljXz31 zPd*ff95;;g;Llpts}sp4+g0N2WOI=r=ef$bjdBe0ISt;Yd8ZhP;|+pTGT@yWMtH z4yWatPQO*AJ$%1fHu-83*GF+rrmb26`-63_yxq({LqczS(x`}X8YaY>8e9?Q~tx6QuxcohQFfL~|_8sSxz zPbv=6?~cS>Jh^Lw+tel4)O3Ce8$YpMesL$fzjKRN*oIJSIQ2dKtT;Q{bjX5lPlMc} zR>0?K&z`DNAn&`PGyP6g>L|((tqQJ?zYP^e>Fa!6DNK-LicR$pK)RgEwXJPROCG2Az6cI08LB~BYC3zde?$+k zOGRV<0bmePB-Clkbrb=(X`?~Xquh#j(niPuRGl$_o{kz~cbt?wB+qO`ZRR7I<$~=- z%WBp|%|;j39`G9;cnb`iu)J9o{o5@reLIT_01gvVv+gLw9h$o2kbFw#v}1NeVF zmeDkrpaS^MhN9Gk@#FTB7sX<}wK1I-}JAL-&CNU#9b73~I)AEg2 zzS(1RQsUp9ImSLE&4G`3YH>>VDmwx#e-C#)*J8N(zlT&|eHFlIX~UY)q-k?3Q-OL)5Jx9%S2CZ7Dn?)Fex zjtP^63#%$VXHVq%jAa@p{l;%^=f`Hrl`z|7bHa|sq_(oQ_m89-;RUUk@srFb@60D! zBqEIEs9~*xp)E(&10*MVga(%QM2fXY^kh3KD6CkU*C+kdiD31DaDoY(rwHqNo4)6q zT+c_9K4b9XI|(a+#rHyFyU=O)7s^`ZJFX=(lZEIaP(?91%zER6qS>@ry*cir^d9;lR~*L)j? zkWnN%OG`0Nmxk+F%eGa7qc>@5btdp;yv(p3u5BB}!TOqxS>}E_jN)#ZpEX?qJ5ePC zoyE$#t0s!_FcI$wt{SC1aej%xPgT4)4QpY5%Dzy~PLFmSAYU_FGLf?ffhirP-4ijzAkw@MzG6Kolmq7R$L&r8yi&7kg`s);zvrF}Z|sP{?{zg3~=oNZE{iWISU>5hZdv0LholpV2Q3gSO-h`6>uX@qntv}x+%z-zT zKBrKZQm2qPRcLJFH?mR#$uLcWNwE(kRjOJUHTSa)4>`9QI`_TQ*CSQnm{*3mb4c0_YuXZ5N272! z3n?eMs~@}RwB;ka-MROLJ zp<4MGKWgje&&m+?`^)DK$4l>>u1ww}ewQ0oCAu8P#Jua?d7p0?TKY7yk-`nmy&>n^ zNwp2MSo1~jOSU}mF13=}Ho^JAlhbYn>_euWL-s1QnPV*yuHFkt4Tny_;0XYDZ}4KRIu49j4_fk&mSMtm*e0-K zv{mR)G8yw5?~{u{$ycl>oA;Co;MKBpHjD~e9h^1=f2}{E_=iq)9%L9jEh}aoGP`$N z@=e+(t5aT-+q7HRkB7L^FyIbRM(1_QCOAVke5$`uW`pti<6_7%(gbc7)14ZNCo=bM z5m;X*4_!)ji(MPZJ!A*nD&;DWZI7r`l%H`gDmOv+WDVR7j zodDYfT=R)YChEl_NmQMqo!F{^?2D51#%RiGApv^a{OR<*g~1~9=r;{{bp`sj2kO7? zu6=C-fhuTCS6DWRwBF#K9xvs_#-cqS-FhkiU#TcH8igIac-7a_!jvM1hf2%K^(QBX zFLi}YK^oT?DOXP&;i zY@VGAX51}&V0hd=%Y}IYbAf;H86)es+K6=-`ZK~ze5Fsqi6|O)y}YT^KF74!Wy=im z-Tqaw&6+SBOm-d#a==qQeugDrpd5+pw=j^hw~r_-fo2L-sumYNnPWX^Wh=3PfPQTl zr88#elPsxPK#8{@X=9*B>-D~OwY{Dqbjl2|-ZEBM%aQ{wp50#itp2Abe=a86OMR`Z zC1F|Ng)OZ}nrkEJ@h%4mtkimQ^X{6t(SptDA2g?f*2C4+sM04dD0ro|SnBm_{IoRG znb8AAd~3aAZQP+cNX@gvXc>t<8tO9W76;7CYDppDi*}}OmFw-<5TvvY3+X(l zW2b+_0tYxd23BtRiZwOi^JB1~jzmHKcM^=WL>gpuBx~=N6u^{{K$O5--YV4^D3zD< ziBMU&(b2=RfLvn~|6hN%K@n`HI9F{}2?zw?TnWQ2*O#C?8S^QLaDjxOmuKgaGB3)? zDbR>f7+I~DxMWS;!LcAlr~-V#Zps{p&~7P=t?`vMPuB zm3BddsQzc%Bflh~atXDRk;12u2yGnGUq}z8eTLNmb!xA5AsJN%;K5T6r_FMFs6DEf zBo*8pa4b|Us(oCZW}j}?2#SZxQC$VJ(%L=9qj2f0XfBTE70oDGIlm73Ay(z;U09im zDfvJBM}&qJP^)e5k|D@mRb>}bO-_vhwnJy-<>bUXX;H}KkewLWsGjmVLSJHUmcQ2^ zdqb9sj;nS(^!5k$mdIUwt3!Tn?g`;8mw+gsmp1;eJJ?=p&kR!chFW}MA?Fyg^O+q^ zr3%A9F)m~{wPUe6lI`EjHC>sONHAKZT)i#s*KBOje>jkSB!DC6+=ri;7Kr{#(8#R6 zW#-6WY%rnoBdGiQTX-j#qe5$ey!er-VG9^Ejht?!UXrg#E8oWJPA}5(+IJLvKkGP| z?N_jTsn?S-LTwX6U-={df(ukEb2`p~H0g}P#DW)beyI81hTrP9@};Yw8SYr%z`wrf zY5Kw;r`K$l5^13q7DTTvsBO9{NB>pblvK7i$zmjhNh^3c#qa);%Dtxb#dcuZ`I7!o zJBXE5aX2ER{O(c(lho^2*RfJn4vt$dyqxp4x2~UsJ%Zt&Ia*??iOQSHJr8f3R2Zke zTw!p`zqCpUfcLJHzQVWrpXo29am+wpR6+``>}wPpvanfvqwQ>Jlpc}hPeU+559|#5 zu4jiMLHc|6(?&Vu7AACH>!H^W%8-dQ_h5Zj-=^NRHm+cFU9Qcjq|Ngm*K8eSHjIV9}&h#!GXm-3*e?@h$F-SRU1)5IS6~yU>uTNi3?LN{+ooEs|RO z14!SCb|N|T2}bY%qJ9+#yvvU#HlKDC@cMmF;Ls{1KrZ5Tkcoj7`EYBk6jtMizjsIu zna{9pCHU86$p08E2nL*IBF*5X)@6_xWorW0z@Mvv-Q7vi*|o!>N`r52%(SJ|WfgRS zS@f}@XcM(cNa@VX)KM44K z#$tva$iDE3bU)QI!ZHBcs-j6SwPDLJJ_{kk4)@Zvc) zjEfaFQzl8y^$WhdAThpCU}#L-#eQihty})av&U;AIEQgsG2wxEx*e0p69I(a7c}O| z&EE`m*rrj-1WUw+`a>7Vv+C7`{cc3%oO0Ygn_JN|Tb$yn6LI%H%9(IKwJ{|ZO(JpC zlLg+*iupS}Y+~;9|5V5pzFQZ@F^w@(tA_j;p*r_7t{GOfn0r3icSZ#iEHpH_&Ahy1 zlcH!o5GqmTM%4YCrdF{r1B~;FiNV$^*e_ib)+1kWZxET@yVK`poCc(D)`K&VO&xKJ zRPjEw-3@)-gwT)v!~mLp2o-5pnf?}b)F)7&d^5l1oiClR*kKin&d8Na4Bq2dgRnZR z-fasdr99!suL1<#G=j1deyz?d3IZBeIdFc3|F*Pi_TDU!XR*4ij%MA~&r^2>miJtu z9R57f++3&6PNvP?I5+I{2Jo0eID*!Bvu)lju4qVjcXx|j_QeIB{&YgarpP_9a6IXn zxDY!_1UNTca^3{yyc<>d;KId>`6ST@RWHuO>x;7EQpUsPe&bjG#z>kvT;A6l&d5e^ zgOC4#Sqqbm@-MjtfjQc2H17}?f|aU}!cE+iV^6;~TH}rQ=zr+UUvvVV`lD_ECf7)S zO4=xhY+GpN8F2%v&@O6KgSYH#AMCx=I6o^5_`e!33-ehWPV?+)D^epsA`jTtCo%IU zRZ(vGa@1o=xqPcyp4w|WA0^d#9Xx8mnquE))?yX>Jh9(c?6UhV?%~tY$P8}OU281_ zR|Orutf&-Ap=wdYrjt2wVJm@Fc`)XD*KF3l-B`*vuCQ7iz zNKtRJm{8dCcegPUY5-ahfq}+%7I^`9&yMklH`6?^;aU$y{J1YU@_qL6mcb$GfZgtZ zrHLcAZ@0|6-~x9T5BYuI3qgkyJOrG4_sI)Em!hM&@~Hwbjb_5^{~CD&9j(2@hmjI> zM%w3#lq5N#libogM$7rZt1i-sG~5m;l#F<8;G}4@PhTeXE!1BylBUazr{g@d934Hx zvFbYo*xy|rHczuU2nm|Ax&8uwn#90p2 zhoa};p*>wgX?ltux2VkX#z)xE)>~8b>Sz3Yt1qqE>gl3p^xsY0fts%a=r3kdp3snd9IATycV=f9^tZ{7%#m8#vv9clhm@f37x543yQ; z{~t`EETPK9!t6<&I1}D=&*?x^k;5rRBfq0Ye}TjzzT4Jvrp1aHo(m&u{sSR(bmIiG z1!L=HZgQc^O=~0Hv#TAJsKxzeAKz)mqGVZhA{p-f*yt9fFfzDDy$+{8%b+$JW?n*2 zFPQMsCEgX(0mfz6Fpad%HK$)PfAb^KBNu-%g}(NEgE4;Mr{Vw22>Phm2{K{8L*?O0 z&=7p}y86@4tzVR;jX<)6-`Wo{u*tAIC-;!0^>1a2UEp+jbEeJ$R(Z5idg_5HT0qIx zMS;x5_GP(;x@*WiTkYU8;?sHCDXp&0Yv#)v%4-S-UpTX%!*6#>KWLc?Mi_V^-ho>L zQ(t%d-Fyhcx~&o=gB%s&{#7d5C!dlnm6*o(2R7kJL-P817;;%(e_i3rQ?&=2I75IJ!gXEc<)Oy=(my;h3Lv;drKy@BwiGgjzEXp zKkFJW;LQDkr=FV1?8HwgHs)Q23;lh}kCvt6I1~Gv%@(36Qzx-eFqkQoC04rH%@?y9VC3&f z(Ct4gTX}WCYT&{PupehdHE_7}6lvfBCj3KDl0^{_{76ru6GSVG zIr70-A^n3`z2$R538UN<2{!9KqH_K*R;>3)eA55ZIY!W71hr;MeJRvQk_;1-mh1)& zdQogSaiZav7R0XFk|DHW18Of}pNLxv%>WU}r=wY_4Q;HrdL_J(RoG&{p++@3)TuO* zi#+w{(i@SCy86PZGxN?9^j?=@v_*if%L9LM)9Wg&soc@1c?rsk>!R`m@GR>r6V=SQfdN1}OB76%X7u>yT1x$@V`+ZwybwVVCa z!`Ch=SNYmobq<2MW9h#o?o6>Bw?+pOGzr$hQPRKcU*l-_F3y+vdWFRh)e3W5F`eJ! z+9~~aS4$CE!Yu^%kb622zVr9U>gbKu;TX+&x2eShv6uA4k6pL>F&1&YupM>Oey=Yy zYMhg5>(>%^F8ogx?IV5!%#4%K?^_^xmEZm0!bzXE{_-ng(w9kJsZ2jedF~#|a+-20 z=(6?J>56aoyo{N$-N;t8=w-r^ho0G`kFIzrp+(8nd_Ja(L_Cayc@+A0=76 zW=mEiL2H#u#s4cZZ^Fx^ISzE?_;@VN<40fyQ*^%YMdg|O{7v7pi8gc7*)WC*jJcvi zW*-S!y2m|d0ibVi!smVl+o3Cdi4;ub&*Sb4?{K?sjyQN-BgIlyKZr6dV;1nXN{5BZ zU-iGJAkyL-{e4Yeu4^O1iFXbh9#iyw--`FjK(oi_-2!n{(iTc8Zh07Ay#XaogCxQ5UDu9F^hArPt}u9n}o$>0oc8*&_TX>S8v(m;XDiCfBC4+mo&j(AL(Z)p+pfF z8lr(ZN~Kh@fMSMbzT~3T=x1EzNXw0n&&4|mU(;2+8OV&C31ywtYo(eah#hga{&{)I zxd@>HQc&zFT3+ePWO#R8k3p zeovuva&fWVBrOiu@%ckzmdHH~+nIbkE9*lxTIK^~C{9)Okm{Z4UvJ$a~J zu)C_l9lIV5eKGrrGoqS`iWqb1{u6oSzl(*ytkX&zJV*dWKvWY|aRs7Ctkze)t3L#A z$v>0scqO?pqH~o7?dz#6*N|B;;9453;aN43K=uJu6eGnMl2mz`A; zx~PKk-`haidxDtfIsCm@Dc-c4CG|tp57WqMA-*0^057?U$Gabi$zEn%Xw$20UeSFy za1VA=pM)-_)?2BSH5KZt@%?3jQ`36JrjJE`r}#K)GU1`tc)U-Ef#`9((yDadRG%GXxP4dXMwd3bzh!+j$)+a==K>MxP92a{$qIUL!0+tJve3pYG-|1;f~o)a2J;;2?|#d%ao zs%b;~ZSYnfi8|coEoS@lHI4OI9k${-d)I5JrC+nek3u{w`uTZ1?zAWFbs`PDRhG(5 ziVE;*NxM*@r{h;g)OB$aNHEokvagdfXCMY)@w^0BS<(N*qJceO_f5IFgdhX+?=M2Z zwp8LXfPg9exQE6M?^%X)3L_H&UM6;_Ct_T@kO1T#k3oELr=N9JA@p1($fKf7EdEM) ze(gZf1FDzvWH^?8!uyY1WfADKD|k0#5%Isx^Y`iqvJAbTz(6XL>sZ=eLPhVLB2fzh ze~NquDJJ~8KZC!YLWa=ryyk4qr1rdlh5d+M;M3*LczU4yivH{MhZgyCtdOfiv86@(Mv@vq%kp_}wQjcUX!Ptj_G4o%DT znoUNN{Tqn%?;Q%PB*fg&=YIpQZob1>bhn#53Ss>jVZ;0Ga64Y2d6~@_^9I zfqJQUrklrf#_y+5PXnI)eRs2i0Eyv)e}bV6Ft$P46H0woy)om#uu83lFK_- zP>&>H+y&(o-~8nlwwyVMTVIWGQeg(<>ZAYP4I>t(x*Z-(iz%TXPiI|^h|mCu;EKDS z(`H>=8>cql%?D@P9*=x+S`wa$N?z76{gXbn@tpwM1a; zC#^;L+zjg{0Tx;t7>0vg`qi=a#R}HYJJ7R_APs2FNG0PN&Pe4Z|W&8(rMs)%2DY zYKp)K35gL`#@4!c!j_Dbgk0CMZGrAe8ygQ5{s*R7=-IdwpqbZ0Tj0;ie|aK%1uqh z$=Aa+yZF$hwsq{C7whpV;u^rG=Q6tsX!}=p-8Mvj(f@EFp*i!Ut<_9jou+Jf!e?R| zX^ne2K+OQh2F;ne^nGwpJ>Qw|u#Dkd)}AKPwF^0aFLl=*NIKwE8?DC~C$c`RPvw{I zocQGkG>w+gtv1%p%Ts9BBF(`|AM7COWM{I@jPI-Tx5D^Lsk`2o8M0P6cSNDykPZGT_ZZgv7$M9_8mWA0t zi3mO4Sz`$1R&dkZ)NSg~oPgtz&>&wa;UaqWC|?(&D|8?Gl;OI`Sai)~Ij7BXvt9D%`f|rHMcr;_EgR zVjO`JMn^mWNfz*#a3a1p-ZES7`WUglS!CYn5BfxWSG0^0n9-2pJ=~*D-}ZW&DQ?}E z6h>ktvMF-c`|>vYz3Y3H9I5nzYIW?|yQTo{gT+g&1T^Rw2O8wa(Qq_IF5~C2_aplD zgwF}G3VHz46AYRbx!?>OUM@{4FD)atmm+&dZO*9eCJ@L-rut9rv3vPqR(TX!C>|l=-gCP^$ZIxBGs+Waw`dsKnaZpSZ*+vAjG&Gi4)8dW!G; zYdGdX#h(*+dHE4J^?Ohp`ET-1UVy2NIq=l5$Zy}oPAVgi+T%uk{_KL-lX*qLsKH#p zM?HX~v+nk?2{iae$lnN-KV2vG=BRoit{tw`s@FB8r$RF@&)Cpu6MNu$WCKrTUVB4V zt?J$Yh4Ldg4Ktnu3^BOVVR7$)bp$d=PDbPE_eZQw%>&n_;BG5-qrlfN1_-hSBb4TL zk*$TD~j0ECE`du1{oU+^?>B*7RAvA>;R-gP>d?d9O$;2Yr~XbE%}0gj6uyC&ao%qwl5-){LMl}+GQ@tlPD{jT_<4!1F=G3LGC zX$?&<2-UW>mV-G)<+*o1Uuo#wbw8SAuzr0I6p{p_UgF<+ptJwO%F0=u?tq*SsP@GH z{(QGN2Iq>;%8`h{-b()NhpZyedpJM=+IXb1%7dvU|79 zg9*GHbSjw`1OrVR@#`{;qeVk9{1d-2Ua!KqmFn9<&CLO;=otz_UT|wV1rzw4)MYhO zr0l!3lh7G`=TefvWhO3k!FmwXxr$^hRh*ddg}4A>H}>&Hvt-aMI5@nb(E&_ncL*p6%(oGMySsfk_tL zr0Xf!94QScIl1{|BK8D*BmqXeHF{i~gF5+h63s$-_pY1s)#Os`2SY_;oaYLw$SZ5p z{8RF;@BEw(4rc&=&#M$pe&@F9k3hNyF4SCA*|xJ3vRN6Px*xP4XbKhs&I=~wB@(=c%Ec$=aY3-iIjC&EWds_@ECVU z3|=V5z+IoDC_(&JrUiIJ#sH_*Jg@y<{rx2S(hF6N{+r{C{$lU`yy>zZOFuHo?aoWj z?3xnI1cc@;E`c%y?t0d5m{p1%-w>g|&Af^2Ue&;Z3VqW>wQaAg9Pr%*Tajg5Itft- zM(`q`S`D%h*PVtN!Fc|iZwLY^+U5MZNn+C^qiiuNB0p{FK8tcW|C2-tYaL)PJOknE zs^E`NpW50dQnEn9KUZ(wu$F37e{igXdwa1PbPZW3%7$#1e|)(*mGlojg?K)xJ^+*0 zxL6K8y2ICF2wav1eS8FmH*_I}K{DVHo=ej~2QJgc&@S%-=X?zOBPO7kNHr6(=EvAz zc?&maSj8*KX`CuDS}M&Sy5X%f_NmCX0$tMFc?7X=CjIa;BjUNL{!OM zuI!pj-s2f^`C4^4{D!UUhs9lG%Q#}(>sLf9#a9A-+|X#`tY?lf3p~I(;Em9-wn}d) zZn-x|zh>&fRX9L+PKuAc#c`Eu0P^9?`1l#Xe5dx!L>67G3}&gsceb{+_Rm6&&b*pk zyAKjuY=e`Y-=EdmOzsZ-hMwsTJQ3{vOVX`5llUrK^~M81Dctgh=}uPu%KX{%3hbOw z_AV*zM%83KdWG2AyMk@=v= z9Aaph7942nIO#`)8_a@@fu6oZ)vnY`B0YCKQ|mpFs3$zpg@wvI)n zgpnmHuiDQ9CWVO$Gr+C4DVht~$>Zyg^F8?;S@b%wD-@B=qhT8TS3cG8nI*vk$kn_k zq($_dj z2ZAp2fqPAG}D8xnb&=;w&EL_noc)X9|}J>LTzI&p^}GflTtw42m_C| zuZ%k;s?#J3HZ6kOy9KYO*S}wRMqcf zn>c&uIL9^R)QC@0s?(p9Gol@ zRmRt*lE)j$ZwBf|%HnJ#Y76>}S#wG+^$Q9T`Q&=W1wi!yddW2369EKo3cXxc(rFk7`5c_JVWIA&ELC@DTY}dR!f_0wR|P7(FWSc0|KH3@6J1X zCV44Zgcm{U9hj>k7WB^YYYZlNYB?QY7~B@2N3o!WerxMADs}kBU5@D5GN#E3E4szR z476ViPr3$3i8=+E%`MG$d!tQ{{LWa6+)!u8wE1b`&E;K4H=GH(!on;j0ksJf5p(UV z7Jm0G7lyE{Fc8vZSLK!caD5A>g}8OO=0l)kD(-rp-Ce#ck4)qD?*&t!1{%tZxP(9t z9Uprhir-lC{UgctKWz*;_P|YyCgk{06-w6RqKIscz>u^tTY6^ZF5k-pro=@YN+@($ zLG464AzvYy_g7nIPZ*)`3Pn1+0e5lJ&ds946}beycX9PaSV(Bn6!atE@$PEZ0T@2- zgZ(ASvx2!(r3tO;-ja^A7v1T$onYV1jG@fS+v+gs%E{@@BBSP4yf*2!s&!f;jO5xN zkZ4m=YiXP*RheR#ob7+!iShLuC|@7>cT4`h`g5Y^LR@^bclav3yp3LsGug1qMd_0> zwb|qB9{=R}3oc-hXBuqBK;6g9kQU>DKZN)1RqNYicHR}C#J#eKBl380lXR?KsNh}?&%k$&FdqtlTO-#S-k+A!*IQ!;R-FAl$n>=yfNw5j=#DXPyOv4%qd?@k%w z^z_C)HOz#LL0v>H2Q*YD;J%Z@L#0?;R4MWsBYXre!ajtt&_q2^s10rp zRQcK|FO0(;_tYxS36lKEZE^3K+&>;-naazQdF!?ItJE>A-W3KaQO`a93AvhpR2GW| zHfH_pD?{tz2Hv1_Zj9sW=M%IvhqmTXtc`Kmh#UW2S+ zpyXF0;6nz935W~kRQ^5ptX7}S*N3=K!fb!M+ZG-IAdV8D_WMUUd5~w1m)zCoMdeFp zkFODxL67HFuJ}Bn!D^jQ>8XH98u42S-IUfGp#a&syYd+=ko{k0J!1?ll zqmfd{+aXm&6_=jU+$-|>M2Kc+ z_0q|rM)w7p7t2+3WndCzyu0_dbwxk@!aiJHd$*@@KP?)``;1w?$!&&c z-1nrX)0t;Y?>t`~AQ_Mm0VC`u#KtEBbW3dy0$>yFp%CnX*BbWgkdyPy-t6JGpV%Q| z$lLRS^d{HIgnY6|Ki4RDe7{m8acYBW+}8gL6Ib6pi+zGLZwBMaY}Kf$z&9JeQQD@~ z*R~S(O;vzL;*q&=uH)3^e!p$AfYvHLr;TMj(@!yBQB(5JvLq( z0m0VMGGJ8OMqAzXW;seUq@<;!I^y>p-SYND`2UxcQYqx`5=i~~a_b^===+O#@53$M z2k~OYK*Mh^69}^j*i`f#(`}a(^aK;A3lV+D`{KLr_FLGffSbUECZ^hJ5=8)zBMsbdNJG^}-b8Erhl zzx5#bYr}qisxzZGj#`+20~cKCeYcy7gnkGt)U;4gP#CefdTG09-Rk39FwX&|9JkI- zHMz_#Ra1!uz^9jX+1I!6__hJKd3YYCtbFtPV2jQS1+p$r68p=^*}Dq4nHj9+#Eu+n zUq54TmXH_=*n(1r`;U94n9d>>nSjky8elYXP=_s55OT^%r`#4eeoU7AMZi4)d+ilA zQnV{S3w>oS0iyMo6*~iO-jP{J!^lTz(IPkjph?rXL74GKk6d0IN?!1RAzh(Q9u>4r zH;q)9p2EVQFx!=!KA83V{9NpE++HTNLRA+t?xP+0(nAl(^RR|uhwWuAMl`C016zdU zNhh4HnZ(*Vc3OFC?I!og?(RkV&SJhKR)UEZ9+xjU&qL}lVA0SG4U+|QaCif7Jt7L! zbxAG?`8NxXHaY-MXVqlUk1_vEyYDfUoWSehnLImv_r1d?2-V3^jEpg4n|8jPES}{U zU$c-?%>BHncVH{P#y-NtR%raZlX(o3C1PyKOLWTs_FKJo1z4-Q)8g~U7UbedW*PP#4(TO25ox z0t_sQ-M{oU&0(J1nl&bxj0FJ8%pCf!t9us$UDvsES~Iqo6=$E7PRahG=hkhyx^Jep z<(qNCc@QBhFd<*->iE6(HFYZjBA$UXmNYr7W$0e(3GnxFQ)N4mCU7m33e4DDuaRPj zaK}7Agxj3<{6>zddb&tE4HA>oFx;%JuQGV6UO+3DhNvhveBV1h1~_kdp@4M}j@i`c zQ?0TeSX5#6dY-E*zp0$Wo#&s3_%mT1IEeB(ky#i&YDt-NM;LeHJ%|BWYI0dyt@DBz z1}@p0RhP+&xHA{bEyL{+_vhc z!o{Imv9tx(Ua(jfxD;?OckRJ|;D{F|RlYAH{rO>PN!rU!3xnXkuWStF?vjsjMz?k^ z>*(rT>`qldd2_H zc!XX*+Ga1+klJi3)40z~gfNl0(17rg0lb(Ba-K1FE-SV!83(1S1MD2K<1`GDlcT5a zCL?n%ThxTyARc2paCiSLfc|oA?^ddPGVXnG3hIPmL$;7dYl^Rp4`8dgYQZ*SWN8L< z2IMPW4X#l+w{(eev8Aw(P5RLHlw5tbf0IG6c3ecA&vbXvZdp!_0b+tCo-}sgsRm6g zm}>Vq$c3HZl-27JpASU5=OKGz_HFS%7mvt^dRIa2PnFFnhV`{cma$R7ftHN-C( zntGjUI>Eu0lPd*au~5n0l+9dh2N3?tlLFM&gyS>*GhDEU*FyZP6Y~~e_2gwjMQgf5 zoiDRu)*iLz=E?fN4i;QGe-llf8xy5Ju)Yd*o(F|^Um7=?LmnlHLB(tI5*Gt1N7bH@ z4rWvJ^Qm7$_Tt&fp6RtUwbpN4-sw-hq41o4)0TfQ@rLuMG>@L8pv3YH{(^S+ufb#O zUglr<985I~C26klRauKu1U7U}))oGVJ(k3_FL`cPFU`x#2_D$cKt($(Z{@FY>k~lI zPf+3o!6x&9fPGiD+SUXSb|t^_oCvY%w%)e=3Sp8%(Z8Bb2}NphFv#GLePsYpC};JG z-o}m<3c!%Wctzm)apq*iHD%UZGa+3G`#H09O3BZ|HeYoV0#yTH_qv;xP4pRwT@dv9 zs~4rzGtbchnLRv$HB^J&aZ^@%^8r_y@-K+FELHQIF>o=|k~`JKJW0udk5_mZUn{G0 zVI4b-%WE7nZQ=7fcORUvp?BYGnuE&Frt5Mfjqr2v>E@+o@Wt zg%%LxNx9|)p8ndy zrQIvSl~4cjQr}(z92IcO(ekp}uT*H2Zwq*$6^>(+8P8<-t(@;X7a#J%&Ynt!{3bV= zuLiAdrHAmCRMgZ!Mq$x3#DNzZUz9odzc^j*P7~7EvdUH|5k(t~X7)g(=TW;pRo{?t z`GJ~o6+Np=%{tULz9ejpm5BF`>Na5-n^O>sOH1BBBfGAr)s2OniuMyH)$I2@A8C$u zxot!P&lA9XsgFkuDmF#hXoLp4EYo<*PHw#PLETO`2>QHHU+)Sp;wYb^nQ%^;f*BTS zzA-C6jeU?_1!mjAM_}ux4W&ue&pI}<=a7kt1(v#YCqbzg#oU$9JurOT6DG;h>~h9> zGb7E`5PNzmNiwzmFJ08Un>|H6G_b(u!~uZSM?!L`gH1`z*PAr9aNJiDxY=Rn?KW-J zUCKGmxfD(LokwN9Mo+-YaRBA;@`eCFf^<{$E86Lx_${bAHsc~;wZwhrc(>}N{)doY@Vh^4EM3Ti(5#x#>D}F9CItP# zYU>Q=1K%uV(p8;FK6ui95H5cl^#Vw~NFCLh?Ujh-TQ0fy&NKr#aw8t}MwUD&xcDdT z_qZmP`be6xFE^}?hKPUcSBu`?YPD%?@O~qFHY^urNtf9W2>4cKeF5<T^tE z4B&#ePnBSi1$s$0!&aV^c_`%Sy=0_%-|n%%xx<4k(u?(@R8QY|^7uCwm7 z1|>{c>rf>$I#@`gWNLVOM?Q-vzV+bAN$zWl$6%#Tk6_);08sm%2np35U=5a5?ITfr z)>`ZO!(Z3TfloJ2-M6$YYBi$*Q7-kU$0JhcG%P#s>N&u=2U%SyEVwNGZFJOsQCpb> zW|lSx~^AmWv#@Crl zdJax1phM7b5sB5Z(zwoS=vTn-F*kEg4li)A%JZ)9ES;lNo&9)#Jy~g9sFez0GeV9f@95neGzbJ}xSv3H0-6{Jl96Sw;cIVw^aJ;5W7%{Se6LGXWtu5&boJw9VL5}XY!Scv5AuFm@bX&PmB`FPO`9sMfOaq^1ZIOA^96(!HVw4{h~{vr5g+XW8UEl+7&)`pxe9Q7P>s;k0&l~`kewMk zmVObp3MK^Oaax)A)01$Yl*vb%B?m7!GES>xk&~`@rVLmT>#M4pTqyv~x&VXFqn8OR zlBYEV!56n#UkaXR+%8Bp+?!?i3IZ-1si`D?n{0{u+^|{x_X{a9%<~`PQAw-~soy@I zz9N5kgBSUywS)E86yc%Zwec((-v!(4Awy z-aaqzoA8+e3~IdIRMg-+D@tKza3QwUTnXR5g63R8`~pAjd+nco7XI~m{`|aKkfF%G z?u~Z^&M@NPjsL75ZIHC_0fFx8?v0Ju^i$pmI#Zfb3024DD+xZbVplx*qQj@|s9hq_ z&ffB#e;a2WOhj^1U5ylN>U*nzHb@-ewt|3JZ3u>3cx{yON_6n)12EANv+Jz-46W#Uusn2h(Fa8+h1qKkqx5 z!@FnHh#HLnh|)qj4stV|4Ad^N{jl_jc9-tpNykIBz=4Y5uBVSL@9@+;t!6Ju$e=IH z@h$_ndinBW?i}gWaZA7%T;aG%;Eaa;X)BX`Y>)=zGe&>?vd2Q*F?I+2V73Z#5A zgxs2&TJ`MhS$JV(_pCCj`oik2qwMJ`bZuLiS$LuHpP3YaM^Q!-X!nu~u(+#{GGBF& z@~EzMn~z${jX4}rd!A=EM~KL)$e8`*Lf5N9$r;Cr;c;h2d`Vd?Rq3f~82R4sQ&Eod z_D%#AH|B~h_8f#_pO1^@kOR@p3eb@jYmwXe-ePX{23nsSDB5gCp-^Up0jZY4fp0rM zk|+c2zS50E7RG-VK2|QyUS;n@2V9J0*cv2yB(*$H7CcP9HG7)l?mq-hSVeVDFSMrt zW4T8zuycJ3!64BZ-IQLU^x2P-M|6Hop;v2St@w9s-~5+O2{VkesI>4MuMd*74z$kS zYSb~UU;Q;CbNc>huI8mH)WNQ88{|mjVeI>Dw)Z1av#|(6tLpH6Wu*QIq++S)J&f@> zI-6{@D=+a;hX<1HKu*GI zjoiHxVJd5$RyaGmC26XS&zmt8&^g`sB4=Xdu?G%|0@p#n$)SD@4lV2QkL}fMceMb8 zfsSCo{l0*D3@udO+F@i+)OO*f=t{aW&YPO>Rd=f_ygQf7)}_hR61bRj1sX(wj*Fe> zX8!myGl_eBD4$IhP#^>90yV#z+kx$Y5Wdk)fyA4D6o{DW0@aK!LC9w(`)#8;qCN92 z&MLdtA{7<3hPPrDqC*px3R@pKJpt>tk@8*N#M&hZh+7wHj;^%1y#I=G!84mnRYc<6=cxh9Q*{O*MyatHl zk#j4+>4FEI5s<1_AKvN$dcjY@mbMaQ^K)`#jz7UmWYj!#1&`jB-PaCvUH-I_8sy+G z{K3SI_q8x!Ht-DO1EeG}8UOhU9zKMNi`oYs-YSK+HCMK8#~)o7wpKX}S39{G z__unPh39BD7XyUEBgz&}w(P!r5YKYE*U1F$X#OTJqln{byagaxuHr{(P)QwkL>=-L zi)EK_4xs)3RhP9{bVck6)Di%7o_U;~y?~zj4V4YvPwDJ4wdw&&)?rh$zOk{k4>bf`{5E11RxXEudz#1!QCMq^xhc@>ef?&b5x}xehBuRva%-hyn)k zL0sM*RcRwSE+Ik7AqZ*EdWaez73yr2X3X+ExTCN+V3FpmRx9(gE9oDvXAthEpc@nh0FxO%=XGj5H{n90Uu)? zdLjL`w!aZXELA&(e+E7|cPZXKzH>MA6c5M*B>I?oPk>H7pXd=}_BVdsS*pL-ea#&F zrwC>9)4k=M)fYP|avf{nabbwklovL2-wj$-r9|uFL2861F7Vps?cs~vg3AGjgK27@ zL83-8MGs@#BGd=oKIRT&{!nCzUI`RK`!`UX3N}W{C%eavb%w5Yucb|46;X6pWQD$Y zRe35EZGDDhcTy66L2g+I_6c#-M^#5>bdL>C<)_*q6i!<55d=u{=fFDIz?)czv{-bp z<{Uge(RcXyye}wYZ$hf;!VPIBKprzJBcG8))uKcgW#T5nuC)CxX}L47d1<)JC7i@k zWc``o%<8JQOu-aK>P%YMD9aL%P9PAwflha9=7BY!I)COkPoa;X!9`PgI0ohHb1S_T z&MVixvD^AeC;K3aPG42S-G&PER0S-OqgA2NR~p?X|lPn+WN2xk|=D&&D|t>NsD!Nc;;BQ5WWKgjK6 z84T+)ax7b}Y%&WJ*JM!!1eRMnK_se8jw+E0ukmF!r&?-$gqwDCz5(n`IuTHxV*k~d zp^&iOMJz6wt1#m;(eoetp#r429K6m0k%TWfZ49NWzcW@*l%D?S2-&of*${5zivw!P0}3qac{ik>3=rwA<2iph5D}bX)Ywb#7&)I`jx!w>1lv z)vA+Blo><>1luRwQj1KqLEqcL>B7P)dNl>03=FL0>K{kUQr6R+_VRr!8=Q9Gdh{f8 zFt=2+R8fiCJ-AH#;M<=la1+M3-Z?zyZ8Grf)zqaGEnm8a;gDq20j2gk0|jRtzqZ|J zJn8LJ-&ScA5%j}>{5(+hHoZ8)fFCvBq|V79oAR$?0IDb%vNK#5v@l4P$Oojm@nF51 zS7fFPB2xb{uxMTc>WN^QRJN`(@OF5Uv~e$dGB3?_pb>FBBzNiPJd?={QxBzZI0a4xcI# zl~1Ri>>Y{XxSx20Ae|AlYEo~3lB^A|%|XpspB-<~1t}_L!pF%3GL>-I0)t3uDwAnQ zn%qvvQbh?i=i+X$ZC}MexfXv>oO8--3FUW7-&{G8=vPjvoE!|h#F638rp9~^S4XbZ za!?#93nTdkF*IIiYzhSdBt@1_t53K6y17m$(C=9J%S)U(d3+{UjM)Z|B&UzLZkd|C zZvB+7bkF}Z;ot}ZO0dRq5VX7i&b148i@|*_6vBepDoVVB<{9aH~q3aISWAXCj&XX&!eDF)8x!mu_&27ta&*A=N%VPpF(E)mfYR2}#_^ z53-_ZsgRUEJ2U0B;iT%Et4RrqHnc6bgA!R-fjvTXxL@@+1v`9`=If%r(B1vFMs_Kh z-)aqYSb_Q#V&~V3=YHM!v}-y0Vi9=#7uLmE(bAantyW|Ac4Pzi;EEof8-GD9H~^3^ z^MsxavtkR(&m+yn)R7bsN~EG^ZvBInxQ&=^RT6*&WDP(b3TU3Rz8*5IT&$&s%~`K^ zL!(@=OVCr^`#qE%jAza3r?avR+dI7q1&s8UGe4@D)n~Oe$lz3?a%H^fNV#%D*=v>p z&!TB&rKhZ>=7qF06+6ecxZz{oZ`;GF<2(JZRh`;?-#^mc5v0nT(zp9~1bb6G!xP?- zT>Sr#s`6oOYMyc=(3ixdTPuPDk>&Bnj`(XoN~0xTLXyP=oTTaS$kTVCoTQ^Q7jV~q z|Nd=S0j*<%fCKoeWkB|P64ZV948iQ2|0*8uMgSSG@k~1DxA5%BpYK}*`nQ@uo9zy3 zQ7f0+AsuU7!7GwA?Ino8p?hXw#U-Lh3~(z|vs71SGZwYQ8!rZxUbxN<{fI@O-{1~L zZF#YlxUT!7LJqogC6Z`CqKIFDO-@knw+_;cUz%R_WD4w(42TsQ*B`6ld~{=Dr0C)` zDC-fh3n$5%|GO;g+}ZrCVH4#Ij>#O6YPsqsQNtCp`$U+P>s#PH?KxCQfZj3j$?)F= z^Gi5+oBB@yg5ZXY6h0od0I*9)fR)u|sCl#7S^>P_iLjBfP<8k<1zP>wjvT~E9UC_a zLI`9DIOhuooP=%P6F)j%>g)D3fsvo2%~SNSeSsJo&%3S~xs913R%FkIut;4?mHd&_ zd$Jt}JT4a?##I^_6z0Ds>J46@EM_xAm7oyyjaD;pbMvV=sY?m=3;Ib>yYFDl{R zx?bMl?O`i{fDYUwkj0}q+o18lsoY`?p1u1t99$8=%Jevu?iCsh#sF&9kEP}|LK4?p zBag{hHRu6m+Gh>h_X#L}%*bXgD8Y_hXJPxA`tEIJt^uMEpi@)+R7Mq$0YqM=J$rdg zZ|g+@J){&A>>O2DR6jgY(c7s0K9ip{%pX`7V21LR|7TBcPtrL6SzuiVdrRr?kb_OM zv6={!Sx#D}+IUYF$l+}$E0y!u^iAMs5omiEof%&*A!6-SZn&MF95LoGlx>&)#H^Mn4+i~AoF(@(mf0)m3l!5{5Gav2`6T>nNOk0! zR@0mducc9R5hv~ps+TN>%e-5zUgWdKtgD&~{YZPW{Fu$2UpC<))pA3s?8%v9WPOX~ z6IHS(VzoH^UplUW2aEj1@A2`YTIs7EXX`%m10?@7ou|vLOz?ROdGLrHK??!*BxJq8 zXAES{5O|OQe&_C=zV|*7Xn*X&2uy~7bmoN<@6U$}oF{ddV|62`Xg4sEKRN4MZ)VKN02C-ATD0qN zE*B6h^H7Mr4oV+4a@ARI`F;gTqS9HkT_2?piMBth%Qyb)nqSZuz44$9YGR>Pn8rfQ z4`GQq7330c;WO7|0nfClkCZscs!#H#?SPKf)f^tZ&G`IB9~n?;x6~MCfrGQ#^HsqB zmTCDp9>}-Qb?S)77dldtplejp5(ShZIP3LquAk(Uavw>FRA?4+gIa4y;;{dg>Ct8U zes&WOk>csoTQ?4L&_K|qXIF|68Q%5M47uZSQvv6{W!09I`1JlD_X+;DA;@wB!xJ^*x_dx)z(z2b265mm%+{q_o=3TlfPvZ!YFxRF$9q+W zM|sf;Kts9%z(N)qzjSBqun_t35-R!B@&3#m-Zvv={1DwHCzJ*O#B@fNhPuA05vZjj z1+=xQKWSoLMN(7xjChPck)WrdUBm@z?MSe3=nyzBk+MesNYBa!{5isgz`dtrE?H!c zqcv0ySDJY*@SHn9YDseZQLZ?)lq=Ox!_tO5bZpyFM7lHC)!W;bV(GNW80FQv0!D*@ z8ihxIDy(U?8`6zfGae+5yXqC|l$uc#9aK@ax2XTeld8A;xRdc8byLZFNUhOc}Q z;B3IES8*HO55QO%oU1P%)q@RU0t0~Ue^rjj?oPu?e-yjI(rPyrf23Ay9E#ay33dP0 z(8*g;^m6OhGh~$eq=VTf{Qod4)fb?PL*fd`!N!JV`)TQi%KCj`_eY*0C7mxRN}1D@S!#F!bPAC zPo+3Kn$IHl8sa^EYRB1+t6CY+*=J1NYW!nTUy0+U@&Rs2{{OhCv`z4SZPkv^%3F{C zy9PLc@7P-@;qjp3WLuYfCO+OqIjCehe97r+w~W;$ClQa(lyD*8vr3x2=@2_%ePajL z3ba{$kHaz}`j|!b9ZhtToK8iQlv83=_NUJL(jP^Mhtzg=3ufo+MF2`RI2C_zGEefN zE;<@wIy;Xs{Xk<;JbkoTwKSiu^`9Pe#7XC#qsix+KDdI+^rk@&NVnF0O%4EcJG-SLHk%`jnHHm>ouhQmw2S5V z0eU&B5Fjawtn<#&_p>+b1e0Ik09Mu%YCrpLR048+WgHX+^Z|N`SdAzy+mABG06t^Id-4dSB1qK zjI!=+BYq%n@_Mh)t-$>U_RO*AKBmU&|1xrqn<;yUZGbAKZ3rT^SuY_zIA!M}tK9Tx z0`o+4P>4Y{B<AhLlhkpob`#-e)a4eXnY(-74I!5%u!!j)0p&u`Wxn6 zLjcM7U^@AIBa5k_z7VLB^F2?AtT@agXfA&xx#>uXfTV*t6@JuU{*KZ%VM9iMwT2I7 z*|}zIzRv9c%NJOGzlV5G40}->&)~+u%p9VRgA&3r59UThW$^(xRiD%tM{5&N`6hUO z0i<_+mH=IxWQJsA*Ii7YkCtba!Kb|A`TI9(!wuuQ-Pch0#@tz`i@o14bN(_^(d%gD zkaLYCuzUd{Q1n@xX~S)p_<94UHM1~$3)BArdi9SkM!+uV)S%1zyRn5t`gu?>aaH^0Wks+b`juK zv0rxZXr~c4H@hBy<%s#j(z;yEoeM}`R1E2;bO)afgBz-jnhZoFh1yHUbnjVMaM-)I z67lX3nPJEw*pHCfR)i!<>a$04q}b161Y`lb4N6oY#oBc#6I!VU%EW z5IpizB@Jx`#4C5lZcIzmQ^j;_$Hw+n@c3t)bmr6LfR)OF7Ozfmih(DD6^i(>)I)k~ zq?JHfS(t5qV^pn6vNz+BrMT02kbt1KdmPcG@-J^^gaS$3#^u(uKXYN^Y(tA?9j*Q6 zv6-8Zqk(hUrlx3Y=;=BkU&3C;WyoaKl4(_Az1}T42oHlo_|3tD*(Wz$(xE?-UW7HL z%~C%VP$P(Bf8U6|`3=db_T~-m3!|q`I$k|+pyWEbeN&6l%d*#mk}`rfxkbF58^Tj* zYM9YEk&DQKjgLF|n2)6R1csbCz`7b5Bn4r)Kn5&PB68U>WUw^J;xJ5>F=5N7+AuB_S(V`lC-Dr(}<=YD9&FqubirOlm~*#}(rHZ^}{J^rV!NeIgpn z@Up1Ks*c3})hh;G9wb6@m$`vQ+ePvD)gMu^{to6lR?xRl!K>ziZfO5+Naxi_b3H}W zPiyd6=1uBolc!d7Ph(t_e!QT-jmXbd79L3wktFpgy1YHiTYb9;C>z#gXYx~v-^A;j zBvsEQn&WmZdlN}NZD=yU`Gl4-vzLsH_`ymc-eQ_rd2}WHaNhfecQ*e-JSXXT0q#4> zu#K9krcI)kKch?+*&3(fdBclJS0R^ECy5Lfe{!X1W_I`LyM=Ai2NB_+iEn(?grw&T zds`h8(JvQokV|QbA2rTm3Q(3+1mT&znI%UoKvuhkSF*(&TcE3}dw>t4F27xt=hB$@ zo|Exgwp@yNn{y-NHJ`zQW_LXdX$1oE$oMCRf+ zq3CwaG8)P-QC!x4%90RH^>)l^`pamsnwfvv!#m-?zs$!%A|i5pE!63seH$9dC@9#) z`5t_Njek*O5xk*S>}cdHHakhW7_o5c*6DN^`;VZhyk4;1<5ZF5%E}qG57|nO9cHQJBGvxzR%ad*-|>rLQq5Rk$byP-h9GpJK7L`XQU1sG7oOi(Hjl+To<=MP zd|b+oQKX9(*%6`?RO?CNda=FqXX{Sr$!*z|TA>F4FDA@x-Q*|X6Ekn}+x=P4%vn)rE zdOMYj?5&}$Zt);_U!hI(UwZ+HuQniSf*ZzW9%FwE5xS8c;-dQ??}YGM6c=Ue(|K~l ze0BF|Za6x1Ch=PE4!Rz2t9((oQJe$3BCX_w^E0fZ@Sk-)_r_Q z!Vx9$aa^-zr)Szz=Ob@nRaN$@BC(ePs@kefFv7<1*Re#oX6?Q_fl!*Mp!Fm2D-wzy z+-#{zq39wO%+7+cii#~a^?g6=z{L{_++u2?Al7B2qx%XON&t>G@b=ElU`8-t#G~Fy zEu2;b9NC6$Auil+hi4Wqf_+|Z@fKXLe-2-p|LN+JpHin^`ILr?Ozp5!P&tA0f75Ui z%;>gQOSna>=$~R-YpYf0RYsq2{ZNlNZQ3 zs!wXxDUGTnbgmQVT6iY>WAWLugNtlt%jsxd$i<`vFJ3!q-T{%VpNQ>2b47?rMCzLdH(k+mqXw{S#{~QLys%W(U7Uo3xu~ED8zb| z3bi2}M&N||KK=wTTkR$N4$VDLC^uynn|(p>-rgCu0Q&^@$NZ#-`3Bh&4y?HS?@?_| zPL*+Q<*8p}f0BnI+YSdhwS*!l3r(*clMv#Ym^=i-Yc9{xM`+x`XIQw zUC=L3VhhVVCXheC+|?olo9uZs(V`Q_t&J(dK+~+HUC%tzKZ|; zmA#oV?nOFZ$Rh#l!*PMX>mSd0zO>P7O7jOfc`>cC9-Bptku z*4(>hxV7ajcV3ET`URxlstS_L`C^F+OZS)U|`-Dd#oa{Vl zEUKNXHUWFnQ}@K)!AJ|3 z?uMC0U-o>yFaP&Bm-Ap1DZJiAu~#(wq1}U$N3o>cx`&97B+D4Sl6AV4^h3uFohVnI z<_9-^%D;K@#BmiN?K-qv{fzh~ z){uwnh8#t3l9sQ{Y*zQ>&TCl|WCiSssPy-y?yUauo<17b7V3=3kNDer);B34CFTvZ z2S#(Yf*>7it5R)KZwSsOjj^Trs2Bo>O%D`Ljsm^=rd0d?@$?oBQFY(r+dEdjobf9$fxw8+h79aKS)C5K7FgDJ`pgYBgaO#Lfqz=75(Jj6t6B3|sbLV<-_tV8RDZeffphlq`vz;V?^U^K>rRH`4#TE1cV zHp!(iR|+8tnz3h4XfvO)6}|0H9r}`EK&s7yk) zJvDm*aBlhtE_MSQ()_z_Q*I6;%j1aWlFdV|$Nq4@VB^nR(!}i+#Js|3rWY(_4a^2I0 zLgxJ~(MrfV;XcO{m0vYMMHvugH`Q;nfoQDkuKoA*O5qb>AwZf>MN<>dL1&(D!gJy- z_4*wyc4DZKfC-vWFmH3%MA+!JiXaaxY?;>YP7P3%aA1*)N0{~L6fO?j;qxfhBvOH? z{3hXI#h*sonqyn7Ha9)pTr9+|%}3HJg+~2dTOxs6=MksgN-gbZtm+Su0G)JJy(b+7 zZPF?dp(fT@KTHU1yV3-GP)7teG<1G5rN9C|$Ro@6x$>pZ))W<4-F7lY(<06=Jj5ya zww?9{emb(Te|^00yhmwYZ=x(Ki>9Hiy>Xf6>lOHaA6UfhWG?nlE)3h0%()P$wGBHa z%Gu;c?M3I7MYB;#7cE0|31IUK37|K zXjl-CME!G%6*GjcA4g;pOYmOvoWrE5+yIWLkpFg462JRRVkRiag+T54SAw+zCAe!E z$0_({u%k?$XAqjRPAVJ~x%x(<-_mkbT=h?Nc&Cxa#l`$V9W~}cR2R2VzE9O8Xa0K%nI5WK`NV|k6uq*a z_fnF8b;&$T+b*0bcm|lFDZp~M24-upt-o6`qsk=IfKhOQAn(#6cg4uU{n*Ea70JijzafBT838onCvmJk@6LnCeBr>%(PnClAVK=q zTJTp~&v5PQ6WhT-s=7lS<$>8#lC7LEsbj~vZ>yfar;Te@xuj5<)LdW z`fDkj&%P}ijNX5~m9ap_YOyC6L*KMoO5sV|&y8+BjiWDLYfhZ}0~$e-ho|Mbive_0@%IXWN&6vje5uZiqzMF%8$wOJ4}b_C*I$US1_K#37- zcs0!X#rj0X(^DvKJzOKkxz*OD#C+V6Pd@yHl>?ga_cpW$z=MPuUoNuyQ;3wv?;3>$+{b2{>1CE90iB zbWlBHWpR`XC}V@0Cw!ix@G3Ay^=~S@w0;1(OHJlLfMgB*Fz(fA*ej>;&4#8Tfj9c9WYHtX4_S3s!&k5oC%~VNH4Wr z1HZNh7b}`MAoa9+;}3pFXu<%MISEOu^Z7PM>bl>15y&ph%1wV4KoijC`MeVU+}6K% z!m2;%0Y)4^?vtT)w#z}cZ60@f1PQ2O*l~W{6n*ufnE!)pummY5>h8e>Po^PCM;3DTFc1*d zi9*85yL_^J>0}Lc_TzjE>As28Vwv@QgX6nIafeCy^5%?+^1{2Zh-GqFZF61{KQWqF zEdhG(BTlZS?fTgyYHX~CmUni)ui-3Q0eMYnt00ih0u3e0;8_fyC9SXDl=uj3#!>Ww*Y~S#s{AkR0mf(uqDxH7jj9kuxg1G*f0yZVlWUCNJ(LLMyAhg{Hrf4u0CKOztl$%!Maru_p^doJZ~N(%5SgYR>V&~7+a+z*$QZmi@OK))BgP_L|o9ph#mj5P=W8ZZwo(y z3iBIpu^K37!4h~WE*{!R$P0D4ULvNpSnl%9>l|w{aD*v7SreSNCWN@92 z9}xu%n+`;9M)2{XzGMEwMkySxl$AB(zS0X@%><50&OVl@T%S3F97QXB{gY1gc_@Yy zR}pwz4kv+%o@ei6F1NZd^FGYW#`_qHrS zMW69)p=3HnS`3E_{Zub$gpA8aeBt*^l5X(isk znf96F>UTJ+AbJGR_P}!r4gSe7o-jqaXENqaYu~UCZRv9Ytr}~9wvlMV3m`E=)=-fA z3+`V;sEtY?tV`YX!K{qi8MV6&65?CG9NHdAjF%FRW_#Z&JYMNVbb%~Yv>g*;I?d;< z8=3y2$#fvZ{j+jd-~`_ca-j;ai-tu2U2o^xPTL(x$&aKVe%lOw$f5;t&FDkB9uPs zUlgEh0qddq4hSm4t^c6&RBmWu@f(l?OprQ?(W(m_2oCIlRbmp90ZR1 zKi_njaXrDw>a+`)s3VYg-OB@N|1y!sPDcLuE>LopuYr@MsHXZqfzs3!HP|w(7JqOk zF(F3(wGQI@F8AmJS`2zH_|K`V1NpwX99*!~=;>ln%|cMZk#TXoB6Fj58JxjD<=w37 zgZxrSd35Aq>8fC(a5ZiG$9{K(lQ)<#C38%ulcO^h;Ja&4kVz0z%6iB5F%-1?48>1M zS!1KJ9X$L}Dqq_rqg~(TNh-cJ#E&mI3*%x?WeZzJiyt6`c+8_Mfg;yK5fHN#ox*wh z`CA)^DRO$cr4qJ3t!j5Zsjj>?^UG1^I;QQ^!6AK;80We$s|_9Ws`c_v1Ue;DGI^TR zf$;URVGjFQ3oTjV*G4r5gC)HM?$0_SX8vwP8<$sAuH-^|%KXUwt0j>m8=0d2MB%N1 zukG5u_89+_sITI1xr#ygFvLdZ$}f1xl7r_;At}xKizI8FShQNN(gEFh;{;?UC)a8x z4f;XzXvjPDU~U%{@tA7IbcH$LgxK2$ zyDx78Q(w5kkiVv+1Y3`)O02zzcvC$0d(=Kn10A@1HH{$6#FOpQ<%UCk4~ewKI(m9n z0m6f3kWigk+ncVr{rNkF)N+nQ z{nOq46)UTwC!sA$NS&5M6iCS@IY2i zGwkO!djkOrWaTZ^wcc&)2lf2CE)31fNk_?3BlJ7Z`ME@vaGl{m`VuPHA>+Xr{HxL$ z#&&jvHs4~BM@-s%5y0c;=CPbZ+xGloWolvh;TCtwEI#VRTq z2>4j*W(!qp1gM{Xx}|#&zE~>*qLwtKZum;#6`MYr?LOqw5xjFWrhNga*OEFM>Xf2R zVlG$f2O&CBK*#A%1U_ofEj;?|%Q=k&(9W3DpS&L>Tgw75^dOThbX7aUSpbZdR5b z0Fy4l>WwH3I&Og!(|Nu(nVIFSI4ern(>j`~F?94{J#92sOiYnGnNuIw=3q_bmJ#;` z{a~@?w{jyYF}}e=RW!3~cR(B(4T}a#cMUW}*pHYC|4nw*SmDFhhBh%$iK|QXLFHQt zK_#}o)B$Sm6}hatUSl&Z)udNpO}W=Kj!K6`eVBeBnWe+1)z-#4?eASdVB^}~?SF{1 zlr9oBSU$CAZrtufe=Bo&AIuGAtWjv>(w68Rf_^RhvQBh)|DJ}{xveWRlN9EsA*Cvg z?%{_UQoC<=CO)cjTj9${X@YIgRPX&;xcvR+E!bsvFb6_a8LOWgwze+T@w^4K9&2+s zH3{05r)t~(sJY7(xbfLOlvHGOYyS2J6Fe5WhNv#$sE7g6UAM>2S$N5u|4KBu*#flD z=hr2MG{B9Lx)f~TX9ovN+|=t604qln_zyo$eBRn0Cb+FjPN9OQmE}nD4yVUx!I7oU zWtu~+*}RTb(TW}xZI}Xs#b9p~u)!>PLUNOjhM?Q_)5iKp6L`an0WHhs)rN`v{c*R` zcv?e)xatqE*HZJZ<^*?dvvt4YI+;IhSfDI7CyNjszu0Btw%3O`LvxQSZDWnWY|rk6 za67Eo-VK?gvcCP)d;r{M5>8N_S#}BOaoo30f*+J?wKYu@m)BzNJL1t`aL~GmTz*&I zH!7Fo(ol|Yv`3!4&MU^_IUPgv)tg*5Y}Vh!lBc|*XYeJ3P+Kn0b8K)HR}iR^#q4>B zk6=wDpa!Mdd!V3`{Iw^y&M^86c(o2q;o+6>728Q}Xy_l>%-4(Oo21dce;7Gy?D*048T`E~jL5@&s7hu|1H|FLK z$zjpP$x2j|)~^@$J0-7j3`fQ0aIsko5@0ZxJ%bYt2yv5~-_`FkiM#guryh&D@-Qn< zzM13<%4owXm-yw**!Xz1EcT55mGI3c^1Gc+XL&;Bu;tR%{9NKUxQ=ZHQ zQkW>8+F)$mq2nHIOl}s{zB(LPUbIO2lu~qN^^;A$Z&Jgg!;7^l#;~OC9+C0W8##J@ z$}JSxRSC9Q@jT2q@){S5Nd4Fx7b$3>^X}S)W!vrdRHq8eAb$RK;Jx`gOMm>uQQjz9 z5+Q>!yzZBis(W+%t~D9V0giprK_cH1&(%=dUb z^k82x(-R^`^sMFyLp;v7IZRb>(2f+DG^-xp*Y6Vxyp9X- zI+5*48elA$%#44B{b}Y`LWNqHCxPM#Hvu#1^wO9RviF!4Zp{}`F7UeUZrRSd8(tu> z5ITV*8TtU{#Jwt@yrPy;tvjm+3wu)hXRDtpHRYyDG0C*yN|8h{Z0wPtG>1vR&S0_T zuH!MA-d`P+TL@u2Z3sem1w?L(%3R5?fCV2=6X|+Mup!sLv~V|9Nido0E__V>1gW&& z3ARfjCO8BH(K_D;wu;@H8`9o2uVh`O`N0(VRg_YgeRd5$;Bwp#k*UNm{|N3_JucGT z-B*XC;v3=;_(Nq{cg9L^XA}2hvQnz*DJrp~iz~p$qXh&rxR_rG`dI_%aawo$!hJVDlq(xQ(HGb70?P1s6>AlSS~NW}%s1RA4M0 z;X&aK6&wq+%a@o#CdYGZD3vXK#w;LpZkzy#%B2!UdQ2 z8L&lIZg6>BJx?)ZY~PyTWfcY7`;%kr5rO%+Fj7fZ(}goG;g|5KQx4{6wV-w-d>FR6 zTy>4+?^?CCaZlnKQJ#`7bCyGqdqfWtxv)^{k7_@xq>Yd~c8svXD2xiGM?N`!{re9n&eMdVQH z*3cU#dl3v=IuUR+MHhFw;>4Ln1vLsN{VoKtfOHH;3d8I z)b3!dI`d(IBLjn5#M0I_2RFAB~mVa%Ccpnqi`(C~QK`zW;FhYz7!F`c)(>8>yp@VA(I zct_AYOKQ6-$)nVD_}?#M0ls|vd~w#4dB2;m@&PQU^@#E6`=i{%+D5Rcgq_#4#B8E> zCk$T348HOuT4jINNfa4g8}&u4Oyz(04qdvl+E^vvWW)7SHryCeLMSjcN6io4aQJYQ z8GYuaemH6+ne2lwZIxX=PH0H+(M;vgA@yl89Bf?PE0nW*Y8oI!&uR~P=azj$kZgG!Y#~XHGxclvR$a!wzh25E^-}kV5|C*2Qw2^!-@>dF=zD$V7{< zl9K=NQA%}uHA7PGJC=>U6gMh}TKrsSX(TrkJMaa6&PiY@|JPd-*=MIG+8Oqx(*@?% zwAxtq>e*7y!*ILVS}8%ju0N3aHD;+{6>Jfp*3mLKGT5L!i=vln)P^4ykEFVfW(N*; z8cI78N)3bx8Kb3FsF)p!c>yesObgxe`Q1_u5_KB+q=g~<$=d*2)aWB-NSJ3uCUF-yn{9jWm zVweh!8TfVq1K@BrOM7t)!;G!J+Su$Ehz_tBLx4qcEDJRW@@_-WS2kqjMizDrq2X24m6%lsrkatg@ z#Ph@Xnx>7`bN{N3*s$Z7_wC0D-veC<9lS?<2iDRIs_WB23dqsI_kh&vIg!`j_`d9e4?{d2`#oSgHg zoA3MO;qsH**yybiM?|B~j_Z*{K+xAXcN*h1|CJ}@or16Z!4%;IC|}A(z6%$L{md60 zCgHMV7vDPnJWY!+wSmQ-$X!Nzw|wq}IDN!Z;eOzr4*`5!U0b`) zv zlqx0NzEF#AUzNIlyL=NwfMnn{Yn#G{CjNSak1KH`{JNMwd8c2 zuV7$6iYYE?Up0mjBxvA}!Op!CqLc5ZJOi0%9dL>N&KoW`(Kuk!r~%854Ydu*2}w_+ zL7Hwby&al@ouRpha}>vsQd{|Za1;wKF3ry&Q(K3-zoCyvRMOWcQin0~)#8`_bx$4T zL`DcV%q^W5@!7wm((82)3Ve=AP!DVJVF&JILC0VcX03N!qBu$#1~Op(y7)MxtgLJ& zOgZhIp*k&Gx4qbpl@@|8ra=+mC|HUzsj!yrG7@N(%HK|c<@!=ME0-Z#7E3>K<=et& z4{a0J$m{l}II=S_uZP&P`{Clx;603^;~*H>g=gDOg_KyY@4tQ(tyo=uuUlD01d7Vh zQVg6~zlIK7SiscYS9hu_i=~NV&KSG*{`ADh&STnx;; zQ3MaXlx#orSJUVYNcDvDj%KQ>o=$p;J0AV45O{JMeR|ziOh;=TO=fJ_XG#YyIoJC6 znvHwov!Zx*P(Ik!P;AzjxYt@6jU+?6$S1U;Nk1j57+hsh1JQ!uZF=%KB+LljfE z2QHZ({9MNt)GkSr$pXe|Z29cwRhshl%kO-%#Z7Gm>^g#xmIzvm%B^vgk~wOqqj>Yo z47dp{>0nPZYJIUek%-Gsl(5Jpm&;I|m@n&c<2DR-vZ`n4wEFpX>l28c-}LrClOP^@5b zggoesaYIEp`HcdJL~b&sE`WaKFIlXd)7S_WB4E(Q*KkaW1f53hS1||x0RMb$VRIBu z4M6sWSExxjVgJJGNkMj9eGJ~o*a5CxTVEq7YOe+&dvIaI2l^<-1azF-AGP_9J;s@R zF9PLzwQFG0JbOO-I!@t_fCLsjg55N4@|(n?z>uH4&{g-amGQSI=%^WF_AOglVo#Im zal25+PLw(Br;bd1(v5l^glFnla@vI1kLOBb}sH_ z?dvGi&1?<6mHJll-QdSC2<_C%8rJ9qtYMN*t|7QPiOhBVov#Z`cp@@L^<~VRdkMd1 z+?tR^-|~)E<#6*k2i9_d+m+7Wq%o_Pn~*pbgneo9E8iT8m%sHza0eP&Ek63&dyVWl zL~`lhyTlxo#P@~{I-nvX*{3RTe!sI**HP(20J+dR7Z&M3?!g+Bc=ov!+WB#gm%nyT zH7q^Yxt1P~z2VW9wNFe4mLJPrD=w)B@_y=a0-tmPCoq+Lxb+Cp@|60?xU%S0i%tGY z(|zZ+3nDYHh}II|>w~VKMX7)u<5X0FI?+a#ko(SMnHoO+QFs@~Ns!u$z@w%<*a^5G zmk5B6Kb{zu$E!-4OAn42K1UQ56`k5bQ(t{TRGjZ5uW3|c-&phJP_%Qd)E9c=>6uEq z4;&m-WoGIHpOdCC)#Ie4G>JvHD$`9tbP70YL^L;6KNfeKH&A%jzNsQKq zz`C)lyHDvkS|NC;&AHD4nLY zRE3hz+B7AdyL0i=?o%8>Negl z6HPXjjpNFOs>nljXnZ`;iGu5HeT`$X@)PAk85_Se25`%IGWprge8d6L`7*xH%L;2+ zVF|{BI>XWNs-muax80`aoe-E4RI*RcOod!wr{PWtMpHZH$>D!#@x~NHuPf69L4u^@H(4K4;a%B=h#27n~0I__3*Dp_GNN(T(e4cmsHOqtp0io@LeQX} z&w`ArXRt*{0GNzZCD2v59bu7RL6#)PWf5*+i~vhazdLjNX)h&c9=8YW$LuPzvbbiY ziiscu7Ew3a@rH@(f{5P7bAlHfE68)>U7dSobmhEM?VP*!^I0X?JawY!IINXX~P zMfRluT^L%Z;<3;%0~z!Df_TIIb;ciLe^yC`*{WJBSy|cdlLL7gx5USSw;AP zWoYk<&qI{NQyIvrn(la5pInx`*fDb=a}#SDY9G5z<=e^?)p_=7dmG;5u1&Q0*lV;xR9a2 z0C0b~Xf~2C6Lxa(M+gKK-nZ=J%+Er(CNfCI$F5&%Yw(6>0<(I-p!(AQ&&tZ#W1BI= zp@XO#Dg~Eq5nCt>UwGGThoQH@+uK2=eUwC-n;v z3?hw1@o}?DcmE?Ytc(Z5R`{B_$yimc-kt9PR#KzRp8x& zg-BB!F4zh>_lL!GkM+bTJo6&0IY+6D@=pr1@){VOZv0;H9?&-OTsvhB?g*WOLaYP? z`eZ+R0Pym@VHB(qVap}t!DU^=#0K+Zd$;Z67E}%YV7`bgGWlK1fFQ!(z1%yU7vfk{ zrUR`qqD&VZ|Kgp@6U=l;-XNt!b{66cLGUhpqz*Tju0QA_it@>>0dc0TxJ?I%l^_y!h-#>pyoK+9=>8z2Uha;A=CiFDxj zdv6rhd0%{%H`zZQ#cL2XCa$!en#GoA)}Uo5oA~?|`=0u6X+iiv+-=J@d?%epM8rQQ zf?h*y{-0m2?x}k_gDetv&5vQjEwe1>^rc_dLs-oCMf;qY;vfdJf^t})m?ANi{zZ*K z+={5Q(%%sFECMCp0iqwY0uq_~C$C1*+n=DZsg}1`-JkIH1jII6q?I|p2r1(;o4xsf z19qr^xjYAE5v1DOFQ3cPKL4`1b3itg3)o{ootsMM->S+JX6bxDE_PlBXtUi2FZwuo23EZSIVDDtPxj z1DF=`hmDP*rI?Hok^d#(M7px?+WYp6LNnFsqw)Rx%%FCK?jzF1L!zW5WLAtYRc-ak zVG&ov{gBP_aSywZ$iQIL1|4))=sM(faX4 zB7fVESL2xNGQ^|V&v-yC6_naba>=n_%l95X)J3;#4~$QQ=JHn#$tHwtdKNp0wVpsadG z66*sxhJ0K2=dx9beh#EkEk)!D?jEzFvydQDs2>@c_yqUxd$TRXDOZ0?9mlGCrQ;|4 z0(4l6E271AE)plqB_&?nNz^;jopoa=KPlc^%c-1)=ke-Xx-^{_7!qw9eSxNNKpRo| zPv0*Dbb1`8o5`oN#&3y8<98G1Of4ncQ!%kC1Dt9rD zBCeg>kqsd?mbfi*H2yIw#l>ciT5d*aZfHmodpPMzicaBpY+$nxTUy@O>vZ3asFOum2{~^ML=(uq#N>iyi0RKy^#e<}{E@SXn13OzH#tbLdl<=(SX3LWz+K`Z zpA}qP&>B%>o@HYY9@eCRFStT);u?pfj?LO52Lo7=*Jo)Dm-U{W_dM+c|7_8^t2 z&rU42Z4;p?V6qh8|H|yL!kLq3VcRq@Rx0n7!6YqZIg) zyo~0#qljT(LCmU<>VoZTYo!$NZp%4!`?_rnaj|FnNc4PQ_9WbVRh|s^p8kK;-VJo_ z-BLT@Ns_`)#!fE=mOHdPX$hxgn!=FHe>-LLA!!Bs0qa5wg3+#lEOM`St!x+2a(CW? zwLe`EtiahojPTQoGeBYUJ7sl?pzZ-=G^X2iw1>FzW;om9Blab=r#;S_wX74Xp1}#@ zNZNS+G9RT|_x6TA&}ykSxvI+Sp(?1RgheabXC8;^Bg|ku50p1b`LVk%@zuM`#gc2P zxYs%9BF3H6+xD1|p!r5zi5LJMr7Xv_)kKG?)#jIVhm~m@T(2eS;qeu(0f7rsJm$Fg zJW-dlXIp<|=n&?%2sA_Em%FHnhE5jwH6^EnkkpI^DMJIln9WT{{3B!HD2~pI zg#>Q0o2*XkC33%V+LYOc6Hr-+Uxt zlg(#yTVwVeIMlLg$TDdD4m=5~Ba7Grx?L1%%TOy;j#<2xJVg!`kCw)K~6?^szv#x`7;^O}w1nWjZjL1r&?9)Jg1T0@Kdn?;Nd{N*E zJ$_!jhY(JVS~#AKa)4xwd~O*9Rma2_jG<>+2zeNh0E?|(!eYovWBESeUAjvuC0KZ` zkY;uBbeVBeb#}Q^abJ+1&vfB6LdJ!!4zT^|YI~}j6TADaPb0RXR{8qP&?P5BL}wA| z*!1A-WY>k+`zsc;QS`X@h~8c{w~JjDs%?&qC%`ed$Hi_1E^&8Khdm+T=cd^G6^e+L zFT1zmEyJ`N?jly2EpfilyjX#)E!F^=&j@K>$D)F$I3E&}kG*r+H>6T9loe0Qe=7<) zCwP^A01Q5uj|=>36EYL=yI&L_8sKzODq4Ztqmo7&uvqike#Rk0)%rUo8uK@kV1k5% z#ZL+C69c^NAc`+-x@|x!=T93$ru$^E_i8*C=cz!M68mGjnaSQAPvDy$d_B zSWEk(h*dg?#RI$D|H(Fjfno_`W+Z&mtcWI>C43N{7wa}7t%*%VV^H85Ep^02s8~(z z$K{HyG|y;}WMSLm^4{KS#+CMlnPy@J?JIso;PNL|RoYXO{J5K2<-fs4=K@@2 zE>?_DQQ(K9`75=W3Z4>0j*#}5>&)Yc`{xIa4ong*kLK~oh52ZEyvKNE2@W844dlZ! zhmH8jTG>vy&vDdM=}FC>{q2mZIvs@M8vRR{sb)Ges;Ik~X*;$ufvZ^lrHzyN01grw z-qG$B_y)FJMQ-PP{Bx?yi6r$YXkXmv?DMZ{(Uco&6XMNQ0j5`2B2SS71(TfcVuUxa z&j+K)M4RaVgkr&OIEz2w^IXhXPf`qY` z@0V){0H(r3nq4-l5#;7bil(5*-d-j_w^9Q)n0Y@e>c0y5v=6gSxKH{dn-=p{VxH0FNh^*q{ZAUPA6?csM z3xXdz-I8Tpi6Ijc-8S87INx~c!;^Zeb39wV(!3-9sL~k+1Ri0*Adf|wT{MnrC5lSH zVn$JaJ%N*wfOuKX&5p;Wn8&`3UF$zLY{_M@>CW(GX7Y_40C$<7HWt_i5O$sEVC@8G zB-t!*lQik%kr-%Clz5NVpzNyO4iklJp|*~Om7wTh1Xj@Ah8Q2OxX89^5IgXHFi3g5 z*v4Eec}I(I~v^6SM6K;gs%n1jTApA!~?P2jET2~>t z?MMos6jzs|_!Y0aInEb*)39um1EVxfQN7dl`ao9EW?>MB)s?yo>KruxIq1^ArG5vC z#kJjOV3Pn!gx7E`rzI>3|ojovhBjtxZ=LPS)sAI}oSi=mAOe zb<9UvlxG;^Z{o@g7h_jKUBq9O$djSd<%=Y(VzghkPZ9*ilPl7z8X9gXLS2y?ojVW> zQUA#$ogY+5;`_3Hw`eruDBWR1!b8`nz>CWxUXtIJYiwrD`urYdcXqxeKN@HXCL?pR zuo%_fd?ee+=l1PpOff=X1_2u#8n(0q$q(!@j7E@HjC5f36_C~Mo<+31!EtX)ob(=! zXgxQ_O=bqjey_TF9s4f=(5Qp;>c%!wF;~U%t8GUd;(Oxljt&kckGMPy7Ydy`s5+Lp zYNADQXGQY(MIkBHc4oiS_$C*-u}^Kw2}Nw-=K9?Se@**+3{Sv2TR>{79ozK z=Uf-XGi*QFTrebOv8Q9y5h4G z7Eb%N&M^m6n zm_nEWyOtBUa4tK0%Si|>oqsKjB!M4!jN?5_vunpStVHQxL}QC`hO+!ddTU$_YiTx4>v^- zNWG=}l~mK6;5x9yP^tT5n9J`}Xjl~i3D5w~nUM^~CD@FNjS)pK;>bq_OdB@Z7U4E} zw;Ufg>=@N=EVyGg93wIe)Rilz=6k;$CV}2Ch~FNsbJ)!^wR`*Rj;~65E}-il^%HnH zj79Se>k!{>#PXW(TfM&6Wr%N2(l+GV6hCU&p+WS$IK;8MZa?;2x`PRY$1OOt8D`$U zs%UiXw`)H3=Wpb@c?_(0x*!{UDSovU?}s1|a#=8H!LqL9*jT-LWvXGrXdK?}^piLU zeXaq>3QK;~@oVY#VY)Rj*CZ`V8LhPid>|+}t40N*`15uLfN=(lmsae|m5?$vrDszYbdAzE13+mm z7w>KAs!@z==Xl=3aU)reVosB^%G+t;P!)}|c;D0Al>=91SNjumzeCaStDbw5o%`8x zsn6l$_F<#M;3X3=_Nu?Hx?Y>TWmC0+Jh`m6Q)Vfyp$G$kk#J-B7cSXq$Z?|5LFFs(2fIc|$ zTxBxi0Jw2FE>P?<+7Uuun4xhyk-f549kbU!9wt~$+aUl9{9_XHmWf7f2XjbLSgc*8 z<2q6_ACdQMn)D98bbDl_47t$Sg_DMe-B6hy|2!-B@UQ&6^70aARyJKiATbD z&VIM=NVG;K%zq$(bscyg*Iu1HDt@jylYkINH(Mbei_jDs00Es@R?}MTlgj?lZcs~m z+`+$p-F4pb5im~hlNkKh=OSqGnz~qVO(3henM&=HI1A8)nEtVMX-1>b9)Q(SO`%P$8o_A zFOYrnshrMs6eXmvwa%;px7VAMpmPoKlBtkPTo|w!%{C^B5^s^@jo1~J?XR)X-@D%g zSiMw1buQUgTg#Y0g!T8#O3qr*56!c}s72g5l_!PCN}G@np81pbz0Bu#tz)YQuROfq zhsM2rTkEJ$hp*_}Zf>r#eEo6j{Tjj)OA(ntSfm-GZw;LHem3}S{A%&mpYj$L;$-E{ z=RwYVrbP@*TBBWY?7}HDkiQp3bZ-PKu@-gm;zEzfE}iLJ>|XKNN**KGCaA@J5$>lA z)(s)$9#%F&4Wm}w*}`J@!Q5+_@b(W)6ERB1O7}-dr8W+~Qce5tPr`H>*-J=ZCJCPu zc5Yy?;yk#_wBs=x=Qled)%9xB$VE7kEJ9jxO;VmJA3GcmB!)jh)~4$^^k6one0=0{ zh-WpFnK9$U?+|huRJ1@JMLrJ4$1oM9-*R)LjC7Us?RIr~k^e0!-IYl3AYd-YY6-1Y zxnRUG6q`F0RmH=wsSpdWHdE-pkz*kYOB@u%(UhjPpk>CvPMFc^oe40LOWREG#PxoJ z^A#Tzz~RH>cKgL86N1WkdZB2!(wNc*Og{8<1%`+AQPQ(V$p z79a+Y=YKPPoI0T(OiJ~)yfRdzo#Ym!A#k|cHtf^Ugjlzot;ur9H#0<2SQVME-!1Y4 zZ+Vz53W(HRB9IpqeDidc|3TP8oC-|b*AAnHvVpLF89!9T?^zwxtjtad?l#``#rK`G zN%&HLkb|GR%5Iky|0{nr5@ZK#xjkPKc}`LEhZ&;;R$L;&QasD-E+yK_)f^kF}ch8 z>J~*1NA+={VzQ&%m2l$!(R9v%b+ldAk8PU`8r!yQyGa_`Z0w{_W81dThK+68#y7p6 z_xt}OGw00AwddMFDw9`kb{)VD7L}>ONh~7dZ=W3u5{Z4tL*OOb&kdrZcY0m|V`Y zufr^jvZAi;Hw@I`uYHHf&u3Z#2aP}#pW_R<56S=A^@b+SWY-*{LS;QU zjS$A^qujmDE)$JH7>hq3|F-zP(MfrI){x;!P}>6@JUh1&j(vAEEs9K^Bfk?;7qAml z?|`f4(8qJG#FXgJx7+=CIeTgn?_Vm!(a!KT5yROV!?{+vNxn&8+}Q@G14;89-UZ|M zc#f`rCWjc82j^@j$}-HC;WFL#JAi@mwH(De&ZCr(0^<9jRKuCqm)}W>h;57**g&sR zU8go6KMdGk9l>4qC<^lLI38T$58=l3p8-aZcYF7@=9^zVtrm)^s!48!qq5Eb+G;nz z&PHbZZLNta(mUHkXxxpaPr3jF*KAo1jS=Gy!~?aH-ED$YG-Pm?XIRz_e^v=jNlu#^ zcLX-Ta**RXB-ZnLcujppI9Mhk9e_G3htVB12?q3N{SkAD3T)JZs_V}R zcby3BQ`!e+QqfWn3PBm`9@~ZSAhj~@<0TD}5CF*P-#dmFY&#M{nn?bxCAx0&nOce* zk2YC+9{~=RPGd_`Q=Jx_1%cJ~S06{mt5qR6>LqewRWy$s2n-rPd%~?lz>I6O0>(!a z2hY%IG!wfo)_WUy$oAO~eZTe~NY$F-id{L|pLoqRS$6lXD6UermrA2h_7n%a>#hLi zVf4S2YC9nR*hM>gsGw9*{u3x;-|{_-XS^`DC3q36jo|Go!FKOvgDrqjLXP6o+hy0) z4E|f65e9JPlnG`mxbN$jAl&DdGN^!w;kvOF`vP{5t#Noet zOUrHyNLsmNNDG>qvtgbN+y>?oJg#*+PJ(6t&S+Bx>3_td=-3Y=1b{bWiXD(8{mok> zoeQGj_5jz7t3Q^EujWiSvJRyeN3OtCG3GjAKk`4AMtg|th_xeniE zD(rEmTW@}<{csdrHL+W5M!b~w$f@rV7s7M}`j$Dhykc_!k2zx?FME{}(C1FdolF$? zS(Je=(V}r=a_EH@_nmchP${nOytF5oQ{MgBvoh%d1Bb69!2C>(rTF*M_1LaJN7!mn z(SGOm4Pz+j|2;6&KV{h0U=_Uj9o-sum6`5cg58Xcj;>fo0zv;>X5h$>F1j!;qAP)c z#QcK~7xam@2i#+|`dm~-4$i+>F&MX8R8`UGv}yxbgL6C{o&HUC_gZ?MCz4MDTmw%r z$gf?0H4p%>wFWBtPBaS!YM0ho!f*rYK2+A|I>`jHm>9&$w&#wu%bY1;vsUAAX1Oha zVccXz*q}5QF3!~OTr0e<#!pGOL^4x{lB}SpF+@R@KgIrsa*2+_N;CN%C4ilPw^<`a zR6@iNwy-Vd59fbv=u9TzaZ=+|6#H0FK5d57tT?X(aBr^>YiF>$jPJ`6umuF&TCGNb&u5@zg z^_v%u3_8*M`*k@~>Rl*J(?X}VGI7hy)^rm11~X|~C->vwD*ihU1jzR1ye(ch)f5_b z0*I+VQW6$5?Ef%ECU4ZxTof4Jyxk{L?><@4W~yDiae2gXSL_LpHtOVRD8Sy z)&-M9PvcV!dbJ})9g=InyFU063LiiWg3cZc8vSG|C2f0xp&fInZ3B&3?Kg`47Olqf zj`iQ|d2IH`F>rSsJ$2w#7pFd60T@A3%iCQLjp9HRePn@;pg<%@${}KxZBwS@??9T> zn#fWAaGUz~1%O`$=}R8`Jv>U7OS5z;A0W?WR)tv6mTA~rsr-BztLc2}$$3hWwS)dw ztG@VH4~6*k^ytUvq{~S_fr0X7DXaoRbe3cpfbF50xX;v8<`Nc;$M9oY)kLfhc5aG| zvPu#a>H}C*8k<~1y_5m)D;*35Ci6CYExDv_4svq#?c=}86G0%b)Ts^=1^JeU`D}9l zEjVHqAwSRD?0IXEaOfcQ-q4L&YYr~{y3BRP&5j9w&x*yB1PE88~-vyN)H`5{OEXzuh^eFZ+P|2xP`S#}Q5_f>zc zDw$$o-HEG^+O+%@Dol#nOEOv%r>gO<%7>mB&}fC2L_5`wRaR4c!j%S$*UJl2sX zAtB-P)zWSbwg|z0S8wb0NX`?sX6_a{*!McHmm9CI6fh>mwT_O5`v3flwSZ7d-T`cm z;Uap=@2>lRiZ&1jxw#?_i{M*C7?gHw-`%2-M@eY9Tohh4h~yw@@O7?m*0_v)K04hj z%Ce1$6Q$WHv7dh!Mm|1ps+55XT{VmF#!)1!MdrBq=azb@iovhE6xKBy_8W3LI>09o z)l*CFPrv{to|#Gw)c65Gh!v&FNHIW8VdDUU0Nb}~qb&56K>K%-3TILq=#<~wV1k6j z1^7SuFkX}#9;!&~-5X&6CE@Jpdbd75U!=LI>Gx@MSMz=T{pqJ?U(oX>1n|dH3BW4# zx%*AvHoePQE!*p)tg^He=HYTPNB3f5+`=53nM1~Ll7V1dgXP-rRfN-96^HIybZSgv za=Zy6QyU?Vtm!?hhEfqe_z?omgOs4AuV```Bm$bXUsU`JcWunbDKfNf=&IrebL8as z#C^n#vib|N$CR&0!-l~q;XSm7d!L#_{n(dL8P{x+t-EQQT|cv;6~!tdPI`pq77%8G z#sG4&@bYc#BU-KA>k%V2Ze}^9I5Jqolg%L-982slamSFa@5hZOP}qqX4}VRoplNwh z`T0uDex3%BP{sqZnu@-X5ni?Em(9;M=%qELB;pPC(fDFpd~IWh;2EQ42wrN@ z50z!L^)J#!vf~yXUXD}PUy@cRq?3DcRQE4mL|X3ue8|XB?I&XFRx!WA&RbdR6gvml|uWnD*U&fPNEEFNs7bohsif3HbZo9HA|VFg%uQ zmv|&k`WFHkxmvw11WkQ`6#9S7yWvz1KBS}Ocx_yhgrRvlEN2z1eF%igmXHv3c6MVX zCFK46eHB}}DZREE?+N;e?Z`+^f9QYLJTMp<+176n_Zz^@{26KtGt1Vb1yB`)=&)vQ zjUN&>snu~3n*x%Y|7N%a?UVOTXG)gm6_yk=7oJ{hi-7=E2#pG=KUp3=p1eoiI3$i) zV||U6oxlU4pUv%34!mwejih@y;2r1~Wh$SW*BTFC1q{2T-#~sLxLwsAz_5tdB}3#u z##y-Z@42sYo@^u0<2jrIYX$ESom<+sW9H z8&JZ^z4bs~@5qQnz%l+p_pm_sHB}iHdiqR^`+010CB~rPGf~h(KJ-P5W zhZ0{ULqta>tpziceAzn9=uwF?gAs`J|;3FP7p#|D{O@doM1q}pgYImijkSE zm1c&VUkkyj#?AUdSFN@w34!o8|0Gqj^wQ*2>GkiuqER^?L)cm>9@%(f50l9{`XhFV zvB650_W21VZ14rLQf`68eEH@d3#;3x(pagzz#e)V=dv&>BsZ7JvSqNoEihkFBM0N- zufi!OAph6ArA$AY7l-oMpSiqDtkWupmME~fJ(%bMgL0@`PS+vH_PLQZ1zu{M`hp;D zrt?I(40%~AOZBc91uZQ-4orz(kHDcgQvlxIAcPUT_oltU%RTnzp5n)zcqSb!Z38b4 z&t3w#B%fn)_>hTLriv7o5x8z(1ri0CWr7?Z+nz>r&RAimUK5LYd4un_^Tb{H`_}`$ z4S)6#S7B&p*EsAFY;-s&cmSs5Vls-&JXdZEof#fAJVyzIzE&W$ zzA`Vq_+_!0pIY;&qR`MxY~`|LhRh05i{!g*N+jVQx!uXCxN%KP9p_V|xn%GYH-0=NoVK_{t&QG754J^-(R6sUS%ptO53;o#hPW<}V*)1
4s{_FK|*82ekZqdHS{V5YzUM z%j-XX{pG1e6ctH64s&od((>m{&4XDjyX6B$(HG{QO+AIvURcafg-QA6ZJ_t!>eU+N zec#tEceK@ObXn7+Ku*1*m&*wl1};8KPHI+i#&uH*Shqr28EY=e{Ib4>@=#)(#FlxT zwOtgxO!SKS#>Umj+^^1!6vs_98dd=A&c6<1@Fp5e!qqNG5kTTQ{?<+u=S_j6Ew!ez zinOdg+Fq{^z-F}kU)uObVV1l_!#I`|liVzn6@kFon+0V?Ot#~2n<0zO zhq+L6`aoFi(xn>ejW}#b&}uFM}F!6l46>4x$-j4#|Q7&E@#W0 zJ-&$_<2d7!77g+0sj|ZgvK?j#Un_UlU&4rE7L2)Jq)cVUCFU+a;GbwHyfc~4^Rbk{ zQyxv3bmkr)7f|6u>wlEzh=u8jFIvgzc6^tPSH?`cNW1xZRQ$ER!)(Ig72}f}X&S4zLdn zJEtt@fg=#+NfUe%FI4am-x^2@j5OJ9l7+Il0x)WOhvtBf>-Q*waTP};O-ZWGz=RYc#(r-#xHS?v9j|jeJYcsv>am* z)LuhqDnc?c*)ugkB>-U&R$}bSHi8!HWn3p%f|Obm&J85O)`QtXSH^s=X0_hej*gC@ zJkhY+%F5^p60+0dGRIZ^A7Fixq^kY zG0-YC#Jkd3^r9fw*6PW_u`}Pf5rs8v^wvUR6j09)3m!fPnYtQW7^_9}ab8*(nVM|< zh}42`{I4FvW6Enl_x!#;MaEQP_uU;9K2kf;2keI;BG$Kg=D-w067at3$Lyj4UhgXuBk zA1CVisHe`TSx|+aufR+iQt}IHrbpj$>SNbRH@{B2pj4I$J@OdcoHZ;x55~&d&9;iD zk+0^wA+4j~3zv*f(}||6N$N86!o3L-`1N2~OpLt4%klRju9Tljl~`YZ>N2?rPwn zLp(Z3DGkSLKG@DSy;RK{J66s3I8nw zNDCVp{G+Vd24kED4Hc0sX2GO>YDPW~&BNulqXGIG6iDwO%Kb_~=9$I>R zH8xGkz{kd|cgI?ZV!8B%cQC$M!&d97&z7Fw8j_$gDy@>!*s1UeO@Vb78%k97x@C{Z zA2dR=UkZmNoe!Y>tMOC4eU&mG3TyPGIgUe;X652x89Qlpq&~-Z6Fja>tt)OTc}0AH zcVlv9Nh&5vEEdaQhGs3DlM(E_Ciz$rfih20`SP7`tcaxJff04Vr1m`DX4s!^fX)KP z2+ocvNw?XI0MZsJcsPsE&eMJVHE~-NORo=?777TWUJ3t!*S|kW=nlwh&}xtvjkp@h z<0y4(2PEbA>s-WZIz(kz+W;vKGO76#uk9=G#{hoMYjQho5f{!^{Ij2CZYt|ODoYOL z7Gj~0Z_75#g)n$F zi2kKiBAlsMYZRNj6o(fMv)>exY_9^;#gJd~k}Lp< z+GybiOXy<~IVb68->~*ZtJrcNCyg_$)FtL>;gxUZU2*;ByTO3yV*!C4NZvY$9yW~T zRX{&9hS--vwmToW2;m@TlV+3UufFO<4SvmZrnM({(|92tU%zx_kFWUlHGRXTr>9ra z)`oq8OT%Cym@E&-|2sPyTorqkd2#~XeP3nI@4Sw<_&LosJ|vWh(QM}ns`Gd;zw2Go zK?IolC|rKyOfcjY6o{UGqXl-ZCd&&Sua{kysx9koy5E7vM(GjDXXVuRVy&&&7>5i6 zCY;;(U5Ol|RgL-@vrx-W^xk-?atjBb8@ZQIM3=i`aCF=1&CLg0Aj&z}+_hPi zy7JczNHHMzu3_64q^3r-7^Rt;PCWyTG%ejUoi0cf?U}AR#e&p+$Dc>BM<$LRm#U9@ z%J)dKp@33gv7-MtJjSFtltT`R{NVbNO; zWN=K%6L0L^gedJkj+|n#P#hTslh{y_;)Q9ELvgYcTLYPe)5X|Lbh*l}@+={ipSaRn z%?c6ZhW_s@ia%4QUe`M&UkYk)LOp3v3c&Pn`u7#sjSaF@`<2wHk2)sP)yYe;(*%3m zvljSG8$!N>NAx#Wxo=o|?Vp}UGzS|A{aTfnv2gnXlHOP(I3!%m0%aBnG1dJ0Cxr}; zSlB!Rr&u<6U7D8*cFJ)ii3Q1-NDCs4euhi0R&orp5HYT*19tF$_NVy@Y+s{dbwx6x zZp{gQHnpo%!o5Q)=m)gxI>^|(0XKevf~WUx{zVEj@=v4y%HDoH>%2#f(00CDzmridm1qRf&g2@4N$i*W2aLR3#^!vnO=k$c+ef_yb7%*Xk0&%@N}^nraps!uXs3%CIe(yDS;6; ze__zHHepkoxsEVNj)q01VEQnL5KxHkppKSFc%kf9m4h!pe2Xcx%A*s{1r-UKfw}*e zrT$RT)Ho=+fHH;Qctn_JX+1lnv?=l0OSZdDKPXI>JN|DzS551fMIH68G`H?~;)pDY z5TxEA$k|VQ871o^p)P?(n^-~BNOZ9#o@J0oTHXB^*b)Tmga9OOTu$W0zBe_&|BA|7 zRaBv3`f=yct38k1TVFp531!DjRyInn)1z+Gq9FyZ6CciCVI>(9JdS3xb^D~Joks@^ z&KDy*2@Vd|?sNFuVWi~}Bq&Y6#m$ZXcHh7CJ__0QG7pK}s=yJ4RqW+Djo{rgCV49NX=%>CTsYnPs#X^^u)4i|Dj%s-Hl1z4oQE zX&fbC`=+p-uR?7dJf`maB2XP^Y3glH+!T|!R*91aYifd!b(X&*(}hr^EAB;s>9dCC zG6DKAOu?Yw`VMM{D52AeU}SUJrP6HBNTwE7re*$jf%<+vP$f;=GO;mJS<)w&tV(If zZ>90;wN!sXC6z;c6_?Dn-PE=3yt&_})iBh@pfi9M>m%Twqg_8?R5(KBVS|XwJY;U? zG+K!uV~@Fg(}& zxF<^VD(3LZs|O&KIXY(O@u2O#wslUiZ5&z`C(1n7csE=Zz4CN;??9zmQF_QYB(N1_ z)qg3Jqf83jE3=WYKuRanW;oD|;{A1&nt^BA$j$exp8{iI9dCb_L;N=3tc&mJp!t|q zbJiZTd{w!kEY>w5U*PD)i|S&;Lsh>Jc%gi~D4VnhW4v4Eg}4CQBDa}eQY*)(ZGMrt z9|-E`y3u*lrV4GWz&}_4ZK(|18@$MHVc=H(W4Nn7WW$M}{n855+x_=0`u55}CiyqM zOOZCxt-%W%R?$=;t?)9?ELs+@wd@32ApJ|xK@Q}bu}lf9E$H!HL{|de$JS)P%V7rQ z{}3n#lB2{2QC+LUq&TJS4}v%IGWzp(RS!AehM1+G?2>b&DWpl%Y>B-?Iqha&3Hw@w z!iAg=d1Z&^$7uz@62~6Gl52fbjQa+5UaE94`L}UWJ${e(6+;GxKt5^ty%!Wn8uwvn zOq@4hJo2}XPmB!OgeUQ=&P==wmQZq+E=jsps_>Vm4y1D};#zL5d;0SC#p)2-)#ki# z&{zU)Fq+!%l*<&`RTW4)0AV;~We0$dZa~qj+D#w3iLMT)1pZG_s>4c#5rn~~! zc!7r@5w)L=vray7veVpTVJmg!8qjNY=8CHT>QpSCDVE>fu4A{+i3=@JMotMkTRwiu zK9`u0vsud)jd&8j)!o^@3=<uj-xbe{7&;PoHyznzQ#|;t%@^2z`r=Tt9BIs2f*k;AXtZtUa z0Tpg(8%^#=Dm}4IFO$EYeFst`5zkIAJkFKxrhj+Y5K3!Lp3q2m)~e0YwKYIrd33%n ziCVLT;Ct&byYQu%uYW9i{B`NKQDc7+S{JZt*UA&mYw+gt`7UJL`%MgWcRL zBuTJuwWdMi)E3pYQO9 zOC*ICyjEYuBGbDu%*8w|C0Q5=f zEHe|0o#kh5FcbT3EE2|J8OxStu$kWbl5UDd{bb?OTE56&yttUMZe}ckzd}WB!Z?sx z#Hjt%?o{Y@=^xd?s#E*WJ9&oBYpBp`YoE6Z8g5k<+7D?|Y#N7Vmts83xm-iD`0EF46+3nl;B51nQAZki5R+gRpy34gf`Dra+iy3eG5 zz%*2&+>|;*C3Q$(??Th-FrA(w2PQ0stCVY$=HzD}6UKDt{#j`CB?%e4>R_5?& z&BgY5=Q@rU;sW;~2?FKAs8h zVWNYfCV)Ty0*!`&5kL(#FvRjankEH?nl<9e*^V#A?nBxBANS6hH7}AfUmcy=?o)J) z*I({yU5-^q!pkC2cAjo^4WG`6h(DjAw$=n~CfiauM1YitG9KGOpq1F@VSK*Qm%(JI#r?$$gA}S%r2n_|G=w<2+eCh> z3_is`{wsyV!6<1-NjJ6@@nk|S3!_q8;Dgv2^0t$Uk<#OxAyT(LLZvr>0AHi*fT3@@ z6!eiK*bo>O01dWRs_QlSF|R*?E(NqAH{;2I{sz20RZKx_H$ryBVgwUTwOulA7oXc-Xzb>0TCE%$CGQkyeG*?jfT? zW-bC%MEuoX$)c5MmclQ=x1eG5mu6yfQq=0X6oMFn7a=07Npl|LpaBmkB_J5r;~%-4 zq?MI0{%Pc?mi*}&$45CwY}%i z9#au^Mk5=Sx_ZUvt?V`lPdH}u5rT%n;PM9don<1OEc*&K` ztS$(qavD|C&wFr6X-HrFQ2F^AmY`D&GdnmqYd>t>-bXD5AdbnPtk%s_YyaE2{&P?-pgxj_VnwRQkAwrgPOs0FvNuwb!QRWm|SFmphcPwP-E=&{o`!}{tPu7G52 z`+mw!(!bvRQz4)?LH$b}G$E~e*tZJ3zQ*~=l8>1$?3(2uzuO$ZV7}HeKRJCXK?vTj z3*u~!k+LUW?+rch8wBBhV;+(;uXmz$hs{T3?sZ-m3lWOOrJ@dR`Q!;7%1#2psF)bj zeiTSP3M()CBvqzS38nCqU?+6QxpE|BwXC#+3%qfcZMwSLXgJ+BjUcbO>a|tD@9>d@($DcpEdF`5Frc{xC?o~yXF;G|1z+2?0v?yTCqfta3lxd}ex`+?F?OEPp)4Gv9Ab(Seaqg@^v574A&nZ)qaOuw)Svy13h=|0VTrOimur#51$Bwx#sr+jyFa!d*r7aGitf#a@a z#S$JTLRh7ZE6??&<}hkQ@;Y@=_GrvsU1Oj#EkIhT2;gA6whe`llRljCVGg53W9a1c zGk7LA^ZruTGR5#6Nv!3=z)Bm{D7ILa1X;+Ydod zVJ;DMvI=>Q;+G`IQY{>RnqUW|{VAnMqgk;D09>w?c5Y}=0F-lDvM~NIKi4mKe`(g| zDW>i;XphHbjNlMrd3D#w&nPx; z=_%<}IDr_G;!$tmr zLK;xHZQ2)dG|D9C9yaWd%iZ6sGl&u+;aYPT8VdFH+)!Rl)9Z5wk41OXPz@(0^6eWH z1~&GX{oMTd`-yE!R?oMo?1_@eidL*}Skqa}9x#9)1!Ln2SaH|eF)!ZZythJox=<+- z7rMNgEWw})D=X!Y0d`)Iq2dpHUiSkC+zmfQtrjE&j5T7S$7=l!lFzge)6KzA0^Wzo z&-WWao9^3JWd6B4%JalTw7=`?RYUNh#+J~b?i-#|sT^cz5#d^N#l&sZKbZ^%h3-VL zP+B#<*^MxbkpYCKb6h7=cp8d56_-49%f?Mqm&|dC*Gzcuq_leMDl^E}?Dh`?Q`oh(M5Dm77f6w@DV zp5PQwM!if}!M3YNzi%fi5|2Sf(Z(1XjNrG*pZj+>7SIvz8ENalf6&h*Nf6l!uDtw- zQuN(En?L<5cbQxh1T~RT3f}_gyz;;d6uMe9{jjk1R=p(ncSen_PH#YG#~*_D1<>HL z=VOuwBr+@^st_Mz(Bfi`{h_?zN1Uw@Ta+~P!pXS;o#l}?Zs<;Xnd*|3Xh^ZV`yy%gvv zFR~exZ!M{O_9k)xKX^8TiaADl`UOX8io2BA4>4L;tqS}>Hr8G(Fk(ba_DtUTt?fU& zOqqn=AAVy&=5kx0EP2|wi}vq5=;4K2%juI0opd(hu9|E97N?SJ4Y)&Pl)6@W59Iy} zN_)=(q96b@9~Yhu+dOEDPQU|XvLu!oNp#kE%r zEBZItz|THXaU?Hzfx_OtwpH1@lGgT_!uG8>0q&O64&Cgc0d~7v=WOQz}(J^yn7~v(U53;eyO#l-f7u_gfjZ|qJvn#W5bh2%Kx;gr62Niv9Dac z!icZWw}5NsAC?lJq#(6z03GbDKNZy~1IbF%8{L;}j|N9l2!8h605ouGm>Guk-|XdI zheKzL8OMCPcH^t~+ zno&9DA^P%ltknC*w75h5-4y5w@|p0DYN%jXa3_vTs=5*BYxWZPkQ__-2BS_pR#uJm{7p?n5cfR@+NX{vZ8dzbZS8hFMln3mk0S$oKOmZ z#uN>XpjL+wBL{R=k_VxhJo(q_J6a+7Z&_Gub5>c30cIxA|sV4?&EAt4Yw2L%I}Ku!lGDlFL4f=;!Z?+|drK=6ip#w^x> zli2CG!@G|XLcmuPJ_$5N5^4=0;UKEaUQG;;=)L{`Ft9bZ!uwEDHo(KI_k4;-X)t;Qzj#Q#zd5SdLXvXjL?S%exPr zJ{}|`Ye&`bV-&)SBk9;!yN7zRKj_E$GHy#=nVuL&fXnHq6GH401hZ*xJ);N5=Kxo3 z8Py~D2`+(hm2&E0=q_g*1tbI8FR4j z+UmQ{(PwTA>)(FDRA@_2eQ%=Bccm^*vrKpb0OJ>G_oW$k#=uJQ(VLlJbY&{Hmrkc> z_x{CDj#M(;4}i)sDd_6O;C%6cK!!IGMpr9s;e(w zwB`j(FR55ern5>C3HlMp$jB7Jso-6DUwRWi{W+MJQj&>?Fg>m>;sNph4OjU7I7MJhfjv(+3Xba+mzowF;mkEYt=*KmyBeG+-c zkR7nSe&X}-Mm12=bC0d9-b``b@37Q-uONGSdCBKuIb|1&Z{?aKrENiJ?v&k3M{6Z3 zh50<*h$qdOVJ`I4kR{PF|M$LF{{TK5bM=q7YomljK{hRd@VH%?S# zji~p;&su6GQXTp?0P#5+9V!XH>$5`rwzkH*!zqpj2_N8VG?oGrw!Bdl#34AScs=!* z916<&0q47pfivQ^DWCvnx0-a;5d*1{+shkbWaXlyL_ho$gBs>HUE>j;U4$|+M{H3N zXiW;Ha$rJ06@i8xh^OfN5r>Zw)O~@-?HF8Q4>HvzEVHfGCAZhcf$-Hv$@OvoEV?y0 zJKNmMgbYTco{D~guPo>kFh9DI!{NN-5mQXR5}*QenWy+`o!_k)0|uYr%AEnMNxuu< zfVZ+HC7NgoHaG_{kfU8%4h4>GvsDltt4NgtMrOcEi>VMr|Z$7q#PBTr6 z^caZ_&;Vc=ZG45h`JD+Z3cP|W>@uexf{oaduUS~>( zom3^ZfQQOVQ#18k;rmF4XqXGO;x-Q>%-1^iN?oI&M0!XCMXZRtppPYwqT30Xtgs6O zoXP~HKj&>E?mSV50`yF9E(2-q1O+I9r8%SPC$_|D1MJqFPAu!#wU6oFT+5g8hV6r@ zryu&xG=P!P{4Vo=N3S~h6>O;D7t?#l8aj!;;4vhWD4M89HqJ3#T61+%)%v{;t^xA| zabunq5fD8<^aus_qx9%fxr({biEx>YG{jPQB1Kx36v6W60)CCp`(5()>RafbZ=eYS zj*{^XDG21QY3DTJ z4K9rqssfqk-5%fS8#=O_(?R7h`_9!8CR90s)p|n<_sEit70eID>35D)l4?avcvV;_LUj6lT4TMnGH)_}{PB5*Ummp@iVHhbWvfeSxoC4~lJldjPi7X^z1w zQT%4OtXgcGB+bUMG_bt7+~&9%0NC7@V4K!sQrzo3oF{&9@k;Cj*B=ZNqf8!U6Gy+k_=pxF7@5r#o6@ z?1oOKP*(t5?;t~6^vmgREHbjHtz^Iw5~1xUnrBzR^~g~KUSY<-MkcR&wT->~lx?Vg z_kYc7;{jd!p8Fw4c`!B*$Z_s4-}UcplE8=N%vpa-f0>Uc&bn6QB>LBXewcm^11 znAXD$Zx0 zn%ce^*nU0>X1P9ivYYQ?YJwzyXJIPn9;so!lmr9ff)>2L^(FS?2l(<>K6~hX-#Yev zArYSYgSxz-IQ$M6?blh`hyRj?c|jtS$ma&+^>~1aLjfh~g;+5^i{~Q8hKFZ`==Hri zwHXJDrJS980yh#%><^pgpL!qZS}`74o>`6tmhQPsn^l3)$hM+~9i-Y-td^)qc6F9^ zNTsH&@DhLkfRfg~<)r>GZ?4?r`ie{bnp70k=kTI&1^t=^?pkJY?n+tGB)7`fq0;H*v0Uc36ypU2c_ zzl~!q{&4IPIZb|f&myyueGg28eFii|H>H!9I;lTj)Xi|If`!IyZie78qhz+6`jkvU zHs+AY;W4M>`%r0cvNTd$6Qlx3iYdFk(mm)JeX0z$;U7x6{HA}B{l}< z;b<+R#wku?HUi}w)XvrgFD^|9B1B6hZ2p%okJu3S<%@UZUL+=|)KKur*X<=f&vW_$ zET%o}_CsD~50BaAslz1a!^w~bw{@No(L9vWt)h>*Y|r8J&xa08!*%yzT-yKiHq3Up zrl3%#f**BJ_jmU>T}Xg>b!^4ah%Og}M6hY#Q{bH~v*9F@bWw8g2k%8IxZC+!nLauS z1~kxLI_kN!FFTizL3Z3%*R+O)bdkY%XeB;(HFyI2mZFi^d>m!22EC2gQ;)?xIP&Fi z{lYMgK0vsi|05Og^XODiU#=8ol1`_Rjt&hEFYhZCqfsO-f%_1AKe{-xHAeO?Eerm% z2Mhjcyoy{ya8wQ(|##S6^rJZOV^yL_B@cvsJ4dBw%T3>TI`iJoJ7 zeQu&yQ}iK?S;?{TL4V6`kom_z*9b5Zmu~e1xPM`%V&xm>}(h#NU&Ky`99L zY=iTBNDM{kya`}1U3N*xXe(o!dGlT3ed!ujf+&l5`k@puo?Q_3t7o;-d$9)h&}u&A zmr3tz3gXw;CUc7td5aR9TR?O|PV4oDMSDI4?v}-E94 zHNe~NB+tRb+Tlb`8-Hp_ACkD>PTpgSU}$#>4tMU=V1QsGCLC@{l7r#IE@7?Pvrz;J z(9a43*iyf^ASHPickiG`5lH9(zZx#v759^Sf@?pU!guaN5sj1GuSUK>X$D8YmLjqb z1Fd@!EnTDOi+^qu-kAjowFZ-qNq1gQ#6SdQV?|I>ha~?7S1QQlfI3KrlfkEZ(lbg@|}{wze+$ zU3zac>%2K$)dh!rpv$ znG3k`Q2Ncy2N*^gF|s|taHTu95-!!Qlc;jQOkB2_q8IO8;2ECE+69*~oiATnczJp0 z|I0I&G7Iqf*DKNm_TyE@lUz~ehZ;5 zeOIoss~5`iC3dyi$ed<3=bVP``x)|#_Vu=>gd{IqUyNSB7%e}m8@$-3?Ku#$o$ly4 z-@)OCJI`;{F{kNCvwr!-`C`y|#Aq!gH{uw=w7nRtEIBH%4{+WIvWf9eba(8Y9yC0h78cor-C2wL5kc{;gw5Y$zx zh-0{Lvp{nneZ*YiK1G28^`{#z{_q--F9=G$7~GzAMuvO)h~}#r-j&5&|3;rhAB31$I_eyr z7aWv&EmLBPVfI;g6H-3t%-^}g>{Ggl8lotZ2lYSDnf<08&7TMC0Zbs(AK2%N%A0w5 z+P`6y@mArr!cm#TN{l;PLQp7|=7L4q9upN%Ebp(*FB{xB?PP5E1<^YXsgfARKt zRh7OaL}Nq&@`FPV(;E|dzkhg;UA@M779WcERDP)W?&1CF|7O=C9!T`I2Gec*1yP5c&@G&T+DF%HmOn$ z$a1ngFPmhDC5CV@)SyCcR{4J^x^xEL&$b_gtV#|^(vuqK!S?`@DbRBe*5dEkX^oSF5 zKe#K2e75${Z_B?v($Y@{O^7lr-B;igvWw?n;acO}yFog`*ZJJe?$Ghph&};U%nG3M z;1Q8awhje+WQUzNRNjE6ki)8$Wd19?YMalH9G|!TY==S_)a^+lEd@|;<`1u(`Z8Dpe}!3 zB_>jL@JFeAtN7DEMZmUTuCK09qwhkXO2(JLSW6_U#G|OS4Y7djy3VG>t zLqa;|Hd)jC#J}2g-AY;0mvpn@mvsx`C=WF(a1t>JOEF@RV&W_0w5blnn zIk4Gv|F8;;MW?5vG|6LP#*{qx=zj(NwB>nd1q~s$>(KY&>Kk-N9=N)_rCf^AYx>^q zdKI2QX7WuCRP@s~hc$8+FT$rzY#GK5QS@(n3ubu#Qb)`E_y2X0UuCET(Ug;HL*X*(?*lVav!G_?033iEDx6=5;sjr;_o%Lg>)uGn^zq)2Jd-k*E-Zw6oHuHFC6%5rvD=qrR zOu|-tX$Ks?+G-0c#2NdiY?qB5&?~0KjE@m?1^RUCp?^>2g+MR7@WW&a^&RPH994zz zZ(MdDukk5*fZEhsHMB#;?gE}w*Bw(D)m_yxQ(l;Zn4X*J@BvLbh~OuG2-Jz>Pe|CS zAz_Ro(Fb$CgAX4c^JwVlk!NJ=vY=d!xk$nmCR%PYa+qMlp#Br+Z82h1l4r(RnxzbB zK#V`P!>)ci$9~B6J#TBOa~S*x)JA~A0Dz{+Mn_JfybNk>M`TRu1M$JZ@Td*~JamRa zo2v|)fqAYl+riC&3Af6Z*!kx@`D?X|2{D zy7^*%E*F{z@gdC$stI<|-<6~7<(0g{6V0pf8wcr0R+_LG_FrX{zBDTb-exSEpU>wc zmTAs)sU{CWr_|z~kACpkJ*|}~O26ZTv>AOb9Xm6f!Jot@ZV9{rFS-l<3gt)noLTzd zkGQE*{vdGxfn&brthQ>+bH9=9oLpmyi@-G4~n!6(%Fgth$La^>l6O6GvS zBdZ2W@8Ro?_rm5nfRy}36m-W!VZG30{W5Rved0RFaK(!g3=#}L&=3*j>&oF&e^o)c zz$1qSr1d$dh9I)GuWd0IKweIMp~HmWK#|sg1jQA&y$z1_k=jN65y!9@Z>?GQmEa`Xq%!ur@uhi@ZPfVQ4K9GvESE?$12Cd3~*PZh+kA#K5! zj&_WiH>r4GzS)UhLA;ifHaD)T##9Np1Y@edxC-m3n>IJ;GQN1><0xqL1l+>&mH*kX zQtz25hC>4%27b?BqoXpO|AtYOK;;*5=YNBz>z2sQJ|#r6PI2uTd>r_{ADCksTwh(J zhqVHIV<<@IjIKJ^PrD3ytp8fwA8$uv1Cz;Fv1r3`mTc^6ZsGZIY|zix$(#n%L+J za;$lunJ2Tp_=pk%$Z37oG3Mvwf*AVbayB?nYyTG34=agm3Rn}3@X0g)&561m@7Ux}1}Q+7O~1Odmw{6bbq6Xp2`N;QYJjC7 z2~;)(AtXs;D2u6+yNwM80SjkMA6y_yfR31Kla$V))@G9cjZF?U!Rm6CloA5epcSG* zCWg}s-(VRqq`02;9?V{_Q}71Rb{yMK^sZ6H-rns1`+m#T*0y}Q0CO=Ly6SbT`*m#7 z|3qi=dA>yOCH(^OlSHYIhG*A-3Xt`wm!4lW&Vn3XIPHG$cKJ#QuQoMhYYF*t<%xB@ zfF`^@sJ8E z5_`=XoSPvU+~RMUzSTB_$EA?Plf|ODcgK8eOZF-}^pG`eyGC#kjk3P@=oWn(l_j-W z1Jla)%y8e9oFVBv!*9vY^({Hy*+a(FLT|2BtQ<58X%h2L-u*J9Om_r|VT zkB{Nk+rKd7D3)9f`x*cWkIai;6tWoC`Vh>JGxB!1s#wz6s&gQ=i1OF?9_ZtB0S)`f zQYUF~fyS7YsRARmn2Y}m%>&{4DMQs}6wI{p&!on5G|CT7%)4GXIq?KO1~9aWnVzUg zl!z{Yg0BLZ*Pt22lnHF{N#yVvBZdTEb_!(rQL4^u%?)>%>r2-|+ci z$x&a$D?Z-2OsIoCZgiJG`AEFljOjkW@x@9anFZho($TIz-|JMrom}1bLzAOhQj8}- z((TpHT6Ry#xfO}6Fp7`fL$YpfWy)3D{bZj~$+#msHK|+ga={Cy+R_tkL%atVZYV9F ztIP)EzrB}okutwiHy2QebRx8_2cvKCD8U3TMmU+;s3sr$aC5&r)>~P~HQ>*UYfJK0 zVEB7s9VN4nlo9Z|OtqLWw#_EzZSSvnM-nn*_KDFw_TaYINj(H}R#{nD;2(*vyf9|w ztq5Uiy40`(WPweG#e&|8H9?R^Ewn8(ko(D2{OD>~aD6*hHer-D8SX|sDBhxg?(YCb zq2|1yl`{B2xyeDjXc)1GKz9#@agmk&3=spONNk>LFc#ZzP46jt7~$fh8cq}hqAGas zOA(^qrK^dFmpJYi+r`XOrK77M4g|e^kLF(rf`cQhj$~TR(e-xsgRc?$q6ebAkvsUZ>kUpJ1s8S?IJ`PjQbk@t+vMX$$tO~2i(YUz`XeA0`~C{#_0%M70; z-D}@WTwywJSHn!}XzbRq+t@7<%iSyxff=42`qE)knJ29NE%&E5R&)PMfqESsD2oz8 zkhXTbI{iy;fZ`ICDkdKByn)V_;wre&rW%_WRQuJ36N8U`W9B)&!t&algP2TFLbZXZ zi^A}zi#(tnE?S2dEYmrhzbbW=K70=zG`(9*J4$Yn=B6f0V(MOTgp#wIcFIa#3(zSeLtHonK*t}e9qB2}d$E4K=BJ{fla zD#$>7!TFv3dyYkCpYNt}=YvNSyaLDWS-8+0sd3Ye3txF)IgN{94+a`EsI%G+uEwJ* zy05)Km254IJ~c@wPeGx&JTQx#_z-Qh|5Suc##>D5F@T&!PW5_Dk<8mraB??{?)t@6 zj$irruj{_wvf%FQx=S#-{U|OBX9KxGVH-*48exr6y+`_2B(`YE=t2 z1-lh-zvn|Bgj=uU-36sSAGLZv*a6nJ7E#qltqA~>5N=jV1@H_@!`wwm9+ z56u?I+U;jfff|nkyZ6%NE%-VJ%;Xgf{MYX;XU7(RV=30c}ydx zGlL7xAhDLs$dKds?8KU@cXzkjoWFkbX=!=Z|0;%3i}Ckg;9EXdipt8Ojz<4455Aw=BA@qjQvRZO28hKRUkqpS+vpY_Vw0S+ z9A2+@q-?f~cHC`XJgTt4%CH%0ehFQoscFiT`}Fx~^y zOvI+NLv7L4x z-KYPajV9ANkLiM<@4%*iiVztm82*m4tB)QO8ra>-ufO@qo2H%J$jh33FPh$9wI>ao zCiTpH5$}b0vHm2!-ZD|PrUJiM&;fDmJTUY^8T zT!BjC>!%ju=hr7ej*bF6lceQ{9Y?7h&9v7Q3IXAeXxhWNEn1msY8nLtWtCrR7?z$}~2+(c!{6(UrdNhN2Yc2<0F{IDKc4Q6yFi_|~iIrqo& zrb_ex^NxwG(!+>{<Pj3rnh7WuiahfS2v3GZ>`z!>41eS9$KhZz{ge ze2oA7u>Vt?SRXbNFmzrS3fvCzqo!5D2^0axP$gotSua=BiH4?H9LE2#^6?wc-ON-6 ztjO@P^z^3Io*xXJ+kX)H;oN}Sp)w?;royYPkF2h&IPDYym*&Zd4sn{fP8gce^$t0( z+r=P18#riAj`NlD)=h?tu}2Fe$M%j)9o0?K8RzQ8ID+Md)ih+(Sy8{m)7uyp_vEvy zistWTz1eqOjs8uo*79{#uJNEBcR(WiLZv~)CNAnH0|OEKD2ar`=qDr@1gKmKPb7=; z499Mryi9RQE-7H0S}-Z+1`p6O^5ozSV&l!5Kc06}I_HP>Lqq#Dxo4S{8$G`5n3H-* zlY2LHUcIDVDBim?JBtcdz>*CfRab04dpNLd| z3D>^@E8_T1ye#>Y|r59D>&fi&Tno{!h9m)0X- z>f$tDdp!sOq%RAi#B1*{tIoS)DfGdvZx9H>MV>*Sk_4WGTNs$U?&n|Kxcw$wPrNrT zvc1nCjcvBr-M#j%6$Jqwo2#9jo%i4H8hixJz;!A6oW1*&(#gr0HfITz|Tdv9uTGuur zUyW@%TfJ}3d9#+3T>Z|}FWf8a_Y4fm#d=8s-&YB^)wJS4z~87Ogg8UvgC1c2x`Kv+ zB~t5O0yHzrJMOGqq4>UM?^JrzPdztp8L1RQ>c%udf7jex&x#W8 zhiv*49HFGmYV6c`&dl73@#xwqRg)Bo$Yt-yh!)n*$Kc|d5{|#b4$9P}>OX6@8%1F; z#);zmciRjwv9TG#MNnw9pp=XfW59tjb1<=sDLX#z5Z(WyA6?sOr8CT7JgXz>tu|&C zSFuQi%yHrIW1%KWmdAUkeQ+46-O>Qku=Cr-xc*Pwz@=Vq3&m!CfO5LkiFr}$QH{yr z61eN|eXDcH)x4*7b1179m~}ijO=W>zTnR-75+$YZd>>jFHKjm5o`d(QYZGH_mgI1b z1%>YS-2(>I?5nG*SHv;F$|@@P1Um^2x3`xE1|Hkr0cmLOddsvZBP>-t^(<$`mW%cW z?9=FgHSZ~Y2LN7(l(xCN&TdE$2{$Qt5xmj}D*QKu5I8oXJYTk+H(vqbaI6<{ETjDj z1ki7>>}dlmG8PU>N9C(kUS8jk{XqaXBYKZbQG8A~dI4Yt?mTxWaJ47=)2Q@eL@|pQ z4^`x=zrg*PIb-MTY$_3>ycmE7J_PW5S`cM?)ODf6>{G6{0l`8fI=Lk#V4 z2IIcVoUWZy7q{o>&qY zKD>EM04#1fIHOX@s}{uF|sKoXpmcI;B=MY`n0?yfFPtRyJpxjCX== zDn&cJ)vtZWBEur%FPU-e1V>v9*Pt5O(*Gpz5F`PnP(dwfv>}s4isyy)np?7mOZy@~V!eqVMtnY)5~eJ1qM%6j{!yU+xd_=#YZ!(gg7nlk`L zNZNAh?=4KQg3r^WJ3%{$jX%#SIIi5&(+`tbD=RA{<{>EC7oKfuGN^)_tPa2tbQ_?+ zxv8sTMq=a^6+se?cXV{Dg%9oaaHrB(M1n!Lz6ZhkJy!t?p`$i{A+&9S4NP%?s$aPz z?ove*0UGN1>gL~wl_W;AYczG9csM@*Eri)JxIXhBx}D9DR|t@NWPqUb2$)FX+0L7Ksdgf zHTRQRd1uCoM5tFxj#Nbj0jO}DyQ}V+J4k?TOWSy8+>|h4@~g}J!(GsM6#xb1p9Hm0 zGzJ(=W^(TUnuA_Gfa=%|pgJa^SD#;223u`(R9c5G)$u8;DTjWkT}1dzr)fSM5${WG zD>u>&&+IG>{P6v$k9NHLa5N(o#1K~*u?YvgTsJj~98CW;>_}a{=hW|1^s9dFsY&r?7PKr!VT-#A$n3WB*nm3c)Z_gsCu0k=Bb>AcRDb^yR+Gq=M$}*IKoMdBAJ{^m z{5D-iOvaiH(L6_i$3`F0#6=0bi~`Z9UX8|*;16gC^Loe^8iY9{XK<0-TcOPn#=pRqW%&0Phq1E|3UBlHi3&4}+wryz0ip<}jx}5f8nt&STtc%J0 zTz6w*rr*AJ0-ap}2T#3KL9QUpxvcsW62y@MYb2G_d zyYwzeJ2wYPk}RAa&d}&ZJ{@ih%HwV_wD066-{}yUo$jX|#??q5R}rlL>Y32w9rjNxvSg2n8b`b>BVPo z3%Vc1=lzC|6dEdZI}58YPh*S7d2;UDrv|7$Sa1vIv##+eQXzrr?MS$bH@NLV^HwJs zsak}cR|`Q)pYbaitH-Uo;Um!Q9(e*qXF2TYsiz)0y#!|LzKi> zhD-&pf`DwyJ5ag3vlAb@L}SA+H7QA}Ol?$OUmMrfa#;s@pj2477k~5Jt@osWwpW(1 zyHdY;_BULAUMT#-Ci2J2?Hp+&b{qgp60MG1MTCUIzU(ycIU@gbopq&&5&`^LV&vr^ z@j1Jo3Hr%Xb>Zsc7Z-h9J>)-a?ar)b0nEp<7x+V4e_mA;;vl>+K&t(=1hA@{0p-6r zNkGsK_=ySL-b5X3?PH-v1S$}?n)B||NDtQwKx=u?*#!9#AURy(AcoMa6oW+%Co^R@A^lc6qv?l;7(^Xof0T5 z=)|8(2MHz$_|aueV&(k08i2pqXNL{vqs-3@Z3_BqGXU#A3EfSn9s}b_KhwVo6T=4#cY^G@UX|J>VDbj#pk@w50DvmE;3WWW!F;1Qz z@2h6mtpV1L1Oqx6X@kHi6e6MUXdZI~=(~Vwne#jMy%pQ(J>OY;9HoKWxO3V`<9Ml% zA>-OiQt(?I#x2ABiMiuJz)8e+dGqxQQ7|_NjS^YzA)pNZ8Xq@>K@YBCQXA5An+1m! zKP}QYwcb7!5Fdw5q1SU0dAaM^{quctQ?R&v`a(IE%DC7F(7^#}glpEEx09)Nfpoe3 z0nx{A`ucW*yN(@}OHtU2SkC+7wHd#D@jXELvi3${yPW5g86JuN;=e_u&`ySFDO`Ih zUsrqn{(fT;EV-52)iTF!Rx@HHUG-TrheXJDfea?;Mny2GltbmzMzJB-qQF=dffzlQ z)VV#c-{_72EWig2Yi&=mQua4D0ZMoFIuy1gs99uSRvmfaYmZ8b7yWg*W&GKQi}^%D z4z6~)+){e;FWb_N>GD#GPe6cE{n}}GBF?Ac`)81p_+i4%j@SX*w{wcfzse`34`ki{ zpK!ie5jZ$hrrNSgtL4f8=5d3eX>@;j!;D1Pr-6(aheA25b>KF-(VZC8sLsEExwOOG z@!Veo=pxtr=?xd!MpxQb(T5j3msz~PHTwF_1~|_Vr}=jF*Zi&YoVK#Cn7)!kr@*ad z9)b63lY!#nqBjJLpusSi7NT;S-J6vGNFIPua{Oh#(oiPu@Lo3M_q^hd2o)s=G7wU4 zwLrrU7(A6&_|<55~9lC#wd?LgLl!Xx?I@~RZA>} zXHV_F9|f?eXtdMc4MyM~`}qn+-LoEF6O?Fzi@?m#(@6R*nim~JyE2@;Jukpq)D4SC zzbS<(G!IDUe+I;5*o~XcmvJxPLc1K46fe6a8a4usv`&f8jYX*uWNqzx>#_T@ zbb4bbd8TlFty>G86LUb#*v7g-1CbliYoibz=Xw|xw??!X8^hMhs;?jw00|_OsmYvg zb{oLs0wBqmP~k8U;&tQKg;l%NiZI1bpa3VMR?8Lz&@<860QCH;_mam7dJUC$>2e)P z0c9MOzbRNu1G|p&W?WAH!Yj+muQPF#I*8V+lfDE44ld!i($zIsLq$C*R{!#O4!Kqs z#zxDDdU&@zQsBE4Vba-cnCNs^xM6cL*+&dz@5tQvSe=S=Fm0?k_OK}?_p zU*kex4=Czs=6Hv*4om$>?D71@j#PqRMA3?3`Nx9>{ct3}w1hQc7Qtj+D<9$1Vbv&J zcg;@HE!YFANlFI$zYquEXl&SS@E=v(M&&9Lk|$WKX30dt%Pm}V#Ar&>GZ%d8pv zxI@AS8N?edJ62EtBPny43f*EFt+_*5kRd4bmzW)@uJS&W7%f1INKZ{I=>-2A$qnd! zEzHe*Z@sB<)~tGN=G^o-xd=fS0{SMPTa8!y8TPMl*Q0wG3xyNh%_4bm>+9=TcLVSx zS#ICD@W6{>rgtF46*>aUsQ*(yXT?4`v`V)tjvN$XV^?2Jms{gG*NMM_C#9Vgx4bl; z#i~#A>pzZ9nfcDr4$XA}`p0`dh5Apl$?i{Ve~*8c8yDq2ERcaH2GYT?vU2b1{`dZX z@HSmSMbC#ntb8oU_JJwbHJI2-NbkFZGEMRkgUn9jL5^l-XtrHNLBWWhm%BFy+A>{! zjkd8l7Yoe)dIMa{H6-)X(;$Gu-}%=hCYM0`;qj#etTuXxp!6sinZyjR_wmNHnn{tX zAO%!nZJ{dEo}R?WN(apbhtijpkH!Js-YUJJH~Y_C5=_t`vH)J&PiXxME-WA19Nc9# zK|0q`g54V`Gs9!<{?}*i0;ZNus2t(Jynonlfwy1mBALc5M6AL^dGO4qAy6XJSShXF zFff^j0Ots3IY6h9uA-vCR9rlC4M0a?s|_;#Rk?BMgLFq3F`3nOe}B?>zsdGyY_hQy z1cc|ot0O*M&pWZJdHvT@02mED#PRw0q6iT%n>)V?p(ZUaFTZ6E1tW0~7p2l2mVFW7 zUjJmo` z%Zdb8I;0eGVx<;xq!+u!lYZ+7BEKoiMY~VAlcoIGCze9))i%!gvl-gp{PeP0_%i^( zdA-N%H9U?ICqJUS)P`hDU=WJ`SKCww zp~-q=YLzAf6jY*hf$Qcq$}&-SKzAfJKObzq!mgBto<0;9ZIY(TjRM1^D%YW(_=GR_ zf*;Oox&-g6MB&;BBm^Hbn%&n6hoS)g8`L*Tu0g4nP2L+QT0V8@Qc|!wD*=&tRf)p} zfUgBa{CXU^g#2*OMxPlAeBMWsKYqm83^Lv{V$TiWvq`!TL2|bMLGRmI%DBc!5K1u$ zQLF8jCONX7ryp+>Q?@+y$FaW|7}D#JV2o4d)?Iy0&YPRw+!f8uGiB2?Fz=I+NQxM; zlBkJIrBf2HWN#8|nqIW1Y_q|ZT~_RM7Fl>(TSRQ4_$ixgUn&c4&4A?q|8@~oZG+gA zR8=9%)RY=qyZe1Opm1NA(1n^IyEnh}BR<(pGRud6-v^lXFq`#tE-G2p|F&7Qkac^x zuUN@~{QM)1L4^tXMqlThrTAk{9dfza*QA+yrz3hVRpd&Wkl^)j5xP77T#Qjv()n{F zB6UEq1{f;6)rxw(CPflipL172g4*FBXB>k;#lHTbLa3|*TvX}+1t-rPNN%8<0aVQ% zz^Wc0$A$>~`L33}$G)vk>hV4RQU8(xcmy)mV;gM`T@Rffoau6KF8YT@M>BTtrlzL( z1qJKyZ#|wkl%Uax zQqk000sxY4iGY}ewrW}O$D*=IEb7xq`rwKKe z+94&?MVWZdPfI6Im7_dDAp1)R6x2ZG`Axho-`M)MulZZ8@@0zlQ&oM=p#_;v^V-{O>^L~6N!a$XZ2t3MaEz@+-i{4&+?x~vc21=l)X8mm{5RD#k>x9t7wPfj*H zZtgY(%4iCtskLHhEbjy8z36(|icIgA2u~)Zvcdc**3TbG-ui}(SFk!3fBowDc^ip6 zOdb6hZ(xi?;wtmTYf-@hrcQ_(>k=Z0etZ%lD&hkTBM~B^6RciQ%%&-Rx-Pu5#?aKlQ6uTY3CXN75Q}l*F$Honitbj6g4u)I% z24wf$8?IBG556>63Y?(A<6w0fO;>(*86QhS0{1F_{$-ayPb&p5)1P0c(5f*E%x?Do z@B;=2>m2&39=5{vqBY~6_7E^Z!-0WwrE7Jj!CJNXx@3vM-r{v@a3Xve*V?!>U`Np= z)C7jqpd=#*hbcTlv}(C2ccmh>$HRGmT#47NyFqh(dZ!RdJ;<0$^SMU%6L?j7von-* zVm*Nli>vJtzefyCoC1`Gv?4W5Sq+nlfTN66wz?T^l!ChHUm5snc52RWDjNxBCYqf8 z_15H2RPw<^F)MyUae1JK!lxCt!v`^S3T=zjkaaR|P3cDU5@b@&SPIt#Omd!caR(3A ziG5CPBJT4yDZHuFKjRFur^jyu{GlMm-FoVb{U7V%VSaAX7_3rRM!o1P5K*fBfm3{G z#U9^Av?^z0(c$LELqmT2l^X0Is_+_g1oHbYHon_2WQo#`0Q>Yzy?!pfOMF!TG z;zmJd)t|mvmjDqectts{q(m~x*C7p{+S%fSW&0ma=lKjlc4QZN+k(bfd|vlN5+W=V z=`_FKd~6nlf?q||xs&igakUmUiRQ+*dIEe}g!BSfU7+6E#^e9?rm3`n`DO>X>7j z{V$l|qKM(*3EvSrv`-%(6jgK=)1E*)Cq+kTvsR#bn#_Xp6E((H<=swVVX zor)A1!90&h0DZkZc|bBB*wt__N3X!$xq9N`!lc7A@OwwXDYuLW8_Q|A;LRK?LrT-d zV^F)>w3PMlOr#@)%k-g#wSh%n+&drp@4rJ6psL)1{O^}#Itbb=Qn1v23Bo?KPZ<<{%W<25TsmI4R)YQ~m#;ldl>G8*0UQP(r^&sL+N6=V2y zI|k%*alX4LDk>_BJHI|U`qH<|8SjlI8w0^0zqt4hIy2Gp7vuS?nE#F}ynrBiDT}3R zBdPJ>>3FHRO|7YW_OB5rqJFSl97-FdLar#ibNWcRnmpWPrCk-#%^`D z@#-Qkz(N9frN8)Z$lG;wLRmzgsQWgVZn9oH;a%5s*gKCpyTcDkkziJv9wbK*W2ptw zMqgUl0LTZ~jCZbqAT}k+$Nj&C4kfq^!GKd5Ya`0&f}k8RsB~Ha28nJi8!B*22jqHW z8S_FBu!g=WE-mz{O?8n_r*Dq*5cR+3pBik{JC+xxIDE}kTL#-%2_#PB9-n?2agNBb zqb7Tv6I`MJuCFlGrT4d&Uc}i?DH9DHos5}Ta)E80{t*a}r}o|aA) z+9-ZspDXeWuf75BLK%uIuga{qX(9dYY5y<@)nj77SSv?#&{s`G`F{X}oqxfB;HQug zpZ&%f_kU_Z86>)D30?UpL!qBd#Q#MrGn&U}-fWoHu7b|PJ)2*IZsvGahw}8j1KOXp z#^dRLIh7o_8Y*el-3egX9x0KvN5HE$g#jA;18JB9Cl^w47^HLYsg2Xi+`MD~5RD3M zPg4Lp$oV9xKCd)s3W@^h)ltx=8e;8%fa{+l^EGW<-AdI0!{blE>n;1$mx+Rx@b{`} zy>_gUJNDV>X(AJX|Ei3EmaUdA%t!yC*Xc-6QD~czqvR;;I-5Qm7B3hlBX{x}VEIg$ zc`aGuA;u1w!8?Ay9yXD~;i}dA4f*#QpYdaB`hB05x(I?@MrSzc5%1P7l~M6&(YI=Wre--?6piMH&-^|;u#iJV7yqXHnO;Pqb2`8 zm_lTFA=2Rdx@}NDhQOlW4Tadvr$QOm#Xu;w2P)&RkpKF;+e9`g$2Jt@Wx&M5ZFb&#R(R3_8a)2pO< z$lDHF{wo@whP8fg&p8|Z7rr?psNl>CBPJ~JAD&a4*#0}{IUdsT-}w3YpC8s17EB2R z{npS={*}a`r0RQ>5j-$w4C08eg2c{isX|&x{{W0+0>v>;a z#WkmPMZLw*y)C6MIco#4s%e#yS(Ul5p~!-``7pmQ=vi$!=iUBrpF{u&f?$jj)5abx zHs!98=8Z47{S~#%wjPt%=D6W*blz~D1RE3JWt$My|Jgf!5?u7&a2?+fajKH#09D=5rK;i|{dB0pH z`1j7-vZL|ZdcQuIA;Y1XB!{p-i_9_>l>N-BvI+#e0{^IdA|uKx-6vkA2za-Xmsi^p zovZhLTWiNbwzvLEW9(bWL-u&qBN}L>5T#=FsK-cADW@J5603Q|FT4pcsaN7Jq-sMy+f{V-wML%b~lV zI*(cu0AJnbv8>)fM;d4lop9lq{VuSGm*$UWGzN3yN?&O#0_+-y!}8t3!_2?JY-u!d zG_fl#J*VP%*N&<>N;FKuN=WRlZ}haq7ZE5y@W5-MScJpUeNJ(!Q9Ko7XcFm1daQ+^ zc*T}X1$0)?_C8#=S7cz0mOw`ft9npmik?q`Q*5dl<1927W9&-+V52%b^L)vE;HH-> zEbm9N|4Z|n*Cpr8amB8`5D4wCbv6E%j_n~koY&FRE@Kr{l6a}58WEu9x9(Tnc1*L% z&^I0%8@p<|-k{2C?!OoWe>V+=1P2ZRqRVq%@8R*NkX{5jv#Gcvxe=e+o+w7Xxb2Ku z%{5A$xHSmlE;F8IJP?7A75=?~+2K!T-9gstHgdhFDka577>Gt(S)ydu^J)EZo{FX0 zXoH|586zw%#)KPP`ZXB;NaQ7z1MQAUkxJJ3)|5n(rn#BKphY+AuZzU~{-bY8Sf6JK zBp{jGhS=y?RVTFuqTi=I{Mz1;;dkR135NLHmaN(Zn-f3bp>R4jiZ1Y4uKK!z>@;p% z|5+ciNoZWT@Idhib6V!s)(wpy2I6j#+aXV*_cs3|(|XdFlzeOMo|;^?uCjr+39}P= z@`0cDye2?)^=5OIdsK&N=W!Fm`M%KqaXD>kK@FOku?#iQwB#@Pkt)|1*VM$JBV4FN zwKR5gWZ6zn$m=F|e|NWhrfq++&!H1Kfbgs4k`Us5n@DhHShc~3ul`oxOX8_2kd`K; zCOgw=)hO0g`G*EHF?$PD5mwpQ3?)rf*+C=edFiXP#uxmz4btk4>tMx#A_!eOGd!QY zWOD(2p6hkVWF-R1bLwKg`p&k@R9*cm-qoY|>^vQ1K!57u6^rZS&Z*RUN z1t0V#Ea1iHhP#{4!0v!lekCPs#MKETp0UvMt!=H%{hQQ;v?`v)imAQC5R~>Ezz;0@ zy1LQ47ZN0K%K&e1P}pa6W+3!m@d;bBa~e0dcGihl40+H=E&tMQ+>TKUCaRTTXsc;h zS|5sYyntgxPi_KSZ<%RnrO-HX7J1wQ(=BhJgwNAA{+=HOYkQtW7ic}EhE4qSh`(%= z4G?9D5~#4EN}%k&RFQJntyxqs+&(;T(!|YY!$F4vk%xh|o=_k+%u>yE`C55q?k4Fl z)1vGTwZ3f>mjGlfT!|Am(UEd;st>V@%y5E|+C1+$!C&yf){(#ZeovtnA9)5>`a}U^ zJ=IAD9-g;lQ~!R(bdRTBf1P9KGa3xf4v$J$cI)KUPsqMDuIr>5)AQpdtZqt6L0uMY zG7W*fHnHewiW1Uf1@<8zc@RSG?n<1u+Z28o4k1(|kby(hHLs}ANr&#(e;}!;$vkYu z4RziETY0#dPcJhnTGlc)0tiGsXz5c5H@G~k3)d!%eey7u4NYjk<^_kRW1+#8yT`R; zyD7OpNEJlBCZibET(qPg-^yEkyuVFUkzq9?GN}Uej!ri7He@6uf|VAd-7HB8$_vr} z*~5P$=mxW^8+vAVg-=q33(l>>Tv(!MYA^R*ErInO16 zndu|^Vj!tXMs~*;ih@ruz<*5O?|bR6t(L5Fm@3_Zt9lB1`c>~LI4}sBlA}>F)^|62|`a!fo#d7oPq1-Bd)VvY=M|fg!LSQ zh{`hUKsM_c30Uvw=j-j$=5vZdmzUs68k1{sI#Y|kh%Si)Q5jgP-ofGPI(%TF8kVvW zL;WHgPnJ)O0o|(j-8PDZGBq)=d@^XP>@+EEZEby7e6o4l3f%Hf%|%J|5Duv6wx>m@ z`ma-*uSM3I`kg}d*|H=4p!==y z`w)_^GL1S(K-93!cQYMchpj8K(CJ18?6R=%IBZIh5ul58{max6s;a7Xvh4j|HUvNV-kKpv(ue}0 zdTeye@VpOJH=o$2ZbHhyjDz5x8JIaa&4D}Ptg#iW-}sEg!4L;Hc90c6tf8U$QqYPk z-`6r3znY82mH4n<7H(?X(RP(5=~3{PQA#4CK>IaXjNYS19fF!D3p`%4ZhATXTg|~D z=>g?$p&D|d-@lv<9Mpel!>*&!otO%r#dpwWpF{KeG%q-Zltl$|L(0dQlje6A(Rd^Hh{RS$ z%osbBVsG`c=xSQVp6S9xd1E8-u&yqFPeP>U&HyZ)KA`jxlkf&W*3&@uyw6(?8V%l( zW7j#da^Ahcv?R8)eIlgbn@jWHEDtT6oXmPjYBpPhwFB94qGQ()n`UxVQtaGYA{5`fJ z6jLJW6;XOvSXSq6yjFir=AsT?%Sul=Vb9=I5F3r&XHYl@dr~YD=sBYz^Z9Q*}b^_Y^e9RS{!f_spkv@cbA5+LWEG_%)r+ zi4c@7_^&=MUukpMrq^XP0UPf=1=IHm|7jnsUWLxi&WT<~Jgw3q$N!x?Ocu;Os|6gY z#F^A|-`SgDu_=Uz<>7lP)8hWvCf8(AzVoZ4=q?wT{Yv*iacYOsLopBO(Vm;mz7g=v zkHt6ExmJIo08=K{>tRh~pO4y?F|XAG6;GK_v?E(a<8lUw4zWNgb#mGp9NMYuXid4X z@Zpn=6;@Un2}$YGO@FbAW~9>S>}d`l_=6ZW{%W{_6qU-EVF~|J&e`0cCa7YMYgEo~ zm?i%E0K6K-8%h21c{A|eEL#f%cZ&QA{l*S87o1R`9$#H5&gpn z`mPp%9_jw?r}lf{d0GBDl1wKZdw!{qluHSZP$AN;MS~z8EGMt0|1}EF zI^^joLr^*E;m*{&f!$x=>t(Q@riW7+iiM&NECNJJFYWk(^TpY>VNH(lOse5o_-&!2 zXptp<@Ks3LbS~6qY|b=x*Bb#iv8HPGYX#18^w5R5U|B0FBqa8%IeQIsaQEFhO(?*1 zyS`2uvn#3T6Oi{OyH_8X$;I7$H3h9q%}TiS{pjeZCj{r#C)o;obpQJ8!uP`c{h9IO zcv7Z_MF!-c_u*}(yW@lL-0S0I^8>%Uk(xPKVEpy;w6YAucqN@J2(F-@;NA(Sk@P)# z`C_CQw{n?)6@4*sib#(pc*G}AUu~w`yPE934yh;RBESf?8Y`C3iPG85*X5umCsXgR zBCoL}HyKEJFejJDe|^qN&G!6ESRfWgtsxu;f^x4{#ux?e(nlhKr=&XRnU63+^GQae zl&`d?RWQjp39i%zsv6YSBRM%W5qYA7%P`>Uc3*1B^|Ksvaf+e}X$ zzC2}nqrEZvtu=;0OGNQW{6aV&uhbp>F-Z%>s#LN2oZ^ZmZR&w%KxzT9SX|^=a25>#DugJ+%VfjhBUikqthH074Q^m z_S2>wqc<2kRX6-zP`5o5BhEJBg+^siP_N*k#A$VF>zuMwLR!i9m6%DiYiUd(95qZb zN9TZQqw8ZGDEQY-=Q&{wVd0!9Kc2i5->1BbZzw4W(M)Go6AH@@SoBroQocbj!l~$o zUOAaEY|A(Qp&hiCedsWmc6N{{93k7Ab220%Mg6QLK(sphoQoxvhQqlH?BPR$;UOdc zY-qv5{xOvWinyWe9kW#yk=nGb1y~&kF@2`Ja#-L!8ksw~CLOwwAvtrAx<$bC_mkAG zm|po^ai2ANBV1J_zRN!eIE!~DFidp$Zr|Dw84KquovDEauzYwPfiBEqm5w<>qcRKl zWS}8=@of=-1J=Ok6#v5`xGH6=kNRikBA#a3Z$m1`cR7%%*Es>9lJa#@)})Diu9ysk zJeAsBBY6xXMq@?1Lqt!#bLty{nVD4d&xAuD@9VUxhGjr$%lKSJrJb3-0wK2G-Xa8d z@}VbTV#9W*#}g@VmDqap-p(%WqRFVVGNJPuj;f)5zFbRxCMzxDrvUw(Ip?yCFpe=1 z1!r^9v-gHnIPo|m#+Sbt-=xxrmBK*!Nwnrrn}KY1MR*s$pjH3|)uX^IjHeYYMsn|D zXUD(r@9D zP{!e!?*%a9>7%evt=L9HC0Okh3EBT2p3XWh%C2kMpwbA^Akq!eNJt6N9W#J*mo%bu zcZkFgf~ZIi-5`x1($Xp2Aq|rA?csjj&p#db&1PRa);iB)t;;~=O>&Pqq%(6zU4;>z zy!Y_(YT$OKpoDY;GL8r421Tdg_O^O^f4}ND>WsB21lRRB_vt|}b=VXHpF1rB%x86M z!E-d^4kj7$wI13U`l}xqWMmsjS9;E7m@msT!;p{X`EdsuHJraFb+b&@vb=^yENnIki*w2` zzoixAg*C~{lL7eq!8PL}+8Hm;=YMOWl}9=LknPw3@xCY}>DX*{$x9$yGsrdPho!J1%& z**Up`XNH1OlsO_A!NBmt!IH7bIPDm=r@hB{BQi*A{2`IV`uH&a{(tF4QTW*BKZ2dK z4qKr(c)Slftil@XSM){>Ts-Z5N)wps1?NdwXh*f9Q|KEQ3{V`^_6NLsS4ZWLr+UEa-j*7_F`#tcS-g7`JdtB3*$vSTTFs!O{n2zIv>bOUQ<$ z-U(yrftdfk|2s1}C?`AmfRL;g;GdaOh#TkI$#mbpbl{8oRsMdZ5j%~OEQUmDcMw=b z942BlJ%D}rm1&!xs8gJHz#a`VVDBBGVh#}OC}QCMM(=u@N1O2{zpe|FvY0pB85y?( zJ`3Ts!jC*ZkKab9O*o0Snq%2ubN8?9O=qBjJ>3Q;3MXH-!R38=>J@tO>bNCrpT_F+ zFVvQzXb&Kl8Vv`6aQ!l%&3D*p|+iaqn<4*w3aEtH<(Y4j><pM>a6*2w1Z zA?=!;wBB9>+^FZ}K9I)M#alR+M_dE&`|Z`06QXv*OR5l3IV)QtF7sR9S?woq-c8o= zUOi#9k^w_P!AT>vv1ECyrmX$*^G|#vHXP2 z=L9i3`5%J#9hL+48SXKU7 zd!IkW>ZI9nDtB`h2Bs3PFcDVfgxYboN54R#^gX|o*8XN6aLv$(!|7wOKTbdWtvdYpx&XVs`5SbvM zl-E9^Nx~h9iLLvtqOeiSVjxRDPH8Aq@KC=z(vMzoZTZcv$ba*cA{QwYFriIu9{F*0 zpyfr5?DfBCYRbRbjHGiZN~e?O{J*7_eL?>86(e;okUz|(uMfh`^lop4$v|I!`~G;` zHtycuUa-3pMp^3-tZcQc`D~^?p@-p|pjff2`K@R9`S!e2b7_8|j+K=2{WG^e-#y|| z2t?oRmd}Pxzbzm&B_}5@&^_0{M23uQBWXtHd0Nw`ia!{AC25Ld4(wrApV6GA(~o-- zJp0|#kJ2JZ84Qzivhp#x6$`}{I5Nc!`dcupt>25wlryV-5<42V2`Jx3Z50q@vRcGa z>|#%4(hibxHI8=4Cyv62&mLBK4f!`d{ZPX3DZ02gc%y9L%sWb=^8e+au$!ZnTezRgg2h9!eLU;vE@O#M7oQtgwdQ zxk9(`&a^mZ!_Gsw4~1)r{+ovcp>i_L7kGbY=dtOtsd{DCi=Ywn)BwWmI7VU7I*1w6 zh_S=aU=;IjZcY)WOJ$O7l=@1=RMtkfH_g&LpIUv^U^KpuIhiK#s<;N8Q&C;~{g(Meo$%c7r&6b9 z&%+Z4(~yqQ?^*PWC>=P{k+P=z!cpQa-U<#;WREpQsMh@&Kj_^7&e3^rQ1F-nYa^)! z^X2u^zPip{gc{Av?ud!1q2IU)BrQ%N;sE?)+YIxmO|x!fi)k5aGya!f!|t4idZ!^a zz8Ze{^me!Iv9`&qpK8$_^oL^OlKrB7ny0TErGF710vaCMa(-p`ict_^$wvX+Q#w<1 zP$*b%^EKdnKMT=!g;?fl)uLe18y71(F{5J;X@xxd74X46@D;}5=@Sn^VC7bY{dQ-X+0Y%6CPdDS|#Bx!)KJ;lRzb# z{Hfn7>Ao%>#>r@&kaP6zNzK)h8jH+=HkYscloZzFDw0zwevkBXMz`cBqYSxh(91*8 z)6=yv%DX^v^wOW4y9)*xJ$7xTf zk^QXl4Ivn`=`=2Wz7;!BxSeXGRPWgf5>bxat`?N0b4={GUXhhP|5fzH!U9WOowIX;OHU92|X6 zZ*D~zqhl_n6W62kdztC}P)y9tRXiqa?v+$-%c==@aDA4CDyceZwNtc5HbR+nK#CC`9jU)CB9?D*Od*NwMFM zB9;f_DN^DOwK!gz;SqDV>-ceY(gI&$6{>3lb>#KD=*TSQX_7+j*_Pwa-F&{VZKU;wqRZZ;f)nN9pV)!UO60P=g|;8;1nMsa{hBM(Hw9acIAvC!y{Fmo$hfGhPbua_W?&+ zZEDAe(vhe=k}lZ;5-CcXtoFhJkBEW$Vnx6k zfi#1CoO}T)kP4ly&%ui?N!ksX>>ISWQkw{)+R!OZs7qR6lZn~(7nK(fCfx6R&qt*n zuwd#4RJ}Kz*x6oGmi%X8{U;%EYB88lTD@|0M>$n-$ciAo$2M0VerauRFp0%0v@v@S zJPnEm`Dz_ynlF*(8;YUVwmkRM)zwKroyL?*NA<9{->Zn2j0^+$WaR|i|3&q>PuSQB zJQMUkB*<=;Fb-Zvihb&5+RKs`6dfU#mJKvZ2lr_DHjMlu^YF}!Jw`PE@eT30r|faP z7e#Gfzve#&`cUg72Th}+^;_#SQ`wK{S=6@!CWyFMiPJ)Ls*#;yP1%>~?Yp}UABHw| zcBbZ7@TtXERXl^%XJe$^fL`{9l-bAc_Nd4lHF8A^u*1gG81`?OvhOfKY< z1x~Z5fMe}fWP_lhluvTH#CRr#jc`^E7%y8pw$+gMo#EG||63_|njP1s#tJ84pGEKa zAK|%r=W<1BIO1OZrCIC3a+ux$;Dl-}(SiH$fOg^&=@nVVl(@m#!@oLl! zsiv}Cdo?dZ#0`A>uQ*IDqNAu{W}a6`C41~lJcHYBdKluD|GE?iV9t5i@Cb~pE!X_5 zxXKkAv}YA`+D*yJ4Nc~kVv_Us5i>r7Q!#Pf@Ff+qKFPyqE-a|G6JI2SQ9n9Ys?x@2 zL6ltC9CcIE>uKhxQD1Ba?MuE5_^Kz&_G&SGF}S3uO;%CHh})ct?Vvt7Re2P`r3QnG zyuz{uKJ(8wM(t@ox3sg~nJo5uwa6JVomO(Xv39~~aChvj{t&5nULEq$>=ezd-`9-m z=pHihAt^I->)wkscXBi_BQW!U_b6uXl(F=#T~hYZwYelb`((9Xm>{L5q7pejZ#+CQ zqLu4{(j@VSOGOk*J8tQ6rwlQ-qJG_HWXtAyk?wKe`4+q_O6Vj&(FB%fjy@iYV`O9` zSp6tU<}qrxQRCYv2)k7XPTr@;EImhqHl!tmQjgo=Hxh)(+}P;OUgrv#Uv1_1a%k@2 z`M2JLGhzRy*R*C+e-C#NEo%dF8|{f{J@up19I^HCBljV{{e8T+){|Et?Yi!NE?hj> zrOco%32zE&DIwtm=l$i~7-Y$Zl)(1Bcmm&vx%1!_5dNY_ z%b2#)h2o+`(te^H=&rZ>n?>0GCat~M+}tcT3j@x(J6k-Js5Li;e{+Ev;2=0_OUcW_ zLyJ`67WuG?UJ&g4$w(<23+&&gNgA7_bU06x$~+oxOY*PS*7fb`MUC;y!DwgTY|05` zD@ya0q<&#RI9eXH|H3G~P=TUH8<>=74$~MPBFl0`63uW;(JG z+)J{Rd|&tSLJ|JlKxit3;HsxOmK-W`s>1+A0H#Q#zGgwkbjnmd7ufb3+{&5{5Rq z`uk(Cc&A=~{{U@;7GE2b&Rka^Ga$ltCJ*}5tE0=+@3Z~Szwk7f&iGB8@&PTUpraf( zEADoa8)tj7dzU43u{_7N6v!xxBkY3kLl2LL%Db5})V#+(;+rga2EIWxBXyVEvEs4H z03f5w7DaeyJB%B$`gx`Aw{3SvlRA)nrCz-2h9AMZikxzuQ*IS} zOd(({NL1PzRVS*ArQOX({_SlPp%LW~JrL70hv%Gp%zw0pMk;L}Dd2*;Tj0dwe4=(X z#O`XXLQ=^?+Pjuo7yiLOT!x4mhoSsn(%Iqkv44inkK%M6Y49d#YinmWHqN4xKY#vw z8^A?+^MGYBL3G^&+*X0}=rHa4EFq%RUl;LE==#~XzTD<1Gj>>TNi7dQzr~~?BVi$i z#&(h&nbhg+)6kF(Fud~0>7ZSe@Is(Tes>q3>K|x+dhH?IFVaF&ZzMsK_bK!QQlqcs z9cYMF2_eZQc6milr(8-1R8i0EVX`d5ne*4&&Mu4j*@}=MzxB0*eCmWCRt#o^j+4uR zNWvtDrIj(U*s?4!MzGn@OYqkBv*4as&CA(Ee?fMnwnI>{MFB);iyGdSGjG(EY-x+f z2c)>^a%rwq{>l5Jd0(okM)A@z&d}D6-uZ95Grrp4z8U}I9@ZgA^t4E+;b^t&=4bc9 z)o{H3$;HHk&Xs@0Hcpsy`=F3I@YGJd8vmZa1(L9@l=+{KBiGTm^h3+E-rux)z!~fx zxac@e45EC3*PGC`ZyN{o#DV4Rr_jfy&arZf1g8=Ug={7yn)mjBJKp~g@GEB19D6K! zI{vxW9mg11|5gtzEXow*lc}2Vo;2hAf^*aD<%H-I>${~>3va04=gBL@Hp(_LK+H>5#Lst=0)#|F-qPqR zHMwdPj;yY7oHrlL)wTXqNTF~YvZ|Ev)|*kJ}%?55BY&DT$73o=cq0#LKmQQ}DXnbd9Qi3QcEabJCX+W?E7n zTH~Zng^R7pbLO9wC3)^14;a9_6YKxq=SRr-}Zatg(I)+bU#aD*O^b z{e;?arWLoFJjWcVq@!d8Va|!T7epWb zmjP3M6u8Yijt>iT;@WYW?hFV2Q(cL-vr&eB`yMFseHsETz@M8aQ49-+<%G7e3}aH&1H~;k`!;w9=*HFNSWj~P>oxOJkFpWro0&^DQDo{r41-@^z8&&ad{ z9`@`<3gN3}$>hE;;Dtv^dFcGxJA4S!)AS-B%ql=vj;C(ibEnzZ?yD;ryV0+*85cEuS+ID;=Jo!lckj?MSD2(w zj=sr$B5D$J#Rpp6LoNaRAmkLGG-<*G+jt%uuf8^-7b~+%KR~ggCBKJ!gf4P!{w8J; zZIAG|6BVGcIBV}0ekk9I>BEhW{^|lBGus+eXU2H7)w2u zv(3qf`+!`hbY02~)?>N*;u5S6zH#!+lLP7F)5ZUA)SJ`*k9?EDSVBFdyLrcUTyDxs zUOpLUbw0aMVom~~l25x2IPu{Z^wMzr9Y0y$*;Cc9;XBiJ}qGwx--4ATCH zca&VFR^tf+W@K2Bc1n|eG>xUoTrZ&Li3AEBE+D>MCo_-A5(znuA82f99$y4lm7K>0} zw%8cmdh?C`{y1Qdg3d3$_>3De6Ji0`gB=7p{u?!*Xr=Z1*J{GmCic6d)v?q&?-D6T z{yJ`wL&*Wf)h($PzQ(I=4c_BuS1lh8g8diBs$>6~5$zHWQf&%9CfLZgZE~>o3lF-ZeDN4*q?+$v3(UggXZ|CJV-&(&;K(E$W(X z&f=p%#?#6}Yj!s?WbLdaCNyPxubCA(uk#3pmMXMvp3zBz=Z0d_CepApxAmhq@7L(% zU$NtxZ^ec8Lcm#jS}uh?+WTS%7%4y=1hXTD^T54F1mE6~VE4s!MrrG^?M7EE2{>K?DvT)kL@1fi9WHC2N!uG8TiOWurJ+pK^eCv3gzohubl;h|mM$ zPsx>KfMk?|AQNBug}q~b^;hIojZ$w4z(tAZu0-~y?HYP60aI3~zm`_-bQcO2`I=C7 z6~TqSuzRq{M)C>;EhBa71%{?SyB4%<<=6r>T;)q>{GSJ`DPiQ7mzTTkE8sqvh2Qhm z@Rw4?;&(yR*0p%-{~ucMo228qbR8_H*@as`h^iEPo9B}TQI&v6mvq|s!D4#|pa=!) zzjQRaJi$fASi-XPpOc3%1mt0MY_3hmBtT}hy_ z$&t$9+{&?4NA0PfFD7Cb7H+B^wPSskPFpy&x)DVvAi%p~#Ugkqj|%uS-LiZ6ON5g9 zltG4H>W{D3nri;HUQ$qyD%|Pns`J1_pPfYK2kmarU6v(lcJ@quS8A)LM;{|fyU@jC zEDl`}`HArTZmi{{xofkhbX26y9(yfvxsR>Ne&Z?Sc90TU8$Lk|hZdwbmFQx80Qi?+ zr_1WEgr9*;>EFM9cTkR+X=~=$Bl6#~Kx{RSUTn4IKXY;_Js4`e1P-P_iB)HCd2rUf zQNCq(QK^0P5-19fb5U@l&+I|YN_oA0TMAz~DnJ417W`#^QT$0t8Fp-2!%6rLC=Th1 zR$KMnHe;vvoY-+cIC9cKG%Oi=`B~f}K3}!BsKWsN`TMc%nbOtg0Ku!C>z;2BrxnKW zjNE8iDa5HvsmEHmIb}&>b@Cj<5O$FpYUUFkihhVpyuWi}mB_QE$=W1_@Nfb!3=|j5 zfjU-$R*9j+ycT|@U&o|embWY4*`<)5*(?gj`Gx(gFZ%Hyn@F%e?TYArj1lrO*T^u%}h*XpWYwHy^-CFVIZ1aG>_{MI;_ zpEv{{&d$!fPs6@6HX6`8_)7q?*4Ne3^*3j;1G#>ZnqJ6&$y8B=B=CUmWwKI2zvB}a zKR=_j$;P-+tgmL_X8S+;uJQY41b;W%y@AE}6=U}s)3Q^7NVehAp-;jGA&Aor{tFBI z;m6PTlYDFu+K7F`0!P0ZRz&Z*UxC}JXg78_PYv@#l0WrF!12gi;D*lWL~WkC$h4*! z52?}@EzFuJDVxXj-W@e4WU8LGhOLh&C~}in#9>1boB*ZC3riC(Rhmn{1o$6_VHz~Y zDF)ei174>^XJ#l)HNSsdD$+Z$b-?If{Xs9+9){Y%hkG2dH!yx5S;YiyO!S>1AORu) zh)_y^nyh7%28;&yT}S!7E6GhE)aso^a-839zwi9by;uxy&dkUtiXf5UO2yIF*MH0@ z+&?zQ7rc}#2);b8Cxnn$7z)^OILb-%vej&?;+km(jZ_L6i?INnyow z7)K?9CE`M)u+|i%=7CPQfz$~PrfbHuq}id*1#U-d<~z0TO(XG!S&o0LpZ5FrZ&0_fJ~?$;tfgANZ}DPp?%8Ns632|1VY@$u1k%|^F?m_HBL7(yJC{WZj)3%A#xx;W;Z z3z%)a5JIerjzPk*HCV@rxYCr5@fmpb1o5cFY&p0D(8zxCAvgKrz0fNyif1bCdiQ3N zyjyot7XP^(9bfo9bD6!ag&Ww#)swW-8$9QuGa`B}H=8ovvn1h5N0I+pJeu)gK7>Wmft;*GLd`hwASQV}sR2YjbIEjCK?w6rw6-}>g@#Nn-X z&Qre`+;2+R-Soafk`trscY0wNV*SrM=xb8X=i+E~8TPp_WW#b^~!V zi3r;KNq&~g z{?N{NQ1+fAX0((#M$}3-e&3t#>iZdZW9RCcbjVN_g^TSc#1TWFxa|dTmN1+7UKM-4$7bVSt6?4<MH zsE#kv%}k+K_EUE{qYQn5_lIv1pJv$aEg`e=Kr6z(=DaZ8M!g>)-Yx38Gcer>JqXl> zs*9d;qV&IZY*@VkCA zh{#?`37uIJCge;7_E}q6CNCcvC0EuejeI+XvS;^UXia7`N7#*I41I%bv%7;I2VPb+5$vMm@j?HqU z?Hqaz^9O3bOv$QWA2?3zR`DByq)2B~w z+ZK+SzSPt}=fE~9yIH#laGf;#dN5Qi?PYw?_#dkTh{>-Tr?2q<;vL8qiOYEAAit`6IG++S%3FnFJLQeo`%auN4rJ`UVA0 zKAf;ajom1?+1iye+2R4;M!2@MWkJN<`ftoetnA-?{jW&+VZ*PF$!Nzb&d z^$P2iTf?oNOmV?=8rf;`2D|Yf?6}3QsKRwcaF);f>MUn>p9WfUqkU&JgKA$&YU(C$mgI8Q(?CAdzY1t zjjPJZ2y)ZcV{8}_(C43Yka)dkyUP2ONbtTw(E}59*zOI;Du%x9c$eM8BdgiIk*p*0 z4ne0WmyHF)%|ga^tMz9%gAEVOKpL7hX@bx5tEHs{jg#Tyf3-0vTrDmy<_8Oxma6;n z&U)e~6>oou60TbMe27V)VPbdh>UC=B(tgUy_ETB9FqjXDlVgw!ieo@<6i|-HK(wY` zS47x`rGvt@laTz@3k=xN9F}4)12u6GP{Z!KXLs2yroczIb(0OUN;)~Z3b(dNZyJXH z8_;r?T z$*U^!<43~L(b0htvfZW)uVqeW|Gt1ftq2SnBP>p)oa8DDJh5XA=kK?dL*phiozdGz47C4EI!A=}0SM6+-OXj^xkd#IE+r6ceq8jCsaM z6%MpOxi9ol@&gVZEd-ZSW~$X|ji}xRc1MSYKd*gxo_}nOx1ar=TVMRZ zx>X>FA{~EneW1ZIjGud-b-_^1yprWHkrDDz(=B#~n(#cKhDLSf&D0F6Udg`_6x9-y z`w3t-Y{v8b1#YCV9{)sS1oJ5<+nAi?7MS7j%wo~r_ns7LaRH^J9wIMs*KH8z$a}wW z{_uSOUs(~rv@{h#nad2yPXLdM7qEW~5K2(XAA>T#zu`q>lt3~|RwU_tCK_8+>E+n@DoT-^oyZ(BQ>%@hdbCv;7>)PTFE4ysw!xhb+P8z8dAp8N zaDCxU{tARySHHDW;$i%CtY#w3CP3(yY~n~2PA_3=<3OQ zbs00(fqxSeNLQP!I@)stZYrw1jawl=)ysZq(9AijG#ENb`9y&@`sX#`?o`??^WCi_ zY9KPnQ+9YRd2@buEM@pVi~#ZF-p*3y?1wNc?)p*=Z*T8lp!qzj`^iJXLYn-T4!OGZ z2(}xo)B6hmxDdbXS2w>XZ_KA~AYH@vvZW}|O83C+H{oeoB*`CgZr;J@Og135v$>9H z(Mx*#yF(XvgickzcPfQVv&dz6U1x0-S`f{xWzCOr!3+H35=meyQ`-cnx>?F29BRjI zB8302s7t&r-<>q&ci!H}%LE__{E5)Vd}BZvD>!vv7Pq$cpR-i(qnesYO>)9QP2#R! z3U~MO8gx;?3R2^cb_<8H;r@LWHBL))c3tcK6u8t$09VVKU=@Xt>JJ4Ae8R%E?~{2S zke5D<8=*Yq39;iq)}K|Oj^D?vz3iBqGyFyvl8!7r^Ido-Wci+2+>P6a9$=HU2da}M z2m_OgUR@PFd4M}yU7ozID(eX)Q#mj{dQJd=@hq2|RoJDrnIz0VH>|WuO9>FG72CT|mJg$Sj81fQN(}+pKY0`vJ*2L#%VO zQ9{-I-uMY?ZErW(6S+O;UqfhvYN_yT=9ce%X$69)wOtT>VF7{#EevX;e>JU5s zdPpX5H3A}qZEg)?TFV|@61o$nB&p%*SQi6&`il>1z3V zi->>!JjMHIfmltel7a5ttoIZ>Yh(JLO&;+h!1I=S9}&~Y4(ny**D!lZ^D>f#rh`2c$6G*FIem@ag_T`Y9IUW)a1 z;`-&TVj7$Q$0;s28S*IVpdhtDKIn(mKdh$8xIdOetv!&|G!|RzBrbOS%?3RBrBG# zxe@kDPEN36OHfLft53+;-w1g^5>ipvVE;_czj$hSOT5*M5mTu@iuBKm?USsnd zu`VyW)eDwZL64);GNoLl6ZG`%b-+~>v;vicKYw){` z0asUx)QH!3rjH#uPujdtp^UX#Y9LE}TYDZ5{V65MU7d?j3fd-miEz|?_H)dc>8{R(~zWeMM zOm_N800=POnRqBA_PGq5veB1WPsEg9h#y8S%&nN$#7@MtTqzR_OEX_1lWRr%>O%^i z^OIsy8;YCVO#+vz`Jl-K#)4MIp@NJOqgM~a@I%m%0%8s`-V@Zdq``0CtnpMo9Bgf{))unTzx%GPRN++GuYD1#C=wmw^74Mt6EEV zzot6cI22TJp-hk-PdiDh?2+X8UQL&sqq4)Mydd%e+D{Y5x(D8Bu#cN8481@@x8-3H z+09K4kBHcz3-WepR|CD*fC9k<-XP;p8cmDzCI4&YGpI6z%YBNWwvmsxo%CN>*>BDV z1-*qw!G2ENu9jS>Nr#?Ix9%rFyT$NRlh;ok2$oUabpt_}Agppu)x5d+wtY4$$u7pt zrQc+^NKGaMB8SrDwG`VLdcQixqNbd4y%-gB=Ke1~)YPZ~_AY34ubQ=7&!Tc~ zAyvD-_~RG&KS}boKhaz#_KXJQuPWnVm>W!z(pFn+LrvjFDvKrnhf;5?9UM#(_!^^P z{UX4CQX}+6`);{)bztM>?UhkLFqL^aQxAlh3JtlsTO81NPKK>}$fc3@$LoRMgL^&f zRcNZy)zcUsP(Mpdu7tgL{Pma8v){IW#8%-vi(Ai8Rd2K9~vrhQ}&D*7I#W{tC%{;NEy!> z&~ZX@n_Vmo^uxRb$klVM~#CRmH#yCHw5a#yojs?+(oh_J%o-Es_2;Y6bVC!=8 z!OYj;A+biuE1N9;*u=`t`}u8|Jmg>N%jwWHHK}!r#S97|{jWdlSO+JhYYs~)EuY=U zhhiC7@~O}n{KSuV|m9utGi-k6c<^IC9W+LD?5$6Q9pa9r)4vZiZ$vd%QQQk)cF9%oa%za`-Awybry)h+DVe^LrHv)#H{8eNwre*PE})__%Tz zdUQ*1LzhXGE3+Xv{khtttI$XhSY#~3a;&IVt0TU5uq|J4>B=!Xpj!USW~6r){_(%! z2$W)Pm52EGuj#@Sv9Gj0f9C0Ybm69~e9GWm*U|;b*9kTXC=OgQqYcvHnx(-*FNary zE23LpM%}YSr0LNsH_!G1v20Jsz8PbZfcv$=-h{w-cizx+6?Um-?0OxMdrt<}rbs>x|+oFRq<;)Wl0?;-nFl}F=l zZT(Q4JRftqA9&o|@ux$o(K%h|Pshq3c%`&wKh~U}?ok0{KN`Wrt2kMTng@xpQKGzr zVx&ua>pY}IT8{{VMF<8PmaX%(w##G;x2W{#l5tP`t)UBAOly#O?A~aO&pX_vDl+W! z(k%raOwyvElT7uVX6sPbgT>_e2&!Rj`?BJ_BlEiwJ(LN|u8}P>2|m8<&XtTmj7$s( z6Vl1ow!1}mrakfD1jS&T0Yx-8t z;_iL>^pvO!8cc8!?oILNE3`ET=qqVxZ+!pbYD{%144Wwm1GiG|qh%^~%c4yM+8-Y&vlc$+u~ygczks-+{r>3yp-23zWm^ISN}{Y>o=!w7iAQZTX_)^u}c2gj|! za8X!I{I}^#9Kh58mA~wN&DBM9ob9^H*K=;{{6AR$SMcPyhg)trAQcYYxl_b2)`5OB zRbe3o-WvFj7Dyi$RaI5(b90#`y}!rD#|>T0+N-^et&jCIi#C6!ouI?FC9B?Qkl%4D zC|}gr9&0dnA{Ci}a}6HgejeD1rNO;n=8uYm*$*J&+!YGNztBjiCfN$$Y1*9|5RHtJ zrZQ)DcDQLU9#oSpTB!#|FZYSEu+`Bwm~dASps4OG3}K9^XMdK1I!+Q_6@Wd0QZ85c zv&Ry=mumz24c#^!y4H0h#lPQ@KQf!yR3hoJ3g(ENkIRt4v>tE2jk`faKovFI^d7ob*( zgFUSm+Z)@wBF&M=OkY()yDt)41jrUn9Nc3ek7^!jKg7jVxVpc=fpp+dPz1`$b3?Pj zt=_%+yuFo9NF=5F?C3dYXrcG})qAFF<8f{3L;FH{2rpqAF?`Lfor$;Gs$r28x~N4g z<`)neN(SZS&YSQt?Br;`zQ_}o)oKnXANP z@|x~z*H&;^(0lW*YeQi!Sz{4tjtB?$qK2|u=*e5KAV3BP0>Qq%>R6u7J?mA|hBk)r zdmo1!WR(>Bzd)iTTcn zpen%aey}Lj;JWe?gj?vOAHfs^+_9c*6jOfkK!<@n{&4f{$Fk~#TgmlrK)yy8k#?wqdj%17>D-$LFs8FOiOHX5OzeQV1(N zJ@}FFHj~ZF^Yj#gb^i2U;SV`-k{?o{LvQ2UD%R+jywbS#AP}Cq*@;JuZ}%#L!t#2!EK-K7t8m!4A1NK) zCB0q+`|%(5g?`wVT`%7RHn(@zI%YA*Uqfwr7uwd^Hxs56Alpjrptv5RQ_6H8Wg!N}A`X`#hKtfOQ*pEAXk z`Zi)RnX1SLKI2z8PnT`%8U)5N@t#iHuYT(?YsL}9B_YkIue}|2$g=;|hNf>#Zu)Sg zv+Qt^w=Dk?6ezMGG$sdBn=)hC6?&RflJm@}QFC#WJ+(RH)M}IWaQetRv@J7?oEN~Z^O-P$t zmFqPs_z8ZkT+G}xz@SO>CQdYv7r5WKElwMUIO~eJ@m$@8r}$oYtw>} z33KNe$tX8@mw+H@{d?i}7kC$!tm)#+*6j4@8gxeY4s}u+F2uU%T|Jup2|U(u86F^$ zPOy8FLcSpPHQ*?wu&v*)DCdLvtqg2juj~m%5fVh=uKZ{HxFGV2%Ms0JSrvn*GcE)R zoW#G{Ft+6}rVcasMPZJ=1t}AP2z~AGg11TZdA>Ik90QtsY|}+8G#5X+O23zDwC6Au zY7JCH*~ZMI?MzrPutf;^ZsaQWRk1fuk;Jr5DA!3Mm2Eq{cjKUF<9HY;Gn3 z4h-n+!!q)K#gT9_Q20q|RV4O|x5RkhwM-z`M*shxy~^tvfAP4H6ix$Mr$ODQ2S%hY zx04<>kuxj03)HJ#a{dr5D>-r@1ifsZG+W$QkSZ@v% zS|EJZb~dz$d{gwO@#b0R{Tp2zU<4z-h1>72-}J_n{cYAYaibL~=;0j0T3Uo1(Y@LM zdKH`BG+KV+17F_ov1c7t=c-Mr#Ws_qtn5%sFjOB=h!WH5A1kV6GKPVKCmGF2jeNkZRa3z2gx@Q1)KUgQSM;j6V# z{eNGY8632g`a~k753}*vAD9$1!Z%yZQw{ZD1-x0T(S(VN5vSVpPsRLt7na|nwaa&8 zHSxexYfG0Iv#kTJ6wc?&J2B}RGKKq@x{Er2ijFt5uWvB!?kDbup8>p=*sx-gF$-nW zYQ;B%dInJazk%_;k8FR&{*rt;AqT+Vu>Ls=i$VU=@}`W7V|8LHbL-V{fA27n$tR9C zsA>>T_FG1Opo#$|FN3Xiu9r8`bpWJH^MHVW!CcbOkALsvzrSe(9Hg}g+o)d4o+^=G zqXE(MnLqrm+#v7kKrDs>76%CVo%NZQiN^JlAIDxryci` zG~oEo1d_OgW$Y{_g=velxuyT=?r3Pf4G&V8f^8 zsiH*hDW-)YA~HUe8IJs>!PRe+n}+VL^x2{ObieV!Zc^6$iTcwymCmRofweF|uACw^ zf|)%e)pmpxO6-2aNAFPuCo{25Zk@Y5tni8FAWtm`j?a#K`1e?rqsh$S@y@Ng-!duj z*(Eg+`LGc`efsLS0prg;aBC~=!CqfAfd878!Y3FvuaV(GCTr3ua4!Q6E5~?%aHQw2 z3vjfFnN9cFHvsrfa5kMuas`NQR;-sgxsi( zIxSqN(yb_RU9t~~2>fdg4dgKh(fhZbb+&iD956wV<9|nC@-(La81&F;-p<@VI3uj) z5vvF12e`1OE65P9fV;;soz8>JA^j@i6C~ut#!~?h^yQ9@0vPiWUNkyYkRS5mQgZpB5}A>z4@sLn zeH7e4XP^%%s^k7Ej$T&XUxY_x&yJPG2sGY6QBHHhJ)F2@b z&>*gHp)=x8NEm+{p8{jbK>zWv5I0}q8r|K^o)*0GIscDZu! z>Z`Yz>slfBF1yFBKC26E?~tYb^F)W@87TZ zdFnprO-NgI;&pZ1p61E3XO=y6?BPni4B1>&p-_BXmnGIebPLaFiGeC)dC?z}e2CbS zpBPNJB>2YU;#gjuH&Mc2e_SYLO?~Y!;oY)-PH6Tq0?2_hi#cmUGpkj#zjwNgT`eJ6 zGd6tEC}{_JI#8Qby{s2soURZVf09kBpuqn<(r}t-u^3&ubZLTxJ5@!@J`*W*)jH+7 zgM+bYF9eoT7VEMe75R)GVUdcRg+K8hw=`uB-nNwqLwQ3KBbPXNcN9U34f2tZ&t}so znqQBlwPf6?p{V_hHCDsfv#-DtT>Sjf(hqI5!!-$+7%x`K58stBie7WKkCA+{WXBKd z$2JSEM;1WH$T(#>#&27f1Bfc^3~~{nnh~>`Vqz{T5uxCDgaSrRDh~N6;=ol@gtZnh z@w`}Sv_;J*-?rJj)XVm#r30pteYYg*Pz~nrq+RNDEMoSU&=D9A=G|5<9ojotat1jr zsXZ39q3J;;&ACKEN_cgyhA(x)^!ZNzz%=dzq$8iNNHtDY^u^e)-V0f_DHpr9VLbh+ zFE>M)!o;km9%!yA==ly(Nva&4n^2jvc}f3=NHLWNbPXSR`z~TaSiZXSR%U+z{?nZz z)EZT(ryxpEz+_3Cr_NRfy}}!#N6d(-=(EB!UyVc98h;+NMH(6$62_4fDR>GLY@Qwk zSIRR8lH*;>{P8ekkv0pt#{=V%U35fW4lx%^$~p0^k!e6O;Cyq*BQmu zySDcaiz@nj$B&rS1%=NSwj#6_s+k8ks_Nbpvp@Z4JGrS=!BHN7!?Xb+4`~qNxYp1H`IFJkuu3X0)lVvwR-oK$Zh&WkGOiJij zgIZBfIxENw2rXY5Ns|wZti<=3$YG-YXOf_W$Rsukp zw1=gX#qc*UJF&tMCd{Y{0V+?BqO3H)3TrR!BIJRHwJr%(Xl8Zx;MV9E^+dB02C zki6U-i(8;xS4}6`rapHkF^*`3`TNA?DL*i$yr{c-^%r*xEM#%V;Un;h$=ZOZ`kkw} zdB_Cefx~juM%~c?uT4V+JM60#T{*?Wk_+tKV}Vizw&7d4|w z8BDsGOkoL-#jE$qfuo&9>$L+WKP-YJKTj^?RsnM>pvVgx#Q#}=waz+UGIA3dm&?jZ zT{p@M*KhhWjRR-*eZMd39}b)6=bK=IH9nc9zsTAut(W2@?PhPz+jJga8zZs3f1a&W zDr`C~tq)(YQSn&b-kc)*clO#<-$4>m8vFXBt?Y2MC119JzmtU)<4UBFn}OrVNRgv> z2*df^4I%{ar>4l!n=W#sG1T$$n?!s1vZj!Lmb;0TC||jh1a;NGJ+@uzNFecrzFP-h z_0cOlEk0bjj+X?nBP18475{}(Ns*wX@c(X;g`w*!&fkDrn4Q&3udk>ICBdhuV;QToT~Ta3@bbAmdKA2uOsRLN?1J&h(%BL0U9e%}rocQ6GoI8h z0Mkk%AtAv)xstb|Fni$O&-vLZQ6w7VAjqFTh2)~@FF=hS7M2cJftRaKa~m?>A;Pt9 z85c8)8BHBqs5+-FlXs>X8Y`H*%6d#rDNfB-Lz)+DF$@ADbe+AY@GwobZFS4_*1`Ta zmNqa|j=IDOVE*}Pd4{+DPT($8X3Dw#p*!lJiPz?lFr>LnECIk6oPEo<%lo$2ec<|oKfI>WqXl`7el4QjI$RUOcT%(Sad!nRb?ta(i^N%ot!2{bMTJW z{20bWV@VZ$^kiy&v~nvieyctAc61;O)bI%oN{!KZ2Bhv_Oug{PA!Bl(2FAh5(-o-x=F`5S?7t}AlBqgqjNxEDKFVCJ~X1H6e zbVb{Ss&FUH(Y;x!z!ktbn6V`!g)(COEjj=fInrqF0{o(cAt9`#k2Z;8w^6^J@+D)gPsQl_Mh)*yA{&rIMO^S1(Fj)u480{CO57<@I182*eI z#6l_MC<;P<2W$Cqau5+ymrJF;0)LQWI7cr?E@5W2!S2w*`w@ley@~vqU+bP3aV47Z zJW0ajlD+>+l+Wa`F&0iTV<}nKf7j~SU`+%Y=iDL7y*w4jJOr0&XB|p-yyi^pIG}jn zNiR=RG6y=-T+mpC9baU4-?i7n-puK(HZF2^lT?0|!_tH&B8P+fxKjxJ0IkC_kr-`keJ>h^ z{J;xUs|jvMl!LKK$63Y7i}4nDSJ89LIs5(lc{*RA-!PK~;i4k3s$*iFpOkN=ibbYI z@h&%bZ%hfmf;j;zZzz|!u50IU5=E~l=go1I_Q*Zm>Y&i(ZPd8P0$jXF{b0_z)0b1S zvVTu)sl6dny#Q2)3N-hTOOWJiG#)#ZKn;nsN(s7=PF;i2)B)cV1s|s&%2ZyPOyY#9 zIMAe-m*4Fh)6Z!jOSCQ@{ba|Vg}Q>_-P4zZfvv3k0cbqq?QBXIR}DlVNe`Q@dm>=2 z>E7aqPsqG`izY3pWV)t&W|iOuOw{!B(V~()m?^QGt&90D`(mXy&oi+fO%>;oA|ubV z;ThOC3v|-G2MhI2Z?`r+>Iy8p`=QTGgliFG6xn9ybv2!`d^>T{_z)CRpbufy!}1Ty zShNp)a!s#>y@(IW_j z)DgmgwSX3wL$(M9xmwyHk@RrU);=I)j)N zs{>9u?KaqiBJAYx0eRUy6yjN>`^AnverP$njSE*sI~@L>xYnRY#Rib?fkX?*XxANP zH#QQk-8|ZV=9-uP4&oI7bT~HOaoFVO5u6pqf+pcKdF;f(x{m3OEw@Idw8a*E}L%+Ohf=eJrBUhENT z(8EV2y|4H>dY_0CAmaW#S*rw`D=c07zfh8jnnt}+uP=vlbL<=e(0664hxG~l3*K9H zEz@k<0rQo!V;^TIr=swe=(h5xt#nqDC1DXRj`0GyIP~mD%h*>y;$T2DyOeUWPVHWi zG()AQPxWr`k`2B++5B>f>_7tu!OpH}Uk{xsT2plN8=x$U#SN}5{M>Fr=i<|`u`M43 zzZ}K;w&?&d;-`U#IW2_cGh)jq^P{Ufd<4TMuZ89GUP+Rn(P zFe`RCP+rh{BXzB?TWvhvy_%|~<{p63LWp6w%yr>uYL*;q6yC=9FFw}cFvLg5?`&-f zV2*1F>fIROCrnG>%Sd)wxg4(7Z}#q*uGlxV%~tDoSc--~3nuF2cYskW0aHF?DT$ex zB}|6i8ZiK9)X=x91@*F-yZ;oSUA)%g=ga`j_+@2GxG*0V{z$sRTzca(^ZTw_&DE^8 ziZ$lLw=E0dF80~Di8pVlY!@axS0w9*sKCOC#a^Nn^uZPk71et;o2idsS$mroMptRk zDyWOX<0{WbecU`opnMOXxpSgZ1Vuaha9%`3Pw?xV1hwQ7RVJWIRHfs#XDUwMvAxXOebGGgTRL2ZJ~xjUSXW;!>9g3aeOJJdX^aksb1 z^#{M4_IvRfrBv%a#qxAZ zT_l^Ph+-=@If-QUTL{C$krX#p`NOOy9k;=qNhM5d$rbiR&xEO8Q;_`l7XI=NBLZfL zZ$ZJXk_;W8yJEnubqQU0cRI;LQ~KS;q;A}o(kvr zA3tLNv-;Rf0MgYT8~8HLSkPvtyHsmCki**oeQ2Y+8{UW>L8E^uC~-EXn2sx~|%u7g)Mzz*@KQI*Km*x}y(- z^%yu19|{Dc+;E~60u)7Sp2-|fO44=pI@e;2bIHGTUvc})w_5JQd+x3x>y|5MyNyBU z{@c?n8}xnn(*5l!cLxVr7bXu^PNzsC@`aKeq0Q_=Oj*!f>%IFrp}9wKv<=f@No+&& zS%>GoQ;avOgv;Ta)+E*XegJS{()kz8lh}})g4{q1TKD`O&;y#gTla9>gMgz9J{1d; zRb`B>I?MeNdd37OX;*Fq0mV<%Q&0c65v;{N-~7K;l#^9EHIz$@yt9rlm1m?r%^=&M z-;rf~# zMuDn^g)QnUVDF0W3t<9*#C#=D(k!7!Tj%ZqMcdGQkfo}Df%)u*zuN{EucqnNF^FBI z8h<@o{@%JEySl_Y!s+_jWHUZxBIC=_?#VcLcJwt!mX;v;0&*wl^X|5-4Jyfb@Z!%x zPHc|83JncEBoM@)6z%-78>k-jjJi6yD=KziRpnpUPqN5*8)%#G%1?Niw`7bdoF~Xq z>A459zVgC4Kfg69GFLSnvA_FreGuH{juNj{b3heSa4OyMwRX}C-(BnJsur6nT&2GdWF)kC|6ta<=vr(Ais1? znP5K#veSM^7n=GDPB4iofaN@Pec+k+>Mu|_plWF#XS=SWYb`XY01~XUC^Qtg3%kk4 zL4Ah#ao0~=)f%pWI;j{#flKP0PHp1%&d#6;Dzq<@!l}fa zgKB?!iqYzewIX6Ex z$MK;mQ=&^8drbFnc0NAc6r=&tgYnb=evXeEX~%?JU1a^crW_{4%{c^!a5~QcV^{6o z{mS$sYIE}+UGw77_RxauiZ`5XyO|Km(G`Au6#b(d**@nJ3i7u04&9#o5Isec;W{bf z(>W|vPV&){g;6plW#(RMf8O$0*eT+K&90ow$?=f6CLX(pe;QqP=FHY=(SfqvL^IT@ z`G~0JSKO*O7XGg3$@D@u#7yt8*}bazUn;3MX%Yp;uigar96+>Jt6e{cpOY@bjmDjYc9gv~eU)HH=+0m%Hu+kGI?@rttcjlkf67=LoV_;b)Vsnpnc63fKaZk=G&Qi+_qG&0U zO=s@NgZ)@u&t@jz$}wRm zN@S(6UBWmCc6SRBXJmR=YQGh>KulnWjM4?n%J;&}b`ORBw@lF>^=!aIO3|gNvU13C zV^E2;&ZW^!_@tEvI`Eh-N$pdL`_&m3mw?3mLj$yQb-a12==G=%+bf~yMz4Hyjc}h> z^q5S$g>%UFQ8n%SMu5f+*Z#**EFM9xEk0X(o-`1#ym&^>(49|RX(kCpDXDC@{m|fK z^Z;bd(QeS#!TC(A(`|V9Ba^SLBCSjC5!P?@pQ8syWQ??cQU3rt)~iS;o{EKuH(mVE zy>!s|ro#PaP#mOzSEuD_FfzUF3w!>298m>VFa8qE{Bwetz>J+qMUsZx{>%(dYv4|E zwAkjx_6oYnA`%hDkjue}n-Dm2m#_wjZr6}0FHjEiCo{Yb9k3H(Iqvh~u@|9?$WFzy z9dTQ7apZD^e$@t?=2nCow|1|7rtFk(v!y6&%7FczKX^u|KP%7xjgDyA22#)>7cRnq zJ@;QB=ebx><3F5K5-YQTj0V4KOR^HEf_dMnt98WdCDQdpOZ9w{*`mbvYZaqF19t+Q z`d1>LiP!D`g+M1l|sj2)Cw2$%V{v<-YD-LBMVKb}8FyFSbq_IL}h-KJFy@&PM0d(c4W|}#2S=poYb%>{& zro~8I=NHm&K5OJ>z7ec8k7nDq7hKY@Gd>8~&{9dbs85L8LFA+fEDB<_Mv=A5h0`C3 zP0>40)UCxktcFH*N_Z?Hf++UmD00~nj#Fd%0U?#&en4{=9(jG7`-qZ{VJ1pB1-3)V z@A4X1bdJv-M&fokjr(=%G@DX&^f9*JK#k}^Xr<=Ib2YV9=D``}>afGi<73qQg@pfl zEAfOXw;so>iI3wci z&k2o>WPoLoNPEkm)nDSF5(uMr zakoN}ZIh?l`WqU#qK#`oB6bwbNi-_%OUq4*S^ilwgw4=%v)}kGvqvMql(!M5o?}@u z6Er4`$v19=H$o3h`$S~<&;!BfaB7YcUc3c5|8)hKwvwT>ka-Ue?^jfU?>}LRGs8}v zt&Rigy4MQ(2M0j^?CaR#B04d?8S@;yZ%-n8e`aUzkf-yQL4ee|RGuV*0CfIT<;qt& zbquq3LIA)5o4Liya4Yz?%SpYlw<{jL~MTetK3WtblOgM zl{FIv<+SqJRH?MDW^TDn6X}h1iPOl|iC+4uUiRHM_xg^dWv)~~acovo?7M^!*i|71 z_G`}h-H5d%6O5jVqn6PQ0ZVls11zDd7=&LNlHxzJWEyx!-(A$rT)RPEcBQ9(oUT#S zEYWg_0L6v=JOnM`}FMq>|!taSUnD#cQ);Z zN(pbGZvaetaf@{vD)yG*4TY8F`^`*L9@o*+b|*#`e}s7ItgAC)g!(igd14P=XyJT$rNpZ8bw9|5C!?yr zh^a~lPcO>AwnK?Kasx1yl zAb$c=cnAs7+hw@z(82o==aU@S`((L0R4>Dk(;3MGD#~PwyVaz4!P;<#n3+Hx0B4gr9HP7trPO3a^B&Qj3_8X7a}pAxX^TNb zYH-6m9o}hSE9s!skhT1y)0>N9&ULV8ioyy$~GNb!MgmSW|zo0|%9Y`ERM8k9QS#$mv7`H3PKQDJXsNs=~D`hYu zZR~dny)SBj>JWx~(N^!DSVg)xeMe0m0e|8r$|c~P3}bJ+Z7AY7n$CwI3anE@-=a5x zw#``9M$Rx)z@_|TdHRta$HgaXwNy?vY+=Pq!~7UpDNLD^D9|XFY(hgrtIY7DXfkuq zB({oPA(I>tu0_NQ><&198XH|c)@HyIIPpw zM&U=c&%fYY)9VH^Ua!PRID0Qq!T!T zuc}TZll)A|tpfUnL{J@Wt!<0LkaeE(L&M zJp=>5R{L#Na-8Rg?G)L#s>z=^L!fK-st6p6m}1psq+LiZS0%+IQ<*q|w#vbQSpM;k`&RJ8 zaRuu0iS(ZvGcL+6cc&^m>f{PA}3DBrWmUAOBbTcKUL zv-lcpr--6m{)s}xPoBJFwI(98sy4@=&Bjaj#N=1Z}bHDb?X*15Y5_xOS;tE+Stx|h_$ zXD>Fdojjp~74>GRSfO$R(~s293_-7D!)TQZ4ZG^dZc){e7W?T_gd8wO*bnJu#Q>EKaXEbny=%)pf3A$cNM5y#TX?^)itDO`i+oL@~q5eT7OR zKYQx9?l~118By#pVUoAk%fX8VRp`%Pd+O(K%%AJnNnDiOXDA)d!!3`8#l|QQ=wYPI1f2J|vvMCLheS z=oy=Y|Otg65U)6u{$27~DHpPG#!(2wi~YC6gnj+t1pwfd)mm zk~FgGG^%^&;RYS2QNK5#Df`&BB^nWypYek!iGyDZ8f%sv`|z~^RcPCoRm30wHSYnk zj*tio**4v#sy$62!J2e(?$3mL(599h>P6Z=IE;n!hmXN@!B6^itFZlSI+; z^FEm(n_qMi|DgD>a8AqfwZ3~CGvmt5aZUU(ia6-e|y0OD{63@6!0p0#n#(SIgzA3 ztXV|Hd<@Yt0MXRW&#j^MG9>RePD`cs;g}jjtM@!&V*h?XDV6zGGLKKwdh_up`R-j- zdAP*98Yvca&Ab_=Q@ik--%C+zc%Bx zxE1trODXBE??=UYk#UiHatrug=KY>57b-z&+eih`BwB`ax|68VeNI+2{x;)L8vlkAKT zY_KCU_pU)A7}2<@r>2~b`_updr&N9I_hNz6{M{j9N&s$c(y5Wq{iphCm)3mzKlx+l z`Cw63jR`0i+|6*6u|=^}G~&#jS|Xoqg+n=M%ke^J`-E4Kmf2D3bW7I$#PU|Re-Z~^xtI$S{nfMATx1W=ruy{{=ewqCz;WosF^ z+9n6Z6c8MU^% z61%K1ec`2Tpa>_I3F)CfQn9KDVwhJHVY-;;DLJ$Sx_RTNY_i9 zXkJ;iXfsp4m($(Lr(I9FK>!LCjgSBENOAiCG>v3L3`_ZBMHO^)qGR>O2uMK_q;flF zOek?nx{v2$$0m06s6%{Ni6D0sZE(3q_nWjzKY)4_@dY;!$7{;*eHcoyb%?)c*j^22 z98Lsh^|-fZ;rote9Ov(g0>Em23%y}0@z-<%+%{1^HdahK6FYcb~A-_Xs#qa`~E!<*jPO_R@7+Dp zr52kuQJXuwo{0^*=jWB5EG?Lwj0(T;9C;>C<`_tM_hP)OjjnxE9R12x5bFp3jMKhI zDK2mAgl07eeHx4f0I^k~3`5lgO^eIN0C1gMynE!^q})#FY}e+1CVjYp-HX)#0R_A7 z=%QTPX*W0S;-SC1y2%YgZ%jKgyNyP)D*%;U?qX8rTMY6J|Lv{UMfnSznIaEb0IzKa z{~erWOyHNPMdb^t%dY2bw}i({R5T5=Pi;~k zMkuC+@HwRSm_Z&?R9BHgOFni$=NuuI2QVQbDGT}w!FY%OvZmYcET_6Je2R3w9l0$t$F*;G6#P&4 zGkRx=ZQpB_;KgucMF6cy?DV^xDIqIlVbEMw*9+W}Ij!gQWl|UJqYIxdZ5BqKT$Oze zGzj)0SkI1vS~JQWdI1;YV?5M0Pdp!fgUeKEA{;)oMuuo>b@dX zjlPyR8h`}qDMAw;JGGnW(@|C$&pAHEH=Q3$Vg#rD6NM|7mnh^C(f`sk;q{(e zzA1I2QTuBP=uX?`cIExI)6P7g@@;JQu$SR;`Kum!yj$3h=KRCy{#wXZ%Og~^Vtyd$uz<5;8P*ahNeNpQKldKOg_smzK|ut+j*Uqd((MaDJrg2f{%Ll z29Ge0IV&=PwxZ>)JFxmg;)scb%tod=@q!Sq>f$=*Q05x790kJ7=o;7xF+el>JHtAW zpk5R<%~_0|`%>NuPU@*<(Df_NZ-YQRkctlq;VEO2BMod^Z&&tCH>$$VHRz_S9%-xv zJ9Ri+x zV@=BVL0;)=lus-+?m{v9B~Ps96MC$HFZkkOE&ss7LOzxth$(;!o>!kdn1Ad|GpP67 zj;s0L+-_Z?-NW3~B7wE^eG-h?fxE*rC-$9A(KDY1k7%EVv-e8Q8*<{i&b0L>iv5Nh zEU!7XS2=Ll5ZkjaK3v0l;p5~ox^Mf<5b|a?-++QzEDP{h%UjD)vrqVX&llQ+!z;70 zxW+qu2MD_UB$ZT{x@`MYG))}aH^`UDx=Z@~{0%?e(}OOT*6xlv=N^UMXcjYn8e zeWk#_*!U%VBM^CLWMy{HMwah+JVxboEz>P?N8u+5PJ5p&JQx+;Gfs*(`Ghb9;4gLWJ9NFnwweC%RiEoM{w0UKdDJZ`BHscYSrM}WeJ zz>IGF0F;g%prV_^J{P`BX}jt0K_|)n>ZFuv{owVq?tBq-m*Z5kWD-u>>&thPZUs5- z2BwrdcBUNJ)aUUQJj8 zt|H{X>(ND4b{#*EgjM+0EKfD-@|ly17l-sS{DtPl%gaK}I;N3{$%2`z&k2S^*uVtM zJ-->RC^sHqpI=*X$#zVcn(^g$ba5MB4g+;uKJ%3t z1=5M~*AJ<})u)wV=BiWd(cA}iLSkV1)%T^}KiNQTBIA%~y|VR&vM#F=7BTOPf=Nmp z2dbc3JZ){^*V_-bViO)9=Ib;ly|sFqd3`!li$TUYw0m@X8s19}d3XrzZK<@`Lfgwq6FFU-)TKW-jw9?%zk z^f!~jivBU$n`-GwIt8FtrU_KWdflyk53(vZ7K9+KDDcwWPT7 zu&lFZ)F-U*C`b4$lY6*s(LP|A_m5u28UI^@7wzE}5|k<#wEtQxfz?agHba69TG3b==w4`!TGvsF7l;HY&7s? z0QCf;U=}EuUr136Ow0MhVT-XFwcIdsSpV4`UuUD+M zyY2}XY2zb4DE*{QrNlbJGO;H_PC5LEPL%m)pA-U|ui6k!y5a)-=(*1!av%=t=1#eK zZTPc|n#ghh#h$7X-f4v;`m7F+KqLw)`r`hUimXUio{jeM^0r1H(lfIz5)Q0{fIuA?_;X{mh{L&(Nji8QyDL8 z-2DzsJz_Xf&)m+(}$9LaAVW%^h4&z=w+cJv|K6 zpsxfhyX*jy5IlaNeBEd&)b@1(Bm0Vs0zQfbC^Kwz-PnVlPOyO+?N8}&aZhJRx?LlNP79xFdz4Kj76N07)F{*5nOyA3*yAYeh|9 zbdr07dp(gJo*qXc%KZkIP_1D#Bae)N_}?o88|4$Epudfbyr=})#!JtxQqy@8XmrP; z?`KmUr?i`=L9Vd?&d^^?zM_jlps60Vfh}MMY0T?!E|1t*a`j9zo^}f!0?5i~ z0|E?P4Q-h1S}_~mh7sEC{35=hU%tU?TONxWDxNIHKqg8;N#i zT;4kOLK?Rj5P)Ysq&51;g2vHaY}ko`ySpd!&dL5&NrB?-UcfUPv`V#i@NlWoQx{ht z7I~8O^Jm@)rWQful$7}}?}?9{I1Q*92h_$S5m{Y>r)(UPx4!#54>$VZDQ2NRDkv}Kq^=kSbnI04L}!~ct_7t1=vNtp@U77 zqxr=Gu#EG-G8g-Ma4gF-E#4?4WV(%UoxWXNHJ$wqfI0g_TEm&4=Av?aQ%lWGjG`kc zA<;!lz|#gqLBQqOT6LoYCaK3_IH{R_Xg*w=*@H^WJV@u2JMA4|m96>#Pj6U{Qqg?zJCLNRW}RIy^b>`6CQIzd&Qa=j zp-UoQH&^||jd3`MNabW4y)L0*CAlZ1%l3YC+=Zl((h$oywybFYhAlo>P$55U%L0rV2nSXSj`h0=K4a-| zY=I_i5x`#rg2m>HEL=0je@_}P*@GBTg*a}xZ)|M5vQEKky8&1%1rM&={vIOm8@n#s82OST*ZR;Z=BXd?}SO@qZgK^7WKZU`oe@6wx zM>+%CW$?Ac;F$Mm1bh4d;FZv63@FG09gel&l~$gi0H^&2uFfZR0I-K94>*6VJ*&>* z{Z3(8nQD%^&qiih`*&bK2s=Z<(^mlcJ?w}h0n9l77fPGYdB@e`c;(OULrs%AZr+Lo zSLnC4waB)#)$nb;;}g*1UPiO!B~Ss`ZasgfgbfRc^oIS~{^O2Fu9_sW;4Q}9O}hGr zJd>?(bTE+QD`Baohx_)iapZz;s_K15R+#J8qG;}b<4XtSBMNsx_L}F>*4~2V*|^__ zcH7)G+<8i<9!N~G4wK2Gt~kK8_*6SdRfD%Am%cy zOM#$wUkWqV&peOnklZ!_@)f{{18}>q7vH@8DV@yEdIXv~40S)Og!CgZ;2S(G5u7Gx zXZ1GV?c#@`sy2F^TD(|~zt%+Z^DbVW^VYwf@e;?C(wB!b87|#8IhX~X{YtZ2x$!HY z@>!SM5^LwNEe!==FWekIR00CX$|rp>M%ob@Hr*NC8`maTQdFiG^vpy04qg*b%^1r00zTXosu9asScFc02b|1zP8WYQF~9Y* zH&2#T=MDb(=Z_~Eo6$RAGZ@|Z$`4V3cK@Dvb$&sG+}^|?wDy;r^vpYsOUy!*!;Qzm z4I6HHn=b@}udNrO7IPBzUi#9}jaGSg#UD2NoFV$$KJ-5BjdUM45WjWHCkUY3{~4L2 zjAbUmDP&QFg~r+b+R19~&OvYqkQ!!}G_k5S!y8FO&X`|{J0VY2NVnU&U=@7xfEScE zlMrIG5#EPN()KfH#ytXtw|8dquA9v}n-Adw8zz*OB5bKT>m6b^FYCupHO zU78hoRA#!lhfVgI#?8EL5y4MVkSmJkAhZ0qqVa4K6(VfgPR#nm9c8(r>46l`pKVUi zPeWI~W4#OR;E@7-8n4W=+h;x&BU8)AsCp5?QuePo7kYP~U#QBe)bY)Evw5LH$*Ht5 zKPsrpr2oAC8+H3)?j+Tz`dbl^e*47LBn=R)6A=|Jz5iRr+S}HLmx_6zE-tv|fWM5G zW~JwGq5ktNKJHEu{~lRfLVKIoK9E!*yl5T| zve{5xtZ!y|g!1A|`qjQFp|2K38>GAgAN|-s&O|0L^xA@{KITfK8tkeRX}_s~^f- zR_VBC9YZ!+8!!}XUv+C2BuKLqdt7jCj_ME9Pic0!8qC`zU=Nrw8`WyFxI2ehQ@!M4 zBD}qpQgkqIQCw@y6Ed?%PIJoi*@XtWwU#E$9ISN}JS^Hb&suInd&ft~Ix*LrqUZ~# zTp`9T4>xkJQB=q_C%2NPuPz$3G!ltvWRxKvB}pg!n8wO#EmNmSe?5~;Q9{ZFLYbTo zz>2^PbOcgwl$8s%K8v{SzlyHXDbFaZVZ~x0qXJzRjI_xsz7MidjN%eMTW6JfOzGrsUFiGL64+g`+ zU5JCz;exooa|RQb-fO@M&N@rZ+DYKj9r)=Bw%>zW=K@KZ1`W<{R5{p8?;5>RgosYiD3L^9En{J@xO3z5-YotCVIM8^O#@5vTn0EUV3=YV7X|y?}X#rc^q0 zY|v881ni8Anwl+>JS3et5s3VL3X$b zXRQ16`13fVNJG&O`P?bIuWb(UhXa`}ZJ4kU8L?_J6@ph~e&%u8xVk3Z-i}XB{51cQ z+xS`}@NdH$Y$M0#NTF6Nd6E8rvw!TJsb{ACPv!^L&S>QSdmFEXp9ZEN7T89v3|(wK+)%vSBpS58c;B{i=F$b1-SByrBqm~q+a*AUQ|uJWLgZqW7V8C-&`?O zVJD#QvLAb}QS%EmbQ-~lxD|GA2JayrUXcRq7b{rN zRzQsyREEaH{~cs|1-^ZrFygh^^6;Jr%3lzOuW8&idhLOBMe;f9pe{JD4SlbNkQcJG=tHg%Ocjwfl;GMbiKGLn;oH0dWc=;w)Fc&XCd zMGMMtf{-S|WVcs-zjhQd5Hj&6b+@pVZ8%f7g*pPJL8V4?i>)w^Yh6eTO@L2CKtwH2>t zTa~s{Uli3nL#XE9hk=L1-exdNkLj0ua)mNapVa)8sFw3YAxp+kb$|;(GCW~LA$;0H?t>8 zL50nyAINoNac0j?`*k3CBG)i61AE3{OKAa>UyAj1cxfhnge2Wx*lgf#XvHKO?kRtJ zNi%bN!{7y333vTG?B4+5z0Z<6+3<158}aMnap{o%qG`Q-<+>ZGMAg%B!yR%R7C;S= z#wB+F(Ik*?b1L@m+@cY(AcY1B284rwE~cj1r;=v{pv>#>g+48!a$ubuum3O^oq6|$ zkSJ~weTtIIDIbTY1#{zjIC_wk>HZZKDdGzs2j)@pt+Ain9(?UmnGBP!rQ|l&|0P5< znVb|!W?D8&FEh4|4Y(TVie)+1C2(R|&3rqm&@6nKc4=mo6sqja0CtivSTwg08=h=( zq$Xr?NWjgV92!Eshk*k65V4WR6@z6_!loe%h(Uw-tU-Tpws&pHtt{I0M`bn%F z5`2EV`h9!b*|nqN0FW=HeB2kYT6Xvq1?1B1`vyU3>-eo#GtJ7HX!T!h?~H{WA-ik7 zMExWh*B8n0bmo{6{So}z|MC$No!HFxHJ=2#D%Dr%&VI3#JY-Mx zzMt7?Qsa>tQfP(O*aL)YzFtaW?xbEUie;iO(zzn%3tih95xUVwk4){-IDcTG zlJ@O!Ok;3F!(zBKgW+JLKbbR!1>{a`DN#d8Z3eIqDV^^WwsG!$$D619TqfuawgRFp zyrQTzF;z{U;EK1E3p*>Wa`PGqpQ%f3x>57Thqot?f5-}V!*R!J;TnJnbj}wWr-l<6 zG#~Gx@k?*I{85fsry_*>$IzKQ`E(H+QJJy>7lXHmBi>tqn*Nv9#V7QWD$YR#T3Ju0 zHVLgh84y+M<|Mo-KcDc3C2lRC&@8yP`l8RratyX;;`pP~%PejO4_9;;NoGMbF(aL{`4GVA{)<&X5e^pyE*WIJXr{J$BlEK>?X z2DP&9$(6?iI-8kc@!}({7rSs|e=hB(8{tMNpv_15S){|Z53T+{A-c7?Utsl7@a0+$ zT8XGCGTb9VlJjwc+qyrHQ82K{V@7HGeNV;eiebe>;? z)tNw2w$|RSqvO&3PCTPEx?gjwkp3kS$F$#+HC+|eD=}SR33@jDu@AxV;2iVrGjIlj z-|{Bs#J{++cm;&=Q!NzNE*byYo8B4KKOeJ;$E)U>itiSa6w^f9_3r{k@&}p$yO+(R zG^mpexe}(J-dE09y38ViJAN~wfg86cHoQ-a>0DI&;*DSjV(GLsV-&4?r$F14Uhh%! zA<=|Yeli>F%a@LIoHJGkf;YjHojPu&h2LuBu-47ZN9`e2jlRF%@av){ zahiPnfC);){jh#Ig@M!h;NeB%IO_Z!>ASzE5gv5i9!QhpFh&sI(;2cYFQ6UzvwBO9 zb^Or@1(mbYEpdspXA>C>Kt?3@GeWa}3HVxU?h27noM@VrsNlY}a{0R)j8U*3TfWm$ z1zO8Y;b{Nx@dMKV9c%Uv23!*I-l$&t3s>I5w}qeA2F6-6YwQHO0DObmUq9t*?0b z3>M`pKj>%f7%PmHjf<)4 zyu9#+K!m9K@W+!>b?^BD4EtG+NAj#(i7^ zFMS1g>7t2QVWW{7M92Fb{zL9oH(zL9n_ULy5@rbzrHO)&eGuw*NjA*}WCqUTrOs-N zjp4SaAsk-AmvS*~4^`cA!#n(V`7`)yt+)GsJH6dJ(VEfOy3C8>z^DG1wCy)LH>GXpqxvZ0ShKiBjF=25UT*!@o6j4LL=FtafiV9a-z2CnMiXX(SR3m%L6O@*Y1w92kS6cAVO0ukuK03 z-LW8Ww`lPFa(e1pkw)FSyOrxZx>hgGVvA4GMa@yq0_)gz>Cy3CUTEM4OVt` zY1wn5{~mFtYt#T}opwj)HI2b+v{{yUCdilDYGeF9Phdw12W$5nNJF~*{7LSN;CS(H zNA)_s(%L|8S2Blcs_df}Sc1PFpukh|R>Eq@J!eikwcc?{VVT2Dj~-=cC=;fDaXFZ0 zGO|xZGyY^qJ-?`zs7ZcODgJC0c8uyxLv186u6;8*fMa^e|89GF)|hu7|4X3fMeQIr zF~b`K%1lOPM6v2qkU(eECkkptJLfpwhmMcpC8P_`>>qipQlbaXeg;)neEp)O^Whr^ zriS%6vlPw#6UXM$6eo(Y5qZ&k{{Q+A7J0c7K&?P`Nbfr*M4Uv5^|qFK7&uLqcjIL4 zDnqRfz{I$w38vAGA3LU-4o6vvp$3QDG z@E&j66gbSgXnjM=$3L7gSNV(Eh?-u|`d2doE%XIC#LF2G%ASSJztCa+)*m1PWvxMK z2Qvq=jsOs1ePkLq-oPYYc55cy5|CM7UY3KcP$=_Ac1NJnENH3kUc_`uo%rGH8=^GS zf=icS*bExAX$(0-tdDpvl_~eOyq5SOQ(E|-<2)A$W!VgAdED;2zV~Fqt+j8-Ex7vB z?718TF|Lfz)Y|2?;27WO`Ltj2|4St>4;=Yh(QqqBj6)HlFELsQ^8t0StW6YS{Z75n zib~CMI}Y`5W@1@PXc*8p60P|$xik`og0%bMI!%L7uO$VkSiMfI-#VlH@kY#O=r&wP zTTnk{ zd9fe1a0!_e7Iiy_Uw|MkE$Z@Nq3$NJuGG$-r?%32XrEM25C)+azfmrqi(&r)u22d{ zySXYNBF^tagW?JdQSc*_&%L!==Y^GYC7uWu4T3)(y1?F4cq{Ujaq4a2w+V>HXR z8)V3thO!}@Uo*CO(Pu9N2se>FlR7*$H+7B1rKCTJ{B6G6OZ z_t5X1F`L@gqpT*N+g3J&>mj`c-w8C6HjNv7TZvF3Epi&wGtC`d-~gtb;=hK??Cx$DIKWVYu~c-7yX_cl$SwY z4|kIvUX9C{BvH>QjIBquZ9a6b4NMmW8DNj<%=W(uoMpGp9KW&DtN0Zkl2M=yD2WHP=w0*T(~X8=Z+%rVuLy+TB@?&0Hpb?1xOcRF)dJPhh%>r< zdbhtDxq0*;FdvgUNqF#OgfOZxCP>*We8z2LGStq_kI76!^@p$?v(o(q_w1~T{Q$Mc(oZXYf@k}=of zdf5)^F&5nH)=d>l6^Y&fGy}MAD(1BtU6$;Au()IRZCoQHI1W|6E3YHj~K{iSxlrm;G{D&J#=S$s#)&v~R{=f&e17uPR=PP2k4 zI|ZqJo4-~4q;ovXJ3Y21$yd5Go+jtL!_9i7LI3=Xzv(Od7I;rO(br^cw-+gtP#xySUZ%q)VRv;wBXpGMtmFNGS}<8Mf+TFQl1drc4lph=nFuhsNXkFCofBOnq!{Rsfc(7N{X-^rdkjgOPxG2~qydd) zou49&t1fSj>Q15VDY;i~tLg^~aN*U(dQ|4+@ei6?MBt&}$Qf?PbqHnKI-SY7R}H>g zv1$ej8=HG8F7ld^y{Y1$?szxwCdRV)C0cEf2?bY%FnSjfOm zz?eoNtB23faLtPCNz*|ts6j4+Vb%S?RdGG2PS^E9`h0Ta+du#tGLWY}swmBf*Sffr z>yQN0)R$-|$ms;W2zl$uTnXMhQ8Mbw=r^f7tAyEbFI_R3&i{b7p<~`-4^u52 zqcT|Avco5BW@>+@6ACpync+F3Zz%}UhxgR&bGkkQiFCa>C<8yFqy$wEFU@6^F&T|%Cws&=tH^2}I zc2c9Qi9e8p5eFqTZ$zK!sBUIzQ!e5wvEs6yPk(C<+#Z}6+|A4u>Tb*%q}%aXs@SpL zI-OEEaSx$#Z<^3+Qqa`VJiBVo=ndD~I=zT9B#Sy9e;R=FX`;OJZ&rF^SdrU#wN1@+ zX}#A`!E+xP9uudg{@6MDIDx!(%h2ejxRz;_Mxc{ChgL_%d;QXKF@&U(`frgFyknpt-5a(g_wBj+|LI-Jm zO47;p>1D%V0v-qdK+}c!w?Y5QtJb)b=0j=k)+D!#);`HTch0^@hQ5uxHuGbgO8zU; zi%B1d>Oq=)j~$T`tN7QM9Hmx5&h@Nk%~sp3XeBhK`qPJ7{<4gsP`) zrsVHaKfXI~bUTsT74Cbdy8|ZJwKQ$yCtvIZ+B!OF4z1h>sFV7k)1%0Z_|K=oL;)*+ zF6q_dd$Zqj`XJW@$lEjB_tRep-Mb4)oW!}gvpYMw2s`WS@cUN%%OW30D(%=Dk4UHi zT$UCU`|M}ZKJdwIa0lThB+c1Xu##>GOOcVcD>{dA5>8-{RXvfRlO@iCM7Do1HAN=M z2jNQV#c?0s+P{(D#E)^?-v8t7e#qb{Q}YDKzbQO(fy_4mP3RjH-Q&kcOHhQ5aV==Y zXL5u(>~@_l;ArE(N`}nqkDcfBE(SPM1ZLhUT^&|SrL2el6l#Z~)QV$`Ny4dfzNk`e zqV{E@&zCCF?xi0cidp z)B&=y-|7QbThR4>lvvBR%z?o~zH85A!XmrR#1TPfC>QCfP)i!KF8i4LJ9|WO_y-~& z`aD6-gA-`BDrwxb0cM;xF#$+IBit(*&B=5w;iQ>;klU_4e()xh>-L$M>@=y);nZ^g z3;M0pdyLq5C%OO98KL-V%L)~dP{ZMViH-Uy$%ZmGy!lLG47qvHv_-h>JbQH$!=^L8 zFV`_CinBBbBo1g7==K0X)B)d(HgTc@KOJhO5GA{Ff{n^Z)7=< zo@zH!?-5yM-M;c}OZ)l$HRpp@D>&RO1nroQ426U7#yb_YDY@EkNRU>?sSlXTcs2T8 z4bqh~6Te=sGGF|7>WeqK;>sM=V)Qc$pO77ch+G;QFBqLelmoM>9C_?RI1y?89sd=0 z+a=Jd%ZvdZ5kmZtA?kdS1oZQf%L{ytf+KE|RWqKg7U2==swOB)qFsL+XHChFLx;BO zP)}c$C8nXXH8#b~T=OLz9lby**>*-6s6dSU2jDiSMG}+fVf|maa z{#7Bn3Ui~S7duEs@~CUsMR&D!i*Pd?15CXbEk0Rmx2-Ede7hPat^)I%OQ+S0wBoCF z>!qTa;xDXq7CO|X_-9>&Ht~7xu>~XJ4}J;NM=)Dhp%L$mmsulmO! zO7PbH#VyRcKKvE)I#^yCn{T1joM**!YawFV`@-_uZbX*}bjiI%irTZUx==lWPA>HU zRN~?0##@z`lgHtKV}TV+{90ADy4e)CDfUHBmlecrNU#6s*-$szln^4_2 z{;N}Qo`b_Cx~m8GX05iO5EmJiy_pw#7IC-JJD{fhqT`$awuG0lacX700m>70YPOo= zmK?h=^ml=Ji-#$*MgRCOQK0b3Qmt9mP2y)c@wm=-3{AI9`juJshmiF6ZAPbY!@`hO zb0uuc-{!*E@d{9HVZ@ectA7Fp=p}fY(vStNtpH9S$*GE;!g<mssfdizrFn+c@bo+pCN z$U!4}3cr;AT;1Q_pYE29$2Cd%zqp#rvd5_Fmx03h-X2D3$`gqpiIz}H`}RbyUwY-9 z{o>6`(9>F+MZvYpc}c&$IZ1!yZ~|$jh^x)gJIHR<8gx9tT2ONFn*HB@k3qbfJ3R>>Al{#|0~L(pn`NK8x&>;Yq+ z_S%0!B#S-eFyEQpxkle>7eN7mu1H_ScVl^8gQ3)d$J)DR6?2rby*Owe@%C#coc#CD zjT(C){tLZoY9}W;Q629iB0G=H&ayThgS2CdxvLWHRueQaiIRpU8xH?m5iXl>UmpJW z87?b#OD+ciD>RYlo3;uf8qiGPpzXWLUc*yJ%{*2Ish5!!^yem!5~~Io^si#jBt|?I zo*?7g(r$Gp9cdC@kg>*oJfI1O-)wB0lauY}*Q%AwWm-@=5m)7D)=6>HKE{*(1~N@JF(w_b`L-MA=rrOItI8gu>oU8=P#yI7Tk{qqss*=V@)`bkT=8PK%x< zjrc-oCDZg_zQtmoG(tfe6>M@=!Jn<7@@OB;S=+b=Hnop7i~M@f+Wytz(a^k8C$DVq zVxoyE_N5+r^;;>EF*m%cJBXya-7q0*y@WhOGkjG_Cx3f$O-EICSRi@5Y?Ax3tid%i zEvqJp_YVnHI zal5`GbgTTDOvFS0b9^dix6d<$kFJv9%_q!M1DkzQE*P%5BHCV!5&L4x3m?OuU5Co< zpTX!HI!@gAqMGi7f@zYIrkP&L=Is$Il5-!V57B!v6sjX;zWsTDij~o+LdkZ%ssFZ9 zK5;YE+7Iwo@18xK{%LDgYj|szn=Vzja$m{Rl)}TFIRfV? zcTES{i(VoxuA;!m(+VTlNu`>@+aJ`T%>|L< zq4>`p-@^gM2|$n+t~p{f>lKGuxcaYV?Gkl#77d-D>UdF=MXXQ-Pp5TRkcjoSSrg5| z92@o;|E|V}^t7)ytZ1|TvkQ(H?tR%r(ZgFTtOBz=vs{`l$LtdRbZ9&X8Ot-KGq0W|E*-P`uLiqkRo8o;RTfi~?Q)S4?$GtJcnMR7DK z>7P-x{$U+nI>#FpP%s)jFuayAUJ;SkjEs!@FX|EtSETXXJPR4

?<>B`0C?-sL?H zEcJqw-zo-^Ady9dW!@%QT4>N;-Ah;6)Z`%q8Kp(iNjtf}SVZWaDnl>(cjvgY>{(<8 zv~=va1C*1R4t4B?u7b4un^_wyFoYWAExwrREBORQAD_mhI?i||>Hms&aA$UihW5>( z$tcW4RM((3)|HCVUH4;8chITBPhWJl-M4`D1VQuDhD9U}+I&sRM+v^8J(Ie_aAfn* zpmk!YRZR5g8U6DP`2wBPGSRA!UUJXw$ge>~zpFpl2%@hQIeoB@0w3zxU@|~wZJ4oj z-AX1VRgg9t0h|;%S#CwFHgazI@AKTHFx!!{zn_IXKm$vBgh#Bnmziagg%j6z61gw< z6bi7HyxUsklLqF{4xEj(U`$O`Fn;y3ahE5z@tW5fTRj(HrNwdE_-!ZKH=y2O425O# zAAt;-!`ooWtYYtv8kf>3J*~Vc5Xm=&g;r-9%z6wGIqKI`+c98Bt>% zYrmZW2`drM%$BY%<~)J&h9R$KVGE)MjC`8TSHGLY)zs!`lmz+T6VR!NS@TTFUz!alD~{rZvR1ROP))6=R2h7`q^#7IM z7N3YDfM)BcipFY&_x310F3aY)$h?}{ZoEariP6f>nN%&m?8R}yD;js5zVLdo`75I| zJXp=-Ed3Ef1Y6h5_1x4Q`lQ{uELUs12+eQ$qAD20$`P=_CYdL1^Cm74lDB%jCTf3I z2Wd&*5r){+Hg9=4B_5j^ps<$i<25l0!90mQ51bF>01dx^ZA;Z15zn>2zCS$=IExo! z)0hD8@s44%?5KVu8_TF%Y4C2e#~76#v(Inm-ssp4rH_;P#+X*gV;q&KeNfixqv(%i zZ8w(6LUcUD2wAn{tazQBnc`;~v3bm3)fao<`QM4eS&Yv$YWKUNgBIDphTO`1$?{BA z5BJKe8L0s2J`6E=3T4;$ZL_kBNcHV2XN<^;wG$~b8Fr8vIUYA?)bqYqLN3}lV}WnKKkd|B`He$7@# zPU1AhaXkhY__}rq?!Q(^cfT2LZ4DUfYeiH@rIf zY@qhRG+(w&M^hfRd6h3mEpjkja<|B}9-=;kCu(^v>}_vox<0pi-Hol5A$*{;Ff%vX ztf;$krlk#Fv#I8>dPR~k&8tTmPq`K zMo(DH19RfRcb3Vtp2i3*{Adud>uf_X1O zg0URW0w-LhSr;-6C`0VK$i|a?kX_$XTCuYb67PydeN~A6Po@D(4wJDs^{lC+a{N(V zZ%?`Zn4S}#jMzzS9{$Msu8cIgm3U*6e1Wt zaxdEXN>Pd{@b(A=hKE%eC_oEIA-D1SOMX*)CB&xQsat1TaV2_nJ8*^F+ZMI0nZu)Z z&d%X^d7JJgm1gob*fD>?uCXT8lobE&0t2#6r_cZn?IPS~1hz{L;Myzq@c*=v|8R@dMQlCuW{4C-#~@@-Wi1#!5bdhRBK51 zHj=5?G)s#6c+b2Sw=K@DpKx+;c#ijwkl4`DZPaFRn?O>?$#Jz65AQ!72ffhbvNT?w zJ1duiH6vqaF{$yZur~q58Y&H^;ztvbYnX`A$xC)d@0q3plS((HtHCu^s@dA9A6`u} z2F3n+a3buld9s56r_&J>h7@q^w|d-sZX2Q3Mq}?Ks#2j#hCx4pD=vS&`rt=%&E@67 z)%NRW34VtyH@wHGBvd)sDK5WEY?|J2-r3Ngp*c%^(VCj`A{9vGAoud9WVg3~Z9Xut z!GnkU1i69qgpDk-@!3BiEC)|NVvrvgjv9$Z4Geb!A0$7Vm0;D{tjNuXxKLuk^9jyz z4<0sEw7sZVCux!=oQI){bF-YgYuoL^_3J-)KL@}bef=cbMcOIpB6hk?Bm_i6<;|ex7Oab3Z?2p<_eq)uKuomYPxGk|z)xuCo^*_M4L=j^EOZubOgx z(c#&SrKsA=O!Oey?Pp`*;UtME=^ba=9^WZaa~`CN4dZigc? zt|X^N`oC9*Wp#IBiSsn16Ywod(8 zD+FZU=3Paz;wjBYKT7tJW{vm(8040qgD8|pcQNYP5B$fLFY5=ZQ=z6FSbutAd4A^p zVj{MJioHg%J1gX6werWVFzrfobaZz1?o<7v+Xl^$xm{8*^7nLhZs)>r%e|a?ccR?= zsOJT${QFZx169Dg&Lp#7E9wZVtqS7TQ2QUk&OwOJr{_mQFfL~=Wco>-zVSv%#kPiF6BLw@JP5co~1nwuVq4b?k|Ql=XhRFr@G}mw8f6q=@V%CzrmeS zI;+;En5HR)QF7`kyL>a1OSO@Mgc4NkmQP$yE&cje&1i9Kqwv~0Zq3~;mQXH+_GaYWRB=8VK3}PGt2Y*xD@=@jym1a`TERy0792aq}h<%37x>w zB?m1zFL+GI(5=Vl+??HE-f=&+Q4p7PTiY&p73W*T-}J;Hd&%+m_%B%*nNa>0K4EJq z+RI)fA2bMZOY9SJm6=7*mk(_L4AkZwd6CL#lz=ddF@wPP14 zasCB6^QJ>7N-KITD<}&*K{2Jn`%ojiblY!Ws7dt_=YKQ(0S^FkGe$Ew00k)~YB`#2 z=h^&VD$td6Nh(Q#HS9bCV`qeBbNtd9s=c~8b#Y{9+GqBm)Q3_`M3}EEo4xIZyKFyi z0_y}JdLc;(`JC(Eg|(7))a?=5i~hc}Tb8QY0}5ybMfvfVwwKs(NmKL92D155vMEa| z^BPWTYOwBRH}XoGC}uqRT8)(PZlwz=HA=t%PT+Q3^;0(Hq|*8g&o$}0Yd%0t9A0N^ zk*_y9Tx+eRPY=@9`%|h6a*$^o?wX$Gsfqybk_VK2TACsTk2E~nur6mzmeQ&$Kf_2N zt(iA;36@}9y7CtK(aFL^x93AmFQoXp-%hgG<%!3%g7AN@g1;@7aJpx}w{LWLz) zk=73B{HXD~3c)RcD8|3%FT8!;$6269EQc&@*}Hhl5QgVsh68dx!_JKE?d9wumf!jU z$A;P-U6>v1(w#5H&7UtFLavG$NBfujQmp*_Lt_3wpIEIeI}Ujukh7!za&zDVQ~3B)*@s(lDj_dRQrJipl>We&V)FDO+ZJSHs6hD#L^LDe<$b7k{I zln7S;V)&@!&!1`dC2rE*FAI1&WlMcMmvwnr8(JYJ%R;w2>mXby7e;cTQ%he3ybcm< z7Qw$CTW87BM+`!NW|Y+_t-wV@D#}Wt##78mzIqs&h!!rZcks*xBYovA>2>a}dHJvT zMajzO^%~vjWB)_1wd}eBZkL|sjEn}`PgjXQE-U<=+h>5E*}2@x5H-U7%-s5US|e0L zW{kfOhaYxo%8~d!nEha=ZYu}5&1^t@Mr2M_`c14TJHJF-DA;&=f9U9tCm5Smh%j?v zF;iSMf!16M)QjCYM1W~11V!o-u&@=j&XcHO82N`**Nk236YA{nnug+B5Yl)``m)?> zQ@>>9aXH;7RBS){DGcA$aR!iE+%i*EPZCitm}>(`-fdOy*gW|2r^MtdO~m>k`mx4@ zxwLILN~g4kcr#6rtW3VrOt-AjT{keAL(2sG50eHNsOPgc6g6=3^7a(Q(;<#u(~nX! z%Oc`Wn>uN4eFj0dv-1tni-{AutB*QBL`=VV;EpY^5~U{PYb8dKqs^NmKxJ{JSwIvv zLimWK+ES^cLx{jmuzqg7AY!lexDrv^TYII!3=C7KkVLn0DkV)!I^lK8nicXVGxV61 zn_bI*SX#0r$+!}H?f(%+2+~@>*S6A%|7LATFXR1%^sXC_w@`X9XVogXbzaE)VbvFb zYEn%KzC7V{(vY|B7mg2Ez|y9ieVHaQcXO-_!w1mi0UGH)64IjG3%?%~ae%{(Rkk(C zQ&2jzSfuVfGU1ynF^0TKg=a|5R1fg2Af4Fiy zWm?~X)vCL$M7=kPv35aenS#g2#j*wQvgssX(;;*BA6eH%y?O|dl*R*6k)GYsef?zS zwW4(Gu9n1wW33a{9l!karp#L_?HMg~Umt80;`^*g`k*i(jJqPJgM0sm3)F2ysi~-PsvKg~a}DICm|8W}z&q2j?rBDj*$t}Yb>E4Pb5`lb zQ|@#H49#qp(!n3?c}Rd0XNy%ZQJc~*Dmcl2>$yc$g++3Ywq=yXaoM zUeq^U(w^DJMQWUawvC2(m&l1ZY)`pIgx!p%!pQ?Kb~6dN-W6nnJXu8owT8F)!jV~! zD`uI^q9=P`Sa~M3*9k%Kbx&vl-Q{RW^VtYjzDwxR8A-KeFLH7qT|xVo)lVH6Zv+)( z9$yg|4gKG%5b6Y6IsIqC!*>x?<19O~ZUADec7KO zgYOEwaK?ZZ|3!Tl4yHf9Mb{GPwF0DACeE^rjdU!tDp*5}RBUZ0bbB!OykAm_OZv1{ ze}lda#SdY-%<%2jHm4U+J^k~#a3V5^M|4X+`9FOIZC=mu)lS@!2g~vqTSJ<^0Bn{%e~JR13fyniSR$PKOm?ikZC>@ z-S8>mM`JG&?LaM;>~M)4!0E1!LkA^~ixlRCHMA13H%Se~)qe>V+OOQq2%5y*Gm9u% zw;rq)lfOI605`YTKV4m~-UeqsF^Ime8;)4BW6;0?NuMVn3pxjENzIU|U45tcd{002 ziH6w$I{@5Rhn8*t<-Q#J+4FL?R>;pYeU(+79B=!X;U}r*x!Lng23cmSCBTR=f{8wdL+;oZOlB`@gSfIzCC}EjMZyMn>7yU@Y z1&XX3YfwYptVK#xl{=sJlyaM+aGjphUF~gDRC5KIByv3Ad{-0fW999Ju`F3%?6nB@ zYCcVuEcd%QzqXw1r&ZMv=L|S9^>y+hRmp0YM`TBPBx{&K$e<20_1DYX6rX=cMij;pZkrKuM zOCO7btMd_z{9Rj}YCt4il0@KH zknS<(`F8@NC?}-N0dS?7QJhj|6)59;4PBI`K&rWWSl|Ap3EynRl&Pa>akxqgJ zjlc|Qjo0~k%hqeE;V=?o;N(w@De74qH>(^O=9Al@`7=Fm1?!QHzU<(x-C@H6xrYEm zviJ3sajt3$NfyJ0%U)xw8afzT#;)-0?DKKef29ntK;uhf&F3>r5wGuS^2eN;4UyEi zC@tp&W2d25HG%_?YVV;j<|;ZLI`1>aW~RIBzteLs$A>^*9Z(NB+&Cc_4Ajl2761LG z5N~(($$Fh666C;$xK9`dndHj6v~wx#)PAprqdD6=^1TCYE=8-U+q=5A?&VJfb;w*-9Qavx-~KW4vqbZRVZieLG5@&+xD}*2a06G0xGj$#vV=tz$tAJmS4R zM>|)ok>=Kzy5-gADFZ9k=NKr6UDQ{&+R&{$lt5NOjK9J$Pal6a0`3F_chlzY;-q#w zuhgptYwZZ~FR|M#b#1*xjmu3g(|32ad^xl}>s|Qj%uF1lJ2Q-e84ns3L3Dw;YB%d! zqr-9AGj84$3k1Gu1b-&=nj1Y}p1D-pfz_AKq@xhOTu_~e$`YAlpgAK5H88oI3WVYw zC_$b8-tR9QkyNZp(_cAY_-0=v*UmR_F+jjK_FN6Y2adq+dL>OSKQXwDy$@3->J5=h zwnlK)-@^ztJ@+SzNGMXRXqsyOj^F2QEcSOJ$=ybXWQXT!Q_K$MsSWj;g&Lh`GP@&~ z5(tNmHJlop7E~P^fV89VtkQR9&eMhKE)zeoXh6Xd2w5fY;xE4Ss+>t4{|d>tbaH`T zLhbMt&g@hrZ{p2-rou7j-yU;bR2#B=2!Gi&wwQ4n{LBpZR6ntaRyX(Fy$EWM^%cA+ zShT6(#Z52WE&R0UFa%Vq=YN(&MH@I9$dpirsVSK8G78b}iU^;6I}wnf9&opouv)e} z`w6K44AI>^sm02y7gs@wmt4OyBWlbdgb}ze$z#at)Rq&1PnDHt2*86P$9C4TLL(`2fY6YJR=d`(7 zX-A~O2dCQ|CygT~9D923fa2Ogc{vYlXIvD2Y-3Z?EM#yf?xu}p^PCsxa%TeRC6&it zO0uwVt{m+Kir3kAo54SdKD%CPR$X5V3>ZevR~ih10+~75(3K3-3~f2J_Ujguxy}Us zJUW~g5`o?NTMt&Os)=8m#Ra@3EIw_RSN|Pp^6;*Qk&a^`>dzI%2Z_!{S{On+qYAvCMR0Ndydd6w)^+fNLpxh50MTM=6 z#?yHj;K57C=p)cPxZ|2ElAk_JH(l@?kfY&zQLpR|qP7CC8?X8g6cZZ6J)MTzijqOofuA|e5b41(kOMXyj>OwUM0u%bUO|2Q zyY3pCs=FxH?A=-Kb$u${k7OEwS$wek4)_2^bk?984wLQ+2o zzXWCmEVEJdsVts4A?V!lG|Ju}1TpJL(${Yd%~fUwB@54H;GHl+?|zQ6Hqb%f8r=~1 z2la9z<28HpD8`my+``bbF?HE z-Sd3ir^GA%g^+>hd(1Y^DYbO&Hmv#wvrMS7E*4<@i71C0=6$1HJtZi&wgn+rxrj$FQ*8$@1_#q07|Q`UNwmeK zD)Wum0`TdU`%yIe87^xl2h*OZYh&Y@*mD{X}dHMGRBSvgj zk*wVB-nJNGV%5w0H&4!vrC*Z{tXK&Zn19RKnsdD6N%=EgU^=m-T%|;A9(%XzD(dKE z+?f2ICj^9m3qA%vvV?HqR7P&FqTW66>v@3B_5n-O8|=jMd+{OJ zf5S7#%vX&vTPsEhspq$c6y$b5&`eTs|5pO!kQ9g}LEUFi+nEbg7XXsM2d4t=Jc zDIj(KZz)pI&{W^54t2-ZZD^U&_w+o5k0A;3M0i?$#9Ee+c_F3}c<5ZYZ*R2Zs%7R5oFCA+VxL0D|d5(c-Dc6w+vaS!G ze(e_H!lz>d?$~ITo7fr~-109NAH{ftnk)N0omn5BT zlE6X3^_6<5WD7a>Ia8^KbLfjFNlB#8v>fnrhLld}o8!Mqe*NCC!=IIeQHAcU+SR8N z%CcQhZ!I>BSvfe&_SLTO|CQNlIR*!b1T@=wo+ls9HU*2AMP!PjcqabLC{EpMT^%@Y z9XM~OGn(GroqxQJZRAQ+2P;SQ6mfX0;5H{NJtgp?rO}Bz3vr>NU|(ADjE@T*s1s^6A12y@RW) z+mRU<_B_w7 zd-45~5DQx@uM}Is9d$@wWv1P9{q74cUbvBpsr7L+E7fKYc>1Kx%cA{w%>6@35OCb~ zJ&a0EbK>Jn+LPI!55Y?>4=*JW*4?9qTFl5jRG_GrbMWjEI2b3IBH7W3ovrMX{NF&?xWAm)HUf8@%*#`K~*OHEmMRpOwn0X~=T*ew8H1=Y4{Td=rFke;pp&Qm;LcpT4` zgs;yVaBdAgf6!fb?BcUrdP%HJ{q@(nWqj#QcEDvvf4tfbcgZv6e;vs}v_htb3WvRK zsMn`m6$WoSeC=ob&N`3Cr{yU|3R;FFb%f^HWIHXpqVoILw&Z_3u-TlnS3A|3zrF=n zDya3}fHA;Xlj(b2OR(A7;dgTW(Gd~z@EXUNwb!(FeV#4-KS!j^!#Xe4z9u) zYa&^?H@0z>uW)rErqeSep?^4lNMtf@ej(C9)g7@`gqaVQT z$K98_gn-w?lb^xkkI37d=;eU2cB;+Eq{f4jUz`RmyyMUB|7$XC>0d@OGH_n)0-Yv7 zl+yRw3`ATz9P`KRW~L@7%0=endK*4G_f>tE{^XB@b!Qxna82N{sPUfoEg<9vrpd8I zK^j~f7I@r;!-fBbpenlA)4ew{%j(B#nQx5bG+|C$8A)&@fJs-%FQ^( z9U*>ow1i5A4=Q9)5!>&3fTT4I?TR4Fx(Sy9_9cAujs?#B2~*(GKc>c+W@l#uT|i5k zM8Pb*(yG3^l?;65ZP2?YG7!u(Q=nQK2VzvKjMW&E;E@D;VljAQYQ_=1H??FD@~d6#B8cWsCs$%13VkKuAj;dH%qR}dzj3_?*Z{T5*W@%r$L~Xd;8kdarFAb-H z;&_m|_PR|;#F1#JZhj)v3e_==+-^W4x^u^b3|mKp)cj^Y>4qep2e= z9wg^J_Yk(kZtGSVmUHvqfFf$>f*|A434CJsG{++2iszBs<>a4xJKlrX%hbGYvR`&1 z1~{qeF^S9hc+39x$;m)62`jmxR)8`WZ=PTmWjr&~8;Z~QEq(LFvW&XObV!Cra}8$0 z;7INW_w6g~s4+}bpIOTyes3ZvcC96ePdRzC0NJ76GmPEiw?C1 z`ZKw~^+HD`wWZ#>mW+Ej2Hd>bH9c(O*y$eKcgxeF;H-OP zprHH@hF=_fmaXi(L|Ooh<|=Z2%Q0lvId?fREmh*)S5h=Y{5ukAGRXn6ENMSye@@ z7}I+8KSD@QnrxQvZ0|IlAS8_!l0OAu-)~kBw>(W5ABJ2P7qC$L$WEcrs@ERN z^=0))=9L;W9~mH8IFEYMNM= zi%N6{fBBMB+_&TSpT3l1^>_k@V`8(x*@3Qhd_!EYFhB49Nz3|ABS&xZ7qiliiY#yB z-I!ri8xC2zu%Kr`y{qwJ!#->Aha2K0aiXse^4Lq-xyska8u2KGaM|VrJ-0?WwkPxV zkFR3Nu$^BzXHr5KGEfu6W!H-;WQB3DviMb*yc0NF&K+Z~-epwe6g-i!AS;}oe|LU9 zJt~f=D~>qb&(^;NJf~*VYdkwS?Ulym;wsH%>buGC{a{XwP#dxs^8zXt(Z3sJAbp>d z`VbsX8ew=|prpVT`ImKW04R5?3du_ARoCYGLGPC`zj1GCm_Utm z0IypqZby`MiY1(lxhjp7tZewD#E=}R?sGNyn8YQlUnqh4po+7Bfzb{KSOT5QXR}Y| zsh;;Wqco&^iD>ePK~ON)wnjpFdfD9LnSRpZxe@)fYTLc7?W^BkjT-KJMv?=vT<=Y| zQ9ZPXK%w<4p2%pq#`qz^xBk59V7~MOspqMpeFbqZcFiQ?Y`&Ptw^UZ7v^2;C)3uZo z%KDVf(jEZEkK={yYSfud8p#a4?tTC!k$&+my}*H{jw0|@@k-MFXC@XyBHcfrq)0( z%#r%66bncCu(1s<)Toa9>OI!h(rLc>z|Q z93~X8wAwNDz6_FBph@cE&yKTce3o!UrQ&WHLmO)|l6_o1^CDa<9a2djo5RA$&{A0I zNY5*W#j*B!pF7bLGzfznj^6vm@Hv5F|sa4D2;;NxUl6@g7-Y{Xo69*L5RE zz`=9+(>Y|zzd15e|H%coTeE`^*brao7f5*JI0l`S44S&=r>A{TvD3!c8i z3TZmF9`vd%u8<&@HIN zGi~iKcRks-x3UfQ-hi*Hdvgty(5wG34Un2Xx9@wP+UYJoq0(XJ$^}OS=*IP^D~Sa{v;yqy^E6RGd*_ye07!<| zS-ll`e&#S1AOIIfk3YgB~ z=BgOda!ye-e)`fNNijB&Cx%%r;g1RxgoStJt<4MOjlHJLGo))Qm3kTdBu^6;fWM-; z9aGUkW`AY!ZS0@oktU@xz&l6#Zdx<2s`;SrfEkRPkwLsSb{G_+$4s11e!LJ4!JEw$3 zg&q>1o5EHCy}j2J!BQy!(t2EZ9e;S`jI6ukdE{9M|3n(mKOrl$wERkaGjHs3Ng7N= ze&eZPaY;aI@C}ca?cOs3{-BK(2*Ou@tuW`{^S;OI^W7Qt{#&1q+rE1!>thS{wp3Gg z{yGlCGN2F2{dH$LWx0Oe$n$7k>7enq5W=POM>`-bAMb)r6H4bZUYyTP`)08aC%V=z zVg1%L(mB}mphp3`kQ-vf=y%l_s^Md>o!3&p3xrt9g|B<>HWA)ZkXWPUr5L1me+$el z2sxrCLqJ^KPT2}XIJwfh_eB``MBB2cbq91JxUHH>j=_zbnR{S*Bc2(-Z(EuzJ=w?n)@|{ovO~@JtORi3X&cID`sI%rZ1kA*u-79wVBz5MVpok~ zcU{Wli^AAA3|FBmt{U))hq4T4n>c^g5B@KI441n)*5=vO(C~_$#VN<=j0zPtl=g58 zb3((a>LX%4rb->Z;3jm6U3OSX?R0J(?9G^^sY^8W-1fj&2$1(kR{1(MDo^}JgFRwF^gvQ==H!`FEKcuNf<-Cnq!#)yx4%~~%Cx;~5^)mtlA5~+DjkrarI47Q)=htJI5oYs_;B%~im zGU@eav!2{n5p7w)BG*1HKNITVylcY#Q zmZ`_XGaz>#{pqz!FA_Z?7_OjY1BKVOUZRjj*8>^a8ED+POk$2d zIYM^a9QwgK;8G3!H9y*9el+?zl#!;caeNd|{M*Opr*&J<;BI-rgX55`Ui7W_)t z*mS-=KdqR3?4~oTb$6cpyzXW!T*Y_$;_xmcvKG=e>%_ZMlVrkPnMv59FU$CeyO)Q| z65i6I|C&GXu4H7S!#9uzl_qFl>gY?;NGt}BT%20flf|#h${M0?opGr1^=5i->?R-oZZ06edIWb#B#STh(Rikh2Lv(Q|Uwxpl2A=)sy-LM=Ia zT}8#uOq`dnw7pETl!EZcfO^|iB%Frww#?%`_bnL|*f#dQsV_qn=fG2@`sf0-YKx(5 zHTI$HbTZx!@inmn^-Yk~F1Y-CKwPb?8@*_j zvG>+6c+PE~Abpcs2kjwbW_ZscDDdt4!i=F1D8w1PD2aUFt2}vo`s$bF!_DGK<%1>t z^c^_eO?ytO+ip+&ByPfdMoaJ6Bi=|}n32IlQ$?#l(x)v6cDgPVU{Y0|^__Y&&+Kcq z5VL@~*A(O8XWgqCkDmK%ar&s3SB{Wfiqufw!RgnPzpcW{P$f&E8yzlrOh<5EcHJxA zLJBQaH*YHQ&K(1DIVzOi=~!JQjEVnfQROxdr$ih?5B(yc5Mdy3+; z8|NkB0loq>TK9dRlZLHeLh?zqvsdZLYkQh%&_x=p+H7`|f#7Bn7tj6R{ z{v(@FBj#yHa|{WbY?@&YxB2In;?C0s53&8o%KTtG8P#^N=mEYcs+(Wu-F}6WDZVL= zZl!kuigU_MI`W3hQNC~Ti?N?Ky-NKe*0@$5r9TUPe{{rf<)i_4FkWXH%WkQ(&d%(+ z{t7Pf-x%n_clNDsT}m{5j?7&zT=^oL+-VJ8Z;=fNuO$rwztGfojWuVHYwUQjbA~)( zo{1@=2U%1iC~X-_G`l0L$8k8J68HW+1+ZY|I-l<0KhX44(1I9B!!8ARkRx4 z`f!HsS3Vc?2LTQM@T?QP-+x?s=d3s{mL4aU zw+uWdPS&W+*Xml?kRo{wkwWl`g=)jrf;73F72 zG>1)6wj56SR2X(vot?D2R@F*z)?^XCi2itUqEhKb#p|Y(8tISXgb7+;g{5|#^ky5+ zH6!cGX2s?Q{8-FeOqf{}S&|inD<+tdXBH~sa+p~v5{DHpWA5jz<;iGHV}IRVTFO89 zqciB2F>!z?SRY|i03iXm?cLt?*l8)v@xGOFy#j0F&Su2zy)>s;$844t*#M65ePcH97Ur|fTYgNp(0r_cxI^Rits{~Ok`IvQ?sqNuIfYJZsG zMaip>+KO;Q-5VaVI}k42xpVF$ZrHYmy^u)iyE`8`WM&8n@@PSxtjw9;*uOD8 z+Hv@EvJTcvnD^hmsIgOBnh&>Qt}t0OGbQ^Y1RFcAJwZOu4vw4Iff#!s5(y}0>%wv3 zaV?3ase*SGs*$j%IC_mI4$3yOR_L?v`bk~I$o86Km!%*jQ1Mx!}Ea8wCN`SD3gNe2cL+)nAIZT?OrDtz)fRHun+%CfwN8 zkb^e<<*qthkBN8r+d6|0kq;Az^J5E0rk%8+zigCU!U5`d4aMy2bCB=xT?Gmky zYI0a0R2i|DekYsnNB<_V9~&mKtO<;6ZU(xr(`o&*=3>@QFUg^2r+Z3%!8`~kB7pAj zwtTp&{x3@I%GHLh*aMl?2#twvU*;_?a&bUJW14W(8P2M zwPMSkUUL9Gt z{9#!xjSK+7m|TYJN_Ra3pQA218CQT|3qsU^ltaTP_Ngf9>p22Ht}`#BV2159Py2=g zL%|w0!bzuEi3Zl5Q^*wu#)@y!sp2WyQB+=kh|&MR+nTsxPtUScy)e1gWWJ}zr2f-~ zBXfW@m>U=zBE$xO*GQGEkmMlNH%Tg7=B;d-Nfb;YvVo1voub$sUzcg zw(Hou(h2%P1?ky< zplR2r3eW7h#1B#@Y80xv3EsRu&yo%BWYC`?hgUgy-7&VL%1+eTns)a}jQR_PTc&~i zT%D%ldQ4ml<0HkE>zTrAxk91U!y80aDGT-C+1UM!ixE z=R?MZ?RZ`haJ-gn>k=%YcuEoz8RuZt#$!GqAt}3Q$E;P^cyI5yn#&l-2No{buE!?8 z8$RNV<=2rUJkKIl&y7(XqZ83BOKH+ib2<4Rpbz5$r??`|LN9y!ip1Y#avkon5T2LP z@2S|tkYu+f5DIH|;&oJ)^)M+JFpVjC1t^-GMGBs^y!Z23$@?Ns)n%x^`P5@~v)(kQ z-(hr-VI%SWvpJxD47>?~ZsPz`7vf}Ylr@x(K!}skb4a3RTGm2UR3p_S=~E~!(Xemb zafac=`(j^$OyrCJVv zy+qZ6)znx{<*eAw_AJUq44%v%8O1j}?O~tbisWfp6vwtbOUwntIzWH~EM;WC-|T2y zRzu)!EM=)Lx)&Uj2;-4za?} zdD@I5CJc+X&ZfPMh;7rCFW01NFEG>ZZ?)}po~rG{MV7ft!)lE7F<#`Go>vkQM5{lv zhrQiq3>J6(2ucQ~jP=Muw$<4}en_UsS`)htOL;hYZ)6v1!A~n-xw)OEK&{`eHrxa? zBV%$k?qgXxejI6sEzKuF{1p~bdEFJ&&8`M&iRySA6pKdI<|2GLL`;e=c|>HVFmO^WFwq2)K!x-V(T91h``@lXrH?on<~jhEWYIy zo;fJeI0feC^&lvGk76aRC3b5yqDCk7Yl|$ErBq;zFI_Mk-bRw zcALZX)v1cq`y|NQTi4riEdQWIZZ9+_L#JUIm!8Q*ZE zgtQ0zG(=8CVBqMJ^XH*9 z?9t#146^GO2dAc`5yB~a@E6k+;siVg!d88^#X;@fiaT?T4y&syqCq#`z1WlZ z-W;F5{cjnR)PE8N-P&N`e&+GvYzikv?kFv$D!I=pKoaR&zlqBmbvQBXzU(%`kUKk2 z6TfH_&gTSUv>6_FE_OXnmS$jzPfw#@SpymK|HAUZ!dDIi9h38cg?qV zYLsq?4{daf_KW|`WV|)xp8q{y>)x|Cwcp;JCVgibx!K|8hR^C=|2wh4(e4=5Wf78P z$taBc-930U|8AVE@SRm_X*iCE3pGr@aBGKcrBNW{OgQ#kz|Yfq(0wF(yA)r zKCEbfQas(<41o*FG-A5PkAfWoPGS!Ssd2Xi-Nx+&~9Z z?ibD~qmQJrs_2eG^8GA{UP{4aM=gu2fmxtn@9H&4{D{!i_|T0Oo|_|Hv=p;f05HeF z%^y_Oa)eS*X}Tsgt^m~)n*RI=eGz91F#rHBMb&DHAH-#Lx_Z@+qgUbYFI{s}1`U)H zBZ@1y$7)iLsWKSR=B>p+J?nZw;TbMQWxBOX`CLg+bq2X`-*n_V-A}LYHADe#xy8|j zr&M2JqLaU-k+JdHoTi$Qb_D_lgH|5U_o}WJt#F9`@4o15YxINZ&cz}0{%CId~I<#>N*miNDe&k`HN01wg*nnXqimlNKlE* zdj3kUg#{fp4@kY9>?F1u=;?XEdu;LjZ|McV9c%!u6H^^-R;zDwkGaC zGtK=A+$-sRC|8^-i`q+#RGZedulV~KX5V@T1`azoefIsYj|yk55+B^+uqhQroo-|z zXIoRmy>`Eu|8z*QoBRPw8?KA|R+;`qw@-=wq2<+>#ZqOHBbvd=O78UJXF`Wr$83^= zWefYj`-Q(dWY1$;{vYWe&<1=oI+Qe(?cso6%juCgsD9B_oR!u=5Y~ey8T1&#KBRRx zyo5Sn(XQyJFqaHgaQdIl_%bJUyDiBU`&C=8betz=Wf4}r!}{OBu*Ys|8@*U$!FxRh zl}>um=}aZ+3TiU-(t!69c|dob!=i_9Qm_y?)J+N$mH(hMJbTwr8uh9e zpA#z2jCk-XrC8?A%&4Ib=EqxPclm_S^K$%*;ye)~+m|&B{Khb>pEs%4GFng3KPUIN zzz&b!=1dDvyJCLPFL-8ogeg_k%FBX`U77AB3AdLD)`1}<3oqM2>W|` z#VYG*jqy1jPgYY{W1PRYJYfbjtfVFd8edaZz7VTt77?EkdC~G^z;!_2>O(^hiJ?M# z8%UPkVA3I`3YbulG90~Wo=vwpz71fz13rec4mfp6(e#qzPgI`^4(2IQkJilAcDYba zOklPqz?{C0xxA_hG&lKL=7rQ}!Hm*MK1Wa2@33*+LO9~-rvuM#0`AW`X;*`P*Yiam zNSVvZuDYo4n^Rw_Tky(SB)gQf5XEX5zcQYc61xGuFmy~T)uFn z^^(H>+GF>Djl8^tw5)KZ{F*5lN%~z4nWqHGud)8F@sA4cE$D-luRNp#$u$kBH>Uwx zg&1#Y7_fZOdg+dZ6vSMAhwKCjIsLxv)X0J_c`U#fo6|ejOTBwNj^4*~b$6hVN-ET- zWrfCWKf9qDC2I$w4);dDsSIUdO?ogE+NU_S2hpYf$~&%?S%1*JyBi1?cb2%v{#7vp z{%+L5f`s)Wsx)54fKGs{9%rO&EM$m0eB-{YakMZ;DZJr^;0u`)(J%!f^ASrmac)t$ ze$$>+QHj5Gkwv(db{eRIb+D&dnMp#9Ny_enl&-EZS71obY&eHxt? z2Mkb!mOv8-u?co>23Q+2QN$buHCVA+C3-oYu-Jw{zM6=~4w;3Wu_Y&`W^z**j!!R- zUfRNU0|76RhM8H8U3vl`g89E67B>Rk=NgOSs750?%OmFU{di!!g|RxW-tTCK)3?_e z&SQ>CYse*IG^hC!GlT-V*9E_n8Dg%q%7tk;Pm&tJJubSWDAwA)c}-2p@s142e}|*& z(!a2SzrOlc-4_SB(=*DYI6BLLL)`o~-`v#;iRa0M>O9?hcVi+Ax6Fol)pVIS#3o(dSC9%=f0pX5xTpfA%7{gHqG|+{KqWuNX6X}o~rR? zhDZ9~*m@rOjFX;r?QX)s4KJ#dcs<`1M!Y6mDkeWa`e@Gyqc~ep^&y;S< zgn7c8`Rj46VfwXO+S5MHalye11LkvF`vr6P#M3ggC)JMR7IixltMh`E zJB(KTt#1A)Zqe9FsVz(i#eek`AX9^%hd=^$B(k>r{R-~>`z!y=v71V(w??l4(yyq7 zk0o^D!?QCA?JXPY5A8|g@R#!c90IqR#MTIxK-OQVs2KJn@dqtSl^@W-dv9|KPy288VQ48Sr5=UhV|W9ViySs_rhFFIySSpsux{ zk3*4$j6<;CLI7m1eKPSrOphi!Myew2=mlP$Gz}stdXS`^oSasg515njUuvQTQ`l2z zvb-HU6sm~Duc(Ev_DR9z?XvGkyaV~6Va9z`*YptMF5&nRx*v#Tqv{wSP|#-R6d{nK z-Wfe!6ZqshNqV!t;J;1rNu{VHrqWl+jzcis@{Nkpq&r%t`T4%7l^E&YVKYE!i(n>r z#}YZhR7P-KC8QiE=q@W;1{)VWO$h40)1?w*gZKh27GI?buVJHoVO)`0111o;(R)DsW7kd`3!yS&Fxy-E-nYVC2G|gOz_XSU^ldPTpb-%dv?;?N+Zx zSpwLj9+sjPv4j0$U8WrB;NE*v`UNDZsp(%cFf|Cueo`t+wUS0VEfU}&vQ`S&hH+ju zQ#Y0GbJ;{Qo4UrBK0UG|+zv|{5bRY-WjMXQAo$N$RR^la>iJ~pv_n(d`1NX%5A@9oEcl#|R4SG#loHi{5r)6*;UWW{oxew> z|IH4X$AO*^%gkZfsJ|x6c~2dm)Zt|_*yB&UC4*8jex)~*4gHHCFZ?<=csG#L40L0_X138Fz`FCH@Qm2!!r(H z^h-f*83g~eRjmEW^;3P7=Mpz7nM_d(`R+N@tv{{pR8PRtx=tP{+rL?8m}m7;0uAJR zhh}HX0WN`i&I!;wc-$;rME4H=$h|kmR?WTl85`_xdA+uObNgkwZA|DysfARuY?mIL zIqXT%o=0kqwPhgy`EOQsEKG1}|LmE}i!P#~fXMGZqfB4$vdHiMYu#u&eOKF`O+NbP z4-^NOkvk)r)%YWCs7k6)fG9bdhoEuI|8m&>CO0QuTwiN=nQsLge*La2yzw{^(?4NE zgaj{ZhDwmh--Daw2l->2{FfwrSI%|wavtqk%&6&QsP(4hAFu6b4tEsyhvA(OTC5FL zWfK1{5CcQ?2^8}+XEt*YoFHptSY2)cJD>`H zI=b4C(RuC`0m_D&VbRv@eyyPh>!Jtc#eB&+=9RWUGNt8J1SvV z8pPZH>z0Uj0nM@R-PnF64|>dMF@EC!XvinCvx(34^YS*(K7XACZ4cYZ$?HF!eREmb zZE3+iq1gIZ^udNcB`d-pgvLQ`#bUc3y)rW_UzisXy_&`k&(7wkp$S{F1={M5`pVBX zcHj;3=^JTt4aW5jpVEh0&!lA}$rqRsn8@D3<=6T(JT`Yaz8Ld3{;cMGr13R7KPyHvAE~WZ#KkJ^J+fj6$yJ2t z(uPMQ8w?%ZNbV?>>c8#VEB_i9%`mG~YrIt*(dLNZ*xMp=szPhiBO~y6>QvMV=NLn_ zjq2F0C0R2?C7$r$;YMNalh1no{vv*T&}xz99-HNe9C18<-ka76QUYcY1K&^=$i*@B z;j{;P&Z}Fo|Itphf#*^o<8O%?$n~V4Y`7wXLA!nkrljiG$7%h&RrH(1#!wL@o}NAR z|AsnF_g#zH*At5K@tJ}V*MageI?|~`Gu6b^Z9fAIWuHAi?83ua1 z*trlsr{cNl0E}WkrjkIrBfkTWqw&?hGBD>U0<-2{IkO1?tD!XvwNoC7nW@c?P;^Da zNo#8pMt@u3q!z6_uGf};dwqv9d>|&*np3xz!Awg2sNZqfj)%C$YkYLwmqEQNkzyrp z*`26IwpL3L#*tlVq=rI)DSo(S7EUremBia9X$u{t!8`&V?b?)v8kDZ~es*_4T)hf8F<;J&4;v1n<7y|^i*SrnAjFNSFnLnn-|5xUE5(W}FyDNL_~Q5aK}F21 zQsU{U)wb3COfl~dXmL}|?t8g?n**9gCY@^0GxX**@6WaO$~+1AQ@iL&*afgclLshV zH-?;SwS)g8Ii$DrVXnjZGw+%~druEj*)(!yse?0ADfLR!m*;3X>_lIeis-m01d_#5 zh?a=-IVhX=+9*;1Q-#Un10Spl!vp;2l%hBLH_j*ZPM$pdFoTVj5Z5S1X(kfv%in?jv8#*ZNfMQF6V|C@FUR<#FzO?m3bj1D>8CuS!cj%Z z%c}21=g;cxW4|Y?8Yz&OdfeC1pUL&xZ7T-oy_>{iHn`HQ=}$;FtPwfm?|yypso5a8 z6Xi&>adsYGW3U!Po2Av>3|6M4PqsjKAf4Y0T3MYJZ~J&=X$6zo{`Zsf>FDzJpB5wb z%TfW^`9{53rKJ2o=a^a3gS(K7ji4Pbl(B~>>h#T*ZFK4J2rnylC(x#2)_p2w-4Bx* z3HdBuINdB!OQ0fYIdRD}7^6cI*D%iRCpq;p&Uj9mzoQy+K{T-bT=h&E3u({L#0oM4^ zU4j;Le!}UQf84Fo=z0u^ciENimxw=H@U!W=nsl7^Dt-MFnN2)dYY)?Q|20^xI&Uw? zNDYAIY~TNwQ#G_ETgrHZF+Bh`Kv1k2>1)wXUcp?*vrX?}-moRzY&2qc*eq!yk?8f} z2_E0&&BEm$Y;^ZFKkLoLQM3XJcMc#gw829DEr7TjAD_7wK15uP9kfr3cBA(RZcKKg zW~Egh5U##9JxcRjI`xtknr2gEnMBMeid`SrZq2F!qfDn|CJcT?7Q6s10@H>Vc$y9< z&rexz46$3n8w3j=SI_tp(s|%UT|ajp4i?sO&()dv1#yd8fvb!tn3JYbD0upY?(n*~ zZXZxd?Am^C=Y-h{UisdE$^)g5SiKCL4__Jf>vfhVNe+aZ#Z?4G%L6R5InP62a#Sjt zojKNyzGi{hxb5KjD_*T>>_^c28OvIyw_h<S~gFq^F3(KL^9KcVFrzmll}bbnOa;lguN$Nr56y$<2#_wu{RV|dv=>b{UVOun0+ zvt`D<(wKHbPQ29L$B&|M|@;EjEGY@IwZj9=mbrgzumMzkufH$Yx-L+4n>ez0Mwt z1`n{^TImcrrbb0YR91B=o-S_xS z25YNiRwC;fUM3a`cgqIS&z1-MdEfVU^S_>nfUx1cj#r-dF06%;NQ62q=exDppa!inUg*-5JZ|}Qpln~?j0xp zo>@?}sF^Q+=7oY{anjTe=iaNELPtaDq!9oTDXena&iIEae$*e@dVDuBwt%2oNVRVI zExS`5nRkC~7 zV}9HzDvnv8kSVe!dp%&DR;AtzU`!QGtp}~m7t1?uC5;t&b+d5O@rLk}v)E?i`U5q{ zjD)@wr)x~|lXHVV^k}X&?N&a_cT=)o?{v_P)N6K5b$E-TRmKC|?{`)xh^Vxs%fawj3;)wE z)Hj)|{mYCZDzD+)<%lCTJ$c|u3Wv@usq`y!*(85ov)7%;zPH$4woc!Tad?f~H| z^2jAO-)v{3w{n9FC4PkFf~PC($+CPom+BSe3L@@AO9%B@@up)dKH(K*YA?3{c$U36 z-5O9XG5AaMI#{020237GL#S`k-dQVoT}Q1sqj7| zTt;9mTluZO@rOvGD_dv0h>jxiS}Jn`X_Bj(?&>a6(_+4fp!ZET>At!Bm`%6K~5@=2(MuX;?gRE@&2y*gX}_>xZx@= zr_w$T4*1x~0TQHba;1plM5A)BUC?*U!gBU3b4s#51Wx<~jjC&@KbBr0a~Ek^>~G6& zg4_4_6+P9UaTfypw6szwEV&!Cb6C+pmP?4zYX6^U@|NIFWtAB{2MXYzfy0}d+kO`Z zP-y;GzqB^YDV^hmGN9Xr9ORDB^Ay2e_tz*w4&3sv>eByYgq;FYO zKw|fjx6IHE=ZKA#ZL(PAH-(IzT0OL;8JVszHU-iN#kA@fL&U#uQ??VZXW1_^N*$Gm zHEJUYMr#3?!Jk1)K8B+H_>v;g4=9#SmuRL6J4cqK;2G2Q4|Hi-!;3N|`p#?4;zwI( z)(DB`KNC3Wp*A7=XnkNV!S0lK4$0~Ff3b0wnZ)7=b`h~C8=7+{(K?$Gyg^o>5vLuI z>Jilgy!6tYxY36JU7rSKuX zlC$f}H#wPzNpR(M@;FU_NRB^}$hDzLzo+L$ld75bL`CpLQ?f@GklY7-eRg_QYKUxG zKlhOb+*`K?)a5gh(tIwf7{9i(3~L&4(Gd}e5|7cTRmF+MNAFw6r1zOWT~o~EVPHvN zybCvoE>=n{^3F=kqBKC9NMPEDkFX(0R?)hR%5;%Ey1cxQ^Rq``^7Lz0=Bs4Uo}E4_ zs{K*RRKp6@PLfN>lHMEx2csoX{uf_QS#PK|=CI0!3%X5)^WAtyIW)|GZ}UV3ygQY4 zsc^~KzQY~sGNfEzgeQ|aHBm7mae+dMo>?qB>xy91{hqVhWYU{<0%-{UxrfUVy)4Z# z&0$=Yk?L(L-c&vqNjRvzX6{rk!YGr73Dhk|=WfdJdFH-8LxoTk`^yJTbeb~FJDr3^ z=Tm3qet@$WIJ%-9m;dp&J?hL^=-~AzlfHj6W#&FoRo( zjiY&KWFm`N;R{ZhES|ClMB_@rygcg3y+K>i?&-ZPSKMNg6x->bC>FEY`e@tg@V}Qq z`o0{=9x+*9N%|u?`gI-Y%Z44)bsa{cNV?caiSD}2r`9JD-!XeghJ zk;lKg*XZe-`S^VT9)dS3dEcf_G`GenY2%XNl#C#V1|g>#Qld|dK1cy2IGu`eo74#> zk4{)*HZ)$7r;OzQ!cieieRI|wvL{`7l)%H@t=q`f@ZBAOI(bNX`lz;y=@5VTP3+FdS5y#KQl#>3fQ9EL*s)sbHyc9x zKPQV2?5Vj+m+ClZv$ff*({s6 zvT)I1x_YamgQizVuU~--d)PF@=LGZhR~2ss(1U%I7onGop!|}10hld0>Bg*dr`V58`81l*9U7b)?S!DDTv-rZuzRM)uZ7Yx@ZQ-1^Pw%yk%A>EC1cPSt#jdbS_(jeVPC^1M%cMc)l-3%bzT|?GU z%glA(*NJ`h*&F)#My6iK{TTMqFVWkiQy2hd=`^4glYl;;rAaVwV%b{(sHz}ay!Kx# z75pUzL5Vhd+et=GWe%cE`*hYEUW?ih@o2RV3>H$UvcI?tIdR|5D26HLS1*wLYHufR z^V^FyY&pj*mR>sKnHlyWEn?wS3i_{-8p1$ae3Gd@TD$xt8orq%IW}yQdHTITXDc)s z;jk9z@WjX}ue#e`h<-rg!?n=CsGQ_*kb!~kOv|zFobsgxfblsF*<_|Z-fgWMKYb9i zW(S^Sp_ zMmHPt6hw-HyyG87XQrDy;(RoT=(>l!9Wnm z#NPlM2;#i>Cq3f2zy&Ndv_Lz>cm)KYh%aB>(!f3ZmW>WCZ#}HZCdT7F!>^+ev?FuE zhELZh%}3>IE3+a(CTzPHjg>PtmXyiON@Mnj zEjd<=K9AW+f{#T8Akj-VC1OjQv(eaSxD;rcr~enR^u-Ha?@vBfLof6u1UdKf)$Xig z;1zyGmoT0XNu@>@=o~oDbZt^ula+&9=7iIDg)gEe)HY_b+(*dTXQr* z_DxZ|d>j^GiiJR^566_0;I8VMG*h<$FV=GqQx1A{M~D5g(y!kbMS*{Zz0X?NdXE}x zE3io^DA=y`5fQmXcxqC$^_ZfV+k~dvvb=}j<@LKn-AkI-h&~13a>RKX-`DCphj7lL3~cD zsPLQWy%z(S=t|DuvQw^Nhwf}rv&1g!^t^f8Oo&Q>K4snBGBED6DdV8Nh3xRED8>Eb z4%o)cI2yGgY3+N0<*y@O?~dz*lwW)poF+wU@p2+>t)&h24_?KQ!jM0(4?rzlP9638kWSIQLr8|WeFlFRWOHH+@b#JRYzBLy%z#U3-D?9UX z$j>MpIEam18lu`k9=iyt0#U#h+Y~sf<@5gu_l`r3Q1Fpl`&(e`w<+6-s<9Ixt&VT^ zNW?l8V6IWzUFYU1KdeT^6{N<>rgq^8@Nr*wsTQ^(nK1cc6TBblkGlmyQ(@IqN=y#H z`_^sB6}smkh7C$-f0kaPUkg3WG)&C|{cF9{K%l+9ih=a)4Dv~o^q6ApD`r%<-^vAE zc!^6Q;!sJ>Ozi6&x0M*#X^hRn{3>QC3M*y`Z7G3Juft3gFarDyPzOUXJwV_m%*j!r`7hPHxn|h zmg)UXcL{_9l%0RRG4O1-+$k%0h-6>)BfflpaTV)mu3Japqn!(l4kW~%gF?iZN6?wZl$Mdog&%RBrc5cCO<`eW zUFUbZSw{0kiSJrCyLH776TH!Df%^PQn!mxd*N5~9Ow+p2^7Gcwx?KnSs@;Tc!S>5T zLL&k>o6Xz7@@qUAI^mY@@oza?-xc@CT5s6T*el5=aHv;{7mLzpiY{`i8^Aspn}}aW zp{0)UtBzp2UEugp4ijVlmm;v7Iw*(cE)y&*E&Vo9IpoN{?>1&F8Y>{2^Y|>T6~9tE zn6`DCGfeh0IxY-7G^7zD^t5uZW#ilZ*OmJ}+Yj)3tod7^Z52MpNd*N_6zd8*!QfC! zSBrkyShGPzonXRvPA^JHDqKLzLz|a{!L=r(BqOc8$lA64FuplM^^dqRaHmOt0g05I zr-Dy@Q-cr%yyh6VQePMPkQ^>&^pi`BdI(>X%nU0Nl4$8+5VQqGO6W@%|1Ck`bwhRy ztGRNJcH(b5!9R$T)>9obiH1hk;=W~=)T*Qq1BOS9*k0{>p(b^`sMC8>P=wfsv>>iT zc69aw*!|s4^;Mr=bR#LVde+?Hh$2k6k%I$CdURo`lqtyhv5NW5WnuPZ&1vnUNh~u! zO}TFgPnf>wX3ZA%Z~0L+O2k|37C$G$p!1t13dunZnZOz{pPoc;_{B@CKZx+ojQTJD zts2*?63!c-3bPbz9K%|-3tF0!9_Ru{l|CL#XB~Q*?;h7pE)df*nTzKS-Y_wwaEr>C zYvoB303({CQ}-C~(3xC5#K`-w@KrLUu=MK=p0L3`Wb#kgs0`Qx3khtN83FUTwcT?O!Uh>(#@-)$P zxIXF9Htl&KSTSXH?e-hI4^(c|3a=#oOyL``;URnB8z!5-c(J$Sknlk#ZzaC#G#1<9 zpAN2*t@@nhCmp=Y;>|3n&!%gzRJYTR%p8tj)tCOssAa&D=V}|z9-;QW1LXludep+E zm~T2^P?bkWdQsE$&5})O6&=g&N3?$@*81tNDkm_}pnvDz&{uJOcm9yxyk`jXROZDe z@3*M7Gi=a$H2RrB-li0)a63~WP73&(K&~AQ|H|K>o8E0ra2%~X3~LXZ3~#ck_o!(W zzP?~gxDw)Hoyi=q<{j%=vEky;lm*uGxHUyNA$@-aBTK(W872Pc!1KN`M=%%#726RV_K0TM)3Y)dZ!XWsi?$9v&EO8F?7bw} zamgxKJcWdU2h+C+zwiH7uPnVQ(9RHS$cP-LOv`gP8w~D~tW-cclrdD89BYX`){|>w zMk7b1-e8{YF=^?;X~KS&@7DM@`jT9);c+EPLOyhV2-tM=`8o6CxOJ)h+4yrKu~UXq zj)oU&!dv;=F9~ck$!9#9X+o#10q2G($HFHZRf={sc;;x1EPaL znR^0-o1T$Qn-UrB*nG24w*P`4{lU>U*fG}@;`2W1=$Wtgy01C_)JB|Z&CUBm)R{~E zgR2(UKVJ$9$DjN`%Cg$LL@E_g_=c0N4@2oL%QwsYGrO^ zlPa|n6hWGqZiL4^?DdS)PB>>2PsW_3d3&qnyTeRa#q434mg#GI4`~l8jlYQO7t&Yz zW_ArVe;~c6C}8+=TaN?Bwqm;@lzJ|?tiEn-cp{f|+?0sp&Mlj#yJ3PxO{c15hGodd z7w^ejHzgd?2j&Ib=g#1iOLb?iCGoxn7IaXa*YQ>f6S9FCR1&DhT66!A|6Q|lW}`ls zJVvm&{87xtU#{50Qt@i#)N<8t-TX(&teg#`(SkPbR6*BG48?olMs2@-ad zgDKQ=mwcM;MzeKcYa;UIz3MGXsYQkv4N;@`7P?j*hAhr7)}p(29Y+-Rc078)gs=XU zdgz@iqMx(wK3v?G*~cwi0Cgi;@U+{&1m_2K3`GnOi4Q~+mrX4leK_9hcVSZ7GhR1V z>W?7cK7X(LDVQ)8nM%msQYMHiOqkzwtXO8%dv*5c$nj@-G1{Ulwv)qYot0coJwIs6 zkBp5TNM9p5R@m4vWs#(3e3g#~Xne$ND9jEAO3d`d1idtHHVYbrE6)U;GIw+4jnN44C@8SHoimn^9+{6Gk zpVZ#xe^-8-n2k;vkrFxf#-F}Dou0ml7VPET7bqDbW0*S%`#?%JyBFYULWXk3tWCBg@(-g^W%e0XFXN! z>fCmDJ4CW!KD6E%VpTOHKvGp8oAq+g?!fV2^g3B{v9K}$q0ayg<0FR|i7j9L ziA0@>#mn&qrHpfL$;|ZYUpJQYNB5(Yl=ge^6ghLy$Mhmpd{U^z#eogs#8@aelA|VC z2As+wC7h&x6(1B2>e}44$4S~~hoF~`dms46)D&~GT62l5i^q3~{W)3!5NoY*QiMw; zD>@sHk{FcI@Aghj8TvYIN0JOD!+w1s&IM0P-$s?s)}PTiGE7|BfAT1{Ed<& zjoGwXjA3AkjvjtPFX*P_CjXQ&`kU1j!4MiK>j|O9T`n%;?z!<0U9#sR9ab3}>RkjPKR2gx|^L}vdC z|MtraCOADmuYM&{KN5fWOgRbm_myodN_bOYke(f3>QOw%R-Z`J?-|<~sF~{RRijq7 z>(V}H8Q|^Gg?e`jVmTLT;n^(a=FCKeeDaiIQ}z_mmOH0J12TPiWLr4r@2{VK3rCgf z3vw>c;I>(=?Fd|abrNRCMb*@kZBZl$~ae-{+ z1l;D&#rjcG@Co>YcP~ePeQ@7FyK>c_aOLDQa~-5y?Y}=z#X0cHQiFq~scD8YkQ!e^IRf{yi1~Y7v*7& z7>LiAzLcmHY)g?bq<;S$7?TmwYm4uS5K0Z5qvrS#Q}_+g?tf2scxy8^R(WpAXRrJ2 z&p-dt(5=+(nkr{oQFVF>J!>st-wi-3iS?+AWA9&M<}S4h$;5?IxTG^ols`hi=0N!o z34^Xv*R}8}AVHc?xwjA&YRZ4W^gb!MrWH{qz+f>LQA)|4=gvF;OI&Wd(5Z|XL^`1t zS*TT#VAQS-zTEkDSo@6Cxcrqw8NlD0LtNNO{KLlJ6805eK3?6<$NyCzg~Pf2)$eHW zeaJW?F7c}5aOFI^|EWUlhfRrNgSx%ZC`f*fer}|-GFis&<#u0D&)y@n-Qkev5sBf{7NK|81Smm-F9*Y#y{Y%RU2X}KQ!si!@88Hkm-du*2HZxm#ho8 zd)98#&q0pnjX31HdM?@t>;3@gaQ^q&cNr~ojbPu^)yge55a)041n-eg%X+FFxU6UkGLO%)WkUz1yrPBk~=X4 z4yj+{QhVFX;Wy(G=Ub!}e>T^gi)wI8uCyeY_}P^p&oveVPfwI2ypFrUoe!KE*-hOv;+JHUrVbAL#HjGRH7zqIuNEw~`h zTv{yJ{4ftGnNetg#Yi59xDUO?WtY#bjMvQOjkeeGzv71g{oM7+JQ+q_ermT=zks^5 zaKh>2p$~w%$K+P0!n%AVwQ89|Azn@rK~9**wMijfi84YXivce5d9bQbgHnGP_XcT~ z6sz)}onb@EF-JQ*mt6|alTm+&hpo@p!iQiUzy|Z3YqMiRMz)+e)Y5Ey?mS{_{vNPu z`a=4xq%Jngjo)vru`Nn| zkXYMJZ1^jokhM0QyLnE~ID2ijerPN?hrR@S`}km{w0}X3nG!=x!QToav%E3d$4>7u z$`xo!Ds)+re#oV7qvoaI{hJOQ$%C6<<<6aBM-__bILC)%wQydF6;V=|j5^Ua1f0sj zSD7(=jFVeZ()clzyP>C?d*vF(O~)`%P2Kw6Uy<31-gxYdT^+)1y${`M*7c+Sl#iQr z`%2?ZibOS@yR5p&mfdLEW7~y30aD)I^eIYdR+w*wI^po6^h8@KS6QT`^Yr$tP-dQQ}nWtKrVAhP`WW#Ej3akdelw>vUE9#S+@ zA4iJHY@}152@gZz%=tqU1m;P`gAd^9{GOX^16j1%Kh<1Yd-gUH_9yy&iR~AtU8wKR zxQ8hRIRir*jjOnsJx|DvR~MIB_Hy9~U;Cc@B=k7;J1WUzx`xk}D~|W}WA3L7Gt+hR zXUb$!-j!-$NW!0^&!J2EO8}XXWVie5lJxTL7@%>IfSG&PsP>-kO$%D}`nun&?UA|s zlDwJjuwI!Vc^@(6R6qe_OYF#ovsO}7GJTu2Q5HNKAY~8}t>&Vg@9%^oXSoa}u5J6t zPMSMEJ|3)zDdIUt!=k0Jg8O5AErnxPDVXcbK)$_3Uord7N!M)`v%*7;&V6+8+x4tD zWGt7VKLZx(z8GX}1uIlt&=cw(eck^t_dg}Mt^hM zMa}s6&P%FX5sPJ2*^TI8Pn{>PnsP#wyTDvcbEFXxm~8`)UH(vjsP60baP#^?T+8Z5 z3>QUK={I)DsvWuh?e;qJ#`K}^V-XiP z)?x8JDa706+*5DWi?&!yQkER<=I6!UZpa}V%@?D)Nn89PfI0nr@{QbRB@^yDd$ujn zS3emi2wI{x1JGPNE-Pt64J!S$cS^V$!_eNl$G-ePIQE_HI#Fg><%p@pdendFAg}I;75iSJ!E+$92ESGV7Dq z!P>;#^5hkk92XnyoZ;}{Z4Md-Fn@xNi5<~pYle%fMfs&Cxs_7T-8oeZ>6#`d%-eln>QZWstdfeJ`mUNe@S!C*z?ngDpY}fH=^e5 zCsrECt1XW{^f9+Ml`9IteR9qUBslFzvC!y74C31=UR~R3GzsW$N8P%oZ}&&u*P;Y| zj)c#*b(`<&+E$-1BLUK7G!wT)YkbhC29^4{cj!klNd5p5V|Dfj>f!aaLtbAz83-`E zLWVhb7M(Q!n zK<-=|M}%s1Iz%N($y8B|g#Aj%5IQ14S`=j$<=b@pl<{`sj6t+i)DMl$eQS&A*KUUz z1W)5eURFQLTeL7eJibZ*c<-2xKVkXkTH`y3m{VN2aQYYM&z>a#A%9iec-_a2>fk9B zIg#7s?0^|nZs6`f;r_P!>)zOLCLk^tnHzaK9#CI`XJq~3a9%;F9pj+SR~3;UZ*4IN zBp>rp+8qqdk7*hgi?(6&)<~?aWoV8pskE-(8L`Og&c&r)X6@4TW zn;dFhOz`<$5v?n2QZ493(}jLx!d>> zmwvn=mv~t%AK#{>=#KotQZ>I7W8YB0bAGd5bQ96T#Kp5&LmA)77vdQkBl~vYmg@(2 zcTxT({ZygxS)hg?;M`l6>S#w#4KeuU-=u}BPws6zmEloF0%~eK>t86NA7uU*EF>sF zut-O_u|j;@89g04lEB80XGJ*3RE({g-YM4V`@XdU~`ugz4QIJZB-4CHC9i743pL#R7t zl6@`v_5`!T^kh`&`75%qzd{ffD)rJDOlIGJyXd=m5r z1$AU2u~$$z8#10UoN@#YO@bHreeJ5%QOA_bD z&Oi^$Ww*Iv^!?c~&!`MW!2(#1cs5kilHUl7D8s^AK%2&bB9cq%FepBZ4S017~! zmm>QA+RiL?w(ms1U6iAdB4CQy3CV{;p3y%(yzzwm4Nbqfty zFx%AXXBB0oaxPThGU)-jM6SDoY-Z=>MQ?DZvLgD~iXl!q zUmIXG*L&XGD*FQBW&HIS4WJKCP3+Bz6v<32h)E$0^PRb;m_M5`5FUEOfptyhwYgm1 zWIbiQ7vM9MVN~Dnv$$5mRpPE&t%|^vpGq8VzWdDz5XGaf3IFij+Q~C$$BG^w#drjo zHBDszi)jEXTy*-{+5D^u2H$Wm0uV;}DM`ryTfgq8b1-K&VEM7{)fhE)w9t3HjjgX0 zR2r?AXa+OSrW!KW&7k7IyABNTZr_NA8`tBcdNX!-JAd?=X~z1^2M`32(5C;uI`1vM zWUTz0e*pT5M_+Nvu@PXNEVXLvag6|np6UJp(T=cJ$m1~O_?PPva#`E=;X|(EO zJ~z;EL%1H;i8W5g-~EWZ&Wg_3T7TbvGsMv$qKSxeAPTa{MVLdSLRcRi5_h@(0rC!! zj4SrMY8JO*ayj(Anr@KK+~YiVCHEAN*K3wdWYO~q00w6m4?VA0t`{I#aIfFEjb~!0 zz2b}bh&v%_(dTJJRlDZ8NQ}+;UmdVQKd01%u@ea_J-nY&+Yk{`cZ&*0B-D;jrbc4A z77sHobkX+r{}I3#n+hi1glKSoI$8ghKo1M}17uu9Jl$eU#VFMMdTI!ho3E*RSvAY^;d?b{viwRZ7{(g<3T(EN^m7V^p;& zlq)UH`*f;ZV$p4_PHh$W^i%puQ(8yEZF&+1Ds8#2dtpLIrkUI54OUqlR2f6UuJQiIhHk52JDuQLp7z~zbkt- z@3zk1M?6fM(0lkJ+&5`8c6KdzFu|dg*_t#v*&BY5qfwTy>)WKt#Z> zLtMXPd}cJsSIjh~+6n%6m?;IOg8T%_Zo6v|ALmvdTpHx$kpXe8$iNDps}s_S$Ic^= zrI}8zaAL7|{;*|Hp3t^+JhNC+UtA6#&*dXkF?>!ln12rJ5MyFGOu0Ma{avvFI_06? z%VA3jtD~TBWD>e!OpiqcTe;%Wm7^fy*#+-u2^^^jR*I?BjhvszI(k|bAx#vr+Ft$( z?*-Py(e;q&HN%q*>4jEDv+*NNmyhTd&Qp1`!>D+WQ4g(SmXJjuhfl%#+~bbBoxO@B zyrrNj+rnyA@(bV3-Cl+*#yye??&73RKAI%y)t5mg*1Bb4VTj`B&m8I4G_;IqH^YPR zM@!2cSKP6bRPkPdz~B^%^G=VB`$?B6nW?wlYzdn9FFt_uk?cj83_=ll+qfIokc$#r zGrX5&93TeXNTY`#3Bz@&RU6A9vTiIf!^z8p0?r8pInNLIqSDk2EeM#|T3xxv)7q4* zAR^i*&OM~NrGV< zZx)q(@uinB-{khwc-QSvHF$&5Q)FE@3bpQ!8;|`d?g;t3L&N|$Mf>{TE>b=6j&u-@ zn8%Pga`Xk2`g^fK#h6$EVg#x0)ki+kyJ~}p=@}!_&AwAM!`}$qGmA=F*G7$Z{|R}B z+j@$w%sjT1btEyUuNw!^)YW^wT1D@{l~wPFCA!tf?Ng3z8?Y8_wNk^;vxw{`wr3~{ z(t%o#D^x&I3|+(yJg%X_SYzPa^TzOScyX3WW|B!q9^m%=idP> z_&PFf=sPXn)r;tlTzmWNE=%wDEram$$S?*XOX?j7%G?C)b~kxy4o~A4l*y7kz0NvG zWk_9~<6c1*=P?JW>EKP9TMvrafN?x3OXHUSU+Ety4mcT%qrRJ7;xEDD;G|!}-&T55 z8_t4W1A-%&AtP00N-{A857IU-Doj!VEVA3r<>o|wXZ=}eik}~5w2g>HCM_gLGH>(Lm^^#me=i3`fdI{n$QAro!gp|@QM*1;MQaqVn3I`bx{8|W-j*&^t~XRElQs)!Z0*l(v4L5 zO1v!m{X62V!7W$4jRE*@VQz#uuuW*pGNoh`t1os4_xZm246*n?OZ-^1(1sV+&;#WL zWK?G`1u4RqeO$#e)q8L^BM~>_UEs0H_XDr|W&hsPI0{xRE5?I44?BjH`}Psido5S= zi;4r%gFO^JNhf~KM?N<%2%;rKuUi}yMZ4%Z6!FVa6e_c)|1T#^{MIkchhLR{_=QP2 z@RerAr`lx7xo4%?9e%bujyVlI<@?rR91y=4q(DSKBAB-&QW)ZV$TCOLrK!n?a=)&P zNFYAXcSl3d#FVru~oGlBSk}V?Q(JKa&!7X z1kzn;v*DTW*7M{o52x6t2*i<)wpA;;E|r4I-QRcFAne95`5=N7>WHAfhuEoUoGf@} z<>M2rsKo1uRom*wi)VO%+%be1wB^1$f_Z4ASBOg1mXMeB` zxmU@_pB7f9XVGfUu$g__2w+Ira7seuVJx>0PF(WhC$w@*AO?XEbycrgcl>no{exy> zn}CPdcf@a|rDfHhCp-d(d7KM*D4h8uAb#M-mt>!=d@T2eW@b7{EaTfYzS|G|p%MtY zK81OCgMl8*>$q8^3_70_XbjiMwmpJ?-|tIl1xU%W$MR@pW^$CSHELnKXA$L3GP}n- z6csofuBjl$(o4A36mle^9CXJT)YNc1qym8vCXiV?yA10KgveaWU*uYCyhZ6EG*T`27bqQ9q8DpOaED$?taT#}pkUW1u z96JdF2eLj2{GanyD%^ zNE6cqJ&q;;!8VqT>l_(5t1@z!$nTcV*K5#>>n~|ItQ^DuKvy`PjY1%%not@dr~kuO zwPaaC#-^B8S;lE!Y;6@E=gRaXg@sa?MAXvf}(AA90{?OU$ z{kg0B@V7vk&(rfEj%#9RMU5G*lKlvS!?{*1_MQ86TGX1| z@wjP0QubjS1ZW88ng%GF->-vSgOG+)$G;3MKG5V_5DBZ2;Bga?mnYow;_?5U`h0a* z774wZQKIIXVr{z1-C#Hwfh0o%63X$iwr0Q7enY4FCjJdwMAjf28RgX%$`HaAqhihw z_3P_gjz%{++AKy5&t6SvaI?MTRg#z`LjL8%)@xDz3bs*9a!F&G0?PwRU(jlSlh)=|9b*5mJA3aAh}rufK` zF4>q*&;YzTw)4KG$S7{304l`~-Vaq<<8OLl5SLj4&Kvi^P z-xS{T!Qq9#0oq8H*Vmg`s^@TGSnz*8SyZSD!4p=TQSmxFnr5C<3BM%CbQ!ez^9lo_ z{pI`LEx{WQe>pYtJ?zS_crcj7+PLRe4!=nw!yf2KV+7j&`QPxTbaWxamGuX# zep3xMg`qIGtHF=Z463m?;$Fqengso#6yXPp%C zmb1h<3YR5^dY2Mfi|_;>n>R+M?n+nnJJU-}hmgb*zRAl8P343(QGs~ENNPP?ZW4^_ zHy8|2PH|On4DcyvY)+0(j;U;|3`m#Xx|C&0It8lLbL!#I+Ea!oXPL*m@aUMCRfLWV z$TzS*#0c_!Lpho&H7Fv!K|;E85=W5y_=frHPDyuGrXX(P%KK#l=GAA%-4JVQ)x0lD zNUZxLc!P!G22eKCvfl!(>T7iuqbPbUqK-!#$?*wAUU<~h-`BL5^yOW(RJ1g^Xsa&n zNzC0|7abU=E+{V4c@dEElR)|7=)cW9T$VSTrWpD_XE2SYLkJK9_X!LLo^g`Acv9L- zR*b_GbIQtk>*|60&&>_EQ8QWoc_u_h*( zce+t0DAY|86D2mbsTr`s;UO~o75nq}ZA~qh;0N7~4L4DXy@_}?x8WSc zA6X)iI_ZqZqtVH+1N_@=(>+RzuGWPqDQ<0ofs?5+F(+=WRw~VFhK-bOS>#8|;yF!g zJ!~A$!@nE?27x2&21nQJrz0%de7tiN7S=*Pe-kwz=Hc@iUwY**CLYZQ#lDcY(^2MFv!#i)ka2U1CeWOII4g{1yf1 zWVd>I-a~(y72`jC0)zV>ZVWQ|tlCUU$Cn=>z}J2s6c#D9`VWWe7`VbHI5Bc~)8z#< z6Jn}A7OO4cGevZ@j)E5W6QxJv@pJ0yN|YHA1eMIsF9H_kpYaQ~S>Bs^;BL4H%6-3D zX{&m;n|uDrgMEGi>hEv&rLE1krKScBN_H-N{-{HvMOgqIBgNtT_q^dxQ>H|$r<=J- zD=5zO8%(hK#FP7)A_Po?9vK~^=?*RtX&nPeE%7=Ea8&pj>rm0nG_q&t~H}_r{2~>wTMVd!lOi+g}us;4>r9BidLmD z>o;K1!CRlsalSXXfWNj!o3#m6LQnWe7?}+GkEZ=q%aq%dJ0&-)Ay44H`hqXHe>2!E_pVmwwx zkJr~8(7F^XwcMull_#mi=213fSic;9HS^Bl#d|u~ zT?*4VIt4QJz+mR!q<|_o;lp$^C#*(nm3wh3>^EpGc7xdb)tMCd=o2`k-n^MVgHLd{Fy1;#G{+^ z?h`rBrO+Kadn*vW1`p)*oq14HL-!5bJw1ncdASiWpG}IO5aR@!AmY8r&KCQXZmn%W zhWAF6eiOI6FV<M6VZYqup(bL;H=&Siz+!n{sX>UU=z#SO9e5HUO{?av31Rz2vRM5bjYqv!-t z`{jglzAt{R8MhTaXRvs`^{(~iH=|Pk0%_B#(cJ(B3cAP zK`@)N7n|=F4@*BkkXAgnS9Cm%t-F{B2;UCri+bLR;@xgB>8O*SH|?h+z0!b)?HIdS zYZ};OY5_StsOHnOq_kQ2=5JDm)wymgVzCU~cC9j=t>M~qObvx?%RS7zk3#As~y@tbI?9qwDAe;d?Dk3i(zoX0SA~LC4$o-&g z<|i6)T+tW|(p1UC@t^yG;bAWbov(O215!XnZHtrNrA-4a=6Ic$?8+^zlZ1=BnM)vv ze<0yoq?5KWWp_s_>x@_92jrzJZwj@3uFa{NA)Km@hv9r7WulW$0wepiar*LsC`A|E zLh_wdo|VA(3>sb_cw!KO3Cb=DhxeDHG$X@6WRSG3m&d}b4@XcKc6iy3pI?z2V~uob zwhEuA_+y1ai9&cEpBv!64#Un4iXQ?my_nLHv)^N`Zp|$t(MDXRNHnh!Ps!3@jZ;fa zDYOf%!ixKru+dN6a{igF%~&Z)UU2l=2wfw5)W$OXb%CGlcIK<4;~wRM5AG8j9S>K{ zfS2AeN2Ae7rJdPW7sx+H$jI#fh6yE7rmC8n{jB)6LP9$aeyoV&32!8;r`_OZ-go6H zMcAX^*xKlXBs#**G>~GNWAeAsDjpXdD$3CPP(@y53bjA=9Y4aZAH%omH4+mP8pT*X z$Q9X^f5Bfyx92iQYUb3m*CiuW)!sFn-E=ZC^1dIpg?hp2hJ3U&13 z+Cnm#>#NmT-E2CMP%;u&i`6r4;Dt9$={F=RQ=+o^z4mrYV5OpbjCzb9{XWO_#M9r5 z?wx8aVOAN$j&coVXg|m; zjWc1=Yi)!E7tCH1!|C*ny{X?Nm}2b{yt#|ze-1_-+syuw9UgGcuEcc& zDcsA|HH;k0Nrw4-zJT60V93bH51D>Y4e4AS>g!1l_2U%5Frm_L%#UR>sAg1n=P!=4 z=@p;?abA+)=F!aeU86tlt?x4xENAI%sxB6X2Aq9viPdTTWOJTUZH|-jW{#JbVnHYY zk^H$N;XSXP+I*j?{iY|3eqh>SL7rg6n}QREdWikgS1^>aN6gla?8f=$Q`5w6-=2=R z@Q$P&Lb(4ETzI-ZQ6OqSKVP>voLyd;`t576Zs;I6IUDvL*n_6I39zF+nBb5Xq;Ly_ zswWbIUm&J6V)!^eZfsW=v|JsoK9o|^adL000B_Gp_tW*t6QaHRaqF{UV_s&onx_31 zgkUXmwIOcY@60KMm&Fs}LYf{&LRI?ubC+ojese+X+-}MLOEk8Xe&1?*dm4SaJo|%> zRv+lyO)M>6a*jF(wZHMOo?jMPd#0W3^OT(NxoBumz;vc*nxe2Aw?3!${07-pq9M3i zU`N-|m)%B|n-w8^dfslzDeeZpB!7oHEKpV~#bPx*jq$t+@8`;wQFJc|D21C1g;HQQyAYdYV;seVR&e!TJX8NW?`UOtRl{EJHRbh z2MuK8bduiS__nyqZKaf-#uZ6lNZag3)C(!ha+Xrg!50PL#FdK~61h`>4sf=p8!tQB zHVkjp3KE$%Hq_0Jf}e?f`9ilze?D{**?lQ54o?v=iA3P%%+-Z+F8dOX2L3~UVB&`l zt(5#sH(52l<)?6XjME9^n)xId*o;1$cWb}meqBJnMiux85(FXLIE&i8^M9cYeJhqL z3voAUmh^NIchp|{ZUJG3jXI%4k2S6fArbONEz2)8l?32OH=1jra%flEe z>oikHnQU03EDGWV6A}ck1h(R;794Zm7EHtfUOmldct&odF}=^TyStVgo|+b{ z5Af^VF>M{2F0uwwTF`n3N&fFE)my!OLcg>sGc@&l=xAHKyg8=gLN@+81ye3Hiv^Rq zYFCgfe4&CQH^K-mUvONrQ{Kh$;@Tw3>hJtjd{rL=5vk&8&+EYJr#T7{5s^E{eoK<{ zKv(1#e(9_f^`^Ll0g*Ker(4mGeZonel0p&vM@)rBOXUmRWzKRImTst)nh8D5(R@`@GN=7Qb8L8Q%dLj1kgK$G#pVCUtuFc%>dP5RrDZ|iLGqL3=;JXX!6@z9?wkgxGJ|b$=93Y^C)-Kk;O+Q|% z;tikBA`zZ6Z26%gN};50a!4(KT%IyL?O4l8n(AA_Y{7KlY=pQaHugcJv0@N$UB+-X zSItgs5qSgMB^T-%5ftjE<U1k}&?yix==E(1vf}iuVLM;pXRXnMrh_S)|MSF_-426f$WEe4`^r zMk99i@THo1iQS%buv$0O;s#vM4S9v49Y)~PUZR%UL!=RtyuHY*dqp0lCAEW01hA7**yh6uz% zvN$&aZfH|Of>q|)k&a%DhhZhAV`D|W2@^lBrNdBRlwWxX=2(p&2FP(+r*E3HVoINR3gOlT@nu7LE++z>U=IP&+2H|9FM)q zNgy@2dDJe%r(Xd}1n-MIik>E7K3VaR$xgTwAGjKABS=L13K<3O46GuSqw;~R&Mn1z z0;=I&`ikP##ROgwi@9|1p*b#2X9P0Z(@>T1?5B>u!JynhEyRzz(Uo@F>k)SVUkI0&6 z^A9!0cnXB;_L$BIDzSj=(V~&RgOqjhe)e{jMsze?-gW2*di-iw4jFe56|RY>a=*Hn zXcKL}{QP*wk!(`VFeZmlIo=JVafh;p>a9jE(%+$q@BQZmn1J!m>48Wh`y|R9vydn|q;4 z7GVL15gCQagnY@VQdK6s%1{)7CL9>4V=QlVYiOZk#@qkb)>(!{*>!uI?(XiAkWxUp zOHxEaxyaKkWeHC7#Lv?m|;-qI=tt4ea`dYe7nEx>$=z8 zYyH=M|L%Kf+mUVX@=SdpEatYFvU1!opu^Jk9H`i4eCyfT1O>KMXV%m%@>9=!XCPG7 zES+CFZ~Klz^)33mFjy8bS1Xq0BM5)c*^pF;fJo#KN7x{=(h>dQ@%GOO`(H4W8;q}7 z?vN423)M0jO2Y^qtu$Go?3^HP2wJ5;FMY# z5{5lCrFX&8;ZQJ1sHxSWnfz#(WaVRdsYey$=!y!XVV9g94^AN-JkbT7>|V@KGM>+g z6LMe%i6nQef!Jzx=`2!6_B;t-5MfbD_uMP665A>&?Gz@iSw5wKP=*EF?`b9HD`II^ zZ}xs0XzRDMGn9w8S8iNpygO=|7Aq0FF`-77zE7WiE72ZB<)>XW{X$Q>v@q#X}2<&;SBqzQ>Y1~ew&+Y_vQ$sCSd ztZBH2b5g~_Xe$ z7G4}ulM871_EZ(fbYOMNK)GV@p8STF9o1~^z0g)EAU}&X$Sx0WX-lFbEySg}Y|*5k znBmv6@gUjif#16@Yzt?}ILs&8D61N^+<4Hx@5^?=CEu$ zt!q15?sY8WIv%7Rbd(%_;~7avndGHbJ7%;*oaf!9qS`LirI1ywK9~6d28F&F1UkGC zOTRZICNE$!wte3>i>SfmRW&@$b0_~Y>^Xw10M@A?54=T+6ZgbID2Ga?%Dmbc)qI8RnW&Wv*i#Krpu>}S zzB5$AnsNWkj`o$_Nxvm{fUpA}E-%Sm4b8VX|9=fS;V$Q8{^0&u00L zsd0}&Q4BefyrhZV-!EePjRX85yo_&l{Y0B6b|=?gv#-4Sb$Cp1qrXZ#1JUe*Z1W1j zR)Q_B1%X{p2c~p7Ou3pC=7qKpsD*l2NRA7QE3_g>>`n={e6ebX18W2@qGPpN!FRK@;<<=gU_bP%oArWK^q#jgckUQD%A? zTCUv7Q$zBl?+73_k6$YIV5VuE+75v86S}Ng_qx2vwthJJ*}=Y1Oxt3k5&{SCE;c5 z5Ed;p7VIDZ%!`&04Qeq=lKyK4y7%_7jjJM8yC>ZElM}%26|-v>vUYOQsK0pf)wSMN zWd!V+DlDXN25cL~F!dY6ZfwtWch>FaCcFqgtzp{u(0?|K`CLVf8WR`;KSx zdNKDz8|4$VQ#^@k}Q``vk0&M40z zV;>-u0-)Pk1bL9}t-FCkk|KifL}`X76e&Grax3&z8A|YpK7+L|*dA8fLy4XJQsy^9 zUC$^rB05Z_9?yan(P-E7n$3fUGOxC-{&IF@)3+~?6bwik{(u?QGXfbn-#Q5TwQ)h{ zsFlvuxMU&iOiv|oenMKfM@=Q$zJN08WF#CB=wiU}I8r~|@o%~5@@k4++KVTUH%YDh zgtf`v&+YPb`L^;lA@bF282P>6l$#vYC;LZCQ5C~KNCpX!!e#OdtdG5A z%^2!fCHzK~>so^k-YmE{*ivSDiAwz*fPSBWs;fw#qFE%09!~CF(EB6WxrF#|Fhn>n z?4l|BD>a!ji-r6z!JE?R6nli8{Bpn6>MLW)z`}^57oE(Ov44m^!RY8Gp{%ig^z;~{ zrJu4qS9HC>B`z56@h{Vi+B_{s8}W5#w`mR*)y+?>3=$bf-}fmk7y9BUR-Ww7h7kNX zBT?Ay-jD-S-W&)Rtxlt{r?F>EqPFXLrT=&s1SEDn_%nU6)B-e4x@%*X3m3(RQak`N zic)drR8s-I>DJfJe!YKCW#^2vw~eTzzinvfhKYMZ?}#DY&SCyah936YT(J+uRSeAf>NDX*^8)j|j2o z^Lap$8=?4TXUhU&AL{$IL0$zqeizuBpQY=us`12qZm5EuA8hCm(j#(OV2>{{9 zj~!5Pt}sAPW?d&=sL@FM$lNkRqPfkv@2g)T^*`GYn{{%}H!epwFeo%HGJ>Llk1F2J zfN{x!edCtRbD=teuAqA(k3OD_YjN#?%)EzT!$uEA;ZI^r^1AdBi58z^zziA2UYeKZ z4jH(kW>?9cRG>zCts?+WriV2DoYmYSf>^6e-n>k}kJ}GFd%FSYBBfZjNhYEB8{iIn zqfkn5ahwv@1>G2~Q}{30aD~J_6 zB!Ew3p-Wj4<~$yN%&M3U_=!dSoV5cy|9D?FpXY1&)Hel~*>@Sl_klk%S?!fbfme&_ zk^MB`hKFC2BWN~mGyPp&ojqVW(BpU@+}>v_+g2I#y(|WMW(;+nUsy<--s-8*!;ndL z_PFn;?(UHvrJVm_o{JxxoX%suR=?dO4qKea-ZCQ&H%k7@KQmvuuauM_q}87jq#oWsHjQFmtPJl zaNwCpF1wan2I+)*>J2PUwDtHtW$eIH{-8td9yLvlz0>EX1%ze7E=qGL7W7>%N{sAq z@Z_HIVv0{8LmZ=opb3-2FaQC^Xa5>9pAc{L`Fw*&lzvDPjYqYQs{r>-CwNE+RHF7& z*Q*AwNXkx$QjMQXv%EGK^N5~!!a!(1Z^#M}U$aV=Ay+iHb0 zllZmJ71LP-xFIk0VdU7l9Td1$T2s3vX<&Kh>vHt##}DUvIR$}7+`J5U;g6QKDrgvz@#dak(IntU=WjB$FO_pq3 z!$*kQp~dHGv`KLh+pYhkxAU3C6XvIUd4sP@X;sO?e+z5WtQ~U4@|Uo!l5zy5`aLhb(fFP(%t78WGywvlq>c6S^eG#MW05I$w&0(`;#A~*3=8P zq~(0@6mH%!6nxPa1-!~0$rx@au#GiAP1`T>p} z@G7RRi#YwG{R)qf;VWf&Y|Qs03oC-GieE$-a1M(P72NtATs-^&zJjH!TBs|Z2Mr=y zLK9{eXs@`RKHFi^_5mb%Wyubb{fCfz8jdA*_q~C=B~>j|SZ@Ecz1TBdjI%EWJ8X!Zmg zTDK%W$dThbLMp^Z!31A8$DB4cj>grzxqd@X<5Q*h`t9Z@ip#Jy(a3X^mi(K9>d}n= z_Vr{nkZ$qr%KYIvHQO@ta~b{v&`qi%Y~;BhA{;yS1$T3(r3Y!^K1>Y;R}jIfTjgn& zE_bV*9B3&S?iUP9a&h?x|F?lyr%vW>uRk`MFi){u=fUy)KDK*bueH3+dL;*L(id|y z67rC@lZJ=!K9zu)*|y~wYt^Hy=LRSG$qj9bd4WDH5ZC>os#8jNT}R#8v4BE8UFc;SOs1X$UNp< zd=i#xy5O`6QV*{Ajd3@4=72>@F=W&hDBNg&a3DNu+i-Hb6ifze?zfMH!}&Dgt2WLG8{2OS3yiGw zV=vJkUwGAOjdFQ2E9&4TQMs$7z78su5x0FB~aBgJPwMWdLpo+|XNJoAi4=1qBgMk$?`6ZS>CUG1^r= zLlCO+lT0U0;N#!2@!qN(Xu9{|!HbCdCg9<^v4s- z-CuI~(_ngU*h3wD{BHNum~xEjMkDu^k>s2qeYQptKI+B}Gsp((eSs$K*Yc${fGS(y zAak|pl;miXbFhPDY-RLp%5K z4|Ds95Pz@u*CpP?G-wYugBWxbw13QYO)3@h^6??kcV8r6)Yx~uzJ@Gy(Vrj$n~B6< z{~n7|l0fXbge3pQWWFq{JPe4eBi3}dy7!yvz_hqFIxf}+4lod{dLZv#GIJ!9b`W-j z*M#!w4rrGe3Jy@BuHwO+ta`VMg0I@%X4{NJ!5Cl}dZOgiD%BShnBHK;#99iNRdR!2~ z4WX1>P}}!v$(>Fm4iXu(x!GNZqGfQ6&uTP$>(uv{w-q_+m$XA?lD8 zPT?u0Q$(P7kvJrqCd>CP-t;MSZ*q6H<1?`I=hl|$DzYe8W!v;+Vy#eqkI>6~Q-ukA z-a#jc{^=B^Km~ue%%aDpO{D`@rF7GxCTVd}mD*zuCL2k9&i(5f9zGCd5{cgKu??o= zADoq1AB{v+K3eK0-g1Xu%LcLjE?0)}7Rv_DHtBF}|I5IPs`(J=rFo_zUYC$2^tq;5 z!0T5=YyFp+nkEUnpSB6lEMj}(S99~fGn3iXBquGWWa>LMHVUOZduDTZybo2|6eXCe z-S-4aHWrX<^_AH#c2U)8utwTMe{v+wC*}!+X_+x^oqKUB1)|qSt38c>V6iU|K%d&6Zz{6{fy{Fi~)Pxxj~t(h!G`a*UhNQ=W8~c z$0SBv7`76HD3+RsvFjv(f|;;(vSCX@msUJ^xC*j({??9n0oVTQhM{!%j*yWx#&v1XB`R9xJNdeBqnTUbetY?qm9j?@Y~ z(+y?lkszBzs)cG$EJ-JmSGT@t>%I%)t`_kAsU$l z+x>h=E~lpRFm-xjLT;!&jmN@L?7G^^~Z1WYGBFJ_h5vIvk658*+LyH+8tM z(6GZktkuA)!{92gm3sHDs>pu)E) and [NumPy](https://numpy.org/) for efficient interoperability with your ML/AI stack +* **Data Catalogs/Table Formats:** Capabilities to effectively query table formats such as [Apache Iceberg](https://iceberg.apache.org/), [Delta Lake](https://delta.io/) and [Apache Hudi](https://hudi.apache.org/) +* **Seamless Data Interchange:** Zero-copy integration with [Apache Arrow](https://arrow.apache.org/docs/index.html) +* **Multimodal/ML Data:** Native functionality for data modalities such as tensors, images, URLs, long-form text and embeddings + +## Learning Daft + +This user guide aims to help Daft users master the usage of Daft for all your data needs. + +!!! tip "Looking to get started with Daft ASAP?" + + The Daft User Guide is a useful resource to take deeper dives into specific Daft concepts, but if you are ready to jump into code you may wish to take a look at these resources: + + 1. [10 minute Quickstart](https://www.getdaft.io/projects/docs/en/stable/10-min.html): Itching to run some Daft code? Hit the ground running with our 10 minute quickstart notebook. + + 2. [API Documentation](https://www.getdaft.io/projects/docs/en/stable/api_docs/index.html): Searchable documentation and reference material to Daftโ€™s public API. + +### Get Started + +

+ +### Daft in Depth + +
+ +- [:material-filter: **DataFrame Operations**](core_concepts.md#dataframe) + + Learn how to perform core DataFrame operations in Daft, including selection, filtering, joining, and sorting. + +- [:octicons-code-16: **Expressions**](core_concepts.md#expressions) + + Daft expressions enable computations on DataFrame columns using Python or SQL for various operations. + +- [:material-file-eye: **Reading Data**](core_concepts.md#reading-data) + + How to use Daft to read data from diverse sources like files, databases, and URLs. + +- [:material-file-edit: **Writing Data**](core_concepts.md#reading-data) + + How to use Daft to write data DataFrames to files or other destinations. + +- [:fontawesome-solid-square-binary: **DataTypes**](core_concepts.md#datatypes) + + Daft DataTypes define the types of data in a DataFrame, from simple primitives to complex structures. + +- [:simple-quicklook: **SQL**](core_concepts.md#sql) + + Daft supports SQL for constructing query plans and expressions, while integrating with Python expressions. + +- [:material-select-group: **Aggregations and Grouping**](core_concepts.md#aggregations-and-grouping) + + Daft supports aggregations and grouping across entire DataFrames and within grouped subsets of data. + +- [:fontawesome-solid-user: **User-Defined Functions (UDFs)**](core_concepts.md#user-defined-functions-udf) + + Daft allows you to define custom UDFs to process data at scale with flexibility in input and output. + +- [:octicons-image-16: **Multimodal Data**](core_concepts.md#multimodal-data) + + Daft is built to work with multimodal data types, including URLs and images. + +
+ +### More Resources + +
+ +- [:material-star-shooting: **Advanced Daft**](advanced/memory.md) +- [:material-file-compare: **DataFrame Comparison**](resources/dataframe_comparison.md) +- [:material-file-document: **Tutorials**](resources/tutorials.md) +- [:material-chart-bar: **Benchmarks**](resources/benchmarks/tpch.md) + +
+ +## Contribute to Daft + +If you're interested in hands-on learning about Daft internals and would like to contribute to our project, join us [on Github](https://github.com/Eventual-Inc/Daft) ๐Ÿš€ + +Take a look at the many issues tagged with `good first issue` in our repo. If there are any that interest you, feel free to chime in on the issue itself or join us in our [Distributed Data Slack Community](https://join.slack.com/t/dist-data/shared_invite/zt-2e77olvxw-uyZcPPV1SRchhi8ah6ZCtg) and send us a message in #daft-dev. Daft team members will be happy to assign any issue to you and provide any guidance if needed! + +## Frequently Asked Questions + +!!! failure "todo(docs): Add answers to each and more questions if necessary" + +??? quote "What does Daft do well? (or What should I use Daft for?)" + + + + Daft is the right tool for you if you are working with: + + - **Large datasets** that don't fit into memory or would benefit from parallelization + - **Multimodal data types** such as images, JSON, vector embeddings, and tensors + - **Formats that support data skipping** through automatic partition pruning and stats-based file pruning for filter predicates + - **ML workloads** that would benefit from interact computation within a DataFrame (via UDFs) + +??? quote "What should I *not* use Daft for?" + +??? quote "How do I know if Daft is the right framework for me?" + + See [DataFrame Comparison](resources/dataframe_comparison.md) + +??? quote "What is the difference between Daft and Ray?" + +??? quote "What is the difference between Daft and Spark?" + +??? quote "How does Daft perform at large scales vs other data engines?" + + See [Benchmarks](resources/benchmarks/tpch.md) + +??? quote "What is the technical architecture of Daft?" + + See [Technical Architecture](resources/architecture.md) + +??? quote "Does Daft perform any telemetry?" + + See [Telemetry](resources/telemetry.md) diff --git a/docs-v2/install.md b/docs-v2/install.md new file mode 100644 index 0000000000..5f773aa5f7 --- /dev/null +++ b/docs-v2/install.md @@ -0,0 +1,47 @@ +# Installation + +To install Daft, run this from your terminal: + +```bash +pip install -U getdaft +``` + +## Extra Dependencies + +Some Daft functionality may also require other dependencies, which are specified as "extras": + +To install Daft with the extra dependencies required for interacting with AWS services, such as AWS S3, run: + +```bash +pip install -U getdaft[aws] +``` + +To install Daft with the extra dependencies required for running distributed Daft on top of a [Ray cluster](https://docs.ray.io/en/latest/index.html), run: + +```bash +pip install -U getdaft[ray] +``` + +To install Daft with all extras, run: + +```bash +pip install -U getdaft[all] +``` + +## Advanced Installation + +### Installing Nightlies + +If you wish to use Daft at the bleeding edge of development, you may also install the nightly build of Daft which is built every night against the `main` branch: + +```bash +pip install -U getdaft --pre --extra-index-url https://pypi.anaconda.org/daft-nightly/simple +``` + +### Installing Daft from source + +```bash +pip install -U https://github.com/Eventual-Inc/Daft/archive/refs/heads/main.zip +``` + +Please note that Daft requires the Rust toolchain in order to build from source. diff --git a/docs-v2/integrations/aws.md b/docs-v2/integrations/aws.md new file mode 100644 index 0000000000..fa89926970 --- /dev/null +++ b/docs-v2/integrations/aws.md @@ -0,0 +1,53 @@ +# Amazon Web Services + +Daft is able to read/write data to/from AWS S3, and understands natively the URL protocol `s3://` as referring to data that resides +in S3. + +## Authorization/Authentication + +In AWS S3, data is stored under the hierarchy of: + +1. Bucket: The container for data storage, which is the top-level namespace for data storage in S3. +2. Object Key: The unique identifier for a piece of data within a bucket. + +URLs to data in S3 come in the form: `s3://{BUCKET}/{OBJECT_KEY}`. + +### Rely on Environment + +You can configure the AWS [CLI](https://docs.aws.amazon.com/cli/latest/userguide/cli-chap-configure.html) to have Daft automatically discover credentials. Alternatively, you may specify your credentials in [environment variables](https://docs.aws.amazon.com/cli/latest/userguide/cli-configure-envvars.html): `AWS_ACCESS_KEY_ID`, `AWS_SECRET_ACCESS_KEY`, and `AWS_SESSION_TOKEN`. + +Please be aware that when doing so in a distributed environment such as Ray, Daft will pick these credentials up from worker machines and thus each worker machine needs to be appropriately provisioned. + +If instead you wish to have Daft use credentials from the "driver", you may wish to manually specify your credentials. + +### Manually specify credentials + +You may also choose to pass these values into your Daft I/O function calls using an [`daft.io.S3Config`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_configs/daft.io.S3Config.html#daft.io.S3Config) config object. + +!!! failure "todo(docs): add SQL S3Config https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/sql_funcs/daft.sql._sql_funcs.S3Config.html" + +[`daft.set_planning_config`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/configuration_functions/daft.set_planning_config.html#daft.set_planning_config) is a convenient way to set your [`daft.io.IOConfig`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_configs/daft.io.IOConfig.html#daft.io.IOConfig) as the default config to use on any subsequent Daft method calls. + +=== "๐Ÿ Python" + + ```python + from daft.io import IOConfig, S3Config + + # Supply actual values for the se + io_config = IOConfig(s3=S3Config(key_id="key_id", session_token="session_token", secret_key="secret_key")) + + # Globally set the default IOConfig for any subsequent I/O calls + daft.set_planning_config(default_io_config=io_config) + + # Perform some I/O operation + df = daft.read_parquet("s3://my_bucket/my_path/**/*") + ``` + +Alternatively, Daft supports overriding the default IOConfig per-operation by passing it into the `io_config=` keyword argument. This is extremely flexible as you can pass a different [`daft.io.S3Config`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_configs/daft.io.S3Config.html#daft.io.S3Config) per function call if you wish! + +=== "๐Ÿ Python" + + ```python + # Perform some I/O operation but override the IOConfig + df2 = daft.read_csv("s3://my_bucket/my_other_path/**/*", io_config=io_config) + ``` diff --git a/docs-v2/integrations/azure.md b/docs-v2/integrations/azure.md new file mode 100644 index 0000000000..f803b5c421 --- /dev/null +++ b/docs-v2/integrations/azure.md @@ -0,0 +1,80 @@ +# Microsoft Azure + +Daft is able to read/write data to/from Azure Blob Store, and understands natively the URL protocols `az://` and `abfs://` as referring to data that resides in Azure Blob Store. + +!!! warning "Warning" + + Daft currently only supports globbing and listing files in storage accounts with [hierarchical namespaces](https://learn.microsoft.com/en-us/azure/storage/blobs/data-lake-storage-namespace) enabled. Hierarchical namespaces enable Daft to use its embarrassingly parallel globbing algorithm to improve performance of listing large nested directories of data. Please file an [issue](https://github.com/Eventual-Inc/Daft/issues) if you need support for non-hierarchical namespace buckets! We'd love to support your use-case. + +## Authorization/Authentication + +In Azure Blob Service, data is stored under the hierarchy of: + +1. Storage Account +2. Container (sometimes referred to as "bucket" in S3-based services) +3. Object Key + +URLs to data in Azure Blob Store come in the form: `az://{CONTAINER_NAME}/{OBJECT_KEY}`. + +Given that the Storage Account is not a part of the URL, you must provide this separately. + +### Rely on Environment + +You can rely on Azure's [environment variables](https://learn.microsoft.com/en-us/azure/storage/blobs/authorize-data-operations-cli#set-environment-variables-for-authorization-parameters) to have Daft automatically discover credentials. + +Please be aware that when doing so in a distributed environment such as Ray, Daft will pick these credentials up from worker machines and thus each worker machine needs to be appropriately provisioned. + +If instead you wish to have Daft use credentials from the "driver", you may wish to manually specify your credentials. + +### Manually specify credentials + +You may also choose to pass these values into your Daft I/O function calls using an [`daft.io.AzureConfig`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_configs/daft.io.AzureConfig.html#daft.io.AzureConfig) config object. + + + +[`daft.set_planning_config`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/configuration_functions/daft.set_planning_config.html#daft.set_planning_config) is a convenient way to set your [`daft.io.IOConfig`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_configs/daft.io.IOConfig.html#daft.io.IOConfig) as the default config to use on any subsequent Daft method calls. + +=== "๐Ÿ Python" + + ```python + from daft.io import IOConfig, AzureConfig + + # Supply actual values for the storage_account and access key here + io_config = IOConfig(azure=AzureConfig(storage_account="***", access_key="***")) + + # Globally set the default IOConfig for any subsequent I/O calls + daft.set_planning_config(default_io_config=io_config) + + # Perform some I/O operation + df = daft.read_parquet("az://my_container/my_path/**/*") + ``` + +Alternatively, Daft supports overriding the default IOConfig per-operation by passing it into the `io_config=` keyword argument. This is extremely flexible as you can pass a different [`daft.io.AzureConfig`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_configs/daft.io.AzureConfig.html#daft.io.AzureConfig) per function call if you wish! + +=== "๐Ÿ Python" + + ```python + # Perform some I/O operation but override the IOConfig + df2 = daft.read_csv("az://my_container/my_other_path/**/*", io_config=io_config) + ``` + +### Connect to Microsoft Fabric/OneLake + +If you are connecting to storage in OneLake or another Microsoft Fabric service, set the `use_fabric_endpoint` parameter to `True` in the [`daft.io.AzureConfig`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_configs/daft.io.AzureConfig.html#daft.io.AzureConfig) object. + +=== "๐Ÿ Python" + + ```python + from daft.io import IOConfig, AzureConfig + + io_config = IOConfig( + azure=AzureConfig( + storage_account="onelake", + use_fabric_endpoint=True, + + # Set credentials as needed + ) + ) + + df = daft.read_deltalake('abfss://[WORKSPACE]@onelake.dfs.fabric.microsoft.com/[LAKEHOUSE].Lakehouse/Tables/[TABLE]', io_config=io_config) + ``` diff --git a/docs-v2/integrations/delta_lake.md b/docs-v2/integrations/delta_lake.md new file mode 100644 index 0000000000..d0c92bed5c --- /dev/null +++ b/docs-v2/integrations/delta_lake.md @@ -0,0 +1,128 @@ +# Delta Lake + +[Delta Lake](https://delta.io/) is an open-source storage framework for data analytics on data lakes. It provides ACID transactions, scalable metadata handling, and a unification of streaming and batch data processing, all on top of Parquet files in cloud storage. + +Daft currently supports: + +1. **Parallel + Distributed Reads:** Daft parallelizes Delta Lake table reads over all cores of your machine, if using the default multithreading runner, or all cores + machines of your Ray cluster, if using the [distributed Ray runner](../advanced/distributed.md). + +2. **Skipping Filtered Data:** Daft ensures that only data that matches your [`df.where(...)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.where.html#daft.DataFrame.where) filter will be read, often skipping entire files/partitions. + +3. **Multi-cloud Support:** Daft supports reading Delta Lake tables from AWS S3, Azure Blob Store, and GCS, as well as local files. + +## Installing Daft with Delta Lake Support + +Daft internally uses the [deltalake](https://pypi.org/project/deltalake/) Python package to fetch metadata about the Delta Lake table, such as paths to the underlying Parquet files and table statistics. The `deltalake` package therefore must be installed to read Delta Lake tables with Daft, either manually or with the below `getdaft[deltalake]` extras install of Daft. + +```bash +pip install -U "getdaft[deltalake]" +``` + +## Reading a Table + +A Delta Lake table can be read by providing [`daft.read_deltalake`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.read_deltalake.html#daft.read_deltalake) with the URI for your table. + +The below example uses the [deltalake](https://pypi.org/project/deltalake/) Python package to create a local Delta Lake table for Daft to read, but Daft can also read Delta Lake tables from all of the major cloud stores. + +=== "๐Ÿ Python" + + ```python + # Create a local Delta Lake table. + from deltalake import write_deltalake + import pandas as pd + + df = pd.DataFrame({ + "group": [1, 1, 2, 2, 3, 3, 4, 4], + "num": list(range(8)), + "letter": ["a", "b", "c", "d", "e", "f", "g", "h"], + }) + + # This will write out separate partitions for group=1, group=2, group=3, group=4. + write_deltalake("some-table", df, partition_by="group") + ``` + +After writing this local example table, we can easily read it into a Daft DataFrame. + +=== "๐Ÿ Python" + + ```python + # Read Delta Lake table into a Daft DataFrame. + import daft + + df = daft.read_deltalake("some-table") + ``` + +## Data Skipping Optimizations + +Subsequent filters on the partition column `group` will efficiently skip data that doesn't match the predicate. In the below example, the `group != 2` partitions (files) will be pruned, i.e. they will never be read into memory. + +=== "๐Ÿ Python" + + ```python + # Filter on partition columns will result in efficient partition pruning; non-matching partitions will be skipped. + df2 = df.where(df["group"] == 2) + df2.show() + ``` + +Filters on non-partition columns will still benefit from automatic file pruning via file-level statistics. In the below example, the `group=2` partition (file) will have `2 <= df["num"] <= 3` lower/upper bounds for the `num` column, and since the filter predicate is `df["num"] < 2`, Daft will prune the file from the read. Similar is true for `group=3` and `group=4` partitions, with none of the data from those files being read into memory. + +=== "๐Ÿ Python" + + ```python + # Filter on non-partition column, relying on file-level column stats to efficiently prune unnecessary file reads. + df3 = df.where(df["num"] < 2) + df3.show() + ``` + +## Write to Delta Lake + +You can use [`df.write_deltalake`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.write_deltalake.html) to write a Daft DataFrame to a Delta table: + +=== "๐Ÿ Python" + + ```python + df.write_deltalake("tmp/daft-table", mode="overwrite") + ``` + +Daft supports multiple write modes. See the API docs for [`daft.DataFrame.write_deltalake`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.write_deltalake.html) for more details. + +## Type System + +Daft and Delta Lake have compatible type systems. Here are how types are converted across the two systems. + +When reading from a Delta Lake table into Daft: + +| Delta Lake | Daft | +| --------------------- | ----------------------------- | +| **Primitive Types** | +| `boolean` | [`daft.DataType.bool()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.bool) | +| `byte` | [`daft.DataType.int8()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.int8) | +| `short` | [`daft.DataType.int16()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.int16)| +| `int` | [`daft.DataType.int32()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.int32) | +| `long` | [`daft.DataType.int64()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.int64) | +| `float` | [`daft.DataType.float32()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.float32) | +| `double` | [`daft.DataType.float64()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.float64) | +| `decimal(precision, scale)` | [`daft.DataType.decimal128(precision, scale)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.decimal128) | +| `date` | [`daft.DataType.date()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.date) | +| `timestamp` | [`daft.DataType.timestamp(timeunit="us", timezone=None)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.timestamp) | +| `timestampz`| [`daft.DataType.timestamp(timeunit="us", timezone="UTC")`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.timestamp) | +| `string` | [`daft.DataType.string()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.string) | +| `binary` | [`daft.DataType.binary()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.binary) | +| **Nested Types** | +| `struct(fields)` | [`daft.DataType.struct(fields)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.struct) | +| `list(child_type)` | [`daft.DataType.list(child_type)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.list) | +| `map(K, V)` | [`daft.DataType.struct({"key": K, "value": V})`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.struct) | + +## Roadmap + +Here are Delta Lake features that are on our roadmap. Please let us know if you would like to see support for any of these features! + +1. Read support for [deletion vectors](https://docs.delta.io/latest/delta-deletion-vectors.html) ([issue](https://github.com/Eventual-Inc/Daft/issues/1954)). + +2. Read support for [column mappings](https://docs.delta.io/latest/delta-column-mapping.html>) ([issue](https://github.com/Eventual-Inc/Daft/issues/1955)). + +3. Writing new Delta Lake tables ([issue](https://github.com/Eventual-Inc/Daft/issues/1967)). + +!!! failure "todo(docs): ^ this needs to be updated, issue is already closed" + +4. Writing back to an existing table with appends, overwrites, upserts, or deletes ([issue](https://github.com/Eventual-Inc/Daft/issues/1968)). diff --git a/docs-v2/integrations/hudi.md b/docs-v2/integrations/hudi.md new file mode 100644 index 0000000000..e0a28ec7da --- /dev/null +++ b/docs-v2/integrations/hudi.md @@ -0,0 +1,76 @@ +# Apache Hudi + +[Apache Hudi](https://hudi.apache.org/) is an open-sourced transactional data lake platform that brings database and data warehouse capabilities to data lakes. Hudi supports transactions, efficient upserts/deletes, advanced indexes, streaming ingestion services, data clustering/compaction optimizations, and concurrency all while keeping your data in open source file formats. + +Daft currently supports: + +1. **Parallel + Distributed Reads:** Daft parallelizes Hudi table reads over all cores of your machine, if using the default multithreading runner, or all cores + machines of your Ray cluster, if using the [distributed Ray runner](../advanced/distributed.md). + +2. **Skipping Filtered Data:** Daft ensures that only data that matches your [`df.where(...)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.where.html#daft.DataFrame.where) filter will be read, often skipping entire files/partitions. + +3. **Multi-cloud Support:** Daft supports reading Hudi tables from AWS S3, Azure Blob Store, and GCS, as well as local files. + +## Installing Daft with Apache Hudi Support + +Daft supports installing Hudi through optional dependency. + +```bash +pip install -U "getdaft[hudi]" +``` + +## Reading a Table + +To read from an Apache Hudi table, use the [`daft.read_hudi`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.read_hudi.html#daft.read_hudi) function. The following is an example snippet of loading an example table: + +=== "๐Ÿ Python" + + ```python + # Read Apache Hudi table into a Daft DataFrame. + import daft + + df = daft.read_hudi("some-table-uri") + df = df.where(df["foo"] > 5) + df.show() + ``` + +## Type System + +Daft and Hudi have compatible type systems. Here are how types are converted across the two systems. + +When reading from a Hudi table into Daft: + +| Apachi Hudi | Daft | +| --------------------- | ----------------------------- | +| **Primitive Types** | +| `boolean` | [`daft.DataType.bool()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.bool) | +| `byte` | [`daft.DataType.int8()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.int8) | +| `short` | [`daft.DataType.int16()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.int16)| +| `int` | [`daft.DataType.int32()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.int32) | +| `long` | [`daft.DataType.int64()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.int64) | +| `float` | [`daft.DataType.float32()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.float32) | +| `double` | [`daft.DataType.float64()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.float64) | +| `decimal(precision, scale)` | [`daft.DataType.decimal128(precision, scale)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.decimal128) | +| `date` | [`daft.DataType.date()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.date) | +| `timestamp` | [`daft.DataType.timestamp(timeunit="us", timezone=None)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.timestamp) | +| `timestampz`| [`daft.DataType.timestamp(timeunit="us", timezone="UTC")`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.timestamp) | +| `string` | [`daft.DataType.string()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.string) | +| `binary` | [`daft.DataType.binary()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.binary) | +| **Nested Types** | +| `struct(fields)` | [`daft.DataType.struct(fields)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.struct) | +| `list(child_type)` | [`daft.DataType.list(child_type)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.list) | +| `map(K, V)` | [`daft.DataType.struct({"key": K, "value": V})`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.struct) | + +## Roadmap + +Currently there are limitations of reading Hudi tables + +- Only support snapshot read of Copy-on-Write tables +- Only support reading table version 5 & 6 (tables created using release 0.12.x - 0.15.x) +- Table must not have `hoodie.datasource.write.drop.partition.columns=true` + +Support for more Hudi features are tracked as below: + +1. Support incremental query for Copy-on-Write tables [issue](https://github.com/Eventual-Inc/Daft/issues/2153)). +2. Read support for 1.0 table format ([issue](https://github.com/Eventual-Inc/Daft/issues/2152)). +3. Read support (snapshot) for Merge-on-Read tables ([issue](https://github.com/Eventual-Inc/Daft/issues/2154)). +4. Write support ([issue](https://github.com/Eventual-Inc/Daft/issues/2155)). diff --git a/docs-v2/integrations/huggingface.md b/docs-v2/integrations/huggingface.md new file mode 100644 index 0000000000..104f6aaddc --- /dev/null +++ b/docs-v2/integrations/huggingface.md @@ -0,0 +1,69 @@ +# Hugging Face Datasets + +Daft is able to read datasets directly from Hugging Face via the `hf://datasets/` protocol. + +Since Hugging Face will [automatically convert](https://huggingface.co/docs/dataset-viewer/en/parquet) all public datasets to parquet format, we can read these datasets using the [`daft.read_parquet()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.read_parquet.html) method. + +!!! warning "Warning" + + This is limited to either public datasets, or PRO/ENTERPRISE datasets. + +For other file formats, you will need to manually specify the path or glob pattern to the files you want to read, similar to how you would read from a local file system. + + +## Reading Public Datasets + +=== "๐Ÿ Python" + + ```python + import daft + + df = daft.read_parquet("hf://datasets/username/dataset_name") + ``` + +This will read the entire dataset into a daft DataFrame. + +Not only can you read entire datasets, but you can also read individual files from a dataset. + +=== "๐Ÿ Python" + + ```python + import daft + + df = daft.read_parquet("hf://datasets/username/dataset_name/file_name.parquet") + # or a csv file + df = daft.read_csv("hf://datasets/username/dataset_name/file_name.csv") + + # or a glob pattern + df = daft.read_parquet("hf://datasets/username/dataset_name/**/*.parquet") + ``` + +## Authorization + +For authenticated datasets: + +=== "๐Ÿ Python" + + ```python + from daft.io import IOConfig, HTTPConfig + + io_config = IoConfig(http=HTTPConfig(bearer_token="your_token")) + df = daft.read_parquet("hf://datasets/username/dataset_name", io_config=io_config) + ``` + +It's important to note that this will not work with standard tier private datasets. +Hugging Face does not auto convert private datasets to parquet format, so you will need to specify the path to the files you want to read. + +=== "๐Ÿ Python" + + ```python + df = daft.read_parquet("hf://datasets/username/my_private_dataset", io_config=io_config) # Errors + ``` + +to get around this, you can read all files using a glob pattern *(assuming they are in parquet format)* + +=== "๐Ÿ Python" + + ```python + df = daft.read_parquet("hf://datasets/username/my_private_dataset/**/*.parquet", io_config=io_config) # Works + ``` diff --git a/docs-v2/integrations/iceberg.md b/docs-v2/integrations/iceberg.md new file mode 100644 index 0000000000..8469b9f0df --- /dev/null +++ b/docs-v2/integrations/iceberg.md @@ -0,0 +1,110 @@ +# Apache Iceberg + +[Apache Iceberg](https://iceberg.apache.org/) is an open-sourced table format originally developed at Netflix for large-scale analytical datasets. + +Daft currently natively supports: + +1. **Distributed Reads:** Daft will fully distribute the I/O of reads over your compute resources (whether Ray or on multithreading on the local PyRunner) +2. **Skipping Filtered Data:** Daft uses [`df.where(...)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.where.html) filter calls to only read data that matches your predicates +3. **All Catalogs From PyIceberg:** Daft is natively integrated with PyIceberg, and supports all the catalogs that PyIceberg does + +## Reading a Table + +To read from the Apache Iceberg table format, use the [`daft.read_iceberg`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.read_iceberg.html#daft.read_iceberg) function. + +We integrate closely with [PyIceberg](https://py.iceberg.apache.org/) (the official Python implementation for Apache Iceberg) and allow the reading of Daft dataframes easily from PyIceberg's Table objects. The following is an example snippet of loading an example table, but for more information please consult the [PyIceberg Table loading documentation](https://py.iceberg.apache.org/api/#load-a-table). + +=== "๐Ÿ Python" + + ```python + # Access a PyIceberg table as per normal + from pyiceberg.catalog import load_catalog + + catalog = load_catalog("my_iceberg_catalog") + table = catalog.load_table("my_namespace.my_table") + ``` + +After a table is loaded as the `table` object, reading it into a DataFrame is extremely easy. + +=== "๐Ÿ Python" + + ```python + # Create a Daft Dataframe + import daft + + df = daft.read_iceberg(table) + ``` + +Any subsequent filter operations on the Daft `df` DataFrame object will be correctly optimized to take advantage of Iceberg features such as hidden partitioning and file-level statistics for efficient reads. + +=== "๐Ÿ Python" + + ```python + # Filter which takes advantage of partition pruning capabilities of Iceberg + df = df.where(df["partition_key"] < 1000) + df.show() + ``` + +## Writing to a Table + +To write to an Apache Iceberg table, use the [`daft.DataFrame.write_iceberg`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.write_iceberg.html) method. + +The following is an example of appending data to an Iceberg table: + +=== "๐Ÿ Python" + + ```python + written_df = df.write_iceberg(table, mode="append") + written_df.show() + ``` + +This call will then return a DataFrame containing the operations that were performed on the Iceberg table, like so: + +``` {title="Output"} + +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ operation โ”† rows โ”† file_size โ”† file_name โ”‚ +โ”‚ --- โ”† --- โ”† --- โ”† --- โ”‚ +โ”‚ Utf8 โ”† Int64 โ”† Int64 โ”† Utf8 โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ ADD โ”† 5 โ”† 707 โ”† 2f1a2bb1-3e64-49da-accd-1074eโ€ฆ โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ +``` + +## Type System + +Daft and Iceberg have compatible type systems. Here are how types are converted across the two systems. + +When reading from an Iceberg table into Daft: + +| Iceberg | Daft | +| --------------------- | ----------------------------- | +| **Primitive Types** | +| `boolean` | [`daft.DataType.bool()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.bool) | +| `int` | [`daft.DataType.int32()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.int32) | +| `long` | [`daft.DataType.int64()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.int64) | +| `float` | [`daft.DataType.float32()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.float32) | +| `double` | [`daft.DataType.float64()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.float64) | +| `decimal(precision, scale)` | [`daft.DataType.decimal128(precision, scale)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.decimal128) | +| `date` | [`daft.DataType.date()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.date) | +| `time` | [`daft.DataType.int64()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.int64) | +| `timestamp` | [`daft.DataType.timestamp(timeunit="us", timezone=None)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.timestamp) | +| `timestampz`| [`daft.DataType.timestamp(timeunit="us", timezone="UTC")`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.timestamp) | +| `string` | [`daft.DataType.string()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.string) | +| `uuid` | [`daft.DataType.binary()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.binary) | +| `fixed(L)` | [`daft.DataType.binary()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.binary) +| `binary` | [`daft.DataType.binary()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.binary) | +| **Nested Types** | +| `struct(fields)` | [`daft.DataType.struct(fields)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.struct) | +| `list(child_type)` | [`daft.DataType.list(child_type)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.list) | +| `map(K, V)` | [`daft.DataType.struct({"key": K, "value": V})`](https://www.getdaft.io/projects/docs/en/stable/api_docs/datatype.html#daft.DataType.struct) | + +## Roadmap + +Here are some features of Iceberg that are works-in-progress: + +1. Reading Iceberg V2 equality deletes +2. More extensive usage of Iceberg-provided statistics to further optimize queries +3. Copy-on-write and merge-on-read writes + +A more detailed Iceberg roadmap for Daft can be found on [our Github Issues page](https://github.com/Eventual-Inc/Daft/issues/2458). diff --git a/docs-v2/integrations/ray.md b/docs-v2/integrations/ray.md new file mode 100644 index 0000000000..0517248aa7 --- /dev/null +++ b/docs-v2/integrations/ray.md @@ -0,0 +1,93 @@ +# Ray + +[Ray](https://docs.ray.io/en/latest/ray-overview/index.html) is an open-source framework for distributed computing. Daft's native support for Ray enables you to run distributed DataFrame workloads at scale. + +## Usage + +You can run Daft on Ray in two ways: by using the [Ray Client](https://docs.ray.io/en/latest/cluster/running-applications/job-submission/ray-client.html) or by submitting a Ray job. + +### Ray Client + +The Ray client is a quick way to get started with running tasks and retrieving their results on Ray using Python. + +!!! warning "Warning" + + To run tasks using the Ray client, the version of Daft and the minor version (eg. 3.9, 3.10) of Python must match between client and server. + +Here's an example of how you can use the Ray client with Daft: + +=== "๐Ÿ Python" + + ```python + import daft + import ray + + # Refer to the note under "Ray Job" for details on "runtime_env" + ray.init("ray://:10001", runtime_env={"pip": ["getdaft"]}) + + # Starts the Ray client and tells Daft to use Ray to execute queries + # If ray.init() has already been called, it uses the existing client + daft.context.set_runner_ray("ray://:10001") + + df = daft.from_pydict({ + "a": [3, 2, 5, 6, 1, 4], + "b": [True, False, False, True, True, False] + }) + df = df.where(df["b"]).sort(df["a"]) + + # Daft executes the query remotely and returns a preview to the client + df.collect() + ``` + +```{title="Output"} +โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ฌโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ +โ”‚ a โ”† b โ”‚ +โ”‚ --- โ”† --- โ”‚ +โ”‚ Int64 โ”† Boolean โ”‚ +โ•žโ•โ•โ•โ•โ•โ•โ•โ•ชโ•โ•โ•โ•โ•โ•โ•โ•โ•โ•ก +โ”‚ 1 โ”† true โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 3 โ”† true โ”‚ +โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ผโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค +โ”‚ 6 โ”† true โ”‚ +โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”ดโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ + +(Showing first 3 of 3 rows) +``` + +### Ray Job + +Ray jobs allow for more control and observability over using the Ray client. In addition, your entire code runs on Ray, which means it is not constrained by the compute, network, library versions, or availability of your local machine. + +=== "๐Ÿ Python" + + ```python + # wd/job.py + + import daft + + def main(): + # call without any arguments to connect to Ray from the head node + daft.context.set_runner_ray() + + # ... Run Daft commands here ... + + if __name__ == "__main__": + main() + ``` + +To submit this script as a job, use the Ray CLI, which can be installed with `pip install "ray[default]"`. + +```bash +ray job submit \ + --working-dir wd \ + --address "http://:8265" \ + --runtime-env-json '{"pip": ["getdaft"]}' \ + -- python job.py +``` + +!!! note "Note" + + The runtime env parameter specifies that Daft should be installed on the Ray workers. Alternative methods of including Daft in the worker dependencies can be found [here](https://docs.ray.io/en/latest/ray-core/handling-dependencies.html). + +For more information about Ray jobs, see [Ray docs -> Ray Jobs Overview](https://docs.ray.io/en/latest/cluster/running-applications/job-submission/index.html). diff --git a/docs-v2/integrations/sql.md b/docs-v2/integrations/sql.md new file mode 100644 index 0000000000..95676d0082 --- /dev/null +++ b/docs-v2/integrations/sql.md @@ -0,0 +1,159 @@ +# SQL + +You can read the results of SQL queries from databases, data warehouses, and query engines, into a Daft DataFrame via the [`daft.read_sql()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.read_sql.html#daft.read_sql) function. + +Daft currently supports: + +1. **20+ SQL Dialects:** Daft supports over 20 databases, data warehouses, and query engines by using [SQLGlot](https://sqlglot.com/sqlglot.html) to convert SQL queries across dialects. See the full list of supported dialects [here](https://sqlglot.com/sqlglot/dialects.html). + +2. **Parallel + Distributed Reads:** Daft parallelizes SQL reads by using all local machine cores with its default multithreading runner, or all cores across multiple machines if using the [distributed Ray runner](../advanced/distributed.md). + +3. **Skipping Filtered Data:** Daft ensures that only data that matches your [`df.select(...)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.select.html#daft.DataFrame.select), [`df.limit(...)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.limit.html#daft.DataFrame.limit), and [`df.where(...)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.where.html#daft.DataFrame.where) expressions will be read, often skipping entire partitions/columns. + +## Installing Daft with SQL Support + +Install Daft with the `getdaft[sql]` extra, or manually install the required packages: [ConnectorX](https://sfu-db.github.io/connector-x/databases.html), [SQLAlchemy](https://docs.sqlalchemy.org/en/20/orm/quickstart.html) and [SQLGlot](https://sqlglot.com/sqlglot.html). + +```bash +pip install -U "getdaft[sql]" +``` + +## Reading a SQL query + +To read a SQL query, provide [`daft.read_sql()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.read_sql.html#daft.read_sql) with the **SQL query** and a **URL** for the data source. + +The example below creates a local SQLite table for Daft to read. + +=== "๐Ÿ Python" + + ```python + import sqlite3 + + connection = sqlite3.connect("example.db") + connection.execute( + "CREATE TABLE IF NOT EXISTS books (title TEXT, author TEXT, year INTEGER)" + ) + connection.execute( + """ + INSERT INTO books (title, author, year) + VALUES + ('The Great Gatsby', 'F. Scott Fitzgerald', 1925), + ('To Kill a Mockingbird', 'Harper Lee', 1960), + ('1984', 'George Orwell', 1949), + ('The Catcher in the Rye', 'J.D. Salinger', 1951) + """ + ) + connection.commit() + connection.close() + ``` + +After writing this local example table, we can easily read it into a Daft DataFrame. + +=== "๐Ÿ Python" + + ```python + # Read SQL query into Daft DataFrame + import daft + + df = daft.read_sql( + "SELECT * FROM books", + "sqlite://example.db", + ) + ``` + +Daft uses [ConnectorX](https://sfu-db.github.io/connector-x/databases.html) under the hood to read SQL data. ConnectorX is a fast, Rust based SQL connector that reads directly into Arrow Tables, enabling zero-copy transfer into Daft dataframes. If the database is not supported by ConnectorX (list of supported databases [here](https://sfu-db.github.io/connector-x/intro.html#supported-sources-destinations)), Daft will fall back to using [SQLAlchemy](https://docs.sqlalchemy.org/en/20/orm/quickstart.html). + +You can also directly provide a SQL alchemy connection via a **connection factory**. This way, you have the flexibility to provide additional parameters to the engine. + +=== "๐Ÿ Python" + + ```python + # Read SQL query into Daft DataFrame using a connection factory + import daft + from sqlalchemy import create_engine + + def create_connection(): + return sqlalchemy.create_engine("sqlite:///example.db", echo=True).connect() + + df = daft.read_sql("SELECT * FROM books", create_connection) + ``` + +## Parallel + Distributed Reads + +For large datasets, Daft can parallelize SQL reads by using all local machine cores with its default multithreading runner, or all cores across multiple machines if using the [distributed Ray runner](../advanced/distributed.md). + +Supply the [`daft.read_sql()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.read_sql.html#daft.read_sql) function with a **partition column** and optionally the **number of partitions** to enable parallel reads. + +=== "๐Ÿ Python" + + ```python + # Read SQL query into Daft DataFrame with parallel reads + import daft + + df = daft.read_sql( + "SELECT * FROM table", + "sqlite:///big_table.db", + partition_on="col", + num_partitions=3, + ) + ``` + +Behind the scenes, Daft will partition the data by appending a `WHERE col > ... AND col <= ...` clause to the SQL query, and then reading each partition in parallel. + +![SQL Distributed Read](../img/sql_distributed_read.png) + +## Data Skipping Optimizations + +Filter, projection, and limit pushdown optimizations can be used to reduce the amount of data read from the database. + +In the example below, Daft reads the top ranked terms from the BigQuery Google Trends dataset. The `where` and `select` expressions in this example will be pushed down into the SQL query itself, we can see this by calling the [`df.explain()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.explain.html#daft.DataFrame.explain) method. + +=== "๐Ÿ Python" + + ```python + import daft, sqlalchemy, datetime + + def create_conn(): + engine = sqlalchemy.create_engine( + "bigquery://", credentials_path="path/to/service_account_credentials.json" + ) + return engine.connect() + + + df = daft.read_sql("SELECT * FROM `bigquery-public-data.google_trends.top_terms`", create_conn) + + df = df.where((df["refresh_date"] >= datetime.date(2024, 4, 1)) & (df["refresh_date"] < datetime.date(2024, 4, 8))) + df = df.where(df["rank"] == 1) + df = df.select(df["refresh_date"].alias("Day"), df["term"].alias("Top Search Term"), df["rank"]) + df = df.distinct() + df = df.sort(df["Day"], desc=True) + + df.explain(show_all=True) + ``` + +``` {title="Output"} + +.. +== Physical Plan == +.. +| SQL Query = SELECT refresh_date, term, rank FROM + (SELECT * FROM `bigquery-public-data.google_trends.top_terms`) + AS subquery WHERE rank = 1 AND refresh_date >= CAST('2024-04-01' AS DATE) + AND refresh_date < CAST('2024-04-08' AS DATE) +``` + +The second last line labeled 'SQL Query =' shows the query that Daft executed. Filters such as `rank = 1` and projections such as `SELECT refresh_date, term, rank` have been injected into the query. + +Without these pushdowns, Daft would execute the unmodified `SELECT * FROM 'bigquery-public-data.google_trends.top_terms'` query and read in the entire dataset/table. We tested the code above on Google Colab (12GB RAM): + +- With pushdowns, the code ran in **8.87s** with a peak memory of **315.97 MiB** +- Without pushdowns, the code took over **2 mins** before crashing with an **out of memory** error. + +You could modify the SQL query to add the filters and projections yourself, but this may become lengthy and error-prone, particularly with many expressions. That's why Daft automatically handles it for you. + +## Roadmap + +Here are the SQL features that are on our roadmap. Please let us know if you would like to see support for any of these features! + +1. Write support into SQL databases. +2. Reads via [ADBC (Arrow Database Connectivity)](https://arrow.apache.org/docs/format/ADBC.html). diff --git a/docs-v2/integrations/unity_catalog.md b/docs-v2/integrations/unity_catalog.md new file mode 100644 index 0000000000..4e1599687b --- /dev/null +++ b/docs-v2/integrations/unity_catalog.md @@ -0,0 +1,69 @@ +# Unity Catalog + +[Unity Catalog](https://github.com/unitycatalog/unitycatalog/) is an open-sourced catalog developed by Databricks. Users of Unity Catalog are able to work with data assets such as tables (Parquet, CSV, Iceberg, Delta), volumes (storing raw files), functions and models. + +To use Daft with the Unity Catalog, you will need to install Daft with the `unity` option specified like so: + +```bash +pip install getdaft[unity] +``` + +!!! warning "Warning" + + These APIs are in beta and may be subject to change as the Unity Catalog continues to be developed. + +## Connecting to the Unity Catalog + +Daft includes an abstraction for the Unity Catalog. + +=== "๐Ÿ Python" + + ```python + from daft.unity_catalog import UnityCatalog + + unity = UnityCatalog( + endpoint="https://.cloud.databricks.com", + # Authentication can be retrieved from your provider of Unity Catalog + token="my-token", + ) + + # See all available catalogs + print(unity.list_catalogs()) + + # See available schemas in a given catalog + print(unity.list_schemas("my_catalog_name")) + + # See available tables in a given schema + print(unity.list_tables("my_catalog_name.my_schema_name")) + ``` + +## Loading a Daft Dataframe from a Delta Lake table in Unity Catalog + +=== "๐Ÿ Python" + + ```python + unity_table = unity.load_table("my_catalog_name.my_schema_name.my_table_name") + + df = daft.read_deltalake(unity_table) + df.show() + ``` + +Any subsequent filter operations on the Daft `df` DataFrame object will be correctly optimized to take advantage of DeltaLake features: + +=== "๐Ÿ Python" + + ```python + # Filter which takes advantage of partition pruning capabilities of Delta Lake + df = df.where(df["partition_key"] < 1000) + df.show() + ``` + +See also [Delta Lake](delta_lake.md) for more information about how to work with the Delta Lake tables provided by the Unity Catalog. + +## Roadmap + +1. Volumes integration for reading objects from volumes (e.g. images and documents) + +2. Unity Iceberg integration for reading tables using the Iceberg interface instead of the Delta Lake interface + +Please make issues on the [Daft repository](https://github.com/Eventual-Inc/Daft) if you have any use-cases that Daft does not currently cover! diff --git a/docs-v2/migration/dask_migration.md b/docs-v2/migration/dask_migration.md new file mode 100644 index 0000000000..359d2d91e5 --- /dev/null +++ b/docs-v2/migration/dask_migration.md @@ -0,0 +1,132 @@ +# Coming from Dask + +This migration guide explains the most important points that anyone familiar with Dask should know when trying out Daft or migrating Dask workloads to Daft. + +The guide includes an overview of technical, conceptual and syntax differences between the two libraries that you should be aware of. Understanding these differences will help you evaluate your choice of tooling and ease your migration from Dask to Daft. + +## When should I use Daft? + +Dask and Daft are DataFrame frameworks built for distributed computing. Both libraries enable you to process large, tabular datasets in parallel, either locally or on remote instances on-prem or in the cloud. + +If you are currently using Dask, you may want to consider migrating to Daft if you: + +- Are working with **multimodal data types**, such as nested JSON, tensors, Images, URLs, etc., +- Need faster computations through **query planning and optimization**, +- Are executing **machine learning workloads** at scale, +- Need deep support for **data catalogs, predicate pushdowns and metadata pruning** from Iceberg, Delta, and Hudi +- Want to benefit from **native Rust concurrency** + +You may want to stick with using Dask if you: + +- Want to only write **pandas-like syntax**, +- Need to parallelize **array-based workloads** or arbitrary **Python code that does not involve DataFrames** (with Dask Array, Dask Delayed and/or Dask Futures) + +The following sections explain conceptual and technical differences between Dask and Daft. Whenever relevant, code snippets are provided to illustrate differences in syntax. + +## Daft does not use an index + +Dask aims for as much feature-parity with pandas as possible, including maintaining the presence of an Index in the DataFrame. But keeping an Index is difficult when moving to a distributed computing environment. Dask doesnโ€™t support row-based positional indexing (with .iloc) because it does not track the length of its partitions. It also does not support pandas MultiIndex. The argument for keeping the Index is that it makes some operations against the sorted index column very fast. In reality, resetting the Index forces a data shuffle and is an expensive operation. + +Daft drops the need for an Index to make queries more readable and consistent. How you write a query should not change because of the state of an index or a reset_index call. In our opinion, eliminating the index makes things simpler, more explicit, more readable and therefore less error-prone. Daft achieves this by using the [Expressions API](https://www.getdaft.io/projects/docs/en/stable/api_docs/expressions.html). + +In Dask you would index your DataFrame to return row `b` as follows: + +```python +ddf.loc[[โ€œbโ€]] +``` + +In Daft, you would accomplish the same by using a `col` Expression to refer to the column that contains `b`: + +```python +df.where(daft.col(โ€œalphaโ€)==โ€bโ€) +``` + +More about Expressions in the sections below. + +## Daft does not try to copy the pandas syntax + +Dask is built as a parallelizable version of pandas and Dask partitions are in fact pandas DataFrames. When you call a Dask function you are often applying a pandas function on each partition. This makes Dask relatively easy to learn for people familiar with pandas, but it also causes complications when pandas logic (built for sequential processing) does not translate well to a distributed context. When reading the documentation, Dask users will often encounter this phrase `โ€œThis docstring was copied from pandas.coreโ€ฆ Some inconsistencies with the Dask version may exist.โ€` It is often unclear what these inconsistencies are and how they might affect performance. + +Daft does not try to copy the pandas syntax. Instead, we believe that efficiency is best achieved by defining logic specifically for the unique challenges of distributed computing. This means that we trade a slightly higher learning curve for pandas users against improved performance and more clarity for the developer experience. + +## Daft eliminates manual repartitioning of data + +In distributed settings, your data will always be partitioned for efficient parallel processing. How to partition this data is not straightforward and depends on factors like data types, query construction, and available cluster resources. While Dask often requires manual repartitioning for optimal performance, Daft abstracts this away from users so you donโ€™t have to worry about it. + +Dask leaves repartitioning up to the user with guidelines on having partitions that are โ€œnot too large and not too manyโ€. This is hard to interpret, especially given that the optimal partitioning strategy may be different for every query. Instead, Daft automatically controls your partitions in order to execute queries faster and more efficiently. As a side-effect, this means that Daft does not support partition indexing the way Dask does (i.e. โ€œget me partition Xโ€). If things are working well, you shouldn't need to index partitions like this. + +## Daft performs Query Optimization for optimal performance + +Daft is built with logical query optimization by default. This means that Daft will optimize your queries and skip any files or partitions that are not required for your query. This can give you significant performance gains, especially when working with file formats that support these kinds of optimized queries. + +Dask currently does not support full-featured query optimization. + +!!! note "Note" + + As of version 2024.3.0 Dask is slowly implementing query optimization as well. As far as we can tell this is still in early development and has some rough edges. For context see [the discussion](https://github.com/dask/dask/issues/10995_) in the Dask repo. + +## Daft uses Expressions and UDFs to perform computations in parallel + +Dask provides a `map_partitions` method to map computations over the partitions in your DataFrame. Since Dask partitions are pandas DataFrames, you can pass pandas functions to `map_partitions`. You can also map arbitrary Python functions over Dask partitions using `map_partitions`. + +For example: + +=== "๐Ÿ Python" + + ```python + def my_function(**kwargs): + return ... + + res = ddf.map_partitions(my_function, **kwargs) + ``` + +Daft implements two APIs for mapping computations over the data in your DataFrame in parallel: [Expressions](../core_concepts.md#expressions) and [User-Defined Functions (UDFs)](../core_concepts.md#user-defined-functions-udf). Expressions are most useful when you need to define computation over your columns. + +=== "๐Ÿ Python" + + ```python + # Add 1 to each element in column "A" + df = df.with_column("A_add_one", daft.col("A") + 1) + ``` + +You can use User-Defined Functions (UDFs) to run computations over multiple rows or columns: + +=== "๐Ÿ Python" + + ```python + # apply a custom function โ€œcrop_imageโ€ to the image column + @daft.udf(...) + def crop_image(**kwargs): + return ... + + df = df.with_column( + "cropped", + crop_image(daft.col("image"), **kwargs), + ) + ``` + +## Daft is built for Machine Learning Workloads + +Dask offers some distributed Machine Learning functionality through the [dask-ml library](https://ml.dask.org/). This library provides parallel implementations of a few common scikit-learn algorithms. Note that `dask-ml` is not a core Dask library and is not as actively maintained. It also does not offer support for deep-learning algorithms or neural networks. + +Daft is built as a DataFrame API for distributed Machine learning. You can use Daft UDFs to apply Machine Learning tasks to the data stored in your Daft DataFrame, including deep learning algorithms from libraries like PyTorch. See [our Quickstart](../10min.ipynb) for a toy example. + +## Daft supports Multimodal Data Types + +Dask supports the same data types as pandas. Daft is built to support many more data types, including Images, nested JSON, tensors, etc. See [the documentation](../core_concepts.md#datatypes) for a list of all supported data types. + +## Distributed Computing and Remote Clusters + +Both Dask and Daft support distributed computing on remote clusters. In Dask, you create a Dask cluster either locally or remotely and perform computations in parallel there. Currently, Daft supports distributed cluster computing [with Ray](../advanced/distributed.md). Support for running Daft computations on Dask clusters is on the roadmap. + +Cloud support for both Dask and Daft is the same. + +## SQL Support + +Dask does not natively provide full support for running SQL queries. You can use pandas-like code to write SQL-equivalent queries, or use the external [dask-sql library](https://dask-sql.readthedocs.io/en/latest/). + +Daft provides a [`read_sql()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.read_sql.html) method to read SQL queries into a DataFrame. Daft uses SQLGlot to build SQL queries, so it supports all databases that SQLGlot supports. Daft pushes down operations such as filtering, projections, and limits into the SQL query when possible. Full-featured support for SQL queries (as opposed to a DataFrame API) is in progress. + +## Daft combines Python with Rust and Pyarrow for optimal performance + +Daft combines Python with Rust and Pyarrow for optimal performance (see [Benchmarks](../resources/benchmarks/tpch.md)). Under the hood, Table and Series are implemented in Rust on top of the Apache Arrow specification (using the Rust arrow2 library). This architecture means that all the computationally expensive operations on Table and Series are performed in Rust, and can be heavily optimized for raw speed. Python is most useful as a user-facing API layer for ease of use and an interactive data science user experience (see [Architecture](../resources/architecture.md)). diff --git a/docs-v2/quickstart.ipynb b/docs-v2/quickstart.ipynb new file mode 100644 index 0000000000..f25ee74898 --- /dev/null +++ b/docs-v2/quickstart.ipynb @@ -0,0 +1,807 @@ +{ + "cells": [ + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# Quickstart\n", + "\n", + "In this quickstart, you will learn the basics of Daft's DataFrame and SQL API and the features that set it apart from frameworks like Pandas, PySpark, Dask, and Ray.\n", + "\n", + "
\n", + "

todo(docs): incorporate sql examples

\n", + "
" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Install Daft\n", + "\n", + "You can install Daft using `pip`. Run the following command in your terminal or notebook:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "pip install getdaft" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Create Your First Daft DataFrame\n", + "\n", + "Let's create a DataFrame from a dictionary of columns:" + ] + }, + { + "cell_type": "code", + "execution_count": 18, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
A
Int64
B
Float64
C
Boolean
D
Null
1
1.5
true
None
2
2.5
true
None
3
3.5
false
None
4
4.5
false
None
\n", + "(Showing first 4 of 4 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 A \u2506 B \u2506 C \u2506 D \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Int64 \u2506 Float64 \u2506 Boolean \u2506 Null \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 1 \u2506 1.5 \u2506 true \u2506 None \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 2 \u2506 2.5 \u2506 true \u2506 None \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 3 \u2506 3.5 \u2506 false \u2506 None \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 4 \u2506 4.5 \u2506 false \u2506 None \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 4 of 4 rows)" + ] + }, + "execution_count": 18, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "import daft\n", + "\n", + "df = daft.from_pydict(\n", + " {\n", + " \"A\": [1, 2, 3, 4],\n", + " \"B\": [1.5, 2.5, 3.5, 4.5],\n", + " \"C\": [True, True, False, False],\n", + " \"D\": [None, None, None, None],\n", + " }\n", + ")\n", + "\n", + "df" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "You just created your first DataFrame!" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Read From a Data Source\n", + "\n", + "Daft supports both local paths as well as paths to object storage such as AWS S3:\n", + "\n", + "- CSV files: [`daft.read_csv(\"s3://path/to/bucket/*.csv\")`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.read_csv.html#daft.read_csv)\n", + "- Parquet files: [`daft.read_parquet(\"/path/*.parquet\")`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.read_parquet.html#daft.read_parquet)\n", + "- JSON line-delimited files: [`daft.read_json(\"/path/*.json\")`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.read_json.html#daft.read_json)\n", + "- Files on disk: [`daft.from_glob_path(\"/path/*.jpeg\")`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.from_glob_path.html#daft.from_glob_path)\n", + "\n", + "
\n", + "\n", + "Let\u2019s read in a Parquet file from a public S3 bucket. Note that this Parquet file is partitioned on the column `country`. This will be important later on.\n", + "\n", + "
\n", + "

todo(docs): sql equivalent?

\n", + "
" + ] + }, + { + "cell_type": "code", + "execution_count": 19, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "
first_name
Utf8
last_name
Utf8
age
Int64
DoB
Date
country
Utf8
has_dog
Boolean
\n", + "(No data to display: Dataframe not materialized)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 first_name \u2506 last_name \u2506 age \u2506 DoB \u2506 country \u2506 has_dog \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Utf8 \u2506 Int64 \u2506 Date \u2506 Utf8 \u2506 Boolean \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(No data to display: Dataframe not materialized)" + ] + }, + "execution_count": 19, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "# Set IO Configurations to use anonymous data access mode\n", + "daft.set_planning_config(default_io_config=daft.io.IOConfig(s3=daft.io.S3Config(anonymous=True)))\n", + "\n", + "df = daft.read_parquet(\"s3://daft-public-data/tutorials/10-min/sample-data-dog-owners-partitioned.pq/**\")\n", + "df" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Why does it say `(No data to display: Dataframe not materialized)` and where are the rows?\n", + "\n", + "## Execute Your DataFrame and View Data\n", + "\n", + "Daft DataFrames are **lazy** by default. This means that the contents will not be computed (\u201cmaterialized\u201d) unless you explicitly tell Daft to do so. This is best practice for working with larger-than-memory datasets and parallel/distributed architectures.\n", + "\n", + "The file we have just loaded only has 5 rows. You can materialize the whole DataFrame in memory easily using the [`df.collect()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.collect.html#daft.DataFrame.collect) method:\n", + "\n", + "
\n", + "

todo(docs): sql equivalent?

\n", + "
\n" + ] + }, + { + "cell_type": "code", + "execution_count": 20, + "metadata": {}, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + " \r" + ] + }, + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
first_name
Utf8
last_name
Utf8
age
Int64
DoB
Date
country
Utf8
has_dog
Boolean
Wolfgang
Winter
23
2001-02-12
Germany
None
Shandra
Shamas
57
1967-01-02
United Kingdom
true
Zaya
Zaphora
40
1984-04-07
United Kingdom
true
Ernesto
Evergreen
34
1990-04-03
Canada
true
James
Jale
62
1962-03-24
Canada
true
\n", + "(Showing first 5 of 5 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 first_name \u2506 last_name \u2506 age \u2506 DoB \u2506 country \u2506 has_dog \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Utf8 \u2506 Int64 \u2506 Date \u2506 Utf8 \u2506 Boolean \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Wolfgang \u2506 Winter \u2506 23 \u2506 2001-02-12 \u2506 Germany \u2506 None \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Shandra \u2506 Shamas \u2506 57 \u2506 1967-01-02 \u2506 United Kingdom \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya \u2506 Zaphora \u2506 40 \u2506 1984-04-07 \u2506 United Kingdom \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Ernesto \u2506 Evergreen \u2506 34 \u2506 1990-04-03 \u2506 Canada \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 James \u2506 Jale \u2506 62 \u2506 1962-03-24 \u2506 Canada \u2506 true \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 5 of 5 rows)" + ] + }, + "execution_count": 20, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "df.collect()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "To view just the first few rows, you can use the [`df.show()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.show.html#daft.DataFrame.show) method:" + ] + }, + { + "cell_type": "code", + "execution_count": 21, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
first_name
Utf8
last_name
Utf8
age
Int64
DoB
Date
country
Utf8
has_dog
Boolean
Wolfgang
Winter
23
2001-02-12
Germany
None
Shandra
Shamas
57
1967-01-02
United Kingdom
true
Zaya
Zaphora
40
1984-04-07
United Kingdom
true
\n", + "(Showing first 3 of 5 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 first_name \u2506 last_name \u2506 age \u2506 DoB \u2506 country \u2506 has_dog \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Utf8 \u2506 Int64 \u2506 Date \u2506 Utf8 \u2506 Boolean \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Wolfgang \u2506 Winter \u2506 23 \u2506 2001-02-12 \u2506 Germany \u2506 None \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Shandra \u2506 Shamas \u2506 57 \u2506 1967-01-02 \u2506 United Kingdom \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya \u2506 Zaphora \u2506 40 \u2506 1984-04-07 \u2506 United Kingdom \u2506 true \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 3 of 5 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df.show(3)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Now let's take a look at some common DataFrame operations." + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Selecting Columns\n", + "\n", + "You can **select** specific columns from your DataFrame with the [`df.select()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.select.html#daft.DataFrame.select) method.\n", + "\n", + "
\n", + "

todo(docs): sql equivalent?

\n", + "
" + ] + }, + { + "cell_type": "code", + "execution_count": 22, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
first_name
Utf8
has_dog
Boolean
Wolfgang
None
Shandra
true
Zaya
true
Ernesto
true
James
true
\n", + "(Showing first 5 of 5 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 first_name \u2506 has_dog \u2502\n", + "\u2502 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Boolean \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Wolfgang \u2506 None \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Shandra \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Ernesto \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 James \u2506 true \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 5 of 5 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df.select(\"first_name\", \"has_dog\").show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Selecting Rows\n", + "\n", + "You can **filter** rows using the [`df.where()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.where.html#daft.DataFrame.where) method that takes an Logical Expression predicate input. In this case, we call the [`df.col()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.col.html#daft.col) method that refers to the column with the provided name `age`:" + ] + }, + { + "cell_type": "code", + "execution_count": 23, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
first_name
Utf8
last_name
Utf8
age
Int64
DoB
Date
country
Utf8
has_dog
Boolean
Shandra
Shamas
57
1967-01-02
United Kingdom
true
Zaya
Zaphora
40
1984-04-07
United Kingdom
true
James
Jale
62
1962-03-24
Canada
true
\n", + "(Showing first 3 of 3 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 first_name \u2506 last_name \u2506 age \u2506 DoB \u2506 country \u2506 has_dog \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Utf8 \u2506 Int64 \u2506 Date \u2506 Utf8 \u2506 Boolean \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Shandra \u2506 Shamas \u2506 57 \u2506 1967-01-02 \u2506 United Kingdom \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya \u2506 Zaphora \u2506 40 \u2506 1984-04-07 \u2506 United Kingdom \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 James \u2506 Jale \u2506 62 \u2506 1962-03-24 \u2506 Canada \u2506 true \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 3 of 3 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df.where(daft.col(\"age\") >= 40).show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Filtering can give you powerful optimization when you are working with partitioned files or tables. Daft will use the predicate to read only the necessary partitions, skipping any data that is not relevant.\n", + "\n", + "
\n", + "

Note

\n", + "

\n", + " As mentioned earlier that our Parquet file is partitioned on the country column, this means that queries with a country predicate will benefit from query optimization.\n", + "

\n", + "
" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Excluding Data\n", + "\n", + "You can **limit** the number of rows in a DataFrame by calling the [`df.limit()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.limit.html#daft.DataFrame.limit) method:" + ] + }, + { + "cell_type": "code", + "execution_count": 24, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
first_name
Utf8
last_name
Utf8
age
Int64
DoB
Date
country
Utf8
has_dog
Boolean
Wolfgang
Winter
23
2001-02-12
Germany
None
\n", + "(Showing first 1 of 1 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 first_name \u2506 last_name \u2506 age \u2506 DoB \u2506 country \u2506 has_dog \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Utf8 \u2506 Int64 \u2506 Date \u2506 Utf8 \u2506 Boolean \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Wolfgang \u2506 Winter \u2506 23 \u2506 2001-02-12 \u2506 Germany \u2506 None \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 1 of 1 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df.limit(1).show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "To **drop** columns from the DataFrame, use the [`df.exclude()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.exclude.html#daft.DataFrame.exclude) method." + ] + }, + { + "cell_type": "code", + "execution_count": 25, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
first_name
Utf8
last_name
Utf8
age
Int64
country
Utf8
has_dog
Boolean
Wolfgang
Winter
23
Germany
None
Shandra
Shamas
57
United Kingdom
true
Zaya
Zaphora
40
United Kingdom
true
Ernesto
Evergreen
34
Canada
true
James
Jale
62
Canada
true
\n", + "(Showing first 5 of 5 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 first_name \u2506 last_name \u2506 age \u2506 country \u2506 has_dog \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Utf8 \u2506 Int64 \u2506 Utf8 \u2506 Boolean \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Wolfgang \u2506 Winter \u2506 23 \u2506 Germany \u2506 None \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Shandra \u2506 Shamas \u2506 57 \u2506 United Kingdom \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya \u2506 Zaphora \u2506 40 \u2506 United Kingdom \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Ernesto \u2506 Evergreen \u2506 34 \u2506 Canada \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 James \u2506 Jale \u2506 62 \u2506 Canada \u2506 true \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 5 of 5 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df.exclude(\"DoB\").show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Transforming Columns with Expressions\n", + "\n", + "[Expressions](core_concepts/expressions.md) are an API for defining computation that needs to happen over columns. For example, use the [`daft.col()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.col.html#daft.col) expressions together with the [`with_column`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.with_column.html#daft.DataFrame.with_column) method to create a new column called `full_name`, joining the contents from the `last_name` column with the `first_name` column:" + ] + }, + { + "cell_type": "code", + "execution_count": 26, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
full_name
Utf8
age
Int64
country
Utf8
has_dog
Boolean
Wolfgang Winter
23
Germany
None
Shandra Shamas
57
United Kingdom
true
Zaya Zaphora
40
United Kingdom
true
Ernesto Evergreen
34
Canada
true
James Jale
62
Canada
true
\n", + "(Showing first 5 of 5 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 full_name \u2506 age \u2506 country \u2506 has_dog \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Int64 \u2506 Utf8 \u2506 Boolean \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Wolfgang Winter \u2506 23 \u2506 Germany \u2506 None \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Shandra Shamas \u2506 57 \u2506 United Kingdom \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya Zaphora \u2506 40 \u2506 United Kingdom \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Ernesto Evergreen \u2506 34 \u2506 Canada \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 James Jale \u2506 62 \u2506 Canada \u2506 true \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 5 of 5 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df = df.with_column(\"full_name\", daft.col(\"first_name\") + \" \" + daft.col(\"last_name\"))\n", + "df.select(\"full_name\", \"age\", \"country\", \"has_dog\").show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Alternatively, you can also run your column transformation using Expressions directly inside your [`df.select()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.select.html#daft.DataFrame.select) method:" + ] + }, + { + "cell_type": "code", + "execution_count": 27, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
full_name
Utf8
age
Int64
country
Utf8
has_dog
Boolean
Wolfgang Winter
23
Germany
None
Shandra Shamas
57
United Kingdom
true
Zaya Zaphora
40
United Kingdom
true
Ernesto Evergreen
34
Canada
true
James Jale
62
Canada
true
\n", + "(Showing first 5 of 5 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 full_name \u2506 age \u2506 country \u2506 has_dog \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Int64 \u2506 Utf8 \u2506 Boolean \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Wolfgang Winter \u2506 23 \u2506 Germany \u2506 None \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Shandra Shamas \u2506 57 \u2506 United Kingdom \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya Zaphora \u2506 40 \u2506 United Kingdom \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Ernesto Evergreen \u2506 34 \u2506 Canada \u2506 true \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 James Jale \u2506 62 \u2506 Canada \u2506 true \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 5 of 5 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df.select((daft.col(\"first_name\").alias(\"full_name\") + \" \" + daft.col(\"last_name\")), \"age\", \"country\", \"has_dog\").show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Sorting Data\n", + "\n", + "You can **sort** a DataFrame with the [`df.sort()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.sort.html#daft.DataFrame.sort), in this example we chose to sort in ascending order:" + ] + }, + { + "cell_type": "code", + "execution_count": 31, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
first_name
Utf8
last_name
Utf8
age
Int64
DoB
Date
country
Utf8
has_dog
Boolean
full_name
Utf8
Wolfgang
Winter
23
2001-02-12
Germany
None
Wolfgang Winter
Ernesto
Evergreen
34
1990-04-03
Canada
true
Ernesto Evergreen
Zaya
Zaphora
40
1984-04-07
United Kingdom
true
Zaya Zaphora
Shandra
Shamas
57
1967-01-02
United Kingdom
true
Shandra Shamas
James
Jale
62
1962-03-24
Canada
true
James Jale
\n", + "(Showing first 5 of 5 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 first_name \u2506 last_name \u2506 age \u2506 \u2026 \u2506 country \u2506 has_dog \u2506 full_name \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2506 \u2506 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Utf8 \u2506 Int64 \u2506 (1 hidden) \u2506 Utf8 \u2506 Boolean \u2506 Utf8 \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Wolfgang \u2506 Winter \u2506 23 \u2506 \u2026 \u2506 Germany \u2506 None \u2506 Wolfgang Winter \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Ernesto \u2506 Evergreen \u2506 34 \u2506 \u2026 \u2506 Canada \u2506 true \u2506 Ernesto Evergreen \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Zaya \u2506 Zaphora \u2506 40 \u2506 \u2026 \u2506 United Kingdom \u2506 true \u2506 Zaya Zaphora \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Shandra \u2506 Shamas \u2506 57 \u2506 \u2026 \u2506 United Kingdom \u2506 true \u2506 Shandra Shamas \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 James \u2506 Jale \u2506 62 \u2506 \u2026 \u2506 Canada \u2506 true \u2506 James Jale \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 5 of 5 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "df.sort(daft.col(\"age\"), desc=False).show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Grouping and Aggregating Data\n", + "\n", + "You can **group** and **aggregate** your data using the [`df.groupby()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.groupby.html#daft.DataFrame.groupby) and the [`df.agg()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.agg.html#daft.DataFrame.agg) methods. A groupby aggregation operation over a dataset happens in 2 steps:\n", + "\n", + "1. Split the data into groups based on some criteria using [`df.groupby()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.groupby.html#daft.DataFrame.groupby)\n", + "2. Specify how to aggregate the data for each group using [`df.agg()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.agg.html#daft.DataFrame.agg)" + ] + }, + { + "cell_type": "code", + "execution_count": 29, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
country
Utf8
avg_age
Float64
has_dog
UInt64
Canada
48
2
Germany
23
0
United Kingdom
48.5
2
\n", + "(Showing first 3 of 3 rows)\n", + "
" + ], + "text/plain": [ + "\u256d\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u252c\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256e\n", + "\u2502 country \u2506 avg_age \u2506 has_dog \u2502\n", + "\u2502 --- \u2506 --- \u2506 --- \u2502\n", + "\u2502 Utf8 \u2506 Float64 \u2506 UInt64 \u2502\n", + "\u255e\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u256a\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2550\u2561\n", + "\u2502 Canada \u2506 48 \u2506 2 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 Germany \u2506 23 \u2506 0 \u2502\n", + "\u251c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u253c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u254c\u2524\n", + "\u2502 United Kingdom \u2506 48.5 \u2506 2 \u2502\n", + "\u2570\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2534\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u2500\u256f\n", + "\n", + "(Showing first 3 of 3 rows)" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "grouped = df.groupby(\"country\").agg(daft.col(\"age\").mean().alias(\"avg_age\"), daft.col(\"has_dog\").count()).show()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "
\n", + "

Note

\n", + "

\n", + " The df.alias method renames the given column.\n", + "

\n", + "
" + ] + } + ], + "metadata": { + "kernelspec": { + "display_name": "venv", + "language": "python", + "name": "python3" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.13.0" + } + }, + "nbformat": 4, + "nbformat_minor": 4 +} diff --git a/docs-v2/quickstart.md b/docs-v2/quickstart.md new file mode 100644 index 0000000000..c744d99c9e --- /dev/null +++ b/docs-v2/quickstart.md @@ -0,0 +1,418 @@ +# Quickstart + +!!! failure "todo(docs): Are there too many sections?" + +!!! failure "todo(docs): Incorporate SQL examples" + +In this quickstart, you will learn the basics of Daft's DataFrame and SQL API and the features that set it apart from frameworks like Pandas, PySpark, Dask, and Ray. + + + +## Install Daft + +You can install Daft using `pip`. Run the following command in your terminal or notebook: + +=== "๐Ÿ Python" + + ```python + pip install getdaft + ``` + +For more advanced installation options, please see [Installation](install.md). + +## Create Your First Daft DataFrame + +!!! failure "todo(docs): Simplify this example, take from "dataframe" section, but will they be too similar now?" + +See also [DataFrame Creation](https://www.getdaft.io/projects/docs/en/stable/api_docs/creation.html#df-input-output). Let's create a DataFrame from a dictionary of columns: + +=== "๐Ÿ Python" + ```python + import daft + + df = daft.from_pydict({ + "A": [1, 2, 3, 4], + "B": [1.5, 2.5, 3.5, 4.5], + "C": [True, True, False, False], + "D": [None, None, None, None], + }) + + df + ``` + +``` {title="Output"} + ++-------+---------+---------+------+ +| A | B | C | D | +| Int64 | Float64 | Boolean | Null | ++=======+=========+=========+======+ +| 1 | 1.5 | true | None | ++-------+---------+---------+------+ +| 2 | 2.5 | true | None | ++-------+---------+---------+------+ +| 3 | 3.5 | false | None | ++-------+---------+---------+------+ +| 4 | 4.5 | false | None | ++-------+---------+---------+------+ + + +(Showing first 4 of 4 rows) +``` + +You just created your first DataFrame! + +## Read From a Data Source + +Daft supports both local paths as well as paths to object storage such as AWS S3: + +- CSV files: [`daft.read_csv("s3://path/to/bucket/*.csv")`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.read_csv.html#daft.read_csv) +- Parquet files: [`daft.read_parquet("/path/*.parquet")`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.read_parquet.html#daft.read_parquet) +- JSON line-delimited files: [`daft.read_json("/path/*.json")`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.read_json.html#daft.read_json) +- Files on disk: [`daft.from_glob_path("/path/*.jpeg")`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/io_functions/daft.from_glob_path.html#daft.from_glob_path) + +!!! tip "Note" + + See [Integrations](https://www.getdaft.io/projects/docs/en/stable/user_guide/integrations.html) to learn more about working with other formats like Delta Lake and Iceberg. + +Letโ€™s read in a Parquet file from a public S3 bucket. Note that this Parquet file is partitioned on the column `country`. This will be important later on. + +!!! failure "todo(docs): SQL equivalent?" + + +=== "๐Ÿ Python" + ```python + + # Set IO Configurations to use anonymous data access mode + daft.set_planning_config(default_io_config=daft.io.IOConfig(s3=daft.io.S3Config(anonymous=True))) + + df = daft.read_parquet("s3://daft-public-data/tutorials/10-min/sample-data-dog-owners-partitioned.pq/**") + df + ``` + +```{title="Output"} + ++------------+-----------+-------+------+---------+---------+ +| first_name | last_name | age | DoB | country | has_dog | +| Utf8 | Utf8 | Int64 | Date | Utf8 | Boolean | ++------------+-----------+-------+------+---------+---------+ + +(No data to display: Dataframe not materialized) + +``` + +Why does it say `(No data to display: Dataframe not materialized)` and where are the rows? + +## Execute Your DataFrame and View Data + +Daft DataFrames are **lazy** by default. This means that the contents will not be computed (โ€œmaterializedโ€) unless you explicitly tell Daft to do so. This is best practice for working with larger-than-memory datasets and parallel/distributed architectures. + +The file we have just loaded only has 5 rows. You can materialize the whole DataFrame in memory easily using the [`df.collect()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.collect.html#daft.DataFrame.collect]) method: + +!!! failure "todo(docs): How does SQL materialize the DataFrame?" + +=== "๐Ÿ Python" + + ```python + df.collect() + ``` + +```{title="Output"} + ++------------+-----------+-------+------------+----------------+---------+ +| first_name | last_name | age | DoB | country | has_dog | +| Utf8 | Utf8 | Int64 | Date | Utf8 | Boolean | ++------------+-----------+-------+------------+----------------+---------+ +| Ernesto | Evergreen | 34 | 1990-04-03 | Canada | true | +| James | Jale | 62 | 1962-03-24 | Canada | true | +| Wolfgang | Winter | 23 | 2001-02-12 | Germany | None | +| Shandra | Shamas | 57 | 1967-01-02 | United Kingdom | true | +| Zaya | Zaphora | 40 | 1984-04-07 | United Kingdom | true | ++------------+-----------+-------+------------+----------------+---------+ +(Showing first 5 of 5 rows) +``` + +To view just the first few rows, you can use the [`df.show()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.show.html#daft.DataFrame.show) method: + +=== "๐Ÿ Python" + + ```python + df.show(3) + ``` + +```{title="Output"} + ++------------+-----------+-------+------------+----------------+---------+ +| first_name | last_name | age | DoB | country | has_dog | +| Utf8 | Utf8 | Int64 | Date | Utf8 | Boolean | ++------------+-----------+-------+------------+----------------+---------+ +| Ernesto | Evergreen | 34 | 1990-04-03 | Canada | true | +| James | Jale | 62 | 1962-03-24 | Canada | true | +| Wolfgang | Winter | 23 | 2001-02-12 | Germany | None | ++------------+-----------+-------+------------+----------------+---------+ +(Showing first 3 of 5 rows) + +``` + +Now let's take a look at some common DataFrame operations. + +## Select Columns + +!!! failure "todo(docs): SQL equivalent?" + +You can **select** specific columns from your DataFrame with the [`df.select()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.select.html#daft.DataFrame.select) method: + +=== "๐Ÿ Python" + + ```python + df.select("first_name", "has_dog").show() + ``` + +```{title="Output"} + ++------------+---------+ +| first_name | has_dog | +| Utf8 | Boolean | ++------------+---------+ +| Ernesto | true | +| James | true | +| Wolfgang | None | +| Shandra | true | +| Zaya | true | ++------------+---------+ +(Showing first 5 of 5 rows) + +``` +## Select Rows + +You can **filter** rows using the [`df.where()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.where.html#daft.DataFrame.where) method that takes an Logical Expression predicate input. In this case, we call the [`df.col()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.col.html#daft.col) method that refers to the column with the provided name `age`: + +=== "๐Ÿ Python" + + ```python + df.where(daft.col("age") >= 40).show() + ``` + +```{title="Output"} ++------------+-----------+-------+------------+----------------+---------+ +| first_name | last_name | age | DoB | country | has_dog | +| Utf8 | Utf8 | Int64 | Date | Utf8 | Boolean | ++------------+-----------+-------+------------+----------------+---------+ +| James | Jale | 62 | 1962-03-24 | Canada | true | +| Shandra | Shamas | 57 | 1967-01-02 | United Kingdom | true | +| Zaya | Zaphora | 40 | 1984-04-07 | United Kingdom | true | ++------------+-----------+-------+------------+----------------+---------+ +(Showing first 3 of 3 rows) +``` + +Filtering can give you powerful optimization when you are working with partitioned files or tables. Daft will use the predicate to read only the necessary partitions, skipping any data that is not relevant. + +!!! tip "Note" + + As mentioned earlier that our Parquet file is partitioned on the `country` column, this means that queries with a `country` predicate will benefit from query optimization. + +## Exclude Data + +You can **limit** the number of rows in a DataFrame by calling the [`df.limit()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.limit.html#daft.DataFrame.limit) method: + +=== "๐Ÿ Python" + + ```python + df.limit(2).show() + ``` + +```{title="Output"} + ++------------+-----------+-------+------------+----------------+---------+ +| first_name | last_name | age | DoB | country | has_dog | +| Utf8 | Utf8 | Int64 | Date | Utf8 | Boolean | ++------------+-----------+-------+------------+----------------+---------+ +| Ernesto | Evergreen | 34 | 1990-04-03 | Canada | true | ++------------+-----------+-------+------------+----------------+---------+ +(Showing first 1 of 1 rows) +``` + +To **drop** columns from the DataFrame, use the [`df.exclude()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.exclude.html#daft.DataFrame.exclude) method. + +=== "๐Ÿ Python" + + ```python + df.exclude("DoB").show() + ``` + +```{title="Output"} + ++------------+-----------+-------+----------------+---------+ +| first_name | last_name | age | country | has_dog | +| Utf8 | Utf8 | Int64 | Utf8 | Boolean | ++------------+-----------+-------+----------------+---------+ +| Ernesto | Evergreen | 34 | Canada | true | +| James | Jale | 62 | Canada | true | +| Wolfgang | Winter | 23 | Germany | None | +| Shandra | Shamas | 57 | United Kingdom | true | +| Zaya | Zaphora | 40 | United Kingdom | true | ++------------+-----------+-------+----------------+---------+ +(Showing first 5 of 5 rows) + +``` + +## Transform Columns with Expressions + +[Expressions](core_concepts.md#expressions) are an API for defining computation that needs to happen over columns. For example, use the [`daft.col()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.col.html#daft.col) expressions together with the [`with_column`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.with_column.html#daft.DataFrame.with_column) method to create a new column called `full_name`, joining the contents from the `last_name` column with the `first_name` column: + +=== "๐Ÿ Python" + + ```python + df = df.with_column("full_name", daft.col("first_name") + " " + daft.col("last_name")) + df.select("full_name", "age", "country", "has_dog").show() + ``` + +```{title="Output"} + ++-------------------+-------+----------------+---------+ +| full_name | age | country | has_dog | +| Utf8 | Int64 | Utf8 | Boolean | ++-------------------+-------+----------------+---------+ +| Ernesto Evergreen | 34 | Canada | true | +| James Jale | 62 | Canada | true | +| Wolfgang Winter | 23 | Germany | None | +| Shandra Shamas | 57 | United Kingdom | true | +| Zaya Zaphora | 40 | United Kingdom | true | ++-------------------+-------+----------------+---------+ +(Showing first 5 of 5 rows) +``` + +Alternatively, you can also run your column transformation using Expressions directly inside your [`df.select()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.select.html#daft.DataFrame.select) method*: + +=== "๐Ÿ Python" + + ```python + df.select((daft.col("first_name").alias("full_name") + " " + daft.col("last_name")), "age", "country", "has_dog").show() + ``` + +```{title="Output"} + ++-------------------+-------+----------------+---------+ +| full_name | age | country | has_dog | +| Utf8 | Int64 | Utf8 | Boolean | ++-------------------+-------+----------------+---------+ +| Ernesto Evergreen | 34 | Canada | true | +| James Jale | 62 | Canada | true | +| Wolfgang Winter | 23 | Germany | None | +| Shandra Shamas | 57 | United Kingdom | true | +| Zaya Zaphora | 40 | United Kingdom | true | ++-------------------+-------+----------------+---------+ +(Showing first 5 of 5 rows) +``` + +## Sort Data + +You can **sort** a DataFrame with the [`df.sort()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.sort.html#daft.DataFrame.sort), in this example we chose to sort in ascending order: + +=== "๐Ÿ Python" + + ```python + df.sort(daft.col("age"), desc=False).show() + ``` + +```{title="Output"} + ++------------+-----------+-------+------------+----------------+---------+ +| first_name | last_name | age | DoB | country | has_dog | +| Utf8 | Utf8 | Int64 | Date | Utf8 | Boolean | ++------------+-----------+-------+------------+----------------+---------+ +| Wolfgang | Winter | 23 | 2001-02-12 | Germany | None | +| Ernesto | Evergreen | 34 | 1990-04-03 | Canada | true | +| Zaya | Zaphora | 40 | 1984-04-07 | United Kingdom | true | +| Shandra | Shamas | 57 | 1967-01-02 | United Kingdom | true | +| James | Jale | 62 | 1962-03-24 | Canada | true | ++------------+-----------+-------+------------+----------------+---------+ +(Showing first 5 of 5 rows) +``` + +## Group and Aggregate Data + +You can **group** and **aggregate** your data using the [`df.groupby()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.groupby.html#daft.DataFrame.groupby) and the [`df.agg()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.agg.html#daft.DataFrame.agg) methods. A groupby aggregation operation over a dataset happens in 2 steps: + +1. Split the data into groups based on some criteria using [`df.groupby()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.groupby.html#daft.DataFrame.groupby) +2. Specify how to aggregate the data for each group using [`df.agg()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.agg.html#daft.DataFrame.agg) + +=== "๐Ÿ Python" + + ```python + grouped = df.groupby("country").agg( + daft.col("age").mean().alias("avg_age"), + daft.col("has_dog").count() + ).show() + ``` + +```{title="Output"} + ++----------------+---------+---------+ +| country | avg_age | has_dog | +| Utf8 | Float64 | UInt64 | ++----------------+---------+---------+ +| Canada | 48 | 2 | +| Germany | 23 | 0 | +| United Kingdom | 48.5 | 2 | ++----------------+---------+---------+ +(Showing first 3 of 3 rows) +``` + +!!! tip "Note" + + The [`df.alias()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.Expression.alias.html#daft.Expression.alias) method renames the given column. + + +## What's Next? + +Now that you have a basic sense of Daftโ€™s functionality and features, here are some more resources to help you get the most out of Daft: + +**Check out the Core Concepts sections for more details about:** + +
+ +- [:material-filter: **DataFrame Operations**](core_concepts.md#dataframe) +- [:octicons-code-16: **Expressions**](core_concepts.md#expressions) +- [:material-file-eye: **Reading Data**](core_concepts.md#reading-data) +- [:material-file-edit: **Writing Data**](core_concepts.md#reading-data) +- [:fontawesome-solid-square-binary: **DataTypes**](core_concepts.md#datatypes) +- [:simple-quicklook: **SQL**](core_concepts.md#sql) +- [:material-select-group: **Aggregations and Grouping**](core_concepts.md#aggregations-and-grouping) +- [:fontawesome-solid-user: **User-Defined Functions (UDFs)**](core_concepts.md#user-defined-functions-udf) +- [:octicons-image-16: **Multimodal Data**](core_concepts.md#multimodal-data) + +
+ +**Work with your favorite tools**: + +
+ +- [:simple-ray: **Ray**](integrations/ray.md) +- [**Unity Catalog**](integrations/unity_catalog.md) +- [**Apache Iceberg**](integrations/iceberg.md) +- [**Delta Lake**](integrations/delta_lake.md) +- [:material-microsoft-azure: **Microsoft Azure**](integrations/azure.md) +- [:fontawesome-brands-aws: **Amazon Web Services (AWS)**](integrations/aws.md) +- [**SQL**](integrations/sql.md) +- [:simple-huggingface: **Hugging Face Datasets**](integrations/huggingface.md) + +
+ +**Coming from?** + +
+ +- [:simple-dask: **Dask Migration Guide**](migration/dask_migration.md) + +
+ +**Try your hand at some [Tutorials](resources/tutorials.md):** + +
+ +- [:material-image-edit: **MNIST Digit Classification**](https://colab.research.google.com/github/Eventual-Inc/Daft/blob/main/tutorials/mnist.ipynb) +- [:octicons-search-16: **Running LLMs on the Red Pajamas Dataset**](https://colab.research.google.com/github/Eventual-Inc/Daft/blob/main/tutorials/embeddings/daft_tutorial_embeddings_stackexchange.ipynb) +- [:material-image-search: **Querying Images with UDFs**](https://colab.research.google.com/github/Eventual-Inc/Daft/blob/main/tutorials/image_querying/top_n_red_color.ipynb) +- [:material-image-sync: **Image Generation on GPUs**](https://colab.research.google.com/github/Eventual-Inc/Daft/blob/main/tutorials/text_to_image/text_to_image_generation.ipynb) + +
diff --git a/docs-v2/resources/architecture.md b/docs-v2/resources/architecture.md new file mode 100644 index 0000000000..a6c9db7324 --- /dev/null +++ b/docs-v2/resources/architecture.md @@ -0,0 +1,94 @@ +# Architecture + +!!! failure "todo(docs): Add information about where Daft fits into the ecosystem or architecture of a system" + + +## High Level Overview + +![Architecture diagram for the Daft library spanning the User API, Planning, Scheduling and Execution layers](../img/architecture.png) + +### 1. User API + +!!! info "The user-facing API of Daft" + +a. **DataFrame:** A tabular (rows and columns) Python interface to a distributed table of data. It supports common tabular operations such as filters, joins, column projections and (grouped) aggregations. + +b. **Expressions:** A tree data-structure expressing the computation that produces a column of data in a DataFrame. Expressions are built in Rust, but expose a Python API for users to access them from Python. + +### 2. Planning + +!!! info "Usersโ€™ function calls on the User API layer are collected into the Planning layer, which is responsible for optimizing the plan and serializing it into a PhysicalPlan for the Scheduling layer." + +a. **LogicalPlan:** When a user calls methods on a DataFrame, these operations are enqueued in a LogicalPlan for delayed execution. + +b. **Optimizer:** The Optimizer performs optimizations on LogicalPlans such as predicate pushdowns, column pruning, limit pushdowns and more + +c. **PhysicalPlan:** The optimized LogicalPlan is then translated into a PhysicalPlan, which can be polled for tasks to be executed along with other metadata such as resource requirements + +### 3. Scheduling + +!!! info "The scheduling layer is where Daft schedules tasks produced by the PhysicalPlan to be run on the requested backend" + +a. **Runner:** The Runner consumes tasks produced by the PhysicalPlan. It is responsible for scheduling work on its backend (e.g. local threads or on Ray) and maintaining global distributed state. + +### 4. Execution + +!!! info "The Execution layer comprises the data-structures that are the actual in-memory representation of the data, and all of the kernels that run on these data-structures." + +a. **Table:** Tables are local data-structures with rows/columns built in Rust. It is a high-performance tabular abstraction for fast local execution of work on each partition of data. Tables expose a Python API that is used in the PhysicalPlans. + +b. **Series:** Each column in a Table is a Series. Series expose methods which invoke high-performance kernels for manipulation of a column of data. Series are implemented in Rust and expose a Python API only for testing purposes. + +## Execution Model + +Daft DataFrames are lazy. When operations are called on the DataFrame, their actual execution is delayed. These operations are โ€œenqueuedโ€ for execution in a LogicalPlan, which is a tree datastructure which describes the operations that will need to be performed to produce the requested DataFrame. + +=== "๐Ÿ Python" + + ```python + df = daft.read_csv("s3://foo/*.csv") + df = df.where(df["baz"] > 0) + ``` + +When the Dataframe is executed, a few things will happen: + + +1. The LogicalPlan is optimized by a query optimizer +2. The optimized LogicalPlan is translated into a PhysicalPlan +3. The Runner executes the PhysicalPlan by pulling tasks from it + +![Diagram for the execution model of Daft across the LogicalPlan, Optimizer and PhysicalPlan](../img/execution_model.png) + +These modules can also be understood as: + +1. **LogicalPlan:** what to run +2. **PhysicalPlan:** how to run it +3. **Runner:** when and where to run it + +By default, Daft runs on the PyRunner which uses Python multithreading as its backend. Daft also includes other runners including the RayRunner which can run the PhysicalPlan on a distributed Ray cluster. + +## DataFrame Partitioning + +Daft DataFrames are Partitioned - meaning that under the hood they are split row-wise into Partitions of data. + +This is useful for a few reasons: + +1. **Parallelization:** each partition of data can be processed independently of other partitions, allowing parallelization of work across all available compute resources. +2. **Distributed Computing**: each partition can reside in a different machine, unlocking DataFrames that can span terabytes of data +3. **Pipelining:** different operations may require different resources (some operations can be I/O-bound, while others may be compute-bound). By chunking up the data into Partitions, Daft can effectively pipeline these operations during scheduling to maximize resource utilization. +4. **Memory pressure:** by processing one partition at a time, Daft can limit the amount of memory it needs to execute and possibly spill result partitions to disk if necessary, freeing up memory that it needs for execution. +5. **Optimizations:** by understanding the PartitionSpec (invariants around the data inside each partition), Daft can make intelligent decisions to avoid unnecessary data movement for certain operations that may otherwise require a global shuffle of data. + +Partitioning is most often inherited from the data source that Daft is reading from. For example, if read from a directory of files, each file naturally is read as a single partition. If reading from a data catalog service such as Apache Iceberg or Delta Lake, Daft will inherit the partitioning scheme as informed by these services. + +When querying a DataFrame, global operations will also require a repartitioning of the data, depending on the operation. For instance, sorting a DataFrame on [`col(x)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.col.html) will require repartitioning by range on [`col(x)`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/expression_methods/daft.col.html), so that a local sort on each partition will provide a globally sorted DataFrame. + +## In-Memory Data Representation + +![Diagram for the hierarchy of datastructures that make up Daft's in-memory representation: DataFrame, Table and Series](../img/in_memory_data_representation.png) + +Each Partition of a DataFrame is represented as a Table object, which is in turn composed of Columns which are Series objects. + +Under the hood, Table and Series are implemented in Rust on top of the Apache Arrow specification (using the Rust arrow2 library). We expose Python API bindings for Table using PyO3, which allows our PhysicalPlan to define operations that should be run on each Table. + +This architecture means that all the computationally expensive operations on Table and Series are performed in Rust, and can be heavily optimized for raw speed. Python is most useful as a user-facing API layer for ease of use and an interactive data science user experience. diff --git a/docs-v2/resources/benchmarks/tpch-1000sf.html b/docs-v2/resources/benchmarks/tpch-1000sf.html new file mode 100644 index 0000000000..0fe0e3d35f --- /dev/null +++ b/docs-v2/resources/benchmarks/tpch-1000sf.html @@ -0,0 +1,2 @@ +
+
diff --git a/docs-v2/resources/benchmarks/tpch-100sf.html b/docs-v2/resources/benchmarks/tpch-100sf.html new file mode 100644 index 0000000000..3a700f7594 --- /dev/null +++ b/docs-v2/resources/benchmarks/tpch-100sf.html @@ -0,0 +1,2 @@ +
+
diff --git a/docs-v2/resources/benchmarks/tpch-nodes-count-daft-1000-sf.html b/docs-v2/resources/benchmarks/tpch-nodes-count-daft-1000-sf.html new file mode 100644 index 0000000000..45b8406316 --- /dev/null +++ b/docs-v2/resources/benchmarks/tpch-nodes-count-daft-1000-sf.html @@ -0,0 +1,2 @@ +
+
diff --git a/docs-v2/resources/benchmarks/tpch.md b/docs-v2/resources/benchmarks/tpch.md new file mode 100644 index 0000000000..44ace03e07 --- /dev/null +++ b/docs-v2/resources/benchmarks/tpch.md @@ -0,0 +1,153 @@ +# TPC-H Benchmarks + +Here we compare Daft against some popular Distributed Dataframes such as Spark, Modin, and Dask on the TPC-H benchmark. Our goal for this benchmark is to demonstrate that Daft is able to meet the following development goals: + +1. **Solid out of the box performance:** great performance without having to tune esoteric flags or configurations specific to this workload +2. **Reliable out-of-core execution:** highly performant and reliable processing on larger-than-memory datasets, without developer intervention and Out-Of-Memory (OOM) errors +3. **Ease of use:** getting up and running should be easy on cloud infrastructure for an individual developer or in an enterprise cloud setting + +A great stress test for Daft is the [TPC-H benchmark](https://www.tpc.org/tpch/), which is a standard benchmark for analytical query engines. This benchmark helps ensure that while Daft makes it very easy to work with multimodal data, it can also do a great job at larger scales (terabytes) of more traditional tabular analytical workloads. + +## Setup + +The basic setup for our benchmarks are as follows: + +1. We run questions 1 to 10 of the TPC-H benchmarks using Daft and other commonly used Python Distributed Dataframes. +2. The data for the queries are stored and retrieved from AWS S3 as partitioned Apache Parquet files, which is typical of enterprise workloads. No on disk/in-memory caching was performed. +3. We run each framework on a cluster of AWS i3.2xlarge instances that each have: + - 8 vCPUs + - 61G of memory + - 1900G of NVMe SSD space + +The frameworks that we benchmark against are Spark, Modin, and Dask. We chose these comparable Dataframes as they are the most commonly referenced frameworks for running large scale distributed analytical queries in Python. + +For benchmarking against Spark, we use AWS EMR which is a hosted Spark service. For other benchmarks, we host our own Ray and Dask clusters on Kubernetes. Please refer to the section on our [Detailed Benchmarking Setup](#detailed-benchmarking-setup) for additional information. + +## Results + +!!! success "Highlights" + + 1. Out of all the benchmarked frameworks, **only Daft and EMR Spark are able to run terabyte scale queries reliably** on out-of-the-box configurations. + 2. **Daft is consistently much faster** (3.3x faster than EMR Spark, 7.7x faster than Dask Dataframes, and 44.4x faster than Modin). + +!!! note "Note" + + We were unable to obtain full results for Modin due to cluster OOMs, errors and timeouts (one hour limit per question attempt). Similarly, Dask was unable to provide comparable results for the terabyte scale benchmark. It is possible that these frameworks may perform and function better with additional tuning and configuration. Logs for all the runs are provided in a public AWS S3 bucket. + +### 100 Scale Factor + +First we run TPC-H 100 Scale Factor (around 100GB) benchmark on 4 i3.2xlarge worker instances. In total, these instances add up to 244GB of cluster memory which will require the Dataframe library to perform disk spilling and out-of-core processing for certain questions that have a large join or sort. + + + + +
+
+ +| Dataframe | Questions Completed | Total Time (seconds) | Relative to Daft | +| --------- | :-----------------: | :------------------: | :--------------: | +| Daft | 10/10 | 785 | 1.0x | +| Spark | 10/10 | 2648 | 3.3x | +| Dask | 10/10 | 6010 | 7.7x | +| Modin | 5/10 | Did not finish | 44.4x* | + +*\* Only for queries that completed.* + +From the results we see that Daft, Spark, and Dask are able to complete all the questions and Modin completes less than half. We also see that Daft is **3.3x** faster than Spark and **7.7x** faster than Dask including S3 IO. We expect these speed-ups to be much larger if the data is loaded in memory instead of cloud storage, which we will show in future benchmarks. + +### 1000 Scale Factor + +Next we scale up the data size by 10x while keeping the cluster size the same. Since we only have 244GB of memory and 1TB+ of tabular data, the DataFrame library will be required to perform disk spilling and out-of-core processing for all questions at nearly all stages of the query. + + +
+
+ + +| Dataframe | Questions Completed | Total Time (seconds) | Relative to Daft | +| --------- | :-----------------: | :------------------: | :--------------: | +| Daft | 10/10 | 7774 | 1.0x | +| Spark | 10/10 | 27161 | 3.5x | +| Dask | 3/10 | Did not finish | 5.8x* | +| Modin | 0/10 | Did not finish | No data | + + +*\* Only for queries that completed.* + +From the results we see that only Daft and Spark are able to complete all the questions. Dask completes less than a third and Modin is unable to complete any due to OOMs and cluster crashes. Since we can only compare to Spark here, we see that Daft is **3.5x** faster including S3 IO. This shows that Daft and Spark are the only Dataframes in this comparison capable of processing data larger than memory, with Daft standing out as the significantly faster option. + +### 1000 Scale Factor - Node Count Ablation + +Finally, we compare how Daft performs on varying size clusters on the terabyte scale dataset. We run the same Daft TPC-H questions on the same dataset as the [previous section](#1000-scale-factor) but sweep the worker node count. + + +
+
+ +We note two interesting results here: + +1. Daft can process 1TB+ of analytical data on a single 61GB instance without being distributed (16x more data than memory). +2. Daft query times scale linearly with the number of nodes (e.g. 4 nodes being 4 times faster than a single node). This allows for faster queries while maintaining the same compute cost! + +## Detailed Benchmarking Setup + +### Benchmarking Code + +Our benchmarking scripts and code can be found in the [distributed-query-benchmarks](https://github.com/Eventual-Inc/distributed-query-benchmarking) GitHub repository. + +- TPC-H queries for Daft were written by us. +- TPC-H queries for SparkSQL was adapted from [this repository](https://github.com/bodo-ai/Bodo/blob/main/benchmarks/tpch/pyspark_notebook.ipynb). +- TPC-H queries for Dask and Modin were adapted from these repositories for questions [Q1-7](https://github.com/pola-rs/tpch) and [Q8-10](https://github.com/xprobe-inc/benchmarks/tree/main/tpch). + +### Infrastructure +Our infrastructure runs on an EKS Kubernetes cluster. + + +- **Driver Instance**: i3.2xlarge +- **Worker Instance**: i3.2xlarge +- **Number of Workers**: 1/4/8 +- **Networking**: All instances colocated in the same Availability Zone in the AWS us-west-2 region + +### Data +Data for the benchmark was stored in AWS S3. +No node-level caching was performed, and data is read directly from AWS S3 on every attempt to simulate realistic workloads. + +- **Storage**: AWS S3 Bucket +- **Format**: Parquet +- **Region**: us-west-2 +- **File Layout**: Each table is split into 32 (for the 100SF benchmark) or 512 (for the 1000SF benchmark) separate Parquet files. Parquet files for a given table have their paths prefixed with that tableโ€™s name, and are laid out in a flat folder structure under that prefix. Frameworks are instructed to read Parquet files from that prefix. +- **Data Generation**: TPC-H data was generated using the utilities found in the open-sourced [Daft repository](https://github.com/Eventual-Inc/Daft/blob/main/benchmarking/tpch/pipelined_data_generation.py). This data is also available on request if you wish to reproduce any results! + +### Cluster Setup + +#### Dask and Ray + +To help us run the Distributed Dataframe libraries, we used Kubernetes for deploying Dask and Ray clusters. +The configuration files for these setups can be found in our [open source benchmarking repository](https://github.com/Eventual-Inc/distributed-query-benchmarking/tree/main/cluster_setup). + +Our benchmarks for Daft and Modin were run on a [KubeRay](https://github.com/ray-project/kuberay) cluster, and our benchmarks for Dask was run on a [Dask-on-Kubernetes](https://github.com/dask/dask-kubernetes) cluster. Both projects are owned and maintained officially by the creators of these libraries as one of the main methods of deploying. + +#### Spark + +For benchmarking Spark we used AWS EMR, the official managed Spark solution provided by AWS. For more details on our setup and approach, please consult our Spark benchmarks [README](https://github.com/Eventual-Inc/distributed-query-benchmarking/tree/main/distributed_query_benchmarking/spark_queries). + +### Logs + +| Dataframe | Scale Factor | Nodes | Links | +| --------- | ------------ | ------ | ------------------------- | +| Daft | 1000 | 8 | 1. s3://daft-public-data/benchmarking/logs/daft.0_1_3.1tb.8-i32xlarge.log | +| Daft | 1000 | 4 | 1. s3://daft-public-data/benchmarking/logs/daft.0_1_3.1tb.4-i32xlarge.log | +| Daft | 1000 | 1 | 1. s3://daft-public-data/benchmarking/logs/daft.1tb.1.i3-2xlarge.part1.log
2. s3://daft-public-data/benchmarking/logs/daft.1tb.1.i3-2xlarge.part2.log | +| Daft | 100 | 4 | 1. s3://daft-public-data/benchmarking/logs/daft.0_1_3.100gb.4-i32xlarge.log +| Spark | 1000 | 4 | 1. s3://daft-public-data/benchmarking/logs/emr-spark.6_10_0.1tb.4-i32xlarge.log +| Spark | 100 | 4 | 1. s3://daft-public-data/benchmarking/logs/emr-spark.6_10_0.100gb.4-i32xlarge.log.gz +|Dask (failed, multiple retries) | 1000 | 16 | 1. s3://daft-public-data/benchmarking/logs/dask.2023_5_0.1tb.16-i32xlarge.0.log
2. s3://daft-public-data/benchmarking/logs/dask.2023_5_0.1tb.16-i32xlarge.1.log
3. s3://daft-public-data/benchmarking/logs/dask.2023_5_0.1tb.16-i32xlarge.2.log
4. s3://daft-public-data/benchmarking/logs/dask.2023_5_0.1tb.16-i32xlarge.3.log | +| Dask (failed, multiple retries)| 1000 | 4 | 1. s3://daft-public-data/benchmarking/logs/dask.2023_5_0.1tb.4-i32xlarge.q126.log | +| Dask (multiple retries) | 100 | 4 | 1. s3://daft-public-data/benchmarking/logs/dask.2023_5_0.100gb.4-i32xlarge.0.log
2. s3://daft-public-data/benchmarking/logs/dask.2023_5_0.100gb.4-i32xlarge.0.log
3. s3://daft-public-data/benchmarking/logs/dask.2023_5_0.100gb.4-i32xlarge.1.log | +| Modin (failed, multiple retries) | 1000 | 16 | 1. s3://daft-public-data/benchmarking/logs/modin.0_20_1.1tb.16-i32xlarge.0.log
2. s3://daft-public-data/benchmarking/logs/modin.0_20_1.1tb.16-i32xlarge.1.log | +| Modin (failed, multiple retries) | 100 | 4 | 1. s3://daft-public-data/benchmarking/logs/modin.0_20_1.100gb.4-i32xlarge.log | diff --git a/docs-v2/resources/dataframe_comparison.md b/docs-v2/resources/dataframe_comparison.md new file mode 100644 index 0000000000..3e900d5822 --- /dev/null +++ b/docs-v2/resources/dataframe_comparison.md @@ -0,0 +1,76 @@ +# Dataframe Comparison + +A Dataframe can be thought of conceptually as a "table of data", with rows and columns. If you are familiar with Pandas or Spark Dataframes, you will be right at home with Daft! Dataframes are used for: + +* Interactive Data Science: Performing interactive and ad-hoc exploration of data in a Notebook environment +* Extract/Transform/Load (ETL): Defining data pipelines that clean and process data for consumption by other users +* Data Analytics: Analyzing data by producing summaries and reports + +Daft Dataframe focuses on Machine Learning/Deep Learning workloads that often involve Complex media data (images, video, audio, text documents and more). + +Below we discuss some other Dataframe libraries and compare them to Daft. + + + +| Dataframe | Query Optimizer | Multimodal | Distributed | Arrow Backed | Vectorized Execution Engine | Out-of-Core | +| -----------------------------------------------| :--------------:| :--------: | :---------: | :----------: | :-------------------------: | :---------: | +| Daft | โœ… | โœ… | โœ… | โœ… | โœ… | โœ… | +| [Pandas](https://github.com/pandas-dev/pandas) | โŒ | Python object | โŒ | optional >= 2.0 | some (Numpy) | โŒ | +| [Polars](https://github.com/pola-rs/polars) | โœ… | Python object | โŒ | โœ… | โœ… | โœ… | +| [Modin](https://github.com/modin-project/modin)| Eagar | Python object | โœ… | โŒ | some (Pandas) | โœ… | +| [PySpark](https://github.com/apache/spark) | โœ… | โŒ | โœ… | Pandas UDF/IO | Pandas UDF | โœ… | +| [Dask](https://github.com/dask/dask) | โŒ | Python object| โœ… | โŒ | some (Pandas) | โœ… | + +## Pandas/Modin + +The main drawback of using Pandas is scalability. Pandas is single-threaded and not built for distributed computing. While this is not as much of a problem for purely tabular datasets, when dealing with data such as images/video your data can get very large and expensive to compute very quickly. + +Modin is a project that provides "distributed Pandas". If the use-case is tabular, has code that is already written in Pandas but just needs to be scaled up to larger data, Modin may be a good choice. Modin aims to be 100% Pandas API compatible which means that certain operations that are important for performance in the world of multimodal data such as requesting for certain amount of resources (e.g. GPUs) is not yet possible. + +## Spark Dataframes + +Spark Dataframes are the modern enterprise de-facto solution for many ETL (Extract-Load-Transform) jobs. Originally written for Scala, a Python wrapper library called PySpark exists for Python compatibility which allows for some flexibility in leveraging the Python ecosystem for data processing. + +Spark excels at large scale tabular analytics, with support for running Python code using [Pandas UDFs](https://www.databricks.com/blog/2017/10/30/introducing-vectorized-udfs-for-pyspark.html), but suffer from a few key issues. + +* **Serialization overhead:** Spark itself is run on the JVM, and the PySpark wrapper runs a Python subprocess to execute Python code. This means means that running Python code always involves copying data back and forth between the Python and the Spark process. This is somewhat alleviated with [Arrow as the intermediate serialization format](https://spark.apache.org/docs/latest/api/python/user_guide/sql/arrow_pandas.html) but generally without the correct configurations and expert tuning this can be very slow. +* **Development experience:** Spark is not written for Python as a first-class citizen, which means that development velocity is often very slow when users need to run Python code for machine learning, image processing and more. +* **Typing:** Python is dynamically typed, but programming in Spark requires jumping through various hoops to be compatible with Spark's strongly typed data model. For example to [pass a 2D Numpy array between Spark functions](https://ai.plainenglish.io/large-scale-deep-learning-with-spark-an-opinionated-guide-1f2a7a948424), users have to: + + 1. Store the shape and flatten the array + 2. Tell Spark exactly what type the array is + 3. Unravel the flattened array again on the other end + +* **Debugging:** Key features such as exposing print statements or breakpoints from user-defined functions to the user are missing, which make PySpark extremely difficult to develop on. +* **Lack of granular execution control:** with heavy processing of multimodal data, users often need more control around the execution and scheduling of their work. For example, users may need to ensure that Spark runs a single executor per GPU, but Spark's programming model makes this very difficult. +* **Compatibility with downstream Machine Learning tasks:** Spark itself is not well suited for performing distributed ML training which is increasingly becoming the domain of frameworks such as Ray and Horovod. Integrating with such a solution is difficult and requires expert tuning of intermediate storage and data engineering solutions. + +## Dask Dataframes + +Dask and Daft are both DataFrame frameworks built for distributed computing. Both libraries enable you to process large, tabular datasets in parallel, either locally or on remote instances on-prem or in the cloud. + +If you are currently using Dask, you may want to consider migrating to Daft if you: + +- Are working with **multimodal data types**, such as nested JSON, tensors, Images, URLs, etc., +- Need faster computations through **query planning and optimization**, +- Are executing **machine learning workloads** at scale, +- Need deep support for **data catalogs, predicate pushdowns and metadata pruning** from Iceberg, Delta, and Hudi +- Want to benefit from **native Rust concurrency** + +You may want to stick with using Dask if you: + +- Want to only write **pandas-like syntax**, +- Need to parallelize **array-based workloads** or arbitrary **Python code that does not involve DataFrames** (with Dask Array, Dask Delayed and/or Dask Futures) + +Read more detailed comparisons in the [Dask Migration Guide](../migration/dask_migration.md). + +## Ray Datasets + +Ray Datasets make it easy to feed data really efficiently into Ray's model training and inference ecosystem. Datasets also provide basic functionality for data preprocessing such as mapping a function over each data item, filtering data etc. + +However, Ray Datasets are not a fully-fledged Dataframe abstraction (and [it is explicit in not being an ETL framework for data science](https://docs.ray.io/en/latest/data/overview.html#ray-data-overview)) which means that it lacks key features in data querying, visualization and aggregations. + +Instead, Ray Data is a perfect destination for processed data from DaFt Dataframes to be sent to with a simple [`df.to_ray_dataset()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.to_ray_dataset.html#daft.DataFrame.to_ray_dataset) call. This is useful as an entrypoint into your model training and inference ecosystem! diff --git a/docs-v2/resources/telemetry.md b/docs-v2/resources/telemetry.md new file mode 100644 index 0000000000..0113203c87 --- /dev/null +++ b/docs-v2/resources/telemetry.md @@ -0,0 +1,22 @@ +# Telemetry + +To help core developers improve Daft, we collect non-identifiable statistics on Daft usage in order to better understand how Daft is used, common bugs and performance bottlenecks. + +We take the privacy of our users extremely seriously, and telemetry in Daft is built to be: + +1. Easy to opt-out: to disable telemetry, set the following environment variable: `DAFT_ANALYTICS_ENABLED=0` +2. Non-identifiable: events are keyed by a session ID which is generated on import of Daft +3. Metadata-only: we do not collect any of our users' proprietary code or data + +We **do not** sell or buy any of the data that is collected in telemetry. + +!!! info "*Daft telemetry is enabled in versions >= v0.0.21*" + +## What data do we collect? + +To audit what data is collected, please see the implementation of `AnalyticsClient` in the `daft.analytics` module. + +In short, we collect the following: + +1. On import, we track system information such as the runner being used, version of Daft, OS, Python version, etc. +2. On calls of public methods on the DataFrame object, we track metadata about the execution: the name of the method, the walltime for execution and the class of error raised (if any). Function parameters and stacktraces are not logged, ensuring that user data remains private. diff --git a/docs-v2/resources/tutorials.md b/docs-v2/resources/tutorials.md new file mode 100644 index 0000000000..6d7592a0a5 --- /dev/null +++ b/docs-v2/resources/tutorials.md @@ -0,0 +1,45 @@ +# Tutorials + +## MNIST Digit Classification + +Load the MNIST image dataset and use a simple deep learning model to run classification on each image. Evaluate the model's performance with simple aggregations. + +[Run this tutorial on Google Colab](https://colab.research.google.com/github/Eventual-Inc/Daft/blob/main/tutorials/mnist.ipynb) + + +## Running LLMs on the Red Pajamas Dataset + +Load the Red Pajamas dataset and perform similarity search on Stack Exchange questions using language models and embeddings. + +[Run this tutorial on Google Colab](https://colab.research.google.com/github/Eventual-Inc/Daft/blob/main/tutorials/embeddings/daft_tutorial_embeddings_stackexchange.ipynb) + +## Querying Images with UDFs + +Query the Open Images dataset to retrieve the top N "reddest" images. This tutorial uses common open-source tools such as numpy and Pillow inside Daft UDFs to execute this query. + +[Run this tutorial on Google Colab](https://colab.research.google.com/github/Eventual-Inc/Daft/blob/main/tutorials/image_querying/top_n_red_color.ipynb) + +## Image Generation on GPUs + +Generate images from text prompts using a deep learning model (Mini DALL-E) and Daft UDFs. Run Daft UDFs on GPUs for more efficient resource allocation. + +[Run this tutorial on Google Colab](https://colab.research.google.com/github/Eventual-Inc/Daft/blob/main/tutorials/text_to_image/text_to_image_generation.ipynb) + + + diff --git a/docs-v2/terms.md b/docs-v2/terms.md new file mode 100644 index 0000000000..eb703b37e7 --- /dev/null +++ b/docs-v2/terms.md @@ -0,0 +1,91 @@ +# Terminology + +!!! failure "todo(docs): Should the terms below include a link to its respective section under "Core Concepts"? (Except Query Plan doesn't have a section)" + +Daft is a distributed data engine. The main abstraction in Daft is the [`DataFrame`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.html#daft.DataFrame), which conceptually can be thought of as a "table" of data with rows and columns. + +Daft also exposes a [`SQL`](https://www.getdaft.io/projects/docs/en/stable/user_guide/sql.html) interface which interoperates closely with the DataFrame interface, allowing you to express data transformations and queries on your tables as SQL strings. + +![Daft python dataframes make it easy to load any data such as PDF documents, images, protobufs, csv, parquet and audio files into a table dataframe structure for easy querying](img/daft_diagram.png) + +## DataFrames + +The [`DataFrame`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.html#daft.DataFrame) is the core concept in Daft. Think of it as a table with rows and columns, similar to a spreadsheet or a database table. It's designed to handle large amounts of data efficiently. + +Daft DataFrames are lazy. This means that calling most methods on a DataFrame will not execute that operation immediately - instead, DataFrames expose explicit methods such as [`daft.DataFrame.show`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.show.html#daft.DataFrame.show) and [`daft.DataFrame.write_parquet`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.write_parquet.html#daft.DataFrame.write_parquet) which will actually trigger computation of the DataFrame. + +## Expressions + +An [`Expression`](https://www.getdaft.io/projects/docs/en/stable/api_docs/expressions.html) is a fundamental concept in Daft that allows you to define computations on DataFrame columns. They are the building blocks for transforming and manipulating data within your DataFrame and will be your best friend if you are working with Daft primarily using the Python API. + +## Query Plan + +As mentioned earlier, Daft DataFrames are lazy. Under the hood, each DataFrame in Daft is represented by `LogicalPlan`, a plan of operations that describes how to compute that DataFrame. This plan is called the "query plan" and calling methods on the DataFrame actually adds steps to the query plan! When your DataFrame is executed, Daft will read this plan, optimize it to make it run faster and then execute it to compute the requested results. + +You can examine a logical plan using [`df.explain()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.explain.html#daft.DataFrame.explain), here's an example: + +=== "๐Ÿ Python" + + ```python + df2 = daft.read_parquet("s3://daft-public-data/tutorials/10-min/sample-data-dog-owners-partitioned.pq/**") + df2.where(df["country"] == "Canada").explain(show_all=True) + ``` + +```{title="Output"} +== Unoptimized Logical Plan == + +* Filter: col(country) == lit("Canada") +| +* GlobScanOperator +| Glob paths = [s3://daft-public-data/tutorials/10-min/sample-data-dog-owners- +| partitioned.pq/**] +| Coerce int96 timestamp unit = Nanoseconds +| IO config = S3 config = { Max connections = 8, Retry initial backoff ms = 1000, +| Connect timeout ms = 30000, Read timeout ms = 30000, Max retries = 25, Retry +| mode = adaptive, Anonymous = false, Use SSL = true, Verify SSL = true, Check +| hostname SSL = true, Requester pays = false, Force Virtual Addressing = false }, +| Azure config = { Anonymous = false, Use SSL = true }, GCS config = { Anonymous = +| false }, HTTP config = { user_agent = daft/0.0.1 } +| Use multithreading = true +| File schema = first_name#Utf8, last_name#Utf8, age#Int64, DoB#Date, +| country#Utf8, has_dog#Boolean +| Partitioning keys = [] +| Output schema = first_name#Utf8, last_name#Utf8, age#Int64, DoB#Date, +| country#Utf8, has_dog#Boolean + + +== Optimized Logical Plan == + +* GlobScanOperator +| Glob paths = [s3://daft-public-data/tutorials/10-min/sample-data-dog-owners- +| partitioned.pq/**] +| Coerce int96 timestamp unit = Nanoseconds +| IO config = S3 config = { Max connections = 8, Retry initial backoff ms = 1000, +| Connect timeout ms = 30000, Read timeout ms = 30000, Max retries = 25, Retry +| mode = adaptive, Anonymous = false, Use SSL = true, Verify SSL = true, Check +| hostname SSL = true, Requester pays = false, Force Virtual Addressing = false }, +| Azure config = { Anonymous = false, Use SSL = true }, GCS config = { Anonymous = +| false }, HTTP config = { user_agent = daft/0.0.1 } +| Use multithreading = true +| File schema = first_name#Utf8, last_name#Utf8, age#Int64, DoB#Date, +| country#Utf8, has_dog#Boolean +| Partitioning keys = [] +| Filter pushdown = col(country) == lit("Canada") +| Output schema = first_name#Utf8, last_name#Utf8, age#Int64, DoB#Date, +| country#Utf8, has_dog#Boolean + + +== Physical Plan == + +* TabularScan: +| Num Scan Tasks = 1 +| Estimated Scan Bytes = 6336 +| Clustering spec = { Num partitions = 1 } +``` + +## Structured Query Language (SQL) + +SQL is a common query language for expressing queries over tables of data. Daft exposes a SQL API as an alternative (but often also complementary API) to the Python [`DataFrame`](https://www.getdaft.io/projects/docs/en/stable/api_docs/doc_gen/dataframe_methods/daft.DataFrame.html#daft.DataFrame) and +[`Expression`](https://www.getdaft.io/projects/docs/en/stable/api_docs/expressions.html) APIs for building queries. + +You can use SQL in Daft via the [`daft.sql()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/sql.html#daft.sql) function, and Daft will also convert many SQL-compatible strings into Expressions via [`daft.sql_expr()`](https://www.getdaft.io/projects/docs/en/stable/api_docs/sql.html#daft.sql_expr) for easy interoperability with DataFrames. diff --git a/mkdocs.yml b/mkdocs.yml new file mode 100644 index 0000000000..cadf521f08 --- /dev/null +++ b/mkdocs.yml @@ -0,0 +1,141 @@ +# Documentation: https://squidfunk.github.io/mkdocs-material/ +# Example: https://github.com/squidfunk/mkdocs-material/blob/master/mkdocs.yml + +# Project Information +site_name: Daft Documentation + +docs_dir: docs-v2 + +# Repository +repo_name: Daft +repo_url: https://github.com/Eventual-Inc/Daft + +# Sitemap +nav: +- Daft User Guide: + - Overview: index.md + - Terminology: terms.md + - Installation: install.md + - Quickstart: quickstart.md + - Core Concepts: core_concepts.md + # - DataFrame: core_concepts/dataframe.md + # - Expressions: core_concepts/expressions.md + # - Reading/Writing Data: core_concepts/read_write.md + # - DataTypes: core_concepts/datatypes.md + # - SQL: core_concepts/sql.md + # - Aggregations and Grouping: core_concepts/aggregations.md + # - User-Defined Functions (UDF): core_concepts/udf.md + - Advanced: + - Managing Memory Usage: advanced/memory.md + - Partitioning: advanced/partitioning.md + - Distributed Computing: advanced/distributed.md + - Integrations: + - Ray: integrations/ray.md + - Unity Catalog: integrations/unity_catalog.md + - Apache Iceberg: integrations/iceberg.md + - Delta Lake: integrations/delta_lake.md + - Apache Hudi: integrations/hudi.md + - Microsoft Azure: integrations/azure.md + - Amazon Web Services: integrations/aws.md + - SQL: integrations/sql.md + - Hugging Face Datasets: integrations/huggingface.md + - Resources: + - Architecture: resources/architecture.md + - DataFrame Comparison: resources/dataframe_comparison.md + - Tutorials: resources/tutorials.md + - Benchmarks: resources/benchmarks/tpch.md # Benchmarks can expand into a folder once we have more + - Telemetry: resources/telemetry.md + - Migration Guide: + - Coming from Dask: migration/dask_migration.md +- API Docs + +# Configuration +theme: + name: material + locale: en + logo: img/daft.png + favicon: img/favicon.png + features: + - search.suggest + - search.highlight + - content.tabs.link # If one tab switches Python to SQL, all tabs switch + - toc.follow + - toc.integrate # adds page subsections to left-hand menu (instead of right-hand menu) + - navigation.tracking # url in address bar is automatically updated with active anchor as highlighted in TOC + - navigation.tabs # moves nav bar to top, with secondary left-hand menu + - navigation.tabs.sticky # top nav bar remains visible when scrolling down + # - navigation.sections # fixed sections + # - navigation.expand # drop down sections + # - navigation.path # breadcrumb navigation is rendered above title of each page + - navigation.top # back-to-top button + # - navigation.indexes # section header can also be a doc BUT! clashes with toc.integrate for the top-level page + +# Issue: +# If we enable toc.integrate and navigation.indexes together, it doesn't show subsections for the top-level page that is a section header, but works fine for all lower pages +# If we only enable toc.integrate, section headers are now plain text +# If we only enable navigation.indexes, page subsections get moved back to right-hand menu + + palette: + - media: "(prefers-color-scheme)" + toggle: + icon: material/white-balance-sunny + name: Switch to light mode + - media: "(prefers-color-scheme: light)" + scheme: default + primary: indigo + accent: indigo + toggle: + icon: material/weather-night + name: Switch to dark mode + - media: "(prefers-color-scheme: dark)" + scheme: slate + primary: black + accent: indigo + toggle: + icon: material/theme-light-dark + name: Switch to system preference + icon: + admonition: + quote: octicons/question-16 + +# Additional Configuration +extra: + social: + - icon: fontawesome/brands/github + link: https://github.com/squidfunk + - icon: fontawesome/brands/slack + link: https://join.slack.com/t/dist-data/shared_invite/zt-2e77olvxw-uyZcPPV1SRchhi8ah6ZCtg + - icon: fontawesome/brands/linkedin + link: https://www.linkedin.com/company/eventualcomputing/ + - icon: fontawesome/brands/x-twitter + link: https://x.com/daft_dataframe + +# Extensions +markdown_extensions: +- admonition +- toc: + toc_depth: 3 + permalink: "#" +- pymdownx.tabbed: + alternate_style: true +- pymdownx.snippets: + check_paths: true + # base_path: ['docs/source', "."] + # dedent_subsections: true +- pymdownx.superfences +- pymdownx.details + # - pymdownx.blocks.admonition +- attr_list +- md_in_html +- pymdownx.emoji: + emoji_index: !!python/name:material.extensions.emoji.twemoji + emoji_generator: !!python/name:material.extensions.emoji.to_svg + +# Plugins +plugins: +- markdown-exec +- mkdocs-jupyter: + execute: false + include_source: true +- search: + separator: '[\s\u200b\-_,:!=\[\]()"`/]+|\.(?!\d)|&[lg]t;|(?!\b)(?=[A-Z][a-z])' From ea8f8bdc7e6dfa1f07b77451a9bd689b0f50f9fa Mon Sep 17 00:00:00 2001 From: Raunak Bhagat Date: Thu, 19 Dec 2024 01:34:03 -0800 Subject: [PATCH 09/11] ci: Add build step to run-cluster (#3606) # Overview The `run-cluster` workflow now also builds the commit by calling the `build-commit` workflow first. --- .github/workflows/build-commit.yaml | 24 ++++++++++++++++++++++++ .github/workflows/run-cluster.yaml | 17 +++++++++++++++-- 2 files changed, 39 insertions(+), 2 deletions(-) diff --git a/.github/workflows/build-commit.yaml b/.github/workflows/build-commit.yaml index a8a77417ef..794f94ddac 100644 --- a/.github/workflows/build-commit.yaml +++ b/.github/workflows/build-commit.yaml @@ -1,6 +1,24 @@ name: build-commit on: + workflow_call: + inputs: + arch: + description: The machine architecture to build for + required: true + type: string + python_version: + description: The version of Python to use + required: true + type: string + secrets: + ACTIONS_AWS_ROLE_ARN: + description: The AWS Role ARN for secrets access + required: true + outputs: + wheel_url: + description: The name of the built wheel + value: ${{ jobs.build-commit.outputs.wheel_url }} workflow_dispatch: inputs: arch: @@ -24,6 +42,8 @@ jobs: permissions: id-token: write contents: read + outputs: + wheel_url: ${{ steps.set-wheel-name.outputs.wheel_url }} steps: - name: Set platform substring run: | @@ -96,6 +116,7 @@ jobs: ) echo "wheel_name=$wheel_name" >> $GITHUB_ENV - name: Print url of the built wheel to GitHub Actions Summary Page + id: set-wheel-name run: | console_url="https://us-west-2.console.aws.amazon.com/s3/object/github-actions-artifacts-bucket?prefix=builds/${{ github.sha }}/$wheel_name" download_url="https://github-actions-artifacts-bucket.s3.us-west-2.amazonaws.com/builds/${{ github.sha }}/$wheel_name" @@ -105,3 +126,6 @@ jobs: echo "Directly download the wheel here:" >> $GITHUB_STEP_SUMMARY echo "$download_url" >> $GITHUB_STEP_SUMMARY + echo "wheel_url=$download_url" >> $GITHUB_OUTPUT + env: + wheel_url: ${{ env.wheel_url }} diff --git a/.github/workflows/run-cluster.yaml b/.github/workflows/run-cluster.yaml index 2a6a2a531d..f5c41aaf75 100644 --- a/.github/workflows/run-cluster.yaml +++ b/.github/workflows/run-cluster.yaml @@ -45,12 +45,25 @@ on: default: "" jobs: + build-commit: + uses: ./.github/workflows/build-commit.yaml + if: ${{ inputs.daft_version == '' && inputs.daft_wheel_url == '' }} + with: + arch: x86 + python_version: ${{ inputs.python_version }} + secrets: + ACTIONS_AWS_ROLE_ARN: ${{ secrets.ACTIONS_AWS_ROLE_ARN }} + run-command: runs-on: [self-hosted, linux, x64, ci-dev] - timeout-minutes: 15 # Remove for ssh debugging + # If both the `daft-version` and `daft-wheel-url` parameters are not specified, the `build-commit` job is entirely skipped. + # We still want to run this job, even if `build-commit` is skipped. + # The `always()` guarantees that this job is always run. + if: always() permissions: id-token: write contents: read + needs: build-commit steps: - name: Log workflow inputs run: echo "${{ toJson(github.event.inputs) }}" @@ -80,7 +93,7 @@ jobs: --python 3.12 \ .github/ci-scripts/templatize_ray_config.py \ --cluster-name="ray-ci-run-${{ github.run_id }}_${{ github.run_attempt }}" \ - --daft-wheel-url='${{ inputs.daft_wheel_url }}' \ + --daft-wheel-url='${{ needs.build-commit.outputs.wheel_url || inputs.daft_wheel_url || '' }}' \ --daft-version='${{ inputs.daft_version }}' \ --python-version='${{ inputs.python_version }}' \ --cluster-profile='${{ inputs.cluster_profile }}' \ From ae74c10e74ff6568590d53d02a31046d111eb6f9 Mon Sep 17 00:00:00 2001 From: Andrew Gazelka Date: Thu, 19 Dec 2024 02:44:47 -0800 Subject: [PATCH 10/11] =?UTF-8?q?feat(connect):=20read/write=20=E2=86=92?= =?UTF-8?q?=20csv,=20write=20=E2=86=92=20json=20(#3361)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- src/daft-connect/src/op/execute/write.rs | 6 +- .../logical_plan/read/data_source.rs | 29 +++--- tests/connect/test_csv.py | 88 +++++++++++++++++++ 3 files changed, 109 insertions(+), 14 deletions(-) create mode 100644 tests/connect/test_csv.py diff --git a/src/daft-connect/src/op/execute/write.rs b/src/daft-connect/src/op/execute/write.rs index 5db783f5e1..da06f04887 100644 --- a/src/daft-connect/src/op/execute/write.rs +++ b/src/daft-connect/src/op/execute/write.rs @@ -55,9 +55,7 @@ impl Session { bail!("Source is required"); }; - if source != "parquet" { - bail!("Unsupported source: {source}; only parquet is supported"); - } + let file_format: FileFormat = source.parse()?; let Ok(mode) = SaveMode::try_from(mode) else { bail!("Invalid save mode: {mode}"); @@ -115,7 +113,7 @@ impl Session { let plan = translator.to_logical_plan(input).await?; let plan = plan - .table_write(&path, FileFormat::Parquet, None, None, None) + .table_write(&path, file_format, None, None, None) .wrap_err("Failed to create table write plan")?; let optimized_plan = plan.optimize()?; diff --git a/src/daft-connect/src/translation/logical_plan/read/data_source.rs b/src/daft-connect/src/translation/logical_plan/read/data_source.rs index 90164dd0bd..863b5e8f1d 100644 --- a/src/daft-connect/src/translation/logical_plan/read/data_source.rs +++ b/src/daft-connect/src/translation/logical_plan/read/data_source.rs @@ -1,5 +1,5 @@ use daft_logical_plan::LogicalPlanBuilder; -use daft_scan::builder::ParquetScanBuilder; +use daft_scan::builder::{CsvScanBuilder, ParquetScanBuilder}; use eyre::{bail, ensure, WrapErr}; use tracing::warn; @@ -18,10 +18,6 @@ pub async fn data_source( bail!("Format is required"); }; - if format != "parquet" { - bail!("Unsupported format: {format}; only parquet is supported"); - } - ensure!(!paths.is_empty(), "Paths are required"); if let Some(schema) = schema { @@ -36,10 +32,23 @@ pub async fn data_source( warn!("Ignoring predicates: {predicates:?}; not yet implemented"); } - let builder = ParquetScanBuilder::new(paths) - .finish() - .await - .wrap_err("Failed to create parquet scan builder")?; + let plan = match &*format { + "parquet" => ParquetScanBuilder::new(paths) + .finish() + .await + .wrap_err("Failed to create parquet scan builder")?, + "csv" => CsvScanBuilder::new(paths) + .finish() + .await + .wrap_err("Failed to create csv scan builder")?, + "json" => { + // todo(completeness): implement json reading + bail!("json reading is not yet implemented"); + } + other => { + bail!("Unsupported format: {other}; only parquet and csv are supported"); + } + }; - Ok(builder) + Ok(plan) } diff --git a/tests/connect/test_csv.py b/tests/connect/test_csv.py new file mode 100644 index 0000000000..7e957dd394 --- /dev/null +++ b/tests/connect/test_csv.py @@ -0,0 +1,88 @@ +from __future__ import annotations + +import os + +import pytest + + +def test_write_csv_basic(spark_session, tmp_path): + df = spark_session.range(10) + csv_dir = os.path.join(tmp_path, "csv") + df.write.csv(csv_dir) + + csv_files = [f for f in os.listdir(csv_dir) if f.endswith(".csv")] + assert len(csv_files) > 0, "Expected at least one CSV file to be written" + + df_read = spark_session.read.csv(str(csv_dir)) + df_pandas = df.toPandas() + df_read_pandas = df_read.toPandas() + assert df_pandas["id"].equals(df_read_pandas["id"]), "Data should be unchanged after write/read" + + +def test_write_csv_with_header(spark_session, tmp_path): + df = spark_session.range(10) + csv_dir = os.path.join(tmp_path, "csv") + df.write.option("header", True).csv(csv_dir) + + df_read = spark_session.read.option("header", True).csv(str(csv_dir)) + df_pandas = df.toPandas() + df_read_pandas = df_read.toPandas() + assert df_pandas["id"].equals(df_read_pandas["id"]) + + +def test_write_csv_with_delimiter(spark_session, tmp_path): + df = spark_session.range(10) + csv_dir = os.path.join(tmp_path, "csv") + df.write.option("sep", "|").csv(csv_dir) + + df_read = spark_session.read.option("sep", "|").csv(str(csv_dir)) + df_pandas = df.toPandas() + df_read_pandas = df_read.toPandas() + assert df_pandas["id"].equals(df_read_pandas["id"]) + + +def test_write_csv_with_quote(spark_session, tmp_path): + df = spark_session.createDataFrame([("a,b",), ("c'd",)], ["text"]) + csv_dir = os.path.join(tmp_path, "csv") + df.write.option("quote", "'").csv(csv_dir) + + df_read = spark_session.read.option("quote", "'").csv(str(csv_dir)) + df_pandas = df.toPandas() + df_read_pandas = df_read.toPandas() + assert df_pandas["text"].equals(df_read_pandas["text"]) + + +def test_write_csv_with_escape(spark_session, tmp_path): + df = spark_session.createDataFrame([("a'b",), ("c'd",)], ["text"]) + csv_dir = os.path.join(tmp_path, "csv") + df.write.option("escape", "\\").csv(csv_dir) + + df_read = spark_session.read.option("escape", "\\").csv(str(csv_dir)) + df_pandas = df.toPandas() + df_read_pandas = df_read.toPandas() + assert df_pandas["text"].equals(df_read_pandas["text"]) + + +@pytest.mark.skip( + reason="https://github.com/Eventual-Inc/Daft/issues/3609: CSV null value handling not yet implemented" +) +def test_write_csv_with_null_value(spark_session, tmp_path): + df = spark_session.createDataFrame([(1, None), (2, "test")], ["id", "value"]) + csv_dir = os.path.join(tmp_path, "csv") + df.write.option("nullValue", "NULL").csv(csv_dir) + + df_read = spark_session.read.option("nullValue", "NULL").csv(str(csv_dir)) + df_pandas = df.toPandas() + df_read_pandas = df_read.toPandas() + assert df_pandas["value"].isna().equals(df_read_pandas["value"].isna()) + + +def test_write_csv_with_compression(spark_session, tmp_path): + df = spark_session.range(10) + csv_dir = os.path.join(tmp_path, "csv") + df.write.option("compression", "gzip").csv(csv_dir) + + df_read = spark_session.read.csv(str(csv_dir)) + df_pandas = df.toPandas() + df_read_pandas = df_read.toPandas() + assert df_pandas["id"].equals(df_read_pandas["id"]) From 33f95c134a2cce7fb325884cca8cdea0ab7dfb77 Mon Sep 17 00:00:00 2001 From: Andrew Gazelka Date: Thu, 19 Dec 2024 09:44:41 -0800 Subject: [PATCH 11/11] test(connect): verify `show()`output (#3610) --- tests/connect/test_show.py | 36 +++++++++++++++++++++++++++++++----- 1 file changed, 31 insertions(+), 5 deletions(-) diff --git a/tests/connect/test_show.py b/tests/connect/test_show.py index a463d5de72..3d4234d70b 100644 --- a/tests/connect/test_show.py +++ b/tests/connect/test_show.py @@ -1,9 +1,35 @@ from __future__ import annotations -def test_show(spark_session): +def test_show(spark_session, capsys): df = spark_session.range(10) - try: - df.show() - except Exception as e: - assert False, e + df.show() + captured = capsys.readouterr() + expected = ( + "โ•ญโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฎ\n" + "โ”‚ id โ”‚\n" + "โ”‚ --- โ”‚\n" + "โ”‚ Int64 โ”‚\n" + "โ•žโ•โ•โ•โ•โ•โ•โ•โ•ก\n" + "โ”‚ 0 โ”‚\n" + "โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค\n" + "โ”‚ 1 โ”‚\n" + "โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค\n" + "โ”‚ 2 โ”‚\n" + "โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค\n" + "โ”‚ 3 โ”‚\n" + "โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค\n" + "โ”‚ 4 โ”‚\n" + "โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค\n" + "โ”‚ 5 โ”‚\n" + "โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค\n" + "โ”‚ 6 โ”‚\n" + "โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค\n" + "โ”‚ 7 โ”‚\n" + "โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค\n" + "โ”‚ 8 โ”‚\n" + "โ”œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ•Œโ”ค\n" + "โ”‚ 9 โ”‚\n" + "โ•ฐโ”€โ”€โ”€โ”€โ”€โ”€โ”€โ•ฏ\n" + ) + assert captured.out == expected
\n", + "

Note

\n", + "

\n", + " See Integrations to learn more about working with other formats like Delta Lake and Iceberg.\n", + "

\n", + "