Lee-W commented on code in PR #36916:
URL: https://github.com/apache/airflow/pull/36916#discussion_r1461687786


##########
airflow/triggers/external_task.py:
##########
@@ -36,6 +38,106 @@
     from airflow.utils.state import DagRunState
 
 
+class WorkflowTrigger(BaseTrigger):
+    """
+    A trigger to monitor tasks, task group and dag execution in Apache Airflow.
+
+    :param external_dag_id: The ID of the external DAG.
+    :param execution_dates: A list of execution dates for the external DAG.
+    :param external_task_ids: A collection of external task IDs to wait for.
+    :param external_task_group_id: The ID of the external task group to wait 
for.
+    :param failed_states: States considered as failed for external tasks.
+    :param skipped_states: States considered as skipped for external tasks.
+    :param allowed_states: States considered as successful for external tasks.
+    :param poke_interval: The interval (in seconds) for poking the external 
tasks.
+    :param soft_fail: If True, the trigger will not fail the entire DAG on 
external task failure.
+    """
+
+    def __init__(
+        self,
+        external_dag_id: str,
+        execution_dates: list,
+        external_task_ids: typing.Collection[str] | None = None,
+        external_task_group_id: str | None = None,
+        failed_states: typing.Iterable[str] | None = None,
+        skipped_states: typing.Iterable[str] | None = None,
+        allowed_states: typing.Iterable[str] | None = None,
+        poke_interval: float = 2.0,
+        soft_fail: bool = False,
+        **kwargs,
+    ):
+        self.external_dag_id = external_dag_id
+        self.external_task_ids = external_task_ids
+        self.external_task_group_id = external_task_group_id
+        self.failed_states = failed_states
+        self.skipped_states = skipped_states
+        self.allowed_states = allowed_states
+        self.execution_dates = execution_dates
+        self.poke_interval = poke_interval
+        self.soft_fail = soft_fail
+        super().__init__(**kwargs)
+
+    def _set_context(self, context):

Review Comment:
   ```suggestion
       def _set_context(self, context: Context):
   ```



##########
airflow/hooks/utils.py:
##########
@@ -0,0 +1,120 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from __future__ import annotations
+
+import typing
+from typing import TYPE_CHECKING
+
+from sqlalchemy import func
+
+from airflow.models import DagBag, DagRun, TaskInstance
+from airflow.utils.session import NEW_SESSION, provide_session
+from airflow.utils.sqlalchemy import tuple_in_condition
+
+if TYPE_CHECKING:
+    from sqlalchemy.orm import Query, Session
+
+
+@provide_session
+def _get_count(
+    dttm_filter,
+    external_task_ids,
+    external_task_group_id,
+    external_dag_id,
+    states,
+    session: Session = NEW_SESSION,
+) -> int:
+    """
+    Get the count of records against dttm filter and states.
+
+    :param dttm_filter: date time filter for execution date
+    :param external_task_ids: The list of task_ids
+    :param external_task_group_id: The ID of the external task group
+    :param external_dag_id: The ID of the external DAG.
+    :param states: task or dag states
+    :param session: airflow session object
+    """
+    TI = TaskInstance
+    DR = DagRun
+    if not dttm_filter:
+        return 0
+
+    if external_task_ids:
+        count = (
+            _count_query(TI, states, dttm_filter, external_dag_id, session)
+            .filter(TI.task_id.in_(external_task_ids))
+            .scalar()
+        ) / len(external_task_ids)
+    elif external_task_group_id:
+        external_task_group_task_ids = _get_external_task_group_task_ids(
+            dttm_filter, external_task_group_id, external_dag_id, session
+        )
+        if not external_task_group_task_ids:
+            count = 0
+        else:
+            count = (
+                _count_query(TI, states, dttm_filter, external_dag_id, session)
+                .filter(tuple_in_condition((TI.task_id, TI.map_index), 
external_task_group_task_ids))
+                .scalar()
+            ) / len(external_task_group_task_ids)
+    else:
+        count = _count_query(DR, states, dttm_filter, external_dag_id, 
session).scalar()
+    return typing.cast(int, count)
+
+
+def _count_query(model, states, dttm_filter, external_dag_id, session: 
Session) -> Query:
+    """
+    Get the count of records against dttm filter and states.
+
+    :param model: The SQLAlchemy model representing the relevant table.
+    :param states: task or dag states
+    :param dttm_filter: date time filter for execution date
+    :param external_dag_id: The ID of the external DAG.
+    :param session: airflow session object
+    """
+    query = session.query(func.count()).filter(
+        model.dag_id == external_dag_id,
+        model.state.in_(states),
+        model.execution_date.in_(dttm_filter),
+    )
+    return query
+
+
+def _get_external_task_group_task_ids(dttm_filter, external_task_group_id, 
external_dag_id, session):

