From 1243f90212f5b64bab72bb5954f5e739a6b891b3 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Mon, 12 Aug 2024 00:43:59 +0100 Subject: [PATCH 1/4] Remove deprecated SubDags This PR removes SubDags in favor of TaskGroups fro Airflow 3.0 Subdags have been removed from the following locations: - CLI - API - ``SubDagOperator`` This removal marks the end of Subdag support across all interfaces. Users should transition to using TaskGroups as a more efficient and maintainable alternative. --- airflow/api/common/delete_dag.py | 26 +- airflow/api/common/mark_tasks.py | 106 +- airflow/api/common/trigger_dag.py | 30 +- .../api_connexion/endpoints/dag_endpoint.py | 7 +- .../endpoints/dag_run_endpoint.py | 4 - airflow/api_connexion/openapi/v1.yaml | 17 - airflow/api_connexion/schemas/dag_schema.py | 2 - .../schemas/task_instance_schema.py | 2 - airflow/cli/cli_config.py | 8 - airflow/cli/commands/dag_command.py | 3 - airflow/cli/commands/task_command.py | 2 - airflow/dag_processing/processor.py | 2 - .../example_dags/example_subdag_operator.py | 70 - airflow/example_dags/subdags/__init__.py | 17 - airflow/example_dags/subdags/subdag.py | 56 - airflow/jobs/backfill_job_runner.py | 28 +- .../versions/0003_3_0_0_remove_is_subdag.py | 76 + airflow/models/baseoperator.py | 5 - airflow/models/dag.py | 174 +- airflow/models/dagbag.py | 59 +- airflow/models/dagrun.py | 1 - airflow/models/mappedoperator.py | 6 - airflow/models/serialized_dag.py | 21 +- airflow/models/taskinstance.py | 7 +- airflow/operators/__init__.py | 4 - airflow/operators/subdag.py | 247 -- .../celery/executors/celery_executor_utils.py | 3 +- .../cncf/kubernetes/operators/pod.py | 3 +- .../kubernetes/operators/spark_kubernetes.py | 2 +- .../fab/auth_manager/fab_auth_manager.py | 2 +- .../auth_manager/security_manager/override.py | 4 +- airflow/serialization/pydantic/dag.py | 3 - airflow/serialization/schema.json | 1 - airflow/serialization/serialized_objects.py | 5 - airflow/utils/db.py | 2 +- airflow/utils/task_group.py | 8 +- .../js/dag/details/taskInstance/Nav.tsx | 17 +- airflow/www/static/js/types/api-generated.ts | 8 - airflow/www/views.py | 11 +- .../administration-and-deployment/pools.rst | 5 - docs/apache-airflow/core-concepts/dags.rst | 92 +- .../apache-airflow/core-concepts/overview.rst | 2 +- docs/apache-airflow/img/airflow_erd.sha256 | 2 +- docs/apache-airflow/img/airflow_erd.svg | 2206 ++++++++--------- docs/apache-airflow/migrations-ref.rst | 17 +- .../operators-and-hooks-ref.rst | 3 - newsfragments/41390.significant.rst | 14 + tests/always/test_example_dags.py | 5 - .../endpoints/test_dag_endpoint.py | 78 +- .../endpoints/test_dag_run_endpoint.py | 16 +- .../endpoints/test_dag_stats_endpoint.py | 2 +- .../endpoints/test_log_endpoint.py | 6 +- .../endpoints/test_task_instance_endpoint.py | 60 +- .../api_connexion/schemas/test_dag_schema.py | 10 - .../common/test_delete_dag.py | 18 +- .../common/test_mark_tasks.py | 40 +- .../common/test_trigger_dag.py | 40 +- tests/cli/commands/test_task_command.py | 16 - tests/conftest.py | 10 +- tests/core/test_impersonation_tests.py | 5 - tests/dags/test_clear_subdag.py | 72 - tests/dags/test_impersonation_subdag.py | 55 - tests/dags/test_subdag.py | 86 - tests/jobs/test_backfill_job.py | 148 -- tests/jobs/test_local_task_job.py | 2 +- tests/jobs/test_scheduler_job.py | 4 +- tests/models/test_dag.py | 279 +-- tests/models/test_dagbag.py | 269 +- tests/models/test_dagcode.py | 2 - tests/models/test_serialized_dag.py | 12 +- tests/operators/test_subdag_operator.py | 376 --- tests/operators/test_trigger_dagrun.py | 2 +- .../fab/auth_manager/test_security.py | 40 - tests/sensors/test_external_task_sensor.py | 14 +- tests/serialization/test_dag_serialization.py | 13 - tests/utils/test_cli_util.py | 5 +- tests/www/views/test_views.py | 4 +- tests/www/views/test_views_acl.py | 26 +- tests/www/views/test_views_blocked.py | 90 - tests/www/views/test_views_log.py | 4 +- tests/www/views/test_views_tasks.py | 36 +- 81 files changed, 1367 insertions(+), 3868 deletions(-) delete mode 100644 airflow/example_dags/example_subdag_operator.py delete mode 100644 airflow/example_dags/subdags/__init__.py delete mode 100644 airflow/example_dags/subdags/subdag.py create mode 100644 airflow/migrations/versions/0003_3_0_0_remove_is_subdag.py delete mode 100644 airflow/operators/subdag.py create mode 100644 newsfragments/41390.significant.rst delete mode 100644 tests/dags/test_clear_subdag.py delete mode 100644 tests/dags/test_impersonation_subdag.py delete mode 100644 tests/dags/test_subdag.py delete mode 100644 tests/operators/test_subdag_operator.py delete mode 100644 tests/www/views/test_views_blocked.py diff --git a/airflow/api/common/delete_dag.py b/airflow/api/common/delete_dag.py index 1cf7ffec8b9e4..11b046648c5a1 100644 --- a/airflow/api/common/delete_dag.py +++ b/airflow/api/common/delete_dag.py @@ -22,11 +22,11 @@ import logging from typing import TYPE_CHECKING -from sqlalchemy import and_, delete, or_, select +from sqlalchemy import delete, select from airflow import models from airflow.exceptions import AirflowException, DagNotFound -from airflow.models import DagModel, TaskFail +from airflow.models import DagModel from airflow.models.errors import ParseImportError from airflow.models.serialized_dag import SerializedDagModel from airflow.utils.db import get_sqla_model_classes @@ -64,18 +64,6 @@ def delete_dag(dag_id: str, keep_records_in_log: bool = True, session: Session = if dag is None: raise DagNotFound(f"Dag id {dag_id} not found") - # deleting a DAG should also delete all of its subdags - dags_to_delete_query = session.execute( - select(DagModel.dag_id).where( - or_( - DagModel.dag_id == dag_id, - and_(DagModel.dag_id.like(f"{dag_id}.%"), DagModel.is_subdag), - ) - ) - ) - - dags_to_delete = [dag_id for (dag_id,) in dags_to_delete_query] - # Scheduler removes DAGs without files from serialized_dag table every dag_dir_list_interval. # There may be a lag, so explicitly removes serialized DAG here. if SerializedDagModel.has_dag(dag_id=dag_id, session=session): @@ -86,15 +74,7 @@ def delete_dag(dag_id: str, keep_records_in_log: bool = True, session: Session = for model in get_sqla_model_classes(): if hasattr(model, "dag_id") and (not keep_records_in_log or model.__name__ != "Log"): count += session.execute( - delete(model) - .where(model.dag_id.in_(dags_to_delete)) - .execution_options(synchronize_session="fetch") - ).rowcount - if dag.is_subdag: - parent_dag_id, task_id = dag_id.rsplit(".", 1) - for model in TaskFail, models.TaskInstance: - count += session.execute( - delete(model).where(model.dag_id == parent_dag_id, model.task_id == task_id) + delete(model).where(model.dag_id == dag_id).execution_options(synchronize_session="fetch") ).rowcount # Delete entries in Import Errors table for a deleted DAG diff --git a/airflow/api/common/mark_tasks.py b/airflow/api/common/mark_tasks.py index fa6ce835a919e..d0be6b86d21cb 100644 --- a/airflow/api/common/mark_tasks.py +++ b/airflow/api/common/mark_tasks.py @@ -26,12 +26,10 @@ from airflow.models.dagrun import DagRun from airflow.models.taskinstance import TaskInstance -from airflow.operators.subdag import SubDagOperator from airflow.utils import timezone from airflow.utils.helpers import exactly_one from airflow.utils.session import NEW_SESSION, provide_session from airflow.utils.state import DagRunState, State, TaskInstanceState -from airflow.utils.types import DagRunType if TYPE_CHECKING: from datetime import datetime @@ -40,6 +38,7 @@ from airflow.models.dag import DAG from airflow.models.operator import Operator + from airflow.utils.types import DagRunType class _DagRunInfo(NamedTuple): @@ -101,14 +100,14 @@ def set_state( Can set state for future tasks (calculated from run_id) and retroactively for past tasks. Will verify integrity of past dag runs in order to create tasks that did not exist. It will not create dag runs that are missing - on the schedule (but it will, as for subdag, dag runs if needed). + on the schedule. :param tasks: the iterable of tasks or (task, map_index) tuples from which to work. ``task.dag`` needs to be set :param run_id: the run_id of the dagrun to start looking from :param execution_date: the execution date from which to start looking (deprecated) :param upstream: Mark all parents (upstream tasks) - :param downstream: Mark all siblings (downstream tasks) of task_id, including SubDags + :param downstream: Mark all siblings (downstream tasks) of task_id :param future: Mark all future tasks on the interval of the dag up until last execution date. :param past: Retroactively mark all tasks starting from start_date of the DAG @@ -140,54 +139,20 @@ def set_state( dag_run_ids = get_run_ids(dag, run_id, future, past, session=session) task_id_map_index_list = list(find_task_relatives(tasks, downstream, upstream)) - task_ids = [task_id if isinstance(task_id, str) else task_id[0] for task_id in task_id_map_index_list] - - confirmed_infos = list(_iter_existing_dag_run_infos(dag, dag_run_ids, session=session)) - confirmed_dates = [info.logical_date for info in confirmed_infos] - - sub_dag_run_ids = ( - list( - _iter_subdag_run_ids(dag, session, DagRunState(state), task_ids, commit, confirmed_infos), - ) - if not state == TaskInstanceState.SKIPPED - else [] - ) - # now look for the task instances that are affected qry_dag = get_all_dag_task_query(dag, session, state, task_id_map_index_list, dag_run_ids) if commit: tis_altered = session.scalars(qry_dag.with_for_update()).all() - if sub_dag_run_ids: - qry_sub_dag = all_subdag_tasks_query(sub_dag_run_ids, session, state, confirmed_dates) - tis_altered += session.scalars(qry_sub_dag.with_for_update()).all() for task_instance in tis_altered: task_instance.set_state(state, session=session) session.flush() else: tis_altered = session.scalars(qry_dag).all() - if sub_dag_run_ids: - qry_sub_dag = all_subdag_tasks_query(sub_dag_run_ids, session, state, confirmed_dates) - tis_altered += session.scalars(qry_sub_dag).all() return tis_altered -def all_subdag_tasks_query( - sub_dag_run_ids: list[str], - session: SASession, - state: TaskInstanceState, - confirmed_dates: Iterable[datetime], -): - """Get *all* tasks of the sub dags.""" - qry_sub_dag = ( - select(TaskInstance) - .where(TaskInstance.dag_id.in_(sub_dag_run_ids), TaskInstance.execution_date.in_(confirmed_dates)) - .where(or_(TaskInstance.state.is_(None), TaskInstance.state != state)) - ) - return qry_sub_dag - - def get_all_dag_task_query( dag: DAG, session: SASession, @@ -208,71 +173,6 @@ def get_all_dag_task_query( return qry_dag -def _iter_subdag_run_ids( - dag: DAG, - session: SASession, - state: DagRunState, - task_ids: list[str], - commit: bool, - confirmed_infos: Iterable[_DagRunInfo], -) -> Iterator[str]: - """ - Go through subdag operators and create dag runs. - - We only work within the scope of the subdag. A subdag does not propagate to - its parent DAG, but parent propagates to subdags. - """ - dags = [dag] - while dags: - current_dag = dags.pop() - for task_id in task_ids: - if not current_dag.has_task(task_id): - continue - - current_task = current_dag.get_task(task_id) - if isinstance(current_task, SubDagOperator) or current_task.task_type == "SubDagOperator": - # this works as a kind of integrity check - # it creates missing dag runs for subdag operators, - # maybe this should be moved to dagrun.verify_integrity - if TYPE_CHECKING: - assert current_task.subdag - dag_runs = _create_dagruns( - current_task.subdag, - infos=confirmed_infos, - state=DagRunState.RUNNING, - run_type=DagRunType.BACKFILL_JOB, - ) - - verify_dagruns(dag_runs, commit, state, session, current_task) - - dags.append(current_task.subdag) - yield current_task.subdag.dag_id - - -def verify_dagruns( - dag_runs: Iterable[DagRun], - commit: bool, - state: DagRunState, - session: SASession, - current_task: Operator, -): - """ - Verify integrity of dag_runs. - - :param dag_runs: dag runs to verify - :param commit: whether dag runs state should be updated - :param state: state of the dag_run to set if commit is True - :param session: session to use - :param current_task: current task - """ - for dag_run in dag_runs: - dag_run.dag = current_task.subdag - dag_run.verify_integrity() - if commit: - dag_run.state = state - session.merge(dag_run) - - def _iter_existing_dag_run_infos(dag: DAG, run_ids: list[str], session: SASession) -> Iterator[_DagRunInfo]: for dag_run in DagRun.find(dag_id=dag.dag_id, run_id=run_ids, session=session): dag_run.dag = dag diff --git a/airflow/api/common/trigger_dag.py b/airflow/api/common/trigger_dag.py index f22755ec640ea..70fb999f542a2 100644 --- a/airflow/api/common/trigger_dag.py +++ b/airflow/api/common/trigger_dag.py @@ -43,7 +43,7 @@ def _trigger_dag( conf: dict | str | None = None, execution_date: datetime | None = None, replace_microseconds: bool = True, -) -> list[DagRun | None]: +) -> DagRun | None: """ Triggers DAG run. @@ -90,21 +90,17 @@ def _trigger_dag( if conf: run_conf = conf if isinstance(conf, dict) else json.loads(conf) - dag_runs = [] - dags_to_run = [dag, *dag.subdags] - for _dag in dags_to_run: - dag_run = _dag.create_dagrun( - run_id=run_id, - execution_date=execution_date, - state=DagRunState.QUEUED, - conf=run_conf, - external_trigger=True, - dag_hash=dag_bag.dags_hash.get(dag_id), - data_interval=data_interval, - ) - dag_runs.append(dag_run) + dag_run = dag.create_dagrun( + run_id=run_id, + execution_date=execution_date, + state=DagRunState.QUEUED, + conf=run_conf, + external_trigger=True, + dag_hash=dag_bag.dags_hash.get(dag_id), + data_interval=data_interval, + ) - return dag_runs + return dag_run @internal_api_call @@ -133,7 +129,7 @@ def trigger_dag( raise DagNotFound(f"Dag id {dag_id} not found in DagModel") dagbag = DagBag(dag_folder=dag_model.fileloc, read_dags_from_db=True) - triggers = _trigger_dag( + dr = _trigger_dag( dag_id=dag_id, dag_bag=dagbag, run_id=run_id, @@ -142,4 +138,4 @@ def trigger_dag( replace_microseconds=replace_microseconds, ) - return triggers[0] if triggers else None + return dr if dr else None diff --git a/airflow/api_connexion/endpoints/dag_endpoint.py b/airflow/api_connexion/endpoints/dag_endpoint.py index 1895bfeaec762..749c3bf14ddf3 100644 --- a/airflow/api_connexion/endpoints/dag_endpoint.py +++ b/airflow/api_connexion/endpoints/dag_endpoint.py @@ -106,7 +106,7 @@ def get_dags( ) -> APIResponse: """Get all DAGs.""" allowed_attrs = ["dag_id"] - dags_query = select(DagModel).where(~DagModel.is_subdag) + dags_query = select(DagModel) if only_active: dags_query = dags_query.where(DagModel.is_active) if paused is not None: @@ -179,10 +179,9 @@ def patch_dags(limit, session, offset=0, only_active=True, tags=None, dag_id_pat update_mask = update_mask[0] patch_body_[update_mask] = patch_body[update_mask] patch_body = patch_body_ + dags_query = select(DagModel) if only_active: - dags_query = select(DagModel).where(~DagModel.is_subdag, DagModel.is_active) - else: - dags_query = select(DagModel).where(~DagModel.is_subdag) + dags_query = dags_query.where(DagModel.is_active) if dag_id_pattern == "~": dag_id_pattern = "%" diff --git a/airflow/api_connexion/endpoints/dag_run_endpoint.py b/airflow/api_connexion/endpoints/dag_run_endpoint.py index 96fdd42fa03b7..acf4b44493924 100644 --- a/airflow/api_connexion/endpoints/dag_run_endpoint.py +++ b/airflow/api_connexion/endpoints/dag_run_endpoint.py @@ -425,8 +425,6 @@ def clear_dag_run(*, dag_id: str, dag_run_id: str, session: Session = NEW_SESSIO start_date=start_date, end_date=end_date, task_ids=None, - include_subdags=True, - include_parentdag=True, only_failed=False, dry_run=True, ) @@ -438,8 +436,6 @@ def clear_dag_run(*, dag_id: str, dag_run_id: str, session: Session = NEW_SESSIO start_date=start_date, end_date=end_date, task_ids=None, - include_subdags=True, - include_parentdag=True, only_failed=False, ) dag_run = session.execute(select(DagRun).where(DagRun.id == dag_run.id)).scalar_one() diff --git a/airflow/api_connexion/openapi/v1.yaml b/airflow/api_connexion/openapi/v1.yaml index 0394da4f466cf..fbd9a64eacd89 100644 --- a/airflow/api_connexion/openapi/v1.yaml +++ b/airflow/api_connexion/openapi/v1.yaml @@ -3106,11 +3106,6 @@ components: Human centric display text for the DAG. *New in version 2.9.0* - root_dag_id: - type: string - readOnly: true - nullable: true - description: If the DAG is SubDAG then it is the top level DAG identifier. Otherwise, null. is_paused: type: boolean nullable: true @@ -3125,10 +3120,6 @@ components: nullable: true readOnly: true type: boolean - is_subdag: - description: Whether the DAG is SubDAG. - type: boolean - readOnly: true last_parsed_time: type: string format: date-time @@ -4903,14 +4894,6 @@ components: type: boolean default: false - include_subdags: - description: Clear tasks in subdags and clear external tasks indicated by ExternalTaskMarker. - type: boolean - - include_parentdag: - description: Clear tasks in the parent dag of the subdag. - type: boolean - reset_dag_runs: description: Set state of DAG runs to RUNNING. type: boolean diff --git a/airflow/api_connexion/schemas/dag_schema.py b/airflow/api_connexion/schemas/dag_schema.py index 799e4092ccaee..32eca2f0b8903 100644 --- a/airflow/api_connexion/schemas/dag_schema.py +++ b/airflow/api_connexion/schemas/dag_schema.py @@ -51,10 +51,8 @@ class Meta: dag_id = auto_field(dump_only=True) dag_display_name = fields.String(attribute="dag_display_name", dump_only=True) - root_dag_id = auto_field(dump_only=True) is_paused = auto_field() is_active = auto_field(dump_only=True) - is_subdag = auto_field(dump_only=True) last_parsed_time = auto_field(dump_only=True) last_pickled = auto_field(dump_only=True) last_expired = auto_field(dump_only=True) diff --git a/airflow/api_connexion/schemas/task_instance_schema.py b/airflow/api_connexion/schemas/task_instance_schema.py index 5d0eb72091272..74cd0585dcda8 100644 --- a/airflow/api_connexion/schemas/task_instance_schema.py +++ b/airflow/api_connexion/schemas/task_instance_schema.py @@ -177,8 +177,6 @@ class ClearTaskInstanceFormSchema(Schema): end_date = fields.DateTime(load_default=None, validate=validate_istimezone) only_failed = fields.Boolean(load_default=True) only_running = fields.Boolean(load_default=False) - include_subdags = fields.Boolean(load_default=False) - include_parentdag = fields.Boolean(load_default=False) reset_dag_runs = fields.Boolean(load_default=False) task_ids = fields.List(fields.String(), validate=validate.Length(min=1)) dag_run_id = fields.Str(load_default=None) diff --git a/airflow/cli/cli_config.py b/airflow/cli/cli_config.py index 7814c86bd6d18..269916548401d 100644 --- a/airflow/cli/cli_config.py +++ b/airflow/cli/cli_config.py @@ -449,12 +449,6 @@ def string_lower_type(val): ARG_ONLY_FAILED = Arg(("-f", "--only-failed"), help="Only failed jobs", action="store_true") ARG_ONLY_RUNNING = Arg(("-r", "--only-running"), help="Only running jobs", action="store_true") ARG_DOWNSTREAM = Arg(("-d", "--downstream"), help="Include downstream tasks", action="store_true") -ARG_EXCLUDE_SUBDAGS = Arg(("-x", "--exclude-subdags"), help="Exclude subdags", action="store_true") -ARG_EXCLUDE_PARENTDAG = Arg( - ("-X", "--exclude-parentdag"), - help="Exclude ParentDAGS if the task cleared is a part of a SubDAG", - action="store_true", -) ARG_DAG_REGEX = Arg( ("-R", "--dag-regex"), help="Search dag_id as regex instead of exact string", action="store_true" ) @@ -1330,8 +1324,6 @@ class GroupCommand(NamedTuple): ARG_YES, ARG_ONLY_FAILED, ARG_ONLY_RUNNING, - ARG_EXCLUDE_SUBDAGS, - ARG_EXCLUDE_PARENTDAG, ARG_DAG_REGEX, ARG_VERBOSE, ), diff --git a/airflow/cli/commands/dag_command.py b/airflow/cli/commands/dag_command.py index dac61d0da5971..2f300ebef2144 100644 --- a/airflow/cli/commands/dag_command.py +++ b/airflow/cli/commands/dag_command.py @@ -101,7 +101,6 @@ def _run_dag_backfill(dags: list[DAG], args) -> None: start_date=args.start_date, end_date=args.end_date, confirm_prompt=not args.yes, - include_subdags=True, dag_run_state=DagRunState.QUEUED, ) @@ -334,10 +333,8 @@ def _get_dagbag_dag_details(dag: DAG) -> dict: return { "dag_id": dag.dag_id, "dag_display_name": dag.dag_display_name, - "root_dag_id": dag.parent_dag.dag_id if dag.parent_dag else None, "is_paused": dag.get_is_paused(), "is_active": dag.get_is_active(), - "is_subdag": dag.is_subdag, "last_parsed_time": None, "last_pickled": None, "last_expired": None, diff --git a/airflow/cli/commands/task_command.py b/airflow/cli/commands/task_command.py index 6e0fc80fbb300..c4c6db386a4e9 100644 --- a/airflow/cli/commands/task_command.py +++ b/airflow/cli/commands/task_command.py @@ -764,8 +764,6 @@ def task_clear(args) -> None: only_failed=args.only_failed, only_running=args.only_running, confirm_prompt=not args.yes, - include_subdags=not args.exclude_subdags, - include_parentdag=not args.exclude_parentdag, ) diff --git a/airflow/dag_processing/processor.py b/airflow/dag_processing/processor.py index 86db0b5b881a3..0b19d8f2db76c 100644 --- a/airflow/dag_processing/processor.py +++ b/airflow/dag_processing/processor.py @@ -688,8 +688,6 @@ def get_pools(dag) -> dict[str, set[str]]: pool_dict: dict[str, set[str]] = {} for dag in dagbag.dags.values(): pool_dict.update(get_pools(dag)) - for subdag in dag.subdags: - pool_dict.update(get_pools(subdag)) dag_ids = {dag.dag_id for dag in dagbag.dags.values()} return DagFileProcessor._validate_task_pools_and_update_dag_warnings(pool_dict, dag_ids) diff --git a/airflow/example_dags/example_subdag_operator.py b/airflow/example_dags/example_subdag_operator.py deleted file mode 100644 index 196012024c2c6..0000000000000 --- a/airflow/example_dags/example_subdag_operator.py +++ /dev/null @@ -1,70 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -"""Example DAG demonstrating the usage of the SubDagOperator.""" - -from __future__ import annotations - -import warnings - -with warnings.catch_warnings(): - warnings.filterwarnings( - "ignore", - message=r"This class is deprecated\. Please use `airflow\.utils\.task_group\.TaskGroup`\.", - ) - - # [START example_subdag_operator] - import datetime - - from airflow.example_dags.subdags.subdag import subdag - from airflow.models.dag import DAG - from airflow.operators.empty import EmptyOperator - from airflow.operators.subdag import SubDagOperator - - DAG_NAME = "example_subdag_operator" - - with DAG( - dag_id=DAG_NAME, - default_args={"retries": 2}, - start_date=datetime.datetime(2022, 1, 1), - schedule="@once", - tags=["example"], - ) as dag: - start = EmptyOperator( - task_id="start", - ) - - section_1 = SubDagOperator( - task_id="section-1", - subdag=subdag(DAG_NAME, "section-1", dag.default_args), - ) - - some_other_task = EmptyOperator( - task_id="some-other-task", - ) - - section_2 = SubDagOperator( - task_id="section-2", - subdag=subdag(DAG_NAME, "section-2", dag.default_args), - ) - - end = EmptyOperator( - task_id="end", - ) - - start >> section_1 >> some_other_task >> section_2 >> end - # [END example_subdag_operator] diff --git a/airflow/example_dags/subdags/__init__.py b/airflow/example_dags/subdags/__init__.py deleted file mode 100644 index 217e5db960782..0000000000000 --- a/airflow/example_dags/subdags/__init__.py +++ /dev/null @@ -1,17 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. diff --git a/airflow/example_dags/subdags/subdag.py b/airflow/example_dags/subdags/subdag.py deleted file mode 100644 index 748582f4b84ae..0000000000000 --- a/airflow/example_dags/subdags/subdag.py +++ /dev/null @@ -1,56 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -"""Helper function to generate a DAG and operators given some arguments.""" - -from __future__ import annotations - -# [START subdag] -import pendulum - -from airflow.models.dag import DAG -from airflow.operators.empty import EmptyOperator - - -def subdag(parent_dag_name, child_dag_name, args) -> DAG: - """ - Generate a DAG to be used as a subdag. - - :param str parent_dag_name: Id of the parent DAG - :param str child_dag_name: Id of the child DAG - :param dict args: Default arguments to provide to the subdag - :return: DAG to use as a subdag - """ - dag_subdag = DAG( - dag_id=f"{parent_dag_name}.{child_dag_name}", - default_args=args, - start_date=pendulum.datetime(2021, 1, 1, tz="UTC"), - catchup=False, - schedule="@daily", - ) - - for i in range(5): - EmptyOperator( - task_id=f"{child_dag_name}-task-{i + 1}", - default_args=args, - dag=dag_subdag, - ) - - return dag_subdag - - -# [END subdag] diff --git a/airflow/jobs/backfill_job_runner.py b/airflow/jobs/backfill_job_runner.py index 961c4b7e020b3..028d494219677 100644 --- a/airflow/jobs/backfill_job_runner.py +++ b/airflow/jobs/backfill_job_runner.py @@ -65,7 +65,7 @@ class BackfillJobRunner(BaseJobRunner, LoggingMixin): """ - A backfill job runner consists of a dag or subdag for a specific time range. + A backfill job runner consists of a dag for a specific time range. It triggers a set of task instance runs, in the right order and lasts for as long as it takes for the set of task instance to be completed. @@ -327,7 +327,7 @@ def _manage_executor_state( def _iter_task_needing_expansion() -> Iterator[AbstractOperator]: from airflow.models.mappedoperator import AbstractOperator - for node in self.dag.get_task(ti.task_id, include_subdags=True).iter_mapped_dependants(): + for node in self.dag.get_task(ti.task_id).iter_mapped_dependants(): if isinstance(node, AbstractOperator): yield node else: # A (mapped) task group. All its children need expansion. @@ -359,8 +359,7 @@ def _get_dag_run( """ run_date = dagrun_info.logical_date - # consider max_active_runs but ignore when running subdags - respect_dag_max_active_limit = bool(dag.timetable.can_be_scheduled and not dag.is_subdag) + respect_dag_max_active_limit = bool(dag.timetable.can_be_scheduled) current_active_dag_count = dag.get_num_active_runs(external_trigger=False) @@ -500,7 +499,7 @@ def _process_backfill_task_instances( def _per_task_process(key, ti: TaskInstance, session): ti.refresh_from_db(lock_for_update=True, session=session) - task = self.dag.get_task(ti.task_id, include_subdags=True) + task = self.dag.get_task(ti.task_id) ti.task = task self.log.debug("Task instance to run %s state %s", ti, ti.state) @@ -636,7 +635,7 @@ def _per_task_process(key, ti: TaskInstance, session): ti_status.not_ready.add(key) try: - for task in self.dag.topological_sort(include_subdag_tasks=True): + for task in self.dag.topological_sort(): for key, ti in list(ti_status.to_run.items()): # Attempt to workaround deadlock on backfill by attempting to commit the transaction # state update few times before giving up @@ -839,9 +838,6 @@ def tabulate_ti_keys_set(ti_keys: Iterable[TaskInstanceKey]) -> str: yield "\n\nThese tasks are deadlocked:\n" yield tabulate_ti_keys_set([ti.key for ti in ti_status.deadlocked]) - def _get_dag_with_subdags(self) -> list[DAG]: - return [self.dag, *self.dag.subdags] - @provide_session def _execute_dagruns( self, @@ -863,12 +859,11 @@ def _execute_dagruns( :param session: the current session object """ for dagrun_info in dagrun_infos: - for dag in self._get_dag_with_subdags(): - dag_run = self._get_dag_run(dagrun_info, dag, session=session) - if dag_run is not None: - tis_map = self._task_instances_for_dag_run(dag, dag_run, session=session) - ti_status.active_runs.add(dag_run) - ti_status.to_run.update(tis_map or {}) + dag_run = self._get_dag_run(dagrun_info, self.dag, session=session) + if dag_run is not None: + tis_map = self._task_instances_for_dag_run(self.dag, dag_run, session=session) + ti_status.active_runs.add(dag_run) + ti_status.to_run.update(tis_map or {}) tis_missing_executor = [] for ti in ti_status.to_run.values(): @@ -948,9 +943,8 @@ def _execute(self, session: Session = NEW_SESSION) -> None: return dagrun_infos = [DagRunInfo.interval(dagrun_start_date, dagrun_end_date)] - dag_with_subdags_ids = [d.dag_id for d in self._get_dag_with_subdags()] running_dagruns = DagRun.find( - dag_id=dag_with_subdags_ids, + dag_id=self.dag.dag_id, execution_start_date=self.bf_start_date, execution_end_date=self.bf_end_date, no_backfills=True, diff --git a/airflow/migrations/versions/0003_3_0_0_remove_is_subdag.py b/airflow/migrations/versions/0003_3_0_0_remove_is_subdag.py new file mode 100644 index 0000000000000..eab9954b329ab --- /dev/null +++ b/airflow/migrations/versions/0003_3_0_0_remove_is_subdag.py @@ -0,0 +1,76 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +""" +Remove SubDAGs: ``is_subdag`` & ``root_dag_id`` columns from DAG table. + +Revision ID: d0f1c55954fa +Revises: 044f740568ec +Create Date: 2024-08-11 21:32:40.576172 + +""" + +from __future__ import annotations + +import sqlalchemy as sa +from alembic import op + +from airflow.migrations.db_types import StringID + +# revision identifiers, used by Alembic. +revision = "d0f1c55954fa" +down_revision = "044f740568ec" +branch_labels = None +depends_on = None +airflow_version = "3.0.0" + + +def _column_exists(inspector, column_name): + return column_name in [col["name"] for col in inspector.get_columns("dag")] + + +def _index_exists(inspector, index_name): + return index_name in [index["name"] for index in inspector.get_indexes("dag")] + + +def upgrade(): + """Remove ``is_subdag`` column from DAGs table.""" + conn = op.get_bind() + inspector = sa.inspect(conn) + + with op.batch_alter_table("dag", schema=None) as batch_op: + if _index_exists(inspector, "idx_root_dag_id"): + batch_op.drop_index("idx_root_dag_id") + if _column_exists(inspector, "is_subdag"): + batch_op.drop_column("is_subdag") + if _column_exists(inspector, "root_dag_id"): + batch_op.drop_column("root_dag_id") + + +def downgrade(): + """Add ``is_subdag`` column in DAGs table.""" + conn = op.get_bind() + inspector = sa.inspect(conn) + + with op.batch_alter_table("dag", schema=None) as batch_op: + if not _column_exists(inspector, "is_subdag"): + batch_op.add_column(sa.Column("is_subdag", sa.BOOLEAN(), nullable=True)) + if not _column_exists(inspector, "root_dag_id"): + batch_op.add_column(sa.Column("root_dag_id", StringID(), nullable=True)) + if not _index_exists(inspector, "idx_root_dag_id"): + batch_op.create_index("idx_root_dag_id", ["root_dag_id"], unique=False) diff --git a/airflow/models/baseoperator.py b/airflow/models/baseoperator.py index 7ffa596ec67a1..ea100cd4e2abf 100644 --- a/airflow/models/baseoperator.py +++ b/airflow/models/baseoperator.py @@ -854,10 +854,6 @@ def say_hello_world(**context): _dag: DAG | None = None task_group: TaskGroup | None = None - # subdag parameter is only set for SubDagOperator. - # Setting it to None by default as other Operators do not have that field - subdag: DAG | None = None - start_date: pendulum.DateTime | None = None end_date: pendulum.DateTime | None = None @@ -1724,7 +1720,6 @@ def get_serialized_fields(cls): "end_date", "_task_type", "_operator_name", - "subdag", "ui_color", "ui_fgcolor", "template_ext", diff --git a/airflow/models/dag.py b/airflow/models/dag.py index 1c9d351c1d292..2f9fc479ae757 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -497,7 +497,6 @@ class DAG(LoggingMixin): _comps = { "dag_id", "task_ids", - "parent_dag", "start_date", "end_date", "schedule_interval", @@ -510,14 +509,12 @@ class DAG(LoggingMixin): fileloc: str """ - File path that needs to be imported to load this DAG or subdag. + File path that needs to be imported to load this DAG. This may not be an actual file on disk in the case when this DAG is loaded from a ZIP file or other DAG distribution format. """ - parent_dag: DAG | None = None # Gets set when DAGs are loaded - # NOTE: When updating arguments here, please also keep arguments in @dag() # below in sync. (Search for 'def dag(' in this file.) def __init__( @@ -1135,10 +1132,6 @@ def next_dagrun_info( :return: DagRunInfo of the next dagrun, or None if a dagrun is not going to be scheduled. """ - # Never schedule a subdag. It will be scheduled by its parent dag. - if self.is_subdag: - return None - data_interval = None if isinstance(last_automated_dagrun, datetime): warnings.warn( @@ -1217,8 +1210,7 @@ def iter_dagrun_infos_between( If ``align`` is ``False``, the first run will happen immediately on ``earliest``, even if it does not fall on the logical timetable schedule. - The default is ``True``, but subdags will ignore this value and always - behave as if this is set to ``False`` for backward compatibility. + The default is ``True``. Example: A DAG is scheduled to run every midnight (``0 0 * * *``). If ``earliest`` is ``2021-06-03 23:00:00``, the first DagRunInfo would be @@ -1234,15 +1226,6 @@ def iter_dagrun_infos_between( restriction = TimeRestriction(earliest, latest, catchup=True) - # HACK: Sub-DAGs are currently scheduled differently. For example, say - # the schedule is @daily and start is 2021-06-03 22:16:00, a top-level - # DAG should be first scheduled to run on midnight 2021-06-04, but a - # sub-DAG should be first scheduled to run RIGHT NOW. We can change - # this, but since sub-DAGs are going away in 3.0 anyway, let's keep - # compatibility for now and remove this entirely later. - if self.is_subdag: - align = False - try: info = self.timetable.next_dagrun_info( last_automated_data_interval=None, @@ -1347,10 +1330,6 @@ def dag_id(self) -> str: def dag_id(self, value: str) -> None: self._dag_id = value - @property - def is_subdag(self) -> bool: - return self.parent_dag is not None - @property def full_filepath(self) -> str: """ @@ -1755,25 +1734,6 @@ def latest_execution_date(self): ) return self.get_latest_execution_date() - @property - def subdags(self): - """Return a list of the subdag objects associated to this DAG.""" - # Check SubDag for class but don't check class directly - from airflow.operators.subdag import SubDagOperator - - subdag_lst = [] - for task in self.tasks: - if ( - isinstance(task, SubDagOperator) - or - # TODO remove in Airflow 2.0 - type(task).__name__ == "SubDagOperator" - or task.task_type == "SubDagOperator" - ): - subdag_lst.append(task.subdag) - subdag_lst += task.subdag.subdags - return subdag_lst - def resolve_template_files(self): for t in self.tasks: t.resolve_template_files() @@ -1866,8 +1826,6 @@ def get_task_instances( end_date=end_date, run_id=None, state=state or (), - include_subdags=False, - include_parentdag=False, include_dependent_dags=False, exclude_task_ids=(), session=session, @@ -1883,8 +1841,6 @@ def _get_task_instances( end_date: datetime | None, run_id: str | None, state: TaskInstanceState | Sequence[TaskInstanceState], - include_subdags: bool, - include_parentdag: bool, include_dependent_dags: bool, exclude_task_ids: Collection[str | tuple[str, int]] | None, session: Session, @@ -1901,8 +1857,6 @@ def _get_task_instances( end_date: datetime | None, run_id: str | None, state: TaskInstanceState | Sequence[TaskInstanceState], - include_subdags: bool, - include_parentdag: bool, include_dependent_dags: bool, exclude_task_ids: Collection[str | tuple[str, int]] | None, session: Session, @@ -1921,8 +1875,6 @@ def _get_task_instances( end_date: datetime | None, run_id: str | None, state: TaskInstanceState | Sequence[TaskInstanceState], - include_subdags: bool, - include_parentdag: bool, include_dependent_dags: bool, exclude_task_ids: Collection[str | tuple[str, int]] | None, session: Session, @@ -1933,7 +1885,7 @@ def _get_task_instances( ) -> Iterable[TaskInstance] | set[TaskInstanceKey]: TI = TaskInstance - # If we are looking at subdags/dependent dags we want to avoid UNION calls + # If we are looking at dependent dags we want to avoid UNION calls # in SQL (it doesn't play nice with fields that have no equality operator, # like JSON types), we instead build our result set separately. # @@ -1948,15 +1900,7 @@ def _get_task_instances( tis = select(TaskInstance) tis = tis.join(TaskInstance.dag_run) - if include_subdags: - # Crafting the right filter for dag_id and task_ids combo - conditions = [] - for dag in [*self.subdags, self]: - conditions.append( - (TaskInstance.dag_id == dag.dag_id) & TaskInstance.task_id.in_(dag.task_ids) - ) - tis = tis.where(or_(*conditions)) - elif self.partial: + if self.partial: tis = tis.where(TaskInstance.dag_id == self.dag_id, TaskInstance.task_id.in_(self.task_ids)) else: tis = tis.where(TaskInstance.dag_id == self.dag_id) @@ -1990,36 +1934,6 @@ def _get_task_instances( else: tis = tis.where(TaskInstance.state.in_(state)) - # Next, get any of them from our parent DAG (if there is one) - if include_parentdag and self.parent_dag is not None: - if visited_external_tis is None: - visited_external_tis = set() - - p_dag = self.parent_dag.partial_subset( - task_ids_or_regex=r"^{}$".format(self.dag_id.split(".")[1]), - include_upstream=False, - include_downstream=True, - ) - result.update( - p_dag._get_task_instances( - task_ids=task_ids, - start_date=start_date, - end_date=end_date, - run_id=None, - state=state, - include_subdags=include_subdags, - include_parentdag=False, - include_dependent_dags=include_dependent_dags, - as_pk_tuple=True, - exclude_task_ids=exclude_task_ids, - session=session, - dag_bag=dag_bag, - recursion_depth=recursion_depth, - max_recursion_depth=max_recursion_depth, - visited_external_tis=visited_external_tis, - ) - ) - if include_dependent_dags: # Recursively find external tasks indicated by ExternalTaskMarker from airflow.sensors.external_task import ExternalTaskMarker @@ -2089,9 +2003,7 @@ def _get_task_instances( start_date=None, end_date=None, state=state, - include_subdags=include_subdags, include_dependent_dags=include_dependent_dags, - include_parentdag=False, as_pk_tuple=True, exclude_task_ids=exclude_task_ids, dag_bag=dag_bag, @@ -2103,7 +2015,7 @@ def _get_task_instances( ) if result or as_pk_tuple: - # Only execute the `ti` query if we have also collected some other results (i.e. subdags etc.) + # Only execute the `ti` query if we have also collected some other results if as_pk_tuple: tis_query = session.execute(tis).all() result.update(TaskInstanceKey(**cols._mapping) for cols in tis_query) @@ -2218,8 +2130,6 @@ def set_task_instance_state( subdag.clear( start_date=start_date, end_date=end_date, - include_subdags=True, - include_parentdag=True, only_failed=True, session=session, # Exclude the task itself from being cleared @@ -2319,8 +2229,6 @@ def set_task_group_state( task_subset.clear( start_date=start_date, end_date=end_date, - include_subdags=True, - include_parentdag=True, only_failed=True, session=session, # Exclude the task from the current group from being cleared @@ -2339,7 +2247,7 @@ def leaves(self) -> list[Operator]: """Return nodes with no children. These are last to execute and are called leaves or leaf nodes.""" return [task for task in self.tasks if not task.downstream_list] - def topological_sort(self, include_subdag_tasks: bool = False): + def topological_sort(self): """ Sorts tasks in topographical order, such that a task comes after any of its upstream dependencies. @@ -2348,7 +2256,7 @@ def topological_sort(self, include_subdag_tasks: bool = False): from airflow.utils.task_group import TaskGroup def nested_topo(group): - for node in group.topological_sort(_include_subdag_tasks=include_subdag_tasks): + for node in group.topological_sort(): if isinstance(node, TaskGroup): yield from nested_topo(node) else: @@ -2387,8 +2295,6 @@ def clear( only_failed: bool = False, only_running: bool = False, confirm_prompt: bool = False, - include_subdags: bool = True, - include_parentdag: bool = True, dag_run_state: DagRunState = DagRunState.QUEUED, dry_run: bool = False, session: Session = NEW_SESSION, @@ -2407,14 +2313,11 @@ def clear( :param only_failed: Only clear failed tasks :param only_running: Only clear running tasks. :param confirm_prompt: Ask for confirmation - :param include_subdags: Clear tasks in subdags and clear external tasks - indicated by ExternalTaskMarker - :param include_parentdag: Clear tasks in the parent dag of the subdag. :param dag_run_state: state to set DagRun to. If set to False, dagrun state will not be changed. :param dry_run: Find the tasks to clear but don't clear them. :param session: The sqlalchemy session to use - :param dag_bag: The DagBag used to find the dags subdags (Optional) + :param dag_bag: The DagBag used to find the dags (Optional) :param exclude_task_ids: A set of ``task_id`` or (``task_id``, ``map_index``) tuples that should not be cleared """ @@ -2452,9 +2355,7 @@ def clear( end_date=end_date, run_id=None, state=state, - include_subdags=include_subdags, - include_parentdag=include_parentdag, - include_dependent_dags=include_subdags, # compat, yes this is not a typo + include_dependent_dags=False, session=session, dag_bag=dag_bag, exclude_task_ids=exclude_task_ids, @@ -2497,8 +2398,6 @@ def clear_dags( only_failed=False, only_running=False, confirm_prompt=False, - include_subdags=True, - include_parentdag=False, dag_run_state=DagRunState.QUEUED, dry_run=False, ): @@ -2510,8 +2409,6 @@ def clear_dags( only_failed=only_failed, only_running=only_running, confirm_prompt=False, - include_subdags=include_subdags, - include_parentdag=include_parentdag, dag_run_state=dag_run_state, dry_run=True, ) @@ -2538,7 +2435,6 @@ def clear_dags( only_failed=only_failed, only_running=only_running, confirm_prompt=False, - include_subdags=include_subdags, dag_run_state=dag_run_state, dry_run=False, ) @@ -2563,15 +2459,6 @@ def __deepcopy__(self, memo): result._log = self._log return result - def sub_dag(self, *args, **kwargs): - """Use `airflow.models.DAG.partial_subset`, this method is deprecated.""" - warnings.warn( - "This method is deprecated and will be removed in a future version. Please use partial_subset", - RemovedInAirflow3Warning, - stacklevel=2, - ) - return self.partial_subset(*args, **kwargs) - def partial_subset( self, task_ids_or_regex: str | Pattern | Iterable[str], @@ -2714,13 +2601,9 @@ def has_task_group(self, task_group_id: str) -> bool: def task_group_dict(self): return {k: v for k, v in self._task_group.get_task_group_dict().items() if k is not None} - def get_task(self, task_id: str, include_subdags: bool = False) -> Operator: + def get_task(self, task_id: str) -> Operator: if task_id in self.task_dict: return self.task_dict[task_id] - if include_subdags: - for dag in self.subdags: - if task_id in dag.task_dict: - return dag.task_dict[task_id] raise TaskNotFound(f"Task {task_id} not found") def pickle_info(self): @@ -3198,8 +3081,6 @@ def bulk_write_to_db( """ Ensure the DagModel rows for the given dags are up-to-date in the dag table in the DB. - Note that this method can be called for both DAGs and SubDAGs. A SubDag is actually a SubDagOperator. - :param dags: the DAG objects to save to the DB :return: None """ @@ -3251,15 +3132,8 @@ def bulk_write_to_db( for orm_dag in sorted(orm_dags, key=lambda d: d.dag_id): dag = dag_by_ids[orm_dag.dag_id] filelocs.append(dag.fileloc) - if dag.is_subdag: - orm_dag.is_subdag = True - orm_dag.fileloc = dag.parent_dag.fileloc # type: ignore - orm_dag.root_dag_id = dag.parent_dag.dag_id # type: ignore - orm_dag.owners = dag.parent_dag.owner # type: ignore - else: - orm_dag.is_subdag = False - orm_dag.fileloc = dag.fileloc - orm_dag.owners = dag.owner + orm_dag.fileloc = dag.fileloc + orm_dag.owners = dag.owner orm_dag.is_active = True orm_dag.has_import_errors = False orm_dag.last_parsed_time = timezone.utcnow() @@ -3474,9 +3348,6 @@ def bulk_write_to_db( # decide when to commit session.flush() - for dag in dags: - cls.bulk_write_to_db(dag.subdags, processor_subdir=processor_subdir, session=session) - @classmethod def _get_latest_runs_stmt(cls, dags: list[str]) -> Select: """ @@ -3526,8 +3397,6 @@ def sync_to_db(self, processor_subdir: str | None = None, session=NEW_SESSION): """ Save attributes about this DAG to the DB. - Note that this method can be called for both DAGs and SubDAGs. A SubDag is actually a SubDagOperator. - :return: None """ self.bulk_write_to_db([self], processor_subdir=processor_subdir, session=session) @@ -3621,7 +3490,6 @@ def get_serialized_fields(cls): """Stringified DAGs and operators contain exactly these fields.""" if not cls.__serialized_fields: exclusion_list = { - "parent_dag", "schedule_dataset_references", "schedule_dataset_alias_references", "task_outlet_dataset_references", @@ -3753,13 +3621,10 @@ class DagModel(Base): These items are stored in the database for state related information """ dag_id = Column(StringID(), primary_key=True) - root_dag_id = Column(StringID()) # A DAG can be paused from the UI / DB # Set this default value of is_paused based on a configuration value! is_paused_at_creation = airflow_conf.getboolean("core", "dags_are_paused_at_creation") is_paused = Column(Boolean, default=is_paused_at_creation) - # Whether the DAG is a subdag - is_subdag = Column(Boolean, default=False) # Whether that DAG was seen on the last DagBag load is_active = Column(Boolean, default=False) # Last time the scheduler started @@ -3818,14 +3683,8 @@ class DagModel(Base): # Earliest time at which this ``next_dagrun`` can be created. next_dagrun_create_after = Column(UtcDateTime) - __table_args__ = ( - Index("idx_root_dag_id", root_dag_id, unique=False), - Index("idx_next_dagrun_create_after", next_dagrun_create_after, unique=False), - ) + __table_args__ = (Index("idx_next_dagrun_create_after", next_dagrun_create_after, unique=False),) - parent_dag = relationship( - "DagModel", remote_side=[dag_id], primaryjoin=root_dag_id == dag_id, foreign_keys=[root_dag_id] - ) schedule_dataset_references = relationship( "DagScheduleDatasetReference", back_populates="dag", @@ -3898,7 +3757,6 @@ def get_dagmodel(dag_id: str, session: Session = NEW_SESSION) -> DagModel | None return session.get( DagModel, dag_id, - options=[joinedload(DagModel.parent_dag)], ) @classmethod @@ -3963,19 +3821,17 @@ def relative_fileloc(self) -> pathlib.Path | None: return path @provide_session - def set_is_paused(self, is_paused: bool, including_subdags: bool = True, session=NEW_SESSION) -> None: + def set_is_paused(self, is_paused: bool, session=NEW_SESSION) -> None: """ Pause/Un-pause a DAG. :param is_paused: Is the DAG paused - :param including_subdags: whether to include the DAG's subdags :param session: session """ filter_query = [ DagModel.dag_id == self.dag_id, ] - if including_subdags: - filter_query.append(DagModel.root_dag_id == self.dag_id) + session.execute( update(DagModel) .where(or_(*filter_query)) diff --git a/airflow/models/dagbag.py b/airflow/models/dagbag.py index f384bfcd84ea8..3fa3af180c92d 100644 --- a/airflow/models/dagbag.py +++ b/airflow/models/dagbag.py @@ -259,8 +259,6 @@ def get_dag(self, dag_id, session: Session = None): root_dag_id = dag_id if dag_id in self.dags: dag = self.dags[dag_id] - if dag.parent_dag: - root_dag_id = dag.parent_dag.dag_id # If DAG Model is absent, we can't check last_expired property. Is the DAG not yet synchronized? orm_dag = DagModel.get_current(root_dag_id, session=session) @@ -272,11 +270,7 @@ def get_dag(self, dag_id, session: Session = None): is_expired = orm_dag.last_expired and dag and dag.last_loaded < orm_dag.last_expired if is_expired: # Remove associated dags so we can re-add them. - self.dags = { - key: dag - for key, dag in self.dags.items() - if root_dag_id != key and not (dag.parent_dag and root_dag_id == dag.parent_dag.dag_id) - } + self.dags = {key: dag for key, dag in self.dags.items()} if is_missing or is_expired: # Reprocess source file. found_dags = self.process_file( @@ -300,8 +294,6 @@ def _add_dag_from_db(self, dag_id: str, session: Session): row.load_op_links = self.load_op_links dag = row.dag - for subdag in dag.subdags: - self.dags[subdag.dag_id] = subdag self.dags[dag.dag_id] = dag self.dags_last_fetched[dag.dag_id] = timezone.utcnow() self.dags_hash[dag.dag_id] = row.dag_hash @@ -476,7 +468,7 @@ def _process_modules(self, filepath, mods, file_last_changed_on_disk): dag.fileloc = mod.__file__ try: dag.validate() - self.bag_dag(dag=dag, root_dag=dag) + self.bag_dag(dag=dag) except AirflowClusterPolicySkipDag: pass except Exception as e: @@ -485,25 +477,15 @@ def _process_modules(self, filepath, mods, file_last_changed_on_disk): self.file_last_changed[dag.fileloc] = file_last_changed_on_disk else: found_dags.append(dag) - found_dags += dag.subdags return found_dags - def bag_dag(self, dag, root_dag): + def bag_dag(self, dag): """ - Add the DAG into the bag, recurses into sub dags. + Add the DAG into the bag. :raises: AirflowDagCycleException if a cycle is detected in this dag or its subdags. :raises: AirflowDagDuplicatedIdException if this dag or its subdags already exists in the bag. """ - self._bag_dag(dag=dag, root_dag=root_dag, recursive=True) - - def _bag_dag(self, *, dag, root_dag, recursive): - """ - Actual implementation of bagging a dag. - - The only purpose of this is to avoid exposing ``recursive`` in ``bag_dag()``, - intended to only be used by the ``_bag_dag()`` implementation. - """ check_cycle(dag) # throws if a task cycle is found dag.resolve_template_files() @@ -531,17 +513,7 @@ def _bag_dag(self, *, dag, root_dag, recursive): self.log.exception(e) raise AirflowClusterPolicyError(e) - subdags = dag.subdags - try: - # DAG.subdags automatically performs DFS search, so we don't recurse - # into further _bag_dag() calls. - if recursive: - for subdag in subdags: - subdag.fileloc = dag.fileloc - subdag.parent_dag = dag - self._bag_dag(dag=subdag, root_dag=root_dag, recursive=False) - prev_dag = self.dags.get(dag.dag_id) if prev_dag and prev_dag.fileloc != dag.fileloc: raise AirflowDagDuplicatedIdException( @@ -554,12 +526,6 @@ def _bag_dag(self, *, dag, root_dag, recursive): except (AirflowDagCycleException, AirflowDagDuplicatedIdException): # There was an error in bagging the dag. Remove it from the list of dags self.log.exception("Exception bagging dag: %s", dag.dag_id) - # Only necessary at the root level since DAG.subdags automatically - # performs DFS to search through all subdags - if recursive: - for subdag in subdags: - if subdag.dag_id in self.dags: - del self.dags[subdag.dag_id] raise def collect_dags( @@ -627,15 +593,6 @@ def collect_dags_from_db(self): # from the table by the scheduler job. self.dags = SerializedDagModel.read_all_dags() - # Adds subdags. - # DAG post-processing steps such as self.bag_dag and croniter are not needed as - # they are done by scheduler before serialization. - subdags = {} - for dag in self.dags.values(): - for subdag in dag.subdags: - subdags[subdag.dag_id] = subdag - self.dags.update(subdags) - def dagbag_report(self): """Print a report around DagBag loading stats.""" stats = self.dagbag_stats @@ -678,8 +635,6 @@ def _serialize_dag_capturing_errors(dag, session, processor_subdir): We can't place them directly in import_errors, as this may be retried, and work the next time """ - if dag.is_subdag: - return [] try: # We can't use bulk_write_to_db as we want to capture each error individually dag_was_updated = SerializedDagModel.write_dag( @@ -739,13 +694,13 @@ def sync_to_db(self, processor_subdir: str | None = None, session: Session = NEW @provide_session def _sync_perm_for_dag(cls, dag: DAG, session: Session = NEW_SESSION): """Sync DAG specific permissions.""" - root_dag_id = dag.parent_dag.dag_id if dag.parent_dag else dag.dag_id + dag_id = dag.dag_id - cls.logger().debug("Syncing DAG permissions: %s to the DB", root_dag_id) + cls.logger().debug("Syncing DAG permissions: %s to the DB", dag_id) from airflow.www.security_appless import ApplessAirflowSecurityManager security_manager = ApplessAirflowSecurityManager(session=session) - security_manager.sync_perm_for_dag(root_dag_id, dag.access_control) + security_manager.sync_perm_for_dag(dag_id, dag.access_control) def generate_md5_hash(context): diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 523f94cded38f..1ff2f6316a5a7 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -590,7 +590,6 @@ def _check_last_n_dagruns_failed(self, dag_id, max_consecutive_failed_dag_runs, ) filter_query = [ DagModel.dag_id == self.dag_id, - DagModel.root_dag_id == self.dag_id, # for sub-dags ] session.execute( update(DagModel) diff --git a/airflow/models/mappedoperator.py b/airflow/models/mappedoperator.py index 2377fdab00756..2cb7d993fc9f9 100644 --- a/airflow/models/mappedoperator.py +++ b/airflow/models/mappedoperator.py @@ -157,10 +157,6 @@ class OperatorPartial: _expand_called: bool = False # Set when expand() is called to ease user debugging. def __attrs_post_init__(self): - from airflow.operators.subdag import SubDagOperator - - if issubclass(self.operator_class, SubDagOperator): - raise TypeError("Mapping over deprecated SubDagOperator is not supported") validate_mapping_kwargs(self.operator_class, "partial", self.kwargs) def __repr__(self) -> str: @@ -306,7 +302,6 @@ class MappedOperator(AbstractOperator): This should be a name to call ``getattr()`` on. """ - subdag: None = None # Since we don't support SubDagOperator, this is always None. supports_lineage: bool = False HIDE_ATTRS_FROM_UI: ClassVar[frozenset[str]] = AbstractOperator.HIDE_ATTRS_FROM_UI | frozenset( @@ -347,7 +342,6 @@ def get_serialized_fields(cls): "dag", "deps", "expand_input", # This is needed to be able to accept XComArg. - "subdag", "task_group", "upstream_task_ids", "supports_lineage", diff --git a/airflow/models/serialized_dag.py b/airflow/models/serialized_dag.py index 99495f81b2ca1..dec843451a98a 100644 --- a/airflow/models/serialized_dag.py +++ b/airflow/models/serialized_dag.py @@ -305,8 +305,6 @@ def get(cls, dag_id: str, session: Session = NEW_SESSION) -> SerializedDagModel """ Get the SerializedDAG for the given dag ID. - It will cope with being passed the ID of a subdag by looking up the root dag_id from the DAG table. - :param dag_id: the DAG to fetch :param session: ORM Session """ @@ -314,11 +312,7 @@ def get(cls, dag_id: str, session: Session = NEW_SESSION) -> SerializedDagModel if row: return row - # If we didn't find a matching DAG id then ask the DAG table to find - # out the root dag - root_dag_id = session.scalar(select(DagModel.root_dag_id).where(DagModel.dag_id == dag_id)) - - return session.scalar(select(cls).where(cls.dag_id == root_dag_id)) + return session.scalar(select(cls).where(cls.dag_id == dag_id)) @staticmethod @provide_session @@ -337,13 +331,12 @@ def bulk_sync_to_db( :return: None """ for dag in dags: - if not dag.is_subdag: - SerializedDagModel.write_dag( - dag=dag, - min_update_interval=MIN_SERIALIZED_DAG_UPDATE_INTERVAL, - processor_subdir=processor_subdir, - session=session, - ) + SerializedDagModel.write_dag( + dag=dag, + min_update_interval=MIN_SERIALIZED_DAG_UPDATE_INTERVAL, + processor_subdir=processor_subdir, + session=session, + ) @classmethod @provide_session diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 284b313cdae4d..cedc25423900e 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -2098,12 +2098,7 @@ def _command_as_list( should_pass_filepath = not pickle_id and dag path: PurePath | None = None if should_pass_filepath: - if dag.is_subdag: - if TYPE_CHECKING: - assert dag.parent_dag is not None - path = dag.parent_dag.relative_fileloc - else: - path = dag.relative_fileloc + path = dag.relative_fileloc if path: if not path.is_absolute(): diff --git a/airflow/operators/__init__.py b/airflow/operators/__init__.py index 7fc63e1c8f507..f3195123b6a48 100644 --- a/airflow/operators/__init__.py +++ b/airflow/operators/__init__.py @@ -190,10 +190,6 @@ "sqlite_operator": { "SqliteOperator": "airflow.providers.sqlite.operators.sqlite.SqliteOperator", }, - "subdag_operator": { - "SkippedStatePropagationOptions": "airflow.operators.subdag.SkippedStatePropagationOptions", - "SubDagOperator": "airflow.operators.subdag.SubDagOperator", - }, } add_deprecated_classes(__deprecated_classes, __name__) diff --git a/airflow/operators/subdag.py b/airflow/operators/subdag.py deleted file mode 100644 index 7cbfa03198758..0000000000000 --- a/airflow/operators/subdag.py +++ /dev/null @@ -1,247 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -""" -This module is deprecated. Please use :mod:`airflow.utils.task_group`. - -The module which provides a way to nest your DAGs and so your levels of complexity. -""" - -from __future__ import annotations - -import warnings -from enum import Enum -from typing import TYPE_CHECKING - -from sqlalchemy import select - -from airflow.api.common.experimental.get_task_instance import get_task_instance -from airflow.api_internal.internal_api_call import InternalApiConfig -from airflow.exceptions import AirflowException, RemovedInAirflow3Warning, TaskInstanceNotFound -from airflow.models import DagRun -from airflow.models.dag import DagContext -from airflow.models.pool import Pool -from airflow.models.taskinstance import TaskInstance -from airflow.sensors.base import BaseSensorOperator -from airflow.utils.session import NEW_SESSION, create_session, provide_session -from airflow.utils.state import DagRunState, TaskInstanceState -from airflow.utils.types import DagRunType - -if TYPE_CHECKING: - from datetime import datetime - - from sqlalchemy.orm.session import Session - - from airflow.models.dag import DAG - from airflow.utils.context import Context - - -class SkippedStatePropagationOptions(Enum): - """Available options for skipped state propagation of subdag's tasks to parent dag tasks.""" - - ALL_LEAVES = "all_leaves" - ANY_LEAF = "any_leaf" - - -class SubDagOperator(BaseSensorOperator): - """ - This class is deprecated, please use :class:`airflow.utils.task_group.TaskGroup`. - - This runs a sub dag. By convention, a sub dag's dag_id - should be prefixed by its parent and a dot. As in `parent.child`. - Although SubDagOperator can occupy a pool/concurrency slot, - user can specify the mode=reschedule so that the slot will be - released periodically to avoid potential deadlock. - - :param subdag: the DAG object to run as a subdag of the current DAG. - :param session: sqlalchemy session - :param conf: Configuration for the subdag - :param propagate_skipped_state: by setting this argument you can define - whether the skipped state of leaf task(s) should be propagated to the - parent dag's downstream task. - """ - - ui_color = "#555" - ui_fgcolor = "#fff" - - subdag: DAG - - @provide_session - def __init__( - self, - *, - subdag: DAG, - session: Session = NEW_SESSION, - conf: dict | None = None, - propagate_skipped_state: SkippedStatePropagationOptions | None = None, - **kwargs, - ) -> None: - super().__init__(**kwargs) - self.subdag = subdag - self.conf = conf - self.propagate_skipped_state = propagate_skipped_state - - self._validate_dag(kwargs) - if not InternalApiConfig.get_use_internal_api(): - self._validate_pool(session) - - warnings.warn( - """This class is deprecated. Please use `airflow.utils.task_group.TaskGroup`.""", - RemovedInAirflow3Warning, - stacklevel=4, - ) - - def _validate_dag(self, kwargs): - dag = kwargs.get("dag") or DagContext.get_current_dag() - - if not dag: - raise AirflowException("Please pass in the `dag` param or call within a DAG context manager") - - if dag.dag_id + "." + kwargs["task_id"] != self.subdag.dag_id: - raise AirflowException( - f"The subdag's dag_id should have the form '{{parent_dag_id}}.{{this_task_id}}'. " - f"Expected '{dag.dag_id}.{kwargs['task_id']}'; received '{self.subdag.dag_id}'." - ) - - def _validate_pool(self, session): - if self.pool: - conflicts = [t for t in self.subdag.tasks if t.pool == self.pool] - if conflicts: - # only query for pool conflicts if one may exist - pool = session.scalar(select(Pool).where(Pool.slots == 1, Pool.pool == self.pool)) - if pool and any(t.pool == self.pool for t in self.subdag.tasks): - raise AirflowException( - f"SubDagOperator {self.task_id} and subdag task{'s' if len(conflicts) > 1 else ''} " - f"{', '.join(t.task_id for t in conflicts)} both use pool {self.pool}, " - f"but the pool only has 1 slot. The subdag tasks will never run." - ) - - def _get_dagrun(self, execution_date): - dag_runs = DagRun.find( - dag_id=self.subdag.dag_id, - execution_date=execution_date, - ) - return dag_runs[0] if dag_runs else None - - def _reset_dag_run_and_task_instances(self, dag_run: DagRun, execution_date: datetime) -> None: - """ - Set task instance states to allow for execution. - - The state of the DAG run will be set to RUNNING, and failed task - instances to ``None`` for scheduler to pick up. - - :param dag_run: DAG run to reset. - :param execution_date: Execution date to select task instances. - """ - with create_session() as session: - dag_run.state = DagRunState.RUNNING - session.merge(dag_run) - failed_task_instances = session.scalars( - select(TaskInstance) - .where(TaskInstance.dag_id == self.subdag.dag_id) - .where(TaskInstance.execution_date == execution_date) - .where(TaskInstance.state.in_((TaskInstanceState.FAILED, TaskInstanceState.UPSTREAM_FAILED))) - ) - - for task_instance in failed_task_instances: - task_instance.state = None - session.merge(task_instance) - session.commit() - - def pre_execute(self, context): - super().pre_execute(context) - execution_date = context["execution_date"] - dag_run = self._get_dagrun(execution_date) - - if dag_run is None: - if context["data_interval_start"] is None or context["data_interval_end"] is None: - data_interval: tuple[datetime, datetime] | None = None - else: - data_interval = (context["data_interval_start"], context["data_interval_end"]) - dag_run = self.subdag.create_dagrun( - run_type=DagRunType.SCHEDULED, - execution_date=execution_date, - state=DagRunState.RUNNING, - conf=self.conf, - external_trigger=True, - data_interval=data_interval, - ) - self.log.info("Created DagRun: %s", dag_run.run_id) - else: - self.log.info("Found existing DagRun: %s", dag_run.run_id) - if dag_run.state == DagRunState.FAILED: - self._reset_dag_run_and_task_instances(dag_run, execution_date) - - def poke(self, context: Context): - execution_date = context["execution_date"] - dag_run = self._get_dagrun(execution_date=execution_date) - return dag_run.state != DagRunState.RUNNING - - def post_execute(self, context, result=None): - super().post_execute(context) - execution_date = context["execution_date"] - dag_run = self._get_dagrun(execution_date=execution_date) - self.log.info("Execution finished. State is %s", dag_run.state) - - if dag_run.state != DagRunState.SUCCESS: - raise AirflowException(f"Expected state: SUCCESS. Actual state: {dag_run.state}") - - if self.propagate_skipped_state and self._check_skipped_states(context): - self._skip_downstream_tasks(context) - - def _check_skipped_states(self, context): - leaves_tis = self._get_leaves_tis(context["execution_date"]) - - if self.propagate_skipped_state == SkippedStatePropagationOptions.ANY_LEAF: - return any(ti.state == TaskInstanceState.SKIPPED for ti in leaves_tis) - if self.propagate_skipped_state == SkippedStatePropagationOptions.ALL_LEAVES: - return all(ti.state == TaskInstanceState.SKIPPED for ti in leaves_tis) - raise AirflowException( - f"Unimplemented SkippedStatePropagationOptions {self.propagate_skipped_state} used." - ) - - def _get_leaves_tis(self, execution_date): - leaves_tis = [] - for leaf in self.subdag.leaves: - try: - ti = get_task_instance( - dag_id=self.subdag.dag_id, task_id=leaf.task_id, execution_date=execution_date - ) - leaves_tis.append(ti) - except TaskInstanceNotFound: - continue - return leaves_tis - - def _skip_downstream_tasks(self, context): - self.log.info( - "Skipping downstream tasks because propagate_skipped_state is set to %s " - "and skipped task(s) were found.", - self.propagate_skipped_state, - ) - - downstream_tasks = context["task"].downstream_list - self.log.debug("Downstream task_ids %s", downstream_tasks) - - if downstream_tasks: - self.skip( - context["dag_run"], - context["execution_date"], - downstream_tasks, - map_index=context["ti"].map_index, - ) - - self.log.info("Done.") diff --git a/airflow/providers/celery/executors/celery_executor_utils.py b/airflow/providers/celery/executors/celery_executor_utils.py index 5dd2d59ab0ead..8f25f040c90ad 100644 --- a/airflow/providers/celery/executors/celery_executor_utils.py +++ b/airflow/providers/celery/executors/celery_executor_utils.py @@ -112,8 +112,7 @@ def on_celery_import_modules(*args, **kwargs): import airflow.jobs.local_task_job_runner import airflow.macros import airflow.operators.bash - import airflow.operators.python - import airflow.operators.subdag # noqa: F401 + import airflow.operators.python # noqa: F401 with contextlib.suppress(ImportError): import numpy # noqa: F401 diff --git a/airflow/providers/cncf/kubernetes/operators/pod.py b/airflow/providers/cncf/kubernetes/operators/pod.py index 921fdaa92762f..afe6cfef9228c 100644 --- a/airflow/providers/cncf/kubernetes/operators/pod.py +++ b/airflow/providers/cncf/kubernetes/operators/pod.py @@ -495,8 +495,9 @@ def _get_ti_pod_labels(context: Context | None = None, include_try_number: bool if include_try_number: labels.update(try_number=ti.try_number) # In the case of sub dags this is just useful - if context["dag"].parent_dag: + if getattr(context["dag"], "parent_dag", False): labels["parent_dag_id"] = context["dag"].parent_dag.dag_id + # Ensure that label is valid for Kube, # and if not truncate/remove invalid chars and replace with short hash. for label_id, label in labels.items(): diff --git a/airflow/providers/cncf/kubernetes/operators/spark_kubernetes.py b/airflow/providers/cncf/kubernetes/operators/spark_kubernetes.py index 82df0a2ec90c0..d8a2867c64aca 100644 --- a/airflow/providers/cncf/kubernetes/operators/spark_kubernetes.py +++ b/airflow/providers/cncf/kubernetes/operators/spark_kubernetes.py @@ -202,7 +202,7 @@ def create_labels_for_pod(context: dict | None = None, include_try_number: bool labels.update(try_number=ti.try_number) # In the case of sub dags this is just useful - if context["dag"].is_subdag: + if getattr(context["dag"], "is_subdag", False): labels["parent_dag_id"] = context["dag"].parent_dag.dag_id # Ensure that label is valid for Kube, # and if not truncate/remove invalid chars and replace with short hash. diff --git a/airflow/providers/fab/auth_manager/fab_auth_manager.py b/airflow/providers/fab/auth_manager/fab_auth_manager.py index 344df7588de7d..ceec5c0e37bd3 100644 --- a/airflow/providers/fab/auth_manager/fab_auth_manager.py +++ b/airflow/providers/fab/auth_manager/fab_auth_manager.py @@ -503,7 +503,7 @@ def _get_root_dag_id(self, dag_id: str) -> str: :meta private: """ - if "." in dag_id: + if "." in dag_id and hasattr(DagModel, "root_dag_id"): return self.appbuilder.get_session.scalar( select(DagModel.dag_id, DagModel.root_dag_id).where(DagModel.dag_id == dag_id).limit(1) ) diff --git a/airflow/providers/fab/auth_manager/security_manager/override.py b/airflow/providers/fab/auth_manager/security_manager/override.py index e2208e5fb409f..86d76de76ff2b 100644 --- a/airflow/providers/fab/auth_manager/security_manager/override.py +++ b/airflow/providers/fab/auth_manager/security_manager/override.py @@ -1073,7 +1073,7 @@ def create_dag_specific_permissions(self) -> None: dags = dagbag.dags.values() for dag in dags: - root_dag_id = dag.parent_dag.dag_id if dag.parent_dag else dag.dag_id + root_dag_id = (getattr(dag, "parent_dag", None) or dag).dag_id for resource_name, resource_values in self.RESOURCE_DETAILS_MAP.items(): dag_resource_name = self._resource_name(root_dag_id, resource_name) for action_name in resource_values["actions"]: @@ -2828,7 +2828,7 @@ def filter_roles_by_perm_with_action(self, action_name: str, role_ids: list[int] ).all() def _get_root_dag_id(self, dag_id: str) -> str: - if "." in dag_id: + if "." in dag_id and hasattr(DagModel, "root_dag_id"): dm = self.appbuilder.get_session.execute( select(DagModel.dag_id, DagModel.root_dag_id).where(DagModel.dag_id == dag_id) ).one() diff --git a/airflow/serialization/pydantic/dag.py b/airflow/serialization/pydantic/dag.py index fa1cac535f60b..a1fea6384aade 100644 --- a/airflow/serialization/pydantic/dag.py +++ b/airflow/serialization/pydantic/dag.py @@ -108,10 +108,8 @@ class DagModelPydantic(BaseModelPydantic): """Serializable representation of the DagModel ORM SqlAlchemyModel used by internal API.""" dag_id: str - root_dag_id: Optional[str] is_paused_at_creation: bool = airflow_conf.getboolean("core", "dags_are_paused_at_creation") is_paused: bool = is_paused_at_creation - is_subdag: Optional[bool] = False is_active: Optional[bool] = False last_parsed_time: Optional[datetime] last_pickled: Optional[datetime] @@ -127,7 +125,6 @@ class DagModelPydantic(BaseModelPydantic): timetable_description: Optional[str] tags: List[DagTagPydantic] # noqa: UP006 dag_owner_links: List[DagOwnerAttributesPydantic] # noqa: UP006 - parent_dag: Optional[PydanticDag] max_active_tasks: int max_active_runs: Optional[int] diff --git a/airflow/serialization/schema.json b/airflow/serialization/schema.json index 84b2e2ed4a75f..d76bfcb1a40bd 100644 --- a/airflow/serialization/schema.json +++ b/airflow/serialization/schema.json @@ -157,7 +157,6 @@ } }, "catchup": { "type": "boolean" }, - "is_subdag": { "type": "boolean" }, "fileloc": { "type" : "string"}, "_processor_dags_folder": { "anyOf": [ diff --git a/airflow/serialization/serialized_objects.py b/airflow/serialization/serialized_objects.py index d110271c3da08..a10916852b653 100644 --- a/airflow/serialization/serialized_objects.py +++ b/airflow/serialization/serialized_objects.py @@ -1273,8 +1273,6 @@ def populate_operator(cls, op: Operator, encoded_op: dict[str, Any]) -> None: continue elif k == "downstream_task_ids": v = set(v) - elif k == "subdag": - v = SerializedDAG.deserialize_dag(v) elif k in {"retry_delay", "execution_timeout", "sla", "max_retry_delay"}: v = cls._deserialize_timedelta(v) elif k in encoded_op["template_fields"]: @@ -1359,9 +1357,6 @@ def set_task_dag_references(task: Operator, dag: DAG) -> None: if getattr(task, date_attr, None) is None: setattr(task, date_attr, getattr(dag, date_attr, None)) - if task.subdag is not None: - task.subdag.parent_dag = dag - # Dereference expand_input and op_kwargs_expand_input. for k in ("expand_input", "op_kwargs_expand_input"): if isinstance(kwargs_ref := getattr(task, k, None), _ExpandInputRef): diff --git a/airflow/utils/db.py b/airflow/utils/db.py index b5e722cb50056..a86ca5dbf5628 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -95,7 +95,7 @@ class MappedClassProtocol(Protocol): _REVISION_HEADS_MAP = { "2.10.0": "22ed7efa9da2", - "3.0.0": "044f740568ec", + "3.0.0": "d0f1c55954fa", } diff --git a/airflow/utils/task_group.py b/airflow/utils/task_group.py index f3d46ea6eac7c..69a5d015bd426 100644 --- a/airflow/utils/task_group.py +++ b/airflow/utils/task_group.py @@ -491,7 +491,7 @@ def hierarchical_alphabetical_sort(self): self.children.values(), key=lambda node: (not isinstance(node, TaskGroup), node.node_id) ) - def topological_sort(self, _include_subdag_tasks: bool = False): + def topological_sort(self): """ Sorts children in topographical order, such that a task comes after any of its upstream dependencies. @@ -499,8 +499,6 @@ def topological_sort(self, _include_subdag_tasks: bool = False): """ # This uses a modified version of Kahn's Topological Sort algorithm to # not have to pre-compute the "in-degree" of the nodes. - from airflow.operators.subdag import SubDagOperator # Avoid circular import - graph_unsorted = copy.copy(self.children) graph_sorted: list[DAGNode] = [] @@ -539,10 +537,6 @@ def topological_sort(self, _include_subdag_tasks: bool = False): acyclic = True del graph_unsorted[node.node_id] graph_sorted.append(node) - if _include_subdag_tasks and isinstance(node, SubDagOperator): - graph_sorted.extend( - node.subdag.task_group.topological_sort(_include_subdag_tasks=True) - ) if not acyclic: raise AirflowDagCycleException(f"A cyclic dependency occurred in dag: {self.dag_id}") diff --git a/airflow/www/static/js/dag/details/taskInstance/Nav.tsx b/airflow/www/static/js/dag/details/taskInstance/Nav.tsx index e24088085a9f9..dd03d6a28c073 100644 --- a/airflow/www/static/js/dag/details/taskInstance/Nav.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/Nav.tsx @@ -61,24 +61,9 @@ const Nav = forwardRef( const allInstancesLink = `${taskInstancesUrl}?${listParams.toString()}`; - const subDagLink = appendSearchParams( - gridUrl.replace(dagId, `${dagId}.${taskId}`), - subDagParams - ); - - // TODO: base subdag zooming as its own attribute instead of via operator name - const isSubDag = operator === "SubDagOperator"; - return ( - {(!isMapped || mapIndex !== undefined) && ( - <> - More Details - {isSubDag && ( - Zoom into SubDag - )} - - )} + {!isMapped || mapIndex !== undefined} List All Instances diff --git a/airflow/www/static/js/types/api-generated.ts b/airflow/www/static/js/types/api-generated.ts index a892e327ace07..30948df332bb1 100644 --- a/airflow/www/static/js/types/api-generated.ts +++ b/airflow/www/static/js/types/api-generated.ts @@ -1031,8 +1031,6 @@ export interface components { * *New in version 2.9.0* */ dag_display_name?: string; - /** @description If the DAG is SubDAG then it is the top level DAG identifier. Otherwise, null. */ - root_dag_id?: string | null; /** @description Whether the DAG is paused. */ is_paused?: boolean | null; /** @@ -1043,8 +1041,6 @@ export interface components { * *Changed in version 2.2.0*: Field is read-only. */ is_active?: boolean | null; - /** @description Whether the DAG is SubDAG. */ - is_subdag?: boolean; /** * Format: date-time * @description The last time the DAG was parsed. @@ -2090,10 +2086,6 @@ export interface components { * @default false */ only_running?: boolean; - /** @description Clear tasks in subdags and clear external tasks indicated by ExternalTaskMarker. */ - include_subdags?: boolean; - /** @description Clear tasks in the parent dag of the subdag. */ - include_parentdag?: boolean; /** @description Set state of DAG runs to RUNNING. */ reset_dag_runs?: boolean; /** @description The DagRun ID for this task instance */ diff --git a/airflow/www/views.py b/airflow/www/views.py index a485f84ed4b1c..d7b670303c86f 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -844,7 +844,7 @@ def index(self): with create_session() as session: # read orm_dags from the db - dags_query = select(DagModel).where(~DagModel.is_subdag, DagModel.is_active) + dags_query = select(DagModel).where(DagModel.is_active) if arg_search_query: escaped_arg_search_query = arg_search_query.replace("_", r"\_") @@ -2341,8 +2341,6 @@ def _clear_dag_tis( start_date=start_date, end_date=end_date, task_ids=task_ids, - include_subdags=recursive, - include_parentdag=recursive, only_failed=only_failed, session=session, ) @@ -2355,8 +2353,6 @@ def _clear_dag_tis( start_date=start_date, end_date=end_date, task_ids=task_ids, - include_subdags=recursive, - include_parentdag=recursive, only_failed=only_failed, dry_run=True, session=session, @@ -5488,8 +5484,6 @@ def _clear_task_instances( start_date=dag_run.execution_date, end_date=dag_run.execution_date, task_ids=downstream_task_ids_to_clear, - include_subdags=False, - include_parentdag=False, session=session, dry_run=True, ) @@ -5638,7 +5632,6 @@ def autocomplete(self, session: Session = NEW_SESSION): DagModel.dag_id.label("name"), DagModel._dag_display_property_value.label("dag_display_name"), ).where( - ~DagModel.is_subdag, DagModel.is_active, or_( DagModel.dag_id.ilike(f"%{query}%"), @@ -5653,7 +5646,7 @@ def autocomplete(self, session: Session = NEW_SESSION): sqla.literal(None).label("dag_display_name"), ) .distinct() - .where(~DagModel.is_subdag, DagModel.is_active, DagModel.owners.ilike(f"%{query}%")) + .where(DagModel.is_active, DagModel.owners.ilike(f"%{query}%")) ) # Hide DAGs if not showing status: "all" diff --git a/docs/apache-airflow/administration-and-deployment/pools.rst b/docs/apache-airflow/administration-and-deployment/pools.rst index 5a906d3f6fed2..56a830c33caa5 100644 --- a/docs/apache-airflow/administration-and-deployment/pools.rst +++ b/docs/apache-airflow/administration-and-deployment/pools.rst @@ -83,8 +83,3 @@ for the heavy task to complete before they are executed. Here, in terms of resou This implementation can prevent overwhelming system resources, which (in this example) could occur when a heavy and a light task are running concurrently. On the other hand, both light tasks can run concurrently since they only occupy one pool slot each, while the heavy task would have to wait for two pool slots to become available before getting executed. - -.. warning:: - - Pools and SubDAGs do not interact as you might first expect. SubDAGs will *not* honor any pool you set on them at - the top level; pools must be set on the tasks *inside* the SubDAG directly. diff --git a/docs/apache-airflow/core-concepts/dags.rst b/docs/apache-airflow/core-concepts/dags.rst index 482b604f33ed9..acc7b0ff16f48 100644 --- a/docs/apache-airflow/core-concepts/dags.rst +++ b/docs/apache-airflow/core-concepts/dags.rst @@ -543,7 +543,7 @@ TaskGroups A TaskGroup can be used to organize tasks into hierarchical groups in Graph view. It is useful for creating repeating patterns and cutting down visual clutter. -Unlike :ref:`concepts:subdags`, TaskGroups are purely a UI grouping concept. Tasks in TaskGroups live on the same original DAG, and honor all the DAG settings and pool configurations. +Tasks in TaskGroups live on the same original DAG, and honor all the DAG settings and pool configurations. .. image:: /img/task_group.gif @@ -680,96 +680,6 @@ This is especially useful if your tasks are built dynamically from configuration """ -.. _concepts:subdags: - -SubDAGs -------- - -.. note:: - - SubDAG is deprecated hence TaskGroup is always the preferred choice. - - -Sometimes, you will find that you are regularly adding exactly the same set of tasks to every DAG, or you want to group a lot of tasks into a single, logical unit. This is what SubDAGs are for. - -For example, here's a DAG that has a lot of parallel tasks in two sections: - -.. image:: /img/subdag_before.png - -We can combine all of the parallel ``task-*`` operators into a single SubDAG, so that the resulting DAG resembles the following: - -.. image:: /img/subdag_after.png - -Note that SubDAG operators should contain a factory method that returns a DAG object. This will prevent the SubDAG from being treated like a separate DAG in the main UI - remember, if Airflow sees a DAG at the top level of a Python file, it will :ref:`load it as its own DAG `. For example: - -.. exampleinclude:: /../../airflow/example_dags/subdags/subdag.py - :language: python - :start-after: [START subdag] - :end-before: [END subdag] - -This SubDAG can then be referenced in your main DAG file: - -.. exampleinclude:: /../../airflow/example_dags/example_subdag_operator.py - :language: python - :dedent: 4 - :start-after: [START example_subdag_operator] - :end-before: [END example_subdag_operator] - -You can zoom into a :class:`~airflow.operators.subdag.SubDagOperator` from the graph view of the main DAG to show the tasks contained within the SubDAG: - -.. image:: /img/subdag_zoom.png - -Some other tips when using SubDAGs: - -- By convention, a SubDAG's ``dag_id`` should be prefixed by the name of its parent DAG and a dot (``parent.child``) -- You should share arguments between the main DAG and the SubDAG by passing arguments to the SubDAG operator (as demonstrated above) -- SubDAGs must have a schedule and be enabled. If the SubDAG's schedule is set to ``None`` or ``@once``, the SubDAG will succeed without having done anything. -- Clearing a :class:`~airflow.operators.subdag.SubDagOperator` also clears the state of the tasks within it. -- Marking success on a :class:`~airflow.operators.subdag.SubDagOperator` does not affect the state of the tasks within it. -- Refrain from using :ref:`concepts:depends-on-past` in tasks within the SubDAG as this can be confusing. -- You can specify an executor for the SubDAG. It is common to use the SequentialExecutor if you want to run the SubDAG in-process and effectively limit its parallelism to one. Using LocalExecutor can be problematic as it may over-subscribe your worker, running multiple tasks in a single slot. - -See ``airflow/example_dags`` for a demonstration. - - -.. note:: - - Parallelism is *not honored* by :class:`~airflow.operators.subdag.SubDagOperator`, and so resources could be consumed by SubdagOperators beyond any limits you may have set. - - - -TaskGroups vs SubDAGs ----------------------- - -SubDAGs, while serving a similar purpose as TaskGroups, introduces both performance and functional issues due to its implementation. - -* The SubDagOperator starts a BackfillJob, which ignores existing parallelism configurations potentially oversubscribing the worker environment. -* SubDAGs have their own DAG attributes. When the SubDAG DAG attributes are inconsistent with its parent DAG, unexpected behavior can occur. -* Unable to see the "full" DAG in one view as SubDAGs exists as a full fledged DAG. -* SubDAGs introduces all sorts of edge cases and caveats. This can disrupt user experience and expectation. - -TaskGroups, on the other hand, is a better option given that it is purely a UI grouping concept. All tasks within the TaskGroup still behave as any other tasks outside of the TaskGroup. - -You can see the core differences between these two constructs. - -+--------------------------------------------------------+--------------------------------------------------------+ -| TaskGroup | SubDAG | -+========================================================+========================================================+ -| Repeating patterns as part of the same DAG | Repeating patterns as a separate DAG | -+--------------------------------------------------------+--------------------------------------------------------+ -| One set of views and statistics for the DAG | Separate set of views and statistics between parent | -| | and child DAGs | -+--------------------------------------------------------+--------------------------------------------------------+ -| One set of DAG configuration | Several sets of DAG configurations | -+--------------------------------------------------------+--------------------------------------------------------+ -| Honors parallelism configurations through existing | Does not honor parallelism configurations due to | -| SchedulerJob | newly spawned BackfillJob | -+--------------------------------------------------------+--------------------------------------------------------+ -| Simple construct declaration with context manager | Complex DAG factory with naming restrictions | -+--------------------------------------------------------+--------------------------------------------------------+ - - - Packaging DAGs -------------- diff --git a/docs/apache-airflow/core-concepts/overview.rst b/docs/apache-airflow/core-concepts/overview.rst index 767b7e8990fc4..e10cf9d5785d8 100644 --- a/docs/apache-airflow/core-concepts/overview.rst +++ b/docs/apache-airflow/core-concepts/overview.rst @@ -232,7 +232,7 @@ To pass data between tasks you have three options: Airflow sends out Tasks to run on Workers as space becomes available, so there's no guarantee all the tasks in your DAG will run on the same worker or the same machine. -As you build out your DAGs, they are likely to get very complex, so Airflow provides several mechanisms for making this more sustainable - :ref:`SubDAGs ` let you make "reusable" DAGs you can embed into other ones, and :ref:`concepts:taskgroups` let you visually group tasks in the UI. +As you build out your DAGs, they are likely to get very complex, so Airflow provides several mechanisms for making this more sustainable, example :ref:`concepts:taskgroups` let you visually group tasks in the UI. There are also features for letting you easily pre-configure access to a central resource, like a datastore, in the form of :doc:`../authoring-and-scheduling/connections`, and for limiting concurrency, via :doc:`../administration-and-deployment/pools`. diff --git a/docs/apache-airflow/img/airflow_erd.sha256 b/docs/apache-airflow/img/airflow_erd.sha256 index 4e832da49f0c4..e7e8667275258 100644 --- a/docs/apache-airflow/img/airflow_erd.sha256 +++ b/docs/apache-airflow/img/airflow_erd.sha256 @@ -1 +1 @@ -bbe537329f9e97dcb4a395e3f3e5d9df4ccd51b657aaa714ce27b2b80f9ca79a \ No newline at end of file +37de7143e49532e7650ba09d0172788c23e132b57123a5f6e863e7234cf4f79e \ No newline at end of file diff --git a/docs/apache-airflow/img/airflow_erd.svg b/docs/apache-airflow/img/airflow_erd.svg index a7d8e961dca79..9f253a53db27f 100644 --- a/docs/apache-airflow/img/airflow_erd.svg +++ b/docs/apache-airflow/img/airflow_erd.svg @@ -4,11 +4,11 @@ - - + + %3 - + log @@ -583,573 +583,565 @@ dataset_alias - -dataset_alias - -id - - [INTEGER] - NOT NULL - -name - - [VARCHAR(3000)] - NOT NULL + +dataset_alias + +id + + [INTEGER] + NOT NULL + +name + + [VARCHAR(3000)] + NOT NULL dataset_alias_dataset - -dataset_alias_dataset - -alias_id - - [INTEGER] - NOT NULL - -dataset_id - - [INTEGER] - NOT NULL + +dataset_alias_dataset + +alias_id + + [INTEGER] + NOT NULL + +dataset_id + + [INTEGER] + NOT NULL dataset_alias--dataset_alias_dataset - -0..N -1 + +0..N +1 dataset_alias--dataset_alias_dataset - -0..N -1 + +0..N +1 dataset_alias_dataset_event - -dataset_alias_dataset_event - -alias_id - - [INTEGER] - NOT NULL - -event_id - - [INTEGER] - NOT NULL + +dataset_alias_dataset_event + +alias_id + + [INTEGER] + NOT NULL + +event_id + + [INTEGER] + NOT NULL dataset_alias--dataset_alias_dataset_event - -0..N -1 + +0..N +1 dataset_alias--dataset_alias_dataset_event - -0..N -1 + +0..N +1 dag_schedule_dataset_alias_reference - -dag_schedule_dataset_alias_reference - -alias_id - - [INTEGER] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL + +dag_schedule_dataset_alias_reference + +alias_id + + [INTEGER] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL dataset_alias--dag_schedule_dataset_alias_reference - -0..N -1 + +0..N +1 dataset - -dataset - -id - - [INTEGER] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL - -extra - - [JSON] - NOT NULL - -is_orphaned - - [BOOLEAN] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL - -uri - - [VARCHAR(3000)] - NOT NULL + +dataset + +id + + [INTEGER] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL + +extra + + [JSON] + NOT NULL + +is_orphaned + + [BOOLEAN] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL + +uri + + [VARCHAR(3000)] + NOT NULL dataset--dataset_alias_dataset - -0..N -1 + +0..N +1 dataset--dataset_alias_dataset - -0..N -1 + +0..N +1 dag_schedule_dataset_reference - -dag_schedule_dataset_reference - -dag_id - - [VARCHAR(250)] - NOT NULL - -dataset_id - - [INTEGER] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL + +dag_schedule_dataset_reference + +dag_id + + [VARCHAR(250)] + NOT NULL + +dataset_id + + [INTEGER] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL dataset--dag_schedule_dataset_reference - -0..N -1 + +0..N +1 task_outlet_dataset_reference - -task_outlet_dataset_reference - -dag_id - - [VARCHAR(250)] - NOT NULL - -dataset_id - - [INTEGER] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL + +task_outlet_dataset_reference + +dag_id + + [VARCHAR(250)] + NOT NULL + +dataset_id + + [INTEGER] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL dataset--task_outlet_dataset_reference - -0..N -1 + +0..N +1 dataset_dag_run_queue - -dataset_dag_run_queue - -dataset_id - - [INTEGER] - NOT NULL - -target_dag_id - - [VARCHAR(250)] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL + +dataset_dag_run_queue + +dataset_id + + [INTEGER] + NOT NULL + +target_dag_id + + [VARCHAR(250)] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL dataset--dataset_dag_run_queue - -0..N -1 + +0..N +1 dataset_event - -dataset_event - -id - - [INTEGER] - NOT NULL - -dataset_id - - [INTEGER] - NOT NULL - -extra - - [JSON] - NOT NULL - -source_dag_id - - [VARCHAR(250)] - -source_map_index - - [INTEGER] - -source_run_id - - [VARCHAR(250)] - -source_task_id - - [VARCHAR(250)] - -timestamp - - [TIMESTAMP] - NOT NULL + +dataset_event + +id + + [INTEGER] + NOT NULL + +dataset_id + + [INTEGER] + NOT NULL + +extra + + [JSON] + NOT NULL + +source_dag_id + + [VARCHAR(250)] + +source_map_index + + [INTEGER] + +source_run_id + + [VARCHAR(250)] + +source_task_id + + [VARCHAR(250)] + +timestamp + + [TIMESTAMP] + NOT NULL dataset_event--dataset_alias_dataset_event - -0..N -1 + +0..N +1 dataset_event--dataset_alias_dataset_event - -0..N -1 + +0..N +1 dagrun_dataset_event - -dagrun_dataset_event - -dag_run_id - - [INTEGER] - NOT NULL - -event_id - - [INTEGER] - NOT NULL + +dagrun_dataset_event + +dag_run_id + + [INTEGER] + NOT NULL + +event_id + + [INTEGER] + NOT NULL dataset_event--dagrun_dataset_event - -0..N -1 + +0..N +1 dag - -dag - -dag_id - - [VARCHAR(250)] - NOT NULL - -dag_display_name - - [VARCHAR(2000)] - -dataset_expression - - [JSON] - -default_view - - [VARCHAR(25)] - -description - - [TEXT] - -fileloc - - [VARCHAR(2000)] - -has_import_errors - - [BOOLEAN] - -has_task_concurrency_limits - - [BOOLEAN] - NOT NULL - -is_active - - [BOOLEAN] - -is_paused - - [BOOLEAN] - -is_subdag - - [BOOLEAN] - -last_expired - - [TIMESTAMP] - -last_parsed_time - - [TIMESTAMP] - -last_pickled - - [TIMESTAMP] - -max_active_runs - - [INTEGER] - -max_active_tasks - - [INTEGER] - NOT NULL - -max_consecutive_failed_dag_runs - - [INTEGER] - NOT NULL - -next_dagrun - - [TIMESTAMP] - -next_dagrun_create_after - - [TIMESTAMP] - -next_dagrun_data_interval_end - - [TIMESTAMP] - -next_dagrun_data_interval_start - - [TIMESTAMP] - -owners - - [VARCHAR(2000)] - -pickle_id - - [INTEGER] - -processor_subdir - - [VARCHAR(2000)] - -root_dag_id - - [VARCHAR(250)] - -schedule_interval - - [TEXT] - -scheduler_lock - - [BOOLEAN] - -timetable_description - - [VARCHAR(1000)] + +dag + +dag_id + + [VARCHAR(250)] + NOT NULL + +dag_display_name + + [VARCHAR(2000)] + +dataset_expression + + [JSON] + +default_view + + [VARCHAR(25)] + +description + + [TEXT] + +fileloc + + [VARCHAR(2000)] + +has_import_errors + + [BOOLEAN] + +has_task_concurrency_limits + + [BOOLEAN] + NOT NULL + +is_active + + [BOOLEAN] + +is_paused + + [BOOLEAN] + +last_expired + + [TIMESTAMP] + +last_parsed_time + + [TIMESTAMP] + +last_pickled + + [TIMESTAMP] + +max_active_runs + + [INTEGER] + +max_active_tasks + + [INTEGER] + NOT NULL + +max_consecutive_failed_dag_runs + + [INTEGER] + NOT NULL + +next_dagrun + + [TIMESTAMP] + +next_dagrun_create_after + + [TIMESTAMP] + +next_dagrun_data_interval_end + + [TIMESTAMP] + +next_dagrun_data_interval_start + + [TIMESTAMP] + +owners + + [VARCHAR(2000)] + +pickle_id + + [INTEGER] + +processor_subdir + + [VARCHAR(2000)] + +schedule_interval + + [TEXT] + +scheduler_lock + + [BOOLEAN] + +timetable_description + + [VARCHAR(1000)] dag--dag_schedule_dataset_alias_reference - -0..N -1 + +0..N +1 dag--dag_schedule_dataset_reference - -0..N -1 + +0..N +1 dag--task_outlet_dataset_reference - -0..N -1 + +0..N +1 dag--dataset_dag_run_queue - -0..N -1 + +0..N +1 dag_tag - -dag_tag - -dag_id - - [VARCHAR(250)] - NOT NULL - -name - - [VARCHAR(100)] - NOT NULL + +dag_tag + +dag_id + + [VARCHAR(250)] + NOT NULL + +name + + [VARCHAR(100)] + NOT NULL dag--dag_tag - -0..N -1 + +0..N +1 dag_owner_attributes - -dag_owner_attributes - -dag_id - - [VARCHAR(250)] - NOT NULL - -owner - - [VARCHAR(500)] - NOT NULL - -link - - [VARCHAR(500)] - NOT NULL + +dag_owner_attributes + +dag_id + + [VARCHAR(250)] + NOT NULL + +owner + + [VARCHAR(500)] + NOT NULL + +link + + [VARCHAR(500)] + NOT NULL dag--dag_owner_attributes - -0..N -1 + +0..N +1 dag_warning - -dag_warning - -dag_id - - [VARCHAR(250)] - NOT NULL - -warning_type - - [VARCHAR(50)] - NOT NULL - -message - - [TEXT] - NOT NULL - -timestamp - - [TIMESTAMP] - NOT NULL + +dag_warning + +dag_id + + [VARCHAR(250)] + NOT NULL + +warning_type + + [VARCHAR(50)] + NOT NULL + +message + + [TEXT] + NOT NULL + +timestamp + + [TIMESTAMP] + NOT NULL dag--dag_warning - -0..N -1 + +0..N +1 @@ -1275,813 +1267,813 @@ dag_run--dagrun_dataset_event - -0..N -1 + +0..N +1 task_instance - -task_instance - -dag_id - - [VARCHAR(250)] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -custom_operator_name - - [VARCHAR(1000)] - -duration - - [DOUBLE_PRECISION] - -end_date - - [TIMESTAMP] - -executor - - [VARCHAR(1000)] - -executor_config - - [BYTEA] - -external_executor_id - - [VARCHAR(250)] - -hostname - - [VARCHAR(1000)] - -job_id - - [INTEGER] - -max_tries - - [INTEGER] - -next_kwargs - - [JSON] - -next_method - - [VARCHAR(1000)] - -operator - - [VARCHAR(1000)] - -pid - - [INTEGER] - -pool - - [VARCHAR(256)] - NOT NULL - -pool_slots - - [INTEGER] - NOT NULL - -priority_weight - - [INTEGER] - -queue - - [VARCHAR(256)] - -queued_by_job_id - - [INTEGER] - -queued_dttm - - [TIMESTAMP] - -rendered_map_index - - [VARCHAR(250)] - -start_date - - [TIMESTAMP] - -state - - [VARCHAR(20)] - -task_display_name - - [VARCHAR(2000)] - -trigger_id - - [INTEGER] - -trigger_timeout - - [TIMESTAMP] - -try_number - - [INTEGER] - -unixname - - [VARCHAR(1000)] - -updated_at - - [TIMESTAMP] + +task_instance + +dag_id + + [VARCHAR(250)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +custom_operator_name + + [VARCHAR(1000)] + +duration + + [DOUBLE_PRECISION] + +end_date + + [TIMESTAMP] + +executor + + [VARCHAR(1000)] + +executor_config + + [BYTEA] + +external_executor_id + + [VARCHAR(250)] + +hostname + + [VARCHAR(1000)] + +job_id + + [INTEGER] + +max_tries + + [INTEGER] + +next_kwargs + + [JSON] + +next_method + + [VARCHAR(1000)] + +operator + + [VARCHAR(1000)] + +pid + + [INTEGER] + +pool + + [VARCHAR(256)] + NOT NULL + +pool_slots + + [INTEGER] + NOT NULL + +priority_weight + + [INTEGER] + +queue + + [VARCHAR(256)] + +queued_by_job_id + + [INTEGER] + +queued_dttm + + [TIMESTAMP] + +rendered_map_index + + [VARCHAR(250)] + +start_date + + [TIMESTAMP] + +state + + [VARCHAR(20)] + +task_display_name + + [VARCHAR(2000)] + +trigger_id + + [INTEGER] + +trigger_timeout + + [TIMESTAMP] + +try_number + + [INTEGER] + +unixname + + [VARCHAR(1000)] + +updated_at + + [TIMESTAMP] dag_run--task_instance - -0..N -1 + +0..N +1 dag_run--task_instance - -0..N -1 + +0..N +1 dag_run_note - -dag_run_note - -dag_run_id - - [INTEGER] - NOT NULL - -content - - [VARCHAR(1000)] - -created_at - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL - -user_id - - [INTEGER] + +dag_run_note + +dag_run_id + + [INTEGER] + NOT NULL + +content + + [VARCHAR(1000)] + +created_at + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL + +user_id + + [INTEGER] dag_run--dag_run_note - -1 -1 + +1 +1 task_reschedule - -task_reschedule - -id - - [INTEGER] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -duration - - [INTEGER] - NOT NULL - -end_date - - [TIMESTAMP] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -reschedule_date - - [TIMESTAMP] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -start_date - - [TIMESTAMP] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -try_number - - [INTEGER] - NOT NULL + +task_reschedule + +id + + [INTEGER] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +duration + + [INTEGER] + NOT NULL + +end_date + + [TIMESTAMP] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +reschedule_date + + [TIMESTAMP] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +start_date + + [TIMESTAMP] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +try_number + + [INTEGER] + NOT NULL dag_run--task_reschedule - -0..N -1 + +0..N +1 dag_run--task_reschedule - -0..N -1 + +0..N +1 task_instance--task_reschedule - -0..N -1 + +0..N +1 task_instance--task_reschedule - -0..N -1 + +0..N +1 task_instance--task_reschedule - -0..N -1 + +0..N +1 task_instance--task_reschedule - -0..N -1 + +0..N +1 rendered_task_instance_fields - -rendered_task_instance_fields - -dag_id - - [VARCHAR(250)] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -k8s_pod_yaml - - [JSON] - -rendered_fields - - [JSON] - NOT NULL + +rendered_task_instance_fields + +dag_id + + [VARCHAR(250)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +k8s_pod_yaml + + [JSON] + +rendered_fields + + [JSON] + NOT NULL task_instance--rendered_task_instance_fields - -0..N -1 + +0..N +1 task_instance--rendered_task_instance_fields - -0..N -1 + +0..N +1 task_instance--rendered_task_instance_fields - -0..N -1 + +0..N +1 task_instance--rendered_task_instance_fields - -0..N -1 + +0..N +1 task_fail - -task_fail - -id - - [INTEGER] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -duration - - [INTEGER] - -end_date - - [TIMESTAMP] - -map_index - - [INTEGER] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -start_date - - [TIMESTAMP] - -task_id - - [VARCHAR(250)] - NOT NULL + +task_fail + +id + + [INTEGER] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +duration + + [INTEGER] + +end_date + + [TIMESTAMP] + +map_index + + [INTEGER] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +start_date + + [TIMESTAMP] + +task_id + + [VARCHAR(250)] + NOT NULL task_instance--task_fail - -0..N -1 + +0..N +1 task_instance--task_fail - -0..N -1 + +0..N +1 task_instance--task_fail - -0..N -1 + +0..N +1 task_instance--task_fail - -0..N -1 + +0..N +1 task_map - -task_map - -dag_id - - [VARCHAR(250)] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -keys - - [JSON] - -length - - [INTEGER] - NOT NULL + +task_map + +dag_id + + [VARCHAR(250)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +keys + + [JSON] + +length + + [INTEGER] + NOT NULL task_instance--task_map - -0..N -1 + +0..N +1 task_instance--task_map - -0..N -1 + +0..N +1 task_instance--task_map - -0..N -1 + +0..N +1 task_instance--task_map - -0..N -1 + +0..N +1 xcom - -xcom - -dag_run_id - - [INTEGER] - NOT NULL - -key - - [VARCHAR(512)] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -timestamp - - [TIMESTAMP] - NOT NULL - -value - - [BYTEA] + +xcom + +dag_run_id + + [INTEGER] + NOT NULL + +key + + [VARCHAR(512)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +timestamp + + [TIMESTAMP] + NOT NULL + +value + + [BYTEA] task_instance--xcom - -0..N -1 + +0..N +1 task_instance--xcom - -0..N -1 + +0..N +1 task_instance--xcom - -0..N -1 + +0..N +1 task_instance--xcom - -0..N -1 + +0..N +1 task_instance_note - -task_instance_note - -dag_id - - [VARCHAR(250)] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -content - - [VARCHAR(1000)] - -created_at - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL - -user_id - - [INTEGER] + +task_instance_note + +dag_id + + [VARCHAR(250)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +content + + [VARCHAR(1000)] + +created_at + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL + +user_id + + [INTEGER] task_instance--task_instance_note - -0..N -1 + +0..N +1 task_instance--task_instance_note - -0..N -1 + +0..N +1 task_instance--task_instance_note - -0..N -1 + +0..N +1 task_instance--task_instance_note - -0..N -1 + +0..N +1 task_instance_history - -task_instance_history - -id - - [INTEGER] - NOT NULL - -custom_operator_name - - [VARCHAR(1000)] - -dag_id - - [VARCHAR(250)] - NOT NULL - -duration - - [DOUBLE_PRECISION] - -end_date - - [TIMESTAMP] - -executor - - [VARCHAR(1000)] - -executor_config - - [BYTEA] - -external_executor_id - - [VARCHAR(250)] - -hostname - - [VARCHAR(1000)] - -job_id - - [INTEGER] - -map_index - - [INTEGER] - NOT NULL - -max_tries - - [INTEGER] - -next_kwargs - - [JSON] - -next_method - - [VARCHAR(1000)] - -operator - - [VARCHAR(1000)] - -pid - - [INTEGER] - -pool - - [VARCHAR(256)] - NOT NULL - -pool_slots - - [INTEGER] - NOT NULL - -priority_weight - - [INTEGER] - -queue - - [VARCHAR(256)] - -queued_by_job_id - - [INTEGER] - -queued_dttm - - [TIMESTAMP] - -rendered_map_index - - [VARCHAR(250)] - -run_id - - [VARCHAR(250)] - NOT NULL - -start_date - - [TIMESTAMP] - -state - - [VARCHAR(20)] - -task_display_name - - [VARCHAR(2000)] - -task_id - - [VARCHAR(250)] - NOT NULL - -trigger_id - - [INTEGER] - -trigger_timeout - - [TIMESTAMP] - -try_number - - [INTEGER] - NOT NULL - -unixname - - [VARCHAR(1000)] - -updated_at - - [TIMESTAMP] + +task_instance_history + +id + + [INTEGER] + NOT NULL + +custom_operator_name + + [VARCHAR(1000)] + +dag_id + + [VARCHAR(250)] + NOT NULL + +duration + + [DOUBLE_PRECISION] + +end_date + + [TIMESTAMP] + +executor + + [VARCHAR(1000)] + +executor_config + + [BYTEA] + +external_executor_id + + [VARCHAR(250)] + +hostname + + [VARCHAR(1000)] + +job_id + + [INTEGER] + +map_index + + [INTEGER] + NOT NULL + +max_tries + + [INTEGER] + +next_kwargs + + [JSON] + +next_method + + [VARCHAR(1000)] + +operator + + [VARCHAR(1000)] + +pid + + [INTEGER] + +pool + + [VARCHAR(256)] + NOT NULL + +pool_slots + + [INTEGER] + NOT NULL + +priority_weight + + [INTEGER] + +queue + + [VARCHAR(256)] + +queued_by_job_id + + [INTEGER] + +queued_dttm + + [TIMESTAMP] + +rendered_map_index + + [VARCHAR(250)] + +run_id + + [VARCHAR(250)] + NOT NULL + +start_date + + [TIMESTAMP] + +state + + [VARCHAR(20)] + +task_display_name + + [VARCHAR(2000)] + +task_id + + [VARCHAR(250)] + NOT NULL + +trigger_id + + [INTEGER] + +trigger_timeout + + [TIMESTAMP] + +try_number + + [INTEGER] + NOT NULL + +unixname + + [VARCHAR(1000)] + +updated_at + + [TIMESTAMP] task_instance--task_instance_history - -0..N -1 + +0..N +1 task_instance--task_instance_history - -0..N -1 + +0..N +1 task_instance--task_instance_history - -0..N -1 + +0..N +1 task_instance--task_instance_history - -0..N -1 + +0..N +1 @@ -2207,39 +2199,39 @@ trigger - -trigger - -id - - [INTEGER] - NOT NULL - -classpath - - [VARCHAR(1000)] - NOT NULL - -created_date - - [TIMESTAMP] - NOT NULL - -kwargs - - [TEXT] - NOT NULL - -triggerer_id - - [INTEGER] + +trigger + +id + + [INTEGER] + NOT NULL + +classpath + + [VARCHAR(1000)] + NOT NULL + +created_date + + [TIMESTAMP] + NOT NULL + +kwargs + + [TEXT] + NOT NULL + +triggerer_id + + [INTEGER] trigger--task_instance - -0..N -{0,1} + +0..N +{0,1} diff --git a/docs/apache-airflow/migrations-ref.rst b/docs/apache-airflow/migrations-ref.rst index 1dbe67bf271f6..37640e2b5ed86 100644 --- a/docs/apache-airflow/migrations-ref.rst +++ b/docs/apache-airflow/migrations-ref.rst @@ -36,13 +36,16 @@ Here's the list of all the Database Migrations that are executed via when you ru .. All table elements are scraped from migration files .. Beginning of auto-generated table -+-------------------------+------------------+-------------------+-------------------------------------------------+ -| Revision ID | Revises ID | Airflow Version | Description | -+=========================+==================+===================+=================================================+ -| ``044f740568ec`` (head) | ``22ed7efa9da2`` | ``3.0.0`` | Drop ab_user.id foreign key. | -+-------------------------+------------------+-------------------+-------------------------------------------------+ -| ``22ed7efa9da2`` (base) | ``None`` | ``2.10.0`` | Add dag_schedule_dataset_alias_reference table. | -+-------------------------+------------------+-------------------+-------------------------------------------------+ ++-------------------------+------------------+-------------------+--------------------------------------------------------------+ +| Revision ID | Revises ID | Airflow Version | Description | ++=========================+==================+===================+==============================================================+ +| ``d0f1c55954fa`` (head) | ``044f740568ec`` | ``3.0.0`` | Remove SubDAGs: ``is_subdag`` & ``root_dag_id`` columns from | +| | | | DAG table. | ++-------------------------+------------------+-------------------+--------------------------------------------------------------+ +| ``044f740568ec`` | ``22ed7efa9da2`` | ``3.0.0`` | Drop ab_user.id foreign key. | ++-------------------------+------------------+-------------------+--------------------------------------------------------------+ +| ``22ed7efa9da2`` (base) | ``None`` | ``2.10.0`` | Add dag_schedule_dataset_alias_reference table. | ++-------------------------+------------------+-------------------+--------------------------------------------------------------+ .. End of auto-generated table diff --git a/docs/apache-airflow/operators-and-hooks-ref.rst b/docs/apache-airflow/operators-and-hooks-ref.rst index 9fdf9d97cc510..6742559a02304 100644 --- a/docs/apache-airflow/operators-and-hooks-ref.rst +++ b/docs/apache-airflow/operators-and-hooks-ref.rst @@ -74,9 +74,6 @@ For details see: :doc:`apache-airflow-providers:operators-and-hooks-ref/index`. * - :mod:`airflow.operators.python` - :doc:`How to use ` - * - :mod:`airflow.operators.subdag` - - - * - :mod:`airflow.operators.trigger_dagrun` - diff --git a/newsfragments/41390.significant.rst b/newsfragments/41390.significant.rst new file mode 100644 index 0000000000000..37ddf0732449e --- /dev/null +++ b/newsfragments/41390.significant.rst @@ -0,0 +1,14 @@ +Support for SubDags is removed + +**Breaking Change** + +Subdags have been removed from the following locations: + +- CLI +- API +- ``SubDagOperator`` + +This removal marks the end of Subdag support across all interfaces. Users +should transition to using TaskGroups as a more efficient and maintainable +alternative. Please ensure your DAGs are updated to +remove any usage of Subdags to maintain compatibility with future Airflow releases. diff --git a/tests/always/test_example_dags.py b/tests/always/test_example_dags.py index 2b5f37631a427..f81bbf82caacd 100644 --- a/tests/always/test_example_dags.py +++ b/tests/always/test_example_dags.py @@ -33,7 +33,6 @@ AIRFLOW_SOURCES_ROOT = Path(__file__).resolve().parents[2] AIRFLOW_PROVIDERS_ROOT = AIRFLOW_SOURCES_ROOT / "airflow" / "providers" CURRENT_PYTHON_VERSION = f"{sys.version_info.major}.{sys.version_info.minor}" -NO_DB_QUERY_EXCEPTION = ("/airflow/example_dags/example_subdag_operator.py",) PROVIDERS_PREFIXES = ("airflow/providers/", "tests/system/providers/") OPTIONAL_PROVIDERS_DEPENDENCIES: dict[str, dict[str, str | None]] = { # Some examples or system tests may depend on additional packages @@ -163,10 +162,6 @@ def example_not_excluded_dags(xfail_db_exception: bool = False): if not result: param_marks.append(pytest.mark.skip(reason=reason)) - if xfail_db_exception and candidate.endswith(NO_DB_QUERY_EXCEPTION): - # Use strict XFAIL for excluded tests. So if it is not failed, we should remove from the list. - param_marks.append(pytest.mark.xfail(reason="Expected DB call", strict=True)) - if candidate.startswith(providers_folders): # Do not raise an error for airflow.exceptions.RemovedInAirflow3Warning. # We should not rush to enforce new syntax updates in providers diff --git a/tests/api_connexion/endpoints/test_dag_endpoint.py b/tests/api_connexion/endpoints/test_dag_endpoint.py index a546e6bec4751..d3192723a7077 100644 --- a/tests/api_connexion/endpoints/test_dag_endpoint.py +++ b/tests/api_connexion/endpoints/test_dag_endpoint.py @@ -195,9 +195,7 @@ def test_should_respond_200(self): "file_token": "Ii90bXAvZGFnXzEucHki.EnmIdPaUPo26lHQClbWMbDFD1Pk", "is_paused": False, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": {"__type": "CronExpression", "value": "2 2 * * *"}, "tags": [], "next_dagrun": None, @@ -238,9 +236,7 @@ def test_should_respond_200_with_schedule_interval_none(self, session): "file_token": "Ii90bXAvZGFnXzEucHki.EnmIdPaUPo26lHQClbWMbDFD1Pk", "is_paused": False, "is_active": False, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": None, "tags": [], "next_dagrun": None, @@ -371,7 +367,6 @@ def test_should_respond_200(self, url_safe_serializer): "is_active": True, "is_paused": False, "is_paused_upon_creation": None, - "is_subdag": False, "last_expired": None, "last_parsed": last_parsed, "last_parsed_time": None, @@ -395,7 +390,6 @@ def test_should_respond_200(self, url_safe_serializer): }, "pickle_id": None, "render_template_as_native_obj": False, - "root_dag_id": None, "schedule_interval": {"__type": "CronExpression", "value": "2 2 * * *"}, "scheduler_lock": None, "start_date": "2020-06-15T00:00:00+00:00", @@ -437,7 +431,6 @@ def test_should_respond_200_with_dataset_expression(self, url_safe_serializer): "is_active": True, "is_paused": False, "is_paused_upon_creation": None, - "is_subdag": False, "last_expired": None, "last_parsed": last_parsed, "last_parsed_time": None, @@ -461,7 +454,6 @@ def test_should_respond_200_with_dataset_expression(self, url_safe_serializer): }, "pickle_id": None, "render_template_as_native_obj": False, - "root_dag_id": None, "schedule_interval": {"__type": "CronExpression", "value": "2 2 * * *"}, "scheduler_lock": None, "start_date": "2020-06-15T00:00:00+00:00", @@ -498,7 +490,6 @@ def test_should_response_200_with_doc_md_none(self, url_safe_serializer): "is_active": True, "is_paused": False, "is_paused_upon_creation": None, - "is_subdag": False, "last_expired": None, "last_parsed": last_parsed, "last_parsed_time": None, @@ -515,7 +506,6 @@ def test_should_response_200_with_doc_md_none(self, url_safe_serializer): "params": {}, "pickle_id": None, "render_template_as_native_obj": False, - "root_dag_id": None, "schedule_interval": {"__type": "CronExpression", "value": "2 2 * * *"}, "scheduler_lock": None, "start_date": "2020-06-15T00:00:00+00:00", @@ -552,7 +542,6 @@ def test_should_response_200_for_null_start_date(self, url_safe_serializer): "is_active": True, "is_paused": False, "is_paused_upon_creation": None, - "is_subdag": False, "last_expired": None, "last_parsed": last_parsed, "last_parsed_time": None, @@ -569,7 +558,6 @@ def test_should_response_200_for_null_start_date(self, url_safe_serializer): "params": {}, "pickle_id": None, "render_template_as_native_obj": False, - "root_dag_id": None, "schedule_interval": {"__type": "CronExpression", "value": "2 2 * * *"}, "scheduler_lock": None, "start_date": None, @@ -609,7 +597,6 @@ def test_should_respond_200_serialized(self, url_safe_serializer): "is_active": True, "is_paused": False, "is_paused_upon_creation": None, - "is_subdag": False, "last_expired": None, "last_parsed_time": None, "last_pickled": None, @@ -632,7 +619,6 @@ def test_should_respond_200_serialized(self, url_safe_serializer): }, "pickle_id": None, "render_template_as_native_obj": False, - "root_dag_id": None, "schedule_interval": {"__type": "CronExpression", "value": "2 2 * * *"}, "scheduler_lock": None, "start_date": "2020-06-15T00:00:00+00:00", @@ -673,7 +659,6 @@ def test_should_respond_200_serialized(self, url_safe_serializer): "is_active": True, "is_paused": False, "is_paused_upon_creation": None, - "is_subdag": False, "last_expired": None, "last_parsed_time": None, "last_pickled": None, @@ -696,7 +681,6 @@ def test_should_respond_200_serialized(self, url_safe_serializer): }, "pickle_id": None, "render_template_as_native_obj": False, - "root_dag_id": None, "schedule_interval": {"__type": "CronExpression", "value": "2 2 * * *"}, "scheduler_lock": None, "start_date": "2020-06-15T00:00:00+00:00", @@ -773,7 +757,7 @@ def test_should_respond_200(self, session, url_safe_serializer): self._create_dag_models(2) self._create_deactivated_dag() - dags_query = session.query(DagModel).filter(~DagModel.is_subdag) + dags_query = session.query(DagModel) assert len(dags_query.all()) == 3 response = self.client.get("api/v1/dags", environ_overrides={"REMOTE_USER": "test"}) @@ -791,9 +775,7 @@ def test_should_respond_200(self, session, url_safe_serializer): "file_token": file_token, "is_paused": False, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -824,9 +806,7 @@ def test_should_respond_200(self, session, url_safe_serializer): "file_token": file_token2, "is_paused": False, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -869,9 +849,7 @@ def test_only_active_true_returns_active_dags(self, url_safe_serializer): "file_token": file_token, "is_paused": False, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -915,9 +893,7 @@ def test_only_active_false_returns_all_dags(self, url_safe_serializer): "file_token": file_token, "is_paused": False, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -948,9 +924,7 @@ def test_only_active_false_returns_all_dags(self, url_safe_serializer): "file_token": file_token_2, "is_paused": False, "is_active": False, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -1119,9 +1093,7 @@ def test_paused_true_returns_paused_dags(self, url_safe_serializer): "file_token": file_token, "is_paused": True, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -1164,9 +1136,7 @@ def test_paused_false_returns_unpaused_dags(self, url_safe_serializer): "file_token": file_token, "is_paused": False, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -1209,9 +1179,7 @@ def test_paused_none_returns_all_dags(self, url_safe_serializer): "file_token": file_token, "is_paused": True, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -1242,9 +1210,7 @@ def test_paused_none_returns_all_dags(self, url_safe_serializer): "file_token": file_token, "is_paused": False, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -1306,7 +1272,7 @@ def test_with_auth_role_public_set(self, set_auto_role_public, expected_status_c self._create_dag_models(2) self._create_deactivated_dag() - dags_query = session.query(DagModel).filter(~DagModel.is_subdag) + dags_query = session.query(DagModel) assert len(dags_query.all()) == 3 response = self.client.get("api/v1/dags") @@ -1333,9 +1299,7 @@ def test_should_respond_200_on_patch_is_paused(self, url_safe_serializer, sessio "file_token": file_token, "is_paused": False, "is_active": False, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -1470,9 +1434,7 @@ def test_should_respond_200_with_update_mask(self, url_safe_serializer): "file_token": file_token, "is_paused": False, "is_active": False, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -1566,7 +1528,7 @@ def test_should_respond_200_on_patch_is_paused(self, session, url_safe_serialize self._create_dag_models(2) self._create_deactivated_dag() - dags_query = session.query(DagModel).filter(~DagModel.is_subdag) + dags_query = session.query(DagModel) assert len(dags_query.all()) == 3 response = self.client.patch( @@ -1588,9 +1550,7 @@ def test_should_respond_200_on_patch_is_paused(self, session, url_safe_serialize "file_token": file_token, "is_paused": False, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -1621,9 +1581,7 @@ def test_should_respond_200_on_patch_is_paused(self, session, url_safe_serialize "file_token": file_token2, "is_paused": False, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -1657,7 +1615,7 @@ def test_should_respond_200_on_patch_is_paused_using_update_mask(self, session, self._create_dag_models(2) self._create_deactivated_dag() - dags_query = session.query(DagModel).filter(~DagModel.is_subdag) + dags_query = session.query(DagModel) assert len(dags_query.all()) == 3 response = self.client.patch( @@ -1679,9 +1637,7 @@ def test_should_respond_200_on_patch_is_paused_using_update_mask(self, session, "file_token": file_token, "is_paused": False, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -1712,9 +1668,7 @@ def test_should_respond_200_on_patch_is_paused_using_update_mask(self, session, "file_token": file_token2, "is_paused": False, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -1746,7 +1700,7 @@ def test_wrong_value_as_update_mask_rasise(self, session): self._create_dag_models(2) self._create_deactivated_dag() - dags_query = session.query(DagModel).filter(~DagModel.is_subdag) + dags_query = session.query(DagModel) assert len(dags_query.all()) == 3 response = self.client.patch( @@ -1769,7 +1723,7 @@ def test_invalid_request_body_raises_badrequest(self, session): self._create_dag_models(2) self._create_deactivated_dag() - dags_query = session.query(DagModel).filter(~DagModel.is_subdag) + dags_query = session.query(DagModel) assert len(dags_query.all()) == 3 response = self.client.patch( @@ -1810,9 +1764,7 @@ def test_only_active_true_returns_active_dags(self, url_safe_serializer, session "file_token": file_token, "is_paused": False, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -1864,9 +1816,7 @@ def test_only_active_false_returns_all_dags(self, url_safe_serializer, session): "file_token": file_token, "is_paused": False, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -1897,9 +1847,7 @@ def test_only_active_false_returns_all_dags(self, url_safe_serializer, session): "file_token": file_token_2, "is_paused": False, "is_active": False, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -2114,9 +2062,7 @@ def test_should_respond_200_and_pause_dags(self, url_safe_serializer): "file_token": file_token, "is_paused": True, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -2147,9 +2093,7 @@ def test_should_respond_200_and_pause_dags(self, url_safe_serializer): "file_token": file_token2, "is_paused": True, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -2201,9 +2145,7 @@ def test_should_respond_200_and_pause_dag_pattern(self, session, url_safe_serial "file_token": file_token, "is_paused": True, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -2234,9 +2176,7 @@ def test_should_respond_200_and_pause_dag_pattern(self, session, url_safe_serial "file_token": file_token10, "is_paused": True, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -2290,9 +2230,7 @@ def test_should_respond_200_and_reverse_ordering(self, session, url_safe_seriali "file_token": file_token10, "is_paused": False, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -2323,9 +2261,7 @@ def test_should_respond_200_and_reverse_ordering(self, session, url_safe_seriali "file_token": file_token, "is_paused": False, "is_active": True, - "is_subdag": False, "owners": [], - "root_dag_id": None, "schedule_interval": { "__type": "CronExpression", "value": "2 2 * * *", @@ -2376,7 +2312,7 @@ def test_with_auth_role_public_set( self._create_dag_models(2) self._create_deactivated_dag() - dags_query = session.query(DagModel).filter(~DagModel.is_subdag) + dags_query = session.query(DagModel) assert len(dags_query.all()) == 3 response = self.client.patch( diff --git a/tests/api_connexion/endpoints/test_dag_run_endpoint.py b/tests/api_connexion/endpoints/test_dag_run_endpoint.py index dc77648784ce5..d1e965aa0b1d2 100644 --- a/tests/api_connexion/endpoints/test_dag_run_endpoint.py +++ b/tests/api_connexion/endpoints/test_dag_run_endpoint.py @@ -147,7 +147,7 @@ def _create_dag(self, dag_id): with create_session() as session: session.add(dag_instance) dag = DAG(dag_id=dag_id, schedule=None, params={"validated_number": Param(1, minimum=1, maximum=10)}) - self.app.dag_bag.bag_dag(dag, root_dag=dag) + self.app.dag_bag.bag_dag(dag) return dag_instance def _create_test_dag_run(self, state=DagRunState.RUNNING, extra_dag=False, commit=True, idx_start=1): @@ -1690,7 +1690,7 @@ def test_should_respond_200(self, state, run_type, dag_maker, session): dag_run_id = "TEST_DAG_RUN_ID" with dag_maker(dag_id) as dag: task = EmptyOperator(task_id="task_id", dag=dag) - self.app.dag_bag.bag_dag(dag, root_dag=dag) + self.app.dag_bag.bag_dag(dag) dr = dag_maker.create_dagrun(run_id=dag_run_id, run_type=run_type) ti = dr.get_task_instance(task_id="task_id") ti.task = task @@ -1734,7 +1734,7 @@ def test_schema_validation_error_raises(self, dag_maker, session): dag_run_id = "TEST_DAG_RUN_ID" with dag_maker(dag_id) as dag: EmptyOperator(task_id="task_id", dag=dag) - self.app.dag_bag.bag_dag(dag, root_dag=dag) + self.app.dag_bag.bag_dag(dag) dag_maker.create_dagrun(run_id=dag_run_id) response = self.client.patch( @@ -1814,7 +1814,7 @@ def test_with_auth_role_public_set(self, set_auto_role_public, expected_status_c dag_run_id = "TEST_DAG_RUN_ID" with dag_maker(dag_id) as dag: task = EmptyOperator(task_id="task_id", dag=dag) - self.app.dag_bag.bag_dag(dag, root_dag=dag) + self.app.dag_bag.bag_dag(dag) dr = dag_maker.create_dagrun(run_id=dag_run_id, run_type=DagRunType.SCHEDULED) ti = dr.get_task_instance(task_id="task_id") ti.task = task @@ -1836,7 +1836,7 @@ def test_should_respond_200(self, dag_maker, session): dag_run_id = "TEST_DAG_RUN_ID" with dag_maker(dag_id) as dag: task = EmptyOperator(task_id="task_id", dag=dag) - self.app.dag_bag.bag_dag(dag, root_dag=dag) + self.app.dag_bag.bag_dag(dag) dr = dag_maker.create_dagrun(run_id=dag_run_id, state=DagRunState.FAILED) ti = dr.get_task_instance(task_id="task_id") ti.task = task @@ -1879,7 +1879,7 @@ def test_schema_validation_error_raises_for_invalid_fields(self, dag_maker, sess dag_run_id = "TEST_DAG_RUN_ID" with dag_maker(dag_id) as dag: EmptyOperator(task_id="task_id", dag=dag) - self.app.dag_bag.bag_dag(dag, root_dag=dag) + self.app.dag_bag.bag_dag(dag) dag_maker.create_dagrun(run_id=dag_run_id, state=DagRunState.FAILED) response = self.client.post( f"api/v1/dags/{dag_id}/dagRuns/{dag_run_id}/clear", @@ -1900,7 +1900,7 @@ def test_dry_run(self, dag_maker, session): dag_run_id = "TEST_DAG_RUN_ID" with dag_maker(dag_id) as dag: task = EmptyOperator(task_id="task_id", dag=dag) - self.app.dag_bag.bag_dag(dag, root_dag=dag) + self.app.dag_bag.bag_dag(dag) dr = dag_maker.create_dagrun(run_id=dag_run_id) ti = dr.get_task_instance(task_id="task_id") ti.task = task @@ -1974,7 +1974,7 @@ def test_with_auth_role_public_set(self, set_auto_role_public, expected_status_c dag_run_id = "TEST_DAG_RUN_ID" with dag_maker(dag_id) as dag: task = EmptyOperator(task_id="task_id", dag=dag) - self.app.dag_bag.bag_dag(dag, root_dag=dag) + self.app.dag_bag.bag_dag(dag) dr = dag_maker.create_dagrun(run_id=dag_run_id, run_type=DagRunType.SCHEDULED) ti = dr.get_task_instance(task_id="task_id") ti.task = task diff --git a/tests/api_connexion/endpoints/test_dag_stats_endpoint.py b/tests/api_connexion/endpoints/test_dag_stats_endpoint.py index 2e36701bd8014..36fc54d3a5b17 100644 --- a/tests/api_connexion/endpoints/test_dag_stats_endpoint.py +++ b/tests/api_connexion/endpoints/test_dag_stats_endpoint.py @@ -77,7 +77,7 @@ def _create_dag(self, dag_id): with create_session() as session: session.add(dag_instance) dag = DAG(dag_id=dag_id, schedule=None) - self.app.dag_bag.bag_dag(dag, root_dag=dag) + self.app.dag_bag.bag_dag(dag) return dag_instance def test_should_respond_200(self, session): diff --git a/tests/api_connexion/endpoints/test_log_endpoint.py b/tests/api_connexion/endpoints/test_log_endpoint.py index 19390d7f46d7e..7c4fb613e4843 100644 --- a/tests/api_connexion/endpoints/test_log_endpoint.py +++ b/tests/api_connexion/endpoints/test_log_endpoint.py @@ -92,7 +92,7 @@ def add_one(x: int): start_date=timezone.parse(self.default_time), ) - configured_app.dag_bag.bag_dag(dag, root_dag=dag) + configured_app.dag_bag.bag_dag(dag) # Add dummy dag for checking picking correct log with same task_id and different dag_id case. with dag_maker( @@ -105,7 +105,7 @@ def add_one(x: int): execution_date=timezone.parse(self.default_time), start_date=timezone.parse(self.default_time), ) - configured_app.dag_bag.bag_dag(dummy_dag, root_dag=dummy_dag) + configured_app.dag_bag.bag_dag(dummy_dag) for ti in dr.task_instances: ti.try_number = 1 @@ -286,7 +286,7 @@ def test_get_logs_of_removed_task(self, request_url, expected_filename, extra_qu dagbag = self.app.dag_bag dag = DAG(self.DAG_ID, start_date=timezone.parse(self.default_time)) del dagbag.dags[self.DAG_ID] - dagbag.bag_dag(dag=dag, root_dag=dag) + dagbag.bag_dag(dag=dag) key = self.app.config["SECRET_KEY"] serializer = URLSafeSerializer(key) diff --git a/tests/api_connexion/endpoints/test_task_instance_endpoint.py b/tests/api_connexion/endpoints/test_task_instance_endpoint.py index 6d04cbf3989e1..4fcd66affea71 100644 --- a/tests/api_connexion/endpoints/test_task_instance_endpoint.py +++ b/tests/api_connexion/endpoints/test_task_instance_endpoint.py @@ -1219,57 +1219,6 @@ class TestPostClearTaskInstances(TestTaskInstanceEndpoint): 2, id="clear by task ids", ), - pytest.param( - "example_subdag_operator", - [ - {"execution_date": DEFAULT_DATETIME_1, "state": State.FAILED}, - { - "execution_date": DEFAULT_DATETIME_1 + dt.timedelta(days=1), - "state": State.FAILED, - }, - { - "execution_date": DEFAULT_DATETIME_1 + dt.timedelta(days=2), - "state": State.FAILED, - }, - { - "execution_date": DEFAULT_DATETIME_1 + dt.timedelta(days=3), - "state": State.FAILED, - }, - { - "execution_date": DEFAULT_DATETIME_1 + dt.timedelta(days=4), - "state": State.FAILED, - }, - ], - "example_subdag_operator.section-1", - {"dry_run": True, "include_parentdag": True}, - 4, - id="include parent dag", - ), - pytest.param( - "example_subdag_operator.section-1", - [ - {"execution_date": DEFAULT_DATETIME_1, "state": State.FAILED}, - { - "execution_date": DEFAULT_DATETIME_1 + dt.timedelta(days=1), - "state": State.FAILED, - }, - { - "execution_date": DEFAULT_DATETIME_1 + dt.timedelta(days=2), - "state": State.FAILED, - }, - { - "execution_date": DEFAULT_DATETIME_1 + dt.timedelta(days=3), - "state": State.FAILED, - }, - ], - "example_subdag_operator", - { - "dry_run": True, - "include_subdags": True, - }, - 4, - id="include sub dag", - ), pytest.param( "example_python_operator", [ @@ -1321,7 +1270,7 @@ def test_clear_taskinstance_is_called_with_queued_dr_state(self, mock_clearti, s """Test that if reset_dag_runs is True, then clear_task_instances is called with State.QUEUED""" self.create_task_instances(session) dag_id = "example_python_operator" - payload = {"include_subdags": True, "reset_dag_runs": True, "dry_run": False} + payload = {"reset_dag_runs": True, "dry_run": False} self.app.dag_bag.sync_to_db() response = self.client.post( f"/api/v1/dags/{dag_id}/clearTaskInstances", @@ -1362,7 +1311,6 @@ def test_should_respond_200_with_reset_dag_run(self, session): "reset_dag_runs": True, "only_failed": False, "only_running": True, - "include_subdags": True, } task_instances = [ {"execution_date": DEFAULT_DATETIME_1, "state": State.RUNNING}, @@ -1454,7 +1402,6 @@ def test_should_respond_200_with_dag_run_id(self, session): "reset_dag_runs": False, "only_failed": False, "only_running": True, - "include_subdags": True, "dag_run_id": "TEST_DAG_RUN_ID_0", } task_instances = [ @@ -1514,7 +1461,6 @@ def test_should_respond_200_with_include_past(self, session): "only_failed": False, "include_past": True, "only_running": True, - "include_subdags": True, } task_instances = [ {"execution_date": DEFAULT_DATETIME_1, "state": State.RUNNING}, @@ -1693,7 +1639,6 @@ def test_should_respond_404_for_nonexistent_dagrun_id(self, session): "reset_dag_runs": False, "only_failed": False, "only_running": True, - "include_subdags": True, "dag_run_id": "TEST_DAG_RUN_ID_100", } task_instances = [ @@ -1732,7 +1677,6 @@ def test_should_raises_401_unauthenticated(self): "reset_dag_runs": True, "only_failed": False, "only_running": True, - "include_subdags": True, }, ) assert_401(response) @@ -1747,7 +1691,6 @@ def test_should_raise_403_forbidden(self, username: str): "reset_dag_runs": True, "only_failed": False, "only_running": True, - "include_subdags": True, }, ) assert response.status_code == 403 @@ -1794,7 +1737,6 @@ def test_raises_404_for_non_existent_dag(self): "reset_dag_runs": True, "only_failed": False, "only_running": True, - "include_subdags": True, }, ) assert response.status_code == 404 diff --git a/tests/api_connexion/schemas/test_dag_schema.py b/tests/api_connexion/schemas/test_dag_schema.py index 1e91972d1fa65..ae3a87db151e8 100644 --- a/tests/api_connexion/schemas/test_dag_schema.py +++ b/tests/api_connexion/schemas/test_dag_schema.py @@ -37,10 +37,8 @@ def test_serialize_test_dag_schema(url_safe_serializer): dag_model = DagModel( dag_id="test_dag_id", - root_dag_id="test_root_dag_id", is_paused=True, is_active=True, - is_subdag=False, fileloc="/root/airflow/dags/my_dag.py", owners="airflow1,airflow2", description="The description", @@ -57,9 +55,7 @@ def test_serialize_test_dag_schema(url_safe_serializer): "file_token": url_safe_serializer.dumps("/root/airflow/dags/my_dag.py"), "is_paused": True, "is_active": True, - "is_subdag": False, "owners": ["airflow1", "airflow2"], - "root_dag_id": "test_root_dag_id", "schedule_interval": {"__type": "CronExpression", "value": "5 4 * * *"}, "tags": [{"name": "tag-1"}, {"name": "tag-2"}], "next_dagrun": None, @@ -95,10 +91,8 @@ def test_serialize_test_dag_collection_schema(url_safe_serializer): "fileloc": "/tmp/a.py", "file_token": url_safe_serializer.dumps("/tmp/a.py"), "is_paused": None, - "is_subdag": None, "is_active": None, "owners": [], - "root_dag_id": None, "schedule_interval": None, "tags": [], "next_dagrun": None, @@ -126,9 +120,7 @@ def test_serialize_test_dag_collection_schema(url_safe_serializer): "file_token": url_safe_serializer.dumps("/tmp/a.py"), "is_active": None, "is_paused": None, - "is_subdag": None, "owners": [], - "root_dag_id": None, "schedule_interval": None, "tags": [], "next_dagrun": None, @@ -181,7 +173,6 @@ def test_serialize_test_dag_detail_schema(url_safe_serializer): "file_token": url_safe_serializer.dumps(__file__), "is_active": None, "is_paused": None, - "is_subdag": False, "orientation": "LR", "owners": [], "params": { @@ -240,7 +231,6 @@ def test_serialize_test_dag_with_dataset_schedule_detail_schema(url_safe_seriali "file_token": url_safe_serializer.dumps(__file__), "is_active": None, "is_paused": None, - "is_subdag": False, "orientation": "LR", "owners": [], "params": { diff --git a/tests/api_experimental/common/test_delete_dag.py b/tests/api_experimental/common/test_delete_dag.py index 693a534bc6dea..e7c249ee02ac5 100644 --- a/tests/api_experimental/common/test_delete_dag.py +++ b/tests/api_experimental/common/test_delete_dag.py @@ -67,10 +67,7 @@ class TestDeleteDAGSuccessfulDelete: dag_file_path = "/usr/local/airflow/dags/test_dag_8.py" key = "test_dag_id" - def setup_dag_models(self, for_sub_dag=False): - if for_sub_dag: - self.key = "test_dag_id.test_subdag" - + def setup_dag_models(self): task = EmptyOperator( task_id="dummy", dag=DAG(dag_id=self.key, default_args={"start_date": timezone.datetime(2022, 1, 1)}), @@ -79,7 +76,7 @@ def setup_dag_models(self, for_sub_dag=False): test_date = timezone.datetime(2022, 1, 1) with create_session() as session: - session.add(DagModel(dag_id=self.key, fileloc=self.dag_file_path, is_subdag=for_sub_dag)) + session.add(DagModel(dag_id=self.key, fileloc=self.dag_file_path)) dr = DR(dag_id=self.key, run_type=DagRunType.MANUAL, run_id="test", execution_date=test_date) ti = TI(task=task, state=State.SUCCESS) ti.dag_run = dr @@ -158,21 +155,16 @@ def test_delete_dag_successful_delete_not_keeping_records_in_log(self): delete_dag(dag_id=self.key, keep_records_in_log=False) self.check_dag_models_removed(expect_logs=0) - def test_delete_subdag_successful_delete(self): - self.setup_dag_models(for_sub_dag=True) - self.check_dag_models_exists() - delete_dag(dag_id=self.key, keep_records_in_log=False) - self.check_dag_models_removed(expect_logs=0) - def test_delete_dag_preserves_other_dags(self): self.setup_dag_models() with create_session() as session: session.add(DagModel(dag_id=self.key + ".other_dag", fileloc=self.dag_file_path)) - session.add(DagModel(dag_id=self.key + ".subdag", fileloc=self.dag_file_path, is_subdag=True)) + session.add(DagModel(dag_id=self.key + ".other_dag2", fileloc=self.dag_file_path)) delete_dag(self.key) with create_session() as session: assert session.query(DagModel).filter(DagModel.dag_id == self.key + ".other_dag").count() == 1 - assert session.query(DagModel).filter(DagModel.dag_id.like(self.key + "%")).count() == 1 + assert session.query(DagModel).filter(DagModel.dag_id == self.key + ".other_dag2").count() == 1 + assert session.query(DagModel).filter(DagModel.dag_id == self.key).count() == 0 diff --git a/tests/api_experimental/common/test_mark_tasks.py b/tests/api_experimental/common/test_mark_tasks.py index e90938ca69877..578fe7b04602f 100644 --- a/tests/api_experimental/common/test_mark_tasks.py +++ b/tests/api_experimental/common/test_mark_tasks.py @@ -61,7 +61,7 @@ class TestMarkTasks: @classmethod def create_dags(cls, dagbag): cls.dag1 = dagbag.get_dag("miscellaneous_test_dag") - cls.dag2 = dagbag.get_dag("example_subdag_operator") + cls.dag2 = dagbag.get_dag("example_python_operator") cls.dag3 = dagbag.get_dag("example_trigger_target_dag") cls.dag4 = dagbag.get_dag("test_mapped_classic") cls.execution_dates = [timezone.datetime(2022, 1, 1), timezone.datetime(2022, 1, 2)] @@ -408,33 +408,6 @@ def test_mark_tasks_multiple(self): self.dag1, [task.task_id for task in tasks], [self.execution_dates[0]], State.SUCCESS, snapshot ) - # TODO: this backend should be removed once a fixing solution is found later - # We skip it here because this test case is working with Postgres & SQLite - # but not with MySQL - @pytest.mark.backend("sqlite", "postgres") - def test_mark_tasks_subdag(self): - # set one task to success towards end of scheduled dag runs - snapshot = TestMarkTasks.snapshot_state(self.dag2, self.execution_dates) - task = self.dag2.get_task("section-1") - relatives = task.get_flat_relatives(upstream=False) - task_ids = [t.task_id for t in relatives] - task_ids.append(task.task_id) - dr = DagRun.find(dag_id=self.dag2.dag_id, execution_date=self.execution_dates[0])[0] - - altered = set_state( - tasks=[task], - run_id=dr.run_id, - upstream=False, - downstream=True, - future=False, - past=False, - state=State.SUCCESS, - commit=True, - ) - assert len(altered) == 14 - - self.verify_state(self.dag2, task_ids, [self.execution_dates[0]], State.SUCCESS, snapshot) - def test_mark_mapped_task_instance_state(self, session): # set mapped task instance to success mapped = self.dag4.get_task("consumer") @@ -481,7 +454,7 @@ def setup_class(cls): dagbag = models.DagBag(include_examples=True, read_dags_from_db=False) cls.dag1 = dagbag.dags["miscellaneous_test_dag"] cls.dag1.sync_to_db() - cls.dag2 = dagbag.dags["example_subdag_operator"] + cls.dag2 = dagbag.dags["example_python_operator"] cls.dag2.sync_to_db() cls.execution_dates = [ timezone.datetime(2022, 1, 1), @@ -766,14 +739,7 @@ def test_set_state_with_multiple_dagruns(self, session=None): altered = set_dag_run_state_to_success(dag=self.dag2, run_id=dr2.run_id, commit=True) - # Recursively count number of tasks in the dag - def count_dag_tasks(dag): - count = len(dag.tasks) - subdag_counts = [count_dag_tasks(subdag) for subdag in dag.subdags] - count += sum(subdag_counts) - return count - - assert len(altered) == count_dag_tasks(self.dag2) + assert len(altered) == len(self.dag2.tasks) self._verify_dag_run_state(self.dag2, self.execution_dates[1], State.SUCCESS) # Make sure other dag status are not changed diff --git a/tests/api_experimental/common/test_trigger_dag.py b/tests/api_experimental/common/test_trigger_dag.py index 8d4dc47e25a92..e65a3dad6ea14 100644 --- a/tests/api_experimental/common/test_trigger_dag.py +++ b/tests/api_experimental/common/test_trigger_dag.py @@ -55,38 +55,6 @@ def test_trigger_dag_dag_run_exist(self, dag_bag_mock, dag_run_mock): with pytest.raises(AirflowException): _trigger_dag(dag_id, dag_bag_mock) - @mock.patch("airflow.models.DAG") - @mock.patch("airflow.api.common.trigger_dag.DagRun", spec=DagRun) - @mock.patch("airflow.models.DagBag") - def test_trigger_dag_include_subdags(self, dag_bag_mock, dag_run_mock, dag_mock): - dag_id = "trigger_dag" - dag_bag_mock.dags = [dag_id] - dag_bag_mock.get_dag.return_value = dag_mock - dag_run_mock.find_duplicate.return_value = None - dag1 = mock.MagicMock(subdags=[]) - dag2 = mock.MagicMock(subdags=[]) - dag_mock.subdags = [dag1, dag2] - - triggers = _trigger_dag(dag_id, dag_bag_mock) - - assert 3 == len(triggers) - - @mock.patch("airflow.models.DAG") - @mock.patch("airflow.api.common.trigger_dag.DagRun", spec=DagRun) - @mock.patch("airflow.models.DagBag") - def test_trigger_dag_include_nested_subdags(self, dag_bag_mock, dag_run_mock, dag_mock): - dag_id = "trigger_dag" - dag_bag_mock.dags = [dag_id] - dag_bag_mock.get_dag.return_value = dag_mock - dag_run_mock.find_duplicate.return_value = None - dag1 = mock.MagicMock(subdags=[]) - dag2 = mock.MagicMock(subdags=[dag1]) - dag_mock.subdags = [dag1, dag2] - - triggers = _trigger_dag(dag_id, dag_bag_mock) - - assert 3 == len(triggers) - @mock.patch("airflow.models.DagBag") def test_trigger_dag_with_too_early_start_date(self, dag_bag_mock): dag_id = "trigger_dag_with_too_early_start_date" @@ -105,9 +73,9 @@ def test_trigger_dag_with_valid_start_date(self, dag_bag_mock): dag_bag_mock.get_dag.return_value = dag dag_bag_mock.dags_hash = {} - triggers = _trigger_dag(dag_id, dag_bag_mock, execution_date=timezone.datetime(2018, 7, 5, 10, 10, 0)) + dagrun = _trigger_dag(dag_id, dag_bag_mock, execution_date=timezone.datetime(2018, 7, 5, 10, 10, 0)) - assert len(triggers) == 1 + assert dagrun @pytest.mark.parametrize( "conf, expected_conf", @@ -126,6 +94,6 @@ def test_trigger_dag_with_conf(self, dag_bag_mock, conf, expected_conf): dag_bag_mock.dags_hash = {} - triggers = _trigger_dag(dag_id, dag_bag_mock, conf=conf) + dagrun = _trigger_dag(dag_id, dag_bag_mock, conf=conf) - assert triggers[0].conf == expected_conf + assert dagrun.conf == expected_conf diff --git a/tests/cli/commands/test_task_command.py b/tests/cli/commands/test_task_command.py index eed9c400c7c74..869d5a5a22005 100644 --- a/tests/cli/commands/test_task_command.py +++ b/tests/cli/commands/test_task_command.py @@ -692,22 +692,6 @@ def test_task_states_for_dag_run_when_dag_run_not_exists(self): ) ) - def test_subdag_clear(self): - args = self.parser.parse_args(["tasks", "clear", "example_subdag_operator", "--yes"]) - task_command.task_clear(args) - args = self.parser.parse_args( - ["tasks", "clear", "example_subdag_operator", "--yes", "--exclude-subdags"] - ) - task_command.task_clear(args) - - def test_parentdag_downstream_clear(self): - args = self.parser.parse_args(["tasks", "clear", "example_subdag_operator.section-1", "--yes"]) - task_command.task_clear(args) - args = self.parser.parse_args( - ["tasks", "clear", "example_subdag_operator.section-1", "--yes", "--exclude-parentdag"] - ) - task_command.task_clear(args) - def _set_state_and_try_num(ti, session): ti.state = TaskInstanceState.QUEUED diff --git a/tests/conftest.py b/tests/conftest.py index 0206ab9a9f714..d2d9ecf0e7f5b 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -844,6 +844,12 @@ def get_serialized_data(self): return json.loads(data) return data + def _bag_dag_compat(self, dag): + # This is a compatibility shim for the old bag_dag method in Airflow <3.0 + if hasattr(dag, "parent_dag"): + return self.dagbag.bag_dag(dag, root_dag=dag) + return self.dagbag.bag_dag(dag) + def __exit__(self, type, value, traceback): from airflow.models import DagModel from airflow.models.serialized_dag import SerializedDagModel @@ -863,10 +869,10 @@ def __exit__(self, type, value, traceback): ) self.session.merge(self.serialized_model) serialized_dag = self._serialized_dag() - self.dagbag.bag_dag(serialized_dag, root_dag=serialized_dag) + self._bag_dag_compat(serialized_dag) self.session.flush() else: - self.dagbag.bag_dag(self.dag, self.dag) + self._bag_dag_compat(self.dag) def create_dagrun(self, **kwargs): from airflow.utils import timezone diff --git a/tests/core/test_impersonation_tests.py b/tests/core/test_impersonation_tests.py index b17900820d504..721d180ce7d3f 100644 --- a/tests/core/test_impersonation_tests.py +++ b/tests/core/test_impersonation_tests.py @@ -217,11 +217,6 @@ def test_default_impersonation(self, monkeypatch): monkeypatch.setenv("AIRFLOW__CORE__DEFAULT_IMPERSONATION", TEST_USER) self.run_backfill("test_default_impersonation", "test_deelevated_user") - @pytest.mark.execution_timeout(150) - def test_impersonation_subdag(self): - """Tests that impersonation using a subdag correctly passes the right configuration.""" - self.run_backfill("impersonation_subdag", "test_subdag_operation") - class TestImpersonationWithCustomPythonPath(BaseImpersonationTest): @pytest.fixture(autouse=True) diff --git a/tests/dags/test_clear_subdag.py b/tests/dags/test_clear_subdag.py deleted file mode 100644 index bd41ead6a0dc9..0000000000000 --- a/tests/dags/test_clear_subdag.py +++ /dev/null @@ -1,72 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import datetime -import warnings - -import pytest - -from airflow.models.dag import DAG -from airflow.operators.bash import BashOperator -from airflow.operators.subdag import SubDagOperator - -pytestmark = pytest.mark.db_test - - -def create_subdag_opt(main_dag): - subdag_name = "daily_job" - subdag = DAG( - dag_id=f"{dag_name}.{subdag_name}", - start_date=start_date, - schedule=None, - max_active_tasks=2, - ) - BashOperator(bash_command="echo 1", task_id="daily_job_subdag_task", dag=subdag) - with warnings.catch_warnings(): - warnings.filterwarnings( - "ignore", - message=r"This class is deprecated\. Please use `airflow\.utils\.task_group\.TaskGroup`\.", - ) - return SubDagOperator( - task_id=subdag_name, - subdag=subdag, - dag=main_dag, - ) - - -dag_name = "clear_subdag_test_dag" - -start_date = datetime.datetime(2016, 1, 1) - -dag = DAG(dag_id=dag_name, max_active_tasks=3, start_date=start_date, schedule="0 0 * * *") - -daily_job_irrelevant = BashOperator( - bash_command="echo 1", - task_id="daily_job_irrelevant", - dag=dag, -) - -daily_job_downstream = BashOperator( - bash_command="echo 1", - task_id="daily_job_downstream", - dag=dag, -) - -daily_job = create_subdag_opt(main_dag=dag) - -daily_job >> daily_job_downstream diff --git a/tests/dags/test_impersonation_subdag.py b/tests/dags/test_impersonation_subdag.py deleted file mode 100644 index 7b006f3f96909..0000000000000 --- a/tests/dags/test_impersonation_subdag.py +++ /dev/null @@ -1,55 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import warnings - -from airflow.models.dag import DAG -from airflow.operators.bash import BashOperator -from airflow.operators.python import PythonOperator -from airflow.operators.subdag import SubDagOperator -from airflow.utils import timezone - -DEFAULT_DATE = timezone.datetime(2016, 1, 1) - -default_args = {"owner": "airflow", "start_date": DEFAULT_DATE, "run_as_user": "airflow_test_user"} - -dag = DAG(dag_id="impersonation_subdag", default_args=default_args) - - -def print_today(): - print(f"Today is {timezone.utcnow()}") - - -subdag = DAG("impersonation_subdag.test_subdag_operation", default_args=default_args) - - -PythonOperator(python_callable=print_today, task_id="exec_python_fn", dag=subdag) - - -BashOperator(task_id="exec_bash_operator", bash_command='echo "Running within SubDag"', dag=subdag) - - -with warnings.catch_warnings(): - warnings.filterwarnings( - "ignore", - message=r"This class is deprecated\. Please use `airflow\.utils\.task_group\.TaskGroup`\.", - ) - subdag_operator = SubDagOperator( - task_id="test_subdag_operation", subdag=subdag, mode="reschedule", poke_interval=1, dag=dag - ) diff --git a/tests/dags/test_subdag.py b/tests/dags/test_subdag.py deleted file mode 100644 index 9a2ebb91d395c..0000000000000 --- a/tests/dags/test_subdag.py +++ /dev/null @@ -1,86 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -""" -A DAG with subdag for testing purpose. -""" - -from __future__ import annotations - -import warnings -from datetime import datetime, timedelta - -from airflow.models.dag import DAG -from airflow.operators.empty import EmptyOperator -from airflow.operators.subdag import SubDagOperator - -DAG_NAME = "test_subdag_operator" - -DEFAULT_TASK_ARGS = { - "owner": "airflow", - "start_date": datetime(2019, 1, 1), - "max_active_runs": 1, -} - - -def subdag(parent_dag_name, child_dag_name, args): - """ - Create a subdag. - """ - dag_subdag = DAG( - dag_id=f"{parent_dag_name}.{child_dag_name}", - default_args=args, - schedule="@daily", - ) - - for i in range(2): - EmptyOperator( - task_id=f"{child_dag_name}-task-{i + 1}", - default_args=args, - dag=dag_subdag, - ) - - return dag_subdag - - -with DAG( - dag_id=DAG_NAME, - start_date=datetime(2019, 1, 1), - max_active_runs=1, - default_args=DEFAULT_TASK_ARGS, - schedule=timedelta(minutes=1), -): - start = EmptyOperator( - task_id="start", - ) - - with warnings.catch_warnings(): - warnings.filterwarnings( - "ignore", - message=r"This class is deprecated\. Please use `airflow\.utils\.task_group\.TaskGroup`\.", - ) - section_1 = SubDagOperator( - task_id="section-1", - subdag=subdag(DAG_NAME, "section-1", DEFAULT_TASK_ARGS), - default_args=DEFAULT_TASK_ARGS, - ) - - some_other_task = EmptyOperator( - task_id="some-other-task", - ) - - start >> section_1 >> some_other_task diff --git a/tests/jobs/test_backfill_job.py b/tests/jobs/test_backfill_job.py index e46f27ed62459..e18e71e7d7e4b 100644 --- a/tests/jobs/test_backfill_job.py +++ b/tests/jobs/test_backfill_job.py @@ -68,7 +68,6 @@ set_default_pool_slots, ) from tests.test_utils.mock_executor import MockExecutor -from tests.test_utils.timetables import cron_timetable pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode] @@ -1447,153 +1446,6 @@ def test_backfill_fill_blanks(self, dag_maker, mock_executor): elif ti.task_id == op5.task_id: assert ti.state == State.UPSTREAM_FAILED - def test_backfill_execute_subdag(self, mock_executor): - dag = self.dagbag.get_dag("example_subdag_operator") - subdag_op_task = dag.get_task("section-1") - - subdag = subdag_op_task.subdag - subdag.timetable = cron_timetable("@daily") - - start_date = timezone.utcnow() - executor = mock_executor - job = Job() - job_runner = BackfillJobRunner( - job=job, - dag=subdag, - start_date=start_date, - end_date=start_date, - donot_pickle=True, - ) - run_job(job=job, execute_callable=job_runner._execute) - - subdag_op_task.pre_execute(context={"execution_date": start_date}) - subdag_op_task.execute(context={"execution_date": start_date}) - subdag_op_task.post_execute(context={"execution_date": start_date}) - - history = executor.history - subdag_history = history[0] - - # check that all 5 task instances of the subdag 'section-1' were executed - assert 5 == len(subdag_history) - for sdh in subdag_history: - ti = sdh[3] - assert "section-1-task-" in ti.task_id - - with create_session() as session: - successful_subdag_runs = ( - session.query(DagRun) - .filter(DagRun.dag_id == subdag.dag_id) - .filter(DagRun.execution_date == start_date) - .filter(DagRun.state == State.SUCCESS) - .count() - ) - - assert 1 == successful_subdag_runs - - subdag.clear() - dag.clear() - - def test_subdag_clear_parentdag_downstream_clear(self, mock_executor): - dag = self.dagbag.get_dag("clear_subdag_test_dag") - subdag_op_task = dag.get_task("daily_job") - - subdag = subdag_op_task.subdag - - job = Job() - job_runner = BackfillJobRunner( - job=job, - dag=dag, - start_date=DEFAULT_DATE, - end_date=DEFAULT_DATE, - donot_pickle=True, - ) - - with timeout(seconds=30): - run_job(job=job, execute_callable=job_runner._execute) - - run_id = f"backfill__{DEFAULT_DATE.isoformat()}" - ti_subdag = TI(task=dag.get_task("daily_job"), run_id=run_id) - ti_subdag.refresh_from_db() - assert ti_subdag.state == State.SUCCESS - - ti_irrelevant = TI(task=dag.get_task("daily_job_irrelevant"), run_id=run_id) - ti_irrelevant.refresh_from_db() - assert ti_irrelevant.state == State.SUCCESS - - ti_downstream = TI(task=dag.get_task("daily_job_downstream"), run_id=run_id) - ti_downstream.refresh_from_db() - assert ti_downstream.state == State.SUCCESS - - sdag = subdag.partial_subset( - task_ids_or_regex="daily_job_subdag_task", include_downstream=True, include_upstream=False - ) - - sdag.clear(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, include_parentdag=True) - - ti_subdag.refresh_from_db() - assert State.NONE == ti_subdag.state - - ti_irrelevant.refresh_from_db() - assert State.SUCCESS == ti_irrelevant.state - - ti_downstream.refresh_from_db() - assert State.NONE == ti_downstream.state - - subdag.clear() - dag.clear() - - def test_backfill_execute_subdag_with_removed_task(self, mock_executor): - """ - Ensure that subdag operators execute properly in the case where - an associated task of the subdag has been removed from the dag - definition, but has instances in the database from previous runs. - """ - dag = self.dagbag.get_dag("example_subdag_operator") - subdag = dag.get_task("section-1").subdag - - session = settings.Session() - job = Job() - job_runner = BackfillJobRunner( - job=job, - dag=subdag, - start_date=DEFAULT_DATE, - end_date=DEFAULT_DATE, - donot_pickle=True, - ) - dr = DagRun( - dag_id=subdag.dag_id, execution_date=DEFAULT_DATE, run_id="test", run_type=DagRunType.BACKFILL_JOB - ) - session.add(dr) - - removed_task_ti = TI( - task=EmptyOperator(task_id="removed_task"), run_id=dr.run_id, state=State.REMOVED - ) - removed_task_ti.dag_id = subdag.dag_id - dr.task_instances.append(removed_task_ti) - - session.commit() - - with timeout(seconds=30): - run_job(job=job, execute_callable=job_runner._execute) - - for task in subdag.tasks: - instance = ( - session.query(TI) - .filter( - TI.dag_id == subdag.dag_id, TI.task_id == task.task_id, TI.execution_date == DEFAULT_DATE - ) - .first() - ) - - assert instance is not None - assert instance.state == State.SUCCESS - - removed_task_ti.refresh_from_db() - assert removed_task_ti.state == State.REMOVED - - subdag.clear() - dag.clear() - def test_update_counters(self, dag_maker, session): with dag_maker(dag_id="test_manage_executor_state", start_date=DEFAULT_DATE, session=session) as dag: task1 = EmptyOperator(task_id="dummy", owner="airflow") diff --git a/tests/jobs/test_local_task_job.py b/tests/jobs/test_local_task_job.py index a4e7c4f387752..91217703e2d0f 100644 --- a/tests/jobs/test_local_task_job.py +++ b/tests/jobs/test_local_task_job.py @@ -863,7 +863,7 @@ def test_fast_follow( scheduler_job = Job() scheduler_job_runner = SchedulerJobRunner(job=scheduler_job, subdir=os.devnull) - scheduler_job_runner.dagbag.bag_dag(dag, root_dag=dag) + scheduler_job_runner.dagbag.bag_dag(dag) dag_run = dag.create_dagrun(run_id="test_dagrun_fast_follow", state=State.RUNNING) diff --git a/tests/jobs/test_scheduler_job.py b/tests/jobs/test_scheduler_job.py index 2e96728d5ecae..8fdbf4826db7a 100644 --- a/tests/jobs/test_scheduler_job.py +++ b/tests/jobs/test_scheduler_job.py @@ -3097,12 +3097,12 @@ def test_scheduler_task_start_date(self, configs): dag_id = "test_task_start_date_scheduling" dag = self.dagbag.get_dag(dag_id) dag.is_paused_upon_creation = False - dagbag.bag_dag(dag=dag, root_dag=dag) + dagbag.bag_dag(dag=dag) # Deactivate other dags in this file so the scheduler doesn't waste time processing them other_dag = self.dagbag.get_dag("test_start_date_scheduling") other_dag.is_paused_upon_creation = True - dagbag.bag_dag(dag=other_dag, root_dag=other_dag) + dagbag.bag_dag(dag=other_dag) dagbag.sync_to_db() diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py index 3d39a7290d909..fcbfaa6b153cf 100644 --- a/tests/models/test_dag.py +++ b/tests/models/test_dag.py @@ -84,7 +84,6 @@ from airflow.operators.bash import BashOperator from airflow.operators.empty import EmptyOperator from airflow.operators.python import PythonOperator -from airflow.operators.subdag import SubDagOperator from airflow.security import permissions from airflow.templates import NativeEnvironment, SandboxedEnvironment from airflow.timetables.base import DagRunInfo, DataInterval, TimeRestriction, Timetable @@ -96,7 +95,7 @@ ) from airflow.utils import timezone from airflow.utils.file import list_py_file_paths -from airflow.utils.session import create_session, provide_session +from airflow.utils.session import create_session from airflow.utils.state import DagRunState, State, TaskInstanceState from airflow.utils.task_group import TaskGroup, TaskGroupContext from airflow.utils.timezone import datetime as datetime_tz @@ -292,42 +291,6 @@ def test_dag_as_context_manager(self): assert op8.dag == dag assert op9.dag == dag2 - def test_dag_topological_sort_include_subdag_tasks(self): - child_dag = DAG( - "parent_dag.child_dag", - schedule="@daily", - start_date=DEFAULT_DATE, - ) - - with child_dag: - EmptyOperator(task_id="a_child") - EmptyOperator(task_id="b_child") - - parent_dag = DAG( - "parent_dag", - schedule="@daily", - start_date=DEFAULT_DATE, - ) - - # a_parent -> child_dag -> (a_child | b_child) -> b_parent - with parent_dag: - op1 = EmptyOperator(task_id="a_parent") - with pytest.warns( - RemovedInAirflow3Warning, match="Please use `airflow.utils.task_group.TaskGroup`." - ): - op2 = SubDagOperator(task_id="child_dag", subdag=child_dag) - op3 = EmptyOperator(task_id="b_parent") - - op1 >> op2 >> op3 - - topological_list = parent_dag.topological_sort(include_subdag_tasks=True) - - assert self._occur_before("a_parent", "child_dag", topological_list) - assert self._occur_before("child_dag", "a_child", topological_list) - assert self._occur_before("child_dag", "b_child", topological_list) - assert self._occur_before("a_child", "b_parent", topological_list) - assert self._occur_before("b_child", "b_parent", topological_list) - def test_dag_topological_sort_dag_without_tasks(self): dag = DAG("dag", start_date=DEFAULT_DATE, default_args={"owner": "owner1"}) @@ -1374,16 +1337,6 @@ def test_sync_to_db(self): ) with dag: EmptyOperator(task_id="task", owner="owner1") - subdag = DAG( - "dag.subtask", - start_date=DEFAULT_DATE, - ) - # parent_dag and is_subdag was set by DagBag. We don't use DagBag, so this value is not set. - subdag.parent_dag = dag - with pytest.warns( - RemovedInAirflow3Warning, match="Please use `airflow.utils.task_group.TaskGroup`." - ): - SubDagOperator(task_id="subtask", owner="owner2", subdag=subdag) session = settings.Session() dag.sync_to_db(session=session) @@ -1393,12 +1346,6 @@ def test_sync_to_db(self): assert orm_dag.default_view is not None assert orm_dag.default_view == conf.get("webserver", "dag_default_view").lower() assert orm_dag.safe_dag_id == "dag" - - orm_subdag = session.query(DagModel).filter(DagModel.dag_id == "dag.subtask").one() - assert set(orm_subdag.owners.split(", ")) == {"owner1", "owner2"} - assert orm_subdag.is_active - assert orm_subdag.safe_dag_id == "dag__dot__subtask" - assert orm_subdag.fileloc == orm_dag.fileloc session.close() def test_sync_to_db_default_view(self): @@ -1409,17 +1356,6 @@ def test_sync_to_db_default_view(self): ) with dag: EmptyOperator(task_id="task", owner="owner1") - with pytest.warns( - RemovedInAirflow3Warning, match="Please use `airflow.utils.task_group.TaskGroup`." - ): - SubDagOperator( - task_id="subtask", - owner="owner2", - subdag=DAG( - "dag.subtask", - start_date=DEFAULT_DATE, - ), - ) session = settings.Session() dag.sync_to_db(session=session) @@ -1428,81 +1364,6 @@ def test_sync_to_db_default_view(self): assert orm_dag.default_view == "graph" session.close() - @provide_session - def test_is_paused_subdag(self, session): - subdag_id = "dag.subdag" - subdag = DAG( - subdag_id, - start_date=DEFAULT_DATE, - ) - with subdag: - EmptyOperator( - task_id="dummy_task", - ) - - dag_id = "dag" - dag = DAG( - dag_id, - start_date=DEFAULT_DATE, - ) - - with dag, pytest.warns( - RemovedInAirflow3Warning, match="Please use `airflow.utils.task_group.TaskGroup`." - ): - SubDagOperator(task_id="subdag", subdag=subdag) - - # parent_dag and is_subdag was set by DagBag. We don't use DagBag, so this value is not set. - subdag.parent_dag = dag - - session.query(DagModel).filter(DagModel.dag_id.in_([subdag_id, dag_id])).delete( - synchronize_session=False - ) - - dag.sync_to_db(session=session) - - unpaused_dags = ( - session.query(DagModel.dag_id, DagModel.is_paused) - .filter( - DagModel.dag_id.in_([subdag_id, dag_id]), - ) - .all() - ) - - assert { - (dag_id, False), - (subdag_id, False), - } == set(unpaused_dags) - - DagModel.get_dagmodel(dag.dag_id).set_is_paused(is_paused=True, including_subdags=False) - - paused_dags = ( - session.query(DagModel.dag_id, DagModel.is_paused) - .filter( - DagModel.dag_id.in_([subdag_id, dag_id]), - ) - .all() - ) - - assert { - (dag_id, True), - (subdag_id, False), - } == set(paused_dags) - - DagModel.get_dagmodel(dag.dag_id).set_is_paused(is_paused=True) - - paused_dags = ( - session.query(DagModel.dag_id, DagModel.is_paused) - .filter( - DagModel.dag_id.in_([subdag_id, dag_id]), - ) - .all() - ) - - assert { - (dag_id, True), - (subdag_id, True), - } == set(paused_dags) - def test_existing_dag_is_paused_upon_creation(self): dag = DAG("dag_paused") dag.sync_to_db() @@ -2151,8 +2012,6 @@ def test_clear_set_dagrun_state(self, dag_run_state): start_date=DEFAULT_DATE, end_date=DEFAULT_DATE + datetime.timedelta(days=1), dag_run_state=dag_run_state, - include_subdags=False, - include_parentdag=False, session=session, ) @@ -2213,8 +2072,6 @@ def consumer(value): start_date=DEFAULT_DATE, end_date=DEFAULT_DATE + datetime.timedelta(days=1), dag_run_state=dag_run_state, - include_subdags=False, - include_parentdag=False, session=session, ) session.refresh(upstream_ti) @@ -2357,96 +2214,6 @@ def check_task(): path.write_text(test_connections_string) dag.test(conn_file_path=os.fspath(path)) - def _make_test_subdag(self, session): - dag_id = "test_subdag" - self._clean_up(dag_id) - task_id = "t1" - dag = DAG(dag_id, start_date=DEFAULT_DATE, max_active_runs=1) - t_1 = EmptyOperator(task_id=task_id, dag=dag) - subdag = DAG(dag_id + ".test", start_date=DEFAULT_DATE, max_active_runs=1) - with pytest.warns( - RemovedInAirflow3Warning, - match="This class is deprecated. Please use `airflow.utils.task_group.TaskGroup`.", - ): - SubDagOperator(task_id="test", subdag=subdag, dag=dag) - t_2 = EmptyOperator(task_id="task", dag=subdag) - subdag.parent_dag = dag - - dag.sync_to_db() - - session = settings.Session() - dag.create_dagrun( - run_type=DagRunType.MANUAL, - state=State.FAILED, - start_date=DEFAULT_DATE, - execution_date=DEFAULT_DATE, - session=session, - data_interval=(DEFAULT_DATE, DEFAULT_DATE), - ) - subdag.create_dagrun( - run_type=DagRunType.MANUAL, - state=State.FAILED, - start_date=DEFAULT_DATE, - execution_date=DEFAULT_DATE, - session=session, - data_interval=(DEFAULT_DATE, DEFAULT_DATE), - ) - task_instance_1 = TI(t_1, run_id=f"manual__{DEFAULT_DATE.isoformat()}", state=State.RUNNING) - task_instance_2 = TI(t_2, run_id=f"manual__{DEFAULT_DATE.isoformat()}", state=State.RUNNING) - session.merge(task_instance_1) - session.merge(task_instance_2) - - return dag, subdag - - @pytest.mark.parametrize("dag_run_state", [DagRunState.QUEUED, DagRunState.RUNNING]) - def test_clear_set_dagrun_state_for_subdag(self, dag_run_state): - session = settings.Session() - dag, subdag = self._make_test_subdag(session) - session.flush() - - dag.clear( - start_date=DEFAULT_DATE, - end_date=DEFAULT_DATE + datetime.timedelta(days=1), - dag_run_state=dag_run_state, - include_subdags=True, - include_parentdag=False, - session=session, - ) - - dagrun = ( - session.query( - DagRun, - ) - .filter(DagRun.dag_id == subdag.dag_id) - .one() - ) - assert dagrun.state == dag_run_state - session.rollback() - - @pytest.mark.parametrize("dag_run_state", [DagRunState.QUEUED, DagRunState.RUNNING]) - def test_clear_set_dagrun_state_for_parent_dag(self, dag_run_state): - session = settings.Session() - dag, subdag = self._make_test_subdag(session) - session.flush() - - subdag.clear( - start_date=DEFAULT_DATE, - end_date=DEFAULT_DATE + datetime.timedelta(days=1), - dag_run_state=dag_run_state, - include_subdags=True, - include_parentdag=True, - session=session, - ) - - dagrun = ( - session.query( - DagRun, - ) - .filter(DagRun.dag_id == dag.dag_id) - .one() - ) - assert dagrun.state == dag_run_state - @pytest.mark.parametrize( "ti_state_begin, ti_state_end", [ @@ -2726,50 +2493,6 @@ def test_next_dagrun_after_auto_align(self): assert next_info assert next_info.logical_date == timezone.datetime(2016, 1, 1, 10, 10) - def test_next_dagrun_after_not_for_subdags(self): - """ - Test the subdags are never marked to have dagruns created, as they are - handled by the SubDagOperator, not the scheduler - """ - - def subdag(parent_dag_name, child_dag_name, args): - """ - Create a subdag. - """ - dag_subdag = DAG( - dag_id=f"{parent_dag_name}.{child_dag_name}", - schedule="@daily", - default_args=args, - ) - - for i in range(2): - EmptyOperator(task_id=f"{child_dag_name}-task-{i + 1}", dag=dag_subdag) - - return dag_subdag - - with DAG( - dag_id="test_subdag_operator", - start_date=datetime.datetime(2019, 1, 1), - max_active_runs=1, - schedule=timedelta(minutes=1), - ) as dag, pytest.warns( - RemovedInAirflow3Warning, match="Please use `airflow.utils.task_group.TaskGroup`." - ): - section_1 = SubDagOperator( - task_id="section-1", - subdag=subdag(dag.dag_id, "section-1", {"start_date": dag.start_date}), - ) - - subdag = section_1.subdag - # parent_dag and is_subdag was set by DagBag. We don't use DagBag, so this value is not set. - subdag.parent_dag = dag - - next_parent_info = dag.next_dagrun_info(None) - assert next_parent_info.logical_date == timezone.datetime(2019, 1, 1, 0, 0) - - next_subdag_info = subdag.next_dagrun_info(None) - assert next_subdag_info is None, "SubDags should never have DagRuns created by the scheduler" - def test_next_dagrun_info_on_29_feb(self): dag = DAG( "test_scheduler_dagrun_29_feb", start_date=timezone.datetime(2024, 1, 1), schedule="0 0 29 2 *" diff --git a/tests/models/test_dagbag.py b/tests/models/test_dagbag.py index 936852dd082e4..5ace985adde5d 100644 --- a/tests/models/test_dagbag.py +++ b/tests/models/test_dagbag.py @@ -36,7 +36,7 @@ import airflow.example_dags from airflow import settings -from airflow.exceptions import RemovedInAirflow3Warning, SerializationError +from airflow.exceptions import SerializationError from airflow.models.dag import DAG, DagModel from airflow.models.dagbag import DagBag from airflow.models.serialized_dag import SerializedDagModel @@ -364,14 +364,6 @@ def process_file(self, filepath, only_if_updated=True, safe_mode=True): {"example_bash_operator": "airflow/example_dags/example_bash_operator.py"}, id="example_bash_operator", ), - pytest.param( - TEST_DAGS_FOLDER / "test_subdag.py", - { - "test_subdag_operator": "dags/test_subdag.py", - "test_subdag_operator.section-1": "dags/test_subdag.py", - }, - id="test_subdag_operator", - ), ), ) def test_get_dag_registration(self, file_to_load, expected): @@ -500,161 +492,18 @@ def process_dag(self, create_dag, tmp_path): found_dags = dagbag.process_file(os.fspath(path)) return dagbag, found_dags, os.fspath(path) - def validate_dags(self, expected_parent_dag, actual_found_dags, actual_dagbag, should_be_found=True): - expected_dag_ids = [dag.dag_id for dag in expected_parent_dag.subdags] - expected_dag_ids.append(expected_parent_dag.dag_id) - + def validate_dags(self, expected_dag, actual_found_dags, actual_dagbag, should_be_found=True): actual_found_dag_ids = [dag.dag_id for dag in actual_found_dags] - - for dag_id in expected_dag_ids: - actual_dagbag.log.info("validating %s", dag_id) - assert (dag_id in actual_found_dag_ids) == should_be_found, ( - f"dag \"{dag_id}\" should {'' if should_be_found else 'not '}" - f'have been found after processing dag "{expected_parent_dag.dag_id}"' - ) - assert (dag_id in actual_dagbag.dags) == should_be_found, ( - f"dag \"{dag_id}\" should {'' if should_be_found else 'not '}" - f'be in dagbag.dags after processing dag "{expected_parent_dag.dag_id}"' - ) - - def test_load_subdags(self, tmp_path): - # Define Dag to load - def standard_subdag(): - import datetime - - from airflow.models.dag import DAG - from airflow.operators.empty import EmptyOperator - from airflow.operators.subdag import SubDagOperator - - dag_name = "parent" - default_args = {"owner": "owner1", "start_date": datetime.datetime(2016, 1, 1)} - dag = DAG(dag_name, default_args=default_args) - - # parent: - # A -> opSubDag_0 - # parent.opsubdag_0: - # -> subdag_0.task - # A -> opSubDag_1 - # parent.opsubdag_1: - # -> subdag_1.task - - with dag: - - def subdag_0(): - subdag_0 = DAG("parent.op_subdag_0", default_args=default_args) - EmptyOperator(task_id="subdag_0.task", dag=subdag_0) - return subdag_0 - - def subdag_1(): - subdag_1 = DAG("parent.op_subdag_1", default_args=default_args) - EmptyOperator(task_id="subdag_1.task", dag=subdag_1) - return subdag_1 - - op_subdag_0 = SubDagOperator(task_id="op_subdag_0", dag=dag, subdag=subdag_0()) - op_subdag_1 = SubDagOperator(task_id="op_subdag_1", dag=dag, subdag=subdag_1()) - - op_a = EmptyOperator(task_id="A") - op_a.set_downstream(op_subdag_0) - op_a.set_downstream(op_subdag_1) - return dag - - test_dag = standard_subdag() - # coherence check to make sure DAG.subdag is still functioning properly - assert len(test_dag.subdags) == 2 - - # Perform processing dag - dagbag, found_dags, _ = self.process_dag(standard_subdag, tmp_path) - - # Validate correctness - # all dags from test_dag should be listed - self.validate_dags(test_dag, found_dags, dagbag) - - # Define Dag to load - def nested_subdags(): - import datetime - - from airflow.models.dag import DAG - from airflow.operators.empty import EmptyOperator - from airflow.operators.subdag import SubDagOperator - - dag_name = "parent" - default_args = {"owner": "owner1", "start_date": datetime.datetime(2016, 1, 1)} - dag = DAG(dag_name, default_args=default_args) - - # parent: - # A -> op_subdag_0 - # parent.op_subdag_0: - # -> opSubDag_A - # parent.op_subdag_0.opSubdag_A: - # -> subdag_a.task - # -> opSubdag_B - # parent.op_subdag_0.opSubdag_B: - # -> subdag_b.task - # A -> op_subdag_1 - # parent.op_subdag_1: - # -> opSubdag_C - # parent.op_subdag_1.opSubdag_C: - # -> subdag_c.task - # -> opSubDag_D - # parent.op_subdag_1.opSubdag_D: - # -> subdag_d.task - - with dag: - - def subdag_a(): - subdag_a = DAG("parent.op_subdag_0.opSubdag_A", default_args=default_args) - EmptyOperator(task_id="subdag_a.task", dag=subdag_a) - return subdag_a - - def subdag_b(): - subdag_b = DAG("parent.op_subdag_0.opSubdag_B", default_args=default_args) - EmptyOperator(task_id="subdag_b.task", dag=subdag_b) - return subdag_b - - def subdag_c(): - subdag_c = DAG("parent.op_subdag_1.opSubdag_C", default_args=default_args) - EmptyOperator(task_id="subdag_c.task", dag=subdag_c) - return subdag_c - - def subdag_d(): - subdag_d = DAG("parent.op_subdag_1.opSubdag_D", default_args=default_args) - EmptyOperator(task_id="subdag_d.task", dag=subdag_d) - return subdag_d - - def subdag_0(): - subdag_0 = DAG("parent.op_subdag_0", default_args=default_args) - SubDagOperator(task_id="opSubdag_A", dag=subdag_0, subdag=subdag_a()) - SubDagOperator(task_id="opSubdag_B", dag=subdag_0, subdag=subdag_b()) - return subdag_0 - - def subdag_1(): - subdag_1 = DAG("parent.op_subdag_1", default_args=default_args) - SubDagOperator(task_id="opSubdag_C", dag=subdag_1, subdag=subdag_c()) - SubDagOperator(task_id="opSubdag_D", dag=subdag_1, subdag=subdag_d()) - return subdag_1 - - op_subdag_0 = SubDagOperator(task_id="op_subdag_0", dag=dag, subdag=subdag_0()) - op_subdag_1 = SubDagOperator(task_id="op_subdag_1", dag=dag, subdag=subdag_1()) - - op_a = EmptyOperator(task_id="A") - op_a.set_downstream(op_subdag_0) - op_a.set_downstream(op_subdag_1) - - return dag - - test_dag = nested_subdags() - # coherence check to make sure DAG.subdag is still functioning properly - assert len(test_dag.subdags) == 6 - - # Perform processing dag - dagbag, found_dags, filename = self.process_dag(nested_subdags, tmp_path) - - # Validate correctness - # all dags from test_dag should be listed - self.validate_dags(test_dag, found_dags, dagbag) - - for dag in dagbag.dags.values(): - assert dag.fileloc == filename + dag_id = expected_dag.dag_id + actual_dagbag.log.info("validating %s", dag_id) + assert (dag_id in actual_found_dag_ids) == should_be_found, ( + f"dag \"{dag_id}\" should {'' if should_be_found else 'not '}" + f'have been found after processing dag "{expected_dag.dag_id}"' + ) + assert (dag_id in actual_dagbag.dags) == should_be_found, ( + f"dag \"{dag_id}\" should {'' if should_be_found else 'not '}" + f'be in dagbag.dags after processing dag "{expected_dag.dag_id}"' + ) def test_skip_cycle_dags(self, tmp_path): """ @@ -681,8 +530,6 @@ def basic_cycle(): return dag test_dag = basic_cycle() - # coherence check to make sure DAG.subdag is still functioning properly - assert len(test_dag.subdags) == 0 # Perform processing dag dagbag, found_dags, file_path = self.process_dag(basic_cycle, tmp_path) @@ -692,97 +539,6 @@ def basic_cycle(): self.validate_dags(test_dag, found_dags, dagbag, should_be_found=False) assert file_path in dagbag.import_errors - # Define Dag to load - def nested_subdag_cycle(): - import datetime - - from airflow.models.dag import DAG - from airflow.operators.empty import EmptyOperator - from airflow.operators.subdag import SubDagOperator - - dag_name = "nested_cycle" - default_args = {"owner": "owner1", "start_date": datetime.datetime(2016, 1, 1)} - dag = DAG(dag_name, default_args=default_args) - - # cycle: - # A -> op_subdag_0 - # cycle.op_subdag_0: - # -> opSubDag_A - # cycle.op_subdag_0.opSubdag_A: - # -> subdag_a.task - # -> opSubdag_B - # cycle.op_subdag_0.opSubdag_B: - # -> subdag_b.task - # A -> op_subdag_1 - # cycle.op_subdag_1: - # -> opSubdag_C - # cycle.op_subdag_1.opSubdag_C: - # -> subdag_c.task -> subdag_c.task >Invalid Loop< - # -> opSubDag_D - # cycle.op_subdag_1.opSubdag_D: - # -> subdag_d.task - - with dag: - - def subdag_a(): - subdag_a = DAG("nested_cycle.op_subdag_0.opSubdag_A", default_args=default_args) - EmptyOperator(task_id="subdag_a.task", dag=subdag_a) - return subdag_a - - def subdag_b(): - subdag_b = DAG("nested_cycle.op_subdag_0.opSubdag_B", default_args=default_args) - EmptyOperator(task_id="subdag_b.task", dag=subdag_b) - return subdag_b - - def subdag_c(): - subdag_c = DAG("nested_cycle.op_subdag_1.opSubdag_C", default_args=default_args) - op_subdag_c_task = EmptyOperator(task_id="subdag_c.task", dag=subdag_c) - # introduce a loop in opSubdag_C - op_subdag_c_task.set_downstream(op_subdag_c_task) - return subdag_c - - def subdag_d(): - subdag_d = DAG("nested_cycle.op_subdag_1.opSubdag_D", default_args=default_args) - EmptyOperator(task_id="subdag_d.task", dag=subdag_d) - return subdag_d - - def subdag_0(): - subdag_0 = DAG("nested_cycle.op_subdag_0", default_args=default_args) - SubDagOperator(task_id="opSubdag_A", dag=subdag_0, subdag=subdag_a()) - SubDagOperator(task_id="opSubdag_B", dag=subdag_0, subdag=subdag_b()) - return subdag_0 - - def subdag_1(): - subdag_1 = DAG("nested_cycle.op_subdag_1", default_args=default_args) - SubDagOperator(task_id="opSubdag_C", dag=subdag_1, subdag=subdag_c()) - SubDagOperator(task_id="opSubdag_D", dag=subdag_1, subdag=subdag_d()) - return subdag_1 - - op_subdag_0 = SubDagOperator(task_id="op_subdag_0", dag=dag, subdag=subdag_0()) - op_subdag_1 = SubDagOperator(task_id="op_subdag_1", dag=dag, subdag=subdag_1()) - - op_a = EmptyOperator(task_id="A") - op_a.set_downstream(op_subdag_0) - op_a.set_downstream(op_subdag_1) - - return dag - - with pytest.warns( - RemovedInAirflow3Warning, - match="This class is deprecated. Please use `airflow.utils.task_group.TaskGroup`.", - ): - test_dag = nested_subdag_cycle() - # coherence check to make sure DAG.subdag is still functioning properly - assert len(test_dag.subdags) == 6 - - # Perform processing dag - dagbag, found_dags, file_path = self.process_dag(nested_subdag_cycle, tmp_path) - - # Validate correctness - # None of the dags should be found - self.validate_dags(test_dag, found_dags, dagbag, should_be_found=False) - assert file_path in dagbag.import_errors - def test_process_file_with_none(self, tmp_path): """ test that process_file can handle Nones @@ -870,7 +626,6 @@ def test_sync_to_db_is_retried(self, mock_bulk_write_to_db, mock_s10n_write_dag, dagbag = DagBag("/dev/null") mock_dag = mock.MagicMock(spec=DAG) - mock_dag.is_subdag = False dagbag.dags["mock_dag"] = mock_dag op_error = OperationalError(statement=mock.ANY, params=mock.ANY, orig=mock.ANY) diff --git a/tests/models/test_dagcode.py b/tests/models/test_dagcode.py index 1d30c94863c8a..e566e9ceed080 100644 --- a/tests/models/test_dagcode.py +++ b/tests/models/test_dagcode.py @@ -105,8 +105,6 @@ def test_detecting_duplicate_key(self, mock_hash): def _compare_example_dags(self, example_dags): with create_session() as session: for dag in example_dags.values(): - if dag.is_subdag: - dag.fileloc = dag.parent_dag.fileloc assert DagCode.has_dag(dag.fileloc) dag_fileloc_hash = DagCode.dag_fileloc_hash(dag.fileloc) result = ( diff --git a/tests/models/test_serialized_dag.py b/tests/models/test_serialized_dag.py index 531ffb031925e..83da1916378df 100644 --- a/tests/models/test_serialized_dag.py +++ b/tests/models/test_serialized_dag.py @@ -164,11 +164,8 @@ def test_read_dags(self): def test_remove_dags_by_id(self): """DAGs can be removed from database.""" example_dags_list = list(self._write_example_dags().values()) - # Remove SubDags from the list as they are not stored in DB in a separate row - # and are directly added in Json blob of the main DAG - filtered_example_dags_list = [dag for dag in example_dags_list if not dag.is_subdag] # Tests removing by dag_id. - dag_removed_by_id = filtered_example_dags_list[0] + dag_removed_by_id = example_dags_list[0] SDM.remove_dag(dag_removed_by_id.dag_id) assert not SDM.has_dag(dag_removed_by_id.dag_id) @@ -176,13 +173,10 @@ def test_remove_dags_by_id(self): def test_remove_dags_by_filepath(self): """DAGs can be removed from database.""" example_dags_list = list(self._write_example_dags().values()) - # Remove SubDags from the list as they are not stored in DB in a separate row - # and are directly added in Json blob of the main DAG - filtered_example_dags_list = [dag for dag in example_dags_list if not dag.is_subdag] # Tests removing by file path. - dag_removed_by_file = filtered_example_dags_list[0] + dag_removed_by_file = example_dags_list[0] # remove repeated files for those DAGs that define multiple dags in the same file (set comprehension) - example_dag_files = list({dag.fileloc for dag in filtered_example_dags_list}) + example_dag_files = list({dag.fileloc for dag in example_dags_list}) example_dag_files.remove(dag_removed_by_file.fileloc) SDM.remove_deleted_dags(example_dag_files, processor_subdir="/tmp/test") assert not SDM.has_dag(dag_removed_by_file.dag_id) diff --git a/tests/operators/test_subdag_operator.py b/tests/operators/test_subdag_operator.py deleted file mode 100644 index ca669a9e4e8cf..0000000000000 --- a/tests/operators/test_subdag_operator.py +++ /dev/null @@ -1,376 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -from unittest import mock -from unittest.mock import Mock - -import pytest - -import airflow -from airflow.exceptions import AirflowException, RemovedInAirflow3Warning -from airflow.models.dag import DAG -from airflow.models.dagrun import DagRun -from airflow.models.taskinstance import TaskInstance -from airflow.operators.empty import EmptyOperator -from airflow.operators.subdag import SkippedStatePropagationOptions, SubDagOperator -from airflow.utils.session import create_session -from airflow.utils.state import State -from airflow.utils.timezone import datetime -from airflow.utils.types import DagRunType -from tests.test_utils.db import clear_db_runs - -pytestmark = pytest.mark.db_test - -DEFAULT_DATE = datetime(2016, 1, 1) - -default_args = {"start_date": DEFAULT_DATE} - -WARNING_MESSAGE = """This class is deprecated. Please use `airflow.utils.task_group.TaskGroup`.""" - - -class TestSubDagOperator: - def setup_method(self): - clear_db_runs() - self.dag_run_running = DagRun() - self.dag_run_running.state = State.RUNNING - self.dag_run_success = DagRun() - self.dag_run_success.state = State.SUCCESS - self.dag_run_failed = DagRun() - self.dag_run_failed.state = State.FAILED - - def teardown_class(self): - clear_db_runs() - - def test_subdag_name(self): - """ - Subdag names must be {parent_dag}.{subdag task} - """ - dag = DAG("parent", default_args=default_args) - subdag_good = DAG("parent.test", default_args=default_args) - subdag_bad1 = DAG("parent.bad", default_args=default_args) - subdag_bad2 = DAG("bad.test", default_args=default_args) - subdag_bad3 = DAG("bad.bad", default_args=default_args) - - with pytest.warns(RemovedInAirflow3Warning, match=WARNING_MESSAGE): - SubDagOperator(task_id="test", dag=dag, subdag=subdag_good) - with pytest.raises(AirflowException): - SubDagOperator(task_id="test", dag=dag, subdag=subdag_bad1) - with pytest.raises(AirflowException): - SubDagOperator(task_id="test", dag=dag, subdag=subdag_bad2) - with pytest.raises(AirflowException): - SubDagOperator(task_id="test", dag=dag, subdag=subdag_bad3) - - def test_subdag_in_context_manager(self): - """ - Creating a sub DAG within a main DAG's context manager - """ - with DAG("parent", default_args=default_args) as dag: - subdag = DAG("parent.test", default_args=default_args) - with pytest.warns(RemovedInAirflow3Warning, match=WARNING_MESSAGE): - op = SubDagOperator(task_id="test", subdag=subdag) - - assert op.dag == dag - assert op.subdag == subdag - - def test_subdag_pools(self, dag_maker): - """ - Subdags and subdag tasks can't both have a pool with 1 slot - """ - with dag_maker("parent", default_args=default_args, serialized=True) as dag: - pass - - pool_1 = airflow.models.Pool(pool="test_pool_1", slots=1, include_deferred=False) - pool_10 = airflow.models.Pool(pool="test_pool_10", slots=10, include_deferred=False) - dag_maker.session.add(pool_1) - dag_maker.session.add(pool_10) - dag_maker.session.commit() - - with dag_maker("parent.child", default_args=default_args, serialized=True) as subdag: - EmptyOperator(task_id="dummy", pool="test_pool_1") - - with pytest.raises(AirflowException): - SubDagOperator(task_id="child", dag=dag, subdag=subdag, pool="test_pool_1") - - # recreate dag because failed subdagoperator was already added - dag = DAG("parent", default_args=default_args) - with pytest.warns(RemovedInAirflow3Warning, match=WARNING_MESSAGE): - SubDagOperator(task_id="child", dag=dag, subdag=subdag, pool="test_pool_10") - - dag_maker.session.delete(pool_1) - dag_maker.session.delete(pool_10) - dag_maker.session.commit() - - def test_subdag_pools_no_possible_conflict(self): - """ - Subdags and subdag tasks with no pool overlap, should not to query - pools - """ - dag = DAG("parent", default_args=default_args) - subdag = DAG("parent.child", default_args=default_args) - - session = airflow.settings.Session() - pool_1 = airflow.models.Pool(pool="test_pool_1", slots=1, include_deferred=False) - pool_10 = airflow.models.Pool(pool="test_pool_10", slots=10, include_deferred=False) - session.add(pool_1) - session.add(pool_10) - session.commit() - - EmptyOperator(task_id="dummy", dag=subdag, pool="test_pool_10") - - mock_session = Mock() - with pytest.warns(RemovedInAirflow3Warning, match=WARNING_MESSAGE): - SubDagOperator(task_id="child", dag=dag, subdag=subdag, pool="test_pool_1", session=mock_session) - assert not mock_session.query.called - - session.delete(pool_1) - session.delete(pool_10) - session.commit() - - def test_execute_create_dagrun_wait_until_success(self): - """ - When SubDagOperator executes, it creates a DagRun if there is no existing one - and wait until the DagRun succeeds. - """ - dag = DAG("parent", default_args=default_args) - subdag = DAG("parent.test", default_args=default_args) - with pytest.warns(RemovedInAirflow3Warning, match=WARNING_MESSAGE): - subdag_task = SubDagOperator(task_id="test", subdag=subdag, dag=dag, poke_interval=1) - - subdag.create_dagrun = Mock() - subdag.create_dagrun.return_value = self.dag_run_running - - subdag_task._get_dagrun = Mock() - subdag_task._get_dagrun.side_effect = [None, self.dag_run_success, self.dag_run_success] - - context = { - "data_interval_start": None, - "data_interval_end": None, - "execution_date": DEFAULT_DATE, - } - - subdag_task.pre_execute(context=context) - subdag_task.execute(context=context) - subdag_task.post_execute(context=context) - - subdag.create_dagrun.assert_called_once_with( - run_type=DagRunType.SCHEDULED, - execution_date=DEFAULT_DATE, - data_interval=None, - conf=None, - state=State.RUNNING, - external_trigger=True, - ) - - assert 3 == subdag_task._get_dagrun.call_count - - def test_execute_create_dagrun_with_conf(self): - """ - When SubDagOperator executes, it creates a DagRun if there is no existing one - and wait until the DagRun succeeds. - """ - conf = {"key": "value"} - dag = DAG("parent", default_args=default_args) - subdag = DAG("parent.test", default_args=default_args) - with pytest.warns(RemovedInAirflow3Warning, match=WARNING_MESSAGE): - subdag_task = SubDagOperator(task_id="test", subdag=subdag, dag=dag, poke_interval=1, conf=conf) - - subdag.create_dagrun = Mock() - subdag.create_dagrun.return_value = self.dag_run_running - - subdag_task._get_dagrun = Mock() - subdag_task._get_dagrun.side_effect = [None, self.dag_run_success, self.dag_run_success] - - context = { - "data_interval_start": None, - "data_interval_end": None, - "execution_date": DEFAULT_DATE, - } - - subdag_task.pre_execute(context=context) - subdag_task.execute(context=context) - subdag_task.post_execute(context=context) - - subdag.create_dagrun.assert_called_once_with( - run_type=DagRunType.SCHEDULED, - execution_date=DEFAULT_DATE, - data_interval=None, - conf=conf, - state=State.RUNNING, - external_trigger=True, - ) - - assert 3 == subdag_task._get_dagrun.call_count - - def test_execute_dagrun_failed(self): - """ - When the DagRun failed during the execution, it raises an Airflow Exception. - """ - dag = DAG("parent", default_args=default_args) - subdag = DAG("parent.test", default_args=default_args) - with pytest.warns(RemovedInAirflow3Warning, match=WARNING_MESSAGE): - subdag_task = SubDagOperator(task_id="test", subdag=subdag, dag=dag, poke_interval=1) - - subdag.create_dagrun = Mock() - subdag.create_dagrun.return_value = self.dag_run_running - - subdag_task._get_dagrun = Mock() - subdag_task._get_dagrun.side_effect = [None, self.dag_run_failed, self.dag_run_failed] - - context = { - "data_interval_start": None, - "data_interval_end": None, - "execution_date": DEFAULT_DATE, - } - - subdag_task.pre_execute(context=context) - subdag_task.execute(context=context) - with pytest.raises(AirflowException): - subdag_task.post_execute(context=context) - - def test_execute_skip_if_dagrun_success(self): - """ - When there is an existing DagRun in SUCCESS state, skip the execution. - """ - dag = DAG("parent", default_args=default_args) - subdag = DAG("parent.test", default_args=default_args) - - subdag.create_dagrun = Mock() - with pytest.warns(RemovedInAirflow3Warning, match=WARNING_MESSAGE): - subdag_task = SubDagOperator(task_id="test", subdag=subdag, dag=dag, poke_interval=1) - subdag_task._get_dagrun = Mock() - subdag_task._get_dagrun.return_value = self.dag_run_success - - context = { - "data_interval_start": None, - "data_interval_end": None, - "execution_date": DEFAULT_DATE, - } - - subdag_task.pre_execute(context=context) - subdag_task.execute(context=context) - subdag_task.post_execute(context=context) - - subdag.create_dagrun.assert_not_called() - assert 3 == subdag_task._get_dagrun.call_count - - @pytest.mark.skip_if_database_isolation_mode # this uses functions which operate directly on DB - def test_rerun_failed_subdag(self, dag_maker): - """ - When there is an existing DagRun with failed state, reset the DagRun and the - corresponding TaskInstances - """ - with create_session() as session: - with dag_maker("parent.test", default_args=default_args, session=session) as subdag: - dummy_task = EmptyOperator(task_id="dummy") - sub_dagrun = dag_maker.create_dagrun( - run_type=DagRunType.SCHEDULED, - execution_date=DEFAULT_DATE, - state=State.FAILED, - external_trigger=True, - ) - - (dummy_task_instance,) = sub_dagrun.task_instances - dummy_task_instance.refresh_from_task(dummy_task) - dummy_task_instance.state == State.FAILED - - with dag_maker("parent", default_args=default_args, session=session), pytest.warns( - RemovedInAirflow3Warning, match=WARNING_MESSAGE - ): - subdag_task = SubDagOperator(task_id="test", subdag=subdag, poke_interval=1) - dag_maker.create_dagrun(execution_date=DEFAULT_DATE, run_type=DagRunType.SCHEDULED) - - subdag_task._reset_dag_run_and_task_instances(sub_dagrun, execution_date=DEFAULT_DATE) - - dummy_task_instance.refresh_from_db() - assert dummy_task_instance.state == State.NONE - - sub_dagrun.refresh_from_db() - assert sub_dagrun.state == State.RUNNING - - @pytest.mark.parametrize( - "propagate_option, states, skip_parent", - [ - (SkippedStatePropagationOptions.ALL_LEAVES, [State.SKIPPED, State.SKIPPED], True), - (SkippedStatePropagationOptions.ALL_LEAVES, [State.SKIPPED, State.SUCCESS], False), - (SkippedStatePropagationOptions.ANY_LEAF, [State.SKIPPED, State.SUCCESS], True), - (SkippedStatePropagationOptions.ANY_LEAF, [State.FAILED, State.SKIPPED], True), - (None, [State.SKIPPED, State.SKIPPED], False), - ], - ) - @mock.patch("airflow.operators.subdag.SubDagOperator.skip") - @mock.patch("airflow.operators.subdag.get_task_instance") - def test_subdag_with_propagate_skipped_state( - self, - mock_get_task_instance, - mock_skip, - dag_maker, - propagate_option, - states, - skip_parent, - ): - """ - Tests that skipped state of leaf tasks propagates to the parent dag. - Note that the skipped state propagation only takes affect when the dagrun's state is SUCCESS. - """ - with dag_maker("parent.test", default_args=default_args) as subdag: - dummy_subdag_tasks = [EmptyOperator(task_id=f"dummy_subdag_{i}") for i in range(len(states))] - dag_maker.create_dagrun(execution_date=DEFAULT_DATE) - - with dag_maker("parent", default_args=default_args): - with pytest.warns(RemovedInAirflow3Warning, match=WARNING_MESSAGE): - subdag_task = SubDagOperator( - task_id="test", - subdag=subdag, - poke_interval=1, - propagate_skipped_state=propagate_option, - ) - dummy_dag_task = EmptyOperator(task_id="dummy_dag") - subdag_task >> dummy_dag_task - dag_run = dag_maker.create_dagrun(execution_date=DEFAULT_DATE) - - subdag_task._get_dagrun = Mock(return_value=self.dag_run_success) - - mock_get_task_instance.side_effect = [ - TaskInstance(task=task, run_id=dag_run.run_id, state=state) - for task, state in zip(dummy_subdag_tasks, states) - ] - - context = { - "execution_date": DEFAULT_DATE, - "dag_run": dag_run, - "task": subdag_task, - "ti": mock.MagicMock(map_index=-1), - } - subdag_task.post_execute(context) - - if skip_parent: - mock_skip.assert_called_once_with( - context["dag_run"], context["execution_date"], [dummy_dag_task], map_index=-1 - ) - else: - mock_skip.assert_not_called() - - def test_deprecation_warning(self): - dag = DAG("parent", default_args=default_args) - subdag = DAG("parent.test", default_args=default_args) - warning_message = """This class is deprecated. Please use `airflow.utils.task_group.TaskGroup`.""" - - with pytest.warns(DeprecationWarning) as warnings: - SubDagOperator(task_id="test", subdag=subdag, dag=dag) - assert warning_message == str(warnings[0].message) diff --git a/tests/operators/test_trigger_dagrun.py b/tests/operators/test_trigger_dagrun.py index 349bba463800f..9ec22e7e7a3de 100644 --- a/tests/operators/test_trigger_dagrun.py +++ b/tests/operators/test_trigger_dagrun.py @@ -76,7 +76,7 @@ def setup_method(self): def re_sync_triggered_dag_to_db(self, dag, dag_maker): TracebackSessionForTests.set_allow_db_access(dag_maker.session, True) dagbag = DagBag(self.f_name, read_dags_from_db=False, include_examples=False) - dagbag.bag_dag(dag, root_dag=dag) + dagbag.bag_dag(dag) dagbag.sync_to_db(session=dag_maker.session) TracebackSessionForTests.set_allow_db_access(dag_maker.session, False) diff --git a/tests/providers/fab/auth_manager/test_security.py b/tests/providers/fab/auth_manager/test_security.py index 5ff7f34d018c0..6dd48cc6e6e29 100644 --- a/tests/providers/fab/auth_manager/test_security.py +++ b/tests/providers/fab/auth_manager/test_security.py @@ -1008,46 +1008,6 @@ def test_prefixed_dag_id_is_deprecated(security_manager): security_manager.prefixed_dag_id("hello") -def test_parent_dag_access_applies_to_subdag(app, security_manager, assert_user_has_dag_perms, session): - username = "dag_permission_user" - role_name = "dag_permission_role" - parent_dag_name = "parent_dag" - subdag_name = parent_dag_name + ".subdag" - subsubdag_name = parent_dag_name + ".subdag.subsubdag" - with app.app_context(): - mock_roles = [ - { - "role": role_name, - "perms": [ - (permissions.ACTION_CAN_READ, f"DAG:{parent_dag_name}"), - (permissions.ACTION_CAN_EDIT, f"DAG:{parent_dag_name}"), - ], - } - ] - with create_user_scope( - app, - username=username, - role_name=role_name, - ) as user: - dag1 = DagModel(dag_id=parent_dag_name) - dag2 = DagModel(dag_id=subdag_name, is_subdag=True, root_dag_id=parent_dag_name) - dag3 = DagModel(dag_id=subsubdag_name, is_subdag=True, root_dag_id=parent_dag_name) - session.add_all([dag1, dag2, dag3]) - session.commit() - security_manager.bulk_sync_roles(mock_roles) - for _ in [dag1, dag2, dag3]: - security_manager._sync_dag_view_permissions( - parent_dag_name, access_control={role_name: READ_WRITE} - ) - - assert_user_has_dag_perms(perms=["GET", "PUT"], dag_id=parent_dag_name, user=user) - assert_user_has_dag_perms(perms=["GET", "PUT"], dag_id=parent_dag_name + ".subdag", user=user) - assert_user_has_dag_perms( - perms=["GET", "PUT"], dag_id=parent_dag_name + ".subdag.subsubdag", user=user - ) - session.query(DagModel).delete() - - def test_permissions_work_for_dags_with_dot_in_dagname( app, security_manager, assert_user_has_dag_perms, assert_user_does_not_have_dag_perms, session ): diff --git a/tests/sensors/test_external_task_sensor.py b/tests/sensors/test_external_task_sensor.py index 8c5ed3c42920b..91b224a0de44d 100644 --- a/tests/sensors/test_external_task_sensor.py +++ b/tests/sensors/test_external_task_sensor.py @@ -1205,7 +1205,7 @@ def dag_bag_ext(): task_a_3 >> task_b_3 for dag in [dag_0, dag_1, dag_2, dag_3]: - dag_bag.bag_dag(dag=dag, root_dag=dag) + dag_bag.bag_dag(dag=dag) yield dag_bag @@ -1254,7 +1254,7 @@ def dag_bag_parent_child(): ) for dag in [dag_0, dag_1]: - dag_bag.bag_dag(dag=dag, root_dag=dag) + dag_bag.bag_dag(dag=dag) yield dag_bag @@ -1479,7 +1479,7 @@ def _factory(depth: int) -> DagBag: task_a >> task_b for dag in dags: - dag_bag.bag_dag(dag=dag, root_dag=dag) + dag_bag.bag_dag(dag=dag) return dag_bag @@ -1531,8 +1531,8 @@ def dag_bag_multiple(): dag_bag = DagBag(dag_folder=DEV_NULL, include_examples=False) daily_dag = DAG("daily_dag", start_date=DEFAULT_DATE, schedule="@daily") agg_dag = DAG("agg_dag", start_date=DEFAULT_DATE, schedule="@daily") - dag_bag.bag_dag(dag=daily_dag, root_dag=daily_dag) - dag_bag.bag_dag(dag=agg_dag, root_dag=agg_dag) + dag_bag.bag_dag(dag=daily_dag) + dag_bag.bag_dag(dag=agg_dag) daily_task = EmptyOperator(task_id="daily_tas", dag=daily_dag) @@ -1603,7 +1603,7 @@ def dag_bag_head_tail(): ) head >> body >> tail - dag_bag.bag_dag(dag=dag, root_dag=dag) + dag_bag.bag_dag(dag=dag) return dag_bag @@ -1687,7 +1687,7 @@ def fake_task(x: int): ) head >> body >> tail - dag_bag.bag_dag(dag=dag, root_dag=dag) + dag_bag.bag_dag(dag=dag) return dag_bag diff --git a/tests/serialization/test_dag_serialization.py b/tests/serialization/test_dag_serialization.py index e9c8ceaf03979..8ef09349275ea 100644 --- a/tests/serialization/test_dag_serialization.py +++ b/tests/serialization/test_dag_serialization.py @@ -364,9 +364,6 @@ def collect_dags(dag_folder=None): if any([directory.startswith(excluded_pattern) for excluded_pattern in excluded_patterns]): continue dags.update(make_example_dags(directory)) - - # Filter subdags as they are stored in same row in Serialized Dag table - dags = {dag_id: dag for dag_id, dag in dags.items() if not dag.is_subdag} return dags @@ -640,7 +637,6 @@ def validate_deserialized_task( # Checked separately "_task_type", "_operator_name", - "subdag", # Type is excluded, so don't check it "_log", # List vs tuple. Check separately @@ -714,14 +710,6 @@ def validate_deserialized_task( original_partial_kwargs = {**default_partial_kwargs, **task.partial_kwargs} assert serialized_partial_kwargs == original_partial_kwargs - # Check that for Deserialized task, task.subdag is None for all other Operators - # except for the SubDagOperator where task.subdag is an instance of DAG object - if task.task_type == "SubDagOperator": - assert serialized_task.subdag is not None - assert isinstance(serialized_task.subdag, DAG) - else: - assert serialized_task.subdag is None - @pytest.mark.parametrize( "dag_start_date, task_start_date, expected_task_start_date", [ @@ -1254,7 +1242,6 @@ def test_dag_serialized_fields_with_schema(self): # The parameters we add manually in Serialization need to be ignored ignored_keys: set = { - "is_subdag", "tasks", "has_on_success_callback", "has_on_failure_callback", diff --git a/tests/utils/test_cli_util.py b/tests/utils/test_cli_util.py index 395db77e0392f..25003eec6d0eb 100644 --- a/tests/utils/test_cli_util.py +++ b/tests/utils/test_cli_util.py @@ -83,12 +83,9 @@ def test_process_subdir_path_with_placeholder(self): assert os.path.join(settings.DAGS_FOLDER, "abc") == cli.process_subdir("DAGS_FOLDER/abc") def test_get_dags(self): - dags = cli.get_dags(None, "example_subdag_operator") + dags = cli.get_dags(None, "example_bash_operator") assert len(dags) == 1 - dags = cli.get_dags(None, "subdag", True) - assert len(dags) > 1 - with pytest.raises(AirflowException): cli.get_dags(None, "foobar", True) diff --git a/tests/www/views/test_views.py b/tests/www/views/test_views.py index 44f11d3f033a5..ae18bdc943981 100644 --- a/tests/www/views/test_views.py +++ b/tests/www/views/test_views.py @@ -367,7 +367,7 @@ def get_task_instance(session, task): session.commit() test_app.dag_bag = DagBag(dag_folder="/dev/null", include_examples=False) - test_app.dag_bag.bag_dag(dag=dag, root_dag=dag) + test_app.dag_bag.bag_dag(dag=dag) with test_app.test_request_context(): view = Airflow() @@ -469,7 +469,7 @@ def get_task_instance(session, task): session.commit() test_app.dag_bag = DagBag(dag_folder="/dev/null", include_examples=False) - test_app.dag_bag.bag_dag(dag=dag, root_dag=dag) + test_app.dag_bag.bag_dag(dag=dag) with test_app.test_request_context(): view = Airflow() diff --git a/tests/www/views/test_views_acl.py b/tests/www/views/test_views_acl.py index 17700749f6e32..400ac2a6bf384 100644 --- a/tests/www/views/test_views_acl.py +++ b/tests/www/views/test_views_acl.py @@ -147,7 +147,7 @@ def init_dagruns(acl_app, reset_dagruns): start_date=timezone.utcnow(), state=State.RUNNING, ) - acl_app.dag_bag.get_dag("example_subdag_operator").create_dagrun( + acl_app.dag_bag.get_dag("example_python_operator").create_dagrun( run_type=DagRunType.SCHEDULED, execution_date=DEFAULT_DATE, start_date=timezone.utcnow(), @@ -238,14 +238,14 @@ def client_all_dags(acl_app, user_all_dags): def test_index_for_all_dag_user(client_all_dags): # The all dag user can access/view all dags. resp = client_all_dags.get("/", follow_redirects=True) - check_content_in_response("example_subdag_operator", resp) + check_content_in_response("example_python_operator", resp) check_content_in_response("example_bash_operator", resp) def test_index_failure(dag_test_client): # This user can only access/view example_bash_operator dag. resp = dag_test_client.get("/", follow_redirects=True) - check_content_not_in_response("example_subdag_operator", resp) + check_content_not_in_response("example_python_operator", resp) def test_dag_autocomplete_success(client_all_dags): @@ -376,12 +376,12 @@ def test_dag_stats_success(client_all_dags_dagruns): def test_task_stats_failure(dag_test_client): resp = dag_test_client.post("task_stats", follow_redirects=True) - check_content_not_in_response("example_subdag_operator", resp) + check_content_not_in_response("example_python_operator", resp) def test_dag_stats_success_for_all_dag_user(client_all_dags_dagruns): resp = client_all_dags_dagruns.post("dag_stats", follow_redirects=True) - check_content_in_response("example_subdag_operator", resp) + check_content_in_response("example_python_operator", resp) check_content_in_response("example_bash_operator", resp) @@ -413,18 +413,18 @@ def client_all_dags_dagruns_tis(acl_app, user_all_dags_dagruns_tis): def test_task_stats_empty_success(client_all_dags_dagruns_tis): resp = client_all_dags_dagruns_tis.post("task_stats", follow_redirects=True) check_content_in_response("example_bash_operator", resp) - check_content_in_response("example_subdag_operator", resp) + check_content_in_response("example_python_operator", resp) @pytest.mark.parametrize( "dags_to_run, unexpected_dag_ids", [ ( - ["example_subdag_operator"], + ["example_python_operator"], ["example_bash_operator", "example_xcom"], ), ( - ["example_subdag_operator", "example_bash_operator"], + ["example_python_operator", "example_bash_operator"], ["example_xcom"], ), ], @@ -484,7 +484,7 @@ def test_code_failure(dag_test_client): @pytest.mark.parametrize( "dag_id", - ["example_bash_operator", "example_subdag_operator"], + ["example_bash_operator", "example_python_operator"], ) def test_code_success_for_all_dag_user(client_all_dags_codes, dag_id): url = f"code?dag_id={dag_id}" @@ -494,7 +494,7 @@ def test_code_success_for_all_dag_user(client_all_dags_codes, dag_id): @pytest.mark.parametrize( "dag_id", - ["example_bash_operator", "example_subdag_operator"], + ["example_bash_operator", "example_python_operator"], ) def test_dag_details_success_for_all_dag_user(client_all_dags_dagruns, dag_id): url = f"dag_details?dag_id={dag_id}" @@ -673,7 +673,7 @@ def test_blocked_success(client_all_dags_dagruns): def test_blocked_success_for_all_dag_user(all_dag_user_client): resp = all_dag_user_client.post("blocked") check_content_in_response("example_bash_operator", resp) - check_content_in_response("example_subdag_operator", resp) + check_content_in_response("example_python_operator", resp) def test_blocked_viewer(viewer_client): @@ -685,11 +685,11 @@ def test_blocked_viewer(viewer_client): "dags_to_block, unexpected_dag_ids", [ ( - ["example_subdag_operator"], + ["example_python_operator"], ["example_bash_operator", "example_xcom"], ), ( - ["example_subdag_operator", "example_bash_operator"], + ["example_python_operator", "example_bash_operator"], ["example_xcom"], ), ], diff --git a/tests/www/views/test_views_blocked.py b/tests/www/views/test_views_blocked.py deleted file mode 100644 index c3e8cd4e88cf1..0000000000000 --- a/tests/www/views/test_views_blocked.py +++ /dev/null @@ -1,90 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -from __future__ import annotations - -import pytest - -from airflow.models import DagModel -from airflow.models.dagbag import DagBag -from airflow.models.serialized_dag import SerializedDagModel -from airflow.operators.empty import EmptyOperator -from airflow.operators.subdag import SubDagOperator -from airflow.utils import timezone -from airflow.utils.session import create_session -from airflow.utils.state import State -from tests.test_utils.db import clear_db_runs - -pytestmark = pytest.mark.db_test - - -@pytest.fixture -def running_subdag(admin_client, dag_maker): - with dag_maker(dag_id="running_dag.subdag") as subdag: - EmptyOperator(task_id="empty") - - with pytest.deprecated_call(), dag_maker(dag_id="running_dag") as dag: - SubDagOperator(task_id="subdag", subdag=subdag) - - dag_bag = DagBag(include_examples=False) - dag_bag.bag_dag(dag, root_dag=dag) - - with create_session() as session: - # This writes both DAGs to DagModel, but only serialize the parent DAG. - dag_bag.sync_to_db(session=session) - - # Simulate triggering the SubDagOperator to run the subdag. - logical_date = timezone.datetime(2016, 1, 1) - subdag.create_dagrun( - run_id="blocked_run_example_bash_operator", - state=State.RUNNING, - execution_date=logical_date, - data_interval=(logical_date, logical_date), - start_date=timezone.datetime(2016, 1, 1), - session=session, - ) - - # Now delete the parent DAG but leave the subdag. - session.query(DagModel).filter(DagModel.dag_id == dag.dag_id).delete() - session.query(SerializedDagModel).filter(SerializedDagModel.dag_id == dag.dag_id).delete() - - yield subdag - - with create_session() as session: - session.query(DagModel).filter(DagModel.dag_id == subdag.dag_id).delete() - clear_db_runs() - - -def test_blocked_subdag_success(admin_client, running_subdag): - """Test the /blocked endpoint works when a DAG is deleted. - - When a DAG is bagged, it is written to both DagModel and SerializedDagModel, - but its subdags are only written to DagModel (without serialization). Thus, - ``DagBag.get_dag(subdag_id)`` would raise ``SerializedDagNotFound`` if the - subdag was not previously bagged in the dagbag (perhaps due to its root DAG - being deleted). ``DagBag.get_dag()`` calls should catch the exception and - properly handle this situation. - """ - resp = admin_client.post("/blocked", data={"dag_ids": [running_subdag.dag_id]}) - assert resp.status_code == 200 - assert resp.json == [ - { - "dag_id": running_subdag.dag_id, - "active_dag_run": 1, - "max_active_runs": 0, # Default value for an unserialized DAG. - }, - ] diff --git a/tests/www/views/test_views_log.py b/tests/www/views/test_views_log.py index 2607317c5fccc..56df22eb050a6 100644 --- a/tests/www/views/test_views_log.py +++ b/tests/www/views/test_views_log.py @@ -139,8 +139,8 @@ def dags(log_app, create_dummy_dag, session): ) bag = DagBag(include_examples=False) - bag.bag_dag(dag=dag, root_dag=dag) - bag.bag_dag(dag=dag_removed, root_dag=dag_removed) + bag.bag_dag(dag=dag) + bag.bag_dag(dag=dag_removed) bag.sync_to_db(session=session) log_app.dag_bag = bag diff --git a/tests/www/views/test_views_tasks.py b/tests/www/views/test_views_tasks.py index 4e4b8d27afc83..1d84f31d511a0 100644 --- a/tests/www/views/test_views_tasks.py +++ b/tests/www/views/test_views_tasks.py @@ -83,14 +83,6 @@ def init_dagruns(app): dag_id="example_bash_operator", run_id=DEFAULT_DAGRUN, ) - app.dag_bag.get_dag("example_subdag_operator").create_dagrun( - run_id=DEFAULT_DAGRUN, - run_type=DagRunType.SCHEDULED, - execution_date=DEFAULT_DATE, - data_interval=(DEFAULT_DATE, DEFAULT_DATE), - start_date=timezone.utcnow(), - state=State.RUNNING, - ) app.dag_bag.get_dag("example_xcom").create_dagrun( run_id=DEFAULT_DAGRUN, run_type=DagRunType.SCHEDULED, @@ -200,21 +192,11 @@ def client_ti_without_dag_edit(app): ["runme_1"], id="graph-data", ), - pytest.param( - "object/graph_data?dag_id=example_subdag_operator.section-1", - ["section-1-task-1"], - id="graph-data-subdag", - ), pytest.param( "object/grid_data?dag_id=example_bash_operator", ["runme_1"], id="grid-data", ), - pytest.param( - "object/grid_data?dag_id=example_subdag_operator.section-1", - ["section-1-task-1"], - id="grid-data-subdag", - ), pytest.param( "duration?days=30&dag_id=example_bash_operator", ["example_bash_operator"], @@ -459,22 +441,21 @@ def test_last_dagruns(admin_client): def test_last_dagruns_success_when_selecting_dags(admin_client): resp = admin_client.post( - "last_dagruns", data={"dag_ids": ["example_subdag_operator"]}, follow_redirects=True + "last_dagruns", data={"dag_ids": ["example_python_operator"]}, follow_redirects=True ) assert resp.status_code == 200 stats = json.loads(resp.data.decode("utf-8")) assert "example_bash_operator" not in stats - assert "example_subdag_operator" in stats + assert "example_python_operator" in stats # Multiple resp = admin_client.post( "last_dagruns", - data={"dag_ids": ["example_subdag_operator", "example_bash_operator"]}, + data={"dag_ids": ["example_python_operator", "example_bash_operator"]}, follow_redirects=True, ) stats = json.loads(resp.data.decode("utf-8")) assert "example_bash_operator" in stats - assert "example_subdag_operator" in stats check_content_not_in_response("example_xcom", resp) @@ -972,15 +953,8 @@ def test_task_instance_set_state_failure(admin_client, action): check_content_in_response("Failed to set state", resp) -@pytest.mark.parametrize( - "task_search_tuples", - [ - [("example_xcom", "bash_push"), ("example_bash_operator", "run_this_last")], - [("example_subdag_operator", "some-other-task")], - ], - ids=["multiple_tasks", "one_task"], -) -def test_action_muldelete_task_instance(session, admin_client, task_search_tuples): +def test_action_muldelete_task_instance(session, admin_client): + task_search_tuples = [("example_xcom", "bash_push"), ("example_bash_operator", "run_this_last")] # get task instances to delete tasks_to_delete = [] for task_search_tuple in task_search_tuples: From 1d9b0624a7c302d4d41c7f6f72898e2253b445a3 Mon Sep 17 00:00:00 2001 From: Brent Bovenzi Date: Mon, 12 Aug 2024 11:53:47 -0400 Subject: [PATCH 2/4] Fix UI issues, remove subdag from dag.html --- .../js/dag/details/taskInstance/Nav.tsx | 13 +++--- .../js/dag/details/taskInstance/index.tsx | 2 - airflow/www/templates/airflow/dag.html | 40 +++++++------------ 3 files changed, 20 insertions(+), 35 deletions(-) diff --git a/airflow/www/static/js/dag/details/taskInstance/Nav.tsx b/airflow/www/static/js/dag/details/taskInstance/Nav.tsx index dd03d6a28c073..22d7fdd696fa8 100644 --- a/airflow/www/static/js/dag/details/taskInstance/Nav.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/Nav.tsx @@ -20,7 +20,7 @@ import React, { forwardRef } from "react"; import { Flex } from "@chakra-ui/react"; -import { getMetaValue, appendSearchParams } from "src/utils"; +import { getMetaValue } from "src/utils"; import LinkButton from "src/components/LinkButton"; import type { Task } from "src/types"; import URLSearchParamsWrapper from "src/utils/URLSearchParamWrapper"; @@ -28,18 +28,16 @@ import URLSearchParamsWrapper from "src/utils/URLSearchParamWrapper"; const dagId = getMetaValue("dag_id"); const taskInstancesUrl = getMetaValue("task_instances_list_url"); const taskUrl = getMetaValue("task_url"); -const gridUrl = getMetaValue("grid_url"); interface Props { taskId: Task["id"]; executionDate: string; - operator?: string; isMapped?: boolean; mapIndex?: number; } const Nav = forwardRef( - ({ taskId, executionDate, operator, isMapped = false, mapIndex }, ref) => { + ({ taskId, executionDate, isMapped = false, mapIndex }, ref) => { if (!taskId) return null; const params = new URLSearchParamsWrapper({ task_id: taskId, @@ -52,9 +50,6 @@ const Nav = forwardRef( _flt_3_task_id: taskId, _oc_TaskInstanceModelView: "dag_run.execution_date", }); - const subDagParams = new URLSearchParamsWrapper({ - execution_date: executionDate, - }).toString(); if (mapIndex !== undefined && mapIndex >= 0) listParams.append("_flt_0_map_index", mapIndex.toString()); @@ -63,7 +58,9 @@ const Nav = forwardRef( return ( - {!isMapped || mapIndex !== undefined} + {(!isMapped || mapIndex !== undefined) && ( + More Details + )} List All Instances diff --git a/airflow/www/static/js/dag/details/taskInstance/index.tsx b/airflow/www/static/js/dag/details/taskInstance/index.tsx index 74f317867aea3..44311249b0cec 100644 --- a/airflow/www/static/js/dag/details/taskInstance/index.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/index.tsx @@ -54,7 +54,6 @@ const TaskInstance = ({ taskId, runId, mapIndex }: Props) => { const children = group?.children; const isMapped = group?.isMapped; - const operator = group?.operator; const isMappedTaskSummary = !!isMapped && !isMapIndexDefined && taskId; const isGroup = !!children; @@ -90,7 +89,6 @@ const TaskInstance = ({ taskId, runId, mapIndex }: Props) => { isMapped={isMapped} mapIndex={mapIndex} executionDate={run?.executionDate} - operator={operator} /> )} {!isGroupOrMappedTaskSummary && } diff --git a/airflow/www/templates/airflow/dag.html b/airflow/www/templates/airflow/dag.html index d3a7995440c05..973ca812e3fb2 100644 --- a/airflow/www/templates/airflow/dag.html +++ b/airflow/www/templates/airflow/dag.html @@ -110,35 +110,25 @@ {% endblock %} {% block content %} - {% if dag.parent_dag is defined and dag.parent_dag %} - - - DAG: {{ dag.parent_dag.dag_display_name }} - {% endif %} -

- {% if dag.parent_dag is defined and dag.parent_dag %} - SUBDAG: {{ dag.dag_id }} + {% if can_edit_dag %} + {% set switch_tooltip = 'Pause/Unpause DAG' %} {% else %} - {% if can_edit_dag %} - {% set switch_tooltip = 'Pause/Unpause DAG' %} - {% else %} - {% set switch_tooltip = 'DAG is Paused' if dag_is_paused else 'DAG is Active' %} - {% endif %} - - DAG: {{ dag.dag_display_name }} - {{ dag.description[0:150] + '…' if dag.description and dag.description|length > 150 else dag.description|default('', true) }} - {% if dag_model is defined and dag_model.max_consecutive_failed_dag_runs is defined and dag_model.max_consecutive_failed_dag_runs > 0 %} - - {% endif %} + {% set switch_tooltip = 'DAG is Paused' if dag_is_paused else 'DAG is Active' %} + {% endif %} + + DAG: {{ dag.dag_display_name }} + {{ dag.description[0:150] + '…' if dag.description and dag.description|length > 150 else dag.description|default('', true) }} + {% if dag_model is defined and dag_model.max_consecutive_failed_dag_runs is defined and dag_model.max_consecutive_failed_dag_runs > 0 %} + {% endif %}

From 457290e8a0aa9fa4aad3a5421afcdaaac9aaf255 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Mon, 12 Aug 2024 21:07:01 +0100 Subject: [PATCH 3/4] Fix tests --- airflow/api_connexion/schemas/task_schema.py | 2 -- airflow/models/dag.py | 2 +- .../providers/cncf/kubernetes/operators/pod.py | 2 +- tests/conftest.py | 1 + tests/models/test_dag.py | 1 + tests/www/views/test_views_decorators.py | 17 ++--------------- tests/www/views/test_views_tasks.py | 8 ++++++++ 7 files changed, 14 insertions(+), 19 deletions(-) diff --git a/airflow/api_connexion/schemas/task_schema.py b/airflow/api_connexion/schemas/task_schema.py index 03bf4b59ef2e2..e78c3ef4af1b2 100644 --- a/airflow/api_connexion/schemas/task_schema.py +++ b/airflow/api_connexion/schemas/task_schema.py @@ -26,7 +26,6 @@ TimeDeltaSchema, WeightRuleField, ) -from airflow.api_connexion.schemas.dag_schema import DAGSchema from airflow.models.mappedoperator import MappedOperator if TYPE_CHECKING: @@ -61,7 +60,6 @@ class TaskSchema(Schema): ui_color = ColorField(dump_only=True) ui_fgcolor = ColorField(dump_only=True) template_fields = fields.List(fields.String(), dump_only=True) - sub_dag = fields.Nested(DAGSchema, dump_only=True) downstream_task_ids = fields.List(fields.String(), dump_only=True) params = fields.Method("_get_params", dump_only=True) is_mapped = fields.Method("_get_is_mapped", dump_only=True) diff --git a/airflow/models/dag.py b/airflow/models/dag.py index 2f9fc479ae757..7b762aa18dbf8 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -2355,7 +2355,7 @@ def clear( end_date=end_date, run_id=None, state=state, - include_dependent_dags=False, + include_dependent_dags=True, session=session, dag_bag=dag_bag, exclude_task_ids=exclude_task_ids, diff --git a/airflow/providers/cncf/kubernetes/operators/pod.py b/airflow/providers/cncf/kubernetes/operators/pod.py index afe6cfef9228c..6b3b635d220c6 100644 --- a/airflow/providers/cncf/kubernetes/operators/pod.py +++ b/airflow/providers/cncf/kubernetes/operators/pod.py @@ -496,7 +496,7 @@ def _get_ti_pod_labels(context: Context | None = None, include_try_number: bool labels.update(try_number=ti.try_number) # In the case of sub dags this is just useful if getattr(context["dag"], "parent_dag", False): - labels["parent_dag_id"] = context["dag"].parent_dag.dag_id + labels["parent_dag_id"] = context["dag"].parent_dag.dag_id # type: ignore[attr-defined] # Ensure that label is valid for Kube, # and if not truncate/remove invalid chars and replace with short hash. diff --git a/tests/conftest.py b/tests/conftest.py index d2d9ecf0e7f5b..d41ac095c2858 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -846,6 +846,7 @@ def get_serialized_data(self): def _bag_dag_compat(self, dag): # This is a compatibility shim for the old bag_dag method in Airflow <3.0 + # TODO: Remove this when we drop support for Airflow <3.0 in Providers if hasattr(dag, "parent_dag"): return self.dagbag.bag_dag(dag, root_dag=dag) return self.dagbag.bag_dag(dag) diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py index fcbfaa6b153cf..2b3961f1e6c6f 100644 --- a/tests/models/test_dag.py +++ b/tests/models/test_dag.py @@ -1337,6 +1337,7 @@ def test_sync_to_db(self): ) with dag: EmptyOperator(task_id="task", owner="owner1") + EmptyOperator(task_id="task2", owner="owner2") session = settings.Session() dag.sync_to_db(session=session) diff --git a/tests/www/views/test_views_decorators.py b/tests/www/views/test_views_decorators.py index f10b3d66847f2..fb095b78d2fd0 100644 --- a/tests/www/views/test_views_decorators.py +++ b/tests/www/views/test_views_decorators.py @@ -42,18 +42,13 @@ def bash_dag(dagbag): return dagbag.get_dag("example_bash_operator") -@pytest.fixture(scope="module") -def sub_dag(dagbag): - return dagbag.get_dag("example_subdag_operator") - - @pytest.fixture(scope="module") def xcom_dag(dagbag): return dagbag.get_dag("example_xcom") @pytest.fixture(autouse=True) -def dagruns(bash_dag, sub_dag, xcom_dag): +def dagruns(bash_dag, xcom_dag): bash_dagrun = bash_dag.create_dagrun( run_type=DagRunType.SCHEDULED, execution_date=EXAMPLE_DAG_DEFAULT_DATE, @@ -62,14 +57,6 @@ def dagruns(bash_dag, sub_dag, xcom_dag): state=State.RUNNING, ) - sub_dagrun = sub_dag.create_dagrun( - run_type=DagRunType.SCHEDULED, - execution_date=EXAMPLE_DAG_DEFAULT_DATE, - data_interval=(EXAMPLE_DAG_DEFAULT_DATE, EXAMPLE_DAG_DEFAULT_DATE), - start_date=timezone.utcnow(), - state=State.RUNNING, - ) - xcom_dagrun = xcom_dag.create_dagrun( run_type=DagRunType.SCHEDULED, execution_date=EXAMPLE_DAG_DEFAULT_DATE, @@ -78,7 +65,7 @@ def dagruns(bash_dag, sub_dag, xcom_dag): state=State.RUNNING, ) - yield bash_dagrun, sub_dagrun, xcom_dagrun + yield bash_dagrun, xcom_dagrun clear_db_runs() diff --git a/tests/www/views/test_views_tasks.py b/tests/www/views/test_views_tasks.py index 1d84f31d511a0..d0e7c168e59a2 100644 --- a/tests/www/views/test_views_tasks.py +++ b/tests/www/views/test_views_tasks.py @@ -76,6 +76,14 @@ def init_dagruns(app): start_date=timezone.utcnow(), state=State.RUNNING, ) + app.dag_bag.get_dag("example_python_operator").create_dagrun( + run_id=DEFAULT_DAGRUN, + run_type=DagRunType.SCHEDULED, + execution_date=DEFAULT_DATE, + data_interval=(DEFAULT_DATE, DEFAULT_DATE), + start_date=timezone.utcnow(), + state=State.RUNNING, + ) XCom.set( key="return_value", value="{'x':1}", From 8620b324411b982fab3f0589c0ba9d6baf86bd27 Mon Sep 17 00:00:00 2001 From: Kaxil Naik Date: Tue, 13 Aug 2024 15:10:54 +0100 Subject: [PATCH 4/4] Remove screenshots --- docs/apache-airflow/img/subdag_after.png | Bin 30245 -> 0 bytes docs/apache-airflow/img/subdag_before.png | Bin 70382 -> 0 bytes docs/apache-airflow/img/subdag_zoom.png | Bin 145017 -> 0 bytes 3 files changed, 0 insertions(+), 0 deletions(-) delete mode 100644 docs/apache-airflow/img/subdag_after.png delete mode 100644 docs/apache-airflow/img/subdag_before.png delete mode 100644 docs/apache-airflow/img/subdag_zoom.png diff --git a/docs/apache-airflow/img/subdag_after.png b/docs/apache-airflow/img/subdag_after.png deleted file mode 100644 index 166a6ded314ca4aa7ee6338aae808584d3d51ed6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 30245 zcmY(p1y~$Gvo1^sfh0HtcMtBa!JPoXWznDuiw1WM?he7--DPoix5eFM@ymD4`Okmv zJkQMZ)KpisR9DZ`TcJM{rI5eke}#d8L6(sgSAl_nrG769e};d*&W3yxc)!7!{ZRM; z15+J^_-y#;{h8EQT15c{#_gTaCjbWK@tx#z00ZO10t0ho00YC93y19#r#!}gH%tn7U;IM8 z<;&iSiA0n8ar6awea-19{7g5E{!^Ucn?y7U?DUde!?<^5T>5Bp3jg(Z>! zq3V~OtqF(%AODQStyb_5;Rn{y((P7EDLdZ09Exb8a_J4q^sj7Y4ud_kKzZ78rI(5+&QfG%Te`Igt<2e=50{GzRY287K%eMC~?AJi8 zmvk%P{l`XCsk)}F#GCQA_BmX?>fa0T4#!FCDKq(8?oIlT%fhagM)fW4x=6 z?T{lU?rn{A6~@A~G@tbRQB?gCmp_$0)^kAN_YP~09j7**G@E!m%WWkcqKbVvciN4G za{03dt@eN~okBocwupHby!6#rTY5v!DY0W$y!IU}DTI-}*zpBVUd%cix@SR!Li=v= z8JcHZZh?sbgT$G=xJ3~WBG0*8X9gNqpc}65(9NA9M^KI{k^ZM_2MK8%&x^Klc|}=1 z6TNc1mFFJu>J?!&Lw8k1xbkFi#>(;^ak$1V0yhiDj}>nsM+rryW>^fHGOS z=Th0z!CsTM1Wxdd)`*r)J!S#kM4einkCVm<3jtIS&}^TUC&#Fl)1(&KGMzeskkAo(@0$-vCeG10KH8TB!fo-&ms5dWl8>tq4LUm6dT%9y z&}I3UwZ7kiv?qt2&4`!sQ>}<^DbusYfM4p~pmuvIBB$6i5@Nj(Dg^>j1QBLO>lk)k z)2ETHrqACYjO0HYIrt`L3KVJs{j1UAzlJ(%8UPUysXvH!j7|Af7Si?eKoa@P*(DX~ z`e)+QmXF~7NiF{sn&k1!$9>Z4qIZMJhVCCP2Dzmm{dTpiBu&+`_&D)erl`Nzb?|BK zudAVE_Pu}rV3Ua!vohGK^qU|mjgH;08@V;D=1s)iEHhbocmzpdFZCU2TVCP5RYyrx zUhq}G;!%q7$nUQ?f)L!S@UN~VCSLQ>kxY_VE?uQP1Dk$K62^tP_JZB?jze4pY3NY5 zK)laAG}jk3-*wiK$U;qkWd?=CI37{PruJE7`g=jfpvN(iTe#AjFKJ@i%g9Pvcv&sZ z29ioPO`!mtwx2pI3j zo+!Ldu~WU;h&I4^{Q=D6haw1-P^wJYUPt*(9H>S}l=Qn=C*xr|((wG*p%1sd+aZ-8 ze_bFVglDBloVH-lN?16MT8)lRbCGi^Xx6S>fGlkNZ^d zj@G<8WC|I! z&!c|!sS$^8TO(=WyC>1ap>K9DRa!2)PNP25l@K7wU5d&gfqZcM9o0xd=ge?Pam^qz zsoTTG6r5|r0>BomX$znqV{AuSFzBBkAkkbRv1;VXtzCs`w^QT9tJIt8g6r=(TwDcm zsknJePYj$!e08o^00l(L`WFHj2}7q7*HzB`F$AOb8}nKAGPcGEn+Y8*y7X~Lrlo0_ z{WC`eBrfav@XVJ;15j-n7osDrKigI=7qvC@S}1MiR@6g_D$?=uqK!JCS&S36H({>b_ zS9M-i)}VQ5LA*IdAvR{(cowT#!azYqE$%kiNCBD5is^MZs!N#qM5UvoY8BVwiQgu< zBYa}HNIEz!pl=}^q3bT=qXNg#J1I6V36O`Iqz2^9Dnp&8a2U7EjHj)B2?^{vyr}-! zM8i)u>dmN`SZ%M7t(%BX>(8wOHc4EMQmOjFp|h8&PUVNW%V_*_b*_(ibQ4IgdMC35 zE2Gg1{dF*;{_!W&b3(?!14A8%s4jRUN3r(uvZi4hn{M8ds`d!$LVT2uW7c@FD0pIv@;f4uRKB~Gppyi~#N2hg8= zTOqmta@78AbaDkWB0F7hg7ErB85qD#C$Jrx z{hNT0+e6C`tW6#Kb@LQnrfu#Cq<;f!c|lLa`AS7xwmSh*lj+7nhyVMH=m4wO+JAk7 zcEwYt&_3s~#YyxOPrnf1GVjD{0_jtAq#3u{1!IDJzW!aKkyP$=^TqX}PMh+#ZNQd` zTUcS2K*!-1BsZmxC3E?Dsm86l#R_(^Zrzvm;SoLN6t1biDmIl_b?N;z63Z2BJQxu6 z;x4Z};P#&W9Z-1^5ry;**fU#64pVj3L<)Ukh)Nh@bmD57h=|&)d#D+>-L+}6h!a+> zRG>1QLR@t`xbE$xzs;hP3l`sK7-APYGN23>Ve?N>4zI3!Vms|nOxbz(zS$ntI0n0Z z$JGGOPW7COSEhPq2lU{Fbi{2=n<@X<8&A-h>--Z}!M*~&m$p7}Wa4Ob4>WC#T)63Z z^(HRVx1L|ne5_{mqADwM7ohJXxKixc0*VWJWYpYl*elBCt!nSaZxMD41IAEPlKH%jOM*!qrj_{AcGWviT{*Tj<9O#i;7e5j+n72Yny z%9?A3J2;AMOpEXP?GM#QdnpB0)(bj`Pyp~$_c*YE znQ-H5BqP5;`!Otwwwia_!kc3@BF1OMWO4EL?+OFpMNN&ae;S56{zv%XTev2$H~(KQ zsFVF10#NATxQ%o{&(^b3cI~jT9yRvwFz3Hd#@LkLAmkYzBzs{M@8~E06;v!XO9Jml zUe~Lgi+LT}!%q<%w{ybm>U(sba%oSwOa`M_91kXaUVf`s9*cD^Jm)jF`B}GM^Vj-G z4b=ji(!HniWcV`Ufx3LG-5baKGyUj%%Rpil1Y9L-D$e3gu1doxnxam&i{RJko8n&Y z&xYzZxdciZdm}S5FWQf{y@?$t9cxI0vceVLgMH|7d3kTf+^u3u za@LbGAal-*)*CyeH!BpArh$;`Ck~1M?BnWQaSUI;x10vl#fFcE-N{vB5-)PiuaR^D4@O27U^j?u6*Si^PyWG z{o!MYaeyx}Z{OyQ6vhXm)h0Vj4)&izV1i2PYbSPU#ojF5m_>Dj-etit_g%bN7~emU zyn8=|12M;iy`G^b&l&v{n5$OCJm6r|h0D4I+L?j?3pSi}xpS?5o!F#jXx*2$yUSiL`xkL8>b0`T{P$jR$+cYmEYoz#N z;OeAZ4Tsk%l*M&;M*yMaaOnaO6RAhhduMPdz{Lj+Vqy3k&)uUlV^W_hnKB=tQ9q{i ziMG@s*lnN^4L@U-Lxp-t^F}RdF@|YskSrP35`-%R;(7=RV+z0BOS8RmWo38Hus`W% z3heR`>%Q(MutjRz@N#$AOEGigo)B@8>1foO{9SZ-8qF&9^-eZCVn5ZYqUxk(B#5zb zm94C4>#ywac%Z?A%Z=fAMb1AGE)PvyR*!%!g%oNW^b~}+R7@vB@@}%}By7f`ngATd zzzKIk09S-h3<&3aQPd?l8Bi*bkj+gTS}4_M(;YIxoiON?)tnb;_`>TiBuAV4uBlP< z4_2m55t{$)YJK{wwUux?9$${Fv^|U)3S~F%+GH`>JTW}zGL&U!e@T$w@Xp)MbZ64_ zez}EYx}Qz;eQABbOF`|KzLk`CZuGcm1@yvK#2_*?8EAUm&Vki+fcd6KJv{-Pc-8fN zOxRSqs?wN+9r+6o+OZ+S)aNiBw(t5+HAsfMI6l5oHC)4)iG0N<*g7Ry#7e_lh1<8z z(a@(smQg+14VUc*A)M{cgQ%T}@P^7){W}`q0E_KBnaEB(rC%<=bM+Rdb{XHG@;05kMy_L^sKgkFc4_9LXlKH}JA%gb5Kx z-^EbVvN-?h?}9nsj+YDnKgUn^5*r%s8hhtml0-O)dG9lp3A!4h3Fe&F6r1aPb@E;8 zXN+(dVDcRR67WV$xeLV=DtxU}@Zy!XP&A5w6go%|ui=Y#9IYb{G>%z6zg}IpjGkPr zWRKhoC+qXoDiaA_2J7t*`tQ#hATy=mVt@Cu%}eF?R`afQ-c*N=;VTTLkczY(_G`b~ ziBk%480V!~b=}W!>plS?){t9K|9XwPG?g?_;$}=+F3fbJv#uGUh73|fA>~R&Od~U0 z2Cwkjgwt)CHT+(cMo~IW|7UkJejC7HI1Igbj8D3a_OVLZq!$KyAb7ZVdf?5KEUfSz zl-W%4E;V==2iG9UlXzhf^U;GYy2ap+4HS~un8LFj3sQS*ypxF>V^lJvmMJgHPqA~l z~$G-YC@&gru;@rUBBK>wo zA`4*kbYsAW@Hn$j;jtePWD!3;!VgurRIS-4XR-hG3fJWoA~lt!rLnj^8`3)CEynGr(%F&n_N>QxTLgV`T4fed(Gfu z%ZAs};VDr+Z0i%{w#GF3Jdh~i8^6isYv=qu`H3_!Hgq1K!e2m)Dz5q|uek6zyaN@k|^N%nn@Tg!FkE$(jq zzCI4Ms_1Fl#I8FrZDMeMWlC^bZiKEDL?0VSIpe=6x^>_fo1P3po5%wy##MBy>Wp<7 z^?3l$Fi6SN*aT{hqB^vmYw~sO?s?>=k_;8~obnwa+X87`r1fME3NXS-1Z>H@c5u*B z$<|xd2@s3sRB7NJ{xXqhNi7g-$;7?tCoaofk`C9@f1JX89~>oX)kq&{JgH%=0uDD- zODqvtIz*2h{Hi+d_>@P2>dh=CEPK;v1KO}v-9E<}!sv6vjb0F1a{H^kTJGJ#DNYr<;h%gf{6K#daRJ@wj#K!%iENEaD zIG~qSp@sC1nD_#&*odjhyNCGezAOe6E+Z(dl%G6kFYQnM?V#)3jLh<>@ogoQ7yguMbAlzG6GP{Vs^WE5jwON(Db;jD@HCLieOKf{-9Q z8czdKp`O97g95M?+Um}l6R^Eae?A%3);7B3^o#mXX zg9gqcf36>qxoza*Azf-jw7>i`$crY)zImuFUr%j9_?Sb&0pA!v#&8T8@hO;y`l1|o zei)qb@HolV2IjjzB-zu`ePDBJFzPc6fReS6r>bS&iF`NvZ~(KpYC7oC*3O(QiY!+5 zP|t*?nX{dSTlPx}e|~(`$_C-)Ddb<-X$0oMj1p1%Is&W1v>JkAuMBj%+*$GQp}?m8 zuoejZ=mM>Vc`Vj)W+_G!2m?DmCA~5gW`v#Z)Ayy6cqDR_Fst4O1|-W!dA(aF5Aya0 z0Z;U@)y$W5%_#(mp%N33noqZn@vI415WMSI*|~zaaV4;v0P>o+iM~qFCQeAl5e{Eh ztx^9k`zS!U2_=?UFP6v8*mbP+4(1MPXi2z79jod}4Qp!82wbdVSq`lBJ;{(%Lg+o@ zZzVPHq#c2;J9HJkC`?$=tU1Yax21s(7wQB;y*AypeF$%G%H_s0J+mmY=55Lt&WwUY zvkUbS*eIagYbEC?l+YB%N1zz!tfD6~$!|$&C@sYEJY{|}5v2G_xA>Ig%Vya#7$@rZ zNy^xlyzzOyNdjXcdl|#k*Ax_W+*Q2h2%D8;cHy2W$VwpTI(0U*fJ~`Lgb|`_E!HEa z0tW-q`_3b}nc8LC^=+)q3??tV^vgoaA*8s>UL?%P6!0TXR-)h^`Rvfo7(4|1x`yPz zGZxr{?>&76SYL0rC?3;mUhpvOt^2nEOrY8+w zd}jE0)c+}C`0~QVROd)c0)S^{=OKcUzVF**K659o&`(F5t2)_?PNvDI!$h_uKYvKh za%&+b%X6XDuPse7kL{m7t{2Ckuzp@#oJoy{LPvh}GiG?sg&Lg~j4>~1>pZJNce8Y}L(vN{Rll@(pXhVh3{k2XEmW9RXZvfINZ6B;;OzqX z@(8O*M=V35vxm5jJ!-y4vonXeG+KLcdPqar1GH@c~jV(7Tba|S9$j>z2wv$&k6sR)HxM_oZTg5{xd;?^_W6|3Q}orp zI)D0F2J=utli=8yWzn%k|PuM(ku1nxX|cY9B658_ij{1LEK>|ZcYVl(#bphQ z8ktf87tzsPo4->c9v)riG-+gqVsN5YPVMpxJwOiPQ;`MFnKsxhmnEboW@GKaBt%-q zX3yl3j`!W&tK&oq_I&oYi)TH-y0^}{S?AiK#YXw~>YZsnFTsFp;(m||9^=|7rxp7A z(W9x@4vkR?7zG@9M|rwc8f`yL)$D4y?`O6wwRCgjLSL98YGpL&+i?67V;Y4vuMkVC z{_W?SDxJQBqAv}05t0d6WJ!tcp*XS%38_Q6&}5DOZQ#BB6aP4@pnHgmE`d!SeSUpe z5~OWf)uOZjSZTjV3y$K_)F2g?G9KEN>WG z*b8M>(@{2$A@gPp5w~$;OXHCWxf-5Td!cBjg;x=eS4s(zwC$}H^pX_(RCwVIj!4gw zbcfH8E5o&g?))}RCHumCH5zVdyjFJ6dL~D0kV+?@k&9X*S*uVJ5 zKbnSNj>x6goDq(3YoQ99 z5b4~#WTH@QaC+9W-ns@~mL`9A>+{_r{73shF1X-|J~bwWMIjkxl9t^4BNU5lS~JFV z$==e*pgY3-!Df3V2VF5nD)vVrM?y}C>JP;#?8Li1>s*6;X4!-(8#SYevMfpCW=OvC zO*B*gy07tRP}Ys&4@@{)oj#1%#8Au)d6b4!vL$jZ6Iy&T@&%I{6FhK;iMoFaRcD2G zE9Hr5z2-4Z<8mhU{u*}9?cXo*tWqMJ zbUaZBcfaIZIH8QT;R{TVa?<=s7Xu`$=|?oRc8@r5RJ4y+GG9Do@c)T%C*%V&lZNDw zg)o1Y3rQ3#l3$J_-=cP=O5!mVQxA8UxWzi=qN*GVrQTC!E5yvheDMsI9S%_(FmWxJ zH(^{&?ML}{n0?rF1pL5`wrg;ZbZpAs&S*LNm6oUu+k*Pz$mQ`4Rg2uDh_V5T%uMHB z%QD7c@k#UR)sUT-l;2kX<4@)5jU(09n39F&@Z#n1{KjEtgB7EO%ve7p^g>6X6p7H5 zXMgP*XUf&(SG15fMVe06=G*@gzE)06>?n`L6Hz$rPKo!bQM0GDLIVOU9L$rIZi*IJ zQh(ik`zS7%k})}dQSw18AWxQTfW2JITcb<%iS_MO>{9B~f@^OK-?7@wJ6x+jTCpxj z%ElBVK^@2LCZVx`mVmgSsO zuqIW$4JCx5KDHza{oC28K?srUE-tL=iTaU`nt*7O=_8Pc4neq9A&~3XDP}6vOePlL zpY_2{W~URMy9(HCI1k1u&(>nv33upCGVC|?Nxs5M)bWVjYKuJAOTc3EdFk6+|6D0v~IRnO#KF|0q&V=jtpyO0M1f3*WTN zQo~9;ii-n8p=GaGFGieYEK7P1L&5hxqdDh*LrZu(Ic1tx6_!ghLCy4wcIllX96esB z))G;5;r!99w#;O;Y-y(YCuKR!sf6Obi%s+t!BKQ>PgRGgar04e>8lX|vpN~vnSh(f z;`VZNmzVN%zKR>^lm;H-0Gl`-c;=n25k(^+Bj^g|ga5YSW>pb#d-HOx>MliejnE%V zcWXYJ5ZOEB1?U)_9_k#KNoIV-TH_=8qPy#C;Z4JNnI=-%`P!l=@Wg522P^C`kbHrd zFF)Pbs&tYJ{p#%~#_Tm{seUwhytr3=Ed)3D{2jdQMfPZ8uXU~WL#Z6gIW&Tc%JZPc z?zfFaM#Hb4k0F%icdyb zemsDsEIv&5L6;teafbm_^LW!)Wh2q)B`8gYb_7qgQlui#8Y*GqJZ*c4jlc6$efMHK zB&EW7<*O!7;!g`Jvb;m$xf^9$ozBI{`jhe@#5^Vo$Hcr?I7o=-A>1JuC7V4gz|P>~ z&9YVH;a0B~a!PXV`SYZWus(OKiSw3!7MjE0Lp!76d*l@%G#HV&t?Ed6=}C#(D`Qz; z^%*mnS&Go%o076>KVod9#cvbXg=Rx1X=Lt%Io~qD`rTL8rb2e1oz$)LV3bxZ;rK=` zN)BNJjH1y1^+lJQkg1s-IL`?6k4ouoc*vr$#+kB-xz^c5yM@sl{>SC0ABH-Nf+}%>?0>o!>OUAsi z1d1_ZQ%hkYfkSZq{yaYLUgNduAX|;`9MneD)tYG`$suxK6RqEF-`nkXa#Y$bY3)3A8hVuiGryME-C5=NqV8aMJd zyHdo{IiIjB9=nqB7-0kU|8@KKTu-#4oi!h$8t%@iBjqM(>h8(5fK2segQ}y^=i<3c zX-H~VbCav{I}=(|G=x1`6Q}aChOW&i`b1!R9X-qeACwbKdS)y1K;-s0a&>%nEZO4{ znSdX}f^Lq+Jpo_v-8J*9+i&L$PZp~o6W*^5Pzb7bQ5fb0mHwfNiX=%PCPRHi;Ef?` zB@>gI0Ccm^e$9M%VThY+!^7dl$=<6tJ!0x3ssNfz^KN2+bv!ZRifGtn*CY!oi@87(I-Eu z1w(n#{e)u^(4|E~J6G=)K#*pfzyT6>(T~A&6u_M?0JFL&{ET`XAty_Yoq#V_VW}D~ zfTkk^;V+o|O41)oufEpySnN?g`eJ4ydapOVBpBT2rfX?-ho$qGF3apNU_-Ht;{d_N zb$dITVud(J4~!8B>e{q~PP598O@CCBFvbQ=m%Al1r$1H8mZgEn+^a03+#4Q`X#|sE z16fJ?PYPO!Tss3PK}Grnol_}iYt~oK*Av0Q&k$7KbeQ_HmgA$SXyxFym6(I9CGCM+ zr?sMxf$TO_wSCdjPsEIGx4VPmEXY!d`1sA~{VO*qTnP`~Tv-ky8h29OWAj3fg1{Qs z7>4P~6^$HM_90TbkgrpfDx}JNlXCJgPv$q1G4fKAMcb;(&&Pom5tW|ot4>dA2lL;O z;hw%g4#XnA+^;)i9hK7179a`-BxAk;JtzqA1|wQ}NsPmn?sMg;~XGg@+UIH`uM(4FoEAY+=&*2f}oYw_Xj? zAFuAu#(zSBl2?-1*PT;OJVyMSsvR0CIiv@=*-25~WLO&muX0NxKBDl5Y?Fn>p?Y2T zL%<}jAfB5^F)$>Uq#_LCrkjb##i@?|F=n!{GC=5`WM}DP@YHW6`3N4tI@)o3@UW|x zhw=lf#%S;iOQ@~lNHs-+g4$g*A@5(E9g~~It`pEUv+Y+J;)hRyyIk&)L_~&r|C~dR z{U`AjDOamJBBI0WFis`7tm~d1wkLF3jfOk!U)0`&WACncK0QAE1hcgOg+} zB8Fz<>1`ioVX=JdC3QKPZpnQ4e;)*)9QZ%$AIb(zP9ea*4$)+d(qP@7uisfEzzLiT zQ+h!Haq5te-P+zg?1-CzF!fvK>n-!H9Djou4NPS)vDdyDFu5cF+kVEMb(R4_ha%5OFwIfH z+dF=j+xfMv7enr#;*-Fm5-25RHjDKK$+d8Qobh8zFVdA@D4XVNU6pS)Du1+&XDXDf z)hhNji-?CNco>10+^0jTV`x8ElEhd5TKnJ|At)#?m0_pydbr@3f_fEqcriN@I}}GP zHKF)XzPC`wp-w}ly34IghEWWy@*su_(z*(KZiLuf_v>V9=UJ; zRU&b&S7_wNpgOO;)njR&vbby>)tyF@yty&==8uH?gr0B+a3)$kHV2fYNuWCmz!y2j z)^DtZtaex8_4GHcx-48nOS(Wxr0NtZ?5siX`cI^9#FPYxCPudHxG3SBDvG19>IL{e z#nJmO4&x`oxJGEx^sMz^k8MS%?Se3ww@~@ONgehoU7=Ll@~{(IA3Xb%nz3p({SfI; zTo&)x4QV!BA@t7}Pw5Wh#WZ=F0*?o|`0mkxOzE3{ zZIHL{Jv`*aGY@7&h|_4j+R&avXk1mdzJ;X`N-q5kX^cA@C z`tGMyuaCzRov!cpz=f^nIAx6hG5)BAXYeRX-YEQOF!+G--*BI18dBu=!PiK=FrT>s z*!0qnLh`F!onW2nAG2yCiVG?dnCnm;0(hg0ep0ZU1L8w9Nej%c&VP{`+YVyE$jG_a zo>^C1g7r}a2qIHbCiLPfU5I62d%*Uvjg9U1Iiyy?;eUs|Mh;vGI~jJ z9D7g4K*;W3X^yr(ihxAl`_ySSI8NwtG0w2pn&KBzB>(-YLy(y%H~R+cb(n)32t!SQ zxzb86r7Pmh$@3a|VXR`WbQMHk^ZB@}R&mqsuWHO=&VAc=inxDWlpEHyV?9?R%oECm ziZ#I`UOcOw_iIO=;l5B$A57(fQTR{Lu(eFk^S2WRi2kf*UuC<$=dg0ZoKz1G8;&Nk z4rfUWpL@T#>%~{mYZ?Y(ImXT;p?V&O;o}m!pHzd^oECNgH2G0*fokenRAtS3srVj~ z*CcPy;9-i-nm?Bf{E>ScCIkz^WT`noj=TE{SzeD8yGm$3GO5xDaJ}ErsABk!Rut!z zM1tj06uCG;;cM;p>!pZw)y4xyhcOXm1RJnFuL`3EO+|&?D)+&ieFI;vDEgB20U$3T zg4SrXrpv?q-ChXimkR%2B$Vs0b2fkrAF@8f^ZO`9u-pqc(4K9-d5C@-_d{et!^W@w zsfz6vQJCqXz1$zjyO*kpU5prL$hMzM0-c9x{>e7M^P_9v9dbSE7xnXJ=tjTgw3n`A z7`S6?nnrbLd%mKF^zEApg;h|Pk>FwLn)|X0q&pmVc<$C~{CW$NjQDChG_*UTB#60} zVVjBDW@6~eEOLJnHxaX=kZICA({2=}&&@9&jXwl8-TP%fbd7iPI?Z}nPZm<7?F^x?b` z>BmDkF3Up=7e9w8_52~8N`WJR3f|v1*xPy1{@Xsm@Wzg=CdfR%nu4Tebtma)QkZ2i zj&k1X9*R;MYMrEDs=}xaU`DU@7>z?sR#85A#5HwEq&E-LqGdeSe+)zV?NN>1u>Ovt zaVE0fhf;XI90S1b(>uWObW&TYm%N>BDLra6F8ub4jE${3`e1ABAn6^o(nDg;!X-JN+_lS9p@tf3JP%TR&=@jL65?=a*xMI=YhF&lB;az?jp9&~ z!D!tPk8Fmd%11EPbU!DJ7BA>0nBDF04W*}gjxWs>9wGo2gETKrW+pf{TwWsJuidim z8H=DdDIDU<0Ph#%J`$P^Kiep6lla~gh$TSc=!y)n+>%+7rg`U7bm?(&rYFznE9>_h zlVMtS z)UoMw<#{6jOu;>?`e6Wd;o zs*c-O*KVsrSij8Qza4sUZbo@SXFyA+OkY+`nw-x5nQ2ysSp7qNb_F->>2eTKUmFG! zVXFf?Z>yfpT95OX)aOzPR6l4T2|W}FzgV=3us1WZc6+_gcyQ(Wyt~QPOi2<1z7MsO zZ6_{Dsi&d{Q}YhS+E~>AbcaT`xX}tJI9+0bcRxLsP3S5n${es%>@!la`<{+6A379y z5C(Y8fzNxn-vE3>!tBF%A)&`PxAi>Mkrn&5^k>79(;V%4@@(RJo>Mz%E3#hzJgQ#= z)uTM1`GP#H4>OF&57E2M-6JO(=ZX^TZ6Xng)3KLlayC562{ z0EB7ExSjQof1$pe8a4w#BGAS4)*$caQxeGVxn6!#nstXA&*tXKjnH|L5--h6mgD24 z;rYkA={|wG*5kX4Kw}g^CIQDcF}(c~#0ez+vj+bQlHxF|$8$_QVp1GgWwup^dlD0& zEKHl%TSh$7q$qC=Lg&%!F)R)JY`fFBg`k)GEC#Mf;VsIS*3*tON>DrVDA&_q!6f&q z<9Sxk!gw>!OqT2Gp<$AsbK?GI$)d5>$3g^--@0$L0Mm41C0^>2`i%iFc`h~z>ho==wNug$*de0>UOn#lNe_ICYZxRXu+POEIkQ+ItEhKyEb63^>eA-8M3>^%~I*ZI$r zoU$*PCrI$`9%k1KM-QgxFmdm8Z%k63Um$<(gBZCNxvX1fxFC5mkqZ%;Z6RvpW*vE< z6@gO_+_>9;BhzsgyB51mUxc9YhGkhLQXVH0p?isASM&^X9--o&iu9U92{ zdM8)p6uzsD;&fgz#kr_Gvv#cLcs#mzIrj!IFZaUROrarq8hyBpu=St-tpwl(0%FLoe>Bme zlLuC7gx;R7)v$4n>WU^m!SnQzfcGg=0s}D>Fd6o%5Ol z&h55o5be2EJ!{+88~) z2Jp~T$)w70;m=2{CiBWbM@s`%mt|hU4em}I49P>t`4#i5skt|PA{n>Qwg!fcw~FIX zY3M)sR~>9221~uZnlD-vdV&k@p5?a(7WAf8ft7mrCjuSL;}Vc}+1kYES~^PdHtmPb zdhvuF&>w8gjq^lv8I5aLkuN7-ATK1kf()%UY-2-Lq|?GA4~0zT^V;YB@6O9j8@F}` zcS%{{^TtE}y<_9mmoLO4m31Di)hdk{%jmJ;gC%bZosUk*wyzI#r6L=DK9^PMcJTr? zjfSuB1q6mo^4b@A^L$X_w6GBs)F&7)cz~ObU)0#$5GC)&xYyJ7+_jb!gW&EM3aKJz zrTDeM%K;LSj+=aDTB62ZJX8>8k;BMi0S@P@F2m@x_Lanb2m z&(vww7?fy-hsZn!U9f?qT^~yrijw6iu_;fq^<8%pE=mC}vNTK$kPg1>drvREz{lE`j_cs0!a!AXHS3&E;`7q&T;_&iuIo7=1n;0bX=@qSE z%_X?%Hd1F8n8Ac=|GQM54Y@<+%AN2m36~^ zb7qS1`GuSgUM-^}aCA5Xt}ZByqKLpHR%Q5JkMyop+StCmg`Un!%m%G$>HuEQV{eX0 z<;y*i`><&2+EzLD8oDCi5u!agrxed~^28n@!7KfJd5^7aJNrr1;ke-pqcDb|!0jR$ zNk~M8{!wsbqULPuV)NU^+Y4ZGHZ}~z{5otw=a3cm`E`-Bn&praQVJEU?#HI8kujHj zwx~MX%5%IqaN(nG1f*gx2Q&Wmx1LGK^dBt&7aq91cU?s$8w!@z3wqe!D-D55Y!7~PRYAn^0k(nk%$;;6H`PSodTu~&;XmQ0!+OX~+<1I*mF%J@ zas5ALK>VV^`vw;n*6HBOu%!WUq|&`f2m_CCua-FI$5*2jxn;8_sde5Itkb%-rj}RH zM3pVz={}GuF9%i*QR8$UJ|JU;_{+y5E~7GcvYk|KoEai($usc0`HiV*XxCRyW*`P` zJYV$}5YlrC)!p$uhJQhfV{BXzC}SFM+@=|}YY;f>Sa@|0piqSB;&R@LI+n9;JYQRa zx|3CV;)B-i^}KVPz|BeNYr6KuRI^GhlV_1S-ty#*XWn;^;?>Dvfp8{w(&|Xx7U$K){QWmL6!6A#=V3QYU(;N zG(wGNlToTLumi41nMBST`^E=L2KViYTVX*?>dP#;uVd%Is6$4kMXLzZcG;#=f<}v< zwJw;+3qufwKkg(QY8KM?z)E zb(<6M!pw8rQ#B>UH5$T67~V6%9_==*ZDe+jo8)LLN+O6`*Hen?0}beQ1(vuHXJ z6{zFoV6}K2mo;hNa$**RdTayDu4+%m9R!9JtR`B35dX$yY5G-oU_CuLl#Ul zHGf73#JAEEY$)U`%ee2v4C^`7f#X~*;{Fli))Z0)pTNTIloRtD=WW|(@L{q3LVx~U z;niO}*;_|%yHppTPE$({C>M%$#@zxB-p z1j8EGgt|xwaJhIWc{#0h?fS6X>`Y#>{;a#UKuoH;;6t0c12Zv=6mgSu?T0Sb&b>60 z@JueJ2stg|g~Gb~(xIXZAJ1o1mF85{nqP7YYo^!ZU)hV!nW@fQ)9icJaZB^%>27&R5We}46~Xd3?TQy7{9#9Q4sSPwF? zZZSCTED74E*3uY<<(&cBH`WizcyN@fvZE@s7YN@4IY)|E;8@M z^hUMEK-XOBq1FeN{df5KW3(?P1%YK2Z<}Vl18U+^ak?}JvY+Rt2Nwnl`|17{ZwuuV zH#J|Q>47kf8YMB|wn+e~=s%-{i3(pn{?PX%c@fjfK9d;DbmBuQBglS6e+tY5ZFVAI&W z7ERnRA9Yw3ZCX!nl3NKo`PLV2a5|w9P4DoSV^W{;m683ZZq}qy5We7LR++71ZLwOrZFfmleWv|#wt2f}->rPG&23_3s)e2cFpzYmgf;ZM&l@@{LdXB6; zJT;nltSc8)B`2xaNi%U$Z_;JTp;lgV5#tzdeI+1K{67H;HT23YxG1v_Vcy%V88#8uTyu^3APaw<{UAv! z&<9j0b6bSdT252gD8UCF*bV=XMS%7Nehu4ycZ^S@0g`7xpON>6UDEYw; zeh_ZD=_ZX|=wLc39_p_Pi+>6g@ zY`4=kO$jm;rC8(;NHHQiyo=BIMp$ggMZ=1#ET_9Ul+=gh&>`(UcsPfSF|g2*vUiCqDU3nnRvmF?)!qqWQj`Vf4XfF_$s2@Ge%PB{5@zVjWuZ_05c`bQ^^58Rmz zjj8WHyjUPMYxS21d8 z@V7qJG%Lv3(T4~8xDCcB5J8wa$!wa+qDxxnCa042sGoeohX=Uv&r{F(g6`JuxaGHD z-(B}tUTe$y{-2cvx?DW-n;k4t@NX`7*bN>$uR-F~cXM8SdR}gipbvj6$mQ)CwO@pZ zFGxE{`lWM>huyCE(a*v^A9-AOy(POhS+I`l%}8P5beQ3ci5T49Hozz z8_{kMl z>j}#^dBB#3YdP7`SkO{QS(N``Buf8-cthtWdu8;*53q}MmXBJz0v2gj1 zzZJOUb;qrD%IVH!s_xc1Y#Baz(7s`VP1eu8&7x$lPwo*ezwGC34_TnYiY0*)W0+PjtZ+_#3*gDFPRsTv&-!(DQIUw@}!Oa66vF!iw)uViCki zE;L<|`exaV4#?{^9<-URy!60A30=^2pv(nLZaibLI=m= zfBGmMh-+*v_pSV3*)Hwk^DYhFy7)WcN5A+%SbDjoWfeD{-UGW|9*mnLJ3`SS%fnGJ z4}lVhL`Q{LkTh-0Fjlvmj=QAqbi-Hl&vH_GS=WmzD|#N5Bp@(pm=G*h`kSETsdJIt z;KIn{#YtJ9a00@yCM$lM94um(Ff0~;@yFzmT}OkH2NDwx$3Z7@0tc_W!5imiabQjW zjx8=scYpC<0&~R4g2;jo3knjupp^80PmM2~F`k>@mcJs8{L8@h0nlZk~!EvS9G{Hkwd+cS3Z3|NE3 z5+@}539P}X&NMkS=e0uYbL)+F$OEi9s?Xb4qaVRd?ZJ!`{oqqc$#S$2^d`&P<^lHq zIrAj#FSmmS%RE9?Zs9IIW^q|qJ*wvBG|Fs%U-OI%v8c)`#cMx%kUr{juxmNe)NXmz@7~jZyGn(Y=7Y@x-%1AD3rL_>LByYukIL{q#Qj zgHQcgPp)9kL-;(zyQ(}N!|9kj3)kFgyDjou5FMqPwvaW2Qcfl|u*9q6_?-2{;3>E6 zXqEBf`4Ub!h&k8_T4HH`l8%qi@oW`yl<)1}WP**>5=$(haBi7MTSH`^Y=e_nZkMny zAg%Lx&D(UoV@{`Y_XS7RL~H`o?FrjsgO{iuwM^~-eW zLYMoGKD9rMK$MOE15$!RalVk%4KkqHndPx8}IJ zV77`fgjoKw=`pd=bxoW|@|``zsez5S=z8VlS5&{2$d;yMsTUlh!QQ0S^4D5w@)&Gt z8U(~U(>M_K&-alp9^#(V>~o9^%P+fJ_{i7~>%DO(d3BNjV<1SZ?Bwga^tHXGo_JF4 z(aQHAgk_dkx-sX-Im6R(jLU;?>|Vu1Q@|tsS~p`ZAy9du1I}DtP>@xn08&lx?DY@B zalz(~AVHs|C!r_r`*T4&>7)KgC>Z~6tC3C~mRBpzJj z=#+;-!D9l!pMG<@34D@waD!<>YU9Y1+a&}E`pB!v+^Q;rsX3+TVm$hrqeEYR_@Uvj zXrgZShO@)e2sU4$$!=SiM?{HgGTqrWJ(g0_*@^5ltk86#3@Q*@QZf>WiX312+O1Lj+A zzh(ICZFkDpG{fjc7Y|F0Swi2I`GeddJ^!q)=?MZsp4b2EkJ}~$^C7Z8TX%yGg)6>$ zr9SY=4vvTN80Y)`@Mk%#cs4Ax%u>7slTT^}%G_f-ZF9iw8cr&_Elpt-EXd~e3a1T| z^3#v}5evOWtZ3W%+&yNciK!~C$<-29b!$e5J= z(M^`M5r4SY^ZXjmk@36-CmY;KW}Xu}NNhx8?rUE-z2GfD-mc+-P8)Cev5DeANfH|? zVg~fQZcLd!ZtF0wZH}WiF^k2w!SHMN5by9M9-ilPh==`&T|AVJfASm)x1}xC8$I%| z1$Fqtk5dtDGqb27{*d!3F)sngZIcFz5^hEF@&LC|csqw{2{AB^cTbl{L~BC6$kpd4 zx{W8g(M~_prcIw6f(h{2B>j>W+`A2d{$h`1Q26uS?;KV<)pAE=#JR)Q~2{@!ri z4Ujb#AF~)ct_F)1(iQUR<0~(}DkmgM$?ola@-W}U^3d%2<++qGP5D;E#v%J1u6Pgp z<)QH8vrmby#ofm=SqMl6P(qKa*)_~A6Xcs0oiF#tR}UXtdrkdfC5UlQ9c0O+p!{k+ z=uK|SG!e1V1_*vQfhFZ{m44*V;TVt0{ai^+d5)vOi3SS*jwP8eBqk~Mby<9{ley@k zi*!-KfDZTZ-v zk1GzRK7M2vv!<{YeN_)dk`Ma9$q8=PVvfE#`Iy}LI_i+mhl4(Qpw{_*6=Ktp%Pc9+nam^K{Q8WF;(3?}Qzq!G zyQ7XjJlyoloAox)NnbiaU!$z_-IVjW>x{XPZ?;6#?&z9e~Ottp$3262K-lw91K^1Md_98PYqFBf*;vXE$S z(!i~ejW^y{7oRLzNIYoDGjl99NTWxO*5|s2H5Lbm?(mPJij2y?p}>FA=cdqRb!kEQol(mjxKSfcKIF&y6l* zI(8LOco|O{Pm7m&ef1omOrr>yRzx370*8;PuU-z4tHmpsZtv#YGOvOJu>o6dQ@k7!M~w*FfV zATE&CkSEEtoj%yJpHZ2lCl%C_c2SO6D*DpTh|Wez1GhSp$M6&(eZU93ZGv)COMPY} zlZOmTz^GdraDy36eJ(th#2OsL{?cW;BtXxE<|23f?LQBvop?srW5<1(3oJB$Jn9SC-+1VhS6O)-RvtbRQS4~=&KLPsNcm@J zxO_2~+!{?+stC7lB<0j#%Csr+oYo5AXE*&+78hR%yKJ?)T&Ne&x8LLe^#-r@vio6y zlkX#^@k0jlnFnWY{Qa-OSI@kl!E-Wm%ImUxSrfOY_zn>+^ek}s^0&SA-6MQv-$TRg z@@nm^f4otjGh2N0;<~%aYKxkBJT3guadLAqn~-qh??lE~7s&0BedWIO?DCAs8{v?n zKGXcfr}l2lH3tty#|0`UF!a=ng}rV3Se%d;Q(_EXn0DO%91{*Z;PAl0(;wB=a;d2 zQO0ibZ8i-@op3~0VE*~!iyfa(S+a9fb?{kUHRhXd;j@*T$RNx7$#hgAbs5!_)l}DH z{wnlR%ojG@j2TbEp}h;Uwl?UQlU(e1pq)6x^K5pZh8OL>`OR-QZD_Dq;Y~Yv4o+j0 zJuuJs^M>G0fBI8>GtXiav4ih8A)zA+2^J;9Bw`P@j)-I2O5veaPAN#lA2we+|NrydUjF4*D_;cW$;QK@U@iNS|Dq3fkr$h#UE`%;?(x26AzaK z`n>WJ6UQnpABRiq;=9OLoJ+jVeWoocmz1jO2S_d@`((Rd5A4v8H$t$S4kE#d!I~<8 z?WIXC$o*Q`O=)lxP8IRhnlcg|axoDHeY3&+Cw77ofE#nlf@SV`=GNETn8e6p^%Wn4 z9H;@06)(F=?ty`WBS=utr}LkC_Bjcv3pIxg9jfoxbK#U-nm&+6|KJA=54VxCgQFiQ zRkcn)cBMlG3=Q)PSpa(d22J!4J(Mpzjoup$IDI7<4t23vQ~vaoUtVvs=O^rfoOrcm z@UTh5Ef*#r_iveiEFd@v#fh*7!GQ$=!Gnj$kjw29778M#DL4&IKA^YL4o*K<)JR}x zN}KL-m}uM*A~0ZgPGp#X>`(|aJf)9LyeR|>c}!yLgFK#dVZyV>VJ86&_i&+uAALqY zf)mG~oB#pumdpO*$ z0k@pn1l*>0YwCpX+T@oqJmKfLhbyS7qyqFT%;3c6q`0UN#sP4KX9-gW5bE! z%i{Np3H_K;ZbxuS0v!GbuJ|<%sSTG`ZN+UxV9@6;@O3r=w#!o7*WV1oXB#2DI80QC zCvv>{$J;h6n&fs&Qx^RVUA#!(0jEp?%MZH_lxIrl$1Cy%3-_eC-NEgikzyb0YyHJ< zc*}-5!H&MLh>_b<`kg6;hmJfuLX3eIiSYs^-{aDh?$DrHdbh{!%Ol?@RTX(>(tYCt~?>H67yOz-)Ko9eo4QZ z5_skJQic=1^ubp8!HE>NJ2h5HoAr&?uJOBgf;+ngY|&N6L7Aox>?>dE-xx7)_Auv= zxt0EnH{~l4I*bSW>&6Lo*2X3QxqROcr;GwI1+wN&rBy zJ`h_t72;c;nOn?vz7gHNMV$`3>wmxaZ@Cu9^K&xy z`4(&ZmHBKj$oJRySI6QDC4zBdv7vugk zbrG#}$i+cugP!}zI19--ZpfxnJ5yd^x{uo=yhQNZIaoP zg@FFzrs)ivNe6HGgM&!GP#LBJHsh9zOfuayP$$r^s4yGBPmW!iGEo~$8u)_4B8EEm zjhUPT75E6Q?mieJ796&lK_)t&6O)>t!-SN{+mwl<58r}=JSGK!f<+RO%=9(8)#cC; zd{`)>1A(O%I{MsgVd8lp!*ujH`uMg^3lsBwbv*U_*ppaSE|I^uCZ{6yyag>Bc_zOU z(Q${1jYWU4TC5umx{(+w`bQw)&vb;hw8@taPSzc?;<~vaaX&SKHA0GCUd2ES33cX-NJJ3ddNiwhF zt-S=p2AxvUU*%VHlKDeiKn^-shjl!J2smTQ2@?Eu?9@O}624BqNXj?GAwMR-;ll~f zWYN`Ynf)dyNcGM^yRtB8)GSQQEYMh) z96RY;M>ZQDwTFWBB)Xw*h)BQaKjXvZ23^qw*~BJ^2ez@*SB6;#u)s+C@dNz|d;G!r z2oK_l@MsFX>OlMi>f?q8y98hN1Whs3&oXwS>oENZwe)qnV;|gU&?xIrjDO$ zU@=0`mAX8T-WWD;cyrFd^9qT)A?Fq*UT<`eB$?>JN4Er`wZ~D*RRtqh6+$t=Ez=)3GD8-AidpKzoY%Pcb3k%&!H zpEQZ}X>XH9AT#HO9{Y_RJzB>PIp{!6kYIAs4z0<6hQ!KRj$Uct+Jroq%C7@WM6z8?B3aDx$JQ95iSsSIw|0IJv^+Q=2 zJn1dfNKRhYA7rpVmcPhYv{B0@84?2;DF$SjDyaBKeaJeLIw2o9KACQ+t9D9k!Pv>^ zP)z7v8N1w9XpNrS%SJ}#pnM^s%1|J&L3PjmryWAK zq&dpKGx1=IbJ<`%PrN0IXkEYg)d-EFrN6+A2jcQTXeYN}M`;U`$$)_0=w< zR;P_X?(4vOQU^~LP7fz2*#*_ZyOojS>AK17#-~$Tw{h*()`_3OW!x%`n2^!Es(<$F z3cGf^O0)9Fe4H$HDsm>N5GZtZWSIoD)nFz{DqQ^VQmsCONbrj~sIR z*XLTP|EP8bN0G$5BPPJplF3MQZRjH&C{7uFJiYNw3kJncGW82HO}^yX_arYXRp_KT z3op=(wdO+SdHc2sBK##)V7?CDHs+CANfV)o7kV-#(6ji+jgem-w^DCtNM%1+kN{7N zGCyE0>17z)#5Y<6mO=q6@`N%ry#WT&2VyOkeW$ z=%{1C+S3$GJy&N2kDi4IBJ#Yq+EHLdue5epH68ZobsPrdJPmd49fZJWE(v2Tx!Xy%Vg$+xV3^ zwD9coSK`m=I(lfzqv9o6H>JN;^v$j;J&G87I$i?LRqyYJHPJ0LNlu-t1XCXFdL89D zfeFt+U|(|7X8LZihB9Y_EnTc+f>YgN)S(+$x17mX(sr^i@;q*>jy1w6#`O$pa65r| zmF4NDhsM*tSAKge(CavPe^<@aA)}6^Tb1=rJ+Yuu@^-9Sq6FA0Ux-Sc6&>mt>tKnf zU2fb0l|nq8v19(VUnVNrv?`uK3)-~=i6`^ru7o`_7QwRzis?e>Sq%P?g0*CkOma1~ z)u&5?&O|-yR>NXOXlvMIkhxX{)D&h!_KeV?N}u{MR`yq;yMj8#rHXb`)K%}{otg?S~QX@;1+)lky-|Npdm4o|$ zvkH!|ZUG@G2wo*bRdmzgtf&|HMz~ErsU<3%kVywHtuvWvDpd7K0X4CHFPmi+$ivV@ zLN7RVTpg6oINh}J z7BM_>D`(ywF#-ux9B|}00l1AW!Rp`zOJ8LTLS2JlqV?^F(o_pfO=%%Rt?QRXz=Pu% zSqCY*$g_+*v2R(P6k@p*$0HkqYYo0qD0;6TJzb+kVyvK$u%?xzN`nldWC>_l1xLsV z_R6b|kIXa4=>z?}W;d)|jNwM63s!s6^w7;x8u1VDsx%+)L+_`hWc<%ymbdpYx+ zSxd)cYH@ET@2gLVqR$hP80dtXF(mS|YEq&tHA;p~scIC8pU+PW*BE9j2wVNozPWl; zj_S}t2G9NsDstKaXqIhAnrA8E7_#2URT^`};N(^|F3?%IY%};Q&a?aua{6lTK=vO2 zPnWKBK>Me zEB%0HRsvp*pOy_~d1Rt^Cf{;vGCGm%xv?#YbQv)fP|`3h7m!~FC@Yl!?oOxm? z83yo05U7YYfAWJxvft2BO! zD%;!%YE@iq@ARQ`9-nfBStrVRxVK5ig-PXzq9U!@qWiPlRW$ROIt8-AR$%5^xrJ8= z2*yi;1h@&`Ed*MW?Xm^CDkAF%r*_#bUhO4Y&5~QW!~)KH^CWYW66e3L-eJJszMk?% z5w9rf_fZrR-o}dY@MCB2{%TqEPodU}hL}Ko3!m!49IZ91iq1Zo$dnx|75A)EqKrHT zaPgK*=~wjml0>EPfv&mIDr8tLL|l+P$jP$4YS&wwrnBVw;5dlr)*J?z-{?)LPPyT?+8w1Uw)G?u_`Z9aRBq*%(!2`9uLw?&uZ>17+rG>|CTS~D&?t35 zQp1>%;J7<2Q9V8*x0pxTsBa?7ynTlRPm{n>01!P>iG~C(NrWYMS*D|aYM}g@tM#eV zYB$m&HtC17G=K$65x6k&of2dp{yzS5_15*#^Xsb$2`tV_=fs~z^qBJ?Ze?3NH28J%#pJ+Xw}U# z!__=vwV?PmvSlG6iIPuID8LG-R1}tSPeECkc!Jn>o!eDL6%%Oxl{Q{X&AyQnpW4`x zraDkHS=KIf>4`$IY1u1UJPbr7Rf@CwSh6ebS$&qdKd!0Z3&94iC(^?c7%RE5pl;B>7M=_RyK>8T4pAz zlbc1&e89>wFp}X?Hr~e5x6y8?4(|CIULA~a&{wU~7tJYd&9%hq;`uR_urQeB@>ZQF zuK}eou%TZ%srEIp7RD9MKscA`maq7r-pz>(cG zw6a2Xa4AxQMkv(bm0D`rKk4V70fU-uF|OE9q$Dau4uY?wd0_$r-a$uc7z`aaJWQMx zf2gNwJmr}g*R*BZm{iFHKaseBsM3OMybU*Vkj^F;Sgu^hH&O|HCdq}=zlMHAE_l*s zbuM_CaUy;76T5i1zK-2Ue7O)mcEu0eaiO$tNUd_3p;{d8b{3uBhS@TJL1tb)D|vU- z68rL%*8JEPzp+6wA2vc8GscTl)(y|vz_an>;xSl$y5Oy;15Pe+E#H~OAEy$2m{w{P~^IDDk`pltvvYX{FR@9ApY>XHhh&kGR^NRMDL7B?w3=i4o0(w zm*u+HSK>GkU5jDHV{Op4-HJ2|( zYw%60{A>zG1oG`wRlRH;C>O^{y-*i009^yzlZUpTiAw^s;E7TJ@?i~DH%%SfuuQL( z@OW*N*g0gtFeSh~=C5!neH>C?d0_$)D{X#9f}a{6K5(`$dD^7#*0gbghapW7FeO

8N9r1@v z-)sYBX9uYm-TXDNFzy;h1!f%8$MJRIHuHgRecoY*9rTCs_?<1j*>pN@?PQsy4yj&d zS*Nn$ZK%(P?6RLd^CIT+%ggfPhm9R~+)?8NV^E%w%Y4EF6}|~pF%9D=@kXyjd^0-N z;6ZY#KXTBV;q@u6H24{V>f+aNb$*wERBPT@8yZMi=Xc$;)U-)>R@L0JH(mfeo@@8~ z@pqbLlNicx<&f}q?Bs3W%ivJYzm)?3<8X21tHEiU)|@IoBJ`##PA2FN`qBPwP28Ru zBu8T`xcH-gI2YgBPfp$XN15d7l*sa(wX9SDAwn_muQM{mE!piJNU=*t2) znYnr`F5|$76brv?w%JC1>i;C`Aw+)UT^6izepQBKlWDa(roCj z3(7^-j3&QyAiszyKNto){3*1`3k=GJ3I7O-V_-d+j$WPW|aY1IJIj?JH zwK-6BDLgkoV%5e)OjfvlQmM{DZ@kKSt&<6o;!AL7%Kb(mkH65$583e}RMu`xeI9jH z=gmTWdoN{pNUG3#m@oIjuRl05n|H7Ln6&(s4-=TgN*r6;A`=*P7k(5gV_-0CNNvCj z-*Oh3%morKj#TBBw}oMJnO=2O1|2@&k@$BUzdBM=w`}VK_juj(F59M8Kjf9XybisV z>-y74r~lX9xiGnH!$7#xPMiP#Yn$|q58~1*n51Oc@m!NIbH;1203Vih_T_HI`HDVf z+-Lny*0_#<`P{KQBO=SKH+sXFmi zcbH3D!*k_(r~F#=XT0Ih>q2~U^lw~;@_Ea8>sn7d7OgJ*H8~Np9FZr|K@tNsfy9yVi3Ix{clQ&-|%Y9_z=~WXEBc9i99O&W;zhJ3{firuTL5nCD3MIJ)-8 z{gmLm#$SNu+QXS;=p3dof9R~U^NK&NGx^8mU4Niw4<>!|Jp@OtMm+<>8`{M^1>f8-N>BsHyez4-f|c)k7W?)5h3?{jorw8I+{1+_4;C4(qJ>w|CxBR-c` z^PJ<1uEhwfFTd;1RC`FxIvL;Y;0+VLA?K_2>nb1X@8{L`L(5(FN&b;^tn-er&-kv+ z_nDgS%6-M$UcvFM`94xd=hM&GB;T(=u6MZ3C!ErI@D0khx<^M#r+FXkrSI`dr=GY} zyY8oIDqbaVe3AFo8@1X8w)diWX+E+&&(~)6zj7SZh?CqWd4?DaIVz4FdOoR|o~2y( z+T;;a4;9z!yC8pqpRmNM&+~O4>V+)+!z@$!v&QI6wX zSFWTuB;^kouG@1=e&HnLmu&t%2l-9bSfuQqM{qD|-y`mdu{uqy<@N|_8^;dZ>g^L2 zmg8MlBh=y@4<34)xyJslM#nBdGmOn`k-ZMV0spKj^OGLN9)JeS>Rj*)o=;d z1M_h1T^r(b=g3DKKcn=WJ{}ZDKkBX4SlJ${ykQUL=D3BvTW9)aI37Fd_t!LfVF=S~ z;u&y1y6~CQ^4zx_^X;tTSHAEkrLRHnRk;JsD7BqU?{WE#iOAk8ePVAeeZJgkAJdedJ??;z#1wy~hUbpzcpM#!aFW7QTmF%J1Ut%?Z^kq8D-T&QvL%JB>x^A7!d{iN z#*D}7R^9Dc{*m7~Lce>SkU#P(Pf~S`Y{xm)b;n3QlFFAn!V?FpuZkDXObV~KY@~dW zipiFgze?i5NP=i%!8LP>U~vn6-&YwKfJr%s{OjgE39I* z|7r_|B*v(Fa+Fi?Nzxu_ZnYiz5%sJ6UfHWMe}DeEUSOBZV3Ih>w3&} z$H|Xma8~Qy2t(X;Y#%N5$ow9`QtA=>BUqotL;7kxD$yg@l3VA}?W>$8^Lm}q&l&TX z+%x=B40F@4Yj3#sA2aelENAbSBts+WY}hlO{38s_*qz*tPmLlQN4kusg?d`r35o~d zB;{|iCpot;%CT6-TYmZsqjZ;vvBmb-Ze9db*FfRwrKR_}iXo+2Zc^sMLA>Q8zj%!E zuAXWLQ*)XfV(T3#-TMh&zab6(vQCTVjjY#~ZnE=m!N#xp>OuaN6Yg7ozE{FS(Xdsx zKY0KcpL~*92l9GvA|;Q}WY5`LUMgW;XLB?+0uJ%DU?p&Sr+c1wns0}5Ug4?_yTyF3 zYj2q*j#jHt?bazmeO{?Ie!h#y7ipx8#Nc!P%%1qwcZ8FBdnlY%Q>yT?k-O&7aqCwx zNzdy#XO}+w42PZb@1Rh?I8L@O*o=@lKiP^G*?V#vdb4|cqr91h>6GS6V6%H4jkC|e z?ZSU6WNX{-#Clvlp5NkAL)g{lxzdGm=^W@Smir;jkUB$k-<0k$;Jxi#UlYH;32Duuqom)f6s-d)ks;eiQ_!iGc7Lg+?4QNuMdC51TdTY>=^OL zCrJ(0RkQlA8LPf(j<{CxN)CML2c4c|ulfV4v8wA_D;!)NcU^Tx@+@wY`!#&t4?06@ zdS*R6=WKsoQ#qAK7)fvyqui17yiweJk6=9yUAdATANdp~#u%+}jX96&;*wm&ggKHU zj*;((XBDq})n4U#%=27#3_3B!Xst6~4@W%W5Khu4`k8Pay0Az_h$ly4R|Hx1}TN(Av0Pc=@U=|B+O^s6pYo7h8+(*8PZA@vD6)|E%;rM>*=l lrEsh-zgvu72*}@@`wwB7%z0UpZHoW^002ovPDHLkV1oJ_CSU*n diff --git a/docs/apache-airflow/img/subdag_before.png b/docs/apache-airflow/img/subdag_before.png deleted file mode 100644 index ebc3e589b2221660a01865b0e54e33bbc2ecad73..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 70382 zcmY(q1yo!?vn`A>IE3I1!QI`0CAhl=cX#*T5MXe3cL?qf90qrHcZYwzd+)pN{j=80 zv{YA}K7G2+uIk+pit&Q5D+Loe~2qXKtP^=)3*RP@bAxlwRP|xjJc?sCX<50wN#;0^$W+6mSFq;mQI5abgGo!IugFf#Z zqS6~ZTma3XAnsTyy#`d;MMke;ZOqo1v9l+8M5CR^2;H0goixH`Z zt&N>CpNAm%e@XCx)Bh=ECMW$b5f^Jga!ol!QZaicQ&LVQ4ki|IAw*JAQUNCuGd^W; z$^U5%&IyuRxVSj*F*CcnyED17GubTK*}>EL2%Z%6tcy+*(6U0npp$^T>Me?R|yoi3JU|JRb8^Z#@U+(G95)G)I$ zu`vIyZg5k9|CI76I9Zy4E&oTq5Uap{N&f%J{>P30^M8#0KNs`gJ^imza94#81(^T$ zwh1B1_EBp=K!`y66c<(XfIQ2BH&j(y2tcF#cBd~sttcS^gH;>61nVuT^wk^3r0TTV zR_{}7vHn57R^tq%{TFl*;CJ#yMv@3A+1nvMv-=I(xU3nK?U#+v5&D)K?-jwvwO6Yu z&AML=Fk}HD5P=X-kT6gH81x@M0fj{-6+x1)=m1i55z_yzg9{WDohM_xX!8@M4a~b(Mxo&oC9ezJ+ zJG8m#M&+lllPcHNbTL;r^)&6(f>ttqE+QfvcgUTZ>hp<#X9r+R+V@9zBd{#eJj8Zn2% zcn-XjSuj{r(<`3#KW<_xl=l)W%TU_%cuwcEE-ap^Y+(|u64z{Z15OljV;_u^zwL8n zs1sEI2t?6Q3}eIcOa7oqIBtyPcYlQVRXSU8`Zct>_e`1F~KoyDhXw7LpKqYD%)+lCbmk{=JLk@*!M7od(pbX3So1&)t z#H8gRj15=}+uXZe+-Iifkk!x#;lNnM*Dn#yX>aMmQrqiZe7ZV0mrx>5)p$6XP41uX zbY9H=`(^)Wr`f~)_}(_ekIgTG%bsO0(ga{V^yO4r4pk>PmuuEivN-0jEl3v&6#7S< zzSF~pf>e@*E#dD~Y1H&?-ZsLPJt}% zgOi)5&F4$Kj`o+ELYcNb`n+mpP3n7gf^^WJml`@0U;r8ta}Mv=OlU#{vQxCm>e!DM zUH=CRVRS_I$KA~6O-Y2?<n!;{&f6a%$hj&V>`-itbzWZg|<7W7xzrDSJUu_~JXGjU?5t|GVP#|a87m((XyBrtI$7u#PgJh%Wk z2oxucS8p6dBUb#ymGXm3bcHS$0*Fqf2RN1ETP|0lpUAGL z6irsVRv+0Xm51Xe1xfeRB9`~v(kiagFKOtBqF1*Lrnfd9xY^U6rVUS6dzG}{i^jkm ze80}}#?-AKfeBKo^fLxpF`_!q%u9Z|eqagX^d|(%5Lq%2xp+>14W}k<UcX>ZRR=U=iEBhdXPwAENnG7aHNk(@DLMkYU+|lTd}qWKci-%Yef4 z>(?XfzdWTA^HKzs^v~mz!o?R~*x8ng;h&)Zs)#Ua&EJm_ykq-Agm#P4JP0iBYTjU1 zg0PpZWr^yZ1I#Tb?b!l-s)gc!GAV7b@Y9k{QkETbX>MwV0y;6da2T1>KVaSop;|?Y zqZN6zsjiulu%-AwV&4GdlU4eS4hfa|O_I&_-C0tVu#{6o?xtZUGd^~b@}99yl?asVXEP5=ZCayG^qXRJ-$ zH`MS>oJ?-LN}3nTkoxz1Zv)R8kC3QfJ4D`U)zzfD@EV(ctEAm;ZxxNIUY08zWi~vXpH}Af#Z-bwc z1|-K~wjk+LqyDCcX`x-xEozXEAqUsg%F-)UPmo(#0wOw#r#CrJX3 z-e0W;4cH`Cn_WhfN2&`BNpUcKp06alD(KX!_S^*LK4X=dZe53G|GDf=GqjZ;%Ue&= zmi*Q;`{gvedAr|T=npG%mtAxpuK0-AO9tT@Jo*m+RID9u%rvvh4lRrqq*NY;1ujx* z0a^7{d6e_LVs#4h`c5v!&J&qH_w$Vo-sAJ)pR8nCgTS`k&kySEkGBkyLT_hki=k15 zd`1led;yWu0_2@{fssv|uP~z`2m!^+e9j>P9*_M_L(|TC1&_j9WOBUKx`uu1-OR#7 zC3pNAFVfx-&H~{d?LK>y+3t<7xYx)AxzQ=ve!M1Rq;bME-Bs0uR7?%#lR%wM*LDW2 z4so_70i>dvKxNFQiVEN(r?w(L3`7J^0Ev;2)C3la8bhF*emv6NuE4RH|8eiT7p#%z zCHmJ{R7YJ=@s0MsZ!ym|Df?qBKIo_oL&GHajAKBi6sUJ?g%)DH{WGK~^sYTM_Zx6O zqTflr>+a*^_<+aSB|MY=42pjt*@@A1020x=Y#kQ$EeMS9zQ7UGcAF=1>`r__>&^FFk@}fQ! z_qtm!vaok*rq7%u6}sN5uFz<%?Oqz2qty>3cU%1?tK|EcBZJqYUNXIf8Xfz$-yMa6 zPR&0(MS7{G2ITpVpmT~t6<$`Q*9SiHI!=D>N8x0%e$V#Z@suH{U%+yo;N+A&X6E&z zlUH@Vkal`bpJf)a3MX_v7-ucCT4^kst5V_Ss1qxc=z~O?;grO1UrDmO>1`(&8QrIs zO(o_y5;^YFVHT^-DVd0GG4R&_9?N2}Ut$a+11I@i*vQk_93y?6Hl!qT?@t#Kd!^9o zIqW-bfa&cxW3zF>=4KWg=blYV;Z- zf0?@y!0oMd)JJRB{Zz=h3kbaidO;l$&T8a0F+6&!ay@iLl?FM~uS7f|+V%}G)Z>wS zXD|fEWYy2N*#A;fHsh1&)kjt0glxyltXqioULALvNQ|p|lCpwP`9<8$+oRYLIB%}u zb7(|AO^WZU`P~xwn?6~H0Jy)#C;tIqqp<%~Fh?iEX7T^qczEIWGAoQeBm>bO}7A4c_9oLd2q^^XC9 z=N0nfh{Iivb#D)2_R`)NMVr~ara6$_WdYgWNIKXTK<>+R4QF-k!dhKEt#f4=jH}J- z>e!{rPyO!0qd8Ux!KLoUvtt&xD=&6yk3+(WJ|&-b4L9*#P6Kt}1e>&qm@RBVIFQm{fs^Xk96 za1sU?9OmT3NCw+u?F+kix8y+hOY_9Ynt8UXWvQHuT02l<#5Kf|WScfzBkjh3*zp&k z=f%-(lPjfggyU%S?XH*23VwryYCU>rH2hqj`qm~SYr@``f=AW66duh0^}d8!S(i0< zuOn|cB+eQs35tO&Hbc?@8RpJ!!SHB6DPmu7^-9e=+DJ1SCCS8&$}}deM)B&dyPWcf zMPQ;l{|w9_nKv32CR_2TbVXus2@w9b`^0tPLBVCpLdvzlB@|dJ|fm3*G$|`-VogdMHACk{FC-!{}Mn#bFSA6oN4Ta*&QM>CUT>%pTS^uPf1U|Q=N#A&u zb7pduDHJMMir(t$o3YqxZOJE_-URrcc1#s(sg3qxrZS z4r;e5DxOa~GT)72>s`lK%rj{SMvv>sLV_uM3KGlnL&asGN{YOh{BR*^e_6>W(k^r+ zJngl<*E41LIN`xWNAh>MCB8J>Uzq}swz?vl%sS1wejm@~1m-W_9i)E?!uU690|Lah zpn?oP-kzQ2l%z&H4|Av!spQyF(b1};5M%w(9}g<*N1PS93>%|&#*wjRnMPTqk2KZx z;_#dr?>4);b^Yh%sD5MMD+s8m$|<7qoBc)XSd zow}@wrN-`!RuvrmBsrunH0DtJkF1V{EkmgV7muj6C`|53LKRAa7#cIK2+T;qal*tcGEajTp}TK z#<-d-mTO?PP$D)aw$<&g*T@*g=PxjG5Dk!w!T(vK-?_K>`O%{5zkbhlwr2#YHyulb zHW4pBAb{ClM&N3L;Q?PHSfj|(Nn z<1VP3Xgs;yGv|KLFJt+pQLa?$lF7$Ie6JsR7wzwmc-;HW$4#@72QI*Ov*xvYAu4Fp zC5@Ac#Ch11Z0eFsAyw^pd&{7Qg$z6Hv1L$@X}u;j&Rv>m5s*>m$LjgY3R7atGr|68 zO?Lvjs^Q1&f&UV6=FhQw*eC|ZJ5arv%(C;Qpb02WMKz(C{vAsaDTRscW6^t2f2f-c z3)nfA|2u6HLF@>3tVeCpfGq03d*A3|2$AQd&U`#c(C)?@)KH=gw^*&-@Jd+ZIlhzO zTCX^KQq&3i@~jQxVw8FVQ`LXj3`mcCyoy5`_OlT4e7hbVE zLuR2-{W|s=y~-K4E#4sR) zEN-LbP5cjj9v<1W7MyjN3*-5WlV-?a`tNRj+B$T-gwfk9=;j*G_1zMNW1sXRyA8GW z$FTN%>u>||apb|oI;=L?(wk?w0)1Z3Tbw@MAIG-CNtnDZT1#RC7rw5138_MZ+QI}9 znav8nT?Kc%UbNSHT>rxR`zf+q^vDDkKaoDfF%Q^+oT7y$W_!z={Tc_7`?v41r2kUU zV?vAcW{!tM{ll!R&`1w&mRaT}5nq!Az7<95bL)@-dQh5WTD{#nUlD7OKh~KitfxAf zbz6oStsx$G5gx5euNu)YZkS@yDr*2*Ed$&H8(wNvgJsBWtW}-&UMu`QR}(|`7*20<_Ske8Ex%8 zUbOl1t`~HUaWpWgA*AkF>YnwCl!@4ln4JIt?^}P zW_KtihLlu%FynXc4oBF%9*0CcLELeKGyo@uSNXzYiwnykO|o|Q_zaVsy$?>@Y(e08 zc*N{ORHiaTYw`EZm_m2Gs|5_j{-PSYYnaC*l|aKfbYUNDsEtKxGe7_!2pZmVh>p@fD(JS z4A6(|f!z~4CHMb~F>&N)*04uW6Z=S?GAPFIi^&U~ew}Yth;MB;NIwvenlQ$BKlh<% zCa~+2ha+Axh$Z1r#Z!lG0Mx4Of?L=7fw;y`d9pDqGTs|I9bx4ZNEN_Q(fy->IG}tM z-$*~2aOJ_3snY<46igo&go((){>3xt6AqWDRW|_)#E;B+g@?*j` zyJRtb8eO_8x9#!@3hLCe)!W3(|5J8Mb7NlYfR$c!P3NeyXYLQ*g1#d3fHB3lwLZ0^ zJ}|ZTn=Z70&cYMmc8si2PMODjKTYdFi5jZ))kQUQVvx#ysHGZ*+ zUl%F<5SRciO)BS2eZjgKo%QRx+!FMrMSfz`|0~}0c1F*8{LkoSVc58mM2grwMAviO z^rr79W$$A3g44XYcp^P*5LE`&2NP3}Uj-dRY_F5(`6Sy`}k$8wR`0 zVFium$d=f-vs2-!utmxVA;#TZgMW{I1&9 zy~&5iTLe9EzV7TQP^v<*vcZ}&!?-%T;=7}5-XUX*gCx?r)`S(sBr&B5wF3LE)IwA` zjkvdP#JLNqXd@_?xK5{__|UvCSfrRSYyhXKnI>+Kdqf0pwL{}!^zrw^Vsf! z|NCO*m<@G!Ge%A_80q45rlvF^>Q;JScD*ZY_`ix}WHD>kYi5(Y=gUEql9(&Ed@eU- zYcb_D@r{UTFf@rSRtmGv`tK&U?7K7}Hd?JqgE8#HT5B72wdYa;O81f|HN`k6Pl^&R zn55w#*;<|MQYRNP>E{VxQHsy?$U;40_`<&rB&&}3jH2;5_VsbyO$ELo_#u&!uJ)K) zAYbh8p*|S&TFV}u9uya>>z2jf`s?%?Kh0EL)l)YME5bFMEd;;lTL;}q^r+XlC}b4H zuHQK8aH<>?X>f8qako?WQ5f$7D6N$jZ@m-e1QX7oeKE4nCh6)VYA}CKkq~7#5TxqkcTe*05S+|9myuw9jE}41D-TG@_-pZ)P+InTPe391d_Cm1zSUpn4B%Q73T$O!T=kCTo znk`U%xslu86Gb)dmff{8+t%Q-N=hE6<*rebxVBPX0^IX;F(Q_jyW-hO0J1fGj-CM?$;lkodlg1+yzI#vLuaQ1yXLNim zmMeUhLQ?GA;e)Z@ec9Q9jL-3JG(YQ_9;&)|OPh$a=Wj4HK%Ie3LEt)s)&^jS+k{kT z><+{8M<xYl9yGBFMk$Fi((sNW3ULiLsGx2OUzAdGDB;V zo#nYBhe&&`yV$e6O`{Ae1*>HxHy~;JmE=s zLv;i3)2x&IvFp%0hfJ0kcM1LG9s`nn2Hy4AU0!UWg3z*u@Uf-8D+;HyB0CE@*=sS0 zhY082Sp}Sh0+BE1I590o!C{hwW2%ZcrMYegRX?ErZ3R`n{TmYr<^jAbW`4<)OoRF- zi?Kr*kVyjvRG63CxGg}|pqdJEkS5$KIvZc{;t5zN!B8*VpXC7neo>=b>JnKL){PIU zcRgfv&>$p)6k@60g}Y?NCK<$EHc7p1qZ1w&i>{i9=UK|pGeJQ(1g6z;LXW{t!JuJ# z{T-2HR2lx4Y@Q(DkDM*B-|F-6wBt<#?J7GaUsDp4{=Q2vVKklmllRSVV5^tNea%Sc z77mrr!ZO0yGHML@qMRKDR|Tr^b5=n(O$@_2$pcU96^8p@i$wa{Dy}oqLTRo(o|UoDR;@;uR7%*Xw8FHhW9DREvLsI};H`xxtJ7mAu1H&`_uwOlkx!K8g+gzQGfyNZNYC}>aC`_?)u6ZT*Vm?d=2#Q zi@n4(2olDgmrA>v>HXHx9V->Kcsg2Kb$vy5B6EvyGGC#A_vV2{Jf6-umee#W&o9U4 zc82>0av{R%sbna%3y(A&Tl8)>PB`5dW0iG!x-X%R;xyhThRa)}$?4*FXs45wrsVa7fkjsuv#i zpaa}3Te8B$V@waIcSChop53ePG&f5}f(81RuQT>EIY5LQI?Y4*k3k^i5)~XIquortgj;T}(CeA;RM-Hi5&}m|x5Mvf|0IPg>ynEup6Va@ zSVa__Wa0Yn|3*5;#yW*3HxiXrPmAc8*ad8#8cd_EhKqK!wXaH=ywAIl8eKv-Hje&MidEuej{U zZIkCD`!;m_SnuvZwT6(}I7ZNbhp6C{kBR8zMrX_c4uhs4k8+k}mn6Unt|u+_>c(!i zIEKh>D@Ts~iRmxDyX$3vz9_L2I>r}?D5TEC4unu#S4vg+KM|HK!XYnt%#7J zlntx&Bu|}(h##jfJ+c-p>u&6DvB4vtqL*F_PG5EN{3^p~wTzv#iN5%}+ru1x{T5=^ zIx+iNr{2op;L_$=C9j#zmFSi@241W-L^Qxlq(T!MDpz>Ik$~#A!sV_kSDRapLXbA@B~%A zkOl<7O*9bpReB9Y%}0}0vwGUjp8V&rF_A&L!ITaoPHZf#5Ugo!)=z@!JAynzyq z@=v_k))nYe8fjZ^7uzMuu*8z4yt?L=z(SQ5`tyH zt#bO<0`2#(?Dm+x3tFqezfGnOnvOcfvF}uFT7CxU;8iWz!EsZp--c($A`<@MH9`-ZrN;5-yUqfvjtlO>kx2v70n=M-Clh$ zJhk4a?no;XTGMzF{4^R6kZrF-FLq)&$1Q>67smk50QJD{Jf&kB$PT<^B_X(w{ip{e&=~B%7i|s1>~%!DI48yzE%E_ukVyczS+s@hz@BfJWru1Uf{16x8Y(*d~2r=cu?N6#hukq$*h5^m!!$I7 zNI-)DhS1nllEdvxb2V&4;}7~8jlng*Qv1juK!DH-RO^QMlUZ@fQEi3$lwo39|7?SY z&S`OHj(D<9N`UXT_+xeKuQ#TcOT{jFUMKAuSy1cKAD6$TGCy?)bu&D6yP4%NHnTM7 zJHnmF?>FeFwTU|1$pl))605(rRnwg`&34#pHB@5JvMkV;MzedD^BNQwbo({K9Bg8I zrL`~btY{9x@1qpStyHVS4%A8vjKId_xoMq^Ox5C6{(c`{z@ZweZdKk9nq(=Y3$o<2h4&6|f$2PWU=yB736SaOD zGj`gkVz>l^EKNwoohjQb-P%(X&)WEwHc7vdydRfX`_@yE4w#(45@wT>>jLMiMI|fH|62 z*?)Rv_4{qnpgU!%Ga@s)r6~DQ`bw;SeRx*E|HIBzQo9U2_dpwgPpvNN57L{{bjKIga!})O1 z6rUaq{z-S2N(WJ9(b#$YulGaQ(YXuHaWzxXXPWo-Z|RPX`iRUi!ZqHP-9T&w?UW&d zu8PgU=drs@?`uD`L4!zP(>-th=Zg(Jio>gou#)ZNL2)bo>&l;EpU+;%bce<5j?TRA z8arD!6L+`iKDog^iT2#q?>8#uWDaE~@`F|rjYjm|^A3ogBu#-!R zG3?wai+`TpuvPSZ8}j0WH#3xW9o5foXgq3yWi<_iXrMeIr2nU5-)TDTHYNDTA&G*L zVw%|lJ-UCmEso(N@wF!6AzxrD|ANtydRj?=^>D$h|vmirZC(3hY-R(EW7J+HYC*5rD|JlMO#Wn*W&JVGFNu+JmSU9z9%l`ieD-I zid`6o0%>N9&eCAfrQ-{}GfR*NXM^VF^gS_4R~1K^N11h^5jzb{B{_a=jTSR~LS>r3 zG?zj*cw-_2I?i1pr*dFp#)3uSKjXH?~F{cdUAF zh^pTWk-yNx1+UyaIU+|{D#9~)fne_+J+ATD3Z}3~4z^;x8 zQ6D4R0q}iaq^B33(sDG3md$RBhxg-YcLHXT4R#j;Oxhd~=}GJ6rGdZYmiBP%EwKu} zM`ts;Ty8KYfF3j%;rC-%AqDfB;h@kNsIG=$2r>{d#W)>Qkz;Y?;L=A917le|8xn)7 zG{I5Ngl5~-74^!KKGnoa-}r`n(f}KWAiUd068|?NL|$OV33_>QZrE=!b%t4UFX4YjzT}oyWi){MIk?lQsMl@bbk7;XVrniOO z7qkwYNj3o#Hd~Mnfaz=786v{PtOfeM-!C3ab9-DFdgIXSz=dro@QOg_S}oNY{d*yyx2(Uzou|9@x8KsKSFMx(1l~7zOMd&CucwAdui&{!z2U1N6lhf8^tXnu*PRk-*i*sYsW@hIK z6R4O#QAYTm5lGEiCirCr43O1Rf`3| zEgI)vU($ZkgjM)>=z3{k*yz2}Csx=vGfHV+$)n*`$fl=TSs|PwgL9k}MSqT;&HdJz z^akeX^G1`P*$xTgeP=8}j4IXz;0y4c)%NdIw_p4LgJ^X)WL(2bm9)~`V}tTS$XG(A z-R560guI{-&PgFG#zMT^$}@(M(eX%|kGtLCPa)zh37JbUeb?Po((7<%jWe5r$+pm% z?KyD-Q{TOjEx!|O%HR%pat~)OgFFV5NrM((U)vZOH~VzGJ3%t|%qLM+KWdKm=;eC)Q@hP8{uF+f+ioM5O}&Yon=k@!bS3c7alO|52Hv#t zN!Vx=sN4DSa~Jed^HIP4`r}ep*HU9b=0V)(e1GOrsl;5`&8wN#+6NNFD4iw`xFC*$ zHd!Z$SY0P~low9?Co_}qmFx#yxxR+KTsL1>tIx9&yXP)ufE1IorbOSuHKtHTL^5jh z_-2QaxvEbbAEMotgde|w8=o8gm!q+FlY_0+0+zZfKNbPcp8FOcf09vRG6fo@jSLGg zO*T%mdttk?hVr-xWV@Cr!I?n$QM-^pWQn3J1bh^Q8qMnn1 zBUhYA?hPg~USFeD-3e*o=x~R7xu@ua6!!dv6e7WNZZzoy4~3GxAa-H^eIeRfg4Jy_ z6azebs1M1E3yREbN942rpfNpeQb$saszfwz)fmR(GV5a5E{-^!K{8VjoG4^^imA*M zx1WdzCm?4y+VU?@KCU%DZIRRB#l)AinDLjO0J%Ah2mzld!cEM~QI_BmCqTnWeHKmCF3r9KG}_uGcXKN26!ll9*7>dg)=tZifQamFV%vc}|SEE&Py)6Kvb6JTOe;szLEQFaZRkk0A({X9T?r%q(K%7W>1r0Ieoh`ws0{_5GT7l+aTb!sHyD z4*EEH(g^if;YoiIMqhzT+Hmy$Ae*9=XbDg7$pBF6-ya(OtqD7xhu=MX3LiV@M;ls! z)e81Z@tm*~-?;soXpgp(@_V(MdDyp84Ezw}UX+uwRrD31d2q>L#%+_wv%bm`T7t2Km z_7%b#gRLDGS~Zqd4=aVEY8%D(T(Njoh&4xBS7<7L$Bt;t;zTl4)nhHxa~jfzCAJt zrCGyh(>A?V+xhE%c3Mjnx-c<;=5vIU1un_wG0%NHJV&7cr)s@)s<2Vlvf`G-cGU6C zPEIDv^`>T3MTCz+Kj*4zZRWo#^4r^+&1KF;)`l)YdUW){i6nY^PLFc=MMG@V&ILt8H>t{Q(Q8yUY8llpWQ|(5gWuSdx#kOb%gpQT?)Jya4J8Pc791Gq z^3C?us2(EdoP>FD&Y88K}`f)4*Tz2d)Jv0oCfoaQ$l-Y?YyPKt$rM#|`h|K(Vc|@`#kO+5OCA zgkNS%nL=Ut{aiAJNOBrL5LUDrFPcp>zphJPCUa{i`fG zME%wjkU=QDXMD(V@EU3?N2oO~rlpuvZ(k($Mhb)63?U3}`#)raL~4cR#IYbW_W|ym zY4+%Z&Wg+K&n`mean?Q?IP=cYFQDrtjvs2XWLq$t9w}f45VKdULOl&n5DTj{31=>~ z6_Qj6>DPlKfw|c(PDcc}8cU-5*I%Mu@Q_!iy?wN%nc86JRI|MO0XVJK_R=NdygrfL zuwJOFbG4jQzFcXl@FmP+Xw>D>RO_nOXnufe9m9kInD|YzjdE|sIS9$_YbkgpC4+*} zQS+}I#tA|uyIO7x!yr9@@FzA|rlhK*c4iBmUibCP0*+RAQ|P=N_#uDLA@ntea`&f% zUdJswz^n^y&1AJ>CM}hGg}~8lngJ_quf3naa40rk{^1eXA;4wI6~)&FtzdAQ*_(kq z+6)fMKS|5{gj;2tFxyo~*(yX(0K%&;r;}vigul!hz_(|k(TVz>N+(+KvXDg@9+LUp zE^8bm6xvHUw`WQ0X!b9sJhqvF(UTx`jPYsK#2zq=3hz=P0-W%K!-_zP$eLDKI<948 z)Tw5bF-#5(|B&oni!-^%ww4I>IIVRF)qfoYVQUz4n8s(Psa@CX7-HCGNXvxR4mqbO zfS<@0Fri^p=+m&X?b{x%%0sJ@r10jWSvN;`1_ZyMW>enDm5mHOE&sYiuxTF1h1J&6 z+(b%d?3Hm9o8EQrZ^+nE5PZ_ccB2#U&HAIjqW zr0^DQruPuK*#a!GBTPhq26~7$Zkl&2Li{&!Q6cntH=z48lB*B&lqEZVp z1w!#JRZRZx>NVM4G)nA9p$i=5Q<;pL{_GpLLNGxOGQ?o639P?X#%}8KA7VHNLjP<&u@CKM4OKG; zUDiWTbI{w4LO8HgWh`oeSOsv0ju8J5el0wDCVUw3dp&vkWxKi~oEOWEE(veeFF`@* zO_l0{Wu)tOSnciN{*7v0y4qSuKEI))KLR|lNA3jvvfG}Im-q}41 z1_I4#f|K#2w+w$5e&;UY8r*l+@P~Vx)RT##Ai@s4KV2Y*aj*kB=YAAOhs?+`CG=x@ zdTAG}d-H?9jz4gG+TB7^M~E*$Xb}D--;&0+CyH+u_O)LB*^4q$p3;ZuJb7ocC&j&g zoRZwk7M2ZpgOZrVcQp#bV=WHU9&j*zvO!l zKPnsGJ+Q$pgeelU^7e9v|=Lef2v zgd^F#hhdAaT}j@ykW$2Mc+>t;N-t;ht3iJ>#yHHT7Bu<6;aR?R7nfnWANel`pA1P% zugfO1$#$ks!tDeY%q=dyIy0{4eMD`$Ehv2PY)iNFZA$n#NcP3C-uG@`rRwrHN;p1; zHO#W(d1ck<<0@(|N}>CY^@m&4n)`6D(kh})hoZ~84D0wir|P7jubKLUO+wQmn$S~E z<4x5%461w}PUcI<2*FaW12ldA*eG7MBI8z0zvvN1!j{9lg@}1XH18!q*OMuhJ{$Js z7DJ)?Iq)$j=3Q6lc}rz1rJL0t+vAEh?f%dvs$lJpDfZ$pH{$OCG+ z?z2vXJtN%T0=cd;i+#>sYlg=Nn7b|C0^BoU4i8E6={vjkKKBm|_UognlTxxhB~~BH z!RH;mr@ifdwJjLt*5!YH{{9#(G2;=kG3Y6_sw>>1(4{mX#b=>1&bQ0Z-SH=RFMD6j ziZ`i0_Z9=S*~}3?DWi2jheHG|FEvy}^8MwWvOP9kg|war(easd~i|jigV_@e9!e* zpo1w4X54z-pki+xM?b~S`yn|Kl}F2is+O~2QPcM*65Zs5xN{>LfA&aYA|pGI|1{V{V?~abmr((!(HI*t zDGZ$Z9ZI`lR4#Qcwxb$tjcvE4%b@*8}5L?mw7jy&_DfdU+h^NgCV(0}Z-gHkrd{PbZmqw61rlTotYOnlpsZnO^b(q#KEoWbxxqI(T< zXA{KS5UHih8BVwiquUO!3vKWh!isLkol&fcxZWlUm#jYKPR|DwmwQb(Lrby{Cj1YD z1*Th~uW2lOChu!1%;hYTYNEOd|6$ruK)@WP@bQZ~!=_n|m6>Ogui9?PXM5dQrsb#a zOf0UGb8k3hA*|41`u`C_6V|kvFQX)>@I0B&*Z>Jw{5^I$i!Uew6B=n84@XyxH|o-; zLWEpzIF{?YHSpYq6qnuFZ6|ck(W71fGyq?>ON!6Vn$Gm3E_F@A1VB+e0qFa0FyB3K zn8ZW*U^1Pa+rT?Y*LhY=^513Wb08hI{g3sRlcEZ!$0f`-;5!;|X~X6I&5d(SdTwah zp3A=k_DKS!iFEF|>)`kX_WJ$DH2;6l?tFS|gc zb`;0U*3@{O^OROE+7`ycxNRREjP68N)puWqPGO*H4~PFIaw`S7PXF9HxkCPBzB&Wu z2=0N=eTo>GAje$wsD%`{ZS_&3rbXB51wXrY96sl$0~CH(Gd+C(G|X37)RJv_z4ptl z_OsT*?B=KegeHq~$V+rKqyxtIlw_cP#&yPw$m9P9DM8l0p55fcQ-?^~cH7GQdGqC^ zXO#|AxY56Z1|2Aak2*{msnBJH{NmFu%gjG#nxNXd|6Vfa=)+{fw?7DiXv6xAvdy-w z<$y!>mq|ZNmM%MXHh%Z}Z-2#GX{`VNKmbWZK~%`|qhD03+dNZLck8#S9Cy+&D##nl z#~*xRvrm?brceFZ@Ef}8wOcQ}_Z+ZjxA9O-v+A$aa?9N}OSc|7O2e!MDyaL(;l~b= zC5xAax(%F6sZctG-k(#8SY~Ez70C5v$SEgi`)A1`_dRAjW}{j}`l=#-+|Xmy>Xs!R zzWb4U@b(8vqY6zO+tbfERd(BbS6iQk*I*gM#Q)^Oqa{yG@TZ)4vMyt08?4AfJDJR& zY@cw-@zT0gYf;;W95VPIx#fo2G;Y03f~uM#!1LX3H|1OJlEd?UHYUDL4wDyzh z{_mgCTM6ag*ZxNdV2S*p>oZ3mbiDN4qmK%qkCmygpg;^e_H-l4tFOOOh5l~x(A|&d3}K~wp=Pwl9(YPFx$Gj-f+tOwZ0#vfji0iy zUd)+4M{c<4U#bz;F-;zGy~Syr#KUZMQMjFb(OK4x0@3e{E9^!;@{9l*h?sFK8N6rr zet{k>`}!NN%QxSAV;-BlM@Uc)0XVp{CCe$_C)x_5MqxYdxZ~vGk3Y6$8^(hAE29nN z+oR-41Kz-Y@#4ku>Z`9BELyZ^Q7+pm<86@}bBAl>!{q+VGtbBgC!Ao%m+}sN5^Y>~ z@9^&xRn&j@?gzQ_su42s%8TW_w?33_zWml^#?L(QoGe_hP_Dk=3c2WS7pR%}DVxQv zRV&=X_dIF}^@W$7FXvryj$A$B?`GQV*tvrorwZ?&qYn{;FyYxJo|nILX=>H#RdUOX zx2ccEY`Ngl^Gx}@?O(Uc3s1kK^sBRM6|mP{euHe6)6VFwy(-A}-SL2|)!8nz42{{f z2=5L!WS|^4+RP9-_wsh zYs&s1IvHu&Oci;xJWwar0LFkB`eoN#DxJG@(hu%zl9jnD1-{lvXVYfEEHS|_aOofT zfXa$mYTnnrzx&?1<_Cf0gu0#3cBx}sX#+UJc;mDT?z(cK;YY0UDk3iBX4C);!Mb9_ z3KOb1s^I{{L;~qS1T*9O^pi}SIMK2reEC{d{DFflFnO&sg5GDpedPJqRWohWK;C@y zZRw;0hv^z2*k=zFVoI13zWY%YEm&l;!sAaqR%eL~RZDIu+p6}B@Ij03zEd|7itU=W zlkxI{3e@&mPk|xlmDgWx+H<|EdeUzDw({#Qzsa~SzLB(zYnY{&5wBgd)({^PJaqxbeEz9`_Cm2riWA$46)Uwh%B=JsX0U#H_7Ri;H2>Dk zTFdgLE9Ap(-ZdgkS7F|?c~j4fwUKqkG@13sOu;PPd52E&?>lZZ2F-dFro8XQeXpjg z(FUK9S6pK67ON67clKOsW1%Wut+r_;+qBt635#!w=garsj+aZ$zEsY+_)Jq$P|gV8 zM6>`iARwp_j(h@@4?S>-9cE6-K!7?zj8;y0yr-QgnC;uQw{1uWD!yHf@CFxS^`kuM zuj(>jYA{F?LO!^`fiQeww?RGNmqayo1;cRzB6nW^W@nxp<0FUlPc+-9>~1kPbc9%6J14gRb%-kWZ}QF{06 zB^x(yl(FA^V6)3kT{=qloxAJoJWqC3_$*$!SZCmkrCx)2rl?}Y8}s4I(zSCJrA(2> zUwlMvxcVkJ{FuR}TwnH&OXbib4v}Vcn##H()QmoDhQXMb|J=FrBwg2tS$=|lG;7|> z1mk+0nIfQ1I^q<0;Hi5BWr0Zr!WO}aS-stMZKb>N0r(Sb>CvOREL*YM6qF7fbIh!d z;M`GX=Nr_zn58_Hfa#l=^NFXPU@$`XGuBwe?z-h}8Ta`&^4RkaNvDpTWW}nLTIX!D zFkm&TuQZC%(PrB=Ru;T-^kdJM;`00(qgBa=kMjYZ@Nx2<;a!X;ChJL)CfO%w;g48O z;0?|vA|Bj>N3cx`<$A$_1!mo_My0@?loHesnFHu7HG5<7ZrQS>FBBarX z_=FzLEV$jDefHUM(n%+UfRBKR7VHyvs(~Y1Uwr#HGuxan=p<>ag5}H$&M=_~GkWjM z52a=6R_3kPxM^e6K>O(+XBvWb?Y@H{(5iK+UFti`zUl&^hd%sfvuDge%plPU=^KOs zBiMPDo+b=eYd#R3Sv81_`H1zeCR&dgcV(k05O5G?rkLd4d+P(80Z*|mG4nln_(_sk zkCU==-9`}*0c!9q>h}-d|0wTj9@d=Dq_4W}3S$a*h~}xVLBP`|%xf<^^^%#s5y~(? z5*|GGKx0twK=*>1abs*o+WWrC|mTqAcsdZ!$I^dJ-Lo&_NCI5~U(M~*!5NP_`B4KQW)G()8B zm`VgTWg)Q1=ki7y|771&E-Wp}uuCpF_{NZwvzGIx1=@QWE>NNGL`hw`-oB6l5uKzAqoNlXvuSLt|vMzs} z;b#vO&Y5ZIW3qmonSq%EY}mBHaE846FaxuW+`L6IS(}$@yaNG>g=@76M8bACo)W%Z z@=OtBqCl9UPNSoBT^A(*Gx@ofoGmSyw~#-6|5KiL_$hhjvFFqmB`M?ju8opJ@HS^ZmxZ z|J%xa<-O-sndoQ}82YBe5g{-}o1~ul4)xbD?%SuYnXTdbjQ@}!Lj-FM_&|s@)-DeI zfWT+DmE%NFGE;?aQ%X<+FaU!FjwbBP-e|_LfJ{C|tT5<>3m3{`k3DAE(Rt^cXXU{x zTB%SqM<@L_n#n8F)hK~>;9S*QHPV}J|CbTaWHkj3KkiI<K%@QgJfd+#!Fh|dqcp|J z26gMp#!dOQPK6*v*pt?_Z6PT&u#S$S8G9p#pFONdd-Teo9%hr|W$g%|Z|m+wd0A)1 zLO=j^FqA)kpQ&YKtB|ZOS^Dpt*WWeI%^PpK-e%31rE&Uz!NE`l9d)R!*)LwWL`Lh{ z`>i+rSFXMJ8p%*kO=cOG(C{+FJa_uJr&$>{-g3RyZZ&&jnZRuIjGD%P-NZk}OXm)q zjG*)K^Q>*9{h^O2IcU|KUdN0YAvtBr6!XA_S?{;se&*=qH0+9;U4y3(Jn+W^e&?Nc z+5sOs?X;6Iy3+8twEm`QJTL{mFgT7>J>`^BY)fT4v|E-wQ7p`IXDU2K=DU$BShr=MU8hQsO!*}$@fx=6@m;)Qg%A#-!#Kt=oCDj z`THytE_u?dvCf9Im8>&EAK40Dlo9&O&+CIzJZs5bx^&gER}U*ssanmeMSEL&;OJApSVX_wP_{i z=u#H}7y}2KbiY*#-0e5trAvC38Q-XzQCITxd*RS`Oq!wX%8!`gGhW@gbu;JVF=NKa z{rBH*6EwWJ5!SKFA&$D>lQ=K`@WT(*zf`e?Rh*h@Ly`C&5D~LCpS9!!BL*{`KstyO z*ZKz@ctARJ>LjP0da9Wn**XKUf?yzItl2bmQ5KpxYtj98J3x`>J87UorGu_5@3>P} z8*t1+Y+;!B$1GceMZn&wYe{&4@44^pwvL2`xOC+*bu1evr>Vd)23{viiFuLDewVLb zVMM{&HTCVj_wI6~uK(cpwPe{+`Rt?5O~c%+uR3Qa0kgNz5SJb(V!=f45C=ga%9d(k zZ`rzq&bnC#@@9AR6~beW;wJ683wW-gRPaMcQ$Da?tc(UnwdHGR~p`jBb}*b}`G!bI(0z z%o{jU4={-rx1!b^CBnu=+kxV6)KN#-_9Q;(NZpRXM#)+&U(+Y>2iP;4>DRBHbno8X z#se6~Vvxypm~vgcC8 zx7};>)^@vX%~HfVERJA1s`CBa*WW8&@jlb}#w%~iw_km)%UNnFR)?~eo_Sg3Et;XI<0wcB**vuY5r15#da^ z;3?ay<}8}4&Z{eJi0CUO^6}sPU~BJK1Sji+k=5tbwkui z4Bqjg!UFzF8{q-XTX+`JWx8{Otyws_F`0oMPW)+-`2iWf);1N*FRc8Me>6@ZL4XG^ z*2n>@r!2i73s`doM@pkT;LeYAY7`P=35o>osyd`!dF7SYS9K$(-vI-_V735An5V7_d_diddoZT5sF4Whz4DBA>_4mb`}Xs+uw<*FtYU0Wj$>oMD^)5YEYz0VxP z-g)Ca>8%7aSUoo}Kd~KwHJCrtOB2)AVMiV+`>0t7M-2#Nw9XWN{rMLe``*WL?Z5vn zd#ZB=IEATt)OnW+&J}3Y%a$yYeroRO(5bz&P=aAw!tcNQp+xqUT&fGkS#?>t4I>-Oq^meKns?P%}EtHRp{uN4J`FR8R2f z(n5uM#~nKwb79NIl!;TVei%A^p?)-B9M(`k)+>R1^5JLlwXVmrzZ8bGovyRfK7xPz zF`;Fosk2JDZe7!O`c1q`<6r?!pzI+uu(I&|XzIoshKb9;%H^|)%HfYIk3J%-XX!Ey zKTa^?9Y2?*9IC6*B3xg9HJG14k zM)>JRpP3SiZwj=GS2)`vS&|z2-bcFr+f9a_bDAmA;Nd?nyUx~U!F9G;;a9@C@DAP8 zq^vtC*d4+lB{*YVF!DTUxlK!%{?l~hOYi~u3!hxHV6i!VW$Pq>a`ATo?(7cv{+kI3 z18*xAZTRNP?^M`Mu(m+Qqz5ki=r3g=6|d4|sGJVl$=8LV#PZ`rbi^37iARQ8NH*fOa=X+W`u=Rp6{ zbaKGX7WPgvL18_eNkETY-9w6X_&8ySsH105=+13hUPGAEZ}5&u2NqeD!#H0a+CzbN z_z(>zsRP`>kr`Oz*PCNrsh>`M8EKpQvw?CbO5ARqf=L9p> zxR@1VDOk2-scPHxYzDDZl?sSDr~UTU9vnvKD@4yMnZVX97#xZYCVf0$nF%!2nLcKE z^1J#meXmo3q)isI4Xmj*)IK%T`sqV>Kv~FEA>E{QbL$&c1y`OwEqVDtc*Jvf4qNq5 zMmTdCGa2u}6RwfU@Ryh9J$(Xa(CF{D;||mEnc<`LGcGkD&_@L3pMU--4?g&y3?Dw+ z;93!VbmIwbLD!rj#XfB;aE#~9JMV1thwvtCBpACffobO!tdZ6EE2BD?QU*8yzrdjs zvo`se2_n1=XLR{#mC`uQW(bNl$^+Q$x}yX?exXp zPR0uh1!*j3z3>3WxtY$?;UO&f`RUWzS}R?eX0?>G&FQ*?wyFeQqi*0ynNL3Xq?L8@ z$tN4$@Q!vz3Y1Ix$BrFqyGV{d{`gS)qveTIPIV;R~WGe?jcBKHVi*QYX6xlm2l)j^pc7k%;asYb7*Vy>r7Ylp&zG&~b* z-CApK?Xj0(e_Z(xF*9lM`w>1Yxz+F5Mm@$9)r%C^(O%j?K%ro~GV~d^P!CRHY;A); z=&Ng&rPF>n?cHBL76o_ukQJjP zxAM_P9~ook=ueKPg;~(%aGY^vBy$BO1nQ+uoZK+UQePzeOs0;i%mcg_TgLT~M;?(J zoqZf}#1ZyP-y@A(vh8)WgQCjJ3x_pk@R-^A_wOGXL-0G+7*Y=7;R)*1DyaV|l?Tud zXbo>_wNvR9BBf`=e>c9+2!0tz>ikX0ACzVZ88CED+YINm z7UL^L!lb}gCgoW@MOHEL;df)pdYa$C+`*spk>Ei+w8y2l22--Ohw~d;+rU%m;78x- z9|1l93|t@WrTzgIm?_uJQl6=kAfCN4>2H?R>8ESr9B=ct#mxE&XXGFmp z3q6LnjXJG!q)-^(4Niav=|diU(`N)Z?VxScG{~s(Jz_UC8 z?SQah@ZifpC03NcciwqN?z!h4Ir`|M%`^s#-MECeGnsZ!hGV|Wr1{Jv3gxh2!;Co* zS042w)3&naF-ECyg=V2W#u%qP`ozzTeOc=vKjk4P7&~U}tb@bsv7BMvbYlxF%d=kU zw((IM$5b1l1Kt5lnbbv~ zpR~<{e-Z!iu;`= z2kcy+4CApnp5ei(znI^J$Cr5CvWteB$O8Nnh7KJnk3RaSojAsF81D(L53VepfhqC_ z+-92=_u=VW`pUYd#2PgNY=feMjHb*QB5Mi|sEZ2+2ytk}e3lH88w0@+jvyvwMFT9e zJ&hwF=LgBInb_2OI}%6Iu(UH&rNJ9l+)!pZ&>!Pi9)UUt)Wev3rrz8yz4Vgplg9@IFKz@Eaen-Z zRX1fip&x6FxeA?&|M@YfFhT-OHk@izC1P$9n~CSnA@(5^K~^X5N$m zey~59>la^qVcV{d9nhuKQCy_mIo!sf8P9fH@{~gpgCbH20ufZ`INAqZ5io(nurZrnaKQzp@G-k) z9UCGh4*V;R06v17>^$Jx)f@(brj7tAuR6-3o$;1Io0;9h#OMo7pL{Q#SxR})9(6EQ zzz9b`OzSKk4A(XAL4yX_agU54yugiVyzNK$1Yzrd0Qd-wV1XGhV1RX+$sq5+8{+(d z_uwA%k1~ygm!&nH!_)fnRrGyI2xS>gEF1vi_eDs_rL!&C5OXGcHD8tm`f>f)m@S| z$CV{WRDB_UK!(T);CxVn*)<< z5GhaRpb$c4nwmY#r!s2lAR+}J03JY|wQFTr>)d#jwW*F+5jt$o;6$p^Bla7q}eB1M@8fV?$93XhyGApSeq`Z$=ksa%LRfNraC0z)x24^0R^FYK|r~_Ih6ACEIi6Rz7V?);12@Y(Kkz6 zIE*2XBjx8^jtXjgB%purzp6q?czwg9zW@GvBgEZy+bzTlAu1OjRMr^S;(|7hmnoin zFu6nt7$(EXR_voL*sN=7!M|z)0q}tn$r@zpp$W=gVd4;!A6FdCBzkpapiL7{8abGs z{rMb#fHN3B@C^Y)aCz(@jz6Ramr+hx1H*g_K4Owfw#+!HEprlJj4qH4}IaRfXwuweay-S0v=GkAdp!r zQ+YsR`^;tD2K~U1(qp_h0)6u_A57nT#0P~LnvTRrz)ijzfECQYWjNltV&P#TaaFpH z8o;2XAZ%Hm=JSMTH)z&=KtQlCGfdv>L&bRo6D9fl=}PN3X(4>_F#w!wwXPsn@;7^g zTC5gCF3A~LT{>@W7ddTN--g!}t-NS$i1t3boOsiP=kq^DYO4;dj*UN6>G!3zMUB)_ zTP-0R8_2ABoqk;iqDuq^N^1pW@dM|;6V|CtJMATGqWI6F9^VB*X8M_1M~Dyg-Fz1 z4&8AC3lm`W3sJMr8Z$VvN(3wfj8ciPNmQUvKEenOaI8oupv+RC2bfQy<&{I1BNhSu z(H7eK;fEjEESl|*w2|Q86ftvzmyTvxM&KM~9737ZaCiwb8{(a^l4#ixs*CuFaydm% zBv+R+>gsW{-Y;U};AP4}0NQiTg^A#bH~t$9m-+lDw1=+~XDCpn!*_?P#C`I~CpH6j z=PV>rUwHtqN(D6?04QiHc-Jw*066{tqd==64QpekpMJUllmSYHnE_zRBtPGDuALeA zh&_KBg8S5f0OJOJ@Eu={DI|E#Bo)^_a10F^;gOS*BNttCk!|0)`|i8#GnTvVx~o}x z5a{4myaY@-`ofP{B%j0N;}Qc@A)r3`8gE$@*L$7uX*&?A%sOb_s8OSAc8!veDE-kd zX67h9e0Ya55Lil}Oz597fk&d{Cewdl3sa^|C`U{LQUTt{^kr+y1CH>a5sqF>hE5#q zp?fbe4zhj%a-Xm<&CNYPpV8%0}O}*2IsbA!4S~Y9lqd@fCr zRfn=sPw=Z=FaSpwv&s(CN!x(xXbVh#(n5UUlu#SgNBg8ufj#d%U?Cd}p`SHge<=k`6m4SNZy6b;c zk1Op6ejW_m_@+8M3=zxlhyFp!Ich0GVRWCtg@&CV27h_RCyL{8nOHKxtn5%_f|`!f z;n}lioAU@>b$-SKpl~*E-x226KuqHR$}y1223AmC00D3h+{9y)T?d?KhWY!UhaR$Z zEFc8lg3F{UEx;^D%MZbY^=RCella`y2l=Sn9>ctV?}bb%atGg zDNB|tF^9^kz}G4f)O4KALiA`Um>W3V$eEysM~fkz?JJxkjH5}SCbz8t*@8Y+J_I0a z@E4)V%$M(`Bd|GA^XaFbHd8S(NYWv!N(+>QW)BTu0%vbE3ap=DlvYN0#?xk$4USI4 z8=JK|Ox|uCzO=faU&=txlZUTe@r`-Pp?^5!06RieFul^+AH;7x5P(L(QMPMg0%u8v z@0&w=;4tyXCE}rJPAlXc0X(m)P-TJ|2FE6HI*zu2U`D%ffp{EF*u%;iAO!8SAD1S$ z1`w!92w)LHnMBB9y5?Kx;4vqffx8ep&k%fRMC}*`9;P^#At0S1K>o@Kw3B{u#T1AV z%_llxR=jgeDq4GJE3^tzPNo33o& zlov7t<4$ zGmVL?Z{NNKMBW7z4Yx1Qh)@3;kleI0sDI6N$7ak}Bfe6FkBNUk7 z(BPe*uV@-@N(D||n5&OC;)uAsyj@?Ib-={%5a;wj_D17d!h3`?fj$NyfIzB20NTRR z!!m2PZr$WF{q{LB2}K$FMhSH%VM5O+qZLh$ei&6c)zE1=&$n#RPIA=a4lUdXYG%{8 z!84m+GdkKlag_-aKY)V!00Qws03igfLhyKrvgew!s+n1`4;efMzX|@CvcG)CFwovH zfwQ)Y;~Ij#A_I(ww$XmPyfMM^jdx}m&}=07a|{fog)+gncs;4VrmW^tBjkx$UF6XNmKQI>R-e%3d{ zjqpWCvyO*bJWezaGmL^6#fvY#XnWe+Y?=-RL|&F*4)RtO1i({p96Uz2qMfr36DKp? zvE~bbBg|k5;53d_o|*gHEt?p)_MD zgNC6?mb)q=R5}GY=7OMx>CjOa7{mLxuamsc1PUy(YK}M@a>5Wh84S(7yy<~9VxGq2 zO?f<~a*FTVJ~mW9}^h$A$(PM(Sm5l<$IRI#qAiEf9tugq{I_=TwdKD^N` zFP%k0;c$OCz00n}X}io4>SHlm-*>N2)7YyJFRsNS=zpfg-%t9);t2c)KI^ z8G@w<=vJ?^5u!)9@}(@i!e4ypMVoa}9)g%t$RKvoA^fad&5wX&SHx$!R`&SgkCWWK z9LE|kvsI42O;7V?y2WjARTtx9ahylHw9OAYXp;IlYo8Bpbn5JxyjfF*KJ4e3Hk)&AsPM zS%aSs(A?-{yWW)EpYZn5M%Pz&^}mbMhwyZyi3%$k?u+;5X{`+Cz z^0;ew9`{^gjI9h}O4jMhuzD=E7*}06TnWW}rEgk}4<9VvW5nE+DY)@>v;f`V$jq_6 zgx~-8-RKkL`AJoJza`r4_1^W*)$d^Lu2xCWpAv%FWq`Ws#qL z(%&GYJ_Hyu#-5)WzhqlYeu5kSvfxr_ZE)>Iz#g#w0dnuX_gWu#MzHxm20&S)r(K1c z3+-T#2OoUU1oJknx3RTgr^S~C-cpw{@3n2$)+|}fu9*>2Pk11`Gu=;{K1~J;*w6Bi z*5#=TSXMV22!~_voIC`e3%)@O4wMzwz~{6ywSXb-_@^?VapOj|rL zKDJHb=!_*8s}nTH$DPXB4xm9>3RUQD(!LS=W!BQo-Nvy2l6xHOV$j`_#1)g_(Oo@A@Je?pQjt7Ccb%- zmbkh>BF4cOiKCqa^T%CLgoM=L z7EdqW0R$$9$h(YmowKF@3ZQ&A`Q}$dOi(+4hw}@9nu+6Y#WK~Vt-#TGlT>#w4ysMy`z ziDF`d3Me2*Nl8jdNq09(FW`56=gnQ-4l}zm+kDQ2**EjvjdRaEedB>q25HZ3FFY5` zAW=8MnS}wAOVXFdANZH8Vcx*Gwqn=ZhXH6Ne_mZ}sqdr3>fE(!r-h`V9e)%v?d0c!q9nQUsg=o$XV5^L z%7A<-3uv1KJ<%#_aEK$4P}tGHLD_$hchYAM4P*u${>ydbQ~u|@^dzQ&+mE2}6{f3SO<`_jvCxKYG8TX*Kn{c_pvJJm%{quc-|$vb7x{$!!8_#ERv#18s#If)UG*Ll|KE8Rvp|3rn&15+=(r)R;exbo`+e z#va;X%oF!-l|6@c8x5tT0i}mz68G^%GXMJSWdc5cKkh&Q({DJFC~1^6DL+8tVK`4Z zQI+&w1v;-;6KYXp0sxs+1+b!68LSfVr4436V98;)2pV#s&fN!g$<71Yt%UV8SM@Z4 zj{HZnU`Qq&*s7kjTGep{JYlP=t0PI!4~80{PNGGLNn2%~Dr?4{PqwAp#8lbV>C9^g z!)k|)9iCO33bg$Dy!Qe`(gHNV96NFV4{C?f5>Yjr0RF{N?#BRQxa(k|cUyKHFY&fHC z*f~~=AF{yd0r)$uL>~A*pXv^#yKqVwdvr@qC*VLHnI6`@d2o8??&n2|J0hji))Bqq z`TRKzdT{$X5DJ>D`cqi>?qCnj8T{J@k^IV_^!cEcr!fdiu$M-k_=cu z87O+xpy&TTpy?aMt*i3z)`5E19p%zhuk#^yt*Rp z8b#4q)?9;u^o?F}spK!Yrb976GJt`oRBrOvCx6L6%dw(IJqC^fGDzM>ee@tZz>|7neytDk(?K?knAXxmC}xwQJ9o1UkqkD2aK0|Xnj)013T*O5?l{_6mw3sFsw)8g1mbtXr^4EPxkAx(-t#;5)1UoJKRm8sg zFyO|~-LmoUoo?ptXUbKF&Ks|caV9Xj@lPLIQK{pm|64RBK}tCb%X21{;esUmFdd50)J!&Ni_B68pi&Mimh$I@9l@3sz4c_61 z33g1cNJ!s;lgt}Qh|%9i`pN!v?zuikqu|nl%n5AT-~&$iglfZJ)BW`MH6O=wa_D_D zV;2kyZBS=gi9B&}`6XX|-7l#0g3)qNVNBZ30S1{be3NTT8?>}Q&$Z7vb&13qxm<$w zlUGT*n;E<^c8oKJhcI@~aC(!`3!I(qaiL)H=!OR)MV53o7QX%npTx(f93$h;z{@5g ztRl^9j{i8%8X4p9S6-tFXyXVu5I`a0$v_-X8J>|8jSK)j=+@CFWsw2lz+gGSiV349ivEGo9-ETd5(eNe`bb?-iP=gM4oZdjO%03 zVnt=%?0;m%vXydj`&Npxgn)vwLMhs#EysBJ)V@=n`*1Hf(I&^_CF(JoGpGG6RjXH( zvgOKzJZ_JA_U_qd=cgAeR8UX*iPt`z4j;4qhmud z`qI6`j0^2<+q%t+t>Pt$84jmUnIUD$mXTU@Y9^oHVCyS&swd$iH>VR(oDt z-_!*Cj~V%=G;G{JYSyk{V{}+{1jZy#r+-YS-{?s32af%TXC!!X>*lR8XVyH~vU!VC ztaOa=7iZlcp{%tE2OISjPAF{sw0`(#E#-J3mz;b!?1;?O_UKo$7RMToyOYC$$CJ7p zE*H*UC=32ssK4_`?Ygz3T8*mq448uZ=kKe1Pucw=9=jXV<=S)iBDoq4QjDb+2~EBq zsD$6Z>OS12O+FYRyLRoCSu1R5(*{A z+*$gL+V6fcK<$`5eQg)dAq>k~@bIfExsRZx14RoLHNyF<`xmlu`6{a`7zqJfc$dJy zI3$F-q9hOoytOWCk)gA~hGR5oqgEC^B?Fl9T+0W0OGo8R<}EBFGZv;{u-?b;00i~V}E zChr-ZFm|GJyZu2EsOGssaK1$SoPE4w`woTe3)#GBvlJ;*#FQ-gIU7B>|9HMyo{ygf z_>O=-sBg9D^>uIgd&cZ!g^tmM5a+Nv%8|HI_rAURAj7 zLsK@4m&@-1H|1BYF5K92pYf*-aFxRNi;%HVf_Nm3e)&$KPCjw!89~?J$NvnP1}EI( z8Dqv6dN|W_%AqA6E_qhi(Ne%j4cB+>liyEY&Um8UabqUPz+Zlogo56SXZrR2RTeK? zED7=OYl6Yz=!UwW>B0$xtRLWsK64+~83SO3CTY{b#5;KDJN5MX=0}254#`elQ*W|);mAkJ@eZ1#vd91my8p2jT$~i{unw^3K!tKX_3j} z|1!P$Y89|%=#Ba)N*|bI8f8#^>KtIcuESDQQ-7Gyn`P$Bq z&L5yZ%7HuRfVzhc8ZLj18fP@>#wVH9pmSgU^T$Cu9{oES_YALT@@t;~7Fnf!U;%$l zesa$xjETRe&ypK1zgebCoGN40Xu0^zPUc*YpA_6bS2pla9STnwhN1z+pZmao3^1B9 ztf#0Y><`yAICpXo+&P?aADqA=^pWpM$jLOqZ)9lE0!0kYpT7IqbR4Inxw^wP9(WLy zM@^*=5{TWY3TMOm4d!u`Q7^4{@nW)T_byp9f3ZwbqC4mOv!zP4%0^Hqi;e3y8BnFm zl#*g-PpJ*u;HzL{n@wj?HAum!zK+q z22&WUp#TzG{(bxR$r^2U(62+~5T(QZ%5unB|Z#Fi(>xfh-z zCpKs;r=N40v_Gw_8E+280LX!hH+sBBHiZKAWulyiukA8-?p%A(d)2B{y&Ng!ny*yK zuh9CA?TJdi+D{d-L?nt9EuwJg(S<5_d4t>Xb<53zEmxtey;~Zb@m41c7sh<8;*V$v zIK@y~vwE!w!E!o&e%f{8URt9HrAn8wvXpo91Z4DyGNknyfgn0VWCpnCE8g@xjZUms zzEU=J+Gu&9lXw*-8@6t;vBcA6ydVYY0^a3nB+(c8P+aMi*9hV%W598qK3*AKISRzV z1l?h1jvYD9c%!*`PbQ=kHu{pNd|yZf6#cOXjnd5SU3-*f6km!z_@$K64DaAAqBvz7 zZOqgF*rxqLE&wZax;CxjL4P(;E-@A;&{F%=9)c&m&Dw1b?)O9Z#bkj<(*XU&*x zY`A{odNZ7Ey6pz*D`SB%vtyge%ne?9<&P<6^g<{YV;Y*|^^Wjf(PBk(?8_S8IUE5u zdpUFVzHlMbQQ$k;9woZ$@e{fkd-S|DPVdgss4N5iEi4F9qJwRHtQOpbl;=* zn&tMsdPuxKn6NR`-1peM*3rJbf6@0r=a``yca|PKddQVmTq(uWsNgv@g;Lf$gU|`#|1oTo z^jFJ}Q2wI%OXP}cFO!R}xX1=(*x=!+1b;9Ev~j~mx&7|W(oSK9NeA`+UHbIuD-|nO zGzECw|E`e+8rtg9tDnqPCVlPsw@m55z=Vn~yXq2auiuZq%5T365||U_zx2wB<@`%J z8lmC25N*J!Te4`W6w!gdqucFLr(P{3mIEH4YLC30i3$75uf9}HIitM{`em>Yly{~r ziqgG?)@ zUw!(G3X_epd*^N?!)C2R;=WoCK!TO1k#bUne{w{`_rAwB{mp#5R_*{i0uZ?-%-umNf$84p&dSYGR2Ef;sXQ0wX_e@&QTG)bF;QZLZ4LMdac zP5*1UDR7kF-4EU+C$wx~f*+b3{l^$Hs=?j858rK}$KiwjFn$CF%hVft>Fwt|g`u<- zUm#8mwVg7c?{6|<$S9q@Sz+`>A4iQCEnj@xQ$y{w?Ecy3ovCzlg&HY4<(tpHlSK;_ z8-5Yc4?KRKG-=-0eiL3U7++8@tf86qr?r!g7oC$lo<|aacV~~jmwMwSRfdHB12n)B zzB?LNwrrU^_0&^Ry+(C;;Qj}sQKLrWNahvthf0ho(TE-C%})RTKmbWZK~!%Pe+cUB zYLsKF-*K<FmnkEBY~$`YR^P91~#a{i?qWt;ZJB@8YU zCQ27Czpv5*@`Qc@?;MSmy#3018aA(B$^a-q*S@GhqXu%zo&PiEqR$WgG=ekU6yRc& zN!=dqYIL4Y*SqXj1%^^XfxZ38yJoOm_rI$pPhpQwjlY!zF?!Ft=v+De#O6|>P;uF- z@0-B5gFezq$a|c=&6_uy+(m9X`2r5W2fWE!sDAwMM;SkUygd5YqjJt!=ScnnUUV=u z89@6eX`G%O?|-VYEyj!Z&;WCVMEmxpBHpOfUG%Px?=b>0*K^5K{789`vR z&X_twmDZ&yWa=A|X&Z$5lPP4qhnyWfYP3w5Iz>tpFOi(zpBI={ty*QbZoBO^IsLTL zO;P*v1b=5vx=^8bS+snKTzl~i^4a%2)WhpwVVO5xdPmxv+QvL=gvs--J!4Ad&(Y)M zEhXk!%DjY|Zoaa!eEstma)Nr!lg3S!t1h@se*bfz3DfZ!ioW62Yi$k}Z=`>pfu{VK zr|WXVUGn{}U(2z_H8r7oW2akW==edZ&6kqSEf-5MGsV6!{EoSC(%t}>3 z=e#(qyeiczOOv{dW#P(2GIH2x#gEPv2XKjHjOD8+R66xM#j`q|sf4yjZn))Isn?*M zj>SxQ?YTGQ^Ikoqk$U8V2mCHyssL`QLV@tqpkIHN-UGfhCVx}$Ldf+JHDZVk5DuC* zf1W)1$n$dA*&U>%TJjhKj0cQCn;0~;m8j|4rVX3rmp}T;ml~EkQ58s&V;h@S$JjWS z{o~|_e9fA*@@3C2)ZmLVhD0#C{NzE<;_=XK@7}$oO`A5JqQ+>n-ur9yzEjTN)Sm-1 zBs*j3-}3fHuUTD;|D^F#%pk!~VJ`03SD#XEuz-C0-ly`ShVHxfeBWsO{WrSH=RbTR z*Q#*p_v0^e=S_D@k8eIS`W`ZHs0puj9oon;^=3c$;4^vg&F5?$872JI%kRn?AG{(p zYSgfE>46M*IkbQLpA+Poi>{YXRRFhY+e!uL9J%7WYozzUk|yjgx#B`Iq)tEkG&AN{ zH`4#-0n+83&IUJa_3t~-@)F`4F=V7X{qhs4XiMs}#gA$L^io6N4)w0K$RqbWAuqrC zqMWD3BJ-)>IbVVo!BV?NTDNT}XPtMt9M__`v~Jf*FkT++`k1`?>D%^f!njG=?|W3~ zwigVP7oK=o#{E576fhK!?sr1{ntto1ZNfRO|#fvrJXaGU~)AxPN7<%FL zXN+cFe&#iqu5zku`7#zp#;CskF&vfc^6aB8NDURp?G^U!Z+~nsb$#RxbyW81^wa%j zXdx$W=zN_SN%Rp~`}))GbPOIhg+m?4zyGOnW=`|5ZciBP-K$Cv8QJ~q9`e)RUdodV zfY$VVz;k3IfaDPOj%M=JpyN2#+oW&bKC)gT7Z~QwSfPv?M`X=9bwVahnk4t#f1jM(x{Z`6pb=2Zq<_ND z05pRa{m!fJnE`Un1!rjlVU0Zgz|&H@UTt-fYO9QVL#|hOe)j`+nL%*fB{!bXY1$h`^lZ%?l69)`AAa49yOKGX%ZpKjqBE{=R8LTsEJ@D z@w&OHm8wY1IyG#uLgS_lZ6G_T2Z~aF*$98b91sOU2XE?`SDzHTzM}C(WzFU_>PeQD z#v1OZR=Jua?n$)v9Mjnk8(uA~ut1M^N7buWH$wKFdw(6QD2HLghS|;g?z>OJF~$7Z zLVl;-Ls~yx!-=h17*p0$VZvgZ_dk2vJmC>)*;Z0-nb0$Hx`ZAl{W-;y>$iStUxh{;2gfvcNmOg+t15mzW$!I*>a4orpedtQaXS?Yll&=;eZyQrKz z_~iYn7$%!Cu3NvZGS6vpT$ASZ<`=vugag*%Y$XPm7+e)sWIVIwL`Lj7f?VxKUa|Z5Li}p$R*qE29f9Gc<<*1A~qpJ=%&lY}nAQ9(@E?dbkBg zKUnr7)yrTp4*k4cy%#)GqDpw8-~9ZgG;7jS3+7SgyIn3j_bLk`V<2?tcAK2p@eCDc3%817N%HELIvCm(#q1Sdl3e_d{r>opowqDTpuHFvgpmrImpyiKEV52n=da-h$Om5(vQh43~B53SHi$60g=j|&As*c-1NRlqYlwKS z@)dHjriQQgNxfQiWbLN)YT(zjDGcxn?Z5ZwTT;DBb(3~azWBIK4?Q7Q-O$PU^7JcD z$|)LcD53AaQN z;GfVi5kLBevq7}3R*jmnZO1lw^}Ux&!4l~~)>TtOgtaII3wpA2md-EJPCIpA+IMJY z5kHjhygC1vkzsi1+f%Alsw(TZthf1SA`B>0_zN9UEJ3GE_wLeq4;dbqVk2q@&mk}T z%GVoP9T@GG&P*$<7nb(zPqC;KeeoyCfeYwp&6+j#=$^ap(M7g}bhKCm$Ebx*yjgOI zMpULuo+j7be2wuEMg|MPhO6U5zi;e(z1(#B4eAs%l0Etlo&ldI_;KpMVFWi-8KLnW zGk7rWc^*uw;h|8kKDCGbXHE#x@zc)gAU~@`*YBraoFlYgD(of;$Zp za*u)Awr!jA>eWkHwQ6Pas2B-6%M)gVAPg2cRjgD|neYSo<>&rpL9JZ2O73~+PC5Jh zvus{u_RP7~H^RSIiVrzv2~TT^x>?vhf+)+Zm5vvlBX7R+wuV#(T6mT@=tC-Ots@4x_s5Ml^QldmG%;9xkL{~d)TK(5 zlG|^;UEX~2O=;Yuu{~o|t*axr9QWx1q1cNqzrZ|A;CV_F7~x_<-;fTG4Cd1I9N24u zizyW%2he_!;tC-Iu6D~VZA&aK{Ig5{!6V?EkQu^Y=G0l{xgiWuvQNJBxEX&eu)_=D z9+3*>w-9E8LxB@I!JEejWWJwpAz@2AxZV4AYuV~;*`1EJ>T#3XX;8<)~u!c8h z4L=4akBYs&@E(FN)OgY^MgRsn(;f{PGzf&fwMHGAP-SR~`;_Iq=NMQhYrMQE6Q@W! zT|2=v7x)8DV@Hj%55F-+PCfG!Z@yp4uzp}yk{Zj3BZNB!7r5F3|EqG{bHGzrJZC{1 z@)JXgdUo#JX-WqC)TmX%cm|kIVqAAB9g&}1GVllMUq-31Q&0`8!U`91nST5|{cq)| zcT8rm76iutoZ9*Vg&l(onx!2=Sp();gt z$UOX7>ij{^p~lLlOBDWk_3NcrALQgeR~J<~`|Pvj(n~Lu7A;zs!Q;J$Jal9&xxC7W zB;^;jtqvJD%?u{}uR zPsEBEzyS~@XTA!z|E=z?o5jq$>bg;?0C9#(Zvri2I1BDYRy(q;ZszF0*iB~r$j<= z*}P-3&11CE`74A9`-HLbHCXF4tU$oPco(i5tKj$S-y{F1*NPHDA%iE%k{Ap6heAMz zUwLgOTl7Qdn`bVHQ6LXYtofDk3?BjfVUZKUCgHV}Pyx-{Aq-YT7eeUzSeu)kFlM4W zr(QlnW{w&rERcf`SL$M=#>X~P53YkT+4s6~8wLB)ThCiNOlO>-;juezyvvmKJr8v? z#m2e?-~jK8GhQi7%$#ZA{5tL>;!P;gZ~B{G`w}{N1IdD(-Fv)u%7)5^ae*M}*s-HA z544$9as1Fx6!!LFyr0ftW#|*1f;=y;xg?s1_*g6 z2aXvtLYf$y3?49>5>mS78463+J!@FCk9wKk%R4>ZR2flARw|r$R@BV`w<=Yu$ZPMv zYz910m#(*TlRj$n0}oRO*Xgti(+BVAD&-g6c-E9GV+c)$OGX3Ar@&!^<&j2mb`%f% zcHVjC84aXXmn&rBraXguLEfFHLKy=ckMaxku2Eb}^Pp5OyZRChLpL%-NHmDD7(Q$@ zA_`Txg3b@?!pYGi$12~fkP9w3U-@095Oh2X=GWEqdDNJPZ}o#wK?H@xWTmT@*6!r7 z#b_98C^8mKVcZ#f(2@2TIs(DS7raxpQ1ZNJ@uD`r&J-QI$aRlO=rD|-VZ1J3t zgBU3bb!_3qg4%z!q*6a^JKACuItD7Ce`G`>)j>R8o4D5W1d+C2UFO;jbq!Db!MOv( zk9ER!pBr^4Jz*N_hyu)_7FMz`*uQFsg)K+hRInxKicu045sRjtxc_Oho85kIrYj7P$bXD`d=Zxt!kz4zX<&c)JMJhwpvd6el5|mz40>=CFMAa$8vTaMwqTsPTSw zXy2I|T(fB@nQwCSZ2H`=e1i@A*%qOZ?uVG_^#*o@|`ZM zV(k9VdDOdZzE=&4;v-oc5%4ITi@Gvv(nC~>>#0Rs3J;hZA(-|kh%yF&lFgc&QJmv z+dP9W#*7#%OV%wh!RmxPbd4c~r_8Ef7P@t8e}M@q@B+;v_z=h#y6}IwQe~yH8ah3D z=o*;*LyU$QM~{{R=nKARw=nc=`~!xnpBBjOXaE@yDGk79$TSvIKL6N@HdVxy8fbv} ztJkPzk*HsF9EhqR@7P{SWP`OA?1^SQz=9)hIg2`c~$)FX+5nhDGT)&udU>arqlKHma4g3*-q73U~*xp67Wx>3K z(z;1Io0@~Z!7+(d`1A?7AVPEDW#^lr##}WK0OSO1@{BfF`44Zho@vg4c`{?p-#YK! zLykM4nHj1WB2MO61Hpg5g?_@x0iHSe#@N$V3f-KFLV%WQ>JB6fhY>^mFdVQ8hB<4Z zlE?^lwfOZ4#vKbSSx|_9^t8(KY1606g2fAs*BBGvF(88b66SZlhaZ0U87F`a{F@|u z4+bpL5Cs5+yo9XSBJ`s!pP#4;rSJmpyswKcK&lJ>nJ3M{DCSQZAKTb~8?P$^QG9q{ z%Tz#B&?W2;d*!N?EJTJ{cPgB90ohbbenKk^HFTrYine zY{&c*^T$|>tjtCLRIFSwYEu=_#D=Hp7Q!j1&=y57Yr^#K)PtDYmXPXqBID!x!0WNS$*zIRkAd@HlWpmvR z=~{+Ul%C)r@Wuj01V2+SJ5@;%hOefI1vX)=2*wl!0*6Y#(_ei2r7Z?z`h;;L+&Evo zhO!zCg|ET6qZM~W(10Tp?CZLV0h(`d(sAZ3f2G2lMQ&_YVonw>wR|^j4U+6h^wP>yhIY1z8S51jX5v_fex*@v1FKlCr)XgV1Nd22>j23$CKZV z1{iXfaX_Gfk;6t=Ue=CO ztWr^GDnC!vxy=!}(3a_oORv1h*5jZ^Q9OeO3{~SN-V^~-7Hrcbgxj!b1DQPGFRO2? z8uTZ&K0&U(Z&*lJw_Jl5bL z6pk!_mzjz*UevzQUucIu_wDtI85AuvG+m=sbzAs1Q_B%OY_1A)i7qN@qw<|~3Meh` z!y22%o_pA6lQ2B>vZkW6PCYefQeS@4#iv9dN|!BVaMB(!3%+Tp>m+7Qn`O9ymx>lG zs?o~MHvMSMoc1R;@bNDL17c9p+xF{%FGWD8qabW3{NSZ9z9=TVX2umdaSVr&uyTrLLgU~g zu7J|Nj7v;KO(H0q+WWW6XdZm_sFC;W)12aMyuE z*^#t0Z5mJa!Eq;IUhP$IB11fVf+%rkJoqCKW9*nbR<(>D^& zNc0bUKo^cazz20QR}3w1-(-N&730fq@IgD2b#!JG2L3QUc(9b?yB44*q^3CV4x0+j zPo3n27Pw+mc(Ps>s%mD|mCz=%hd~9*j&`6!6cloOui}eH9)`KwmP?%AY)3 z?KOEq=pLoVnhanMm4Gd;uJ|rhpq%9WH=oL=?MKi^xN^XZ9yqA$HMZ>AU<=YhDQe2o z4md#$Ad}&BWH5#=V?Y!N833(wpKc)+DPv=)_ZR~Ve6L?bI;fK|M2;T`CG+mZ;z~+F zeqG<7P8@CIiAGVoDgmC(@?rE42#y#4koyQ)?tuV0Nj?ah!GrjW@eC2(eV#)>dGdMX zsnZd(YuCF56mn|HFzaKX8 zGnOzO^`NM1%(bu+EcB6d7*-59Zw`p^);_di0^OVcqHiwo9Q;8eZf=cG7iInVTz;;e zqE0Yl4IvVZqDDX^pr{Y3qTZjjq6R+td}Kepakgjr+|f7VOfozLXx?z3{ZBd0-fuT1 z9^Mt#fpPNFi&g0RV>nYB8qO4V@DyX?#~EV)P|&KwqoZ4AAoJV7MqT8?bBB&e+~Ya* zhD)wI_;Ebz;RKpdqd*M;c!0X08Tv=P;Oj8Wc+m^jR&t$+S77w_%fXR3ih74C>TL#} zt2fjYcsyJyzXitB;g0@ald$LU6zRW!!h30cUVo_dNI-44akn8MVObQaBp$w?+GmHy^$S!)owppRr2) zDL9}1nfE3;a(y;>P~08yTkGeZdv1jSf5PE_`>D~M_slnF zLG_uz%+{2^c+fTmk3aFuKaT!q{_o^7*nPB4xqytdtSZ8ii$u~^a-BYWC0CHCUwoP? z)p`%(!k-N|5lE(h;paetX851XcfO&pGMdjne&j!J$jij@)XJqrSIIB{nEwscPuF%|Wfx<|7&=&? z2U7BHl?4tT{=T^9{(IrRr%O>&S=dI5M$?bnu=NIdK+Ps>)87QNXvtjzKSN$pgC|4J z!ZaRDovGyw?o6b4EngVeg6jI;4z$?xE{f`3O+NA!6jGuu2}5rzg*p^w!15d6$f zW5Lh-{R(~-{M(i+`4{}=k0tEenX zfC(DKpr2uzj5J@^3Stjp7>HpYcVQrXOr5?`In(na6m^&YPtHJ>TB{CFcTFvCm}eO% z8m7kB?d-!q(C7vI3a$V*c!%Gccn8mAsU<(z<#t*LE@nSXvlx+v-?+7fo`x{d0oRl#k`jEf~<<5q3tr zx9cH2C=v_c_~?3V(*?Or5nw<#`R2N1irQOo>K!1!AtEdf z&#j3Sr1|w&%(%P?r>+X+6HQ|K35m_*Xzg5Wlo+J3KYk3bUm4~skXQF&YU1SpktPg0 zRy@Pk9F~Eg&&{+RX}|t!`4}^3z~kY=+h8KWgV6vo!)QQH7-RcnE|zzZ^y$BtM>lUJ z^Dr_U#T!Xt{E;D)VIOv!>H+_*gh=Yh$~@3wUX@G0MVDAD4$S%a!;k%XvI_?R6wO~k z_ZDuE1OFlbBkO#60H@~Q(1U9&`2OMg_0R@Y(na;HF?iH_^(<@@HGPOQae@sS(G%i| z>giP8No=|yD$?Syd^w5%#Iv3&tp@>c;0a1M=|IYDU8#ZRXq{86paJM8QTNdvG+Ixs zWad3H8ixa+*#HxyM!k+FBFURkC>>al)Z_I-b1Lwu!5>Lo(dL2oi{#fMFZF%rY=uc} z3sR`M$q(Jc>G|8pyTk)}ysjRg5=%z|20Uux42>dj-eEax4?|H$st2HR%5VS>G64SH zucy=n>$_wc*C&1z+KWe()I-$|A?ywP z(Nq`9n+gWN=8+?qN`VL-A)viuBMVt>UdDm44yK{hs)D|Z*7ZE%LirNBV`vZRDN1Ug zXGgLKjw7`&bnf8r1cJ-#-HoQNuAWG*;dnB&^1)@&8uzrGhil7~g9h-VQPK!pqk$-l zb?^twc9NPNL{T_@5j~DJPWBrOXtYHE$ZWa|*QeZcAG*W{Ci%bT1dM65Pt+GZfJclD z2;RpX3Cxr+eWxcdmMlNA?zsZy(sUN|}v7akfc(1(UNKe90w6}QiVgs&j zU403FE)pWY-@GNvO(*~+b+D#vA@X08=V&+Ji7D&I@s^W088dgpWwcWs!EihJ;p}DJ z_LJGPo}xV^W4r`teBHWr`bx^hM%TRRuwK2odJcSv6ikT~ZD*Zpv|#1R6|#Q)dYhi& zWghkF={ulRxcHekMO)GS(6+$Ah7B8xCmS?uAcghZdU!G%PB^c5%hoM2XU<%E$8eoG zbqqcPj4gu6LYLHNKn*qepIJ133~)zYMvJ~!{_Mwq=>Z*oXn+)sPSb)6S|T}p=kx$H z41F(My3`aV@3E*?x1PPvV70#7WWj<3_O1$Epu&3tTzERI_R@L|Jk;fmqW1SIg(>MC zz;JvJE*VYmEVU3Kz!PE~oP)-H>hqJFdg`gts#Pm{9iY7cLJ#ln{!w?i|GxXBS~c%g zfd*56@m-jKdk)t0!bf@F;S=0X2XmTrJM@GRO#XVpeGk=!KX0VZQ!B$Wk^`6TspaQ+ zYAKqs(d137P4~=C1MXgWu0}tK5RNY0um3;}R<$2Tu3mp1{NLSkuA%M&E6m3N5nkzu zU>P@VoLqI)RaW$oM;?|MHM*)W*sak4?;LDskutp2bi2N2;QQ~tm*<{)&hp>h<#u`W zkw=a83E!h!f@wD#ELK6#A7EzHJP|Y$bDw^Fq~)5Ha@N^r*-II$F3pqzC;Gz)w4FN> z<<(bTl?yMtP};U_YcD3)y?3`E!&|i)3c4U5^k*pC9hvL!oy?>GaP2~;erThG=`eIT z`z_sYGsqMs`M$gQ+?B`SzVj@#@4ox?J9+G} z$E?X)Zn;ICefAkyvv!TV_|l8=?Kj^_xns)8?}LU&-8ywGvg3Hd!5H;b`|Nr3V1w`8 z)zxFKxpY6=l}uUnfiZY<&9-gZ?5!(!2y_xJxk3efeWPBdPMs>vnl&>^hx{*+UB2U?Vw?^bFu<(z|K0RIGgwjBwQJY55T5hs z$Pc~aHNYqE1RnK2|NJAJI(3p0PB_6+(C~%wjcY5tSL$G%AFuwH3dcy_zI~-;jhfaj zMog$S=oiWzPa6LH?YG}78isMr8}i%>2SSxiFPx#uWd;p|syBA;=#T}OG5*xy#v6Kt zwhQQ6<#@$0%6rC)8Pc(1M>*-Fma=f+Liy{jzw8ZP>(;K9OE0}ts#dLPIxDkjJ-BZn z6?Jf?%=!&N5I}g<%rDHGInx-Oke-{|KjxTYeUTdl#8Vp2972SK<9+i zp!F3iR!GmDJxy6(cinYz?6Jp6qehLSN|h=$^+Hs~PvZy+^Op5GVZsCx23@;$mGb4w z+jpLm{ADtV8{nmWB58mA`KL5#(!^j#SOvGkZ}J0o*|KHLsAk$~(4ay3{_%3A~4=si19 zwkm2mMUJvVskuZM66L2)pKf0GO*h>n4I4Hz!Qk=+D^FKGJy)lAsk>68 zN-|~26nj53Yef)Tz!cmbzqyZM$MbiJolrex@H7x;JSPR0PY;~2GBFHfB?h2d_>Ona zfAh^Z(z0bsIrGdjjqlJ`7|-wqk)w+*zF10^E^V})9khK|LS!Wde09?)n3vE3VZCL` zmYE`U9t6)CG-zO9wPC}CSxE$l6JWloVqaqz$W<7C(Mee3+qZ9*-o1NE|Ni|2FZ%rR z&o{-)+_F<}p%ftZ2rF;gxY3MN!sB?YgwB~i%c=yA5!GVh40Fz-Mvby?HW7hvyaz8+ zj_4SDZ`ZD!+;h)8W`KY3#TT+>%^Jf6c^q!C3jSEVF$`n`1CAG=)yb15+f)#aBCD^V za}+f^P58UIMtU7RXNQueUpbU-n@Ahf(s`s ztWfMv3q=#*2iRyLazyblt66Hwz7;%rHU`1Tc3f28YI}gSrL}9&E;H^XAP> z5t82thpg%V(|X^@5{7l)2vVM zJ9WhPIIBi4R&Pcy09|5~u3ELqrkhSb{d6;0-9kfX9OD@p#=&$E;Ec2xP1$T$)DAcl zGIMlz3TzF*qjm}8FeiYrMpz^!CYoYTO<1_Hv1<$i*@*!d9J++bNh3#&l#f38NKQQQ zL?L9&)XwhRyG=pC?BSAAU>K={yywrKFXx?io+);CC7f~(S6_PW(*|%Lgxa@nZ*AgP zlU&5heuVf#SrYtADEzX^F0=69k3aq>zx?t`@*zXi@A?{|n%J!v2C^Lk@Gk~3bU$Xy z7+c`Uv=7lQ=-Tlu+qIY)BI-mLNAuYMaCpg)g{{*;ybEIde3ma?ZXI_B0SSCq;R@4( zJOl;hVoaWe0~4z{DhxPgg+}$@Ao=R6uVn1lvBGwaGtM}}XcjLMx=xF!p=-QOR(SvP z(@&=82zg_)I=+YsxO8TwE!re}&paIcAwmFbX(?Iy2%i!?z_>W+q?6>1JMPfE;49_h zk3Tj(r(W6scd-=1KyJYR^v(Wgc2}@%7&>-qqa07LV3H^>#&mY;p&V1xFfv{U+f$gQ z#_PbU0Rav%EF?So?6YN*TJ_{1teD*h$}NNxgClog07V1C5*B8P=G}MSwcmH%d8aM> za-nV*KCJ{zlMkL?3s=*oO>J?O3%jLNKC_<#3kC+OTroKM_3LMc6gg#`R(nnnQwLEq z3>U)b@4fe)?IWjt7fz4yb6VJpJTbO4)WNmX=gMC=I~s;*foZM?8E>J z3f*EfVsS&4pMCb3FsF6()mPh&g-Al(ewe8f#l`sn7^E1d2niSR_UFyc??k;=AVy!= zGD(6*fj6z>gc!;igPbt_4L96i`1;|8A51V(FDa7x(rPdEJcfa2F#z4UC=ff4SSy9D zfv?>&uFyPl*PJ!LmMr)`yHkpj$+EL?5QHFL?NrLHk)$- z5{S%7dysy3XT3d+a=Ep*p%KE#GKT)QaTylxL^UgbV<^el0fGv_l9ejukfHzjr zvJi)LCvU&~w$N7$aq#BCj5#r`@PAAOL^7heBafpIcTF!;j5)l_8Y#T#i!Qpzb|b;l zP7iT~Hktlmp(Ol(Udry2ql%gi({Y3x6ZOX*f4ng?jLRT^B!mK1%sJuakNp|(ywZ&QIoL`<@?Phdb{_EwTCFFopnCTfe#vc8_m30_6iH#dK zHX8KTM;;bPVn}0XLd)=LcBdRu)N~Xik+7;!!W=8gD%oH}z`!7J_Uzd*bm&kEG2&?> zup<=~)==ON?Zy{~mqG>gm6;h(_Wn9!-?I+`j5|W#8FX3MvQxe&5?&_5>05-Tues(L zV_ez>7MMF)S~NOT?h&{_S7=SPL?%ON- z_NL#s!|=vaCQ{3mLg)dR;&=}l!I0*ZGTMVaxyQ%l$u5NCsG^2>K@1);AqWrxG8|KL z1tAD#7z*zNE1dIJ+5Q4UgEaC*n#fOEcmGWpAe z`<}AIk-z-q+|y9^+%x|*T)yD@Y5n%s8T{Q}*8e>?AHUOj9;!^}drA%VuGaYc1?u}$fELma;%!r&pw>gn=R8iAmgaAsDW7)8*3H`x5euVHF zOPu*^4oGKZEcpqeao-^{Ed;s&;w1?v<{kUD%ASL}?E8_75e@OA)iTfh_4&VtD(ipl zzH=p|#lSAVm;3PFaQ9t~w4S?s(O&)a`oBk85BXJ9D`O_)D<(zaim9S2taOrS)^4OU z5J{^jQR;KySVCja9E-eIFM%Zv-I3B`SyMj-0Clj(m~BkZJ&_|)X7*;H1;K{V!u@D*Kz?`;MUS%NXjP(WUw{3zoP6@hw$ePk(+O#Hm3Co8j3nd={GXV# zT@L>1*1e>~Uum_F*7Hz2;J?RrQZ#V*%X_6G|C^Drv=R-x;o1s!KU6Q&*wcRn@)ne0 z@g`&4{ArO?J?rTR=v6$9v?1KA>+DH7|*R@0|e|Nn4qjCr1@ENI_`9 zxV(jj)0kK$f5iKF0|*3Z%wx})Ge?FD8DbVV2xp)o6&4_ycm3`f$i#9>(;3Jg-qc=7nzlyM;Mf-3-69*;S&oTZ@>L^^Ca;GopI=xI-0f- z0F*;fV*qe0A&aWGhjAJy9Y@ns&eT!7;HP3EV@F!PI!w2n&|fo8r}HAU2XjdD6avd-Shoo!O1ppQ8c;7*@JLO7{r&gfhObC@FI0Qr8k`^~kpZ{ecAL%pqpU}a7-2F2@0Ucrj3!_;no%ci z|3N*NC8ne^iqae@k8`#EkT-seCvqXJr>jj zb2+o{ih?GLMmU?fKp4_l>i%ZA2SYQb&6W|uhA4KtpR|O9Jy+&L;1KrKv(J))HvTqZ z-;WLqAYe(E;R!)+&;_Svv7nJRw-MeY%fI3xRLiw8@lpMwBS^@&U(qF-$Ssdh4w+e*Abl?G3L6Wz92$Y-S~-Kr9`t z81Uq{jiJ&`Bnce=|6yb@&3f8try1QrQ+~cCAG(G}4*JNw2`)HaByDC^o*Y-yAjlyG zuLZ=oeQcpRNqz(yA#kiA76X7#m^d|IVFMB5bO-$m)m-deuEjuL^djTYkiCHJU_QJ= zKI|&^@WT)76ry|Yz1QZBdG5SjKQo7{FqNB*#<9{-K+rfx6>{Gh3aJ#&6^|UR!f(eN zcbqK-V<8;y(=Ymxk>rFU{9V6(eOqY8G#D@U;Jw_mkHV(CjJA~#IAdjV69y0fY117v zgOcWOEp`}TXu2bMgZUcTVrq(MDU3uEHT0d6e468m+6_4Z5rlx8;Fol`JX}GlvuxPg z$^L6zO9o<{Ao1r<@jU}VJw-jSAGrVn85>NPj>JMnJUI+Lx0s2SX}G=maqa3bcQ5`B@Q8r!!S5`7Obej68YIX4V|-ynS7Z+-{~EJ zj7y;Owj5Lr<%{*~!ru;op^yVJ*43IZ~xyh z0DU0fVL<4GeU!|Ha%I67`+H$hls9x1iixAe5$)lzvz?6T8V>ltV{~mqTVGD+2Pg0! zr;IVj&O5hRc!#$OA7?aCzvFT0;~*H$P{%;vyl`Hf$@pO$fZNo0J~e1!5+F6~{C{}{ zeZcFym5vv0aCjE<13fvz(BD_+k21_pvu*-|n!Guga*d*P!V_x;q3Obn;gAC|@kTh( zkU1?*ic0%+m|lp)1xnmCTpO|bxfuh|j~+Be!Xsk=2@6Vk^yp!S>2&Vg*-pqqd17gY zV&L>(1BRzAK7aoCr?pGS8wJEO==f+$PLP2^PHTb(@p!=-0?^?(qx}MB)Ww`W=PR(+ z94{WjoP$zWPvCzW7x;`WMVmjm+GDwM9s}?h1|+MzS$o1+0C>pgmr%S1y)lgn%@M6- zwKqmK^@PfwT|LYdirOInBvr3o-R20HOTe=UmncJi5QyNz8|N5Cm^ID$>OiHrK|}&? z^a3UvR>I%Ujy7ED8R=8xWg{Ow>V#HcIOjz(7tKjqC}<2ln3-@ZDYKL`b|9PgVJ=0Gqc(T1}7-khsgpKPS#^@uv0n?&ZG6pIMWyS zl`UJ$^E0o9(lk2C#E#Jxh5-=|geT{P<6X1wnzaS&{=nG6aG>o}+s`0k z6J1oMOqm=Hdjn^_haZ0UnNR@d)IGcptmXOh=bHrt;vg)Cz+kwz4+FEkiQK9eut_gi|*wbMGX=hqCh0uFZptLZRWJU z<EPiyJdq(QcO=ZCyf)T~Te4dMF!H3*;W0F}g(;gpW0w1QlkP~>>BS((3@H_W7 zH{FHdfrB!@$ahB{kt%BX;7{%+b@Ghg`qfK;tnQg=FynO|I$UtwylJ!SOxz_!ixt(i zH9MtmuV18Kp@LGrLV4?#%WIVe)oJbf>Lm~D`I9S;(p|TDos1YVQpS&+VEkROR7snD zF}xs%^uNo`b<@U8vS+uxM_l1a?uWnf!1v&1lqF#dOqn!I{+T;ps#dRJcxS9g!EoBm zz#qyZcUCQ5rLazvij|Ks9&+R7&&&59{wLz!-0tE{w!4$A7)*i&bC&e>7bu|EjpHY?%a6kl!J9f%DufA{0Q#emy*}G@2 zY*@b`IlN*-7AQl0+MrJQ2>jgRgAzax5n|$bsD$EOw|<=r9XwpR+;q3>-gWprtN#3y zFIb?UY}>kB-hTBx*|=e&6fRIWSpd5}a0NaG5A6d7xba}o#XkoR$xv0)ix(_W{`Z&y zczGUY*o2?}06+jqL_t(8QC|UN(E0Ng$VF#dCW{s<(tf2i?twmY(GPf{4ca21pgFYU z(@#G&Z;zKhaHcl!!?3i;m}EtQhJl5UH=$X=!gzn-7O-Rm4B1&Hc%*NH3OR^~Eik-n zgs}k6S&`h>xv|8H=M6|LTeh_MNQ`x&a_~MOaT0Tmj6a`9Qz%+5P~h-ZQ1}wsb$o#L zisnl$XU{x-HE&jB1lOUz4VNLm4V8j%o@`jRcD*Xc-N~M*$K#&diPQE-)QRGa*Wd94 zY15MhPDWU!z#sC`&e~OL<>`l?li|M)*HiWqX{GFN0;| z@KL6y!L#)-Ad_g5K61~&!##sVuhfC!|7XrWGUku5W_0k|WS2gtT%gGSq}~Odq3u1p z_Q=QYeJU?K{fdp9SR#>1IHW|M~~L4TQ9jM zN-|-S5w0rb{rmRIlEq79$F?0(T)ojsRVtd*2eAR`@?|Sz%jPX=T$EM|u%Znxa2`lH zAj_66SLL_Oto|z1DjVS~*LwgqLCL;zX3f(&mq}IiqU$xRD?BSyxRCZw)YCg4%a^XO zIh9gs`Bto??I}~D(6?^cro^1juGCwhVg)HzzO0oyq@JtsfZ~AmF(_uwoGWj>__mB3 zHcE~=@i=30&dPRzgwAVr%fEZiZdsvynKM&iTeM8dA5%WC$9S)-TCqy4|D95#Xc2|y z7%7yXf=|nBP=&r`)f&{X!BRoXUYj2Hvn? zll=TcU#Y10rJb}H#2Y>8a|(ee6BGm|53x{;SEI7e4caug_>u*QcBr2*U{yI5 zISZh2L-;HUx|>`j{dK&@x!r6-yXBT!OhD83tiU;K@_`TeAf$O|3gb>h4nd8l?GBb9 z1PW}72gxa^b9o0&0LFcZ!!RZl?n7(x|{n zl@GvEx~$4Tl@BOoJmbPD(@31Fj`6>3+jc3Vd6=32R&eVQc;thDFk$Q@Y2Trp$r$%M z>@~n|I2zywc~4}82pVfp*pUETlKkxj*XQ^axG?ge8@6RZyQ!56t|zO0=bF$r9RwjD z%vrpXCQY)i7UwMo4{RueIE0dy1uifwOYGS?!U|DB8g1LPwey>KDTf^+>LV6cCosS) zcnSUc43tkl{9M+mvU&EA=am=^O0(m=7l`-!@mG1}nb(YAK6vvZfvD>>@D?Y1*YgK? z;fa@JyDIn(-~PymHX)&~eDm3NGO*uoGHKjoX?JQHWAdMS^^@aIIt~oT7ax5oZ>VzJ zylIPc*K+i!NwdaAY~A1aSibD>wK34}!6T&CH@%JVj#0)Tkr!g;%0DdMHm9_fb1pbb zR;xm7)#fBAQ%)C0DDlxh%cs18qFlXljlBQHhiZ9`kSUX=$;s_nnFapATOY|=FTW$Z zbRkyvw?9?`Ag{HF!SU*IZ>TXaM+mFG^33aUa{Jb1IQ%hmq|~TcT^g&wG-2FCx%$HE zrRi}^q*k3;hv5rE0vKzA)3;xIZ)MLo=QLB~brp}WF=GLo{)B+dpC``drxJH0nqr1A z$BY?cbI~Z~3of|81hQi`Xo3`6m%rZ3_zt`nn#|Rr7;e1rMw^Do4dJuQ=wG-xz^hYS zgur>>2#Iwg+z&O5;p&R^z8h28L0EH$4W|~e$`>Kcd?TyFd6CE~ue>6spMJX3s8PdS zZ@c$^?w3#HoM@EJoWJK9PtKn=U*Nm1KmSe*-{$%fkxDXq)*N~E(dU&9X3L;oe>YFE zQPW0Js&pxts(kd&-H)159;FIsqAH*owW`bC(`L!z_dO|-C;TPVYF0A?|NF0c8^6`8 zSwnedlHAwjL0PtBxxh=KhL4uUS|>_w(658#LmXr??`Em=tu0Rnf_kK7{kT%}qSsjrU2 zsNrLzW7|t4e|$a})c<#xGI6RLf8z0`NIzG3g&bVEWSQJ{-5mnoA_M2oU7)gUvz&T* zdows5xbq>K(rSKuGjmFy2Pdl-H+Y6=!2W#)O6RUO%XW?K0ds;z0eJ4l*iRdTz6-|{ zm8AU#Y=1Q2YTj~(OyOl4jBzANnmU089I)UN+GRS0Gd<7&z?su2*Ll<+jDbPXVtH|p zNoxDvf)VHjlu!rrR4m{@ND>C+WTO{fd@;Dl5x*VbKzw&<$oQMSJ>~54&Xg;z?Id4* z@|9V&Sjr#1)m=XR>O-mDsJ={8#rydEPbot+G^_NXu8+!ZqklD}KJBmRa{j3o%h2(I z88ijXS=h-owC>%zOZDp2llP}*B@SHOjJNZGm_lPs0td`64aVMU5^qDZ zq%>-b=lx3YmwNDs8Pne1{-oD~^4N2aD&ORhPe1rVmZ_IY1g(dLX-_$=y@g?aChz7Wr}dE&RN&0T?3{tIpAalhJA;#PL-|? z-l=>Ur}FMHi=_ShLm&BJU{5*q%u{Sit5u`+a_#@Fwn*0VkG~{CCk&S3S{y4K^!Z;C zrU-o}td2)bl#Wn#1C?k0yZuHfQL2PR*PNpOk24pJ$NgFNFRWhXwg1pLb>MR2nqqL= z4;>?o+sTeyJ50Hf_tHx*wP-9#H&P|-sskqE3AzK18vHr=;qv6vHPkj>y>6|(?< z{im$Zb;OkFzy)Cu3lYJARmAo!=Fdu(Ds8!7(QwINP{u?lUUlOYW@X|H6WY1w!MjWW z4*hMIRMXJ)4rMkJYLd3IZ1D=UqNl6(y;fd)^Eo-G^@$RfKTb|)*+Ql+oTLjy%9)2v zSQ%n05m($mD5v*<^RJ0h$a`rnOC2Z8B=GfGHzsHrE@PhM`~8B zDF^f)9xTkx-1DUIlg&F8Y0!z3ocTu zi1w?hTB?PmPQBW4|Kr`v(#G0dyl}CU(2y6zTdQtORj%_*c{gp=L{2^96b*@$xB3bd zE+|A4h724g&%OS%hS|>3{^U`sd%3|$n}o_*wP~qJzM%Z_Q-8VX_8X*P>54Xwh!;cX zJ~c({=m78e)>}K9MGm7e=8QK?Mjn^X^*gJs;Dq*Rmu*TI4A3n~!-T%}`M;3BH-fHC zojT^ZvXHM!mo6sM!3ATVmC21aIEM~MEJ(#*Vmt_k@~Tb}ksOp&xWsdmUh@;0sRDjc zejhNzlw-#W&#~z$LcYKL+}{)wUN==CBM9A=ELp-lyUVV=RLYmp1xdQX{OOmUvS?PB zvSl;`UPi;##iT;H3bKCVdXsk;dCQhAQ~6R>F1zXyDO*Ov&ie1RuD8fz_dR9dcWCUz zx1N{g>Jb;oUqs<)sG;vl#w)XC%r;qs(H^hs15Rpvf>ctkA6yVI`(AnS(+|Fsulsy1 zm36v}btID}Og1?8sUd+gP`_b4b&|@O9Kg`oxof99^yCA^M{~80gzU?dD`hAvS-PY} z`|!ev(p9Qb$?(I}7v3&P`-iW3Szo{Z^=qk6wt}qLu-2l9c&3opyg8IMf`D z)@@p8D1D_IR6aru*qS1R-;HaEu~LMyAeiyRFa&?}opDD=M@nnphx)k^_I8UAoxzu) z%^doXD-|`u4vVOUhAx=vVhwM9wm8*5`Ot%oJQbciLY|w8O zbR1MeV3!&I7$i#;Ew%no50*Lbw^a*%z{p=@goerqeQ9UqfyeLDuy_;e%Nkw8a_=K| z%k(KTgor`S+BK~WR&?f5X4J>h$=7Ohw6eKyA`=LWGKzZ)78uVd-IN36 z30QEn|G<7zh)ytDzq42&q=bd;d#k*e8^yTn+O?|*nrH+@aGgivZ*b3eVKAL?$|<&1 zVAQBl#(x-I;K}GQi#taft{v!%7ce~c>~qRb-jR@A^N0B%7#zw1E1z>OI9tn?u`o2W z%F66+O51pi7~QjU`sW{w2tj{@yxXV&jp5Clb0PH>cPH&OrBqVANfP<^z;F8ws8Jbc z2V?bRwQH#zu8TJvcgCoH+sB<>FGnmCJl^tt5>fk zpM3j~4AaG`AHCDV=E868a-&>)#l`BNzy}CF}3KnDd92a1R)c0o;!P< zTy*dPDOI?Xtlhd!K6tacV2QJ!3j2TsQ@Z5Bgb9igC0dg~0$nM(54%qg<)pGDe6UYV_MvUu#dSDrCr#LdZn{PhP$ z^!Gf}RnF{qhCF!J!y3~1Ql5PAF{xX>j#TT`Md9#Z1RloZt&d+frbW=e$gIv~@tXD5 z%W80@{Nv#Q*fAWUWzLj!W$YI)Ko6`I#js?$g%zVPI&l5hk{ef|V&LL~4?Zvtn+RD> z;DqrZl!?)Y0)^&~3(6T`hX&fiC`RVYocgy$kZQ?kXLOM5i96(T4bk>^|5KYPs;Nrv zLS2}3d<%`9983}vl9N-jX3Vm+CRHm`m87^N8Q6c2E)-pE;azx_PoezY`_BnGgE!`6 zhyVVEE!=8Uw}JBOzhPlE4u-0XOs=IEuv=Gef(}4R^`d=_92@v*Tpl)(xt-XPE)A zeA#k4w)u{3w@cpp^2+3if63V=pC^rvZETT-epWY)52Ht&7a4OduS zX_JMZ>(_0Nk9)pn(MHzN?A*Ca%jePG+a*a2KI0+SJ>+oFSI;m6Zg7~M1Z}Y3m={8| zY1784puXI~zv=5$dN2`18IXgLWg$@Q+O^F?;WvW-@*+(R$q*&XV`Y7V?faUdO7cPc zgmXH!zff4j#nz!!8b0FwG3w=>(DHa0J8GQt`u0beJMSO)O^NaHbFR@g#OAEtd+h`H z_RH^O%HLDv>rcP2)z_@}M4)fcJ(Mh-TD*3NF$l~Ap`3R1Y4XMkZ^>7hfAX{`I`{OY zhN?c-d9Job_=KKeBp8&0GJ}{&#;~gJQTiU^+HL*g;gXOQ7N?_jaG6{NMpw9DY?!9^ z*FVbQ)k}oX7AwA4H9BU(7~R_Os(hmhX}Cw|{Dk_gWb~*pDo9q?%GMT&6FhEKqk`|! z#Y!2YvIwnzp85#b zo)=gcAB+T#2O=wr9DoK#jTj^6ceqH#kDH)TtCbppUT+GAsi8LQTH9jF$&>%G#a%2O zBcfJE6(D^3j4nVOH)*^K8!}uMRbFS|Zj53+58wTmjGs6`BUrm_CHRi*J9I5bZ8`sv zj`Gxl&&U{E1UGodP`OyQEHVd<_X&S8)kQ)MAP?runrm`|Fg?5lZ)2E4gMD?O>hA-F zsuGTqF(b#x&pH)BsG2!p3=Y&XoAt+OU^j=bb0NCH*FUTa)x#= zP7dl6Ba7tDzvMiV0XN=yy-k%7wYWl8!ZSrkl+QUJ24wJ`4D!F@QTN_+jBN0ln^LY? z_3aP?;+%ztlI09xR&W!JKw&$?y8LNf$!}#;sAIw55$@5G$x=J;1Og)5ewr?>n67ig zQ~#P~9w*jdYhA!mu6#KQ^G^J8l3Cnrxp?a3$82>b3Yk#u)Jaoqei^Iqm3Lm$6_Is} zxmX=eIEfXN7$a=^Vw)E8HLZ2EHA)cA8isu8rN@Ppl7zj$NevCHmn&b!zQYV9HPlv9 z7jPj|@GQCVaX5kzDa*EsdfGnoVoq3DUbVcbmxPint97%mg)kVFF&_I!UF^j{DD6*c zXMJFc*y(Ci;OYM{bd-7#v$Sjhx%$Q{?U0lIYwtV&q^hn4eAtCu>PklhkzxVF28u{g zM6qEnv7sg!utbe5G5#qU6O$MdV-$^=XkzaTETC8r#SYlT0tglqq)P|?^PM+m=FRNP z?9Q^g%CFO8RGJ0)i=Di zT;0NNzV@bVl{$IUNICqNLyfK$>TVFWWTL=|8W!npjPG^bNK4s%2VK>zJPvPzUliNw zm8)fuZsXjh=hikg_UVjIRndQAhBT|Q$>-^v*T4;Os-riiP1| z1guLeS>}_NhgqR>Im7`gaVSdy`CuaQBDk@jT|pAhLJBNk2!Xaz7X-tVHlkc{X_Jrn zYh1!Mx>N%8_G7dVs4H8bEuA2|SAl*7mw<;qAISx3FVPbf0Df{{8SV`MORNey8+U~p@lz-N6^UPoIUEVyl5 z+Fpu&C{_c1iNTqQgVcIbpN$?=cC&StGvnF12Yu!n59j3^wIdUPU{u?+YghU7(@)JB z&$T1Gtnf9VCc)FPZBU*P&#SJ!+bByLR)x8 zxe#EWfOP~6;p(1X%7pQYiZ)nzFoETRnU!ITft4vR5O_~HC{K(A1VyGmozw^OlaDrK z!Xa@mJPMjJff=jW7<*EdFfw_J?4n{#30)wxo70U`%KEBM&XK+ zPW&cobEIbFTa_Ho ztPUqoCJVfnpN2P)8Spac;X!jDD{t}a{N(p|$rzpTFB~W{l^2O8AAx}ffpqkd^EuDJ z)>^%WC}W=S3N~oTT(P!sU7+AOP0eK!Wl~pT-QmI2*OXWVu|EzskJN`8FkDrxi^4@! z`rdIw-ONwZU(hCP%D0fWiki-%qfms*b8(mjM>1luF*>Ua2B!n*OzQJsB(H=|h{r)p z`ik^UaKs^qJcQ$n8I>r!4eo-65v@$MJe3y{W$Jq)aK%TsoONqa(AgJIkUm{rhbw#~ zEldQ#0vpfVn{^y(pD~YSA|8Sc6Sc!e9>b-e832ak82+kSZguC&Yl#BE@WaXhc^p)LHCNT(;vn^!tvm>e&h(!%8hPSzI$|WA zXAkkXSl-k)cU5^hkN0Orq2V(9y{Drdn3psag>-o{<)M8J8)>ZFAz17=R;SF_vz8XVe&a9V#yo>bZ7>9W;JK0+>OgB)av-&Ohs zK1S9t4TGX)R{}JekH%xgQohp7u9<`=Xil0!S#umH$fYB><{CUl>NVGLqV)9_128e} zN9N@odE^nw;bu~BiG5Cc`{ zQ#S}$2eoL?BKvTTdki;WOVU=I*NPeo4NDx$o0E>PdJvFUJylWEj)5u|U!FC(<_!j_ zz?mBfyIbHe_zQZ72i_e%0)a8<2INLoQD?J*GXh=*7KQG-^Umf1=NZEX11Jg+2C9aE z++~6DD_0hSav%<8b>ySl8=T~|q6R<^$&Z8nvB=rtfq>*)E+L#gx171u5~Zy@48+W> zTIx)vLyShgG0hixILw0)A#}7ZF$iqCEy`<`jttH~A zE^woA(qMo-tlH2Ew>RsL@YLe=c7xH4$~2X2vR8R8U0`YiAs^V_q%!8Wu)aZZE{tu+ z*FGhW&aBuOx|{H>e~^a6(XvF z1n@1RX!x14_xThko-z!cT==RVxKAFk^9s>ppNZJihll9lNvf*hA91cI>xJKss?G&q8aDh{|?<+1TF4fPa zvgI-J_Qtw~`?}B`-R!0R)NpRlSTNX;0sd`+%CygAKD59BWj>>f>pNfF-n6OeOQ#9Ks)LB}<>Wdfqxy;}&D3^4o8}Ef-yMk*$-6M$=WpF#-~P=Eq_@rV=Ji znq(`&-5?@OO$D|aty#OqT%9c%wUlLR7uzX?>`#tD{$l`-D5nxOZ&YT$^6?M`)wz%i zbjARX?SdRr%~=ufbH0XxP}J`JX0jj4hdBjC*StrNR+gzrhj3r9GB_=)tfZBcZfGLw zHt1rYh)MI6M&oed<|-*Fkw*ICaD|oHj|lt9_J6#f9B!a5yg3+$xV$+cl&`SC)HPXe z4j>j}Qg|0bo6!uPD7@~v>+HMLy?O;-W};CwfnUm`Pg$jpux?!3RF9+8FR-oGBYO2; z6!IGbm~X{;KvZKj)R~)x522%6X~o^%eAcpm|Nikt=UPrYV-4RFgrWv{SU((g0wYhK zKHXepFgf#GxgeLcA@cgvzyKjjuOQ!nmP zj-F+1yrfZy_H*-Giipx~VhnIp@#@w3DNdE;xdeEcfTCu5AjTS72GKA19122FI|5-Y zn0oT$$#&!$hfGu@z^GVZbPh)Y+L_e)uHq1kyK%jE;7$11-&fBSqDs`2uFTB!SDZIp zc~yBAkbW{AGQ*X*hKlmQWQ?HcH|}dvQ9488@}4+xqMbQ^{PD*d@X#TQ9)*p=0R0D^ z;YH@EaYIg+FhMv#rlwusuB`1`{o++tUq$qk%25^3YbNXj`H0QT+o@&q9OTH9r#y5) z`9VI*Z1tt43u40cZ7gqjS-~}cI;bV_*VFhLf96WaEK`vI9JQ)PIjgd^w3U+Gmel^h zyYMjd#%gZ{fEeTX912EJ!$cqx_cyLb+}}NV^r$jJQ=ZL}tqIP|4fBi>NrGsCQ~2WE zape`*C1#u@@f&#)`BLu5V0jbot;`hOvod0Z17|wciW7E9neqAqdXLu!5jtGlmv5ui zV`W*(NTMNeA1~M5Cg2alYRN61{)qvdEgwm}9d?(J$6@q1!-b}SFS)PD04|ufVwb_V zapUC7Gtabh=NSP;O1kDl-!h~RG`Z|t6T0WSt|@n>$Jeg4Ek96keH<4j&C z5c=QSXURjbI&V@$D%@6Vs0}>F>#&TmJSHoGdrxC|g9joEh7hRnx~$F^yd)39%yUw( zrp5}3waMejSCs(&VxY~OInzGD%5zQ9Syf<6%~Mc{8Y>8fW+55w?>+X|!-OgCtI}zn z$FHf#2{S~TUP6g7(MfW?1Tlr{iKj_Khe{5J*I%Sd;`!5+X>n;Xc^)s9YxrHFtZ?SU zeNx&G2PX1`N={@;t|OE$xqdA@Lmq=N78gzvs;ydYG2j?Uvl98f`|gwd_S?_aL{w!i ztXcu9qlFjkN#?+D%V?PV;_E>Vgox^4`*f-i7ld-KQbqaJ-e3 zk|qIX5(IJH3de`xh`$fPmBNDM;7%Lw?Tb|(FV~;LnY=E)hCudqrp=!weIlzyW{ppz ztmuyEEVg8!oY}#F{>b-GaQ-`x+P!;sS+Zn_xxaCdMIpjKbzuPJMRD*OKYqL|b{aZ# zs4;a_4Z`Yzt*)1YvgX9HU3S?;9(w2@bBR}V8)jX{Rn)%RG2o+S4Dk;?{7|}f?P{Oy zjTCk6xXhXZD=h5(=F^uT9c4`?L?Ob!#$o_E!fV4eEVg6ej>QGeXd2}mg_^+tyzK&$ z4|ja`-FLPYh2w$=5r5YV-s@Yb=n6iYq7mCa!ZECyiZW~8X| zoy-aYJ`@E?dj9Ep-tC&ch7wx^?Sj z2BCX#^1ZzUr>H?T!n9SZR?@k1XPGf$hPlkJ#@#x~cce$~Q5OqpWG3Zf*i zzM-|qeQ+7 z)?07cv_xc#*KS|bmD*fUbI1vE)U5XA9Y1tRzK4PvQG-wr%ic%a7hFL;%v}fw&v$~T zZ}2gCWaxw85jD{1lK#W}kv>A>h$SU5mH(`OqSybMOu)22<6H{N)| zRvNSFJDQi?SnO6$e!OHp>e+w){mo_h?6c1rpA+YOxG}X=KmP~xnZ4Y~q}f=$%v=Xw z3(N*zMe*74R!3ZA>#uCT0~5v_o>%AzFO9=lD8_ifoJ^LkQxK>ioqev%FS1IPKqqAD z0A&F?-j2}}9SGA$p)fE2qpj8B9yjPYzuEe@qSinoC}o(J)5kc+8(LzqNkk77VXyY$ z=?g~Hc+ZYG<`}u*h8v`-9#p~l6y_Q?mIf%lVWWmpTommbsl8s^xSBB5@p|ziR73;B z!|xn&(xpomI||tIZX6`Fm%E@9H64bM-f5?u*c zt-5MLS##!gRgd(9K5%_e21=TvXgLv!=ZcuVy4nzxQ+W)~C-e;`5*~8MA#(4%_u5ys ze8FE;co;c=ktNK z^Q(%`4}%=tS3d^mEBXe$9yoBIeE#|8_Sw;sPd?d{ZLU)l&;UxBuO`voYc{Ns)qIay zUBXErh5nM_s`^k9WmXlIT=L}w+$xZj78>gJsGFz^K#oHjcvk3B?$ZVOdiLzu*4K=v zN$YYTUHW`{SFnoOk;&+)UAuPX(s=K^_e@dKVd*-KcVCr)!O;YpgPWGQVH9OqQ=(QeH`?%e=|oF>*++jH(y3+NviV z)IK`O-p*C)zn9G#wXq!=xUtg>&gg6UnD`Gq_`tr(*0N2Z7rITK|6SGF8&dap~x)R){|zy1VdU_(4ZKE7mWQlJ&vJx>3VM zIvUFELGe~ID#Me)HXlyBI`PC4&9EypH<-7lzFiTz2xNrqDW{wwW5$e;jvYJ7jyvvX zu5nkw72ySC-$;LJH>`;(X}(Dxg~GwW>J6*4@0Lo-k~ZebPIrPJ^nh;Q^h&%i?8&F^ zNee9%QYb8FQ!wjR5dyet*RHl2ltpEXn(3sZ5PM|N2w|;uG#Wcn(uR(L`O~0*et<>) ztyXs>p}4qmwk9-!k;dUTx8Hud?7jEi(zkD4V_M2_TH-zm`HKPi$>*zCTt;8AAP_G} zIi63`4APj8R%0C*fV(?V(rFNt=dbkpc4la;|6!fU{Y?BzU$Z64PqNH3fXnmd7UuS> zRHTCZva&K8RkO_oi^=!LdRm0L4V;xhxySXbw^1lK3S%gxJ>zFV)BARO+5hE`wA}zyF3OA7$EoDbC0c8z!0OaD;rEER5o#xF7Gfv{{?+m zUIsu14A_{z#_i2IrWP$)6ztZ^f*w&rIWR7r!j(PMxHad_ool0~iY)AuF?Ze(Vgyh& z3{Y2?U}*4G?tVdo^Mz)q{Iad!WuJYC6YVVJtAgvi|C;ugtnmotMZbI~v0JVMEOe468@$s*M1?(+bE; zl`B{dpI{nBGZ zd8FKUWrdxw4$uBuD9NLYhlicjzciR2&t}b&r_RM z;0ZG&3t>2GxTXX31eRvzno?oQoHBb1QxL(b`|RoBu_`ZNqVO*sm*Wk%#r4UV-E7@q zrRPyc9cA#~afy_4jtDJ)(PlN< z(g9BgGO&@#8OK$&R&T6Ar%sb=>Tb>@g!$j^E1h3uh}X-NbxQtq#avG>taP2Vn6OSM zY6u-gjYTzN$Pnu+_RwW@gViFkA)un-B2y#~G68s3uU=CQ3es+DKp1dp7Xcv_%+aJT z)rlM>7bw}P)lWGqSFDt^Yu2hzNMBd@EzJZc2&(WeFe)k!1N9=TSX& zIh+hw;xa(%2i8gl?Mli)_Oq{^udH!a0`d&H0H#6?#TQ?E@nwbFfON1!@NL?(u|0Gz zz4VgpujXuMI;^xriB=WENZ^lG{x=DCeb1mddQpW=V0wVrkM;&q7xL!p;Y;$FB~XKT{{L(LUgR z?7?xS7`tq>rM*W;63W3D!^y`Vej-nfeM+Xk^|nlVbGp1U{atC>uC26Ecmv}inaQ+A zq+t=~{Wi1sVBT3&Avdu;XW*fdO_GT-N9B(&U5EW zTO4>nK%w(PpmQ=A2j`Td%M|QD=V4m+`*M5`;}!GVp@7rmBM&3ixu4IGh2Jfd)@@oF z-`#ZGzs>v8sY@s0PvgB9*;9_)A`>)kxZE%-VX!Taribo7nenMSeBUGT{417o}Ul>TFv;5l5aIi8Ib^^@#*aPD% zo%IjG2n(SyL6cE$^XARW)yc7v9C8p=(%>ix^#cR+HKREc@o~o;XCrIQ1;?|-w*XkT z;v>q+&=_<|nb0?$Ieze81ASMH50u8O?3j{HL`5C5B?!v7uXEKAW&K+Bgcml+>@ zEZ@!l&aB^Udu}6To0VBkMrvN1w>wuee4n=@(?+UIXo2z$A&$It*>~E?pw; z>PR+#ZJJu#Wzt6BUc7Lz%=qwQg9{J9*4y^bk!~v!C^MBAXMQr%aIsalt)#~`-L0(F z8jpf6R>r5k_uu`%(s$pwTO0XD1MGrZ02T^bfd+LjA%3eeTNo*kg}1Hzg)s%~I{ke7(Wj>L8Y;bYQ#$UfBfX*+?aiL~x$!!4!0WowlIq+wJ#S7iW! z=_R~hjL3UxohVQrpd{e_!HnQWjY{Na7n~^_+jf-h%A=WCH@_x^kOAq!1UJl?LazWmw@7UT|w1-yq`{OE%j^8NA^vbhd8kXMYvF(%43 z3r6H9QQBfMOL4cEmK}CSV=#TKW&iZdpU9}QPm`wl!IbrSv@2PMvgp8};cQ2n?4O1bqMf_sh6(<7D)x(RPF+pSQGQMAIk&ln5ic z@0aU>q+9Niu@8-t5+(M3UO7f)E5QvI+}{+?tvCM9;!8^#%hiAWrzzx1$)NHXUp&d_mQo-cG0P; z*X8^(F4Aq zL-syG`KW;`UA$B-zvK#?&T1*W`}DF=CH)MKp}cTQW2_>tP_BHy2Cw3?6Q0v|1C$?w zPd<`|(J1(OL1DiB?>DOJ`W4xJ$KIyc$36VG{Ak2T>Agcwd0F2-c+bN!c<3Nmuk!V( zKVB_w>)>O}>NWDy6HZqJzPYst<9`0P^JVYhL*+9y=8xF_Xu++&bHANTPEsd*Wb#V! z{OU`u%f~Z5QJK5krUTckU1QT<^q(6NXxb?GmEFLyt7ha7Ooelp_lgXF$D zA24Nw6%Esjx%FDOj$ zo^tX}N9qXkLpkT7vt*ZkJL}x+c=>kTd{uZGWYTk!<^DS#lqV)VW|r&$2k$R?_8lg> zse&Cmbf8(ZXPo;JIqSUBP02B8ed~>>Rwo3RO3=(hm6ew1Tzp`7fD?$DfUeLYVXk@YsE4%1JZ@T>`$vClm$0|xDGbpj*7VAU%ggggFomz;R&@iJuS zV0m1Pn15&;x7~lM;ee450<(Mw-mTZga(q3h&-Q)Hil*L3Ntee~eSCw$!YczWqY$P} zooZ-s*LOuYLD^yCk9+tDRlIHGjGrstDbMxZp|{fhB9#yG6R}MUM zf1NU#s|xQ-8S~$(<{Hm_s5=e zjJn7dsS5KO+#=7uK0*5J)>j7`1I*pc6dE{Znt*v~ z1}*3Q;%vhM?s4~&A&*!`1HaCiIm^o0K8D-<2ab@N@BEhyGB8RoI?5?s69Emhyfy0K z=xi(gx7cC}8}QJ#<ha^p+s1cJF(SCaO4m2!e@m{GpED|aK;B8+hT^5I{*BED&aY^=g4pWbcyup)6=Y5 z++hzsd9N)V!~N5;)n@6G%k^=F8{l^#Qt>6A7t=egUrpxRKSGCpSHznz>IN&3-PJR2nI3^{x<)#)f5wxA^zRx&C)?~Dsok^IN#zHk3s%#F=F?BlA$iGSv)iB=Yh4|#wa zn8lkI=`EVIP(``MloiHt^A^q3W7brshjg)Li!xKjc)E}$bLV~`yARsU__vu((Uoc5 zBab^metqHZit6@?l)q% zbk$Mw+O_M97q9!<4d#mf+x1uKRNdw_U|{r)@v%Z(>HF+A%-Vyo|T2W(NV$}^CIM4(+BV`zm(WJg$SeZ7MCB9~@Db{l? zJj>>j-+XPV-1@KEl#qASd6b`;g}QdlI-R3gsH65}W`SOE*)NRHW_&oqT-Ox7)(A&k zOKQ==OjyX&iy?sP9im#NZeLfVxV(W6MTAxK000orNklR$(y#pxi^@`Vi>i&M|?HAxy>V z^E>R=#}<7t_l*@zzfdo5?5Lvx=Clx~2oQ`BgdP2dm7hMuI#2P(RxoEyZ%g_-(cAin z0r-xU-+Z&45b%*>BE3L|FQ6&be&7kaf81TF2;Y(i?tVy()){i?pMRriHW?bIu@ z)TVA2H8a}oq4TiI)P>tn`4!$-yMC?FHnfZ)htE)+z_oru!23L-#Ho+n4ty^g9*5=O z4czBE6W|%-!RlhhYi`NcWE(vkle^1DC%S_c~K?LEAL z3;?di%8$q!%7YJqQxTxDXSqCb|D)!<-hJS1X3QV)qr-GyvPv+Lul(nqYY1Wy$*OOFaikhEpzL%xl7lrZ2p_~7#bM=%yBc- z*11bI!#j`z$O`u+(06q$WRik)qMgoc5FSiNfEo0Y##sYtVFngT;ileu_+E1GK_ld( z(@vBh|L8QCKJ{%=j99fts3q8@MQd5G^gEfP^E)h5>(sfEDKOkoZ8vKxbH1J{%;%nQ z?kICH+g1YRi1nXpN@G|Ui$!Ozz5JSuG+VW3B?}fUu#xhRVS`Pv(T>2n2$U;~27@zl zKJbwJjZk?9{4g3nCt6cr2p_Bj4Z9EOuM1cP*fTgO)e#5t#pDGRV0-ehr;Krbf93C_ zZL_xW;;WO58Oc9o(ko_hk2?J{Svz{I+;_(V>gu~ucF;LD>N-ZHPgQ z@{fQ0&0N{c>p4s%TF+xT|MiN_dtLPF3#3Ea4zg&Ou2k3gybdZ%7!7&9QD<@pn2~sX z*1qHf(NlzhI0m2(1UZT^K`WUBIy43SF#+BUA@b^D-j+pGXPh<4+}8iS={C9Nw)<@1 z6LZRg_ZldB>^{J}T(7+nY%5}PIqi+?EM+p^BN@Ml)`dNtOI zi-I*J{<&7Bs~*1hQ919I=a^xkTx?!C7LnnF1Fj7*zQ&lQEv8hprG0nXMf&W#y&2i0 zgO3?yU-gHp<%a)`QG-&x`RPnK_=tn#*kg~Dw@#QQd+svKya?2P-mlITJdiAyJ^s*> zq}Tf|>AIDka>$Vf8{E{{Sa-sFK4-36dhz8lS{IV-q3(5L)aN?2xR9MODo>G5nL}Vu zxp%~6OXl>28B?bfwRZ_b6AvNY4Bf_~xgk9rj-`3(F+Y{jKOZGnkzeR&l6}zKl{r}e z!eX)Ot{Nlz95Bon;x`vwDsO)Lswwtgoc$|v*$h%9e)@@twn7s(6;?LPb;C6`N$Wqh zF~*t2s!$aQ!_^hMd#^!esSn(9pl+pjP9D1V5u4`%o-gJwnpNei{|GBrtg?AA?p-kh zds2IeQ-=8Ri?5Br16P915i2wP&>+BzS@!B}U^GuT|Gxef6BH~6yX$}V%FC0cP=!tb z{7YWcZ6zoE_;{NyKkLWm$ZZeaDjjt7`WvsjDZ}>P+eY&XbR{SY@tT%2l~Z-zyYDu; z%RpT$hoM6|r~?@BZfvdlEPtlu;PzvFjKw*R+&@L~?Eqv|Si8q%Xb z)q(+NoZ!4unL?q_z+j&xCms0{TdRUch>;=-tyq7u&jEYeqRvbI@>?5$9&^y~a>M_w zQ)M(l=VXtPpPzn_T&{-hB6aot?9_ARu19Y(K4a9(0?ZwE-a%MJj$FZW#$wb9{_i|Z z`*WQ>`?W07vkgu={&czV_UonH=Iv|^i5nWU1$n@}XJDm`#@Bj}l4hGCYd)sFG0o=G zAANAF@g=f``Q%_FxsIHbhm`#-5V!dj9ps$T&QtF|FXh2vQ|QdC<9f%%&p8S3I=eVf zz?c8^Qd@jF_04IvaE z`RG&S+J9ed2Gy_5`;DBhiXMZ90ZLXLSDB&-th}HVb?A&M18jlBd(+>uZBFN1e6G%q>FR9#x5btnbv5^VRrp=4oQ}#ESms!s`>Wd*YjTEe zdDu+r|HD<6nLC};o6H%pa=Nw7oij4Pt&SUO%dI+DQ(YMG;O(NVYn0GiDKm7}MQ^ys zVP*nMul+LB$6=0;$ zMMuu9%`e9i@f#vRO9yxe|^5q*J7Y@02AjYFy{md_PxK(p1;<&?>c*f<8<(B3jNM%X$Ep8D>(i^&1p{!DxI>)+2t(Co`* zFu`It*48lcW?Jv3XPsv9iUo2geB=TH0Je29HGr&R4GdF%+93unqjQV`@GxE5%p$!L zPC3qGC~+bZLWgS+~<=SH92MfkX$1g20!`wR8O&tcCGB^vv z6lFpC)Iw);0W-ol+*3c#l*x#ga+y1GxSYXg18Mx(VNAs__-Uj#bv$~!r}rx-$~D$g z$y$}PHQXS0(zeMnMArxdF$_S5@MAMQRO9<~%Vg==MY`=Oy$m1=G#H$b4Xv-T9Temz zFw%pznRjKe6M=(0kO{V?L}dbWY5RHg$~>^S-<5yJ2fx9~&Xd%|J)>dr`8fa#E>LX9 zA?Huh5l9EW!CUD9an2i>lpOkFYAjeElQxgGhkVE+^73;T#KAADsR_$G@DP_*YIx9g z0{MWGc3FG1YNqOdA%Mk_#c-TxGx?Aw)DOORP7Rb{}KoUYUvYfi>|)DK(^n?EBQxJOXXUg8}O;8Jtg2;b+Di1%P+=!2i+1Pr0* zI1U`gFq`8sT#lE=HBi(|RZ+K#6m=Le(S3y&fW8undZ{Yv#aR_K1v(!X?L+IT6#boF z&8XG1@CdPwT?R-*FA+_jK$N=RSphgEniPjCiELC`1_8 zgcxudPDkrmf5Uo5jnuccDL=p`e9@-UPQ8 z*Cve0#iWPL+Ya;&8+G!}lqywFWUPIuaPmGZJ5sClHhVq9ds(Sz>*I4hQ`9jgiS=~w zRB^W(!DL9BjhHg!3urk*feKFMe93vT)t%a^%BD?CmkLWbK71dpFZF%ub2zQvS0;_8 z4X(+!NREr1A`E1~K%8{qH(3$~QQ7FTI^t5xNPTty%e~hon9}tGDc5r8Q{po{hVzB* zNuP=}i%)r(n#c0Rlz3(8%#^0SzRx^4QztPJSe9;I`n66rHEL9Zfd~WXF%W0`_)YqB z(c1_Ed4hp@qo~~#lP7qOz}RREL_&6>QCRN_NErs!dwfI)$X5*18%3S210ps}hn2#3 z7kBqX`noVWEH9yqh$k<*B`2WjM<3Q3MQw8?+NX@H5%R^lVz7R6aZXfUFp2qk7J-E{ zQHU^56%4QsHwuM_0bOjT|JYW?zjTY`3b9q$V6JD2$@GkEU4h9y#l}TVW!3tX+Q-3) zNt8AtY`^jZF%bN{J!JV49-(_VK?#X6f;2(d;c}Djlj}}YnMf5bKPb~-PvrG{L41%m z3=i?)XTML*=TFILJ#PY{M7r=@F!BvB8BQC(h%iuD3}CSP_9~9ZY}7!{;{Jj24RqU~781~zUlV*v`RVw>-$CatFG%b38!j(+ zpPVlBT{vH&oa8)SSK>-ahdHkU{UQkLLmpa(uUultkFF*%eYt{!J;|XLpFwh zsm{xz#<;TPlrju=KQ$|lK#V}igaI9jbEGOV0MAH$kOB2lQPcl0DjjUc-|0y}!Td(x z*vUj6<=?yWrY>4@x!@v~wB@l_(Hk$FKI3#FI|)fM)dvQx2scrD)iD4)5s(2W>inlC z=Y*=aImnq>PJoBNSxt)6_>Ha4^H;Xqbm{yZG9Wn0*=4wwUYD=+ZjRbJ6Q-o2nXht& z^|P!L vP~R{RDeC&x<57Df3`7`+Fc4v&pfK?NCJ0TYT$*~+00000NkvXXu0mjfs}f<@ diff --git a/docs/apache-airflow/img/subdag_zoom.png b/docs/apache-airflow/img/subdag_zoom.png deleted file mode 100644 index e6d4728602181f5209e1c1e536d71e275fcfb8ef..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 145017 zcmYg&1z3|``~MIGMM4zmP>@CthBORCX+%Ikq*P)kC5(;%q9D>D9aHHVN{&lafdgR~HlHzyGR{vbtg~CEn3eSbUl9E~#>-2)rHZs;7mc=p z^45Uq{=U`|5hhWnN#yCYcl4IuNDFQ4GEWF(H~Xi~v7D-Yn(v#z?G%f0@A3XTvj04A z8zTD4cW~n2wF5S7*$)Q?PH_4M)ihhiaPt5BzJz6=U+oH`XQ#+GP0I)yNMDBj=LaRY zjo2WUI#$)sN;KM{FAhCI5dOsXQe{I$66N77rhh`2UOnTEb>C ztd3Z2)3twZpZ?!L;DM3aXLBZtj+lM$e-}Ec)&O&z?M|1CyyQn5+<6-3=Qf$%(W5B{ zAysNX0e7i^K+F53mlSaQQkj(J^*iq+Z1BvWEoR`S%PgR1M1ptV%*9xZJLDu^tQ>1p zJd+Gk`b+2Ew)7 zIZ@}SDaz1*dUtwpE`Ba>u1u^H)}i`_xV6fS@m2CZhW%7Ba>BTwF`(P`NkO37Pe~g? z&HdZweT{{}dgO-|zE<5A(dmf=?)SWKU&8)#cWZqO%qgD))R#^I0zLaqa+^l1*q(71 zMKmlgJ<0uJiZ>37JTj$UUH{9R|E@dWE@-(BI(eIv@vIjl-oumeNsdKZhj=GuA0X4jjl1$w3#mY7<) z_m|G%@5}(wlAD+eYSFO;_kEZ$+gZ=4YC0n9l3BGvyWS!Qkhhq1DiG+qdN(HaT7YAE zJmc&^ZGg#mSBWXy@mXN~YOCn4_U#cw-^SEESNF z4~L!Eb_5HsLT;x4$^$ZzIjfO|*B*c7T8*bT-gI%1=TOz?0a+*jBK1eodSRW!14~zZWUO$Djhm~Vd8AMY+Wx9E=NhGPc zfHDVsdFZ`+rAUeNnU39&O^lCSRr#q49;Lg58XLxVx5?me2SS`a=%Xooqo&0^CakL* zJC+p^nf3bd)IBsOqb_Q2E%iA6S(ZY3dCtsJIJs@Q%a&JeDVk45SLB|K5x`}owZ{rN zAvDTQZ1KzituC*sIe%1Egtf;aQby7(F&Fa_)KOJf<}K9W_sOUXh1una^0#Wv`&H5( zi0ejmE=R|VrtcrJlydpB>uKzR1CCPD9FgntqjcD&n*AYZ0QLK7S?PRFq68l0h=BsE ziweyl564V&bp-Y|@CKN7&n88L1<*wa@o|q6e*{7PzPj&Y7|nl7z4TR_0S(&8_X%T>e&i! zglRBh?}p<~3zzZ>3JMHmz4omb;@A{@U%Y!^VUmIilJ_|(9BO#;dbN?pSt7lXhO{7Ol@Mp6$x~bL$k3bDgO4 z5FNDZJw3*GxZhl%rlwxA2^bX+YgOf{F-ZuOw9kz#cr9Y|W#|GyeLcp zzq7OZiuhEcSVLO2h!q~kxrONWc|maW*aK z5TPxh4F#x!3C61JV%H_|x+ok&JRbXPd6m)4y!Iq^ycBZ2S&Gq;704_#*p;L;*8BOm zGa!Q5jAiR{TuuhPWoBh4!l>w_x`I<3Ud4*2{pI664cm)qDq_mqw=g3K)v@iG<8-*+ z?7Z2GjX}=SG;!;Kf_ybvyHTVWE#%-;bzP8IAa(1wM0kC+4#Ituy)vg(Jd<>DIVUFP zXHKrpm%U8;g~cG{gx1dz>JCU+$cd7OO=Go(Pkzb%U}`B=I5oYc9^IVMl8CQ4kloNI zK07_0^&O0uI9cTj^ByU-C>9EQjp)tGeKv~OzQN#n=zYYDe9`Dwa~PlFb9A7&{x46y zY}AlxP-OZ)r-ZG70##a7zMF}R3W+xS`N$XSqbIAJWAN=5#;YHj`X zYhP}=mMDeb{2wFaink` zUrme2`GZm5b)Y};{n_0hwux_Hz8TQIRhCn+(|a%kaE*f#rg$!`!7TCq)i-g6u5ZE% zHkz_t0p%-qc$jaz1VLi?eJ7aCr}pONZy3p=qo_TwJKx^Rt`=;aj8CM>ls;e&x7$vY z$?m{mAdKWMhnUAQYU85VWb{7XiTrWyV4jL49Q`}FETb$Muh+Ji##Gc`B*R2TokyaJG^!$152Grx~wd&>*;UQ}udI z_~v3Swrj-N(CW>!)8ySgh}p^b9<2Qh=E8bK$NxNy0R(fYaa{69vq^0+czJVm_+lsK z3glvU_kv+}>i*`iiO~p2)T9%F!|5tQGZDptru^eT3upM{u!|!-o*jh~g`Z9?R8!wt zu}D+9?QBMt%+a`R_0;UX>C_&qkI*nF`fS)TN|8&3bkjn@_vF{d(=4Z^-0a*5>-{D5 ztD-c4`joVg%Ca(P_<8vrSVeeSs5SkbXFPMreYeQi6|k6JipD);#p8szY}OhUZnh`c zheJa{t1MS&hrjCU{y-w8d4yNJkCJjwZ?105_-wCeikeL=|0~f1N&fE4dl59wtM@}*$vOHA4rI%zisJ4Ut5Y~kuE^vrv}6xv`AWa zS;6FyEAg7IS8IH+Uf*A@9oLJ@g^UhmBeoxbX2-}cum+;KIZBe}5ANJ?CaM0oN2ZoIFO8|&)i>iN3M z+kJB^M*MRxCdg z8q&H&^y;^(+OPe*kG)r$DD*1M zeKJ&@TsHg1%%e<^gAa`r*?;!OE3;lnlV_+@op(o3iXL$5ge&;Cm(u)Bx)QMneN?qZ z^~cHb^6w}`VjtOp%@_U*o+N=A0GqP4&d6vjD|%m<(X1FXV|rr2y2aF8mTD25jp7*H zPdz&+`aX+?qq}q#4e;qHROur4VzR<XEM9JVM#R2II?@9p*QXmk9}avL*Y zN0?BLDXOHYIV9}NdfqjfGcH_lZQD-2qpxWLVM0rA;p0oE; zRD98OTf?A8%fqU$dvVdZz0Wn_<&WHTg{}*w7ztlpZKW zv}s#ej22cNvb2Btzwdqf(V%W+`WORRxkE2G+lu`P{_*#$=t3he~f#Q8RQ-T^z^32d*Shc65>(y%i1kc7W-z%r7Oa$`qjk`dvC{a*jjx?QaU?Y>(Z4)P416DcS80`gS%wfF)X)~dzBwd zE;2eg_?_gcM*bPHe+1JHdg?(;z42+^oT~UO$;(T$c8}BONV%&tx)LNft=8-h4^0jY z`=tAlT^-0k2q`BK%fwp8Z$M~dil9dp@V21SEH3$yg%TMoS8+Yi4%OMCRD=5Ed( z{P;%|4dv5CmU0C~f$7D}(WK_K^6DK}N-t{&GHOs+V5UmCW zOM%a^qeNYobr==feI>@GDj2Y#W5Jl3n4Gq;7I#w%)Xb%|+AWV2^ZpG%fk&8+Pq*C3tAO(Ga>4%n`9Ur5aTM>klaYW&H#kkx**MM>KxA~1j8EmyRlv|9Pci9A* zpI=+SKt}R2ih!_kuJ4Yrn$Fw%Zy!l<3V8P-AjNhGB5w+_3op7UYWKK$drvL=4h<^W zicBhV4*Z$%(GB~9Iy|3P1Ni&)K9ue6FUL*Y5s=z@s9a_zIFxeIbys8bve&~^dGFEw zJ^CDP*PZNQl`i@0Lr%{*k~+7}udT#3>jJU*J7fK0wt|I7BqjaYcm@S0zuL-|FD^F* znYD?nj23Lp|88o`sM)nMq#G+FCVtw^_c1)OlUltW=uWY^5Z}F58u8WPr=;Udt-8~4 zpfVROTDJe=t3RB|l441`<_6>tyT{aDuY5#W-N9yXmY6e|i?xLkJJy}{^UgA) zzth9whT02>BSsH4!nKs-iO%$k;q2@kQ7bm2nY5j~nB~Uxp5*rNdFcKU2c&@XX3o|t z;~#S)u;d19t`1|$392J2Wfx*kP5W)7*oQuM7W>sG=j9|0pHE*p-M1043_X&j&<}Es zZ{7VRoLG)%eQ(`T#7reRI(X$srQ+A`<`9t&v#(R0yoWeI;0$)OaPtb+ymQey9T`w? ziTB|CayL6<5bRhh96ur=#9C|;T`$TwH3zk3fRK8VZpw(eJO41~Q|*gkx|87l?v_XK zFX$_E#b|wNJacMH_?#OdY)GvODCQ8tySSw@F73iO+d>+f*&JM9K@T)(5gB-&lYA-s=O%PwkaGds4i^-oxEz?-mxh_SVt)wiD4)3%u%gP-Q z@#FTF+CDHhEBYs+9{J1Bz{&YSG%xR#y1TYTjqlncL+iyeHG_9G#v&1F2#p2F%6{pG9{EI`C z8Hst4&39nnr?BVZhHJuQe#6U0hoM|rDV4;LA7$%yGk9!&&Mbh7-kh9;R`R z^-Onvylnn;VLAMFJ9p87&0xU0g^yP$Oj`X0vS3(|wBjf#+RZG@1aykj&Nvhi^Ef={ zt>Hv{)?R5o<|a#03eJ>Rbuc_v68rgP{!`+~Vw5Xgu$Q+GwZz+jKevPocYp7idCWZg zA~`~y-SM?GOq(E7T-zglbdsIo8a}rrJwnSWsm15w^w=loQJT52F>_Yz!yK52%k#~J zd0jdEpmmNwZ0BYhosR8DAGDdNH-f7OJ6qCKvO-7UiB{np{FW*Sl9_ax1a;T#4^0@GThNy}!qoQg_6<3)i%r9iFC8Izj zqBmKD_d`VJ6nd}Y{*4c}j{LB3#vPk7%EX?m)6Adp0SIH86k~Gcf+&%DaoQf!a0)Pu z_z*UvExG;!!oq)R7+2(7ufy4d`KUD)g8BFeF;n-zCs>M>O*&x3hWCR3f~WQ(I#1<2jTaVq4i z3qEmA8G)A;4)4aEp?+v{Pu3F|wFMF`Sw2_c`kb&K99lS56jnP~j1Vbt?`0W(Zef8r zCi50*x}k>M9td@+-wUQ^miLUF`6Fs1xi=p+^SI&DW5J=&?)ZCQ;SCE%Zc$iLtwwkL@jf7K>{w9bA3{rde=Z!a&DNb^2X@8hJF z6AJ`xN;E9YGZw<5ggE>+e|O{_bya@dY|YRuCvUPjMK z27O#aYeXUrdFux5kKdgZKJZx(=dq@zqgn11f`lU?oJ8>R%w#4uyZWz+S{lP=fYb}q zA^{xF4Thb4y&|&k!7d}ia#fgH%1p<4IcFu|$u*T=3f7=TVC$Bwz7LH+tby0Gbv


?Ypf*u>+e?ieK%)(4N~o+{7MkTf)Ix8;_e^U1LK(cEOrb z6K{u)3Oa0z?T*tiqQo8&i}DH>kvQ;y)1`EvvKi zle&GD<^#g{k3`MWNW>?R?g1J-R;R*~$*`g&UEMCTt9zyIRd4G4ffJPG2029wva?m3 zE`>9{x%ad(X#r}wa7i((C4@RI^3m(hF$0n8a*8K&eyHJyd1aLfj~Te8UUM)XKmLxd zY|OmH$^D5aL}>anoETn3+KalGW9Z8s8u>0@? zdJhK?dHa^ur{3{w!}%!OOuyfF5xP0ymQgJ4lMM@&E52=T-=Q&aD$3=?=bE57=((hG zioPI+l=L>FmowT^KQ6REyJJKeLDRhlZULoH_Xwm&^is0!xb3$u-nHDPetEZVbaR|j zmE2b1#JK)tAC;VA;oXdb!IE?vx4PKzNd)~U^ai!J4+q98I0Gw`hR``l19sT)7*OW|H$tpFdttZ z|G%a}m?&KMoVVQdnbwEf>-5IieV@mlRPdGond(041S7*_D$j=dKJmLliGw2)sp7Pn zQEP;z9ssp`ja7u1nL!q8LG}hlKf1Iq5&;`=k3|gWI+Q}L-3X*@e-a4az8%(iVP$%SD`S})pdjvUR!{6ef@fq*weXQR4ACgqpbFB|F{5N%5W7v^E*TA z942{!^13)j#Mv^|l_MSNY%t-(`QGMqFn#6~N$HQ~xw)f3?w8$g}*OUAJY)8~O3+~I2dzqc)hPK^$ zQsA3teKO9aDKxxcy+hcfMnim+bhKCFTmm#(7Kos|DNlJ<{#lC&p6= zO)buXT9j)pk@K-PFe{DMsJQ*-9i@$+XBg; zZ%vhJDya>H{GVd3ItQN~Y-izg5R&(FSf5aguPkJ4N=C4QRS?39WqXTlpMJR2*4W%e z8oRS_+9^^uoh+{QXWSjYgEcchH-;EdLQE*UBcGkaSCVpK8!Qe}%58T`553!sK5Z|Al0$<>E&QpE!aIZNe*F)? zNboGYn@V|#T9&J?<2o=XZ{KSQ+jfkI(T@eFO=y8=_n-b$3Fn`o<4JNv`>0cLrgMI2=M#`Ekq(E)Q zq`$aFWaG;FB?mjiQGk*K^HGmf5U-%yi{wLhpY=|_4zC$fDt+*vbCG$O8R(Bsp%2tr z`w7!jb8z0IG3Ae#F(Zdq)C$WI3Z;a1oW$6m4|pE@kY6Bo9dfd5qtZ2A7bC0w{k#7_ zP%V5B+QuZi*Ac@lOCEM~RpcIfTuwx@wG?6~sp$rSeB)%2sQm`mof7m?kW2yBKQN6i zv?8M4yf49zKTVT;GTO$E{`2L>;g)w#0}C8!Tr)y1Arv+lJr_D#%;Zl`3s-bqjV7ur zwZ@P^Xsd}bcg@kqGcYMyhbD{`h5Y%64}s7V?W-=UkS9y7Ge85dEM4IK;#i|(?te8ax>~=NllhRyT_!h&Q7qa+Zj!_ zDAQPaIb`h{Z>!jpkJ{_CmRy&XbdT*n%Dk5I64u#GNhQ~+6vMb(MoyBdf}M-+C&jDQ zq$KxH-Jx0D^Zw!6{Dz6Ha!YhWwaj-fCSv2IVl@O-Qspm4F)R@2x+a@tZJR1ESN z8N1~%v-$bZpdGWYAWSBk_4~nlQ0nI&Z=7Q|R1PB%EjI5({a0Ooe!hDrufVVoN<6qOa#j53^~f=i&m)p4h$Cd%Xu` zG3#+xjPE3gBF#d6w9*`UG5)AoU!mZ9De`U@!kqk zZ(-gqsvv}QdN0#nBk3XTdRP!G+eh+m-_MD!C7pM_y`zPzc*^IbnsLf|J}WlIv_aF| z0vwGNe*^51i-d((ILpAzf5~)zLR?G@lz`ypi^-F(dbaNLDMyxB`5Q8?~2O7=9iz8&^T*fMuHv;3P0+`$Ps6%IOO@-fU*M6@~k-DG47Nu`64I zp0v&zi^&R>bMMlRBzOj?>wFYU>>ZZ=J>2?xQTgP1%0XwV_-uqs7WMbYdm#qMG)c2x z-Yg(^yR1+`aI0#HEfx0T1H0p55C(on-fAzV3Qh=)IP_Y$rH!m!WMg91MF5CKC^k>< zg1J^}QRDMikWU*juT}t*-g^F>aQWwZY^p-aHcKC)_46+@rq91X$}YqNo+r96z9z6D z?`Gsn-&Nwxl@@;?p7j;+yvn6wz4R&IVybkL?~gvy(dH+0a%Sgcv3zzXrGC7$-P4N2 zQU+VUQ|qt`Zn-L-*+P-JBNAsi|3f+C?pn-s;U|oLoMoaB#gj~^q*ubPFZG6e{E8>y zVG~Ug&$gx(us@W^L-xS+(>8V|n#$bm7~i%#ms?Jf+#gQK5Sd6f@hHVQFAdwHo^K>K zDLiJ~a(+R>FFV<)^0Is;6S27VZ=UY&rNHxJ0$92E8=36&50ix;Ki$r~(r$yXlkvxp z2ot|E!~G>P$hBSc9k|MkWw}l3yVvW|PMDoXNGSMFLv9cF6Z}C`B%(OD zI#MBp9e!R)2q6VFz6>6pMEFX0fc;2r>E`(I`hCg@n^e`EP)>mP3Dff#Pm zn!bW7mOw^httgz6S0_31YVK_Zw!--|;241(aKd-`hGM^_Xf(!Nx{gSn?@ytT#~`Fc ztxf3zBw{KT@l9~vhJzsqJmqT$?*2{aeht$OR&hFh8Zejm<2cWFK*gcAtnpsindG^=VBHj~(YSBE!a%w7bC zaLC`J_8jgWeeyr1R**vsZl#ucwe0bH{FCpoa!PV=;5bjIhl_l-w8yLdq#5^BdxX=< zPpX1^7j`1knE8ec;xt~uQ+dL?mq-1(c<>Vg1_@(fVM**B#`3EKY<~;W zszF;2cp7$btp(x&+b2nq-Dt;?H%cG~pA}|;nxF$onRq+-rJ>S&2<@a_g$2?YYT)xi z;k|A`-!sg|7E);jjxAxSYur$}H!1>m84{Ts`)d}+?raO&YDZC$PpurjW>N`RyD0Tw ztaWo(V-M-xGD0DKE{JF3-im7#1OqRaoP2_lw2eE{_Mdrja&hO)&2zVe>hkg^KW{6} zl4BgdmW|L3A+6ld9IReDzA$y63wL{&o|dL=QR}koa&ZkJe%;2%bw+QzL2+O;g%L~& zo#{$?Vsb1C6>&HZS&KSfrKP=C@*E#QW8`)^PpxKXaWhqAW#8NadoGMGul8c_MZRh) zXM#?J!aPfmabv^WNnCG=6i&y0{cojHaMSZxn?Q8q3rbg+4?d05eOs5NBZyRyI%;~o zB}+u2sZ0^hwv!KOPwe=x<$p2i4_|8GM7v{s*I8Sj`NBQcGE&{$<5h8(>N5man2DS! z4+Lvm)TRSJAIq2yD#z}eR_YiU9I~@Q%Ar2zg`b?%bQO0KjAoAAC=UWjIIbm4A2poH zH4IkoEfdP^ZgHbAhtDm7iB!b?cbK9gjIne_(5;tSGEt)HKK1Z3?_qVIvdrpk&nsZP zDy`K@lOP3W(3EeP&+~m;FJH|*jx4pN0ycMKN+d2rHhVD^_f7V3Ro5gy%U{<<5>j*Z z#Y&E*_#lv|x_#JNjInq}GjN z)e;g}<>dg@?-ZRFU6RIE>2`eUWUuqt0!TD2t_R)N5)lmc)D*Oc{Bp%rOqK@ zCbqddm4cu}9ikjDGM4xFS}ag5SwPM3_gL{8Hunp!x#DiDQhjCS>n2~NN^F?`vEKqw z*(@KS4=S<4HFCp`J1`B0ZKW+`Si?ZaW$YeM*a<&^`e!qz`|mOO-xWQN6t(%XW>^=vkKQjNtfn;=Z^D0@G&_`|wGnJpQiD@B%-q89bOdkd{V0d&CA z>i6@H$Xmigda3&a`nIV1f@VPqLqsZ2n2CNJi7dwafxtda>SBEGM;UhJQXN#&@H@%- zeRsPJ*aQr=0h>;jl9Cbv)+sB%{hGX`S}B3LZ~4mh3Uyy6=8$~#D}WYB^`(G4`+$l= zV>ogWH+Pv(I(y7RDFI^d#K?)gfIChTOe3d}m5QV7j#%eyR#w(qz;J10_4VmZzm9>S z$XCy?P9a39Mzx~Zv;pcq)?@elt6vczW9cTB3E&w*zSZJ9!|M2H_pmM~fH>u|2a6|7 z=5soB^mF8>Njs{os4xWT^ZL}8uY#FuVHAMC{On7H`FE06^n6ax zr0+}E856`t5e7Ij_=os!F~95s&~#JQq^<~DCD@+E^W0qyBGdcC(!_#mUe@PXeIy=rC%sX;^QCW!74 zY*toQek=fShgwxh$-orI4AEfb6yS(xP{<^jJ|`s&h%3sgB9q0aJaFE}q3q-`>@cmL z+KqrJY3cyQE!0KlWG`_F7!tYxqtUZ-@H00xEtLIRsr>2FC#v9H*6(1+cMc8(k{&$%# zO%PZtw%{qCS-1ZRl?y3g7pTISS;6w#9_K)PGLE>Vqk6G{ScZav*YW6Xl!;ud9Y)TJ zJU@>ILR+c8$mzcq%9(cNcDdRPs1YA&#Te78!M7E>P|h&7wWV%Vqf6SA^

_831m(7Wgo0u7|7DKfxw}fM%ei}Sdw4YST@$=ai;WNj zFac*cSO*3aJ`|)=XUcZu(1)kMVGQ*(;patC>Dmna~~429%quTqbT=73!~GU46daby{ChsbtjnF@RXheu~Ai1&?%L zWbWU?%XoNLU;bfAOjRCU`dd1WPL~1X zFySuHi*HI|mBY6Z4bPR%$Iym93oHY2hM0OxmkUt1`VNkM1;B^SO$-#Xv;;$-x%SIgs&pO*ls*nY^0y75`YQq^H##z6 zn1%qHQEC{%2Z!s7<@>KELHf@CPepHP{B81p0$m{B835Ns^h-P*aPFvt0yIX-aVi&w zBM=B(T^ebK(yqQe02rWC0Ri!cJ32Z#JO2h~2dLTj3j-R%EK`&IpQ)FyKrNFb4mLtx zQWAZ586Xk#ofiUq)idV#>%U6(7b@Lej83}zS}$g&{?xJauRopuLlDx=KmYak>56`Y zDHP0S1sfjQOH6%Tx`Yi!{6}a1dAtjd2o6Iyn()Yw`5)=Rd%3&2zkdC?ftX4HYb2U7 z0;TBx!21~h2hV(Xx`Z{UbbNrrt*qFL(8VVtlx9cAefkYpIS?qjw6q?FmILPbi|BvU zEd?)QY2yS(DKYvuhu*K^BnqXSyX<}Jg~OJ%sE;M%18 zJc^4WrwhUZi!+161mEDi$WM^7K?r`?;yi0V%QrEK0_b${Us7Q1B~)Vf+cg3?OyG(@ zT{yr@wlmpq)=)fsr(<7Jcs|5*s=z0y9+kYNeSfKOTkG$#aS z;#7C2J$pj!^{_1!K>`wQvgX8@UnXc=9LdXp!NC$K*sWK1JWvn+zuE_|xzJ7mha(bw z0l6KyKcK(Dv11L0Z1$aN%u&aBG|8-tcxF5*&A!+irRU#?{kg8znbX^FmePPvo5dG# zp2ct?E&i%z;%BY*J`AfwQM-mCc_}%Q>P=}JOr;etqP)pS>|eDZ0palrtzkhc*nT05 zl7j|iG%=j$o4Pb;StA8_r^b)h4oo=L_EuP(4xIirZQX{Df^MQQ$2+qPK!ubOd}eH{ zL$-NfVBn&}gF)V9$rf}L*51?8L)G2$+GfS-c8@XDJ>^}dR@r%lytt7{gu-_6+7Im+ z{pl@aH&WcduG-xH$mb~}b{=SB!Nmy~*CZBj@+41411$5K)5u{cw7cI_r_?kNrn9yl3fnSM2`FMc)jx^4CNUJ8R_}@`ljYoofhG@IssodQuOIT z2rcJA3|9Z=1mF9mj(Pq!K=DxAc}?@To8SHHWPwWW!71;>HfpV$z_T8+E+PGSg&L2+ zw}3J)lYYrDI1}dL;_?UxME`m*B+N2ih`=}J0$Xaj7$m{?@#Dusm;_M^xYHWc)Y2(u zT@bJ{Zz8teJv0yN8wjNwpgkC~Is{KAB7?Np^&nt90kR~ATCdsX4#F*Fb&7&gZg0t~ zStH~|y^vSuY4#R|_*&Ifa|O@ZOdY7iU{Unv4Z`O{90vS^)hr~45A!$!K`z!r%<&#h zdAj8Kb}g&X>X0vL#xhENf1b9s;hn5MA(r!I>%a-wi@JBq?U1AXk~xoh+*W}RQsUO> z#fvx9Q)7uiU4<`x%R5fWm-zcncDS&YG^H!lP;8&|U@{&aY-O%Ke$lQaBn)@<5TeFiE8^-zptnQLZb*d$r}IjS!D>!)t~>%_d$nk1fzx4wwK~MwB98j@#HUi0q%=m=t7_q zg!>PT{*;H+FfNUl9e{1bZfS|t3}zvI)Uc9|eA|BUi}Ojp8DvmqTByy_`fMs`h_CUv zlkEwaUm`~qz1ZyMbSyrZ;jBkFi7SA$J?|#pn{j{Q*$&f)$Yl@v+4SvG!bY)Mtw81X zjBH&9C&WGcPb39Gp88Dw7rxR22gdPMV11t!BDCcpsX&V8!!sypWhkt!9 zDPElj8%jiXsHcOt3Xe4Iy(Du3$aKq~Z9ug%zog#$%o;^7A*nj)@%U%?uCK71GR- zOwd74^IGb_=nY)&A?n((hR{peA&hBsQYWiWMT}-nu7j|7BzN>{F54ySsQs4GttRnSr*MOAm*w_`_hCjvjrVWZu_-?MUbTiJijJEhP}+%*FGGZP*iN}B%fxz>O}jGUc+}5i^uyz z=ZlRGaT2!y#MYs7bac89)%%~UtsJ;zIg~ zDpOvK=bEzYvzFkd%Tw>FT`V1Nfh~_m%cWXuBqa}E-x91_#I|)ED!K>wT;DzbCFgLT zy}L6)`5xPu3@e{M#OdoO8YfSTaeZ{{X?!=CNt(eh$mzWsSZ2Hr_-M;uitUDC|8>AR z?F~sa>>avVu!P-8@2E(Zc9u^E3TdS*$1C7A`dH6(JBu^QQdx)Ew8x^lqts_TGsFEl zVE)+?!swgeYgo+ee+-o?W{{J#=&$9XJ1hlsH5b7>B$+aoJ*dDr^n7R_+KS`)H6J{~ zfx0q8)h}gehiVNfi1}PuSz^PpoBOu3iTb_3rcJ}|o{!svB!SUqqh4va+64Cz%-YCs zjK&5s=ZIHt@~w^A%C2ej*eZrz7UL#zx%y&g?zbw(+ZsJCZ%&|XU}kSunYQ1}`eYvq zIeyEr9mt4XUcy((7n* z#_#kW{`4zAk>)7zZo=O#oyxfXrAgnZPYJ|gVjkG0_bVUqJ2>;Mzq)97!Si)2{lvA} zcXuYk*P(`49AhsIydp=P@QU^;5Rx_yR0Ek`xJ|Mj!tg42Q=h6+Ijmh>UVcB_ zzeiZPMPcFi#;n)x$*lIFNp`;F0}jeDT$H+3iO z8Y(Ez4W9)a(G(;8x*Csi*UmRL3a0!<&BU+`o*&*3yrw70t&Po5ZSpCS4x2jNtlFMH zrz8$w4c#;jhlyg?%@LjTIMFjquNDsvG=zlxpKW!p72V=uODcLD+pA4ZcR{c7BqlVe zO@F}1;ViY&xpS}>F~6aKVi*wL)ebeSsQJaK1j&_6idDWdEnDf(Wm&wznU;d3w-<5#xB=nxk3%3L z+YtzZ<0)|QP3F~to!X55W0)58An_Gd6eGWIcZsJu%KC zgtl{JKu({$Z_Cs5f3)t=kIMu&rSXtp}OItQ$@{6R6UBL;*y6&hqs;0zRSP z1YbMw*VuCr6{g-}-ghf;-YeZpsQuTa0pE+?8p2$CcW!D_GMBm07f%`?!2J1_@F8sS z7nY&(BVg`ZSey3i__V}4vE#NUX;BDzudj|dBuK*LZoM681|&s zrxgwn_6~K`mZwLFCs7R-ej^7sGmkSd*~7MbCxu2=UQgRB`q7)SovCH@0 zh~wYI(7_iiA{MNrEZm99htyBVDW6tH&S~~A$gq+ppZ5@CQkNt1XiVMHSF!X0XqJw* z+M8q!Ye+P#7_Z(X=CN5O6d@yo!O* zZR~T^vz}XDJ#t99siQ)MG0K}~J=4WAA1Ys2kEbuBDSLW4X8|Xr#F5JSB>FYLh>oYY zi0$2Ipk1XhLaIDZ0Q=9*)SU%!Qd1QpLYvpBwxu$z!^~gtQWYV**v7y$&i9#D{$mHJ zUDwhzN&OC1V}4AP{vV@%!X3v9CEgu!y_nZ~KXA{8^g!n=S*&}ONj@RT4HbLmX}TNw z5G&4o@bubw`&W<9=~p5Q;lMbrLm z{%r32?9`hAGtrp#!#JZx%FJ>h{$7fHp7aYw(exd5g-<HDAf4) zII}nyteu^cvs#*(nwpr%#L3ACB!Jo3*_oM{Ie=3iHZ~@v)t#N4jg5`9>Hg{IX&}uo z-rwIpbB;G0#Q5gUJ6Siwo^g`z0Kw<&JNo~RvF{FOs$2F2Q9x-TN|7d_C?cTJr9|l{ zMFgdT)QAvT=p}#%NEeVMCDNt$-g_q?y>~)~5IRYCJAU6i=iKwoz3=58T(ZepGi%nY znctLc(eX+LYJOM(xk(u|Z&eOkZTYD*_M=0N9yyFqOh00_;lAu|%5}g8cQlHiCL0N|SLB{e}K?qeB5+h!zZU=MvqH`xxmM{Ce1si5c5*@^AAqBdqLR9DU+iY*V zE6->$8N1>9Eq7YL9nCnmckF^W8I(Oc<@G#;Dr0*)f^Z0zU8_iMS(Ea!-F)KT2uV`5 zFW0$`#6P~0ye8Ex6)62m=9+A`Y@pn$&V>cGZiT>CuU=nM>{bj^dZm0#MZ^1T;M>R> z8gSZcCK4eAhzO3T5j^)&UT9v98C%Jc&d)U)c_zmdFu5pvxBg4N>&LhI#j;t-fm2K} zsyAg+U$yc38nN}c8_t?qC24y`_xARGTaR1MTF>gr>I&!zKn0+RP(`ZW$JROr_6Ft$ z<{Re&J!c2!$n?Z#^k(#!6W5SO0B<$*9N)w6GWhs2J$se=;Et%Ju3Qq z(yXHmCvn=a0LB>@dX>VH1g5c9FsX{cFaVWGhVaa;0AeC|uAKi;1-=Zv>_Utg( zRm*2i`>Ssc#(wxRrD#hyVo_nMJN+ZVX0bg7d}~E$^inrBTduMGo_J+lQzUIP)xkJ+Gp*KwOGBc2SmMAo5J~(dgspj zonYvAgNuVq(1bc@sk2XX5#)(kV%4j*4|m_KnOhj_Ewt6oJAhi;1vCTcI{2E@7N5-B zf9%M;)Y69!I3ICxb3WnZ;}ql+;S>YYJd{DwT(h_-{%Vtx3^+MPJzY7jT!Jcxw%>^t zqgoXvvqywqUDBe0yeo}4{2tHvN9r*JX;=L1*nbXjPMfe3ia#rU^b>nJ^{sc#N(7J+ zYs-bymPbRPI&@Ro%%g}S(<$m{rC~%PWSrJ+sn+gVOFEt8oxF*_JcEyf9$tz zIh**nOGH9>a%1xA_)uV|-e@|>{UHZ+Z(0W&& zmiQ12IAs_(I*8|FSa@q*4a!8n0&yK;PKY*J2M?uLd19{t(!Mz4|8MLw{(UAU_G1l6 zp$$hql&q@L6C$Tm=NRgXZ9k;}K@xnKwg4Xc^ z>mXcMvLY(6#E}Bst>if)VQ=`G9^!PK)_Cr^H&6IJ)+tbb#oz?-(?noipy=?R{C5;% zsV$S#abW@}<pCTZM(L@mj^4hpJWNtFw=sj--f;`wvjwBLjHTo_|(DX<9JHP0K;)yLQTocMjH``!IaEO#MWV*QNI&=VJ0@^~T1h-rjX6%JJ;6 z*>BVgy19yv2X{$75^E7Rs z3=P!aRbP9tRk@HRV zH2QmO!km7uo@xgYss+bsaqK3g7pu!5=t71=b?3AQ3A}WE7?7~GU)9h7N0mOO2qlSo zQWHpvoyM^MrhJMMD|?*#cQ*{1Rh<`Y2;IOw(R9p}gU#cL?ZM*;Ru?P~7$G3Yg%{M< zLUMSm&+F+fZ}d#{Rh5E&x5 zaOMujnLK1weStE(ML0BH8J?4^o{1Q$LHa_$#O?r#bSkzW0q$?>DmI8845|tue~H%a z!hoRl%Ih^0Md>VPQ|_l4Cu{~pwF$2BbY$u%&PAaSyaX)x*sLZfj1?BW3GQ&vJKQUI z?h(3lvT;xqVYZt@*Pj@63`E^@^ZZnnIEM49HkZ$|bTnmnpWgr^rcbB-3(MS#%`2s_ z%g=k>vMKP8t&Jh!dZ!q4p%)B`5!^(QTmin{>kxhJ59dDJE6HJjMQ0*57LYkZ$n5-_ ziGQwts^Vj{U9?_qHB^XyH|=?7wF&{jhtnIb?nHUVh%@SsSWL+z2uSnBT$*PC76g1{ za|>eQKua7WChn0`=cX_0Br(GZ^eLL9bE+1}LCcJF;bMnMmz40xB}(JXQHPI}(>T?( zQ;OMG;6{1YkLqlLF@$OOiq?;Gg&(Z}x*&gV){}gs)lA;@cr=6|gG~%Oe|Gv3n2X>8 z_OYy_)Dw!Ha?6g{YMn7Cwvnp&PGG8VKqS9H2|w?C%HvlqXJX_@N_rI`KV(NtvNMKJ z)2`T6W2fqOc%-rKNG`#@A6x}XTH}~na|2|AmM1w=Xh4Ja1=1y->gNa=@$&i_5)PRf zMSvOzoPnNkz@)jXz@4T8!Z2NaN$+=dAktI3zK-6)A6cl=1;Bz?p&Z%w$hoogYJAvr z6KOyN`~Wxu^rjPT*c97AzW4B?O_aF^-oZm!gC`EbhJ<-EWFM=1W_y?ql=<~u(6von zdLGJ4_HmAtc3>)x9quVeVQfslNk>2hBs>5oiozH|&)D;9r3DcR2;Z)+;dL3qh*t9- zCR#s`EoW8P-PQThN@^95lBu#!x8Fr({dn5}-5f!RTl2lA0j30{&}|~0)&U3Eb|zU= zV5Vp-bRrLbpoiE-n2!T?3WH?M-{0K6tvWvXQtgVvx<ZzR>5gvAlyE^C`V^~ozqCG4uJ~NCr48K5vN*L_yA+eMNT8liuNc|>JDpKOgEXP5J zFn?sHqA}O&#Jf|`RXVry-?=)h2)!E^ZOXsuq*Q7xQi&W1{)60_t@f8KN~7{`$v|iX9W*swzB^5yeu12D)C4_nyrH9N;^)n9 z!idnTnR?K1W)^S2y%bb*TB_yg?4pP#sqxGO`;y!*8ubs7@`*fYh|QSd6fDyN#B<@$ z3MNMc?A@uuEq;e_!G-(~M@a2lzzvb~jbsOTs*>!B51%;z7Nd7ZN60KzujJqgo;&5? zHmzBJC2|``vO3=NAYv|e5N3Lz5ks_gcZ!l^ARw{*JyV?O2R~f9)7b2hxdoslrcR8k zBpAY~|4Z!|-Mi_N%sqzq!{*(5DNmX!1(}F|#GXUwJ7btmH4Mu0lP zZ~2hZ-(F9$b0K)Q=C6G-uI54T^s7{HVUiIW?51IrC zYR>bUDS@4cOqLBUk08@Kl6JuZcsADOEdOwsPFXobf3D}+c?3+aUH?U5N9K(9`J}+$ z3PJUEl7Z7HLi-z9MYV*OA0bj@LY-=nBeIM77psFYIH5hHzy!OTK%?DMM{yk9{tyT)xUgeR_Lyl=CjT>{Ark zvAChr@w!g>`DPu0y9IG}Wwn3^i}gJ1F_0ZNj)$vH)EaBfd0vQ9+l8k0?*oOly4Chs zX`}`Eb!Q;TD0?3q2Eecn4`(WEjTL9C78mTIZr*CKi;=^8wN32_*l@j&htGL?d$W3C zY=>75k5^t5Csjq@Vmr?0!ad)e%N-p@;mKM6H<)Tgn`)i7oE>CvJjNtI@FQASH14DW z)E_T;?0vp_e(ZUQ^u%ReggsFCaWcwiU3-C>W%I<&l|wf5oMseanlZbh-0{hejlud2 zw7mNK&{Zx6@bBU<5bk8bKZ7>Y&+|CmfLFV8Z-3==#h!yVvncMo3Jjh;I-eWg+&fAT zsjI82c+$);6n|phYLz79R7ek`zpLtw^MRGA82F}lW}IeSfUA4AcIn8-UDj^wJI#`E`Tt;t(#Qjsbet!9{^2fD0aIlv$r^ndV*suTZGyh%Lss z+B)yO8)Wn`3s?0`k4(k5TH|xaSq&TF(OFpTdYt*iDeHxP!u}Vxog_YF9~ItQ z;k@Tf68H;t*kd5Jzt?hpw7q1~6F)X|40IN7zq;o&{!YkY!v|=E5e_nIz=XM+=1d9| zrxxJ3dlhuf-BbLyrx{W7!$!?;0gPuZOAjqf@N5W&pM~i2=NQ#+sjEe=d-vP7CTv_izpN~P ze?R5Ebw0nhaKK6A#g-KiFz-?UxWmp3-WB(WEox1NGNjcR1xrJO$$x>*cnWl@nio}c zZq2X67`(Dyeia(~f{$HHf_}hmqLmZf{H1C=*}7r-xXIxNvAH`nF%gF=HnO>>g`KjX z_ch9lO*{?oN8<)eSqGpb+80jtU98TEeGe@^%n!+ajHlJOTj3_fm-dU-KT#@|UAqya z2R;{aKX>CVdDlapU3P8WeK3uVciTBT)N{h@=TE_GJ9aNN%jd8ibM564*n^=-kp0$p zk&c8!>}YfC<$?58+ukq*eMP5xd8a@j#|UDRW<6k;ladU?t5qz_%%MoWa`B?gIba+CeBk^mWUE8tyhdiFOqkv!5fk<0&d{wZ-|rd- zE_ATE@mKbU35zhhyp@&x*@TgzoE;q{xLkj@$|xd)ve$?K&+1H76jV5M2337rki00f z>qf=KRA)1@$FTl6L6~0n7mT5E2c%&>Od6-YOeKr6Tn>}NeW@+@fS{W0V~s6D%CYi{ z^Ub@wB2S{J)$*LjvuQte|A71|D6AFucQH_nh*1n!6}}a=)iQsFBjnkd=;lbAtpSr$ zuBCp(!(t_Y53iH#_Ru>sHA^%BB*~7YV0QiN^>(!%ImO4x@_p4-GgIuKJ+BJ{i^eojZ-QCv6|tcR8}#jfNct?;S? zo)Hb%Sa!FwSAa}J?3Mi4|7OG=ipW4Qq3{Wqz)4q<*0^R&*tkU8wu8t=i01&ve(enb zfdppHCms~wd0A|}5r-W?h$ehm=FWK^UU+h#rZPGB48@oA`{fe1j-VPhCiAS0`Cj~F z*!YU8$?;B|xZCH=7e6eq>_%ia6qBQHqzalKw^hZhpV@oew|W=_8P|5f-x5U~-`FN{ zYjYFzv?9>)@k9~boOE8_lw5pd+O+r8uiR-?#4z3Qyq4s7IHGar(BI%w{JZ6vF(PTE zT=WW(SGV%b&Cw+B^Rf>VH!~HFz}H3Mv&LATMjy}C?Oq|@Z4IC~Rb>`=OJ>7SbDK`+ zvq#2urivJ@(xdpZhGJ!=J!iQ$)tYe7G^n7)`Xry6iZtwMW0Fpz4~;H(W1(jC@c+#^ z19#Mwb2stzq?bO-Qx#LhX{cMQs=p^-((jg~!p%Zq0aWTsAM>H4ata=U`sJ`E52&Ld9yB37w^vV380>Ekm^w;RPUdQ% z{SV%pKC@;?L-E}oXdrb|<4|Wj`py{%Av((M5B9E~*L3S9A|hIsqaW7-_3IC=%aIe9 zGxo-d3h`wNQ%PvL<kKU&pjVgsE#vUzD)B@mS)-SDJ=ST(D;jSi82HDT86@-{_Ja+gb5` zs0`whnfH-iu>bV{9LkCs+uXkn(>I+8|Zu0%2=Wn+EF|FZku&tNCSUVzQF%n zy1+Gy|LIb@fUK{tKRP;^yz|*yRdl!M(-yV#lKy8IuJ)&%?CiY8tqdf*?EBq)XBrJ2_XgdekQ4UA)iQgNaEHJ-jVyzVg_wrk zCvd!sWM@$It!m!-F})`qMX#!MbCSgpqBrAgJg-*+Q)aD7ld|Z^P%5WWD5ZdZ25#VM z=yK_VaKtP6eeEV3o)0`a<-V|2$YScb_*ZmD}xI{V)P;0gid9cVK9mOO5^52E9Kee&@oZOV|k342=h zkt9g`stYtHQOMv9T>0Op`D39)AVlFSWSkASufe<*M zWs`4uuLCiRlFR}vA=hG7+xH+x(-M&$zG0I*+l=sQEP~5&ZOU5wq+Z3^DG+4ZCJM9H6 zWmB!11QnFiH`yakIiE*r_mhIPM;r&Hhk^Y`QNDjY{jk9n5No-u#q_D6OS9R zWi-3$i3+=>+Y&$$e@WYz%v3|Tz#urz8{Xnt?vp;=@9K~-;~-!)KitnOYDxcs!}<#) zCV^H+hy=u!3u%8%P7z-x4i1GlF17|3E+Bo$Xm3&^Ib&=?@1dZT4BH>BW)aNWyq*mH zH6Atr8!i++0>Xk)-Skh*pExQ@x@>NY*v22Yu`T}?Yshux{h~N@_G*buufi%sOQWjV zar-vsprH%VMJ*u0bu@2XNw}=2Y1fW=HyaWC9ksRpqtxpzfSup1tS8hu2<#n1NSu=&n5xWW#GbR~UI(e&o)k>Av2QWkf-Y~`9Ie=WQ|*f zjgwUZ)xrsc6S{Fz(%4Tm1`c)05)XC~##o`sw_JBZA*02gqAc`DU`L?UmAp^L&ZK($ zS-;lb9 zpN*3)hfQ|pk05j;U`cb+mhlvVw>iW5uNAFdYl*g0eOWzBDn|l>Hl6Z&=!k~0QYph9 z?egpI6MbMh4-wK=q3R@~p%9fgT&D4;tHZf?c(l|sWRy7+(iHWr)?=gpe@Hw& zIvNd@cMYAgcW{^xY-vL*D|lK-8}XB1Qv$8Mu0KRu1Wl}}SCp3(8@l}=&u>9%{=3%g z7_pOxJ@itT__goc0?mU>h_CjVQxTKHf)C;$N!zmJ4h1?j%P6pIjqZ`5K=T zN6V8(j#0caAN;DpaAtmcYEDTeifz{D>Z8jaYFym+I4y{ZKThzu3+X?S>UUf1HJ@rb zJDP&A8hBzRij3JC5fBIjfcFzU^}z;KaxK}D>$hj7<;H3$F=pu99@#3)@zANSvLC526^vY+1XzL-+0T{ z>GbujtE=lYwKj&yb*elMN6OWK*xv5=j7}jC6{U@wRa!TT@6oADH+`o}{^U{3hkX6I z9|7>LVT9KKTibLu<1NZ?TFXzt@s|!|6C`b3YBo-1>D4&Cxindbc;fOrufM;)a;el@ zB~nyB?Y3I^nN}``)wE0|c+|#-)fdV4XIV*bqeHqk5xnwqjme$mPN*!|E(X8-s!@i1 zOCU%Ok_B3(kdu`87LYJPfOe6N{Lx=Pk`V?IQgrpCZ52;)t!N`a&7V+wuBbM;kCD5< z(;e5JzFms(i90z3ns$q%msAOqPn_@WGNWj^gZ;QN10|(a_zK=GdW714;)DU&KRy+o z2!1uiUzZDm{cslbCsWRXmdpBib`;Q^0Oa{DPWeDK$EY0dPTwT+!;S?;;k$n|x|e9Z zXBP5u5&hq&Mft%E$$ygWW5y?|qTf55#NAQ27m}=wuC019FP*XtG$0llkHxDTes+VB zZWq3J9?_#q+mt;KmfJ`1Q8_51OtKdr*jQ%#N`mBc(-A+XEEnOwpG(sNbSD)gQ=;S6 zjin*t_E!u7SqUz*Kk4sqt*7pt64a!K^4@;jOb+)ukBn0~c=z=gIJJ=|JstPMA5i z=;f@gx*WNSQ*+$h+MSx4+YK(1#8}GITPAQJHc7Ya`s9x8h^A7B+&)D)mK8$s1bn}5 zdxeQ%Y_KOljrCg3$xLJr)(jA@U_B)X|9T^RTT42L8|3KTv zY%5?QaA5Iu$7|94V10wRy*r|xsE*!C7uuG2>Qljv#8&U)Rq21Ghy+y92j;#ZGT>#B zowo1xyiFPKkh1}0vwX{P#I91h&Xhg?qIhLS)=gxt5Yow&=QsBOaI&oQMR#%88^`b7ileK~e)z1{!z(ICPSP*Lx-*X_$OUWFbuv#lm0Dn zN;9Mqx$}hG9aAI+Ep8;LCk?lg)qD!;A6QnPA-bvY2qr7Y8MnBHIw5?8Y4gN_mDP_J z@5KJ_TG)x3L0dD0nJ5?3>{JiQMq~^-ViTS_JbU1FOZDNb@HZo%qZ z&>3wG^!mTYrsCCdr;fC%QV6FsXMP(^>DIev6}!xIcOXbv63p%~6MtJGIzOfoiUmxW zaq3?w(}fc?yS2vgs-xqqhQ23pihT++4~AzZt6e*%yB#dXLNCEG?v#itvp8bDPY`c) zC3@P{|JX?$;NRcg!aniDa>`7}!lE`v z@cbciUUNT99DR2Fxog|@?KVy4Q0>8dqEO;wJdgQ~OYduK;IhE}M~jvSmN5_MTLaui zig=M@7;%*z{~6#+?6U(EP$AeNmf8-Ej5rU7-`y{kA3ogq+HJ)50eXlIrG?>pSwgx( z$O>6aogqYSJ8^fFrbA!so3=QSy$CC3=n4fckygZ!{fG9}xY^Cg_qLfEjp6KN&4SiR zJ$6u=!NE|4xij0O9apFB+f&54A$PLxzosQ&UzRrgJ>54?*3v|CBBJ-%8SnfM9@hC3 zFf+z_G%R3F(dFzkZ6$~#><c!NELy+)qxmAq#nXZj7Da1M@sE zb^3k3jDI}3lE4nbPT;+q{a#}C*8=iq=aUB(B`ib-%5~ERwIcWHQGiixNz|zShM#_T zIjm3{L+)P2{nGYm2tW-;;=hvI-w~nDtxnjxbVf=L1Wa-#lYbA!({wEI;P*;|dT zzly%V+P%H~7t3kv3qV2)d{P<_w2`TJf17Pt?aPcP4zO{-m$?azZdjP)0@9YeQvx$sEYKhTgr zey(s)pfyp>r_Q~l5-=j7@b$C17Y_%D->Ilj(VIW1b_YyCppMSL(Gl+o60<)XP^veE zWi2?z8-9BtE$F5BQ<}+gL$RBEJaykijsk=Zd@;l~^jR(TX;{Z&~v+|s?)D=~m0&1pkeIy1G(4jLCbPmLBJr-}CMt8=(Q;O6s9i1+b= zR^-H)qqb1A7><$Y!7%36YOf2lQstUcA*rY)(SSwq@_hq}GI`5^wV%8W)Xtf$A_Wb+a}_U%SZH_-OZdz#cet#3 z`}cqfNI;@2=o=bkp8o(8|C{xJ@`4-omQi=zt?;9$uBic9V&E2<8V|R62{_s^2w&8l z;oV)rk^~#X4F7940JC8a_2;zYTqtO5nP|+k#s<6{(NU_mZeBg4-mS&euVIgmDJR0E!D{K#PXQwdSZ2ik zxJ+9wxT>3*8#kwy?In~B<;eND>lZPA4!Zk_?A~*`lCzCloC)aXdnL%HydM9G>;G}7|F%_GSWn{S znpEHtdVIxErJSuw@s-}pgpUaissTu(xjas>=UK9I0=wKN06+XV9!v3&D`Bjll_ehl z@Qo>1GKL{cZ2VvQ?1L2L!sx$zD)q7L)+*>v>vfx;evaZxZ-$;&?d^N#u~=FN_`4t$ ztUB~Dip;A`+WWGXWW3C2+-9e%=G@T$WhrZ=;z{eLKz7Sb1E8Nb`d;IIr1x%LlU_&Ys3Bb?uawS7E%P z>QRx{ha=sGch&$Pk#LaP-hTFL=u*erREhuHSuvm5(3wH z9_R9UyXW``Rp*XgjkBh+iTo&-`3#x25%jAW@UC4fB`ki=O_Y$VRHhJwwy3_PkQfppJ) zvR!vemo)A6SUV$Rv*xe1pqJh0v@HUN+#jc7Cb=8di)L9Dl#suJV%#WZJm>tz-rV=s z;>q>3eaL9<|ECif%352$Y>AV33= zyXvD}#43{S8OXva7kJ9uwAWny!sQ3N4*?W+5)k~w3HS5}#-V@_6qZ}b!8as%rDsV> zCGZj5XdglYfwAO?I;|>y)d7x|kO=(2YVCCUU;lFUZBg8lMthvA?G|6`pKnE%oNBFt zff929ovlQq9g|3;46A9~@oexhYoH|0w9WH`G^|d?#n{ z;hpebKNrkISE*<))1 zM2|?)MR5x=C$@X6Ze|;f^C47`{2roX;3nEiXO7u&cDn&;aJ8+hIYNiQF;$7sM}oN) z5z9|sl5QTe;wAwAM1xe(o!G?j+k* zQ8!~G+G(?B*b}VFfeEgNlTa@AyoF)T%gF`iWQjEvQv`V%fST}d3wIst0S%Yz~J-i+RZ;Z~Zm$rS@yHEW(< zVi++wFWZF2aC>VyFa{jv43U`}=%iBWJsCruQ<;Xko9KtjXM?rUsEEuRCt; zR2*3vN0QZ1n#|~7T{_DXE?d6%apQ7?Zm0ZS1dV|pY{#bTgn=&VUPM6T@x;SSMK=$( z<9+|!bM0^^;Qb2MPNNIsnzr%`^lMNqRZJ%Dq6;GjU%gX!LE=^PRgl;|fU!=VQi!k3 zdJiV2c|Vwkr7N51Dnnhagn84YrTvUAO#Ir2Z+*Q5?lT8N3NM*f9PO{fn!L5h|G8Yy zifA7fx_Ghnj0LkR)11+c2#nt1xLXozNNZp8by3_eLa~I_5?Dbs%wOM0rsUQ%PIsy9cyRC!JGL2}Oi%_qWD#9) z9#><>=$>p#VPump!LRoIC^TO}<}<60KKgKKL&t8vxXTNpgoXZ5T7T)TJt#hlSf&Z# zBfr3%Gx!!Y5+*iBH~lc557pnC*CKZ(ftJJF@eaj)L5eqj{4wV};)X8k4{%e^Gy2MP zQqb5y%uR8Z%!*V*m%GK+Amx(j(yx@Y{Mm+FGfzFxTX-p61hdij#vE=xR;6|RNcF}< z^%Xf0bAloRU0?Cr`=u=)o2(zNuH{SX`laKTBtyRs4Su@KO~jd2$EW@3UgXj9?qrR7 zRcF*U9T!UAD>n6)?@UBJ9SfoUYO78dzh1K_09zeeBFQP2d!Yt$NUy$o%~C{nLZ^ZM zIa`fBbzDwd{q`eLi03nps}Gak%5t2Kaq>l+qWF?Kf5+Ia=y2wYN$mc`!se4WZmo8Z z?!=n{6d#$ia|{g4Amee)0oxgW`8cE^^{0%BC`(%t!oLaeFY-$U3&kfB&m-r-^et;@ zLlokXw}MqO$z2_@Y;vC7iys^*u%0zkH=(^0Mg0*lZTJ;bYm;03U(lhTiG>!3jI|Kw zh!@rEBdhy%`#UE4aE8d9b_8>NK8nxcIf;cx3fRz-C@~Oj_-i($b>yvrsL83{Ql05Y zYWtbNxa{n5?qhI*iu{XDZ6U7%OswvWTNJvOyYj;Pc1Oj(ifrn2AwIw59jy%S%mQGs z-Vl00!fLJ*CMvM1iY%{Dr~ouQ)|=Z6h5k6E!eXElSZ?B^j3HOgzaU!uPQ9ygAVKq~ zOs|L@c+pjB?;6Gdl#ME}CjV7NqcH%c0Xc2mVQyZ{3ljrV&w(*a|4k>(BmZRQe}+G} zP0D?br$C=6^xHCj%D2~j*E^*hFYUV=Vtyt*Q2MlLLPYRDaX9xQim%XxcCQ9EV{ISu zc#8T-zHJ$jMu*^ZVK%S&X=Cu^MbkwjOd|;Hn*~w5SGzq0+(nd)e6~{+&><;9@PkzdDuk?hV|&I1R`iE-j1tet0(-r?XoXA(F z$F-phqkb>=c2>eQ+RlZ7;TN#;4a2@T;nq@dB^1=dDbho(D^Wd0+~IDu48MHb;#vLv zy6FN^AE3Wa%u1eVwnx z)5adUkIEl8TvkI`cvGzm0YuH+^U9FA?w$B-7eK6xRU_xZ zhgI`Toq1BX0ie9<32YVF{`|)YS1V)*d6cndW94YE%a{CqJFsL4?4*0nV1%^#<%kr3 z?aaLdUT$}F?+Mkv>lXBR&R=&UEkpS?A8<#1vVmBDqo2Ic_lo2j!7i2ixUOyaob6`B zUb7&q{JO1~sXVYUX=h=pA)@H$ydF5PZDCjX?Ykv(2dO^9! z>>F9Zn*IBXMdAAYCF$jVY6(t6P8cqKY-b!uVI$Z}aHnj&G*;^Dpzyepff<&riTaJ= zqYM`l;Mo6wd_Pes0n_B4@`bM#`ohokxM5})5Z^$B0Nw=6eidiN6woy9FK zSff|1rOgRjB}FcDp9P!P?`;eaWNWy3xsS?t4^Yn%Whd@+QUMuQJ?dj!K9LGgp|)sv z&0a~HKR25`+)5c)vd8D{YA1H)O&D_$w;B2->NM*z*jl>I)$xvS0T)B8zupq~l{=go zRN~_o^u5{Y!Gus{%}JFW&+PPp+|R|cxPgLluE{U4+fabgszbk~Rz~2(7XPN&t+?=B z3Z7C^J1_s+B71aHX(3@=9RgHp1Eig-X=$=mZ>o#)3%n?rS~xN}a(3DfDZCaQF_$n; zs4qiTG)VgfVrvn^R9vW%tf>>5;Urzfe4#yPMb0@2?Fl_8GM1RM`kq7#I)`~>3XDmm zNU4ppZq4%0eJvWnf<$K-#Co$Sq$91?lXtXKZnV5^MKtyhvqWMRkisb_2QOguN_1{fONQJfd7kYJ-;_~j;@b@& zO_~Ka0~W&<=N)};g~4Nt`)N=^qResDjiy83B18VetDz0L0;>2^hJ|NMOB8}TL=lI##S{U=8B&qYwA*1d|D;MbjZ#6=%6fpemYXeJ(TlcR>v+6iHy@`P zsUcS^jj?1JC2YL=gE54wTj)XeP{Gt)4Y$k4%XRqzPl}HqAr)6aMfbm!wj-VxXMiS8 zWFLc$WxoSmwD$N%%4xBZCso@jK7{bvRayt%4cQ;gBVi1u?4zB-kMjpE6g97dppVZK z)OEAxiYhfv;$m2Gclx;=udw&eh&Lk?zudF_0{sj({+HVxfBOG2mYNU^>EwZB^Q;6n ztO(VwW0Rs8Ur~iwUOjQCbPAFj_^NDTSpma@sU|(VJ3$$Edhjs_*Y=wn2z||i!{e`{ zb2h%#e@4el?8H3AVj#*|JI*N-&oi(=} z6zROWmuff8SW$0Qs%W#h;}qPjN2FDtmQqD&I<0l8YB?u-9zA-lZb3@c-<^i_6M7Sv)HZ@3H_rYQ;U zv>{yUY3p3bp#gA4>PyL&sd%rezHlOhFmK#sdHD3y2v2-vF6>vEa_P>o*h$r3R8;9X z{Hn*vM2xlBsprzD1|21cMsHX-HlhEhJ8Tx^bp78gg}(t2ux6VH(Mgk7XOnA5ioo^n zz3;L&7)mL$RI65X>`9{uu8kruzXso$x3TAcJO^1st~bx8NnVrrcBSrt2u;>6LE8Hf zcg=B&=m)!d@)1t2c~N}xajwnkWHb91LmEA5^4yt^ z@^2o-q*6-e*l=;&@3`U8d`TfO*Ff%$fQz~-b}Rfb^iqRu5Vh$S-O!!2&(R?$ePj81 z6idGzHG>Eeqh--k~@VwswC;26x~Hd<^ZGWO&~RB-Dgnydiot84-dB8 zzKCZPaU>O|+i-XUEQ#^~L6aq|x2FZrq-+V#->rkT$&2GzTRJ%zEpzUFnVFqW%qkN1T5 z{$%R_xZMX>KUHiWBFY!dhe>yC6{kG*7uJ=4+bz9u^PrP)TLt3-I}rfF_ymL`iiI1B zc}DZ@Vb{&7Nm8bd3wLWa^02pJ|EeJXkitex!le%|zt{wLP0&7dJ1v85%K)uTW1;?YkUT(rbdjz^b5BP9Qy;a`jR3#9$8B>|%f75oxRfcl3&4*-Dizl=ow{POSh{XtXzv07lX9f@Cj z902|PAppRZ|7E27=a+aG8jlU*vD&{t=~y_Lhydt9)1Zi->AU5BoVcfg2LHT*-Kik3 zU-+frAO7b$uKpSR4>MbR`{!K$eP&<;E&lVxf4GKU+Mfi)e~hLc;Aj2AR;0mEzobwo zGkL0topoGX+|%A(EwlE9pzQg=hBCxiXI zH<<~1WE^0lD-oUSvUG1NSHDJGmX`Y3--_|RbWHc)%x&9sl~h!GHlrywbz1Q^ z=S&k)DQvEDt=0~HblZMX_Quw;Pu6_`CH%Mz6N`Z|NT|l6enDbkixO7Fw(Rte^P&Cz zCFX%5=h(qLy3BJz9eGGBt;ev3wD-v0YxII~;0tnr`g6N``b_${% zsRrM5v|8DwxVk$UeH#AYc0NTFj506eM;kOiU`k$DqkVo8Y2?_EYv{H>cII-jgaq2{ zr{V(;_zzYQ-%;ZL%2tx7l3&45E?^3)mDpb`hLnc)?Q4KGxsjt-g`NJM)Xuw`?2LT@ zGuGmRc?wf~(w`@ce^}jWIwH@*saI{)=B7{Y?6-2AXX$6NlmgR-ZJ1ifPHE-3t%}+vGipQ&)@5mAG?!o_Unh z)-x%rP=}dUZ!gbIde2I@p$icCt1Ujm2%MYRXBg$D0mK?|^ylBSG^z|B{^n9{_^Z?( z>NP{(OdH-5+#5z)9+ub@#*`4dn(jL|`P)gDAg4zq#^Xxe^7FKVwF`2uh#x8KzHnnq z#LAYx58V|iAsRv4miWZ=JH!W_F)5w;V@>FF)l8|=kFsI-C;n&UrUQfJOuWi6B)|9s z)X%hTL3v|l`-SM@BVmF<*7sEl)_y@VX`(0Jx9byuY28SojQU(B537m{1{jHN z>f1RxJ(wz6+1CSJ8+L0}e58+dFZZCt!;DxF6mu=y{Zoy4aMcT&R`1qyU7_P5B%f15U)0UZD049cb;2M% zTg$_@I+is%QKJzWt)}8+?RdHqqp~xs^fcct`ojD;_%zD5t&U zyajRdR)h1w?pjTOWA%F8&+u86S@)x^BZ>mUw71ma0{E;4FPsR zBglZIQXiFTB1G3D&&&r1Bc3UL8CLw>KlNE*BQeN!Y5O)TqOwx)Rx1=3RyB zV&C22zT><4B4&1=lyHA4o(mmcI0fTA&b*|;;ko*6-N3ds3<61!A4a>Z$VU$F4Clf2%-00f{2JpmnJoIlopU0 zT0o?i&|By&Kp>P*LjvDI_t|Hk^E}u0{&@er>$=jj)*5rPIp?^?oFky^`y^!DK-^G8 z(x+%h+|_!`SX6WU&Vv>knvaxm)V*dkqVz+Not2+pnvf~z?o0L3C4{3G|dA!{SJX(Ys{sBAv9X>A!DUthd=3L+DG?DK_I zZPoSo%KL@|w;x@`YzxFR<<83A=`JYK6zz`4a+fqbdvQ%gVdC8^-&2hn!4x^>fs_cu z7dCXmg{r8~vVbTaR9TcgYGraLnW}T$L-=P&p8DHn*Y#U;g540;3y&tOgX-yS?4BV4 z(M_6xcy&lQPXrT2D#44ar|?kWDGR1<{-G1cWR>#HL8G4G1tC5V)ujS>{uKL7_L=WH zW?7ddrDXQYm(z*Wz1tGqJA1n(myR1V<6K%jLJ7lSe*|8Hc?8+ z0p#9;_mlL!q;_s}JW2Jc4{JhALCc)C8mE|#H-szT%t24sz#G;Y8~DgQs6lD9V@5E{ zolQx_XA7em3*Fu9LrD=as7apxJ-T`A)R!cQ>mR1J6y-8Y1a5!jG>aIJl)7bk>gb@l z`#CYBuz>jzX8THp@|ciGX`?udMD>L@uh&#@ZbC5P^Xd*e`W_SxKQ({cD z+lk?h%1Zu!N39fs1<^wgo8M`61>)elIZw67l zpKZZv@>1m$t!l%}ISK$1HDJyYd+Z8+a^e}(2+}v4OBO0RM zAW%AQHAdnk{Z1$6AyWU7Dfa9uRSKQ4TJW;rC8+FO8X{r71qsV%8LLG6a9fl%K?O}6 z?jSN0XE80|I5yFZL)AE_+6I1^W4*ADSYT@qS&%K!^!m%HcqmAa2US>Xi z-0DHJDZkaddTrSk_ulz%vhmrY}AGULU z>iR8&R%Owk2c3M`Jd|ju#x^?AI3Yq0rjt;yzQ9_e3-1bTvjiDksKH&*r|Nz_>NK)< zPU+>|GqNdGi&c6XKWean$@KveGpCW+heJ3sB-uR=G*v4x7 z=Ns63*hSvwRxtYj)hU1WX4%(`E$E%9quR$M%Bk+*t`dEgYK@?qHYL*-Y+j3$H+KX-3pyn9tlnf0MTnO)0s5P9GJL;pILbAANt&Hd?29hQ*^ zE{nLt#Nv?q92Y-tL=!sOSe}>UdMW(bJ;1Daj0` z4Wq<*L}lpHzHgz4?-9&jyC!37Y7f_G1x4L`f*E|zZ?-G-W>xmGCt77YxEW<6;Q+W`FZjm@ieKh`EVtsE=Qh z?y*VVbE3_y)aSGfy6?FqPm9G&s@Wst!lQfKbL$SAI%?kd3(w0d=~?P;t_=pvGJm-9 z9%p?(x92cC1`>-+hS*~Kq;Drh*bytJ-r_UL$=-VwkNP!i`LS|)ft-ZjRE z5o%Yjw$4YJA?>nGx?q<4h9Nb&u=Dvk|Me-qT5`2SHn93awZCOkyvith@q^`?$2gYg zJ1R}6@lv~-xXak=Ce-d7^*zVEvmehvS$(Eva6j(UsN4&`+MXMCH-K{x!qbK$id+ZN zhJThzo<&Q%yJyOd(-h2i-8m@I4E8c#nLnMYJg^vZb}0CbE^YO$(LJb?-~2kU3mF81 z8%Y4J8yCPdjR`_yJM2lPfdY%Rb?WE;83_>X{Wn0r8mkKn412!Mlf!eXKojE?_6udc zC^-@TT&wjPC^4eTrTWY5>bnB3?I)Vo*gZ@8W7Yy-Sdy|C*Zj*TcHeBpS;68ugdUcc zHx0$z`5!1l9cIR(tFPY_jK>`)KbQURke}j|)`F62X5=j+4wK;L58=fOL3V?%@DaL({ zO*=JY>dp(3W}zFb$DJ;ZMa6XQfeDmUe$h*+QM~~B>dLoz0gGOCU;s}U8S(CWF0_jO zSK#|MrYWr1fZ7^6H{(2q5#8SD_nzA<@o1y@xXpNlY7_mnDK86w7r8jg4aF_4j>S)g zwr(1nhw?EuvE+N74u6=g9R{26kuIz9&WfyJd6faOP@eEwX`#p&$4#zM8}v>qHluK! z`ono~yPF(wF>LEb>$D96c%bn3lOlmqhIjyx30|o%`+|GnZ;wbOso4^3Jde+vD@SvR zRdn1H*ueK3(zwUjLHNSlO=GTgW&B8-)1zmwejoGbG?+G+a`toB&Yjj!bai1~oKLM$ zeR@j#PC$?{vb zrly6@{Jaqk3v~WCFz8ey2VqJtweMoVq=keOHlu!OC*0N^Os_+InnMImL-0>wxXH|< zvJF|n4zMPKrC&7ZG_U;>;?$elCK`p~wk;u^-;2OMZtlI_1Wi0}t?aUEOY^yGS4u7j zE#LKMI=>!6wUIe2Ov$e}{R=hahp@Hj7j9M=y03EQg^m<{4#E+dG~8TOe!hXd*s$tp ztXqnW5yWxRo%5X3IWXF`9Vu*A`C3YZ1dvC(naP`Qqmuo>W09RTm`H=BuDA8~Efz21=-u zJCXF?;J{z00}UuciMuQ}0$_mxkIo6p+S>E;x)wQ(et~|?v7f~>$8Z+Ja^K_*Za~Gb zSNCvKKfSXDq;+3mw}_Hd%jVkMo#MUiEPs9BxZ_pjmt{R3=m-AtfX^LImHAJDpUeUP z$2T3@q3q>u4UYA5KBnp>nO++kh)2ERV5pnFbfskFCDJjbHAx;Ge5;> zUU4uLCIrr5d@DP;dK-$*{?fgl@b36F`Q&{LC)sICgPIyu>1v-ekB^D%r{kY*J=r@D zW>s^r_|Fv%?M>mr@Hn}kJa22?XN+BRotJBA_C4KAUvA(x!;k(O>W=*z0Q5O;bI(lM z+NafC;)T3Q!?j8$08KVg*`J}$jJC1GOKdmdFW)Se5+n~#5$3v?cnooa@5l2<_S_#Y zmepq;;iF*-fE@_CKYjr_YFUfXUm2W+TpCjYbyzD&ard2h7M)|q=^l1L+LSXxqh>RJ+ohl(6q%y|bs^PXNvygaFH-7` zM7k}qf27N68SV%zVh(^AJe@}h8H&6TfldZhzk%^lxVvXxF1DE2el(Z1?`fg&Kp^{V z3(Y#2hjOiQ^-T?AJy|?X@&UtEXX$IQ$$JePCF`Nd2nV*M;WJ>cZw)Z^%27mCI%8k{ zGlE~+Mo%2q`j;`qakTI{uC5F4UKL;l2l$N_UvovAKl7UP)$krJqT!Axu7jDTAoi&@s6H_5Aom-o@#EMR{VR_Uc$u`Ig8mzLyqN)AR>j@f z_@s9V`d;|Qj1V;ThV-aZtv&2Zmv1yibTE!gds4{k&Ua5x zIlvi0VWW?SWZJhQ=AGK_I~PR{?_LFITH%MNEw$L_uO)>%GwOZ#4v&|^`ixgkRdan~ znwN#FpKV=Y*)A=8BMouZ>rK+>@wtAWF0vL6>TB3^oMREy2Ik}olCQOf>5 zCft?|-k!g4IZ31>r1UPGLl-o3jjaP1=u>Eo1B zr=(>i?o=&CG5kyGOUY(<3sw(w7_d($1cx5Re zIm|Bocy=WhEi`=LU5eF5++bPt?Pw2EF6rBrUzq&K5plv-QiraFcGL%P9y>O85JR3{ zdSf(SWp6`JlzI(U_ChO*+s!dQm;ruiHEg1dp^G&!@6$SCiGBDS;VSP`ASfkLP(`f} zT^xUoU@`Z3coh|D$EOm#dn;((;AblFd4MFb#(D3SJBDv3$K0&!-GyMh>m#_)3kgIw zvu_n>*^Dr2^(a@_=9&P3aNc=>JoYnmEa5fE-XwS5^GM&#JTuHn>&&{IOG)khRr}8l zOyt0h!<$7D<)&xSbmkV>j|W!^?5Y@#}=Dl5JZQg+)W?w9++NY0R{{ex=z-NgnJZH3)u*a52v1iu*1vF=l~ z-Ju=+Unukm1`Rk{EWhXt&q593*ioYWWg@xQ`2sTfFQJl<=9}V_jS}@-FWA22Jhv}S z2MMUq^W66x?=DC+Vj}y%7OC)Y>V4Te*;z6LUHfAjVBezEdDE|woo&i-4EIi_JYe6t z@QGj(3iGW!bv#|=R@1iHv12?YS~zYd_5u#}iG@DmC+FA*hNnnQ57qdaL=3BOorlRi zP+2^^s$k=OEcWzpyC=Z(X*N$@^MZsH$M_DVfEX8hB{SrmNRv|6aY8G0qmvmf1U=e2 z4wu646Tda1e8lVB&-kd<@0~Ruc2->Q$*C8nqQ&V~d%Bu!8Ldu`z0q(y)#CW}%#C-a z1TGanzb+TM_#wtDa^?;1>GGF7Zge-l*iou|lR0Je_TAH)LZ@E8ri$F%wZm!^Vs`dR z9=uw_)mHn~NnOH?O|ULPkJh_m`pZj;#*?5E_XcNEtV;d7u31E;SZU-of#8f(-11xi z4)PmbaTaeFl0-Eg>`T5rB#tZp>}qZ#vujn`1PVz8GyG7!@dL%!HxzdfCHvR{Y4cv? z1Wx%427TCVrFjzMO+&l0=hiaSu!z|#cB<5aif~XpGlmvQ7*w2IJj;a|QV0J32O~<$ za_3}S0ZTYl@!O!{J7Q*V9+r!kRct6KMa;|A$p=GAhIe*{d7m?1Wcm(ZIDZhUf`9|j z5cuQrCny!wcv$sf9F~%wrV~Zgq1<6~xjuc}{{)o!4U1Kr*RWR!iw&oRNj!QqA}poV zeS{dE^k>q(%auVMk5!M;rv-{TZTCXz$t?cW_!F>@Z*cY&wY{_@pNhEfKflL)mwk49 z@@>H2y|wYb$pK(K1bZV>k-fR#$?idaL2kej1;`)7?hnuhz**9hzag!^0X*v8NYq~_ zoHPJ=ZJD2})%ZjD38MEO=by%dB7}cKYJY=zz4)kD{j6Y}3ZJRg;Z;F}Mll&wc1y|cz?ttB0_hU;4bvx+H(q*0u0)?+s z`{d%ov1oX#x4jU&W?aM#a|{JGMk5S0(rUO9zO8}@MFaXrk@oC2@)D!=Eq z&wpSw0EsF;*#&h80z65BA!;OCK0cUiNE=&6m~K{GS6 zgYLrW`iTmOmefT{^6_Wqr3b{!{%=ha56`B-#7aHM*}*&KWk`Q2KPSWR0eR#5)Mf_E z-plHNdysD-7sAflpx8tiU%#O|L2!9_;CsMv;J(*xFA*Jj0&gu!Ef^!wbq-FH&65Y zX!$0IZLBP3tpYCF0~XIW>i%(!l&@M=Q^D&o+QGz(V<%xbwyXo)>jWZObmf_LO4{>X z=WK^jv%+%QLDrGSzCWdj%d;uZ!QuN;3tyd4ZbaLKH$l3RMaDW-d$TEij+g%+kO2G& z(Dn92!f#J4VN}wxTgoinyjf?$bLJ74_?JKND3l%lm|W^%Vx*VPQ{D#UrN0&&HCia* zdavnf-`t)1MUw^D^iWv)O_^ZwSkQCX$xZ!RKh5X_nO$GrD75_LgKD$ zl{?E==ea>VwXfkcb~kSL4{mw$%4SG_~Tz9-uZZanP$jlBLx3*JF-0^vO04?ZU@#?fni<#7p zA?!3~FX-^&uqL;Ju>%&abs0MU-@)wNc4wJiO?95t0qpt23hJh z&v9+5?loz(>02{lK%BcS-CN)9xL&HwHO-etZU>e-9;%vUIBGfSbBGr#AeKWY`enTE zQ3gHZ5!|fbvll9qh=5(L>yGqaAgxvp38z=rkr0ti&St$$O+z~+j)NZArsba(fn7cV z5@FsKbML6G`?T)pfF_{-Q#R7zim+cr{219T%OmI6x9%SgwG(UMJO>F zO$BXU#Z{TwZ75;%wRpdV z{pKHr-Q5vOiy`#&vR>;eICmh;$$S99FUyU;}Q*w$V^SVHt`WIor6^10G{ zLBNAmHEW;kvAmJb%NlDFwDQH`;&TL5ETu}V!%0gHZB9;ojFClkesO zxF@VbsFq3n3m-kxtR_5ysBo%$EX^5V{JxdBG7e~KIlaw#th`}Km5v2To>P635Dl8`~is^Uv^1_0DWkC^vP&wB&~!1;5`iez61e+hO6!${A& zlJt5}{yIMsBy3F?hfqV`682U@5eqeSWpRO&bvtJ=)3e(8>9RGq(4V1o+5QGvhua<$ zB(lfV+a+2|7`*pxRhI`Z<8oQy-Kgfn9N1TL%NC}cdJslCuuA0%**~3Zk4bi&F2VDt z)6waV&!xdHJqp(>hjh)fQ0$$h3cCa=n?Bw>P(+LhPi2H0lk9d)FkhoC7iol4s0E=XF<1W_I_@1RfeN&E|kA; z^QX+xtAEN^8WV99R|$1HzpFSi6>)GbXS0``B?cpxMm0gJ%V7lK!ir<3?m#gwK^xMuA*Q8HI0R+*` zUmz-}V_--ThCu4w&b#bx7hr+H4kE2Pob#f{7xiCIYb$oRyOlv>n`q$;u9HG_Cd%H` z_Mq@5h5X(x_P!Ey`m^P*80ij2$G)|;gTd7*S5iJjR-oCKub=60UE0hNKhw{<+6?Z( zpWWq2ET}-%$V!Ev4A_N}G$bI&2Oy&P@v<|PFnZ35AcFAx%^gV3h%xq!j?!KWO_4eK zLqCR#!JPsv6A2<6JA+=bY<$?R-XuQ>g%x2z+RDTo{QL|E8=2d?y|g^G3->nCq0Dy( zR*&+Id_($acrnjg^0W?jNX6B>wgYgP_wEYN2ZnB!-vG-3mWU|hZ#wP@O0D|8_RWi2B zgxj@d^2e?UI}h;1(XZhFcPZU`ThAOAp|xErEDVy`T~zS$(|IYnzMs(&H(Z*fw_T#) zRr8SbxsAXT@i24C9D_Py-ipgag+7;9k&G{w98*rtC11Pnq-0klG?$Ni`1$ggAq zc3<;bs1_N4!VTB9)>0VbV=m~UbHPQKb!DeMtkpy{jl>!a<11a zQn6+t_aXL+V-sC+i+(0scQR^tyT|2FhHzZGAZWYI0Ethw6DW{c(GAH~_I~QZL0W~c zmR57=rIZ`SjC#>bLL122j!a0aY28O`R;vwS7j5rDkPPPDG|omxDsy=JC!B0|)8v2!v) zr^LJqs36oFa-%;-6((dmHUR80rxgFlb8G$+-CbPeT$uQ3QKK8$wqmUuZCAT9q-Td% zpx3i09yL$2ty<2eyQ8#3PvZF`KA>;2KmJ2c6A|hb)f&y%8La~@6@mDPH$CHnu3aJd z?ACs|R3_b3)prgtZ-O3wC~{QpG%ewVTIQR|y))d{*%hjKQK%2K`D;l@zmazWoe8Cp zqeA1`Lu;z%U8+1iP7V4UN*o%gD@?6ZS-X6B@utCb9iH-cTQAUUzy3D-q?U6+Lt?$F~wHhmfFZsWEbq#c68;b7Ceji4SGNm*0A($T7i{RCD9&k}r|1N{^ z__wcE0_w?d`}*d$Yi8LSK&>J@TbwTjX0=;svLvc~;V%dsNEWX}@6oJczd?6XhER2{ zUxBeJ%SR(uR@5Z(V6`eS;=@=S&3r{{|02+V`tAw$>@pS<(4d#YMpL3!k<<=w`}oy$ z+Nn96J&jpOjKS9>$ibGWUEzM-wi@e!#2u#iK-&uF@z|@H;S2{*>Flhp{K8l{N zw~V=*$G;s;Tkd*v{B*5FkXo;4$WO8mJgzSm(yVxl-zSTq$(z^Bb!}*Bnqn3G04EWk zwF%i0o1O2n>_HY~Rzn3xgN5I4x3)iOTrZGw`MG8$C}E#%afn=<@V7)uUvKHXaoU9? zy!Z5_)2BFlIN7`B=Vw8THz!hA($-j)Y1U2^Ubvomb%*|&^X)UfpC{+Q4JIbjE%GO| z)c>lZyoEml0-fw)06czIVjk^0DFY5Xq=6a3Jmg?h37B#J-~4>;6#M_*#s3dK|DWy+ zRQ&i~DIaS?nco4XWhb?we>YWF1ii-bUKr$Ef(}2azz*d8b^+PY#F{Gb)ARP`Tijwr%U7W zfIeU857&ST74k_*s%TDhuFm4;fZgyTfr!g*TAYdEJ?2_<%yOqreO3I;V&Knq$22aX zJ$Z+Xqjtwm#a*2%99nFpj45nB>whG3{w}S!+&yC;%2Gftwn~7flW!*x1$-6xQ@9i#j9bO27YgxaCt}AKp?;s_!{`w2Wr;A; z#zIBzW5#(;%_lKLpJ?E3p&(cspw+RldRnwrBKNZfcu!?U)Bp3=RBmD1L>FCKfgbf& zzdzb^B0Xtnio+J@%x~3XJ_8WH1!4y(%Kv2y|M3y;184#}IDlxg1=>;Vx(S?f@>k%T zRumA(!qwjmtqz|Pk>BD`E$6XK-~(9?xi)7px}>Uma*_FywMvtnd-qvYN7x{*YOU%R z>?t+Mf3zFZZw=qHv~N3%>3E5QGGiXIw|OJ0j}@Uug&KODh1F)F6@pbBEUfA{+~LE# zz$*s_lGwe&L4_vX{$&A=@D#8g&Zr8#c5eNR1nXh)(o}kJ4o+k3( z6)m@%A4R~XlA*oCV4xkq)<2!pJ+$MKlPP?^)|yvwxN~iCCyfAf4mhOyfNN|yKSv1P z0^g-L5alB|E#taz*LGG2yv_`>Lk0QO$=#cq#YbzD+rV}~Bms*8S{(fe!6%*oIOZW+ zAZh5)_7SNe4?HOa-c#jopCFaJG&8xKI62b3zA8kUGh+f)f);MK*!8OKqKvfn!nG>6 z@g3a3)=XYn$J<+ra}uUTwVU}pI5)!@C1a>tsIvs4U}Bd7LVs-1PJoeQ zsA`nkz{|(ieK+b;_GGNW?W|Elc%J>7@FJvZVX)I+4XIl|6GZN7I;CZ@b8%{YFUAkPm$qbjXPeo`h3{=rLG?kG#k3p$k)xS9EpUx z%dM+bPxO5XJtq5j{Nc-giWLZFb0di8c}#B;4B@5j9-kE zPmfs)K$`jaa~l2Cr&d10XObBxJ%#s%`{MkBqBc!^?6t(oXs1M&s{5CTl1DmxV&E3B z=hd2OV8ZdK4UQf19Kn@#sIo`zcQlFqin1N|J$Q=ZT;$B0cl-b13C zLgG60s&cAzS?-ZFDzSM9?m$a=W4GXAw)nA;dmazm@grn;{uk-gFEQ;CjXvpmfFlA% zcZMAdpD%YAwLNygb==Je{zj}Bv_N$l9IaRqO5}{jV)j@im{ZQq5LYV*n}$%xaOnCG zvAsL*XnxXnZHhIGhX+FU1?G%U)-$A~rH!Gks0bFKBgii4m4&Ca$8&RcxGgL$HrpPE zMz!U`;a&O6G(mR6(NS)u$7On5xUM&`_SQ3EWAVw>H82bBy#vvq$IU2|cxdxrQ5Cd_ zsMBR{yPU>UlF&6qWrtOm!=)X~@MR|>q!@fmO%0+#V&1Q{&lq3oMoGzFFD4u~Z$F3- zVSD=u^ZCTx1K#uG*m8accz7FV^*B>j$nk()#R7)S=*S$sF_#m3s9h2ZB!Wh#+n8K(8}b$7L3 zdKBYI`ekOIhsMbnHO&wum}$nRqoU~aZUZKjtcAtrP9602Moxq$>gU1UV$CX8?b_U7 zusP;8eRhQlI zHE+#(MuxpelD;1DE6>W}1Rv_XYNGe>@X*iigS<_z8(8Ekjl2+D%E<4d zu$GkT6Bx`s#r>6EVp(>M?k;tq$bNi7!(-%BMM+YvZo$eBwnCNmkd%_Bxmvz*)aA!k z->qW2#FE6@p9bu=ptP!7wswcq-TXhbH0mrn7;D#_+r7Io1n6>?LL=OV*wr6DAw-+K z>()I(tSPOtZ1JluTN9Q{Fmg~&huGn5;-b?GQ{D&LV=Bt+YHpVd?RiNiUO%i&VT(}j z4>fm!qbdyZMoSUH%AZo=KUp&%CmD2@iriQ8UxRP#;qcC#0U_VD3aT(xS4xg<`7PLj zrIB`CM&r>I6|%!a<#4lO)zc&?*_z(Zvopl}r!3dqqNp$rrA|FWUHhlo$GjnHMd>@Y$^*akutL5%qk&grdz3zbF*E-yY?pb3>P`*_+b$dBb8X$?CE=y-) z54H9(b^D2Q0VZzndO5lm;STh`g1y0ax~sOVUR&{3+KDhv`a;9JBOY3Mm3rHyz1T?8 zem2_0G_Jb;xa^V}#JI>h*5Rl(Xfw2*e5Bi=b=+}p)vX}EIL3!VO|aMMf61Cv4{atnNzjVseD5I#|m8^O>H;(+rt zvqP>x8AI)nUF+ycz*-L7^9HLDn0zh6*H%|~Dhpp9CwWa^Aje$22ckF%_9Mit6J+OP zy7XJ^+2s#(6!8v>$*>B>wT+2n((83ewsR<1mLmx!Xer`TYp0ix zfp;bbgeXG>TP73``F>CNn_fSB;FG@V?oP8l&KuCcZU+qmY|-yOZIS%>OXw~>kz>o* zRH7J*K9(#-B^w#Wx3xLq@k_I)Xf-C))qTQ}#m!_#MMwLM27HP6sCnTxk4Y_uvcZex zYwAx;w}I!&!zB4MLV1O+i(sOpq@-iTkM3%3&L9Qn5`^iAe_hi@vSht% zgNJmZHfbJ3=P}u}yJkX>n3nZb0fC@b-a|Kr))T79w}FAztXAHl+oNS*!h>s}7hDk_hu7~b6{mS29z7hU8WK$Fw)7Uut!|a)+ zRu5|CVycXeJb3R?!*uxLo1`Er3T;U~I$|Q(t;WZp8$LkPiJ)@=>;5itTe4kpsSm|@ zIyiWz)$x4BMGzH9Uw)f zhqUn~Zd1kL?!-ry%afD@(Hb(1`+55Wod?^VNIDOubkUL5Ax-zV3dqk{1sX*Uk!48g zn)K1sMcQP>TzYM3`Dv=xCOtgUfMegY_1Y?f2UrLMit3=||B)#}@Z7lkJGcxyk$9ep z^r13gbz2LMNJ@;g;FcgWVo&(-z*sLu%#O43#bx{BuyrnRshMGHn60oiWKIx;RJk#%{9@$HCPLs)(cr zVJTGL$RyI87OV3O_USz{8ykwn)7Dk6TJoTn5dR!&72j!BzdD6_^0*Mz+1)Kfp6qP- zW`1aKF}q}aWrfo0^;}3|o}hLQia4faacaiwHc8?+VN_SI`BfPtHL+FZgYQHV&67rf z(||TU2U9I$&i_j%BcSv$1W_nq<3QY7Zmt6ht{yD9+%}Q3HW$YoJ_#9w^!>!m6+poD z6`#JtSSmf7o=u?9rWZ_HY*aMgX(A9z0ty5lsq1)eLf2E1A2Q5tdJTqE`P{oCpIuqlI^S<9X=QlyMvjGPP*ZJL?zvQ;$ zmkA4ufNFh)!{#`C)ng$a7}z9yKXZje?BA;6%Z7#-r_wRVI#~u4DZZqr;ZBP*q6SXb zE;(VrU+f_9ypMp!p0FeyG9r=DOev@0bKht}oz(epWmM<7ahmCYgr%IE^? zx6=U{q4S;%7`> z7b)|eJ~LO4pC1x3Ugfr9rS7tc6tD#@*6M)HSueTkxQupgH$-r*EtG;rTphE_O66t4(NF zNNIaeXN3Rgs1BH1HAP|+)$`@ocGub(1+)KLXv*hqbbIZD2p}hm$0y5->A3YM|WFL!8p<((4yDyo$ zbyVv;pw_Zumbxptd4Ler)!4NFf%Qg)sJ7+rt_;zOaFnZM}R(johRG8o=a! z_k)0RpM!_%CPASQ`Q$8*!Jt9~YOP3J-&gdY(DZM{k#rliu7iEXZ*F|?{AMj^ zca_S#N&Yzlr?QoCX0T8R+TPYS>vk%3bvHOTI8e9MJpX|S_ol4n1zz~5(lDGBf75g? zAO6GA*0#*`3^5%>md`nN6k#z|TrBSM(PNkPc`Rb#n;O`elpamwf9BTL*LTkWsWEcJ zMkXQFz~?~S&eW>?;N@q53Q5oNy(swhKx0YuEsWOX;A9nh}cko8XjmLP6yqyTF9{8ZeIriwW{QWrH3ID>!6r0{X zMx+VrH&3kNr0h|-{aJPS>Vo8DRY;+NLO5f%)rS319oXtc8Jzo*2nFDYZj$m#S&U9z zsU?cL<&74x5^Ja3;JwmOwH}1g7nSEr`8BTR>9maZUH{}?oZv_AdSJ|nEh!o4OC@+^ z!fqfL8#NLr&oYs{3uIz&yFUAsYN5D5yuHEQQs^=dpqw{af|woL+&)YIot=!(Vo}nw zkI=;;f5`isdGl0nl9I8RvB@ndj`xq9*9X)VeiScX-P{Jr6gl~hhAC;-P4`Cp$|8(b zLT-(bGDhk4_`gQKnNB-D7Y2v9ySq0gUef#0=0NgsaJb|&!tb}N=#CrDR`Dfu&b>t- z5d5`{s5)rfMn70mK;T!Rz|_)hMf-8kF{u{0KXrUGHSATxafZbo*R#TS)T}&Jl+T(e zb}b#NHEfS;%59h;l^kCvXfWz`ee%>YC?Mpd4_Niv=6YJkrS{@Gm39qACW+i)MQsN} z_bjMIvL)RAj`!wH33yX5gAXz`^Q-=2*8w-V_;D>0hNtRo7_@xyo9)=hy%#n8{rzr= z0!(KUxl3h01Nk3$lo;6NW0iD`jO_DZ@-q~)pL+z!`eh+ns-4|k22agMofBt466i)k zA)n_K=h}@@H06TyZAN}pIj}aIznaG}=gY_)1(YGP$4=VMw6AV;ozkeTZY>dGW|Gpw zb2|!AU%QiiNrm%wfvca9$Frv@f%`4&Cu=@=(+r?Ek0$#?D-vl-j_|7)FK?bnPCqD( zvzzTijc;EJxXq-IgDeoc=G(KObBjvG6u@^_p(wG2@~ zB)I6jdQ~KIo@}whSfSd1@cQ({HKn;Q)-R%yc|?o&@&J0X7f zY_9KehFp*(?et{xr}91P-Jya+-$Gb2;q;K}BD@TXSzPH=i4^0Fbqz~?xuQx^df~AU zQq1VGMu`742v-8*#ZJ#6I7mt>u1lcD*LGE9&R2c7DD;CY_*d2Ue2pBLnu!7x!B)Nd zIE&AW0~QKT5nrfzPKM~4oBV!PZW+Z$9WAJ$3r+X+KRTaMmP~;jO)vMyE=YP2_pwHY zW0uMe_h?=PL+?ZZ%7A?1wEt#ee{P(ynI%viO@4Z=oU9M^^e#j0LR6F~J!nDyTl6E( zHbrW;E9*ab^&Za$VkfQ$B{N101xCpTGm7gppr#YA#gX=HjKFx&Mz66eXovZN0%bYD z%N6aZu**IrVqd$f96*e+mEASMg3pp1`(HU97gZnjFnVDJ8h1bkr7MxDuRIgDb@6+N zVQ+u&@+2A7s64(wFH7yWzp7Jm_~K`2Au#6JT8@3-2b6X2W$j}1=K zHswD~d)UU#0SwVDwHO^go2ubbdun+~e>MdN#mBP#h8=|Dh=RTE? z-#m;mud*?XR<^NwT?D;Hd|Q91C$fPAhv^s&tjMb5I&|4k$Co|&ZsP4-ZDRll)31Ex zG^1a62BDj%DRgiOq=-J7_+%S&qoig0VNTCh#+|&TMHJsdbg#go4;4_Od3%O1`T_m2 zJTULPv2F5$J;qV=MA?yGS@gcHz5&2tMc=eSDnZ*B<|`VvZHI-72Al;%K*lO}~&+U*cfk5YFFpscKQ&CNQb&QueN zC0!M8czAe!k*8t3D(ms|O1twJzVLajFzDh#`ei1WkKKo@*kRc5jvn|%K1D;ij&-hj zFQgz0xy$r(K&`Yl`jVC9iE`htKtOI-+}vYU1tWb9ZsmDYU-T7nsIS@v>4biV`RXZ6 zTgJCge$p_1fKlyP^Ait{nTqqfPAzJ3C zAM8W9`RUW^0Oqh;RSH!?K*kuz7Vgw}W!Jsdtp-HS>}mHY@YIZF5x zmUS=0Wu4wy_>;A&Vk(z2ld6i!uVK{0FatxajGQ8Jm(lGlR95c;Qb1bE_8NNEo;(;q zE#`Vv8BFJ8hqUfZTGx?8TDZ>$O}gC%^ASr*dUTu!@dLo5pAat;<%;6GDp+tD_$KE9 z(TDotMfsDCkLO=w{8?V>-^}Rx*|pVt$W}Wh<#~TW>E?^8S3iO<(d5QrU}7xAj>$xaP&Eh63jSLY#3$Hu_4-akg0FSTPFF4%`MmU zp?VnE#R1fr=|v%F5<&SYT@rd>V7CcXAIXF=;c*nL?!aKFw;LAidE z2O%H2_#>(xCGj}*V%#!hwe0r@4A6}~)0J*ZMY=!pjE||X_+i9?JD9LEUrOOHq6_HA znlovYe??z3d-Ql$cgLBOny<8kH$nH6JKqTkQb6n2_KT&m#?x27gW9dGo_Xq=Fj{kw zC*7t?OD`EwRh3X4Fs+yv8V2nALtC#TloF&9*u~j5vMpA0Zh)sCG^|X0R&N%I!C`~1 zk2_?oB~I8aO0Yi|13Z)A*+>a7F-D*nnaOSEe(DJ_oI!G%Y~uar&KdGci=q!I5()7} zrAglAx9k!sGGO0zqRaoQU8etN*Oa)J)q#QRwH(Q}jYH|MHml<$%+DgQ6g8|Q;ovhJ zxoKs68I0l?7|pJ&f(cYiyTFZySm))1c`u@GT*@(db>SC#;zvwD0WTfpja(S%tqOJ# z2#NPyVU@Dc=s{gIx3b_?hCext!{IPqSLfMOXN%d5)(s-*=s%%W|D+6PLH0 z-xFqigU5|9i;I1yED#ewG-PM|`xPy5FxY;gU=TGjK7Njyo7+-+Xn2@o3DVsZKQr#n z{b3@KX4WOEprEUq5RdkbcIFdY#~-XW%5vnu>?f)QiG$NyP0nXh?i6L$&oI%@(CqHM zy5<{Kcrp?FJ;Md0OVIgEl!dR9WqOYUi_SWY-}T-_fWKo1OX1}e#geS=}){7f0D<35b_U5Ua!cR&mm zlkTV?u2PAgi!DOBiK__wf3&>^RMSfrFRFq_5s)IiNiWg`qy|Mnq^TgiiV%?AdjLUt zM?h*s1f=)gdna_HL+CvaN`L_OkKZ}xJMZ56?py1vb+fWy(8+&h_TIBwW`5hMt7RU7 zd1v$xtU73v)lG;Y8`|?x6@NYB#rzj+7Z;o_?{tPAO ziF$HYc@r)EBhOC%@I4Ev;?e1|1qP?C<-fgUs2M-zOB=<1rEotbiKNow-5tN~Z|8~p zW_lk+LOz1vQT`7V+8Cs4?d@CX9@Rpe@6*5wLLh~@N}d!zDAph~28%0`YnR}?Jrw+7 zWBq))uh-B%21}a!COkM_TVJ#M<(LheXv`wXOV;kMdkU~yWT7?Qx$#~$mz@&9Fig!t z8ALKXI|jqUL;BsAwhT$IjduaJZ~4S$raAARJ`o} zRnEh2?&XcQw!=NMeWNXCfzZdatLg~UKSkSlWjO8L=& z^itQ`3$Q8eedF=wRp*JZxp@#dIayqx0>UzRxgnUR!m5kbHyNoaUS8>^dcj<7LYHX7 zQd<@zEisXj`M+6_ztqKlvKvZ`1wB<(h?4h}VGS`!d1Tf>y@`|h9&0li)%&SS zL|-fXla{`|v$Nbi#vV;NS1+${ccY|P-%v=Us=Q7$*w*>}H_waecuvz+WO;&ZcKi9% z<8T^^nm!ABfO5OX9aY~p*4v)=h`a2e&J&a0H5H}5;+T(HHF3wy+}#Fi+gSTa>ZD

`V#H#N|8gtAUU{-*cdo6B9q7UO-moCXez*JksM?|3hb@_n;;h z$FZ2YFivHUTY&`~lCk4v=To3NE5c>5+AcEA^Ts|uv1K9VfiKcLXF5Mjk6YFGw(#W5 zpA$#NrBg1u^GC2Msi+8|))r;i3DO{0r;Mh^=LzF>U!FqXeJKHL*XNR9n#4f6iTtmOka85G+! z&bQFj)MU35RT0P|QlY-Auc%n1i4StigEEb27=yE|#7kB_#}z=VpBx5O3{)^(S}uRv z#hjezW1I#wopoC3i0z{D++H^>@Si5JfX1Jn48qxjSzbjt8AAdnB? z`!wn&YPh_;4n*uJ)>FTKznby&_xPI5$4q+MF`U1|brkd^{GMIh%>Ayl72LU}w&#h9 z`?z`IgZ+vwKLK2y&p3g4)p~A@WYpI4EQE9tnxc4;lbbu0N9uwH)#`6~843Cx1a+kqZ^d zbNjDJ@wgWzVK2(pX9cOQ5*DOmv$$4|1Q8f3+v~C!Ek@?!|3}8YNnV9sI4%K;R{{wt z3*7F?08ZiT?!escM63i;@2hPb(kfE%XdiD zL<}RRim^rbD_2lmU;w%(i*_)FBTH4Y z;5$6CKV}L)Dg;<)0HF?CLX^KnSoE?V<8wYFi`nYd6CwR9XpVUMe{yO6wS52fcpiz1 zo(wN{31LY)+=Q(jtg#$wz-1MnD45naHeqVAwW3~{56%T&519&L8B5>#ILF&~ybHU# z+duDVrox#7aW?JgjCw2X+@Q&oh|sKgwq-y3qD7H8S@KJ(*xQyKKWYJT2&RJu#6AN+*rsbJB#bO#<%#GU9s25 z{u3-XJT+?XO6rs9IyrdDX=L7@<$s3CRc^<}KbwYsNreSv`DYcT^3}} z*ze6Qn+G-LBj?q9ng?y_nb&$MTF7LFimYzs5;HYzOg5p>q|xj)HvR95g1~h-@uGCM zZ&j;O{gDcM%OFR8m%6X?J2%(WY=OFGc-%4TAcTEChR36HC<+&J>Z@WtDX~0myZ2D# z;nUL=k%@MbH(vb2bVCkwgA1to(?hn7gVuJn@vXA!aa@nsCSBCGuIt9rOeg^w-5U7P z-0;PE(_+Js-EelgYC5l+Us`r%0mJ66WPJp23v3hm@Lrc!PR};f*eF-H6ZQq(eir#P z;IMIWG9UTK^;j@XG>4Tphi@VeHRXw^Gtwm^C92hf*q2-2kp;1kjLF!SoR3wMv6p8fZGCJK`Bx=A$=cKjzP z&aZ-(!4U4Qxr-;UUM>9TdxgK#Z@)4Y+(RY4aI{kl8rG_`**OPyL3G9FN z1tExI6xA;nN_K3x4qPC}dhy#qXA!I)XJ4yuJ`V{$V3pX4pvk=48`6G#aU$iHQg`su zcwj?*{t9|(n|8GMirSB6=!?mNYtQxvlrj6zt_6oaO%gDfYSq8U&p1&h$TsPa+dV5f|e~<8KSjY(D z@vtG^Y_gt!sd6q(-o`!EZ7GqD4?*HmpILg!3|lyu&>mfRoy%!4^!Ly0Plf8hm9>sx z+Yk#I^SS|qazl|;oEtoqk$crQ6D3;ljsv2eqsAHcI_?%k7S*L*I$uyV{VAcg0=NJn z6e6iA$(i^B;%o?eGRmK=m%qaJ%>JY4)Yn=FOEA<)6eHk;h+ zswFJvqQVMU3(QPRzY-ICiBUWUyPL)81LYo?lRRknvZ`*cgau)}h0^bChV_Sgqub~K zk@VX4SUqakha`;gpxh)tXGO+=TqOIAnZVgDDYdgx6L8p1Y7pVNr}@(~X^v)Cli0^M zSbOwcn8w=580%s1{;{yDw`fHhjd>viWPUTjgKlA=$Nwivbhqq>FLcIBeQa^2PLD*9 zNu`N;Zl<|Oph*oq3rA_uwM1_wo~9y@1-w{gB0?&Xka)Sao#nuXplmJ94j4d6V3!d$ z!|HbG9fDBAMGgsK?VF?{zg`^2bixQCy0N#+O$%GzCco5Nanc&8*G~pc+yE<&czzC^aKb~T~06*O(5j@Gw#+4p&Tn5K(Uy%3?#4h5;_PjoDsOt|2 zfj$I*E+Kp#`%ZdCXo=lwqliPABes5dAyE#9vp1&B3xt(yluZmNhzp|Jq!t2w|A%@k z0nq5p{4m0!uNRWB?BQEV4eZ4Dg%(O($4X1qfS6#Kg~&PqdujIz^vvkiLGI|? zsR-zRjR3az&Z|xA+jBdKM97VejpE66FVf#36tNDs+h$FG+FidNfrq&(A`|!(-ef75 z-!klMjertCt*h(Y%MOa`x`azIcJCf$V!LoPL6l4vEJ@(3KSU$&!BR%JPEBzeA3wy@ zErMG`eUeX~^P>{o?aiIJY};Wp#u#@xV5*@2fkFdHr3y8&*sNPGVFIq=wkLvY>nk zszW|ja4ez>yVt~}H$}*THBV>s;0bY`k#n#VzOYQa|gSXCqYGucq#$%zu6>>OWexlb;e+ zt!mnnjd2iM+zA|>jy))aaURuY3U*Lb{`N!;Jz}r+s$cQA!^vPES{<37=UCB^L)DiM ziSrvev$_T!#TO4<{iUcoTZRd9X@`v~y**8`vX|X)*_=eK7LlBU|1KCpef+Q4xib*FgxnQl&(Si&rMyLEmqz-4iWB zju9Pvzngt|{41Twy3YityKKqpr8|p`;cr!rM@p-^rhdA|T|e8OA~NlA(w*xdH|07` z1OlBqN2@y5exlLfn-AfU^zuK=X7*G(;1Up}iMG!?^dCue7{P8fRpPG}TeTGn^bbF1 z3Y?2Cn2ob-!;>}t`V|JM-taKEpvg^V9RWp5A_)3iHunga0};r_-}15zKk`{rv? z@yG{LL@^$B(KSR-bHbiG=jLMa>hkm5+Wvf}_4*fGd&0{~;6__J{#VOtv(qETqL(Pu z1prR&aVtd-M}uL@XY*ef##h>b8fG7m{lq}k)L6WMZ^GeYb8~mGyB>EKNgClSt~54B zrYtSjVcsW0~nS3|?s(W$8(Zw?X+Yqj?r^n5+e(_^->Or3b~{hN4X zd}J&KgkipJ8I>c&nTo6!zti51JeeLWQimj|PbUX~PEL=bgG1pKeOh-=79@`zJeFF; z38HJutIG=|f_7+dgo$6T@j3=wzDSV1(+}~)WLWbr5==>pSbf<%&j?2Md)#K_QA{F& zN_w0$U0>QfZ@IqQ_f8gaFa^F2_Vxx!W_KLMKe~%bJguZo;YB}AVIW)8fYb{L31X}J zhIjbJpEe%#JxkwrA9px~Gy*Zes;dRgG(T_e@L{lFO?SvJdlJ?PAEEEp72SCS*)+U@ zrBurbA<6-jLJ5&twj$T9rk&Kgk+aIga!td~#Ft(D{r&ep|2HDQ<=IFpELBnVFn-@T zKk8Ts`SemSjjnF)O>Dt>u9U#h2^@=datVEOqPQS3zx6xwHxtTRZn9N_nrNi#oUP^* zxpFHN7nNu-!n|;Wh6x9H*z4$@W-?9OuCm2^+#VG!w+{B6r}{xlK3t4?d4P-#-}S*? zPU3kdm!i=ABf|$Hc0krJk|gmn1mSQ`y`%%S{qZ5P%;i}gw&tg`Yz_bE!s~p$+LwOx zq91DQ_Tg0%zgXji+an;h++34SVe+m+ZVCy5T`<^n`x1$gaHTfAfX$oj>gfT4t9@bi zdT+Eu$tkk1saxf`I~{E!_n3X+X@>D#po{1p{p~}x z{9!-=J#Jv_+93gnKYhT|F`wuot^Dv7Ywo2H8+da(O^#s_3%~7IzeQ{>yrRn{)L`68 z!~ppv!`Cy24F0NF`^QHgPtp%w@I?iX7xPpM7W+GFG3IZ-o34#B`v@gSKij zbK%=owJ*Tujr(XZuYoSFFH%c753u1SG^7HJgZoepbPC1y`Gc}PIfyi9sH#L$^UaYc zf9N~0`rQG`^x0FTdmA(%{=wuQ*s`L=LgoQzu0SfR1HSnZ}_@XSxC=RUH<#})4{W)R7ZlB z2>D-Tf`1@U$`5UrVeu)$s&9G|q zz(>5y?zHEtc3}h#@c|K*)?HJtoW)A|II76*ojEFEcb|T@|Q^#<=m@4PZvTL>a z0SEQ1C(|@UZ24ghchR$zxc(2GfhN6{njAv|>qI3Ae$J$hf+*1`G~am;f=3CHZ*7NU z)G;woLt0F=7Z{!QRrOy&=2CV&d(Cz5Q%;Di$f>npVG#^oER%AcWps)i zhFvCAZi)4Dyat+~!kashKNI=H)cjyHwT5p{@VWb+Tk+4$R$Vblp4B96E zsgSyQheDAapzLry)RZ!H;XwybQZ+n}tJvUuz9UynjH+Fa0e*iPs)H39fmMU4!XG3s%B3pTzb4K^AN%+$Y+&i@LRY3 zs#wcATEs)b)*2}>Z$Bg1-6fdfEh$hoyTU30)W;Ak%etM2E2o5kDr{b6I-3KAx|OwliF%)@qkolpau5e%P%C%@2J_1nP1eWaJaxu1K@vq^@8M4Os?GV>>ZtzUgNa1ZZe1R(5V7{&cov)F=d%xri+Ki=e(@z4DX!1Z%cyr& zqICV93XX?qNam>duM8?ngsa+0!DUOoQvIO&1x0QTUik}F)sWGeAcj(|5<6jux3{oC zBrrJ@ZB151ZNW$Pripm%CY7KhesuJ=>5jsZpK2_1xnREi$&odh&&PV(&Rmr~GMkJo zc#s?<(#}3*nZYNIBW_Ko`U3+!8~NVy06DGMv9hB#!AZqo0WQUYUIe$@k4l~D$Vd9d zm~NJ^AoftA4dtr?douLUQp@GLgwnQAn;$XO7QunZ&l^wIm}HEN2N8XH{^sp0wQaB+ zqub`zuZx%B4gmi>6zb(}X@hVOXOz-eRz~))h=xOjINVfiOHFY{t-N??{hms^(%<6& zY~0_v4XC+CC<3wcoOCTX(B$n%7JbOJQl65cF!mt+^#u*cVBdXG2bZz%nqHWtbE97n z_6^$zv>Z0t1M~WU3r=-neKMiSE#&acpLS`z73QxP9q}HPn)c#Z=-ym{*GW+Qz-MDp z!tdWrJh9+QYm4hy(>Ai-?-I*M!Llxw&V){vDaxzcEPg(+3c2CRZ$`VF4VuyXgD;(v z06Pxc6_~mUB<&t$%%A3(x+hGiMe-oMpCf*FY)a)Pw6@|Z79Gq_`4tetM+ni@Fk)<(j z8Nr6hui69nG_2(90I!w*__(Ig%w^Er)WDoqa%$AZ8EUFV@~Zol`j6U%yAvJNR0UhN zKDT2=6eUF6;_q6E?H*YxDn5<1nu~3bj%1+`|L8nZ1EP3m*X&D!18te3I zK`PdJGtHvx%d_I>$Jp(2I2>Uge+nDlDSwb{c5zTeEYl`xB#3eb?90-p^{vDi|K4J* z^}rhHDOmd@*^)pU+nCFI~e65q4BmM#h4yYTh0N*QTyUVY(_)oO^3_ zC2~N75k%9DKepd_2SLqqvtJAAFq{X|?>)cRyU4=$f9jZdt(WiIjO(Ox4_=DY?tPfJ zylgJ92H(Rh1NWJ9-B@=b)xm|JAbVEE1^GeOXDA9Z{^%JuE@*F;XVgh6I2?2k(M zL(Q8%h`meGT+-eMzvrplcz#8(RtE?#IfRwgg61~uCYI3smxQ!Za>Sg%I@rsvI4d26>yH9)VJtKODQScx;0t5 zkW4^<8K5zy@^B81_?B}1&IN0SwHgX>&AIq1ylPJ|-?rGE`<|ohwc^tH$}OjGC=rQ1 z1w$nfHivNQPZ-CT{2htYFCDZtcaMjv=t)6!b)RBYHH4d;&NR@oi?EMn!8j40Ecxfv z99kWHpyDMV+z@^0yl>EkKJAzK4hGU$Mp3nn^tj_4`LVr%G%NWLolVDia;yjk;06Jl z5*vbN=qvD&vmIx-Kg!bjlN98Htw#+v#0azukE;)5=ng+saxs?_5E}um$!sGTGp<|p%ysMUN(M!Q_Xe`E<5G4M|#&Ty5U0sT> z&-&E;sEeOjx92nbFvEU4w2M{+2SF+8mfG%ZEd)k#^B8UwqC^)g?Vhq(^2NlK0ctt; zFSU%`(Hr6vSb!fVZ;boiUvHS%z8>w%_qZ5*Iubq|kAqIpohEqp`^5EeXKMGd6{6-I z7FlXIwB-31W$erBhM9+TnyA#$h~#_wK;(!irN!j=4n)Lbm?OUKzr_k@l8Yw@QQSqX-%XyGAYnG3^N=47m=$m0tXl4~qlr@Lw zKGPb5Gn0)!5;bMO9=N>SG>u)^uuDuGZF=A!!O-d$p7KCS1Vef4mA$3`IB*fat|~)4?Ewn?Fq(pr6p*5wJaL;I=EVFGa)&64_7;E zR$ywU6h--s2r|!J4y`;p$%Kl+$s1^UEA$b(#t$xkSoSD4S0q^myKzY3n4r$jpigO> z!m!ID#_vp-dM+&gaL`I(Ss&U|DIsBO4^aW1z@9*q@!wpd5;0rf-o7VklUx`YylZ>U zrVm5nXlEYmh8=C0D zwQyTHP@iLR+9U%CdV6$P1o$o_Kgw{Qv4+{0+8!_8Jq(J>GHe7?d;b4IGc}ak?Vb`$ zZfiP@Jichw#$gSwE8F^p0j_IWq2SVSyqvaZbl`a@4(|Q^j}CgOVfKUO==?ewkTLN& zak<>%h$}gyV$y>VpbQ=>enjvh2&46=2eNaI+LC``ZdK>Q?9bGkg?QJBwK~t2ocRhG zk~Ih` z8MB0ty=Knm$%^m>b#;D8kA$j_a^?@!+(hXLN6rRi5f`N(YKmEkarB6K5qfsXPW|H# z^W{Z7laaE6vfD-9q=H=)Z{7b2;>RuSs(M0up9L#01;G=gdLM)o7w7-(NX2BPe3(LH zec&LqNJk=>A`wAK)=$X}Az)SdbW}{JzNW?H;y%bLpN~uQTR#(z3gH0)-9Iw>FC)-#)a7mESN*%fkpi=gEFE<_I8#O}N%WOF7_Q-iYg;5lG=N30 zB+YDO{INh0SA*%aoA$OPX;IBHm#P7U1aYnovlkZ)Alt zU(UgMx14;S>%lxBCLN(?9~PS_Kqhx$LV@Xb2k9VDj91{aB#f%=jB3)`e=+|nOF5-afU02B%B~j46<3;G+ zVZWT*CH-_z78x1!FvYu|7#-7}{8>_1_1a{gz$gRz$F2Jw!Dfg%ya(||2r1XcxfGvU zWK=_*8-0590Kd?c2A^1hhGVIf_9(#RRI$yz*@W12#?zql{b2IE>Gh6v4D{ymI&j@S z3fe^|n+D~Gf~IebMcc$g-g{@<(13{WMKily5jI_aU%`-?m(1kPh?Gq%pv=eg@%L-Z+M@ zT}`ECXtg~RBg!y6XW0G?;It;Cd^Ur@Ufbe_`x1xYGUOf66I1A&j>;ZNb zFwj{I{r{%pU*yAG3i1?4x z0BxC9e{Tl>4dx#-GN8>M`8WChjYbBveOCUSzzrPP zcO{I4$3iV=5=0A+)GDG`0{&U~!D2X@L*_dUve3I5GP-fqe0Emw!){nH-`h=0g$q^k zNpS_NGf_!k0szb4=DM1gL0rrrnI?kpg?H?`u=(_09j!avu)nDCCPxhyNfjj*@ZpHbGITCUNWt*jsYS<~(MQ^1jiKYBd7woGd02FJK8 zTJVjB8s$y{w_qIAbA<(E{m?Z3=^7MhWi-E74nA{p#d9>$*X0Brc5W%G%`46H7q=h4 z-_Bm7&9v^qMPKgM*>+{dK)tamX&RX2r#CETh83pF{-75DKNx^u2{`zq1)<9&|GxaL z2L{6AIti1H5f%%@gN9=PYyGp}=Ovn9*^Uy%UuHUhb-c3kuU0!m68h4aXH@8cONstK z=m0-_Y?FOdyJ6qulFRRtPBPbU}N-*a;$)(paU!5KO@-x zZ@S$Ex-Cuoe|Ed~|NCi&zy$y2x$*wLJ`Fy46eY%BxO9S%EGq&V9 ziGwo(;y!KsGs+b(fLI9({0E-x>A#zV>iGfu2Z&oM}OL;;FlMW`j@R zO|>*!rxEmt^~d#^J<<^2GtpF$RrfvbYg7uW3`80C&q}2sdrnoZ2JCbbZB8KyRZz&4 z(0auM+n~Oi?YjJW_1wIgm zl5_cahUl>sqLDTirA^-B^WOVzq{uR~#KoT1en<5e>&tY9)Wn)FP$=4x74)|bSYZHN zLmSFwt!Z#tHP>SSJ{&w3J3syr2)MsU4K4B=TjH8kXBy9*JOKj+>!g6_u*@3!bHR5u*v;% zQ8D8&mt$V+|(l+g0w-1O+LQNVAtTzlC{(y)B^ZgDr2KLpZrQO9USkE-l zi$@Xvo8s8Zv7kN5ONonfEdik)VZ->}ttps)WW5FP7R1#L>bifM&F&tX4}zr0U!oCq)c-cgWy{Hd;%dKlm@SQnHmAMzl}SmsUc_%%6wV}4RO1$rVp!8cjT;+TF5 zv7bk+p08=MZl9dJ-}wF;faE>|OG^;QOG@iAa8D`yOo2S(PW`Qe@h~+F|1!V$0~MDA z3zTY3pPKt~h6hvn(et-9E`(+iLoglZpQkT{mNGWL1uZT8mR`Tv#jh`3UyLztgLjFp z_pFREVp4gxw204p8LRYVZ|vF4>s*xQrWOg=WZKVOWy$%PsEcB*m8Pqcy6ryYj55u~ zXMqtKW%;wOiAT^Maw~EuX7Tm z(Qw>qUfz`neV=@4fKy%&E$3%7xS+y*mQdqKl5}V{GMllGX6DiMx^flh2+QOPwe&`3Kv?M#Kh6L>bk-( zTn^(hpIWTo;QL!)HYz=k_;IBU)hoZ$MsO2; z1XrYLmr~%pt#u2rS3i$x23{KeY8V)cW8=8)oq@$V_>z>TzEl9DfsAAQHU{lZ+Iyn8 zhk)SH2ALrBFzuDE^(jl`$v>(P< zv~D`|sNGK{C3xfq0y9?&RarZD1Xk^>q!3qOV-!;9Wxo3~JX%Q8Ey>&lgG|5=1lQzT zu&Qd(UcOi2g!V6BeeUzi$DpXt$k(UN4-HF58hlVaFt{l@w87HDHt=PJG$r{Q-fxIK zm}j=F+32CJEP?KCN+|rj6QEb&oT_^<+>&1V-uAiN*NfGJaH`4tg=V{VxX-yu<70%9 zUTS{{P1KR5tsbDnqg{;tG^W~$5A|ePDxI}{t1Lfos}t5kVpakk--O+VtZB86Ms`KJ zjO09`O_OL}ZDHy=9-ivTEJzzhCDt&^M(#y-1ErB6vQBv9_rPjT?c8F~K9;yf=-HB# z8BUwAuk~92>VeR_ZJhv>9@;cX7s@fWo{^NZxQxx#*rDQJN}=*-LvGPLhBjnAsv zdO2W6*m7t9QT2U?T%D~i&R<_$?Ca0XB^W>`XXkjE(8M!8 z&X*%W=w<#yMb_A!j{=E`m#+e#Nc-&UMs z;$W_devz(6+y}!|_?DZZ1(`6-Q)&?CvDS_yNteB0r8tP79=EJWfec zS4tY=V@Qur7c3fc8Vs{F&=v@QHi-46m4UkB@~EdX-0hz;9S92gJ_!ahQST|zgP7(j zi+2aZryYn@#RzV31|B##=yVCUoi3Q7#?Ib{6M)F$C#@P_V^$|=Bc-;(dn91%hSQaT zpj!|+5h3u#-g01Tn&4+>-)jY2yu&)$NQpz+$9}b2xml_ zuW`FpXqpujK0-w&x|}=ZYU5S>*Uw1@mLueyBgnjT!Hen={y2DVjT2Cb{GS+*ueY1~ z9hZ$seVr|3{Z*)sipzttn32CaNBjiP`LWnvl+3Zkv;^4i4Giqk_BDK1vfZd%bGFyA z9mcY6;*Aht%%U15S>~1N#$w5Rv(0*(lEWPuKVqp1cY3Ml-2a~E;jY=8-Lmx`pKh43t|7EZbdQ)d z8bgEE+Vprmn7olMYanms!*@1V3B%inW>{!Kp;LJ9Kq7Wh3Qh-| zQEpWwqanSYTwVCYi$3PfvH3u&rnoE|Px3b%kPgVu)8uGllqeE)HWJEvf~E6L?RQ?l zGUytO{Hm%fbr{s9CviNc>PHDm{#)SxLwV9iZ7?6c9e%Hmr1ee-l>M1p%KG*L2NOve zjhHop_#i@Cw4znt(WW65qKx5*zKCAVn4$A-K{8)l0wsmG{Z^`%+pMzg3y!Xa{u#GS zCnBJZ?<^++{_*{39VZ@ytVi|B4B4O71Tm;VWg7usiyaz16eTqyVaCRg`~jlujq1@B zL;u%zDrE`Q%i!RU&8I&r)D+#}k6lJORyfu6>lM)VpU*#9yp_dSMr<&|9R}?Qpj8}@ zWQ}^OI}?npRi4{NMMS6$dl_t{bnn*8Ui36<#FquAG(&jeh!EUK zWH=MH!FynX@9I*lGI2i;%4M;27SM8WcU}Nd*n)6aJ8-a4a&oPAnT2@l??x|So;s~Q z)S!UG|BJ?dYZZN@`b5=n#oKPzO0arFkZfs;RWL=Oy1YRf>{42QDvJa)F5m|>g4s)A zmDm|&Cfw8Ud9x^7G1y~-W7W&XL)+`Uy$s$}7*uYmEH_@UC70!FU4Iai@Q}>2)Vz;N zqAUBs&aGw`M?fUhL)h`GC%5ovadrGYQZFD#<3UmGaM>>ID5o=(pz69v7<5WX<&(|B zD=afPRNNO!$?WRq<5_ab!_{ysmtSFh`fFA-NG61!AanhAL+^mKd3c?XobvYlbh|wc zF9x2=YUqKC$g*JzOv;yll{ru*PsJ1=coiz5 zNJoHu-}Bhv;@BL<(5+~{YVp?O?M9RZ&~%o^ITdJga>P1QfMR{7yZVj6M5OkJPri6L zxFES={k!BpUn6iq7EHK$3Uu?K*aYt~qCCXA*J|!EA_OTr^zeSx^l|9ra__=sy7O$g zd6YzG`BXeUCK+gy8W5VI^SdP=6ZzP8+-kIb2|Z|sB}snIrXh5*;(RQ|{-|({N0|zH7z5L0e&1`7n<%G_X*(Xcvn87YO zkNiGs(B;vnPdhB|df~d0X^-CPXgIzV=6!v6TDRYrM|j;u?;VD3_HRWT1CW>!l2t<2 zI2tJT=xGdb$ejm1ou0qJD7WTiaLg?{_e@gpur&Gkv#=#_k=}{HgI7-_&ST!KoAr1r znxT5@_0+~meW4~-3a-rKO1r(L49yw36%pk%ZW!pvfM;W$R`mu3pqQlC{E^8ws4S z*0|-(SwmtlZ_PEN7ig^$4p7^x+Iwq=iOg|){OQ6?^V7nP9`8CYw>jn6bRc-$jgl4% ze7GA%w1JI&>?gheTC7%_P~G)ofSi0%E7cGK6!sZO7}Wp@TJ>oz(n_*N8Sk92fsD3HRrZ zK-cx&1*rDr$QwbO>x4Dg6aq#}F#d;i!`a%Rewz2y)Y=;BrOivXDFg4z%t0Kvt}U7q z(e)Is?Sd+a(^-cFPoz>c^}U{S$nk1I%cZ{aR3_EkM(`vu5_E;2Vlq4tgh@E8#%UbZ z#AIEx5rV=t_N}zSi|P-1@h1(uC$zw&sdDsQq(z>Oqopo($}aZn_G>K)Mb~%Acxw^Q zb4>)dtW)m(%~`;4nF4rxA5?cb5BL%neeAuT$lbBhU2|G{)U@ABRU>~-ZQajZe1FZ* z3JZY70k|>Esh!%CZLlRABGPE5P=pKFbvKf;s&L=L@{JH5)P$5Y(ez!TmY>*H&i$_)HC^tU>R&{L(yl2 zeM>_3_(26msD-RbpT^}f&jE6*#ze0UszcWEeRdg@*rSx{t9Z+vId9An6B5AssZ&*^ z5ANC`SHS&v>P7x(e^TD(?ZVfXIms1v2_ZgA!yy>dAU4>I0H|Liie0)788MmGVoiQa zPHM}f>x9+rYzC(reAOmYSpsK%UwYI}gSY#|6GJixDoVVYdcpGuVtVI=1X_2p`VC5} z0(zRl=Qj;oT(>&ur4lq5%&+-^40+whm3J-xX#^@R{R+xY(??HwU#-vBG|jDs@HQO1 zTTc&c*EvA6UGHC>&MQ0w$;dbC&N{&6N=U!lmBda?uT0Q)4^9Y?h=28FW;$t$ClI(v zZ~@wHx^8s86JQ<|K`4dtN!xEY)o-1K(QsFq6NuMG$NYFMg(zw~=d7O?RRKNR>5-K} zBQT!1>@PN<5BoGVYJoaX64}+CiI<1W1~~ct(UX6y5=Z)d0D6PMg`IHwN(k3BPUF49 zdHrje4+WtK2hZcEs!?#mh&ttzT{lE=d1jLfVB%HAqQeJU3tTQg>j2y*0iTuo)>Brw--?`@$zt+{&wTT`>Qw4|eMUJVYjBFPFi4 z*psaAphEhcP+KH?3Vot`FwZ2a#1#a$oh+4gsu>6B9hXKftgyCMw7 zxJVOayNW<)jgcOnH~Evc!g~jT9%=-B%3rZT2|Q*Ifza(_PqeO@dc$~{*5Z0Ymr|aM znc6ORUg$^*?Fq+Wdm*j*7M5V(Ijlp(nbIEu#ANP=sI#QJqLWIWNYVHo^9wWwY0lF* zgTvI8m~dI~wCmmCLH~@l_1$yTK~Z@$8VDL>m%$MZQzO7FpZBvp^C3@@{k*;}(>0%H z9#$j{I=S%V+$gN`|WF~EgOkz{8x#mN|cbjgwLEx zN$-jGf27lrmvS~)YoYV=$Hw&3GcXfcpUW*WDpfd?fBmTWRB7UqVhPTH!wG90=f(4DHq1*XUvUzhxj6CB%xV#6kGvq)urZ;YU!TN@Wv4!GCNsk;jxqZB$ zGm-kx*Qj1=hMf=+f7w>xgAQ^MAc7p2K8Ssg_Z+D!ZkM0(HJa3p;AdzMAlUd7;XmA= zxx={KV`glx_A~G4>H*Fn6=ZS^{gX2jw*&H=s=R=I)7yXla3+il!6!=>e#i}s<=H>lx=q`wcmI&EKjUsKK%{~Km<*mpgw<{mP){3 z@D_-x=g_8|^YSr{AUAf~HFv)ZQu@%>%Pw=W`n9C#bSuSeo?!UjCz$NzsiA!t5?GRUi_Y7QVBP4z2M0v3$K@mVK7Ow8QJZL` z?pDgzZz1&Qet3uqMyTwoLzjxsz*heL8_ypJ*{`3Nr=|QdBXucDYP<-Bvc!r)*gmL; zX1XN@_>yYDV+T0$@-1yBcs1}qUvrr)3gPLEH*+7To8X6)jre*;ug>+!?mtr-bm^>O z3JAB@ltC3%z3-aHus{t!nTxPEp?U`0x!tc{Hcx%?AhHV@aZ#4RTiYJqz{Y^A-`O}t zd>zTw1LmD;#r-?#Dy~BW5ie+g?X|SGGA+S|ZwCB%wzC!%gX-&tkj-tj5B#Bl*;kui z9zx7lb~Mn1aGo_Na;?I!W~hcyTWlfFMDyq~SUTT|*GmmYksS$ERI;hj=E0Gu7kG_4 z?&aj36Kl6R8#jS`U^j#Yu;bjk|dH_Ygh!rA@iqsaIy>Ph;N&yU*i`0hyKOht6T2xn$b zxJw#IjVOki%ul#rt+f77P4T7LmTM%p`|7T3Q7aMJIv?@n^t&iTLEwVtwz(Fn_T9Vx zHR0*iG8v=A^TKyx8c&QX?Bo|1u4~e|N$epvb(@)+%pFW3P~n-`c_Nyd?V__1{@cC>-yTBdtv}c` z+TWnweldO}lKT-#d@^-toZb`yZFNaykzj)8DuC-k%0uAFSWd!u#x^>v^LIh^j;H}M z27dml9!`(f#;^{U@WS@G3~^mgCZgg{zDE(zuw>ZJ<>q9iFp%t;UA6j=a`u_iJ*fo! z+h4n3MMb+(5efJZ`ufZB^o9tUL(R{0k~TYE-eg`fLS*;noP0=WNBbUdkE5Uda5d_D ztc3(DOLy)VYGe;fl*8i_vuUt2il=h(-N;^#*gfCl5v8Rij%Q4JMUf*C)96zlFf-yF z*C`zc+laTl;jyog^`Kb2+9L(&z0^ek`(@E<^&*nK3iqLt7o!k_+;-5s^H_x7|5)mf z<%FPF4oRsgiuK}Y>)eN}M`0-;#Y5CHb;pdlBEVVQf$%V&2N0~9qIEZlK`WRXKgJwx zfsQ4!mG}_i`XaEfLcl(%toLB+2g_^TlhBpxx;4lUd_HywJR{~0mLz-p=B%TcM_$^7 zhZ?kvPb`(I+@~IC#uZRtCK@E-2c2OvBrr{^#1U)ObTxU4;*s|qMGu@hA&gZg1#OQl zinnVxGCk#w)vJR%S6(b#atRp`CIs1&pmt=cO8LE>Q*C@|PgrOHHrn#aT*k}>oH6yX zq$z8&9gnv49Hmi(>_x7a?7ChZf#Y7MA22>-BJ?y9=RZPOlV`}x$*~Ye2b^^iCq+0V zM!N@2GXXh(7IVX?IvLC?2jd`)CL5w?KrXHf9Wa5;eNoPyDAr*%i#k;ls^#Wr!6f^Q zjv+UNb5IZz%R;7qUzNY<{&unSXZ+z4gPF<3eB_xv`GL=^y&qLU(=18Q2e{4XBa{hraZpM44*M zJhOYFmaMWmzsuM!S@(levY>Cf9tR;q42^+D;1ZNe5gzJ&eLySAjycO8z!Kuoz@k^a z)=VD^B~x}Ha-bpetlb2>vjS~qduc3^*Q13Q$yq252dNu4 zBmn@&9z+&`s>S}g+prDx$;r>}J6A%)Y^Z;4 z3v>8G-|eYz8nicnXBf5tmIDFxdyj?Fpq*P2+&&t*Nd2Y;3SI*X_3ta`c=dh~01@IY zTknZ3RBEkktC{w%eL{}lu**yl>q!*Q_y;h9Ui?nbb5xv#&sx9+7;%nR!CUDDA9k5E zdtWRu+~UIia0D&X5MT%xE-)@B{_;i1#6fs*o^GaHTP0FG%oAmZhNQYu1I}cx8sD^g zM&M}cYtI*eVVuv|N{s5Bs5Ua|CU$#1Zkey}I-3^Ev@y@huI|X)hA&#e=zsetC=S@q zH};AyFv(+E+Y&jRY`x^AINO^)LDV_H0S+~SVe||@7JJh+&SKQb7Jdd)=A(=C6cOdH z=>n^_9)#l$jVSrsy3zM)-upv_nwur2&h@M_(XT#3*oI+#kT}azys;ehmsi#T?BlRc zjqZr$;nqN%sd+l2s?BUUYd&x~5&qDpO5l}{%T{b*%mh*IV7su*S&_O>-KEF&WKq7} zHEDMwUB4-4q27hNy?LQ;96XdD*p!Bpcrk)>Uv6 zq4Mrls^Oz=aM4p%XN20b^~R>$Mc3&XQJuxMz^3h0K|5SB#LFv zy&YAcWO?4|&(aaH;7ID5j<5VEx^1%C@%HkG6tkbBY#A(_fayd{m6(Pr03q-Kte*XG z>aIR7U4ZfjAk=~UENc3xv2hAPgn{@hz{df*yppH~CfVC=3m%uio*v<3m2M1TA9gk7 zW1Y)y+RWo|RT$i75^qL^oP|Z~BhAuPH98&Tv9z?hL~^J3CJ&LWmja(RO!t zwY@<7i@)IPnyC)x&B!fosi&RE00>qs8b38DKr>73OwT9e3p>#A6%aYTV$NjVHi znr?(fWh%wCJCgYdd9jLEVLGW8W}v;)dGqGYxDMPzeW!YIa`mL*QJs&8vGMMm_h9gh zV2$0p^WPl3rU8Hy9v|?5LtSJd<1p+8P8!tO%IZ2RM33hzr{G?VI+m_liqOne*oOdH z5py#Wrdow@H*C4ji7l%0f0Gl-Wie3TpfzRSbQ+`6;8%nIHw;JOw)(oS$4XGfN$TN_)i6RlG+qv&T%r<{7ZB%80^lGL@nf zlnnvQB5>UvH6s(BJN!!?Agre2@|u$K4PL;`ZBAaEmQR{3#6cl6zsgBTsUgR@RIkM# z|7E29*vqG0-I2TVb3;QzK}dafv$6+$m9C$(YAmG`o5GBl--DYXkVs@*lZuB*{)B?GsSJcIni+SFE)wxkB6z3^%7-!$?bvaT~cNfx_CKvwUby==p>1(&Wny z4=uVxRi~@0caU;FupfQg~#2Uxh1V9F~!t-D&uErkzhzrTUcAY0woRUh3PD zBJMsccZTER(G2IS5b9NF;!GlREOOQK4&Qe{$cK_5(O%C zDps?XgfPG}HP+YnSUX%(rQqYJ(=T7RV%a#;z#5pqHELPO)lAXZ2Vd#fcU|pYBsNJ6 zJJW_Pi=)`i@bLLw6=S`9tj$wOu=p%4h!8i(<51_&aGI4UZRxPNx4sbOH_7 z9r~W1HqGmC3?Cjj&N`)PDXdCNj6eS`C@9d%qMd$Rt-+MbY_-qAN5ht75)Up90S{5f zZy_(f{)KDg`KNj>+PqwQ&F=-7+q%Esdtv|9BVLpm&W`m+?RF$||M^J$_S-f4|9r^j z7DjywLiGcntY4t&kQ@vE+59)u8R+ldaJk{sKZx@`&;1852*6o+y}gNIvQ8?b&sA`M z4*i2E#?~+eD~}>)HsN)nwXx&7)@y&Vt@v|wp21rb)Y- z$+-MR4vVe*BdqV6Y*d2&A6{eglJpG?MCQ>2awcrTdwtDbEj1RHh9P`lD-ZueIsovF z2MGQv7~s#Z|GYQ)|3SuY`HS^0OdR;*{w*{90Lo$TqvGu+xGu(eH27#~#oPCuOu2-2 zGJ_}^txKv+O=V}k+wP`b)!3doO!+EY5^oI^^k2(!|MTseU(S+6XlPH(B~#4l{bZ%X zld2=PS#E1n|7p$J5I5bxYF4x>L8q@YI6s@PmgGD$R_Ak>?W`4WN+RG?lDcb}mr&j8 z)$IFkG{^$vZ0?czXTiWqY&%RKN6+-52(jUtU^m$M-G3h}4TTl3?owIaox}ZHA*ZK$ zb|N#}ysuw3_j%^A7wbRN!Hc(beVwHKdXS}1;&HB{w|8>*yUR6f86Qz#n%oZZZK3RY z0lBbe){9SnVr<-d!WY@iZ(hD;K9ITxfT92Xgwd4C%a@$lgrn58B$U~)I=<8{#_;9o zU*CAe-k(~C_5Bl`p@Vt*iJvZWQQyB8>e~(M+@t_V;y>aJ)bS3*0ori+Bh!CIZO$J> zxQt#);y~8}Ge2~?tucMeIYxRv8uTgu+qv*(lmq}O5omL`wM4hLQ~_x>+Bv;zK{+`p>Lx37Z#CmkG*|C`+ZZ|4CSBo;px@Ho28TR1sUA4+sS z$-J>G6m6&-XmR`?X?QhIb1cK3!rxqcy>>L?L4R1i>3eueBtGPHNdz(h~$!d_sy z9)Hv+4yyO9f&xj7Z0MBZt2+yCOwHFH9EDZ^K(@w9!4&}jXX~8^yh?l(AE1qr@0u3u znD>cQncw_TUu@K1&vUlw^D=C?9`5Sa`eSl87xp7{zBxEW_9S4`YI`fOxp4E7TfF!6 zr{0^R#fGzH(fz@??eD~?La%88cNi&tH7Kpo8M@MO`i`O^SojxYKSLv+1+VP9YF7|u z@k(68e0x{jHTAyN&8K}eVk)9y`$f*3J$S-9vq%h#k(Jm%VFdAUAT@kYT{@T%By6%g~qX>2QFLlkPVP z(vW?Nqd!t!_YJGh6dQB>-(1O*ahQiIo~H9eAOI5ARwKm*#(n%*$u;A!4^tT&&V~K?Xg?BqqAvZn+L;$Gt%?+_a7b`Cd)=uJyKouEs!D z?Juu(7p`|rZ%#~s{^)^+suYP4%2uAw5l6*=xvmVdbog~9?EEtyZnlqn&xAF5(Tv`^ zzpmG1e=uyM<6(YyILu_~;c0L@lG+Xqz8LCghjrNDrx#iMQWF1=7m_Vl=!^OY?#R06 zIiDfY$;HLhpi|(NY!n0=ACHU(!`aaw#ap}Ww_|-?=l1%XTi7e}aAUN;fQWEwUbWdf z#-(9QHtswT8iW!+g%m930_yf+rYA}X^5jrH2Kp71x0;)6@dzYwvCGLuSL)0-hOVJ% zdY#w&_76W4Vh~Oyx5Hru42*a9R#l?4yuG(|Yt8|bA0%>;vkXrAjQ}J z5TEI0@$fsyJzdqfFL*e6Xyffag5ach+rEAmtP8f||I4Y)_eDr&D#j$0VeiV}kZR_v zc~yz8-|shLxf#F&>82f4wqc#AmLJ+pPk-~4yW~vm%zYXBXIfFN{h-ay{ys2ND{1ib z7%h&@ScVq$E*N@=gRN8{n2)Rr|6~B(v#7VLbSeR&>2eqZVcqB;vZrboRC{XGCif4#zf9|oo4Bx%fkUpM#qCM?fefBuE*~fyo}GwmrSsO%*dqsEWkJADm$-v(vsCMkJbP{*k{W`k+k#mS{Egk z3;Okfdoqg)S)JcA?jf59s?*1l?9IJ%CsuY0i&TMWyTF1jlSA5V8W2FsR z4>3erjR}JBC`@mQmf0>KzE>}bL2wVx6Ni^(O>5V$xx7D8CDe~T-{&QW&+-VRu=YBU zhF}{wWeht+YIf@)PW^juiuoqwj+&nxlgIy&f}_JLj8>cNGVb(s|Bp@a*uk7NuwC`t z1&~CK(yokrzU3};MlkZ`X0V#o3+Mb<`}i1W$2dFUf655+GT4FTo29?Z(IW}zfGB#W z77_cv;g2fr^**cPGaP=T8?1qOKc&Oc-;KqKe)LRT2`BvY^xhTrlCD`t*Ve<~w_Ho% zOI9B1^w&Z0gv3$YkT`m8r^f;qI{D?-Y%N08>Uo;GiV2Z5MqgwGFic{SIEf`5R%M)1 zeSYXl8to#gNL2BntI>tTO1$Tp$-NY}-mc$261DF7r~q)8WpSP0dO_x+YtD10vb`)RM_YZTAza>LLf;B5QD!$i;y$~{V4Vmx>RY@U!32hfbb6;bkYU}aQc<-C0%irsj%>)u;fVYxVgKN9h=2SX|71mZv+I3l5l`*-*r zC&V(3WZ?#kav1yOfqNdk#Ly(Unsh2tEa3sQ8F@SYBK!jp zGpexd_$NJ=8nH!M;=x_h0~#qz57a;!MIFz`1g4df?c$r4eXqXp3?1o;tuUU`*=5gb zJ#z9~S^SNwKl)yGC2U?ctzW45@>?BkqMR#Iy^GrH&w>y4^*W0|oxx1@xB{WsGQ4_h z_PgwH2KKf~gCL(MGDpC~_^GHH5ITRydvnEZJIBn;fB& zZ!}REMg3_}!zXJeCxb-g3>fmiAOdcPgOo6DtzZY5d{7W7xdVFsDnR^D>OZfxguL@+DuZY=M zsKvb%=yxo>F^)kX#Opl-8+WNWlKKO_)Qzi^Rta4=y^0}IF!{Q0Nj1zDL$~_|_LfwZ zng8tW8AR5nT753k)-QbNSCRnZG%y;YL~UC5h2$%yO!3Uajby{aSS<+oFWs=Pvh`vL zv-I4Fba{uk(l{<1=~6@EZ=*3|Uq$R`84Yu?-K*n{x#nfveObq)B`huTq8-QI-7>TW z7qYi^xnK$@uXuHRoL$VXwm;*39A>n%;lZ?3wg`T+?(1G*c(QOcD0;CyWiEkL^Opq( ztovm0e+`cfN{#mWodg}PI|)MPRML$79Dc95DI%E{b<9Z-uRO`2qqrG;VO?pF9B2j4 zuANg{q*=I5<=sdEH^))FC`8353kF^H@OuP?K`#9IhYag$iKo&iWi-?GYV_~&9!3v; zYlm5}8M6m451KTw1VTTWvj#+AZ5>AjWMEA6S`xB$hh!+8C*p1yhGGt7^Nz13oKdi7CitC5R& zMu98vY^xQfei^A`kM>O9I#l2MpaN<7`sLgAj%9xD>hk|ghbK3=7P?a2+vj(U4`@KZ zOeo}7s?@xOx4N|}-J8q9NIV*uBOnTh&lV#REm%)F3I590kWoXt_7mK}^3G&GjjYEC ztx*gj3TbvZaC-U{27-XSJW!~ib^jj9HXmDc6F6%2Ulpc_duw9>RyU9#Tk1h1`AQ9t zJ$yfJfSEe|xKZ*QrgL`l_2Utu5!|Jw3^TBseLac2`r)A9QOeKxWEC07(ct0NjfifT zx>w|nnp++RZuWl{nl&zX(80FVHa+pcIVmH)ZwcI_fj*?R3&`C2gNkZ5bvL%_uA_jL3$$Q+-i@yw~9Dge2K%v^!UY(tHqDdaNAW-qr&JUReSDnSF7WF zPNMjT3%dzBhF?FQ@RQkC$R^{hivnB~h0s%zoAZT3#|UTLVB2>Y27d`Ik_bZnG1Q*) z{N;2-Ku-uI@n7gPz6*nnX=&r(XMY)FZo+uo!xkgj_Wt<*a|E!-Sh-{^C}?kRY`Oh$ z$eW!bkEpS=E3WfNn6}TO>+?i+Au|_x+pnF(l~uM}e$S&7(T^;0h?pNnQ4LpSoa<}h z)5iR=UYNWODxhJpq8apk4HsDnugaYvc&=vxTk+94m-r>68`4KyOj{%k(XDcApc*$e%PcS5LBb9U{P zBtjTmGIk=ah;`tEheHA>{^Vb?-$Jf6lw~y+uyD@QvjWERQwhf%VG|D8!{UUHx8;V) zS6GYlAD8Ckh!K4oxTN~l;KLZlMyKGlRB`l3dyz-vk8+@fU=#Dt$=v(4{#?EZgQ32? zJ)A$j1J=lzK)oP)Mh!{*4fuusI0Q!>U;gn8xgV;EYDbPm9|)IoVQ)a$1S!<{^j6Uu;yC;n%Hi{_$u5PycVHp%@LqM$Sui zMYzKm2=y>wxPF+SA$qZ!p~!+pqbN+u!mY&JA@)4>xGE=fvG6EedevMSA&iojj``Uk z_(&gxhSEK{|2)p-E_O$+;^VSMAqgNec3oT+ZO@4|SjYTYpw#9npozsPz@tNZ!Q80N z)(UZLdOJGFR7TV~^q}_CaFQ?^tid%wP29=%LV%s|Bc^6Y@^HEGn^l!3OD9b3Eh0;Rc98;a7|$Nw@H}uLW}pUv!~R`^ z{;WXi4#0;B!P`6aNxv|7`xe2ig!sKLtYv5Jb1&a0QlksT=pz~(0!Zl?=ja(sf9Ww= z?F32MoqN0-gDjD!Em9^Cf_lK2J_mzXQ_8T=hU^bkr`O|HPA>s&rbWlHslb0diI1cM zkyYnie8tr@`pgq5Kvt>oJ6%HV-$T`|2ZnJS2~vOM zQlgEwlAMRauU|zze3jP5ENs4&{gK8sf*8WarSZZXS)NlSqL1=JCK)A0JwiI%kwn*3 z=rOQD&BZ>r;{vd&ffzbOD45>MiV%Jit*x$|#jVFqHUt>oMryGLha&M3 zk0_~pw`3wMQOTVDF2;Y%QIREJF&5sGKY8WhGIkxy3aCRHG|tq=$8S2e?sL`TTQ1zS z?AV?nCUOsTo&N~=7=^n&^a#pt&U}+qvu-;)cJw_0ik^&_g-<`sw9_qJ4OpI9&tj>w zr*wCywb)gzy)#(X#g3!UysHj`We8>QzQNQ?OdesE=;)Jz++5GBO5^N*>pf!Ay>^N+ zd1ue~$_nsor|ry0nW$c-&*dDuKRzGjn52cr zn;y~2UkB)O2#kqX$52#RwDZmMNCiPzIT~n?>V3M9*VX}D?&QF(02%S1B87WLTsldc0kZE#`kYVhV~?Va54LxL zGE;GcP5TrUEK7}WyG)pZIux4jm{y{-AxFO!qDm^wDC#(%P7b<9_ohZ0`;JOJ)9|o@ zVWh+zn3&|moU${0M5(m^QJI~`fU>1ilyyG10!upw;quDfPeRbmE z_AbCuabHz^DI3R~=1MdBi96`(AS?Lc0=WmPdGv7`qphgRB%~%Oxrjni&^+fO_1hFn z3VU|H59krzdSoIiSR_5;vBjKqjc5|uQ|Gd>&(zrYJ3=XR7)?g3*ggzQRxq&(jF}kD z2T;y5rTT5Uk=w*DVCgL{8Vs*u)k^&6nUOK@Z?xKPoESWp&2_g#S_!_0u>1&MZXfrp zz@PG--@Wy;0Z$@=x+$?E2<%zv&Xrh=yw|eMOEyvJCp?k(F1Bi)A$E2&kqY%Wv?WT% zL2vKW-|zwdXSK^hQs>`uB7+9D?v5Sj62MmHxyMSG^|{AZHRO$XQIKtue2M+WB}_T~ zT+gyT0!m}jyt~~7ivV^_{;>pZ=Zr3})x83yq=A*oH-)(xtZ9ExMhy%lX;mICJ|RK( zoRDY9Bb*}prbhbCawezA^Bz?2`TJ7(La}=wFn&kCM!gU)Ot`fiyxNbB78f62RZQ|Y z=nim&K);zw@R~xbxRg^>m%)dVq@qdmM$()P!=4Ai>0$WXY6L6KN?ic^{B?I;XxjFT z+GXjWlX)K}hU+{*kg&ef;rDgs)6h;KQhA!M9Gtx1aZhs`{Td*QopuS0q?UiK+f zk_Z(%U$^M^txE`++Gk`vaFQoS1rtBU2049ZeiI>pO5q=fH1S)Uh)C*6I+$ZUI zZ1wO+dumc9C!ktM_xMISLYZ;7C!?S)RV34Xg>np^q;=FVEa>JhM@jA6g&1^ z;vru3FJoZ;e;EeOF;kg(BK z5p}jIEx!s%=+}8m63Awi%00%f-!TEjCuEhD?8_#kACS^O_L+hDsS-T6>>*BkI=Ka%Fm7M~_0)ZO2kyErlEBvr{H%#GQi{FRDYN zP9Y8_SU;ZJV5HiILUSLTcq#mAsr~U93J^jNKF-c!ya5Zat}V8{&U?N!PEjd0t{2t= zcVum6-ICSXGJ+>wUI5&+;0L!!&P>HIf=GMUmC>N2QMm1X0RoaLw8keFJVf|rda4*? zSac$*i|+WccMRWJFU5oepy(tW;}EP5N@+{W?o0`a;p(p|Bx{_6Qr8M$Yd3q&v%Gox zbF1e`iCrTVDb_>KLH#a{e|RGrdd54^$h(POn!mdQliJb@ya=I-biJw@pAE4desIoc zQ-!p?7MFMiX@y$I%Yhv5&FRKcPbxOn2dKDQOkC*pVC8sH@aF?|t+P1w(<6ghk0)`k zC4V_x-Ap&&q%;I@pk?xFb!=38bEFo_w^ghHpkTlAs(*{uG}I?7B&@zI+Tr4W6*ngX zIs4QPD>jAK$I@Od1l!(FF`Ut-AZUdC&Q`{GApzw8JGPjsH3rP|%@jG7nfY(U9S5TF z5gw6FsW-AP#|EqPPwqI6Jlq)aU6punug`l*f#cApKpw)svmgqGe^m@;&t%)@r4x5+ za38fqoa*L(1!8if$)qGq#pYq$^f+`S)8uKVZXE%PK(-y4+k`s+0bih~ImEU=_>{7S zezRC{R0IuQ$iUnelVFtxU!mdE3@rxU&MSmV;$pJJGW0dC!QO&(4=JsD0U{2C4=V)! z?m}HJjIPDmwPUWR^YqKWWTxKXZRl;olzj0-4WR zmU|11MCrn%9$!WnNxSzYS#qBhfgBL(2q4B|{^U|qcSo|q6ILhAX)Nu2iLicDP2yD3 zrQI&LLb3N5n%qEa>Z$Ps%8OfsLcEds0*wGN68(K>51eSbV%h-5yn=^LME^N7rbC|v z4vjyflJEST#14VBgF6iz8oyhGtNjg$0FHj8?omI4Uvn-Py zD&JRpLy7%(@v<525xFv5ZL1$=nN(jqW`i{iLjz+jItxJ9RtM(kEV?Fw-q5y$p7X)E zo1%5Z{W(;==SWF3xD6XU;Y;6j`Rpu)t|2F?ipIjcgqssZ)m1ioY}04+VF=~>lmi_= zl$JrqUFI^B>w!u!a}2m)YEbNTp!V+i#dbY>LPwpTw*jUOu5-=u87XJ zOKwD7B4umO$bqE6Bp?j=tV3ldPOh)*vS_&X0V<;bDy;)a*}JP)nC?VpE^_;6tM2bj zM5}?TYp3RN47cLn)nGQA?eF`WcwrnH+`nR)CQKKRNXf|-VJ`x5XpB9IsG5|3qi2Yu z^-!ZPCO@})q`u-tbmQAVr*z!&wn?Ur6++J;a8}5yW0^(Dcz*-y3tOILp35;zjtpdd zqm7ltsRM+Hu2RaOe-_Mlb9s`|moLZbcxi{{S+benTxhHEFlU4_M1!f4{+>WsS`*a<@1MEHHGD$dte82GNH?E6@(X%=rXY&ow`M z7VMrymrF{U9Qg`#H}_^e6S~TvgN3A2&mEh|Lr=hVF{XJJd%w%D_T;PGO!&OiZ!NdN zk7K_dA)_RY4RDLtaV-fAG|3-f?^e{cwAAPvpgjtm1X%p~Sih3u~(ECuwZW*PrP$+t{@HqPNMNyue@;)mhuD@TEdx_i*ZZum>&uD;=NAf!B zSGCJfr&Nt14bx4=!QOEsDn?}P$qh;Y)F^?W@10|$+wV%9xvFT zu)^hUgv?0ld-|C{qAFn)e0>4!+R8gMOQr@NZ<-Dp(F! z9a!(l0m!)9RRYkr3plrlVEbVJXw7)tgDCOapa1*=NJGg28W})(rRMyo&x)l_{r2at zzrVT$pwMn_0)GNI|Neo4T~Pf=l?%K^1Nm?PdRF1?0B8z$qBm`~c%=H+i>UnV`t5o% z{f7V@K+atFDDWXh&;kJI>46>-2DRNCj>xm7Z;PH`LEjr zo-z6XD!m`J2J`?x`21-RKHIwf+ICju?_0o^yj5;*-8S)Dew(H1r2nsU&-cQ(Yw)guI(alg%2B5{;cV}zrsCoHmr=`3lq(0sjS!?@9pr%P- z#}bx(zo#+*H1&f{`3Qj6n$IqsE%<^Zo85M6?0)~+xdSS(Mo(&OnXh$Z;S>R4EUtHJ z7J?({oHupWF)v$vo^kH}Zg8VR`hK&EDqYw-1`%BisIQn~v$M0$z08M7X|w_m>BH@k z7NRKSp~;b%c&Dvg%VN`(VhB0OpX;Q;yFU*qak2*sOztI?K2>J&sj}m|nJCZ$G>5Me z_>eay|cm54qLVqVzdy8FvK0DzrOrWZK>;xNR-tLSC@xT1LQ>iGo- zh^~0A3I+MjkHfC4;eC~fi$Hzi%ZYhK(8?E4fg)()8U@HWaP0TW2b#eC$D~(t={I^C zt|vz};sWEa{-t@T5}$Dgc2M8VVr$^wqeh?v_dm2Ap)+Mbu<2j^fD%(`1AH%2_^m3- zbM+Y`GLJDJv6q=liCO8Zco@|hG=qsj##7WOV^3a=G%@|6&7@ z9u!;Q9}L+umh$2PHF_MeWkfr;27$s;vSj-K=_#aBel z@PbWEkm)r_?03SIV+U%u7iTKKTj0NVuMG;%gZjfSZfte9*5-AYs6AP>s<1y!vCZ!+Nag7BoFQ@2BlwZFkRjSA3 zC{o?Ey-J>_mNx`}`JD)~p_@Ofa}gge!-BAsKyKSNCk;&jq=)8DWrw|vkrED!-)tK1 z&lFhIN7KA;Rgj`_%UUCfyZL~Ll(2Uf$e29(y%WB~sZaXpi!LZU^v{LVe=jl)tNb!K zWRwz1mWbNk0Mq-92++HVq;9jZ-wjXK0)->>uL>c1jfbK-OrMrUvsKqj6WJm$>FYC^ zgA}ax^u*`0BjxYncE*a{SRG%Yoj#KXofXlEJwL37abkHAZ?~i4B@a@{$zkw2mb|( zvO|(Lm6zQ6@yY&6?0A%}2I?-(?`y1jf0E#oR$m_B+jfr^@@M~$Tzzxq!6uK{qM~sv zwIXkNMQed^mqj{fwU=!ws?ZWmv(i0FqObRh$aBzo-jVm?N_j7hoS#wE=eQEE^OcF|hY5WutBfHzk;nbo`DJ8l0cHNZ3s`32vEf7yI; zeDuN+b(1CcIw%tY~;{9H~0##-q zWFF(cW}fw~N`VSKyO;AJ?$Gwg(e(N6FZ^O|lQ2ss#iW#EY5Z`hl3q5kV__nWi~hh3 z!9VkG0dl?1`E{F8N#6Wwm8ZnGwOfgdSF+{6({@!7#8|dmQ@;&=V5P~xc1Q`DDmZOM zN{qnVe{_w`Vdf8wz~;;e;}45Zjvz8xdi9VRjIa655ZUI-qD`Yac7##bga9T1|=LaP1-mhe6cP*fhCSY6ey7!Jd&qU0_Sh3Lc;W$RxlMM4!2jVYm#>TU=Q9;{E zrdN$mc8#D~k3Usm6?@ZJ?-xMHdi!WIOyVB)iC#u3^l>_T`FwbDz7lmo=~M(g+w-p< zJ!wc0egD`Vs6Ox8flM?jzQMo{h@MsgN=1rwx$%v|XkX8z5n-epY3@YF^xS(vL9~Ow zlw;VKo)rggh9Kv94_8F8LnNetZvMG4AmrL0id6N3<<4*kWS#g<5HC8reUUpAnF>Do z@W_i#7n{XUDsT?|`>|&@Pg7%J5^Pd~BPPfAHpg|&i>R$wB`h*L9 zJg7S=>H{LE+D%P*-gfAw9ajSWf!KRrgLLiOT1{~Qx_}MsmI0C|AcwY*O?c7iSP{q0+m?Z5= z6nb_vi3C#OO{8+rPo-C?*aID?HhF1XLwYCPRZa7 zgj4{vD*jpC53vJJOs}w3xW%!s*nA;#>fJfJnj`+LfjTF_w_-n_@ht!LtmVpxU`2tF zeKPTRGpcvxholqAS?MYn=Pe=s>Qe9fJB9Mzm=S7kKMO?<5}MkWPY^%&R0c?O0Wb&e zX3&r^88|Cm;cIp*eI%fW!DV82pIv$JG30zGOHS^s;^UKX827k|f9U{)m0OVy)>ZP? zq%VD=rIqBfL9SfB)$BMtgDFJm^*>v`K;vWiWY7v+qzq$;U@B-zB~X$sWkSW55!q)nMv5sB2I?yut-V|3!csNZ7?#TX!ep9$9#6KrrYq=D2`0HDRb)*|1hhCCZlDb zxr9M!r&H^&!q=m?!ve?kK=a38i8ZgHe(G1c{~%bepH;$?I;LIhA^#sD5j@RjxQppg{RV^rV8|2WQVCl1=!r6Vij?tomd3 zivR0rwcrd&ONZX?c;fHJcb%fZd??q4?`rF6DrycBz$^Q7rJU@?N54ilaW{XuQ=ZQA z7vBMCBSnzLj5k)h2y+1UeMe*hnMM@t6QIY(YP+p#xCS`m$hUhe3`BCn6*WttOTkTZ|8|2;$mZ!j+I?939HQ$?7OML zvR>$37`-eWg%K?KzMQ@ydn?;)9u2{^7lX1#ZGbas6hH(5X~%MQuQ|M*Uln?>$r2&X z$d6fa{oRfWp;UWb`=hYGmcD36b8zUfu>BIebA+vM0#@vmn(L9+l5}!3ARyJZE;wsW zo3KUOYtc79A0Wkil?;&0LLYpBei_3OoDeT@CTMr;$>;QQ!P`<8M-{_X&V!8iA z*k3?J*|qKCFp8o`igYPbO2ZH$F(@UH(v3(A-5rC7bcy63Fobk>cjwSZH%K=OFu?yp z@B4k9=l8v9ed}6_VKIwq$9e8^?|sH`q$;pK^4UAkTB=Sy9=m|uZ<{Nxzu%Up>#2JW z&UrocrEk{9Q~Qk{YrqY?{)E+Cc$!QC^i&#i=FHaDd9tcS0pNzMf@*?6PfSD#~QSgUCNjHS_%9f@dOes>~K6(k!PswWuBPb){jTx`6v zQ#`VDaMoB-w-&)1`y1&56)g7;WU{Wzy3l%qTCzh>_s zS@Q>zplhq!md&!voepFij=b_+_MT1JGgU7Pl5n)c2yS(3UJJbJef%1HYCB?4=Zm{H zk}1y}z@4q!?NRisENPv~AE-vIIt)(=87H1z6kq5(xo|bz=oy+fcncO&dr+qfJ-+wJ zE%A@)DN}oH8T1uw5XzGD8jp?-cC^zE6|@MZz~#k0vK@Qj2*@|Q@dM*Ek*+TPhQyGE z8+tf?=(lJS_PXw4{W#j*Xf<*b1RdwJpKuXAm+6(%&=t_Xx$2me1A|<6xWK22{U$zr zo}7ds6RuO3_cs$MwI^?${HvyCjKKa%GK`SnJ4j27jq%pwENz3^8|6bWJUOJ7vjlfA zjY5rjxWXQea-y3j!j@ApmtOZ1dgXS^1^e_?@>VWnsIC=}D5jNZc$Hlh1_1WteD`89 z%?8f7T+QWMtk zQTM}bk{aP{w`S+Jvch3dl&OC*}O}E_N z9;|pnSb3u5!IFQ@W;5_(`!>sWU;+7D6{4MO%_|w9bw$C7$(!(Q-T0{fAk3XHUCOWV zh}^6Z#9uo>QZ*Ngaj+jTT9dA)1_xt&6Y+pcG`?W{>R;}zwA|SE&h@0$X-To7d%+!jkh4D9LU7AFj!H_ zUY0LC?{UFM;*L_M4|-#8{i8c_D2goFmV5jDgFpT`Z~p;AT>T&K;4KZ)=XegZ|8{@5D*ScPB{-gs zG%-X5-UH}g0B@L}1kf5#R`=I^MYpwFo;PX0cNg*}Rs*;Leg1j_$8KGg zyzYM;iMRg7e;s$FYCr`Z(0I(h-n{>Rj!yr79G$nWKw7|g_#Y!<@kT?xWk31fR&Hla z1gKZNouz+h(6d+;MPlsj~QBLZI7 zBSk;)0un)c2vTMV*R8%f`em#c|F1|wR3;Bk@&(Wm52<{ypPZlPB*@u?9_~FpCo_xo zi1RO-xpvhAvG(i!~*_h48vAmXDc@4?<}(maQLlCkkf%N1u7>&=P;VaQ`vVkH*E# z_1ek}VABbs)UYe=Iq9jUlf=g~ocTPnQN7ix#`PS}A|rkBwxAV|mMD(hWnG0`Z@^$B z+9LmQsv6*yuARR`8+{>flCi3{a8xL)gLFAfi4H>9IlZDR5q7aMiY}VpbFQ6rs-3mq z-8!i;jUO{B*JdLLZuby|_ZwRS@=*pzc4Swm?z%Oev4AB9o>!aAx~tJ*e3L8}{JDC- zl7=V~`zs!5@bAV|^sa!e5$k@SHnFqG_`(wKtnnI)X)M-|34MgeW$Ql?K;YvV$O3oP zw1%K@Qg6vr!~)m=@OH!_(c z4SVv5ucIG1xM(5QR^Ut-FP$5}=b3-2WC2#Q{U6iWLI~s1qC+tyW~nDayv(}lcLaGY zU*Eo9A-&JIR5{tx_p;sP%%Tn;f%+#hJu z2aQ#i*RVG~g>G_lmZv?=X?(Qi`L_}n2o0B6)>c%+G*pM0n9-Z2>g#Q%O6P^JkrV&@ z$HFGHs)_OOsS2DreMzs=Tpdr)NWLU zeH;%KaIm;OAh9KUVCIf^?XvOU0Wg zlPT%dkyr{1@QP2mCAfcJicgEt*v&37zvDca`d=9Y#pHlyyw_8ci{_;7&{vqr#%*na z*%_O`HrxkRlOEL)+r+X4p)3HS{HIzOu$Z~}fLxBdc`l)#b+C3 zOq6ZBw7ebs|J|(@LUyuxY6H=>q$Fp|EgWWRF|da9grwmZccyL4CY_2Ghoyf^^xs98 z=NJ7maQNmjL$V;Pr|Rhmkv^Ebyze-Q_bS%r&jgcvsg=16%-@!&le1C-N)I*Zc;AEA zxbZxqozNz8XNuNZ+ka_!@x4^(!|rXLdPSkUz^?SC{`)g(+uu0vCo17-H7@h=*C1>y zW9WMUY?z#QbuDoWtQH$6JH0g)e}q>0G`{%z;nC06b#Bn4{qb(ZZb>@B|If6)xM{b> z4ZR%7;<{FSCBbSB?x6wQL=^qg_+jZwqyL>o{!4hhJ(?YTV5`fdBa7~a#Vw@JX(^CP zQqF5^}mu>2NlL=Dd(Q~Y{Fa_hs?wCq`w*v(; zQHV?bzO-bqc2Q3=S!KKw3E(1X(aAvB>5-Y`_4lM}YGqQB*}VX}>#Q(&mLlj#B+Uvc zxFo0%5o=~-R!ejTHBsYAUD$T2QKL2Tn6u-IT~!|W1-g+K5X(A)5Z6^r^;sVG!6;=d z`Ex~|FQl&`vhTn6WICuksI2b&QCBKdNGuB1;nmC9uZJMOogRAo^X)^Lwr>hhhOhW< z9|B=S4Q0n^_a|4szD7TYFLzqO<{xL+QPAe?6Pf!=gGk<#>EbpBp;sASWB@l_OcPX@ zx25Vp4Y5~;OQ!sqa)qu2_`*;Y<(l=6LWkM$`iP*y1J@;@%3A5RrWV67Vp4I{s<=3H zk6=|NHQ&fkd9UK>ue?gY&JIVY<+tQyJl`z47ab?I-;_FewY=0b^Ypy(R?7-%gT$%q zXKGC2Cp7Z3P(P2^4UzEGlb;ql=YRJCJYo_B$D{!|e|~-}`jFVWqO!^`9* zJC`Q}ulp&9@I;Fm=ZO!m8P}t-E-WkdPy8|y^C%Ny&KIl3vTUxJ=`L?@SJyu#$e*7l zP0H2U=71z7T@B)$Gfao$LaLB6Tv>;OGL3}<0 zMTt+mBkXxO(lE_(GFGs*sm};jf_8LlCsFn|WVFP`r@xtudQluvRqfQ(gV33+e~2L; zlFh+YUnErlGzeXROwm7S&wG-1?&uj4b3M)`00rf^Rl;hl;q>)uaF;9tIk9yTt*I}T z@e=1Spkp-u&K5uFzFNXnp5nUIYhmnFNDm?k{u1n-_?Dp8(5O{6NWQ-&z?UR#DFhv!v4Ti-D`{hPH})k`ZqwiZR{7ZR9g1s`J+< zGn7=etao76)t%<>yn6hKn5PN-*K_F6B4K|16DepyxKE!gPOELE`g}O?2b*!fv>d|c zi$`GMZ82Z=s}h@z8rt7<3E^~97RnUx52&LV%dTX;`U3?;XZgaUbrEge_$EEHD~$$5 z3{eqzj>QS8cgmMDp9Jn0ifE92jZs&M_yB;m42oc=>sDt-^78ibchYfh0Ehs2p-_($ zq(ST}4Q9tRYLY^>CvXd-{FeHM(09MW2>W@UDtTfn>l9QlT6?OAdCQ3305A(QFbz7^ zR?M=|k>s%BOaI#^&7UOXNqFBAW&=u3bYGRQ3I%OP>Q*tE8DUIZ$b0SZ8G0K}4pC@v z_lMT_SK04afebOh{vkFd%0@lo7`G4QLwwtkp%P=)E8+dl3*vT+8#nunpB(z}yx7F0uF6O-4MiO}P_(muLBO z(dKxWUm_Zg?pnQ^?gLTN*GPPN&j0Pf%d9MOu{Ya~3Z_~_DG-LDy@1l2Lc*-VcxrDZ zYA!9T%pBY+ccxx`xwVUb7IMIYD0nFLt$FEi;h6IMaKmNy`SDGWGjWq6trj`kn&t~# z7HuY7V`JlvP5`N6&qy!Mbp9(0X1Zj{VB2NeMHoi4@!BHi>if-wvX*CBL}xaViQnsF zK(}1eGkrH~w^aCQ-D6V;?!mHX)@-V8%4wQ5v<$JU!*Tz z-6D=rCqht|m}Y$KJZ#I2gDSRTjn@F-Mn^^RnG$Dwho5-wOLfdcaBx1`?8W~9pOg%=7I4eA2PZ3t=+-O9jjol zP~_t-GLL;f;no6{X8tTKU9VTR9)#bfn=o%~dP^tTEsulzcqq<>iOrJPQ&1 zAe^92Yi25l4W^>`2!-9Q=2Ng6WcI8fdEP|?tqeOhd*f|R-_pTDz9aXR8+_B=OuQRdoEac2@%Q|h$4Zf%ukMP2`mel_Q&!%ul?E-LX=^rU-_Km5pR z25Sw<=0Yo8r+V_!-Np5wbQ!|VI)Pzf^E%R-J-_knX#QRK)>)@oSlQX+T!DZO7**YN za1`tVKaczd#y&s#1|~RHiU8m+N+ZD3p?tyISRgEpV0y2^Vx-UUNcahi#*=q&{kdmD z(yCpxHd}1(L22*pZ((pM@bcm49DnYqugqhsN*8JLj4bJinhES;LsUPkvuByaQeXml zbr`A@CtyUB{*j=)F*r8Xe<)`WbRR!kbbB(O1$lou&NpT}_m`Q9Ph{RUa&oEusi{?~LFfC@_e~PN_=AJ!AEg?= zihpH&$l6J0CV}#Hy#edNN|QRa@4A>w1X+fDi!)%QLZU**AOo;$nmcBlnPv~Z@laPRqJJkvFN&MBt(ojl%Frx0dxJswC41gjj! z@7^Q30Y7iZu3XA;N1RdUShuE|1E3r{<@XDZr_$jmVo%>B|w zd2Y?klJ%j}-;LEz8rCG${OL7$j!vV|z1cZAgvQT;FE%2UVNhzw<86*gUxF`fz;;CsRbC3{V#48_Zk7 z3db22ASxnWtwLq?ggGQJLzHW!7gmUh&fV?qJ-PW0_fay0? zSiAmJa1{+j6d;V=Y2vx!h)MT~LgOEW)-@2zcc;K#ZSr0?T3}Gw=rOf#ziqlEg#APIgn@N>Jj;zq z+8b`|q((DyNt_XYKQcx4)$YX-X>L+cM-B}s*N--~3f{NXOb-x0hf{sBQ{JS7r~X*1 zev+zjA_kWDEV|YCl`OUpf3m;(ic&LSF+lMwpNW#5t;{q3iwoB4Pv_BXTtv0gWhE(Y z=L(9gh~i8#u^hLM1y#(K6KeQl)dl+?B}}!7yG*#D?X_~M?A}@1In5H3;ej~9Hab5< zJP8pjgKj^OlXK`CPkBfAJ@A^oRcmUZoLUhC!J)DVlQ6VdHxe3;ozM4;AGZ_v>dtxU zMrSdjC`p=)rwDIs5uJ#3>z_4hN>w!e{Z(qQBwzvP>6a9ZR=cWlaOR}9y{Fmu9FGDe z_=(A#8;4@hB8lKtaJCR}OxnBi5h)>IpD_4eIr2lD{uqH(@zOGNJkM#H3*>&yi#|#VXGSKn91AHV<-yOskD@jIWm0 zm+etz_o>ZW5f>y8QqDQXr4!p&%-w+nZUo}7cflifa9*r1NqF(|N8B0x4DYsRPS#Ez zC3-l@5D|LvPUoT1%$mMrMB>CJ_K7~`kV)l}WU=E--%W;f+Q#w}g?+lds)x|)-z9BK zc=xX$Un2k*BgnP&QpMNEPE`uCmhrYvZ~T#Dy^dLX7CGZS39W9=y_GQM^Pd8!>LEh3 z+Ota~vr;=_Cw|p|G`cPx1oV!^&#RpJhu$54toe(`9=O~f51EjaCx9bZ&3qYhsiENi^`_FK!o2vEY^(8()PG zJ*2`ZyK;MeFX_O@o%XMO8Bguw$HCA!mxWilV_84TTu+f2=;kGPh|k_KDd}7yvE<+< z(Lz>rXYYc^OKX0$RrNtYPYm8;T+LODrFDq!gLalnjnI}qsJPF#(?l;OR8qYbGC!FM z57W!R;Dj2-^2(xW2R9!v2ij|~p)67YF%2!}^x|yEG2juh!>nd~v&zU2O z!|rayV@--A!%}4NjncwXYNzb5y-W_$OuA$HVDaH_T3wSjKkh7wwc-ZfSI^*kk|`Nw z`f>e|IK7cyF7xp+gx^h;_)I`xO5I`$AB^5khQsV7z;lOM3&Z@aB1Bi9=y7Rkw-K6Y zh5;JIDr zH_uLradjw9S?6yiwPGW^t6*=ozKP_PtZp{NGYm|wC~Y7o|L(>AYh~doyX)#--#~9F ziQDt0k`%P)5aIIb#1#2mhjBjeUJi1D&v3H+B3Z%f1F=y&s4;ZEW&GuZ6;*z1D9!aD$WOrN!IsIA8X?uQ+>_lBFVfhruP)~FbF9j>cv>pfEL|Ay8eCymFy zQc%-T^4~YoE(nF=q?Udvm(sQ8)hL@<(K;;r@TPt10eBfqeHHF5e0vHi|55!(&9F^C z$fykw2Cq+O{E*)G#98PCR2dg~U4H-s^WX2{=7EQ5ivUWYUl?BU-*nwNVY)vazdH7N zQXlCgg!VxTZ(9D$xM6Fzwb_QcSoV_c_5YZ>e`kHYxEb*)GoedS|KXXaN40eahNIUK_42a1>Gd%h{7=DJV1@{bgbR6cm1nl3 z=m|>nZ+PAe1$N32I@qp~+yghdHy34%zAr#M9yFlKviHMBPa3PPPS8gfr(pC484-xM z|2*dGJHuw92NkomSyY`P`755u&uUn9`^Vp}dbrKbT3nUJu!U!43IgWy)S ztHVk~l{rs^{PR86%GQpjI<{&j6YdzGc_aJ_*7*@kcU#od9kWL5_e{VHDRd4!aDi2~F>kFK9jRWWVGigr032uV!BvxHphLpp(wID8u zT-{y$sowNQlTa(W~XI8n>#&ch;vtP)4HsX1r z$m0#u!$*b8ae0w8c0MZnIbcc`(uGY0a3OC1=4mBRta4)LkMpYsleq$>Uex9`4frSem)lG$zo`dvc%cK<7U-eWvHD~5?86vp% ztL^^}VT8boxrYuD_1s_PwOzb4Gm+1lzgTs(_@4U89tp`Ph~ejHLHMLo;9z(glX|rF z98pbV=LNz8e`%xkJy0{;gw=$iHtGl^PkzMbGy~!9CjAD}lt!p8mIrh8J7dB3)eQGR zzUJ5z=yK{h5*aWmo~`$8X5`2&CMrAt06!yib0n!&-&1-^Wo-E+pPa|BjQe&U^zlBZ z+cR{{N-+LCT=zr%q#X;6_^^>pBK<{l-)|Ho^63@HW?ql$r6RAFS689O^{JS=8c*K}4}uNJ7fOQS5}OK;H_%HS%z!y%}+uG)B>9J{eTm7f z_t(ET%`mfu2uSUcrMll-dMWRAP*d||NGR<52bcTp~|Z%4Ap$8axktLfitJ~ z3UD^gGM-Kuxw=C4qqr}*A;zPwoXde1=CD53I@drD+QGMGP3aalCnNmP?4imd4L)eX zo17IoLB5$4l+PluvS`clAwwgqJ(C zAz{+XJNI;^LbZgFAC5iQ)=Zuq2VFjp_dNYck}d}cIR(XoyJ_L)g`W4erI?!{H>N|vdA^qVcjltrf3%QS;3 zbX_US9z$80LvRbSJ2%=?q4M@bS8`MDs0C`%P22ZUUpt0>8o7H?5JaDgVK>6%f2X%S z7j1}3zEVf3)3QLdh2e2^E~`YPnDlBWFZ|3t9B!MNduQ2T171!?OeJG<*Y6SYG@tMB z;vqf7t0B6>p1aa-1LEIngyxeKvT*CXdwkObPE}Dx15dl9K}u;$f1a>f$Nlh!@8q&h zdB}R9z%tTtdWoc>KfQ4Kp79tpBkp_6939vYfgh^F?xAJNnFDHG3eWGx2&IM z7~BkS9cSxfr0d>fgUz?)=O=jB6kZ_{%g%G)(J8Nd$igB%E}|JRf1?3^_5YsXa?pWW z*}2FFKac7@gf<{%3Pz4k6`s#2J!xQ0E$aWHD*5;Owox}Fx=n&mzl0vA4}_jEN7Kk+ zp>d!}>vknNrT1dMKebBJj5u8hmRL8qh(nCj%tSPO0}8nTHJJh$}Jc(sAI5$1SQBt#UxFL^@_MD!iPYCqKkFF@uj7Vs}0 z5jgt)@Q}uZ@`h3WK++eq%k>qg35oVde}!XR|8u}@qv6>*`aA7q+Efp(dd|gN;tnT9 z+@7N>0^wLvJ6_fv>1#CTKJsmdU*(?T|9epWyBX!O?E@FxGP8-t?(un<9j}BBSYh^O z!{t`-i;ypOb2=U&vTfh^*0atHe?%mzS(vj>g;YjP(7I#S&TDyb_`r+R2n?s=i3A0o zf|r#t(8ebc9Ow-djEK6DlPT;~|$f{Z@)}?Z}yxuHgL|+p(!xe`%7z$ukO3 zKba}N)ClGGu|{fCY=p2m=)pxR7BxXLw>jsYKE<{j-y1AjzPb8U|^X$?( z{L<}k&z=C)AW55LtPvwFBZ?M6_Bym}0)2ByWO7ekK}b8$TNb|=xP5WX#(fI?GzCM< z$M%z1(6nKuCANN!UOH1^enJ~;Tx8W4A7A9G=zr$D-2L+0(dTRHkimUw!ARK(Sv7qM-sT&g0}vMN>-z1*)CrnJ z)zw44?PB{gW#>h=@bn4VZ54Eo=<>BBeG8)9Yv$zq9PJs_1q!jC;n5P9E^yqUB`B)+QaRHQlNIlFw?3OlrRGHXMtBZSvbK3>| zqnM*I=v+en=pI$RmHa`exHBIko^*l@pYYdEou2y(x>bH z8`ds$*YuOg$k+%`B6=x5pRi!NM(*^}v!VHSvO>;#@EV)IOGx&Onm6EB1nwuDeyc*k zyDMdM%XY-dl1SG9?*}=s^1br<(UCJ%fw~a^tzvuHqAzQNs_zrBHX z!&vtB@=p7>P~ieq`kJ@;8Y$Xp0;y>*xPcE zaS?(tAN9iJ^O(R-JrOo1?~!r4-Chy(R+QK`)jDzPCG_CLv(}}}1^j%q*Ia^VRJyUZ zSvhSWyK1z^r!el)`&%?WEx)GrE`?+v+Up(1#fN zir1+-hCX<{F#P#Sq+~@Bem#|Qy&vXm0M)&aJ#WNe7vl1GUHKDB6!)ZWu%)1+n>$@NhP{2#B+gSe6kq*yR1{9^+4zFND42s&8Kl-TCJYj zX>_*rj8WW$`Zb%^5%WcB#F>|Ro0^rov!Eb_C+doJu^k!bc#@_>O`V<8eOn~Cf&Mqe zqd;i7s9THL^Jpb^&+9ZX+^mutNK+y^pR5O>-4l(ksmF&sP0u_PzS7pNuXBX+Fw~$N z-^vx>PG97BtJWTIwlB>;=$JadL%6#s-&SsZ2G|sClQO1@q^__2cdGVL2kcinPMHs{ zWas9xFBEo7SJpW?=L9Ihz5@{qGnvyY$N3@lr`pt(JrZvIDur7Qwf&p+h@rs6)?!`@V?rR;+Q}=C2ccay&+4!CZOjr)|K0R< zsIi42bl}Ki5&p?!J9PZryj@NZ*@|m}U6+H(=xJUwiusId1@b3p?eDby@{=VvI>G~r zI01gztHWM)PVR!^Ec0;vbAJ+f`?@iBjz9C-j_T>zSpV5IS# z4q%|NF4sH|A0qq_1z#q)QcdAR9Q`6+?U<8}EB>_Rs_WZ_Bx>O_96!jh8Sl|Z88h?M zBd3@8Zln=-3(fFPR)!$0X}s$F*L8_HguW)5^&f$EHQrobGI4iE>KK^kv1}8yeI;Wy z_6VqW1Y7)+^`}#PC{Sf9yMKIth9qDd07wUoe_$S{pFS;o#NE^?bnl&UFt@B|5Zj+& z|C6HxVeBp8mZb=l0CFLl%wTX+-XEs%7k!qu2oV2dyHp#ap)K4%dSWspjr&2D|B@a6 zRnN!?3;VIp7N%|nPE~GzIaw1qCf3?c?TJfUw;$SKqFjfuTaBU5qi*6RaQL=L-x480hRJCz@(D2%^x2bCaaZ%`VZM}F2eaxS6e~{dS z%*n;8iS14@VD8Y+{6?=f)y$;1s5d;Xw!Eg-!VDhlzJl(DMf&f)c03*kOmmi}u65sR zn*}lM!Gs(U+KWl&fwgtr?m_}9D_+CTKnaIMd zSIzIA4tg%aXN=j={&=o*d3DNqGxJ#w>cenyJ_s+@laTCIrtwVLz<7hd)gR9{Y*m0jWeqaNq=1MW4ETF55Bi( zz5c9oSLCWr2N=$0Ys3`~^%yP9eBx#Bg{ytQdy`%nS-Nx!;WT;cNfpsbUwYMi4dc!C z8zGgn)lY6SovrL!hR>^m68FkP;-jhkZGIem5 zd~1xjKAyM9f>(P3LzcdI?||Mvd&D~|$!+m03F~7z7nh8p;on7Ft*;_B@_j|pHz5mF1t3eur_ZmfR__H6B03D zS>Hfdh2t=|ryeiis)NIS9>?0RJ_-uq^IV#e5j^OsDJ(R4H)2s}t|g5%ZlX!3AuJxj z6Hne!11o#2hOCEF(&rCQTjeXKWm%Jc!N}q?i`%u3JaJCWhPQG!)Ix$FsU|ox7@E@G zjl7h(6$Py&F!S3N(8&MjUQ@w>bF#|Y#;bO`%9AsP*>?e|%E38H8q$|~uQws@O?U=( z?>NMaKpYXS%tLSA8^}>_9(2T(3A(@WU#~MK&FGPmJH$P6;sF?ojcJYCs^9E(XgK_2H zvo>ft4u+FY<=`VWXo(fK>QnlM`AGq6kCr%!2>(hXZGY_d({tsRKRk6{^}d2t31N;} zq|HSgLr_9@6pOgg(#X%Jp%waWlp^LuQWm2hfC>!)KA&-f{7>Lze^m($?F8f(JBq*? z?}_QS+R$5xEJcTHLa<+v^$b~l^TAXCN4C@7w5=R;SHI0a2fIU>r(Y3t4n3g+*^h<> z*}Z2oReBT+^U*RkzSw>s&sS5;Qd?R6wo#2c^Z~Zls;gjU`1j;vkj_Wih^}L>SlST4 z<(7H(_xASBI$Byte{%;&jg9kse!cL&1!mqm#!M4<{hHEip#$z^TBPkLejPF99z{hK6 zXH)E^BU{%00*twSc6N=%=kBP_jiuR{Zp?+Nwey5<>dFyM>G6rZm5{xiXV=+cyYNl8 zT&TK$xxjtou}7wDUpBVo`oVd%JiI;dG_VlTIxfK=ZzSgR&9+JMgPMh|#Lx zY3VWW;3Xs_#jh>b?Kcpu6VieXs~wCsAzB_A5a3y#cz#t^QhM0(T-iZKG*P3*vrc39 zqB=dTy$Yh$hqBhLeYw^*JnHj38pqec>i#Bb)Z6|=!BWBm#mpP z<1}w#1zTl#{wiws-lExfb^%VjA;K!$z$YTU#!SKrSNZ;uDCdX|ENp5UtG17$o87P> zw>DCIr8$OEn9wsZ*i%ub{n7SNohi_BW33ub>s{+_Qm0+dOSri?#ZGm9Io()P3J~Cq z-_u|mB_K~E7p@jAk-l8^qytFneOuOVaz1`Ew2M36HsR&(mu}2{`lua$)C!Vo>6>!! zrs85{M7~Sq*}lNda+hU1g11Cx5%>G2@Spfn$4_;rf2FT{JIR(&0*|8CF3c`Ne?wXn zo@mzBJ)D&KDPCsA0Mri$nyaa*>Z*rk#h!bWI#buF(CLt0qyKPjqNSpyYVU?*6{@rnzkGm<3!7>%qJ5CAXr>DGUWHprx1}py9kwUyX-&88| z3#OA}l?|~5diAsicCK4m?s6(opn|8}QWIgYmjy<0Ue~B5NKX1OiyF12ILdmQmd+pu6C{_c?qmYhvw zR@1;S*h%7mAO=c@RTQ+in#d0f;mPhL>E5pgD^SoXIl^A!9-ttppwX!pNrFqt5QG6JvfSF zl<#8IA6z;=>H~|8w&rW`WuJcnSu?O9Rvv@T&K8|EF9YC0s5gz+Plj`Hd#R%JyXZvG zm@EwpX8n9|PC{@eNN;i=*lh_F)x|0NN}B0oD}Uc+w3}VtuDg^E zysZ!etY$Z z!{Y8$Ef^IBny}hyic13>6*eT=y7Zn->Ls93krRCrj6{n|!=F59B3geEp>bIIZ{qIT zoHgSkCmYKF(4W^sgin6IGceH^2>%9f1lS5qowskcAu8Syqpw*_KPIkBirC|#3BzvH zL-{{k-83|ChFFadVa)ZZF>G~MhVo?*Cp{&T;OE|0PnvFQUZ5wb_%@paA|hU|cIGsH z>Y5Vz)0VXWl!Tm7{C`)|2|x20bnWAY6kecBM4Dt6rGVRLXR$$N(?fg!Uf=)hsf?^_ zBEM&Rdgii{5&*3O2Z)q?09JTSk9C;x(^mnJ5407E5lFOUSlUB#q6fd9yS>Hmp3v{o zZ$r>7KSpw6?)x9%3(1W~^S^**#2!+M$PDSUcmUL9MD1jb1mhW-OCwvSsZ(3-!8a2ivao>B${wto0holgP=+3;n8>40?q<;p=r(>4oD0;FPB7d4kaQ z8FEDzAi9o$KVzmz<}{Zimz4HPU%reu;$rwcaWMU<3IO|-Hd5}VDr(*Y=#i_Ac;Bt5 zHdRc`6@Hl}VLN6P>_F5Uln9-@dy|1lD`$b)?pdK?>&eO)^h~M@3o9i5;QjMA=rQ;* zi4i!U#pceOkVFV5q^H2%ob+{zMOkTImG+p4Ol79sSU20bq0h#_?Y=OgkCgv)!I$4J;q0()`R7CI6a>dkFJFd5F-;lXd_&rYHV>(Tz?&v@OOD6gP z4Oy+D(}$VE7VlFXP#E{#pfVrpYU!QU&QNnmb8_;QncHXxO-1V0xJ30I^rn7jLSiHY zACD6z{o&IZ@ye)>rxy3Wsck}0*u@6a=hXpks1KGI2S!*UJ{ zGTIB)FK&=nHV(<*FLktxL};6DEq2hT*Os}_EUmAetVYy|Ek|h;bQ$4MwSI);aV*#a z5h{Yzc&)hH(t9p3p)Lk{kk4hsHQ}>rU^_(~0P!FF_nR}QM4f7fkcrR@*Y%rdg9Qjk zE%@a$alH@{R%>?ua~JuHMCOR_2C4Er{`d-{bsGZPR11&KaKvzV{dKNkhr}?(^h{zW z$7o7R6eAwQSZ$qQ$_SLp4@h7T!%MqLSv{1MuTB18R^N*?i_taxn66Fpq z<#k*^c=-vnf|uqH->CONop@kDUZ|gtflc3$F=7s2&f)!B(W{g=lv6|R60-bBxO7|% zgo!Ppj}cBSv;y)Xk}9L6qiDfK^6(fs4igxSN(CHD^}7AP=_i^Wu$kjFYc~ z?n+$^9)NZTTG8%0QWp)4D4qys02t6pPU9Hn)gEg`zHyYv6Bga5z+81nTT%i=gLWgc zkZXP^%MbulR31u>oFuEYGn$T_zWcp6Bi(o`Iy7y&-OU7rUBWasCbFV!jZKJzg}s^1 z{;6qZwY=U!#^so$s-)S0P!?Z|29baNjx#Ab(Z9kem*7YdF`aTy8u?K%R0#flDgBXW zd$=2AAO=p&H`DPeSk0K4DFh-JNdP}fq3&o$BzjgQUf|!VSxf!bK&t4l%(Z8JP6StJ zX>*s8je#NNaKntlWKto>>EgOm3jfuB{!QpPa*G9>z%1&&3`VG4JZY z>=)t6soKuN)KRASW_@je@7awd9gZ5R1hAS*ng{V~Sx4@bYFtq3AM%g0l68 z(9dNwQgm?Xh{)p*EZPgpWk{S=*^;k>JpxB&%nKH{?HoTb|GaPPfiJ5`Awe$+GQDTS~2TDQ+t%#&*C8LDTu_7Ul- z81>H?8u$o=7gO8;1CV*glckxD!AA<+NsbGUyJ9Hp`z!p%z&7Pnc+Dafn5JC!j*z?MB+}gG=$8ut>LKDT2OeQ;~qr zRx(53)e0v~-{n}imV1*nH7He6eR%{`gvyRAN8aQO!`RH&S33|cJoxSz+ZZSul*Pzg zTugPz0qBDw1+BC!+Ok&PWU#QPp9(1NI}!G;mL><2_36>T-?X{ACv3|Jw#+0(GoIEg zy${aT?>_AaT|djE?r8cC#SuA(*h^IK?tusdUJvP`0KGs(2c>#zgAJC~!&qb)px3?U zuGd&(SL!J2W=Aw@;}DWv7lOt?0Ajw&7k;2XCExNn6wW`umbA)zDJnVr$-`=EYa6EI zVbmI#akMk>o=VGX86ruF7xc0OD`-tr^p^w$pnk8^+R*9dP_d!b#*H+edjh7AWb@09 z3|>(8m+a8aB4PUv{NS|e?V7pj^vHBc#=x8qUEVP0|k;~r;fhp@|*0SbG$Fzrjm6NZu-U~dnC@+UK# zJp+`Enug$y*5>wVj-`=Cy1Q%X?pnH=1>PIK z@jSolectPO_pjaio}HO9bLPa%XU=(Hc8;}lxJjhR=lBl&#n=1FdbQK;c_&fTZBemL zvR-=LkG_6_oQb$ZxWkN7nUp-Sm4$g!axlOGhc{EnC5zI;{*vfF7pab%s=qYcQ;0f=OzV@lh2eNlt`G(UTdMX!-1)9 zJ7jgK9d?{5qqnZCt7fmWF+i$qlgg?$@d`%lSNk4~t!0^MjwUnH)S#%p<@u{tUZg+(K;`Qu&!O{OYrG2*DP5oM=1T4r~_oIPq+n zv-ZB&IgolvaqXSQbH#eag`3_c2fPo+gpah)B;LPV=|9q736dw;1h)ty1M5bkLz7_f z9MG}nL%{s9xH+3^eVvn_(zt|F;Y0H0s1=az6y3ANsI@0zUw@>eQpP?`;HbeoT~L1> zY&&DT z5-^F6LgH%+B}wsGg2>BZ(RzKxtI=THKib@6pXXqI z)t=D2x-!Y=n5EO*b{@ZW_+dmhHqV^mM8Un4fst9mbbiNu=x7Vcc~uF1g*-I~TyY35 zjC?KD5u)8{u#|*nS|%ZZ3f5Fn4|a%o*knOM4Q3?)+kZ1$+Ap~cl-xvrmIMad$DAt% z$jCClw(U%;O-nTTC&*7bQ9(>w+G>vE;1Um;hrPV9>QRZo)A%al?O1NB?Oz2ALuW>c zWLY#d0K)HesAnv{p|naepnAo`$q@M@?21TFS&h={vW}xhRe+r zhx1s!ui%SfJeIv=mt8=3PfejwO+1zJhHohhK99X-Wl+KY#^pXUH)oX2QjhtTq5sHa zwjDkx{L@#0Oh6G?a&VwD zbl~RXbVqegtb$+zFBpc++p$nwM|UUJ#O+JEiS5b3#2N!MVtV&`&SN_zPEnj?i1ynn z%Xgmf$-Ei6yqs!0uVQ+7^yoy(9ILnV^z0K_=D7lwSYZUlo~{sYl_yGYJA6lY26l?xU{`zbS z)63pWok4q3dl_L|QDC*E*l&n#&^`u`m5+ow<6Ujy9V2vb zP#e+7y$TBFW~c&SmWlcA3+AhcSdo{o24MPYuG%LZYWHVv>uLG=m%Fc?9&pskH^s=3 zoBh&G;MUY1;dAc`F-C5$BoI7K;CW`Y44KE>Px@@rDh<`~#(BRCxgD{Tr%Kb=3-*VF z&9ADdspUmvrvL72hT8=#Xt89w+87(@-(&pP7y9^Oh1iu9+1uhS=+F_Nb(d_8uCUJ2 z`mv!#(m7w`N2K83e>6U@jW%v)*9eygrSOOagg7JLcC1%f=5-sKU@6Ynh%khfx2Tlh zUTvR-(N5Vkrna@U*lFhdBX*HQ86s&;LL*hN+tyCE%(n?zh9so_kuV@b$& zGP4Cl$OuLwTwNI$@kC~xd385Ep9sL`tE{+u7X@FKe?s=Ur=AvlVtrbwl5jCLOheT84D;=BN`J}B zc$YjN^S65)ncrq4@<=Qp`h{m?uH6qxvX1w$eoE~6)M+@8nA?n;$9n7~Vlxq0O^?A@ zJe_?|O6WcMnOy77N$atBR^9c4UF)#jBC)sby}*8t8goDWft#IW5(i-7yeZ*lsJlEP zW*4L#a@3iwvJP(Kzg+`sCTa@99Ved))E;msjbA*IXMvY518Q*5J(l z2rS=%RkS^Y$*{CjyYEteJ~za_YZYAX7V^XQd~=#qarVs;; z=`!T?LIm1Vb%7>3E+?-|e;5NphV?3eIDMDPjLVdF%+X38A@3t_{40*0@wJ{`?tLsK zQemDoPfl4v0{8AVV3!ad6#wI7(?ZvtS?aI7IFEHrRL*(?U0ZxOgyXFL#ShX8Fx2~+ z`$ywwarmBbB&y^@#VdrQqrk=~2Y%sri3saYNvb0H;Vy_Q%x($N-K-c!Zbdb|wSy6Z zVd#1y<2iRb@xG;fU=aP`J&Z1czZuI_nG3D^=O4Bo-k1*u39#HZ>dc zRAZ!eWQjOtf~9&`*7kN?i~Z}yX0B%6BI9R9ODFWe3Wo{f{uhFfQuFE#vO-FSN9(;q zkU*Fu=~!`g5vnt_-Zi%0D1qkUgjffh0)3IbLGW|*@!~Ejs_ii%&F#s@=+2S=Tkj^i zVK?Lv_4Vl29+rF0tr*`{z5ABA=o0?}>$C%WgYwu&ueauno6uF6r*_}=P6jaaOE}~* z0@DrWqpUjLv2)tJ+Tci%C(igsTkC^_JMPyd9%zabfX|nd-YP9*`F4H~*K+w$lZA_R zx}6n_4OC}5C}Ga2c>`KzVoB;Mb^4pE)yd?TC_NLoU~{y-l}6rdot;bUgYdAwAj*fKFO zVW`a~A$(F+}<_YLp>S$e>s>+NKr3Zhg;Xh`UN>EH^TP1hbBIFHrdf{L~m1)hBRGKhsV zaV~Ds03YgdmeY|}G@ z8k>0>i8P2z$`Eu4Y^h|dy|$p2osD*4Bm1T5w~vdQ*=^Yw!M_i7CNU}fa}p~y z)#+fS&2ng{Cc7LE2M6p=8mPv6JtvU4a(wLkB8x}%NlUQvxC9YVW005efbD0c-ax%R zwFL-Pv5n&gQZ_$yU4fXe+9gg#ePYNhaU)=F;-sxAO^-S_nGu;ITnPH@xBNBIVhd$& zwrj3-HZJyha|XfJORyE385BHp@N|q4d4W=9OzmkZ8+wIPSsy{%)w|DMgx?~2SxZI9 zCP;K86S7nAXntrKhi(5VE92_W-_lThz#-(A1u=U~5&j{9Ew!z7`7w?6hE?F#6%XtS z!zY!@`y8ZRsDu>N{q))tkRA1e1VD5{e47jJ8L|rzW+D{w9Kj>q(en5*gUq|n*=lH^ zG8vD+vZa5HkN)Dh1CG$|1Znc~P;A@Nrf?o?WkD@0u&y&dI&{s&1IC=nRn6hy&#jui zn_(>$h+)4+_)51Ooxf;xVFNfb`>3+FY9gCnZupSwVDTL|aN@ZwI{AXh*!va-xrFu? zzJ6987lHhryV9;8!0Gw245^mD`QS61kXe#p@bGk#n>Q@J7f75p;%h+}7#nsC1n|Z) ze58Bh{a|_jyy_J=D6#S}_vy7u8|Lc*)WW>H^+!B@#GmW5ByrfE1#vY=wk$$ax4MJ4 z;~)lCayVJo1m;BH>&hToH@li&3;5#L8d`t-RD!rSdl8!urEZMIi{;*2Ii5!m&*PIygHJ z%;cJR?pTX@HV77U>(w^8;-lc%)AXwn^#sS{67MA~d0zFWshZzzMaqV>!Vj~gKTvPa z9idM+|K7O*&$(XORW!&XU#S&eZFW>Lb2^YeMkMvX>aGIR%24rvH1;vHA=!(44lqP! z3JTK$aZp~cYbIql=|wBlf8t!opOi+Q?#=WERT8PA9h4S)Kxt9HZH%wj}%lP10QvEdC`Z$uFv^$4xUJ<2v+UBQ~mJdk=J%+4bjCj#NqvG zhE(1ct%r5NvPNOJ%>=~{@%d6-cZ~{X2!o)zojC{AIEJA?*UVgRRXbb#Z%7|hOzSz| zMpCzv9mGhFrE)8+JX5lS@*+*}S3Q050rBl5 zu-ovcnZIgrzeE5vEnte~MEoJ9n5hHZ_PEH2i0bqs^Z82)ffGAid>Z`*ao_xFO3D$& z$WJzgbOa;A_33S)ha}2z21r4a4&cxs=AhZfdRc9*5gy@&69S{xwDNDdwih&v&>LLG zzg~)(qS9ikYQL&T z)UwEerfLSj=vD6wk-xV^Z*})r9H!ql^9Idh@Dhl{ezxgV&=>C@p0gF~GI<}xgByGm zutzh~z&Y2P&^}^R2bV1@iWwz$;iLj^(vI@;N}FwpC*5uhE(SJUYi)tVo$wWeoI!dv z_o4@2N7>iyTQ_+>G(*Oe}uM5O7Tb|cj+vpFt=pS*d7!q~ z#o26m{5>>H?Di$bwN&Xb&_$D=5A#ZfdWAzYEe51RJf$z#uhPPuh|WLh-~;HnLu;nH zMgh{g#(d%M`9kLVB|1CrD})PVYjT>V(ly;>b_^_J9}V}xcV+^T6YYy z#a~&A-4~L(b=WyJ^?SVrab*vyALNuybj+xIt%AxdcIg}SLf_4(Y)Swl871GFh^AKZ z4$OM@IpjzU5xy-pbA{HAYFMMdBO0nNs-RQU?gePuX7+S4m-ii#kUOHqk!BE6rg~lw zu8?=TPDM8)x2e_q=D?LRzA6j0SoO?mlbu+spFSNz_%$9=>VIjG|?<}uqu!?~y& z`gPT74=~WxDgA@KuEj@tQ8GFf5EtSjWy2sRh$V|mz;IYUdvK%$2(Xg-|RQ8 zETay~;`Iv(@u~{&^85y8Q%lj~b7kT&OHmsc>Ep#twgi^=5g|DJc94r1D;7}{WnE#b zrEI}1uYi2C)IM_hXv?9(5|r-xIs7;4)*~neV5#$U>va$(hbP1BAmp$_$$0n-0CNuR z;ZM-sLEnLo$df}3>w>cF=Qf!Q*;bBv{KNC&J0FIjPwQVxZ#~a@5h|gmK`Nx7XtaW8 zf){y~ej#i0IBQ^0&Fy)Nh2{ja z8~VJ#P6p7KK{|K9+JA@ppd~@g5ZX|lzHd^&qh2_L%@~$A_3&&V*V4S8S21yFka)4G zZow%M?EEn0vHZq<0*Mfs+q`vcpNn+OEl9-Ob-$1U&`cc34sl z0AN=T&gM>O&338CgSJojw4VumK9jG0e)dj|K0}WAzC4GXJYSnUm|nEH3IfLicpe3W zX{HfO<&P;swL4a+tG1?#0=nG*E77Y&rfII`TAd#0Ode`D!Kc427Ld!5}@3F>f8mE{6CG|P0&Ae&H$bGw=v-P zKXq1N0F(kf`McczsZ;2#$A1$*@V^=Z2uRNQo3j6I3>Z!k*?%beuf_lZ+>`!)H1^*F zIQ_5D0_q(3hcckC|0V$QPmh7bT~FHro?I9*HZk*ZMf6Db=I0$kt zbl2m;a|>n7_wenvs0+W|?ve~Ema=Yku*$j`{(Z$|1py}SUs=Ytj zk1?5n(fD&mrQxR>R*5#rM~1SocGBj&-vf$`cp~{_KZo3a{3A89+zVK9V8J;vSI)JT z79gp?oqPi#dLHgNmQ4PXBTJ12iZ_~nWVrS_9Kq#W*qyBWPXNy>L#&eF9n!&`S6UdK zl3TWm=RMHpoxig5ydlm`F1k7QDO=D zyQ570$OJj7OBdlkwE^&3Iip6O+6x;caVO5T+3S?9yKdd}j=h~gG`}p-dwrF#2=$Zk zP7jf;Nu$q9K5;lzuwxy|;ZYwd=%VVDVLw7hpizqiGnLK7Gl?y#Gmc#!M@9Nky5jD` z@Q??;g+wA9V0BF_wb$deeWBY802jMwu=8YcW(I)h017pj()`|8cZTgG?x9SAZigf0#WuANUrwxt4f|fuh2o_Hcy~0!HQIXAYDE&LE&+C zM!ids2?1&S0Osc|ptGnJb|uOTIgr-!zhaL5cds7&J(UhZs*Sh{bD)1OQ@98IO<5(+ zAIil3z9{y8N8(N(6JL-PLV#v0%z(EYzX5&>3z{dVGgd-ASwpW7x~AXpmk z@vZ6&YU!l_zO8*r_&^~;*Zb{xv@x=R32;Y8Y;HISntN3<@ z*xCyU-iw=(?g?dLnfpisFIYJ>Q@c4nLH3m+k7VuJT@hy4i~Z&uXqG7!?^sOlav&=M zX`-cHE<<{XnR<$YV7k}H25`7K^oUYcTD`n(M$_@?6#J*c%|{Gc&2>Z1LHXEf7iZm_ z3Ivw6NRX#dHissr7cz_5iDCNiM%&tJHKeYCta9K}lJIh4Xg!bDJTmZh`eP>oNhgg{bzNeo|%ICeQI%6yL*h#^vQ3uwE zpp#MCx|qAf^yI*UyIk5x7y7^3X@`4H%>ijAy4S`k*7D{qH1R_MNCC&@biK`oWNj_F z9d0ku4!0iP9kH(1TFAcGvTxvZTy2;KtAuVgKp&Z3HX3#Q$*z$wJPwEgJ{khiVm!1{}bIPE?i39{)bi3i$G zC3Mt-1N7Y<1m4RR*C4%RkSyOYOf3Bte_&nG{bz_*N64e+of-U^fTkJ_t?hMi01Okr zL%@W;hW<&j46swky^%RNW$rKXuN4_lO%r{}`wM!#(~7q}iNCj#g6A!v*@in?ET`>m zU+bp^+sbzJkWvSl0`aIe>{U({EsA{^pg?xn+JyRbLJMT;Mle3ksIunX6Czd-yH(`Z*I#m_!b5wQ1Em#C%lprWFVlFnM}r3+B&QN8%!o|{CZtsB zy^e=U>12)TRhJF z+Gy2MoQtc$IkGabgm~JMy@+|l#O281*8?@vp(SQ&2O})gCH$NQI##uYcuBvW16Jba z$NX0h;>f|&f>7b0-gchQqVf%{U^McmrHJL|1YUbCGsMhle{^QPPJq!V>F*p<+={*e#bCvFCvIh$@C~Y1% znk^l^fRDwIavL)v%{Pm8_`W9YGr@}k-T4@Dewa@m;{E>Wk+;ZEf?lPk z^1&i`@a8;_Bj$2sLkc$lzd8L^k#}e4k8NR7qZ1v9f=6p+Bel3LP=nTc!Z1O%NJVLf zj6Ja3S8@jNUX4&`-o!Ur7y0IVHm|mV&v0-La~>2X%hq#7;N`?GYA@nGZkLuGd^lEh zV69usw2a`9Irh6k4g`sQ=@Cgx84-|;%mwbhJLXtV*4Jy%MP6cXH+=DJ=-y`aL)2WM zzl3Qjp((%(7!vpLEKuDMI7peHSnV#l; z&Ls6I^r&F$7aU(nFubg88-Krw)ZP;F+Bn2dJED*dp-Sohz;!2XU81{zWw*rRKc9EC zOFF1r4Lm~pD_%a~R^fD8%kaKnd5ZSZr+H5FwUKi33r|{u^qi(4X6JZxaN!*EDG92; za@wA&m6a9;$}>eu+=pYJE-gb#fg)4dCS0x(#d2@5AEEx4`yB6a$kMx?EP3K{0nxOq z#eO{y6lOX2R%g2zo@}Mo7;z${d5$%y7ZQ#>KGsE!VXQ7{U*IR1_#Hd&?b}brzE%3; zq9#(;*+-^ML-NiaEo4mZA%nD9nzk1|V@I$wXf3F;oFAWOSXnfTL=8`5st{G6A7vo} z%2J-4SNau#Oio|6M(b{dCRY7CX2bhcY4lkbYU=ww5>x^{Im}}Vy$Vel9s*8O%06xB z2|L}i`;L=ek%-z)bjqD5FU(CG%9SxRYmtC?o&rMZrovKvQ5ausJ+x1lw~ipB-m$uN z$H6TK=5|pf^WJ*KI7ON7af7ju{V$4|j59Tfcid(BC9hd9EV0;}Ee?+SAbM>D?}yAB zic)_4Mv=TO)yLW z8i(q^4Xqcud(X5CR6yWkf)DT*VN>D#9eHcyE2unE?FwmfJ>?mjFY#{e6(zJHyi&x8Y7d*GO z>^b;g4qR|tW(VA8F>)h!@&IHj`MLoRDA^AWoZC$YFUO>7LPW+_8nN!94bECzie~g< z9fe33?l*m*z2bIYaS#c=LbanrF7>^Hv1bhn(2?=Tn`sRDMO%Ll?WS7jRtr@vVt99> zq)^&2MZ#{M$Y?oYhKF=nnjyrkXFQnsB75MV-}nIihxc(%e$Z-84;|_kEJ?rLqco2x zGR1^KzdQmPcYTJ34^vU1m^jL8MRuAL&#|?O03HK}e#HGujD)gQ#Y-R6l6{Uq4yN~2 zBPEL`R*GsS}X70QUo>Jm>NlD#BHIqM&?fFo9WuKU> z&<3ZDIc-DU^J6_;W)rGxs&Ly3z$?>yiMxO*xQKg70eb4tSPkR)?@2y+?YBkq5S2iE{#_g&j z*-GwON;TE86cl%#1cHDPLC5eOq}myuU^xd;VEhicA6eCfKBoEzTtOiiD|(`{`8Fq1 z!abf7Y9kBI0iOb%pMOtcD|{$61S=p) z57rEaCCIM~{T{Zk`{1Om*`>(Z3c_!fWnxCx)zPFUzc_?fSUd5e-D_Y#O39~KeQ9Po zR-e7N$Py6Kb3TYWm<;(YCoInfQqN^&+o-|*+eou>C6cWpZpF`~NdDKbq2hG^4l^!( zX`TQ5DJj!4(mFnd7-e4Xt#7}cb(;Qa`ogJ)T#tDtLL-<0SO$`z8K)X!o8J>nXIvtgB?a^*q}DR20L|ofRJpYfI{3-;L}sJg$AA_+2y=0*10ubP#k6TDB5dyI8Zv}n}kmL$b$0y1-Nsnz>V7^rcL+OG`kx9$WUeEo~ zjv5Bi&ZKvuZtW=)ki9S#oC&-x*H}(OXoOu2HC*>Y2;`+Nkr-+_93-66rYH zo9AA78hMg>iZsKw9@R&}rB!6uO!-bfNV~%Vy3}$TbA9hYzcy+=XlVI_KdzvM)^hLH zD;^GL$l*!5p!w7xC4;$wa3iACSa;M5R#8eNJ=VZ$tM^>bT*Z4GT)?V=2lrgL${f4m%$6 z+hYDQ2Rx;U=gSbR`p!i&x4n!6lNB^%AlxoF-NV8^$LDAZwt6MlXp62_sWy1vg3&pU_adXJ5dKmPhZAjaee{4@YYWRys?6s z7=Fp$ashKv)oQgFs~Pmjy!W%IT#OT5a{hZ$oacPxQGk6+(J_91gw~#cl-{=I4A`pvrW;w<*B&LQKKJA5#r@R`u6v` zdFh6wy~dHEz((}X~5o+qZSFH z?F69#ldv<2t7bsv{X`CVzf&!q2RRj!DVaCJ$#^zW9%8S!Q-NOtY-Ae#9!{*olEsM!y>n5;5-Cf(mvgC(4h}V zCRkONHFx^c@1FHb>yD6pz|$J?Q4fG^G5E=VeoH`dJUy=xpQ~_c(*g(&M%JJxRZYpOa(?LE$pM6`YYK+{%{KKbL`*01zKhKP9Vz zwdIlDjg~l?!y<%sE5EGoISHs&r*I4Kn-&PAU2Jg_0H;}*9;)B+CP(_SX>krjN&<5d zw_11UZT*L0Ve?rQeQHHp7UIWx%aG&FbssD-%Jxr9M|u&7p~_wi#74#A#{L#g`mN14 zMp~=<^CZx(Bo^h;1Zyg!nQenx9q{bR$!~giCodZNmCW;UoyOu1jtIi^NWt<8ts}|! zGOYVt7RoW4Z*q(m5pI;z89ewXxdFsNrV<+*N8ek==-O?Wky84rILKbfv-x)0z8U4A z_3J_7i>5ZX4!n48mps`%jVCgB1AOYx5fmkddak@I_QJZ?DDua9=c|)JDD;c$*v{R? z!QH_AH=mgx6P{(z(5_kVHE|4d9VzM}DFWHrp40xcW%dh#J>&QCglUU@0dB;pNi`?M zR|=XTSX5e;x(9T=wAWB`Du2r*g!cP;2YofG}ZV8Y2KKEK$6F1glTN=Zg`*B0QL#_JFOP2{Kz1B^~K&qMkeUuFZWy z<*Q#DJ=nx+&&5Ca!Zp!t@W0j~FhZ?dQMf2H=Y07 zu6EVugu)~y`ur_j#b!)k72Q_KcnVbi*n(C*-;cgpZPy$J{CF+#L|j|D741DM%Jc6$ zzOa+}wqzrt+oCy8*Gb+#+xm9)B3r~1t z{G*4Fg6IEk-&A$&m!#q@duMh3cu)8>WuiRWp*gF+B4f&f-A@c9D@AwQvh$ZCd7Sxi zC*`YqaixAuFB{FhVF4vV=EtweuN zF0JQE3Ve(x==M7SoD%tS<@M^Hjo$uxAFgc$H8V3;ohq%K<7XA5vh_DMZ*TN8!qNDm z9}Cu(FgWG-1rnTnzmRHrCt&zs8`)VhO!#&7oUb*OI4`4!%I3W_Sty_3#wvLqnY@r( zr6D1^KD#V@ZyyB$`rqeHNZ3w{AUOp7b%fU=0v%lQxfDP4$fG)Jg zbYy-Lo!#3AfAJlU%Z;50R*Jliw!XX_EZ~T2-{;U9OcD-Wgw-ZdzeOUaS=YQ_4alS0 zy@8g%?<2WcC(DqJ1xnegAXi$qe6P2scwFjrWT_tyeXC%u5K;i-`j;EJvWW6 z3kITaUOH(blT=74(AR&W_8Z>a2F>lwEWHMPBs-MOIPWPDukr%q@;hZQJsz7zcnw}s zIFIA?IcerQ?6}uRve@Kcb1EYKaQamOi8qbQD^t1(5BzakXw-L$wk*R@$uoP{p;3Eu zov{xWnAVdw#dcNSni+m+?!p8zOJos9Cqmv>)|O-5pr3&7n!~4oyNU?HP;01#dg*db<}G&5*M~n(_6fpE#rOVvvM3ng8#&< z?^qq12HGfgZY}uWyA|AbQnjE$&&Uk{)d?W`V?BqDizB(#x{;D(~g zwNJ77wEo*}vg<}TU{swp19ta8Je_@06yU~M#O@_^#|lhi&N{|o=|NdiHj?jKof(pe z!yB8+F>x<`MyM{s8(d1%!YVjT8nmaW^z$GC*;sT^2N=KPW_^&+{|`x&;VJ>w(JWio zCH)j7c#q+5C|Biab{AS5zk0}EAE#4uJ0sRk&p_KbsiwvxCf9wuII>?az`4|58b)KT zjF$sq)L^g5l{uB%0qI~7EO6kb_WRuvuud$T*#1himw5FtLavkNr1&N;wPuQUImW-> zM7bn4P0+FFUkuCdU)M?OTid00HCYN~l=t7W4zna`%2+O$ApPO8zLb3bc^`9RT~yT=DOyn3&*HBl93FvK~3p~M;aJ8OCMv^FO8DS<{q5vNaY-7_eW5+HHxX`ka6)x8=ks+Bpo z$-5tZ@GyGbItaGd%JlBS(nxTbvjWrJ;-ZHR@$ggM0z_7m8yfYs!SWM48p~`(2ma=~ z8LMWKTj{+wmg4wJce%D+^RGxI6Viy!W8Bb{csoiDs&7nGseQ&Mn5Ut!HJ{`0y(NDyYko|cG z>{5k#0e;XdDJj_RjPo{g(^4msDsk&qI-aRhE(M-YapkX%pKLIjmcmWd{jr5*bE08^7{ zj;yiG`D#A_Pm|tsj{{1T`u~~@DtpBu8*^A z8vV31R|}z5M@u40So?N5Y$umw&+i=+_c|C7IL4p(p332*HXf;Yx?rtHYf;qBG4%kt zuuny+&U^M|KkuH;{%n`n+XCOU@1*!mo~ow}!CTE$7q0R(4QF8?E6rt>Lnp zHdsX{Eh#f=^Qx*wcYtXfL4pFhO#7Si*4!L^eZQUs7?C{0%k5>k%KN{pANv;0Z7DnerZ_VJ~m#WLe{Acelm<)mB%lF_*>j` zM|6MstFjY`P^ydBl904NmB{^%N-Xb#ehP*^fKrQXykcz!T`G%TM4*09UimS|6IC;M z+$)p@OS?nC{ExpLpn{I0Yo?8+yJ;E9(i`mclH=e5*ma|CTt|Y-DkJv~moSoH96q_w zGGtZC$c0*&gPv`j<@KDG-?%wX+1jZhmLdk-(T$EKuaGMvUm@{2!r=}KwhlpkfHLDl zeT}RrblaN#PJMH%ST}!sjFJo-x&I*?G>dUYp=P0KnARuW!hqoZ<^Da}q7>VbdO##M zyazt8-*!DZpw39h5j&~IlDoZdHPke-Gmy5jzlj(c6{k}NrSKLv0hPq|6o4jeC&g++b(=oQcIjF2r9 z?>guXQnN0+4oN<##1^3j0YoLk9^3<9EbMc8%CYJC5-(8+{alkvCWAC9jR=mOFHQWW z$zfv`+@^fQsdg1FIvxt$4XPhZxL962>0{IUeVxP%;9fu=0)RR0Sj1fzbj$-#^B4_4 zrBr*|M5(41T7Bq4O)iu$NVavPTp5s|0Port3=a=eD_y%>4AG8bLf}b*BKhru1i{cb z*63&+DJt2u+{7BlA9j9=e@SW}w0my=1GH`q%#J58QX(dwd|!q;!qwwY^Lvg?1LV~E zZTfqAc~$e=Y`;)TU8^xhsu?Tf5*Er2z+1Yo`zfgTBI?f;v=O_c0Hq)hJIXaqDV06S zI;w~pz(pKy065K4tDihZlY3R94$((oRHGiDBNRBE)UfqQtS3*c?JxfnKZ zU^YP@|NH+AJFtfS9`>-3@^YgnMdjzTF>w!5a(>!=e+%AvepVJ$Ih=rsdhJvrS6EXK z&{{Q+k%v53KmAwpFFKL< z&Y1`wA4hsFVg#|8a33;PyeO(UBI5Bb4X!~`cH=IMqjFtj^5~Y<`?M*=?$vNg^MIx% z6aihUI3RC4g`it$*|LOm~ zY}Ww-eQNG3bV!T4Gy%GQT*5MnE#amK8;&gxIo{sR69|t@+))A>zaROglx@C1lNhXLj_%N{)Dx|bW5@4!=5Bq zH7Gb_iZDl1Rcg9nq+Deh8iKUBs)J3B6gnxC2oyj(5Mt#M>16^s!Z$-1l1ca%h8*2* zuJgu*OIjMbD_qkv#lUH>8hn6L{?b<<*SnGaTUWgewh~*jkK{M9L!nK7J_xT?U`1Qg z#1=0VW<3y`T<6}q~+rrMV5N{79%-uj{#CnZUA%szg;=%(8HVnye6Qg;|Y zfO{_vmODPC-R_vX8EfMFAegzIj1OA3%5BNph~|!Oe&+A=!avmP79Ef&pHoVI zT&Q0&^VGQK1iuB3M@KiC=$S?-7NU}i#!H51fTv$*1*gu*ym<6LQKTh%Zqw$KK#HuF zlhU%Og{@`Z%IUpxMY3_^GTSAelADoHH|2QxH#nx0@mRp|*SPu34|EZac50+GxvvE| zc_-G2NAIzZz7#LrW$Tsba!|_Ky@IGH{|t(+>YXftZ3bjmso%?6$zEJ_FQL4xLv2UA?Qs_ zE$S0=!&PcBSq7u<(>73S&<5g!WE0_xPX)Dlx^ZlMU_>8jZVu*$HSi`eyfVsV1vLC! z-v{7(^;ZS2H}-0eWUh4n?)<_|#r#*u)grzd$rm5c{sOt1xay<2hxiQ_fN~Jw5drhm zv?dC$N`Z~VbU2btby#4bYT#-8t{PlRY;5d;MkKO!PG58jqMm{f^D4sg4XCQBV!NE& z_L(}iX5_M6KQ1AW(XhC@=!`7|vRpMl7jGe$+gJ| zxeCR!>rXghYQ(%uc}^}g2rwnCldkh!ms%eP(K(5LH8tjnPAce+MqOPiHfC?PwYpu; zY;qcxrhz`XLqo!0$QAmJnGFZ+7D6j?un3O^UCrUQPX4pb3HI_u6%EL?Q{pi-k;HfRQbJ=u& zwUg^>zd;q^81|ZY-@i=i*nhjMDjiE6mvexcvD8BQ5l2nU zzC_8~#JMO(?w5`$Gkj-Z-3}<3nc*EtDGzHnu1(e)-fVee3~ZY!x@9XUcpiH5LS0O3 zMzFO^<7Kj(#r{}N7zTa%^_4cJ@WPo&>$A>DWox)flPbpGowef*-5sMa0KQ;0*Fr#R zB9B|K zT`^xOI<`g`_T@~v4sMvLW~;*eqnwK2XBAPuc5$znJ}QoX3uErjz>b@#)xFbfh@2i6 zWkPwgt1h1@S@50p2XH0vna2&Vrp}KyhJn~)X?LjUBtgg2j#R3U?K7Nxol}n}Zz|Uv zd+IrHqes8-Y(LuqY0viTAq{sX*~<^WTui3w6MrfTTZgFUR0tf_=<`n_#yG!>M*?@M zZF1xaO+S{3gM*#pPK;Uje!_04E^z-~tO^p0{CIzhU1002?Tbt0<)Tbs47g6c8qLm|-` za2CogEcjCgetyfFSZ$KSCI!1+{Yz$h9U z=Vm%YzW8o&uQW%_C#({pzJkqmKz6WsQtivT*SL;&b)7j9FD_ZwV}KSVSuh?QdArpP z$~{m!88_k*x+i6V@wr$IACoJ2x2iinH6)2-9iscMD@wGLN0SwP6c`P}ous~wt`ive zL4&@)i_Cc!!B-u-@i{YFhq~^zIj1oySY3IZ-eB)J#}6^6cme940O#|BKratDepsh1 zeqXf?pKcop4MjF>uz|s{M;FV;N|#%?de>4otO2S?{{zsP7mja$KXk%Hm{;e=YAWo6 znF4!_;AK&GK;TQAzailuEfk=}IzgzZ3k3o-WV!QsiMU3g*FSj`Kp@EsD3!#{Mr9BAPgs0b@buPfU=Q~ll#KAcCg zHFSGd(G;c#{T*z-_61OM1@W*F!|BFi9!Mwf2ZwHsch?n1v~Tv~a!J%l#>|d3|8yY| zV8f^2PJ@z5*iG+8>3Piod z%7UTmPyb1r0BYOZ`c;Y)muXLe(9Ac09J}_&%*#A#+?;X};)Aa#HQBkUh zG^wJ9NR!?Pii!$?iZrPT7^Oq#C7>uERYZCT7C^f6-b8u{g7jV!N@$^lx|5*K=kxCS z?H{{)?X~$Mx#6CB=FFLM&YYS1{Lbh^P@eFfT+!aVr6-=n!VpOymSD%gmvk7~1t&XJ zD&hahkGG+Y2s>CLEc?KSv_8wLF}iv#$aVD?ruf-^KI#?ha@1ZuG2oNrI;>bhkzmRR@!~*4$2{-- z^;hC~f&&XAiqItC2V!&Lw;Y2XAQ%9za`+qsQu>=RVzIv;S@u*qER&d)w#VSe-yOEv z#G@*4ZTrKgK~`5;HVIZ&NtS0i8#=EM7%ot$JUL2;(O4W&AYJ4JSC1LgFLBK^?=KRC zT0G+;5^*2^&?0lC@`&>VEb;@;7rmHlM+}N{ZFpEe!qmHFY-VO=dwY9zb@gK;sLt;W zW9`+MCxJhr{0Y_8JIlU7chSZVc9#stZmLnSCq+y$#MJCKsf!XS_gWr$gsv}{A@&X( z_Tl|WN}5jyZj;4D)z*F(f}MtZoV+&xES~ zJbSqcBHbTE^$bq>#<@3d|l6y3Z2Z|P@1WMp*tdYsUJ21 zWk(45H)dKiZyfI5hA5HdV?$XXq+84!r337?o&9Iaqgo3$ZIfeH?Bxmlreh z`{4P?VyS$>1{d~c9PQy@hWhA8hT4FDh2fppqNOausuK70?b?>eB_#A8O6z3<|0dMB zp;w68_)&d+D_WB1qe0vZ!ZdqN)l3y4IdcE6f%ZN`QB6+W-MVyWXzodTQNGC64QuOj=r6Rr+1iKpTP~7%Oi=**-nm zJ4$zU}K}9jTh+wIO^`avL?LqV-sp1q;g#n;4o^pQG-zqE<2^TO207bu}Q zrfTv=6-ExVi)FH=h7HO7qQfVw#boyO+_{}cNq5__4r?W9yah9;pQ!dYP0$JA)o+_Y zG~>Vj0a0lHi7r~=tNyW4>_elYn;z`O6z4@D%cJDye=zP;8uHN}@5{9w-o)40IXad( zx#Ga!kMl*I|h+2LdLHJ{-Nc9Q2n#a z`BH!>D_N#!dt3v6J}QserILO*=;t}QxafHCOUhRb4sSni^~K2$Cw*Z2@heU!%faLi zWS=4dW@t=;C#lfVgM?CTbvRFm+c4sv%25YyRz3N!pDmzPi(~}}^L^4%f!$G@=-g+a zNg)8!Ikd$kOFO=!LjryV0Mg_Yl$DiLZ(LScDWJdwN6Is^EBrnyNSzbf!055f=SqMu zB^(^F56pptx3*iQgt7%m?Yis!v9;*a*Skce(62D200Pu^87<4Ed-5YBd?8--qTD&$ z%-XF(W2=COup~mBsw--$avp~_eNi$tKoS9N6p&rcL;MmzQC<1hxj{q>(f_0Ws*kUK z`vcGVDXwKT+hY6- z4+VI4l2?qD+m@VpZ7iEXP-w_r2KZJ0+gDl^Sq7Cn${7M#{Am_*RDHqTuNj}OI|c`{ znMaA=WO$gGf5q;OWx7<}r%HX(dfGckjc}z;_R#q-ZDThz&hU-s|UPKvYcIfXM9tnKtI}4BUS6t z&CnVQY%LE>q{keP)`7yH!xmi!SZ5Z~+^|=!uj#xp(l^N{Zgg#nw#&efMkZkwU68rr9rlF3{l)&K1YlOgy zd-Lb^zeyINn2)UEDh$YXb01mJjz8zl^qejshzC(tz-0AJ1au#+dF>MPtjp&!0*DEb zwY-n6mzEomz~C9lfASVOz5L**_^ zVK=E(II^B++?EGk5B<%~5r(fTH}0ZMD!Aa6CfAN#kgIvO;`3QrCvVAR-PTzQ$R?wjymd z588IuIK3NhZ%YvRkQc){X(4xBQfp>GUWiZ1WgW+ZJSKS1F27oFBu>}8ske}K8$Y*X zDfa3%Mo)^@yz2_g>_KO^@FLznV^n@>*lWJ&3=Cm~J_yq zEK+-$Ux(iUyECIUd*h~tgIM6xTC-tVCMt&w?5oy{U>i@DMN0%OQ^I}6U%6_(E!)s} z<#?pNL;t#jp_3Y}?6Fg;*lys5Qb+Vl+PT-SRHBNqu*jXC*&DUcwU;clSKWWP?5APX zpc1XjO+q*ZhU!B7);=lo%YvDo$Ma{nEQbsCq`Z;67Q<+(G^2*|4tG7c`6e7Un9Y(}$G5ECh!7*RRZ%cVpxmdCYy0~w-VK?^T8soK6BpplrNbD$GwOvCD8@sP4 zeB#{s?(hbRc=9FS*CcM+Ps$)6Q>@Erf^%y$sy}_5Ms;r?@dcGxf+Vw>%NAe4)8St8 z3(w$jL)EAZxUe<%;Mn&YyXi|GmiDqzeBR(6mm*RmPGrm{+ik7URfbR}l($xhSx-BT z>Rx9u%N|+>vq1YlXCS;h~ z#=G?1=C@_a$@+V~6Swl)DWS0{KFHq1?6)(O!P1JUz3dqq?3S7-Ir#MePz5k2D`oS^V2cpCstln_l2c9OBe>@7Ev$5 zJ(y&{-rg7CyVtrR7SbH$HthpSyixu1N57Err`s=0#BFTAck5)*1DemoDQ~u5^VKzG zTfMszqS>Gi-gDHaN+xJmmYuU$Aou1Jd0yGv6KGTPW;*)T3r%7Gtyv7~EgaB`kE`@<8y>VDrlZkNjoXJ+nd`OvrXA33!?ieZVJvbw2 z+h}nd_JESl+phhT?=j|+NA7;iFM1bf5_MzpD5DCVDz}fJkEa)v85yNWCm}0E zn{$|a-zh37zV5yzto?Xd@|z`Hje}g+OeS28gPrEyIP4vx=#q+kCSo>{WBki3Anc_KQXsr>Wi z+zbK)YzX@;lxt0AY@l!TjO*o2BSyfwcjIVPMde|aBKGW4K6`e0tX;!4U(e0koQBqV z*VHyGzOidDIEs~aVpCJ;f0Pt;a+gAaKXWU)ytkd3)#6jR^w`{~^fNS&mEcLKAVD7fkgu&aCS{{Me}CUGfZj;+@GX;)~(iUQ@R0r)0HtM$g##!BODx zn?oxJo0Z&xZ>>u9W>OFwRSc<+)`vWyqtpY1)wr@! zbruwbWE0pyO32apmQiXUd_SeO^PKKxW8+oqwo69&mX2~TA|h@BbJD0qDl_}Wv#|AB zSC5d*1fMh(*D-Rf)=@5>s{AHK4$YGse)Szy`ha?7q?nGmvw*eJq6J!441I5PcIpz!J_iZ(z0rm6<2VtST#y5B-aA@ zHRKpHGKREjTR6t-2G987~Im4eXrY)4m5J^@p;olsHPvmjmhu2mRyOs)C; z)Cokr3g6tU>+M^agNEf<9n1hYrc04J|N5KjW}D;#J%!Lp&f^`Xy>AJ?vC*$1X4_K{ zC#}|gx_Z&_cDiYM+1g`a<1p+y4)Y?-?Aj5|hBGbN_oD+^Z7T39xYF+=@{a_QRg<^J zPTGX}Amddo)uL#7w0vT01>Fn%Af$mVO#VnyELJxESLDpVy7z03EuGVkRV4h7OIUhu zcHVnyrnBA3l{~GpE&Oo;E~Gc1Qgd1*3#+dz}Pp)mzthdA=GDX_aIe*xD)NO1vUC{j3785ePDAg`A2H?tSVk`Q2qoUo& z=fBLKZ_1pnxiUwNuv)fHq*2ANRHJBuY-K!`MHZWk9y_R#MUeLZT$u&gmd*{$qesa4 zh11T_I%VCZlg#$KtlcWw-31c-K1FpO8RBhuvFx&vb-9rIvQp{O*R|dmk2k+3`JCul zwoErnu}Kt_Sk7}=9!8Lt#0i}y^==|fyw(E`r2x3HU}Tx5wNRZEz?Hdg;T!K9WRIk! zszl|Q`JOR+)|^@NLRlgy1nFR+Ar!f!6JOc=)%v|kZ|t(j7umJ4;N;cLYw!|g0Rl6| z(aeTCStJ!n1tU=ibLQ(xwTwaIE-onHpfC9!zLYj-$WWa z5b4jb_*M1SrF$pjt@M+qqIeKK2J?*{Vm6LDF=?jdnVeG@bdnx--G9%n6}54s_{g}| z;)H_<`#R~M$Xn6Ev&+Jt8|!_M=H?!H(ce*hn=u1rNf2 zY55>u_|DHuNX)aTQ&@C5yx)?J-idY}7NvWv+4js%NofqmdfpF7TBT8=l^EB>zT@_6 zlWS|Qh-Zb)E!}vUrA2UTQ%sQF^d&NO)2}2Foo1$p+VS4!x^EEJ9(|%KZZOu3n$kcA zx`k2+!R0PH2WgTQSA=!sR+*me$1KA$0e-10X1L9h5pR9rz~xIs7cJCdAr&b{<-gz8 zAV1T9>OJ?LPEZUZVt?k%Xlr_}R^>h&LgwTI+A)4m&*taxptI00wZ2}+#V^lTMPLpK zdPAE$(|5VtM5gU*m0Ji1Hx+ zcr1@BOmSrW{)}ENC)K@p{X(5@zeIkZ_7Rt@rMqg>X9sU7NKe1SD@(;2rP&y*#e~*r z)?5>g7unJoF}19FXQ_peyBv<`>E(#hIn}9u*RjGR_Ej#7H>V`)?v|9~Tj^Yi#hQzK zcUAU^};$W!Q%84|}FjV4B#NYEkc-0NScf$JWX5 z{i8CrW{NVlE7D~N90<)aY>+naBzZS`??2M6pFX+E_jLgjpW~WrZQo{l^T|^#C(s!EGX^T;ufN#p=cdcky6X~Z>=LSYd|qSa*=X-s zLvw?Al{*dzX9Y1HFZxs@SiyO&yF(reyY|Z>(z<$A%D!d4be4Ltz16RjQQ-Wru(dZ9 zxL~O@fS)3s*)5%cq`eK^hVi;=FOjL2%Qx*gKd|^t!jfV^(#>s2o>b|SbnCll7MKH!aUAO2x-uK)H5jvKmn zXFn{80(yX9P#bYTvHmv}T9;#eWJA98I^eJpMN0$i;4B~e?I-P%XhrUJf3Dj6(x4b$ zuCsF_ai_Cr5fPB9986}ZWM^kja+yy1AkRxsz9OtZ8T^qN)!L5@ofTB?>&4OpmH$9v z`K}7byjk^6^Se#4ClE-n{2g_rC)OhN4LKz0xPC8A%d&y)9>U@1ul+nDkC?4!O43SW ze=~Irxu?>4%~l<(wA`!rLf3cejp)6olLb zD7*Lc6IaIz^#ECC8jqra@={6;Ub+p>B2CYD88u3QG&ufL9qPTXUBHJ9`s~3cTTGw2IJTHB zo%KiFoOyL|VVws4i%0B}7+)3zzb4FP)B93@S;2P{P4v5;<>3zM;wE#ee&m;joG9G# z6`nQEhOHaQ$v^cS7n{~kPure6VEc6DSLL+HmQ?ZBwCaXb`?|&q34fr%GLu$;@y1Z` z9_hv04Nbvkau@)|i{(d9^}fv$65qoZi0<&Lz;mrHQVda6ju4z)%FDH=N5?gDF{3Jc za8GN=i}vSUA$R!swOl6M%IdDG(fmN&G<~q;(y@BCH_zRvAg)8AB0~E3=1Y-N3xShtl`yrlk2g$dGD2;v##Egd29YRXLKGWe&L>_wd$ZC*P>h z)(gyBh<(JM(x(+%i0AsRx=4fH{w$btfAiX`bPL~5<(@W8!hcl`&DD#RKPXAhst`u6 znv+9(d=+~XQJ+8e>ZQuKz+nDLo|pNzk*m?sU58jc z4$@ckHPO!GrH%=oIezk`$#4fm=h&-rOoMbD96HXo?5Ft~`dgpQoE+R>Te*33>Ll%> z7Ahz?`<1sv)bzrv`8o%YSIU=O-yAW=ktwEejZ#^;JdrD7oZ;HKc1gm%wN9^Mufx0- zdVbl)MLDyKyR@|QlFY73j8hMD&>NbI3fpHmK07-C-?r%3>xd1fdCGTjcvlhNBZ_^Y zfOaX=Wx)+SzXx`D|2Dq>n-<)uy>`ElY9h6at_$ zbDNvf)TvsGeohpLW0OU=5D`AVCIwnQWXWdU=+vK%mq)>_RjLfEamBuZ=Nw>|T(sSS zb;oK^*L%a+RQ$9f)_N!Lb50ZUUEyIaeNOaAwkPOssTm$aTxu#&S$2kbtUaP;;8_gE zMf=&KRqJ($aj6}?hpEp?Fwpdp3iwxSKmwkpO##i_$=c0`bPdm!@= zx|$P#PD*NY)tjaD{Y9%Kp8l=a#Sh+Up5eT0Pm*mtn^d^=<1d|4zl4%Y^gcI#l>&ayv zOyagaR`-VG3y1Gbw+S%9Py&q5rTlY>;8EyjA-LxJ*(ta{pY1!}H>#|L$WLRXJrJe( z^iueNt+YqWDF7oZn;nDAyr~e(T4$;93U!jm61QxsH;D_sW$St?kd#bUWHajxjeB+Y z{(xNjh+*9NwR39fAk}k#Zwypou|Fa&Uf!iLtsA|;=Q7>9l3j(-T2DxwIsqQZy0vi7ER)TAHoOT?&ZNj$dR zvwFy+Ii|9bmnIlcJ`#83_$JseNH``^pp7_q&+eP~@J{K{doq8PdHuzkNNN3tyticB z=4I-_N23h0RKf0JrY5(9@N*qg^v5L6*fUUj@sB%o`1qSwu_`Zy@2v3V(RqFJx}%;M z9jU79s8QCaEhkl9)7ls&84uqxe$0}VdEwBda2vrT*e`=krI9CFQi{ZlK`=!qnI0-P znj5hyS;e|cetob#bIB61X{x2S5sh+Bo*hGHx&@zRANb}b{eeq~Sl&M0)EJq)l$gEOSbn%6ah<}7ey{)Hk~Usu9aT)epnAmLTM#I%D5Cx)<#7r&on9v1pt9ThV_qMh0AS+m3un(*Dn#(} zp+u439s9!Kz!JZ$vrS2Kd^sZ3I~{K>@t=Oub`DAsdx1^uoUZ4qq|NTDx0EJf8UboO z)lL1G5@3a*UfaB!r&ppnHu8MeeULfF=$_|Cy`+*`s8&CiFLJymT#V_%uH8 zx^Whqerw~fbKR*Is7BzSfn!XhqoXOu=HbFGQ91JD7PXTXZkVm8Twu0d#X838rhr3V z^gIj3(G-7}s254tW(8Zwc-EJOv|IOH8q%{lPwkTnR(!J^hm9wxcrG?vB(+=qD0bCn zp*>4V?HE-#O_$cO^JG6>1!iA6N4B&wfo5T@01M_>^q8x7O_l#F$!zn{zBH9`4f`V9F0uh z(&-bveM@I>t1*&c;i;4IkBRex<@8=g2V+ewX79(yx7Qmj2H~P-F=>tYYmRqxq=Ei4 z<@C1PTC((<4+{~I*Pa}5W9V1}ONvmRYvShC(yqB5U=<_IEEG@}H`tPf-{Ty@-(@n= zxW7?R;O@j4;Nf^nFTT%Zy1#q9)=SVe3sVwp>o76Vb{3kxhj^%SE8|0Mvz~0twFidL z>12pwVfJG020wTDA3XK07r`1lwC!gOLFx9_g)YuFJ^1uX=S0|N=GN?hP#iP2G*4Sa zXmP0%8+FM&u$rHaRuX@@yW0WI-+hY|EXEuqtYg8jf!*@MnNFRy4z)ZJL0$M9uMj{_ zE$X>NO6@ZN%S|tsAM)|K<%ML%aNo{w_#&BnO|3iRD)}o+^8}szV?}8Div`z?;EDG( zP$jF71fQc>wfC2;v1PSm>>4Fhm`_16`C(Hy4q;*5{Z5|4#OZcn!oRBzo z-D5&Wq2%kK^(+l3)!M>BW7?0H+!N)$-S6jSzJF0D*+5d{JO2%q(P6=mkH$frCC3BTqyZ;YT_Q{7;;9PEfzE_OkR;pm(N3Sku`c zB(ri^k?w*7ZOb>Bq6M7^Y}MlQQ+HIG+mHRtNPDbu-SXrjIWMG$BvRx1w}UE zUnyO?I@IO8lYS1wU`)aAQU)LrqxKnN+y%oX0|96X*;_Z34*~00SpQm2l$QU<@+`RL!B8Q!7E7pdgn)F4ObJYz7ZAL zJLlKs3nDwclAL=MsI*$=C z*9FdFt0DZ^C+Ob31fecMx>~#@Ba@q8F+Gt8;#a2He0J4(>8|@7&SR=a zWV}2`2+3-tG;U{DW!P^zl4`BByIHHXsl>IRE*w4u=K~qTC7`MCU#JZ|CcDRkw@+;A zeGWMNW8>^MlD@;dwjy=Eo|qk!Fd)eVrpFuDbOy97JV&AukYpghg&3du4#~`}WzMb{ zu)h1m%+nHP>54G3ursl;eP#-G*)ihPxLtgU;k+M?$B6XXVFp~7@~k^o*r~4AvU&1! z94(wy*_{0C!X51AyST8&YbmB5ke{I<(Cysyl}Idx8ihfr;iY>1G?tFFOVTHcebU#+ zkX8b!;ZG8&`>@MGbKVAIZ2ryWH)aABQ?<{uUqq&1?S3Ty`VI)`jK7@t1fgJmx-3a% zaATQmQm;npJIgh>2E{Nw-t;ddI`nPtUOuHaU9dD3>3kpxGF~5ZvEEcS2|0`lA_SKF zi7A`uoMfD^3!_o!-HN{XRS#KDoR-Y0EzQGpcA)>7Q^XHjp)qu_1?(7|A4X z&gVR5@{2U)BUVs3>;({_G>!}VQ>9*{>DzG9j>9mk<7!B)mfS~17YrL<{hbTOl|hGb z&x8(9<7Aa?^g2L!JBU$K3T}&~0-Q50S=Gr_Rn8DgSwdv37b`J5*I?a)oqJMpCQg}$ z4v@OFWDU8?BZYp#IPqm^2Y@A)*x1%vJe^v1d>cyt37bWYDv&_Xi1A^OG0OavMV&t} zR0jMl!uQngroK2AeFQR4c#r2^*BWeplVYEw;uc0?vmsXI0WO;@F9GBF)!f-2q%dm= zy|mEZ|3vXT1?1aD_-ggoPZ@=|HP^lCj}-yZgP8dp0zr3ygx>%B@prCw?*AKq94LcW zIDpApgU7~h5CdaM?(OX=f>@O!UVe?=g$5G{SOqVH{IizpH}bWJE!Q*ZMDcDc5Ta4n zaC)I{gl@SV2f>Aq#La{PH@$UGOSRP+?n0K%d<6l4O9DYQKQ{Cw#KH6hAr|Kw9}}Ri zko(c_XILgDLiOLM_jUu_dS>D%K}Zfn!898^mtx?2X$LQ`EOu$g+i`bfu=H zwZvu2m%fN8v@FY--$5|?*s`eKMQgOXJg!TzGv0sdqR=iScajhs=;cR; zDQ;rH{eYEvo6m+0>5dNkZJzykBouz1PLm~)-*ZoTbjo@IMIpC;X(qukE^ov-A$l<@ ztu^qZlZ1!{T_{XQgvR555MB&Er)fl>$~PW$w+_j13g@Mjx;+~KJ7VrVa5>wWn@hVFMD|L;gvh}Y^rdHN5l{@pYL@4F5j{2W#^eX7g z-{k*u9Z7)8{e(41BF@{XRE8Li@1Q8|9%E?g9~fM{IvY9Dmw74uCAr+Y29Y) zu5vB8qESK!11DnNNEMt%TTwuoa1_~ zD_TUy;(jo1ODAyR6VxwaXUsdx@2*UK^c;Is-sYBH7n-E*Jw7fXs1h@3HytA2`E;~q z-!A!|!&CrcR^b-s1RS|jva_Y2O6P->=rp!+ieRd`1AF_e;RltQnG-V~j+=M-HH;>N zkM1^gtJQVUXD_~8&nYR_$j ziS-%m#x^%}8##qmqtD`hx?EVf7c`hLx@-S}CS?8ssVu|Nki6YuPFJ?d;}4_At7W`Z zw4|zNrwOGE_C>V|K0YYf<@)^H^gd?&@A-dexB_C|zJ{S-YsCo`1ckzMt^}JHlse8c zqE)KI99$47+M{J?8#9|PRcuO14RmrcRkTVCU+%y zzI^u95UVFb-hS=IzL0@m{Nq%s%xQ8f537d=bZF7Dv$~%?-(jFB)hX|i8GRK0*u|xp zp)5(udZD7D>o+QlXzKA&j%U!_8=rPY**p;jGOPJJhRB%5smOD(OJ8R*%Qlpp1yo(z zovx08qlwk%GhWCxN^OWTkU#anX}HV4b=I$|5s)W;tnC!~c6r@{w%L{+^%Kqw4tgjL z$!t?Z!eD{W!B*$gn-^zqUJ+Pik$0TfKqvo*pW&r)2xc&Ei1F~<4u~~2Ttex&d`$WG z_(vq&E^LEc3E6u!UnH+PAPmjUutGuQ_ctZEmLP0F4KLH^=FId(qw=&-MaT};*45&A=pcUmt z#nuKj*shjI4QDN*o*~b2+4UAJ_3+)fV8n2I`?jZ?)&A66ioew`+utSx8lo8bdUEb@ z)rFv^bMGDMa)Am*;9ZKxU(ZPXk`j9x`2sy!FM!@+gwhG_1)G15%bFS31M1z*kUjA2 zvrw4ztmVp7W4G;dd%Xe=Uv=`o>Q>nC#>5MHJT#eXY)x(IEYJ-QiRErj{nBv->Z^5$ zzj^70OnykOTGI8}^pZ*cy*8L(@t({6ik5_|Ayh1?olf^F&e~Yr!CkK+)hVCbdZ?27 z;L0k{_-U1Y51a?-4o*uK-K!B&anSQ?I@GOb9gEC+s%RWnIq`9mb166!`I%FRq}{r{ zaK^9SmJu#Hm%52QOYLGi@m+KR$Pc(bjn+?<5ONa=v|z##7rC*LB={ zGO|9*FzOqJ{LCgnsb2ht7_L#r`R5@&j|JP_bdz__NcfHcaIXUfp)!2wW;g0-7kp9( zbu-J#C9}VrH%QWE%Gtt$&2k#pO|y#if0j?c@D{(jCfbR8t)uT*G1)zn%Pi&c#1wud zFdC`2H#I9cQ-W!(+nNw%N9-wHX+<^+<`JSYt?J@TMCwvS^awUP4u= zWaxH!*k4m5z9a~!`448084w$6zE}A8pt@=89;=i=$m?>4-u)Uu^V$?rU$Y%5t)iZZ zwm}(3_A9}!*xIU_x>owMJu#S-6>Haj>0Y#WD3Y0Q>JQN?fxNoB3U~R(M*=f5K>q#X zuWguq{J7VFodB8iyinVQP^I);XZ|9SVhkx?RVAH?HYp}a4nvsq-u z?q7aM9|SmB!<9!fmaE8425>{fd z=lBvc6l$lEi04!P?Ik&1M~oB_(Y5c8n;r+vC>|~_Xn*z~AYP|dafOy4VBKf>n}-U$ zyLxx=t%v1^5|MD@(6|$Pyg1y}p-1RdzpE!UgN`FWvmpwoU3W7SIF*XH&d3^FZ$>rP z0?Hu}*ecGJC1e$++?{~ThL#h>{Hh8f>+vc)a76%`J?{#-Wz@V|o;9zJ2o`DvOCthn zRFH?A6O3$G!k4&!9Z@*iyF%vnTSoGuSKaSZiofogusT7yCa_^phWbc)e<_sKMEE8H z?wxGs>nHRv?xRO?vvN9{weggi_nHD?^cw4Z-FH#tKxE2^N^VY;Z_$Z-y+ zZ^>@?=+XfZwh3GWkw%pA5rT`BAqs8yo__~nHCQ->uJU~{#0V$?rYnE?a)mRs1=|T;xW3^eisq^QV0hR ze0N}sKpZ77W=vuReKisZKX9BuXU$il(b?JAw#K0N*6!|5e4TS&p=GZ_G}74l1G1MK z4805JqHhac$^;>RMv6c^p$G2L@%2HVe`EHt?W~CQUA+O^MJRNacMvzdySt`H6zgT} zR2;m*vkWW`AksM85FWiXPz(;wgZ#Lm!;TODCwOklHlaL^gC5YoL-rD6tZ5&+bJcSn zzqjWxL?Hhu1z&7YBs+o%9$PexCIjp=2Xcg)#-Y~tv0Lp|DJFr%hX62U;7j1Yarq*r z(IABas4#ho(7{?!C@RiVV5g{d+=fvCl=NbRIDNDj$)W=qN z?qg$PW4mTWfgpS{YnT)|QEh=LagiCswT`1-AP^^Uczn+4;g-ihJ5;c*iZv!#iiN7s zCL1(tgx&!A<6WX<-;8h@-)jj5s1+!qm%|kL_)gL0jz_PK{^GS&49WU-MX!69s!_ z*6#B|n;wA zHLUy2&t_fobA}7>0d^>ZGEs98DM#$(pheV)$}bHl6Sa=J$|F(F@@DO&qg4c?(P%Ub z2IDqwi2vLUHTT@y)z~EL^o(UMZzg;81WPx>hZW@I<>A`v>gvFIMAekMehYczXbBv> zI9#@uH;C?{TR?c(gpsZGWZ<{qz)&n<{e9JZ^uZxo2lkm)zbHqR-p0#{}9I_EBxv zAxQFX4S53?o% zE#|3eqR%A2*Yqq=7m`t+F!3#p*UljRh-=Exco9rsN4$YQ6@g)a4wbW?QXZjmo!fun ziZ}FrTn$VF>}HIM4t@VD6G!SAh<9Eu2ZNC?EZC z7S5f5sxl?c2S*`{?WoVB9R-Aj5U)oTI+%H+33)FV%}ZbOG|kDfBsiI&@arxnhlYI2 zXwf?IXgmof;%K~YvGVJ7stVyz(ex;SxVKq!l+!kq(54i^*W$2JC@TqfkBu$-X|RnI z2V{gz#LG&sBb>>;LeXjQR~taKL#pG?HkCX=3(}*cT-mHpwvIceQwUpzdJss!BW49B zJ4iftF)5jSd!iOiY0pPxe}Toq5}Y8=S(~D^ib|Ds8o)PM8@LS0idqesCzoPM0QZxU zKqwL$=rD2DBDw4xF>6M#Zf&4P&i7ZY-gFQcFM+zU4JS}r0>(9>Y9ZB-(#9ie%~zmB z3>abWMJ3Ea2X9CZ+x}_ri0qZQZmC+hx!YoX5jvj_NDf{B;&sHnmKMX;A>}PVQwsGl zaLL^$Mn9;l6(>B9<~D>g!OlKTZQonot;3Y0%JrhGV\DeaK=@zT629bDJ7B0A#x z6tiVdg#b464Tde}hv}ltoCQ9k+poxaCdvKt$Wae<3I>2Oz>J@)nCpoC)YbJ&Z9fw< zm9U7>d2pwFZ%&UKH9&9x3BFzJ#u0nPmsTMiyn?+3YqFy=ge||(?>HixgO-O<-=KX! z4hMD)P<9ZQt#Z$oSxT7p^n}P*gGaKX1W2LaoB@4w+-GTYfhCr(HADiqG@7d*aRYmTnh1!2NPhH&ucK0Dh^$%XTLF2b_TAmA&2uK|my`=dW3Bf09N| zl&~cX4(82@K=Oe<>{Z){!X7U@^HJHiUC4gLu`BjkwiDR-td6~AEm1M=GRj0nkcUx1 z@(JP#w)=@9*U|ioEQ$*6Y1|~(qiU422DA2+55l*4+KCqUvSWB#~!C$RzY)I;R-i9na;0&U9n51Ph`@hm_E#ZyR?>1oB9jF2Qti7Gw6B3R#3-1E>B%mP;mtO`NlW6KbD!5E3SU zZr)Ffm9{U=*xC&Pq}Y_SX{7n0YpoA=&v(a;>g)NY`?oxhCjpM=j2(lPgHF8CYji## zw*#^A`jhclG5QDs23ik0QTm)3H2^<)*a`ZNLf2E~{nHHz(_t^2#G9W4SqUM5@cHMk zO6xCS%n1C{>kU$EzvI9LQg{%&GQRH^P@^X$xHIJ5hD}^5b3R<7zZ9_Au&E&I1A|{? zy+MLYSpk$)yvWiD@q|#kP$yp}nBdZrl0mc=CLBM+vID8|8hk)sroBiFdV2&yQ(C;Y z2l(*qaz0_I+E;}ryw)V-!CRZj2gjQd8FKn#5+TJ{Q;GJzGjSXI>(5RxFq$Y-)BD5K z4kwtuAeOHWC%dzR!WR#(wOI(SBO(qxV=cfk4*tVU*z1IAS>iP)06xUN60eAxTZF5> sF5Ih$w+K#k19+0~A?^bIZx{O@BCuG|M1_LD2gse9syDK(8@>8J03H+(xBvhE