vincbeck commented on code in PR #24057: URL: https://github.com/apache/airflow/pull/24057#discussion_r886123715
########## 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] Review Comment: We try to be consistent across the AWS package. That reminds me we should definitely work on writing a README file to explain all this. Here are the 3 variants we are using depending on the use case: - howto_operator_ - howto_sensor_ - howto_transfer_ ```suggestion # [START howto_operator_appflow_run] ``` ########## 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 Review Comment: We also decided to use `chain` instead ########## 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): Review Comment: ```suggestion class AppflowRecordsShortCircuitOperator(ShortCircuitOperator): ``` ########## 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): Review Comment: Unless there is a valid reason, I think we can use `AirflowException` ########## 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( Review Comment: Why do we need this? ########## 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): Review Comment: Please add docstrings to explain parameters ########## docs/apache-airflow-providers-amazon/operators/appflow.rst: ########## @@ -0,0 +1,146 @@ + .. Licensed to the Apache Software Foundation (ASF) under one Review Comment: Thank you for following the format of other AWS documentation. That's awesome! ########## 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] Review Comment: This comment is valid for all the `howto_` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
