This is an automated email from the ASF dual-hosted git repository.

ephraimanierobi pushed a commit to branch v2-7-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit b0998038936d290e1ca5e237930c884fec1ce9e4
Author: Daniel Standish <[email protected]>
AuthorDate: Wed Aug 9 11:07:27 2023 -0700

    Remove add_task from TaskGroup (#33262)
    
    This was added during work for AIP-52, taking inspiration from the method 
on the setupteardowncontext object.  But it causes problems because it's 
assumed in the task_id label logic that if the group is set to prefix task ids 
then the task id has been prefixed.  This results in bad graph labeling in UI.  
Rather than fix it now, at release time, better to revert.
    
    (cherry picked from commit 2d2a1d699b8069f59604c6d6ea3e29853faa7945)
---
 airflow/models/abstractoperator.py |  8 --------
 airflow/models/taskmixin.py        |  5 -----
 airflow/models/xcom_arg.py         | 10 ----------
 airflow/utils/edgemodifier.py      |  9 ---------
 airflow/utils/task_group.py        | 20 --------------------
 5 files changed, 52 deletions(-)

diff --git a/airflow/models/abstractoperator.py 
b/airflow/models/abstractoperator.py
index 11e9184735..458d55ef5d 100644
--- a/airflow/models/abstractoperator.py
+++ b/airflow/models/abstractoperator.py
@@ -360,14 +360,6 @@ class AbstractOperator(Templater, DAGNode):
                 yield parent
             parent = parent.task_group
 
-    def add_to_taskgroup(self, task_group: TaskGroup) -> None:
-        """Add the task to the given task group.
-
-        :meta private:
-        """
-        if self.node_id not in task_group.children:
-            task_group.add(self)
-
     def get_closest_mapped_task_group(self) -> MappedTaskGroup | None:
         """Get the mapped task group "closest" to this task in the DAG.
 
diff --git a/airflow/models/taskmixin.py b/airflow/models/taskmixin.py
index 8c19749104..c5e7cea3b9 100644
--- a/airflow/models/taskmixin.py
+++ b/airflow/models/taskmixin.py
@@ -113,11 +113,6 @@ class DependencyMixin:
         self.__rshift__(other)
         return self
 
-    @abstractmethod
-    def add_to_taskgroup(self, task_group: TaskGroup) -> None:
-        """Add the task to the given task group."""
-        raise NotImplementedError()
-
     @classmethod
     def _iter_references(cls, obj: Any) -> Iterable[tuple[DependencyMixin, 
str]]:
         from airflow.models.baseoperator import AbstractOperator
diff --git a/airflow/models/xcom_arg.py b/airflow/models/xcom_arg.py
index e8cde16db8..0a54d18ff7 100644
--- a/airflow/models/xcom_arg.py
+++ b/airflow/models/xcom_arg.py
@@ -43,7 +43,6 @@ from airflow.utils.xcom import XCOM_RETURN_KEY
 if TYPE_CHECKING:
     from airflow.models.dag import DAG
     from airflow.models.operator import Operator
-    from airflow.utils.task_group import TaskGroup
 
 # Callable objects contained by MapXComArg. We only accept callables from
 # the user, but deserialize them into strings in a serialized XComArg for
@@ -209,15 +208,6 @@ class XComArg(ResolveMixin, DependencyMixin):
         """
         raise NotImplementedError()
 
-    def add_to_taskgroup(self, task_group: TaskGroup) -> None:
-        """Add the task to the given task group.
-
-        :meta private:
-        """
-        for op, _ in self.iter_references():
-            if op.node_id not in task_group.children:
-                task_group.add(op)
-
     def __enter__(self):
         if not self.operator.is_setup and not self.operator.is_teardown:
             raise AirflowException("Only setup/teardown tasks can be used as 
context managers.")
diff --git a/airflow/utils/edgemodifier.py b/airflow/utils/edgemodifier.py
index 14aa3ef25e..e760beb50a 100644
--- a/airflow/utils/edgemodifier.py
+++ b/airflow/utils/edgemodifier.py
@@ -169,15 +169,6 @@ class EdgeModifier(DependencyMixin):
         """
         dag.set_edge_info(upstream_id, downstream_id, {"label": self.label})
 
-    def add_to_taskgroup(self, task_group: TaskGroup) -> None:
-        """No-op, since we're not a task.
-
-        We only add tasks to TaskGroups and not EdgeModifiers, but we need
-        this to satisfy the interface.
-
-        :meta private:
-        """
-
 
 # Factory functions
 def Label(label: str):
diff --git a/airflow/utils/task_group.py b/airflow/utils/task_group.py
index b6c40a14a9..deee607dd1 100644
--- a/airflow/utils/task_group.py
+++ b/airflow/utils/task_group.py
@@ -551,26 +551,6 @@ class TaskGroup(DAGNode):
                         f"Encountered a DAGNode that is not a TaskGroup or an 
AbstractOperator: {type(child)}"
                     )
 
-    def add_task(self, task: AbstractOperator) -> None:
-        """Add a task to the task group.
-
-        :param task: the task to add
-        """
-        if not TaskGroupContext.active:
-            raise AirflowException(
-                "Using this method on a task group that's not a context 
manager is not supported."
-            )
-        task.add_to_taskgroup(self)
-
-    def add_to_taskgroup(self, task_group: TaskGroup) -> None:
-        """No-op, since we're not a task.
-
-        We only add tasks to TaskGroups and not TaskGroup, but we need
-        this to satisfy the interface.
-
-        :meta private:
-        """
-
 
 class MappedTaskGroup(TaskGroup):
     """A mapped task group.

Reply via email to