From a0cae7c58be1210222d519b1947fcff68006dacb Mon Sep 17 00:00:00 2001 From: elijahbenizzy Date: Thu, 3 Aug 2023 06:16:25 -0700 Subject: [PATCH 01/11] Implements spark integration, see #248 See issue for more design. Basic overview is: 1. We use `with_columns` to group together map transforms 2. Map transforms can be UDFs (normal) or pandas UDFs 3. THese all get run and "linearized" -- this means that we have two sets of edges: -> edges that form the "physical" dependency -- the dataframe geting passed through and consistently appended to -> edges that represent logical dependencies -- these are the original edges in the with_columns group` While this muddles the edges, this allows us to visualize both the structure and execution of the DAG. We will likely be adding metadata to edges to help with visualization. --- .circleci/config.yml | 12 +- graph_adapter_tests/h_spark/test_h_spark.py | 365 ++++++++++++--- hamilton/execution/graph_functions.py | 30 +- hamilton/experimental/h_spark.py | 417 ++++++++++++++++-- hamilton/function_modifiers/recursive.py | 29 +- hamilton/node.py | 22 +- tests/resources/spark/__init__.py | 0 tests/resources/spark/basic_spark_dag.py | 59 +++ tests/resources/{ => spark}/pyspark_udfs.py | 0 .../spark/spark_dag_external_dependencies.py | 45 ++ .../spark/spark_dag_multiple_with_columns.py | 115 +++++ 11 files changed, 973 insertions(+), 121 deletions(-) create mode 100644 tests/resources/spark/__init__.py create mode 100644 tests/resources/spark/basic_spark_dag.py rename tests/resources/{ => spark}/pyspark_udfs.py (100%) create mode 100644 tests/resources/spark/spark_dag_external_dependencies.py create mode 100644 tests/resources/spark/spark_dag_multiple_with_columns.py diff --git a/.circleci/config.yml b/.circleci/config.yml index 46f6b4e07..f573dbbcc 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -55,8 +55,16 @@ workflows: python-version: '3.9' task: ray - test: - name: spark-py38 - python-version: '3.8' + name: spark-py39 + python-version: '3.9' + task: pyspark + - test: + name: spark-py310 + python-version: '3.10' + task: pyspark + - test: + name: spark-py311 + python-version: '3.11' task: pyspark - test: name: integrations-py37 diff --git a/graph_adapter_tests/h_spark/test_h_spark.py b/graph_adapter_tests/h_spark/test_h_spark.py index feecf4c1a..a05368a95 100644 --- a/graph_adapter_tests/h_spark/test_h_spark.py +++ b/graph_adapter_tests/h_spark/test_h_spark.py @@ -1,22 +1,45 @@ +<<<<<<< HEAD import sys import numpy as np +======= +import logging + +>>>>>>> 7d0b719 (Implements spark integration, see #248) import pandas as pd import pyspark.pandas as ps import pytest from pyspark import Row +<<<<<<< HEAD from pyspark.sql import SparkSession, types +======= +from pyspark.sql import Column, DataFrame, SparkSession +>>>>>>> 7d0b719 (Implements spark integration, see #248) from pyspark.sql.functions import column from hamilton import base, driver, htypes, node from hamilton.experimental import h_spark +from hamilton.log_setup import setup_logging + +from .resources import example_module, smoke_screen_module +from .resources.spark import ( + basic_spark_dag, + pyspark_udfs, + spark_dag_external_dependencies, + spark_dag_multiple_with_columns, +) -from .resources import example_module, pyspark_udfs, smoke_screen_module +setup_logging(logging.DEBUG) @pytest.fixture(scope="module") def spark_session(): - spark = SparkSession.builder.getOrCreate() + spark = ( + SparkSession.builder.master("local") + .appName("spark session") + .config("spark.sql.shuffle.partitions", "1") + .getOrCreate() + ) yield spark spark.stop() @@ -89,25 +112,24 @@ def test_smoke_screen_module(spark_session): assert df["series_with_start_date_end_date"].iloc[0] == "date_20200101_date_20220801" -spark = SparkSession.builder.master("local[1]").getOrCreate() - -pandas_df = pd.DataFrame({"spend": [10, 10, 20, 40, 40, 50], "signups": [1, 10, 50, 100, 200, 400]}) -spark_df = spark.createDataFrame(pandas_df) - - @pytest.mark.parametrize( - "input, expected", + "input_and_expected_fn", [ - ({}, (None, {})), - ({"a": 1}, (None, {"a": 1})), - ({"a": spark_df}, (spark_df, {})), - ({"a": spark_df, "b": 1}, (spark_df, {"b": 1})), + (lambda df: ({}, (None, {}))), + (lambda df: ({"a": 1}, (None, {"a": 1}))), + (lambda df: ({"a": df}, (df, {}))), + (lambda df: ({"a": df, "b": 1}, (df, {"b": 1}))), ], ids=["no_kwargs", "one_plain_kwarg", "one_df_kwarg", "one_df_kwarg_and_one_plain_kwarg"], ) -def test__inspect_kwargs(input, expected): +def test__inspect_kwargs(input_and_expected_fn, spark_session): """A unit test for inspect_kwargs.""" - assert h_spark._inspect_kwargs(input) == expected + pandas_df = pd.DataFrame( + {"spend": [10, 10, 20, 40, 40, 50], "signups": [1, 10, 50, 100, 200, 400]} + ) + df = spark_session.createDataFrame(pandas_df) + input_, expected = input_and_expected_fn(df) + assert h_spark._inspect_kwargs(input_) == expected def test__get_pandas_annotations(): @@ -119,16 +141,24 @@ def no_pandas(a: int, b: float) -> float: def with_pandas(a: pd.Series) -> pd.Series: return a * 2 - def with_pandas_and_other_default(a: pd.Series, b: int = 2) -> pd.Series: + def with_pandas_and_other_default(a: pd.Series, b: int) -> pd.Series: return a * b + # def with_pandas_and_other_default_with_one_more(a: pd.Series, c: int, b: int = 2) -> pd.Series: - return a * b + return a * b * c - assert h_spark._get_pandas_annotations(no_pandas) == {"a": False, "b": False} - assert h_spark._get_pandas_annotations(with_pandas) == {"a": True} - assert h_spark._get_pandas_annotations(with_pandas_and_other_default) == {"a": True} - assert h_spark._get_pandas_annotations(with_pandas_and_other_default_with_one_more) == { + assert h_spark._get_pandas_annotations(node.Node.from_fn(no_pandas), {}) == { + "a": False, + "b": False, + } + assert h_spark._get_pandas_annotations(node.Node.from_fn(with_pandas), {}) == {"a": True} + assert h_spark._get_pandas_annotations( + node.Node.from_fn(with_pandas_and_other_default), {"b": 2} + ) == {"a": True} + assert h_spark._get_pandas_annotations( + node.Node.from_fn(with_pandas_and_other_default_with_one_more), {"b": 2} + ) == { "a": True, "c": False, } @@ -136,21 +166,14 @@ def with_pandas_and_other_default_with_one_more(a: pd.Series, c: int, b: int = 2 def test__bind_parameters_to_callable(): """Unit test for _bind_parameters_to_callable().""" - - def base_func(a: int, b: int) -> int: - return a + b - actual_kwargs = {"a": 1, "b": 2} df_columns = {"b"} node_input_types = {"a": (int,), "b": (int,)} - mod_func, df_params = h_spark._bind_parameters_to_callable( - actual_kwargs, df_columns, base_func, node_input_types, "test" + df_params, params_to_bind = h_spark._determine_parameters_to_bind( + actual_kwargs, df_columns, node_input_types, "test" ) - import inspect - - sig = inspect.signature(mod_func) - assert sig.parameters["a"].default == 1 - assert sig.parameters["b"].default == inspect.Parameter.empty + assert isinstance(df_params["b"], Column) + assert params_to_bind == {"a": 1} assert str(df_params["b"]) == str(column("b")) # hacky, but compare string representation. @@ -161,18 +184,8 @@ def base_func(a: int, b: int) -> int: return a + b base_spark_df = spark_session.createDataFrame(pd.DataFrame({"a": [1, 2, 3], "b": [4, 5, 6]})) - node_ = node.Node( - "test", - int, - "", - base_func, - input_types={ - "a": (int, node.DependencyType.REQUIRED), - "b": (int, node.DependencyType.REQUIRED), - }, - ) - - new_df = h_spark._lambda_udf(base_spark_df, node_, base_func, {}) + node_ = node.Node.from_fn(base_func) + new_df = h_spark._lambda_udf(base_spark_df, node_, {}) assert new_df.collect() == [Row(a=1, b=4, test=5), Row(a=2, b=5, test=7), Row(a=3, b=6, test=9)] @@ -183,18 +196,9 @@ def base_func(a: pd.Series, b: pd.Series) -> htypes.column[pd.Series, int]: return a + b base_spark_df = spark_session.createDataFrame(pd.DataFrame({"a": [1, 2, 3], "b": [4, 5, 6]})) - node_ = node.Node( - "test", - htypes.column[pd.Series, int], - "", - base_func, - input_types={ - "a": (int, node.DependencyType.REQUIRED), - "b": (int, node.DependencyType.REQUIRED), - }, - ) + node_ = node.Node.from_fn(base_func) - new_df = h_spark._lambda_udf(base_spark_df, node_, base_func, {}) + new_df = h_spark._lambda_udf(base_spark_df, node_, {}) assert new_df.collect() == [Row(a=1, b=4, test=5), Row(a=2, b=5, test=7), Row(a=3, b=6, test=9)] @@ -205,22 +209,13 @@ def base_func(a: pd.Series, b: int) -> htypes.column[pd.Series, int]: return a + b base_spark_df = spark_session.createDataFrame(pd.DataFrame({"a": [1, 2, 3], "b": [4, 5, 6]})) - node_ = node.Node( - "test", - htypes.column[pd.Series, int], - "", - base_func, - input_types={ - "a": (int, node.DependencyType.REQUIRED), - "b": (int, node.DependencyType.REQUIRED), - }, - ) + node_ = node.Node.from_fn(base_func) with pytest.raises(ValueError): - h_spark._lambda_udf(base_spark_df, node_, base_func, {"a": 1}) + h_spark._lambda_udf(base_spark_df, node_, {"a": 1}) -def test_smoke_screen_udf_graph_adatper(spark_session): +def test_smoke_screen_udf_graph_adapter(spark_session): """Tests that we can run the PySparkUDFGraphAdapter on a simple graph. THe graph has a pandas UDF, a plain UDF that depends on the output of the pandas UDF, and @@ -240,6 +235,7 @@ def test_smoke_screen_udf_graph_adatper(spark_session): ] +<<<<<<< HEAD # Test cases for python_to_spark_type function @pytest.mark.parametrize( "python_type,expected_spark_type", @@ -350,3 +346,238 @@ def dummyfunc(x: int) -> int: return x return dummyfunc +======= +def test_base_spark_executor_end_to_end(spark_session): + # TODO -- make this simpler to call, and not require all these constructs + dr = ( + driver.Builder() + .with_modules(basic_spark_dag) + .with_adapter(base.SimplePythonGraphAdapter(base.DictResult())) + .build() + ) + # dr.visualize_execution( + # ["processed_df_as_pandas"], "./out", {}, inputs={"spark_session": spark_session} + # ) + df = dr.execute(["processed_df_as_pandas"], inputs={"spark_session": spark_session})[ + "processed_df_as_pandas" + ] + pd.testing.assert_series_equal( + df["a_times_key"], + pd.Series([2, 10, 24, 44, 70]), + check_dtype=False, + check_names=False, + ) + + +def test_base_spark_executor_end_to_end_external_dependencies(spark_session): + # TODO -- make this simpler to call, and not require all these constructs + dr = ( + driver.Builder() + .with_modules(spark_dag_external_dependencies) + .with_adapter(base.SimplePythonGraphAdapter(base.DictResult())) + .build() + ) + # dr.visualize_execution( + # ["processed_df_as_pandas"], "./out", {}, inputs={"spark_session": spark_session} + # ) + df = dr.execute(["processed_df_as_pandas"], inputs={"spark_session": spark_session})[ + "processed_df_as_pandas" + ] + expected_data = {"a": [2, 3, 4, 5], "b": [4, 6, 8, 10]} + expected_df = pd.DataFrame(expected_data) + pd.testing.assert_frame_equal(df, expected_df, check_names=False, check_dtype=False) + + +def test_base_spark_executor_end_to_end_multiple_with_columns(spark_session): + dr = ( + driver.Builder() + .with_modules(spark_dag_multiple_with_columns) + .with_adapter(base.SimplePythonGraphAdapter(base.DictResult())) + .build() + ) + df = dr.execute(["final"], inputs={"spark_session": spark_session})["final"].sort_index(axis=1) + + expected_df = pd.DataFrame( + { + "d_raw": [1, 4, 7, 10], + "e_raw": [2, 5, 8, 11], + "f_raw": [5, 10, 15, 20], + "d": [6, 9, 12, 15], + "f": [17.5, 35.0, 52.5, 70.0], + "e": [12.3, 18.299999, 24.299999, 30.299999], + "multiply_d_e_f_key": [1291.5, 11529.0, 45927.0, 127260.0], + "key": [1, 2, 3, 4], + "a_times_key": [2, 10, 24, 44], + "b_times_key": [5, 16, 33, 56], + "a_plus_b_plus_c": [10.5, 20.0, 29.5, 39.0], + } + ).sort_index(axis=1) + pd.testing.assert_frame_equal(df, expected_df, check_names=False, check_dtype=False) + + +def _no_pyspark_dataframe_parameter(foo: int) -> int: + ... + + +def _one_pyspark_dataframe_parameter(foo: DataFrame, bar: int) -> DataFrame: + ... + + +def _two_pyspark_dataframe_parameters(foo: DataFrame, bar: int, baz: DataFrame) -> DataFrame: + ... + + +@pytest.mark.parametrize( + "fn,requested_parameter,expected", + [ + (_one_pyspark_dataframe_parameter, "foo", "foo"), + (_one_pyspark_dataframe_parameter, None, "foo"), + (_two_pyspark_dataframe_parameters, "foo", "foo"), + (_two_pyspark_dataframe_parameters, "baz", "baz"), + ], +) +def test_derive_dataframe_parameter_succeeds(fn, requested_parameter, expected): + assert h_spark.derive_dataframe_parameter(fn, requested_parameter) == expected + + +@pytest.mark.parametrize( + "fn,requested_parameter", + [ + (_no_pyspark_dataframe_parameter, "foo"), + (_no_pyspark_dataframe_parameter, None), + (_one_pyspark_dataframe_parameter, "baz"), + (_two_pyspark_dataframe_parameters, "bar"), + (_two_pyspark_dataframe_parameters, None), + ], +) +def test_derive_dataframe_parameter_fails(fn, requested_parameter): + with pytest.raises(ValueError): + h_spark.derive_dataframe_parameter(fn, requested_parameter) + + +def test_prune_nodes_no_select(): + nodes = [ + node.Node.from_fn(fn) for fn in [basic_spark_dag.a, basic_spark_dag.b, basic_spark_dag.c] + ] + select = None + assert {n for n in h_spark.prune_nodes(nodes, select)} == set(nodes) + + +def test_prune_nodes_single_select(): + nodes = [ + node.Node.from_fn(fn) for fn in [basic_spark_dag.a, basic_spark_dag.b, basic_spark_dag.c] + ] + select = ["a", "b"] + assert {n for n in h_spark.prune_nodes(nodes, select)} == set(nodes[0:2]) + + +def test_generate_nodes_invalid_select(): + dec = h_spark.with_columns( + basic_spark_dag.a, + basic_spark_dag.b, + basic_spark_dag.c, + select=["d"], # not a node + initial_schema=["a_raw", "b_raw", "c_raw", "key"], + ) + with pytest.raises(ValueError): + + def df_as_pandas(df: DataFrame) -> pd.DataFrame: + return df.toPandas() + + dec.generate_nodes(df_as_pandas, {}) + + +def test_with_columns_generate_nodes_no_select(): + dec = h_spark.with_columns( + basic_spark_dag.a, + basic_spark_dag.b, + basic_spark_dag.c, + initial_schema=["a_raw", "b_raw", "c_raw", "key"], + ) + + def df_as_pandas(df: DataFrame) -> pd.DataFrame: + return df.toPandas() + + nodes = dec.generate_nodes(df_as_pandas, {}) + nodes_by_names = {n.name: n for n in nodes} + assert set(nodes_by_names.keys()) == { + "df_as_pandas.a", + "df_as_pandas.b", + "df_as_pandas.c", + "df_as_pandas", + } + + +def test_with_columns_generate_nodes_select(): + dec = h_spark.with_columns( + basic_spark_dag.a, + basic_spark_dag.b, + basic_spark_dag.c, + initial_schema=["a_raw", "b_raw", "c_raw", "key"], + select=["c"], + ) + + def df_as_pandas(df: DataFrame) -> pd.DataFrame: + return df.toPandas() + + nodes = dec.generate_nodes(df_as_pandas, {}) + nodes_by_names = {n.name: n for n in nodes} + assert set(nodes_by_names.keys()) == {"df_as_pandas.c", "df_as_pandas"} + + +def test_with_columns_generate_nodes_specify_namespace(): + dec = h_spark.with_columns( + basic_spark_dag.a, + basic_spark_dag.b, + basic_spark_dag.c, + initial_schema=["a_raw", "b_raw", "c_raw", "key"], + namespace="foo", + ) + + def df_as_pandas(df: DataFrame) -> pd.DataFrame: + return df.toPandas() + + nodes = dec.generate_nodes(df_as_pandas, {}) + nodes_by_names = {n.name: n for n in nodes} + assert set(nodes_by_names.keys()) == {"foo.a", "foo.b", "foo.c", "df_as_pandas"} + + +def test__format_pandas_udf(): + assert ( + h_spark._format_pandas_udf("foo", ["a", "b"]).strip() + == "def foo(a: pd.Series, b: pd.Series) -> pd.Series:\n" + " return partial_fn(a=a, b=b)" + ) + + +def test__format_standard_udf(): + assert ( + h_spark._format_udf("foo", ["b", "a"]).strip() == "def foo(b, a):\n" + " return partial_fn(b=b, a=a)" + ) + + +def test_sparkify_node(): + def foo( + a_from_upstream: pd.Series, b_from_upstream: pd.Series, c_from_df: pd.Series, d_fixed: int + ) -> htypes.column[pd.Series, int]: + return a_from_upstream + b_from_upstream + c_from_df + d_fixed + + node_ = node.Node.from_fn(foo) + sparkified = h_spark.sparkify_node( + node_, + "df_upstream", + "df_base", + {"a_from_upstream", "b_from_upstream"}, + {"c_from_df"}, + ) + # Superset of all the original nodes except the ones from the dataframe + # (as we already have that) both the physical and the logical dependencies + assert set(sparkified.input_types) == { + "a_from_upstream", + "b_from_upstream", + "d_fixed", + "df_base", + "df_upstream", + } +>>>>>>> 7d0b719 (Implements spark integration, see #248) diff --git a/hamilton/execution/graph_functions.py b/hamilton/execution/graph_functions.py index 89d9d999f..a6837c6b5 100644 --- a/hamilton/execution/graph_functions.py +++ b/hamilton/execution/graph_functions.py @@ -12,6 +12,12 @@ def topologically_sort_nodes(nodes: List[node.Node]) -> List[node.Node]: """Topologically sorts a list of nodes based on their dependencies. + Note that we bypass utilizing the preset dependencies/depended_on_by attributes of the node, + as we may want to use this before these nodes get put in a function graph. + + Thus we compute our own dependency map... + Note that this assumes that the nodes are continuous -- if there is a hidden dependency that + connects them, this has no way of knowing about it. TODO -- use python graphlib when we no longer have to support 3.7/3.8. @@ -20,24 +26,36 @@ def topologically_sort_nodes(nodes: List[node.Node]) -> List[node.Node]: :param nodes: Nodes to sort :return: Nodes in sorted order """ - - in_degrees = {node_.name: len(node_.dependencies) for node_ in nodes} + node_name_map = {node_.name: node_ for node_ in nodes} + depended_on_by_map = {} + dependency_map = {} + for node_ in nodes: + dependency_map[node_.name] = [] + for dep in node_.input_types: + # if the dependency is not here, we don't want to count it + # that means it depends on something outside the set of nodes we're sorting + if dep not in node_name_map: + continue + dependency_map[node_.name].append(dep) + if dep not in depended_on_by_map: + depended_on_by_map[dep] = [] + depended_on_by_map[dep].append(node_) + + in_degrees = {node_.name: len(dependency_map.get(node_.name, [])) for node_ in nodes} # TODO -- determine what happens if nodes have dependencies that aren't present - sources = [node_ for node_ in nodes if len(node_.dependencies) == 0] + sources = [node_ for node_ in nodes if in_degrees[node_.name] == 0] queue = [] for source in sources: queue.append(source) - sorted_nodes = [] while len(queue) > 0: node_ = queue.pop(0) sorted_nodes.append(node_) - for next_node in node_.depended_on_by: + for next_node in depended_on_by_map.get(node_.name, []): if next_node.name in in_degrees: in_degrees[next_node.name] -= 1 if in_degrees[next_node.name] == 0: queue.append(next_node) - return sorted_nodes diff --git a/hamilton/experimental/h_spark.py b/hamilton/experimental/h_spark.py index 0883fb692..863a5a3d1 100644 --- a/hamilton/experimental/h_spark.py +++ b/hamilton/experimental/h_spark.py @@ -2,7 +2,8 @@ import inspect import logging import sys -from typing import Any, Callable, Dict, List, Set, Tuple, Type, Union +from types import CodeType, FunctionType, ModuleType +from typing import Any, Callable, Dict, List, Optional, Set, Tuple, Type, Union import numpy as np import pandas as pd @@ -12,6 +13,10 @@ from hamilton import base, htypes, node from hamilton.node import DependencyType +from hamilton.execution import graph_functions +from hamilton.function_modifiers import base as fm_base +from hamilton.function_modifiers import subdag +from hamilton.function_modifiers.recursive import assign_namespace logger = logging.getLogger(__name__) @@ -206,46 +211,42 @@ def python_to_spark_type(python_type: Type[Union[int, float, bool, str, bytes]]) _list = (list[int], list[float], list[bool], list[str], list[bytes]) -def get_spark_type( - actual_kwargs: dict, df: DataFrame, hamilton_udf: Callable, return_type: Any -) -> types.DataType: + +def get_spark_type(return_type: Any) -> types.DataType: if return_type in (int, float, bool, str, bytes): return python_to_spark_type(return_type) + elif return_type in (list[int], list[float], list[bool], list[str], list[bytes]): + return types.ArrayType(python_to_spark_type(return_type.__args__[0])) elif return_type in _list: return types.ArrayType(python_to_spark_type(return_type.__args__[0])) elif hasattr(return_type, "__module__") and getattr(return_type, "__module__") == "numpy": return numpy_to_spark_type(return_type) else: - logger.debug(f"{inspect.signature(hamilton_udf)}, {actual_kwargs}, {df.columns}") raise ValueError( f"Currently unsupported return type {return_type}. " f"Please create an issue or PR to add support for this type." ) -def _get_pandas_annotations(hamilton_udf: Callable) -> Dict[str, bool]: +def _get_pandas_annotations(node_: node.Node, bound_parameters: Dict[str, Any]) -> Dict[str, bool]: """Given a function, return a dictionary of the parameters that are annotated as pandas series. :param hamilton_udf: the function to check. :return: dictionary of parameter names to boolean indicating if they are pandas series. """ - new_signature = inspect.signature(hamilton_udf) - new_sig_parameters = dict(new_signature.parameters) - pandas_annotation = { - name: param.annotation == pd.Series - for name, param in new_sig_parameters.items() - if param.default == inspect.Parameter.empty # bound parameters will have a default value. + return { + name: type_ == pd.Series + for name, (type_, _) in node_.input_types.items() + if name not in bound_parameters } - return pandas_annotation -def _bind_parameters_to_callable( +def _determine_parameters_to_bind( actual_kwargs: dict, df_columns: Set[str], - hamilton_udf: Callable, node_input_types: Dict[str, Tuple], node_name: str, -) -> Tuple[Callable, Dict[str, Any]]: +) -> Tuple[Dict[str, Any], Dict[str, Any]]: """Function that we use to bind inputs to the function, or determine we should pull them from the dataframe. It does two things: @@ -259,16 +260,15 @@ def _bind_parameters_to_callable( :param hamilton_udf: the callable to bind to. :param node_input_types: the input types of the function. :param node_name: name of the node/function. - :return: a tuple of the callable and the dictionary of parameters to use for the callable. + :return: a tuple of the params that come from the dataframe and the parameters to bind. """ params_from_df = {} + bind_parameters = {} for input_name in node_input_types.keys(): if input_name in df_columns: params_from_df[input_name] = column(input_name) elif input_name in actual_kwargs and not isinstance(actual_kwargs[input_name], DataFrame): - hamilton_udf = functools.partial( - hamilton_udf, **{input_name: actual_kwargs[input_name]} - ) + bind_parameters[input_name] = actual_kwargs[input_name] elif node_input_types[input_name][1] == DependencyType.OPTIONAL: pass else: @@ -276,7 +276,7 @@ def _bind_parameters_to_callable( f"Cannot satisfy {node_name} with input types {node_input_types} against a dataframe with " f"columns {df_columns} and input kwargs {actual_kwargs}." ) - return hamilton_udf, params_from_df + return params_from_df, bind_parameters def _inspect_kwargs(kwargs: Dict[str, Any]) -> Tuple[DataFrame, Dict[str, Any]]: @@ -296,9 +296,77 @@ def _inspect_kwargs(kwargs: Dict[str, Any]) -> Tuple[DataFrame, Dict[str, Any]]: return df, actual_kwargs -def _lambda_udf( - df: DataFrame, node_: node.Node, hamilton_udf: Callable, actual_kwargs: Dict[str, Any] -) -> DataFrame: +def _format_pandas_udf(func_name: str, ordered_params: List[str]) -> str: + formatting_params = { + "name": func_name, + "return_type": "pd.Series", + "params": ", ".join([f"{param}: pd.Series" for param in ordered_params]), + "param_call": ", ".join([f"{param}={param}" for param in ordered_params]), + } + func_string = """ +def {name}({params}) -> {return_type}: + return partial_fn({param_call}) +""".format( + **formatting_params + ) + return func_string + + +def _format_udf(func_name: str, ordered_params: List[str]) -> str: + formatting_params = { + "name": func_name, + "params": ", ".join(ordered_params), + "param_call": ", ".join([f"{param}={param}" for param in ordered_params]), + } + func_string = """ +def {name}({params}): + return partial_fn({param_call}) +""".format( + **formatting_params + ) + return func_string + + +def _fabricate_spark_function( + node_: node.Node, + params_to_bind: Dict[str, Any], + params_from_df: Dict[str, Any], + pandas_udf: bool, +) -> FunctionType: + """Fabricates a spark compatible UDF. We have to do this as we don't actually have a funtion + with annotations to use, as its lambdas passed around by decorators. We may consider pushing + this upstreams so that everything can generate its own function, but for now this is the + easiest way to do it. + + The rules are different for pandas series and regular UDFs. + Pandas series have to: + - be Decorated with pandas_udf + - Have a return type of a pandas series + - Have a pandas series as the only input types + Regular UDFs have to: + - Have no annotations at all + + See https://spark.apache.org/docs/3.1.3/api/python/reference/api/pyspark.sql.functions.udf.html + and https://spark.apache.org/docs/3.1.3/api/python/reference/api/pyspark.sql.functions.pandas_udf.html + + :param node_: Node to place in a spark function + :param params_to_bind: Parameters to bind to the function -- these won't go into the UDF + :param params_from_df: Parameters to retrieve from the dataframe + :return: A function that can be used in a spark UDF + """ + partial_fn = functools.partial(node_.callable, **params_to_bind) + ordered_params = sorted(params_from_df) + func_name = node_.name.replace(".", "_") + if pandas_udf: + func_string = _format_pandas_udf(func_name, ordered_params) + else: + func_string = _format_udf(func_name, ordered_params) + module_code = compile(func_string, "", "exec") + func_code = [c for c in module_code.co_consts if isinstance(c, CodeType)][0] + return FunctionType(func_code, {**globals(), **{"partial_fn": partial_fn}}, func_name) + + +def _lambda_udf(df: DataFrame, node_: node.Node, actual_kwargs: Dict[str, Any]) -> DataFrame: """Function to create a lambda UDF for a function. This functions does the following: @@ -314,15 +382,16 @@ def _lambda_udf( :param actual_kwargs: the actual arguments to the function. :return: the dataframe with one more column representing the result of the UDF. """ - hamilton_udf, params_from_df = _bind_parameters_to_callable( - actual_kwargs, set(df.columns), hamilton_udf, node_.input_types, node_.name + params_from_df, params_to_bind = _determine_parameters_to_bind( + actual_kwargs, set(df.columns), node_.input_types, node_.name ) - pandas_annotation = _get_pandas_annotations(hamilton_udf) + pandas_annotation = _get_pandas_annotations(node_, params_to_bind) if any(pandas_annotation.values()) and not all(pandas_annotation.values()): raise ValueError( f"Currently unsupported function for {node_.name} with function signature:\n{node_.input_types}." ) elif all(pandas_annotation.values()): + hamilton_udf = _fabricate_spark_function(node_, params_to_bind, params_from_df, True) # pull from annotation here instead of tag. base_type, type_args = htypes.get_type_information(node_.type) logger.debug("PandasUDF: %s, %s, %s", node_.name, base_type, type_args) @@ -335,16 +404,17 @@ def _lambda_udf( if isinstance(type_arg, str): spark_return_type = type_arg # spark will handle converting it. else: - spark_return_type = get_spark_type(actual_kwargs, df, hamilton_udf, type_arg) - # remove because pyspark does not like extra function annotations - hamilton_udf.__annotations__["return"] = base_type + spark_return_type = get_spark_type(type_arg) spark_udf = pandas_udf(hamilton_udf, spark_return_type) else: + hamilton_udf = _fabricate_spark_function(node_, params_to_bind, params_from_df, False) logger.debug("RegularUDF: %s, %s", node_.name, node_.type) - spark_return_type = get_spark_type(actual_kwargs, df, hamilton_udf, node_.type) + spark_return_type = get_spark_type(node_.type) spark_udf = udf(hamilton_udf, spark_return_type) return df.withColumn( - node_.name, spark_udf(*[_value for _name, _value in params_from_df.items()]) + # Sorting is a quick hack + node_.name, + spark_udf(*[_value for _name, _value in sorted(params_from_df.items())]), ) @@ -412,7 +482,7 @@ def execute_node(self, node: node.Node, kwargs: Dict[str, Any]) -> Any: logger.debug("%s, %s", self.call_count, self.df_object) logger.debug("%s, Before, %s", node.name, self.df_object.columns) schema_length = len(df.schema) - df = _lambda_udf(self.df_object, node, node.callable, actual_kwargs) + df = _lambda_udf(self.df_object, node, actual_kwargs) assert node.name in df.columns, f"Error {node.name} not in {df.columns}" delta = len(df.schema) - schema_length if delta == 0: @@ -447,3 +517,284 @@ def build_result(self, **outputs: Dict[str, Any]) -> DataFrame: self.df_object = None self.original_schema = [] return result + + +def sparkify_node( + node_: node.Node, + linear_df_dependency_name: str, + base_df_dependency_name: str, + dependent_columns_in_group: Set[str], + dependent_columns_from_dataframe: Set[str], +) -> node.Node: + """ """ + """Turns a node into a spark node. This does the following: + 1. Makes it take the prior dataframe output as a dependency, in + conjunction to its current dependencies. This is so we can represent + the "logical" plan (the UDF-dependencies) as well as + the "physical plan" (linear, df operations) + 2. Adjusts the function to apply the specified UDF on the + dataframe, ignoring all inputs in column_dependencies + (which are only there to demonstrate lineage/make the DAG representative) + 3. Returns the resulting pyspark dataframe for downstream functions to use + + + :param node_: Node we're sparkifying + :param linear_df_dependency_name: Name of the linearly passed along dataframe dependency + :param base_df_dependency_name: Name of the base (parent) dataframe dependency. + this is only used if dependent_columns_from_dataframe is not empty + :param dependent_columns_in_group: Columns on which this depends in the with_columns + :param dependent_columns_from_dataframe: Columns on which this depends in the + base (parent) dataframe that the with_columns is operating on + :return: + + """ + + def new_callable( + __linear_df_dependency_name: str = linear_df_dependency_name, + __base_df_dependency_name: str = base_df_dependency_name, + __dependent_columns_in_group: Set[str] = dependent_columns_in_group, + __dependent_columns_from_dataframe: Set[str] = dependent_columns_from_dataframe, + __node: node.Node = node_, + **kwargs, + ) -> ps.DataFrame: + """This is the new function that the node will call. + Note that this applies the hamilton UDF with *just* the input dataframe dependency, + ignoring the rest.""" + # gather the dataframe from the kwargs + df = kwargs[__linear_df_dependency_name] + kwargs = { + k: v + for k, v in kwargs.items() + if k not in __dependent_columns_from_dataframe + and k not in __dependent_columns_in_group + and k != __linear_df_dependency_name + and k != __base_df_dependency_name + } + return _lambda_udf(df, node_, kwargs) + + # Just extract the dependeency type + # TODO -- add something as a "logical" or "placeholder" dependency + new_input_types = { + # copy over the old ones + **{ + dep: value + for dep, value in node_.input_types.items() + if dep not in dependent_columns_from_dataframe + }, + # add the new one (from the previous) + linear_df_dependency_name: (DataFrame, node.DependencyType.REQUIRED), + # Then add all the others + # Note this might clobber the linear_df_dependency_name, but they'll be the same type + # If we have "logical" dependencies we'll want to be careful about the type + **{ + dep: (DataFrame, node.DependencyType.REQUIRED) + for dep, _ in node_.input_types.items() + if dep in dependent_columns_in_group + }, + } + if len(dependent_columns_from_dataframe) > 0: + new_input_types[base_df_dependency_name] = ( + DataFrame, + node.DependencyType.REQUIRED, + ) + return node_.copy_with(callabl=new_callable, input_types=new_input_types, typ=DataFrame) + + +def derive_dataframe_parameter(fn: Callable, requested_parameter: str = None) -> str: + """Utility function to grab a pyspark dataframe parameter from a function. + Note if one is supplied it'll look for that. If none is, it will look to ensure + that there is only one dataframe parameter in the function. + + :param fn: Function to grab the dataframe parameter from + :param requested_parameter: If supplied, the name of the parameter to grab + :return: The name of the dataframe parameter + :raises ValueError: If no datframe parameter is supplied: + - if no dataframe parameter is found, or if more than one is found + if a requested parameter is supplied: + - if the requested parameter is not found + """ + sig = inspect.signature(fn) + dataframe_parameters = { + param.name: param + for param in sig.parameters.values() + if issubclass(param.annotation, DataFrame) + } + if requested_parameter is not None: + if requested_parameter not in dataframe_parameters: + raise ValueError( + f"Requested parameter {requested_parameter} not found in " + f"function: {fn.__qualname__}" + ) + return requested_parameter + if len(dataframe_parameters) == 0: + raise ValueError( + f"No dataframe parameters found in function: {fn.__qualname__}. " + f"@with_columns must inject a dataframe parameter into the function." + ) + elif len(dataframe_parameters) > 1: + raise ValueError( + f"More than one dataframe parameter found in function: {fn.__qualname__}. Please " + f"specify the desired one with the 'dataframe' parameter in @with_columns" + ) + return list(dataframe_parameters)[0] + + +def prune_nodes(nodes: List[node.Node], select: Optional[List[str]] = None) -> List[node.Node]: + """Prunes the nodes to only include those upstream from the select columns. + Conducts a depth-first search using the nodes `input_types` field. + + If select is None, we just assume all nodes should be included. + + :param nodes: Full set of nodes + :param select: Columns to select + :return: Pruned set of nodes + """ + if select is None: + return nodes + + node_name_map = {node_.name: node_ for node_ in nodes} + seen_nodes = set(select) + stack = list({node_name_map[col] for col in select if col in node_name_map}) + output = [] + while len(stack) > 0: + node_ = stack.pop() + output.append(node_) + for dep in node_.input_types: + if dep not in seen_nodes and dep in node_name_map: + dep_node = node_name_map[dep] + stack.append(dep_node) + seen_nodes.add(dep) + return output + + +class with_columns(fm_base.NodeCreator): + def __init__( + self, + *load_from: Union[Callable, ModuleType], + initial_schema: List[str], + select: List[str] = None, + dataframe: str = None, + namespace: str = None, + ): + """Initializes a with_columns decorator for spark. This allows you to efficiently run + groups of map operations on a dataframe, represented as pandas/primitives UDFs. This + effectively "linearizes" compute -- meaning that a DAG of map operations can be run + as a set of .withColumn operations on a single dataframe -- ensuring that you don't have + to do a complex `extract` then `join` process on spark, which can be inefficient. + + Here's an example of calling it -- if you've seen `@subdag`, you should be familiar with + the concepts: + + .. code-block:: python + # my_module.py + def a(a_from_df: pd.Series) -> pd.Series: + return _process(a) + + def b(b_from_df: pd.Series) -> pd.Series: + return _process(b) + + def a_plus_b(a_from_df: pd.Series, b_from_df: pd.Series) -> pd.Series: + return a + b + + + # the with_columns call + @with_columns( + load_from=[my_module], # Load from any module + initial_schema=["a_from_df", "b_from_df"], # The initial schema of the dataframe + select=["a", "b", "a_plus_b"], # The columns to select from the dataframe + ) + def final_df(df: ps.DataFrame) -> ps.DataFrame: + # process, or just return unprocessed + ... + + You can think of the above as a series of withColumn calls on the dataframe, where the + operations are applied in topological order. This is significantly more efficient than + extracting out the columns, applying the maps, then operating, but *also* allows you to + express the operations individually, making it easy to unit-test and reuse. + + Note that the operation is "append", meaning that the columns that are selected are appended + onto the dataframe. We will likely add an option to have this be either "select" or "append" + mode. + + + :param load_from: The functions that will be used to generate the group of map operations. + :param select: Columns to select from the transformation. If this is left blank it will + keep all columns in + :param initial_schema: The initial schema of the dataframe. This is used to determine which + upstream inputs should be taken from the dataframe, and which shouldn't. Note that, if this is + left empty, we will assume that all upstream items come from the dataframe + :param namespace: The namespace of the nodes, so they don't clash with the global namespace + and so this can be reused. If its left out, there will be no namespace (in which case you'll want + to be careful about repeating it/reusing the nodes in other parts of the DAG.) + :param dataframe: The name of the dataframe that we're modifying. If not provided, + this will assume that there is only one pyspark.DataFrame parameter to the decorated function, + and use that if there is more than one, we will error. + """ + self.subdag_functions = subdag.collect_functions(load_from) + self.select = select + self.initial_schema = initial_schema + self.namespace = namespace + self.upstream_dependency = dataframe + + def generate_nodes(self, fn: Callable, config: Dict[str, Any]) -> List[node.Node]: + """Generates nodes in the with_columns groups. This does the following: + + 1. Collects all the nodes from the subdag functions + 2. Prunes them to only include the ones that are upstream from the select columns + 3. Sorts them topologically + 4. Creates a new node for each one, injecting the dataframe parameter into the first one + 5. Creates a new node for the final one, injecting the last node into that one + 6. Returns the list of nodes + + :param fn: Function to generate from + :param config: Config to use for generating/collecting nodes + :return: List of nodes that this function produces + """ + namespace = fn.__name__ if self.namespace is None else self.namespace + + initial_nodes = subdag.collect_nodes(config, self.subdag_functions) + pruned_nodes = prune_nodes(initial_nodes, self.select) + if len(pruned_nodes) == 0: + raise ValueError( + f"No nodes found upstream from select columns: {self.select} for function: " + f"{fn.__qualname__}" + ) + sorted_initial_nodes = graph_functions.topologically_sort_nodes(pruned_nodes) + output_nodes = [] + inject_parameter = derive_dataframe_parameter(fn, self.upstream_dependency) + current_dataframe_node = inject_parameter + # Columns that it is dependent on could be from the group of transforms created + columns_produced_within_mapgroup = {node_.name for node_ in pruned_nodes} + columns_passed_in_from_dataframe = set(self.initial_schema) + # Or from the dataframe passed in... + # potential_dependent_columns.update(self.initial_schema) + for node_ in sorted_initial_nodes: + # dependent columns are broken into two sets: + # 1. Those that come from the group of transforms + dependent_columns_in_mapgroup = { + column for column in node_.input_types if column in columns_produced_within_mapgroup + } + # 2. Those that come from the dataframe + dependent_columns_in_dataframe = { + column for column in node_.input_types if column in columns_passed_in_from_dataframe + } + + sparkified = sparkify_node( + node_, + current_dataframe_node, + inject_parameter, + dependent_columns_in_mapgroup, + dependent_columns_in_dataframe, + ) + output_nodes.append(sparkified) + current_dataframe_node = sparkified.name + # We get the final node, which is the function we're using + # and reassign inputs to be the dataframe + output_nodes = subdag.add_namespace(output_nodes, namespace) + final_node = node.Node.from_fn(fn).reassign_input_names( + {inject_parameter: assign_namespace(current_dataframe_node, namespace)} + ) + return output_nodes + [final_node] + + def validate(self, fn: Callable): + derive_dataframe_parameter(fn, self.upstream_dependency) diff --git a/hamilton/function_modifiers/recursive.py b/hamilton/function_modifiers/recursive.py index c5cbffef3..1d54be93b 100644 --- a/hamilton/function_modifiers/recursive.py +++ b/hamilton/function_modifiers/recursive.py @@ -248,12 +248,11 @@ def collect_functions( ) return out - def _collect_nodes(self, original_config: Dict[str, Any]): - combined_config = dict(original_config, **self.config) + @staticmethod + def collect_nodes(config: Dict[str, Any], subdag_functions: List[Callable]) -> List[node.Node]: nodes = [] - for fn in self.subdag_functions: - for node_ in base.resolve_nodes(fn, combined_config): - # nodes.append(node_) + for fn in subdag_functions: + for node_ in base.resolve_nodes(fn, config): nodes.append(node_.copy_with(tags={**node_.tags, **NON_FINAL_TAGS})) return nodes @@ -302,25 +301,32 @@ def _create_additional_static_nodes( ) return out - def _add_namespace(self, nodes: List[node.Node], namespace: str) -> List[node.Node]: + @staticmethod + def add_namespace( + nodes: List[node.Node], + namespace: str, + inputs: Dict[str, Any] = None, + config: Dict[str, Any] = None, + ) -> List[node.Node]: """Utility function to add a namespace to nodes. :param nodes: :return: """ - # already_namespaced_nodes = [] + inputs = inputs if inputs is not None else {} + config = config if config is not None else {} new_nodes = [] new_name_map = {} # First pass we validate + collect names so we can alter dependencies for node_ in nodes: new_name = assign_namespace(node_.name, namespace) new_name_map[node_.name] = new_name - for dep, value in self.inputs.items(): + for dep, value in inputs.items(): # We create nodes for both namespace assignment and source assignment # Why? Cause we need unique parameter names, and with source() some can share params new_name_map[dep] = assign_namespace(dep, namespace) - for dep, value in self.config.items(): + for dep, value in config.items(): new_name_map[dep] = assign_namespace(dep, namespace) # Reassign sources @@ -398,12 +404,13 @@ def _derive_name(self, fn: Callable) -> str: def generate_nodes(self, fn: Callable, configuration: Dict[str, Any]) -> Collection[node.Node]: # Resolve all nodes from passed in functions - nodes = self._collect_nodes(original_config=configuration) + resolved_config = dict(configuration, **self.config) + nodes = self.collect_nodes(config=resolved_config, subdag_functions=self.subdag_functions) # Derive the namespace under which all these nodes will live namespace = self._derive_namespace(fn) final_node_name = self._derive_name(fn) # Rename them all to have the right namespace - nodes = self._add_namespace(nodes, namespace) + nodes = self.add_namespace(nodes, namespace, self.inputs, self.config) # Create any static input nodes we need to translate nodes += self._create_additional_static_nodes(nodes, namespace) # Add the final node that does the translation diff --git a/hamilton/node.py b/hamilton/node.py index 3bba90aa6..c1ff5e709 100644 --- a/hamilton/node.py +++ b/hamilton/node.py @@ -1,4 +1,5 @@ import inspect +import sys import typing from enum import Enum from typing import Any, Callable, Dict, List, Optional, Tuple, Type, Union @@ -241,7 +242,9 @@ def from_fn(fn: Callable, name: str = None) -> "Node": """ if name is None: name = fn.__name__ - return_type = typing.get_type_hints(fn).get("return") + # TODO -- remove this when we no longer support 3.8 -- 10/14/2024 + type_hint_kwargs = {} if sys.version_info < (3, 9) else {"include_extras": True} + return_type = typing.get_type_hints(fn, **type_hint_kwargs).get("return") if return_type is None: raise ValueError(f"Missing type hint for return value in function {fn.__qualname__}.") node_source = NodeType.STANDARD @@ -255,7 +258,6 @@ def from_fn(fn: Callable, name: str = None) -> "Node": if typing_inspect.get_origin(hint) == Collect: node_source = NodeType.COLLECT break - module = inspect.getmodule(fn).__name__ return Node( name, @@ -302,3 +304,19 @@ def copy(self, include_refs: bool = True) -> "Node": :return: A copy of the node. """ return self.copy_with(include_refs) + + def reassign_input_names(self, input_names: Dict[str, Any]) -> "Node": + """Reassigns the input names of a node. Useful for applying + a node to a separate input if needed. + + :param input_names: Input name map to reassign + :return: A node with the input names reassigned + """ + + def new_callable(**kwargs) -> Any: + reverse_input_names = {v: k for k, v in input_names.items()} + return self.callable(**{reverse_input_names.get(k, k): v for k, v in kwargs.items()}) + + new_input_types = {input_names.get(k, k): v for k, v in self.input_types.items()} + out = self.copy_with(callabl=new_callable, input_types=new_input_types) + return out diff --git a/tests/resources/spark/__init__.py b/tests/resources/spark/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/tests/resources/spark/basic_spark_dag.py b/tests/resources/spark/basic_spark_dag.py new file mode 100644 index 000000000..4f6d95a90 --- /dev/null +++ b/tests/resources/spark/basic_spark_dag.py @@ -0,0 +1,59 @@ +import pandas as pd +import pyspark.sql as ps + +from hamilton.experimental import h_spark +from hamilton.htypes import column as _ + +IntSeries = _[pd.Series, int] +FloatSeries = _[pd.Series, float] + + +def a(a_raw: IntSeries) -> IntSeries: + return a_raw + 1 + + +def b(b_raw: IntSeries) -> IntSeries: + return b_raw + 3 + + +def c(c_raw: IntSeries) -> FloatSeries: + return c_raw * 3.5 + + +def a_times_key(a: IntSeries, key: IntSeries) -> IntSeries: + return a * key + + +def b_times_key(b: IntSeries, key: IntSeries) -> IntSeries: + return b * key + + +def a_plus_b_plus_c(a: IntSeries, b: IntSeries, c: FloatSeries) -> FloatSeries: + return a + b + c + + +def df_1(spark_session: ps.SparkSession) -> ps.DataFrame: + df = pd.DataFrame.from_records( + [ + {"key": 1, "a_raw": 1, "b_raw": 2, "c_raw": 1}, + {"key": 2, "a_raw": 4, "b_raw": 5, "c_raw": 2}, + {"key": 3, "a_raw": 7, "b_raw": 8, "c_raw": 3}, + {"key": 4, "a_raw": 10, "b_raw": 11, "c_raw": 4}, + {"key": 5, "a_raw": 13, "b_raw": 14, "c_raw": 5}, + ] + ) + return spark_session.createDataFrame(df) + + +@h_spark.with_columns( + a, + b, + c, + a_times_key, + b_times_key, + a_plus_b_plus_c, + select=["a_times_key", "b_times_key", "a_plus_b_plus_c"], + initial_schema=["a_raw", "b_raw", "c_raw", "key"], +) +def processed_df_as_pandas(df_1: ps.DataFrame) -> pd.DataFrame: + return df_1.select("a_times_key", "b_times_key", "a_plus_b_plus_c").toPandas() diff --git a/tests/resources/pyspark_udfs.py b/tests/resources/spark/pyspark_udfs.py similarity index 100% rename from tests/resources/pyspark_udfs.py rename to tests/resources/spark/pyspark_udfs.py diff --git a/tests/resources/spark/spark_dag_external_dependencies.py b/tests/resources/spark/spark_dag_external_dependencies.py new file mode 100644 index 000000000..0a25224eb --- /dev/null +++ b/tests/resources/spark/spark_dag_external_dependencies.py @@ -0,0 +1,45 @@ +import pandas as pd +import pyspark.sql as ps + +from hamilton.experimental import h_spark +from hamilton.htypes import column as _ + +IntSeries = _[pd.Series, int] + + +def to_add() -> int: + return 1 + + +def to_multiply() -> int: + return 2 + + +def a(initial_column: IntSeries, to_add: int) -> IntSeries: + return initial_column + to_add + + +def b(a: IntSeries, to_multiply: int) -> IntSeries: + return a * to_multiply + + +def df_input(spark_session: ps.SparkSession) -> ps.DataFrame: + df = pd.DataFrame.from_records( + [ + {"initial_column": 1}, + {"initial_column": 2}, + {"initial_column": 3}, + {"initial_column": 4}, + ] + ) + return spark_session.createDataFrame(df) + + +@h_spark.with_columns( + a, + b, + select=["a", "b"], + initial_schema=["initial_column"], +) +def processed_df_as_pandas(df_input: ps.DataFrame) -> pd.DataFrame: + return df_input.select("a", "b").toPandas() diff --git a/tests/resources/spark/spark_dag_multiple_with_columns.py b/tests/resources/spark/spark_dag_multiple_with_columns.py new file mode 100644 index 000000000..5901152b0 --- /dev/null +++ b/tests/resources/spark/spark_dag_multiple_with_columns.py @@ -0,0 +1,115 @@ +import pandas as pd +import pyspark.sql as ps + +from hamilton.experimental import h_spark +from hamilton.htypes import column as _ + +IntSeries = _[pd.Series, int] +FloatSeries = _[pd.Series, float] + + +def a(a_raw: IntSeries) -> IntSeries: + return a_raw + 1 + + +def b(b_raw: IntSeries) -> IntSeries: + return b_raw + 3 + + +def c(c_raw: IntSeries) -> FloatSeries: + return c_raw * 3.5 + + +def a_times_key(a: IntSeries, key: IntSeries) -> IntSeries: + return a * key + + +def b_times_key(b: IntSeries, key: IntSeries) -> IntSeries: + return b * key + + +def a_plus_b_plus_c(a: IntSeries, b: IntSeries, c: FloatSeries) -> FloatSeries: + return a + b + c + + +def const_1() -> float: + return 4.3 + + +# Placing these functions here so we don't try to read the DAG +# This tests the mixing of different types, which is *only* allowed +# inside the with_columns subdag, and not yet allowed within Hamilton +# as hamilton doesn't know that they will compile to the same nodes + + +def _df_2_modules(): + def d(d_raw: IntSeries) -> IntSeries: + return d_raw + 5 + + def e(e_raw: int, d: int, const_1: float) -> float: + return e_raw + d + const_1 + + def f(f_raw: int) -> float: + return f_raw * 3.5 + + def multiply_d_e_f_key( + d: IntSeries, e: FloatSeries, f: FloatSeries, key: IntSeries + ) -> FloatSeries: + return d * e * f * key + + return [d, e, f, multiply_d_e_f_key] + + +def df_1(spark_session: ps.SparkSession) -> ps.DataFrame: + df = pd.DataFrame.from_records( + [ + {"key": 1, "a_raw": 1, "b_raw": 2, "c_raw": 1}, + {"key": 2, "a_raw": 4, "b_raw": 5, "c_raw": 2}, + {"key": 3, "a_raw": 7, "b_raw": 8, "c_raw": 3}, + {"key": 4, "a_raw": 10, "b_raw": 11, "c_raw": 4}, + {"key": 5, "a_raw": 13, "b_raw": 14, "c_raw": 5}, + ] + ) + return spark_session.createDataFrame(df) + + +@h_spark.with_columns( + # TODO -- have a pool (module, rather than function) that we can select *from* + # Or just select in the processed_df? + a, + b, + c, + a_times_key, + b_times_key, + a_plus_b_plus_c, + select=["a_times_key", "b_times_key", "a_plus_b_plus_c"], + initial_schema=["a_raw", "b_raw", "c_raw", "key"], +) +def processed_df_1(df_1: ps.DataFrame) -> ps.DataFrame: + return df_1.select("key", "a_times_key", "b_times_key", "a_plus_b_plus_c") + + +def df_2(spark_session: ps.SparkSession) -> ps.DataFrame: + df = pd.DataFrame.from_records( + [ + {"key": 1, "d_raw": 1, "e_raw": 2, "f_raw": 5}, + {"key": 2, "d_raw": 4, "e_raw": 5, "f_raw": 10}, + {"key": 3, "d_raw": 7, "e_raw": 8, "f_raw": 15}, + {"key": 4, "d_raw": 10, "e_raw": 11, "f_raw": 20}, + ] + ) + return spark_session.createDataFrame(df) + + +@h_spark.with_columns( + *_df_2_modules(), + select=["multiply_d_e_f_key", "d", "e", "f"], + initial_schema=["d_raw", "e_raw", "f_raw", "key"], + dataframe="df_2", +) +def processed_df_2_joined_df_1(df_2: ps.DataFrame, processed_df_1: ps.DataFrame) -> ps.DataFrame: + return df_2.join(processed_df_1, processed_df_1["key"] == df_2["key"], "inner").drop(df_2.key) + + +def final(processed_df_2_joined_df_1: ps.DataFrame) -> pd.DataFrame: + return processed_df_2_joined_df_1.toPandas() From 1f3282e63e1d35e5e031d683a434fb21e02ea231 Mon Sep 17 00:00:00 2001 From: elijahbenizzy Date: Sun, 6 Aug 2023 13:00:53 -0700 Subject: [PATCH 02/11] Adds pyspark examples This has basic documentation in the README, a notebook, and some simple hello_world code/a script. There are a few caveats that are noted in the code. --- docs/how-tos/scale-up.rst | 2 +- examples/spark/pyspark/README.md | 130 ++++++ examples/spark/pyspark/dataflow.py | 124 +++++ examples/spark/pyspark/map_transforms.py | 20 + examples/spark/pyspark/notebook.ipynb | 570 +++++++++++++++++++++++ examples/spark/pyspark/out.png | 25 + examples/spark/pyspark/run.py | 15 + 7 files changed, 885 insertions(+), 1 deletion(-) create mode 100644 examples/spark/pyspark/README.md create mode 100644 examples/spark/pyspark/dataflow.py create mode 100644 examples/spark/pyspark/map_transforms.py create mode 100644 examples/spark/pyspark/notebook.ipynb create mode 100644 examples/spark/pyspark/out.png create mode 100644 examples/spark/pyspark/run.py diff --git a/docs/how-tos/scale-up.rst b/docs/how-tos/scale-up.rst index 6f237a9f4..ebf26fab8 100644 --- a/docs/how-tos/scale-up.rst +++ b/docs/how-tos/scale-up.rst @@ -10,4 +10,4 @@ on larger, distributed datasets (pandas on spark, pyspark map UDFs). 1. Integrating hamilton with `pandas on spark `_. 2. Integrating hamilton with `ray `_. 3. Integrating hamilton with `dask `_. -4. Integrating hamilton using `pyspark map UDFs `__. +4. Integrating hamilton with `pyspark `_. diff --git a/examples/spark/pyspark/README.md b/examples/spark/pyspark/README.md new file mode 100644 index 000000000..0f541e561 --- /dev/null +++ b/examples/spark/pyspark/README.md @@ -0,0 +1,130 @@ +# Hamilton and Spark + +Hamilton now has first-class pyspark integration. While we will likely be improving it as we go along, +this version is the first we're considering "stable" and moving out of "experimental" + +# Design + +The idea is to break your code into components. These components make one of two shapes: + +1. Run linearly (e.g. chained dataframe transformations -- aggregations, etc...) +2. Form a DAG of operations (E.G. a set of features that have inter-feature dependencies) + +For the first case, we just use the pyspark dataframe API. You define functions that, when put +through Hamilton, act as a pipe. For example: + +```python +import pyspark.sql as ps + +def raw_data_1() -> ps.DataFrame: + """Loads up data from an external source""" + +def raw_data_2() -> ps.DataFrame: + """Loads up data from an external source""" + +def all_initial_data(raw_data_1: ps.DataFrame, raw_data_2: ps.DataFrame) -> ps.DataFrame: + """Combines the two dataframes""" + return _join(raw_data_1, raw_data_2) + +def raw_data_3() -> ps.DataFrame: + """Loads up data from an external source""" +``` + +For the next case, we define transformations that are columnar/map-oriented in nature. +These are UDFs (either pandas or python) that get applied to the dataframe in a specific order: + +```python +import pandas as pd + +#map_transforms.py + +def column_3(column_1_from_dataframe: pd.Series) -> pd.Series: + """Transforms column 1 (from the dataframe) into column 3""" + return _some_transform(column_1_from_dataframe) + +def column_4(column_2_from_dataframe: pd.Series) -> pd.Series: + """Transforms column 2 (from the dataframe) into column 4""" + return _some_other_transform(column_2_from_dataframe) + +def column_5(column_3: pd.Series, column_4: pd.Series) -> pd.Series: + """is a combination of column_1_from_dataframe and column_2_from_dataframe""" + return _yet_another_transform(column_3, column_4) +``` + +Finally, we combine them together with a call to `with_column`: + +```python +from hamilton.experimental.h_spark import with_columns +import pyspark.sql as ps +import map_transforms # file defined above + +@with_columns( + map_transforms, # Load all the functions we defined above + select=["column_1_from_dataframe", "column_2_from_dataframe"], # calculate these + dataframe="all_initial_data" # use this dataframe as the source (and inject the result into the final_result function +) +def final_result(all_initial_data: ps.DataFrame, raw_data_3: ps.DataFrame) -> ps.DataFrame: + """Gives the final result. This decorator will apply the transformations in the order. + Then, the final_result function is called, with the result of the transformations passed in.""" + return _join(all_initial_data, raw_data_3) +``` + +Thus you can represent a clean, modular, unit-testable string of transformations while also allowing for +complex sets of feature UDFs. Note that we will shortly allow the with_columns group to include pyspark +functions (dataframe -> dataframe) as well, but that is not currently supported. + +We have implemented the hamilton hello_world example in [run.py](run.py) and the [map_transforms.py](map_transforms.py)/[dataflow.py](dataflow.py) files +so you can compare. You can run `run.py`: + +`python run.py` + +and check out the interactive example in the `notebook.ipynb` file. + +## How does this work? + +The `with_columns` decorator does the following: +1. Resolves the functions you pass in, with the config passed from the driver +2. Transforms them each into a node, in topological order. + - Retains all specified dependencies + - Adds a single dataframe that gets wired through (linearizing the operations) + - Transforms each function into a function of that input dataframe and any other external dependencies + +Thus the graph continually assigns to a single (immutable) dataframe, tracking the result, and still displays the DAG shape +that was presented by the code. Column-level lineage is preserved and readable from the code, while it executes as a +normal set of spark operations. + +## Why use Hamilton and not plain spark? + +As you can see above, we delegate almost entirely to spark. However, when you want column-level lineage and modular functions, +vanilla spark is often suboptimal. It requires "linearization" (e.g. chained) modification of the same dataframe, and can +get messy and out of hand. Thus we group together columnar (map-ish) operations together, while still allowing the +chaining of dataframe functions that pyspark expresses naturally. + +Furthermore, this opens up a few interesting debugging capabilities, that we're building out: +1. Running components of your workflow in pandas +2. Unit testing individual spark transforms +3. Grabbing intermediate results to debug the spark execution plan at any given moment +4. Adding `collect` halfway through to make inspection easier +5. Breaking large spark jobs into separate tasks by arranging their functions into modules + +We have found that spark ETLs tend to fit the patterns above nicely, and that hamilton helps make them shine. + +## Why not just pass around spark dataframes in Hamilton functions? + +This is great when your functions are linear, but when you have enough features this gets messy. Specifically, +unit testing/zooming in on transformations often requires tooling to manage/collect UDFs, and that's what Hamilton does. + +The biggest problem comes when you want column-level transformations so you extract out the columns into different dataframes +and join them together. This can result in (accidental) performance issues, as spark does very poorly in handling multiple +large joins. + +## Why not use pandas-on-spark? + +We support that! You can use pandas-on-spark with the `KoalaGraphAdapter` -- see [Pandas on Spark](../pandas_on_spark/README.md) for reference. +Some people prefer vanilla spark, some like pandas-on-spark. We support both. + +Note there are other scaling libraries that Hamilton supports -- it all depends on your use-case: + +- [dask](../../dask/README.md) +- [ray](../../ray/README.md) +- [modin](https://github.com/modin-project/modin) (no example for modin yet but it is just the pandas API with a different import) diff --git a/examples/spark/pyspark/dataflow.py b/examples/spark/pyspark/dataflow.py new file mode 100644 index 000000000..eb0c999ff --- /dev/null +++ b/examples/spark/pyspark/dataflow.py @@ -0,0 +1,124 @@ +from typing import Dict + +import map_transforms +import pandas as pd +import pyspark.sql as ps +from pyspark.sql.functions import col, mean, stddev + +from hamilton.experimental import h_spark +from hamilton.function_modifiers import extract_fields + + +def spark_session() -> ps.SparkSession: + """Pyspark session to load up when starting. + You can also pass it in if you so choose. + + :return: + """ + return ps.SparkSession.builder.master("local[1]").getOrCreate() + + +def base_df(spark_session: ps.SparkSession) -> ps.DataFrame: + """Dummy function showing how to wire through loading data. + Note you can use @load_from (although our spark data loaders are limited now). + + :return: A dataframe with spend and signups columns. + """ + pd_df = pd.DataFrame( + { + "spend": [ + 10, + 10, + 20, + 40, + 40, + 50, + 60, + 70, + 90, + 100, + 70, + 80, + 90, + 100, + 110, + 120, + 130, + 140, + 150, + 160, + ], + "signups": [ + 1, + 10, + 50, + 100, + 200, + 400, + 600, + 800, + 1000, + 1200, + 1400, + 1600, + 1800, + 2000, + 2200, + 2400, + 2600, + 2800, + 3000, + 3200, + ], + } + ) + return spark_session.createDataFrame(pd_df) + + +@extract_fields( + { + "spend_mean": float, + "spend_std_dev": float, + } +) +def spend_statistics(base_df: ps.DataFrame) -> Dict[str, float]: + """Computes the mean and standard deviation of the spend column. + Note that this is a blocking (collect) operation, + but it doesn't have to be if you use an aggregation. In that case + you'd just add the column to the dataframe and refer to it downstream, + by expanding `initial_schema` in `with_mapped_data`. + + :param base_df: Base dataframe with spend and signups columns. + :return: A dictionary with the mean and standard deviation of the spend column. + """ + df_stats = base_df.select( + mean(col("spend")).alias("mean"), stddev(col("spend")).alias("std") + ).collect() + + return { + "spend_mean": df_stats[0]["mean"], + "spend_std_dev": df_stats[0]["std"], + } + + +@h_spark.with_columns( + map_transforms, + initial_schema=["spend", "signups"], +) +def with_mapped_data(base_df: ps.DataFrame) -> ps.DataFrame: + """Applies all the transforms in map_transforms + + :param base_df: + :return: + """ + return base_df + + +def final_result(with_mapped_data: ps.DataFrame) -> pd.DataFrame: + """Computes the final result. You could always just output the pyspark + dataframe, but we'll collect it and make it a pandas dataframe. + + :param base_df: Base dataframe with spend and signups columns. + :return: A dataframe with the final result. + """ + return with_mapped_data.toPandas() diff --git a/examples/spark/pyspark/map_transforms.py b/examples/spark/pyspark/map_transforms.py new file mode 100644 index 000000000..876268414 --- /dev/null +++ b/examples/spark/pyspark/map_transforms.py @@ -0,0 +1,20 @@ +import pandas as pd + +from hamilton.htypes import column + + +def spend_per_signup(spend: pd.Series, signups: pd.Series) -> column[pd.Series, float]: + """The cost per signup in relation to spend.""" + return spend / signups + + +def spend_zero_mean(spend: pd.Series, spend_mean: float) -> column[pd.Series, float]: + """Shows function that takes a scalar. In this case to zero mean spend.""" + return spend - spend_mean + + +def spend_zero_mean_unit_variance( + spend_zero_mean: pd.Series, spend_std_dev: float +) -> column[pd.Series, float]: + """Function showing one way to make spend have zero mean and unit variance.""" + return spend_zero_mean / spend_std_dev diff --git a/examples/spark/pyspark/notebook.ipynb b/examples/spark/pyspark/notebook.ipynb new file mode 100644 index 000000000..a06468894 --- /dev/null +++ b/examples/spark/pyspark/notebook.ipynb @@ -0,0 +1,570 @@ +{ + "cells": [ + { + "cell_type": "code", + "execution_count": 6, + "id": "4c8c7cb7", + "metadata": {}, + "outputs": [], + "source": [ + "from hamilton import driver, base\n", + "import dataflow, map_transforms" + ] + }, + { + "cell_type": "markdown", + "id": "068cb1ad", + "metadata": {}, + "source": [ + "# Basic Driver Instantiation + Visualization\n", + "\n", + "Pyspark is effectively the same as pandas dataframes -- you're welcome to use it however you would pyspark\n", + "Only difference is the with_columns decorator, which chains together UDFs while preserving lineage.\n", + "\n", + "In this notebook we:\n", + "1. Instantiate a driver\n", + "2. Visualize execution\n", + "3. Capture the result (in pandas and pyspark)\n", + "4. Explain the execution plan in pyspark" + ] + }, + { + "cell_type": "code", + "execution_count": 18, + "id": "a85bb3cf", + "metadata": {}, + "outputs": [ + { + "data": { + "image/svg+xml": [ + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "final_result\n", + "\n", + "final_result\n", + "\n", + "\n", + "\n", + "spend_mean\n", + "\n", + "spend_mean\n", + "\n", + "\n", + "\n", + "with_mapped_data.spend_zero_mean\n", + "\n", + "with_mapped_data.spend_zero_mean\n", + "\n", + "\n", + "\n", + "spend_mean->with_mapped_data.spend_zero_mean\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "with_mapped_data.spend_zero_mean_unit_variance\n", + "\n", + "with_mapped_data.spend_zero_mean_unit_variance\n", + "\n", + "\n", + "\n", + "with_mapped_data.spend_zero_mean->with_mapped_data.spend_zero_mean_unit_variance\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "spend_statistics\n", + "\n", + "spend_statistics\n", + "\n", + "\n", + "\n", + "spend_statistics->spend_mean\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "spend_std_dev\n", + "\n", + "spend_std_dev\n", + "\n", + "\n", + "\n", + "spend_statistics->spend_std_dev\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "with_mapped_data.spend_per_signup\n", + "\n", + "with_mapped_data.spend_per_signup\n", + "\n", + "\n", + "\n", + "with_mapped_data.spend_per_signup->with_mapped_data.spend_zero_mean\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "with_mapped_data\n", + "\n", + "with_mapped_data\n", + "\n", + "\n", + "\n", + "with_mapped_data->final_result\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "spark_session\n", + "\n", + "spark_session\n", + "\n", + "\n", + "\n", + "base_df\n", + "\n", + "base_df\n", + "\n", + "\n", + "\n", + "spark_session->base_df\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "spend_std_dev->with_mapped_data.spend_zero_mean_unit_variance\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "with_mapped_data.spend_zero_mean_unit_variance->with_mapped_data\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "base_df->with_mapped_data.spend_zero_mean\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "base_df->spend_statistics\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "base_df->with_mapped_data.spend_per_signup\n", + "\n", + "\n", + "\n", + "\n", + "\n" + ], + "text/plain": [ + "" + ] + }, + "execution_count": 18, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "dr = driver.\\\n", + " Builder().\\\n", + " with_modules(dataflow, map_transforms).\\\n", + " with_adapter(base.DefaultAdapter()).\\\n", + " build()\n", + "dr.visualize_execution([\"final_result\"], \"./out.png\", {\"format\": \"png\"})" + ] + }, + { + "cell_type": "code", + "execution_count": 19, + "id": "ac02f09c", + "metadata": {}, + "outputs": [ + { + "name": "stderr", + "output_type": "stream", + "text": [ + "\r", + "[Stage 15:> (0 + 1) / 1]\r", + "\r", + " \r" + ] + }, + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
spendsignupsspend_per_signupspend_zero_meanspend_zero_mean_unit_variance
010110.000000-72.0-1.590827
110101.000000-72.0-1.590827
220500.400000-62.0-1.369879
3401000.400000-42.0-0.927982
4402000.200000-42.0-0.927982
5504000.125000-32.0-0.707034
6606000.100000-22.0-0.486086
7708000.087500-12.0-0.265138
89010000.0900008.00.176759
910012000.08333318.00.397707
107014000.050000-12.0-0.265138
118016000.050000-2.0-0.044190
129018000.0500008.00.176759
1310020000.05000018.00.397707
1411022000.05000028.00.618655
1512024000.05000038.00.839603
1613026000.05000048.01.060551
1714028000.05000058.01.281499
1815030000.05000068.01.502447
1916032000.05000078.01.723396
\n", + "
" + ], + "text/plain": [ + " spend signups spend_per_signup spend_zero_mean \\\n", + "0 10 1 10.000000 -72.0 \n", + "1 10 10 1.000000 -72.0 \n", + "2 20 50 0.400000 -62.0 \n", + "3 40 100 0.400000 -42.0 \n", + "4 40 200 0.200000 -42.0 \n", + "5 50 400 0.125000 -32.0 \n", + "6 60 600 0.100000 -22.0 \n", + "7 70 800 0.087500 -12.0 \n", + "8 90 1000 0.090000 8.0 \n", + "9 100 1200 0.083333 18.0 \n", + "10 70 1400 0.050000 -12.0 \n", + "11 80 1600 0.050000 -2.0 \n", + "12 90 1800 0.050000 8.0 \n", + "13 100 2000 0.050000 18.0 \n", + "14 110 2200 0.050000 28.0 \n", + "15 120 2400 0.050000 38.0 \n", + "16 130 2600 0.050000 48.0 \n", + "17 140 2800 0.050000 58.0 \n", + "18 150 3000 0.050000 68.0 \n", + "19 160 3200 0.050000 78.0 \n", + "\n", + " spend_zero_mean_unit_variance \n", + "0 -1.590827 \n", + "1 -1.590827 \n", + "2 -1.369879 \n", + "3 -0.927982 \n", + "4 -0.927982 \n", + "5 -0.707034 \n", + "6 -0.486086 \n", + "7 -0.265138 \n", + "8 0.176759 \n", + "9 0.397707 \n", + "10 -0.265138 \n", + "11 -0.044190 \n", + "12 0.176759 \n", + "13 0.397707 \n", + "14 0.618655 \n", + "15 0.839603 \n", + "16 1.060551 \n", + "17 1.281499 \n", + "18 1.502447 \n", + "19 1.723396 " + ] + }, + "execution_count": 19, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "# get both the pandas result *and* the pyspark dataframes to examine\n", + "all_results = dr.execute([\"final_result\", \"with_mapped_data\"])\n", + "all_results[\"final_result\"]" + ] + }, + { + "cell_type": "code", + "execution_count": 21, + "id": "71fd52ed", + "metadata": {}, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "== Physical Plan ==\n", + "* Project (3)\n", + "+- ArrowEvalPython (2)\n", + " +- * Scan ExistingRDD (1)\n", + "\n", + "\n", + "(1) Scan ExistingRDD [codegen id : 1]\n", + "Output [2]: [spend#263L, signups#264L]\n", + "Arguments: [spend#263L, signups#264L], MapPartitionsRDD[49] at applySchemaToPythonRDD at NativeMethodAccessorImpl.java:0, ExistingRDD, UnknownPartitioning(0)\n", + "\n", + "(2) ArrowEvalPython\n", + "Input [2]: [spend#263L, signups#264L]\n", + "Arguments: [spend_per_signup(signups#264L, spend#263L)#335, spend_zero_mean(spend#263L)#340, spend_zero_mean_unit_variance(spend_zero_mean(spend#263L)#340)#346], [pythonUDF0#353, pythonUDF1#354, pythonUDF2#355], 200\n", + "\n", + "(3) Project [codegen id : 2]\n", + "Output [5]: [spend#263L, signups#264L, pythonUDF0#353 AS spend_per_signup#336, pythonUDF1#354 AS spend_zero_mean#341, pythonUDF2#355 AS spend_zero_mean_unit_variance#347]\n", + "Input [5]: [spend#263L, signups#264L, pythonUDF0#353, pythonUDF1#354, pythonUDF2#355]\n", + "\n", + "\n" + ] + } + ], + "source": [ + "# note that this only goes until the aggregation statistics, as we called `collect`\n", + "all_results[\"with_mapped_data\"].explain(\"formatted\")" + ] + }, + { + "cell_type": "code", + "execution_count": 23, + "id": "8986d1ca", + "metadata": {}, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "+-----+-------+----------------+---------------+-----------------------------+\n", + "|spend|signups|spend_per_signup|spend_zero_mean|spend_zero_mean_unit_variance|\n", + "+-----+-------+----------------+---------------+-----------------------------+\n", + "| 10| 1| 10.0| -72.0| -1.5908267|\n", + "| 10| 10| 1.0| -72.0| -1.5908267|\n", + "| 20| 50| 0.4| -62.0| -1.3698785|\n", + "| 40| 100| 0.4| -42.0| -0.9279823|\n", + "| 40| 200| 0.2| -42.0| -0.9279823|\n", + "| 50| 400| 0.125| -32.0| -0.7070341|\n", + "| 60| 600| 0.1| -22.0| -0.48608595|\n", + "| 70| 800| 0.0875| -12.0| -0.2651378|\n", + "| 90| 1000| 0.09| 8.0| 0.17675853|\n", + "| 100| 1200| 0.083333336| 18.0| 0.3977067|\n", + "| 70| 1400| 0.05| -12.0| -0.2651378|\n", + "| 80| 1600| 0.05| -2.0| -0.044189632|\n", + "| 90| 1800| 0.05| 8.0| 0.17675853|\n", + "| 100| 2000| 0.05| 18.0| 0.3977067|\n", + "| 110| 2200| 0.05| 28.0| 0.61865485|\n", + "| 120| 2400| 0.05| 38.0| 0.839603|\n", + "| 130| 2600| 0.05| 48.0| 1.0605512|\n", + "| 140| 2800| 0.05| 58.0| 1.2814993|\n", + "| 150| 3000| 0.05| 68.0| 1.5024475|\n", + "| 160| 3200| 0.05| 78.0| 1.7233956|\n", + "+-----+-------+----------------+---------------+-----------------------------+\n", + "\n" + ] + } + ], + "source": [ + "all_results[\"with_mapped_data\"].show()" + ] + } + ], + "metadata": { + "kernelspec": { + "display_name": "Python 3 (ipykernel)", + "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.9.10" + } + }, + "nbformat": 4, + "nbformat_minor": 5 +} diff --git a/examples/spark/pyspark/out.png b/examples/spark/pyspark/out.png new file mode 100644 index 000000000..719e0854e --- /dev/null +++ b/examples/spark/pyspark/out.png @@ -0,0 +1,25 @@ +// Dependency Graph +digraph { + final_result [label=final_result shape=rectangle] + spend_mean [label=spend_mean] + "with_mapped_data.spend_zero_mean" [label="with_mapped_data.spend_zero_mean"] + spend_statistics [label=spend_statistics] + "with_mapped_data.spend_per_signup" [label="with_mapped_data.spend_per_signup"] + with_mapped_data [label=with_mapped_data] + spark_session [label=spark_session] + spend_std_dev [label=spend_std_dev] + "with_mapped_data.spend_zero_mean_unit_variance" [label="with_mapped_data.spend_zero_mean_unit_variance"] + base_df [label=base_df] + with_mapped_data -> final_result + spend_statistics -> spend_mean + spend_mean -> "with_mapped_data.spend_zero_mean" + "with_mapped_data.spend_per_signup" -> "with_mapped_data.spend_zero_mean" + base_df -> "with_mapped_data.spend_zero_mean" + base_df -> spend_statistics + base_df -> "with_mapped_data.spend_per_signup" + "with_mapped_data.spend_zero_mean_unit_variance" -> with_mapped_data + spend_statistics -> spend_std_dev + "with_mapped_data.spend_zero_mean" -> "with_mapped_data.spend_zero_mean_unit_variance" + spend_std_dev -> "with_mapped_data.spend_zero_mean_unit_variance" + spark_session -> base_df +} diff --git a/examples/spark/pyspark/run.py b/examples/spark/pyspark/run.py new file mode 100644 index 000000000..227f8a5a0 --- /dev/null +++ b/examples/spark/pyspark/run.py @@ -0,0 +1,15 @@ +import dataflow +import map_transforms + +from hamilton import driver + + +def main(): + dr = driver.Builder().with_modules(dataflow, map_transforms).build() + dr.visualize_execution(["final_result"], "./out.png", {"format": "png"}) + final_result = dr.execute(["final_result"]) + print(final_result) + + +if __name__ == "__main__": + main() From f09bbc7392e3467b578e604580a07e457266448e Mon Sep 17 00:00:00 2001 From: elijahbenizzy Date: Mon, 7 Aug 2023 16:54:20 -0700 Subject: [PATCH 03/11] Adds pyspark -> pyspark UDFs --- examples/spark/pyspark/notebook.ipynb | 577 +++--------------- graph_adapter_tests/h_spark/test_h_spark.py | 80 ++- hamilton/experimental/h_spark.py | 250 ++++++-- hamilton/node.py | 17 + .../resources/spark/spark_dag_pyspark_udfs.py | 56 ++ 5 files changed, 418 insertions(+), 562 deletions(-) create mode 100644 tests/resources/spark/spark_dag_pyspark_udfs.py diff --git a/examples/spark/pyspark/notebook.ipynb b/examples/spark/pyspark/notebook.ipynb index a06468894..e7f704ed1 100644 --- a/examples/spark/pyspark/notebook.ipynb +++ b/examples/spark/pyspark/notebook.ipynb @@ -2,548 +2,133 @@ "cells": [ { "cell_type": "code", - "execution_count": 6, + "execution_count": 13, "id": "4c8c7cb7", "metadata": {}, "outputs": [], "source": [ - "from hamilton import driver, base\n", - "import dataflow, map_transforms" + "import pyspark.sql as ps\n", + "import pandas as pd\n", + "from pyspark.sql.functions import col, mean, stddev" ] }, { - "cell_type": "markdown", - "id": "068cb1ad", + "cell_type": "code", + "execution_count": 6, + "id": "a85bb3cf", "metadata": {}, + "outputs": [], "source": [ - "# Basic Driver Instantiation + Visualization\n", - "\n", - "Pyspark is effectively the same as pandas dataframes -- you're welcome to use it however you would pyspark\n", - "Only difference is the with_columns decorator, which chains together UDFs while preserving lineage.\n", - "\n", - "In this notebook we:\n", - "1. Instantiate a driver\n", - "2. Visualize execution\n", - "3. Capture the result (in pandas and pyspark)\n", - "4. Explain the execution plan in pyspark" + "spark_session = ps.SparkSession.builder.master(\"local[1]\").getOrCreate()" ] }, { "cell_type": "code", - "execution_count": 18, - "id": "a85bb3cf", + "execution_count": 10, + "id": "ac02f09c", "metadata": {}, - "outputs": [ - { - "data": { - "image/svg+xml": [ - "\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "final_result\n", - "\n", - "final_result\n", - "\n", - "\n", - "\n", - "spend_mean\n", - "\n", - "spend_mean\n", - "\n", - "\n", - "\n", - "with_mapped_data.spend_zero_mean\n", - "\n", - "with_mapped_data.spend_zero_mean\n", - "\n", - "\n", - "\n", - "spend_mean->with_mapped_data.spend_zero_mean\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "with_mapped_data.spend_zero_mean_unit_variance\n", - "\n", - "with_mapped_data.spend_zero_mean_unit_variance\n", - "\n", - "\n", - "\n", - "with_mapped_data.spend_zero_mean->with_mapped_data.spend_zero_mean_unit_variance\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "spend_statistics\n", - "\n", - "spend_statistics\n", - "\n", - "\n", - "\n", - "spend_statistics->spend_mean\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "spend_std_dev\n", - "\n", - "spend_std_dev\n", - "\n", - "\n", - "\n", - "spend_statistics->spend_std_dev\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "with_mapped_data.spend_per_signup\n", - "\n", - "with_mapped_data.spend_per_signup\n", - "\n", - "\n", - "\n", - "with_mapped_data.spend_per_signup->with_mapped_data.spend_zero_mean\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "with_mapped_data\n", - "\n", - "with_mapped_data\n", - "\n", - "\n", - "\n", - "with_mapped_data->final_result\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "spark_session\n", - "\n", - "spark_session\n", - "\n", - "\n", - "\n", - "base_df\n", - "\n", - "base_df\n", - "\n", - "\n", - "\n", - "spark_session->base_df\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "spend_std_dev->with_mapped_data.spend_zero_mean_unit_variance\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "with_mapped_data.spend_zero_mean_unit_variance->with_mapped_data\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "base_df->with_mapped_data.spend_zero_mean\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "base_df->spend_statistics\n", - "\n", - "\n", - "\n", - "\n", - "\n", - "base_df->with_mapped_data.spend_per_signup\n", - "\n", - "\n", - "\n", - "\n", - "\n" - ], - "text/plain": [ - "" - ] - }, - "execution_count": 18, - "metadata": {}, - "output_type": "execute_result" - } - ], + "outputs": [], "source": [ - "dr = driver.\\\n", - " Builder().\\\n", - " with_modules(dataflow, map_transforms).\\\n", - " with_adapter(base.DefaultAdapter()).\\\n", - " build()\n", - "dr.visualize_execution([\"final_result\"], \"./out.png\", {\"format\": \"png\"})" + "pd_df = pd.DataFrame(\n", + " {\n", + " \"spend\": [\n", + " 10,\n", + " 10,\n", + " 20,\n", + " 40,\n", + " 40,\n", + " 50,\n", + " 60,\n", + " 70,\n", + " 90,\n", + " 100,\n", + " 70,\n", + " 80,\n", + " 90,\n", + " 100,\n", + " 110,\n", + " 120,\n", + " 130,\n", + " 140,\n", + " 150,\n", + " 160,\n", + " ],\n", + " \"signups\": [\n", + " 1,\n", + " 10,\n", + " 50,\n", + " 100,\n", + " 200,\n", + " 400,\n", + " 600,\n", + " 800,\n", + " 1000,\n", + " 1200,\n", + " 1400,\n", + " 1600,\n", + " 1800,\n", + " 2000,\n", + " 2200,\n", + " 2400,\n", + " 2600,\n", + " 2800,\n", + " 3000,\n", + " 3200,\n", + " ],\n", + " }\n", + " )\n", + "ps_df = spark_session.createDataFrame(pd_df)" ] }, { "cell_type": "code", - "execution_count": 19, - "id": "ac02f09c", + "execution_count": 17, + "id": "71fd52ed", "metadata": {}, "outputs": [ - { - "name": "stderr", - "output_type": "stream", - "text": [ - "\r", - "[Stage 15:> (0 + 1) / 1]\r", - "\r", - " \r" - ] - }, { "data": { - "text/html": [ - "
\n", - "\n", - "\n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - "
spendsignupsspend_per_signupspend_zero_meanspend_zero_mean_unit_variance
010110.000000-72.0-1.590827
110101.000000-72.0-1.590827
220500.400000-62.0-1.369879
3401000.400000-42.0-0.927982
4402000.200000-42.0-0.927982
5504000.125000-32.0-0.707034
6606000.100000-22.0-0.486086
7708000.087500-12.0-0.265138
89010000.0900008.00.176759
910012000.08333318.00.397707
107014000.050000-12.0-0.265138
118016000.050000-2.0-0.044190
129018000.0500008.00.176759
1310020000.05000018.00.397707
1411022000.05000028.00.618655
1512024000.05000038.00.839603
1613026000.05000048.01.060551
1714028000.05000058.01.281499
1815030000.05000068.01.502447
1916032000.05000078.01.723396
\n", - "
" - ], "text/plain": [ - " spend signups spend_per_signup spend_zero_mean \\\n", - "0 10 1 10.000000 -72.0 \n", - "1 10 10 1.000000 -72.0 \n", - "2 20 50 0.400000 -62.0 \n", - "3 40 100 0.400000 -42.0 \n", - "4 40 200 0.200000 -42.0 \n", - "5 50 400 0.125000 -32.0 \n", - "6 60 600 0.100000 -22.0 \n", - "7 70 800 0.087500 -12.0 \n", - "8 90 1000 0.090000 8.0 \n", - "9 100 1200 0.083333 18.0 \n", - "10 70 1400 0.050000 -12.0 \n", - "11 80 1600 0.050000 -2.0 \n", - "12 90 1800 0.050000 8.0 \n", - "13 100 2000 0.050000 18.0 \n", - "14 110 2200 0.050000 28.0 \n", - "15 120 2400 0.050000 38.0 \n", - "16 130 2600 0.050000 48.0 \n", - "17 140 2800 0.050000 58.0 \n", - "18 150 3000 0.050000 68.0 \n", - "19 160 3200 0.050000 78.0 \n", - "\n", - " spend_zero_mean_unit_variance \n", - "0 -1.590827 \n", - "1 -1.590827 \n", - "2 -1.369879 \n", - "3 -0.927982 \n", - "4 -0.927982 \n", - "5 -0.707034 \n", - "6 -0.486086 \n", - "7 -0.265138 \n", - "8 0.176759 \n", - "9 0.397707 \n", - "10 -0.265138 \n", - "11 -0.044190 \n", - "12 0.176759 \n", - "13 0.397707 \n", - "14 0.618655 \n", - "15 0.839603 \n", - "16 1.060551 \n", - "17 1.281499 \n", - "18 1.502447 \n", - "19 1.723396 " + "DataFrame[foo: double]" ] }, - "execution_count": 19, + "execution_count": 17, "metadata": {}, "output_type": "execute_result" } ], "source": [ - "# get both the pandas result *and* the pyspark dataframes to examine\n", - "all_results = dr.execute([\"final_result\", \"with_mapped_data\"])\n", - "all_results[\"final_result\"]" + "ps_df.select(mean(col(\"spend\")).alias(\"foo\"))" ] }, { "cell_type": "code", - "execution_count": 21, - "id": "71fd52ed", + "execution_count": 25, + "id": "8986d1ca", "metadata": {}, "outputs": [ { - "name": "stdout", - "output_type": "stream", - "text": [ - "== Physical Plan ==\n", - "* Project (3)\n", - "+- ArrowEvalPython (2)\n", - " +- * Scan ExistingRDD (1)\n", - "\n", - "\n", - "(1) Scan ExistingRDD [codegen id : 1]\n", - "Output [2]: [spend#263L, signups#264L]\n", - "Arguments: [spend#263L, signups#264L], MapPartitionsRDD[49] at applySchemaToPythonRDD at NativeMethodAccessorImpl.java:0, ExistingRDD, UnknownPartitioning(0)\n", - "\n", - "(2) ArrowEvalPython\n", - "Input [2]: [spend#263L, signups#264L]\n", - "Arguments: [spend_per_signup(signups#264L, spend#263L)#335, spend_zero_mean(spend#263L)#340, spend_zero_mean_unit_variance(spend_zero_mean(spend#263L)#340)#346], [pythonUDF0#353, pythonUDF1#354, pythonUDF2#355], 200\n", - "\n", - "(3) Project [codegen id : 2]\n", - "Output [5]: [spend#263L, signups#264L, pythonUDF0#353 AS spend_per_signup#336, pythonUDF1#354 AS spend_zero_mean#341, pythonUDF2#355 AS spend_zero_mean_unit_variance#347]\n", - "Input [5]: [spend#263L, signups#264L, pythonUDF0#353, pythonUDF1#354, pythonUDF2#355]\n", - "\n", - "\n" - ] + "data": { + "text/plain": [ + "DataFrame[spend: bigint, signups: bigint, foo: bigint]" + ] + }, + "execution_count": 25, + "metadata": {}, + "output_type": "execute_result" } ], "source": [ - "# note that this only goes until the aggregation statistics, as we called `collect`\n", - "all_results[\"with_mapped_data\"].explain(\"formatted\")" + "ps_df.withColumn(\"foo\", ps_df['signups']*ps_df['spend'])" ] }, { "cell_type": "code", - "execution_count": 23, - "id": "8986d1ca", + "execution_count": null, + "id": "7489e4dd", "metadata": {}, - "outputs": [ - { - "name": "stdout", - "output_type": "stream", - "text": [ - "+-----+-------+----------------+---------------+-----------------------------+\n", - "|spend|signups|spend_per_signup|spend_zero_mean|spend_zero_mean_unit_variance|\n", - "+-----+-------+----------------+---------------+-----------------------------+\n", - "| 10| 1| 10.0| -72.0| -1.5908267|\n", - "| 10| 10| 1.0| -72.0| -1.5908267|\n", - "| 20| 50| 0.4| -62.0| -1.3698785|\n", - "| 40| 100| 0.4| -42.0| -0.9279823|\n", - "| 40| 200| 0.2| -42.0| -0.9279823|\n", - "| 50| 400| 0.125| -32.0| -0.7070341|\n", - "| 60| 600| 0.1| -22.0| -0.48608595|\n", - "| 70| 800| 0.0875| -12.0| -0.2651378|\n", - "| 90| 1000| 0.09| 8.0| 0.17675853|\n", - "| 100| 1200| 0.083333336| 18.0| 0.3977067|\n", - "| 70| 1400| 0.05| -12.0| -0.2651378|\n", - "| 80| 1600| 0.05| -2.0| -0.044189632|\n", - "| 90| 1800| 0.05| 8.0| 0.17675853|\n", - "| 100| 2000| 0.05| 18.0| 0.3977067|\n", - "| 110| 2200| 0.05| 28.0| 0.61865485|\n", - "| 120| 2400| 0.05| 38.0| 0.839603|\n", - "| 130| 2600| 0.05| 48.0| 1.0605512|\n", - "| 140| 2800| 0.05| 58.0| 1.2814993|\n", - "| 150| 3000| 0.05| 68.0| 1.5024475|\n", - "| 160| 3200| 0.05| 78.0| 1.7233956|\n", - "+-----+-------+----------------+---------------+-----------------------------+\n", - "\n" - ] - } - ], - "source": [ - "all_results[\"with_mapped_data\"].show()" - ] + "outputs": [], + "source": [] } ], "metadata": { diff --git a/graph_adapter_tests/h_spark/test_h_spark.py b/graph_adapter_tests/h_spark/test_h_spark.py index a05368a95..e1f20fc61 100644 --- a/graph_adapter_tests/h_spark/test_h_spark.py +++ b/graph_adapter_tests/h_spark/test_h_spark.py @@ -1,20 +1,14 @@ -<<<<<<< HEAD import sys import numpy as np -======= import logging ->>>>>>> 7d0b719 (Implements spark integration, see #248) import pandas as pd import pyspark.pandas as ps import pytest from pyspark import Row -<<<<<<< HEAD from pyspark.sql import SparkSession, types -======= from pyspark.sql import Column, DataFrame, SparkSession ->>>>>>> 7d0b719 (Implements spark integration, see #248) from pyspark.sql.functions import column from hamilton import base, driver, htypes, node @@ -27,6 +21,7 @@ pyspark_udfs, spark_dag_external_dependencies, spark_dag_multiple_with_columns, + spark_dag_pyspark_udfs, ) setup_logging(logging.DEBUG) @@ -235,7 +230,6 @@ def test_smoke_screen_udf_graph_adapter(spark_session): ] -<<<<<<< HEAD # Test cases for python_to_spark_type function @pytest.mark.parametrize( "python_type,expected_spark_type", @@ -269,11 +263,9 @@ def test_python_to_spark_type_invalid(invalid_python_type): (bytes, types.BinaryType()), ], ) -def test_get_spark_type_basic_types( - dummy_kwargs, dummy_df, dummy_udf, return_type, expected_spark_type -): +def test_get_spark_type_basic_types(return_type, expected_spark_type): assert ( - h_spark.get_spark_type(dummy_kwargs, dummy_df, dummy_udf, return_type) + h_spark.get_spark_type(return_type) == expected_spark_type ) @@ -291,11 +283,11 @@ def test_get_spark_type_basic_types( ], ) def test_get_spark_type_list_types( - dummy_kwargs, dummy_df, dummy_udf, return_type, expected_spark_type + return_type, expected_spark_type ): return_type = list[return_type] # type: ignore assert ( - h_spark.get_spark_type(dummy_kwargs, dummy_df, dummy_udf, return_type) + h_spark.get_spark_type(return_type) == expected_spark_type ) @@ -310,10 +302,10 @@ def test_get_spark_type_list_types( ], ) def test_get_spark_type_numpy_types( - dummy_kwargs, dummy_df, dummy_udf, return_type, expected_spark_type + return_type, expected_spark_type ): assert ( - h_spark.get_spark_type(dummy_kwargs, dummy_df, dummy_udf, return_type) + h_spark.get_spark_type(return_type) == expected_spark_type ) @@ -322,11 +314,11 @@ def test_get_spark_type_numpy_types( @pytest.mark.parametrize( "unsupported_return_type", [dict, set, tuple] # Add other unsupported types as needed ) -def test_get_spark_type_unsupported(dummy_kwargs, dummy_df, dummy_udf, unsupported_return_type): +def test_get_spark_type_unsupported(unsupported_return_type): with pytest.raises( ValueError, match=f"Currently unsupported return type {unsupported_return_type}." ): - h_spark.get_spark_type(dummy_kwargs, dummy_df, dummy_udf, unsupported_return_type) + h_spark.get_spark_type(unsupported_return_type) # Dummy values for the tests @@ -336,8 +328,8 @@ def dummy_kwargs(): @pytest.fixture -def dummy_df(): - return spark.createDataFrame(pd.DataFrame({"a": [1, 2, 3], "b": [4, 5, 6]})) +def dummy_df(spark_session): + return spark_session.createDataFrame(pd.DataFrame({"a": [1, 2, 3], "b": [4, 5, 6]})) @pytest.fixture @@ -346,7 +338,7 @@ def dummyfunc(x: int) -> int: return x return dummyfunc -======= + def test_base_spark_executor_end_to_end(spark_session): # TODO -- make this simpler to call, and not require all these constructs dr = ( @@ -361,12 +353,13 @@ def test_base_spark_executor_end_to_end(spark_session): df = dr.execute(["processed_df_as_pandas"], inputs={"spark_session": spark_session})[ "processed_df_as_pandas" ] - pd.testing.assert_series_equal( - df["a_times_key"], - pd.Series([2, 10, 24, 44, 70]), - check_dtype=False, - check_names=False, - ) + expected_data = { + "a_times_key": [2, 10, 24, 44, 70], + "b_times_key": [5, 16, 33, 56, 85], + "a_plus_b_plus_c": [10.5, 20.0, 29.5, 39.0, 48.5], + } + expected_df = pd.DataFrame(expected_data) + pd.testing.assert_frame_equal(df, expected_df, check_names=False, check_dtype=False) def test_base_spark_executor_end_to_end_external_dependencies(spark_session): @@ -377,9 +370,6 @@ def test_base_spark_executor_end_to_end_external_dependencies(spark_session): .with_adapter(base.SimplePythonGraphAdapter(base.DictResult())) .build() ) - # dr.visualize_execution( - # ["processed_df_as_pandas"], "./out", {}, inputs={"spark_session": spark_session} - # ) df = dr.execute(["processed_df_as_pandas"], inputs={"spark_session": spark_session})[ "processed_df_as_pandas" ] @@ -437,7 +427,9 @@ def _two_pyspark_dataframe_parameters(foo: DataFrame, bar: int, baz: DataFrame) ], ) def test_derive_dataframe_parameter_succeeds(fn, requested_parameter, expected): - assert h_spark.derive_dataframe_parameter(fn, requested_parameter) == expected + assert h_spark.derive_dataframe_parameter_from_fn(fn, requested_parameter) == expected + n = node.Node.from_fn(fn) + assert h_spark.derive_dataframe_parameter_from_node(n, requested_parameter) == expected @pytest.mark.parametrize( @@ -452,7 +444,9 @@ def test_derive_dataframe_parameter_succeeds(fn, requested_parameter, expected): ) def test_derive_dataframe_parameter_fails(fn, requested_parameter): with pytest.raises(ValueError): - h_spark.derive_dataframe_parameter(fn, requested_parameter) + h_spark.derive_dataframe_parameter_from_fn(fn, requested_parameter) + n = node.Node.from_fn(fn) + h_spark.derive_dataframe_parameter_from_node(n, requested_parameter) def test_prune_nodes_no_select(): @@ -580,4 +574,26 @@ def foo( "df_base", "df_upstream", } ->>>>>>> 7d0b719 (Implements spark integration, see #248) + + +def test_pyspark_udfs_end_to_end(spark_session): + # TODO -- make this simpler to call, and not require all these constructs + dr = ( + driver.Builder() + .with_modules(spark_dag_pyspark_udfs) + .with_adapter(base.SimplePythonGraphAdapter(base.DictResult())) + .build() + ) + dr.visualize_execution( + ["processed_df_as_pandas"], "./out", {}, inputs={"spark_session": spark_session} + ) + df = dr.execute(["processed_df_as_pandas"], inputs={"spark_session": spark_session})[ + "processed_df_as_pandas" + ] + expected_data = { + "a_times_key": [2, 10, 24, 44, 70], + "b_times_key": [5, 16, 33, 56, 85], + "a_plus_b_plus_c": [10.5, 20.0, 29.5, 39.0, 48.5], + } + expected_df = pd.DataFrame(expected_data) + pd.testing.assert_frame_equal(df, expected_df, check_names=False, check_dtype=False) diff --git a/hamilton/experimental/h_spark.py b/hamilton/experimental/h_spark.py index 863a5a3d1..2f4b24f99 100644 --- a/hamilton/experimental/h_spark.py +++ b/hamilton/experimental/h_spark.py @@ -3,12 +3,12 @@ import logging import sys from types import CodeType, FunctionType, ModuleType -from typing import Any, Callable, Dict, List, Optional, Set, Tuple, Type, Union +from typing import Any, Callable, Collection, Dict, List, Optional, Set, Tuple, Type, Union import numpy as np import pandas as pd import pyspark.pandas as ps -from pyspark.sql import DataFrame, dataframe, types +from pyspark.sql import Column, DataFrame, dataframe, types from pyspark.sql.functions import column, lit, pandas_udf, udf from hamilton import base, htypes, node @@ -600,45 +600,63 @@ def new_callable( return node_.copy_with(callabl=new_callable, input_types=new_input_types, typ=DataFrame) -def derive_dataframe_parameter(fn: Callable, requested_parameter: str = None) -> str: - """Utility function to grab a pyspark dataframe parameter from a function. - Note if one is supplied it'll look for that. If none is, it will look to ensure - that there is only one dataframe parameter in the function. - - :param fn: Function to grab the dataframe parameter from - :param requested_parameter: If supplied, the name of the parameter to grab - :return: The name of the dataframe parameter - :raises ValueError: If no datframe parameter is supplied: - - if no dataframe parameter is found, or if more than one is found - if a requested parameter is supplied: - - if the requested parameter is not found - """ - sig = inspect.signature(fn) +def derive_dataframe_parameter( + param_types: Dict[str, Type], requested_parameter: str, location_name: Callable +) -> str: dataframe_parameters = { - param.name: param - for param in sig.parameters.values() - if issubclass(param.annotation, DataFrame) + param for param, val in param_types.items() if issubclass(val, DataFrame) } if requested_parameter is not None: if requested_parameter not in dataframe_parameters: raise ValueError( - f"Requested parameter {requested_parameter} not found in " - f"function: {fn.__qualname__}" + f"Requested parameter {requested_parameter} not found in " f"{location_name}" ) return requested_parameter if len(dataframe_parameters) == 0: raise ValueError( - f"No dataframe parameters found in function: {fn.__qualname__}. " + f"No dataframe parameters found in: {location_name}. " f"@with_columns must inject a dataframe parameter into the function." ) elif len(dataframe_parameters) > 1: raise ValueError( - f"More than one dataframe parameter found in function: {fn.__qualname__}. Please " + f"More than one dataframe parameter found in function: {location_name}. Please " f"specify the desired one with the 'dataframe' parameter in @with_columns" ) return list(dataframe_parameters)[0] +def derive_dataframe_parameter_from_fn(fn: Callable, requested_parameter: str = None) -> str: + """Utility function to grab a pyspark dataframe parameter from a function. + Note if one is supplied it'll look for that. If none is, it will look to ensure + that there is only one dataframe parameter in the function. + + :param fn: Function to grab the dataframe parameter from + :param requested_parameter: If supplied, the name of the parameter to grab + :return: The name of the dataframe parameter + :raises ValueError: If no datframe parameter is supplied: + - if no dataframe parameter is found, or if more than one is found + if a requested parameter is supplied: + - if the requested parameter is not found + """ + sig = inspect.signature(fn) + parameters_with_types = {param.name: param.annotation for param in sig.parameters.values()} + return derive_dataframe_parameter(parameters_with_types, requested_parameter, fn.__qualname__) + + +def derive_dataframe_parameter_from_node(node_: node.Node, requested_parameter: str = None) -> str: + """Derives the only/requested dataframe parameter from a node. + + :param node_: + :param requested_parameter: + :return: + """ + types_ = {key: value[0] for key, value in node_.input_types.items()} + originating_function_name = ( + node_.originating_functions[-1] if node_.originating_functions is not None else node_.name + ) + return derive_dataframe_parameter(types_, requested_parameter, originating_function_name) + + def prune_nodes(nodes: List[node.Node], select: Optional[List[str]] = None) -> List[node.Node]: """Prunes the nodes to only include those upstream from the select columns. Conducts a depth-first search using the nodes `input_types` field. @@ -667,6 +685,141 @@ def prune_nodes(nodes: List[node.Node], select: Optional[List[str]] = None) -> L return output +class transforms(fm_base.NodeTransformer): + """Decorator for spark that allows for the specification of columns to transform. + These are columns within a specific node in a decorator, enabling the user to make use of pyspark + transformations inside a with_columns group. Note that this will have no impact if it is not + decorating a node inside `with_columns`. + + Note that this currently does not work with other decorators, but it definitely could. + """ + + TRANSFORM_TARGET_TAG = "hamilton.spark.target" + TRANSFORM_COLUMNS_TAG = "hamilton.spark.columns" + + def __init__(self, *columns: str, target_parameter=None): + super(transforms, self).__init__(target=None) + self._columns = columns + self._target = target_parameter + + def transform_node( + self, node_: node.Node, config: Dict[str, Any], fn: Callable + ) -> Collection[node.Node]: + """Generates nodes for the `@transforms` decorator. + + This does two things, but does not fully prepare the node: + 1. It adds the columns as dependencies to the node + 2. Adds tags with relevant metadata for later use + + Note that, at this point, we don't actually know which columns will come from the + base dataframe, and which will come from the upstream nodes. This is handled in the + `with_columns` decorator, so for now, we need to give it enough information to topologically + sort/assign dependencies. + + :param node_: Node to transform + :param config: Configuration to use (unused here) + :return: + """ + param = derive_dataframe_parameter_from_node(node_, self._target) + + # This allows for injection of any extra parameters + def new_fn(**kwargs): + return node_.callable( + **{key: value for key, value in kwargs.items() if key in node_.input_types} + ) + + # Add the upstream columns as additional dependencies + additional_input_types = { + param: (DataFrame, node.DependencyType.REQUIRED) for param in self._columns + } + node_out = node_.copy_with( + input_types={**node_.input_types, **additional_input_types}, + callabl=new_fn, + tags={ + transforms.TRANSFORM_TARGET_TAG: param, + transforms.TRANSFORM_COLUMNS_TAG: self._columns, + }, + ) + # if it returns a column, we just turn it into a withColumn expression + if issubclass(node_.type, Column): + + def transform_output(output: Column, kwargs: Dict[str, Any]) -> DataFrame: + return kwargs[param].withColumn(node_.name, output) + + node_out = node_out.transform_output(transform_output, DataFrame) + return [node_out] + + def validate(self, fn: Callable): + """Validates on the function, even though it operates on nodes. We can always loosen + this, but for now it should help the code stay readable. + + :param fn: Function this is decorating + :return: + """ + + derive_dataframe_parameter_from_fn(fn, self._target) + + @staticmethod + def _extract_dataframe_params(node_: node.Node) -> List[str]: + """Extracts the dataframe parameters from a node. + + :param node_: Node to extract from + :return: List of dataframe parameters + """ + return [key for key, value in node_.input_types.items() if issubclass(value[0], DataFrame)] + + @staticmethod + def is_default_pyspark_udf(node_: node.Node) -> bool: + """Tells if a node is, by default, a pyspark UDF. This means: + 1. It has a single dataframe parameter + 2. That parameter name determines an upstream column name + + :param node_: Node to check + :return: True if it functions as a default pyspark UDF, false otherwise + """ + df_columns = transforms._extract_dataframe_params(node_) + return len(df_columns) == 1 and len(node_.input_types) == 1 + + @staticmethod + def is_decorated_pyspark_udf(node_: node.Node): + """Tells if this is a decorated pyspark UDF. This means it has been + decorated by the `@transforms` decorator. + + :return: True if it can be run as part of a group, false otherwise + """ + if "hamilton.spark.columns" in node_.tags and "hamilton.spark.target" in node_.tags: + return True + return False + + @staticmethod + def sparkify_node( + node_: node.Node, + linear_df_dependency_name: str, + base_df_dependency_name: str, + dependent_columns_from_dataframe: Set[str], + ) -> node.Node: + """Transforms a pyspark node into a node that can be run as part of a `with_columns` group. + + :param node_: Node to transform + :param linear_df_dependency_name: Dependency on continaully modified dataframe (this will enable us + :param base_df_dependency_name: + :param dependent_columns_in_group: + :param dependent_columns_from_dataframe: + :return: The final node with correct dependencies + """ + transformation_target = node_.tags.get(transforms.TRANSFORM_TARGET_TAG) + + # This should come from the dataframe + # We have to reassign this to the linear dataframe dependency so we're transforming the + # right one Then we have to replace all the columns that are in the dataframe with the + # base dataframe dependency name + node_ = node_.reassign_input_names({transformation_target: linear_df_dependency_name}) + node_ = node_.reassign_input_names( + {col: base_df_dependency_name for col in dependent_columns_from_dataframe} + ) + return node_ + + class with_columns(fm_base.NodeCreator): def __init__( self, @@ -736,6 +889,25 @@ def final_df(df: ps.DataFrame) -> ps.DataFrame: self.namespace = namespace self.upstream_dependency = dataframe + @staticmethod + def _prep_nodes(initial_nodes: List[node.Node]) -> List[node.Node]: + """Prepares nodes by decorating "default" UDFs with transform. + This allows us to use the sparkify_node function in transforms + for both the default ones and the decorated ones. + + :param initial_nodes: + :return: + """ + out = [] + for node_ in initial_nodes: + if transforms.is_default_pyspark_udf(node_): + col = derive_dataframe_parameter_from_node(node_) + # todo -- wire through config/function correctly + # the col is the only dataframe paameter so it is the target node + (node_,) = transforms(col).transform_node(node_, {}, node_.callable) + out.append(node_) + return out + def generate_nodes(self, fn: Callable, config: Dict[str, Any]) -> List[node.Node]: """Generates nodes in the with_columns groups. This does the following: @@ -753,7 +925,8 @@ def generate_nodes(self, fn: Callable, config: Dict[str, Any]) -> List[node.Node namespace = fn.__name__ if self.namespace is None else self.namespace initial_nodes = subdag.collect_nodes(config, self.subdag_functions) - pruned_nodes = prune_nodes(initial_nodes, self.select) + transformed_nodes = with_columns._prep_nodes(initial_nodes) + pruned_nodes = prune_nodes(transformed_nodes, self.select) if len(pruned_nodes) == 0: raise ValueError( f"No nodes found upstream from select columns: {self.select} for function: " @@ -761,7 +934,7 @@ def generate_nodes(self, fn: Callable, config: Dict[str, Any]) -> List[node.Node ) sorted_initial_nodes = graph_functions.topologically_sort_nodes(pruned_nodes) output_nodes = [] - inject_parameter = derive_dataframe_parameter(fn, self.upstream_dependency) + inject_parameter = derive_dataframe_parameter_from_fn(fn, self.upstream_dependency) current_dataframe_node = inject_parameter # Columns that it is dependent on could be from the group of transforms created columns_produced_within_mapgroup = {node_.name for node_ in pruned_nodes} @@ -778,14 +951,23 @@ def generate_nodes(self, fn: Callable, config: Dict[str, Any]) -> List[node.Node dependent_columns_in_dataframe = { column for column in node_.input_types if column in columns_passed_in_from_dataframe } - - sparkified = sparkify_node( - node_, - current_dataframe_node, - inject_parameter, - dependent_columns_in_mapgroup, - dependent_columns_in_dataframe, - ) + # In the case that we are using pyspark UDFs + if transforms.is_decorated_pyspark_udf(node_): + sparkified = transforms.sparkify_node( + node_, + current_dataframe_node, + inject_parameter, + dependent_columns_in_dataframe, + ) + # otherwise we're using pandas/primitive UDFs + else: + sparkified = sparkify_node( + node_, + current_dataframe_node, + inject_parameter, + dependent_columns_in_mapgroup, + dependent_columns_in_dataframe, + ) output_nodes.append(sparkified) current_dataframe_node = sparkified.name # We get the final node, which is the function we're using @@ -797,4 +979,4 @@ def generate_nodes(self, fn: Callable, config: Dict[str, Any]) -> List[node.Node return output_nodes + [final_node] def validate(self, fn: Callable): - derive_dataframe_parameter(fn, self.upstream_dependency) + derive_dataframe_parameter_from_fn(fn, self.upstream_dependency) diff --git a/hamilton/node.py b/hamilton/node.py index c1ff5e709..4b4223604 100644 --- a/hamilton/node.py +++ b/hamilton/node.py @@ -320,3 +320,20 @@ def new_callable(**kwargs) -> Any: new_input_types = {input_names.get(k, k): v for k, v in self.input_types.items()} out = self.copy_with(callabl=new_callable, input_types=new_input_types) return out + + def transform_output( + self, __transform: Callable[[Dict[str, Any], Any], Any], __output_type: Type[Any] + ) -> "Node": + """Applies a transformation on the output of the node, returning a new node. + Also modifies the type. + + :param __transform: Transformation to apply. This is a function with two arguments: + (a) the kwargs passed to the node, and (b) the output of the node. + :param __output_type: Return type of the transformation + :return: A new node, with the right type/transformation + """ + + def new_callable(**kwargs) -> Any: + return __transform(self.callable(**kwargs), kwargs) + + return self.copy_with(callabl=new_callable, typ=__output_type) diff --git a/tests/resources/spark/spark_dag_pyspark_udfs.py b/tests/resources/spark/spark_dag_pyspark_udfs.py new file mode 100644 index 000000000..0573f9b59 --- /dev/null +++ b/tests/resources/spark/spark_dag_pyspark_udfs.py @@ -0,0 +1,56 @@ +from typing import Callable, List + +import pandas as pd +import pyspark.sql as ps + +from hamilton.experimental import h_spark +from hamilton.htypes import column as _ + +IntSeries = _[pd.Series, int] +FloatSeries = _[pd.Series, float] + + +def _module() -> List[Callable]: + def a(a_raw: ps.DataFrame) -> ps.DataFrame: + return a_raw.withColumn("a", a_raw.a_raw + 1) + + def b(b_raw: IntSeries) -> IntSeries: + return b_raw + 3 + + def c(c_raw: IntSeries) -> FloatSeries: + return c_raw * 3.5 + + @h_spark.transforms("a", "key") + def a_times_key(a_key: ps.DataFrame) -> ps.Column: + return a_key.a * a_key.key + + def b_times_key(b: IntSeries, key: IntSeries) -> IntSeries: + return b * key + + @h_spark.transforms("a", "b", "c") + def a_plus_b_plus_c(a_b_c: ps.DataFrame) -> ps.Column: + return a_b_c.a + a_b_c.b + a_b_c.c + + return [a, b, c, a_times_key, b_times_key, a_plus_b_plus_c] + + +def df_1(spark_session: ps.SparkSession) -> ps.DataFrame: + df = pd.DataFrame.from_records( + [ + {"key": 1, "a_raw": 1, "b_raw": 2, "c_raw": 1}, + {"key": 2, "a_raw": 4, "b_raw": 5, "c_raw": 2}, + {"key": 3, "a_raw": 7, "b_raw": 8, "c_raw": 3}, + {"key": 4, "a_raw": 10, "b_raw": 11, "c_raw": 4}, + {"key": 5, "a_raw": 13, "b_raw": 14, "c_raw": 5}, + ] + ) + return spark_session.createDataFrame(df) + + +@h_spark.with_columns( + *_module(), + select=["a_times_key", "b_times_key", "a_plus_b_plus_c"], + initial_schema=["a_raw", "b_raw", "c_raw", "key"], +) +def processed_df_as_pandas(df_1: ps.DataFrame) -> pd.DataFrame: + return df_1.select("a_times_key", "b_times_key", "a_plus_b_plus_c").toPandas() From 4744d3d3162ff692fa766e8541aceb36b1cdf988 Mon Sep 17 00:00:00 2001 From: elijahbenizzy Date: Mon, 7 Aug 2023 21:27:13 -0700 Subject: [PATCH 04/11] Adds derivation of dataframe columns in with_columns Dependencies within a with_columns group can come from one of three places: 1. Other columns inside that group 2. The upsteram dataframe 3. External places in the DAG Previously we were requiring users to specify (2) with the initial_schema kwarg, but we now allow them to specify (3) with the external_inputs kwarg. --- examples/spark/README.md | 7 ++ examples/spark/pyspark/README.md | 89 ++++++++++++++++++- graph_adapter_tests/h_spark/test_h_spark.py | 54 +++++++++-- hamilton/experimental/h_spark.py | 63 +++++++++++-- hamilton/htypes.py | 1 + .../spark/spark_dag_external_dependencies.py | 10 ++- .../resources/spark/spark_dag_pyspark_udfs.py | 8 +- tests/test_type_utils.py | 2 + 8 files changed, 212 insertions(+), 22 deletions(-) diff --git a/examples/spark/README.md b/examples/spark/README.md index da5684881..df7a588a5 100644 --- a/examples/spark/README.md +++ b/examples/spark/README.md @@ -1,4 +1,11 @@ + # Scaling Hamilton on Spark +## Pyspark + +If you're using pyspark, Hamilton allows for natural manipulation of pyspark dataframes, +with some special constructs for managing DAGs of UDFs. + +See the example in `pyspark` to learn more. ## Pandas If you're using Pandas, Hamilton scales by using Koalas on Spark. diff --git a/examples/spark/pyspark/README.md b/examples/spark/pyspark/README.md index 0f541e561..97ed5dc03 100644 --- a/examples/spark/pyspark/README.md +++ b/examples/spark/pyspark/README.md @@ -70,8 +70,92 @@ def final_result(all_initial_data: ps.DataFrame, raw_data_3: ps.DataFrame) -> ps ``` Thus you can represent a clean, modular, unit-testable string of transformations while also allowing for -complex sets of feature UDFs. Note that we will shortly allow the with_columns group to include pyspark -functions (dataframe -> dataframe) as well, but that is not currently supported. +complex sets of feature UDFs. Note that the following kinds of UDFs are all supported: + +All of these can rely on data passed from a node or parametr external to the group of functions. + +This takes in the following parameters (see the docstring for more info) +1. `load_from` -- a list of functions/modules to find the functions to load the DAG from, similar to `@subdag` +2. `initial_schema` -- not compatible with `external_inputs`. Dependencies specified from the initial dataframe, +injected in. +3. `external_inputs` -- not compatible with `initial_schema`. Dependencies specified from outside the UDF group, +somewhere else in the DAG. +4. `select` -- a list of columns to select from the UDF group. If not specified all will be selected. +5. `dataframe` -- the initial dataframe. If not specified, will default to the only dataframe param +in the decorated function (and error if there are multiple). +6. `namespace` -- the namespace of the nodes generated by this -- will deafult to the function name that is decorated. + +#### Pandas -> Pandas +These are functions of series: + +```python +from hamilton import htypes + +def foo(bar: pd.Series, baz: pd.Series) -> htypes.column[pd.Series, int]: + return bar + 1 +``` + +The rules are the same as vanilla hamilton -- the parameter name determines the upstream dependencies, +and the function name determines the output column name. + +Note that, due to the type-specification of pyspark, these have to return a "typed" (`Annotated[]`) series. + +These are adapted to form pyspark-friendly [pandas UDFs](https://spark.apache.org/docs/3.1.2/api/python/reference/api/pyspark.sql.functions.pandas_udf.html) + +#### Python primitives -> python primitives + +These are functions of python primitives: + +```python +def foo(bar: int, baz: int) -> int: + return bar + 1 +``` + +These are adapted to standard [pyspark UDFs](https://spark.apache.org/docs/3.1.3/api/python/reference/api/pyspark.sql.functions.udf.html). + +#### pyspark dataframe -> pyspark columns + +These are functions that take in a pyspark dataframe (single) and output a pyspark column. + +```python +def foo(bar: ps.DataFrame) -> ps.Column: + return df["bar"] + 1 +``` + +Note that these have two forms: +1. The dataframe specifies the name of the upstream column -- then you just access the column and return a manipulation +2. The dataframe contains more than one column + +```python +import h_spark + +@h_spark.transforms("bar", "baz") +def foo(bar_baz: ps.DataFrame) -> ps.Column: + return df["bar"] + 1 +``` + +In this case we are only allowed a single dataframe dependency, and the paraemter name does not matter. +These are an out for when the pyspark computational expression is more convenient than the pandas one, +or it is not possible to express it in pandas. + +#### pyspark dataframe -> pyspark dataframe + +This is the ultimate power-user case, where you can manipulate the dataframe in any way you want. +Note that this and the column-flavor is an _out_, meaning that its a way to jump back to the pyspark world and not have to break up +your map functions for a windowed aggregation. + +This has the exact same rules as the column flavor, except that the return type is a dataframe. + +```python +import h_spark + +@h_spark.transforms("bar", "baz") +def foo(df: ps.DataFrame) -> ps.DataFrame: + return df.withColumn("bar", df["bar"] + 1) +``` + +Note that this is the column-flavor in which you (not the framework) are responsible for calling `withColumn`. + We have implemented the hamilton hello_world example in [run.py](run.py) and the [map_transforms.py](map_transforms.py)/[dataflow.py](dataflow.py) files so you can compare. You can run `run.py`: @@ -80,6 +164,7 @@ so you can compare. You can run `run.py`: and check out the interactive example in the `notebook.ipynb` file. + ## How does this work? The `with_columns` decorator does the following: diff --git a/graph_adapter_tests/h_spark/test_h_spark.py b/graph_adapter_tests/h_spark/test_h_spark.py index e1f20fc61..6df7c7f01 100644 --- a/graph_adapter_tests/h_spark/test_h_spark.py +++ b/graph_adapter_tests/h_spark/test_h_spark.py @@ -370,12 +370,26 @@ def test_base_spark_executor_end_to_end_external_dependencies(spark_session): .with_adapter(base.SimplePythonGraphAdapter(base.DictResult())) .build() ) - df = dr.execute(["processed_df_as_pandas"], inputs={"spark_session": spark_session})[ - "processed_df_as_pandas" - ] - expected_data = {"a": [2, 3, 4, 5], "b": [4, 6, 8, 10]} - expected_df = pd.DataFrame(expected_data) - pd.testing.assert_frame_equal(df, expected_df, check_names=False, check_dtype=False) + dfs = dr.execute( + ["processed_df_as_pandas", "processed_df_as_pandas_with_external_inputs"], + inputs={"spark_session": spark_session}, + ) + + expected_df = pd.DataFrame({"a": [2, 3, 4, 5], "b": [4, 6, 8, 10]}) + processed_df_as_pandas = pd.DataFrame(dfs["processed_df_as_pandas"]) + pd.testing.assert_frame_equal( + processed_df_as_pandas, expected_df, check_names=False, check_dtype=False + ) + + processed_df_as_pandas_with_external_inputs = pd.DataFrame( + dfs["processed_df_as_pandas_with_external_inputs"] + ) + pd.testing.assert_frame_equal( + processed_df_as_pandas, + processed_df_as_pandas_with_external_inputs, + check_names=False, + check_dtype=False, + ) def test_base_spark_executor_end_to_end_multiple_with_columns(spark_session): @@ -584,9 +598,9 @@ def test_pyspark_udfs_end_to_end(spark_session): .with_adapter(base.SimplePythonGraphAdapter(base.DictResult())) .build() ) - dr.visualize_execution( - ["processed_df_as_pandas"], "./out", {}, inputs={"spark_session": spark_session} - ) + # dr.visualize_execution( + # ["processed_df_as_pandas"], "./out", {}, inputs={"spark_session": spark_session} + # ) df = dr.execute(["processed_df_as_pandas"], inputs={"spark_session": spark_session})[ "processed_df_as_pandas" ] @@ -597,3 +611,25 @@ def test_pyspark_udfs_end_to_end(spark_session): } expected_df = pd.DataFrame(expected_data) pd.testing.assert_frame_equal(df, expected_df, check_names=False, check_dtype=False) + + +# is default +def pyspark_fn_1(foo: DataFrame) -> DataFrame: + pass + + +# is default +def pyspark_fn_2(foo: DataFrame, bar: int) -> DataFrame: + pass + + +def not_pyspark_fn(foo: DataFrame, bar: DataFrame) -> DataFrame: + pass + + +@pytest.mark.parametrize( + "fn,expected", [(pyspark_fn_1, True), (pyspark_fn_2, True), (not_pyspark_fn, False)] +) +def test_is_default_pyspark_node(fn, expected): + node_ = node.Node.from_fn(fn) + assert h_spark.transforms.is_default_pyspark_udf(node_) == expected diff --git a/hamilton/experimental/h_spark.py b/hamilton/experimental/h_spark.py index 2f4b24f99..173f648c9 100644 --- a/hamilton/experimental/h_spark.py +++ b/hamilton/experimental/h_spark.py @@ -17,6 +17,7 @@ from hamilton.function_modifiers import base as fm_base from hamilton.function_modifiers import subdag from hamilton.function_modifiers.recursive import assign_namespace +from hamilton.htypes import custom_subclass_check logger = logging.getLogger(__name__) @@ -604,7 +605,7 @@ def derive_dataframe_parameter( param_types: Dict[str, Type], requested_parameter: str, location_name: Callable ) -> str: dataframe_parameters = { - param for param, val in param_types.items() if issubclass(val, DataFrame) + param for param, val in param_types.items() if custom_subclass_check(val, DataFrame) } if requested_parameter is not None: if requested_parameter not in dataframe_parameters: @@ -741,7 +742,7 @@ def new_fn(**kwargs): }, ) # if it returns a column, we just turn it into a withColumn expression - if issubclass(node_.type, Column): + if custom_subclass_check(node_.type, Column): def transform_output(output: Column, kwargs: Dict[str, Any]) -> DataFrame: return kwargs[param].withColumn(node_.name, output) @@ -766,7 +767,11 @@ def _extract_dataframe_params(node_: node.Node) -> List[str]: :param node_: Node to extract from :return: List of dataframe parameters """ - return [key for key, value in node_.input_types.items() if issubclass(value[0], DataFrame)] + return [ + key + for key, value in node_.input_types.items() + if custom_subclass_check(value[0], DataFrame) + ] @staticmethod def is_default_pyspark_udf(node_: node.Node) -> bool: @@ -778,7 +783,7 @@ def is_default_pyspark_udf(node_: node.Node) -> bool: :return: True if it functions as a default pyspark UDF, false otherwise """ df_columns = transforms._extract_dataframe_params(node_) - return len(df_columns) == 1 and len(node_.input_types) == 1 + return len(df_columns) == 1 @staticmethod def is_decorated_pyspark_udf(node_: node.Node): @@ -824,7 +829,8 @@ class with_columns(fm_base.NodeCreator): def __init__( self, *load_from: Union[Callable, ModuleType], - initial_schema: List[str], + initial_schema: List[str] = None, + external_inputs: List[str] = None, select: List[str] = None, dataframe: str = None, namespace: str = None, @@ -875,7 +881,12 @@ def final_df(df: ps.DataFrame) -> ps.DataFrame: keep all columns in :param initial_schema: The initial schema of the dataframe. This is used to determine which upstream inputs should be taken from the dataframe, and which shouldn't. Note that, if this is - left empty, we will assume that all upstream items come from the dataframe + left empty (and external_inputs is as well), we will assume that all dependencies come + from the dataframe. This cannot be used in conjunction with external_inputs. + :param external_inputs: All dependencies referred to within the DAG that are not part of the + DAG itself or the upstream dataframe. Note that, if this is left empty (and initial_schema is as well), + we will assume that all dependencies come from the dataframe. This cannot be used in conjunction + with initial_schema. :param namespace: The namespace of the nodes, so they don't clash with the global namespace and so this can be reused. If its left out, there will be no namespace (in which case you'll want to be careful about repeating it/reusing the nodes in other parts of the DAG.) @@ -886,6 +897,16 @@ def final_df(df: ps.DataFrame) -> ps.DataFrame: self.subdag_functions = subdag.collect_functions(load_from) self.select = select self.initial_schema = initial_schema + self.external_inputs = external_inputs + if self.initial_schema is not None and self.external_inputs is not None: + raise ValueError( + "You cannot specify both initial_schema and external_inputs. You are allowed to " + "\n1. specify neither (which means we will assume every uknown dependency in this subdag " + "comes from the upstream dataframe)" + "\n2. specify external_inputs (meaning that we will assume" + "everything not specified comes from the dataframe), \n3. specify initial_schema (meaning that" + "we will assume everything not specified comes from external inputs)." + ) self.namespace = namespace self.upstream_dependency = dataframe @@ -908,6 +929,33 @@ def _prep_nodes(initial_nodes: List[node.Node]) -> List[node.Node]: out.append(node_) return out + def derive_initial_schema(self, nodes: List[node.Node]) -> List[str]: + """Derives the dependency sources, which fill out `initial_schema` and `external_inputs`, + as only one of them is allowed to be specified. Note that: + + 1. If none are specified, everything missing is assumed to come from the dataframe + 2. If `initial_schema` is specified, everything missing not in that + is assumed to come from the external inputs + 3. If `external_inputs` is specified, everything missing not in that is + assumed to come from the dataframe + + :param nodes: Nodes resolved in the DAG + :return: The sources from the dataframe + """ + node_names = {node_.name for node_ in nodes} + all_dependencies = set() + for node_ in nodes: + all_dependencies.update(node_.input_types) + external_dependencies = all_dependencies - node_names + if self.initial_schema is not None: + initial_schema = self.initial_schema + elif self.external_inputs is not None: + external_inputs = self.external_inputs + initial_schema = list(external_dependencies - set(external_inputs)) + else: + initial_schema = list(external_dependencies) + return initial_schema + def generate_nodes(self, fn: Callable, config: Dict[str, Any]) -> List[node.Node]: """Generates nodes in the with_columns groups. This does the following: @@ -938,9 +986,8 @@ def generate_nodes(self, fn: Callable, config: Dict[str, Any]) -> List[node.Node current_dataframe_node = inject_parameter # Columns that it is dependent on could be from the group of transforms created columns_produced_within_mapgroup = {node_.name for node_ in pruned_nodes} - columns_passed_in_from_dataframe = set(self.initial_schema) + columns_passed_in_from_dataframe = set(self.derive_initial_schema(sorted_initial_nodes)) # Or from the dataframe passed in... - # potential_dependent_columns.update(self.initial_schema) for node_ in sorted_initial_nodes: # dependent columns are broken into two sets: # 1. Those that come from the group of transforms diff --git a/hamilton/htypes.py b/hamilton/htypes.py index e5f70fa9a..613a29c13 100644 --- a/hamilton/htypes.py +++ b/hamilton/htypes.py @@ -45,6 +45,7 @@ def custom_subclass_check(requested_type: Type, param_type: Type): """ # handles case when someone is using primitives and generics requested_origin_type = requested_type + param_type, _ = get_type_information(param_type) param_origin_type = param_type has_generic = False if _safe_subclass(requested_type, param_type): diff --git a/tests/resources/spark/spark_dag_external_dependencies.py b/tests/resources/spark/spark_dag_external_dependencies.py index 0a25224eb..677a7d364 100644 --- a/tests/resources/spark/spark_dag_external_dependencies.py +++ b/tests/resources/spark/spark_dag_external_dependencies.py @@ -38,8 +38,16 @@ def df_input(spark_session: ps.SparkSession) -> ps.DataFrame: @h_spark.with_columns( a, b, - select=["a", "b"], initial_schema=["initial_column"], ) def processed_df_as_pandas(df_input: ps.DataFrame) -> pd.DataFrame: return df_input.select("a", "b").toPandas() + + +@h_spark.with_columns( + a, + b, + external_inputs=["to_add", "to_multiply"], +) +def processed_df_as_pandas_with_external_inputs(df_input: ps.DataFrame) -> pd.DataFrame: + return df_input.select("a", "b").toPandas() diff --git a/tests/resources/spark/spark_dag_pyspark_udfs.py b/tests/resources/spark/spark_dag_pyspark_udfs.py index 0573f9b59..28f0821d2 100644 --- a/tests/resources/spark/spark_dag_pyspark_udfs.py +++ b/tests/resources/spark/spark_dag_pyspark_udfs.py @@ -10,9 +10,13 @@ FloatSeries = _[pd.Series, float] +def to_add() -> int: + return 1 + + def _module() -> List[Callable]: - def a(a_raw: ps.DataFrame) -> ps.DataFrame: - return a_raw.withColumn("a", a_raw.a_raw + 1) + def a(a_raw: ps.DataFrame, to_add: int) -> ps.DataFrame: + return a_raw.withColumn("a", a_raw.a_raw + to_add) def b(b_raw: IntSeries) -> IntSeries: return b_raw + 3 diff --git a/tests/test_type_utils.py b/tests/test_type_utils.py index 4670e9104..4e449d4a0 100644 --- a/tests/test_type_utils.py +++ b/tests/test_type_utils.py @@ -57,6 +57,8 @@ class Y(X): (typing.Dict, collections.Counter, True), # These are not subclasses of each other, see issue 42 (typing.FrozenSet[int], typing.Set[int], False), + (htypes.column[pd.Series, int], pd.Series, True), + (htypes.column[pd.Series, int], int, False), ], ) def test_custom_subclass_check(param_type, requested_type, expected): From 7b590835453fba57ea9427762701195e2f1632e2 Mon Sep 17 00:00:00 2001 From: elijahbenizzy Date: Tue, 8 Aug 2023 16:18:31 -0700 Subject: [PATCH 05/11] Adds mode= argument to with_columns append will just append to that dataframe select will just select the specified columns from that dataframe --- examples/spark/pyspark/README.md | 5 + graph_adapter_tests/h_spark/test_h_spark.py | 178 ++++++++++++++++-- hamilton/experimental/h_spark.py | 96 +++++++--- tests/resources/spark/basic_spark_dag.py | 21 ++- .../spark/spark_dag_external_dependencies.py | 6 +- .../resources/spark/spark_dag_pyspark_udfs.py | 17 +- 6 files changed, 276 insertions(+), 47 deletions(-) diff --git a/examples/spark/pyspark/README.md b/examples/spark/pyspark/README.md index 97ed5dc03..d0a0b3fc3 100644 --- a/examples/spark/pyspark/README.md +++ b/examples/spark/pyspark/README.md @@ -144,6 +144,11 @@ This is the ultimate power-user case, where you can manipulate the dataframe in Note that this and the column-flavor is an _out_, meaning that its a way to jump back to the pyspark world and not have to break up your map functions for a windowed aggregation. +Note that you can easily shoot yourself in the foot here, so be careful! This should only be used if +you strongly feel the need to inject a map-like (index-preserving, but not row-wise) operation into the DAG, +and the df -> column flavor is not sufficient (and if you find yourself using this a lot, please reach +out, we'd love to hear your use-case). + This has the exact same rules as the column flavor, except that the return type is a dataframe. ```python diff --git a/graph_adapter_tests/h_spark/test_h_spark.py b/graph_adapter_tests/h_spark/test_h_spark.py index 6df7c7f01..e0ff607ff 100644 --- a/graph_adapter_tests/h_spark/test_h_spark.py +++ b/graph_adapter_tests/h_spark/test_h_spark.py @@ -127,30 +127,38 @@ def test__inspect_kwargs(input_and_expected_fn, spark_session): assert h_spark._inspect_kwargs(input_) == expected -def test__get_pandas_annotations(): +def test__get_pandas_annotations_no_pandas(): """Unit test for _get_pandas_annotations().""" def no_pandas(a: int, b: float) -> float: return a * b + assert h_spark._get_pandas_annotations(node.Node.from_fn(no_pandas), {}) == { + "a": False, + "b": False, + } + + +def test__get_pandas_annotations_with_pandas(): def with_pandas(a: pd.Series) -> pd.Series: return a * 2 + assert h_spark._get_pandas_annotations(node.Node.from_fn(with_pandas), {}) == {"a": True} + + +def test__get_pandas_annotations_with_pandas_and_other_default(): def with_pandas_and_other_default(a: pd.Series, b: int) -> pd.Series: return a * b - # - def with_pandas_and_other_default_with_one_more(a: pd.Series, c: int, b: int = 2) -> pd.Series: - return a * b * c - - assert h_spark._get_pandas_annotations(node.Node.from_fn(no_pandas), {}) == { - "a": False, - "b": False, - } - assert h_spark._get_pandas_annotations(node.Node.from_fn(with_pandas), {}) == {"a": True} assert h_spark._get_pandas_annotations( node.Node.from_fn(with_pandas_and_other_default), {"b": 2} ) == {"a": True} + + +def test__get_pandas_annotations_with_pandas_and_other_default_and_one_more(): + def with_pandas_and_other_default_with_one_more(a: pd.Series, c: int, b: int = 2) -> pd.Series: + return a * b * c + assert h_spark._get_pandas_annotations( node.Node.from_fn(with_pandas_and_other_default_with_one_more), {"b": 2} ) == { @@ -163,7 +171,10 @@ def test__bind_parameters_to_callable(): """Unit test for _bind_parameters_to_callable().""" actual_kwargs = {"a": 1, "b": 2} df_columns = {"b"} - node_input_types = {"a": (int,), "b": (int,)} + node_input_types = { + "a": (int, node.DependencyType.REQUIRED), + "b": (int, node.DependencyType.REQUIRED), + } df_params, params_to_bind = h_spark._determine_parameters_to_bind( actual_kwargs, df_columns, node_input_types, "test" ) @@ -172,6 +183,40 @@ def test__bind_parameters_to_callable(): assert str(df_params["b"]) == str(column("b")) # hacky, but compare string representation. +def test__bind_parameters_to_callable_with_defaults_provided(): + """Unit test for _bind_parameters_to_callable().""" + actual_kwargs = {"a": 1, "b": 2, "c": 2} + df_columns = {"b"} + node_input_types = { + "a": (int, node.DependencyType.REQUIRED), + "b": (int, node.DependencyType.REQUIRED), + "c": (int, node.DependencyType.OPTIONAL), + } + df_params, params_to_bind = h_spark._determine_parameters_to_bind( + actual_kwargs, df_columns, node_input_types, "test" + ) + assert isinstance(df_params["b"], Column) + assert params_to_bind == {"a": 1, "c": 2} + assert str(df_params["b"]) == str(column("b")) # hacky, but compare string representation. + + +def test__bind_parameters_to_callable_with_defaults_not_provided(): + """Unit test for _bind_parameters_to_callable().""" + actual_kwargs = {"a": 1, "b": 2, "c": 2} + df_columns = {"b"} + node_input_types = { + "a": (int, node.DependencyType.REQUIRED), + "b": (int, node.DependencyType.REQUIRED), + "c": (int, node.DependencyType.OPTIONAL), + } + df_params, params_to_bind = h_spark._determine_parameters_to_bind( + actual_kwargs, df_columns, node_input_types, "test" + ) + assert isinstance(df_params["b"], Column) + assert params_to_bind == {"a": 1, "c": 2} + assert str(df_params["b"]) == str(column("b")) # hacky, but compare string representation. + + def test__lambda_udf_plain_func(spark_session): """Tests plain UDF function""" @@ -362,6 +407,29 @@ def test_base_spark_executor_end_to_end(spark_session): pd.testing.assert_frame_equal(df, expected_df, check_names=False, check_dtype=False) +def test_base_spark_executor_end_to_end_with_mode_select(spark_session): + # TODO -- make this simpler to call, and not require all these constructs + dr = ( + driver.Builder() + .with_modules(basic_spark_dag) + .with_adapter(base.SimplePythonGraphAdapter(base.DictResult())) + .with_config({"mode": "select"}) + .build() + ) + # dr.visualize_execution( + # ["processed_df_as_pandas"], "./out", {}, inputs={"spark_session": spark_session} + # ) + df = dr.execute(["processed_df_as_pandas"], inputs={"spark_session": spark_session})[ + "processed_df_as_pandas" + ] + expected_data = { + "a_times_key": [2, 10, 24, 44, 70], + "a_plus_b_plus_c": [10.5, 20.0, 29.5, 39.0, 48.5], + } + expected_df = pd.DataFrame(expected_data) + pd.testing.assert_frame_equal(df, expected_df, check_names=False, check_dtype=False) + + def test_base_spark_executor_end_to_end_external_dependencies(spark_session): # TODO -- make this simpler to call, and not require all these constructs dr = ( @@ -419,6 +487,10 @@ def test_base_spark_executor_end_to_end_multiple_with_columns(spark_session): pd.testing.assert_frame_equal(df, expected_df, check_names=False, check_dtype=False) +def _only_pyspark_dataframe_parameter(foo: DataFrame) -> DataFrame: + ... + + def _no_pyspark_dataframe_parameter(foo: int) -> int: ... @@ -434,6 +506,7 @@ def _two_pyspark_dataframe_parameters(foo: DataFrame, bar: int, baz: DataFrame) @pytest.mark.parametrize( "fn,requested_parameter,expected", [ + (_only_pyspark_dataframe_parameter, "foo", "foo"), (_one_pyspark_dataframe_parameter, "foo", "foo"), (_one_pyspark_dataframe_parameter, None, "foo"), (_two_pyspark_dataframe_parameters, "foo", "foo"), @@ -533,6 +606,24 @@ def df_as_pandas(df: DataFrame) -> pd.DataFrame: assert set(nodes_by_names.keys()) == {"df_as_pandas.c", "df_as_pandas"} +def test_with_columns_generate_nodes_select_mode_select(): + dec = h_spark.with_columns( + basic_spark_dag.a, + basic_spark_dag.b, + basic_spark_dag.c, + initial_schema=["a_raw", "b_raw", "c_raw", "key"], + select=["c"], + mode="select", + ) + + def df_as_pandas(df: DataFrame) -> pd.DataFrame: + return df.toPandas() + + nodes = dec.generate_nodes(df_as_pandas, {}) + nodes_by_names = {n.name: n for n in nodes} + assert set(nodes_by_names.keys()) == {"df_as_pandas.c", "df_as_pandas", "df_as_pandas._select"} + + def test_with_columns_generate_nodes_specify_namespace(): dec = h_spark.with_columns( basic_spark_dag.a, @@ -599,10 +690,18 @@ def test_pyspark_udfs_end_to_end(spark_session): .build() ) # dr.visualize_execution( - # ["processed_df_as_pandas"], "./out", {}, inputs={"spark_session": spark_session} + # ["processed_df_as_pandas", "processed_df_as_pandas_with_initial_schema"], + # "./out", + # {}, + # inputs={"spark_session": spark_session}, # ) - df = dr.execute(["processed_df_as_pandas"], inputs={"spark_session": spark_session})[ - "processed_df_as_pandas" + results = dr.execute( + ["processed_df_as_pandas", "processed_df_as_pandas_with_initial_schema"], + inputs={"spark_session": spark_session}, + ) + processed_df_as_pandas = results["processed_df_as_pandas"] + processed_df_as_pandas_with_initial_schema = results[ + "processed_df_as_pandas_with_initial_schema" ] expected_data = { "a_times_key": [2, 10, 24, 44, 70], @@ -610,7 +709,15 @@ def test_pyspark_udfs_end_to_end(spark_session): "a_plus_b_plus_c": [10.5, 20.0, 29.5, 39.0, 48.5], } expected_df = pd.DataFrame(expected_data) - pd.testing.assert_frame_equal(df, expected_df, check_names=False, check_dtype=False) + pd.testing.assert_frame_equal( + processed_df_as_pandas, expected_df, check_names=False, check_dtype=False + ) + pd.testing.assert_frame_equal( + processed_df_as_pandas_with_initial_schema, + expected_df, + check_names=False, + check_dtype=False, + ) # is default @@ -633,3 +740,44 @@ def not_pyspark_fn(foo: DataFrame, bar: DataFrame) -> DataFrame: def test_is_default_pyspark_node(fn, expected): node_ = node.Node.from_fn(fn) assert h_spark.transforms.is_default_pyspark_udf(node_) == expected + + +def fn_test_initial_schema_1(a: int, b: int) -> int: + return a + b + + +def fn_test_initial_schema_2(fn_test_initial_schema_1: int, c: int = 1) -> int: + return fn_test_initial_schema_1 + c + + +@pytest.mark.parametrize( + "initial_schema,external_inputs,expected", + [ + (None, None, {"a", "b"}), + (["a", "b"], None, {"a", "b"}), + (None, ["c", "a"], {"b"}), + ], +) +def test_derive_initial_schema(initial_schema, external_inputs, expected): + nodes = [ + node.Node.from_fn(fn_test_initial_schema_1), + node.Node.from_fn(fn_test_initial_schema_2), + ] + + assert ( + h_spark.with_columns.derive_initial_schema(nodes, initial_schema, external_inputs) + == expected + ) + + +def test_create_selector_node(spark_session): + selector_node = h_spark.with_columns.create_selector_node("foo", ["a", "b"], "select") + assert selector_node.name == "select" + pandas_df = pd.DataFrame( + {"a": [10, 10, 20, 40, 40, 50], "b": [1, 10, 50, 100, 200, 400], "c": [1, 2, 3, 4, 5, 6]} + ) + df = spark_session.createDataFrame(pandas_df) + transformed = selector_node(foo=df).toPandas() + pd.testing.assert_frame_equal( + transformed, pandas_df[["a", "b"]], check_names=False, check_dtype=False + ) diff --git a/hamilton/experimental/h_spark.py b/hamilton/experimental/h_spark.py index 173f648c9..caee39c16 100644 --- a/hamilton/experimental/h_spark.py +++ b/hamilton/experimental/h_spark.py @@ -237,8 +237,8 @@ def _get_pandas_annotations(node_: node.Node, bound_parameters: Dict[str, Any]) """ return { name: type_ == pd.Series - for name, (type_, _) in node_.input_types.items() - if name not in bound_parameters + for name, (type_, dep_type) in node_.input_types.items() + if name not in bound_parameters and dep_type == node.DependencyType.REQUIRED } @@ -265,16 +265,15 @@ def _determine_parameters_to_bind( """ params_from_df = {} bind_parameters = {} - for input_name in node_input_types.keys(): + for input_name, (type_, dep_type) in node_input_types.items(): if input_name in df_columns: params_from_df[input_name] = column(input_name) elif input_name in actual_kwargs and not isinstance(actual_kwargs[input_name], DataFrame): bind_parameters[input_name] = actual_kwargs[input_name] - elif node_input_types[input_name][1] == DependencyType.OPTIONAL: - pass - else: + elif dep_type == node.DependencyType.REQUIRED: raise ValueError( - f"Cannot satisfy {node_name} with input types {node_input_types} against a dataframe with " + f"Cannot satisfy {node_name} with input types {node_input_types} against a " + f"dataframe with " f"columns {df_columns} and input kwargs {actual_kwargs}." ) return params_from_df, bind_parameters @@ -616,6 +615,7 @@ def derive_dataframe_parameter( if len(dataframe_parameters) == 0: raise ValueError( f"No dataframe parameters found in: {location_name}. " + f"Received parameters: {param_types}. " f"@with_columns must inject a dataframe parameter into the function." ) elif len(dataframe_parameters) > 1: @@ -818,9 +818,11 @@ def sparkify_node( # We have to reassign this to the linear dataframe dependency so we're transforming the # right one Then we have to replace all the columns that are in the dataframe with the # base dataframe dependency name - node_ = node_.reassign_input_names({transformation_target: linear_df_dependency_name}) node_ = node_.reassign_input_names( - {col: base_df_dependency_name for col in dependent_columns_from_dataframe} + { + **{col: base_df_dependency_name for col in dependent_columns_from_dataframe}, + **{transformation_target: linear_df_dependency_name}, + } ) return node_ @@ -834,6 +836,7 @@ def __init__( select: List[str] = None, dataframe: str = None, namespace: str = None, + mode: str = "append", ): """Initializes a with_columns decorator for spark. This allows you to efficiently run groups of map operations on a dataframe, represented as pandas/primitives UDFs. This @@ -878,7 +881,7 @@ def final_df(df: ps.DataFrame) -> ps.DataFrame: :param load_from: The functions that will be used to generate the group of map operations. :param select: Columns to select from the transformation. If this is left blank it will - keep all columns in + keep all columns in the subdag. :param initial_schema: The initial schema of the dataframe. This is used to determine which upstream inputs should be taken from the dataframe, and which shouldn't. Note that, if this is left empty (and external_inputs is as well), we will assume that all dependencies come @@ -891,8 +894,14 @@ def final_df(df: ps.DataFrame) -> ps.DataFrame: and so this can be reused. If its left out, there will be no namespace (in which case you'll want to be careful about repeating it/reusing the nodes in other parts of the DAG.) :param dataframe: The name of the dataframe that we're modifying. If not provided, - this will assume that there is only one pyspark.DataFrame parameter to the decorated function, - and use that if there is more than one, we will error. + this will assume that there is only one pyspark.DataFrame parameter to the decorated function, + and use that if there is more than one, we will error. + :param mode: The mode of the operation. This can be either "append" or "select". + If it is "append", it will keep all columns in the dataframe. If it is "select", + it will only keep the columns in the dataframe from the `select` parameter. Note that, + if the `select` parameter is left blank, it will keep all columns in the dataframe + that are in the subdag (as that is the behavior of the `select` parameter. This + defaults to `append` """ self.subdag_functions = subdag.collect_functions(load_from) self.select = select @@ -909,6 +918,7 @@ def final_df(df: ps.DataFrame) -> ps.DataFrame: ) self.namespace = namespace self.upstream_dependency = dataframe + self.mode = mode @staticmethod def _prep_nodes(initial_nodes: List[node.Node]) -> List[node.Node]: @@ -929,7 +939,12 @@ def _prep_nodes(initial_nodes: List[node.Node]) -> List[node.Node]: out.append(node_) return out - def derive_initial_schema(self, nodes: List[node.Node]) -> List[str]: + @staticmethod + def derive_initial_schema( + nodes: List[node.Node], + initial_schema: Optional[List[str]], + external_inputs: Optional[List[str]], + ) -> Set[str]: """Derives the dependency sources, which fill out `initial_schema` and `external_inputs`, as only one of them is allowed to be specified. Note that: @@ -940,21 +955,49 @@ def derive_initial_schema(self, nodes: List[node.Node]) -> List[str]: assumed to come from the dataframe :param nodes: Nodes resolved in the DAG + :param initial_schema: The initial schema of the dataframe, a list of columns + :param external_inputs: The external inputs to the DAG, a list of columns :return: The sources from the dataframe """ node_names = {node_.name for node_ in nodes} all_dependencies = set() for node_ in nodes: - all_dependencies.update(node_.input_types) + for dependency_name, (_, dep_type) in node_.input_types.items(): + # Note that we do not support optional columns in the dataframe + # This makes the API simpler/clearer -- all optional + # columns should be handled upstream + if dep_type == node.DependencyType.REQUIRED: + all_dependencies.add(dependency_name) external_dependencies = all_dependencies - node_names - if self.initial_schema is not None: - initial_schema = self.initial_schema - elif self.external_inputs is not None: - external_inputs = self.external_inputs + if external_inputs is not None: initial_schema = list(external_dependencies - set(external_inputs)) - else: + elif initial_schema is None: initial_schema = list(external_dependencies) - return initial_schema + return set(initial_schema) + + @staticmethod + def create_selector_node( + upstream_name: str, columns: List[str], node_name: str = "select" + ) -> node.Node: + """Creates a selector node. The sole job of this is to select just the specified columns. + Note this is a utility function that's only called + + :param upstream_name: Name of the upstream dataframe node + :param columns: Columns to select + :param node_namespace: Namespace of the node + :param node_name: Name of the node to create + :return: + """ + + def new_callable(**kwargs) -> DataFrame: + return kwargs[upstream_name].select(*columns) + + return node.Node( + name=node_name, + typ=DataFrame, + callabl=new_callable, + input_types={upstream_name: DataFrame}, + ) def generate_nodes(self, fn: Callable, config: Dict[str, Any]) -> List[node.Node]: """Generates nodes in the with_columns groups. This does the following: @@ -986,7 +1029,9 @@ def generate_nodes(self, fn: Callable, config: Dict[str, Any]) -> List[node.Node current_dataframe_node = inject_parameter # Columns that it is dependent on could be from the group of transforms created columns_produced_within_mapgroup = {node_.name for node_ in pruned_nodes} - columns_passed_in_from_dataframe = set(self.derive_initial_schema(sorted_initial_nodes)) + columns_passed_in_from_dataframe = self.derive_initial_schema( + sorted_initial_nodes, self.initial_schema, self.external_inputs + ) # Or from the dataframe passed in... for node_ in sorted_initial_nodes: # dependent columns are broken into two sets: @@ -1019,6 +1064,15 @@ def generate_nodes(self, fn: Callable, config: Dict[str, Any]) -> List[node.Node current_dataframe_node = sparkified.name # We get the final node, which is the function we're using # and reassign inputs to be the dataframe + if self.mode == "select": + select_columns = ( + self.select if self.select is not None else [item.name for item in output_nodes] + ) + select_node = with_columns.create_selector_node( + upstream_name=current_dataframe_node, columns=select_columns, node_name="_select" + ) + output_nodes.append(select_node) + current_dataframe_node = select_node.name output_nodes = subdag.add_namespace(output_nodes, namespace) final_node = node.Node.from_fn(fn).reassign_input_names( {inject_parameter: assign_namespace(current_dataframe_node, namespace)} diff --git a/tests/resources/spark/basic_spark_dag.py b/tests/resources/spark/basic_spark_dag.py index 4f6d95a90..48eaecc14 100644 --- a/tests/resources/spark/basic_spark_dag.py +++ b/tests/resources/spark/basic_spark_dag.py @@ -2,6 +2,7 @@ import pyspark.sql as ps from hamilton.experimental import h_spark +from hamilton.function_modifiers import config from hamilton.htypes import column as _ IntSeries = _[pd.Series, int] @@ -55,5 +56,23 @@ def df_1(spark_session: ps.SparkSession) -> ps.DataFrame: select=["a_times_key", "b_times_key", "a_plus_b_plus_c"], initial_schema=["a_raw", "b_raw", "c_raw", "key"], ) -def processed_df_as_pandas(df_1: ps.DataFrame) -> pd.DataFrame: +@config.when_not(mode="select") +def processed_df_as_pandas__append(df_1: ps.DataFrame) -> pd.DataFrame: return df_1.select("a_times_key", "b_times_key", "a_plus_b_plus_c").toPandas() + + +@h_spark.with_columns( + a, + b, + c, + a_times_key, + b_times_key, + a_plus_b_plus_c, + select=["a_times_key", "a_plus_b_plus_c"], + initial_schema=["a_raw", "b_raw", "c_raw", "key"], + mode="select", +) +@config.when(mode="select") +def processed_df_as_pandas__select(df_1: ps.DataFrame) -> pd.DataFrame: + # This should have two columns + return df_1.toPandas() diff --git a/tests/resources/spark/spark_dag_external_dependencies.py b/tests/resources/spark/spark_dag_external_dependencies.py index 677a7d364..3de1469f4 100644 --- a/tests/resources/spark/spark_dag_external_dependencies.py +++ b/tests/resources/spark/spark_dag_external_dependencies.py @@ -7,15 +7,11 @@ IntSeries = _[pd.Series, int] -def to_add() -> int: - return 1 - - def to_multiply() -> int: return 2 -def a(initial_column: IntSeries, to_add: int) -> IntSeries: +def a(initial_column: IntSeries, to_add: int = 1) -> IntSeries: return initial_column + to_add diff --git a/tests/resources/spark/spark_dag_pyspark_udfs.py b/tests/resources/spark/spark_dag_pyspark_udfs.py index 28f0821d2..eefec6338 100644 --- a/tests/resources/spark/spark_dag_pyspark_udfs.py +++ b/tests/resources/spark/spark_dag_pyspark_udfs.py @@ -18,15 +18,15 @@ def _module() -> List[Callable]: def a(a_raw: ps.DataFrame, to_add: int) -> ps.DataFrame: return a_raw.withColumn("a", a_raw.a_raw + to_add) - def b(b_raw: IntSeries) -> IntSeries: - return b_raw + 3 + def b(b_raw: ps.DataFrame, b_add: int = 3) -> ps.Column: + return b_raw["b_raw"] + b_add def c(c_raw: IntSeries) -> FloatSeries: return c_raw * 3.5 @h_spark.transforms("a", "key") - def a_times_key(a_key: ps.DataFrame) -> ps.Column: - return a_key.a * a_key.key + def a_times_key(a_key: ps.DataFrame, identity_multiplier: int = 1) -> ps.Column: + return a_key.a * a_key.key * identity_multiplier def b_times_key(b: IntSeries, key: IntSeries) -> IntSeries: return b * key @@ -51,10 +51,17 @@ def df_1(spark_session: ps.SparkSession) -> ps.DataFrame: return spark_session.createDataFrame(df) +@h_spark.with_columns( + *_module(), select=["a_times_key", "b_times_key", "a_plus_b_plus_c"], external_inputs=["to_add"] +) +def processed_df_as_pandas(df_1: ps.DataFrame) -> pd.DataFrame: + return df_1.select("a_times_key", "b_times_key", "a_plus_b_plus_c").toPandas() + + @h_spark.with_columns( *_module(), select=["a_times_key", "b_times_key", "a_plus_b_plus_c"], initial_schema=["a_raw", "b_raw", "c_raw", "key"], ) -def processed_df_as_pandas(df_1: ps.DataFrame) -> pd.DataFrame: +def processed_df_as_pandas_with_initial_schema(df_1: ps.DataFrame) -> pd.DataFrame: return df_1.select("a_times_key", "b_times_key", "a_plus_b_plus_c").toPandas() From f86984762339fbd0d7d3a206692967ec388ae5e0 Mon Sep 17 00:00:00 2001 From: elijahbenizzy Date: Fri, 11 Aug 2023 18:22:59 -0700 Subject: [PATCH 06/11] Adds dataframe_subdag_param argument to with_columns constructor This enables you to extract columns from a cetnral dataframe. Note there are two available approaches: 1. Specify initial_schema to get a set of columns extracted already 2. Specify dataframe_subdag_param to extract them yourself --- examples/spark/pyspark/README.md | 107 ++++++-- .../spark/pyspark_udfs/my_spark_udf.dot.png | Bin 50177 -> 37352 bytes graph_adapter_tests/h_spark/test_h_spark.py | 97 +++---- hamilton/experimental/h_spark.py | 245 ++++++++++++------ hamilton/node.py | 4 +- .../spark/spark_dag_external_dependencies.py | 9 - .../spark_dag_mixed_pyspark_pandas_udfs.py | 95 +++++++ .../resources/spark/spark_dag_pyspark_udfs.py | 57 ++-- 8 files changed, 436 insertions(+), 178 deletions(-) create mode 100644 tests/resources/spark/spark_dag_mixed_pyspark_pandas_udfs.py diff --git a/examples/spark/pyspark/README.md b/examples/spark/pyspark/README.md index d0a0b3fc3..5156bb3f0 100644 --- a/examples/spark/pyspark/README.md +++ b/examples/spark/pyspark/README.md @@ -1,7 +1,7 @@ # Hamilton and Spark -Hamilton now has first-class pyspark integration. While we will likely be improving it as we go along, -this version is the first we're considering "stable" and moving out of "experimental" +Hamilton now has first-class pyspark integration! While we will likely be improving it as we go along, +this version is the first we're considering "stable" and we are planning an imminent move out of "experimental". # Design @@ -31,7 +31,8 @@ def raw_data_3() -> ps.DataFrame: ``` For the next case, we define transformations that are columnar/map-oriented in nature. -These are UDFs (either pandas or python) that get applied to the dataframe in a specific order: +These are UDFs (either pandas or python), or functions of pyspark constructs, that get applied +to the upstream dataframe in a specific order: ```python import pandas as pd @@ -39,15 +40,12 @@ import pandas as pd #map_transforms.py def column_3(column_1_from_dataframe: pd.Series) -> pd.Series: - """Transforms column 1 (from the dataframe) into column 3""" return _some_transform(column_1_from_dataframe) def column_4(column_2_from_dataframe: pd.Series) -> pd.Series: - """Transforms column 2 (from the dataframe) into column 4""" return _some_other_transform(column_2_from_dataframe) def column_5(column_3: pd.Series, column_4: pd.Series) -> pd.Series: - """is a combination of column_1_from_dataframe and column_2_from_dataframe""" return _yet_another_transform(column_3, column_4) ``` @@ -60,8 +58,8 @@ import map_transforms # file defined above @with_columns( map_transforms, # Load all the functions we defined above - select=["column_1_from_dataframe", "column_2_from_dataframe"], # calculate these - dataframe="all_initial_data" # use this dataframe as the source (and inject the result into the final_result function + initial_schema=["column_1_from_dataframe", "column_2_from_dataframe", "column_3_from_dataframe"], # use these from the initial datafrmae + dataframe="all_initial_data" # use this dataframe as the source (and inject as a parameter with the same name) ) def final_result(all_initial_data: ps.DataFrame, raw_data_3: ps.DataFrame) -> ps.DataFrame: """Gives the final result. This decorator will apply the transformations in the order. @@ -69,23 +67,74 @@ def final_result(all_initial_data: ps.DataFrame, raw_data_3: ps.DataFrame) -> ps return _join(all_initial_data, raw_data_3) ``` -Thus you can represent a clean, modular, unit-testable string of transformations while also allowing for -complex sets of feature UDFs. Note that the following kinds of UDFs are all supported: +`with_columns` serves to _linearize_ the operation, enabling you to define a DAG, and have them all operate on a single dataframe. -All of these can rely on data passed from a node or parametr external to the group of functions. +You can thus represent a clean, modular, unit-testable string of transformations while also allowing for +complex sets of feature UDFs. -This takes in the following parameters (see the docstring for more info) +All of these can rely on data passed from a node or parameter external to the group of functions. + +You have two options when specifying the initial dataframe/how to read it. You can: + +1. Specify the columns in the initial dataframe, then refer to them in your functions (as in the example above) +2. Specify the initial dataframe as a parameter to the function, and then create the columns drawing from that. See the following: + +```python +import pandas as pd, pyspark.sql as ps + +#map_transforms.py + +def colums_1_from_dataframe(input_dataframe: ps.DataFrame) -> ps.Column: + return input_dataframe.column_1_from_dataframe + +def column_2_from_dataframe(input_dataframe: ps.DataFrame) -> ps.Column: + return input_dataframe.column_2_from_dataframe + +def column_3(column_1_from_dataframe: pd.Series) -> pd.Series: + return _some_transform(column_1_from_dataframe) + +def column_4(column_2_from_dataframe: pd.Series) -> pd.Series: + return _some_other_transform(column_2_from_dataframe) + +def column_5(column_3: pd.Series, column_4: pd.Series) -> pd.Series: + return _yet_another_transform(column_3, column_4) +``` + +```python +from hamilton.experimental.h_spark import with_columns +import pyspark.sql as ps +import map_transforms # file defined above + +@with_columns( + map_transforms, # Load all the functions we defined above + dataframe_subdag_param="input_dataframe", #the upstream dataframe, referred to by downstream nodes, will have this parametter name + dataframe="all_initial_data" # use this dataframe as the source (and inject as a parameter with the same name) +) +def final_result(all_initial_data: ps.DataFrame, raw_data_3: ps.DataFrame) -> ps.DataFrame: + """Gives the final result. This decorator will apply the transformations in the order. + Then, the final_result function is called, with the result of the transformations passed in.""" + return _join(all_initial_data, raw_data_3) +``` + +This requires functions that take in pyspark dataframes and return pyspark dataframes or columns, for those reading directly from the dataframe. +If you want to stay in pandas entirely for the `with_columns` group, you should approach (1). + + +`with_columns` takes in the following parameters (see the docstring for more info) 1. `load_from` -- a list of functions/modules to find the functions to load the DAG from, similar to `@subdag` 2. `initial_schema` -- not compatible with `external_inputs`. Dependencies specified from the initial dataframe, -injected in. -3. `external_inputs` -- not compatible with `initial_schema`. Dependencies specified from outside the UDF group, -somewhere else in the DAG. +injected in. Not that you must use one of this +3. `dataframe_subdag_param` -- the name of the parameter to inject the initial dataframe into the subdag. +If this is provided, this must be the only pyspark dataframe dependency in the subdag that is not also another +node (column) in the subdag. 4. `select` -- a list of columns to select from the UDF group. If not specified all will be selected. 5. `dataframe` -- the initial dataframe. If not specified, will default to the only dataframe param in the decorated function (and error if there are multiple). -6. `namespace` -- the namespace of the nodes generated by this -- will deafult to the function name that is decorated. +6. `namespace` -- the namespace of the nodes generated by this -- will default to the function name that is decorated. -#### Pandas -> Pandas +There are four flavors of transforms supported. + +#### Pandas -> Pandas UDFs These are functions of series: ```python @@ -98,11 +147,11 @@ def foo(bar: pd.Series, baz: pd.Series) -> htypes.column[pd.Series, int]: The rules are the same as vanilla hamilton -- the parameter name determines the upstream dependencies, and the function name determines the output column name. -Note that, due to the type-specification of pyspark, these have to return a "typed" (`Annotated[]`) series. +Note that, due to the type-specification of pyspark, these have to return a "typed" (`Annotated[]`) series, specified by `htypes.column`. These are adapted to form pyspark-friendly [pandas UDFs](https://spark.apache.org/docs/3.1.2/api/python/reference/api/pyspark.sql.functions.pandas_udf.html) -#### Python primitives -> python primitives +#### Python primitives -> python primitives UDFs These are functions of python primitives: @@ -124,14 +173,15 @@ def foo(bar: ps.DataFrame) -> ps.Column: Note that these have two forms: 1. The dataframe specifies the name of the upstream column -- then you just access the column and return a manipulation -2. The dataframe contains more than one column +2. The dataframe contains more than one column, in which case you need the `@require(...)` decorator, to specify which column you want to use. ```python import h_spark -@h_spark.transforms("bar", "baz") + +@h_spark.require_columns("bar", "baz") def foo(bar_baz: ps.DataFrame) -> ps.Column: - return df["bar"] + 1 + return df["bar"] + 1 ``` In this case we are only allowed a single dataframe dependency, and the paraemter name does not matter. @@ -154,9 +204,10 @@ This has the exact same rules as the column flavor, except that the return type ```python import h_spark -@h_spark.transforms("bar", "baz") + +@h_spark.require_columns("bar", "baz") def foo(df: ps.DataFrame) -> ps.DataFrame: - return df.withColumn("bar", df["bar"] + 1) + return df.withColumn("bar", df["bar"] + 1) ``` Note that this is the column-flavor in which you (not the framework) are responsible for calling `withColumn`. @@ -218,3 +269,11 @@ Note there are other scaling libraries that Hamilton supports -- it all depends - [dask](../../dask/README.md) - [ray](../../ray/README.md) - [modin](https://github.com/modin-project/modin) (no example for modin yet but it is just the pandas API with a different import) + +## Why are there so many lines in the visualizations for a with_columns group? + +Good question! This is because we are adding two sets of edges: +1. The single dataframe being passed through and updated (immutably, which is why it fits so nicely with Hamilton) +2. The original edges, dependent on columns + +We are planning, shortly, to display these edges differently, and ideally allow a visualization mode for these specifically (to show the DAG with (1), (2), or both). diff --git a/examples/spark/pyspark_udfs/my_spark_udf.dot.png b/examples/spark/pyspark_udfs/my_spark_udf.dot.png index 35f92814c59bc23d4c2b01f2600d8cebf633d3ce..985947a8dfc8bc8fef7e5570276b790beece3067 100644 GIT binary patch literal 37352 zcmZ7ecRZJW|2~dCrI4};$;fI*gpiSy5osY=*&38WW_FonHzZ}ND3nMjTO?an$x1~e zl#%^C9Tk6v|pg{QDvu z4gNmb)H#oT&{!YQRHv+x|0kC`xlN()Q4Xsg)c1J!tM{sD#QduAO0C@quA4$H)rD^e zoDNGDk2$Mn%+=G)V8eXwXWXEq@wxYJow>aB1%F6XJS(ccb| z{r~%|^yB)neE;t!wGzH@=CzD~(T-3JnW;_IJM2 zXED{GzqGyd{^!kgVv;w{KU4vIw@|XXSf_zkk10 zJ9JP@je6fc{bxJASca!0CsTIp*derMkJ{nG>z_V-s$*h8PqFPURcvT%9J9C3(eV#q z;$1`0*Vj+RkDfYpD&OuS4SvqHr{GeFpJrhG+TFW%zpt-X^YvY!O_JHaKR7b-c*6M5 z*RRnrF}h(;Hy$jG2jie1NOL`Ab7Z#YWFf9!^|s)`DQT4%WV&hC@AsRD<+iHV6b zKmJTlr_VNaCx+I}j`w`R?{&YA5j0+1SvGD?R5V_k9lw9`COvhzs+!u^C5Q9p4JVvR zeV08;#x*0kcbgcPnJ9T zy^yC*FJuo33k&n@+C?i@R7~rj=s8U%CnqN)B&4FDvF^i%4?0Fhp)X(V`}Y02)%Oo~ zM_SV~6UGq=k=@-!EPbaVveXo+e1_p1xjyMHu32x)KE zPk-`6jgL7r@uZ<)El$npN5g%@;P9~EzI~fWsOjon!}C0I_H0T{4&%W1$X03TSMoiE zhK6Z{g&Y)I09}esXj_}!o!Hpa^z^m3s2BP9S$RITwn95kB!;J^3bfk%`SSN?gb?dccY`4d+_w_$_Y@#B#ryAB-SGB!4TUsqR=WA;Yeq3^AHk3&xZ1O7-r za4^m1f=e4*U1v-tXx-f0#HFP0`EG9eZ-#}XJbg-k_3HdwqMr=slf91~J<7l}|IJ0n zbS`@Ot}I=xe)lc~7y9nqJ0W4=P0PqSo4-FAIV8^=O&H%IV&s4N_2q*{jvSf(m}{fy zGSWhoaKJq{JNr_}L`YzCX9PX%CjZ)@ivE7yoSYmyBt zjeL~hbMwLP_4l-n9n-jQLD+NdcZ5UPazT^Zi6kWrRn;{O4Gqapp6sfrPM15Sug|w* zhlW1KHU$syU`8I3;Xletlhin#U~)wA^BtX?!LhOI!A!i2)YQ}xm#?`LIS#JDCH3zq z#xv)al-xYrln{`XW|J_Ehq9hB@+ohZi>vF;KL=Dz?lZA7js$AHjep}^|CMM3gd`X^y+ZOy!8OGvIwt@ZRstLxs?$w4(ljjDwO zZ*g()^yg_Y)FF}-Sy}c8^*CbMOFab|WxmUn*4Aq_Z{EC-Tb6nJ^Q+A^dTS_1ve3{_ z6&DvNTq_G3+m3kZs`hr)sBMa0``z1$64KfnzEv_gJ6km8eIT=h*F*=Y0+%mePQFGL82y-JeM^guy1M#6 zMF6eG^azXao;|IPk8vS1Y~Dxn&lK=V*+$3R(oQP4)YG}BV6*XJXBIgsYa1IDR@T6* zED?&zL|=)-nfJ`x@-7*DOgGc*^iQ2iu_*J+$jJ#t=->&9`!0FlmaNMCsZx*7yblpI zEoA=q_*hk%W|Vr+z507%Lv#Ipny#+WZ{NNhK<#pEFc6iI*>d*m*^$q$E|%24z517C zd10oi`o?CBT@P_8E_H0RqkB}!oQG-{&YU^JCns0bdE)5Nji|y5nwqtq8)!Q^I;MTq zLRr(w%vH7WioNIe#qKq`NkvUlj&nV7Z|^(YZ|fU(2hJbe4IT>Hel#apjO8{o998g` zq%dt$D1T=7d_?VG9IDp<3hYSl8<{(I?i5A}-fi#bu(Gi^dGYvdR2ua|hdK}3Rdc=FE1qhbaU~b#*o0o;@4LE%y7Znw@`>dGd)h>ZSEj?RXP}Xls%4u;;K;<`INI za#q$xJaG~))U=F&MMYNwG#ph!&m^A+4~%ZsLz|p<{pjP96SY~!dFi9_zhaM`KFu{$ z8&&&nYKY5p|BfB&e0_Z#9UaSCQq(eA{_?7(Fu!)_uL;_)#c}GZ#;?z>HlPZ*?(KhW zp)B$EA`k9@({)Xrp_+&Q^n|WfEgr-|h~)VuO8^DZDV{ys^Z0neO~kD; z&#|Y|zk9iqm!~(=Fmi|DF{9@MjcKWCPkJaEy}xhW$;U@o6k}sz_&JMi#l{}epjO?^ zefP_H{)j&V71SaYrQE&6u2sng0|thNX}P$#etpUd`Jrx?^6S^H$*=D=pe~b^6&exo z4k6f@qDCuw;K0!D-Z!S2tj`d>X()0W`_9nqH7)eN`u7LFlG3&li3$}cbiqnK3ywT7 z?G^R)8}Lg9y}cDkE8DPTuO+f;q~qBZ-}&!ba)z9cb{o(PYjDf0Wqy8iL6!3NI@;PQ zW%|9TQSya3qnkU6oX>~zDsKa@=|<$1)GK+-ZtVA66ull0plfWr`*C1kv|kzv?Isl= z*0!Tl>ZuXi6gev@D$s(eAIdq4JAGA;d+@-SCoJ?vZmvXlL_{v24mzr!kPzLj6N&2P z=G(ejtF=+#Ztpr#15}o=dSo|%i2avB7A~2~>WFVri_sf$Y;JVC(Y4tkW?nu2cbG&x z@LE+(4GkI(dRppL+0g2`4ppX=ynRs-#!q3210M2j(Ie^~aAN-GxqqH4$93t_rFq`E&HN*REZ2baj1HNyQ%+ zeWHd#&Di*UdqQ}7dwYv(LAu5U1_q5cZsX#X= zF??&TQ4p-vILpGwwljz%%X8-tcmgwb?%tIESo_uc=H5-3z)-4Vky=`NW)Wrxs;cg8 zj^`F-Kjazh4x$C#)CglAtmX9!%*@=KV_H<{n$GkzC#U9l*$O>s20qBDI+Vp!Q()6} zK$7+A*NaO^2Hd;1<>|9$MUnbX0JhNMEC(xt2;@t}4G%2LjG5N*sgLRo`bQgtN(76Z zE-1%WBCln=W<>zJDqTg|uG`w#S)q85-j7JBiR1|rD_h!BR8(ZDx%n#KBDu08zm=`t zO1$n9eL+D%>$-A7*S6xxEHBPYeLSITg$K>C-=4#~)Vt0#%LKtaHTJwr^~4D#w2x~K zA97!~aG@|#BeWV%l7<}3<;#(%xL0^Wm@_p{K}_zyeXAfVCpY*c$+z-^@^WBp6t4{m zkZG-es6HB-)1PmL*V57oqR8NRqqz>qvkM3ah`WsF;4b;(uCA`m z3e6)lGBQ|&k5tJiqf-uT{kIp4-+ro!JKY&OxdVQIPm6a90Z$jne<%OKVpwuAU(-uuqD@=+VMiAi zb<8I*(b4`9Tr#O@A&d{t3aH;=K!130BB=(bpP7qm1Jcst>ebc}ZdIj9FZWY?Uu%b^ zn3krCHRN)&f(rHe5@{*o}C{M)Qt)!E6uWv{7! z_(q?5+FEK#ZyYE8P1z{;E){m#0FBmU=%y*{i7TwGt+fH7q+?~d{2H*~-@?LIbk>Zltm;4C>m|-LaO7q^kl5*zX8=5p6y2(=t!-`Leq_{; z_f}F86KbG`hsVIP=cQle8DqpnOIDq~16HCCibZZyyople$fKDRsG3s!@R!c&4RvmeQm@_9UHpnag2&@J6qNk_lkarQH->_j2760_y}zI(S0gHEH8 zg7V(Ij5~MkM4!;;FZFrgYQ~#}Is0Mun@7{niZL!6Jb7|6KK9sx*Wvn>wnb&bTeojJ zAyo{n8gd-o%}Hr5dvN69@BXhy2S(&~jt#Sbbe~t(dOEs)BQk<&YHHJ?9UBz3F-e|l z7_c81254eR(fOx)2_dK{#>vaqSUa77nlpfdL7BXUQ^xQZiSqbSdYSve`sJs4N!}su z9wsC_n4Y^8o3>-lFDIN^Zc}V*Eb2n?N-uYY-)7bAhi~6~cK6O5fS7vYbxTO~3j8&H z_!0pH@5XXr<><)UwyExB;*L=X#cUO1sQ&I zJ*lyhCyB~_j20FaOG|&7wY0U#tS;~NEA&&|`7m?3+qZ5p14<&YzXDhky8PUfIKL(8MF!+Si{y z7yIX}L<+;de;(e8ir`%5>gtMH#tbK2I3DpSr{-5u@Sn}U^8;7%8HWt3~I{*l{smpzrrZj6fZ2Xp{XiuIxbru!ioMzOvvnaPVn5{Qb zoZZ~cg-aOd>sOOc!r(OG{^ZG%JXGD>;|cN-H5_jcj1MGiXsFAZl9ZcCi{r5QHQMoP zBjxlPw@W3TdV70~QSUzmGjMgIbl(ROq%J?#^l-Pyx%ofe8$N%I|NiaU3)JR)+oEbQ zJ#g5B^C)@8xsJBqpt8h8WB3TCvb3_YiumkKi7PAfON(7*1%PUJ|9&K`=kSLwE>Lv~cZP^}meIupk<=HUX`Cq^6 zA3uNI3?`=mrJx&_b}OZ`#LFG*Qb)!mhXyz4bF1z5k}f(AH=QNmu1lgib;s;#&1@OU z|2{x;x9=a=W=C2D&skenPS4C31OBJYQrm>Hv9i9 zdw2O^l|bv-HB@`V#D?V$rRl8xDD~q#;1=%svn`0q5+P~dnPqHW6TbDRfkCX#{P%mm z5YzS;SN{7o5FT>!@|_nf`rX70R)+^`F7M)3mzO?0=r6HzA$sX|dW1v`8eAQ6R(;jY z+1Umh&@TWK>+ljEL4pCO`8-=K3ha%uWj54sJW%#49cjESQ3JZ_Jw-f4N60g)xHz61 z`S%|`Iv?P9CuC-3=7xsS;po`d*c$r!5~(bKug``{JWffe0O7Fxf{nd>12Bm2!W>GV zm@v0^O*dfHg|nGa`K*i_w-ap^_G%_6`CL%k@3Y{tA7h9~)Uo^ff_CrOlLvTkNbDo? z;oaQ$w-+)FeNx3@Hv{r#pFMplOxU(fn>TAjWgF$HPOsY8*-fx98swPPVdlH9!e<#S z5gQ->f%1ZZYrl7Ufy}u#oS&5$z7Mw{Bt8MIJ{X_F{2#xSlk+8t<&#gHtwy4+R%Lc3 z1q4tZ@L9;uXnb_!-h~7msdLIA#xDbp#Y<%x6kynKHjxF=I^WdPBy!-u8)M(q-Z$=B zDYNK`peK&qSa^MAdcu8iuz$?t<(c)=QT#+RZJTxdqhN76+2tpq^7SHO7!f` zTv}fKbZf_v!pMW6)fYO$S};6PD3OtoskKLs9=*`W_zWcoMFxn5y1b`nZV4Tq;y*B8 zN#RYDE0pMJ24=Ue3|tSg)+eC;f}I`VASlTh8A6w6M;wV2W{f-|7;S4d4Z5Avrz|(Eq71!HkepFqp{!rdkds;3z6X~(8yr*lk zo%-_WrRBxm#h8eQh>=`i7HUdXR+f4c9`qUt2s28uZcl3qGK1UUFx#e0*W=1%rX}h#oyKwiO=9yd`OzGfcS#XOO zJe}enDYdmdUzsagwL(q+p89~c`8W8{&i{F~fwWNeISU;_YiIetUO*!Tvn0W+m} z{!yFPB@&3|neW5e$B#3jo#cAXjGj9UURQV{1B26W6VK6yvbmo+|7Mz~t~vAbfLX@KrLw)9(7CAigScxB=?nUKS!r^* ztG-7b;!KNIdS}&MXL!5c+kH)iLW%es8LY%4^W7_67FcW$v#?RF^|~%yx)`I0Wmo5A znvHLL_)qHV2PP(NJHJ&*N($tOh`g2Q8#htLZ3>}2i?dc>g!NpbEP>k4!H-L7RZ3)M zW>QTS_)j`a^l>w6+{pRrs13-i00|l!o1x|pOGRjj0d==`Rn^u$(Kybx^S7@whBF1f z74AKTIb`OgVc-nJ|K#tj&h|J+oc+L>W%h<0UA+ozO2Ye}Bf&J*b&LJWe{<*GS)o~h z6?`@`uJ25y29Vskr%rWbjj`-HJh8gkH1~V$4q@TZlJB5{xn8Zf*PlIe##?pahc1|F z%oO_lyqf6R0H`Wc6*iw$mCe}@Mbmj2X>VZ5$fn*UoAsTx}C`P^~f z>(@0kaYjJ8`t1wu9v-q7wIn+$awMXNYACwyD(3E8&D3QqK(laqwKy*=~R ztvA3|TK#N$^y>1&uHVbsk8Pidi@$&Wpsp?py5*~F%}q@-6r?*hN*XtoYxSlss@%Y&_hy=;t%PF#VGn zB)f;FXQy!Ane*p|5IzwfCNCS!yh-`CT64;(y{Cr@Ou12^qg7r`ZEr7+;GuB;ygbRi zw^u)AjeVt`arnLHuGZ74r_6+2z=>J?U1#>`2Qx3_RMX9FH#+H6D=%^o4fMDKQa|YM=`B)lx8rh=A=+2i(Ux84#6pjEpok8&_6VFx+V3qA^0~ zo;)eYd7K?*il`?X1P=Pzvu9&oyRPx-85ry=_T4}MeHsk7Th-LWc;!oOBi|Yd zK+h!5i+D|Re7tbyU`rIQGEGu*Er^Mfv@|@pTQM;QF&?0(Nmy5hnk*XM(pilSUs+C> zOLI>;x%y_sPpk_a@5f(9o2b|6XfM@hy}95sixnmK8RoxyVzaTf?tXc;J}x|bBd}sa zTt$v~$%m$ga#h9d6HSyCYP!$ zkD;L|b`hg;)OjOq?VGkd5NjH$s{^SlKY#xG3-gYB#r1Wl18(*+qa7kBM~@r{%Ujt!zWg*J zBb0z4g2QnW1TvCqB6t)*-;K#=E9D$87MgA~hmDAkP$(o>3Cb(q>FH^u3q4$UW(?y= z3xC2M=m?!b)jG!`?_vp>#rpJfpd=vaEDQlz$;m4AV`sC@&iU0CrTr~>^CoPC5kN{K z@3oAZG8p|}S3PuQfiDyPf&;DV|$wE+SlJK6f!1z3D$>Y-1x>eDP*s zXAd}Rw{P!Ra7(HM@pa*|z~4-u=6G z*{I7OAH81*dXei3W6kKQ!k6yxJFGb0qaXo&{x-^Na-G|)y1nE47DlD6FGbGxQM;aH zWt{<+(U88dGyKH#bl7b+{;axHf+;@Q{eG@Y6nP6+D%lB&M30BFDcJ-5wK4 z3LNv@!AfszXSaKgu&{MZOiUK?yP^T`9K4s!e=&x8#1nU0UT~cSCnyfqf~Ln014F)_ z(hfgodDl@^ucd#v-CYk75*pnm`b1}EeD|X3L^RfR_t>E1&zlEFO)bqk$Hv9I4^-tp za{PFX>j60@AyHABx~L5}3~Mtn8wZE2y=^+HIT2$kjYi8ot+$kzl&}5?9HR@=V%TQ$ z#XL^Rp-;iI)VruNJWIW}RpPEG<{T)%_d*ZrD97f{JR5FKs%mT7MBsDpBSG-mh^gSD zB#YbGCqBY#8)JW~&u_aTFMnXD8mjSRCbHpzSwIKpl!wAv8k#FSX>~35 zY!Eei#!{5@=N6A1KNi4;<(+G+%s%}Z)#E0ZXtaKYe(%2wP{JO#__P)Hy|0a#9@_JB zKJ?7(=@r4B^K|^`+_HPP!GHbgE8!wl4;_$}+cL=A1}>Tqcyg|A0ha&*0AmGt3!b8|7N-@Vv>@Y zUGLBw_FRog_Xetc@%pvH6$$3oGvyw zbt!QQ`!~>GH}c5e854>i3?HxWKPs=;Uk9PEf%%be z{&BSX)2D-ng8v?T{7q84=0dMP?%pd`u0$hW)CB7E(GW0BRW>&>y}sP53V7^=-lxzJ9e{0$d^jEVNAs z>#1|UefHL2Hhgiek#qXjr_B(?O7y*Lj>Mb$`YP?W@1{UXA|-50mRAD+6lvozGs@-* zVF}z@RYe7rr>Cb;-Z_RRuLX7_g}`&7tD9Sq)M+FIa7;%}fBnG&E@VI;5Qi!RGRzQ_ z5Xmw@*l$jur{Z5r0ee9E7Mut=sS0!n0xvL3KUBE-Qn&v~bmti6Wx@iZ!a*J$938!h zyBWeCf_n8;-aHKjkjOf?TY`c-r$?@JJTnyBz5B+Pt@=jf{$#Ive+tT8_iG1UP)Mq( zs)dm=hslg-|7joORX%=xs=V{fH8(czdf~I^mXpTP_U(A0!i~eXck&~um6Vh`{{CRV z00$;C&OkSRm%3?&qY531F8H-njj-srxKP{=z)5*W2OGu-S+`#tT~ncJ7kf)9zCpx<;_!M<;q&XI4k!YEy zvjJmTf%#fj7{pZR>@H!V8U!ds{H8yCj2L=exAU){Xgw4)f}J;V?GGOlI;L%A#sh}l zsKiqmlpif<_qU!i+D~)bcSSfO`ScA8zGA3$ymIA2u3E`eg>C2?V`k?7WQ0US0$;t7 z282ycOA~zT64<_zm42#Od36BT&g0)CzqgOi825jos-mKz#IEh>GlZ~DH%FU$2WZki za0M-1Pg*2F3cLd+k!_!wmyDV|hF)!bbi41%moMWKuI_^wq`?&;8VSGno}a6&EiEcI zDHwdh02#r;9?@6lkg{bC4-Yr7w+eE61~Um-Mxpz}Hb@3R7;cGU3J4qOs_MamRA6~( zU5~1oNDFgoQBriYx-3~TL94Gr(FiaGZ{iqTPMNIcNTH8F|7FEOeZHNRbae=o*M=fq1L#02JG;X{YbEeR)s2ieDl01;$2vv7 zym4m=mw=3x+E+<+H3`EfH#hgdzyJkio-|Bho?$e=wJ<{*fPQIp=1d^I`{PB;J%$)% zp}#3y{j)ZJhT*CE^k>ZJDVdpv?{PB=RCt6-%udj*p{6Ezy{Z7iSVU}MVkB}VFeqpc zKdA#h5!4n4IHTfFBJwZwup`*2P`<4$T+rtIr>uMV@?OBmo6*tCKnb)rZrn(fQ?tF^ z`lEuQ3Ppki*grm=lY^795}j{$b{2xX|KrCyyI4*K0frhG8S#mVGEkB+8z7-DeiEGz zN&$zmuR`ta!_J4bw91KeHvRJ?s+}MT4H6D?$Te`wQmpXeOf1N z!aKYKpm9ZFwdxPfO)XMO(-(vnsnE-vnm z5X`VVV{aecSK_6H3Lk!BJ%8ghdfGtdCNp@D@GC?*fxm^K0x=pNh<;m(8Yb>FYkyfy z`$gje89{MzW)M6=yLVGf&MN)J{RiE+u^uGd4fqkSJiOI@zBZDF%t6o>=)k)Z3jqxc z4Rkkkm@QgBwBqxE$Ko0vW57YRB$X^t;p%mes{-JTh(?dIL-|JhB4z(<4OF!%>3Z7# z_Uh(>f&xlkUq?FJBgB3Hq)6;e@EWwA_7+UqwSPZ5#CFoO{?APZ7>f&_#>`0iR>9?7 zlk;BjeD|Ycqe`0u1064admqCgcV#<;Pg$9_IZ0VlPw&>q6MnQEO-Dy@=m{|GFaTo0 zfLGe-x8v&0N$9y}&Ycs0!46U^aY&+1{-0s%zU2A;d`bmf5Mj4icvGR9wlbOR5;jAL zQm<`9)kn6dh2favc{CpMJbr0uHh3hpPMpxhNJ$tsqys8O$7xv< zno5`kk~1^c5nv6bPztS`lv{{5ozOJw&l5 zR#0sX4UIzY`6~&{URO--ZM?FoDkVLA`+@;Vjq&tB+sT0n4slDBgz*VfKZpcsLaeE%$rP9Yh~WhQ z2nV$lrNFhI|AiGbxDAcfSokE7HuCPjH=3E5!Q) zRm=P8PzWnKJ7%#h4CYVD&epRNFAt>!-mKOew^3{eZv{$VC`_z0BhLpV^y}BJsLwjj zCLxanfB2vc_NMCn`;4Ypu+h8FW7V( zdnV5Y^c0z?fAyE~Voc$k@l;7a`C$L07FbiJC;C@!e+KQFisyzhm3sR-C<-7Q3I!4` zlp~Atiw|TRtYIJ^2Z4?{qBzj&*Vx#I>T7a-5r>FEuA-(kR3dN~&QS0QV?5HExwyih znGka+p4O|C(*U7RGpxXB;QXC`Y%GOev9%1zl#-Id?yp#0ZmUUPeibL7*)0~2p=?-I4xi12V`UfvVq4>A>5qTt+Wm6<~l!Ctlp z8p@dI3huuejX>AjJQBZ4QNfe|7X{VZb~qt9`Fc&b1h?FkwZ!s(CQxAi={j-l{z<=s2+}%ym?c3R z;uU!uIAZvvt5BInMa@Bqxy!nYL}%zsM#7Lv9y0>J%S+E`pqwf&Bt#8o1{|Gw`>tP7 zzW0YE24cfOnCP0CZWI#}^Io3ml5*NLFGN06Sok{37V@w~3}2a>=&yxWO2dyDm7t}) zT^*l;iQm$}fk{m4xqMHOtkW7;A&`cxUjHcph)keai-hrj>(?ij=lazUqWA+C^|mY< z9y`WBIwYvb|9QHRKQNT3n3*N~VC4&xl8YZorh?{5zR?NNfsu*H6`m&=b$pYh8lk=>K91HR%RE~ zcs&%$SP|pvpx$*`{h3u)f(~&@sh&E;Rujb=RahuX+AM6)1M+%d;-1sC8PA>#LBNI? zC;hP8fdgp(jfOdf&6_s?$Z|0<+(>+JQg8j*HA^dt6O+@^f$-tX zPV_VV=L(2num1FDD=6d88#hwwYv}mDS=d$|gpEsx=;LHS1ZaZjTj#nx^Ik$i1WI=7dU~7S?~{NljJTE%6NGDLV^E4ArBFcuHGDlXapTr~YsWgX zH#Z(reLpPgG0BIh2mpxt-~9&A(?+D{-!6b`kSb;kiM>~su_;WLH)QO0M?>)&U3Zjnd_m`r8 z{CugLG-ipzM~(!%d2;{^QtLBAF-ZCT&BG^Klkfi=85sahnVg%eg`L65);1JskP}as zRt%;V;u@0m{wL1HK*yiXs4x3(Epfs^#3Hg5rrj$%nxovxzTBguqa2E!QV>h*_M?z$ z;Nsz;M4-1dK>J1eA!r#gm}#-gdPExzXh`1V8G6QbXkM^dRUo8b#ndkpWbpj+Z9OJ( ze?l9eL7LWL)8vN6Jz?))7?r#s5O<@Y5h4>jGco>xnH(s;Mor?{xG~|tmZ_;JG6{9* zYDpHkjFEKhG|egFP_ zM|bxS7@r{yvheUQC?%cHpk6NllLm1oK?XB|oQhB)pc@gkKJV(^bHvXDb{9w@Fq%eZ za6qy#UV6jooZo85sb>Zi&CO9`vYVP=Ng-oSUfz&AJ%1yyT8+Sqef*(p1rA#s#2OUZ z{{lHUxDre_rZhDO5QUM4R?+~kC;$9WRaak&1}g!_q5Y?qK|gj1Q2a+mZ0-BrZUqE} z7kyBkkv~vNp;$lVMwWRAC%jmdn6!z%Bxgt~3y9P5-{fF-iI)tXHcJ=~!Vqd93~oQ< z>F1B4E|5C|C>A68*0nt?EGim=!UfI>M?S9( zI|->FG)&tzqH?HfYF731Jh(}tN1Ubj-#^gXEd`fOgP3C7W00i+s#81R>K{z331rkEfkdSgTX8u*Vzj9yTPH&a?V2#&fhy}dE_?$x?VwKZrTIU<3{ zMpjJ> zgoC7uk=K{CrQzh{EcBQXK(T=12gt3`b$iqN;-VT%n*p>;2rYhV86>;QL>~`64qrS& zXg64wP+M|c#|%Jn5l=HBtr43{>xcn5aQ)_i(NTH~4e5^_p`c0S;+sjSAh8ShU+v2M zf8V|n+doj&$YD!d{c{Ec&=sB=CqaJ4Df!4V{N0hg3A!SxDbAa*I40uTVVJc*0PQ^f z5Sr5=qWUAElIx$E#JG)rw!$<~=({YBheEmxsIIpDiyLAdJP1Q&w}H{c(aGt5DYiqH zTPh86%^+}0q2Hg2oDv# z*nvnY;qxeptXdCq7nxP5X*R+=%>PgVPF|d?Sq>jR8Ajoc#o$y?RYgs>WVkFzu~Y#f zANc+~2=YTRrdfD62ma234mH&jOd0?Sc*xR{j{qh7ThcsJnlQi?2VoYTn{QNBsT!H^^*F&}7lk(G;a5CZsv;b-m5G zbHT8Zl5%po!2TLcz@P(APX4g8ef*ep&Xvt3TBpR5`lySbX5>@`T?% zj5oR1hRYcWZ3MQBiH)Vfju!dyBG~Hq@ulv+zYNN2KNDo88yFvF0A~VJFx2+w2njie z|6Q%2Yq5tI5E4SWGC#~qcGL(Oy~(x*_P-CvElLdsRyv}fx>nZLhXH3Y3(RCV@*N_S zKGuy0Ck?tB-yZ~L>cZRvlZ}nde>ab>0s?INPH*i7ItqSy8?;-(Ct%@)^bUeW+R(iL zqRN3%U@=2pzr!d=xT7glxUV1X?wOpK2?BNX-;Kj+tV?)0SwVR8KnbSV z$l;IDTLs{5hW7a+|;;NAE?gUua2>Pb5VG+6+Qw6_=42 zme>Z2~Oeb+9s%0Nm55F|Y90_%fRmjH#x0bq2T^g}O1R~?pT zx3ju_{W@td82l{J;#?jmuZQtHI?2vo~o31akJurHwsvx5*pP*m>LHTG}90%O& z!%8}C?%n(r>Fkuj}{x}uxekp!oZb#B>$T|bcJ~4KZFCWu7hQWpGqomZc(#?}k zA#68>NgTKYB&HYE@8}7s3H(i{(bunKa9iluCNJ<6)})-kr=FFS zel0E0V@BD;(GDXI@T)%tBS7$20+-rSTSBQLL1hfWqC#dsY{Z2Xy(5R7m7NVua?IMh)30wio|_&{SJ47dhODGmpk7 z+*kNCvIPfkC6Gl3+BeqU(68x8WaFWdjTtiRk-9=G4O#5t06FAfut!Vw|8j=FgoddH zQLncGqey}qLPnhAdB*fX`c2nVCM#|y0?MNR(eWOlhIF$WRX;ZTPKv1W(C>$$Vs;rW3WNOG`XSz?$+1>1CN`Uhmc0|yQu z!<`>SGuL(C^SeKPzPZ;Bet2~q9cB>da16-vNlM-x3cR&r_wMzu@x@_H>QFexH30T3 zqulGk+c)2Ul{5+lKEmT*2Wtx|8Tro~o5Km{q{K4A>wOs%%KL_fYCz|o`*vVFKBO-j zEslW(Bm*>S zVm>zJAt=Kg@sm2ZeaySy-pK~PsVR4aiU9UTlXG8evIjd9Cw-Bf^b%-**fLw=abR~! zR1_2FDA~rY7M)QvJsfTY`VTgnC}f2Y1qW8gt^v`{CZmt&p&`7;&8+|o>-RbiEfdfi zn=r`#S`0BPtgPx9)NNmh`U17O0=F`S1@!5$E+&Ad)VX;k=;|yss&#NXnqF-G`o;&z zO;ba7&%&D+ShBV2m(F}vh4Ll{!9z7=vV@7uArJehOH&Sq!s*ls_8*y|a`XmWj5oaT zs=zdSVqz8(S449i!RS$=Ky2T+)4J{!d40iVML53x{}2FaD2{Ql(Lc^#x)gztEz7iM z6QTD|@PW;h3V#8@k+Ltb>sudYTxE|ZnP6%mtXN`7Bu#YK;xVe$k*+^V_Y0W8nPdID z8ksQqR(dF8=Lyy)3&-tYD8WTVp>{z4x*@^Ff9n{SI*jaC>b0MA$arLoZ6jbp%G(lC zvb%)b4#cp5Nj4=rn_Lt~EXX5H4`Z39P{N713iusf&-EaWCg+z-(r%G#!=MEH2Zb-7 zD*x*#%-bz3E%KVD(f9l%f~SEZzGU8PTWd0A*f<$$s5nqpdmB9IYv;k@4?#R2r|Ts zZC4Ht_5*J~WN^L{$y}#nX2yUR^6nSg z00{{7W0x=|vtAVZJwqrTS#uK^kH~H-F1z3BA^Zi9sIeA@M!42t;%Ghq0tQ+#l2wH; zg(M{KAc^=`>G_%Ih}Z##oFmUE5h#Qv=dYU@%EBlKU^>;MF*Nt>oHyjeleJ6Bb z?Zum-oVUMJ|D*YvTEl8{G-N-Ep=4c~!BfAisSfDg=-w?*qGi$TygIqhP~dE9 zknzC0uaftj-rdcKy>bWs+BAi$rU=a#)xBc|#7%kf#Bk_-=!MO9wSxfFh9Gn{+Ye#1 z2nG*$v`%Ul+zQy`#L~X#Ul$t;k)+Dq-g)d6KTpi;Ce?woD5Xl&-Fk0du>7to29YqQ zRD+XBl#@Jyc@b;@JWI5A5yGC6@&c&~kA5#5eioUVmsbV+=`{XXI((ZVCnml}Il`u@ zhK5_vC-3nO=55hz|M)Qg>@F5sjivbX$lfm+6b+0{=L;r2>{T-1BCkZi0E067eDdZb zhGb1p84NoG+k8=4{=uVKzogUw2@E*O5#O`*R!2Bk12Q>0x=FL+yfvYxV5k=1WM=8$ zV`t1aMHwK0L+qF(o-=E77Ff?c(z^jX|KW=CpkI=9_OoYU#A8F;`coq6ukHemU~b-^ zs92)+?)qmX{eyJZwo0Dk+qds!<;7QY{AAb0&u^9UsAl>y+#J|74A>WTS(OfcMk~|+ zB2EFP8s!-S4Xg2zfz)R`=2{Xgs(zAGKR_URc~pON^W7P#<{C&E!LfA_)ad zug$O>=R}lzZ2y@J&JrUv1Y(dPv0gD9pGBmk^ovWc;mpZP8nE9pxj$Q7eV^4}a_kg1 zD>$J^1B5t*9VfL%dV>!fN^v45c)Q>JZ8`wc#FdEid{6g~*JxlT_()mT4@pLHP|C_> zdVZqb@tBm_{iH5^W=%U3QF%Ve$3QG+(+U0iP&kf0*q{AneArI=(I)Ko0&!_DY=34< z%gC()do3jG$@$@=A;c3pN6Z0s7TuC}VS=63$Y>+@zGKBKBFPg*reiR9o;$PWt~Qb3 zae5d6z8zO?-JmCYcci~enJ8dz8JH9uBU&>8mJAAk%#fHvnAac&ZCjG5;1FfR3`N%L z@B@Jf2?<|fZsco&Do2YB0j?>G82F>+>N9`gWy>)A<2AP~M_BsY(i^v-yp%$<^9vJK}1fgQsU<8=c zL{m6>;5OOhc$sx?M1yt9mLVi7-d<7#Tu!D8v>2@}_1#-PAgne3tC^N~vH=is8oy>< z18=0z-o1x_rtWC(c$t?+rBF0!XKvmn`4?@BPe$f-!*@`tcZ-(qDdeQ3QKP58CqNh% zLMnh1rjwMEjCi|E;HeRGbS!mQ{Qe87gpGX)TmK)puh@cx#tfsHqLU2#!C9yMC*xjF zz~=(Vs3+(^?|A}73qAjx(sYexUIQ_`mR$X4`)+PUPgWvfVZ)Jm z4cXfVa3lt5z+wVA!5-iOANhiXz^h75jph_KYh@ zymD901dS;Jg`0bgztZb@3nRnPW5=v;DS*k|lNGezn^v)1i?2?_MS*7!b(i<_ZSL!T zo(P!u%_3PV4M!#B9%vG_Zp+iXfC!inQol%qnj67W2UrA?B$-$Xi;FA4l@fT2^wJ4CX`-+|BgZ44M>eH?3T(_Mb#v5t(8cVz(7l3xbt!b-o+e^=M;-4|<@ ze%d7(TN_VcxMSCM-+}aHOaTV29r`6Hr-zo6y$Vt-QypBtd}XgFeXA?5^K4_snd6m7Xy=dTG!fQ=-;sWYGYqHL_1j98l2v{?BUz{R2y`w;oGN zNd;nO$YrtzKJ%aD-5`{+O*v5X z%?)SH=gZivoy5}yru_=8i0CJPTJ;J2c&y;#+pb-%n>~GnVt@Iv`QQTCFo3{4RrFWJ zzMGk>xc#2Ty3B_!O&vT<_6`nF$TFq5g_5^#nYV4*n~h=UKP{qEzmE-w0@HE0Pql<{A`$o`lDUT;NShk_UT zfCY+GJp2Nia{%US6i5oI5W+PT`GXG zZ(^sB7_abrzQ!$ao`L>{Cz*yfPkldr60b|cE&eQd3r7R6Zv>=QVpwbQ`dsSc1?q^* zgKcF*KAj9PA&w06cMTVpX-Q?644_dxo3w69katyuu$28)a1#~kB|#1UO9{N_CL`Zs zW%Swe7jEMwct;1R$;7$_Y8?b1tN|9gl(vX)GqbR;DDDC-g4+;xwFVxgT12DM-yg?- zYbszaLh-=mU&ky6@nGQBuZ?Je=nHi`3GGOe?QCuLP?khCbsR)% z!S_CODF(Ztrm0B_6DE&acjSQ2%}r1s}j55=nl~Q$bKrO?ap2TMriG z;k$V0ityt5Yto0bqHZ#P6qbd@ADhGB zn7=T&Q$x1G(^5fU2XnU(%addbc zN)V_$$o$07gRWN>S2|k^j+3bSuqg`w=0Hc($<&9NGYGT^nki(rQ@@u$COxaj*YDcR zdUM}~u(QUnOk`L+6IVbg2oRwu;@Zh`+NQD9q#SeAj!D*>e?V7%5|)jPb}PXPY>pR;&3 z7d5D5@(Kx}sdsmGzvqd_P$l3IFMlAAkKE1n?Q5SIWM9{Qqzbj!X?jHW)!u(52}ZJf zC`X>~r7@6KV!$uyI5}_-#Kx-tnzI_Xn z>GPm2zn#K}7$m+tyanaVak|Ne@<_0!?gS^=ecVajtpw+b$5{63XviISMFR11Vt_F! z^_IiByOC5fCzK0nOwIP)FQ_005^jiKa9n>C7*zaWxQc1uG+)mj_{Hg2f?j6z>dKt* z+nLVO#Hed%;d5$m8m(A_QzrpAK;$G)F@{FsSc$8`HV&q!R~H(qDyeXp9UD~Xh+#HD zrq3UY2ckX*m2C){@hUKlgOVUV&Frlwv}Yc`97#O#WMvJn=0kz6fJ2G6WFcOYB7^I{ z8uXejD}s>k?i1!mlazRHtsF|;vhY|)fQSsoixhu*sGAI<78;fLDq?Y=6)KzepSoR; zE}qtO{ogg&A-u@@SGak2TCpEwTI#(u$!B&eHh#!bET(&q%=*9M)Y&e4e9X}2x9UT3 zmkdD=(a0`BcTbNEgb9c->0ekZx^cvoc6K+APsCA9;2r7h@J$f7eW8~ zbGs*A#<-W*x>@k1IYbGB6KlL8ZKUftFAnotgei~6L0{-LtpHoSF%oY}eN#0ccn5MKK8W4t?~>F?wupcyTBBN<$YHkf!l zh9YF}^0}SvQin0+A)&rS9@KY$Gdc~pl+&2~f1f$_9rHdEFJf*M5fO2OlMFBGdRWDh z(f1S9JupOsCxaH3jLHK2IQ9TLA(Y@a{hIx{@74i&mStroUZrX*47JC{{_m-D5J7`@ zD1a@=<_=kILF;=Dzl{NSM&ei{O&N)94U-5;pmZWxVgi3$gStdwEbH{^oEk}H-A}~Z z>@xnD4MiZs#e|(vD~-&5klzth^*lithdSOh`@+sl7E{pR>+nmafGO~eQecZjSG@+g z^Pr}t=5HgD%)J=Mq54WLO@BPmoo}}p^I^d2KF?=|!f1iXM|xh1k;;$U;fJySoRpC} z^Nb`H#Mj+3&bWypn-0$_Gcxfe4n9A{ma$3Vx8mK9sK`ZF5;ClQ9PP5)`*a;L&hpHe z=`Vl#OT4xaO9dR=G$&c?%8LJe>gwv!1kHnY7z((vrtZTl8LW{TC}Wi$WnGxkAJ`NnxFb4K>8-?Dy;da7T01$Pa7=CCxxJWfZBkj83|uF zNf#{9ygYNYb=mQfp4sxTvzO8{XFBh#w6(M_6Sc+xE523Aix5r&~2UinoDKgThW$L~Wr6y~;Uql2gi-f`yfl(GSiZ>6tj0)SoMd zdiS@rdybvkG5tMI1yk zrw|4XBRn1kdAX9^R>;Kv;Nm1B1oFcg6-h^k_5Y0Qzx&IGl1Uf_PzvPTMi3Ch?b^0O z03SjZMQR1ZV29Ntstp^$^tR?9dO0{kv5iP@EYz-h$0ab=?uSW+SnG)I5z{Ckw^6pN zEG_--s5Z~exvyec27`|NNQs; z#AUvN;6$Y$7f0Us)aNs2X2*#+4zKom)ZSDcW-qowYU)#$dfx%}->`pEj>L)_#FE$l zSKFC~^_;iy{%23NvNa4tO}1<^$kJj-kt}JYqQW#1St6pcj4fMe(b!5+S`aCevLstk zv}h>VrHqmmL^`iK=3Lh~*SXGL=ltwX{t zHd!JN<{(qat_rMn3QPz|s*6Z+9bcUz+Fr0NexOlV-&fJC>H<_6Lq86#9K}W}$p_gd zI#ru-rjK0qyeXSwBlvj^|JHN}C(S36jkmmuOY14s4ASS+`}fa0dN7m|*~M<)xZr#pI?8eR85Ig-@Ak6dN8kuhZ0%GY-EB$e+fZIT7S#eV@%|WtM_% z3ZuH1EYqjAHzXQ_dYoFM{Yt=B&fR6tQk!Ap3tVfSJ6>Vv=y>E&4%#;d=n_-X7JX(g|ge_^Cb&LF|Ok0$F3#Rf+3zS#KsDwHVRC5`V4 z;xGEHJ_;YkP25{>TK#EP)0kS<*&ovu#-{XU&1?qS7~CZ5!Dj90T8LIn2Tyl{(0b_8A|Ff;c0|2 zrg3m#R*z;^^aro#2s!0)eu?{~n)ISE?vEzSX4g(FD7a`+d9N~|0Crb={&)y||2gM+ z5H+2ufzlb;8>B!Q0A*?X;pVev&*Dq(tPpkZvCj@GJXH;bhImhX`It`th3Kqc39LsP z9;4_mQA_-)@Sz|z&YAHbC@N=dY1P5fb5otr&KT*}MaNVUgVT@Uc;#z$ zDkVzz0G>|FUrSww z*$=lyP6Cn)BQ@?hyG)SocI%4w?^USxv&&19FPARLD)lN3Jl|t2F`A9WlGk=R^gFi+@+l~a^ z=>T0Ew6;7q${!RpXl=-dCH7I7F#APbm3PfIZ(Q}ndq&@m%AuB8n^i=3r>uBR*|#r( z@;^$^6PDvw{o$+1W;Ghi_HMIgQN{D?jn@m*{majK9;Y!5232|D9g1p`^m6jI5~p_E zMo#C%)jPX3&?LOZ4K}$yV{1{Gup6i>{pW@M^UZc2&`mES`kdXIIibKX+b*oGjHvnt zs^?g}>7kKkz&^(4CykUUUS=es*KWMNs@KtN?~E+uO;%S{`mapitnwVuc)x`1LC53H z?cBY4UBuEO(;OnEeTm9h5}h+V5OBM!G(R@v?-5Ia3x7|y462RdjH-Y~n0$H6nkr+8 zl~#~~2=wfe+-G+w%PIZxM0%=GH$ia9KCy@GHv)({MC2UjI*$d(5z@ z~V>$q@L<+k2r+*7rGo#Jq1uadBa01j7`uphEmE9Nc>j z9&Dv>&W%)E-$(66KN4!^FQ|>PJXc6TE3E~c{~+xUj&dr~^x2DTqAL?-)O^!7E!gA5 zZcZ|mGTsA;Lh%4vBU7)B{*y?vUzszk>c$Gz-Fv9hD`>^|vIL{7efG}CU*41U@ZrrB zQw|JksmL8&9UpgYz}v;jrBzO)i!4}OfeCkWuJwuhwxB3^6o$tBt9A~_3twx(CU#se z#!`zPEmZ=FD$w^0wok4(Rh#w70I;!&s=m8_XbT{%D{Pb%xBG-d2JZc$V&#tvgZ^&5 z>G+8Gk(J+~D`$siuCljbOGBRjM)`QuWXh4)S5Jl|*t_pLI&IxEx*@40IphV}P&@_Q z3D{zpEA%u=vrX{oLz|muQ-+HIO4uhk1(}<+?`9QljXVGrHI}rwb9QYX^e(-0 z*=^m7yoMS7VOYZ{7nw6nAwrYnx|#y8iA_u?B%7PX4sNpgWfj?+_*ImkpS)g=vY&0z zhMd(j3DO0Tbo}u0c9#3PJdSix1kAj!R9#BqcB4Njylbnj(yiK$bhxed3{`ya>M~lt zymN(qPJJ{r^*-v@2k6Ak?oi%iK@fUTfo(l76D*@25d;CK+dY!kDGvM!0%X>ql+C_&9zM49&m*6<`sL%2I4suW5qGC{c*&uW*#a7g3 zgkM1#?~l!OB2knP>#P;1%>N9>ppA*OHnj`Y0M%^^gr*%kcQ)CLDpv$gR`&Lj%F_4Q zxurYU9B-VLtMA-DBQo>L#026Y+KM(?T72m;8Bc)nT^#3=a-d;@4ir<7;W-%#_oE$` z;#_;y;qmH>qtooZ9gL_Q;$A!T$n$fdwRh9fnqtJz=G5Jak1)Oa@qnZUro?rmQVwD^ zy9pOLP=75!M8%m*uNgme=_0vL;lG$po7P-GF%xUjC1j#^0rJ(S0sw;8kFKlA>1q># zVhrSI=DW9kc1};MO4k-Q>RWkcdu}*&bDf``l+JlJzDe)?KmkmbHHWW?cO-mR?&N}& z>M{XE7tJDZ>O9G2WTYYT(MAgKoZv!~NhL@pOcJ16DrDR}x^AJrC_ac%ZQlGjU!C*K z5Q+;Se?{I5hi6EoMD`8j2p=S&p617OoaWDO;t_X0pzo-|WhE6WdQ}EY5z8d} zmkXC^x<`d*C7xosHG*F@se84E!#nHoL>pV%x!hnOt`GLh-ujo|I7O9&nt6KcnXBGP z?va*S(u+#1PX!V@s@KINN!4ccWzU~}Q2#qWmtEX)dij^ukzX&G94=2@SH6`f*cWu3 z=txqBHamAbFX3B6SpJ9d;b1wtCOn9lw${QT$D|-CjrqANIav>;?7{Y-nk>?aM+;-0 z!H{30uHS&bJ5Ya4wY|TYZs_(^RwRjTr=jQs-XuGFdrBWQFwy55+Yw-{wAbJNR}7j1 z6jH59SuAlG5b^4xKRs=N##RFyvp?-xPenlOf`@V8V%3xX#P8&_?x6bLPNAoS!nDy? z44^hJZQBv}Y^OU?R#2tdf!mHx7bn4jMCHu}Nu|Z$OZWm(6g>$X>M5rj`o=-F=NAuD z__Q1Ibvk(HcIKrrPa{u3TzHU4Q2#XlnEoHB^c5-)72^Dz%ifp@1Ip_%WniryUCR6X zsJ~}@uItb4?@uXb9glvuFECKLK`DNb8|=a)vJGr-3kG~9bp;-r9fXi>W4L!<{27N< z*`n%Q!3>1(rnXyppHrLaRinLk74B-<8#9Fp0HYc(TAs1pjkGi$IBX?F7?TvN)lf-Y z>s>~!>lpR`_eT*!l4A@W&zLo90JY@QP!d)pHaGEwl6tB2m;_V_jm8pm;{zHxKDpw? z&k~a##@!HM1TqBvc`|ah)}TS1>EAEOVfN|G(cNiuoM{*fj$PD_JbLQi@s`I?^AHuK zguh9XcVl<-t#y>8R89r}EAk4G?uHLf1TvH&Pu{y?Q*7*#z)M-VkAwSv$?U(7OD3LO zYrG!&)!-59W|uC}@*L=tus5RSX0FXNr@HGoTUvL~kobbAgvE*P9$%a>Y;oaCpx+az zC8e*5EUj9#VA2ExXnH$uR)HcunOh0LBd?uaPd&~5ix&yiCRsGAJJ>oUm_=xlWhL3Y zW}*t&URt)Lnm=&Mq5a?a+SOnx~%zqX&d{*nhAS3GJssKT(R zW!^reabUIgjA8eO59pRV82@WW@XQWul&Nfd z3)@X|$ysTdU}0eJmn`)a5#p>b+-qoSd*IX4jqMC6+zi==T&x;JXgs>4ztD4Zw7$o# zja|vr-a)l1*9C}v{QD&r%RarIBLWv&Pb25BM}5&h{^xI`RgJJ_eoQxqzN*BIS8Mr93X6-uZl8H^|qNg(~(OB zgs%E+er7ieShu=Q`ZeC$^FfW5<ZRQ1D4-Oz!V+!J|0_=DoQW zD!4cqRfVcCMm#O(j9JMFxpaau(50tpo)h4T$V)|*2j?o#x#%}x8eH^t=!AGf=;7Sv z#*48pR>v$Xh>5y1?USBJ8n22VM7t~k!wVszrU;@YC!90wxf z;FP_Ao=&E{nr*PJNHVfKlLr9I)K)QTU*92Ma^$4%n`9eUrh_#AOF*ww_;xtyw0vXOReVM2PH|ElT{L z#|oC@&2al@5qO`kaD=K24~@WH#AdYt+41`HY^w)G>V!DQ>7AfkD7YuUAieZ8=-M;C zW>$|cs_1rjpN?#Yg0s%r!0L9>!;+?zWv!ih z^75rBC(DZu3kqIr?XmVspAfIVpm_z*b?~n4B>pwDWR3`T+-vS!-hS(`snvvn%G>>* z%&1iUA!9d~SL`Klcb{g+eZ~0P-iG@-OODL4j~x zU8PCc&H2cP&n)vXw7m(n$1tX$w_8e%-2l6DQ|1{pSwCMGXEXgUw0H2j^N^6B+!;#) zBAo&r`cQao_MPkK+bh!h$2zarx+7AHLTnk)s3PSSNi&W*Q6mVU3+ZBh-bpxAemhuNqEKvDY zOH!&kqg@yB;NODt?d0`+%c}o829er07OHfP$im!`wXI?}y`OW!WShbSh?^e`tbkgKYs0(U_7t&p+Mc?ty*%u}bG{jjQ0vcYENS{iEXGLNzp{|N~cax29l-BoRASY&#h znRBiAL$|!kMO%Rd-Z*p}zNjHHfgjHN?Xi^6kVVEE6`wxvo)Sxm{Kf{72^Qz(teoXA z#Nq^EFBfuFs#xR6lgBd@;!gD2aZ-P;Zz#b>Ss zC*Ip?Vx|+NI-9dE{4^2&MlqQ~vV%2Yx(sb25qM?0t18fGw+kBVaDxU=zASTm&ciq- zTJ57n)ET&`K{~eh`*)B4hU473_9}x!dztHKLB7)Ii3~!n8eo*@d#>_t=+r%koqzWF zb#vZTXXRm}PBtCCrS(-jDaUT&L=m+yl7FOrr&2$2v`S0HX)t5W$|hzu!b?+JpfGqr zzXe>@8$;C*ORZ&4(r(2IA17&ZG|gH*F2?7(6i16~dIH|$dxT7uoWZ3o1FC#xaOOh1&o)&+TxsC-xq);jON7q_wzHd3JU!AIM(E`IUju3|h)NH|$` z%d!>s4qz(i3dW`gZXL4>%>zOZF^ckPGY-AKetPLx-b>YAT}Le5$+DZv*_b^KMKENZ zR&{!=JI^%H2~07#p5;pQ*{14~2z8fMfg9_c6)RnWI33l|E!FETqc zZ+FhI&^QR>&GAiV14GW}-2A@!GU z-h8^VoUshdtk2Xf7qK$$;2_eXjhKGo?`G~Fx>YSDK(f6MK~WoKC~x&m07TJQuH@nN zcmrQ*(zSaiY%&IRQ(U2VA9ld*;P>}sxe;D4L;tsaT3QBPUV3!0sz`H0?i6d^XWAj6 zrn>#2zsAxf81N^IZC_4dcBcC$Y=)MsVvVC)xq9bLYa$A=>+jyV<5IVv^5MgW^yOZ! z`w+Z@Y)%}yD9admp}nZR#o-tkX>2QO7Q1^}EDiB4dCH=Yf)18F4N4`Hc_Hn_&RDSM+@9FXe~sXWC^ zU>-#yB!)(0@%_3h=}o1yW4h6Ki}X`ekg^QH?#<1vlTNgzC_R1q{MLIWfG1qK_t?th z)Ugn9KE>k6h{?eOIffJo+5}*l>`BO|IPi^N6+-;-r9t#!vmZYgbXXN(PAR z-)x0J7r%|E|Js2N$1J_Yj*bekajsrXyuHE6&&%E1Ob&f2EA5Z*WRI@l)E8<^t<=a* zBibaR`%&vIWV{iFtb=&6an((L|} zRIbSx14bRGKY#pqKQp&`AC~lLww+)0GnUg7S}=UO8e|Yw-ofsjuV2nO@)N=tNgkA) zj-Qh6*uVXP-Na}m*MIvWnsH*8c4I_0&T{24FG}Kx5DoDIgF3#3AtpS{=Goz1(q^!! zjjr^VR4+meDCU!epB>he^qQ6E)=YtBz_L`Q9jfUooXr^CZ@LvTG}@dwa^)eyHe{Z- z8H}4{qHOF9tp@FdTU8~k;R=U_fHt?g zZ!2wSA00$N<5McWObNh1HQ9&Oeuq@PoHcGYHL`uc4kZi>^db$sSAW=>-NPNiNKmoh>otC5B4BXz#-MTYLagfmDYClM?!67ub@ z!PMm@9sXtSaJ1&4>8gP7n#-6_hA)qL+gv5D<99t_ew-$0@%~hJwH@|P~VLGl*pcvmKf;VzS%MIP0d#wrnm8?%my^uS3C}_4~ zaxXj+89eXDNa_J2Nc{($eiIC2O7r943a%ZSu@fwSWv~aIa@Xy}|P9u8k?R#?W;0X|@uFUw@bs8rLs}TP? z`#ewF@Kis>EzD#EcQlIOm%3l-177J6vi_f;E7-TCH~ece5CB%>nmPYHSF>K@)kOS9 zIeeAdNco6nyxZn*rxb*;NbqX&F0+67F;bSzq$#I{mIMRX)^2(p)60xCf2GBSY0V>66H1LHE& zuucY-l*xkmP40`-op? zD->a9c#WK$BMbMwX{Mxp8t6@4ApL{r40klQEdQ~u9h)N-8DJUdgc?SQa3D*qKxY1S zY!yNmiq+&3&G4Cp`GXo@s2!0vmPLvu}52+vdH zA}RbTiNC7e-qO&xtEyTZ6~{OniAeEf*r;?Fwr0$Ij$MgK7S**p0kaHB}8j?nXt_Q?1)plyn##^;Sh_0F!$pFBPU*lj$l-z5TW@C}k0at|q zhkVygiUAd# zO_t-E(hu)_A_pXY{|bBWV5Qp4S2O_IW@|(Qr%;HC1%Q<@B3+_k=>Z~Kp-KprlMbz+h_-;RJBJ= zCI&pI`#}Ns>i&6B>Liw(7%tF)iaA5hOm5h7L~8QBvhv!x`Y#T_PGgTX|1p&Dn^Z)V zx(T|DY9DxbJw_?TdenBTJYR~pj6-oNg^aDZ)@}@U4Jj#(#KyMc;Ac#Kq9^gsEbSSV z{$rjTGvbLzP+ErcyX1(Pf`@a2?Sw790#bZq)1z`~WO(=Tp=yV6v4M_w;?^9TsimqacbO3vsW)^ zgQ;NsPc582F>A&lg+kSJlHoYZ&Zz#RZNuL-v7uKCt)a#oPrYIGP~8_cQEuOa4bGoF zB<#_y@tx$%^ohTsTb{fTC4))q2M4Y(!Fu$ycuaj%=pPB=Db3M;&sy+>RJpNA8wfYx zO82u{vQ_gI>%9=;l82~DB@sP6Wts?bU>+3^oR6;;#W(;# zY2ED3VlrSm+1E8pIt*xOo`-Zp zBQ9wLzqgldiocWiV?9#gxc!*&Jld-t^N4qs{-em{F9+`U{>e;fEw%4oi@$3*=*O$@ z@rj)EV*vku{a~|{8VY!b4{q(UFY9YLmDcQjltmvh=O&f)?G!z5RMfZsD#`>wL1{aE zcI@!iRkrHDVv$yv+9CXx!F~Jg>1XyZJ!6vQi0@AyY5z0X5qf%hsKWDZkrE?8S-g7_ z{W7lW3^eIw5j{UJ$lrMJ_it#HG9T^hs;1F%*45XI5Nm(pJe54OxORjxOH+T5!nhZ z4UJ3|*4gxR?wt)B*`l3BACHOI!Io`(pqiy1_&YjA6*5KGuAXv@!bVm%U9_C`hsa)q zv6GWev>xF*g|z^7kdQI5{|QxiMDm;*!`*^wRJ#>#+?-eeR#$Rv}si5cfSK z(}9ydmjW%=;-hz@@&8R1oRA;*zvyB|a+f5{fVfTAb+}VPZVQMVmcvGk$+ek+Z9qJ0 z%52b*Bndg$R3XL#<;`KKHT}v{2Gc4BTKK!p&TcMS8ZKB#h(y;vIcrAES_}O?vIF@9 zb7$CPa$)8LzP7orG?dyWhTITUw)+mu2yS~O{G>m~gxDwL4>_VmZr6A{T*RSa>sG9Y z=O(cL;z*BBNNgt%`YblTlcWNu!IPvCXQ)si4krjH-HD%mpSwxXbM)tz9Tw2*bP?UN z1bk>{Yg@@Y1&&l^uSjs1xOnj?PU(##4DArH!^ovi*P}2BRrJK?VzuA_piWC5wMJu+ zYxuAYAc3Sk(HK0~6hwVHW3>a@WrNO?3~WpYQ7bP|0Jj8S;-5hN_OpmkqN@^pIa_*v z@mFB`)ozf~R##t#AeemD!EB}aroZKiL^O$#5RCJ`J;&^G++%R`2~W1B!}PYKjRRXc zb`vtFFI~sxgoK1gZ|Tk2BTZCQq|qz==>vHbaNF3~5;pIjbHInf98PftL(>qu}7;gj&o2djjA_;_&$n_&lqkT$sog8$q5 z-y~WlES-*o7_+PfisgR#=V$;+>XABk1cMZtf~tYuC}UJ9=M_D>|I-HT4x9Sq1L(7k z<&HDIef0~o_Sgamwt;Ur_T)cVs8#{#6fu?~!Sy-<;+(DeLW9F45D65(fo*SSWt5^e z?5?M*p2na4N@e@LqC!=O%7oE?Fl0jw5Q$7CLh9`Kk0fXjttBX1m0WYwOhGbJKYeSP zmEc^G&{pyp+i6S6*^*W6MJ?RLIcWLpyAF10)Y2nmN%)FFM)e%LphuvIRrr+usoboZ z`CYkrqRP|hokfwSgY~qj@xU7!MYbOPw~%HhTq*Y?6(nh6V2OiitJRf*0G62gl1?>{=F;p)=fiuPH|P= z=Ne())PM4hr9L_f#?RMrKa*&As&~%oq>J3VPc zpE$81@OO`cp9z6KG%suPmlK*%g!PW9p?BJ+N!8zTvhDrXeF`5(Ez)|k$BO-XprFbMm+++1fwH<_PN+wZBt zX1Gt&eZ`8%gaj+Ucz8Me+BZk%d};U5X!+Mq6{T!6wD@o~t_Pkq>x6N7w;NNSbiyYk=Wp`)1Xq*S!8 z$6~uw$0sK+I%>wNX6jHZ@RduUn$>Tn*vD!{oj$#EdY5^s13l-> zBMmRw(#$LrFjS}b?tn5Iy6t+ zR8;o$;-yRWypQbuvB_ZoIVWc&n#1ts!0=X=91n?(h?oH1y}V|7JtN}RF|I{&Y^>?p z%MQdJ!(z_OR z6<)r0K0d$%%#!T7CG5tT6>#{G@F-ZzR*iU=bkTHnlHKW+%D$hUd@ez}Rs+a3YCiyV z6(na2x3EwqST0-`!{vElr#9M|>Q!vwgHRc04$a~1oZ7ObIoHxg#l%b|k}q%Ov)3A* z5*IF7l#AS2?}&3J@~<|d`B=e|aEba?{>Je8VTtpU=N@ zN5pAAk6D{M&?YX8t2$gr`a2D5&56If_DxyYIp(MTAAWaURE#v;IEK=I)wW;L6RP$d zHUBy6<_C4$qbJ4D{uHogqX|D07lRZg!eT78qwZc+KAFQ-VZ ze2Dzxb@IupyZHZ^M6xaRTZEd3!ef*!WzpSW{Q^y!Wy{WJj~G$#?Aq3oY?_)6DcoGJ zJDZ(7xIZu;U?x+L>ql|P6iBq&zC(u=0q`tOX}fsEs#UuABplDd3croZkh#<*n*=EL zb988t^A78+&IVj_@*6N_%oFj~4;^}OMTs9j3U2iPYVgO^g*6|a?-*?B?OnJ2#m567 z#tWZtukY~rdasjWVlu?6HeVkL*}td_)GNJWt9`ee*V^ed>CbjggEe?vJ3Xj2%w1hi z60NV8UNJj`Nx=~B-=4{RsQ5ZoTRo*!Mn}p zcKa}Z2e8FRox76RsMW7um%j`{+v|t?yv?J{w!d2%Yc7(#eC(9xDOT#Lf7{&ZU!p;x z@qK*HTY8#B8fFzw(6Aq`@n`LrIWeiXZgy|MU%qmg@R5K62XqR5*D$H_TCDk8TXSyj zo_pUnJf1?KqPkh0^Z(&5%+i-Od~kg}3)j-lPPg!PMeC#VQtehX)pKmv4pq)(W==~% zvvjR{6UBQ)ul@VmtWs)tiamdW@HPC($EV>`P~eaG?-zZI%pWh^+VH1}Z6!VW9kYM( Ua@@{h`He|N(+p1;IR5oN0H`sN-v9sr literal 50177 zcmYJb2|U-^`aS+pB88C35HhPILW)R{Au?9xgvyj588Ss>tdJziP%?#(N|^~!smPRM zjEFLY%>35w-0%N??|I#G&xz0H{oec8&$FJj*0V!(v{V_^ajc_IC=BXq%6b&a>RJ5% zg|#&J->&q7Dfk5Z~Po&{^N`qUizF-y&xpH+`Z)x^ej-H=#gBE=Ra&1V^7Nkx^qUN6<-?z550 zE90G>0vvDN?pYsS?=~VR`uyUkQT>Pbg%9^6SMl+Q(^vi9UrniWbBz8LY(f9~E4H?l z#pnP2fh}l=#_($W|NU&0c&C5J7zIDO9I7d#WM##F|Ni}k_V(-Z^IjsX>)!rwk&=?? zcwS;xSl;yIOK@Uhm;4wfC#RAK>tPcU&Y+;64j0QjHa=##1A7)Tv$B+YedULL{rcS4 zcr_@92ERQzmY<(r;LX9s&(CjTYkQ$Q?*9G50xYlJzFjLMBxLGq9sAeL(J@GwuXi}< zaFU{#8P}boXXDNq8ymO#jE=cw8|0p9XlXf;d#d#G>C==v!wJhy&1&I$GT-pUu(9B$c`O{Jw5jyJ$h8f*!cRNKQ8$9_^i#-)6;fY zD_Bp{N+$;g201x7LnEWtt*x8)?%n$|FYnL;1>c_j{%Q{O&YOpmDn`D&ihS@O0>6Ie z?%hYvp0PwnM?Zf2SP>7rW5^Pd6RGMHLm5r%#{y2L`SY-L=cY!h#Ch zH!4p{OS?~2wz9ffIeS8!D>pN9ovp3y+qycXLx-r=tzWNeXc)4%=yUwUiKL<;4!w-2 zygFUMg}Zm}I=i|CBq#H4-MY1>r-!m-%a%)CUeD&M_zO+Lz zyURXGvuI#_bagM(x453T?Nd(uwYizK)NA@`UY^8@7cWj*TK=in`713gZKgUkIeGQ! z)z`ec%8R!c9?3ZVC?|)JvMu^3Z=_lQw`EIou(%xGQ5KT1hU=NpwL zCUQS{^5nF&HLZq*hM|ecMfE#)ulR(7-k(3;%C0I(%FNu6l9IAbT6)vhuV3H3dw0mu zQOvQk=#AM>Gom{>`b3<-w@;rK5j#vb-jMK#jAXuf^X9hAH$G<>*Cmh*Uo{!9!i8jjr&skRME)0Z(kgA;!I_;F`0#vea^ zi0s^{boQ*^{kXWoX*PED{QZ|@Wn~|kmtP*Ano{mA^Zi!+##Ib^+}wQJw8)ii`}XZ> z+S;{YoH9D6PBBt&LIbZ|Tm5@v1dF#;SXkK5(9qey?8FLE!NtKNR}>qBwwHT1>nSO% z7GRy?gg1Z*Qv#e=4^9E&>M*9B5O!&3IB* z_ex#NZhi>~=8TLCH4P1=bLWJkLPRtU#HVvmQ7<~V3Xhlf zg^`5fpQ+GM=F1rv7)VYX0vo4JQJHC7;LoP?4J9QdJwJW~URh0B`Q^(N-}$ND!S(v9 z@AgR2UyK+SIEP5)-MMqUj8j+t?Kv!X|KQ+7+LM&0FJ1(Mg)y9WaIkQ2=oHF0VaEL8 z*|V#7h*#Wy#iEbuo;VSB{rWn5ZJhnXOqR!uje0r!DMdy17ve)h>AvW=xww4icrrRV z>g?urmf<9W$FJ61)ipJI+qaW7Ex5cSi)W;egyekjq7oLqP}yiYH!LhnW$%uiJ9l2d zkM|D@xNvRUzkff8Nec^Jf4QpMg_*GpJUl$~98zm9U%ov4_pjqcZ%Xnhq+T^m&FXyn zFSjBiuYCBRnUbEKjBHVnu(MB0Ow57BqNVQZSN7J{R|#}2-FgJ@`1rU#0%P}u9nnWwSy{L3+7+g;7uoqjUk!tlj7$}(0Kdvj|J>Z& zHQ`)0Dk>Cz4iEofu{VfLObm|_*8WaD+I5J<#l?l<|BLOUTjnKC&)1!uOJ!RuWv?CI zaRf(FPhY=~@$uA@Yf5VBA+eaM_wU!?BP-rU@-wrt1~BuhxPG1F&=_Y=quse{*PH2I zUm~NU18?4>7t5%xucu``q@j^@>`%r~#S>M62?+_sOX9+AHs{axbEG3|98{DZ2Z?8v z`B__A7Zeu0uBuw?=;(-@sj9fL+QQNC)6BW4eS7ZF$B3C7HZ|R9X=TOC!xMo$|M~N0 zMQv@ch8Hb88wDHF_Vw!omrF1X>KmLHXS_X9OHWN0r`pk@hjA*ZEhSWLvQhc@`OVJH zTTcD_EG{M0FW2vwl$1m@F)@KnR6cxo70bqrg)}b^e%OoNfq@#qmn4u}UHeB*jE>^y zM&7#@icNN$x|&&)oSM1@8MD%#YPC4Fjh=lEHCgGHm>3oJr?y4EzU21JPV|Lbzh3#~ z#unb_qrE?VtT8t?zqxfkOIcaj_@6(+Eb5T}1V@jq``*`gVW?5Cy0(^IPL7j&Xsm>l zxNW0=X5s<%rTOVTx#MB_1_r@z->StQ@MO`|)fEvH4M4arE-n^Odsc@qo24Z*Ha7>L z*yLQ}6BA=ZeGr$Bc#Q{O-n#YXyC~twR`qG8r_>KDyeLRASFf&0NlWA7<-Ovaks}2> zQu*f18pNxqcGhsBbjvM~v`Mv8J=RT|E*NmST)Y?=7k2~ill{ym+T#H zPef{H&>2Xx*`9b=w{?gP1l?hEbvp94Zo|hvB+7FhI&`QrlXtd&b=?{gY*RmV z7pK3NlWa)6ips~wN8D@5<|em1$CWErMDLAJTne`$S?=>&KJe+m!H64Mc0YRbXceFn zn~dYu!-o$SnKjNkHGlbHHTWr^t;lU>j(K@_SE)C@loac8>-wPFT&D;&u17i|&Z8as zM@@B%j6x&%RlmR4VqP7;|Du<2x)^is@2}5imlwt^%#K?yF$-zMhhV)AEKN1ZpKCmv zar`=VhiuzE*Pl$ll;U>HA|VMGRN-2*uJ^V_T4NFWlPf6mO#&E;j)NA%85{|dQz zvwC@H;lkXXvp}Sl?JsO&C9N5+UcKtouQLR^yj9MfUNwB{p)+R$GBYz9UOAZu1O#jo z5m}1`o=H%*357u08EH&QyMRVg(QaW zJ5E^d6cw!oOnBd8_@F=Y)}mdw>19 zk)fY%iEm+(brtF6{d;NGu3ZI}Ciw6L7sh)I;TToEdq;P1;KS?6!1cu=Oe1&hu%2&! zK}XBD)juwd^RB4T)xkvX@O<9U#YN4ESg+AAe3z37JB|hejY8kTSa~h}E@Ay?Ysu2zZ}%nMN3aEb zzH~J8HVW2Im`v9;#NjF=ATN|?WuD8kyn;ePxAijmyxY2($~>Z%q^SWaW6LZR!)xC#{{{UPx1S!{w8VNO-$6v zEnk+sckf;#x~ssTpvLH(2D6M=RjKLeP0!BOy{)XgiZi~3hNjV@+gBBBj8AaQf$AT1 zF}8Dmew?wgQli)(ZN6~(b@H~FfKPBM@3ZI6ZRe+k1en&^G(0@y+EYR0zV*Ctp{bUf z`|oWsGBPZo$}bRwfX4%JPyA8~3-9f9XjewBdB)Q6Fv>*w@UUH3)$ks`aU5vv{_o!p zx3{;yaPB?415jLA2ng_Iklv1H){PsL$fNJyw~vW}DwDiv=SiJ2XU@E`JuSZYI5#)F zDtJTF&fJp2T3TAv{y221_N`BnP~ys}^lzBFa{PQ_cbOVMbpOwvYCOJE%+WD1lUyew zzI1eG;z=L&Rt0aQ+~0F{-J9ClNv>8a-5t@ci?bKJzK05{ZDYEXm9^8MtK=T(H#}@U zly8fv+rDSd?Sc9S+C@c0XB-{1R+h&6GM+xYHP)~kot7rl{?uai(9lq1M8pQJ1D9?I z{$06kDwdug=W!cJB=n8t{O{-WYC=&HTqldUdK*;9y!RG=tk@eP+eCv#u*svXWIpM_9TrT71UdUV|?(WZ+|*DuLv3 zPuy>Wh9;p?x@{<$hKzIv+u6>IYl818Q8@Ak@`P*sBJi|t9z&^U0awU zg)@K#`MX?-qY__aLy_ArH3Nefq~f<4|GhHchgR?go-|;RC>;?2wBg1$ZGytWlUy84 zRgZ5kAqsAxmA19FZ+c?%Ttrs3sLrgHyg-xo!AShU*Y@_t`hzPUIG&qT<3nu6dtRp` zCo3T~hFJLThVvZM!s+Bo(otK#S+rr6Mq%<2%C#*jRsZOy76NgqQDgUng6PblJ4Xc_ zzCH_7=3_o~Z%?40tKIGB=p)F_DY?1Y-(EWO$;B_odW>%U`t4g3`qm?=s*_w=uiQM$ zQ1R^3?|wvz?A6uKSnoUCVn9OhaMG=-*RBl#x0=~q+S24JEhEze0Ka|PHh<1zEZp2$ zE`{-94ct3+?~XvtQ5RtO{O3mxK#k6Pq4~{~$Yv#@+^%3Y5bekA0nqnnKBLzgi=8L{dB_2i)3!9wM#ThT{yY zR8>_QzkEsma>_dlxnPd*hQ+cjisCNg7bvjIKz$7;_R{Wc6G-&$K76=IL(f*B^4fJZ znr$ho8hVl+&em-XQ`f=@`!0;UI%8v_wwi|iVQeg0nUdut&^*dD3r%!J)GB(x&Th!mb*Qy52|t+1%*HoEd%W<_@xqO6MU{QiC3>+e@WLD@B2xKNAF0L~&DE{x6GqSRh1J`e1p@0{%dcIxK%-GIu2MJ14 zg3^f^&Jtl$Wse*!uzR0py8Y@sL~h;(Fpm82;e*J&eX*~0TE24Zh_F&9NjhuI%*@b?zA}nX6Cex|9=riv?9_0J z#z>w=8k+yb*}jOjQg4|*fBwuerq|H{)7a;h(jYthZqHw@6hz-E7>>@eZ)v8x76|9+ zn)RDqXXjTwE(>hFp=)69<(&L^KQA9t%c=mnjp9DD&RZ3H*iaknwhvyu9Thc!2&n+o zR~ycC4G(Ype#owN3`7->)amKrmP@0BgUxy>1~2Tth>04%Sc|5xt;|;r=S4RA_p9#9 zoIHMu5_tLb=H}guM(JrSbuV5>o$oB#1SDDD^5cDZN9Lys{co8yQiItTk9FknZx#io3e(>yf2|x%f&Hu`^+j(t7d!)%|LEqAnj}*Kyzm=?3 zm(i0aPpU97GD=-uyp)-nYhZ6=mR;8J<%>Sf^3p^t4_Db}j+m_M{i*3`+m_4|2`2}z zk2r~0Ro0`eFT6a_ilFxMTU!r?w;jkgDZebw&c^22o_!Dd)IU6Y6jf!cA?wZklN*kM z|5!s!t#jdm1Td4={7|~O=w|!=UB@5z{Gl8A`SY>E@-BicI7T`Oqe$E%Tg32sp^>5X zN7J%=;a;itOe4aobwD13sf&k4x%F4{n9uU^c#(?$!p6og&HceSct*V<@$SqX;}iPQhYmzDQH3Kqjr(FQy%!FPW2zE>k){ou!qcH!IpX6OM z^^E4P(ppwOl3Tck7Int^;>G7RT9yD1C6wu;R5p{$mX=|HJoEa5H?{RQtQ4$+1jyQVp@};}!(*upItpjoa zV%h>M7N~e@*RNNyu@OPemrLz9k#+Krw)O^dpNYWsI?Dz91=rSPg5fk1WH)Kt73~eW znpv0r;PQ1ehu<5Ll)y&Ioa`=&06;GJ^M2=S`O3n%A@2vgg|?r(nH|^OJL+ zZqI*v$*}X}lM0z`pKEC8MxB6*s!g0q*QDx&qN1GdDq+W>UDFh@f6jPMGbkivE$|8r z+PQ-Cn}5NzSK;6{Kt3rcIZ!hBNshF;x+W%d4)r+n<6o^GB&DSdB-S6ak)vz_>tlX- zP7oC-q^~yO)trgPI4=mP`6Zz(*DOnTS0JatK7dDg0< z#B*aXBbS~DJ6O!wo@*P%5yF-rdF^(lOFqsj%Lkti_ybbF?zxY?{zlU107~{o99&ie zdk#wA=Zf~q)Kl5N2vV#1M0w=mg9UEOIJ|Aotv4`oA9!2(rE7j!d%`1Vm&r>-(an8W zx8wHvK`8^)(yD}Uq_(=qHug1ZKprwv3`K7YOv}W~e4*ZDgKYXhe}B)&tL`AEDgX=< zpH+D>Mto!e6rjI!%G89rey^r$Zfzy}06vs!sJr#r?b|F!p0pAY61UU@$V2@7{k3YT zG(Ue6*bUO1m7&dAaL;*ReJ^}72(HUZ^ZB9qRgce_QmUq$QD_Mw^z`&BsUhU|$MzL% z#;yAlIrpBonM@1|`{*02lTk9=YC`S5Sa48VWBC+8|?)e+7gk<@u+7?*qREcP3`+$xZCR5wvz-?2Z7^ zygdK)uyXV0NPkmHmVx>W2i>hLlvdFfBx zZV=b1=ql`X_FXPI>Z_rxosK4*nKF+vQut%%jvbGky31bpF1X{IzOZZAu}bmNr%$YG zY;8HRC(X^Z(aEE8tInSJJMyg=rJdx|M}NP}_+2lT^c{aqi({A8>V0+O_ivlNnlL*P zobEc*@LSKHOOiD0CN)c)Wn2*jsW%vLH!(L4$) zp!;n;bkt{evPziVw|VIdZseIJP>n!Tx4dRXkKrslQVM2d{+wVjZ&e$fimb1PIQLze z{sOdM3;wKQ@Z0m7MNja>U`k|tO8=#hSSeeUkN2f-jEOh9Q@Zl3&?$v(6lVFH;2p}y zB|B9?^Yr)*!HX9!zW?x{zssb|XaCHkWc>jp8X6h`8~BQvLMA6C`6(n6@DI0EChHFt z)^9y{x#XCYPK*C>uC~w+5O%77M;&9kOQ#y;XAT4g1RQWV{0v;Z_j^Ewn6;FVw!CZJ z+HTLlr1W)px4E;SU-TX;tzGH_IfYd!?EdyV!Dl@^eF|`xR`%Hs@%y9Fv>)WO?isDY zo{nfo^Z~jGLdeL^_8aXeoa&F#&U9E>N!i&VbK;4I@N91BUk-M5b`a0}e2*$C53f?R z|C00LuB|qg_m60Uc7a)C<(0X5zXIw9KYOQ_EG9heK6xU#N)eFO&d3kJO7##GGylm0 zwdccvpA(c%if+Cb@#MU%Z4%u^AtuW8>(}`n1wYXLl5J8eh>yZI*(J!>0+*#^Wo5PQ z38H5|j7AL`D4osC)VZH=(YF(V>CA`ElJ>2lr1(`4h4li5HyZEt^Lg93a7&VTyV@7%f51IjuiEbI+3*3)wzIS6rw#I1xp`HHdU zllbmUIMX;*R0Wf>chGzB?%P*z?EK2Pwru1C%4rA*9p0;H8L27MG&C+_=#3}^lJQ#= zi(eAT3~btVN)Ot+ZIAXw#CW#3+g-&wY?F}C%|6@T-|yV*WKi=ry|!;?C=|eVZ-m3J z8h9MD%QnlM&AQ#kYpaUST3f>i;8K-70_AUbm{ePrx)+0n?&S+_SC+e~h8L>7kBI^c zUb}LIYR{fM&Cf2k%$##_%Dp?C)l{Alvt#q-&1`5^D2fIKTky*?l+nA9!3p{h!>K6f zGS^TFii@+{x&)wB0~3JeJ?wCnb@S#&IE^@^MF+IxXN$pp{QmZ8!!_Csj+4s}jEE-y zFgM?Hamwhdn6&?$5MjlOFDbU1;iBnf^S905pWBP^P zcQnqa?DgW@>0C_C622&UbpIgP(}|_ z84E?duBl1;xMyWRIcLYO(RvWWUC-m~4zsr``HXxG_xGoemSdlD4@KzMeQE&8y|gRA z4(UP=LJ9}j$GNf#3k&rh@Ynx$PL!erAhCtK`3|T7Df9hLO|y)$&fFaA<9BRbfzR9? z05Rgz04CrU5C||ykqpP@Ri z7~lT2R`kTeQz+1Y$9CJ5I!79EbT)tu$HMe4TFTq^@9nOws;ac$u!wBM@?*hlljB&lT#JjKpkM^6hpJt@`xC71Kf=@D zTaYyx-ZC!Q`(nFyYmXRg&^&fb*T^X2Up7f!*j+ogOikIeX%hrp$nPiW=68vT>YO~8 zcGvBonnA7;A@9z8ydMat$SW($vC|+YpuGGbP*$hvj+ySF;}ZHVi3zYL+< zta;ewtJgIAV+nap3FZ-vMyZkb%;8HqgZlwR@9tFPDmpB$BX1_Sckj!O3z=FDXd2yj zC2I|5uK$;#*^o|+t=$}Vb8&G&1Lf!CP5S8kqBud;?Rs(JXYNd+EOYOXeAQu#XuyV- z4xDH-#}N~K027cL&r5pSz={x850d#29*f9z&|T5yJ+-I`+ACj!S49)*QEaKhFu1%x zUqq307#It7P{ryJpjEQ?r_RBvY-pukwsRAlH)J;e*9`3UFD=aAhprme6>xrg(B2rZ zL`Pv^Vd)ti&3UP^0Cdc}abxnoMay#X6J}ZqPed>%tgS~=SqNRPm3Uz8ZL%n=4WpiC zHZyll+EmlF%=m3Amo+W(VTTrjYKFxp3jP&Ef56H7Cp-;LyymozYG{0h69uTszJI^; z?%f;E@XeIwrzW?3`<69*Sl{3=Ee>r^PR^)&t<#D0YVgbL`8Q?ZMt~^$D>2eJ;)dWX zeKHLaLk*C^h2L!wWlFC>6;a>QbnmX6E(OYRzHs67!$a2!X3p&S>g#<5(K&%;&jD*bxyCk)F#DfQ49R(tQxI z;oWV)kcfy{2%Y0#CXoiVqC@Q)jmyf(q4D1<%voqkKsDqt2-3Zw!ifYsGIMh?_Gm?1 z(9>fHSqt@egqtIi!!o1DTywg!Gk38#1{L#hP(06pOH5d`&!0c5X=xE#5;_{_`q{yb z!resfSorH8YFeO^{X_G<6dfvCg4ddRBih4XA1v?UtAvyy0p&G6S^|BRfS@3+px_aQ zvx`wVdflTQ2Nr)OM-WvDDU!Ie@NO_n*y{_oUlhIQ;zBxlY$hFFT_c$5FV;U4zEQou|gSd2(`c?2O9G$`r_oMxmIEIi0lXZQs5M;3r+DH<3Jbiejx|155%9m7t zx1^wC8^1iyj>Y|4aG~!fz%Q8pPrE#z1Kr<$QHToG7G=a#+E7z^6KrwYkr<(+9EPep z^W-4fd`gF;$#5rJDe5@h^D|@G=DyRLO40sIRj%KC3kZtu5ty#qs8*zOPXP}%et&ag zJD?k3;1nEc9VU80jpIA&fWW=SP|D6iA*scI`^l zO;%z4ry!&!dR-?Fa*vc$ioq#Q7)7pg&--?1MI(m8miWDFSPL%ggVCs(#4n6?+1_a*(^-OWBbcUNQ@G#i;v&FBXcEcKS=KOh0 zCvO}s%UYh`7qg@lF3xtwt-JUkqTqh(Z9i1>)IA*PcSVe{@&nlNO&aQSgG z^T`t*8(+V+^YIyyUTaazgx zH(cMe^KnK-sL8;g!DeU%*YIa_6lg|zAf1DKRcG|g%#MLI>)6QCEVzIFJ&Mlx^G}TH zJ_mTAPdP%k(V^jCm)at{zA%`&8-uMDNl4_vO9!f~mduz^Q)Ea06&w z%e>gKxq&!Fa~b$URF2u%*^xGa1R0DPKjgNenQ(k7W=xR zj|ze!2ceV>;nbU(?ir{D9OGmpgd+D5jvoU9%Y8)<8bnA5pj*`KEb)vYYmsqW4+uN1 zw$1G>t%E9Bz_=P~oKowczq!OR1Xa%#@}=EDiA*2x_M}Y&IQ@0!&vESAW}XEs02lNY zeH)v{?RDg_uFN{m>Z{&CV z9np7g-#&~E^2W^Q{QT4TWvB0BYJ2Y81m|xHInCiVw;RkG0@&*L;jG+dV@Bqc=|7IB z&aOY+Z?ZdozGSRh`bRw8(Rh!vbSh|w{1|CaMqv1Lz4zZgzF}D%&=m5a!c&LF#Iv%E zAQRrg*0XENS>idhyAu_qgoY=LYBkN{C8PUh37J=FT#tdPZcf+c`7=KLvzeJ1OmIyE zk9J~WVnp;qcNuvLzx&`$jEszQ;ZRJ!^UAhKsPXgXq}bS4t?jux>pp!FLO*f)-I(H~Nayw+Xh1gOXk0>@R@6+L-FeSctoG578x2@iD8euF-=lL7 zJ@+B@N~%VT`|w8VSXzXFVSzI*76lZ()8lled{<6!`X)GWlTd6Sex<}<(!7hCU;g^w`8u-c(nDX7* zw;=$7mG)bGno^e3%@pSPH-dAtD|DK$0N?lXo^>KG82I*L52y5)?&$b^7p@{|RIvY~ z=?8X6hh*&jbANShFUG)t(J93rM^T zz8MSLM~@j87^rf}I0{%R>P;e0+SD zwjV@Fqo=3;3GS_Q%t2Cfa1KZY^hg5|d1k8%IX?+c($a!XW*Z+t-*Lv;TDAM~!meK@ z&K3oL0Bi?6c`q(53cf2iDAfC8GjdD|4}%8j@9C)+jUH8jdPCGcd_}Xuf~z-C>R>zb z^!9H0C&>qYrXUaOb?VXtIwzz%T(3UC%IJrA;osaIH7YUV^(*@o#A(-d-rCmK{(n{D#S2292o zTU&J`V4^B#W@Pk2tB3M;gqN2$B|H0=Q|a{G8n?sZBk9L?Y)20UMe1!+Qxv>@bBq~b zz?kdSt>Z(jue-aeZ<;xpzMN=nD8@v!n+K^NWtZ|LnCCJP&_LOK&4{4+{SFkX{L11a zY=Ws0{30YUo15>zizPL1@^0co)Sb<{jFl0PQ{WffkKQAQK5^6L&5fK+C5NCjk;j7a z;tD>ETP7v)vncWuV9sIvG)-Q}x$f@nF96fUB&WHv|qVO0ii-QM$U$|}uU-uBm-TrJb;uVzAadFv01W1q-)q)E|VxRhIaBoyQNk>Oh zlfLwCyFFrGk@6Pi1>#mg3GoU3VgaR01LPbZg=j*u7YFFEJAP4-k%yqj6Z{DD3rVX> zaN{jFz=^K{Fe?h?71&w~vY!h|Ca3}h64-U>)G2Czm;yBbju5~2*J?Izu5S`x3AI5v zaD^cb?uyN%xZo!VD~n9>o8{HvMIpNE z^rCnjLdWtD3^Ft2*~^z{$jR-{Hqi=BaB(-S0ja7BNp#n#l32nmQXa#vJB7mSzOb_c zI`YF?x0qIu=bQq2ZS%WfEKxB@rvbKv?MR{!%L@w%8n80Oze6I6VbCDzDx?LMX$Npj zDAiT4p728?b?vQOV~lc2m^tF_0s4fvF$@0@Sk(qdnnamZQ&Wp!-=1uXKOrd^3Kv3H z7p}%`UisT3KM{w@&!4wmy^;Z5G)o?c_<0nEkU(_&EVxD;;Af{ivavJUrS?B@;AX=w#2giO(Z8GfGP0$HrxE;@m)A`7<#=G6CS{ zxzNjUW{|-DvFgOXM65k!N?xLVXf~W9 z5txY?-bp415GYi{K8eEU6I@pe0~9fRkU(oeY?yE~4a$``3d+cAS@=6r z1(yHTty>hJs{_cJh6n&mC)9adN;teDns3h@mST^w8aO8#Fbx5I+Y)CRp1RlI6L^`{ z&N4n%1xu_13m@=N5O@e;WCJ`sVra+)7Z0t6hew&xaZZR;;3rQ5Y^kZMUx5!CeaH2$ z&#jfPXJg}1D)8Du@(6<|jq2UIcjMF3Z{W0=K!*RrWgHA-QGw6O^B6mYaUTm}5F`&d zHH5(BNX1YMxUpKqH>|wj=+CV{M6O-84hAL)g&4x)M0V|Rh-wxT5O@f)G-g}=XUqKO z`R%E=0)~JG?8_|UJx@)Tyb=ARVMoqEh&$tSLlvsTqy<*Zsp%^Q5RvxmYghCzwbcwo{Cye1e~U1BY}-Xdkr>*DqlsqHZ5?bf`F!kt%supKwjePdsbr9 zg#X?r_-jVUTI>46H-JkKD3FzK+}9tNq(4`m7>2&|kO=D^F3a?&x*vd8zqEJ*4M!d!+CJ2oq ze!7A%zEvMTGC;F`B9U?nLT4i15^&=6~|bB`i1w-Z;b z_-Vxmtwe4Ed?ZHT#9WEjG!R99D%A3&I_rKhik8(RWNfDBwfhw}+;7X)p^ zi*JKlzreY7RnDo>b>N7AA8&FIAHo&WR41HIHNI(!nersbEpfE!|^?KdZS z4R43t!}ZtB1(r~1Y-iQ$YqS?&ZUiW@KsAY#^AOuCYS@rG90FvIQ}+~Ih_Z?b+}K1& z0FOdLXl3x^I8|A2T*NE*a?ozzuExp4%v%WPXZXc=MMX8O(52Zu?0i|6x?=(5X}0XrXRlgaR5fWV7!$f7pyQ z6sV;Z75-EmHaB6F1&y^DmXHHZhh|Q|UJXrV)IIh;2lVXh?6z&&D3~!I#6KEd@8-9$ zP{$^Sn&fhj%)czv|BwaEzr03cHgpgL>u^SkBc zxtN)m+g>^GU`a5(wVV75$^z2G)mN`PVif+uZ~?llw>d+f*w=A}VK~IMUPVDh>WpL1 ztxYgt1s=SLLI6WSQeh!Gq;V2U$n4Cp%46!Ktjb69+*!C3%Z9ze#^;^TXM|E7lpuC2Yjvc0`Xb=vM= zCI*h6Ui3lSd|O|yia947nZgG*u3z64^^QWpz{cV7%S-L9FCMEHp=%y55!A%k5J{_` zcUdsK2~c_KXlet-J7Z$Dz=B35R_NKpDMYv_E%l9YT)Z0{-HV+@ilQNw%2H4z+gR5@ z$)>{0$n3&G6-KC#l;mARa*Hx@6uXx&thAZ}BDJ!!lMDF6*~23kT0FM9x4*x6m+E~c zW^h?#gb`&5X<(bA5oXWNb)WNzSp;}5)>Z%BY1 zo1LF0gIlN`WMR6Kx?=P)lo6gt&P-dj)P}j7$5<)IDY*CF3O)gO=o6fk2YBy1@%>0n zpB(ZC1+_B>N68WenoYuz`hdK=^j|iCsCUjt-vR;x+Y}VIi4h7(0(Ldj$-*M>%ZsyO zU&_t`NsF(53VQhX@!J1J7yE91E{E6YBvUq(=H4M^_6+eD8bzx(D* zpf`GrB&cWPu@VxvoT?V#SJyQ*e&5o9^VyFa24=)tdhgy<6hJ2ASEvNxKw*|}99#m= zShTn)F6T)~3N;$(w;w(z1K-8`+|0`AT!#vLf$KT?O=o9kf6OSMWrLWB0l)RAanSc- zYIn^&0Acjs!0PQ^Y`et7RIp#@iWHDDX@si(i+UPEW9jL_Gm>#?$4gNF0D`Ze z^AqF4-2pB_iq3qpMk~Nv_5z1RV^-Le3SE1DRbzzmP=>TldeFtQg}yuz(mY zP{EPGvATfJ$Poj2M~XQFC5zNISs_T@#fxtxVeg`!G$a$lCr`Et-P?tH2|CB)O%c8v zJU9%Jg@cpGi>8&k{|$-m#D3M0ApilY+Q+0 zBy+?sUi8b#Vs`QC84oiiW($-VLpZq62*8qYb#8L-;8O`h%v&P0i|KxX=jgK7&YgY3 zJkIIMWPX>+0xhlho;`TVf#pqWsU1|De`3CK=Jor3L*mYU_UF$B0N`A_D2Yde^i$Dd zae}r5nxih7L!omJxAyJd{~DVH9O%%r9jSo&iXw%t(!kvJpGt>eEcjy;$GS@8hO^Ge zDq?6C!gE2&?5F{HUaTv4sBbVNQl+b3M#hL=fD&cR*og z#tR)bCV@v6yuqA;0Ki{(G(64*U?(gc=3gIzE7j;Y0m|>RlT*E$UWfd3&1p$aiAVeG z?Cmcs&AW+(B2_5?DPO;FBkAaVa8G2&6)!UyYmFlfuO8XPwzk-XgVGWbR}{DTA5Q8C zH^}TW!$P2_>gwxX1LQt#U_d6+-o&@lS^u9G;LRJ;rB=XaP*5?aGnc3T{v|dfeQo(b z9Oxd5Fs@y@wr5b|R}cb0=Fd*dr;!<0zBHh6WP6674u)O%%MMB zpwj`w=iuZVQ~!Q~OP?~TUUYo=ApIFy0bhJFgY{=>(auevcJ$na;!bBo3Y_Ul$+{47GQnqcPNaC`rdTS=QfM zE(Ykn{Wp)tmt%Q`^ssW1(P^@eoed34kod{0FC6_cl@*PRx4BG;=`xPL4_kY@6t%4| zg)ni&Ot64dxTSn)ZM_=O-ke-qU+*Ct02PG%CaecC#&14-8ohF{8QujxQBnHH$Vg&d zD{A2%lRDw%CJjce&`U>8?<$A}*v_}1>?;T0+63IYFuE!%lWXn6k8cu-cdzR-Bht5p zy?tHSW&nZTx);^+02N}n(~(#JAPEi3#KH02W8cjnaA~jQ#!9Y1G20ADZ5G22-W^uY zxv{tGWM*Wd#lYMG-Yv3Z#r+O4AFf5;y<6GU#Z5Xs*lp(+gDqt5APS*-<-s|09@>uE z$v@DtSg-}e>;_F827NOQ&ZcT{%?K-7+fX=!deL7D%CV}fW@dK(-A0ArA_HnfaNTFTm;gR**?;00nEX@KXe+%OI%#MVx#tf*B&DDO}*HaaoB`<@uZ|%R9IQO z2qGmCHYaK@vrt7t$8?GgkYwA$FONnAx?`Jf@RX#y0hCuzYyLji_v?+)Rsb*N;}6r;qk@9iv8S=M$5HlB*GRAB6I>~7_sVP9ZV{aCEaoI8 z-tWwP8vb6Z5rGT%Odb(0z2?7{zIXpVADs0JO=G>I+~MWeB_G(0iC#V4TQ$d+tpPID z5Ed}BHp@KTGhCbBw3IU}p37L*m=b!ujf#2wX2tW&oR+IMt>Lx1x}D>glpTxJRdr3x zN9s43p_%pAtdXBhdYF|sQ~^!LnCmGvNPvg1OhYqbjI7K`Vs|gX2=ZD0@I|*STbc&(odWOggVnI3ql2S!;O@{?Of?82 zzve+VD@%S$-CAMImR1XIat*YqKp>H_Wbp$SJ*|XV{1jpn*-p$;wv~8F7XLU&OQvWY zyGrOlA`rU>nw2oeuxeKxFe7+x&*IUi72=(pm_WiY4g&cB!+}iEMe3#rLGm#lK?|?0hT#9mB+b`r9GvZEh@)=1xBHt zkIo?s;1(e5H}qxF#Nj;8g$qPMnT6_OIJ3xD%`r<~HD!p)1+E~)|4Qq=h7PR?cUh2$ zC`gZ-qyq%8Gaz?Y(`xm&n|Zoc!c|l>c;-HNum<(M0k=k)Q{1dUA|PyMBc0f(l92D; zEk(ijjfw2JCOys`H1ruMowP-OZxdXBL&_Vt4^)t0%u5rNV6d*a)s9)7r<9GVcgUNc zI+ZvkLSMxRAW9lq;0l~-&>91Bw>9a|*TP#Q0j3RmLVxnP1zVb`2{a<~pcq1}^0)CI}DD zzv)GOKD-v;Fsq%0&6Tj_-H}D5nC;`2l#G%dmt(UhxuFkAX&>BesF#Cs>zb%AXd&UZ z?1PnK1934})c4%6rtum>h42y?rw952DTn~L*{q|lujK43j@w(rz5kv^cQeVAVIt@K zSM+9Bn8djcY*-xD{{j{-#f@x5#y|0!yOS(D7O_=W*d8=}4NXmbavV*o_EH|W7NK#% z_Z!Xl1lb0m8k5T>0IVTUlPd3{I{Xqs#8cNHy)vt}TmSoPeLcNOM6jXElB4LpeQcy* zC6lrcgE(Lcd18M-<~2=ElmbGY%$VY8m$~ptaXob)YH&t7{~3ao=YNrw7K@Vrwxy%o z?}1zW81<7Yxcujub>~zr=AtooiYFf-OYiRKC^g4`bx5FA`pTT4sGNL8y-t2q55{4l6;Y3e64`4X0PoN;&#!Xf-wbr!ij zhSgL~?-dgpTL^~oaXZZwgncjxdyDZW#R(W%#Ia%sDefDC2aupgkU!YD+3+{6OnWIj z<%ARfkN`%)duqPIwFxvBKLpCR!h9+21w!B-hlX5(u@9q}MXkzz&%e_5`}fU&0P2oX z@0c+WO-bapJj*xh$nXu>2PoJ*XhOi`lqE|f_%l>|nFfX?WbF6vl{mLU?l5kLz`w|g z#`W*tzqnkbPwsXzY#afo^m6;U)eTpp(;x672Ltj7mLnKnvl>hl834}6*baLZsT#0z zvZ2B4M;N_pqxhKpLaKWE>C?H+N%ny?q@;o4ufUL}I0)Q-t~~V}n54DrD)GeSIlXe% zH7yx%?^Emw^MCb#vqR%LNmo`iH(z67H~>lDC-o_z3-3KB*t`XmVQ02$>|m!u{w+nzIP#mvj-v78_^a1%=Cn3^6e zjnvB2-rn9gY@*~q1ld;LW)E1+&R!dtQ10OB21Nwo@C_7q!f@l7h6%2)$W`RSHzGHn zgrJ09CFx=#7~^0lwq)jj@Td^HX2S(Gky3#4jizjZ>&Ur08UWxA7w_&i4~GvXZ{e$Z z9{CmU7tL!vqfe+DP9;h!1@4#+9<2BNB|Y)_3-0bJ=uZC)r{3Y_#drlO7;X;`A~aS}pCpR~YC$~k zYP`JnUF6bOt%sSJ*P$y^zJ5&w=`8>}d3WR`@}3|E5S1Y~6Jh_(M53hoZ$t1d&m89y zp1|AnE1ZXu^9D#N6rSgjAvdeZqw@oc0yX1yq3`6;v2lX&xS9cwaWBN>KrHeESLhbZ zniA7F8K3b1IM6Ej2Vf-usc6VikO4 z;j+Sy21Odd#6v)5o^?HU2}CXa~(;; zZ8wBNl05gpAESA=9Et%=y8CX^fm9@@Qy-AYnORsWiv}e%w|s@H3HCXJco&hGMZ1i-~R!&!*+v4TjClyruk(?8dj4CkP6 zgIxj{%);v?>=(p~J?pnJwqgPK`oFxC4zD}dS$^~?IQaK!|7DL|&hGC2ZkcGC_&4xa z)!4{_QJgAxxShwkcq zcyk8y%DS4lpM3xFcGFde^^b62*1svyqpt{t{1_*L6V2Z6`ST%ILBV`e(DNWO$y4wm z4rkAvorT$-l%C#qP{W)76yPwmL=OO->4%(NI+Qy_4Cdi9ANlVzufY_ifK=MELmmE$r;xdq}0xTCN_K??a^qgl0x5?YM|H z*~WKr=o}mOF$`?R`4CnPQi=Wv$5I(cNVN9$_9dusLkdP`2^z=dA@s@JB?uMG=U&4UAGjst#MPBe3*CX^o|wF1 zgtLaQc?f_kA4z&`3`sIWiGi(+o(wOguarU{NfAUQg`2DEvkq zK4~U4V-z3IptJRU%T0lFa;G%T-UUOZCU6*3&S51bW_I>>?`48|BeAd08%QoET>?0* zzEj?dvl1{_Y=YOL7v8Dw5@+jRDp<+xNL0OA|Bo;w2` zlE?)F)02LuRv)A-44p8Zgu>EUft#>E5tIG^eoqPoKW3GD<#$Tpvu7?h7<_t+j3j|d z$q1fVY^Jrvzk)sJGt=o2ixEK>EluK8Hb=_dvlW}<9mWkYa3v%qF%Fxh;3%E*DYf2!rR%0 z;o+*ZV{w2j|0X}=N_u*tR>(reH?%yCIuM? zFbc&p|5DniR*qTXu|DXh0LmIBY8o^Q(q2md12EOU!ANTml9o&7#8<}qk()j;RK(yi zAxQLBL!fuw!nC(Ty(}H|)syv^bF;G`U7 zQ8Vt1d%-Q`p3KdjnVclIq!o+n*7KYEdp5>Lxw#Rjl`q@;qf=?h5Hd(3m?u7@40Yj0 z&dAQ~+h60!BrGba*9IO&Cj5>_1&`CSU%JZm09bY~FyE0dYth2fX=v05Z5jq3A`q?i z&hE7QXTPBb)}Spd+Q(ca?vRn2TT2QS?!`QxYy|L4O5uig#%YfpHJ(2SKfF`f9HCwh zqzoIKNe_Athuv#v0c5mO%0%LBDh(=Pc?2}09A0vT7H&~lN8Fm4nwsNe+;L@jsQ`Xb zqG4P#4jM8g3o_3jBk?iX2z)wN4=RjNzE|Q4vM^Rs zHpJ92%Gv!pN|r#F2q6eic3aZz?_IxsG-?nbCV@7mWTwmefWWZl{~nTV@A)w*Mcu$l$vL+AK$Wi_?6#S3ey)G`o}1XtilW;4#4VCj)K z$$*YoY|hz_49!D>gS1dhxErT5|Es$+u-xGM96E6#T5f%7XZ^W@mlxS?Y~E$(wrwjS zl_YcggjwuOi`nUR`&`J6=^vk;!!!)>AOL6=iF~c|Ec+nq`9}a;p65{%6BF26BM@8Y zS~sHO-H-H&U;FZ#jt0iy^IxB(w)lD(*j}PB-^SDfqiXe??fREAni$3Kg`O*SKgWOYNYar9V ztPiJ13#o>1T+7P~PNt<^Wayiu@8r=19FQx@3x7WeC|t|U%DMqK@Ahq8&*MMrJu(h| z5OIohzPzyv(=2Gy_TM2TI5ehT4gxBXEA8U;I#eRjKsjn@(Pa-ZfFF{46GkUO(N=)h zJPoHE>011)zui#WM((V={QszW4|uNo_Wl1OrG;om+T&``&?IGKwD&|i8k$51T~rz> z6+=svk3>7MMRBb_(BQwVpIPgZ-O-FgArB@3U7|mgh`Gsk4|)_yvJZ`e<>RN3wJgre z{%W=>HbdiLe*vH{Zp6;j3&Al2(u^;<2vvT}KEwmM_FM-*KjDQA2}S}Hm1^FLHjOxd zNHXg4O?|F!o$ne;-vOH=EIPV1jdT>B$_g4n#8ULk^mEJf6I={DLih;OLH65UTe8AI zoLMkO!OHm}k{PG+f}uFgO}}8P*6DAwmdLY6djOY&b_Iu0`%lgw`*K<4qAi8@AG z+OY1yjf;*zk;OO{zuJUdZsXJ__(@uKt+Pj3H?f~ttv?m86z{HXsVVpG4{XzRK&MW1 z#6ctW=k}ELYiveNnX)f<>B0jmh1Qf6L47P{v;Y<73nq=Z^=>pBGa*JBU!K}X^Yp7l zuE!=m|8&5Rr;fSmt6cV=aPFvZVv2vmommgfnB~;=ea%Ic>c$##gL-<1-7Ye5H=+eA z`b63~Ub+wxKooE_a~w{59R(r5*TEfHC(p$}hi~C{@FD>S+gd&lH5#OsDGbC5bJgV1 zV~DgTgcusbPZ3H4%SUqA(z9i&R)gdAz1hd!lf;AD7lT!#@o+QpWyGv~q5Q&=XV2(= z(uKj&`n+237#df?xN)#ENuH2&_^=WeRF;2vw97T6@7OHve7pE84tHyr7-{?)N-8GS zEBeUNa*Yu)c&?D&Eqgt-8OK4yW&i)wLiFYE@U<4!?|s{U@=;25^!vm0zD8?iJv;(7 z7XVYl6dVX){UhSzj~Hl90)DD+FmmJSd8U%Xo_XWjp3C>EUODq_KYe|l^XFIY%lgu< z1~8^(tM8tmpsgYa!8R3|N*)P1gb>3GH>-3#3`g`0V8LZ-v@H2{OKtv|H)!Ym3>N5C zpZ@2c2cIpqtLtioW-qriv{@H=jCb-PwYyQXhF(QoJ5=}ZIQKf0lnvr$;4l|>>X#q9 zMy<2>DYO^iS>6PhE@>*o&n9Ble+HzjwY1<>eRWjly*vlRNoS=C86hZ+cz<%0=R1ZT zKAcizi&5f-1kjw7lP7Jg{<%eMndExTJIE+m?J+8m7*Qh5%IFeWUX|J`)X&W}`SWFV z6;Yjh$9nNx_dzARFi@nGm;5_aCs44ZRU*lZx#QvB(1&m?dM%L%35L@S&Gx5}c= zGl057Xgojw@vr33vCRN*IuixC*l~|f(9%Ar=iCradwY9Kont0l2rfwJ@a2V*;!((a z&+3N(Qt^VXxn(TqJjDA zH%BL_9i8-|^E2zz4oBaGn5nr9L=J`H^7$9Tc9s9!^8N6Z`c}7h7wGk&H#+Y@Yxu|F zpZx4RPGVsW{rxBHHq>~sP^+AWgva41@`rZAUBznfe7{i4OOdyjYi~zu%Z50Lu`_J} z#i~xpH*TxV?9=C5ySOmgsYX#{IV-x17}1`r+WNMI`^YYE{ya6m$J>#m9H;U_2&P~q z#N4^Fr=w}D`)S)D7qe^&9>%xsWCD>ZmOFV#%I7=Qlz&+$>JtNl!&$R915t)4cugo_ z3L2FccxGZePI5UXVe8qm#6@F%8PHqAXnzcWJCl4Flmb>HPt&+{`^MhAllST5i;Rsp zDPBWVG6lc(^&4Bz1N&iAB>b*FKr4!Flv8U1ing?vK{)wVl-UFTK;%}Ldp1@(J2-qE z6nFPQolYi54rV;_ZQ`N7!q+!6_3`8P9~GoAaAo$6N7vA;BP1HOv73;J{q*gBxbPrE zwcMDYOQ{0w6JVYFzeiWOT(o^wU9Ys-=W< zpAgc40w7KDs)CP$StdTzqVn~4=#rJs?3Q_W?7&itxdp^vo38fzh0o9lj9OW>iIKtE z%T?E+GZGEof8wF#-`Ws#W@V9t_FIu$PZtj5o{i6#w3@70moi{5i8ETiK@HVQ^ zm@PG6;o>oT@Kp!5?V#iWxxxBa-=1Bm5o$wV#hqzQn8_@66>d#*cb^POo5CW+A9vxdjij*3^q5 zd?!Y>)1$~(IawaHnwGxrcSAQT))Pm^#Yy{El@`0(iJB}i(snW@JF+#Gu*-J;o|^Q@ zt8+#lJjZwh`BQ&y0XakLs14jd&_6cI{is^Jydazm%1+bDHSB>Z2_M7Vt?DV6PNF z#RZ@S#%5%+Zy(nkQ6qlY+Ru$2(52Ryjq>6yiXCdJ=$cE&Gf_R>u4)VD(MvVll3ay{ zzoy`^&aKXg!RPoH-2L9r5qpPk^uuoAH9yL2Y~qY` zHgkkP6Zd?i7UB}IR2Tetm!ZlNW69_qH5tq&<%@NQ8E~_7gzal8}w&;KP4A^&;${T!v z;USQgh{Hc??u}MuesHJd_WeE^n?*?P;d3nB z;;dzFUKu#yuU#pQdnt4wJnM`15+4;@kYy{hs(7$|97yqT8%lM9a^-VY04~0nQQOH;3{H)y7*oH*e(6%Al}iuXpU$O@(M^M%gfb;X-AE zG2m&1-BkLkoACtV=rFsW!&-j2qVX@-ZH1O?8Fdf5zLw zU6V$!$OT~njbRvt7sudNooPMAXC6B-I?2Rwmke3FhbdCmG@)nv`ojL%V{wZFw+2Am zcIp&u9nA$Ty%1m4hz2kpSWtbpB0cjT^ys1E?FcZ9K84V0EF(ye-~`l$vzD17`=^JI zIRn4(7V7eSYig=Dvf`7`+6mlDDZNEq=YjLsE!`3DqZv1YQwDJ**s5?L^7KK~1r{{$ zy?2FfVlG$-MeVC?7a<7=wDag3f}%k^>JZen^~Y~29^?b!xldD%NNCtGn~0l2Q*Zq3 z2wEKAps%`$H4aJQhh~IQtTT&)g9AI{<}i{CP=}+Q`hM!cqV?+sK{)4?lgFn-NTY>pQ1(uo zI582duOF>3@?3z1+$|Qx#_$}3z`=Y07arN;q=iC~_nvc272>Dl%8yYI`lUN|?05}D zz*c(=dO#g=7xliZ2RivLr(J4u8nRLR>&4<4J{1KW#X=Id^BRg02B6HtW}F=*popKP zSA`=$q9VL4x9x*&BkpV6jK$m~UiGSgp2G4`p9{Ce0Z<=g_Suttnr1!_b9v}*kc^w~ z=XCg@Z}cIGTwXrwpss*>JZi)Qol!X=!WH#9Y8b2+hpnGu)P@S$o9EFMUr2c#Ajtf% zsJ&_4{5WiEgalW(5K$U{mdJu-OWlImgjCNtmh$nU%5_jL$WZYOg3B`PpfL{*3wd!o zq@w=Zrzi zE&IivA$ZBxV@JS)6lXR-S+KYdrNJFD22#a{Yr9-N&zCi@@IxGvJ{Ub zx-9beZ3{AK85V+m+|8|`-P6Mp0!b+IZq!ui$s0i5GGxe*#HUZ?C9Lpq+tLc&1V;!} z>4-teATdrFv0!U+`h8#JY6SpS+jU4OjjO>Snjv&>h9n!FyZ)JK=V^F1i~^zR%+Xe7 zBuCKOqM|V%BkmT*bc_2_t#M{m9pBAfBb7k_t{vt1QNn<+mdH!I~o9HWBKc?(s7&q^l3PFYE0Jm{C$gew;yp5;Yc$DnuOzY z)qmC5Hvgp<1sGL)8i1im7b>vdfbdy1k1by%UO0XF7*;V8IVJI;^FjZ9;&R92zV3`# zIYz^@KPU!93e%T8-bqhN2FwQSU~92gzNhtL=U6N{BTzf;7Z(GLIP zH1Z}(a4m*TJeCiFhaV-c+edCRj<5}9naBsnSrx?#oyA0N8N_V9{+Yv%gd!z3uc>}3AYIU(1`LKhq<<>B;4Gr!#)-NQK`zWNJ8e@*g`}!G+j?y;}u@G;u^ZvAu*5mo!6z&}W138FE(z>c` zSmL>;u;TWW-o1M5ApBX)nst!A^(5yb?cVSZj}a^V(f-)n@CYs$GTF3cN-)dq^3|*1 zL`|d+J)x^!qQ8YP!5)73VW(~<-%ueR7PE@-($}^9#&7Oj397P9;b8<&7vHC49w6&} za&p)s+i&Lvl}0&anG^1=Fv%b%?m0FcO%-cbueM*K1UPEhAu+wj@KqEFO-vaem)g`9C#78Y9>*%7xQrqZk?CdD+cw6 zjN21l(kkOJ;Z>T=3Eycn+LZ6={8c)dL29K7c76TeB^TF5*vXU(8eN}oB%hY4YD4<# zAp(+qUV806nVGlRbE40ameX5akB8Qq%$r?`{^H!VNA>yyzsQYq))PINko=Tn?-Eib z5rmW!R7HKpOMa%N=GQHCpdCv_^rt|)aw?NI?rNOUdDN)e>cP3(Rw+@*@x@w$VI#y8 zNa#4sjoQh02Iv`sjR)zZd>*r|yNFA$vvmAj7OOq$w@1G|%^K8w&IQOmgmkA(X**^r zy?~?WN}XI~Yzk&a7|gt3+6LJf9}?M2RkN?f|g$hr~Jf7P)%Y4x(D zedAt@=jPzvA^90;jV$Iq<0GED+9!f13}JVcT(3|E%l~*~k9%ZP)LjI&w39ZZ6-!QY zir?ofx+C=Uu6=uNdTx8_4Ssk&Ae;s|FvviK3^k*>&4pJUvs4BTkq9)XTjM-797Z80 zCnuV0LoRFJ@@I0=EB5T&`!`kM>&R3i;rs}T0x0rN^~$2qXI}Km zY-GnBzb+{eRe)$LkOu~@zt|g?iIzGJ5#=1xl<~97-R|A|VQj{%qGdk@(nunndXem5 zF~~z*RGSU4orT#H>W>>?{D!8Ejv9G@G!B3}In21j)|S~i;!Gtcj6q_oPN>_s^cv;4a;BgeY(;9s-H5C|*MHGt zvNX(eRp9Hn>90B#-5vz*dXP2y#n=8tmx<4aF#sIzlBPu$GUZQ(%j8q$NfpG4<(yaN z-i8j18I&0ck3=E`V|!`Ei~b7f45te(jdn#^ZBN(~ZD~TN6>Y^uEKuq4RFUorUd{1z&joo)$myQ@I)b)bZn|0 zWVM}KpWSYxpw&O&sUcq$f$qVZ)lf+K3C2UOF+flkQl^jKp5nx`hWYon~ zXqbW$z zv997BeJ-v}Pq&LfCQN@FF7}veAdx-uXEDy1z+ov~B#OT%P~EVA?}X}b7!_4qnd3(-^r$QP5iJB!RLEBQoyg^_yB_{9t4*(_+whRu{Or692w?x&v7Sgg(UDXC0d~`Us{YV+Zebg ztMFk}e3z`n>l`*+d!IE)0>qV1D1xORp?Vo-yF7hV4dIPO$yVryo>^Z|9Gn7NCIcbs ziF@2a1m)+hOSvaK5MWp$$dfT-pk!}tXD}=gAqQ()NqUWwr4NuTA_YGws5w4aG8GWn zh_(W`%O=cd@}S#Z8G6|TY5L-Iv@T=BhO)XCD&M3#Kx=X2#fws+cB!s(*ce#g^zYiP z2Pe?QdAR645=+hn(UKr_s-wU?qae14)jmGFfVwcae+ir*G7;$}2<~6=V^8>JV7K6+ zz@AOFxHhKwEC~o#I;OVu-yU~p zVDCr}8LH1k_I;^ng%uioQq$j5MmZFwp0S*TE^D}Dp#C1cOb*qYk z(B;ds_2XA~>0wE<3HpRoD=9nuU}0+xS7DQzH$Rl{AbQNztFl9T-# z^=>>%Ct@)oTRHAAW>)Ui>x}t@fiGklWpt6ziEZT}fnXke#xro_X|dbf?H8U7f>X(? z@c3QWT_z*aK*rfaCsu3cJ&KY0iq7H&(Le!BGZNlQAqpQrm!<2gADzRdc^bg8lFRy% zY4qbWyVX1+{@(X&-r1I6%VPFEwAlN-u0k4`ai0X)76cb(1QFkUevr|Eb_+(1?)sxy zasChwp09NDeusVT{=BqL*!bt&DH2{xasPVQqdo%$4Qgp}?4nh@#bn3|FOD3`BV*!q zwR9t5mLI#VHiy|=Ykp98h$R6I65_G|vI|T6bkPg7ft4dH8zbx1&?E}Z?lb1V4|MMV zhx*swld!g6@z?8PH)OtyU9obwsMhuMn}d^YB@j_Bb5N>*wXHm~tGC|lIfK)2;AtlV zMt?jSdzXT@R*BkAaV25*elxdg!H=J(-nh5^#WVNh0bM#Jy4?o^Eq?#^??&#*;niZJRSx7Mkxm-mcIVoEL2}ujY*{>GZtYu5-Ff(AF<_7 ziwsNI7e zL^{I9^wrn@RC;H)0%m=!@+C`U;+{g<^2Gy&cEe<~KxNBnosG84x%IMBtJ-qQ;Mn;g zZ#r91dPTi~))T`?1SNf1c%Mq}g*HUpa?wA~;J~qC^MhXLm`%Psz1Z+V*w}&>!7yBDIHW>$uc`VeNHW`B(65z;s4dJ(w)y#uVy}oogUZH)H;h5b z(y(?7IZLIQ*!B_4Z4`B*90+FF}` z!W&kt(*GfYjviDp5CNM37obAt2bEc-f!CVky~u7zx-}1!mCorksdw+*y|MZkU6;mW z&LrAPzr~j;-eXOX74T)$m4M;AeQ}0E!uRBqU2{z**K51pQ%Zn=TiK*uDF)S3CuQkV zyf1F=*#H9@|ER8epEcG36Y|D|OsXkJO%b$#&oL=?LR=uVZO+?kx6N)nHNGVaTDA*? z;qdosGtYmcQ!$UkjcEU7;MOh`mR8qOb*_Haj|fX?xTcs_k+%5tjt>o;jsS1cU^@?7 zigROfRvKoqAUc--wN4xCsKH(k&@rDvY(?2ipu)a%p$W$0HvX^XgWT@iAb}Q}Q*o`J z4N8N-AN79D1`C*ec4pd%{A>m>++^OzgU|CQO3^8mn%gK58qj4vuBxJ#7urg|bSD4g zxPKidR2g@67x-7i|LA|$Wma?kA~YDbC}!{562A3tk<|(ev=G$5;2-<9ZPKa_6d2LKgouX! zU7F&8jh?`?#h061zxVQo7^fqID=#nC|JWvF%_;Ur90%R%oYJGHFPh(_^8HP zUnAFijMhq>9?Nl6{X!DnEpt|xap3=RzASz$9umORV$8#t4Rt{8&dRXVURE3WG5t}3 z5NESl-BqhrLEIg`a;3TGV3fnIIweO@?6`lq-3kS_|Dp45FE8TcCkt?x>RjOcXHo23 zVGw*Zw0@T?-OyEy>C`{-^w;`Dv+jKy@~9X3P)_iRBv$Vy*y<1NnM)0HkIr%&Z#uLH zn>q;7y&nDh_U*tMb>(?TF16(G5i3~`n0K$qu!(gP?MMR!*vJ!1O6&{M!#U5*D8s)o zZG?$k*86^0n3-vKbqo747>5;eOi_+=US|MzS<=sV1z-x#SL}Z=qx0uhWSW=w2P%={d_g^1NQ5k z(ecV`AQtYJ50K;dHDD6lCmgHI65k7lq;o_RsgEj%Iazj}2< z@dQ&5lo(d%2+dLdbx~YFLOdGe7}8@8Egec?dzqFU=bZI$ycsP!=Cl%UQ-x@#kLg`6 zml2N%(05I1+LwcFoAHoN^3`7tnFenB^)5)#Nnx1>ZtWdlpyMBP^>V@DXNwU>CDcwq z;YodfWn|J++qPqg!qzRnFgV(mMi9L(zt5aK`}To8N+T&=#6gh+;R8oT1e`M7oS}w> zhM0kzv2iw?|FVG86&yOcr2X94Sr$$G&2kH_asIwa9TRIaDD(d+f-hXZ9wKg#$ngA% zwVq6jRC7OW_cMyj1+74LTI>t2?9n?8Rt42IPJH%|bgPHfa9gLS=5J>rdYv|X`edRr zEd-O`?!z*fIbnffWh%X;!eg>;t`51t*$%Jdc3Nc?l%ycymR;CXS6bG*&?}qe8*;AX zeObc8wOI!3t)5MBb&WNPQhXeZ6C3JLfdfzb7EwAsSgRPfkZ(;*!#F z`MdEm3LGr=x+Zo^oV_DI|+|Tu|tN1D4=TX5Y?^e^wfK)h^rd zDm&@`8$mmJ`i?@GwkBJpBI)+EAN~#-Lk-quEq#;{*sRahdM^*Hn(;JgcIPopZQ&)P z_l(pigD(0xN6z0?F-nW9a3PlMgT^;#`a(TFxJgE2Fs;U7a|pd!ls@1i{rnxUN@IxT z_Nkn8Kaw1;ky!|fgJyIJd1vmLMYhl{7%KYlROO5>8S?1k%6H?P9~?e%WGvD3QM+4@ zQ&KwP;nMCueJT@e z?x&_+c)E0s&KenBSk4xuR8a!0Xgg%}(fcF2kDWr5@8Ec@K}IAM0JF<===GZk8jKbw zxr9x=_DE^z1+bS47n#IAU;cV1TgjDtLT=~94u5MuLHAGL#zLGQOiVk}JhyPgnp zIWBM1PG{YTn-h}}T()Gxt>@G7&vZ7zqS%CtR8(+((gyY zD<=85{;qXddm<@C$w%+Nrm2Aj4z;&={nop&fGo61VtMlXtL1-w^Hz;tH55 z%{bR9iVZk!%XH4%{endx!Kxt(qmr^(x+*Vhh@BII7y`$f9kekxyW~9XJwql(L!?WB z`8s*4u(9tS2k+oIDAKA%bspSlT+Ns@Ap*3@{I+gm4|oBK&x4tcUsPDA6=*=4&C<+~ zSKW)YJ6OEDIBXZg;67gTtnrZPU(&`VR3%>=^~-3RR`~+-W_KxPR^vS-1A4*3f)35u z+t*KXQdpMB6*FVwW2Pl3k2BZ)0@c_I=Dd(XocEpt-TwuUX|UFs;0eeOg!jQ&S6VGu z)b~G)H=UebJv`utd%EZ$sYdV8yyJr==QSAv0Cc{=^xar@ZRS{SM#d-2n~dmsdVWsY zgPzx;iq6kA(lj1FzIb!4-e)d$iqjo*ID5-JJETAI!<6Gx>QuPrS7v&T zc^0M9?|RD3;*njNCQ%zp9l&+hx{CmL?F|~Bw^q%IzVLAe`fgG@LAj5u_}GssgBKon zJbmU&OX+vOLdqYFkvO%sTr+G(U#m?X=nrfkTN5=+&uH_6^O$aeC;&4u=X5;~n-@N+6W>-gYdqhPx` zu-jy=YeoBh+1rV^5G;F8c)`=GU9;vGbldvWg3Y`(*is=X$c*XH81Nf!j;`11sU|yG zaAdOeM~OKy$vmQ><6>M33;t_YH-AcRp?66Lq&E#S=4~=O00!3oITEvRvS1ND?}Z|w zINHsDMzi`AG(#b5`1`bJa99oUelZ~Je?$j$0K=OsS$jYO-f9{gVX}P^0&?q>w#Hx-)6Pg8VV5Q8t-*QQ;H> zC}xJN$*;>MsSg_UFxa4$cZrJH8WLh_(P#VoVJ$Q}lAnu@U&DsR?(T--ZHFZztw|{l zzn?Jk0kUj6%&`FssVFYsV%qBH^kahAAj>w%httz1i)4gShpi*b3A=}E;(3P)+199` zqRKDw8yia+# zOXbfSHMdF2CeD~qAJkcNEz-3KbX&gb?zawbpWbXCmIXSl2AnApO^NT`Teg^Q?Hy*S zU9N^ZULxC1nz|@afsbT5h(P~*Pa>~o*uqO}2yk~X%%BZigjQ%-kvY88ibwObeQcBC zjPLfMY)Qt85I~9S;AZ@WucEU-gfWtxh$?7j3dVwT6~gog)70rES)_0fLktXlmM3*w zJdT=)o=nl<@FS@8=S#iy$FoqNS7$TxJB*#ngHZr{yUWrR??0&iBxFH!@-4|VZi6^f zox%dohsjsERd%NWRYi(uu-4W)NmcHp-)B8eu-;`!Qx{TX0t;RJW z$9`_X`M6@Oem8v0!p6`bBgdN@_0(;UTHY8EP^)=i1^#+d{dTBje>u2S&Tm+r&_-27 z^iq`1$#4W~sCw=7wJ5hcA$9|$AB7GWO&lJh1CE5JwH82_6i&!oF<%l-8)#rrZSmRL zHqP0bsTu}Mv<}0}3M}sy2kaW=`x3Bae6G0gkjGN`;+dm{_%?h@xaN7<2ntd|@1k~D z%}1%eOwK&*L`6>$nanzuwz&|k{*Md4yCOjnds)VR%N&P!eOvcMfha^FvWLNt+!X5n zIG)zO$vu(aGI^&SNk9_UaBTV+d_}fkSlvp#mFgpPifIFJ@nzZwzzaflYsO^h&kWj!0=B>HM4oK zoW?^ab;KftN=z(iF{_r@s92ukR-}mpl?I_L4Yc-?%*wO>V~3c`lf#Kl8J9d_z=KU5 zpu8}rlVe{^3cqq?s(JZ@wTTxQNJ|V^_3aH^BTNC=2r@#^w}Lgew%HTP4_SINXfTL0!7}OxLdTBjgc7 z$_5pn`+H)qZ|;|l_)}W;-Tczk#7kG+1lb=&dfLmEjq2C0j}lusa>*kT;<&tw{QTp* zjE6oNwDsPm){)K%q8Pb+D-~}g0;2RI%P0-E;uwodQyRBuQl8n=Xi076&I(C>@V7}u&G!$p{>k^ zwzJ1EF(6=C=$%k9_g*&r{T2|5`ZZbuov_I~Y!k32{EXkVq2zvTepJ8M2;~DT8#4( z8(<&~!o89L^|(gUrk%D{|A(gIi%_|sy|Su`GEjvyq9N*&ZX;x&{GP-NjXE#}axep8 zqknrlF?G}HEpzScy6aj;%Xvh3sx0=afPArWaT4V?TFBQ$@%+b{*95XJb)9xS8Z;sJ zeRv7{LW)5qX(4e_9Hj8!5~FR-d0EiIh8?=lCwsa{1GcXi8vzXL(>{a&s*-^J@M0sG zL>j+)^<5q3)qT%RN}SjCXneeNPO)Xyk+WxelQ&`9T9;%&mMPUU{!^-a?Uz$Mv)3aK zU#q;3-XeLqXs{Gs(758SKnP)R-LDu%DcoqrI9jqYyvZy?q|8t|^>69bXDD&GPWj}k z4F$ncvB0GbTA;0NF)Hy!2z!aky{{yuz#s|I7Me6qvHH7dODvHXtuypnea3|2d{4{i z{Gb}m<3t3I0K7c{yyL$`fzi!A*Rpo5B<-1U{Da_;I5D1PE9*RUhgOo~Fdf#p->%Xn zc6MPn3M#-oRYr^$fy{E$b$XtYNDQ$1fXbRy-ezZeCSU*O;7CRMZp@@f{_zeZ!L?@1 z?b_+Kzr=$~q*vMeQ;?3I^wid_d7D@ttbS!6G+K}k#l-GYnm^q!Y%jDk0Ygc8 zb)jMa8f2x@*1+orDaCR2=7)h_%F6mL+gZ3Vn*%t>&eZf=LPEB^#d+F}*p#$idCfi- zSg_8?={D8|{nq50Vd`-|vpY`SI4IJ2y{`Kz4UTB?Iy_5vQxuat-fLBO&GFx6(Z9Q9 z#-)vleYNJUPGyUFS^=B2CUDuXMwPnD}h}AIc|eCgCtGUu>?HknW`Zk7by@MjhaB^NPXj zEdszTlH0V54CTC!gX}Q5FI&Ha-zYOy7X#f58!lz0sqwr|JzYjGnfB)l_77lV*tKiZ zjTE2WzU{JV;fZ!F(;bNEn?oO+)841#i6Chpp>@CM5^OQ$F+%rJks57k7X5PVr{ZGW z`W}FoV)=_TJp6^EmI&<8IFTTB$!*tbcwHV7@%-(L^zWE3>+9>cwxLOalNle{Cw9Ya zvyS!$OFDa05DmCKdDMOuC;tj~?>8-y=q%apb1y!v}e8W0T~ zHE?fFBe%Kj&mkORW_E8+_3$w?^KylfxnG^_Cw2tSo|&ARo?>xRfnDfi&hB5o+H7vK zG~#c+Zu%jP&d%Bc-L0|?F1yMkCJUM;I&HkE`#IME1=Y+BMM{i z&>M7`&;zosW^~)OZGUIC!T6|n-P@y)k6co@5Y1h7EfVYKl6Hrks?}aV*wfQ#F<^FESeOZs~#r=@FZ4em}H16BcG# zonK4>y?lAEE3^C1CH%Hx_MADgZvD)?i8En+bKCP&PfhK~%6c%jdUBJkImTu$jq~zO zm<=wVh*+!FdTB+c&9jzvk7Swz18HdAFLZ7LAh@V2C7GyHcI=d}K^|e- zcY)`}%muPPN1B4hLuT>6Sy^TS;_eSX7Az%Bbnmh!Qm)#?+rX&I`x$MztyOI<}Y@!!AShsvO*P17_d zpqnPJBwq3$s(#ENZuhWhN}8sb=8#-{&~Zv}^c!^nSy3R}VdU)ARHKTqg?oK}YnaSK z>Ov)^DIS5i8fFQpB9glzkLEBspzK>&>Q54@~cxHpB@tD$%2`q)Ct#MtA2&j<4CO6uR;S zjWv4$10^?*59GRk^^-q{#NUa&Jh2^=7FC2A0lM2r0(|YC_sh%JLwSB1Y^c@=njVLl zaqAsIL(@8FE#tkqd^{adn!iJD0Zxz4=}V5T`8=ieo^H6P8>z*R2E9hKb7!~5x6Z)9 z!E_{d#(a_L1 z>S#ssX=c{Bjf_BlB{~-|2a715f_)O>&M- z$88@5=z8551oplnJGKRok_#g_r7vkkPJA0wBr(oXFf%=uFuZFoL|J-~N%gC_oUS|QJ#M7i{)8qfU*rstGGA?e&+f;w` zk59A6>pLTeK~*=6i+wE?R#LN4$PB^VG{q1=cW{1GzV-41N`g19djztYr=LBsmimXy-DOk#K+M=W{qO`G)uS+EXk*rR~z%# z_+RB;`pYivFzWH~(%p8+lurJK5AP7=n1R8Z!o8LqMT-u;>5Z)Vh=KF2*CDbRKXmJt zw=bW1%0Do03zVs-y`EoP9>k6H{a9F-1Ud2LRTUaAzH@^l#CevkM?iU^~E`1-1LMlU5zoa{W$>H0sc&j~Y8zy41jG{^1Ymatfh9 z+##m3=o@9F%Lr)4S+y6skY(*RQ=DDEX`~Ua@`J^Qo4ACYaI|u4ETELE^Zyw=(i3v? z)e*afvSKvVS~N@Ye`xR1%p^~wJFqX?LaRR6m7{N=YCG*X%dweMMPlC?ks(6|xTx6w z0+HWP_H5@6#I(X?v7Bm>#2gA{pElp|g_rI3$T9Qq(7?v*xc@I4b&4Iw8~In}SBfhS z>Iviso%-|{SGf0s%89R^JVe{A6WY~Qb`uuvHv|CEuPLWU0q#K){C>8;jg#*WCf_Ic&9H; ztAPZR(OW_)!GgSr7*Tcle>d2e7t#}0;Pr0L%Ffrz^BcP}qDf|<1*nij zXGrNyJ;^z;dd3%(|7ii_38zITc})mA*>pBG{{{%pm#4MwD(4jTfJ$YA3a^Uv;* z2kq;+FfpQ^EP7exrGVTzwces8Hg>~fdO{v5JvARz#&SR2X-9Ss2jf?t!jRD z?b|Wq#@$8?w4F1Truh|nBE}SCjSzU0r!MOSG`Ppi9`ax{mAtlTnEYi^H|?_^2#kzu zz3i$MOt(yLMS6`02peZ*06C5b@PpZflP!+mGb1iY6mWtv(1I&e0Fly=k5cbz}0>w0qRgK2S9eBKldM ziHN8#Jp??`ArAv;pLN*!EG&5u~X+o_a^(@dL~}8;9OJ= zY(a)vasqAoibNb$iH=(5(`%QJy@pHO&^z%240qPxb7pS}dDOV^4v)B_hZ!M98p5tV zS7%z!UaUWSwyUX=8>_@x58};AFD&t?4(kyEdhvxOgtv3aQ=&m*Mq$&fkLK3=o*_y?MRiTuZ{&Ti zPe+8voONo}=I8g+-Z5Bc=$(YekH?Fp;>ImV?EY~S=QlWhIr--f1pJAp2+80+eEL*ZL3taN zkf5VDno)?O)sk;D$R;zZw#DR7^>}rDsyt)&DT5(HRELC$%>>>RC+Tc>U_(v$`JGZ3 zLKc*b4Hc3wHI$AYMlTS0m5tPEf85WS0Q!&o!nt#5RIQW(#vrp`e#{uCn_#;efNZGC zm?Ux`o$T!FKb1m|#r8}<<&HLonnb%!y+M6*KVn#KakyY6jzapAfmXe0a)-#&AE;fK zOLOyP9SXZL_sOQ4U;^2fe($edx+MRZt_y0sw;O67=!YM~)%h7YIlCZte=(oouem`t zT`4gI8=&gTT}iEC9!Akwqo2Db8Ma^k;m6Tib6Q zz5UFUxp}Y-v3I(9wFM}jIAXC*rk@E2%BU^5y`Tn^LL+Go{AK8ejq%d9Tm_qcKQ*ud#$bS|cScv-=48G(V^BA0+9 zZ{0|%BS1qUC=bnKYfZ)67IlXLcA4o9JbVEs2x)T6K$lP(AM^lI+4tiIrmr!ycK{va2T=--$kq}79 z?bY4879sD8in0p+RjrL^{$5I+s?Z;xwV!`Z2K3^T*kH>+iyCwu6e`Yun}MM?FuK%l zRrpu8RHu_)$Y4#?t3cz#YlmjkSAR{XB4Rm(UDDuX<^frHB1F-H>}+II8#uKtv@W$! z>R1;QI12LJn8X=%yngA@q>xOM#DCda9?OP8=4QxnXOte)@Us<2lP{+lfjLI-HUHbt zvuLpjWHJ{8Z?_2{y**pf>LQK^bRE_UttacoZ?S55XG>@2!QjH;;EfIf)&!HR`e0#Y z6JnU7o07fxLV|z>%j!CJ(sE_Z3iQ~cN8!32y1Id$i^by|3|HtZuzi?Gu@!{LD7!a8 zej&T5ihDhfMHGhM@5(qn;f8Pi-)tx!G3gP4x@ptDIrjLKluR3I>;1Z+z8(rNIwH0Z z5F|np!G6q1C+elq^|s$S|jCZ2RMo?|IMiTW0VHB^7r&J zn}u9RPKH%q+59a}VrZ;k+7OCM(B^r~DWB7kuq+v=z`c%T6u^HoNEj*bkK=vWIw$(7RL*(w;>9-7sQa6YX!#_1vgs3HJ~Z36 z*W3ELmUGh8^jmAv0H+x-fB9wMzwSO?%jw|ye6Ls_ zBYfIQBw%GqJq`&ypU)EHr^k+!U8c1F;)-)Y)Kv5is!LBgCxw(o|2e+M`@tq`bdZ&q zeh2R)FVK*Ah%cnKmyC%akr>IF8UA>_P%84F^V-W(Xr1I$KoW938313hiL(mB;? z(`6nCx&Slo-l($F6gHt276L43U0h#WKc&E~->hviXATag+i79$Y4bqMFK}2ueo@hm zf)F=#G&|U!wIH{voqa$V09l#De+B*wb37;uo-RA*wB{OARiO?LL zqV=8bG^a|Ef!Ap_*1Si*oJ`hRe@e!hEZZR`>HNs2o)|6&0(~dVNbg5Zow`4Lv2571 z4m-~bU)!;+Ko6jh+6cR0N$)atDRK1(GX@gw*WI+`1%;1zREMV=?Z}w>-6;Bnl;r4_ z1;L+Q_*3)gH&{ZCdMaBO%`vaLcVl z*9JTNZi7f}+O%1!U*-wiEhe`jXA~nKHMPxTh6=GS^8A%DcQt8Q82?#rgg{1E!&a-C z)CR991CqpqnqwYD0p)_!YmbhGz?{mU6HKE*$KcOH7^;8vD!(nx&>~_a17Cm-80sGt zq|--1yx!i^tVE_M<1u9fXMX|9?ZgpxtERbJ<4kjXEpe0tVd_(P?O?UXA*1d|Xzshy3< zLKp?Hs%P_?A|gUkb`k9m9E~VD;4}&M+80kEddVXpCcc^w66HF`yk%SoNvA1JZ)|-0 z$()kEuS>|ZjmT*8bm;z~*|Vj{uyAuT;Ggc)x~AfqV9^XVMW#Z5ez*XlrID$jeduM= zQkF=zZ|m+JKjU(7_8zka8FToUn8gV4Vkrds17Z6()&dQ&y#^5BqoYMx z*rSt2v{0B$p1hX{3IWV~3ydtOgP^OnkwV zd~w@uq*y>m%||fg{m*H7AONZxD8yI3%+WikgLHZTXc^`Gj>%bG^TICl`{{M3o_&XA z9A#qi_j6o2Aig!wm^6cbKTfUGpkHax@I!dAj3aH^4qDs8rcvz+PK<6)PS$plOsbM- zG(C3zoM+kDeOULSJUp%&e#SD~{{m>q2vLJczfp4&>y5=k{N{?cib1G>q%?X^6UoJd z_3ga*6H$A72obed6HQ6kXBvMeF+%G3scW1w*;5lKGZr9tE&Ghx26C<|S;X5X`V-F) zp#$5N`DmaIoX}cTMaBnxK`*I2Jg4(+!%kEpob0!cE|;pT;_MFjR)q$^BgXJEKi^W} zV;<%op*A&+f?#IU%3yblU{8q{V^x#)POW!>A-1mP-qEN0MXE=)`jtsW(@CR3^Ln3a z@13-I%0QYF8Y`r;wM_rq^j=I<@0r`qzRJ4fYRQGi$7W_CAa8~8EpQd9YYs+ZzNr2m z{LKCEz1?Sz?`P&p5}t;l)v<>JYe${;QCHm(8^~r6Q!?7?M+dDe zf_1kiNE`V;DoKRI9(gmBlZYdT%&;tZjf1Y9|yFVnjjr znUX@RL9n9IO&##$cVWrZ8@9nwvFDi&BBqdnhWx}QMuU_ZSwZW6i-x}x!E<>=XtfXN z%PaQn=pZ__%*?7p7V_zr5OIqlpr=^GXc%+B2cL=--zO;h5g z84@RtFFWSS$-tzO0A)*0^vRfOZU)4=Z~P6V=*IWU9UOH2eKY1O3e!rjj}^mE(g|1< zUs;yo^48ztZQ+P3(-Ul)qwO;6?7Hs@}z%(@G_vCXXO_UZS^~s zyN9^cA*hvSQy1Po`t-UV9*YQp|>Nqs&1fT??h_+DS%2dg=2J2OUMqs+)3@0!WIj!c89#NcA zM6e2$=uwfCdps^~PKX6?3tQOxALEa^a^^Z$9=5r`F)AWtoEl3(n5{xQUfslC|m416eH1V(UN7Rh~(@M$%nZFpO z`&-NJ&Z(%IH0p?8k>)Yu@X4=*Rzk^F%RZFURO?Bf11HCWg8qXUNbQw2^Z_&F;H?26 zNTY58X73g>nB-D%Ud4-Umj)dXj`Ta_)H~Su!qa76I61X-5~}C`RGD_>->%g#`eik) zE-ra+istlDK#cU9kb8A`D;fI*v8%pv@tcnu{!M(krNy>y2Qtrx=6odt! zzOLHa@E}_gG;Ttu$O_J=AL;Si?Q_MMb$>m(3hEx>lHbre-bE95_%40LrT4ps=4IZf z9$1VEf^otOi!-3DVu=Hx@SK*vKEc~-V@K4%1<4gXq)3o+?-dF2@ynMd7q0byu5F3m zr@#QWAalCpY$(u?b7%)|u%nO0h^oQ~n}dUcT|OM!dJChgPST@It#Zq>&b!r7pM*-l%b(nyqJ+qSlI zwKS|Qe{(Y@#}+f2B@>H&^43}@h!ZDV-XFBPH6Ycnv)pq(CfGs8^qqIEruyfNQL9&9 zRCJJ-iYW2j;-c@aUs8}~P>=2yoa16DxGS^yEmNFZ^u3(Z-DbFD;m0?Y>`mXVc*@u zGTrp?hSPwMH3^@*4Yaj~O)UtjmT2ai^`yH?-4vRrnA5SHWJ78b+P5B^(sG|rnfoCZ z+EvPnMveMwl@}ef3rS*cVVJ)iiJYLDxw$uT7mJt&vY*4NZ0McAXUNa_E8HTIq*vkN z$3_+wciUe%tNrWPRdOTgY~YjT*^yf_{Fco)^Hj!~38sGZ^y!8@1D4%eu3;i0i@|G< z29Clt5y6~iX;IGM@e!sICZsm8osK4s|E*$pf5ZJncBcZT+1RK7#319`Fe^drcERYS zAiy$JzBI?JM7pencu`fI`iD-|N{nt?p7>NZ!m~WNy?jEN=W4!=sPTr|nSDtyWd8}T zTsNQ9osK8TAnFm-Uz)_zTFGqJz4dRz|7g`iVv_wLtyTqjb5j89NaS=uroPZy@`jx% ziaYV6v%c;=tFUaP7@4IFO*b=BzK@=mp0ssh6Mds1yQK8tfK*fjb(zi6vz8-I`>Vh1 z?!38YIICq;2x3i{B_?Jfm@UpEXgL(F^e_OX5@w&%S1f#`>D=(=t92yeIzML9^Ha(u zY%(Mlc6{5kGt<>fN8K@Z-RRw~X=t+L*=^@b2vubB^(j2IP(n#{F3F4tJffqc|FUT( zscw}*wLAgzcOI2OIsBQ-Mj=JZgMzw4M*(@m-$fxQ_$O>EsMu$Oz){j+9w=fHvL47AJ9|+Ytp}Gf+5Uzq$ zL*O2|Ltr68hW{n6`_5k~mF=apCTSF1fHWMScM~d-Tyt$+DEv8kYaj-~oL+JA+co~)^D$~SwJUzVja7i#o zo+eXU1dqh=gvMpsGT3{KU>10G9q4rs;X20KU|B~UW8{IJP^eU9iF%)qGsL;LEq@^_ z9vPj&3&B*|AK`iJH8c{F&iOV-e41p$v0CBti%Bhtkt0WX#aBsjFGE&ELwTl~z=;AS z9{_C^iLD$s)Vu=1Qsf;&dd_=g>5YQl5lOU$hL>omM~zye=zYB3=rd2nO+#eaJ$e{Z zd*chjSS~>TL3%plrA$B)NS42*?Y^NPAY*D(L7Ll@<{EQ+{8zI>cL}~MlJNceM@D2M zA(y7n*aYD@Ai{RqEirOQ%*|~tYGRrWe)rHwQ-hv-4HN+aJ_@;#2=4g`%&+K#R8o-Z zX*O@}1BCP-^gEZ#?AWgZ#>?{?yjjV!E=j6fG9UW^Ht_bAvgBaGA^SRCbqXj#{~5)*GXmc z%YISGQXUeeTHoIzppcOGCM72;qflI1PO-QZPXtPTu~rHgT@Pbab#=?+li}g@L@>E* z87RK;A1N+0%sUM|8;He{uQ}HuBfVMZlbW=m;^Pe&hrGuqP})O6I?9ts%j=sr*)xEnYkTLf7)t(~lm? z&qux~c8x`zus5Sif)#MW|UL=9kt?UAPPpFj{ zo>HvX*$sZUe95Q(vy9waS70*|xa13SFg4Lrv!n`ti`}A3YyylF38je z(Yd)(^MOnE9e_(M)~lL8lAo_gpH zRJwGjDN#=g8%E%)+2Fa@lW7FnMPJn$VjrU&u>aFP?WniP2c6vN_z8`V|s z<5+=*WPEzi64ZQXE1R$p`>T}-tpNRoD zjtknab9w3i*KgiTVb7Q8*pdgFczntx0W6Dv3n&-6Nv&M&_P$LjQ!cYkE)rBuDDAQY z&fzY*b0+W^usI5xcmY-<1DeU8pugCBZP@MSHt(~))~%5S-jUnZ z*?E%p`9vwLD&Wap6OKC?n45#n%WPO}zh0yFSsw5TMBs^!cKxcp)BpLul1 DataFrame: ) def test_is_default_pyspark_node(fn, expected): node_ = node.Node.from_fn(fn) - assert h_spark.transforms.is_default_pyspark_udf(node_) == expected + assert h_spark.require_columns.is_default_pyspark_udf(node_) == expected def fn_test_initial_schema_1(a: int, b: int) -> int: @@ -750,26 +773,6 @@ def fn_test_initial_schema_2(fn_test_initial_schema_1: int, c: int = 1) -> int: return fn_test_initial_schema_1 + c -@pytest.mark.parametrize( - "initial_schema,external_inputs,expected", - [ - (None, None, {"a", "b"}), - (["a", "b"], None, {"a", "b"}), - (None, ["c", "a"], {"b"}), - ], -) -def test_derive_initial_schema(initial_schema, external_inputs, expected): - nodes = [ - node.Node.from_fn(fn_test_initial_schema_1), - node.Node.from_fn(fn_test_initial_schema_2), - ] - - assert ( - h_spark.with_columns.derive_initial_schema(nodes, initial_schema, external_inputs) - == expected - ) - - def test_create_selector_node(spark_session): selector_node = h_spark.with_columns.create_selector_node("foo", ["a", "b"], "select") assert selector_node.name == "select" diff --git a/hamilton/experimental/h_spark.py b/hamilton/experimental/h_spark.py index caee39c16..719f33882 100644 --- a/hamilton/experimental/h_spark.py +++ b/hamilton/experimental/h_spark.py @@ -12,7 +12,6 @@ from pyspark.sql.functions import column, lit, pandas_udf, udf from hamilton import base, htypes, node -from hamilton.node import DependencyType from hamilton.execution import graph_functions from hamilton.function_modifiers import base as fm_base from hamilton.function_modifiers import subdag @@ -411,11 +410,11 @@ def _lambda_udf(df: DataFrame, node_: node.Node, actual_kwargs: Dict[str, Any]) logger.debug("RegularUDF: %s, %s", node_.name, node_.type) spark_return_type = get_spark_type(node_.type) spark_udf = udf(hamilton_udf, spark_return_type) - return df.withColumn( - # Sorting is a quick hack + out = df.withColumn( node_.name, spark_udf(*[_value for _name, _value in sorted(params_from_df.items())]), ) + return out class PySparkUDFGraphAdapter(base.SimplePythonDataFrameGraphAdapter): @@ -519,10 +518,11 @@ def build_result(self, **outputs: Dict[str, Any]) -> DataFrame: return result -def sparkify_node( +def sparkify_node_with_udf( node_: node.Node, linear_df_dependency_name: str, base_df_dependency_name: str, + base_df_dependency_param: Optional[str], dependent_columns_in_group: Set[str], dependent_columns_from_dataframe: Set[str], ) -> node.Node: @@ -542,6 +542,9 @@ def sparkify_node( :param linear_df_dependency_name: Name of the linearly passed along dataframe dependency :param base_df_dependency_name: Name of the base (parent) dataframe dependency. this is only used if dependent_columns_from_dataframe is not empty + :param base_df_dendency_param: Name of the base (parent) dataframe dependency parameter, as known + by the node. This is only used if `dataframe_subdag_param` is provided, which means that + dependent_columns_from_dataframe is empty. :param dependent_columns_in_group: Columns on which this depends in the with_columns :param dependent_columns_from_dataframe: Columns on which this depends in the base (parent) dataframe that the with_columns is operating on @@ -554,6 +557,7 @@ def new_callable( __base_df_dependency_name: str = base_df_dependency_name, __dependent_columns_in_group: Set[str] = dependent_columns_in_group, __dependent_columns_from_dataframe: Set[str] = dependent_columns_from_dataframe, + __base_df_dependency_param: str = base_df_dependency_param, __node: node.Node = node_, **kwargs, ) -> ps.DataFrame: @@ -592,6 +596,13 @@ def new_callable( if dep in dependent_columns_in_group }, } + + if base_df_dependency_param is not None and base_df_dependency_name in node_.input_types: + # In this case we want to add a dependency for visualization/lineage + new_input_types[base_df_dependency_name] = ( + DataFrame, + node.DependencyType.REQUIRED, + ) if len(dependent_columns_from_dataframe) > 0: new_input_types[base_df_dependency_name] = ( DataFrame, @@ -623,6 +634,7 @@ def derive_dataframe_parameter( f"More than one dataframe parameter found in function: {location_name}. Please " f"specify the desired one with the 'dataframe' parameter in @with_columns" ) + assert len(dataframe_parameters) == 1 return list(dataframe_parameters)[0] @@ -686,7 +698,7 @@ def prune_nodes(nodes: List[node.Node], select: Optional[List[str]] = None) -> L return output -class transforms(fm_base.NodeTransformer): +class require_columns(fm_base.NodeTransformer): """Decorator for spark that allows for the specification of columns to transform. These are columns within a specific node in a decorator, enabling the user to make use of pyspark transformations inside a with_columns group. Note that this will have no impact if it is not @@ -699,14 +711,14 @@ class transforms(fm_base.NodeTransformer): TRANSFORM_COLUMNS_TAG = "hamilton.spark.columns" def __init__(self, *columns: str, target_parameter=None): - super(transforms, self).__init__(target=None) + super(require_columns, self).__init__(target=None) self._columns = columns self._target = target_parameter def transform_node( self, node_: node.Node, config: Dict[str, Any], fn: Callable ) -> Collection[node.Node]: - """Generates nodes for the `@transforms` decorator. + """Generates nodes for the `@require_columns` decorator. This does two things, but does not fully prepare the node: 1. It adds the columns as dependencies to the node @@ -722,23 +734,26 @@ def transform_node( :return: """ param = derive_dataframe_parameter_from_node(node_, self._target) + with open("./debug.txt", "a") as f: + f.write(f"{node_.name}={param}\n") # This allows for injection of any extra parameters - def new_fn(**kwargs): + def new_callable(__input_types=node_.input_types, **kwargs): return node_.callable( - **{key: value for key, value in kwargs.items() if key in node_.input_types} + **{key: value for key, value in kwargs.items() if key in __input_types} ) - # Add the upstream columns as additional dependencies additional_input_types = { - param: (DataFrame, node.DependencyType.REQUIRED) for param in self._columns + param: (DataFrame, node.DependencyType.REQUIRED) + for param in self._columns + if param not in node_.input_types } node_out = node_.copy_with( input_types={**node_.input_types, **additional_input_types}, - callabl=new_fn, + callabl=new_callable, tags={ - transforms.TRANSFORM_TARGET_TAG: param, - transforms.TRANSFORM_COLUMNS_TAG: self._columns, + require_columns.TRANSFORM_TARGET_TAG: param, + require_columns.TRANSFORM_COLUMNS_TAG: self._columns, }, ) # if it returns a column, we just turn it into a withColumn expression @@ -782,7 +797,7 @@ def is_default_pyspark_udf(node_: node.Node) -> bool: :param node_: Node to check :return: True if it functions as a default pyspark UDF, false otherwise """ - df_columns = transforms._extract_dataframe_params(node_) + df_columns = require_columns._extract_dataframe_params(node_) return len(df_columns) == 1 @staticmethod @@ -801,9 +816,12 @@ def sparkify_node( node_: node.Node, linear_df_dependency_name: str, base_df_dependency_name: str, + base_df_param_name: Optional[str], + dependent_columns_from_upstream: Set[str], dependent_columns_from_dataframe: Set[str], ) -> node.Node: """Transforms a pyspark node into a node that can be run as part of a `with_columns` group. + This is only for non-UDF nodes that have already been transformed by `@transforms`. :param node_: Node to transform :param linear_df_dependency_name: Dependency on continaully modified dataframe (this will enable us @@ -812,29 +830,86 @@ def sparkify_node( :param dependent_columns_from_dataframe: :return: The final node with correct dependencies """ - transformation_target = node_.tags.get(transforms.TRANSFORM_TARGET_TAG) - - # This should come from the dataframe - # We have to reassign this to the linear dataframe dependency so we're transforming the - # right one Then we have to replace all the columns that are in the dataframe with the - # base dataframe dependency name - node_ = node_.reassign_input_names( - { - **{col: base_df_dependency_name for col in dependent_columns_from_dataframe}, - **{transformation_target: linear_df_dependency_name}, - } - ) + transformation_target = node_.tags.get(require_columns.TRANSFORM_TARGET_TAG) + + # Note that the following does not use the reassign_columns function as we have + # special knowledge of the function -- E.G. that it doesn't need all the parameters + # we choose to pass it. Thus we can just make sure that we pass it the right one, + # and not worry about value-clashes in reassigning names (as there are all sorts of + # edge cases around the parameter name to be transformed). + + # We have only a few dependencies we truly need + # These are the linear_df_dependency_name (the dataframe that is being modified) + # as well as any non-dataframe arguments (E.G. the ones that aren't about to be added + # Note that the node comes with logical dependencies already, so we filter them out + def new_callable(__callable=node_.callable, **kwargs) -> Any: + new_kwargs = kwargs.copy() + new_kwargs[transformation_target] = kwargs[linear_df_dependency_name] + return __callable(**new_kwargs) + + # We start off with everything except the transformation target, as we're + # going to use the linear dependency for that (see the callable above) + new_input_types = { + key: value + for key, value in node_.input_types.items() + if key != transformation_target and key not in dependent_columns_from_dataframe + } + # Thus we put that linear dependency in + new_input_types[linear_df_dependency_name] = (DataFrame, node.DependencyType.REQUIRED) + # Then we go through all "logical" dependencies -- columns we want to add to make lineage + # look nice + for item in dependent_columns_from_upstream: + new_input_types[item] = (DataFrame, node.DependencyType.REQUIRED) + + # Then we see if we're trying to transform the base dataframe + # This means we're not referring to it as a column, and only happens with the + # `dataframe_subdag_param` argument (which means the base_df_param_name is not None) + if transformation_target == base_df_param_name: + new_input_types[base_df_dependency_name] = ( + DataFrame, + node.DependencyType.REQUIRED, + ) + # Finally we create the new node and return it + node_ = node_.copy_with(callabl=new_callable, input_types=new_input_types) return node_ +def _identify_upstream_dataframe_nodes(nodes: List[node.Node]) -> List[str]: + """Gives the upstream dataframe name. This is the only ps.DataFrame parameter not + produced from within the subdag. + + :param nodes: Nodes in the subdag + :return: The name of the upstream dataframe + """ + node_names = {node_.name for node_ in nodes} + df_deps = set() + + for node_ in nodes: + # In this case its a df node that is a linear dependency, so we don't count it + # Instead we count the columns it wants, as we have not yet created them TODO -- + # consider moving this validation afterwards so we don't have to do this check + df_dependencies = node_.tags.get( + require_columns.TRANSFORM_COLUMNS_TAG, + [ + dep + for dep, (type_, _) in node_.input_types.items() + if custom_subclass_check(type_, DataFrame) + ], + ) + for dependency in df_dependencies: + if dependency not in node_names: + df_deps.add(dependency) + return list(df_deps) + + class with_columns(fm_base.NodeCreator): def __init__( self, *load_from: Union[Callable, ModuleType], initial_schema: List[str] = None, - external_inputs: List[str] = None, select: List[str] = None, - dataframe: str = None, + dataframe: Optional[str] = None, + dataframe_subdag_param: str = None, namespace: str = None, mode: str = "append", ): @@ -906,16 +981,23 @@ def final_df(df: ps.DataFrame) -> ps.DataFrame: self.subdag_functions = subdag.collect_functions(load_from) self.select = select self.initial_schema = initial_schema - self.external_inputs = external_inputs - if self.initial_schema is not None and self.external_inputs is not None: + if (dataframe_subdag_param is not None and initial_schema is not None) or ( + dataframe_subdag_param is None and initial_schema is None + ): raise ValueError( - "You cannot specify both initial_schema and external_inputs. You are allowed to " - "\n1. specify neither (which means we will assume every uknown dependency in this subdag " - "comes from the upstream dataframe)" - "\n2. specify external_inputs (meaning that we will assume" - "everything not specified comes from the dataframe), \n3. specify initial_schema (meaning that" - "we will assume everything not specified comes from external inputs)." + "You must specify only one of initial_schema and " + "dataframe_subdag_param. " + "This is because specifying dataframe_subdag_param injects into " + "the set of columns, allowing you to perform your own extraction" + " from the dataframe. We then execute all columns in the sbudag" + " in order, passing in that initial dataframe. If you want" + " to reference columns in your code, you'll have to specify " + "the set of initial columns, and allow the subdag decorator " + "to inject the dataframe through. The initial columns tell " + "us which parameters to take from that dataframe, so we can" + "feed the right data into the right columns." ) + self.dataframe_subdag_param = dataframe_subdag_param self.namespace = namespace self.upstream_dependency = dataframe self.mode = mode @@ -931,50 +1013,14 @@ def _prep_nodes(initial_nodes: List[node.Node]) -> List[node.Node]: """ out = [] for node_ in initial_nodes: - if transforms.is_default_pyspark_udf(node_): + if require_columns.is_default_pyspark_udf(node_): col = derive_dataframe_parameter_from_node(node_) # todo -- wire through config/function correctly # the col is the only dataframe paameter so it is the target node - (node_,) = transforms(col).transform_node(node_, {}, node_.callable) + (node_,) = require_columns(col).transform_node(node_, {}, node_.callable) out.append(node_) return out - @staticmethod - def derive_initial_schema( - nodes: List[node.Node], - initial_schema: Optional[List[str]], - external_inputs: Optional[List[str]], - ) -> Set[str]: - """Derives the dependency sources, which fill out `initial_schema` and `external_inputs`, - as only one of them is allowed to be specified. Note that: - - 1. If none are specified, everything missing is assumed to come from the dataframe - 2. If `initial_schema` is specified, everything missing not in that - is assumed to come from the external inputs - 3. If `external_inputs` is specified, everything missing not in that is - assumed to come from the dataframe - - :param nodes: Nodes resolved in the DAG - :param initial_schema: The initial schema of the dataframe, a list of columns - :param external_inputs: The external inputs to the DAG, a list of columns - :return: The sources from the dataframe - """ - node_names = {node_.name for node_ in nodes} - all_dependencies = set() - for node_ in nodes: - for dependency_name, (_, dep_type) in node_.input_types.items(): - # Note that we do not support optional columns in the dataframe - # This makes the API simpler/clearer -- all optional - # columns should be handled upstream - if dep_type == node.DependencyType.REQUIRED: - all_dependencies.add(dependency_name) - external_dependencies = all_dependencies - node_names - if external_inputs is not None: - initial_schema = list(external_dependencies - set(external_inputs)) - elif initial_schema is None: - initial_schema = list(external_dependencies) - return set(initial_schema) - @staticmethod def create_selector_node( upstream_name: str, columns: List[str], node_name: str = "select" @@ -999,6 +1045,40 @@ def new_callable(**kwargs) -> DataFrame: input_types={upstream_name: DataFrame}, ) + def _validate_dataframe_subdag_parameter(self, nodes: List[node.Node], fn_name: str): + all_upstream_dataframe_nodes = _identify_upstream_dataframe_nodes(nodes) + initial_schema = set(self.initial_schema) if self.initial_schema is not None else set() + candidates_for_upstream_dataframe = set(all_upstream_dataframe_nodes) - set(initial_schema) + if ( + len(candidates_for_upstream_dataframe) > 1 + or self.dataframe_subdag_param is None + and len(candidates_for_upstream_dataframe) > 0 + ): + raise ValueError( + f"We found multiple upstream dataframe parameters for function: {fn_name} decorated with " + f"@with_columns. You specified dataframe_subdag_param={self.dataframe_subdag_param} as the upstream " + f"dataframe parameter, which means that your subdag must have exactly {0 if self.dataframe_subdag_param is None else 1} " + f"upstream dataframe parameters. Instead, we found the following upstream dataframe parameters: {candidates_for_upstream_dataframe}" + ) + if self.dataframe_subdag_param is not None: + if len(candidates_for_upstream_dataframe) == 0: + raise ValueError( + f"You specified your set of UDFs to use upstream dataframe parameter: {self.dataframe_subdag_param} " + f"for function: {fn_name} decorated with `with_columns`, but we could not find " + "that parameter as a dependency of any of the nodes. Note that that dependency " + "must be a pyspark dataframe. If you wish, instead, to supply an initial set of " + "columns for the upstream dataframe and refer to those columns directly within " + "your UDFs, please use initial_schema instead of dataframe_subdag_param." + ) + (upstream_dependency,) = list(candidates_for_upstream_dataframe) + if upstream_dependency != self.dataframe_subdag_param: + raise ValueError( + f"You specified your set of UDFs to use upstream dataframe parameter: {self.dataframe_subdag_param} " + f"for function: {fn_name} decorated with `with_columns`, but we found that parameter " + f"as a dependency of a node, but it was not the same as the parameter you specified. " + f"Instead, we found: {upstream_dependency}." + ) + def generate_nodes(self, fn: Callable, config: Dict[str, Any]) -> List[node.Node]: """Generates nodes in the with_columns groups. This does the following: @@ -1014,9 +1094,9 @@ def generate_nodes(self, fn: Callable, config: Dict[str, Any]) -> List[node.Node :return: List of nodes that this function produces """ namespace = fn.__name__ if self.namespace is None else self.namespace - initial_nodes = subdag.collect_nodes(config, self.subdag_functions) transformed_nodes = with_columns._prep_nodes(initial_nodes) + self._validate_dataframe_subdag_parameter(transformed_nodes, fn.__qualname__) pruned_nodes = prune_nodes(transformed_nodes, self.select) if len(pruned_nodes) == 0: raise ValueError( @@ -1029,8 +1109,8 @@ def generate_nodes(self, fn: Callable, config: Dict[str, Any]) -> List[node.Node current_dataframe_node = inject_parameter # Columns that it is dependent on could be from the group of transforms created columns_produced_within_mapgroup = {node_.name for node_ in pruned_nodes} - columns_passed_in_from_dataframe = self.derive_initial_schema( - sorted_initial_nodes, self.initial_schema, self.external_inputs + columns_passed_in_from_dataframe = ( + set(self.initial_schema) if self.initial_schema is not None else [] ) # Or from the dataframe passed in... for node_ in sorted_initial_nodes: @@ -1044,19 +1124,22 @@ def generate_nodes(self, fn: Callable, config: Dict[str, Any]) -> List[node.Node column for column in node_.input_types if column in columns_passed_in_from_dataframe } # In the case that we are using pyspark UDFs - if transforms.is_decorated_pyspark_udf(node_): - sparkified = transforms.sparkify_node( + if require_columns.is_decorated_pyspark_udf(node_): + sparkified = require_columns.sparkify_node( node_, current_dataframe_node, inject_parameter, + self.dataframe_subdag_param, + dependent_columns_in_mapgroup, dependent_columns_in_dataframe, ) # otherwise we're using pandas/primitive UDFs else: - sparkified = sparkify_node( + sparkified = sparkify_node_with_udf( node_, current_dataframe_node, inject_parameter, + self.dataframe_subdag_param, dependent_columns_in_mapgroup, dependent_columns_in_dataframe, ) diff --git a/hamilton/node.py b/hamilton/node.py index 4b4223604..6d55b4195 100644 --- a/hamilton/node.py +++ b/hamilton/node.py @@ -307,7 +307,9 @@ def copy(self, include_refs: bool = True) -> "Node": def reassign_input_names(self, input_names: Dict[str, Any]) -> "Node": """Reassigns the input names of a node. Useful for applying - a node to a separate input if needed. + a node to a separate input if needed. Note that things can get a + little strange if you have multiple inputs with the same name, so + be careful about how you use this. :param input_names: Input name map to reassign :return: A node with the input names reassigned diff --git a/tests/resources/spark/spark_dag_external_dependencies.py b/tests/resources/spark/spark_dag_external_dependencies.py index 3de1469f4..84610ee5c 100644 --- a/tests/resources/spark/spark_dag_external_dependencies.py +++ b/tests/resources/spark/spark_dag_external_dependencies.py @@ -38,12 +38,3 @@ def df_input(spark_session: ps.SparkSession) -> ps.DataFrame: ) def processed_df_as_pandas(df_input: ps.DataFrame) -> pd.DataFrame: return df_input.select("a", "b").toPandas() - - -@h_spark.with_columns( - a, - b, - external_inputs=["to_add", "to_multiply"], -) -def processed_df_as_pandas_with_external_inputs(df_input: ps.DataFrame) -> pd.DataFrame: - return df_input.select("a", "b").toPandas() diff --git a/tests/resources/spark/spark_dag_mixed_pyspark_pandas_udfs.py b/tests/resources/spark/spark_dag_mixed_pyspark_pandas_udfs.py new file mode 100644 index 000000000..db3f5f53c --- /dev/null +++ b/tests/resources/spark/spark_dag_mixed_pyspark_pandas_udfs.py @@ -0,0 +1,95 @@ +from typing import Callable, List + +import pandas as pd +import pyspark.sql as ps + +from hamilton.experimental import h_spark +from hamilton.function_modifiers import parameterize, value +from hamilton.htypes import column as _ + +IntSeries = _[pd.Series, int] +FloatSeries = _[pd.Series, float] + + +def to_add() -> int: + return 1 + + +def spark_session() -> ps.SparkSession: + spark = ( + ps.SparkSession.builder.master("local") + .appName("spark session") + .config("spark.sql.shuffle.partitions", "1") + .getOrCreate() + ) + return spark + + +def _module(user_controls_initial_dataframe: bool) -> List[Callable]: + out = [] + if user_controls_initial_dataframe: + + @parameterize( + a_raw={"col": value("a_raw")}, + b_raw={"col": value("b_raw")}, + c_raw={"col": value("c_raw")}, + key={"col": value("key")}, + ) + def raw_col(external_dataframe: ps.DataFrame, col: str) -> ps.Column: + return external_dataframe[col] + + out.append(raw_col) + + def a(a_raw: ps.DataFrame, to_add: int) -> ps.DataFrame: + return a_raw.withColumn("a", a_raw.a_raw + to_add) + + def b(b_raw: ps.DataFrame, b_add: int = 3) -> ps.Column: + return b_raw["b_raw"] + b_add + + def c(c_raw: IntSeries) -> FloatSeries: + return c_raw * 3.5 + + @h_spark.require_columns("a", "key") + def a_times_key(a_key: ps.DataFrame, identity_multiplier: int = 1) -> ps.Column: + return a_key.a * a_key.key * identity_multiplier + + def b_times_key(b: IntSeries, key: IntSeries) -> IntSeries: + return b * key + + @h_spark.require_columns("a", "b", "c") + def a_plus_b_plus_c(a_b_c: ps.DataFrame) -> ps.Column: + return a_b_c.a + a_b_c.b + a_b_c.c + + out.extend([a, b, c, a_times_key, b_times_key, a_plus_b_plus_c]) + return out + + +def df_1(spark_session: ps.SparkSession) -> ps.DataFrame: + df = pd.DataFrame.from_records( + [ + {"key": 1, "a_raw": 1, "b_raw": 2, "c_raw": 1}, + {"key": 2, "a_raw": 4, "b_raw": 5, "c_raw": 2}, + {"key": 3, "a_raw": 7, "b_raw": 8, "c_raw": 3}, + {"key": 4, "a_raw": 10, "b_raw": 11, "c_raw": 4}, + {"key": 5, "a_raw": 13, "b_raw": 14, "c_raw": 5}, + ] + ) + return spark_session.createDataFrame(df) + + +@h_spark.with_columns( + *_module(False), + select=["a_times_key", "b_times_key", "a_plus_b_plus_c"], + initial_schema=["a_raw", "b_raw", "c_raw", "key"], +) +def processed_df_as_pandas(df_1: ps.DataFrame) -> pd.DataFrame: + return df_1.select("a_times_key", "b_times_key", "a_plus_b_plus_c").toPandas() + + +@h_spark.with_columns( + *_module(True), + select=["a_times_key", "b_times_key", "a_plus_b_plus_c"], + dataframe_subdag_param="external_dataframe", +) +def processed_df_as_pandas_dataframe_with_injected_dataframe(df_1: ps.DataFrame) -> pd.DataFrame: + return df_1.select("a_times_key", "b_times_key", "a_plus_b_plus_c").toPandas() diff --git a/tests/resources/spark/spark_dag_pyspark_udfs.py b/tests/resources/spark/spark_dag_pyspark_udfs.py index eefec6338..4424e6d26 100644 --- a/tests/resources/spark/spark_dag_pyspark_udfs.py +++ b/tests/resources/spark/spark_dag_pyspark_udfs.py @@ -4,38 +4,61 @@ import pyspark.sql as ps from hamilton.experimental import h_spark -from hamilton.htypes import column as _ - -IntSeries = _[pd.Series, int] -FloatSeries = _[pd.Series, float] +from hamilton.function_modifiers import parameterize, value def to_add() -> int: return 1 -def _module() -> List[Callable]: +def spark_session() -> ps.SparkSession: + spark = ( + ps.SparkSession.builder.master("local") + .appName("spark session") + .config("spark.sql.shuffle.partitions", "1") + .getOrCreate() + ) + return spark + + +def _module(user_controls_initial_dataframe: bool) -> List[Callable]: + out = [] + if user_controls_initial_dataframe: + + @parameterize( + a_raw={"col": value("a_raw")}, + b_raw={"col": value("b_raw")}, + c_raw={"col": value("c_raw")}, + key={"col": value("key")}, + ) + def raw_col(external_dataframe: ps.DataFrame, col: str) -> ps.Column: + return external_dataframe[col] + + out.append(raw_col) + def a(a_raw: ps.DataFrame, to_add: int) -> ps.DataFrame: return a_raw.withColumn("a", a_raw.a_raw + to_add) def b(b_raw: ps.DataFrame, b_add: int = 3) -> ps.Column: return b_raw["b_raw"] + b_add - def c(c_raw: IntSeries) -> FloatSeries: - return c_raw * 3.5 + def c(c_raw: ps.DataFrame) -> ps.Column: + return c_raw.c_raw * 3.5 - @h_spark.transforms("a", "key") + @h_spark.require_columns("a", "key") def a_times_key(a_key: ps.DataFrame, identity_multiplier: int = 1) -> ps.Column: return a_key.a * a_key.key * identity_multiplier - def b_times_key(b: IntSeries, key: IntSeries) -> IntSeries: - return b * key + @h_spark.require_columns("b", "key") + def b_times_key(b_key: ps.DataFrame) -> ps.Column: + return b_key.b * b_key.key - @h_spark.transforms("a", "b", "c") + @h_spark.require_columns("a", "b", "c") def a_plus_b_plus_c(a_b_c: ps.DataFrame) -> ps.Column: return a_b_c.a + a_b_c.b + a_b_c.c - return [a, b, c, a_times_key, b_times_key, a_plus_b_plus_c] + out.extend([a, b, c, a_times_key, b_times_key, a_plus_b_plus_c]) + return out def df_1(spark_session: ps.SparkSession) -> ps.DataFrame: @@ -52,16 +75,18 @@ def df_1(spark_session: ps.SparkSession) -> ps.DataFrame: @h_spark.with_columns( - *_module(), select=["a_times_key", "b_times_key", "a_plus_b_plus_c"], external_inputs=["to_add"] + *_module(False), + select=["a_times_key", "b_times_key", "a_plus_b_plus_c"], + initial_schema=["a_raw", "b_raw", "c_raw", "key"], ) def processed_df_as_pandas(df_1: ps.DataFrame) -> pd.DataFrame: return df_1.select("a_times_key", "b_times_key", "a_plus_b_plus_c").toPandas() @h_spark.with_columns( - *_module(), + *_module(True), select=["a_times_key", "b_times_key", "a_plus_b_plus_c"], - initial_schema=["a_raw", "b_raw", "c_raw", "key"], + dataframe_subdag_param="external_dataframe", ) -def processed_df_as_pandas_with_initial_schema(df_1: ps.DataFrame) -> pd.DataFrame: +def processed_df_as_pandas_with_injected_dataframe(df_1: ps.DataFrame) -> pd.DataFrame: return df_1.select("a_times_key", "b_times_key", "a_plus_b_plus_c").toPandas() From 63249ef88319bf1e719482f2781c8669435d316d Mon Sep 17 00:00:00 2001 From: elijahbenizzy Date: Fri, 18 Aug 2023 11:24:48 -0700 Subject: [PATCH 07/11] Usability upgrades for pyspark integration 1. Removes dataframe parameter in favor of using the function's first parameter 2. Renames the dataframe_subdag_param parameter to pass_dataframe_as 3. Renames the initial_schema parameter as columns_to_pass We also update the README to be a little easier to run. --- examples/spark/pyspark/README.md | 86 ++++++++------ examples/spark/pyspark/dataflow.py | 4 +- examples/spark/pyspark/out.png | 24 ++-- graph_adapter_tests/h_spark/test_h_spark.py | 10 +- hamilton/experimental/h_spark.py | 107 ++++++++++++------ tests/resources/spark/basic_spark_dag.py | 4 +- .../spark/spark_dag_external_dependencies.py | 2 +- .../spark_dag_mixed_pyspark_pandas_udfs.py | 4 +- .../spark/spark_dag_multiple_with_columns.py | 5 +- .../resources/spark/spark_dag_pyspark_udfs.py | 4 +- 10 files changed, 151 insertions(+), 99 deletions(-) diff --git a/examples/spark/pyspark/README.md b/examples/spark/pyspark/README.md index 5156bb3f0..7fd24a0dc 100644 --- a/examples/spark/pyspark/README.md +++ b/examples/spark/pyspark/README.md @@ -1,14 +1,38 @@ # Hamilton and Spark Hamilton now has first-class pyspark integration! While we will likely be improving it as we go along, -this version is the first we're considering "stable" and we are planning an imminent move out of "experimental". +this version is the first we're considering "stable" and we are planning an imminent move out of "experimental" into "plugins". + +# Motivation + +Spark is immensely powerful -- its one of the only tools that effectively handles large +datasets in a distributed fashion, and has a variety of plugins/a tooling ecosystem that +data teams love. + +Just like pandas pipelines, however, spark pipelines can be difficult to maintain/manage, devolving into +spaghetti code over time. Specifically, we've observed the following problems with pyspark pipelines: +1. They rarely get broken up into modular and reusable components +2. They contain a lot of "implicit" dependencies -- as you do break them up into functions, it is difficult to +specify which columns the dataframes contain/depend on, and how that changes through the workflow. +3. They are difficult to configure in a readable manner. A monolithic spark script likely has a few different shapes/parameters, and naturally becomes littered with poorly documented if/else statements +4. They're not easy to unit tests. While specific UDFs can be tested, spark transformations are difficult to test in a modular fashion. + +Vanilla Hamilton with pyspark gets you part of the way there. You can easily build functions that input/output spark dataframes, +and your code gets more modular/easier to maintain. That said, you still have to deal with the implicit dependencies, and the +fact that you're not really building a DAG of operations -- its more of a linear chain. You don't get any column-level lineage +(you'll have to look at the spark execution plan for that/an external lineage tool), and you either have lots of steps each manipulating +the same dataframe, or a few large modular ones. + +The new spark integation is meant to give you the best of both worlds -- we want to allow you to express column-level map (cardinality-preserving) operations +while simultaneously passing around dataframes for aggregations, filters, and joins. + # Design The idea is to break your code into components. These components make one of two shapes: -1. Run linearly (e.g. chained dataframe transformations -- aggregations, etc...) -2. Form a DAG of operations (E.G. a set of features that have inter-feature dependencies) +1. Run linearly (e.g. cardinality non-preserving operations: aggregations, filters, joins, etc..) +2. Form a DAG of column-level operations (for cardinality-preserving operations) For the first case, we just use the pyspark dataframe API. You define functions that, when put through Hamilton, act as a pipe. For example: @@ -58,8 +82,10 @@ import map_transforms # file defined above @with_columns( map_transforms, # Load all the functions we defined above - initial_schema=["column_1_from_dataframe", "column_2_from_dataframe", "column_3_from_dataframe"], # use these from the initial datafrmae - dataframe="all_initial_data" # use this dataframe as the source (and inject as a parameter with the same name) + columns_to_pass=[ + "column_1_from_dataframe", + "column_2_from_dataframe", + "column_3_from_dataframe"], # use these from the initial datafrmae ) def final_result(all_initial_data: ps.DataFrame, raw_data_3: ps.DataFrame) -> ps.DataFrame: """Gives the final result. This decorator will apply the transformations in the order. @@ -67,17 +93,26 @@ def final_result(all_initial_data: ps.DataFrame, raw_data_3: ps.DataFrame) -> ps return _join(all_initial_data, raw_data_3) ``` -`with_columns` serves to _linearize_ the operation, enabling you to define a DAG, and have them all operate on a single dataframe. - -You can thus represent a clean, modular, unit-testable string of transformations while also allowing for -complex sets of feature UDFs. +`with_columns` takes in the following parameters (see the docstring for more info) +1. `load_from` -- a list of functions/modules to find the functions to load the DAG from, similar to `@subdag` +2. `columns_to_pass` -- not compatible with `external_inputs`. Dependencies specified from the initial dataframe, +injected in. Not that you must use one of this or +3. `pass_dtaframe_as` -- the name of the parameter to inject the initial dataframe into the subdag. +If this is provided, this must be the only pyspark dataframe dependency in the subdag that is not also another +node (column) in the subdag. +4. `select` -- a list of columns to select from the UDF group. If not specified all will be selected. +5. `dataframe` -- the initial dataframe. If not specified, will default to the only dataframe param +in the decorated function (and error if there are multiple). +6. `namespace` -- the namespace of the nodes generated by this -- will default to the function name that is decorated. -All of these can rely on data passed from a node or parameter external to the group of functions. -You have two options when specifying the initial dataframe/how to read it. You can: +`with_columns` serves to _linearize_ the operation, enabling you to define a DAG, and have all your operations run on a single dataframe, +in topological order. You can thus represent a clean, modular, unit-testable string of transformations while also allowing for +complex sets of feature UDFs. All of these can rely on data passed from a node or parameter external to the group of functions. -1. Specify the columns in the initial dataframe, then refer to them in your functions (as in the example above) -2. Specify the initial dataframe as a parameter to the function, and then create the columns drawing from that. See the following: +You have two options when presenting the initial dataframe/how to read it. Each corresponds to a `with_columns` parameter. You can use: +1.`columns_to_pass` to constrain the columns that must exist in the initial dataframe, which you refer to in your functions. In the example above, the functions can refer to the three columns `column_1_from_dataframe`, `column_2_from_dataframe`, and `column_3_from_dataframe`, but those cannot be named defined by the subdag. +2. `pass_dataframe_as` to pass the dataframe you're transforming in as a specific parameter name to the subdag. This allows you to handle the extraction -- use this if you want to redefine columns in the dataframe/preserve the same names. ```python import pandas as pd, pyspark.sql as ps @@ -107,8 +142,7 @@ import map_transforms # file defined above @with_columns( map_transforms, # Load all the functions we defined above - dataframe_subdag_param="input_dataframe", #the upstream dataframe, referred to by downstream nodes, will have this parametter name - dataframe="all_initial_data" # use this dataframe as the source (and inject as a parameter with the same name) + pass_dataframe_as="input_dataframe", #the upstream dataframe, referred to by downstream nodes, will have this parametter name ) def final_result(all_initial_data: ps.DataFrame, raw_data_3: ps.DataFrame) -> ps.DataFrame: """Gives the final result. This decorator will apply the transformations in the order. @@ -116,21 +150,8 @@ def final_result(all_initial_data: ps.DataFrame, raw_data_3: ps.DataFrame) -> ps return _join(all_initial_data, raw_data_3) ``` -This requires functions that take in pyspark dataframes and return pyspark dataframes or columns, for those reading directly from the dataframe. -If you want to stay in pandas entirely for the `with_columns` group, you should approach (1). - - -`with_columns` takes in the following parameters (see the docstring for more info) -1. `load_from` -- a list of functions/modules to find the functions to load the DAG from, similar to `@subdag` -2. `initial_schema` -- not compatible with `external_inputs`. Dependencies specified from the initial dataframe, -injected in. Not that you must use one of this -3. `dataframe_subdag_param` -- the name of the parameter to inject the initial dataframe into the subdag. -If this is provided, this must be the only pyspark dataframe dependency in the subdag that is not also another -node (column) in the subdag. -4. `select` -- a list of columns to select from the UDF group. If not specified all will be selected. -5. `dataframe` -- the initial dataframe. If not specified, will default to the only dataframe param -in the decorated function (and error if there are multiple). -6. `namespace` -- the namespace of the nodes generated by this -- will default to the function name that is decorated. +Approach (2) requires functions that take in pyspark dataframes and return pyspark dataframes or columns for the functions reading directly from the dataframe. +If you want to stay in pandas entirely for the `with_columns` group, you should use approach (1). There are four flavors of transforms supported. @@ -147,9 +168,8 @@ def foo(bar: pd.Series, baz: pd.Series) -> htypes.column[pd.Series, int]: The rules are the same as vanilla hamilton -- the parameter name determines the upstream dependencies, and the function name determines the output column name. -Note that, due to the type-specification of pyspark, these have to return a "typed" (`Annotated[]`) series, specified by `htypes.column`. - -These are adapted to form pyspark-friendly [pandas UDFs](https://spark.apache.org/docs/3.1.2/api/python/reference/api/pyspark.sql.functions.pandas_udf.html) +Note that, due to the type-specification requirements of pyspark, these have to return a "typed" (`Annotated[]`) series, specified by `htypes.column`. These are adapted +to form pyspark-friendly [pandas UDFs](https://spark.apache.org/docs/3.1.2/api/python/reference/api/pyspark.sql.functions.pandas_udf.html) #### Python primitives -> python primitives UDFs diff --git a/examples/spark/pyspark/dataflow.py b/examples/spark/pyspark/dataflow.py index eb0c999ff..01efbde6c 100644 --- a/examples/spark/pyspark/dataflow.py +++ b/examples/spark/pyspark/dataflow.py @@ -86,7 +86,7 @@ def spend_statistics(base_df: ps.DataFrame) -> Dict[str, float]: Note that this is a blocking (collect) operation, but it doesn't have to be if you use an aggregation. In that case you'd just add the column to the dataframe and refer to it downstream, - by expanding `initial_schema` in `with_mapped_data`. + by expanding `columns_to_pass` in `with_mapped_data`. :param base_df: Base dataframe with spend and signups columns. :return: A dictionary with the mean and standard deviation of the spend column. @@ -103,7 +103,7 @@ def spend_statistics(base_df: ps.DataFrame) -> Dict[str, float]: @h_spark.with_columns( map_transforms, - initial_schema=["spend", "signups"], + columns_to_pass=["spend", "signups"], ) def with_mapped_data(base_df: ps.DataFrame) -> ps.DataFrame: """Applies all the transforms in map_transforms diff --git a/examples/spark/pyspark/out.png b/examples/spark/pyspark/out.png index 719e0854e..e86907761 100644 --- a/examples/spark/pyspark/out.png +++ b/examples/spark/pyspark/out.png @@ -1,25 +1,25 @@ // Dependency Graph digraph { - final_result [label=final_result shape=rectangle] - spend_mean [label=spend_mean] - "with_mapped_data.spend_zero_mean" [label="with_mapped_data.spend_zero_mean"] + base_df [label=base_df] spend_statistics [label=spend_statistics] - "with_mapped_data.spend_per_signup" [label="with_mapped_data.spend_per_signup"] - with_mapped_data [label=with_mapped_data] spark_session [label=spark_session] + spend_mean [label=spend_mean] spend_std_dev [label=spend_std_dev] + "with_mapped_data.spend_per_signup" [label="with_mapped_data.spend_per_signup"] + with_mapped_data [label=with_mapped_data] + "with_mapped_data.spend_zero_mean" [label="with_mapped_data.spend_zero_mean"] + final_result [label=final_result shape=rectangle] "with_mapped_data.spend_zero_mean_unit_variance" [label="with_mapped_data.spend_zero_mean_unit_variance"] - base_df [label=base_df] - with_mapped_data -> final_result + spark_session -> base_df + base_df -> spend_statistics spend_statistics -> spend_mean + spend_statistics -> spend_std_dev + base_df -> "with_mapped_data.spend_per_signup" + "with_mapped_data.spend_zero_mean_unit_variance" -> with_mapped_data spend_mean -> "with_mapped_data.spend_zero_mean" "with_mapped_data.spend_per_signup" -> "with_mapped_data.spend_zero_mean" base_df -> "with_mapped_data.spend_zero_mean" - base_df -> spend_statistics - base_df -> "with_mapped_data.spend_per_signup" - "with_mapped_data.spend_zero_mean_unit_variance" -> with_mapped_data - spend_statistics -> spend_std_dev + with_mapped_data -> final_result "with_mapped_data.spend_zero_mean" -> "with_mapped_data.spend_zero_mean_unit_variance" spend_std_dev -> "with_mapped_data.spend_zero_mean_unit_variance" - spark_session -> base_df } diff --git a/graph_adapter_tests/h_spark/test_h_spark.py b/graph_adapter_tests/h_spark/test_h_spark.py index 67935ee06..3dec1706c 100644 --- a/graph_adapter_tests/h_spark/test_h_spark.py +++ b/graph_adapter_tests/h_spark/test_h_spark.py @@ -543,7 +543,7 @@ def test_generate_nodes_invalid_select(): basic_spark_dag.b, basic_spark_dag.c, select=["d"], # not a node - initial_schema=["a_raw", "b_raw", "c_raw", "key"], + columns_to_pass=["a_raw", "b_raw", "c_raw", "key"], ) with pytest.raises(ValueError): @@ -558,7 +558,7 @@ def test_with_columns_generate_nodes_no_select(): basic_spark_dag.a, basic_spark_dag.b, basic_spark_dag.c, - initial_schema=["a_raw", "b_raw", "c_raw", "key"], + columns_to_pass=["a_raw", "b_raw", "c_raw", "key"], ) def df_as_pandas(df: DataFrame) -> pd.DataFrame: @@ -579,7 +579,7 @@ def test_with_columns_generate_nodes_select(): basic_spark_dag.a, basic_spark_dag.b, basic_spark_dag.c, - initial_schema=["a_raw", "b_raw", "c_raw", "key"], + columns_to_pass=["a_raw", "b_raw", "c_raw", "key"], select=["c"], ) @@ -596,7 +596,7 @@ def test_with_columns_generate_nodes_select_mode_select(): basic_spark_dag.a, basic_spark_dag.b, basic_spark_dag.c, - initial_schema=["a_raw", "b_raw", "c_raw", "key"], + columns_to_pass=["a_raw", "b_raw", "c_raw", "key"], select=["c"], mode="select", ) @@ -614,7 +614,7 @@ def test_with_columns_generate_nodes_specify_namespace(): basic_spark_dag.a, basic_spark_dag.b, basic_spark_dag.c, - initial_schema=["a_raw", "b_raw", "c_raw", "key"], + columns_to_pass=["a_raw", "b_raw", "c_raw", "key"], namespace="foo", ) diff --git a/hamilton/experimental/h_spark.py b/hamilton/experimental/h_spark.py index 719f33882..dc4d7e674 100644 --- a/hamilton/experimental/h_spark.py +++ b/hamilton/experimental/h_spark.py @@ -543,7 +543,7 @@ def sparkify_node_with_udf( :param base_df_dependency_name: Name of the base (parent) dataframe dependency. this is only used if dependent_columns_from_dataframe is not empty :param base_df_dendency_param: Name of the base (parent) dataframe dependency parameter, as known - by the node. This is only used if `dataframe_subdag_param` is provided, which means that + by the node. This is only used if `pass_dataframe_as` is provided, which means that dependent_columns_from_dataframe is empty. :param dependent_columns_in_group: Columns on which this depends in the with_columns :param dependent_columns_from_dataframe: Columns on which this depends in the @@ -656,6 +656,32 @@ def derive_dataframe_parameter_from_fn(fn: Callable, requested_parameter: str = return derive_dataframe_parameter(parameters_with_types, requested_parameter, fn.__qualname__) +def _derive_first_dataframe_parameter_from_fn(fn: Callable) -> str: + """Utility function to derive the first parameter from a function and assert + that it is annotated with a pyspark dataframe. + + :param fn: + :return: + """ + sig = inspect.signature(fn) + params = list(sig.parameters.items()) + if len(params) == 0: + raise ValueError( + f"Function {fn.__qualname__} has no parameters, but was " + f"decorated with with_columns. with_columns requires the first " + f"parameter to be a dataframe so we know how to wire dependencies." + ) + first_param_name, first_param_value = params[0] + if not custom_subclass_check(first_param_value.annotation, DataFrame): + raise ValueError( + f"Function {fn.__qualname__} has a first parameter {first_param_name} " + f"that is not a pyspark dataframe. Instead got: {first_param_value.annotation}." + f"with_columns requires the first " + f"parameter to be a dataframe so we know how to wire dependencies." + ) + return first_param_name + + def derive_dataframe_parameter_from_node(node_: node.Node, requested_parameter: str = None) -> str: """Derives the only/requested dataframe parameter from a node. @@ -710,10 +736,9 @@ class require_columns(fm_base.NodeTransformer): TRANSFORM_TARGET_TAG = "hamilton.spark.target" TRANSFORM_COLUMNS_TAG = "hamilton.spark.columns" - def __init__(self, *columns: str, target_parameter=None): + def __init__(self, *columns: str): super(require_columns, self).__init__(target=None) self._columns = columns - self._target = target_parameter def transform_node( self, node_: node.Node, config: Dict[str, Any], fn: Callable @@ -733,7 +758,7 @@ def transform_node( :param config: Configuration to use (unused here) :return: """ - param = derive_dataframe_parameter_from_node(node_, self._target) + param = derive_dataframe_parameter_from_node(node_) with open("./debug.txt", "a") as f: f.write(f"{node_.name}={param}\n") @@ -773,7 +798,7 @@ def validate(self, fn: Callable): :return: """ - derive_dataframe_parameter_from_fn(fn, self._target) + _derive_first_dataframe_parameter_from_fn(fn) @staticmethod def _extract_dataframe_params(node_: node.Node) -> List[str]: @@ -863,7 +888,7 @@ def new_callable(__callable=node_.callable, **kwargs) -> Any: # Then we see if we're trying to transform the base dataframe # This means we're not referring to it as a column, and only happens with the - # `dataframe_subdag_param` argument (which means the base_df_param_name is not None) + # `pass_dataframe_as` argument (which means the base_df_param_name is not None) if transformation_target == base_df_param_name: new_input_types[base_df_dependency_name] = ( DataFrame, @@ -906,10 +931,9 @@ class with_columns(fm_base.NodeCreator): def __init__( self, *load_from: Union[Callable, ModuleType], - initial_schema: List[str] = None, + columns_to_pass: List[str] = None, + pass_dataframe_as: str = None, select: List[str] = None, - dataframe: Optional[str] = None, - dataframe_subdag_param: str = None, namespace: str = None, mode: str = "append", ): @@ -937,40 +961,49 @@ def a_plus_b(a_from_df: pd.Series, b_from_df: pd.Series) -> pd.Series: # the with_columns call @with_columns( load_from=[my_module], # Load from any module - initial_schema=["a_from_df", "b_from_df"], # The initial schema of the dataframe + columns_to_pass=["a_from_df", "b_from_df"], # The columns to pass from the dataframe to + # the subdag select=["a", "b", "a_plus_b"], # The columns to select from the dataframe ) - def final_df(df: ps.DataFrame) -> ps.DataFrame: + def final_df(initial_df: ps.DataFrame) -> ps.DataFrame: # process, or just return unprocessed ... You can think of the above as a series of withColumn calls on the dataframe, where the operations are applied in topological order. This is significantly more efficient than - extracting out the columns, applying the maps, then operating, but *also* allows you to + extracting out the columns, applying the maps, then joining, but *also* allows you to express the operations individually, making it easy to unit-test and reuse. Note that the operation is "append", meaning that the columns that are selected are appended onto the dataframe. We will likely add an option to have this be either "select" or "append" mode. + If the function takes multiple dataframes, the dataframe input to process will always be + the first one. This will be passed to the subdag, transformed, and passed back to the functions. + This follows the hamilton rule of reference by parameter name. To demonstarte this, in the code + above, the dataframe that is passed to the subdag is `initial_df`. That is transformed + by the subdag, and then returned as the final dataframe. + + You can read it as: + + "final_df is a function that transforms the upstream dataframe initial_df, running the transformations + from my_module. It starts with the columns a_from_df and b_from_df, and then adds the columns + a, b, and a_plus_b to the dataframe. It then returns the dataframe, and does some processing on it." + :param load_from: The functions that will be used to generate the group of map operations. :param select: Columns to select from the transformation. If this is left blank it will keep all columns in the subdag. - :param initial_schema: The initial schema of the dataframe. This is used to determine which + :param columns_to_pass: The initial schema of the dataframe. This is used to determine which upstream inputs should be taken from the dataframe, and which shouldn't. Note that, if this is left empty (and external_inputs is as well), we will assume that all dependencies come - from the dataframe. This cannot be used in conjunction with external_inputs. - :param external_inputs: All dependencies referred to within the DAG that are not part of the - DAG itself or the upstream dataframe. Note that, if this is left empty (and initial_schema is as well), - we will assume that all dependencies come from the dataframe. This cannot be used in conjunction - with initial_schema. + from the dataframe. This cannot be used in conjunction with pass_dataframe_as. + :param pass_dataframe_as: The name of the dataframe that we're modifying, as known to the subdag. + If you pass this in, you are responsible for extracting columns out. If not provided, you have + to pass columns_to_pass in, and we will extract the columns out for you. :param namespace: The namespace of the nodes, so they don't clash with the global namespace and so this can be reused. If its left out, there will be no namespace (in which case you'll want to be careful about repeating it/reusing the nodes in other parts of the DAG.) - :param dataframe: The name of the dataframe that we're modifying. If not provided, - this will assume that there is only one pyspark.DataFrame parameter to the decorated function, - and use that if there is more than one, we will error. :param mode: The mode of the operation. This can be either "append" or "select". If it is "append", it will keep all columns in the dataframe. If it is "select", it will only keep the columns in the dataframe from the `select` parameter. Note that, @@ -980,24 +1013,24 @@ def final_df(df: ps.DataFrame) -> ps.DataFrame: """ self.subdag_functions = subdag.collect_functions(load_from) self.select = select - self.initial_schema = initial_schema - if (dataframe_subdag_param is not None and initial_schema is not None) or ( - dataframe_subdag_param is None and initial_schema is None + self.initial_schema = columns_to_pass + if (pass_dataframe_as is not None and columns_to_pass is not None) or ( + pass_dataframe_as is None and columns_to_pass is None ): raise ValueError( - "You must specify only one of initial_schema and " - "dataframe_subdag_param. " - "This is because specifying dataframe_subdag_param injects into " + "You must specify only one of columns_to_pass and " + "pass_dataframe_as. " + "This is because specifying pass_dataframe_as injects into " "the set of columns, allowing you to perform your own extraction" - " from the dataframe. We then execute all columns in the sbudag" - " in order, passing in that initial dataframe. If you want" - " to reference columns in your code, you'll have to specify " + "from the dataframe. We then execute all columns in the sbudag" + "in order, passing in that initial dataframe. If you want" + "to reference columns in your code, you'll have to specify " "the set of initial columns, and allow the subdag decorator " "to inject the dataframe through. The initial columns tell " "us which parameters to take from that dataframe, so we can" "feed the right data into the right columns." ) - self.dataframe_subdag_param = dataframe_subdag_param + self.dataframe_subdag_param = pass_dataframe_as self.namespace = namespace self.upstream_dependency = dataframe self.mode = mode @@ -1008,8 +1041,8 @@ def _prep_nodes(initial_nodes: List[node.Node]) -> List[node.Node]: This allows us to use the sparkify_node function in transforms for both the default ones and the decorated ones. - :param initial_nodes: - :return: + :param initial_nodes: Initial nodes to prepare + :return: Prepared nodes """ out = [] for node_ in initial_nodes: @@ -1056,7 +1089,7 @@ def _validate_dataframe_subdag_parameter(self, nodes: List[node.Node], fn_name: ): raise ValueError( f"We found multiple upstream dataframe parameters for function: {fn_name} decorated with " - f"@with_columns. You specified dataframe_subdag_param={self.dataframe_subdag_param} as the upstream " + f"@with_columns. You specified pass_dataframe_as={self.dataframe_subdag_param} as the upstream " f"dataframe parameter, which means that your subdag must have exactly {0 if self.dataframe_subdag_param is None else 1} " f"upstream dataframe parameters. Instead, we found the following upstream dataframe parameters: {candidates_for_upstream_dataframe}" ) @@ -1068,7 +1101,7 @@ def _validate_dataframe_subdag_parameter(self, nodes: List[node.Node], fn_name: "that parameter as a dependency of any of the nodes. Note that that dependency " "must be a pyspark dataframe. If you wish, instead, to supply an initial set of " "columns for the upstream dataframe and refer to those columns directly within " - "your UDFs, please use initial_schema instead of dataframe_subdag_param." + "your UDFs, please use columns_to_pass instead of pass_dataframe_as." ) (upstream_dependency,) = list(candidates_for_upstream_dataframe) if upstream_dependency != self.dataframe_subdag_param: @@ -1105,7 +1138,7 @@ def generate_nodes(self, fn: Callable, config: Dict[str, Any]) -> List[node.Node ) sorted_initial_nodes = graph_functions.topologically_sort_nodes(pruned_nodes) output_nodes = [] - inject_parameter = derive_dataframe_parameter_from_fn(fn, self.upstream_dependency) + inject_parameter = _derive_first_dataframe_parameter_from_fn(fn) current_dataframe_node = inject_parameter # Columns that it is dependent on could be from the group of transforms created columns_produced_within_mapgroup = {node_.name for node_ in pruned_nodes} @@ -1163,4 +1196,4 @@ def generate_nodes(self, fn: Callable, config: Dict[str, Any]) -> List[node.Node return output_nodes + [final_node] def validate(self, fn: Callable): - derive_dataframe_parameter_from_fn(fn, self.upstream_dependency) + _derive_first_dataframe_parameter_from_fn(fn) diff --git a/tests/resources/spark/basic_spark_dag.py b/tests/resources/spark/basic_spark_dag.py index 48eaecc14..78b0f66fd 100644 --- a/tests/resources/spark/basic_spark_dag.py +++ b/tests/resources/spark/basic_spark_dag.py @@ -54,7 +54,7 @@ def df_1(spark_session: ps.SparkSession) -> ps.DataFrame: b_times_key, a_plus_b_plus_c, select=["a_times_key", "b_times_key", "a_plus_b_plus_c"], - initial_schema=["a_raw", "b_raw", "c_raw", "key"], + columns_to_pass=["a_raw", "b_raw", "c_raw", "key"], ) @config.when_not(mode="select") def processed_df_as_pandas__append(df_1: ps.DataFrame) -> pd.DataFrame: @@ -69,7 +69,7 @@ def processed_df_as_pandas__append(df_1: ps.DataFrame) -> pd.DataFrame: b_times_key, a_plus_b_plus_c, select=["a_times_key", "a_plus_b_plus_c"], - initial_schema=["a_raw", "b_raw", "c_raw", "key"], + columns_to_pass=["a_raw", "b_raw", "c_raw", "key"], mode="select", ) @config.when(mode="select") diff --git a/tests/resources/spark/spark_dag_external_dependencies.py b/tests/resources/spark/spark_dag_external_dependencies.py index 84610ee5c..61b16473f 100644 --- a/tests/resources/spark/spark_dag_external_dependencies.py +++ b/tests/resources/spark/spark_dag_external_dependencies.py @@ -34,7 +34,7 @@ def df_input(spark_session: ps.SparkSession) -> ps.DataFrame: @h_spark.with_columns( a, b, - initial_schema=["initial_column"], + columns_to_pass=["initial_column"], ) def processed_df_as_pandas(df_input: ps.DataFrame) -> pd.DataFrame: return df_input.select("a", "b").toPandas() diff --git a/tests/resources/spark/spark_dag_mixed_pyspark_pandas_udfs.py b/tests/resources/spark/spark_dag_mixed_pyspark_pandas_udfs.py index db3f5f53c..e5809c9e4 100644 --- a/tests/resources/spark/spark_dag_mixed_pyspark_pandas_udfs.py +++ b/tests/resources/spark/spark_dag_mixed_pyspark_pandas_udfs.py @@ -80,7 +80,7 @@ def df_1(spark_session: ps.SparkSession) -> ps.DataFrame: @h_spark.with_columns( *_module(False), select=["a_times_key", "b_times_key", "a_plus_b_plus_c"], - initial_schema=["a_raw", "b_raw", "c_raw", "key"], + columns_to_pass=["a_raw", "b_raw", "c_raw", "key"], ) def processed_df_as_pandas(df_1: ps.DataFrame) -> pd.DataFrame: return df_1.select("a_times_key", "b_times_key", "a_plus_b_plus_c").toPandas() @@ -89,7 +89,7 @@ def processed_df_as_pandas(df_1: ps.DataFrame) -> pd.DataFrame: @h_spark.with_columns( *_module(True), select=["a_times_key", "b_times_key", "a_plus_b_plus_c"], - dataframe_subdag_param="external_dataframe", + pass_dataframe_as="external_dataframe", ) def processed_df_as_pandas_dataframe_with_injected_dataframe(df_1: ps.DataFrame) -> pd.DataFrame: return df_1.select("a_times_key", "b_times_key", "a_plus_b_plus_c").toPandas() diff --git a/tests/resources/spark/spark_dag_multiple_with_columns.py b/tests/resources/spark/spark_dag_multiple_with_columns.py index 5901152b0..8d00aefab 100644 --- a/tests/resources/spark/spark_dag_multiple_with_columns.py +++ b/tests/resources/spark/spark_dag_multiple_with_columns.py @@ -83,7 +83,7 @@ def df_1(spark_session: ps.SparkSession) -> ps.DataFrame: b_times_key, a_plus_b_plus_c, select=["a_times_key", "b_times_key", "a_plus_b_plus_c"], - initial_schema=["a_raw", "b_raw", "c_raw", "key"], + columns_to_pass=["a_raw", "b_raw", "c_raw", "key"], ) def processed_df_1(df_1: ps.DataFrame) -> ps.DataFrame: return df_1.select("key", "a_times_key", "b_times_key", "a_plus_b_plus_c") @@ -104,8 +104,7 @@ def df_2(spark_session: ps.SparkSession) -> ps.DataFrame: @h_spark.with_columns( *_df_2_modules(), select=["multiply_d_e_f_key", "d", "e", "f"], - initial_schema=["d_raw", "e_raw", "f_raw", "key"], - dataframe="df_2", + columns_to_pass=["d_raw", "e_raw", "f_raw", "key"], ) def processed_df_2_joined_df_1(df_2: ps.DataFrame, processed_df_1: ps.DataFrame) -> ps.DataFrame: return df_2.join(processed_df_1, processed_df_1["key"] == df_2["key"], "inner").drop(df_2.key) diff --git a/tests/resources/spark/spark_dag_pyspark_udfs.py b/tests/resources/spark/spark_dag_pyspark_udfs.py index 4424e6d26..cfa54c560 100644 --- a/tests/resources/spark/spark_dag_pyspark_udfs.py +++ b/tests/resources/spark/spark_dag_pyspark_udfs.py @@ -77,7 +77,7 @@ def df_1(spark_session: ps.SparkSession) -> ps.DataFrame: @h_spark.with_columns( *_module(False), select=["a_times_key", "b_times_key", "a_plus_b_plus_c"], - initial_schema=["a_raw", "b_raw", "c_raw", "key"], + columns_to_pass=["a_raw", "b_raw", "c_raw", "key"], ) def processed_df_as_pandas(df_1: ps.DataFrame) -> pd.DataFrame: return df_1.select("a_times_key", "b_times_key", "a_plus_b_plus_c").toPandas() @@ -86,7 +86,7 @@ def processed_df_as_pandas(df_1: ps.DataFrame) -> pd.DataFrame: @h_spark.with_columns( *_module(True), select=["a_times_key", "b_times_key", "a_plus_b_plus_c"], - dataframe_subdag_param="external_dataframe", + pass_dataframe_as="external_dataframe", ) def processed_df_as_pandas_with_injected_dataframe(df_1: ps.DataFrame) -> pd.DataFrame: return df_1.select("a_times_key", "b_times_key", "a_plus_b_plus_c").toPandas() From 67ba4cbca5c02b0501e08efbf4169f59278a6a7a Mon Sep 17 00:00:00 2001 From: elijahbenizzy Date: Fri, 18 Aug 2023 14:54:17 -0700 Subject: [PATCH 08/11] Adds a few TPC-h queries These are in an example. We don't have a notebook yet (as getting the data is a pain and I don't want to link our copy due to licensing), but its great to demonstrate how it works. --- examples/spark/tpc-h/README.md | 12 +++ examples/spark/tpc-h/csv_data_loaders.py | 62 +++++++++++++++ examples/spark/tpc-h/dag-query_1.pdf | Bin 0 -> 21704 bytes examples/spark/tpc-h/dag-query_12.pdf | Bin 0 -> 25015 bytes examples/spark/tpc-h/dag-query_8.pdf | Bin 0 -> 29888 bytes examples/spark/tpc-h/query_1.py | 56 ++++++++++++++ examples/spark/tpc-h/query_12.py | 56 ++++++++++++++ examples/spark/tpc-h/query_8.py | 91 +++++++++++++++++++++++ examples/spark/tpc-h/run.py | 52 +++++++++++++ hamilton/experimental/h_spark.py | 66 +++++++++++++++- 10 files changed, 393 insertions(+), 2 deletions(-) create mode 100644 examples/spark/tpc-h/README.md create mode 100644 examples/spark/tpc-h/csv_data_loaders.py create mode 100644 examples/spark/tpc-h/dag-query_1.pdf create mode 100644 examples/spark/tpc-h/dag-query_12.pdf create mode 100644 examples/spark/tpc-h/dag-query_8.pdf create mode 100644 examples/spark/tpc-h/query_1.py create mode 100644 examples/spark/tpc-h/query_12.py create mode 100644 examples/spark/tpc-h/query_8.py create mode 100644 examples/spark/tpc-h/run.py diff --git a/examples/spark/tpc-h/README.md b/examples/spark/tpc-h/README.md new file mode 100644 index 000000000..18e348f2b --- /dev/null +++ b/examples/spark/tpc-h/README.md @@ -0,0 +1,12 @@ +# TPC-H + +We've represented a few TPC-h queries using pyspark + hamilton. + +While we have not optimized these for benchmarking, they provide a good set of examples for how to express pyspark logic/break +it into hamilton functions. + +## Running + +To run, you have `run.py` -- this enables you to run a few of the queries. That said, you'll have to generate the data on your own, which is a bit tricky. + +Download dbgen here, and follow the instructions: https://www.tpc.org/tpch/. You can also reach out to us and we'll help you get set up. diff --git a/examples/spark/tpc-h/csv_data_loaders.py b/examples/spark/tpc-h/csv_data_loaders.py new file mode 100644 index 000000000..3aca0f042 --- /dev/null +++ b/examples/spark/tpc-h/csv_data_loaders.py @@ -0,0 +1,62 @@ +import os + +import pyspark.sql as ps + +from hamilton.function_modifiers import load_from, parameterize, source, value + + +@parameterize( + customer_path={"suffix": value("customer.tbl")}, + lineitem_path={"suffix": value("lineitem.tbl")}, + nation_path={"suffix": value("nation.tbl")}, + orders_path={"suffix": value("orders.tbl")}, + part_path={"suffix": value("part.tbl")}, + partsupp_path={"suffix": value("partsupp.tbl")}, + region_path={"suffix": value("region.tbl")}, + supplier_path={"suffix": value("supplier.tbl")}, +) +def paths(suffix: str, data_dir: str) -> str: + return os.path.join(data_dir, suffix) + + +@load_from.csv(path=source("customer_path"), sep=value("|"), spark=source("spark")) +def customer(df: ps.DataFrame) -> ps.DataFrame: + return df + + +# TODO -- parameterize these, but this is fine for now + + +@load_from.csv(path=source("lineitem_path"), sep=value("|"), spark=source("spark")) +def lineitem(df: ps.DataFrame) -> ps.DataFrame: + return df + + +@load_from.csv(path=source("nation_path"), sep=value("|"), spark=source("spark")) +def nation(df: ps.DataFrame) -> ps.DataFrame: + return df + + +@load_from.csv(path=source("orders_path"), sep=value("|"), spark=source("spark")) +def orders(df: ps.DataFrame) -> ps.DataFrame: + return df + + +@load_from.csv(path=source("part_path"), sep=value("|"), spark=source("spark")) +def part(df: ps.DataFrame) -> ps.DataFrame: + return df + + +@load_from.csv(path=source("partsupp_path"), sep=value("|"), spark=source("spark")) +def partsupp(df: ps.DataFrame) -> ps.DataFrame: + return df + + +@load_from.csv(path=source("region_path"), sep=value("|"), spark=source("spark")) +def region(df: ps.DataFrame) -> ps.DataFrame: + return df + + +@load_from.csv(path=source("supplier_path"), sep=value("|"), spark=source("spark")) +def supplier(df: ps.DataFrame) -> ps.DataFrame: + return df diff --git a/examples/spark/tpc-h/dag-query_1.pdf b/examples/spark/tpc-h/dag-query_1.pdf new file mode 100644 index 0000000000000000000000000000000000000000..93c6d0c55c4ebee82c8a8b3f0651ed254ebf1077 GIT binary patch literal 21704 zcmZs=19UDyvo;#rwr%H)ZQHhO+fH_D+s2NyW81cq9p3$&^PjW+d)J*c(>?u6byrod zp7m6Bkt>Rd(=#)$!;rUL*L}dS5it=t7+b^e@)9x1n%P^pS`xAUgOp*2h=>@)t!!P* zod4CfMy_U}W+o1%W-$ExFfOjnW=3`}o;lMRatU~1aJ{efXNNOJv}UWE(axIC#3W8Y zFBoF;RVi|T+d#NT0r?gleos@T$_`I?^kH|4vO)iwgY ztOA{lkDG|7(*Z`;T;BnC`aLX=JsEG7M;;Vi1XgyQ!-Ewgeo?gmvGD~o2|jd}O`*S>Ci zNt`R%;Va&ldL;*PL=|9X1D*}ki@tsWe4V{5>5hJVeYyI&Ti4_IpJW2J9;u8wS5!3z zgb&3BUpL7^Bh~n&F}tR-L-C&5&Eq!ftP_`X&OM%#3Y-rRxaSAg)-T7;kLQ<{$OPuq zV+m-drWTl|5uB$4aa5h5~d(Q0CN>#@bK6ng!DAYI+Esq_t6ZEFBs?LPH)Wo@yZVch5$wW;_ zJV9E+)LU*$+&(FEb-W}_%K9!}lDF30%%rdL{d0X-enP&mTGeAT!&y&~IeU{i ziH}q&D6b{dgf-M$m)t4PnWvRiY+O_)ReLrs8tiDH<})@otFxY)co|ufqur0hM=6Eq z!uPHR0kp`zyYu(upH!ZG3xUS0&(jL|Nq*MnT7Hdn@f#bt)x+PvQIjAbaZGd zzZxY_?lcjnF)ERAv$?z~59B869!Ef`#>~)ky)gR5DN>)|!$5qzF09>z;uu7P)wKQ^ ztHC>A!8bmPkPNE~-^i$KdcTAF&KLGu!8mivRm~v0YlpNNU0mn|{4Ow3NAt~zQd;BK z*)%bS=u3mW*^-d@O%c{mW&pwiXE^VaXzL|ThF4tE^$gFog$yO5q4S>~j0|&Cl1+_3 zPCGCCrPIo9+1JV%S7A0j)VI1U9~AjhBMBg84Lk~}{HOFPNt@UaSNW1ESI8QF1XTGF zK#;i6W+7u8LGUmI3CBRUx7t}Fy>!B|BnF88Yt%7wsT)2sRf8gyN)Iw&^=yQD-5tfk^CTF-Syitx=p$l$ACD0b4d1sAcd|-4LLVuw&p+g;1JO!bwhp zD)U-`5AtuPhANcg<*eO~?0D?}JERq!W9xMQFp#tUTHCuv3FJ?~1U3+G0Id9MbagXX zXtt|)@;gb|j@gm-obW98F~qnu6)UA~O$fPX1x<1GKJ-e?k-a)cRQPcSmM|<3-SQAA zyJsYd0UlfNa!d+KbTnltO%db(d5(fs?9e$jR2m4Axd7z121(Ep@cMv~PXAswXmI({ zHMb+ToXDIQbVkZK(&>|7CUBGg#>HJ^53gL=zJG&$lrw@dE`Zc7teBD`^{7F2oGJcz zfF!r{`Axzb@SV|&VBrJv(ad-D7vd0j{+o9cl;(|pG~^UCww+HW7O|JF?-r4RT>RqJC%g6^v$afZNikm)+LRW(N>r=PEG-6la6NfYL_JABHj^~AR_ye zlLz=h8$n=%oiOR0mK@twRbtFQ-o?*IdvevMmbxFAY0)}Pb|cWenq86(Za1HnL(->g z4i>rwWJoG{iHZZ7kf9J01AH=RXZh*ti4!Ob&zyvyZ?`VmjBgH#Hi}!Xhy`SXk=kV zL{7x~U-l>(Ih*|(p80?JGRm2mS{VsDcoOOSgP4fen7N2JxLAnv{!3cqpDF*uU5Ng} z%l}Qn63z~8j{lc=|E=S{(EssI<)3C}BYPLe|L8LD`X5+|$ko}+?0;G!|D;6C+^tN^ zlqH1!1O6ZHmCaln+?-9!T!{WNlgj^>XMlh0|4(ed^}lBQ|HTIXnX3N|BAAI-Sh?6) z{$J6;agUdw&PXSKoAva}GCiKm#$YCiTZ4g(#gWibF6KsQoK9o9x|B}4s$A-S0<9J9H_ac2YDxWh@xMfQ8s zW4AZpw+rv%bYHGRpPjfk7IBHlVWh~zsLe{k=h#C146&;a7IBmN;fu|2XI8sBaVoxq ziXZd7iV>VmOC)v+EqRO^LM%W-RExR&|guBE1q0r znmSD*Ax$tBL*ZmvuPmB#ugdi+qwtp4MpvY)9Tz+|h^r(h$}q z!0vV>dz_IGj}-B9p(I&@{CZb~60a8jgMjo2_ZI;rF^g9qO!brP?+h`*`4XEW7HPY< zA|K*kFegBG%Nva^wDZUaqRwm!1ExYd1J7b~6B331M132DCA2olo)KXJ|F6!>1a$Ow zsEJRwSg>MPL%yl1?nL~6eIVj%TF7`Ji+zz12exE2GYH`7#&8H`2aXU%EVw<~GS~n< z)8-^Kq_;!jEU=(mf}&%hyA5I`)1Y>QMQ(^?mo0}Rh#Z{tj$ImWoy?-rxJ_c>G(T2- zPTHB71JxY*na9hN_y_ca91O1327RaU*mJwkQNS!w_z_ z$Wda)cO%62Mswv-go4$p>@jN_N8TGh>p`wV-s7X3#jwnf^*RWcv|m%f z5LY=Y_SRP(#v|*uQ(O``s55?wiguCUOWVf)45XsMri#XC464gO;!1G*L8IcrCAhF? z%H9H_!N3oAfCUz_EY=IQJho~$_1F^U*%%HEe$1H8dii9VKhp>EHxWSuGyJ|1UPRJ< zj#-q){GRG#0F!S> zs468)GZ)#TR9INhE{>^Vp+C<0d1BDc`4!x66JbT2&I?btwT`)kvW0vUe5?bs>kNW{ zrY#*v?j6lRlnTr7Mpi#^C^8m4%o!ow;aGDPBViCbHu9$Q-g&}2NW~fp#ggCVQc+%x zW@$qUf##u-(jbk@gz^F%>JGEXpjp>8f9_ZDW+_+8fycsv#)1;fC?2lrYAz-k+#dm# zr280oxo>A_9}{XI!L-7b=P-S*&vPZ8WAPj%OsZA)dc~O ztrN|pr4m{Q6(Zfbxwsb%Zo7Rvs@cDR4ET_X$S5zB4RRECNS+iTA=ntg4@6Z`qQYP&n^}7N_p|>}?EUaavO+V{ zV%(an#tsp!-a5r;H5HFEM_8IQD*WvzJ9MWtM3okUg|I!bWL}uv*CnT_nD~Zc77>cK7uM;;7I%^*;ql^$;<7ySE)=D=8p{QfKi?_nmvce>~3Wb)>jv$kPeD1{)= z1^ib_Nrl}|u1g?SCZU9Kp^7X49fKf3lb`-*acq$+f0RoyL(0WJjueAQ!Nq(;nIHyyaQiq* zvwsQyVGN|uYNkXr!n#<^jy-4tfe>bSxfwe`gt&RV#D#nC0vdkCra_CfD08W&XrF!Y zh-(D5`Zc1uL^FU66F2JqurW%Lf3z?Ii;rTPtOFk__o1gM(#88X!S6!etX-c^@{1-26LLef%=)d&u+cE?NY9l&jog} z%X&I_43qt^36SYmXyJAl=dxO#wAIjBgnq*RS(z;^z+nFuRQ>%Okgki%3xh#0k}WNkwmqU<_PhXEU! zxP#kRk)*AP%06AzYPH8i4><_yh>R25kC%DV$$@_KTBhGIv_XyfNLWS zrwpRopt9giSDgafFpmwY8Qit38Qjo$jx0qA2JQauwAU2B`zKbc+u7NiVo!V=x;Tt5 z*qrxegUxuZNN3Y3AWi4REN2fdnvrx~>3pLbmra?+kS~s)rpDHAg0`v(wocxd%jF`+9L&63 zpH1?|H+V!^8Pvt{yyXB;T zqy7W)}1&cWLL+Tk8v`Jc&A_6~8@ zLSptTzN~ikoC+m2CR90n7hzIM0@ih}mXrhZbj2EWijgIUgG4E|35uIk=-ViS`;GqFw6TwsRomTU|&gPb_UROp1w{gl>(85_o9A|a}CC<=%J3}sK2 zQ0$zfhUV9x5n!Rv*!ZSB+5DC@U&ctuOLN)$smxRpvB){$O>2{}T~HOi%*XVjJtIBJ zvKq1H(aiKQ)D(833|*}D>&eW)BK=Z_RQ@xp0DvWG4uUj94hWNavV8+cRD(n?dTb%g z@4;-2?Y@_`64Rgkbi((=0DiBZ7z_KcgeuY3(NXmFWN?ALmyNj9wChj!QHM|H>VPNB zKY|@vo#1blTKfW0v_E=BBNo!#JO>&z{bQdnX2#y2CUy|r@YS|-mL|G09C~j&CiWbY zwC5)g{nTMjdfECZOgxisR#&;gM5<;9xm^Bq>^?2S%{w{B2BEdW*h`+*~Zw;PG6KE2zP+4;Z!)kX`r`69EO;<*v>8rT@-% zkIBYYoetmm7!O~^?3@K22OO?|BXC@McM-#kkw+&VUYUNT2jR&Td+EG0duZTbYFUVT z4@mYYqx+1%Rp#rMq(I!R21*k!Zk5V+PyQS7C~rMX`t#;p2Rv!E<)2M++c7&z+hF=r zA72Y8#bQa>-NHqY0#;ytP!Vny6i<|4qD!(i04r1udScGO>)9F-PBGq?vIAmJi8ymj zp0TEw*anI;C&;GN;C6klx&gMXWmNbmPZeZU-&_m(dF=ge?uXZErUN2FteX9rra(`U zE`h6VW66Q?@DVGlurU0%g-u8E!#YvQEChNqy*zA6K3%o9%}H1-P8F5%XCZZTrm>b> zPFnOThud`eMHZu&N-7ck@^V{8T$+tZvKDU2#D@1DONZXaXYbRnhMm6<-j!@8WoABZ zEtxxjY66SpLpikEXSPDAl4YXrs-p7eEZ>%W3L16k7E4s4zoBjOrbc#}nz$I52Q)S! zb%ka`-@=XXozZQ;_XgD7+Nu<#p9w;*VeY08JZh_*p)ipT2Wi!4+7zuYueBRZ)?+mG z%hpCtwY14|)jFs&6! zE6>(x-p5tT)*9Ewu$#&lndO451VUDD7WH+u^%%>eOSAPsk3^xFMKG| zEWiMX?(5;i(A?6qj{?sjk|cl`F#%3okBbntYZSx;OARYjn>Vwg1+C_0Yb-O9rckdG zx)?uLwmFN4B7@`Z)!tLP7}0BGXDGLyJAaD&<=7xTO6sgn@xUfeds3$9#=E7v7%GwV zAXgGq#`UY#3ikCy`pY&)(xynUZ+ zDMIN{sX%k$hxUb?qF)>tQI~aTaNl?P>PR zFL`AH=%hPt9auC}?Qr_)$}QCX#JOy0{6V0`bB`vL5!hi@t}RJFJ9OboR~0b^*mgoK zSfd9NSO~^6^f$|Wd9wJ3B2Qg$;z_W|_yr#6qD`WnoGdS{aaX|@(}LMm{@RCr?y>xs%1}-x`w9+DTr(Ri?wiIU}Dc5#~{qOk2 zFDZ&Ps~)ZUN#_7ZGRe^pEbtLSegG;jfhxEQJ>%0QdfubS@0rkTw1%#>|8rEEPo za9!lB0(QM-=c(qM_bJ- ziy@Q|Y*;bVC}!)Zs5F$rz#1Y9uJoZVIHGn~H6ZTn_adDlo+8~ATGjo18agFi)6YYfLQ7TqYA8{jM0o=< z>J+i5?$Jssc`#&$L{K|4z;Ht#Ixy_AGJ2ixuLd$H7J_t2(_m9%z-X+muKGmh+R)Ev zYIZYQz}1oEJQd|k{4|1AbPHd|4D(D}9Eo)6b*eTdybC#vZqdf0sLu|^YWMSd89Ds^ z98ULgUQHu5CVftfM_RWVjmbEvnA_US_0NRD2I~CiOw$^jmDnuaG%BkSWx=(24O15Y*?L zR5bawHbQ9zvmupFizOJpdnyi44RMvYU9B`n0a7W6{2>nd{kr-CttxG3+6C}!phB-! zs~&s&Xh=lQZbTSwkbnIG$Nq?Ct4*2%Al2pAcT>-#Pj98$7WEhO>d;%ZX}}rLH^X<( zJLJtP8>L@Bs)9+nU{|e8O?T<`gXfv5q+QZ|bAi&Iy^_Hw>LbYI7MV)q5CA3~B8s%= zkzqjMj15Ugy{Mu*m;!j*z@JZt?8d*4@saBAi$3_OW-zJdHV_n*h~Jtlfh3J%f^gJH zTCC2NU~&b#SwfVFzbKHN*3@Tf2^MY#QV6axM$!R+_<} zg#jx1A`;Ig1I^tpq;$l|qp@YtwU|M54Nn3yjI3$Pa7L&Ytg0*tE(2#Vqtbd~=j$ly zcplzs4ZJW9McM&Ysu~>9LAG)G zUZxW^s_P}ly@$W#+%T=le55;$9n87IE1J)+kQX{LoK}yi@%cafzVG%$mDJprs}8FS z2YKZxpg2m3rSOiZ*I}ACf~Cq{DX6^)Dkr>9e&tYFBY7GyFJ4Hk=%f7U>6C*k`J>%( zTqNzNv;<^2m19n+b1sh7`i9Dzwo{3Pn{Rfq{YZ^XZt6x@ys z1sJDPPP>SEw7^Z4YLywM+@+BWQ^pxP8WMe44lte^LdW&5nDA2Kl}fm?gaG3=XYpcL z-+B^YKlM&iZB|ZOVOPQ4#ADWoPjgDVc zLbWH03f{M-UeBK*mq`BfbS9B{kcv$tRiW!d7kD&3p(xx0+F%Alt`1?#kF**NN+<<2G4UWH%>aqK!lE z(fTI1o*`4 z7OXdXhAZPr2K7O5agYL`d%=t5=A|1Q(dM_tCG2SZ`q6{|3hKPCAg~XLWitc%jg(Y+ z@7+3YW4-yv+NMHrIfy?}XY$Yr)%Y-tx zXvw7eanVo>l5!GL-e6J^dNi-_7E<;m(5PQ5?leabJ(aSx+RSFAZoj>Cr=zTWrQ<$? zWE~9F{MDPVVi>de_;4|26_;6dyeahAXj=!vgZP}^rzVJ<+|h;&);J0>F%dab#3tEB zumgSI7F?=G*aK^u0lod0%IjHKxY8`4((y1_xmYAp747>(a(V`{585R-a zxh~Yspf)^$-cv;5wVM35L7uGH$z@S=6rR6iWzfoZ3Vu|weLZ94eoK{0L%RUF0I~+Q zW<6aut<=Ie!A4#hNs(WhlsgY632>Ro_g7ZROA0D#*)c6YSAt!lW)W(7d)yy|4Hv+V z{5tz$r)^4tNv?`4c}U@;GuUhXcdRgP=z!l@(*_Jc=JW&X69B9o z4bjO+o!`j){#&)NJqa|rBi^=tqc6CCrck@XvVF=T`+ddEYD1m5$sX~ygj@Ava~XXCCKCGB!=(Z%hI zXA;O&tg9CikM3oXEIIKI{qp4vTQhC=f^v3Y>L~P*ZYaJdDppf!Fy-P|dGQyCyadN2 ztIxgtAOMvK5lUYvmX+3}sbj$1Um{GpWh102u)>;yth3Q=m=br|#HHK<)#~owaOrX; zamJfP13YX<4T7v$Ee-AX?Fuff(F5DHGM!GpeYWJbW|Q)Xfvq~tD$tu;gE#z$`Lt#zN^sy*AZQ)X0M4r%YDxj9Ma#Q#xtqUI=w zL!(}Ia#6+%l&zbYXw`6<>jFY~xsd|-O*LL+P9?AYghrO-#H82XqT~x{b!k`H_K8dD z73#~EfvoD4`6~Y2LZe)(SwFpwnCsU%?e(DPc;qo!gc)GCo(dB#xTm{XOBD~r7t77; zk@XFnx_r_vJn@`yWN)s$g3aB2qL#_&-JOD<@-w_n+LbyV90SOQo|}Go4c>3!ZIFiN zAee@JTi>e^EW)ouu>+H*P;!2nJYM*DX6sl6f}R|&2!bI~Aq`}Ov50V>xd%n8mzzS3 zUgUo(yyrygAmRV2z9ItH7H zf1vHq{m_?_s-l;Z()h1*knRfeo>;Ky2eq*E3MStb1DrDv_0SBc4-qud5$mJt?)R5% zKI!$TZuA1ZZMx027IW=YCA;AsPK5iGBht$mo(j<$Va=A#ZC{DSk+GQHhqE=VVieHF zA=;Tp*oWb6V(F;6tdSkr=(VZUxZV0ECI1lBQO(B|-6=w-lAzH{%)}Z3)_qn0f>8)P zH5d#LAXkx*Szry0_HYO1X*LPh-R!TzTf@*aGk8C$kMj}S7?dT3aSlh6m~eKT5441S zuWbaq+cFNL0*|mU+t;a3mOTwmo07y2R6In~6|tyh~8aZSznm_G5{_6;MO zn!6$KmE4boGPRUMrQM`3D|Nb7$08iMiOtG~XmJk*jL<<)sIVv{$6I7nOwmCs!HCy_ z!+QMjx%Q#$K&TsP81}>U$QE)FNKl!;AAmLz=BvzH;-})boSC|!y&zQ`ZOAlv{q8fEzc!m1Az zHC1ENm~Gj0hP$WD@ya4fAzeBlAat~&nX>5z9qB`&hcIn3n2fxKxovTnr!(TS!BTkc z*(e<^iqyGOdC~MZWatafzI1?lSY4Ye6D;lcSXZ+6Kp%vxzad1)v`;D`*Jw%Nr3s^d zh521k;IcA=By)|%{!ulrW1N_ZdqS%snCmu0Goe(OYNK1bes=$CU~F<6$>cOii`i09 z0gaHT$Fj770$g^JR~BjC_s%}&d3pBmA|q(H+Y;HNx7>Wrw8fjc5aIWXnpbasZWNKL zGTcz*9H|vZNK;99OKWN-8Eayp>Q3<%QfJX*tb!NgykMLWQ#Oao7R4H z4zn+*9ipdC|qY$5@*DUeUZlKjh)ik9QN{Uv?lL08MmF1Yg>q@{uZt&38zw%tJ znkU87E5c0q#^$T~kOkB~xdiTIye-9V4LvUm#p$t#d1%wtza%oz5Y3dAnbpJP>KS{Z~lCes(mP5WG+<#lFY~;&+WkYBn@+G zo%4O8d^2yX|3Y}7$W10xCR7;IENEETDv4yX8(ubi6vuNI^zRZ2vGe6OG^?ec0Fn{` zy`fJ+A+ljQ)qJN|9#7GmvvmwKFXon4yu zb*a@xWry^#(G9ED*soU8M3*v(tzNoO0KXdrLJ&1YobeLp9X44mzEVoW2oF~V0)|s+ zDcJ-=8!)LSKsAWV4=68Su#C5Uqmk&>t|d`xy)b6Aw@H98D1C2%fS# z9IhI34GmD^b(P4Nes&{>i`tHg(WZ_Rzr2+8IkH41DCP7EuRu*8z@Lnk$ZP`*DFWj! zlj}eowysrw$^<(@u4@t++K7f6_8Fqd#Yv6Utn)AN@6l&kRWQ-&+y*SYksc<n)$z4L+YWzWWv{?vIW8z$(9xY{4RKFo8v?QpPgR%o4%ofaLlf)7?m z_pJOpP8=QMnylfE*u{UOc(H!HAV|E0-bP)~IUJo7^(dm@Yk@ zP=QtfeFIm?t1Xj=VlKNhZ(bK=*<_N6VgCADi*8JVaSteT=FuoSlb8Zn5HKrbaP7^+ zMe|%ZX~+;|Q~q0*nD|lg5?ox`sO0nIjjBGm3KcNMLe|DIi|f zRe{8kNpECFpv0w9XC(j~?sHf34WT2L2w|h*zA4M4UHhu|YUs7O)$H5 zEb{>p6#S}Rt)mcVgr=(f8rUhO$f}$ZC{ISjW%tBdV1>$hl@I$Q_=e5|v9k02CEtpl za}P@7h*f7;IHl2*2YJIOrpeh$yAC-G3Z4`_sC~W11~76I@FF2khH2{>kZ681sW7+l zyh#2;g*Z4RgGHsv?iMX+H*o5%*dQ}HJ;Fyu?HO*F){@dp<T$CHj!GI3I|RlMOf=ctxsU2bM8U)|y~R|u>Y>Gt(X6eRNl}I)H%OaNbAcmnYJPYvommZU+M4 z7FqDA{-OECD6mW}@h@>bnUSA2A{RL?$h}9S5gmX@zMSDzX>J-cTat>(!dcMjE*i93 znz+xBBvYo{?bqw&I&16Fd^_~f>hKJRK$Ca2=a}OA$H9A0_(@g6By*^Bnj5%=mZNGEYVoZc8aJR zE}17zjH-BogCV4fzfHIQ!Gu3l73Y=!Xcsf+V;`|`_tto13i5`l$H-kYhEH5~9n=eD z@G|;j*d7@@Buhvv@~X#`37dP2nHEy9sA*O*=U_C>?7o!)o1?dD7o(rqs6Trp zHU}*yK_ZBr{UIx*+Mn7b13j0>Mfi1-E<~4g!V&CBDkkR=AaDP>(PiCoaJ2=j9p3zK z^~rR_`8DtF2w94aK9&KS=P6)kQ~h)#i4`&SmGcGV`GEojxRVc&WdEP1AlL8 z5m59NmyJug^|2+Yn7C1(`D{LdP^2&v>X)CpmOp}gBrGr#`GsM9<~mO?z+evq%>OC@ zdkP~gMDraeQ?h}fG=3a|qU5%Ty;)>ziM&k8Z8VDqygnZLIjp$B-p3Ns?|(V`1PscW zgeyveg3Zg@+At@JqiD;PWHI7tAzF+{TH)DV?K!0-`d5L0$Ht1E{sm_Lt4O+>P85&K_&2N}0#2riYSHgO=Y{#L=PY)+Z7s$JB&TI?wC;1hl;T@?ZY^ zV+nVtzIm8$ujdjIG~rU9%sIW~GPToBL=-zI^}6#Zx4#{)fkh0@fujt(=Qn~*EBx^9 z{~cw*^$sud-_bodx2CWR`0Th8-HAcUpn}NS4g1>+s|u0y!RU!rG-UiSJsKmx z=4nB4t|T^Pjb?dq-dlf`$L>_yO9QNY4qg4}&z4~BMXb?mOR^h;@4}Vk)U0>0b+LQ2 z3iJaFffTQOj`|ggnUjm-_9Rw}@WlmZ;zph#XHuMEU2nFM>=a&b$Z62}m4;ux)$q5B zuwK%j)$)P$(xK0ox&3Eg^EKf8@IJyl;PNEEw~GE_FOG4#xAoSi{B|nsM&kiuPAj|P zR3-!CmFanD+BcIpE9`H^-$Y?X9YBSFki+7FD30R6NXYD{WoxL6L|&{3`1*F7Av@1v zjk_X^K3~9#7N0|QP2NwZ47-^ni401k*a5Apyh2H`o2XpI6Nv=4G-y&gM-%GYD23`i z!!_hwME(*^?A5BtmILA$m^UvUE+c+yUexT=l&`Z*rR&REk>`_`cUWT^#Iy{S{&6H1 zsad2N67%PF@mG<(uz-g8IyGW+4j0nh=Y}?K?(>+way5mu U5e)lJ20|n=f1IUK|88`PQ2<{y({-PzZ8PWD^Kt8bz#J1H zg5KNWPS}3_$hba?0GHJ&QRFFo0b)4i7zk7{6b*GF2*wj!KcRkz^RRm&=4jP9OqaBM zTmkG4EciRFc?+UJs7qpi3iU3;-^}K}KeveC{QhybV+I3=gWdH{eNeYDZ>2wN6kpS` zzx^k_H?>!O64Yn?mLluGzm7YwGvcAKg+PdrJo-DpHL-aX1Z82_w8LzFV%&cMUV(mu zRS`i?LG^}W6!$szbmb5FKE!Rx|i>u#ilSQb4Z$wh4V&(U>pFpyNw$F@~ z!r$k6Et$Q+uOk?p`M!~TGm$O?6N;dI?OnPOQ>qvCp252^@a*=NHkfv?F;(5+7Vql5TV#$j{u5ZWVKp< zl|WoWq}IWZGLO1P?+x@FAz{9BZoy&>pXq>_O{A*ReVKnv0sRc51+O+@OQD#v0e;n4 znpj&?bq%&Mq2PBiz$-~*2ykFYjU%2fVGy$H37;!UXvimjSbs3K|3&@XeBp+rgSZiH zFL=oG5K(x8dInWD(mE`a)1WPnCQ?o0faneNE)W_2@8B}h;?AQpLY3WRU07>C{|)qN zpN0*HG)HtdjQ?MDj0%*`@p!A%H}JQ*n#T!7#y-Upx+~rn+UPw-%?IS!Z3oQGZesRC z{9x8#aoFyVCR;%aXEE%&b_=+jM82>=MEyGmcl0-EfM|>21N!$z6|QPIjK|Q1dVv_z z3qMsa$i@!5ffbD0&{U-7i6 zIS_iTz)8tKa+|s1J8=8(d_~(1xsp0h&*69z!^0U-F&k07%k&q6h#gr86r_c?; zjs6Y!4do}+58#J(S$Wxe`S9Y|<%i>SQCMNW8HyPP1xQx90AdO3%>~h|z z?hK;^b{qW7nR6q8Ai_%^L0|m0phMB?J}ePlD6;`@uK}p02W}tiPVXD^ryd*$-#7es zVqXc~j`V_LQ%SCC()1CB=Pmizgw2uO%$VJgyW`;<|2W@7?*M7uDQp$DXL^NuH371J zNu4UAlEd8R+u=H^ngzsc4|~K3ugm4hP4>sl79mVE?7ze$DOFUJQAzs{)hh@BKiI8F zV3a};=6u3DBedn(@9yVc%_*`d)Ha2Kj@N8OiZFzs;SaBQ$2 zF!_5Xpd?u}x9X1?VwFBpk8ntXbcga=Q-gu0#+xR6#m6%45@(fx8YUZZfx1AT7a@Pb zQE8~gow^!!0KOOQOIQr9(ssPYKs2-|xd*!k<}2&6*&7<=p~1f17W$T2n1iF<(d`yS zPZn>&A13Ro8^j6$XPiL~SPZ2g=S{K6XjT|eCy z+?uyxM+mwjy^{iw{E;fSFQe2%39VSI`qj_`R7= z)`)L-_x@fo7`9r}iiue#e9Jon5bpr*8!hQZwfbsz&^*65nSvNacpMS|VfqG6f5cm$ zcf2YQ1yF>YAnWn;`M5Br9_X`gmV$gRxkFkdn9Cq-ZX;WsKm&jz*PO7o(aEjgPXLta zZ^$2PKbSua&E|^a3NANCJV7kSf!{ab9;i6MYZ`OggGS4J+nlP@3O1uF!Wm zyFxz%OE8BzF@2+yhD$7pCrTCfX`SSfL9Z@n5*Y>AhEuRc`LB;Mfe$+x9Vq6W;CB`J zdr^$Vv@iz3*LT+pG+WwhapABAw7Rhz#pq!YLErX_vW4QV-PVaU>LY(43K|&?i~AG4 zL2n_tdcnS6&XmBq=Lq+|IAQ||FR)u;e#qx$Z1-md83olVMUNYVaRX!WL{|$0hPvqY z%@x14oSh5iM&f`Xym9q3Jxhp;@Vj~(*%fyz=0&X0=klD;6#2k9y0d@BF7LhfNTwKZacahV+TEKnX%h%fYEW^Nq#N;gs~5Wu5@fK?vga~oc-{HiQww)88R?nN|Lb28P0K}WT~`+XCbD@b4kJ2a=KiD`xk~A zw?&6PF=O5Bv?Y?`=pbCKQ0|hEVBF;r(u(jG;Au0-W5;pfSqcrOy$x~8-@Yz=bNReW zxNKgp+dlpAg8>h~$tn3YAY@6ZK%&>Fr#E+>+Yrh}YkC&5#Q{hhYZr=O$%V{SadJ-& z=eCl*k>;w0r-#w&yPU-H#hR_-?K*qkOC2l5qs|e`llNMk3{qsue-4LO^>{-_E!4x* z3z(}Nzj<^?jsh(z?Wlp2JyGsA$# zSGipA18VTCAY5_(GL(VA5Wp(Q z_jqjy<`5(k@ZT3?W+;Z=p)M&o2=PiT=3;&1dQoGu@Z~?Xeb$A#dqY}$x$=@5Sr=$N zYjD%wNgj*;q4-!Z_MeSboDH*qJetw#T-bQ2;bl~xa^TO_fY~>Op!3X2=$EF42fom2 zaSx>E&!<^Hzvv-p2W!KLWp=P3-0LxIz#`F)X9s`txnKWfW!f*ec|Zy>^dq-u4Nt@* zGvKVDKSfXd9@1ra7ocg?95UX`^Ak`hGt67M@5K?vB3?(GjFan%ntzRzT0wyPb2_?UYFNzm`LJ;@YttJ?H4J8v%XvMau7#2`b}; zgcAj@{jCoL0xtOyy+{nGEaQX4g9iOyA-v(Si96tb(HOuzflHo&uYkZmU~>rVf<5_$ zp_isXufQy*pPlf@>O-Le?dv)*x&j-{p+jvG?Rmg1{sM&Sc!l+`^aX_~de94_HISp> ziBcs4LC?G0p@rQvNK?aE&i$a+anF9-O`4sGeuGf{(2Re8MHQgdh3EPI`GI!zbq1?F z+7TF6lUnEC@jyB5c%37A^?Bj|5JV-C2vfn@gAIZ^Rt>_vW{2eFx=ph6d$Nb9q&0St zvYAXCR{O+%zO^@^sRGDsX+Hv2|KVdqC`o#F)0=TQ) z8k)LpRI`y$m^@fnLITMP{kF2H5aWr$K9&a@#kP+gw-ShdBDVezP0d*|Dqz#Q&Fvv} zlnaU_3qk{QC%Js*Oe;m~boT!kvVnd(Ia2oLf{Yc@Z;@I;cJS@Jgd8Se`$Fr&=n~59 z6Pyx6IPrr$Pw$oY+X1&j;vvesVU-w&`Q0epNGD2@-N@hQCn}QJb9x|#?1G&Ezqv_5 zZpVG#3{&UOjSvD!KHf5x-iCF6?xoAz$t2!@dlEKtNN!=u++S~TsNZg&Y>nCF!blqH z06myFvhM+)wqXl>jLvULe{>;y;gp}3Ysx`^E`jt_LK}2B<-kR*aJOCO&g(WIWxP@8 zzT_I5IR_@UMG@ILa8LhiY0{+}*@v=l3TNULPQ^9Quy3iKIncH;1~CSw3^+C1=ooEV zc@unMJC13+TMXm_wI_^rC=76q8mjVi(w5VZ3pPzdvP$c{U@MX^vH7eEwg0xCrDU#+ zCm|73fzGm8m(jW@g=~e$@;(7AclH>B{F3f+^Fsj4>%;kwHateuq>f0auqd)o4D zO<=NK|GAiik-7^TURMp3Zb*(Irm8pF+=>sU#eO9E#I@nwr&c`=4og5DAI<&o z*SONY0H^e6?L=g#j#DHTXJO*m2l?4E4V>SdP8~0g4zWzJEmAL&X{b#sPb!NJX%;!t z(@^p7xV?TU&*0k6tB@%hJO|b=@F6_=5mGs(io9hU3Pw=o2WtZQH#1lNs)ZXsixC8ScSaqUAiPi|8BvzRlzQ^jBcg)``5n9 z>2|h;Rn7@XJ(f~C!yCPxo2n)E)loskwKREhT)X=a=CNL#N#**@ol@2B6(yTbkLwDR z+%TvSLA9C$O*iUy3uEfF!{<(FUfcZFJE^#|RW7jgX?D%98q<6vua8^Ch%RjIvPa48 z0Esy+OyY5!%K}&9o4$^#9eD9(ymilz&UZrX-!J@0^*E zpY^l4m(ul^w+R;!cUZ+_&Lh3WHf@dcwMM?QHI4dNV>h__M@%eF#Ki{uYvrO8^6=Ig zTdwyQ@f5B{jEwv}9=Wwn6frx0g_EM1=UNCcCfB;|a3GO3-!AW|wUf;Yr+(BH&)^YF zy%8u8X`$0^c2OY5c=GQk^>%%Eth?$-lzW96_OQZMiUmfzn8!{`*uZSh+^rEw&4WKU zu;yCIGvD@~Z}o_pl~BDB;4s-MR_tI|zwHE7<_KZZ_={xkR6F%iz@Cf7dp+Tu6HnF3 zBU8DupC3^+8H%f)ed^Y0FP}JN!Ov*7G%wha%BSadngL_+yiHBfKLXBN2T(eF1(1#GM(BtHpa=-$hBqRbH#3s!8cS@r>f7zW5UJu)}H{ zV(s}|TZi8E?BA#OHPP~K|IPdKod-XAJBak|&*AUM)VfXSx*llxh+^khq}(bqSnWC4 zxLT%c{GM)9SGDY$4Vf}9?o{K#Bk3)H+{DYp1H%Ud=#K`5!^4Y|baKT;ykdvt?u8ur zvUAopLgGQJ>pu+%`N%v{vF2<<(zb~nhh*lJLkX|D{%vi{b%D>F4@Nod=Hi)iHa`ly zD~gHz`+jO}C%IhDddb3U`!#*_D6*UAhm1;&c^lb^v% zU+&l;9G|paup}g@UN1yjHCyk}Des6BLH9dXIh74Lm1~*=;$R5gaSN}SK$u~&D2 zT#lT|z`WW}Bkq%moAREArtz=Ci&9-a!~~y{J-IFt6Se8?RsM~n8Ru!^3ol$G%fbkf z*F(Yg^Hx8b8({WcpEDo^`pjhEyl?u)INTm4_&k{YHjwo2zfNh=L|K|slF%mc2dkx3 zZ~4rmPB=v(2cFXf{GRhQ(5Kgbkfx4KT2yi5kqzgQYpH8t z9&p<6@94YIPA9d3aBPFwR6tRW6KSmWYb4c4D=%wfQ{$5tPwr+VpOR=#o(<)a%8P2T zA`ijq`ki$?KPoRb5I#tfZS+OzP%zNY+aYWTK*{~WDE z=&uU>yAvF$;5&}C+5|N5f#3COu(hjBY>>_T$4<$9tT3a%TlyHOf4D0vKCb;gMAwcN zWAWyRgR1M@E1z+hZ_SMB3RdYU;J~(fw)cK@4x(}K2df1P6lDI;tu43ldG@WNxnqFK zyUSg{=-~#%Va5BQf}wZ#3OO$e1c^*v4WE)NdK}tWm)X|J^-a1>xPBe7dh-rPTi@D^ ze4X)Mii(QE)q9J(sUiKU+bYC8Ph~TYh!qsPdjK;d_xpLfi9P}Xq^9Mc7l4=Fp5)aXf(Go zBhDW-*O4_%R?=ux&6k!i(yky$Ib4&wyc%y_=q#%FNXU#V_0>6XN@e!d`+x+CgSoE- z&*b+jStrH@5_}F+2wbA<@EgAggC7x*c+!T~C!n zlRxpSxrq?CuK|%TLk`T`s9VL{a;}DSa5S^LwpcI&ZMsc)s+92M>*Mncg!+;0CSL3A zd->`qa8O$Y*tiS%i`PD-#TCz_;li6_LP=%w++n`NBX3A@a;DeK=+lIayPJe!|_2^^r*!52#BW$yxdXt%Eo(|%Wlrk#g|1du>zSP&^?w^kH{pm@F;(nBPxUGtM5)^EKI{*;(QqZ zaIA)w&OaQ0JjCXk7|=kj7?nnWtJ+d&x-=$rLBOhKOC|ZbGv`?;&uocx|512K?m1{3-FMzswP?K#6i$xCKW9hkQnw4%X+d zJ0WC*0^nPmzrw=iZgIB{>@VOxr0RIxBFw*A(+5#bc>MmTM9aFEXCSd%|+CF(BO$%1O#o!{0@qAOEs{9SBK2Fkb3q@-? zea1vCkawcIPB^}NB4n?L!-qAe)h6Hg@}LykZE?Jg7TXMgYL=%gK1G{ zxS09aiTG#YTb90%U@E|4z_f=%tFvt^f|o$}A{1X>L@dbzY}V2*!2}vz8n+N=ipjEL zLIwZD)*jBvs*5NoKZrZ^OXQ-V5iYaQWhiThSh%p{BmPJv0i71?O>TU9dCDWnfpJ`Cg(2 z%2xCRoH;en(@XWh*<3z0Km)GhoUvRFjasRPhStTh_R!$Yv|Ps8 zBbVu6G2jwdCey%!Yh|en08n7=Uy%7Sh~QHRgT)%Pqy~^7|3WcOpwoTf(B#iAG-H|@ e9nK0?);7rWB{F1C%H+vo1Q?W81cE+qP}nwr$(CZQGvNW1DyX=iGDOeQ&*9=}PD8N~)5zQk^ep z5_w@!8U|VxNRpQ8+D}Mke0qF4LrX|*ZhSgv6I(N9b9|P6kRl{LK0ckOg|)MZoTl4AzYt(o6S1*-8y)S0^$n+P0;TMh;vS zjYzFr6;r%6%EW2sNpfnzDyZ?|QZjZf)}?Zbt!(6_lHYbApNm%K=Ir9q$LmWh-d^?V z94Z#8Q8ZLR%?NysLslEDAeA9%Qj|^}5A|M;uVro~%3598Upe2Nrjv;=E~~um`$`aFK@G>B)>h4$K&fAxjA3$pEqmTW;lf0i4T+V7Ql=rW@4=zT5+Wol4uDJpNlvx;j32XmY$Nfx}uRL zso43+J|~>Et}=A*xs%V98q>oV4YfBDJgxd+6+ab}_mgLX5uJK1h{L!nwuD`*!_zZ! z2AfgRahFGosqhg8?eDV&@jJ1;l*BnbUsc3UIQ{l zRUVY+Zk<@qS1D;wxL6o%CB+^b3Y8+NjJS$*$~BZzD?y1RmB}&nAb}FwEl46~Luch|)faBN|@Wht*q)D=(?L zy~U;0q#c*+T^ zInroA{K<&s&D;Q;3)HC{5coNUNvE?qNDaU^W0woxAR{}9Hop&xwyNjAord@04VQC3 zwcFcx3FtLL!0Hgftwvlcjv*i7!IV2SvEn7Z0ydjJ){Kj1(of8e#e%o!F(~&J;RJeT13)3NSpU= zv^QlQRSm-U#&#l*cg$SJh)kJs-$9U(<-$Z!!DkH)s*hRQe^X=hd=7H%AcJAH^{!L1 zd;D4B+G%(rBP;b|!8pH7X`b1&K-;}RHtBtywZv8CU6s4^;+f)=SF@v(<0z9){{SYH zvZxuCU5KQtksAkY!FxWVT~R))Fg9=cOM7H0Dw3gBPE1QAAuE0~hbXcYQ(Q-PZ#Y!H zY8m6to-`hX^L4Ax^`lBX6qXV19Ch^2i9Y!ZRk6(gXMc5;b5~8NVV2@?1;t=$M!H#n zIxl|QugugDc|fs~-#b?N%0zL(Y8t{@KLK5HYhYhxWo;)4;ZclQ*({q>kd~dnPkMLn z*Pt$ubWDkpuFG7?Ou@7+H+h}ait(TtyG9{wuz*pFX9m-(UI50*WO!7m!L3L<1K8<{ zXhd*NhvAl3RZuJCh5~R~oXBL(3nc~0EU)=xR#aZ9r88bOF;jsDr z2C?j6eF}XVuuVM$U~CS{m2QP!`f$6C=aZ{bO1!~oJ#Ay!H9hs0_ZNrNWWvU`Vs9EB z3$JFcDo5)+AR(h`IbQ{~9%m@xs1fF84G|6iZ|nh%#Nzl%D0WNwy{f~Mn67j$#z3V5 z=mKsL`eVgc;7E2s=E2^bs~a#9GE8O|)4AlWoA*Yrlyfzq;fa1<#gidOG`7<-ddKKO z3oPttza^j|Ixz#~P&Uh=VOAXlA`dJ~qfc8EYe^61iuejBrKM+d{hLr6a!Qn&0g@EE5|gSTH~X@d&>DZsaR$hI3RB*se*1Z$<-k%A z%fgtNGX71Iuc8QY-R_p^Vj4$P2kU}HV;xO#)$2gEBc2cpa!k;5nEVzeRqAE@3rpnc z`OJe^FdlVrYXfSTVg<-vmNfdMEtU}sbN89};6TD~e?;&F4-84lspv8S_L|nt%p4!d z#8+&&MvDwn=9->_wlLZ&--bkQx;1k&%(2rMJTd(&pX2U#ohT=Zckc7Zj-%LxUfvDA zI1*k&qORHSG`a+6?5xDeF9QUykVfJAIYd-`mXzaC)))qmje`F9m`F5B?{`-JNVI^J zRS&NO?4U&I0Tm+=m6`tA;5%7kpfio|mK0_gL0Oa7o_T(JQWc>V@|o+PcGnb})KkFL zdTnvkC5)MJF`RZ1ZDvo^zk>H}6l|~y2xkh^x7OXgAweA|6xoOoO@|r}Y%Q$#Nx7xS zSR;EXL&a|{BwPPa4DFo&-7XppKWBV?yY_Kvom>3Vhx0Wd1KWu$`=>slA8mTC@K{-E zA}ZW#lbRy+kBRdYHP8}sB0Qxz@!M1bjk<+Y;cJ$g<9s}^`dI&$DT$I5ea!f}@h^)K zpQ4dUXkNl*(?U=P>BcA{XQF|XtHH~ntbS1D9HHR^o$TP z`;B@&Lu#Z9DyYDqg)GVb7^7F;0d4}$0*B&bFkUPGTUL(4`esX=@pD0;D{@hwS2CB# zT|5=b25*Kdeqn$nUVl9<*yRuZ?|3&7oHz{J@BTnq9o z*H=p6$AYTCOX?wld(>10?Kcf+f?@WCF7LA`&m1?QG+jG*qAtoCfR$BKKTIlIBDRH=oh--1qpE*`Ag_~bAUFr@x~1dUwA;@yj~gIEiWAjyvGWl z?-RJtu6YuejX%cTG8a=h`HL83%l|3TtF%z;x5>Y65wf(tk&m&E#e9aB!EFm?|F7jy z-=SApVV8FdNbd<(jNJAq@Z(n7PYAZVPwZCL&xEKIfsV;NRi1S`byEzPNR^0SoPoBd zyzjeW)w0EI1{Fa+7FjkroZdS_Fl$U}q7sec)tBz3I&HMwyRd9Ctf%;5Q;GNSP(+65 z-=T=)x!TIfcThZ0MX%$biefR|?c0sdyJex|Og;E#I=#_w#4{mPAfIqG2sb znjBiU30&}}B3x-}Jd6?TPuU_y(>`n|0*i~y(c#MPq9zgkA=TKK_diHE*}hU$qOlJG zya)U5WSfiuxXB(auVs6`A6GD(}AJ`HIH5 zgb+UOz=RMs0(~I(euA~2GR6^tJ#9|m`PboXnB*jQI8QfeeYC&8Qkp+HkS4aq|C?I; zd-@M4`41EMPyUaKFfp(&{zu_IMW+A5S^kTqh`2k8DLMaRDEQpm{{jA8bO!YQ3ixzF z0{9I0bVdgM@5O&hI#D}Y=l@IFl!l&$9-sApc+YWMtlx> zc6^=x(iZyX%0G1{{Qoeq|E67Hj&?5g|Cf9JZR5Yt|LLF7Kg*5=wodl{v1R1(Kd=P8 zv!jd2|CEIOX$hOSS{RuqiV6M){NKG-G;y+XaWpb9|wXI8$dSA}F~jPH*wn z5V;9aao+yy24bUHeQH5=Youy`000O^33k*$PinkzY=L^%0AkqLQ223Bd&Dq){sDqe zHi*7>(nyt6f zoIX5J^Ocr(H9rVrb>7uu{G2f7<3y^u&epjqNWl*maXrV8QgU+KZ;16b&cEsj7Is-ZvY;P>?-y7dNcw z@2h#TXXOU7_Lwu?(?d&~6+9+I#tL)L=C6SPn@X`7&6*k`d?bw&G2(!5s&3K{xqhV| z#UF*esmvh!YGLj^h|%cgTewxX?gie?U(xrRsDPzhaMG?@o>k#N9j0B^d7_hqqd_(` zHT2sjMOrVMq&lh}+%X)wIRS6uvrV3G`Wjh5F#8qRBl!!@x}C zybnLjdcVF~y_^F>z5eM4nOc3~j(ZA`7!H6!CB?$R^3w9k6~b-BbQm91^~3kw=b_>t zk$p}@UG~gTjQdV4XBZXsVX#S#%aCD(cyACJgoPQ_whY&RDGctRc6ZHD60}8eH?C^% zK6OLF9i?aU0DnDL#U1HR*_hO;>)CbhInf+bT~p1QU)u`%(4_NEJ4Bk%pkwE<{@~R~ zeOlhyimJ+i?TYz3WOA>rW8>4)6#e&_RZ7OiN5_VHG~b9#*Z0|KORvj!sWp$buFHFO zU2S#8{oLF2R;caz`{6u6%$3LGXs^yxr8Gd_nr=x|hPi6H%W^>Hx*pE+`8* zuA4eaY?w$T<$?M*RKktHyi21=XUIGs7!8i8B;V>(IHAnxYYZ?kbUY6hUY|6fOhSWR zJ(n0>8ynz5EMt42$fu(%F0lau-7XbsB_bSPYM{+K&95yq;NYOv0BtvYSmut$~0IardTegPO7? z%;V|oIF5;J%_RH_$dRVC+3U}hSxtqzPZy2H!SrBlI~!sI`*3{+mfgBj?i?}(adSnD zeV|gsjj(|7fO9H7X{y4wl-vU?(4TJpU6wjB4erD*Segl!Gh#u#X%Jdso&czb)zIxe ziez;O+C{+!9^m~0I%(WR{z<%MX@Kl-RS_d&RZSam%$0H_3RDvSCMUZoB@BBK_6%<= zUP4*Z$mMJ3e@h8|D&lkXROV-xkA_3MJEos}*v1$Xm@RY z3Nm7)HqLARD6ywNHg@7KhCD5T>=vtE0mmYY>4IT7M{}Z&g-zJvo_6vj)RG)5Jzl=8 zI$yd%rZUd2oFDM+>l}mGbFe^5ftFJb8bMG~R6GqE%Z7^?2_l~1f$0k3q}X>*k`O2t zhqCPifB1i~9v1L2gXnLInQJA>v*26uPiMO{(Q6sC0#d87v8x<#I%N{v*|Z6OJ{GMe zh`i$dZuRN3EfaTeHv(o+NfU&%39>mc&al~yKQ-6Q=ry=1jd|q#-i5jU9QwV#0L7u> zetlhs{r#LiC{Vl4vI*KKN$Y+%(o^u=`t#+=ktN1@Y}PiTC@^5qMAQS}507D?U(CH` zUZ)J7Do@fZMCe|x0#?mhF0$LSVI7LT=uu?Kq8Mc_~sz3-5 zF49swTNMBVQ!cjbwVeGyd&9NOK9*RpT+bXB#V~nfGXPMXBy7+E@MhcskB&n;Wlr7r zP8dTC^wUs8zt&6S(gI}i zDgbF5p_$p!&8>5JvXhB>84~OJ7jsQB4zmeUw+7*f@>9?k%n8gd%eTxu>bGk;sWxbJ zEu&4k4^@uA&!i?5@Y$apylSh!L*YB}yjXL;e)CQxbOm1e08j~a#fmfiJ+ov0Wg2qE z(=>glF<mo?M(oP(;Y##>(DLyewK={(?8e`9T z%%_JDK|l@x0)id;Mkpr6JbauG%sm*xv^-6m>?tf%0GcCunLd*?dxZ)`1DG996i?20 zWyD`T!oiN=geEmbann#zOWML*lEJdkbs@(^>a-dVn*>yx_Yw>!HpLCdbB&E1f<>R4 z#eTi0Ks_5C8F~~+%~@kbFzsdI^TpaQF7j0^cMm0vofeZ_h zx}q(x^iRYA*(gy0(SlycUU+wkXPg(^m+o;;@@%5ib4jmq15FFot*L7h(F?*{xZ7}B zk)hkB{vfI7l;u;mT$X3a)k^)o5e{qA+<_ZePsJz8s})}I4eTDyf`+jWVGDUnb7|k{ z=5b4fR!m&j>uUF0@82b|kV*j0T!?-AS$$ABeGhD&yxe}=5$zku_Pke6ZUHz{9>gnb z3TM7s{16jVDPrq+hf;PJWxy|@DGG>KjmY%e5<0us!R7x^5=L*9ya z0qYod1TKOnLn6N3NMH2R(hJ_TKhj^*Umk&%mm@>q!&2kUaR&^h(Kt&y`sgijo}xWP zy`NJzP;n|}0vZYT@IaA(E6y4Aq>WV1`o{ywm33+5^rykcvzCZA)T+%IpxJHWh}^-b z?JXP3#wF!T#?OZRiA<8iz&>E*;4s@8;tBPkQ31GtLaWx3CNwW56fY)?{qU&hgNZT{ zsEen?hKvF%N~tI?^sxcX_KFSY`q;qBfPWd4++g3|>G;;ObSVw9hLXS51sYvBqZ(ba z!O0DrU=ZZ^-y=oJ1wk93Nc5)7$z5mu;8Yds|LX90_+Ca*3A&aMAV2Ul_xe0{X1GyO z;(j@PA7VC{FEn|X>2`cJilD?q7<>MN-dcC@@MO~;s?+(xC?I(`N1=I%_gMLuLt^oRceA&=)FEs!iXASO| zp*G9`d`A^gE2$RxPjDmysaj1*78V^?xhMPrB?i?4t zF_ODxI32$Xm3P{I;1?LKXg)lY-_@+lS_Ez=7>rE9GmKH9xX9&7U(7|OP2M6dOWuvL z584t*i;9^Fk@7dcXR9yY-eFXns!LQSK)`e>aD|1FYd93Nt8N>d+R7YoRBUI@E^jWI zoK7if1z%3DmGn|_7yj_Sb&bk%l$DjXy+mrtW&|+-Wr=H6D>Cl)woFSp57S46rH%rZ z$25#Xq)J6Sr%wkfByn`;j9ygcyDCpXn~ODjwN$?dMKmB*Y*R_>6b!&KAr6ughdvZc z%s0eSGS5FLk~NSa{7Ech5SOz7~#9A7oEONQ9 zgfN5NUopQR(Jqn7Lm01+j{O(bv2THCr$w!|pSy@7!*;9rD%Bmfhm-7}VR76ye zo}>+Z@CP}9cpBcj(r&u-&t3+|P*=z*b2er%CU)T4Hb?CHQAS!Yk5Z=CW_f;rO3E0K z44Q`QwA*eX>w0;g_g%Xx3{Qigtbi8CUC%068!x@6bVje4VpdWu1~-ahnkUu@0MceuRuA#Mv?0+qR{OG2|0e#`oSk&fs2Ht#lDxk`@M($)?~u@Ot*F{-*%&d^-qMM_9L*1(h@{F&F$GQK zQD;ybP+lJBMX0qrbv~bf)MkJeVFB&wZEX#mM+~fHnLCsr4=gkmHbNB%T1YrEPgW`< zE1DKZ786mV_@G{jTpovI=1XruB8G5h+MsMPoR++>l?PN;h>@#wV9WQo++Jl$MK*-t zW+91MVq}0|m75A4Nc`Kxo1a`(D)$tjB;?QApK0RM8VA{zLHLbf_Ka@M2VZHufDLlS z+-Dn0SKY|UyvoMdEjkYgjYpX@iQX%lmJ+{V_e>DAg3SD8ui+j`c_|blSQC6{>4KVf zHZ;n3m-%LjXTV7h(a(LiFGov9M@H?Ao_-FmPhik>!~qo!+FnNhrU?{IRdWm&K%m*v@M^AC=uOE$~pYCrO@I;<>v!@3wrV(RQ1jFYvckke6*joVSLshcH!S3cgdbNrD_NFV*ocGP%OnN&L7GZiYoIbRFk`4es6 zk!Kf5RW-9^38?s`@PS>jgGR(iVk2v#$TGjl3Is*c(5Q_5E&Cw%Bu);mD>IWzv9XdT zmb=oaHpb5NvXT|T^R?NC0I5gbV6z$PVQ9FZjbspKHIqlfw<4CV`H(aFpJ4`ct-m9b zlatCo3SBD69Tob4ex*9`8H!yb!aw)SCf=%jw@_Ur(k=JfPYu1j68Giq#c=C>3IRbF zc%q=`{87AHai18ZfBW{l`#%4CLd8P7e%d~#*|S-Uc192@5fzNe-mcCmWi_k@F^-L;$ALc`0dWUA&g>| zMXq<&xDQ7rs5GC}B&`~)9;xnf?t3|oQ4?j+78>nU{M$Hylt-<1`~YF_h*Xky1_IU- z+NbvS6P6>vwp1qt(um=)ZdFjI3f+t`bUC%h@wfE>)_oeqVu_g7cH-t>+n?Mkhkrw8F{Mcf zX|4sA=;9_m$P6Jv8{mc%rG+%FT^Bx(lZ-bW7Ho03hr~V%uizH`1Xqzyuf~5so~TJ@ z$x@rfTqJl%FeIFzGWSDKKa{4?jIW{IFWg7mKP*Qjrc%l&3}|(ECbxYlV|N@(nt;cE zDz1v-WfahA%FL|Ue}LV+dMBSy|3`!L! z>Xukx{q;>FGYyGiiej{T9UAY|iKrgiyBzWz;*BX{)Dg8O{5=9$C{7zAjq!WSrL&;J zqXr9%@mU8jQ=Hprb8RbNmRk%cbmPt_j~dc9s4aJD!%dZ=9yP5OVJ*j+>nu2~K_0si zYxoPqIn@_*7~~n$2V#_?Rq5X$qbY|YCnAkdv`6@dinb`Y(*Qg>(F3hT^|4kLt!GW8 z1~D48fQcGByeR|XJdJo$w!&5d#jalsFpzZRk%#5+KvRVYa=;Bm4z(CBk@E9U2Eto@ zqZ@v$wSEt0^ei+O&ss2BQYNRhyA*3of^1S$(r6mZxJ)x=p|-S3^T%=-45Kb)&!5-Y z>N@#73{N{&5~t7rWkj_Dw{CjqmAafq-3&OF-t0Bi;gcoSlp$X)P~G$7xY|tHc$tZ%gQwob9hPtVgnUKEhNgcG<6GoIg8?b z=YRp{K-|s8^ym%`434S2OWz<8*PpmmjUvU^L3bm>kRIx?~vv3%}U*e7WHXcEMV*Du%r_2r{w*xwuW3r3$>(;n%{`z~x=Ayy#`*|*NoO7QW=M3{_UY3&- zEGFkDMdciY#ys!HR!gTua&fX0dv4c?1A7YkQ256Emgte>yFvf8jX<4pL@;-n(aWX>92pT7=pO<>G$Xji z=KpN$a{ZB}O?yEPqLmwro~h*Gv(s2PixcLaq0%@RcyFR_LN>p8uH=fJ25;)_3Us`N z3`yjr9Pk2!h~`F(o>=AcN5G?m2IKjNY8cxsdml(${C)oJJOF==^qrw=&7G<_4nhAHV zx1=Oo)IiAM1urat!6TB*1D@9<)w$>E7j0v-^#rxuR^N7}0%y;z)WE|$8CN+L%rGN# zD0J!_B6=*svZS|5$HY6}j*q{s5()>hj8vTuG8E9z4+Ti7A1j=regn$T)yQqZ@EE8` z<=Nl{M7Pz4C432S+T@^3E2po(K%BU4$Qb38A&qBvk*`6%vX~)=@7c8vK=8`g&{VFvisFB6UE5a7b_nEi`C89)gjoB)-`R;vC zjSWgnK<9jls?|y<+bIoxrxBC8Wj-CJ@+ft72|a>3UkN=ba0rJ`xMq;hGA4For#ei+ z2zbZ>ycK;#Awh5mve2MfCzmKBw*8Vm^fII;UE@RMKV4@R=AUexs$n9I82HS zXgB&kgB*i@ZpT8) zQxmd<1%)WGn-FlH;sL|q>m4)(G698hX=~z%PwAVuR17I1tGquOJ;*o=GQ#*|q>e|=|X zYG{niNPS@=dT0@|@}jLA4ZZ7SA)^qZKABY4XBifWW=GInp% zFzdSJnBtaxZETNr9a}pLLK>l&PBQW;+e&PweH#>9JE%@kEl+I{ZHg7QH!4MO>bep< z>OhvdI`u|V>~7TQ057}*gb8<4oyJ^A;=KlA26Y}&S$DU`Y|Kq%YR2knvsw|XyEg<; zYsC%mpP7>rCsVmzIjV{~4H2Ir zeaB#k7U?AAbvy}3{m0C zU!r7#1Z6&HlqKZ9c+*2{7NQycCA5%;DsgN?n6iwiE>aVqvJ#31@Qyl=f^woMg2W(F zxsr=4WJjrhbRdQofj{$;=7nFv4#?(+WBejJj`pz;f&m7&Dp{GNvcR6huhQVBB%H5L zBhQaU&Vq*hRKF+Cgslwc#C~C!khl7@1kFeDv=p-V3DBetm>`!+%91$Ix%pco4@JJ9 z45nBkFIK!c*?H3$&^ge#z^U>l3DL}R9Xa(EdyE@ zH3eeA-$7wL4>~vs$X2DhyM$4GDVAxi7LK#h#%eRNUX#rk71IxI*dA8I^j^3ccVoX0 zJq!_(pf}0>hUE)%?z1BIWyxYhlQV>+iP{PEqB3GZ5syI%?g87 z)G;}MVpWleFI3H%AVYP?p{f*G59CJX#$acr~IT|z8Yxu;|$D}8e|fe&-GQ&kjtN~Di?3Q5(k)Cg~}qo*4fF`kH6 zrR4Ng1;F^iaen{7ta2M2emL#Fjc!E(7e=c4#gr^HTfK=|?UA`UM-e4X7bi)T?znUU zNGmV_o(8NjW)Sg*ypXA%&%~1UYjHAt5<50#D*5t5gvFW4q?& zs;aiK;FnXTUT~%|*i`?s%(zp>;bFjJe7iI4g-4(Z^6t3J0h4|e)Lp(ahYp>5RW!dI z-4V4a-qp{BARNGqiZprX4Sz8&ft{2@z%>x34}J|HM)^MoB5JEC(9?Rtw`G6=?C_~c zfcZV4zz3(Q1`UZaq#(tyvwAI=-Rb6%bt@GfvL(5D3M%K0 zmR>osaptkXlqB5bCnTbo{VtIe1R1GBF)1+a^#ptU%<gwnoRw`$Hzf--J_`KI7sq3E?pzxlaNO8JtRIwY;Fs%d(4xI z{Gtr%{QHsRM4HZ^Gcs06rwBch2ry$so(S+*Gbf`Lnp!O6OHHIlWb!GKm%orwI-ONw zC$lVD9-(M`Nws`ZYkgqt>IQbOXi(CYt2+mJ&u3>tgb2a}wET9Zn_OhaD^9xaRb0~C z7@~VBj7lr1s+2+$r7ka1)!HEnCRM!3vE_@E@obn!CfQ!}wl+E%&qFh+)Ffc#dvczB zc4h~f72Ej9=A0FMxDgrXg1RY3@+{LE>MFoEPji|O_1yI+l@l@22SYPpyR}-djCY86 z6$?!}gHtNCSMd`*WO4A{sxw+5EKNsLU>01eMJig!5v(Ly3D_0V3uN#g7Os5e_NI3M ze2~<+I^H7QhnRN!SL0NZ9SBIx8K_R;-;nI(GxTf9AhV{#l(k8Qr+SSL*5&j4yTgF=`2DW5}rI`Ch(6e@RT*6YR%1MvM#O7}>$*j0GUE1n77-lh(c z=XjXRZQ)TNh+FHXJ(lF^jpuM%I&|#t5asl_4@$TRh(?%q5Vx}$mTuQS9UBVOQ6aS+5Xba9&MsHhP@T_wWYAJB$ z>}c4%bp`7&V3@Hm`cbD$jj3_wJ@uY`R+pX^!6JmYFDZwjQ>8;b72`aNLW)Xis}dVV zcw8%NBeRu@EyjbtB}gfniwr9YL!v9zi|9>sr}|JmRObv=&dzLel9f@$StEWk&S_Gy zW^t=Ztx)SoC2>1*jcV(1ZFdW{i`&J;lHjsi+P>(f8O3DodBeQNWGY(#&j%MP(yTTG zQg_Y-E<}d?L!2!ACtkt|Iz8})AAtUD^&5nO6mkoB1SLj1P;h?h63n5V+z^|DvT*fpJF#*A^ z7=QqNL|;A;UWk!FQUFDsLTB!9N*)>FO<^QiUcWIv?6P>NG^*Jh=$4Btt=tUiAo z#jbQK=C%F)@n&uNwo~I(Cjpus%+af~R4zO70?RSyLApaE@gxA6+%@M;I-)E*i!!1EH-VARQJKLjUOie4 z<@Rd3lu~!TVsnoII$PqD^<+iJo-x ziNkTigfCdtk^F3qa}Ls)n4RNhT2w;6titEzDq{|QoTe*2CwrTtW|f^^v>P_@!Z1_Y zgrd|TeZ&Ad-mIUMgM1yl=nE*2)!g!#w%VjY*T7Um#yyS~VakCX2S`hmh&P(-L)4{V4yeuDbKCG}tE| zzSEollmeYs-UCWh(P!`a_cL`R{2HTLnqE(~z=?OJUaQohBaGCVt?fD{mUdUxR!!%| zyUN;4XO5Gd?cR>Z<)R&X-$AF}If$FOOTS0|v*mSFPLfqz4NvdOv1XlE<65{&>&L$4 ziqT-ZPA`o}yWonxEkbm4^g{ADv{%!=g_IFW*N1fs~gp^>Vhey|K_L1ELGqtQao z0$QQ9GZ3G1y%;XU0lcybbaAZ-bu-%&PzRN&wUCfGVqj;al{12FG5-zW!^C7EvO!@~ zr_H*k{a~j?|JI-I^a8Ac*uFlLHraZW$lV_CEQcf8*YGR{zPT^1cy*XAbQYn6RJE?T zvA<=Cr6YX$R0<@J{&^x%8 z6)wLyva2eMyEIC(M#51vLeU*xy1rYiBUtO3Qdto_GOH7MoM$%8?pnx$TRs80PUgU~ zk)ZK%OA{zO-h#>Dd_<8R=sQrT;3CMtf$>PuwY#agzHvJ^p%UNe34ndz<3Hp(>RSE+ zbU=ntQyDuRUH4lpxD~}SNuL1>_!e#j73En>@1nQ&I=Lu$70>>4>#V0lt*Nc7 zsH`!3PhDSB=hVd2LeV)G>g52$yQtQpF7RL{D|l4qz`LkbzY>}{MnSZ7a(Vl==CH)m zZQ0yA4!PLnD6g_i24 zmiI;*uit&DY)*&ljvkd+gU`Zf>*h`E^EC^T-;^?|jrUJ~5=#r*7b=~^?vB?dg2;$f zHf~L%9zDC`IO^2qbYiBF3g*y$V*!j?O>M1PN8uwgR-nqjUC6YJn%?jHg~!lZTup}C zM`>oRxry=3%byllu(6?BS7h1xD1X|JyRsvoFf%($hTO$4nvX*<+NU5ivE^a~xOgco z0?#QteWqiuovhu5u<hbggMo5lHcC#gvBv;vG8PL2&CW*MO}rc!Q+>VrFzYg zw`!{Plgg7XNSC;_v9jIbPED3TQ9-b7zT{Ymu?0NkOi*sTI>(x)=P)pB;NB{oq28QS zu9~7#g|(tLuj$=8-!HbKK4hU2tnBKeIw~p%JSYAJmlS_nrG2u! zwY_b0eWO3aJOhCy+DV_Ok@oYS^Q=I5!s0K8*v92@8_!eY^kJ07N(=j5>lptQlL^2mssu) zx}3(k49QP}Na+aFl`4leR>Fp~3e~UDCtJ%i3xS(Bvh9vs;O9oB6 z9CT+0V8AsRX^0V!RwIzoP#u3YW-*PhDKa01raBqSpcEgjA=NpR*+iG=>Nlt>UTb># zAjCQGCaj8Uxk`7rKN6qjVnGEgP!=*MMBokbLOtMGhE-W!jW&W4M)QEO3CWUf3->9j zIY|d2W{j%DT9cYJWCsl`CZ7*n4BD*;y?@YBzMpW$B;pUp;W92+ax9pOVxe&orH=vm zA4%2dQ2S!T`~-vZa$OciitERun0mb za|HAHV`I4Rf8YK%Qxpx%c)Ph-PEy?(THnP1NX?vLCk!G5vM0?e#jrn~rR6la$V*M4 z6GAESa4bq5%}-d`QTW5emY%6^g*ud`1ZF$MktQ$1p~B@S6NV9Ev#s0#WQtht3ZCQ= z%L@19;xO|aZBcud%LnWXBoYpeUOaezEnQ#+NG$~H1BxHOS4)mTt;{X(G2x&bwhLjw zZ4zlO&`9mO3|9)xm#<4N87y5ib7+P6$e9YcM@hNE$L_4shn_xyN3J8(sdrWtsF;9s zcPA4s`4!Iv;q7q3dz!zu>F~-vL&x1r30r98P9y_aGu|_R(F^Ild`O$-j_(N;u28;t zg=u(?ml(In=5W=}`mnQl2$Zp+Dg=^gcCi?d1k!f&Avppaj;8adYI_TO?&T-f>H*rG zPLHX$nyl8nO8q!Q`_Q~BjCF5uh~9ouE$L!q`PdhA_D0InnCTVrjot?#xF=5&U^y1L zeX6~H#8CPSl)lTo;<64?o=1?L@e2nN;{lJ!?l%7mXOwidTrp4;$qQ-SgBiB?;9*T&0{^#Sx>2w;^sxw$mpc`jK__#C+3>khYTcweq=X+@>o%~)HlXzHj z8)3*z;%RY#BydV_qoMQx#Kb_==@E=1Rb-t_(BSOjQvEckU!@$`DPXZbgoP_A7h+G- z50z_MBtd>n9MV4y?!$MdGAEi-w$exzc!2Rh>)j#5|d%i@-4(5Z>26 z@bB)Lx{j9S?Qr7Y`%8!pZ%l|hD`1dM8A_F>g+Di>Qlx^M0I}=a(k@PwtxzT4fsnO} z7IZ3OQ9m}tSK*pFQoTL6xP*oy^yFv9VWIQz-@3BIXuJZxu%!G*0Dsj0?hs3m9n-821L?>Gn@JvK{~in7mf%q@ zLKUF*I&GxT;Q6-qRx=*vaxGy#lA8XU$fr!J0&Gii2#`%wl$1oT}RuL5UHX<}f+)a@kq;u3^*WNCn;aLcTdYXVghr zW5#ID7Fr##-?ZJadL|8u)~`z!nIf8)Clf4KV%Cth$4It+BXD<2FYrdL>UqCl&X14V zF)^*F9a$P5AmH`pMjJG4@W)rya2TO`cin>9a~h&$Q{%Rtj?AL)%n{ig*p2J3M{wsa znU5?sOeSF#UZS7a5o1`{Y&JCfn6A)p7cG=i2Ws7!8)uP~MyM0!rf-K|(zxutr%ExW znbW!GU2W|55WCda=e%&fvgB5Eua&x*yS4X4^j37tIn{kpbE=*LGL zJwZvP+i)Yze#o@QmW|0&gYE8Ue$~DZzegL(e>SA9*iKobT7Ixpn`ZGtANan*ATgN7 zNklhy@r5u|Xh%wPSXustV2{8TpJVBI7uuZ-KUMHP=$tv9znR=WL3D((HsKWkdU_k| z1jdth{?;&DLII=VCI*m9EuSfWg+^*-gx7R}uXbZTnqf3S*xB|oSf(!?kzJ#p6u8A@ zt_WsCPZdF=jq)^A`QE-ebz)KqZ3MH@^DQzyHd7p<=CEAxEVT3O@+!srCP=F@a|$s} zb_kAcpVRF)dTCX5j~ZldagAQVl@56-#6=>KMveOwFA&9Q02u&8#2X23DFD)qUEa04 ziF%_wGCpEakg7<^EWQevgaEzkF_^>YhJxjLYgKGRo@_9k{C>j?q4y2I?b{uM?QbY~ z>W2PB`I`J)M+^wQpYxmjZAkh4Em>>w<7NlUhH(uL0yP%VT!r}A9KK$_N#8Rwxy zS;Pk$rrzKFMf?8!Jq3F}BkaUs$D0?U6Aal4}AKW!ZRUyYpySQE?l z_i0i>7etCc=%Nx*=mMcBq99E~krt^z2ql1_cThk;M3CMDQKTrnH>pw;q&Go&iPHO< zz}0)b-uLSCY#F>uHFW zlXWQPrB-1TI-oA-)iF54MKAuDJ%g!m* zlf>A|J<9*t6 zjeyX;_HkarUNzN0sU531tNdb*$u{XeA;8C>CM5ftGZRP|-7UXBF;thm>AQ@Ck4^x$7>VoZVbHj6YUn_{|XUHlyNfDxW2LlqXA7Ku&3}4&Zxz zW&J_K9`mmM)7Y51qV0-TDK+7{tgg(?Q`C+`9>GqfG+u0QimtBj$vU?6BT_xL9c5U3 zV&MRrnsAz_3;K-`ECNf>417J;&ksN0@cF)#Zc@sy^;Kn`d+*U32Y%iBGfM$*BAS?V zju(^6uJlb30T)wgLTjVn>Dm*nT+rEASIe(|GIwu|c~`Q1ihW>9y=UA`Qo08Bb*$pE zx6g72?-SBn1JbTK%If)vrDyku&RxddHd0XTHp0$^l3GTiV2}7=@ghTZ=JYzB-xf9Zg{U)*H?V6!VuL2jNixbf1rYR-CtW$9~igWxS*3t z;p#x}oPsFo?bVoCwo-cMJnUD6OPN8B0n>QM8Ria>xS!#{wD%s0A?(B&9ip{VgD}oJr5IZ<`)W+zslVx-l^r zri|6ac5G*FKiFnl-{F6|OusL@O((5BWiheC=Xtg!=*SEz`7QK?AeRzb)2wJAZM2YtwVnXfS4=f3OBk74sFU4=V{C z{xI5J)pU^H_H0l8%&nqqkfzK_*T-8wiuSJ`LJ#Bj4G%pIMZoi@4c86oh5o)%T!7vM z<@^=D*@#N2Bi6CB_xAnJRU*UE0r@@~4BrxS*IVN}<5klm11E@V>-Ft^jDVYvG;>@!?SkU!T7;=%C2<&KE-W=KE?K~rytlY-p(1+E)2ieOX=x)aU<3sD=1sX zBMcPaadft?D1RIsJ8UnxC$&YlOU!0|7_9!)&By9oV@N4En(JAkI#FX{6QM{zP&e?9 zR0~_MZ6oai_Dt}gdI)jF+;eI2W|Q-lXbZ$HzGd2ol>EwU(mdL(V$QdB&MP#Bcz&6A z_o1VVxjkQP=aMa{2L*V7gP%quhdrxRST5Hob{OJs4J;4*7;VeJHx03a~+${iZJPJYYK%sOPevf#tngXo6A^AGHuer z2zR9|@suMt>6`J#N1?fOr;WexUblNWO_R%h{xd=MxB=E$$s=NysXPRpwR=i0SiWiE zWo!^Zo0lJmBx%i!dE74!;rXDuK_6f)L};=6y;Psw@ZfkM+=9pm4`+|7tA z(@S}m_m!tM{E&E+sJx2cfOSrjv7$S9Vn_GIk$zu+rNGw3l*F=n_5dT_k!Nyd`HG7^ zAxUz&Y4C<>A!7l}j`q{$MyY=6Bs3c!sT%AZu17_4k@aWfT#^;TvvlO#u4>3_kk_t& zR_o{3kYN$?q=yk$ZzxOO{N!vWp(An9|I4e#z3fNOa6x13>(MQ3!7sNudq1T0hQ?jg(sH}g#%V)&g8|dABSUwjcdt~pO}%IPJ}YO2 zJ@E%cE=yoh1qk`<2S(1S>!H3!Yv-nq*c>YP<}fR&>e{`h3VVf8j~0N@4+F`z5VQKK zJfQ_uNKQWzV$}<)6eUHoqmyD#KWR1_qY}utho`z*yg1`R&b4p$i-Fi(X+YPcmAy+uE zyDXzv@>(JPbAkuba1sDA(10O3KGL4r!=u1;w^X9LTTDNUNQVLa*;h3G5l z5>Z_)4ONRn?t9jjv+ox-E6%ifpj56{C0l|}i?Zpj8GG0|!}J|9Lz^s}PA}>^n;Su5 zH8EV?iiShLI#t*D{LHIg4SF9rqb&Wcu#1h|7i#FonloI~7He<5qxf;0)rMmI{nzV9*> z`(BIcETwV7nm(hm6@ppLb})`3lF;jorE=-PC!bT^R^aB!qN1P;XjLQglo?W&pyq7C zd@Qf)`b%L@oA&m&L@E=3Hc`hw-?Z^;>VU(TC;R{bm5Z$IJ#Py$-JiLmTstM_ z(V4HtTkHMHm_L`@@90b%LbpkI-&r0Rq>q+zrdiS#*JQrS8T|p<*##*R*JRlY?Ma6xi<&tQU-9^lNyC`AewUM6o;=Z6+3Lm2K;i;% zZU%Bg`NAL%wr_QiYY49@_~~p8H=hH+sG}DiXZ+87TI#d+D`S<~r{5FFx+$+gLlb?S zW<@+MTk|6mhKE_r*NbYz-cF#--V(OQJVb*MFRhx8FeQSMgNRXrt%zjPWPcY~qGh!S zi&`7f61H<>_qIu5=FUbJQNCpj-Mn(2H{5i;Lh<8#xzoS)*^Eb&@ z_oIvaL3Vv-OL2&BQvAlGN5pCX*3c?|{Qda(j6Cy_PtLl}>t&XrzsqS@Q^=K(|4cPTL6V7N&Uu^U#r{IEQGt2P?tmorX!xrEvo z#kSL2u|!?qggXD_wXvA4@Ow6{FJIR3)ArriImzMUJ1|CKxlX;yU zfy5o@V3q5Zxdb;^2by`?t;pO4G2G;G#YnH)uRCQZ+fD_dgvVAlbyp9$GkQ+T`FGy| zsJ&ddQn;RWz$(*PWm(f~M3wsGo6uEf;$5f&>0oWRclN1%(l8V8P-SSH#Q@*jbwa_W z31{=m6^X;SUoQ;je>+vo>O|Z}!A*J3CoWY1Ixrr}Mk?YLM^AmxidQY;YLDZCTk-Ph zqaE+%#TsS?GOXd*+_NMLnXLDTlDfqELKimYew3?>LJ}`@ssCu+p4wM2)TtEj>7d2> z?UTdXRwqTLd9=;t#Ix9X-lg6Aq_eNM6VHKQN-j>Ha3G)WWzz3cbZ>xWzOMXP7QXrn({Xe=L4*8ypQup;BgU_z;M&mP zz+Sk@_1ebNUh=Z51H)4j$-EG|5TKcN6rmPrmQ$!CHH@(?rq;>N1UY7JM<6=U6|EE9 zMppa1aGy5^b}uucxE*O`;0}Gfrcx>JSozri)IsdFGU~P;eJEgb3|SuSt(gA z`EG|Eg~T8ok@x3$^g#(BGpEaxOI0NpI(VVsc6D}OR#g(tN#}>l%dV-5MWEc82k8Dy zP_d7`0b9()Y~&p#zR5cKl2_$*z0jaR%8&D8xD>CLxWh;SCkBzBU=m@2l%5eT0EcAX zkyqXx(x+^ad^m~u)C*+d~LTchp=- z;p8Z~5ktFk6{JahOI9<2m}(rrMolpAf*YM=me1g&M?&1`WX#}+=GRAR3#%!fVY~E= zr7hS&?gm;W7LFF-Lv>M|#-1l1vwc{y}NsY-VVsjMX;sSmQRB42uQ1KEuFlaTvsH%}`U=^%< z>&X1}jGwJ(#MA}(x@pJzss|wFc@XCVbY;Hi)4PK!`7vIMpQh^KwYr`nQ!XMJvr|gN zMa2jD_D%D7qZJfFRy zK4_~vpMBKTQ`*wargJ4Wx{ipr^jXRKMAg|JE)ur$N{JU|zOhYrQaB;*R%@=bQ;t4x z+eElq2*3Q%zRbX!Ge=0kcee!dSk{9Xkx=&Sf#K;fyN~^@ZuS@TE%y;&x48c^kO}o zs;)|x*PdSAOViAX43508$C{>rFe>z6g6~|hm(WYNDIEOCKCLt+%cI?$;g>L2Tbu>|b;DFMhYRE_AJp21%N`LSy@*&5Y#-phC-uY~>fw(5i zLw?jy81+VvGozku<_u%igT*7hI!zXPSSs!XCwZ_%g$y zn3T{&gSU9^lw8W*>YihI!p9wk``U&ADfW%^O-sbI~RI6d3(*=JF%Yu>@OG z*F|ERs&&2AFQ4*?lsPAh8WiB0OrI@T6W*pExAE*?7C4 zuW&8rO4jnIZ*dEArh4wA{*_9hA3E#Fw@luT>e@UtE6l*CkTtNS9I?*omnvgE-X^(6 zC`>|>rGzetFu$iFvBz^tF8GL(mCVHFPmU6v;qwg5YsP|Kj~X{^l0_ z7w`v%5c-9GHHiF(@&{90+1ddJzY3wZox4WPW3E!i8)9jtJQe!rK)6_oG3?Uat^bTx6c@ky`y(Q~uIj@Y> zKGP`;Pd`;8cWrf5(TC_x3l3UN+rrAR+TN?yQLtE|EFSn}* z8>F=h@cgj{5a$9s_7OaButr$mSW(bl-U=r$D0iX|a=!pOp3DZVTOg@_&3DPT16KodV_vCa=_Z^R6UAJ(EEPUOuRRMuE zJEnpCk1;|zpdA%05e`3xK^5^=83H}&(eL9Agy17|g7dn9!hw^KC<~ylE)pe=LL+|~ zjxRtLY3gW+{tFF;pO62wK}G+|inl?<@r<(n{|1BM8DIa`O%xYKFi=DUe+>Q=U&g=V z&w}5_Kk+T%pFj7-|Jwgq4?gyWNl5aZ|fB1oc z@@p9JSImG0y+G1)2l>R(U9#a|9$EG(7UTbzv-pesDYp=jUvu^+&F&wlZOAX?-2Y^4 z!@6Nm9k6!;qauia-H-Upf3*8E`EkPpk@(dtK0kh+62Il264Kn<3=fgR;o$Cpfx@^S zh`~RQaFD;5Z~%Pmad>cvEerudPDtT@gYh6g7n*-`5qc8e|4Zl$BGgbHfK&ln3H2Z6 z?~}AjDZI&yzqFg4DML`le%Q%Y zol|d!`;=%7V2SExjnHS=vicMsOJN$2C&M&4yR4TFZ|;vK9~u%@=4E>tZF=b)1;gW; zg4Bs0k?lfFNQZFz~#kqoW;KN*Kq8#91vI5O$VG z6Ew)y!9wt4?Nh{|?j4b~C?z~!^1PB16byxdB_NUzaTrV#Dg+i60D}ep*(m;+el~it z0x2SJtD&vM@r56E;V(4g-}46JU}g@$DJu+sTSxzT0L4T_VWL2D;E4??0>vvi{sBb& zWdq~x2yn6fmklb0i{bA!Fc?2of49M)A~>D;%?5>v@f~*dOh} z#BiheZ$Gg=#|78yAJ-=WmHeY`A|n4BXGaGF?jFVA_+Kk%BVEn#@x|x7maVNL@OVKy jnNun#b6enXvL2^D+7aR4c$^Wq2#HAm`1r1S=pJQ literal 0 HcmV?d00001 diff --git a/examples/spark/tpc-h/dag-query_8.pdf b/examples/spark/tpc-h/dag-query_8.pdf new file mode 100644 index 0000000000000000000000000000000000000000..221b283a6da86b66e4ba8132887c7dfe05cc2344 GIT binary patch literal 29888 zcmZsCW0)X4)9%=|ZQHhO+qP}nwr$%scE`3oJ7=F)*Z1S}l}>l6ld4M6ol0_7lPZXa z(K6DpLXoy!*L^~<5HJwf8(Bf|@DR|;nA(}USP-!OB`HA>5D?IdS=zXmI{m%c7`m8> zm>S!gm_qUKK{>lPnHt(cdF0gUNY~RyI_|93pB?`iWqJ40h!{Jo$io$az`|&_ z?TiTrg!bz_j21U@Td(AE9fNQ@6>$|+Q&Te;L9c&87ytR(TIT0VaqH~0`)TIvd&B>6 zdY`QSeZ0+Y|L39K<@)sL=V#0Bemc6lYucXumhQ(5az|GfGJS98*i~sq-}?bw|B&^` z0}`-!-TFN5r%zX_qf0GA;{C$+H(2rl&%rFj?)jfzv9s+gW5c`c_asGajIf7VomEl3h~69a~?=+3VNj{Xbj2vh~jIC$HP|eILu@ zyL}&jl3(?|(fq$}lIey1yeG@+>+<``qVxMd$k~0d`juf{s`mrEYj?N#J`b1E^+Q4b zJbcRcf1R9e^ZPUNi{HItQTHmy_H0SDR#fZO@hw2TGN^YSrdp@Jk`r-u`ycjGtHJxH zj98ntpD>LV^xHeT6pbZ^T6Sk|yraIXr~6>AKQ2WiucM^ zK?7yiuxuPFpqx}6J7gm}HA%U(Ybxg#4<|!d&6sYq;(2C$nO6VZ+78y(Q+D79>D=9g zLZ3*MZJh32)vJf(!ors4e0;%0YZbFDGShy2U3;oiUMq8na&9VgG^k6mNw$Vk*J^aT zD6QvQ1{$-ErIRw-e(JM%hiXMg86S5h>H2Q8sHeG@hQ_2{To#Ot^0od z>**^~QWxB&oT`8B`YHVP>usxEGBXb&MJO0=QW=Ri{KkTo_B{x#K4C3Hm0OmFE<&iD z!l(Vp0Ll~;fnhLI#lpC)%$81!ZSMIyD=DeojyP)hTFX)UohaJq6S1w{<+&Cwbk$bR zn&h*P>d_2a{o*CCZ=w)Af{njzEz=hlo2N-#9U*%kbQxfAm$jtEV*wZ(bv*fpRCjz{ zux|S)aq=k;Vaf(wE89-LSQ{0Q&|`Jsk~&>_-bllN{YO?yy)>Lms}L0o!o+|2$MT^z zwzY%P!myM!emf&4S+TEsG7lZe%l5ZN>eYRC;396@O!WRos*lT742^g~w(a6rkE}mA zQOo*trw+5=E#62CCT6(iunaRTJguTJ5nCEZfv5O^~ z6+>&g$3QyT<$JS@wDUZ6o04CsW={@1-2K?l^Z9#Wp{J&Yw$>FwfqZS+qp?x%KW1$X z$=7e+dZ1=O5+yZ~a`lyBY_u(#Mm;Q9RCP~6VF@=Dlue2kqBMfRKVmA~YbtHh#N48>zFZqE3-){_J9mrGY;-pMjo224%V^goHHE;1Y!JN zZ%beV!djF_O_!-ye95gbCDYplj1JHGpB%MmY7PrCwd*hI+%$Fi^CTglUu;Ox6JCr9 zQ6VA$C1XNM>)ZTDOiRGA)@q-s?HA5)@j<=uoKg%SxOk#=Fpk4cM5o+*?!pXoC$B9w z8`)VV>V(9&qs%rin3h^j^9UD-f>8h(AgB)oQw8!E<+dESlqV|-FzjKbI}%Nv+43WI z$Nn`KfcFrlH+&Ns)8Yd+;}aVDf0J}sAr@R~1WhQhuWJTvaJ;IW0rgVP!s{oHLqW)3 z2Q~PNNg=L85gBcU17{le;9Cw##9*v7XV!Pclmzh%yCk)%FsAvC1fFxt1)VQrQI=(@ zkHq7RN33{B$oBlA-r>Tf<_ z>Ba3Wh?^`}vBj(*p)yuL!}iVb;5Me4YMXm{stcme+`OW*V z^~sUR^CV@a%?`T6e_LuLM)Y5}?!%1ZPqYtSlex|vy_)$wFz9%{-FKYVa(EvKdpZ!e}rELT{ zR~sf;5^?mAX2NHY4#J+pO_L>?hN6>JgB6cjM%p~syqJMXPeuhG@4j?e@yNmp4lzrB z2nLw=pfd#}bddwGM(GJO6&ky-n23b|xSvwSd`E7+RM?TmqI-yl=Jqy;4`z@^&`el z?~TtppP5ZP+t`=g5@k0{o}xC_1>D9i&51vpCK@A)ZN{yN_Dx$&lVwbw&xnnBxWjK( z;Nkm4xO0aI-V zMaWTI6XMH*jq105>4=rgjdG-2=D~M$9<@<@{)3O}YW($pE|6{?Ua3TBWSODZ;d@of z{_$~iZgOOX?X4wB#05&2fHV1OcSW6kRhN5*rQ;^j59RpecVtdz70jR{sP;f{OF2I+iGg6`mN+(aZ|8h!W{%hI6{m70l&BX(G0VkSTfY16zZ zvh-)CI4OQtgHaIauv1J$@yM8sl@#ktv+!p(Rt2Y-AwD#r%ZBbWDsxcrq_3FMSQJ41fbrDfOdaV@z(PjM!LW4>1n-ccMh_XA!Xhv2 z-DdOJCFlP$oXk7{-*sfC#=mb&@k7et6ta2UgRh_=zUwJylSFhCv`Mr*`>votX6+QW zZEOQuHG8YAu)QN-guf6T5j1v(;soB$iK0g=da6Gl^*zR+NQ!vzw8|#Yc^T8>L= zs0#hEs36$~^o`D?x`Y&;6OsXp;;iiNrF4PGjaAyPj#F3?&W$UXFAVrdj|lhRQXZ-{ zzYgIV9Scv_980m&!WR``D>tljXPR^x?B`ihp2xX-y0-;q(5S$iVR|g@5aDC&13W-< z6m1-t10;oI6v`a{0mVH@i5FM{Wx19Bl076L6vRW8W3nQLI8)RQw3Z-Nr9&;oU_JiY zgsR6{FJY|^mC0ngOyZg@TnfE%;9-DTkt*S%Y>jMn4obbjI_1#Xc!Jr6R8bm_T*3OU zDJ4Gwjbnwp2fR&gct)# zn%=<_`9c^_hdReGKfQS%y?gYms^N(L}m7`kgS3B+TqOyY(5)Zq0{a zNKvwzlyWBYx+-HWfE5n_9_6ZVrKE6{e^I(TJ;0EL)`|2`kO9Q0B<@{~%GOqaVC#?u zMf8Z+h6XAgT8}VL4OH`DBT)>O{jnDhnv}@xKhw(XAA@oN=^rC{I)@K^yomp3dK(YR zmsK=>wOm{lNNvt!$N;~5y?>P)dJV(NWCWxI9(ZO=<{i4V7EFQ})I0hpqkTf6c^-4m zY|91`$M2j9oR-|&shMoT&9{ZZ=qV>tEzT_QXwl#XBhb-YT~n*ecZO#YKjw{sK%XPl z%*$xiK{`f_t5>??$?~G`XR(Y$W1TYp)k5gboG0t|9Ko~unt-fnXYXOp$cUrh^$Fut zmM0O^6N?1l4<+tfp~U`Bc1Y} zfudzK?!|fMp0iu*YWzTWHBLNS{o}^tYMv*bRApKH!kg~CslMy#p2gQzbKgxrM?%p* z)6iQCmJ!wEw6u-6l9BQnQv3Gd26BR2QvqwM09Z! zabc+9le=md(T%muyzurDCe`c@HAPPK2?_MYT2APR%U|TbGSAKU?f#t4_k319@-O43 zDVEvHX26rb{_!TKDTZZpuatVWATYuXQ34;CMnVT7v}NvrK1kQu=(vq5{HZFqP0l(E|es; z1W}=M4+{&OFsfIITq2pv5Pc9AYojS|6LkL(QMva#oq9}Vd8oZ?S3V?)@Om51M1A%- zi|!$L=RA2mhOKHj7$I>E<~Jxj9q!#RfacabR6qn7dSL8V3&Bt0bOlqGMNujH@Y~(y zsC(I1Q9EEAF+jN&ixB074ZxjzHFP7b!k2s&A(t+N9$eW5LZ<25sV4-)p$Gt|M^hfB z5Z(dneLJ-V@dFoI7BSUEM?tgYi8C2dw6z+fLu8~wwUPyCB!{@ZM*L#~AJvF><;HqX zK@D*O1hDQ)j}(Ca^s3YnoD!1SWV>+@q!|D*A7v-y7YSMVVwt#i*2yRz*#{aBB^XWD z6aBETuXoUdY4Ky6u}d5yzps>L&O{>a&=>1evh|@i<1lZ&kLl?#_4+T5nh{;-9&1Z> zbb|*9rD#2vQ?x?SME;9Rb{@{rG1eQ0d)`VXlHWY^TN|w*PCWe_bnKOn$4+K8#v!&6 zlgL0ZKBW(0d`!d?-*Tz?m=K6&A<%G|5xmfBzfnqUquMN6!Nz#YX@G@{HNfIZ4N*(X z871#AyRx-TR0V3EznKK|kbC#S8p!&{*Qe3X>ShKxQ9|f{yXdk(T+(qc$z6zgSRWFe zH6Ed`pb>@_E8y;y4<`voM_1Sk8t0bpK#B)^+a+ZGqt{lse zith;Ns%_}xp9%sA-pxHIYnB^wO6IGpm(HNwVo7ihU>Xt{YMH=(Ii(UXfZ%0~hVvPW}UXSwPN#SKl z`JD)`y3%jWR)Jaqqrn;B|7srDB4M8bS=UmLP*gB|*kfGgJ`b6;3KZ zu)PXG48ggQV8dx}?9md@2s(!j(LnLtGVzBb0CX=`OHQEhPr9x|U(+|TlkvyRxTH(} zugP4VyG)*1##;syr|}z7MBIf#eW5+5(P1^FpuDG$pcHxQFjN2t$H}BNft@24lqIOO zqI)oE_9iYsT0XXE0O;H**CrwaDf9;$4$}iN+0hns!6df8O{2^Dnn$2X-ZClT}6^xCRAlIULulvqEl_I6u-zCm z%etNhvvZ>he|;=&^tZy!_z;R6*H_PtpVHp=yN%{1%gg1*yWSh@FP?;F-|*cIYukHm z4Slu>k2+^K#YRPC^lq__1wLm4oIs5>tx|_n9a0AE<`R1N6G?0UtISv={J(~^5`@_5 z#s-S-(OiU}V@UgwX!q|z78YaWa-X<9T@o@6)KBCZi+F!ajUXorRrhXF)C8&xR8aag zxX7RwW(teipbmDbk;FG=Bxy`1JZC33r0)(VuIPA6n4C3>j=l+LOo~ zUT)ds+cyZQg}A*rL|!m(hoXmHNIpMNYex4EQVuzSfFi*q=t!j*abIU?$Cm?uC&NBi zoe*yv5+s7B{$kQ>;q?f76|#t%#MsvvH|Pz_6r!|+X@G7*b5y(?f42)W&)=7{@t!lcBRyB`pTe_|l}b#Y|o$vIMWRiIhF;a9Qb!QREDS%~;dS z{m;=10&$axB z1SS}-L*n5cuyzAgxI%X7LNg5LPNH!t`sVIM*7^NIFE7H`)kkY` z6hvDjzyZP-4VpSFkUmjAu=9n(FNm9@2(LvX+mIjW<6=b*qWsa@;nSCvXawjZ_bo>t zU$08VWEk?u(Bv7M87QyyMYqsf0^7{Or&Vu8!k5JPT@lfWk3Tg8z!%NE62M9E;vHoA z&j#pVNg>u}g+!vc@SE>5#EutHX!A=>-h59MT$Nttr*=wOJaGtG`)u*hhQCHT538_e zP-LN&a#?b@dkg$F3Ra}CYjh|w(~cL%Bc0h#j86M#n0fWx_~b>sPJZQ5bGQ5LxPSLt zm=qrj^s8(}O63%i{{SbNhv+i9w>HX$iQRNUaFl|^aha4jlKLm;poWB4&qMA_PAaB` zo)CCM2g`7!uR1{PPzUSfo3uBG7!X7%C2~YWTy+*$Z5{l`S5?=>4F9lSk`WCBsyzSf zH~)VB@e=GmDR|O`R|^rF>f-v~Kulyr=Z0|2Jojvl6dvAr{mECK&P#~~Bx9j5Mb82$ zjk&@}!lJ*)6D_f-L{f7?Q_X@l8ze6wTJz>8=sxt`B&rhNyfCn3st`U=eo^=(14VW~ zql+Px6STT;T-~X^F*Z!(>7dHj#OzqJmzu5PD6xN)IEY#=?t8GgRQE9<{*itX#&TCd zTyYfJ1Ym^=*hik;VT}v8K!L?#ZkvAIo5TnMn?Bj)rB4b22hfxf!|AtP;M=!l`!w-J zv}glT)RTPnT~h7IC&zuGBR!26lhi%$GlpFe8xIE}hxOOO6}L*7@x~2yEF<~4O8VyC zYuL?VEl-_%*C6xbcg%UF#CCdJ{mJ1cxn+2|^6`M-Ef)Kvdz$GCc|JaD);dJwcWdMc z%hRQbFEsDIqNuY8>%UDS#&bJO)8>6~nSM_LZ}(}K zW&R*A=pMcLLl}4a%NJ*tS2}pspvStKqtE{DX>C7p|63*$bg7eKbcm7`V6q;^E!m<> z!pTJmiXQafi~(`Ba@@7>9D%&O&#f>27g%n$e{(mY#w|SAv)zY1+m)gYMkwyPu_>d+K?xh28 zl3cv*QL_~Z-{aU=F){kKLg~RNg;)#0aX|W1sZ6obc|1H4jXnKZkqE0~UNWgPpf&#pj11KySoSVv8kuiu&VXnP^I-9#|!Dw65 z=hcUPyWjo?+&QSU8_LwqjaCT-3uw zT-oI>98SQ)^AF+gL2t3{MG=v6!&ObO@}49!gmNC_DKrH_K4lj&dfjQ=f5 zFKcRIX((jxL7@GY#6ZBv%)w5;$i&7$p!=`}>?76a-}~AtWiC0)c|j zLR#rSwy5U41{7gsC|l*FG^I0BNqc1|(DWAAo|cid*BoNb>2<%??%UhfKd)9kbIvce zd5<=yF^FOim5UHYi7hV|tHD2>ufl90C-kBbnw>9Q)}@(G-Q);b8R8P}-O}Le`n}I@ z#ympkzXB7CZa=E+I#@i;{r_NTc|aAujMI0UJa@6*eho7SQJY9qcf%mbpt&(FUT`S}@!#z(!gW-Xk}K zfpE^1uYUN7DYKc*lpPSYprgl+SduR^{4&F8JQ%kDx8)zJuY&X5Rk|oxMxbK2NV^&gmCjS7$#Nr`OhZWo*KyM==vO4!R*b?vPB@o-xcE7LVMWg>?x2 zMUj5$absKyliB1ZGi{;mLz^4NQS<2Fv#6rKyWr5ntlw9{yz4tS)vL8T)az%rA4=_= zC%!dAaxm}?oeT>L+e^zkPl&D^%VA5;RPH5>Y}F}PH>{_ufp0jJ7r`|n z9s?E~(z8Kom<=TaM;bE2r8s1V##22;M_3NT)3n0j_u&qWH^$KRhHx`X%iH8g+??32 z=v(|pZ27q@HZB&gu)ZDsxm5>*ev}xi+0H>tciz4z)P=6o%R!A-3#H)1FpH{A=<}gturLl8CeR44J)4N#O)9skMb54HI0p)8;FGLhz`DAJR*7 zadr`Q@lA`)r9x9c>G%?-+b7w`<>UZu&8Yf2Z}49w9&g7T9vVD4sd4j0b_p!#fnC@t z*{U&6SPpj#N>N>6awwi$WBxbfju2m-{DNjk9$824->8g1gQIW-zA00-nt61>xgnR>#*1EE+f=~vVBpPHUsbkT) zXVr{hO{y+0FD{wJwjdm)R3bm{b} z`XsxQv4xDMkcWhaV<=e>2JzNevi#1~i`cC!UOxM3{3^?ikkq;?{y=0%{_1;TSpjO< zz2~M$E(7ew>&8aE?TXk&UzE!e!+qUjE^B5^Xl+mKb2&T)Z3L9Z|o4bmd_N z(}5`>zO;p;Bq?>0Imkr^kqE%=!vSjbFjAdD2?+?)%6;1Q0zQI%SdR*LSU{{-#4R*3 z71#(Y1ZHnDG;`}1w1MLbuWq6YMl#HxT{UPD27XJ-^N!bzQ~oZ)MnR{J11_5y77&{o zso9xGlHC@z)#WxS|JADJ>r>zRL;S<{=)TulP&!7Aw^45R`B#i!;rau1ZSdx3dhg@% z{?e~D=(lSmws@L}X}he_prBbJQ7vEs!X@Q`avpVi*=Y!A3fSia2t1lqzh+RW#8(>@ zEW!|$n+C0ty($(gfkKZj2c8{p4w1X6`ucdH z7w-GXmBdNrt1W9>!EQyoMiU682r?IhQ9RJ4NE{*w8Y)s#JxCM_0nVadwMQxLj=Lvg zoM|qlA(e@}7==7L2d{%Xk_cwd5%Olx0`|m`UcQO5=$0mk_wjgvjW4)2m(l9WjYATU zL(_7~{l)W~0bn%t4N&DrlUswECh}|YVx@NH6{r;5ugWCU^`(>2R7tINJCzrV`ZkyC z^)59|l#JB}sZ@g14RlBCjP7^Zt=prtJGTt-TyPos<~vMZay*k?xvbe>DZoPnGj`b* zqQ}%>$WnpBQoq$P)kNK(;JHhS z*gaf+C4t@zc4P*yAA^?BXdXuY%IHBno(I+L;&1h** z_K-Ia6+|;26Vl@#yBk9Gq7H9E>@h-aqO47S{_TZ!F4|jSE`QdT7b5~{UCB@TI1(Nf zjxGz&8oBH%>Scu zcB!gaeKx~IYODqxhXjnme+dp0m)So2QdixMY&D=@WzZZ3oJc_k@pXIG<*Kc2SNWvv z@AX5B|NXoNOfPRg8c!{9y46M6&gAUpcin1T-){5CJ7(`2lFIL0)JVQUunWXA*EQsL z!cK6g;@DdH8@INQNiJ5##1u?bqdSSFfrlHy}KR_f42%;dq0lcYN5Q_id z1M&WusGe)KrV2Q$VnB7nr>rEXvO)V0IRl=?!IbR)Fb2S7vI6iaQLUwfuS-QlYxTC&zK6Nc@Se} zQWIG=#kvV@n%?!)MDPJGVW?Imo%%SK6_~Y{m51;wNZL_tL$ZI_XXYP=Gl#eKL3SZ~ z(w);C_&)p(hC;?;B;UxNieD)kk?u^~n~0YY)+4+Jn`)iRmCOdf#b&BnJH>K6N-tIl zcMWXOlg5u+#y2B$t8~x7OTP!Zg|U=%)bq|HXzI@EJNevv57@pB2YFfX8*aT@7z?Mu z^UQrV;*~Q7kvaI{^5o?V(rs!S-g4%m+Y@J)k||MEEIEWzfI{Ja zKuDEQRSOOVi@NOGxNwnex@{UuCx)2#*$r*HXz%p)k^!(Y3}~Co76QOwg2NPntuqE( zo0u=0$i9Klkc<_15O8DSL^LV0;+PiA1GrU;E$vTh1lldE-bjAuE}P(^0X4ukQ*P)T zg2D(kDaQ-*L9oagC41=PFlAM=n9m2r6~KW86XuQ;EnBu-Bs+eS-8<9+N8;z_TwsqK ztV~uKWq?IXwD40_y0lX4NOv}t^PHrM5Z6Cp1LMt%gTP1R^1&;mQS)?6#M=krhb2x{ z(Wm4`=419#oPK!SGXUKqA>|r-#9$hav%sH;)fU$&zE+s4Z~6{4O3_kKJMx4C7zr@S zQe{A0OYmTLF%+I%l2XMq>NuIH9?#LFt!jehbW0%VgrIS(Zjz9YRw$dc7R{$DOOF8i zf>%I*&e>E*XatKEfI2MX$a2h#w#kgP!H}H|f#_y5K~9QXn|#9 zD&R@a>3?3!cKcYN=lPH4qMK~f;n~jAVRY@C*bC4T!bVNwkvw9lsZGXW zrshmPRo`V;!CMGux2XS;uz?GSLT;&&3O82;#=sGy?BvX}vGa@km`ss*>VFgfU<%-| zAJwN{8WWz-dy{!V(i|^3ZbVX^h5vULj(*$r6XI_bsV0&{c%!cZvWj#NzD7X~b%$3_ z4n6QhI0JQ%G6B=nGj5xpTIxVW68IgBtm>^62Cj(}@-}=hZ7zAI*c>Urf<~2-U5F(l zB?F2!lRzSK?8;Pu{3)rUy&`S7H?q9qQrNmJwZwG};8n`R)uaEqT>K*W3CV3uE+kpk zn? zvYeJhsNiOLvJz+I*p3Eo0p41?^?M6hE7a{5+aKJAq=?j@2IH!u+{KHJI|Qj$wBcCRSTwQt@--O>~LY>1bpcw*zif ziyt>aN$1FqO&;=`QhqdlPnl>C^w7zKZZj;cDNw^FN?F^ahz@y|(6@v&2vB<#v$ojC zs!0|$F-ek9R~AS$fxUNV5b0K@s*W&`diRd7XHoF=(0xh})~0P=4Zvkx6p-Q=ZIs(c zIjB?cW(_mXKBXJAh{Qpu+mMcXHDO?6=%|6;2zSfeIYX+vC_Stff}8$3x7WVP&P^D_QcJ`v5{ij9PIe(3Y{_2u}jj<$Y3itl{i(#6l^Wl zfq=MNQcmq)Dv+US(XzF*wc)Ou!f2f*dc7C{DjDxt6Xhjbe5lw^4lW;pnb;5qO|tN= zSq9pMhI&EVe@$wIC!agDR%z;^rsN9DF%vcQEdNal;!zX7;B4~z(Dz^kVIpN<>|l&WbjPErLwH=17K{thhgzE@7hpFtKl2chYUUZ% z6&9u*X$5E)e5!;gY~G>?iOFjg&x8s4iwm9(dQQ=Vw?eQY)uBwy_#pzXh90;+(f$De z2sStX77*`il$wePq8kfrj5#Dc3^E#PJ~JTpps#4As~^e?3d7Szr(~~yPn&-57bNr| zk^_uMUyblmfLGC^7ere(izaK&)Q1z$D!?j$DuC+G5CH@~#N!l^Xl;FUHTaTztI{V; zDk%+shlr$OZFPIn69t1oUrE>8k56z;Gxkd_C zS=iUMo!HN;oXEhdAisim2z0(o%TpOKPV`vM|0! zk$qz(9YcGD9|RUh}J9$ExR(viO&0*(mXmVc!&A1T!FFz7Q9N05jV*{8wg~ zx$fzX>Gz;#(Cv^%;Bu##==RP#qiZAG{x<#BO5s87nz)MKR!v+v^yo-)$J?gl7EC6KZR-LXKO@uHbd6v9;#rVijX4P|8rd?Qk9uR#3@03-6HB&Z#AY!wMV$6| zEcmSW%#SC4^U2#%K4BETN9Wv|;5}$<+&!=7ha)W%a)N7)HZA6e9LQI?MeuO2uYnrk zZ}PFM7A*>iWno+9M3}}5z!v)I@!UHe#brDU>lm5=FG!}79G$(_XIqR@L-$s%}OtqDtyKiHsTI{5AZFaGA1XYr&uk<{ziTq^5qURm0`$_ zJ&bw-K{M6$gvo{xyQd2V43Ahe*c}j{4ULTz&6q|ttcViYJb_$^yI^EweYymHPW~wN z(z})68GfiiLhTe;y5?KP`kU<&9kAud^OO3)%pERhY#j39)+(u)oNC6HU1VAoW<1TOnq@Q41 z@d6Lb$0LzAuO&LMFzUIS#-+n;!X*RaR!*q7WJOJFrI3Kpcl!+mddNtBoJNs$C!NZB zgBS1Ad%!gcfdzTx{-eP~g)8i@QgLzb)AOrwW*HyH0p_pw0gQKB_6(HN{eEf7x|h3AERhr+|kDt6NuP43=;Qy3SZ zSfV!LD_+0At*93H1Fwk}1zU!Y3kax07^5SDD3JX4JJn~>;0^$OhdE3(g_z097Kx%z zYo-M@HRA5G+kc;cW8Ch31XqpV%;~!CfWq5rJNx<4_MHvHGyLpiYxFxSlAt!~ zoxZx(c)oS(w~*Kuv>lP%r4-0pDquD{s1-v2;wi*Vl@zncq8Ff*1GI90&cLaOz99_K zqc(CR5>XXNB-%iH9S~5~E^$KWla}H%U!(=2ke3qNjR7i65hem7e;y2k4V(aAQ)~lp z@DCX0DQQ@(mxp_s_Fz`5U=vv5pCj*{Q=jXX4jVeHp`jm!L&j&Dj&ns?83JA;yS=!( zr2);(js{%S)Sv$SUBdPD1apZ}NU1=^l5F`?<|QGEN|mCb7({g(fvBoDByFaqqK9)r zlW5XmhRPmJoWdjRL5;-(ar9o|MnVyW*2<~LnI!~VD7bKO;btK+K?uq<93#_%Q0Szp z+M3eFf@WyS1rcl*WlFNVdJXY0lY7dAF~5=+BZ4cNRKGy=awSXLkV-RTszs_%9PT=h{#T6(Eg=|d;a ziMNg7h6Ga<71h}cXnE7{fE9FnsugLnq@IB5za?nt<^(LGEFJfmIp;Sj!rS(8i3m+v zC9gC^F3)&XB2jrl>9TbYpIs*-vARjf3;;vEOBpwF0_IGN8A>WnnxzMb(K}53JkBh` zI}Bbx$Rx}qstselVG!%YSd+#K`UM5D5udO40`?;)xkhgejG8EQ)^cB!8a$NR`T}Nn zkYKQ2ecQIslp^3Ms0{*GDC0hBATn9Tsi}L-7v&Db;e8WNx+{}wJQ%0eAyE>TA~K}t z6NE2BI6*ix22PT4E6sCSjH@Z1vnIS^aSI zz9xauS1d6x5cifF=F}U@pEc<>&3aVdThi#XX|AQ_?6O%Hn-gFJludzGkuEuDQR=Jh z^J!N3w4zE!0F@}$nKDx)Y5SwoY&6?FF}iIz`TEBYQsS#V z#J^j`BK3$84qV#DEwy_3GRLwkYdWQg+~uRzp}a04n`DwMqqUe}zhY0;)?Zx^i^P$- z%_IRp_{uwVb$fYBZS8rkqZyno44@+B-X-Y7SGWs6Jatc(^3VnDz5!YTr30nW`8f8R zIB?;_h40s)qs(g`EN;o!+(1ko^n@4>xOldrNd=?cagb*!zfGSR^-)1gWHr@i*!zQb z0s(^>_%Y(&170p(z_X3}+#n{LgQtt4v(i7(TA><0M4i~w=m<YY>Z)SP|DuDQIu(&0d9IA3u+on^^9 z1-Y`C!`02}Y>GCliM?i4)zyq8O3=_r*Hk(u387NIE^-n`RPybaL#WhP_xCkFZ5qoq ztJ9|2{3=Yy-MQc7%Sd5{X1ULti%w3!fWo8?X`$N8uE=k!s{!UZ$!mtu_t7VE&@wX+ zNY6666*MH%Ei!GRC^Z$5bX}J?F8$W&0wjBv`h4%ODf|2_3f(1>;+A0sD^YgPN{huI zIjW@`Ve#qaWIlsHPqL6=R^m`2aDI$-Y4SiBA+ve3C7t=*Wc;|JaK4 z`Hz@opnqVA1LR*vZE+S1jVoyJU?CKCYG0x15=ie*$X{gHs}wkHT<1PdRLMT@jvyb(8OPS4=4qvr8!QS9Dkw9*5fshI+kJ0Oh~ zVY=238()gwGx}8Z-IcbB3PZ~M0PL?P2kP!->P@dQKV~sIr=FH3dbha59_r4m*x}{) zIOlbjMk%N3%!UIC{xdEO^8o#^MKsE4LrWVtULT6Jbp!&l$SKO)Gjb%2jLf1E%_8vp zC7L`W7&45>dPd8NE2h(jWODP}W{AqbAgq{c>(38U6saPJe*>5th!WXd5haz25>YZ0 zRCaRm!SsNG(&;Zv?vN_zayW#VN;PCVY5Kg%Rc9CfRqRW%Xfp zph$7Ie`sg!%-p%(^L`0Q&RKh%vt_Nl)*(;MZ+97SnsJ)Jgc5Id=etEJE^qVdG>zjw zxpc4f#;Q><>j_(X>YVwinKfvO-o=#;goW?fuU+9%0LI>`C8 z^q+b)0yaq$l|{-Thf(oO$2LA6T_s(nq=qy=+peL`m9hb6 zNiSX58>N5Bl!)26U&fJZ7@kc|mryO(aj6aadOF9kQ}lAbnB;SR;W(GKEO62tZe%)g zl}2?vIeLYq`r|#@jYrSh()d~JPBTaNqK5pr(qhkYcct%NSD%t13*~TOQ9R37rBo|& zd0v_*iANhWCtrt$455`lv?33ZE%Q^yTX^y=wHD9hd&MA1P=Yx;*TVSu)0%G5jLB+j zc?o^=)R35bof*V%48i@OiM-OW%#}zr$?oJ!0`<1x$(BBbWgDC}i;>`w)}QmLa`xDy zJK_mrI#wy1*0E(j!j)8wdaQ5f1lHrl4xDTMPR>Xk&t2_I<-vo&iIGJA2{!bp)Sx9x znN}*uu&zL!@+69~Lu8PcxlAkF>cgk0H0vIfI;=76U=}0JXqytcLNfLB7BcdQkR8yO zt&PG9R4)d!(4_l;p94Vy9kLFEE_F%$nkH(@8VVXzl^aiycVc$bmml<6KQs!U1z|VS zQH6aANe!0!A*j1U^TjdORQYb;(h(sWJe@F-WVs zMH`p?ONj`sd7*Ef2BT{*tY0(6O_SG#e{k`9WNnl7H`)9|m=v<3y>``jwiKgUA7d$? zW&tWEe_gvpc;Ky)*(=ZQgSU~q^DPB>yFkw~EjK>`saD1%u}UW*z!Y*Erxm?uI1(vd zMRA7>%1;RakfK75aQKrnzHzCj%XHgxA$1BbR>{p?8@)Jrfwd5xBq0nUIZUy7t|CQ- zK-^PGB!vfsWo21!Sl44(>8%kuk5$(VmMoF?ol&heHS|fs@9fkJ)Mx)wC7{#-~r`jvH?3TX6arS*!PTzJhjp049gVOTKZPKpJ z#aWd7j{We>*j?F;-bHVYgp}<=AF}#W8Y)FKwbGjQ%j#L{qjcr`xd?fif)N5-`jW4y z@0E0OMQEw8-yxt^P^i6gdIzabgxOXrN1(CdP6tZ*rYs$Xj7h3?R7@+O)3&<8p?~lx zA!{oU5|zHoy~YQeWxAC0l=%~u&mrVT8=Utp1EpsGaX~X}wZcKFQF_5L5L=rzscui& z&I?R-z9k+v-)l4KPYn=#yhY*{)Ct5OkMj=`8hM_AHswM;BH`)vhJ(79SrqQes|dmv z^WJ$b(qd_0Fxg>g#d!D*TF8E*>yU3BH~}P^-dEJ>2Tz?r@E8(^gsm$pFj_piO9=+n z8fo=@D)CynPwam(Qqc6{OJvo&*^J1r%;rYO%5M#CeL`_2Id4&RBy9SfF;$Wn%QZew z$YV$NYNe<%Ujobf z(^B4BrWRozx~+*liLCtYd_mS4t9mT{HImeH=_kG48k`o4QyE8f>!Qc_&$dvQid<*4 z1rkK?SvXmOeL)QPY%zV7Gd~C!q?`eY2rkb!Eq+W?sGNkkt9o?iI+7Oz}cl78xmaoXIfs6vF~E-qkm3l7iSW`Szs`=Z)#?liHi z8b0Q$Of-oChoi5U=11Y}>HG)1vvTIoE81N@3LMrO`uTiGUQA%&fmtjcHFY%~nw(W% zzMg85FV4E!8#N5@Iwq%!i`xrwRO#f=7rJwQIrNNuBgaEG5UE$hq7lt1moPw4Asu13 zBhJ~h33PYSh^oMA@rCb3QeR!4X9wh_tIcCMory{8N=&x29#581qzqfx3?m)5AKNs9KPP83>{t$3B}6shV$s_`#Eq=Z$z ziM9ihwfbIj$rCIIS$%c+PK4)slFRTFGeg?xG?)1+Zw38jhv+i|OwPvXM8REjkatcl zacvL=RtoMMeg(Fcb71wj_c`s*JD&8Qbm-5|(iio3NAiClA6}(-%t>In58pC(Yd}IK zDZ#6%Rmk|`-klGzBErQhi*kc85 zRQ$f41^QI%`D{C{;!ZZaU3M4zd7tMb@>0of^r-HHbh`Fu|A%FWkZQo@Wv4oCai{s| zt!3`(PEU2*YZD1bfIqHefdmN#QQnwz9t*!z(e?n(?Skdl#|qF0xo9;7XMcgnajAy_CE% z1cs^=$Us1h-Ng3^2xGM-_gQ&ODOQlLhtge>PTo?X3;vyS%GHF!YT_Y!5Fz)wEOPw~ zzKl>cph3tx4T;|UQx%B8TbdPXd!%=a>ajL(ZB9xoU3BI>lYuG-RGe>Y;WsiORb#(I zge#uoVz|BFU-Ah4d;s~8LF+UFQoqHC+qv3!-#3LOV@A5w4PCfj}KRDgnA;3lLh1M^Xs6Rzty#k(-| ziK6F(SF~>=wt`wNbM~k>ky)R&?)e>S%76TML)*Ngm6)#ynpW3q2o)vtO=@$B=>EP# zTs5cE2I6S6dD5WjTz*#`|2BQ!)zN-_)@y#vai(o`rrtA{zi`HJ`M|UOd~T}qQu z_oYedJcG7vcWlvco9xk>QROsOI&xj&Z6`X*n0_X*Q@%>cN|p2>qEsMWZ!wS;2-bdM zZhSn&GDi{g0coS6<}mvZ-I!Y-h?;K{A32N{M#7wu*_I4_54A<)XC?8j5{f#Ul7@S% z`wAAR3V+{LEhTFH&Pz94=r>Tdo0xD*EF$eFLnx&Fn&=McM&)Lewc>AG)*QAXlbwY| zZqd#^MeB!V0OW?Ay(L~j$3yMPr~Y7_A}MrOYlWMMgb0q~DfB4*GHy2~`Z?%nc3Kg5 zmY)9u0Xu{1x1dnCpq!t?tel5zlijcJY9n-z%Vx%_+bGh$-7vdr?hm`PH2PNiez>*b z)qMYi;j87#aTgPK5H6_1x1jE6tE@pi`IKHwwAz_fU48i}|AGCN{nL0epH#Chr1{)^ zE^=EsYNb&}4SSvABaNf@^te*BO?}?1a@pENmTyBQ-O(&vueq_qMxXNg&?>?yCi8ws ztI|8e_)LXZ?ajU_l7mZ+!>s{{N_amJ!E(I z1hzXbRZtuF<^(;p-NKOB4Oi{m#GNlFCe$ZsR)^!|-wxJEx4Uk-yrA;+-+{x{!D|)~ zvp*-oCJc%uhKzU~xi%Ug-k}9)jo)k|a(bt#YCMU$PsV+*!MGLGTD)dCTD=(a!Bcq^ z;VKFnNzQ|Jn?a7>Pl4R2PvV8A6{}uLkKwfu*@@*BK(=<%aoZL1@cFP2s(DP|Y3u;ju2F^xcCTUNrfYarKJRxGLny9mjP4tFmE^$y^{U1^x`& z6-A{k3C@hau?hsOqR2Bt(Ow$v)rO2d%3zo@z-dL{Oan6K?TP#6d_#5W ztWl7cHq6;r5nbxzaeL~!@CwNx;fjmmU6D)8{lxL>%G?Q@ees+7h_LVx=2Z@Ui>0m& z$*bmt^bc(909|cub$**gd-IAB<58W_Dq#LYN#22(^T8N;jo*3##mA;4*?lq~t zA&n$yIF}Zy(()L_yGFD80!JnhMOAnin-IA<6b8lNxt*k1HxRFnAFYf*u5|KqyHi)( z`b9#v5=z`-8KHrMTnwx5V!krcsmX%K+S35WNY>{?lVkJJ0Jbu@kBLb%j(kGAyk3&l zEVBa!MA&We@hauxJ#rHh&)xeH&n!vt*-b8+bwNcWpPq&dkm9=ME>=_7TG&uSALFrUZl+KEG= z=~0VR(TXnofY~Ujrqg47DdzmH8?lGbGi=rGNL9IJw6XmmX@}vAdIGI^fbqK3KY$!O z(rz1zR6AUBM&|1++!m_e%c^`DEf*&@)befR?D|%Yi^m*e zfusffc0_Oe(?-%`ukyz@xC)NnMb!hzDq@N=z@M8mQpCt=nZoJYBsya^cp}^VNzr>T zUMFtwKxtSc6ZzXp({yG}c%qsq=5lN~ZKy(-OTOhz71q<~Y=#n1Pf~s32$_>{85wb) zb~TFln$t5Y!W1R;jzBiL-`9o1i}T&K!rH41t0WmLPO~S6XE$&gIV3o5Z5y#CEN2wA zV!Co(t;o=#Wpi_l`+WJt_#~afSSb)omT08)h!sp-(i@tkwYj0=Gz>RuwS45hDx!Y8g) z#S+)tr#}r*(%VIDxS&*2+Pe5QObfh=vYsoYeI9UvO-vLplSc@L>$?@E$rEromhr)@ zkJqD^VT~>4DjEFAigA{z&IZG~-@FoNqU^l&u4B0_qfRF#?G-&E?^b2{vOjO)c)2)@ z9oM|=;dOdJD<&2N8Y$O#JK|nT&6NWk$aOqfi|ExuMFAq=m2aRBy;wAry(D4_LRIhd zdghV-;Mc)0;{ge%1x9F_GJK&(OwUx8jU+zuv? z#(ZX8@N%&4x^1lMfkAl1FtU$wMPN^`FHULoVkJK0#;CZ!N!FPGZCFnk(wmWYcp-U+ZS`p zix~x8)A5B5C_nKg&dhzw&57d3tb2)btKjCdij$&JKy(U!it!nZ9d=+km1NENAFc?w7+Cm*NB>@K(s0 zU+vlTP_LI^PstZh^CY&BdNRx4x1W}tQQCb;4!b}L5EI@2HX-O-qcu@V3dA#})6eJ6 zVBRnX4~uUVOeWr#pAvDeKh{M@zt^vDivGPafnI|^j3&PzUh)X#? z`Pq|vDRGV~;mxR7O;8ag+(GqTL=e~sI;A~z#w*FgIBkDjGwmYuQ%B6#&SU0>@$acrv*E_)fw1I|Q$hL%8lAKB+{T!7>Ts}seYyE~dT+OS&rITWj)ZH=9T zFvU0Sre;mDC#qAw`XeqUcJF5rS@xih9__khQuv20u(g~fs1{>^Bgjp)R0rJ`YpAzG z{xPfRFyQXn^r2c|)8)+_f@m?iIy3mE>&PtHL2MyBPT-1jec%1y3KeRuTcWd5Q?`^}j_DX<~I$F6Kg#FpKlNnbdPReZY;Q1>C>CD|=9 z$rV#=&SZ!Uu^HG7+?=v+?r3>nY-k+Nf$6H~(x%FyR-qyAPEYWJ8r;U>M)WxvWq;jR zPdP=TzLe^4OQpT|nXzxkAx1&wD618J6aL|MwmNb_)DwT$)@c8bB~j`wauEoO>1PV> zHnM65^w0R&`j^bNwD%x?!Uw7VNvdLuY?tEreh{lXe{J?G+=$?y@+Rgc{8ZO_bDmnd zLxFB8v*k6MFV2V(Tmt9>`~(R{!m6*I5i?Dr2n%y@>4O&e?|x!rI1g21eDc`rNf#!3 zP3#bO6rs!@+bP}9$>afxB?LEu9~gx^_H}z}yTPp@0REFGEXhH;XnQz9OFcz)0&!<8 zO33Ggw2oT1s!&TVgAn!*OE6W3VMoJS)mqi6mmk7^Rmsj`qzVD@m9aQfRZtWqpWO>1+)%Z)$`7k%ZU;a!UJV;L-%z->J81Rk! z`l)KYBpuTZUHntd+H_WGv-%M%FmE|u(gHu*%ztocU`fZ5it8y==K(X7iGLLRPZCqf zgO2x00}IHh{vCS5dm@5*m22p3ycIVW1k+vKBjyfY2={gu#J3ad60i7$f({VNoDn>n zXf{n4V{#Cp`Q!Yo&{B3pZ!ziDEsyv;;f!w~HzYqz>JC31$6`%~z&M9uu1srl?J$^j z=O3ag99#djV9=wYC>|;U8BXq`8cI3cau2 zHPqzUlG+}OfX|y3x#-GdXCPtmxN4qz6h-VNO!Wdw7-UB&>rH<*RbzxI_@*|L?I$6O zhlEUFTq3Odz>h=i>~`{`qhiNF>sulZ0cdJ%_z#GFsB+b{jLN)HETJk^K~yEVtYhutWlw=|=pHxRk!-gkC|#f*B_ z6&^xHmv_VP8XW@*D8rW5+wqbM_3ogPNTVKhuV&#jCoA&C{ z+?J6%sT8zaLAt&YPXs3qWDn-#@T1;D%o4yEZ4Zgqaf`X!_l1R%j7?4p@2 zhG+Ok(OO=dcvy&E;gXiO&KmUwZwOn6~QmoKFVoP83iDbth5VMrhAFbF( zYx!w(iWmeJp1XeCD1z0I=W8g$U|_!sNKJ`h`2LNdhKVb$cp!9wDCJm6CuAJKz)KW6 zv~7xPFZEcJvn4WI%32F~RRrd#w0-^FDBt%xI$CFhh`+&@I@&IL1&AVt3^WH<+Iv}6r^W(C!sj)rzIqsrew*6CLhycu z@3ruOyOA*=_~bYo!Wyh&#pCsvp{j6EIxTrNmvVfmd*hs+@p+vE;JrSvUc+^Mwm@@J z+Np7@cw42{P23ks%ly&CcSV-Fv!5K?eKOZwqMkQ6Eu?=`+vBV!k$MIx->FHPynW zWepe5=wyZRu?w`me-q2ph%-vVUYTk{FDUv(75<3LT4a8jTLg4@W>T7JhUI zM75H$#Mkt8;WEp0Z3{!O`E3}iD{istkAeO9$*qlg_?o%Tt%6MTj?^=sz4L(|7@49v z_9JGfK3d)w&A0FE<}eA=cD;3MDFs4@I`v>djGO|G~rQ_Y{IS5ZZQ!d?1H#FYR@=Z>Y!b z7ZHgdNb3zO=CTqrH!8-g#2F)=zCRJfO&uS(UGRN<3)Y$F6$o;J(VeZzW1EVZ0-^Z5 z?aw~}d`=V(Qv}!ram3IpVXZPUm4C18LrDy}4S=CPtH-&A8|Vhe9}kD{aVZHYt`!MY zMLf-wd&&ajOAOTpPqY0+f}jLbcJ(WyR818hqh!a;}|PrJ`MuFetv0)n>qMd@9A-Qea~+g?Ba znpxy4yhct4k=TXEHkDSBPoJ;?0(HoQAK|_=d%I1bCNT}mbn%f$;^%@FH+#c7i`9jh zMn8Bu2JC}V=bzud0{VQjOV5lphyL&)ckqSMCL@z8euZpCE7Emy_-%sZp@b72glOuH zOv@uF8#w@PK{?}I{ZOi7pAmE}R~-)`ndCl?K5`KT%wGFxf8x7fVIn2TXm!i^rrUZV zFJHtMiK1i}jHGgom*LYdl=`~R;KRswRxzfR=C#@SF`qHwMTri@S8O`!gV$&3*Yi>y zNY8!U>I2PM1HiTEi-YzWPCqp4yn`ww7e)MT`?@vq8JTcSE_hb2VJCBKp7%sQl}Xj; zFLOvWf!{9Y*Pa?Fo}p-dvu{b*y;NpbK_gXra@CumJmm1g0Q`=(OnKu<=AJ1aR=DF? z^m2^H3H&WUxHI9n*^BTh0(CIZLJ6F2ED={IV{7)JL2^}dZE;9%P#l?-4^bK*jS%S9q(^jmY;VxF23!-I-Wfg3da5Ycc@=VB-L~+6(ok!wIE}7f z_0suHpS+3NqvmYm(nlcseQ`C5{7R5>NqN)Ks6fOl{I?&-0T>HDD_R@K!ia3h4)Bop zS@_R6S3-sU)GBQWL?OG#C4pqiPc{&J1DF#E6UGGaXs-r&uUO=7-yQX4^bApoxZHHa z1)1jUmjE7^b(%=##J`=aP#64&k^AoP%}w~S_N5lU=^1mnF6z7^1I|NgDfI#Xg6JKm z4?Ad#Xie%X+$ieg)9$<{>#0%w?9D;f3Bd_LX5veA4dD;C;M#ts$RXMSM2gYXFYf|` z&l;44rS1|d4o3Z^@@|YHetyK#CxDaZWh&8?QYU@rQ!q+S7 zmwkCjx09oI7ix#PeUCeOCzyLQ;wwkB{hwsUmSKJLDnn512QRo~>sZOnl{*>a7REYKZ9MU*j_G89ZUb zKqjr_Rxv`CFk_v7)SfP^Mh>^DuvMYgT$fsLxMQI=T&0> z2Z%$98dZ#~Wv8ZC_93Jr^du#qiCTtgY7jL+;VG zL3oAY+8PfS$)d6?$_h+X>hG1=G!1gI3M!N}o-!0^{!rFINCu#Ji#{tw6kS17FhZc$ z>hr6H+(oDfq-uG^Us7PkVBEpgFwJvl85tdH4B0*T_Q{NDIFnC#YuxMV+H))PgOl-R znx%B&Ce5?Gp37l_$nWX%Pk=ge@F=p-ONuB1KNRDvxnnM#ARhZG9Y^+`gNpAZ-r5%4 zTFMzw*^FWnba-KzHM4E?rEike?pHqak~3XdE;=}8`AstXJls6k*Beg`=$y~2Vhjpy zXaEc6SZzeSp|KFx-KDWe3|z_i<=jcJng!% zs$tW?7_F;cA)lV>;)Q*tK`v$+DS8(w>xz;oHJ0h4JZvCk z`Yfv*pSgvwFtYMRPMemVJmf@)f_IM1QSi=Vh-+F;ZlO2M`LJqnJJ));vlx@*#~}3U zqgLkSiPve!IzB&+ZhOb8ilm4Pt-LFvW=HEjSzCV7`j*Yu(;L^CXy=``SI<%0W+y(o z`Qfd(skzU9XOgq>i0CKlioIkGcRC~d&qV(Am31R=FsljAs}?;muV}Vc$een9u92^^ z3J#fDu}U|S(4RnWog_xkgm*xNAYS8Mwjo_0I|U$r$=6c7Z;iMfu@NBwlAal2nlDfn8tBUxd zM;gK&u>&m3xlOp?$PLhB>R^vuPIY=yiArvyft?BgBg&Bx}hurW_fSulmOxEmq# z!d%WMPq>Y=Yv~QZV4xX=!Z7F3R$GCjb(tf>VG|LmnUG)vATd_|Mms%1h5kT0fgqfm zpg-A4phrT|e*uUMv#1k^`XgD_L zU+-#rz3$mCHspEZ!^2u(?{$ugJE_$9IbB_B2c+54r3b_|Dg z-d#QiqiA>bfwnSI&(}!_tx;R<^9i7R#iLI?E9P^=YDoqd6$aKQ1NwpWUQ0*r8GJ~? zxBSl#i(-@SHG=9*qQ?ai&~44rA#5N6)fP#INb(7_!gsW3i4ZpB0-1tW=xekt;`c9( zpD3maAX+1Ec%hvVJXyo%xQf|ujKsWF66Ban#@{OSq93B`?U!PtZc1zSk&$o?ljrJF z3!#sT)l+qd`$YKT*+k4MPE+rhdGs~>b(RI6k}8_+k3~T;cby8_$QS9yF<&Z9@8BA& zE0X_hzF#K!!+u~+&i`2N-)O8y)YfxzS5p^jCpSlzM+Vg|`j&#RJ(QcJ^ITIwR_U3V zwY|Bkg1Lu^qrI_%lIjb!|ENhC+gsat0hoSi0H7*>Upmac73_>Hp#&?A|7eT-hHSBN zLD^cYU?4Yu0|5AaH)A_%Q&9&?J97Z=*I}ZrrjM{L2=8xp*JDc`_gOhW z+`r1jjh&>;tu3vfJTK^Jeu-4w%KPRjrFW(nlEw z&m%kS|N99BKa$7(|30~(W&{E_Il+L(t-pUhp65~a`1)PvpIbc87W`TA_z46&>VhG> zzx4hrd6YkHKc4fKk3g|%j_MB9j|;0oO%5&nyBR@$q38ajo&i9AVcY(%=*0E42CEZ@ z1FvKe5Ec-)!S>={yq+KmLmCJIFdFT_1V5A@tnO=nMDSqO&sPd|-{f;*?NDi(@h4g$ zNa$&`b$I=;|DBm_gUNYi6}UHA>%v!X*`pgDeJvh-&2vTx_eplDi|E_NYvlQ8Iw8td zQltMdZyy``XWRw-HE;hU?fnh=2Ko;V@BdtM4m=9xPzMHf5s0(FP=>%0{*QY9oB&YA z13~^1E*^d4pVQ(WKKR_)!ovI!ZwCd{=>h@l&_9s=-(Yp1zi4%6kEOrNqXOLo0YJZ( z%=-)42l_MP{H==|zwP@gCjB4l>ggxGcuXu|7g?Rq#DdVzCstOR$#Pv#EC|!VD3Jl< z(ns6CPE(+^17~)!WGTqvcqd2gi*(g7hLzHB6(}#KH;Wg`c?HNI8lAu0-m$k#4rMv4 zW=-D|YGi7{+OWh;8zk4u2F0J!!-#watUJ~#!n!r_V~;`%$q z6SDPXF4$j$j30JLhu#k|s4q&z)gm7S4W|~paWBg@Y=v>&`M1e_4bC5~!ok7)r_KIt zvi}jupivC`D~ddp{f^2&04tDO)Qbm{$H13x9{^ze3@E1vEBjXn=sBfsl=Z>vtXh z;UPk~hmZaRJ>5Uzz;6WN-<;`Re)~v(bp2J2oVD3w*!&e||42LHjxQg9kbm=Bj>jJU z>G{x}n7uTGuIk8SYHaP|2mrBxxY>XJCM!2LCs#gpD8CXawRACdva&XHWpi|~Wd5E1 z#Gz|Gx>-9qJbz?gGCk+x0CIqV5D+hj2Mhs0iIzN1fI#NISNiYlEN%?Vh>n)O8vd&b ze_$*B>Z+%Uxdj?Dwt~^1x%7_*fSZd8%muIj{FZU>f}k1pR~f+J4;i!_kE!PI@t2H; z7Xpnx|C9lNk5l!ZG9F%T==l6Y#=*n==raG5K_Hy}Iu-(cwCq3ixFOIBf`7?)|E0$R z{#RQ#c);L)9SZ^S{C7V%plgRg{qvvoLO6J#?)+b}NBZY~%Q%7m?i(i$G}-*uv0RUH z;J;)Lj(_z7>H+^6KVA;V-}>R^Vhp`baruS$RJHatf3)wT&#O2(x&a str: + return (datetime.date(1998, 12, 1) - datetime.timedelta(days=90)).format("YYYY-MM-DD") + + +def lineitem_filtered(lineitem: ps.DataFrame, start_date: str) -> ps.DataFrame: + return lineitem.filter((lineitem.l_shipdate <= start_date)) + + +def disc_price( + l_extendedprice: pd.Series, l_discount: pd.Series +) -> htypes.column[pd.Series, float]: + return l_extendedprice * (1 - l_discount) + + +def charge( + l_extendedprice: pd.Series, l_discount: pd.Series, l_tax: pd.Series +) -> htypes.column[pd.Series, float]: + # This could easily depend on the prior one... + return l_extendedprice * (1 - l_discount) * (1 + l_tax) + + +@h_spark.with_columns( + disc_price, + charge, + columns_to_pass=["l_extendedprice", "l_discount", "l_tax"], +) +def lineitem_price_charge(lineitem: ps.DataFrame) -> ps.DataFrame: + return lineitem + + +def final_data(lineitem_price_charge: ps.DataFrame) -> ps.DataFrame: + return ( + lineitem_price_charge.groupBy(["l_returnflag", "l_linestatus"]) + .agg( + F.sum("l_quantity").alias("sum_l_quantity"), + F.avg("l_quantity").alias("avg_l_quantity"), + F.sum("l_extendedprice").alias("sum_l_extendedprice"), + F.avg("l_extendedprice").alias("avg_l_extendedprice"), + F.sum("disc_price").alias("sum_disc_price"), + F.sum("charge").alias("sum_charge"), + F.avg("l_discount").alias("avg_l_discount"), + F.count("*").alias("count"), + ) + .orderBy(["l_returnflag", "l_linestatus"]) + ).toPandas() diff --git a/examples/spark/tpc-h/query_12.py b/examples/spark/tpc-h/query_12.py new file mode 100644 index 000000000..903566e0f --- /dev/null +++ b/examples/spark/tpc-h/query_12.py @@ -0,0 +1,56 @@ +import pandas as pd +import pyspark.sql as ps +from pyspark.sql import functions as F + +from hamilton import htypes +from hamilton.experimental import h_spark + +# see # See # See https://github.com/dragansah/tpch-dbgen/blob/master/tpch-queries/12.sql + + +def lineitems_joined_with_orders(lineitem: ps.DataFrame, orders: ps.DataFrame) -> ps.DataFrame: + return lineitem.join(orders, lineitem.l_orderkey == orders.o_orderkey) + + +def start_date() -> str: + return "1995-01-01" + + +def end_date() -> str: + return "1996-12-31" + + +def filtered_data( + lineitems_joined_with_orders: ps.DataFrame, start_date: str, end_date: str +) -> ps.DataFrame: + return lineitems_joined_with_orders.filter( + (lineitems_joined_with_orders.l_shipmode.isin("MAIL", "SHIP")) + & (lineitems_joined_with_orders.l_commitdate < lineitems_joined_with_orders.l_receiptdate) + & (lineitems_joined_with_orders.l_shipdate < lineitems_joined_with_orders.l_commitdate) + & (lineitems_joined_with_orders.l_receiptdate >= start_date) + & (lineitems_joined_with_orders.l_receiptdate < end_date) + ) + + +def high_priority(o_orderpriority: pd.Series) -> htypes.column[pd.Series, int]: + return (o_orderpriority == "1-URGENT") | (o_orderpriority == "2-HIGH") + + +def low_priority(o_orderpriority: pd.Series) -> htypes.column[pd.Series, int]: + return (o_orderpriority != "1-URGENT") & (o_orderpriority != "2-HIGH") + + +@h_spark.with_columns(high_priority, low_priority, columns_to_pass=["o_orderpriority"]) +def with_priorities(filtered_data: ps.DataFrame) -> ps.DataFrame: + return filtered_data + + +def shipmode_aggregated(with_priorities: ps.DataFrame) -> ps.DataFrame: + return with_priorities.groupBy("l_shipmode").agg( + F.sum("high_priority").alias("sum_high"), + F.sum("low_priority").alias("sum_low"), + ) + + +def final_data(shipmode_aggregated: ps.DataFrame) -> pd.DataFrame: + return shipmode_aggregated.toPandas() diff --git a/examples/spark/tpc-h/query_8.py b/examples/spark/tpc-h/query_8.py new file mode 100644 index 000000000..4daaa13f9 --- /dev/null +++ b/examples/spark/tpc-h/query_8.py @@ -0,0 +1,91 @@ +import pandas as pd +import pyspark.sql as ps +import pyspark.sql.functions as F + +# See # See https://github.com/dragansah/tpch-dbgen/blob/master/tpch-queries/8.sql +from hamilton import htypes +from hamilton.experimental import h_spark + + +def start_date() -> str: + return "1995-01-01" + + +def end_date() -> str: + return "1996-12-31" + + +def america(region: ps.DataFrame) -> ps.DataFrame: + return region.filter(F.col("r_name") == "AMERICA") + + +def american_nations(nation: ps.DataFrame, america: ps.DataFrame) -> ps.DataFrame: + return nation.join(america, nation.n_regionkey == america.r_regionkey).select(["n_nationkey"]) + + +def american_customers(customer: ps.DataFrame, american_nations: ps.DataFrame) -> ps.DataFrame: + return customer.join(american_nations, customer.c_nationkey == american_nations.n_nationkey) + + +def american_orders(orders: ps.DataFrame, american_customers: ps.DataFrame) -> ps.DataFrame: + return orders.join(american_customers, orders.o_custkey == american_customers.c_custkey) + + +def order_data_augmented( + american_orders: ps.DataFrame, + lineitem: ps.DataFrame, + supplier: ps.DataFrame, + nation: ps.DataFrame, + part: ps.DataFrame, +) -> ps.DataFrame: + d = lineitem.join(part, lineitem.l_partkey == part.p_partkey).drop("n_nation", "n_nationkey") + d = d.join(american_orders.drop("n_nationkey"), d.l_orderkey == american_orders.o_orderkey) + d = d.join(supplier, d.l_suppkey == supplier.s_suppkey) + d = d.join(nation, d.s_nationkey == nation.n_nationkey) + return d + + +def order_data_filtered( + order_data_augmented: ps.DataFrame, + start_date: str, + end_date: str, + p_type: str = "ECONOMY ANODIZED STEEL", +) -> ps.DataFrame: + return order_data_augmented.filter( + (F.col("o_orderdate") >= F.to_date(F.lit(start_date))) + & (F.col("o_orderdate") <= F.to_date(F.lit(end_date))) + & (F.col("p_type") == p_type) + ) + + +def o_year(o_orderdate: pd.Series) -> htypes.column[pd.Series, int]: + return pd.to_datetime(o_orderdate).dt.year + + +def volume(l_extendedprice: pd.Series, l_discount: pd.Series) -> htypes.column[pd.Series, float]: + return l_extendedprice * (1 - l_discount) + + +def brazil_volume(n_name: pd.Series, volume: pd.Series) -> htypes.column[pd.Series, float]: + return volume.where(n_name == "BRAZIL", 0) + + +@h_spark.with_columns( + o_year, + volume, + brazil_volume, + columns_to_pass=["o_orderdate", "l_extendedprice", "l_discount", "n_name", "volume"], + select=["o_year", "volume", "brazil_volume"], +) +def processed(order_data_filtered: ps.DataFrame) -> ps.DataFrame: + return order_data_filtered + + +def brazil_volume_by_year(processed: ps.DataFrame) -> ps.DataFrame: + return processed.groupBy("o_year").agg( + F.sum("volume").alias("sum_volume"), F.sum("brazil_volume").alias("sum_brazil_volume") + ) + + +def final_data(brazil_volume_by_year: ps.DataFrame) -> pd.DataFrame: + return brazil_volume_by_year.toPandas() diff --git a/examples/spark/tpc-h/run.py b/examples/spark/tpc-h/run.py new file mode 100644 index 000000000..2649cff8b --- /dev/null +++ b/examples/spark/tpc-h/run.py @@ -0,0 +1,52 @@ +import click + +exec("from hamilton.experimental import h_spark") +import csv_data_loaders +import pyspark +import query_1 +import query_8 +import query_12 + +from hamilton import base, driver + +QUERIES = {"query_1": query_1, "query_8": query_8, "query_12": query_12} + + +def run_query(query: str, data_dir: str, visualize: bool = True): + """Runs the given query""" + + dr = ( + driver.Builder() + .with_modules(QUERIES[query], csv_data_loaders) + .with_adapter(base.DefaultAdapter()) + .build() + ) + spark = pyspark.sql.SparkSession.builder.getOrCreate() + if visualize: + dr.visualize_execution( + ["final_data"], f"./dag-{query}", {}, inputs={"data_dir": data_dir, "spark": spark} + ) + df = dr.execute(["final_data"], inputs={"data_dir": data_dir, "spark": spark})["final_data"] + print(df) + + +@click.command() +@click.option("--data-dir", type=str, help="Base directory for data", required=True) +@click.option( + "--which", type=click.Choice(list(QUERIES.keys())), help="Which query to run", required=True +) +@click.option( + "--visualize", + type=bool, + help="Whether to visualize the execution", + is_flag=True, +) +def run_tpch_query(data_dir: str, which: str, visualize: bool): + """Placeholder function for running TPCH query""" + # Place logic here for running the TPCH query with the given 'which' value + click.echo(f"Running TPCH query: {which}") + run_query(which, data_dir, visualize) + + +if __name__ == "__main__": + run_tpch_query() diff --git a/hamilton/experimental/h_spark.py b/hamilton/experimental/h_spark.py index dc4d7e674..c98723746 100644 --- a/hamilton/experimental/h_spark.py +++ b/hamilton/experimental/h_spark.py @@ -1,3 +1,5 @@ +import abc +import dataclasses import functools import inspect import logging @@ -8,15 +10,17 @@ import numpy as np import pandas as pd import pyspark.pandas as ps -from pyspark.sql import Column, DataFrame, dataframe, types +from pyspark.sql import Column, DataFrame, SparkSession, dataframe, types from pyspark.sql.functions import column, lit, pandas_udf, udf -from hamilton import base, htypes, node +from hamilton import base, htypes, node, registry from hamilton.execution import graph_functions from hamilton.function_modifiers import base as fm_base from hamilton.function_modifiers import subdag from hamilton.function_modifiers.recursive import assign_namespace from hamilton.htypes import custom_subclass_check +from hamilton.io import utils +from hamilton.io.data_adapters import DataLoader logger = logging.getLogger(__name__) @@ -1197,3 +1201,61 @@ def generate_nodes(self, fn: Callable, config: Dict[str, Any]) -> List[node.Node def validate(self, fn: Callable): _derive_first_dataframe_parameter_from_fn(fn) + + +@dataclasses.dataclass +class SparkDataFrameDataLoader(DataLoader): + """Base class for data loaders that load pyspark dataframes. + We are not yet including data savers, but that will be added to this most likely.. + """ + + spark: SparkSession + + @classmethod + def applicable_types(cls) -> Collection[Type]: + return [DataFrame] + + @abc.abstractmethod + def load_data(self, type_: Type[DataFrame]) -> Tuple[ps.DataFrame, Dict[str, Any]]: + pass + + +@dataclasses.dataclass +class CSVDataLoader(SparkDataFrameDataLoader): + path: str # It supports multiple but for now we're going to have a single one + # We can always make that a list of strings, or make a multiple reader (.multicsv) + header: bool = True + sep: str = "," + + def load_data(self, type_: Type[DataFrame]) -> Tuple[ps.DataFrame, Dict[str, Any]]: + return ( + self.spark.read.csv(self.path, header=self.header, sep=self.sep, inferSchema=True), + utils.get_file_metadata(self.path), + ) + + @classmethod + def name(cls) -> str: + return "csv" + + +@dataclasses.dataclass +class ParquetDataLoader(SparkDataFrameDataLoader): + path: str # It supports multiple but for now we're going to have a single one + + # We can always make that a list of strings, or make a multiple reader (.multicsv) + + def load_data(self, type_: Type[DataFrame]) -> Tuple[ps.DataFrame, Dict[str, Any]]: + return self.spark.read.parquet(self.path), utils.get_file_metadata(self.path) + + @classmethod + def name(cls) -> str: + return "parquet" + + +def register_data_loaders(): + """Function to register the data loaders for this extension.""" + for loader in [CSVDataLoader, ParquetDataLoader]: + registry.register_adapter(loader) + + +register_data_loaders() From a1ac01de78210c605dcdb68e33b70674c04d86e1 Mon Sep 17 00:00:00 2001 From: elijahbenizzy Date: Mon, 21 Aug 2023 16:00:29 -0700 Subject: [PATCH 09/11] Updates the blog post to be a better narrative/cleaner --- examples/spark/pyspark/README.md | 185 ++++++++++-------- .../spark/pyspark/grouped_transformations.png | Bin 0 -> 514850 bytes examples/spark/pyspark/illustration.png | Bin 0 -> 31632 bytes 3 files changed, 105 insertions(+), 80 deletions(-) create mode 100644 examples/spark/pyspark/grouped_transformations.png create mode 100644 examples/spark/pyspark/illustration.png diff --git a/examples/spark/pyspark/README.md b/examples/spark/pyspark/README.md index 7fd24a0dc..483905936 100644 --- a/examples/spark/pyspark/README.md +++ b/examples/spark/pyspark/README.md @@ -1,33 +1,64 @@ -# Hamilton and Spark +# Hamilton and Pyspark -Hamilton now has first-class pyspark integration! While we will likely be improving it as we go along, -this version is the first we're considering "stable" and we are planning an imminent move out of "experimental" into "plugins". +**TL;DR** Hamilton now supports full pyspark integration. This enables you to write a DAG of transformations as +python UDFs, pandas UDFs, or pyspark transformations and apply them to a central dataframe, using the +[Hamilton](https://github.com/dagworks-inc/hamilton) paradigm. Functions written this way improve maintaibility, modularity, +readability, and clarity of data lineage in spark ETLs. -# Motivation +

+ +

-Spark is immensely powerful -- its one of the only tools that effectively handles large -datasets in a distributed fashion, and has a variety of plugins/a tooling ecosystem that -data teams love. +*A spark pipeline representing multiple joins (in blue), a set of map operations (in green) and a set of join/filters (in yellow). This uses Hamilton’s visualization features (with a little extra annotation). See [TPC-H query 8](../tpc-h/query_8.py) for motivation.* -Just like pandas pipelines, however, spark pipelines can be difficult to maintain/manage, devolving into -spaghetti code over time. Specifically, we've observed the following problems with pyspark pipelines: -1. They rarely get broken up into modular and reusable components -2. They contain a lot of "implicit" dependencies -- as you do break them up into functions, it is difficult to -specify which columns the dataframes contain/depend on, and how that changes through the workflow. -3. They are difficult to configure in a readable manner. A monolithic spark script likely has a few different shapes/parameters, and naturally becomes littered with poorly documented if/else statements -4. They're not easy to unit tests. While specific UDFs can be tested, spark transformations are difficult to test in a modular fashion. +## Apache Spark +[Apache Spark](https://spark.apache.org/) (and its python API, [pyspark](https://spark.apache.org/docs/latest/api/python/index.html)) is an open-source library for building out highly scalable data transformations. +At its core is the notion of the RDD (resilient distributed dataframe), which represents a lazily evaluated, +partitioned, in-memory dataset that stores the information needed to recreate the data +if any of the servers computing it fail. The pyspark library gives data practitioners +a dataframe-centric API to interact with this in python, enabling them to specify computation +and scale up to the resources they have available. Since its introduction in 2014, spark has taken +off and is now the de facto way to perform computations on large (multi gb -> multi tb) datasets. -Vanilla Hamilton with pyspark gets you part of the way there. You can easily build functions that input/output spark dataframes, -and your code gets more modular/easier to maintain. That said, you still have to deal with the implicit dependencies, and the -fact that you're not really building a DAG of operations -- its more of a linear chain. You don't get any column-level lineage -(you'll have to look at the spark execution plan for that/an external lineage tool), and you either have lots of steps each manipulating -the same dataframe, or a few large modular ones. +## Limitations of Spark for Complex Pipelines -The new spark integation is meant to give you the best of both worlds -- we want to allow you to express column-level map (cardinality-preserving) operations -while simultaneously passing around dataframes for aggregations, filters, and joins. +Just like any ETLs, spark pipelines can be difficult to maintain and manage, +and often devolve into spaghetti code over time. +Specifically, we've observed the following problems with pyspark pipelines: +1. _They rarely get broken up into modular and reusable components._ +2. _They commonly contain "implicit" dependencies._ Even when you do break them into functions, it is difficult to specify which columns the transformed dataframes depend on, and how that changes throughout your workflow. +3. _They are difficult to configure in a readable manner._ A monolithic spark script likely has a few different shapes/parameters, and naturally becomes littered with poorly documented if/else statements. +4. _They are not easy to unit test._ While specific UDFs can be tested, spark transformations are tough to test in a modular fashion. +5. _They are notoriously tricky to debug._ Large pipelines of spark transformations (much like SQL transformations) will often have errors that cascade upwards, and pinpointing the source of these can be quite a challenge. -# Design + +# Hamilton +As this is a README inside the Hamilton repository, we assume some basic familiarity. That said, here's a quick primer: + +Hamilton is an open-source Python framework for writing data transformations. +One writes Python functions in a declarative style, which Hamilton parses into nodes in +a graph based on their names, arguments and type annotations. The simple rule is akin to that of pytest fixtures -- +the name of a parameter points to another node (function) in the graph, and the name of the function defines a referencable node. +You can request specific outputs, and Hamilton will execute the required nodes (specified by your functions) to produce them. + +You can try hamilton out in your browser at [tryhamilton.dev](https://tryhamilton.dev). + +# Integration + +Breaking your pipeline into Hamilton functions with pyspark dataframes as inputs and outputs gets you most of +the way towards more modular/documented code. +That said, it falls flat in a critical area – column-level lineage/transformation +simplicity. For complex series of map operations, spark represents all transformations +on a single dataframe in a linear chain by repeatedly calling `withColumn`/`select` to create columns. +For dataframe APIs that manage indices, hamilton improves this experience by encouraging the user to +pull apart column-level transformations then join later. With columns that share cardinality, this is generally an efficient approach. + +Spark, however, has no notion of indices. Data is partitioned across a cluster, and once a set of columns is selected it has the potential to be reshuffled. +Thus, the two options one previously had for integrating with pyspark both have disadvantages: + +1. Extracting into columns then joining is prohibitively expensive and taxing on the spark optimizer (which we have not found was smart enough to detect this pattern) +2. Running pure DataFrame transformations does not afford the expressiveness that Hamilton provides. The idea is to break your code into components. These components make one of two shapes: @@ -37,6 +68,14 @@ The idea is to break your code into components. These components make one of two For the first case, we just use the pyspark dataframe API. You define functions that, when put through Hamilton, act as a pipe. For example: +Hamilton `1.27.0` introduces a new API to give the user the best of both worlds. You can now express column-level +operations in a DAG on the same dataframe, as part of a multi-step process. + +With the new `@with_columns` decorator, your break your pipeline into two classes of steps: + +### Joins/Aggregations/Filters + +We simply write functions that take in dataframes and return dataframes. ```python import pyspark.sql as ps @@ -54,6 +93,8 @@ def raw_data_3() -> ps.DataFrame: """Loads up data from an external source""" ``` +### Columnar Operations + For the next case, we define transformations that are columnar/map-oriented in nature. These are UDFs (either pandas or python), or functions of pyspark constructs, that get applied to the upstream dataframe in a specific order: @@ -92,23 +133,27 @@ def final_result(all_initial_data: ps.DataFrame, raw_data_3: ps.DataFrame) -> ps Then, the final_result function is called, with the result of the transformations passed in.""" return _join(all_initial_data, raw_data_3) ``` +Contained within the `load_from` functions/modules is a set of transformations that specify a DAG. +These transformations can take multiple forms – they can use vanilla pyspark operations, pandas UDFs, +or standard python UDFs. See documentation for specific examples. + +The easiest way to think about this is that the `with_columns` decorator “linearizes” the DAG. +It turns a DAG of hamilton functions into a linear chain, repeatedly appending those columns to the initial dataframe. +![](illustration.png | width=500px) +*The natural DAG of steps in three separate configurations -- hamilton/pandas, pure pyspark, and pyspark + hamilton* `with_columns` takes in the following parameters (see the docstring for more info) 1. `load_from` -- a list of functions/modules to find the functions to load the DAG from, similar to `@subdag` -2. `columns_to_pass` -- not compatible with `external_inputs`. Dependencies specified from the initial dataframe, -injected in. Not that you must use one of this or +2. `columns_to_pass` -- not compatible with `pass_dataframe_as`. Dependencies specified from the initial dataframe, +injected in. Not that you must use one of this or `pass_dataframe_as` 3. `pass_dtaframe_as` -- the name of the parameter to inject the initial dataframe into the subdag. If this is provided, this must be the only pyspark dataframe dependency in the subdag that is not also another node (column) in the subdag. 4. `select` -- a list of columns to select from the UDF group. If not specified all will be selected. -5. `dataframe` -- the initial dataframe. If not specified, will default to the only dataframe param -in the decorated function (and error if there are multiple). -6. `namespace` -- the namespace of the nodes generated by this -- will default to the function name that is decorated. - +5. `namespace` -- the namespace of the nodes generated by this -- will default to the function name that is decorated. -`with_columns` serves to _linearize_ the operation, enabling you to define a DAG, and have all your operations run on a single dataframe, -in topological order. You can thus represent a clean, modular, unit-testable string of transformations while also allowing for -complex sets of feature UDFs. All of these can rely on data passed from a node or parameter external to the group of functions. +Note that the dependency that forms the core dataframe will always be the first parameter to the function. Therefore, the first parameter +must be a pyspark dataframe and share the name of an upstream node that returns a pyspark dataframe. You have two options when presenting the initial dataframe/how to read it. Each corresponds to a `with_columns` parameter. You can use: 1.`columns_to_pass` to constrain the columns that must exist in the initial dataframe, which you refer to in your functions. In the example above, the functions can refer to the three columns `column_1_from_dataframe`, `column_2_from_dataframe`, and `column_3_from_dataframe`, but those cannot be named defined by the subdag. @@ -153,7 +198,9 @@ def final_result(all_initial_data: ps.DataFrame, raw_data_3: ps.DataFrame) -> ps Approach (2) requires functions that take in pyspark dataframes and return pyspark dataframes or columns for the functions reading directly from the dataframe. If you want to stay in pandas entirely for the `with_columns` group, you should use approach (1). -There are four flavors of transforms supported. +### Flavors of UDFs + +There are four flavors of transforms supported that compose the group of DAG transformations: #### Pandas -> Pandas UDFs These are functions of series: @@ -171,7 +218,7 @@ and the function name determines the output column name. Note that, due to the type-specification requirements of pyspark, these have to return a "typed" (`Annotated[]`) series, specified by `htypes.column`. These are adapted to form pyspark-friendly [pandas UDFs](https://spark.apache.org/docs/3.1.2/api/python/reference/api/pyspark.sql.functions.pandas_udf.html) -#### Python primitives -> python primitives UDFs +#### Python primitives -> Python Primitives UDFs These are functions of python primitives: @@ -182,7 +229,7 @@ def foo(bar: int, baz: int) -> int: These are adapted to standard [pyspark UDFs](https://spark.apache.org/docs/3.1.3/api/python/reference/api/pyspark.sql.functions.udf.html). -#### pyspark dataframe -> pyspark columns +#### Pyspark Dataframe -> Pyspark Columns These are functions that take in a pyspark dataframe (single) and output a pyspark column. @@ -204,17 +251,14 @@ def foo(bar_baz: ps.DataFrame) -> ps.Column: return df["bar"] + 1 ``` -In this case we are only allowed a single dataframe dependency, and the paraemter name does not matter. -These are an out for when the pyspark computational expression is more convenient than the pandas one, -or it is not possible to express it in pandas. +In this case we are only allowed a single dataframe dependency, and the parameter name does not matter. +The columns specified are injected into the dataframe, allowing you to depend on multiple upstream columns. #### pyspark dataframe -> pyspark dataframe This is the ultimate power-user case, where you can manipulate the dataframe in any way you want. Note that this and the column-flavor is an _out_, meaning that its a way to jump back to the pyspark world and not have to break up -your map functions for a windowed aggregation. - -Note that you can easily shoot yourself in the foot here, so be careful! This should only be used if +your map functions for a windowed aggregation. You can easily shoot yourself in the foot here. This should only be used if you strongly feel the need to inject a map-like (index-preserving, but not row-wise) operation into the DAG, and the df -> column flavor is not sufficient (and if you find yourself using this a lot, please reach out, we'd love to hear your use-case). @@ -230,8 +274,7 @@ def foo(df: ps.DataFrame) -> ps.DataFrame: return df.withColumn("bar", df["bar"] + 1) ``` -Note that this is the column-flavor in which you (not the framework) are responsible for calling `withColumn`. - +Note that this is isomorphic to the column-flavor in which you (not the framework) are responsible for calling `withColumn`. We have implemented the hamilton hello_world example in [run.py](run.py) and the [map_transforms.py](map_transforms.py)/[dataflow.py](dataflow.py) files so you can compare. You can run `run.py`: @@ -240,8 +283,15 @@ so you can compare. You can run `run.py`: and check out the interactive example in the `notebook.ipynb` file. +We have also implemented three of the [TPC-H](https://www.tpc.org/tpch/) query functions to demonstrate a more real-world set of queries: +1. [query_1][../tpc-h/query_1.py] +2. [query_8][../tpc-h/query_8.py] +3. [query_12][../tpc-h/query_12.py] + +See the [README](../tpc-h/README.md) for more details on how to run these. + -## How does this work? +## Technical Details The `with_columns` decorator does the following: 1. Resolves the functions you pass in, with the config passed from the driver @@ -251,49 +301,24 @@ The `with_columns` decorator does the following: - Transforms each function into a function of that input dataframe and any other external dependencies Thus the graph continually assigns to a single (immutable) dataframe, tracking the result, and still displays the DAG shape -that was presented by the code. Column-level lineage is preserved and readable from the code, while it executes as a +that was presented by the code. Column-level lineage is preserved as dependencies and easy to read from the code, while it executes as a normal set of spark operations. -## Why use Hamilton and not plain spark? - -As you can see above, we delegate almost entirely to spark. However, when you want column-level lineage and modular functions, -vanilla spark is often suboptimal. It requires "linearization" (e.g. chained) modification of the same dataframe, and can -get messy and out of hand. Thus we group together columnar (map-ish) operations together, while still allowing the -chaining of dataframe functions that pyspark expresses naturally. - -Furthermore, this opens up a few interesting debugging capabilities, that we're building out: -1. Running components of your workflow in pandas -2. Unit testing individual spark transforms -3. Grabbing intermediate results to debug the spark execution plan at any given moment -4. Adding `collect` halfway through to make inspection easier -5. Breaking large spark jobs into separate tasks by arranging their functions into modules - -We have found that spark ETLs tend to fit the patterns above nicely, and that hamilton helps make them shine. +## Scaling Alternatives -## Why not just pass around spark dataframes in Hamilton functions? +Pyspark is not the only way to scale up your computation. Hamilton supports `pandas-on-spark` as well. You can use pandas-on-spark with the `KoalaGraphAdapter` -- see [Pandas on Spark](../pandas_on_spark/README.md) for reference. +Some people prefer vanilla spark, some like pandas-on-spark. We support both. Hamilton also support executing map-based pandas UDFs in pyspark, in case you want simple parallelism. See [pyspark_udfs](../pyspark_udfs/README.md) for reference. -This is great when your functions are linear, but when you have enough features this gets messy. Specifically, -unit testing/zooming in on transformations often requires tooling to manage/collect UDFs, and that's what Hamilton does. - -The biggest problem comes when you want column-level transformations so you extract out the columns into different dataframes -and join them together. This can result in (accidental) performance issues, as spark does very poorly in handling multiple -large joins. - -## Why not use pandas-on-spark? - -We support that! You can use pandas-on-spark with the `KoalaGraphAdapter` -- see [Pandas on Spark](../pandas_on_spark/README.md) for reference. -Some people prefer vanilla spark, some like pandas-on-spark. We support both. - -Note there are other scaling libraries that Hamilton supports -- it all depends on your use-case: +Hamilton has integrations with other scaling libraries as well -- it all depends on your use-case: - [dask](../../dask/README.md) - [ray](../../ray/README.md) - [modin](https://github.com/modin-project/modin) (no example for modin yet but it is just the pandas API with a different import) -## Why are there so many lines in the visualizations for a with_columns group? - -Good question! This is because we are adding two sets of edges: -1. The single dataframe being passed through and updated (immutably, which is why it fits so nicely with Hamilton) -2. The original edges, dependent on columns - -We are planning, shortly, to display these edges differently, and ideally allow a visualization mode for these specifically (to show the DAG with (1), (2), or both). +## Next Steps +A few interesting directions: +1. Improve the visualization, allowing you to differentiate the dependencies that just exist for structure from the central linear dependency that flows through. +2. Add similar capabilities for other dataframe libraries for which series/indices are not first-class citizens (polars, etc...) +3. Add data quality decorations for spark -- this is complex as they often require realization of the DAG, which one typically wishes to delay along with the rest of the computation +4. Add more data loaders to seamlessly load data to spark/from spark +5. Add natural constructs for `collect()`/`cache()` through the DAG diff --git a/examples/spark/pyspark/grouped_transformations.png b/examples/spark/pyspark/grouped_transformations.png new file mode 100644 index 0000000000000000000000000000000000000000..f2ade28501e0debf37be713213f37aaadb1d0e34 GIT binary patch literal 514850 zcmeFZWmuG7+b>LaNH-&0k^<60Nr-fdGzdtCv<%%Lt$++j3(}<`L#H&-(l|;t15CVw z|ND8Kd++`1_tQS!pWLoYtHA)Y>Fi11U`{upmM2f}HP+yJ8zGC0 zW(xF5+c}zPzR=dj;78vRU|?ZVVql~1FwrlJI2R0@zxNmz+~_L?26jd)20r>qiGJ%9 zVf{xf&Tq-Gyu& zJnbBX0^QLS=s&qY8T75Yqo3{lKzBC}UztF8_P-P|==-~3VfOofN&H;p+0C@|?yGqE zINp~O5)l$%S0K87|Gu1$!wVS$RkeSyqyLjdUAwk+p0E7m+1c+p`p>_A z#_1U7@?Sl9`2K5J=m`qn)d-6Ui3tC9YaJ z_zq<>|9yUW#&SNM$;hp=rYYD+{DZQepOGbXWSz9KS$WE2y!x;Edqi%rWvqmac4!NjFLlDx|kD4K3b{U{f^6g^y*MYFD(p%WAb!r?GFx=G8uaR12 zW}E$jfv9kO`#N_@-+{AK4iyVKhBJ8|Bk@7VCq8dx-;FMBT^r6XgnxUPm!kaGD5cVQ z-NF)1mHNXqORd~NfR~6J(Eo)f$~Xv-3K_I^PHD9ayd9R#evr>V@k!GsNlnSL}Pv&;;WO#e0w{$U1VN{$_ z8qRd~Ned3dPdq^s_dP*4(Fx?#awyKH2EA-H8HLgy)G;&-CmsmGA@pEuK7*zyr`MlA zh2!wBBhC8gHDP)ycRgZny)!BV$)lc#2O?=W#sBoRKK5zEErXz__~Re<@ouQ>7CBgY z6h(+rGFc`+)LLBaRf2(y{fZ6reQ$N~yMq-eUokEU*nZB`O~ z-OCV}={C}g$ACf%>QmO+)-7sOIeRfdKCn zy6pz(fXpHxtTNRm^%-sCz@B4{sYH*M_{x<+)5+|?8rQDH zKaFdw8OkpG1d>M29<418f8y^oc#Hwn@DD%ctQU}cBvOlGnRjZ6O0`n@ik<2C zOW7R3$J!a$RBFx*1|0agOR0`p`k4+p(G)Lt1sc1{TJ0ANW;xBJZKG)gBz(N84$NAq zB{WTR=!?rc|1bOhUAq7MwxJZ4Y83S7Io{?f<-otuWisf+n`Z=FD5>-DGi%+J680B; zPd=fcs{{6^VK7XRG8Os2aWgR9YH*jcXRv~%+c`gUAcBy^1GY30;LlP`KRSvh)|9^+`e^=+f**mU&&PqfF&MFG$_eVFn-*fJ=kZ?}lKe0-MN(<_9h_@Gk{MI?V zO&=(w@__2V`dyaFTuVeTH6ij4P9CCr-Td(knC+)bOFJ7H)_Ah z@aGYiV)H;7!bGX}SNb&!NN|A$fw%V5Ub%XNfsbc95EP1w+!~r+sTd3HJ@N#R;Huv{ zwBzK9jRf#@AVi7UwJLEs*k7(935wjPN=ttdlIq0gw*g}#IS2woGpJR%)2PvHccg$DN9w`Z|2Rrp*Kxj)+9cd1fk-wSN#azcQ5at}$*nos8i!inNkeI# zg+3dxCpM2<#a9P(%^l@X_pxY6?`qn(QM5A4MBUNdT;Kf5AFD}5zbEZ%)SagOW?n0` z@bLlo38j$Pqa%(^AIstV&zv7ijLofBvIytKZYzD=O8@7gruj6zXg*^V)rTH5>*m=z zR2e|GFtc9%ORXeYOb6<8wAA3pb zSMM?KXjj>MgWkYs2e@uHY=p)4v}o(7-3K1=M_Eh78nJ@n(=m1LJ>*XVrm}#LaSLw2_f)Y9i(HLXky3sgh?xIlRIS#xz2mJIO_2{O)#ZPt+Zd96 z(+o0=D=;E(l$R^4A4=IMaV86p3__4Zeb|G=)xQZpL7S(!ZTBXCqUM3fA2kUSXwT=h zpWc85mOV!!5XAK&8m~x|vVbxiiKMF6kCsEJ@yem{-X6pA4dS3{Sqj%bNqyZyS+CF3 zmy-2BNWFLhMoJz=5obcl3q7X^2p-i0gwM}^2A=fZ-KW=1GTAN*_E`R;K>C$wE<>q) z-wXDBA41-Bpif;Aewj+Y=*tgp_b=76OvaMuJx{#gO&XM#Y@4OYjf3De-~^DZsT|Hv z%5_In2%@5uT(h$nAt?l*sk-m{vEMu7D1F}(I8%n&6P9Vc58R2vhu4p6H?HT|*C)Uv zDe&AHaREQHON@h!(w9AjGc{_C@KN&WQ}sJs!Sn0u4@m9#9>Q)tX*Jp3!eO|VAb`q<>mEP_F#s~k|}ng z=92<vqye=xl}fg zeBiU3d+*Hk)&I8lJN}pVi`Eung_S=y3Hp7ES(R8WGf`j5o&Lq*iHW}^i@)9a3pWC!h`A#cu1En-Ecm7^0^ zryaBkTBs&+G8TC|5blPzvcZ0~q*2PIM6dPp6gXNUbgu4ugp!QJ*!Mp#D)y}{a!60h zRj%)-ABf|fEHl4|qtY%5MEsg_Z0kO$pig=e2oJgyK!%>eGf!hQ*;gF&a>_Fc;)zp- ze_hWj<|_5m6YBlmo&YL8!C;#^##5d9;8ue-zy8*Z6@(`fW!R@1uFR8mS9|wOarHG z{9!h9Vq!R1xjx_7uWHMbOph}9OkUVAh|YXV>*(*sf5r z1@GvdZv1@_ED^&cX(U#{-IpW*cbIL~%8Vyv=u9_U?O3mfee2qkg0NwG0{pnW4HQ$BBl9^a$zLd{~i6TvK2xqcwyV}~cQFE^hqC_&RQ#ioY* zn?3Xqn>CLaaev%rH(Uxr5aAZHwG%()Sf2U?qf%sGtnAXRXSe!3!Wh%il;_V1x~n%M z`dfdZj^XHBg!QO9F;G;%m3nUcL{9aTw@pMg%YPJV84R1BbUoofDV0v;v$6-p4(nql6KqPFU8AI_z%BN5 zn`7SU8CG&MsTx!x$29xLt7F2xrpfxKcujcVK9@>gY z3g5a`-{KS;K7rIzyN z16@5Afgx-t1IeUfpZLv`-m zP3fB_ALEEJbymn$s9k;cat+Bit_@rWxu${;kFjsvy>EiPtaHpk|^U&IxNg(88kSsOh6)|q1MBUl6G>aU}PCHW9vP(X}e@HW4>U@ zqX|II(0W;oNFfN9Z~{_=OUS*7YRHubDUXOtv$cTs-Fk~(heGZ195W&#BbC0hDb{Ff zt&(K+B<@a^q!Wgo?dBT9%qd{*ZCuDAMGfST9SCpUK%|QfGPj0*_RLcd=YcX3egaJc zg6J{tWKk}5HyDD1mnjnH#vZ$X6}ZeMboGp4cwEl67AP#K`4fO3ss~79ae7R$YDOC{ z@l8BlW>;K$)gda~giE~NEWL^K)*s&TWKvb{EskC!cf{j13_!}CH_5~%CWNEZAwpyN z8_wc3(NM0(37Rt(Gu9=_9E7%ZY3uF9J{@5=!zWiTEqB3YF;7q-$lz(T_~CV~_c4^b z#|B(3Dzxp*2wu92F{S*t|Y=s_N=brLW=^4Pu_rF}&z;xMH5d%h0v+zW|I-LrVRR+>Qd zx#s33`25f4BjLxA?>_YPZGo-dGc-gSOyVvsL)uWkT0tIfk+-^HWNl)1KD6PN3+P1Q zB|ITTiYJt!3ia5OgDd;z?*76{F&hhwG|N4{j)sx%QBTLxWUAPcf_$a zl^MzuyU%$Xw>?ccx7wTt}0gawsvG#AL<& z>qmWYLBB$7GuPjgXO5O#3A@-c+xjJ|E$VoJ_U1oD(FmCI#F8>?Z)ba~)Dw;akL09~ z?OR2@gF2I+wVb40%r%Tnf5P`PDzJOe?qNt*AYJ9co$^#Hgci!YQ>m#8Kp1Atd6pt|)cVN7WN_ z`mWU1yd{{X#;ld(_m{an2}gnond60Ucx~_6Ud8fn7!^^6rpGZZ)3r zZObdQ$4XCy(21Il#*iwRY{%@@d$V+Xeote%KXxE@Z#|2-SM74Tw5_76$yh&(aZWXe+L5rw8G|CYw&?b z>m>k$GX}Snmx>Cu+~va7WP_o4D9|lh(tSxeSIDxcv-9<&F^0YIvH;2hZKM$Yn&|bd zj*20s>p9&SwdM1#e1ArMq$&LMBqwa+baW%7%5bVP%i4`h_FV5b&;}zENt~-vk;?MH zRA*nhsjP@Mez**sZCc*DaewFYtEO7SEPMWF0tiB0pQZW4juB_5MCZ7n=d#^vW{RHT z-A=rWh8ymahx}=kTnu>}|0?mtPq(+3(<3?fyfmrr-EE1(zEoKG{rQ~4+QA-`tVGK) z(~&y%!}^a$mKCr&mG>V%uJOUvHU3#7bWM)?dlLu4{@Kv!REaXzM2$@kIyEo>39&(z zjaPDM(c0%ZV`A5rXl_1R8jxMPCX>1)*xz%sMDrtrC)D_=0Zq7njYk4g@tX`I9*u!rS zh-DDSM&K9<0iv$DpdgK1NvMTFIh^Q6XC5jEVQYJAzKRSQZ=DT!s;@*attN+bQpEl^ z*CSMBdCc)DUU-G77k^VHY@IM=@KGD`&7i@1pOSB3gKJyDtyp)m1)W!ndm|SReRCE^ zaSW}4UzlZbZ7Rzlq4P<{soi6|&j{MT9U$!c6OE7KfGc`~<7 zBw8gfc5v@xF@EPh2*AYY51 zZV*?O08mZf?Hb22D70wst7#~5tD*{43VjrR%vk(@MZLk=zIu00yD->9E(Fxl_03;D~dJ z4-@~(M#*f##}nVqu7U11es9GbiR55i*s!HptWLxyPo-+a_Zxe>SW3Bq;6Ug#zBf7< z73ByR1)fK#7W;M$m0Z48b#d zeEp;^IVa9(5RAUoHi)sSX?JcX?8xhnV{SKvI&*5j-8)>kisdD@404$m#DtoD$&Bhi zgup4VR*}jcln>%PLXc=9d+nTapaWrU#5;XCd->?)`Hk0&H!4G?npP-b6Y}`C7D?_< zP^>l9wp1`%JZoPgweya`4Ym7*Z<2GV)Xbs$t~0SPbqDmlU^HxO9?-ERda372X$35% z@AgJJ{2hq?{dII^8|_b4io(uJ73+m5`i@wR00C>S9(5eOK8kAF_}$WAKQJRWC^!Lf zzjU>iS&Dq^`59@6K+d#RQ3PM^;#%L1m~Qk;0U}-NzR{Miv!k6B0{`>0y zQz)Qr4TLf%+)&+upy(4fZ*?_#mfwKvsvff=bOkGwmjZvcei~#h9xd9WVv}P`X&}sj zt2E3eF~!ud)`4v`71Q6kr-G7YIMYXu7Yp9lT)^!2D?bO_K4e2jlDdxJ)_3_cIn+Q+ z5~mMg0k_mCx#0IRVf(hsvmxpaC#zf+ehALGfJ(}6N+Tc^BbD)oYU-8p>B?t z@J&7UVtehNY#l3L)E+DZZhKql!G5RR^;kb8lTAP*(v_lsAYQK?69xq&{HOWEns&bA zCJr*7=NbwY4#26}f;1i1>3}g&<`12e)t~=tHXKa-wW>O|2HGmA5R1^j$@S()fv!bs zafPlT1L7`qy>}rX`-i#|Oi~k|LX=d1F?an>Ayn>7uSc)T59fy!b>Q?ywh)cRN!q*!{O@9Lt;B$^NQ(bxeo~(kk12i3JBR^T=f>p%w2#4j01Fd%1$OkU-*ZNG~3e zavxXf=buy1pcDXZTmpK0#Vvuz1oNma>fD^vShGB% z<%v@*Oe${eB{^Jp3?EuF$qHoA=Hb80+KNw9;-x*fpekyYMpE)Iv+xGO^`GBoBuOr! zdZpSfuW8XkH`>=I5jf-yS2)MS zm^Oc5!JnSLb1IApIs8l)e6c&`(Y!nd|7z2%4Eyy6l7Om<_Ox+YO?k9xW)6zCmKH5u zwgP>8AP*WyX2EuPA4D69Nc6lzr1zZ1B9c|Xc*qUi?2o>Od1aoT6OKo#5lbhD)xs!nT z-?8t+WLa^u(_O+4#njSPyEMBO>RBu(YESAG0xi{8h6DhaGiivU*(cBCe?04N1D8MUiW0G%=7enlPz07={0Q4bR=>FSL)6n)y{>n1 zdq1_UVCe!cjDil^=pQfppun#W7e6IJ(B9EoNq5Ev&zkSmwa6eP-Cu^U!EffJ4i;NC zbmxI#@IbSMQ0Zp3DXejsTQ{k#Th0I^LC<3XO^vX`o> zzRg{Dx+02+JW;f}^qM9yp|9Z5NX&9~cMz=`T2-@k3(luMp*TEEABvkQ1Wq=UNxaDOIn? zo19C&*$G5v7w2^e?yTJE@7on&pY{~FK9 zGM@iR_!mUT@~a}W_UHO~l)54UdArr!U(yozDL`uvocul-2;U&zIfp$@yvP)rrN4~A zCynfUeOvZqN(d@^(FxlmK8|v&+c|CBxgl48+5DdS#w4!CVm1s!Yyo?IlZT1q??7lZ>3u0VtuwmEKCv~iMNYBs^foha*kuIuDKS>lY*Pyk%<<_u`vPK^43LQ&?{$qrQkUMuiP%wjEH*tU=SDO_jmS0|F^CzyC zbhyLjH&jv;a%+06-Wpe%`a&1h4j}-Ai(%cKi$^O2_{@_`VaO!w9)Kr=Qf-9kCDPG8Hx~>x*<_VUlCTItt_01`y_xDsu!@dlxn>-8y z+;@uH+O1xHlOEUOq;SUIE5C7lJJg%jg-&oh^xK;wMSCxJ*!Y*wt5q;!zoI_0pNJml z&&=#SyK#)XdXH~?Jz{EexbpcJymqY%+nw*k1Kv!5dY4-E!WER{|3p)5G){!l?oPY) z<2C8cySM!$n~+10G#^c}<_KGk5Ol(Nw2m{uNSvFCBXSh1O1JwJ@P;B8EJC4{Du*+W z0>*$K#ypEc(hkcX*yg8!h~)--C!~2+d-Vz-pwcw}61j6^{a+tnZxDusmyUUbD7i*K z=}A}Pl}VoC?Sj_lPlCfZMD50(e0~&@>jr8IOpwMH>^>{%W~|CxDVqdsr;`)4&>GNS|$H0LXmY!z8dn$|?BUO^Do3lY>w7<{OS#lY7PzxKEs&xRx|xhS$($4Jf;>k~gsO=HMg~%t z)`5^)OBla@?fan1cMBAOEX}UVWI~-Vd%5YOJ=0JYQzuJI6as`|>OB2X*R!k(WRYDB z!_mqWs=7vA!-<8v+By+}u*-nXl=nX12pR#?lZz|BB)}>JAyG6H`11?eA+D1Y4+M??)KsKW zp5|KgA~qP!Zi|eyUrt}{)9=MrzhI5$7<yoI3fQyE|3`0dY92z@zL zVSXy;P1Y#FT19s0+5}+LHN>@lPIEdzv(_GoaHLzZc zqqg|G2Y(Jlkw@IzTq+J;?*7P}RXl6^(N+yLVbw6i#`WM4WDxYdnx1Hp)+gCG zri0Q4&+(n$7#gZ}bp-&SmY1Mu?m-b1IdPtZvm#?E3h|W`+#TEuX~{cLLF!6;OtkND zj6U4k84%&a*;PKHi_+NS2|gv3yqHwsBbfem!81gW%Xz0y7|BoEyJxVcy zkc11o%7q{fY9w4#G=fz98411yC0f=EqO*A~212yoGoPdNRw8Qd)72Kmius^&r*y!$ki*V=^^ZFjnd%=wKfkoJ z(|5})%qqUZeO+zxX^LxByJQh2Rv-8g!nI_ezMPtXLS2IaJw*}N$8t1mj`Aq_r?AvX zL}`kro=s5NmDtKSeaMu$n?_6N?+W}Pu7z2Z_bi~Oge_q<+7Fsr@sZ!Rbh}R00Mjp` zV-+bhy-wjAq|Um-klFDu?uvJ|O(gn>HLiN=2;H7gHuk{F9&SzTtZ!8+@81`}nEphhd>+B1aJ+b}Y zPR+%$JVVEW`%-K5(wcQYrYQ(ob_5cBe=8dG+GRfcm3$<}5)yd@>8%U9aD|8Zfo?_{ z<{&RG_#ZS;9Dcj9qEGVuh=2{w+Ya9MPwnQe;V7`@B@tNvM6uNDg57+-FN@F6W{3YK zG^+OLPvb(_$3V>EOlX=HU7St__t&9!<2dE9pd z8gZazCS*b(Qf$_NFV&IAf6M9C{X}hD^RzTj^3;D&S^}hja&x(e1+?RtB#VD>m&vBj zr>_au#v*?RCIV7NV)qCk@;4U7Y!>lDCyE}bn1$|!lKI8TNnhRP~I0myrv%0 zb$^IS66=F)L>&`EUJi4}$P}3%E#s1y;0K=M$If-R*1?zq9SV4?ug^uzARgFq$iA)P zUDvvve5u|2Djsk+msU>KwBzVH6j-6}aMcAwo$a(wb~}cm)KZxVA&8$xXfGYa8g>SO z^pCe*v={XN0+FqzRzbK(#2N%=#>=X0Mh8e*+i^PLMaqpt?^_h^(tKN#me94-c-bY( zTXAo(r*EY&caME%ug{unL ze`b!Gz;GON?3c*;W;)la{u%Sli&}8_hkiy{VJjQ}+UDD^n13F%Gco8IM$H=bTgj~5 zi_O1JyFX~Tylc1#Pf@JD|5{1qR+?n-qXf?9->CAH| z@`dgM-Whxlf{Zb7d;XT|Q}bEI2MV^qmxy~kmCMH#1 zGT7&A6SZ*z?~X(O+JxKzBWSDfq(ct@geWSNn7u*Rq|C1+Cy|I&Py`gWVcQJs zzYEog!Y7ZdTA+{&oK$p9@~zhH8ECHyf|0vtNuH(z1017 z8*k3{TIrBuQ;j^X;Cz3=jqMn8!?S+kC$$LFCBv9- zo0Q7s(Gjz*?k(Z|1TSUWV%4brrEF8Rj+Ds7`!j|A0P3w1sWzgCloRm&as~d2E952o zlLZ9x#Y7?L!wTKOe(Z4dL_tecu&z!VDJ)^9X0gfUU{~UJuA{Bp@TF*~gX_Ep2)^x& zoBB%zL&i@1$8441hHLxjudk?$pY27MuiNiU-s_7N(_0j><%Lz%iTQ-5FD5JvJ6(u z8k(~iEoGG=UMZsbroq?1M)$cMjv)cNzs?3{J#88(h(R-~n9dCO3-FRe!AC349B8Vq zFt=A>as<-~4TkH>1>O?dd&r?|L^L*2T-pm%+IUgEiF|;&vqMLnH*l9OiWl&A>_8Ax zU|2(}_PNlHF0i9PZ^?vcRf^o%{*2rd@bw2xj=oCs5oL^$R2iaG{}Dbof$MTD%X9ct z(#H3P;|_qJTD}y%gU`^y7<3R8-)pI=LJ6Nag=;xA6gk!53c1;-v+ezI;e|K?O;pGF z?}xuSDnA0Op^#VW2|R@ka2H$r4od=Aq07mSH1PE$z&cSizNnk50BVZpqY+S;Q|SsU ztC{XGeEaBTX3_8gL%VrvB1;hJZ>a2Vd@pE;eFrMa<79%)J`c`X!Ea|)f?o(60u?5< zh9raP>Bu(i^>~cFXZQYkVztSVCQNIGXlq0oHOjXc;O-9mMXMltX4n;Z2ndZ8Ni*FN z9=>?7XduQEgASNE>F(WAqGVG}@BEnQYmqU;w8wonQFx@0K~i<}DT*a@hioe3eL;PA zr#xBFrfA`CJ)@C#<$|$Gc#PR`+sE@j6eL}|eB_d&Cz$#!q}zwDpF!Q+D7{h1D0 z!*0}JC`F&4k|7hi+QKN^pL@ocYRc1vISvH;Ij` z29jA4y9M-qr#EG}(%m1PQ}lg!J~MR$(ypX_=x`^Xf{?v{G>I?;S|)w{xmKh9Acr4a zgEu{CYGpK0eAG;m5?#*<1YCcNWUt_#ZByxPyv&{O?(~Sr%+c=o3>ipa80>UkYVId% znuAA%Zd0hAj^zLJUKP#rN0n0PMpNlXVO}nKLa2RnmLpEug2x!mKHK5ebWdoT)>;wj z${}$kTrOiqAzdtyEM9#KuYI1)izSyeR+NP{P|)n;r6S>VgvNczCKj)KFY~>ANSJni z3;0P~QRvSfdXi+l9av1(g|+}_4hP2Ds%FYhd#GnlGVQ{wyM8>wUk~-;VFj^=)ygsl zZ^@$r`zoKK>cddMm1`hSfqGr%`EG67h?uMhGVe>60%_eE`p61$>j?ef)a0|7q}j|{ z?X;*jSLtgvIlKmHJG%A!gsjJr>Ez!XlAJ!(UJn>vH!Nv;m)zfD-G@sVNZ;YE=mUK> zo%*pWx1*BoAyXtpl+EZa)cz@EghZklE0_$8+Cc*^?l7_<6gGu?4oROzK%(!Q8;Klg zo=~Ur9sC25ZF+d7%i(X(0o&9%9;p`gwXke;!+i>}mq})M6uTyoBV?6aZ6?&~HyJ7W z{maE~vSuP4_qYxIyf{D*b=%V6&yaP_YZmWWD*qx|!^6cAe01o7MOIch&f9s{b^?D! zN430p8F>J@ELikfdwtdGS1^Bow!OAfdtOQPxXtR9*ep&vF0lF13l(~CGlvcl(7)br zW}a%HTxxN_rwd&Vg{{yjkPH5ieX@8=jX-D_%0xzl8^K#F*Qaljh&uaClysCB_y{q? zh1asz+9}mlJtDOfPQ0bg-=x<-S-w!8Sa%70m+Z0d{Dzz4jqHp`qFw($O67IFoqs&7 zC2UWVQjZSeIV={_{@vd99d%g8=bFlJm0+yO=xBqu-paHG%T8(DZ)D2F$afEM_o$XLuBKHyL)-77U=^^X5P{ zx1|M)hfZVBajb#yvUAUl<{1k^E~Q!J=PPspcVV|8*IXg5dts@8H-9SfGkkJnXd$;h zk9zFW6cS@)+W-8@w;`w{8387yRL=s8{@<5QYk!u{@c3*~o8Gn8mMb+2KAZ+38Ds%r zG5arQAUU&4Jb&cwCyM!^AE!XJ0UUBVBEXCa`1$%#?c5EwaB~BR=~K(jP=?gVy_Bm! z5fUZ^EE&JbID3&*3tI#l#u9b7@OgzrHU)ts&WrdZ!ZcieO)>VS6ul4P|9`j-tQbQF z-n-9XUWYp5GYUL;Q2ElowF)ejeMCL{F`Kcx=UMS4wwY)02>;dvhu-jN% zyleGBPwjxUxnuKc>B)^CZ7PT&g2<2m4!(`CW!I%|!hlYUWkMlx(DL8Bt@0THH8lJR z85aYUHDb@6O&iU1^?yVEmc&fCZq#sQMLisj4&c?8H-($5K974(#N-{;Ou9GU8y}_K z70k>fZU^gL*qEhE?V@nzdGAN*UnF4{r>yR5y+)=^*s^|Wwbm)`*M+h$0Gj3n;b~_l z3O=voGdFiUpDbO3U!V>mAhuJlydL=%Jk6I(k)=&zcqM-RXP#Wm7J6T>zzbu1>oK_T zaQ+La-@)8F#LV@*+?P~3yyo*9sZB4wM8CAk^(}G)LJld#HnxvpX=V z1;5F3rD2sm+gNRa(`A2gu_wB%u$8reY2@$9_&-DV{`ww~@9~8&46$e3d+Mll;yV$w z(J;swB`O73vqr+5U}x5ceYz^p)6myW1+TcDi{B_q(G};r{>N_t&_EpoJpZixFe$cU z+Dp-2RNLRS=x2E?P~ViMkishK!Th~IG-`Q9tFD7&ro6KoI*lh==)&3^MsH1*%1U~H zPDA7D)1Tkg{hlXV7+VJv%KXv$CSg{yqb->Jsotb=;oUz^#B@3kK}(E2R-lM0LxV=% ztA=}vuA{M$ZNO+3PY_C|PVI?d{G@Tkb2JEDs$V$OGI#2fPjx}qx3x9zz5Ac~p~aF1 z!l_dCIvFmsG?mMm#adW%=ydDWPS$Qd_H-a>Bn^~|<7f?W*>>CuK;F9?q|r<3)A`Q& zEx0>@hc&GPI-XjW_6K)5r*K{%=!N%pHw;iZJ%bxoUF7zzxlESBM#t?hH0}Lg{vGFUL7t{0rd~LdD<5N zuEwUN_+s|*4*(ql$w4f#7LU5O0u?;R^6?*f7bgX^hxhfw5MLt!f$Mq;@jMGnP671d z&LZb~%XGoEa_BR>^f&ee)8ui-ne25EnyoscNrU6=pi^v(mlYw6kNT{?6tX*swspX+ zFK!W6SGph+$#s#8N_35jdk=B8o59})`Z-`5yo9j*lQ%ZhTIrC%eD0`Z98qQ29&V=X zF1iS(K7GzoycOpmR2!)y4nD{pdlYRE0^@=?&KgH`s~AE&|%-A7f}8)dmxT}_p} zK~;=_QdaxB%*iEFGMmtXL6(pQ%;M+w4^>|#2q?A|0rust%vr|S<|uiN1}_Mo zt*5pH4qW!-!-4?hHJq()54;iC+4PRs{zmX~Ct!a+;c7VAt|SFUSYoE+OWHcw(Mh4O z>F$%MY#4&!oIH^xpPrpvw7LDn(t4BA@VXYH;O~q)R!3n2zZ}ZcqGrII)SW>;vR8_g zuZn2uSQ$@4IfTyYDS#0x?JgWEAk=F`@RkB{$o}~2nspkO#a{yYLdcJJ@qG&Dq`9>< zEYISg0*GMfkZ=-8?jE0lfcgw*^hh1bcTQ0%DCH|jK^E3WA19K8p@Ne=Ri@1C7Ns%$x| zv^ds4GFzhd13d-izUwXXR&kKk0P?2=BI%~f z3l|qar60Q_ovr1Z$Gsbu{LB)uv}WD?K$UF9cm>!DU^hIhi9QiRWs~ z@jcfo3X;71M(v&}+6-rK)%%@LzD)wt9j?_11{A&8UtmywEEs_aQwg~@e*NP^EhZir z^LJq9@+LH;GQ1lbzdJ%DZvSEe)vz_toBYSH!O)I*vLbqxp&{@OrE?5REEx z)^Yi(=TbMnOWIr!YuA@;8-)-gjV_ri)XVB~p%i}-tyBByZZ+)YPh3eK%8gk9ok$IP z?F|N|FWC4m`yg&k%{Wj`!J?#+>&~6SQgw_wCx7Vopzi}%CszU*r;5?=Ki$89($7*) z4&qhFqd3x}8BGyx_&kMP?E7f~;fIy*tDEY&-Zkq`R;%EX#~z?&b{D|&*M8d+alOa3 z_$SO!_$=aXaX)nF#bbbi;*DM#SkFVZg9;42_)^W@V|uRla5TTc{uCXexQ1=BnBj*(6i( zGI893AhZ;a89$%xCeP;tSM(Hwp&?;%O*vXM*mt_9YBt?}?AYnTr0{m9(4f z%4E#j(Z?#~2#)c0F;UAmSZ~YB*aw6vlJ?kEwPC4>J`vaVPbzV?ozsElLj~r|{Vxi(wU6}0E zME8qv(L0ZyUbBWiH(D^_Z!$cSa19kfROU|c7jK$3d<`Ntx!Nq+CuqE%*c|JrGs*n2 z6kWoe{x$);j+eUk#O^47C-u|QL72`$lB4&DrkCngh zl3|}ut@93+Z)tr1{0an*+G{XHIXQ1RYuWqK{AVlf9Rgy@HD_(64ajOb6^7r_K!030 zx)mr`Ze_T`JrBgY%2Vq$9@igf+m51nsuS!%x39AtguVfhbgk+mSu~&aBV0v~KXt;t zhXca&9a?c@liS1SahNaKZ1xZUD3RRbpTW3Ws@iuwg13kcUc1y`oX*dD*AAOkOa>b- zjv4ZCM~kw&v9kKUS0y-w{E6RDL_$}&JS_2ZT-0!7F<9nt-)m z!2GuY@3r4iRE@Ad!;X<52<^XOus!g`f>`$H+*{(R2ToWKe<+@h>x4W4-iHsY?_L?% zy;D%|ED0sI{w$gIK|1Gf50ZR-4QkOm$5q0zchwnEWejs zJaKLYPgMW?9W96!q;)UQP~N$iVyyh`1Ac-c0!%Ifb3cRRjiIImN1ZNuW`x^P!a6_I zx#PTHsV3L1`iNTQ^h3VNusuh0U98&df-Zx;Le%38bx@}D^`g$eZF9=AN;SFh-8G23 z?%nCCmx-&F_F2>&2y?8+9qxD6Wk6kUwND>+&Kg&@o)JvLagw7mPTN+y{k>kU3#mQF zqYCA3<~u>OS3@|WzHL&epd_@sklU&?+nud#T%alf@ZLB6n{-`VjnQ)9lL=p(AFyRe=Suk8s>(I`ZbD-qgQZfZX5Xpb~TYFqS=_!fs*0= zP@*19`(O%PNhBTWywiYx{QU%@e6b}gQqT$d|8}imj z?E%)VzECT_&)c7R9B?+sad|zaF^@?Wu)k1Z3hUEmpUe{SLxYOASHehzxr};L^Es)X z-a$qRlDqtY;XhZ8xyw0OuuPmwxOBBhU{`yi;a?M&w&>^*AVlNL*}jD0qMyPSsGrFi z8yH^aM1B6#MZaCtwaJbE!>|1X!)s|hIky$ulFyM5(@iOq)GKESk(~87X92#fK%CBZ zGX#wLM<^D$TByaPPP*-_ft7!j@_g|2qv*F(M4x!cMIaLU3pt_yqZRr4QiL6Jt^Y*J z^5eq|^xMTlO^IGLJy3fGa<+B`ARNEVs;T@BK>k1FI7~1o^2wNUHS7$1H}sn+WpJw1 z3Ic0CG?WxR%I-!9b#)kkS@T&{P&a&=ule=xt2npygZGnb6#?piIQ9cRT}0GCcD;;L z6b|BuxNyzl{82QbQ$&LYHX=2!44_lnt_oAUnC{U6B3lu~%IL?63(c2*93@(swlw3p z2;T=n4Zbf;w|Ti3Lqjkj@_R7g1AVkb&bZ*?%}F@hOWfefPpzAK=~A2dL{1E8P2izh z0f4aY+=f+y7x1+R$A}{$r4C92#vdYz$00FP{9AhtnHLGaX&uJ|_e{{oUU~xj6?9px zn0LVxJcZjPSlC|v2oM(N#c30*a zVF&QG?a?_Q1l5yX}JrG71Z!{^UZifk@`?HEI)p9H3 zr%j-cHA<|mAhfv0DXVaMvh};pLH>Q5-bVgyreoijcZO7j^0GU3O52n5n!;!l(JXzO z$O}`)KkKFoANWj{Ty@*bUdO2&ErU5dzI4D5@bBO-oN!#=P@E9g!BsaY5Xo208A(UQ zRQC(4(HN0&2qKB$d+4Xai}N;P#bE@eZF?%@;$P^fgUaHkEK30o6QRJ!5>Dy>dVdI> zO(@S>KJ>=ao2xEg`8HD)xu&{Ga48J?P4hF@ws>WKq8=W06VvIUz;>|A@XKZGt*U|k zO@WZVj2-&>cO+5in=!dEx$rP&HLSnwOFIhWXvF`2*P z#BPBThee!VDB*SAtW+GO8?gbTYm&d>USuYD|MRbi7_>|c-%cMN#8w?_hdM@aGY9p8 zO8H{GVL6{$@#1pYdY&Q-=g!ZQX+jAi>E(Zr&S3 z=@YpGd(lG3VOza>C);gm`s(17)#EISAWsIpw@2^KQtz^5T9rC+@hMAWAaP!P>}q>@ zW&9P61p|Cs=7+>E>XY*xygjeIedl%9?esmmU|wyy0eqMf9OPo49E#a|$mnNrcZ}dk zoNxAQo}782w>_5E89%Sw)w3NwwPyMe$CB!ia8!g=R3usSX%g04cXC+P^PAtrdXdjO z$SKke;VS-5V2+c*bcb%0Ccu?tzmCxrpzSwuwsQ%xkLKp5?@FWP>q%)^shC|+yVvOf zdBzS1WVaojhOvl$%i=Fn=I9cs4H^fK@b&&Ijqr6`Pi*OzO|2B%vd;B6>-+K{rE0@K z&Q$ptnl+HN@!l3OG#~PHAS>i!o#iDoAc~az4^%%c)A6ag?mJ9w5>LEI?6leR{^4lZ zqf_v8_y*!h?^N0BexN;3a%wzZRb+si!fwr$+C&7?>^Y@`9~^opKVCf$A=848u$4L2 zV2z3}W=#(**z4PH5^?$+m!8+l`*<=I+rj|zY4%*b@`CP?I_2m>UW*Uw5qQ(QNMK-6 z;f|C?&E)sq=`c(3**S%zm-AflDE3zW79G|HQ;}^?i`qGZbos~MfVv0wmI%yxoCp*n zHkuMevAXBDA=xM_wa#L5D6`HBDJTrhAo`j3Q$FS#UiU1rb?&9t?LoP>~P5MJk#2q~qf8pfPwS`bJvWmSz4*5Ju*wJ6KZ~X`!bnt1rH6R73Bq<=L-0 z+AJdOUgwpwu(Rypi-ZHW1wm0Xab@s9Kh>t8!3THx*d<}eHv-3*?|B=3z^MeG{GP_W z{`HoIBS?$(R)p~oFR$gwQ56`?V+)zug)4l#@BVE6>LRJR6LSaa;RQV7E(4ohCVGLb zGW%@(3pr@hvPK*Zyo!1g9#!=iFY|8OOsJbk5uOhe;q^Re<~kgYOh(wcE#NO zH(9$jSU|9HGKJb?bbgqSAa9rXfG`Y+BMwZ?_e#h!8&v_Tl=AS;#PehtRph^2eq*u* zxIkfh7m>5T*7N-t|{y;_9lR8@K1RxR1 zMAsrN^b2t5#0m<%-p`9e)6+G#ZCZVLU{=ZKz1zRhG3p_3s`Knh?Tbo2H^G@nRmj`K z?e(3eBHR0U&*gJ_Ja{or<3te+QnI#LUaiWGN6I#J&pjx$DNc|gGH@5ctUt-QXU>4m zojg5YH!YumknkNqD>AiV@k_e@Rjjb_Bm=}h4`8|pAD1g}p2Rm=F_YA~{(gVUuhT3l zLZW}8^I|)dI_7OH$xU)*DPM_h6&R{R9Q+Y^h=u+LV!GWs_mx~8n*6SGjzRbD`Y#gH z$HSd~+=tkEQZ;xx9{}R9$*-j7t;wlDqU8S|fYbQ|dZbce6~~mt84au<>hH+|O2+U% zAiXc6{xGm8VPLR<09+4@rQwQ9LJ>@fL9rGWXnLn zxgUr?HBsr{w!h3t3{Vi-6oK)=#^BG!78^-2?J?&Non5_x!4FzPc8sujST;P!!RU$u z0*tL}y9L9dWEsal1`e_8Hkkujup9-gKiIVK{odB&ad{$ z>&Cm+oe7A#(f0gTG|s>VU@-gvfoO$Ytb<^WWJd2|c$zBC1$h z=8!R+?eh8X;)(+DcSmh=VXg#)Z;S_7^mh(?o_vk|K8Ni9v&zqdgK-kIy-&3PvL~_8 zwOX{aaWY#cV6@mQIP10pJ8UCwym7A5Xdu&y*Z5YF+K(Ru^G&&%6A>$vy`!b}o0;Uk z{)ys6tY-^VDkk$cl)JiB3&17k!hhKQv&dWK?iOL<=nbXUPTaS1do+r}MzpSf)3-hB zB`l;ix*ohu5wU(-->qt6-Gxrc;e5?4d$nSNz7==h0gQL#Qr+nAJ4|*Z-j5v>2R&?O ze;)er8tAZ+2Q>E4*Nt^~Xun%=G5(Vijyy^wTlV^GWfr_q2tJ%)Pw!-=?`HP0-qpvv z%yp^eF8i=Zb;P0Vf(bI52@3m=7aPV2I|83$b#2zW`#w3}&%WfV@*k>k*+X=2IirKo z0siFuu@pVoBTcsRxUq;z6L9B;#s&=#{8{)@HR`wt*(R92(r8#Z3_pXV^@-h6r5?6Rp$kh&yh8s*1O5n$(2Nc-LrMXBb5&RLumO+ltxm@M)!C7O&)a>nF|U#B8|#O z!uStlm33e41xmR|hH4~l0+fuO_e45WM#i&*M%GPz#;=zG3DyAet3T^7nR<30hM_wqj^`#xH8MSQgCqJD46R}R@N zW%;UJ3}nMWYg(k#5Abg+fd%hW^M;G%5=lngxg*9EWF`Bl0}z;em~~6b)5TKJA*Fo1-WJ znvX)0xvWAF*%P|9bbm)>GfphS_1_tM%ov;FDZO6)=e|huYWEZG=9AlHFT;FUTA%2> z_oY=1c$q?OA$Q<25prHLeBa}(C@1L`?J>Oq@seJecK=LJ{{s;w!lhb1gF*kLG|LAzOvltp&7P zHV)sm%>VTP_ubzxD8)_f#`xE{l<-lqaAb__ZQXo0+T4l~M@u$ze|>!r!?tP?9WqL6 zvS7t-VB(4|{+~=T#er1w(gC9QhlODAyyPV0`79w$F8V;10YAYf=e8qv?HO;g7UNz0 z{||`N|AotF-{8=$hwnu%J8z5~ zQH)wiMNs_kWS*@@V`3DGO?-I zr`o9O=`)!B@ggqH-|tcKOw2C_eJn;Q)3)Ai0KkI4ID~3*9Mm!1cMUPEYRzUpGT|ZA zaZ3vlQ~z~*!?UGK%`@)o`jKf2Y|PPW6%`}TlVvn9n7xj@ro>Yfy9ZYJ3z6u|BYcH(eK<=sCUW%xc(-!5CAXcU zYw&wjFOo~rJ``2#UmP?cnW_EapZLlD2=*8CXddkrr<(*YYgZVejEi0IB0Bwu)u~7< zFH!dvW0Wz-U}fiLj#Q528Ol%4x>&R0P^)JMKSxR}$4&T=++H3cKu=J~Apbfe*lv2< z|8hb#ed9p9e+^i*B9dT5`+?emV)mKyR#gf6zcSKdOAyl)9uafxOiO>D@oOnGm~Yzkskw z-*gKWYj>x)0n@DSv7o(HZyv&9>pwP6Q9Y&ThLw4Yr3=LET_4PrkySnLQ|layJd^df z9w5N?tL8bO&WK~P*Ea-vhMB%6OjD@O&`bR=*c=hdV&70@Bodr0CmGz>ti#5`N+B>w zP+&hltC}}su@Tp)1@yKZI~Fj;QrU%HYpC|LpFJmc@FyKW;ksiWpNXE^-*<5oJeHk% zVvip^ZBIp-w<4>0N{TZ3vGJdmTXS;P+^xV5ri*$qJ=QSqRy_bCSv=^s4)dffZYQ{i z;x0RaNN2f$^|L2O>JhhDlE=@x?(P$)rRm_1KN)W2gr)}!JDkxfV9?)16` z;%VUKM!vRO=QQ(PbWCanbE#<^0H4OFJkbF8-~Q6fLvxMf+*ZOKu-kq_kCFC1{k6BT zXqmmEV)DRny^c_q@-#CSfKDzK;ye|vI^8UnEt1TGg%MIQGOWcI4VKX#e#P9aCut-8 z#(M1ubeL+naXAcr_Lmq^<$r<@ycqXi)7IH)@_1@xg;(~XN|J7gq;EU*?O9K+1yK2e zMB4;C_}%dvx~!k61WuC0hIwQ^ownaT+;^W_eBpqxoyY0fUc>>e2iTShHzK+O)>4oW zYky95!!``~OD;VAP3B>Hy{3LaWLc;Cv%U`@e+d+bNLZ=|QT{ z6za-cjG-kJ;3Y5y)?bF%7`!sHyt)L>1Gc{gR>;iDXf%Xdg)Y&%pI(7x_HRD!a86q0 z_d#S)OO>S5-=&u06O;-q7r#8zCt~g3Tk9(u)vI`Ws1rr-SKav`>i>`G&R7|pRI*v4 z-STbL^etu=CRrO^yCbPk9k&NN7Z?<^?OZ~Zx=S+-BzP7`)QZ=G+we*%Y(vEQag$*j zy(b6w;TJLI4c6>o@Bf;P^-mK7JErYtF9Wou)$Bk$%hv89p3xV6fJcOGQe=&2FK!S+ zHx;w}Cqwp>?bX{5irH~Q|Ea2W*hm7ia<@wV!TG!oeCR@)N;cv8OgumbR}_u88SwVH z6Z(=~{^GAHH^F~;{Qr$Ax67FDa?n45+TK6^Q_*emw4>u>@B_*Cy4xUQa;Nb6g$gX1 zc0G6UtN(Al_J1_Kb+V!NSJ`AvqZVqiTx--5N8091;HBrp&vz+% z;f^6OB4!M(y_1f-<|q#83u43|@4@E*SVPYZ`0L12f1OF*;Qt?^H~#0>FY8%`q8|ux z8m&Jh<}IF#TAB7Fdx52zq%O_VhO3b%wJo`#(}~~8Sr{}z^W^)ePupNiRZbLh1OrNY zkZ{@^_I`fm=hwf31y}ywy=NaN5*r%GwNt;aEgMN8q%jUuLH*gDu@u6j7Ao0+MEt5q zdSpceaKbgJy^w?DD$WUHl#?2blj6MCAf9ngGL?;0EV4s8BnTcnU@9K1aI@$qFDuS? z*xH{aPi#xo(~-@wZO~pHf;PR59w!cau4*Xu1?p+FGV?5*Qeab5ECJ-`cd3!RHHlZD zxB1lnA)NQ}+dMm$_;n9W^qmuDD_k6YKfW0TA7Ar$X7z!=o=kA>C7}lr!8-|wZ!awf zcq~*^RmWYky~A49v9tZXEkxUoQ?D|;ElKGk>EiRu1_kBEs@b{V8mk(ji1#qP8E(PG z?m!X#RMOf~=HvZva!?7I2U`(p+s_4ZN@R_Aod=>9OZG+fvhn;X0gUX#iT4|Rlq!^6 z9=`!~-S15UPTS*z$g}M)R*E)_TTzE4niCsgmh+XLc-@b6TM#v_>h3a;cIZ?Tu%R)5 zP06|zB!35Q6SBRl%&eyeG5aAT3a$sl#U!K1IQtIzY)h1ndiHCF-{ir6T`;+L2Jol_ zN*q*yGX1J5rNU&FqxRzF>&|4Bw`TWz_wkqA@$H1XkI5e>nWe7ueW(5m_V=Nvb~bbZ zR3UTyulNzG0c0&nz;B%oH|f8P?UHc`+3d%&M~lWPIBF*9cJ?k5tP(t%3$I33^Iupf-{dmlgm%`1Z`2P8FGGv}Q13sZ z&*%0!#Zl&A5F{i0?W%70@e?t+*-a1f66MFoyurLts&uQKVp8)iys#J+>M7EWq2xRGCKImq zNWYZRo|I#dfi}oCGqX$hyKHVp0^Mvm+u_w+>$L7~frJ_@SGI2yUrD-PzARn)+@@AL zktDp_GS3#BPXJ$9;pOp*TAD z;3+*k`1u!Q)-#OH1W%rA*4`7C&qv&Awy`XV{}&vtej zyORdTX7Je@Rr^WL_`{n%_?4hF%<_ujvI(LKbz;~af3^bIIt~fIW2gjT8@mPe-a;bM zxlPfuJ2+p7ozry=wDc>dZAs)_&J@^^l}2fe9~adU;&R)}9&)*))z^IGS6V55XiwU( z4pQh1#09dPR4Ib0!JwzwLgUg(n*9K~r@u3VE(PCiLe>&O7y79=4+>;k?$(P^ZtPWebG>i9^QV84*3H({MChIJR5g)fzz!+< zc1YcWG1i^i^XeM?$O~Tip6`sPwJSm9(%MUI4E8@U+=9~bjhQ!|4Rh%BV_F^o3V+Xm zX7pYV=M?7Ht#{BEU2}KhxKFG2E{^vMnnO|3p6bdCei3fkQDPd2?{I%rM;neCGJZBf z#Jp!(8z6irHgKDRfH@Fm>ym=YOXZBy{9}paA&w%Vn>P3PO42pX-|GYEW-$N7WM2t! zkgwW1v85h<;BO0%ETUul)}NSz2j=bBm_w;jDGU#~+O%pHqX`|%gYDhKVbMxD*;0t~ zd8XxWN(W&^F(JfF3rPyB5JSoR_o0k3^c2A^%$5)`6Z?H+hWSVUKWPM%rC@h~()$Y> z=6WZ3m)|lpePF#%;wI7ckg1E&TY!_5E0MrUt$H@_0KFKJ%CZxFD-u)xnd-fE-T&3* z3{O3!wjaMGDf#VEJ|6VPo@IOOezb{0kz>_l0WhY>eW+2FNUzMNZOp=O<3{*gr4uPh zzdD*o<~Y_$yf6qr5MU%8{pZN5-v2OlQV99xGg_nL0(( z2Uq(USmD^hL`-b=T$;KnML749@F6BBYc=*jg*n5KPKBuI89Zqxdetkj@M1F3uGhu( z{KtrGd{|yQhS@LF;Rriz(25?KDt8u>HLx~c;;-euElvPUNnIMzMF`v3Uw=ac7Jco< z$AJp-bK-65_%M1!EVEy9bw&4F^kgk|SB7unGjRHTH8NnccD_me<7IM;|EA@Ko%D-W zoAOqJVukbtM@@ABsV3&RE-l>&TxCZ3eR@*sb+=7p=2|{ZKT+zto>}JBrJOU=fAAZb z3Q3q9KdSl>iny!Ub9JWfvoncc+G_&pcsGfgp1jUmh>L1D^c#_x-6|33efTS#z>kUb z#y#Gp7|a7SLhiluj}0@V%nf`(^_ad7Gr^n{VdwhO5tK6y5)HQVg=q~7f__VT-$n{v z-Nwnc}Y9>_f zW_vS2ow2j7VJ~C9-N_C|CEMbiYCU@o_hg7XOnzn4iWuT6#1Pp5QR*i5Tc>dBWq5g~ z%`6hI_)it+G29E5?YApfx3f<23fVPGi^9|_SV5k#{;gQy_6hh>aSY28?gs2~0Wz(6 zCeBUdNbL)kk1aL2wSJuS@mxkUEf~eS6iyAw&nsii?lTXP7Y1QX^>irJyq)}I7|NVeVf)W4?cPsw)6s}&O%2b18lMK1e+RC?Zh05` z?E!4d8*Kw(zhHG6vj__Vq+c|=9;~DN=9t&W?iW3E^<0alcmKn4^n1cfb&N$? zA~|hv$!__RG1}?T$`0QTyGNH@$ACIb>T|~0@pYRQ*G+b7ky>)X)r_WW_MN4aYgR4KU*~3qc#(JDrWoP| zs`eqgQ`i}8d9UwbNzpz+0g9NvX+b*VSRdww?mokf!s>bgYHZ3*Da=M_ey|Y97;swPTjjn#l{hc`g zwkgVYdP|-D!~?Mjs7*N?tt`)^hoU z(0q=r0RMw2^XpZQY3uDUoO%*{Sm|f|btk`v4eT>*mko(4QjTJ7w3~41C32PU;cZ(7 zj#7gHRrLaT@599auk9E$rJygl zeZMq2Kw`30N7Y5o0Fh#rkZpwFELxJMR@*eNlCv#w{#;|ooko6H$?~UfFlbqAorjq; zQ7EXsK9F&foL+_0z?M{xc(zDdk6zB~UIJxJMb!P8ocgw!3N|M+A-I>JENhB=lM_R= z%{!nl38ebYu(LJAdGClR9E2*J{S9^eaBtmb3CrXaZl6LD>8$s2?tyZ<)`17RsP{Dl zPInrSjunD$h~yW|5R?9g7~`8#uj9Y@ARa%Q&fNODHugz>7gL?(crJ6ccf6P<9Hr%= zpXxvWeum51!#v;nM!bN<{GY!lIErK9ORP1c3!`Ah{vze4V||hX71xA^x!GC^LQ}u= zqgQ{=lb!AiG@AG$PgH`q3-#Z0%QI(2c({>mw|;jaX@cX;I`~`y)?f{NCQL5uHBI=> z=ab0Ne*bWNLI)NbKlR7dQ!T3|n`o_QmkUwzUsKsV;q15Htx54C^Td{_7hv((Fa1a} zCOj?Y8An@5ady^M&yzZCc95+_VaAcfW%PYpJdoG?urn1a7Gc8FWAGCfBH>#3I z`j1HzebO={If!d;lJ^H?yJtq-gDEGV5$CvD^b?ZK9E|I`wb*4qk8UZ)N3~UwaSTe4 zWmx^FolCGW?$6sM6t*dg@igV*GwazQtp&Q8X!q^S&tNY7bFB(`z-7w!b0Wy$0bZ=S`8K!l1cms>)}MN3571p_G2U->B$LMD zf!pX`S=bc}eyfttycOloXZLAnbTB7DRcJ5+;;oy`KT{`*H&1W*Xk;xm2ulV<`w2VI z#SCCgzBBTB&$e0jWw?-ICU2fQ#XvX>Aqo=-@OymZ76nGjJc+p2UaBt)qnQzJ2F zZQ!=B(SC`N47UQ$HYPX}7j>3$ennImg&P=sJh`|vGQ0$?K5^i|wMs!*ckBtn98$FK8g+Asrp_)oSTHvc3*1z!m{%ey>KRJ-d8g@@~pX3 zwZa`JwGT0`e~4zRnn*}W6V=`d*ej*?rgARB@Iv++&Nr%R8q9zvYwgPrrF0>CR%45p z#?5a*Y2-6}*1jy%93K2GaykZHjC}UpY!<8<=H+RdD*3ETORS&W;*SXHZUA~41CuZ# zpC=py@4VFk{Mabi7L<7bn0_L+Nllk(IGW4LHgY3@9?s@mI@*lI=9Xz8ZsMj7(KE3< z4KGm`q26kPQpS8o1CYhY(9-ZpJ zfTPE7JUIkC;eo2XW#e9I z4>@AV5rmR=X^o?@p7qxJtsmEhlwPAJ=4b}baI;?VGx4TOZbRW(KVhMhYo21&vo)Yv zzDbI>)(14rW*>*%fRG3#YUtVe%vUynh0!GVzyqv7RlX6neUsf%JL*P3s+M4%v;o;o zj6W{xU9e8Dr%tZ2a^_P|eu-I;-=QD7E4+Y2SA~IA4_v<=I&5;^co(QE^%Cj6;vz?1BHDN*%tsSAE z=cxSXMLKUXnx35_rwQ7UqCJvBw$T^w*ucz&neML1mAj5Xmn1`zHSKW)hrsfHs;!Cf zgXT7&fD+%=|5)9D%lmY%D=%cXp!RSa#c72A18HK4XAEsgw{AOCzk6&v*^lNKn+TP} z#H+kaSHK?5S0GJPCyJ@2BcIsv-8+&f1En7ozlaAZBF!U7lbO_>&?+^$b(*FBdchev z%=@*9UV)Hs>GuO|!K;f*#5fH9V;pyhF!==LDI&OMkVTKjCgN!%CEgCLbyW2Jp&H0g zFF5NlJn4%rEb)h=D}Z!Q*yynZ`u0eV@WFCC<9&kD)WM7B*Zr|nie=R#i31onRQx|H zKec=gLI<5!4Hmq?eCkh{YwK^kE&R7R6-qe<7hPWAh9EAHlwit!)s)ZuAeYU`Ks>V7 z;Y7>BrrjLqUqpsBGrPZ_L3(iJsj^o-0k%8#Bkfpj>xkum68Zmb1)-7b4{}01dB3=+ zZxEHa{~9#$kdIH_GoQy#=qarKd&ubgw~#qEwEl z3j2KgnAlyu-HIbM-ks;OHDMYuzV(_idl;v3nD|kmv7heH@vdfg;uQ|nRDt;YS1-yj zkt6m)m*bj2v2I5vmlzg?{4L2BQsQ#*D9GqJPaY>rU-txlww|_Om(8P1OD}Y24Qr<` zG-rHiJCJdis;n`2eF)0mCA`U~0NEm0{@k*MubhDp>>3MUaSvsWPSwRxa9gV020E{i zz0;JjZN}U5;^Y}91~jL2=tXSDT7vY;`?R`pfCK*LBxh z?>rC)@+TDX0UtQhlRag#yg)tuOIqpvwbHbCdwbI8PCr`UeRR`)r>@UJok_+cmgVVyx zHP4Y>DBB^Ix~Mu7tyDIeRVtsVDPvX3cQowu|L_XsJD3D$ zSGU4ipnHVHiSt)FXmBv*MW14BOEUS>5xj>4jvLKKU5O;lS?WvIKAdznMhtx$3O4R_ z!k=bdj{l$xh2R{ZdR}cTt7e#9GMKU{4kYdCXc zn61zd@Qx*-(Q1U>bKil|1?&qemYXJ-GQ9i6X4ZUs!AJC254+^RYQw zzEwgG5;glW8xNx?keqx5oS0M17+$LYkzfAsg9CtPaHExTZ13Ae%jhc8tKX?r;Jh=Gnrznilx+71;IkoahxH#_ zPlf{~Ng;v5qoW0gW^8I@Zy;$Rt7h&)_)O8ER+(bmO^+vi?6!)~pqzS!<8uiXeK_ho zyqG)B3f6XiJ*V$5X#zRdue0FWF+SF-R1$0kSWEsbz#Z13X1Y!dp5y2sZBGfEGwOY1 z)xJegrJ^f;mOs^D%zrkg;S}K2i z(Yq|qoFz>o>?ZAFBe64-T(FY#i=2F0l%x^cn%3BMv)|Z%qV*u}_z`_Pxh{++0p$l0 z1(ep`Sq!#M?|nb3W3^bj^JAHAKq>z2#D`KlgP(W#3AgaplF7ez8*iH0PD2eRs3a)u zo5TAfkk?i&vvy`$)JaS}a6_wv$6}1jtZZlvC(z-aA51&Y6|OS)CA`^_C(t!iBf7L- zKeB|!pS3^IwnK*$dzo$5{O$n7I91RvBt{=5Q+}647@qrPqj}}lKHPZ6E2aKpwWC-J zl{hY9EfqpkdgX3cq~~TcU6K+Yd^J*6tWl&=C|QeO9xlQRQv1%KC*7=*e86vJ6e`8* z2$k2--dJqf`B-Yccz)yZ8dP-PxlKPMkfPz~`$@?v!AbG*bG~Q?$GeCZzu58ZDZOOkB-FmPh{D6JZ#G*b?2WfHyWo8&@I)Q6LBDV!nmD#6A?K*qe!SxHK_SaXpIdvv$=?u7isv|Eecku z0;cCQ-=YTR>PRx4{P)g48nk@XI86ruZ$5Y-))S+*B^A0|UHVQwH;34TxcInRaEq&sqC;{efWJUUwkqW8M3U z-E@b--O=>z9yP_bb(@ePUA6+O72ARf48-{(&;EOELWI+oPAAiWv|!#aJb(%h7e$lB zsGctu(-m!btk%uItspM*s!mZJH>M8GS^w^+zFNcKY_Wnl_3yrfpu@{q7NtOiHmjfc zj5!K@%&-5qZ-~@D*gIl=m_H@jzj0THiVJpE)#R#TMk!~kLB|)LxM}vE6$#{;-Pr20^jPw{7h;4}rr^4E;?;@+ zq|Xs~F;$s%M-hU0*5d`m-<}Tw|7malF+lZZLR6#BTs8OJSlqmuWkr+_D$+Db^^QI# zeqe-v<1^%bht4=!zX1>H7rvlsF%g6u^wf*@tyU|1*k0D!z>nbq^%WgMZoOV z`WnxQviu|Y9oCSIY(*WBw^Ci-`c}>MT9@CPk1Ur))MGNdnr{lDxT_-mf;HI7?b_%1 z6D6ZEB7U~Yeseji2Jx-6(NgvkM^=QEM23L>)R923gLaZ;_gn>niNQJ$i0jxe_@lvg zZXz7uFNY9V&Mx85MuSnHzdk=mb5_pc{R+Fi15Y9p7FZalWCjZ+NTvgE&##9wBztcX zhKt!Xo=fQ{AzGL-M|*LJw)2&IfH#H8nF>{~BQ2h_Ev-IL?`tmqJ4n5)SDzQDFvieX z;=s!ftAd7DuL-pf$9)FRih}`O+Y8R+5L~WU+Nd60(k`79M3QXHsJ-dX6$xAm8uH(S z{A%7w(pOq|Tv+S;Bag{}Pr}S#^^dFVgI_~3TyP#yhdM`zyZ*7zv4u`S8{GJQ z4(5cNuge^2W4f)#~Pl2o%7$-1`^L=PLA7e^uyhg;Xr*zMmKX#@Nr6CQ`l3Pn1)zkmATO{=Nj- zNrsv)!#ix@h(c-3tRsu4?XD8FXH6jDA`Ii~v+bts-cekBi#c=7C$tesEG zc@TCnyRSr_KWT(N3U&KTtg|s_L+Lh_-u7Xi>s;%alFWyKX`td5OjHpQ0XaPh{tzP@6hd2!d%^z`fA; z2-ybZ+E#X9t~E0xg``@^0G$-03PN$CZwKtzD;UxW6-m1EBzx$-8~Phk^bt9%z+-8| zX^DohW1>P3S-hH#(ircTsSv&0-EMQqIy*R9p%PBRC?9+G!7bUFa6-jyC0-ilplOr$ zD~Q*AOd!;VY_&U;-}zO246ngw>P0XV?0i35o_@<~b?Z@>wy{|D&0K0Yn2OtgG zqPS9SoT))i7-qgA53&lDd`LCD;7g8s(J2GFXCI#)Cb2n8ku3|#pYY;uo6Lu`?e4rI z=QgkpdohW)2DyXWQCly%af=NWon6O%?WJ@X?Bxkf-;-sa%MR^0v1)i}z)4s&d$*8Y z2#i)yuJYePni-!X{`ieNJ&Ci@Mt-z^-okh4Z*r|%&&jAHv_XRiUL?s~dd~Vpa&SlP zSx5R8mES#z=wrF6In)rf*3%{1*8;F3s#Ia*=*L7KS`zc z;4j`LBQ&TIZ7!i_xci<;mzP5}rxXZ50x`rJk>G5;J=lzOC*uyFHZ8bzd9l~Y&HVnJ zI5^Dk@y@ni`FWKb2&x)4Ju0yJe7I|C$#D_pQMH^YJz>Fqd=@mCZ|mE6Bv8dJXkip7V!M{x@C7ThuN z-ybn|v5jS&C06P~yFb}I3-yE5jjwlj!RUlr^;lh>$SlN^lpF8u-3G z+*(VGP;c!}^Vz6yd~c}|QKl=*7G?tRWhrP#5LqB{E0ALo7@FB?HS^ne!MUj~<3Yf0 z$yKk#_ce!JbwnDU0~Yw@@Mt9Qt-TgmS=hoyui=cO9|`G-H;1!5r42%hdz=L<;vS-(nJc-kxh$ZI^Lda4E1DuW2Ofz(Ur>TloaMH}Rxbc4D zeD!qUO(vobe=LOctzA^}F2NVcvWgIWNnHDSZk6UtX&kCKfyG8IEkV+Wki`ol<9<*= zORb~y0I>_u6L4yxnA=mGx-az zMZ-NsJT!~)ijDK6iUGKV)-Dj^#dRaAahR2qP=sd{*(N3?zJkO;A6Y4}FZGT`hUnBNd_3`(7v4m4GdTkgy;h zsvh_Fct3n7YecRcX3tjIhhfzX7hoITA*T6;pSX?yQ!#igeSTe2m>m5+KEeJx zbMwvkQtaXGLdXdQ^iAAx=W;FB$ZS`8itVW28p4OM!FG^ zP`Xow5Rj6V?v{oDhJl&6+xI>1d(OSzx!*m1&NF*I^Q?IGo@cG!T0dfZ5Bqpcmp8J@ zt~6vg)En<3oF1=J#IAeHut)oF|b@Qfk_S<46sn3IEr z4r)i*@ATZdd{UV0vdDCqP@CKUk_vT6`LR^-MV#MaU#V7BBUy>@DF%Mr7o3t8u74Q8 zX2}Hu&L#a(YNO&hxRY2QS8Z-nl&6B2pbYYI!gU=<`}!n2t$XZC_ixNyvApHx!0+Q0 zxXf*W`ay?3fhdjgkh=WjAkp-wz1anOQq%);8yzr@sY8-EjT-q)wl|i^U05qh)P2Iy z8d3S@U7l_p`Pn{C+^0zq|ET2dF_!UBR}tfhM1iiD2j-5RSj2WQBQK1QU|HR%;xITc`3JTl{5s zU~W->8KzkyMg?<5Iiw1One`~r;jWZUu7YCC&;or6>io?ockOrACpLM`F1 z9@qK6AVhE3gKI6^1TcLMW;x;dT2k*1SQ;d{2J?4GuCG5qedzn)$ldNCY?0LW)MyL~ zU+34-#k^q&?&8#2_m21wc=ng|_2`UgXOOk?p}P&#%m4dx{)EOZNWY>OCnOTqvKxE- zl0S6h z9gtF0kUTqIo2?U6Coh+EJWVYkV2gd^{2NoRSB#ow?#z^IZg}Q#V?vn zsKyQiBFGe=2G?r(ms&Rq4eso`h8;O)UYnT|C5YauQ}%q*nrbi=EAiC{?A56uZZ)n{qnB5en%aRTsQLhI^@PRND6j2fX6Xu#jfe}q2b5y~P4U?8 z#!2|zO2Iv>w~9-zb491Mr_)BVZ7Jt-zs6U`y_L(}p#C^j#Mx*ERkh|@@o7kHxuaXD zok_F*6V0U%tHF(uH(AIDd#x;U>+0CHIxtAO5yn`#~A4IMhuen=ke90^;9xrY{Pu z&M7|pEurf;+vkAk6-3Bs>Dw)TZqZkm|ExfQtg_o3HK1P}H*U*BmkA-|oKzZW-WD2M zeCt~&aO7<+nZ-n2f~A6vR$yQqzSts}P~6Nz$j>)EI5evK#l)o>RULHlg^IBeqwSBU z?iu=wN}M~i-UnC5zrfgo#G;-Qj5L@QSQ@q7w4(U+YiaO=+r#-p_6wSSd`9vTJNVy?L1jb;7t?jjFx#yX&uG;<`my{7Xy-be5Q%EvJh* zutwArpjJNR2P*+>+2T5ik}~zIe7$=|ENRkqk67Pe@UqZ{z6&;6gKvL&(Z&kuOtKxF zYvdEIhT3qFY+VO#DU02!ddJZ1&N%vmM*a#TMfK4-c|VOi@6g`q#r0B8bu8HJBu5Cg z^uUo5+OJ-Pse)8h`dZP8IM^W3Tr__2ASZ_k*NMxC<_N7>a@OIL)IQXmQYce`$H6RC zT4%H1hDH>ux?%vd`y^Lu#OVff+Kw_Litvo0}v*}kc0H&u@l%cNkEo0>%XjC`Zw2m;kq!;Aoftu4oA3?0*b zU1|+>W~n5r@r)7^X^z=nRnjZXhJ#wYC2=k0xU6CaKx|7lD{+EqRNaQ9qaQ^o*NE05 z9JUs}r%YpT+pkL0vlU6D;g6Q%>P7~ZZ`Wc2E-!x!l*FBEz{VfPl@j!A{6c>-G8-6K zqt!Jz75)=!{P19fztw0!jWrx8=s?s0>z6`N zH05sFp0GeinLN-*JS5gp-zorq8S{30=ETyid_KboYItghixZs4tD8Fb_T^RnKVu!+ah4EC+KV)~o@LM+(s_h#z5wO-r9k`)jeo-#((|4UXvAOgVC z^wzb`GW`4qd$N*qX+#>Z^|jlc^%giKKM z*lB`RGw9;@1!N$=q#dP}FV2Xv>Db#4b)NswOJ25!sMG8nw7TUy9GxW%H}jXAud5ei zvN*F{08pJmmphmJ8>DC#y*3?3hBS?(X7txy$LNJ(Z~ek%KhAxqs?OV+^V&R{H?1L0 z8TynTD-|xzY$d#Ioa?pz8D0yUQ~YM_>}b^@V8)we)K;FYDa4-FJXe?{koJHyS)T4I zBhjotM|mE1d7!%ZU~0XX3VwqkRF|sFtpu;7mao z+sC%{pEgmpzaPFcf8Xt8FQaf1Y^fA#wZYYsFBh~LKh3jGCR9iSWQ9S5V(P|xOnP)o z6Eu=uL@&PpjR+@%ZcSH$LAX-QrEk85WEy=7#&0J1xwH232B4C3p4fL)CEYa}TO|5v z8#z~9e)0?cW;O)*Yh1sQ<`7i;p?$Vp-;=KKp*!C7eidG}%q5u^O@2JZFVWSPlvQL7 zi}f|vA183QS$!L{;<6S==*`JYj9G0bw*Ztu?JZWI(f8|L+4VkauW_pil>4v^YxovVFt$var@@>_L=|zMc>lQ>SHqw6mkJT+{C?2>Z)bsv-h5>9^6zzdaEFh7 z9We!%aC@|WlPNH<5dAWqPKo;?`!8MA-Lf#y`Ym>AK%1Ax(#1?6$kti9?-go6k*YyN zUDrTQ{^IYh;rWUG^|IvprVZWlI$PW#BJA~?y2e?7K&rrRUgc$e+P z>_e%?W{x3#&F{Zc^<~Kl>ughZ?BzUoT#oVguV0GBCs7Zb^Uavrkl2p7aPGrPpZ-Z$ z#k6K{?X}Lb5gGp)VjN5=EY(;M;Vt_eNnC16;RIc(_Uy>MKCvfX3mc*Q%~^=ob+a=6 zlt)Xi`@+Y$LLWv&v1)Hi{^vCLHMl6MR4Z@sXqY>%jX|wx>Ak((bqi=S!6@LhcsSF0 z!~=KPZ{uru@o(_)`o1(%7NGi9FzrG%^wQPa1;3g$LNCmC4YN}97EL5T(+|E4W~_*m zC$n=PP@~DssVApnyOH?AClB0!0P-eCAnFRnd-vXWr|2jCaTv52b@~`*c*Fb@k=&m| zVFoT9{&8>n+jN-?r8M6`XGjFElzybB5u-(@(PjfavK_s1`3&al=I)+&x9+38HN6GT zo2%D13t{-o=%A5E=aYBaW6m7~y(-k>2v>^gyDq8vhq}avp{ToF-R{fS>_tqWp>zkm zzE=ww1=-AN+k-UGkutViV7n*7sm>V3g|pp}CIOOO=ZM#*fg7hgbnTPzy_XJ08$Jyj zuu(Xoj<1jD>?U+_8kJw8C5BEpjbv;)Q1qJ*7L8ch9cv+mW%dV1{_0#QNH7VC(lE); z_-fAjEu!Oq<3Mvaeub0T$mQ{L)o8$GmxbFg$zYAto`K(JN_oqjZH$|_>H{Hv|_xlP~9+5T4Vj@phl~PZ0A=}+~t0`rB zF_KuIH+{nD_(Qj^2vPf!e{?a&sP1lv?nQl9LN!~)rrFD^Z7$Z!9g+x$PET{_%M#JG z1^iu)#*DY7p7t}n)OLOc;@;+qT{GQWtS#+aS7aYQbl;;7!-?6RY&&l6Pq>FdUM^I) zU>!m<7sAxNnHZ)jdsV#0yA^0QM|wBsn@$QF`}{38PNCnCzikiAA{ng$%X8yFYyS03 zDtW@%gYzILuTi5vZG~RV-HQ`mou9Ek+~5~hod)!fBwx+ogeTxN1jVTw4ff3T$gX@2 zmG`IT7^zYOlciDpI{-c;k+Z$_Qq1`!u-)otJx&Inc^yFI7oL<#vK5zEB+lV-XJ?5x zvZ-~3nPi`lz2@N@N~uP-phmmD3_R6^r~&P$?>1h0sw@6_v6gCg%e=h<9r#?N7blUs zJ)AFR_{_PN<%Z#0q4o(Ih2QG_)3O^{ozs-!%O)lY1Hx_oFLUnO6Q4%G&ysaxYHFi& zS406g9^Goc25j;sbEm|T&$)0L8%*xN_OR#tiZ_pGIDgy8%>>%4#nlOJCm?h9ZGZ?>3)$_*#X ziSK&*`0aj0oK7CsPja%CC?JFyx!~Ww430`%!f#>7KV|TfCkduw@mO5T;Dh$ebAMze zs`@=H#c_@zbEw4CUNv>ET+$i7Cr6fem);|GJd8w>JAe73_++}J~qO|vH^ zdDWY;ip2Z)?Nv}z3h|W=3?#{TJ^@es;vX^+c6TsewM)~-en(NvEfztPL9ynqo}0demL=Vbon^+c?%D7kvMHEI-j~PTz=^J+Vhd|wC3C0+=z}$QUk8TtsUvC zm?5A6{v*yjc>Ys&Hh+Dm?w1*KE(03b66-x`hVU8x<~vuDadfA_(m0U3f#)o={9RVE zoZm1}daUT9w&RHpAD@n!*#jCpeh z7a9`LQhS|&g-Fg!j7YRD8>x~q`>dZv;L*l4>S}%;Q$Q;W>-jQCr8?-?tJHG6UH#PL z#aKbWOL*$VZem~Ln`F6cmb`q>XRtn<$|+#kcd@{uj-_wD=uKID_3ueEXGipU>ur1p|#037O%qJjrLA9bdNW~G#8hac3 z%D8y*apf?1Qh4^O9QK8|M+(-!H7oMMO_n4bF}iu3_)FyN9gM@EETIQ&L8*ht+Q&86u}O7w`<2*L(wjM973dI;`2 zs?>V>D#i`Ki4U*~ZbgTJ_4mwSrt_()?Y!H<3^4Ssw&O7&qBI>oh|@d>gnNQHY7PYs z1o!0Q=0EVY1{3O|>7AOfAfsrN{TatkhNlYG08y$+hIWdVjcpz34?x|?W>RoIzz)ik zItI$cCXuHxjiQaC9)%rZ3i0{SKwq8mXs-{`;+IL?%9E~1u&#jtLOQAHphBMZGMX}N0_W3l!exJ%Ps_@DQ0g=L-&F;!7JZq4 z@5j>fH=a1_OVDvk^R>W*TFDLR3RS2>%tGq+6ZX!A`1-$%#8qQ0r>|WJZlBhPyKTDE zk&6cyVufA0Ui;efH;w$oa~U)^Z0(Kq%mktfy2>(qo=$eb(I8O|nv0ihjvsiVsm&BN zQgu;08iAF<7GzGt94H{Gnn#2=rmva|A_a?32r^cC9&?m)w5&&S$7O$@)4TXJ%2V~i zV@BEus$O*`d}=iQhxg{rn(vZLBUfPD+bihJ!b)|P7q(+J%i`LJhVL=zN^A6JX+%^| zJum;wmFE)k=cfdOUsn{zq6CI@YcWmm^=bZN43OKLYNlKxt@9U}MtJpE@wP z#O7P0J;&j>2PD{KaIelSf~qNce2^W`w2?*UJ%3+xv;RiI89VqI-#Cu6>gLaN({!&Pq2=Ss^I6X?2CWWkND6dV&*0wD$LscBM}i71I0Vu4?cQ(QuPW@j-!M6#y0y~%Q}=i>r}(52swuAgb%RcMK2E zo5B5wvG-DgmCDpTf7|6C*8PpXgu0=j(~ThLaAS|I)$;r#YWK*BbznThRHcnzg1^E3 zom3Y`5z*!=Q_{qEFk-06X1MtjelY`~h3K_UpV~NV>_u6ZN_LkT%SZh|JZLycq|SfP zae00diKhUx?d|Ne;X$6x^#P=sd$85SoEsW+>-!yJZ?oFNCw22l=dr$`n7eUJW=wtu zHPXgrtj2+t5F!6r0lJ`yt&Q>gYSM~dh&4EX&dlUJWvzfvKBZhtN4EUhUdNujIff-1 z_GyifCPKeo(lyvr>U?zPyK=t)LNb@^=K#m0#b=-TQ@{H67j*5^^OJMdxp8m76>*qp z?LzMLj(V8pe&_lam}ztmsE@TBO!+Eu&oJA!cT%vQ!9vG9KM31`pfy?!X3vMiIoPQ( zIZAPLTV9Ki^}_EGx<&6e)}xTQc>$Qn#?6}5cd=O3-OhrT54jQ3E(i7*EBeL)kEd`z z{?{8iGP+qP!kb&yR+HP1hnB0pr)mmvV^2ATo_fC)P*zQz%ikMt`8>+)KhnGR;t^3b zF{~ytBj67AlF!|E{&+A=pjnA(rfiR3OXIseA%6!>XPx*n;i+mFxj%7Ny1aV|3kDc_zP#7#oxwNM+@G#Wpmcqi#i;v{ zR4v;lD-ezH0|;~`Mo(n+cXTm1SwlBZS-~d$iU@-JeD5G)IykFZwsfwK0zC#T4sfw_ zJC5s2#W*YZd0wse!-pmv0e^a>0C;{P*80j%&|2XvS$os$GfA-h_cfi(Sm9Bj(fl|o zEq;7d5-&d{_T+7JJ~+E;_h<{u?Dt0I+>B3}s&OtRz1gzWD^m3y^Y81=hr#F}nrT6= zw&wURSW0@6_teMv%b$m9)Z$eJicOOW>@(f|xrE^Cez@EIBLs#zJnjOz6I^QhTb^S# z-O4=P2ldb30eh2>M7&-j=$BY$0%Nm35ZM|^sG7kFWWb$0`#BnP-Q#|2T5L8QUL$z9 zh6lSjbPlD?gD%h0Dk<6diTHDn4ID#mHn=Sl(_v41uD8;$ZxczY^;vhse=|9J+x-p-Z*xN$=U-b#U5&m2dK*$}IqO>he`{3( z>jLVOkG?#YQ-1e5i5u+Sa!=2UK?<=N+STg0-`;*eh|24x(KNVFNnfv}x19RO@TfbH z^VRSG_oF0!f0X;mkK>;zd@U?ND}|qqzn0)8JipA)U@9s}d3ZrsR)1brznZae4Sq#s05&l=-!y0^YOaJ7GqpZ+OHS zy^mMN6eF`%SZ47bGE5e}bU7b+26Wy5`KPc#ZxaZ>g?p?bkd2sqBatkMi;_w-K@Hqui2XsSI3m$( z{P=l)GX`9cJ@@h%j_uys_fMHp=k71QXceBjo0o|BklMGt>O%HBl#c-kX;s31bn&8) z*AGQZCeZN7*V(CtyJ5pr3d__G5{Rf~Oy8FDt3hYN43oi-KhSo4@$;v|2-7=ndoSkz zqDERT$;tW;06>t2#26&fflJi2sE4OXgl+Qil%5B~r^{qGBc#{z2vszIZu36=?04J- zYV=t_&`Ulf*{W(o8w`%vMomf>TKL*WkRPpGgWnA1n*^90JxGzizSbUoz2@N4jlhRP z90<3noE)pk*OOHES&4@C&iQda2cW2=Es8S_xhvPMN66T*p^@11^NJ(dGI$}V72h3W zhi5lI+ot=p`60e6{?WO2F-DD{vSunj(M5aL3k#S1(#D9ezA8M@;+Yy_XB>u>^K_j{ z`@RyGsSN!dfX*cb!t7ob^LOX~z)ar?-OZsC zQ`zZzvsAfEH*{lZ(%A8Y$BWtbWGO#xno18>EE9rd0W9a-NY8I&rouX{+n5t6i_e|q z(n5Ph(qHhLu6;ds^sV#o({aUE9<|S61didWS`S;kLnF0zbUUjSG;$>-Ez=IDg!xJa9u$R^P#1>bU)U7bwJSC^@TL zCZFQ(<%&AvH>xK?C&QjuWLeT+k7S5rpGa*qd9$k{9G12PHdK;HMQKCd7Pr5B3;CPN zUn>+@d-6&?eh#x&lJ_~?#CM=ssu|U|z(FnPlhxn!-2FgS8uO7EC)NDsyudTR0Cl?$ z!H7o6<_4`iIZ)(uW&E<$v_)jdAF}=)!?N9jCTz0v-@Duyd8;_-A|yziO-zJC_AYCn z?m;R?0|1^K=#%MC%XJKf#73hj&8nsA~O{li^$UT zh)2+xzsS;1fHPY=Z^DdiXuHz{ z`bac--+uau{6a?YIgI&0c4+lR7ZOP80r2N`lF#2^H^rf>u_GqQS>x&tjN*nAFo1oR zSN`_Q(o+tJAW*0Z>-H)DChw+^l&91UX5LShRsBF7?LxD?#qAMiC~AN+SWR}=Ee&64-cKK?&f+gEemwO5kQB;q*FBewqG z$=L6)%Q3XeXX-YGvw4##s-b(J0C)kWlz4>h6#Fo;Htc<@S8n6gN*f|8MQ$~HEV2* zoVF_^M=c^aHWe#3M7p1m+8=%0d8g!)V%5PzU;0s@Fy4YMd%7Rix!$}BuXngUTP3|h zKNj173x^n66|u2|asC%e0oVLL<+l^{r^2B&@i7}aNyL@Omkt;nyb$q2y2=6iZ4JYz z2CU9u1Q|WLL($#^knxb4O6s}{RW#uqcU+mYmddBxWA&K0s|PIoVI_-&>;G=!|5He! z@U<0gL@$5fgUwA5Qs5;@MZ|z;S^M$Bd^B#I3&1L}#&cxAOaomm-st!{g1r!Uov;U(fb2HS&&vZ_%q6$Xt*14_FHCYi^7 zy|i93BSsW1pyRs!iK$NF!@4i+QGY)d=+K1z*U$g{zF!kSG19lAelK*5zx00U zNWvb13vl;%0mQ2)zFjYmrbq10*zIC6?gZ*IBJ8$;V!nZu*#-;#3P7&@LWhO(CwLwp z@u2PWHWF}~`gsS^)YI%b_YVcJ)hpiPPSO_`h|*$Sl)3wg=Bo;D!+&bvjcvHQ*CceU zI_VBcYbZZ1@!e+%0

S(N{Y_Eyy{zxxa6z}{ncx{+NPFP$hZIHPIogS+_ljtc*qWw7*331P~bbsa(4sT z<2`zW7`@jqOg~8EPS*d#5g3<_5Ce!+1B3gBz{Y%G3v7z&YSP@EqPn&M>WIZ_U`AMt zzKPj@WLV()&>cN956bVsC-*&QOQN!dwip*2*gV#7N!uouDPpm?=X4b;pl)Mc%DkyF zSZtQA03Yd36_S44R)38}D=4EzPCZ!o#OHXbFCph?ScTZuZaJ=Y{0LU2=(7y+QtwQw!_rXTuK zPGLp&iT2#Om&WX|Z+oUpJoV;f zc(xf8O9$!R9p&Q#y-q;oupzXc*<^n?QWBdVzfci&4M2fBF7aMP81cTh*< z$EXQpw133Zr)FYWcl|-i^oTjzqf4y!VjsiuC=4wuxo49*Meb^`P-(%Uo`2BW%GrkY zG768%o>$UQ=}lx3S@hUurnSpPk|&48DvjE#v+pZ?ZIvvdb;_+-nM|LdKdMNhR9R<2k2ns)u5gsklgyXK~3XZ zyj;s!vg<}{>}MY#dMUS6_A{N^OVQ+R5F3#|;Rn;MXYI)ZUh%bLpywBDLL9R7`{nE; zUp;J{cvg9jd;azg>Fl6qWsO6!Z!b4suD=`J+7^t`nWuDFSlW$dhcqzsP!d&a8{>F3CXzvb?k$Sg4OeW> z18ykcq?%!#KVQ-Q4kG{>rwWSpm`Sa*e1I@sb$$O(hZp~0Y#Qx8#RcolI>D$Q4jUGK z`y8!s(ANb&4~ve!DCi2nl|jUep2zlGEE^V9*Z07tI6-3$OvA6trXG0+7;QUE#^MA1 zrSUgv?~JkSrbU=V({LD7#m^ucZuLP-DxJ}L&n`j)E6Hi}K^QlYOuKAobGaD*G*v+QisJ!kC`ydvcOcD|ItZ{q_? z*_(!(wDk?rgJsCt&wpO3cz&9)=nB6vuD;yz9Uw~#`Ud!l7l=;$ATS7IVEY8qyG@ja^g4`w$?n-F zv+z9{Oo#FAlr;f7@(nTv_HnhN#vQaAM|c3qTU=#AQLU~k)OAx;7A55q0nS!&2!OVG zEV0+iXeOC=t?q-%&<`V~E7VdozYGNR!cv{C-$~!2B@c>S@7J!<)q{?8AV_D1l}dhC zqCSV>0Dwz}NpQR4sW07`58|F=d-uxuXKXS$Itq@1;$MHex^V3Pq@SQkl%s__3V{u5 z#vR^jx((j^IlzzI#%yeA&SqH8My%{F?>5~gNLc%$&D;q$xjvqswnDapYK57?(cjc) zU^}WUd*hjS*LD0*_~n#(eo0Zt^;ix+fmccpkonP2_So0h_jVDP_Zm203mHHNyqC&dy0v((6}PpGIwGLbr2Of=@c&#-L(zb7XPyo$`jSuNB&3odcNP zqy0rnOCjBWwjIVD#c%cqVO9xWk3?R+b_Tkqd&2u3c^qAe{0yZZDy+wU3 z5t8So+sL)k_ue(?Qg0^?dRO-LZcMH3a*((X)p_b8oJgyvYh#noFsC_lI4pGS>2NGSzpdB{d{z zkQkxTM_S-;x|Mqy}TGdCVcWC!x?CQvis77$}y5M@%nFDGt5&UHu8OmSi#1IxIj*L>(WU`|!mXh;Ytn1*0H`LjeKR)-PRnSr&;ofB!_Ag^nL0)D`V7=?INqxZAXBr zEt4PUvQTYX>qS7C&3p)~N^cgs3b5XGVJv|-uh-S*q=~O>&c`;noV*qG>~bx1Ui>3w zQ1QvU(PKu*>}(Cwk3Ys`Q)Rl9M#b{2vR(R;{zkyr^G78U4Y|KUABUnV+qo+7nFJ*E z@j-0e9AgVq0u2k;C+wfkfUF}<7dG6}!!$1f>QL{kI($x(zTu8*e)0d)QzOxObrg;~ z5p3t-KxBE?X+2gsnu2EcPcP|{){n-!;GdX%OErQbINf#b*(|z-e<~fn+J>sKwrFuF za)>;3B#Jhi5I8s9eEn>DP31wOagFben@sJ+i)bEp>r4Q@%*sd_FE|1J&%3&1YV#sv z&Yg~B(geL01ezUg|JCTbyUZ(_#|}=Ke4u)xzuDRf2}VLulN_)$u`%-DrYB|NA#r5= zLMRb>FOO#=TdKKeICha%d@(@4`gp#|R=}vn#GMBW~6$S~mN!*_E1dR;!kkYk|gy zF+WDRi*CRYlisH?>Rvd?zt&mGt%-2TO>cYept;&cMWe8`=Kb*&!Zo5BrvhWT6(sSz zXzD=pAfKp9MS5rT_V(lTE11EvW@LS@RI`Uzj*^etT z886X)3$l02b>i#>8rth&U8>|`{j_wOz|7t3I>nIWvYoL|xrp1Pz&1kL5GeJm(40X{ zE16d!uCmEz6@818B2%;#t+4di^xDA{^v5NXN}*)_a=nC1!{a3SE1rUgHtzpt1^1 zYkLB_-CI-Z%lA`RqHMe61VaETcH4Zq24NZ0?s_5K#`wawMS9oeypp0jHr~0(t!T;c zvD7IQGIoejd#1o?q_S>%I!&Ni%E_o1&|}>X2T#DlZ;$54KxU}XaunZW$N_i$3AOZm z#ao(>BaU-CH_|P$X?s(eNavW4At$J{=J85ng$afvEptWl^Eb#Gh2igC>Ca?PTY=d| zpKk5oWVo*z4|lc5N9tN6Ye7)N&Rv-d6tKJp@|*w3eFDjxX(4f67|D14o8K-)xHci! zBix8`8GRxMc#!y`sn>a4L9N-hvk>V&QL-mW+!X0C?k`L?zQrQtXws9!;U>x z>T7meer-|>z*zlkA}GKF2`+NXIwSS@=1_)o+i%Jjy4Y9%7{ur|x(W5rdhbnrg|9uk z1qw$`tU{tp?}aiTug-(fIKN4$q<4XBotn3I#-Sx&l4aU)j?-}!1*lgEOOqBd6AM_7 zdQYqvx!H-Egh?HcZHzYe*C)I*V({JN$Pi%2tAu0p`^?wGc=wit5zs%aJ=Y4!q zrK0(YlyiIh%b^O@vld>|c@H@)%%wg@MWv*o$;Htlas0^5)^HC9zu?UtP2q~xVIU#Biwo=juv86ap(5wlv|{^KacK&=~k#) zHO)Q7YtqC03628f9CH>P;0IqHXDrKSyC%7LCg=~E0i>}U^-(#QKxsJIvM9e;=+5{j z@cz6r?P*ub4#%Zk~=j$D?*_T#UfVOf3QYajQH z5{Wppem248WjkkQn78+(p`L?AiBUKt+3}YpB@v8>8L2!Y3<#`}4Ek_-@>*!oA6KCl z`gvyXARhVc3q8E-dofmT&+Xn)*g-#6z@KH$+5JE9Xme;17B{a+7leg)d=b};@|EIz z^YsNvA=Ji&Px;l}atooIjojy7AHr-42#YNRV-jV}cxf2-`o!{r1l=eU!r(WKh%7-% zyfp7C%l=?7u@utZd98)0W7?gw@t)trshQz$tC8!6m!))Xz-6&A??#3R^PNy6(96dtPR{AF*ht!{ z_7ILZU<>>PVTW3s{x%I^;!W0YLkE z-fdBjroar+Db?aq?U=MS1f;d4vtfa_G+!b9^6vDad-iyrN=G3S#O#w|m`%8vAQ5QS zuWWW``qL&D+Z%E?Op`D5Q|^0Sxpj-$cpEbiU7KeAoet#;v8D&g44vDuKiseVZTr1B z(A<8yhM5{v&9-^s!?638`RAyVVEW~zxUd{dFE0mW6 z0&rrfKb^vOL~P|$jH5uXn0e1j8jTlW#NcY-8Eit3=ZD-`0!TC6iDz_ht{lw0=hvQE z*;nQo20wb;(5sC5LgqgBs&*9x#b%Y(@}OMsmj9M^wDJ9i-&IO<(2dkH!nAzgZXXYU zEa8;c(BC`9W@Hsx!kJ`^Y{1H`tcuB1z3d=B{dcVb*g#qF&xI2RqySr99JCk%E_)7s=i^MDEbHKu%-Cp}Reg>C?nXF(`FGWyR|a@lOg ztIhq)lnsL<;`@VfIugj&J=QPJ;~6TAgt2(Wb+GVWTXCUKofi&IZRQCs&tH<|awf@8 z;`kaH3brkREHi*POvwY?B6@A%O0;Uo0(e?ze9a!2xQw2=y3NJ}nHG{tB@63L5$nM? zv~kpuGHs%rdG+cg-i5UA*sHfNy8eA*ia@@i3z=~uaqfoU4`g?1GYZ=lgf7mk7Atk0 zELT4f3uYf@2KOP^sMwHX&E>KxAh0`&0XSck*~2B}54+f&LBZ*~Qt&6tKW{iA<48Uv z`KRW2Ah;*ThKTk+wk(P_Kv_Mc=QJ#7ZqK-Oi(Tu^7-`K=Ce*8ph4WQnk8DpTb)*_j zFb|Ec&gYbW2~nJc?8)Fzo`ILXsR=7Dgz1|QbG$F(q0+chqm3n&Cue_9{!IjnH;j6m zV(To4S2os%^;%i=Z)%E|#wg@vBoJLj+WU5b-fvQ5@(YBsH|Qb9jQJ#nMBp)7{d>EU ze^z=%MnmRw09w>T?n88p!PX84w$EmQ;onS`&rd`bJ^!g&?JM~?#@m8Iu_kHuQ^!Rl zYkOc(?H6p)xy`zP@h7^)W-=4iFi}mlxzZcC4_)B~0Ne;2hwCY|>rA(IDo1=hL!=Sh zi&AddQqv~9>fTk<-qhL*ZUbBhFFw_MJ6Fl>xXayjJib~8<{s!>p>isj@Z*m=JQb^K zM(MjJeZtZZS3;{|$9I?s4SJ}Y3~;P-hwNk2w54?tpu4%fX}D}l!^U8WA313)UPpkp zoA;r8B@lYSdspn082XD1`^zf_ZtYw|geWT;5T24Rl&fIR$WsgIb?OxgJoOtg9Hw(+ zq~}X|bURxXUwHPJeyjNen>W9pAP5;Qu>o>&pz1dU}pO2k9tw_SK6 zb-!8!e;T)h9^zyvpz59LNTsVS9Ywl2qqTCmD7h%iak4Z?FI(F&n8Z9h!T{f0KcTg< z)7C%9nj%l{pc-Y@ZKQngciR$78M8Fx@De$pE@|m3YN_|k?{C7e0 zw{B9~HdF4n?y64kSe}kTJBhYyG3sZmj6f5Mq)jPb1Bw``UuhNGNT}Y2Pr}F-2ZE`5 zoTJT;7-*yklp4$Wc;OGo^m>N;;R5WYI`CFld*Mj%x{H6DHL0g<-~M>M}dpBNcNAP%nE?%>S;YlW&R_ zgiAzAD6C_u69qjruom~39eDKdy~`7YUbqZQwUPd!oAD|m-5p!IWCV{TP(HM`mIQGz zlaR?>^mM9vFp46zd5U!r!0DWnGs4B!3pn`WHDRA+5l5aSFj@`bi?y=7kTyr8-J=yD zlf|2sNV+7Y%uunj-;WdSQ3pn?t>MSzgM%dYnecwbXcxAs#OIa3yqC6_w(dSbQx^M#`RhCU1&6U? z>}}|6+0@4KNjh+F5-78uOWhDDw%|I(me`%YQWehsro1$iRL_(piGB?jIO%C&Bu z9Zdi_oPk>a4>F8n?GJwg&SwrKMF*dM0FFi|$8S4C0`)W9c#0(kz{x#mgpBwlW3|QH z9S*pQAD9Xlc4w?M0r|0e)#L6AIwawTtT?<|Va+VbL$&_Q|45loZua@q$DudmGckbr z;_9#e2-^eMFHSV`iLYT${?gX4`Rl^OVBMz1;B~9*?>F=nPJ7C7PCoDoamE6})kiCP z@U)y>AHQL?5cztjBL`}(Tqh;%{myV=TAHtfh?MIc*~fUNn?J-8S{sx*W`U}FsC=fU zkay>-b!{ywz1kRC^dBLY_dv&940^x&jn0e3yqNr@GAhi>yq%nWxa9AAW0crb{XLmW zoILX3j%pu(F4DP7c5MKyP?7E1N)u;?vvF4`s?4E2llnQpYSXTD_LOa}9s&nChJ2DRVMtL@I5 z^I~9VoLox=lboDWL?C!K*lx`8QSO=7)Yj8MDm@n1OM0*To`3G1hM~iAc>MKY#vE1G)L9eSD@x0*JxG>H;HFN}1|sd)S7mA>aTD^FovLv)8t(}uK)$SUgi z+)9J1(ay6khNt}63@o_#NH_@+3txWQs~1)|82pE(P`9m*XM-cofLQ@YhQXRa&y3+Q z3Pb{}W#Cuzm0TYY9f_uw{PsnxNlLgxoPcX(82@#Nu5|y$!}6~rxGNItFIb3WY6v5L zfN{~DV2y|NvP5zGRk!hhPT5;%o-zgFy{DFtK`>>qe%64%6Yni~tid~>clR>tY=hov z%k(oJlylWgl?C+ZREF|%2FNL|=K~AWeX~@#4(<0F;}~Xnf1_JGea(E#>xcJ@vZ@FBt)5%Q z4`)=2EW4IziZdeaivB@Qa=DD+G4n3Qp=B6+Mh1gcBVZEjzv|ItzM2+g;#7)FMKDvL z>}SiSGI%X?P;w|(SrX~%0|x%ZJ$!k>ArttTJaQTB(G@x(s_mjEg`(bMR5_p35LJBa zsWx+@Tn3vgq-HQm2rrm_*vruR%3cjb>y;JpWpzBM^ku1ta7E#MF}TCxce??rqL?=C zZI*#rkH!?ils+cTs0HZq&NT41FyZYAnFfjm>gCEpF7<+T72J_l=|!+;(!5Zemr6Sm zNBp~u3fD#5WV5zZ#91Z6*+7);BoXnFgg(xyN3Tj>MejJCOjZ#z?mzOd#Qe;=8+Od3 z_g1R}Un0j_EE4F>RouzyiNnlXoUd;vR*2pU#q8 z5XFm#Gd{Hf`wiL~#b}F|T|~OKeVtyoi5PjZ7S-^Zk?;^42vwUY3D@#qNOS59hhhk} z=%1C^<~;j!NWFQV%eOW)GV*%s!zI<*I0tv>Yv4>I9i9wK_)Vd>B-VjlkhPr`S$O^<4(XP~}V zlOgF6JQf&!2}+_ru$~d7zuIH>%UW*L=Oy*o?E6{M`PAN8r~-&W&LY#X8E1fZY9 z1>J3_VwpV;CzhmQNkV^U0yW{&YNWnR<4IKUd$vYY!H#7dh7I{R3!J7WoU34+kt@Pa z5xcy+oLbbK+L#AipLr7hK%)~}TW%YWIRrIRv1YS%s(&tyU!HFp+2z;UG`r64X#CE~ zC^SeIqHym)e0sEPrauhn7Bh@+>B96W*xJFT9?v316AF0Glp)=%u{9HD4N}Vx5|xy8 z+DQ1skOgrT$y(NLJgVDoxOt#IR+H?7Z#>bD~?(<&?3>(IcXI z-wTfO3DKa7nPk-)PQ!AtlkbZ0w3B5ueE5H!diu*}BTi~(QPWO8Pp9>UZ*zj4oitJEr-hGXSGMQz#|ep}{A*{9de#(X+}dRoD{R5gy6d%)CTg7wD%2oo74H9ewR9 zlxX*zzHY@QmczL#nm;ZU=GErAXwFoRxmRKnU>JBqbX};C>lVc6eK12$vh~aNL!}1j zi2==}4`aOzWJIiGCE%=^3=yiK3}LiC`t!?C<96^iy8%94ZE0<6otf}3-yg*!SFDp7 z)#Z)61&j#RWXC1w;!Q;=SeH*JD0;DC>iIoQL!k-YrrS@+S|OkCcfXb&%9QHi2sBqv z$T6yHq74M;iA1}0^-}`IgoIp*pEEDTxJAT0dBGQZ3zsMlo73+TyYjhtguSDkD^aY> zyc;6M?~{7h6s!{EzCJ3Bl>U7+wR!xl=H{EIX28^)ZQ9#=(XoNp=~H#tkT-h#7mubb z)m93BIo70zbRVV`>h-@0lOwy-Wp+&wPqf9ir#F(L5Pgn9)uDv+o>ac8a9;-AWYj0V z${7joh_*ag(iy6$w;dYuPk{D!Q!GtKqD;O#>%W@|S2Cb-LHT794YFE)Dd@4049K5q zgxnrjs^uE5Ude=saecQ;DPsCOTUvBwp&JqB_&8SQ#K`V&hy@gzsd&^?NIk{m2rt+stJ^l&Z#}RArBY5nt$TU zXI8PMDE0QX9E$kLNV7ZjDMbTEKQ}PBBR{DpJ&`OKmd2qV!4DoWBEB%iaKw?97oM}Q_<_=WV zPpzeAk|lWV+%X&aChKFI0jWr?4wc?;*r1m*t(l?26jV7A;rew#E>X_oqGfXY>1A!; zf|l0GL9T2@L@CZY96D+9AKodE%Xglh+z>P@Z0^x7qqa=^vASX!y~Gg+JTkT!7}Mq2hpNtGB3W32ctl>EycQEraHmtv%4T7 z?WRaqs}you+Totva4Mp{R|(F+C@nA z>-~;$mGX4=Hdr(3?Dp9EEKRk|?Lo>YIEiUJJ&kPM6l2z*9&XVy?$z4*AJW ze2}?j#%zhiX{Upin1a~t2NI69Lx90;$hBXebj9Y4-KvnQ2{+EzzO^}u$6mI(<(BS*Px${fS9gba4tQK+e&)wzku>>+y4tI- znIY80ItJt{0NX$AMG#viaem~A7ySk0eSMd93T88Ue)-bb`|*0pLHMHvX2`)F`w}Y- z`DeXK=Jd~_<7s3bohy?!6=7aMm1g>^5f8qzyf8T>H5G#q(}#Z7PT7K63jsmyaL_ekWAITYNHuPFlgbs){;LGDRUSU+^G zE>hkCMx^K%>gLILkHL^6I;|8{D<+g1V8}~Cv8!W3;fI;h(!;6UZ@B|IWlJ;IqP^zO zux7V=n)Fe_MDk;H5gND=TkJFT2JN$93(xrwR7ozWxp*{x0TcY|+pgIXJ?nh~-ZDr!L3k@xe}7qaJJM6?sq=+#ZFEDG*k%yygxQm8ljIyVonuQLoi z4PHi~sj%+!MOrI6WcfT5OncDsE`5J{bfGw?=It%Wm7mcBLE&o=x$<*XC zZJK)f59lQ!39WXUdc8pn>6>a|^H;gVdDt|YLtl04yT^dyEYjp9-YX4y@`+@v9?Pr} zKYKOZH#$~?-nFdr@7+FNM9sZUR=|at{5CxXnY}a3n(E>MR)@MZlylWIjAQpekOBQY zay3T_weKMLGZ$0)HEU<7x37lG!HOf`z*5(h-C7@Y+=F}LFOjZcSC;?=%R(_r@Wh=c z+M+P-Mmk1Ys)dei7RoqDXht@uN;5O_+8)OE3d_&3EU?po_WEJ>q<~ofFxT(BX15u~ zeLb5Q?QW1?O#>URhNnh9-vw#e9@a*Lf(%v_B}a(cKas_MN@l2fuBOj^&4I8((NT#- z(EX&5M>+>f@EbR^wuxa7y=ZhDeWJ&yWJ_xZ=!@R72n~)Ss}0#s&+mn%;UZ;%m`jY1 z%CmJCLS^CjB8mEH6emPv@zQKdDy`du9JRI!;h{J|^sjL5RG@3OU;!beqak@IQ@lsd z=ZH?!neT#-N;?!$rIe%b$0JVU6{;ZCHic(n@^OOhhK|W;r#*7WiI{1=$4}aS+)P!& zJNGr-VAqk7`EUTX{gM6CM~07uTy7pv#<&3)d?p(zh@H3L-yAb%{o6;0_D=1UUIxVM7>fPj_sfiWY=Z)1#9#O5 z-e@f!He)_Ub!OW&u!Z;-G|~#t@M*}Xq5bmE(U~Ph_;q=vcL->_5h-y6^o{kDAoo#^%QY6WUU=;eeACke2Yb6WyIGc>b*#2VU zMn!1K)9jl2xrjCXkOAdhG+*v8rm1O$Bv+=2_9e;0V9K^fQVwhMhOu&Jp6PD&V8oB$ zv-)BXN~*>sRga5@uD(IAx$dD3cR<$PopQZJK8mtv^?lm6OGx|vWf~kxWn6Ah@e98R zan^a_e!0l>A?LCq-{o)TgGk!cl4S_o;SG|k{C>E+^RU+UIwPxS&ME!KKlt+;lfSiH z$|b%(${1Ouk)_5ohC%TO|65^Jt>ICsjY+lngTdq|Z*xj8KXGRH{LQ-MDFDr{pJCG{ zNCR}=sW}s^AH%S0PLi+qWv7Q&B*B8^zm#PeSHT8?;1Q0?~ z4%VV~Iyuh8pEs-o-m=66Q(Xf{pVf6E_L=;iBW@sszx~Z3^s^{y z$n!3d{vucXN81^R&8RgLZbCeo&6nW3vF*es52U8`9Y1Znfw1j>3^{^5;wU-{*uq1L zMeWkE?hT#07P^xkXLPe|vu58L!ItxG!L|UBZ<_=l!2)l&^I1MpX@gX#s3iLy9+wF4 z1_b}Tdi(bBZ^&noYnGcX60ds1geBp7qLcfBt}0-b1?BWT?nW3Zmdr1HG=GtBhR7d< zeLD)b*SQA=XJ2X#4ilL+cSFYn?cJEh!-hpzP+|tD?MwGv0(jd_aGi>(e3$A-cS}>< za@-j-f;jtAMGIXDjlz#*VExuz8T-&G?^zd^>aRS>+H9REGC zxfP!FU`L$Ml2!b0SPxSi$>y8p zuk~X~b2B?E1U%CZE?;h5Zzd!m~SWL53qyr za%(i=Z7`KnXT;&&$@JJi;{SQ||BA_W32p!MNHs&FCS>iyQ#ot$T`FVC4CHvp6m6yE z{57Q>H60@w!JU0{`ySv$NjtY>5kx|*Cw^;e=*jI_nZhwL2 z3iU50sEPx(UC}{w09DZuIX_2C%6N6Dn1ZxO3);3y!fD#@Jb#XLROXk3K}T1(P1*Rp3SJpTKDkn9Q|9- zB7DJDoUH%i#PMHo`acJC1zS?sr^|4g4;@ifKjM6G+o;iAo}1A3C}W3DdovA{Co}&U zDm87v+Ry8r(R~?mu7Ay(kkF-`@@A(~ZkB%os5cUPf-0Vr0m=6IW}>?}WNX)7fldr= zihn6*_|YtOFgn_DP;7%!Ms}2pS9Ax*jpFn6JASGf^mx-T|9|rGe|KC{aeXIZCSrIp zehToxeUPM3`^&R2xFN^C0sZd``@ez{z&>k=S3(3KVdc04QLYGZVJ6WcWG* z&qr(#P>qlOvas_my@fa4#?vhC5n2S0+}=leu)fckFb|K0`xy7xG-h-YydL#66nUZ} zrT~QO*;f)sVR%21+bN}KFuQmz3M^v5YZsWIAt7gCE$;R`so#Df&VZ^Ol zN3$!W7V3RM*j65668xt}pT;3NzkceO(PiRsTZi3Zp|6h^0^jEp1nsY2u&IZ;oBxQh zP}|q@Dy{*gJt>9XrCVDurbqny^X9KjIeOYc#kg15FD-l)-iYCePk4T`Bi+GvR+=mP zG;C=wUwKS<)~`<2{_jA9#earu(^n-ss2{w;tUZFKyY*4Gsb{ew^PTy2kns_Rw3ZWN z`ZN{U7rS`|O>-R|YpxRRgdCFlkA)iAC11xe*gi#&zNX5>&ZodR`xo*gqlgt0ydG2JmDIVonz!1$ z-vuFS*uTb+mO-D^22&8sB~vGUWP&)fbPo^qy7#^ZkcU-vTlOF&f{Pk+KUBeM$n z6HO58nB2ietD;N5yGF3>Wcx?PCQw6L`f}dhX;G&IRv&2Wx4HOH>dr;|`2G8;8lA0v zb-#0HMFD!nEp9suqI-H%@#Ry8Y>mCfT#MVgL+}WAMU_F=`0(ET#+qBscbsi+S}rl4 z?@b3K39J0_dBoD%hc36{`R|Ff430TyxoHlEXU7mrSo6l=HbM&V(Bp2kFj)<$`<4Ic z_^2E$M757&R@=u@+Iic41}V4&3aB05`;ZbPI4p`bPON?CCSiGIaPikYo%7G<4WH;L z{r0aqW3>0^>|^Uhg`#b$K2M7Fzy*Tdep?*QNn2}@oUfvI-UETIJ-Mgc?l;B0fQl-D z0ecnTpU4y8@WJ?_klOm+(?D1~&08Y_4WA%Fwzp$4s4ME6+a4EB|8n{(i9Ys-?9wGe z&Ly3h=nR{)y9y0GgO>ktE+YU_qXQGkk+j)vW#2LNF(j| zo?4+gT@O|yztMRI9WO_@)Q&EpZHA6?_QQxoB+5JpJ8^1u{*Rl-0K|{@pJerTU*$IZ zr#WA{b^y1nkco9!uSqmF)Xuj!kyo+)aIo3V%ray6;HY1=A7CDr+W@t!vnuN2X=eaf zGhaFHOQT0ASN3mPCK~4L zw0e&UGC`KYFCu6TAZNhy?v)1Tc-cSa4)hN?;=BdE#Z|=5^YttY)o=eIG@$dUithl7`@}(Ai38Ovtoz+019}jjrd+?EKd^?}+x=gL8t6)kwcXqt~eaYH1sB z2J`uVAozQmuy>bvB-rttAa8JOvkiMqcVK^JxC%9;d6%7}aWDEkg} zqVMivP&K}P1Sb9+!IuS^Hkx1&Jv%^N%V%SWF!zQidGwQX4HhrfjOB_&JCFLg$B86z|pLc(vwQHgc;(cj)H!k~NM zV{g9V+!>Q=etF{4b8-8vFT)Rz2@r(y>J(6$`5%=*EruRy#fF225IDEwH))=ArCXO5 zF9YQtea1r#(#?>v{30%N;R-3J-yERQCLjg;IWG*j{cEG*VPx`=AKT-c6$*KC|AWX& zo(uzZu``U%E%wO^-u-v=BuvpVXwP1HB1c9fi{SQx>$bo1&;-Za6@xUht%&Sbrf}Y1 z+(x4G?-)2O8QOW2GVuAKpfOey_l0t`P=k+M5-$cm@7?fAB>~kx7+L!T7$Q;__>SQy zK4T0Apk=4;MWNYBLto>R0`HCs&#Or@T9&D<-80H{>=0V|r~4Br21pVsy$0V{N=zNo zgCtb-K%M=7=lAs9cO4Y{(ec~?dlbn*hPw6#iBF8m-62F zNahdYfep4to${lg?YqYc1rfo<`<;9)CyTX2h53L=G^c7`nH#%uAVmR0>z1P_&kY8l zhp+J$pfSHa9aF|QCP*PT1_bbce?Z)#aM7G{6`+BT%zb4^7!vWI2DJVD)#2@`VW%Ji z20k0s9_R_vl+XJ|pA)TJO<4=1d>E^n3PRRwNy14T3mAkRKtN%M=?0<5R_>XRved<&K1I2~vdM@snFol~`K~uvATRUo}(JJ+ggzd{1%t9$I z!7}J(@fNXVzNw-35>JB8e&+ju|BN}Jj_-H%pXfmnF5k7{jo|!2tS!M>jo+gsJ_!RC zGHN(-2fZgrO&UU>7sf(w#3K}Pay1MHSI$-uUX?gJ{@8{_+2kF2$Q6x;cPwdBY zsEWMyuI5m#@AUz6-~H-;0;BhgWbfgZp>mP_C{Lx9gCyDHV&oK>vh4C9#p_M=6})l)=_Z<+rDlB!7af;kaP%6aEBnlf&_O6?oM!mbqMYW?ykYzY24kN z;1=AW<*n>}?>=Y0ch5a{yf@yzJ=U64HEULPb=92TPwTFx9UxT%u{N$JEl!M9KM#Ar z+8#Gi5I8X9=UxDmzMOl0NT66F1bs$Bk?1(T8DeF#@5 zGBd)!`#exPY}zeXDuQc^PLac;KO&)T!>Y8}>{IGk`uO*ZjLVU@OuAG6!;mcMkJ`EP znMSoh_thn|X1p88q)BAvPQiW(Q~JH-2SU6I0lK??sp4B?!DjR2yH$6PeC3&YTpG6JQ{*S?J6FLU?)l-Y zf$gt#K|BB~izr=j=18MSh2~0>*$nD{spoJanv9!p&P(^OUOJR{1K#S7;EpYXM%*^$ zX?grvs%z<#dXa`Nc#TJiVA@`*6>*(W8Kad;dIMVzawU^sy;hc@2vkkz&t|ov&_0ns zK6-a01;9P}d$k05M>1lKXn%j49v49{Is2ntAEt_QH(TZg;mBR)rfzW^u8oRRGa?|j z4~4zg0PCHoDNO|2-t^oPjf~scc8q_I+`>-n(2#Y*7073XiL6;Ww$#dh-O=xy=RG@$0l%S95JAA&&K-WTk&wQU4jDS-S$?ag zwF+M@W&Yh%oN8Cu{`<8Zqigq*>t9E%#a=@qfotcqS=ci$*ahM`(O_g;wV-2jaym#O zrE$>>AdV#CQV1AL&f!wH`m%e;4cp&a2`t!k-Cy^RezFZcDS)Z&cQe*U&9yWymVosK z7XlQF-^<}w=rm2W2E1b-k3hvGx3r8u6VMVa4HiYG6Ej{xNFQg?dzcg1oiC9eHsAKt zi!g6qpROB36_5ALim=qCyq+6@5^1!N8opH+wqGHnSIH|HP|p4ie}B3s@v10QyFTZP z@TsMNjZ%IDbRod`AXA4V1>MOMqbM&k zSdUAtH+x27MdA^QcV8rqtbo!}?$L=o{m2qgE{%ywM=I64FFy&9c^R&ElAbG}9tQax zs?SfF>A^Kiwe0o3lS5!*%jU29HaX0O!gjP+lC0$Go|L}3)Ty%5>>n`y&~|9<57xr( zksHzE`k-r;^$&j#K+h+aY?oSP*5>7ZO!Df3S4VoKihMWfmLOvHUO?u+&ckH;Z%D(% zOLaeDrPb1S@Nr6=b@?0s%~l zKvwr{cjoiKwb}V>Rvt=l5Hx>n>Bn#22QAPT)hPf5xPQySyEC*H9S!NfS0tDCY;+J#Uq0K_!JGDLoT~3aD=9Db0N>s#F@Wu$wjdoy5o%)hgWD;=--h*n-+uG zzinyXVvPmYt=<%9y@=>ctJxlXH|oeXD>84T_OuGcW=FYk4&FyrkqYd{F+u*w(=`^X z&b!GLU0T}1(&Ty#f4dxpqIl{*_7szO_J@6y~!~dFjEg9*vYAO8o={ ziT1{WL$72Oe$?6~_0SQ3@Kg)eY6UW+-^$C#ISSXXbx4`@0auCh(NvAk-fl!nPd&RU z5)Vb>1Y(U|9ZWkKn>M}jk9B~IriK%Iy6nATe<>xkQ7aFCZmKZcf2GkEYqSY4PxZ}- zmy&?$M21DaUYMp4KBy~TZdm(kx7e3u3&$;KeM@cqDp#@<)vpJhVso`Oe?C6n5UI8g~%_r zCfNcJC)$p*GoHC#8!Q*^hjH|PL#nkZAL=RZk?(>g4rA4CM8Bw?80qlpmOpB8-b+&F zYhrs@IFCupXo+j%MnX(@!!#e#`EO6$6vqPg%Cq6uxH1Q@suCIn733>vMCRzZ^LWlMF~*ddKG)t?cI3}B5u zNMPk#!R^QFoMme!V#&*i0w4PWDmo6X>5m#F8o2Fu%dpV5|16q-&aWE~Dc?lHn z5$lAg3$3e`(ok6LDIY#Xc0V7p3YuSL~X1|C`#8)4Th)iv@JU+1u2@u zl=`Jo??ebOywi@uTRWIr=iz#|z9c8If4PAk!^iMq%c;F#XP!LI$a&&oz0oQ*ncc2A zS=hPxRVV>Wq;u-Tsn%dLCCNR3!OnWF$f)^4@QGiX2nnP!G9+;KeZ=QamBOik;=H4d zvV^~$N{GDODVCY;<73w0NavGiQ)Wq)a~P0 zIX1PH!0l0{kcM#bNlM?>KVcB+cdPBoM}n#H~xn(f${@#YuF2jR5}LlBR7PsoP(%EGz%pT;UD zQuDXO$ATF3j+Y<#LzQ1^Dh=&Oc?>FvGK79p ze)TS3w7iRNmQuoq1R`7v9c)A0O%Hv;ct4H)%+1^^_qj%Or6osFPb7P4rDeX@1kwU^ z9#lLtL_6XrQ_|>l8_9N*RjvK3?1dKYe07VcW7CkJc9E_&AGB^UGCOVzb%VD7QFFXC zkH5?jet||IZ9cQFyz52(EaQ9Nq#~bLqvIB8n#6+np^@^4Jd>7fLWx~mYC$l!U)M4| z$H71%USu<8G=q9`-T;WROE~mG6Frsa&pf@r``-q^k!GI=2Q%wwc+=hO*PkY9JJ5rH zbZkzp@15{)r&{w-QcmYSxh7QAB56wJ?H5uh-Hy!l#E%s$`;dVi@J$v}d7G;%0X6?? zz33gzPBd0ZZ+Joy&cG?Y+*PI*oWFS3tnsbv{a%04Yd*@6(Vsw>j?hTict`N`vz80~ zF_rnD%+BF2PDiN~5DtX6=JO@t~AXUb#f32=qpZCGQx@a zi%sH&b~c|}i3}%J{1|Q6%kb%g^1ag3`VwYBrr>VB*g88?q(6E_Wx(CYZL@$!% zf_IqdI@0=*>6Ja8|KfigJp-QwpjaheFd2hSfAx<+m%(p zu3;wR89dwE^j*-sXoMg|Pb4Q8&ToE2@)JQ5{xVq->!rAcMkim-v!97E3)I^mUHPxL z(*;~(Vj250zq}P@A@{$E#W{TRxr<(f8y%>Yd_uAnYFcmm1Lhz@@CVh5@<(rD$ldre zN7+-JLLy$=Wl73xkOp2P2E(M-Z$NI)c^oG5rnpkCA0{`$Ss|nB8+%}!Wxuaa_CRqh z@?^Z6&E9!A`)nf977DT5IM96#oFHs~Wf2tF`JKc7;9rUpIAbgHUGzA$+#J)`L;EkH zzqGFhi-&R1+YyolthbedKYLt!>Sn@`J@9pa7Pnh%_Q)hbypjD4YD`U%OxQ&RRN)Gp zgX`7#!(mWI*oZvqmNgNqSnaryT>(OnA)x`xptsjY!G}REv@Ps zh&NuNC*^e`e^{nLZ8q3J?tEpD@sx@;&B85k2sfkIm7BHIX<5V@+^zYMoClSToFTfb z1^xKZhcE9;X>gRs>kj+b+9Tb(G;HJBR?h?*er{VyYNte*&~7w56{ z-+qU5Nu_6wql8Y7`NHJ`dU}Sc35+wH187}Yz?!@zUyMzO6kX68c{IsJ{c1ZTs zlK7p2v>2^sf7QuFC4R_M%FYQKVXZOGMYY3{=0*a}8|@v5_BuW!2bsd%ED$nEuqw7L!yZ1GHJT23*X%g+{_9hLJUMp7988FV;bS1~eJ% z^%^|heF@1C=K?z|XA=Fq*zS3h3Qt|Gx9{uvDn#2)R#ns=@T^diXM=Q=+}0i_Pf$Aq z<0}lDyRHSv3?uDJcQk9cU|$Gu?Vz_%896%{M@edSey1G>N8hhqh&Za+$iC?JWnz86 z?aMu3RmhvC0?j1SA`^f7C8)&*@*Bk2i6DC;=&IbR<=6z5NT&B6)H#?kb9w~b+E*sr zleeC(1Bz9qjtx%}wXDER;zS2rJ#|6XNkHu!rIQQ4Hsb2<=a-mf%Gy{Qi*JiHr zsj7BHF%j+aFHZ+aU82<(e03>WoPeb9mqlEl?|A-3%_?fH=0_FPhq*2pP1#U7UEo6Q_M&xN&rW zlvetR*D)4 z5ZF+*u8-u&5R3r1_{?M!aXTDIe`;Wum((l@%1%^AK;rap=ou*=kp(Ai7mNv<{bukj z;m4N~ybc|-dn%kxcj%XQ1d1ucp1O<|jX0o#f=xAc5p<3kluD~1g6#M8xd?Ma6kNpN zby4>Z9Hq|@*LPp`ZV}t+&wKYczA#*S<$q z;4EC}gmd?yde3BrvHOn`UeT$Ebf4c@q#2He3;U710oecX=LV)Y)xGDL*7&7RcG@{H ztwL!A@%x&~@m8m)pCGQr^R9OlWR$~c;MbwC>)wBYm_<&qqy0wCk)Q6NO1=mNMd7I- zY~JCTx!N%;x{5SiK=v-iao{*foGg9)sPPa-6RF%^mn35%#SQU=xu4>gXd1ZwBEpYA z73%D^DUQ0vkVapszN$*ZyMSmBfbq(6>G(XK-n3eO^}$J}Q6oJi>9a-xUJg|0DY9)+ zv|68swEF9t)!N8|?((k;QyxysV1ps>1FW(Vr|SX0Gc4=ZXK!mFi59^eNc!DlLY?WN zqa_yQp>c-&T!iU_NSV%_e8e=nGWm)av|Y;+1V1L-l;Nhos<=!tsLkjfeJ{{3z>F{$ z)>Vv)U=8w#r$J`^#bt<#x_4&;5&tf~%g{JdOMziM!tfZJH4Gt5rS0Fux|{b$O$HWr z%L4)j%l`H1+AIM#dZ+cH{{RAjT8p<+& zlS3=YpdK6~s4o*({6vw{P?-#>1Zf{6M?>k-Etk?}`fB^NtMIT zxKg38nBF16%R4o}7eZaS9+Q&0erPAWdSge%$hTBr4!J>n7KH6!=6ZFNK;&0%My$eT zbcYC!fkU9UPo$|YgY>-+ZG!notZXKJ+HZtZI>Nc*pYDi2=a6iudVXcKLi>oAN25WTJUKSJsI3d+>|3n@;tlRkA#)RPJphi)Pv05| z36#YqDDf))(?W)gdyzhw;Hv)!8J|8!og0=9PE*e;OqUjurQ1#otwfKFshX3|LXc~%_ zt-q@v(GP?JJYPei7jzd$`gWb_@8};+R+?(8f>e7F^_@Z)A_){{Dh4Y4pczAtyxlC6 zAE1D^UXm)pg;9Yp{$kB0>H%%+KtgZYL22%3Aye|-KD##Lc!ckWZgr_wy9Bx|8+sR1-)eL(7YBf ze)3_cC3-s(HEqow>0#^3?5?35aB=ASh!k)+>#qjKbGiQ-?=v%?JYl8YrJ;EO9me^@ z$Zyme&}|$dkLoFbmV?l|8QIPr$ZYWTkiB?34c>U5D}N<5%?7djur~)@8k*D*kXT-C zcVUv4IUCd*d2nur0s7ymc{i_p?>3OT43ffi36>B$z{xuAY4xl*KeUclNL-N~abK7F z-0>$!;rn$}W%P}B{7_T!Gbi40yIOX3o{S0ncg4#FnLnwm39crIxA>M!y9```1pS?h zbsrc6C4UxA>j9lnuEyIz0>6MwWn`n94v??+C7r+o=AA~G5q^})3x%OJ zov7f7d2ap5@~Z%d=Xe3mChX^G?fAP-q|9a@vaJX^R?iQev8VVguh!T!xnhn2vr%fU@DmWCwsCCD*vnt*$H$ z^u+pQBFw@xCBH-24>>KvlEE8KP zTNNMpqQ&lhZZr?U7Ya^R9`&(56HkHF*3I3Drm`065(49fJB9sVx2V_EWX9;G&&^uM zB>Vmt4sNa{NYsO`40vA*(#Z#NjrEt~eShK{pUxAdbY1crCQF;(EKfJFSSI3vxgj z*u&k=Sd0=8o^hOeVwD;EE-R9ZmjlPpr&+(v6MYa_$WnKz4~;z`Gjif)J63)Vl4f$! z3d1#S?VsAa0T;!1%r*%ODLmS2Bh!EvcNC8<)o3$CMCzU{??tQmGS{5$CV*5%RzLDR zwjhL1U~{|%K4xR@ZJGM#=b!_mrNc&4ySwBId9GMJ^Q<02KvOt;SWAe_crn#_N{Qm_t$EZ zd*VI^)SqN1$p@8|BVGLrFE#Tr7f>@29;MCAPGVh`KJJ(hn_!HTP?8Na{2}w-uvUD_<&(*>~c~)0OpWS;faU4 z;}HH-%z)-5nDWctbhK~+Geylos@Ylo)wFLks*nxnh}bXVs1OmY(2f@V+4TY*YGwKDya< zbn<>Q)uGi4xNk-crz1+Ir?(Td-&f6Rmi?nkmeqMk`(3=^<)NP}nN*;#8wy2agr|M+rSRZ)Ng>AW#p@#kTmu|yS@mYx*x|JR}Y z7hbTl%oO%9Y^!d)-y21^m_FNd6wiFSVegM3tW&cs{;6rj<6%2C`irM!eKPLRPfkNo zPLXoS_%8FFMT<=2aLZo#Hi7GrP&);}Zq#V_-V08jC2W2>HQf2fvA%;$lPLbaH0PVe zn1X8j=-rP$50^fi&ZAmZHb=b)Iurb-zp&EHh7*#3nyiki?el?Vk(w9s&H>bJ6Vxf9(oXU1&^WTF>Y?_ACL??nE8j!|GDQSK(F@KT9} z6I0PZ-b!6@gy>TxuKX>8oop`lP*l@EGm34L|c$pi&K)a*akMKOy~gDP^EWOf6J zDPVkU-;*^$*4n}vMomeZ{ZYoG$2%EAl8bg-7L>iKI%m{|6cudm3 z=^K<8`3+La|6xFG+x-X9;m1EJZ$8EF8(B-^6WmkeeQRdWr3(009|G#T!t`5Imy)F_ z&nF%l=NQP^Uszx*E+~^8bo&64Ihz*h8k&BJ?x!ScZ zRmEYkf~^V-JY4lkWX;onCbe1E%Xjue;+$zsR;8Vreq z%M$5b42^tbN@ZiotBYh>!+#qoMn-!g^>h`(^eeLO@$WQHJ)z{hmo*Aih0A3cyEthK^zA z8Tb9UN~u)r~J80gLKreCPKvl0y}mn ztHOz+(g#O*>2&L%hc<981f*kpvhEJXsVWgzL-2ZxxOBDqXi&NPaA2Teu~LpdqvP6V1$O!aa;b7VYN_{ zy=?o*X`W#!U+xFB$z~GUA0Q$Xw@KeJ$#`~|%8?TU0Ri|x+49U(W{QCJUk`l2d67UZ zWW+)H&&FXt^pL8l4=`fc9py;efDlI0{c3r(8qUfg%pXGnH z70r3w^`9H_i8dgwrB?1zOSLl+_=JUw;iuZcF#hH#1=W+1R*?_ta}1Lw{pQk3jcd4w zqh1wC#ixQx;=3RNd$T(mgPlScuic7iI|6o^`q{pT2;_gQ%NjH^y^!H@@E|e`7ud9Y zZ#6(R&GXh$Jd=?V4%-8799yO>&vW_F@}!{f**Pusyulphbb1(`8=?n$9>Y@GTq<_~GW2Zj=9;q5GD4qI%?;C|$~Q zaE5tvr;5R~^{itZKi0$jukfh$1m@2q_NB~zlYxDaq^ayCebT~P{n5Mtccoq0i^b1g zLchq6w7FokGCOEDmmp(r;D53JxK}X<`jT(gz<4oEUS=lcxOqaa7BQ!?jMf3Gh~lv`e3RcoN&f>;7|XowxIZ0s~Ee|Gf1wz|C$iyX~Cz>?rMtumJ{Yy`>Nkr zlO;PdMXc&D_$q({I^BTiP1OC~Ym0KZ4axXXLhhzaU?(CHR}d(Q?~UC&Yn{Y64L?{G z;e`9Y2==?)3?EMojcw-v?(_`!JdBmhq4}O-b?P(GJUYAgpAb-G%~>EJ+YNagn zgamhgwFNIPtx?;_WfakRB5W~6&YqG513lDxT{mzEfgg?Njrwnnb0BtH5eCqgQ|Sg^ zAq-;K)Gbd-GUsuF6YguWnZ#e~u6yab`I`DXncmP)TiI%#%_9hNqf?vDqehM0Fi(x+ zv?r%8lzLzC0m&05F$uUP?f5d_J6&l+B@I#MfjL&MKToh@cskk&&{e2<_QZ5tc=9%@ zO=~Jpu6k#K${T@No1|Dp0qu+mc>nEqsYrCz^{s~1=yBw|+ufPPmov9C;g6ezU%M$! z_EW%bBaKBezR6AtB%fA@eT6fOt`Q3`ONlb3qZ)}79+&%)h!=5W*0_V;rZ?PEx3=HHu1JHq6ND}|xE7p-#rGyA6jjBpo}H?F zc~*0Y>-77~WKHbo`qSf)D$-yMuHYY$gEr3$I(c7K^XLEEZQucBiC0R}(2X1nH;c%P$|| z_#~zOVOj0}&~gW5&Ijrx6!PaZ3NKXt8lzJGYfNnb##K&E;Z9Dx5Hd?!m@dLr)xjQ!eR!fm`;Y;k~SxA>M>P5-KE+Rr&nNz-7K&s^&t^qR<* zOIJU%lsbAMl?eE%4Dz2`j#QiQaQLnw#Od+Z?5~vdl`}HV=fq3ig)(u##C-hUx-X=O z^2;dq`~cBFVTSiqTxUUv3Zucbp4k6t>i)w!QGxp2=QNyleOy0XR_biAWo{apDC{xK462!0_tL;i*O`cFnbaS6Y_(fURd z>c2(pe-m>5#p3#(#@_S*O6ZyVKfIpcC)_aBWcvT3YmxuEuKgF|lo)EA`E`Kw0u^7Y zc0!w2`Qr;+rP#ll4i;g;U7F{$o-Ug7QayuPF&xUrd{KRWBMdc z7~;_Ee)`&q0hQx;d=9f;E=QhYYy6~$?xTW$#6kknMn4$7CpPE?@rCtl;xaF2AF|08 zz)Z$X&7qQg>p(7ueL~mJUg=`Z2pz~L;9Rwjd;zkqguj<$65ZLS>42Th5fr}DPGUhc z6whlmoXELTDWmcfbIE6(5VQtxZIOL!yn)E9_SM*I`_C;lM)kV|hzFQ>4MeA!t>mes&E{rWS** z_5JY`XHPQ=Z~vi~rV-B-u`uk4v%2`&?g7yO_wgvlOKS7F-CP{w{*=hf4#YjjxX5iQ zK#;`X*)lgN!BMnZbSijX-f*+OIE~G374k~Z@`rS%W2_6;)r6fcq*D;FiPebPjx~Pe z!keh1Qs))@+KJRh46!9IBHim~avG=lhNj=!e|=lVDaGD>KQ|Yl=SuJaegyFtG^UROUMsXwVcijl(2crB=dJY^-|{b1u#w0pBYusO8i`|QJ0 zx-@jFlcwZzO{DqEgWK+|J(2oF;*a}FVg{}P(e#pUol^wlugfSgxZ`(Y#%}k?_rFOf zJgia@Dp!;E3sMQ`M34F5(i1_yayT#UXeOQdoHACdVS2i{W+BT;!s-f&eJaXvT|N}R zEuTWTF}Lnpb`n0DPWuz%P&K{(h&xaVt_@sq&#YCdP0lOp$`qs$=YY8D;iI9agBT;= zw@Jt&y}_3%HMn#88fCgPF-%*M?6$A=zv&py*=ezEugLim+0pAd*dNZ7nOKWEsAAd1 z)mcwt24SXz-~tE3WS-iO`8N~nXCAt+R(#Cn;(gev`x^J-ncuiguH8~CxBr{b*-}X? zUDc!QJj1XY_zBHn*qvK1Q*L+`(^6k@LSglQ?{PhYMH!DBK^@ z4SvPrVO4c`xAmCZV#LqxAWuFiAPrN2d$<6@t<`?c_zND*F$6+XWDh$m^mdPk;|ZIN z@oiz(SWdnr(MoaV7_zEp90a zpQ9w*3l&GC|4e?V~Nv<-Gz|s;zfW$Iq_O$;C zXXJ@x31sG#2+jHZuxd3k$$Zr)k3&QaXIofRPWva@`i&@S{jpu5U3w)F7a5;8y!S|G z3k$7Q;-KXkI#_R;^-XXi$HiZaoe&^xK!)a3OfJYndBTBmx18WuG~SeSfHBAcoN zem*73Cy4^QWK$---nT0so1rt6Fa0Oin{Pyhk}Q=sa_X>#Tm_2|l_ER(!bs}(lfR!) zDZGb|3-8%hIZ5p-yZHJFlkes!ZJfDty8kE}pAL?<*|)g6tV`4+J1z`(1*iX6jRz`! zyg8|wV$fn^^5+I1ohSTGGl>IKFJn|4Qkhd5w2C=xzr-`E&XHpcF2nmlte|djMGDNp z+IFK$w6x33(-%@j3=}nFLksS|Qx1YbVAGFL{U+g`W#j_g|7?#ds1Rfb1d{ojB5;M7E^! zTo6ne9_J>0$>BUeSXXfSX1pcB=tXU9rtd}Z zna6eA{7B9{)(CZ6XsY$2BSoufze063aOENHC|QSGifrQCW+G*ZG&ma;FBcfLgI!jq z^46Z83DcR=ai(a}c*N)-;^6FdI3;YVl7fbUrQVp+ej%`1PF&i~xjt|9_=4z-a`^Op zHN6^&bdScj6J6pkGT)sqZL>ad=zEp8%F;H3zTOgBx!HVm0n8j?`@lSlQ9lLdR6dDO zfq6sAnAD)@=H+~-JySu1WNWOzLT6f3KPbnE+z|R=O0~TV(02nTCDK z0mZl!h~Ce*=u9Otaj`_X>R__bpkN zI|GN`+`r#Ta|TL_A;+G*d$t=nI~CBJT+ym4PH35vb44Jzr5}T8fmn=bX!cu8mM7*P zf~Q;WFRQ-R6Uk0AB-<$kDFp7;8A?{l#P*5x8aX{VvR!#S=z2* zk`$`DHb82q89W&eI>RoY(cW%w&j4%H3#VNA)I#C$G>x+FZ(rgeENueH)Tc5bI}2QJ z_-zGGt+dyw>wFW>r}(JZ8V{GN#{IQ-H zC_h;$FxHS%t?`s!lsiHmn_qz@NSYx z7i9ofalf{J;DBBVy*!tMfUD#>d|Lue)qW?&G`HqDuQ#M?4CFPOhaj{0*vUGo?w^i!bueAL4#d7XC4kRv42pmm=--3`vri@quKQh!VSBECac(AgW+S^u;!L zs9^KGL(g?dtfKHNbSpDDJg{{?eotE5Y-yc47`ZE()IWzX`P{u|9C20{ES`Krav2R4 zd|h8^qlOmX$WAZ&(+VMO2VPFIbvF)%o9v2ZPEX|z%a;k%jef8#)eAux{z2*QKS>z%mNaVPV@FxTYFOiQ z5SbebNj@J(|9Y*J>S!k1rN#CG{5-lL`BdLqrjk5lZcKd%Ifqcn97X{9!?^RLqEA_g zfyc-spS6?@4#xwtCF*%L=?X>Rf|Q82?h;k3H@K9ZL7V)9EGItfvp^ZBmwfNa=D^Oo zW9+=^F0skNc0tmfdT#gmN#Z)G08Jq5JO=ZJXLyS=QDmR;)J8@a|6d}vz!03&tIp}K7xN~jCDb^v?FR}IzCd&8yNGw z-A@HJwVk9#d>Er?>{!IwFH-}JVTk!xE+fq17fH?2+SAZR^Aapl*1g>;kRpE zcz22Hkhb#%Us07W^+jE~tWC{w6hAgH3s7>HuZ|xfc@2n509yiw4^U04@`%j|N_~cH zb-z&sBSw`4w-97S(j-r;9u{jX#C>Eb9Cle=!{Hhr5<=&1Q{z%0>R= zQBPJKL9$nj-DFzVZA0%P1%V5gjaj%hW`ChI?!hKGhg ze#A}U9D9he!n5%L4*oe z9+{=5Yom&iDDKt?Rw_*y3`2dU zV}SZKfdh3)*Q?TshiE=?X!C^cv)fc&g(HP^ifxK=V>15BN5|Dzd;=is^QBD9&9Q)C zja{(SQfZS_6a<+50W?#9-F|kig#r4hzP)6op<$+lUOV3rK<5i>w)z_O3wB@i0Gy?! zEdC(52lY)4sh5n%{D|7<-)`DVvmrH_t{~0ZCGC7L)W2MR`Fy;vGw>x)@LNOzjz(;O zc70n3EcLD&z+O)h$VzG+djfM=0=UNRm#T1${P=XbC1(-rH^Dpi22aw3VPD$zvAjd- zPA9Up`VoLzm@hJ%rUrNaIFGZtuRN8(8@B?#lkL7g0F!e3-lgGP&+v3=)UGtlSoCWu z(TvtN-=9!5I=_88%42W|3dIYAn7@vCcs|9qN41sS;$|GR!hE@>U>jg3de1P+gMtrU z3CP`EZ@xYnEvHyEs`Kz?2oC*u>7XuX1F%*TN^-f>W-;+OU9%L+KP9pli*j#UQ|cU= z4n?N&$OaZ^##n!F`-V7K^`f@#x;t0Vls!=Hr=f{lOeh7Ljm$ec@~I#b2M%U}XP=hX zOy=h`qEB2H1mhlyiet2NSan?II=e`POe z9VdY>DLX|pQ$L9t+7v6N+*ZXRm*33Uvc&nkj&PNELLc87s4G5DNX(bgBiTPmTHlU1 zig}{M7>?Bx^ql6Zg<={I`h~$)0WftuB`s)amTDvJN z1*0+HJ$hc+2xDLbgr*?RNorh>l+D5eyy6qmhhioczzKk3CaGBj0LNUz3=*Gx1JtJn zxB~$7={D;7MLYr;DvhNG*57A!8nk>EI9}85s5{kI3}5?NKJBDvdtxMnm^W8>10VTV z{T8na-SqBL>H@L(@@@*`10qT=2~S8Oz?l-KXu`gh!(AFSj!hja_h``v>PJHDrW6Dl zC}uE$Nad@Z#=HVg?l}B<%}w7Q>+y~?#~WXl=P{fl`b&?g?u?Y!x$!gU(#?Up_~82^4!LRaxFkL?Z$ zT85E)!88xbVikSuQX)A}@sh3zU$_>I*U0uLeu$6^f0=sUILB?*r$+UVsDx;8-c2!A z`VIDmY>n2<>)Vt!A2JTkC*Y)cn~O$7Pn}E25XQgZ-+zpLB~4r8%|2(KHK^EmcR2)T zKSF@7$+Pb2;)gym2Sb8**tj)$*CPVG41$Y#AV5K5IqvWXw+Xaq03#l${(@IJ%#S}c zp3LJkQEjO-!|%@(eQkq&5A9L7uS5yRBFnno%LUkZ?O*SG6hSmLfv0}N{%|c%j=(9g z2of;^k`PaK>qX#`*sr*HYVFoPUj0h{A_i8i!{mv?J%o#h z&yxFo$Y^yKAT4rJ;Oie$J*bhdV>YtEb@IWlu9{7FQfX;;j|Ro*$ioTm-Ug<*IrN8A z`f@7gvS4(f!vysXaUKlXIx_p}JDC0cf0d0Za4xGa|T^3 zX1XCGLuidu7Rk^NolOeAoxf@fDYVMQa;^WWTJUM}o1vk{23t6N+?(?kPw_xIDi(J~ zbkm6w-OjWwgK|l1`b>v11Tv}lwtVy7Dbu4brcIX&mKGfF=hc$W3%3+sg6xU9crg3c z{VR2D0l9%>_QsDXQ#4C+QsmlO4mW?a&1p()rp}X@2&5B%yg3EoditlLD9dqbhRh7b zy%jk6XM1RP*S9mJ$`p=fa;N%j&mmI3efvd~StV#J8DgVAdhuDdE!aG(j2p6rBY(8+ zO*aITlR0*_e-+k4SbS)Veu0Rob-5{jQ-xK)?%5w^R27qf#O9<#gRirHdd*EPcSq$M zNZ1i%w=*NpGwamZSV6^bDA z?h*ciKsF!G(A#Z)L@*{0bwf>l7_5qIg3MzF1pCN4pM^%#x1YZx{_`8kuW{Obq;NT8 zQ;|Ap;EQ23Zr#atIfc0&D2}R3FS<#E>i`4a0otKWBJ17{ll{gM)cR-Ng~30)p@*^( zEPcoNlZZwl%{5S8->&6IK9-SH9Om#}W<00u0-F*s^j?Gn@nwuY$$TKmXb6dos1g(-s1?2;M#Gji$fI8f-kiN6TvMhGQ zN;I3taQE`#&$1(NI|oh$AzXp;ugh1BKYZ_zB5&IouU9G83ONntJ}Xm#y>LxF-&-TFAK=Rx z$biJf*hQ70;Ll6z33^U+;!fDhw=I6;X#N7nBVf#H&%XlOMSxBX(dgTPpHBqce@fhN zSoVu~2MN4peV5>{+-t-CY1_A(XXeJa7FWp_k1`AAGErgLi)tW}hBcqY4Px{@3Gi?@j!2H~WAY^(QX6v`;`! zpp{Ja0W2UT@F9f&hFqD{^ep^Qm+7Q<+mrR`?Ci{&pnSu2u~9qk-QA=(mW!^TGTp-K z01StYV#F?UYDY~(F>>Hs9AOcXRq*k$`x^t>6v}a0r5v9p_p{yHlsX$G_trB-7*+uf z^v19DsjZQ*^<%Q^a52glvnUZal_63(kY?Fpj+2S&>(@}4nczlmoT_HK|*jRxI2Rr z+&#D_xVuYmcL)&Ng8MLYJKtI7uDi~;|9e(>;~7EVK-tIVyo_)ek9VK73%V zxPx&Ob&5P}NnB&Nsy7AQ;AE%j_=!pT_X%1(l2;;f2maSImL5p}8=>Ncfxmp?5_3Yj zC`8vihOqbAuw1{t0&`_Jw2`!IieKg-c24~W6HgYYUe6#0V%<8jD4XO?N1_>AZJKu1 zqpy%PR^XT>ay6Vc%FqNy=>R=s%9d67nA6#to8&q&k;+bsT| z&M)ruiSiW81xJ0i{i5Hyrqd0LG5S`BzKOQw&`bnRQ|OH9q(n3RCYAmj&Koj5@!f@7 zWQjVCSlQ|`!)gP4jY`2W#+9fkT*>T-Ip``8ijb&=Cn~&MdbjJYslIG!{F6M+6Yo%* zM-5_h2ETDo>*>{)(XLt3FpZ*uncCHTy0=jn=u38`s0VigOo6@A7J9jCx({T+xLE5i z$tcWkJvvz=U>9+)`944@PL|Y^9%z+?G0hQsnFkld4j1%rZH+7Bg%j%Nk5Cd;zYfQd zM%7$)FilICh61K@>S8tukR#SIO9qfBI>tHQ*!sS~x_*CX%+zW`)<#K~H*Q012t|hA`>MQa$K2UQJ z^lyHFdV#@GwJZZ_!&9JV3~htEQR!8}cU?WpLme9XaCl$hr1x7b&Q{&{KN`LdKX4a| z_Pk&YW@bP3R!<{)%GDlV)Tig%+9X9U`?@Z{aKx%G%YcoZ;IkVvR?%N^zK#nV+O%E7asUA3%TA2 zPk9diY-jWc1+<1FoUYhU?<{Yv=n}&y|3a?y_xZXzMGlXw27}{m2(XZxY7f}=h@2YW z`eYcS45GD_i*sS*&6~#1>rI%r^F4Dbo5SA`alBoTbRp#-IHi|1)aIV{cUfT#-j)evx_l-tHG5Yzvr}(b(==5 zwTd>urE9V|bCgGE7ufv0>^O>8u+5I=naU?0oiYj(2^?j1JqxH<|ElzS>Rd^Z^gWnN zEI^TXGDQOJJ})q=FE}|btx~ArJ|yBT#w~s1^(vT_)~%Sf))z3Gp(kzEGpsFJg|)^9 zG?-f=@K;TZ`@@aXEdiCn*QCW{hSacs`TCjLlO@89Q0P4ac0@90$*SY7T6rs=o6bb6 zz(!!rUiYCV&YLBbHOJy~!LV@Acth5D8-yjXwfgOWL)%P8s^&^^1@2I+$Fx{n|TAM@ono9~2e;{(fA}1b2qw&zr|^foIc! zu_*>PU+&9Nx8Z*keaQ2sL$5f#YXq73Z5sNpe{2afPE$XKt{e&u`Ase#@bPZBFnw$+ z@^7F(O;!EdB~1c$T^4q<)U=itorZ~Akcx5*pZzMF6s)5p=!Bs~xgW{IPA9!+b~o^%ou59YS-F8R3(0ED}oqr)NKpg@yd~*Y%e~U(vcH@glZt|SC zx&auJ#dMxniIb`9{>WMYlH}(OY|qpTWreb?L1}P*%bQ4bvYe-9^Y}!%$OGz*Ck=moa{(qy+g|=ldlE% z#8(>~EQ=d@t27Rj4Qj=q^V{D4O5skXwJQV_pXJK%j(sA?|KIRbL;ZD&RX5`__<@MW zXc+!y1bCOFpI}dJHpS;fP#woh^oJk#%x4)4@!Y&MIAOwoV?fr?+acD$GlVMOG^kze|51RyGT`Ts+lhK&3TOPEkYsXwL^6!|iq8tO zT7lFI?RPs9@UbfGkH3?D2>R|&CVsem01#Kq9J!WRK#@Vetj_==pQ`nmmLBJfT3!v- zF7E?WjEPbVJl0As-yj{rW@^6q%}wxS;k=U>^~9F>xpV=aZVl^m0Na^2*-b^c+o%HR z#Z0NSyL&0g6~obj~ziy{+BhgE9J|fs6}+QsjWbK4RwMj&inQ+-Dsi4$1|#l8?bN3aK98k)H))rE!+y} zJVo1((?+WrM{0}vZW?|m*c07jVqwXsSC8;z04J$$P>7SlT`mzvF8Uo7{F`DQ*iC3x zA#-(0A4a=p`buCNa_kOP%VbttnVh|x;;%(9uM5IVPx{N6+yre7d34)=#y*9vXZ@LL zZr=w-F3$@inX>?zKU0u7f(6cI$)i=nUw<+N`q`?@Byo4XNx(@C4(3vt)f>^>M=<0D z(7cQ`VQh8Z_ZVbs_KAC`cHmytyi<~<<&5QjCF;s}3;tuMlHPvaHRgLU6W-#!EI7^} zcs&fFL#9e5o8U;Dx6__Vvvf9D z1GE~oip8}rpXUlYp~CZpA@my`Y;ot_vy<~VElhkDm^)oT9^EZY2|l{( zhbW`ct$j;TzUX3}J#9+l&UuuSZCd#sCgn1_#Rzavj-gCX4Kg8Rh8OP}1bMEeLSv=MjIeveMH82yEShW1UD_)VHSEell z6=i0LU8RGIIA*I6kUpMMe_PCZfC$_?jlEY;FL{`O&}|laKw=)hMtj|N!ZOrazKTcm zA3r$St+zxF3p>UiA^(iNJy{s$S(gTEd<72^6Zv4UKZM;JV`;2un0*4*`+aRE223Zq zhabPr7cmu?mJ}WOE*;@oh6nLceOmzT#k6|qfx_#a6V6s?OsH>W-+snhtIUk8!Ff3a zNNl%f7iM7{A7H_hdYEYWxBCFJ+IAVbLs3X+;a=dk!gmUtcf*KraIQ~`4J2+2N} zp`s_*bdhiooJ3KI%R@Ig>dbDZ9`_ddRx=q+4AqAzs}-tr8fSg4W@K7)@8LzJn-yOC zpRRL_T{=6E|X}rW>*iSlkIzE!=L zC8J_VtdC2GVQoIpjEqjeM*KtlM;4ShL0`awOA1WO+F3vKQ@h%Fwb7c{(MbfH0kyPo ziDdRVQDI7FU8nAb<@N=>ujYnbX?JsbygW{+#?>!=hh}p*)}N&~2BigR37$+@zPAz# zsAZ539#n-r_@~P@4L+<*@Jg*Z`-EzT$h-+);?ll?JY`ls8MHBuzV^zKG2l?WR^J?> zi@XvCumpq{^Bwc|&JhJ2@hDh5Rd=%A;%BqL(8q_eO3TIV;tN2S=_9ZQ|JC5wr${ZC z@JckU!zgdRxGPNoZ!Z+Es%IxE^C~~z!?KwUy-N?jEMcoUEJ~yQt>u`UGN#X2bpx}SOxsV<Pw$n($>A)y0j4MC8*+){CAu%uAas=?$fit;3TXGSp#ry$}Fh^gw-*j**m19 z2fi}u$w2x)5lVD%x+T7O`>%woim}-z44nc~IGs@%zNxbBDgMuldha43@~}T#=eM7d>OA`% z=7&4-H;D53`Fl?AlrTS48W$p)$cm<0Wd=D#m&^hS!KCp27d)zqobL5t8`X3Vf#$3E z7%!B+Rqf}z|B9x>*oB08V8tAcm2nA5yS0Xe@eICU+l?mt9hbEH;`#62$pl^%8HXaP z%BdnHN}ckNQ2)Sa#A*06VH2bWR{h-VqDy$IFL{&&?)VX!xA^6SgPxb}KWvZxcU}J1 zC~iCT3K&V-&Q!qMDjHQm(Ir-`752V|~6)bLnpRuo~U32WYGhVW=L|sx(+I(;V;PS<2KR z$;==Oo0I=>zDNB3Thl>-|E7-Zf4>Uxv;F7d`o91fy^Y+6aj(^}a0v;{@!I&DHSk6Z z{O{}k=d1I!&+^iys3=Chsdb*oIXZvVzDYn3lmZ;Zi34?HCWUQNGR5Y_Y}YeEC(~CX&sUjz2PKb0nqxZ;}pp z1a!S^mzpZeqcAxld(sB2^?-(y-Q?KUnr+7mWfcZJhjE`qe`X+(8&#nK2DoU!Xn~`L z;h3ZZ5%Md7#_npmy*>$mj<2?<+mrwS3tb_(F3h0QcN8ugfsXS(j}zc922P(-A)Qlxr*ZKfG2;k%trxG`l+kXc z+N17L9fS8#sg^zL9ZMQb&IUz@Is)lgh?sE)p)bD1&=WTKtvpgqU^TAD1+VV4PnRH1?}MJ4D7NK zQzC?wA?WeyMBw9s zU1?G`ud|i4c+K5T2w(5*^k~Rf<1JYoWxN7 zFD*^3qLDgze|z;j=al%`c!KEI5pUt}CtRkwsZfTiY9&Oe?3?WR0h`xS`51j4_b90G zzSz(D0WeZO*Nru;SuX^sU@4UphR-1~)?cF#DxXrb_``U$!9GT~G9eD%+PZU{c_DjOb}&=2eUdf}Uu zTSrm+@7)vZT+n_3j4%9zrY8Xz0;X-g$>RZ+L1@#BE^tx(*2_g_tBqOZOL>8hbGZ(D z3_1Kg+-S*IHp<6qIepb*I)RBG*oEX{1=kce_D2Xu5u>WT>|Kq)L>5m{3~%|n8G~EA z$<2pmJGJ;rK9WTjO9{1``|kUX!@1gRxn6gluSbR{vtHM#`HjJz!BvlO19$Ado(fx5 zb1}aVy{H^bNyo=O9rl-3RWqhMth-EQ0-Z~^F7;At-bjG~bsP^|KZ9esh0Z(XPKQcf zk>l?!1%E=q{u~(vdZd_qVun54l5X`k23iLrmgT}A2h%SN>%xEo^M3slC>X}>HH8(# zCTTzIaXP=^8pft}ptr=NdUepY72*_JAqaKW z_zr&;Qky-!f-~4SBP=@r6h|j~p^pc%;Tk21y%X^`%Xw>P!Pr^mbCuTD`}=TJsI#iL zbbFbiF>tqzd%I>?$jgq!lDK?4wpB#chZg#VuT{?6`EV?s4A%~fuCdpuX@z0B21hEa zZ=Y_P;~fvAg|Vq5!l8hBAZ~J?$RziR3R5A8iD*ceB!^Y@>!r<)9Jw`65Esa!_SLtn zz*}O6g7J`KGQKT$v2L>A^KS`;k+Ei96%4xlgOh`gb?Ry|)~l3+Feddlm1kciW`}%f z{)%`s>8-Q$X|P$GeB{OPBdmhdCT%+ohcG$|5(}%J6j|%InN=vj&ftnCr<=`lwu#v3 zw{J4vr}WE-GKbdc2N-6fcH9l8z~8ZdKeISNazvnTbAzt!+|pEE$3k>EFATz{*m3cUK9}g@?!Rdpw_$Q=f)UQZ^msDnYm6olnO$vk8UL zVCxd4tk|T=&bkP9xVEXf_n$tT)jSCOZTWq8{=otc&j>4wMzKq$e7ce4M#8a=jqor+ zc8Maq5UOq9R_yh6sZBv0I5FXB(uAHJ`5tn=~+6+lj$#0d|&BKhqy_0fTi{~STci4X^@&CZm^ zOc(jK2Wu}tR_&V9u^b++2nIl25ShyToMWI+pu9@i@!1m@5Eg*ap$s?SZhInL1bb^& z9%l&*X*UUUG2qRer}RsmpLKf2cYW~)d~+4A2|ejsXDWMD%2)q}y7Hkf$b=qSQTFW(nw zu3g0;dRxq7H{YU?ByS-H4H(7xAbCR@8UiRQr~%voMMFjrOQwe=?|ua2w7a8>C+7P5 z&xT)UQVc)6mn4OWCv~sO!hWi(8;F;xLCExw-UO!;yzlyw&%N zaho;9h_v6dvi20xvMl(e^x<6&Plb*x-W+k=Fcwc8^HpGzWAW_a{Qz>dSBm82JFrOR zYTRr)L7nz{{>6@q_1E#}hec%})p0dZaDQYgVcXwfWn8pp7}y3bekajM68PY?=x=1jx4jETT6Pj@{3 zXbaF_WDIWnnoG4>aj)8aZvbjAq%6@%p#co@5h{ec4;GPpxZ&i3{k`9(IE1=jU4IP0 zAhOeKu(BrDDP;UxbmB{Ul)=`6Z}D<(nzTJT#x`g?^-~P$4q<-IX|caQ=iB6-&~0MG zg^mv2N7X|`3`4KSas^cleO{CfeX_zH-CAwC$T!bD=juIFT#&0goBMkYfFTM@%YT5K zR6U12K5tSkTJQl{%yKViDBV{+XxM_e6EvS zbjjj%2;?3iDI;5sRf^U--1_={k>v}wjj?XyBI6EJ;vtk)QPrV@-Mb&b3aMuN|ia zDVH?wO$#$@`lVXfrY!BzhVDgdmlpzFQD7GESg+2T@i`cia9c;m433r{Z2@$Kk*7j^ zp-Bh#!QJ1=bRE#*qsN`+0~Br88LuJCSpRLJY+!!GLJIxWKrTQ|f30~ZHyaTNPT?=fgflRDUS0RD&s+$4o z)^~#_subNOkMb+?jBy%;7A>)h2&6U})U2O=+S|=Fy70Noj?~<_hs1VE*ug-MZR~wE zJ+*?LOL<+y;Tlb+-)%$sE;A0N#;@ZD&39c8&+ywkea`w92n|y5DYEce+=YD~xGefY zoC-p+1wqP(n}$P_DUh&<9BZ&XRo%zdUuHtAU%C`m54TY$KP*B2wAIXfWc&>^z!SC8 zz75vF{BeGmPU8zx8#OsBmzc23^?y&wLc6n&H*T!{n;FDM~@`B}1 z@0}b?_xQDk>{Ra316-s20$&)xUuBUaGcHo?m9pgx557&``qt|zp_#Y8uMMKlx)I(p z#RfKZxY<-*k})Ai8juPaf&Kb#_hB!&9!dg?o+cd+`0{Gth{b)?&6#4NsX-lZDyv$q zLocj2&48O-7)Ha=8auQv@55y^P33|(@BL}=hwb}q)0C$)TkZBR#!t}lAwxMk7(hH- z`tG*)SzY)?&x6STVTnyzRT;wS{+r+6#K7kncTLc@>AWW(2ldZW(n2v8ZKP#ze~{3y z&*I6Xb#+kbtl~$Cl6#cx;c%r5Ajij*9c<)TVXaqsjNDTyj!dD_*N0zVdmD}8AmZp& zFKW^u{iiYLn9$)RuIN&G`hGJQ`y=BNH>Ql5R8Og=_BBb_UF<}&YXbg3HSSBA+;64xt z7o40QAHqjG{rhp($v=_;*K4g&_p=Sm4Vr6e>YJGF7F7nFB(IAX`6t#b>hgvQaa=#& z1}AO^x*hl3cqnPKTiAU^ z_)fjcsU~FU^%HOqmpDjMc=KC@hAr$H5?%3@y5xGuL|V8p{tTN{@GVp1#fjmj5!Ac8 znw_3SlqQRahG0CYTjk68*;G)91r#tHA$%u2-&|OQ^?MD-R&%q~*CT1J-`xWg*Mx-yy4K_{DDz+$ACzggKbaNvQ_nEsclF}GEKi$f>f(qp4sWG0ACajp?WT3+*TK4`p5V%t|4GV`89Dt};_ zNS5`$q~2oj^z$@vGd}uFvzWHN(oCbIT+3Cw$zP1@06=uisc&5SQA(|tviSSM4DVaB zXnq0=Id#d0vc4bPxuqP#aX`Et$`VI&{eZNVb{0^`C!a|Y_V0TIh)Xf?-2CYpsz?YQ zWSn2DznNX{E75$aK^S_nGW+_@{%rae9xKvRrod~&4}{!b9Bsmf+($D_({vCAk{}G#8$_T| z>jnbCa`Q9{5CS?8wz(X{o9rfLPW3uu{g=a)C`y!Bv!8Fx<5E;s_175V99McnE?edq zizEA5u-EM8STDe=|D(p^8w(aj4sjO24jvp$vl#*f;iX2>p*V72wm67bi= zH!XP)fc_0|epx$j>Gy^CfH+_e!=_ez4GNWU57?)}_mgkkp{dZl^iv zdL)L(pT?*=FZaL03l3T>1ZkP9Ipa*PQ9uuYhq1M|2vz$T2i*bJh96n5%Gnd5MjWqX zIhWP*x+WfW++zQ^si^09d+UvwLRBF$RCwDb|m}+Z1U|QXlCH$=XAb8D2iUJvjw9@J2{7} zAiH&AY<&0YE-L8AQvI9wUjq2CL8tvpkp~~@T>suv-Fl2fM!?2YzrFx9$@J=>SR9)M zawFpFVbDwY$fP~fQj-;u-bP`Z2m&JJQlv=n0hMMSGfcS1nN2E*>?^Fje(4+PZqQ@Wg)RKh zSI8N$K5QOpvBVEEnMHaq@M6TC$wcfj1kGM0l-b5t2WRWKF2}8l50WQFA zJs~YAew9-0TW&Jt^47P}lOt>YN3NZkPpg>(J5FqF5bNFw%PETGL1@H3>2pvTd+|za z&QrZaVdL%ddbr)v82fZ?f8ug<2A8pgPn6gB25kGrT&U{0#|#o{S3-BQ-!M9YQ`})G z!Q_1=*hn3;hT%JE@~1k+k#&1?{Rq9!Q2Ino5b6M%I}4-4GBw_-1E`)&IGvlb5D4hi zNC$ih z)~t+(gdmxtha^HI@YQFS&)-g&=oCdF5Vhw>4g49hAZbHc4$SgW%k6Y{K*S%f$LEM#mNK}yugQrv9$x?UQ3}G$HcBQmehH4GNIB`oB;_^n&vhtK&b9wN zxKCPbt=YrV2b4K}S$KcZ!R-JJ+~XIeImt5JfED$=N;7N(^2(+Q*#6pr@y;LKO^va& zkAE7#uiqw$8vF6WYp<-z>>_itvkoISzl8{6Rn7=0<|l3 z7r4JJ!wLRb{q=OKcX(qP(WvE8)m)@-nKX4zxi-+b2P~3IaT00~E`w9h@ z3afP@!MgQ1rWh;mhqE~MZ?WAQ%T{YxXf62D!7ZlGSOSVAu2e_TV4l^AFU=H*c{WM} z^K_b(YWTC+8;Uj^@5&Ryk>TP_=CoH02TA(E7FT9WcJCR#g5`f1y!6|cX+URWC^56> z7NG_w$)}Gh6dxZGpcbu-;S#oZJ4;GiCWo;3XJaur1gMr}R`M!yo1}N6e6mRE%22kj zVELWcEJHzPBV-wcQ7payj`I_(9POKTk{mcnA$0ym zx0Y^bZF#(AuWk6OuEF+Jgc0ju-qEXKh}~+_;O!)Ri2%L5!4YZ(vW7=T0QJuXRbaI7 ziizLo3%;%83U)3!eebR8casPkL)b6bara+0g*a(;8}HCFs%*S-wFT?)6o|BK)f-;8 z)6(#aMRT9h$988dk0-Ms#ztz4X|gcvI9x3yIej&hD5_2mtn4O-othaWsX_;?kQ7WQ zo0Q_=G9v3#ifpHAsDqXRezDq+@Y=)a@FRPnmw!M-*d=mH?Iu)`q;3YSodV{fcur4i zP|vJZd|ZWNUnn3&4-Z+Tzn(bF*iT6_4CRMf)FX#+*VWfM(KX}?py4L_J>T+>uX;@H zGWmg~tW0S~-Zezi7sQ{i>)9=#9>VbG!3I0O1DP>>q2<-FERWPQ>eggiMng83$tIe7 zK9gjJ&F@&tFNYs_iC|>lu~Zh$w@#JS`p@nqEG!;w0r*>xXjoOef69AP$kI>krDWDP z1lculL$r!lbExPl&Ne0#CXzd0I^VAmQMolrRX8nk-LP(+SrS^>+>SE^`DM59B_=;3 z=Ko|BVi2la5UR_shp^fr!k>EK=>92=zIq!ppkVO|>o1L$G{Q(jgq1NXs5rHIvRe;i z+w&A?Prmq?g97Cw)n|g-<}WD|HjNhI4HA>mO+$#mv@TwW?} zvaM$OqXCQS6&|oozs1W@;gDY(AfAh=R`oT?2vVs2U9Eks%I3Ntz0?h%+P9j!WiXbn zDIx`bdqY9WXV3A|sW@@R541?rcsxJgT@f1M22mJNgd47vKkEJ(zcqUgIygnXZKXhc zq>MN*w2pV+gj$n(yx&JRID5Vb#vcEX^3zxee_LNjQM2lKMwENCC%7%=qtECwgOJb~ z&=%3p9kdz|bOqjWz)tTC4(oA#gcZgcJ`td#3qM0yDqW)k!b^J7ms0#cjY!LMG;XBxR z!_V@f!u52PlN0s<@r!WA5l)c5h)(O-GVzI!*AVBdT8{TEKg`+eb;K8#RuL^x5pgSY z@4;b!=apX3XhMLbj6d#-q;J&a@c27m@^Kb%WhD=KbHDnQNq7t#yz6Q7I^e zfRb#qciqX!qe1p%+K9$wlh^g90qL<7W#{`V!)mB+Ktg_;=^=ywIP0Rrcj zUT7X$-v?Y-dL_xXm#l#8NjMt^{mmX*#!NLNSvY$7J0HlhXYv z++L_vHzb(e1lt|U z$TFQ1aG<`+Q+k}K5Ln$Ue^dzwnMa}Yw;Jlr<#)n;vgV7%BYX2gz*o>ZhT#VKar@eMi#->32ErnHa>v zmuAfsCggR5=w<>xpp5(LP|>_}z4!#@r$(ju61u=4Q#{N}akhIpFxA0z;;9gof<}O>I+|zZeii`9dq0 zgAa^T&2%Y9oycIjvhS#URX&xeVrl!d%Qa=UyYjBAtkjA1H3aiFi6z-X@CPCP0cmc0!m)T#oCf2xnPL~AmZZ;Tl+mtHL)DwmRcSVL_&4~jkQJ6b=u{M_ z+CddeGYo%VXdsm%jCO)TJNuaS-dO;)Sgx_eJz$dhVu-U7kQ(S!tYNU6&hKN)MlaG! zxf8Mzcg?2%F2XBxBUsdJ$I{Jq3b07nAteSk8loxHfI zZZdEZgG%CUGx@tr4gMzHEmd<~tlM-xaz0A>XiAh{vj93t7E52f5&^^c1*6#CSR zyNEMPIpv0x&FlvEmhkG{l-RT%o|i%nL2&EkKt*GQJ@OXivoqVu>&92fuGFyGti`N| z9eR@u2>i`gRt^^`xzAm;pNTBWXqWC=p!_IV7$3dehkF^ja0Y6{UCU3Vx`YGH3-TpY zIDvmP4_jZ}H|kkK&#XgFDt!+baftUoH+%i61=-CX)0mP7vv z_SX5mrzpYO7OcO^<%qA|{usqKsD$r6t1y?jE6fHrnjd15c^2$4V9b1Sy>WdR8#Z05 zI-7;!%8Bp3rGypA#34{P$^MdPz5GyRXAgnyS{M@Bj%$i`C$N(r>?bxs8|kk7i{+uR zmPf^*s@PSdzNJnB)~sSUdH1HV%i)+BcZ8euKDAANAW*1_dN?x<1YI7_0!@wg8h;gN zc+0ny$!BN5_V|~sL^De(K1_(ss7SbAigMFU)2JsT3;NI4tjwkkO&pr};TI#LRzQpl zz{2o>T8gs+O4_bKyw>GUYjB#J*Jf;H1mD3-r~S9Dh7DzOxx-YJetI~pd-t}%SEesP zvNU}HI*7BBg(qHuUY4V5-l3QSAS!_madoehpN;=sGl!HfZcyj_ zIB;AP0(=V9JD)i&)V#J?SgUWO8gY47lbC3s!F$)y30|c)*YA#q^X8sjVX|}Mh${?j zav#Q1VOwnnAz9$KJ9~NmtITI%o-9gGF1@d}x)f)uaiA&f!~0AZSF(ImiER_%8{wFi z+P>>nOSMmQx}1Z!{wth=&jo7`K+3@BbI#{#nf(eQ4RN`+`rPH)x`LLV&vtPzHH?gU zlc+{GfNuNCajr@)qi(sU4DO@qf}{L29D{MC;%lAs?Qk(ZTN4g0wkqs};vcXLaO>K- z_erI_*s$=X?~ano@5ckOk9C5_U^fvt)RE~+CnZADE!z;=RVv&OeWYoSZu z=lePj=Yo@Z+Gz0~cFcDJyjuS6LX4HIr>T3z@?^b{53dfvIjZV)VWgO}2jQerRoY^I zB1}_~$lP3Osy*e^u-+QHlcnVDJ)*nbD;j$n(WE1qfcc!;W_jo)j?NWc^deNtq%inwINalLG^OSy z6kEDQciLwbt~8M6t5#YP$FT@g_=HuH9B|2a`yIUfqgf!)Mxi|9Z^w(A z$Q*4-{H2i6Dkk-M6kP}H7`1qK`1UN=nVaL~`JS*`VLKsAjijMYq>NH-3U0&rFW4U4 zWyh(12-D)J;poF;cLwKMQ{Q6!H>OUtC|SSEYif4&*2!M_kdSW)W{y*IOr?2_jr}d! zDdeVLfHu~trRK|%5%0bgTgkquq6=mN*-qMPVgs(VFv}cm`SQ|;6XIAk`;N=^ysY=N zzlSM&Djv$KEEPaIrL=djZ3^(?UboSHulAb0l4oI8q zqutKvy2IR>&ST7*u+~zs@K-2-G{8$K5$=2)W;5bsI&9q8FghMcjlGylyFOr@lEi95 zxY+#O)#~0KzNRySmy=%V29C$%i)%fT9i~FT+n>?*==XtItn~1i54+RVuN0)JLE490; zDqwMeGhy}{vAxtQpIj)SG3vh9;*6y;vvrR{n9>dPJ;m3Dp_FrF&Wsa2*jmdaX_S{KgWGGQEIF!eCGP0Le>*~h8efE=i%sEiatZu{ys-4yYMhyPCsXH#WAxxb z{AeD_q{|sgMNgCzE)lXh>^mOOaWz;xKD#CED zc&k{{$*oD-U;jOnX)Y{eOrD@*WSWk zBuWp^(zU&GdQV1Jb-5>mVdXMuXF)p8;JX#eI_pQ`A&vtC`={{%>LJF z*F)+aBTy%)1+iuAyXAd*N@3~ewPJm)|6XWEfS(GE>+M(?hJN#Fm-ujy$ z7)DoyrFhIZX>~WXHvtnyf@V3*{HR{?3K(@Y?QH6tT|S>pn)2{oTgdn1{&-_d&vQHU z_i8|{a=PA3)6ptu(T+_=Z-;%uB5MjX$TOSw{X_2?}adNzBYP@xw~e!NnD@T)H)(r;;dAmGnx$KSM+6guzLHdBpyo`EW_!GN|f z*9XVBzlJKp#{XOrohkxcY>V{fx+HwAtHDqOTE0vkJ9KNI_0CxgvPz`Lb|~OUG!GTF z+t*wvZmANhbsc2WRJ)UD z`mkb|!>g_5WiCpX+!e9_QjZYcH=Hd7Qp6=!13vG0k00}82b%%sp|FO*_lKch9a@aV zn9xf8hJNo0!R8ByNc{qI|^zE56Jt@Pt zWOWh3C75S%0dDntTHR%-5?inaBGW3YFb>46kjBDQC=>HRfrCNgfOrWzs~KQQ)6YK- zFgtt)KdNQOf8|(s&`w3!InT){EPHW#>A~*cS@z6|#rBv4=uR^-*&3fer${#2-kY{( z!87Ebn(xFYF+8R#&jeZ>&aOlC*w>w&&-l0W!VkQ%Wl1NbgfxfR0&qKp;eMdWp>~Dy z+plN|$J@+wgvvH(2NealFlDh@brxDdTJPTEZQUMp$Xn-oEyo}1BNl76h~(j+R4!`= zzehMO-J{0k3ju^Owt}JvEXU}RhSzA$(?;`_MQ!4z;N!HCc__DD3XCi~dFs!5E;rF) z=XIj?+r_o3!ugtu3XP_nJRj9@DaZv0h99Hu_9L zHPZ(mV9Uhh{KBk0)`BE8B4Z!qY#Ljt)*&$P7^WWGL9cx3!4(hBlui|zvS zLqTec2{Xanw+>7~U)HM;9{0^GvkW0uY5UsUPbql8d>^_VAg^aMCMG=nQCzrUn@wVS zebUs+&kT3{W%V(WJJ=(Kfn|#>$C(|s2TDy0s0N~ zv_>Rb>CEh>SjM0W1b|JX_UVm!(UbhCFKPbF@o+(K9^nZ9m6AfDmikdWFxiYlQt8hk zi0WPYe=ntiA`9*_+Ff3av2O?a;Lx8Uy}_@vQVtBd&uEGLfTND!e6OdB&4D0*dK;Yc z*rVngB}n+`IFvt5iU_z7Z8ZPC8wn+%_PzGZo0V1*lTA2A8PM~*C~|mtYTQqdp?Hnw z;}74-zqgk;1DKPAPizwWAJX1BE{ihP~SwM^n0(Kuc2yt1_fQki` z#Y<%u+wajhHvjM2`agARFCE_~I?1-vNNi&)sk%f=^`uJ7z2ES#y7s$iGPBO2|2nQ) z_}(aGH0u7<^9F4FIqtJx0G%#)t(rbAtzG%Vvy+9k8kjFZ_eWfJ+`iPrHdy(K%G7!T~Wq zlIEqjaY*3ca^B@9K8b?4IB8UIj(NCoC=x>=Qo|@B%kljWv8g*H{oxJ&yT0c-zcjXs zl5)**aV-lOZmUf(Zl}ptvPJjJOzOMD{~Ys+M`q*qJi5LyF*r8=!-aTsA=<8Zx2AA) zQ9&tnq3m7q;(xy<8OsUk?{|h?d-U81({Si#E_=%bY}1CdN-s*>pOm+JN0<3Ghi&L_ zd-U5HcuI8*w%rC^{x7d23o}I@uP}!Drn$`qW0EcJC}dw}NN0l_I1S70KVFHAef+lv9}=iVHN_|F5Zh zQc%2g={em$ybsrieI(3d%nYNO<6>0LLS)0hCE*mSYF;SUM6!)!<*$PX({23YX)|Vq zp6jT7`me)A;v29&Od}a8c5!DV(<`XzfW%pQ;E^jHVOO4%JJka()W*Chvy&7V6Ao`a zpM133H~K>28WCIfLH9TtWh?t2+ZP3F0ZPG!icZ=qMvZj1NhQn8I?Iaq&c6jdGxDt% z_lZ>$_67FB3-`ljS^Lua%v$koyU*ASK zRp~bWpy4_7DsPuC#Pa-appoQ%JLQNE7I1>fb~AVx(chGp7) zaz8_a@VT9>MqlNO(*B1Wi3AAvHy_c%yFs}1>OW0P@IFMBP&NG1ZA24#bJg+mod&;4 znkcp2q{IAZdf@R@IW?J639iT*7FyE9v#{^u%4PFY@SZbVv-FH*Rs$c9$El$lv7Q&$ zZsa3_w$}cguDseYIFF4koJF&4&_$me-h-VZRJQ+WbQ0|&`8VHE+|})W6*9dN#pSd* zwPd($sM4*U_dzp5m@TU(%qU5U?mQ<3#bXQ1Y3`@S)O>rpz4I?+6fPkn+*$U|J}q9( zC<9^$Um{8mTxE5k!22#lm8kq@?|8hUH%zaY#NYJf(j#khf38J;jd|7B?VzC{CP6PH zW+z3Lj3TXykera9RHU%{_szsj7?_`+GE4;d+eoD(W%Cr{!|Lss(;udn~dpAIz|NRUA= zXz&N+R3z3W%~&h}39S6V`ir4|6Q$)uHvPDtU#R+jzb;0Fpnc0Gyl5qmRdUs+}oET_ywGcIKt{ z_joNU)Z6mQjKlASl10T!J&drE(_b9yOk9WH{eV6gjQwv{HM0V%{#&EnbBw>FQ9skX zmhrBuNU087BHO{ULZhP?V5jpC^ZRbyUhUg{?;k5o`mYX!7XO!9g#HMW$@QJm#3QJc z{6*lSK_h}()@(5=_4k+hT(CiHC>~{=%%KlwiB#NVl0!_V#OeKihRQvIw`c9&Lxo_F zVGa+~ARs~!eTUr~>E0n?VAVP<={`oQQa`UNq-k++%FM(s@LrlF9Qz*x7BMjMGE1#& zUkm1vf9KHTT>%OmMw(SU+o+kbWQP5DgmkYE2N9VG%LR#WtV}jrup;NHh=|ky2SkP@ z*Cn;9^hH!k`#&Bk^Hp$WpXsZ44|53#EC^Z^+BX?-H8bFbDt>gX9*dk15SN z9J48n8`c*$W%^B18vnk(mK{)7p7Pt-C(Qkb6EGb9+@oS?;t}&3t-4Z&TE0v9%pX|- z}@Gm0EII&Q4!zn`TI$C(juHOW*n#H6^vhiF>h(*{EaG?j_G8q@|xMEBr~uNQMW&ijD;`8-4NFKkaLU_ zWg_W(IO%R8X~7qLNTh*nk`Y?sDLl|X&%4*a<;`DK3py`D)*nTxx9+$BU(CUPAzgRr zOC`!ws`$$&`!^|i4k8Lx6l`9$`;d`YYka^O}* z?D~(8p;JFbFI@csw#aHX+BBw%Sf+qYR|OUo0{LYsZHkLz{FH1^I zjY9KJ7ZF2LT9R;yl`vlRs|yhxf5RGM8rg?8U*Rxhd|sbVz{t*6V&84Imf-N1C2v*> zLic~PjdEWxlUekmPC(<|z5~yv< zP+zKv@kEpkj_^hSmzF6mu82WY`@dY!Fyhg53v2uiBR5l72x3EIab386%l)#9rJG0(gosT_*n{XDkhPMJEAzF6x?%i#om zFD!Pc*P;n_N6X&3TcBIPhJ*$fgPquEU`!^uQLNTQCyage!;io#yWfU~*N95k`Mwih z6{mYWOb4}k*X4q`*4h;vVF27}o`TP@Z`^SGc&a+CnLQGIGf^J8?EQV06hx0*T@V8U z4pA-3f8uk#2xWWPxPbN3H4f(+5{LLj}IL%7|ZRjN0nZ?p>W>)$%0X|@lDdhUX_6~{fA}#Ce2fk z{Uhg0Afc)U^LhZgztF>=#bAiSEYznOrguSgCUdsh zY9R8kdc$c9dec<+qf~*tQ==0N8jwxr2_XdMB)o{wAk4ap0h6hG6ybEa0YT&<+DLJs zzAum_ilRkyAc5qfhIiaI2;(f^9qa+<`|>NL>CEAxtQm%26+kY~f?HlEWSU#DVUN%vsyB-94et_4JZnkqG*yO1j_f?Ren&lLIa&6Ii%d9~% zZTH#)Ww7)+L=Y|(QRyk~h ze^{bTTJ6eCMSOZEHy}`|fZ?RZn)c2&5+WRKzB-luIxL^ikMEZ&%F*Z|megGGT85sV z)z8|3h}u`Jzp_<9WnU?rx-~|%h#^PB&@J1&r#-J+Y!A8HNTT2j{P3*G}kDV!O5Q`@9hlAC=H;yFV9W^H&H*!RetF8S3Kzz2txP;X^#O z8kiNKI9FB7;8XX^OYm$M$!~T&TidJu*EMlyzweKRp0zgSgJmA3=B&K)o!`%0x1~&t zhA+LORCNl8VQv!S2mCQ3b`JR96qpv$0RMA`m_fx;Ns8^n-_c75sD&rSfYerboBX*+OSRvj_1m46a|~b&vl4 zyo>lh?{dj?MtjMVknmp6C9@FDlR|aWNDa?h_~7a@`p=I1>sSBhZTXML5~cxwqNnCB zrQo}Z>ib^kl0~Wk%8F`pucWJBSpTp*K*=vAKFbxk`dvt^7+lWzDe zgPyFOV>uQM%r%V^Znby9smIS2A1S?eZ$bsk@Qm(;*!ju1;1gd`6QZ&e7CBzvEdReB ztp9E5anWvcOYSC>shej>4d;3bx%w9boFw=~+57z?Ki~fMyg@+7=Ce;HeHak%=!F{M zWM=T$7CVGip4J;PwvY z*W~Hu2>*Prf|}tE0akqKAttKjtt_~IN4YdAmqE9blYepg&zo0)x!7 z@mR9fob7`Ql-2GvpVpHQ)VWy%P6yg9?>0@2zl5;p^L-j|_@vM+#^F z6L}wo8&(W_x#v_rpMs)a1Wj;EHDR$7_ebA}^-eGFt@S6Pt{3p_M~k~%rbMJi>V=I- z=-Ep1jOWe%%s~z?AXLbu^C8an!_tP##uP^SkP0-li)UzK5(dmoT2pl6nr7*NVOa+`@OI5dZGqUp$%#y*Pq^W_ zU)#H0>O$M%yDk6^*5RizgL2!llMYR<@v+snzo%KMq7Cvvn&c$i&|f4Sd`H-tkJr3^ z*b1mb%HX9EY$xFP^(F~yZtA@0?GJ2##ldUomsgse0J=cv{MfnZC=QHq*vgrKt!$qI>wuu;LB;;{A&eV*x zkRcCv1-73I*|3ID(I_qvmwUT*u4AcXRT>Ml#xIhi z9k7FVV=Z1Pb=p7Q9nUHFa_^AAg*Ik_CWSi${K}ss*r_?nsf38YP(7+pkyG|VMA)

$RhfZu38T*VgPVDBOu|k=j zPJk8Ad;6DED8L&^#cAwMiX72wGdHe^D{@<-K8$&gu^9jGqcs_yx1>HnDKZ=W z^SuUKOEH_}zK%ZERxh%vO1>VHmWv9HY)` zU@WU*9np~!-KMybTE_>~PK_w=hHZ*YqBWKuCSq~@=91QTptaAwd5!&kS4re-7N!e1 z6M#52!V^Og^X2WCaty;K+-HJ>#J5|2(8B0d3aUtJ@mWwHi z@RGIxvzBjq?O_H-$n5$|7Z+wuZmcNLGI8KPAR6Hc1 zAT0h35V7@MrFJxdK1v#XashUM|~mbJkR_L2)@Cg}E`U^D^Hjsf zMx?v#w@;mzvm54{DGKqrI~jSN2+?_R+1u%%^^lAvz$jqP$!V~iaF~1AQnG=Y(QR%I zr*H<5u?lgiknkb~|20~}xH9o_{lbl$K+~!kVJYF?zSXR*c(?`Zn2Eb(!U}s)$0|~Y zW%n?XUtxkJlxJ%nLzGoQ(0s|9C($@Z4G@K(!@w)R#&^;Bn^vkOkKT6AbiU}VuutyC zh3i3)F9+{MgK8bFbFG_!%(u<s63W*SU- z^=l^yl;eMm4>t@p3#6?)cJ8*Q3?}%5+lZGqOOmdz#ZD&NBiVMVjtADbR|LZ+%?ck#R1l9cF=t24n5wckIGBn zm)qk|5wnn61WXYWrb4@XpSSid$lQEKPOio~s@X%Wqx!X22m4j{c;PW-P!+Q?;)>{? zI8eK&mEKt!o6#nT8bV4khNpoUH_|k`^|g-)cR=AxIj$B7c-Zq_emLoucteqt={7Rn zJmCU=+#NkzG|}%0Fy!pUogMOzi|Kk$$>clzGmpBSv%N1X!^~>02sK+V_B2wup+>^v z+R;~dei(TVfjKQt#TlAc1oRp?{Oe8|Iobkx2gn0AV(KM1^3RDXtiFSH&bq)qZ85fF zdP}+AM8{;cX}k6;Z)U^LRp>efBv21&Z_S3tzj~zg7d(HUxz>)}N|egeKKxrl=0S7% za(}v}U@?|HXTjyfa2!VAk154PX%C2=bl?xiH}qS{sHu_gFr~>C-;V%cv5G`wyQ=;8 zsZ##BxfP||lL2Ys{kK|gIdPqYDMmUWz?4O8$OKmAaG%C>4yrfLtuEhb;#SoUQBgvr z80c#2%^U21~gDMFt^oHXHu=K#7c% z-ru2RC;;+EXcxDNuv3~CNNRo8_pnj+Oy!}nhSoMyqSt1q6O@qVa=jodqzmc$XnRMU zwp0y*%6G0q-0oiWqJ;EVT%t&=vO#XCLNo@E9snO4MmIthq}!+VXZRO|GYN!6Z|#4w z{Q!wFWG+8)KeF;gt#vIFj}|-lUxhD`8>*#WY!t_U%1Uzlm)M!lB)Q^IaPCS zwG5ebW+Z`E`SizKu`tyjRNU(dysz_}V3j|upL}nlGWC&B61D!K-@}i|=ZjK6-{E-*(5G>M z&2vxYD+@lYesA`|Q39M=)IQ&FZNbhTcl1AL3rjB%EEmgG!Z{|3)rP>%k~a83{W&V& zsq_Uxj+Rc;ngRx;%!zWS@#-Q+4N0rGc;mE;NyaqM#%{ztC^srknIn;{+MX%OTaw-E zw(Rxg+pfIkb)P8RE+5ZHOPLa?N+C1dt2YlOb{eABvX-7c4dBElN>QJdw*BLZx#cR-vV3>ODED9znYGUX5hHox#{3#q!v4w+z0>^}mXOii5XJAQ=X<(6foL z6fGlf#ZLZPk#{9QD9)@DIadH6RMh>zPWR>jQ9VT>d&9Ds0*9d>;bpl+}Lh&U)+xIb(CDU6I`r|XfR zDXTglYessM^bNfrB!=WU4sg|SJiRB46lXM3RW`%OHN)Q&)?&3$($-hIGBD}G>G8IU z35W)Cx|>_*uI;gnq|An#cuaAGBeledg0Xt2!@+KUIco+05 zMSWmMSA`CI%iAZ8r2P#=m7pScJ}zr|J?&MM-@K2L{hG$ClKIeDX600pBl^YVgZx%Z zAQyCzMdZ$s67FR@bX;k(Bm9H@pDX}H5!fnbje(Z}vS{Hi=soPPAWkF7wn~%bYgNLj zOabiuY00|B&X{(+iy7eQnK)%2cv=rO=A{3C^{Agl;iB7M(e(;gufMZwZbUEzHqP5-JF%j=g z#xE3o|A?!{TUZL)%B-2T~QtQ0lwsbRm<(NXnh|@x;fQbi~Tm#*>8# z?~6{YZR@!W;d;DRsm8i@#7RG?B+Tl^sJXIC@bSlKn=R#VXg|uLTeAx>UH8>1uXzq- zZQDnzKOEvL=g+(eNElh?Rst-~(fBguvUN>L(1kzWHruVSW{-ZY)ZO~D)Z2>XVJdGk zf*CB%iSN6ZpQuz0>eZNIC*i9Ye_oy@y1CEbjZz&v?zZxRk<$UM5-@ z86G)6Q3(UA_jfL>K78@IFfuxsk{p-k6>%9v@x~J1UE?R8!y1|hj1>Y*F_NC|Qh_a% zAV~sYC$^*OC9!RAVgU5`&eZfaZi?v`7*VDwpxi&dQ# zTr5#+*Yv5Z0XP&(VG#hnRafo}2+&&aw=2KZl7w6guya)4zyBOq<`AF`r2UA!W0Mej z-#n+3+|?izc>|y*G*3?S){@|NRnG(N{xRW2jXs8xOG@BGnZSH)pRzgFWS*o|Enx1D zt6fdd!=`LJ2jCdgp1=*rPaLZQg4G& zaC>XaVsBfoS12wla%&C^E=9V&fisfpy>)DTsW)=|~|C{*=uq~O(LW7P>ind~YQ z4v$p3JLdP7IZC~#KgtD}4e`k) zeNn;8n;$@&x#2neUPj2(MWIUjMdtZprCoU}gI}xTH4OcTPcK{JBu=?{IW~y>I%}ob zk@2ozn$eNpbKBep1pIhOmz@iRJ0jk$^Vc)&>+<`5e@D;&%kFHS{ERs_Yd!NCP*euJ z$$6d~E_u^~^3Zo|Kt6c3T+5O>re_L9>BjpKBDw+_%y2-adZ{Fh8at*}NI}EaazzaO z={m)Td%tKFC*ReKilf0RjTu4zVmq|4-swFHFUy+!P31?;mCSY$!Av%2kn;O$`bWsE z@}GovsK%&QYa=vw#_)q5px5&|E6|!9tnb{I-9dgF7JPtV5X^e&fZ^|(-?sOs=z~%4 z=!MVcWW=i%D}80rs+6i+E&cgvncT;+Sm_rjQw9aDj5n;waV3w4&G7fPiC^?jp0E0` zA6cq9Fi^L(N}D$*OPfrxH|=3Y|wDX^;B;(J>&u1X&3XDzZ4aBh(aTo`Jfa52^=C7!GR8m-<>+j z!3sc;SEVIQpuH#gatU5vCyBms7(jD~SxNUHUeRtZX1{CkiB}wT?J%Qn@S+ zLO<8zgjRQ5m%sX}fIkaJJj4ms6eI#8l;?8N_?AUf5@j&$|rqzx2xO2F&VYNZ~ z*T()qTz;{!WbkA2)2Ik9xJW)(&CO;O!|Eo53+yCCLy5zLz@jxtZt~ne0cfI^~T`(o^-T!5YY>^MetsiWLS(juHb$I&}&$VHbnNO*_VkTqLGc184UWGVa8Q*ca z=5^;p&jEZ?37Y}L!mIsVB*!!Qo6lOzJ8>E5{A;d>UZ+z^<#w$f%Rn&EE$U1RUm;i_ z7z%p1RyZEaaFiFLU?cuPn_aR7ZFMy76dk}DW+f&-Zx)xiCeANxDryhm?N0!5p zZ9Ox30WY3(?<-{F+U%B;k`%KZLUA-XOtb5s^zY)#U|PXL&u~_RMBBMnwck%e1%8Nl z5K7sB)1>UM>UT)zP8^TAHL!_gg{Sf%=jB`1%TwW}U<+@>5Uno)=RX@3a~ci_aX~Y7 zD$K)0VO^WJZGHCu++g;uU4ouOLvYFVv#+?@_4vL#4|L(T0jjm&wQ>^j{#%vkyqrTw z5cm5sy_D|$eo^xH(R5zDEZ=Vy($Ji2Da<#l#*-NP(F*{4->9@R7#dwKds`_!-|v*# zh`ogLMKm>nxs9f3S1jWXs{JwZCYSjmEIyRJusxH@#ILaSUbL#+I}XkAhEdEE(2Y)0jEe|hlH!5_Z_FZiKHjrWVW^bq*+lN7d}rApheOhM9pt zlq(=)db|cgTd0tfXg>-o`uK<43qlMjd@V-yS<+0?29gd@Sluk z<1Ra|Nz5k_@{goKy~9TeOX}pnl&%dZ1bjz4e467d`|D8tD@+@lSabC!s00p>q#t`y zwjwWrKwJt_{k&FkGVbFuI>eQW|&*bnsb{#Kn8Is>qM(4l2ojNvZbG6niQC^w3P3G&&7Zvc=Jle^#Rd$`zWJ;cP$q)rO6bFPn11ZnEuiZc8G(`Bu!7A=qt6r>Z z9*Bp038Xsh^S!=eJmy~%$~RwA6nNSyyrf_4;m~Q)$cmNoo&Qaxq1EA3fAI3bG&m_8 zZh2DYtTk5y?1W@S2XuMeF}op<#ChEvm(}hJGI<;ZonPWN+AdJeoyXqW^rjNlyfUho zm*~hYsa>l3#+{6Htn5uJJ{GT7F(Ko}Zew!UMSJW5pT#AH+&o_G?C}T+i4v?Dj3x5( z>*K+21|bHubDxKsE8J*YPgf~of0NFXD!DLXl2Y- z4q!thRoJ$|^6i%D>Z}S+b{dU^;V;yp-s2t?LOieD`h73JW~~Gd^ioGb}`jFY}vd=)l@A?N5Opt0bkdqi>Ukzcx2X z01s5(Q{SbsxEnS)j4^AL&mJ8&{orJ_UJ=A(u(Jlz<-$>CSylVvITdgpTRGf_M_HShe@r$=LNV_pQvvXAfebAzR}{X=DE(%G%4GUm_r}x6mFayVyXcL z((H%pJq$OFAV_gbe9c+hU~jT0IJ575>tbKeAvymU0RQOG{+OyUtucojBr|-ZZsYn` zlCvdsMJ^lfhpoq*xpQ3#r`Sd?!?(TmG&1jQtm6Ks z@_eIAbjzZr%^V;f7sf-cj1TO~)GJY<7&Rg8t4bd<=|^kynq!XMVY8~%)1bH0zH+@F zTr}SnU`Yc}7cS#OW&LefhAkMSitjy&BonHEzV}2s>DN}4NR9zTNo3107J@xP_QyEF zFs!_5`?$^j;LYVrV>Z4l>=}O|f-l*G+>w^TWj0uAo4!C*?D3@wEYfgy>*~zUg#7d? ze+nrDYK(lJ_K1(E`GR%zM?9zR3y+jnD!u>=Ax0J7?2lIP_=i3p-vDC#L3e{0w7-U~ ze_^(m9y^9ISYP6olHw(@{(3w;yd`GW4^CV$Dc8Qrw|S^qvk@gC#$AWymiv6Q_=%Xq z;~A;j=_1(mc3`gZG4@B$Uv|*ZY5Vu_GTM#PqsPzpKAj~zALrJ-jXsiY+`E-KdM0lVM4ELpt1mg5g5bqxtX~1*D_P-$r$&W}cEGfDSOs-!gOTrLkBm#N z_wGdBwa9kpDCOm+zN?tMyw~&s7a_|}=B&Dg{x%{-3~|Qmm44!GNJygLYQ7ZbN7NZu zDU=bmr}lBZh%qSZn7b1kZpHvrgA`_sLV$6%m}&%zMoaFI%@0QKm}}v{w@+Griku#? z;r$52g)8ICW{y9|KG=``i0bo=oS#ck$t!;@L6p zxN$c2_egAQXZKPK*nRjz*hMKszd!=HGAR<|SL=4@4ajfnt)-;svgIoEYcZyys7$lR zSfffZ1;78Tcg#f>XL6(b`t!hy?B$><_9FdHT*`8BidguYeQ2sEQflfg3mE2W9?BKUD|({`^^S(1}tmk zvsC=7-)kT{cI}ERKN52K^VxebJ#J;z#o}fBlErkcw~rNt{d~x87-?jy9xxGE#?taS z%J}pVsZ<_Ns02u~)6Mp8MM$vX?h07G6u9d*Q!a(0eKB&3o_^QQRV$>HY940bJp9nm zGZwC##XR*ZZ2E2HAMng2`>xA*xZ`F zQ4c@f4kFC$38MVtl-?T%a`Jl_99ay%N{0Xppx>x^Q0=d5T-2Tk(^_RYAw9t|+Got~ zJn3JjhwI}gxj)YbX>5QVw-q=;Ir<+wOG>>R*6Vb-=FYfe)bWBg4sneoUEX?NjZ6FR ze|Sg;ht*dKB6Z9rmCZPjBuG9|ufS&iZlxKZmhDzh`+jZG*GU`j6zdW}Lb=syZtN7H zflCCB60Nd9#P9u=fpih5r3u4M1eLi3!hi_+i=3k#Ho_{_rf;*E?%$v{%c;duR+#Az+but7&d_nz^5>0;9stbaD zC)D@_oz^FPs!X!M?y|z%A8Ow4(>vz2-bVhd#b3VBUD1pf2juf$nn~E=c%Wzz9Di_J zs@p3~A+qm!Fgf0*rYlD^9s6!QQ*4zwo$Gshk1K}wiIx4D+yc6jC#%@70;k<;wqEc3 zGa|#MLw@KgYLc>VKbp*aci+d=Jgzp zQ7L^?+>dAFCNiHY4M8SB>_6mHV|pmmuK~P#Jg#sXb)nCqYq0l@H_s27JKO3`l;x5m zH`@g()>7AnDvlFuvUywCYOB*K8`g9lJ_DY@ew8>1|JJYii)4rqC$MvG$-5&9B^l58 zlQQ*^ky%w^C(6@mth~LK8#N z3$i&)5C<)BVqSL*nwPR@lu}#$FzGP1G?FxsHdB{}Zx^CCY&P4nPR8FyQHD1V)_jf zP)IZSNHcKS1dd-#0oNYJx=3xTRdvI~*}mrJlqKAv&epiejbc8c^-><;%du$J_-6;S z83!Rv4$Z@QsH0gV>1c|fBHmq9J&(xLWivSdcm)rsKN$>PH-eVyw!lJ(3Nlsj+18Hp zh^4AUU$XOako9y7yG`N2L`H?>Xd2BIBdJi_60q2EL_G&LzZ6T-=qmUDsk@8V+$sEkW_mpQ;ej!)&k~Yb7&}RJY?!f>* zerd_vsDdx!R*aYl!%b-7K}S>cm$934`ukh(!i%|@l`zj|!x(G>YYJ9oUXU&8dp?w> zy@ls{=+E4}-Uq4-e2=hw8;|a3?cJjkHod<>I{6-4@J$dE(GE*;GnKU|U@Z98)LU_Lju~^qagA+Ew<&J9D1@m3(Kr)_!zWin{@v54!;181nG(-NL`u;{+Y_ zc&c-+oasvu6Su`)g8L!3Z;3~REP8*lM`0p+bb26O!|Nkw+M6z-J+}|!zHFGItpLQu z38shS%Bx(z(Ho$A|7$*Kz*gOEs8yl+7IIy3i3;Cu{4}^t>#JL)wu$}OEOzzx&F)x* z^I|c-3Hx@-r=mk92|u!Z?$Vw^xLm=8HIN5~l`}__`GzA69s?6>p%QS%&G-zi@TJ%n z3dIJvw9f1Pw+<=uJJK*hQq1}kir4=is+qrP--9j2_?dC!< zb9Nr%m(0@Uf(>|8%sZc%&Zb0T5=P)`sp|V*4?UNd9CfdJ2yqT>?-n+wcS_Dfi1U9x zj69$wBzVAUpoD&$O9?g3o-H@e2yD(Cn8Y4rj-&io#D9c7UtEnF1$#caV{_wOuU|qA zgfZllwt^n9_^R^lx8fJg0b7e;SmX%D498EP?8)^odS=dL$)!{n?qHIj|M&Q-fZ!bt z6E1zj=UKYdU_mE1RfhCFbPvJ^8*V*Oyex70oNMG;RFM?3gR7LoQ)oEF5qARr!k!Uy z*`K5W-q!jZ_Mn=dhw$_H!Z9A!B;kB#0SH5$809N!N&>|M0_3t*nJ*_NlN+ZMa8cKB z;AXW}m3EnPe9=wdfzN%bB`?t+IIh!M9|jUJ&K=y3S+BQj9vlp;`r*La={0B5iQmDD zqw5NuaZ)TX=iuRB{%d6mc_JUo?WuZGM*hTkXYj~C$h`&3h(`V$mhn;g1W5cS zcpdc9GP;Fp7ZLr)Ots2(uBylx4R7=EW|+bsB`n3|Em+6sSV4~>3w08}Td4_O! z>n&VzmnsMuAa2j?^5jJp!Fu<6@-rjSje;lAoUQxmjRajC^cS~xY?W>$^Zqu48nCN< z*WM6SS+hf%Ypji72gLSn-GoYi-nl>~HsPsDE-}gt`?n-6xj^Z{CK)Otc^$NG%Qyb) z@g8JEM2vHc%OvAVJb$-toKPctzx-!U5~=0J`#9#4WneS~*ZS(JWg7izxmZuC!@dxfPv z#)}*~o=~QnO$^g^KV$L$K!;VONNqG z7cm@dZatpYC*3ubN(-i{u!od+GwCeG)8Abv>5#y>zOEsVML(FaE{@Dmp^hh2oxU!^ zvHM$|KcG+wjz=5leZ$C_vsqY4uC#ov1?cg^5QQG&mU2`2H=rZ8RKZKKT?v$WZt+wfbsNO0@f zg}bl_Q#?JlOqCD=S?Emm?m357Aq+ zJk2zx=}g=wPx*;1?oQids+Cl=npeHO&fzUJ$oaQ~cL0z<5zV_Z>X|ekwjyjnWyi4x z@o6ql8dJS^qT`(>48CTIWIgb5+(zcmExe?{8`SVeO_pBGYm4UgxiwxY2RTDEiD>sB z=cXIoYNMBtSWTL`bsp~pp+X+)cz;$>_iZ>|KKn;Gv!@OEPs6_A64~LszrPgd=s(an zZH3$!0|&I?OdNtFy)Yk{<|NCC;M}Y{j++Wg>atoiW}+J!`eDJi1dk2iGezM++#i0^ zk9KY5LFN)WpJQ@y(x?bdHD6omLSRgGwZ?teZp^vkeVLJpQt#Nanmb`< z?y7^y$Xt)d_&46%ZPmt$_K$~Z3`GNqYT{AZUyvO9fG{avCVDBe9Zk+^Rf0OHT7q~= zW-PhFp@0jvWQvX=Us$cOX*puYE{l9BGuy=g(}(v(f`1Z;St46;d|sb1`#$vH)@Zh1 z-oPC09uwm#qDUprND^`sD@n_wv3?Wm#>Gz_+xt-A2E+;CUd%H|M}|uwAF1=Ue?J)a z;Hc|iUu2b)qI#njm0ukGqMWtRGrgqa0Fn?E#96Tcj&5(-8p z7!~5u%JTj3)+1pI%dic3zeJ(`+f8!(kAmQH?)dS3l$`R&r^N4-)^Dtxi8R?GYGXK+ z{t)w*YEco0J$3UNqExT$_20@MEoOPO+G<@EZ3Iwl(7hadTAYJ!_Qh?!+C=#^L&j%k zeyQ4lrltSL*~{Ee52(IbbNcQKsCTmn zo?$Z5yww!@S-2<9;d}ixNsJ1o%*7Bm01aaEe!_UGfz#lNDex|@@JT2hrLaI!zK&ao z`wJdU{M`h5bPs%SII@*4$IH9>)m#N*=CCoq|GNOaW-HX)KEQvpue4#7?Cx z{@*xz%c!`brr#5Sy9Rd+gy62h-66PZa0~7lTmu9M?(XjHE{!(s?$-45^St-owdTXj znl+#M^p>i9_Bp3^)$b21L*Aca+3Ug8VE&I-Az-RY{}IEw=jjQ^6FK;;sFfivJjCG- zMyIBHj8LC>Vx7q+8!;TSx~U8MrM9D-ujLg@HjA)1sOp~rBfftk;c@>g`|vAa1K5>p zS;$_Il^|@niefA*M1F<-0?N7eS4;j4MA5@v|0u}a6#D=nUt_#~PQ*n@;ENfa;vI zb-!-utT&C`cpor0n1lhN!#q0?j0T5i#xO{La==32&_sT@A**4s-fZ&?l}FMeBwvj4 zh41g^7w?b#WaTkf<|g!UhtminJ2A|ooY{e`kSXkWNW8$VoQiI8Nd?3Um;Vnx%L96` zU#@(EKz}jMu-Uqdz>dpwLe4^*bUzuxEwvkofUy#0>4XA_3hMpZYIwi@R-;ly_2CT6 zO#pkmZiYjHBhAj9nFX&`#KG7XK31Wv{f?|z)&ND#MXB2c_%QnZIg*@NK8G;?Pi`KT z(jU`gc3YalmtcGMn;ejet5uvN30frbHDD;}kG458{%6=AoM4l!`}rFE7X>&5`r{F0 z)F`Z7nP`8zE`_jx&XYXlfxi58^cz1V4P0ktUv@eU-&O{-FQ13TNbWtevver4OuGg`Np8sV#SuQn%VXW$? zb7Ia7H?8CI-naI>x1s2P-)qOV(O5Unqp%SD%l;24)_ogY9{(bs{O19*y4^|goP{cd zD%B}zm-pij-m%26>g_@CI(djN;3UFVe*@@0>n)mhmQ#gO-%zgxIqo{hHGXu8toB~t zP4_!-+~!INwl0(j-?-(uru}rr0!Jf zM8pjx6-$D2Up8k|6iiY!H)D=ByIP<3Ehy0Rfdm6mZ50w*6KAvg9C;1z7GLhY)s`;( zCxg=@zjXc27aeR@H(@XE7^dqUViIS5v4nV?odq+=n2JLXBiB;6Qtz|Ly2}R+Osksl z!kXO-;L)hnDQGR2ZQ=|zrbrFYxY=Ws!x$;as3~Txl#7m6lodePMPoy>+93Lmrg!2B zB_ibHy7r1ma;+!twWX`xsz^~RmMx2zI6-II>x6l!__L;U1Cm_#i~Gk^Nc$aWK3ZdQ zOA}o7c{o!d+`Y3Ha#JRx!j~>qfuMzc7ey>JCXFGU6r4o$a{f-02$j|q_6rktkm|GB zv4YEYU_U{yr^sx;hLd9qT%takQ{4r5|KP$=IKCo-u?u^v!W{N))BP%Qr2u$eE~s0j zPy66)=6$se$qg|xw^C$c9vbDmoY$NY>I14@B#>3Z4s10XgyTh8tdzlvRh}NPm=h7n zmCUP|ryit`0koKWI@a3ysrRt$Z+52`HNGKAs-Rys1km1lE{H@e{->Ky!>x?+JhwP{=B7$%kkTON&)_1L|?Vuvy)=)VaYMdkV_ckoEfb zYh@ui4wq>z-A;&I{IS96yFVwblA2X+KDN+#BmF%rB#Sh^Jtx-n=Xc;_4Kz0=7LSAi zQ>BQ@#GmnJYV6eU6jW%@Q`0zMv^sU6$ak&pyyEv|*_&nRR{OMhwdZlB*sq6ef|Hnj zWNM=2idIobKZPPFC)D@pM}~18QmB5GI4KJr zzv?*<37*dr(UEwEX&;hL{k?V#sdcO(3&w$x3GDZ0QHa|nyts5mx7>WD$4>yfM3`Bw zrTpZzTg4JwVZ~B`k3dAI$N4( z5$; zzG1NJEBBHNQY|57+9bpU>2A`MK3(A4;Sc`#jL<)5cIL=R4UK;`gC=IJ3g^l>uPx0` zlQbo?W(mT~^k}kKzBx29PW)sgHyY5fhKQ>1nrHbJ#uoaJ5>)9)PD9ct^1`A-?*iv( zufcvzKYBJ!4r6r`aHmf;j7vRe;;cN}(&&-1b!rO0MstDnBKfSoT`((xL!2@Yfi}Xw z#^R+B4U?SjPG$=%UyW021XEuw0*rxG6jL}cJ8M?dtF=j!m`qfAI+pzNR zTftlnzaCEH-}a!}C5-cK0=e)N_$gstHX)~k1B)}dd<5s4Z)i<~KIEDe{}4RUL?hZ5 z<%^lpJL-}BFU1l6wKU}ORFB;YKA2w19V)TeK ze{2Noe9R2Ws!LepJe$4caL+%3c07L1Gz11Ty|eW2D4?zwnPjF9Q+{`n#S{sZPVx~e zG+~PC-iQBhAz5$aUCt2YO!_)!b1s&fFJ{O}f|G{*GB59PPD4x({VqVzJ&`c~cSPL` zO$`G!nyNbgKC=uWV1hH37IQ-pgM+v0gY6<1WNH7CCo0W?V3aFmPz>EB zzmF#Bc2|VfXF4=USPpp>9hv1ftN04Y^{3up3;xCSY?xnwO#k&_2)O_vQWRQFZkR=2 z!HK!^ggW(a#7xp(vqwrigtWh}<-J%!O>B>&DbBEd$1xWHXtT`v8#M&b#KWk{#ei@nwY!0Vk%(lWV4r1B)qh$a+RuXk;D6=tUgJfZ; zlWsIjc&x4)_b7Wl0$uFX#dQpR9}a6po9H1x=}IK*;>eSSiI zEmX*Lfu`xD`gP5UaB)>MIA9+50mtiiFgySeKY)GE2jKi={g7wxm z@=+I%0zS4Dkodp>E-gZ^nNTH{!Z{^9|tgMKyS8A4nEvOW~E>;X`lj+@rG$Xo~hZvUZzIe zGi&9Ed3oCU$gZ6?UDgC8_T}R$Gb|0%2RN-Ae*z@<=oMV(R15JGN*jBXufu|XqKj4b&saMuFkG-Q$r ze?98p;B5p)AyU=^CU9FSJr{c?gn)Pb1D?V&J?`boOh)G6d`aDp`<=rp2kl=1s`yC} zkQkN0Ui|)QMdVBPK5A5)lfvIIb3(*;Lneic`;ZM1t5Jl{N`PM7OF;|!u*PqMLmPle zM-`yVtCLa@DT+gLVs|UOv|;x0;Yb;Ya--KA8Qe&Kre`|4klCTI*KhJi5tiweP8pGH zM;h`X*bx&2l3-G%AK0fNSt>)-sUO*v3-fo)x}G1VUX)hY)DP;fjJIVY|&0gmY^Nxz4o~BNDX0tETyx?yH*BJ^jHIR_VYZ|X8CMQc!k9Oj4Hs0eK z?`KBGs`r3odlVpvVtI1JE}er)6!2{Jr1KU+7rCMrgDJ$%%3pt#%K)!ASGVU)$$xt# zH;K?5hNF_9La9h()~!A}ECq5x6)AlA$+*WtkfWb<>l6kgC>=xO@QY*$|44DCU=k|- zL}}-Lh=;^2r}TUeA#O{KeQ_|9(}KG{2m;9-JtrIT$ki>_x!!7vc|tZU0UxY z9e6&nykKSw>Uwk5LZM;eCuH{^20w2GyGsWw0Z}n*r^X}X&#Dg)#b_~`OM$wa(WjZb?{BUj z;Ln`|MG@BW$;AAf3THcSSd+lUXhcmbKzs)C%*((P#)SF!=3PlWc(wyDJp0+&5pGBg#^AkE=6QI$`N!>!-R-7qnL zg1RP{FWu8AQQ7z}&JFO_Bq%M z=`RAC6XbDP3hLMC-8-GQ*)#dlV=lu4&q7WTE3Cj>_O+%P`|3FpIvZ+?tvjGuCAFJ0 zpuZq!(81aof_#6_DWHp8j5aVtJ`W`l4!kD0v;@UN-PW4FH{W*1bh>|F>G~=`>62)Y z`(NjUj!}~fx_!4pA23JB62^=8w!hq~_)f $g{ywhd8PWDh!O<+}Qvvf;*ng1sGH zMu}DX3`ax{|7jJIVn&?!0rq06s{?>fPd0z_!SX_pN>u*%e0ij1(B4V$jZWn#HX5jSG-a%5uLrIF=uytw3w*=p~?Go?0HFqahAZAsq28&0!>xxzzD& zE2vvg6iLEL%>8E1ellU1DQQRkjB7_~ddoj+-HRJn(wrk69BdlS$vuvdz271OC>9%} zgNi1VlsH~ep}>5i5K4lt{oeh_l)&#j4M*ZnO!(0lQ-546LGM|vHJ(Pfohc3xV*nEWj)VQtOQ!VHb}e-1`rZrvX6Col;e+WmQ#xpq}ZFm==jlj`f_B5 zc^DrvGLm2zgvL^;^bbzcPG&~b?tI61_j{*J5Mf`ovg#=X58dxOA#0n?` z^%EmoIAcPk!CJjYc`Pu3%v*a&Iww9%^#=zQRD%{$U#zFh!1C{>n7N*taN;~ARqTAQ zW+=B^Q5nZfITH@1MV>1k;S@;b~IUvdeT(VF2dyBzWaHRCJlmoV#hU&GOUx@!I3*F&z z`xWmLge`dKdw8@;6QnQOul1LLo!H#Q(dHs?=t{%x@J)Tmv8AS%4QbTj-*Q&txIbD% zZyj0j`Zf@+9=usfv9hdWr2HXA2O7j=g66fSYY(57j;4fnEjRK6!e4lNdGfDBjslpa zQhQ6E5~(9JlcO_B8>abDTo3+~TV7@zC}auBZX|+3V~?IFk=o8zEC?Dp zhGee7e-TSG)gr}AEFeD2sK|am#YlyOKja|;zk@iB=uP;4en|}v9y`AkO6RvPFZFW~ zOhTe9n3_6QbmE*pZ05ebi2zY@A>8d>@#ck3iI=whtNg)a6ofXtBH12wr`w{{u@g#U zqo&dpOp15Y+J=*Vv+3iO{6wP)x(e%aE;msL3zAAdNeHzG5iy>VKG{C%5X2}!&_d$4 zg`7o{i_~ft55PFoD-M{TD^s5QOLX0&WJ)4GSwsRRrER=<%t^+YhrH!QkgJ(V$75kQ z_=2rtvLmE6pbF~Vg$SNH3uBsdPY42%#tvaRr)(Cc3?@G1+MjVp(N_tEjw_xlz-lqx zqlkYF0tz#~UIqu5m*3JAD68!>96#ShC*t`z{Jscevn9&df+;hZ#wTH^#ivb5f&hvF3o0p-a7M^dP;vwZ~0Er<&=G~@uDi@4f%?bK-t+2zQfQ|<#E`c zux~OwKvt~)Jg-YNIl~+s80^xf&EI!=>MH*7kTZ0tE?VO}sey9r^{V#=!RN~jq9rSh z4!q>h5Bj?E7jn23=xg;sqhdgN@j_?j)Kg|we%x11I0G9zSh%dODOBrrhlNMT@k2TP z-H>W%_JAzIW6S_op+`5_#?3z4PG2fyx;^0H#m#22lwbU_VIG{zN&E5ASr6{{U(Y#a zN)2=SxDV*dD^UM=cM+h%W2bnHF6L>WAY_Ts(bmchOV-9w_JF9;bAxjz#Mrh1y0 z#*P>2urbb#yKekFcZ&v4QU|}bOHw|WUt9&w^MTRH1SLq&7C*&jUGlTMu{})%+JN3ywS!Z-9!8!}28{Iv8u#}55pbq`*+JL@iQjy~p zT_3SpuS==P;aA1v0cQ`ifEMpQ5yktrbID+yEk>JRlVu1yd!;Zh>?VxWTX(dIE9te zVRtNaGwo4)rd-lRAB^L8ZcpCBXI*(QSovXi&QXx#r6VQCkS&ENQD`c!lH1N$w>(nO z+_8psCPCXRqaEs!cY4IfS*aF;cYU`zV8fuyHGt0w72 zdV#Puvhs2u$op%hk*77X-e1-;b*%^DSqubg`Z3=@#p2CvLQfg7kHqQ7W9t~#rBls{ zUV2av*D@0cHh$vG$W1$B9ac7Rn{{uLy*^#AxD5xeK?|PX>Rip&VcY$TLf+xH*4@B& zJ=eExL?ygsra8BP)Yc&wcRf;O4SWZ^%u;fp3RZP=%F^5+(U)dAv?5L8|M&{%g2VVA zd)~hA%;`xQ@OoUZ1#ARPS@`m}Z-XvlnnZ=I)h{$X!~kLCId$(mD=I>US9ww2Gs7c- z?y(h{TyM3gjo-81Sx8eAg2a&=B>q#d2?~7rO`JCO)fcSrS{Ey_yWJj)K65G#_a9i_ zk6$(`sV{QF$|=02CnaD`jS#+ogpxJ1?py7n0QHnFe&g=W9jUgbngl$x2aJ}!U-ZQz zFI)0-?r1cQUi`qGfRTD|?_&uj(vvfI7q`h|TMxbY5S#b|>*kd)(uBctin zWckTShD*&awyxLz@HK)ipBLB@%+{yq>7Nz=)1%j}!9a+yLA+yv^Ai6Ae50^ay?PU` zigHNTI^IWeuiJsDBhMm#XMS$2Q=F|&$XmA#aKw$ff(P1A|14@XgkQaw zs{M}hg=025EGYZ1g_cumQH%yzH1vvpD{YK9l9Df~|8?vP9h$O=+HM=Tkz@!Z^S7x` zX8nFgvg~BmFpHM`LoHG&LW1*!U>?ucLP^Aco1;80I|cR?`(hfc2l?RZXfzVciFGun z-fIBoIry0%7EWEM%ef!SdnEk}P6=u>-KC%i4=?JR#rrcPtGyVYycQCVT?8Ya#z;%- znAIp3*vF6XDCCkQp;=Z7Ud_S&BM<;cDti#Iw^BJk9=2LK6Ef6VX{;r-51vfqC}a|) z{ES{(Fq;!wcin$yV9rr;R}de#+B!%fWA=^`D9x<-`raE)h(JtuY!ug|X005jReS?8 z)9A>i^flWHZ+HY+XWKa|h1o!NV>%h>u$bYmJpnM3Xw-p_N1&HA=SO==d+ClO`uhmY`(IwUcz)C3&m!|=;^6|xPiDJ?NVkdQ=)?DCCr08n12d{2CYwI)fz>KSVF9(T17kYfOT;KXrC;k4w0D>vZn3byq%l!wu z`xR6Sj-u=K_RU6M;>r`^bfQ8cwQ8YdSeov$Ced^pRAL9hC#XtgtRei|DqjiJ)gWxbDM`F18j3t1k?~P@JGJ#}?oxQoHdJm4(uCpOH7?XgS8NR$Qw)=M zDA5{+&@|P?4fa!#`$oydNGmFeqSMMhe@&~PtF@4%BUAEbm8uwrM;b8~k_jL93ybG$ z>rCa1bnY;nQtzzPd__*k;c%G{w4YyUU6*fqhB$m_(GW}a!}VyA_MuQn`EJfaq=*|# z_x8~MZu{AfhFd;$-xp~mEVI&%^cSjFHCTYUYz`Paq{TD&qVcjLmPNmqII)Z0S$Iy1 zIV*KAoiljPV9$qNj%Ut3Gt`0;K5u~&<1X3N^WrQvi$?;_O9|7+-^h|Q6(V-xQU&F zU3F_IUQgwCv+1xP1(*&L8)9c=gs8tb6=glnY{Q#mc=)_SpZ%=tiMhvZYl zpG|YK9O8Ob2~xZ{Jn75#NXGyzt-I^eI~LS%lpL{{>i0_*--|#SkP3*^y1;n|f8O^GhKK)$Bpse!|;vT;>S@593Q`!g( zUZCF*No6t!zPjV7o%ySCSk&M=TW|FITecqB-kzwBw`0;{UlHFJy(mQ!mxKU#)ZH6t)} zk;(VBqknVL5|mz9kK@^p1%3d`^HP}Y8Rt>xXpNAAfV)pQYV3k3nX!&sJTYIjs$;mM zo&RcidOAO@YC2snLuHyzNLVDha@wcgzVVU~QEPaxM8Meaj`lqdGc~ztvmIYuwe_bhes;X1P8V!Yr_N%hq*P1dA9yXF|7zz58EJ{)Qrv5!2&moExK* zFH|DmKUx%4*>At)PVL%kkL{!@yryxd_6&ERABpXPvNA0#uDf_KP9mQjc^3yd-eTJe z41E52oX94N(bJovNq^+{YH&5auvh+SaRky~d%ef0cJvw;dRR4rK`-&#{~W^(vElSv zy=3${f8LN^zQuhiiV()YFdQ^t2vaw_T7^+N{?+fO$ZCkw)KlSTM^XIY^iGFBu0PPr z7hBmeDsT&Ri%-H_2cGHoa9B^Zr-GB}u*eK(;25$T2#+RQkj7fVL@4uaLeBg;FY(V9EqE;#7VC@x>ex)08)#rH)-`2b5BqaTj3QVMp z#-A9bdJ<{<7)Ry04b@V3D@g+r3!kgoNpUdJZi(_{j242Qz#1$Qx-C=Q5DS7!gk#zo9m1v%^LkC_F!tc;b^Zu zneZ5iPx^r4c|axB%-FpYY@`^Pl#F;3Yws!6LogvF;h358YR406#{N)eE<$!)?vTY5 zS*mps{#`Jc=Wy_U2CAJeYl2#mw2MVRY(#_rOT-v#SdANkSJmaS3MwpHD2XcB^VwD; z%rxDFo`sq6oW-v5z)=6c+voqoRO3;qub&aTHxxW|^dB#wCI83xLuOngRlj+hWvp$j zk)GpmBL3r<+!hrbu2*-P^%U1XkbYvl3eAZ~UjX*9l0G0vNXh!KuHyd`idRX3q|JdkNVxfUNHIi`GwOU8`loaLJCeY9q<1bj@6)l8ykrAhhCafeuO%q&ITc_CG&ShojVh*Xx3=Jqd4F~E zER8irxsG-6GD~eOtyR`-F|ri-2J7>G*m@fAevf~xD3iL4l%mQkC)9&`Lb@3ujOKIA zk&^Z!3w(E|^8CovVIw-Oy)dySU84s2w;S9FP4ZtT0 zCE~iq?L>&bg2NM}_5n3cQgCdcWM#J6|77aRvYuukwA-my^^6@q8EVvEP6N>_nmP~z z?Pnf|UITJ0(y%n5Mkv{`i9* z-KY!}qA2u|Wt!ccM=JznM1yGm!^bM0yjf$wC%6PD3D$7FD*{5Sjr)(~p@GuJN?3#>ntf-%oRDVI=Qx<|dl?#uZlqkbkHjlM z^&)HT-^gT&KAbM!(QVma8ZWAoEpdD|Xss1iS`)M`s^^5Vg&vqw^#aNp*_w4oZOy+) zTaCS&x>u8UCl}K(cA0jsR+wV`FQCPn?zmZ}=g)Dm{H4;{2lJLq9@sYnBhlU4NSNy5 z6^iTY60zw(O|x}uZrD7c73gB5Qq8#@N0GWV>2FcTe{$WBlEl3KPuFG-IY>XJ`&52e z{`>}>2%3C(T07H?;?JVSgV#USK0#3T4KBWA?Z`zT!SRz*RBJF7F1#}gEB6d}eTb#E z(?h1jnRglN4@B`&VV=s5+(M&SqHXzp7wB%-Ax3p9D6^xc9;p^^`HFHPK!o}n_C)?3 zAeNK#-K=bmMicGDs5`T2n?z9~%*;rAh1qbgH5a9|7)3par{t$DIlJ{atn97C|9DLH z`yt@ zFfY4>Hq)n^mvsCOFA3<}wLnfCuKLle+T3OD#m z9iA9bb7K_z$6cgY8K<4`QYI;^fd%kpC-*zsflL0jXq8UGc zp{)_w+)#)0`NmD--Z#7|WSm_QvXe{TV_8bzlJyGkt26=pr!gu%5N7{aiBDpbjX`#2QqVZ2QSDFCaOcH_v90MQ|T zE!(0Co~iytXojLBz|j}a=CFj8M7eqKaXM+ewrhQD@PFD>1g(e)RU)M4W&zkD~uyvVQ zQ67~$7Q9o zMIz_COu?Ltv^3E{2b&b?SIZ=S6I3F55yW+WkszTOVAQ6-nj9%wGSOm>z0+jFDg88C zIM-0*oN_!@Msj2p3{UNav(jgZn&W9v?iTwkAaibXJ5{w;J9J&067%bt7* zh^>W2rZ+sda8c{uB?JGGqCSuD&+b({jUay!K4-eb_Q^DQIZb1uSI*okC;iasBExIq z`?0pM_c#)XbM+^BAkqdzUvIDE8fP#?=#Jsi@pL}c2k^S=tTS_MZDaH}`sZ(_*H#URr|xx!nVm{+bnY&fCm`6v#jUWI)OX zDSKnug_vGo+_-1Et$JF7b%N?o)#kg;YE;bV^fn|%Nz1)AWze45^8>!D)!%pL@`s}F z4Y?kJy?|V>Pvx*1-ojvx#XCpMCgyAWTgxz2||?S$&+I*gOO&t zPH8GG{B7hp9FXb#_^%Nl)gdO6Msco)Cj?J9w-c3ek6%4od< z^r~P8mEx#2FeSvq?XL(bIrqjixytk@nerX(iI#LhduvShp3=8#@-*dr<4WKZdG#N& zV~7wzX&gxO&?d;gDHIclxP+=88JL|#4B^e8b9=c6`8*_;`3Ui0r%1zl+2a6asXz-um%Bl+l~zZ?=}{Jf@bkj>z6#%;+`9jJzSur~Dt^ z3Q#5?(c};XM!*gHkBO*;og*uiNB7iWu4@b3{`a!jKuT1gcv*xJWFq)Kf0Z{kn1%6i zHKl@0`xNzYCSv>@ormwpNmi7xBwa~?8oh=OGx)b3;5#-oIP-$&<24v z9{5A_OFIfnAbG@5C(lWg|6PsrNVfO??RJ0LcXyD@nHOuXl_)W;Oon?b%aq?)fyCeR z{;KV$fR8El(3qP0=|nkzzr`abBP2>L-p-71i$2IXCd4lR0sD_g;zKQCTw?Lc_lsLP zyziivxA#Atwf562+vE9UKhP5y+g=H(9w}{!0)0=*O3W?N!&SV#YNt6E984hLZSPAB zG_$r6EuM?BUrfW3(KD{8tKTmDRCb4)baZ~ka5 zn;+$K4DH12Slunre(4DS3%6BSfnH--?SvF#{sZ+f|zKPJ0KI`f{}+g#N`? zzGN2(2=+Radq6A#CNMRsQcp~@vgaF>6)`FQFV-|e-?9xxVW`*E_7~|qhf=EZy{OiY z=1SGb&$umu_g9AJquzvRdxEiERA@4H`H{<2db>#}UrnCDj0-jOEE$Y!lx|UFo~JV? z+}XqSRkq7jsl+L5UsKbF=bi-|+s{{Oa`c5I|1Qu??$0k%45(9K}Dbtc?F-2HHw!=6wB zS6!|4f_X;W2axqoS!Lox?mjUYvTL6|1Y9(&Okwyx!|1oxNifrqa9Stmm)~v)IPg?$ zT)t#%}9>MZ*6|K8a`Leq{vhONtw0DnxY63275A8Q_yQ?^xq0ois)#C&N>ylWvKRGu zT#;>9y5dHcpd6H+@oagDBYt#S!h2N=*G!Et`XYUqTRDH z_M~!z{TQ9A-hol1)pUv(+m;!0#ai5{Tg`( zsRi`KabiL1haY~N_B%l3W&s}7(UCN0otCBhS?unBC4jb8`mD+UX_-8YF&)@X26sV3 z7Ot3P>x?spAaX6*_7K~GH2S6q?^6IO|=Z7yPg?Jh`AnL|Oeq#Q?v|@RzbH z2xAwoX`uC=vk2Y-*}z3uh123Wy@;2F!_K~Q5)nG(9R^D;{)FZ`;5Q)>N^9>y7X5*tL85)mb4wapU{?=)}*HlN{3{@Yo|{>yF=Uy%lvulumuAzXOWA z{oQbyA~y5VJ8f!Z$l`{iVwWnv;y4@!uk|f{i~Bh0=wSJQ&U4Xi{XzH`g|{aqw&kL& z6X`Vo%IS}-#UGD-ie+`o?P?o-j?Qth4=@_RUJ*$F!=zJ_3`Icc--pCQD^J9Zh`d}$ zEAEBIPN}31wSH=G2Kqf2$z~okSw$`fAmd>IeecjVoZjg-cH=yFCgTB)BC&d4sqN)i z2ceDu1xb`qS8=96&hM`49+fU-J%5ZYwom!Xe$=!ZQl>0F4`j+=>}uc4Q7u8sS&I*96V!7*+{=7Lf96_G62AepHat#<$F@VY6& zH_oxm%OV7JS+(q}JUw0rjINxmJN68q4y^RRpL0dV;oK(qRvbezr?*sJHR1M$U{w2& zB?d){hQaOSBJ}tWx=_|?tG`h9c6X?4W!kRysZcGDN58tP8_}ppS`6$q$h84Nc;q2n z`z6z8JE{K7jNvW^|LbHH-@rMbTODr;Z*Rzx-o1Y=m9nPf3VIze&Cop(Ft@WZ=~B_t zoY(I4AP-mAnWXUFPD|#&ka`Jo2;!2(h4h`+6IwTRs3{zcI140V{^* zJwZ4ue|hM8elAz(AQ0<+t>Y*zIo6MhJj&t0vj-U2eavSYx7`KIY;CsYngvY;l^8F; z0yiPgr}+59$AS$4yWHdDqOZ)@?Mw?jst8|(Kc3|E^Ci~})r%Fl8;n;xAbV16-o|pH z{koDnM>c}qveh2wvNBH`jR&6wUUi8}l@uv8>HUs+w$lq`Qmk`b$;$`X&iRAzoj2U0 zP1qD@xAA;)NV4eH+Z)uJ>P&58U>^hiR`l91TgLd}jT<9RS<;#GC~+!cHjLU}yybhh z4g{lf|I^x|-hf^}ACj?+>fvcOJ6-SS5Hof zoKOvY5OCA6xH2`bj_kewJ+CU}DP;xQ%ZvZj8Vfbk^#HcLU358!l^2 zK3c8<4s=8i3`CV))8M$Pt3v+zZ_%UFZ@OdE$>ko3v_CbZjmER3;y>Br`k$44V3pw# z3t2S&x|jtbnx#fuH+ccOL9A=-Xwq?(e^;EMC+BJ9r*1z_C{Cn)J7}Ioq0=`G&k`CG zbOOUDJ=;cfSLIe|<=4XnH%{4U+(2q9N(mRNGI|8TSsPFs_QP&3-ay>s!$(qnV}uQh zV)3`fTd}zcD;APE`>?RF;$nvGEY1-)g7@}+$6e+UNJ}EC4qu)b9s|Z|@J{2;+NXry zFZxWs^ejRUMM%;o+!IP6C!R?9p9q1G@~G~+#e9adP7vLpUF$)LzpSqGDwQhsr1p95 zj)eE%R>Lv(P?nv3>2aRfTqF!TOl{jodffhYZjbyUf34wZfCU$(u8 zb&ukDGxNGz$nXc`Bg7zB=*U(PtYb4InZq-55g}YGx@FMeu@gS+$LiiO@P6pg!YHCEODuP;=Ra(9!T+y_i?0lfCaNqZcb{aCF7q5|VT8;R=ayKUtH!cbrJg5^-8NEL*5yc?Y(c z;k%2H^sZQ+0=g_$rb*e2d`P2v*aHHyt)cy(1TAoD8Q&?YXafEy>ikxQL#XNS{n*lm)$!Fso!v;vyT@tb{l z3RtZyfWR1;(>*A+-3AP)fsA~2p%*wBsI`s!_c?&Iv(54GeANZzn};^1{g%cdVy&XZ z*4?FE?Qadj^6dwBdMDVGyd#|s%$UxK007|5cTKQq{jZOIbli3bS<~uk(>h+f0xs7PQhVtdYOFI z>P6yiM*E}L%--a7-EmKRp~A%_QNWzfiU5*58e|~D;dzb8RBPaPm*9Jy>wZmGb`o3-Pie?o=zZ-$cUbR+G16Eb9sHZz-mD#aWwbh@|LdWt zxZcrGA9{V=?TQTOujTsMU1?SN(IgpgyFd1f%{9}To4aAm?YD;7Mrd`%{l)COZ+9wm zLvAASW)Oug!(E%?6wpmtO?4HThI@%z`>nrOR)V4N)3wtG zoly$k>y5g!#pX_p2}Q z5`sgRB(M>iXP37G(s7lCK4G#PWODn#J5XepJ(6#*jCed=ts@i(!|EmJ?TlYm7dryw zINT?_>Y>)E#r2r!donRhK(tyWWfKIO>>ySmHiZ zsJPhKZoE~`2hEkhD?de}bL8{5&99Fpk|@`$qi05X7hrC@utO92=t=Uezp?T13w4;C zBPtk9_`T0aI22)gt7Y!ccoA!6%*)+o2*&_)s)nG5P!jU6qSTuHVN7b%Z{)b2SHb|k z`9Stvks~9Y$8&aKV-9zg;mS6Gh2=fGtFm-~N~0D$6w%DOv#Cp_pCLmzCId5EUCuZR zG5#=Q4#PX)C->@l7e?rJJ#DT>>9Q=!pQ0JB>#W27!n_5x!Sngrt$%Y7^1L=4wGmh~ zaOOBPsme9WZ0YrZbANU_*}x)ycoEw-yF0*4!OwTJB$$7RC8)?fKK(7?wlb0*k zjhi?FAOZ&YpwHb8?Uu6Mw;^L++zUo}2dNl>$SFp9g_%p<$Mtf)L+q++Rov(&&!j5S zJHhaRAG}I$X(-?piD3w}GfRr+YB{j1DYhRlvsKCA6l7-gJmP=Ro@$o2ht`BKGX zl4La7$eL+(P?d3afa(XY)IYyzJ^<2&fiT5Qffa%;=Po?1{yd!;y{Sgwt(K?IHqRHGgE6TmQbUeFQ#M^RG!^V=aEpHH`xN4xxZyJeV0Ku zYU_|A74W!3e$%4}{j(tTK8{FC6aoIGp6tmxL%D- zYdByv=5zeem(cn>>$q7jc{)!f4dm6aj(K%b>#KRwhwb_ZC*J=OVy-fz9DIPJ>>u1| z4Zo=JG2cdatKd1G&Y)xBhP6<#;Syvj)C^?NZ^Xp85PFVkN0wG-Mwx-AS5W?6jJNU4-`H_|ZDA}yUG-3`)=DBUS7UDBO0(hUOA-Q77bGw&V0xa)p*-QQZj ze__pe&%4h)=RJGxXFm@Z5tI7@qb4|wE7~gCBSzD9?mb0Ypx?)UUSkTWmfnoWg8GWy z!DcK>Qq(H<80GvR39IQ+TD$wt$1Sv5tMbnSut_;@3;xQ_d?>B9n(c28jO|yvbyl~= zX(ctty7RA1roFCr1gZFx3mH$UU(KN*(!e{fKe>d0Wko-c(LhB|ny7f-kKTTjqKuB> zBWG@*G_&*wJ|p1%U3Qx|OzRXcHa&hG~N$bK8z|D-g%KA}~8ZMH6ZJ4In?GWruxX24T6#zuv!m z`IKYks4qoy>BLgILNDB`xow2kZ+qP2Japg7J&=o*L%%WZJhWdR{Nm0^ zmCyDsbE$Ej$}`p08|`Ht%|2H2p7ELlp z!JcMl^+Zbg{3yn#i`g+;Dxt^WZbfFZF|L^+Gthc!`Pck`CJr%LpJhi8VI0Ql`a*(;py|Ewt zQZ@^<#4Ko~Y351yAwFfkFW6PRE;P1f`Enx8Q!XFLWgozs#CJL~sn5+^e2~5zM{!!R zAm~-F^b3ynNGDW&)I0A_= z>1cgMo+Z?%mNHq7yQ;p+naNb=HnW+fclxPYgKY|(n%zJWe>`M#Xnhwqksx{d4P^hi z`ikdcqtp{FUgZPgc(GY2^|F>v(WAqekq)oZ9Kk`mLe9R&g<+c6G=CYVgzm8}C7&5N zQw+a6tfEc8@_TL+hLn8GElUAyr}~<6_HvMB&T`?4WNOGsT3l1vnbgx2?g3MZ-@}uv zO6JylDz$PxBgwfvUBVt6e{WA`V_li+OXXv7&2TAs`HvoT(?4a)+Ibn*e82%ZQ$&A} zkWNGSG@;2wZb^&Sd*k|r7oYRtYih>QM`*8U#Y4g|5ox`~x@+xEfz1ftdGWWaEPWkVl1iTC{DV}Y|;#JaqtU*m4Q1WM7 zgeL7zy{D(98W(o{ME!>Kms#AVbVy3%nO-!dtRJC%n@{w7Ee5#ija{ZSs=YzZ7?)hq zIhKa9F#2Mo%7AxIxO1Bm{~;)(7#{uAq1G}pXH6Z(Mo*!_I$*hqG<4Rjc9!@k0`t6L zf$+IVfQrT`nc#GsOn(ji-XQ9nMdPl>dZKzvyLo~LUi0f3@7-;OwEax{HsPlTEtL6U4F__-anDf)^C1$@Ws>+mk55&dRg?9C434xkNpdCg=vIx~CX z3{N4=C3%FT?_`iU5X`O}b-RI(0bP2CSf26&5ABG|_rLDf3PYg%OE$P`6b9*HVPGAW zs0Pd;k)UnDKfm zE)m{I%-lQ)J*q_zNHV$14jTwfzk_9n?^3cOPX(9O3oi$h=Qx@$m}E)}Jdk>U0z2`x z^n|edsIy?>eT||L;4VJ0AN_KHnwRSWlm5$y%${BJ-sH-L?Fg7&!$3V3>bxS>9ncec z#iJ;vM^lF{KkR=J<~AWnz_SN<`Za!r)74}4Ox+ijL)NRbuNfO$|u!lv`mIF%?C z$Ru|~8j#H`lFIzu|{Ux zwbIdH@U`XVZb6pH=|jGjMC={URMS9S!z-jz!|I1jBgKw)T-Rj`_xY>+0k5fiC#PWQ zc>BQI3c(+|Sr>6>5fl3an(2A_$QMub19J%h`+5de=-ZrZ)xv`~nZ~QLbCTMci;Q>m z)^HBuc1pJ9rME>s-Wc{mM_rs0{$8RuPGfIgu?Kp)?ve3!8nwFDNRJNUn0X#m?nh?{ z$ZTO(3+m!AurJ!Pd`BnO(9>f9Tc--{3-;h_|3+&!U6c^pc?Al;V0%F{Y-YAC7+7*1 zd`N8?OTbWD7^*6_deNyco3;k>92!LDc>V2)Kx)3KPXN3qvvl4vZQnPRUJq~i+C6Bp zJ1WrK@urqJIqnm?oc$Nc3GW>g({6v|_*gZyS*%V8-0jPnw8wC8P(W%kHx}*5z8fY1 z)2-&76#*sf9n)ogZ$kZddq#-G0wB~f(<`#L(m&GG8~LkMuiZJ`=u_BMZ1=ALtbiF) zG8V85g6<-vTc3j#ZTyFLe^e-sVOcgqa!x{)A$KJ8N#4;?Y27R!Hq=X3cag*bY~Hg` zGzUHRx%+#0?>1XRk&)+$!RHm&O_lmv4N-iF`ILAPs`#B_7p$FhvhR-q4Z6eph)Ib=O;C1LiwBY!XpmcO0Dh2a-vaD49v( zj2s*#Yx`^(!&dupc4a1tI;bV_YThcFk4&lkcg%SV)SVgl=6n!D&}%H=T(CpE?ELj7 z9x1~+sUfL*v8;(wH?lN9AhctL0%rchbQg5UYN15T|F=G3?Ta%7bNmk~2?lo5M)-qE zSXTW?bg1z>m31vRbrBmy%ebBd?Ovh}>a~hkj^tAA;@rM*t)VY;So#Jb^)|?yml&h4h774kBIki-7>dQd|`m#Y{upsaI(&CZ)SP9mT z{_WX(^FZxQZ1LM|HUORgU;745SZtZRNF4+h0TCwf?Ox%gRqx_7kf*V}Oa%zbHoz}! z94PG4Lp0=nw||(r8e2rceIKT6YB*QVnV_9g@1Gx*BO7>tzt<_h*AH@~;E+?_RW=+4 z0*Sj%?O8Ko7OxVjD=p=UG z-JO4yOyQ!y>pO9vhuw49+QRh+s)}jTj*aV zM{V*oV(A%x4?z}-Uz0;z3fH-*=wUz+pgKVw7SS;>4m%hGk0VETH*c6ntlVKJpUN7=6 zelv+rww1KB}DwK5~fr7iE6V;^zWV`E|(bYal2r z$r-Q`KZ27v{6zmzV#o;WBcVRiA6ksOe}4)1uA%qf8?J6_aSHnM>_J0c%SVNCaGmxT2u#8Cw-E%yWZMmoA}-m;Q%-B1G3EF)PEQeCQ7B3vY5L zT!^T2&^)qti$xE3L;tl=Yza$KQ`vU;OB^+|6gw##{jQQUnSCqq&Fd$NOZC|L5nrB{ z*cf8f=*r4v_fFCN;0A_EkoHC8NNwHW5=1C2l_?df52wVXL7q$x7~Vp??%O}0lr_e_ zQjMY^yqLzRi>9FdY$c?k0aEreAFR%wCtYLV5!`ln#yCA1RP2UCWCJh^KF1}YskFY8 zLSuH8@s(G(kFw$saPij{II*Jy65)SzEr0IsKW%=F zs9x&1LQ82mVm@>gey*{l61k9Rb%0kduApTRT!;(;2{Qf9wjZa0rN+ z>}50sQHuKXG}zk2g0_D`K3Kp(7RX^Swy0zkW<^V++t78&1Vb6r!E+m0UK-QsZA%058(O^vWaM%=VftmdaoyI zjEwXo8D4z&X_3r-<^`}!G__d6P=(DxDIm4JV6(}p4KvANgAp>6k6Z}&Cn>-&epp}QKBCx3UVcYVVlhKaQoFemK&7ZvB4VizIk!2A;_Jz>~CN8$1SMV zx0q9v^rX{+??ZQxA$UqS-@Q5`q!A<{@i*g!%M=Fkxhk1&HmPg5UTz9W!)dKJZoxu# z9=ZVcTsNVo{Q~Z87^R2G5wR5FmNzHMY@&jC;?vz}NSBHqqmXYMZ0N9E_s-+mp(`#g zb%RZU41s`21oiieyg-8L@`o>e6t=sZrD+sIMO1>jmnlpO)&)}%{g1qiR3sWANrAyX zvsbVDhd*wAyS9LjZ;dN<*&IW%L_E95KueQJ*O0QsyNrEk?H^#)*fY!fV!yb+sP0^T zZml0-53hk&LyJ-0kRWWS5^bXRGdi|0ZWLV=7l6RE#1&Lbp%dub}?A)^@q@4PK@&84vek_3iK4c`5ED^b1r@tr2I9Qb2mj{-*K zm$@VXJ{wJVSjKO4Rhh%bdxy<*B~;+ckAnFqjqya#O~}+Tu+U&*6+<-b!rw zs^aOuJ!A&3g96#2eyK3zEtLQ&>~F|75J1kIe80`OhPIwU)_7tM_PX2lzNV_@jjd#Zhi?FCWrfYM|hzZ^`dh953$L#Cds=x`n3b<O8kaG!segQ z=gqv>`yY!=hmBPaPxV0q3gUJEZv~KAV^%%)xo0GUZ~S`u&-xQ=*@*W_gg$Qg8~^G% zwVX9^hWoC>->G5;Tj>lWSe93vMZaV2dm)DCZ*Y3A8;}cE4SE3jFV4sF$lYS=4}vFk zq%$0$5xER_dfU>3m67${=d*p!U}336VP7*%D=(MiSVuQ>tbD3on;0x?a*H9-k1+6(^**%N^Z1hnTk257?1IZDsM-k+Y ze1%$As&#DiS3~#dOURI36!oydYep*4MNL?nb>V@PlkF1UkN~=Lkv?8y7ZTxrV1|SaqLL zww%SMpc}o19dms0dIEs6J*vbH$Z;FT&P&^rylR7yi%YpET=?s0@@W6eS@6;AQ0WhYj* z9-2Z#bmpf8QbU6$x#OE}h|C=bIc^s27`2V%6TjH7Y?Q#p!O%AFaXwu><{&_15Lkf1^7^Kg#Eona353;V8Y}@ z$a5xH0@*_6=$FY{kTjCG^^llkXEHV3O;wwV)tGa~C3h9XCbW$*cV!0J(FAdsfgJp0 z*+Io87x&y$Flp-1KBzssDc`cR12|f3n$^_Sw%mjWAl2f2b=0{fu5pnf`Yku?RkM|V zIHCq_Kr6(DOso$=d1zh}9nuI3a>}2nf=1Z8guQ(I!KH_!=e>n%=UA|}m_oV|g!f8X zpnyh3j3e8ZWdc*xt_!Fw1N9DJN0N~|F+7aW`EdVS37126tdNw z<~Az(-$omV(1%Mu8Cw7)LL%e|2Y}{hL?b9D6CYbSZRY&luCFIcG}O8?ffDli-5IlQ z8a({tz}EOrIz!4`i>~0z#`D`z!4+=t)>|>NtHKVeYUC?_=Nzgt$o$2&oo z@eChgvtC);8e((x!T_??_~Y(bZ^|&h-)_u{6t&P7m&FNo>ymow zez~ZGHa1qdp-yhs3J6LjrclSW%@UD)&pDs+>SN`x)1h|9Jp?5KIx@pyfE%l%-lymR zAKV(ZWuDexcnkbq3e>3hcVy|X-|rTqAaJkvER(c=J*9=J+VgDo!9EDn%Xl*ene772 ztJ5O93}uXR?Rwh@-YKZ{sObcVR(JmX+aXv1F)T1ND_J-X9Q}SV6u`JW&Lb+s0;}tP z&#+MG*wg)m^AV-$xq|Co!HJ-cKwL(lU4|ahjR?Adb{4%q4dJ9u1cMxSOih%^)-LNg zFkFD771u))5+rLp2$ooX5^(BtD43+KLpoh2a?=$){f5t0d_&+tqZx3#T4+ahiM)mU z9?9fkBM1I;6^bCg-``2+_iy?4K@MJt<(Y~L)}HV3$3j)%>(4Gb36?Du(V_EqPQjBO z%mt8L8Y92LabZkd7P`awP_ty5M8a| zJxIp!poha^STOx%#uNKW`qzgviKEH_ufEM1^=S6uDIqM5*!z6a{Z@->F4#@gZ^B~j z3WGF;;qOvv4S`u1*O2|Gu4tbs;6oDtcG)J`=}j?KQ#Mpnwc4bfM;$>y&VSR9W~=M$ zUmz#y+M-U3kWVs<5QX)Uw=n6(&(9O#s3uv{g(fGfR`>m*J>eNEIEVvIMym(h*2?E_ zUV~SMI-bGC?+EbCFPKboKprd*-UIo2!9*zE8_2?+1JXFaIf@vE9H}rqzZPpMt}mhX z$z^~>!DsX_OmA4=^`nBsJSUvjt}G_^^=*0M&n}@+2jJC3sM6uMSpmj90XGy_KK=S) z@E%eGn47$r^hTvpbim#X^I!7|h|k`8lnIdO*#4YGMdOKpeiHCBPLxeQ4Q_$p%KhBF z>wb{EKUI|BxZGyeiTAmD$xURRZFFvQ?;E+c!r3<46ttlLLdL0YebRi4&$XR~4EHtq zQin{rhF;8qBv+=MZ-dwyd9;(b4Xcu}4%9#{KfTzCsEAzO|Mc$JR^_G|-OrVKU?+TI z@_}nX?#RXTH-yD)KTbapJN?D;6i}}s*pXV6u&_iw+(#4QP*sjxlSylU#GZ(c&?T(QSVul(1g`ykj~BV-(Y zK`Oyb#uxQ4%lliUD9F*_D}aMcB??GF+9PB=0;5P&!swOKbiZAbWy|9-qq+2|y)=FM zTtD1HHReodJR{m{=gu zuix^g82OZXUq%a3(nZgA__xj1Cbuo0mHfdtd-F}}WLjL|{zuSgmhl4?4+@YZw_5kr zUcIBtpcX!xBIa;bjtIsW!e8A>gZpa#&4a6^@lI2Sc*O~0FX~hnTHIgl=dUOMUPV?H zyL#%87xKAv304qUEa^`bL#yl3p`#P|@*L0N3_^PwQKFVt^%J*l+sy%ARCN%1_l^78 zf-jmTH=tQuN#pR{kzoiu37RJCc~ckv2b}0$b2DZ}=|^r-LpegJ>ihENeqZDM%F{&u zPETo-sCxY;B`Pmdd=Cwr@Z5E0A2^f-c97A#l+-H0NJj$Iz+S%5d`}>qRRBDtwq&W)H{l}XPVSg(9}Rr^{G;)8 z6A??l3|IjbnpT;?pgRX_%EeqjrYBO!zy~#Qk{04}1t?6< zt?Z~pJ5vj?=tdcVBk*4q7@1q>j%Dqn?Pl|d#dj7zS@K17ez|4WeKS3V^bP}I z{^(C_bg&{F=JX>sQ(MD_qiBDLhOzPCueKV=xwC9{ra$p znGmyJ|LOL#uE5T)#o6$Y z$aU&e=vzqKwu}q62oxF2I$D5%kCYI*GDKu9(&wGz<8d9OK$Puze*+&gAwS*!_QA_s z5-lxxB=XKJ$RBfy{C$eJ9|OvflIh*U%^}m3 zNV}!x8oj#D6?)Zimg+0dTO=S$kDG=00^nYkqH~NV6Zx2M4{6X=`Y13EsRU@?YqHYT zT$*SjQO31H?h3|cJSD*k>|wH6NfREX5O#fuR0Uz*~wp3gbwEgFhB0s=Xa2p%@0ECt7c*2zt&U2Xjaz{{d$N# zNrt+0R+?WxcrUq>v`lYrStpCN&Bl==_&>C_@;(Y%OGD(d!} zDi66Rf$jB6*zSyFk?F<{W~&79B4RtMi2I?A%dP1!w1Hx~qJg7Ze&j;n+{K@?u%#ubGU<*-Wq_$fG{;j_rhCJ)+rtZiW%73U-*~ zDP$;?@&6LZta9(U;vH(gx>ea#vbBS83W2^tb$k(%Hk+SQVCKt^j^HpK+@s*;RRq_- zf=)-@$^P~qpQ!x_9HIoD6`<}uB@_wV&q5vLV#T@A{Y1q_NQi6n!I_(?jF4>_$lp@p zx(IASfihDM);Er*j1)d^|LBj$)zubheY!zjDx}r+5}cvT8aO*o?p(McSQPJ4VC0aAk}azuZW zGQHw$=E7!r4!(C37HPgAe6!2kTJo{D>TT4LCU6_+ppg7Xu4f@?|1y=xg7s66NJBdl zXNMu%&l{1(MzPoLf=NEfQKocEzNy?Q^!{M_tY)~rR{^SXx?;K1e5pRBb}IMWXQ5$3 zprk&?M%IxKv{q?80BfD! zMiW)xo%}YFf14Nr&cENfIN-@(%>z(d|_WB+Ge$0tBKr0WiLr<6}~cPPYL3Y z3-EicB4&$pxSG28rl6W=e&O4ydJWLO!?UCJM*{{+n@+QpMx!}s7{`h)(`9{EQ0fm1 z$wJex(9t=UV&frbneS1-7BtXR{NRPafK7s7{az6s@oup*FaB31G|5Lq#$@c6^hRCj zld$Qfpce)}O zBbH`E@9i+huc(D`yCu{10ZyaVRNvFKah=$AM020lYhL)*-f*qnSdph}!0D!-dnJ|Y z8?dC~GHnH|w?Opyq3=VY{KM_(?R}N}IMq%l=uYzQx<(a-MG=7d*KE`M>h*e?t_4u< zxV+$ZCQW@2khZ7Gu+jZuMF|VWP3|AF0Gd6a zE2EWt{;L#pM2%TrDoSxXwpZ5M?T&Yn2=ePt8Bv%eZI+1B75Q9q<)5>F@SFIJoIAZ{ zOWBkncSe=#iawL6`In%stObhlBGJoY2fMj-DfU3|#Fi82Dz4@Jr=7ip`Y->7+uVJ$ zxu4%X5PJK6q8hsjxsjIlOcTy$bB&RUwF|K9#N%=VAFLRcnGAW82(cTZ`iu{M{+x+U zvgjqnpdRnnG}pyBzn@^jVl^@G1f2e!DzC{ zU5w@YXOj69A5!BD_TZ{K=w5xQ)J#5}jCG#eseMAJ{g0zxbzr>l_b*3?2bq5AVd$Ab z7~mSkU%pZ^oUs0VpZDn50JU$7u9;LLkNNpaQ&CWW|9ABd&@Wa~KiK`jG~l4ge8|ka zbm7&K%U#;IA1U`?PCNSL<1JN*f6i0x=a0(cS3$mo;Ei|!2v9qPXJFub%+E7DZ!x_G zd4%{Okc1VF^GfW)d%*t|)L|y#c~br@dw;pkQvhkP2_>=e=gT*r9fZrjQr3}uUj1}d z*#Be0Wl~fCVgsl)>7Lv2a@~nwtK`mxFREHzLe}MHfFX6jZ;OsPe#T;Xj8bAm>|%g! zCVL-r7Kov8m4PaD*ua@)V?oFjk6U?r9Fy}kC*+P(Dzh(e;YP<*-bEI_@%k5}+1%NW z>rAgzbn}>0YLH|!ShsRRw@Pt4u~kW`2%B&j;#)%%UF$7FGy=Egh#%g3ydU~;e&ahW zVsYg`VqIhL#$mLgX+@t7nIetQgC!Vj&v-BFhh_W2x5Yx0_$N6i-FPuP|+^u(8o+5 z4zpMvvZFq-NzJM|gMb=ke6~lQs*oqR+@uEPP!TsP$C`x^WJWv@bn$y^fRb;3^dTu1 zPCzbTcyOZwL8!ab-Bf{($K{$P9U+Q^h^k*Z`cv%|ihR~*@7%htXCsd+H@Oz$IQGo% z=x4I&UEpSQbu=o#+c@rcGB*BN$OqbRE7|uSn|oZS1ehM!6!<5LJzrv_(7hc~eEmHg zha?uSl*M0ha;Nw+>}ZRT)rfEk3Xd2r%<)`04{$j&KolYlTpngq?pF~yo*5FeHD zQ8l=IsOvPaln&|7YLfp(@SBzjauA7R*@emD-^SO1k~iQ?i68-r>EP+GLLmT!cp*dS ztabzNz|!*adaau2_%kH{Ri4`}g0x*Qvn0)mQ@3IA8n>Qiry*r8lbewh;cz3@L53o7 za}@bjM-hz*rNuE=^wjR#MueC| z7kt*r)(TP>pU8|pIqo$3T&^#;#5`HA#}^%6eM2dl+UT&bP|aruVTt21dZa=I@ZAUQ z@@SI5!5w}r)fwJ(myXM|y+`wHf{bk3`C3R!liUhQbB{^Is=1gqFDEbak=Y}8d6F1Y zZJyM0-RQhilu(o^2v3%Xhr9C0<5l57inyoi*OGwwMYf~&Vb$mj=#lgqN~ta`ip6on z7uD;~$R%_^X5^c`wnD^EiSoZy#fdb$+qfH-VYyC0v z7NJjvTui^~2d|<%vhSQ5H+oT_-^829s2q{6#9(~j4!3%I4OX3Fa9k=N^uJ4?0yaPB z>T(-8Q?S9#<#DoNlA2NLdf~*N+=JFt?=vdmwlzS*Ch&YOUp|(`KH3-Xkjhsfxqy)3 z4rlwGIP<|OnjRH8L?!&THb#EkXf`Yjjqg4J~D%KziUvIt;5*=#cOqrJC|b(Y0=H;95=)9FgJ zbz`jRy6U%7%rxQm-QbCETDl5+!@j5hu(cXvIubOq=PmLrjsxm&r?bMJ-sl}5B#r%t zD=L-RN9;b~?FUA~FHhQ*c#5jWYb}OHHe@45P*6YKA)zfw?+Nem{=wi>mt&`5Y0p%#~ zxf3gZ=YZVz)TZ*Z6h%w-=ZUg@PU61u`@TP4*Vf+baWXd;MJYN{oR9WS5d&I+zTRGp z4?DF94!Mv-#fH1rpz(~&N5oc(MRA}ZHTqShR+QFy9XgJc`8g`EySTW$*&5tIl`4rP z_&g5&K1Mtp>HiR2Z2TVi$|i&4E*_qZVOBd@Z7qgM4Z9tsE+n^6apgEC??}0ky9ZhM(IWLMR*6`&Hv0LA8*+c8--3=-Lsf2MsF73f?RL$!-i*bn|$>?>*Pv1 ztb&eN6~7y3^gFEb;Mn>ujj&K*jd=9Yo&^2p0s15H%Mref)fTtVGztIKX~L2_RDG8b z0hu1svb-#c4nW#mYps1hroQvg`x!$M^0@FG6y15c(6Cr;{EO8bg@>TsAN1IN5*K^6PzbMzMdOcdZ-{j3Zi4<@7B>qs30}_Ub0BW z!(;m>5>Fv0(usHFM6QFq43^Blz2Ic^N$|2GzI|^mYK{9{e9&yY? z(EXy;oZU{L5lT$i@QHy1XRG4xZcji&3sM`Q=gl~x_(E1&ts%wOM}c&dv9qxJs9Szz zf2!QE7>;TPqt<F{P5uQQqMfBR4YBl&9Hs;}Qmj2$#2N zPLm$jDerjMPK9CjcG|BHZ}cy7{95u|1sQ*%_%r!41f9O4@ccJNZ&CJ13O`5s`))@r z#88TE%`9>lG)A;veT%9KZ(X+|hbn2j*O!VBefRX`1PZ}#BJ48YCLvyfQzw}(mk@(Q z^#p%_zG080ozze~hPw7*t7hvR%%;CN?4uBIAF$VT@}!JMspjkvk@RyvTf|*CO1OHs zJ?1+UEKjLQV+9O;SAD)ipD*|3i-+J-A=n<$LhjP}9)E70BfAT(MJdo1X|bJQ5tQjx z+rCyvQI(`md99-S03GGe2ZsSin5ahhKk=w)`CC*L<`LkwM{a#e7s~T&mA1CcV^O=_ zMzGGUUq$0RvUF2epuV%9$w<@=5;Bfo$E#_a^n2qWDY=s4{BPKKNzn@DdXRvv9Q288E_tM@&Li}vKLA|^0#qaY_Som1o#7z_F1E4*^ zb$hgOyr0Ou%b4jog!xEqDFQaIf|z?1E9Dxs<0w3IlE9!?D8>!~d0k?l?;ehF%deLb z#*F$cND~P5SLUwx`wYJvOQf|jIdc3{-G$r9S+JEj2;mt7;NF)sn|64f2CgYWyJm+I zGrl(`@hxm*Pnn692>XmglV+v&2oMj~-J)K>BcKN~Z8QzNo$&0&(JR%uTfMQc&@y^-F^4OXy?sBlEX}?8`A)ibPC&H`1brJZZu|&qC1q)iy35s z91O0;_U5v|BcJn27qGEyz1^e2(br1GxH(^h>{e!3KHN|yG!ZEe9J9uQI9n?^A4tfy zut!0{F%8x$UG^)jde$z;WkNgpCG!ci>(~6&)dN6JBxiz0lJUzbp;Z);;&OGD?cMPq z(B9kFzr4dYi=>ata#%XpkTQI=@s;756pj3<@$#5JWUMh~n(aioQH4HK!b6Jr+Kk3wYk;s>((-ZCV*x>;zSxw` zyM#!thAn(K3f@s9+Bu_9&i5q;*ztUl~R>6P5>t3NI5+EelPQw)8=)4V!m^m5j}%;x%I zp4A2_g(TU1&{;m%Q#7Q?+bLk7{JK$EHu1j)&Y#25Tr!Wlw8?3WI5Hy7oj54U%My%; ze#iOKwd_HXJ?vv;i!T4a9tT(02X@*KLARZ;&dTB5I8~+Anr~%!AfLbE!Ncf3t(O6G0*!*Z?L$wnlXfZsFQFTs%lFYnR?9B_^D+aknf#k?)@@vyEE z*S%}7oSV~SmgDNa*$2s$9ZtMJi)TV7`Ab8Iz6jk*8@k&U^`!Q5EvweFZC znJ#2@hE zIg}qAc~zHV7c)=*Jnx(8{6m&WWau`kRiwK5!EL=KTsO>)#;3@EZ9wqzzDs{eL_X7D z;Lm-W4~98Hi89Tj?Em32mBnRW{tfey z8Mv+zc--wh8cS!MZ_e7sTtf`c3V z(pGajU3^cLTJCZ>7Ds-FlTp9MPp44%;cR00e5ZVCxd(?SJy$v;J(9BpE7EOLi70B? zL4UrNG-lo02~Mwf^luwvzWemg2me3!!T;qyHAHSR`KfJG?w7N|Zab#k-MsCI96kMc zMmxbML~HuNOVa3!1(@Biw&kC=&N&7?d||EBm0v!oLjj1^S{a@Txm@H z-eLPW8~IfiTuhp0lIa&vvwnd((_YqgQHq2t8O$~OPG2-Kq1NBVIy5k+$@=hI`Z+55Nd63Mvp@6N^{<~T8mgFi=$5I8lm3%A`q9A-$ zqjACV7ctb4A-LkhhFdDOhAzDpMJD-uwJ*oK{rN^jW#7G(_7FFE^3UJPY6b(SdgcDx zFQ2%sn3Ajb*w(k2SY?Eil>NnSmU*Qy$}rd+4`bh#g{7{$~_iDrLe%yuvQkDuESk-{7P$9QqIc=iAN2QUpMMf=r-qUAHv@M4vW z2sAg$&p(d@AF46V{p|TxBeQJyZ58Hsrk;Q$Ti5JJV}m`qXqlm7=VfTrqZ(pQq|rlA zQx>Ej`^NrZIt<3MAGdZJhkmf5j)z8CXxjr%b$@^6`yR~$LpV^tu^^LtUUH~W*y|Ih zqvwPNCHE(AIuan?(c`9`)IZe1J@HdZalTS~!;t-?yRm<7;wG$YRN(nJYP_Poe&fiG zZ%qS~MCk9t3%<1SbQ0UE|CAevRS10y4JH24MoE+el0e8GnSY6aoHMvRs)bUuz4Ey} zU;sM`heKb=NBgYXfl)oLxK3@`-OCh9`?-3-rp5nhVDcBclK*42mn9*O{`ccZHaMXe zq4=rC22ExO&Dvq3-97%xA<@;MP56=MIOg^a;Z>FJ1QB#zaeJXHpB`&I+mRAAp?B$4 zSW5pfxBoxZX%QR9d)#wnY7jP2@LwjhokXvcEq%=cVx)Wlr~3GA8QuNuK%QoFDs>|M zvj75?oXQt$Yx+q)34x2Q$T_s?uJ*f)NTJ|rh0hhvRJ~IKOM$&c1-qRiXucmTwjWKs zpblukDb~u+E|L~)%Wr?AN9`_l4`WdL&neDo3}gP!1x&oAL77A*dOg!-{6ySR@N;Z= zkGj4DJg41x0{tj+;nRxuE@h8NlT#gIUcJvJmtu$o#BHLNtkYP%@Rhh_eeo*{rVpTk zi5E8i=>Q}RU+n&m1ygH!WQ14C3=2b*Y*|%sIRvUYZqf(DJd0gYyjA_K92Oh1M!;<* z^45 zUxGino&x6wsvfO609s94+@P~y1U`MV?KFBxhKPfXgKo?9zD+&m5(B=b#kNdvve!<; zP_G7#_wkInIlqV!ZSeZ|#sH!guF3cml&kF3w9w#|qSx?L{ zJQX-4TIQy>R9--ITnw>|rcPwKkvf*d`AUeBzu*=_Id6JB$`%P>!K8~)A0x@_8(}X; zgE*%(=LH7cq;r5-g7()8!HOf?tOt(N1+K)~n zpTCXb%%04F%7_fuUS3V9n&cQUwN*85XAmg`lx{?xc^%Pj#h01vX5vlB!z1!M8s0WQ zUu)}M6HMl8#~zxmFRM|MecCmrY>cV`H+8~Kvyse;?kuxQ=wrQWXo!b297bbi9R4NV zkzq9YQ%%4uM&ZMO#_XgLaIcHj&LuU)-Rk#28i_7g4{yBM2UD|sxs|nkEtjG?dI;Ps zYK++X^`^mZ%>7KVqn{xhVHdp>>(V?}1bedtZ6uuB90m=)BWH8==pOB7iTE2>?j!@& zC*PxPWyAh+2uR*llfx=&KRrr7g0}6iklt!&KAqbqNFg6tVrSK1WgAsnnNJ3JW!nUq zY(!9kjY47SN(yd+%ld3T_L@H$O+3N3!%Adf39#}5F3|CJ?5)$U^P;rqULlo)o0CPl zlv_hcceLf6p0teB^|vUu;tIivHK8NnALrD0+bu31E(F)n=j`>C(y2FGw5NV$xe+vaaiSDEAsn&0!|ix&9a>{;}w zzI3|lm>}OI^hATTtHcM?#-+0vN%X!&{!zkICOIqF<)2&HJ{O`&T*8{7;BVpfyW{2~ zywss`Xwh% z@TQP`nvODS1qb2j;z~T_|6^efI*{LEkc=pL04@g&PH5a_7e&XnwnekVoO)BYty^Zd;=I^XJTi8=e?C@_HgE zgli@rXDB-MWbU@R>`HRoCgD&l45@1lDXq)9RXHGct3q4M3eR69gUmR3)-#^TM?D1(>DC!h+JK|!dxrt9_c3ni;KC3T`$tbgGU|?reAuMHgp-nNWj25|-L- zk}`JOv}&0y=nzwPPx)q+W1HRAaQ7wW^!;TR&%=H=ifF0Bra9>ln$bvb)=w8#PbZI2 z4C$;4MbY;RIwU(RG=5d?S>H*_h6f3IKE~Z@w9TsQ&`Q(_(e9-Dc|*Gia#FcRoI;$C zxcMuJH3KuR<>lpKU)ePOFUHQ>(jjgV@?g;b6V#Lo1lN6t{B6toP8&% zEW)2z(KF*xOSMsTmrW^V(fqVONRC&-S|4=HJ9?;<7i0D`aK$0Jz?U=U*ni#y(G~V< zA*!S^B$VhghH#?A12LHL4C$D`<)34#@6p2TC#o&|Zs!wgJ*vlUj)egvPkfGVA->mJ zEAK#U3)2$^TW#Do26Yi|P{#lTqlk{E&liF>!19AH2j)~ z=a&;2E8YR$EEx1@-CIX(41NGHN)m$!>>;}hS-lmtTUhC0T)|cAB1Qi^Mz@&E1%7$` zM5*4OW!F#c`7xgQ1DXSQHZ=~0<40wG^k<9tLaEj8KiM6fDBXnyYl{K|8#aBK0HwQX z&-@dKQwE6wV!nQyzNm-^zIAwNga_a^F=H-2^xTpiZOYbqX|@U+s+-YdP|XX0yv6Eo z1UGV=|CJTpf6oeG?9vvW>#()v@_FfvM)4*<>yX@{lAV2(CEa!2C-jq)_CWNvG>fLN z1yDc6pAWugy>s!Duj4=lKTL|gWTSqVOpc_}+xfPqd#{W5VaT(FE1w6SOghW6UILPc z-$)c5kUsp=o*w?ZJS(;=pj0Nl+aA>YAo zXu)4jC)o0_+JC~P@xO&_l^{il1NgV3XF{ubuSzScRrLu$NrZPNfqFi4$1AktGxfx6 zL9nV`URqI@6!|&K4B+T1M*l0YfAvLEr7K<0{e-_KljrK%9B;mSL+SoYV!+^kBq9Ip zzlEyb%62ic9a*%GFF$#0kf#}%0>tZuAPIljjIn9r_y0X!|9XbpRJk@SCt^YoCr6*O z)oFN5#R}OImd)kvj${{3*qx3Ak%pmxBFm=};(>Pm(+-_*eInj}UFm-b;{V>2{_mJ5 z&0O!fh(I>8+VBR#eQTfA03x!0gmq#(9>eha;DlzAQN@UQKGp*_`PGwe_;h-ERsUz* z#dExBw$?X2@q`e_d-N;`-L8rZnZ)L;3e=!zVk%}kJLZ4zmroSp{~DL`(f?@N|BrJ0 zFKtEW&VCI`)X4JrJXe}&Cd5K)N6?nGfx9!>(Mpo~M$eTaBA3(0h#LIA z#XDT%K>y=a=ZMRi&eoqIbzTvDt-+zrOM?KEiE0IP@sCJ-Y2E>p_3L>nMJX6=<*L_i zf7$?Sr4TS1HMZ$<@`2*%k@%(7(~Zf~!ZYZP){OJ;Z?{y;@Ye<)`oc-wmQmu*)WdDg z4`M#xt=yv`Yo#Mm6vf#artnV#?~RWE4RiwwZFA^O_^08f4$OUz5y2+4jJ{EbnY_YI zsXF2WCgTNbVC3i*#!JRlt0yb|NB4NS4u*>ZrT%w1f~WiOb*27wtf8=Z z=xlj5u4Ihd%#OkBZbAe4yj)RiLjH#+6q8gP)Sh6Ogh)DQqi~*$<=R4AzQCb3;Tw)JvM%t{6?rBZ#+Dw1~1r+xC%SE zpjVS|;@WZ-B}T!Cj>RYGNOBx9(K4*~fgpp=w{s&-g%s+b^^<|*W3A8(a?SAkXPCX5 zc3?Yp*O{{~c@I61QLu);R=%`b2;P$3=-tz%rB+0*C64yMaBMa8N2ZrGnTlm9G?H7V)ZATEW5WHn$J$95>;aJ&YN>d7+ zAIC5v(N}9UQ}kOaM#YIOD&6~LKthlr(;K6YwQUm{c~$siEZ~|S-1S^T^*OfV9MWN3 zy%F#;dH_ZZELnAW*Z6bY?ZDONn5v>5wUCy5jCT`#_1{PNc{^Zn+cl{FHFKFb148?`5#TCTO-rXZB{GC|Jk5k120V^Oqs{REqp{!diurfhO7+j zb-b?Fz=flS0twOX4dy?SX}rR>l&SxcqX5@f2?!cKg}9hC%OoBBVX?L9u*QXW(fZ@6 zuarSlYMc-!)%CXT+o&q&k)9`Zgjpe*-rvn0_+iPDDUL}v3AKsv8Ry~je#7_Sq+9f$IbCNha z-`@n!7qM^)+=)iRyw}sueUkLDMo|^rk3_o9iiioNF7sdBl#|H1r2Qh6*Wo#b`Rxgw zY2}9rEt0 zq7!8h08xsXdN`pTBxR+X-PkMqYyUBl^Bvo{Qq}LwSKrW|{v`)cdmbQOX!e71YWy}7 zL88KIr(f#b2(g^rFrs=1DXY;$hN>@1K$0J>!3X1#foZgfqp+Z`t; z7MUg2oUnteCCrs#(qf&B#^K;3=jS@2BJrkPaImkN4#k$Oy_H)l-<1L6!T{=Zu(-M6 zIU)#6=YNYR=IFJIVg)#*h1~xymE?_-4<-JWKO6KN79{ska)vLvp9R=!#Dot@Blcf*EP;+ZT)j%X2cQ>y8}6kWsE4%;K$DwmhU z7cYp#{{$~$q>_1L4Fp?vsRY}#j*q-3H~A&$^gK!dSxW5jV}xtaEl8Br{q^?7ju^4J zX|NIyjh7P5|C0Dt%ViqU3)C$tkFKtE8qVvSf}=?P_%Go9zA@PUHnF(bHbG%4y72nE z+@txq!fc&Q>{63xRId{P+M_CB#=si@B=?#7zd|XMDH|hm26!I01Evgnz)$DZB4wy> z&>Dy!G;AGhDNQ|juN^4)@H&xWNa!qQzq05)jog2A6yMVSXT}SE@r;XBFQS3+`4dvl z0c|CB%2Wls{gJyfyvoaRn5}j?3~XA?2=mRK62kowE%e#`3>o?9G{x-=NRV%0k9%ns zctg~KEra@9@zv!@dm!{urteM|D$Ic*Z@$3+Lh_taj>zLBpHZoD?LsOex&2CpS(*Pd z)%W#Y2aE72>J#*M;|9vK_554^7$-$=v5O)jClhn{l4Z{<8GI?`j#8EF95l zG_H`7*fOLu#@QUPvq&^#&E}T#1NSieo!V4IE`_F*hu}1`cj7THtlrdp>;6>cw!2WG z4$d=uQHTb|ENX%L_Zg5JC#_5ntj`&MQ+zRQ!Qj004Qix^#jP_WW)B3N@Gm`RVRlEe zov|_ADBOJacvalrtU29evt$+^!PcM^UGDS= zjM+6n5Gx{+zwR$GsAkoOfqEo&=$r3B;y#ui1rQ1b*cU@As!*?vz*t>RsdN-mdMO8aSgM^rl|$#;KCabWXy*eK=-!aVTo zhFL(Nw^+l(xAUSL*9Suq(AJ@5`_X%Y$;Aj@XO&D#votx94+8{H8O$^NN`ABm-GIEA z0uO-uV)9Qf64`7HyL6$ku5ZK(IRA2C9%zK8rL;&&?!@X@pBk{7GDv^RdwX?$0s=(< zz-R@Fpp#f|G_-;`=(q9_qQ{y%_a%s-gW{=>13R9rc;- zZcaJ)(^P3oFLO=eg%bAVO6?L}z-KIa3#VPJU8n%93EXsGGq7t8u+e)>=TUc_9U=T} z!&_iS`&l>bB`w0@oErZ2Ad5kW_=N`f}T4xlz+gBY{o+i4-lk*aFU%*iM+of$`j%MI5&5 zRht0B2OHH~&J@?X4?9=G6%X973kV3Wj{TnjC-7&oa5oY8$0S}bgAZd;f9AIUYXbJ2OJht9NuYQt-Oxv1>8<~;V`9wgW6#~i_5SexLwX$ zpvvmQCk5|_B?t}!Z4bObV6Tl~F^uj3SXyZHuJ-L;d<&0dsJ%lY>W^7S~E5#%>N8accZwPTJ>iY{S;W?WpTYTVH&&Bx!Y21Onuxl{+f0Jm;IjC{c;yOq5I6TPgNGj8a5f zI0MK_cK^b!#409aFM(#WZO5I~;Bxj1=B2KFcgUw1HwIkWGeHir95lT7&Ol}I0n+N7tvO2+}uuk4AC9 zJ^?6)neRhwILy&R;3~%3&$<+Jsy=yQMosvaF0Hr12=nyqkVyg$ggw6|g%F^bh<| zV(;V+(z9*VRH5`#xz#!|kIUsHN;+_z3fteFai3O$qPVQI3$Pv&@|j^wsY_c5SMy5< z&D{IcsZiqn+r0gQ#?-N#LrmzXE^$mj-%owH@x1_&)#2TfLfgW?aT5_|x$2<;`;{Ct zJ48R4aOsf z^LAW}tm(VT<#(N0g5i2#EFUR8>0xJYW2lKD%r zfV*kMoB4@h&7{^D%w{G}_6^9Vb5iYz#0N;)*0Y^dGPeUgjZ*{b^CrjT$Yn-ql@~{s z7j&IJjh$vdds@gZXtAY?k6(=%9g~wtuVq7rPg0At7xDf=X4mR6nZ(K#^O7cLwdE`n z&bz3yD{I=uNAv7^_;=NC7Rh6+1i$R1I-RGgj@s@?k}9ktna@Fc0et6$|L9sl-9B@C zCLEFpSZsaanRvx2ZQdJdkq`AaRl>0o6v%1vH`{;<1Q#{+1dHAtP?TfE_eT=r5Nx{oQWRtAx$&*iS!>&XDlLU&jkh zxLT_Tu?dBb#B$oGrCr{^wmEvpqT$SD0G@JcYisg1h7G2)hb99$et(<8)_?Tg5?X%n zh$NIAGue&z_Uk7e|9Wg6Vu)1O zb#%l4B07D@!V=uvjB4tzQZaTu7ovuP2i+&8fAx0 zOEcet3)aa(L7nkc;#g7ZI^k{Y+WmzX-1#}h)K@**`9v`j*QUfB@LKhaL;lX0IVcb3)>n@&*gOJyTbfmUd)#_UIz(1mD#1QSuSy^Y}ZYL!R_Dn^lz^Z5mwK@=sC8I zzsj!4Zqu&%K>iHzSU|)k)^#3_f4EtEP1x~H(8ulu{(}+}wAVaerel-D_*3xl#_SV} ztHUyJl_hX`k7j!EvEr-Y=P#hhXk?9y;Uebeq&o$;j)S5@slD`=u$cfN`^MM+xS~swvXA$x%{|Tf3I)ujS!w1+9XyZ~Bm& zxbwJM84){J=6UfgLR{}ODxPID4?3rVLLM#Om{lk$_Q;WCTpV@a#szvqK;(|>-|k0f zlMRvvbTi5LrFF$u?|i)8^x{OopRbV~J9fev19N@X05AZxVw{K`Hj6{s!)Nt<{Fq5l zpmd`p@_4UiBUKN16N9ZV!QVRS{f?eJZO$@KoDt+fGUTao%K+jG@f6Oan@Su!$K?}Q z3GJg@K@-PDGifH!vG3VpV`ZAg-Ctt)zvEu~pw}*tn~$`$L7(kSw50LscE>S^y-g$d zovSSe3z2+Gre3XInZa}nyFv$biBFUz7gR_6i#ic zjS&X7x0cz&W|VzM=hK3k2+X-3@}nIvO~L)|gB7HlP_m0ia8$h5hMKi6_R;H-A58Ke zN{r%r93nuCf{Xp@xLlWZQ7^Z&Hk1cjcA+R}0(sE2R_H@^nkY8$m|7m41SKPG0cewW z6dlR|{4@cU3lXgSlN%@J21VvUGy)1FxqVQN_e_JHBX4gsHZiCz!uC=s!LL<$Xw=0L zE#|&7^76HV_W{Iu7UpsX_J`gW>K-9;)Oto?ZJc8yUG(JUd%b1F@5bZyLmU|d@)LLI zBs&T?c8q=Diq*xN*cizro}ft-eI84nN_N12T+!-+6SA&28g@)_Xx+lwG1|uct&`VB zyftq55tk#kfNOG^fB)74lap*=aosDZx!)vq14Ev`#R-KZy}L~gO>MLj)YC1>$LtdKu2DFU}GVd(>4!5 zCulUCjmyq1C4@ayn0J_Lg^r0G8pwsBj-0U*FEL$CkM@exX(0m(&a#iMMr+MG)igC@ z!yMge{gD!XUT;FH9E&H-dWx)F!0Rv-)kj=YyqG?~6&@E_9ry|Ny7XoCufrA%x-m*t zl!He>48daR zPLw|MM8;=n+g?YlhOcEVHW^9{Dgj6`;gQ#vW$ZxU9%3waRG?w)(rKx(G|RTo_w5XX zE;}{k=XU5d`I@jYQk??aU9SV_=R}i9*ZZNe%QXQ6<*|4A8rmwpF%9Sl@5*66kiStk zk5pnUEN(_DjAIJHK9YSZRJ`PE65zHstODutFYprS| zoQD00U{YJbYhXljb={k$Z#pr_Yvz7+I3pgnhKgaim=HpnK;5%vs*H0VmyVDaML~b{ zd9z1|Zr@BzBwZbM-82f;-Rb}qrLwo0c0tJk1aWHc8%6xpAUn|!Kp7;n=GKARfzw(t zp6KblhUmAvV6EmSJ`M(L?{?YN>hH*e)F6)*YYKJ*1)tfvuFGE{tqe6R3B2yC z+Wy3y{XmqYYTXr&=<^m8RF;(cc>5f3)ON+RQu1;yoM}}y2tAa~|J*r|{GQN8YLF7U zBm}gNYsVB`qdi=e`xu+vtS8vxOrRbkU+%ob*HIwH?bWmczW2lVBdzD_e%IWlmNA%-;%4mP66gbxI&X^%>nfc z>AeGU!Yx5hv!q~tv{Gm%@oVQr2>4tPUm6gTU7}Ny;*|3^6@oYDr z-p>yb9)KvrQ?pg$;Jr){TBL%Gak*o+?U&u@Z=dhO9v22(W?$dQ;iP6jjX2+IWno3y z52`U&^thnCm%yG!WCusRge&^`uIigF7HTVaa3V!~?p-krm_4DSE4sLf{&Mxv3B7(s z^(B^`pYI->%hyV62IZqMUyL65f?vWSm#%PjFsd*eixZm(JE}i=PlGe*n4uesm^7o& zfatVl6rYr<5Pb)qbVo$?FxWP*XVn7(!sW?FX|v|>Z~EFl*G?TEmTMXViz6pTWx0N# zEEs$bq2{TR^RW;_=Xqc~K5o~cTJmar{U)cfCVl-{tx3k32ItI9_?~{1f!DAzj|f`Z zTiU>kDExKUPM$Lb6+YLLTrG?ZbARl52OE>lKh|~yvn_896Tv>*fi;5PZfoacP1ZMK zADOjQFrmiooJ~c?G0%V8mpBc3FUV`R!|#I6wbV30 zV*0~FZ-<-<)NBvxdqBjgvS~d*;i#^*!gctq5wd`Gfjw6_v)lM2E$kG-tYX-z0j}Fj z<{3$1n&0Ts>=NOS0y$q1A#V|AZTUKHcaM@gPK7W$ZVU#w%D4a9jMr`cI0U3L!hwnC z)9xbbJK%<3L(8-aiAj~?7Wvl49nq779|k(Twek(0Io}IJKbY+lq+%#R&@bTRuH=lp zH)wj(W>yWdY9c_rBx1*(i9~3)TzP^gSR*NMD`-i!^6tIaZ1>G!mb$V!3k8gIQOJ9_ zGf^#-i5F0)xc_CIO!xAOPGeUN^e%r*NMOKEqffcrEY7Wy|AgWKZpL(2 zSqrQbklRl^AV)W&;;!9-eUF^{UbM(y{!Wc5vfJr?=me!XooDHGPj{1sG*w(J`wKS@ zzs+uU>9BcQW8{EcX{MR@01`>(*MIzIE`Z#aJq7zI_gB+Ofbr_tN0eFb*Wjn$LqI{+ zW#d`zu`3Nj4fgX3Z@#`ZA}sV+LmtW=WsOYAkkj$nvAlG(672&Qq(Fif1%#M`Zk+NS z8CHk{=#AQj-41!m-eWyl=)xT8I^^Yd6xcRap<6r#&@p0-VUR3X`76;_20_WQ+Rq>U zeE$kXLJ$TYzKOskecd#|m`^vDfgwWw61<-tg-PHI}=c#dNq$j<00E_|dK9D$t zJWPong0+Qo5NMkz^hCOI6xcfw8oaupy8o)jk8;$X*8mV%dF9_7VY)Lez`1tgEkf?STwTLMOjaZaiMjdk@_se3PJ88021ztg2f`FW%kbU4fsoN z7_u^A5pku|Q=aek#17gqU+eh`I8#1C4F~F2n9U1Ufh2W>4zpNu-O5| z#g@OP>{ka3UCjDuKUULaVI$dQt+aSQl?xM4EPhi4Gy!RsPIXzkm)nlI!V6Vh`&CY6 zu<|BeNo-rfSuV)YWNl~=3yy^;y8lPx_rM~P9bsns4ccQOS1VsOH^c`u6hwBg z3cRwls2l{UIt>D>pQ){3JcIeyHls)|7XL<(W=m3|wGle>K;!{+oQmSnKu*!SMf{?4d zD=LIH-=8j&=eb=RFxR-pY4^p>U+l4_#Yb6HzD3&)$}g5{gVb|bqMXG`a7OjEU$j8* z2iDL>rr#QN;>mx6X6-5VxA-nJ?7SD75fOa%Q*yfUt>~;NB>!uq%nuOa5a5?tv^V_V z61%Py6lZfNq}DxhH|$90LH$%yKKCG`>}E|al-ou*&FYLoDdqVT#)`wsU$XMZR+!lI2@%kkFl$HuBNs~so(xsj|E zv-q%iQ`}vS8f(>w9E_np^6Gq5Yh_saBY z)zYV&Dfs8x{_f|7V&kIuLcb^uW?$LaSoGrq3mEX!Pu3*e0JR)%c}chJ3U9k(|6%4E zyLN&v)0tS--RkW>Of_UH5ow#<+mmGNoHewc~V3_8A(;b>c%B4A$@f8IAC ztX*W4=?!gjz`!9%YG~2 zq_hvwq2xlxSj-^jN{M=ED}Qyf%C|5cixu+0DT%poUmtD+a7G;I={!A3S4dcn}1XUkDiIOc@y?kwt#Bo>oqy@1Oqe>S*7#L;<-x% zTaS+C`S)o;pI5bMWn+`q!GcJkK=%ZzS(ZLvITfO@O;CQlLJSI6O@9>M7;@81E`08a zc-BmMEr8m143Uk=NYeVnP#<*b3=uxwN}gi`sko*91Y^JV8W z*Q<$=Req~1O<(W=6WApB`E?$2T-_XX_#2q{Mhsvzx$vb$j;otbU#d@6IY=F?_#fSq zatpeLp6>wq6WtMt#cwX}zBB;Jy?XL-`vG>UOzX-WFG>-M3=4pHZkZ0=1ms3`q-fFJ zc>R5;LmYoZ7-43yqKr+*1l1VSn60RS0vimDL0!^5YtI?xM}gxmZ*s|0Z~Oh9c`p~< zNH@^Elfj|S+EhXQqb#ys-+|{{K!VxM77)yP{6hYumzJ%Wrp+oxBJ&wwyYOJUu1v}j zNfKvMFTa%L3!DLq09l?cQtOTrXG0QmaNLs;8t>G^%67 z2=w(J7Q*I;j(~$%azpO{lWpD(<|h^0aJhv|NURqY0ZJpJ`myb=nJbwNC7K|9I1v;G zy9h$AsZz5@sNay-6~YxM-O}F4Q(|mV|JHoaXCYpqW~3f(C>e@YxMM=+Ycrg75@}BH z00TE??T&L0jg$(QbFFb*>(&v+FQu&}CENDrM*nAqz(YSUCdTEQ#!XEeXIP9UiZLq_ z+zqNMcTuep=rUk^l+`)&eVTKv$?pYTrHjc5=`M(Y`1qM}_w3Vqbw9&i1-ks#L=)wN zDUEz5(a};ebvAxejw90OmU!K;zRMD|D6p9>1$5 zy&E7H{~pPdDwJ|rF{v@WI(&@%=Zjl6v%!dWowbs>-IT{K+>HcO?1cTA2QS7^14e&% z*oBl3@5z$3c$!I*H#;cUs$oYoWXKM$0;sPwU5(+08J{gdZa7|NuB>Z}>DE-eCRIZ)S) zw>JD`W*bO#>A=i%!@_~hz{Ksn*#ddzUepEs5a)viJ29W;?TgtOh+1ocq46&2=fi&M z>d7Ef^4{zhFHDSKY3++I*%BU&2JI%k5X_Ta{x?68qT-vvhY~v%xb_N?IUzJPQ$MtJ z=S~RiFFUaBj?#I}RLluKxzAzQTOX7QOu`ZwJA^w}yO*XnZYdPdj`#@{WMqGR?MHXA z%R+cW6!3?W>T%mq*WjrFK-2d_(0dJ@& z>uWj&X~H(v);^}6izuR~U*{*Rvw29-8j}P~zft;wl*O!EtOPc7j7BWMOD8l2 zn18lL5j_4(`Uf0K?-hS%70;LJ3uN5aY0(mQId9vAt&*5&H;LuPh*52X34V|Ed`K!C1?5Q*53jNx5Kbo!W@B=i$?rsB50Y9P)%-khU zmOLid+mi!dl#j^zf~e7R{JM1|nFR%Fx00>RsN+t|zb7S<{{XEO>@*s?oB{5c>>t%p zvN!wnTVZXcXwDPXmz9v_0-y@@$sPnA;s5-Y_b5j#9 zrU+3rQDGAY4C8oqUiXjZ%UTJ~zZzB|db?EVywcFepVSdd96z$MF{CF~4e3I1y1i?# zTMI-D+16ouxniX}zg4An_i#Z~^E}(WRi=emJzZ(_m1k`ok@Lhi2hg*3M5(Yp z!ee9kK-mw^kE!2t9aRcE>XPDPWo##kJqe$ekWVQA&~_WR|@0$+Rda z!pm-^m&5C02fhQ-y?BIcShL^7hoz;E*smL7WuO8o$qF!aH?zaq+lt%GL}cW)Iv*}} zyQ?GudR?bVJs6Th-n4omTa$hc+@fIhEi=BVzPp0iJ-kg!zbcT0=iex&u_4GpZ{FQk zBvG~`JuI91*oH5>(Ux%qOQ!^7>!#$OTH8P*=Gl0<{sC@vi`XrXhaamI9D^ch5x^ab z3a_=5U+#-|=5MnP4v&XZ#xt)*+4Y87|&PJDnG4+AC`s*O-_4&u?yYdM$4< z`AlC+Cn5LM#dsU?H?KYXp@vG;Hmoa~;GYxV3X<^EaUt%6wREK8x* zV-8%TSl>oh{Z>`8Oz~${K30lomM|9QRR3JA)z%i^26=TL<@ah3!Li#h*M4&=DR<_$ zO{Ul5nl|jW1{oJ~Hpsf@QpsPb+0gM7KRD&@)CH0Nx0+5$K&)BW8CCDd zyiMloHYqj*DYjuC6B{S|ittV$buvbK?linigoK`5!*D@_tSLRO;4k7I7?(*%FFPk~%`S_S!SQcTc4HI6-sqyA z3PFDzLI*w`t=wmwnA&)I>Un-YrLQ#8RK?D+0pr$8?q_Ce2BCEeuF#m|X>^?ba63dS z{#^a+akH?&dVgUya)S#i6tLF?o;ct4y+-7;M%2;ZRyI6opVX;8kE%e%H$tN4+BxJ~9PEk?tZkdiRE|ba-qKgoir6#~G z!j5GopjoOK8GQC1W~<^Q-aF`;b17{M5DWwd+D=l2mivdmFF(XNG`sf4dQs=SwR*td z-5`Svhf~fni_=d$T2){or9z=^=Y-l+`K+to7roj_7X3*o9e!PM{2^`;z*r_$lIo6jm_;1VhdQPSwYB-Um;^&jlu05ZzWetp|4OE^{q0%0jE|iRDp5`kaw*a&9}Q2abKE$? zQluqmkw_ZLb(;um#oxG^yZEUD$mZ7GCq70x6l47S$C&=NcujI)XxF> z-Lw1M{vyhvU?Jh>y38qa>BpVdNydhRbv8$-PyP}{iEEJL}Yg;{t(;{ZjuAC99wH*;ICp<}nRBOitEZ73~b@gg3edWS1 z`eqa*rKKvLV2~o|1Bjt*JYM;X*7*TGw}hZ(bFdo|-#HHB-E$D#)iSUex4A_wdsxb= znqo4tG?m_^@ag3@Eom%a9Msnd)MfY$$6MIM*1rrMzn!m zc;(=q-=LUqRxi z(KsNoK}tr2RrvW{gokG3v6RN}cOn^ENXjg- z%u=UBgA0E|7SKdz$_|EE&P#-s%a%T`=jh0|8kLVXsN`(5lnXKxkR_a%+We#Ur4EZ= z_c=c1JDzP71Xi=oiHu$*2k<%zG4<|j&42)Hn+5MoIZd|cPZr_@k6a8pl*I#1M8YBk zuFQnA3))X}iDw+#9w~yp%->bTTjk`GCkS*eXZjQv8G5t@JI2zY9`c=Z3ys19{!o)CY-j zNEbGgbLq=_qKRh0L4#&#YJsOiJZn?=w-l)N+~Fx+SpYqWWYsdtb^8#}n{eNzexdDV z+G{xuxi^dB>az;^E=66{{Gup8Bw3pjrg##LOVw6P{VBNDcfo(p$c=#eA_Jsea0j>A z*|cZStz_Z~R%LqzAWRc#fX}gF z>9X47-<9~r)n#|0A=H%IX!w?5_ZBMl<8TdHC#dH-v!Sa;Fb08F z-t=%CB|=T*ECbPiCa<)ascRSVWEKMBsA`m03ih%7x3shzX)k_%&3+R)GPTEh)ZQ|w za9xoG;~i2##5QC0ak8&@^d+qyGWSM`HHjZn6XH{3Dy;-%TzLCPFYP_$rPD!e=EjN; zfG9~>Rb9{xTHRb#j7tjT=^}TloxQcLJRG_5F*!)w>CQTFuu69k9aIfMuI+z4ni8c4 zEN5W&`*K`znMYW{nI-O3WaMSil{9hG4-?g(^AlA=skg~JHfE!BMFPa|&F-!=jLN3n zNA%}hvuF|_?fP4m-fVuy>}Qg@&vfyvTbd~t8r`GgqphQpThvobI5?t))t|sALcf&Q z@m|7#8?i=Ps&gf*Q=)@IqYI^QF$c-zni$_%fr94~1jxVksiHr-`4*`9t4nV-zM0C} zAEb@-7#*vrMTz+%pTm~n)zU<~rpE8;^HKXRXEXBrUp}^2E#=S~3i;*a(&3bksd+{c zGuVkAw~GuJozElf5( zH)}$Hivdq1;1+Q6)E5bhmgA+0dqYDl8B|Z2%T<;+Wq_oh5IG=k1Q$eb2} z8ojUXWZ4VlPALmbJ$OeHd+8+ITK4@RLbVy!@lB^sn4@M8#6=TC6h?Eu>=uLYouemX2x4Yz9a)Alchu!yly(f6#Wt*JbJ0I{&VRp@(wuX<;UYIS(gUgKV8 zRL>QS;cf;YIwT5d&xLNG0g;Y5Zzr;3R=Q=W%$uc$FQ&?DQ78yM9OMcBEeKkA5rEh2 ziTcJB&~#rF`HrR@6)@!jFG|ng5@+DrfKpe{q}1!etwP~b=Y@t3HY>Pef;9>RqfiBb zomq|!J-kULR5{G{tqNr04(<2UG#c%g;b*T+Dt$j?3)~)XndErRPVgb7OAj=-yO;R1 zik_oBKB_xT!3yz_b3jN}YowLEGk276sPxhx) z`R*YLeuL~O!tKu{(Z|oT)!wW;X2Cq%h8)AF{QS3;vG4g2oC#}rI5o&V?(qy6<&hFN zh=9B&K)MJr=eQj5)aB3j&hEKTjYlfqgL@Q*Ls67c#X!_B8Rrx2N-QovblZQJHP0t+n4BpkI@^x>Z#0H3L5JT`wpDV`ZHJBJ4VIqt@ zn9YI@o>dwh%2Km{Ms4j_8>u&+=YP_uq&t`Jy6t}STfnsLZ(1kgi>hK)7OpJ3peL8U zYz?W!A$i2-I8ujf(J!tMfeapZ;XQ*94358o0te9|XyQrK3<#bx>XwBA2b~ao(umKW z2(FI<)SX64o@@~7tvuYaXLiR=Cg;6JOl{IpP1J-x08>cv>>Ih`{I=0~kpoTw=lRm{ zE{U`EBt6smF0tnZ3ciR74AWewM0^@3xI^X56eU9Kfu2urWghk~bzCfWHP_GDGswzc zJXj-}9TO0r9Y|wpBq8<=pSYl&{VwkH-5RH~c{fh+?xRav>hpo`z0qTZy%_2VjxkTe zkz~EndC{>;vLGqetk-}?S2N=&Qmjn+ULgNx#0t5-}4uRl}yF>8c zn&3`?ySqCy8r&Obnf}k2nTvVO#W{1!v)NU<*4_oR*81MBJU<1p43fU^y0aNe)p|ZU zpu)#CVaPL6yu=6G)Y(d?&vh3t8{S2-EhHIwH}1>`QQk=U#U#t$-F~Z9dERD08~{Pw zxeDOXYA#sa&HhH_p(HQEk@PA~{xyyA%5N*gU8D*0jvZn+FZ#U-|B9(X0z_y$@u-!DerQ;Xo;C z>H({)=Ndm zVw$dP{%+cXx#c!h=>?q+BU0WGTc`6CWS!!HJQ(NbISajIwGbej(^K6uV0|b+QEINb zF#dG&v))CA!*Dk}#|6)lW8g63%CPbCdNj$sdzb&wnP{V2`heiF5feW%29Rir!y1mRr@dco=R>xz^#n!H zP-uaFqRVx!(|)tVHl9l|@Y=Z(Duj>n>M+sx>(4));u~bXX)aQ43``Xs=@pIkZ#C5L z555Iq_dB4iilzBTJ;^lMOb6qH^3z(;OGPnlSw1ESZ>i=8+)73zUGovS^TU$9pDt=^ ze;hL@z(J>;^Wo{dOUck}jCkBZ!5{nUfc`ny19=Q<(v5K%efs9A=9wb7`gP$dZCJF7 z?&9a3!Ei4}sccv%@mpWzX*z6?Tk=M=VBMEBEM*;sTw&ZnllT&Sm3$MNJZjh0Pha zFE+n#dumztSNc@OmmpLhl^qInLjG+{-IJ5iJ0gw99Bw+3Qk8=kCP0d?feD$AP5%CQ8=h_j_<0X|Y8_@6XKY@7k_^^-#Cey~T>p^ZmQD zTl?w{Z*_lRWZB~Td9UqCg~onA%t{+Q^huO>t{4$1=EM~bCG?k0t%CNyckyxK zK_+5&`)fXWwSNM_-q}Y)BmIvy8P#rn7%Z;;BmVS7hqN4)Pnw@TReGkh}ieKRdQQil0Wsu5hI^{}*snBiZuvh8jg-g)N--%W-FrrZkA z?{5j3f94*`J+Y;2Ud=G!`-mi^Xkhe_z2Ty!5`A#RDd1Fn$a+t27}o%1Uw22d{G;iG zis#<>;j)96pGC+_rAx+AALkAKfFFgE${WJ8S#OK!9QxLZvXS6oBlHM^Fg)F`S;XHMvZF8S1|FzTg*8B-T+HXyY2H`>- zsr@0z=+H;jeJv6WqXCCF0Z;qodo;-Rpb>>49GsR_Peg5z~m|z^%hb=pUkh2Zm&5*f?FSU1}L4sCpRpV<*BeMbCD7 z>`s4Qu&1rJ!R^9!tD$}8@qHXkBH*E|2~!p;N<5)1PSn+|dBLkegJ-IlNgdWNsAJUafe1+3YF^=1Yh8&Jv4c>T%adn06`VLyO)5LFg;r!BF8FyY z>S6!Y{zJQL=YP?WonDA$TjLj2%T|1fUN62D;A#Fj^M7AR$aZk9RmO2ptU1MeAm%&> z_8g}9UZk#jchU9YJaXm4#n06K)fd)IE{-S_dG`yl3?n&iSp^~wcRen=SC7uk6LLzt zFW_4Tfu(%f)6V}|juItjeG;`!;~Tt=bFXn-eRm3&f@AXyH+Jh)YJDCO88dtr?PlH! zdF=%?XljcCqy;EFr5HW#xy$XBX{kB<^!22NJ7x_F3b(0QLO*?}%1z}E&N?WlQgUs1 zeX3VuAp)0r_cJclF_JQ=ROyWZwZ#BCcR-GVlW}(!>*=z!7j*5B>r2j8x5Xb@a^zo` zLGO=D$1e5^;LoSE8BZAwM#kOl@wKOVV}DSV3HL6O8r;u&>Qin2R?z3&;VqjarXhhp zfr~|Q*fF)XLAX;VJU5KAg@Usm&hO?K+hu!2OcLIw^gQiQCY(!kxNhcE&UP$#G0OkX zWhQ5PG)QBU6A#4A3VriN7mfVUNF+=WAVPX>i2qWsN}EY1J4AetI+c@iA*GWrVVpKQ z-eQbPX(o0tO~(C|-Mw5no%_JpZ?ajszeL-0aZxWW8TA?dF0%F8pfh`<>c}7M zyj<&R0uK*=wptki97!lhqOP-XSqaT~L} zbkj2K2$l?`!8NC;Ip>sn%5PI>4{CIx;anLc@?3=-D+)+(|A`LR_cZ=A~Bln{{ zjHsj{xAgU=${kK7yDA^f=>o>--70gorj}7Y3~e*=gtS5y!n~(MZC?A>YBgsKc{n)_Ik=<)U%vh zI(9O6Nzue?(p#!bBTsf%7g^y&?YLp|({iH^D%>>oIm*8^`%N1NyoPuCBdIVjNu0a@ zFcE6DIYU>FzRnhvOdF?K<@e0%9p7GSs)iZ^DLg}qkyL&Af^U*5$qsvI!T=GRk*vX^ z9DV<2U&B789D(LA>6PZ)d1Q%Xh+n@vqD4e?e{2@O8$upS-Lk(o7)x(>tfx(RIWnp> z$wne*za+3A`v=Q$3b%kY^9Ie;6`4zO5yuMaTBi&1-}JhYy?do$ZH;ZXBL-k8{BjB& zoTDotc+ePU5(xXUVp;jqt^YYbseYDV8cP>c+3RaISsP+FozmCmV5gX)=9IqaiSCI> zjVW!>Q4=Aa#DL)amiK=29{lk$d9Hk!{_75gX}HASNC273S*2LO_)_%^bqYUoh)M|m zGv4UJsHiJi0=?aFs>lbg!EjTm!%2Z=zvf_F{xNaT`Fe$k{N5&@ZBbHqVP&Djj`+RH zSF!yVr&o{5Wb*_@fXton4^|s4AXVHEw@TfQpYx2h9q)iYFf<*E0*jLPnD$o0n z0VJ-fAp)VehjAWvb$0Ru$EJP8U`ek~xiGtplIW4Xz4OU2fL@5-``TV9VD{!5QRGm8M~q=+x5A6y2FDL=W_}5zFJ=x zv*$C-+TNH9-kUEOKJ%N1ud^kD$?VlL#R&kAXOa4oA~L406f3xUj&{EAOO)e+NF2ed zOvo@v4+Va;3s(tU>Ut%GEs!wpDy6=f`X9n;K+q*m9GQlL=cynWnI+$ujD&4EizCl%<`~g)|NU2fWvby04mI+| z44XkZJAL44rlauKQvcrO0y@`0FD^wsp$BI<>k&dX8Oj-DPl4Vgo&SbsI!@1mx`?ct z!6d*A_6X%omQs4czdv0h7ArxsqMpez@U*v>aER+_q*?o6J&H@edhz6#<3rA^qc5Yna=haKM>GP5Fx51RhvT|}81eJc21f&DDX6nO4krXA z4V%*ZIb5RMi4n(rQx8q+xJ-Nx%4hUE$=ndIpK5&u_0DA4O~uDmI! zWPM9kTTB;EBvA%fqVRecwKA54Q4uHg*h()g8cWP2@LzeMeWO}U&bB6T#El)PRZ>To zO5R>s*`5M0A#$gkv8HnQhv{ilA!G*itCv$&-eJJw?h!r@Gcpuq?#t+{AGaY zCKa(;WiS&0*$C9Ch8gD1*lf|aI6cs+r>38`JVN*S#nq|Hc>75O)K!8Guz}3(uWeoSOBHsyyKmMr zHJN&aGeUgQ+yic+SzX=p^?E3V8?KZd5NLYVgZ@Je0e7P3>5*nB9 z`D^awS`A572h_3owC3vyzAHn^gwTptV~Nb>^L);UwD;JDoxd6aS^UQUG?C5R-0haR z?!08Cw4aB)JM}H8ld1Af(P~VdEbRlu4qr_GjDYH(XHwAjX8T1x1vqj-j8g+_H*ZO@z5BFBjqoo;aqigDreloH|{iheH8przjiRI zcrLY-Gb(y;h=Knj7C}qWbU8TafoN0~9%y6oxZXyW`k-)7wjVIYdcUb*(FHlMV5sh}a13mtrO1e(oDZ;>4brpy5?M>SePi;ZgE9=bX$aZew`h4+~!ayt(=D0wS*0XqTtbxQvHt$_v~72n|_15IgLhb`$zl=NGVe zVT&xzY2f9txNf$&^|R(OKv-Ys5=uo_(mH43PPBX5A6j!ACWiCUkLV`}^VO?GX|LjT z#(B_Wv>#UI&^aQ`^wL5bV!r_0+gpNG=r01=(Rb$Izu4)l<`m*c7CBrHTVh4EarCdr zJH)r%Moo}ay1|yz*n$w)+~KJ@zQQF0P78Ef<~&DzbTe&ZTPzWA1ZmPlM6EP8r0OvJ zo-A1DZ;3{^RgMwf_MhLe)DC<5{c_EC;lpXshpreEY2CfE{^f=Wt{!4o8=q0DyjmY1 z_e$M!dq$TL-7N9rb43Nlqmmw#z>wGahNnXllETi}|HA^9ScfB!WyDYNT(0_SjYqg; z7Q@6=d_!s3scYGW9p^)?o0RW8z-Dn1=&`b*_F@iy5%(YZgt-5)iJh@wicmhs!|Ae~ zko*pB*hI(QW;Zhj72kH}_!Q^^x(!8|gE#|vhTiTfR{kezXQRv-Lv2es32l3|4a*y( zCp+<{f;kikB{Um`wCxTa^{W0%=eOZbP`t+ny=+$p(m$lyEj7DFReTS4aOaC`iv=%> zHs^R!{CsSF*H1YJ8uNJ3kzm{-;>d=zmdhc?)-2fSy&k{&N6=avqVI42wmC~{)84`P zx`nFW(_x}B5+9G|wg=^Ly{3U z9>qy&ASinzt_7j#;wwK$j!pys;q6Ht)TepQ4WCdz8i*I5kwnkFg3PNC15gGKsQzA?QP{H(xqPUA29 zXxsICfQY6Ucvb~IGZDiFCVo!7(IPP~tFM9bZ{5olkg>6De~`Ec1~VU-rHQG(;|PNh2$nI#A2R8mvXcrSvd z00#Bk-`#1eWA8N`M9KW&X}=>v5$ygeH@<5CuSEl5PUbxBCOHw4eDr53-`=kps~L5M z3OgF{P+GWL_z=71jo+O=k~tT19C6sN{=EvF`p5ds7Xjnhwe}F10|w!D?{&H{$*TcQ zu=z8u&~H$KieX8B&~jfw5*oYv%a!`#jc>+VmBe@vaIPDF_g!7#_l>px)KxR~Ia!3! z+8MS&&S+<2vfh|(15JAMcyqfhh9*A>CapOqzlzT%54G5Ny=%JQZGY(i&UVo~uj&Jy z)eld~I;OqyWm75+@wq(^nwG(Qb@1z8z>>5pQ6=a~yAAUZ1il?QbqQhu(Gsc!3}xAv zhI-Ss5%W&b(Sym?}LQcim}f5zxbSPx*;k$|QrHh2SVrco<*>0vgYoziz{S)2wPBOM4DUNW|d+N|nM zOtExfFbW`|YLji!sdeM<`zU?n8S;Y#dSw83S7kRcigZBw?Q(urn^|DlYTZcZ6(TN5 zLRH-=pGSxc)vAnn1T!Sf^ux4ucVC;LC>cU{E(7*LP{$#@}|sBR<8l z8)6l)LHeFph~KfZ^cex=J^LXges%1jj=Ldp1hhb65%V%Bq3!p5$iRYAI*yOO+ywdn zYbS6&SLXmh?>e88(hPhGXd)FrZQqcm$07Bou)3X!J8CulAsq&bh+D6j*@FN&0=YsEN$ircv)?+8#POZn9!BR(FM`GIG;)DoBxB#9Tl^#W@CU{kX*LA;hBNLp z(By3@%cS0E38mcU*m^+M#A|&nmzfxz`0~DB;lRm__g}peSj$ zRFA|jfF(w@@Rq+ZiMd7xZhlDe#r)jdD~)ewJYX_KqYJ_n1%60zG^|~n64AhD%Z{)} zu>&FK64M+0`M0W{qK;eKDuB?Bq-$VNhS-_VHH~xrMAa(&Mcriwb55ZF} zR4y3ea~0Q>J}E~WOG5)MA~X*Y%g>hE`GExMGJ?%*-+sMDC$ST zE@GZ^f!kZ1k|L_VTaKixL%*v&{{C12x-VGUc%hKaL5OC~rMRm1GmXJ+*Gvc6ZT2t0 z&y_9OM{SS?4R{+^35#T6g9?DzN2fH+`Qdf@rKEAys4o=sFSVq*uAaDU?}{|K*jsNMe?h$&{~D&wLX;%T z5u*4(#_Vd zp#Baxo&hx7nRC@)hLivwo!bwr3+>*1>up2;E55M|ce6{FcNsm$>LnGN*Rp$oU2Wuw0NZ{#A;-++VjKLd@f^vKyI~YSht$u)|&L6{2>>h=t1v*#eaUB7$^7*v2Q*4i3k9=YUXfIWr~cxMwor}}TG z>=53*kyj$v@uk7Ls_Z#}q0{o(Xi9}pwh~>%KBejJWzhHBi&6iZ9&5A0d&gEIn_rlp zn&hC3wOVd^SE!$3j-0?Y;WKefBbGU(_l>Y{FbUqL!*&1xWImm zFtxr`3K4xU8x#(nPb^Avd3EUh)~pimz;U=0@O8hrHKd3wyX~gZftyF#vpn!QVk`R#nQ-c@z zyprIJTPS-or*;`m@rpn_=Awx?y&Bm%;6dDRZ+OORhm#ByhrhV`OTU-7>@NbY&qjw` zA*j_Y@$n`ZkGx1IKsZwPTEfL&cp%2(6z_t&{x2OnPUnP3-hOP{g#l>~&Cj+}oJdY8 zY10SkxcRe;PwnVQgnrXk&5BD2lQU4SO+t-XPKfmC3V2fnS{!X**0x>qauqrqemX`9 zl$N~4nsrxLeVuly;Pjp4Dy~>LW9rarq%o?c-q2rt0zoYN;?EBfaQ1h%AXwUz` z9(V8jbI)+xhL!i6E*u_9MmAZkp!}lnZ5z5)U8_DPbHex^93B0W4BeF0nJZI zo;g$<`Gp5I4`*MB2qO(Ns01A?)g%rqJz!68fyzp|EIlFQPv0~)h#@G^@Vhvlq7(2e zTpZ15Lb*V+Dozv`oy%WKyU5#3`;B@lmU&$HPzh5Z+@=(Y&kVCDpB$lrD5A%)us^PQ z+Nh-f;hltQdds_emTv@{;PyiaFj50eV5%Hg)cenDA=K>b0ND@};YNGo%lVK&{VO46v$H8O)fN8=< zzzDW2fFm497I69)CRg|@dIdDm8pgz;&h+PiKgNDAptRN5&gqA&lY@B@fFSA7`QRf0>ubkE-+_MyGC<;hE5LmVz~*= zhor8LSMXn^S8jR7{n3m>&UyIPH|9UYN~MngR){vjF&2_mCaQ(Bs1n^6Beoo~!l)dV zHn+B_!|V;XU9HVPS;>6ccf*BpTN~}FFzF}N*h8*KyK)57_Dej@V&k(Y+I+S*vvWZA zC%*`~L#gG9(`ky9mWPb(e-@ilJjIrUFrjeSh})}3I18#7w^5V6bF^M{pA^-8YDrn? z-cm7&GgNIz%HB#;okWTuXW&A#zU%Bq40 zh6qVJ$2kL_z~~8U2h3r9z)gz`-LCFV0KRiw>7uaskCVnHIwfp^`?OZok2oJlhZjh6 zh_)%%hc@2(YyY-BPxA7A=+4YbrUnKxY%Yn8V9n5qJb1wU&YPD@nJ=NO-GwtdKil-} zkecrhEdt2PYxNo_Y1U)r*9IN3(FKY%GDpBu^7%1{N(&&F;KO^=yKUX3WQnN$miNqg zgT=Rh^Vb3Tzrif-^oB=-NWjT|&DNMAmmD4^%O=B_Z{}^X5#6lWxTi~II6S!VDfb8T z?c5?+YfW~qT_YHb0xMqbc5DnW7Lbk@_{+PYPV$|MH`X8FhAzQ%7e6FV? ziONO+8E@5KwFFM@M0S-A7M}=dr6IR5(m&P;p~hka6sBGi-5CPfqbGCOe58o1N&Ugj z!!g)fxc(Bt#vPXRPbZQ21xoh8U!q2N@RRflOc`))xK!oXUl$08W8FDywAGWei;F}K zSZcw}dcMbW8rm>Vk0|&zw3+%^62t4;Q z>N`XPz(0n0BYf>ezn1xb(MD}WF3X40$HljYOs|vE^`=_`nVaVg%J~!ee3&s1d2z>`ygtoe@J9p7O|9Cn$N@k@& zPgZu*9(3z6D~>254HMBDD)fJ9IN&OO-?sCqi0;aCPB2-ViO!kxGyyRo1da&RPC@Fm zX82R;cxXGps}8GqGL&}KHXK7vN)oocC-|?$eVwnb*M`hw`q9b|OaOPg8Cctc3fD)Dj~-n1_QY) z;7TA%%#S4Hf;uuyy%M$LL|gH|!PoO3-W(vST4q@P;iz|@*q5fl+TIPx+#jg%%T=1n z+;ddI4xG0_qM=%a*BN*Dbf?Qqg|PZ+%1Z5hCZWT^S=jB-^zhoZ&eghWmqK`WEP0T& zNBbAtQZ#JB7M$D@ZYcSP9@0x|qslA>xr=_Q;twV|GiuNOh9xM6D-pAedR!xHJ2MeY zF&JAhJDytoXz_Ke)usk?>+&S$3lS7m`%j_UbG@kLTrTHTEkIwi2X%`4qq6Du@9NcI zhdG4}>1Z%oG3|;}H70pMxfM&wA)Y^W8q(W-#Ccz@g_75@1A}tI zzNw^bLoW@ZQe43{T#ZBHC=R_$@G@2X9ML;fR<5maj}6bvc} z@$3r{Dnb7ruJp*ar+~L#u7?>=$eT0K z69(UFxwZt$WV`LD5W&&IU|#1N;hrbDD01rj&TbvqiQ}O-wxtl~fk*4nzCr7k1R7SC zF7Y-Ic8gyYXZEl0q942sCJi4GybXAM3fjtg*=D26!?`&7bQr}TyO$vydHKNS)i~;p zM9rW0wn6+NG&#a!>1zpsdX@ZE^@0Cv?X}Zw?ccWj&WSD?@PE45>mfPSBQ*gP4-Vp8 zTXnz3Y{K}VsBi?8rQuQ#&cGLcd+LOdX#niD>#$+D(}4Z2lH0QHKsq?pxGgY z{Rel(cmA(6Mj=4CALa{z#I*uS&&%zCfA1baiGb|8_S=a$((#G2tj4RkD0R?L*5r!9 zWuLxIE%Jl&&I+Q((+UAf2oLt)K?oS`yr0+6HmZkv`IpNrb2Ae^_7+S7&_SbE2z?NL zRL2-PURy*N*260VddEmn%m-HYazFiTTxD&)h`fk6;Aig{{mI1@N~Sl%TJY*>qZHl$t5pRS0Nrt!0(8o(SkNG&!H8_y}d zRzEuR>0V^xOK5+iU|akqy}GzC2$P^4qwq8FzhgbKSy;8s6yI4RnG^RbrzOd*%(&6~)Pp)KIBW&bP{=c$QQn0ao z{wi{!zoF_-VqR9ltZ?SW!V}G2__f&kIy>a3V{xA9qG8}TJ?AFWaH z9i%{1@qC&sm->Nx2a`#a@i|hL%c+ z2q9q`)FzRL5-OVZ57;)>oq{dg5L;&&cBoB_6`$r716p*a#&SZ5+<3-4on__VmsqUO zlu1)A=_{LN72?n{)0udESs+$8KM0>Vd2b1OgYFaY&eiNaxzV(SKPx|;-i}5{q`0{E zcx1C}Jf+=UndA|Fsq!1!^MlUZ6{C#wp$)ZEL>AG@8o&+d_k@@Pd48M;=^uZsHHn*< zh)9Sz6;!WR!Lc0}c2XR4oxMaa&U>U@3HK|S|C2mZA*7}!03Nuu>DIfG&bK&GeK|e7 zkai`J(utZ>>FCGsbCsLB4UKqBY$0yrSI7j z;m?k6zra3S{-Hn4P4;XOfA8R2WugS%P&`G{vW=DZniXal!!5toM%+7;j{Shh%zT`e z`0dNS00_CD-a#Zpp2lJY4clg>ZgGxEDNgchBi+p;jwpkS^aktjHJ$-wcJH&`sdv-z z1@#LI-GtKhg=TSAe&4zC096Ap-;yd-IL;KEh{UyPl)%@Ig19_sH|GAe$ZOoe4{y2M zt5=>T_i)@@N zHD0#GA<*Yj5tJXxNrSHFpTvi)OUE`_etG-ZhFhF>{zjNyjOULa2 zxK%0mxD}I#Wq{YoL@!~hw8mGX_oFh!yG#)%@b-xjlfd+cqa6j^<0z?@hG@g)h%)7s z61Y&c;dro|v&tHYHQhRVS_GtGDRuU%{_cp_pJ0HZq&q+rZe*$9p=+y!qq92BHV$w; ztG%g89ZTonO>M39D*}==%H5JBaebziu2WBA!xxfSU{nIml_l5n$p)5jufuR8uAn?L z!U>2|_z>3%C*nh4CGH8Fa&w~;Ar_VjvN!dTfwSd*BYKu!ft51&Rgn%Ue)v`IW&gkf z1!7zAuL-Z_$j5<^yMq$pMl1E=RQ1TELUJw13AdaK`B!N&?WdrYW&t5?U%!AgyYkT@ zYE5Rkld>DaThIdB)gEr!j3*AIVF&9>CG7y@278DX zu*pL)eY}6@IJ!5;54+(H@;L=tG*kOB*5xNZHH6$qSV61b^e^qr{3rqbCGwad7L$<; z5Bw+A1I=%ecuEAIKI6|MiBSav-Q28Xf&!^{nhFOPJ>HuyReix8yFUvj2wz|R?Qr0m zANBOX3pyGob-2NLToYC9RM{3Jkv#R(picH1kj_pD-XgATIO8t;CH6v~p2h4=6V# zXkcg;He0Fh3s8KvI;Hd&dec?4#F@~YGV(Nv1$7zl{zzoelpeoFj5z0V$n9Xax`Bo+ zy`Wz@4L;9zd{+Jf6zX3M;cC>5O46WF-yWSJ6owBdDDM+n|3&0o8*h2Z14C>|oL`?< zYY*g1@MmO(6|kDQcVpBwT?_b{Vuw8a;6-6*9AHpv$hZ;?uJb;Xp4nKKPeM3m8YirW zR@3w1?TJ@m$C+^XOkqtI$;D(d5_orEZFo}kdi^?aLQWPHXS}xw-(gGF25vdth*Hjb zAosbtzbh#%%T7M7pee}CD?@4%`X#BHM7i}U%jmDGzW(N6G?GLoW75JxlQ%d!sRd>G zURQX$&z)HSM<(D{q-+@aI*{<0 zLJ*%EEdv*t3Aj}SO-gim%QU=+WXkS(Yb!2{7xhLRS5zyOqy@V|Ix~}1d7a)*ER1paqj zv9F&ZgO@mG0Eam4Rq32R6giMRVj_8>@7Y9QSTPU;7{T;~0;h`!&xzgl)#SDH_TDya zOQn#O+t>1&5U z@!e6@w~1W?+1+1~1Xh`&9o)!5Y29m8`;6R)eCqR0S!)=9vI!nO?nive89YBsRrAx+ z506mk(mF|oDuW!zzLJjpAg}2f+xa^7GxAkU*4)m$l3?(3ebP{Ea2qJFsQ{;L)S?Jn zX~u%6JQre&5N@!@?nn&-Bh*{>U60=3XNTXOi=|(%XehQ2^GMLMD~ZrJfKvOZ}O|=wB{*P^A>A{u^B0cSo+b9 z4ugE9;DXnT(zl}~^xY$E7XBG9JJp-I+XbTe4ZzC>fU2 zKd_BWnDVVz#E-YSeWI`S-@cM)XD0oi<6dex2o|(ol>L^|K-qR>d$#s)jPND@85xjk z;<{I?xUu7ig8%oKAP4!-X`>{i9IOmNxf4^gz&t`gMcL-D>(CjTm1 zXw`sbOZ@OeznTuR}82f#|UvzpC6V;%*C z1al7P?qyAcxwf;`i%7_O{vuOVeTml8s->%y)rEoA5OuZ%M7uX)@X|bchy9)wWXdWp zl{)xIn3!5T)%{KPg0@pU^@@;W=wbrq1K_8|aHU)ES6GJcuj*u!bu`<$HNIY0Iq^Rc zLFa{WxR!depu6ch60YtC))n~O1cJhgXd*cv@t9gYvh;xv{)rveXRyQ%L=un+{GYSe zt@~fhzI=^Rw9Y9A?Kcha8BnY+K^r9B3E{cdF=Rq-!JRQ}wg`d#Z%^A!+il1q@uTze zLx5qV4WMPHuN<^cRrJ642;Zb>{{Jk0xyvw33(FC(AaNc0L&NWC5!_b;{r_)p{tq4x z`U?S8d@{cxPr65)YRiO%sM>(+44MU}nPI`vNG^C@)a83`{|os4!9D*Mkw!zbYE9oo zw)7)XU`gjl^9x~ z4~8K5PlxiJ8!-&nbgw@nC5vn{t{~w`r0jM-kNtLEcRTv|9Zg>(`yZXonj7;e(0dlB z%I;4A88ltbZxn|yzX07K9%7v!@oz8P>Kmg^Jhog;zxN^dX|5}32|#40nL-DVQ@iez zQ-C>>D14A0l7+B$Vz(KV9MZVW_5x=RgsBF5?Z$GK+^`6aUPopsCUl2{uZ${1Th>m~ zJwKc~7znvIPklTxBBpKPdtruR9=^tnynvL4nHt7cTa$Yx?Hy2gqEaEt*Ch{l;<3E>{pkz+=cuoMR%R&2C(e|+ zQ`6b}Z5>Rmby)kiIcyNS*$mz$vT%-XIHrFFS_SU&&R-k8!VXr<@YYFu6CybfUqe|r z%^%`k{L%Q6{Sz=IY+#~ayVTEb+uZa|y7loa4xKYHx1OG;c@39^M=HAyas8*aGzs#mQ8Aa2i(TcGw49$FN z0x?l@-2x8&QZL?U8P<{C!66wIkhD$X*Z zin6#KuV$}ta0K>BaTvT-ib&j{O33YRZMXFe|NStF+D(Ew@p$@@@MJM&>9OwA8Y{U` zTnQ_A=k&cf_5RUsm0=5xMfUa;a`V9iLRER-(p4&GRWXxq+M$78p9;;lRZIX%(11Ob z8CKz%Ejx3xgSf|U@SFe)CyofAb|-YBY0I8%7(h232?n<=pI_ECHf=7dwZJv!VCbcA zkAhDLfYB%6-0p81nlG-jKnchdiuPYGuB??QcyXCX# zhwTol-zT%hky-p!F+S*i;JKpDk3GH152?>+rYMo#m($O599PD^g&)#k9?|8*<9P){ zvkbmmYXSQX>h5zD+S5x5EilC+8y3->;c2B*yE=b_(@nb0?yrERULM1irZ-}i6;Ia3 z1mavY`yk$5_ zhidSe0}&a{L4|yP>%`Dk&tEkkH z;k>MqwH?-A>lv7OXOmor_^tcRkI>0*-Pw8d50m@7w6+*&)aemInjfU!>3Vjw=wuqf zU%W+7Po_CiKsf6rct4CDbnrpP`K>G>YFX8tO%l~e`_vxc2F4Tc{fpImuB^!nB`^4f^WI(a3|v42 zk2Q=L0Dt%87>LD)M@r0dor!g=xAMIGZ_!~Vr-*JqQQm5BDP|v%&dsv|%3>x3R50Wi-@V52X?S{{>c0*}Xt=2p$>hu1&OL-+ne> z@Myl0J*Rm=61WeI>FeW}a@DGNn>?WMMb|KfreXa)Q7TF}P+Rt@mJih*+Aa#YUpAR( zbF&aZB=idq{y~&%p}IZ!(i44zdx?ZsFR3BJm-^}Bn=piL;RpRTSHAAe^l&$-XE;JO zwlQep85H6G-`m-S{$_rx=7qTe^qka>2!O22t_71%;MnwfK#qZmv61S@E=+yadUwV( z9Q;1Xk{Ec0BY8SI(~;v)E2WZxQ8OeqKvzVvmniAlFSjQ<`nky)Ex~8&BitW0cr?@W zMccN1nD%0j L664Hn-PO1Y9g?A)&;{%>C)FgJ9DPA57XFC196Wdsa;hdYa?QJ)i z@=M1%93luTS$G$=F$+zO&h%sz-xQrFm5Fe5+P~?Fd0mR6QYYAcfPMZ_)7go^G5IO& z=a|gg-VPi94vD7tXHrTG1y2{;hwBqLJ@X9I8^T~>5lRvMS8`b znJ*{Oa0O}-c3SfU#oyR6FCL`1+FLEau*(_rSOOrd67hPACpzcoCwAr);s^$j-h`?0aUOCrM`!ioWXLZei04K-K`YT#3M>ls(W+_8$yz3F$ zJm-afNje;1#2-=_5Q9}qfLoK^LDVG=rF}!S%pph3gI1HdsjEfJ`?u`d}d)~2nwZTuUlUHy-Jjjf; zin$jnxAXCraEb1RxegHd(!Qdlvq}xhSS=V>I+zDB$e~Oze{&)@#oucga2U<;Op&< zEU~WXeh?Gct0>;R>cB=g4Kx#!$--zo3j`~FKNdnUWA zHGAzxe!&tK#o^|9!(n6E1q9u8L9W@O$RM@0CLjh z_2>ZXt&<2>lP&kkDN9_l#%e) zJ>OUKYG$|s=hqLoxI6J}dux8yap=5;(HItK$J;v-0)4;zE=jOtsswLzdnWDfD-DaZ zz7%jB;#`MR=`z+XHocJ=8pqJDrxa;ljnlm<(Z?10W?*0)d8+dIZC7S9j>by2`SnYN zU8P>keTAQ5_-~rn2O;=I$ho(bTHuD^Ve%byN9=Vi`$$a2)EDx4A@Y-12AY%Y)5cF3 z7W*ljU&)ktxCmgxYJE_(ey;YRG|Wq;z$mwgMl!Q1lOO&OjnA#s_5!HUhS#~73xO)j zac^2K7>F-Ub&Z%ZybS(=e1-W$;CK6B=f$Ht;E0juPO;nYiG6%=&ZMI0>YHuxFV*6> z>3r(TX7&EzdE%6j9Y9z8`}egnduk3_1A@ENmM)@}ZPIvmsh^tA!qaDZGk``V{QtU%57op8gPa0tY_lSsyNn($_vKGK7 z1xn>NiSSL=igVY)0lAaQoKGL-HoBxO8?vs@pq(Vi&}FnbNBFbL}d@ ztMEH_Zr;3pos^W2@aq5jYd`fXE~rGLe-pja-HK5i7Vz|2+G>CHM7Q!QvG4;QA!NIcK{c`7XQ;s1vV|jiMXP-u|zs|Fi%7cfGp7L}+*9 zaI7U{*yN3SwoGJ~$09e9=D=Co@|1Cx~c9_*rJ%6 zmbQvWkh4L+QruRBaM*seG1g&vokX1V6$ym4gSOrKZfV9*mXb8T^-Y zY4?}2Wr|T1X_+O!Pd@2n#tTrt6R?F}Y&|0IIueub9oOcF=gpITLf(}(A(RNS#V($p z>w=HABpI1fJ(t^($mAKG(|NH!UxN4IGL<>jZ(KmzQXn7!K#X84DQ4Zz|4Bw=!T{lIH>gIGerMu#_l1(d_tQx|LFFE4hp?@w<-n(L;gU=PH6D_ zYYJ1~ZM#3@7-HF(|CA5#9BEomDm?br&5HZP%|Y;pY*;A&IIQe(?IC2hsPT&f8J zhDzhBomOKx)h|+O)z`m%f~2c^O(;}vO4kqhWpdtC=AlhpG6Gn{ux8}-ALih@Rz&BV zQV4eHW_w44+Q;8x;@RmXUh-4RTXTe0NpUrj$@4G zM2|mEEX1ee_f%^CBRHwfuxmtx-1hMw${kd9$t^C2{fDRRiFU5PGgaDraygk|XWGf# z&rSaKLPqzS4~$FvTKs#)Oen!{6&a)O*r)%q&*?As4f~n+fwZs!LXn5H;afkPo4;O% z6#~W~_hCLt`I>ELgfmh@0c50n@7nC)wff~ueJJ%sQBs7^F@#D3%?X=w$nCp9M9lRE zC7{Bp^QPbZk+@!?{0HvDuX+xWlbeIXee9jLF!DDqx}0xvI8eEercLXK;z8zzm_YoQ z+)|P>x;K_)mWmE_W=oBBiug+6n7NU1P5hxFctW#{ir1_92(f_-FxHp6 zrg0pww0>9@q;EC(7~=WN#y3qqDqcy13_WN``Zd+Z>4Rw;gp2McU(g$ed{iEJ@0iPt zng_RZ8G3>k1ygtP2!QLbFrAx%-5Vl}*6GZ5Vq~IdAH~b1)k}sRE}uOGk$WP*XsZu* zkqLKvp2I0a>(*+N8hs%yG;5L#@OPc<8WMz4?w`(+cp&xzmec7Z<@~Dor0^P-?W8w; zUTd-a*KK$Xcjt00uh4FC!iu(CZ^So*3nNabQB-3TevjD1SC4t?|01@MfmLxVXBPUQDgV$s^sraPcL z`rXp3e1PjnIF^IsR$DJlex>%d-L2pqVEzpF`sl^%QUE3G0fbIp@e9!MC5=b256%UP zid}k5K8qtB;-)~kL14&pk)u5k2ya+=NkY?)~b60|G&9SG_kR(IP*Oy#&ctMi<7JyOWr ze8lP5^8iPdd(TXD0$k~`~S z>)^JyA9A_4evRQj-$(lCU-=cl=04h9%kvL_DxmHxQN9vfC{N0R4*pqG_zsg4CuYgM zeXH~c!ZI$GXD;nro~)l0cZ=uqN=9#n$o-cFWBr6u@1*NR!7L0S;5-EYv%3nRh)$yl zAAaVG-(xDS0Ln3UAM%R)KidTRsR&*)h~>k8x<_9?l$TIZnUUv|nz3rH5lPdRdIz$; zx!}`ha`=o}iX?%zz!6t)G^{k+T>6Kn;KEL*bhfj7j(Z-xO@^9-kHN}A)3i+I_wLo4 z(mTytmRlk%G9u)7imSk%&Ef8^Z+UI||7?ySU@*&e4>8#sF5xMY_Bs609P-A0F)@s< z{BSr#*s@X!Xm5HCE|5#Q%}HWa-P9=k!q}>~`fc%%J@Vgyuk2*~GIjG?BLxdAH#q*> zP`>cj$+mhs(j_eZBlG(Am2MXFht)SneqRXz{D-2RvNbfcGlVEF@SxSG;yFLF+_7Rj zoAk%u&o2j8-Aq^ad_UeKB!a>`NNPXS2n}JKEtHy0E+SG<*;P?x*8%EV?;cD8FOL;& zs9m5m`^44^jR$R`K1O3qKiL=eX!qTFSIVrQ!f*#zY;=)s^q3|u)rXgg6HAX5Lke~TMl9F83qhn6M8+MF?D?74lWW#6#8Thc9HGW6=xBX>&;!|&5POH zi#KZau!&jD)3~gM%Oj;1^5UOQ2*q|o?A{7s zEUF$d-0Q%zb|RbTM%cVK-R2vG3>Tn1Q4?eMUz7MH@g?{3KKHs?({>LlUwC?)0j2u6 zT_U|ub*CBAunUQu3ld=BIJy~kKmBXei}vY_(hsH}#5_`FsatP(KV4x7RuS5ZgBRIq zAHZgx_);wnSGLHy&c4#i5FUD39r1dEu=)N1)Kb|i=!e0io-UQR8BVHxuk@2J%X;q4)8(=Z zwA{W(uf@>7HW}~}dq0eegNl;GXPXlG{E&T+C7^tHib_OcLn6(_D^X|XeUy%4{*$U& zEht@dj#7G-gfWgFyiWij>mz^Vy`tWx{{3uYx&nmEc_fVND!~)^NaBZ$!LbmWWe8YI z`gFZ79<}pc-G`<1=jsjnxH0@(4>z7A5_h@`)`a2}8Q7FiN6Pq;Slu-eptS0CA?j;8 zy6ocywCFCG!dafLVnKzK)3ayly)o_4m+)lHWUkJWBQkGw@JxDxSY>C2cF>Sn_2O;I zcEQV{43m}6Z5yLP4I&bV+=e)x93R;z6nSoX(qt2ie<&sBbWpE!&mEvb50c&#F9dYG zKFnf{7D1CXl1BAwG0$i77T~G!<_0eFGS*j58%u&bT=@%aRuH_!5?c0|sk%r>1&Bd& zwI@+}mjXKq>WWD~Hv7^odu6q^CG5hPA_1QrN`l0aMEs?WW3QeoGb{5DaPe(UM8bM{ zi5_oger>f=Po54k>3BDgvEXchmPnyN(W{*^ljN2my8*1WwQrj%{!7tB6P^gnj5%5D z2=>LiSeO6!{DhUvkK<5=Dn{^Mu>j;e0>tn=82SO|flcrW;8!rlR;B4}nQBgR_&0{C zoh#I6rghKRbk~2I63UQ{*u#UQ{&OY*l>?Urd^*Px+pSeUpHSU<+b!r$(C6{<^WeavzX8Xx#FJ`gF);`!PipRjcf zn#=NEMDN@tH7+xjxPx__DkrP!P+>oStr29Yg}#{BdmykBcB$Oj(em++UpWM-Pp}SU z$8k**lejI`6NO!#J26la0Fw9xTW;;C(?NI#0+_0^pczry6YWH-1Wk6$g~}b`L}-xs z`=7c-6;?t^y*6OGLS;UXokf}rk)gNLXs-1X*3$A5NF6h zgTpp_Vb&|dSis*-Fbp2%t8^D@-k)K+!zMK>x1&=5#H(&LHjX64m%4RFJVDi|e`p&2 zYRkjSx4b#GWR4@DtaF?wi;*F5?LMui61$czv zx-K+()v;%@5jQ@P&P$Yg+l>1%aQCmdZZ>fhHJbcd3SWqiyCdERTkQ-5U&V}#o~xV` zQ^xQ}3=L+R8sF*-h{T~iU+%9~Sx9{S_ALU;9sDE&pBfvob&Q=xVV`{>#Tl;O<+b(v zO{X=l0;5fSv8z*!lj(!AuG5mR`4$3?L??=Y<1K%_A9K+=nbMi+Vt<`aCt3+?w}VJkPL54cAio>`LC640 zM*hzXvv29)I&<-*GlATA4M5sfqmN2{h19(2VKiGUf>V)0m(D2lg<_mBx$*ty-&J1q zDtwsx|Jjcm`oa%(ZYCwyKYx(=?8s50nNTA!kw+@E-0@7h`pFkoIpmfU$j1v-a?1DY zj2A!smhQqF-C}TU#b{f?n$&muyE=~1x~r3?(Y-;?A-yX(nI@eo(DTZpq})4#d1j>? zwR)Li*BH>=15>h6YkLg1IumhBAr5Tg3o(1F#?Ys1tCiIZi?M0 zNyvB#4|VzE6@Hr;)?zI(HfdX7Qu5wmDAlM`rj%dr6X3Hs+#i{CRFT#u>2F{sKrR=+ z9Gl(~@^()xcYmQ&`V1-|@9{|kxXXRuEbjxOgs&IP+79Ut|`UBEkMzq`eg9|)DaW=Ei(3dY0^$3>)W4R z9)4jPQ7c32SJK@~)iN>tc>V9%Yl-yVTKRVi{eP=l|E`DpzY3EFL*MFDO6R=1dX!+g zdb79?+rO${9F;Bgv}%uGb_1kn1Z!II1~U z&(9u-jj4?;1`E(XUV1DVm&!DDobhWn5E#T-1O+l-SbWmf1iLE{szY#n*HGccF-98? zhj}n3hHOw-w!aYj4wZxlzH(b^5(CLVId-RNC`h<`B;{F_ZCs}-;~N)KX+@uOL_Lq7 zXUazH)j@hm4q;bJ%M7C#h9BP1%6kKDcEaLetBfN%lT56#ZdYVo$G61`ab#0|&Hy|N zOOVS80=OSjxGi*$Irq4#8pQp`lDa0h-nX~#+4X5}4An=Fs>Nd)wD`L9LNuGrhaX}} z4}9j@{MiiNe7Mto zw2V!I@l{!V;xTM9c9E^L*r0HmtfX>yC;EzBqQ>^{%b7rm(dWnhSc1hYiC7E@TZR%8 z#jG`rhq8>@>OJ6NFwNgh%=@5;^?h0)9Au&MJy`{a7aOcdhSDoozB|hRA0D?&oyhB|W-%$JmfW=*`1G1^<+1ix}SYV>)`ZJgfWHC@!P5a1MgU`Y_gbetKSsn#HooXLk7b)5%Y%F+ zq8Mrp^3cn?c+b|1SI^C>dc%)KHkx%Ge|CxYPtGUurK{AqCro{kf`6Pk zkYw4lPSv~TGqJ<7XWjVA!$UdMNjm^;Gx`^QnUswWw%r`1-?oMv4^T0P%dYy;;n%bv%e;jeCs)m~|A$yTem5!B>fUH!(% zZu=5)em#W<3E#be6>Jifbqi~R*$%=Mrd-BK^<6_=cXW_LfAJvCwNsd;+IMa~eG_m1 zX;sA6Q=FNFVQKmPs`1_U0?_Mt6t~u}{ z==fBaC0lf_=F2kwa0-qdZYJI0G_m@`+|oN>GVtw2_0S*X z*IP(|7NgFP-~tdBJ5w~Q34$+4K(Mwn@ZlC=fWgP-)50OVE<_*%5OXJfS0rfd)#Z6o z#U#k6OibNDa2sS7d=krsP(DPo^Z|YG^tJ*91tZzY6a&ejB%m9PT*RWLkLziM7xFjj z7$qcgxDU5rsDiwU{asI;E@43}7>i00XAc%OuT1i3-M-6t6>o4r-5tv}JttXX+Wml{S!D=40gD0}vTOqDuaUB6p>3uVIIEqw51 zoT{;4tFO=Leyh#d?Kw2LUA=OL>j3z5f_?p?reGU)YVp?dFjq?oFLmPwZ2^Sb^Ad=I z@daVy1y9DrPTrSh5Wh1UO4vJUCO=gP!JAU&HLQud;E@oowL4k2-C0!t?CTak4VS|l zjmy0Pq|AM0z+vf)S$`#=K@Z5Ac06b$-Iujm_wG%Hkm-;i(fCr%2Z2at=2wP6&wH7B zCV7Dror1sepyte|PGe_oX22 z$!d%2UAmM`9nq4NR8|ow$*XYGRE$)VXI>QjeY8l06vH)M6q%trvnFOhSK-qQL@KN4 zf79p=#M-)(Dy%4Q*?;z&MON_Lx`m0oDDk~D`ho??7*tZ4mz_`;zig@9Q$jUl1(iMN zssh=+G!p*bZ^W8SqZY(f5cG%4^5au(&$F;In?Uu3tsl4t)Pm>zbsq-qQYLl@s(iMk zM6yg?ctw)X#Ee1!_SL2f5(C8(8HLS8)AwqfLVRFHSkjm>tlizHY=4{3{?oJ5|IoAB z`LENIj<&_Z=Y1W!>GD2e4=xXb>wUVh=PjI%`r!8tBGWKb=>4iT%(>^N=1Yy>utjw} z{dGwJx_{=6{(p1+u2{x+QjO%x(+E7s3iqVMUz_5bM8%GPHOpSN+u04<#uq*J7e4nI zR#tfd#fBBb?}Rqei*$sZVD+9;_IHtn^q8^@T<`RzGZ6hd17`m=18joGJ`=Ffga6q` z%IAl-D7!TAteb13YZH?D?;<^H`#b-s@RI-gwg3MRkSjZLh1WvRo0$qo=t7@FRZ01M*86INOe>x5j;U4OH{C8N>0b`!GPO} zT5>CQ{b+#PX4ToXlvp-!q;e7esP&i-~S4HQi*bRHz4?E49_qt`!l-`$t zTzz>Mnio<*^6cB!-k=wsfu^_>p2rmOOJ2N*%u-i9IyW6#*8hm4`bDfW0TkYv2~GvU z#P>SnBIHZwVydu{;ve15mgtlBLPZ+~xgH-sG#9cyhBVwI3wcS!Br+L0JDWD70gN`_ z4cYX@$~to8-G3SyL$EvcK^SDL*R}ZI7P29|rQd~w>|l}vN+P7{XiW2^PMW)*SQz^7 zTI%JA=ZYQ8(oNgOg`3@h@1ll70VMvg<4G!^H}nQ-kl;19Eg$N5y030+#>yia-7@JA zBYm3oR%6)i-A#%OSZy+ZUJW-#$-fV^Pssh@nWoYOfA@Nh9!?d5P-1TjgxmrXksb?F zR6Pum$|eB=m10*pXIc|sZuLJ{94K8#1NMC;4n%z!P8Yrm>t7cn&C>ucHU+eefUUOHm?Cqq!^WwJ`3cZ9Q{D zL#wqjaO?s*s%V#~&1|dL60%FQclt`IcK*|J$JdbyLkrrUb_`#O3BtK$qGz3d*chQ5SEGSZW}%(z-kz9 zjd-K17J);vN$lSDjJSx+9iGMH**ocIeykXdhrMehM;QIAH&>cT&IkKG1^gU>^+I8W zF%l@>Rf*1vtGjD4SC3D*#Aa5*5Fk#!y8F(2nt-Fjl-Cmfn`8-1R-OuADGu_IbXpJE=UFc^(Jl>T3b6WcO59-IZ2T4GK(Tnty!m>ZigClQZo`P)Y zu^5RsjO#1UrlkI59V_G(GZ7N{!pFNGG6nTt$ae>Z(J3Y^N@Oa3P`pnL&kjsWi`jUN zVRyTV{aV@<3kMWXiXK7$7P4O1}ThBQKCx&3=IQ4grg^KQ@q9S zd8$Kzmbm+UIz5+$X4=JdB#co!D?}3iZ0(-4^&gOix}QXfP1Urm zQ{I1$oh0-C@!yfV-HKX)C;66HgE5&y78%%|P2 zqeFuYaGAmPR{+_Eg10{IcnxxD4G;XWdv)(Y>brNk`T+_}k86ILanhkj`2-t&dfjdN zy#}@ACFKr!AS`OrAv0z7^wEdEnSKc!U5N36_FiS+lB)x=PJ#M8=G1(++F+Du#ibQ{ zDj~Y>nL8P8F19p{*_$Yl{+^?FKjU2~8lHJFo3Aznzy7u46c+kZr{!nr{k4DQ|}afrBN+(Iru@pr8bQXoV&z}hN!00YiIFhR`mq@nFXx=Y6abqx>z#(#PXRcWF- zLod5QUi+LWK-)9I&|Y$T+*HtK~Ef@~OeYzcyy zZoz_wIr{YOg*bDHpMv;)cdST-2TzAYjz(@#GA)yapMkWq$-WNXa+C*oa*czN?ClqY z0-eUbVM2y1l9Y=3~SCHreL4qxQzF-@M1?q5t=xmj2TD;g|x z47G>l5a`5kNQX1lqRZqCNO71A!>%{onyAURqqQI7#kS2CC*+&Bz>{5Bl}EQ7zFx;% zUwx`{{xc2vyJ7TIns>XpieB{n6$NOAD14Qm6CY4R<%0V?jF3*Q-X3Kv%n z&6OXAU2f2*0;(s+*D;KNd1bxn;vPS)ypzY*3|@y3vBAz`A^2dup2WO`@=11b=j->w zP)dsT_|+?7Buqs!!`1lk{sLBxsJfeAl3=sjG`U&H(v_lQ4#S zy20BA@9!C`R5(G5*f5^|40Z+_MZ%ST#Q;{^@ zL|?oGb#-SI0b!r3VFM3se7R_O)kDN6^2| zszgOk*f-aPQYYO02^g8`k9a)7xyT08wycw5I0NCKoU4!XmHclX?~GqNXgdn9MA(Lc z$vXCNFrw(>dK7lZYfjCgaR7eUdC5w=ep|jNj2llOhyQWsLtMxX>^PuU1(0!gLcG5Vx2Q*)5B=;*<;ia--gs{hrXLy~z(7Qq zC<6tgocqPGBt8lgcpyYNKz$5=iIQEmu^a<1(w&JCij=!r=Xl>={F&;CShuZJ8kIMa z_=KDJyO7WYKkLH=h{O7Ym4%HzLoHsJ&Uv*?==oPkt^uG(1;0ITwB1-?iZrlZ zO=9dnm@K35OwdpsFEzz7#cabFe?0EWYIcyH7z27Rk(!_8g)|R@(@osgS`q=Lz@JV) zv5ny1`V05JDe;-}27Fg(VKs3_b-3Uka*qaAmUb$L@_IQ$vIpkn5(52;y3MS-*0KD3l#OmRdW|eeB)K-3 z_A&`xB@ak|=&Qh&v*)p{q@hGi>jo05p?j1?ySw%^56&E#OKKJ}VotTK6Kt`WB2Z4@ zwW3*GRLzHVOV}?eZhW8^9?FJh&4aC6c@rcjS?uXP-n4 zB(e4Wo^MbA1}ZdvzJrP1PP&ASz!6Po8Z(Bdb2ea!u^)A=-6J){^>%LPv`jvy$enho z(pYot=M?p-hheiot;`zY2qRsjI{UPaHzHr$Pa7?GBVrRJxskjiDhFl zhz}$L2xGfD;?@jd>TSmp_t!$O)$xq=2FrM1=7h&M^BN88{q|B1;P)l0L@xs!i@{6M zuRbX1rR2@~u|>6!;yc_IYvLb;;pJl#draurp-4XR+n%dY=1$fev)v=kBm@oqj9cL{ z4+OC){4)Ui8%8*j`hrY1HQekE4{QSZ59eFw0XfeFLMEsm_k-@{hd@trvUvM%ldrd4 zd9stqJ@}~o_J$4sZ4Uc_UTyZ|3Ob11{Bdx&qRQ~rKUoJ2K!&Y8G1M@Cr`P9&jQXUVEIVa8&!0$JV z`fJmudmVm?;}@4E+!y;A%<^e80yL(jRWb-bLDRK>U{aT+yu-e2l-a*w7n2g=B zv~=(`4}bhSQNXb*ERcy34{mG>lsk|sFV?ATcQ5?Dd?@7C`qUbh9(_8iP_ZS}xNpc* zz=odvq^S+2|9hL|^h8k12y!Uj>m6eex4q+(@L$m7XvE8g3#7lz59PK zrgSj&yx3eS_xbvaH&V=-Q^0SQ2IO)ekhyUR!G7$uO>zA3iM7&nhw*;-tbGckv{pNN zUS!b#$0qLqYWGVOys;&YKWMU4%xmLB-it#US<1h?c8|H__Wkk~!a)oiBJyP?DXXwb z^D6S$C!u10S-8CjJ=%^YQ_@;IsOw!AU=^s<`3vc+HlvJNZ+xv`tv5%W9_3)G<(I?< z5iPIJs;*Wc5(Yx{fI>Vg%F=pY=Q5>NzJp{Xh?nuh_%R(@Z>dY~A(SXO;@)_BI0RXH zkB}zlRj#SB-OmFY4BgZ3HOSH2OP>OvmVLhCM0t1{^wn8?aY^xefuGRNKw&@e;vnYO ziu$8Y+_cjNLJ3%lo7Y3aPEIFbA>n3uXO_i}OP`YG(a^>z_-Dz?hn0di`-uQz##cE{ z4E?T~uQS*3iBe+DRI4QLWBY`Hc|knY-J_55{*=8+B;;6AtFN3|LXa15k3(xNWX53z?ZNmzGC%mT%2ookzmTiO=C~@?(5(|{x<_i z!Xum1SYG!bIlH*c3NC)0o7&VIV8+Q$^=i}xlwfx01vQs*m04{fg(-Eo?A}a{lEC#s z92MV^hz78ME;GnidJ_%j_v~B#IaPSIKbhSY>)nDp_vPRf-DNl9y5UE`bd!+t**5+vybEsAUIiw9(fFq9gUNdNjtPZGPc&BV%=l+9=K z&O=cnBz{}{S%#{cW-D4581pKNV6-;Ave{8LlkHa35dbnu zbYoHmdKogILl-doIY|MBTsp*;^Uu^|$Ni5E-(R0-|&7t3~elJc?SB92GB2%o-! zDgF;%U~PBWp7fYt=n3FEcDEF7=&(>j2RNfR;!) zonu7$7Wl5w+cd=%Km5S^KC#C~PY}CU!pkHZkGb#&A%l93@Ug~ zOcF+R0|Wee$OP{WGkre8kDq+w_l?%Pl%^Ciq_za>HPUs#Iteq>zNdJ9t4f=hn$ggU z-k0q64HeK)JDJchsndb<_^dA<^Cl`#v(R0(y4uQ0GfS(#Wf2LUI3{uH>(69es@Lyt z@SvC<7_SZ4ZP0Y0Z*ahR(6k4RtUt2UR4L z*-u0<7~j_NObp>&s1Npyf_$V4f80INe4a4Dc-iZScJ6`6uo}EdK-?KPn zm%gaWoYpjd_aA3K=DN(yWE2@f&Si2So?%gNqkAwcTH6d{$lBrW7?59O=a6+@klCGY zaEkKG)XJzbOZJMmodbGTjKQ4PB5=tRS<$=~b~DDPMu3kc1OR)mXYIftJ$M+q9sB@(*xjBl%pX=oa7n=BF5< zy=!1n@XtDI&niB8Q1%nAf9|}V@un1?E9=UL!!LTS^jGek)~@V=KNcCon!iL*p9u0z zG~*^u@Pg=Gp`LB-tWw;nb%XCGP_iBPM9cj2e*bl}r$&eV)WCVGj+DD*){dw0zfQYw zg7|1`x9&ryBS(hzBx?J`F@e{IqEDHuCkcQD+cbPC<6^$cDy?oSa=xWG922?B+Yx-t z1*ME&w}17wf3`;S55ipD{7&V^2)3xm+$fJ`2fk$pBiTH9-&@buSE?PbRP9Z-kj}vO z%0yV!q4J%5e$)#H8u4-*xCvJ!(usZ*nxK6X4JwvUB7~S+L(?nsMn<(Z$LjcbDkf}00oFiqAKsr<+V*V^v>KoZUQ z+s_IK!y@FL?cQMTk$&&iT5nVsLWH*{@!=;fVC;a|&K(&wIEoRo2^z0`1Peh^z?x&> z_1bos)8=JY3G1)TH>Rp>=-&;MO-?+VyI1;&Qm95-PbI3~4;c}S8JNNAeb`if3={Vi z#8ti4{SD*r;uz3{;J#~3_7CxIKf^v-Gd1MzQ@AEi@4nndWmap(Q2DSf!IMFJTb}G! zO`nx#JJr1;&Lp_Z1P`uqV%JnBygul#X!-3gJ8z2?5q6ur!LFyj4? zY*C3`9D30a%z;*IQ+DzTJ8ADN8vA z0fES0Ax?(bRzu83r$*nH*v>o-El1odUt0kBnVv*xzGN@Wc_449v;-)aI0W_XrILOd zMb~v*!VkpLe>&na>nulH!>K!5-oF^>nP?5!=W9d|ceV^d)uy5PO}bIJ?=zz*wZe%< zh~m>t-@T^e{~uLR^oZ1)h)5V$YEdlkUXWA1CSND&RAP63)c%#3i3vSv#Ug@My!y-2 z&d0rRKU!}(IY@or?!gp$e~!C>KsUR!`WO}35f!j=D=<^UE9=S{d^pki>Vmp``{Y_~ z({iJ?w84y7pW#_@1TT7(+}f!|Ju*geXIy&?0pKLqVLfC3|JtMDcbc|ve0Dc998Ne9Rlr`IQP0P~|0GjF z!yw^^LGjd3>ppGFK)r$Tz6`VCb3SYmJZ*vZ5a(I^LdS;GI@FsN3gf697=?{;d2t@^ z%^E`KSc}vm9hv<&vttWS%SIPmQeV?FRzyF;(1n`$*Wu8!lUr}i&-(OXQdTgxtV{ttf8sV?sO4sK)t_SjXPPIr0VV>5c z^H|R-D8h;Y!|IB*Ia3V)UA4$kio8) z&`>0dP~9uj?24ve2(8c6g1RvuWAcw?Z!*aEE5`L6rKU!tfL52`T@2}#bu>@tp{V9t zdq?N68J|L_N6{ad4r*`RTv$T{twTA03HWbPw08O2in=w3O+nW~0I6H=Z+AflDfM79 zZz-VdbU+yu(P0mIYVpL%edB_P+D=&xzb+WiaxnAU7jT_#W61M%ga&P9WZHuDP93*! ztk$ByDu#GW|C(onNNo#yE#lxs$DoPwU6uJe6sSxw8{)tQ6_U=&1FI*3ZX5=+*u-4! zyzsr)@vFx-_Aq3^TSMVuEw!_4Nqb3#&%I?0wnXtx$XpX?$M0S79m}^wxFX0h%-);< z!U_bKE*wo)aLWKwm|Df&v$|w)If(&;{v5HN+kAcYETlMzK+bLHiv4PEtbEN}q7T+A zIUg3wW|>}Plq@fdA)6g3)~!v#IP&Nl$b#OW-=ELaS|+#KRpJ}3Gp#>KLVJHLjO{6< zVmse;xkmUL)w+MRffNXdRY;!RiB+dk)(Gw zZ;N-VU1dJ&R(QYcAn&pAbyj3=`w~3grtg}CIhjQb zlp9N8H<2x!Yg8?lkEyy0A;?1n@Kcts*)HT!l=~%@&sT2L$jifS44Ug|O>vV0+Ad4h zm$Bn@eTM5pWDHq$i+a!9c8wC{=Q@Cx?W1s%VN!EI@Z* zV5tgZOt%kewgsn->8{LXj4xXgF zfNPEM4!=ob(T@^e(SyI z98DoE+bf&665?w%q8=!!KcvH~vvIZNWS0$hgq`$XAx=8On;Akrmc%Vi389$%amm z&f2rxcF-}a3k!WR=(^LOS*+h8Hv?9H(fzGgncaL&w;ZtO-ZHTGuH2%om={YatMp-z zdpvITNz(J#3XbXWhpHD2J!LzSh~2#Kuv0}R?Uq}rbsNuocx`vsXQ1bQd^h_LuJ%6d zcyIUQvI15*B=N0r-`)aEU;Ae+ytkvQ+>%|seF+&s} z-&4!obDQ!I=2bJa2On9HeWp{D{#E5>6(+PE>9jTh^P3$rkHCyA-jLT4!jFm^WQgpY zCO*JH!`Dy^8!Dh>O*3cdJJ7S*yKZ=h1SF=s(8b0E@qzu81rkOb1CJ5=pEFGdC-Ziygu0(lf1T zZ`Gby&KHtB&3}rjtE-)XWDY4gNIbYE&7_dm$p@X6;>nkS-uZVPZ`3@&y%-p!2mJ_0 z6FTFhUD=OM1hv}_S0sDl2g{!gt1?$zYv*7F6DjZ7d)jr&v~#oP=B1^+T%V;c&Y26d zX%9g9GlW9yv^JkF{5v6iRHtZQog^ZtX|a7HPxBug$g}t#YOz?f)#$i**y6Zoz+URW zNBLv3CG20?Hg7OJ{VM`sME2YJmaop>6F+UNSxHcKLR0q!LDlg2wd)f=h8x*sTJ}Br z;^qc6cDCr%uqXzF;_gV^VD$qJK1!QJoFJCi!N}S{#{F(g9JlK{5yh4^INh&7(2W>| z=imO($jN`z$jRt~J9d8`H1cnKZoZcMx#F*`Ju3UxVU$rUzv}x(yP^N8-5wm>uGoL;`j-&5F6FoDAFI`zavGOtB%f8&Rn+_DmkeUrETU1%#c<6gymmqtof}m-a)24hd}}>ujN>9~ zZIcPw)BJX(7qqBf1-$y*2ou~1wlox-kjP)*DUwgwfwOl;e@p#_sQ33FjsL|3!&7e` zLo!zbQtLPHHA^NYQE%Trb;3@b)hQ++4*wt0&N?ot_iyvkB@Gf1(hVX~f{Y;DAdNKA zNDER!Bhn=)LyCw<3P=r&NOvkNAURS4%*@%tZ+D;l?(Va%eV*MvXXfO6-)GJ_pZmJr z*Rgwr%u6`)9DC($$rW$0@zJW^-wA+q*QtK36fl%7LWAVN?g8E0AOQ9`A`WzHQ2+%xBxaKiOtpGA___=DQ)v+XrD+T9a zc)h3a(1o%`_pl&4N0XL{z$5Jga^{VQ!`xsK)-HHoL?IswvXVFDqyot|Vg&{I+z=n( z?k5z$9TMWO{YBQ5dl3-e%A#9YXLBgKJ10{(0fHOTPoNgwOrH!dhk4-(n$=`O7V5(& zxK>qnkh0-Biu8CNAN2YVJs+sDZ2t+`QKMi`1VeB*^-GU+v*%!4I+hiR#OmL8WOjYf zTK$8%2gdDmEF@JnwY|R%(_#YtwB%P#wP+!C*GwJXlg_O&ngxI78bGDLH2wqo0q^GB zM$BQ&8Nc3!0+HJ9WfU%6{#>2Z5blcmA&$)9IV29#&SwlnZoh-#QEA^<`+&>kWrvfi zxgIB>e@}?o-7PB&%DuW|jdCuJBW+Z(gyuSNOlz!`=ahL(Ru~CeCwqKRwnYWEK+-rW+?dqB;&-V*+BLYaYxb@5}#?a-Ds28fMCeIT<=>BRQv zshS|#Jn>;jznaSXM_-E`zD&DE?anC>p8g$NC3dm%DIT@fFCq-`4y(tG{8>3EFYidF zS~vegQ?=M6_#Z>pCx-dGC2A5mPD6XhlDHui)=*qY3Fl!nOzyH{zfR+_Zn{iuTeo<5 z2P(q|{I*R|g#v-)WGzM=o_oRb@wMqZVjI+YLhiWCgdBHtqWLZ~^VLyCWx$AO+qNp#kj|NSjR7*l8Uc;` zY;AaoC~x*kCSVPw&>nzGkO%I2?JSm&RPOW*Q9vKvp`hH=>-z>+c=gfNNFL^@xUyz; zvPS#v*hKB#$5>}XA?SLZ1ja|>Ko+7z?FKrP=q!3E&K}p3$Hp(gXnb474tkM36Mb|~ ze?B_1o-9>sCR-Az;rUW6etyv8rK0|oq{Mf1RO^p=I$T~5lunref952p6a%Z(-(c{1=+c1n@)EsIA; zN8Ge0?DYJ!1yl4uukjmrUvz(L51bV#*>-;A8n)MLEzT8?W*`yw@&1gJN^N?tz~*1! zs(MaJ+N>+stHp}r?h}#qrMsu589}_d!_tWxEA1{y19xxsx~<91FMKdYxUYcbS#!Wu zVGVh>dS#Bxc>&ObwU9l36#dOd?+4=;9tHH|_eiYFQ45L0nq%@9Y0xjSqN}>sO6LmR zK;Ks1Dp>qkQ@U^*D~L?b_FMbXdK9sUFR=Gtoo@Ptap}=6XAjLTq&apS6@w$eRv4zDi2FdVgRjy z8;-yJ1~>>02N<>A17p%4e^@I&+0at{sSXM?Yk+QR6Le-8TY$-0-k;pd$AKvalm;jP zmE-Wc)@QHEdmw=!ZS%Spe2s28W_x*T7idEdr5e})OZrp*%Ir-@(3``2nh(AdiGY-g z)(#^nC4W+867A?utDlhwO0 zbms^uk_b}_iIkCMnXjshe^W#Ps&ae*fh5e*S6715!LG|~Sh8z#V%XfVm0$3lR$@`M zr$`U)dQtgt0;EK_ov7(&=-rM!AHnB-TbZ92eJ>3t87{cEL|Jy1F*yNMP0PVzq3iTr zM^F?Q9KgsthxVKu(;bE_SYO1910YW*BsQrA%pTO$I)?3DzU6MDn1k2!S;?y^kuk6MD&7&4m8=m(goTJ-L>yrg54*2K$!{QDRa!C~I# zVH-ygmn7DVhlk6bxV{L16u!kIsLR7tv6auMMV(zxCVl=d&q+;NH^!zbraYMk^^hue z0$t6we*0kcTOB8_5}*uk+uhqJpO7N%!_M87aO?FxetER%hnuW6oKr}K5#?pTU83o` z1!UHeQaO2P5=EQ8{D#+o{7aI3-$udq7FVjS+@p^kK5Q{;-4I!G{kUht?y6}%Hi~Y* z@sHE&-ZmnajfY4{T?)5_RbVqc(&kSCA1d;#=3~vrF%6j1LJ4gg3_fv$;9XRRB8dp* z^v?WG z`{J558fjIBWNijkF1i{jrVs~ZuWMLV)@8Obo)vO@xi%^Z)N-1y!J4ix=VAFUOc#AM zAnWeL@`&*ElIA|en>(j4v)3#dR`I%vo8gc-)8D~ zm8+!TE)7}|t3bg3YFOJ_Z_JatkR(*$UxWEG(pWqGA#w4Z{M*6|wlOU99rGkIMfFc|aLH2{!#?w~q; zHE9$w9`huYWtO_-)xi%dek25c`JE1x04IMj!is(oW?q`T3X{K^b2Y~K+=n*v<+Mv} zrMTCpVnc!|FrYr0ubahw&NM|oU?#;x^NqH8IZd^0yhW8sHe9$b)BXv;k4WIO2O>BA z>AjO1hox8F)_woO=K&judPQh)!8^>F! zR(gO^xD)ac6+jQJ(oAA!7~lr}tF(xtQ%XaXR03C^FTY)K2mH=)Tx-MH?&8X_j38Ma z;LfRGR64^nIiEPpw>fiYcL>H1vFfazAst5Zi5m9oaq>N75K+`c9GBg&yiNO%_TDW@-vGo>>`xPUrTT0x#o=o?cN~s2x}9 z9*5axATt!)NE@HN2EXPYJ=V!~tBhXR%iF`obWMY`O^o8sub@M{xpwMp-zxa0w=Iw4 zg6LcXa{q0ep4HC{!RNO#8Qwp8Q`Pl~R|F#JyI*{xVSKXG=)MNZ528RN!PmSx>A&4j zi9;%bF6ifPL`4>E0B%Y>nCmx*-mai}PuuUfmLxAR9lh5|0@5$Wn%TRbsI z7M}ssZdA#nB8Pp^t_EoElIr6e)M51=4&5!o|4}Kv!^N_2Ul4M z2ojRmW1!mji$ciXpda&qz%lbbFCE;oBMYi#K>-&r=|ji;7uuygBEr!4}c z(TITsmKyReyN=*3CUtL&6)alH1wUA$e*VmvF`+N+ZbWvVQ{}s=RnOPx=#=qgIbB8B-ldKtMRO+MKv*XBKM_a{?kGe^?9PZL@>#_`aVX zEq@aWj&2pHJSRqEA@8J5`8F0Y6>*=`09%`qg{B_l)uM91F`*nm$o)f{0MOY@RehJM z(X0p!-u|>o7GfT-Ki#xJnX;;uG0yu8Les`nzXrW};`>+NJK5bNPYcbz`;R2K5u+Rn z1_UIS_y?E;vIcF`ggG(J=wrI1!x@q^N&|o|~$RA@RyUtMlrUk1_F9 zR`D>Z*D9hFV9*iQJq5J86ek{^BLeO{qn@9V(@QP3;pP1SoZpc}tZaqh6z+xpEVTR? z2vXLz*fFOiuMq2f$6)cG)1z6!@Www*cUZhwW-d8jqC_%RS>`hV{iLVN;S|zh!1fnn zH>;T~8iFrue4jvKefuLM7{C#5};iGwzaA>g!vCEoc{7A0&hDz=^ zv&{PFN5AZs3!gI0uWc6qCyP>$=h5Z%T*mWp2iqfNI&Q#m5K?2mOO>5Wk||==9ERY? z(=XFdFxucA=M*3T=OoF3zW$_knfvr95}H~FMwAjt?P#(#FlhH4Rlmvs3M^TlMc%a!442E|u42{K44 zS4fx`LI>&uibuA+5@%x?nOrQT(;Pqo_G%1rYz~v`(FpQGr6v4stLN+giFZr!Ryoep zmbjsX^mkHACHyy9$p2tJi0?Tu>$`>gzi}>}cH5ipi|N;8wP|>g7O=!J-3}SuC%KVq zXYiVK^J{e}Y!1Wg?vN*m@o~$Sx^u;sCbhHQpjH_FE0g8_fLi$n4&on(6KyBS!LCSx z9>Qg{Nk~TcX6E$#f0%IbpFB(B(;*Q^&yYwvwnxV$_qgJ7P@I97inf&UO2K>o$Uma) z#n9hvcUpmW1CD*SnM&0EaweMNXI0|u%~cOL77+6ANYn%z6FG%;v{a=cjZ1-)JuXk^a#r=0gs!T)AD4FO!}{UJTeQ zDc^UVX!QBPEiR3$h_sv>&s!a3N@J*oNN@a(qRjAG zJu04jb(>(_YsV+t14+)t4Ip12zioK*^*3wLl)VfCzW4PvC=k*4notfo`{E4f_&l;? zeRqJT{Xiu>VX5(Df537#W^V0~^9mGT2A$!=UdYHJI2?r{5xjIH61mW&+S3QoTc4lt zeKf0j@UVZdA?ez`_bMw4jUc6dYvU6@w+X^|{!;^i@PBWeb*J(U*-P`?DAicEzYt z0XgE9%iSF4@p60<$L)c>h}#tQrKQ~ zNmEXm-0tOt{wS-+^xqgN1^oe!e1m^(!K<9C`oC>q=&m}U8Z*>OKc`eebGn!7on!_@ zTL1e;Z4dN7F>PySmKAi@$!<*la~z$^&CygC7mx9#PshaNHuwH(+bd-QeEO0BPo$}1;r4=w*-F}l9P#nd*@*}H+DEkeMnR=EmsjE@- z8qpob)^}@)j3fS^O==TX(p)oU2#tiT;^N_~w^}8anIZ0OBwznb_ep+q38E_$7Cno; zX`g?c-Q#4rl7F6UNES=^KR^HLcXpE~4<-k3r3=G(_8`gZfL^#K`MCdyESueeg>p`LV7Jrz}%UeswtiP*3I*qpF3wDiXUZ+X# zA8~Fe#UbkFAASAv=a^cq=#lJ^Tl?%U-jmoB*PwFl1!Dxb!l-#|5Y zLaqIWS)0Dz4DW|sm9+oU0NN7{*Z=3>B(kh}^53_wgCxz~j3(JL)yudZ5|N87h-Hr1 z3kI2cjFNvf-FDfN{_7=_`B_c>pNEj!Tm9yrmr#Y2MjHP=2I(Ycs8Z=a&#BKe&EQ}A z;wh6>Ivu9!yrPRRtSNG zGi!j8+BSJT}%N~d?GlOgCgtuKVJ&B{M{z(ZvnKzjuB$y7fJLk+yCHe4unh+@14 zZQSC&*lTsL_(yDV(a-t8w7cX?jy5p2`-$j%y$PRBPi>zhHGEBukJjSN9^Po4q}8CO zzD_juhn)F=`<1y_Lt#rV;j~w4$_fqrInN2s}zOdAd53&pFa1hOY^ zSH2j=GA6qVuvHI0#Ejz90x^gsDJu3h3pVI`hl@=Zwe_!n z^kTk*GZDVXq;~gQy@3O)-=-QY4XVzplAPn21`j656K@~poX(fWhzm217_KO3lLvw& zB!AE{@V%~=lI|6XOB-dt&s$*yfU~LTbVi3Wd-PJ+bow90-1ieL7t~*~Vcv!Uu1k%i zw?ocNo3H-3`qnQs6CGHcE?FhM-Sq=f2gTyx+V3x7osmYuRjz&rB|3yB*Vva};F)BX z32IU5w~WvY>Lxk_wapXZ2@J#-B3hy4o2`TT*3Fb(aQalO0(bGGHd9zUr}b1!X`Rz$ zp|x0d?yM(YQNQ{CMwVacowZ1nxWVT$r%$bb#?P&;b9 zg9#bRdjUpoX%@JKSW$GwC}#qfMdudz9{fl8-A7{$XmpkNckam}F0Mof28G;6uCZQY zZ>Euaqpz|91o}FBe#mQs()x?)(TiTwr?_Cf;^0}4Dho$;^xej*dAvNHtWm)+)ddyu z(-)sm#y+{gd9cxZe+v)jy9=eIJa_9JrT8p{hT|oHkU4zs#jGv`{MZpv3T1(hA_}F#qJrWQT)xcVjG7-y`EqPx2 zsx-}N-ExuwVEkH)K^0kw6OLK63`HT0BNcz0L$S@zs_$c*m=sW6tF*qrU0_Yc^Wwe# zF5Tnfw6|AC5p>P{ei-AYebc$F*A%wzWdS#^O7^8~U47>Sdj18c^DVnGouD*<{1Z+B zEcrSjP5S%W`8^Cr!*Kud;(~%~pHT)KJuI>!!cd&UF`^)RM<_Qo{Obx63H(-jN`LG+tly#u&D0C?F z66Qq2>?E1mFc*9-PR`f2@%h$Vn2dQkp{nS1NoYhk1(gAwNPKRBeZsqiIQuW_N8&Hi z(MQ-QL%Y=x$*ynP1tqFmF4ujbQBO~y%2@tnv>TVZ;55Zo#;e@Rb%O?oP3(8+o%^xP z3$h$e}=tfCme}j@A=w$oi2qQ(8x>0~NIYF)N}xhV2bF5X5=yZ1#cMO1~CY z=v#0PbVeOAZP`na?@N{0uVA-m(BzM?<*k|WuL)&``I3;3z-i<07qQHrV|Jj}o{d+q zZDMF3AogmQ9>v?uFLjxbep*8hy-h>q&n3PL;>*_o*@X|5@iya9!iVDQfwDBqtET$D zLrSSz1B2hXJk0d^*}O4Lr?)soduDDRt4TQc2C|{w@%**a`Z0$M0h7V?K|A^ImHWFu zJyk_uXdBZsr7hE?|GYP7KfAVQL7eUYHg?V& z8?b<~jRhawOEganZo{x!(@QUV<>K>o4`!^%okQYNz{NBHG1+aXLCPInd#d6IfROoI zXxU3WA!DS+3~?I3%en!3D)JuC%9F!+w;JpC$+-IE1lQ#b6c?W(RwlMET9tZiPQ4G< zTt2={G6B7Q@82yxcRLIl^~3Mcwd=Et!dc~OY9==jHu4FHRCHc~z|VNNmxlT8--ej= z`sjUqBSwR5_ab|n*kG1Yzxw>%v!)?4z0{u1RWiGb+FH}F6Qm5mg4^{<9PIosLPzqw z*DON~F%+Q+kF%1Fwfpfx(i~z)iMD#8nk>g=U-&9aY^yFW5sI#Aj2LZxx%62&TB}ii zCTB`V>&-bG1>v5rdG89c)h8fb$CgWq4Uz;!Tt)uuu|Vj~ZlU&APe1f#GO>P=K~#c$ z3u=MiMEMYq#3P9uB&nd^2}S~~!7O^U?d(QjNNV8w?l+L|9XrK_)n5Aq=xv=H6XERv zNF-ex{87OD2B!&Z&;r-5lJ|b~&#h!k>~|?CL8Az22TG@B`xHEESsKakD&ADpc5RzL zlv24~#x$sUwd$1I=$Ef9XxF<(eYGV!wO6kEq7khJ4^*u{S~(!*G#)ya`fz&*UVg|_ zt$~?0J&old*IbY_HFwx_^7zqg`m4)3YM{YynS^es??0wHp)iICsj?WrRmzf zo#y=OI#2AYfw->VAjZW$7#QJ4oAl_47cA_DH}+*^Jt|H9))v*W?+#bI4Szg%?iyq= z-_Vye_xIQ~Y(aUH>1hHz^Z_+AfwVW5+wu7K%2TayMy;)=l;pb~qXL9;r#Ec0ldQ3_ zp7#ki>Z7f>-r4E{>U1@p4^i4X#_(qfSS z|26|04@yYe>@B;f?&4zZ@yn+FnVXRkNN1NARr_kvuO0q+Qg@Vv(}M30UEvRYOn;Uy z91r6^1^0*ogJwFdM&<6_I13Gxfv37B&46LG5l(22_;o}9C&Ouk6M$-j@9r!-#vBU( z&EHj-fMF@R0VqQOr>^@TSV*#O0=7?(=_ClAb=D&YZ}%o>v%Xu61!6()4i=mIn=r?E7(0JeW*vu*crxHS2t;H@A|SF_Dz2BmlFW~K~Dk` zRKD=Md9j|DYB0RZ=P~9xlB36XaDesN57+y_vbrVQkm)Ro5;f8DeQ*Zk2=<>T zPOi8IzDGOy4gr0lgeXQ47-IyQ?97>f=4AEZ@@We>lLV9k|HWU;?FD30-+@T-M~eHm zwc9Xn5i75<6B40&BWPwEt&V{tI<&|+#J6oS9&$tmr3>^uasZ3zB=*C=2D-X6Vc{>v zqr|Xia3dLlFLP8!0@m{TZhLv5E>}=fvG|bXz=3U zatWXW2qf;UfN4ElNFwS&qxUTf(il7ABB3N8Dk6~ZMZ}0Am3OBVTI`;^iyby~(XjgyJ*%=KbXYnsntbRUx(SIYFe} zc#F;VQ=1=|;QH&q`v#xbS*xwO-13`e2*$G=S$GW-?6JBn#Me0T9`SAgZjtxO6HO|q z0HDwDuIznJA=uJG`>at|+db=KO1r@IVLdBK_L2u_H=*DYPz+PN-fluWs}{`{Avl)y z;RU0D&}*yNjWJ9L968l}VR`_(PJ-r~KpV?(pP$MOs{ljm2%fR)mDZ!%vD}G_Pxp+f zb`%DFnWw4b3Rwb_WMBiBQhTin#)J@Uvl>;KlG5@UMZK#szbkrfr}I{AxerObArkl| zP`O*Dt_aARQyolnw$&P&Mv6aX8QpfP&*yxxqswpI!Sg<4gCRt$N~C)_Z$~wVcQVed z7$Z9Xt2#P(yMPHg`blmHMf)|M)@iJv$stcvXz?TxF$1e&jMBe=)>lhdpXNT|$KOHQ zFsW<0{}kwF#bv*OpT$h>08$-FC=E&=_#E2us@Rq9s@^;S8i89bis883cL%DlZCh<; zbPDlnpl!OYoTLNZ0_nVvjarCq}(H6;aH0RA2a@NVKQNR#81$zy-9%3P3YS z+8Fb{K@iCEZlED_i$|6mnRNa$L2;fEnj0G?F8IdA?;O0}xYnM%xyR!0E07MK@9)lL zcH-;m=~-2^X&qiEH*Zqtqw9y}qKi;)$|DBLm^=&&;v`j0W^m82L4yxm_S@2z@g zIm!>1D_6$YI-$(p%GWBucI{K9h?WQyIy)4lpLRZSw2nyS+Ha|B{zpeL!_KLNw@ zB~`V)1GwCFjr@*8P_OQm7es)5dA+AiB6l`)_JK|E2cPY)ksI7gJD*=uoR0nqO3Zls zOafh|yllzx>}y)%R5qb1LsI~^1&ey^?A^1wn!Q1PnV?@=DH!U2Z~`Vm^2ew5@yziq;uUz4TogW8UM0Jiyl)Dk;{}vCB6{bgEGtbP7fNq%#orlZy5RAt)%^PTa@Ub&5=R#wQ3gMA<~KpQGR*_e{N44c*l6~(^Ma_Tvt~z z!%IhBkmh?aK=}lVvA;slC>SQt$N8g-CGv0DR>rFdfChBdp|te{H@yYpF)Fw4obKUB zcD;2nmj%^W^ByZ9%#lBw(tD)KEKVWO5#tnsTzv|{19enqkZGR0YT&?sC7cJGW{=;* zeX6f>XW_S$^x`bHskSrSLA5RgBX(t}$@eW2qvCTBj;j&-ynAjT8HHq~*(bO3MA@@N zqSNNE*-v$KejF>tl?u8i5t{13Bj~Zymynh1>52}hxdx z@Yt*b2*!9_IWosz$!MQK`W4nYN|~M{mPZK=69w3WZ)p@A*(dR2lq%yKmI

gf;E6 zKremEMDx}0bu87mGPa~90D6+z*3#LlDDCgv`C)W-Ogd=ZX`z4k+lb&DL9-${nksX=2HqB1uCLg{(#FdkaX$mdqETh?QOI> zNl4z(K)vcnJ+MmI4IIALC`*g#X!_;xNI!nf746A$Xhoy$JZ&|LFeZ0opThWqqVuja zE{)aCVq;)H>F!ouT@n=20JrIh(JfPdf!rxc9{)mRV~e8Xv_<)GWLxVOqphIozQ0V1 zul%Nyt~B~E4Bb$ifXZ;Eo~Snnos>6->VS3D8Ful$J!p#?E%}MsS|5f-)xH=&htK_e z7J5mWE9s#ebF9f`#({}Yj@7BZILDa@Sx*lIF!}+^AN*0}9pk(X42S>f0JRkk|6luw>-j$}6wYGvqO`&?1EO;K*Di^Bi3q zu&*Qju4(4#Yxy7UBoRH?E3_GUu>|8wh$P%%#nd~Z@kxFINji-O8D9|JhmOv#t2D^N zL{bya$;WaJ0f(^yJRAbDD7Yn3QgC?X+DhzB=JXlQT7KfWh|GZwS}o)HgCmWINt{p0 zf<7!}YKZ<@d@bK7(-hV#F$-2{Riw=mA3X@Wp~NvAjBt{w2AcWFa5cxVASM;Oj*Kl| z`>p2m^9|u+F6V*3i$lEX+o3b_88|vOMUWOkyWoz8{j7G;lT#k|-&_&JPS!wwAD3^} z4`qtMRIP<3*Q65>I;;2GiUDe3^WQ+!_SG4_nt5JD`hy0d)qG`uT8c1t2RbWHLAWhX zmQ!pAC1-p{NG7#&kN4>3pfbQA>ort?an#P)hd58_=(a19&UZ|mfA_Vty-FP%xFm?r z$~Jv($sp|jb#>JKdx9~4Y9Pf~9Dn^w^OX{XAX5ehRyx&ShB5yOP?uk(`h)HFg3cy! zB(F+(z%=W6DhwHKSNb8R5ojFA?Q(>L5Ob)6-oBY`{cO=BAr+^&C;1~ku>_D2B+MJ> z13+A66f2Kb6jyxKJU#7(P6E8qH9kHQG^$CY*rXPCisj|&G6A+IpN=OTdeg`z`H;8d z5{Iydj+QEY4Ds2-44rr&%+|c1;(8^0LdS4L)CjV~=^&aQ|QL^pzZWExN-8!VTeHqDA=WgGZnnU97W?#ceY(oVPTjKXw>1-Em77A@}2;E$^hev=Xk#{a^!zz zs^i(Bar6=(lYP8pGN<%VF*(fimuGixoOoh<-OX8%Fm9l?WEnK&lj5BoYA^w^V}h4R zSS1ojLL7k7KZAYn!cEjF68J)ToD4;ASVFz#RiJMXFgs0p2U}F~hSlK;bYh9beX(JU zeh0Obz=pu2w}g=OLYTX5*Mb6{Y{-$nQ;Y}8L~xkwPLHpY3AIKwtQd{=eFxaTG?l`f zxRyQ)a+B~jEpC@hZ`k-!$;GLIH8M!-zHE76RyG~yeMK$994W4d#wP^x!PBjyS4GuY z!E6YLf+Z)$=XD3w=H+B<=-=j^6G~Z~n}g{zfX}EjISceuG;hobby5W!e48t{5i>G? zM?LV0Z}%B)eB7ui;+@BNxpK~(^$wbYm3CKuIck<)^r890wdK=gLu5|)ETJ1n@Fn{& zwxGVH?d&c;hkl?J801vfjPMaiDg)=sqa@sy?4yA_qQ7xNIUC>lksSCN59ijaza6Sr z*>9ih|M=xBtAC-Wc+G*&qCEk`_W@(=tFk*SPwA?KyGIXc09lm1@m#S*t3%{Iv;t7O{alNmlagMO@i{+(J?Q-#Af~Mv!=wt z-vOiwxxJSr{P!7(hm~W4a5P;+iRh#CAhz;)y`47l+`+ew{4(T;&0{2s0_J_K_%N-Z@xVK>9aoN%BhjsbmASw9i7IHkjiTKTSn+` z+LLv^3C!m2{w>lw6M)b?hJ8T5=yQ|@{1P8(;xuN)j@gX-2Ie<%XZ-TsJE++k4Jnkx zY$WE{FDeBsDHv}ihJ0JZ6!%-ct_N zHv3^3BJI2CTdj zi-4t*xER=S08ZMX2IHuqVIdbwbEj-9bHnovu~KTM_n$Ey!HWS$IFPkHNJ?g0$5B7% zP|PjU@Z!3R^Tpq2nnjl#d7(?@;ZxR%=;lca-Ff5QfwcTgUd3n|I(_;xG3a$yq+i)P%t~SncyyW#fp=LJ5X#eiJ1dK=RZC# zBW?qONrUmZs;u6ZWGu3t#S&(G0Fh=cxb6PE#+dt!dI2TE-cz0C`$YW+kt9HNCr1Ou zo&crgHwZI-*$-#tah2@?zIXH>IfG?K8?a6G!QI=`}-G^#ZI67#D)$Xw#m zuno8nAxUGK-PGs+;(iu~D%egP4>*boJ-62jO1Q;W5gNnt0u^rbL-JjVOS=^U=dm7S zjmN4?GxkO3#?e=)W0C=pit~%zB?8M}1jMKoVLXZE4}B8ev;#;_0lR*mV4)Au zo?AyZ;c@*q=wj1Y)a-U29&|=J>N+Fq95?=e>#|qP>RgL4S+u zV+%^ZYB79YJAB8M-S#ftd*TDw_o>vRMio0-KI2+?kv~oM(%c2Tvk7C~=B;j(8#n=q zasr5pF=@4|8Z?-nM_#qm5aTja$=C0zj=RxvwzpREl>Egp-dI~F>)oP*l$bbXb$Z~I zFMh%3aKTRpBox#;TJP`FT7DJ9NKutQHRBXId0$tLYNIj<#C5fDw1T;`9`3AdZ@Q*MGUO7}n z8x=Z*{+)HH!bq02>1u z7Sr@FNl9~=3en`=+BjZsEW#fq7jmu87ojewoQX@Q0*Ez_gHuY_Y?Q5^BLmPL$&b(P zH6SxTaW6fzd9(}dFt2^mhN+X{qBswKi;1N#yMVNi3#+3}Y!WfCguR$1CA7!P4@2E= z-YM+$$0;8;RtI@z8SNtdKEq;a1!yCq3Wz#NyVS-x#LLq|jX1$FtTC@4h zzZK~9N-varefsP4EPm%UHsb>;`S+%T9Tdmr?#?bnT(2h=EW0U(+zvrzf%S3*SK0ls z^-Y&XO~vBSsDw_6>Uv+fb&q_GWyf&z-ZW^$V)4HXy!YzQk-9t=Mx8GS27qeLwS9QL zJ1jt~>Hc~nh)oNZ_W51+)2P}K2mJ6=j;zLWO@Wt}H_Ys-OOc`Es!G+6e3`moVPN7X zFdFqSXlXAHm1axD19NWx&__nrhWb0?IV?KxF*%1f+9)0o8zR(iwG>&#;o$(3VBl_- z#3n8FqJFj}4NjDwt(t!Ma%~ElHzr&In=qcImoU*+CP1Z8)gyz7x3#yPU6pmbI>^=4 z4d;#Wcf(4<=sz=+C+YWi?KSLB@d(!UHY8W9XXYEun}!W)qkB^yZe374SK`02I)!+X zj2ZVTVeC<5$cquljny|jWaKDy*?4{CXT^XiF79)AzMv~u?sqEi9k$k>4I}y_<$xkS zxD8lI!0701{y{zaM!JWnv%<+gJ9!v)yUZrllS?Z@e|@kJ*X z+C!$oyo~QOHCf+0>;(Y`{v_S=8`5clIv1S z`1M}hFNs1C8HWvkNgXcF+MOp~`2)nF(qDoj7#q$#hnpn(hu3MM4R6+FPPitKZ{&Q6 zZ?hN!vZWbkHv)rj2!7GuyuS%66pDH5BE{^#>t;XlBTis>jNgGizk3K&&V3G_5itH+ zYSv!$=fvI|lztZNky%)aYBx^YHwWJaz(+UP{@9t|30)$dS#IEzqodIhkKcfIHLeMS z1>4+q!UtWF_aA~r;a9(2Z|Cnr`)Hv);utrzYn8QLFs&#g1|!Zmkj-$P88E6g!!tY< z7!opT=p0r~$Jy2lholT%B@&vzpYpG^vfSW-{C12~1K^Q%?si=Rd{;8O~Fd>ETEx0Cc zJ~vNVv9UfaE!K-DMcvN~TncLsrBi9z(>{G9 z$YrQUsjIAt+n1}yPNSVptu`UB3-&f49aw)sVtFY1)YNdn^OntWQjVL(pi!Dkcl za&Si2j_^_vmk-93?qnJ$DLr9eqoi?Fj?eyZ^_PLb(e-IWc#YfWpk<9C>j7A{B2QUqQCC?#k`# zH|bm_zegke_GT6O)H4f7v={w;770LzmP%A~P*hxBCZch^${ize;n&L=fMDL zNUxX6=J!;oZxZi$ESi3`Q2OqSV?A28DnDq7eOF~BKf);99VyB)6K|=emJ;kTJ8XKq zyAA73aGtX1Dvp*?lchClbn4pMIPE`E^B?eA`Li)K16l_BGM%gV{^--+of_+APw_6b zx#@x8loBR$&a$6rpQ8Sghn+2!?*7!Pu7o=4U;RP;Z)547BkupF)AZOWFd*b_?XAF< zu$$EE%bT&Ibu)CnBZg=GA1(+f_AQngEICj46MyjTd7GY)z=$oQQ=nPn)B1n#0{UN1 zr|Op6o2N=I#e$`mk8v#3wGQV`Gb-cxmUo}wIg=6kF8mFnJ3ApYV%#CNVL1Jm!ngc_ z1j8482j)iN=59`NtUiu;R*oS~evIK=295hqk|kMvrYBGhgW57j*qQvsg2Calom9W~ zC20XLF1y<--_5+;MW`*Zy84&Tm<5t}&n-X99BFs5BCwY2iKJAyRnd~a7G+_<-PTT_IQnewbQ+6g)E(0>jv)H&58$0{79{vrK4*w3RKfe+R&mOzykT9Y(jIlhc8@z95x+GGg}iMq{VhqRsYt z>G+(RSEs;lOjaAg=fsOJFO=w;c)tg6@ST*0oL9d zd5-p6nbxwX{8O3Vmv|BRAMOen@O}GCL>oPD_3h(ILTf@Cc|z<*Hc7J`wIFYGe#%@t zAM>}+2jjEONZMH!#}mUUs=hBUf!nW?=jxGOQ9Q>Ed9$s%%ddliTRyY?@;tJ#y?Uy2 z*IHQa=U>n^-Z}4O)dQA@Z1T9ZR>3hJi9;qu_+#ma$OEM|Ptj92DBqv{Oh4|p;eJ!u zIk^FktMONLPA&;GX#5?awY6B5D9mhV#iNju2@QC~F^pgeoY3hpTH(|-j zURNuUE5SI)vM7p#p1GdgnGH*+UC+*N0&DnDve{3Ogq%qaRA;4y8W;985Jf!*N*Bn% z*_arRMup`v^O20_y*Cw8jO)U5pEysC&}R<86Q7%n*A&78v{56b0bd0w6fJjP!1a+Q zM2`7!aBu6rV|jr$zlh8eo%;m8sz!|1?GCJ(UNPThc!WF9Heh|&wqgUaTf%+Dr0HNE zN(M8A_qJoW%*$OeRn^_}QIdYKeo`T7KW-hl^{jrC=0kU%9YM4$oyOT{VOd z7v$zg%Viiv<=h+!C)bZs0Oq+K=Q$FFRRf}xG6&DEf2ELbc>H;MPi{duPr@KBMT^~} z*JF~6d4oK>lcH=Pn*IZ2ji8RU8kb4`m=Y@fH#BZgVr+oYjCI}sCucHosJSFo(Oc1%bp;M?lk{dlxO{2KgHd#JczZIN| z4{r3_6}^5UvMS;Ti`JPlh#i0hU^fmJFB0<`94qNNJe%_Aq=eQ;O0JjiJgUB0g8m)KYS)EkQqotCDy%hGGmhX7DP5(VUb6^9Prcp3*;wWarIQWg~oqj?Y+aAirTba z6#?l@Dbl0}(m}ciQL2D|Qk50~k=~@&ARtJWj&Ni`bpFcdtK(VQt2pi(*JNp01PwD zG4~qK#zxoRLNqMwKBRVWQM4)a73uA0ji>W3xu>p7l(Bd6_Ea#rQe^oIt5*)TUUq=H zJoW9n-JMfAq_7f$7VPhcZ&L5%`0s}^S0Jy($>Dt(*VZ#H2*}>-nE`@{k{Ig}5Aa3d zOgAnC;gn%ARvOa6D{&ec?g|6#;2$&7QvAEb9~^^FkPOHC-YMCiD`ZQNcxSD z;EkF9%%H>5ltbAk^od>D$x8*TClw}0%IW2% zBmj4e{NuVq^z#S-Qo6S8{mX)lDFvkJd<~}kyMy>yz-U+WxaF*CgROa^bAxa7z+%0d zqZYW6IPoEmXoY?kT^$5hb9CkJ(tPSDxM-9h2JgU2lEfOk1Fq^RF?@!A+e6+5cGKA2 z=Sm|z6t;Ar$0S+7|D*HCYp1FN@gC}Exg76GmZd7kx#zI?Wq7xzMkgLT!;&LRP7YuT zH*{7as6iV8nWSf)&?PoL=i%S5n2W9YB%KE|lnSfQOpW%9sI<1U;;dZ@TKEJ6a0iv0 zl%a!2bVqk$^+eE7aJ4&lE~++FhZ>+=_n%1Tb4HX^cJI#&|EqvkF0jVOAKh*>?O7Nv8*1yTJ%j*o(R3r6k6@82RXagHLvyp z;`yLUR`h}q5c(ICLfKQ*F!<=-SO6r?cV0QNM^0Mm$eKh=6h7sy4D0=aH;vqX__9wR z3MbOKX^95w$DbuDXc`6;{8y;tHWt4YLhScGjf+I}%cccf<_Cp7vOv4Eoge->ib^@c zEWF}oRcGiLgk?IL!n*>jTfRm()UjWyxf7ECPb>B$0n+$gHToX?Sa>qAiSSA@N%Ds zebLWg^gLpv>BB~ryOg5=TzJuKB#28HgSSzxn>c>=R@n{(!F*6`07K3+0u)_Vd$@Ad z;`b}h=lwf<(Y5olvhG{SM`L8NHtGW}HHFh=pwnnxlt>ps5B~NS-{HXyK*)gKbJ=w0 zJW6?v3E^@I@$p(-_@ZtVz>n5Nv%vG_gLRVs7(MNYV`a|r3o?bh=EVzBTU%js9Re}D z3QwJB5I_=c!Gk`OqNCF!2DvPr=XKM$ zPUaQ3>;@w!jb!eO?9$l5NY9|a#klfDXY9>fZRg|4ZTf**gGtAkccWAMEgGl^<^emA{hXw{0hb zPq*@2{urp%bW5sK@EjRMc*_Fcp>i9r!iptybuqZz?BmPD8x3O^gp9AKvG{Ul1x@$5 zf@jr^&mc%ly`gnd?zjO5lJ3M}uppMXbHe(3(#qRsZOmu|ZK62A4qd@yfcUw4fBbA+ zno&|&ZtG04zBna^kYQays>KKHr(zL1E{$x4ivm@(T|1EQ_Wfm)3$^;)(9Ptv9jdo+ zlR+LtSRE_;x;d=8O)O^IKpyKFfSPv}!{4AyUbn{{&9n`?3jeOqe|`xiNxlfgwG`kW zOLwyd>hObzgkeSR(U*Y@!bU$u76tj`-{G!{7ha>&`o`0J&0D|QEtRRe9NUr*4fdm} zqmf6~-rjN(ROSlBtFbf5u)8#hwDET4_^vbAPE{Lduf(4#Rj$Ozf#rvwFc|ytw4PC? zF}Boh7v}RRdhzcG-nj*37w}APX{4~ubBgfI*aLQ z(wlb!g-2uxevWW9E5&`iLUpA5iRUT$3cbYGH-ZAc16O*(R%(T`tzrn`(A(=wNN846 z)VTx)!*AJaA{Sa+X@zxhX*Gjy?*0xGM=XeENmVwUj(ja}L85qKZsTr&NPxP@TUGM2 zU#e$-Uku3+&Su_iP!*)4PY*g&#s?q#ZTT2-Bv~#`hIQF%s(ZLKROcYwxU?WWS-8Vg zt+VQW*~N2O*GF>@3+9V!cl4rR8Lh9*t#fhHi?o{nK?TzlmzE3qEL@uVF+aZEZidL5 zU5Xs;{GD>YXDiFROiHy&4Dpph@(Za&ErARDA3DwO5{6!q`^@_tv)d#<-fVYAzwBAU23%{g z(oBiYila)e;tYsIA&n&++QE;!3XQ)Ehr5~}-;H}G3js4aLIbY>xSR{yZipt`ZmpHe zW@8|)hxkz;b_{NA>vjxh3TsER8C$|RUvf++QR#q`ccoypq)2~U1n z-^CZIIJ?Z;iCL}(tm5TK!CdT~IvZ^^ zXXr~63@y=vMIJ@SbGimF$DZ^61(DKo0Rpg#qryxVz7f2R&l9XmEo%G_q;)Js7_`1K z5HKXs=cefagjirXf$J}2c0s^YDj5Dsx+cK*l*u9j$pm-*4%9i%5kb{kSt%bL-tli= z3h1e&Vjj>M&B8zxY8RxYU!*dJ$?eEZF9*0FJuSpPr?#KX1hLM5djEO+;KNHn5DPDK zw?Y}KNd(*4?`1an2I_}>QYuEFF$UeU7ZtD;XE;j@LUlNiKU6R^5<^I$$VXUI_G2NdWX9$XzGl5YoEea(&gpvm9jq3q+Jxg= z2+m4$j*iaV-2xkXK?+M8KSiw_5o*NPn)QKW1eB*aaV8_>Z1B}(Qq`A({lKH>n3FPu zH5>2+56ZmM8{npxtzunHQK%DMuJD-n6YS!!%t))g5mzv7k!eM}xS_DW7NGKTqsHP6jM}y(ziKMO*aHe2 zCd|CvgVu0va~RFCqsxJPUDzJ98q?X`wR2GY21aN38g2)EKZYWnA)Mtmr^qH8$jst( zI7To9*;%T%t1QV=x>!B_o2tz~{0aLqS>n1Y;EqW66saiss zxbU`n^9{?uH`dY?pP!AoBQ=_Mj~m zE@nlP_4l~9(#AY?eJTc`^-BlLUU-3=Zyo>i4xQ!*B&e~H^sod926ds;L!Yz2#*Cyk zFdJajL+~7CpK?aZVSoFmm>b_EhVRdkrArkw>!=F68^qy(sbwmD9uiYz+Ni7l^k}2y zT&>N05VBzxGXXQ7pdkN9a@qI3DLuszdLN@sW+yofngAX#qv&Wa;s9Ruz-u7(wf3uF z34eHlT`g>Wa-pG2Ou&7zLF^#lp~0V&6n$)x`v<(*6J=}{yVO3+xjJ@r$a^yIy8MBp zZi(=diw)yWF=yU*J7d1i+7#XHz=!V)Q$Jdk(=qi&V}y$7Iog%pRV&A>z#S=>AbMx; zaOs+&?4VMNsX7|^_}ev^vvjM^K`od`efyVSq6^#%^QezrXwf@kQ-BahD#cv4I&-j7 zBtk&RgvqGZ=)ji>l1)QXL2cME_hi#^uFVD;2%Tj$3ji9FIvfx!MDk7NqQYE93Y{wl zw*a(U43qfuD`o#5k-7XFxUxRG!IwDD(*3>)3X7#{=JjF7A<}U+C&Id1R9m}lwnub~ zLJO;-Mg||!5w)PN>34{nZSh>Oc?8*M+le3h+Mnd1YS4^Ns<1k&O8t2UZ@H5Qwuk54 zv6zupJU1+($k!uj#93hZK~(;nBa6r&98pZU)e?D(3!e$7T>XL8jq8D~Tys9-IfGb1 zBh~DkG?f7(<+tsCodu0}$H&J@Z#AlLyzxYNcGwR9@ ztNyA68|pODSMPD(5`4`(dtLqkk#z6dZS@xl$S$>7{rFm;`(@-SgBzO%_lsexyj)8k zl_!3NZ4$wb3)^Y?!0cCw79G2j(x`^n8olcEl9i;MEP6G@}wE`l;U(55kX^(8?q1i9?LG%(d zpv7IVKkJB@#p7rDpK?tj##lOi_#*KWO1WLpfcv|AleaI?AcBYr%-yU10}>89&li+V)`eaz$J4oGBe-7utTb9W?4;b_bpfMu zZiGU838DTu1fUR&b+_hV{#ov4&Ss$fQWSrA^z564wG2pYd-b)vetBO-4&nK$=Bdf1 zI%DGH!g80UDO|Nxh(7@h7s0vgO-nvXf_~qvj_#aivd%}-EuLma9f8UcX>%9>W(kuB zkHKEwv?60L^Wku;8IU@hJ74H90+eOnC7^odx7!Ax)X17Zax?joM9jBgX<&q)sTOibXs&T z79$z-goPu9(Fnej&U^q6y_(DP+r6tEegLt;R)y2SRS&uF-M5$$Ezck)pfbBm7@p3- zOZU-hI4pHG)C!(DDUEX!c>v>vf;>SIrW=JoXtwC14!`J zr{Usxp|PD3UiP-q{z60M2u)}4dsf!0jUyNl#2PQSa7zucf$Ov(1JBkbic zgpilbYjgHfMt>%VjTHMt|6SAm$G7AcE8Vf7IJfGi!0&sXPtLNJK6J(f*U3~JEr=k) z-l3!T%Yn(SHZF~rQh|SPLbfIC+a6|df4e>bHK__&+Z{u&@3!`^p-C^Dv8y9s z!qPSH=*lu1+%uRiOR$+T@v_FIqmxSEheUP<7Up`>a1t-1#_maudF8)ZuOuyV zfIab{^#W&nNxiN@&ZCF7Xa^h~Bt~bSvWq=nypv%Q4i;|j@3vk#?<(13xe@8i4>!F9 zFcTJ8n7)9PEj9zaA?KcWLJ}d9yxP~4AT2HQt*o%YFB1At=e;OVGbvEy_zAL0FIn*LnOqLa+BpXYKo0 zPQDa<%>tps>RLXEm+48Nogun*$ecncWo=3(4gOrgo+`^>diA>H-Fjrs8YUigoJV`v4isFBt>dH~qc65r22NPg$bIy*{_rIk zWRRrgji=gE>~P0UqnSe+{!c%+A|!N{8rj+&I=I+O75}|D22;O_wLkSvCINm0OcGs1bhh77wExIB9I+9lfYqkS&!YxAysb00>@gU;1p7OZ(7uU-qilSb0!2Y(QWkH z&ay>&u1VDX^xlg~*Q@77e#WP*#B9MkRq=*DT0Q&z`fs}*KNK5hT^F&k|vQ>G%Yd@ zLp}P{D1O4a3QZrq`_a1lzGvxTQ73#q2RnDMI0el{glS8%HXQbKG z%;yk$*ih_ngzMn2Y`!PloSzFjMrVdDX!|P()<>?H=`XJOckW!Li2;k8|(?EBy1cqg% zzl-LY3^eaSTOTS|i!!@>__buy>tO5Ndhv!u!tu|?lhZZiwdP#&1~2*k97$qi@o2P5 zH=(Uoyr`JHlq53!Pg+PY0>F?~V{HFu-DCEXrc&UHgHDX= zN-Pkb?W)*fNoY*vK+rn;vMn&@=Ci%i*Ux=T;R#Zh;Uf%c$Mu>8L4OBHwHI1>_$5r1 zjCWi(gr;+$Qyrt{+tH%i^~*<;)r>NJEsPQrz2GS0yUt1Xyf!T&9;N-BInmQPJ>99b zLn;yk1SZEnW2I*pnSNzsx%ohqTxZYXaSd*cBc6 zQv~vt366qUDy(i(ZH??#u8u)OJ|%>pny3&>o@fZi;8b<&mp9BHO5*4nnfe1Q?$slc8y-J}zi)Yp^}-2@2rG zKZoQtx&ge(*+7BXoEsNjfj&x*Hk|Sjb_!9{d>+y;?@RZjY}<25$W2CZE-*KB|fq{YVoo1wo}P~8!bw%pG70w=J+BQ zxj`S-{}Hz%T}%wpAX~{^6!&mwZaY;_<3No~@a;PL^8Af+B_)r_7_HfDB!_?RH@Pje8atkG?Zi!;461AcuE< z?oO*aORCb!?B#e*&roUWnmBXjGTcbg9GWzC0)xlL&VVVi1t57g)1`@UzvW!P+&=*% z7b!e}?M%=;BdYP7^zHlfl2*??$T>`vg1P+zP3KW$9{cCFYZ`~QT2-}onw;7MG@^*t zZJBu-D@M*(eFGb*AlMr_*8;YkNFIhFZnAmrR=G+9t*+FLX!npXW;H|AHXhk<_ApOd zO@?&L09U&^4a=twIsBvobiX;l=)4KjJ7z@C%vT(p#iyZFZ>_O+)CkbD(1lg4r|ot_ zr_6TI5H<)5f31-Xiu;S5$u*c69|sP*^v)nxj75GozKek{e|F0BYF^*S7;|6*H(II^ z_^=tAvtRJ;jeP8j$=}C~BW`NVRUh`?U~J%S>qJf&vG<#qkh9E@4|rZ`d!+Jfig=3E zDJ}H=ho_eiJ6eD%lzHH85!|z|Skq#XPQVxdt%93x)PVe=M&#w0|0WU&dr(q#1XD=6 z>8)v@3|KvpsspMbof`Wn0dbqx%R)0*Oa<()3sc=$q4zX6#1p*DUcdSqtPbx?hwwXv zgNlyPjjS1tzNe4<_cu~Bn2KcF?d?C>MwYl{g(~AH$wIz5E{en@WYBwG#zXdH7^Tk~ zUewU@_DXdE)3r_nGb=k0AXS?THg}R@1PwoN!)3+IcrKD;ZH?G;fQQpYj?=LbUbqjb z^LLz~&ruh!IB-J#l(-!_hzy>v+a9&>H-CgH{fY?FMn_QzeNdo)Zsf*u{P6+^&f6cF zTE(@#HYRZyb{H+|4gGj}n?6w?oT>9%!)dB76dIy}mN18#d9NwWsx4RjFZfONL_PahIJ_ zHaJ2^qU)9ZYS}kRfM3ePqax#aD1a5El7@K=1sp{>TC?s6Q zS@@00uPGgkDH;nnFaL-<9B6=cvU<;U_b`kD1B}^0ShEPkH?|M;xsVd(qSDet{N)-eB8yuYKU&<^|gfi`4C;zg*=wEV>=9U60F zil)J>`I_>&Vh|%TcwCjMSXsN#^Nrt`1h9_^51rk2f%Qz6!>^^8D=?yQMp##i6fsP; z-s3Q>*KCWy{NQ8j^Xs5NOE=L3g z{^P!S^d&xiCq=l1=^W%{gs{|Eut?GO^lrzEnzLtRFZYQ}_!1cP^8(CN)?gUUu4DUT5E43selKOL z7h4aWoy(Iv7==G-I|v4%kcdN6rPtrY-%i9xVLNRzpZ!FueCr#-cSyDgiV0JPaSszy zbO60XcabB=Gj2zO?MEe~z)X@x}48D$Bb8Z|w*?Hb-e}A5MC1n4twG z(#-I%Kh@S`*x`K+tQ(s-Rjr+#%*ATbw`OTJ9R~olJ;&@dXKj=&;Y+L2pX2`f+n?Jo zL5w{JaANKouQ2HgQqp-c9KitE)FKdy6E+ZCXmxg7Yl-{%xR-a4%l|sunvAc^%6xQuwoB?+sV{ zYpoxGU&yD_-BKl<&dnIv*J^M+DVX=YxZ6vLYu1^u8m?UZ@|0#jKWP7{!mp_iAn#jC z$VlnPp+LV}5rfI?T#0V&&P1!>DImqj{JjH$>B=DT@Y?*Y7nw)V#FD_Foxknv@$S>F z9psmQu_^n0NfA683{1UZ?i2Lv24A*vI4Y^b1D&RSDP0JDW1W{(_Cz}4b-p;Ohhlu; z^=gF{)}c7*@gRkh`<4Lcj3MFC!I11X?@;C(>iVZ8oVklUje+eP_L^fj55A%G_tns) z{6gsm^~MqB9(H!#z4tHY0o{YEI8U8Gf=iDjVVXTeYCM>`u2}OExRBJ9UQp%Y zWs1hzc7oPFafWBQiUp&TyaKgMfu1_oS(O>mtMBqm=LNN^wpTx~$ceU1L!zTi(gld^ zzecEu)dR)(aX&J1GvAr>74SZL-6suv2kUH$&dp3sc8ddSUqblmo5;^0n7oxibe9^e zPx?}C=3e5Vxst|I4Ipa+C(vq`qo?fQ)m=eb?E*jabdHVmM82VQgF>R3xiQF7iao*W zWX|auoYS}q^h}GIk!`FFKU*l`3Ze}$xb6V>A3;JF$Bdw%e}VZk_a-@dYgKcaogj6| z_0@hjhiNmf8J9Bl8MqyQF!&lr!Y5Vn>%)^dR0(bI5EUp3C0&?0)WpY@bc%g>SR@k?v0RxbAtKaIK5j-<=QE}vd5 za1gU!<6U-;=FReXh)7U2KIsLvP+&#H|Jr`bhIvp+#PhpFk+}F=+u35VX6!G4+vXoW z>6~q-77|U~w*#6`+wFB|e!HUI*`GyP4K>JwM^Z_tJgLPOUq8Snr{WD7RVM<`m0Pa)KGn;Cw~{>e%9m;Kmo+CV!by1GXJ`B}tzc{! zZBg%8<^4H8;~K2x(VI5$Xk=}~a6k)wI9kX#-_$816`a0f1hy5=bD8Ns~i9D2F9Q6qS-_ae>uRz4e`oua0CB8E(Ryi3n z2|OLNn`@FB?Wn7o2cc16-FY3k5#F|#**e%h3g};fvQ4x#oz%-ME-F!WqiD~=9V)(s zJ2{qR`{Jhl06`fNOnBclh=oj0E($auhIGf)$E_eF$<^Ll?gI_8e&V%VlEY_nHqdcJ zj=Sv3BB*Qu&^}L%aQ9^P1AlUno#Qy_}-PbtGE#7dA zS0ux`T@aZ8+B$%>Q4~!;C4b!}+mq;TEs{a5M?`kMzeYN?yn%_Ba%%PEGtu0p%6nX)up z#3CJR)GyiR$Xm{)sT)zLCZBnO#7>dGCBplDT>l&Fg5Qo1{@PsGl>1`2_k4&SdOHgV z;wUTzy5OzfQSI-&AvL01Xr)KV@6k16K@?~`Eay=y0YCwYmMC{apuSo6*BH)10~`Pf zbluS0;n);mH2t#CeQ1dPK?`xc^J{1yN%yBs;L|azx}6x!y+m*h35HV3-&n`vzmurS zh5>76j!(bQ2BY{(_A8;#Ai`p*Th`GlI@@SE^Nm$|fGn%=S=xuQL8D6s`{U2A+;_oT z9+@EUk=rYCU}E91)yQk9B3!Ti@poMBc;WiVcigwzfZkqm935_<3m_#`eY?9V@`fhh z#T}hqYCD>-6;h?i0`Vl#*TF(qdUD|p62)$J{{&;*1Xj_bYN-_oe zPRY7>BGMYho8!YG3?Tb*!6&o7kC94)Z1UB&0e?4K!A4FXJ$v0jzs}a~_QLf9_em|& zh}&GPdgGT1B)lwCYAYEUJL6@c$G6DHg(+D=O-U}GF|dlv$icT6eAp@_aFgsGq9&l7 zA1BO1$?zEIV*M)4T_g-9b3W@n$M;$TX~(iZv=rrwH4%kfo%Ne3$EFPN8I&(LsX@NlVf2_n;b@wcS{8jr)B&AxXW^{|m?HB1rVF%8Ja z2y^t^eZekRcMOy0n^W6HCJ2vyoxLQZg5~wIPx6y_%Sz*+Kl&|h?;(6v^MNEPG(yF} z>OBZyR%8X&<%8%CW9m)FI&-C=}bS3gcIgJaTo+MDbl;VA=uJYo4 z=OV{i%^<~L%^*n;ReYVSg+}QuG7Hi#FLe1*#Aa8`ww59Eop;mBNR=YU-+?0b_H6HY z%IxpE!43gqVPIqFW<&Mc4_}O{s0$xO-A&Og>0WKojlcQQqTHl`avi5YW$t%$w-I+X zlWVYJczCsy)(DIhr$OnuC$k8YkX|PjSxMkzu6EQ;PmHQ8{QD7FH)qO>w*S*bR5X=5 zR@BtrRx~If;V}9rzDoV7hWxyu4WY8OE}`L#zmub}wT?#0j*oNS`oDLewGBHNXjnY% z6Ny;UOzZT8$-{Y7L2Lx*l>WwQQ~ENpfdkORXZ+4-}+j3+R^{^_5-(c-72WTp#ps) z*YTo5+(TXE!7Q95hJs;#=dM7aZ1jJ$=VdeBQcY4HTx}70IKCdnSHkP^$!WD+(Dm-hh5Hy^@cYCY@)+3&*fS*y^38ZyMJ-6Y#RZSUmQ| zwMNry0i%LcupcAV9EpgF2bTPO*e9|D+F-GnQtyyB6yncy7wt{e7t^b8TK||5ZcpN( zXa78k;hW`g7XLg<&u!g%T+7#u%3icXf*(dWT$6GbCl_wM0gu_AVg-!{mhqm|KhUf& z311va8l9dY>7WF~PF`sl4)~Fv_JwR7Rtn*xw$C31*HA6?HUU`Y2vKAAy*XY1C9}!$ zBH9J1D1S=yc#EHL%u&heW`4EOJarHP7r$(@kNKY$TRu>I_@5U$Lw|cM_Q_I6$zAoh zJ^?WFA*Ypls?f=VQOe^6=7!KKcvFF4!oO}L<5aWs6UqIKN8 zZoUO!ANA`BX#Z(au8o*^%@KlV8hNl&=U4i6mGQak9Ru!vd$mNJ@!Egbmj)*M|6vNX z48F4b`$F=a2ivhVKI=*qRr~&6IPG%wOW8k$k6$90{tx%}XUpb4++SiKH3gGbK>PNvHmo$MiFC$%52wbol9c1VK9tlAyi;TIzfLK?O1&Aaqr~QZ`rK!-a0NmE8NI)larF>2iFha6HC=B< zafw=hTLppoJhbHRZOPKdYf%PSFC9^qZtgjWNu#dT$XD&x*sM}lek_(Pyy@Vgg8T|X zoeb*vB%Z+mG;ga$3nZ%VD0w)w6{pXcsQlgZr=+zr;_up!*p}<7nv4!kz8OM6XU=4% zq+Z_(x|g4{v1O^L9*au!7qV7#&q+VLR{%26W_CqX88n&X)U|axwf5_hNd`(Lzam;g zcB1G1HgnioJ)hBB4B;;v0Tp-uuX~ZjV13BOT{!Y=(B?7|KM-Co7fAvR0&*r2;gYcI zZB_8D(0A+XuQT1whuiF7>dN_^-FxDL?7*984#e(GDR>`b{V8on$|&TTr!wwkEWqwQ zqE-K1D%q5SzT0w!Nfw0l(-+|;%AxVnO~^wEvX%-Ih_h?ookD)f%-tS4j0|6|9FWq= zwTjsL?0qo5yKovV)e_VMqRguSowg!tdLc|G?(Of{Z|Kum{d6%WqULkF>x*x1DFn^2 z#I=rNw8jng1fZLQRKk9g5zLUI&^jW?9Cn7aZn4K9vp_G2)8YIf&8Fg($%en4AIXx& zHapFVj@ubhky^D)d;tK^D^%%mJ1>NKjP?`&thmqRo;QeDo&vef_s5~~L%FG#-{m4~gN zjiLW&(-g~}H=lea!VfQi*>;nDJF3rG9|p}l$yemq{Bb10wX!gx1EBzok`j2v`!p5J z$NVxZP<7tS^h_m*=((t60L)SO+w|H32-wJd+*$4zn0~YR*0ROD>*L)2PY?P3#469o>Gav)i*;oIR(E%epcuQ9Ad%m-nUYH~w_*?>;_5VyD!s8n5?XxElq zyE(}Uni`G9H+S9%mpW$zm1_PqSnD#maNfC4zWvv-9_C?h_z%lE-+yIUCu);McYV!1 z>b>KaQAt$MYcsIirmx=;O3G2-HzZIJ75zU(0x0j#xFAH9p}sIGYK{vnCa zCc5>Dq0Jgi^Le$9zvPj)26vaW?3 zgii@U8pGR%?*G5=h5zgw|F4Ha!dF=r7cP?%!|? z|80xCtMV`T&#kIW38Aw~PygZ9?^6D6@Du*asrdikQ5Xg@tQ{(4_%1)ClwRhs@#T~J z_VFiG)e=YI_E2iO!YR1UVd4l8bL+6m$L~W_Kvrw84@nTJR(sCRb(w_sXKuaAs`lj;h#Gx@ zOdFLTok@I;L%OF4KfCIZyv5@OD)M5`Cf6;oLdMv=;R124jdPggpRkxy7-!+rPuHX= zSw0{_6)INPQOr>!_&0t*y8zL{(5J!bB=InZiSn+Z7@Z!;@o+_?{CS8Gs9?2UHvW{t zFZ|5Sd~gQq;5vgW2w61wU7GNzaA0RFi9NQV(YJ93` zW|~+)2i#c`OkRfNhL#ABnhbBk!mi|{t>FeIdyUBk*j8LvVA$eh5gdG8cSk}9|;qaO>85}fcxlu&Ss zMe#2u@qv+I9Y$JTAA!dkpOA9S`7cAt%ue2Ofzm;b56MPu%=@>p;TD#cEkd%Ew}I2! z^N%A~VxSZ(%tuDoAO2^Mrm)9Dbizv z_Q|`oJDwB4{EU6Cx0(lC!`~Qg&|%-4FBfQjkMN>ol_K9>BDQ~>S1!>5y=_8DY8y~; zW9$&M{OguL4dnT}R_*h?;yUI-;@s{=g*8@JFzaY(=TDzm5&=I3enc_bgSV0IJRv#e zZpBMA51+ywfS&#I4`v3qvBTNmz6j@-o@uR<#h`25s6)%xHUe5<05s)as;wo zT*o6VBxw7ThxiZ0`gR%T?gvV1+Q5n zaQ1#Y8uhrnnc5Y}*jd@+vLNrryf)ORosS>mkD&XML$kBmPM~t{@n;VltSc(T6%^zt zB3mxo#d_E}b%Fygz#xO_KH<Fs zjS~0!y&_WFr>n5?L6PvG9<+gCEowq`bwb{LuAD-d>*H(32Y8$1AL_0cCL2c4U#~8E zH@9i6t3fxq^#}lR>t+DYYOM#~$kyLlsIyCd*5VH1R{uou<^J#4}H8bpj~h3vu!d>oLYr~oD?roZO_az#IDaR}UmC%SH8s|Qrt`TiU1glG5yOx*i(QtZU^TyljeAef9u`BRAhu~m$w|{s4 z)8w4{nX{lVIf3h=y(C+c2s;|AS-A&kQ?Ka7GZwA~ z+N^a=-1^reN#N?i1MzuGai0UqqtJhlUc@9Z3pc12&JGrk>}60S<@!K&zQi5N%qZ%G zV@-MKI=R?2kr$O$cH~=r>9|~k9;b8`wBo8*vZ7c{VR?2WBN{-H@7Mvm$FP79N71_? zfePCQO#v#)fUC`N$7Z&Z#Wcc$t9y=O&jad^vA501B^}1Y(_D$*epZ_7xXd(}xxq;3 zbmzeqXV8D?f77d8^gmKeq}wepz4$+1HF2lX`rV3qw9#j*H}i@oWb7`5OXN z#(bQIuyG~hDPXc{<*=QY{ne+(mz_~mSP=tkd>VG;{_tl&57j^Jvgc~24^Xm9k{zs& zM@5T#y9tUWu7jZl_+^wJ5#Z-8*AcjYr)izQQP&(~|mV2i-k4)6|7fYVb- zGv9NO04~AO+mXjih7bBFn{&k8N!R7MPF@4j(b=DD9zp=9QT(vN6@6Ru%YU9R>7deo z)0*hE`weCz@|d6I61PPLh`*A%fi+!!8zkv3o0hv3_!NeZ4YtIJ%A6MqZSQp2Ze%@A zC_etW&J~D-YhRi)y3+WXeg>|W!|aw|qt)N!;ir+KX9X%3`r4dnIyIBdI}7Z7r;SWt zX^PMVe@?+D?aVS8-N!GWU;J}gJ^8F5;NsY|Z$LiRq`|q_RO%37AV=GUuNS(r`E;7y zQN37fiHOidYVFbKwH}6iCeDmnIa>bGwS?8jKxQ%C`}hm7xXto>`uRc;%Z-D`Q#^cg z4aQ!TeURQCf@!0I$HjTWSc4sASrF~B~Iik+rr=Q$>Qpa>> zXzRXhTO3`qx9Ce}#?n3>Z%x|L--qs1Ig(?xSo*~NH<+6iH|k$+39#|kuQLPy?oJ@wT18A(z`QA2H{fw+{+h4l=HAy{?QGOtQNpzi&N0+rV+-2W z+kbuz$QLI>dv=R8RZW-e-><+=Z5ys%uZF#uEnDv1K}0;yU+77T!)efva3dXj?Rbl6 z#6$^Jch8m?=h6#hwsK4X*3;K^o~wljL}*cQ^E5KLrwvLm5}3+A9%Y$*c<~9 zj@>TH0bwAo#J2&+c{@hrDn%&v#O+zM1z!cV=gcpV76dUYva}kh!i1k{eg--2bA2(r zP;Fui0(uj;h(RJ`4;Vsuqde?K;zC_>2*HIql?K@jtO=l`tqsd++ZO(Y6IuT(l1a>k zFAuOJ{S^Yf2vZH1CsvU+s+y|zBjjmkO>!3mORCS2R8P%BX_xvPxO$TjrVK>obl!H|Su zP}2M`=$OOq7Wo=rcGdRS9+ZnG8>@b&jX+%fC^OOb+GadzuWhU!= zhL%6$)E6;n7#zFb+dsfoy_TFYDGGqcU$FcYFU$LTMa-eCx!rEREt{(q#^3JS0wHIEt^U+EQMegIam^7#INZ7YT&-gqsm307 zJ&Hjfqw?6@N#hkzw9S4LHH}Aq3q2!7$!nYK0cZ9d)54X>^hcYql@mSV^*SKP5=;a?dOx(7df<(XuN z;cd|?d50fS1N8aLn4n<&MkJy$f=a1{2$k~0P%y5>B|m+wrt< z3$sU&UFmK3;_xbSA?KAA3OeVgRH#(_&wyGr(EW-nAIgjMN z{~yxcGOCI%>;si<5Cmxs2uL^5a0mqfr4i{6P>@DCj&w*!OM`Svhrl7Flx~oa2I=O+ z-0}ae_g(9*`|*CAIcH|iv-j-T`+0sruweZO{|C+%PfA}aI`%ita~A|PpAO~O_6X6& zWjM;7+Db2O8gC*gQ~~wiV<&whVVq^0Y+( zKVs=`DSwGcBsqc>f-MdgKCe+bxhL}*@o%c%z5;$LvyrbtLHd`ZMBoMXiwp_X#)8S! zzFS1Zjn&sB;Kk<7B7HlorCdqk9jtKAqW?M0#%HH>)auXl`hW0h`Qy&%8^H=|iU#N~ z{3(Rn_W1M_Ft2p-_zd^Tpy^@2>k%H#mA{akOY@rA@mEg^qD}P+c0So*Nk60({z|#a z9QpDq!{kdyPTr~S!(1mzDmWX#Fh{^mLlA!ZYav3>yZ+KYl<50X1$z&T@8&e^u>7AU zFq0qV@_6xkZ|{}({P+=-#Ido0uGInDcciF2k5&i}WPdf^0j%o}TG2(t$mKKN^pwEH z#5euG%Up_rtJO|-X#AmlLZp9!@6x0k#Q zS#q1yboMxm5rPibf@YBv;@FrkbRyK(3rz^pjf+kSg3n?pP|-v4cj8y1I23b0b?Y`= z*0?93r^D(IC;JI7eE}@{xGA&>+kl1CTBDt)1DF~{5E5;B_4fdB8F1wl@Y_M1g(8Cv zvTLpV$?Nxx7d5Dtei$X*9_KgvvPO z$F+UqFgBPf+ik%uY@yD^;%K?`1eJWO{p`+$Bt7I#g^ATjcV(lApYVaNm*m%GXpvf4 zFt|)+(*q|G+(YPX%q%V*;j5fvol7#*efW!)qxhT>paT^k<}iP6rXri{SEhK#-c(6% zpEF1I&KtQ9dgm@1<fjZ+)*XctB=6Ix=m#8ZUKbt6VX znf9SRdbWqJs=W+zU0KptzrkeYb1QjXrHnVWKs+UpAQ#JlmQuj%(GB_y)m}3OX-rxk z_eKYb`8z0qick>?Am9tDFzQ=A#LGl2~E{cdVpHLzInswF6jK9p1T zG3TkXiJD3MP|)g}NuNpVDf5=uHD8e3WSWQYx1T)vw>hf32?(pA#_!KWRlQ9j+kGvy z0CG~9zBw+5y>1Rfc?=}Xe= z8iN9a$A5k{VcrtWdt$KU;yU8Y+z5the1vCN8MP*ll84CW!6pd9E+%uMt{s=!?TGwO z!?H#l8VfH818f)nSDxFJlxeiz0xN?&BeXn*&IW)7z1EVq8D!r?GnGpVj z$6EapNek%3p3>@hq`r^BZCtpjg0kse0ie5Q66iYuNfsb=d5l)6|MJgtp3g{h!-1j- z&~2-iOWr*LEv9H1fLZa^9IuD;WT)uVuCG?*R@0|p>|wuH$&t|d7%xdBE5f4ft91dI zYQ;kC&+7rjX%2l4E+jIx)=k*Tm2#LX>b=f&+oQQSp@Emq9YbU|S1JJ(s0zdHsr7bn zX;H>$XmbbiacyI$)x0Ms$5(vw&UeXIB^E)Lo~uoHC|zHoMcAQ2AVj?ym?$aZ5O}VQ z)Fh1FxjF_31rf%G--K!6?@7TIEXYo`qTWO5Q?Xx)i=P5KIQ{O30pZ6k2!XSy&d==XDdC{@%Dd#t5^y1r_`v~~ z(gPd<0KS(aJ3XZ@iIAF=|7Nm%bK8M|Z`fNt)k_nTm>zqL8&W@lJoSRZ)X%h_E36^g zwI8f!1q5TxA4nK~>Jbh(Gvp|IUO^o)DgR>{?&FPxjaDhYFTMJ)UdE|)>!Ad4%(CxC zF3M1YJT-th;1ITzV$joIHx+|c-H9M&{KG+X0lCQ?(;wpbu&SI_#AW>&7$Ek22YY_4;Zsh~G6>knprMv4LN`ziB<;e4} zF+VOr@GL1W$^pg-HC~3*>pu!atqwYeJ<~fuzq#n)CpW(uQxl%}D>PG@w3X7Wulvz8 zdVlL4f7ld@VO90RcuU(yd!s3|A5xzu#VILn6Ax>CK)n|qH{0+vv_hEwZlmrld%$6N z`t=!TJd$>2|9CG(ZZ`ycSr~r5u9Yr($mMgW8AZWvIRtvt-Hk!lBJ7|&*Q=UA;Usij zA!M>$oqN8vc|aF{u0lKhuA@S)v_wfs%-7k7%)I_04$E=Vt!$+>{lH>98k01#(Eyk> zxRC$MdP2YAd;Su%z{T)C8ye6md`He1D?+!-W45)^%xoXovfh>1e*L^O<$lH4T*cfy zr^6}g&YYVka%I0&R2r)|ABxv-&7tc${W};Qb}dv0?`jxmbxtZ7?ZZ}G*?RAwenxAs((vROn8@4F;O14|Bu2?sOYknV z-4wcf_QjDCPp;S&9-2A^DD1zX=8yd$j`~dc?A7-#&2(jWs-aF)>&Qrh%NENr|MsUO z!`B|~qP_wFO9u#lR@j8kEL9{$Kg>VsL*I^W=W_`!8p&|fF<21_2j!Q$LCQA`e`Lt% z(B>zJ7j(p|1K*JAnh@HJq#PNF;R?kWV=axlsnALKXMa~qTq!vzqGbv9!U zz0St6goswWH$1+Gz)1C~U(=fTp`J)UG-*@R^(>cJH4Y(Fz%b{Tzz+x$Xah2Q_5CcG zr{AAkDfb1~c0Zc*`>wA6kv7!t9Ra;>_y4ZcXU3{tK7@`X|8nU&Ze~~a9wB<F1Zkg>ntfm|4%K^?^YkV}0<5n9}f#BM60-yQ3ScsVpallK}+UQlBQ-l05wk<;>xPD<3MarC(4fj#Xm z@ZxgpZnG)Q`JGgsi*2FhSNm85mNx9g0o=1{abI$8LHjX#e(=FuO+19oaOPY@L0hB_ zDbOiML$~%~JP_p^hB$(g*^Xs|)GNUw&c~!7ThqAKry*!_L+Pef*#30uy+GtC9?jM0 zw{iPYB;E0FxdKAv>LP%R8;eIaKT)_1Xm(tBHDqOLQ>Ud>XRTVQ|Gn70t|+=*m{NLm zLq5>1^XbzBjeaoE6bFpuic`C7|0FQ1-{mj} zV}dUZdf&s&ejLqKf?|Q_R=r&8gxT2cJWQ%nn0dqbp1b1vyEHW!?&{k7rzufpPI*zu zw7+0GpPJb9%=)~VnO~Mrm^2hl(_(MY%d*@rXQ*OpTxNcFDex*8uV|bqc&)DqCT8;0 z73YY3wgd69lFdN+l`$Q?+9qXrTRMq+nJ~_!=KIGFtjAKRaLJg6aflvTXZctv!bfri zAe+M}((ho;MYK&`j3y|`H&nH!)hC?1ri+V1wk^P0b(r@Ir!X=4F+H^W39CDE?_E zXpC;t^}1FX;}or31s#GBD&h$mYgdBUcvoH(VHJ!zi|NzwHH2ovm!{*@mN)fcX}1(E zNVAY_8naEy-iZ3A{qqjl+D{WwbaA$l=q`AFMEb@!qof}pkUf7kjM1=b;B9Uo-Qx}7 zgcP1vxR+yX{dNi^Q)o&)Bg$}6pRn`#!%RG%QQ0)ts3TYqa*;>y8`{6PqVV?Q_!9c?Qq%7U^r7x;k!hgIlJbbYwncdSs2MNOgydR1H)FRP2kH-1 zY1LS0d(eh1n%zrxmmy&Om}eI@{;?c=XCgE7y!{4-ffOp&>h6iF9F7vG^ve$+!rOQj zn93V;n5%PkyxjO;do&vh%u~*eenoP+!eJDq=o2wk$_Jm0q4{Gpd`u0nv$hBAK$%V9 zCRZo5e9VTx>+KoE;yBj$Z{$z**;R!o1A}=hb>HN%q-opGC&&AaFGd-lR6Lj|vaRFY zZ?$FkEECbW1W<3O24^xz;6;9!(qmILMhb0UL5g}BT0vALODu{l>wHAuu|N5EY?czK zF30*ojhKy0b$~$8`RO#QN0W>1zZ+!MZc^RAUp>O5Z9#pn`%R@)uE%BL=lB=Ys42yY zvKmK&Z?FLqO+T>WcP;yFvPDSd;zY>>bhYjRsk&=rpouK6p|H~S3%A8`F~;Gy0dgm> z3Zg@Np?5GF|2l|lf$!-1?;auD0`2!Npzq{s^PBRt)5z}6SaKq<4PTf1kXPa_7#^O0 z@iu@pFsfcdhwPh$WQP4AJ5BhvPtWhhhO=t7sJovry=fL)ulievwLZ3crO9>M?7W2Q zbDm8EmNBg$LcR^rI&`r@0C=L&f#N-JjTWI?Q=z zp!xn7%nS1?B7xH&EKAg#hYP;=g}8ugb*IB{%o~8F0&s4>!nGd2z%H77@lPi-tijZ1 zoyKMA;qL0C6w%w{D-IH5JhBV~eg7z_dpkzz_1hN?_ho14m+NLN7&=c?>j${;Xgj+_6~fe^d3P#P)Lut=Yp*M15cMxG^sa^A1`MZVO;Ti)t(_MA|hvRIe^8!kKPR^Q{T2J=$?qa@Kg)QGTE;tgB^LbwOFOQG z*Vxl=S2|1zE%ekSkjSpq^KS{~EV+EseJy5afg!m}9V%!`003W4JEWN**W=kViBFcl#C z#zfhFYjrVK9w-a~cWoQ^nB#t*-G`*x75j6MLBM;Wl5>Q!Q3F(>{-rtQc#V#~_HGwx z>Fa`+J4?DDwYQ-`{iP0qqaS;DcolAq91EFM_DDP<|3e_iuz zOgJE4y_o*PS5gFWFQ$xUCHv`c}ei?QLj-n-Qb17}W90^`$M zojwnz9zN<5iVJlK+suWnO$;z?*-U@;I^53jWgh+7an$DYk25S^634ifW)1^81C1@- z=+raD2t{e$9hjT>{p8UBd7^m=A$eeSS@8Yri;Jy1Ac0@OgMz}{Uyb~=0VX$pgKx|< zuz6>mP2^I6^FogqhT$-!7WtayaX6B#c+zkUuyWnPrchr5(g3uyUYQAd?;Q;6gG*Qm zhPDOqH4OPt0Y$b5^$gu?2JZga*l}`E?*(!-eETW$gtJ&pijYr0`R2^~_miM2Y}Rc> z&lJcH38JA0@8ABQaj9lox4K?W-fpZS%k#+cSRYi9e14h?H0&Yi-*^s~$3Fh;*-i-5 z02orMpOGzCgdC3W^9H`KGR*w$VzdaDGK|Ymlc{i;wh@!=5P$WrRs=<`W<N31;hQF? zw7WX^6@#@2+XXx)E7^d>%OUYU?D`5hbp8uiJ&@@n#CKe9n-Mcp=Mi!-u{Pq5F z95YhbMqbBT4+}ie3>*GLp^CQZk5&Q!#`!iQ4@-mllAsfZFtIcE<*K+VeOR6Mupbjb zj%pg-cLX3wp9x^y3EvygS7#L^2)T+7O?Zxqe{B{Fa(RpK4h_i)ANwxmDpUBeQAC10tGC;rvH+BT1 zG>sSJ+$~_H&JDI2IT9H9t{Z3r;GFxz-*Iidw5RVpXy0GOUySeD9g36`qhNQZDmUjf za0?LTgya+fk$nq!exAp{GZg@)6pom26K`a}>^|kVz=LIz$+AxpzdJOtr6RTY45^l7 zI(5PbT@*mPHG`&8g?4@M!?9tC-Q1anB8Tx}p9SzQ;dam$ZxDd{4KuIFxAA${NE6>T z&tG)aSp)Hq2%C|#eD!f+r!DeivhTQFHN(ytSuz9*b=J>rsi=KO&xamZz+V-qQ4JHtK1LaSW1Gz|+u2h+O+8 zy$?8@GQ%bO!4#0rFW_puGX~6phwQ=8!8X?6V6yQ~MN@=`DG+Y@-SZ3j_#V?_va}}u zU<`_tmIG8o)T;30n?if^^oo{e!{m~A%t^U7x+?AQ8yORzuU4SF9RhuGqDpHpYlR}5 zBgp6);$S>q$;EK0_++sH2vW}!-L4iIhqd7Pcc$J$Q{Rz%A%kPY*06s><}-bMccsa8 zTYndpqKV_Xr%VVZmy~$wGe183DY5-puwwmsr&L(`sYZmjDr8tStb944cXMYg_zVlh z(E`}~pk4|@-F0U=BdM#~zsd&2RHUr6&nWaoKO5|;gq`pFiumn!)rGuux4b@*{r2wk zk%Zd~gfkjG>bz1%01o?pb*r=OUqsJ7Jq(LGtO-plQrp0@Ce@~GQemRx*C9UJ86o<> zy^#+iJ-_9@=v3XHd_RlqubL&5nM5DI%(Rdr9b}O8rEGFo=)n?6KPf0OX%&`;)S<$A zuBPy7|4E(%lc#o>PXEbvy!#`+b@Op_(L2EX?>xa5X=lv1w{cWL9T+wJ=DZsC zMeYqQ(tQ*E?|f&r3^thjP(>Yu$|++7q}SVqTRr=Pad`Y1H@DUzC|E1cLLuw(7XStT!R5L->$){Yiq4wYI=RJ+0_)LX|6VE6e-H-=7Ic6)2#Xg2TN1L>|DPCtf z1#kZ{?)jN6GOjePe`{+pgfXZ~mB8#~Ldr-=B-Lqq=I$~`1A70iz`YI^35clq!r9?` z+gHbFxv=#+4XA_-xao?vKxj8P7;+jkF^~>6<1J3Nf^JX_B9pR>T3ZRqhZ|z{5uY_4 zE_QnGF!Vv#2B$BuX2-mZPF0bvmaI+MqK^%#O?(*OBOvS~dkp0~@`^7HoXNgSUg8eS z2BAEjxqe6tLgn6xo~1-n@!aq_9mpxfRR*CphS6-qb06AUpR72;@>x_Y0PF@>)*LV> zvZO~Ph-NZkm8dRNJ)aNFUuKB^JpZiq+b>n-XD>5Uyo-*EoWdQZ1=~2O#fWjWrVD+D z>z$-|#g)YDX-t!e=y9<)?I&fzB_EHhQhV5D`W_Y9jNF{5+{wGzcv`+!#hDO}1G6m= zel1&?b?1p6@)!Sjhhmx}6RFFI>46vf>h9i#9|c}$DGOUiZ(Y(8Ny7m3WVlYxASqCe z4R-Fl4X53kyRP{DA?WPJY8Ad(C2^t^dYYv{p`J*aUhvX_jOLZ+#UYEQU?8OO$4LFx5xwZJb7F};2_?? zbUDe(3^Uv`GheQ;NnX=7QR^Q;KPqoj(sol>EydyJ-a(c~EvGmY&aP*zF3&w_^uZ`O zl>a;O{O?zj`E9n(UmyQsZC}?M*?Yw1sm<@;PVrnjX`6gynhSo9bUP3f3Arg4qiV|I z)Y_eLlj-UG&Ph!=_6qz-ZJA=LBoj;RCpXK?B>CtJ(fE_%IP~6o-m9 zDi9d)CHPxB*|!i17I7%`m5}Sx|1J~hJkf*W-!=86_RXyd)^VTe zJFhJ&i5r-R==XM7;JN(RW5|Z;PB!w))X{YxAE$+rx+xLA5bP%Wn%x;U5$l5R32=m2 z3VsBii-brcFi0%iZ8+(2A{CRg*e?etH8}0tr`84Y zFqT0-(%uJ|<7|4WD50C_kBS6Hc7s3 zES*-pGGCT<@*EYeWrAK$Lt3sa-$Y&SD$JY!{wFb;gdY7vfs_He$0i8$xB6JzvfN`< zTLru2o0Jn!2>h*FeCvZ05`SiWd%YIZUp7_TC6=Wv5pegS2})8oLEHoKJk}gyqO}Tq zUuJc@#st*PA-@Di5;hoqX36a!+3(%}W}UvOpHjK$a?YqSLd$t!yjvsCAV`yOlS6oc zgvik~6&!2Nd9@Sko7)y4C^?`ri!J7(=z+fp9VCc5$Q$fCaEJKAT#-~>kD7V>CgBO-Z1;1die2UEr*2DR~uAZ~F_^s9Li` z4=jP;k*&30X=0}JMqwgK!qG|r9F2#-sC2JOtLwwvsffUNwRUuxePY|wDT=$=9-H2q zD`4EFa2-sH^1=w|4HAXo*6}B`=jp z>Mr!$!?vuZZrz)a56|4Ow|GjX?@qZm#4aJV`A^0i@4Y{{KRA}~`pO)1B{c~a(Ruub zxcVWVQJyV@QlQb-R{I%l8mSMQNar)_GLq!E;G zB&c_Fmd$8XsK;6|_hA?C8{WMV#}!k4@{uJBGcc~c2EZd__{jb#t*a2Mux2#Ext4 zV`~RB@No1RFX06(9;?s_I$SzM^PZ9?22|w^u_~3-eRpMi+=V!f5B6^{LB7fw3ILU- zH(1{iqrlHBZYPa2X<#=bk8oWWMU4mVgUYbG0XuiboDA>LX?iI5x1(n#+e^+01>v#N zvv|rhaLOiKf^Paiq8rIcIGUXuHjZA;#6**!&}0Z)-rxuPH$^?prtdw7ZY41hVD(_> z7$PyRe57e0I!EGa)n9ppW(%i?PHiFl{arQqql@b7A1Q^~uLK6h#0Y)c!Tfho72_S5 zZ20(TJ%&}rgIl)qxqY9NCP2Jphl-Ihclx{zADid{wI8_oAGI3#9EUo6e5D#)RGGeB z&hr7`2h{@s8FlygsNDtNY_oN=hUgB^d+rNZr6WI%>%zj1R&qpobiTc2(t0RvSv)nd z%)>eYCrgv&H}7P)Zoj2n@$sW^ip-@xVswElFV~mwd<|B@nPK>!R(&@lOta4JL<8Ns zWEMwRKA=ar;(VmvR^z)_OhPJdd_LpG(7K`#MShv5c_!&@U5LU~E9SpxeLqZxlzey% z90r|@8HvIoY3~NZ8Mrh1l6ZgwSll19E5emvJolT;`NJ=vh<3iNB8vu-Ox)LMX`DD% zxxT^I{=#9b%v%H+e}?$XJEV$okqWmLGf53-ufNL}JL@qf8C%M73TgA1tQs=kprG50 z(R~u_-MNgJwlEW?aF0aGW8wkyAC4QxAqpCo)c_gm2Uv zKX1)wc|Rt1QwzKWCBk|ann!BkkdZ^!qbt~}gNJH7C|$`WJ)&$o@N8AcRzzJe<$L+ zO}Y~T4;rp;;yhDOA62+*+Ww?bsez)Y@c4HyIxaXhYswgqnSWgHORjd4K>Opjh!5{x zY)Oz9vx>y4y)PrkhfPqvC=}#`%>39-A~TszEKtXh@`^}JxP&Z@Z}Uh||5lOy%v!+O zs`E){WbjPgRDvFWZzg`Soc1eNYq$3n&es%3mVPAcjRGfQ+87WpKHq-o{5pPw4^Qrt zt>OLD54*R|l_~Z0neiZyv$1Sxyc~bfk0|g0{{nvK`EH_)opLa3H9-VivmkF&O@>*X);Isp)chORFI_E>=9k-&(4IA5@2 z{mxoc*p#cM9n(9v1)K4@XX<8kuSlvp%6-w(nxq;zMbEbG>a_0`qdr@_KiPTu>qk4Q78O^5!&(|P<_r`?QUQZ zp0t|2P@HRunJXXqf!z1?=rdKRO%WyKaHo6Ou@A)e6h8_+q$W=ulk(KJnoe`QfHdZ_ z8}?YclLw*t(E*6dts+xRVt++H`5(1#*M;ZzGW;I%Q~zCVeehv-o;=6*jDRy-WD!iW zj*haNMZgF9;*pP*YTpcaYt{`+{NOpkI=J1zQ+O+5(hW=qsQr9>M#hB191rAgc>8)s zBYPc`wiqz{L@jSA^GxKy(VMq5dFUnavUxjz6e3BybgHLlG>e{2(p<;=YnHus$AP{of^RZO| zDk$6r-o*~nr$5es!p*s!FudxP7art6NZMMobco~dm%yZOI1f*aOAAedU6d1Su*n}& zwe-yOSd2{7*_(tJNgAv`RU&PgxZ>fa%98)*!~f@m9Ovr3Dl>tqW&)j2nv(^V7Izy1 zb_lnIR~v;YFp|n2GLrcujLQ3u&HAJL!e24`K#$M)dQ?d+w$zi;ICQfTh@~+p_!{az@SNyL z3D>Mimd@c2BcML1+3ny=U%1{yg{JoyP%rttgZBr>Zm$1XdMnShed=qpdjpmbI z#A#x!#2H`N_$kH9-@mqez@nGGVf>$`|6>TapEMlqoM2BV zwmAO@0Zp%F#%h8vIpUCHYfK-&(t2pzjm8Nm8J|~mikFU^XztTeC*Di^qZ+kmk@HG| z9(MDcHJ>mCeJImP(Z5_ptOFyd!3bSY5G?)kp+tR>LkFR*^Y-bj5*&jyN4flGMa+m7 zUUy#fRCaY2T8%6SFou#pSDg4vLe>9YKo0*WAXL8q@BNQfiVv}QS$jm``X#9`da<7W zdY>fh!Iv-lWJ0w)yUYGTzm%xxk|EdsL9@{ovX8$3d-HJHs*Lf*gD2ivJ#4et6Xv*8 zFp!u4cTwF?&wsHeEO@(gd_t}LWV|?&a0b?5)`Pj!#9A=ooC7&{%7$omUd4EYI>n>x z?`1AGsKJ0DZ?If5nwARV6%g?Zl9<;HTm)XfJvc@~?TCrC`ML}VALKJKxvdQ||59MP z4M9gVys510p09=V6H>WsZ=DO;{b8dy8?KO;a^x|Y*|OiA?_~Ykhw7M4I#~QmR7o6! zO2HLQ1jHT2?}K1}7GAT$wCH{dgi@@v@)aDcuHI_|49tJ0G{@y#nG;wa#3hoUI5aK+ zVSYpO)||QJ&B^*n7Z%y$EM&fCtOp`j#)l$qE&r81_bc0ypM#h2##^6#sSfT8WC>n9 z2t1m105)m4L=UOEyKh2tV(nL`7;bV`)#ojfgYh>x?YN(6)~*4&V~Yhg>k_i1?h_(O z_b8;;u#7Uoh_$g^5U@%|=K1S2xKb6!O?(awU%QuT%M$UES+{49^bz9)A%oQthg4y`36fW91`Ew8Ap8@}H)aRPe5U+5u5V|KP zN|*w*pF`#HKlDxVQVtI!3)Y255TTY#c#|J)fy#_yC*!?j8#zvz3dV~AqwjbcKD;FG zlfg?+JDV!Y4>4yT-WbsQXd>y>0~)@>awI+p&1!YK%@t1>rhfUYFs_C0m^t`Nw+dKj zea|swoh9fblgzK1p-EN^LgXx9&7D6$AHJ(4iruLavi8f_rM$m9jRL4eh)-~g8Su}= z-P>mOK8@5S)w`Lt#>oDB$9~?;H(k$_)1Z2Fpuw>MnGAE8#&0Q!QXm>VNVrvJ6e)gN zW$jc0@jHWE&KR^%AINxLE$mExj6Z%b;Iks}4lq$~xnhwxf%2;wW%rP9>x~-Cf&wR7%f074|44sbT$?TNPm&L4>=adN(^mix!>b>gy zv(1`r5!kRl|8kH19pSL}{wHS?y%0*?0#*<|RSP3uh8KeCeZiZs?l$F&P24sAoA6Sa z3Yh2$aP`e=&13oLwXTe711`#{re-R^yzlE(pDW`~aPR9>bJ@I<9;!!Nw*tLeK4Ws% zyvvj4wl{kN5TVTZL|}YfX}rc9`bFG}>0;l5X!Q9F4yd&fUNW$)NIs+!_frArb?Ooy zh)Sx1`-ZtT(imS+2N6uMUBqB4hGZ8I3y2Xxe9imswoLfGY=_Gg@(LHfG%uYojTa=xswo z9$LEGor-E2IvFlO$jv3Y0$pY-T^L$;y`A5%$95zFQQj>S3gFU6FWy($A68;ed-`?q zVV#uMubHJX@Upnx%=uNGkqB0|>?}b@N_N{<#we3^LR}KP30CrlG=55y> zcPJfNo)V?~iu<=!J^HYhqzOugA{UV5(Qiwx0xP#Ed`aJ++4m%Ef6~x=^jr8G5K!rG zeNhX%m3X&!llbG?`yCA$A%C{zz$Hf^5HUTZrU^n|_>7xENtpYLfE}L_?}cI>vR)xQ zz;E5+XDGpgwGeGcR|g1sbX;o67%1}O&Oq}eVM!iBNpvr-hx3vtDKw{CsUT@%7bu!zIG%GeT(P&dlGNa%Yh73PKqKxOv$m9loqJlfW>^2 zcv}m3GLP{?Ccg`zH@>(UO2AK!8G29?cVQYnQ(>?%6nfp=8!tn=;c8LjLN3K6P>0ej z@E=WbdPE%Y)(oAv`mh&=+T`A+RNlBD^bY4yl+vW*Yd8j-=(~GU#d~7RVVlE7ITb~MA9LfC1p4m!&mgX0)p85MlMf1@ zaBH4YMMu@sj4QNUfRlDzE!8xc+AWxV4RV)kG+C0hrW1zGmcACs_pGr`VDrJ`j)cDKEW79P--YWaBb#9k zUag?aoaPen%zROuGea8zRY?x50ZJTUc2l*T&r_mQ5Eb2G3FF?;05##Opo06o%J(+H zI9-*P9ptgtmVm@JKrGBTm_-2I_b9EeNPpEI zMPlNA%1)!1d8RMa#A{sHPr$|^>yTj8@BR2_Zq=WpSA}?Ir@6WQ9GVi(_mY_)YV_Mp zN9Z<2@EMA``wQoo=D0TtcfaTEI626%z^z#`Gx|WFuNacl&SvfO?6N@caTF>va;w6D zE$F4a5FLk)Dox1Ns>htEi*(K)kaz5<4tEPj8TxBdhQPjs&dmVCIT~z@Fq0@v;}8+Rb_-Tu#m&u3?^;6CvDU%=c-M= z5d9IX2eGU@3Kvv#*rxEns@W*xUeqDLav|yKnNf9xw+-SlSc}n+r>&_1=A&r!F`vyr za&NFY;6gAXCATL6RzO{xR@_e|HOtOU1J}9&&4%kAq@Ak)el+K|^d1T}l}3#eq>xvE z8VHA~fg+2-csTlQM207&sQ+|B&{GTrTk-c<2YoU^K{@Pq`*dkGl?Bo2RcGv8#s*{U zXxjeBH?*`mbHJD~P{t0cHj!LE@;eTL6i=uQMRC9G#-{~FfkKVb|k$at_} zTb`2@h$?97u$x+4wyo-J>wqkA?}KPunU4X9MuDAJOW%_Qy|59#z=&EQq5|lzVi8x6 zwKKaWaXu=(8jfv0bFCs~|O+l--V>z*|2p9z?$O}!_xI~2@blfqTj;P_Sd1zUNVstE%XJQ~)>q~1pbej} zg(Itzf6&wH>4-XL;YRNI->m=DuLQK@Jr#=zi>&-_R`;}Xa^UazH?@NRo5GuP~Vo&@%IyT zVb_C7WPa>w$rr_%zzo8z2fN0kwT_8@>5}e={pQw4<{U^bNfQ6Ct0e#$l9(kt3X?^C zUfYp!{S)d%k~&f1WnKdoEX}T&WZO06Zu@xj>nQ)!Y`M7-e2aj{s>RiR25xt92i|(m zl*ErmPB3BO+CFC7=qhUQw)$X~-|A;}yb>T|GW$SdoF<75154!Pf8ooaG0oU{;F(uza(hE#bQC2c}j! zwtr>%fUCC)qelCFO{m-EcQep$u>*~Aev|pk_(K(%%{TkzdumLad8bxp@aMyO3R+fEqsU(nrs|F1`eIA7OiOykph zu>%Pp>aM4KyOSS^wSyy=0wWKK)y2xl1qDTR{~PlU3e?@*{Noqd!YNBV z9J&6Jy$Vlilp?NfRJG#$ zJ5h3Kbs=s`a$Emi9WMD@Jbck=`oCHLUWAYuaG^aPv3>r~D*FdSvK{z-4zz=H=YF{s z?yDlc*6a#TvnMtFVT3E}X(llNma6|eB-L;qj-_b#;@aHH{drB&G6uS{&H1-HC;xA3 z7W{AI8lJPW#ftn32FHXDsY&!F4&}7e4~7Y){{>k2|Du_o_N@h%gnm9U9PAh3pQ8VF zLTbty``>T3Fa9sgiQ3<6pZvSz6d1U)KYyYoiuPV4k$?FY%c+C^4Y|5RHw-P5)SA!Q8aAQ9s<9t~BJeXzz3V_W5eqO%=(OD{XWG@mpQMuI3xw$#j!*XAb^x z=QUF@uNeg>wntzR;9-z;SZFIgfx7~FABJrV#CQLs6|n%{eEAXQT%T=25LG8i>~F

CFWIZWHkNW@p&h)ldEo=JSFd@`D2j0=GwgrY+ot=ia|C;=+ z?@Q`0GCsnhAPPXrW1pDErFb#YlGS1~Ye6vJjyO+AVs^u!p*YPH_c*pEphs&gcb#Gz zoQo6%mdW-O-+wB9Q^$&5W%T$x?g}z41<|{ImurbS@9y4vAN*=|u^rNR-C_SEZDy~0^aM#HARiQr zvwl7Xzh_i!3?lsBEwquy07&L6mMe)YcMZ=h)!=&7Zy}x6M}FEIKnRPHSx^MT7vIH# z9F+0Hc`^YsIsL|j0v8Y3k!L*LTqYi61vN9JJ&OOR&fkH(!g$N1Ck^Yxp&$-`&F0kb?qjK+nlxkJ>vI>}^wXp`_KHT}ZQ}Erm`$w$yVwyu)a#U`*&NpPt zH~=m=vKepZstt+yKc#&9nZ8_=(ui9X!YOo_eg1$hq8x=9JE&1a-Cee#TEDG?eS|V& zpm+NYjYdsczd5$hzHMRpfoj}$K{Qh$n!dZf$6M5OC7S(}c6zVa*!Md4mjr!vKm@5^ zAktv|?N+cscxMm$+ev!nNwJdaRWE2Wv_vB-IIJbvWu_duhQ>Pz!GrkUw|$XIIuHlJ zVHl4%l)yrPyO)0S@snl0szRMGR(Oa@UoVg%GnSAm z6}?@1)%8aQtDPV6lzMEby>QO3Goaxiv=vk(=eW?dD>p);x0u~a$!<& zr@l=GziG61g8bnQE$wk1Y}uq(JRo%JK1>}5 zaaD<9vh*$HP0xprIytN~(Bf&4%A+xTT%9w{(_a<2*$ifiLqWqy^3i^hAkbWE6vC;; zOj4Yk$w7vU8hcft{L&kBIhXISclKG_TyEL8f?U*TnFcRHDly%EF8v0|r|vlaEuSLU zO+Df&?Sr80-<|R*&8;(6T-G&Sh2@eH0-CTA<3D72pFzP2&Pa8+vEB#cYf{-}pO0o=E>`UYK zxQK!@WX9T<9SpA@Zb{EEP z5|=Z|8Ap#YgQ*uE>vunJNKJi*f_*--z{;AH9gaBh8D9n@Y$~>rKM!Q2l^WD;; zu|DACzIC#IUGOzzk0DtZ|#27!Q+}Yon7OdX*DiG#XNl$!$Fl)to@n2^Y% zFkJjoNl;+I`3~;xA}Mz`_PW)I%H$}Ada2KUoB|J38v82_O3`BAb_nLSpyw8kcE}Y1 zf8e1@^c)=63G80=0bW6Dp7=8TwK)6Ex-jtms2QEKb|YKePxSYGYs> zWOZ+vuW~(DOo|q3D!GdEto@@PI>|8%BMf;npt<&^C5A3R1Q8H{p+Nyj0Rg3@M7oBV zS-<6e_c_;o_qBiLde8ZHt`*OEV%D?n`~H5uFi=hqnaQKqhkW2{?g^H8Zuc@F9n!wz z>aH4N*=Tl;`i%|*&#q1@wv*$Dy6sIBU78qCQai*&T^V;|IRM?g-wh;6vWgE5A6Pc7 zp+f>vo-%0j(!9TA`L5&&InIZ|Os_e`^7CD-=o15OL7)`%C>1u=-Oq&}%#V zj{_~kKnE5?Xwb(F*1-!O!`XlZSv{j6&h)Ih`&G$ab{uKe;lsBqXMKx!kYo*l-ZW}2 zx%>5ZzfT4p7q{CM48Nwnd;xmRs(?>&WGIs_dMwxP8S|q*byYX5x>V@bA(BWTE{GxG zsIR42pY1?J)AR{b)R>>1BnzF-C@thGasIV2ccZe-v1eHx?p?9$EmaN=v>^hbeM7#1yxXhAW68wm} z)>7zYEj@yt_&ULig9p`YocGAx?UEYguLN8g4Cw1bMfoCp{tDxhwL&J&V{Fz;AYtn@pGaJLm}tKb^XRZ zZVZ9~N-43Q9P`h8QV)UYreD`?%LH9eDhC)r0Z>oAKAva)^>zFUxd~s~i)wRS84vn{#7JnAZ~JW&qqI@H zm%ccW zL;g}{^y&5YuwfO7FS++q5h&WeN0s`ky?8vcMux#Lk?hqT<&GKQea~A8DcAIv<<@?; zUf#M~SWhm|SUoXR)a{k4m5kQEjh=}ecW%Vsy*T$6GE_U)EfiMFH3Zkz^m9JW_Z1JX zjZ5r|TFl55$=hVUnn`<*;RP8sp~(9+)~A1rp$rk&q&+3c-SZo_sU;o;F&*?rdtX-@ zNbOk@W)$JR*zKtEwl9p9!tKWVVq~cZ-EWZcoMSImP7LB?Ej=>^c!gz+d;8^uI*!;c zlK-09hB`9K$Bu3~$b9q9?2aB4cE{z>E9w~^zU=?g6?|ln;)=qrCeuKca?Kcb6VIwm z1D;H`s#rNcSG6=rXTvCB6QF}i?-GUTiLOt#h98ZSuAema%LixV2+gQ(!5UY3u8aGG z*C>bel|ldEz?lS#2RTEZbtIjK3%LSdfS+jJrGjocCGzCfrE%ovnQr^nso8YCKiDUysl-|^=eskE z_Gy;bltZb*T<15d{-{!{OSh;4xz>%__%}A6r@5A^r<#M1yd^4DHANMzK8z!fgH>&5of{{9^SpYvAkVNy zXi@U(ci9vrYSK~%q1gFHJl`LW5LhaL{1g3xk;TO9tyJ09cel>I^GEJNT)cI2WIZ=& zIjVEehwW&;Woh(OULy>GKFjnMcQbB$E8YeiKi5!@@T@z3eDVu6 zd1E|9irJ^)NMJbIxDDs z;9LJzT!^*uDb4*(5K15P{^i6O7)AX>&1*c#csXeHGk^i_S!c0AM=&(x_|A#xxra>2 zC3^GoPe^QPfU8B@u~(sYiYaK1aF`5quRXfP1UG7D*R_j=gt>#Z#P&Zh>jZB)A7f(9 z_XjFbY;R^=Wt2zwmNCB}@8F~?JzJSSI|EOR#(^9mXiNb*_Y7pFZ~Ta7yxAU(y#MFp zHcKOzzh`m%iRNQb^}PIX8Sv42>!M@Lh6+@yoPHhdS0OBC?ndr8liFJb6v_IZcwBRD zSPeyrf$pLVC&s5Iv?GV1S8EzSK^{UErtSRHBQSd&JCq8ddz_Z|$EsL9u(ND1F;KWm zdO)jnUYq2H(p+J=j0fW`3FufiGK+j~bdxZcb+sc0-~qIgK{Bi

5ub8r0gy{u z#yrZdujLMkh^#=6NS8}qx5mVufVg3@Mu#Tb05cW2Mg_N*DL05fP5}I;TOA+q)K`*k zUsFs)y&L^zDZdQ++)w(nVr%F1o6bP*iw;Z!3n90~RPttOa1@Mk~vfo-g{r$=9Qx~MwlLxefbzj4{S{F?!C{_dGjvWqSdF?6cL7vY(B9H_1!U_ zhdG%-P)8~!b{cYbM}(-O)oVk1BlBHt^{S?03^FPK@ec@o&ga`Wa9Tja6iR zN}&{v$qvnaqTHFJ1&XY~_A|o2kJvaUN5QbUb5IqLJwP@rQ>f)VuR)of4&#^FtLPt< zgm+c^YI!BtZx*>)^ z4srTYKsn_f(S4`VMikxI;sHfju8oJ@DJZq=I~V#&0Q78$z5Umo^Xi9rXrw@6`}B_N z+};Wp%U^R^UhM`=#E+&&RqmOlFzuO7yhDHQaweU?@I8H-lunPQ>hW?(lGnVMvmS)J zNWFoiAnI9%p&L?b8Lv;MN0lyIw%-9$W3Bb(@4@cgyM1>?uQsJ>-S z4XOI^8Ww2kL-#dl=WB2RIZQhzl{Gowo|4})?tBt^fj_+p(S)kJ4~q?>OSjz30I5wy zDIV4&8)KzwP;t*=I6R&#oK7gD;++UaFSi5s z6{ES;!e#A_@W-;7(K*j@gvya>(~xu4k4I_r!ZUn@o_Zgju$1Vz1L z9{}Uy20butEAG7b zg9GKAUrUXj|Dvms%5andQTGhwLysEY(MBrEa+bXrh-8aBwOb+hSK$%M$ zSeiuwMyjL>Zec9YJR#TPrlOX>#m4YM(&hPjpM1lk6{(-IUYYP|>RG_y0Q%-h!*NNB z)6uk!@J7juOg(?4_;zv10qa5OeC4T$$I%E|fY)kf2zyn-#_kek(!0H(Y&x$cnF{0q z?QZ_wUrDi)JIb2zpVEhd3aNr#P}#hlJcEs1YAXx4M3goX4;m4|qT(qLN{@FN#A{&H z8^Ex~?ypBuKciyzHlxWoG@5^9HEQbAEeW>mQs5Gj!wbPHz+t!bviK+*gBV!KH62 z@3{#nx`0^RXe1CVvQDQ=N8l*NPHQJbU*NSRJfVJl7(%I0UCn9c?a-n_eRr_>I|04d zltXb~Wrf(-5sOXaiI!!2EnuKg+P-gtgNNCEqVrD%hbpctwUfY~kIf#v%C@E{G?C}2 zFGHt>uUqssnVLrGBb>Dq^RE4*i*L8q*Xi?7baMVnZ0 z&_^75yUB-3nka*M*kc7cnRSJp%0`zJRcC;7W2}h>SMd&5;zqk1&HC1V#h0hw-xjr7 zT^ysL*=9O)3DAR7wNTP!QXdc8qGjZ`DYT#V`S9AqrH=%Tus=J`-=QBRUXs2J{KS0e z$|&_m=U~0}W@Wbp_TA%`P*GtLkyQybO9*FdaTN@z;YF~?7<<&R{?U8;;knbdRRQ_9Sek*!?CC9?l)^v?G=!uj6m>j&^e33(+rxrT zy6oAHJQ|pp`_N{6;p^ugP@t1tv-@WM%-j~?nZ8w}0FOxljw|q;Re6&Z6jm(7ku!w= zo;q*7tvki~RINXbv#3=%ntRUj( z4I`DVJ!}7}-ulK_uU5W)Lx{pt!^7LN+^O9-nHmy?wb8@pry^H_v^GaX!aerzP+Zez zo_8c=?RgLxGGU8w(<7CQ<6%6I=-Ji>!<@3(Q(hl+JTLfmmJ4t$dR|YzXtzNs1egst zwPQN=O95B3XP^+vo;9mP1ljQm*oo&QJHSahT=cnL@8gSZ;vJw}m|6aO*639*lXC8x zjw3cUb1+2m+u^k~6{wUrd&)15^Ee7ChIe*hIw$50o zqE`d7BwNDKMzfel9!`LFB(Bm-)uX1DLt*Fw%zmTvj%41YUOzT*30Q$QUFK24b+o1% z73ZZ^uh=AvV6f(7tH1T$e&Uz8LBIkQ@vyn=x36;g6=$*Z4>7wl_B}1NGuDz{bM6Hm zrD5;*SxU(td|)c(Eig(}Y}$DeW;&t@hNwmnApI~8v+t9qVBearwp1QW>hQB!CQBIl zGt?(a{3Y5u8Xq;`n*VIv;gpcS0?8tSB3+){;I1)$Pa}W6Y1Dhey2qocrtSUsC+w() z*-G*nwc6|9S+4E@`ASp&tTMsjX|-=zy0``|X9=Z!*5#o}0M==uv<~cK^Yi3c^Jod} zAEVbG2q>uvHe61ZG>e%k{#(%%9oBa74_*vcJS8Bc01btusc-bPol8(m)Y8*nN�T`LpeW@CN!qc=W(8_cB-la1Yb)wloe zw~oD$Mn|!xLHj`t5bmHhRqScqlj!YY#;Dlao3YP6o<7u*5m8}zCYCj6C-@5NDdkda zN&Y|o*PSXN?L$Xu)DueM*dP3QQ2jyxlY*z(z6F90KbXH zNAVZ^z13&&caQ>&H~mv7*a>e@{}}Cmj-k*;kZFQd{DnB0Xy!IY#|-InV%K$v$RjNW zF`9?q(3Ki&3}TApd}FfTJm~{%qQ=F090hQF2UAhAz}n|84}&M(LfdF!y2P2lbaIgh z8nNQzm5=*e;WB%aEiX4uau5aD(x1<>*>DRgA_%gh3maw)S^@?llusD zI-&CB$$e3y^YG^ZchW>U&HnwATNj5*q(;?`z@3n~$VF)^>rkeSz{7m87tBILG4|vM zO`g~U_e#fwhzig14lgMQ7Zl z|7e6x_Set(L{8so?%wxkWVbM6nmFN6cxCs);A7kdZ_a?h|L&qRc|O@Y`Wz@Qe*R8K zFB1Ip>yeShwlxTn0i0Ks?1?gWGtB!RB1o zCtgwG+JBsxRY9)&ua{=akz+t_$ed-W?gF-5Xj)SPZZG0Wg1`C(xj*gLG(r|sv>2X~ zn3$p5c zcnaeU2BR66WIf2@)@^;wCTgB>EM#lQ5TNg6FT&Ctv!Q&pII4e>=lZiV zo>bnf-Q%|Kwd?GTZ9B>31M&UP&Ese0SG&>oN;xvH8ThfeX~H8Nd@9{<by_{-(wO+(yj>FF)frsh{}AQE~70)kYIDF#Kxt}aZGGzAr@7!Zv}D(4Z?2j-ux zCyEtB?7u!^vVI!RlakPaA~pw$*5b#*;^llppO2LsUSTsi)uW zLq(Ei!%FQGw}!u0NnueSgn^1zanDatw4OvDCsVuUk(96g4n~;c8&G}p3JxZ}%evey zuL~K|7nu_=)x}1|BG13Up1$9@wrqI1U_dV-I$A;;=~*_F-GPRl&1lY4g>b<$jDM){ zC|zr_);a^K(=hI5Y6;a736UT5wqrTG@t^diNn|~6eV0Hz?>}(Ad2}JYZYh1{(?_*m zWc5*Qgn}Hrkg9WpdvMk{GECe{Fd;ga7JiSTp4$tcb)Yb`P$BrP zTZRIS9~u%of7@DZ9EB$>t9i)GcOeurY5fI6`W+?Cq#~4n;a!TNEnyR z{s@PUU(MiQg@4-5!>b8hF)hE9PmuzLf3gLaUy9}1Nh0#I#+Sz37VvK_4x+&U=p)#f zk6ja}0k`ronXik89o44SI{=PR>a(jhyJHGe$EpF70MW)g-f({*Ze15n)})u zna@3*oIUhP?<^cS0=jP8LYD0SRQ};W% zpvg@{Cy*M@eE*DrZ$qs+3RVSBRIRXZujF;gxgVysv1aZ>w@-uQR>Dx#2lJ98IK0Yl-77f)2X^B7#@?0B;6y| zlq6Tdi7p>d1>FQ4RMmP!B2@2vS0yH=N}-_Cni41_r@q~9pgNl}H^RW4B_PHRTFEJM z@L|P}b7Fl4Z(HJ#m8Y3tMUY&BPp6)S&sh-17SZ?EmYpw%OQMYZRZ`&TujBF4q}`cP zx2_<;bxyf*i&xGG3F>hbJK2*X39oE$;vY80)3Nh1ab4Wh=ZgeRPyAikSE0We)4GKt zX$f|6Jr@IjwB{`j%3YOGuS#tLVyO*M*4vN3o4*TCea2b&lY&8s%B;hMuO9m8F2?+X z)Mfcfn3z$P3S-_$cPUs08U=leNEB)(d)xJAF=_y?28|Y~Hv67uR*HzHi`nK=*a4t} z_*&V%ommpmJDps?4NnJfav4xSS|?jR+ZcSXpmtcUAy>R(8aY}UZ}VaATcDU>`OEQV zySX|vpL86A%>KP^NXdQrG?Wi~ipX*nb%Peu^_EROy1yO4$DW zL5b#=cYXN={}`opc56B%_B0pv7<@vF_`i;OlX3rTT>iQT_JO6a>zR>2B>fC5IuQ0X zn&Yk<)9a#XCU}S(fs^%$#z8lCYw+g@HE{gP9Eh9$F`Ft>Z-H@K7>q zDkR6{mFt2&G?8(SHT9;sRi-hyo8D6_KcbBCAB~y*pEO39ga2RWADxz4Uv){r1Gzg@ zdIeZpqb94z^KmRCHGk(-A5|tjTZW-KL2~DRnxFrJC;!LIY~tgBAD%Ke(&W4NRsG}H z22}s!O8E~R|Npq`zG0C!rt^2(nXccqQq`&&(ZzpCV-z5Us8jn)K3O?~E#C-ja8U0R zlGQ2IaMWGnvpP!MaX2$-tzj0Jp#I)&Zu6Xd5%sHk!h4X5 zVOATlZo>EWhb(!Z$$<#?`VK)lvCh6V!n1&Ta{qp`f~Row0N(uK%fl(#!K3qe+g}VP z$_TzB0u%Q1zs*~O9m$#q3hyEOCs^L@f3=kJT=QieD^|b@wdN8U!rT_f5srl%e;5h$ z-d@jiS2Gj3f1hM;tE3tNg6%CXcQrtl~H0juBY_=056mGvgp-R ze1-wl{v+228qTc|B2=XywbQalA60S253oSY5~sNppoNAI6v?gNLTuq$j4J|;1+m(M zN9z^nj}OLPxBA);>R%rKHy&dv%`b?44|X4Eps>y~u<=%l6M-JtXG!IvwRbU7SkDAd zhSvl^@)GG3Qlec#E2kpo?JU(A?g z>wMpjO5-YZm{fo{Sheld=PzptCHc;~R4)D4cDN=kfLMVyn_de46dhuQEKJIMfM-my@7JS(QjmIuqB5}Y>QauTrZYR}INv+znP}{J z9s(hpv>kH_v5m8`9BA`f0Gwv)@5p(EKSvLp|8Dzmr1bzP5J5!W@Dv^m4G+Z+It##h z+l_R(Et@&nyOx0W8h5%t?0u^{RyJ=~^equV z+S!Cywkj%C8kQLo)L6ce%<5HJ-n-h=rVOck!mLF6KwNwE9|LeW)CrWnn9NFwC(^}f zHs_X|PAOy~NpHC#Ws)X)xt6IXj6hM$k>9TDH(nxBl8m>`7=~R3Ycr~IQ$e-H~c7!{7ycQ+kC@XHss4n_lGq{DOI@Y;Jn|A{$X18%0pSCP(*(ZBA$ z?hU>C$vW8i_FEYzV-APCrb@sK2bK^+{2yT662{z#-h&PyRkL#EH6N*w^m@4oPVKYS z1ho5AmBrxeTpg2^YE>LVS{7)W#vH5^&g0LBe*BJnOQxp#KC3YMbYCf>OExq z51i>am^?oe3TwXve`)0`HFla4V;5+*aIA8f#qSWzmBflMs5Kv@TWha}wJ0ntBimaV z;tk@VfCEvKTal%OlWYI-Hvmkr#lmPXB!4SygbB4izSRRuq#1wdLAF!MdR}~N`6iI1 zJ#L|H=lv6-zCla3n`^oNwGobvxv- z*eAisF6dH?q5 z$?S@S*6#}GV>1{7Dabo}xpQG4g*w@^yjn80c`n?>Vk7GU0Av;f8~^=kc4e<+#%5sh#|3-3p?)jhlQ~}<6q0rm&00=-#~OU5+Z>*%l`ro< z_l~ZVVc~i|>gD(}!{F~sNKd8&hM{#kLFd-i!e0$fJh7`SVkDxd?fJzOxwKzL@jHuD z1!UN|KR%Nf!)NRpf93Nsgd$Udz+}!^_n32VKlkZC=OE-nM`GsL>VpHcFb$>+7oH)r zku-O-nW?a*jjF4@1mb)f<@$dVBk7{VU%*~A$|2Nb}R8Nk)4!&E4`c79MmJt3AanBoeGH@Ub?b( z{~`9?!bh=NT#evsT5Xw&onB9`F-i5;k~%G{UY1QNuX`)%Y4PODEBcJMM{K3qX?Y|0 z#8$A3jm95JSQ}Ex0f3Qfe+$`+nK$T`VDjc`@Kz@XVGp0%S%NN*pAG?kI!^VZ5w1@Q zvsxp!MxT^WG*ur0_?}ATB`A0CFoRdc3njvNU$4$hpE)=;FtfS;7Ct}OfSXw;9|Ff9 zSC`~?22~7Gh);A)W?)VT4yjeZKYa>Q0@b%?GtZzuaQ5^lupvM$y+GiS z%hU*0YQ5r=!zjTi{B?_vw1t1={b-M!(qwDUoN!tDJ4H8oupX^|x!Rbl9Br7qh&Vmg zsnBWJpe;myyAyzZ?lE&@q_04|H{VDq+SrE! z%)BR{d@W{Q|6Bju_`}vK{ht^@@)e-M4_G>TS2bCpguRnOpz4;dz0S9l3>ZRhFTyYQnPL!M^k1KW z1Rsx&m#`HKb7M=zk^u4=@4>>tIH1UKC5k#3l`#9{mn`~fc;#WKg#+<%+iOAUcu3fv z=&CdMhGAp5SK>2c7ibh~6B`o=`)i-|Jf}79^s10OkB_>v?P%8<0IUK+5&9F9M0?e~ zsV!D*(^~e9#<719o`vm)Xw;71uI*h_|+zPdC_JzumC`V4<6(x`tpOOFQa>BS5F0P zUqKWe2P_{CD-EYe@ej{08tcahrozUK;G!F*R}&Km1_`4${(N_!u zzz5L}XO#2X?9%&8bUF@nqABM9P}Pa`z-e|sVJRV<8asCHb6T|P(xk4z^ESNUkQ?En z3#j!5=||;=@6aywFN#T~DSG32^`RYULEVbZ!`sA?jh|Va?m7KP?y!~h5huiHC2nAW zJ^Y5nuhpcoRfpwjWzt5+go3kEAm`}crR$u92mQwCtohw}8) zeTd`w4Z*Ni{S#mY05C~#RNG2PPPuWPfeV7^h#Np+INFIL zUr(u9Av_1=5r9!+y5V)pyEXPl@*(mmOZZb>Q7_Ua#K!z9RJ@ko>TXQOBR@-Jf7{oW zK@C=O5??2QYs49Lwr@ue!dx;hcNq5|cFw-#ZF7r;|b z_k5Ng@Sb@!xt`#|ZcWt)`?7$MtPY$$-k zVst6i%4dDF)O~HA$V%jXS-SRrcZ8aEf1{N(pL%ce4f;L?lrzSNPdHgxBfHR)qrm*z zr+7wHD6E5}HmOtL{YPwduZCE?sjU#Kuw-!ucIh+GQ|nedu(u(%5;q_;ka1Qb2h44Te5d}ZjV5Z z7s%0qB6m{o(^s{}yHd@&CgDYfYoe#~b5pS!WM#+-KKqfG5q#`{I$hLY_aD1*Vj>{c*GeL@~epG`&|V<`wKRnG8q{aBehhqyrc3M41 z9oc-3*2(1~=~{uYa_bipBPI&-50Rg*YMlU3EUVP$AKbeR!}O+d`$Q=OUk0JPcY#pk zMf9l9Ct|llFS(E?eHRwP*1{9v13Tn>!=>t~cYr74_ddKCtuMcba5~>R8WY#=A8%VF zCG(Otdi-2a*XWUjluLRxZ>W&*($CL^aNN~zxrM&ay2GJ%9^AI>8)Fo@e%VAF#3dkGbnlN zv2j*Pt-Oa-c(*r>yf;KEt6l3}pPg@okA2_g*4U}XU&uQ-8&_V@;czz{gKnObKQT(; z+l|0V@S%do)E!!B_0Ldr?ILD0iw|1`mo#QD5eTP+2J~*YlZ|u(KYX{DCqG@@h=6aj z5Q}$|xtqhR*fp#$0>WK8E{UV7V;+31izRr`xj8@2!QofA!-#RfEhZHStEi{!d^b=G zXj*dWk^vi|)X%+m$?IRSk=U5!ln++?q~O*>kH4Vk&GvG7sj0&G0&TyR+C)!J?{Tb_ zi;qu8W90hxN0S$od#!J)nsY?5I4T3yFDYcBan0#6$9kipP7t)vAo3LvUR=Y_V z_9;FDgF9ZxbU{nYEg*hBJIi1!wN~kg*DNdT($P0uygO>RZ}@wpZ3zXjlGT_4RS0du zy;+&EjT<4~j-~D=5gfvBB3+rOMh`DQvO;i8JoY+XS8ygWWF9(sF06?@lfYAm{_DLN zaFK5>kNJLvfB;BJp^uvsW-S2(ovEzKHAdB~j*r4nO&sk9Hilqn(|PLlH7h6ww}C58 zmfMpqv;>M@#`;j`U!3YI_qV>(ArwB`UWw`&hI?o*(Y@?tppT(qKRmYY^64utIh!ax z#d{XGm2Cii`*#x7z5$Px**`?XI}caUAaOTTofxvr*%`rgF@h6i-aggj;&?j0Nt|vy79Xl2ZLR;FKx{%ygP9|Db zkTnCd{yJ>hBeDbS095wnJyp#(9kDDgLE8zAO)W@ zeoHZVbAGVWQ+6X&olBX~Uw96Sf^^9~Sbx*F=6FdUwX39tXBETFK_|;*k92ix&3)2& z^&S90B&*3%RvT)q!z{s6e|Eg^CL5~>^EMk{bxSMra3uSZEy?Hl%_E??IEYX^0+<7GL^Jkepp{lYrCu)4;;$EvzLO#PKq8AAY#Je%AQcb*k#R)vNH&ThNy?peN zIBk-%H1ASc7pX}5*DACbVuOt2`Bj>0>>w`_*InT-%nh{>p3Jf+PG+ z(eUtUYa~pLhB_&1$*(KS>(0#RQY(4+vHSW!yg`?`3208kds8wDfgPKc@T34Un6QY^9hlyZ27$KD*hAx;;}2n zZG7_I6h8sF?1&Z^8kYprS+7l^n}%mUNc(v$2{eV?`1h2iKxLY2aLc~$012y<7H3nS zlhRPn`w2z&FQ-?ueS;q$CH>To zs=E40jW0nCk??>M1fwTg(OdheK13IqXcF|ByS~WnKu_#EE-%sA;BH5!4fb3jg6^%~ zlevM;4}Ew1Ys|uiF`LHcH>s_EIPF$06$i#{U!PyyK%%2hCyn%ut0SnYo^k4@Qt6{4 z7U01s@*=t`s6(~#3%xd?56n(kRcmG|VTV-*h0VBhMZMQE-zX?GKR!vB(3*qDoXgc zoM1v()=hF$gK*=ocDBd7VIn3bjwDpKO#6jG`b<$7@|f!OG~IW_JL#tzq@VQ5t`O3} zJuw5iHh{gZ;X6#KJPIG&%lE&_SIkL^0K*ST@K7`fQ>(i{$rT1(q9oXoY8DdLkSDVX zh);`KBhPjxTO5r9hjX0xQXw1@ufT5{<5rm#S3Z`;tLj6YSO`!^)YyHJ@bUL6!2E6L zatGBJI(1%n4y;0mi1M6uX@!I(&tihK1|Jg`5)S28WCo?08ETf&PxAJ~W^fo<(iJ~S z*VB9ds>(b6+f$Bu3ec;?XlwdA-{3w^dqkXdVg7~7J>RrMaa1kMo(Qe;M#y)c6O0D0 z^pi1KKwlsOHxf6L5F0mySSLql+#ju7+6Dua`!)+lYa)Iq=>Uw{dDPRo3q+*i4Hf98 z9!Ph-;Sg}WmFojl3O}JN(-n5;psL;)FbB4Wp-su6H2GwtN!}UF0!3;WhDP6j+CM%O zw&!459}p<_GUk~L5l5gjq1zyynPNK27ys2_^+s0xr1Si)g>ETp`j-oXcK_BarnxLl1+QP>Uhc89au{bCB;( zB)aaU8<>z-AvguJ8`>>vhLr5p9-BlzVcNo%HBrk-!_QF)-@JM9L3TNKLcv44d#L=Wbb;>F-tQ&Eg{3lgb0m?j2zw?$KlUZe*J(Z`EcB-D zovEZR@&?1h zf@&lVll}$r#6ZhsSVULwW!{t3132u$^y&IYqFpzjRf(~`s5keJR{Zv)N|#=NCh(>@ z1D*Y$c1Vo&^D<^%vOe;yC2JI;d$;K5wJ>d!J_bX2tlM-01Rwxk1qC^bJL2ES|HjN~AFU4t4AXvSMLb^>^_HZUnWL)G8OVcEK+%RnN{a)!EhD-&dcC zD;(T`s}@+B+Yf7SYa+69I)SZzdH`P?L0}sFieyEcW*z7v^?j!VDr+MEr0@P3_)?7S zPm#s~WzwY8ZSu)z5tF?}lKbI~k=>Zz|Dzve&Hz;QC9$Ns|VaLqBea2c_KCuACC*feEFYw^hPf>MKQ~v z%h-yMtVV(A#kSp;sYO`oABEK+M)#=q6Y>HxnBl<;2N)2m@Ol?f>u;p+j928{H72!0 zGwH%KujnbrBID2*Ev^64{KGiI&~u!GXV2E5^hp@r^J+F+)&XjNrIiPkrM!O_(j5-i^7G7^#InIH^dH@CeID^i(Br%W-CnZ2JtsE)ZVnQst}Ug;xQ%R!q$;&u&u-_#!9j0fRj4S zaRb+MwA5=^H(0m?qRMc2q7dq)NAa7_pe1JZWlf7}Jz{pR=Ul(=2BCU_2p9lkN7|+|EOOW7cQ*O6 zswhRtM2pI%`|Y)7VeI-c_Zz&Qm+(D(m$h~7OeU)N1T{D}rR)y0w*dK90^6YYaV@@B zq1^n@vCmTV5a_7x4$u(&*taR+qvcIztK6B@U4*5xtF4X-Bch(VeY}I9xq(Vw7J9FN z!RY914@lenf;JDoH{#vLtjFj0c~>*48%9L5g0BkPy9uf$uBf?n3-Q%<9VP3NW`|c_ zdPzq;j4d6ZeBoHchz))b*4fVp7H{NFa#V6W?MwKrDczN^Nkt^?}zQW1m7a&-J697RmG@3_$iogT;etrdOa>LC@DBxz)27z0^7@7dS1_Je(Cq_0>VQ zsrtCUfo4SIW*Z0CovG^YyF;`#lp>YCUn!95EM)~Z;ksJ}q(1TeMeTv^0?MgdNb1MS zmsZ+eQ5(N}qFV5-LpVf!D$(LmPbQ4GE+uwoAFBQHp$zP zc{u7Sb~%Qz=BN|BF^vIHWb06AM4#~Ux5H{}HL>o%^&G$qLmhjNhIOuk=(&gw`*Y1Q zzl5)dK*Ni45%eM0{I$P!p`ZwqeE(evd{*w@Eaxiv3VpP(z*-7|^cKg@Zk5K+irXh1 zDILPDf5FHXDEl`Kk0+|Hp}9hqB`P4{B^2qipYRi~-GlkCZf*lRZhz1T=iL?i!1VOA zuE}mre%+6HbMb%+bEKwGR(~5JuN5NpP`(Xu8~djJ{e%J$HNT1VcANwES2T?3@UbBH z7buPML__3ldXo3e%{5#st^MN_MDhF3_>=GHnp%&Aglog|_*DhpW)?pDGRvjWPRyt> znkcw`jx@Rp8+M(=Uq=gLQH&bg=Vw`5ac%pNP%x_jyf9j9aR}|U;?Jn?dAq#5ZNrAK}*N4UB{O1 zzrhTjGghHdJh$S{))H!M!%#dDlaC`->6X_&0KGg*XKXki4-+r??m?Qi#Bx;qE2>JO zz8Q2JeoK0#dZgx-obK({CocV2rdCFz`mYS?6x=-tuMh` zkO(A z%|4N{oAG%DkTHvo4*{w}b6gG&+!$m$NMdV=vkW0ulK7f67ds1#gyFVb=oc<(<8+L07_R{#(kt?b*+Wtu z-rJvJ?mV52v+(tKoQ`$IkS4o%Np?Tl#f0UT4S70;V;hrBFq%O-JSv@HnKuiEfPAri z7&^~KHgay{;OGt`$#~eeLsgF+PGwAFJVV5qX4GI}t;M1uIBW!O89D3*oFR-}OZD@} zBKvI~6--BOY}JL5U)3GYR=cwf4r^&uMQcutS z4Vs@pAQ+uDWs}G{zeOB6*zxO+godOYUTrKnDfWPXWgXQasgGK$fHkOi#X<~2o}5B_ zXgeN(;oL=%=+=ff>>&Vk({q3zQ%reWs;pK^A>puat3z=id*dhOvpej?Rb&CI>ug=e zW4AK54+IKMe82kzNVX?fYvs@xJ_VUFkAoWwGb8IDpVF?#w|qvMS{f1RCiiSAaHI&i zl$2Yv>xUrTeBWoFd2gr8SB%5@qqUUk?4I6fh{qnxMj1*G5FH24^TriZjHHtPf26&2 zTvXAzKdd5M(hbrjjWmt|0@8wrgbXR&ARsYxhXMiu5|W~TbTc%9(jXlJ(mCWXOuUR2OmwgKw7G1OxVXo1xN9%H`C z{`g6t!@GRuHFuFX^0*0XAXY#Nu(9a?0`}jap_g^!B8x+Y`TTI%qYZp`;rgoD3fBX1 zd;os$t_&3Y7;&eugAe-mW`fJ2_rtcaN3GaLX7&q7=s~`vynG)N%D{(K>=X7wbr3dn zA5C6gV0Grfp&;X2bQZ`9JEG%hh594AS7bk}O%P{&Vm_UEw$F>sjS!f= z&U1tGQ_Uu(fsFAfvFth=4UpzwcmSbxMg6TACfz|3tS-jwzJfaz=MA$)Ju-fYJK;5Y zyr6eSFZMkE!JOSO;{G+aXQWPWNV$oNiG({98I~JveQ`bTCe8`^V|=HTP!U5!)b(1a zuW$J!PTF_$QA34k!2p%wDx|7|1Q382ev5TtCL{!5C>j6qPn6rV{V9A4q!AXfcpB0E z)rr5qkr}V|>^dz_7If-up>=dQ-e%~iXb?uF>C*IbuRi-oou)6QmN8Rx+f$UCOA*uW z(T08YjeS7$M??%TwE(j9dWWT3cN5z{_ml%P!rMTy0r@Qa2trN3t&tuUZM@U^ELdWzN^n1!cQ{sEW!fR=RItyJJe z*{|tyfrfU61_I2|@(lzgxs_1nq^$79L*^JztmLymXkotKsa-HD7?XSUW_lwTC358y z1O`W;^t!5s)_>8-?3Uzl3S#=d-#~k)qCbTF^avD6LRArTXuMmj9_YeBAj>WxPKqE5 zA8+tJ{Lmd9NvFKWM_fIZjn!NLT+M%CKlV+}T|+M+J1w;f80Eaw=WZ~G96t^SdHbzh z5^5RI)n4VnANgj=X?lMX>W1tJ<&uAXJ!!-I?FUUfZ$oRRs^?!A9loz|pHcLEa($7} z(@RJN*b?KhPsY90J_x)n6&~p>fK%r1i3OnDkQnb<$h;3^b+_J~4ofQ!B(ZoL)w&fJ zRr$i2sB}WmnLP3b?;TOp4-=RbHTL>%t)KnkSka}d5LV9PFCkAaWWs4PFM5V4D@+b! zoOX)28-4aohJ`C5wD|eK#4&%H2jblxkhdxD8@JMvsUTRycB02%CuHu69s{y+PeHVY z%u<^JY(TbqA%2~QUH|WQH3{276Z5~KaY-d96$2mMC;DUK)v!#W7Ppv#5wND`zD8w$ z!2Zw5KdZ}_?O(J>_jHEFB~BrZICX4f=hr0!3!>h}uJ;dy)V~Evl}A1$D=gW!=#tu8 z0KZ*W>1Yp_wYYn|9CV$phj+N45wf=O{Zp+4U;;|94xwP~nVg)2^#O~p30sY>!_V?_ zFC*irqc~*J4U8S0#x=2Umg4JL89BL^IKPOGXI6!0y{M^|A)L!>guFdC%Fnhta08sk zj~Rle95=sznvWX4Gww|4y3a-_O?mVBX>MPj)vdI$CY;51jB#QAL~#oQj&06+#+Q6^ zG>KqW`Xs)dnpyO$tOn28)x#W8vj%l=k(e3N>)5S(Y^K0zyvMdLx0p$+yoP{OnYUXk zwVWUKQY}^AX)yJ6T3>*kc~iz^sHr()F(jF%0}8XX0lcVocM9i~W|njzV>|$V7xmD| z=#-rG`ogxerQXToPCFe2?Mp?Qc^g3OMW63U-PeA13y{1`xZZUMQ4DzN#kzX|ye{uX zUIk1^GU!(%{2lZZpxY>-o}W+cP>id8UOZgN~7wrQ6u$Fs&T=ZDzw#r{t+)-l-B5~q3{=zi{*xC)TcoIyN8B+-u zRsP<2==di}gT^6`UI2$2=QBI?N{+;O>!)z>u1tOc!YBP7M2pxLNpc3vocQ|814Z3L zH{YBfgRnP*H*C473iI2w zsvgsZu`jRK7pDRQ-S!nB7;qcGL$|}cA5NFTs05@V!g_>_1PqXsLNMEW1`Q7+<<9hE z07vTrlf+$xV#SR#15sO0DGOu~{=hDae83F#{9ESJiN#nCm1~2(r}9kCU+Y*X5OAsP zg6kc*buifeoNG&bD%(mra<=0ln#|{M>5EhPk7QFZ6f1>^Y9cS_VGTS_B5b zimwagzTN$*S3(l>CK$x5U&)*%V9dB)%s;fDXP)GTwN^kO_O{ptA7NXV^HPE~HjFF* zgpfO+i433{K3nEqKqO~;x30^R>(L-G(GKizBi!zMclw9JB=iRXq?ZvjXOSs;>ieq0 zSvIXok2MK|@snYoQ_7NdyRF9JC7CvmEU>YGz)tqYic(;Mv1sWbn;+Op9f&d@S6JlVuCDDeG#NgVXdm0Wzz1^LQ@%3M zJJJ!ymk`I6L#_FT4%epLBBcny9POPutvj8H82?BJyMK+4q?}xdSK_uLkuBKnakWFg zc^0<^O^aRjz#d&1H4m+iX0yE*%Wtcc9%sM_P%-;~dQ94yaYedk@h7r3VuO|qy@vLq z*|iA69>@B$=l3J1A9%1t%~3XH;XqT>hHdLj1d4b_K0Tui>VEGhunn%UAtdjHt{aI8 zz9{RwSEP^r+N}>Rfg2`@L(hmB3kaOJtyZ(cW5~BTJ)3bH%ck^#U4W?i;xk7x~j`g&4;y19wCKT5|XVp{cvt zKwA>7{f8?WxS470u@=vgFt_XT#rG1B+sjAPwugjEAK?NHaH&j02@*KI-RzIU_AaIl#Wr_)v+>G zMAZFboV96t@%Qf~MOFn7qNZ!rl~A*&ZQ_B{XApp@Xel+)`VN8!J#FbI_jtRTH<#-2 zXE&V^5CnQzMAiMz@lSbwkH4?hL})`p;(t9WwsW;XX|8AI3k2bGE4v>I@-3xqL!>n8K|lO zHY3uHbw?|V+i9g>97`823x4|Q(s5h$4mkBr)BQ6q1Yu6)hGO4SQqWa0`IpcsW=QH1 zpkn2ZduTv-e6>1ST>f172s?L*l$!IP}+0L}RRqtC%| zz?b8eM*4$|ft9;0Sf(+^t+u5-A=L4oQN6mdHs*}lD=>m}tW9-yasBEVw5!BS&FqXq zi-dLg0VU}&=|{7enOE%52zlLxNW9R;p@Gl*x{2Tg85zlf$u{$d5I6i>vR_ZPmHVc*YDcj94^X^RbkbRr5`!G`u<8Ru&-DWvI<)c zINV5<91ANuf{eYkZL%9pth&fh$I!X+tYF5g%)mr%uX$+M*Tl&x0Vl@RqXM+;L8!Tp zJWQp%^!)&61|jXbNDxKFM5gvMs7ADjqtcOTdRJUAH>IrK^jy?16^S4trdubTU5~jNAgJWO$HVK}s$| zDw|zYOUe{Ln00G@3VK^M2&uizz2^OD~N@-Sdy`DvF!@L{M*pWlF>$(OUAJd6VE=%azUTb1~N~y>YsZYN1~Z8Jme73z_5)0Tjs)V5b$cm$`-f`q)tM%NgvSSAQry zFTKO!Lrp45X@+Hk{Eh?+NYUjoZtoG+Donul>Hz60KDmRbhw>I8b}7+ zY)A_t9I9P_O6KmMluoVxGo=&LigY3x+q(FOY*%~bn5G^J{k|P7wVLfbnH^rg8Z})M zyBl#%L_po6P#}%!t@h>UyfHx2;KGr>B6WSA(s3 zFsBm%#anuaTQ7 z)IH1pn45!c^^S2D&JxC?1>&E!_Cy_1ND6GZ2u=(8Bfs`otn$H;47dF|C^*A%xbOOP z5h**_@`h&%7vBw~b5)rW`@?o<|I4VW>I2?xh*62J*YFu5E}(w>$B0twuQ$bnrD#kT z1Cbw^1Q8j|= z$;zW33xp9E7B%{G)wzu@-_LdtBr>LE>)`gR_x%&vA|HB^wGk{Dk&JhO2FX|ZRh}^< z*$GY%Y^RlZSni6__x55h50~PZ{PONfo2^i=?!N<>Gi!^6#ch;FR5*oLWv(5J0*0aw zUB6s9%Tr!Dr}hxo-^2v8n_yh@^F=>)&Mr5yG%uLw1`_1A`N{4+>7~N@*XqaikB5ld z)M8GbZQeib7UIL|4!|#F7$ov1hd<}5n^0`8KtV^g*+Un0zxtkIYhAA2$`l$3qV6ks zZAXFwl4KI7cK#9Ky>RWl8U&*Q^E9|p;Y%XHJ=OaG=cTGHHH=C_kE7Ti^dvZm>^Fnd z(>ed7uF2+)rKmdzyX}|9tZQz0&R)!>-`4JX`@5zTJmooYOpNj56D*@*j+${Jae7`G zBd6XxsI?%e85t?V%`s<5eG5Yo?}GTIig+)WCYqz;k^ScD59U%8F8PEzWzc|)!`|-p zx6Mj~OOgdHS;bu1N;X`}*UaGSULKeZ*`2YiXBdG~4DuvPF3VY?VK$RQU)ZdkT8;qWj)= z=6Z*8qF){wDm6Vnd$CWIy4{XLyfzMdQN?f zcc%wrxy|?77U?gYnW6$3cKW1Di8W>KOAPW;43cq%?y}XpHl)LBve#Z15z3Oh#>P_d zC^bg6JOQ|J1qP)$?)HVMi;Z$A6dV85oxQg)tw0uw0-P9vZNC6u&dOU0uHi$5?jXbO z30}ppa#@E@n*WG4CLH-IvEB5^|Eq8D%dmyXkD^Z3Y(;N@ZOPags5PH{j8$JI&;BDt z9FwKEAZb*iPJfSV&}M@>2lVA$v_}Y1K7}H~V)6kE?OIxlO*HJhv=bTq?uXDeQxdtm zl8kE+Cl!aR>MSK+9otdh?>7jkr_W(Z-5#85pY9Ecb*DdOuY#_^S8|Nin53*^Tpuv} z>#!RZRSi<)o~nE;FM|i}3s{TgMf}Q^l#ioBIY8@k5w^v7#~(>|ljXdd?zDg~P2XAh z@&>2AasYSxb3j6)b}l{hmlCECv92m5&Yof&Arz_vw}J#bC!?uU;lunAF<3opJ&FW; z^q4A}FHE+P7Jztche8^^>7P#s`HjfIyH(H%9Rhl-n7^+al&_Gybyy~O)#|rx^H|`z z&7gwHz6(a(_2j;Zg6{iK>#CCfx=Hg6WO_y&{&0emd_X)L^kgF%^Z03s3N{I-Lbf2`Gy{fUe{H-Gc!v_8cb z;jZaZ=gA=t4GL9xR6M`elYG>^zC4(u%YC=|jQ-&p$%WoDDvyFWHf-S|cZy0C8CT)^ z{}m4Q4sIm#fZPDvp0PUdi%B@v1lQ?j>b{l5UTZX($@lU>9XI2o5M{^l9DM!s2%Tq7(y6@E)fDWC(zpu@G~RFYGEY#9GVt3z z+fJjjkrj*f?)6gDFs76%zH6-Wo1F#D{NXuddW;))T1G7T_XmCHRN%T)+W*m+IUjCr z!2i!1phEth*#63A5Y|T__q@x@+oMu4YPh^L2y1dOmF3p(wSmeo-E!}64TM<&@33iI zr|AfDJw@OqFE1=u)*mbT_le-Jmfldx*azZ&ULng_{DAPYe%J%v&G$)uBkKH`N~LM{ zf5plcapp?s11Iux|MavX+llE3!zErFS7QzB10YAQeV1@={5X$l*N0TbfcTOg|L-69 zzdgwRc~6CNh_vo|%B;oo6sRS$>ecZDvzYu_Tw4ig|JDKj^*B*nM{`nvnaIassk{qC z)Pe;D|C11{jOBd~BDQjaSc7xL2?5fV^$=mXhZUJagqMaoFk&13F=GD_e)l`T`^aMZ zxZrqjAn!3?hV7E=W8RB@#;*1+DhYJiZ{$l7%0DYhMa&2L{+RolU&`0YG*(M)aQ%CNlkP zNAJw6tFZ=AgQ*T~-3a>)B5Wu{>OPDFj_Eq)yv#Z}zeHNcgW63VZr6^ku7|R%S9Jx! z=AEiwkVzVz)&y-veC8si#U{$U4gtb4rs4M`szibEF5At=Bw_II(G`G%GTqyv;`~a6n2Gvp~-Ej zef+Wp#x@K_#D^AGe%|{Wac=Zdr{ZF!J%Id1KLK?#jg1`uB>(3cMOZ4EPKab^pv+g6 zM~}wxEtMT!UVQ8nD-!$+O0=R%a|4M$ zmB&*_V-zuBKp%X zHUK)qguFZ$HBu=rdO;XVhj!-S;u@I>Dw8Tv;QWKhK*N(OHRszSc1;RDS403GHCxG4oKgevKBBuC z!Y2+g`1jdmdp~M!F?zdpm{dG3-KEDjhxYycQsgI)OM;Mh1#ICUy&V2%kWd9d!j*?4 z34=I&hi~v#$t5G!1?&ek)+kyboxXP7cS;mMIY)Qm473t>N}7Yt!{(39vi7|H9hZm6 zFn_j>OCh;_6=qpT1+uB52NvX9Zfy&7^=kK9WhfyP&o)1Be1+2k=Ef-8G?r1n zqj^H^b1kCvNg?tV?X>|H^aw=eG*A6ztO`2 zIHSF^b?%m7f^ua`s9iZMFRU15C}NFfHV&#NsFnS(b{Rj-v2?#K+-MiVG9t8pyJ*`y zb+&#dqbFsk`9n*vBUQkGd2Og*2qqKi?u-LR3eny*a$FTjqF^XdMOO?P3}W|YTy9}| z65XnLc}IZKC%#nhgI!(d(1aib`Kl9vNi3Q53U+_Fo+*m#n$rB;j_s?s?9Iu2@17RY zF*@+&t7KR%Oi!O9&kF8Ui|L6RVsZ5`kc=5!n&h= ztW=jRKUpDgb_-`VGFvbiu`8G<5g=7UcLO2r{`rBEV!_DrhLKVY`pnxG4a3@!3?tM3V&EVQq$ z1L{Fu1i>yv#Fwtr2Jg4sXUXF$O7vOj78xgFlOowRTuz!<3AxS`iuwRbcw=}Le+7I_ zyKO7YYqP3+W)5E;&uxWUOhU2U?*T*bOMJFfk-Y`k72;74J`{_|Zqb^mhCuMNvLp$? zx`(#`Q`&Iv?+cZ$1+a3c$c{_=qaGmBXn9_4Xoeld55h|g&W+IPU&aJ}@Jpn7?Jbrj zWPrz$z;rYK+BrO-fH=}wxwI18XKubPo>)Bib{qxvjCGOaGVXiNXZunz+8o-YeP7t( zI&XYeO9m4`mG5L^f-s71gwm_QJS?!I_qN-x5|sVF$ji}gz3{~TD%C`?jm73uP|fF- zkA=83+?sSf-%y*k6hAiLJn9wm#!o6}UkPAS4Maao8Xo!+aIwG1yJ}B`g3_TAXKL9R zRykHXlkq!U7uOr459iK-ghscBLFrn~I{D8|xX+FD2VqWX0;6I|6=W-k<(NMScDf}J zG79IvPJV=p*}{vb6<({tj$~ij<(@$|1Yd?^AuKDZ)B-C|I>=WESD1|ZgSlA#doI~n z+)L3)gHkdx|7pW9ct6ug5XxZ2dKCW`lvQ4aDR)izxBRH-y9lG2V1vFCp3bq$35gXB41Ac znq0omUaf_o&fBB5YqH>)y{~|cHX&R;A6!(m?T+iFB)^^d#C^{8z`rWZXKzONo5V=S zQo!Zo9z4=jvp~;-0S{}~g|P1$sg;qL%sU#oA5o{v7L7iKB*2!Cw`iR!n-!av1l5=U z3e!+(-@Ac;wm0G~?B4d`)mxc%L&0r~iZ4(gZM9rPRJ*ThOI!Q<9#2O0xF=Uvl6yZK z7G=ah(HBf+LBl?J(BPo$c;aA}Q8(g`f<{q*JV(TB9K0LtD$5ZzuT?-!;LUdja0g9{ zvH|KTlGyC;LU?>!bQ(y-=Y`C3o&`xzG46}6X1GB5pKvS3hfFvV9Yb}g3v1bPU2Xq} z$&{vh=Y2%B)#wl&U()^q-+vzOy#u^8czkWBLWN(}QJ%i{IB`gqXp#Du)w1=lBdo33!P6w5n= zScj{r*$mu|cPLefj{SofOs?Z=O%r6((Y$-@I}lv#2E~ZF9k#_Fi@7(|sjE>c$7_pdm@Q`_( zAI0KqHRtbs-rkq%R90s`i0(+DQ|68D5~FLSFjmOE7|O&=*GzPQblT}p2%6zKbQym( z;{%!RQV=;NQ;J9jNPMvt0gkk`h}9l)*k-m{6reGd1O166*pBA#2Q#D>gu8#33DKbg=VN2=hYq?K!UjcI!`G@Q=0|RYn}Mw05d3?PfSg zAcOQf(evjD_+O?%MRlnJ97nFo^FHFD2lvzrxUSH=vZld}*zQ>;+2|96U){gP^Y=8p zY_1%&7l)O#&eyrq?Q-T~I7Y{@9YaUdAKEZtzlv0;5cU8&}5Mnd;*43~EcRb+- z<5b%IOPHMQV%7F)r6~KzRJ7xA)!Q zZj(7*(-i5RK@gvRG&1WWj`f4(JhJRoY3f)%2cSBFs~-kfH*lfi#lKeHG-fCbGd|4M z_~_{8C7VHaLL|{WpFYMVK_*Q;&~k zTjOds{FRem`2d(?z@df&Ses65fc(1`@tV-+k9|1Rn^< zS0syXM(<=T!ri9JY1|JU&fGI)5xW5CukDuljSaH^y9xEF5A_#z;MEsfMRArpP$*}NvTrB$i7(>LIB zk|(qfYYG0`iq$$|7J!;M8CTL3xA6<^;9W1B<=JgE#a2tJR9a?;5{lh5^5iocRO3oey!8&xt8m>kx8)jKjWmJKfDY{lr^;s9yblJd zvM|JY=p&?;{%PusnC0t8va_|7j|ckvNXs_+iMnk&?U<;yz{$Yb*;>Yvrj?qx>PG?;Ejro@bT8S zd;_bU(ni4>_ORdH=SKHK9~|l)*{_jYZ2ge%Z0-|7sNoxDJL4-o$hY5nx+fO{qpS<1 z13youjLXQvTRC}lik1u0scmKn6`vyXJXHkPx@-9-1|4XRy@i*qpSCtoh0?D%) zfYqhaUxK_jebGKMU4`TzB0)G!ztUM8{;o($6uYqMIO$z&bpX&D>eG+kBvxTIr|h0g z9f+m`PXkZ_+$lFbwmk2>PTdZ`ahfP^eitv&W6ks~?~cRMc#B8BV}}a~($(zXU6_%g`0 zTQeN*e0QB=Udo(aTQKT;-H|2?5PnOQVU1jU@3~>&hJ2(xFym$~Ce8w;D#86p#v<2g zv50f*6^Kp~EpETsp4J{RD0Vbl?unvpov)KkraSZw)sW4#mc<-&83klKFeg%Dd6sRl zr);QhB{n@OQ}gV#yjZ?DRP}!P#U+CCr^haYe3k04JRC1K_qQQ_J#oh?rmZ*1fTRMA z&6E-zN8b_wP7z#R=DEtbsGrn(InNI4dgU;kd(kb2u;!j%foS6Qf!FYdQ?F5V3xIw&;Dg#2 z9f|{rdfud$s_Ml!F(kD_K7Fhvg6vAH$q+WqqO>^54U+s>!<^@7@sFyt8=`2j_+a6a zXB(D+nyeD=^s{E1pyvL-hnP&~dVSc2W52TUQplA5>drP+uiERlZ&$`#!6;d$1MBmF zJp@?35qVEUpHRhRzI-2v%{D7>p||{;)`%HUEZ$|D`!?SAER^rVw~}~08a-+?mq^ia z<|n05?P0Sz_)MR)?wErLvf>a1?HlTS$WqU100=G_7!}|!43=R#>^EZe-S5>M*|Nl* z0!F%>du|;`0VxkC=_Z3d##({W8_}bqsXRR{jmvnts|MC}od%gaMH5&{XXcr443q772_DsP&MJ%NX`@B!atvME=6U7=|iV5F)5^ z-ov9PqHV$-$)KyuyhQ+!Eu#<-?`G%$i=7a6cP~Ki+uWi3!$yk5JKsCSxL>=&W0Kgz zp`GZ>i?urz&vUeniml-NKICrYIUOQ6w7`ovwN$68ba%chUIc0{CB74%K zx-QY1vt#hW3xY6WB6l;V9x4ypauZc68W%By=VZfuMU+ zSPPBWyC1EdBtU01GqGHNv*F_R(!=%3<_uLfIKJTC((>RFr>TDuGX2n5>Ph;npGD}^ zpUX|}+5Nn06`mJGcm+knoVD%A1Yg>8bFtiLF$#zER zxrlk2%wjxrHg_CJvfP#t67PdmW+TtPxY)>0hD9;ChOjKWz15DCwHsl^#5|EZ1gq+P z8pkJVIGP5*%0NMU7QBbW-NYL6 zIB!?+`k16D2Nb79q6Iz*tm8Erb-R3a>mlt-9xzhf?RvL}b-6gIU8OSfM%fNgziRg6 zK7?FI3IBEJHhLjlgJJEhikYAr8G)|eY1-~42B^mch!?LZ_`n4U zBv)3Vkl*=C%XRw30PORj_qCh!tEN@A#j-0y`^6s=X?(_a1YqCz&Z?v3TfCzHt-MUI z<(1`M@*fVTJb^-IZZ|g06yb_Ls66I+s%G{Fg6yPVldkg340^Ad0p*z(PIuilV8$k0 z{%A$+e)D7d1nAG7Sgio^v_Br1Ha`L|WD#3={$x6?Z^eV(uWOX)72E;c{@Fup8q1=j zv(Mx|jgebgjCGki$KXFr?UDhV)r!km@>1lXp={V@>>E2h)(^(loJ@`e0!n!_^3~T6 zSGV1?(u2`kA0x2_JQuyu|Iz}0KCZf+G*I;RLs2~NeEL@HM0mrpp!3+U{1OzUrECjV z&fI<+T!-;p>m@uM55X|*EW`VXJOHjHiEPq$$jK;PPyGuf#S(#)gQ&739qwGN z4U|-N@xn%Ax15{u4{*zz1b-d&lT(dzJV?X_c#tXU5+8N3t?(^I62BYYb<=~D+r5U& z(;WhW43yGEoduHE4S#qt~~}Y>&i?Uh2Pz}QaVIq5mdSnlO{PsRQ=CE3BB#V zlX`i-33$y_Z$rvgJDkr9MNM9P5h+OLGmHaI7e%s}+*_!&=KXF`pAytg&=F@0kZlmG z&>@W~t%)e4kb{WC;!J|n^W~U$tINdzwqU-3M^FJ{(mb6)a*;87o<~@e#B7wVtIJ#s zP1ff?akIBqwyzEambVh61Rep9xT}oSX2k8tiY5`_gJ{*{kAQwz+PbL+qzP~$Sc?`= zT?=+KbbszH4-_$;&)zL2H9=)dy7s0X8Or!q8MaC(08QvMI2&WK=E0C*{59W$KZ$fC z*)Y<5)*KY}bqx2MhK_YR}>s-hG9)_=U4 zcFaLE_Ep*YL|np&9nbEvSwhan)s*>B2w*Q0kIo#o{!v++Kd`=ZW#s=>NTIJAbJTW9 z7M%7)BFF1Owk_71J$UJ*wZ`!0G%rahfIbp{fM={;AZm5}u*&pd2g%3>!f-jHjwB|?QVenc%y5NzT5f*5E!NjKl^CMDfQf78B5^0@fD3c>x*pg^&eg2(az#%v^WBB z9NyMkSIn2)&I-c*qSfD=d?|Tr$B!r0FTV5Z8&bofN*g+L0@L52vV8%fkiLQ*T#Sg) z&^g+RMJFrNK;+U-RW4akb<;wK^C7@WGn}q5jZ9%rOq1Vd#J>pCie~c}-b^8@0hAGn z4Fh%@_%&u}m2x-ae5mx-MaCa*Pvw~l7R*R5g;z=R7J9E4RBJqH2~NGdf3C&^(PB7! z>R=MLlziO=dm~Oe`r|u6)D^YwE}J&g#u$J&*L7EQr1vr<4yFcgX49HFpCL8P@dOCI zJ(GKdmI^O!3wHR!ifyyU8ec)rtIWJN3LCQo`etjHe?uKci6Mj)^laysnD*=3$_CjYIR zB!cgCpH=NpF}TB;_8tv9hm%Y)Dx#ffN}}-lTX}%%^z8Qf(UT>dD3pkVH0fRmBJ?a< z;pNy564?2`Zw#$k|DnN%*5MCX&HK@Kql< zx_buX-{*H(v1Zqh&Lggr5R4iz%USs~!6H*?a=0^Af8%jx$hogvDbAm~d5R@DyLVjv z=&#GhQ_E-VTsGG^+h{lTz?2|l0vJZRi2d#c?LeICEQ*KH5X8`b5ThneJvq?;Welg{ z75~YpJ%%v4ooYb&)dYCxLXeOvTO!Hs$fXP*nGW@Q$75vPtM%xEukt@A{h$6x=_mf4 zmiiR^toJXKzjTob+RleI2V-3OM)}vIsZw>_mm5JdGy87^m9$mkQ$6NoI;k2KlIfVz zx~I9R1Q5gKu-Y3qDzyRzu|z+)n3JwHqgo;IKF&iZ{&u4_uOQES_;R$$?Ut=VP9-f$ zQN{c&nlmIt^DQ~rXSiU)?8t_6R#!JX1?!|^ge!ly<4wJfolZQwu@$6D3`Kg^(jR>XB*2ge?&kg#CB!-X^x4lb z$uJFjh5IRb9G{3)OCn^YT>03!5?(mTXmELT{Y(=b=qKflh)^}= zb4&gYzi~MYx|x}z)9z8m)$kQi4w{LIbN{aPP$p!zq;Bul|MwtNY^$IbjBUwPMkE7~ zi9FibF-V2R8-q)WE)=fl>iX?tbmK~GB~=pW_N7egZRz)lBil4eufKQ|q~!PfXXFLq zOJ)Cln}TnmDuVwWN0fKVbMBO83mc70>-8u~B_$*F32z)pXM{Tf6!_V8uErK%v*L|m zY5+~OQ;`TDZHHIH8Bg#Z{lF(i|NP&BCMhTB{cjzE!x&ES@6C7T4?WiUkDmHx$K?Ot zHK7#N@e|P*ME>|g9S4HqllhZk9jy&9+0WEZ|7-a8rp5nT5EU(0yCb>PV!o8%4_+@i zL-JF?L+;@HroWA=W!HN*^PXa#`_^Z8s{}sv)2E8(3OGEEpHkph(7xz=tpALIFD%6i zQC{BK^3Cp&UBP^|ap5$4M`KA5UA3Dv=P#kiPp#-8d1q$Vak|_mF6r)9Z7%Ly-?V%B zB`$LWiO-g!zGfBmo+eQ$4OwkN2!kmy){cObLDUm+k}w=OmiHHYzu7l~^8fRsIz_ri z7zuau43k?61*Wrp!?pF`C2DN)5x<%hnb;AI_VmA+KikFa=W^?Lp{>8>uSUpTd?Xwx z&g0J%x7mKV5+0m!77k!7NkDZzv5smFrp#zQqz%bEV@z!$HF)?ecFQ0b=TBF01VSgc z#%MHqzJs(%>_s^bKfJ9l;=TA>+1kGct}zWxkpx;UaZI5cmDeEPPyH;gocY%!v;3Kc z!Vw>P(F-*9GwYN_%l~TN>pvS{>;CtkfzYRJUK$zcIEs9X?2lnrDBc-iCAFQtiGJs-5 ziX|uhE<1d3&k?6$_*nN8OIMF&*`&bcBtYLDTttakFK<1hd;P!$cAkOepi^sJLHlDI zCP>^GU2)H##U3>lvNc;$x+ zckYD1x+75(g@A+$p_@4CPt8|ph2*Ner)*XsxywTH_9ul@xaR!*Xh;BK~g+urQh4748z3=(GpnUO;_=5(+%M)TFURpI3uR#|7@LYRh<1od=A}~Ld z2_qvYiQ$@@R&C@SD#P^vpK`3i-;e&`Tk(?kq-`uE8#YaOTk|6oT+qlZi8;0^=+BE( zvoA^-#a}Y#bg&{_BB3{+Hy+(Ak{eTn!1Ila$8OB;OX3>44oHCB1oKc zx8g@@>}d6M5-M@%$@fzfcrfZbC?P9wXj$q%TkiA&jAlhC-H_UOk67Md5S-sTp&XS9Kgg0~ckSC`C&^}H7j6Sz za&r4v-JbEafwp3|p?W4DCIY(*I`(vNSN_lK|7Xt^5(X@yx&ZrgvfAJr!o0ORzX5D@ zWZzGbo*9+whfa9WY;R>G9qV|R7dBq;#S;^`4f70ZLiJl({xjD9kG-H`pA$u;xnuPI zceoON=g8K=Y2Z{hh96NKbZ4Z%pzDMGH)Eh;pODi$C&dirbm^1(N9WW0UriBZ=fx(9 z%8p$DL?Ft~eEG{M>@(~!>IYR&fq~Rl4zvdUJ73)qLM5f$+Nu9O4T-k-8=g_|cIV@> zMXUs0lQJH5$S-j*2S`vVbNE~Ww_%I(Ox0(NC}M4TL!-PDy*#6R;PjPqgXWqA}}!%j=^#NJYL|_y636= z?zfOu;6SU@;}NyTfZ%w(0`i+ly`vab)MVN2uSSot@o%W?C&)0sZo;H!P}CF;4^SF3 z2@+l%9gH8X8a$QmDMb)9Gla6JB|z*a>KH1)u}%&3lF|zee}e{DZ46RSx?3c3;$1hu z_b_r0;+}BpzW6J|0>}1JqqKJ)IRSH$^h!h9T+`D+txoI^)cm$Gn33p)HK~Kod`o)e z1cJiXU{DUJr@^a{!A|5t%->ycH{rPrb%>rl-5%>utduvFdYqi8bqs1(DLsXp0HV_< z((RFlT?AUxMUMup6HgvEx0A3yPJRrJBdvLY?#KT>ti5+sQ(f36Dk9Q*?*anSt8|G1 zQUw$cq)Ttolqw}kl`b7=0Ra)EcL50e%Ge(0Oj@ zx@du3hICsskqu~2%By0YIlWgkP+I@^n49Wct^*>1=?%4DDG3{3|MBE-*UqHUT!StE za9ztV3Xz|ru7RrEFM7g;s-jfp28O~2j=(3n{R^CFd$9a448qA5tc=HS+0NuRip696 z2_uYS5DQD#mA$_+=}@@NK`xiStx5)l>pEuXcRMH=+}2yXTK^`v1hHSV} z+}1iF$+nRBagp7R6v*?*UsXuHO~))5JC?pav9_7XDmab=)HI2lsW|J3!z_%qhY%_^ z)OVYx`uDOH?8x3#&`=2b>Rj2}CkDZ{19&g6nJnRgEo?jQn4s_8s25cufOx5~H|-&> zQRIQhr%z5jMI(Ka9xCG6&ONm`HJRRm3*B*daT_NTR`)yIxl4j5qpaj=Lgp-9MeYM_ zeb!4De;MVxO9)nb%Y>G{GoE=0$)gvf`jtVXK&!5AJQPN;dCqmsq=?}TP5f2GMrHsM zLL_O&{t;rj_nOX%Qsm7E>rA(|W$S|r3id+trplQ1n5S2BySM~Dy!E1cC^hY>o@6b^ zC&c32gkXt+W@F5Y9?A!?kkcrXl~UVVdpdt<>uBax`r zKI}^tRM?2?QGEKU`;(m!rEO{`;RhZj{tO; z<{pRcMKaOTzx}X->razRrzBmiH#gCYB4$1Mw@(yW8JB`lBG8ngzv^)lI{4KLE(=Y! z(2i4j=9?i5AB;$D1LEwvh79N7J0f_+`lXJf_po+&GCF_j01`gVG zrX*vbB?Z!!Vrzx&V<%giBLNis?c2e+ChWV&k?<>LKfaK=)GI6}3{}25v$rlU&Z0T` z{i8!DJ4(+*NFoMLPdP8)%rljygoNxb#dF8&_%E`(M%;AN-9D|su$uXSwIlIeY^a&U zJ7S8<8J-e`eV$?(%bgcIX+-{CZrBJngmm3{o;ovJT2}rrn}Wrz&$;OZ!i~4$gTA3@ zobCLytk=G5&c+8J2l1zQ~Pi zE8hFUkEiRkq(1$~DS7g^$dP=`rPMN>BM{+W?_I<|^q@{FLHCYY!czlyP7SB1)+lrn zFAMYtF7LEcBUqfWUeRSY^xj{%3&Nskf-n7eg{|9ZTIa|>j4ENG1DDa4XED}0OVy$G zst^W@{r5TVe11|VGhP0K-0w@1RMSHTP*}3i*!}$3&4!#M?N7i|Lc~!FCMlF#5T-Cm z9dm6!N6+zN>ZB35{IW$4BolE6%94ctr5Eg+(!V4O?;3r~D4aNs7w+@#Jb!W>VE*7N zNe=KOc$ z%>IP6e`yD8gTYjx66B%2^;D(Cmf`VJ&}6+(LbT%67N_73PDiP&D|yR;YPxcZ?=+zx zTR=3gh9pDO{CYjktO!)YS#D_X|ao7W43n z+zhyFt{7@9@nUf(h9)mfo%Lmly5lMI%CV{A;t-J(eR+8}yMu3R_vqb+^ki+97b<$Q zLQBXkjD3z`MAyD`i$Eo^A4h)u+ZzauMXEbZZTDp|&$ddYjMKVi>J9Q|(}`OXGWRK& zFI3&7GL998@7j`E?pEeg)j-1&R0kexA^DVa1aUrluCz_9*J5#F9sp}}>-Gu>aM(69 zdNe(yzs|p*xGwa;VNKBD>qqVMK_&@7V+%z(^MSNCSVQo6>Rip*ZWa4n+80^1sUAia z_3l&$dGfQ0Kg7MC^k~Nk_EM9C)T@lPta%X;WNlFzKZE_r8 zu}ZB3Q3@|UUQA2dJgTZ=qI|4)js0w&sN!sa)bqf`gByz!+r2pSzeY~!w-U(T-r=O0 zk;PN+I?6ZVX)#0LL#+|0Bi$Yi%SYBUD#1?#*WLCI%SOjaESkQR;g5*lLFK2%0dH*I zvZ%@)_7)?l%mfe-)nBx>0We!c*U{!XF^HJO%ZmE_qDuGqeKB+wOYD~5UzxBP@!%5% zHWTAM{#+JJGE&4d;aG4I6&%Xx@+6cqThQ~#(_cSQk9cm?t?gFvv$EejuC}TAn?m+7 z|2+!o5?tF3I+!?iG#|d%&7G-y$_noaz2?)!Ke+QFsE&7oD<5i+KZEu>JX{42h#MyA zgk5Ubj-EaFSB1G|IV1gcoxdS0YU|*qoVA#ED*=*LDDCHiy+0eq@8+QE=MeuRld5EG zFRkIBY2DjT?#Xru>ZQ_=dTM|7fBK$EhzIrkwYIqwyRrA5;%5Pdf4+YWi+INT;qa&7 zLiSE|tRa7ISB}AF|Ac| z^y&D*2keZe-9$fTG<~v~mWdnu=oy<5zl~d5dObB_HP=y+5$J7u?8S-WI$m(&IjU-E zm=7%E53$z(rhMboJEKBbrh9hlWyS_%SHrlHl)og9=iOK^yWQKhoBwc=m7I4H5~%O^ zv1z2JNU3M?OVE3qxm9sO(6;P}KCjAGN1 zG3C1g?jt_WddpI6*LBOzDoJQ;9qhpYy10Gn7Szl8AE)vUhz^W^UtW5OuMOgP4NLT| zrQ#IR(U{EHRQJ&7{7Fy)NM-Y%nVJqVAt0`ilwAE!_lYdpedg{m{$R;^Sba$`3QK-_1FwhI4K^$_5u{>Sgy-!t`lv{ zxjvIx=UgwUoQ3aOw_w}AHXzaqh3lMm)|Nnz7!xog z4@C{-Vjd?pTsKEAskFQphRdn2*pDW{Hc360WkSuOHb5*v*bfk41qxEUkzF{_8ZM8A z;6p#Hi$SdAgwa%_jySOn$}skz;R!F^75mo9Q^ob%y0OkuhTrO}CvO3lx`z+Ws#v!~ zUqA>V;VzJ+t;xgHfXfH%W4K-Y_IXDup^xk5E?-owbw2c^-@PnWjq$wpwalFF^e25L z!RIH|6Ta(Tck^=<;(uEgp~5NENq&Vum#1iJ*8fYjmS0 zTh;5`>obcUahN1W8v$%QrTm+`$I@uSrutrt{;gsdps###Tj}r7^8ieGW+ABBxy|{P zyl`_=Gs8&fZoiDZEAf|)G9nHNNw^ZR+1Go}RwSpCBN_CUtypmSS*pJB+e*vCUYV&2 zc=1M+iiij^Vyv5Hi_`Xr3c}i|xt}_VZij~V_0Ne%+wYs?Bh<>JGU_R<!i3{5x0rBzbhmA{d)A%#-hJU{h zKLg?OfZ3tMbBcuFVL0zC9Fs1s)y=sZl@d6j3_0M}!o*0kS}vU$j`Qy}6_fm5-DlVf z8rnF9^W2`*%|tmM7K)qbSJYEL+6S!R2n)PHLb?no7w+4-TI&Qra0qCn@*y<@`r0ly z;$h+(uk%9ujIE&`-rNn!geQ+g>{=^-3AjxDiFDF)z_Dm=)D$4b+SIixrlv?0Cbd36 z?Wy(6CvQGf2C}>Locf_p#G%GV30Xq7rtJ41^ouY0VfYXCQ)<%p5lP5H!-o%sp*^Vg zsuYo(;hbG!1MJs8FiH|>Wfqhw7Q1PEPwcMyL;{BM?Q@F_%$P1Xsvb-eJ8JB$p!X*N z)ZEHvV4KE!%p4kd=BuU|efpIQuOFY^?ho(72uyKEi5>Jgi1tJon&mBRHQVE3qZOi{;7k48ir<-^L7#8L9SS52+-Y`%uy(K&cu-ylFO&%&cQyQHo@au(lZ@CgS(f623Q(aV|+sgXCD zrOZw-2dB;T`{N=G`m}N%!U#Do;mf!y!aAcXM3* zSV!&1lFaaEoPe!i+dYP%fhYa`uIZ!JIe2*3I8~E&jP^`X<((sbMgnu{GwUTp(xv$! zqXG7=>k`pQ0D|0kwFaNf_t%pvZw=a2nE@W0c{pU{miPU_@95Ys+O=NAG|uHipjE4kC9dyPbkkRg`bBaGu`r|vUR7e>U`g9pw;oh=$sKwr z2$=al;o=xBL!zFKtRc|4;`T+r$cPNGnwkHA7Ft*+P;kBL19j)iCnSe-YRz8;2M55U z)BM6T(Zd=jp)Jv)AAPGBMXeq*=`tGv0!Ln%UtOhv*5PAH_$Cz1G&p1ubj|JlS36vx z0RIENlANe@tk6Sv`Rzb{cRQTM^bI!M*B>4m&*k z;ued!lwt@gNn;36;iu!`UyCFWVsz+PUH?U_A&?gTs5-6o#yYh7M79GYqEOKGK@0N^ z5IqSCakUQEJyB+|%r56UZIA8<{1|x@w_0+Wz|$KS`Tmwur+KXpvCsY_!9s%@sfmSm zWV(<&`wmgrAMLQLhRK+;o53?t+^bM-i9Q^&W>N#;{7^*7UpR0nR=?WBQ@&cw0s1~C z%Ic2!53q9gZ$KnolYC)1e(Cz7{Wz@X+FkV_GzO(+?Cu9^feC!iW=;z-HQ_(S|Ae-RBBI}TG5-Lj62 zdBFlL>PtXAo$P-;fKc@rzkob%hQ-Y@0@7y)Z0S4E^W8C!)^gLV!GoNVJd}yEt%sQ& zCHDG(WI5l%40)92X3}c+TS5*Z;CvRow!;qtp|nsAP=qcU0f)OVU`Bq?BlD8$$R4J= z(iHTG2Nep|e`|6{^5g12JrfwZJI5XUD)!R$OO6R?u>kGNN11~#9G(*OK;1 zlDGkW&AOnk%5*oe?;|n%qmaQ430xB$;A6Fk@`wk@TXAzM7=3HiB0S>pm@CWT`l$JtfE&W6XJ;RO%SY3KniwuL2 z{+Qp;NDB7TNLoE5>h_+h_yhPV-r^naM2_^MISoCzMxW4u9^ADFl`Cf42qUN5N``#*k8Y}v1 z^^tu%q=#D;Yt4p&t6*Mw4FSv7LCo*zHUSfRrsF{h?SsKt&WzP^gmgyZIevtUq4k#{e1U7A%Hkfz?9E_8=_vVPp>U z{G|1UN9&~@`ec^i2YxM~aq}f(Kbm+(Hz&p0R0$3K|JzvF`X?fiX`pHOwNNmwHz0t0 zwxGiJdy69f)*$*)PDc-ybI(miDa@LA8F+(V@bb5|I6K7En?7{-quGfo5}V|4%pB3? zCd<%=f{*cW@bX9nD*-wkbcYyNVi&qMNVsa3(!Wlm?RG~@!FMf@k4fJbW6ly-zfHy_ z>a@MmI&n!A+??0vy_J4vUh))1i)OVhT?C(~TdKKGf*h!x0>(Z|%1NdT1QU7nP-}F& zC_Z68jP-FUpLSzd{Y8^#AEzK=JhxDP%#U~>6A+Vsk@dLVqO3wx{6i#Xzf;j{n7JaLAbK- z=Ebc^q5#hUi+2#1n4~*4N5(S(Ok;POejf$j4)+hmk5(?$8bM*G7T-TCoV_S`!qY=l z&bBB)0dV9pQyXHkqY!03Z4z9`Z;s5C+v40P4+0!PFXf;i-E+rxl3MkN-AM#1e9tSg zIH?-weS`XDn0j#{_lfFDYw@8mdG^egN$+|v{WeeLP|#njJXa?LuRj8%UE!3?H?%+ff}M#I`W#N+gI07mui~)0IvH!J?iY)0gYIWg8*6~X zOCdj28fboZs+N4}iC;R`$9dt@zjYl~vMeURQHyNTw+m3HX3R?)ZClbKW4+T8m`9zR zbEav|TBTtZgf~?CxRX5S$sQyJ>EuMHNGm)6T-S~Vx)Oox23Js%AfHqpf|6Fw6Thwp zxdfeHQNb4)QSac^5%QZ*SmqwKeoJPfZIlRsNqYQ1SXiBdlZKURNQK}jNwx4Erb?j* zKkiZk`&9FVC2rLwn!Ug!mG9_iP2>6Hxi6clH;$If8|SHy3<xo?mPCliMyNv^o5~ zSl0ZDD%GbqUcZPhx)&|f0!}{keeT<;cc*(3wCBhB_#qb-*A5c`IrZ+H7;W(!a%MC9 zQKA(z2>x)g1>x{+M!g67D_LlvKuX}4BblI%U&_!0Yj^T7{Z+~5ty^@pKDSPZcBnoS z=LII2N#8lb(#fbjxh)RGaB5CW#H-;F5Z&W2G8GvNmXeOXSxHQCSo$kVnwJ zpL2^dDwKK{^5>zharfk}5hRAOU%&7@2D1fPwnqFG%IGq&*8MfNJcCLi4)Cv)d=b?L z$8g12tv~pCqoLmotM*7HX_cb!aX-Z6@9pOGsGR3ZSvA%Et$y)$3Rw%-k^0Y3UE)Meqv;CA4Ch;PEndm+5eZBCTP)^}UzH6%!CmD~UBgI$Z z{4bwmY=IlkL^lrcd1b-TSvw8`+UbHSz>sM47Vb=`P5PKuZohu&`NFXaQ@-*P^PpOJ zX7*ls0wxLic;4DXdc}ej&Rl)w!v-Il;Yi<00Jfojo6daVfny|`G<5xZ6S!bpx%CY% zL&Va9FCvS^+0^IK4R?)4443up-7IjrkRF_Wj_;I%1ty|7!E{+UJm~7w3!zo4mrZ*xtdkHZgTsmApYn+OK!}YP=4#x+S0PRw z{!;(^a|@M-BIv>lCQKd;_FDX1mHce!1fz4+ZFLQ3m{-4*$==W zrdrhqq||9p&|yFsI-XmD;uDsX1P-P=JfBrR$BTb`Argce+{8MuabIKs6TfBDL$I4; z;Av`?QtUgHJJ(BPjD{W{xEq?piRb!&oRqZOIqvj)*V_VCH>nP}_q}8{FZK+VKuen-33UX6{-%y1|3IR)eBqJq)I)qljcxIs`^3Bxp9QB;v zdM5)cu+8*@t_Qm{QWrlP+nES)BV4y=sX}2X4q4*JXTGN}IX! z3fV+>=iz7qq!0RO6c|hw>YjIk;e)aW9IF9ECitL~iiKmgTKR9CKiI`sw2%Y5Ftwj5 zL5|X)Zwpqz1U24TmBT$6fjIsdZyVf1LH&BIL7v-l&{^CI^v~!9rTdrVxnEVm|3<^V z*`86Gy{v+uWJ9-}<>&R#4`VzhJmc2hhf}Us75VQq!`x*@p_g?X+`0uQM|4;xR0j^9 zDK{n8S3=6My&m8S&_#=qvC90z-JW}ubni%x1P-*d-1&lJGzMl8zEnR`IP61@MWDrm zfvw3C#&&hrQlUVj{%;geQ$uq1Ii&WPQ?aMBrD#K*YcgGqDjs!_irojbSMs2ZYA@n| zSE#eGjB%+JCZLxUpa#M^SHa+iM4poWUvinnm~z>-GPaI)cbrH-Oma$u71NsqMY zJtz#Nt%tkVh!Z>N`s*8jNSIyk{~%>^6)zZ&U3nv`-i~Lt!im2-TY(BxWWUB6j8!Vm zg1(@#ORm_0(WF4RaWZkXT$nV2NBQOwTQCGNNpcKwrt~OBa)e!s{To|r>mQLqqy}Pq z=z1}_FbkRaN-UiNHQWo`U+_h1v{_YCjD5X4w+x$NPg2N-$_>gj#;#*gquUc+kkH#- zK|vPrzaz5imTf;|UZBN4KP$X`5v`&$CejBOTLYU3g>T+5^sWJN05|SH$?`F>DG!4uUZRXkCW{E4X`Q=uGt@3}S195>%!Q_@*wp3?;O9qFA+<(_}E^aT3ev6Qkvw6&WFe5#x5clFj=JvTM0RG+_;I+Zv5IZ zwsO%C%^6CR{?&*fD{qwc6PJsrb}8kJ*eAtvQQSi@kF_QIO2B~*Ml957fBnBYP?4|o z|Jgm#3p5rEyyQ+Zh_`g1`%8Fa*CW$Xeg-DmE>btt zT>eWpG_sf7@bw=M92msv9%qdDw8lKs0IsVKIjIde41;=Vs9Kk$Q;?60#~hK6Nks&x zy3Sw~;>)!6-QPUGZF+IV=uN7ifx`1had5Q+%v}18{)_YNc!huNU47$9x@!h*F2Nwx ztBvI2_&?bL|E1ylujlulosPCQ-%w|DncjR!GeoB0_V|ci^|MbyvF?+hpl_oc6~$T= z&xGVZf>ELvqRw$5@=RTHDVfYUgn&kDKpDuK#sKTloE^GNOE*A)M-iW$1$xJk?B7eb z?NYv(PyV%@;?wm)T3&G0%@+6bAPJ`*t{xA)?l6S$?0AFs7>=tv0^Lk($$0tGw-DGmC*rUOO-k~&xhJS9 z{X%(J9kEe zFXZ&#L|)u@-*1#U9}Z8V(s&4Ly$5s-uvHe{NZ3Hh8*#s-XL9G;MQguO^}|4W8ahxv z=0Ua_6^vR(1-Cn0MqBx$He6R87g-2G^E)Ax7p3gh!$3;jNCdrJZ@s>2q98Hu41u&8 z97QGlO@A~IfUBv|ZcoDbzk2YXreW54_HtjL8GIgG>1Qz*KAuqs0PgEy(siR7`oBVL| zQPl&AI6mE6BL4M;MP{dbyyc`r3cJ4cv03kn0}apR&wBKx?=M?H`Q3i(VTgSh8!Nnd zIntR8PJScX%XLzqws;hBQ6|q!oM5=5Us5`<9L#HL@aM^nd-A%wRF&fWVxY6n*x6%2 zH8JPC-+*xwVrLFw6P|Owt9VVlvGfu(QXSfTP9qNd?E{Ka^^DP}*4JRupOf?( z@5+)LT#hBAWJCXQz+=9rP(vMQFn08W4xzXtw38|>4w_FQ7V@vzbNDXnxph7Asa{cEuR^y55$zzSG|m{ZJ5dhJ;{p zJPYuZd;MJCIyN2YCLR5p(Cb%kgL(cdy$WH~NoZDauTMdDgp~<=F1O5RRs4VLsO-W@ ztB36rZmyPovIAxy*C}~;o85zOLI)UYw#&>H_>xY4}?Clu!#xEKu&lA<Ng3c)(LW zb6frhKFH}HAZ17!!1{wY3zhG$eJ%H%mOrBw?Twr8l6qpggzm>xaK0FPw`EORf;{ie zzq@f_Qez2KPS~{$JznP*KWP}VMWKZ8L8iI zl6y%@#ajn>bH<0SSG8Zaew2017s)1sE2750EI%AT73NYRfg}j_S~wwPlt;gBc%0}q z>803rI%V81JGp|X2jkT?_$-fmRwMIxm>3W)d_OC%VXMsSCVh8_7#PifIj>wOi6vnx z4>#`J8lOS!FJoBqPH|-f&@mcA)6~ilCJst-R2Fz9*lpWx?{g& z@QD?0f`wA>=lgIgh3V)GXrvb_&u1R*a`Bld{U;LUmTj2>K7MhdY|=-TjYDF@Uev&mN@ET?`n|3CafzS(muZ#Zh0`lh4u5G zw#l_Q&3Vd~9@zhBbfwAh&KIY&rRxWebhH#Vo?2JlqlmdP^B-COygYGpaD34X&w_Wz ze#8FWhnhVYlW9!pqi?lng|5*f;`VGX1)|>p!n&qGWpEbd4!E=7-`2B(ATVZ*Wu#C( zrUIc=jW06?eVN|_qspy}S-iJpR4*<6sQ&!((+-!g#mQKO#M%|S+v4o${4K=Dk@0Q| zh5|*QK({Y4N{CVx3c{5VI#iDJOnkylOEP@Yl-!FEBcc|M(p$bj!$1fQ8Iu=5+wsPb zR|g(padzJ-1lRPbTgnp4yp0pat%RIeg4tYEs$GyTlfaBpRubRFbDnJ!8BaYg%lY&5 z+PUn05P)jyXU{ClE4n>D# zIgI)p^3XsfV;BGf1l>I`scX5f#v(!gY5EqnAmOFZe)n7p$i**1IYhivK!~gqp5zda zkS*=E_2%N{3gOcuLc3vjO8o|=vBZ=vJ!HSlI6h@_ z>4*6yUTNH&n=bciQzGDR7x1m7(SWokM{e+>)|G-Is8Dx=zKBlw;7{n?>~4)K$+Ccl z-V6Twga2`>pL;Xze3yAwA96|N@qJ%ruGLa;iu@pM0dV{r+ED}b4tqsROTqO>GV0Vx zN%F{I7$N~=Gp%PGy`{cM_bSE2!iRnc>o2?bOZ-@Z)lAvr7DK@CqZ9^w<&kmwZ0GbE zDPPUeZj^Ef{FLA=-R&{owQM{#D1|*-v!i`q2TGRft|S zs(l9;<{z2Xp~UxE&fY+AsMm6bSk$v;DVMMqI7GP@Vv!#qE4A2k*rETPf>oK!D!}H> z(#s(c22a5Oa|wX;wxkd+B{A`7E42`D$VpoxSSs#fYl?87D*j&Epkn9r=lz{c&K}LT z{O<&kpE6ot)9)re;S?{h`li99R!Y&G;`8OiK;h2%6TzSJ^8@2U)we&<-Zj}^cTFht zYR{K0SoqcW3;p@iwkGw(Spd~I2R}$mhyKl5Sn?+2Qud?X;EUlD`SutF3M zV;3#2IVFMA+cJDuh|Jqpx5l(yYtRBAr}L#=epOcDFXp~JZk?fcm;#-xTD=AeE`WjB zwO`k`CG^WJ8m=e3O{}1+cWaUrKUn;(5E(UISN}D{ntQ<-F3yi#_U2LO;k;7xgq~ z7dAzdx`~8*f2ca4l&HRxiNJX7OqO{4d?U2J5hof2{sbiHH}zsJoVue%svSoJPiF_y zzZL{zY9Cix1S&gZ3g~Y6DM4R}$KvzKSVoxL`}WMI7Xer^;%@RpQAXD^;}*cj_6_jo zlQ8_|J)+iwSZliU?lB}9u};f`Jmc(lZ^L!S4;h+n)_u6sg$d-L)!b~Z8_2?dpmQhl zi~bz<>mLbbYp>sRsHlD-72-0si_!E=5BOG@x>y|FmYkCja$)Ptn{jeJ2W{Lqm+8>YRr2vg~4Bv1zCY;<)``s^xx)9K@WxQh1TbxadWB2FRt+BwxENH1-9?9|G zAX_WL(xz^X&E#wf3`HDjUl<|%*HKmtJ@ z;jwr`)i!fj{?<mG860UxJGgkbO!3X2 z+Ago9S`msjtQSy5zXkbdKbD>`ID_@HV80ymNJJ)yKNq&gU=LRT+W;^syuNaAP^X%J zgjFE*t+@{;ON78E3I4g&c>l21MUmEMvgjJA3(j+)nV16@13?U^vFA5eel0hLvX^({ zyztR8ZGRyEE$$5dj5oILF!bQ564a!>!8A$!lqr~{ihsHaPP(X_#$x>;#QPF=Wtjbs zeYs6l8_9*g!Zk4RO&0JgSp(g?Gr(l-+YyXYTNxn#`!EtN&+#gp=QL>|p9bK|SXwzB zx!@0~1d4zuj5XhvlK_SbI9>aeUR4WfdGuC!NPW)jk~8gof%w25jvfoQ-jW1qSn8zu z>_KFa*0t4kR3vx*B=+mPNt^$~Gd3=P6Mud~A=^%K3MO>jLQ`KMP0_rJHp3f79Q>y- zs%!GSYn9JUyPj7lD$Y$eEZX9d^rov!|{dQ4^n^7@`5%CRl7 zFCYt@qim-D3c#=|uEgtiUtTBO1iCJ(EbDH*!Tj*WX7yP2ZbtVFQVR`Ud@afUdMfL)ZJoXyhQ1m^d2FISo31^?ejx9DiORh1UnAz;fGMKK@-AqPdve4;<=?1K^lb`v-Vs2TRW@?HiaZ>YN}HJI$L927 zO2LrpW2gR>qhlz{5wg`FqzR1WOR@IFGm{pneNBT+Y7Wg|{6_PV!#F9CY) z)6Ek3`nyNpG80{vTgaPxdhL&Y@nuPb+!S?35V=G2DQ8Hhy@ow>Kp}%HHr@B-7T8ZBNuRZ-rXs z79DJRrbd9zjOxNE{_bs(=9DmKe#}e+^<|@R4#$gQs0`X#UN(LDz_EfM?GYemS{vaR zUjgd9hvvhtIgaKMgHJNrwv!q2YGO~bkb5zr0eIGTUEf0N%{uHY5bh({KdUJiFGa#- zhZQ)zZ>NcwP@FD&y=!%RfTxS>Gy3iN^hnuS0Jo44NM3xsebjWc?a$7wS86p|t02qk zubc30jYX>;`Kk}X#0Hu*2y#iAd;JS)=ODhneBuZW;8e_J8ouk(dZeVFr{QaYS+Gv~(5;_o|U@ zpzUz|kfELTUN&hM6mYi6*ja<_p%wXM#Q?jh*IE*No#M<>tIR)1O>pYefFn?gUH%(saRoLhbYM!WWyCewSPo(3ruIw_= za?tTGeqY!L5Jk>fUBw}8gWVc0M6`=j!H6nM>uye0G#MrzA7ZI<-y0xj{dj+$WkNQk z(Dn@K%6D5wC}RGfJY5&GV zDz|@I*ZtFvyQc?s`+hR0XyUboqRQUCZ8t)?0mk6(5x5O3CSvUQdKJ7gf*4`Lz7k+o z8%!`xYuPI6d@V?Nf%Xa|QQ?&U%k%rvukZh1#swa7d4WKQXy<%!SdC+NN&jz0BTa|< z5h;_;{(PTszBkpJ^LN?5`X+3R^`cYmS%h1^Zmu+S29I%$jK{AN$LuEOo2`3oqo}|%$s_HEk`V6qyuI!-+xVz1xk^W}tO^xT&s*Z6adpRA)X;J@On*mim99I%erLVu#N;il5os`WzIQDJi{c#5MK zm4GEw0PPDRqY49hioVhf7t3aLq?T7*pkiThT1K#owArDv&TzgWp<^h3f8|DfG*kue zqR#~ATm1HK{&X|yc5D;@mVypLK(jEO^=1E+hUm2mV6RAjEl$h2OA*b`pni*@&9MP+98`fle6)S?AD4hwsMla1vRQ8S9yQC1X3?Z z3LEh^&d)*A6N%X$j4F1ij-^Ra?NuK?ajKy&q6$dDSwP#iU9t~1MyMOTKktO6vhoOSbI%V(asG3?7G#h;| zK)b3jOT~}|;?j+PDZ@AQr+Ak?a|W;ABUXW!r&&sYG}t2qW&D1A&>kMphI@S#jT6rJ z<%X;m>E_Od-rxvl+ISn51!X2DOyIe3+nn$JxqlFWq;%b#p??*O+L2i%?q9EhBU#vD zWtY&kr4jLPttq~?#yb^1b{f;{G<3a0G}@TvC53Cbah$KWxdM(lqt^_BU27aGvMD%T zRf;qtr%Z-heRi%_ASqv7sf8Kcv8IMt6<^D&{6&<-SSZqM`xRM=IvnP70`vKKaXJsN zBuV#9nb`ccZCtP-d3G)r*7eR3$2(K3uuaZh{f4pZyQ#3rQXVL#By;@_2sQ=Z@d-fl zi5J`cS`g>?FT9XT0S(bv;V%xV3iw~OhT_IEY-)ZYC^yU~)UsE&#nf&3xpkS%8MYu4 z3~cC>)vl6=CAZ=TJBoOitw81_Ob2nM-&od>5!{JPRMd_Ty;VOsX|+{W%xhG zTAzJUC%dQ=oBxu1YaGj}gyypHJW1AwIx?VS2@mUn(zh=JF7ErUkvC3wp>$;tzG_l+ zi7fuc$7s0JIeYU?Moxk11%(%IiI2Qv0PP+xoJdXp_Fe^Nqh4f5*xif?X#Ml$>yg08 zZ*j`Zskw+()t-m#S+)L-DnCsY^zO^QUo;0rBH+?hYB@f(DgOD7o)&B8*6@$t zIftV`o;ZpV;ez?=AP%ZtqB(J1=c`gkvNI)0+jO`&+O2D@#9BMz1N^})!1zKq7jD18EWcmR!5wodKCa-yJo zxlje)KUWSTn^pH)m~{D)Z^KHtqW#{thDl>IfKv3?MA@Ya77nlwv)BIEKo_!Le8=Lls9Eq)HZ7K+;s z*o~JnSDI$yp{uN$I9}(E(}wm`8aVt*dv@ zf7Q+(_juq99)2);4?mM}ihss`OlL0^#*BgN<1=dU^964GFnaiBr*awnOVaq5Mi z_wj~`ljq=Xe%~T_N0SH@XR!SU*kKLX(h4vD*~+tBza?)ki?ajq?al{Y(;)7PAAPq9 zgWBpQ2kK=P7nF$1EBz3QQuiT0h0mzs~o$&K9&M;e|+$ z?V`a>x(!1>v>wtl>@Q`B&cbk8?5y>M0zoVRzZ6QWqQ-ZCix8wv*6A4G*7q#Csvy5D zdTLY@>8c-vISWCd@vO2G9355mlR}-xc$`gIgdiFh*n+ln`l*c3qBrS8azJ59!xwrK z(;TvmhrEtT!NLUNaMo157r^hq%y3!t#2%CTRblx7?9vC_l=1YZI=f%20y-nHk;re5 zc~*Cyw9iGHZ6KuUhJR6ly^-O~nsl@weWl4G5Pi%@h1=-ghLQ-}_^H;AL#h7fj9C)E zJjQ&EmZFxTkPW zyBq2JisT+{el@hzT1YY2QAysQD;q{>Xqy{+G5)V#yvGLO#_n4KE&DTzXrTfV>yW#Z zs*@T&SF{bX(+!@IK8!UimK@_0aMvm`BmU1WV%)zU|J4!S`?-FEn>#BAtlRWLjmUzQ zC+r0;>9K=f-Jbe>(JUPm;{sb)_)lB-pSS&g@)wsyh2!_#M@e^=L4S!#zx{~iJ9^2Z za`3A|mMcG<0EL!S9XVWYKDJ%GHJ$oq%5Qpc1tB>Jz+I7xv12d(W^6r)6Lu z2}I+A)|RP_#YX83&h(+*vGG%55z8zxZyw-tYGNureQ{?M$kQxyjpptZF!-SO5`@ls z?N&5(U0GYx3mJzEiGt{vb~HCkXGGuIwS%nl@tdmdmFu!g0pH!i-&M2r3P^r_YB zVFPt6U8$x+KE&Yag62QFpjjMr8)^Kf5;X6N*kKGyxHeJjf z@oWeRn`mn&bn)tmY4J@Jyt1z^`B_5Uiz5&uNaSH(A3sj1Co(T68_zJ#RqtWR5Gnx+ z%}_K`-II}lB8g=pq*YMBZsBx2b`Cm7Z^@$BI1BJnLtAQ2X{?m;S1*FMf9}5cZ0Sb$ z`N>pp!Vuy?8X*YwRu!4m-m5QpT0&hD^;>&KQ=^#&NfY7|QyEfJsA4knZ#T)v%PDOV z2<`|!81vKQ%ATNOv~`aRz|F!7rIbsyWgu+)wqgHNx5uKp@u;_wmUkxKJ?np2;#16H z;bxW6NI3h{tfI19SQl~CNe_-%Y_NeN`TdL+io1`gO->we1rC3?h9oSF-J5s}-5Vybb|AlkY_vO43XJ8E?$5JpWY! zL|+@FVX1C3sK&Dsbx(wtm{&w1vAuit(dDNFS{oMXum9d#c#lrKe=)wLtB#O9PwXYn z_4v5!{mF@Q+_)Xi<|E%clgW47xWy!Rx9qLSH+kxG$WZ&;=H zy%^;xZj=RtO`mpaE>1o;dB!1a)2^ z*cmOn@Rk(4ZW&cr$jg=3o8d|kg+Tr9KPUYl?Y2NDl>GQno7NwXdg@yQ!8m}}v-UDx z#~?oRj6GT&;n6y%hP~J@#2rg7zRjbE0Ds4N!&QWir|?jHPZH9806JbGKS3z!JgQ~x z&QQRmUCom$s@L!0ELLm#9`N^^S_8GEB;-&jj`^I=@9ktKyY?Yg(4O5S?iO4J{Ewk9 z!m(?KE<-O_NO8vfVzePXLW^!veGfJ^O%|?7+&R%qWo*Vj=N_`-%JXCiT^bURMLEu1 z!dXQtZ@WuaA#CY>J2nVg1NC(m^d6yjX(i|fc+NF*P$=$sVeyRUIfA#joR_9Ti8{^@+xChu{_YTlQB63v4D+<=G3+ZR)I^<@jVf1!iO$4tO z&ogEcn?Fq6n1H%B@UN~W6u=GqUW65G>s@p9^M-n&SIw&7*38@V=|++%v>$>Cb5r|> zv;$nm`=;Ldo+`RE77jlWWi?UVMastE^-|tl|1hku{VvG3H~(uL%!R!bs*vF6Z25*H zmmY0kwxERd7*WTgjKhaNNnLjm0*pKiZKpW6)0SKDsIs&|095sv=ESm?|; ztQo#8i^%k?KA8Uc1_72ONzs=D2wPuXQ|LGhRoilTn{h6j+2oBt`ms5=o{+-8RznWB zS~Xf98tjMo7ttiX|1J$hY8EOR!9f^0TD*=u4}Dng;K;aK2vj%{gy;8eMa}A-Rf3YiTmSuwBzF0NKEBxw-cccxuf08kG->jP{ccY|4f0IB-(Dtsz zfcjM^q(?3ppq?oU0?p~L1=f6TY8UNgLx!Qbb`jqQPQ%P(_rj8^j5JWXmBx6Y$d&xH z0CK&14Lt9(^Zxeu+}or;K)cJL6N_h<#q)akVLQi?7cmZ0Se81y=k(bd-+%0-8Ny_a z2dWU!Y%s;y(?A7B@25Upfgp~L`~L97zpTKqKk+rsca(Dm`jgAsYH z?u`rI!$IhU(U4g4op`hRr9t19C{XxE_$)OZxi~uvhpxkVE7E&2)P|Bo|ELhO5Ck=i z_b)CEqxLLL@uD(`oZ0FXt>UDtMw4Jem zkK)LEi*AGymG;M46U&H?hbs?a+B7y)m0|fC3bh~~lHCqvXZpAoq5s>Fjm8w0A*{8{Ij4W15AI>4hlSi$f;ksr{m zvBUx$%UDx9mTyi$oYI+H+*4rUKD8WP7jXj&5k}IkpmQ`L7X~@t6H+n0g#B#8?@a2i z^Na+6!KRP!wT(X`Ah0eH+mq(&_I}mk+4ycd)LkW<1#ZuQnl9B2JE*+$1@|^>53g7B zpKhcJd8k$BLBm~s^8(gez8a6#v}zBox((+i%@8>=AVnlX(E~3a5G=9tDnMxUjl7a)#P=rh z1zwrW$4qR24OYW03nqi@!9?3p9}Msl05F(yQ5_u#>b4`@uf~x1fO?83C z&Vu-_j-uG-iTN{M2|5Dm2|B4m4^Df#Km;GmcwDAj8@aR^)0*pLCY<4Ss}D8c2VC$kYwt`KO7TEO7+sUFB}J9R0y+q7 zdMyaciH2J^Grz^KpwC|au;J2dAa7xp!9?(RU zO*SFUWJmLD^xB{C_f#(7~Fu)L2qM+ zm;M1U*tmQGzBJgnMhLJ!c0JPrx(*=yjD`l{$^2FZCpAYIT-&uEWrSZ3YSMN>&@lFv zen^uWFmb(~PUxadw*whQfc1^NkaC0`z;)}z16sk!D79F+X;iyE1KaBPO;-zg9@T+c z199#7=}>ff?^A}}1uT{{Oh;N6owvVX?Gjbn=KJVOl=Y?Ejel{bm>$k(J$yGivO@{a zmv7{}l@6N!P`EuX;-Djm2k1s{?LY()Mc1-RTuXs3z;l zNP~V%eRm3nNDdlExoVUrFg~SJ#9*8M(-i&FY8@fFoxTT=^BVx%>)WH&_$a{Ix8KoK zFFkVVdbU*8V79pP6ezZxH_PvAbZx&wjP4kW`uQ1vP&V5qdL?=uOMIt?K$_o#0t9b+ z_zo*laQ|IGm_QXP>hAgkgIv#%2lNFcK%K=$ulYz2bOgCZ+p;d_X!{-v$fferUM)H( z)}589uVzg%8C<`n>Ui&(UF;PU39>&;Bx^Ss+WFM=A5OpmXLejGk6^v@oz$9Mf7OC) z``Uv#0=&4sC5cS>hjHy=Y_Q0EySAfEyk3D4uy0UF?QO4I^R@(ukzWLOoDTc~eJFQW z9tDZsvTnoW!OaN}nCDtqc3-beF%7|o*q{f#fu@ihZab+MV9d#0*>L-79XBhj={ zkPq?cRHGGT{IY7Un0C?Wdg*jaU4PA8f*KaMFu-kov2eP(+J4$GN`A9#IuOT^;sPoX zM4c?trg>dyQrKSJJ2bfifo)16FoXqErQvHp0bvAyV~3JP;5`7BI0v^Dqf!K`L9R({ zJL!twkXK9O`4Vd>&@*Zv=Gsg)9@siGm9tfRlrCt;GDE9u|2#1rNb8w#?~O#>1N$#@ zdRDO0aGOCT(dOoje6F;a5>1hn8=I>V&C`(sl#jstwD>hB(KF^+>IpJ~U^HTMx4NEK z-tx`NIxb5qH&NzB{V_hbjmOwf^h=tt*G!yO2w0-cYMmd+XCd2(Qc6kv|=8DI;U@V-nT1q^9eCLO&yn zf&jN6{&R91l781Y!z{sIOWtNP8&+(m%n1L%dQLW6shGLCtg^%+=F~mzt0)a6G9Fuv z?|9B2Q6xz9Df*{kc{GY4&mk4QJ-MK6pH|9sBa2-f(^nO2?BqT8Z3PrVpLbilPBu zbbKNF5I;3|JX{~DiKz?vBiWp)`ih*?JzRE0;+c-uv3VlKWI(KIkgZtD@*EXxZBe51 zy}Ye8VsGlzPsj*Vo{Dch|G~^Ul|xl7HRL&VA#{J_;!krIiW1%Lo%eRS#1HOVe$*n! zUZ8+V^MLMlzwTczdvS>dLv4AY7cY|7V7=F0sJG8wlsMvsak8@wZFo|pazeBMBO$G zlBo3Ia$7FT47B$k7_=Z++;B6F&OApuqX^w88cIaEQ3Vg|{ ziF|A+J3fhFVnGdmB>Zm@zbJI{DeF$qu({D4pur0WvTKx$!=GNI%fgdSNAGq4oinKM zFhItUyiuIkDZlafm)byvuDj>JK6W_^|D#lk{sR36n_47+U9B!LHY&dql<<@sj z-3Hv!fZud2w`b31NdZQ$ZA}yLvdUrKij$~zmZe*VC2Z2?;`P6`>a-}Ee5xrIvT++t z#~3}09PvgY#6AqNq_{q4*`6b7a5=$J6uWsUW_Lw8S*VS{S5*KrM$BuOH9g_#6!Pq) zb@7jYKPG#P#gv>Ww_pJ)+g(|edBktE+~@Qs@2@(+zzEKu>#1+Db=??EI?os3D4!Os zw-{;_pN*{T_kOMzzK%SaoKV4~?D4*N|NrU1s z2W&y3YL8gf={KUJGFQu>Bl}KdoOq{9Adh&7dG@etB;|%HF$lkGwb$lM;F1WxiM1CP z2VE-vu%tgygj~Mjax-}Q!J`!m8!Z(u+Y9IpBR7<}p@(=;gbdG3zB!_`=v06k2I)hQ zq=2b2-69ka*J#=mC{SyrDk_XpcGdkzWjDVx^iI!;u1Xcxa%f;{dT=pxOU)@@=Zv$(`bH-~v(_L46Fb{v! zH1>LE79!EkYMQ|zwms=DM@L9)NB6#+A1|cc2-vMZZTfLR?@FrAU^QJr$#Vb3mb_W{4;VhtX10$$aPPSg^Opxz z=3~Un@@bf$@9D!9du0-R5#30|-_0YH?^g)3AwT0ggcL8$XG#JTOI7`eI%Q60L68N8 zx~>yx3TWI>zC-6Bxw7n)sm0fHkLAm)&0Ri1syE|K=9j!II$A@!jJ~-x$eU|v`JFUb zj^D~TPn@n0{ceX^()Go~D}A=65CNn9TFc(vFyvI*jJDdBr8QU2sbbZg8C}y~0(eA$dJVR|joMT^mx!wuul9dhW&T32&zUpUZ+X(>*lkt0O62%qe zBrdqDsB^r?|CP_AZdC0GKm;X``$GsSQq(;jMu1{|K7Va_H{8=&H|M17Ru%Kn=0dZf zd&&JB`_KoRiQ;ewstamx;&t^SvTNd)5W5(!w}`C!QLCMAgY*EWa& znOrKQo37+v{E@5q;lbg|(HofuS+!pflaH)n2pLvmW>k-qJ3aDbEEtw9f1Xa4X|Qq@ z^@Mh3c?0n)}ck2CJspF*Aad zLuvs}!-miM)PM^iE&<|;+?uQY>OA825a6T)ZQWKMtWJ!Gb+fP?2biRL;P-6YTKUm? zQy6aqk*lrCi_U}8(@3kb9j@?-Cl%SATwq zYS$qlVFntzE+W_H5}M`RMyi&(D{a@U+<3#i-?^vTeFCuJ^quwIyQ4(p^-B5?*q3T_)b>WCTLyWKR$B-NlC>txXBc+o zxQBU4T#lp~e3@LXv8(`Gk%kJzu-mX5d>=RfZDf|ghCxXfF+l;&;c&q-&WzRnswHT zO*iK^Q~|>}&#X{bgbRMq_u){=+L!RO8vbs5jb%d(A9g&#bI1i#u)4SuIuwAWZ(UBL}?YU6Jje?@UO;*OXp@qB@q z7OHCc!?95sFlh$%_Md;+fl_+r`jTC?u7^H{S!6Uyffc1$`k#50=+UNlvdZ&TvNvVc zsSQen_C^T;CnkC&b)ErdP1f_E=uVf@{OLf_>|7}6^P6ooT2ESky%PknKCM-RMd4Vk zMc6jSY1w4Z28l`T>HDUB+dWW!i>~pcy)53ji{ME~C$%XkMd)*VYF4`h3(IrtEaLD! zDxH~YhL_D%UColZO8scroTEw?({48A^A4#J)A6;ajcsR{7d~FNug_Z;Syp%a*8$n* zL0&N=9 zR5*vj+97bOH5+($brftWp30V?)I=Ta;It+dZX=J7@8;?pFRq`01@(fCM`0FgiRunG z9f0;kE0!>|lZHlUk?=gO))9^^SQCBldVOS!bYo} z^Z>;OgqiX;p!309>k^2E{e}6df%FxH1M4xmSi=zdHQ@|s^WjUKfeQt}HUBi{(KWbP zlk^&NKWKQb8Af2+q=^VwPC^B>oHD9(b2&}Smrmp>1}#`O;yKTov4{cVNo7MI(vI%6 z$JtFYe(8`J=#t?b>u6Fnm+5R}3MxzL*nfVVV&Bj9PQVtmGLJfG*x*D6-}>Oogk?e8 zY2M`N0WcE2Qi{6l-BO#B~kH2-&~i+HkR&W?i%EgSdL|{_;@2Z*XYTiz4^b2RNDD^x)lD;5j%g{cLlH6UJ322KNpZ`dtNsB zyn9;wFsF*+rUN8v=1yQd=w|)T6FC3hR9aMe^Ul4|th(EeIoLEutjhSC;-T+INplOn zKk}GlC7bL@V@@g)P6`F`EuYhj~4p^(cm(X8avsMh>eN18NAv&&j+((1g{~G3~xYo$= z(i-z(RbMePSL%~T{hBxbG2S|Lyz$F}70@pm$){f(dyD%cL|k*ugc6LaIVZ_sz|$E# z`(2Vi|%!EJRV>u=tknx5Gk3*HMTu8=%B#%|eWpXm zNq*-L`swy?3uNE>=M5#XC4H$axdf^z51eN7Fl6I7eZXDQK#to~jk z%hDaawG;!Y?;&X=GBxVJ%sad?dM_Gsix8u|rN7MB#>mJ2DC&^>*-mp8Am`9|)GBXQt4yjO#D%hhi}sPqx=TiKiR$=Rl4XIJFo zeJc4N6qhJqARPdsr%WN+5#yK#R(IN7>3^c5IU3A+8hn7M&U*d1r0^4Ox6<~W?Q_-O z+!WuL145Bmt}@h0x$w(Md1O&LkHq&#C19%>wI!8aY9X9&gPbV45m>fmat`pY%S0p_ zoQ^>8@%Co~;yKHb8#UG0do*OpCzM~j?GyEN%+W_EriziqZM?WQUnGuHks>&8rOwn< zjuJh?EJ*&0e8r9z&$-m!>U5xnNV%4A-CPSYIII&`BleW22RTO*;{f~99IihV)0NYm zq13N63ngDXn@o`T9(fVgs^qQ9tMf^0%b|KO^_L1moISQ44Ux!XtGRYA0wH2@b$!}f zjKX9}qrES0+wLZenEw)*sT`780v<~2KFIV=rU_Lk^0kvXyH~t-IJKTFD@{(u>?gtW z_v51+{&8MB(!4{%a8{H{6KcP)~e%og17ndvD#Fw@rzABM8M zK0KVdnUgk2NzvPmD{0pWe2F%Xc{MQ)P-mzAc$-H!FfaG9v!H|4p~)xUmf)zY;EsNt z%>lI2vAqK^ahw?|ouEYj{Z_=5*(lD;yD&NiRR3GV5-{Si2CDG|T^`J|bJQ+Bc!)uV zkr)A54pZKy=g0_NVM~n|t)ZyXAmg_;Qm=UmM&%>*DTn@04H~h2dVF-wVkI}xst>7d z*?xV!_yxB@H?Z48Un`@Xno4PJv7U+Q#odU$R8Sh4DnS80!3F_G6^Y9@!@$YrpwaGA zf6^%=C>l|1d%@G&MjEe=4{f{LTx@ry6vi@?Hqze;$oFTl>~y*^8BD<6N%f!$_e2Tl zXp_1y5S&0&NY;?#*}XjkfjF(e$pNo&=xbpee>1K0!~v!L4=kuH(wMh*I9I6{<3m7f zA=;>T!fRIRDMKP@EIcNbjQSM1bdITZC6}tcj<~H{nU3*bTlZ9r9SI#b4X;1ASFr-K z&_kC%qYgmNi_x%niSYx|By{rQ8;i1-y<4s?N3CG5#P;#=Pa*3?4Odc^QJoe~Gz2ON z>TlQ?*xpAp(%Iy84yvnP3LCp6If2C@s7=Ro&5m*bpjm0s9(2-50D6tRtQxS!7>RVn z$B>6&IcyL5R1S*{;8q2oWY8*v=5BuujsmPU{wQ@B42TQ&3z8A~+)+TOvf_v>{mSf1 z+yV+l0KexFq|@YDF~cs)f+~Pc=Cab>ZTfaB~MVp5IZaG8AHx@WFK3m;~ujs z1k@@@8vupjzcj-N@Nk!D1n~7d_oSPKf#9V!*&Z=nb?<(N`vtWvqqVgo-Z8hgX}r{s zI5ji=2&4rVO`J{-J}gF^A3%>?DC0s<@z^W`)K_F5l@GWIEt>jJ8}_^}NVdv$FZF%; z>F>g0Bt>r{8hf+fZO6akI{ic>%=ex&kcDna;X`J%E>*HZeoq%G&Q-Kdyzh=@A-8C5 z7!d6}HL8A4Dm2+@CcuE=d0b%9GyAUcs`a7EC1QPmYqg$;Y0b-%vM55tk1o8i_ceUf zukOf8+iS^H?PEl~qZxFvCM(F=?RM~GRD2+6=Rhv-c7HEAsn_lY>?2ppqsr|;Uw%!i z)uiIGK(N}q!fk^R)p2cNKkiW@aIvuhXT*`w@o<*Xqz+F>GcM@tlZ1-U>Em_9%cn)w z*(_zH5u+QiWAOYe|Jz=7t_>If8nt(->qiA)(0e|iYG5;^ZI;#q=8i;G(Xp~G(XrS{ zgq_D0xQgz8Bj6f(I<0NfDRR={*XX?7BjdVT>@-8O5vzQMu}JB)5g4%_)FR;rMF&vV zt&^9>cbolL+@hp10MK7t*5*`aaa>&1g8$n6iZEXF`v}+- zF?fw=U<-!`rvG5#Gjh$H1ZIV?u5b`2N=i$TSj&%L1?W%Nt{T_h`ekKnD04g3_g`Ti>yi|e0QY-fNcB6I(T487m= ziWfPZ_XUGs!wC-~ZuR^y}JcwIr3Z#7joIRjudIY;-BOd;^9S?`dP7s^LW<7OUEr0~m>aFI8d48Ql zIb9ykRfLUKuW-@fU3zwByAn_n017Kb!_uc3NJm%Nd>&e9jHK~Vb2cMhf{eT%!}GkM z?-7;x?)2V*Dey3cx<4Hw8N~RzD%}!7`-)iVR-%)D*g^XA*V#%Ia_Gf~4-H}qKwoV7 zynR7w9>;JL?!F$vvOe&RAM6PaZyIUG=i$AtoELW7%|NA~uK@RqN7C@X3gm0i%ZpFt zu4A7QqSCQ+vIE_wdSW|{_LhrL)=6RW}RjxWBkgT_onli-gM za90AHBG!7t!fIK2+&Nbz5r!lV$E>_ZW$%rdaa+v=5h#MzUi!x^-PJw#<=0`RzeE~L z57Lhn-Cxtp$n1yNl6jSw9vw8{z!70i-gmSvZ3AUZ4^|XX9;xDgFBA7nHD!nZ#dsHW z7AZa0Xgn4x_J!88ORfPim_FaC^Uaoe8~8USDA0;n|7Om-w`yPijmz?GrP+gg(RD+A z%?GqJYh89_lIx@|Ql^)-NoVA`01nkJJd29dV>{fX)dA^k;N9r#bGhs*7Ot^+p)P$> z=wj*F9O^XhJT_(=>})D)>)Hw(i6#lF8-uG@?D@*b#UXBOEHEJDR@ z7QQ&1$@g9#&NYY+$)kV`k+B4QzMQ*?7GEH&7dXSGUDtP)LN^%eSy_J;27&|%%iS-{ z69i$@349jJhx66c`bnp}J;Ik69@m}U^b*9BdjG8jP;g5?aSyD)1Tf3LltLtet$*m#R&ucrNWyvAsn8FYJ~eleBD4J4LAW0if0qNh^~1F&q8+Tn-k^cqWtMz9S~h zcttjRGn9j85qft;woT~P9(?C^CmF|sd-dmGi$G5A`sGQ?r`!+S@o}EC>Db^G-4}>X zgQ9{D{OOkS1yaJo#(JYB$2J+4W?*cIUlzf4@O*KiRq7!+mU8;(WW7*g*~X`rt#w}U zUPUT}@O#;MbZAguVZ~sx@5gf9+rm$ee<6_^K*2|@wZRHNvbh1H!zf;_^w>@Zl`9)x zU>{C>O?J_xC}4&$T4O#JI+;7Cf0x6xVFTjA%w-lqJ~`#S(PRJyo$I|5yHulqTqh)X z4WcMxx|s}X>ss`CKUYrIOT16RmX$X*u>9GSfT00)>Mr&-Xzwb+Gu6NmznG0b(s%0- z9$*|d49;JVrUGX^=kLt3ab3Z9$8}!hQrsHHt4eye-y(zQP3JY+(cuKl>;le}@tx1`w; z!U%bF0^Rn6$lx(Hi|!jCV(i!u-$YW6=TaYPYHAjV|8;UM)oomVXpMexJS97nQKZ_) zuTrLlhu=1b$7ik*#I8E@0^`HSxBaa~1IDurPRR`hw`kC%c|1Xj7Z}V6N%!=_Z~jHy;etMKFF6vf=rX0)1e%j)q%1Wf3OqK zR+=g@iQ6=7^*du^O{R?jE@T0qkQu_$nqDO%`%@O;953R9Et@sFVz}@6}N3Q_<=|?_w(9cgKw<87E^hom7aiJpN^j55Iwl#vhSB z3aKz&cn7P?>Ln91m=tq=sKlGa8|S*W#xyT>St z{^nfts$tP{ITJzqtw!e~-@h88X>QJsP)ak8tQ+>qMw_)=K%=`5&9Kr{n4;`U%cTa@ zC8$gU6v!|#Ko7mE^&rfeY9`<2E$h^j4(87RO5rZ!& zwwa}6Uk_zuKaOt^Trpi*z7Ko)Ce8SU3N5Vl=S8bVCC5~*3^qg9F-yfy)?_ohtvO~1 z|LRvGnobFmYIW&dq1l#sd@&};kBFwnMNOs*m9^KriJwRQPMlUsvm$<@A$y*p8uZ>O z;O>2iV85}Jlv~YvLPL%r2Q^yeYEm%h`rn{SZW~3*{CEtMb%M8;FD@TikQ={KAHZ7Z z@27NNn?F%sZh7-kBdRfVd^oXKt;FA4&>$Pb!o0|*mLERgjJL>BjnjfT_CyEdzh_$1 zc4oZX{uy=yK87S#Fx8xzldR>xw}N`6%~N4^MK>gz{RQ^(S-JO-R+S>+Pmw6E@u0GL zg<&W9EC@sXaJ4=!G^Y`_FYaev)iS@k)%|nrwhs!2pJVdi5?E@ zyjPo0N)ux8E$P;Dm?SSjOp6&enh%+4L{u(Mm+BLQk#l}?!y#*f!$|DZ7SS`#Ch%)h zg`K)6-G`>~%Q&>FeEBgfhl`8AcAaUdd#k4R>tK8=<{ACLX zA(fevvtS+J$6_qWbpwOu)N~-9s?q$f2bCB61$mDIf{B7+81_CD7n}1T2HT82<;Nc%YyRzx zyyZ4MHGt1$Y^<#wUVZnu9`#xRc0iIhy%&PP6W%UU=h3s zL7ET4ZF}UXDU5hIICw{%UH+ZU6l-iLw%HVi@)LKutB~`;5j#DXgbDez-N%;0^;O6+ z^aaO^FjBEDn#df`fLiM6zw&cJBZ8dXzXL!zTT z>YIp0JcBwc9?D|B%Yhj9rJ8|%foF~q#=L)%w0 zK|ndyxc?3aKq#pG;S6m1O#rIqLXY?Qw-IM z)$oWJH7WO}M#O1z!_(cIdlGen3Meg}=2LpS_PBaERQvf7s)7Rj6^Y_rXky!d+8Fpb z>)W^!c^ml!3wAOK^MZ961d*cq?@Y&j^En=Vk8t!d9WP;fRkA{8!gH2nI}c<6*%{J7 zjJ6mz=#+FL?3AWXF9kmZP1$h&k>cnDzE zON*htVG`F`5C3d^MV2RZr(a`93Ce++oJK96GN(U#T>Jn*vR1DbG+iv#OFlXv3z5K( zK=kT@O^ct)vIn_oO<8n;DJBX8aS|hWa5qm;HTyr9m#vV?RuYVP=M9#^Dn?iQj#QD_FXve-8O_@y3U7Yo zpmqKkc(WRD|KyWakZ}QR2HDT;eC8u4-?DL!r|H|I`OW(d1>t^Hd_UBddfho!2htBcS`6_9;2Ti}?HvK9Hzw&Figjv>;FDED#TmHt z%Y$Gyf=fb%c7pttmc!*6Zwo_|R=j4{=_A_frtL13r`d?xl?Ev?b)IKTF;M6&Jy9?N zN`>N&#&4n>Ch$78FoCmxoz)s`n*jP2pgo5Rb)_g`5J`8A?)W}sP7QIz7JNnx^{~Hc z17Us`Ubi$p)J%)`s_bjRX|Plt&lQqqOTTh{{(#=y#=g%{Ey&=pxm9r(U6nBZcJNpD zxzp}EC#wcrknmqJDGaC4_%A#OQX}Y)T3H}a;_Wxfej>a-HSmqf6_}nW)o}eeqf@nI zfjb?*W(C*PiZO%J2?Yxgz<5DlN+z_`#C93B5qRMtrg zK1GwtOv>Y>J*G;f`+{4YCu$KAqrJ#j!bO2S^(u7_^QxMD6YuBip6`4#j$&T{P2I5a z8Ut393CBFYiIV|uWtr#p&b2OoRLCCv6X&M7tor8gZ1p0^&pr&^5UTt#KF&2ok9qS2 zg}nlPgQ#HhR5hjrPM{>8yos=&SNRE|i$dy%&25^Pt+q~DL!bL`k(uxW5uyx9xIWU2 z^#ylM3g7%+L%PRnhiq^=-)Vo9a$mrP(_1b4p>45RnF{aP{ziBJXehVp{(31x!Vi74 zaDzcfAiL%EFsRaC_wZ%jGi7VLf5!4l+(cLsVFB@nyZCfcNL_vR(_-8rtNpJ7V%I`A zb;>(&wjSEo9fm&tkb`(N%fB5VaFNGlb_k@~PM@OHwP-{gFRhk(9IM)8GD+97+1vAU z!M-?gbiDtM6r_0TA3+iRFu~_Avqe>BVq6hGw9XtpwVa(-{h27Wbx@s!HNk6GUMe0# zzm?B|NP*#>T(4f72@9b~{bC#8Bv7yc1K#>(D#=eI;w#V}ia>aJlsrD9Rln)yuqK)& z#|E;;K6JAZR!>`rc%1KuNzFgXzA0V#(UVs7ITV6-;Zye~QRi$7Wp8i) zjX$n$&M#TEL?0uXS^h8iM;@xvSX~aqi+a%$^BTcMD31T+0 zfm}sG562U%=W3j_+Z%3~eZ#x#DagaCIc{|F{`_b64-Ih)1w&r(;+W_+4QZUe-=cLr ztf<>CRjdPP$dmKA(yA!L{Pl&(58ay}T%r45y`04LvRZu=F5#C+Ra!>H^>BYD8x>0; zXmRz9aY*i=7r4IkC?WVkgg6OOpOl$a53c(d=&!*j#KKS59ynDvbA-P^u$w^$?WTf0 zuB3^@=aJCux`ql=%73+#a|@^#d>kp)d>UD5}OZs{G6`?J^ z+{4CMDcsy}I~E6uKc?pad=}GfwJTmY7IuCSCoK}e5AC0;mnc1Sl((yGz)iV%d$7&- zXRzzi&0Sorcv`7BPtPXmY?X$Rn9E%-v6I;8^XGO}p1$kAw&)fckDSd;{qnvdhJp$(_iqg-bCb6A=M>@!@=LhSDyAZeM zc{QzW=VXFMlb##{=AB9?+ZDCob57k+i$+p$**Do`}u$1AqP)>X$Ed6 zdZdP~mA`{c^Z?7>zBete0?PG(rA9!yTv%u6oUtp-N8ErS>C{0(HP-Wx5Hzq5UN{N0 z69f7wl=wJ`vD^92W1J#LgVm3}>%_wvV5{HN^0YK?R5IX#-K zyl%lG2E0di{OALffYp-B4<&#!3ng_&fK0c#%>-&ozYC0YTmY1ml!6IyK8;N^4sImI z$F1pJXNk8@FF{^Fx6>L~Thf%p6BZV7;dOK}-3Di$`$&v7u$pBzEzJjR9?(^~tJjOv zJF7hpGeykFyPLvmRLg_M4W++8s-Vu84QySl5wTgJ(0IceZ&XQSqW}aX#*Z==zI5!^n!n<(FJ5C5C%||hf6$$$*0hy@hAAopkSnFZFQMvJwmPW%@tP;IusyxM1 z>`J?}%S%CO>xn#h^Ht7bd9o_YG2+|n1B{_0zTOC`hQvGUZmXO(a_sN&FB*W;twNj$ za`Iyc>F!N-(-Z)Lit$Ip&R-mx-vr>Bac1~BvL{`|r>5fBteWJccA;a}A?1ySpDv?%g!S}`s489ZpD?Nq?hVWK{C#zmQ zolrHFpce+(gXR~K{=VWz(qxaxPSu}|Uz+Uwy;bBf6-Js{mjlM>6@8CF7+d9gg8b#z};tb)*u-kJ}JP&>OQ<(BDWBoL8c0 zUn;&$IYGm3O8gy4!CwEF4E+0&|KmGT-woXwFzmx(X)8DETrDat4GrCR1!ln%yjO;z zX-kiXGnU!yH61i598y2x|CQSYNqx5)Blv;5K>KXPmrSp_tB-am+f5YMc>m;=;+DyJ zSe!QyWM}Eo7>E$jd0mVpDC`2G&z!j@U7}8mC9ozHKi&%k)BF4M|2Ms#^$Mpg&xbHS zWT5e~E8FSelDtWJ_V|DPLHK{B;kAANq;iP)eIJic?j)%`CRp3*x_4CJ-RGkUDp)HY?af6lCm0F+Rh^Df!k zlE+1#xQSFf8LA4TC6U?5StqZCMOsAoVxL5mJ0CBGJhWW^ZPQ8X)F##7)@P4FSZ1Nt zg$FY5vi!>(zu-NH(9u*ed6nf9wgSAHaQ9nd_H^%v)?V<}{GoXV-&g-0Dv`y2ov+s( z&Vh5-(|05q?ZEE+E%G&*Fs+<78&UkS6feJGdR}gkD-~b)l!n5UN%L)CD|99`5!-mg z<2i`G#4ud2gKe_de4iG7Z#6c~!k3|5e7)z{gTK+(eb#XpvefZfvER0$sC2xyY}QgT zr7~3yrlpsG`7rtNvhWb%vnigOD&kr<&8HGcZOZ$hT>fO$!PQ%W?JXApPL0yUIanoU ztH34}IaYwYL2S_30{iuM{|@GI$(yl<{ahVFVAE#)0#awZpqKJ190f5&|S^xN13~f-em}YDjtBu-(tLs(fD*=xs8BkW@PqIEg&|P;|49f3B z_7-z#JB8O>Mfq7RjwN!dw{gYw<7zfYy>joNPRVtQjFQD0TjZv8j)^bp*IO99NeGjU z%O^j4PVnyJ*4yNRc}_fVgZ*rBB|7Lf-?=b~5Zto2Yy4}$q8guZ_x6~UqF-#XH}~zI zO8~?V-@m1PrEyYI>#)N)3${lrJLha034zUxsI_94erL>|m3C0IBok!sfdx2pN7I%2 zpk3A*!OicH<__rCh*4ApKDZz_(&yXqss+NhPe5WE(B!tYm#(Zq}#7F7~$Yjy`qRK(}+xuYzc%g zI=*^n<%rg8qaV*ZB{b`j#&!#jUUqVR%FO5Mwd9crCCt1Ac8QjOQKqJXBzpx{6dTtF zlue9D)vCQw-gaa4iebij60eV+DPE)EG@cq+_hLSFlvF3A=hgR2jGZdA@+hlumB6dm zQ;KZtyfPpht>SW)65<9s2y$ ziK#N1_%eJah*%S%EvZF6r4Y=Q>j?Mo@9NY7+QFqtmaF|1IH za%FXM7a80x1C|R4;&qs6{^7R3^K_WafyR@7U%)je-c9GWYv}5AVIhq|$WuQJr$eIS zZ#yZgmz1gM@ML3?>ECJl^BRu`nXoSUCNtvFInwBbU5i^TONEZ0K#alz#nrsWIAomu z`udLdb?pBbHK5BDAdHYV`u-lcM&w%eUrGWdOp6O5;&&fh!#|>dGZw~(Qf_`|Pb7(? zCdiPIM8qG#dlRj0gZban1a$s{Wxw7J#_)=j3~w>#D6~il9jhR=mf$x}khqqHzT=!N z!$b+R$KR7gs{4ZwmUJ8XOaKSjdW$x45kOi9WXqGyyO7(n}B5x?z8$G zgZ>z{a)b5X{4R_{Q3aRxaIhC^Am1dqIZD&{YV* zs`vuzO6zgQ33K-(%E8`#RS}9J5%o#mM;*Jg27$(dEpLuIjNV9kw0uR5ILXcsABu1h z8hd?`gYC>caY)J>PAid{b&2Pw2)#f^ZQ+jqTfj{#`}A+~UuP2f6@%gndY-$#y6V(& zXWP&@g~?5|gB0qCD+@#5)X zUaMR2`aU%nwH&|UECacFz|Hw~yU7N*!NaEP^wKXWPV4BOCI>-kg>^>#zzx19m>a0>!!6brEFuTy1l&Q zB`318n}y@I^M#c_o!Wz~l?pG66zBQE?>0wI^6X)+o%S?frD~V%#gH z_BpH<3eoy^9TZt(xX(E3qS<}A-AswnWa|NftBHK%hNZmI>MZ&7yT{Cq6k5^|c(iP= zq*&rEUsVUvWQFO#wMq(+R)bd1l(0S<50;-SZ3^ut0N?%IyceyGkkAX2W?1k{I zp`n0pAA<930U`xER^+l;+DWT6zm3md?!0dRj=|78AGgUBO@fFEUB_Ix-T7O-U1wJ7 zZSls;G}{iszcP4LRu1LP-?7 zj+I#{WmHAOD0&Abm_8TX&dC_lA7+=yE(6r+msjj8Ob*Hg(9i*|uCkq1Qm*es8B1wN zNU|UAukfFM+e1hfJzvKtV5k0ODemY=aNphO?W?HZ{IF0K`IJZLo>x4BNkY`yTpF0k zwg#ndBdC<}t!Z}`YPqUR=ZJ-!1t0A*F5%7uSbVEAXIF3MTkBfe+9Q(ulflEoBZ-J6 zkwiGKDwRC`h|u>Mni79F3_`;ona)S^{k?iuca%_f;|1zbO32W5XP8-(0UpMeb+RS@VB2f?|1 zz{`pdCG4s*CGLwAt=~elK85D#y+NAQXd%0?vL$>!cvPUlTxyPI2rFM0ltti(&T2^B@s5c%GJ%g?#juZu&+u+C0le#UTY&UhW9r?-xy(tY z$3x`a7p~)P4Ww#idUzr($KPn{&=+SwAQuF=r}(!!+1IZ5jjveM&w8m*s!Xr>zcBaK zVNt#RzPE}JQX&llh>A#yNXH0BDAL^}-Ce>UDXjw142pD1N)3&4cY}1-P&2ch#rOOB z?ftvXKG*X+XYYNkbN*zQSnFQ*thw*c=ly!WuW^w8KXi1*tR4gTG<-LGD(f4-(!Rc+ zYmX6zIGubxn96~-aD@;wAN-+}YEDPkWj$0;Ia{E0s3$S+O}#nLa1g*hr~yH8{W?)O zUjXKt6dfE|ssQpAR>N`l)}oyx1`uTHF?=tla-?gd*kScIA!dU9>Gomg>F;OtfN@mF z8I+WWY|b?<7K!k1;2bM%4BPlx(8Ju%GAZ%MVN_1>Hr*_IdWQF>h*XjEI+<-wDRHkH z!;XzZ)(0RHVg2q!$NuTA+pjmLTLWdsQC^#p=p1?UHvUj$X1{^duPymz_q$h3^(U_e zjaS~vjZLYJTDvOz+hZg@#<}-FXN7d#frrb(~iYnKD}}d1{vc^xE^G-aj$lJF(r(2 zQK}x3EuXU5`n}|f;nhCkiXL@#2;|d64S-#FaFtzu;x~Tr{rY5-ae-}uBN!3#(H;>d zM&QT!6$967FkrgxFHp|m?RiK`z7~ia4n^jOuBppzjzQ0g^P}s-oyk)IcFB+*yf(u! zGRe9Ppb>9%r*#u z2FAC~?_0ErLW-dY;Nfd#)~`SZOLheb58SWv!temLr3h{~sn zGKAT&xxiL#BN)Qt?FnwaGr*m>lGOVr=?Z8??I>vYZQg1T3ch5LODMr&qJyha!v4A3 zkmH@Y_X*&csk=#80`Uh4kG2pip)`w(?%!YRr_j$Woy)$pxBq~vGA|O~yK?yo=PN$2 zeo-Ku3pg7BWNqKjIUB~3sL*pweb~h=z$S5JSZXBJdx62x`sqo6Iv-IiwOo?WI$F6u zS?R+6qJ3ZlY&QDS&-pCoSSJlO$aws=o}$U0Xgkf70JjB*k2AoZu2?6mPozd*ve@8# z9Dn?2jAdWdgp;{$f*Q|E_{IeQD(yb_>^Bg1MAPA?3H>@<+8Ar87eCMqAwISOcB|(@ zs^`6Ev*pvU*Hd}OIO}$vCn#Ei1$J_$E7IoRr{?tOFymHhgGOd* zDvnd>ztjrgqHzeo>0XeJz_OVL*c!8XKomRdLL-^N{pcDUVm}?h`)?e<*CL=Z;$*on zj)CW)S0=}gTdRfdzh?iOSpqXEuM4DLo%Z)tfI`lx0Ec8h5A?8_Vn$-$Ky);OsXwSI zjOX7ktZua}7+Z3AkF)mRmO%dg=V$giDLQ28um0wqoIC!zSg8LLT2=UaQuqOJ6}1_( z2n3A=&3iwB^bn&QS{gjz|B*EMZ=F$dy?krWD)3f5YP>F@N_Zt0T}AbXV8xt6rr?K0 zq}+FijBX@J$hZz<#+U@G(P|9Z-Hr($Y*a$d)SM<=x}p}IYaEYrKx_F-Og24Ve4R0j?U!hz-_%< z*E{?#Z_h|8=l_ql`@inxuiSU1?$~G)edtmJyBRbZMS6HWpu8$i;E@%M{kYOEXEV#u zT#^dWbEA#X!+W!%OO6}vTKtXd_;%?mf(otRU{qNB=f*m0#df*^k4UrvKa%G3`8B$| zrET4b+&$Q0D`~lC?T?A()K0l?70NR+PsKwsVLJj>k)vMfr}@V<3?c(^(w$jn-6ql; zk%Qk-!-93vGclbQx>p%}$<3CPw$$NP`KAVv}h<@WcL5ZnR2Q0D*LJLFOP0l-Bq_}C39 zZc8H$h1jnSnG=s&iA#MZ>@COZS^95n1;eM^IuOyP!w3HBs!iuiu@|4zS9BJJa{CprDQ9lN)v_vA1B~Se% z(4j}B>}1a|ldTYau&_;H3!~Mb>n~P*_Q>TS-T_wybT`ov;NU_lGNHoBCWsuw)KEX&uGSuG=(V+_Jw|)YaB@ z$x`V4dQ&f$vP;*UuU32u><9-**13kraUmc=^ur|29X-_Yj}K60d(2zopWg$EYZA}2 z1Ah^i-F+L|aR&Fp!fpyD?=e2@RxfoL!4^1Zsx7q|kw)AIpb4;PXh4ta24uqPQh05! z%Pj^}u<9jp2T^ed(8_#c2BMd+%aN;FO_U0{5ghawrr(79!U#FU29VK93wuF<^|e2s zuT?r0DdLJ^ukn}@bk;Wo>j~x`VXi6OrYh0$$l}6*cZ6%t<5-4;!HJbXf69VZ(e;jS0dsquv*t z=gkk=0%t^9eJcUbCyvN3@hb3*u zmG$of%^=PL71-`AUu;*-tgqmMz7l9q!99d)0po^v2JZl;2A)|>bU|SNz~+wfIt+Ss zz0}O>@4&w0K{_NWu5f%8(Yc;6Ld82IxNqOQ(m1hs)3tSTPRnA;U7_`y3Enfc60QVYP0;1hkai zWN=#dz3Dr`Rd$*-tsHu#_tyH-@E^%U2bqCJb`rBpr~cMs8TxhZgQOl(qN}4LMi2V^ zzGv1$VPpwF{PAxes}mGx<9F7a2t<`BCaP7t4&K;#Kl|g6I^e@t?0{V8qz`PD`qhRVHPX~i9`x}F@Ung{> zsoE3Uv(*HdW_3;dJ2~m@`O{vN_H*8EikGPrYa^#Q)7VtdBai2m{1E+;=(?&;`RExR zXYP8K&^9C`KTD@7o1lbh2h`9ND@*~D7HCZ}KxW*QRNNjUZD>A|bvA$`RDp2dDht`P z#K3n7Jme78FfAynJY;D@H8is4vO8U{B5>aahX2$VefUO3#jF7%d67wfuhK?2BIxp( z&Ivm2Ou&AxqAd0~IxHl9y)seh*{dgDt7|w`?ErRh2qmm3(r&ohh}yEzD#zxxI=6rB zXHY`|TFl*zqP37{#U}h_wyiOE7l;<9Cw^fwO-A#=mTb8svJwFrOEi0}69%4%d1_Ys zSP!mI9Yw%fQydzKSY`%!f70Su$LviIgHrX3KluydL#T`6Z}Kx$SLgR<%sW6Eom69w zowCjrA{0R~vG?Y8?c|uHjQLN71Rq${3m$YT_KbR>-g)0B=JD9AgoWtZH>gkLR1&~F zQLiV=|+K{ z&##Z0bF03S-k1B@w&G=NH7~@euBJPL7se#F%*otQ@xp;_+zcUP7c4l;`sQm}QGnF- zLj|@0qgHIapEZa|T(7wYkzvq|9rGWP4%XFjHW4LSm3p2zPQ znY@OqVW!7l*~98msYZtD-L&{C5V_T4X^SFVUuupYo9ZN26K!JyG7VW9x7S~PZA)XO z{`+a?&@B!rW&y>YkM~5n62Gn$wbq&~02QCm;El!6uBhx>d+Y6Fqp9$pwY}h0N91f2 z>;fO+t)*!h9 zy+L^3q9F9W4BZy1*u+F>_vp>LGM-thx#g(M zcs`yBED^ttSm-JD{%k75GGUI$!jw~m1uSG?b)E>*RN2pA!(o@y75&zB&dv|44Tq99 zhxxn-9|VU$FaMt-udfhk1rB{-&#LqEU6&u~aqMBbcOopohcu?n$Ns}e8!_lJ z!pGwm?x!+q-}gv=_g3zB+<0#EkaF&E34QTYhbfWQy8a7f5w}a;aCV#iYbtys`lAWr zaqC|LZq)U~Q$&-aPq#BG+?q{@vFzRJ+10kcNA?Nk@c7p7nf-GGDY+B=vWL16I&F$y zWbFIDz{hjmx!3;+7%a_|dUvZ$_3(9+8G)M$d|2db(9uY;bBiQ6=Ul0l^Ui;C{QQw- z*;_(mVE+vxi6U1-ZB`S|*+SwCHdr3E8Gb~7+OQ!Y6xmrqzE`ZJaR~>po|a*Tgd5FL zHIiE>5Y*kP|5a+*sepQ1#`liT>Q)pDU&s@An%1tI)LUszIcmO+iamqtU*nIq6J8() z_NKvO2?URY749tf+0W?&0h-l28&@fR8@(mpTUGqCh8XD+E(cBXR2Tnsj1 zt#9wB!pcnA?^1aqmCjE_6e}%C@hSy!$@PGQZAa5!LkZPXr_G~C!=%_U{UVTZN^>xL zUHJ2V94|5%bltWQ+fRNX5T24iQ(B<1Qm4$>9-Jctz=S{|wBq3ztj0XWz1(-F+`uA; za=)4eneTe9X3P1-ma{XHbPGFKDJ7%4cy-e4t8U8ix^aidI1ixUh7&Vq6=BAafuH_j zxGwMwHA)$B$*&IDJdJoOs9d3%#YaSVLWyooP1X+3v_WQ@=DAeNIgB#_xEIuuy04s7&|2>`RO zn?F?AtSBV9Wq?gSA!gd@$e{BH4us_l^a7=c5F&+=*yCEHU$8+Obt3`|ZH(!N>NlIK zY5<;;cP)pV!~rA&Vi9oC>JI96cDB3wiokIjsp7XH!deDNo+xfk1<+E@^OnW*FF!VW zSZ6eyBc{~&peqNiRrf;OsI}vCu)&Al@;mmlEZHS=UFHZ!2o!vM19HqiyPrYB6`dT* zt8YRsVROgZ}6+=BcI!SlZb3;K_5y3?my)x?+jy3%OKWU|o? zKav4T;-^yunFzk3z*%yaB=V!s0&$y-*L82j)v>e(>=PRlWlY;#p7-+k6MQ8j%`}MsVHSQ!?64`)rBkk#{`KCUo5R z7TWxUN$&GBeomS@4Y+~S{}cre?oPn7P&XBJ>iBbQ*`4Wlj76R=opSWUm2wH2NKy1O z?)zl=!-F;uz3HX7tu1%_9l4Sf$uxvVI+#YLiikTRF5tle?1+PLF@8lXq^2%K#t#jdLZ$EK~p-E-$_s%q3pY|`6cDsHH zPRdZSoT;+|GQ!i|Nu=CrZ%aRF4gUiZc)HspTlmf7BjTgt!zkjeL5#qAR*kC_S=>{G zEyTll7^t}>j&r{PH%Ir`4eF`HFn=CtJ%imW)cbXd&wesj3=e`TZ$2l}#v|GvBL(rp z&O0Y5lGuBkQ@9rIk_t3qZY*I+=pQ^fyF@_&_NFx1P1W9HUh5VR_=!|33N6Dvk)+&} z_HF|Sy9Ea{t)Ba)S?)`|dw-5EY(QpDpwxpNlpm_`-VdvO$2)7@^rOzB)R6f<)3yP zfJZ_&5nqvLuF3TonTtJGTQ48256$7NWUVj8wPsyEyQC8ARw=lraHq2)9)PF-E>d0V zO@h{0JROj;W0}WZ_VYDZPCils_}<0d)1G_7Ne$lT(JQ891bDQPW;sB4LTXT3_Y-PH z@a|uC4xl0i)NlJn35N}5$a1E$Jqo2?W>INOdVvSB%{9@_|ERZw#D=9%SJ=%+AA;3F zBHzoCA!o53M^%xor}O7CKh4-|XKL^8J({ieB7GkvtmyZ_EEQqI$gWd!k4PYcl{0EF7S6Ub5^2v^j4Y>O2YD!chw}@bxZ8Rl_FJ z&1G7N2~vZi!qz!ph&$23_7Cg}hv=Zp7%$sLgz>~_uWCfW@B&9rr+TYBSS0s0l;H^9 za_IY=RJU>M!CG}2%ZWWBx3tGDULSkOSS^#jwVAShQFa8jZN7ND&b$?qCo4TwiXgM# z;?$|f-8Sah6Du%{>{dh*w-1BiZ1|utSg3go76$_;>}lp@*v>z<8Tv_1)NM|!JjDik z-MIkvVvK0)P0FqNbP}N+#alogn&i1}$J{Mm9Elo$qkdP}l1F zZ={FX4ZA&FS`}06c6Rqg8o`BJoMc1E^wIcOVyt~EVejJPuS`niDLc5n99F-R#xH)y z$q}Yc_bVupTW!jdIRPi9x|U+;_CK}$CEgU0PIp6eRMYk8Q|GsIf=>TIXrziwdia?{ zGG4IcWyU*4=mx0_{R14p#Z35p@-vCyen*ItMmz=6n0Q#o@lU3px$&431Uqs-+uk3u z%M`}G%}Tl1n{Q~#S9l*cix;w{8;24DEV zrLVolC$m7Xkm>!VPT6}Vzm8ux0(`p$HEGDU7F?3SrXc%nqx(fiiF&Yj(V zOrK;(pKe*Uu6`f-QUDa4zusz*!XKhEY|STUTLSsiVEGjOmal!f8P=87=Yl4D@4myl zPd4yQdA7u5P%kQ#Td`}zW!bhwqHGtO9-dO%HBY&>uMk+5AYomF<| z21TFVqpSGOc}O+M_^$_(1|UsXT7 z!dWt+ZhtYAbFLd~1E@=9F2JBT{43|mW_rXpCegxs0+!SD)F+)c5flQ8I$73ckW#DS z?(y6FYerYKx06lk2*#K^nK?dRPZLtyzhN8J0Q8j0T5gSAJr)1!<|vhG$6j(6C~640 z-lT!yqmEzSx$UHEz2wo8i&-Wvm>EY*Ye{mmHWNeNmu$gT9M+M$rN$%DS#{X_tGQq@ zsamnh9m|C2oxmA!8$GVl-DaGX4l10yONX)AtRru!17M=&Y9Gu+Fy1<`$_K!xGx|M( z26;huVWO}6mV+-ZbUPWnZ_l{;pkAe5H^Ex?D%@uH$l;(PktX^H?-sc4kgz+kLrX`+ zEBY)Fjhu4B#boGj7Fj)rhHMAW;Qc*~Lsi|pBe_SDL6u0`oNQ)T_@I~}PQvTtpI><6 z4*ux{0CGjTe?u*0(A5c+jlb|DQz-@BI0j$i`X8x(-peSM0A-3{osI8gMF2-HNAH4v z#d7-dn=ScOQbr#NeXfmNmX9Z5`}uLm_}(h>aPNyjoR*JWamDPDn*Bxtzru|=%S-On z>L?sAbqP`~9QH=7>i1$|2&;HISlSGz54t7L1$Py+WwI|<+!Dx$QzJfz$$q3(!^IF| zWQZOA2n1mbye#-c+4lzf@+H-hh!SHP+!@=4^PFLhai8Rgnu%W-(Tdi0*XxhXqK8E| z$(MIxuBbqZ3JqV4Byh{y$fn-WzM>dX$Y-~B)Wg~OAyfK%S(`fLPQQU9wRo^*%Lhi9 zti`~Ht)X7mI|FkmSifZV+DWG#Ev&7r<*V%@&gf^uwylP^?JiyQ(hKer{TT&Ke8liM zn{VbRqzSPk2|11TzW$M9l|g*>56s&;YkMw`_L&m)iwCc{K)#*5k^y%4xPN>Wa+R?tXZNq=_yq5&9ZEdkuck_KuXeFg znO`$9gr1aW5ub5(tzfKNFiENXwD)W7p*EIaFEAT+t$!qvEGq{(cAW||U>PzNV$CL!ok(`o4V55qgKb>A5 zBcat+tT0Z@P;zu!!14Kw!(WjICv=K_c45(As@E1lB(Y(OTH)_>@5+t6N1YpG;YPG0@ zo#za8_cb#L6ef$};^iRujF)eZRdqzs;*8|Tw|>~kw@z4C(0k7fnX7gXF?6S><5NtI zCwNtA?B<~GEdw1H_q0~ldNgPm-opNUa*yoazY%$90hJ#aC5XIIr}(v41boP!?XK2 zz4&rGZ}6@KGZS5KC0Fr_uMHr_6u&kAC$^a=3eHnKef#O!|LS=vxwNqEaAafPfy-!N zA$#I;ujhNt7kQ_=Sp&a+`5s!$*e-@8<&qsYwaS2(fqOd>Zia-4b7? zFd=DFk7SwAP9`QeyH{gLVXZXx%U_5W?v;<@C!@pfQkKpo`7>^yl`zf8MMxB_I^AGS zBxv*y0=Lw7fXVz2yoST8c^OJKPDsOR9dwv-XV+2V9Yu$OTk7j}9PkRP$!)wNigj-J zi`RLC2!!@LBLnRn`H0Sk*j0q|IEhT_uF13c2t)p)X64Kc!m6-()7OHg-yW5` z`beJK?dOSk?hzk!$=q4CKrM}CbxU>EV>@SfLIqS)E|>+Ky!yH()*@0LD1z=CDTR2*j$IgYj`Av-uhndpz=F3O&hba{wIj3z z{L4wf;(4&@s~gaP>AMb*URc?Sy)Q0DPeH>gH7_uFTD^!xx}0=VU^PxPgSN4dA|Alk z_0y#adzD6&Rb;PsZ(YkQG(*KI0q8V?w#@FYxBOn0&YyAub=FNcYk57R;t$H}Y&V80 z@d=tLx3P&JxMPlXLYHx6)|FTpLd6an(VkW-kQ}?R7f)ed0RH90KcMdTCK!cqgXpI( zVd$9{1e<{B2I%c*`l9!(@^V&`i)|jAO7E&+FkERMRgko}am^YMbVcHKoU+jL`J5WH z*c`ZYqp%e#AELB%NO=k%>1K@S`+xg_^~j|xO2ak=z1r5(3G^L{Vmbx{gO*}P1Kss0 zkhK8d>=c-FpRrS_+`8$xP)n^KviZt#(x1PeYegl^Evs!+G1ZZT<5gUhSNlJvM44|V znbwQgr*`Vs7l~E)c6Zlo8*rj1ws~&=W9a86u!m2&yU-3IoG)QLMKDd9D7WT; zG_Zoh-k(T=yGy+3zpXkoU)(zZ<>;2@nNO!aWbc2WL*VA*TNDAxtWtSSAcdnRzL?CY zo6*8Zf%BsRuHDS1jA}e5^=~XLE8@gvgjOPXM&DL&AZEKn5% z9DZ9#!L*9t-J;Kt^e%7U8@H29T|d?Wp@aKu&OkYV1+~hB)z=kkJudsPaPm@uiIx=q z@;b{yujI|dF>K1XPPgTdbYDK#c4~u@DOS4t@IGyB$O!I8HAW^H@&Rj=bc|qTje{rv zpR49&AM!_1C5jr(5&O#Z8#%Ln4>0l*zF4jZZ5?<9ZC6~%F|I2lo-p*GJK0{~($URN zfAa{_1(Y;g3bD6LB}op5;`q6(B@JfF)7I0USToTX4)0QMNdY6w*#lz2w>qL}@#h@N zV}RlJlodk9i^pDzB( zc-{Xy>-Jx!_z2J>l$q&aa6PdIp6reC9-HDu8b7!3sjZQh1-)L}6#++wsMEzj;%3k` zj_b6=^YmcJk+nEsydn>CB1<0M(^neTjc{sxvN2VM(+lS})VsO3Mp|h^%lTAcKjuXp zOQyJQu67Z&-*nP9Y!aW-o#1W6l92Qr(VAHTDY5@A;_;uLo0T;(b?TYhk(%fBZuKa^ z7^q%fS#K{Iy|L3=V+3P{*!Hz4Zd)pG8sg9zKfV9=nD6t~g5p82x=lChxI%ca$qaAi z8W13y*@D@MU0Yk-9)mc80Ku+r_J+(}cFB!sl~kx=s*s zjxJ#^W8nNKSF^hiuo4W~E0PApsOCtc(d)4Y?dG*2aD&d`^9skep~*^`4S7?DPGSU$?WY{<%R{jwUc;*y&cI5biS*agc);bT;nqSH_%jZ znSoBKY)xc&rGG#2wS23b5lh%#|1jAVQYhu}Z8q?D29j}aG=Z2V#S!~vMO&JsT3wGN zY%I{-u}#~+%@#7~mcts26bGrk_~O-xSlW|2BmQk7da9dI{PX@pV1Oc*qA;O-%@TI* z2jQG-IY6<_z{htzk~WMhi;WxT^ygyyk1fHIGKlLMJQ`_E=U5l zjaROyrCu^aoqkcjNk7dDm6d-a`_F?^VMki=J+|vFrJumLS`749T2T~JFJI!bo4qY_ zDtb3$$(+{C$+vu5fT~lomeqm&i>X$0y|a zIl@{aPPR`1vXrt|Pe7Z`hy{A*dOL^V!C6q z%Br#N8zUBwQ~c{-1f$Ej*m2dYY20tIftHF-5xDDGqo9}K#uIBifyt1o22h@oUbLn6 zL*jK>V&|vCNFLW|<+s+dsi%%z!$t^+OK4uEeDtSLuAbMI5N(<_gkj|x9}$BX34rnv z@N`)6{x}F6EC%|$Cpv>Q#yX)QY|r75)uqhwFS zfdN-VrMI91R|wz==VWV4SM)5-{9?TyeG^AcETDQGlEarf>Cd&?aM+$Q?n%REo0$K& zm^Y;DT`J>*Ipq2P%snwr`z~{m+2359P@j@2irl=;Jrk{AX}u)ec3hvA8>0#Fy`m2j zwy5o+biGYh z?i^s+qSs5>gagrN-{17_WZ!pC|Fnm^iE20(JTgB=@56}8oie|&71!M7tlemFUIoCW z7d~iv_Jn;x3Vceiqfy%Q#(iY&Nib6Fgl_OEa#SEzmHRQ@3PF_htxq)fEyljs!1nT| z_?nFn-%p?><|%lu?z~fa(?~=#;rVXx4{^2Nqh())o<12CEe`U??sPNl$1vM3E^WvnCeHlLBCvJxCmGT7GZt_<-`v`m`*9~EkHYMI8=$yH1r~ee!Jii z0Z{lgA$`Wy9k2cjNq(*}C&~MWS!l2z21B$oIBMrv3wY~~kux3g9XxlnxX5RYJMpy? zVI7n^bTie>+6W|XJ&snSHW%p`-!C3W?3duI&dmP-nafo48 z)`zFxnGStPm7&k1*q)RIv?W1#oid}o6ar%f<1HD@o#LjbwtW(atxah8-kV{u>MyQ^ zu<5Ro+U*imVgJUf{<^`R>iG4u%T_?EqiDToO#nJWzm9InO;*fHx%=Zh zqBUFkN6i)-JJm`xypT`(nRA=${;f=;sUiPT?cCBur#GIjg=5dpkLwet`{2m6s{>y8 za#B^Ghs}FR?VM-ADuT|t(nV}4&+;Ec&zf|$>nHR_lM5URL|Q$N#Fi z3XktGz8c>ZI6ip3F`CEo&Z3%)n`&{g4GX(IOXmSe@KUBF}IMo+GC<{~&aANFC1 zSaY@RfqOOflc2G6dQFF}@;I;~xX=pr(+P5xUM}!X=mnBc^oX`Gc>9APn+h3Vq$ri^ z*6@^=&C_+VpJ=<8TEgPCv>YqK-#KuvyCCn&yynvNaZHJE_t+)2ilj$BS;O;f<_+Wk z7;>4bdiK)uC@;s?F#KHZlt_yVS()Sr5Uy}-a%0}%%$P8;ESQ_ff3Ds3Bp2efXC;UV zdL8n8sQm~ZIX4sMQrK>S`FUSrvY8<2)9Vlou5V&;O+cQIj$0*@(ZWTZYd6Er%*!^} z$P?yom>%CNVof*b+AQ?QI9z{&|Y^`%h%fvm9g^8IjdVE{*vO#CVxeTecVt7j}bBf zTDe814dy5^)ZBoEHC|dnEHfYeoMikMyv~&mgwp%r(!W0tKR=4#`UctxrE4;g94C(>L78)Kf}wAg#0A2A7i)|dA{k2LwpwLBg%2wE_PqKPYMy>SUth;J3)+2N zar!EOVj1!H3OYA79iDp+Q}Rl#s>_GqPL|8f<;Q$bM!CibiHBBk?B5}bX>S5v{tQ4L zZ3Ak7cHb~_91q)I7f~?s_2-lFIjq`X?E@WpRpN*Db+_dOIljqH+(!}=UB{j3nrM=$ z<;dVp2H32RxpUG?PehOMk7Aakmd6E;r-9nj?Tr2pZj1ibIBlE_HM$ z4WG!tB9@drkdxMtNPeD=$H(OkXa@ zSxP4h=KWqaq=_?IzAIaeS*YWbB+CXhm*p~OZjQCDu4v7#mFGLKaH zCG=uHo76g|U&}A|eD=kq0|}@^&OH-fSbZ7fNJU$udtrM}+eN{nrLm_~;^184V=hZO zs(u$y(A5{CIdc&Moy|$e-I?`GU{)_ub4B+JTS?Le3JH72nDza~LEQp|0`_X^uEUB6 zaAy}b=?8k7QAL+qfo=`J>`Ps|3n--ro*9E6pv0|@Vv1kbq-ey<(9IHgBX@{k%^&`x zLbgmFB6&N7DYMYSoUqc8g7Gn{9~j>cD(;0XorXdehJCu(L#pfdTkaGo38|~-ef=cn z@ib^#a+{=>F2q9Oqn+KS!C#~-WKP8zNt2!rie3pB#z#@)x|Ax~8!d1b``QeQXGw7N zp-Na)({|cvD2BIwZjvp>)IYH1u7HyNNtaPeRXTV|l7^=oP3mn$0jecdYRlrK)IZ`3 zzj>82i0Q)LVL4K$0a&;OyC6aeA2VcCENE3)T%-mupq-$(=S8JvugP>bReJb1#-=1{ ze*YW+$}o*rvZF3IVem&WN&|-0A)#-i%!~{-k!LGFxd*ww0hul^sDD4;0V5*|MQ$S1 z$0CudDHD_GII$J#0iqKYBO`p)#(^`X+sevhS!x7vd-j|S3xYg8{i#zmO<8*lJFGri z{4VbOz5N)Ey`A3P&GWpJw1!R!*25}m*s3G;(2ZH6WiT@#_?OFi8_GrRG=6mrw%isr zh{?^&7@Nz=o(&`viNJ5+TADlzPQfx$zg zs8*NPKWcL|=R@!Z20yaw=_-tV?3*)0102R>JTsSi$rS*1qW88G&Q+;&rfMpR0oLLW zuMU?Fez(RX>T_FpbTdjqb6DFh=l2uEbb=9!-hk_PxQ`ApO{$8b zc=%3O*wf9V30GiU(4wW6n&%U-1#ZTKpvziwZRNJnTA}vAO775k3x>Tn&CbpFCLTs} z^-t4dNL;>(7k|A7oPUov91+wZppOYUnlTi%ebQ8g zMwh>za=!j8AI$l-0|Py#wG3U1FadW;Z&=I^iQ8Dq!};iNJo6K6a4nk%O>2$O(uqQD zG*zLSrHqC9>WO}w*@iVnVi?_bKP!_uXBa9IsLRb?mYGO>v%UB^jNw~kugyCTn}1s< zECjA3@EZ*G2pNEdPIC8HM})_I zX7wCiQY2SNf`$kNxN@bt2XcLPA+sqjs7=dlle#`mGxW72h3)##ef7DlpuTo_H`T3r zPi?@}al-=F3Me=4=R503=6$g!vFl1$l}7z?`&d3jXwKGOeES(1+&m3#w;Omea6IR$ z4eCMr3XKSO{&1}nop&|9fNh^QjDY3jLBI#S0D={lc^Nw|XNfo9Xv zg{nf9RonGm_;2UlCJB*iBo81PPu%WwjVp>d{&aZHce?{_{w*9 zdaOWR`H5mO0jOJovNw76He5q(k>h`)6|9_Vao>BV|z|JVwnZueDplsP#e}JM$C2 z=Nqt6M-Ywlhm)#a<%qipQE+LSTrzk^_onbK2qg0QWN)VMC3_HW$C*=OE+*imkSjW& z-|$nbSLE?0N2EW;=_@lx?QcIPpTVVL6_Fm#*+IvcC;~0HiT)|7=_?MT(;`nNEK&UH zm>k!*z@A{Qf-C z{y^D$3HsPMBi*le>J1YvR;wAFdWmbO-lRc7sCCb2;;EKcKhiW<0%sYVq1$U8~PN0-wQG;GSXaejpwxrs+d* z-BLJPagdMqJXW9Xi%=3U-n$dxMutKM;4ePZJE?!yrJ_{p`ud!FLd0eY)CEtWI(^St z^gZVBijY&CVw&7PF3%>q<9nPa+jy21tL&D_2SS6Y4V0;P%n4?_XE;g$Kb)pe20wo| zeWSUU4MgbO)BfoEy90|a=gDgV*YDI!x?!PC;>5DD+_H83#r$O{pM`sfjfzUvM znmyII%hN>!97X9N@E*r}AcfFl!HH6>M7gzh-X;>>Md%6m3LF>md~?c_1_Q<|T^gjq zZQDKsn$`3VEqs%X+IrG4q36QU-#dUOw$@iQAXO?DZTJfaIbjYOrE&Lk{HgahT zu2w{Q+Y^rN*1Wt>G=&8q(k}QRar*e6pq{2eG?(!G<)(!rGRZJi?bQFII$5*77%fN?x0IEEZF-{5H)E3(n7 zIYrJa&Xx}ZANLCqe`PnkWk^Jaecitr84&2JKukEQeh!Z>c-elJJVSQqyFgRjJ*3s# z-H`WUWN+f0M#JaPq_u9_*fvhjE@oZ&gFE4#x5ig;#^y@vsclzDr!Qy;rx`vF+f0h``(hF9`jg#il+r$OFj9 zrcti8i$~32c8XEQ$tG3kvUxetv%mo_8`v8CCtSwBsf{bHBDCrk1*((@1K&G14Vn_G zto0r9Un^lpz?!Z@i#QI0ZHWm!idht9`Lcm+e0|o-<87@sV~X3mc_kV z>TqH;$Ng|bab*!)#`Ag&G9AaetZ5*x7*s3n9e_#6&7hOW02A-@kE0pSpfY7NF`>pt zvX5zJWN@Gp#nA)QpPJkKP4Dq0;)F5hL`$dV7z90ryx($%FrHO2pzgrnVx@Y)7>N=* z7+q0p31d#XU4p={)<^jc&rgODT$oUaphs1+(UhIuS7%@>^C6v3%XJS4+=V|EUX5k| z*a$E?E@0g97Dd7W8CBpbScUeJz-@@scg(jHhfH~QN>7EZyh2bT-?^}Ouj1F4;Tl6= z)-*iIg#u=Ot~-J`D3Py$Fww8sYCqam4cC!$UTrve+(woJBfZbX$@zoD8fWtMdKNQd z_N3sUp~zX&OW?SqhrGK^FK6DiUk)DA?SOAo(x2LZ@76A-EZp=&fJ+pQyS+aT<}-yf z>7AQvU z5^q1ZF9fwkMBrPi`GQWAIAG(ueYu6D{7i})7^IwI*w)$&xB z5}^)_9#YeD$*E(_PMz+!znSUIRp{07)mW%_+-)6aimLAbAoCBdX(n)d_O$u0U1e^i zs*^(&%1j;C>8fc(8tdP1SFbI!;W@oHv!$sH%$ll=&ChRKZ3c0fm_b^~* zBc?CEyc@22onGKWG+@ zFz*X2X{4clf7Hc%(|-EitaDG_noy|Up_EdVbNnK6j>a0GV6GUISzt`@|z!sAQj z5wwd0iAL3P`Pc$R_ZK6xGH4At)7`9b@F_WQLkOtvgOb>e3dFbH3GC|6&u^K^IL$Y7 z>^~%zMsIDMEEX2G3QF*{#!4u`2Zg8x5itK0yxTC;q0Fao%pa477^e=4 zQpN4@<_v)F`6rpu1~M*#hoA*dabBInL3+A~m5P7)$t{QxZz9Mrrqt@x*>lWOhTi6C zJdik8A%N4+-r=Ip~+$xFv#k;83MrI1cG%J zk*1^q@u{C?0iY|CXa*+7E~V-w5eg^v*|;mPlR0{yZQ(a6<&&qAleql9fBB=sU{2J^F+@{` zgWSUpU+||&3x6IYfTCdK#0f=LY$5Q;HFM0TxGzp?h-aOKl_7-~^=FWjN}y#H+;YU| zc#i|B_hv;)e(hie49{SJ*o2PAeouX=Jk%NrGX+DS{ms|jbGtt6-|{4N%xL-}W3?%; z^|2My^vWy|dHF8pS)}v9vEtk4*tVCK*WJzv*~$g{Fsl}U#4TK0r zf9yS*%$841HkUbDfHGj1{j!BfOIB}z%RtuUjFgLAncbK*A|>-cC;+ioNWGbP5Uc)v zDX^6oI{PQqJ^V5Onk;u~0zeCF$xK?n$4!jQ<)#5~9J(Ml2WQKal1*QKy6!SecNSJr ziV>RqaSP|!A+rmK=f-E5-olsR7G1+x8Lx{PxXnY86`u@(0&)1^uj6!v@*CMky_bL^fd0l+ELvr4MihCzCkzVYqOD+R**S1;8fbhW z5pQYa*g@=T$j3lx9*f1^n)~2qK0M~l( zqfUAh{b3K9`kv$`UNX&&$H8v;-IXsJkVEbA-l}NEd^BGjich@I#9L)%Ux1}XvoxO- z>eF_|ap0^k5&U{0e};{c47-T@y0`m9fR>{=5KLa!U(Y;T=nULyZ8MX6 z?jVmojyJ<_hkGuJ^i`;zDh?!Tp>?Hnu5yH)vL9B%KPTUJ8o8TCZG=weuJTLxmW87~0|t;p2H2)psT_ zjwwevOy739G+ud7_3RY5bAmyvM1%V3>hYSYF>i3qS=UbFEf9*MNjCF$u)bjqfY7U>csq;nHniA}3C(p}Qs zwc}en&+qrX&-b1)PK@!MF}`!opL?#g=9+QOwdcI=>%K0+p=TG|b#}AZ&rp{Zq3Ud+ z`HP3674}u%di+(dR|xnz^ez=2>dmS8e1Q)01w{T-EZnL_1IXL-vy%Sj1S?xVpfs;u zC^b?2-VcgL?wB0jIK$JGl~Z(a-S)iogX~%A+o<;5s)6P=QC(_R<_g{n1&dk7ZVMnJ zf;{C!Khucu21NVPP8Y&=c^E=6VAEi*vdE>th#WOt%ex{lBdYAyX2ki~o5UH0sX#&G zZ2Tl9vCaRDoIP8G?zdjss0ZCorqBa3%5`>KfNwguf!Vqu-&~is;q6`gnn*gl1dB)4 zV)pxwaNka2UI~alkAmugd73f@YmFTW5>rK)%W%H#i1{U+J$sW7 zk2#y_=g^&%fiBJG- zk1p{{#OT#Y`LgiL#p6}3ii{pAEZ>n{6`KT^G;+Wd4ZF%|B8BcPcSQ4c$yBebyZupw zd1jW39?nF)xgqMyob$U{M+k_=pv?_;4uE}hC&uM_(#Y_v#;pzDP|yj;9gA2WuDh-7|8d=^&$CtVwJU z&Uc#14Z3=Y-h~c*s(VxQA$kJ-M{aLOoTaE?Y)s)}=_6wKlqY(3C0c0}t!R#&QDGyp zx+I9Prd2%lrRN4r{&DUNkl{yY>?rZBq^dy7Yw@4jWH_ZdE6)*%?AdpkGJ|{rmrWp! zb2SP|Kbk#C2Q3Ln;?V5q3p~RiWXMre2}RtIEX$ghuX4w;#xYRgpSp>fGx)coA%w-f z+RN8;y{~UNXLahmyJw7hq2_fj?vRawMHcFM4qvKiz3anfEujx4`g*L%7Bzr)dx-CY z0F*2Ywb8uH>l|7?cYuREQtGKaTJL59*YW}+&x}5Ycz_$)nX@Dky&aYhasXA#;r^dj z4#PI;7&z7_G28nOxHQO2o=q;zs7S11*$ECFilahT2TUii4Kt&#X>8YNDz-Xt;0Rt_ z@uBAC<00<(iebVq!NT75*@I@xOFT+q<3CG&XY#o+un0zfT-8pImUpZDU&dkJXKU~W zNcC0%aF8SC0mco}ox1QT?<|#CfF#Hpb5cEu!9X`aNi77DgV-R?YS$XHZlu_|C5Z~Y zg*Yt)TsS7C^1`)3LCTGtCmW&UYHeveqD{GP5F5+1RvtU!-RU|_`S$d%HY~(e#1Z3E zfGy>Cu1wC9Z_lN3*8qn0L&qR!#rG9oP2odXg3M9Ea#!mH3sY?GEgKkzOu4D3yGo`= zP;1p`x5(@|Eu=1-j>i>sIy#gl&gA3mcN!91Xc1%A!lQ2yO<818Nv58py~B&eT5jMaenzMK9kZEOvf; z6f*FfC_wPT#h9xQi!09rDN%5Sy`aS5FXP#-E6(y1Baq)n8&`8ob9a&pf=WNnXO-h# zVwGKv+6UE_FeK6I3OHlO}P$0-t3PH26&565zB-_L*S)^37P^U zX}cHL+ZVRbm@`cU-J)@@Hp`jXrJYoN4!q65_2`lu-Jusn4g8F2PJ-}UcMhE_!-yFi z*d*Eehb=FR3p5T5;qOo!X4l&V6co{@hzBl$xYZ# zR0AO&V3V}hOG|w6CdxDj=uH(3N{`;z*D6*|cv*$eP5^7Kc}?&?H`oLz7uE{sIO8wR zo6)9S{qXh~k&%do+r9L0gdAUS3MtoIC`)m(GEg_Z6j7y&qxot#sNwu*s}1(J)TI{O z&p&`Dl5Y!tBCYQ7TGL{!>) z_GvWGg|3kq8yrTkX&{s&*dp#3Tbih2hTdZ7{%1dI_RsFF!&H;tQa!*KzQgaa^}D-# z*#mhtiN&7d>Z)EMTet)$cnaGmEvu%=Ag|9tUVsN`L2QUelb!IiS3!I~+RLh%$;L{x z`pR!cYgQv+PHNLjl~M=z-#vF|#}@KE!RzTZ*b0n3mVrW3!2n1rZ*MS{w$krxD1s3` z0C}bZ#nvrUq6ndFn#8?^PnulSlyfm;=Fzf~M_{VAmlS2uK0W$Fe*H$mC1z%fH~vW= z_i6xbgo`-q=0v?gAYUTshu2)d6tHmA**@BOPG2o(KfSzLb_cZ4DKgVa2wvg__+NM{ z!y4HurOvZ=4#3q&SjnX;VLMfARt=nbSFi1H(mim8Bu~okP`Q)r!w&-7oVCi; zNhd$P3#qx}5l7aIhTy}T=2-xyP9Wp=?i@YqlZcub=>F$+9(;O*2X>@nFSt;_^egpgqSKMeRyojJ2c+Fi32#Z`q102>vB zq2URCtvf=w?+#nFcW^WZ^d0NY7XaPhpc+JXT@8@3u<||LIsFW#l*-Fs7-qjQB59(y z7eB#J7vUyLpPaKl8Z}i!@9{@NCW4qD7=MlHp4zjcdzDlZ;Cj^4o7^zaNyq zKw&%t00Ams(@acE{f&f)-F%Pp;`ls~*`IKTI7AT@8~5l@%TW(qNR!L^-hxh9aT)qA zJmR?Y0C#$%wbAnkciiPPv+cHaTtjFj;9zP8Kx%Yz?k58nPX43-X;;4NCAZbcL2LkQ z!AJ|bzYZ`F5d}_M7M$&ZhgHcmE1NSTFJi^!sOdc~&Du#|gO7|(R^z1^HGArxmZ0qb zS#%6y2PIm+NIWW9=DZZCgXYesYIBkq#H4;W?rcX?R{#7+X%s0n{Nppi3DKC4{T&2+ zqE;<|Hqv^OTK?l3RB}vZ#q*CZ+`k_-7dfSh{MoLa7p23^Zkn8|XWsiHCu*FZndAP> z2TtP}LhJvIGKzFv1^e8Xho#=#u5snHg0Gcn*5208%ClmbGiY>krL_P5t;}YH;7N<& z3=2lvb2%C-Xam`PE!7ft_7dP~{2@2vGqMxgUaH0Q*crhsp!3Aa?#FC@hy3n8cGZ#C&R4mT`J%y6UMqJ`-EQP^}H%bmxe^&XqBxW$mO%Z#RMo{P=kQC;6(D8J9^PBSOasFf<1zK3uy? zrqyhY2c+qh-|=F7Ec%rBjy6lTUZ(Z)2+hzABQ=+YdsXVp6%2=8E~^Ih!svvRh1Pxz zfC|mxyQF?i>SsztkMq-o)W8Jr_Fvg!GXmOfub#!om+B72YH3?X+6B5W+F^fL=piD$ zykBPa(p>1x1gXc*{tJY3G5z(s zd;QI(ajpvYW^OEK!gO{$1pA{czg8ot!H$!Wh~-nTgK$WT@N;Ra-aqQ`;^BgF(dNm2 z>^U$E-3Ae)mgA-LsXc8gx~W{#v&8OH|l-q2E8RC`YGj<6^@(;1cO^$Ug6vU#42NQelt=N zefi0u=3TdeLKG3o$6u53w$*n`EEm;mWP3ex?>?(mO65sJq6(JM+Sn9Q`~6v2bo-J< zQsigRr*@;cBD{!rJ|vjZiC8AOX{0Em;@mFC{+rP@d)zAy#{0>B$Wiui&7X*K#0?&( zbARs2^+{09PPW^=9vDzUU)iX(Vkz`AH}1R5o?Py@-e@BdTf9eQ1Wp*ZFEmSuG@0^N z_U~+z@M;bCi>z%6f`W4u7pIA^MNPo3frseuF%X=o&@%0KC&=crb z9DGXNirVICOZW3vd|-x1^#lAuumh%xj;yN_vqu5eH`q>XZ}G)c&CoDR{{BoL#arn~ z{&6gH_g7Rq`>s_0`C;mM` z(GO!tR6kPx3P{IK!pJ+M8R8D+e1 z{mi(lA>^#`wI*NhS_MAgPMjLS*G_v?3j6u$yifBZ;r*~7Z)?)fqjo$pmTA7XL4wa8 zJX=i%UXH{5D#jA7jMaNb<1)sEQihr->!H9Wq#(r;%$LhTpY`sU`GbSIZY#KW305y8*I0q+C@}qxDI_jz1XB%FwbF;{ zDB8cB{!XMU?Hp?0{Eg^*p(*lps-Jd%B9=Xkuax)xw9Rtr5$uK+K505=SvY{>d)kaM znKRp&q5iF85j_M#KP9B*zvE?V7)DU56UYUe55VEJLD6u6%GfYG72dTU)32UqMiRC& zL`1B>(}k?t#$j8(0JmI*J)%BGcwr*R?;xrHPP!O?qhsuW<$f?jG$?T}rz@JZK>hMQ zTX;O@U$#b!K|sY5iUWqmYSryiGnR6+fA-js^`Z|H!LeCa=$?5qm=y{bhWM>vI53KX z9|Ea1Au@&zsy@iXFm(MF=vqu0piXZ1ZGL=-c>c@*UH1gB zcj-ZrpZ=dFmz)ZyXbKMe!ro9RR7siTgmHy5 z`*wo{VyG>hv1zgxL+#nwgQX0YUC#2gy$w)X>S!3OZ@H61nh-`ww!~fs?jlu4GZOLo zp+e}Xc{OT$>-u!`eIDKQ)*}s5sGwbwtZym$>E4nZNhZrtI18cK3;QJ9y4No}XSgj^ z!#&clan6;I77M4r@eOZaeyn?UG6P`EKj*@`7>|)8lZ6nTJB0z4Yr7X zhRx`=;*vY>lt?@@DSE257k&nVf8H6QHrofH=>8L1WP4K-O9{>v&o$C; z;Ze11QR8~qM7d)0mT^@LbwutYevsw%MW(Z5i6&z?1?=4_>qqRwAJ`O;b~Dt4mxCT_ ztZFAx*z6Q`TUt9ts!36eu5+EYsc3;;;7LX;tJY#$++|A3HXMj>Ok=9Vv%x<4gz)=5 zz>U7z$Z^J|2DJrwY_n|gUo9#A09_O7;w@s7xX-en6j1|VC5dNQS3@iCy#w@5)6934 z9rzX*1u2)ULXwW^v6HpmXesoULn+u!;z@@`o@^e)d)6h}$8?c}#~R{uDr=*|%26s| zzU*3)9r%gk5a4$y`XnQu&f_(diY@9RjVAVTC$rwhJ5Aj%a_zK!ko=*@Cm4L*B0KXyf7F(kmfr}nH;(#eB1xv9Ftp&nhc+9R8j z>bJ`K5%jzWo^575MbX_3@_J&@fe#%=Wi$O4=@Y3O$8uh&EFCY?>dw}^xM!%$G{f+l z7~JlE&&dS6uk_(P5Yr<@^Rls4el#eH!{XMu8Q-%>^hOJrJ0-zVpNA-MrzEHt6rOwrO zP+U`#Xe}!5TeN{$-Ut6H1bX=g({c};Q(%ht+bbD_d!29Ww)cDhjb_jz6frgLV^+mb zj{9qqR&LEE*hTTKe~nmqVLAH8$X$9~eu0so$Z(O=^Nkl;Y_}tG6i?W2)fjfr1Lu}s zk8`o(9{fVI@e<=*@;cVTnELfNQZThsglHByRckm zpf=F-O3lgY^DXy1`coAAaFBS5w{)8^sC9fN?rb8#@1Y!nOm|{~B$b*ZtGQ1Kj2lW&Az5bM!bW6&IaxFdXj2Y_pBzR^}(*HO}u%YMW^`j0<&Eao1L@XUBRDukn`!~h(ecl!a;)Tm1!C$k4SrrNJ(zfaxMQ#DcW`c*m5EeSnhs-YgW z)4NlFIad|!=@mopVFDb7#b7ua0uP9nhQ2)%vbug*r$~MyQ8m!FW-HFFbPkLf2CfA# z*!OnFCk~f8muvg@BIY@@!Y8j=h+%F|4s#m^)lPWX*N^veT=`4L>1~tOr_1tzS!1J# zZF20;F@X?;JiFpMMVsCke7cLNz~GUDv4apka5d*WKygIeFnQl$?9m%$@;?Q#UY22Z zfgF+EyT<;@a)`QmoTni<%PT#?^iN*oj|d8WPU*jx+-ne;b$FB6(Yw!m!DL1_-~jp zmXGD(VrENi@wvprQ}muw%P`z)EA>`?3GS?`IQD`tC_=NRs`Z-I_acCXO*IP_0Cx!0 z*?Zvv`e)V4`?>05R9k~XLpe0!1h`&Dq061{a1c7{#u__prpr6TR_Eyn0;RT zf)|tR2(zL6ptgm#s`J49!l6KGfe=^Pp2=hduZj#WdW>UqNny^x;&+$gA%JaY3k9JA zXX@=czkARvnP$~D9JJotLhej&oIc2T<~&;aaa^TaL;vjR4+Lf!01|q!4J6iu#EFmc z1U#rbpDdd^fnBxlz-Z(oHb+(&Z9_5h$3O?0H}TiWfJzSyC9h#1v`t?|CJsa-2tWM| z|MG6UbeI8W7C3n>4mhDpgk}pKQC6Poe7pf|jsCGFm4DYKXy;p78@u)O&4bU^fV|!1 z1`X2mBQ>r?eggQMls@VHCXEIzXl%VCo>z)wb6b|P`%!*Yj^K$i4i>&Df?P_!MAIy9 zoVPaq=Qccd9A5DXPC4($H1fJasz<@{EN2`JUoG}4H+%i-GiY5PM;anPxZqd4811lw zilBs{aGbPQ0_ow zG-IyhoS_XS(W5Qjsi{7j0I(u%<62z68G-@m%`gJ!@>#;lo0zYu1g>^ zdsm++`k*G57L7(af4nO2+n+a&O?0dPvY`m(>^Zr*v)0$j~@Vz!TJJUhtop`N^gH38U8 zBmBDc`5RcYJ#?L?Imhtx$KUAg1&l`1kkD+sfYOA1xKCF^tlN(UcrZ2OjK)rXH1eJR zVa1~WvYMCU+yt&y8i^qyK=e+FK5a~(=_ z$!Z=~s@I^I^B7MsrdD!e+=bTB8x)zA|1O1R+(t2&Fu#1z$0JLgZb6NCy*R__^_kwM zhn-K#+>9BDM#3)=KS-2XPrrGe!upfHcRNexzo|+#i8i%V`FI9`RR{{t_}~J&KTU!l zPI}!|d?^s~Fl6ZlNCW4`8{poSzG73LTZ_ED`lD`ceABToUE}# zI_uoUtKx*P-><~f4;G9kxIc!1-%sEn`)~Uwd_=H5a0@=_7%LeyyE-mB?EIMkQ=QzX z#s_f~Y{|-jHNAiYw9Zs@LqfQw;->eCqB@C9kI1{Rm*y+SbFOH-^t4?>R)fhEK`q)x zT%F%>`vG)UsBBZUsO z$EGnm%drz4V+kHTq_zLcyt6*1+k3>ZoDTDacZ{k@$A*M~Uf5@K`w*ZWg?YM9;M`a+ z^vwwEn1?vRWbQHm)-4VKOwqEj64GX|xLDcxCC1r7>iF%`Jz(EH568=SH^G^5dztI!UmJLE$UUF( zV!_f+vus}-u)ZPuxh(e}lMZM-^ZuG;XTA#KGq*mG%wnbyBD5 z{t42!06HAo%l_3-uh`K|{Ikid4>@ZiGoGV1%AM)vbca@TLc5Iz0oD&cO?dW^tZhuX z0Z6gn4%as{3H3zOC0x&l3}dccgnPDabVH;Bezy(!X5lFa$=Kenl_2gI*$)y5cLXocP4pJuvArA(qe3WmCMqJj zVXM1Jb4Zu*Uivgq3rWJ86@s?{!XM9Bcvn=VOjFwR-X(Kt1y59LoeXKKe|4}EUp*D- ze@`VUnb^~1*eF}ZFG^=&Nxlt#_#+2<&?Ml>iW)TleiF1-BRDwG{O40&F3On5wCdV# znR_1pQqRMOTO7Ny%VMJ$T>&uO+B{ShlZsSD*fSGeyWc-o%r}ddewQJ4UDXv-fl2)I zVR@Nlt@I3N!2dnYWHEg#L^tggOKIDRFX_39>#;KtYxHsqohKI!b_>EW{rT~c_3+8> zKJdlNSC#&Co@A~A?Br@*Ll6{*i28aE-=+bfRYIx`g|5+^LveN!8&*@qg8(Y5*jY4j zN)kQd^;}QsXB`I}XJ09_c5nAQ_3B42|5=UOK05dTL|eeyAh5B6u^e11u5?ESVJ}{vzsHjIzai?e#GrRqH-N3r*e~NElt)|4_S24= zsPsAKCl(QGH}@3L^36vmh^s>G z#SOJqeUphsH67e1Z!^r&5+^!H)N`{=Pk#=Wq!4EP&O~KU?Hbo}75e^yv(t&yhXV62 z_GTE~9e>5hbG?i-2IA6RfM}41-iVHdig7?SwymsrZTlRiVjJPN5wv62 zjB~X`{}Y}7;zg28x3~KK#Sfeit}7huGDWJ{{eykuWz*U0DvK}IKE9@88{H;W;(V0@)tB|;I0^t{6TqX5){%6wH^o^a%pe?^?RX$ z!=^}*I%KY`VLh`%>lZ`I3Y&h5l2aL%qfGa(Nl^4Q9P#0#i|^wxC%BfDUJwV3s!37I zFuf-vKGo`4BjL>Sb1^`ml-}cvEqewD(POzNJViHnpbw%*xeRrA<;?F_{OmQkzhgqw zFx`7vCcL0Cy`HNc?qT=K=~@iaU3@)l#{YDBt$hoNi*Pl$qgwl`5JQ&n*?B|!i%ccE z>is{*OQ*pU;Uh|Bx?H2_#Hv*-F8E(5ONMw!RQYtxsH+}?*9&HEEuCJ?L*6KqDFKP^ zTq=0~st2nOasKXVx3_K5S;UsA=LhYtDMS6=F230xp{BhO65Ic!>YOVt7xTE1*raC$ z(G5UO!@rxOhj65?J>QtYb5oX(jnn_@TK>OYp?3pynaVi7xM6~4PSbX}R6r%%tUH;= z+1fr8zBXkUL~v?-e^5!Fjd$13qSUNY-a54(kXFt75gH2#%mGgXuXIOs$d||F**;P_ zlcf`zx^5S0|Y4^eU2@4n*td6GleF1KPaBT9JU01>Op(@CssVVl$L2N9v`k{S+DJd zWxg5it10`&3=`F4nZqLW(vjma6Agz#dzWLXI4`fbRFqiJbth~CbG3qboh9O9aUVoz zGU-45V8H=Z7{y36WjsA9pItnFAcANm9c~QxA&GV&dF_-*N&IJDp=CW6{o+U>;`zh6CR?3a#VwtClvjr4Q6#k;+0 z2c8+;X8sr*{7(J!LHDU+(hUG#^0*A5``1hrssCN>;EcAAj}PanYQMQtS#pJ?@;lEwzv=p?n9K!{mSEnLDQw;|`dpG)QyyyU#S35U6(8^T7IHEk z%{LwbhXa|M*7rwm|Hy%5f2=M)%c@t#v@oP|chuxlPKc$S$lL3P=zoCN^{#nSx6w$4g+?XI2xiElf8r)eR7dGeMrPLY@K3u30M zb05Vg$vBaMaKp?zYRs`fb<%E{^yEipit9O@ECO;^$U7Y`lsovJ6llE9>+k(|{ZeB2 zDQPtW65zb=trcI7I~tN%E3W2FSi71pe$~ydd&ht8)w~gvbzAP=_uj8&SRD#mDKk%x zPMSvpN!{l<-Zv(#i_ja;Zt3IuS5;`Ec>C6orI{Y0%XnIXXahQ4Dm1+STdR>Ozkd5hvzsZBuk^7}QrB5}oUF$kIhVjnT>;ua zjNH_9=H0?m+;`3rMkA?zsn9Ch*1*KWIK# z>gB2m#U`RM|_t} zB|X&9QZgJr#A+p$(n0Dp{!i@XfmD_XrMA}7!8AHY`r-AqhII{@&hHpR!)DpjHaLY{ zm>z!xVyCnoW1uO5;5le7*w;Q0@IeduK6(tb$?rI9NJ2cJL04eU{9eN$^Sdz-t(t4l z7iJA>t5@$oeZ;x@gjmvVklCGuwNx!hkLFFz16=eXAsSn?U9VwC`hq?Sg1UNxAlfN$ zo;I~6dsD9gAj`K*AV|CG;%S%*{uxNqN6uySzECE)i^tAnx5pt^Onp05`TZ<6N&B>B zES*R&o8M|K=eTqO2JOqQ+Uj|hct>2y)O~YS*kNPrK2d11%ZLvn;+L;>W{IeWnb3zC z{rh$Om6yRW{Yr}h_N$K6JRE!R3o|7q1w+3h%{N_Iehv-K$|F{P-M!qKH0i86NAbM< z&Ry9useg^uuRjlDi_J{KxbQurOS_3$d40i<+)lrYRPf&x(J#0gim#Y!r5*m@*Qcwx z@Xu0zgxCVZ`6xWmMjr_ zGT_-U_St#hkKrpVr3v52&o#Tm!x$G^0Xz$SV(RT=<`X`m*TLb03Li=u1P&bHEa=`^ zM5ZU3vv+|+rzm;jeoZPTRek&~*OC^KiK_%p;C=PLJ`Q@L7VyRL0^*5|0C8WX%OxG$ z|2ost43&s;RE~XYtsQlD4E0qtGhFCsb9ADe-*WT58$rG8LdWj0@2e)y^~az32I}~e zbNaf*#`9LEdW1nKklzK7;C3vkjC7?Iu$gy8I;Ax1rEC9g^uZoc9vhl_r|AAHTR+_s%Squ}*JOk;WGm)ri zYdFACVT98e0U?t1Tfl^)P;|l_L;aFo`-DZ=0s64z=ZdW-+ynb?e-AXn`_mCvxW3}0 zX&a7p{>R;`kO5&PX~@K%@}NE`6KryD3aw+*O5EirZ>5XvGX4}&VXa7w*Y9EvvkHDK zY3Mx?2A->70EaIs>arSX_%10Z(b?eZURhk^Cp|Hqy=`=UZ;tpfbB3a3O4dp^Ouj8n zM*SR>+JfHOya0(oV>9yXLQ8L6(d;E@m+3cHB{TOAoyy6pu)7Ay5t@;@uQOo=L-&j> zjDD{E&1Y&bERLv)$>`Hc`E;`{@AGQS4%#a0sHCP!3k)7MJ}`g*hJ0FEP4 zIx2f}uSXaUDqWGVd&x(?Q(gY%{s%eW*4;(wa)BEpjSm`d!h3y&pnTRxe{SUXF>-3Z zfs#nHvQ=tQD3SJzj;Ue9qAznG;Ydp?~tH z2W3k><`yB2+d^*^V?RB}WG|8=-RRbw{?{c73BIqKRR6bX`(NmhqFNR}gp)6vD8y?q zfG%ov#@z!va=>f!J?$*FbSZBo*~O-1lN!Tny-DMf`P*!#${=8feqV7u=Fo0G>2U=H zlDAb!F-q&Oxh6~Jm2=ac=p{g^{Y9tBxSE~!bl*6X=7ZCNOtB(Pij8jJc1tb%TwSao zx;`HPt#6nAtfW9cF4M^{jP4PTzd^X*jU=*HX)Cof`0EYt{&R3S)&&7ZrL;(UqjTS7 ztV{nxQ`^N>={eV(foip~iHV4g^??#4fWA|OzdMV~RI8Tfuc5B5dh^Mg;;A$DdUAtx z=7GFo)IjFjoolHIJjlb zUo6y(-KQa!AR$y@()97Nk`6?AQKBHa$r6z$IrE3|(}EuR@;58JQql|?%?VWP z4c`vB76n=`fH5-fKz;{sv!4~yH+sKJ90oq{XK<+^G=rEE{nM*IN4@=e`b@)T_X*qqAyIqu$T=FwA(sUX;7gG|m?GQ; zrNsm$=oP=x4~UNgcL+v9gi$7h4EU*ru6Ty-%VCFYAtaPDm!sSbBl!yLCqwW%Cm)?B z@+6*~caf{->x($ihr5=UDJiX|`{F&7VG$kzq%e_kix@3KuR8C$jHP|b@cDP$u9I;f zIYD3asI&BYFmNT-=PT{O3GEjj`8`P#pc%z~KYTOSRvPQothi~W+G7#Hg^Q7_Dh%A- zEAnI@;;Q)wC`yOiklA&T%lF)m{OuVFS$VL+un8(dXB*t^h%cLPZsHCgwdbkz@N(o) ze7?~I${;%tvoD1&1jr>JT!0+B2fS&M>ZO`?u;Ws$Wu}XTXp9bRW>944a+@P*^4L#H zq?#}Kh5Z48|Ic>tv{mYguhV?(Y#dU}%nALjT4g5RX+@RB zF3WfCsR79wy5As|rIj1wC*>zgE&}&BPX`jNe#}1de_LJihSXw0onE3Rv$I80J&k55 zY>wOXw7{y;`Cdwr4fC%f$Yh!Q8^nw4zn*4oq zomU&f8mQ)2BmK&2=(+=p_Iq#F>}fGz>Nf@_Z0HUk+cbGw^ZH8Oz24(Os*^+Kway>q)*vmZcq!m2g^!>HIM7y4AZ$yg182_E@~gHNX{8F7m|jAFD%xgpD^dWPBa}Z-J3^w+FsT0Yc?DXce5s^h~(TAdl#Ju{p`oc_8MKL zI2HjEUFm#TFZPw;z&(m5X^8C-+?>^${I=ebM>%z@z)Rn4OH*HdRgPf0i}z@mTA%zg z6!imM_yjz*#*P}7q*dp!e`XKSyUF(qMZ0@d1e>dP5AMUgV`^`zhEG9>MarT=3#hC^ zc0j$3Y?4L_ckt;6_mmeLnX8&DoojjYMh~fQ^WkaTZ63AkWaYf+6jkRE5c3bj2Ez9{ zkcLWhHVA@Y=syz1?>50rjkq!`)@bo%eKFr|zu1f-RPvjR!dloCFtycgl^b?ydOrQ54|~qvq?A8O60BM1%(bLb=@oES|n{ zg2@{ENR8MpXdlz|EKkyB(LW{YMz1o~@ z;zIzu7Ed&qL$Bse0V?ot?VDXp3NGzX0ADiOlQ52^97h9$;zVfQT|DozdYxRW=oGf_ z1?e@sX(Ux_8|&QZ>Pj?UFhEm9Rk`>3vA=>?>~L$FGjqTQ;??;lZ&`Wm_!tM!7+1%%YFHkR#sX{R@_m51IJeFHqpwFfL4QR{aO9yYSGZG5Wh zr=Jz~%JUB6;4PQzPP=L(Z9Dw)<5EFNze(I$H+8)?vWNwU2qFCj+UoEnrzgl7Xy)DT znaNz$AV4dN1U|EZbhm4b2a9_`_1rY0yIpc3#^n~U9*co z9`_xPI{%x#P^(!#d7RXg7hLmU{oYKymmebQM71IS2CsRLd?LN+VhdC?`n# zvvcZG$c0gEtPpQ!EU{|43d#0({>_hiQZ4E}>$4o^S%`ngy_Tf=^>UJi^V<`Olw1)r z#R+%ZbDuS#&k{dlCL>WJ8VNnj(#+kFagJjs@DIv<4W!)cfln+~B`wRJM0g5vr?UD&gws~6W~RCH{Ten)QG1MAJF zK+s^0bx>4teHW`YOiT_yy)HGMvJNL)HM$IgsG6ULS+oADilG(f{N#Fg_4w3;8K4U0 z63h9YcXu!LJOUW*EchL}?4Ouod4#;BC=R2%Zk?R5DelU{2@ zj`>EnFGqLk-s4m>AdnDlU)73T*@VJ|ZEthED5%d6!esF&C}t^kA9<~Imm$*!J%W0_ z`2K={kl|Q@kY4;g&ct@7-R0Fz7?rW@$#l%g#AVTL%W38^AO6vAXR|AUCiIll!$F5= zMq@-?&%no)+kxp4JwdTlQ^5y5cs*DtTnt-v2C>X*zO3mKTD)Oa)ISiQHRNe}&K{e` zS%ArD_0t+fV1*DX$`|2zScaC>LodXBo=0!v6jRacx(kTcE!+i2&{as@vc7BBX@eq- z5~Lw3knXpnefuD%C!9|nWzIwN(9rDbMIch!&N-)RwmHlPPzfEhXN$4RB$qa|3#myH z@fg&H0>)~Cgz#Q00@!=PRAXNbdcR$lLjWDty>B#Gu{VQ+94zp4o{25shPX_Qbb>3I zrk3wg2} z+^fxSAN@z)Yo;7^Wut;Bhm>&WXylQ7{Wu2cuqiMjRl$f%eWcDICGf4ghnglk?WuAs z>)D#X2~+qpfxs|k)HlyMJtWojb&-75Jeot=;V+cgp{FAnLoFw8YB zbs0FA#NXBW>2i?Y(y7UfV=>F$ov9m266b|s{t zs?+Vd=%ac|b$hN$)y*Hj#U`l778Qz3d!eC6(Wh%dp;T_!Yx-rP&H5{;+Y`<*bDEtp z*~nyN8tyULseWPeh4P_S$ht3GN?$(-2;JW0aOTHprNwRWUh7HCCytOXeay^s`Tj4( z!L{sX&#Kw?T$k$Uhy zr+K2>h(K%}jH#R>L5?~Ap-Ic`_1FR&BUaV_(|k^U00iTLM9L;R?5kR^-`|{um5dh2 zR2jD{WkH8TJwc9uJ(O7X#{d?`VQ}*H>RSct15p`@lu&{$P5hq!D0a&R2T|>s_}$+9 zAch7omA4Jo>Hk~9`){4d`F_y}{^U@Xo`2>wdvVeJGq2e@T8Q!!af|_wXHM{ga1>@R zqWcf3g>Hkan!j_TbtP)@V`W$OJ6i?+lji|1K+ozQJP%ejcw_%~RmM`P)<5cg>zVTW zpDEP;UwQbE5j;S;KU<7MK86~DMIo+h+BxSrfK&e>H%bZK^O#)G*FMV?>~-&ge)yZe z;sso>>z~8)M|@206+O&#TnTm*P8Z}kOD}%NGPhSBoZ4Zv^&9^O;ai+Imwyc8XqrwD zz@G8Ypz`}QXPSr`<=}%I%<-~GFikOKhptAXY&%Q&zt}W-0djUqC-4YC3Xj3v?7_`@ zMcFR2!j8Wm^d>h##aNu=>Pqr`qV%Ewu}nLFX`|#d{pvqc)ZjStP&gp;jtrQ`r_YZ5c8!P$rPZQ`hN%|*~hkW+m4@Zx~ zL}poq!pRClz6vl8ogk+i>bF_mit-oM9la zT6vtOP#3sfaPQvq<}3LBAn!e+nrQ!ZVU?;hL8_D}Rp~`Qq(r4Rk&Yk&BE2KMg^qy0 z14xl7y$ec}7J3u}q?Z7pi1ZRl2&A3idG`ODz0cZfoqgVQKD_UH)_lmCWaggap2^I8 zU%#smRF->?2xj7(hKhLpO2@B0XZ=>gMeYfi}HHE1yVeZNGe!U zD^0)uQt;1n=evq(z&UPGkOXjhB!mO>wkHHU&I%wKH_b)bsm!@PtXE14q7BPpJ9PIE)n*8!{Tv-Aox9&V=3@J-XyHPomvKcvxRhK#}VE%EPRRyWDij|?}1uG zG6i&4D(Ebv4T3|kBJM}(e7y*f_A9MRiwuKV?54gU1j6p|Xs=y_Le!7cvd021BMt#a z+$bp*YDx?F)B{kkG!LLFHYl6*I0MOB+(JC7N91Q?aV*($>2<94ryMmRdt1s(>d|iE zc&WM{$fB0xpOQ50zZHG^nVS-Aq`EeYR7C|> zzlbZPO@O%v6wyCnrpe|OJpvGc=)5zp7`W>g7*03JH z)pfc5l}OKEPMZ4)LL^l7pi>~7q)eKWHZpdl=myX*MdH5~IluV>8G=9tmaeW#=p{Nn z?c(BBu|*<|(E9geLU^kI*>0@Q%QtTr2@7i=AoPhevDc1hXX{}hGPBJ7<*Vr4RIH)o z-Kc*QmXP1t{CD5w4XPv=MEB~4#6qfz`k{_~-t6hU6E>$7_S*=jQg!d#;vwLgha6}N zS1>r_?#ZiJ|1GZ6vEVs=Wx>QRZ+gYnC&GcN;c zKC1Vh5&-gI53MT4i{(KCFlkzucdA8xZUda;-C-`x-7z4lNFXRdAiG9hkC|HMWe5q~ zbfNiebtb%c5`jlG<;L;QD7e?5MRvEaKYsZD0EjOO)j$XXfSU9BAv1{M<{0sYKHpov z0(-;Tupc4pVL+b+lT}9^gen{4>OPZ-5zW6gO*ov=l=+Hk9$JDVcT4$Nx1c_IjUQHQ zSxP)j{iq%qv~N*(;h)(T%Lk|hnNy!?ek6pm6YN#Zz7423syErFDO03Q{jDK%B9OG< z;L)E)D}ACEcrWeR03g_zXyq0s1{Y?oh{6a2@m?$z?;CJ+sPz4g<-cVWC8j3C(6@0= znI^DORU6a$?x2$-%l`qgHWn>o!TL89|G4*70bnlg!8fY69TP45MNi*g%;4rcqQ%47 zqi#>*;rWNu0=uNVQjUZD0BOL{+FEoI60w|Vop{|WjK8X`=Xd-CA-D(R-e(m9L;%57 z_`c#knS3cc9P3J2S%Khh;ZM;;?r1C3NSm=fk%%UAd(U-&_hF9eGlW{``XWVUl9jj-qNzU-)mZXP`fxv&>%RKy8y@)r#W7!4b2gtq)w7z|PUah9f z_FOX{KIzZq>)bx&v3ck~OvW37tQqDZ)wl1t8f;0Z#X0=@R(=snC)rQ7h{+vx5!pJg z{^hf`445s_mZK^F-Cey6{$KjOdN_wXKELB2w7KdYKEK^SPwnZA;+p~2I~4E4j>kSA zQF{DWAy$xlT$!LzTj*7C|Bf;-vGYBlPvf%1v~;EcYS*?paAmu@BYAlrhM934*| zA6uWZU0*xfvwhk140o1SvOk{5?maN^T?_%NCo;N!*+uuD;4Za}JQb5^j&JyFiaKpx zJmOb+%_C^hc@2+0g8&I13RivcNMUrpNXYdv?4jzflBysE=}OMK-X*s`YRr{8zDQ5J zk$iJ&V@Jux4ODOskLPcGVvB6>da75E)eS`_%YCcYo-dC_#j}2B($;jU)i;&EouO_O zD!uqAlQ`G+x15qI7ZosFL-9;niHaw|K?P5|Nn)Q!3(g{>h^i zLfs!ThTk?laFC~d#HfKsaEACld{ko$Ywk9C}!$<`K-(5y+;17_aYtWB(Iay-qg)=J?b)Kd< zT()*-F#ZzjhHdcxN)@8m?>$wA%ZfBBGukuVFv5Iy*At5{H+Fww=XNCT4fV|GO2wK- z3Jk^{OQZOCoJJ&>2xZ#Yq_CA3BEW$Aey4!)qc3#$^HI(iyXk*<0l=1>{ERKW2`RTO zMT_It6gX&P_rBixnTfw|Az(fSpn$}HW3a_0yjdsug?CkKRu|z1UtTyZS9GuZ*;2xu z8>K3n2lsAN64%-09M?-lg{D6}e;Ir%M#6xI*N4BC6~l>&&h*&2itR@8L_>P~ z?BS&sFuT+lzDi*nr2lB$eR52-SFz=~sa(iWYW_0U7h}aCqnZi7H%u`f4Sz!7GX(Jq zcnI)^KEwh;G0=Vs(b7N?2b<^jyY6e+(oc5K1qspe_^NG4--A`g*n9kR-=fc;bOe!e zDm010U(0BCE^9MXW4Bg#2J%O2d|Rto^qth|Z{@<`@6+wYD7N@J1*SckHN1n461hyV ztDI)XDf4dy!y_H9i!^54s~Yxt#T^&#Jgn(B`h}dXlBa#rv9MZd!aw<*@K;HZ3V9f( zT6y}RURCVZd94lQ!j{WMw_fgx>D6!pyQlRV2qaS8jdLC@ORvWeKXsTIe>Cac5v*Kz zujyHUe(pqAK825P6#p$WcyDMBs=hbBnP-b}=ec(|{|^3BKHkSxh^8S-~B zklUb#kD_3WjqUF|DJ0d0Q~92Sn)|R`(dX5E?Z4pAt9O$4C2;>6>%p&!(h5am`7MpY z=dmKcLS|>G{P3XTCFWV1j9jac1)1@??W8sJ!5Hep4KRWRC<2IDyg+_^0Gx15)x5xc znU>!v@hd<0^2hV;s*DqW0s(_-$DWVhc}WHm=Y(4+KFA}6yVeTseel#4pfE2K>)tF?yl*F=*1eV*;iivI0lGjA&R^w-R9oUxqX(V)Pbb1w}5$v@XI z2$kkBRbLe~{QF!tkr@7S{`bC8kwmrgf*}39oVD$d4kPO2eEeHRL|&54HQzenwS_xH zP@kdE$GrCMz!$D%q^ ztHd9L*n|4Rp-Ipr$lFAM>)#7iAa17F%?*7cHv`Bd3ZdB| z@UAeRIu7l;m^rk-_H49&JMKwwo!TLsqlFt=4+**liO~sC3pWGKwuoHj8@hKioraa; z8M~*938V@fXC5v4(T(`h+8-QV%QrJ6RHf$+rRniS99}z3jJpb!(!2UC0yO?d6!EQJ z9kC$4<*UqcexeGdvh*YJXD|7Qtq-{<2_d_P!&AV>nhQugc(h`yr23vY+8TFf7fQJL z%ZYu#z9@pVQ0iLx-E+=8eMxUIJkpcs(bnw_+{HltK&c-ZA7NTvy81GRI^+*t)hNR9 zK{~qVz**q7tg0uw9pgB;0N1XRsZn*;u9AvBH*>aOP^*EXxUvSzk^-TzjF{_bC34wf z(L&O%UGu`0MFFpEh3&7cl7X)Xi;;sMAV|=gsy>zP&u{Bss|Jpu%*UmN+%dkF3jy}L zK=X)MYg-C0^s1HFz8)~7L>~SELb6kJw*JY&y$2Z`m6c6rejd627GnWA?aV;?TLFP^&dhj@Kz zn%6mb76j17&Ip7YY5B|4bIKsE^1q?YWhxy*1#C~DqmIh;Z`5{P*F z%6_`L6d}#Ur=3L^c+klAGU!K1%h+@>jE_|@*Lqm8l)?mu(*G!KpuU){JBNNN@G_yM zN(h{Ds-5^}CcUVO;{V}3Yvq2djF+04nu^$zWj~*4* z{UpKefY%{O8#j2c_k!DqV@s{P(@ONd|D%bOJql`{f#-b>3==-^QZhixS+7q?QS$QW z*ga~%R?dqs*=r9wbk=LRc8g|ykjvdNSwtzDjaE<#iLiZ8<(ye(8`68R>fmqoyy};M zlZHd3;*jd1T6xnyjTfwU;38A?vTo?11nC5lytoYz|Fq(cB_V5uzTpFQATKfa=L`MY zJ}W4`hb=fBSzQf%<$EQZUyjsNpEgoOsQ&;|K+WLpf^YMP@3dUYfYW5JDvd>?@vC4e z()};8lmkqgZgUWdSQRVX0Yzu(K9q1ycn)rw$Z$^p?lIeQP6nrydS>d_*K7Bu@>A!cU{PmSxX~t#LM14FxT{{ z{#e6?K}>p5Yx48pUQ4|ruTsUKK+JW%k_QaMSK^>ohe8q>Oup}lykN2Bx-#}pw{)L_ z7UvjUWaw62tJ5v)mdo3 z4oG)iwab4?jXfj-hc4}$L>sx7l8~hKX*39y@<3rqTW|#68We)2HEH(~Yh&sb928)E zNmwM<+*~_U;RFaB${};@;)+>(12+{*yq6*Mr3cmW%MF@-cC2>WpKylf`EZMgCTJ@g z!OJdN%-IP?a@J6&R`sl+S%u-FYRc=I%+bvtru6HLk>yVY<32w73go7l=y6KD59orv zsUIet|8=$FsdhZBFertmvK*E|_r+>bk+7F`{B~m{CaS{+_^?K6i(B)<-~$`=*jdqb z!te-T0T`>TiHM&;&LH^^uh#b!D&*QlT{qR!?{V3t@bJ9uwdcj3-RRtLnT5OcN{lK{ ztR1pDh^)k)?L7hSjXXToYQY`}d~$(HdAQj9^>tf|??qvphG6$Z<`-)}8pmDCLD8TF z8gLJ{{PXjpC>6BnoTYSB;h2)L$2U_uUo7I(Xt@QDgt8cY?m7&0YkB3i7$R!3%~$6< zr}tqf5Y?lo{1Vit2@?F&cBa4~o%ogQwZEUHup5up{z4SI?9?K=$t8?qy;%0A!D-`+ z1+0zW6Vn+yy`G>xU%L;l1%=2oB*-bmwF)8QKbSMeDotGf=@D%^{k`o7!(Hey0fPD$A#T2(~l(q9q1$ z;%E5aS?J06+<|B0_JbE`@^NYO$6uaTeEM)J?N*q^QV4Dh+}8TojIHmrLgJf$y{&*h z1>^S#9E248+m)Mr{Xqso|JKY890G?}OM>Oo(^DE!a{l{I)f|D~1A9mTh~@5df;x`%LzL9uAVTzjJs0T z=+Fw0*>(v_+WOojFzXQm6po}?UVPgyK8s9$*L}h=Vfp>4B6=~vi`L8cWp=vD>bh}b zbvVkY0p`Qls|}RTVJpVAYq~CF8YCq-CYqc^V+6XJRW%I;&j;0qRR-a1qR6scDyIG? zf`9fNGoo%*8YbBMb%=BmU2}`!8NYnH$`FXXG&4i;EzYU1TXMZeG97=vR}VM-TpO-> zm0{RZgU|6=MoU=jYcmkekWct(2aY!ZqW-BLrWTA{#4i#Ql}^wz#cTPa5Z>t@t6gi* zT1XJCjAE6SjR3~&>TfpV1M03o8`Q~Mf+pBsqc-O{WOJ}MP#23`PNK!)E-4UIbayH3 zl&-2hl+{7MRI6XRcq_=)6ocBcsvnjL0|b`6rP*$z*{f~C_C+<(=T1vqVzz zpW)1$y8@1rjN`ywUvq3cqaSRiC7WiHmZtw-uws!JbQnfVwreT6!NjimzlN%X%jsVIJMDoaq*s z{9dVA$YZd{@^hCXsSitfa}v3zFlmmON)J*s^ZWEIPMKk!54qL0H$PZ?i9A;h+IojYsGCSWK?rWOZ-SnPPh6Ze4HSfl5GDMO8WJrn%^!c zX~lN#+2OTL#a+dqn0W*q?;9jNe`B-3mJdC+T)UiWzqtHMk*43bxiLziBR~<8$X;MK z${}N!&_HnTI@+Ivp(on5ZI@?E?1u9Ile!Cjt;~l&2t1x7BxcbwNXt8dN-7ke?Lw`} z;Z48W1pj>D zqn^s*YTVhgoyE8xN~yQ_b@Ws2)i{g`>lyWqSH?&u1!9Kj2%ce0?NJ6G!NIL68+AuX z!5ULn$UcOFAcxOTiKo9Shc{?85n$3~Xx392atOqpqIZzZ-}(?9YXiL)pwI&$oLTr$ zK%YMf!9K)(daOF@lU-wVWvQTPj5qtRC_yUPe)89cHyTckr*G#SO-vi*@IkMz1-0Kh z-?O?GsU$2Tv(D%2>aIyEL4B6P>Sh4h@_|CIRhvt*uINxP<6F(D$~A&N`j$pqR_PRy zeoRitc+hcD1mBMcowF`p5pN{RqBC>Qc1iO5yY=m6!er_nIs9-1;m=5u<+~h{qJ|*x z=K}f>=C}vxDx87f@adr&x&n<%FOp4gDhalN36aerU`~>}P)a`964QR@u27%%Y-bV# zmk1D9(G$FhY~Fx18!+Yq7NAdVUTb89hCb4*bEI$vzLJ2$baprTA^H);i+M5*QqoN&lFkn)0=V=e#>o1aHVCuVOmzv`$0;Z&k=m z^HlCO2AvTQyjp%;H8BYa2M@^RsxANzQF*+&*9Q{^&eD_|sfBa!07^X4=Wg6zE60$v zSjZ?8Er(t2hIVDUbzD7W5{!I8M7C=C)3px95!ltbgX*`MQsYp4$Q6X8AeVfO$yW%z zEfd(KXrfLjVxPXV&YG#3Dgw}3P!cztCG$nMu43Arr$mjA z)<@rr-u7#FZXeeIzL|LA1!n=V>li2AY*HTm!u+i_)9Kti&NhE_Ge-Bpb)cnm;mQ2n z0vhk0zm%kUI_!A+beoK@eND)sCJB-XBD^U(wT3K(P!e-Mg7H5fbbi3)Ue)%|>fAE5 zv<`oWwx~zuFCBOT8C6|E%?p&b__V_HbueNt{nrhDI5)^>@A%`k)<9&1DjNg~!i6IM zVGVtHyYbAPqPeFNvrkcD*7#JG93m7ytWu}~*ICQjw&R+tE_H>7^ZE(LZg@W+jAk`a z&?MxoQZac?D!|{!qY&^6aO-3SZi)vlw3+4-I6EKAm8zdJw*lL_ZaordH{o;~P&I1~o` zIV-k?Klrv)-r&IXtsdw1;4To-Ot+<_=wo2WxNWL!G z7!WmRwW#!q)@^70G_)4*Ao_#cK)y^P96#JBdqW4nQz6(1%OctbE`(Z6iLym&neq*# zY`Ih~$7FRiuxq()G|d>7o@n3OYP}uq0<~}CdF{ddyx|O=y5-`DjUQF<-haS=RkRax z$iw+!jH^&cCK_o!QFXo~Xk<92qF7NyANUNyZg@En+*u#C#!WEeAd}^~b(l|!R`Y!N zD}Z_^2E2fbxtaST#)b#Bu{plI#!b&|)TeK~3pQ^X1;4+R+w;WwWeB#m^+w7o$Su?=2i-kIBkOX6=qzmhb<-|rgyVFK0f3%2AvXITK1Nondxy7812Fu;Br;oTB zJDfRiwQkss;8Uuj`1yLUza8roXn!M~`$fopYF`DaE`ycpspPtZd8L>XKlt9c-a89z ze>#Sw>JdTxNv*2_MhAN{Lulhb3MBtPt(Ovob!*%a&ub7NfCX{}wfx}Dl?+oTDOr z6TbQiL47^tt~oh)4ysAAGa=Gd9vwiMH~`* z{8t1`w2xDD@>K5BM;md;kq_^*F>n8jM9jSMCqej2(_YjdXfEUX9->;Ttx!Mb(09W( zK3PCRp6|!!zsP-IdkW8wv0w?_JC(NF-<@;+em7<^3rEAMzy=_xY@5G2i53!lfg4(* zKvVyn;{0&0xqV1UgfSH#>yeG-yn7#>%$p#S7T-kz(DQBJ+VOEpf@}iQ)%)ZD*kkle&>u(=AwdX_ z@PxaF247GH(acb^j}6R#rjMztaNytzuNG9R)AH`ykwhmAQ*Ou{0@U!s51_+!`-2HG zKybXswD~T$1L_KpjvwX5E;%+q{gAH#mmjz8>bd&miUJ)V8zzKRxOMaCr3ox0`y=`g z>>8-I9+tU^kp)ShPd9B$1qT`W7yfe4h|IeHLJ%R=EeJD6iJqC^X(ANSV2j zB*`6-yMFZ!OYIEQ`;~T0fpFJ^7CPUZFUr`k1IDW;-T;2#@omUgcd>!e1A?LR!mP!< z*S7zI;HN!bNmA|@<%I7zrW8LN+nYW@>i^WY@gzvn^{vl$1cwh^)AlEAxW6`zAW0DVE6gNk5T%}xIwk3K^kscC9|j$uBj8WUFv~hX>asM#lOn|7tAx4FJEOa{ zbHDoaWTw_Hpnx3OeR+mD(g=Gxl;s4S$jV)Q$3+cey|LS5z680ry`uxU(D%bN&S)R3 zZ;WhX@gGG)uCW0UHndryInFrm6Vd|d-x6`asIvXr~#%jR+7)rC_euRm(EmQ0PP!5JoF9QWi zw;L2y%IDa%R4hPKK2Kzs4xE7WgIC3Z77qEFMPlB}_bW$YYPj&rlHzf{#N+^EYg{>(>SA}>p$l(uQy=~*B zT{jlEgRmBaXjy_Epi{F7{600geE_8pZn?RS?gfQj3;amS>re#22O%FPB_j|J-`pcRQak*jk@=VNDmpN`!7Qz7mvkB4|=%9yt3t zes~2ayM(3cD6jx#yA(_oLDvBcZ0aFh~#z7fsp`kUr4cA3yglVRac z0t4Nylo)+PAx$XlEdT<~EV{qafU%;#qTj%qT&$=&EnGM{1NX=fgu8;~O~jXJE|cds zusaCC9}%x~46fdN?CFx+%P*cJTK&Vl`-`oBzsal6>^Gx5G>{Xn@n@){RFrgG0{2ukANwwY?TOEZGhcgchA(%Po|Zu9GP&sndDVau(Qezf5)Ux2$c z1N@L!R#Yi{7Hyc?mDsGB)Ez&>&MI6r&|gD9 z;)N`a@n&5B?rNF)YKI)P@OZJ|01BQEno>&Cn4XJ@l0Bs;X^j7ZCD#$Js6zn~g`e z5dwi5ds4{$v736`G1}sqdTKT>c9+gVCt-|Q3M3<3Ql9#lSA(})n;6hV81!1whKRAk z;V&Zhbq3>}w}e3I&+XfQLWHlZR{^u%n{8m`)*%V}c}al?X5xub?I@vnTucDgu1Y z_lrS##B|lV1~@(q`pe4O`U{NR$b7`$(ZV~b=^sFctOjY3Yk#CmR7Ui z?zfFsznpTtEi-a`ug-JJ?YFFbvKdOhq)Y&m5|O3GvwO}R!_##W!m`~43zR+UZ zzwK+^hmA&k`7mAadh0oTkxQA1K?au`gBqvsCvP=}AO4s~(ralETsOZc5%ar#N5kTI z!-Z#IP~w!nzi7QNd`el-xIiq0ZX9+A829JH2=x0An%vPaNKelOSf$|E48;79-DR=W z8zx(mS0RCgwM&yq=bF}^8ygyShc1$KE%?8+!WWXXGmTa0h@=Oj$5iWpeA6X+E|no8 z7qoD@dMfWz3PE4d&V%>+k1=or{>ItTTagCW3A%$1PCqX4r3us^DK>aT_~#aNRaGEw z4rwbB#747t2;?(_=HsCG-8e`q$h%ME#dRHlzQ^antrMdUi9p`Cx4#O6H03HWI7N|D zF=W#IpQ0zvMu}Ir@(Y6V?mf>>YzEv(e3ty}9^S;LAG{AlmKYb1bs}nbmSS6|J9`tS z36Y}<-`+0Y3Cdp#Bclr2`P{+y3WD?Y14%Cd{+!=gT^fj>} ze!>H)YWnlE30sdG+i;_-5H~0a+1xbHkOU!(g+rr?{;cVkUw}jI+-1|~v)3`IO;4)0 ztK8TUl}g{hUo+oYZC^lo>r0h-Kb)qH_VIOnO(}M4YLwC-49b!K#X#PtX|&+la1z&s z@8a7CRdAR^+U1>=>K6gt=mJaP&-1wWD0&M)qe9_ho;w)oFoO1C{?X7f!ysfTU&U(f zO>9j#(GB-iO^BRBQ%jAPC2@FdEig#OJhI$rLDDaYOn2gL*SW7;TWYwg{Q1VItoJ7I z$yoR$CIfJ!D0ga_739uO^n)@Mwi^(oF04T@bDqp#kDOO6&e1E2K74uZoH>AdFm~3l zO>*8!#~66*6}NbjxK>(W_}C!Vay$p`qZ!8Z&yjio;?#r{U2Vb_6N(8)zz}0W``4&R z7|)k$+n`g(+}HKK@K+sy-N3-(#tzDy3G!aLt#-Cv{No4O7qDm;!QmfM9On!1V>UdWgaA!$a^Iw}b)N9Zq z=qQ}G9o7_e_6J+CPY_z|EiPUQkHKjamATReCQ~G3@C9E!Jj8p)LE(^4v#@eyOcfoU zURhzZbrlOc-Xyh8aLV(;yk};=Vz`w4nDGmWsS<^s9t@^5Tk`>QdwaK}{O)mU@w*#2KEiYzFi;&SwPCo$3}e`S+vBsP`~jT$e7)yH4*l# z_H^1P&snROZr?;OW2f)YnHsHRBU_1jMwH*hSG+HJ1rd3vb+RU4nSR2C8(<&^+cV6M zq%bKwqtL5sw+@4yyiR&mIRV`Q*TaiUYjn|-(bYxYe~daX^a$go`#BeD`pabCi@Uq= zNjc+1DViCA5m9K}$aBBunT-U(=Qirl&*p7C%1o~)xl|K+RE3ZpTJ81w6=iunt#+5D z$=7FA*Kful^h^wFRaw)L#q~l+k;Ucx*jRxzADT>o^a6t*aE!=c zoY62a{yH#XqshT`q^_u7)s*0p^G8J=*9K@+1Pb2H2-m?~+X{A79^z&Bx{_7dNhlbF z!?~dT+zPKBUCtj~2bIC^F=M*PL`lF#qMpC+31$p5Nd3t_AS}nor9=CZT#LCHU%>a? z;+%UNVO|VNo$pCj{;DfaKFp4cVT2rC5 z5tiGb-z5!N1=F1TFSu?j>ee{4cifhjY@4|4{d%}K>04&fuu3@Vdrx@Ky^{4tbL|dX zs3}oDWDEK3n7COH@&5N*!Aby>&^q-&ZBFa!Po?3U|C42L*S5NI=bcJ@^r zA?c8d4BC4(~0(9)y^Wm#rtfTMr0R2(SCIVO8c#^XRLez z(81dMM!p7@2~rGN!Ig9*Y)Ax3hmpJSN=2q|s>n=d_DyQS$`$PM-Uf$><;Agso(tSX zH^9a3Mx(YD$)L|J+@=pYp1IB17g1U1RM%@}eBMp*iko4TiNz?xASZ!ED_wUL596!y zyE@Jmf0K#LXtDd-~R26u3_=z7@h~rrn;~ofFsafbn-Y4^mMaM})+2~10 zYi7@_y_5`)T<(q@OXW+?u#G(~zo`Oa*L!s7S}PR=%V4MZkvS6It#ea_=_ma12ol#=crS7Mskd0O_pF*aa_0{fRQx$}f~V)O zehw9U0({kQ(INW0&drOrEUtPh&(hkVdvA9kBnsjhAOrhYR;@`!mj}94CoT6E8K!SY z*Vt!QrJ#&IcO9XrB?AQ0whOFfmgHwr5LGHQn>xb9NHM(tV>b4obfNW~()<@e+6)b& zEglXp&$98geIgm-vg;=%(Y{xWR@yd;5rb*tAGmH@?P=RcKO%jF6?^C08u6FH>f?uv zu2bSw&8?`4Rzr|KX5&-G0;Vi*<#l`NzAV}*<`R}MOm1Z0W*I#J3z)d@y5;B5VG7p{ z_N>1PSwp6t=3Q3+5mw#0M$oLHO$^%}SSK2SbGc}$)gJn?KP))$^iuE?>vFgI*hlwN zxCVm%ya?lmoxF|G6>t(@j3wTg%Ry!P8(BKiAH9uIfp02{(=91*T8^Q=qM3+XGV*F) zQ&@3{Nzq@2L8Fm&kLy#rAEhL_W7!F}PYFLy;-(pfh3S;3@*y{GUn~xjuZuE#!o4gQ zlMG0H50sP0O&)aaV(mg{iYw&**^AT$QlttcyA-0eM}X)aqPvJzWck?0Is#AcOYZDe z4!z?VnhoIQ>Nb@!zJ-@|TaxqJ{Q_%oy-3zEvz@=$LE&<^{3Me~uDDYRV}Mblk$ZQ{ zI31H;I^lC~4~WhRu{o(yD?vpy2!(t@RpgF>wak*=6Eih_o9ZXXx2Nvy0d`9W{C@MV z+tPd@-+qhF1}9y$5$>Pl_@QR75Rz#6(Uxndf#HXmALd=Cvb8r1EW*%jTX-V9>{NJ@ zHIDe$IXJPSmExJxr{*SNjQ~ryP?z!KFma+C6mDcmjRmNi zQJ3mTxf{~d7r4SX@0ZxN<5ZF$%%>mFXw?jN%o(~jF`1LYMt@|@q9_S6HYa&402}|8 zA25a4K#IhpiB%Qj2!K?8#q|@GI*0Hm5Vh836bAWrnUIw3l zuYgjrdLbmJvB@eBd60sr=*i!w}Psc-o&k%68!8OoD);qIxeuoG(j2w6zxO7 zH^T;Q5w;X*xl|}4+77;vf!bUWKvU>PA(>WJC~qtIl8+Z~)i{i#<;b@Yq)U+&Q}&E7 z=)5ev@H!eTy^fr*8Yv3%yzla8Eh*dO1h?C;8UIC;wrPVUm#Fv?ahle0M0_r}@@E#jC07hc zscq%(?hldZc+odjk`nZ51|*#ri|f~9o^n9s@ZVz`XP=9&b-jlnY#AUH0Z5*fi|wkF zb>mmj&)8+V=Irt*WOp05PI{OQYwJhm(XYg)dOjBTVH4by3c2_Bxn|1q(XxTlf|*eS zP|)F3KV(S;fQ{$HxHQe(4#C&w_bQ-Ijb#os775vDgbnbsN)RCh5h{NeO+iid&AVzC z{$QpYgqgXg5;n8}$tM)azZtlX0#2($JsbS3c&^e5MH-E6F;Der>bOhxGTigOMq&T2 z^Ww8c>W6~vJP}Z@;nlDan~LL7d6&O#^S_+y*y|A%z-|5I%JbZv_qw$uH#LrQ>gozT1#PKz$lz6jqi?rh~_LpR>7 zJ07`67Bq|dtYXX_8%K37nMv2RfUm-?(DhQN;BWK4rx0*VVv|XMzPwn@2Zdfwp8S5b+u-$same)IKBZi^ueg7@E@^6_4P}c!nXJSQWx=` zUsMv0f7qJ&wcc;cDxVFdGyU>M(e#beQd4!i!Gn!rqg!cSRSxC1o2Ga7qz@3__PJ7D z**CT&jNzmPuentQ^hVb5UJdXKDXO63RqA@b#x^IZd_UZ}rub=sMblDpgVL^t`Lv#i z`TkGt-nccNs-xqE&O;H>`4MzjWQ(-x#l2z~){nC$YrD|Ir@<=+dlc6aq0vcy7=WVs zBL4bSpnOI6(}J%|LQ4mtoLhE}hE~>+6v6@5#G-?m$j|08;lZdmGM^5;pTV7@q3~&mNhEE4zQ4j36juDW* zTgBI6NJ;&Zpqnix%GUjV;%D=})epCV$6rKo*T1h5vU~BRW$tVIky@mqmO{F~O&bejOm7CKJAA3)t<2ZSpVVu(Uzh6X1%X%jqz?BVGos5hF` zCDKC19OgB>X5rEY^+)G*1Rk`}07f=61T+R=Ikw94Q0lE!QW%ldVrTGj%K=-`+AoUv zW(%U}-G%-Q3t1kROgj>7(f-?0NgW|r^aoozj|gp&=~Ngh|Ard!?kN677u{^ZisWOZ zkX_IwhSbmTcGL&I_u{6f{)dpvHB3;~xv#A}k*EB6JvlLQo0T**rA&f-sdL%Vx0+r| z+mNf|x?;b`w#t7eZwo;yQ9~rA`fumm{@+6<@7@!ci`C2Ge|n$cdamk945JG4jDxb* zYj^W(IaQ~$XZ8#clOR3EeBst4BMdX@NtE3qHlN<5bXp}0W86p=+q`3Z_TX>5T_JIf zPXq6Vn0xott+a$SF_*3sbu<*9mNLgggXbyHyY)IN^*VN4gO&WV_Pq(8Fs4r3pSmNg zTo#rW>}oq|SYqPzJHARivsS&42)U1s%Pg%-9*-{p|(|EY&Ovh->QTOPZP*uIW`X_*%OsmO;0F@+mt&_Wu_jf33`$ zPDj&TqaCaEPa8s3|3*AN2jo{;>bi;lBA(m)i+EmUL=@HRF=1X^c=Nh&#O?3oZmEfd zYQ*f?aoDeav&VNE)oYeQQU{>S_HBI`)~z_PYpw+CdAv92czL-$RRsrxmL(s;O~3q+ z4{0wrrV)AXXy{KG%eWl!+;~OTULM~{@W)!Yz57gi-84r;(clPwd_2~Tgzo%55*lbC2#N0hRJ&4PuNVT?V z9p#{FSmV})lf_HgL$*ewtW92{JdG45CkH?L4|;MhJSn;+Q@4`b!O=zoU|t)P-q+OV zi`JzQ@?*Y?R?HJ7@k@kdB2+R!5bpVXU*8hne%^9l znq8r1_+e|BWl!Jrx}ftLomYzkllJ|jXC454kRkyC8k^%K0!!@8ygShMmxt}}gKT~__xW0v!m;@WL$&R^3^YrLjFQ7dhzJ>A9h>C_;n;(aDVeM}EOFo^fZI^2Om zhnC#KP#;-5#6Uu=0KGbPF!nJ|J|qZVv2$K#`1Nt|Ecw`ochj%MSHpIL#Yy3JZgGiF zelk@k^D5*mj`uPWqfokM=oq-vNg2IL8+Jp93kIEKZ-y@GJzDzvuPu-7-%@;f^;*x} z;XUwjSm>`e)qeCuwp>YoqAjYx8&F>H`5Hsc&Y{z*h*m#4GYhQF^JD8$db26QgAq#tg> zCk)v22<%dDKzxlpkKZrrx{kv&BHm(6qA_L@p}ba~t!pvCYofE>&E_jvPM-_@pqUI& zM0%M)LJRk{(FY;~ocf=+=OGZ)2x{VNLGwH8zgau+1>O9ROv@EK;&3frPNnk69Hqnm zCLQ+QF>>R+e}znuOa8-+{lAs?{GU*~8DDoCqu`h6J>B`B77z;V3rS8Nlk6W%;mZ#a zKiOJj6>i@weEHei!Le9o=r^VMb_^)wVGu9}zjTydS#%4`yloV5Nj_KhO#i_$UT&-c z5SOz||2J1&n@@Gr3%L#arhM_DsPw~hi33Y=-TUyn2>~|6(v*Dj>&%Y z;v77(WhZAeftHE43%HnwJ8$R$c!-kGDzPMRS-6HiTMD7R{)fK()Djv*vZU(JzDQ-J zUPEJZ9#)suKH*ma$A<6N0W`sKxhw2&(}IwtR9WinVETk`(6OP=f-a<3`Jn$S?05vo zhD|n>(ZgssXn2+aIJKgB${)d>4l9>mW-`v2;$@MBWP%>>J*OERMtX7>M@W_w)(V&J z_y8R0H$2at>kCTOh+wVTfO^)tVz>-@`^#n2Xv@8aRB$|U1Sl9!OSM-#pyV|7`G(}q zOS@lF(W>rn$s*43D{5>;2qHz6=1Lh?D!7)(A6`og{SPmXCsaR4gm90ZfsQ<;&2MR| zS2k#`p2z;n3!v2hdUpNobR%{Iq@oJP*IutJJ#T+U6c+nSoXskO<;5!T6P-w&MA}*2 zFQ2&%9IWpe6-oY3vH0%6*=DT+r2A?M0asykXKc~z4-5Y4g zBDYdTN9%4Y`4a2>Ti%+Y0ik>Mxx!T(8v?!;L@NpbDW?PB-<1G-M} z6;t}~Z(iuRaMo^Y|DGsa3{xvD+n@+(INggd-S3_rm!wU`&gUa=UumbRhH>MXTD_sGY8G!3qYZn193JSNt zZ96h6yngu76WL$4kr(*im^#b2sM@IAD~NQ1G$SQlN=c2hK}bmn2q@BxAT@Lfh@iAX z3W#(n%FrVXf^?6R)F3r5vCrmt&-**)17Bb>wP)Y^UTa;~|C(~f&Sf=KyfFi1r^Ee( z+fUW|um?L3TRikFt5BC}s5AaA|fC2rt`G2FTpcyHM+wwxMI0?yHUH^Zqcv5h8GOF^`laWOxg}#_LbeaOkoNKn~MS-jcHNS zOoV(z|44GiTZfpbb&O{%;rdM3a;L@1kK}@I9L#IC1wq~La`@q^Pw3monHN8&O@Jdu zU8S!#tusNt!DJqOT}p}Y+~&;#8Y(Mm`@5hg%!S%EjL_M3g86xixmh9qU%zBGs$i|i zS-OMQMz4hOp;1N5=uGk##R=c+*3 zX^mhCl!G!7b&9;vCDAK*>bLe4kiO0Hk6i8pU*p||2dm)Q0*~W{{nN_7i8_q)es;zH zagbO4goM<+jeeLCe}6bTqB|W4A%4iq5sW)uIw3ZCndfu){Vj;#Lu29JIcsX`kjra{ z{I7J$;V^n~TUaA+@Y|NpX-=^;r`FT<;6JeM;{KLUY9-%*NYB(*5x} z;0d~P5irT13u-&;H0(gtSU9F`sSmII{T_;32EXN!nlY?U^(9oQw&nF6RLu~J zDuBTa?vUFAq5G#H?F5(T9Rq9X;J=rP)V*$d$XhI#KM@*!LWL|8HHS=8E{l01L$re|UoP zzA-d9yW&<+z>pX(pLhW%p zPRyM5x+>yMLBR$E7*VIHe1-tjOidqAXOxf<Kt7lZ1zULdg#bz7(14?D5fD z<+A2cB#g%vEoR(NXNU#*Hx>vCH?FGa60H+5I}R}#WPvzpwQG#2i0Aa2e3N?i3v>^{ zZrNdKE=&~03uSIa{KKSo(2Fesk@R+J1VOn7xgtjfyf*)NcJwOZZyT_F_!3MJ@ItGR z{-=F66(4kK{2d82xfzmxEM!$NoA3AgD3L-b4(#POC)9e}#OGlkIb~%IT>EsmL$6&b z7BL(02y#9;c%XmNf@(l3H}+sJJz1J!+XQtA|D0k%TnB6aXn6qRhPCP)z@p+|{;~vc zP{tLazuj~7iD$i;M;5!3-8_}W`@W23XG1mk6Cp$6tAC%1n`?Q(uk*0+MR8nV2t++) z^)~A|48ZyVN0@H4Z@O7%jA+`gO?XHwSUkUcoCp2+I)u=4i=Ho`20C^d{j*cLneU2o zmbjsD1QNc`E%vx+7X}i@#Phs2vTMOO#ZvFesM9w81Dc7lP!V^z+QVgs4sP_6m1pNh zt|v{~zT6c8ZWG!KW>_++x}fdB-Tssf)x(fnhLoyruM=`8*k>?1Xdg3|34p0`V-DC* zB@8zLpfB1|nTv@&UQ9FgrU11vzo2=cy#y1pvTMGGgoMRJCeyp>7RZV}u9%CyP>x@> z1=`h#Pc;-`kIN!kW&EP;4l;10%S*o7uOG5d$5Gu<>pm@!>&bowUQ7R<_URQM%5FN) z29SvI+>h5Ct5vC>_U^-{+MzKTQU!&7`gm@2G0>N{vcjr=?6bG~?i3|SbXi6|;hJ53 zOH31v1ww55o9Y)QN@cEK5uku^B~6Drc6pf)JOeT{?RK;3R)a`HTcDA}K3LaN1T0FH zJkfs#wjcv-5N0u)CLHYX5e+lJzBh<-KGeJ^^w;$da`J7FWnS02$S0+y^A?7UkE0Tt z{{BB`o1lQL&iSPL%Vp*3}nMKy8Je3nPYHj zQUaYg;o9JIYdmGr>jjkej2W&dcX zu|&vP=Af6{LVnbV@1>IFlbrbfC~1`cM@eHGJ#{G9gZ9E2KeF5Dc!OcwFfTCbdbk(& zOS282|9^@b;%1reQUl0^N-6i!{p3i#JHQ?N6CGJ z{k8)GbQyG!7KTMGum0pqjUGQpPLcc2oJ)p?elOCBudo)tx>g^WxvNinVuoEdU(L9H z&qCUc!Q;9fP6E|=R@*+hqw@aPsvd&JqV6I#qu+Cmyy1%z0 zJQ>fR>fI=V+Nfu@xnaFJAq4Vg@|; zHkFEZ?{3uL&V8d!ukug(Rn8*zVr8MXt~+Yz75XvxF3r>46m}xAP)5FIDi#n#LRaX* zy|8!s)Rr~~$Z)TPL$EC}{f+b&J2YYMh_?7`A@5OBUMPWFflhbzF?Rg5jxl|4JJL2b zGOgQ~UadDZV*y|#Q5G<{RugWSeIoz9Hf|oSMeMC1^l~WcDx!5Ir1f8-sNJwcepRH} zx6?h(VSU%G=GND;-oHu$shvw)5urZq&mP#8<6rH~bp?+_!^@%*F_n3tcxJDg zSzU$qD;8Zv>?dg{o-!EpvNtMkPk+K6I$E_lx`;ewk%QFyL}ToKl{zm>=$8bEX0#y^+~zSjmC7O;ww4+f!xvU31jk7E`CAXjV`k19_HScjE;;?)C;x#PB7NL551 zT<`EvDcA;8pqd4)uqt>-J%!;<{_RNEc-EPDz}g94rW7dpexdi4HxaS&K%Kv-t39>g zrTpQSF-C^3|3uKM|Bew27vGkYBsm2Qe`!f|G?}q5#{jsYco$x!_4KVtAQ!9!dT4kC zJtj|uva=49u$wvfRX9d`35qOp+U)q0hlE`2H-73U6GX^fsM9vcR0rY=qgfNK={ytF z&(bPp@J?(k$?y>-#CE~coqkkw=C~mW=YgN}#Vd*nOp0fDzP#Eev%uBXZTZ*#n9RS1 zXnp7A@ynAsBk4n^qM-*bH#;9m{vbO8X=s;st?m-xZT%nWWvzGYUk^_ZoTPGtfh0LY zI6-H6ZBLZ!)$>XHwCN5MZSphFPmdgyqmJTiB_^ov404%jkS3JOtQO=7YRi*p8Vt}S z#r?}60j5+Pru#c4QS9&2PRhAeFf^aTY&`k@|c@Kx`~n&!FH`gjtMs&9+yyg zF939!#2<@X%|R@t;y%ot!-a>8B7N=AlnrZjxJ6OS<+J5|`v*FcGP@Bs7iih&$L#ao!y3`R&=a${!EDY%BrM(T`8IVSXe_;Beg<30ZJU@1Tp) ze#7{1YtXt_o9f;9lU#iY`+?Wrd%vmsi>?=%%LV*0QTt+@`em<9TaLSC39oYk;(pSt6Z(ej1xo*nku8E#WZy5&T}j9+;x`A9+tZ3KzI=4cwQ6KEp+m zH#q^eR6w-nLk*! zL#cO!%lmiB2gD$d1OmX>E5>|sp)XS-5EPd!I&h4 zBCrr8@F!*=A@$U$2p8ISa}A@==IuwVEt`?vx5ByL;tO;`R6bD9@EwbT9StDj7DSkf6!hMd%h3uIkAE$xYphJ zN2_3*Mzkl&+B5x@o}2@Z_PohO4HTa4ett1n4SmtLo+kzBF^*0roLT|;8%+oTYFI94 zJWuAO-s8Y7$uW$eyUhnQ`Ctjf<8iTBr|#%V-6U3HsV6y!!B~1G4Re#MSGZ(09^U@& zOZ#?@+)r@r4aBX7Zyy=t>Hq5CvJeiL%f6L}f(*u^j%XuRcHqyoRe>q&1>aW@W9d2B zqw|AmC4daHcHOaOpeHZsPgjbo&D_`fd9}%~?$g@x2P+TqRMiV1c@xOqJol6zmcz4N zS{XakfLV{h`(uq?#0!wgNF%P@qq%CX1*F+O^BDi zZ|~9Z4Qpz?Uqx}|o8QXutA>MVdu3`C%!Gfe!p^bdO&W9R`;sQ~uKg?Q;#~f22|tDG za1IjRmt%AzXxQ?U)gF{bnWF#fk>*`vk0=eVypmJP7{a`k<%?sJZ-LguKQps<=k^nC zK!?UKQfcaTqiA_L-j=lfJ7eq`rgWgBrPpdIrtq&8mlPE%9p( zrrYwof(^UuJE}qp+PI?!Yg$F@e~qn;TDtmc0UlT3VTwQcFM!c70wo3vfH4NVWG=lR zNC?1%*|7!KlP$l4SmmqCn+83fg?oXm_zz^Oi%9rAZ>;CQslibySl>G6akx%Ryv9w~ zMhgw<7?Ni1iVd-}2$vxPn0~sG2+_Qc&W8#A-4hji1Ib4dEThV6xd5_M1`pVC(dgK^ z50UxcH(r_I;e8dt+wp4?hah($bC@STgiuFs>NQUH?(x49TD#6x432TQ=$T;CU7$}+ z+%RrWzVT!lRP|lIR1|{c!EIinT#VTqaA{Nje!gL)=R*MX6fajG0H$rlHlCXDhFJ8` zC=_zATU=A;vV}g;F}8ul^DvFH6CCgLs4Nh2>+ut~bZ)-iq-D;S*Qa#IF@^q$+93<3 z^e;mjkH;LjOndDAO)e=lyJirIgnA;JoIAe@8){Un zo|{!#|HcWX8Pudo5IyQ^9Svxs*I=d)7NtzRp8d)4s84K`vanvPgfXp1?WJ(H+g+Y+ zu(7&Gh%^B2{&xa#(rUlna>Gyy{Jsez5`Upt7ob;Cl7loKTr|1Yc4|H~5uT$YKJp96 zaRBg$T(`|U{~I{O5sVzvpOE7n%aE@NR?h*mQ=G_C-#yP?<*Nd6@0X!b*Qp}~A+UFy zu6b|pfG_#>m&EXHy1v~R*8xC7UZ=)fSAQ9Z6=ru08?`Pc2fbBW9sr@GX zK$G+$o<3rer?7770uAixf(`A0(967giuKyIGNzcEBM%(2v(mNWIHp*1^?X-bKm z`P~LB$NSkpe4#(DxMjrp9%Evl(7n5Co$AdipSBFylw}4h#>qlXVMR?rB~kayQ=r{h zS8ki9X?7<5??1P|nw0eDIo0~{a2{`fPqIscnqRQE?&f?T3-eozJVPw4KP9zuiLxD@Vkhm$xw;DZ2n2LeFVSA|3sEjaNN5IjcoffHbAu%>*j+2 z&Y-~6AZfr3G;ZD-C%b_9{}t4eBS_fPU%Me`NftV_M;Ur?+&BE0aOv^wsdK)XY5R)p z`@exouL(Uw@7a!C?6|{{mIozft2h;XxcmeZ+CFl|DEe)ML&LopNdxSh_#Ee}n&}1A zg5vf9!as|=R;biT-Tz`xZK52nW7)`jt+ii=nNA?r>k6376~OYO@CJcHK8%x!LrzRv zc`iH`;icMwaO$@=uk*Wmygg59iOLV2cWa;)>qzz9rQ0uBaG$pR`gC(7|Gb;iPu^B( zp~SVNkG=kUP<)5HCQ{;!B!9_$;}0}9WGl{fE?$fdR|2=|LujIQ@j9+I#U^Rr(u~Qd zVLh+b&wf&Q^YQUa;Vm+%n}*3`8L^K zQd{ve)%IL>Fe5=KU`crkalR4AVimAE_i%=DTZ0yR9rx#W;_E2aVNzX{cT|uU?COEBW{SPq|$mZSf** ziQ5vb($g8Z)ajqmcF__HLjxgvigM zx1Dk65sK}snyME=8c!1HZpwtr3&#BnHx~P>Ni?$5T4Ut7{C|6he1{V#N%8q#yZ6e$ zN)-Agl#u-BqV>k%iB6|&(T>pSj%2c+ojRrr>JoA3-(5^X@I7Tk!nO#=-S4pj_8TBBtBJCyPum=c;<8V$)K+iDpY^Lfem$$J;9M`MJ zlZ&dsTyN{c@!$T~UX84NLl0bfhM&u4{3mw6Kg>ay&+j1y zSA&^6!=M~FY$#HbqG`|hV#fYbh1C5VIM)0baqI1Ge#I?*xy9ZKna7SNJ`+o-b^2?i!W`rT5O=Lpd-FO zZ%1ydj&Ug_c6za0BRH(ZVyS&t7J+F|QT_W6c{)EB%SU}+)Z^X%u2kyf(o^$@fS=-7 zh$}Wh+XAg1m3GCUCr3`simzH}x0M+5@?_Wc-^x{o+l2Z+S1pfb1!^fCtfM5EtL6X- z>KpaVp?vB$l?lN(FvG#~%V#R*y_zr-xCa`@WR*x{^fh#>`+0Ql=Y(zg^LKv~lAfmd zm_GDLRGOAaxW>x=zH5Nd@Ta%E>my5}sy{z2=QRe(Iym)_eOP?BdS}IDNu>&GiLpBx zd{oZZJd)(08(Zm_hOn#iNl1UWPN_};hxPhitBf|c9)6tO#KF}8B6 zBcbBPF=ICDbUHd}!blWjQ%NxgH3Q(*z1h8hx_vNNUN~yrBN1Y&K!|2kI9u8}d$vn@ zfa3dl(B&BxN)7!m*W}Bx6$kO0391yoWLb5$kpy%ok+-q=nicDOct`ycI9J~zFH4g` zUHACuZFF`Dsp*?X9(v(^#}rxL6zO7As^TZ~4mXE9?7QXW{)c8JRz`zOc__F(+OtH2 z(mBJ4_W2_YQN!zor-Fqo2bf~_Etijb!+Vk!X;?TCl3d4>*w5|e_aqx>+b3|5tS_$$ zT=O*qXQ3x2aW#lvAC&bZlV9RHSC>EL7~)`_$4;%A`)!wT%H$E+ldzztUZ8 zb``rYt$v;(qbVI-ziG}!TVPUTBQWfg_SWZ2UCj#4#4PaNa!1*IFa0rEXh*Wg2Y56S zh*J{*wV{Mgp|vB)I7$wKP`p|-pFX%t>#HMB3`VvpI!PP?u^gV!V2Qvo zwf%eW-dO7f0yLR0#Qsp>e6?Vwv#;Ua2iM_#eT@)h>mIuQb+f=XQ=)deFBgp}IKCRS zj~S6wEDu##060>BgVu==FC{zz|BCgl4}X~K30L3^dL6ZBzq?P;W8RtnUra2}3@utoOdY{_4?AOG8z zUXPaXw%6Vm>G=nCWDt|=?57ux%H;HNyDw6g{Ivoa@!$#n?V$tl(8Vvi9{gC<$MJ3_ z2y&%P+cqFz=ndJdi+(1|U^i{Ir{@LN5b`V$ZbBySfL0;6PPAwgC(IEnrD%FZl6R+hU8SLo6(=K+m zTi;ZmRO?>q-rO?!uQx0G);rq7M-}L4n=WpkyZ>)_6B6CE38^Oq#jDSpj-3>Cmg#=# z@=nLU?Y#Ox&cV);Dm30^akaW_KN{2!(^#YNwx>>CP>9cj24)#MYzzJXBq1vw zPNJlIH^c(hf19nWXWH~3AJayvT&IbI=Lc|uoVp7{Q)}KUEXa|ouVBe%rjFi{WC7T7 z0JF5&L7Vz$pE(3d=%d!Fzk$1|6FDr^L*!3CTr+txSXq?4@~GDY!Gs1!87Qo6rJmljgy_VA5H2_bZ>*Bjo&ERIijky1;U&3go7~)AwD0#L5ly=HNS_% z|Fyi<@Z3Cr8lp->IGJ%k=c5e%R6ok(c!}EvYqSu# zojX}7!E!X#FHoLXpl`>s%Nxl}ssjn8;`E{S%lD;c?vLAH%_t#YZ;qK}=HN;zI+AjA904$ItmePbpy4D!bV^==Fh5GP4Xyv~ z#P|+w^H1G+y$mZS#)abPlWW@7Y?(P!kd)-?xU)yRcU>h!19k4#Q_}KS60YdJGgmFQ z;|aP7=bnshywRFRpVE@5poRNic9KZoKgh}hhCxzUu^kNQ6*g|LKQ4C$ZKk-cK?imS z<9je_Th|c%>2&5Me-Gn$C|Lz?%7oiQ@kygl7tTL6s=)omlk>Q$$WA&%QlO*B(2 zbBT^ihG_ob1D$`r$x0ar8rC5UxpM`Gg+sNcF-e#nlk8th?J>PY#bjDTiRwlj8XodA z>i=mj3^<7$k}vPxoP4PM>xM<4gv0Z=;czj2tNfCXS~ge3FXDo57t(t$ zfI~v<&?}v_j%Bj5>^XB7D@K;aCn}fuDB&Pee86*v-5eOnz7r?u;6Gn8ikW`7*p5yD z&L~@{pelyIicxn)UdZE;VfK`cFq*g(gFMnS zoSzdxhqLi~*=e}$L{O}*nNUA?kCl_5v#wmSZiiO0tI%O5wU&tbzWdK3?Q;gC$rhoa z-5R)&3gkmQxkE1c1;>ED1nIR0cZm<7yXfwfmamA;3}NHvVp_*w)J}3#aqwBJl}9_9 z1On&tbD#?=c-srNRO@#uNg>SCpr_(c9^`qMbR^F%H0|WE3$H|^rAOL5BER9pmqmY$ z2=d?Ynayc(HEDPTz7YxcQ`3|CCxw%6K$hnsx6 z>&JDEsAjWBlqzl26L$xD2S7c^nW?HzmjXoFe@(&vP%tv~PTBs6uR=%*;7H=EVK?uf z??>{f#;CUuMCx{GQguS8lG$Zrr>ZR93*$KC+{k{U?oxw`2?ICtIIslNz<9WmS9aT9 zdNLE%s|RCF&Uftga%5{A{b#(2+Dbs$IH{`?R1zZJGIJ^QmP& znOd)Y6gcDG%GT_&{_rbmAQ{Hh+!l1=0>)&EgIAI7oyi0CA*%+FS>W^gN(|>D;ePIj zFKN%KviyL)3>&%!x@U^(8`bNIuV23&66A_=t9>6O8>l}C3sY@s=4O|1x{`b-p{e-K z=cu)Mf#gV`7Xswy(vf_u(|H$PI78wlW2$78*Kj z9l&)7;nhyQ5HU%`TRb?eQPDdQ~!~3F7mJXtMkD&44KKF$j z37K9qAE)6jfg`bn&`~Hk3BS%et}c3!v}GB~9AQG4{trd;-j-*fF)~FlBl#a}YRCeO z<^xVI_%nJi&fN)er`l8y8YFiBr*bl+{3gy5aq`N%-j7`QcuZsLo4@O2XTooHlKgV0 zbLp}zT&1C2O_EfN12@5`%fc;gv$rpYHibUxB4Ag1xtoOLX}qnj3Y!z-$v6^1(xDX! z>PFmKmT9cp{32e$xu_1>k0fN=T8TqIYUfy;18HYJo=|dDZ5?vB{Ooi$8+S4?Vy)DT zKU*5jeN3y0xrsYF-?UqZPLP6NEt(u_KC&{0yZQ(_9oZ#7o8%ShGn7u1*$jf-+*WX#cOp6|$ds>RsT zP$@s5?}ry07_B+2w=vt7%CFC+m3^C^IbR8m7<$hFKhBn-Fh9tD1aWt6^gI`>JJHd8 zOY^?6nZR>fW>z2;*26y!^L|EZyX`HhhC3XQ3n=ekRC_%siZ3Qc-Dfhj=qg#*_KF-5 z42|Y=zCIzQ`R%hkUI|TBKz4keshkQHqj_b*q(|65Lm-pa1-zK~d^DdLCT0QNPc%-M z+(d2+gCph**n2K@Ry{2_0g^ZG6N_FDrg4(eylaW)w$f?CHn7RAKk`=h(@Iu8>x}AR zLF&rcvwctBLf_3Ffx3UapMS|oR1hFxyF(~WQJ?@b1U5!;`WX9|l5XeL6NdUywPd=K zTI?64=Bq30vjB=&=7QiB5A-sc+f{c~AvY6LCQ>E(FD+cEZ zw2H5`t{}OBIVWqZAl2XhJyh7O?e=cid%8e|CFc&p@Ptjz;?EXot=nG3s6}1x`o}!D z>Y0h7CV1naDnin>|MZ@8Ht~BidMetT}Xru~k@m)R~L zN7nUTjZIv4d5w46hF%~qZfSngKZSeOjweutiN4(S}K zbZ?>AeO^t!loPYx=E);i#>)xA9e$56FZB^nK!+21Gwu-x}Y;Kb6znk7SebL$2;GtgZ=06vNabx7Y!WzvZNRdbI)`s-WXLm-Rx9WoA({ya zA~P4oAMAH%jfgshmQKu6^lwW%nudv*)kK1DxcAGypL{iX!R=&$#gEq-YOvzPL(?*7 z>N9FTxjEs7M@OT^ixCgDjNBCyeF+WqgWX`<7y0kcX5D&aN-NQf2GMGP$Qvhm2hx?6 zP3)ZhCY~T4aU?qQbcsVdg`GUV&2QsqEBMr+7VLUsz`(i{=%{L(;!A0u20&7yXNprh z*R&`P564ZV?0d)pS>#Qyrz)hc7X9VREgIy&R+a@C2177a{~#!qo7HeXUv>9rd~ht| z@r0WDSY{~iF0tx4#vx1b1v?en?Q8m~=g$?m2eGyj5X)w_1P+IFuWnJ&Z~G1M@JDl? z)^*|VF&H+#6ndtTl?ur*Zw-|8h~=UPSPYflZ8T=AHZJusZ9V=sc$pTJCd`BF{mMgl zFKPQRT>FO)t~2GI#X1k^L9OfTXMrWdX*zkL(e65#7&%bD1m;X!)eCT@KyvzWidoe~ z&a?g_(3M-3-hRV*m!iE_ukGmKBxcH8#(`cU5{Fj%9#W}T7$G_->2 zjBSu(4L&&+2+V!!>)iq5mY~jKX&+&+`b36w)MXXsb0@%u5yUF(TfGGam3q_ z$8&eZSsqkR3Bonn2<+*_DB_n7we%gDMeWpfz=+$YPIw(5lJBhi)Yr6n4ZNpk&;eq1 z2-*18aeRbcLBq#byUKqaNXopuli$ZhPpfKPmDSC{HHLRbcO~U+W`r`3G(j4mnjLLi zlA0f5dxhwjn4u?1?74uqm?(-M$a^j6g*O4wtyezh@1o&}W4)&^a075PsktheS<|B< zqSHYj-A(QG!|C0(J2nYB9!IV?G@XHIoXQCSW~L&u%eEN4ln@G7^i_i>n_75G?xP*+ zFu5hXHu5_ts|j_y<2dxk@Av25h4iZWIjIL)#Y!t%hvURvA*Lc`v09k>8@vlEMQz`m z9K_4)X{Ic|xv<_Jq`ollKGQb#l0g>>&J)j8%(p$hDjj?$j{OdBFT6>8vG$;2RfQI* z#NCF*Y!2trUM4a_rv?PWPvOa%+n#f3=FGDLt>gZ9s7G|$b*gYVcgyYvf_c##P8Jx2 zU8jZJ%HH3?PP0WMf7$~2408w}>htcC49V~#H2qA-&?x8-9LOB}r!YQZPqQSOxqdA2 z16(!}R`;S#5 zEF&%9bdz)~x?TY=Kw0a){CG&3S8%-hItUAOdj1!~u{(CfWvZG440!9!ruvdW-{xJ` zXK(<;ZX@f^b2~NfF&O8DMDF#W!R};U<*ZF$hozdwwPLlU~4T4J_p% zi2u{nk8b_OMRNBNK};78jInYN-xzXonD zx{qCjyuimEV7k*^J+AIFaPT#RwZd?Oi|tSS~oRQ%ly%1zY%naf3sV_kbeRD z>Zl6Reb|+w)P2@J!SCy==M=2MbA$c9SblvY&vxV$B?aF|z`oD=E9H{q*YvejD@nzU z$0I}qyW6nP7jK{%E597XxL!02H8s*(|IxQ_lX^zDpa)fkx$*r`++dyzpqMq}V)9RA zeN>_MB=O&UVyj<$v+(lI(+15#$@h7tyYKCk$yNbmuBk6GUSc?aA;Nb+3W#c-#MOwI zF4=xEW43g6!xx%nPFqtc=y&&QJ5?P9YQK&yIftae&ml+9(oW!p%Yf>gBm8pYIVI?* zSM6pQ0IG(taNmp57;x7>GSZ_X-aC$K2TI!tV^<%Y)2gCxn!gE%&rgzE8tpG^>pm80 zKQ9xA*jI1c{={Ji;LoB2KL%1%x2{oOD4>>7u6<&dm2SH+&jBWpTWh7MI?(o#wa}dgOn@bcXw`(1*b1b3MObZE~I7 zpyfDjD}X$eaVAntnPWcP{zlvAzRX?SL0t#BjmVct&1e-H56JS-4paspNEZgo3xEO1 z=OU|%p~rRC=O&f5R-(`zE!DNlVc4!qf$-5`^*1|}pnJAGTY0|NE-+K6=jQJ(7K}m9 zR6~yoq{C+4ha)aN6VS6|@U`HXlqiqIZ+w7i44A@B`Dr-oDwKfXTZ(x?elW&Cx!4wf zq?MwJ;=7%C@VU4qX#}a)YGxbC1xjJ+6r?8z_GPuRPNQ@(o6ltP(L-fesEL_7^I3DS%j{>8OO4<)>)Ss@X| zhjP{I&QZ*X#gv%6X|SIa4ODvwvqg$p0c0fjbsfFr(znO=dM)a9nZMsGMbx0)Q^TUW z(6nxL!u%?RYhTPyDQw>UTAn@}=z~UgVOVCmjP3^&_56RwC&S2M4kc@2z{k-{`2U`O z1E5Q+A3%hi9WdTvFBK%5zTnkMkg-y*CVLG&ti;%}h#QnHJ$HXSaY_UK^|SYf0eRFQ z8UYZL0zsk1cbiUnEv=aNUmu#cT;fV<=wJ5Mo1r5!2<<6m65%(dMN9sH>A>N>BiodG zhoP3gT_^92cx(yHz~*1=6}a!nq2Zf^C|LBPDP333asZQIpFI+(dLOLOL|GM|3;kKO zcH>g=i}t~gy{s?uYTucvLIr$l@|?cvtQV+ z1GY`hAVg|$U0ReZT`piZWSG4PjrUD@hRlCO<)Tk9_e-%%AYImRBrhR#Mk0L5eQssu zJmLW21VMNP(#5}4Us|!$eh z-d`9L_ihK)3Mhx|Aa<(}!W5PL0`{x9^KPQi0UyKam)V>H$0-wH{dsg>T%68;QC+V! z#0{;cpnEOaWUt35h0EnL3|uTf)tTshTHBz4q2Pz$X5}B9&9VBf6fY!wV54Me+}!V7 zMeE1QWoH961ZZ-1CE3JtT_4~W5R?_4F#mkZ_+6Pm=_275^f??e&l32T+-8&1mhN!X%k!KbCTeAQUEEP@P`ExmYmYWa-CLyF)PR`MS_d)oit) zRfU9d-*(7D(O>LI`nQ><=bLfGM~ibukkodC3kX=%FNB^!?bua6hV-$Ob%V?-@q@Ue z5X}cD1c);Dg(uZ*5=>5pMc-pc4u9f}mh=!m&e)aCPIP_Vj^B3jqRfwF6K{Oi!^!-o z@7w+Eq(9MJY6V*n=R6VFSzg;G{~k+iI9m&kW&Ay-C>MNT({0v7;M1jRy>fRWE_&7A znS$~g*Z^BH39Fv4KDSf@^a#2;rJ^weC@pMk?L!EJ-Oq0ck zDaBr8(V%z#V-Gy37^@S6mB3A5|G~&?_n~u)Kp*rPF(AmDjZcsz$u%l!<;=JoBRcBTu}By>0`t zdB;)M^H(+tCFf^Nnq-*I#Xfg80Jef~gN;K^t0P3?scj0FuWobvXr6(7o{^WjQ)TF8 z(H)4c7yEn0a}(VcSqF38%oLhaQ0r($%UHzWDBGjwk9f81Om4}=-dbPIV9#X{$-!p> z(f|jC*45c5h(Dy6FmvIsL#q_;l$`v;=4(&#z{Bo~U;=)TK~)@dW;IL$zcJj^ivBwt zNQ}?LNU(pGvp$`Q%f8CFG6Fce7vB zh}{dr}_p~S@*_jZudhVa|fsCih_>?Aui0%zso-3Q{Phyu|F4mm_{E9cz~!?$7ggNc4f!3Hua#+VSb zYz;oXeFpBz7hA=(F26;xl$~!QCX$!tn;eJP+vm4al=d&Vz`vMFyYk{u3&Y#CEtFpk ztzb#+_#eO$7M=8|=5R5|{ji&kYc8OJx&=_fbFmK~A>Xru?KtLOkwyuc&L_~IcYQFe zt}5Oa3qb66qRG=ENa&dv)vsKE)0G!D)+Ctyl-VR0^(ZQOTrZL?&_TB6_)Xc?*lUH< z6213)YBOSTOU25|Ca^u!TwaIuM?Qe%mN++lJ#h(u4qiQh&XP* z==px|`gsX!gNuh;gcA9yY=FlL1D|_$4!@?6Cu_*0P|{qQ1^$vkT7+A&H@@!hD{tU1 zRSN?7F^~%SMk*e`kj-9%>nn)3{eUQdVRvBca3WJOz-f{$j1^1l+04uWh@#q7ESsE@ zZNIxwZ{{!4hH+x~z#P&+FHjJrzW=SM*8aq0tJy$afeGTUS8JwIo(yp+H!c2zC)fwQ zuCy1qwObQC4ZWD#4#Y+Nl|&&s<7uI5$urpvj+X=Al|SkFu~D>l$-wr)x^)C}!l(ym z#NW^*<#^&IpyR%f>QafaeR`_HS9=0oE#za*z@Kjp%dy=ux*8LxhkU6kppp1I!eKog zBX;9U?yd5fA4|SGDv2pa>yzJ3JTb`0+hBE`rok}%j1c$X))AsYEAgIi>#Gv)`jfb4 z^(SToCWmQ=%bm`a8;v||1Yeet(l9&^Bumj~NAuPN;ysvi+((x@WU>3IfpBuSf;L3q zm{{SWe5tPD#tvq_(S_LE!;N^y$Je_-oMnS>$`OJeo|anKZ^L+0;JzzwEtFE``EOW^ zxt(p)dY=1bJsNz_ z8?a#-`4BS-F|;0Wcua2`;O2^*KDys?`!MCY9B_zZ)Ndvz8f9LbwA>7cnS6jvN8WdK z^mmAQt}^y7U&V<=iSU+Ac*ehk>%vZ8Y5cvrwQigG;gx-zMq)XS1#r$pY!%HWY-9N8 z(169c0kL8gEdkEV)t-YlrJ$LM?pJm@l3gqN(ak%MXx^sQxf3((r?74I%y?ttZ0DV2 zYWamL!Oc!-^q9<^lT9^Ju50p(eiKjFU@^OH<>9$^+g4DYKqfPz!E=#NFa%u6XHp1* znMl3#I)o2yJf(Q=E26gd&A*UKwb>kD6T0wbhpNeCI#!#SL;d9M?-{tyI@KopP2ry~ zK*FN_+QtZ;`j&nk73wgh$=SH9D>{LZhceW|CQQ|2s6S$3IH%JTkJZ3moD?`9^)D*3 z4a8|4M#6ek2$+x`ZqpF57;9K<y{DLmFj-g=-R7SR3mxXV zA16*uod<3QUyLu}!(};pe-wU~V|1&muq`sqC5#%t(;Q=eyG%mKI) z$J4f-Z*Z8mhur;svSC##=PCZ@`Q|;1kf(v0`2n#K)@>dB)7GTcX#Fmkr96qr>N_v- zdhoDg#}~qhl2S!l8ThGJPC!uAGn7tHM%wviOa77e7c5F8DHJ0h}I;6LVjj@r^aRbEuvXMWm!6wZcRPZobg-(N*hz1KW6$UF&to zi^Ftjs`@ zhHmLr7)nAKDFGQ8q=povLn%o`KtP7>M(IXMkd&U8wHM#tfA9T2j(zOs&2v2Op0~!- zTKB!~>$*PYdHO}g>9snzIZb-&qN|&fP!F$Eezr{5?^qV9Cer5aKw|arP~(@7;PK`p z)QK9ytVe+VLO2$_P2UJjo$H>#U2--bw4VO`6U&D(;;#r~4(F&tZrz?VxXnJy7V~&W zNGJY~_Vu&hzVS2s%=jm#*gE>=mauVPRxTuqZGtK-Bfq_n01g$G)$Bv#Kstq^%E8~V z<7V|e8hYqDt)b9rI=#aOC_yQZRi`~~UTSN|R`d6^=YC}+EGvZsw*9hc=S}6?U%l^q z7383-7x0A|Y$ePY{0su06CIs(IN~Y0z^)B?R_YO^nw~e60NA=#ipzCp3#_9LzAtO~ ztTREK>RlhC*T4=_jd_y~JpIE-^un!KNM+6GHL+OO4efnW%wGNV;E^+!wL_qa6C0f- zQmEn3=t>Mfbthq3AGY@kC)4xOg2df~{h)gSk;I~VZ#tbeH$4Cv1mgUwb)%ane;WNz zZ{P&X4BB+~sR)h@j%@n=?IFMY;!cA_9U*S-Gr7oCy{J!h+1gmPaha*v>gSrZ&CSTe z%HS~xxAb$vyWT&gjssI%Kg$zGKI$w{+w4ocuBjhHpz+Rj#(IUP2Kr$Q?6ECz?1!Gu z@`Z6J`Cs%H?-4x>{3t>^9+AZJtFl8Eg6^TSG3GU1M}IczrQUDq7T7W=*So*aR)&* ztv3OzPqF@|*`5P9!`d{~!?i*8%cL|^*D{VjvPu2f>-Z zI6B;)t!0G6uHz`cez3(%(PScnEbYwZiurf%f5(;gzOF12W=2VYo#AkeEBNVd%xihf z9Hny&GL9~=LyTXR3+5iiDh~4+RoyfGJaxNM{?|KLjlqqe+VMN(r84x@Rs^r}vk~SL zcyd{4pV7#du}^&lDO$q|M^K0;r>n}a`>?%Gbe|S~Z=|n*PV^l!EjHjf{Mfi*Ne-XIt!~0EA zAV0O)rxY)H$#u3_8$o9#+K(9hfbmvGjs^tQap1BSS&aUHXwc@z6S_TUAg?}a(E6ws zq(nNwL16w3D#um%`9zkc@snYsA$dSmuQhCeq1T4;;IbBlUX}|198NybiWF$$8PdNS zq1iQQFeB*M)3C7@eNrM%J`#6x%3o5*KS=DBxuFiusU8V}Pa+<<>blsMR4%$UMY$RApJ&bqjq4)#`(C9b{p3AvZzo!8(b)KEU;_$z{0K;1!56QC;k@XYK0&9d` zDzY{PYJ{3aU48)OdC;bO*-AbqW1FpsI(Hv#QXGJ$BAZZ*%e%a9(CZF43>!mRRBwIs z>D|h{Pp{)4+gd;KT&IMU4+y>WXQL14l#jA!r47U+PVmLm~V$+@KLs%PzYsrt$ z(K~)XsSHFCYJJqVrnK2Jp#Olv8hN0Z%IW+S@)e>YA3#EFhwf+YPR;@nvux0r6=AW3 zbcFdC!j^F*&z{5R7ZpE-Ix+$7Kh7!VU;qXQk&4>p@n-JBmLFA++av#bVCb35(_3GQ!ZkreeBG~s`mhu-B!bz?6J3u6vPADAmn$sgzUA0;;>D`~^j8S$qkTFhqDu1%* ztC#Bu^=-gg4jPVe(9=(@7`V~hP-s`zx7#JR>(UgK)J~^CCnWVhV-`b8)o?L6fP=;d zRI&8iu(W_I;FoXTcOvk6J&KEmz8nbnI5Go0Wm|7ic{Vfy?SG7AKRg4~Ri+S5hs7Uj zC2ewQhvxqvYC7*np<5_cXD`@8^1fBoLawh)zujSCN&Ta>i|zKr-wvrOMu1*)o%cCXpX7Kv`i90|<2J)pk4CVn$Jq%%6JZXp zF!r8@ca5XR#9UoU!IcAnzufoX3%KS9&(md18qtCt#SrN}IgY;gbNX*=1}~KvZTSG`a?6Sf;&Bw*aQ5!6WAyPWl@)Ts5VUMh4lc6u)?XAcuJ} zv990z!y5f9W;K%OV_upFyFT6WWLg;w0Hawud#RuYrSAKqz+PDaQA7d52z3x&A}m2o zA2r^0FdjjN`@tI4D;9FH=YCA{8|I1(n^(}Rp%VS0QbBAvcq6%|VGWy9AM!`z&qv?b<@A&x@F;8AypWSlht@)SR4 z3q?`FM0PFVv-?OAK!!|y(*z+CYrf|frbUY4MpZ^6I5~o<5|d+%TogmXF31AdiBx+@ z?Gt+MIp({-R|Op%9b3NVn>U+aGGbIgKPrJs)o<_-9T0~Pix!~}bj0wwJEe)R=@q-q zGeT5p{REt|etZ6mx64++`z>X`&uoMY*TI!jhAZDEP+GM&n)WFCZh@}tw=%~o7dB1v z<4&*u%heh-maOfOM}g|Bzrm^37-Ju7E1EOD`{DEp<*<1OxCfEQZ-d?zcuUMriy|)- z^9@M6w_)UhNuvOl#8Y#6xx1bsDMe4MUb*)A3qRp^^don#rl}GLo@yWQcn6RRJAq#f zjRk7P{J&kB_oeMPmssDnp$laeanh))0#b@%&2mp>8SPZz(Gbfk&H(iF&Uz(p&2Opq zxU>51dDpcj-*~hOqzPAo&)oWdR4)R4^=Q^FbkdFSC%3Xh%0(kJkc~3ud=W2H%GXk% z^aRwIL(Wnv-{UnamWxk1I{If5Qjc)QaP-o*$W>}PD&^-%IyctUWjKjSPynjX>^YA)Z->v6#@u-OiaOXAE#;z9xN_+%?ZXNLei3@dKw}WUWVTWccIsKxAgE*~95F-IzST6|8Z_lSh&x02BG` z5$qlT;3WCG`njn?2#gR+jSM{hgVnU^hn?$*#7#Ejj^S?O4*`Bw>65iw?h=8&8_iTw zvm=t%?^a8)+y#Vj=g30Kh*q>rp&%8jG1vv_qSVoy{VdUs@$e=&kir?pQRP;W$ z!Kbl$aJ6`DdQ89g=Y}p!L?EyBdl54;j;W znLMfee-BkS<_aC7xoA`*DF~>2-tiYaSeSA3-HYl6Ss2uz-%wl zi0-TVm}#cby5v*zRroo~4aGlyK1rzpgrPBDdGuhI#WChm+|@ z5|uv7`4e)487n*rBy`C9(>Z+2vn00m#A&)QZ|qPU3cFYd@+f~^_Jq%(nJaYI30%i^ zsCfBA#-^1*_wy<~=fX3n{i4#lEd@OZ#;$gA+KRXmzWU#6AgUowX z>qlme-~klhL7xpgIJEelAr4dq`ZPiOXWHSZl~)L-2)onGgGpQ?F=y% zD<}-`<_ZEXXw$4m$Wkp%mZpnj15e&YG$!9tWkgDvZ6W5+-vOZJ#do|??Ysx0*_*5% z16|cNcbKugaUN8hFuv&_5p=r|_0Y|ih%^1~ZL*geFD^&c9M|gA7@{~CVDt-(Zkv5( zEmy1iu~vd1$N+zLJQfZ&6qR;w*}gR{%0g?0z-vWKJBs`u}S$=)*oTV{~h<)ufkO0S= z*TKJc)e|1)4|febF6MnP$xFnKU*EWAp(AZ%uo?7T1?u}9y1#lXfyPo|G2~`@d7)Ma zR;4N3A}Vhyy*a%Uw7~5G-}>b~@yP?=18}~?q~NA5%>ELcIUxw-u*B#^rjX=w`Tq|E zI&GoCsOq#*_5GTWMLRV5`Xl`-J%rrBk4w3`9M&e&44MlGYKa8~*nxh)<+A_DCRJ;E zV*1oRoTF7VOy{w5*95cD@25gW$mMsh?a>9+a}UtCS>c~pwA+BkXf28lgxv)ba}SVO z6{w8!>wcNVrK0cx99zi$@QJ_vu`BCMlzw~?#jMsKI2w-TOE3cOOS=2yf{7~K1x&(d zZkb5I>^N6VJZfGFm0_?bA-+QoTb#iA(rCaS{8?}DH8gwxS^Gr^r=((_^dsnkrr$6E zWCQG~$}i?^fzJf-wT)uBbf$2Vma9uJXYPTU&#}!uJV*4Dyt6Dh08>s=I$%PiF66D; z`-~7khWg09mG;OfXtyZlWOB0cY0cCjpd*7a0z=Fte{XG3c0jS2W{hr5cFwj6|IHyF zoE(Z?HwbH9n}&tw1KfX+_%NI@5kk8-l}*M`m+g7zd+lPKN`i1j^Dh}6CF__pgv~8W6{*sjXv;rAT_lFY6}lU6kvm{> zjG$wwVjp6Z>^47*o6G-Rrh4)klOprG7g|b%k)jq@3;;eVs5$RI z?`4hsL^A5qZs<`S2*huTEI|Dra?Y`ux{qsm2K5FC&(9YuHocKcB4QK>da!nX=4_9m zLk}@-zc0(9l3YBCGNh#jaBzFUz&yLJkbc@TNarf4%_B+=T`n$IIal-BuaLfOEQVUi z>7w$hie+cliyb|1tG!S~DwsVj(9=T;erub56ZZ9oT$yMx2#B>!&t2{4@opjw#hr4U z&mb-r9(srnS*f?WK?0ubCya&C?M>Cso80R>i(DHksJPuObdMb9Ja_H|T&>60G%fon zVP~2nfNY6HLXc7TxNVPA!fgDBCbjPSYtuahojGwh!WvtdLX({ftxt2A0Str>x*uB! zB1ytz!#8PX1A}Rt-5#&E2THZ0H)@gkOJeW$ zxU6nvkagW)h&H39_VvsGqsOeyzi027fiMj0u4RA1P%tZ-Sla(E;Zv^+I`Mpbr-W1a z-RN#XyXK9kJM|xCR=>@afvXA~jWJ+`8WAvX&(l{D6B-bLto9+Ya*)xbe^)?I(^EoV*FzWH7h0A^|23Hs6 zYj7twgImIIRT|IFO>N%OpK4y7gW0ZUOuW4Kl_u4iGg~2aYf>ODXh>}nE+r%8{J^d+ zVf)LJ11zF|1`|MEBG$Q=JQrVPS}t8@x}(3R`J`AcH@eAOxsNoxdy4w?3Jp6$!ASSJ z3`k~2(O~@}--D~|_yzSze6j2KJ*Qt}?bw^e^)ck(Yd(?!ZIhg)GsK9_)7GV5n zK#AP197;%(`6<~a!-zO-8y4S&CXWsf*7YI`&7?%`j2B=B`L5u<`dQwYb~rw7=d^Qk)gi1L_D+#PFX?BvTHy@9X^;sT2qNM%4!3(qZIoYYg9E4 zZ%R~0X$7|zh2#s0>Od7p0BPd0?t-F(2l+P!(m-(bHmvajD#I9wa>zNA6X#Dk-)gT6 z)2mbejtcXJG7Wvr5^Ff1{lP+t`+&Ckr#I+x<(IDFK(7!G{>d}?c$r}bv|GT` z^blRqF`CGH9!PkeYbG*urlV2lGb+7e`@H5m$V5Ep=8rO1dp_mrtag zI1^wiwsZ1rk5e*ix;~Af|8Lsit`0m^+@b+1_W(Equ5anJfrsfdqvsN`K6W zufs{x>ok<&4R{RN3LjZt{is2<-hn~v>5`BYN1#YJPyG7vF zwi{xCd7k~F0>GOY2{P7_44cqwm^4LB16sx*QvI5*=V$uqPqLX~f6t)EMm&HJwj&sy z0&AWxP1~jQ*Z3PJPwXv7kvhbV_mlg(^OS==Jdt$zqZPA%Do4Y2i zI<9t#YRNLpN@H>wVpDjqJHs z`?^_l`R}V&fbCHLd5vG_l39~L3hy|8+u)Q^Ajd!!tK>P_c9Cde)f_i$=L#Edr zEdH~`lJi=tCTj;a^&7fgn&XbgrnEOS{G=Z)_7q1chA3jDdNJJzc$Z{Z%!bm%@A3P$ zD>P@=0eMTrJ`b}wE0{!;>BxaLu7A8o&@>gH@Dl;+6gK_Rc}DlfBANBciNv=yW?SN( z*t9O?v~&+SNIm}iUiBrS05$vdC6eS~`vsnN{mfTp|CIWwr3baB>-CI9Zk1Y@88`2F zrLb_m1RDdy?=~lp0?gL?#6{pSp&HUsH+Pe!zc28?!>1fE9(nHZtI}BSs|o+ppfd=F zeSkJJvHr=HgtL}GkCCUtl4k9;Qopx{^Bl}2*QKPSo-Qj#q-Fu}$2tNFuC0|Ym|3gC zV~?93XDcACya`trrVPTl9hx|w9pua-NIR(+4lN}}Qu0kB5al?y`Ko>~f43GShlPMvUu3N`Pn6{}O(+0nd>0vVzCliezLj+>)*o<9LFg$5m6dUUpUFf3m@`>E;JpbU#hzHRCz^ zHY}EOMUfuDg(sITG77kN-H>(07OP2MAmJ*3H}vYPj4d;lnaScj4%v4Usuv%=7nswRZLA1r8E}FN;*>3t(>& zL~Z>El%yi7&R3Va95q}IXq!%gBN$T3qn%1Qea+w2=1d}H9 zSO2}QbOs3zy)G!eVSK4g?ih=jlwo_B-Uk#T(ceL*6EIAe_EcHMuTXn#knwvD!rDY= z6!;_d*=8<=5wE`P+26C3pu^sNi(0SDIh(_{R@k9g_T|o+bcU$EnJ~)#a1IxZ8bJ8I zS^R^_WCu9cJ}Ihkyz8Q`G%=?IC4UoD?yIvf(6DYZG~KcPlTknQ(UH;4pV!mUW*ySG zjLv2M0@O?!eJgC7$530(DlGs!t&W9bIQ5p~=HyBE32~9&t7Idp!m{hWn!BNUV-&`|Wy(E01I4tVt3e10RS1gde)CmA zFw&+wI>Y?7*0AY`cJ@;#7PdI<7Znr8f{iaWXc8SPqFdB>vf!sS)&xCaz_bH+`2`i# zuYTU?oW0*LYTfdPag##HoSCXDNv5gOM^Y(528DMEdrL(JH;fGw>{36oS5A0T%K9-pzf}(h+}v1pS#s$^@cug=_3pw* zl^5m1D@^gOGc&eA!m|KE5h^K?{4eRia;+M1^Y2uIl2kajc{-Rzg0pBZmP*2lOxnupvXtUC+stH&6*|(=j z+!N3HaJLl+rf;YmKhcXKqtxSKN;2RHu={xLN6@53LI5jL5Qvw&W6^3YVdYv4`?X(u zQF{WIwhElBmT{$-9Dn$J~gH&!a{!hQq$hQ#y3WxQv~`+zha` zUFx#e_Fw$qif9lW6v7hiNj^Gc`IQlV{w?lYSCV~#>GuN@HkW#;l7!~muiS!2S@Ou} zDGNKk-`=3kheRG%R1kx>H_o2_!Zw{G^R+luAA}gIsL1S9Lez_-mwlj4BkZBWT9H{w zRj*Zo@P_wT>D}XI+=EbxcRj|o`*QY+)M0;lBP&zh zufz4OUG_vvV?m|x!(LET#2wuWHdP=pzXrYrPU7cFGxv9vzL?6o>WR57NH|4EyS-ct1FU{UzdLaM{r9Ek%cJiE#ir_h{%j0ox*UTz86d4tT=Z7kqb@W4h zG9f!C=X(Jg8cHD&c?U=`9{LQQ-ZTZAD4I3;Nb#AsQJd6Q;DWK^F(BI88AtJ%3BMm$ zzFh6T%e^}v%6R(WPT=J{UAkFHv-a4*yv}D&*lcSkeUN;%!jf~zZQDuy+Wcxbp>Zkw z@31Lfgs;nM)nr1UIa`&TM&YZ3!-JO5c1cwRYQ$v($=4|%F~Bu zT}U$bz)0GQ+u0BRgi~Q&U!P&~97cHjP4AeG5~1XKk2SgXeac$&mFk}4Oze}(IvQW2 zcYfdB>?v7%LC=3+X2Z=TfT=P$m}b98O%emEiZ6PCw`urMk(NU84D271?`qIprDRgv zd`U;LAgVJ+6!Rlid-QA&CHa4yx7F-pUhPNYsb3K2wYA}es~wOsdq_x|!1%}vANuy1 zLOVx}ka7%)x$ixz=x)XEduLnmvq`nhW4YpH(=wgpR`uyP3YKw>HiINN1xs zg3B`OmR(TfLJb z7@y63*C2EHqDAJk+(LNJ>fv97;kg!uZ9DKopA>6AfmbZDph&lQoBmTg$tnooT3)mmZNpR|KI)pftq`|R+n zJ-+Jo&Y0wjJSK=8R^$WPEan`t;ua|cjK{QZR9X(N;qX*RnRYZ=toxX0?CH+IYrMYz zh$!`)z#~?-N4jK=-ghT}ZSFzN{K%+ys!obb=>4seq&C$=|47i;LXm7Cxf_<~?~2(f zvG`bG#}5}`t62e5n=xbei>3)Z`Wac469Ez1c;(?g%Sd2N&-5rf%Ati^?b9N9FTo8R zu@`5lKJc}IrT^kq3kJI$ZDeD^u^-`36m+AXT=uZ#7&9}QE&A&d{H_4gOh&lQ&;Gw%bIZ0#dG_ zNGay@dd5SK5$)MWv&-`3YPb+Y<@H1{T54V$#Bph;(OAHqH@f2f6L_=REcP$})cCJ0 z3axgr!sQ;=!KjDflQqx+V#^kvDA3D{Vd0wxb|Rga>eF5e%|yU>v)4?8;qwAD#}W-T zP6Hptf$n(7=Q*hSfJPYO^G~q9H3XpY(HV))C0MTsw%nf=z)dr71xQ2ECg0VT+3wuS z`24icNUxADBSq8OZH)IL(-`>EfZ7ylCs!)&v$I<+ImoA6&1@9&g9Hucjvn*7aw!X} z9+ZegF7OZJOA6)9c~~=kXXl!#E9AZe@5b>yUWavetV33cVJG4FgQ z?yN8-+RR@|w(YDR_mIKALOf$tzxlxn0VFuVA|a46w?D4)<$s$@G*`z z6Z2=+x6Bzj=?Y|Ley;k$u^^fsI{> zZiQHjf^hH#uGO=eT{b8~=>AKv`;A z+x1u7ZsDVab}p~+(hBTFL3)Wemw8i9dP+qc0!DmZui0p%O^&G(G z^(Hz6#k;^E2i7|C&k=BP^WQliytEuvu)T#oF9mG)ymy}R7FK*y5zvpI9x&D#tbrhq z!b#8-7>F2m5g*dtr_^c zs;z~786YEo3X5-e^ufE#Mp#L*+mg)y`u0w)6UKamz{aNcWHPETz^y?XGERJU)Gaq} zrF=yJLI1<9t~9PZks8qcO8tp_cDsP-S{8}n&Nk?(mH(4R5;*umZPhtq1gXLT5?G8DpdJlcio^*1Shj%O<2m=(0MbFaR zaflez;m#ypV5!^WXWw^2M=8zua^l9cHbjJ^*M--Qg__U2l^`}OOh@pqh^N@XOk$fSBQIY{|7=CP_WEeY?C-?s82+)x2Fd#R^#Dm0+v};jvFG#5fc3eFx$sC4uSUC+CV~G5h-u;5 z{|uNCy6<+L-gb1L|BG_HY?OyvaKh`}Qws1;5*TE&5llCeF^pZ@WqF`C&Pj`P_eGiD z{JCza6(H(5_h@%@8l37-BeQyAx}!ULVLzz-TpH>A$+imoo2$i62Tdh71GTOo_Pf|4 z&wd2!KsVGEwo36Ik2ZdUy($I|&r)0lDf7_Mg#w+ACk4q|*(k?~V%L9Mk%L)~Zf$d? z^LO8PDomW>9%E$=KvS;TNOcCEndl4fW1+znZs=Vv0-yc6EEg5Vo-f^_-sH0aAH zp#2^=j*ZFw^{NB;=4}0^p-$9L2G92jAtZkH>Zo%4+L2o*tuLThzz&D?;!&)}@z#WD zY;tyWVd1YIcL#Q1D@D6bB1PmH(YVo)AEXQNB|@fHizcq&L&nOoAWv~H+(0rSh{Vd^ z??U|kt6QeywmO5b7GQfh=$1glF&@kwP)sh8_dwN^uD)F@`%m11WjQUiJDlQF`BMtl zLkCoUasZpq^szvKnYP4c!8{{fY|Dp_gd`rhcEUhJ2%91&=_o*|M!Jg`V~3udVYGN< z8&uT5unijpif<=gmfm~T&@sd4G*QSP_fV4d^z1C{q6bnJzYW`Si3WLK>N_Q1+xg7% z$+Oh44+MEqrvdZj*VU|wvvRiTK&!FJ6PeW+uX4ORC*97L9XQcs%>Z( zVW3}k^_ewV;7t^b0C~?kdJsoi%Tg`4d(#1=+aJZ*KBNlYvWngvg`Q=_>CWA-hA6&`sYdLnuz$?p^+Sbi zD@Edhd2!q^JS#CKG6?2*CkVIPSZl8Me^fPvY(T4aQz>ypBN`Wkw zX1&7OKj;G1@D4PFfp5&3%5=}wLnK9rRBOyKQ~>{(500I{${TD()bK;CSaqWkTX^4o65y+QQjFY@vRW_gqp)C40YzC1taB7bmy5PCyne4 z4LZy!xGe4}=~xmH)1S^cfN!bdd(!`Gp-BJVY@voIZ=o;En$w@~l{K8ynvBxwVF3Kk z+qHH4$z}`0KF@Zo0wK6d0h@#ql`lWDl``DJGL^l#u!EVKn|m%eJ4&+sbo7919`N$gH1mfx9v~UMXh=p`^}i6ZxP$8u6e+$YA(tN8RqC?0wgPAV`^J zz6%YjhU~Kblpe4w2R=#;fYtY10v#WZJGo0(-`$jn?y89ND+XC_5K60uFTfrTt#&Dg z*9R|Hdsuj(Jt!Qt>|7-l>O(VJSmp(sU4f4Ch1 zU5&tTJLC2^6sHz(Aq@t;r=P*l*Ny?(9~`Y0esZsLPsQ#&X{UmQ&`|Qdh~?C%+_wR} zw~rTGYCGvb4~YGlEn6$8gtN70+QhvAwnRVxCU;j{2JhPSPQgywU>b+FmHVNz4g^5L zY(TI#SFIBX8aYORdAn1FNa{&)0Ho(02>iGb9_tKvnil$o7DjssBF)hr?bi=Wbqm6> z_rvX2^27xh$(di7&r04$xOUj=9ZR?n~DlnwIdx;fiSLEw*8sNcdtTayi4jJAMqeI4j49os&O^m@` zH=r|3B8*6E6S_vHk8N`jtf+9^U3J=U_P{adt&TI(cMIs7vig(pyEh4D{lmnUe@D1v z8Rz?1t-02Ya}(l?kBXun|GK5Vnseh|ranA=P=?RwER`i>$06#xKn}X}#OB}|7;kPg zAU=pV$TuA>uN{0Hj|Mr%`!&s{cyo4_vYsLb{LRM#u&_-q3Jv;1cj6$RS4^47MI z9yV_htsF}t>#PKK_sYDAx0by#t^ByPdlOso{FdIW@{7A%&`a)DiFlp4<1Gbmj(Q#l zscn{g`@B95YkzdqXF+)9u(#y5UyVEz`%@NteR>R9_H~+q)+q5*{9$%NN1#7jjKoR+ z*~N+@W0uH{M3$%x3!*kbeBiMCmf=lered5mPkz&K2wl=A2%kmUqa>+q>VT`x81Yz$ z+ds>YYYL?LpJJ6}{n%)9{Ak{m;`xHlF zIRJm8K!)++@62P>BqZLGmMa#?hZ1JDc`!L73P2|v8PRRu^r(*u;k+o_p(EEweesPk z&p&bVdiCWFNAuuyyb-)l_nFh%C)($TfD_x5k>xwEN1_NOW%IbARP?>cInsBDwtsVV z2DMM)axjjKB@26^wGNesib#&-p4^^Hy|L>m)2}-T64e|*moPNapv~P6KeUjq`I1Ck z3FPJYp67%(VX2;zZs1x?QqvPAhU*EE(XCR&OINX(SnM$L_j`=-dET7t@ z&3qn7Y%(f!5_EM44Jwsi=||5qmV_$Es~mfme?XDQzf(#vXq`4oeH@D-wZKO!`4Z3|?hSVKu;SU!P9SzOHS4ln{N5gd$uHh9s78qEhMSO}BOK*P1i zY$HEtOqMntVt7&@;wav*^c8FNmzN8lCpj_Tzf>=j8-T>sE^!Dtdqm$JBH$8}I?~r{cS!+Yq3^LC4O_YgwCyl7 z|NAn$mWn_TdVBMns9Jn`^{Y@?PG_;*4*TlhZ2b7}d+%{)9dVIjFPcjqVt)O-=fRXe;JF=kk;%cj`phzemx^EKHMbNf+0B&Oun>vyVM3Tq%uBHpn zI2ea5X+;B^#cIg$;nA(LV25Mt|AGhJgRXm@k5LX_3otIOlc&dO=4^Y-662vfSRC4n zdAx3Zw%~I+E;8O#a%I(QE>fnDx8Qs1wL%%3xs+UVbhd22!}8?qZ9YcEXX7Z`t4|Ym zKGNQ2r)xY*$;gd(fs@ZgF<#HfUH>2rAIlnow^v01b^>0|K|S`E=~mCxr>8$1-P{vY zUl5h*UQ_5FKWnCF*zldtLfixTGqT%Xu+x;m(~Xs+IL$mXxh+w0aGUz!Xfzz z1WVe1)UHT|IMT1MKQ_-d$8+2!_3}2m_AV19-kKKm+zg~uEblWZ%G4{Le-v`RkKxQc zqc8Pm_{P6bIqOksG$NH9$)4u@6dGJlmH@*8`51#c&xh59_~X5S*+3ZSR#E=-YF*&s zjX;l0Wk7p?uvF&^<8xRfJ^MFwDNUw!W(eB2*6J~1$wz^gs_iq2hUsdw75l4T0!9~a zz|L&k7H?|zMa2dU?7O%5Rn)Ym9>w@Ks~>@NXJDthqAd3z*iMW(ou5yR0{+f>>n_Ds zVnEjI(*8+0{^Bt{;T!4%y?tWf<~5PT5ZZ?nJoZZGmA}GPKKaS4T>Jg0C_A=v=C^1F z^#lFQuuwNAtVO$U)YM;<9BohvM2fjKRiWhSPz<@#35>yi;aKZ7=HH6 zX*6qkOuzD(vkhX8GZ8cdhI(MNiq1#f)>^hwmUK=q)6-sJWB#9B03%8x*Q>2Z^5XeH zQFs*1ZN9_rlOz0bk{;itW%F%wo_H)4$@=sVc4B*BBs09NPM|aURXH144@rn>{JW?` zwWhk|Ka;9}}&93mE2?=>GCj>m*y2k;fK1>CO2xS(4N)1QP{6@Pex3 z!YcbKt)LcoMI*r5%Vq4*9Ert``-rE($-0SS-TeC zoI^@m+J_LSnJ$A#v*ER?rpxQqmHvVSSA5;hG{;BnBFZbmig<+Gd!ANY$1MsIem}6> zIut6v5IMKcY15605@1_dfL;v!?nikddBU%n5i~iSDoe-u7Ya@g2NVLvt8jRU*p-Iv z?f_R5Gu5m8+HiS>;yR_-jJh@Qof|_l_+VS=UzpYo0-T=eHMv_}%_0~)U5N5!#d(~t z;97Qg+$RXN^TR~=wQ)E}L4rDwR0>b~$r&`@x|U%7BzIrltt~Bntm4I+110z24K<63 zpuhr}$6}r$l@Hu)|2AwUUDbTuyUN;e1#V+jyi6#Lu8N1dC}e1r-NYJ2IduUi0?UQS{`}13S&%qdWy5Md;4ctpwKeLug-t=ZRjB> zz1S=AOBu(crMIdACnW9c*QWKsSRwEQP`73VX;oBlul=x?&O-ozO7P?TQus;U+kg6T z46+v(H$zs}cw06gv@?M#fpOU<)XF-PA>S1=G{lv?(!U3nn^0K70jMM4QcnE z+R!fYRl&L}l8S2)O-sS}s~WR361?YYjh`?^yCmlU`<8BE33Hd(n!mo{Wx#~{ZXFjY z-fuTWhvj3O9C~U3fjxn{xOnyQ2|F8(D-ti96;n;)maO5yrUv^*;dnrl5;0#YiW4yz zwl}%y96p-pROZfwI{Qy`4 zV7m9HNyUIcGv3NOf4s$K5HnZHXnp6wDfWQb1w6Aa0o);M>n|lYV)NU8_wkHkX3f_3 zFK0^n+x75|ID)RXOzT`_+a!L!m9z#h^e)v`BBc?lZ;mG-h&$%r-k!FO-8BPk1Vhx< zL46{2p!S)xh9c=a~1=F9042+LQ39TlC-6@+ScMFx)7F=o~Kufjb+vQ!N zuhTlRgbW?6qc{JgXm0ng+!nt@g9U_vg%f&7Tx!{&_mnNH8|6bcg%(JV{h9n#E z@uLG!*Hc;o;H;$`pEVjT|F%MD^y9Uu8DQvHl6JU}Cj4v7{q6H`l5fbb?_&JVf)hBlI9WMPFeU^NBbuXXSdEUWB^~IOdy`a;Y?@HUX=SY3O;Fx3H za;8MnSXT(I@le>ED{tG`0c7GK<39ZjT2B&{Fm2NlyFSsZuN1J30HUf+&5xH}3v_T_ z1?=8=^Kj<-kh<&Sjpq1*>OPu~!GB`Z;r>-WV(FAKPTXa0q@8Xr9v9-aW4L#OwpyD8 zAEd5$`H{POPHyISY&@K)d6v^)d)-<3EN{8;i_0tYbm``rQCh>%z|n@sz6WkCf^G*Yg&dCm ziplPu-QvZ{9id@z61C_0Op89I9S^UP{Y}vppT<)5Bqb;o*00pI372GB zxA$qr!CunOr%C&ZYQw`{X;|%wsn?4S5Aw82=2bg`OroiR!Z$X7f3zRXa<=R#pt)l+ z4Lx4Fp9>2uQmFz74XQ@Hzr+dXIJ#JDNd7a{)8ik+|M8G}#zD%wm#XZm7{3?&4G&lO z``=Mk5#%mkbE`Jur{}ZZOA@`MAUD!FRjgHr`)00Va?<<%|G59(|GEd2xiCQPU2#8f z@p7)MgVasx(ELoX*Ioy_rV6tf|NAwykUG8mwf?GAAJv<)^OhAGW^sBl*e}(hvB{Eqt zBt-7_rp)+zf$qyr<7zyL;}o5|T?*dgJl`wB8lDg*1+~<86rW`a`KzB*xWnL>IvY9Q zBFEP)b5^)!xD* z7eu%re-!>4#24~kQ=;p?wn5Zt(}!@udv#Y@13Q~1OVn97_q7hRus;oZN$Q?OqkFHN zF%q{S+&E3@=YG-E*msbrEB;6)SCN!z9BlKyuNd%c|39yzXL})ZC;IHmFjtsL9_+*U z1`Xo8R)p_B{k5`IhVuP$Cg%@ueDN-o|Fs6#{~s#@Og#FstV$I7Ws6HqrYP8n?}g#CYE#sd$(LcEApf^fv?d9)8S1wq3NDT8-ES3%*GJLupY z9BXPd%Gx(~qs0ZfjP2Jx;RC&ANZ*#Q&iOf9Bb10`$ahsPd|b!QbN_TfjH%Nz|j&w_xf&(H-H-Zu)-5mmgbW4Yn3?Z!&(w$04$-uxcv!BiH_x#s7 zXRYVWd3n}3FL=S8d+(Xqv-iF4>$<+z=j+g~@oN#{nmWMZx(}pY@C)~2+VFS`addZ?QQ!A$bI$#W7uLR*$QcXod$HEUJ;QN6ab zoJv}R1SR-Re9b_*V^!&vC2Rbd5Pf)yBMkNW!3g8wd+jkXuiX z-H&<{|Cq(q=NtL}*#f(_7JsnD-VI=3KzUTZp#~e?5SyP!aPEd^;OktM=7>kW^a2`* zeb>+~530`tm%k&VGDda`z9syG+t56X4!9R{dN6)DA^S-HQ@7{30^ihY_(U@ju>rq- z0bLw|*PMRA;QVXF15PB2c)AbX;-D^x`fX;sw&@K&=8~JvO8uG7Fzi2?RcrM(-Be&r z#%G^}_bW+wwKd zX6n*DDi9oZD-UnQq<=!lWeMr5c1V2(*u5rtJ;ou&S_yn2$0i1}fYf>|md8RVKwxe? z!~SmlgW^)FBaYvuwIdi!dqLD5C@-5E2E#-^@A#*iW0*a|pF}i6z4tngjNbi*vPlm; z3r0)5IUE;%yQzA)qth2l-g`I&pdc9WZ;w_wI>+uEHfNzeV3aF<7?Y1xB3|RIqUQ+U z@3mQ~fiBd&wyVr%+jgYEIbRL7uVA+kBKn={6_4yJ(0Uj4>YV+zlqlT*ruhk|t7qEs z6L}$|9{4R}0crnc>>6VnGNft8?7u*weI*d4re&X(I#GOM`BP1h07bjkMq4c)+6+A7TyM>2V#iU6mk7Pe7Fff4S_ zFIv?qN&~~k)Um;uG^(+Wler2w^m5M0jf^>naHoX#i1rX)-A8{O<$527ZKuq==Im^0 zYU+FqQJqH|G(@!JU(lNzhdY9 zO2W>ak-VKgW%5kBGJXC`Q$x9wEKF;}SXZeIlAvH%Rz^6~Mw_{1Z5OZ}@b%u+ikipf z2>-xCXD*ZZ=Dw0`!7>{ZyTqAY8@lz`hEyP6ndG)&+Y^lH#USg<>z@Dj3Sl58R#lGp z--kU|ihZhLoYL&+q;jCKpjpvnD=zppR?q>xxC}5LhfkdAx?_*q0*D6n8437vJ-Ark zP8&X`W3jsl2!5@PBgxTy{h%4Bl?A_$Uw=1$wh*4zc(y%F08hH4J zl3sSVbN}Z0p^Y`M<1p}?BiOMurxw+PciL-J}S~b!2T$iJiJR;qAJ%%6X51VKF z2Vbh~>|^sI2l?Lq*=t+AXx}-d{70St%;oc8M{kcb4Bnusi4K6|UX_B^Nust%j{;8S zObIBMNWpjz>*k_wt#8)*Bc-Eu4x}9#YdX*#tNraxKROkXgVO9(1KYdu$Y)DPc;~fxgU5w85KzItay;DFksH@ zB#a$R*xKK|=sCpJel@LK)C|>YFDaN8*9E>Tdyxb)ZqWH-!^x|ubDRQX_3UFBcT~N# zlp1WaFs{}a6VsW3d85>&_Bmd++e|!wfgqC|1axNW5)%ZZkuOaP3ChU!90oobrKhL$ zD)QhmsG#+GZvU|O0L^Xk#@LLnA_6ZW7dN1U6*dR?0=a}R@zZ<_LjhehKmKr=+`cA| zOGtZ$7@vc!0rPrKBq`PaPI~$r$-G!vT7c8}N|132gDhzm(vMCjiga~fa)?N4_-4oas{Xte{;XnGkQm^cU+Y=z$1uX6YXOEnA z3;cqBJkK+A{WIxW1#kG#2v=lY#8`Ry+{ z97G81B}dxHn#KExA2V^;$7watdxy|!r!D4Rilr6$s}^8yAUiSb=mv>4fYaFExC(=1 zJVk*&gcd6-BD}iZI;cB_Y@TVA^#+H2q3u{ZK{H^7+X&nZHHxjC#0Xob7n<}sX918K zf~YN2V=xYTz7+PG1=lacL;2)BZeMrIZmzeTvL5=8lcILH3p9wt*d=l4gIgH2wP%&U z<+tqz>o*~yUlytH&)M&M4Y*Vf(CXkcIM}EDT@bkmWr72%fm?c6r#rg-{}6MwKXdP& z4E)(Dcs>59-sy*vJod2_i0;zgw05yB)cNA3Ug-T2Zyn;+=kGApOw{bSg8zqszdmvv z)-nfQp1#72T|)Qo7CUhi%N|3vf3}7i{_7q)l!$-N$*sA=+bpW5r)UW>o2H^aBu%$U z%hbHI zWu!Pc9sPg$MIx=5UXgvzQ)C#g(zYGh95|Ut=o_ryAoF90-)r)fd5?XX4H zT~$N&QddW(#gRtcpULW7fgm@y(vf#e^1cf5=~>ERW?5Or|Lxm1ilj-EP>Hcuf;&C^ zKkbCzii4;>{a!S>AAVIPo|+O}!~0Wi;d2J*mO$So_3w{;n(BQhjRJ>L{pgo_@9VWb zzqDbn^(6YIXUoG)h0l)QhKCjP%_5pEuyT_$a>$ALzRSm7TXYwdod&f5{`n3T#5T zYq~H}Vk!wM>P1#t`5+5A*JCs0%s9vfz}F6aXckel{_fC+q8M{(Y8bh-jEI^Y@zu4-ju6c=&a0wH~M2?z3n&yW6wC zwJgVj{19ZJXBm z7kSVa*ZmYXAn3K&U0C@K*)TJIfd$97ABFb+=oQwjoa&@$>17OX)1^h+K zoje&dTX3!l#~4ScDG2v1&TrKXV_=@f7jYvhr*R|A{9<=wNm~iy4kZ6`Lg}i2DgS-p zH<{6zOqIj`P8n2sxadaX>YWyEw(LsUXxG06ygR6m7j(w|u9C@CH|51&my%RMy*KL= zuvL=z4oHL_|cFs8Philz_$im8hWG?u@vMj|oXj>%3@ z$8aYMChkVbc7mxk`(my5wf=6RUpyLnr4wnlzeg5tCti%df*-# z%31P`T=Ccbw1GDxnj+N%Aazk(#Il3YOaw{X?=v5SrNV}*7@QvG1@EPu8M$O6-n7X= zsY#2Qd`AC9j5Nc_^LqR>asdy;xePBrO^^cr{q<~-|Q1r6yoGovQJ%OCv zw6^i_m?y1kUsfj150g~Kj*wct<&b&tT#&NHP%!Qm6xx(1 zO6v-I%mRPj9me3hUFL=N+m6s^hpDXjILk4XIyqcRq% zUneF%O_!b?cj&lVw0O~7u6;TFbhFx#I_10k^LfBA9^w=mg*c#?h7}L$o#gDF5^(QI zESBlzaMn>>{DYDEFWQP7EHx~K^^=* z)U0oEX}2uG`Ms`>hD&-X)u$a^rG(AFMXO2rv8_0!^J2LCf%q`cemmWl0z0D()9@pP zs9`jEN6UrL*4`844HTPgVdVTNps+wR0!<)B&wPc6OQ{G!iRJZX?VG*-mD7n~e>Yr+ z+z4G-|I~RL@<{HI2qNrsgsCH1%}@uv>tp2YRnPnCY>ea)f652*4Zar;{+YT&#@AEh4`oZvLDcy-%aS|Lc-DrG9XNHQ={nH%ryeL{2#v`QTK|r+XUy}+ zc#Y9zAeCS1ATuW&jIb1mqfu+_O&C=blOp;B_7&H2uzJwjGH3nqFnX}kjwf~KFDA-n zce0qE+4T#N%Veb#dviN*^$I`@_b2t->dg8`#9rzZ4PU}WKe1G4FV&++10|;u)ja&*2Hm$qwC-qg|Kr22!S*8_da-_ae>&VgQm$~gm;6e zT~oSxbCpl=1*K8P);AQ$^@N!w7J$#H7v^p-OLl6n+2_$a82-NS@avf_J-qCmH$b(# z{hX9#2QCj3NVVq`ieqrh5cNhIeYi!sCy)Q^~O99HlHi63I%>GQwuNwIvxF}K^|^d(|^C=MRRKs#X~*GV^KXb z=GJV!V`vKHh7b&YF-+|upUO%Fd9CqA=)DE(_w|=kguXc0D8up#`vWNY(6i-xm+vGZ zbTY+^C%A35KpgmOQ2wHgc~>8@KPTnC%uO zD!mUaR=j?YR0lugPJ?l5Q5s}@5FH-5#`Ir8t2kx?fZkZ=qQz|hbu4!$7>vF=_A4P$ z+^UjD*%$jFf=#jsQ2ei0*HLZLd#7nm^~#i#q0gr%glPFr?{%HXd8`&FO|Yz4m-lJS zqJhCPK$2u3%kzniB1XxQhNl+Bq61KumAo3k{OB{>xCUE8%HVe=cqb!pDb&pSc+}2V zTQWJg?@Nz*P~PXV_h202)BEB7p57^GJ%k)Gkx(>`dLHs!;M@(!vD zUXHN0_o%-XS@3H0Ad(CV1aNy}B*S@zsEop_YFZ$7n-jat5@08#DmD*-iT+(NJdY0Z zLGug5KbJ}%76az0uF91rHKmH}xyL2ug6y{wJSeX`5c1NgP0jn@K2Qnth4VcVjBz zzwz7!F1WHQbgf1NENmrOIN9C-;SbRyClXz(MDawXQo>wENRlVW+&Hm(mgLjR3Nf$m5r9s`R!^tvk;#w_Y@O07~w^tf?L`8FXEN>Wbe4xqlT zX?@w7U_(Lu4v|eaz_}uEtum37@u}%sA4l`IH3ygx@1P~3M3xnXJ%pg}E#x|{V@ATS zKX1B-y^szJ*5`RW5UxoSD(cYp`CF!IYxjrb18r}vs2}qT6NcAFqBYKynS42YHieirx zu}MGi2Kos(?j6BQ?=J&xHwgX?Zq6d0%KE|&6nfmT?(hwBu5FhkpGS)+r9fenbe2Ey z_%9)Nn9EV&y7!Qq4Q)_UXp1F`_5Bi}`dGTsxcRcp*eOloGD6c zwX7#s1yW|dMd5XDIyfM1!@w!r4UB#SP)F?qtHFQR+Q3GJLR!$e@RM*I&=qSwhOyU* z%Q5))8(7+eI}K>JxeU|QQ~y$8uqvqj2w-?z8%WrkDGI-O&cfVkInF>?1;)_tVhzdY zXmQWs`&)hbc}tWZlpLAz`Rw2>7`9j-Z@WNZB5XndOn<8cOl=iiTCi z9l!+PteST!w3C9?xy&mS)E_KavcU!y#H&h;VOIolXHMk>A|m>T~w&;iitX$DJeOKDjIXbQB` zL1EPNwllpv2J`|UQ)sbG%88MyHBi*t_w{lGM`MdYJBo}Y)9~R>gO(F9+vk4gb=tgWn@6N_^ zjDCA_7McjVLzR{g3R^jaRRyH;wNzc74h?B85ElJDz&+4Qw2C#PoKUiOOIMhlIO3Up z-E0(+MPHy>Mj$*)oGFL^r^ZQ1!@7#ZKIw-4#sHUa+tQXX;s7Ej?3?BF^#O`v#W-2r zLIspaV{fALtEml@s(LZiYu)Jeh1Z{Mt&V#~GlU279j%5gVTfIN;x*vq1q5_vh$Oj( zq`SG6gNFoN?(yer9zbXNE$!+w0rtPre{8{L?;6mKp~22@0d_MRG>A)fRi8#si0CR) zT27kMlA5-d7`qzB{d-2={@S#}bQY!Y;J?W;U+&y03aYa0Wq=aQ#0H_Ms)vU<%Bub% zuRr=#3D#L$VQ^Z8FTtXiW?8SBT2M4HYrh3^*JB;aW4Je}=RFpV&1kydBw3Z-LCc&B zV6M!AEphZwMuGq@1h%e8~tU1u@-q6Jjkd@6e&o1hWZWvei{`T0lL|p4&~ng8}aVs z`b>>rl`4n>M2sa><2OTNRlFGfM1FTzWx>-8=<6G(%r|)VZZzV%Q&u|P(Im720rkTo~juhU@;jJo-t$C^eKsXtGM7Y*Kf|2M1ZJ{qnl_af&RSAA( z%ko5Ve|;>?$f(ADB48U}^sJ9PGiKW;W#U~rJ#ibFH!%H0y7<;LKlAprhJ3{vTwMWA z6bU0C{Sl0ifOP*=w#nB*6B22)nkp$2J9e4)bjB$(6v}7)r4~2w{>8knASI2oec4DJXa@o` z;*=sz{h0FZ?_U(0SS@ldH76Xl%p#>EQr}&F;aOR7c&ZOQjMB`$+t!e!p?u)o)Uc!} zt~ff>9Ya|=I~Od2z~DIDM$#=TCUFk>nic4JP3g|t5le5DG1xCd6R+9yRsh@@Pay3u zK^osS<{~_N398zE$v*j4Jv1m@Df%BZkvT#Gomh4R!**&}1{fo38f$;R`qdLQZUw7) zi_HU_%j82E+U1huzNUcAu{cKhh}y&o83Eqf zw!HPg>9k}ELZQmn&NBEJo*KTV zH`-3XB21h(hZdxbM{isV3D~3?P=TY|V7LZ+x30H2v6P#?LD$=Qd|O&+!50-ss{X%8 zhCKp>T>f~|%NW7}U4xS8BWXI$Z$JJf721&X2{;&iGdSf|*K3B$CqJH^xq=_Bc7g_{ z7g2UeR}!FzLb|~4@*MJ&C7PomU<>g1?jV0s5N2~6bP)*(rDoAknNy&bgCsTQ2!^%bK^rE;R<&iA$_-RBqu5QzF5%sn?FXu(IW4l+H2o?{P}r}dfjQ_T zD-<>&o=Rt584HCqp?AFL)?mKbfO-N;|IP;}c@!BdA6%jt zb_wh!kS)@-YkhNYnDul1KTyQuseLJ6;Xo);Dj;E3pNyg!i3$5{9C{4FCQnxlqIw=% zorL*AzVuzk{gip<`q%c<9V7Atj$v^zx)M)y?Ipcv@r*_#qn#4&<9wSD?2lfzSLb}i z5;P<`c_XVJnD;*GC4^8vl-s6M-j+UN6fM@9`oK;04`vQ5bwSUlx-kNT+nlj@i0Y9I z$&ALr)KS|vI5+MMcbgu#%K5fQpz7TI!4~X4fI`H(gpjKdeXq9JgwuZoQe$Q-S^vR& zQsCbufBoq#U#`!Y)*$ZwAo9n1@0T-45I?2HFS@@OV-X4xomiN zquHTxLV+hO-k2@hk3TyRlgbBUJEqtnHmTZ(>FgBAloT>6N1c7jI9obC9S5?0u7Lh^ zG>L31rAlaujpHd-GY{iJ57qrj6_Fu!3CVTA7m+*HLmgWe{2hhNp#|cCtuGM~8^l?> zPX$ua6uxo z2K%hw1}kEz0%fUbFLcbHjJ5AAiz2AOcgu}!g>$N$2$%YwOnq^Og7{wliJj2{c-4@* z>$8(ti{A}JAgKKm1~#2|e3uyNnVGW+gR(u8SPNSD2(V&jxWd|xNE%!8fJPkYZb@1b z8O$m~Pj%)R_Q1vX4@)!Gehl@sCrJF%~l@xhxJ3BCqXo(iaKY$$mF*F@lcA@ zy))xMv{(q$$q7QkSKSy(-86gKOOwk@P=2RIcw4jVZ=W=zd{NBogz$58+o zXznv0=2KPb_*Ca_-M4+5M8inSgF$%Ul^1e7jADI$D>0<$=Y2)i=61|MU>via2xj-eBh}QpjA@^U|1-Y8nx>dN`&7~Iy$J%=qRWEs(SGU^qD+KYFr2QBeW~4 zi1D8cmw6pSY?Ug?*OB}pR=_ew@{_7m1=kUblJqA3CK?4jEgLDXD#E=Tc)RqPT`Rh0 zPrMAgl-82EEl{TZ8vsVEgnj~iOD~&z&~}f>AMM(-9b*XAKLudO!w4v9AKz4`0V-q_ z-2Y&Uj^(}iKp@V8t8HVC{qZLo_x;fP8g^)`S0_L$0bL-Tz5t&4}}ro zLBZwDOhjp94ZNhAk7uc@r7#f%`Bpl9)Y%UK5R~yfLs}FMPB^l}ZIzczDP3e%fQ--2 z2tWi{&qN1krk4xD9(+wZ?D8Im|2%a*cKbV>u*QCAj7_!f}#isEkYF7(zna@Lo* z;sByAXJ?#u2(62qlGcz}!vq3svpj8craI8~n$cBjUcnThTU?ju zKOx6yJeC`VdJCgiNx=}oNL935taEB$^Ly`OUn^kt^D~myxo`2$57sEIFP9kZzC&`h z-5OehtJZA472mnCUXySP@12JHSOD!e{?)Ia0*2(??bLv_ZT(aXLRY@L*~`2rQDYcu8AEEsV@A$>q__qOzqGrF=*2d#IN~A@)DP7n~{E zo;GCnz+{BBBWH{70W6)PlG2D}v2`}{QKPTPAW8#?{WB3#nR)Y@E*Y= z>6P(mLQ&9a{^sb)pP*P-s7P_MK(re}&4*0N;8}Pw*s%=YCocn4VwK4V{`nHVZVUmB z*TozN>_84Mm8<-@mj_DE@q061ZP?v@yO)Kg)tMHY3-#Bsc}@FAz(;8}(r4~7B%eTe zfhpD#2C?U)WX!VRe_NOC#gl8qgh?sBmGKyswH6wO zvUuzVr1OCJL9mo%*ZFh_V(58*XUFwH{#Q2yRnfb>MQb4551dUiIQ3G{Ng#aaX}In* zlUY%LDdPN;N09p7UPi_yai!bm{4~4e1{}dZed_`;TCU+or=)_zS}B+Mgxel;k7?e7 zx)$SUQ{gA zrd~-rYmD~b8VY!K<5)$EJ)YuZm!p#LZ;c&`BShV1WUu7_(wpj2uCN3mn6&3~2%ep4 z$An-3Ic#Om&c5rE7&&4Rd}>9nsk3UK-YIyqYh6d~&eb#?IlyPE6*NKBpjA+U~>Q@#;k4?Yw-vmYTQ~=B{+bFfELPv*T>2#bVxADx)pF(f9ld>!Noy*0 zGm<|-S1N2$)v*P*-i%-G!Z!;l>tH2&nM#w+{sxFBHw|5`R?6t!xC&{b zJ>L$fu?C_!YYfOv5p%aOZtPK-ZW#I{7*>ql@oIQdj{@otK+V&ZP_Q?a?(Vx?PI*#x ziH9r{3m1#dV*XeZ8<{m0F>>h4z680|ESwML5wZt;^c6Fm_>nJt~0 zYk`Zej!}9W?{Ua&Iuz<)VAoK@FMUO}y8(w14s+cPZ!X(`zGy1$_3ZLbp}(G>@K}AP zQzGVR%5ygKg<}DX%tzm9pq=R)9d!&bc~}Gr7|K8tXlC6P*nO8@hUG0P!TF`ea^JX{ zst6wq)`m-yrcQ{It8<-PT~bW0!fhffa2;4-M(lHtMlMn(4ogWPl14RmlYk0x{2yLU zJlJc0#cWDWxb9&we}RM!{!RK!7$&5d89y58VrOll6e>h>d#n`MbMvISG<2`B&R-trg}*!!lWB$tyR3 z2{_*+x|J?{&x(n1WyC-HlHV@x9Z{IJ--x-{tE%4UMjdZCTZ_XUu?C$_PSgbWF~&lf z*~D$`oDn5ZPMZl<+o(;6yWDRo{hr)1iutkYuuUCqsdoq!6Dpli1DwKuI?O-!TJc?b z>yIG4)+9l45i4(E;r)Gt-$H25obqRy5lvOIO}%Hv_Pk)d?<-n!8D&hu_IVIx>195Kx?5_G&W5?gw;Oxk zZLy(1!`K7RxTc$#d{L7buf^-G@NefrSzTXheL+3D%GOdrI**FZYiPNWqcak|x7FxhJFk&C@8q8*AVjpTHXwH*Kn4bQT9#gDK=xYVZJb{27B{N~po$3H+;? zu}A>#)w#8G6N$>~1NRP$YUNBPV9YZ+1pI)uQPzIFP_SrsqN@`+v#@z+(HJOrbN(6< z0a_$TecRWnTDlHcE%;{xI)FvBBIRF=A3Z#IT77(z#3Y6PlZYUcRLENNuGc+iH};c< z2$QUb&h$-8&bg`sxmnugayP%sR9kAVH^pRYL#Py35AzYR-n!>1K*-x2BYuL zt7hKg{CXk>4=!oX+r``$h-}!p^R#olXB$Ap*v$$+omg*o{ihKht1}UtTxw zofNa9DSwZDWVmE|^X8wF*jr)4`=D~tt+O}7+zRWN&nmW9;TdXt)1V8mWhVAJ zCJ5-2M`B29D=fiG>VWQn(J=U|IOxRGSTtZSwyxlkqcFDO&Ocp1$q|>xH^mhJY4=&y z%o;1K{CMUFc0)$O%Qb+*rAbc==Syg!2(N@_1a;N}d#CdjaFMTu=o9WsdqDj$)}Gm{ z<~d%|>+wO6%nf|#rQdBg^gvv9D0N_0s1Ck*^zY`c6s!$^pV7_ zk^GMf{X-_H@_r(IR*HWfR$7!=q#s>71bvQ{@f2JZsnmlciOJdh+h}b>evC4rFDbbA z4REb!&k0Skk={zYb$};zMvpVa2X=YRp_utdkJR*Wh%KLg8)i-fU2Yi{-YO9t-pf#@z0QDCS%IXI?cqQ#NSQuvte(vPB2s~h%zu4r8CVv$i0^cR;Lf3;ugIE zT*b-l=yt%i)F8qG_)ih5BPC-8&$WLJ8mf$a9oLpfsWE54W!abRlzj{^=0(SMEG2Yd zByVKBZkYULCJ_U0%cfT+Q4mZ@iil`Rf2}W4aTjFmhiYKZ?u>G7tlx(ikg$_pf|D(~ z#8BtjZf1(?{eEF¨mMq+rW&WClSNrymM-w3OyL3w~zT8xhpat3dHg-uy-Y>{?`d5{Zq~Oo(EbB?YM8=tbv-TRiRelm9vCqv{usgB2LFUcq^&rc4m9fl8{VY?;Cw6dE(=yODBc-(kk0 zBPjk7$o=!Bl4WAbSeb$vvB=ner%NKsP7e^2V(?xMe4ztS!evb3>aoZ~K`-G)uoWP+ zzd1OBts;ha)GOX9(^;}kGsONsm3{9;y@2fOJ8jMr0eETl5~j98(4Qi+wa>-~wCy15 zaTDo%f1vSUf=D;n&0odEI`m?ZJO1X+!tfJp1gAc3C|Qn~)tq>I#$lIrZJ9B1VbS|2 z-K*Fn8}?iDqsq`5HmvatWOa^@u5uJnp6n9oF{yLei8tFrh8W%MP@j6E6$ES}G~#4t zv+kvc|6l0O8)~yU%O7nS>9t|h52SM=8L8btD2{+3MVY>a?&*=@rmxEy76``aX`@mP zcU0V_0@?mw!V*E}s@5eJoF!Ly&|&|JQlF6EKa2}1UEcr00%#X8_Xb0m@lTuf)Z@oj}Dz*y8On8F(f3|N!r?A`l^&0TVap_({4kK0o(2ue+{w4cMIR|be zfix7B2H5aZvugK^-YzN)Tq{;l6q9STsB8X%H4wFMMJ}aqZrSBe)a~wi?-BMp$mi#s z__rnm!RelNS8HY~Jh>f~>#1B42o%qEU)I$GY;46pdz=IujQHibhOo2QXykDGJcLjy z9zrab_v$evUL{T`fNPvz!z6_4;X5Ar-boZ3z0%IE&_PA$m;_2)k{nr`Rj& z`vCnJDkE=R%FA0Xc=u(i(*iO3+6wc<5)3v7-08AQ{9WsNFgsDNq~fq#LN#5t)&qkT zEp1v>@S)fh=d@86oRo>32qmhu6pL=4LJvg|950=>Iw}1t+?azIIK7FZo~KTYCV4l_ zm{Kfh$QU299?Q7r+K`jVfqx855DJZ`piBT5je0X|NqIq01Yy#>gK&*Iae~T8r`(|- z{hEj;#+kjF{+F+4GdmJt@23&^G6^Q?J|u-oeF%LY6`Q}l_C_8o$A(=dmRkBOIPkIP0E zp~I0&he4C)ZOjTN$UK5svJf*c4yrc~&gCon6Uu;@FL0)*%jp1& z)&?C5L%5&KY6*#n)LLHXCvo3<-WU&Uh6F8=`tJ=e*zyKj*?fzpLe6tOikxqK2$n=H z3pK{?PWUBtPk@wFh#9g%c6l}5 zaB}hNxCiu|Ah_iQiDZ3w4ixxiIC(4c}P#0|jRX^zQ zpIHT>4~F2cuAq5Z9ZuHfPBp-W(qhW~uJ5){>!Yo@s5+No$V8USkHyq~UwfbxLW$M3IHto#K$~J(g`A;Gov)MeDykVK@|5XVgbIr(9~k2tfhO!qi<>!2Bv> zJji}4Jzrq);d?Of=2LNLeNA$wpx>~x4ij;MlHiU!x4;Y954}&Y*B;yN8%Ks-ih)IO zreacqLdG=7d98Jp3f-Vl1CJX#f}#A35MQ2R0j#^aqc;2@?gf$vL6u&FzJ~{R&)Kkz z2xz}2_`mLgreU=LESM}bBx9-9drL^)P$Th$jPoUFtpMDprv4B0>w`$W5Da>o9}?2yJiD&AX10Yw z|L#Q{f29C}XR>(ICdmy)f7wJa7y&Ez2>9$R&Rnjo?O$r7jM!#^a*(iN2qCUnbZABE zA%t1|bvnBMOd@J90k1=aSMo8{0o~&;QXx{MY9qZGVeG7?!H>kHWtZ$d>Nd2hTsXCf zus&s9(Xv_?6|pIsvX1Z2Uond$RKAaYcDjns5Zc{CeA@>{UKM}L3;0+c{ZpOj5+18+ z@}H5nrLWdc3f-8l^G%(Tn>eu^kcb%dcL#<1)Y5sxJnC2PTa-js-^)|Y$@!xl(6dl| zT^x>rs=fGn;|>;T-(y-F7O*?EHWnL(*htd&o`idD4fT?UbrC`ocWAQ`{mei4PiY}} zr(M^Fxf9<MNA06xS*_hX# z;Pf{Z-H~jUf0h+5!1~Lnq0I0x)vE%%O3oQ=(J>T-gUu%L&$)~c1E@{JRhItqnT*P< zx4#|DlupPHNye=`fO=*|>jA1$RrXU%$<0_2mw{v~HGg6oFF@tw1RhTPThstd%BPF) zP;tDx#_*Sm#hXP}H2{`iC}0I!$v<6jcJDi}?O*@!=M&CxH>%dk|4n1|2m6TJarglY zuMYn3%QE%uT5lBVXl>5<#qz(O5iFQM94mBr^~NlG1;6uR6)#7|{gtTxu9FMwg$dzv z-a1GG2WOO6zeO8o5THjxYyP0j?k5_Wa1KN4O;VFr?WX5YYUA(=+~JqrQlCg(ETbQb zoFc53O6?jQOeni_e?O{nh@_D}f0n?OB*bX1XOK7b0_;~aEbV%+a}h2-KI+1JpWg$&P4F|f}uH_gHuwe|GGrsTXunm8!Q|jI|EOHW0P8N z;>8zumr-fg5_Ez(G<+DB>>zEQ@^yCvTJ(7rzL9sO4M-@VblR|y$8EhnIFN|$fV%8M6j-Xpcm@ag>@ZVE@t(1AcIPWm)5w2 z?Gk>!yy&rQl|{o^ zRIZ(iKbR=#zTH>_*GqUWXlR~DwfX9AfS%wF7=s2~6Z68(Tk z#bZ|DargNafU_-w7${A^ha!i;mRt^ zfdb}H@YG1vBye4I?nFN#ksMNyQeda|l2-31v#j3WJ-s>6xt9}>NCXooQL!p0<*{4R zYQA*GG7+JZ)21;~GNPCHDUXSzEe>azgBX#jhJ_gVU?=%TCTlC~TnrWPXfm=5Bus^y zCi{T54!cP&y6!3QGon{uUD1hynKZtL#{aWSxXGE|5sWDvm#Db7%`m5)IMsI;*pF^< zG(CpD{+@+}I&7i{K3`uuIja`y2Xn6iP2s^x#bfEB>c8l-JnNm`~eB-?-xig*#$xuDE!q^Tn&Yxgq0S+ zv^7((2a;mFpCW1|_G<0*@mCLBvEF`)hqw&Sk3vnVfG zR0~}&{N&9{WPVV+>bQT+VBn3`xmK?Vt5*$DRz^g%2vu40WU=ksFHiAk>E6!+C+^#W zl(?5D{LewE{~sL0(9_L#5*{bk%?QT21nym@^oQLCM-zll<+`vK^KPoaI^^hq-=+7Amy34$} zPRa&NKmh|IV%MiFkx7rPDKkOi7DEKgE*?TIm}AY;Ca~{J%Tsh$+I^Mvnqr=Uf)WxA zN_Qf>gRgYH{S~g`?7l`lX&PPdvd$a7SB|Ex!#`IhJSZ}ork9L*p9IDP%FA%&~- zU3uj&hg&VkV05NbS4!goy<8=hF^qWe)Iy=)sJ}^fMsd}&|2{FMQeFnP6VGzQhw`bB z(m`Idp(ThJc};NZ-sdcVvQVPf^u@d~uJE>Yv;^&EB}P2GSVp1XZxt~Y36PWv+`wyA*M}+uGLV1h+CGp$1M*n8Gyu1tt5=q0@@)pBVW=pMh zlu>pUNKvn+XS}Obz+35+yvE!AVp1SUIs5UH(J(@MQ<}S&>6f|7_(=sX*faVETw~v+ zi#&n2V2)abj!cLBTitUvP+QU>k;igWMt+hu?A5neBbEPQ9>rvlj;EekO<KQSepmLS*(F>kt zw_n7W_pFJ3l|9)S(pU>Peppi8Bl=}E625nGm-NPyGb!fxH3b|V8FWsy96Xd)zwqi~ zUtKW5S)kXiVdeVEv(uwC*NxrcL{R_!*S+3p2jtkim1KRl+k30aYy=!9-rp^&B4FQS za|2y2BSNCczMZLTh{3K%0wesU@`4AmPrx(Lt0@1e7#1k~;Jgl|d~E$@K*LEt%DJwS z{MFA~A{Bg!S6#c8<2qtr3EjI9vbqDo7xUf}Hz$Kk6QB*L9dSlEkiaY#3Ej+d4LiLC z!G4+!@o(SAo0b6B$~y`9{PP_7f8hlMg#Eium&cjz&Ys)~p_Q5S3d{0fCwZ=3d$i@{ zCnFx3YXg?zDPx6HxJ-AuxRO=LO8;udne>>9;#+3)f4)3MHkrMo6W+P4`+iHRf5J0B z*!_)f-#oS769_}{$YWc?{xA04Gb)NU?DG_nEFc1sK|rFCBuZ+MM3F2R1VM6?EYe6+ zf*@HC$w6|K++;)~CkYZH#|D}vG~HcO_`J`VcV~BY_RM~n-7}}YbyeMWRrOurzOL*4 z`%}K8(Bc^eE4Tj9ciFHzQw|7*@Xadi^U^Wu@i2rLJf%7eOCqPi$l^gG60sd`n4M79_(pO0FjzzE0+ZBs6@% zO|LdPL6YS4f^)wYpv%V!h&Asy%O{NhQ@|uoC5~>;dUf7y+&zGG>q<*V189=-?9JCp zQdIn?efV=?<~{U5?oub6-FKW@Oyl6ywiOIKC!~eAN}G9&J+1w{q2weq57y#tnR)cC3%=-k(rEpi`kW%9{;&;PwdGXa&wNR_H3NfOWulGVM*8*c z!DJr}5X8_vx;`wq`@|XGm!KivZCLbiYINE%KU~I;%PqCP$?>3+K7VwUDri6I4;K6q zbghGF0`O$kLzPNYUIgqS#gEb>5wDTlZ$L*2(Eh3L2h($|UX&p)VHYR+YdY1>UHm!- zeIj9!^a&nFERur+W-WlEJd(=RV`8)O`K7L28i4Gn^ommJ;zz$JNuVH*@Ip-10u8>t z0IzfEbF%7x{xy8!jQnHbNj{I zU7I;XUsQQ_X!r2DGhFh85ja2l*<8~r znqTE32kt`5VKM;vm9^&gB4iOyJN$`abPC@r%QrcTvx?u@V6fBu^po8YjZTZtE}mol9h_3Ye8k$v6U6|0ki^!9XbE_K6y0PmV}AED1@W; zkHhX9Dgy^s*xLo|{RaC6ea-JzVIRv+d0+D}tL>3TZ=(yYf0A*W&?HjJmUE6gPRlU} z`IYvASzP1~M%W@sxl0qr3yqiY?$+){`($hOru;bW(nFn(|0)R9?gV0vkg2J+&9!WD zM$Jsby>P}vIY9+6Wwz>Wc_pl_U)ArfT#0OQ1F-x-L7y69*AG2@&()pm*FPqEd4Rh= z=c;RaNgiNml-PG}h5wmc(r_3UTo_G*Rg>bVl)4++qOpvK zZM!S);ZD$4l7Mt)K>9d*JbMS^Rs{=I?@$iGb|exKM8i_PN=Ls?$~EC^N>Z8q^&x4> zAHCa*H3&UQG-P+_k(+zZYNJR-SNHA?`523*hi+iX%*!UcpUXf`B@J(Zn1P_Xf>`ov zfGB!TS^|5Ad7J)=XR|bri2bTnaF>(52jDEDOXV^bc#T6*B%M+_f!dh7dMN9yGJ&QH3DW$wUM-{lp^jpo;` zKa#$l%eGjdheh$#pjlSzUZnbQlx(lwDuGiFdq4CM*J1U5LDl~Bd{xg=DfrDe8El$r z-cQd5Qai=$jhO**X&MW!|HZdTh*Gd(+Hy@;Sv;TwIwfI87D2 zz9g?QgY9gAcY@#c4`!U)OJXofpfMZ{s5QO^;?66&`itVMxz~9e>imSxA*5f24F5V(G;DmyiOqx6;GsvjQkI@fbc(bMkk^TxKn+a_IE@ zx?qz|Po3<$Vle4|!nXtHRCObOL~`wl&ZZ_*K97{Az`yeqe>cudGY0vibPmY!U2Jj} zYsZZx`|a9(@zSU%uqouu%)REVC?u@l{5B?aSuxP&IBJkTR0ehMp1{HeG0zGlWpFhNX_IWGY0*`_#%u@~mWSQT# zLgLiE;QHQdqlE9x5&6t})BJKQEq{(Wp*vsvw$yDVT@0pen6!h)gCZH;e5_|a9V>S- zJeLi+wX9`mNy;8U-rX`Dup&SEbhJ%{Tw*&vtLuZ*KH&-V3^-w+bM1{2_?ae4${cto zFTqr4_09bC>Tf5f!&M@_BiH5>_cIyaZqR)LCr|m!bs*@R75l6EVNdGf+Xm4urAze$ja-{XI&iU{7o7`P@Z-HSNx!!WmUh`Sn=gPkSZ+z`yA`r5!-z> z!PndUjptlqo2H74oLl9^yytR%t&+}LEK77;L4)%cBbn4*l{Vq|_swuWiAT)v<7R4R zkY9~m?hZe%XnDWdzq(q7Sb@UZqkU=kBSOg}30Xvg@FCoSH;}8(j9P7kWUVg+Xr)et zjnGrpw2?DMvvw~o%zfEaZmwR?KRSL9wDkK%Ah1{sc|_K*P{Sq5+$s*@amiat55=X) zlfMFCN0f1p5AY7DtR*_%1&@BZ-<(U!mz5$cWeSYO7QJ(@)k-4`#}}W-`?K7O!!S@D z{I}9Hl!kWiqaTDetotQ68&?N#=LJ59Q(FEMSeD98tn{G!Z=!kuJ8EYWNOb|KV)qzs zL8 z#My`iq*vj|4mS~tEd{;`qWXx+4&{_ga++^o+N)b7t1>Qnx*+;y4AJJh{TMrx4OTQi zPi$~`&avnqb{q2Uus6(OlmWh$t62Nk&u)&pwbqf9nF=}QsexAQ^7%DlBz-R(@BXv0 zE`d+3B=aY85pY-f*hqi)Dw8`{S3}x<=Iqv57-5!0Hafk+p8dPor<43qV7TS)1=7MN znNe{~!>cM<+Xr<=tNpAgSqd#;869M8z9CP+@vWA~dBkd|hZa;2K(lenEUH_+C=kdy zsL^nvy*T1JH<)I6KAl=yg%Jg5r$n;E8v8D>zWJJ+aSXRK{l?54Pi!Y&4w)WWSAp-(%y{#}p(o&tx)hoBaqE+{EBEb1R}UC6Zfq8Q3tPcQ7n=l@{c zIsf0Y?m|`tBpR_CGCJydkGTtUtq0keh?2!_a&l-1;mtrgJCOq0?{(986?l?wn=)^! z7T5hl=xf9hkySv|v_MuM#ec&JyvmamKfp-sSI%}foG6M7CF-W{P#5Z}JNxIe=Zr^m zUV#F&ow0`F<}W~io!wzIEF82zTl=W8VhD+gtnjmEW3A6YhKCk>B+O0ewEfDjX8)$- z7v4MsI_iJ8O9CB4hZNQgj2806dEL|hC76~Z-Am;Rq)*VNN9D)rF_=JpC@ZZ03m7zA zu8SySbNH7{>HjaAvc4K1hnxqK#{$^17OPj%F*2aO9IRJ`4dzvltZou5Uc1zD`3}h6 zz2J7x@qR4AGZSs=`SG8VpL~f)q(GgoKA6eA*sT2cV=^a-%_{4Cf1-qy>PK!N5-#sM zFRje0`|oiTbp&h($dnAJS5%RLlO(%vswsC2yVmY$>MPwF>N)W5t7{;O!V|a;eT77- zxNxSCafXYnT~%qfV{^^*FEIq=Z=2%hx%82!nK~WHa}L=6|2wkJns2fgwxV{Nvd^Bj zeQ`9S zd01K@v{qH>vN|#hO*c$OAwL1{5;+v|5~z0K;gAqrB9ZH&q4g{0P*V)K-Z~5d!^0Mj ztO)7K7}fR#*M*chRHW5{q&@#fSNOe~a7fEI-Zg4gO+=%vY%&M0KtBgwZ^X*E6xOGR zno3nvX`XbAZ&r3xF@3k>d?a~N*&B1PF!0Cy;bQZUq z6Lc3}#}xImZktoi9uFX8{8P4Lr@L9^yV_;RNK==-|WBQp>l9ugzHf){4 z#F|F#`Mg){kfi+UKxzG;YF!>Yc1yg~$^%SgpTOr27bkkM!FljI@&2PDfv7O~-DlGG_>qnT1-E{b)Y74i@1Oh0Zuhbbn?gwVT2HtgiT+~}K6ucY82SA&BT!cT zUzp%`Ox7jT68LFppXg1+|Hqczz)GIAUyRZIN*7UYj}gEsHoRNBUz)yibqX;AZfBjf z=i}(xXm~K3Ihch^f{09dSuU{17}tKhj7P+|gM9U#;&AzjE6{{c%6ygaY~ms=H7%DDAR{?tzD4K=x1rb^g&55EtV)|KLGh#1Bphr^6^kS}rFzL-yupkb-;zZBaTBYz| zUt+WnU@Hj6B+uG|hgGCXZZHa6R3T+LsQ|FwFDfMTF4TT*4pKeP&jcZZylK#R8K(Kk z-X+NPu$V%qyx?$i%)~U|XPxV;r0jM5s2=C2ViNRwiXPALMaXppmbbAGc}5_&=y#cF z?h}d!V+Eg;9a8k7u3~0a1Lv_y@Y83%ai#D*$gafTM@U1`t}_-AYd%i}({8yu zXi;QAhwPEf!#h|K?d9UWKa&HzJphBfwuGkHKmNx{u(1MG=6CVCmLg}@ySU@yBjmf% z*OS<+4SvHRU|D?hpTUpZ|1T(Q|5IqU|6lCHYq4RKL7bEfBKzo@{*P$8GF^1~raG;d zwM6Ur0?FZtuvT~OH&g&uiUj;y; z)yi!M_69$jQ;7@h6&ugd&iH?TSHxQ@Gd`J`@~-k4xJ`R>)PIPE0wfcM8pUALKZH{= zUE$32_cD0zP@J%gvb5K3`u;g_RGf7MF@3zI2Vh)etB8-|JGO$yMvD-(jz(H{Haeny7ON=eGO^Z{n-Fb8n6KyG(DQw3t(%Y|Ty861o=b()Q)&U%{ps#_* z#HfqKR`Q(PhM5Gq0DAot^JjQ}DTQpO$yX3THWHEbsH8hq9x`jt^OZ|Mt3UVBWZy`( z;~u^OjWcP}FzZk3Yt>A_G=fa_|M(VU5sw9k7W01J~et4)@%Dy zJRH0t^|WB1cVH(lJyW7Pf?L-l%wS@Jg?wK9u-wJZ$^1~dw=Z8>(;;5n$ua)N$7fyR zyw9S1HLhTNTZ$f}JeO64X|0aWtfG7K&Hv+J$a8XMM`hB)Es`>SagJyYPO2h4x zXFO+s?FI1YTvT6Dx`6y4nBAfTeIz@4OEl9(-jh#m3}uqk9y!(SA9mC5nTTvEiZl;; zMDsNSU(JP7NKwx&Xo&|1Z)cx74df5=eF8N&J#AGdOsY>XRB3VF_5&b6jT4 zuwx)Pgfh#hy?p@9vy<=otBfzFgLM(#t`FOHQN#$ey^h9a1|APMzbl<7i@0(eHMHdG z>i%}ysLhk36^ml13hX8J12YcKZ!ixJr9Hw;;WH(Rr56k7)1*I=H|tg=JsNHd7zO!v zRq(}2Yg6QN80PX7XdS@gt1nV>+ZFJ52}&D~8yQsrQ;z3ge*cgR%&)vWp#|qvgneln zF324Nv}-$=%e-P``M0;4a}5!RPo;@iXk76%Gr%I}p>#!(K(9uM_K`|?Txm@{!Vl1p z{1T*qIT2sp0q`3z3XV2JG3qPPF;1x>c9pL$5HXh=b&O07Z znDlT?>497XZIOE`R6p-p$#dIVdI@@9xMl<_CDS^|?`&d|M z=^v&ZL)y5UdqrUN?G{F*)0};?jqAgYBNfJJOW@h^%KX|u;-R)H&(EIjZvyS-j9JY* z;b6^_B2iGi=X7h6pJ3hXJN!oP;26h+%vd-utQyxMg)tDMWz@{oIq*f;Sc~R-S5P*ngw<{gzfHcMf{3f-&`9a* z3P&6r-&hJiH-hU*m*oq(?1(S}T^A%)*bsj~-^D=C6-ubCy9M8F`18POmaMlV(6>A3 z@Jjm0{O8@V*Js>d)ud0%B^XX50Rthp2-U`temExDG>2$*6f-ij^ya>v^Xg79APcN# zK>-ZjEm&TrWx^p9*+wJX9Lpb0+N!8IFM#CfAYMXB02r3wy4q22Oh1NgetkLn4dudLWHlOH89F zHxBomh-DU~SKr1>ZNjL?IQLgTu1u)-UY{r-6lD!k`Hx6GNgcnX_EtEagqcx#IA;@k zrP9a*^I%>*nMt=wG?Ve$E&uiqqnE710`0O4uE$Jt=HWM4IFlQnScs6e9kuzDo38eQ zqEvX0FXYMY(OWtev0P`KMIP1FqEK~I+?qS>EcUCAV;Zr@UGVDUGJ(``a~tB=L*TKWG?4c!aOw>T?w;liY+D0O)_@(oFz7bTF*Su4+5;{ zh*EW=F1gkX+uXD(-^L{SUu#);QL0)PkEaBE*?SCzM%c;Uko+jWZ%iP3xQ^8i4|$gP z(3^Pqe*LyY6#4J9-|9wDfPi@e75sr|@Vb%w3R$vcz&b}D+>C3hZE$!+kZgAqN|kuH zk>Q3Ddq`nCY^CHIH&(Cr<&W#{*Y01~4p?SFB4X(~H%z^h@6_XtfRxN95BF7u<4!1z*!a{aY;K^jP(tO=<8o8i`J;?o{qhf zY#jc)MTeXh)%CzdkxWbH?&sV>v9AnkdBV094Z)`B5B7v5%>T|;z?-OVuy%5Ys{01ZZasAGV7kc7GLf8uRwl^8r@*%70AADN8rXAgAfSu}7&i*AoEDV{z&{c!64>Ts(!#@e|!1gBL^(8$0g} zGN!PrcWg2%CeCqEtMd%#9Ac>%`XSuW-1rV3cUg=Hv9CZi-=wap{P&!7i8Pv}+b`G} zbH8q5BZH&CvacOw{ghzRjiz^O-yjO|AK8MhN@}7MZpq|;(dl3X5uUt92hh-V)tV!; zo8vUHl5CgdY5NG2=?j!GpgsdwOD8Ul-2@L^R9L`7E=sPdN0`|<+UIU>=g~}?+Kv1F za0RVt6^S^&wCcKV!@!b6z5GuPf?xE=Aw-vs;PV~jkahLs!x>%EEG12bB)$inIDf{; z>A^pOi?378(aI}QjXz%X_;w4Hf)hlQkcN{*R8xSsAw5F!oQ;2Sxc`*VHv;4Ju@ede zan7?`I{sGGmw-t> zT%pL>BU-zkP*{;PqBx1;_`Q+Z0}jewUU~h8I$^uxQn(`TH1f=r1AupUp}|6bdY%)@ z`Y%&P=~oo|%Naf*8i0mLOM(EMfmbLL`|jgOVlcg|9f;v@1NB7djR0|wMPeiL=FKy} z+ZjV5S%z9hSoUwQa9N}Xj|Dpl0wq!DKh`hc(9h|DhV8&D?hv@O1R;|$cUC+%0Z%#I zCd;`CZJSxE4`+GuZKiwI=TpY*IV_b~iJr%A8iw9PvYsum%G=E?LS9g@M+=U9fEj2+ zlgV>lbPUlXT*IG%REyxSU+bc~nqD732dHTAcI`vnT%{TMOCATkKSD;{?44QK;!pdo z3{v&JpQer&1cDFw82`mbVegB*(6(xEj@sLp{x9K&pg%yS-urmu5)1ZH@iTS5PhKy* z0$Kg(9%Nf_^5Fw|d%9>mXhdqhcEr&#{Vq+x%OM`0LKb*d{923W!^V4opf`p06%%{l z!F04IkmvdvBk-EyxS9^KyGu@&PzmvGyQ_cisz_VF3B&>g0y$5uWF|^}L6eR%8rQFv z=0Ztg&no9*n?XK_iIu|=l!Cq;V7pajaj&^y54I+xOK%vIjaHEHOxC|cETmJd&9;7# zM%)caE=qq-Rxd=P?`SlCjKzHi>RxiGE1+W$xxb6sekBoi!)#u;LW!b-nK?nJeqslO zrYFDqK1FKHpF5<(PRwe$QoH%XG*HkQVNaH|oZ&NR{j zOe^onMILfwzPi*fy3y8k1<}{EN?|ArLbV}>a78Z@Y*qCuwa!iCk{}JLW*x zAlZHmTgy+D{F<{M>W3$65w;3PrB-v)vs$SS@5hb01BtJOb=)S4N)8%ceNIabuFsr(ss3oc#6>@ul}5K zwc^DmRT84>HD{n6M0hAJ=MU7Sv0s6?!c|LrEc$0j63?>_kU#RoiDtOC`I7hS)hb$d zA&awjLT-Do6&vo;=_y9pFNP%qixp_j8yM$nhrE}VQtP=SucDA6($kQ{-T;&Oj*;9k z8!?({I4=tFRj&%rF3fJ6^Lq+ zXWtX=CggEV;w%fvzrtwBQwL%UHI9%p2G=_eF$ACb`sScFgaOxO|&(zbn8CYosrbpp)3rfh5}*>MhRBeEjA zMHn9z97cEjyYol2Io{>qXvhl$n53fW?4N5B5zYfT(6U*rixaR zB0EbMn`c!yc}+k&kWBq@v~%GlUjJ^|BWTXA_uBIpygHN{c+h30hZh26wpd$CZ}Ups zVSqIrGVhjvl;Pz5xx`aCy#Bm{yR}$iCu6AS(miO}oe{2Q@sM4q;E!t8}IWB|}I9v9Ed)h=*qmMoCve^$SV%FG~{XEvtKd!@h@Any6@md7Da?Ov$~gwPsg=*|k7Z|nApq?9;u4X+ zfM1@!%nrewnU>o=aB zr=`7W+JZZ!Zxin6JU6Sn%g~aJck{Dm5`u(}uX;)yD-4t=*J1-~*}^HsPLk)`^-EuN zUhn5jOvk9$e`PAFuK{d;0W7-=+ogbNJeiQaqGdU$VZCEYXz&Z+OE{$k6kli&)#1b; zePI{D%M{Lk*t6011v*p{yJ?W+{squPqf=wVs<9(bguC1u&Hmect2VD*RKhlhR{f{A zJ|iRVy;FKyq=`UZ4vFQvYuKZ%hdMd=@ZQjoY zzeT!dh5ihnUfjm@e=gowgB9_v!9Ea=XMGY?@(gq)PO9iKp3)Tl%*6hs48kcCPv8@J ze^*QIf7s9$6HM>hB)1oBpIg@~6C~X0`YvdiyI<-JEWV_7x?m~R_AZkFs)MkI3(BJ* zF)Anzfmi0A)DQaad1+q>=5p`!J$_HScjnRaGVfREo`%p(_MF>?AJG!rk6<3FH!s*e zq6wZ##KjAJg8tOt=yojoWeojJr^=51Y@PJROF+o=bIu+lJZcZd%2fkQPk)?&G-01# zbci09SVFyXYgU`hHW8ah!-7EEWvJ~>f*>0w8Q>W0ro9Q1rF|Mkx{U7k(r20>JC(Cy z)snID=c`X+KStGRf1Jr;<&yAl{80;|XHdo>R#!4y{5sahF5WA7_i*844Sd`n%HjTj z=&3^K&n^Ravbtm?k=N4@_e;Y+{LnhKY^rMQmPL?1dG{0LBuRAhr7{gi14mp9U?e(~ zeqEj%g>--Bs#z3t>lz@!c=0|jwqZOEeO+|$yvB_g`l7yWec zo6wtCPzMq0iFgdwl;wjXGM$*b&M(S%)gNQrWUuyfcjSA21Qb)i!d@?so+$1!Jt6sU zjmbXh+7pt{*o}-G>r9QZxo0XKZZ_-aF2&=li^#4&&fih@GwaIQ3&eR8vw1GbU-j77 z0QF`({%>Q1Ts<*3ES#(DqlK;OsKi}zdZ!8q!xYN&nL6R*>r;p^L%%hUn(D1P2tEk& zZXaK`sb0j@vele&59_K;1!4w^)ZgEFc#K@)JVzP2HLMUk+i8=RS!$0C&yifxovP!0 zoC{0JrekY~f2fegtewcsXw)x2xrPoWwChXSGLX3-cj$?kyfKU(MAa_Dj}KrIF@_8; zDu1hoI(5JXzIYen6|GZc|x~Fls;u;fUNjl%_*@r+(S*XZ3&?c{&< zk^E;LTPz*Feik9nM|MW757G|C{z_x8?#r1{pB)+w|OZRJ;?tBkG z*=`C=i{99c^rzh*CL{XyP?_Jc-zB9x`;Rjq|M%z3N@@8}k-|TVY(fXk_d5t8OI{9I zG>|4kFKKz0*&uT$Iph8}5K3B88}S?_({jnWJO8o(EKjo%?xxVsLd(Jr z7h9Zloc8_hUAlGaRX|dtQ&==t`BLRYBX974H^Ckxg)r!=85R7Y=6zk~gM`25NlR;= z(rE^)DctosH3P(}FSSmEba|fmTOU1%j$2wX-jRPW-%NYTtBk?<-8mp-f^->@R41CF zZq&>`Z~>X02f*<|!o+bICpcP8;bRNn39y< zk(Hbt|Iu$6?e|@B8PfpKlo^%`G%|A0xmNfCiAO_orSUJaG3{aevT7W00%8-PYsXfn zL$2M2ZT6MCy^*?;6Tv7=BfUqWj~U>36%OLYZ3M-XEjz5tXn8uf-tx6>k`+JJEoA z?5hU|Ekeg%&k5X?|7-UTfzQNNxMm7h-Otk^;KfUiz|!C^SoEqHbXNa23^7;ZE^Ok&8@Y-GOdD=_MHzxt7rQ2Eom@mL-rpvm}oG$KMwB9dx zqew3Hme&-u`L(st9S7t?#ak4{3%F+VI#9yqov0C0nMcqywp)K}f!(-Q+0T?pQ;A8~*rHztYq0N-%Nv>>`ImBS z#+lvt6PYNpcMTbtU0QJu_~}>ya~Yc2+p@EaX8e*sKlZlEzM?td;@gK~cm#uMww=uG zjdXE%(9ju=UeP#MDyx;~`->HkH6lOs{h0tAvy#%Cvuml!lPn<>&3_Qjw1!!l2z$u6 zUjM@R+ou}#)07!7_kTLfUPRo|Y+u%wJJ8uD$$EO?$m;tYX5ZjuWPXM@%QSb_!)7RM z4P7JxKVeVH9h~;g8=pHO^<{Ro{7Qc3mmJyxIFz$cC%m_bX3(pVSyVec$ytI^PS2la zB4aRuKNKBxH3-_heKFOzOZxL*u)6hS9H`oyvx*$R#(QEanQq!~%>ZV=E6fOz z<)%?&@o#yw(+(=qt>MUY*K6-#Eb+uXPEv1(`ROg<3hCxkD{4D-HnzIg%=qlUbW=cw$Js zt$RQ2x~*6>e&3DzF*MhPP5S}2dEL#hxHZA#eWFvw1w18uwQ`@~e&Hrk%K}i1NOIP; zYF^j00M?Lk1^s$$p#70@RACu=MBxpnrpiz!_V;ppwzkHft|VLxPOWt_*(SxS4Rmtt zkAw^2P-^Wtr4bzz0kD3ZN@{RV4IG;a7Jj0R0k)W_^Z}G&IxW{MG%U^xh^KCRQck@} z@}b!iL-aFAG*YGsyMZ#*cvoLDT>KsRZTiW|%;7ja!^H@6P%L|NiHNCAnjzskA4}#|pLFPgMJ7IgF>FZB<6U z1%$xoG?EP=AQ*f-^Z{#HyfO62lO{hq{;MzUBQ}sb+=?CCg$P7e9Q)0%6n#Fv-n{wc zato{#LSRe!U;KXi6WM>Uee?-@s%f2Oxz2mE7z!{`;F03j+^|xRbaigUG8)$>w5T`s zMeBd%UrF#9)>+|?vM5{KN((4P;or!0G8oQ6+hL~y-an=}F%k~#Xp;z?6u*1R+!#6w z)Bvjyw_HiXyY4%deJX>9r2f#0W03>*x_{N%9%L4ppV-r`Efww1X4F3>guII+B~+S; zn1kMVU?c~Y{8YlaQ3p-F6YSA9un6x?br5M_r{r4mAx6cr5^Y6j#8=ripukO4u2vww-gFi0d zIJGY{_91}IHJC(ZLZNj8HTZXlzIipv$=KB@d&f%RX!bl$_C|%PrN4Od?o#r7!_nNw z}Cb>PVc|FC{VG)Da0u7FGhB%*e6yUqGUu={NYHlZ^ zvsx8@8H08^6xf=({#v~?{V7gLC+qi9CW0;xilEFU9tE)GpkHOau)=nDr?IkYx$~z<`&!f4f zQ+5LG!fSw&VhWY3LGYp;Sn!_7JjsLyrUspKvy9^+HYYND7v)Vg7ox~y&u`I-Km1at zm7z?)<~L{V1+Z2RWERJYE5GH zB)(4&x^UJd>(Io$gxixxusg&WI|hP1{&2&&d<@72HOVGoL1@XQ2MqFk8{LEe{>`Y8((M%$=0cZ-*@&ls z!?oKTxa5Vn=bQ@@9u(Qzm;%iS*M(?@Ksc_#~i10EPb^%-bLnw<&yU6 zlSOGZk1$e8?(b>xOJ_{+GWBFqTa~If7H>O^CqHK;hSsuQjG$LV4rRwGlfD$C7))ED z=ciY1O_5xXH``!ruNGk2_g&7HJ(yAD-*24uZ@Gqo_1XbBu?`r}par4{K_A6!T`pmm z+s@kH1>&toq#_RU`{#Kn8Hn}E!b@~2pXt+M%&X1MugKo4X3&g_V4pUP1fAB_oylG% zA=!4Jda?i}S5%8{R~qogivPx*DIG&Q`c((50zx%TItBKfC`i+kLw?~iTYPvabh2SS zY+1~ry1cr`F!lV{?dHvV0fbiK{at}e@|phS+be6M9Ub9dpH_nFXq<87uZF6- zH5kIf4mt=nKDGgV8%<|{Y`}3Y*&`y<4?N2iON^PCz(sfJ==|cT*qn7VUqq&v1(Vos zl;z1N8|pX}_QVpiGh-jg#I>vPaztVkX+&WRn!E920j*e~udjNzqS~BM~-&9--WT0khiM5)W}`o`ri0;LPejh-kW3D*XNs~ z@VM@E@bd)=+HjfuyhZxIH~Rn9B^Wd|92#eEL2L}=RYEblLuoiK&qxW#h^YudC2tX5 zbC18)Qg^K@KGR)DQKYz|p~AY8h^R_~@!PGJ&mHcg;Th`X9i*>TzID}QZYqAcBT>rE;EUGedJGhTZ<(Ij^n|6 zkcQ7S*_Q@rT7!j>Yx@nfDhrQCE;5PZ7i$Pe9%|F;$Cdnfzw0VQ8G(;-UFdDQ`(9|^z9Yu(k1h)sc0)hn_(A4J zY23uCinW-fk1XUCOMqIvpVhYRczVS1wo89q1fiw&HHlklS<_~{r40S{Q+0I* zcu*=L1NjYweKjgiFfJ~D=G<>wr3zRtwbZ=7#t?JTv%E)`>G8S54iNZ?ub#wn$axz_ zwGjM*WaYbLdHA?CGgwi-3z+wi!UIZpWYCq4%$)n&i0E|{^OY)mA< zQYHmNEi39I%+GN?fnm5$(3t$po(|M36pdN8&g zc7}WHXlSj+DSlCcANkhJV%Yo#!u(k=9awGV+kdZjD3i^#_x`N^=?{3h+eyHZ4?-)3 zF>=-x-QE%Nu5)U$&6W__0NafUkF$CP1=s*8^SFI8MFY<%Q zwbdRSc=)ST7WX{Os)l>-1U1ZUBUjgzBc0=43bmgjRTsTgI*vj{E|T#U4W7kRc9|Rt z+z@3*Q&^Ozf9F4l*5efk^)q}p@aP8g&v%sG9X~^3q4ulfu!45&Tb@a#{I_Y}${+;_ z!tZ=J5*a(yU+*uM2mrd}mrcnQ#rd~8+7Y|Xw_I<7`V^T}+ed~w} zldl?n{OOx!K9<@4+zM6d+ZsytXifXIqix)iPDDgt>n2bg2mKrboCU?nVzx(unZxZc zM5MY-uIyCb*r53?ioJE)Gy#&a{CV%8CM_+o#MDsBzNx<8mLY5KQ!A29`n5Ppi; zEZ6Rz8GDkbTa|qW56qKVIYu@Gwk{`r^bE4&zYynr2;>V&^GT{tvwR=Gc7Oed5ueF= z8d?0($f+&s1`F;hOu3;z6V_y74Rt?Ur?hy=6NC4N0fD-l$il|FT=JsC3@u@pVp-e- zGWtv~mCIbxwsRt2B&I3-kJkEyThn^VNo5v_3Nr(JlM!?=Xql2gf%#XO5J8${oqyz%r`B2pYQWL$7A%~D70>PWw0Y&5G)5kD7~1794g1!!2S$O!vayE5BCYgB!pJxeP1iXj9Hz|s zc#+>JTfW`QJq#crG&IL){B9LQ#E$O}S@(=i`SIVFk}W2jY^CIWwN;huPXWQ6yZfWl z4_VlZp;t?e0vOr7b91*FeO>Y9IGGC{f;9n|3NAFfJqCu>o&U`)>~%}+m!lw^1;}Ur zAf2nbWv1M=pi6Lf^I9JPa-83*(Uf5>ym6KDeF z?DB835eKb^r>G5af&7s-SQ3%51F7xP`GagQ`j98Qv3bdFGEYUVrn2d=W?gFgoEF zbzI=Y3|@pQ*?pf{gSdCS*VFrenR|MIuzh`uSTft_R=@D6S5S?EFy3XS=NQ zB`xH*bTju%OLgQHuX=@%AC}udvCH5&-_>%wzK*N}v(%4_cS`^egjHW>GoU-?(&y?~ z^5qx}3aKE!!7%knyD*Uc;n9YDx|j>^)zmI0!!=!z@>{y9C1!4Bt0(b`BG+g4>*lZb zy}cHjEc0uJB{O7E)p6rR!;d1{4|}1j3|_Dt8em0ZYu^pp!hHFt-QRbtmu8I4?z8;s z{d)1tA@LayQYb&-Km2N|2!a)Qb0)QvO6st5Ru-ssvDNUMFKC z`Mq(n3cwMIWuXWA>K5n_QrLDj8@-4*@I}y;ZpAH~C`)c^z1Z3V-iR^zE*#Z4U%XK7 zPG5#vw!d!*ym;7bcko;CIIsi&pzA(p5OynU9;PQ}2xdP1Y|M)WqXf%PD>Ovh;0^)+JB={`!KFw|saPUKX z7Stp>1yPlb%pLZZz@&rTIg?|{C>=T!w9z`kM|IJ(%eBB|L)%a5D<@wO?nBvg&GJDNHS;e+57D0I@fvMv#)dB_gv41{Rt*RhM7B)x$m{s|NmR5 z*${`Ko?0Gw@NV!E17!&DClx+D)?;1p^ZvAlUA&0%Ew3#POm_B0375WO_t4(VOS*wr zLdzJ)Ly&=Qr$JM8h&@8p;Rg}1bzeSy7Yn>3v9~H9O=g8O@IkeC z?9`&l%En|sK|pTI*oNAe0ZU-j(+8_aH&h5#j16l-V3iQsvw!ydX9x00PV|rL zFwBB6(=BQ#jVEg_ZXG%RjiX)s?8aH=Q@oD;t$6ZDj62J$=3D-`HG6$lpdE{-BX@v6 ze)4T6lCZ^uyi?O`z`$=m8k;~K?7C_-nX&bvoHv_f1y9B3w~yl%oMrJYPQg%Qg?L3aj@S=JEg|xh;>(y7kqtD^ z#xC@8f?3@JVWd81@#y=-^0@vvDzFP67Fh?px%yjbGPY~3t{rTb3MZxOml_0*@Rtw1 zJ6=&wVnY2X-xyV;dXwEE!26i$+uKn4FEwkk)Ecl!Y1Eo99LIMd&7$e1$~C#9WPR_( z4)Hj+g7nPin|XW93fDf_1Wo5dm)O8c1LWaFp7eESs_%n`)A}XA5+1?Jn0%$0%t`M{ zr>)d+CWv8tKxaMH<-c2d2o-{?D6!et>@%R8d& z8IlX>OV5_UYRZcmTnCpp^NlXJkHLqIUGrY?gS1S~!Hr{}8_N6h!d`VRJk+!|PEmJP z_GPaa_O;Ibs8QKUYQQRW&@z#7b_ze*!%}|kj8;+t;YqD?PJ3+|Z zkiw)@CI;`MOTvej$#QERs}0ASBLwh~=4gQs-=CS4T_N&$$$Q}F%7T9dMpFq(qJjT7V}Vz>H^F65)%=*Kum= zXMXZg9BNk=D&(}`@;Bt?i`=bA6K7kWJ+v9TQIM+#clpx?zI&S32cy7qZ zk=(~02}Hb)ga63Phsp<5#A-$SVM+UVBF2ef#7m#Ej;SVJzv@6Txo5crR`=@2+e7Rq zv_k9TAqp))oJI8Y!o44NH|)rGDJjTJ zY^6fT3$o+t5_`UM{cPmoFHq8;&~9y?MB|o0ZPYa2vbBr|X!siH*%H3j@?`o4rg}fg z4q&LCi$-NsHf(G-t)jf8sl-XEM6VCM83)_lpy}tvAd&=!%Rou1$`;4W9D`N{$09X^ z7~Pd}ln0N#HzB&LSK_$u0x`7WVEdr58&>3VNo4!fxC`L5ni>E}vpJm4;#-E%dL2&y z5!3ihmwvOuqz@%Yy<1U(BQDCR0L;62(P^vxpr4Ci-^!AEZ2_-o0Jpez=V@KuJ@NH> z>kk(o{zBR#A6~y>sZeVtu)@GGXzrK7>=8y_tkKyy3)OGG7WHcWn7yk>O__#;m^W%0 za|LgG9~6cn8ClPESt!YbM!=1JUf}#fTQU3;O8_%4gW7MoJ@Db2^jqT*XrF&+n}#uPT0BMYq#8E>*Kpt^BH^GMWq<%n zg8&PyM6P6A)>p6h9Rj9yMD*uR6{Km_uMG(}cc_Wr%Tz6nH}3DEk~U@MTSL;%(J*Wy zuIB$i_$$qAnMY#z+^8l)O3emuB-km{_KqgObN(zDlo(%t%T7E6Z+&%QVcAPOgMu_t z)P=XZ#^DCD!%8Y~%BpsNNZ4Hs4}dH(Sv#5oBr}{#o#HH_NkWcJ{P6NrH$+-|<|l=P zDEUFQm>!{_GOw`a+s7$q&2giHDT~qz*H#{T$DSbPU4sKGZo#3G1FbOwVBcq=QIW!_ zT6dl{$?$E>Q zyZ_{zjZ%OiM0ktL{o7HB^2QTwUl;~_o@!utFP+UWJhjlcu8Wy@r`AK4o9}Vs)y)%q zr{%}9JXqznANp*^2oou}@H9@@JDkUMh0xinDynQF7#)Gsui#f*=Q~|ls6vFfj9Q2Z za4~1_P3CRi2bAC{?|&5URP66=89{(&@Km6}X+Zw|U?bfGDpF#M00&Ee{rP!j{cx{i z(W4##uNO@1yL1bsM+*exQ>z=|yFbiuwPVRJyR!BaqXhyN{I%{ddO@imQ4H{<;o?%H z$CogttyLzSwzA$hKtB@B6Gm_END)v%6AKTPlvW%29V%4>M@@(7y-r7s`$ySvU(gkv zq@-`u3BI$CAbxqDbu?9iHe#3Q=}coFrvh#(Sc}(${50p&8P$ii`Or5JV5OYT;jZ8n z{K@q`w)`1zO+5QAP%vS$~lrppYp-4XcVyJ;KPW zgSun!eLZaH$bEE<1YERq49Hi);o81?f?2mqCUimBNI;gQ1Sg|A~QY|d(|4e!#zaC@{YsA^h-_gUS#msXJ&tV z+X#+eL~qBP51&o4$-MI8tK!|G%;!yvw_2{ndJAyj!Zo>HN)(>Ct-kCS%cbl&;Io&m z-F~7df?k)+p`|xq*M_gSCc8>=+1!+YIg?>KD4`={Ie?@g&8p~;vvV+mlu5nPuaHFmow?qi0AlAA`% zNPY=68!z!x=UlR%E!2um)P~QV|G`P<5m(ibw@K!g*^;N7J^isMC_NDh7LH5TigYf} zQ>dE^gOb^GP|s7uo3&g03EwcpC85=U zPH)8^79C0Sfk$uFJOY|*a1;@%lYV=C851*BoB$Ar)$Rn=Lc$R%3=S$`(PP7}0$rc3 zGNU=gJuepfXP)`1u2KqJ$uRDLp-h=V{Lc@bG>H=_yK{kEsI1iP#Z@1iIPV9)1$kk0RLkvG|(YB z&0`1W=N0ad;L~I^x)}_G97i?9Q z(nV^9)kUAAop4NiGUk(2_KjD&iEW~7U1;YB_+kdFtM=yodm}K@lxp^})5A!ltcE-3 z9%X&u8e!-WC``M^$cF@Op~<(xQPy185F{^@!Qm~t9M2^1)e{l|sP=o-;hy3c)~elHF?DSLqTl|_`XG-)4X+RrI)u#0aH*ffThpspsXV6fj!Fw%zCVk4&VU)lv`_( z6V|43zzD&Ux!9@MM}BgU%_E3Ow6pEbm=6^WwbLs|yXJjxHcG?>%rNJ6iaKh9=Kpb7 zX*z<;3MCfkRey`&^$Irf6o3L?2>RWdkOwP)XOeYR?7ST>l9uAF%uBmD`eJmg+ab>4 zU!hmsZzp~Dy@i7&TlH;}tV2oDZ(Y9;u}PiZeVN{o#~QK8ede*F%HeXvWC7;|GA^OtPY{#gI3pS`>qw)|9Q-#X% z?}nuS<;UPFQ(X3So%s7ZiAiIRtmbSl{KzMccw(rGP4M~k#6J7U%`^s`uEe0}d2xw%I#d!tn*j+efC)qyoaw)$byOtt;l#8^wg7yBWsR zy1%t&3+Piyq$1&&`5)8>Tmr>VaD6WaN^rF++sWzLsm6?~WAL~uHXH+@Uy|Mm6+8$z z*T0Td(4PH)(=<1P@590(Ps%TPaGP?lFkkyA;(c&_96naId^Z2I8EB8A8hM;}xQv6>hF9Ll@Zo*KMG4Z3v5Cb@HJ40|6FS~l#2#cqJJDY0rvR&}! zzCqL;9C+F(ZIQYb_*KNZDrmGTa+e0NsA^2{P!Wu|1Rj6)|K0Rc?B+t((M#PsoL#eDB6fecm=DKG>Sb{-lI&2x!sfgsSI`&g+5CTAQ@)YUL1yIpjfN-;=_dPmT-;>DsIR7QvY|(V1Vh zz{GMQI8vk${?z?$L!q>M@iVzL>?5E{OWRrNI?q)THB-QZg<5nB&T~Kc9k2*t>&8Om znaSYMj+GbF{?G@dNvCm-lmH+QEu7TqFn3`eqR1)FP1bbe#r8sFa;@DL@BZ4p%QQd! z>KdDj*qwWef{#msU0&_KJHOIS)1RXAX_*)vv4Af6^&+T>J=^r%-!BQcC>Up}vl=Fc zVG;J>SUwc%`zseL4d8M+0_#Qk=M7Uc=?S?*&Ce-Py4)X~FP#!RMs5EFK+l_Dl(<7%*E;m~%YBH+-D73l+Zy*GAd4b5XV=kFSvoc! zqOo-ML)l(u%Isbc&X8%(8^R-_&lCZjj8o6^l86lcepgkvp8AQ>o0yFb?a(Q8wuxE0 z^jT6Q5PGkl=nKjP1gnqcT`Hb>eYwWct6|*t*|OD1SMJ>5cSha3HrkhuU(nk<)i6KP zPN1=R@n%sVw;j##P1W#=dLcicKwbSKnDN!gH-t^N1aLQz1lrafV|QN31jkOI-@*t%WOtE&mS5Ji_7L$J>RRV2Ml-ev?ofui}v2somuRB=xbON z(%Kr8aV^RQRA4L zy@s9K^?lc=>NKjeQa2^PxY@(6PQD182|8E(Her73LjwJ66vc7bfB382?mCjeLOg6& z@oj*`j;G-{Lb~>%#qzfrX4H+3OU*lf7iPb8K)0NbJabm_ZXo>v!ZbQN33O7|>5RrP zw6bo23K(iD&>DwfbN+V=xBCBvg*%3z`Tuef_-HuSV(Km${}ed#CJUn%&60t)r@FQY&5$KJx#@l7&+>{hV2E+E+sXN%8kmxeu-ySIj4_yPQRV4kIJ1k z65lFz{$(5I_K9J8x6b!W)*F|w8)nlbP@KGKGFeSY*M|IAy0o`k{ng?k|KF9M!^Z&&$M>u#EQ$=`)dbbK4W zRStBQY=jL_kKA%2shD3#EV_?ETjCe+v6*rPaQXL94y%FAaMDmR!WjyG^}GkM(AV=z zG@3`gnYw;>_u(O9G^w&AmNYVuRd8TycmWpky5sg1=VyKEw(>6Y3#A#U@1E2LND-ll zwMw7%n-{X%dKJV~h<8q;12dz!@AROb1wTHex&BPBb9Cy@kHH0EJ9^Q42|4ttpt8`v zCih&n&eP7*KgV2e6w=>MvHUB`CCP_?Y2*t7cPVXFxXNdLXmmSV{$S0YbEA-1R&wYc z>vhB*xqV339cIHm|Ax0rp_W73RHs{)TBhNa>3{AUDVIykDv}987CyW2y^yj6f*FUU z$aS5t`_{JWc*M1WtG#&=vtq#O*O}=>=aNVDXqGW{74N4<(-m7|C(ig%JydC!;h`&~ zKdpPlztpHHQ<>{_Fzcth;G|t9p?6=~RfnpAJ?V2#|KP9!ce8~Np9K1f6)}P7iTLN* z@EE4H%N)|`d70n&6f|e^*RpQ*gkDh|hD0*9QGgBR@!2_%c(ZC>+V^kO%5}aG(wyjf za&6;h*BN(=Nss?=AZ>WO$p7j<_&;lWmohw=u!{@qKtz*o`qLhCAG2VB!oHyD5*&0C zYtk5V|AAnvU+%=-4>i*#4uo5#Klnb7F|KP<_23FEh3D>kMi`Q-o9*p>bnkC|~9xE|mC6@eT?|c>M`nJRGCEga0&`znLD@DK#^0W7d*BmGz~h-%~YpzdR!PYF@<- zCGX#JYTRK1XklRz#1BNwJuU}dllcHq|&jefSzIT6@5Pa4hH+58X9e>>u=gno=lErr*FW3vZ_?Pf>zx#Lqt4EixmuDm4TjsbhzI~o zQHTUZQz@~5tR6qVlSIp~8W8Ro!E>yT_}4giFLzyM?-_Hr6$xL;SW;So zRhjezeX_x6a!r(wzXVjoP(R`P8EKHGlait7@p>4Rs&e7UurU%G?&atRQnpjrI(j9q znk=W_>K8@4tm5%Q4*A@tsS0AZ!TmJ^fTPhvoyi7K{zfK34HL9$>NiJ7h%EsTP zbRFu|Qv$>Y46x-?M`C-o@vRavw{p2+X3tBo?6~>`7Vy}bHs?d1jS`4JeH)K{$Qc|A9OplR7#QEdZ(DDI3ju=Ft} zlBIu<^Y2i^7Sp0)CP%TiBf$s3-E`34>uVcQ^h&to9)Bq13^=$C!ZeKuohTJhP8_Q{ z1{yo_n41Nn-fWuz@A>ls)igxJOaK#0aDa#p+QYF3-22Q(>{h%koRIZ)A2}Y>?|u?) z=P(D1Sl2r|eC4u_d^!UEz5w1%!>{GAIS|&6sWFG5tVJI&?w4w>`+uiWKJ-{*x$pCQ zzkue$qX)|<&vRF!Qy0*>wSHWzxu*R-36WrR(#PR!+R1#kV#CIEHd}5VGw=`$8l1uJ z(}}C2^~pWXYBDubdC(GG);#I4N-*o?=9D@zLzd*%AJ6I7a$`3xU#z?J#_&}vcWN_& zH&vF!D=LN$rt~USTn54G9mQheaiMk3mG*Z4JMf$NyxSUaLcaZ|GFU)xPGVQtA^%YX zUWntE4*vEul73MtjBVzNq|!=nS(Ee~4a7<+Ebi9r$Az|I?RSZB!)LH5j@}$>;2ht1 zG^p{b`RBQCQg+y5$uE?apg;RI=+6%1Iw^rAo}V-hV~J;sxm`_OIB}__f_}PoybEg; zS*zHPJ&F8naCMnMzbW<=t9!al=}tm_Rk*!7VUM_s_K`Rs^jc~xA0bDeZluP{u#+EV z7Cu7;d|yYFNQ1tVP5;gUV5l7%u3g(S_(~=D2Ah66(;uv1Jo>M)&i_{htq&qwb-OFh zD6fJBpF;8#P&U-P`U!;lGSsg~4_&-y^!#^kiSs$GAEzZm=#8HHtLSJJeY!Q1uKM_f zF=OWcwH@RyYM_}49ZlKVhJns$*ir=LUDVPHombMJW1NOLs+ccmbR=k_F} zgZJXOg-5=i5xsCto*YfHaMgGT_WlkQrFfy@jhM}yV+8IDI5W5=t2LMZj39))3Oh*? zJfIkZPei<UQ#ck{|tGDX@IRO8>Uxx!ITl*Dy!y)@nAgM{UGYx0!FrfY70-ek5EWxJ-BLT)6~`n>9!PGv!GH zB;DUc0tzxbb zo?bO};Aw(oj4@fBKAJlBY;?`KFc@A%tV2P~?5ai-ks+h;l5j96;A7*Hy%4pT z1UkwKc8H}-^WwJ*_TlUlPUoT8@ZY~j*8%U+f_k>OjIWxJ?60_0nr6DE&zM<`Cg=?0 z5!VAD0r{l|68zUM_2=>$Ef9GiZ!>$+udsPcbwSde3AICaS?#j;m>zT!v6n%$Fo~-9 zJ*ecaTm;u}h0umjxxjZJ^y;rTq^@-0HOK5t-@L6RTed9V`L5nWL5G|0tt=kv&%rK| zrIUu12>7tQHr$PbM8Op*@w&9&ga0UUKRepCnm_{l8S1-ez#FW z++sA#Qml4?_TG1o(;Ngj9S zUke74YUL%l!?@@Z^2idHkl2eJFfW$)#m#@r=W2N@zYkOTuF^mj!FH9F5i?km96BDF z`~tBN#^i^5Cqil0*~>6xW8kJ2Q0)J?jx)qRsU=6+{=g-F2!i&#JE^Fu!1-L}ighZD z`&}C!npUl^2M!F?j^Dj0)44fJGVW5eTu3`{u_NQM|3RO~M^|)$C))6xR2g~Q@Z-pL zFy((+0pw5*4)r}u!z)d;L54UvdAfHJt0~s6U%^lHu zZH+-K3%jxB%(#(&C;5aC@`~PCk_3OEja=}a_-a6GQ%_Cg7?WCtyAKopxJh02V7W0U z=~`?!5?jY09k9U6TXbh5IX^Jw07raNuL+RCGtCEX@ND#IE4*0%&R&8CNC#y`sO z6v+&JiSbjTg+%|Q96<6IYPBA^pOgtBs6p6OW;lD}aEu;Y=nob#2(uSaOyGpvUwjgc zMeE;?XKi$*%V2wA=SGS|0lQ$!bZWaB(q_7`4SK*AruWYU-^*9rp8z(JY?xo{M6*id zhLEa`mwW|efFgru%gaxGu>G05G5$g0`1-!s#q5y&&-)QsN;(ELsHH>lR}=AhABlyb z+ZdW4RwTLvOuDy&RcyceWlv9h-i?2g?wJ3H&-(=G@}D!0W1Mhyv%mey9E{no0BFzy z*R7s9p-meqv8miY4 zlt({RnCf11dy`ZUY|uE@hK1jK-kbLJZlVcrCCTU7{;a;yWH}s`Glp#dLnDE2kGpm2UG>-K)zh8C0SsO`l!DYnj&+`k= z62sLqwoO;{L0^47n&YAXm$BK{TL+G;P*&O~(tV9vUpvUxw2Pxo3mFFT{$(pE-ZIx{ z>nX`xc>S7Lmhr){a=yT2@NOzGd`R*4IP1_UvST}Y?1LxhqoGWqe~$WJy2B>|jro4! zSefgNp$>5+z3jy$o$7D+=CW^guQ!S(Hea(lf@YFS&_ovPvacGCjCVl++dDG5O8u$^ zhsfMlR`cIyNi!;qfNP7bsETdH7)Qxu+VdxUK0Su2t6urhV!$)+%-Q}H-3Lf*)9ipq- zcd&uipoXrkyy)w(z>i4{K|S!pC!r}6`$&0^uA$T_v9Cqg@y`c`$&GNt@7 zhW>X$IM1>W3w3RW%1gBirHpfhK11f+JBDpBL?${gt^9xHZ3k`j;Q#CdiC_GHz)b&R)`c0#^v1oVY0w2SubN8ZPZuIwiWM+(EFYd1@RSfKIxyN95Yo}v@#72Hix9`Nw^ zKcD~vVo1R6LWe(>)qnq!@gHQUH8Rd&BxE?1Yums6ruaf^ehJ`yJyN>bd#wo~JN%xC zM9+F#>#zC#L_aZjs~gO0M3O4eQnb5&Ou%~0VP!oFEc%Cm9PIr`jNK_50T#cYOvgvN zodAKoyV~MXcia>3L2OxCSQnv@uO;6Z!^9VNu&YF=zz?1&|=?!+fcW#MS?@ zYym<7wL2(y&DK3yX;m8bXF>4?u&S@FLq?&=9r#`!M7Yc+X-Hj}#B_B(lZ(msUjeb{ zkgZ?X3q5~@F;Q=$wBe1MpS7<#6kU9M;$kt_$IJ$q6+$zAQr-xQ8J#)LU@#1+=?&vJ znNR36mG?2_18~R^LeRi{HFNJL#qR1iEc~M%w?` z+CB)~5zYIUEQ9M;#FPMbmq{?C&vm++{#aD@ge_VhBYs~%+wBBj-CCQDb+Ki+~9$9-e|I_Avwlm_nKgDr) zVfJ|8bKkwOqi_GTxvq?5`Akc-&#JFnuK)SWYAK%npEk#k!{U@TFRjbgBwN-}o*OX3 za{be03ADh2mjCuB!byDS<3;L>-eJvu+BAmi955O`D}t=L=Wg7(E{7~Y{nKU}xoq=s z+lP110leD8(+6P^y0CxVBw0)u{nHhCRs5@5u>0oH)~iS+E;JGOY9$^+^w;3lO@(Ys z#$nn|Xfvy_7w_9tiZg@FZUww25BU|q!ZkQ>4sFCa!GF41Vc-pAaN4Yt_;0A(UFi57 z43o=L`N)pT=9DD#s2^dGS5T3*N}T!-0P$wr=!Fh{0J~HUJXeO-Ou00E2zzPPyNebf zA7B6YcifF}(5Tbt=Fe3A{RX&{B)*Gza7#%IYM6YKWTV8Ja?d0Zdb%i-F7KZ8^byq= zfJyRH+iX%d2t}ZOvmF@411d^|8Tf&^>u;QXT+h3v6}GgoDW3aH_cSF^ZVb$Rl0p2! zztI^|RLeB5R)m?8<0K&I+LEPYJcOSb%;(P#9;h?t` ziH2=(j(@DR0iA}29PTYDvi)1QgPbvcgEQKb%(EcIf%?IrR7VWSOmz*)01yp9N5}FY zeq>6ra{439MoDMk-r7i=4%A)5sD(DZcw*2zk(irX`0^wSD{;cSCpQxK-`2i2Xj2jZ zFZ~Hn_24AM`-M-o8x+mZ@nzK{uMG&hl7jPA-<_=s+d01}a>CO~z}gj}m2pQ#Ebx(W zKg^+s*P80`_CY~+x>2aG|D{P0DX!$2_=ZJow9MxQ3J_GsA4YOH#P&85`v+xN zw55cmKd_QkS^-5(J$rf*E*5y{v0hD8f7r|iIye_62u9i05_|66Y_Zpt4v;!g%kqst z3lN^9W)!;WuH&ooLM4+Q{L&HV(+?@+ZJ!^hoT^G!k1t{>ESoufBkViFK)|Apn)%RLkjAE*f$&~{Sl|}gt-QRUBbk{dWHYjdk}=7y3_k*5=x^1);sn0M zcuO9Nj{aew$dp&xsrp6T_~{-Cw28i1n#L`=f$=w^?7kuMlwr=>Wv0b8Q2ptnrSW$!fo_T64uDM5Z< zPX+t+$$5H5l~^CxCKT_fnk;A-;XQMUL)K@RN9d!XvquV;U+ z!O92=Co;9!mmX)Yg(E({xDa;4;6@PhG$%j3-1p{jVaNjm5ZK`Jv-kuE6HjkrsLu7* zo`5KOQxK@bm>DbM+n~c_yANB+9#CmJqioHOykQ$ypm8#jfRIg@G7ii4Q99qwIIGhr zm0R9;^g-~4--K^oG`_NU7$YlzyE5+BTfg#?M3)l8*YLeZc=>iiKL@6<;$tt+NXM53 z!UEMDZ4{R<`YzMbuVQ(I>`h!wpn)CHTvLU;c^t&ZB=};7x)TMEyL8k6oMajNXc!7B z3D1{A;AGGtFWl>PDd`|{cQqSIxNu_m6q(y@T;+n16dK5jk$2tlmot(X6LasVjHLI% zX*IIj$>`#JxNxj;1zMVv%S=Z z=we6m0*s&X=9wXfhf6^z)-VJ$_q~!l1uQFr_{>^~;v%DIM z6Tt|m9b%)z{j~jHLb6yhCTZK=dHMaidO?g~!vlT4(%b~oXQDD^o}% zd51~*zyXwR1_myQ>;y?pNxgiYH;sP0d1J2E;#o{}|JaA8C;K;a2?F&N^CDqk*}FZW zNC^U5LLPF|M`c8lGlJHB-nrBd-){^h64&)co9=VY~yu<4^Px-+#oEW48 zNI|>+b+(J=t^YLmQbvH{$`hPJHTkU4GyAZ|U`ex1jo*Pp&rc08awcgvSPLiZ_zU%! z#2P%u0A%lGuM@U<;A@>qnY%qugv9<1JPqN!moDLmksRUo|^N-n+1QshK69Hh-{bKT*sZ^@wb@xI3=P8q~Ss+Aq<1dL4K zJjDr`!QR5qc|8%$52}n9dy#TjkCvD*q6*U=-hzlEjZ?-1qur3Hmm$%)o1vC0391Uk`T^ zAVNy+MfvkwD@RO)%Lw;rnO<=MTZ9W2k9NtI!m2rkagQx8^PWUur%C(P{@)Le=N^~Ve0 zTpD6BLOsbZJ7WrFAU=0Q4re}t@DsabggNgFVxXswZw3wK%T@+Dc^45}Eb<^HrnV3X zLQo2=+eMd_ws+>i{13&U|DwL*Ru&b@_uehB1SQdpDpu0cxRu3iX!g!+G{{+6X`fHl?Or8>cZz+Cqyrs=|2r( z`F?Op9IM?6B6cM*b~`Utp8#)@J?1m`tLlYo>pK;p@ zW)00qAhh(CB1zUqztLXdOH*(@c+@V>o*Oxh2*N{=$)rTHK?oVTKTWY+k}JZ1c|vRR zfTHXC+q1=eFRoo?`qchJ*+}tvt$c6`ixEY-1)t2Hy6;W#TDYsbSFH@=39U^6?3~ne z^fb&CoQgSTC>pya0nTGxj}6Bj<`&2!T4F=vhkZ+rEsuuXg7RK*wBwf7cyBa@yn*4Q z6Fo{@1ko)esHY!a(hWJ+k@K}nHI9I2eE(KnLi;bI%k(+^eXb5V@59fy>4S#Kx@~Db zZcSij4d+@Nz8jA4Y}(d-OH>iAmp@pA6SHyOk)FYx%9HU9JIxHy^Pyf&guVa?HIVmm z5CujmS&7?6*A`mud;cPj%=bgmTuB%1N6Cug!W|-a4E^ee-7vtJz)na5RF%ccZ@Wwy zb%1&Gz>5vQAb~#C40Taf8X`9r=FF~gBoO$W6PR9i&wJU!If>B5nmg7-m4AW-}&(_$q=~Jc>oYsr%lB{H_Gp*cgM0YAqXS}L04#H@GR>?x=rtKV$EEzQtLYA7YTli__tY$+kT$pT(g?H={UV@0M&)rFmTv@Ra$y8^`Z<3Z z{TmAQTq_+oZw!QE)V}#!Z83Qb%f0xE&i#WhVHe2--#3G6G~f&DqG|Z*iijyGnh+H4rG^($Pw3hI5(>6)k)K zJKW-n!Vn8EocmwnZ%`(@zyK=+;(otr{R4ArRtI5r9$5LsudJbA_iMcG^znq>$J+9u z+`98N47~N#|GjLV~A(ADw z(8g-L2@7MguEn@imH?-ZJ415$qDLB*oEKrxW0W8U4wAvzTlF-fd&yWc8<1&%FIXfv z?5ZNoC-iM^d95fzAE3qz$`6kf3q%fzTUm6KA`={eBIp6V7j<$ZL5WH#%ct4uBox z=mBQ+npX{p5CEikl6UXN!VU>YJ~a38ytl`eK3Uba31n^Xamg~{`${&3C!&BDCI`rI zQ!7uyCtK7I?X|;NRYaCeNW(4y(tjg-4Pe22nTRt(F4{WcNOmc;7q$;F@|HsmX}}Ut zcQl-wh>E6$2kyM^tGb);=7N4+s71{ar0g0C{CUZvhYAg=u#299#L}}l?pEWBrjgI~ z?0<>KxY8OuFW_5Ne#Nna^1KU#z3PHyO2J+wvK9D$?>TBb!Z!V&QR%%|2z?q-MW()7 z@NxU-^9V%cp({z9YbBxtLq3ijR1)pB?V9AlqLOI{m3foF({)(y;r+$X z-my;`aVe;YpVtPVehp@x7ZQ)*br7`;e=8pT&opxLSh|m%v zUr^>2T~$<{;s}K|_lQOVcu_lE;2!irv5v&_ksqI@N^+Bj7hzbaK_#YxFsdn`xz4wEX6o052Wx8(#Lk#pDk0>fovS!;t1c{0O0+KH2@|pJy3zt1U;W_73K^ zqn;ZSlNnX>ryo=%!tT+2sHw&F(s<9FTcm)vqCLOh_;%*jtfU+ncH#B+2G?Xz?urCKTz$rWuvDyYdyx&z!K-kg2x}H z3Ga!Gh2!3s(Gc|bltFq3A7!mr@xz6e$+#p3-8QR0!Vs#JEup6-g9YuaJgK2;^AMfr zuI|=H!J-!G(N(rVh*RDajLDd%Rux=C@ zIpVBWt>Ep4Z&!A6sNS9nT1c#`U+2MCEc3}DRC@wLE%-)Zh;La(YTDSPN(RW|zH!F2 z0UKAMY12*+j?fElNF-wJ#~LjhWA|Kz@Wg%hb)N|uBNk=&3x9v$#cWXsmTnqx zD*Um`-0Tt89WD7qyzFJa5y)vj>VxP69SEtdbs_yb{2qqRA3+j^C5qgmdsOR)f}40& zpFi^Tya^mRV>e{lJJf5V0Ky=GtFnWRjkw~EHS#98<}{R`yf>_*?L754pf9#_&;&KN zeaB=TD983shfiYd85sF}DYtJ}%GGQLQm=MXaP@H{#J5V`(@HT$-?|Zts$XOc?{(jQ z(rG5K%eSa#=b2srr6M&w$7PHqFQ!9C=GuJddmq*;vSVH@w%+{Xy{jn1K(asEEC>Bv zqmjIQL|m4DkynvqdGG796N_`NcsVi0WS2~g$nh_#5Z&~hl7i@wQrn6i+qPAN+6sbs z1tBT6pAEgdQG*L5#IbQr$~e8)~;W zFaZbWs!uON-#rqGB^<_f47%`n&eC*lH7s^7+9Qr{(s)G??|kw;wHdo;nk6B87(Y@c z6S^AV@XeDUndLxJ8CtLBHWa4aT(()S~%it9mN(DvzvS{ zdT|5+C(h%(WOy16SJkdA9emy+P1zF@(RU?@=V)*+$*Ou{tli3O8!S~!D$8cEk#*n< zhg_dK!ERH~=^}pR*~eg>E0bq0%n_L+Emv->`7Vq;u8{h0jqU59-Km^Zs~=PmMy;b) z{~LSn8P!A=?~Q^WO`0Nt0@8~hASfUp1f>ZG2uLqMdhcB!6hV5C-kV78y$0zWktQ9f zN~pmAiDB{%&vVYXWu3e3eb@c+et6fc#RoF8XJ$__d+%TSe^rfV*?{AJ4y($)|1y@z zT!m2Vd2l$6DE?ksm>-Mye>B1d{LeRXhYtXIv_84LBI^>V65!i}*_owCd`R(}aqp%i z*zWxbpV8EVNu5dq5lLh?yEPSTtJfM?p79v{He})Q2!>mT<@3q{MXiDvE8xodG5WA+ zVWNHiSt4{fYk2(Yg7mwi$A96l6S_Nv)tC5GKfIgVw#yd4CE+7UuoAZwpO=x4B=)%1 z8ah>bw&L9cVhxif=WX--<&30AHWX^vV^j5U>+hLXI2aK)xz+y7>f?Q{U#|l8O}WHS zR&-F0ll|jf%6c>uxsf_#-(h(MX4;!^sec^LL04E$;=kQ!Ypr`o-U=f%+JJf2s*V5^CJ_UFhT&p(XIM!Ifc%ujb7C0n}21o4Fe9nyJ`>Su@xH z)wns*#Cc1*Ichl1^>yWJcE%R|y4}Ym#=|XaA9E5LE)Av$)q@pCQ-;2gF@6}da+@F4-RoJr6W{jmI+Gq1t#1wR)K@dBi)=@<0 zCUo}+Wm#jJ7MNkFJDO-UnmT&k)jGWcBd zs!H5ka4n{_T)1;Dvntie8t|+~Z7-HrKX9W_`;yRg3!aaK%XfjVMl}8h#_H*n_osG$ zJ-^3|()b!W*Q@#^ptfHz;+RL{RQDT3C$@c&qcyck%XaAe&`jsH;hh@x1O%k@t9Dbe zd%ijic&=6DT#jjqK{_*G3tS%-(?&eS@C;zs0+FS#qGz4w^5~NN$&_8G^H$3>e`acz zzj`4z^9{-+vkM@&-wuv00f=YIZ{IWMLY0MW9=@^n97IPz(B9t0IP|zEM^RDXk)PK0 zPiZ}LTxJg&Y?D@#8aTcjhepufaJ*L}#>pS?;Qd>+fftzyzg3k~%*dc=;kDnsX#DR( zu6m3k2idPR!mxS~hRnVp@Nn`kdN)P46RB-k{r7;A*?pPqkmjP>_B)4w&G%(e^L*8= ze>*q9CuI3yyFtJQ8Rx*fsIIB`C8qY#^Loen%H<@x*cRK6pmjGXTP3;uV8dw6gN#P| z{Xv#TcZ|xMn|seJ=e5sWb-lv3!L&h}gyPM8^Lz~AOWW_55oTbW*v$tyU~=}W00Yk7 zcek%HfVb(Cy!pY24s4dWG#n3cAo=I6Z6`ZQFYxLyvd(xUG7es$h{%LEf4KmejkuBN zT-W!$3sF!5O6jfj+rGqdV^N|Mvzeb#`fhhUx(VJy;h!&`IfX(6U-jwPO!IU;uvL2X z^9GI3oy8~0&dLV2v@Uw&qC-VvzW>i0q7ZSJ=s0C*)<>EcnkgkNOU`O%TmNcqKF$Ao zreyI;=4ar)f=+Lkbd5EObmk|1{qXE(_9Ojrj;b8VS?=IV2_&sMFOPf^4Km7QLC8!SDquisETVGW};&McagXZlU-=Kc!K zH;#A{`*0f+oC+W{X3uu;wN!rmUHvPKzTfUYs(v|RzFjJc7I{;f?K`Kx{TPQ;KS|Z-wLj>2R|KtMsgVwa==t6+5JfzGFFA;N2C9 zjBb&8vN^h0h^VKHng4S|^p*Nt)+IEyiR|URJr?)>>9{_daba=U?3bC0`s4rgpcL#= zhVSmlEp83j@cm~Is#>B7#W#+VKmva9}?XpPgTU2VA(eem415l z*dMZfd29h>HkSHWq%@1OjfR^6wLlF3#+2wr>9Fd-u&Ori;1SA^cl2VGz~khu1Ppb6 z-Q{fNBkSCEPUsNuC*rp1ySpH$CB1of^5MHRST5UIqK?<9Z)(f)&{r>_9Pa(ot_8q< zhHW0r%ZPtlk32+=TtDq&U0G{XMHMCvv<=N^5Umn_!wM!;`rxI{@(z+XnXvdvpYAmNEjftkqI&JZJwLB*i_}u(W-273XxNWgvdB-`FR>cbu`P@PeH~?DgI?RAHzR)g0SqkA;^~c zIJrHWN?-cHy-fdK?N^Nlp$D<%0IRn_&TqkYtA2=)cookI+?djLNL$6`x}6wOh@;~Yhxi>Lu=W4 z*Za=zP)-@Nx(jNT1(n?->0aOTy3HDvm6Zj()fl{|$JxhXpn@F+xgb!iS<6GcFgIFA zB2)u3MX_H^16lyT@neGoULrYr-^;IvILUxZ*Ma#i6KPz(V3HObx$OqD?v8rr@TW3s z|CsE0?9i_!ncNGO+xrtD%!*hdM>>P~l3dJIQkx8G4M*{ko6(OWfod^#_`6~6(SW~4 z(EF$%^O4#?skVJVx7F7@JzPX4xMK|Q5xdXjAOg;OjuCWfib?rA_NC;hDt*6L>PsGN zn{pc7#qjC-A)G;{VxrzZ+Jt+`qwu?JBH*lePrrSFU5gTAW|tNMr#E`4ZHpTt;ey8G zx5ynmU&w<6PfgQcBe6Gg^f?|zn=)2Xp_XNE%ja62e zE*6RJ^BH%!ff&Jb52cB$bh7(s&LIbyScj{rT?MR#rN78lxr28n>n)Jq@+3>)g2kdI zLA3(%?`ELnW^9w(UzFf%N?$TJe05w~@k}mXbz7NHz=GvA0*Y&U{~VXTUB*|%OK1H4 zt$w?gv3}S4_PWhTrqP>J;vLJo((NfA7o1?JixwGMIefQP$HDAyT_p4H!C)@bU9uGw z`1vmSEqI~RJ3d%CV;>FlSu@-*cOPB%9=j=~4n`=!K<4)qtDNNu*7E_M#q5wn@D-fcZ1XQ(*3m=JOQ!hXz!eO>T0N8 zztdC~u!5=EX%YeJJ?f+;ZqpgV-G6#-y_*$yKxcHE?Y7fAndDvOV!V@$+sVhk1rY)9Lju?QH3i7_OsLFULZgjehs1EhVvhxPYV-d zEsK6lt&?W<5#Hhl6q*YOFw!J0VcgZP%3XVE+4QoJ?CtifeMRzA*sJzDBAp62DUHnEYY_13AYLx9w!zB)%uJf1MlGxm zP9v&1yBK-aikkkK1fP_xj1Vwm^XgD@hNIIFe4{4dFAot{`@~&fG$PcF3wOsK9GNig z-pp{Ab>?;nw_SKft`9CBdEk&TdCS~b!gASlD^>}?bfVGi6P;g#`2L`GXzJjZ*vzi& zW|589eH?vLy7}*?vkx{Y*}VhRfa|x(7|>!|lZ+~Z zrT?)bQZVoJk0H2_?~a~?n&(JM?+{a7A`CyYM*)Z#emwZywM=V(N#PLB|8#Rrcgs8y zgY5Fc=xz^}osOc!mS0auo@XHo8Jq>0t6Jq(B_Gx@$hRN0D(ds9!yi~4!Y8}7xdS%x z>KqW3NTE(nuTvXc^hKEy=Jtx0Td&%h$1iV=gN{=Hwh?j_Ys+F<$#!ke6^lnP3%0x2 zMr++&XWs;X0EH!S?19-$!6Voo_GU5cF9qLmM*vaPJ^}qGozGG26!XmAYdQu3(i{y& z-o1@xNM}C@oFv^9(me3x*(PGG=ErRCWO6E0!>Qx4^dxLFY$f-h!7- zt(dm*qTIDtUt>74VVxc|0>XzNwm7n9=ybRPM_Wyayx>lNjLXXTPDx0fc(q9^sk{yE>O$mIkDA459gdoOQS7NNLi8 z3ms19Q-9{yd3PFoRFjLyM&~9Fy&l6wM^0b=r-QbbzY7-@4z-w6$`%v-;_}+beFTbQ z;`1zovvvYS2KQy0`nnL+kxo6(wPpLi0@MLn-@f2lG{q`-+gt+%S&LPO-*~D#9$dcW zq_~Hnfhk5|%PU;Hesr-bv(GZR-)Lbfy*j#Y&D9iOxEsyRq`~{_`xQ1(j! z5U1SAP5Aos`^};r_VNGYfUVq38Q|`Je*gjSd)i%Wmca;9<-jHO^-tJh`WoiTijP*S zBZYqG^m{h!ZxEez1AQtfpekX_)I?CayYd{lPn1cvs0d6i8NXXS3b5Rz<@=M9+sSs| z`s3}#xwN@8a>!0qr*h`EcbnH29Nyz>={;pc;&;;wyANI(!?Zovfmi~xwZ)@Y54qCg zm;ZUkP82{uOQ|I7k_@snn;nRa~(lh@CsS-LiepOp); zsXC~uuWki=lhZ7xh@a00HUR5aEfLB4R2yjoW=pzVZe#dFg6Ektai~FrnvcIY@|b<( zGzZu5M?)t|_O0ZCSFp7ImX-gIAYj^l& z@8#=J@wwbEd3p7W?pkrau)WgNcX|N-l;XnPsPo7E)3utCUF{6sZMoLLTv*mS$0eat ztk^>zAZBX|$2E>C0`RKmQ$_24HFbNK2#J2W=UQ&?u33RuBL(l0FU_g&(bK-Q@puAp zILmzgbi9WTcc6BIB|EUid>V)Ph?*aHYofFKI_M{6y86*wo+@}-JY|(JuZ$7w`rQR# z=)>B**m>p-fI4_x7$!&aR1fx-EHS8!0OcJa{o*3Umak3Mx=WDE#t5`$t?DN3&QkLc9F-lIc$J7XrR2$P4BwxDm|IPEYDN#mPnwO3`&4W6wjr#`skJc5d6FAW0)w zn9{6=sVC@KLL}z_lM|#T&OVYXs&KyJ!L%)SDj~A^BnC@ExOXnkduJ$FvL1LlHqMXR z8g6QPPQsGJ!V=M-^yk}KvPL}e(EQV~eT`dhubC#i^w_}Cie4{yn<2kZO`QNU{YXP( zF!I2gUWZyL5pQZ%yVlc(bR_(pryX!(EqrIM`E0=KC*NenD4L< zzDaAmbcZj}@#5e*rwALr@@;1IC+^+lE6As+pmEQ(+x#WIm?MLZh-N@`37S$ zaq9^=eV&#hQ6AguBTgK75>Z4x!E%G6-t0?c99p-u8x!}c_TkM?lx``j4gDjT;>Bo@ z$T#H=D&R!QA${NQofbta>^Q(2Ugx4~2tncgI=Q5GU~GVS5>qwKb#55OQcHg^iJH7= z3WT6C6#+pM!dT_2Us!M%b%pB9;X|Ee=Ke;CMb5KMf?EQ7hV#&*Z}KuQ2KtK&ZJ#mP zleZdQD}TlLPPqY*_fDMOmp70~N8o|k&O?xp=NiL?>&Y==5Lp83^OfgdR3m1S@F!+| zmP+FV9X?tsu~dhCE|=_sNRX~*9dPg7I@O+l!Q9%&Jri7`Ut#o3J+@pqJQ#0!MJqQH zS26$n0WJ}@1$WPwXHGaG3+Ks?a&aDoRlx=@<5d8E1*hVum3_+?>*JL`z+-n)++YhH@UEpv@5hk5ydbuI0mXc&E zo#v|MHMbKVKW&r+FZlFGgChcFuCh&nF76}IBf546a{LJEV4 zvgA4o5zc%wqG{^0TpWP65P*vI3prKB;@<$TZKViBC0FltIs|W72osLCZy3FB6PF9* z*z1ROwzJF1SpTW*h?CeqUa@hdmrCCYmD=@o2#oKxEHF<$5oVEmlG(r65oZtq4@~PO zXd9LTv<1XPWnM;eNu&y!+%ft4WPvm8RN9UeN7zSr|C=foanOL&_`DpeRt zUKOdIXpDS%RCs46_un@9u#fj1;j$P^!voCB4`xAft_Er2iwk$-gOyRE&cM&`W^zeF$!s22up(iW0LrU_|WEv4a>J0a4j5y)=lq592q3ufda$hO+a3LlR?5-pH3E@ z?u}bFR+DDiV&C?g(`^^nUwq96XIJzed63g3_cM>vj@J8p0I8V>8F zdd4`(POI@nX?q{j03q4M#zvbn&y*c9Jw1QjCA&CKa?%6j1gC^p*3O-_2_ ztOtvKs-eAv?8h+Vaj+Je3D4(VuY`EjkGrBiL^s@iBP+@(Yhs_4F}9n~R&BZZF)2|q zlwJh^J23XQ0kup&u|erK`j}r$iP~5+*eRZLi|VwxX9&YCQv=wQ@sM4?dvZlvm~vZIhIwd3dFBqL5cryab;l&YI)KVoKc1k8 z7g-VdxWGQMfi;MN-EzDN;d2gw+kH)h%yBNMkB45XXmI%jM*h6p=eshmqOrpJ$9Q_6 zhI+vjAd8pXtRZ*tlwt6S6=b1Qt(iIOTf6x?TzBEeBagf*yzz9>?Wopc_Q&LFB!LlD(<`EYV4aSQ1MS8yI`Z{%Y4l%L4c)9*5&*)!zn95gfz zw42QPnhwwK%T~Y*9%nI!QO@T|qQQy+(K8ofG6s&w_mijX!) zW&{{&HSGPN?G}-$dCm;ZooEWPoMdBpnrF<yg_rY&XF(93_km@^ZxFPrl zywmEWGokNB*&pZ03#iW4mE6RlYA`c;S1r$^_nDyvohafx9M{GAL zAdw2?1r_j8537Qw3V6!FEdY-uo?Jf%H}yELACk*Rss=p-Cp7LU-5}262*+xD#v=g% zXBDHDN)|d(iDKKy#G=dysyx{nHRSAgoN9B-@+5&Z;1`!e;PaMH`LE%qq)&4MnQKyl+rQIlyQCLZ6Y;Z-fCqV)aAPH1%ktSA}&xI3wZ#Sw}NJ`R5)x`)urjai=TdmkeM6F|_%nHnXTr-%tCaP_Qi30tE zx^LBQ68&k2a{$S1rV}cj+Z5fAI$1k(x#>fY=P$(YW}@&@(wM~Ziv2A@>?#k3;DIlu z@G5NjZ39jcz-h(DFnS9D9o}9hknV-2dn$tPOC(W^9G}{0R2)_-P~u4+3qy_O-9zd` z*1zKwl2=H~lR!=b*9_?Ut=wxf`K&|>fypB-|8a>4AShFS|8OdD8oSZ1c&cvlUV7ro zG1l7XuyGga4i$?Rr_9)e89XgysJ9FpmLe#9X{mu?!}F=5uBxRd-{7u zLYSO;IEKN|ibgA>dKjbh$q)Wb{f$Rx9&@G30<%%5(Ig+pV}Oh3>XaLExzDANg@-YG zX(W&q;G&?CkGp0aBRQ06peU@_0d$Md=}mvr;5xy?&BHr@^Ko^o2GG}%LOmA zmn9nMI11LAybcL(n{R9Oh~nH%GmjOzu?DLJ@{wX;^&m;PiD`Z?2SL+62%LMtks%_G z`CMS&p-}%8sln#5zN2=!gMk5pM5vC0eweDkZmwkvO^XlC{_aww!E~r!i@bV!=}IqJ zV}0qk?s6HCtW9Wf3ly8bl<&Qlnk{xU;g*o<_{3(P#njr?X4x`%vjUQ!&xWTo$GrXO zpM8`VbYb`X$7+*(Vw@lQK5j;V@@G2i_ZvNQ>{08&Yq$iMGKkIF8+m;g;iQCUT-BR5JRK(=EFo5+I-rOz*&t?^^ScByjPR?bCl%h@lbx|K~@-~348serh3 zZ&xe{@9jF9PO~{vy0LbXh*?9r5hqH=kX5)q13 z8M7ODgky{vNPnje^p7`nal91RveRt+MrlZXauCd;ID8|w(QQ}rFM@~=RWI1|HLr@R zJnHn^Cpwbmkxm7!?(0U;1a3JvbXOmBY0gh*&ya^P{DjW zFFqcv=)v~3T3&7Umt! z|BSzU%e=4~+p-yZqxG%w70sM`=iVH1V4lsd6dw%~b1|ToDgn|^I?tFiH#5M(5=X0l zzx>e>|IM{G0U4x<8z9ar`?2H7>*t#4PoC~K$GzPV7u}2IyN$0iO(8m{eNHOB;mm=c zzWHJ7#~HqDBs-25-tiU)VtAes8TV`FI};+e0Ppi{mN}U0b8k=goRVAGJUq$q=9l=U zb9F%A)oxeMpjz`QBJ%L-?RtZR-w(}1&6kLaG#L?bJ#67!0WZ0p7Ms8Q+jC2Khuam)4bujYMa#oZ@sch{WfP62RX`g3`UpnO~X83_h3acZ@$_ zT<&xGpVGbFV?3lgV|TNzfKy3f zO5D2fi5nQedek7UCMaN__mA^hghBKG!>-;`naXRyxr!^;vc~tdtxPE;b*HOmEkeX) z&=IuFpBbAnxW=|#UyhwY>B@^u96kM!sgwiTe5sg37Wc$`jDOwha}}JoEdlo5qv{Ok z-KX&|1)~2S2bi+>?>S3rSG5BY^}OSAqqw|G>3CCTl=kaF|BiBF7=pXQL)x>Rq|J(<3RoZxyU1hSR8t4d6AI%y}T%VXf1ZtjHN%$^U2MZXDoo4!{<+f{YV zV0rVIYColeqh4ne7~SYaJ&3*@@7sL~24bAnB9SQTfOPtlH?#V%AUbf$4vw>F-Vq68 z{Vmb|cwjn!AHNWBh0H*W+Yi={DV0r&WnYplRvi4(`I9k)H;|^&Sq+&s7>Y##_}{*# zJ!{WwIi@bvas+)m867M39DC4{Qn(Jc6_DARR)(_UfIU~=Z1no~|mq5vOdAvTwi z$zZ8w4Bjq@OQ}dba>(QeG&cQRn)&Aoo5Zuug83KDIm3s^*S?SWpN^sV{v_hv!nKZZ zfQYb;?|ZRy=?aLB(bYHUWx%WA@c3TFM3XBq%NO55ZQW76CM8S_C`KX^IxIp=vfl^7 zk-x!M7f{S_o~~=eDS6mA2wEk9SH7c$tD0TdZ9&12P3ue96YmV}%E$`b`3{yFN3%JN zio;fN8sFudg4GtBj*y7R++$!LEIv69eAt#U6QS_dB^;i~O1zi7ygG`GfY(QZqM!#f zZk;Yx7hM5z1eR8}iHH&E(6^6YSHOFUoILG5n1gu5lyc&!yr)A2#49LVSTdNDa&E1w z$0AP;#+iAT4zdIRG)*oHPbQ0Jl?fToH37l1O|Tqj|2QlMmV+zB2#yB7dB5Tz|Jt=i z_)WFK&`w4*EoI5lU#S7PwV$7CEh>zuN6~y8eQWWOxSuEHwjul>gD36WEk{XQQ!g$Houel+eIjB_VDtOG>4-o1Fjq*n`&3M;1( zsa7)0&1650oQINq#0w=HrA{AYa&Zdr7H_y> z;4ZU|k3RLYS1ok&Fvgn30N_B;2{S@sX0v1g&&+tmRZw8JWF}v$+C@pJIb#xat3Hl9 zBa}t0_w|yy%v>>dj@Wi-SIdT)IE_E=$j%Z)-$$FdypAB7bI{WIA zeehEdmPb|>D1aE-%e#)DcM{auSYQu1z6%gOR_w6-(Mo%n;BAu*qZQeKZf0sKn*Pg6 zlm~lVwXfPcQTdB%HJ=m5Yq$0P@_A+&41t;%#zoQM`U9|IkZSe+(_LZAryYON>CbVl zcvnxm+wYF+R6ZZ)LWUr( zhmo;Yz@nvkl68F@z(=0(poF)iBSorUX6_r3iA}@%Tbu5Nec*p`7aRXAcfo+ReoY@@ zp__i2{u)H&YMQhl@`kZG^4_UB{+=jHXovmUc;VaNCALvGc>U`^i54f>u=_9ps%N;U zN??2wzT#j%)4eui;aOr|PS&M!5s;$L(`KK4b4|fr>2K?_LVR}IlV~leaoSDJOa&JJ zFZM$6_Ev6@+HnEhzhrF&jS-BIunO)U>R`||w~kT_<$TL_#{|~6{}Yv}1>#W|^diPA zVOvv3+ae%7AZI!+Qqhq4E?LE|EqrnXI_o!xYzhGfzOYFAiNS-h&~aZg}y^ zv4pR3J#A)Yk<_(-tmM#%Rxu=F0HmvWF@^u#E~`_RX|iOB9j!*vBP3dq-o+@3xrQ3{ zykIK7=UPs_QAei#W4m!E)T7rhZ}Jy-jd>#8C@ z=v5X9BKAm(KwHne1;oX=sqy>eCX2@CLdwnNEK;v7WZNVcMquiL4VdHeJ-pmqvt|A( z?+t7Y8Lyx4QAP&OPrdWMJe|J(4lIwfWyPQt(9b#dhUCJP;NxgP*^f25Uik=xmw0W4 z#2RIwTdV(={^CyrNrn$0n++FL;TLNJ4#Xg;o^ar{% z)Vtqf#o+|<*ktay>&`3=Mu4s2pLL9&gPrNq5^d-h0+cSuW@tYtTdW!35SAElQo3=A z;ibOoKn+nN4^q81bw;g{?^^j4oq=Y!gWpNo2ibfMc%~D|v($<87x2ZU!E#jji}BuE zks|uAdw0awt{4jRybviFxY|_|4A{#3asyFK7o#H)&oiqzy8|xY^=bes!o%N)N%7q* zEt8W--4t%A-%8v1lkcs3LA|Pt=Zm=7>{E(UTv>U*YcRqoaU_Df>$ys_;~ZUN3}+-( zt9>qaR1n^gbrA;TtM(tapEdJ08z^(y$2Qi-Ovez)KkEQ)KiXnG)*ZXog;!=!wuohr z#z~|&cL00wAK2DGdLayy5qAK#T@Bc48iyEc{S}dN@H-^tH|Ln|wQ)9Tt7@&J!%U0; zx=Hqy80Fwn$GPxVG%YcRMzGd?pcsP)Qff#sGAt2r%`ty{*=>61S@p)mv=ijZm)9{j zq__B}ek47;0;+wk#JdJE$rK&Hx6knu$vw5TQl`SGJcTG-rhA{eM1$kD{_<^w&9gJ3 zhmiL0GyCmtvQ&I@CO)p95>z?+TT@Iw47Haz!bt~x(*^pp&`IBytiQeevI_?=h|%l2 zxw>5^G0s1XPWRT(yVu?HUTz`5+0#~sBePJ=B#q0{oC#GHCuVlqqS*Pv81|N2 zHIW<1@Yn{W{_jNb;ctXJzBW<%csK`F|_N86BET0+ujC3fz`fg~RO5MV z-N%^rYpiKo{D!Kl`}0u$6a|ytM0>fbjP2;1%uz7eU1RmEsX1jb7u$+dXgv{abo_O3 zzOV~ybK?fpeY0!3mtZ*0Zv4NgU^N4dFBuza8HUdM&KT5MO&Z0xGfgu-hsiGdax-pN z-e%?W6+he6X(nO*2MkYf2fuu<&u|k&bGWSE{IYeZoGjevBJ1z#fq>`Nk>Z0FqO8nr zQ74p@)qZT6kp+rPuAP9KTxQoq^!+BCpN%d7LA8TpPgd@V#%3IXjw8BPpj%nmM$IQ| z9V13v7ky$p%DQz2bu%5;;Z&X>y{CZ7`-*g2}D+nZd@ za?eU5r{exG&A-1ySpOQWI{@Rp1lE8#vC+^}BTcZ%?lJPipj21KalasMF$`6{9fHK% z$6X8#CwkMRJf4(pH*%Yj-yUL*dpPMgQC|cnHznRtOx+brM?sZLagr}O0E!3wuGZY}?A&+qLHp>mBMe8CD^Q#R`XC&l_(#Nda}0q! zL0nmu_nG)M8iK>RZB%#W8*wvVf}s-U@4Va(EvtP(S6tp5AWU?Na#0vBC+9L&%1D)$cAD_=YBITXxlm$JA=&7g|{v9 zbU}T$tCMJ#j$Y@W_N|Ah9#rs81{}JN%2Gl)uv_ZD8{gIm7@9V~LNuS*Idcv5$F2of zK(Ax0J$~PSAmEmoqpa!=hwrcA!tZJ625T6%b9dtExR)N{|LLR5$w%L1+qY-ojF7U|}P^^7JP?R9DBmqTSw1gcK)-m~1U#?)aa~KwC@ef;QKzG95>~ zmK=)%lR&p-QH4QkS;exYb%;}cB<(gB$W?dbEO|oV%+p+@uccFEpfAvN%=VWx6jbz8 zzyBQtlt?Xc`m5s1T2{?)YX|4XOHL@I%}&v`za^>g>&0I_iuHb*Fl@xW{jAE zcG?Jzf(g!4a<+phXYujhMtzhKdUDwqi1qGNY4BYL^eVrwN7CX&lbLXa#CrsaqgdKke@}d7Z~y{l~r^) zs)`@y5sdy_!v%rDdH-fc_p!oJ%!0%b!>=(2Fs&w~?UXl6A-!UOwKU|fM>j86A?h>( z(tg4^(cUA8x!BKj<4vQZ4*u=>C4#<7mCgKyj;$40f94WLXmT$4>SlEeBekVajZs?+ zJLh{us!{5M>LKF;WObo(QCy-j?G7``TX68ZPUm)0h~ZN>YCa$TKK>ip z7J!g#g0l+3q#bWMw-<|`V)==27c_x!kOm_SSYOy;=NSwT+rw4*#^(E_kcj%%zu^Jd z?+2#e7YxBY5%a0LkR!C^U#4b3LI2+>+hIQ>&`{ZqL{#GHtwm9>#XPEj%ZYEBXw~m86y18*wDj-{3T#714A;(@kU8G) z;(ZSFw%=J2aH%Sro1&pFJp($zpTHFxHo0*}{n+X(7hwkez1Usm2hRaPY2fd8(?B54 zoK<9dHi^u}jKd2J+Sz`KZ`37oduk8n_RntD?$MiM?R<DzC?mpWi) z0Nt^QSOPM6f|>%_VsyVX7g9a`8wDQ9ERfS`km8pu^FkhIto?^_#pC<`Lb+av$tWp= zhKMTOgh}>U3U6(!7!F2?h;34bnXS|q&8$|ZZ2sqei7mC3ikQ)$?E z0>3z<#(j3rDSi-(Q@u(fXk8Dk1DZUyGuN15!qC4M^gc(iGWU(nkBZv_(cy>!0cNX&g`o$asVd-QJ2aN-$a3>i{h z#j9)LT9kuo`f2*vzXSX0E11Dzl7s!soveO&P&@)3p=h`Hz>zpa3{VuqmR#jCJ=vHJg@##8AcMkrJ-D)FY_OJVF<-;`#<(V(4kH*K$=A@%K}_WtZN17!hu4kRQ7W7|8gUC zjQ5})-(mVgF|+!tI^H zqyE8v*%$xynGOxLU%66m!k%}#D6NNIkf2XOV3DfuZG{F66St$-#MoJ zXXc~y$$xB{Te*1hz{({yG1F0VV>`iYxzGMKMTT>Hh1N-BvGSEg^SfdxsA}?-e)ZTQ zpf)X%xPE|LO+FFqTaf3;b#0VkdOj;t?loj{Q_{&@F6HL%KPz%jM2a$PdvT1hUd`-( z(D7P-FkDT34UIa#8RsL1R%~5pDqnFJb+@`(jhx73W?wTN#NBOGI&XLORY8xW!XFB< zGP|do@E-XO#<91zAMaxC2932oWDmn`qUOJ}0-vDl{c>t@?;5^&C&ysK-|O%79ni^% zRQY+{Ts}@=$GW#!CA|H^2ld!@GGbWWJ$41}fUbaqQzGZhcS|qT(?^Q&Fpyn+g!`8{ z@bH`^lY2w@p!7e~>nP%Cc$-A)SHz^jV}7e?V;ERpqZN7coA*!!($Wc&qgq^5ge8rG znfYQ))I;pf+L4%QCeOq_*ZW7S0+midMW!yFD^;U-$mRg$q>n6rk?w5 zO5a+^#zS^8+2Ii0AL+z(g|=^hAJVJ`wMLPOl`R$Uh`W?z^SL^z3@Xb5VEK2DvNVAm zV@6Xct8Y}A9b;7NKBp*!Sb%qO2!?iTa|-&_z9-InRk1eUc)t7|=sa7VR@13m&0NPQ zS}pbs9_0~uH)K-6CXV*afE(wBVIX!b>ijR|I&3?tn`rQ5`{`bO$rt=KccBC0imrP! zm*4uGAE@w<^KKhnljyi`5h)CD3)pbpDw9M`CYsfVc?5#g(6_gjLOKm@qZq?ffhq_s z$rVEokBBNs>Hs!SVEN&rp}$my=KlT>6+Le5bd_+){Tf-{AHDavqWoW3%tJg;>Q)`; zFP4pufzhzD6S3KoL02SZep0d;M?s9Nwj%m|DNqI(gGY8}Vd`i0+|-rCW<$Af0l_(Z z4(Og&jtkaw$5td6^FsDVc5_SiztRRztM2!q8R}FuKQJ`)L2PD76admYG3-o?vtw8q z7CX)D_ih!89H9I;zNS>E0+KFR0hbbM0Sc>TUPLVi85qt%*JtKw=Ag!H*&If&#j@$o zIf9wSm$UB0V)k?h)VvuaKDo6NdYxy!m)c;$unkWs zeutgVR&KAEi?*JqC%~_Y2ORrSM5vaA>sM49LOqptB+|f8 z+d&#!EyUon>A75LJiP^x0}Tg{Y$FQ0pR_BTe?uwS!qd=##PE_hXnIH5BqT0(Hy1nq zu=|r00v=EYodf*jEx3{soeB=I?;O@^1MuF^E9a-WyfS@$8E#{V+7Nr;AP6E5hhc|6oY<-+>u0%UB z%YMlL^-y#SXksTqeO}RmOchZ2b^AI?5ne68K-N~Cc`BD&^5S{v2Sx-HUVRFRc$uW` z&HJk#a59;Rt_RhZwAsj4);H_vFQG$ownCk_ggA}6Pw;vIRi4%Yy#8k9m)FITL=;jY zW3Cw9YMihz8Fe*?!>-JZ`xcDs=)v|Sm#_)1s12UbdQR&)R7;q&rPq$?9BDEij$Mj8 zjmh8pQHgIi`(Xpy2i0fG{Lbb&DCd4L;klLwizGdr{P9TOQCLiU7|NPFu3!K&3OW@9 zt06v@{fO!|&AmF)8)XMZeQad9jYy12Q0Y7rab56XofBolo6u1W`RaF5nD!rWu=l}q z^XT0Z`;7U;h&u~iB*@&|H8t47G_SqXq@pkt?JQ<9jTMKz${|z?)g2R@%D@F%QJ50+ z5}(i^K&jfG(JkO>k4${$y4*jU`7H5VZr5=WZyAc_R}Ek=O!!m-!onO&u!*G=Z2A>^D~$A%Rs-N0kFJ-qO?mbqLIly+OBLxC4} zik)Aq9xo_|O=F>UV&05#e%tZoAF4BgwQtAmJMP#qNr=dZ`Y+c4q|b z3T%)<)&z#cB+lo8`I`ODqe%Zpd)FDz#I~(LI?{WQUPOwBv`7nt5I_W^2`aq?q@y&^ zP($yED7^@R5I_MDPTOTTseI4+*#|5hkvNTdIw3%n=f?wuAJlG)5kkPj!8d zAUe!qXNG8_md4DMN$-wkuW{-mwPUQ_3D#bDC$L(PF0#gaj#;d~y~)Oa&zTy-Y8I>z z!&29VeL0EGX2(98N@??nMC(gbQeGz^Goygu?-7-eFbfx|iIBuF61~0f7Uw_so>p8x zsFiUu2H@Cfpc%!JSzLzYRcgHKp+n)2)Nvju(WSQ|iTq4b<)foEbxp$PB|;1Ckxj$Nwn~LP{&DP|i2bc+ zObET~7rD8jC=M@JG}}qSgxC!Xe*Bumo6Duo$wK>-2lS0fLYJ1}%v8FYxPLdcxylLd z9H>P%m{3d|8`(#D?mjZDD3X}atxt|c$^PNogmkrxvLHFUYZ%#~zCxCutgbVx9vUp? z$a$9gGT3v2{nj&J7rW|3=e7mDeWI@Hyysu|)KocN$CLa%)^4@U?r*NdPj-g>HrA&9 zRORokbh_HL{5F=alAGD@uGrO9&HXl(Vz!+4@2*IF`scF$Q+L}d7OM*R%sXK>;4(~b zVkPgKx|Vn30#58EWTWIp)K1Fs!1%8+$=EFnMynDHj z%+N%(V>gQ*!pS)&d&Vwm?hfAo>K_lZGN9WIzlj}2Z@RM%D+x@Lg?cffb7!}-Sm zRk`O7F^!iOZUWoH(s`W_DrR09G}QHqv>_QPz*AogV1g*u&;zFns$}fL7LrmOVwDwa zm2ByKpUAWAq|%gROgp|kAQ`jh!-2BkETJIkF5)S<$;H$j=-1)eju>0cRgWXp%a5RZ zxU2MPZNY>JqmCN}g&f7*V4~-0wrd1Gd6DnGRsG;aF6HD4{$x8+scMwkT*a}AP&25L z)j5BvEKSCzV+}*bdAgbYU@!pQ;y6N)ghY2sKbRG0)jVrGzLP7XMa4B&a(@ z5CsykM&de9n-dckat>0e!4f&Uc>YlrN_0iYM_#GB!5OvGFFAUS8=j6@Vfy6|I&nr0 zOY%ZFs+sS1vm(8)*|}3u98342}f!HT3HZ!I;7>=bYA!%nLa$Ky%8?J z^iP+6;__dA7^+FdP1L91jFa2E1r3oeZ+NJE&ytbv;y&5^LkzC08aPhCsp+d?c{lPC zRi>5A_3&WPozfRyo}P1!BHn^4Ut5T7QnTsqagXBN8Rp&l7o`3i0%9W*&BuX5fkT4Q3hCC{{?>xR?e!G>MOcCvh|NMi$w>0D`z;yOC@ z%DS?@@Xuzt-4T2yOaO_=I&4m`+R!|T&&+)6C;IayqFi{XHAyrk_AjGl;80aq7{CcT z>?&yr%WxyQO!3xp{Gx+BbC&PZpv81Yc36hW(3Ol=)3DH`8R{-xmafr!T^AQ(N24?W zUgAS{kZ|aolv;S|GmenSN6X~oE$=uJqOJQSuOFGlC=^h?q1btK;SXwUt*y_VRJJ`$ ze7B19gP-W$bk~A-ZzJvKbg+@|rz(V&at?3#D0+Z%)x>vs@*R$e{Q+x8K|x>u(yDeu zRN5EDIR~RO&XlC-hIzJjhSUG5A=AtayF$un(fZ|#J9QHCDp0MY=ylzumE0^lovP6l zy8s^9$#`1%Tf~8aMh_&8ymg^kpjCNdppN`xzk{LVk0Qri(C7Qy0IWEYn7dTxauv=P zPwXXU^1kTM$nnY0RW>m5(0~zho=s>c*z9sLqLT}C`kw^QL}1SI3hxBFUzR$JQ}D9{ z$g^^hS5!&YM^R4g59sO7mo$xrur3=XY#orO=U^i94rrlnFY^nXKJa%Q0&BfLZsDm;Ki`IdSjz6o9P?grcyT_Cc2ltkmjj@npJljeS2OoP{#UP1PWX zV8D@6yK7kYLE35&1{>h9GL;2DEa-F?*O-FzHn*o0n z{9JwcZhWXWTxi$#6T4rL3>HpNg`ZwG6TVEu}dsepzrM(t2Wi z`2ZPxk5sWQo(riDKq=-EI<88vv;e5vlot5&b1xgKkl0vPJK!yBBy;d?RXMY}nc?w> z5@q69JPt$>K>q^;cmM}7cbrN|GH!)x(u-_G(^?`XNYw427Sbsw=4wmpof9XL{36>Km(JK8P6zG3&`FZ#QB`fxwk{WtUJ0}rVc-N zpOF^o*_o?S>kmS->09&T9^ccWw(Czar7>Gf*w=ydfE|+Tic;oVus%@Hiuxii@?3y# zph^?pW?R%LwvyBOv5v8u%F`BiL9YzlpNQHyGJO0~_#P-BK%;b)hwf3MD;#3x^WOsW z=s5P>Z~$A}3MAJO_nTi8-bRwJ4;4JrBEZMLld%{X{je&{%r>9w1siwRMVb4j5<#VY zBh@X7{wiggHHp4z8RmR)R;jHwN%rHWdz$gWy7JdDx*Hx{@UfG7^t=zOT|pn!)BK}^ zA^oPd@Y++(wqt1?p&z~fjiZ$!>$L8*4wOPveM%z&z8h?}`RgbcWK#X!e+d~_F>(fV zUr${7Q}UY*Phqoa*q=#JFSKExJ|O2->`Wtf4<*J+5dw5?D=|uK4wmFR?!3SBzJbCE zzUACtN|6!gS67B~B~%H8X7H;0ROl-Mmx~BGn*msr@V^AX%@A@APPAtyg;yc^q1g;e z!82}s^>^I1d79-v3mg!9ywUJIx3pSvU=%{+_mwVe@)G%~A3Ru-z=ULhCkphWyD33b z>k^U~z0*6HTM5FaJnFZ5p>KBiRwD3y}!aqya=tOlQr86%? zJx?g0nL6eOg;L&@vQe&a9@W+Zr(s~5ASpv-A}*biB2($-!Gf?NN#kZfsTTDAqg4Ah z#BKjM<=>S9{u>d4MdR#mH@v)|NWGPW&H`mY-`JE*&NCsR_`*wWy0CRZ?l7(;>>nmY zk?sU)*0gD3iHe28A6@t6tc2>|Oo-DF%0f`0fIzbS0Z&3u8Mr0&=eKp?X?Q#uo%PA4 z5ky8%f7=?s@qCEYThyj2daBB&E0x+JRGoHzYg0Er&u9CEFG#8~F=T#DgVQoDf`!Ze z)r=sA!XZt+M;*aA!b|ORts~;HRA%`?IA+4VisEoMFJKVIevu#&!>Z&A(ei$+%UyoX zW>mU&I@;q|^4fy_UeOU9Uv>Kj+N~CVzX6DIjMo9MX0GHrwvhe^-9-tRj!5zl44I%A zf+H;W2Q3L*al?34N&S}Zi&0lwm&sv!0No`r&&*GZ0;`6i_5a|t#?N+uR782+d#@9N zbGPnYk4IOie=G&k-@zTq>&Idb<3~g!1jh^39@Y*}jnb?i_T-Mme_n^nWuF;^_3lF- zQDtl@=zMeAD3iDt82e+=i=6HbV7d8mPvRP1?qcF7%!B^NRKxSJ*byn4EYt@sJ0#R5 zC)nz8DR@Kvb5_yHdB#2lO~dmG$;kwDLK*miLEsyl?=E9_gzTeg(~(L{&{j zzUQ3%VKYPS^dqgH@zibUZ4b{tLuGgDIJt8Cy)bw=cA3=I`SkP~%JoWu0I~yj>fzgZ zR_RT7VXujH$3!Hbf8TZF;Ko>>TgPY8d^`I#={8))&8j7-!yZI*Z@`64Lr<1`9EY?y zLYUJAnr2WgPK|Uz?ib4#i3jvO=;!I@xinWV(&33Cm4;Cq<`hwFfrY7Upq+5t8~Bdj z^_`2%_R_ykBWydPyewm+`e)}lMdqnPuXL5ndFqc>GOG?t2P0yd&_SkLt;5y^z z7aN)EDli!ywI7lNKcZccEU~_$j>^-E60zn%kc-lF8ncEqtY?l%SjU*Y>#Ov85_I)G z_M*^>SBjlujCtB3(9^U(ETeg%b}p&^YYDjQbrQ|E6&<0ackkK`GOB%jTRDbRumkg& zd>+?@iS3yGIEQiCYdInFk%N&rmer|P`Sg0@LOEYgBMjNi45txqPdSL_hzEF-5RX`l zIg}9BR33I<7C+^Hmn>>BJ5|D}sUdp<*G`YVeR#)x*jt}HO({O^UOf>BrL6*Eig9(g zyVhUdUQg{B>M3vb-ULdlT3_i_X@9gtnm85Kp!Jc1r5o07eG58N1Oub?BY!kaGFZ-8 z*?@CdKg_+toAbhX{%C8>Yd$T-dKUWbBx!f#Z*8dON8gUJOjvw#y4xIMJ zc+#J{0N!&}yB{w%s4mbN4?o?=G`{=*c=g=&BQfsWtjIW54W`t07VudeLO&41hbBHL?6kmZ^x|C zvp?hLe>Y#4dUsv2M3X!<5DP} zZ?dAd8|Xb~56*Lbru;BkX5w0474uF>Z2MY_{7m6~mxIA6EqP63v4Bg0=7bfSiD(?n zUQ-0C%Uz3}$HzvZbvV}^)pKtjLWCJsv4mF?ZUK>MgfOr8hks986C6);5xy&}D=uz- z1rhgZ?#0m}R*E#7wcjc#Z2!r3+C!^ZH((<-T@ryd_IZptTh#|+KN^SgD=RX!<++eM z8)+{}z873um`1459OfYIoF?nno+NFDN_Yj;U5KE{ofYOGiNZ=NZgm|h@vPl!NN8Q( zB{ywhg-=%`yq}m2sI=+aZgRFK7h0p=|JU&s;MI_0X{mG=@_Hz@RH7$yP)O&=`daL< z5BCc&W7vPU`$P4U2VFS|IOYt-Dir39b%cm#&DCso`p-BPf#|VM(>AAt3!d6I^?M^c zsD{v6w}ZqFoWKfnMORO$_9U{7m41%w(Liigse(tH@S#oCEE8{6F8L+ zyI3I|!XRApG5j11tk6XaRu2zhb_U@0TvTN^Pg#_Et zi@RC)p`mO_2;7a;fo&Wux!t8Wz7iu_`;NpjQgZ5n50qNtaI#e{Ju!QzQdP)afjT1l zWF;lp0kOBukq=CU#uN`M>n>w^F>RPXP9ncc4)Dm)4x}$(#J*8vgS#ji&PJ?y40T&; zVtKXV;?w${h>6}^CNXm?rlP=MrP;Tj28fYaW3q9c)x-f)o{B38$`h#5p`6q8R5yjS zyJR75R^O49dlws@a$Nw^6KuFo1xW_5&h#Zb${{DHJ@_=JGZP{IT44G7nc&a58QVhJ ztr?TadyIt5wGTN1dm45eeG9Qrg31Qx{dY?i{?dwu5BUiu#yr%28S#~Od;g^ItcRm! zuRbxsr(?IZUq-fyJ{+d_&41X?!bB5T-)^G*&B*6!%rC!O(Qop88EMFnoBidN+lk!Q sUq<|fSMLAvt9077=Bvqter~1@FaqGCt)T}+L99dn2cOhV1^@s6 literal 0 HcmV?d00001 diff --git a/examples/spark/pyspark/illustration.png b/examples/spark/pyspark/illustration.png new file mode 100644 index 0000000000000000000000000000000000000000..724fe7fa3aae86bc44009b528512f9f00c7af294 GIT binary patch literal 31632 zcmeFYWmH>j*DwgBKwDa9Y4IXOO7UVvgA{kS;$FNs3GVK0p+JJW771FQXdt+kQY2`B z;5O;~yx)BHGyi7IpINgeYh`7fb6s1`_I;h5NL6K-XHUqVU|?W8larNH$H2frVPIg< zJi2PRoh_i z>?N(Ft*T=j6_l z4K13!wHH9kENq>}#t|-_zUf(c(FrM~gwdfjJHnqY4l8Vvssd)v(p5B4p{-HLG9@Y+S5_0M*t83r; z2U9b171Tb-tLgQ8A82T5m5^5l`GqE@=g6z-g8aj?a|`XhcuFd0C~D}7%c!bo8CpBI zM#LpMxPbBsi%QC?t7;k_0C50lQ^Pbn5ipg}e2Lh?tjBuH&^n28I{DoTRv>*TOz5^8T%i-|DI6sn)XLvSG%1y2qB`vHige zoa8_rqu9bD`-UX@{ZCnmR&Lel?z$0gBA=ao3}9gG=E~l962?kVpM$T%-{TuXamT&0 z?KkG#y>4 zMUpAgd$H?V2v3I4!}!QwA+bk>Z4zeb>AT;%x6<{sfFEf z=y!X|*zG>$VU=yGh64~f^{kj?Czz+vJc}u~rX82;7@Nf=THWr4%^e>e@y*6?Y4r@L z-zQEn)?XHmBj`$j=B37^_wS1Zbn;eGsVdv%3R7LOkq+@F2OA@Q&S^y9t}($#pFZ~0 zqBKpa7oY{URL%b&fm;I8) zhMK+-YA$JI)sIkpbu6%`qlyGkvtupN;TXh_roRz=EGmfJ(kAH<+iF8eAz~JJ#qpjA zwB_m^w7n4ALalz=)FL9m<=gaX(ORJi!KY$V8qZ(q{sORG;eXKVT=?6wo#PDq1xH^(dDqRLgqm14HlGRPSK?pP`sz9l;aygIi{ zEE?`Qp7QyE`kH0Jk*CSEgHW1ML9;RgO^(px?V|Qlo&d~5Ay^fn6pPs6N`7kAvLrtI zeK;GZj`?!&z%Nx}&1)@K&+DV0VYq^g1Lcf1^0BXANxJ)~v48E&xAwL*LGtCN-~tZ< z`}~q!XK7;==Bag>D!8?;C-IEuv5(e0bcJ;L7yFb$u2e1?vDD6SDT?In=+`%s?WL=f z^gDK$9h)VWjC_`vz2;)?hVR-Q~i@)blDuo_gPUlBz73%obmAo;PD>^3o zz$fMIz0k^0Jh#+6h3JhP{sUoEnSBP=(lvg+)1#%t4C#1x4zZ4t?^x}SYZ|!hQ+i`k z1tUgUW^>@}?#}ttUaC!NW~E@0^9Az0D!Me)YS!9_Ekj)zc#4#RjK?EnQr(U_UpTF_ zAlS%u$81eI-kUOLrCdE>YVK(2xEm3L$hps@f^`>_P8G&CdxQY&R8OB7p7(2Bjmw^= zx(lU=D7SVhd;Xf?irzsaI>=Qs9Y0Ci90*qqt{>E8I%hL{F@2Idz0bIyyCX<+*{ZGc z`l%lOw~}p-U~9In;i`eEXhfr5hNt&KC_V`ZSH7`^ZV|a+Gm2<@%pNNqE$xiXtKG zDsN!Oe0{!ykGJK5pXXcUSR*uPqivq@NkNGVY+-R(K1B{lhBEnxW2euMP0fBe+oYUq ziMR2|!k<#*IXP9_l2&Znuv40 z;Nb{rER;Z6?xs&kA(rlUS)?tlVgt|245s42{AiR3a!8X|_0Cl>Sh|7>~dps&xa)e{!tDg^|VV-zOyu%DD zs~W&fsnS@CkXzL>em-quQxT2&oP%^+eGwbZ($ z99uNmoLScXa_1_-mhZsQsqDA)X&63YQjYV|Q8Qu1zGQDybNda@UCX#e$t;n4Sl zjZ>mSvMk=+X2`G`AKKi+NctR@!Aw}D#~_&*UBp}oVtHt~8<==i_G*(*h#jD%L~`|u z#oc&+vu1}wwO4Iqm=Z$blDqe+XnP2|Nws9rBkjZbm#LpQlFf;~y%Cd^lv}Mw7FzM} z($x~%z=~g22wRZ12c%eLnc>uHwj3RsZpHK{QwLEYD68Ndx+;~VqIZS_txu+y)M~9p zz&3i5yusr!BO36B*9f}T*gyY#w|&5sIN;+gnIrgA(on&DS0e(Z49%`Phd7HOOm9Xn=m#Y|$&d=0}(&z3wE6osn$j(a2YO9gr~)APs{6C79G z9iE=1s^Pd7ZQ1&N0U2#M7R7)QR+y`^@1rZHRAik*ro>nbVXHA!EVD{lR)})>>5~o2 z*~i(Z6#|=oj#w>mc>$C37&unBpVAzr3r~N{eKu_TpaYdvk%f=+^tkt0000WUx}zl9 zZq2N_GsV_*?e(7zZn4B-*~5Cw>c7&D;_<3cJ^O$sU6p?K^!Rgq1S`>1tf}uL$B5oC zt5QpI1Z$#k`3h<1sF26q#Jz)ve0Kb+(->dK2e;m@;LdtWa3uvZQEZ2>26x1ppz3#x2aIf^?%C!lF2*+u2yT^yF{t5p z<$QVq6-E~}4d4sIHG(CMbp{8CiD4i{Ps)Q`MI-BR7!{(YcNoC`!ZrMFIa)<#fNdHtl*CUdT-O<29E6e6&476M@h&`~W>>|5>Q@Lj zK@K{gDPREU^%H{3j(tZ=&R#Qx#q}&gx+TRYr;eRHv@)M8(0EkP+pk$PVak)xK-{&&JIJqcLVK)cHq{i8_0# zQI`9CX{8F9Y}_EESebzVRzZFd{RZY?m0*_AwSKyf$|QK&~&7ENPnL+Ex}Ix3Mf z{*o&e6(LJD{77Rf#A5s~47HG{3~Crbol*_&8a}<<_{%~)gapdE(O#0Ig^kpM(<7q! z6%@vpnP3?w^es0ODt%F|pSh3|Ax9~FL=aq`pRbWGz)?hlEv!F3$2&U6I#In%G#Uym z+vl8a7pOU=ftf2Zz@qT#2zNln&+_^opk*N-1M2a47lz6)d4hbLw(HL=>-6z$5|!o0 zTRIxRPwXvO4&K>M7)XKcSBUa&gHF#ARzei%Vf54Y&+(C%)ci7_r5JUZ1n^76Hvnyt z&NmRgH!tNlD(js!WCEr7t&ll$$j zAq#(8lwhO^S4;Q^dda0li&y=dO-WZA)P*-jUZ?tA`z@>78vscLy|)G!-whhJBMzG6 zOhvf>Vd<<=CvX1*o!}IQGpNvMeG>=!CH3*JWL1E@G;$T0i!nfcJc#HcH%rP*8V%fvAR9VhfF)XIm_ zrRLcE(D7@+3yXM+lH^ZHj?)cyChC{l!Kl$Iz3E@L{{*LX+k+C;ueJ`WZs_TmZq!F;8-=4$fgf=M{kVgv--UJ&|DnR@T%WCSvEc* zB(jAMWcsio>YQ>-eefGK%UC|nn|Rh`e*{!wk(?urgEalU${fT3b^KMFBd8$jj5Pi! ze4NStYpt1M3(7x99UNE*gM;%iZ463yk-S@LQNV2o9&ju6Zg8H?G?|KE&wK`%&+I%z^;% zZN_2EVMDgq-fVVZDumtm8sBPgyg%#FaU7$3T=PU0uKz?a>@o4kXkO>(dA(iCvdH%2%qfRrKN6+>;P0dRJJ5T2nSKL+Um+I zAh9W+1`~0Tn(#R2)UWH<>z3$jp>f)^*+|2GYBB-VWt1VrYJ!Kha{9=My*Gg`S!3*~ zYts*^At!!%_wyq#VE8SqG&{yK$QDzIn&D!$5I)A+-wPadTZcpb{>YiU4{`b`{I~s; z>ELW3K@;o*aPh0uD~`98n0UWparai9O~?C&-?|u9FILq3`BhzGtvZjD0B+70QTpu^ z1pRVyaI(jcTz&k&yAtT(ZIj=|A?oja@UHtt3L= zvDFiZx}h&!?5~kO=To$f^AgvG;E1qWtLmj?&K&ZzyR$X@`C7t*{ix-(>t9Ry53Dc{CkeSxR93k-B5y)H z<6);i`yIK9G*@6V0?_#*!nlqv(Eu@bC!iSU?!}*dD!_=~Gaqrh!s#kpBcXzJ z7ARxOQNr>ydFumW9LN?WX0^wTM}|AOR{-(K@ai{j{=iKNCn~KKCZwFP$ceD`+3pWp zJ;DUACw<#kZ^dMnvr+r4vU_xGxTmqGPwWy93_CtMf`gO5zgu)x%{cPCodnPYfpbPn zCUkom%kksEs_zA{dFRzvC9g951Ix4M^XjW0d_?li0$%eRi?NK*7FPgDI9PLVO@pWc zH)s8ng1|4e)|=p9MbzbL1sJ;Qy(T$q!H1M{?EFLAcFuhKi5LU2MFO#vn|aEwP5OY{ zd4Jz?CAGu*VvItjSh?jbi#34?N7lF=C~50JWY`hV9vr3&M=QSIxA<`Vt7TBv3nQQP z-0|S+QvQ^8bJIb7WU0J%mAu!z7^n~s;T8{f~r+f2hzr( zo=RMpwy3BIC~EdlMAL?c?|m4Y4;_gfc@Y^LaUFaLloIy5&JFMUv|E*|1NPq3>!(`T zo>%qnK)pJz)`@|gvPHmIXaG)+ygal56HdMyJi<(9zdI9hlJ{TUsc0+8+t+tGH^w>S zLNYa6Si};@<#u2oFCWO2m#-$6Ej`FczjnI6lBjIcIb1nBNU-DU=$=!k?pV3RS~Oq> zJO-|}K1CmftxCtjacKZ5G}EO(op-}q_=K#lT<-g;b*?|C>O4kc6M_UVb-FS%5sAUrjid+WpUSJ+<}^mYvRKmdhNce9kfzWQLzH14do> zs7ji~aQWURz}08~)XyvSP#uKa^#Mm{{n@ShoTtq!FJIrfs?b*(&y}>8dLfw{S32&h zmO6H>d`GT+QYCa&Wq014VNd7rX*T`nZR^d2(#fkhoiDCx zVv$p=!2B;g10w@a28>-itR~V^??c=-4hjU}jTE+ZTlr;PymSiiy`Z*LF;KLG7$}a; zD^z^T1gC1b^I9pjq-VwV9gAnWUv}BAWe~rQ!rCi@#yCawPTzgPiW#~RpG-*VgNeTS z(%>9)@}|z^yr^smP<_mWJN^n*PRKi;*X7QP%%m)o=2b92&DV66E>t zFC3J~@c3XM*|Z-ZTcY`m4KY>VlMK3~sJm00=?tN>gJ91UDi@rp>}KlrzJuHhiK~eL z6_JbI6O4Hxz+=6|;XL8Xn+xkaBeq6PXfY=pd6^R}tnUL2K9$OH*k>z;Osnd5OI!8(v!TJh!By{LUAY1hGR`XDpi-3e7fqfyfC&BKJA-wN->+#YlRWO_ya4Fe}bY*IerxLwYCEH$i zVv!hRr1HJ_O~=qgAZiSlNV>iloP>ji&PCJYlY6ctTMyUa)Rmn(iz;BhJJeut{KmD& zoh0ZGhJ^%NZkta7->pKhulD0p={ngMuGf-_NuY?&BN1WNi7RI4ydOR87J-RGX9r%? zGrgAUva%)E2kDKmKj>1Sc@Bo=2rY|?T2zMHf6Glihjx5x zzjqKwvbT3v)+v)j77pkMu_R5v4+~hje+H{!Y14_-YT<78gWdl;AppyV!#M6!wXgmZ zPKUQvssVS7G?q2zWe*pR*RHAkDi+u*m-jSI)+mv%>hDFJ>SXG8hhKpjfQyf}1H!t# z%z5gfsNcODI7NA2`%3T+LzWk3|(a{Hz3MiD7;u(QM002EK{dj?j{Bs1;Ic=7Cz z7G?>e0Th3I%z{g$4bBtlq5RhG>eQ~Gb3hei*utH`1j!RPFGf4`ptnk%@U85^!8(T< zGjvQXQSeGPTb0+#&lbXHcwSnT?5l5CCak4Wy6}@k-xw-O* zn`D8Um-?K`NYK~4o)<;yxSk+kg^MWxEP*auxZ1x4Jqace-rBsSP}(?{b;@ZQGc-18 zKO$^9ucRQkS@2f3i5I;zJH6o6n7&dx-SPdJ%%|mWSeUB$E7|>IGnoML7+Tv=$;C-s ztxdiER8Q3*FHiol$W_zLh>Y=6@MLFY7?Z(7`(+0qa*~r7M1i*QaJ2Z*orCqG;Wt7j zO-#B$-$_M*hFS-XoM&2EE!kf}P@_Ulb?L@!ZcmW#Mp(-J&eNgusZOhzk@M+CE~h^e z$%5}zS_RL*z>{^@l^(a|?GfE=?xsFfrxQ+nImbld)O45ItysqzYve&(j*?~MMRuR#*$)pBbg9$_` zTE2K)(xbI?9%dj;(tr-f;$xR3j~N9Pv~L1eR1Kx-&_fewyXjpdAiM6*4dl6AAwj#_ zC5icI>HL5z-&71_y`t`+Ee(L=ITMQs!ysTI(jLoDQ zrB|~2E9;=E?eySuvEJ%#*}n_y-`}$&vFY;o{Holk)|=yN2j6@4JitkA@hWvD8BG4V%t?sah{zLBxwkBZR8`577NB#C4 zTi%*i@Em{0{Scf(==UNm1`c44x7c2&CvdqA-1${2=~X^`t21Z!fdx*DtaPgyuiW)G zbm055mGTECtM=&a>)n{FCGv&(D3ZexUc7^a%an$e7T7jnj5o`NrxCRos@58OI!A2!!1s=hS$`Pzq@1HVE=84(Scy+^RL4XE7a2 z7!u%f5;ob(cVOD}oL5k8sJ~={P4D!d?D1xo-BhiuF|EDRyz1*|;gEQ`key0$xPrE$ z#>$Ac%O~De2c)fQg{Y|?KlJ1F{3(u(Q@$j;GfmA9ieN!EJ;X--U5Wrk>h54*gH#mxQomzf)-D$v zm~*Q2M^R@B#l$Tc2_2%3B8FrWNal<$qFt_ya)q9IM(k?*YTm-PD4pYe4x^~kwuRGt zRJo)Cii=#w>t{j#sAN1?_jI5Toxs0@wFX?XIoW-zzPEt|`a2%*FXh{;-rOw(vJ$H6 z@}i-eJOP66-OZ#M#;v%*+@ejC{(Xue)<*oF^enIaVSmaHyL&g7Zzs1WJ^m48{jfoq z{Y6kH)`ZYYy*!U~_Uf6wIZ^Y)AHt)pu=nl-W#=B0`=UagH0w1s9hTZ)HjiUNSg+rE zPtxIDMBp~%V+$gp<*xNVTiIrN=d7b?N#buY-m6P z1{Ch5+1W1VgJWN+fR>9ezD#ybULqWU4c!v`p?7fH+ha6#J>YYTh>-vQFxmzhKJrU zMi;y34jVU6%^xg>uhd))KOi4cJ(E0C{6;!J9cmH7sG-i8hAMfpK(+dz_GWvjhAhk0 zKu}uZMX?8;YYtXvS&Kx*D>zPp=kiVngF;)&e5j_+P9l?}rq&ks=-EY+XgVi_V6V-G z)7w^wfzSM9UNd>rxI@I6yGL#`pk_AXPp2N?7ID&90oEPgo2Bd()>Zq90fm%;(!t_4 zIapa+t}^e5I`AbAznwBgvwaiKiv(KnM+vT~q4^=U;gl(1ONfkS509HLQ6*4!{v>NJ z5I)eih4zIPOKIMy!F=w~FH?=WO@I!zkKUUz%#8b4&fBEVS|+&fF1%Pjf;!OFj8mJ> zVMQF3393qt1JE`NG>85?30e5aV-vYI&Pk3h3)f0}{~U&km*USkxs>8Ou-XU^!Tac7 zxe?cEonC>Vw zwRhG11CEKHphTC?pIi47Z#`s35SGi2=8R0{sq(P%n=p{E=wowo9#gWE*)a0T4hX>9 zJJ9kUtPdc3f(c;#5C-iOA^gfsG7lbtCKBDI0c-IRHZf2k7P2M8wnzU!7R4Tcz5L?; zg7GE$qKoAp6|d)o@e)D18EH^WRkH5?u83$kU%|MrU|WpFJ2pyI_6Ph%M{n zMg!pcarv7i;G1G?h|t4DJl4hadYY-f^JRf30~*=1_KYq8*8Do_|3-)PYNT^|UuNJw z*Xq9)P9GATe6}l+IN<9X;P`Krt%dGpb)|bxq~6(c)Y!+vgH-hcP|~=F`t0Y^!e9O` zq$}S3D%mu|IpWLPCJ5gY!Ai0B__o5@B(Mimm9pLO6HeS`dbH-AYY z0-62DHP!zSR5I6afe26D&x5?cAT_F zmBKFJxX8!;yYG@|cZdBxI2);cdAJv}$Rt~hEenraopxL9dr`{btREx~IXbqjmP9fG zdc5W0xqz8r_SQO5JFYehXDX^iKHkcmOX8}knDlx0;MpdORbs`yoG5F%}zkt71Nu$)>hOKZL)eI~#%+)8lr zx5P&qQi}>mVA`juWsUa^ZC#qD5kPeq6=^PZ;8jpf!gZ)*kIzfJ&T6hx($lRf)Ks?i zZk#Z41L^@#F*L@6a=JrVJ*4SXpoA!7( zJphpNfnR8w*HJe93vWxMY#n!bleWB5AGa2T2A}?MXG(2>Ns^>In>_UQ263^z9Nctp zBv~F5d8uqu7(|p)%bLPb^zoKd%YRqCme^3&r9b+!3~0(AzLide{+O}U=iR+!pQ%j9 zmB~aCcASFBjG~WyeD+HM+_GzC0G84i+pTpHcwDXdgYfBsBO`=y&XRA#L%urYl)u2K zg^S(JxZ+W1DbiUR2V23DoGX$m6Q$=x14tW8POsO^;z|(h2fp~8Gi#g7m9pA*$%#x= zPd#2|(%|#wiNUYZ)EbUn@aSAbyxd!&cvnMnlWLUj=mMwGYmYsOY zM5?t+;P<#`WD4r@;^}S!>Y=)Be|Nowt`n5eDH%^(p&4Q==&g+m{qRK+l-|0mj4Ul= zWcl*(iPALW1A<$hWq5;43u^UJUil2lIECdKTB~@M!88heom$pFKRpm93+Gpx#*LS+ zK2`;NS&@ARW^le@hK>K0c@-z8QsSRB*$&jBZh<1A_1srPv zC+_BxjY>M@ZuJ;iql$lg>)6Sn^sG>+F8#0i1=J<(}$m2GH6q32XZ%ORIpXb zynj2mpmH3!&I?=ks#Tn2w3Cba+5V!GVg%5wgOm%T&)k_7a#aolLUdADmGnT)Ni}rn ze!cvgwFI%Y?Hpe;opz>hA~6!X=xw#4&K?RGdSxH0EU&^i{2GNM3nsK_9l!8U^F<4w zM3SUgMDIH`cDO9NYzKuQ$TaZ?^7SZ_%pxY|vzeQ|(U#X405f1@UBGq1`h2V2JnF^bg#RR%ZZf36Wn0#a;p^RwX4Omf zszuTiw-B=8;(P7ZH@4%Hv%44%ZF7teF`al^ycJcg@}N!clB*1=ViG4SQab3(g5-u{ zqCqPMbae5QA3&tvC!t=NXunkN48jaQk?t>CqxpB@ne2#GT9#wWSB1x$Z7$OvXM&_c z#5zNN7hKBfhGg^YbdJQZmRq?HU0R=5?nHv5iBF|@ukq%=o#DS5G83g9op{_fqVE!= z+yp<@>Q1S)auDcol!s@$j(Kvr+28Z;>|>>`!m;X*s@Q6goZdF+Q^j@Jtm5jFwp5>J zp-t;!^RwY!`7L)IB6FeqtWx6{qlM6*r(k2tF9&LObx=$AiA2y*&A)eEYAY98D~>XI z%uZ^+xc6B?THnEbOj@pX$^JVfjrkyUwObbHgfAXAX)B-GJtZTh{&xb*R0Em+`ikO; zqfaZ1XwS&(`#0w_0ELJ#Bkq@|{&*g}e?tYL_5vWEmvWwS;7Bg~R~2B5iyn4llzM+I z6aH^D@FCXNZ@Bmv`-8I@_p#IVkMy3~;w7wXi-ELNjoN7+Z-;;J-?8~`681D-Iv;xA zulZ~y{5Qx4Rqb?t+-*546>(3he=8LWsV5c@wenO~bG)SyGhrp;p9(`yEQSg&{$EcW zh++*8B4VovX?YSi=Yn*P{hO$D!%$=V(!H3-(RdTmLfk8Eoj@&jgX}?cQ7F+^tO3Yt zm>>A3fLf{e?yM~Lt9y^ekl?p|N$L#Pp=^Cu!uw%; zFLXha_ci@nOW$kCOgaf>>zUZgL?#czyz4osU+(@p56(OGVVO+>v+}Jqr`-V?7?Fb5zDG_4p#$=H6TQKka4vygoSL9@J>s zuPl8vl5Cwaoa-}tELxF!9^8VJiMa?h^Y*D^-52!G0Uf95Sdg6k{3L}`A5ASN;G-Dz zN<{Z@31l^f%xLoHQKvKK=5<#|)rsu*JoXRqeW21;?CipI4R~Z5mo{l0%5J}eO14MC zbnU64PZ6%pH#K>^omE}k<~#!E3QJT<%1NZ7eoB}aJ*=VJdGmI&g)Sol>ToFV)y*ki z484e&DqhVLiV~$pn^`U3Wb9Gw?<2)UFP0L zw>yL=jOI3eF8$+Swpi&C!J^Ri6bIr-ZXHFUJ}R`p-xcEA2IOYpH|}_e6=~`<3gD2d@*sa8G-7 z!;F+-&eLGwb;S{wsHbPuDYo|Zr(&?{el5vAh^l1a>v0_6O-eP0VscNLS+mhI1*cR& zx8Noh&XY~EJ<;$@TtqfycvI!EeTkFRbSoylEQd#VA4Bubg>umC);N0VpniR<37C%I zsNa+4;?Gsh69CP-uBziVTpZth_>-rsjjzm2c zax_k8aZ?#$IB3t@Ylt2pquiNt|ALblt@rcoIS!hzXVYk|Z0=G)SKy26?@*r~eWM>> zL8w-Y{{`sin21MWj#PGhRBJCgL@x#^c3uT|giuW4QjOO-dzq^&* z|0bGfCg?#c{yn#2Ks*qo{C~>8(NWE8^5*@(o25XT;#?%?HL2_&Pr}OHw|a*I$(>DN zbak%vTdcHpJc)U#x{ zm+I1r`uZeWY~(0TslAZ|tVVy({K5a)S*ZeP{Un2RqIwrkm{^w+CJ!>LYFpk}JnGRr zEdiOC4!%qUri>g&_5(bADFY|35Cp~ef~f4 zo?%4DLgAL6($gq~TF$KZa&O8#InCQ?r1hT_sH=oYgGMYk%g%;1edT||*GDlAh8{Z^ zg_XH;PnBsjX^2>s-VAB!hL}#Qv#(b9QnzTDNzH7^Bn_1n3bL{1eso2rL$qDbu&9II zaX+4RCmC*~EEKn~1ql-!o1(|=m~@$YlAyQ$_$dao+${%BVf^ay6 z3aJOGd=GmNNQx)=!*O0MvDpbmB=!L3^P2L%)_EGmlcvR4AemDBU1_7PRYhg?PQHT< zFeyQ=Ik^c&m6U0ee9FCY>v7Be#M()_Xs~?x^i3ZbUi@O?;c^mKI82+}T&%J(NI6iy zcwG2D?t2=g>0a?WCQ%C-rsyE2Lcwe{k%r20iAu*sPnnmQwQ@}c&=*N*GK@Ge)cj_h z$eBnspY8Be1l9d=$Ul8#Y`*cr_$HmI_D|1i+r^4Fo=R;ZbK0r2#UyZ5{0nXS zQzUO{LlPO%h9k^MF(`!LisJY*Vf~aPD}(}>%h|Mu7NcO(gV&|34Mt|S_MC-i?f26m z`;-VzJ&4=L?+qfChiQ-?vjp8)!8wq}yXi!?gtz{r6^_Znzq5NRyw4H3{fShsUV0F< z>p#364ZWp}C9_2TDPniC2{lsgQar<=Pr~}h!w#fmz6C~oDAFL{V>$s=j&OrqSQpRZ z3}Wpdr}o^62MZ$qxLsTht>Kemv5cHE2gb+!!|i5m^V7LdouT6^MdXX3G8UwA=JOJM zLXS-Ow-6K2Vb7p_5}Ph{;_s}}9=6FVGyJe+5rBE^o;-3Ng+#7XaL(F+stjT&WVPlk zVV}DTiO_y+^SWiEl~}n|bICQbg=DON923W!oZ9+4B{@_Z?%zXAv=sQy#c_%(^?ib=C zd>c*^jqjT5SA|@79p6-1Yv2`+7UC9TB7|b;()H}tAZ6~Ss2_Ra}A!W;w*Otu`UX$xhwsww_hzj(9)>bnWZZoJ^8Nq0G)Nyp~Yb$**uSXhEiEY zO4*^76S5XIWzz2tD~>AqMbld^lX;QHp`esO!X{d!{NX4u5!UJ9qJ?);_imRts#<0P zwO6ZQSLG;C)4|iWl4Gg5U)QOEr?C@$O+l8}huS6;n+j(z{_U4c^Gmo7Hy?7JKYa5n zTZ?$4f@Q6UN)O(kzACIbG4i9XjM3xGuIMOBn+BDP&~1}|v3qYFI)E;G)Yr8D?2_wS zW1yuvjCfhLH2L_b#YeI{WL$n0#jhgOZK-9_CZEQ63*k?oTBkK9Di0Y0T9;OMe>ccv zVr%w`N>`&b7K$-b03|j+{Yai!(sZw78QEG6oA?KTkZCQRhl#mkA7l@G`D)}}=&(|~ zgc#0Z)YwKZrGV?elnU&+kk!kY2G-gq=}?)Gk=G3$^0;zJbC4|p5%SKfJ{47jWigQe zWa)xQk7iqhtu(ik3kx^eJ-M+E46XF2~x}_uZ`UVA;}h@$q(m0{NS{M+c#wKE#t9- zwqC14@##+_3$z)iBZ{2Y`r|0$?Rc`uv-a$Htf79x&}*gMcNy4$>2a|xuW`}YO<93M zT_O((9Z{``MJe{C`+QjB33NVrR`#Ts5oR{c8b!u}OdSnx-kbV8kY0)4&LoXV8$rGM zZX|aYBbgRS`t`oQj;)S0=irg{*mAa&oV%;z-;BftBZ9jS*UL6LI?BP@^#qIM$e9#) z=L>=A?~v0rxq5Vfb;b?ICprCk(@Mo#)rpy6nVm}-vgP(+bSx6E=q6)bq#f=-ig#I4 zwPI|M{Y~#|up})e(@3(^C0bZ2SyQ2+PBVb{yi{AUK8<4c9V|5y9^X`yld)sLAZj8l zJ>KC9YD#k>!B_Ug9Bz82!f_v$`DjbYI`G8yFEVC8QVGSd?5MXo*Q8;wum*FXy&Q*? zrC7+PRIAiuT1cj!;WJ?q%aZ;u_Y|XULir~hiBzSMC#p59nsnsbn}*ULL6u}kH?!;y z`4W3uSkTVhhK*i#+ZuXk+n}qC&BdV~=^!g0X*H4A1$>5|C!V}khT0J*Rww=uZY5b( zYJhYGT{&&ARnin$cTLgdpQ;Ko0B&{M(Qojz$)7T*pRRfN%5T{FYADNJQk3repab4k zUDfZjO_e>UZOAKo+%$X7%HnnXI&^Kqe8$yX>?AjEAye4#^3PV8gAELdGfnRKMAE)Ua{O)eOI$XsvRsmoSJ%+B{!mxI|Lio#ZXIZr|QD@{PLj8p|5Nn z!<HIW&JDVZldH_E4gri+dd=0#v!*i zNI#Ee2kb-Vw@Ki!$e(SU&?4&3R>`h*(N_)~6zS`nvRo1`8T%nlSm-R2SZ)vQVG`*0 zB42E2|ESvIUU9?=hoWi!wRD^&kzw6mRY1^JLIn7dF{c>| z9m8Y~p9GqQ`O%6fs<{H%aCPChUFf3m{%h-adEx-K0k)IjcW@=b0? zU@et_t~2K#f!_P;OowkQu#eR^TQ9_&=ac%Ih?FIMjWxwG0@SLT>8lcqYB7*4!h*ip z8I~{SWar+)8!h?0A3P=)_=Jf5bmhR36ZSpVfkGG|uVeo}JEZY<6bpor2H^Si<dSs^F8lR5D3~sfs zZu24I5JqMQXZW6}*UAfnR50JO>oxY}~<=W3@pJw*G&t;08I9C#20Xjz7PuT~wJjFn`x;p&NB7_z{m_oY2tB=zA ztVhBfIqDr(8DHY>fddl5_SBxjg~G}Y0#Nuh|HJ8!?D-~uP4Ar)zgJkR`}Im|0J}xX z8*fsu=WCs1OeEz`eZ8NXO-f`oU7pG$%lmvj$!Y)DWp(k!TOe$U@;{U>bcnxwb4Zq! z_eeUKqVE3Fe2!+(d9aI0hz#_DDs0Oz^xstf_Ginq)YZ4*;d?Kh!rKUS;MCMAH$;%F zyT5%@z5Z|bHp(H4qAooD3FDV(%7hin86ssYw2`1!+w^}=Y@^u1$m>XYfd8|4Mx@-k z_IK4rhj3S&mOZnl+boQ%E<80v`B#|`4Gims#nEu7Nf>q`R1svr#k;~kSG*QTYVw!0 z-2Q)L_l*uG@BgUotD~Z9yM9qbN<_L#>6R845UC-DZijA=78nFcrMp`kYUq+2lzVB5%Tvl@wMfzfr~NQ+5?&ys?qjLL%tS#7^iRx;yv|GHPYvT_OjghQEU? zdtoWnOQZ+YorNB)2+bDQg&%BH9=4}F17T zE!%g&2UrPof%H5Zq0)!ps5Vy0xUGoGtl!l5Kh)qDgTf}dFI!Wcc<(+3uFbAJ;k$*V zi5qr@8k!+JnUV6Ugze_HS-=gg^0jx9`6X=MV6I+BSl@prQ6pU4hK-U6*DL?d?N!U( zJ^5DNb6>kn-TcLVXEj&A^IZyesJ)pFE`0vx5Ou}o{%u|RY!)hRg{?vQkTOL-%nuLdDOs4k~t@M_AOq(34b0mtM;WSFsvfYnjQ%HW}nWZk!Qjg0D)7(B@_M`h`p^Jm_Gx)RJ~KO)twj&D{A-1H4I(pF0E>!Embi+(jQ zC6Pm&%Ykd^?uygRXTOsu6GDb(ssV6SzH%gmz}=L$WPVew;kHt7g6Rz07!YM$=kY>m&fnwf$8v?Gf|oGZ{Psw7kT;EmeX+!d&f917f@_J>hk1{@`d4uqX@?Y zZpC4W!g%yVT~Y78!0)QmC~vV-XZMLa@_kF+**?UhQ6^P0??_ois21D8^G6qIlky*V zpZF_k?u(Rq=y=(H1mqgfP>NwM-`HXD61w3jkZ z^&!s<^9!SO8@ZthL$=tM;!jt(VJ}f5##;3vZ2g|VvQMV0T83D{cA&2BIFE$sI?io= zQOUEh`o$wV)u_PUgh8azz6;p6Y2DwV0lP%{%7ESakWXj2V3a8X9xsS_%e5i&Pi5K6 zey&VB`oV!rw3Qw)cF|0wMC0o!YM+6JG>=PKor`w4(X(SYi&pS^OR_HMH_NDrV$~pG zZtof=TUbGYbZSE%WyeYFJ}tX@YduLX@@SB)&X=k}Jy`U@x7HraEPv;Swgq}np}27{ z0Isl5!Hjx}h-WwF*p6du*MV)aK%R)cQfd`2Dex&ieDmB9ejBpOUXMp+UT8Hc4Ys+_ z5E2e*C8QJ;+Z%KTMfZ<+eFrqGG?2mcw2tTk9t(%HlnwNn=tnL*))H8j?jd#}k z=kTV6u#H@heDcE$)vh`56+M}(dOafJ#Ou_`VJ9?tf*die)!9Eg>!kM$lZYco2mYAy zqut7r&N4uVa4t)lum(R9eue`aMI#+W%&T2124}Xfdxo%p2KO0nZ6}>zLBkY2?>0aYC|z78xDKY)GH+; z!7!@SvAbY&hp=6zNNk8of@Hj#2dyFZE~oqxSAf%jaLHz7<4U!_uFh@y$witDiS)TD zW5&B0%=E>5#*bmhmwu&#Ty4wFdt3As#@0u3r!-*V+#(Yrk58K8Tk#eqgO<8>QM~EL z!I%r3JddWmAf?Or=J>r=m=E}qCB??QlJ)I$9lja#R&^NR5Dd&~d@z$C10RMl1^mrL zSr!}b#Hg9Yp3zD?WJ9$ATAeSs7@>#0w+J7|Y1@~Xwd?Um?A}#SRy=U|y_VbK9@h7( zjomHDb23^@U9#FT;h~b{nOOL}mrRffRa!h^3AJ1-&$jZ{U){r3jxrg-$^ElY^S+s0 z9JioWRqv~}^T+}ebT<``sVjJL<9_dTE^m7N5WT;;Fpp%XC^m?aDK!<^vcLMXkG(MLhAB z7II&e>7Ka%dE50F>`0=dNQ%i2LJyu`*SeoTbo%|qmz&CNi6+lUdK#s-z5?4isDhcs zleeEW{P?a4mC;gpsb!Z7Z^q}}lo3v&sCZ;uq#wZ+Lv6A^f_+t?{mN$iGk5K*E`(=DQ=CVymi(*297yuqhDz58J;o1 z?OAZxhiDn2X)zDe{e%lO%=v3AeINhH^Fh~5H{?J>DzE(XCP^~Rxa$39SFcllbqTjj z@_+c9vBeFATmiTGs&QV|akLJ#l`8Kny}{lO?hYzkhf8;uZCX--@q8a`+oT79Zkh;JeW(dBhr+>kYr zI^g(*_XroYE6X8J9ekP5mUh{o#93v8zn7+N*vhzkn$RdG@vS zt3*Pieybsle5$`_iz$3^)ju(`lyIj~GNZ2v>t(6O>e*I&uXb;S(?KBDX8c$6A|YF- zw!Z;NF2J5e;;f`wRxF0D7+JmjrCXAC?FHyjq3(x$$dNE(e!pq}bhdv$dB6VS7amWm zbaAG6x11-J37hdf#=r2|YX~z}PF~p1{IXJqT0!I+%wwaIfBW#MhZJtd7n*&j`#k!!C!Jb^zw+T z@mc;Y6dl%X4^Lq|KfOQ|hVfM&AWUFEoov3;?F{^>wuKvD;#fQqii+N4?Uv!S+kAzS zov8(He2zsD8L8t0H>u}5Zj8`1)?UoLl=i*#OxJ>X*O3{?!cwWw=@SMv%6=QRd6S-| z@!n>i#m|##o0bF58i)25kY8Vz_}ywwZlmbP^(Uzu3X?KwRH0q1J}g<-Uf+{r4||ok z`$0a->xXAZAO z2zs=QKXp2uMHeT@4;7y7s$`rl0?vunbqWVb^l2S(L#}S0;OUUj&*NIkIj3-`fIVJj zDWfPP)7gu+SQJ zW~ZC{K7;g!dcsj(Q;hQdPda2hN>ZUq9v`f_I%5R7s@dy%yLY%i zQ?5}L3!s&>0sWX2gLt0Z2pq7#euPRdG8W_|Es~GmJ%))rR;5xWwKV==9LdLsu>Y^x zEIc-P_^kD8231?OKK8`nP|@ihWQ)U-M$YbKrvpVAZvSJ;Lmk5ZYTta(aDThfGp(T< zny#ub`uhKBU6&_(QqDPNHf+RdJ>!IRzHB=r(F4ph?yLz?ea#wwOO|e#aRv@4w&W{p z6ic%WUrV}c%-*W|Vu}2(TG=!82T6(Ukg;cIvIvR~|0+We8%K5`uoz{*@Vx}By zi$q8?U)UleVcXljCJzU0FWZ%hbS`?rHu2<%8WlBdtsm=h>Kr{vDj`!@FZrnqVmc4` zoDwh<)DnF#{Db>Gjy)h{R59TlA|vU|6IIiPa9_sU*~oM2I$H1qN%vtOd3r9y!odA* zSc&hQf#ANkWzF{jVfNi*PAx6E6TkC%4pl8A(vDjgR#8hYdO8gyMwEy{*C7CfnJ%KQ5eXh!&kg=E+^bO7~5+}2xu2`VqQ zd*6cN^g;g^{FzL5($@%b*;JG2(0F3}75^Be*vCCeu+yh%7PQ?8zCqYj%*&mvgDwJ$ zt_c(4VAp6{i~EhwBq2dVM1+#Z%SE*{{~Y1i^0CLApK_<^nL;d6WP2olItJ(_-w&3*Ad zb8Pb4>6pNAYY$e291rDLRYTg9?9I)AwmR59ri+BE!Cb?x(9J87+%wuXDub(4VwbLW z|0PMh)2E*c%W}aZEVcz=k0~erU@Mun~vCVqV7=I*SXaF za}~*Fdh0Vy)z%hU=s0c5Td}?HA2GkELED7184YHUV?ENd*=_;cvYq zMQw4zSvmtHphp8xHQpR&hkLgSyfQOFjhh9DBoa{BX6d;T3|(pW+Lc715sCqx>B#=x z-jS;Mw4Od&>)n@bwHz62jisoCR~BB%lQc2Ei;KH3=1|p|%8U&*44BZ8n0RpC z*-0>K?EFKBBC>#^&JHqOKVK%*%n=CeA*>&rM_9|{Gh`)HtVTL`;fP9x1aglIp=Otb z6u<`uOl+p!^?aYsI-ZrX0I!wLkW7WTVyzWdk%5#W*H#SNTMl}(1P#M{*w9-d%AIL{ zD5%C^pDyxB#X3`23%QE(s)ZUrS!<&2D_MWE!2QcN5B8 zc?p~`u)n^MV5C_HT zQBKs_M_#!5nVf380zFs`SVM=r#+0yHUj95>W(f(2%H>;&Uvpd#r+B0I2zm7#^Fbb_ zOB!T5;s5ElhR@|Nb(M|tk%)0goWeK)8-<;ba-c#S5%u!kbw8iQyACz?W;JpYHum@C zcWGxE37dCuV;_Hbs-Z2bv}P{55iTKRbpwShf6qnjfX87OL;A&ZY;dJ5n#kS0VU`&r zc5eN@Xn{JD1%jqpu&Xp%}xD%W#CTt z!T9EDw`VK>ZM~W1QUD-%{ZzHXzoPSyr6}g(AuPB7$Gv^uYFEHIGXEeI%GRLg@JTvh zaw7*IWaMU7N<-d#>LECPyQKJa-#ozFaPS8>SLeLs{yr4&W4HUPtOKJi+LB6x=t2m6 zb?BX!mP%DAJ zjM6STT@=N@Grm85&$Ta830PBv{oGm}`*!mwev9(X+eX%>&rfKIUkc)N|z`uN}fJ26a`&z?sU{pae#CVtrzBlHn$j8__O`f%pKSmiOiV4 z&F*OI`^7((MN`0>Rqh*vI6App?2xXs^69CS7>6j3%RR+)$>tf$Pu}J+d2d9aXSs7U z>U|Ye6^R ztdWU)m9;JR&gK`t&}#tp-0dd&`7 zjHED@=;dO)tlWZ8%{KbEF9(RZQEm#`eiduNO`TO=VpAz5&<)ICK)KiWHwdOMHFDI? z$bTPxhffM9F3An=E|pHjd}AiST9(17=3_)eYkEqEI=w#{HO&j?{OJvr{-L1Rm6<_;e;g&r~#7 zkVKudz{#BEi`;qXp=%$U7ct8{ZGE736pzWKi7zcKRo<)9tIU4gwvyr5 zFE%SInwpTNK?rLyRhtXD%&shqbZv&X!hJo{CjkB9ZF3^P8Y)hF?rl(2rU)ae4=74; z>g|ijnax*4W%ioJNz2j*I8|t^#yG!YPtB_erB$2 zIVA!p<2_zs#C|wof!&^W)aBzc=<9wS>Zl+~C3Z9op*Kr2VC1Mems33Bn)+8&m>fY& zs&cbrVK(V0erp|E!>>Dhiu)^0q3uiEjC==wOJ|I=5tJI1UqvQC5W+E@Q}lv)Lol#F zarUY){IB9BZikAT^ibkg`ih1KoQY4T68DS}%x;-7|J7m*p*(hhJ_I^lFvBki$1nJp zfbT(!mf#6mRoUCSz*iRUl1?T+6Pd>_x}VbBiTNP%Ab3M!_=B7cygC8@HUh{9F4!%s zK$#lVfHxd~X*N}lN+PTpRISaFdUfNl@kKBohJQElTX$)7#hb{$C64@bz^nYWY=R)%o~Ap2U2~X+SFR(jvNs^9v@xP| z0~_AlHSypMdO_*4@<0ObdeV43%h^|`9qvB_A(mb20>9F?r2|ma1HU+)W1(HYALGfx zC`mm8j*LA;JZK3v%|+cdT>niGyYH-0hnt4{@II(gxRC3tf%@b>qDw{jZOdmEjV4DJ zo_5FD=I`J9M_J@+%9dOek7BBKDT0FX;o@0#`^wrE$`jB%pRPe7;L!>P0V;sE?|n zVlX_RI2kGeTIS5f&C&h|&vC&McA>qmt3e43kK$cDhPV$DnSM0dDa09>HT1dz2z-6Q z(-BPR<&GPI?YeLcdwLSbv=KwkeiSRzR{uD^cKuu%J;f{f6m*Fq)M1Y+g;Uis&eOr| z#kIG4zT>zZpPD8OAHT7@U=2;{uE@!<2+=JjW(RelcvYIxRr|@R-J)g+L~Ql@yqyz1 zt9M3JcvJ8p5zlz>$jKE&AA#D*O5hj-EzU}3#DTx)T3^irE1T9X{GY5|9`eksWR=$QS zO4qy`wXn2bBX-g)Gj=yI|D>#;H*k-ysLxG%czV)`fI4T*@cx99r74Xj3`V0-bn#IK z0Ty;`irm<)$;6Q~^txIxMLjU1i)mDabe72oyf5Lq@QECvSegoKX9hWIEym=zfeC(2 z>=|Vrm4pJG&oNza<&BLz3WUx8?GxLk%5$D{vE_?>KJ-S@?ywj-YV+8YODQpbAW_=3 z__)6?ZQl5&)~XR^NCyi^Ef|o(o~C2^!ODqw*vSW(fUx|gTx_ozVSwIHpD~}L1i`mN zi>x@d#A_M9^d(*z6x|x81Kw3cUZv!dQkBEsBUG;$DB4O4n;ny%Sw(JTq+ zGlM?8Vfz}fBU&8lMeH1)u(n6ii}%Dab?V|ZAWvAfU*|$*&ktGnk1Gw zkXV7vQYi8iJGtqkabKrDzShv@Cf-U0^$wR z;7rofQ0f3j&j7$_&-xy^4+TvvHqLbrWsA+AP;;&*_aLB|cH}>@EP2gab_O?L?IU z;b6==z;iR4(ztkx>e1{^rIpNzg_x+%^uWs}5!j6mJYnnrRXslJEvD$+H7^RbpuaE! zTtRN2T>*bnA__!PO0i0kE(bSR*j?jqjjGZmjcg1(Veq;qLnU|+7P!#-JLkbx3 z*r=u3C~O{m96g^8e!BST-U1dfa%}FMDvxIG6L=Ylblv`!P_+=SK1U(qR!M(wdHF4+ z2e#edy)nUNVRpI(pacFA++DQ3K4V8jD+L$96H-@x>+UIC*Lp*WB?->uzsmyQsk7Dc zQEk?%PQz0?tw?ZqLNH2i%i`Z}R@{hpGc-H-L5uaS%*%4v($g~aG@7fs-^Tj)#8>K# z|CGWHf$2;8x_ltM>uHJTY!$S8DCY{%5EHvd2=)rb^oKDCY*BF_u{da{L)<~kR(XWF zlv6`q8gr)zZ3tLYaI)KYA^4{4lph{@JhD*)jYD6RW8v-1uRb>#5oo()pIB=n!UMb* z+;PA~(IthlDPBf2@uH_PIle7nrO3fAy|2x)d)G1-9Ym9}3Z+LvF#W9DJ4zW5x*J|h z__dFpWbdT8C1gPs9e*g0aZbRPkxANlcN@AtE|Yu|*^)7$m(wdtrE=`-Ij;STuEk+a z-NH)DKM*v}(Yf(Wy=-it;YCdcp>R48NBNYAdRHQV!kYp(8)5l9POR8}1^>u(>@bW= z+f?dlT_}wUfIs2_pu8-xYkG4(-sMOV8W1H8(4;r;f5mG11L}+WLF~<-gxc38bocsE zczP&Ax|~A-m?r@M?C$bwu@!0lkUU0xHK?pF;0i5C+lSD~JX0`Rx zrv4G4_g8NH1~(=St|sR_7ij18H!E_SHvUWAZ|Yiy*MN;(EHS*W++{w09>-E~OaN?O>>_|^ew02}gmB^&TQ*0@}& zAe{CiKVbcxjPaV`vZ?C5RXm$@;NQuF&o{1vQo2-(Wqc(p@c!<+6sm=jldlAZ$xHxa z&*oh$Ec*W>)BKGH}UuR3-aE^ix=Eg6(mP3L)2IqXv-vf?rSSoe>1);mmRXn^ur6z0v(sb$OHI)_E^_W)7}mc&rgUi2!{1OpLu zFjJW|peDQ-+fO_hO!NV#qdsZXS5RvLH>fP0?QZe=pOa?Q61`ZtePeN$-sQSr7A*jz zBwq?okg?>(WvSQe8^IMn&Et-LIqZW!2>uNIt#Z5(8>u>ZMT$*{C6VHzFOVrkDtJfs z$#ll27fcyESQ+0%Q~E+%X#?eehf}wkcZ{$;-~ScOSRFhZB5-eX_RFJBU9nPFX5B>k zQ$60(zY;Awia))eGPcJu>-}W88{joi`!@o|w_;oqKy*;3pwG=~>g&exyxsCKjI&%`ZU!2N#p#A?e{va8W`HgAw5`pZY~ zuP9UPN;j|AcRTaNBfhc*d+-F&DvW* zs%pMw`n42iq7Z=+Tlr!(nOU1td@tR1m;JKM;CGCY8S(bcmH3hwi;l=32HE9DM=KvP z8$ao{de73wh@04?P{W&fO=`^NiGAw_FZBgU8UvZ*)eOg`{HPXKZ=?C5Pj}`|diVsT z@5L%l~5?6~H-UmIET@L3Hzod=TOlp+b`$LpKUy3qf{l$6SLxc`F z{B8_jLeGCj9fo6ISin+42A&XRMk?O;fASNi0>L}^sU~PWb<(J> zen{jS#4R!}_c5^87wk4hff}iIj4$CGc=C?xcm`sa1*?63hZ{a021V&!n$@a=>y+I) z8iSP2-8z}(-67|J3W!s3FE?3}DD=G)JXXCF_x8l3;6(AvlP;vXCY>$RIj|XMNx!C-aeJI;F`Qu)Yo$- zf|*%vqn0QzqwLcO$OwNO@W3rSo(SzNX?=znvuvxL%NhX>Zut%Pu^>e+AqHl11Gnr| zp1xVVkWeeD?0xu#DO#WU6>mjBQt0qaHg$gv(uxM=%9dp7YFVSX=EK&XIp4Xp5>$Pb zZIJPH2z?EXDtP(NqP>FE_fdfD&ZIU>0UlNT)bQw?O@$qw^D!K3&BifHXvn(VSG%s! zC&THlTSukBhQT)Ou`k(VO2wa0x4cK$QT}mSeU+fRVs@Q#s384J=7>3`fd6B;(Mig0 z!vRJVp_l%LDZvULyo?^uTbuOyNcRwbZI^MAy3Q1iqSlkQMgCw#g*nK6hc=YMGH>JF zrU`7bv?pu^c(Re_Kx;P;Dpp`HA?4iXlkmJ$_3ISGyE>oy1P}c)SqwdSeelZ&M4k!& z_ms~UrUcJqN_W0_7jFkAoulbdbm7lXBO9PsP`$mzsxYVq0ljtJK6+F6al~|-TVMZ!h`y)ZJp6@WUJP9WxHTn{ zyYq$4w>dAnyKWzUdTU>ctenxnaoI~MFt@%cY9XY$Y8R`}Y)2;1;hkSn1WNs(l|ud{ zBldca(gt6nEbAn%rbAWfhJ})jIa7~#4R2I9fYnNPJdIhI*bu3Bbc7Eb#baGn5RUoa;GLG=Dp$IbpH*czs#pkH+`f}> z(QC-`kXVKPfSPb+jRN76&9!-ytaYp@8Wg#+HM3#NAz2!S(Ycx?F+C%F3^SZmf%%;9 zI61G1@MRvki)~n^S6`PiRnHByn-mg1L<=`!-F%B*_>4)o??^cCh0wrQ=Oz z?P|PIjn8=OPqHIb zok6vt&i8sNSZqtb&`p@THDutLH>NYGLtVz$0@dq(svX6~nVYdRD#=r7Fyf<8<@>l+XYiT^V*>{fWf9*Fdz4$!!yTX`c0lkjnn!dd^l@tBo{&t7Ph%7cWQs z=W-uymR9(TaHD5danMcE<^IAl3msW@I~9mdo3kA8rfV?E`VIXA?~HK9Y~%7e&P~(M z(Az#K9M(LuT%X`FK+KAdJ@-UGx{kV{NFP&`uJee5!hkQ)Go;%h|Tk%qvZO7Sv{KhYg?Cudw@m)vAGOa93XO4(8DMOX(OiVbi!z;Y%~)K8QCi@~67-!&zsh zvL!}?@+JLew}l1$A&uo|%fXwOIr|Ga<#<-(&S;WFk+qTIJhquTM$fW$$P7@N8<3qZcLALgzZoQ< z{^4G-0WJgwR~8RA#O!>=gmUPy#g?~jJcldW7FWC6E~RtmpTi!TWhOr+I*N{MstIke zN@JX0RLNs&?F`dK^ZA3Wb7`9jJIBmQ>#YP{3X^P>J_L+!HMcxqPln^R8q*nF7Xh_s zDQ#K`Lb#Lb{T#^WL_Vde{MC}mVU_oL^h7i?A!dx8 zn7Y&#$3U&dR;VEMbpU`g4{C9DWRWa+dK4z#j^Ln?rTKFt8i?i9gK1RN7nWEoRHNB; zdVShthK3~5o-Xpb<=>6jG`o9b7eZ!IS?Wl#BTjmP`1!7Oe?nKb%D@=U+nLkOrZtqX zn92px`$&4KtCX3)g{g`YrN6h&{~@({p9J;sop_k#;rcEWnw0tgZM>2nbfeJP$pwTn>!y85+O z|DfhZA&-oj02Gd?N9=Hur-$m)62Eqys#%s@rl4);^83MyZ_g*x|nzT-|IO?fn0DhRe5VGXMkUzf16TbhY*J awsdv>-+3zEIT8MDNfkv+h3Y33q5lI%4_0mf literal 0 HcmV?d00001 From 63e2a182ff4fecf5ab0a98e083cd80e934ebbea3 Mon Sep 17 00:00:00 2001 From: elijahbenizzy Date: Mon, 21 Aug 2023 18:00:56 -0700 Subject: [PATCH 10/11] pre-commit post spark rebase --- graph_adapter_tests/h_spark/test_h_spark.py | 28 ++++++--------------- hamilton/experimental/h_spark.py | 3 +-- 2 files changed, 9 insertions(+), 22 deletions(-) diff --git a/graph_adapter_tests/h_spark/test_h_spark.py b/graph_adapter_tests/h_spark/test_h_spark.py index 3dec1706c..9ec4a0485 100644 --- a/graph_adapter_tests/h_spark/test_h_spark.py +++ b/graph_adapter_tests/h_spark/test_h_spark.py @@ -5,12 +5,12 @@ import pyspark.pandas as ps import pytest from pyspark import Row -from pyspark.sql import Column, DataFrame, SparkSession -from pyspark.sql import types +from pyspark.sql import Column, DataFrame, SparkSession, types from pyspark.sql.functions import column from hamilton import base, driver, htypes, node from hamilton.experimental import h_spark + from .resources import example_module, smoke_screen_module from .resources.spark import ( basic_spark_dag, @@ -304,10 +304,7 @@ def test_python_to_spark_type_invalid(invalid_python_type): ], ) def test_get_spark_type_basic_types(return_type, expected_spark_type): - assert ( - h_spark.get_spark_type(return_type) - == expected_spark_type - ) + assert h_spark.get_spark_type(return_type) == expected_spark_type # 2. Lists of basic Python types @@ -322,14 +319,9 @@ def test_get_spark_type_basic_types(return_type, expected_spark_type): (bytes, types.ArrayType(types.BinaryType())), ], ) -def test_get_spark_type_list_types( - return_type, expected_spark_type -): +def test_get_spark_type_list_types(return_type, expected_spark_type): return_type = list[return_type] # type: ignore - assert ( - h_spark.get_spark_type(return_type) - == expected_spark_type - ) + assert h_spark.get_spark_type(return_type) == expected_spark_type # 3. Numpy types (assuming you have a numpy_to_spark_type function that handles these) @@ -341,13 +333,8 @@ def test_get_spark_type_list_types( (np.bool_, types.BooleanType()), ], ) -def test_get_spark_type_numpy_types( - return_type, expected_spark_type -): - assert ( - h_spark.get_spark_type(return_type) - == expected_spark_type - ) +def test_get_spark_type_numpy_types(return_type, expected_spark_type): + assert h_spark.get_spark_type(return_type) == expected_spark_type # 4. Unsupported types @@ -379,6 +366,7 @@ def dummyfunc(x: int) -> int: return dummyfunc + def test_base_spark_executor_end_to_end(spark_session): # TODO -- make this simpler to call, and not require all these constructs dr = ( diff --git a/hamilton/experimental/h_spark.py b/hamilton/experimental/h_spark.py index c98723746..c5ada3073 100644 --- a/hamilton/experimental/h_spark.py +++ b/hamilton/experimental/h_spark.py @@ -215,11 +215,10 @@ def python_to_spark_type(python_type: Type[Union[int, float, bool, str, bytes]]) _list = (list[int], list[float], list[bool], list[str], list[bytes]) - def get_spark_type(return_type: Any) -> types.DataType: if return_type in (int, float, bool, str, bytes): return python_to_spark_type(return_type) - elif return_type in (list[int], list[float], list[bool], list[str], list[bytes]): + elif return_type in _list: return types.ArrayType(python_to_spark_type(return_type.__args__[0])) elif return_type in _list: return types.ArrayType(python_to_spark_type(return_type.__args__[0])) From 49ed8d630894c58ede18349077cd96053c05d6ef Mon Sep 17 00:00:00 2001 From: elijahbenizzy Date: Tue, 22 Aug 2023 10:59:56 -0700 Subject: [PATCH 11/11] Fixes prior bad rebase --- hamilton/experimental/h_spark.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/hamilton/experimental/h_spark.py b/hamilton/experimental/h_spark.py index c5ada3073..817906119 100644 --- a/hamilton/experimental/h_spark.py +++ b/hamilton/experimental/h_spark.py @@ -220,8 +220,6 @@ def get_spark_type(return_type: Any) -> types.DataType: return python_to_spark_type(return_type) elif return_type in _list: return types.ArrayType(python_to_spark_type(return_type.__args__[0])) - elif return_type in _list: - return types.ArrayType(python_to_spark_type(return_type.__args__[0])) elif hasattr(return_type, "__module__") and getattr(return_type, "__module__") == "numpy": return numpy_to_spark_type(return_type) else:

e(B~s87NW`S8h95NjZU~z@)Z|#b49w!{&1nFy2t{wM_RXr zBN|f^%N6x+0hRel3_cErD5>X=uetAxJOdlm42t<2xVF6&uH=tjnAM=$^H+--Qbg3v z{+=!ds5ffB(p0PPhLCU5tTq-eZN(JtvxxtPsW+my7XPcMxdL4OntrkIIX{hrgH4c@ zDt7rM@#=zbn*76=zmmliXaJIuMe5xv$#<1#v|yy@Trmks9{2R<)G+#M4xWhri8*9) zIG9!iShX-G$Bm==;LJKP)^0|Yps?0X!P)59n+y$$VK2L7O&|jbDkz`S-cR~)D*xoK zYB@BtWdEzGGd}CS`rlNYQWU1cLcTemUT#91=l@5kt^##S$alZdVd=BEVr?4D+FzHV zl>AQ>0-w8^HN)#(jbvTQJq1yxitgz&nJldW4U2>pURMEUaN$7To!q7#ZD!WL_ZIyZ z_5a7chW|2$*lc$!@0ztl^ZFon$Vj#*x7yyPIL4v}>B|4w9DNjToBzF+0NZOL*FRC7 z;t*z$CSURDsPyqGg~{P?Ct9#jB;fx@CU*T;t0bwaL>LDQz6lF|=i0^ckQ3}-ge|6u zkN^K28P_m3UR4KS9$n@8kg& zpt}^jd^*_8lJlt%BQ8Yr3SaVRikRs0NKX9%*HTlQ=;}lszKk&Dt;l%$JCU4^|8Ik< zWd1vc-XTEB2M5}`lXqu9S@?(6CJhHCRUcX`mlnkGOZX5TH?vaB*>dmm;x!Q~+2ly+ z23@C_vZf6KmtHV44@(RoTfwXAf`1IY`fmoUEbn99TzI5?LUSgU+@tP^ByWV`RJ9|c z2fu7a{m=a1e|L`hDd5P*1k#NsLBmVHIAU*?ifphTqM7l4Vg!=WK->-Indwu>sysAs z#%4A*;TT8>3j|~G3?gr5;b4HG2TOg2sPg-^0m9R8_1o!-Cd_-pb~}wUqhb9|{Jg;J zy*qe{;hqO}v(qiO41c|V`Cne}-@3;CWw3gw>+mFD;gd}0Y5^XIKxc!olK-Pb;sS`I za$^a)#zKEg{vab{zQH~d|K+7x-?8YWr{Boa88He5qOJ7q=31H%Q}XB(kVGXGgH<0W z-N;{Q?6|(eim$uz`PvihPhU9xpNd!hw>A2b&&`-A!!5<0ONy62OE<3+^Ku>fO&<09 zn{$Mp_FJjaE+@`-N=b97sFv@wY->J$xIU7prjvaO3v9<$!2EX1UK`pKLAXnNJi6|A zu3g}N_9c^uRX(^P05xCZCH7_zl5$C%R{d%Vg1kIb`xlP*6Ihglc9-#p(j$b!rM z%{!=@70hSus`jLBt8haUx+#^XLQ_`vqH)A(1ykg9d)|G{Kh~2Vt=zPDrh>eFtJ{;# z?$28^RarSg1c1zjI!AuPzG5s@(#QDMK&638@58X8hDU#_Ghxn@I-8cfB#m8rx57f} zOR4;iP-8n_L(E4Uq52cq252QF-6SB~#CMPK{Yh8kDh%F@jl3dl%qci-PBmy4EP4q- z6^i3OufY6lgeCc6*bR$TBwRl+@fx*%U<%mO=n zkIDN3LHOO=)__-mvwV8Tt(yQba@~D4sUPR^%up{UEBNNgdyIFBPb5um7=-|cstKk) zS8Tnz_|D|sBlR=o>;d;kQ6GF}Cx!&pof^jVpbQUk$^mQ*u(Pui(SjJ0&nlLo0OxRC zZ1M0T4U!*c?t(0s;#VZDL0q;H97%p!XaG)PO8E zO6DDJ>zhNR%8FvQYGGxMaSC@W`*KO6{fggQZAGR-~qL5#n=D@HKd2&-oP!ekVugd-k8 z>f;b8)=8~18|q^XKbQk3RDWGUfO=#0*2phXjdBdDNdxyjeA!JM8;A*df(YvO`he}f z_E41tINzyj-(%>)`^QCuC8o@r?^!)QGq2v4UIkQf2O@?;p)}52KtYK`X3{=>|e5}cgbtQm3QS8Iv@CV{=KNfg@7&TuZ(mA zxBIEOBg&URS=jFynnXmzE*YLrt?h?b(I)u0<^P_7A`QKEWmdCDz7| z2CY>;3&UAC1zbzQgE@Gy*Qhk4WJ2}w_}9kC(^QMsv_97pg&E49veMQ`LAnbm>jHDM zX&mP3@sD{cELZlZ2_2SNpJoVqA4&7WcGKS&%GjLO+)DHFOQ|aZB8||-HP_E zYxU6>^Ysy*ru(jq*p- z{!|6UZjXGssw4|nMX*Z@hx$*fv>g?n`jA0VCiR`LyNuuvVn8TYJ4RFAjK6JFy4 zcf7J?2*HJHjElEZsM5U3#I132BoLCzT}_lD!B4BwJ7V9gewF=Sr^^VW7JNchWciO( zCIPEtsz}*`gatOQa&yJ}y>K8S&HC88XS!?YYLO9GTi}c9>rX{#Exgz{$TV#R&HBIx zXExv$FeSWa*?DRKa{wy*ee1-T_As(2mb@0tszMbRh|fDcG)X!JqX>|aM06Y>_vMY5 z;U3{e2~?Aq0rvcR7pK2gd&njL75e^Dh5xXfIWn2^2d9PsZg-rV@N&-ch5P5T-UCUj zR65?+>sDIT-Y{qCsE}@fyf4@ypIJ64dM|$#sW_U1*>Cmefvax;Zg0`Z`>SvprDekB z#*h08Q7=$Nh-#KR6B#LEL4fM~O;}n>OIO~Flu}&JKb7*4p=}&svEo$U`_A9*u9?7?1u0 zU2Jx>ialA1rDYQYU(_!m$xZU2KS4w?JY3GTs4IzqwG0(f%3_?4nxAy;m3LJxIZXcZ z=K-wBf6RScLiZ1oMDy!wqw2;JoEDr&dw-^UR%s(c&V8TAJ+Tekukfm!*nT$b$;kt) z9nK=xUwF~1@Q~NqZ0MX-${^y*I*&AFf@{Abpr=-~ZHnC6gw}>n0h=kqvEZ=^p1KJI z?e;_7$uDu)_#5W9x~)d z`zm+b`umHkw->$y1=>{ofw3IIfEQ{vsrEW+C!?gtC*UnR*UGnzgq?H~mMX+h!2v?t zY{h!;0jNR$?TlY328P<=_ZS(iDgt8IQC!;PrSmE&S>czEoBpq@KfBSB7`Y)BasRvH z<%Z|BbQ==4u~;A{c_d zE~0mCANP7&`8{IJtE;0710WN%RyqP~IU02oXF4W9dn`%l{bMLJM8ZQX>BOS@vf?Mn z4hKrp^YGxS&4-6onDbF^{`(8*w7j4^xIHEXa#rhT$Yh4YWcHL2`HdbZTI)3k{9WgS zLIx~jMsg)1)um2Zq{?6%HJ8xox8TD5>Lque?Gi_k&8>7XuXBYsL zss!~vND%Ry*S2T&)@{vsPFX<%FKK|5n4~`~#@tYH@8!RKuHgK+q6;rKN^=VN>`*{c z`SoMtKyWOb_~I01-??3eJA4B;TU2>EUG=JFJ3*M+yy-m4{q+s6z7AAap^X_0)Fbd88jlgo>e2SlRvZESv zvi3v(O|N+JQOR=X1deOp5I#qZd}3(8!Oy?aeon)!j#?NCL|kv2dN%=}hoo?o^>1?i z$3u`t2l4l-9XGb{T@8_wy4O9I=5CnK8?gFtaZ1_}b~CDovVV;Erk`!0yhP@shT0Ts z>i8R4T86h1p0BGN(GTc~`Q6x}1Lb8qDmk=?Jlx0?FR+S1>SMNNP06c%JAt~5Kh9G7Db#Px-QJSV>lo$;(`B}79wg|$m$=EP`)XdK?13VNead=j{}`g0!FF+8(LR9f{&Nm30yXQt}FtItJ}B6Q34BQM;=sV#@eo^2c3J`h7JRfi5}{^gayDJnHb^AiuZ`=m+gF zu0TDx$*EP!AE@oooxs%*J%rXOQ7Z&51~mCePo8zE+F!5NL4kVQ-TFynmWRqW)XwJi zL{%UPBTzsd_uz9s&m4Mk=+~4Gyb6lR{xe;nGPBx)1?jbfs#`k`f=xT`WPzc1;}D=4 zrCWLGMDEFs@}B`VJ9}=fn$%TEh@Wm?cG~xaSvGv!kuRXeOATVFl62$17GAQcw6MT( zqheAp!;9nZSJH=zAD!VZo4#{{F+SU*!t4xCe?5E+WoeeZ;&gBlrT81=(%bSf>nf%U zqdis2gBed(#^eZ2d?TgJBPUNMC zBU0pDWPSiTrt-eZ3{>^%C@j>fRHwp#Z2~}Or~sW-o3@CLhi4bgR7ftrKsImR9N5*9 z>m1Uq1T@%92~jv|JWwF1Dfp@Ul1_Z&z6w9AO1D}Fo(zy|0M+JsN6i}~!ci^Yq0tsD zeAMmurWKELC;k{>L{Ez#0ho!`KoglNi*4@7y5zfaY`eU8ZDvfBfIilBx$zy7V%o{^ ztK>CyVCx@M86SjVLPdGiVhfnDYsZ)#35CSHonqT0}#U z^e3$3(l&Zg*-6B{qQ)JDc^XwIN<<0-+@i^VnbHz!AAePN$ZfqL;0fLD{~8KkSivk? zzHuubvRM1hk11B8-_sg?l>h>F73C)cFX=*vi-Go%-B|07>dYgFrR=8u)>ORdHh)Qj zgU9)#&q+1KbWe~Nv5IX_XVvhvhSyMZV`!4YpwS@{e=bDI2F*b6)}lY2-GR`h^-Fws z^%E!_G0KX=v2$j96byQGzHJsX@>Q0$DiC=#R5d(vKkvYN>FRc6DKYcqm|f_U^pL^gOMi-s_Mq zOTv?DO`bP-j&^0_o^-fZ7=*Z`0}(kjRz3$nhVD`b?_ zVJT0N(I0p5-e2x2P;#Y^o^uGY@EBY-Sv*}y55*m~W4LqDOAX0ro&D}dO)QHMAVbAw ztID1YrSY-n{8o;k;v*~Ced3jgjjdBE6eiw94|JKmM8FUopRR$s+ zmJRlqX=b6Ta7p_Jel4334Vb`h#T8Xyg_4~!kgsm6?z^wWRgK)Yn>H*4ZD!5z4xZk> z`2KD{Q;X#Y15;#bw3r~1wo=86HfBAyt_;3r>?tcKR?k#aq2gD7RtGz^T~J9)+P`6U zS3{99-8Sr%8R^Idsh}})Z3HhWq7T3x-ri04(WZ|=TFZiVE6GK->4Mr?*N-3v@y2b0 zstz+Ri0*xKO4VU{Ju zb6>_qQ85CZAm#W_^v%yX2~lJ8e5;Z!=8HC(U=lIHCjR(f`fHFVZ+@QZ!Nzz*#&Kkz z^RMSuQi#h~@SYKpv%G2oA)iP@?K$F?IY@7e}+0|6AnO% zv?nxTP%`R=b2R1s5UUX(Y#8D$;Y5*{@x5*8SbC`k1IV4WcG?nFN3|jlhLgHEDWPRZ z`MeA7cqpg$w-!@R%Y*(;Q51bDe7Y3|mO#^Nu1x4~hT!B0oc3U0AjD%g@v#|ZRJq=g z`CZrZGU!&-o_3i*uS37NDoAZ|nXWG4dz4A9DvKd~*mA)Lpe*eho&c+6;sq3$HkL)U5=bS7eGh zN>qzF;|O?@wC}^4 zg~WLOzSVc5GrK)A=JDM>G3YCpLSTz6Y1vUFszg1PE8v0eEQ6mX*QbghkK2Y=(Px)4 z6LVreawaa{O&(v&6shmcOrhXB?am16B2)mn90MfPgv-pV9QAK+=U==e^TpDappV3o z#z^4~jk#FFJ=}NUIs{s|ridM2BFzswDQe^dn{H1_K#$$c2#Dk$(k`p%89f#3ntdxD zpS*l!7)ZQft-E1?IuNg0LV&H+Bd+h1oLhY&WyW^;;DT;>7yu>){w3o3V#TAnA8ne= z)fTlmVum;Kul2((ep}~5t%~I?IiavFmo9w-*|X!Omf1Hs(c)5l@(gZzk+-CDa7v^G zJZ8;epJ!(tQ(h;r;nl}nN5HOxLbA7o8lyim*jZKZ%t_Qz_Fd264nIJjrMvTV-!yoL zl8ozfg7Liaj@EC@@AV}M7WU6R)X+ByjR%2$`Nv)SZ1vQ+Y)(Vrq8Kc<_nkXLt2&T$ zMCq0TL%@Ea@L$D ze~DkGQ%jfA!_b-bBQ=n`gY%j9AB>a;b2Gn}e}nznze60=^3_}0yH6`xA(9*BArS|Hxhz?(nB{4-7<72(k&@n(v1wAA~58@#2tUr36+W8)N`EOg5?b2ibYbfkG@uIA=<4;F%%0|#rmM*`Xp?B5=3fM zq>(Q5`D9V}&J(JGDfKMOW-uO48WPxe$XRl9g{~V|q+4W0tDR7AR#U>f_jD{}3yEW& zrlD(&iY9-3yjeF>HL+wzA!VnOJ|RRLb!el&8JOa4L(jPUG$1jf;u({_n%qAKpd!oR zQMQ=1lE>XvG(K;>Z}DN)p_dn!9TwI-}=0a zg?bE(!6Ho)hDUOYpRKD%`u$fPdbZ}OL<`EF`=geyU9b9_)Pho^Gl9J!n3&z2c)PMBO3$~cuvPLN5EBrmdbfR*D;USTNpg~AOLd9Q4Y+g((Gt03V=v2NW zmD`*E-@9^GG&(oB(58&A80U63O(!$Bko%s+j4TBia`ta88%|buTaKOxZT7p3Y76=z zlly)~#=m~B_7X%F&DTm%CjloY+&AA0NxJXvPd1yE?(xgiGe}pP<+x@3ocB-Dy(Q>) z;cj-)Zu2e#u~a|IDk#ra@}>w#l#B*u=P|F!!^Lqw}pue$HiEl@Kb4Wby zyIv|nDWOcd16Anz=#So1*k)JW;B3Sf73!e5jXj%YS4)2{uvhep0F6k{#n-prDeWeH zvB&UchNy4{?@-s^31`sB(A}337Ehrd6<|EHf~YD`U)pK5)H6h|NAYyEdG|S3##6~- zh{4h07uDu_2MnSa%x-Ob<7)giQBPfRqee99X~ujXNjRAuoq6tmeKkr1e5Mt?L!F9GG!KlEH@&Od@^EZSy# zeG`s;F)Q(%^_&GCEwL%G*x2%COL*$PXnPT=)MuY%&MJR7@f1^gd^3oT_iC+(cOUt1 zMkI3LY2xv4+KSJuPD@s|!hy$9nf3>*KA#TLEE~(fuGCJJxnMSh^+!o1v@p{$llBY$ zsa}@on9TV2=s$;gR)ybnS>Vij0ZX5Dz3`K1IYB z9rT=b%UQk0@w&D)tp!+X(G>;)!03;LG${P2?vzP4LjUyNw?$9F5#vel-@(IfPe{E) zM*Qu-<9@v~MivaNt8M&kn_)y@oiHdac>Bx#gYVBr!|12jo8x{o&bPt^ztQ2B65ARn z!gH&zKgSVA^6L@-A!aZdPt83QI4s(ArB9z^EPSbX^UrdLw8eFU5Y{Gi=rulRjnb9t zRfVL(4#sE%OTMO3f5y8@P6#yTd-g{pIzf-^$z0HV74HSII<5T#fb^sq%1Xl22E<#` z8R)o)A~ex&7yS94`8GXU{;?_Jatux_kQp!lQJGlg z+f2bKG`PRc^Ig}Au@A}09B3e^^vLsqkZn-@GsfZhQ7T}%pTqZ6$s)NAiXjH-4R~9o zA!nQrV7>*JjOzQP?x1rJ5dQKWH|bzrCH5PFqXI2RO89Rp=Zq3D-szOv{JpubWbx9! z0>Aae2b;aIj4<^3SfqFK65}mhxI6uiJV_VEbUq8EA#h2O0_A0&=`651M7-IXY#b1H zcisf;u|i@^NLr+I%Hv?`?}_gd!=dAwB_Jg4^9;j7eh)1&7vuB56O;^MfjP4LX;m@A0!0P(r6l2xk9u11Oey}XrXVQ|PvyW+RS z@AcYzjt8}e_;R2A@Lh=ZRI7%B3;cXC3cNdKQ-aAp4pGN7;cY~j)p>M}@E*L#kM=