xinbinhuang commented on a change in pull request #14640:
URL: https://github.com/apache/airflow/pull/14640#discussion_r589825450



##########
File path: airflow/sensors/external_task.py
##########
@@ -206,29 +232,48 @@ def get_count(self, dttm_filter, session, states) -> int:
         """
         TI = TaskInstance
         DR = DagRun
+
         if self.external_task_id:
             count = (
-                session.query(func.count())  # .count() is inefficient
-                .filter(
-                    TI.dag_id == self.external_dag_id,
-                    TI.task_id == self.external_task_id,
-                    TI.state.in_(states),  # pylint: disable=no-member
-                    TI.execution_date.in_(dttm_filter),
-                )
+                self._count_query(TI, session, states, dttm_filter)
+                .filter(TI.task_id == self.external_task_id)
                 .scalar()
             )
-        else:
+        elif self.external_task_group_id:
+            external_task_group_task_ids = 
self.get_external_task_group_task_ids(session)
             count = (
-                session.query(func.count())
-                .filter(
-                    DR.dag_id == self.external_dag_id,
-                    DR.state.in_(states),  # pylint: disable=no-member
-                    DR.execution_date.in_(dttm_filter),
-                )
+                self._count_query(TI, session, states, dttm_filter)
+                .filter(TI.task_id.in_(external_task_group_task_ids))
                 .scalar()
-            )
+            ) / len(external_task_group_task_ids)
+        else:
+            count = self._count_query(DR, session, states, 
dttm_filter).scalar()
+
         return count
 
+    def _count_query(self, model, session, states, dttm_filter) -> "Query":
+        query = session.query(func.count()).filter(  # .count() is inefficient
+            model.dag_id == self.external_dag_id,
+            model.state.in_(states),  # pylint: disable=no-member
+            model.execution_date.in_(dttm_filter),
+        )
+
+        return query
+
+    def get_external_task_group_task_ids(self, session):
+        """Return task ids for the external TaskGroup"""
+        refreshed_dag_info = DagBag().get_dag(self.external_dag_id, session)
+        task_group: Optional["TaskGroup"] = 
refreshed_dag_info.task_group_dict.get(
+            self.external_task_group_id
+        )
+        if task_group is None:
+            raise AirflowException(
+                f'The external task group {self.external_task_group_id} in '
+                f'DAG {self.external_dag_id} does not exist.'
+            )
+        task_ids = [task.task_id for task in task_group]
+        return task_ids

Review comment:
       The main piece that you retrieve a list of tasks for a `TaskGroup`




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

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to