uranusjr commented on code in PR #26658:
URL: https://github.com/apache/airflow/pull/26658#discussion_r1002919415
##########
airflow/utils/task_group.py:
##########
@@ -454,6 +455,26 @@ def topological_sort(self, _include_subdag_tasks: bool =
False):
return graph_sorted
+ def get_task_dict(self) -> dict[str, AbstractOperator]:
+ """Returns a flat dictionary of task_id: AbstractOperator"""
+ task_map: dict[str, AbstractOperator] = {}
+ groups_to_visit = [self]
+
+ while groups_to_visit:
+ visiting = groups_to_visit.pop(0)
+
+ for child in visiting.children.values():
+ if isinstance(child, AbstractOperator):
+ task_map[child.task_id] = child
+ elif isinstance(child, TaskGroup):
+ groups_to_visit.append(child)
+ else:
+ raise ValueError(
+ f"Encountered a DAGNode that is not a TaskGroup or an
AbstractOperator: {type(child)}"
+ )
+
+ return task_map
Review Comment:
Hmm, this feels like a pretty expensive way for what is needed. Unless we
actually need to dict, using an iterator is likely better (for this particular
feature):
```python
def iter_tasks(self) -> Iterator[AbstractOperator]:
groups_to_visit = [self]
while groups_to_visit:
visiting = groups_to_visit.pop(0)
for child in visiting.children.values():
if isinstance(child, AbstractOperator):
yield child
elif isinstance(child, TaskGroup):
groups_to_visit.append(child)
else:
raise ValueError(f"Encountered unknown DAGNode type:
{type(child)}")
```
##########
airflow/www/views.py:
##########
@@ -2046,14 +2047,26 @@ def clear(self):
recursive = request.form.get('recursive') == "true"
only_failed = request.form.get('only_failed') == "true"
- task_ids: list[str | tuple[str, int]]
- if map_indexes is None:
- task_ids = [task_id]
+ task_ids: list[str | tuple[str, int]] = []
+ if group_id is not None:
+ task_group_dict = dag.task_group.get_task_group_dict()
+ task_group = task_group_dict.get(group_id)
+ if task_group is None:
+ return redirect_or_json(
+ origin, msg=f"TaskGroup {group_id} could not be found",
status="error", status_code=404
+ )
+ tasks = task_group.get_task_dict()
+ task_ids = list(tasks.keys())
+ task_ids_or_regex = task_ids
Review Comment:
```suggestion
task_ids = task_ids_or_regex = [t.task_id for t in
task_group.iter_tasks()]
```
--
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]