From 7f60a97cb04513c0a754e19951c0e060415f55c8 Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Tue, 31 May 2022 17:34:30 -0300 Subject: [PATCH 01/26] Add Amazon AppFlow hook. --- airflow/providers/amazon/aws/hooks/appflow.py | 51 +++++++++++++++++++ setup.py | 1 + .../amazon/aws/hooks/test_appflow.py | 26 ++++++++++ 3 files changed, 78 insertions(+) create mode 100644 airflow/providers/amazon/aws/hooks/appflow.py create mode 100644 tests/providers/amazon/aws/hooks/test_appflow.py diff --git a/airflow/providers/amazon/aws/hooks/appflow.py b/airflow/providers/amazon/aws/hooks/appflow.py new file mode 100644 index 0000000000000..643ea517ac995 --- /dev/null +++ b/airflow/providers/amazon/aws/hooks/appflow.py @@ -0,0 +1,51 @@ +# 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 typing import TYPE_CHECKING + +from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook + +if TYPE_CHECKING: + from mypy_boto3_appflow.client import AppflowClient + + +class AppflowHook(AwsBaseHook): + """ + Interact with Amazon Appflow, using the boto3 library + Hook attribute `conn` has all methods that listed in documentation + + .. seealso:: + - https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/appflow.html + - https://docs.aws.amazon.com/appflow/1.0/APIReference/Welcome.html + + Additional arguments (such as ``aws_conn_id`` or ``region_name``) may be specified and + are passed down to the underlying AwsBaseHook. + + .. seealso:: + :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook` + + :param aws_conn_id: The Airflow connection used for AWS credentials. + """ + + def __init__(self, *args, **kwargs) -> None: + kwargs["client_type"] = "appflow" + super().__init__(*args, **kwargs) + + @property + def conn(self) -> 'AppflowClient': + """Get the underlying boto3 Appflow client (cached)""" + return super().conn diff --git a/setup.py b/setup.py index e7d59516947cf..fc3e68b3c2b5e 100644 --- a/setup.py +++ b/setup.py @@ -200,6 +200,7 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version pandas_requirement, 'mypy-boto3-rds>=1.21.0', 'mypy-boto3-redshift-data>=1.21.0', + 'mypy-boto3-appflow>=1.21.0', # XML to dict 0.13.0 breaks some EMR tests # It should be removed once we solve https://github.com/apache/airflow/issues/23576 'xmltodict<0.13.0', diff --git a/tests/providers/amazon/aws/hooks/test_appflow.py b/tests/providers/amazon/aws/hooks/test_appflow.py new file mode 100644 index 0000000000000..4e62dca294f87 --- /dev/null +++ b/tests/providers/amazon/aws/hooks/test_appflow.py @@ -0,0 +1,26 @@ +# +# 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 airflow.providers.amazon.aws.hooks.appflow import AppflowHook + + +class TestAppflowHook: + def test_conn_attribute(self): + hook = AppflowHook(aws_conn_id='aws_default', region_name='us-east-1') + assert hasattr(hook, 'conn') + assert hook.conn.__class__.__name__ == 'Appflow' From 78bf77a42a09809f7deece6b14896e5fe05bb6eb Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Tue, 31 May 2022 17:34:57 -0300 Subject: [PATCH 02/26] Add Amazon AppFlow operators. --- .../providers/amazon/aws/operators/appflow.py | 562 ++++++++++++++++++ airflow/providers/amazon/provider.yaml | 12 + .../amazon/aws/operators/test_appflow.py | 172 ++++++ 3 files changed, 746 insertions(+) create mode 100644 airflow/providers/amazon/aws/operators/appflow.py create mode 100644 tests/providers/amazon/aws/operators/test_appflow.py diff --git a/airflow/providers/amazon/aws/operators/appflow.py b/airflow/providers/amazon/aws/operators/appflow.py new file mode 100644 index 0000000000000..172423b71b314 --- /dev/null +++ b/airflow/providers/amazon/aws/operators/appflow.py @@ -0,0 +1,562 @@ +# 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. + +import copy +import json +import sys +from datetime import datetime, timedelta, timezone +from time import sleep +from typing import TYPE_CHECKING, List, Optional, cast + +if sys.version_info >= (3, 8): + from functools import cached_property +else: + from cached_property import cached_property + +from airflow.models import BaseOperator +from airflow.operators.python import ShortCircuitOperator +from airflow.providers.amazon.aws.hooks.appflow import AppflowHook + +if TYPE_CHECKING: + from mypy_boto3_appflow.client import AppflowClient + from mypy_boto3_appflow.type_defs import ( + DescribeFlowExecutionRecordsResponseTypeDef, + ExecutionRecordTypeDef, + TaskTypeDef, + ) + + from airflow.utils.context import Context + +EVENTUAL_CONSISTENCY_OFFSET: int = 15 # seconds +EVENTUAL_CONSISTENCY_POLLING: int = 10 # seconds +SUPPORTED_SOURCES = {"salesforce", "zendesk"} + + +class AppflowOperatorException(Exception): + """Alias for Exception.""" + + +class AppflowOperatorBase(BaseOperator): + """Amazon Appflow Base Operator class (not supposed to be used directly in DAGs).""" + + BLUE = "#2bccbd" + ui_color = BLUE + + def __init__( + self, + source: str, + name: str, + flow_update: bool, + source_field: Optional[str] = None, + dt: Optional[str] = None, + poll_interval: int = 20, + aws_conn_id: Optional[str] = "aws_default", + region: Optional[str] = None, + **kwargs, + ) -> None: + super().__init__(**kwargs) + if source not in SUPPORTED_SOURCES: + raise AppflowOperatorException( + f"{source} is not a supported source (options: {SUPPORTED_SOURCES})!" + ) + self.dt = dt + self._name = name + self._source = source + self._source_field = source_field + self._poll_interval = poll_interval + self._aws_conn_id = aws_conn_id + self._region = region + self._flow_update = flow_update + + @cached_property + def hook(self) -> AppflowHook: + """Create and return an AppflowHook.""" + return AppflowHook(aws_conn_id=self.aws_conn_id, region_name=self._region) + + @staticmethod + def _dt_to_epoch_str(dt: datetime) -> str: + text = str(int(dt.timestamp() * 1000)) + return text + + def _get_connector_type(self) -> str: + connector_type = self._response["sourceFlowConfig"]["connectorType"] + if (self.source == "salesforce" and connector_type != "Salesforce") or ( + self.source == "zendesk" and connector_type != "Zendesk" + ): + raise AppflowOperatorException( + f"Incompatible source ({self.source} and connector type ({connector_type})!" + ) + return connector_type + + def execute(self, context: "Context") -> None: + self._af_client: "AppflowClient" = self.hook.conn + self._dt_parsed: Optional[datetime] = datetime.fromisoformat(self.dt) if self.dt else None + if self._flow_update: + self._update_flow() + self._run_flow(context) + + def _update_flow(self) -> None: + self._response = self._af_client.describe_flow(flowName=self.name) + self._connector_type = self._get_connector_type() + + # cleanup + tasks: List["TaskTypeDef"] = [] + for task in self._response["tasks"]: + if ( + task["taskType"] == "Filter" + and task.get("connectorOperator", {}).get(self._connector_type) != "PROJECTION" + ): + self.log.info("Removing task: %s", task) + else: + tasks.append(task) # List of non-filter tasks + + self._add_filter(tasks) + + # Clean up to force on-demand trigger + trigger_config = copy.deepcopy(self._response["triggerConfig"]) + del trigger_config["triggerProperties"] + + self._af_client.update_flow( + flowName=self._response["flowName"], + destinationFlowConfigList=self._response["destinationFlowConfigList"], + sourceFlowConfig=self._response["sourceFlowConfig"], + triggerConfig=trigger_config, + description=self._response.get("description", "Flow description."), + tasks=tasks, + ) + + def _add_filter(self, tasks: List["TaskTypeDef"]) -> None: # Interface + pass + + def _run_flow(self, context) -> str: + ts_before: datetime = datetime.now(timezone.utc) + sleep(EVENTUAL_CONSISTENCY_OFFSET) + response = self._af_client.start_flow(flowName=self.name) + task_instance = context["task_instance"] + task_instance.xcom_push("execution_id", response["executionId"]) + self.log.info("executionId: %s", response["executionId"]) + + response = self._af_client.describe_flow(flowName=self.name) + + # Wait Appflow eventual consistence + self.log.info("Waiting Appflow eventual consistence...") + while ( + response.get("lastRunExecutionDetails", {}).get( + "mostRecentExecutionTime", datetime(1970, 1, 1, tzinfo=timezone.utc) + ) + < ts_before + ): + sleep(EVENTUAL_CONSISTENCY_POLLING) + response = self._af_client.describe_flow(flowName=self.name) + + # Wait flow stops + self.log.info("Waiting flow run...") + while ( + "mostRecentExecutionStatus" not in response["lastRunExecutionDetails"] + or response["lastRunExecutionDetails"]["mostRecentExecutionStatus"] == "InProgress" + ): + sleep(self.poll_interval) + response = self._af_client.describe_flow(flowName=self.name) + + self.log.info("lastRunExecutionDetails: %s", response["lastRunExecutionDetails"]) + + if response["lastRunExecutionDetails"]["mostRecentExecutionStatus"] == "Error": + raise Exception(f"Flow error:\n{json.dumps(response, default=str)}") + + return response["lastRunExecutionDetails"]["mostRecentExecutionStatus"] + + @property + def name(self): + return self._name + + @property + def source(self): + return self._source + + @property + def source_field(self): + return self._source_field + + @property + def aws_conn_id(self): + return self._aws_conn_id + + @property + def region(self): + return self._region + + @property + def poll_interval(self): + return self._poll_interval + + +class AppflowRunOperator(AppflowOperatorBase): + """ + Execute a Appflow run with filters as is. + + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:AppflowRunOperator` + + :param source: The source name (e.g. salesforce, zendesk) + :param name: The flow name + :param poll_interval: how often in seconds to check the query status + :param aws_conn_id: aws connection to use + :param region: aws region to use + """ + + def __init__( + self, + source: str, + name: str, + poll_interval: int = 20, + aws_conn_id: Optional[str] = "aws_default", + region: Optional[str] = None, + **kwargs, + ) -> None: + if source not in {"salesforce", "zendesk"}: + raise AppflowOperatorException(f"Source {source} is not supported for AppflowRunOperator!") + super().__init__( + source=source, + name=name, + flow_update=False, + source_field=None, + dt=None, + poll_interval=poll_interval, + aws_conn_id=aws_conn_id, + region=region, + **kwargs, + ) + + +class AppflowRunFullOperator(AppflowOperatorBase): + """ + Execute a Appflow full run removing any filter. + + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:AppflowRunFullOperator` + + :param source: The source name (e.g. salesforce, zendesk) + :param name: The flow name + :param poll_interval: how often in seconds to check the query status + :param aws_conn_id: aws connection to use + :param region: aws region to use + """ + + def __init__( + self, + source: str, + name: str, + poll_interval: int = 20, + aws_conn_id: Optional[str] = "aws_default", + region: Optional[str] = None, + **kwargs, + ) -> None: + if source not in {"salesforce", "zendesk"}: + raise AppflowOperatorException(f"Source {source} is not supported for AppflowRunFullOperator!") + super().__init__( + source=source, + name=name, + flow_update=True, + source_field=None, + dt=None, + poll_interval=poll_interval, + aws_conn_id=aws_conn_id, + region=region, + **kwargs, + ) + + +class AppflowRunBeforeOperator(AppflowOperatorBase): + """ + Execute a Appflow run after updating the filters to select only previous data. + + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:AppflowRunBeforeOperator` + + :param source: The source name (e.g. salesforce) + :param name: The flow name + :param source_field: The field name to apply filters + :param dt: The date value (or template) to be used in filters. + :param poll_interval: how often in seconds to check the query status + :param aws_conn_id: aws connection to use + :param region: aws region to use + """ + + template_fields = ("dt",) + + def __init__( + self, + source: str, + name: str, + source_field: str, + dt: str, + poll_interval: int = 20, + aws_conn_id: Optional[str] = "aws_default", + region: Optional[str] = None, + **kwargs, + ) -> None: + if not dt: + raise AppflowOperatorException("The dt argument is mandatory for AppflowRunBeforeOperator!") + if source not in {"salesforce"}: + raise AppflowOperatorException(f"Source {source} is not supported for AppflowRunBeforeOperator!") + super().__init__( + source=source, + name=name, + flow_update=True, + source_field=source_field, + dt=dt, + poll_interval=poll_interval, + aws_conn_id=aws_conn_id, + region=region, + **kwargs, + ) + + def _add_filter(self, tasks: List["TaskTypeDef"]) -> None: + if not self._dt_parsed: + raise AppflowOperatorException(f"Invalid dt argument parser value: {self._dt_parsed}") + if not self.source_field: + raise AppflowOperatorException(f"Invalid source_field argument value: {self.source_field}") + filter_task: "TaskTypeDef" = { + "taskType": "Filter", + "connectorOperator": {self._connector_type: "LESS_THAN"}, # type: ignore + "sourceFields": [self.source_field], + "taskProperties": { + "DATA_TYPE": "datetime", + "VALUE": AppflowOperatorBase._dt_to_epoch_str(self._dt_parsed), + }, # NOT inclusive + } + tasks.append(filter_task) + + +class AppflowRunAfterOperator(AppflowOperatorBase): + """ + Execute a Appflow run after updating the filters to select only future data. + + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:AppflowRunAfterOperator` + + :param source: The source name (e.g. salesforce, zendesk) + :param name: The flow name + :param source_field: The field name to apply filters + :param dt: The date value (or template) to be used in filters. + :param poll_interval: how often in seconds to check the query status + :param aws_conn_id: aws connection to use + :param region: aws region to use + """ + + template_fields = ("dt",) + + def __init__( + self, + source: str, + name: str, + source_field: str, + dt: str, + poll_interval: int = 20, + aws_conn_id: Optional[str] = "aws_default", + region: Optional[str] = None, + **kwargs, + ) -> None: + if not dt: + raise AppflowOperatorException("The dt argument is mandatory for AppflowRunAfterOperator!") + if source not in {"salesforce", "zendesk"}: + raise AppflowOperatorException(f"Source {source} is not supported for AppflowRunAfterOperator!") + super().__init__( + source=source, + name=name, + flow_update=True, + source_field=source_field, + dt=dt, + poll_interval=poll_interval, + aws_conn_id=aws_conn_id, + region=region, + **kwargs, + ) + + def _add_filter(self, tasks: List["TaskTypeDef"]) -> None: + if not self._dt_parsed: + raise AppflowOperatorException(f"Invalid dt argument parser value: {self._dt_parsed}") + if not self.source_field: + raise AppflowOperatorException(f"Invalid source_field argument value: {self.source_field}") + filter_task: "TaskTypeDef" = { + "taskType": "Filter", + "connectorOperator": {self._connector_type: "GREATER_THAN"}, # type: ignore + "sourceFields": [self.source_field], + "taskProperties": { + "DATA_TYPE": "datetime", + "VALUE": AppflowOperatorBase._dt_to_epoch_str(self._dt_parsed), + }, # NOT inclusive + } + tasks.append(filter_task) + + +class AppflowRunDailyOperator(AppflowOperatorBase): + """ + Execute a Appflow run after updating the filters to select only a single day. + + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:AppflowRunDailyOperator` + + :param source: The source name (e.g. salesforce) + :param name: The flow name + :param source_field: The field name to apply filters + :param dt: The date value (or template) to be used in filters. + :param poll_interval: how often in seconds to check the query status + :param aws_conn_id: aws connection to use + :param region: aws region to use + """ + + template_fields = ("dt",) + + def __init__( + self, + source: str, + name: str, + source_field: str, + dt: str, + poll_interval: int = 20, + aws_conn_id: Optional[str] = "aws_default", + region: Optional[str] = None, + **kwargs, + ) -> None: + if not dt: + raise AppflowOperatorException("The dt argument is mandatory for AppflowRunDailyOperator!") + if source not in {"salesforce"}: + raise AppflowOperatorException(f"Source {source} is not supported for AppflowRunDailyOperator!") + super().__init__( + source=source, + name=name, + flow_update=True, + source_field=source_field, + dt=dt, + poll_interval=poll_interval, + aws_conn_id=aws_conn_id, + region=region, + **kwargs, + ) + + def _add_filter(self, tasks: List["TaskTypeDef"]) -> None: + if not self._dt_parsed: + raise AppflowOperatorException(f"Invalid dt argument parser value: {self._dt_parsed}") + if not self.source_field: + raise AppflowOperatorException(f"Invalid source_field argument value: {self.source_field}") + start_dt = self._dt_parsed - timedelta(milliseconds=1) + end_dt = self._dt_parsed + timedelta(days=1) + filter_task: "TaskTypeDef" = { + "taskType": "Filter", + "connectorOperator": {self._connector_type: "BETWEEN"}, # type: ignore + "sourceFields": [self.source_field], + "taskProperties": { + "DATA_TYPE": "datetime", + "LOWER_BOUND": AppflowOperatorBase._dt_to_epoch_str(start_dt), # NOT inclusive + "UPPER_BOUND": AppflowOperatorBase._dt_to_epoch_str(end_dt), # NOT inclusive + }, + } + tasks.append(filter_task) + + +class AppflowRecordsShortCircuit(ShortCircuitOperator): + """ + Short-circuit in case of a empty Appflow's run. + + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:AppflowRecordsShortCircuit` + + :param flow_name: The flow name + :param appflow_run_task_id: Run task ID from where this operator should extract the execution ID + :param ignore_downstream_trigger_rules: Ignore downstream trigger rules + :param aws_conn_id: aws connection to use + :param region: aws region to use + """ + + LIGHT_BLUE = "#33ffec" + ui_color = LIGHT_BLUE + + def __init__( + self, + flow_name: str, + appflow_run_task_id: str, + ignore_downstream_trigger_rules: bool = True, + aws_conn_id: Optional[str] = "aws_default", + region: Optional[str] = None, + **kwargs, + ) -> None: + super().__init__( + python_callable=self._has_new_records_func, + op_kwargs={ + "flow_name": flow_name, + "appflow_run_task_id": appflow_run_task_id, + }, + ignore_downstream_trigger_rules=ignore_downstream_trigger_rules, + **kwargs, + ) + self._aws_conn_id = aws_conn_id + self._region = region + + @staticmethod + def _get_target_execution_id( + records: List["ExecutionRecordTypeDef"], execution_id: str + ) -> Optional["ExecutionRecordTypeDef"]: + for record in records: + if record.get("executionId") == execution_id: + return record + return None + + @cached_property + def hook(self) -> AppflowHook: + """Create and return an AppflowHook.""" + return AppflowHook(aws_conn_id=self._aws_conn_id, region_name=self._region) + + def _has_new_records_func(self, **kwargs) -> bool: + appflow_task_id = kwargs["appflow_run_task_id"] + self.log.info("appflow_task_id: ", appflow_task_id) + flow_name = kwargs["flow_name"] + self.log.info("flow_name: %s", flow_name) + af_client = self.hook.conn + task_instance = kwargs["task_instance"] + execution_id = task_instance.xcom_pull(task_ids=appflow_task_id, key="execution_id") # type: ignore + self.log.info("execution_id: %s", execution_id) + args = {"flowName": flow_name, "maxResults": 100} + response: "DescribeFlowExecutionRecordsResponseTypeDef" = cast( + "DescribeFlowExecutionRecordsResponseTypeDef", {} + ) + record = None + + while not record: + if "nextToken" in response: + response = af_client.describe_flow_execution_records(nextToken=response["nextToken"], **args) + else: + response = af_client.describe_flow_execution_records(**args) + record = AppflowRecordsShortCircuit._get_target_execution_id( + response["flowExecutions"], execution_id + ) + if not record and "nextToken" not in response: + raise AppflowOperatorException(f"Flow ({execution_id}) without recordsProcessed info.") + + execution = record.get("executionResult", {}) + if "recordsProcessed" not in execution: + raise AppflowOperatorException(f"Flow ({execution_id}) without recordsProcessed info.") + records_processed = execution["recordsProcessed"] + self.log.info("records_processed: %d", records_processed) + task_instance.xcom_push("records_processed", records_processed) # type: ignore + return records_processed > 0 diff --git a/airflow/providers/amazon/provider.yaml b/airflow/providers/amazon/provider.yaml index 413b6dcfed7ca..1ca7e60d269fb 100644 --- a/airflow/providers/amazon/provider.yaml +++ b/airflow/providers/amazon/provider.yaml @@ -212,6 +212,12 @@ integrations: external-doc-url: https://docs.aws.amazon.com/STS/latest/APIReference/welcome.html logo: /integration-logos/aws/AWS-STS_light-bg@4x.png tags: [aws] + - integration-name: Amazon Appflow + external-doc-url: https://docs.aws.amazon.com/appflow/1.0/APIReference/Welcome.html + logo: /integration-logos/aws/Amazon_AppFlow_light.png + how-to-guide: + - /docs/apache-airflow-providers-amazon/operators/appflow.rst + tags: [aws] operators: - integration-name: Amazon Athena @@ -310,6 +316,9 @@ operators: - integration-name: Amazon QuickSight python-modules: - airflow.providers.amazon.aws.operators.quicksight + - integration-name: Amazon Appflow + python-modules: + - airflow.providers.amazon.aws.operators.appflow sensors: - integration-name: Amazon Athena @@ -474,6 +483,9 @@ hooks: - integration-name: AWS Security Token Service (STS) python-modules: - airflow.providers.amazon.aws.hooks.sts + - integration-name: Amazon Appflow + python-modules: + - airflow.providers.amazon.aws.hooks.appflow transfers: - source-integration-name: Amazon DynamoDB diff --git a/tests/providers/amazon/aws/operators/test_appflow.py b/tests/providers/amazon/aws/operators/test_appflow.py new file mode 100644 index 0000000000000..0345db41cf0c7 --- /dev/null +++ b/tests/providers/amazon/aws/operators/test_appflow.py @@ -0,0 +1,172 @@ +# 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 datetime import datetime +from unittest import mock +from unittest.mock import ANY + +import pytest + +from airflow.providers.amazon.aws.operators.appflow import ( + AppflowRecordsShortCircuit, + AppflowRunAfterOperator, + AppflowRunBeforeOperator, + AppflowRunDailyOperator, + AppflowRunFullOperator, + AppflowRunOperator, +) +from airflow.utils import timezone + +CONN_ID = "aws_default" +DAG_ID = "dag_id" +TASK_ID = "task_id" +SHORT_CIRCUIT_TASK_ID = "short_circuit_task_id" +FLOW_NAME = "flow0" +EXECUTION_ID = "ex_id" +CONNECTION_TYPE = "Salesforce" +SOURCE = "salesforce" + +DUMP_COMMON_ARGS = {"aws_conn_id": CONN_ID, "task_id": TASK_ID, "source": SOURCE, "name": FLOW_NAME} + + +@pytest.fixture +def ctx(create_task_instance): + ti = create_task_instance( + dag_id=DAG_ID, + task_id=TASK_ID, + schedule_interval="0 12 * * *", + ) + yield {"task_instance": ti} + + +@pytest.fixture +def appflow_conn(): + with mock.patch("airflow.providers.amazon.aws.hooks.appflow.AppflowHook.conn") as mock_conn: + mock_conn.describe_flow.return_value = { + 'sourceFlowConfig': {'connectorType': CONNECTION_TYPE}, + 'tasks': [], + 'triggerConfig': {'triggerProperties': None}, + 'flowName': FLOW_NAME, + 'destinationFlowConfigList': {}, + 'lastRunExecutionDetails': { + 'mostRecentExecutionStatus': 'Successful', + 'mostRecentExecutionTime': datetime(3000, 1, 1, tzinfo=timezone.utc), + }, + } + mock_conn.update_flow.return_value = {} + mock_conn.start_flow.return_value = {"executionId": EXECUTION_ID} + mock_conn.describe_flow_execution_records.return_value = { + "flowExecutions": [{"executionId": EXECUTION_ID, "executionResult": {"recordsProcessed": 1}}] + } + yield mock_conn + + +def run_assertions_base(appflow_conn, tasks): + appflow_conn.describe_flow.assert_called_with(flowName=FLOW_NAME) + assert appflow_conn.describe_flow.call_count == 2 + appflow_conn.update_flow.assert_called_once_with( + flowName=FLOW_NAME, + tasks=tasks, + description=ANY, + destinationFlowConfigList=ANY, + sourceFlowConfig=ANY, + triggerConfig=ANY, + ) + appflow_conn.start_flow.assert_called_once_with(flowName=FLOW_NAME) + + +def test_run(appflow_conn, ctx): + operator = AppflowRunOperator(**DUMP_COMMON_ARGS) + operator.execute(ctx) # type: ignore + appflow_conn.describe_flow.assert_called_with(flowName=FLOW_NAME) + assert appflow_conn.describe_flow.call_count == 1 + appflow_conn.start_flow.assert_called_once_with(flowName=FLOW_NAME) + + +def test_run_full(appflow_conn, ctx): + operator = AppflowRunFullOperator(**DUMP_COMMON_ARGS) + operator.execute(ctx) # type: ignore + run_assertions_base(appflow_conn, []) + + +def test_run_after(appflow_conn, ctx): + operator = AppflowRunAfterOperator(source_field="col0", dt="2022-05-26", **DUMP_COMMON_ARGS) + operator.execute(ctx) # type: ignore + run_assertions_base( + appflow_conn, + [ + { + 'taskType': 'Filter', + 'connectorOperator': {'Salesforce': 'GREATER_THAN'}, + 'sourceFields': ['col0'], + 'taskProperties': {'DATA_TYPE': 'datetime', 'VALUE': '1653523200000'}, + } + ], + ) + + +def test_run_before(appflow_conn, ctx): + operator = AppflowRunBeforeOperator(source_field="col0", dt="2022-05-26", **DUMP_COMMON_ARGS) + operator.execute(ctx) # type: ignore + run_assertions_base( + appflow_conn, + [ + { + 'taskType': 'Filter', + 'connectorOperator': {'Salesforce': 'LESS_THAN'}, + 'sourceFields': ['col0'], + 'taskProperties': {'DATA_TYPE': 'datetime', 'VALUE': '1653523200000'}, + } + ], + ) + + +def test_run_daily(appflow_conn, ctx): + operator = AppflowRunDailyOperator(source_field="col0", dt="2022-05-26", **DUMP_COMMON_ARGS) + operator.execute(ctx) # type: ignore + run_assertions_base( + appflow_conn, + [ + { + 'taskType': 'Filter', + 'connectorOperator': {'Salesforce': 'BETWEEN'}, + 'sourceFields': ['col0'], + 'taskProperties': { + 'DATA_TYPE': 'datetime', + 'LOWER_BOUND': '1653523199999', + 'UPPER_BOUND': '1653609600000', + }, + } + ], + ) + + +def test_short_circuit(appflow_conn, dag_maker): + with dag_maker(dag_id="unit_test_short_circuit"): + AppflowRunFullOperator(**DUMP_COMMON_ARGS) + AppflowRecordsShortCircuit( + task_id=SHORT_CIRCUIT_TASK_ID, + flow_name=FLOW_NAME, + appflow_run_task_id=TASK_ID, + ) + + dagrun = dag_maker.create_dagrun(execution_date=timezone.utcnow()) + tis = {ti.task_id: ti for ti in dagrun.task_instances} + for _, ti in tis.items(): + ti.run() + appflow_conn.describe_flow_execution_records.assert_called_once_with(flowName=FLOW_NAME, maxResults=100) + assert tis[SHORT_CIRCUIT_TASK_ID].xcom_pull(task_ids=SHORT_CIRCUIT_TASK_ID, key='return_value') == 1 From e83ac356c6d7d26e22bf0dfea9473780cb324a9c Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Tue, 31 May 2022 17:35:20 -0300 Subject: [PATCH 03/26] Add Amazon AppFlow examples. --- .../aws/example_dags/example_appflow.py | 101 ++++++++++++++++++ 1 file changed, 101 insertions(+) create mode 100644 airflow/providers/amazon/aws/example_dags/example_appflow.py diff --git a/airflow/providers/amazon/aws/example_dags/example_appflow.py b/airflow/providers/amazon/aws/example_dags/example_appflow.py new file mode 100644 index 0000000000000..d51ac1ee9b981 --- /dev/null +++ b/airflow/providers/amazon/aws/example_dags/example_appflow.py @@ -0,0 +1,101 @@ +# 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 datetime import datetime + +from airflow import DAG +from airflow.operators.bash import BashOperator +from airflow.providers.amazon.aws.operators.appflow import ( + AppflowRecordsShortCircuit, + AppflowRunAfterOperator, + AppflowRunBeforeOperator, + AppflowRunDailyOperator, + AppflowRunFullOperator, + AppflowRunOperator, +) + +SOURCE_NAME = "salesforce" +FLOW_NAME = "salesforce-campaign" + +with DAG( + "example_appflow", + schedule_interval=None, + start_date=datetime(2022, 1, 1), + catchup=False, + tags=["example"], +) as dag: + + # [START howto_appflow_run] + run = AppflowRunOperator( + task_id="campaign-dump", + source=SOURCE_NAME, + name=FLOW_NAME, + ) + # [END howto_appflow_run] + + # [START howto_appflow_run_full] + run_full = AppflowRunFullOperator( + task_id="campaign-dump-full", + source=SOURCE_NAME, + name=FLOW_NAME, + ) + # [END howto_appflow_run_full] + + # [START howto_appflow_run_daily] + run_daily = AppflowRunDailyOperator( + task_id="campaign-dump-daily", + source=SOURCE_NAME, + name=FLOW_NAME, + source_field="LastModifiedDate", + dt="{{ ds }}", + ) + # [END howto_appflow_run_daily] + + # [START howto_appflow_run_before] + run_before = AppflowRunBeforeOperator( + task_id="campaign-dump-before", + source=SOURCE_NAME, + name=FLOW_NAME, + source_field="LastModifiedDate", + dt="{{ ds }}", + ) + # [END howto_appflow_run_before] + + # [START howto_appflow_run_after] + run_after = AppflowRunAfterOperator( + task_id="campaign-dump-after", + source=SOURCE_NAME, + name=FLOW_NAME, + source_field="LastModifiedDate", + dt="3000-01-01", # Future date, so no records to dump + ) + # [END howto_appflow_run_after] + + # [START howto_appflow_shortcircuit] + has_records = AppflowRecordsShortCircuit( + task_id="campaign-dump-short-ciruit", + flow_name=FLOW_NAME, + appflow_run_task_id="campaign-dump-after", # Should shortcircuit, no records expected + ) + # [END howto_appflow_shortcircuit] + + skipped = BashOperator( + task_id="should_be_skipped", + bash_command="echo 1", + ) + + run >> run_full >> run_daily >> run_before >> run_after >> has_records >> skipped From ae155fde2b68eb3704899887a72b55a2b56f966c Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Tue, 31 May 2022 17:35:30 -0300 Subject: [PATCH 04/26] Add Amazon Appflow docs. --- .../amazon/aws/operators/redshift_data.py | 1 + .../apache-airflow-providers-amazon/index.rst | 1 + .../operators/appflow.rst | 146 ++++++++++++++++++ .../aws/Amazon_AppFlow_light.png | Bin 0 -> 59266 bytes docs/spelling_wordlist.txt | 1 + 5 files changed, 149 insertions(+) create mode 100644 docs/apache-airflow-providers-amazon/operators/appflow.rst create mode 100644 docs/integration-logos/aws/Amazon_AppFlow_light.png diff --git a/airflow/providers/amazon/aws/operators/redshift_data.py b/airflow/providers/amazon/aws/operators/redshift_data.py index 3961833475ae3..80b9232f1965f 100644 --- a/airflow/providers/amazon/aws/operators/redshift_data.py +++ b/airflow/providers/amazon/aws/operators/redshift_data.py @@ -15,6 +15,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. + import sys from time import sleep from typing import TYPE_CHECKING, Any, Dict, Optional diff --git a/docs/apache-airflow-providers-amazon/index.rst b/docs/apache-airflow-providers-amazon/index.rst index 41fba1f06e283..8e504d5b584ff 100644 --- a/docs/apache-airflow-providers-amazon/index.rst +++ b/docs/apache-airflow-providers-amazon/index.rst @@ -85,6 +85,7 @@ PIP package Version required ``jsonpath_ng`` ``>=1.5.3`` ``mypy-boto3-rds`` ``>=1.21.0`` ``mypy-boto3-redshift-data`` ``>=1.21.0`` +``mypy-boto3-appflow`` ``>=1.21.0`` ``pandas`` ``>=0.17.1`` ``redshift_connector`` ``>=2.0.888`` ``sqlalchemy_redshift`` ``>=0.8.6`` diff --git a/docs/apache-airflow-providers-amazon/operators/appflow.rst b/docs/apache-airflow-providers-amazon/operators/appflow.rst new file mode 100644 index 0000000000000..7c65380b9dba8 --- /dev/null +++ b/docs/apache-airflow-providers-amazon/operators/appflow.rst @@ -0,0 +1,146 @@ + .. 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. + +============== +Amazon AppFlow +============== + +`Amazon AppFlow `__ is a fully managed integration service +that enables you to securely transfer data between Software-as-a-Service (SaaS) applications +like Salesforce, SAP, Zendesk, Slack, and ServiceNow, and AWS services like Amazon S3 and +Amazon Redshift, in just a few clicks. With AppFlow, you can run data flows at enterprise +scale at the frequency you choose - on a schedule, in response to a business event, or on +demand. You can configure data transformation capabilities like filtering and validation to +generate rich, ready-to-use data as part of the flow itself, without additional steps. +AppFlow automatically encrypts data in motion, and allows users to restrict data from +flowing over the public Internet for SaaS applications that are integrated with +AWS PrivateLink, reducing exposure to security threats. + +Prerequisite Tasks +------------------ + +.. include:: _partials/prerequisite_tasks.rst + +Operators +--------- + +Run Flow +======== + +To run an AppFlow flow keeping all filters as is, use: +:class:`~airflow.providers.amazon.aws.operators.appflow.AppflowRunOperator`. + +.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_appflow.py + :language: python + :dedent: 4 + :start-after: [START howto_appflow_run] + :end-before: [END howto_appflow_run] + +.. note:: + Supported sources: Salesforce, Zendesk + +.. _howto/operator:AppflowRunOperator: + +Run Flow Full +============= + +To run an AppFlow flow removing all filters, use: +:class:`~airflow.providers.amazon.aws.operators.appflow.AppflowRunFullOperator`. + +.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_appflow.py + :language: python + :dedent: 4 + :start-after: [START howto_appflow_run_full] + :end-before: [END howto_appflow_run_full] + +.. note:: + Supported sources: Salesforce, Zendesk + +.. _howto/operator:AppflowRunFullOperator: + +Run Flow Daily +============== + +To run an AppFlow flow filtering daily records, use: +:class:`~airflow.providers.amazon.aws.operators.appflow.AppflowRunDailyOperator`. + +.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_appflow.py + :language: python + :dedent: 4 + :start-after: [START howto_appflow_run_daily] + :end-before: [END howto_appflow_run_daily] + +.. note:: + Supported sources: Salesforce + +.. _howto/operator:AppflowRunDailyOperator: + +Run Flow Before +=============== + +To run an AppFlow flow filtering future records and selecting the past ones, use: +:class:`~airflow.providers.amazon.aws.operators.appflow.AppflowRunBeforeOperator`. + +.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_appflow.py + :language: python + :dedent: 4 + :start-after: [START howto_appflow_run_before] + :end-before: [END howto_appflow_run_before] + +.. note:: + Supported sources: Salesforce + +.. _howto/operator:AppflowRunBeforeOperator: + +Run Flow After +============== + +To run an AppFlow flow filtering past records and selecting the future ones, use: +:class:`~airflow.providers.amazon.aws.operators.appflow.AppflowRunAfterOperator`. + +.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_appflow.py + :language: python + :dedent: 4 + :start-after: [START howto_appflow_run_after] + :end-before: [END howto_appflow_run_after] + +.. note:: + Supported sources: Salesforce, Zendesk + +.. _howto/operator:AppflowRunAfterOperator: + +Skipping Tasks For Empty Runs +============================= + +To skip tasks when some AppFlow run return zero records, use: +:class:`~airflow.providers.amazon.aws.operators.appflow.AppflowRecordsShortCircuit`. + +.. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_appflow.py + :language: python + :dedent: 4 + :start-after: [START howto_appflow_shortcircuit] + :end-before: [END howto_appflow_shortcircuit] + +.. note:: + Supported sources: Salesforce, Zendesk + +.. _howto/operator:AppflowRecordsShortCircuit: + +Reference +--------- + +* `AWS boto3 library documentation for Amazon AppFlow `__ diff --git a/docs/integration-logos/aws/Amazon_AppFlow_light.png b/docs/integration-logos/aws/Amazon_AppFlow_light.png new file mode 100644 index 0000000000000000000000000000000000000000..fa1d4c64d615245889df988c48d5bc333518ff7a GIT binary patch literal 59266 zcmc$`WmJ{zyDv(YbW3-OfRuEnAl(ybn1FOhDBUfglyo!cE|C@i>28oNK|0*e1mF0t zwZ{H%_Bs1}U<}kRw|89EudXLdMOhXDjRXw_1_nc3PD%|1238FEgMtM7&0uU^G4Kah zu%x7ljijuky{)~Ix`VN)xfIyh!PeMA5T<+VQ4!uYizG>ctjSh1qOqcel;ECnKkeFGF>e zn%%HxUK)HDV*lXw2~OST7BrFZHHGAMXqCN%uN{V-&dy4} z^QULUPt47}K_Di9vDgWBE82Yt(H<;E2Kh#bx> z@uZw~XrCk~!;keAnTXlCMIA}}8*mQ%<~s5gN=h&+!0#w9upwX=1mHJV;D-eGfq{X` zg!}7nV8t@w|M?yIkK!K?hhSjDVC1F5HQvDPwR-$2)7E6TSWHb_Uz}gJ7*djhqxPes z>7y$wd_j{Fn&8j7>Z>3>XYH!LU~^j0G_~W?G`UmPgb#U7QlZeimqe(6KvIYsD8o#R zk}W4rM;#kDa3Zv@*f}#cbDeggDlM_sUq5-f*qX69^L=r%p69l7uKo;tR-pr>w_4xW%wK<+sFdG&rJK~-?fQOPd>lw zf@B6n1aD9W2kYRofjYuFCczHDet7N4VSUqcE~XeU;i8$6uY2Nr-+$rIi z@cjE?BON22G&;N4GU3`orTU=F**LhtTvT#`LGg7?R&KPoSsP-e@+_@D7Ni3m(|tybh53sGcz$s*1;ji->?n67IicpTv$ffq5u-wySCzOHPCP?9gby zo3awwO6fq!q=gHvR-PeP!7;PQW!IOTAdv#XC{Zh5otsd5VW3@U=bCNfRai3~1 z30ZTndE!_?OSK`0jLAF(x@DXQvW{yFLwltP7}Zw)(N$IxM{zbx@?XV$#b>ol@SrzblD=&wJR`$Bs`xeNQpqkm!*l&B9K$jOyKR zg1BDDneu&USB#@beqtNY0g-Y<%M`j6MDX~Kzls#e!w}*(=bdKr+&#HVv@1xDR{#o*MoK&v^eJ{YU2vwJ{!8rC->F z(Zj8%9T*ihjMn@q%RT)PR zy-UG@LSU|`ucS#83=sx=dD)NOB^Jf1Wm^4+cJ!D**!|U#QsHr_A=^D9Noqdvy^p=? zxZogvF2b)JUq6NMGDZoIqxy`2UOueWxF*}oEd#%s+HW!yyoaT}p(`-XYzZz++j;Jq zxjT^sQH1U;FZzLf6Rd!bs*Gu0QjeN03eIZK+N1ZlB14~=S+gn-)g_-4D`^uUYgP8P zquwu;8OZ@ytAF&vMPOwm3FfLoliT`$_Yr43=rtq#8fE~#4IW9{M~_O*lDnFChI5AT zSHLriwq0Gud#{{R6e^IZD4i#}8>Rw@CK*Ydq^X@S0&eXS#1wF=3#}L!gT^4@q}b&7 z5jnOu5+$a(@=a*39luxjC_6yTyGbv?i;@iug^bitn`ziGX5hYPLkWmP{cJXR7x1bQ$##w|46S2qTG0_uw<@+!4Fp3D+`mp1wAPAHH2^ zrukt=NMW_AkT&=WH4YV7>J>L}l!l2~~gXCvf;_hcTyiaWmntEbQWJoa4jC`6YvIYP{VLt z%bM-IZmJA!hS>QFjP;`gGF0B_J@r!~v)b`M#?;7m6UK20Dn4>XwGxPA!&{6l*)_1X z6d2K=l`d56;~r`6H)DgoEUH;5Az~X<)?dis`(M$USP~ns{A}c-tc9_7@dsb6YXxWE zaeoJ!X)}H8=PF?Km@;x3mF}T{X9VhMD5><`yknav6L*0KXKEqp<1j9Nhp3bisnS$P z4s$LQh2KCr4c8g(f==hY1)sPx?0!F!oP@dF2bZ2TDS57q-eP_&NCe5Yo=s7ammX+y zOX9GmP=3G23bQc2<0ClRPdxXG1ZloR9Dk@cWJA6q$<6A7qiXfMM+rjLRyhzPFs5c! z6zT$#$IXE%d4c|Wm4jg?M_GFssrIuI$~=;hf1^ z8qRo@=M-6o*-veyhoj+%e4pXPRn`N~-cGa`HCS9*hKe`r6pP{`Q@3Bf;g*=l80L{c z%_9vtjmJDFshG6iYNP^yds^EV<{@WhBED6~VUc2G6sz%gVn?w3Qi{RCWd8o;FMHf# z^OsF2n)T~g6-sMpe(<-tqv_?+s!3fQ?!9u=R@C`eY-Ma)S7iVq+j{N_$92wuARBgF zs{X#3h3BYc56i>~|M^!q1MWvH_)!B1n__1L6`J%{&CekGh+-ReIFG$j?MheN#+TNO zkpdI-9%oHfWvHZ4P?8x) z)-|CkhEYY1X|XytC2>@A&k@`Cf`0jC1zK;;}s;iyDSzuxiQlq+*c8q@1UEi`GPlt5&S*9 z%yyrf{tz9grV2Br%Lon@=#DuJWw<*#ta!LZcqCw&Q*U&K5so9hS&^>bmJR7XXrU^0 zXH`vuWBUH@Ep!dE>7*EXsoYz$=_zLB8CwPS@iQWd3= ztqr?|jg_(iYgO*}s|4ifjC=Z|sZWFoDFt)}mo7`2@`>YOWK5S=?6Ww%!i7mo4(%G4 zBdOkbDF}A5r!nOKT2!LE8&~Ard=SxoKZq&Zq2?3H)J`GlkA!heD7exS1e=g-L*x$1 z6OcSKWooZ5#cBco5LT=_>YKzTc8q064Mw6h&Aaq3QWWR*i8Fo7n(_s>pN*^u?Nwdr zeebc{ScCSyh#~*AmqX{snnPb1O~ho!jKur6C}S(GyVe=cSZy_D@nbg^QOw7w`lTJt9xB&r>pQ9}jB^ypg1-5XKW-ja2p zdo#Yyh253_D!<*Rh0XoJlLjJR+s|!txR9`gVO`0`qFl0-O+6M=m7$f;4V0)=BjFba zRD>;A8?L%rUN28d+y&mh>HY0|EZO9^mZnk!V5%;oP&U+TpzYH>&3hxif4scYp%tNu z*4(^r6iQN)t zZzYk+5@C8`?_X#ul1}c?ehy1xpxSlq)vfZOB*$`NrrEXPASjmFJNU;+*6w$D)0 zaGqfH;D0-oFbw+&PxL+!Csc`xQzIN+DNh&KrKs?il`u~6geq-bY4>JRxGT_?YL|H1O|d@MhYG0$Snc4|9oNXbZC$O>5_k*;x9EVk30 zaFY;bIv?Q=4K{H3E5xItkARplxf6%T+bx^mr6b(@(87wdWmeMt zp?$t6%;Jmw2wAMM@c__3sT_HVC)|Bc#PeHFn{bI{wSi>{(Jx#Q^VLq89P@aHqLN!PT#gnrm_>}NuRBtcSBum=-VXs2OnAB?Bf#5;+^-z-QZmrFUm zvR*)=ZzCLWQ2<8Nt%lZltAU4?F2`q=hR32=B%Xl4PnZ7) zT9Pz^p1a<6%Gfcm89XQB6W0YybG0A*^mrC0~k z8yO+v)MsuRN$zxAmw2r2&dhy*I8`{bm1}S)|G8dv>Ehtr-jTbqIA^riMt^WZ&sc%^AeKlU{Y5Qh+btyqySzQ)Q z-r;fEhpuwi^|SA}Y^=Bd^i#Y^tuF}+xQ;#8eBrw+E8adTKhnHukX*Pi2m{$pLS0e} z#Lf4^X3DxA%|`wUd{CIX;*I+O;DbBtbjlMoVO321yP!iRZtZSZEUQLPhN9S*v03tw z+x#g5|5%X^W?|d|s^d!fv&bI3!1t}3(tQ-(T)k+5Gdk+K*5jwdAKn*C+6+&qg#*ab zo1{Bym#aW`W=Oed;LBe<2Y{VFOBzunl4k^BFxg2a&3Q;7nTdLmmDO~`kp@@hdh@Ue84N;q5M z=J6N1KhL@co2;I6#=TyT)v@IbZJ3PTC|$dBbze0B(a9xhOOrx7w1<;?O9yu|`yTI6 z{(*9`Q1L-b=U#ju^OhI|a}(geVY*~7LB$7yk{pHvES4A5-;`pCZ9+~dHL4A3U8$_wFkhI>kpVRbPWu2nE<5x z&OWa7Ra1bVy|&Ai$;XkyfrEBH50M8(k=f_sgU|eaJ?~}9r%IZ#`pJpHy5+|q*_RPy z-*iUo{3R)Or$A4tWR-pM_ac}3om?G1J`f-OoUXSrvK|9K!_H?Sjf->b^p7?Fgzeog zkY_-3tv+3hmTs#Y^c;rEJ^Ws=iK<~H#R+2vkh%mUY^KHYfIGz#0{`5_cS_t=IIa+= zhd{FqH1O{8h%7YMOHD28u5z#Mxw7j>*8WQMeiNSzpvfQ zRW&xNh9r1VznflK#*XpVq;|=Jo)QW81p&Mw@sFZV87-oB81r0roty*nMZIY92e{JJ zyDW-F9RV{PyoF87kpcf4t~4uW~FE zc)y3dC|GXrblKRni^6Y&OL-ezFo(430=urtpc*l<(k|lGaf+x*sBT&G6X;iP) zr*lunyP{T`6dD#OH#K-@Xvf6#xFcJv_86GP#%0UcZ3R8HOvnIXPHv!dT)XT_l+E7X z&Y1nzqjCPSaxK}YBx@CI&fEH124Z{P8U{mlk?%y&tCciRfS^j^|EbTEDdlrtWhB9% z(U$4$6_JPTDxuc>`y?mZf&v6P{3*?j%e&0}=a*9GTUI1&D7qW)xNaFAZ@S(EK@?Kq zj^|0UA_V{;sF)cMDg=nLa(dMGz4(_hSPT=zrn_Hr+C`#mMQFzks|v7|!1^_IKJnt1 zfRBEMxl!CyASlsRnzfCxJ|*{M61zl_6Ts6Sk6bXWqEM4=bL+Q)YM2@Agd*di-poeE zWV1KjNHKq5nZ-XOJIG*lY-)v}FlR;Z3y8viLm!!CysjRB#`#Z`3EnmPB#FbvYG zir;fO(Nu^NqQ_cF=H58;I(Allgv_CcyVEpFRnCPRTGf_GjS5Vi>q*0=oMPYGl{)ag zv)){u%x1buE|6w%@YC0#HbcpvymXj;T{I{9c?^cgD|8j6H(tRot`|<~;?=U;f+&Q* zfcg_i_yOvj-b1}DSiN{;b~;vC>{K(8LVV;Dh3bDdPX*nG@2&*9T72S=39*~~LJds0 z+y@e>Lo%^LD<+D{bype{_5{~b@Hjt;0BH1}s54=mkRjrhc$iSxH1E z11Uh`9!oml^C{~Gl5Je$d3}iL>uVC%16XHGeg>AqEx{rD@QW!G$dX5bX>*u@;f|6v zo1&CUZsNz5wfRVZ$Z>yd$6iPT+ST?vY|9uqliod7Cg4Fk#4jy#-j`G2ezX|A;p_6J zr=Bhs>Rh=T~uZM79cH#lrJI=5gz%;KiP4)mrEPP z=Mp8f1c#xtvmuGyg)d7y>%SmqPo!VYz8&9GcP?~tRB0KVOQTZtcD0y()}r+2e(v+ApH*_$TGJv786u5PE6~*i zPwH*&6e1EIcfQ7bk4 z8;mb}5~&z6ZpjLothNHX!C#M@_;HlkxtR-D4QeQ5srj7O)+m*oX$w9r?!Gm9=j9g>ZA?> zEHolY_p~6LyOA&J4&ZkVW_ZXMu`U3D9)|_7un6nD477F~4;`u@EP^0GNeT1ti`_g@ zSOpvG6}>cVf^T!$ehVyd_Vq~C*q;PI%LW})W$LSlg|4g#p5rtehM)T`1FKXLKaa;vX9Q1(7O9f(H7efEP2=laZfE&1Hf|#y6>c*We zosOU;l5853Zd(d*<*{xJr3~MA>ckpw-jejugWw;SS7Z6$FLXuGxE=5>QR1hlXM9u| z4~&$0EZ!h9s7W(9I5~o^h8^<{&U5=0&VwiltT|@WJw7qjBr5@21s6!|__#2n%-JO? zMyTjN)kO=XyP!h5L@gGA{m?FUfkce-7K~#3fH!?i7@0Jy)rCJ#h@OAo5v^z7Dfd^_ zu5$zUa2r%tEaOB3bT1s5Wig^~8ZroD2jJF7TxND~12<35+Nc(nIT8Ln(>)y?M?al2 z(i;jfNwio=P}V+H>aVKom=Ti(x@^}J&9R08yne|4UQs;o#OwN<^G#-*psBG|F{hn& z8jSli$i;j895r}9R*EkQUMZ+I2mbPfVEe8LNnvaumo+voxl9r`?EWD*mg zKG5FGQ%R$oue5szPQT7-RDT*J5=tXl{0;O@Y2N+mz55LOzbfA?OyKfrY1VM?>O)J+ zuQUPJWG`YU&YNWZs+&mi#_$Z8lRG@k4A?kC!*Cgke&>6jhkLt_td#~w<@`G|vQ>$1 z(m`;?{+I*4<3X5t@C=Ba$4VlR3)3Uw$$kcr?&}Uo>kZY*2b9k67qqbtdYp@< zCwv!6PHGXU82@u#M55>EtJ3XN>zVyl>bqRwp;`F4#fxMBSdn_xyx<-@EI`s-S_G(u zWue?qrA=;n$~`T#$2RxH`d&ktx?W2wmN7FUk&6h9BZ^u;Fi*Man*eMgEKnp+ z(Q(3(6R-L;e~0`~8h$bm^v;i@_jF>Xa6}Dbhb8~>BA@s>WStiusesbAo>8U-Uq*IP z<11(+X#;>VoNA_laxhm?(kW&Xl? zS8HgAqwOQq2536uA-q0^Ke$<^Cc6FV{+8Ho0-qQj#yA&~#KnLereg`bZG`XmB@y=f z09d-C64$Kf@t6Ad7OMU&P|r~QZ`Hqi@DlPg*As4Y|9O_6_#7>2m8}GK*r!6z3p-{q zi&jOTrSAOzE8>?M>7m+-o4%0Ffc=;wqiT4vmn`X#eQRLxz~6|`H$$pJ|cl(PNr&0F>Dn z{ekn4jU4%5tf&s16odvV!Z(oBl9CM#1lR$T><%qyb*xg&JOoVU%hJ?S*;C*_F4U~` zqtr6U>|TukyxJA{DEZ)TSHFkg5c}XAUe{v4-KZ=WBlKtTJA#j2WIUvwDaPv4J9T1( z2&NBAQtBo(6%Ne6Xx$f$b>kvWXTTt@z8Ckh z{N5yc6b%*=c#nbj4f7~%by!fQH^=u%3#ECpRF|uoyUIZfIglJ`@RZ@ z9<qG|$!V zN!1d}qywnXeSg?9dg>b{WA87iq_09Tdz0Sv)YiPQiNBEE1M6NYb&R=5>+qHdMq4O-Nb{~q=sYhJ!1&XnB zwYcphrZAPRo%*${1|t%^!1xk@I)1Eq_nMxIaP1iW>ws%O9p}7v#{k-vF#I9@@~&&q zoVKdcE|w63_LWUvp|}6zN>`4eKqe+Boz!)pd=QcTu^S|A0heio<;5R4MfKnEGoTFQ z){y{sACf2b4D!Jafe8ZpWgE##T8a2{B!cUWQ;C#EGY^VQud=07qz=U8e~rcH|M z5}BSVG&GDQn?~yJx;QeD?mkGlaRq85(~(Hx>yu{TG&_L8SMx{V8~PXYo7!ctI$5FP zjGtNW+p{HCqDK6QVs;kukbh#brn$0-JM2QN z%K$hat*ApTJ4=W^1jWFEvszg}9fXo|2PK6}@IxvQHB#`vlQwDA)6SXoXMFts57aAs zbar4;I+qi~s|%DrJVtZ5y=?WF;jeRI?sGn=1fq49YSK{U(4{{Ai1O*{%m7LEcrRV_n&wJ&=*N zy*g9ol1UBhUA;tDx%9B<9Zfx#H=|YZz6y?OQ&m4$VR5Exi29FR$hE51-7<%LN7B7s6nidhU_Cr>YHC!*wJ>1aFZxhR4%a*gpOl3K$qENK7|Csk%I zAcTlamN)C|6oK#k;B?77p}+m$y~$Cnm!XMo{M0j5Gr3V6M%a#A6SLpm<2wBRV_YYN zCxYCN>c~ofT+5cUgM{5kR*Mc>px%O_lSHQR__Z|%IMMAgb%OV{Ai331@ z_<}|uaHzEm7AHH6V^(_L5og~=w!%0?fRUB>@J7?P#g!UhO?DR|Q(N6Cv+&lD#LC`c zn<3)zkXyeGv5KX2Ds|WoR(7q;Iw1shak7{Ob4_a`D1C^*5h{!U^vo>gKCc!^+U=UC zqWvm4RxW*54zFm_^j@`7Yi_bO417o{%q!p*!2AO_$xrz_(qOsrpg|SKW7SN+yYR0=4r75dC(rT@f}3?6_=9GL0}iHsq)8(soutyvswh|>Cl<|vTSi7H1*a6o#$uh#n){LB4q-r zP#Q&ci5kN~7#O}VVS-+1L6y67No#q$2N%)~-zn_n?yaM;W{!~V$gq%DnY@Q<_s=vh z;}X3Z4Si{Sn7^&udoWi|^gmm<8_&Iw8%j9!c_dE~kG&lmG{X&|vkHfq58({oIH?3x z!l5n0i7L9IV)S9J+8 zzaVfvZHpgGbPZ92#cYn-gUrkkOyXalLv=CmqSr}v}CpEs%jUw zI-vYHG4X51h|9ZSZ*J79ELCE~co?EVxN=Qz*NL=nD&NQA8q>vAJASpr3GLMM8LriL zg`XA>DpTDIpotMHkf=aZe)a*32voV_*L#Uz0~&8wCGS``(3~dm8Y}OjjKXQ&x0ol? zC}kD5OYBRete_g%Ue!pdWOG&Z15~^2x3Q`uV)W~A_qBfpHREUc^01Vb>hN|y3lMSI zY!C?f{~E#tz=$C9xV%w#O{4Lnh~}kk^%OgUeC!FHaEYLpDd5Z_djo8OewZixCn=?tLO$s@;^2+Jt-$*I$N<4K8cOn80`AB9L!iTHZ} zt^G~RLzYD~4Y4{=HCE@hSuRwntN0(JI<;L9WPYJM0dAz`$9!Jft7NzUR8IrtgH*?| z7(CuE$Nj}jYx{U6hxbjXr9ufB)D$lGy`m+jcHL95Upu@?m&C9BE6j%a_IdK*%RLY; z4e2ftc!cA)HaN1M@gt5ta=b#QzeHfIJjH(t29|aFi3Z`gihT%Ih?&I8(j&j;_|_#~ zb_9C$+~L=olC$^^O-diJphLNYK{Jn>prD%otw~qaxU}>TQU%aIzvd7^0g=@;Ar&W` z7pGhfuZ0P?PYAaE1+eJ7OlUvh7Sr&^B7#3YE6ydxpQ8T*<#} z6=dvhd!E;AWUK{_OR{QrGnKa028ylzk+31&pqu12WC=DBkHFKf8S9zRd9$_*l zCGW}#NOEowBsJ0?NM8N{*d_fied5qWm@8>892~s zi3RsbNtyGVNP87g0R6;_c8?)0bd3ZhFGy^cUFesBjb?OKY=>(!yF33n%#WY{n+`{o z{co%Fx4e%1&7K1^?7>&>2F8EG6(u*2s}LP?BO*Q%<}j|bCI)2TmUlj#9r%r!KDBZ6xoy*y<=-pOvC+e6N87zan=0rbJo!)$5T0(Rpd&IMq9O5e!tPG4IvDC>cq5@i`t>dE4s2?2*DAnmVY%; z1#-ep;a#)^U>nXCZ?(M6_wq~u9vSH;7yetV}+J(~`IHHeI zC1i6F0GHurMb$l~(rF_Muo$+2`Q}Q|hT<TdMz|WOCULIet~pJ^**5HxdP@jWH{vmp=U&D z8nugW;;OM-Pd?zBbv?i(rN80Q(amv-`=Q&=_s)I2V|ZyO&Hs&!u=Wb`04_x%h?t4yS#1W9|X*Q z%*tnf(R+MQ7jyYd^nSw$rp=vPto!l+Uz_(_bX7g=zjo>c&eFY(LpirtQXIj(FsAS2YD#JEE=>)q)BDk;E0*)FBvx8klN z5h_Pd;%7?~v^;v#ww{S*8IT`-O_dW$Y#mqr{>?kQbv^I9-@ijTE!%z)%(Vqvo10QR zS9V*j`MrlY_h~H#J_rQ*sW;dSB&xJZBP&YkCd9t+r~r97>EZth)5_6aAsBlAwF*BYl&F@16^9qMb4BGJ+`rYu93&+Ds7|G7dZ0w?UO3ZW?nL1F z3MwFn6Gl(seE}F|Sp&-DHB9_V1f~d;iWHS%{Z`-UoaYbi5YUbt3da1Jd;!i;oQ7dk z%MNUY0!30WAM$}{yhP`Nkt)6^1AZ9iNk9C1SxB+1UUkmzR<1z>vK=6|<@FCjrgP_3 zm)=GDrGj~RgMikD_>c$rW{BWMqLi2Y9|JrtGc#>i*!YDeRppEI(BeV zxu~x*uB_%x3L!|xf766>^4k?7E7T7Tg!;iLcK*Ct+AW=1I({)n)Ew+|A~a&Ufob!t z0ghIbWsA}kZ*A0mDa4YJJa2KRLK2tx`TVEv*8r%|icAm0geF^*jmm2*J6t37#uD>+> zi4N82U0#u~e>FkPzUSOVkMW1;hPt_r9r~gJfL!OK3Xmt;E+$E}fH;o5v4c znT`OQu7{QO*JvGyRpWiK9R=HAS&ZI#3XfX55KZ)hZ)pIj5r~RQP=*|+^J)l}ZLe!x zYdnd*oI&JASjFqEXF0&3oZ~c4Mtt{s!l%pblU@sy;qxMdBmv99<@6SY(m(?m0Bh9& ze4VyJb`v3^{mWDD0TIT?+EadEZA7>&p$wOao3b64jE}bXAfu3K6w`gygt_TVM|#+7(@)B-o0|Pejla<+@B}p zKZK$+C-E=)KTdi5$)D>xuy}(7 zgb83E&X9rK!deALnzLX#WImF{0Sf2}Kz8;cs19(^g5P&#TfG39+v5Ow`kw4a3<0gA z(R(2JPCfvXR<`acf?Ll`ZEVt_i9k-s4xH-D4bX3Wlfk* zeNGNv&$XUUn29YA2oWb3-|TTnsVhALatc~SrDJr4XxpK$gqME&8fWKQ7F^LkM2~*! zbo?lziTvck*^AieWKTwUc1DRKR9j|RbIux@jY$<~N3ElN-udLquGso7>8#VIw(JA% zVmR}8&7UGEvS5g4BNj-cKwZ0|(Q-;Hu&P$og3ijUxFe9*+LgXhc?j~)me>M;t)`N^ zPbq{w7OdX@5h289HE(W&k=!L<{irAA?$m8gIwMaLH3G1Z36Ww>Zo|;Hk825$p&rna z64gBY2bnQ_UIG2{e)N9I(p7y*jQ`lmoXJkeApdSmEiirZC3@3lxF4l;TsB}?HVis3 z1u((F=^?Nzqu$2qQ~w%&mCe)-jvDHOSz1uU-R9)yQj!0qh~q_UDK1%HyeWsPKzD?% z`B(A8zw?3`tFoz&4{C({8E$60l=^sF2+a{hzsGGLE z!slR|@c|+2TT4Fi8ubF`kZNgsE+F=$1fsJ+lY|1R24|Yf`N`=LY!%&^FS+yCBp|@G z$y;-%@PtK#mK-~y^;u+bkhWYZCePQ-B3)js1~bk(024v=55IBAu>dkwY zOQGEB2?u)Z1nE2Y!@Zx0OLxm|-zTQraZg-wTu5HHo_69DDO=^r*Pz``s90-j4@Sh! zbZ8-PC+VZE9R{ZeI>8gz`5+NHS4K|m?B$RXI9LM07O;%>U?LQ*1P3Tdl~(98%i}? znHIP?F2DQYw*JZMW1y2Wb}Nuvq+Wo|+_@-=I8u-saC2iZcNZ+|=2yO1RHz~%1Bj;jKfKRC0Q-{zSFXO4w9mul@DcB=QWx0=A5w9d|QpPz2%EzgpL?XJ2u>-9yiKB&TBIAHq8 z=^cMTv8uD;AyaF)OO`U38zon5#X!Q2$Y4bjW#KH|7)W^Y(6f z&Y+jX4ajeKSQh~Y!J?F{`Str*95hB-rZI{4V}lpq>iNl!bGG|*=5(HtjEgEUC4PM* z=f{q60!5whj~HcZWip|%>8Gdvv;B?!w9jf+$%HJVtiUbh()&nVZ!@hj{c>lsivkoZ zdgj_)-d^9Tfn^oz1cNW{(D*HzwW};H|fiLmq zy~bX*Qmr$JY?9WSX8h9W=pm0xKBPUsvr?k){GSC^9i>B*NLQvAwEbV`!JvFY^&IIo}2<&Z1MtvqsT9a!^^+E1(5FepF`a* z+#lvqfDkxXXEZd~k=xwVI|*4AowPwI-hcH%WgbCYKW*lTT~#0hDQg-=O1>6~889&O zwR3bR;5)WoP;mOu<5%cGna-veg%P@}4u%9iTY?#N>Az}vwGYA<-Yz_uKo+N!LIU$rb?U<6Kffv^RihIT26DwzrCB+R#_8E{NZDS zTgZQB+Z&^W_b&3z=+N?7X}IFGIkM8mCYv!zaw+#?F4w@%$-Sxgbn#qloW~6KKTz8f zZY@!$DgmfkS-EI@vKRT9Ar$lt2iZ!L-rjUyY!R3f(tC(y15<0(3vhHmyt2@d=gu0o zfz6TR`(coM_nTke-<3Zr>HPNlr(iwuGI^BHbf+LHyzKoXZ8w4ZR@pOFocGbNm|@Tn zO7xbdV}F;I(5pKgS&b2 zxhTq;2+Wx6Hp$-C1vf`0@cebW784n6qw8Ox!{8bzQYHSB^w0rICc*o0@I5e}n40s~ ziV7S_iqesB!hqK7lq!^MFD6%VdubV1r z_9IW@7}?tm@SJY>Tf;L)l)K>gZ@)(b$#5lsrDu;oryFK!aOMq`05bSW<uO3nx0KYPx3{aSrQd3e$1cK)H2t^E+8hoQ-ml*6+Ao&9oeP0qWW10re0Fq* z+N_ynvG+PYif29u*l%>L@2oi--@8vr-8cm^9hG!EsHH zE=;7|Vjl|I4)DNBsQVFWH^}pS!n>_E{EaYR2PbZKY;&p50(T`&{Ai_uUJr!ixK*_` zT>Tj=csaAAlJqEasl##5zKzJxcekgM_&%!Ue0_PhTnbdps`rL?EOH4rfi~j?^d<|o zPFja zym1}S(I8tk9`p{Y_-$lMikh#%a{AMdf#@li)AM%wp*zdnR~bIUONbOfhXba)_Y>O5 zxHSw|CI8b{&-^9~fi-MVV=fDwCha`^cyk3CabVzPL@Z_d_g9{@4W6)-54?T9?d%1K zozN!fH@7@^=D%|Ro#|N;rr2$GH7|&?fsAdj^6xAnJLDmOykeETq|aBYZez$IWzUgH zoaIQ0W0e!8fsb~44JEj+Y7;FqW4!55oe3R~sJHVO2*|w|2TQcZ^ZYzMM_s&Q^l4l` z@)GnS^ExY<%hLANkMD|J;2-J%zK`+^%&2rA)Z}1};Pl5RTp9u(KStpatODn@jxD`J zohhy7n(`0AfEO+~Y!Y8ua+{D!h_D|{q3)XdiQ(00C(Dz5g@?FT**`hpzr|{IwfeUO z8CB`GO+*6Xm@z^}?Pot;z4Ky#(M^|RTdhRuybu4!tchH6Zd`Y^c0Y>phIHSDN55HX zY!mj~;n%UW?o6#rZGK0{gx0L-Gcp0DGQEf|4dWJMHq+t2?H{4^LaBm|4CS#<0uGMh zwixE6$2)i__|Z$;!&#I3*Tz9~l8`=qslI1|9_7m3I?@kZSJcMnMZvdS?mz(fBmNut z>wNB;vGE3wf9LG?+s4DCBTE&_n3AQ-!@6enMiVCJ(${}r5TZI*aH(NcKyiQJIob=M zBD$y+HWbwZ%8NwR&j6z&Acdf$diWaQ}Zz zAylt8z$Z9lN`0g&B`qf96bx#>){i8yF|A@E<n8~n0^h<|Ub_{G^!BDjL6Uki-W zVHH>a-o-3N`i`|EbJe(OufzRBdY7>Pp9rN{8yhrk>Rb#AsN*JKuVfp8ECBGYhy}D;T!Zt}!;tUgvwTU4iK3rwRxNF1 zMbFrfyQ69CKQC7XzXmhrKz29Nv}9Dw->0J&y;O698LEGW%tWisDd-}SJxqXcnP1aQ zg*}lVEmsU^CQ8&H6OTfL?+p(=4yVp@S!Kp*4Oyt%6UBQkU)2Enf7#NmFk)pVBf5D_ zeBrS)v8HJ6aqjke-#ymW7?tQof!Y)U8hwv#yuBY+XDY9>vZA2s4FqBfbnz>AKx|by-C|@;D2_{6;C9U?_6J#K3i+_-&Z55b>1IW|4JBHqANxL zHQ9!C*DRe#uWaYGMUVXV+n*n#+FpxOf8KZgV9(cfxU>?cI!O~GQSr~FcSYe|IdLsf z?k><&V;J6#Ww0RXw))%pcepiQf_$15Tz{MzCy6}2`?~oW`P0cdX3`yuq>hKPAMl|F z)oj(No;)Buo!LXD_ajXYD(QQg!#BJ8=8b>&DKU#L=}LRSp&`9w7gAWvCOr&FOOJS| z{gkTCUMbuVwQ8Y&7f7_SR4GKGwXxQ`l4QlpW=&;tV1}?wG-`_Q<%qIb2)Xz1e`o41W+xDRurhL%;5PQ(~?(5mz z^~xb?IRNyNMK9m)qj>MCd3QaIuMY>p-xVK$N4rDi2Tac~0Uu!qiz0`H6x3D57hnY4 zQih^_6sHzBP}dRL687e@Nf>9N<~99uV=5e+YVsTKC7-ATAqg!(Ejx`}?^xh#5w(5V z@UtgpIkbyYNP4y-+xaac@F5f zM5fAbI}`^~RK&A=jU*&hv2O=QrL**Y3bJ3F&ppX#&^ck@--wK_i+^>SE*-P=_}QTu z@Tn%p3^5xgd)&1|oWj4x;Yne-pGryVzE=n%m+xrGYQm9RoHxJ+?tLK`$AzfCd8-rn zZ|v*GXKMp)Ex6$5_fBpWHF|AfhO+Vo@P7A)nXhSag*~I@dQH%S17p|f$i4g5>c=vE zk1w*VpD(ygohAApEv4WLCpBa#EKZdE9U4Q2(txsOf+cDJOvc0+(DCS;`UN*ab9VB#Bmo^#&w{e*q*z3;WwRT~&L{n#|NQ56dtQn54hChfiv zV!Lqo#}ADUtiDTUQS)iecf=WbMfXjHQS9{+;Vu65g2i|Wajx~5w(^yjwA0Z=D&~Z6 zDwrmO9u4dbZGgKA3YxY=eQOo9_+RGNwx`bLKT>Xj`dgQc$``^U?2W$PO0)CD*gY6Z zJL@*v*}yWw5)76Lk~!eDc#)BaV!o4mnUH-e)sMI>1_b|CH${90MKKp?;*j@nO4Vqt zuvqT%)(>p+FOa|WZy5WC9mcX0SNATt`@ zX$Zfb2r0R8;1>KXl~GwYSRB5hDv_CgkuSZAH5}es-4@i2xmexB@A(gm>*lbr|I|0@ zi-R;V^K^`WhPlDtgt*Cm@e_M;_kLy-IyhTUl>M(X5SzgAzCCk4h}uOz)fN5^+Zq|X)CpM? ztZ!&%ZKv?vnO2zKB@~hkmO)?Xq&%2JDZFY8W`)$)#Mdh1?X$)H-~UvYd)PyM$`|}+ zcVePH%gznVGv@K9R06@Sf8KQ48cSaOv~94!apTdCjn{rOqb2rY;DpThd4K22p=%p%QJ7 zqIs%eN~~jeaI$v_Iti|qyIBOe-KrVav<+vMu$@H8@Cp&ZR5&msu|MhcvuEZm)2!@>b?GJ;}5Xii@_Q4zgoJ{LaMc*Fpi&h@!mMCs;7-}m@*%_%-mwy{|^< z*bJFHa=kD^mEzfdQ$V~Elwh)28(J`L`(BsbkX(*IWTTUj?(N4;yW3&CONm+=mDZ%PJOQgmM~PlQhtLXH%zj2 zJGhrzmUSB_cwtrWndP5x8SkL~i3^CBI{hmar-)yI31TJQ0w*rLn^;HCsG(SZ(xUTf z=?VE#u;r@&*qI^drv9k%=9`^v3W^h(DI{=P(0qKU4X^<29CmC(QWfN4n5?M%t_79% zE`)CI-xD`U2aCPKjs!FUTvfenRLfWfmEWc^zsD%4N}*r9|MsjG!!;Q8*L=7uPQNt1 zyMnod{mQ!acE-wHvPc7EE{iOaGyDe%Ab5adTtHN1$iHKLyANkpP`s@@PeTvJv*LAf zH`T0w)?|^TcBN-VnqkWeyX99A{x{ z=+I7|XZ`JV$hQzlch_EY)b#`zV=Pp2?us=l2MCYHl&=smF}(O%=~nE9XZb?$wSReD zLiN2mk%t05g&!SmnbBqvJ4R%(dUXqZYFx%@S5*Fg+LhQ*bPKeFIWz+0s9ajTy+9t~ z)UrMaPb8Ea?&zPQz&Zt~{#4HY6AoNX5#8GGOp*(%`GR3bIs2f&bu3(nB@FQ0(NPDC zpi9t_HY$7zKYp2KNF|^2Vj2CN4ZdrUr{3`VVk^1$Dw$W|9M{U6F1bg@cbc5FYjrYUW0` z$YPpMPQ?Mb<#4_o=?o#1Hx~)i$zB`xt*h~|c5If8F~SDG@P>gq-t?4yAMa9O4PG%` z>@2cuWR8H;d3NcK>Oe@2$3=yf%CG1=HNC4JJ+^I+_PdF6DDJ^y4VDb!Pt^}9&xT#x z+1lL&p;r!;Fm~vxrEb9K$w75auxM+ZKj7sZ*Ecu%pOy%!e)lsZuW5PcxaE_)!k$e7 ztYl_N)jcehEgmda-J8{}q2NesAQ|-IMfoU8Sutg;N5mR;EQKH*YaX4HaT4i60lxaF zt!a`+lH%Qwx5|ON&9PVD7{TI8hg!f>!#48hS6qClS(lb5*cpJ}40Qqt!T3SXCF&j^ zp1z;oA55G%ubO^&MID z+{$^$N&qE?WDf#r`OU{Gb*{t)`9BXWE%d6&;eF?15Ya#7kgs08?$23r18p|A5kgkY za3_yDY)0p_1CDFzO`(%0E}~S!n{_(u#YDPRV`FQ`d$oj@aMsCc6^_HeeqUWeqWyvY zv)v7Lp6j;&)Js~X^BH;O;D0(3JTf}unsbUotYdeDh}0sh<7>As!Sh6*L5+4-nKLB> zQTXrmQU3D9a?~=pAJ$b)7(yfD#k1hew4k+Spv`$I_3S;i4|sdq2ee+o!VemY%qp{R zjyB-SOSl++U&RJbuO$fpu*DZ10}a&puGgON**bN&%)D8dMeq4G2+|STDznzm+v?l` zOMyL0O8X z%p3?KvrU76>G??J{IUHkw#Vyz$`Z4MX@k}irnVd8jPC9QOx?MXS=T?@m$k18PpL5D zIFib!)B~^qoPleoU#_3IA?vd6XhbO{vy); zOegCE!c(4sBOKQUt}dUI)nCA;zqcn49~1c%TnKyrjVZBO)KPl)nCHUSbt-t z=2Y^te~RyJ8`jXjezHK8BU2bu!m^UN4umdkYj#Z8i5m++UeFql5{N(iQPHcVOIpOM zxYd3kvC@)t#;mh&p*<7I5Q4t^>yMfDgKo|`Wf5%7N#p%q+WGU83@g`-J?RYOy(iJ} z?ET0OMUMC6ZLjH{2k5;(xapY5ojLSqMrA~k->~vD>pfr>bQr^yx*EE1DZup~SIiD? z$;BQeJY=q^9kw3zx{FnZwxfc6RK-h zO$Bfj-uGw#6n_X&T#z3_*FZ3RruC(m3c$+fwI@dP>n|adV3-7k2^%HZz6J_#=4__E zH*(r0qe-(xa#6^iJk7Typ+H;l>G`9t7gGNgYw=LnERkLrk9C}&t*hAecoy?cKr?*;2_|FKT!k{{<1&JL{zmJtt$ zR&CD^zCU(d;^oXsH6$y<_8K13ZBoiaD1lzVv}B$9v3Kofk^gBUAh+rWUL=#k}HN?=&fVv=b9eDX+*y!)X{r!4Et6)Ghg%&ab)aS-`iD8d-*wxKKGa;2aAi zmZP-DlOU4e{ugco4jtUX3qtWPsyLLb+?o~&@I-b))Fd-76wPQ^&m2CM*epugeu!K4 z0`!CF|7D`yh9tYTV$CqMXJiy@+vP76VfZCDc&NNDPjTqT)C*NyH09y7r(1Y8f$5~C zl20#gS*(w2f`OO*5i!|btA2>6bseTWsQE0k+WagMGkMWr?r&WJt~XD=!4!PvSIy@r z*F66_wK9`JmfyA4iPrz+8+>sD9le=9@-Fuc*Z~Cf`{BvsC6Kp@Ey~Yau({)zXI7@q zM~usoJ()H)HJvLyD5DE?TLi*e-YzR9F$eA9!{6L(PrUZ$!1spU7!Hn zbgQC|`T~q2F+I&wCb*}-$S9U7nMG1(cNtb1!xI0!1>(NT05e(VypyCn@Oyq9_g!o- zx)Fj|_<9QChMWX1-u3UxU_JTScB{!^wIz-CwC25rn@Bj+M?6cWUVW(N9&qKi&v!cZ zaT#t8Z{U%?c66L)UY55D-W~Yhx5Y1(m0NBOukLyBYwZXzqygifmO;j5i_tS3HF=_` zXb$c%3}I6wQl`dd33gC{QGI-R2lQ{>OYP_Xe|4( z;)WW2H}4<5eBi>OwPyrR@t=?WG(BuQ0m!vF3!O4ZPatLiJ{bggY{fRgqi1$XJeR1L%UPYpz|KFNnKXQsL!#rpT{As=C>3qt(SMlmHduO1`< z_OIUEG}M^Rj}B+!_`?g6uBOUt)_pPfT}f!hkC`mNs{5m<*y!(qtl{|B`^(aKCj@Gw zt8wo9WwMQG`ts37)IOI#e!M4nb6b`a!7BFyx5J%#W6a5KD^uflxnL*EyuBMZ@Zj_g zlovsQ+pY$o=y-^QzE&zmqIxi^0-qlH2` zV!mE@7!FkS{>Sk^uD*od;{fyv&VSc7=xAFjgZVqeh)&a%)sg6I>0E@7a5jYQt{t>z zxMB5;mvR>(_O3+`TXaU{4P{8e(JylOE95~v_!6pu#f?pK_9=&rKRtHN&vrX_s<^?Z zF2BS|si_lrM3~ujqqvM^9(w$a3Jl*ClV_I)E#gqIowo65iZyknl^Z2r^jrB-E0GG( z%PwB6NQnBDHBI51?Xtx0ie%n`)~{oOJ$17=(SA_`HwL0l`MvJS}2pX(zr(W5p0IhQdc6sBTb=&zLp( z5n>Rr+rI|a!5v1w8B{Jv8UVi+3lQ3f_D25h z%U)$7nK~Zj3cRN9FN_#}>u(QwS5m9Y-!KKw&@V~>kP&7u3S{?!C9t=%L)HR)x{ZM#a@E>)bWejdMjdFK1GeBE_aZ z4$C#g-R8;vkx!GokG05H;YmE#HY|LrrRpo}9x3qjO>=;4f9n2RjUPdd%KfYA*I1Pt zdr23$xgheUf{$4iA%TJWHxM4i<;a}K_Wf4?ul>Qp%fKBkQ~Q|1%Gjp^a$H3ROxxP` zyt);qoNtz@QI%5uz^%O5uKzle{*m`ZESehz*>>)G6OKe#t7-$dJx+di0;)eS}c1 zNmRxI2|+AyI_@`9itX-0&XD~@Zpe+^#dB6Po>qp%^Xh0nTdd(=FY4>}6aS4A1 zEYAk!vgwV3qDL0_#(+O;yaObVe_{Nfb-D0j6luZ0AKj0W*vBhgQoM>XZMkKiY3r?= zij|}2F8mktEsrw~I@XZT`#CI)2S3K#F1rl-k4cnk`W3J$^wIXuyStv8gE3uL@m&%D z*02_UphbXpLHNMhy!{eOjA8&uJXGWNbX3`!+<&hcgxyEs`;=5QyiO2zQ9@}QE=9#0 z=}6FQNY4G}P-jnf_@)#;*)(o)M5Xs%5Y05Fv9GxBm%@#ghITY`9h zbwI-Iu%mY7(t9qlt6oV655^pUGe)3{AJA76KvNA0=QT#4uP=Ed(h*QI%cvrBaIPZS;Xz*ld zGKg^f1d2&KNYH2(px!<1Y$sa#R9APTj^l-;#NUa#!Pih?s0fr)xX_VaU1Z%5?YV8ms|vp-{Js z>LJeWR@K{g3EBn?{}UGGC-GbWmLRI1@l%;#`xAk2e!k!Fy}r6<#kzl@u}pAhOzz{k zi#GC1(4pUGY4)V!+aRG3jox>d4RVg{YV3F%>pJKkQEO&eqx@`6Bt|C^YJ0j&KttJH z(}dR!w+0XXgks;B!Lm0`HO3zogy$(X(o{CLcnsYdWId!AF1CkzuOOJwTb0=K%>u*R zjvt~vef}=y@K6LSrO)j~(<@+f1Rf#(!fl7E7uFIT1KNEz3B`|9nQcGKTWmw@wGI-t z_}44MeGrbc4<9i5Z+6oBHjmK~PE}YucI)bVV0pnRTHa6NX&;dI_YMh5+WYQ_>aSj} zF_=;|njr}U)6u+zxrnI`qj_;`J4SvKgr;n;A|JXGqKCWm1X;~y4jVj zr5m?43YR6G=U$d-&*uuxdOc zk;01qM0Tm3Jf}R5Wi>W^mpuDrOzfK>wYQ9b^9RqOea9lJ1GXqw=BGy-l%$)b?|l@- zsTJi@IXwK0F*f9adpz#Jjil{g4i+v$LcU>Rmur`BtJdul>yH)5EQtt6wR^wE1J2kl zpQy@R$2^t^nrl0+uinu)yHG34JgvPNfxdlq!IIn=F#VCu@n+v=iB|JQlwpW906IC? ztO^yXbA=tux<8TdwZze3@$c^6P6m+&JX+#Te^&OPEY;71ej@JG9U=CItg9I1_`j}* zf~$UM3Cuh+rNw{$p>YX4i=u#BaeNoP(|G;w=kEF=F@plrMJ%m)q4>o+D-Sfv<@x;R zQz-&BMP zSxV23OH)pMST&WjWc7_;l5k+Dj5)&0!E1Cxp0l{2-Qo3g2y4_Dgn?K(OQcKOUQ1qRF`PIwuGqvsa6G3RB^ zVJ~ys(M$ugp3#3pO6_MvC^?jfGaZ@7fjbitSvmIK=YtOxyt;^wVp?|NK{WN0Yi*snzq(wK#~>v*L}SPEhdg2JVCh^ zU^dHa!oS|&Hrl@zMlnk5+>b;{!TdyY&$XVD0GtYkH1!&fny#%y0-H`>@9tUlkH@BAs1vWbsdOt$`wUyf!)8j z|4|ddnNG3ZRoVX|``|SEQ>(&f2}NiGl?~}TI6rn}bKqa+$iea^CO$S5V*2=q$ZHig zZF61je8!SA;UZmesg2CZ!DF3Z2m9?9{|;8d@&M{3ak|x7oN&I)vO*oUt)zzp15Az^ z0-yoZSoTU6=<*_Ka}Y4zaP~RN_n2CEv)QR+sD`%e3z>M8%HMn33{h++BR=wWb30Ra zf!qy;m0n99S#5emTK5uUhW-L3F1w$Y<2C3Ah7slf%{nVy*U9NBD>wVKL?_P@_V&Fb zU<9If;Ijv-THl@G_g1dvy|&IJrj6uwg@5_Nj8$abru^+BPT{`GmIqPoTPb05M#w1C z8iyW)?MQ!3W@97JYRS-wRr8j5W%Y|PI)V#7mdW?$>le+r=|r+oyG)l6V>j&=y`wyg zX2@s=mTwYz+nDw=i-vw<9fc=j-F8oKIXZ43$*vpMTE(>{vU*H@^UN{}+J*57&=|`; zXegl@#U&u z{;raxE(HS?rEs#;h3JE2V0mDPwgc+&gmYaE)o<>UJIUB@pr%$X`CPD!{0)7uGnjLv z-7O0Fow1H+`YRmA9~Tb;+EJ!hZlArRmi_w@ED!^|v~_vc=-RTq>veDg*hDn9HuoiT zci!P%n4m5u8i>rI>c&Znj97}^>a4j8say;ZaLRgVwQQxH-8}c^eb55YJ5(Y|ehAgUN|1BYsw=C`!;m{XbP63< z_kiKhipSm8$WnM2#>aZT1|Ptic}5s2B=@_@eCcFs;pE!N!aTX{GQu>d3|^lCK(j5^ z*73iQ8^14c7*qJ-oKo8^TG%CN`H74kQZ3mlpE+hnV=r>8N@AMD@#1y!^jR#cxK>?c z4CqQ(m$~VcVI^Y(6XjChkoiu!+WbV`L zceU|P^<)X7FMGYE!jrRQkiB*#ymfevLPky$Qq;?k%$P3vGoNT#qLwC>(xNZ{jR7*+&)@)lI6x- zvLsXXN^_qD(R0*YQbLEXBEqFyOmjN^f9U4C!2gwE|&X{BdH5MauK!rN3Ept zL&!dVdqThm++3SX8oN3Gp_jQ5rCzOy^!~N5PCWj|SDeRd@cHgbUM>n_aMevuDy>um9{&p&>OWQ#x zEuY|@CdoFF!>_>A-0?i<>~dhv)K`m>?zw=0!t{F{rcG%^QLRGa1R960(jpPwTAB)6 zKi#WMl&Unp2yD}1&ELAv&hzRS*5?rBJs90DXC>l48{d#ritfsJmGt-VfIiFu&e^_#;&o`*aBQR6_d7eSF;p*I}pa zH5Jlr7f*83AdkNX8eHao->MSe2%`KI*iAU7?b34JP+2OBbDa|K#~_1VFznGstw4Uj zhd%6mOCS1jhvStRr;KT&7a0d7ZTABMBIMHt7VKG93n^kgjS#s13K5h|#r^fRGBBb2 zS;~&Sm8Nuswr%kle#C$jeRn8%R7N9u4cXs(9YY=v(XW*i@A_FU*e1#yfUO?MC0G^d zi895cS|>r)SQP*@RejWX<6OnHfRxDL_jhR?6}5YtRWRG>;CEjn{J+bZR4KV$bfx zhG?E?18vLV;;u?2%o1oQzPw-{BI=Y=aAo0PA*uLG7A5gR&{YDoUhRbytBFGd8=k34U)e`|LKnKyb-VQz&^b{*L5w!?9@6=M=) zwn++*_y16=x;vm)NLiDhsFhf1^#p0uxM~b`QEKk57;8NgT4+N{$EBq<`!TSL{_m}$ z5h+6uz)(5;m_H}W!y>D@P!Oo8Wo3%}iGf@l!~MjF>1~3VF$W${v zU&_i82u!drRAups7u5ZcAs0Rn$wDr@m*llH$tu`nTxri6;x7V`%bRu&p5V0Nc@RMP zT7Jpx ze}ATVX^1&!+MIE>eEiI&L14LbZ$L5g{rA8VyEj^CnI3N}4py&12}07jZww-)Eyy7I zE1`M7bX@Oa=x(iY4S{r@?beur{2TV%D9J+W*-vIt!JZ=6`^>qK(p;N} zY*m!9z5%B_cjQr@a%El3m^ERF4QTE7l?(|y0Xo~^I<}yKTIWOpQa?qSR@psIvzvM$ z-D2jJx5Sc7Vics%_?&d%8U6Ob?u3>5lhg^W51&+#18NbtWs>Cw3yDalgjfzjYZGtBZ4ReO^1w`N4#6H{GFxTGQ)WFv^_B(lgE< zRps5!5mz>u-|7R#{u`&UMwbLlrG!8!B=Ge6Ce(ce5iXRxC^UKW^!QYRWIC34`M z_+xae1s<~{^TFM!xUyJXk&t;D+cSxKCw8%g=d|A=nEBt?ytQ;Kj)(g8>276pZDV746^>g2M$Ur@PURW7-^P@u^&b({4*BYY+Daj6ZD`eg zn6~>l&iftCemmCMGH29!^`?!DfmxBC<5s@+Mb8$ZpCGiCBMriAor5meO<%qB?Xtu7 z3y&OG${0ANReCQNA^9nU_PG1$Vp6yL{d`qI@LcjHn2k8dlRq|iT5QT0RqW_wHxmE< zek1eq3SBa2Mj#X~gn+dNFeXTM#Q^Y`nVd>ej_?&1GhqEHa53jo$dvwU?!ew*RDz@? zL8b8$ee(_b_ULtYT^p5K0Od2N&r01yYIP@Z`2NF-7bjQ=4io*01W<2HA%tKaxyAY< zxW1DalK0Xz(kkRVhE9B`E-g4CMx*o%nPYV)O9J}<^`f?Ayv+URR7PEJ?U!IW;RpF+ z&pG3Dq8fh|dRpwMYDM0NOd6EH)WUg01c}n#gg3Nxl)=U8+r;`hHzK)-SigMd+ID800oMNfC&kasLzV=hj@c zbk>kl=3w4jB`4(iDpt1C+nrQ4mwX@L96T^}>lamWZv*l6CSqX;nlE z&fDe(NpA7{Q6-K^v)6i)FqVpa+2hV(@f1${9yRtng=D#JMM=~ah#oI^oa#&9gQa^7 ze9gy`mUiG()tPssI}~I*xcEp0N7djmV|}tH@XxYH5*GAR^yeDwq>uFSeDH`^sF;Lb z;o&0n_I+d(%gStHtvMNJq7`-hMEeJ`hzo-XSwn4%c8U{Qifyj?c`u<&BqstOJu*JH z^<#HlFyf{h@Pp~*M?cQ9m5ZhJ7BidndyijZ^-bmRc5@6F4K}GWP7HJ8V=9lvvo``P zvT!c1Gh7WJlxR3R^xZn<%eHqNNWYl+C&4|@OCE382y7ER@x@*C9dEr>M?aeZI}5B{ z{_8k%od>Q`?gOTWGFmkkCIBafCF(qJ0o5w(l@5!g{iCLKkD>W*W%u9?0D|nZLVldX zhbXuQ=PC`lvCK5|Y5qDnO8xulPdD_6bqJR+L3d6GQF2&GnDx3olSk;>J9-~#MMfuGzkhc@zXPs|8~ zv@Bh`1L>1XjrCnz>rARMQ23typ7au{*J^f~&-z3g)(cWP#t|^bixFb3WOJnwdw%hH z-9e(5(6UF6Eo;WZmbsW(l{oexdxSUXkD_)jplT97^lGPiy}Gm~MKi0$>&c+Tv--o^ z((0d*0rAS6Jd5NUKH_Uq68gE`MlOrvwtl{pR0C742$pG`I46s^ojarN&VKr0q|95L zn^>pdTME0NEF+CAJkc|_x;wgHlmP5HQDt$5R+pK9LCg}~RHp*lK8}ugF9X*<(sj5d`BF&Wg3gj`DgN6ioM?pTFDIkmzo3}k@kNk#&ViH!9Y6O_GVKJm-Z!%IAKUf!01Qn>i& zZjiWD`74R(vC~VL{tS5#a36%K)$&-e!Hc?dy`lBZ;^wN*3j#IWUYG&A~6L@)m0OPerz()`~`^A$5D0;0yMhNJ) zGJGzlyT!Od#7(4?On#;zc=G&9@jwZkLP!+8D7|K1r`=&$@jB(cSh#zfFSsH}@q_-j z750S+TUa8S-ELgklZTV0FC@sr+GfBe4-RizJq7&Idz!;EYxn%&7%9x+P$9s=?4CQp za+_<*gIc{hPMx4yu71s7G_$k!(shA5HxKbDxc`QA*3B$!RvxyoKJ+pX>bP7NtkcvJ z5}f6&&O`luMz}VbDf{}WFpWW(dp@4FkcsPdf;9Oi@uQpd^t;y_O8boh)iSYvcl(tM zf}il$2QX4VRQ@G{LiT!tuAppU+mOr8iOC zXLyazkY+$FEy?#c;P;STu9U-G;@k(CtXiFLKAh@`S_zJBywTZ+(B=oaTHUQP;B zo8_K0NzS#W#BM7)A#PTT*5#iP&ss)AT#G@4Hx0Q#Cw4kfy@!DYAEbL-I9sqG9!X*V z7~iH~aG(ub&B$3#AxQJR=D76V$7ZZZR*;za!i?OK1&-cJAR=I6oyBBE4*sd&ta`6G zW9V66&e8v50Tco-Rwa78Rld?|r1^yHe&H$6-xZ|xK5GO{b*Nz9ElE=J9ob>Edvu}- zz*N1%*qi{Ki9;9x=jYYyjC6I@%Qoe*bRnYJTLf8x2V)MeRvwlIQOY8T$OQ>=d?S8; zT6Ilv5i|uL1KDaz9JzV6_gz0s#!)l;?szGiNi#fowm*3FCUUN)p2#T_3!r%6?2K4i zsu)yuh7!cr!HnqW!H6$qcMtCT6c^|}$Hz8|^KpQyTCa|c4alJ=^pPp#jMOuepzZ-J zQ_{EB%qM0Wg&;Gu9ITlzht$whPArK4QV~|6CyON*a_WO7+}rY&e1n0ELYjUxYU!k( zr5+8CTC&nFze?mHdCk?kxY5i_10V@r(~;9dR^wnEdC;!x2}wlIa9I^LGCF;#ahKK+ zNl)9{#~$T+ALN+ah}h?tb5|k%Iwx0h{XfDl#ZnT&wXI`)V69A+zR7A3-$iBRKaiGwmXSo8wpq$ zyHGk>dH;_D=+}PhwTXDrz(b-L{70*Qv4oHV-Roe#Ljh9=u78H{92w{l@}bMCO8`(& z>grvhOh`u*YV88pw@@ge=G26yatv#Z{Mt_S>t*^+J(uq_BzMNPwB?#HeKteUMF4>S zW2?)0=69k?d*pORC)}0lj5_#S7h0u|npZ;czn-t}b2R=<#X5HAki{RqZkI3nCp)7l zRxa+~za_=O(=tU@*AI(+>7MJ8-VUMePx9>zIY+XqsZA%ocTH39wBT?MIf?pm&gzmR z{0b~j@Qbw8FEcO8^)=1cX6if~zsWY%4%k)9;970O72mR{CHAX{;dZOUQjoPkLae)pUW`6)&7N1&M#T&38proXr-3L|=I#0veBdSDcf_ zqDI_r4CX_ZwZteL0NPqukVN`2SwNkrwXf6dyEGv$z8rbr>&Rj$d8b)r!It&?&q#qg z?veB^&xh6bWTORZ;n`}ync~0Bf7onM zM0Q(s^!@2-d#JcX``ytf#+8M(7F^;Uh7VT!h0I$qX+M%WH174OB0zfGfiJ6M<44Og zYvi7Tv4d!;-m$9HkzVV?P~sqi1i`<>2!mi@=eD76Gk`Oaz5Zu=6-O6x#=j<(sFyyF7hMj#(cBfe7!-s zmRC3VV`@fJJ>Ln=PIyIAKzcN3qO|$A@bhY2<&8c9E&LQ5i}S4emLBqpf-xW*jq97) zP2RvP^^BkHc4pO++~yUn@^u3r+=J)O2tkI2qwPZk46$-bTsh-d*gQeHAgZLyNw_nf zs@?I7 z2TNaLd_0?0^AK-hHV?48-dkT8)dL>YZ$+`mp1;nwKF>0r538o!m0(#mZ9X2cY_D{= zxzmdOyNDrPYLZUuY+d+XG=uYwGO$Nlv?|GdzPSic&E#_;(kC0eI0mV9LG`4B{Tt(y zHTfr)o+|T&r%26_8f?MxzppzpaQIO6SZ|cZvL??Lx`>7wh9y;|64+`u5TGo!MEqW9 z4gIG_SM=StW|Q26%9ido9O`CJtd#SA<*30$+dQ4KUGcoQa9Yb96Q{FqDwL zd{7cQoTNDU;V2Xq{%#!-SmIEPGrRKv8g2-xj9-qUwTMW6!%Fev+uzyNIF)IptHRT3 zl|`ZBy!PomNE=*!QdIUp$U5&b(O1C|)c7zp|4w{8rf4jxoxlMcnx2CUsu1 zrF4YjoVQ~tvA<=t8h>ZZ%qy{!wv83a>~+cqJ!5+lzw}cUXdKh=-@Gnk6(>FPw!_$f z`HGMm)W3v#0!s5f0&iSt4e8=tUMNv^Y6a(eUK&yJ$plp<$Mr#xQJc`ahk&LQ!e)!B z3!QJwDehnG(i?D?_Pxi$&sBFhVOle)@@wszY>jer0NKsD%~8|(^H&x-<;Ho$;%6P? z2~@ll#hk{)9Lx_HI}Qc=QB(SZ>Ax>q&2AcUgj&iQ8P0r|N(rS4V`y_Wh8jNnIQ4W0 zdk+nnyppEZ0^HcZSYLz$PyO-hj)9o@Sp-C>4dnU1Wq=j5(d6=RzbshrQgMQmvu+*SY9AMJ^@}`qdhpMK%;(yNadRkfS`(((+k1f zT2|!pH&b4S?omP2F46~NEk}Lu*@6|#Cy8d0Y$cHsnx9;q)7j4l#N#c4 z*P?f31J)W)uIR)|Mo!}2c~ZV&9hh}8HLggU>tpb{9HE$CoKO6iL##M6Gnuhmg zS6i%n!S#KH%|=DdOWQrAhk-=nzq}}IA-n9n-esTVQ^jA% zKN(3^gq6v~0Jevx3d38<+nSaThdTAv%|#531q&V6mah~abnNoEXTjsOcaEat=P7`t?=5d z7{;7{t{9<&67QGgB;W2Db258v){EK){m{FY{l!lEiOjOU^trBhd+trXp}a=8f)DM| zp94X}O3J*7c(wXxA}lTTb=Js&b(o#E_3drpDV-?;2O03m`7oro+X|G^K?J$49+|x^ zrB#vMH47KkKT>0G>de*f8Vx98oqyV6c3pV+ZP9(u5-oA=sq;R#8ztQO@|lQ!EawuR zL$}=)K7p_G`9Dt6ee&^O zUT-t6Y%P=5MVy|)XVcM^A4|xWAsY8w)0ZhA6-Zs%(KDq;3O)o!szZplCY%HC_#Mjo zKdc!0$eo6e^RLwhtnC)L^CC)=)7$)2oLF=eZ}Gt7uaeh;0=(#GT|z3(APZPa|B`rUP_5+#x;t|(CTbafdG|N5`8f4gn0fjVizxb3xbA{Bv z++|ia{Fp$f5Lm!O%FUrm%@@2;wYhXUm|Mh3=M5pbhem`P1k3C3uM&B=qa{D>G0x!N zD&6F0IjN#?b(Y%TU=Qa@eqP90TDGRjxz-pn%kEHOKU3&~viZt(S-304ehZEg?P<7h z>AIm$clEokr#|_SU(XdM0Z>3zTPPS65NC?sYm#=HT9k|?YfcQv=93ii=)P{8upwjAO&j)~uKt1B2(TRI!%g);!~o$7NWe-A$iS){v_ z%wZasebChKK6^ykzS#2TY082-f2-8hQhwSW5sGO;Cp@<5BxsQismlwHuMcXfGL)aD z-S$1sA9?M@t#D`9Q%;J*N#FIiZxBsFGA%DBDw3F??eD)f_b*vaFzi$gq0#gf7&J(r zr92iOhozZE!~P*zU-WvI{@98Pe|xTJ*xxtnKZrX=K>W6MB6vk)n#;#JF`l%qD_TdM-mjEEeKj{e97Psr_8o z0#W!tUo8zVqcnIZS09S%ifGvG0z;NF7=f7IT9!z^&Li%o&5v6t<{2pFJkx{u(t8pg zflsZd*kmW)AR<-Unzby5*647u5Y##jWre}5)Pt`f&=nS%=ATlcG*?Gk2cxF`-$p>! zoz}OMP+Tp+l{#C~nff@>pZu6$gu3v9lI-#tEH4?Ta*p)W{KM+J`ceyoZ-Z*$kv~$uaz>ml7Lep6F979?+Gw71ESdm?8%+?o?LjCfaTRHlIXjg<0lf^WZXp~sB zbflN3lE-O^Ti39SR=&&sN8DR@1@(5_qJkjZ(j6j=l(ck*bOxl zR!ZHU3u~ntiz1sFn|gpGr)l~9z5stocfyd@4!=E2hfx0V{zKcq21VPO3yFiN)~i62 z!v>~l zK4B>zpGs97^%D0coO%UrAVzNS*%yKHVNd?0zj*Hk!wDmEqs{o8l&=Sf{d=gZC*_Xk9*H;t!^i<&?MH+g8&ul%Zy#D+mKLim@gvJ`E^N^v z#d1QWs-9AiE!nolm-%4KD|<%DGo9@7r<01Q#X_>c{-A^s(J!1I20ltflTHx4hhy$|mS zjHzoq@q+tS_3OT%!JkTINc~RXCh?*dr?|e&%?sDBz4!Bzw4O0qdx5Yg12x&3;*0zq zcIr}|)A(aYVA+m`f`uAsZ<11{i9N=&)RJtljiM+5{HGx#=XxLf#RjCM-wB_PHXs&S z<}bN8Op&_89lbg|In0>);A7AuggP~y*6*B_nZFvJ#iy6IBwOH^1$m5 zjR1TcFzgBlr4tgc5q&Q&OX>5Pwt3M=_gF8}T-s(2eh%`*(`yP2sLI4uM9%siEbZL8 zQL5QOyS8RM*(SR@B^?5`n{SXmJLI>=f+H8wd^Hg%TX$`=nZSJW`^ zM_QsB-41(M*6GkXRZ5LvM~t{qEIG&|hW1eV0xNE}PFjUgICl%&UyXNVwQsrM$bqpH zzhB5%!1r*5rwWRk@~$l(F1uVUy>BLJv7Nr`?Ymz3Y&lk4E%F&fogoM#mm~u2 zODpOz*;5>&%80sX^{^KQxq7YRlYlAZ0Y-=cQE~-_cS;nKN*z_$Dw;YU6B6(Wv+v9u ziOMjd#Q4upiZ#U|9_CAYAXDOtyYDx3lY@41oB;%no|jx`hcEbNy+>ZJhn{-YAdUTb zQQJMcRrYCrb^`c4qIBPJVq^OQ-+*$8s!`Ou+g!=IbL|=0ggmNSFN0SMbsBhd zBi|WYbHcSIX-4l63a!n43*k~cOCo>u^(=E^+r>wN#hZt=K?%N~KFR*cpc6-x``V{% z`Zzc>9r9>`7zWE|TfQOAUPK^9c<)(=c2VRb4EY|s&0G&MWYS>}VFtM$9gzZKOteCXX=!AEzM z9EdrjQK!sOep62LZ#%1&7imSH3-AWmQN!h`Em;U4Dt=D|0u%UzgPT8*Lb<62U#0k$ zyvr{q(_W8#>|za|0^Y_^g*$hyCre?vs0_urf&_&Z2g7!j;)iLzvN-Pa8;6&0e+skl zh2!$WsNwsuXO-ZIw9}0!G<^;u(&`;x`WtJ)ooyR)bnm(JY0Vy#5}%%*vf{|};Xa4& zA~K+pR`Kmo{_x_~Z#(SMo!YLyyZ~%5sX@!Th|!KGa8w>FG2_TeE=t^O3olJhTN|zv z$eBEr(SWk$oEa+ZRj2zQ%TPqhk&b;Q(d?oB)K@;GBc?-zO>;?($~00jtAG1~*i+Vz zM`lwCYXWZ4!eb=6?To-P;fMC|0JjX z@-oS1w`fB5CaQ9I(H_HE>|6ZtuSxYF%vO2lWE;B0u?$+3li`36%E(5Tf}LRX#`*|> zwHG=W;@tlPw3R@N#+#XSy4s=D(VEp{=5_blA)|vQGC;RvDtTTRG80Vqf||63b1{WO za0K&hli7?PwmLT?SUU|&JZ$H-r5Wi_W`FZWrT@tI-M1}9 zVLSCS$=$d7rVKKn7Btxpgp*ap>~OXOx)h^-Ax7*q9Szz)o9A(=NjoZHEwrsO<8xYp z^!iI(997Jae6~;Jw`^p^1>dLY+Y7=7m-~=$c8}j1>n*-`UQ@C+>zsx0rQCN$mi?5<{;Um-%$hUoWGnD2^F_6DAijWpih4ASmzviU3wKDC-%voc51IxpX#r8&5BJWut345L3v_>t);^fdZjz)qaC$_ZzjN9H&<~+Sr z4xAJFLuR3GY~c?*u}Ch=8uv1i<9~1&Kdnq96?4>cgk_&pf}5HytK5n;eTAh^0h=XE zE%gl;li0e&G`!%Y`Lf+#V7=x!SadmTL;@TP2 ztX%Ut?Yi*`Fk(&6Wk)LvDi;xS+?DgF=i{@t?j#9)S*NDT$59Tls93We zH-GvHn$N-pXD})TbIx*2=bzA%UCAC}(8wWI4g)DBc=AEE+Im*L`mpJ=f7cDEZt+nI zcgeMewMEh}0}a4A4`U#l(<3g9T;HOL^nf1Oa*ZilNV4HrWZa}TvL$=ZjOXxPe`ihS0d5ma z9wpCI$3t-{hCt?g>`@RiZ-q56Jt@vpU~Iv35!RQjCQBK`|1qr*4eZpb;4YaiS=;|a zXx4Jpb$oZaQE!$s9hw0C2@lMFw-$!kG$m|lk7^e;mNOHzaVgNZ?1b=aJ$$p#3u||l zE1oTG1_LAeccdiy!BccayPZ`fBV; zr&cud2|W)-d(zP!tdsng6F^sAIA4Y=iesCyX~{?VT>GcSLlUkohm7euC)y2NC=DaH zKIfibY82&}VjjtN5hlhVyF?Yp0hm_3RB#N<`DiAlkf zzmm+of90-~Jjj(l$TQ>jf(H$qGb^MG9usL4KJ~uzh?26H6Uag( zGz>BPftf$D*lWgu>OpDOsP5+)FJu(01o3Z2=xmS2%6PGV}#wQgkOmfd56 z;U4|oT*=aC|5Jo8;;QsppvS;Ijc}rB7PWc)5kbAuW<+l5z$@M|-J^=K>?Sk;{XYil zaUUXYs0wOG=!coM)9-q&rm12xI)!?Y>J<(Q}6<_?&Kj^sH*WK8ygiQKY2K=mg?W;0H%NE zfLpyuH7ONREHJ|&4{kmM9@S1S-Y=;g>*&rSK%NMoZpL9n;}ol)flcD&!iG681d^yk zV3ck19`oxV(%+J^Q5`J>LVwmiFR@95|M)(C30K&Y=7{DbrZ}LysdI}VGMxI=>jMWC zb^wS_&^kql`yu%}qj9pBKJU-r)k`1zn@RrNR%f=!ZVpZ>>BMi$$1*^&pUyC+dnXK8 z;N5?eA3XttALV1uw@DW!eB>Ovjv*ySvTml_s#P=Sj`yT!sdBTCK1?CJ+m6G7E)1M2 zHD@kwdGA#_h)cS~uQosDKiO>8no2|Ax%|8{yVLi`M{$Y#yJ>Nj*W{kgp!~u@KtUat{mkQTYLNQ*v(2}L|=JIOBncPzc zb)L(_T-6ewea`PC;K=}YYh_b@le4i>cr=o)_&QHpPxsc9cEmnsF@7utp+(Z|I zlzGlQ?)$lLiVwcQVC`&ln8|TzEA9a6n_6cmC3j zs!xH~NsNBEi0}Q{Ft;eo>P@SFHb+{_ZkLrCMcksdC>K$PAw`kml#jxFqK0<<@-lNM zE~VK`o6gj)eNZ5i1h}bb@DYnXAFW=<5JMp7=8CJ33}|+v4>Vs%$)DQfhMk(EX}VyO z?KS`6(HfFkr^#yg@{=y*8nx+)pA9vNH@^qINqyT9ZSRzZxSr{ndgSY&m4sz>dqS6o z!&~fdQOqP>Ct7dNrPkgPM!TNom}}N;ka3@fJv326egcoB45KhCmXtG~9%QY!+z>^^ z`xaKkqLSLcG*3>RFH8+YFN%S29PorGfUDs_K|`Ng^D`fve(f?(mam4q>g?f8%Of1xpVyZqDAs%m67Cq zmyXu;@sHijS}HM<K1MqmMZa1Vkz|`Q{vF=XIsmJ9UK}Ofxug78J5$T zm`{GO∨%9EMH6CHKqAMdvahXIa$y#zg44jQ$-Ae9>rW#b|{(@LwWOzZz=o!`(Y}VO#w9Zhw9?)!$$VtA;=V>sNi8 zEl0tc*7rWp?p3ebMv@77zS=U=jlUHuTbjnAN=}3oQIGIV-p7}nN1312Obboj6Z;~C7E`It$ zQs?!|%_2N809gYosVWmws#bM4FxMB0VEo;k#pgwLrtD-dERD-&<)@}?LY6#5-ok1Ip@&JJ?xGOC z?dU4$0M+?^y+x0eiJL8$lN4Y2nj>fZSR|8SI9O$eQ8+0Pu4^h=d%SD>A+1zEU{#|T zfzmvmp@2y$wlN=DcaJc>_3jL(=G6p3Z^VEO4L3@JpH6j|N5`Kes-c9BTe!FH)jZpc zu`Ey_vvxMxbPtuCHqg^?f@Z_Q#zL6l*`C9)-}?9~0-w(hN)Av~SUb4|AE)qTK0$^x zMweZkefb&d314fhuW_qL+#>CiMAg#_GH@zK8}hG)ne_H@lu$g!+{{(`-VU_>-2eGL zgAwn+(ecPNeaw<53Zf~m+e%7!jc;M=Jke0gS`6+2jo-snF`N{YDbn&yr;~=SPV$uo>ihdeIG^A;5RP48OQGpKQNjebv}?h>+QmS zHY1(Eck-mhh9i&!kJ&7(>xaCVMHwcLG;ZH*UD(I0+n_(`-CfK|{OYi_QxYI;kGd6% zB^1jk?O5si##d489GnYbEbD|cK-8xPM=&s)YWt`^>Dfan7>>>brLileRmi#l z{I?Kfr~&^XhRh3{YS!O9y^?X7VXeee)iO7bJ&dfBh_}C`5IM#ghA(OJM<@$ZCD0PZ z^z)6sBj(TC5UABbz2>Q@N;{StVzNZ%@0YIauBc8DdXHTqN|Yo{Y;QeIShCgg?`Kfx z(c}us-u%=P#vxXZGV`)TJx%u7t7f14-KO&dzR4qLGyEt){4I;jC&KQ_j6t5mmeg zgD$W6&<%M!SfE-`s$Hq*ADj3ln^cKE-bVokQ?8HssU>N+*jyh_NZgk=df6g9g!4|@ z^`;hdOTFi$NC8t^cU_yng$P)oQI0A|#F$K48Dv2e_{c9YP7QFjYBwmq6k?ylZ)J}% z0TYewBLgA+K)Qc@fnj0Io2F+Qu+uSStdD5QzusM@#8e`+90>5+SyCcPXDhGvll5o^ z)xTLY{ru?bkK=u&2HOStRi^1PfwQ6wb7KC`|M&tds%bNT=3*+{ms5?B>1XPHNbm>= zNiAW8pD^S#(|UKy?mPY3Z`IUo+kbUweFFE1NO$T(nFb6y8h7c>oTsoMimo_6+-!t# zh=_&7atgyhHqfPFRXcw85_TP_t(*LcCzaX91*YTOsCt|fjYqyVSw=Bdt ztR@5{FZ~h*mnW9PWv0;@REWe6mV{c@Pi8I4>_5J`3@PP7cX-aIcH0}pziup!6_U@< zA#zKa81^8jvf-Kh!6AQ%-Y_IdXxch?#KOw$y^BJqUg2>`#q&(eX@jWq>6QQy<8OiX zc~BU>=<}dcjOHD~q4p;lIcb#Q-b2pSmW(~@vWFyANNb?ZLaZ^IWsCo&9fuv2-Egl) zKwXALWXur?Knk9oq0w@3%i!HgS3XZ+69}T<$dg=y8IkA{EOYOc>OJWzlXD<;T?-gw z0zcL;ur@nUBK7DgDN>}5c=+TQ55~N`LV8Tr$T%;5J$g3HgF4ex@~xAX4K=C13PH~L z(l$T`WRt?4KaZ>*0M?l#J<~w!(-=J3ePCJ@w*h}gctb1>q@b8>EQx-IbBn@x)+X=8 zu|n?um(BNmg24qHDU1wh37B%ymkhMOC67^GM}2obJ+Z zww@vlfuBzvcOJ-HSBEgUpHQj4$rfgPS2046hNPevK?F1N3lZqKEsN`HRldmj^Uf&ApT=(`>+l^$TNKI`)<%|54(e*)7KgE0i0^?0`)f5dwO3j!r(iA32$DdaG& z9-g}_?xSD&{^}YcOEk*bT`<&Ite^M^Qxp4*Qh+|8K5%f5dJr`$2XAx z7Cv@)UQXQk2OON6DFNEu;=#aez-9Z%T=DZM)Ot104{l5aiyLa{y@)WXT-+r82i{(5 zQp2)!{UGWs^LACJ@D*J)0Q!S^%U zUcj^NYSg)VV6HFT2mSQIVJ^F7WN?srGftjXcq4-X|4gG{jH`Aj*6PP-+I`Rytnmzg zf)MeMpK}4GxQSw+^e4qUGD%BL`u%JU34U}i17)OUdqx8Mzl$$1MmP{l{1 zpxrbf9b1G+2q@W>lyB1TL`VfNQF6~;g5E;cJh__#+uontM}M})MU3P33o_2o~uQO(csW+y?(Xy zZjMu-=JJ~StR}Hhp4$udxPZ(3SyG6iLI+|6jzL;3vFsvee#jtiZn@hlqlGLoCC~)W zpU>TMBKP!z<;@<)xQDI2BT8I8h%blwc^yx{*29&mOBmv&^#m$TKA0%+pMzB9Xil=6y}E<-*a`xi~qERotR#r9P^_i}v1RXK&7#pS*@7^&A~ z5|wHR-F>u)I~%Sw>8}k;(^Aklg8ye@PVRdoVKTcOO6;tpAb(^k&Z*Vtss#_3+<-?5RknUSdFO+WhaNP5J@_7eFiO) zYq%wF*|}y_+lBR{xklFt$^2}JNSTq^nV0+UIC0F1rDxI+0Vp`^1RZ6PXBkP9t2A*rx7Hm;n*t2vTpY{!r^5` zC|{Xtes}Yra)~(h0ab-5GI7Wx0b~64p8YT#6fsHaD>7pb=F}Csy~~ChpYwJ5t5~vK zFYhNWbp0B>qYr)b`F^ABY%|->C-@8z(2c1qIa(vml}my#w|e&IeejdD$8y8^c9S`} zmWswk@$^9!u?64%5l)6ZIHC^_!L{v%^^q1ZOx^tjiUhD>-$Vkk)WbJ6sMPmP|D=vz@~jYcAIm*)nN1 zuNvXW8s04|;9rdRtw_p$FfZdCI7u@w9vs_WTFHU>E`2DCY8xDURDsc?UB3BY0fi-B z-95j4d8-XR@DSqeNWtiD@Jc_~EK;_bMGPQ3jC~8|6-Z)L0}B6MGI5v?PK(r;`@B2l zC9T!u*H$x!)pGKO7;WD0EZ09>5*qfuRtD`HWt31yJ6a`|M+l7P*T4D?4fGc0R?~_9 z&p4NkD%8R_>=CD_I~5Hg`OVGw>H#voux>#u`*lmzgm{|8qnM1XH@0|-ZM3;WXZuzb zO7XpKz!@@Qa5b-jJ3l?StkZn`LGKVF&>G(6BJX`4+Q(}Z;?nX{scb+AkO2+}Qw9{m z%Xfseobcv|*2o}7x&r4t#98CmRrtw{6MKws zd~H6x70oz#4W9-8P0$(x!z9n3HEt8-=W)YC3cIyLT0K=Gk5$j-1l_7&%Ein&XZ{*f z$rr3|YxWpQ>=jiJb`gsu5dy|9wCpuzb_E{as_;?fM0LMWOJw2*538$*q8GGY_xe4% z)9N7Jd6q`xtcT;4w&hN9vF7CHn_=rHoHhNC8&eW^h4{61BEES0c(l$L-+ePpqY#L4U6Z*G&Y5IEJ}%}$;Sxa+GfG{P_v(cKa+y0x6toQ1MM z^VGWGN{mYX)i%#BN|Tl*1;nkG7r;S89rV07UnEM&6IDRpntDddpjSi|i~`_gCC#^_ z5)al$W~B_t<^8q@u;#E7YcNKuF3_D=kXA%8u z5`v+O0ERZ|5oAH6)yYGhf@Yafoq5pCxsVyo1O|oFhAN3za`l?x#gGZ`>s_rD8Q9GY3Nwy%MF?A=qToD z5a5>z$9e~`$3Mdb)yEisw-~fNWCHdB??B|S^?-a8vfW#Lcxx@@AFYU}+iiE$_x4UO z?pGb4o6iB=TsNyWu#W*6iUb%B6wjiiEu#5-nWv6hgMPZ)VvHZ(Z==4A5pGaJq~dt2 zPNSTsijl>Y9OnXNLy!jfp)Z4u@SB-$nS$a*W6izw(w}L{{by9ON?ek4!=scFsNt>H zaEgSf#lB!d_4Naz#}fprcx~R4m4${)w3cATT(C4{_}jP3=^0NWvFZ6wl^g)g*E@4U zI#JAt3n^1OGj56}x-k_(P@c=05Ro{$~ng=O7a?QTFzx(*cG=Nh5{ zLjX-UIp)A+#OtAUk!AJe`o>mSO<{1oub-oOmlJ7lyN;-hvW)lSz7>$HIen9e%)GPtkEk~%(S2R>7b?ez|P=1yO zCZ(jmr>Ta8I9^Mb5Oqf?Vm@tRwJ*pvD|QhzJu%M6rP-{M-YCbcEf1+Ev98Q`Q4ZD5KzO>-qvtW(@YXWY2U6vo9Fh(@jWyi zWPDS;#DF&x8b0us*TU$eovCf7c|`dBb(#wOy~pe!N&gaJ z{p)Chztw?V{`O=KB-zcw`q4j9PVKDN2OCO>*i&wKs-J1wj4h>+*+be}3D{0Xw%~z?9JM!UyP~8dTIkzz znn};S4}|FqH$dwSQ}wrkmN>1-HLw5;?RfKCzW+-yUZtF2Zj>Z5$|A=UBBn?e3GVxJI&}%n^<1jL~p4@V8la;-KslRZk2;0`9zF zUV#yO^gzaPLJMU1EdXYQC)(6spvbEhf++)>GEV-PnGaS_exnpu*HxU`471Qw>4IQY*r6U|rge z&gMumd?e|~H*Md4JQh~_n?EL;)-n^+Pw6MB1^I`*-ObqxjQ($NtV%jp$ZuF|7As@3 zwmxg8T&rtPm#{{|X;?s@Ve(ofU{9;O3j*>G!}_#sAN-#n-|`7)g{wUMFYUw>RKORG zVyZOw&j>Q86{=)aX+rSTS0FDKU+5O79y~{zm7kRzWtXUy7 z2Jd@#S;xwwRDo%FphB7hKXaJ`rL1FHitv!b{p8{4dqfQCuYnM+Ms38D3Gff4$l2MS z^VK(i`qX|@8TIW#-`PH&&;8>5SX{m;sMa`1_XgT@raRQ`-=r!0v%>Yv4HtpGuV%as ztSu+WDeaq58jp%FCjn+EFk{nv8dQ>(0S&lpYXJoWD5?&s_o1z`GTG#vae$Nxhz%n8 zn3-G0%Glf;&~`U4}hW`Jb%kx1p^|P2F!Fy zs*p+g7`@>0E|x0MHTH&1zFl1ed*WDHD5#Br5|x`w_gKi0;svNBYimfi!v+0ct( zpunJh^bV3GY*u@ZbJmjo19vL9p@DYeL^N*`dt&Y3nOdxY^6d!AmVT(f~*)uKEl?BB1WmftTc8QB8?%LkY@cOwnF z;CU2Py2hU@m`@Wkj!A9LB+e5%-LWK>=SW{=eE+c=6@p^TU>FYa%VlI{23JN|bcECa|VbYj!is&WX$I?zW#!(Gvn5U$p-${Dy=8Q)|Fwy<@4rJ9Q926+%ZiW}=E$k=%Pa+!^Re{}N+4DDlp- zeAE1RW;lmG=&wPP15+0ErwV0of2vxvc$%VM806=cX?X5W!F_wg&P0p+_#&%IqSghJ zUhKdc4`M9++M?Fnxa|XEuIz1N1%6|B zX>8U)OR+{zv2fF1w)$fL8adt~4sywauCYAe8sU|4dpWxG28;ko?I;AL`H)MNC%`-3 z{@;Mc%Cd8;>AtHTXAxBI5Y?$5+zD4_do$7E1dml)DK&s9wu9>*K8;boP{oMoSL8La zDBKXR{F?xwsd98DU{1*GtR~@yU))@hZ4ljp#W@M?=8bE-k}6(-%-HkDe)JPu{D3k9 znCupEr-gnZavErjx-(xuZcVCPVNJmix~evb9EBlJRsrwah#4B=8}fHWgb}2VCXm?7 zEo$eVCZ>@n1f~DMYfQ=gsgNMr)y4Ex#2R7@8RQ_wPzPcRXL`kfki|t%KfOw_O?kIl;L$r5@A&;JI>W`(EtKS(zQ%_Z zeztt9z{6Y4H%{)yk3sRZKk~)vc}5(;V)V#p3&F9-Pd2HAP&M6T@}1eru<-knxtn+mx#wyF>m$te zAk*lcZ%dqo;L{w|;ffNf+twWKfAsrAmRtE=EKfiObR`a}br0Vn9_6#YmO17`hQerY zHxG8LUdz2Ntp8z<{Py7ThAP;BFx}9b{U_(@)((T?=iHs|-lz}kmpvl7>zhuqlcz4C znc*fXf+|}2-%6mo;5AYLK%%O1(@r6KHxmIzEWPNA*Bzkn{MQp+vQ7&_gAR8rJfI5? zP!+hd=M)-ujV7-cG$XRTM#L5nT~9FXeY%N= z76s~5^NQrHSFgELKxEV^fa(OTOF6W9qgEn_UIomT)+}N(g8glwAj37V_)npr8GbwC z3>a@&Qbl+X{$n6|RfL%Ospl_fpx>55j$*42ml=mSqr$2<5ykC%8PrE$zRb|0qOhtM z3s_b3VQ?`mR0IR59C9BMy`5*1=X?AQlG^miqE0~KUE)p$JZ_<7$K4V#NY0M$cv*v=2+88&>pW7h{0|q`7FzD)T_|t^h zKc#gX170i76d$f^Z5^2WJ&Oh$S7myYqiVPC5y&wV8q!0xiTRx$zFk0PvBb^7j3;3th3FB$(BkkMefp=FjUJcZ(7q@lm|DWK| zbss3_FYi@w3;haR4zQ9b>klWCvFX*y{=oUVpPfLAv(k%|ILY);Ze*^h`kNzonrn}D8y^bZ7RbLWqG>%$d>^O{i#7Rd?MtoMS5uT zL2)bS2~l2Q9=MJv=ScE@!k@D_0>|RjAl{l52et}j)BqG7JtVuRy|dRX|362H%$`Cm zVN?x+_l3)jsptHmQ-1TT@K|0bv|e51ksR!fB~-oqhbHcX)}>g;*2Vnxk7%wk0OS82 z?irgUDI9*Z24)up3J{c5Ju85LAR;Mv$DcQ;WJ>NOAIuWh6=gX&xF7;G> z<#u+n;JX;%VqRfT@r#9j`=t<6x)a&5#^Jmp&;ly=o`>8Z#5mQ~-fdp5>d13J=wSPw zFN54?3FW?5Uyc)xisAox#w7@4$62|pf@|?IlqLg3D{LzL-nw%JRTnK^;1tz_jV8~EnChrnSKOe2=Tj|y^Hyj0r zm`2-KC+6n+G#@K;HkIe1U=89pz|I{Igzvb>3cVfSQNLZ`ZFyBKdCO~O``Y!2#i6eQ zABXN=V)637K`a`&r^nuPP@++He|S*vSR@bUD+*K#^iJPP0P4wc&+R1CYykvm)Ne26 z$<@Bap-`AaEliI;*X4Lo{jQ&OOOeIz(b;t~VuUGuY?N=c$p_QT{OPKD#!EZzDbnx% z5->xnaX{=A$TPNpT&K*c3YNxjq}1*Iq7e6PF(ETZ#iKOvl}GcuP(vpId|k_zi?q_# zKk3F|ANELkKu_FZUH+gjn93^mj)L6*5*Yb@PdB`xXV7m}+^wDsW0MPs!_e1#ul`#Ck1H4$>0wJcn zN0)wix(IRETqN2E7SQx;$>3$(*lg)NLnWLvhq%p0Y3QI*?vW`5rSiEacQ3-fmYDm! zr5f+L`J_~0Dq=nm=JJMIZ%3v#rYleQzLlS@e%Yg#PsWi$YqSjt@6IGXe4Yrg7AGa` zomD2K;OcdWK5`sG2p2j0b0y{>dqht`>W$)=)b47Z=&l}+ToJ_PbMvhA)M;kuAa#>+ z%i;jWG;H%Lzf&$1mz*WBaMM2Q1yAh{v6MVJ^V0E4=eWmXT90ydCKRu{BUW@5IE$hp z4VR_ss1k(EvHQue4rDR*Efr&bB7f%HtK56XAoPBmqX4xWOZYIBf}Jj~$H^5Ybfd6Z zt?AX<%j1{onjEa7>$<#;jb10LPObaaxa=%uhQvERKuj=QqV7k=-DNnxZ(dFYL-81$ zG76E~oBHdD36K1%>0#ppfVljr`6cmtNFrTq!myXj7m z>UrAlzDD{{J1HF2Z+4nLvBE@Sn!jL`G#eTSFyU;_6Zxq)QJkX8sqBqAqCn0Go7DEo zKj1-{0H4^$8}tJe@4VRVchDX^lou43)ViuC$@ewYZ{V&Xe2qErxdz8j#b}4JbAcjC zOLg_$!JM;c=wZ)GoE22(=VonSeRxFL;akVWA=7#fID~^g$xaf!?!>mtR4*e_EWR?+ zn@CpgDMBSF=sR^9dOxN=P2o=0+L>GePocLWrvG^(s@DP}ox=MqVxL&-(`_z-Hg=po zh6@aTy!UN#xo>_Hg(k1wj#xItyRO$JQD1%6isd2Hr!PE5i`uGd4oY4_vR2$SbDs3S$j+lZp(~FG6Q?8#NzQT ziD`T)<)CtZkzytCIoyK?wSI!$*=575EPk{=G%BI*lf>J6Bb*q?ZN?4$vn7fSTuR|l zDlSM{xfn!#D#}51o=gKvu%^mAEDmz4PpV9RHr_6@dqGg^v7qw@Iqk97pE(EpboXn_ zA9rMTJza9|WNG9s=*x+AB+1>0*yW-e)IZ&om$2wEO5F<*$Jk(!ERnib&vU{|G^)%q z?kW7jY@#R#hjIS|9CC^FyHB!H(e+*43l2wb@>V(hJOv9wbc8ZJ=<}5PuLue;m{Au0QtOrD?i`W{;h{oOFC?tF_A0_D#M7vJn z#Y=nEs{ee;TvRx{*jR1BgnMxOw*SIuXkak~wUNkCFz#2N1r`IA-dwz%XLR_-PSTQK z6WmWCKS(qcdQ3$;U_w)^N-hekf5T=)j&m2na%G9uw=4uWjT*Vs#c>JFu;y%~|GkbW z&qJl|j~T~XZpSdeo>9W4j8MJ;ujQ*b9<`D}rrhB?p?eDl6^|Pn1Ti zP>HB!Sg-!Gqc=!XyXZwFa@~uTDIE5#kKRao+z$h&8B!)X_C`uu1aZ#6+5RphYhnK6 zKoW+EOr=ruyBU4wsZ|ZkLjx|BI9jhCvtLbg_Xkb4$ck~i@3!tG@=zGNJ~5wbAbrjG z2_vnFmWLX?{d-JYzp$O?M%oU--So5q=>gZ6u|H8iT2njoj}e8jhy^S+VhDfccGe9i z8_OY?3^g?ujungM3?anVPYH5kxDcTdi;JN==C1NWvfE1Tf{CqWp?^%wrjQyCv3-wT zIr28xWUxIx_jZ1GT^JB9fp0-MFwHOO@^w%}z{9Cjb% z9TB3zc?wddoc(j##KjQsEqSvd^ZNKIQ%J?7e$uRthP}!^tdtgv?WHIY#02sA@%YDk zH%)N+Vllet$a#s%4|I>3TFUx}Ne3oUhS_U!2u&AF1=cJ@bq5!alBYdxpJ@6XdppaI z$AV@sl%GV=>&_!9++rL`9NGr$G>-qMr-A!SR`%Fon-kIK zxQp56$w#lp73Fa@m){L~@(lB$i~7k^@*n%4FfG^G#ZPm8M)RGX?j0UccUpzjupXEu zl0Vg7u=7MxK_BuxQ8DA_r6_!T8%KVy5hcRWWx#oOjspOa3Y5aLl zK0o6CMB<*;6x}SJQmV}v{AA_76sBppRq)abvV}hyW-a;i3|cKggt*1%wTYJbo!D zl940A@%jRVD0LFk4F37hTf8!VW*(H_qEf)_hnp!y$>j&-1E#3fN)x(ovqQot?;So2 zF~3(l(LDVjQ94>ITHR#qaQOI@|BjjHaF>Q&uQU8iu=~8R6g*5J)@WaV%|%AtHNWs> ztnkR6xf}U|{iVk}n!X=-)7hyu1fS99g^Rp7u7RrkERnGP2itc)>bpFK@kkwHKWI}djO>20+8nYCF8Lp;S9U@t_A7Eh_M_8I->##X=iIC@ONhV3xo9 z9DKO`H+Bxcy$cQ?e{17ZG@zRJVoJmpbM%HjniMU1DVVBtK2l%ERTQO3$fNRMe z6=N$rU!HsxgA#aoL+-A{6=zPoN+S(F7XiY6A$h$|Eswnea2oXsF;`c$4?5n5FZ_06 z$X1}&=TtEyk|`%Tru#=IQK{`4^*u^u`~Xu0R;=tSbAr40Q0|bbZXd`a#a6~Qz_5lx zz@x%uL}wZBK0IQNli+Fa)mt6g{2lQz0W)NP$t}L{4NM4giOMcK%1-g-JU8{#V-0S+ z@PB`7KOScl-qexsl7es|97VYrWbsiaj7ie&4;1T-b1#uWcFg{c1VxF8fiEvB4Z+k&7{ zp=Ga3Q@+3Xn$JV;nXL-G&Gd?0u@Wf+%v@aj1H-5VcBFC&-33FTU4w$x8qzg!+|KSn ziTjD~xtO}uen^$G6~*!nQqu8G`iJwtmuU55y{=-_i1ERtKYLl4db);IA~eJ+kw+*l zn&C3%+sEai&};N$Zho?g%yeD8ari)WbGBXo;Y;Fc^#}7SUVc9hWq+Avm_QMX`?nSx ztte+7RJDBFG9F6?Fmbwt;J5yuAp{dUVF~2{;7;UfM(C{jpxl?Zg7@uun`}osSv_CN zQ>ow*#FHQ7jLGT2PbbeFhRRRti99VT)zlmsH1a*c(O1fmjuqa``gH@sN~6`zj(rBp z$^U5P+~1N+(>RWb2hx)9kcTZ#Wd~|>N6^3$K~2TRJd9$RBW|9Rv|LXk8JDLZGY{hd zV`ZeFtVA(m)qvD-ONVsK@|2pE#$=-5Xo{$1?fYW8*X}>C*S>$j^TYjq@6YFZJ=b&6 zleXia1-;M?*EZ{3wRd%~!^hOjY&d^n1}-CX9Y-W@ThpeHAoR88{O zX~MUMZ-h5G_{nk88>$9`BZiM!u4qN451I8x#Y)wGB$x|9oW=@<(sf5*9iq@4=1*kX z{1^ZQy_F~kQkNA#U5uH>hH9|cHE@=~w;7)rx>hh$$n{a~#MV~P;b@J1rXbS27y7S- z^93vBA`EWqZ=@uyzKon5wlEDr3$Dvf&`&%^?Wn3xC1A$raj$o!1Za9*&J6vTT6aDk z>6r!lL=DTiSNhhBlylJ(_dZlF9`9ur{&?PS&+>`s8Val^qKi(u;7ymjV6iPFTKyY- zggT5Hux>7?4wMwR)Lrw_8=YzwfJ|!su;xh&3J&l*`L*+=Ub&5gt{wl~w20ODj{Pzp zL0~@ceM4AT*AKs6+YViQVl?}KgE{uoO4E+YuyB;~ybmLHMQeJ@iT&ejDu#V2CY;|o z%kDdnd%IGel}){-J7V@smA85o;H_rxK*&)R`?3k3dDlzjDGZY|yt5Rqt@V{*vbc|0 z*jHAaz|v1E;@ADrIEUiG4TZ=e0cDg$~8_pXnscm-!d6biOhKbg+E&m}) zr6A5e`sfw@ZD^%fab6NRXy&tV_p^F(mNHOpxMnUaS$n>CdDb@Hful392$khk4H-MR zD>Wl64zUL=ML0i7_(#o5!gho&zv|l&7R=DVOhpSO(D{AzcnaUa+uC<)q--5%k1(}O z(2bkT<{IV!r*?f#s%YYwC{^v92Zv0X8pys~`Z$DPP}&E)6QLbgya(Z|LX^B?;F-}D z<$D{8F9o9lmo7JivMc5kSQ3ab(h{NO{1Y8pd=I6y&0xIH&hw!@Xy%evZ}kO7u1Aso zBpi{co?HU0e1dQ*K0rOb8~Go?>#svcK^TJ;z%{I`xL@1cb<-mi;CU<^Xf=vAQt4yQ zKo~5oId`@A6T@bTVm~&QJeduEV7{#_&Ln3>6fZTAI^~0+zZ>9>g!>X{Pvj9y%Am=|vuJ7tNg* zk?Dj2WupLM>18MSv(-}l*NA`nZh>s_G`HJdh2$fyCv{XY+m4uI+Q()b@a8mRTM1hZ ziF}iThSI>D&TWI>WqwRAHUZlGaZ|RKl6Ky`dyLSvKDnV#?|$fW>qQ=x$GdZ4aG6F< zZ@xwXaJdF`@TwO;Sq;*soD$|Qz)C#0Q#=)nERU4Mu=BeFy>Baz?*~+I#VQDJ#Sqdx z%sLQJsVK|X2(C?(D2MFklz5MmW9--t`2ai+n20c?$YF{kD(utd2dHG*ME_2oozEID z>z1GQ+jd5OFB&QA>K@XCEI)8gAL}E#`d^J}yM{0rYw`XLV}u*Okg1 z#g%g+U|xm`W6&&CVMTBeiJ>tK3j>C@r8d#wiHVptg z=RF1}oE)X*SuiiEfVON@>pE1iT9|&kO^u)E(*H{Hz@pn2d^p4SF#GQ%kQbY2=2)Cm zviXo2{NchdJDRLoOSyXz%YI6nc_K_y{&J%Vc|qyAo9!#%7s8rAeoQ2bi#x%Q_Gonv zWJe#P0f0KOThtq_q4iZz|A9l5r?eX~<>$`4@tCh0sdovBn)3X?t^Tm+V2-2a~vER;pnh_(1` z7EwEF9O~BQ52wi0jS_|y$FVrZC~(x=FQIT>q&fe)&i|wZTP5CSdU*-6y~U9Yy68~_-I z$5_8s8Faa?ry!FE6cE19eaUg~R3|{Pt2-B<-lM?y{dc#t)TuvbpOb>xzGo|3(~7ur znL*$H{*0kASJsL(n##~|y_#{LBd+b=j=bB$QdA<4XDj`2vK@}|&`sU=!@ad`2?;U? O@DYxmbmO^%=lm0*waNAX literal 0 HcmV?d00001 diff --git a/docs/spelling_wordlist.txt b/docs/spelling_wordlist.txt index eb8f6d7795c27..1378ba1c2f172 100644 --- a/docs/spelling_wordlist.txt +++ b/docs/spelling_wordlist.txt @@ -17,6 +17,7 @@ Aneesh AnnotateTextResponse Ansible AppBuilder +Appflow ArangoDB Arg Args From c70df4fbf438b22353d07264a7ea70f5d8f7adce Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Tue, 31 May 2022 18:38:04 -0300 Subject: [PATCH 05/26] Apply comments/docs patterns. --- .../aws/example_dags/example_appflow.py | 31 ++++----- .../providers/amazon/aws/operators/appflow.py | 63 +++++++++++-------- .../operators/appflow.rst | 28 ++++----- .../amazon/aws/operators/test_appflow.py | 4 +- 4 files changed, 68 insertions(+), 58 deletions(-) diff --git a/airflow/providers/amazon/aws/example_dags/example_appflow.py b/airflow/providers/amazon/aws/example_dags/example_appflow.py index d51ac1ee9b981..90217b8daee98 100644 --- a/airflow/providers/amazon/aws/example_dags/example_appflow.py +++ b/airflow/providers/amazon/aws/example_dags/example_appflow.py @@ -18,9 +18,10 @@ from datetime import datetime from airflow import DAG +from airflow.models.baseoperator import chain from airflow.operators.bash import BashOperator from airflow.providers.amazon.aws.operators.appflow import ( - AppflowRecordsShortCircuit, + AppflowRecordsShortCircuitOperator, AppflowRunAfterOperator, AppflowRunBeforeOperator, AppflowRunDailyOperator, @@ -39,23 +40,23 @@ tags=["example"], ) as dag: - # [START howto_appflow_run] + # [START howto_operator_appflow_run] run = AppflowRunOperator( task_id="campaign-dump", source=SOURCE_NAME, name=FLOW_NAME, ) - # [END howto_appflow_run] + # [END howto_operator_appflow_run] - # [START howto_appflow_run_full] + # [START howto_operator_appflow_run_full] run_full = AppflowRunFullOperator( task_id="campaign-dump-full", source=SOURCE_NAME, name=FLOW_NAME, ) - # [END howto_appflow_run_full] + # [END howto_operator_appflow_run_full] - # [START howto_appflow_run_daily] + # [START howto_operator_appflow_run_daily] run_daily = AppflowRunDailyOperator( task_id="campaign-dump-daily", source=SOURCE_NAME, @@ -63,9 +64,9 @@ source_field="LastModifiedDate", dt="{{ ds }}", ) - # [END howto_appflow_run_daily] + # [END howto_operator_appflow_run_daily] - # [START howto_appflow_run_before] + # [START howto_operator_appflow_run_before] run_before = AppflowRunBeforeOperator( task_id="campaign-dump-before", source=SOURCE_NAME, @@ -73,9 +74,9 @@ source_field="LastModifiedDate", dt="{{ ds }}", ) - # [END howto_appflow_run_before] + # [END howto_operator_appflow_run_before] - # [START howto_appflow_run_after] + # [START howto_operator_appflow_run_after] run_after = AppflowRunAfterOperator( task_id="campaign-dump-after", source=SOURCE_NAME, @@ -83,19 +84,19 @@ source_field="LastModifiedDate", dt="3000-01-01", # Future date, so no records to dump ) - # [END howto_appflow_run_after] + # [END howto_operator_appflow_run_after] - # [START howto_appflow_shortcircuit] - has_records = AppflowRecordsShortCircuit( + # [START howto_operator_appflow_shortcircuit] + has_records = AppflowRecordsShortCircuitOperator( task_id="campaign-dump-short-ciruit", flow_name=FLOW_NAME, appflow_run_task_id="campaign-dump-after", # Should shortcircuit, no records expected ) - # [END howto_appflow_shortcircuit] + # [END howto_operator_appflow_shortcircuit] skipped = BashOperator( task_id="should_be_skipped", bash_command="echo 1", ) - run >> run_full >> run_daily >> run_before >> run_after >> has_records >> skipped + chain(run, run_full, run_daily, run_before, run_after, has_records, skipped) diff --git a/airflow/providers/amazon/aws/operators/appflow.py b/airflow/providers/amazon/aws/operators/appflow.py index 172423b71b314..a8cab7a521575 100644 --- a/airflow/providers/amazon/aws/operators/appflow.py +++ b/airflow/providers/amazon/aws/operators/appflow.py @@ -27,6 +27,7 @@ else: from cached_property import cached_property +from airflow.exceptions import AirflowException from airflow.models import BaseOperator from airflow.operators.python import ShortCircuitOperator from airflow.providers.amazon.aws.hooks.appflow import AppflowHook @@ -46,12 +47,22 @@ SUPPORTED_SOURCES = {"salesforce", "zendesk"} -class AppflowOperatorException(Exception): - """Alias for Exception.""" +class AppflowOperatorBase(BaseOperator): + """ + Amazon Appflow Base Operator class (not supposed to be used directly in DAGs). + .. seealso:: + For more information on how to use this operator, take a look at the guide: + :ref:`howto/operator:AppflowRunOperator` -class AppflowOperatorBase(BaseOperator): - """Amazon Appflow Base Operator class (not supposed to be used directly in DAGs).""" + :param source: The source name (e.g. salesforce) + :param name: The flow name + :param source_field: The field name to apply filters + :param dt: The date value (or template) to be used in filters. + :param poll_interval: how often in seconds to check the query status + :param aws_conn_id: aws connection to use + :param region: aws region to use + """ BLUE = "#2bccbd" ui_color = BLUE @@ -70,9 +81,7 @@ def __init__( ) -> None: super().__init__(**kwargs) if source not in SUPPORTED_SOURCES: - raise AppflowOperatorException( - f"{source} is not a supported source (options: {SUPPORTED_SOURCES})!" - ) + raise AirflowException(f"{source} is not a supported source (options: {SUPPORTED_SOURCES})!") self.dt = dt self._name = name self._source = source @@ -97,7 +106,7 @@ def _get_connector_type(self) -> str: if (self.source == "salesforce" and connector_type != "Salesforce") or ( self.source == "zendesk" and connector_type != "Zendesk" ): - raise AppflowOperatorException( + raise AirflowException( f"Incompatible source ({self.source} and connector type ({connector_type})!" ) return connector_type @@ -229,7 +238,7 @@ def __init__( **kwargs, ) -> None: if source not in {"salesforce", "zendesk"}: - raise AppflowOperatorException(f"Source {source} is not supported for AppflowRunOperator!") + raise AirflowException(f"Source {source} is not supported for AppflowRunOperator!") super().__init__( source=source, name=name, @@ -268,7 +277,7 @@ def __init__( **kwargs, ) -> None: if source not in {"salesforce", "zendesk"}: - raise AppflowOperatorException(f"Source {source} is not supported for AppflowRunFullOperator!") + raise AirflowException(f"Source {source} is not supported for AppflowRunFullOperator!") super().__init__( source=source, name=name, @@ -313,9 +322,9 @@ def __init__( **kwargs, ) -> None: if not dt: - raise AppflowOperatorException("The dt argument is mandatory for AppflowRunBeforeOperator!") + raise AirflowException("The dt argument is mandatory for AppflowRunBeforeOperator!") if source not in {"salesforce"}: - raise AppflowOperatorException(f"Source {source} is not supported for AppflowRunBeforeOperator!") + raise AirflowException(f"Source {source} is not supported for AppflowRunBeforeOperator!") super().__init__( source=source, name=name, @@ -330,9 +339,9 @@ def __init__( def _add_filter(self, tasks: List["TaskTypeDef"]) -> None: if not self._dt_parsed: - raise AppflowOperatorException(f"Invalid dt argument parser value: {self._dt_parsed}") + raise AirflowException(f"Invalid dt argument parser value: {self._dt_parsed}") if not self.source_field: - raise AppflowOperatorException(f"Invalid source_field argument value: {self.source_field}") + raise AirflowException(f"Invalid source_field argument value: {self.source_field}") filter_task: "TaskTypeDef" = { "taskType": "Filter", "connectorOperator": {self._connector_type: "LESS_THAN"}, # type: ignore @@ -376,9 +385,9 @@ def __init__( **kwargs, ) -> None: if not dt: - raise AppflowOperatorException("The dt argument is mandatory for AppflowRunAfterOperator!") + raise AirflowException("The dt argument is mandatory for AppflowRunAfterOperator!") if source not in {"salesforce", "zendesk"}: - raise AppflowOperatorException(f"Source {source} is not supported for AppflowRunAfterOperator!") + raise AirflowException(f"Source {source} is not supported for AppflowRunAfterOperator!") super().__init__( source=source, name=name, @@ -393,9 +402,9 @@ def __init__( def _add_filter(self, tasks: List["TaskTypeDef"]) -> None: if not self._dt_parsed: - raise AppflowOperatorException(f"Invalid dt argument parser value: {self._dt_parsed}") + raise AirflowException(f"Invalid dt argument parser value: {self._dt_parsed}") if not self.source_field: - raise AppflowOperatorException(f"Invalid source_field argument value: {self.source_field}") + raise AirflowException(f"Invalid source_field argument value: {self.source_field}") filter_task: "TaskTypeDef" = { "taskType": "Filter", "connectorOperator": {self._connector_type: "GREATER_THAN"}, # type: ignore @@ -439,9 +448,9 @@ def __init__( **kwargs, ) -> None: if not dt: - raise AppflowOperatorException("The dt argument is mandatory for AppflowRunDailyOperator!") + raise AirflowException("The dt argument is mandatory for AppflowRunDailyOperator!") if source not in {"salesforce"}: - raise AppflowOperatorException(f"Source {source} is not supported for AppflowRunDailyOperator!") + raise AirflowException(f"Source {source} is not supported for AppflowRunDailyOperator!") super().__init__( source=source, name=name, @@ -456,9 +465,9 @@ def __init__( def _add_filter(self, tasks: List["TaskTypeDef"]) -> None: if not self._dt_parsed: - raise AppflowOperatorException(f"Invalid dt argument parser value: {self._dt_parsed}") + raise AirflowException(f"Invalid dt argument parser value: {self._dt_parsed}") if not self.source_field: - raise AppflowOperatorException(f"Invalid source_field argument value: {self.source_field}") + raise AirflowException(f"Invalid source_field argument value: {self.source_field}") start_dt = self._dt_parsed - timedelta(milliseconds=1) end_dt = self._dt_parsed + timedelta(days=1) filter_task: "TaskTypeDef" = { @@ -474,13 +483,13 @@ def _add_filter(self, tasks: List["TaskTypeDef"]) -> None: tasks.append(filter_task) -class AppflowRecordsShortCircuit(ShortCircuitOperator): +class AppflowRecordsShortCircuitOperator(ShortCircuitOperator): """ Short-circuit in case of a empty Appflow's run. .. seealso:: For more information on how to use this operator, take a look at the guide: - :ref:`howto/operator:AppflowRecordsShortCircuit` + :ref:`howto/operator:AppflowRecordsShortCircuitOperator` :param flow_name: The flow name :param appflow_run_task_id: Run task ID from where this operator should extract the execution ID @@ -547,15 +556,15 @@ def _has_new_records_func(self, **kwargs) -> bool: response = af_client.describe_flow_execution_records(nextToken=response["nextToken"], **args) else: response = af_client.describe_flow_execution_records(**args) - record = AppflowRecordsShortCircuit._get_target_execution_id( + record = AppflowRecordsShortCircuitOperator._get_target_execution_id( response["flowExecutions"], execution_id ) if not record and "nextToken" not in response: - raise AppflowOperatorException(f"Flow ({execution_id}) without recordsProcessed info.") + raise AirflowException(f"Flow ({execution_id}) without recordsProcessed info.") execution = record.get("executionResult", {}) if "recordsProcessed" not in execution: - raise AppflowOperatorException(f"Flow ({execution_id}) without recordsProcessed info.") + raise AirflowException(f"Flow ({execution_id}) without recordsProcessed info.") records_processed = execution["recordsProcessed"] self.log.info("records_processed: %d", records_processed) task_instance.xcom_push("records_processed", records_processed) # type: ignore diff --git a/docs/apache-airflow-providers-amazon/operators/appflow.rst b/docs/apache-airflow-providers-amazon/operators/appflow.rst index 7c65380b9dba8..daf3d873dcf5f 100644 --- a/docs/apache-airflow-providers-amazon/operators/appflow.rst +++ b/docs/apache-airflow-providers-amazon/operators/appflow.rst @@ -47,8 +47,8 @@ To run an AppFlow flow keeping all filters as is, use: .. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_appflow.py :language: python :dedent: 4 - :start-after: [START howto_appflow_run] - :end-before: [END howto_appflow_run] + :start-after: [START howto_operator_appflow_run] + :end-before: [END howto_operator_appflow_run] .. note:: Supported sources: Salesforce, Zendesk @@ -64,8 +64,8 @@ To run an AppFlow flow removing all filters, use: .. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_appflow.py :language: python :dedent: 4 - :start-after: [START howto_appflow_run_full] - :end-before: [END howto_appflow_run_full] + :start-after: [START howto_operator_appflow_run_full] + :end-before: [END howto_operator_appflow_run_full] .. note:: Supported sources: Salesforce, Zendesk @@ -81,8 +81,8 @@ To run an AppFlow flow filtering daily records, use: .. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_appflow.py :language: python :dedent: 4 - :start-after: [START howto_appflow_run_daily] - :end-before: [END howto_appflow_run_daily] + :start-after: [START howto_operator_appflow_run_daily] + :end-before: [END howto_operator_appflow_run_daily] .. note:: Supported sources: Salesforce @@ -98,8 +98,8 @@ To run an AppFlow flow filtering future records and selecting the past ones, use .. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_appflow.py :language: python :dedent: 4 - :start-after: [START howto_appflow_run_before] - :end-before: [END howto_appflow_run_before] + :start-after: [START howto_operator_appflow_run_before] + :end-before: [END howto_operator_appflow_run_before] .. note:: Supported sources: Salesforce @@ -115,8 +115,8 @@ To run an AppFlow flow filtering past records and selecting the future ones, use .. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_appflow.py :language: python :dedent: 4 - :start-after: [START howto_appflow_run_after] - :end-before: [END howto_appflow_run_after] + :start-after: [START howto_operator_appflow_run_after] + :end-before: [END howto_operator_appflow_run_after] .. note:: Supported sources: Salesforce, Zendesk @@ -127,18 +127,18 @@ Skipping Tasks For Empty Runs ============================= To skip tasks when some AppFlow run return zero records, use: -:class:`~airflow.providers.amazon.aws.operators.appflow.AppflowRecordsShortCircuit`. +:class:`~airflow.providers.amazon.aws.operators.appflow.AppflowRecordsShortCircuitOperator`. .. exampleinclude:: /../../airflow/providers/amazon/aws/example_dags/example_appflow.py :language: python :dedent: 4 - :start-after: [START howto_appflow_shortcircuit] - :end-before: [END howto_appflow_shortcircuit] + :start-after: [START howto_operator_appflow_shortcircuit] + :end-before: [END howto_operator_appflow_shortcircuit] .. note:: Supported sources: Salesforce, Zendesk -.. _howto/operator:AppflowRecordsShortCircuit: +.. _howto/operator:AppflowRecordsShortCircuitOperator: Reference --------- diff --git a/tests/providers/amazon/aws/operators/test_appflow.py b/tests/providers/amazon/aws/operators/test_appflow.py index 0345db41cf0c7..6e9ca002ec15c 100644 --- a/tests/providers/amazon/aws/operators/test_appflow.py +++ b/tests/providers/amazon/aws/operators/test_appflow.py @@ -22,7 +22,7 @@ import pytest from airflow.providers.amazon.aws.operators.appflow import ( - AppflowRecordsShortCircuit, + AppflowRecordsShortCircuitOperator, AppflowRunAfterOperator, AppflowRunBeforeOperator, AppflowRunDailyOperator, @@ -158,7 +158,7 @@ def test_run_daily(appflow_conn, ctx): def test_short_circuit(appflow_conn, dag_maker): with dag_maker(dag_id="unit_test_short_circuit"): AppflowRunFullOperator(**DUMP_COMMON_ARGS) - AppflowRecordsShortCircuit( + AppflowRecordsShortCircuitOperator( task_id=SHORT_CIRCUIT_TASK_ID, flow_name=FLOW_NAME, appflow_run_task_id=TASK_ID, From b83b0e2c1b822bde618ab325c90b65bf67f88f60 Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Tue, 31 May 2022 19:18:43 -0300 Subject: [PATCH 06/26] Removing the "private" attribute signal and more. --- .../aws/example_dags/example_appflow.py | 32 +++-- .../providers/amazon/aws/operators/appflow.py | 133 +++++++----------- 2 files changed, 73 insertions(+), 92 deletions(-) diff --git a/airflow/providers/amazon/aws/example_dags/example_appflow.py b/airflow/providers/amazon/aws/example_dags/example_appflow.py index 90217b8daee98..9be8011853132 100644 --- a/airflow/providers/amazon/aws/example_dags/example_appflow.py +++ b/airflow/providers/amazon/aws/example_dags/example_appflow.py @@ -41,7 +41,7 @@ ) as dag: # [START howto_operator_appflow_run] - run = AppflowRunOperator( + campaign_dump = AppflowRunOperator( task_id="campaign-dump", source=SOURCE_NAME, name=FLOW_NAME, @@ -49,16 +49,16 @@ # [END howto_operator_appflow_run] # [START howto_operator_appflow_run_full] - run_full = AppflowRunFullOperator( - task_id="campaign-dump-full", + campaign_dump_full = AppflowRunFullOperator( + task_id="campaign_dump_full", source=SOURCE_NAME, name=FLOW_NAME, ) # [END howto_operator_appflow_run_full] # [START howto_operator_appflow_run_daily] - run_daily = AppflowRunDailyOperator( - task_id="campaign-dump-daily", + campaign_dump_daily = AppflowRunDailyOperator( + task_id="campaign_dump_daily", source=SOURCE_NAME, name=FLOW_NAME, source_field="LastModifiedDate", @@ -67,8 +67,8 @@ # [END howto_operator_appflow_run_daily] # [START howto_operator_appflow_run_before] - run_before = AppflowRunBeforeOperator( - task_id="campaign-dump-before", + campaign_dump_before = AppflowRunBeforeOperator( + task_id="campaign_dump_before", source=SOURCE_NAME, name=FLOW_NAME, source_field="LastModifiedDate", @@ -77,8 +77,8 @@ # [END howto_operator_appflow_run_before] # [START howto_operator_appflow_run_after] - run_after = AppflowRunAfterOperator( - task_id="campaign-dump-after", + campaign_dump_after = AppflowRunAfterOperator( + task_id="campaign_dump_after", source=SOURCE_NAME, name=FLOW_NAME, source_field="LastModifiedDate", @@ -87,16 +87,24 @@ # [END howto_operator_appflow_run_after] # [START howto_operator_appflow_shortcircuit] - has_records = AppflowRecordsShortCircuitOperator( + campaign_dump_short_ciruit = AppflowRecordsShortCircuitOperator( task_id="campaign-dump-short-ciruit", flow_name=FLOW_NAME, appflow_run_task_id="campaign-dump-after", # Should shortcircuit, no records expected ) # [END howto_operator_appflow_shortcircuit] - skipped = BashOperator( + should_be_skipped = BashOperator( task_id="should_be_skipped", bash_command="echo 1", ) - chain(run, run_full, run_daily, run_before, run_after, has_records, skipped) + chain( + campaign_dump, + campaign_dump_full, + campaign_dump_daily, + campaign_dump_before, + campaign_dump_after, + campaign_dump_short_ciruit, + should_be_skipped, + ) diff --git a/airflow/providers/amazon/aws/operators/appflow.py b/airflow/providers/amazon/aws/operators/appflow.py index a8cab7a521575..fbbbf8444caa8 100644 --- a/airflow/providers/amazon/aws/operators/appflow.py +++ b/airflow/providers/amazon/aws/operators/appflow.py @@ -33,9 +33,9 @@ from airflow.providers.amazon.aws.hooks.appflow import AppflowHook if TYPE_CHECKING: - from mypy_boto3_appflow.client import AppflowClient from mypy_boto3_appflow.type_defs import ( DescribeFlowExecutionRecordsResponseTypeDef, + DescribeFlowResponseTypeDef, ExecutionRecordTypeDef, TaskTypeDef, ) @@ -83,83 +83,80 @@ def __init__( if source not in SUPPORTED_SOURCES: raise AirflowException(f"{source} is not a supported source (options: {SUPPORTED_SOURCES})!") self.dt = dt - self._name = name - self._source = source - self._source_field = source_field - self._poll_interval = poll_interval - self._aws_conn_id = aws_conn_id - self._region = region - self._flow_update = flow_update + self.name = name + self.source = source + self.source_field = source_field + self.poll_interval = poll_interval + self.aws_conn_id = aws_conn_id + self.region = region + self.flow_update = flow_update @cached_property def hook(self) -> AppflowHook: """Create and return an AppflowHook.""" - return AppflowHook(aws_conn_id=self.aws_conn_id, region_name=self._region) + return AppflowHook(aws_conn_id=self.aws_conn_id, region_name=self.region) @staticmethod def _dt_to_epoch_str(dt: datetime) -> str: text = str(int(dt.timestamp() * 1000)) return text - def _get_connector_type(self) -> str: - connector_type = self._response["sourceFlowConfig"]["connectorType"] - if (self.source == "salesforce" and connector_type != "Salesforce") or ( - self.source == "zendesk" and connector_type != "Zendesk" - ): + def execute(self, context: "Context") -> None: + self.dt_parsed: Optional[datetime] = datetime.fromisoformat(self.dt) if self.dt else None + if self.flow_update: + self._update_flow() + self._run_flow(context) + + def _get_connector_type(self, response: "DescribeFlowResponseTypeDef") -> str: + connector_type = response["sourceFlowConfig"]["connectorType"] + if self.source != connector_type.lower(): raise AirflowException( f"Incompatible source ({self.source} and connector type ({connector_type})!" ) return connector_type - def execute(self, context: "Context") -> None: - self._af_client: "AppflowClient" = self.hook.conn - self._dt_parsed: Optional[datetime] = datetime.fromisoformat(self.dt) if self.dt else None - if self._flow_update: - self._update_flow() - self._run_flow(context) - def _update_flow(self) -> None: - self._response = self._af_client.describe_flow(flowName=self.name) - self._connector_type = self._get_connector_type() + response = self.hook.conn.describe_flow(flowName=self.name) + connector_type = self._get_connector_type(response) # cleanup tasks: List["TaskTypeDef"] = [] - for task in self._response["tasks"]: + for task in response["tasks"]: if ( task["taskType"] == "Filter" - and task.get("connectorOperator", {}).get(self._connector_type) != "PROJECTION" + and task.get("connectorOperator", {}).get(connector_type) != "PROJECTION" ): self.log.info("Removing task: %s", task) else: tasks.append(task) # List of non-filter tasks - self._add_filter(tasks) + self._add_filter(connector_type, tasks) # Clean up to force on-demand trigger - trigger_config = copy.deepcopy(self._response["triggerConfig"]) + trigger_config = copy.deepcopy(response["triggerConfig"]) del trigger_config["triggerProperties"] - self._af_client.update_flow( - flowName=self._response["flowName"], - destinationFlowConfigList=self._response["destinationFlowConfigList"], - sourceFlowConfig=self._response["sourceFlowConfig"], + self.hook.conn.update_flow( + flowName=response["flowName"], + destinationFlowConfigList=response["destinationFlowConfigList"], + sourceFlowConfig=response["sourceFlowConfig"], triggerConfig=trigger_config, - description=self._response.get("description", "Flow description."), + description=response.get("description", "Flow description."), tasks=tasks, ) - def _add_filter(self, tasks: List["TaskTypeDef"]) -> None: # Interface + def _add_filter(self, connector_type: str, tasks: List["TaskTypeDef"]) -> None: # Interface pass def _run_flow(self, context) -> str: ts_before: datetime = datetime.now(timezone.utc) sleep(EVENTUAL_CONSISTENCY_OFFSET) - response = self._af_client.start_flow(flowName=self.name) + response = self.hook.conn.start_flow(flowName=self.name) task_instance = context["task_instance"] task_instance.xcom_push("execution_id", response["executionId"]) self.log.info("executionId: %s", response["executionId"]) - response = self._af_client.describe_flow(flowName=self.name) + response = self.hook.conn.describe_flow(flowName=self.name) # Wait Appflow eventual consistence self.log.info("Waiting Appflow eventual consistence...") @@ -170,7 +167,7 @@ def _run_flow(self, context) -> str: < ts_before ): sleep(EVENTUAL_CONSISTENCY_POLLING) - response = self._af_client.describe_flow(flowName=self.name) + response = self.hook.conn.describe_flow(flowName=self.name) # Wait flow stops self.log.info("Waiting flow run...") @@ -179,7 +176,7 @@ def _run_flow(self, context) -> str: or response["lastRunExecutionDetails"]["mostRecentExecutionStatus"] == "InProgress" ): sleep(self.poll_interval) - response = self._af_client.describe_flow(flowName=self.name) + response = self.hook.conn.describe_flow(flowName=self.name) self.log.info("lastRunExecutionDetails: %s", response["lastRunExecutionDetails"]) @@ -188,30 +185,6 @@ def _run_flow(self, context) -> str: return response["lastRunExecutionDetails"]["mostRecentExecutionStatus"] - @property - def name(self): - return self._name - - @property - def source(self): - return self._source - - @property - def source_field(self): - return self._source_field - - @property - def aws_conn_id(self): - return self._aws_conn_id - - @property - def region(self): - return self._region - - @property - def poll_interval(self): - return self._poll_interval - class AppflowRunOperator(AppflowOperatorBase): """ @@ -337,18 +310,18 @@ def __init__( **kwargs, ) - def _add_filter(self, tasks: List["TaskTypeDef"]) -> None: - if not self._dt_parsed: - raise AirflowException(f"Invalid dt argument parser value: {self._dt_parsed}") + def _add_filter(self, connector_type: str, tasks: List["TaskTypeDef"]) -> None: + if not self.dt_parsed: + raise AirflowException(f"Invalid dt argument parser value: {self.dt_parsed}") if not self.source_field: raise AirflowException(f"Invalid source_field argument value: {self.source_field}") filter_task: "TaskTypeDef" = { "taskType": "Filter", - "connectorOperator": {self._connector_type: "LESS_THAN"}, # type: ignore + "connectorOperator": {connector_type: "LESS_THAN"}, # type: ignore "sourceFields": [self.source_field], "taskProperties": { "DATA_TYPE": "datetime", - "VALUE": AppflowOperatorBase._dt_to_epoch_str(self._dt_parsed), + "VALUE": AppflowOperatorBase._dt_to_epoch_str(self.dt_parsed), }, # NOT inclusive } tasks.append(filter_task) @@ -400,18 +373,18 @@ def __init__( **kwargs, ) - def _add_filter(self, tasks: List["TaskTypeDef"]) -> None: - if not self._dt_parsed: - raise AirflowException(f"Invalid dt argument parser value: {self._dt_parsed}") + def _add_filter(self, connector_type: str, tasks: List["TaskTypeDef"]) -> None: + if not self.dt_parsed: + raise AirflowException(f"Invalid dt argument parser value: {self.dt_parsed}") if not self.source_field: raise AirflowException(f"Invalid source_field argument value: {self.source_field}") filter_task: "TaskTypeDef" = { "taskType": "Filter", - "connectorOperator": {self._connector_type: "GREATER_THAN"}, # type: ignore + "connectorOperator": {connector_type: "GREATER_THAN"}, # type: ignore "sourceFields": [self.source_field], "taskProperties": { "DATA_TYPE": "datetime", - "VALUE": AppflowOperatorBase._dt_to_epoch_str(self._dt_parsed), + "VALUE": AppflowOperatorBase._dt_to_epoch_str(self.dt_parsed), }, # NOT inclusive } tasks.append(filter_task) @@ -463,16 +436,16 @@ def __init__( **kwargs, ) - def _add_filter(self, tasks: List["TaskTypeDef"]) -> None: - if not self._dt_parsed: - raise AirflowException(f"Invalid dt argument parser value: {self._dt_parsed}") + def _add_filter(self, connector_type: str, tasks: List["TaskTypeDef"]) -> None: + if not self.dt_parsed: + raise AirflowException(f"Invalid dt argument parser value: {self.dt_parsed}") if not self.source_field: raise AirflowException(f"Invalid source_field argument value: {self.source_field}") - start_dt = self._dt_parsed - timedelta(milliseconds=1) - end_dt = self._dt_parsed + timedelta(days=1) + start_dt = self.dt_parsed - timedelta(milliseconds=1) + end_dt = self.dt_parsed + timedelta(days=1) filter_task: "TaskTypeDef" = { "taskType": "Filter", - "connectorOperator": {self._connector_type: "BETWEEN"}, # type: ignore + "connectorOperator": {connector_type: "BETWEEN"}, # type: ignore "sourceFields": [self.source_field], "taskProperties": { "DATA_TYPE": "datetime", @@ -519,8 +492,8 @@ def __init__( ignore_downstream_trigger_rules=ignore_downstream_trigger_rules, **kwargs, ) - self._aws_conn_id = aws_conn_id - self._region = region + self.aws_conn_id = aws_conn_id + self.region = region @staticmethod def _get_target_execution_id( @@ -534,7 +507,7 @@ def _get_target_execution_id( @cached_property def hook(self) -> AppflowHook: """Create and return an AppflowHook.""" - return AppflowHook(aws_conn_id=self._aws_conn_id, region_name=self._region) + return AppflowHook(aws_conn_id=self.aws_conn_id, region_name=self.region) def _has_new_records_func(self, **kwargs) -> bool: appflow_task_id = kwargs["appflow_run_task_id"] From aaeeaaa072bea04b198dbecf5f9d88e300d6dcef Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Wed, 1 Jun 2022 08:12:10 -0300 Subject: [PATCH 07/26] Fix task_ids for example_appflow. --- .../providers/amazon/aws/example_dags/example_appflow.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/airflow/providers/amazon/aws/example_dags/example_appflow.py b/airflow/providers/amazon/aws/example_dags/example_appflow.py index 9be8011853132..49d871ff314a2 100644 --- a/airflow/providers/amazon/aws/example_dags/example_appflow.py +++ b/airflow/providers/amazon/aws/example_dags/example_appflow.py @@ -42,7 +42,7 @@ # [START howto_operator_appflow_run] campaign_dump = AppflowRunOperator( - task_id="campaign-dump", + task_id="campaign_dump", source=SOURCE_NAME, name=FLOW_NAME, ) @@ -88,9 +88,9 @@ # [START howto_operator_appflow_shortcircuit] campaign_dump_short_ciruit = AppflowRecordsShortCircuitOperator( - task_id="campaign-dump-short-ciruit", + task_id="campaign_dump_short_ciruit", flow_name=FLOW_NAME, - appflow_run_task_id="campaign-dump-after", # Should shortcircuit, no records expected + appflow_run_task_id="campaign_dump_after", # Should shortcircuit, no records expected ) # [END howto_operator_appflow_shortcircuit] From 6a2314eaedc7f39653e70d5c2a29afe32ef11fdd Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Wed, 1 Jun 2022 18:33:34 -0300 Subject: [PATCH 08/26] Move datetime_to_epoch() to utils and more. --- .../aws/example_dags/example_appflow.py | 6 +++--- .../providers/amazon/aws/operators/appflow.py | 18 +++++------------- .../providers/amazon/aws/utils/__init__.py | 17 +++++++++++++++++ docs/spelling_wordlist.txt | 1 + tests/providers/amazon/aws/utils/__init__.py | 19 +++++++++++++++++++ 5 files changed, 45 insertions(+), 16 deletions(-) diff --git a/airflow/providers/amazon/aws/example_dags/example_appflow.py b/airflow/providers/amazon/aws/example_dags/example_appflow.py index 49d871ff314a2..d11ce6fef28b4 100644 --- a/airflow/providers/amazon/aws/example_dags/example_appflow.py +++ b/airflow/providers/amazon/aws/example_dags/example_appflow.py @@ -87,8 +87,8 @@ # [END howto_operator_appflow_run_after] # [START howto_operator_appflow_shortcircuit] - campaign_dump_short_ciruit = AppflowRecordsShortCircuitOperator( - task_id="campaign_dump_short_ciruit", + campaign_dump_short_circuit = AppflowRecordsShortCircuitOperator( + task_id="campaign_dump_short_circuit", flow_name=FLOW_NAME, appflow_run_task_id="campaign_dump_after", # Should shortcircuit, no records expected ) @@ -105,6 +105,6 @@ campaign_dump_daily, campaign_dump_before, campaign_dump_after, - campaign_dump_short_ciruit, + campaign_dump_short_circuit, should_be_skipped, ) diff --git a/airflow/providers/amazon/aws/operators/appflow.py b/airflow/providers/amazon/aws/operators/appflow.py index fbbbf8444caa8..94b57cd4520b1 100644 --- a/airflow/providers/amazon/aws/operators/appflow.py +++ b/airflow/providers/amazon/aws/operators/appflow.py @@ -31,6 +31,7 @@ from airflow.models import BaseOperator from airflow.operators.python import ShortCircuitOperator from airflow.providers.amazon.aws.hooks.appflow import AppflowHook +from airflow.providers.amazon.aws.utils import datetime_to_epoch_ms if TYPE_CHECKING: from mypy_boto3_appflow.type_defs import ( @@ -51,10 +52,6 @@ class AppflowOperatorBase(BaseOperator): """ Amazon Appflow Base Operator class (not supposed to be used directly in DAGs). - .. seealso:: - For more information on how to use this operator, take a look at the guide: - :ref:`howto/operator:AppflowRunOperator` - :param source: The source name (e.g. salesforce) :param name: The flow name :param source_field: The field name to apply filters @@ -96,11 +93,6 @@ def hook(self) -> AppflowHook: """Create and return an AppflowHook.""" return AppflowHook(aws_conn_id=self.aws_conn_id, region_name=self.region) - @staticmethod - def _dt_to_epoch_str(dt: datetime) -> str: - text = str(int(dt.timestamp() * 1000)) - return text - def execute(self, context: "Context") -> None: self.dt_parsed: Optional[datetime] = datetime.fromisoformat(self.dt) if self.dt else None if self.flow_update: @@ -321,7 +313,7 @@ def _add_filter(self, connector_type: str, tasks: List["TaskTypeDef"]) -> None: "sourceFields": [self.source_field], "taskProperties": { "DATA_TYPE": "datetime", - "VALUE": AppflowOperatorBase._dt_to_epoch_str(self.dt_parsed), + "VALUE": str(datetime_to_epoch_ms(self.dt_parsed)), }, # NOT inclusive } tasks.append(filter_task) @@ -384,7 +376,7 @@ def _add_filter(self, connector_type: str, tasks: List["TaskTypeDef"]) -> None: "sourceFields": [self.source_field], "taskProperties": { "DATA_TYPE": "datetime", - "VALUE": AppflowOperatorBase._dt_to_epoch_str(self.dt_parsed), + "VALUE": str(datetime_to_epoch_ms(self.dt_parsed)), }, # NOT inclusive } tasks.append(filter_task) @@ -449,8 +441,8 @@ def _add_filter(self, connector_type: str, tasks: List["TaskTypeDef"]) -> None: "sourceFields": [self.source_field], "taskProperties": { "DATA_TYPE": "datetime", - "LOWER_BOUND": AppflowOperatorBase._dt_to_epoch_str(start_dt), # NOT inclusive - "UPPER_BOUND": AppflowOperatorBase._dt_to_epoch_str(end_dt), # NOT inclusive + "LOWER_BOUND": str(datetime_to_epoch_ms(start_dt)), # NOT inclusive + "UPPER_BOUND": str(datetime_to_epoch_ms(end_dt)), # NOT inclusive }, } tasks.append(filter_task) diff --git a/airflow/providers/amazon/aws/utils/__init__.py b/airflow/providers/amazon/aws/utils/__init__.py index 13a83393a9124..d9acff6bc484d 100644 --- a/airflow/providers/amazon/aws/utils/__init__.py +++ b/airflow/providers/amazon/aws/utils/__init__.py @@ -14,3 +14,20 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. + +from datetime import datetime + + +def datetime_to_epoch(dt: datetime) -> int: + """Convert a datetime object to an epoch integer (seconds).""" + return int(dt.timestamp()) + + +def datetime_to_epoch_ms(dt: datetime) -> int: + """Convert a datetime object to an epoch integer (milliseconds).""" + return int(dt.timestamp() * 1_000) + + +def datetime_to_epoch_us(dt: datetime) -> int: + """Convert a datetime object to an epoch integer (microseconds).""" + return int(dt.timestamp() * 1_000_000) diff --git a/docs/spelling_wordlist.txt b/docs/spelling_wordlist.txt index 1378ba1c2f172..43c1d0dc107b5 100644 --- a/docs/spelling_wordlist.txt +++ b/docs/spelling_wordlist.txt @@ -758,6 +758,7 @@ dropdown druidHook ds dsn +dt dttm dtypes durations diff --git a/tests/providers/amazon/aws/utils/__init__.py b/tests/providers/amazon/aws/utils/__init__.py index 13a83393a9124..dafaadd0bde40 100644 --- a/tests/providers/amazon/aws/utils/__init__.py +++ b/tests/providers/amazon/aws/utils/__init__.py @@ -14,3 +14,22 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. + +from datetime import datetime + +from airflow.providers.amazon.aws.utils import datetime_to_epoch, datetime_to_epoch_ms, datetime_to_epoch_us + +DT = datetime(2000, 1, 1) +EPOCH = 946_684_800 + + +def test_datetime_to_epoch(): + assert datetime_to_epoch(DT) == EPOCH + + +def test_datetime_to_epoch_ms(): + assert datetime_to_epoch_ms(DT) == EPOCH * 1000 + + +def test_datetime_to_epoch_us(): + assert datetime_to_epoch_us(DT) == EPOCH * 1_000_000 From 35efeb944f387abe27a7015feb1a3a814a657415 Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Wed, 1 Jun 2022 20:14:08 -0300 Subject: [PATCH 09/26] Fix the AppflowBaseOperator name. --- airflow/providers/amazon/aws/operators/appflow.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/airflow/providers/amazon/aws/operators/appflow.py b/airflow/providers/amazon/aws/operators/appflow.py index 94b57cd4520b1..869f37494e8c6 100644 --- a/airflow/providers/amazon/aws/operators/appflow.py +++ b/airflow/providers/amazon/aws/operators/appflow.py @@ -48,7 +48,7 @@ SUPPORTED_SOURCES = {"salesforce", "zendesk"} -class AppflowOperatorBase(BaseOperator): +class AppflowBaseOperator(BaseOperator): """ Amazon Appflow Base Operator class (not supposed to be used directly in DAGs). @@ -178,7 +178,7 @@ def _run_flow(self, context) -> str: return response["lastRunExecutionDetails"]["mostRecentExecutionStatus"] -class AppflowRunOperator(AppflowOperatorBase): +class AppflowRunOperator(AppflowBaseOperator): """ Execute a Appflow run with filters as is. @@ -217,7 +217,7 @@ def __init__( ) -class AppflowRunFullOperator(AppflowOperatorBase): +class AppflowRunFullOperator(AppflowBaseOperator): """ Execute a Appflow full run removing any filter. @@ -256,7 +256,7 @@ def __init__( ) -class AppflowRunBeforeOperator(AppflowOperatorBase): +class AppflowRunBeforeOperator(AppflowBaseOperator): """ Execute a Appflow run after updating the filters to select only previous data. @@ -319,7 +319,7 @@ def _add_filter(self, connector_type: str, tasks: List["TaskTypeDef"]) -> None: tasks.append(filter_task) -class AppflowRunAfterOperator(AppflowOperatorBase): +class AppflowRunAfterOperator(AppflowBaseOperator): """ Execute a Appflow run after updating the filters to select only future data. @@ -382,7 +382,7 @@ def _add_filter(self, connector_type: str, tasks: List["TaskTypeDef"]) -> None: tasks.append(filter_task) -class AppflowRunDailyOperator(AppflowOperatorBase): +class AppflowRunDailyOperator(AppflowBaseOperator): """ Execute a Appflow run after updating the filters to select only a single day. @@ -529,7 +529,7 @@ def _has_new_records_func(self, **kwargs) -> bool: execution = record.get("executionResult", {}) if "recordsProcessed" not in execution: - raise AirflowException(f"Flow ({execution_id}) without recordsProcessed info.") + raise AirflowException(f"Flow ({execution_id}) without recordsProcessed info!") records_processed = execution["recordsProcessed"] self.log.info("records_processed: %d", records_processed) task_instance.xcom_push("records_processed", records_processed) # type: ignore From 2b545965737f74bd50db67543aebcc7c7cbb4740 Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Wed, 1 Jun 2022 23:04:15 -0300 Subject: [PATCH 10/26] Ignore AppflowBaseOperator during structure check. --- airflow/providers/amazon/aws/operators/appflow.py | 2 ++ tests/always/test_project_structure.py | 1 + 2 files changed, 3 insertions(+) diff --git a/airflow/providers/amazon/aws/operators/appflow.py b/airflow/providers/amazon/aws/operators/appflow.py index 869f37494e8c6..9d8c56113e53b 100644 --- a/airflow/providers/amazon/aws/operators/appflow.py +++ b/airflow/providers/amazon/aws/operators/appflow.py @@ -509,6 +509,8 @@ def _has_new_records_func(self, **kwargs) -> bool: af_client = self.hook.conn task_instance = kwargs["task_instance"] execution_id = task_instance.xcom_pull(task_ids=appflow_task_id, key="execution_id") # type: ignore + if not execution_id: + raise AirflowException(f"No execution_id found from task_id {appflow_task_id}!") self.log.info("execution_id: %s", execution_id) args = {"flowName": flow_name, "maxResults": 100} response: "DescribeFlowExecutionRecordsResponseTypeDef" = cast( diff --git a/tests/always/test_project_structure.py b/tests/always/test_project_structure.py index 1c1bedf7fcf3b..262d0877f0ad1 100644 --- a/tests/always/test_project_structure.py +++ b/tests/always/test_project_structure.py @@ -430,6 +430,7 @@ class TestAmazonProviderProjectStructure(ExampleCoverageTest): 'airflow.providers.amazon.aws.sensors.emr.EmrBaseSensor', 'airflow.providers.amazon.aws.sensors.rds.RdsBaseSensor', 'airflow.providers.amazon.aws.sensors.sagemaker.SageMakerBaseSensor', + 'airflow.providers.amazon.aws.operators.appflow.AppflowBaseOperator', } MISSING_EXAMPLES_FOR_CLASSES = { From 7cfed9b0252dca9f0edcfb49f68257a1d18c4846 Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Thu, 2 Jun 2022 12:43:07 -0300 Subject: [PATCH 11/26] test_short_circuit refactor. --- .../aws/example_dags/example_appflow.py | 1 + .../providers/amazon/aws/operators/appflow.py | 1 + .../amazon/aws/operators/test_appflow.py | 29 +++++++++---------- 3 files changed, 16 insertions(+), 15 deletions(-) diff --git a/airflow/providers/amazon/aws/example_dags/example_appflow.py b/airflow/providers/amazon/aws/example_dags/example_appflow.py index d11ce6fef28b4..3cd6778478948 100644 --- a/airflow/providers/amazon/aws/example_dags/example_appflow.py +++ b/airflow/providers/amazon/aws/example_dags/example_appflow.py @@ -91,6 +91,7 @@ task_id="campaign_dump_short_circuit", flow_name=FLOW_NAME, appflow_run_task_id="campaign_dump_after", # Should shortcircuit, no records expected + ignore_downstream_trigger_rules=True, ) # [END howto_operator_appflow_shortcircuit] diff --git a/airflow/providers/amazon/aws/operators/appflow.py b/airflow/providers/amazon/aws/operators/appflow.py index 9d8c56113e53b..0e9916e6f0237 100644 --- a/airflow/providers/amazon/aws/operators/appflow.py +++ b/airflow/providers/amazon/aws/operators/appflow.py @@ -468,6 +468,7 @@ class AppflowRecordsShortCircuitOperator(ShortCircuitOperator): def __init__( self, + *, flow_name: str, appflow_run_task_id: str, ignore_downstream_trigger_rules: bool = True, diff --git a/tests/providers/amazon/aws/operators/test_appflow.py b/tests/providers/amazon/aws/operators/test_appflow.py index 6e9ca002ec15c..71a7429fae6e8 100644 --- a/tests/providers/amazon/aws/operators/test_appflow.py +++ b/tests/providers/amazon/aws/operators/test_appflow.py @@ -155,18 +155,17 @@ def test_run_daily(appflow_conn, ctx): ) -def test_short_circuit(appflow_conn, dag_maker): - with dag_maker(dag_id="unit_test_short_circuit"): - AppflowRunFullOperator(**DUMP_COMMON_ARGS) - AppflowRecordsShortCircuitOperator( - task_id=SHORT_CIRCUIT_TASK_ID, - flow_name=FLOW_NAME, - appflow_run_task_id=TASK_ID, - ) - - dagrun = dag_maker.create_dagrun(execution_date=timezone.utcnow()) - tis = {ti.task_id: ti for ti in dagrun.task_instances} - for _, ti in tis.items(): - ti.run() - appflow_conn.describe_flow_execution_records.assert_called_once_with(flowName=FLOW_NAME, maxResults=100) - assert tis[SHORT_CIRCUIT_TASK_ID].xcom_pull(task_ids=SHORT_CIRCUIT_TASK_ID, key='return_value') == 1 +def test_short_circuit(appflow_conn, ctx): + with mock.patch("airflow.models.TaskInstance.xcom_pull") as mock_xcom_pull: + with mock.patch("airflow.models.TaskInstance.xcom_push") as mock_xcom_push: + mock_xcom_pull.return_value = EXECUTION_ID + operator = AppflowRecordsShortCircuitOperator( + task_id=SHORT_CIRCUIT_TASK_ID, + flow_name=FLOW_NAME, + appflow_run_task_id=TASK_ID, + ) + operator.execute(ctx) # type: ignore + appflow_conn.describe_flow_execution_records.assert_called_once_with( + flowName=FLOW_NAME, maxResults=100 + ) + mock_xcom_push.assert_called_with("records_processed", 1) From 77f2a591036fd497e9282dcfbc8cedc2d0b13aa9 Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Thu, 2 Jun 2022 15:48:28 -0300 Subject: [PATCH 12/26] Add get_airflow_version. --- .../aws/example_dags/example_appflow.py | 1 - .../providers/amazon/aws/operators/appflow.py | 12 +++-- .../amazon/aws/secrets/secrets_manager.py | 4 +- .../amazon/aws/secrets/systems_manager.py | 4 +- .../providers/amazon/aws/utils/__init__.py | 9 ++++ tests/providers/amazon/aws/utils/__init__.py | 19 -------- .../providers/amazon/aws/utils/test_utils.py | 44 +++++++++++++++++++ 7 files changed, 66 insertions(+), 27 deletions(-) create mode 100644 tests/providers/amazon/aws/utils/test_utils.py diff --git a/airflow/providers/amazon/aws/example_dags/example_appflow.py b/airflow/providers/amazon/aws/example_dags/example_appflow.py index 3cd6778478948..d11ce6fef28b4 100644 --- a/airflow/providers/amazon/aws/example_dags/example_appflow.py +++ b/airflow/providers/amazon/aws/example_dags/example_appflow.py @@ -91,7 +91,6 @@ task_id="campaign_dump_short_circuit", flow_name=FLOW_NAME, appflow_run_task_id="campaign_dump_after", # Should shortcircuit, no records expected - ignore_downstream_trigger_rules=True, ) # [END howto_operator_appflow_shortcircuit] diff --git a/airflow/providers/amazon/aws/operators/appflow.py b/airflow/providers/amazon/aws/operators/appflow.py index 0e9916e6f0237..c96d1fe68a26d 100644 --- a/airflow/providers/amazon/aws/operators/appflow.py +++ b/airflow/providers/amazon/aws/operators/appflow.py @@ -31,7 +31,7 @@ from airflow.models import BaseOperator from airflow.operators.python import ShortCircuitOperator from airflow.providers.amazon.aws.hooks.appflow import AppflowHook -from airflow.providers.amazon.aws.utils import datetime_to_epoch_ms +from airflow.providers.amazon.aws.utils import datetime_to_epoch_ms, get_airflow_version if TYPE_CHECKING: from mypy_boto3_appflow.type_defs import ( @@ -458,7 +458,7 @@ class AppflowRecordsShortCircuitOperator(ShortCircuitOperator): :param flow_name: The flow name :param appflow_run_task_id: Run task ID from where this operator should extract the execution ID - :param ignore_downstream_trigger_rules: Ignore downstream trigger rules + :param ignore_downstream_trigger_rules: Ignore downstream trigger rules (Ignored for Airflow < 2.3) :param aws_conn_id: aws connection to use :param region: aws region to use """ @@ -476,13 +476,19 @@ def __init__( region: Optional[str] = None, **kwargs, ) -> None: + if get_airflow_version() >= (2, 3): + kwargs["ignore_downstream_trigger_rules"] = ignore_downstream_trigger_rules + else: + self.log.warning( + "Ignoring argument ignore_downstream_trigger_rules (%s) - Only supported for Airflow >= 2.3", + ignore_downstream_trigger_rules, + ) super().__init__( python_callable=self._has_new_records_func, op_kwargs={ "flow_name": flow_name, "appflow_run_task_id": appflow_run_task_id, }, - ignore_downstream_trigger_rules=ignore_downstream_trigger_rules, **kwargs, ) self.aws_conn_id = aws_conn_id diff --git a/airflow/providers/amazon/aws/secrets/secrets_manager.py b/airflow/providers/amazon/aws/secrets/secrets_manager.py index 8b72f955ac6d5..c740077a56836 100644 --- a/airflow/providers/amazon/aws/secrets/secrets_manager.py +++ b/airflow/providers/amazon/aws/secrets/secrets_manager.py @@ -27,7 +27,7 @@ import boto3 -from airflow.version import version as airflow_version +from airflow.providers.amazon.aws.utils import get_airflow_version if sys.version_info >= (3, 8): from functools import cached_property @@ -214,7 +214,7 @@ def get_conn_uri(self, conn_id: str) -> Optional[str]: :param conn_id: the connection id :return: deserialized Connection """ - if _parse_version(airflow_version) >= (2, 3): + if get_airflow_version() >= (2, 3): warnings.warn( f"Method `{self.__class__.__name__}.get_conn_uri` is deprecated and will be removed " "in a future release. Please use method `get_conn_value` instead.", diff --git a/airflow/providers/amazon/aws/secrets/systems_manager.py b/airflow/providers/amazon/aws/secrets/systems_manager.py index e45a5500ab003..140fec11ef7ce 100644 --- a/airflow/providers/amazon/aws/secrets/systems_manager.py +++ b/airflow/providers/amazon/aws/secrets/systems_manager.py @@ -23,7 +23,7 @@ import boto3 -from airflow.version import version as airflow_version +from airflow.providers.amazon.aws.utils import get_airflow_version if sys.version_info >= (3, 8): from functools import cached_property @@ -115,7 +115,7 @@ def get_conn_uri(self, conn_id: str) -> Optional[str]: :param conn_id: the connection id :return: deserialized Connection """ - if _parse_version(airflow_version) >= (2, 3): + if get_airflow_version() >= (2, 3): warnings.warn( f"Method `{self.__class__.__name__}.get_conn_uri` is deprecated and will be removed " "in a future release. Please use method `get_conn_value` instead.", diff --git a/airflow/providers/amazon/aws/utils/__init__.py b/airflow/providers/amazon/aws/utils/__init__.py index d9acff6bc484d..a3755843befc5 100644 --- a/airflow/providers/amazon/aws/utils/__init__.py +++ b/airflow/providers/amazon/aws/utils/__init__.py @@ -15,7 +15,11 @@ # specific language governing permissions and limitations # under the License. +import re from datetime import datetime +from typing import Tuple + +from airflow.version import version def datetime_to_epoch(dt: datetime) -> int: @@ -31,3 +35,8 @@ def datetime_to_epoch_ms(dt: datetime) -> int: def datetime_to_epoch_us(dt: datetime) -> int: """Convert a datetime object to an epoch integer (microseconds).""" return int(dt.timestamp() * 1_000_000) + + +def get_airflow_version() -> Tuple[int, ...]: + val = re.sub(r'(\d+\.\d+\.\d+).*', lambda x: x.group(1), version) + return tuple(int(x) for x in val.split('.')) diff --git a/tests/providers/amazon/aws/utils/__init__.py b/tests/providers/amazon/aws/utils/__init__.py index dafaadd0bde40..13a83393a9124 100644 --- a/tests/providers/amazon/aws/utils/__init__.py +++ b/tests/providers/amazon/aws/utils/__init__.py @@ -14,22 +14,3 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. - -from datetime import datetime - -from airflow.providers.amazon.aws.utils import datetime_to_epoch, datetime_to_epoch_ms, datetime_to_epoch_us - -DT = datetime(2000, 1, 1) -EPOCH = 946_684_800 - - -def test_datetime_to_epoch(): - assert datetime_to_epoch(DT) == EPOCH - - -def test_datetime_to_epoch_ms(): - assert datetime_to_epoch_ms(DT) == EPOCH * 1000 - - -def test_datetime_to_epoch_us(): - assert datetime_to_epoch_us(DT) == EPOCH * 1_000_000 diff --git a/tests/providers/amazon/aws/utils/test_utils.py b/tests/providers/amazon/aws/utils/test_utils.py new file mode 100644 index 0000000000000..9a6bc198c1fa3 --- /dev/null +++ b/tests/providers/amazon/aws/utils/test_utils.py @@ -0,0 +1,44 @@ +# 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 datetime import datetime + +from airflow.providers.amazon.aws.utils import ( + datetime_to_epoch, + datetime_to_epoch_ms, + datetime_to_epoch_us, + get_airflow_version, +) + +DT = datetime(2000, 1, 1) +EPOCH = 946_684_800 + + +def test_datetime_to_epoch(): + assert datetime_to_epoch(DT) == EPOCH + + +def test_datetime_to_epoch_ms(): + assert datetime_to_epoch_ms(DT) == EPOCH * 1000 + + +def test_datetime_to_epoch_us(): + assert datetime_to_epoch_us(DT) == EPOCH * 1_000_000 + + +def test_get_airflow_version(): + assert len(get_airflow_version()) == 3 From 2a754d19787ef99e57c0b73750cf5cac6f2843a1 Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Tue, 7 Jun 2022 18:51:20 -0300 Subject: [PATCH 13/26] Update airflow/providers/amazon/aws/hooks/appflow.py Co-authored-by: Josh Fell <48934154+josh-fell@users.noreply.github.com> --- airflow/providers/amazon/aws/hooks/appflow.py | 1 - 1 file changed, 1 deletion(-) diff --git a/airflow/providers/amazon/aws/hooks/appflow.py b/airflow/providers/amazon/aws/hooks/appflow.py index 643ea517ac995..2bcd73b7fd5d7 100644 --- a/airflow/providers/amazon/aws/hooks/appflow.py +++ b/airflow/providers/amazon/aws/hooks/appflow.py @@ -38,7 +38,6 @@ class AppflowHook(AwsBaseHook): .. seealso:: :class:`~airflow.providers.amazon.aws.hooks.base_aws.AwsBaseHook` - :param aws_conn_id: The Airflow connection used for AWS credentials. """ def __init__(self, *args, **kwargs) -> None: From 94471ca5fc625ce6bfe0f8c84068a76b98416b73 Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Tue, 7 Jun 2022 18:51:50 -0300 Subject: [PATCH 14/26] Update airflow/providers/amazon/aws/operators/appflow.py Co-authored-by: Josh Fell <48934154+josh-fell@users.noreply.github.com> --- airflow/providers/amazon/aws/operators/appflow.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/amazon/aws/operators/appflow.py b/airflow/providers/amazon/aws/operators/appflow.py index c96d1fe68a26d..e5042161bef94 100644 --- a/airflow/providers/amazon/aws/operators/appflow.py +++ b/airflow/providers/amazon/aws/operators/appflow.py @@ -72,7 +72,7 @@ def __init__( source_field: Optional[str] = None, dt: Optional[str] = None, poll_interval: int = 20, - aws_conn_id: Optional[str] = "aws_default", + aws_conn_id: str = "aws_default", region: Optional[str] = None, **kwargs, ) -> None: From 485558bcc4c98d6261224204136ea93dc65c8863 Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Tue, 7 Jun 2022 18:52:37 -0300 Subject: [PATCH 15/26] Update airflow/providers/amazon/aws/operators/appflow.py Co-authored-by: Josh Fell <48934154+josh-fell@users.noreply.github.com> --- airflow/providers/amazon/aws/operators/appflow.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/amazon/aws/operators/appflow.py b/airflow/providers/amazon/aws/operators/appflow.py index e5042161bef94..6e9764469cba4 100644 --- a/airflow/providers/amazon/aws/operators/appflow.py +++ b/airflow/providers/amazon/aws/operators/appflow.py @@ -151,7 +151,7 @@ def _run_flow(self, context) -> str: response = self.hook.conn.describe_flow(flowName=self.name) # Wait Appflow eventual consistence - self.log.info("Waiting Appflow eventual consistence...") + self.log.info("Waiting for Appflow eventual consistence...") while ( response.get("lastRunExecutionDetails", {}).get( "mostRecentExecutionTime", datetime(1970, 1, 1, tzinfo=timezone.utc) From 4f2123f66caf887d9aedba8927ca4ab8a0015125 Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Tue, 7 Jun 2022 18:52:52 -0300 Subject: [PATCH 16/26] Update airflow/providers/amazon/aws/operators/appflow.py Co-authored-by: Josh Fell <48934154+josh-fell@users.noreply.github.com> --- airflow/providers/amazon/aws/operators/appflow.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/amazon/aws/operators/appflow.py b/airflow/providers/amazon/aws/operators/appflow.py index 6e9764469cba4..b78f23383487c 100644 --- a/airflow/providers/amazon/aws/operators/appflow.py +++ b/airflow/providers/amazon/aws/operators/appflow.py @@ -162,7 +162,7 @@ def _run_flow(self, context) -> str: response = self.hook.conn.describe_flow(flowName=self.name) # Wait flow stops - self.log.info("Waiting flow run...") + self.log.info("Waiting for flow run...") while ( "mostRecentExecutionStatus" not in response["lastRunExecutionDetails"] or response["lastRunExecutionDetails"]["mostRecentExecutionStatus"] == "InProgress" From d850ea9bec23b73e5978f1554f3f11b2f6c7646f Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Tue, 7 Jun 2022 18:53:23 -0300 Subject: [PATCH 17/26] Update airflow/providers/amazon/aws/operators/appflow.py Co-authored-by: Josh Fell <48934154+josh-fell@users.noreply.github.com> --- airflow/providers/amazon/aws/operators/appflow.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/amazon/aws/operators/appflow.py b/airflow/providers/amazon/aws/operators/appflow.py index b78f23383487c..0d837fc005391 100644 --- a/airflow/providers/amazon/aws/operators/appflow.py +++ b/airflow/providers/amazon/aws/operators/appflow.py @@ -198,7 +198,7 @@ def __init__( source: str, name: str, poll_interval: int = 20, - aws_conn_id: Optional[str] = "aws_default", + aws_conn_id: str = "aws_default", region: Optional[str] = None, **kwargs, ) -> None: From f7705edb65848d6d84b5f4ecf25b6c55fcd88606 Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Tue, 7 Jun 2022 18:54:06 -0300 Subject: [PATCH 18/26] Update airflow/providers/amazon/aws/operators/appflow.py Co-authored-by: Josh Fell <48934154+josh-fell@users.noreply.github.com> --- airflow/providers/amazon/aws/operators/appflow.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/amazon/aws/operators/appflow.py b/airflow/providers/amazon/aws/operators/appflow.py index 0d837fc005391..96e5a0e52a74a 100644 --- a/airflow/providers/amazon/aws/operators/appflow.py +++ b/airflow/providers/amazon/aws/operators/appflow.py @@ -510,7 +510,7 @@ def hook(self) -> AppflowHook: def _has_new_records_func(self, **kwargs) -> bool: appflow_task_id = kwargs["appflow_run_task_id"] - self.log.info("appflow_task_id: ", appflow_task_id) + self.log.info("appflow_task_id: %s", appflow_task_id) flow_name = kwargs["flow_name"] self.log.info("flow_name: %s", flow_name) af_client = self.hook.conn From 8c3a2209d9ef231d13b8c613d0690b9cc716ea1a Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Tue, 7 Jun 2022 20:04:03 -0300 Subject: [PATCH 19/26] Addressing Josh's requests. --- .../aws/example_dags/example_appflow.py | 6 +- .../providers/amazon/aws/operators/appflow.py | 127 +++++++++--------- .../amazon/aws/operators/test_appflow.py | 6 +- 3 files changed, 72 insertions(+), 67 deletions(-) diff --git a/airflow/providers/amazon/aws/example_dags/example_appflow.py b/airflow/providers/amazon/aws/example_dags/example_appflow.py index d11ce6fef28b4..b81b6858bf6fd 100644 --- a/airflow/providers/amazon/aws/example_dags/example_appflow.py +++ b/airflow/providers/amazon/aws/example_dags/example_appflow.py @@ -62,7 +62,7 @@ source=SOURCE_NAME, name=FLOW_NAME, source_field="LastModifiedDate", - dt="{{ ds }}", + filter_date="{{ ds }}", ) # [END howto_operator_appflow_run_daily] @@ -72,7 +72,7 @@ source=SOURCE_NAME, name=FLOW_NAME, source_field="LastModifiedDate", - dt="{{ ds }}", + filter_date="{{ ds }}", ) # [END howto_operator_appflow_run_before] @@ -82,7 +82,7 @@ source=SOURCE_NAME, name=FLOW_NAME, source_field="LastModifiedDate", - dt="3000-01-01", # Future date, so no records to dump + filter_date="3000-01-01", # Future date, so no records to dump ) # [END howto_operator_appflow_run_after] diff --git a/airflow/providers/amazon/aws/operators/appflow.py b/airflow/providers/amazon/aws/operators/appflow.py index 96e5a0e52a74a..12996f7bd988e 100644 --- a/airflow/providers/amazon/aws/operators/appflow.py +++ b/airflow/providers/amazon/aws/operators/appflow.py @@ -46,16 +46,19 @@ EVENTUAL_CONSISTENCY_OFFSET: int = 15 # seconds EVENTUAL_CONSISTENCY_POLLING: int = 10 # seconds SUPPORTED_SOURCES = {"salesforce", "zendesk"} +MANDATORY_FILTER_DATE_MSG = "The filter_date argument is mandatory for {entity}!" +NOT_SUPPORTED_SOURCE_MSG = "Source {source} is not supported for {entity}!" class AppflowBaseOperator(BaseOperator): """ Amazon Appflow Base Operator class (not supposed to be used directly in DAGs). - :param source: The source name (e.g. salesforce) + :param source: The source name (Supported: salesforce, zendesk) :param name: The flow name + :param flow_update: A boolean to enable/disable the a flow update before the run :param source_field: The field name to apply filters - :param dt: The date value (or template) to be used in filters. + :param filter_date: The date value (or template) to be used in filters. :param poll_interval: how often in seconds to check the query status :param aws_conn_id: aws connection to use :param region: aws region to use @@ -70,7 +73,7 @@ def __init__( name: str, flow_update: bool, source_field: Optional[str] = None, - dt: Optional[str] = None, + filter_date: Optional[str] = None, poll_interval: int = 20, aws_conn_id: str = "aws_default", region: Optional[str] = None, @@ -78,8 +81,8 @@ def __init__( ) -> None: super().__init__(**kwargs) if source not in SUPPORTED_SOURCES: - raise AirflowException(f"{source} is not a supported source (options: {SUPPORTED_SOURCES})!") - self.dt = dt + raise ValueError(f"{source} is not a supported source (options: {SUPPORTED_SOURCES})!") + self.filter_date = filter_date self.name = name self.source = source self.source_field = source_field @@ -94,7 +97,9 @@ def hook(self) -> AppflowHook: return AppflowHook(aws_conn_id=self.aws_conn_id, region_name=self.region) def execute(self, context: "Context") -> None: - self.dt_parsed: Optional[datetime] = datetime.fromisoformat(self.dt) if self.dt else None + self.filter_date_parsed: Optional[datetime] = ( + datetime.fromisoformat(self.filter_date) if self.filter_date else None + ) if self.flow_update: self._update_flow() self._run_flow(context) @@ -102,9 +107,7 @@ def execute(self, context: "Context") -> None: def _get_connector_type(self, response: "DescribeFlowResponseTypeDef") -> str: connector_type = response["sourceFlowConfig"]["connectorType"] if self.source != connector_type.lower(): - raise AirflowException( - f"Incompatible source ({self.source} and connector type ({connector_type})!" - ) + raise ValueError(f"Incompatible source ({self.source} and connector type ({connector_type})!") return connector_type def _update_flow(self) -> None: @@ -186,7 +189,7 @@ class AppflowRunOperator(AppflowBaseOperator): For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:AppflowRunOperator` - :param source: The source name (e.g. salesforce, zendesk) + :param source: The source name (Supported: salesforce, zendesk) :param name: The flow name :param poll_interval: how often in seconds to check the query status :param aws_conn_id: aws connection to use @@ -203,13 +206,13 @@ def __init__( **kwargs, ) -> None: if source not in {"salesforce", "zendesk"}: - raise AirflowException(f"Source {source} is not supported for AppflowRunOperator!") + raise ValueError(NOT_SUPPORTED_SOURCE_MSG.format(source=source, entity="AppflowRunOperator")) super().__init__( source=source, name=name, flow_update=False, source_field=None, - dt=None, + filter_date=None, poll_interval=poll_interval, aws_conn_id=aws_conn_id, region=region, @@ -225,7 +228,7 @@ class AppflowRunFullOperator(AppflowBaseOperator): For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:AppflowRunFullOperator` - :param source: The source name (e.g. salesforce, zendesk) + :param source: The source name (Supported: salesforce, zendesk) :param name: The flow name :param poll_interval: how often in seconds to check the query status :param aws_conn_id: aws connection to use @@ -237,18 +240,18 @@ def __init__( source: str, name: str, poll_interval: int = 20, - aws_conn_id: Optional[str] = "aws_default", + aws_conn_id: str = "aws_default", region: Optional[str] = None, **kwargs, ) -> None: if source not in {"salesforce", "zendesk"}: - raise AirflowException(f"Source {source} is not supported for AppflowRunFullOperator!") + raise ValueError(NOT_SUPPORTED_SOURCE_MSG.format(source=source, entity="AppflowRunFullOperator")) super().__init__( source=source, name=name, flow_update=True, source_field=None, - dt=None, + filter_date=None, poll_interval=poll_interval, aws_conn_id=aws_conn_id, region=region, @@ -264,38 +267,40 @@ class AppflowRunBeforeOperator(AppflowBaseOperator): For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:AppflowRunBeforeOperator` - :param source: The source name (e.g. salesforce) + :param source: The source name (Supported: salesforce) :param name: The flow name :param source_field: The field name to apply filters - :param dt: The date value (or template) to be used in filters. + :param filter_date: The date value (or template) to be used in filters. :param poll_interval: how often in seconds to check the query status :param aws_conn_id: aws connection to use :param region: aws region to use """ - template_fields = ("dt",) + template_fields = ("filter_date",) def __init__( self, source: str, name: str, source_field: str, - dt: str, + filter_date: str, poll_interval: int = 20, - aws_conn_id: Optional[str] = "aws_default", + aws_conn_id: str = "aws_default", region: Optional[str] = None, **kwargs, ) -> None: - if not dt: - raise AirflowException("The dt argument is mandatory for AppflowRunBeforeOperator!") - if source not in {"salesforce"}: - raise AirflowException(f"Source {source} is not supported for AppflowRunBeforeOperator!") + if not filter_date: + raise ValueError(MANDATORY_FILTER_DATE_MSG.format(entity="AppflowRunBeforeOperator")) + if source != "salesforce": + raise ValueError( + NOT_SUPPORTED_SOURCE_MSG.format(source=source, entity="AppflowRunBeforeOperator") + ) super().__init__( source=source, name=name, flow_update=True, source_field=source_field, - dt=dt, + filter_date=filter_date, poll_interval=poll_interval, aws_conn_id=aws_conn_id, region=region, @@ -303,17 +308,17 @@ def __init__( ) def _add_filter(self, connector_type: str, tasks: List["TaskTypeDef"]) -> None: - if not self.dt_parsed: - raise AirflowException(f"Invalid dt argument parser value: {self.dt_parsed}") + if not self.filter_date_parsed: + raise ValueError(f"Invalid filter_date argument parser value: {self.filter_date_parsed}") if not self.source_field: - raise AirflowException(f"Invalid source_field argument value: {self.source_field}") + raise ValueError(f"Invalid source_field argument value: {self.source_field}") filter_task: "TaskTypeDef" = { "taskType": "Filter", "connectorOperator": {connector_type: "LESS_THAN"}, # type: ignore "sourceFields": [self.source_field], "taskProperties": { "DATA_TYPE": "datetime", - "VALUE": str(datetime_to_epoch_ms(self.dt_parsed)), + "VALUE": str(datetime_to_epoch_ms(self.filter_date_parsed)), }, # NOT inclusive } tasks.append(filter_task) @@ -327,38 +332,38 @@ class AppflowRunAfterOperator(AppflowBaseOperator): For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:AppflowRunAfterOperator` - :param source: The source name (e.g. salesforce, zendesk) + :param source: The source name (Supported: salesforce, zendesk) :param name: The flow name :param source_field: The field name to apply filters - :param dt: The date value (or template) to be used in filters. + :param filter_date: The date value (or template) to be used in filters. :param poll_interval: how often in seconds to check the query status :param aws_conn_id: aws connection to use :param region: aws region to use """ - template_fields = ("dt",) + template_fields = ("filter_date",) def __init__( self, source: str, name: str, source_field: str, - dt: str, + filter_date: str, poll_interval: int = 20, - aws_conn_id: Optional[str] = "aws_default", + aws_conn_id: str = "aws_default", region: Optional[str] = None, **kwargs, ) -> None: - if not dt: - raise AirflowException("The dt argument is mandatory for AppflowRunAfterOperator!") + if not filter_date: + raise ValueError(MANDATORY_FILTER_DATE_MSG.format(entity="AppflowRunAfterOperator")) if source not in {"salesforce", "zendesk"}: - raise AirflowException(f"Source {source} is not supported for AppflowRunAfterOperator!") + raise ValueError(NOT_SUPPORTED_SOURCE_MSG.format(source=source, entity="AppflowRunAfterOperator")) super().__init__( source=source, name=name, flow_update=True, source_field=source_field, - dt=dt, + filter_date=filter_date, poll_interval=poll_interval, aws_conn_id=aws_conn_id, region=region, @@ -366,17 +371,17 @@ def __init__( ) def _add_filter(self, connector_type: str, tasks: List["TaskTypeDef"]) -> None: - if not self.dt_parsed: - raise AirflowException(f"Invalid dt argument parser value: {self.dt_parsed}") + if not self.filter_date_parsed: + raise ValueError(f"Invalid filter_date argument parser value: {self.filter_date_parsed}") if not self.source_field: - raise AirflowException(f"Invalid source_field argument value: {self.source_field}") + raise ValueError(f"Invalid source_field argument value: {self.source_field}") filter_task: "TaskTypeDef" = { "taskType": "Filter", "connectorOperator": {connector_type: "GREATER_THAN"}, # type: ignore "sourceFields": [self.source_field], "taskProperties": { "DATA_TYPE": "datetime", - "VALUE": str(datetime_to_epoch_ms(self.dt_parsed)), + "VALUE": str(datetime_to_epoch_ms(self.filter_date_parsed)), }, # NOT inclusive } tasks.append(filter_task) @@ -390,38 +395,38 @@ class AppflowRunDailyOperator(AppflowBaseOperator): For more information on how to use this operator, take a look at the guide: :ref:`howto/operator:AppflowRunDailyOperator` - :param source: The source name (e.g. salesforce) + :param source: The source name (Supported: salesforce) :param name: The flow name :param source_field: The field name to apply filters - :param dt: The date value (or template) to be used in filters. + :param filter_date: The date value (or template) to be used in filters. :param poll_interval: how often in seconds to check the query status :param aws_conn_id: aws connection to use :param region: aws region to use """ - template_fields = ("dt",) + template_fields = ("filter_date",) def __init__( self, source: str, name: str, source_field: str, - dt: str, + filter_date: str, poll_interval: int = 20, - aws_conn_id: Optional[str] = "aws_default", + aws_conn_id: str = "aws_default", region: Optional[str] = None, **kwargs, ) -> None: - if not dt: - raise AirflowException("The dt argument is mandatory for AppflowRunDailyOperator!") - if source not in {"salesforce"}: - raise AirflowException(f"Source {source} is not supported for AppflowRunDailyOperator!") + if not filter_date: + raise ValueError(MANDATORY_FILTER_DATE_MSG.format(entity="AppflowRunDailyOperator")) + if source != "salesforce": + raise ValueError(NOT_SUPPORTED_SOURCE_MSG.format(source=source, entity="AppflowRunDailyOperator")) super().__init__( source=source, name=name, flow_update=True, source_field=source_field, - dt=dt, + filter_date=filter_date, poll_interval=poll_interval, aws_conn_id=aws_conn_id, region=region, @@ -429,20 +434,20 @@ def __init__( ) def _add_filter(self, connector_type: str, tasks: List["TaskTypeDef"]) -> None: - if not self.dt_parsed: - raise AirflowException(f"Invalid dt argument parser value: {self.dt_parsed}") + if not self.filter_date_parsed: + raise ValueError(f"Invalid filter_date argument parser value: {self.filter_date_parsed}") if not self.source_field: - raise AirflowException(f"Invalid source_field argument value: {self.source_field}") - start_dt = self.dt_parsed - timedelta(milliseconds=1) - end_dt = self.dt_parsed + timedelta(days=1) + raise ValueError(f"Invalid source_field argument value: {self.source_field}") + start_filter_date = self.filter_date_parsed - timedelta(milliseconds=1) + end_filter_date = self.filter_date_parsed + timedelta(days=1) filter_task: "TaskTypeDef" = { "taskType": "Filter", "connectorOperator": {connector_type: "BETWEEN"}, # type: ignore "sourceFields": [self.source_field], "taskProperties": { "DATA_TYPE": "datetime", - "LOWER_BOUND": str(datetime_to_epoch_ms(start_dt)), # NOT inclusive - "UPPER_BOUND": str(datetime_to_epoch_ms(end_dt)), # NOT inclusive + "LOWER_BOUND": str(datetime_to_epoch_ms(start_filter_date)), # NOT inclusive + "UPPER_BOUND": str(datetime_to_epoch_ms(end_filter_date)), # NOT inclusive }, } tasks.append(filter_task) @@ -472,7 +477,7 @@ def __init__( flow_name: str, appflow_run_task_id: str, ignore_downstream_trigger_rules: bool = True, - aws_conn_id: Optional[str] = "aws_default", + aws_conn_id: str = "aws_default", region: Optional[str] = None, **kwargs, ) -> None: diff --git a/tests/providers/amazon/aws/operators/test_appflow.py b/tests/providers/amazon/aws/operators/test_appflow.py index 71a7429fae6e8..781666ea0eece 100644 --- a/tests/providers/amazon/aws/operators/test_appflow.py +++ b/tests/providers/amazon/aws/operators/test_appflow.py @@ -104,7 +104,7 @@ def test_run_full(appflow_conn, ctx): def test_run_after(appflow_conn, ctx): - operator = AppflowRunAfterOperator(source_field="col0", dt="2022-05-26", **DUMP_COMMON_ARGS) + operator = AppflowRunAfterOperator(source_field="col0", filter_date="2022-05-26", **DUMP_COMMON_ARGS) operator.execute(ctx) # type: ignore run_assertions_base( appflow_conn, @@ -120,7 +120,7 @@ def test_run_after(appflow_conn, ctx): def test_run_before(appflow_conn, ctx): - operator = AppflowRunBeforeOperator(source_field="col0", dt="2022-05-26", **DUMP_COMMON_ARGS) + operator = AppflowRunBeforeOperator(source_field="col0", filter_date="2022-05-26", **DUMP_COMMON_ARGS) operator.execute(ctx) # type: ignore run_assertions_base( appflow_conn, @@ -136,7 +136,7 @@ def test_run_before(appflow_conn, ctx): def test_run_daily(appflow_conn, ctx): - operator = AppflowRunDailyOperator(source_field="col0", dt="2022-05-26", **DUMP_COMMON_ARGS) + operator = AppflowRunDailyOperator(source_field="col0", filter_date="2022-05-26", **DUMP_COMMON_ARGS) operator.execute(ctx) # type: ignore run_assertions_base( appflow_conn, From 57a6f1e24757a4ae2533e36c6a86e4b123205c9f Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Fri, 10 Jun 2022 17:33:55 -0300 Subject: [PATCH 20/26] Add cached_property to AppflowHook --- airflow/providers/amazon/aws/hooks/appflow.py | 8 +++++++- tests/providers/amazon/aws/hooks/test_appflow.py | 2 ++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/airflow/providers/amazon/aws/hooks/appflow.py b/airflow/providers/amazon/aws/hooks/appflow.py index 2bcd73b7fd5d7..85d2152b0d244 100644 --- a/airflow/providers/amazon/aws/hooks/appflow.py +++ b/airflow/providers/amazon/aws/hooks/appflow.py @@ -15,10 +15,16 @@ # specific language governing permissions and limitations # under the License. +import sys from typing import TYPE_CHECKING from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook +if sys.version_info >= (3, 8): + from functools import cached_property +else: + from cached_property import cached_property + if TYPE_CHECKING: from mypy_boto3_appflow.client import AppflowClient @@ -44,7 +50,7 @@ def __init__(self, *args, **kwargs) -> None: kwargs["client_type"] = "appflow" super().__init__(*args, **kwargs) - @property + @cached_property def conn(self) -> 'AppflowClient': """Get the underlying boto3 Appflow client (cached)""" return super().conn diff --git a/tests/providers/amazon/aws/hooks/test_appflow.py b/tests/providers/amazon/aws/hooks/test_appflow.py index 4e62dca294f87..6837e6ba16da5 100644 --- a/tests/providers/amazon/aws/hooks/test_appflow.py +++ b/tests/providers/amazon/aws/hooks/test_appflow.py @@ -23,4 +23,6 @@ class TestAppflowHook: def test_conn_attribute(self): hook = AppflowHook(aws_conn_id='aws_default', region_name='us-east-1') assert hasattr(hook, 'conn') + conn = hook.conn + assert conn is hook.conn, "AppflowHook conn property non-cached" assert hook.conn.__class__.__name__ == 'Appflow' From d7098ec61abab2e486dacafd743706dd32935f5d Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Wed, 15 Jun 2022 18:35:16 -0300 Subject: [PATCH 21/26] Update airflow/providers/amazon/aws/hooks/appflow.py Co-authored-by: Josh Fell <48934154+josh-fell@users.noreply.github.com> --- airflow/providers/amazon/aws/hooks/appflow.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/providers/amazon/aws/hooks/appflow.py b/airflow/providers/amazon/aws/hooks/appflow.py index 85d2152b0d244..074227330cf60 100644 --- a/airflow/providers/amazon/aws/hooks/appflow.py +++ b/airflow/providers/amazon/aws/hooks/appflow.py @@ -31,8 +31,8 @@ class AppflowHook(AwsBaseHook): """ - Interact with Amazon Appflow, using the boto3 library - Hook attribute `conn` has all methods that listed in documentation + Interact with Amazon Appflow, using the boto3 library. + Hook attribute ``conn`` has all methods that listed in documentation. .. seealso:: - https://boto3.amazonaws.com/v1/documentation/api/latest/reference/services/appflow.html From ccdf4bf4ed4bb3cf1807e089167ed405d3a9dff2 Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Wed, 15 Jun 2022 18:35:47 -0300 Subject: [PATCH 22/26] Update airflow/providers/amazon/aws/operators/appflow.py Co-authored-by: Josh Fell <48934154+josh-fell@users.noreply.github.com> --- airflow/providers/amazon/aws/operators/appflow.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/airflow/providers/amazon/aws/operators/appflow.py b/airflow/providers/amazon/aws/operators/appflow.py index 12996f7bd988e..d61ec6be621a1 100644 --- a/airflow/providers/amazon/aws/operators/appflow.py +++ b/airflow/providers/amazon/aws/operators/appflow.py @@ -64,8 +64,7 @@ class AppflowBaseOperator(BaseOperator): :param region: aws region to use """ - BLUE = "#2bccbd" - ui_color = BLUE + ui_color = "#2bccbd" def __init__( self, From 2e03f6faa74396926607a65a818b9b08dc5de559 Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Wed, 15 Jun 2022 18:36:02 -0300 Subject: [PATCH 23/26] Update airflow/providers/amazon/aws/operators/appflow.py Co-authored-by: Josh Fell <48934154+josh-fell@users.noreply.github.com> --- airflow/providers/amazon/aws/operators/appflow.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/amazon/aws/operators/appflow.py b/airflow/providers/amazon/aws/operators/appflow.py index d61ec6be621a1..b625ee7c1fce0 100644 --- a/airflow/providers/amazon/aws/operators/appflow.py +++ b/airflow/providers/amazon/aws/operators/appflow.py @@ -56,7 +56,7 @@ class AppflowBaseOperator(BaseOperator): :param source: The source name (Supported: salesforce, zendesk) :param name: The flow name - :param flow_update: A boolean to enable/disable the a flow update before the run + :param flow_update: A boolean to enable/disable a flow update before the run :param source_field: The field name to apply filters :param filter_date: The date value (or template) to be used in filters. :param poll_interval: how often in seconds to check the query status From fcbdd63ee42ce53e8c82cffc734a379d81ffc50f Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Wed, 15 Jun 2022 21:33:51 -0300 Subject: [PATCH 24/26] Update Josh's comment. --- .../aws/example_dags/example_appflow.py | 10 +- airflow/providers/amazon/aws/hooks/appflow.py | 96 +++++++++++- .../providers/amazon/aws/operators/appflow.py | 145 ++++++------------ .../amazon/aws/hooks/test_appflow.py | 78 +++++++++- .../amazon/aws/operators/test_appflow.py | 6 +- 5 files changed, 216 insertions(+), 119 deletions(-) diff --git a/airflow/providers/amazon/aws/example_dags/example_appflow.py b/airflow/providers/amazon/aws/example_dags/example_appflow.py index b81b6858bf6fd..06903c73d9a2e 100644 --- a/airflow/providers/amazon/aws/example_dags/example_appflow.py +++ b/airflow/providers/amazon/aws/example_dags/example_appflow.py @@ -44,7 +44,7 @@ campaign_dump = AppflowRunOperator( task_id="campaign_dump", source=SOURCE_NAME, - name=FLOW_NAME, + flow_name=FLOW_NAME, ) # [END howto_operator_appflow_run] @@ -52,7 +52,7 @@ campaign_dump_full = AppflowRunFullOperator( task_id="campaign_dump_full", source=SOURCE_NAME, - name=FLOW_NAME, + flow_name=FLOW_NAME, ) # [END howto_operator_appflow_run_full] @@ -60,7 +60,7 @@ campaign_dump_daily = AppflowRunDailyOperator( task_id="campaign_dump_daily", source=SOURCE_NAME, - name=FLOW_NAME, + flow_name=FLOW_NAME, source_field="LastModifiedDate", filter_date="{{ ds }}", ) @@ -70,7 +70,7 @@ campaign_dump_before = AppflowRunBeforeOperator( task_id="campaign_dump_before", source=SOURCE_NAME, - name=FLOW_NAME, + flow_name=FLOW_NAME, source_field="LastModifiedDate", filter_date="{{ ds }}", ) @@ -80,7 +80,7 @@ campaign_dump_after = AppflowRunAfterOperator( task_id="campaign_dump_after", source=SOURCE_NAME, - name=FLOW_NAME, + flow_name=FLOW_NAME, source_field="LastModifiedDate", filter_date="3000-01-01", # Future date, so no records to dump ) diff --git a/airflow/providers/amazon/aws/hooks/appflow.py b/airflow/providers/amazon/aws/hooks/appflow.py index 074227330cf60..841e813019fd2 100644 --- a/airflow/providers/amazon/aws/hooks/appflow.py +++ b/airflow/providers/amazon/aws/hooks/appflow.py @@ -15,8 +15,11 @@ # specific language governing permissions and limitations # under the License. +import json import sys -from typing import TYPE_CHECKING +from datetime import datetime, timezone +from time import sleep +from typing import TYPE_CHECKING, List from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook @@ -27,6 +30,7 @@ if TYPE_CHECKING: from mypy_boto3_appflow.client import AppflowClient + from mypy_boto3_appflow.type_defs import TaskTypeDef class AppflowHook(AwsBaseHook): @@ -46,6 +50,9 @@ class AppflowHook(AwsBaseHook): """ + EVENTUAL_CONSISTENCY_OFFSET: int = 15 # seconds + EVENTUAL_CONSISTENCY_POLLING: int = 10 # seconds + def __init__(self, *args, **kwargs) -> None: kwargs["client_type"] = "appflow" super().__init__(*args, **kwargs) @@ -54,3 +61,90 @@ def __init__(self, *args, **kwargs) -> None: def conn(self) -> 'AppflowClient': """Get the underlying boto3 Appflow client (cached)""" return super().conn + + def run_flow(self, flow_name: str, poll_interval: int = 20) -> str: + """ + Execute an AppFlow run. + + :param flow_name: The flow name + :param poll_interval: Time (seconds) to wait between two consecutive calls to check the run status + :return: The run execution ID + """ + ts_before: datetime = datetime.now(timezone.utc) + sleep(self.EVENTUAL_CONSISTENCY_OFFSET) + response = self.conn.start_flow(flowName=flow_name) + execution_id = response["executionId"] + self.log.info("executionId: %s", execution_id) + + response = self.conn.describe_flow(flowName=flow_name) + last_exec_details = response["lastRunExecutionDetails"] + + # Wait Appflow eventual consistence + self.log.info("Waiting for Appflow eventual consistence...") + while ( + response.get("lastRunExecutionDetails", {}).get( + "mostRecentExecutionTime", datetime(1970, 1, 1, tzinfo=timezone.utc) + ) + < ts_before + ): + sleep(self.EVENTUAL_CONSISTENCY_POLLING) + response = self.conn.describe_flow(flowName=flow_name) + last_exec_details = response["lastRunExecutionDetails"] + + # Wait flow stops + self.log.info("Waiting for flow run...") + while ( + "mostRecentExecutionStatus" not in last_exec_details + or last_exec_details["mostRecentExecutionStatus"] == "InProgress" + ): + sleep(poll_interval) + response = self.conn.describe_flow(flowName=flow_name) + last_exec_details = response["lastRunExecutionDetails"] + + self.log.info("lastRunExecutionDetails: %s", last_exec_details) + + if last_exec_details["mostRecentExecutionStatus"] == "Error": + raise Exception(f"Flow error:\n{json.dumps(response, default=str)}") + + return execution_id + + def update_flow_filter( + self, flow_name: str, filter_tasks: List["TaskTypeDef"], set_trigger_ondemand: bool = False + ) -> None: + """ + Update the flow task filter. + All filters will be removed if an empty array is passed to filter_tasks. + + :param flow_name: The flow name + :param filter_tasks: List flow tasks to be added + :param set_trigger_ondemand: If True, set the trigger to on-demand; otherwise, keep the trigger as is + :return: None + """ + response = self.conn.describe_flow(flowName=flow_name) + connector_type = response["sourceFlowConfig"]["connectorType"] + tasks: List["TaskTypeDef"] = [] + + # cleanup old filter tasks + for task in response["tasks"]: + if ( + task["taskType"] == "Filter" + and task.get("connectorOperator", {}).get(connector_type) != "PROJECTION" + ): + self.log.info("Removing task: %s", task) + else: + tasks.append(task) # List of non-filter tasks + + tasks += filter_tasks # Add the new filter tasks + + if set_trigger_ondemand: + # Clean up attribute to force on-demand trigger + del response["triggerConfig"]["triggerProperties"] + + self.conn.update_flow( + flowName=response["flowName"], + destinationFlowConfigList=response["destinationFlowConfigList"], + sourceFlowConfig=response["sourceFlowConfig"], + triggerConfig=response["triggerConfig"], + description=response.get("description", "Flow description."), + tasks=tasks, + ) diff --git a/airflow/providers/amazon/aws/operators/appflow.py b/airflow/providers/amazon/aws/operators/appflow.py index b625ee7c1fce0..9fc162b57c211 100644 --- a/airflow/providers/amazon/aws/operators/appflow.py +++ b/airflow/providers/amazon/aws/operators/appflow.py @@ -15,11 +15,8 @@ # specific language governing permissions and limitations # under the License. -import copy -import json import sys -from datetime import datetime, timedelta, timezone -from time import sleep +from datetime import datetime, timedelta from typing import TYPE_CHECKING, List, Optional, cast if sys.version_info >= (3, 8): @@ -36,15 +33,13 @@ if TYPE_CHECKING: from mypy_boto3_appflow.type_defs import ( DescribeFlowExecutionRecordsResponseTypeDef, - DescribeFlowResponseTypeDef, ExecutionRecordTypeDef, TaskTypeDef, ) from airflow.utils.context import Context -EVENTUAL_CONSISTENCY_OFFSET: int = 15 # seconds -EVENTUAL_CONSISTENCY_POLLING: int = 10 # seconds + SUPPORTED_SOURCES = {"salesforce", "zendesk"} MANDATORY_FILTER_DATE_MSG = "The filter_date argument is mandatory for {entity}!" NOT_SUPPORTED_SOURCE_MSG = "Source {source} is not supported for {entity}!" @@ -55,7 +50,7 @@ class AppflowBaseOperator(BaseOperator): Amazon Appflow Base Operator class (not supposed to be used directly in DAGs). :param source: The source name (Supported: salesforce, zendesk) - :param name: The flow name + :param flow_name: The flow name :param flow_update: A boolean to enable/disable a flow update before the run :param source_field: The field name to apply filters :param filter_date: The date value (or template) to be used in filters. @@ -69,7 +64,7 @@ class AppflowBaseOperator(BaseOperator): def __init__( self, source: str, - name: str, + flow_name: str, flow_update: bool, source_field: Optional[str] = None, filter_date: Optional[str] = None, @@ -82,7 +77,7 @@ def __init__( if source not in SUPPORTED_SOURCES: raise ValueError(f"{source} is not a supported source (options: {SUPPORTED_SOURCES})!") self.filter_date = filter_date - self.name = name + self.flow_name = flow_name self.source = source self.source_field = source_field self.poll_interval = poll_interval @@ -99,85 +94,26 @@ def execute(self, context: "Context") -> None: self.filter_date_parsed: Optional[datetime] = ( datetime.fromisoformat(self.filter_date) if self.filter_date else None ) + self.connector_type = self._get_connector_type() if self.flow_update: self._update_flow() self._run_flow(context) - def _get_connector_type(self, response: "DescribeFlowResponseTypeDef") -> str: + def _get_connector_type(self) -> str: + response = self.hook.conn.describe_flow(flowName=self.flow_name) connector_type = response["sourceFlowConfig"]["connectorType"] if self.source != connector_type.lower(): raise ValueError(f"Incompatible source ({self.source} and connector type ({connector_type})!") return connector_type def _update_flow(self) -> None: - response = self.hook.conn.describe_flow(flowName=self.name) - connector_type = self._get_connector_type(response) - - # cleanup - tasks: List["TaskTypeDef"] = [] - for task in response["tasks"]: - if ( - task["taskType"] == "Filter" - and task.get("connectorOperator", {}).get(connector_type) != "PROJECTION" - ): - self.log.info("Removing task: %s", task) - else: - tasks.append(task) # List of non-filter tasks - - self._add_filter(connector_type, tasks) - - # Clean up to force on-demand trigger - trigger_config = copy.deepcopy(response["triggerConfig"]) - del trigger_config["triggerProperties"] - - self.hook.conn.update_flow( - flowName=response["flowName"], - destinationFlowConfigList=response["destinationFlowConfigList"], - sourceFlowConfig=response["sourceFlowConfig"], - triggerConfig=trigger_config, - description=response.get("description", "Flow description."), - tasks=tasks, - ) - - def _add_filter(self, connector_type: str, tasks: List["TaskTypeDef"]) -> None: # Interface - pass + self.hook.update_flow_filter(flow_name=self.flow_name, filter_tasks=[], set_trigger_ondemand=True) def _run_flow(self, context) -> str: - ts_before: datetime = datetime.now(timezone.utc) - sleep(EVENTUAL_CONSISTENCY_OFFSET) - response = self.hook.conn.start_flow(flowName=self.name) + execution_id = self.hook.run_flow(flow_name=self.flow_name, poll_interval=self.poll_interval) task_instance = context["task_instance"] - task_instance.xcom_push("execution_id", response["executionId"]) - self.log.info("executionId: %s", response["executionId"]) - - response = self.hook.conn.describe_flow(flowName=self.name) - - # Wait Appflow eventual consistence - self.log.info("Waiting for Appflow eventual consistence...") - while ( - response.get("lastRunExecutionDetails", {}).get( - "mostRecentExecutionTime", datetime(1970, 1, 1, tzinfo=timezone.utc) - ) - < ts_before - ): - sleep(EVENTUAL_CONSISTENCY_POLLING) - response = self.hook.conn.describe_flow(flowName=self.name) - - # Wait flow stops - self.log.info("Waiting for flow run...") - while ( - "mostRecentExecutionStatus" not in response["lastRunExecutionDetails"] - or response["lastRunExecutionDetails"]["mostRecentExecutionStatus"] == "InProgress" - ): - sleep(self.poll_interval) - response = self.hook.conn.describe_flow(flowName=self.name) - - self.log.info("lastRunExecutionDetails: %s", response["lastRunExecutionDetails"]) - - if response["lastRunExecutionDetails"]["mostRecentExecutionStatus"] == "Error": - raise Exception(f"Flow error:\n{json.dumps(response, default=str)}") - - return response["lastRunExecutionDetails"]["mostRecentExecutionStatus"] + task_instance.xcom_push("execution_id", execution_id) + return execution_id class AppflowRunOperator(AppflowBaseOperator): @@ -189,7 +125,7 @@ class AppflowRunOperator(AppflowBaseOperator): :ref:`howto/operator:AppflowRunOperator` :param source: The source name (Supported: salesforce, zendesk) - :param name: The flow name + :param flow_name: The flow name :param poll_interval: how often in seconds to check the query status :param aws_conn_id: aws connection to use :param region: aws region to use @@ -198,7 +134,7 @@ class AppflowRunOperator(AppflowBaseOperator): def __init__( self, source: str, - name: str, + flow_name: str, poll_interval: int = 20, aws_conn_id: str = "aws_default", region: Optional[str] = None, @@ -208,7 +144,7 @@ def __init__( raise ValueError(NOT_SUPPORTED_SOURCE_MSG.format(source=source, entity="AppflowRunOperator")) super().__init__( source=source, - name=name, + flow_name=flow_name, flow_update=False, source_field=None, filter_date=None, @@ -228,7 +164,7 @@ class AppflowRunFullOperator(AppflowBaseOperator): :ref:`howto/operator:AppflowRunFullOperator` :param source: The source name (Supported: salesforce, zendesk) - :param name: The flow name + :param flow_name: The flow name :param poll_interval: how often in seconds to check the query status :param aws_conn_id: aws connection to use :param region: aws region to use @@ -237,7 +173,7 @@ class AppflowRunFullOperator(AppflowBaseOperator): def __init__( self, source: str, - name: str, + flow_name: str, poll_interval: int = 20, aws_conn_id: str = "aws_default", region: Optional[str] = None, @@ -247,7 +183,7 @@ def __init__( raise ValueError(NOT_SUPPORTED_SOURCE_MSG.format(source=source, entity="AppflowRunFullOperator")) super().__init__( source=source, - name=name, + flow_name=flow_name, flow_update=True, source_field=None, filter_date=None, @@ -267,7 +203,7 @@ class AppflowRunBeforeOperator(AppflowBaseOperator): :ref:`howto/operator:AppflowRunBeforeOperator` :param source: The source name (Supported: salesforce) - :param name: The flow name + :param flow_name: The flow name :param source_field: The field name to apply filters :param filter_date: The date value (or template) to be used in filters. :param poll_interval: how often in seconds to check the query status @@ -280,7 +216,7 @@ class AppflowRunBeforeOperator(AppflowBaseOperator): def __init__( self, source: str, - name: str, + flow_name: str, source_field: str, filter_date: str, poll_interval: int = 20, @@ -296,7 +232,7 @@ def __init__( ) super().__init__( source=source, - name=name, + flow_name=flow_name, flow_update=True, source_field=source_field, filter_date=filter_date, @@ -306,21 +242,23 @@ def __init__( **kwargs, ) - def _add_filter(self, connector_type: str, tasks: List["TaskTypeDef"]) -> None: + def _update_flow(self) -> None: if not self.filter_date_parsed: raise ValueError(f"Invalid filter_date argument parser value: {self.filter_date_parsed}") if not self.source_field: raise ValueError(f"Invalid source_field argument value: {self.source_field}") filter_task: "TaskTypeDef" = { "taskType": "Filter", - "connectorOperator": {connector_type: "LESS_THAN"}, # type: ignore + "connectorOperator": {self.connector_type: "LESS_THAN"}, # type: ignore "sourceFields": [self.source_field], "taskProperties": { "DATA_TYPE": "datetime", "VALUE": str(datetime_to_epoch_ms(self.filter_date_parsed)), }, # NOT inclusive } - tasks.append(filter_task) + self.hook.update_flow_filter( + flow_name=self.flow_name, filter_tasks=[filter_task], set_trigger_ondemand=True + ) class AppflowRunAfterOperator(AppflowBaseOperator): @@ -332,7 +270,7 @@ class AppflowRunAfterOperator(AppflowBaseOperator): :ref:`howto/operator:AppflowRunAfterOperator` :param source: The source name (Supported: salesforce, zendesk) - :param name: The flow name + :param flow_name: The flow name :param source_field: The field name to apply filters :param filter_date: The date value (or template) to be used in filters. :param poll_interval: how often in seconds to check the query status @@ -345,7 +283,7 @@ class AppflowRunAfterOperator(AppflowBaseOperator): def __init__( self, source: str, - name: str, + flow_name: str, source_field: str, filter_date: str, poll_interval: int = 20, @@ -359,7 +297,7 @@ def __init__( raise ValueError(NOT_SUPPORTED_SOURCE_MSG.format(source=source, entity="AppflowRunAfterOperator")) super().__init__( source=source, - name=name, + flow_name=flow_name, flow_update=True, source_field=source_field, filter_date=filter_date, @@ -369,21 +307,23 @@ def __init__( **kwargs, ) - def _add_filter(self, connector_type: str, tasks: List["TaskTypeDef"]) -> None: + def _update_flow(self) -> None: if not self.filter_date_parsed: raise ValueError(f"Invalid filter_date argument parser value: {self.filter_date_parsed}") if not self.source_field: raise ValueError(f"Invalid source_field argument value: {self.source_field}") filter_task: "TaskTypeDef" = { "taskType": "Filter", - "connectorOperator": {connector_type: "GREATER_THAN"}, # type: ignore + "connectorOperator": {self.connector_type: "GREATER_THAN"}, # type: ignore "sourceFields": [self.source_field], "taskProperties": { "DATA_TYPE": "datetime", "VALUE": str(datetime_to_epoch_ms(self.filter_date_parsed)), }, # NOT inclusive } - tasks.append(filter_task) + self.hook.update_flow_filter( + flow_name=self.flow_name, filter_tasks=[filter_task], set_trigger_ondemand=True + ) class AppflowRunDailyOperator(AppflowBaseOperator): @@ -395,7 +335,7 @@ class AppflowRunDailyOperator(AppflowBaseOperator): :ref:`howto/operator:AppflowRunDailyOperator` :param source: The source name (Supported: salesforce) - :param name: The flow name + :param flow_name: The flow name :param source_field: The field name to apply filters :param filter_date: The date value (or template) to be used in filters. :param poll_interval: how often in seconds to check the query status @@ -408,7 +348,7 @@ class AppflowRunDailyOperator(AppflowBaseOperator): def __init__( self, source: str, - name: str, + flow_name: str, source_field: str, filter_date: str, poll_interval: int = 20, @@ -422,7 +362,7 @@ def __init__( raise ValueError(NOT_SUPPORTED_SOURCE_MSG.format(source=source, entity="AppflowRunDailyOperator")) super().__init__( source=source, - name=name, + flow_name=flow_name, flow_update=True, source_field=source_field, filter_date=filter_date, @@ -432,7 +372,7 @@ def __init__( **kwargs, ) - def _add_filter(self, connector_type: str, tasks: List["TaskTypeDef"]) -> None: + def _update_flow(self) -> None: if not self.filter_date_parsed: raise ValueError(f"Invalid filter_date argument parser value: {self.filter_date_parsed}") if not self.source_field: @@ -441,7 +381,7 @@ def _add_filter(self, connector_type: str, tasks: List["TaskTypeDef"]) -> None: end_filter_date = self.filter_date_parsed + timedelta(days=1) filter_task: "TaskTypeDef" = { "taskType": "Filter", - "connectorOperator": {connector_type: "BETWEEN"}, # type: ignore + "connectorOperator": {self.connector_type: "BETWEEN"}, # type: ignore "sourceFields": [self.source_field], "taskProperties": { "DATA_TYPE": "datetime", @@ -449,7 +389,9 @@ def _add_filter(self, connector_type: str, tasks: List["TaskTypeDef"]) -> None: "UPPER_BOUND": str(datetime_to_epoch_ms(end_filter_date)), # NOT inclusive }, } - tasks.append(filter_task) + self.hook.update_flow_filter( + flow_name=self.flow_name, filter_tasks=[filter_task], set_trigger_ondemand=True + ) class AppflowRecordsShortCircuitOperator(ShortCircuitOperator): @@ -467,8 +409,7 @@ class AppflowRecordsShortCircuitOperator(ShortCircuitOperator): :param region: aws region to use """ - LIGHT_BLUE = "#33ffec" - ui_color = LIGHT_BLUE + ui_color = "#33ffec" # Light blue def __init__( self, diff --git a/tests/providers/amazon/aws/hooks/test_appflow.py b/tests/providers/amazon/aws/hooks/test_appflow.py index 6837e6ba16da5..f50e6e9a86aad 100644 --- a/tests/providers/amazon/aws/hooks/test_appflow.py +++ b/tests/providers/amazon/aws/hooks/test_appflow.py @@ -15,14 +15,76 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -# + +from datetime import datetime +from unittest import mock +from unittest.mock import ANY + +import pytest + from airflow.providers.amazon.aws.hooks.appflow import AppflowHook +from airflow.utils import timezone + +FLOW_NAME = "flow0" +EXECUTION_ID = "ex_id" +CONNECTION_TYPE = "Salesforce" +REGION_NAME = "us-east-1" +AWS_CONN_ID = "aws_default" + + +@pytest.fixture +def hook(): + with mock.patch("airflow.providers.amazon.aws.hooks.appflow.AppflowHook.__init__", return_value=None): + with mock.patch("airflow.providers.amazon.aws.hooks.appflow.AppflowHook.conn") as mock_conn: + mock_conn.describe_flow.return_value = { + 'sourceFlowConfig': {'connectorType': CONNECTION_TYPE}, + 'tasks': [], + 'triggerConfig': {'triggerProperties': None}, + 'flowName': FLOW_NAME, + 'destinationFlowConfigList': {}, + 'lastRunExecutionDetails': { + 'mostRecentExecutionStatus': 'Successful', + 'mostRecentExecutionTime': datetime(3000, 1, 1, tzinfo=timezone.utc), + }, + } + mock_conn.update_flow.return_value = {} + mock_conn.start_flow.return_value = {"executionId": EXECUTION_ID} + mock_conn.describe_flow_execution_records.return_value = { + "flowExecutions": [{"executionId": EXECUTION_ID, "executionResult": {"recordsProcessed": 1}}] + } + yield AppflowHook(aws_conn_id=AWS_CONN_ID, region_name=REGION_NAME) + + +def test_conn_attributes(hook): + assert hasattr(hook, 'conn') + conn = hook.conn + assert conn is hook.conn, "AppflowHook conn property non-cached" + + +def test_run_flow(hook): + hook.run_flow(flow_name=FLOW_NAME) + hook.conn.describe_flow.assert_called_with(flowName=FLOW_NAME) + assert hook.conn.describe_flow.call_count == 1 + hook.conn.start_flow.assert_called_once_with(flowName=FLOW_NAME) -class TestAppflowHook: - def test_conn_attribute(self): - hook = AppflowHook(aws_conn_id='aws_default', region_name='us-east-1') - assert hasattr(hook, 'conn') - conn = hook.conn - assert conn is hook.conn, "AppflowHook conn property non-cached" - assert hook.conn.__class__.__name__ == 'Appflow' +def test_update_flow_filter(hook): + tasks = [ + { + 'taskType': 'Filter', + 'connectorOperator': {'Salesforce': 'GREATER_THAN'}, + 'sourceFields': ['col0'], + 'taskProperties': {'DATA_TYPE': 'datetime', 'VALUE': '1653523200000'}, + } + ] + hook.update_flow_filter(flow_name=FLOW_NAME, filter_tasks=tasks, set_trigger_ondemand=True) + hook.conn.describe_flow.assert_called_with(flowName=FLOW_NAME) + assert hook.conn.describe_flow.call_count == 1 + hook.conn.update_flow.assert_called_once_with( + flowName=FLOW_NAME, + tasks=tasks, + description=ANY, + destinationFlowConfigList=ANY, + sourceFlowConfig=ANY, + triggerConfig=ANY, + ) diff --git a/tests/providers/amazon/aws/operators/test_appflow.py b/tests/providers/amazon/aws/operators/test_appflow.py index 781666ea0eece..7bbd515693541 100644 --- a/tests/providers/amazon/aws/operators/test_appflow.py +++ b/tests/providers/amazon/aws/operators/test_appflow.py @@ -40,7 +40,7 @@ CONNECTION_TYPE = "Salesforce" SOURCE = "salesforce" -DUMP_COMMON_ARGS = {"aws_conn_id": CONN_ID, "task_id": TASK_ID, "source": SOURCE, "name": FLOW_NAME} +DUMP_COMMON_ARGS = {"aws_conn_id": CONN_ID, "task_id": TASK_ID, "source": SOURCE, "flow_name": FLOW_NAME} @pytest.fixture @@ -77,7 +77,7 @@ def appflow_conn(): def run_assertions_base(appflow_conn, tasks): appflow_conn.describe_flow.assert_called_with(flowName=FLOW_NAME) - assert appflow_conn.describe_flow.call_count == 2 + assert appflow_conn.describe_flow.call_count == 3 appflow_conn.update_flow.assert_called_once_with( flowName=FLOW_NAME, tasks=tasks, @@ -93,7 +93,7 @@ def test_run(appflow_conn, ctx): operator = AppflowRunOperator(**DUMP_COMMON_ARGS) operator.execute(ctx) # type: ignore appflow_conn.describe_flow.assert_called_with(flowName=FLOW_NAME) - assert appflow_conn.describe_flow.call_count == 1 + assert appflow_conn.describe_flow.call_count == 2 appflow_conn.start_flow.assert_called_once_with(flowName=FLOW_NAME) From 263abdd8123b078a1b9be9ecf95211bedd0673ea Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Tue, 21 Jun 2022 13:42:32 -0300 Subject: [PATCH 25/26] Update cached_property import. --- airflow/providers/amazon/aws/hooks/appflow.py | 7 +------ airflow/providers/amazon/aws/operators/appflow.py | 7 +------ airflow/providers/amazon/aws/utils/__init__.py | 12 ++++++------ docs/spelling_wordlist.txt | 1 - 4 files changed, 8 insertions(+), 19 deletions(-) diff --git a/airflow/providers/amazon/aws/hooks/appflow.py b/airflow/providers/amazon/aws/hooks/appflow.py index 841e813019fd2..e096a8e7144ed 100644 --- a/airflow/providers/amazon/aws/hooks/appflow.py +++ b/airflow/providers/amazon/aws/hooks/appflow.py @@ -16,18 +16,13 @@ # under the License. import json -import sys from datetime import datetime, timezone from time import sleep from typing import TYPE_CHECKING, List +from airflow.compat.functools import cached_property from airflow.providers.amazon.aws.hooks.base_aws import AwsBaseHook -if sys.version_info >= (3, 8): - from functools import cached_property -else: - from cached_property import cached_property - if TYPE_CHECKING: from mypy_boto3_appflow.client import AppflowClient from mypy_boto3_appflow.type_defs import TaskTypeDef diff --git a/airflow/providers/amazon/aws/operators/appflow.py b/airflow/providers/amazon/aws/operators/appflow.py index 9fc162b57c211..a63ec8ccdb5c1 100644 --- a/airflow/providers/amazon/aws/operators/appflow.py +++ b/airflow/providers/amazon/aws/operators/appflow.py @@ -15,15 +15,10 @@ # specific language governing permissions and limitations # under the License. -import sys from datetime import datetime, timedelta from typing import TYPE_CHECKING, List, Optional, cast -if sys.version_info >= (3, 8): - from functools import cached_property -else: - from cached_property import cached_property - +from airflow.compat.functools import cached_property from airflow.exceptions import AirflowException from airflow.models import BaseOperator from airflow.operators.python import ShortCircuitOperator diff --git a/airflow/providers/amazon/aws/utils/__init__.py b/airflow/providers/amazon/aws/utils/__init__.py index a3755843befc5..7f127f7178034 100644 --- a/airflow/providers/amazon/aws/utils/__init__.py +++ b/airflow/providers/amazon/aws/utils/__init__.py @@ -22,19 +22,19 @@ from airflow.version import version -def datetime_to_epoch(dt: datetime) -> int: +def datetime_to_epoch(date_time: datetime) -> int: """Convert a datetime object to an epoch integer (seconds).""" - return int(dt.timestamp()) + return int(date_time.timestamp()) -def datetime_to_epoch_ms(dt: datetime) -> int: +def datetime_to_epoch_ms(date_time: datetime) -> int: """Convert a datetime object to an epoch integer (milliseconds).""" - return int(dt.timestamp() * 1_000) + return int(date_time.timestamp() * 1_000) -def datetime_to_epoch_us(dt: datetime) -> int: +def datetime_to_epoch_us(date_time: datetime) -> int: """Convert a datetime object to an epoch integer (microseconds).""" - return int(dt.timestamp() * 1_000_000) + return int(date_time.timestamp() * 1_000_000) def get_airflow_version() -> Tuple[int, ...]: diff --git a/docs/spelling_wordlist.txt b/docs/spelling_wordlist.txt index 950a6d94d33d6..61bf1a7ea4bba 100644 --- a/docs/spelling_wordlist.txt +++ b/docs/spelling_wordlist.txt @@ -760,7 +760,6 @@ dropdown druidHook ds dsn -dt dttm dtypes durations From 28dfacda5aca170a42c78760f1399e91ae12f425 Mon Sep 17 00:00:00 2001 From: Igor Tavares Date: Tue, 21 Jun 2022 16:35:01 -0300 Subject: [PATCH 26/26] Fix mypy. --- airflow/providers/amazon/aws/hooks/appflow.py | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/airflow/providers/amazon/aws/hooks/appflow.py b/airflow/providers/amazon/aws/hooks/appflow.py index e096a8e7144ed..e2b628bf3d334 100644 --- a/airflow/providers/amazon/aws/hooks/appflow.py +++ b/airflow/providers/amazon/aws/hooks/appflow.py @@ -67,24 +67,24 @@ def run_flow(self, flow_name: str, poll_interval: int = 20) -> str: """ ts_before: datetime = datetime.now(timezone.utc) sleep(self.EVENTUAL_CONSISTENCY_OFFSET) - response = self.conn.start_flow(flowName=flow_name) - execution_id = response["executionId"] + response_start = self.conn.start_flow(flowName=flow_name) + execution_id = response_start["executionId"] self.log.info("executionId: %s", execution_id) - response = self.conn.describe_flow(flowName=flow_name) - last_exec_details = response["lastRunExecutionDetails"] + response_desc = self.conn.describe_flow(flowName=flow_name) + last_exec_details = response_desc["lastRunExecutionDetails"] # Wait Appflow eventual consistence self.log.info("Waiting for Appflow eventual consistence...") while ( - response.get("lastRunExecutionDetails", {}).get( + response_desc.get("lastRunExecutionDetails", {}).get( "mostRecentExecutionTime", datetime(1970, 1, 1, tzinfo=timezone.utc) ) < ts_before ): sleep(self.EVENTUAL_CONSISTENCY_POLLING) - response = self.conn.describe_flow(flowName=flow_name) - last_exec_details = response["lastRunExecutionDetails"] + response_desc = self.conn.describe_flow(flowName=flow_name) + last_exec_details = response_desc["lastRunExecutionDetails"] # Wait flow stops self.log.info("Waiting for flow run...") @@ -93,13 +93,13 @@ def run_flow(self, flow_name: str, poll_interval: int = 20) -> str: or last_exec_details["mostRecentExecutionStatus"] == "InProgress" ): sleep(poll_interval) - response = self.conn.describe_flow(flowName=flow_name) - last_exec_details = response["lastRunExecutionDetails"] + response_desc = self.conn.describe_flow(flowName=flow_name) + last_exec_details = response_desc["lastRunExecutionDetails"] self.log.info("lastRunExecutionDetails: %s", last_exec_details) if last_exec_details["mostRecentExecutionStatus"] == "Error": - raise Exception(f"Flow error:\n{json.dumps(response, default=str)}") + raise Exception(f"Flow error:\n{json.dumps(response_desc, default=str)}") return execution_id