Review Comment:
   nitpick: should we add type annotation here?
   



##########
airflow/sensors/external_task.py:
##########
@@ -351,29 +348,30 @@ def execute(self, context: Context) -> None:
             super().execute(context)
         else:
             self.defer(
-                trigger=TaskStateTrigger(
-                    dag_id=self.external_dag_id,
-                    task_id=self.external_task_id,
+                timeout=self.execution_timeout,
+                trigger=WorkflowTrigger(
+                    external_dag_id=self.external_dag_id,
+                    external_task_ids=self.external_task_ids,
                     execution_dates=self._get_dttm_filter(context),
-                    states=self.allowed_states,
-                    trigger_start_time=utcnow(),
-                    poll_interval=self.poll_interval,
+                    allowed_states=self.allowed_states,
+                    poke_interval=self.poll_interval,
+                    soft_fail=self.soft_fail,
                 ),
                 method_name="execute_complete",
             )
 
     def execute_complete(self, context, event=None):
         """Execute when the trigger fires - return immediately."""
         if event["status"] == "success":
-            self.log.info("External task %s has executed successfully.", 
self.external_task_id)
-            return None
-        elif event["status"] == "timeout":
-            raise AirflowException("Dag was not started within 1 minute, 
assuming fail.")
+            self.log.info("External tasks %s has executed successfully.", 
self.external_task_ids)
         else:
-            raise AirflowException(
-                "Error occurred while trying to retrieve task status. Please, 
check the "
-                "name of executed task and Dag."
-            )
+            if self.soft_fail:

Review Comment:
   
https://github.com/apache/airflow/blob/c8775fc0122fdb7243728c76cde7368509edf725/airflow/sensors/base.py#L301-L304
   
   I think we might not need to catch this for core sensors.



##########
airflow/hooks/utils.py:
##########
@@ -0,0 +1,120 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from __future__ import annotations
+
+import typing
+from typing import TYPE_CHECKING
+
+from sqlalchemy import func
+
+from airflow.models import DagBag, DagRun, TaskInstance
+from airflow.utils.session import NEW_SESSION, provide_session
+from airflow.utils.sqlalchemy import tuple_in_condition
+
+if TYPE_CHECKING:
+    from sqlalchemy.orm import Query, Session
+
+
+@provide_session
+def _get_count(
+    dttm_filter,
+    external_task_ids,
+    external_task_group_id,
+    external_dag_id,
+    states,
+    session: Session = NEW_SESSION,
+) -> int:
+    """
+    Get the count of records against dttm filter and states.
+
+    :param dttm_filter: date time filter for execution date
+    :param external_task_ids: The list of task_ids
+    :param external_task_group_id: The ID of the external task group
+    :param external_dag_id: The ID of the external DAG.
+    :param states: task or dag states
+    :param session: airflow session object
+    """
+    TI = TaskInstance
+    DR = DagRun
+    if not dttm_filter:
+        return 0
+
+    if external_task_ids:
+        count = (
+            _count_query(TI, states, dttm_filter, external_dag_id, session)
+            .filter(TI.task_id.in_(external_task_ids))
+            .scalar()
+        ) / len(external_task_ids)
+    elif external_task_group_id:
+        external_task_group_task_ids = _get_external_task_group_task_ids(
+            dttm_filter, external_task_group_id, external_dag_id, session
+        )
+        if not external_task_group_task_ids:
+            count = 0
+        else:
+            count = (
+                _count_query(TI, states, dttm_filter, external_dag_id, session)
+                .filter(tuple_in_condition((TI.task_id, TI.map_index), 
external_task_group_task_ids))
+                .scalar()
+            ) / len(external_task_group_task_ids)
+    else:
+        count = _count_query(DR, states, dttm_filter, external_dag_id, 
session).scalar()
+    return typing.cast(int, count)
+
+
+def _count_query(model, states, dttm_filter, external_dag_id, session: 
Session) -> Query:

Review Comment:
   nitpick: should we add type annotation here?
   



##########
airflow/hooks/utils.py:
##########
@@ -0,0 +1,120 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from __future__ import annotations
+
+import typing
+from typing import TYPE_CHECKING
+
+from sqlalchemy import func
+
+from airflow.models import DagBag, DagRun, TaskInstance
+from airflow.utils.session import NEW_SESSION, provide_session
+from airflow.utils.sqlalchemy import tuple_in_condition
+
+if TYPE_CHECKING:
+    from sqlalchemy.orm import Query, Session
+
+
+@provide_session
+def _get_count(
+    dttm_filter,
+    external_task_ids,
+    external_task_group_id,
+    external_dag_id,
+    states,

Review Comment:
   nitpick: should we add type annotation here?



-- 
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]

Reply via email to