You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2021/04/15 20:14:02 UTC

[GitHub] [airflow] andrewgodwin opened a new pull request #15389: AIP-40 prototype: Deferrable Tasks

andrewgodwin opened a new pull request #15389:
URL: https://github.com/apache/airflow/pull/15389


   This is a Draft PR to illustrate the approach taken in AIP-40 (https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=177050929).
   
   It is not intended for merge unless AIP-40 is accepted, at which point this code would be polished and prepared for landing.


-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#issuecomment-849105605


   On the pools/queues thing - I agree, I think they should subtract from pools but not touch queues. I added DEFERRED to State.running - is anything else needed here? It seems to be generally working without incident, but I didn't see any tests that specifically tested that the various states in there (e.g. SENSING) work that I could add onto.


-- 
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:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r640533146



##########
File path: airflow/triggers/temporal.py
##########
@@ -0,0 +1,76 @@
+# 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 asyncio
+import datetime
+from typing import Any, Dict, Tuple
+
+import pytz
+
+from airflow.triggers.base import BaseTrigger, TriggerEvent
+from airflow.utils import timezone
+
+
+class DateTimeTrigger(BaseTrigger):
+    """
+    A trigger that fires exactly once, at the given datetime, give or take
+    a few seconds.
+
+    The provided datetime MUST be in UTC.
+    """
+
+    def __init__(self, moment: datetime.datetime):
+        super().__init__()
+        # Make sure it's in UTC
+        if moment.tzinfo is None:
+            self.moment = pytz.utc.localize(moment)
+        elif moment.tzinfo == pytz.utc or getattr(moment.tzinfo, "name", None) == "UTC":
+            self.moment = moment
+        else:
+            raise ValueError(f"The passed datetime must be in UTC, not {moment.tzinfo!r}")
+
+    def serialize(self) -> Tuple[str, Dict[str, Any]]:
+        return ("airflow.triggers.temporal.DateTimeTrigger", {"moment": self.moment})
+
+    async def run(self):
+        """
+        Simple time delay loop until the relevant time is met.
+
+        We do have a two-phase delay to save some cycles, but sleeping is so
+        cheap anyway that it's pretty loose.
+        """
+        # Sleep an hour at a time while it's more than 2 hours away
+        while timezone.utcnow() - self.moment > datetime.timedelta(hours=2):
+            await (asyncio.sleep(3600))
+        # Sleep a second at a time otherwise
+        while self.moment > timezone.utcnow():
+            await asyncio.sleep(1)

Review comment:
       Ah, could you add a comment, otherwise someone might come along and "optomize" this.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638314079



##########
File path: airflow/exceptions.py
##########
@@ -225,3 +226,30 @@ def __str__(self):
 
 class ConnectionNotUnique(AirflowException):
     """Raise when multiple values are found for the same conn_id"""
+
+
+class TaskDeferred(BaseException):
+    """
+    Special exception raised to signal that the operator it was raised from
+    wishes to defer until a trigger fires.
+    """
+
+    def __init__(
+        self,
+        *,
+        trigger,
+        method_name: str,
+        kwargs: Optional[Dict[str, Any]] = None,
+        timeout: Optional[datetime.timedelta] = None,
+    ):
+        super().__init__()
+        self.trigger = trigger
+        self.method_name = method_name
+        self.kwargs = kwargs
+        self.timeout = timeout
+        # Check timeout type at runtime
+        if self.timeout is not None and not hasattr(self.timeout, "total_seconds"):
+            raise ValueError("Timeout value must be a timedelta")

Review comment:
       Curious why not `isinstance(self.timeout, datetime.timedelta)`?

##########
File path: airflow/jobs/scheduler_job.py
##########
@@ -1899,3 +1910,17 @@ def adopt_or_reset_orphaned_tasks(self, session: Session = None):
                     raise
 
         return len(to_reset)
+
+    @provide_session
+    def check_trigger_timeouts(self, session: Session = None):
+        """
+        Looks at all tasks that are in the "deferred" state and whose trigger
+        timeout has passed, so they can be marked as failed.
+        """
+        timed_out_tasks = session.query(TaskInstance).filter(
+            TaskInstance.state == State.DEFERRED, TaskInstance.trigger_timeout < timezone.utcnow()
+        )
+        num_tasks = timed_out_tasks.count()

Review comment:
       Should we change this to use `func.count` ? We will have an extra line or two and some duplication but more reliable:
   
   https://docs.sqlalchemy.org/en/14/orm/query.html#sqlalchemy.orm.Query.count
   https://stackoverflow.com/questions/14754994/why-is-sqlalchemy-count-much-slower-than-the-raw-query




-- 
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:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638909975



##########
File path: airflow/api_connexion/endpoints/task_instance_endpoint.py
##########
@@ -248,7 +248,7 @@ def post_clear_task_instances(dag_id: str, session=None):
         error_message = f"Dag id {dag_id} not found"
         raise NotFound(error_message)
     reset_dag_runs = data.pop('reset_dag_runs')
-    task_instances = dag.clear(get_tis=True, **data)
+    task_instances = dag.clear(get_ti_instances=True, **data)

Review comment:
       https://github.com/apache/airflow/pull/15389#discussion_r638671367




-- 
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:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r639690106



##########
File path: tests/triggers/test_temporal.py
##########
@@ -0,0 +1,84 @@
+# 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 asyncio
+import datetime
+import sys
+
+import pendulum
+import pytest
+
+from airflow.triggers.base import TriggerEvent
+from airflow.triggers.temporal import DateTimeTrigger, TimeDeltaTrigger
+from airflow.utils import timezone
+from airflow.utils.asyncio import create_task

Review comment:
       This module should probably be renamed (the new since this PR was first opened) `airflow.compat`
   ```suggestion
   from airflow.compat.asyncio import create_task
   ```




-- 
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:
users@infra.apache.org



[GitHub] [airflow] xinbinhuang commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638220428



##########
File path: airflow/api_connexion/endpoints/task_instance_endpoint.py
##########
@@ -248,7 +248,7 @@ def post_clear_task_instances(dag_id: str, session=None):
         error_message = f"Dag id {dag_id} not found"
         raise NotFound(error_message)
     reset_dag_runs = data.pop('reset_dag_runs')
-    task_instances = dag.clear(get_tis=True, **data)

Review comment:
       I believe `tis` stands for `task instances`, so maybe `get_task_instances` to make it more explicit? 
   
   Another problem, `ti` is also a "public" variable as it's used in xcom, task context, macros etc. Though in general, I prefer explicit over abbreviation, I don't think it's good change to include within this PR.
   
   The same applies to the rest of `ti_instances`




-- 
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:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#issuecomment-834684185


   [The Workflow run](https://github.com/apache/airflow/actions/runs/820974368) is cancelling this PR. Building image for the PR has been cancelled


-- 
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:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638917089



##########
File path: airflow/exceptions.py
##########
@@ -225,3 +226,30 @@ def __str__(self):
 
 class ConnectionNotUnique(AirflowException):
     """Raise when multiple values are found for the same conn_id"""
+
+
+class TaskDeferred(BaseException):
+    """
+    Special exception raised to signal that the operator it was raised from
+    wishes to defer until a trigger fires.
+    """
+
+    def __init__(
+        self,
+        *,
+        trigger,
+        method_name: str,
+        kwargs: Optional[Dict[str, Any]] = None,
+        timeout: Optional[datetime.timedelta] = None,
+    ):
+        super().__init__()
+        self.trigger = trigger
+        self.method_name = method_name
+        self.kwargs = kwargs
+        self.timeout = timeout
+        # Check timeout type at runtime
+        if self.timeout is not None and not hasattr(self.timeout, "total_seconds"):
+            raise ValueError("Timeout value must be a timedelta")

Review comment:
       Should be fine -- was just curious




-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r685281397



##########
File path: airflow/triggers/testing.py
##########
@@ -0,0 +1,52 @@
+# 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 typing import Any, Dict, Tuple
+
+from airflow.triggers.base import BaseTrigger, TriggerEvent
+
+
+class SuccessTrigger(BaseTrigger):
+    """
+    A trigger that always succeeds immediately.
+
+    Should only be used for testing.

Review comment:
       I don't believe so, since they are also for DAG testing by end-users.




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r639924906



##########
File path: airflow/models/dag.py
##########
@@ -1223,7 +1244,7 @@ def clear(
                     include_subdags=include_subdags,
                     include_parentdag=False,
                     dag_run_state=dag_run_state,
-                    get_tis=True,
+                    get_ti_keys=True,

Review comment:
       Yup, the JSON column meant the UNIONs no longer worked, since UNION compares every single field in the query for equality.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#issuecomment-890436189


   They won't work quite as expected since they'll try to defer mid-run and that can't work in the test mode, but they should not explode like that. I'll take a look and see if I can replicate the behaviour next week - looks like the `execution_date` is `None` in the context, which is odd.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r685316507



##########
File path: airflow/jobs/scheduler_job.py
##########
@@ -699,6 +699,11 @@ def _run_scheduler_loop(self) -> None:
             self.adopt_or_reset_orphaned_tasks,
         )
 
+        timers.call_regular_interval(
+            conf.getfloat('scheduler', 'trigger_timeout_check_interval', fallback=15.0),

Review comment:
       I moved it into the config YAML, but kept the fallback here so it matches the other conf calls above it.




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] xinbinhuang commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638420317



##########
File path: airflow/api_connexion/endpoints/task_instance_endpoint.py
##########
@@ -248,7 +248,7 @@ def post_clear_task_instances(dag_id: str, session=None):
         error_message = f"Dag id {dag_id} not found"
         raise NotFound(error_message)
     reset_dag_runs = data.pop('reset_dag_runs')
-    task_instances = dag.clear(get_tis=True, **data)
+    task_instances = dag.clear(get_ti_instances=True, **data)

Review comment:
       Ah I see. I think these are some good reasons to move it to `get_ti_instances`, but I would still prefer to keep `get_tis` and add the `get_ti_keys`. 
   
   > because they are actually TaskInstance (model) instances.
   
   - Though it makes sense, I think it's a bit unconventional as normally people just reference the instances as lowercase to the model class name (i.e. `class User(Base)` -> `user = User(...)` ) as similar to an instance(object) of a class. 
   - Another concern from me is that people in the future may feel confused about the naming when they come across this.
   
   >  different to the old name (so I could make sure I had caught all references)
   
   Would this cause breaking changes as `dag.clear()` is a public API? Though I doubt people actually use `dag.clear()` in writing the DAG, the method may still be used in testing. In this case, we still need to keep the old keyword with deprecation warnings.
   
   WDYT? I am also curious about other people's opinions.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638920862



##########
File path: airflow/jobs/scheduler_job.py
##########
@@ -1899,3 +1910,17 @@ def adopt_or_reset_orphaned_tasks(self, session: Session = None):
                     raise
 
         return len(to_reset)
+
+    @provide_session
+    def check_trigger_timeouts(self, session: Session = None):
+        """
+        Looks at all tasks that are in the "deferred" state and whose trigger
+        timeout has passed, so they can be marked as failed.
+        """
+        timed_out_tasks = session.query(TaskInstance).filter(
+            TaskInstance.state == State.DEFERRED, TaskInstance.trigger_timeout < timezone.utcnow()
+        )
+        num_tasks = timed_out_tasks.count()

Review comment:
       ```suggestion
           num_tasks = session.query(func.count(TaskInstance.task_id)).filter(
               TaskInstance.state == State.DEFERRED, TaskInstance.trigger_timeout < timezone.utcnow()
           )
   ```




-- 
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:
users@infra.apache.org



[GitHub] [airflow] xinbinhuang commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638420317



##########
File path: airflow/api_connexion/endpoints/task_instance_endpoint.py
##########
@@ -248,7 +248,7 @@ def post_clear_task_instances(dag_id: str, session=None):
         error_message = f"Dag id {dag_id} not found"
         raise NotFound(error_message)
     reset_dag_runs = data.pop('reset_dag_runs')
-    task_instances = dag.clear(get_tis=True, **data)
+    task_instances = dag.clear(get_ti_instances=True, **data)

Review comment:
       Ah I see. I think these are some good reasons to move it to `get_ti_instances`, but I still prefer to keep `get_tis` and add the `get_ti_keys`. 
   
   > because they are actually TaskInstance (model) instances.
   
   - Though it makes sense, I think it's a bit unconventional as normally people just reference the instances as lowercase to the model class name (i.e. `class User(Base)` -> `user = User(...)` ) as similar to an instance(object) of a class. 
   - Another concern from me is that people in the future may feel confused about the naming when they come across this.
   
   >  different to the old name (so I could make sure I had caught all references)
   
   The `dag.clear()` method is a public API. Would this cause breaking changes? Though I doubt people actually use `dag.clear()` in writing the DAG, the method may still be used in testing. In this case, we still need to keep the old keyword and add deprecation warnings.
   
   WDYT? I am also curious about other people's opinions.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r678476724



##########
File path: airflow/models/trigger.py
##########
@@ -0,0 +1,195 @@
+# 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 datetime
+from typing import Any, Dict, List, Optional
+
+from sqlalchemy import BigInteger, Column, String, func
+
+from airflow.models.base import Base
+from airflow.models.taskinstance import TaskInstance
+from airflow.triggers.base import BaseTrigger
+from airflow.utils import timezone
+from airflow.utils.session import provide_session
+from airflow.utils.sqlalchemy import ExtendedJSON, UtcDateTime
+from airflow.utils.state import State
+
+
+class Trigger(Base):
+    """
+    Triggers are a workload that run in an asynchronous event loop shared with
+    other Triggers, and fire off events that will unpause deferred Tasks,
+    start linked DAGs, etc.
+
+    They are persisted into the database and then re-hydrated into a single
+    "triggerer" process, where they're all run at once. We model it so that
+    there is a many-to-one relationship between Task and Trigger, for future
+    deduplication logic to use.
+
+    Rows will be evicted from the database when the triggerer detects no
+    active Tasks/DAGs using them. Events are not stored in the database;
+    when an Event is fired, the triggerer will directly push its data to the
+    appropriate Task/DAG.
+    """
+
+    __tablename__ = "trigger"
+
+    id = Column(BigInteger, primary_key=True)
+    classpath = Column(String(1000), nullable=False)
+    kwargs = Column(ExtendedJSON, nullable=False)
+    created_date = Column(UtcDateTime, nullable=False)
+    triggerer_id = Column(BigInteger, nullable=True)
+
+    def __init__(
+        self, classpath: str, kwargs: Dict[str, Any], created_date: Optional[datetime.datetime] = None
+    ):
+        super().__init__()
+        self.classpath = classpath
+        self.kwargs = kwargs
+        self.created_date = created_date or timezone.utcnow()
+
+    @classmethod
+    def from_object(cls, trigger: BaseTrigger):
+        """
+        Alternative constructor that creates a trigger row based directly
+        off of a Trigger object.
+        """
+        classpath, kwargs = trigger.serialize()
+        return cls(classpath=classpath, kwargs=kwargs)
+
+    @classmethod
+    @provide_session
+    def bulk_fetch(cls, ids: List[int], session=None) -> Dict[int, "Trigger"]:
+        """
+        Fetches all of the Triggers by ID and returns a dict mapping
+        ID -> Trigger instance
+        """
+        return {obj.id: obj for obj in session.query(cls).filter(cls.id.in_(ids)).all()}
+
+    @classmethod
+    @provide_session
+    def clean_unused(cls, session=None):
+        """
+        Deletes all triggers that have no tasks/DAGs dependent on them
+        (triggers have a one-to-many relationship to both)
+        """
+        # Update all task instances with trigger IDs that are not DEFERRED to remove them
+        session.query(TaskInstance).filter(
+            TaskInstance.state != State.DEFERRED, TaskInstance.trigger_id.isnot(None)
+        ).update({TaskInstance.trigger_id: None})
+        # Get all triggers that have no task instances depending on them...
+        ids = [
+            trigger_id
+            for (trigger_id,) in (
+                session.query(cls.id)
+                .join(TaskInstance, cls.id == TaskInstance.trigger_id, isouter=True)
+                .group_by(cls.id)
+                .having(func.count(TaskInstance.trigger_id) == 0)
+            )
+        ]
+        # ...and delete them (we can't do this in one query due to MySQL)
+        session.query(Trigger).filter(Trigger.id.in_(ids)).delete(synchronize_session=False)
+
+    @classmethod
+    @provide_session
+    def submit_event(cls, trigger_id, event, session=None):
+        """
+        Takes an event from an instance of itself, and triggers all dependent
+        tasks to resume.
+        """
+        for task_instance in session.query(TaskInstance).filter(
+            TaskInstance.trigger_id == trigger_id, TaskInstance.state == State.DEFERRED
+        ):
+            # Add the event's payload into the kwargs for the task
+            next_kwargs = task_instance.next_kwargs or {}
+            next_kwargs["event"] = event.payload
+            task_instance.next_kwargs = next_kwargs
+            # Remove ourselves as its trigger
+            task_instance.trigger_id = None
+            # Finally, mark it as scheduled so it gets re-queued
+            task_instance.state = State.SCHEDULED
+
+    @classmethod
+    @provide_session
+    def submit_failure(cls, trigger_id, session=None):
+        """
+        Called when a trigger has failed unexpectedly, and we need to mark
+        everything that depended on it as failed. Notably, we have to actually
+        run the failure code from a worker as it may have linked callbacks, so
+        hilariously we have to re-schedule the task instances to a worker just
+        so they can then fail.
+
+        We use a special __fail__ value for next_method to achieve this that
+        the runtime code understands as immediate-fail, and pack the error into
+        next_kwargs.
+
+        TODO: Once we have shifted callback (and email) handling to run on
+        workers as first-class concepts, we can run the failure code here
+        in-process, but we can't do that right now.
+        """
+        for task_instance in session.query(TaskInstance).filter(
+            TaskInstance.trigger_id == trigger_id, TaskInstance.state == State.DEFERRED
+        ):
+            # Add the error and set the next_method to the fail state
+            task_instance.next_method = "__fail__"
+            task_instance.next_kwargs = {"error": "Trigger failure"}
+            # Remove ourselves as its trigger
+            task_instance.trigger_id = None
+            # Finally, mark it as scheduled so it gets re-queued
+            task_instance.state = State.SCHEDULED
+
+    @classmethod
+    @provide_session
+    def ids_for_triggerer(cls, triggerer_id, session=None):
+        """Retrieves a list of triggerer_ids."""
+        return [row[0] for row in session.query(cls.id).filter(cls.triggerer_id == triggerer_id)]
+
+    @classmethod
+    @provide_session
+    def assign_unassigned(cls, triggerer_id, capacity, session=None):
+        """
+        Takes a triggerer_id and the capacity for that triggerer and assigns unassigned
+        triggers until that capacity is reached, or there are no more unassigned triggers.
+        """
+        from airflow.jobs.base_job import BaseJob  # To avoid circular import
+
+        count = session.query(cls.id).filter(cls.triggerer_id == triggerer_id).count()
+        capacity -= count
+
+        if capacity <= 0:
+            return
+
+        alive_triggerer_ids = [
+            row[0]
+            for row in session.query(BaseJob.id).filter(
+                BaseJob.end_date is None,
+                BaseJob.latest_heartbeat > timezone.utcnow() - datetime.timedelta(seconds=30),
+                BaseJob.job_type == "TriggererJob",
+            )
+        ]
+
+        # find triggers which have a triggerer_id not in list
+        trigger_ids = [
+            row[0]
+            for row in session.query(cls.id)
+            .filter(cls.triggerer_id.notin_(alive_triggerer_ids))
+            .limit(capacity)
+        ]
+
+        session.query(cls).filter(cls.id.in_(trigger_ids)).update(
+            {cls.triggerer_id: triggerer_id},
+            synchronize_session=False,
+        )

Review comment:
       Reformulated it into a subquery - will see if MySQL likes it, as it's the pickiest SQL engine for those.




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r685316016



##########
File path: airflow/triggers/__init__.py
##########
@@ -0,0 +1,19 @@
+# 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 airflow.triggers.base import BaseTrigger  # noqa
+from airflow.triggers.temporal import DateTimeTrigger, TimeDeltaTrigger  # noqa

Review comment:
       Good catch - want to match the other convention here so I pulled them out.




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r681735890



##########
File path: airflow/jobs/local_task_job.py
##########
@@ -241,7 +241,6 @@ def _run_mini_scheduler_on_child_tasks(self, session=None) -> None:
                 include_upstream=False,
                 include_direct_upstream=True,
             )
-

Review comment:
       Could you revert this please so that we don't touch this file




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r639695673



##########
File path: airflow/models/dag.py
##########
@@ -1223,7 +1244,7 @@ def clear(
                     include_subdags=include_subdags,
                     include_parentdag=False,
                     dag_run_state=dag_run_state,
-                    get_tis=True,
+                    get_ti_keys=True,

Review comment:
       Oh, this was caused by adding the JSON column to TI right?




-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638976101



##########
File path: airflow/models/baseoperator.py
##########
@@ -1534,6 +1535,23 @@ def inherits_from_dummy_operator(self):
         # of its sub-classes (which don't inherit from anything but BaseOperator).
         return getattr(self, '_is_dummy', False)
 
+    def defer(
+        self,
+        *,
+        trigger: BaseTrigger,
+        method_name: str,
+        kwargs: Optional[Dict[str, Any]] = None,
+        timeout: Optional[timedelta] = None,
+    ):
+        """
+        Marks this Operator as being "deferred" - that is, suspending its
+        execution until the provided trigger fires an event.
+
+        This is achieved by raising a special exception (OperatorDeferred)
+        which is caught in the main _execute_task wrapper.
+        """
+        raise TaskDeferred(trigger=trigger, method_name=method_name, kwargs=kwargs, timeout=timeout)

Review comment:
       I opted against that because I wanted it to be possible to run everything else on 3.6 and the triggerer on 3.7.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r639631128



##########
File path: airflow/models/baseoperator.py
##########
@@ -1534,6 +1535,23 @@ def inherits_from_dummy_operator(self):
         # of its sub-classes (which don't inherit from anything but BaseOperator).
         return getattr(self, '_is_dummy', False)
 
+    def defer(
+        self,
+        *,
+        trigger: BaseTrigger,
+        method_name: str,
+        kwargs: Optional[Dict[str, Any]] = None,
+        timeout: Optional[timedelta] = None,
+    ):
+        """
+        Marks this Operator as being "deferred" - that is, suspending its
+        execution until the provided trigger fires an event.
+
+        This is achieved by raising a special exception (OperatorDeferred)
+        which is caught in the main _execute_task wrapper.
+        """
+        raise TaskDeferred(trigger=trigger, method_name=method_name, kwargs=kwargs, timeout=timeout)

Review comment:
       👍🏻 




-- 
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:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#issuecomment-833100827


   [The Workflow run](https://github.com/apache/airflow/actions/runs/815010545) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


-- 
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:
users@infra.apache.org



[GitHub] [airflow] jedcunningham commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r684460751



##########
File path: airflow/jobs/scheduler_job.py
##########
@@ -1207,3 +1218,26 @@ def adopt_or_reset_orphaned_tasks(self, session: Session = None):
                     raise
 
         return len(to_reset)
+
+    @provide_session
+    def check_trigger_timeouts(self, session: Session = None):
+        """
+        Looks at all tasks that are in the "deferred" state and whose trigger
+        or execution timeout has passed, so they can be marked as failed.
+        """
+        num_timed_out_tasks = (
+            session.query(TaskInstance)
+            .filter(TaskInstance.state == State.DEFERRED, TaskInstance.trigger_timeout < timezone.utcnow())
+            .update(
+                # We have to schedule these to fail themselves so it doesn't
+                # happen inside the scheduler.
+                {
+                    "state": State.SCHEDULED,
+                    "next_method": "__fail__",
+                    "next_kwargs": {"error": "Trigger/execution timeout"},
+                    "trigger_id": None,
+                }
+            )
+        )
+        if num_timed_out_tasks:
+            self.log.info("Timed out %i deferred tasks without fired triggers", num_timed_out_tasks)

Review comment:
       Metrics in general might be worthy of a followup, but if/when that happens this would be a good candidate.

##########
File path: airflow/triggers/__init__.py
##########
@@ -0,0 +1,19 @@
+# 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 airflow.triggers.base import BaseTrigger  # noqa
+from airflow.triggers.temporal import DateTimeTrigger, TimeDeltaTrigger  # noqa

Review comment:
       I don't think we should import these here. In 2.0, this was removed for operators and sensors, so this feels odd to me.

##########
File path: airflow/jobs/triggerer_job.py
##########
@@ -0,0 +1,379 @@
+# 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 asyncio
+import os
+import signal
+import sys
+import threading
+import time
+from collections import deque
+from typing import Deque, Dict, Set, Tuple, Type
+
+from airflow.compat.asyncio import create_task
+from airflow.jobs.base_job import BaseJob
+from airflow.models.trigger import Trigger
+from airflow.triggers.base import BaseTrigger, TriggerEvent
+from airflow.typing_compat import TypedDict
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.module_loading import import_string
+
+
+class TriggererJob(BaseJob):
+    """
+    TriggererJob continuously runs active triggers in asyncio, watching
+    for them to fire off their events and then dispatching that information
+    to their dependent tasks/DAGs.
+
+    It runs as two threads:
+     - The main thread does DB calls/checkins
+     - A subthread runs all the async code
+    """
+
+    __mapper_args__ = {'polymorphic_identity': 'TriggererJob'}
+
+    def __init__(self, capacity=None, *args, **kwargs):
+        # Call superclass
+        super().__init__(*args, **kwargs)
+
+        if capacity is None:
+            self.capacity = 1000  # TODO put this in a config file?

Review comment:
       Yeah, probably should be moved to the config, like the timeout interval.

##########
File path: airflow/triggers/testing.py
##########
@@ -0,0 +1,52 @@
+# 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 typing import Any, Dict, Tuple
+
+from airflow.triggers.base import BaseTrigger, TriggerEvent
+
+
+class SuccessTrigger(BaseTrigger):
+    """
+    A trigger that always succeeds immediately.
+
+    Should only be used for testing.

Review comment:
       (Should these live in `tests` then?)

##########
File path: tests/jobs/test_triggerer_job.py
##########
@@ -0,0 +1,327 @@
+#
+# 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 datetime
+import sys
+import time
+
+import pytest
+
+from airflow import DAG
+from airflow.jobs.triggerer_job import TriggererJob
+from airflow.models import Trigger
+from airflow.models.taskinstance import TaskInstance
+from airflow.operators.dummy import DummyOperator
+from airflow.triggers.base import TriggerEvent
+from airflow.triggers.temporal import TimeDeltaTrigger
+from airflow.triggers.testing import FailureTrigger, SuccessTrigger
+from airflow.utils import timezone
+from airflow.utils.session import create_session
+from airflow.utils.state import State, TaskInstanceState
+from tests.test_utils.db import clear_db_runs
+
+
+@pytest.fixture(autouse=True)
+def clean_database():
+    """Fixture that cleans the database before and after every test."""
+    clear_db_runs()
+    yield  # Test runs here
+    clear_db_runs()
+
+
+@pytest.fixture
+def session():
+    """Fixture that provides a SQLAlchemy session"""
+    with create_session() as session:
+        yield session
+
+
+@pytest.mark.skipif(sys.version_info.minor <= 6 and sys.version_info.major <= 3, reason="No triggerer on 3.6")
+def test_is_alive():
+    """Checks the heartbeat logic"""
+    # Current time
+    triggerer_job = TriggererJob(None, heartrate=10, state=State.RUNNING)
+    assert triggerer_job.is_alive()
+
+    # Slightly old, but still fresh
+    triggerer_job.latest_heartbeat = timezone.utcnow() - datetime.timedelta(seconds=20)
+    assert triggerer_job.is_alive()
+
+    # Old enough to fail
+    triggerer_job.latest_heartbeat = timezone.utcnow() - datetime.timedelta(seconds=31)
+    assert not triggerer_job.is_alive()
+
+    # Completed state should not be alive
+    triggerer_job.state = State.SUCCESS
+    triggerer_job.latest_heartbeat = timezone.utcnow() - datetime.timedelta(seconds=10)
+    assert not triggerer_job.is_alive(), "Completed jobs even with recent heartbeat should not be alive"
+
+
+@pytest.mark.skipif(sys.version_info.minor <= 6 and sys.version_info.major <= 3, reason="No triggerer on 3.6")
+def test_capacity_decode():
+    """
+    Tests that TriggererJob correctly sets capacity to a valid value passed in as a CLI arg,
+    handles invalid args, or sets it to a default value if no arg is passed.
+    """
+    # Positive cases
+    variants = [
+        42,
+        None,
+    ]
+    for input_str in variants:
+

Review comment:
       ```suggestion
   
   ```
   nit

##########
File path: airflow/triggers/temporal.py
##########
@@ -0,0 +1,76 @@
+# 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 asyncio
+import datetime
+from typing import Any, Dict, Tuple
+
+import pytz
+
+from airflow.triggers.base import BaseTrigger, TriggerEvent
+from airflow.utils import timezone
+
+
+class DateTimeTrigger(BaseTrigger):
+    """
+    A trigger that fires exactly once, at the given datetime, give or take
+    a few seconds.
+
+    The provided datetime MUST be in UTC.
+    """
+
+    def __init__(self, moment: datetime.datetime):
+        super().__init__()
+        # Make sure it's in UTC
+        if moment.tzinfo is None:
+            self.moment = pytz.utc.localize(moment)
+        elif moment.tzinfo == pytz.utc or getattr(moment.tzinfo, "name", None) == "UTC":
+            self.moment = moment
+        else:
+            raise ValueError(f"The passed datetime must be in UTC, not {moment.tzinfo!r}")
+
+    def serialize(self) -> Tuple[str, Dict[str, Any]]:
+        return ("airflow.triggers.temporal.DateTimeTrigger", {"moment": self.moment})
+
+    async def run(self):
+        """
+        Simple time delay loop until the relevant time is met.
+
+        We do have a two-phase delay to save some cycles, but sleeping is so
+        cheap anyway that it's pretty loose.
+        """
+        # Sleep an hour at a time while it's more than 2 hours away
+        while timezone.utcnow() - self.moment > datetime.timedelta(hours=2):
+            await (asyncio.sleep(3600))
+        # Sleep a second at a time otherwise
+        while self.moment > timezone.utcnow():
+            await asyncio.sleep(1)

Review comment:
       Not sure a comment was actually added (and I had the same question).

##########
File path: docs/apache-airflow/concepts/deferring.rst
##########
@@ -0,0 +1,163 @@
+ .. 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.
+
+Deferrable Operators & Triggers
+===============================
+
+Standard :doc:`Operators <operators>` and :doc:`Sensors <sensors>` take up a full *worker slot* for the entire time they are running, even if they are idle; for example, if you only have 100 worker slots available to run Tasks, and you have 100 DAGs waiting on a Sensor that's currently running but idle, then you *cannot run anything else* - even though your entire Airflow cluster is essentially idle. ``reschedule`` mode for Sensors solves some of this, allowing Sensors to only run at fixed intervals, but it is inflexible and only allows using time as the reason to resume, not anything else.
+
+This is where *Deferrable Operators* come in. A deferrable operator is one that is written with the ability to suspend itself and free up the worker when it knows it has to wait, and hand off the job of resuming it to something called a *Trigger*. As a result, while it is suspended (deferred), it is not taking up a worker slot and your cluster will have a lot less resources wasted on idle Operators or Sensors.
+
+*Triggers* are small, asynchronous pieces of Python code designed to be run all together in a single Python process; because they are asynchronous, they are able to all co-exist efficiently. As an overview of how this process works:
+
+* A task instance (running operator) gets to a point where it has to wait, and defers itself with a trigger tied to the event that should resume it. This frees up the worker to run something else.
+* The new Trigger instance is registered inside Airflow, and picked up by a *triggerer* process
+* The trigger is run until it fires, at which point its source task is re-scheduled
+* The scheduler queues the task to resume on a worker node
+
+Using deferrable operators as a DAG author is almost transparent; writing them, however, takes a bit more work.
+
+.. note::
+
+    Deferrable Operators & Triggers rely on more recent ``asyncio`` features, and as a result only work
+    on Python 3.7 or higher.
+
+
+Using Deferrable Operators
+--------------------------
+
+If all you wish to do is use pre-written Deferrable Operators (such as ``TimeSensorAsync``, which comes with Airflow), then there are only two steps you need:
+
+* Ensure your Airflow installation is running at least one ``triggerer`` process, as well as the normal ``scheduler``
+* Use deferrable operators/sensors in your DAGs
+
+That's it; everything else will be automatically handled for you. If you're upgrading existing DAGs, we even provide some API-compatible sensor variants (e.g. ``TimeSensorAsync`` for ``TimeSensor``) that you can swap into your DAG with no other changes required.
+
+Note that you cannot yet use the deferral ability from inside custom PythonOperator/TaskFlow Python functions; it is only available to traditional, class-based Operators at the moment.
+
+
+Writing Deferrable Operators
+----------------------------
+
+Writing a deferrable operator takes a bit more work. There are some main points to consider:
+
+* Your Operator must defer itself based on a Trigger. If there is a Trigger in core Airflow you can use, great; otherwise, you will have to write one.
+* Your Operator will be stopped and removed from its worker while deferred, and no state will persist automatically. You can persist state by asking Airflow to resume you at a certain method or pass certain kwargs, but that's it.
+* You can defer multiple times, and you can defer before/after your Operator does significant work, or only defer if certain conditions are met (e.g. a system does not have an immediate answer). Deferral is entirely under your control.
+* Any Operator can defer; no special marking on its class is needed, and it's not limited to Sensors.
+
+
+Triggering Deferral
+~~~~~~~~~~~~~~~~~~~
+
+If you want to trigger deferral, at any place in your Operator you can call ``self.defer(trigger, method_name, kwargs, timeout)``, which will raise a special exception that Airflow will catch. The arguments are:
+
+* ``trigger``: An instance of a Trigger that you wish to defer on. It will be serialized into the database.
+* ``method_name``: The method name on your Operator you want Airflow to call when it resumes, other than ``execute``.
+* ``kwargs``: Additional keyword arguments to pass to the method when it is called. Optional, defaults to ``{}``.
+* ``timeout``: A timedelta that specifies a timeout after which this deferral will fail, and fail the task instance. Optional, defaults to ``None``, meaning no timeout.
+
+When you opt to defer, your Operator will *stop executing at that point and be removed from its current worker*. No state - such as local variables, or attributes set on ``self`` - will persist, and when your Operator is resumed it will be a *brand new instance* of it. The only way you can pass state from the old instance of the Operator to the new one is via ``method_name`` and ``kwargs``.
+
+When your Operator is resumed, you will find an ``event`` item added to the kwargs passed to it, which contains the payload from the trigger event that resumed your Operator. Depending on the trigger, this may be useful to your operator (e.g. it's a status code or URL to fetch results), or it may not be important (it's just a datetime). Your ``method_name`` method, however, *must* accept ``event`` as a keyword argument.
+
+If your Operator returns from either its first ``execute()`` method when it's new, or a subsequent method specified by ``method_name``, it will be considered complete and will finish executing.
+
+You are free to set ``method_name`` to ``execute`` if you want your Operator to have one entrypoint, but it, too, will have to accept ``event`` as an optional keyword argument.
+
+Here's a basic example of how a sensor might trigger deferral::
+
+    class WaitOneHourSensor(BaseSensorOperator):
+        def execute(self, context):
+            self.defer(trigger=TimeDeltaTrigger(timedelta(hours=1), method_name="execute_complete")
+
+        def execute_complete(self, context, event=None):
+            # We have no more work to do here. Mark as complete.
+            return
+
+This Sensor is literally just a thin wrapper around the Trigger, so all it does is defer to the trigger, and specify a different method to come back to when the trigger fires - which, as it returns immediately, marks the Sensor as successful.
+
+Under the hood, ``self.defer`` raises the ``TaskDeferred`` exception, so it will work anywhere inside your Operator's code, even buried many nested calls deep inside ``execute()``. You are free to raise ``TaskDeferred`` manually if you wish; it takes the same arguments as ``self.defer``.
+
+Note that ``execution_timeout`` on Operators is considered over the *total runtime*, not individual executions in-between deferrals - this means that if ``execution_timeout`` is set, an Operator may fail while it's deferred or while it's running after a deferral, even if it's only been resumed for a few seconds.
+
+
+Writing Triggers
+~~~~~~~~~~~~~~~~
+
+A Trigger is written as a class that inherits from ``BaseTrigger``, and implements three methods:
+
+* ``__init__``, to receive arguments from Operators instantiating it
+* ``run``, an asynchronous method that runs its logic and yields one or more ``TriggerEvent`` instances as an asynchronous generator
+* ``serialize``, which returns the information needed to re-construct this trigger, as a tuple of the classpath, and keyword arguments to pass to ``__init__``
+
+There's also some design constraints to be aware of:
+
+* The ``run`` method *must be asynchronous* (using Python's asyncio), and correctly ``await`` whenever it does a blocking operation.
+* ``run`` must ``yield`` its TriggerEvents, not return them. If it returns before yielding at least once event, Airflow will consider this an error and fail any Task Instances waiting on it. If it throws an exception, Airflow will also fail any dependent task instances.

Review comment:
       ```suggestion
   * ``run`` must ``yield`` its TriggerEvents, not return them. If it returns before yielding at least one event, Airflow will consider this an error and fail any Task Instances waiting on it. If it throws an exception, Airflow will also fail any dependent task instances.
   ```

##########
File path: airflow/jobs/scheduler_job.py
##########
@@ -699,6 +699,11 @@ def _run_scheduler_loop(self) -> None:
             self.adopt_or_reset_orphaned_tasks,
         )
 
+        timers.call_regular_interval(
+            conf.getfloat('scheduler', 'trigger_timeout_check_interval', fallback=15.0),

Review comment:
       ```suggestion
               conf.getfloat('scheduler', 'trigger_timeout_check_interval'),
   ```
   
   This default should be moved into `airflow.config_templates/config.yml` so it gets properly documented everywhere. You then won't need the fallback here because it'll use the value from `default_airflow.cfg`.

##########
File path: airflow/sensors/time_sensor.py
##########
@@ -35,3 +36,23 @@ def __init__(self, *, target_time, **kwargs):
     def poke(self, context):
         self.log.info('Checking if the time (%s) has come', self.target_time)
         return timezone.make_naive(timezone.utcnow(), self.dag.timezone).time() > self.target_time
+
+
+class TimeSensorAsync(BaseSensorOperator):
+    """
+    Waits until the specified time of the day, freeing up a worker slot while
+    it is waiting.
+
+    :param target_time: time after which the job succeeds
+    :type target_time: datetime.time
+    """
+
+    def __init__(self, *, target_time, **kwargs):
+        super().__init__(**kwargs)
+        self.target_time = target_time
+
+    def execute(self, context):
+        self.defer(trigger=DateTimeTrigger(moment=self.target_time), method_name="execute_complete")
+
+    def execute_complete(self, context, event=None):  # pylint: disable=unused-argument
+        """Callback for when the trigger fires - returns immediately."""

Review comment:
       nit: This is unlike the others in that it doesn't explicitly return None.

##########
File path: docs/apache-airflow/concepts/deferring.rst
##########
@@ -0,0 +1,163 @@
+ .. 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.
+
+Deferrable Operators & Triggers
+===============================
+
+Standard :doc:`Operators <operators>` and :doc:`Sensors <sensors>` take up a full *worker slot* for the entire time they are running, even if they are idle; for example, if you only have 100 worker slots available to run Tasks, and you have 100 DAGs waiting on a Sensor that's currently running but idle, then you *cannot run anything else* - even though your entire Airflow cluster is essentially idle. ``reschedule`` mode for Sensors solves some of this, allowing Sensors to only run at fixed intervals, but it is inflexible and only allows using time as the reason to resume, not anything else.
+
+This is where *Deferrable Operators* come in. A deferrable operator is one that is written with the ability to suspend itself and free up the worker when it knows it has to wait, and hand off the job of resuming it to something called a *Trigger*. As a result, while it is suspended (deferred), it is not taking up a worker slot and your cluster will have a lot less resources wasted on idle Operators or Sensors.
+
+*Triggers* are small, asynchronous pieces of Python code designed to be run all together in a single Python process; because they are asynchronous, they are able to all co-exist efficiently. As an overview of how this process works:
+
+* A task instance (running operator) gets to a point where it has to wait, and defers itself with a trigger tied to the event that should resume it. This frees up the worker to run something else.
+* The new Trigger instance is registered inside Airflow, and picked up by a *triggerer* process
+* The trigger is run until it fires, at which point its source task is re-scheduled
+* The scheduler queues the task to resume on a worker node
+
+Using deferrable operators as a DAG author is almost transparent; writing them, however, takes a bit more work.
+
+.. note::
+
+    Deferrable Operators & Triggers rely on more recent ``asyncio`` features, and as a result only work
+    on Python 3.7 or higher.
+
+
+Using Deferrable Operators
+--------------------------
+
+If all you wish to do is use pre-written Deferrable Operators (such as ``TimeSensorAsync``, which comes with Airflow), then there are only two steps you need:
+
+* Ensure your Airflow installation is running at least one ``triggerer`` process, as well as the normal ``scheduler``
+* Use deferrable operators/sensors in your DAGs
+
+That's it; everything else will be automatically handled for you. If you're upgrading existing DAGs, we even provide some API-compatible sensor variants (e.g. ``TimeSensorAsync`` for ``TimeSensor``) that you can swap into your DAG with no other changes required.
+
+Note that you cannot yet use the deferral ability from inside custom PythonOperator/TaskFlow Python functions; it is only available to traditional, class-based Operators at the moment.
+
+
+Writing Deferrable Operators
+----------------------------
+
+Writing a deferrable operator takes a bit more work. There are some main points to consider:
+
+* Your Operator must defer itself based on a Trigger. If there is a Trigger in core Airflow you can use, great; otherwise, you will have to write one.
+* Your Operator will be stopped and removed from its worker while deferred, and no state will persist automatically. You can persist state by asking Airflow to resume you at a certain method or pass certain kwargs, but that's it.
+* You can defer multiple times, and you can defer before/after your Operator does significant work, or only defer if certain conditions are met (e.g. a system does not have an immediate answer). Deferral is entirely under your control.
+* Any Operator can defer; no special marking on its class is needed, and it's not limited to Sensors.
+
+
+Triggering Deferral
+~~~~~~~~~~~~~~~~~~~
+
+If you want to trigger deferral, at any place in your Operator you can call ``self.defer(trigger, method_name, kwargs, timeout)``, which will raise a special exception that Airflow will catch. The arguments are:
+
+* ``trigger``: An instance of a Trigger that you wish to defer on. It will be serialized into the database.
+* ``method_name``: The method name on your Operator you want Airflow to call when it resumes, other than ``execute``.
+* ``kwargs``: Additional keyword arguments to pass to the method when it is called. Optional, defaults to ``{}``.
+* ``timeout``: A timedelta that specifies a timeout after which this deferral will fail, and fail the task instance. Optional, defaults to ``None``, meaning no timeout.
+
+When you opt to defer, your Operator will *stop executing at that point and be removed from its current worker*. No state - such as local variables, or attributes set on ``self`` - will persist, and when your Operator is resumed it will be a *brand new instance* of it. The only way you can pass state from the old instance of the Operator to the new one is via ``method_name`` and ``kwargs``.
+
+When your Operator is resumed, you will find an ``event`` item added to the kwargs passed to it, which contains the payload from the trigger event that resumed your Operator. Depending on the trigger, this may be useful to your operator (e.g. it's a status code or URL to fetch results), or it may not be important (it's just a datetime). Your ``method_name`` method, however, *must* accept ``event`` as a keyword argument.
+
+If your Operator returns from either its first ``execute()`` method when it's new, or a subsequent method specified by ``method_name``, it will be considered complete and will finish executing.
+
+You are free to set ``method_name`` to ``execute`` if you want your Operator to have one entrypoint, but it, too, will have to accept ``event`` as an optional keyword argument.
+
+Here's a basic example of how a sensor might trigger deferral::
+
+    class WaitOneHourSensor(BaseSensorOperator):
+        def execute(self, context):
+            self.defer(trigger=TimeDeltaTrigger(timedelta(hours=1), method_name="execute_complete")
+
+        def execute_complete(self, context, event=None):
+            # We have no more work to do here. Mark as complete.
+            return
+
+This Sensor is literally just a thin wrapper around the Trigger, so all it does is defer to the trigger, and specify a different method to come back to when the trigger fires - which, as it returns immediately, marks the Sensor as successful.
+
+Under the hood, ``self.defer`` raises the ``TaskDeferred`` exception, so it will work anywhere inside your Operator's code, even buried many nested calls deep inside ``execute()``. You are free to raise ``TaskDeferred`` manually if you wish; it takes the same arguments as ``self.defer``.
+
+Note that ``execution_timeout`` on Operators is considered over the *total runtime*, not individual executions in-between deferrals - this means that if ``execution_timeout`` is set, an Operator may fail while it's deferred or while it's running after a deferral, even if it's only been resumed for a few seconds.
+
+
+Writing Triggers
+~~~~~~~~~~~~~~~~
+
+A Trigger is written as a class that inherits from ``BaseTrigger``, and implements three methods:
+
+* ``__init__``, to receive arguments from Operators instantiating it
+* ``run``, an asynchronous method that runs its logic and yields one or more ``TriggerEvent`` instances as an asynchronous generator
+* ``serialize``, which returns the information needed to re-construct this trigger, as a tuple of the classpath, and keyword arguments to pass to ``__init__``
+
+There's also some design constraints to be aware of:
+
+* The ``run`` method *must be asynchronous* (using Python's asyncio), and correctly ``await`` whenever it does a blocking operation.
+* ``run`` must ``yield`` its TriggerEvents, not return them. If it returns before yielding at least once event, Airflow will consider this an error and fail any Task Instances waiting on it. If it throws an exception, Airflow will also fail any dependent task instances.
+* A Trigger *must be able to run in parallel* with other copies of itself. This can happen both when two tasks defer based on the same trigger, and also if a network partition happens and Airflow re-launches a trigger on a separated machine.
+* When events are emitted, and if your trigger is designed to emit more than one event, they *must* contain a payload that can be used to deduplicate events if the trigger is being run in multiple places. If you only fire one event, and don't want to pass information in the payload back to the Operator that deferred, you can just set the payload to ``None``.
+* A trigger may be suddenly removed from one process and started on a new one (if partitions are being changed, or a deployment is happening). You may provide an optional ``cleanup`` method that gets called when this happens.
+
+Here's the structure of a basic Trigger::
+
+
+    class DateTimeTrigger(BaseTrigger):
+
+        def __init__(self, moment):
+            super().__init__()
+            self.moment = moment
+
+        def serialize(self):
+            return ("airflow.triggers.temporal.DateTimeTrigger", {"moment": self.moment})
+
+        async def run(self):
+            while self.moment > timezone.utcnow():
+                await asyncio.sleep(1)
+            yield TriggerEvent(self.moment)
+
+This is a very simplified version of Airflow's ``DateTimeTrigger``, and you can see several things here:
+
+* ``__init__`` and ``serialize`` are written as a pair; the Trigger is instantiated once when it is submitted by the Operator as part of its deferral request, then serialized and re-instantiated on any *triggerer* process that runs the trigger.
+* The ``run`` method is declared as an ``async def``, as it *must* be asynchronous, and uses ``asyncio.sleep`` rather than the regular ``time.sleep`` (as that would block the process).
+* When it emits its event it packs ``self.moment`` in there, so if this trigger is being run redundantly on multiple hosts, the event can be de-duplicated.
+
+Triggers can be as complex or as simple as you like provided you keep inside this contract; they are designed to be run in a highly-available fashion, auto-distributed among hosts running the *triggerer*. We encourage you to avoid any kind of persistent state in a trigger; they should get everything they need from their ``__init__``, so they can be serialized and moved around freely.
+
+If you are new to writing asynchronous Python, you should be very careful writing your ``run()`` method; Python's async model means that any code that does not correctly ``await`` when it does a blocking operation will block the *entire process*. Airflow will attempt to detect this and warn you in the triggerer logs when it happens, but we strongly suggest you set the variable ``PYTHONASYNCIODEBUG=1`` when you are writing your Trigger to enable extra checks from Python to make sure you're writing non-blocking code. Be especially careful when doing filesystem calls, as if the underlying filesystem is network-backed it may be blocking.
+
+Right now, Triggers are only used up to their first event, as they are only used for resuming deferred tasks (which happens on the first event fired). However, we plan to allow DAGs to be launched from triggers in future, which is where multi-event triggers will be more useful.
+
+
+High Availability
+-----------------
+
+Triggers are designed from the ground-up to be highly-available; if you want to run a highly-available setup, simply run multiple copies of ``triggerer`` on multiple hosts. Much like ``scheduler``, they will automatically co-exist with correct locking and HA.
+
+Depending on how much work the triggers are doing, you can fit from hundreds to tens of thousands of triggers on a single ``triggerer`` host. By default, every ``triggerer`` will have a capacity of 1000 triggers it will try to run at once; you can change this with the ``--capacity`` argument. If you have more triggers trying to run than you have capacity across all of your ``triggerer`` processes, some triggers will be delayed from running until others have completed.
+
+Airflow tries to only run triggers in one place at once, and maintains a heartbeat to all ``triggerers`` that are currently running. If a ``triggerer`` dies, or becomes partitioned from the network where Airflow's database is running, Airflow will automatically re-schedule triggers that were on that host to run elsewhere (after waiting 30 seconds for the machine to re-appear).
+
+This means it's possible, but unlikely, for triggers to run in multiple places at once; this is designed into the Trigger contract, however, and entirely expected. Airflow will de-duplicate events fired when a trigger is running in multiple places simultaneously, so this process should be transparent to your Operators.
+
+Note that every extra ``triggerer`` you run will result in an extra persistent connection to your database.
+
+
+Smart Sensors
+-------------
+
+Deferrable Operators essentially supersede :doc:`Smart Sensors <smart-sensors>`, and should be preferred for almost all situations. They do solve fundamentally the same problem; Smart Sensors, however, only work for certain Sensor workload styles, have no redundancy, and require a custom DAG to run at all times.

Review comment:
       Should we add a note on the smart sensors docs page pointing over here too?




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r640534454



##########
File path: airflow/models/taskinstance.py
##########
@@ -282,6 +296,18 @@ class TaskInstance(Base, LoggingMixin):  # pylint: disable=R0902,R0904
     executor_config = Column(PickleType(pickler=dill))
 
     external_executor_id = Column(String(ID_LEN, **COLLATION_ARGS))
+
+    # The trigger to resume on if we are in state DEFERRED
+    trigger_id = Column(BigInteger)
+
+    # Optional timeout datetime for the trigger (past this, we'll fail)
+    trigger_timeout = Column(UtcDateTime)
+
+    # The method to call next, and any extra arguments to pass to it.
+    # Usually used when resuming from DEFERRED.
+    next_method = Column(String(1000))

Review comment:
       Oh yes, likely is that. Something to do with UTF8 3 vs 4 byte chars and maximum size of a index column in MySQL. Not an issue 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638983627



##########
File path: airflow/triggers/temporal.py
##########
@@ -0,0 +1,76 @@
+# 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 asyncio
+import datetime
+from typing import Any, Dict, Tuple
+
+import pytz

Review comment:
       Oh, _that_ explains why this code didn't quite work as well as it should have. I'll rework this to remove any timezone library, it can just look at the offset values.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638979375



##########
File path: airflow/jobs/triggerer_job.py
##########
@@ -0,0 +1,418 @@
+# 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 asyncio
+import importlib
+import os
+import signal
+import sys
+import threading
+import time
+from collections import deque
+from typing import Deque, Dict, List, Optional, Set, Tuple, Type
+
+from airflow.jobs.base_job import BaseJob
+from airflow.models.trigger import Trigger
+from airflow.triggers.base import BaseTrigger, TriggerEvent
+from airflow.typing_compat import TypedDict
+from airflow.utils.asyncio import create_task
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class TriggererJob(BaseJob, LoggingMixin):
+    """
+    TriggererJob continuously runs active triggers in asyncio, watching
+    for them to fire off their events and then dispatching that information
+    to their dependent tasks/DAGs.
+
+    It runs as two threads:
+     - The main thread does DB calls/checkins
+     - A subthread runs all the async code
+    """
+
+    __mapper_args__ = {'polymorphic_identity': 'TriggererJob'}
+
+    partition_ids: Optional[List[int]] = None
+    partition_total: Optional[int] = None
+
+    def __init__(self, partition=None, *args, **kwargs):
+        # Make sure we can actually run
+        if not hasattr(asyncio, "create_task"):
+            raise RuntimeError("The triggerer/deferred operators only work on Python 3.7 and above.")
+        # Call superclass
+        super().__init__(*args, **kwargs)
+        # Decode partition information
+        self.partition_ids, self.partition_total = None, None
+        if partition:
+            self.partition_ids, self.partition_total = self.decode_partition(partition)
+        # Set up runner async thread
+        self.runner = TriggerRunner()
+
+    def decode_partition(self, partition: str) -> Tuple[List[int], int]:
+        """
+        Given a string-format partition specification, returns the list of
+        partition IDs it represents and the partition total.
+        """
+        try:
+            # The partition format is "1,2,3/10" where the numbers before
+            # the slash are the partitions we represent, and the number
+            # after is the total number. Most users will just have a single
+            # partition number, e.g. "2/10".
+            ids_str, total_str = partition.split("/", 1)
+            partition_total = int(total_str)
+            partition_ids = []
+            for id_str in ids_str.split(","):
+                id_number = int(id_str)
+                # Bounds checking (they're 1-indexed, which might catch people out)
+                if id_number <= 0 or id_number > self.partition_total:
+                    raise ValueError(f"Partition number {id_number} is impossible")
+                self.partition_ids.append(id_number)
+        except (ValueError, TypeError):
+            raise ValueError(f"Invalid partition specification: {partition}")
+        return partition_ids, partition_total
+
+    def register_signals(self) -> None:
+        """Register signals that stop child processes"""
+        signal.signal(signal.SIGINT, self._exit_gracefully)
+        signal.signal(signal.SIGTERM, self._exit_gracefully)
+
+    def _exit_gracefully(self, signum, frame) -> None:  # pylint: disable=unused-argument
+        """Helper method to clean up processor_agent to avoid leaving orphan processes."""
+        # The first time, try to exit nicely
+        if not self.runner.stop:
+            self.log.info("Exiting gracefully upon receiving signal %s", signum)
+            self.runner.stop = True
+        else:
+            self.log.warning("Forcing exit due to second exit signal %s", signum)
+            sys.exit(os.EX_SOFTWARE)
+
+    def _execute(self) -> None:
+        # Display custom startup ack depending on plurality of partitions
+        if self.partition_ids is None:
+            self.log.info("Starting the triggerer")
+        elif len(self.partition_ids) == 1:
+            self.log.info(
+                "Starting the triggerer (partition %s of %s)", self.partition_ids[0], self.partition_total
+            )
+        else:
+            self.log.info(
+                "Starting the triggerer (partitions %s of %s)", self.partition_ids, self.partition_total
+            )
+
+        try:
+            # Kick off runner thread
+            self.runner.start()
+            # Start our own DB loop in the main thread
+            self._run_trigger_loop()
+        except Exception:  # pylint: disable=broad-except
+            self.log.exception("Exception when executing TriggererJob._run_trigger_loop")
+            raise
+        finally:
+            self.log.info("Waiting for triggers to clean up")
+            # Tell the subthread to stop and then wait for it.
+            # If the user interrupts/terms again, _graceful_exit will allow them
+            # to force-kill here.
+            self.runner.stop = True
+            self.runner.join()
+            self.log.info("Exited trigger loop")
+
+    def _run_trigger_loop(self) -> None:
+        """
+        The main-thread trigger loop.
+
+        This runs synchronously and handles all database reads/writes.
+        """
+        while not self.runner.stop:
+            # Clean out unused triggers
+            Trigger.clean_unused()
+            # Load/delete triggers
+            self.load_triggers()
+            # Handle events
+            self.handle_events()
+            # Handle failed triggers
+            self.handle_failed_triggers()
+            # Idle sleep
+            time.sleep(1)
+
+    def load_triggers(self):
+        """
+        Queries the database to get the triggers we're supposed to be running,
+        adds them to our runner, and then removes ones from it we no longer
+        need.
+        """
+        requested_trigger_ids = Trigger.runnable_ids(
+            partition_ids=self.partition_ids, partition_total=self.partition_total
+        )
+        self.runner.update_triggers(set(requested_trigger_ids))
+
+    def handle_events(self):
+        """
+        Handles outbound events from triggers - dispatching them into the Trigger
+        model where they are then pushed into the relevant task instances.
+        """
+        while self.runner.events:
+            # Get the event and its trigger ID
+            trigger_id, event = self.runner.events.popleft()
+            # Tell the model to wake up its tasks
+            Trigger.submit_event(trigger_id=trigger_id, event=event)
+
+    def handle_failed_triggers(self):
+        """
+        Handles "failed" triggers - ones that errored or exited before they
+        sent an event. Task Instances that depend on them need failing.
+        """
+        while self.runner.failed_triggers:
+            # Tell the model to fail this trigger's deps
+            trigger_id = self.runner.failed_triggers.popleft()
+            Trigger.submit_failure(trigger_id=trigger_id)
+
+
+class TriggerDetails(TypedDict):
+    """Type class for the trigger details dictionary"""
+
+    task: asyncio.Task
+    name: str
+    events: int
+
+
+class TriggerRunner(threading.Thread, LoggingMixin):
+    """
+    Runtime environment for all triggers.
+
+    Mainly runs inside its own thread, where it hands control off to an asyncio
+    event loop, but is also sometimes interacted with from the main thread
+    (where all the DB queries are done). All communication between threads is
+    done via Deques.
+    """
+
+    # Maps trigger IDs to their running tasks and other info
+    triggers: Dict[int, TriggerDetails]
+
+    # Cache for looking up triggers by classpath
+    trigger_cache: Dict[str, Type[BaseTrigger]]
+
+    # Inbound queue of new triggers
+    to_create: Deque[Tuple[int, BaseTrigger]]
+
+    # Inbound queue of deleted triggers
+    to_delete: Deque[int]
+
+    # Outbound queue of events
+    events: Deque[Tuple[int, TriggerEvent]]
+
+    # Outbound queue of failed triggers
+    failed_triggers: Deque[int]
+
+    # Should-we-stop flag
+    stop: bool = False
+
+    def __init__(self):
+        super().__init__()
+        self.triggers = {}
+        self.trigger_cache = {}
+        self.to_create = deque()
+        self.to_delete = deque()
+        self.events = deque()
+        self.failed_triggers = deque()
+
+    def run(self):
+        """Sync entrypoint - just runs arun in an async loop."""
+        # Pylint complains about this with a 3.6 base, can remove with 3.7+
+        asyncio.run(self.arun())  # pylint: disable=no-member
+
+    async def arun(self):
+        """
+        Main (asynchronous) logic loop.
+
+        The loop in here runs trigger addition/deletion/cleanup. Actual
+        triggers run in their own separate coroutines.
+        """
+        watchdog = create_task(self.block_watchdog())
+        last_status = time.time()
+        while not self.stop:
+            # Run core logic
+            await self.create_triggers()
+            await self.delete_triggers()
+            await self.cleanup_finished_triggers()
+            # Sleep for a bit
+            await asyncio.sleep(1)
+            # Every minute, log status
+            if time.time() - last_status >= 60:
+                self.log.info("%i triggers currently running", len(self.triggers))
+                last_status = time.time()
+        # Wait for watchdog to complete
+        await watchdog
+
+    async def create_triggers(self):
+        """
+        Drain the to_create queue and create all triggers that have been
+        requested in the DB that we don't yet have.
+        """
+        while self.to_create:
+            trigger_id, trigger_instance = self.to_create.popleft()
+            if trigger_id not in self.triggers:
+                self.triggers[trigger_id] = {
+                    "task": create_task(self.run_trigger(trigger_id, trigger_instance)),
+                    "name": f"{trigger_instance!r} (ID {trigger_id})",
+                    "events": 0,
+                }
+            else:
+                self.log.warning("Trigger %s had insertion attempted twice", trigger_id)
+
+    async def delete_triggers(self):
+        """
+        Drain the to_delete queue and ensure all triggers that are not in the
+        DB are cancelled, so the cleanup job deletes them.
+        """
+        while self.to_delete:
+            trigger_id = self.to_delete.popleft()
+            if trigger_id in self.triggers:
+                # We only delete if it did not exit already
+                self.triggers[trigger_id]["task"].cancel()
+
+    async def cleanup_finished_triggers(self):
+        """
+        Go through all trigger tasks (coroutines) and clean up entries for
+        ones that have exited, optionally warning users if the exit was
+        not normal.
+        """
+        for trigger_id, details in list(self.triggers.items()):  # pylint: disable=too-many-nested-blocks
+            if details["task"].done():
+                # Check to see if it exited for good reasons
+                try:
+                    result = details["task"].result()
+                except (asyncio.CancelledError, SystemExit, KeyboardInterrupt):
+                    # These are "expected" exceptions and we stop processing here
+                    # If we don't, then the system requesting a trigger be removed -
+                    # which turns into CancelledError - results in a failure.
+                    del self.triggers[trigger_id]
+                    continue
+                except BaseException as e:
+                    # This is potentially bad, so log it.
+                    self.log.error("Trigger %s exited with error %s", details["name"], e)
+                else:
+                    # See if they foolishly returned a TriggerEvent
+                    if isinstance(result, TriggerEvent):
+                        self.log.error(
+                            "Trigger %s returned a TriggerEvent rather than yielding it", details["name"]
+                        )
+                # See if this exited without sending an event, in which case
+                # any task instances depending on it need to be failed
+                if details["events"] == 0:
+                    self.log.error(
+                        "Trigger %s exited without sending an event. Dependent tasks will be failed.",
+                        details["name"],
+                    )
+                    self.failed_triggers.append(trigger_id)
+                del self.triggers[trigger_id]
+
+    async def block_watchdog(self):
+        """
+        Watchdog loop that detects blocking (badly-written) triggers.
+
+        Triggers should be well-behaved async coroutines and await whenever
+        they need to wait; this loop tries to run every 100ms to see if
+        there are badly-written triggers taking longer than that and blocking
+        the event loop.
+
+        Unfortunately, we can't tell what trigger is blocking things, but
+        we can at least detect the top-level problem.
+        """
+        while not self.stop:
+            last_run = time.monotonic()
+            await asyncio.sleep(0.1)
+            # We allow a generous amount of buffer room for now, since it might
+            # be a busy event loop.
+            time_elapsed = time.monotonic() - last_run
+            if time_elapsed > 0.2:
+                self.log.error(
+                    "Triggerer's async thread was blocked for %.2f seconds, "
+                    "likely by a badly-written trigger. Set PYTHONASYNCIODEBUG=1 "
+                    "to get more information on overrunning coroutines.",
+                    time_elapsed,
+                )
+
+    # Async trigger logic
+
+    async def run_trigger(self, trigger_id, trigger):
+        """
+        Wrapper which runs an actual trigger (they are async generators)
+        and pushes their events into our outbound event deque.
+        """
+        self.log.info("Trigger %s starting", self.triggers[trigger_id]['name'])
+        try:
+            async for event in trigger.run():
+                self.log.info("Trigger %s fired: %s", self.triggers[trigger_id]['name'], event)
+                self.triggers[trigger_id]["events"] += 1
+                self.events.append((trigger_id, event))
+        finally:
+            # CancelledError will get injected when we're stopped - which is
+            # fine, the cleanup process will understand that, but we want to
+            # allow triggers a chance to cleanup, either in that case or if
+            # they exit cleanly.
+            trigger.cleanup()
+
+    # Main-thread sync API
+
+    def update_triggers(self, requested_trigger_ids: Set[int]):
+        """
+        Called from the main thread to request that we update what
+        triggers we're running.
+
+        Works out the differences - ones to add, and ones to remove - then
+        adds them to the deques so the subthread can actually mutate the running
+        trigger set.
+        """
+        current_trigger_ids = set(self.triggers.keys())

Review comment:
       If we can avoid a lock it's more performat to not. Perhaps just a comment somewhere saying this info would do

##########
File path: airflow/jobs/triggerer_job.py
##########
@@ -0,0 +1,418 @@
+# 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 asyncio
+import importlib
+import os
+import signal
+import sys
+import threading
+import time
+from collections import deque
+from typing import Deque, Dict, List, Optional, Set, Tuple, Type
+
+from airflow.jobs.base_job import BaseJob
+from airflow.models.trigger import Trigger
+from airflow.triggers.base import BaseTrigger, TriggerEvent
+from airflow.typing_compat import TypedDict
+from airflow.utils.asyncio import create_task
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class TriggererJob(BaseJob, LoggingMixin):
+    """
+    TriggererJob continuously runs active triggers in asyncio, watching
+    for them to fire off their events and then dispatching that information
+    to their dependent tasks/DAGs.
+
+    It runs as two threads:
+     - The main thread does DB calls/checkins
+     - A subthread runs all the async code
+    """
+
+    __mapper_args__ = {'polymorphic_identity': 'TriggererJob'}
+
+    partition_ids: Optional[List[int]] = None
+    partition_total: Optional[int] = None
+
+    def __init__(self, partition=None, *args, **kwargs):
+        # Make sure we can actually run
+        if not hasattr(asyncio, "create_task"):
+            raise RuntimeError("The triggerer/deferred operators only work on Python 3.7 and above.")
+        # Call superclass
+        super().__init__(*args, **kwargs)
+        # Decode partition information
+        self.partition_ids, self.partition_total = None, None
+        if partition:
+            self.partition_ids, self.partition_total = self.decode_partition(partition)
+        # Set up runner async thread
+        self.runner = TriggerRunner()
+
+    def decode_partition(self, partition: str) -> Tuple[List[int], int]:
+        """
+        Given a string-format partition specification, returns the list of
+        partition IDs it represents and the partition total.
+        """
+        try:
+            # The partition format is "1,2,3/10" where the numbers before
+            # the slash are the partitions we represent, and the number
+            # after is the total number. Most users will just have a single
+            # partition number, e.g. "2/10".
+            ids_str, total_str = partition.split("/", 1)
+            partition_total = int(total_str)
+            partition_ids = []
+            for id_str in ids_str.split(","):
+                id_number = int(id_str)
+                # Bounds checking (they're 1-indexed, which might catch people out)
+                if id_number <= 0 or id_number > self.partition_total:
+                    raise ValueError(f"Partition number {id_number} is impossible")
+                self.partition_ids.append(id_number)
+        except (ValueError, TypeError):
+            raise ValueError(f"Invalid partition specification: {partition}")
+        return partition_ids, partition_total
+
+    def register_signals(self) -> None:
+        """Register signals that stop child processes"""
+        signal.signal(signal.SIGINT, self._exit_gracefully)
+        signal.signal(signal.SIGTERM, self._exit_gracefully)
+
+    def _exit_gracefully(self, signum, frame) -> None:  # pylint: disable=unused-argument
+        """Helper method to clean up processor_agent to avoid leaving orphan processes."""
+        # The first time, try to exit nicely
+        if not self.runner.stop:
+            self.log.info("Exiting gracefully upon receiving signal %s", signum)
+            self.runner.stop = True
+        else:
+            self.log.warning("Forcing exit due to second exit signal %s", signum)
+            sys.exit(os.EX_SOFTWARE)
+
+    def _execute(self) -> None:
+        # Display custom startup ack depending on plurality of partitions
+        if self.partition_ids is None:
+            self.log.info("Starting the triggerer")
+        elif len(self.partition_ids) == 1:
+            self.log.info(
+                "Starting the triggerer (partition %s of %s)", self.partition_ids[0], self.partition_total
+            )
+        else:
+            self.log.info(
+                "Starting the triggerer (partitions %s of %s)", self.partition_ids, self.partition_total
+            )
+
+        try:
+            # Kick off runner thread
+            self.runner.start()
+            # Start our own DB loop in the main thread
+            self._run_trigger_loop()
+        except Exception:  # pylint: disable=broad-except
+            self.log.exception("Exception when executing TriggererJob._run_trigger_loop")
+            raise
+        finally:
+            self.log.info("Waiting for triggers to clean up")
+            # Tell the subthread to stop and then wait for it.
+            # If the user interrupts/terms again, _graceful_exit will allow them
+            # to force-kill here.
+            self.runner.stop = True
+            self.runner.join()
+            self.log.info("Exited trigger loop")
+
+    def _run_trigger_loop(self) -> None:
+        """
+        The main-thread trigger loop.
+
+        This runs synchronously and handles all database reads/writes.
+        """
+        while not self.runner.stop:
+            # Clean out unused triggers
+            Trigger.clean_unused()
+            # Load/delete triggers
+            self.load_triggers()
+            # Handle events
+            self.handle_events()
+            # Handle failed triggers
+            self.handle_failed_triggers()
+            # Idle sleep
+            time.sleep(1)
+
+    def load_triggers(self):
+        """
+        Queries the database to get the triggers we're supposed to be running,
+        adds them to our runner, and then removes ones from it we no longer
+        need.
+        """
+        requested_trigger_ids = Trigger.runnable_ids(
+            partition_ids=self.partition_ids, partition_total=self.partition_total
+        )
+        self.runner.update_triggers(set(requested_trigger_ids))
+
+    def handle_events(self):
+        """
+        Handles outbound events from triggers - dispatching them into the Trigger
+        model where they are then pushed into the relevant task instances.
+        """
+        while self.runner.events:
+            # Get the event and its trigger ID
+            trigger_id, event = self.runner.events.popleft()
+            # Tell the model to wake up its tasks
+            Trigger.submit_event(trigger_id=trigger_id, event=event)
+
+    def handle_failed_triggers(self):
+        """
+        Handles "failed" triggers - ones that errored or exited before they
+        sent an event. Task Instances that depend on them need failing.
+        """
+        while self.runner.failed_triggers:
+            # Tell the model to fail this trigger's deps
+            trigger_id = self.runner.failed_triggers.popleft()
+            Trigger.submit_failure(trigger_id=trigger_id)
+
+
+class TriggerDetails(TypedDict):
+    """Type class for the trigger details dictionary"""
+
+    task: asyncio.Task
+    name: str
+    events: int
+
+
+class TriggerRunner(threading.Thread, LoggingMixin):
+    """
+    Runtime environment for all triggers.
+
+    Mainly runs inside its own thread, where it hands control off to an asyncio
+    event loop, but is also sometimes interacted with from the main thread
+    (where all the DB queries are done). All communication between threads is
+    done via Deques.
+    """
+
+    # Maps trigger IDs to their running tasks and other info
+    triggers: Dict[int, TriggerDetails]
+
+    # Cache for looking up triggers by classpath
+    trigger_cache: Dict[str, Type[BaseTrigger]]
+
+    # Inbound queue of new triggers
+    to_create: Deque[Tuple[int, BaseTrigger]]
+
+    # Inbound queue of deleted triggers
+    to_delete: Deque[int]
+
+    # Outbound queue of events
+    events: Deque[Tuple[int, TriggerEvent]]
+
+    # Outbound queue of failed triggers
+    failed_triggers: Deque[int]
+
+    # Should-we-stop flag
+    stop: bool = False
+
+    def __init__(self):
+        super().__init__()
+        self.triggers = {}
+        self.trigger_cache = {}
+        self.to_create = deque()
+        self.to_delete = deque()
+        self.events = deque()
+        self.failed_triggers = deque()
+
+    def run(self):
+        """Sync entrypoint - just runs arun in an async loop."""
+        # Pylint complains about this with a 3.6 base, can remove with 3.7+
+        asyncio.run(self.arun())  # pylint: disable=no-member
+
+    async def arun(self):
+        """
+        Main (asynchronous) logic loop.
+
+        The loop in here runs trigger addition/deletion/cleanup. Actual
+        triggers run in their own separate coroutines.
+        """
+        watchdog = create_task(self.block_watchdog())
+        last_status = time.time()
+        while not self.stop:
+            # Run core logic
+            await self.create_triggers()
+            await self.delete_triggers()
+            await self.cleanup_finished_triggers()
+            # Sleep for a bit
+            await asyncio.sleep(1)
+            # Every minute, log status
+            if time.time() - last_status >= 60:
+                self.log.info("%i triggers currently running", len(self.triggers))
+                last_status = time.time()
+        # Wait for watchdog to complete
+        await watchdog
+
+    async def create_triggers(self):
+        """
+        Drain the to_create queue and create all triggers that have been
+        requested in the DB that we don't yet have.
+        """
+        while self.to_create:
+            trigger_id, trigger_instance = self.to_create.popleft()
+            if trigger_id not in self.triggers:
+                self.triggers[trigger_id] = {
+                    "task": create_task(self.run_trigger(trigger_id, trigger_instance)),
+                    "name": f"{trigger_instance!r} (ID {trigger_id})",
+                    "events": 0,
+                }
+            else:
+                self.log.warning("Trigger %s had insertion attempted twice", trigger_id)
+
+    async def delete_triggers(self):
+        """
+        Drain the to_delete queue and ensure all triggers that are not in the
+        DB are cancelled, so the cleanup job deletes them.
+        """
+        while self.to_delete:
+            trigger_id = self.to_delete.popleft()
+            if trigger_id in self.triggers:
+                # We only delete if it did not exit already
+                self.triggers[trigger_id]["task"].cancel()
+
+    async def cleanup_finished_triggers(self):
+        """
+        Go through all trigger tasks (coroutines) and clean up entries for
+        ones that have exited, optionally warning users if the exit was
+        not normal.
+        """
+        for trigger_id, details in list(self.triggers.items()):  # pylint: disable=too-many-nested-blocks
+            if details["task"].done():
+                # Check to see if it exited for good reasons
+                try:
+                    result = details["task"].result()
+                except (asyncio.CancelledError, SystemExit, KeyboardInterrupt):
+                    # These are "expected" exceptions and we stop processing here
+                    # If we don't, then the system requesting a trigger be removed -
+                    # which turns into CancelledError - results in a failure.
+                    del self.triggers[trigger_id]
+                    continue
+                except BaseException as e:
+                    # This is potentially bad, so log it.
+                    self.log.error("Trigger %s exited with error %s", details["name"], e)
+                else:
+                    # See if they foolishly returned a TriggerEvent
+                    if isinstance(result, TriggerEvent):
+                        self.log.error(
+                            "Trigger %s returned a TriggerEvent rather than yielding it", details["name"]
+                        )
+                # See if this exited without sending an event, in which case
+                # any task instances depending on it need to be failed
+                if details["events"] == 0:
+                    self.log.error(
+                        "Trigger %s exited without sending an event. Dependent tasks will be failed.",
+                        details["name"],
+                    )
+                    self.failed_triggers.append(trigger_id)
+                del self.triggers[trigger_id]
+
+    async def block_watchdog(self):
+        """
+        Watchdog loop that detects blocking (badly-written) triggers.
+
+        Triggers should be well-behaved async coroutines and await whenever
+        they need to wait; this loop tries to run every 100ms to see if
+        there are badly-written triggers taking longer than that and blocking
+        the event loop.
+
+        Unfortunately, we can't tell what trigger is blocking things, but
+        we can at least detect the top-level problem.
+        """
+        while not self.stop:
+            last_run = time.monotonic()
+            await asyncio.sleep(0.1)
+            # We allow a generous amount of buffer room for now, since it might
+            # be a busy event loop.
+            time_elapsed = time.monotonic() - last_run
+            if time_elapsed > 0.2:
+                self.log.error(
+                    "Triggerer's async thread was blocked for %.2f seconds, "
+                    "likely by a badly-written trigger. Set PYTHONASYNCIODEBUG=1 "
+                    "to get more information on overrunning coroutines.",
+                    time_elapsed,
+                )
+
+    # Async trigger logic
+
+    async def run_trigger(self, trigger_id, trigger):
+        """
+        Wrapper which runs an actual trigger (they are async generators)
+        and pushes their events into our outbound event deque.
+        """
+        self.log.info("Trigger %s starting", self.triggers[trigger_id]['name'])
+        try:
+            async for event in trigger.run():
+                self.log.info("Trigger %s fired: %s", self.triggers[trigger_id]['name'], event)
+                self.triggers[trigger_id]["events"] += 1
+                self.events.append((trigger_id, event))
+        finally:
+            # CancelledError will get injected when we're stopped - which is
+            # fine, the cleanup process will understand that, but we want to
+            # allow triggers a chance to cleanup, either in that case or if
+            # they exit cleanly.
+            trigger.cleanup()
+
+    # Main-thread sync API
+
+    def update_triggers(self, requested_trigger_ids: Set[int]):
+        """
+        Called from the main thread to request that we update what
+        triggers we're running.
+
+        Works out the differences - ones to add, and ones to remove - then
+        adds them to the deques so the subthread can actually mutate the running
+        trigger set.
+        """
+        current_trigger_ids = set(self.triggers.keys())

Review comment:
       If we can avoid a lock it's more performant to not. Perhaps just a comment somewhere saying this info would do




-- 
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:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#issuecomment-859785046


   [The Workflow run](https://github.com/apache/airflow/actions/runs/849772742) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


-- 
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:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#issuecomment-834683813


   [The Workflow run](https://github.com/apache/airflow/actions/runs/821180230) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r639923326



##########
File path: airflow/models/taskinstance.py
##########
@@ -282,6 +296,18 @@ class TaskInstance(Base, LoggingMixin):  # pylint: disable=R0902,R0904
     executor_config = Column(PickleType(pickler=dill))
 
     external_executor_id = Column(String(ID_LEN, **COLLATION_ARGS))
+
+    # The trigger to resume on if we are in state DEFERRED
+    trigger_id = Column(BigInteger)
+
+    # Optional timeout datetime for the trigger (past this, we'll fail)
+    trigger_timeout = Column(UtcDateTime)
+
+    # The method to call next, and any extra arguments to pass to it.
+    # Usually used when resuming from DEFERRED.
+    next_method = Column(String(1000))

Review comment:
       Exactly, I don't think it matters either way, I was just pattern-matching to what was done there already. It might be something to do with indexes?




-- 
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:
users@infra.apache.org



[GitHub] [airflow] xinbinhuang commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638221307



##########
File path: airflow/api_connexion/endpoints/task_instance_endpoint.py
##########
@@ -248,7 +248,7 @@ def post_clear_task_instances(dag_id: str, session=None):
         error_message = f"Dag id {dag_id} not found"
         raise NotFound(error_message)
     reset_dag_runs = data.pop('reset_dag_runs')
-    task_instances = dag.clear(get_tis=True, **data)
+    task_instances = dag.clear(get_ti_instances=True, **data)

Review comment:
       I believe `tis` stands for `task instances`, so maybe `get_task_instances` to make it more explicit? 
   
   Another problem, `ti` is also a "public" API/concept as it's used in xcom, task context, macros etc. Though in general, I prefer explicit over abbreviation, I don't think it's good change to include within this PR.
   
   The same applies to the rest of `ti_instances`




-- 
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:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638659006



##########
File path: airflow/api_connexion/endpoints/task_instance_endpoint.py
##########
@@ -248,7 +248,7 @@ def post_clear_task_instances(dag_id: str, session=None):
         error_message = f"Dag id {dag_id} not found"
         raise NotFound(error_message)
     reset_dag_runs = data.pop('reset_dag_runs')
-    task_instances = dag.clear(get_tis=True, **data)
+    task_instances = dag.clear(get_ti_instances=True, **data)

Review comment:
       `get_ti_objects` perhaps?




-- 
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:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r639632028



##########
File path: airflow/models/taskinstance.py
##########
@@ -282,6 +296,18 @@ class TaskInstance(Base, LoggingMixin):  # pylint: disable=R0902,R0904
     executor_config = Column(PickleType(pickler=dill))
 
     external_executor_id = Column(String(ID_LEN, **COLLATION_ARGS))
+
+    # The trigger to resume on if we are in state DEFERRED
+    trigger_id = Column(BigInteger)
+
+    # Optional timeout datetime for the trigger (past this, we'll fail)
+    trigger_timeout = Column(UtcDateTime)
+
+    # The method to call next, and any extra arguments to pass to it.
+    # Usually used when resuming from DEFERRED.
+    next_method = Column(String(1000))

Review comment:
       Oh could be that then. I have (thankfully) forgotten much about MySQL. It probably doesn't matter here as this will be limited to ASCII chars only, right? (Cos this is a python class/function name)




-- 
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:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r640534963



##########
File path: airflow/jobs/triggerer_job.py
##########
@@ -0,0 +1,418 @@
+# 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 asyncio
+import importlib
+import os
+import signal
+import sys
+import threading
+import time
+from collections import deque
+from typing import Deque, Dict, List, Optional, Set, Tuple, Type
+
+from airflow.jobs.base_job import BaseJob
+from airflow.models.trigger import Trigger
+from airflow.triggers.base import BaseTrigger, TriggerEvent
+from airflow.typing_compat import TypedDict
+from airflow.utils.asyncio import create_task
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class TriggererJob(BaseJob, LoggingMixin):
+    """
+    TriggererJob continuously runs active triggers in asyncio, watching
+    for them to fire off their events and then dispatching that information
+    to their dependent tasks/DAGs.
+
+    It runs as two threads:
+     - The main thread does DB calls/checkins
+     - A subthread runs all the async code
+    """
+
+    __mapper_args__ = {'polymorphic_identity': 'TriggererJob'}
+
+    partition_ids: Optional[List[int]] = None
+    partition_total: Optional[int] = None
+
+    def __init__(self, partition=None, *args, **kwargs):
+        # Make sure we can actually run
+        if not hasattr(asyncio, "create_task"):
+            raise RuntimeError("The triggerer/deferred operators only work on Python 3.7 and above.")
+        # Call superclass
+        super().__init__(*args, **kwargs)
+        # Decode partition information
+        self.partition_ids, self.partition_total = None, None
+        if partition:
+            self.partition_ids, self.partition_total = self.decode_partition(partition)
+        # Set up runner async thread
+        self.runner = TriggerRunner()
+
+    def decode_partition(self, partition: str) -> Tuple[List[int], int]:
+        """
+        Given a string-format partition specification, returns the list of
+        partition IDs it represents and the partition total.
+        """
+        try:
+            # The partition format is "1,2,3/10" where the numbers before
+            # the slash are the partitions we represent, and the number
+            # after is the total number. Most users will just have a single
+            # partition number, e.g. "2/10".
+            ids_str, total_str = partition.split("/", 1)
+            partition_total = int(total_str)
+            partition_ids = []
+            for id_str in ids_str.split(","):
+                id_number = int(id_str)
+                # Bounds checking (they're 1-indexed, which might catch people out)
+                if id_number <= 0 or id_number > self.partition_total:
+                    raise ValueError(f"Partition number {id_number} is impossible")
+                self.partition_ids.append(id_number)
+        except (ValueError, TypeError):
+            raise ValueError(f"Invalid partition specification: {partition}")
+        return partition_ids, partition_total
+
+    def register_signals(self) -> None:
+        """Register signals that stop child processes"""
+        signal.signal(signal.SIGINT, self._exit_gracefully)
+        signal.signal(signal.SIGTERM, self._exit_gracefully)
+
+    def _exit_gracefully(self, signum, frame) -> None:  # pylint: disable=unused-argument
+        """Helper method to clean up processor_agent to avoid leaving orphan processes."""
+        # The first time, try to exit nicely
+        if not self.runner.stop:
+            self.log.info("Exiting gracefully upon receiving signal %s", signum)
+            self.runner.stop = True
+        else:
+            self.log.warning("Forcing exit due to second exit signal %s", signum)
+            sys.exit(os.EX_SOFTWARE)
+
+    def _execute(self) -> None:
+        # Display custom startup ack depending on plurality of partitions
+        if self.partition_ids is None:
+            self.log.info("Starting the triggerer")
+        elif len(self.partition_ids) == 1:
+            self.log.info(
+                "Starting the triggerer (partition %s of %s)", self.partition_ids[0], self.partition_total
+            )
+        else:
+            self.log.info(
+                "Starting the triggerer (partitions %s of %s)", self.partition_ids, self.partition_total
+            )
+
+        try:
+            # Kick off runner thread
+            self.runner.start()
+            # Start our own DB loop in the main thread
+            self._run_trigger_loop()
+        except Exception:  # pylint: disable=broad-except
+            self.log.exception("Exception when executing TriggererJob._run_trigger_loop")
+            raise
+        finally:
+            self.log.info("Waiting for triggers to clean up")
+            # Tell the subthread to stop and then wait for it.
+            # If the user interrupts/terms again, _graceful_exit will allow them
+            # to force-kill here.
+            self.runner.stop = True
+            self.runner.join()
+            self.log.info("Exited trigger loop")
+
+    def _run_trigger_loop(self) -> None:
+        """
+        The main-thread trigger loop.
+
+        This runs synchronously and handles all database reads/writes.
+        """
+        while not self.runner.stop:
+            # Clean out unused triggers
+            Trigger.clean_unused()
+            # Load/delete triggers
+            self.load_triggers()
+            # Handle events
+            self.handle_events()
+            # Handle failed triggers
+            self.handle_failed_triggers()
+            # Idle sleep
+            time.sleep(1)
+
+    def load_triggers(self):
+        """
+        Queries the database to get the triggers we're supposed to be running,
+        adds them to our runner, and then removes ones from it we no longer
+        need.
+        """
+        requested_trigger_ids = Trigger.runnable_ids(
+            partition_ids=self.partition_ids, partition_total=self.partition_total
+        )
+        self.runner.update_triggers(set(requested_trigger_ids))
+
+    def handle_events(self):
+        """
+        Handles outbound events from triggers - dispatching them into the Trigger
+        model where they are then pushed into the relevant task instances.
+        """
+        while self.runner.events:
+            # Get the event and its trigger ID
+            trigger_id, event = self.runner.events.popleft()
+            # Tell the model to wake up its tasks
+            Trigger.submit_event(trigger_id=trigger_id, event=event)
+
+    def handle_failed_triggers(self):
+        """
+        Handles "failed" triggers - ones that errored or exited before they
+        sent an event. Task Instances that depend on them need failing.
+        """
+        while self.runner.failed_triggers:
+            # Tell the model to fail this trigger's deps
+            trigger_id = self.runner.failed_triggers.popleft()
+            Trigger.submit_failure(trigger_id=trigger_id)
+
+
+class TriggerDetails(TypedDict):
+    """Type class for the trigger details dictionary"""
+
+    task: asyncio.Task
+    name: str
+    events: int
+
+
+class TriggerRunner(threading.Thread, LoggingMixin):
+    """
+    Runtime environment for all triggers.
+
+    Mainly runs inside its own thread, where it hands control off to an asyncio
+    event loop, but is also sometimes interacted with from the main thread
+    (where all the DB queries are done). All communication between threads is
+    done via Deques.
+    """
+
+    # Maps trigger IDs to their running tasks and other info
+    triggers: Dict[int, TriggerDetails]
+
+    # Cache for looking up triggers by classpath
+    trigger_cache: Dict[str, Type[BaseTrigger]]
+
+    # Inbound queue of new triggers
+    to_create: Deque[Tuple[int, BaseTrigger]]
+
+    # Inbound queue of deleted triggers
+    to_delete: Deque[int]
+
+    # Outbound queue of events
+    events: Deque[Tuple[int, TriggerEvent]]
+
+    # Outbound queue of failed triggers
+    failed_triggers: Deque[int]
+
+    # Should-we-stop flag
+    stop: bool = False
+
+    def __init__(self):
+        super().__init__()
+        self.triggers = {}
+        self.trigger_cache = {}
+        self.to_create = deque()
+        self.to_delete = deque()
+        self.events = deque()
+        self.failed_triggers = deque()
+
+    def run(self):
+        """Sync entrypoint - just runs arun in an async loop."""
+        # Pylint complains about this with a 3.6 base, can remove with 3.7+
+        asyncio.run(self.arun())  # pylint: disable=no-member
+
+    async def arun(self):
+        """
+        Main (asynchronous) logic loop.
+
+        The loop in here runs trigger addition/deletion/cleanup. Actual
+        triggers run in their own separate coroutines.
+        """
+        watchdog = create_task(self.block_watchdog())
+        last_status = time.time()
+        while not self.stop:
+            # Run core logic
+            await self.create_triggers()
+            await self.delete_triggers()
+            await self.cleanup_finished_triggers()
+            # Sleep for a bit
+            await asyncio.sleep(1)
+            # Every minute, log status
+            if time.time() - last_status >= 60:
+                self.log.info("%i triggers currently running", len(self.triggers))
+                last_status = time.time()
+        # Wait for watchdog to complete
+        await watchdog
+
+    async def create_triggers(self):
+        """
+        Drain the to_create queue and create all triggers that have been
+        requested in the DB that we don't yet have.
+        """
+        while self.to_create:
+            trigger_id, trigger_instance = self.to_create.popleft()
+            if trigger_id not in self.triggers:
+                self.triggers[trigger_id] = {
+                    "task": create_task(self.run_trigger(trigger_id, trigger_instance)),
+                    "name": f"{trigger_instance!r} (ID {trigger_id})",
+                    "events": 0,
+                }
+            else:
+                self.log.warning("Trigger %s had insertion attempted twice", trigger_id)
+
+    async def delete_triggers(self):
+        """
+        Drain the to_delete queue and ensure all triggers that are not in the
+        DB are cancelled, so the cleanup job deletes them.
+        """
+        while self.to_delete:
+            trigger_id = self.to_delete.popleft()
+            if trigger_id in self.triggers:
+                # We only delete if it did not exit already
+                self.triggers[trigger_id]["task"].cancel()
+
+    async def cleanup_finished_triggers(self):
+        """
+        Go through all trigger tasks (coroutines) and clean up entries for
+        ones that have exited, optionally warning users if the exit was
+        not normal.
+        """
+        for trigger_id, details in list(self.triggers.items()):  # pylint: disable=too-many-nested-blocks
+            if details["task"].done():
+                # Check to see if it exited for good reasons
+                try:
+                    result = details["task"].result()
+                except (asyncio.CancelledError, SystemExit, KeyboardInterrupt):
+                    # These are "expected" exceptions and we stop processing here
+                    # If we don't, then the system requesting a trigger be removed -
+                    # which turns into CancelledError - results in a failure.
+                    del self.triggers[trigger_id]
+                    continue
+                except BaseException as e:
+                    # This is potentially bad, so log it.
+                    self.log.error("Trigger %s exited with error %s", details["name"], e)
+                else:
+                    # See if they foolishly returned a TriggerEvent
+                    if isinstance(result, TriggerEvent):
+                        self.log.error(
+                            "Trigger %s returned a TriggerEvent rather than yielding it", details["name"]
+                        )
+                # See if this exited without sending an event, in which case
+                # any task instances depending on it need to be failed
+                if details["events"] == 0:
+                    self.log.error(
+                        "Trigger %s exited without sending an event. Dependent tasks will be failed.",
+                        details["name"],
+                    )
+                    self.failed_triggers.append(trigger_id)
+                del self.triggers[trigger_id]
+
+    async def block_watchdog(self):
+        """
+        Watchdog loop that detects blocking (badly-written) triggers.
+
+        Triggers should be well-behaved async coroutines and await whenever
+        they need to wait; this loop tries to run every 100ms to see if
+        there are badly-written triggers taking longer than that and blocking
+        the event loop.
+
+        Unfortunately, we can't tell what trigger is blocking things, but
+        we can at least detect the top-level problem.
+        """
+        while not self.stop:
+            last_run = time.monotonic()
+            await asyncio.sleep(0.1)
+            # We allow a generous amount of buffer room for now, since it might
+            # be a busy event loop.
+            time_elapsed = time.monotonic() - last_run
+            if time_elapsed > 0.2:
+                self.log.error(
+                    "Triggerer's async thread was blocked for %.2f seconds, "
+                    "likely by a badly-written trigger. Set PYTHONASYNCIODEBUG=1 "
+                    "to get more information on overrunning coroutines.",
+                    time_elapsed,
+                )
+
+    # Async trigger logic
+
+    async def run_trigger(self, trigger_id, trigger):
+        """
+        Wrapper which runs an actual trigger (they are async generators)
+        and pushes their events into our outbound event deque.
+        """
+        self.log.info("Trigger %s starting", self.triggers[trigger_id]['name'])
+        try:
+            async for event in trigger.run():
+                self.log.info("Trigger %s fired: %s", self.triggers[trigger_id]['name'], event)
+                self.triggers[trigger_id]["events"] += 1
+                self.events.append((trigger_id, event))
+        finally:
+            # CancelledError will get injected when we're stopped - which is
+            # fine, the cleanup process will understand that, but we want to
+            # allow triggers a chance to cleanup, either in that case or if
+            # they exit cleanly.
+            trigger.cleanup()
+
+    # Main-thread sync API
+
+    def update_triggers(self, requested_trigger_ids: Set[int]):
+        """
+        Called from the main thread to request that we update what
+        triggers we're running.
+
+        Works out the differences - ones to add, and ones to remove - then
+        adds them to the deques so the subthread can actually mutate the running
+        trigger set.
+        """
+        current_trigger_ids = set(self.triggers.keys())
+        # Work out the two difference sets
+        new_trigger_ids = requested_trigger_ids.difference(current_trigger_ids)
+        old_trigger_ids = current_trigger_ids.difference(requested_trigger_ids)
+        # Bulk-fetch new trigger records
+        new_triggers = Trigger.bulk_fetch(new_trigger_ids)
+        # Add in new triggers
+        for new_id in new_trigger_ids:
+            # Check it didn't vanish in the meantime
+            if new_id not in new_triggers:
+                self.log.warning("Trigger ID %s disappeared before we could start it", new_id)
+                continue
+            # Resolve trigger record into an actual class instance
+            trigger_class = self.get_trigger_by_classpath(new_triggers[new_id].classpath)
+            self.to_create.append((new_id, trigger_class(**new_triggers[new_id].kwargs)))
+        # Remove old triggers
+        for old_id in old_trigger_ids:
+            self.to_delete.append(old_id)
+
+    def get_trigger_by_classpath(self, classpath: str) -> Type[BaseTrigger]:
+        """
+        Gets a trigger class by its classpath ("path.to.module.classname")
+
+        Uses a cache dictionary to speed up lookups after the first time.
+        """
+        if classpath not in self.trigger_cache:
+            module_name, class_name = classpath.rsplit(".", 1)
+            try:
+                module = importlib.import_module(module_name)
+            except ImportError:
+                raise ImportError(
+                    f"Cannot import trigger module {module_name} (from trigger classpath {classpath})"
+                )
+            try:
+                trigger_class = getattr(module, class_name)
+            except AttributeError:
+                raise ImportError(f"Cannot import trigger {class_name} from module {module_name}")
+            self.trigger_cache[classpath] = trigger_class

Review comment:
       Orrrr a junkyard where no one can find anything cos that's just where everything gets dumped.
   
   Potato potato.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638663257



##########
File path: airflow/jobs/triggerer_job.py
##########
@@ -0,0 +1,418 @@
+# 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 asyncio
+import importlib
+import os
+import signal
+import sys
+import threading
+import time
+from collections import deque
+from typing import Deque, Dict, List, Optional, Set, Tuple, Type
+
+from airflow.jobs.base_job import BaseJob
+from airflow.models.trigger import Trigger
+from airflow.triggers.base import BaseTrigger, TriggerEvent
+from airflow.typing_compat import TypedDict
+from airflow.utils.asyncio import create_task
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class TriggererJob(BaseJob, LoggingMixin):
+    """
+    TriggererJob continuously runs active triggers in asyncio, watching
+    for them to fire off their events and then dispatching that information
+    to their dependent tasks/DAGs.
+
+    It runs as two threads:
+     - The main thread does DB calls/checkins
+     - A subthread runs all the async code
+    """
+
+    __mapper_args__ = {'polymorphic_identity': 'TriggererJob'}
+
+    partition_ids: Optional[List[int]] = None
+    partition_total: Optional[int] = None
+
+    def __init__(self, partition=None, *args, **kwargs):
+        # Make sure we can actually run
+        if not hasattr(asyncio, "create_task"):
+            raise RuntimeError("The triggerer/deferred operators only work on Python 3.7 and above.")
+        # Call superclass
+        super().__init__(*args, **kwargs)
+        # Decode partition information
+        self.partition_ids, self.partition_total = None, None
+        if partition:
+            self.partition_ids, self.partition_total = self.decode_partition(partition)
+        # Set up runner async thread
+        self.runner = TriggerRunner()
+
+    def decode_partition(self, partition: str) -> Tuple[List[int], int]:
+        """
+        Given a string-format partition specification, returns the list of
+        partition IDs it represents and the partition total.
+        """
+        try:
+            # The partition format is "1,2,3/10" where the numbers before
+            # the slash are the partitions we represent, and the number
+            # after is the total number. Most users will just have a single
+            # partition number, e.g. "2/10".
+            ids_str, total_str = partition.split("/", 1)
+            partition_total = int(total_str)
+            partition_ids = []
+            for id_str in ids_str.split(","):
+                id_number = int(id_str)
+                # Bounds checking (they're 1-indexed, which might catch people out)
+                if id_number <= 0 or id_number > self.partition_total:
+                    raise ValueError(f"Partition number {id_number} is impossible")
+                self.partition_ids.append(id_number)
+        except (ValueError, TypeError):
+            raise ValueError(f"Invalid partition specification: {partition}")
+        return partition_ids, partition_total
+
+    def register_signals(self) -> None:
+        """Register signals that stop child processes"""
+        signal.signal(signal.SIGINT, self._exit_gracefully)
+        signal.signal(signal.SIGTERM, self._exit_gracefully)
+
+    def _exit_gracefully(self, signum, frame) -> None:  # pylint: disable=unused-argument
+        """Helper method to clean up processor_agent to avoid leaving orphan processes."""
+        # The first time, try to exit nicely
+        if not self.runner.stop:
+            self.log.info("Exiting gracefully upon receiving signal %s", signum)
+            self.runner.stop = True
+        else:
+            self.log.warning("Forcing exit due to second exit signal %s", signum)
+            sys.exit(os.EX_SOFTWARE)
+
+    def _execute(self) -> None:
+        # Display custom startup ack depending on plurality of partitions
+        if self.partition_ids is None:
+            self.log.info("Starting the triggerer")
+        elif len(self.partition_ids) == 1:
+            self.log.info(
+                "Starting the triggerer (partition %s of %s)", self.partition_ids[0], self.partition_total
+            )
+        else:
+            self.log.info(
+                "Starting the triggerer (partitions %s of %s)", self.partition_ids, self.partition_total
+            )
+
+        try:
+            # Kick off runner thread
+            self.runner.start()
+            # Start our own DB loop in the main thread
+            self._run_trigger_loop()
+        except Exception:  # pylint: disable=broad-except
+            self.log.exception("Exception when executing TriggererJob._run_trigger_loop")
+            raise
+        finally:
+            self.log.info("Waiting for triggers to clean up")
+            # Tell the subthread to stop and then wait for it.
+            # If the user interrupts/terms again, _graceful_exit will allow them
+            # to force-kill here.
+            self.runner.stop = True
+            self.runner.join()

Review comment:
       Worth having a timeout on join?

##########
File path: airflow/models/baseoperator.py
##########
@@ -1534,6 +1535,23 @@ def inherits_from_dummy_operator(self):
         # of its sub-classes (which don't inherit from anything but BaseOperator).
         return getattr(self, '_is_dummy', False)
 
+    def defer(
+        self,
+        *,
+        trigger: BaseTrigger,
+        method_name: str,
+        kwargs: Optional[Dict[str, Any]] = None,
+        timeout: Optional[timedelta] = None,
+    ):
+        """
+        Marks this Operator as being "deferred" - that is, suspending its
+        execution until the provided trigger fires an event.
+
+        This is achieved by raising a special exception (OperatorDeferred)
+        which is caught in the main _execute_task wrapper.
+        """
+        raise TaskDeferred(trigger=trigger, method_name=method_name, kwargs=kwargs, timeout=timeout)

Review comment:
       Is it worth doing a check for Python <3.7 here and _failing_ the task instead?
   
   (Rather than going to deferred but not having the trigger ever be able to run?)

##########
File path: airflow/models/baseoperator.py
##########
@@ -1534,6 +1535,23 @@ def inherits_from_dummy_operator(self):
         # of its sub-classes (which don't inherit from anything but BaseOperator).
         return getattr(self, '_is_dummy', False)
 
+    def defer(
+        self,
+        *,
+        trigger: BaseTrigger,
+        method_name: str,
+        kwargs: Optional[Dict[str, Any]] = None,
+        timeout: Optional[timedelta] = None,
+    ):
+        """
+        Marks this Operator as being "deferred" - that is, suspending its
+        execution until the provided trigger fires an event.
+
+        This is achieved by raising a special exception (OperatorDeferred)

Review comment:
       ```suggestion
           This is achieved by raising a special exception (TaskDeferred)
   ```

##########
File path: airflow/jobs/triggerer_job.py
##########
@@ -0,0 +1,418 @@
+# 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 asyncio
+import importlib
+import os
+import signal
+import sys
+import threading
+import time
+from collections import deque
+from typing import Deque, Dict, List, Optional, Set, Tuple, Type
+
+from airflow.jobs.base_job import BaseJob
+from airflow.models.trigger import Trigger
+from airflow.triggers.base import BaseTrigger, TriggerEvent
+from airflow.typing_compat import TypedDict
+from airflow.utils.asyncio import create_task
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class TriggererJob(BaseJob, LoggingMixin):

Review comment:
       ```suggestion
   class TriggererJob(BaseJob):
   ```
   
   BaseJob already has LoggingMixin

##########
File path: airflow/models/dag.py
##########
@@ -1177,8 +1179,10 @@ def clear(
         :type dry_run: bool
         :param session: The sqlalchemy session to use
         :type session: sqlalchemy.orm.session.Session
-        :param get_tis: Return the sqlalchemy query for finding the TaskInstance without clearing the tasks
-        :type get_tis: bool
+        :param get_ti_keys: Return the sqlalchemy query for TaskInstance PKs without clearing the tasks
+        :type get_ti_keys: bool
+        :param get_ti_instances: Return the sqlalchemy query for TaskInstances without clearing the tasks
+        :type get_ti_instances: bool

Review comment:
       These almost feel like it should be a different method, and clear should be made to call that instead.

##########
File path: airflow/jobs/triggerer_job.py
##########
@@ -0,0 +1,418 @@
+# 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 asyncio
+import importlib
+import os
+import signal
+import sys
+import threading
+import time
+from collections import deque
+from typing import Deque, Dict, List, Optional, Set, Tuple, Type
+
+from airflow.jobs.base_job import BaseJob
+from airflow.models.trigger import Trigger
+from airflow.triggers.base import BaseTrigger, TriggerEvent
+from airflow.typing_compat import TypedDict
+from airflow.utils.asyncio import create_task
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class TriggererJob(BaseJob, LoggingMixin):
+    """
+    TriggererJob continuously runs active triggers in asyncio, watching
+    for them to fire off their events and then dispatching that information
+    to their dependent tasks/DAGs.
+
+    It runs as two threads:
+     - The main thread does DB calls/checkins
+     - A subthread runs all the async code
+    """
+
+    __mapper_args__ = {'polymorphic_identity': 'TriggererJob'}
+
+    partition_ids: Optional[List[int]] = None
+    partition_total: Optional[int] = None
+
+    def __init__(self, partition=None, *args, **kwargs):
+        # Make sure we can actually run
+        if not hasattr(asyncio, "create_task"):
+            raise RuntimeError("The triggerer/deferred operators only work on Python 3.7 and above.")
+        # Call superclass
+        super().__init__(*args, **kwargs)
+        # Decode partition information
+        self.partition_ids, self.partition_total = None, None
+        if partition:
+            self.partition_ids, self.partition_total = self.decode_partition(partition)
+        # Set up runner async thread
+        self.runner = TriggerRunner()
+
+    def decode_partition(self, partition: str) -> Tuple[List[int], int]:
+        """
+        Given a string-format partition specification, returns the list of
+        partition IDs it represents and the partition total.
+        """
+        try:
+            # The partition format is "1,2,3/10" where the numbers before
+            # the slash are the partitions we represent, and the number
+            # after is the total number. Most users will just have a single
+            # partition number, e.g. "2/10".
+            ids_str, total_str = partition.split("/", 1)
+            partition_total = int(total_str)
+            partition_ids = []
+            for id_str in ids_str.split(","):
+                id_number = int(id_str)
+                # Bounds checking (they're 1-indexed, which might catch people out)
+                if id_number <= 0 or id_number > self.partition_total:
+                    raise ValueError(f"Partition number {id_number} is impossible")
+                self.partition_ids.append(id_number)
+        except (ValueError, TypeError):
+            raise ValueError(f"Invalid partition specification: {partition}")
+        return partition_ids, partition_total
+
+    def register_signals(self) -> None:
+        """Register signals that stop child processes"""
+        signal.signal(signal.SIGINT, self._exit_gracefully)
+        signal.signal(signal.SIGTERM, self._exit_gracefully)
+
+    def _exit_gracefully(self, signum, frame) -> None:  # pylint: disable=unused-argument
+        """Helper method to clean up processor_agent to avoid leaving orphan processes."""
+        # The first time, try to exit nicely
+        if not self.runner.stop:
+            self.log.info("Exiting gracefully upon receiving signal %s", signum)
+            self.runner.stop = True
+        else:
+            self.log.warning("Forcing exit due to second exit signal %s", signum)
+            sys.exit(os.EX_SOFTWARE)
+
+    def _execute(self) -> None:
+        # Display custom startup ack depending on plurality of partitions
+        if self.partition_ids is None:
+            self.log.info("Starting the triggerer")
+        elif len(self.partition_ids) == 1:
+            self.log.info(
+                "Starting the triggerer (partition %s of %s)", self.partition_ids[0], self.partition_total
+            )
+        else:
+            self.log.info(
+                "Starting the triggerer (partitions %s of %s)", self.partition_ids, self.partition_total
+            )
+
+        try:
+            # Kick off runner thread
+            self.runner.start()
+            # Start our own DB loop in the main thread
+            self._run_trigger_loop()
+        except Exception:  # pylint: disable=broad-except
+            self.log.exception("Exception when executing TriggererJob._run_trigger_loop")
+            raise
+        finally:
+            self.log.info("Waiting for triggers to clean up")
+            # Tell the subthread to stop and then wait for it.
+            # If the user interrupts/terms again, _graceful_exit will allow them
+            # to force-kill here.
+            self.runner.stop = True
+            self.runner.join()
+            self.log.info("Exited trigger loop")
+
+    def _run_trigger_loop(self) -> None:
+        """
+        The main-thread trigger loop.
+
+        This runs synchronously and handles all database reads/writes.
+        """
+        while not self.runner.stop:
+            # Clean out unused triggers
+            Trigger.clean_unused()
+            # Load/delete triggers
+            self.load_triggers()
+            # Handle events
+            self.handle_events()
+            # Handle failed triggers
+            self.handle_failed_triggers()
+            # Idle sleep
+            time.sleep(1)
+
+    def load_triggers(self):
+        """
+        Queries the database to get the triggers we're supposed to be running,
+        adds them to our runner, and then removes ones from it we no longer
+        need.
+        """
+        requested_trigger_ids = Trigger.runnable_ids(
+            partition_ids=self.partition_ids, partition_total=self.partition_total
+        )
+        self.runner.update_triggers(set(requested_trigger_ids))
+
+    def handle_events(self):
+        """
+        Handles outbound events from triggers - dispatching them into the Trigger
+        model where they are then pushed into the relevant task instances.
+        """
+        while self.runner.events:
+            # Get the event and its trigger ID
+            trigger_id, event = self.runner.events.popleft()
+            # Tell the model to wake up its tasks
+            Trigger.submit_event(trigger_id=trigger_id, event=event)
+
+    def handle_failed_triggers(self):
+        """
+        Handles "failed" triggers - ones that errored or exited before they
+        sent an event. Task Instances that depend on them need failing.
+        """
+        while self.runner.failed_triggers:
+            # Tell the model to fail this trigger's deps
+            trigger_id = self.runner.failed_triggers.popleft()
+            Trigger.submit_failure(trigger_id=trigger_id)
+
+
+class TriggerDetails(TypedDict):
+    """Type class for the trigger details dictionary"""
+
+    task: asyncio.Task
+    name: str
+    events: int
+
+
+class TriggerRunner(threading.Thread, LoggingMixin):
+    """
+    Runtime environment for all triggers.
+
+    Mainly runs inside its own thread, where it hands control off to an asyncio
+    event loop, but is also sometimes interacted with from the main thread
+    (where all the DB queries are done). All communication between threads is
+    done via Deques.
+    """
+
+    # Maps trigger IDs to their running tasks and other info
+    triggers: Dict[int, TriggerDetails]
+
+    # Cache for looking up triggers by classpath
+    trigger_cache: Dict[str, Type[BaseTrigger]]
+
+    # Inbound queue of new triggers
+    to_create: Deque[Tuple[int, BaseTrigger]]
+
+    # Inbound queue of deleted triggers
+    to_delete: Deque[int]
+
+    # Outbound queue of events
+    events: Deque[Tuple[int, TriggerEvent]]
+
+    # Outbound queue of failed triggers
+    failed_triggers: Deque[int]
+
+    # Should-we-stop flag
+    stop: bool = False
+
+    def __init__(self):
+        super().__init__()
+        self.triggers = {}
+        self.trigger_cache = {}
+        self.to_create = deque()
+        self.to_delete = deque()
+        self.events = deque()
+        self.failed_triggers = deque()
+
+    def run(self):
+        """Sync entrypoint - just runs arun in an async loop."""
+        # Pylint complains about this with a 3.6 base, can remove with 3.7+
+        asyncio.run(self.arun())  # pylint: disable=no-member
+
+    async def arun(self):
+        """
+        Main (asynchronous) logic loop.
+
+        The loop in here runs trigger addition/deletion/cleanup. Actual
+        triggers run in their own separate coroutines.
+        """
+        watchdog = create_task(self.block_watchdog())
+        last_status = time.time()
+        while not self.stop:
+            # Run core logic
+            await self.create_triggers()
+            await self.delete_triggers()
+            await self.cleanup_finished_triggers()
+            # Sleep for a bit
+            await asyncio.sleep(1)
+            # Every minute, log status
+            if time.time() - last_status >= 60:
+                self.log.info("%i triggers currently running", len(self.triggers))
+                last_status = time.time()
+        # Wait for watchdog to complete
+        await watchdog
+
+    async def create_triggers(self):
+        """
+        Drain the to_create queue and create all triggers that have been
+        requested in the DB that we don't yet have.
+        """
+        while self.to_create:
+            trigger_id, trigger_instance = self.to_create.popleft()
+            if trigger_id not in self.triggers:
+                self.triggers[trigger_id] = {
+                    "task": create_task(self.run_trigger(trigger_id, trigger_instance)),
+                    "name": f"{trigger_instance!r} (ID {trigger_id})",
+                    "events": 0,
+                }
+            else:
+                self.log.warning("Trigger %s had insertion attempted twice", trigger_id)
+
+    async def delete_triggers(self):
+        """
+        Drain the to_delete queue and ensure all triggers that are not in the
+        DB are cancelled, so the cleanup job deletes them.
+        """
+        while self.to_delete:
+            trigger_id = self.to_delete.popleft()
+            if trigger_id in self.triggers:
+                # We only delete if it did not exit already
+                self.triggers[trigger_id]["task"].cancel()
+
+    async def cleanup_finished_triggers(self):
+        """
+        Go through all trigger tasks (coroutines) and clean up entries for
+        ones that have exited, optionally warning users if the exit was
+        not normal.
+        """
+        for trigger_id, details in list(self.triggers.items()):  # pylint: disable=too-many-nested-blocks
+            if details["task"].done():
+                # Check to see if it exited for good reasons
+                try:
+                    result = details["task"].result()
+                except (asyncio.CancelledError, SystemExit, KeyboardInterrupt):
+                    # These are "expected" exceptions and we stop processing here
+                    # If we don't, then the system requesting a trigger be removed -
+                    # which turns into CancelledError - results in a failure.
+                    del self.triggers[trigger_id]
+                    continue
+                except BaseException as e:
+                    # This is potentially bad, so log it.
+                    self.log.error("Trigger %s exited with error %s", details["name"], e)
+                else:
+                    # See if they foolishly returned a TriggerEvent
+                    if isinstance(result, TriggerEvent):
+                        self.log.error(
+                            "Trigger %s returned a TriggerEvent rather than yielding it", details["name"]
+                        )
+                # See if this exited without sending an event, in which case
+                # any task instances depending on it need to be failed
+                if details["events"] == 0:
+                    self.log.error(
+                        "Trigger %s exited without sending an event. Dependent tasks will be failed.",
+                        details["name"],
+                    )
+                    self.failed_triggers.append(trigger_id)
+                del self.triggers[trigger_id]
+
+    async def block_watchdog(self):
+        """
+        Watchdog loop that detects blocking (badly-written) triggers.
+
+        Triggers should be well-behaved async coroutines and await whenever
+        they need to wait; this loop tries to run every 100ms to see if
+        there are badly-written triggers taking longer than that and blocking
+        the event loop.
+
+        Unfortunately, we can't tell what trigger is blocking things, but
+        we can at least detect the top-level problem.
+        """
+        while not self.stop:
+            last_run = time.monotonic()
+            await asyncio.sleep(0.1)
+            # We allow a generous amount of buffer room for now, since it might
+            # be a busy event loop.
+            time_elapsed = time.monotonic() - last_run
+            if time_elapsed > 0.2:
+                self.log.error(
+                    "Triggerer's async thread was blocked for %.2f seconds, "
+                    "likely by a badly-written trigger. Set PYTHONASYNCIODEBUG=1 "
+                    "to get more information on overrunning coroutines.",
+                    time_elapsed,
+                )
+
+    # Async trigger logic
+
+    async def run_trigger(self, trigger_id, trigger):
+        """
+        Wrapper which runs an actual trigger (they are async generators)
+        and pushes their events into our outbound event deque.
+        """
+        self.log.info("Trigger %s starting", self.triggers[trigger_id]['name'])
+        try:
+            async for event in trigger.run():
+                self.log.info("Trigger %s fired: %s", self.triggers[trigger_id]['name'], event)
+                self.triggers[trigger_id]["events"] += 1
+                self.events.append((trigger_id, event))
+        finally:
+            # CancelledError will get injected when we're stopped - which is
+            # fine, the cleanup process will understand that, but we want to
+            # allow triggers a chance to cleanup, either in that case or if
+            # they exit cleanly.
+            trigger.cleanup()
+
+    # Main-thread sync API
+
+    def update_triggers(self, requested_trigger_ids: Set[int]):
+        """
+        Called from the main thread to request that we update what
+        triggers we're running.
+
+        Works out the differences - ones to add, and ones to remove - then
+        adds them to the deques so the subthread can actually mutate the running
+        trigger set.
+        """
+        current_trigger_ids = set(self.triggers.keys())

Review comment:
       This looks like it's prone to a race condition:
   
   This code is run from the main thread, and the aio thread could also be mutating this via `cleanup_finished_triggers`.
   
   Now I'm not familiar with Python's threading access model, but I'd _guess_ this needs a lock to protect access to this variable? 

##########
File path: airflow/jobs/triggerer_job.py
##########
@@ -0,0 +1,418 @@
+# 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 asyncio
+import importlib
+import os
+import signal
+import sys
+import threading
+import time
+from collections import deque
+from typing import Deque, Dict, List, Optional, Set, Tuple, Type
+
+from airflow.jobs.base_job import BaseJob
+from airflow.models.trigger import Trigger
+from airflow.triggers.base import BaseTrigger, TriggerEvent
+from airflow.typing_compat import TypedDict
+from airflow.utils.asyncio import create_task
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class TriggererJob(BaseJob, LoggingMixin):
+    """
+    TriggererJob continuously runs active triggers in asyncio, watching
+    for them to fire off their events and then dispatching that information
+    to their dependent tasks/DAGs.
+
+    It runs as two threads:
+     - The main thread does DB calls/checkins
+     - A subthread runs all the async code
+    """
+
+    __mapper_args__ = {'polymorphic_identity': 'TriggererJob'}
+
+    partition_ids: Optional[List[int]] = None
+    partition_total: Optional[int] = None
+
+    def __init__(self, partition=None, *args, **kwargs):
+        # Make sure we can actually run
+        if not hasattr(asyncio, "create_task"):
+            raise RuntimeError("The triggerer/deferred operators only work on Python 3.7 and above.")
+        # Call superclass
+        super().__init__(*args, **kwargs)
+        # Decode partition information
+        self.partition_ids, self.partition_total = None, None
+        if partition:
+            self.partition_ids, self.partition_total = self.decode_partition(partition)
+        # Set up runner async thread
+        self.runner = TriggerRunner()
+
+    def decode_partition(self, partition: str) -> Tuple[List[int], int]:
+        """
+        Given a string-format partition specification, returns the list of
+        partition IDs it represents and the partition total.
+        """
+        try:
+            # The partition format is "1,2,3/10" where the numbers before
+            # the slash are the partitions we represent, and the number
+            # after is the total number. Most users will just have a single
+            # partition number, e.g. "2/10".
+            ids_str, total_str = partition.split("/", 1)
+            partition_total = int(total_str)
+            partition_ids = []
+            for id_str in ids_str.split(","):
+                id_number = int(id_str)
+                # Bounds checking (they're 1-indexed, which might catch people out)
+                if id_number <= 0 or id_number > self.partition_total:
+                    raise ValueError(f"Partition number {id_number} is impossible")
+                self.partition_ids.append(id_number)
+        except (ValueError, TypeError):
+            raise ValueError(f"Invalid partition specification: {partition}")
+        return partition_ids, partition_total
+
+    def register_signals(self) -> None:
+        """Register signals that stop child processes"""
+        signal.signal(signal.SIGINT, self._exit_gracefully)
+        signal.signal(signal.SIGTERM, self._exit_gracefully)
+
+    def _exit_gracefully(self, signum, frame) -> None:  # pylint: disable=unused-argument
+        """Helper method to clean up processor_agent to avoid leaving orphan processes."""
+        # The first time, try to exit nicely
+        if not self.runner.stop:
+            self.log.info("Exiting gracefully upon receiving signal %s", signum)
+            self.runner.stop = True
+        else:
+            self.log.warning("Forcing exit due to second exit signal %s", signum)
+            sys.exit(os.EX_SOFTWARE)
+
+    def _execute(self) -> None:
+        # Display custom startup ack depending on plurality of partitions
+        if self.partition_ids is None:
+            self.log.info("Starting the triggerer")
+        elif len(self.partition_ids) == 1:
+            self.log.info(
+                "Starting the triggerer (partition %s of %s)", self.partition_ids[0], self.partition_total
+            )
+        else:
+            self.log.info(
+                "Starting the triggerer (partitions %s of %s)", self.partition_ids, self.partition_total
+            )
+
+        try:
+            # Kick off runner thread
+            self.runner.start()
+            # Start our own DB loop in the main thread
+            self._run_trigger_loop()
+        except Exception:  # pylint: disable=broad-except
+            self.log.exception("Exception when executing TriggererJob._run_trigger_loop")
+            raise
+        finally:
+            self.log.info("Waiting for triggers to clean up")
+            # Tell the subthread to stop and then wait for it.
+            # If the user interrupts/terms again, _graceful_exit will allow them
+            # to force-kill here.
+            self.runner.stop = True
+            self.runner.join()
+            self.log.info("Exited trigger loop")
+
+    def _run_trigger_loop(self) -> None:
+        """
+        The main-thread trigger loop.
+
+        This runs synchronously and handles all database reads/writes.
+        """
+        while not self.runner.stop:
+            # Clean out unused triggers
+            Trigger.clean_unused()
+            # Load/delete triggers
+            self.load_triggers()
+            # Handle events
+            self.handle_events()
+            # Handle failed triggers
+            self.handle_failed_triggers()
+            # Idle sleep
+            time.sleep(1)
+
+    def load_triggers(self):
+        """
+        Queries the database to get the triggers we're supposed to be running,
+        adds them to our runner, and then removes ones from it we no longer
+        need.
+        """
+        requested_trigger_ids = Trigger.runnable_ids(
+            partition_ids=self.partition_ids, partition_total=self.partition_total
+        )
+        self.runner.update_triggers(set(requested_trigger_ids))
+
+    def handle_events(self):
+        """
+        Handles outbound events from triggers - dispatching them into the Trigger
+        model where they are then pushed into the relevant task instances.
+        """
+        while self.runner.events:
+            # Get the event and its trigger ID
+            trigger_id, event = self.runner.events.popleft()
+            # Tell the model to wake up its tasks
+            Trigger.submit_event(trigger_id=trigger_id, event=event)
+
+    def handle_failed_triggers(self):
+        """
+        Handles "failed" triggers - ones that errored or exited before they
+        sent an event. Task Instances that depend on them need failing.
+        """
+        while self.runner.failed_triggers:
+            # Tell the model to fail this trigger's deps
+            trigger_id = self.runner.failed_triggers.popleft()
+            Trigger.submit_failure(trigger_id=trigger_id)
+
+
+class TriggerDetails(TypedDict):
+    """Type class for the trigger details dictionary"""
+
+    task: asyncio.Task
+    name: str
+    events: int
+
+
+class TriggerRunner(threading.Thread, LoggingMixin):
+    """
+    Runtime environment for all triggers.
+
+    Mainly runs inside its own thread, where it hands control off to an asyncio
+    event loop, but is also sometimes interacted with from the main thread
+    (where all the DB queries are done). All communication between threads is
+    done via Deques.
+    """
+
+    # Maps trigger IDs to their running tasks and other info
+    triggers: Dict[int, TriggerDetails]
+
+    # Cache for looking up triggers by classpath
+    trigger_cache: Dict[str, Type[BaseTrigger]]
+
+    # Inbound queue of new triggers
+    to_create: Deque[Tuple[int, BaseTrigger]]
+
+    # Inbound queue of deleted triggers
+    to_delete: Deque[int]
+
+    # Outbound queue of events
+    events: Deque[Tuple[int, TriggerEvent]]
+
+    # Outbound queue of failed triggers
+    failed_triggers: Deque[int]
+
+    # Should-we-stop flag
+    stop: bool = False
+
+    def __init__(self):
+        super().__init__()
+        self.triggers = {}
+        self.trigger_cache = {}
+        self.to_create = deque()
+        self.to_delete = deque()
+        self.events = deque()
+        self.failed_triggers = deque()
+
+    def run(self):
+        """Sync entrypoint - just runs arun in an async loop."""
+        # Pylint complains about this with a 3.6 base, can remove with 3.7+
+        asyncio.run(self.arun())  # pylint: disable=no-member
+
+    async def arun(self):
+        """
+        Main (asynchronous) logic loop.
+
+        The loop in here runs trigger addition/deletion/cleanup. Actual
+        triggers run in their own separate coroutines.
+        """
+        watchdog = create_task(self.block_watchdog())
+        last_status = time.time()
+        while not self.stop:
+            # Run core logic
+            await self.create_triggers()
+            await self.delete_triggers()
+            await self.cleanup_finished_triggers()
+            # Sleep for a bit
+            await asyncio.sleep(1)
+            # Every minute, log status
+            if time.time() - last_status >= 60:
+                self.log.info("%i triggers currently running", len(self.triggers))
+                last_status = time.time()
+        # Wait for watchdog to complete
+        await watchdog
+
+    async def create_triggers(self):
+        """
+        Drain the to_create queue and create all triggers that have been
+        requested in the DB that we don't yet have.
+        """
+        while self.to_create:
+            trigger_id, trigger_instance = self.to_create.popleft()
+            if trigger_id not in self.triggers:
+                self.triggers[trigger_id] = {
+                    "task": create_task(self.run_trigger(trigger_id, trigger_instance)),
+                    "name": f"{trigger_instance!r} (ID {trigger_id})",
+                    "events": 0,
+                }
+            else:
+                self.log.warning("Trigger %s had insertion attempted twice", trigger_id)
+
+    async def delete_triggers(self):
+        """
+        Drain the to_delete queue and ensure all triggers that are not in the
+        DB are cancelled, so the cleanup job deletes them.
+        """
+        while self.to_delete:
+            trigger_id = self.to_delete.popleft()
+            if trigger_id in self.triggers:
+                # We only delete if it did not exit already
+                self.triggers[trigger_id]["task"].cancel()
+
+    async def cleanup_finished_triggers(self):
+        """
+        Go through all trigger tasks (coroutines) and clean up entries for
+        ones that have exited, optionally warning users if the exit was
+        not normal.
+        """
+        for trigger_id, details in list(self.triggers.items()):  # pylint: disable=too-many-nested-blocks
+            if details["task"].done():
+                # Check to see if it exited for good reasons
+                try:
+                    result = details["task"].result()
+                except (asyncio.CancelledError, SystemExit, KeyboardInterrupt):
+                    # These are "expected" exceptions and we stop processing here
+                    # If we don't, then the system requesting a trigger be removed -
+                    # which turns into CancelledError - results in a failure.
+                    del self.triggers[trigger_id]
+                    continue
+                except BaseException as e:
+                    # This is potentially bad, so log it.
+                    self.log.error("Trigger %s exited with error %s", details["name"], e)
+                else:
+                    # See if they foolishly returned a TriggerEvent
+                    if isinstance(result, TriggerEvent):
+                        self.log.error(
+                            "Trigger %s returned a TriggerEvent rather than yielding it", details["name"]
+                        )
+                # See if this exited without sending an event, in which case
+                # any task instances depending on it need to be failed
+                if details["events"] == 0:
+                    self.log.error(
+                        "Trigger %s exited without sending an event. Dependent tasks will be failed.",
+                        details["name"],
+                    )
+                    self.failed_triggers.append(trigger_id)
+                del self.triggers[trigger_id]
+
+    async def block_watchdog(self):
+        """
+        Watchdog loop that detects blocking (badly-written) triggers.
+
+        Triggers should be well-behaved async coroutines and await whenever
+        they need to wait; this loop tries to run every 100ms to see if
+        there are badly-written triggers taking longer than that and blocking
+        the event loop.
+
+        Unfortunately, we can't tell what trigger is blocking things, but
+        we can at least detect the top-level problem.
+        """
+        while not self.stop:
+            last_run = time.monotonic()
+            await asyncio.sleep(0.1)
+            # We allow a generous amount of buffer room for now, since it might
+            # be a busy event loop.
+            time_elapsed = time.monotonic() - last_run
+            if time_elapsed > 0.2:
+                self.log.error(
+                    "Triggerer's async thread was blocked for %.2f seconds, "
+                    "likely by a badly-written trigger. Set PYTHONASYNCIODEBUG=1 "
+                    "to get more information on overrunning coroutines.",
+                    time_elapsed,
+                )
+
+    # Async trigger logic
+
+    async def run_trigger(self, trigger_id, trigger):
+        """
+        Wrapper which runs an actual trigger (they are async generators)
+        and pushes their events into our outbound event deque.
+        """
+        self.log.info("Trigger %s starting", self.triggers[trigger_id]['name'])
+        try:
+            async for event in trigger.run():
+                self.log.info("Trigger %s fired: %s", self.triggers[trigger_id]['name'], event)
+                self.triggers[trigger_id]["events"] += 1
+                self.events.append((trigger_id, event))
+        finally:
+            # CancelledError will get injected when we're stopped - which is
+            # fine, the cleanup process will understand that, but we want to
+            # allow triggers a chance to cleanup, either in that case or if
+            # they exit cleanly.
+            trigger.cleanup()
+
+    # Main-thread sync API
+
+    def update_triggers(self, requested_trigger_ids: Set[int]):
+        """
+        Called from the main thread to request that we update what
+        triggers we're running.
+
+        Works out the differences - ones to add, and ones to remove - then
+        adds them to the deques so the subthread can actually mutate the running
+        trigger set.
+        """
+        current_trigger_ids = set(self.triggers.keys())
+        # Work out the two difference sets
+        new_trigger_ids = requested_trigger_ids.difference(current_trigger_ids)
+        old_trigger_ids = current_trigger_ids.difference(requested_trigger_ids)
+        # Bulk-fetch new trigger records
+        new_triggers = Trigger.bulk_fetch(new_trigger_ids)
+        # Add in new triggers
+        for new_id in new_trigger_ids:
+            # Check it didn't vanish in the meantime
+            if new_id not in new_triggers:
+                self.log.warning("Trigger ID %s disappeared before we could start it", new_id)
+                continue
+            # Resolve trigger record into an actual class instance
+            trigger_class = self.get_trigger_by_classpath(new_triggers[new_id].classpath)
+            self.to_create.append((new_id, trigger_class(**new_triggers[new_id].kwargs)))
+        # Remove old triggers
+        for old_id in old_trigger_ids:
+            self.to_delete.append(old_id)
+
+    def get_trigger_by_classpath(self, classpath: str) -> Type[BaseTrigger]:
+        """
+        Gets a trigger class by its classpath ("path.to.module.classname")
+
+        Uses a cache dictionary to speed up lookups after the first time.
+        """
+        if classpath not in self.trigger_cache:
+            module_name, class_name = classpath.rsplit(".", 1)
+            try:
+                module = importlib.import_module(module_name)
+            except ImportError:
+                raise ImportError(
+                    f"Cannot import trigger module {module_name} (from trigger classpath {classpath})"
+                )
+            try:
+                trigger_class = getattr(module, class_name)
+            except AttributeError:
+                raise ImportError(f"Cannot import trigger {class_name} from module {module_name}")
+            self.trigger_cache[classpath] = trigger_class

Review comment:
       `from airflow.utils.module_loading import import_string` and then
   
   ```suggestion
               self.trigger_cache[classpath] = import_string(classpath)
   ```
   
   (The exceptions would be slightly different)

##########
File path: airflow/models/dag.py
##########
@@ -1313,31 +1324,36 @@ def clear(
                             )
                         visited_external_tis.add(ti_key)
 
-        if get_tis:
-            return tis
+        if get_ti_keys:
+            return result
+
+        result_instances = session.query(TI).filter(
+            tuple_(TI.dag_id, TI.task_id, TI.execution_date).in_(result)
+        )
 
-        tis = tis.all()
+        if get_ti_instances:
+            return result_instances
 
         if dry_run:
             session.expunge_all()
-            return tis
+            return result_instances
 
         # Do not use count() here, it's actually much slower than just retrieving all the rows when
         # tis has multiple UNION statements.

Review comment:
       You've removed the UNIONs now haven't you? If so this comment is now out of date

##########
File path: airflow/models/taskinstance.py
##########
@@ -1328,22 +1389,58 @@ def _update_ti_state_for_sensing(self, session=None):
 
     def _execute_task(self, context, task_copy):
         """Executes Task (optionally with a Timeout) and pushes Xcom results"""
+        # If the task has been deferred and is being executed due to a trigger,
+        # then we need to pick the right method to come back to, otherwise
+        # we go for the default execute
+        execute_callable = task_copy.execute
+        if self.next_method:
+            execute_callable = getattr(task_copy, self.next_method)
+            if self.next_kwargs:
+                execute_callable = partial(execute_callable, **self.next_kwargs)
         # If a timeout is specified for the task, make it fail
         # if it goes beyond
         if task_copy.execution_timeout:
             try:
                 with timeout(task_copy.execution_timeout.total_seconds()):
-                    result = task_copy.execute(context=context)
+                    result = execute_callable(context=context)
             except AirflowTaskTimeout:
                 task_copy.on_kill()
                 raise
         else:
-            result = task_copy.execute(context=context)
+            result = execute_callable(context=context)
         # If the task returns a result, push an XCom containing it
         if task_copy.do_xcom_push and result is not None:
             self.xcom_push(key=XCOM_RETURN_KEY, value=result)
         return result
 
+    @provide_session
+    def _defer_task(self, session, defer: TaskDeferred):
+        """
+        Marks the task as deferred and sets up the trigger that is needed
+        to resume it.
+        """
+        from airflow.models.trigger import Trigger
+
+        # First, make the trigger entry
+        trigger_row = Trigger.from_object(defer.trigger)
+        session.add(trigger_row)
+        session.commit()

Review comment:
       https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#database-session-handling
   
   > If a function accepts a session parameter it should not commit the transaction itself. Session management is up to the caller.
   
   (And provide session will commit for us automatically if it creates the session)
   
   ```suggestion
           session.flush()
   ```

##########
File path: airflow/models/trigger.py
##########
@@ -0,0 +1,161 @@
+# 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 datetime
+from typing import Any, Dict, List, Optional
+
+import jump
+from sqlalchemy import BigInteger, Column, String, func
+
+from airflow.models.base import Base
+from airflow.models.taskinstance import TaskInstance
+from airflow.triggers.base import BaseTrigger
+from airflow.utils import timezone
+from airflow.utils.session import provide_session
+from airflow.utils.sqlalchemy import ExtendedJSON, UtcDateTime
+from airflow.utils.state import State
+
+
+class Trigger(Base):
+    """
+    Triggers are a workload that run in an asynchronous event loop shared with
+    other Triggers, and fire off events that will unpause deferred Tasks,
+    start linked DAGs, etc.
+
+    They are persisted into the database and then re-hydrated into a single
+    "triggerer" process, where they're all run at once. We model it so that
+    there is a many-to-one relationship between Task and Trigger, for future
+    deduplication logic to use.
+
+    Rows will be evicted from the database when the triggerer detects no
+    active Tasks/DAGs using them. Events are not stored in the database;
+    when an Event is fired, the triggerer will directly push its data to the
+    appropriate Task/DAG.
+    """
+
+    __tablename__ = "trigger"
+
+    id = Column(BigInteger, primary_key=True)
+    classpath = Column(String(1000), nullable=False)
+    kwargs = Column(ExtendedJSON, nullable=False)
+    created_date = Column(UtcDateTime, nullable=False)
+
+    def __init__(
+        self, classpath: str, kwargs: Dict[str, Any], created_date: Optional[datetime.datetime] = None
+    ):
+        super().__init__()
+        self.classpath = classpath
+        self.kwargs = kwargs
+        self.created_date = created_date or timezone.utcnow()
+
+    @classmethod
+    @provide_session
+    def runnable_ids(
+        cls, session=None, partition_ids: Optional[List[int]] = None, partition_total: Optional[int] = None
+    ):  # pylint: disable=unused-argument
+        """
+        Returns all "runnable" triggers IDs, optionally filtering down by partition.
+
+        This is a pretty basic partition algorithm for now, but it does the job.
+        """
+        # NOTE: It's possible in future that we could try and pre-calculate a
+        # partition entry in a large virtual ring (e.g. 4096 buckets) and store
+        # that in the DB for more direct querying, but for now Jump is fast
+        # enough of a hash to do this all locally - about 0.1s per million hashes
+
+        # Retrieve all IDs first
+        trigger_ids = [row[0] for row in session.query(cls.id).all()]
+
+        # Short-circuit for "no partitioning"
+        if partition_ids is None or partition_total is None:
+            return trigger_ids
+
+        # Go through and map each trigger ID to a partition number,
+        # using a quick, consistent hash (Jump), keeping only the ones that
+        # match one of our partition IDs
+        return [x for x in trigger_ids if jump.hash(x, partition_total) + 1 in partition_ids]
+
+    @classmethod
+    def from_object(cls, trigger: BaseTrigger):
+        """
+        Alternative constructor that creates a trigger row based directly
+        off of a Trigger object.
+        """
+        classpath, kwargs = trigger.serialize()
+        return cls(classpath=classpath, kwargs=kwargs)
+
+    @classmethod
+    @provide_session
+    def bulk_fetch(cls, ids: List[int], session=None) -> Dict[int, "Trigger"]:
+        """
+        Fetches all of the Triggers by ID and returns a dict mapping
+        ID -> Trigger instance
+        """
+        return {obj.id: obj for obj in session.query(cls).filter(cls.id.in_(ids)).all()}
+
+    @classmethod
+    @provide_session
+    def clean_unused(cls, session=None):
+        """
+        Deletes all triggers that have no tasks/DAGs dependent on them
+        (triggers have a one-to-many relationship to both)
+        """
+        # Update all task instances with trigger IDs that are not DEFERRED to remove them
+        session.query(TaskInstance).filter(
+            TaskInstance.state != State.DEFERRED, TaskInstance.trigger_id.isnot(None)
+        ).update({TaskInstance.trigger_id: None})
+        # Get all triggers that have no task instances depending on them...
+        ids = [
+            x[0]
+            for x in (

Review comment:
       ```suggestion
               trigger_id
               for (trigger_id,) in (
   ```
   
   I think is slightly easier to read.

##########
File path: airflow/models/trigger.py
##########
@@ -0,0 +1,161 @@
+# 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 datetime
+from typing import Any, Dict, List, Optional
+
+import jump
+from sqlalchemy import BigInteger, Column, String, func
+
+from airflow.models.base import Base
+from airflow.models.taskinstance import TaskInstance
+from airflow.triggers.base import BaseTrigger
+from airflow.utils import timezone
+from airflow.utils.session import provide_session
+from airflow.utils.sqlalchemy import ExtendedJSON, UtcDateTime
+from airflow.utils.state import State
+
+
+class Trigger(Base):
+    """
+    Triggers are a workload that run in an asynchronous event loop shared with
+    other Triggers, and fire off events that will unpause deferred Tasks,
+    start linked DAGs, etc.
+
+    They are persisted into the database and then re-hydrated into a single
+    "triggerer" process, where they're all run at once. We model it so that
+    there is a many-to-one relationship between Task and Trigger, for future
+    deduplication logic to use.
+
+    Rows will be evicted from the database when the triggerer detects no
+    active Tasks/DAGs using them. Events are not stored in the database;
+    when an Event is fired, the triggerer will directly push its data to the
+    appropriate Task/DAG.
+    """
+
+    __tablename__ = "trigger"
+
+    id = Column(BigInteger, primary_key=True)
+    classpath = Column(String(1000), nullable=False)
+    kwargs = Column(ExtendedJSON, nullable=False)
+    created_date = Column(UtcDateTime, nullable=False)
+
+    def __init__(
+        self, classpath: str, kwargs: Dict[str, Any], created_date: Optional[datetime.datetime] = None
+    ):
+        super().__init__()
+        self.classpath = classpath
+        self.kwargs = kwargs
+        self.created_date = created_date or timezone.utcnow()
+
+    @classmethod
+    @provide_session
+    def runnable_ids(
+        cls, session=None, partition_ids: Optional[List[int]] = None, partition_total: Optional[int] = None
+    ):  # pylint: disable=unused-argument
+        """
+        Returns all "runnable" triggers IDs, optionally filtering down by partition.
+
+        This is a pretty basic partition algorithm for now, but it does the job.
+        """
+        # NOTE: It's possible in future that we could try and pre-calculate a
+        # partition entry in a large virtual ring (e.g. 4096 buckets) and store
+        # that in the DB for more direct querying, but for now Jump is fast
+        # enough of a hash to do this all locally - about 0.1s per million hashes
+
+        # Retrieve all IDs first
+        trigger_ids = [row[0] for row in session.query(cls.id).all()]
+
+        # Short-circuit for "no partitioning"
+        if partition_ids is None or partition_total is None:
+            return trigger_ids
+
+        # Go through and map each trigger ID to a partition number,
+        # using a quick, consistent hash (Jump), keeping only the ones that
+        # match one of our partition IDs
+        return [x for x in trigger_ids if jump.hash(x, partition_total) + 1 in partition_ids]
+
+    @classmethod
+    def from_object(cls, trigger: BaseTrigger):
+        """
+        Alternative constructor that creates a trigger row based directly
+        off of a Trigger object.
+        """
+        classpath, kwargs = trigger.serialize()
+        return cls(classpath=classpath, kwargs=kwargs)
+
+    @classmethod
+    @provide_session
+    def bulk_fetch(cls, ids: List[int], session=None) -> Dict[int, "Trigger"]:
+        """
+        Fetches all of the Triggers by ID and returns a dict mapping
+        ID -> Trigger instance
+        """
+        return {obj.id: obj for obj in session.query(cls).filter(cls.id.in_(ids)).all()}
+
+    @classmethod
+    @provide_session
+    def clean_unused(cls, session=None):
+        """
+        Deletes all triggers that have no tasks/DAGs dependent on them
+        (triggers have a one-to-many relationship to both)
+        """
+        # Update all task instances with trigger IDs that are not DEFERRED to remove them
+        session.query(TaskInstance).filter(
+            TaskInstance.state != State.DEFERRED, TaskInstance.trigger_id.isnot(None)
+        ).update({TaskInstance.trigger_id: None})
+        # Get all triggers that have no task instances depending on them...
+        ids = [
+            x[0]
+            for x in (
+                session.query(cls.id)
+                .join(TaskInstance, cls.id == TaskInstance.trigger_id, isouter=True)
+                .group_by(cls.id)
+                .having(func.count(TaskInstance.trigger_id) == 0)
+            )
+        ]
+        # ...and delete them (we can't do this in one query due to MySQL)

Review comment:
       In other places we've special cased this sort of path for MySQL https://github.com/apache/airflow/blob/9f7c67feb5f2f8d3eeb81cb5f2bf158fb76f5b9e/airflow/jobs/scheduler_job.py#L785-L859 -- is it worth doing that here too?

##########
File path: airflow/models/trigger.py
##########
@@ -0,0 +1,161 @@
+# 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 datetime
+from typing import Any, Dict, List, Optional
+
+import jump
+from sqlalchemy import BigInteger, Column, String, func
+
+from airflow.models.base import Base
+from airflow.models.taskinstance import TaskInstance
+from airflow.triggers.base import BaseTrigger
+from airflow.utils import timezone
+from airflow.utils.session import provide_session
+from airflow.utils.sqlalchemy import ExtendedJSON, UtcDateTime
+from airflow.utils.state import State
+
+
+class Trigger(Base):
+    """
+    Triggers are a workload that run in an asynchronous event loop shared with
+    other Triggers, and fire off events that will unpause deferred Tasks,
+    start linked DAGs, etc.
+
+    They are persisted into the database and then re-hydrated into a single
+    "triggerer" process, where they're all run at once. We model it so that
+    there is a many-to-one relationship between Task and Trigger, for future
+    deduplication logic to use.
+
+    Rows will be evicted from the database when the triggerer detects no
+    active Tasks/DAGs using them. Events are not stored in the database;
+    when an Event is fired, the triggerer will directly push its data to the
+    appropriate Task/DAG.
+    """
+
+    __tablename__ = "trigger"
+
+    id = Column(BigInteger, primary_key=True)
+    classpath = Column(String(1000), nullable=False)
+    kwargs = Column(ExtendedJSON, nullable=False)
+    created_date = Column(UtcDateTime, nullable=False)
+
+    def __init__(
+        self, classpath: str, kwargs: Dict[str, Any], created_date: Optional[datetime.datetime] = None
+    ):
+        super().__init__()
+        self.classpath = classpath
+        self.kwargs = kwargs
+        self.created_date = created_date or timezone.utcnow()
+
+    @classmethod
+    @provide_session
+    def runnable_ids(
+        cls, session=None, partition_ids: Optional[List[int]] = None, partition_total: Optional[int] = None
+    ):  # pylint: disable=unused-argument
+        """
+        Returns all "runnable" triggers IDs, optionally filtering down by partition.
+
+        This is a pretty basic partition algorithm for now, but it does the job.
+        """
+        # NOTE: It's possible in future that we could try and pre-calculate a
+        # partition entry in a large virtual ring (e.g. 4096 buckets) and store
+        # that in the DB for more direct querying, but for now Jump is fast
+        # enough of a hash to do this all locally - about 0.1s per million hashes
+
+        # Retrieve all IDs first
+        trigger_ids = [row[0] for row in session.query(cls.id).all()]
+
+        # Short-circuit for "no partitioning"
+        if partition_ids is None or partition_total is None:
+            return trigger_ids
+
+        # Go through and map each trigger ID to a partition number,
+        # using a quick, consistent hash (Jump), keeping only the ones that
+        # match one of our partition IDs
+        return [x for x in trigger_ids if jump.hash(x, partition_total) + 1 in partition_ids]
+
+    @classmethod
+    def from_object(cls, trigger: BaseTrigger):
+        """
+        Alternative constructor that creates a trigger row based directly
+        off of a Trigger object.
+        """
+        classpath, kwargs = trigger.serialize()
+        return cls(classpath=classpath, kwargs=kwargs)
+
+    @classmethod
+    @provide_session
+    def bulk_fetch(cls, ids: List[int], session=None) -> Dict[int, "Trigger"]:
+        """
+        Fetches all of the Triggers by ID and returns a dict mapping
+        ID -> Trigger instance
+        """
+        return {obj.id: obj for obj in session.query(cls).filter(cls.id.in_(ids)).all()}
+
+    @classmethod
+    @provide_session
+    def clean_unused(cls, session=None):
+        """
+        Deletes all triggers that have no tasks/DAGs dependent on them
+        (triggers have a one-to-many relationship to both)
+        """
+        # Update all task instances with trigger IDs that are not DEFERRED to remove them
+        session.query(TaskInstance).filter(
+            TaskInstance.state != State.DEFERRED, TaskInstance.trigger_id.isnot(None)
+        ).update({TaskInstance.trigger_id: None})
+        # Get all triggers that have no task instances depending on them...
+        ids = [
+            x[0]
+            for x in (
+                session.query(cls.id)
+                .join(TaskInstance, cls.id == TaskInstance.trigger_id, isouter=True)
+                .group_by(cls.id)
+                .having(func.count(TaskInstance.trigger_id) == 0)
+            )
+        ]
+        # ...and delete them (we can't do this in one query due to MySQL)
+        session.query(Trigger).filter(Trigger.id.in_(ids)).delete(synchronize_session=False)
+
+    @classmethod
+    @provide_session
+    def submit_event(cls, trigger_id, event, session=None):
+        """
+        Takes an event from an instance of itself, and triggers all dependent
+        tasks to resume.
+        """
+        for task_instance in session.query(TaskInstance).filter(
+            TaskInstance.trigger_id == trigger_id, TaskInstance.state == State.DEFERRED
+        ):
+            # Add the event's payload into the kwargs for the task
+            next_kwargs = task_instance.next_kwargs or {}
+            next_kwargs["event"] = event.payload
+            task_instance.next_kwargs = next_kwargs
+            # Remove ourselves as its trigger
+            task_instance.trigger_id = None
+            # Finally, mark it as scheduled so it gets re-queued
+            task_instance.state = State.SCHEDULED

Review comment:
       Worth adding a `Log` row for when it gets resumed, like we had for when it gets deferred?

##########
File path: airflow/triggers/base.py
##########
@@ -0,0 +1,96 @@
+# 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 typing import Any, AsyncIterator, Dict, Tuple
+
+
+class BaseTrigger:
+    """
+    Base class for all triggers.
+
+    A trigger has two contexts it can exist in:
+
+     - As part of a DAG declaration, where it's declared.

Review comment:
       Is this true? What I've seen of the code so far Triggers only exist when they are created in side a task's execute function

##########
File path: airflow/triggers/base.py
##########
@@ -0,0 +1,96 @@
+# 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 typing import Any, AsyncIterator, Dict, Tuple
+
+
+class BaseTrigger:
+    """
+    Base class for all triggers.
+
+    A trigger has two contexts it can exist in:
+
+     - As part of a DAG declaration, where it's declared.
+     - Actively running in a trigger worker
+
+    We use the same class for both situations, and rely on all Trigger classes
+    to be able to return the (Airflow-JSON-encodable) arguments that will
+    let them be reinsantiated elsewhere.
+    """
+
+    def __init__(self):
+        pass
+
+    def serialize(self) -> Tuple[str, Dict[str, Any]]:
+        """
+        Returns the information needed to reconstruct this Trigger.
+
+        The first element of the returned tuple is the class path, the second
+        is the keyword arguments needed to re-instantiate it.

Review comment:
       ```suggestion
           :return: Tuple of (class path, keyword arguments needed to re-instantiate).
   ```

##########
File path: airflow/models/trigger.py
##########
@@ -0,0 +1,161 @@
+# 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 datetime
+from typing import Any, Dict, List, Optional
+
+import jump
+from sqlalchemy import BigInteger, Column, String, func
+
+from airflow.models.base import Base
+from airflow.models.taskinstance import TaskInstance
+from airflow.triggers.base import BaseTrigger
+from airflow.utils import timezone
+from airflow.utils.session import provide_session
+from airflow.utils.sqlalchemy import ExtendedJSON, UtcDateTime
+from airflow.utils.state import State
+
+
+class Trigger(Base):
+    """
+    Triggers are a workload that run in an asynchronous event loop shared with
+    other Triggers, and fire off events that will unpause deferred Tasks,
+    start linked DAGs, etc.
+
+    They are persisted into the database and then re-hydrated into a single
+    "triggerer" process, where they're all run at once. We model it so that
+    there is a many-to-one relationship between Task and Trigger, for future
+    deduplication logic to use.
+
+    Rows will be evicted from the database when the triggerer detects no
+    active Tasks/DAGs using them. Events are not stored in the database;
+    when an Event is fired, the triggerer will directly push its data to the
+    appropriate Task/DAG.
+    """
+
+    __tablename__ = "trigger"
+
+    id = Column(BigInteger, primary_key=True)
+    classpath = Column(String(1000), nullable=False)
+    kwargs = Column(ExtendedJSON, nullable=False)
+    created_date = Column(UtcDateTime, nullable=False)
+
+    def __init__(
+        self, classpath: str, kwargs: Dict[str, Any], created_date: Optional[datetime.datetime] = None
+    ):
+        super().__init__()
+        self.classpath = classpath
+        self.kwargs = kwargs
+        self.created_date = created_date or timezone.utcnow()
+
+    @classmethod
+    @provide_session
+    def runnable_ids(
+        cls, session=None, partition_ids: Optional[List[int]] = None, partition_total: Optional[int] = None
+    ):  # pylint: disable=unused-argument
+        """
+        Returns all "runnable" triggers IDs, optionally filtering down by partition.
+
+        This is a pretty basic partition algorithm for now, but it does the job.
+        """
+        # NOTE: It's possible in future that we could try and pre-calculate a
+        # partition entry in a large virtual ring (e.g. 4096 buckets) and store
+        # that in the DB for more direct querying, but for now Jump is fast
+        # enough of a hash to do this all locally - about 0.1s per million hashes
+
+        # Retrieve all IDs first
+        trigger_ids = [row[0] for row in session.query(cls.id).all()]
+
+        # Short-circuit for "no partitioning"
+        if partition_ids is None or partition_total is None:
+            return trigger_ids
+
+        # Go through and map each trigger ID to a partition number,
+        # using a quick, consistent hash (Jump), keeping only the ones that
+        # match one of our partition IDs
+        return [x for x in trigger_ids if jump.hash(x, partition_total) + 1 in partition_ids]
+
+    @classmethod
+    def from_object(cls, trigger: BaseTrigger):
+        """
+        Alternative constructor that creates a trigger row based directly
+        off of a Trigger object.
+        """
+        classpath, kwargs = trigger.serialize()
+        return cls(classpath=classpath, kwargs=kwargs)
+
+    @classmethod
+    @provide_session
+    def bulk_fetch(cls, ids: List[int], session=None) -> Dict[int, "Trigger"]:
+        """
+        Fetches all of the Triggers by ID and returns a dict mapping
+        ID -> Trigger instance
+        """
+        return {obj.id: obj for obj in session.query(cls).filter(cls.id.in_(ids)).all()}
+
+    @classmethod
+    @provide_session
+    def clean_unused(cls, session=None):
+        """
+        Deletes all triggers that have no tasks/DAGs dependent on them
+        (triggers have a one-to-many relationship to both)
+        """
+        # Update all task instances with trigger IDs that are not DEFERRED to remove them
+        session.query(TaskInstance).filter(
+            TaskInstance.state != State.DEFERRED, TaskInstance.trigger_id.isnot(None)
+        ).update({TaskInstance.trigger_id: None})
+        # Get all triggers that have no task instances depending on them...
+        ids = [
+            x[0]
+            for x in (
+                session.query(cls.id)
+                .join(TaskInstance, cls.id == TaskInstance.trigger_id, isouter=True)
+                .group_by(cls.id)
+                .having(func.count(TaskInstance.trigger_id) == 0)
+            )
+        ]
+        # ...and delete them (we can't do this in one query due to MySQL)
+        session.query(Trigger).filter(Trigger.id.in_(ids)).delete(synchronize_session=False)
+
+    @classmethod
+    @provide_session
+    def submit_event(cls, trigger_id, event, session=None):
+        """
+        Takes an event from an instance of itself, and triggers all dependent
+        tasks to resume.
+        """
+        for task_instance in session.query(TaskInstance).filter(
+            TaskInstance.trigger_id == trigger_id, TaskInstance.state == State.DEFERRED
+        ):
+            # Add the event's payload into the kwargs for the task
+            next_kwargs = task_instance.next_kwargs or {}
+            next_kwargs["event"] = event.payload
+            task_instance.next_kwargs = next_kwargs
+            # Remove ourselves as its trigger
+            task_instance.trigger_id = None
+            # Finally, mark it as scheduled so it gets re-queued
+            task_instance.state = State.SCHEDULED
+
+    @classmethod
+    @provide_session
+    def submit_failure(cls, trigger_id, session=None):
+        """
+        Called when a trigger has failed unexpectedly, and we need to mark
+        everything that depended on it as failed.
+        """
+        session.query(TaskInstance).filter(
+            TaskInstance.trigger_id == trigger_id, TaskInstance.state == State.DEFERRED
+        ).update({TaskInstance.state: State.FAILED})

Review comment:
       This won't call on_failure_callbacks, nor will it cascade to downstream tasks (which should end up in state upstream_failed) so this will need to be a bit more complex here.

##########
File path: airflow/triggers/temporal.py
##########
@@ -0,0 +1,76 @@
+# 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 asyncio
+import datetime
+from typing import Any, Dict, Tuple
+
+import pytz
+
+from airflow.triggers.base import BaseTrigger, TriggerEvent
+from airflow.utils import timezone
+
+
+class DateTimeTrigger(BaseTrigger):
+    """
+    A trigger that fires exactly once, at the given datetime, give or take
+    a few seconds.
+
+    The provided datetime MUST be in UTC.
+    """
+
+    def __init__(self, moment: datetime.datetime):
+        super().__init__()
+        # Make sure it's in UTC
+        if moment.tzinfo is None:
+            self.moment = pytz.utc.localize(moment)
+        elif moment.tzinfo == pytz.utc or getattr(moment.tzinfo, "name", None) == "UTC":
+            self.moment = moment
+        else:
+            raise ValueError(f"The passed datetime must be in UTC, not {moment.tzinfo!r}")
+
+    def serialize(self) -> Tuple[str, Dict[str, Any]]:
+        return ("airflow.triggers.temporal.DateTimeTrigger", {"moment": self.moment})
+
+    async def run(self):
+        """
+        Simple time delay loop until the relevant time is met.
+
+        We do have a two-phase delay to save some cycles, but sleeping is so
+        cheap anyway that it's pretty loose.
+        """
+        # Sleep an hour at a time while it's more than 2 hours away
+        while timezone.utcnow() - self.moment > datetime.timedelta(hours=2):
+            await (asyncio.sleep(3600))
+        # Sleep a second at a time otherwise
+        while self.moment > timezone.utcnow():
+            await asyncio.sleep(1)

Review comment:
       Is there a reason we don't  do this
   
   ```python
           await asyncio.sleep((timezone.utcnow() - self.moment).total_seconds())
   ```

##########
File path: airflow/triggers/temporal.py
##########
@@ -0,0 +1,76 @@
+# 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 asyncio
+import datetime
+from typing import Any, Dict, Tuple
+
+import pytz
+
+from airflow.triggers.base import BaseTrigger, TriggerEvent
+from airflow.utils import timezone
+
+
+class DateTimeTrigger(BaseTrigger):
+    """
+    A trigger that fires exactly once, at the given datetime, give or take
+    a few seconds.
+
+    The provided datetime MUST be in UTC.
+    """
+
+    def __init__(self, moment: datetime.datetime):
+        super().__init__()
+        # Make sure it's in UTC
+        if moment.tzinfo is None:
+            self.moment = pytz.utc.localize(moment)
+        elif moment.tzinfo == pytz.utc or getattr(moment.tzinfo, "name", None) == "UTC":
+            self.moment = moment
+        else:
+            raise ValueError(f"The passed datetime must be in UTC, not {moment.tzinfo!r}")
+
+    def serialize(self) -> Tuple[str, Dict[str, Any]]:
+        return ("airflow.triggers.temporal.DateTimeTrigger", {"moment": self.moment})
+
+    async def run(self):
+        """
+        Simple time delay loop until the relevant time is met.
+
+        We do have a two-phase delay to save some cycles, but sleeping is so
+        cheap anyway that it's pretty loose.
+        """
+        # Sleep an hour at a time while it's more than 2 hours away
+        while timezone.utcnow() - self.moment > datetime.timedelta(hours=2):
+            await (asyncio.sleep(3600))
+        # Sleep a second at a time otherwise
+        while self.moment > timezone.utcnow():
+            await asyncio.sleep(1)
+        # Send our single event and then we're done
+        yield TriggerEvent(self.moment)
+
+
+class TimeDeltaTrigger(DateTimeTrigger):
+    """
+    Subclass to create DateTimeTriggers based on time delays rather
+    than exact moments.
+
+    While this is its own distinct class here, it will serialise to a
+    DateTimeTrigger class, since they're operationally the same.
+    """
+
+    def __init__(self, delta: datetime.timedelta):
+        DateTimeTrigger.__init__(self, moment=timezone.utcnow() + delta)

Review comment:
       ```suggestion
           super().__init__(self, moment=timezone.utcnow() + delta)
   ```

##########
File path: docs/apache-airflow/concepts/deferring.rst
##########
@@ -0,0 +1,172 @@
+ .. 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.
+
+Deferrable Operators & Triggers
+===============================
+
+Standard :doc:`Operators <operators>` and :doc:`Sensors <sensors>` take up a full *worker slot* for the entire time they are running, even if they are idle; for example, if you only have 100 worker slots available to run Tasks, and you have 100 DAGs waiting on a Sensor that's currently running but idle, then you *cannot run anything else* - even though your entire Airflow cluster is essentially idle.

Review comment:
       This isn't strictly true for sensors in "reschedule" mode -- they will run, then stop and wait for the scheduler to send them back to a worker.

##########
File path: docs/apache-airflow/concepts/deferring.rst
##########
@@ -0,0 +1,172 @@
+ .. 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.
+
+Deferrable Operators & Triggers
+===============================
+
+Standard :doc:`Operators <operators>` and :doc:`Sensors <sensors>` take up a full *worker slot* for the entire time they are running, even if they are idle; for example, if you only have 100 worker slots available to run Tasks, and you have 100 DAGs waiting on a Sensor that's currently running but idle, then you *cannot run anything else* - even though your entire Airflow cluster is essentially idle.
+
+This is where *Deferrable Operators* come in. A deferrable operator is one that is written with the ability to suspend itself and remove itself from the worker when it knows that it will have to wait, and hand off the job of resuming it to something called a *Trigger*. As a result, while it is suspended (deferred), it is not taking up a worker slot and your cluster will have a lot less resources wasted on idle Operators or Sensors.
+
+*Triggers* are small, asynchronous pieces of Python code designed to be run all together in a single Python process; because they are asynchronous, they are able to all co-exist efficiently. As an overview of how this process works:
+
+* A task instance (running operator) gets to a point where it has to wait, and defers itself with a trigger tied to the event that should resume it. It then removes itself from its current worker and frees up space.
+* The new Trigger instance is registered inside Airflow, and picked up by one or more *triggerer* processes
+* The trigger is run until it fires, at which point its source task is re-scheduled
+* The task instance resumes

Review comment:
       ```suggestion
   * The scheduler queues the Task Instance to resume on a worker node.
   ```

##########
File path: docs/apache-airflow/concepts/deferring.rst
##########
@@ -0,0 +1,172 @@
+ .. 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.
+
+Deferrable Operators & Triggers
+===============================
+
+Standard :doc:`Operators <operators>` and :doc:`Sensors <sensors>` take up a full *worker slot* for the entire time they are running, even if they are idle; for example, if you only have 100 worker slots available to run Tasks, and you have 100 DAGs waiting on a Sensor that's currently running but idle, then you *cannot run anything else* - even though your entire Airflow cluster is essentially idle.
+
+This is where *Deferrable Operators* come in. A deferrable operator is one that is written with the ability to suspend itself and remove itself from the worker when it knows that it will have to wait, and hand off the job of resuming it to something called a *Trigger*. As a result, while it is suspended (deferred), it is not taking up a worker slot and your cluster will have a lot less resources wasted on idle Operators or Sensors.
+
+*Triggers* are small, asynchronous pieces of Python code designed to be run all together in a single Python process; because they are asynchronous, they are able to all co-exist efficiently. As an overview of how this process works:
+
+* A task instance (running operator) gets to a point where it has to wait, and defers itself with a trigger tied to the event that should resume it. It then removes itself from its current worker and frees up space.
+* The new Trigger instance is registered inside Airflow, and picked up by one or more *triggerer* processes
+* The trigger is run until it fires, at which point its source task is re-scheduled
+* The task instance resumes
+
+Using deferrable operators as a DAG author is almost transparent; writing them, however, takes a bit more work.
+
+.. note::
+
+    Deferrable Operators & Triggers rely on more recent ``asyncio`` features, and as a result only work
+    on Python 3.7 or higher.
+
+
+Using Deferrable Operators
+--------------------------
+
+If all you wish to do is use pre-written Deferrable Operators (such as ``TimeSensorAsync``, which comes with Airflow), then there are only two steps you need:
+
+* Ensure your Airflow installation is running at least one *triggerer* process, as well as the normal *scheduler*

Review comment:
       ```suggestion
   * Ensure your Airflow installation is running at least one ``triggerer`` process, as well as the normal ``scheduler``
   ```
   
   I think

##########
File path: docs/apache-airflow/concepts/deferring.rst
##########
@@ -0,0 +1,172 @@
+ .. 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.
+
+Deferrable Operators & Triggers
+===============================
+
+Standard :doc:`Operators <operators>` and :doc:`Sensors <sensors>` take up a full *worker slot* for the entire time they are running, even if they are idle; for example, if you only have 100 worker slots available to run Tasks, and you have 100 DAGs waiting on a Sensor that's currently running but idle, then you *cannot run anything else* - even though your entire Airflow cluster is essentially idle.
+
+This is where *Deferrable Operators* come in. A deferrable operator is one that is written with the ability to suspend itself and remove itself from the worker when it knows that it will have to wait, and hand off the job of resuming it to something called a *Trigger*. As a result, while it is suspended (deferred), it is not taking up a worker slot and your cluster will have a lot less resources wasted on idle Operators or Sensors.
+
+*Triggers* are small, asynchronous pieces of Python code designed to be run all together in a single Python process; because they are asynchronous, they are able to all co-exist efficiently. As an overview of how this process works:
+
+* A task instance (running operator) gets to a point where it has to wait, and defers itself with a trigger tied to the event that should resume it. It then removes itself from its current worker and frees up space.
+* The new Trigger instance is registered inside Airflow, and picked up by one or more *triggerer* processes
+* The trigger is run until it fires, at which point its source task is re-scheduled
+* The task instance resumes
+
+Using deferrable operators as a DAG author is almost transparent; writing them, however, takes a bit more work.
+
+.. note::
+
+    Deferrable Operators & Triggers rely on more recent ``asyncio`` features, and as a result only work
+    on Python 3.7 or higher.
+
+
+Using Deferrable Operators
+--------------------------
+
+If all you wish to do is use pre-written Deferrable Operators (such as ``TimeSensorAsync``, which comes with Airflow), then there are only two steps you need:
+
+* Ensure your Airflow installation is running at least one *triggerer* process, as well as the normal *scheduler*
+* Use deferrable operators/sensors in your DAGs
+
+That's it; everything else will be automatically handled for you. If you're upgrading existing DAGs, we even provide some API-compatible sensor variants (e.g. ``TimeSensorAsync`` for ``TimeSensor``) that you can swap into your DAG with no other changes required.
+
+Note that you cannot yet use the deferral ability from inside custom PythonOperator/TaskFlow code; it is only available to pre-built Operators at the moment.
+
+
+Writing Deferrable Operators
+----------------------------
+
+Writing a deferrable operator takes a bit more work. There are some main points to consider:
+
+* Your Operator must defer itself based on a Trigger. If there is a Trigger in core Airflow you can use, great; otherwise, you will have to write one.
+* Your Operator will be deleted and removed from its worker while deferred, and no state will persist automatically. You can persist state by asking Airflow to resume you at a certain method or pass certain kwargs, but that's it.

Review comment:
       ```suggestion
   * Your Operator will be stopped and removed from its worker while deferred, and no state will persist automatically. You can persist state by asking Airflow to resume you at a certain method or pass certain kwargs, but that's it.
   ```

##########
File path: docs/apache-airflow/concepts/deferring.rst
##########
@@ -0,0 +1,172 @@
+ .. 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.
+
+Deferrable Operators & Triggers
+===============================
+
+Standard :doc:`Operators <operators>` and :doc:`Sensors <sensors>` take up a full *worker slot* for the entire time they are running, even if they are idle; for example, if you only have 100 worker slots available to run Tasks, and you have 100 DAGs waiting on a Sensor that's currently running but idle, then you *cannot run anything else* - even though your entire Airflow cluster is essentially idle.
+
+This is where *Deferrable Operators* come in. A deferrable operator is one that is written with the ability to suspend itself and remove itself from the worker when it knows that it will have to wait, and hand off the job of resuming it to something called a *Trigger*. As a result, while it is suspended (deferred), it is not taking up a worker slot and your cluster will have a lot less resources wasted on idle Operators or Sensors.
+
+*Triggers* are small, asynchronous pieces of Python code designed to be run all together in a single Python process; because they are asynchronous, they are able to all co-exist efficiently. As an overview of how this process works:
+
+* A task instance (running operator) gets to a point where it has to wait, and defers itself with a trigger tied to the event that should resume it. It then removes itself from its current worker and frees up space.
+* The new Trigger instance is registered inside Airflow, and picked up by one or more *triggerer* processes
+* The trigger is run until it fires, at which point its source task is re-scheduled
+* The task instance resumes
+
+Using deferrable operators as a DAG author is almost transparent; writing them, however, takes a bit more work.
+
+.. note::
+
+    Deferrable Operators & Triggers rely on more recent ``asyncio`` features, and as a result only work
+    on Python 3.7 or higher.
+
+
+Using Deferrable Operators
+--------------------------
+
+If all you wish to do is use pre-written Deferrable Operators (such as ``TimeSensorAsync``, which comes with Airflow), then there are only two steps you need:
+
+* Ensure your Airflow installation is running at least one *triggerer* process, as well as the normal *scheduler*
+* Use deferrable operators/sensors in your DAGs
+
+That's it; everything else will be automatically handled for you. If you're upgrading existing DAGs, we even provide some API-compatible sensor variants (e.g. ``TimeSensorAsync`` for ``TimeSensor``) that you can swap into your DAG with no other changes required.
+
+Note that you cannot yet use the deferral ability from inside custom PythonOperator/TaskFlow code; it is only available to pre-built Operators at the moment.

Review comment:
       ```suggestion
   Note that you cannot yet use the deferral ability from inside custom PythonOperator/TaskFlow python functions; it is only available to pre-built Operators at the moment.
   ```
   
   (Cos you can use deferabble operators inside a DAG that uses task flow, you just can't defer a `@task` function, right?)

##########
File path: docs/apache-airflow/concepts/deferring.rst
##########
@@ -0,0 +1,172 @@
+ .. 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.
+
+Deferrable Operators & Triggers
+===============================
+
+Standard :doc:`Operators <operators>` and :doc:`Sensors <sensors>` take up a full *worker slot* for the entire time they are running, even if they are idle; for example, if you only have 100 worker slots available to run Tasks, and you have 100 DAGs waiting on a Sensor that's currently running but idle, then you *cannot run anything else* - even though your entire Airflow cluster is essentially idle.
+
+This is where *Deferrable Operators* come in. A deferrable operator is one that is written with the ability to suspend itself and remove itself from the worker when it knows that it will have to wait, and hand off the job of resuming it to something called a *Trigger*. As a result, while it is suspended (deferred), it is not taking up a worker slot and your cluster will have a lot less resources wasted on idle Operators or Sensors.
+
+*Triggers* are small, asynchronous pieces of Python code designed to be run all together in a single Python process; because they are asynchronous, they are able to all co-exist efficiently. As an overview of how this process works:
+
+* A task instance (running operator) gets to a point where it has to wait, and defers itself with a trigger tied to the event that should resume it. It then removes itself from its current worker and frees up space.
+* The new Trigger instance is registered inside Airflow, and picked up by one or more *triggerer* processes
+* The trigger is run until it fires, at which point its source task is re-scheduled
+* The task instance resumes
+
+Using deferrable operators as a DAG author is almost transparent; writing them, however, takes a bit more work.
+
+.. note::
+
+    Deferrable Operators & Triggers rely on more recent ``asyncio`` features, and as a result only work
+    on Python 3.7 or higher.
+
+
+Using Deferrable Operators
+--------------------------
+
+If all you wish to do is use pre-written Deferrable Operators (such as ``TimeSensorAsync``, which comes with Airflow), then there are only two steps you need:
+
+* Ensure your Airflow installation is running at least one *triggerer* process, as well as the normal *scheduler*
+* Use deferrable operators/sensors in your DAGs
+
+That's it; everything else will be automatically handled for you. If you're upgrading existing DAGs, we even provide some API-compatible sensor variants (e.g. ``TimeSensorAsync`` for ``TimeSensor``) that you can swap into your DAG with no other changes required.
+
+Note that you cannot yet use the deferral ability from inside custom PythonOperator/TaskFlow code; it is only available to pre-built Operators at the moment.
+
+
+Writing Deferrable Operators
+----------------------------
+
+Writing a deferrable operator takes a bit more work. There are some main points to consider:
+
+* Your Operator must defer itself based on a Trigger. If there is a Trigger in core Airflow you can use, great; otherwise, you will have to write one.
+* Your Operator will be deleted and removed from its worker while deferred, and no state will persist automatically. You can persist state by asking Airflow to resume you at a certain method or pass certain kwargs, but that's it.
+* You can defer multiple times, and you can defer before/after your Operator does significant work, or only defer if certain conditions are met (e.g. a system does not have an immediate answer). Deferral is entirely under your control.
+* Any Operator can defer; no special marking on its class is needed, and it's not limited to Sensors.
+
+
+Triggering Deferral
+~~~~~~~~~~~~~~~~~~~
+
+If you want to trigger deferral, at any place in your Operator you can call ``self.defer(trigger, method_name, kwargs, timeout)``, which will raise a special exception that Airflow will catch. The arguments are:
+
+* ``trigger``: An instance of a Trigger that you wish to defer on. It will be serialized into the database.
+* ``method_name``: The method name on your Operator you want Airflow to call when it resumes, other than ``execute``.
+* ``kwargs``: Additional keyword arguments to pass to the method when it is called. Optional, defaults to ``{}``.
+* ``timeout``: A timedelta that specifies a timeout after which this deferral will fail, and fail the task instance. Optional, defaults to ``None``, meaning no timeout.
+
+When you opt to defer, your Operator will *stop executing at that point and be deleted from a worker*. No state will persist, and when your Operator is resumed it will be a *brand new instance* of it. The only way you can pass state from the old instance of the Operator to the new one is via ``method_name`` and ``kwargs``.

Review comment:
       We shouldn't use "deleted" as it could be confused with deleting the DB row.

##########
File path: docs/apache-airflow/concepts/deferring.rst
##########
@@ -0,0 +1,172 @@
+ .. 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.
+
+Deferrable Operators & Triggers
+===============================
+
+Standard :doc:`Operators <operators>` and :doc:`Sensors <sensors>` take up a full *worker slot* for the entire time they are running, even if they are idle; for example, if you only have 100 worker slots available to run Tasks, and you have 100 DAGs waiting on a Sensor that's currently running but idle, then you *cannot run anything else* - even though your entire Airflow cluster is essentially idle.
+
+This is where *Deferrable Operators* come in. A deferrable operator is one that is written with the ability to suspend itself and remove itself from the worker when it knows that it will have to wait, and hand off the job of resuming it to something called a *Trigger*. As a result, while it is suspended (deferred), it is not taking up a worker slot and your cluster will have a lot less resources wasted on idle Operators or Sensors.
+
+*Triggers* are small, asynchronous pieces of Python code designed to be run all together in a single Python process; because they are asynchronous, they are able to all co-exist efficiently. As an overview of how this process works:
+
+* A task instance (running operator) gets to a point where it has to wait, and defers itself with a trigger tied to the event that should resume it. It then removes itself from its current worker and frees up space.

Review comment:
       > It then removes itself from its current worker and frees up space.
   
   This sounds like something the each operator has to perform, but that isn't the case
   
   ```suggestion
   * A task instance (running operator) gets to a point where it has to wait, and defers itself with a trigger tied to the event that should resume it. The worker is then free to execute another task in that slot.
   ```

##########
File path: airflow/models/dag.py
##########
@@ -1232,23 +1253,13 @@ def clear(
                 )
             )
 
-        if start_date:
-            tis = tis.filter(TI.execution_date >= start_date)
-        if end_date:
-            tis = tis.filter(TI.execution_date <= end_date)
-        if only_failed:
-            tis = tis.filter(or_(TI.state == State.FAILED, TI.state == State.UPSTREAM_FAILED))
-        if only_running:
-            tis = tis.filter(TI.state == State.RUNNING)
-        if task_ids:
-            tis = tis.filter(TI.task_id.in_(task_ids))
-
         if include_subdags:
             from airflow.sensors.external_task import ExternalTaskMarker
 
             # Recursively find external tasks indicated by ExternalTaskMarker
-            instances = tis.all()
-            for ti in instances:
+            for ti in (
+                session.query(TI).filter(tuple_(TI.dag_id, TI.task_id, TI.execution_date).in_(tis)).all()

Review comment:
       mssql doesn't like this "in tuple" approach, and we are working on adding support for mssql in https://github.com/apache/airflow/pull/9973 so this will need to change/have db-specific paths.

##########
File path: airflow/models/taskinstance.py
##########
@@ -282,6 +296,18 @@ class TaskInstance(Base, LoggingMixin):  # pylint: disable=R0902,R0904
     executor_config = Column(PickleType(pickler=dill))
 
     external_executor_id = Column(String(ID_LEN, **COLLATION_ARGS))
+
+    # The trigger to resume on if we are in state DEFERRED
+    trigger_id = Column(BigInteger)
+
+    # Optional timeout datetime for the trigger (past this, we'll fail)
+    trigger_timeout = Column(UtcDateTime)
+
+    # The method to call next, and any extra arguments to pass to it.
+    # Usually used when resuming from DEFERRED.
+    next_method = Column(String(1000))

Review comment:
       ```suggestion
       next_method = Column(String(1000, **COLLATION_ARGS))
   ```
   
   (And similarly in the migration too I think?)

##########
File path: airflow/models/taskinstance.py
##########
@@ -1137,6 +1180,24 @@ def _run_raw_task(
                 self._prepare_and_execute_task_with_callbacks(context, task)
             self.refresh_from_db(lock_for_update=True)
             self.state = State.SUCCESS
+        except TaskDeferred as defer:
+            # The task has signalled it wants to defer execution based on
+            # a trigger.
+            self._defer_task(defer=defer)
+            self.log.info(self.state)
+            self.log.info(self.next_method)

Review comment:
       Left over debug logging?
   
   ```suggestion
   ```

##########
File path: airflow/triggers/temporal.py
##########
@@ -0,0 +1,76 @@
+# 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 asyncio
+import datetime
+from typing import Any, Dict, Tuple
+
+import pytz

Review comment:
       We've generally used pendulum for all datetime/timezone shenanigans  -- could you see if it does what you need please?

##########
File path: airflow/triggers/base.py
##########
@@ -0,0 +1,96 @@
+# 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 typing import Any, AsyncIterator, Dict, Tuple
+
+
+class BaseTrigger:

Review comment:
       ```suggestion
   class BaseTrigger(abc.ABC):
   ```
   
   And then using `@abstractmethod` where appropriate?

##########
File path: airflow/sensors/time_sensor.py
##########
@@ -35,3 +36,25 @@ def __init__(self, *, target_time, **kwargs):
     def poke(self, context):
         self.log.info('Checking if the time (%s) has come', self.target_time)
         return timezone.make_naive(timezone.utcnow(), self.dag.timezone).time() > self.target_time
+
+
+class TimeSensorAsync(BaseSensorOperator):
+    """
+    Waits until the specified time of the day, freeing up a worker slot while
+    it is waiting.
+
+    :param target_time: time after which the job succeeds
+    :type target_time: datetime.time
+    """
+
+    def __init__(self, *, target_time, **kwargs):
+        super().__init__(**kwargs)
+        self.target_time = target_time
+
+    def execute(self, context):
+        self.log.info("Kicking off trigger deferral")

Review comment:
       Debugging log?
   
   ```suggestion
   ```

##########
File path: docs/apache-airflow/concepts/deferring.rst
##########
@@ -0,0 +1,172 @@
+ .. 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.
+
+Deferrable Operators & Triggers
+===============================
+
+Standard :doc:`Operators <operators>` and :doc:`Sensors <sensors>` take up a full *worker slot* for the entire time they are running, even if they are idle; for example, if you only have 100 worker slots available to run Tasks, and you have 100 DAGs waiting on a Sensor that's currently running but idle, then you *cannot run anything else* - even though your entire Airflow cluster is essentially idle.
+
+This is where *Deferrable Operators* come in. A deferrable operator is one that is written with the ability to suspend itself and remove itself from the worker when it knows that it will have to wait, and hand off the job of resuming it to something called a *Trigger*. As a result, while it is suspended (deferred), it is not taking up a worker slot and your cluster will have a lot less resources wasted on idle Operators or Sensors.
+
+*Triggers* are small, asynchronous pieces of Python code designed to be run all together in a single Python process; because they are asynchronous, they are able to all co-exist efficiently. As an overview of how this process works:
+
+* A task instance (running operator) gets to a point where it has to wait, and defers itself with a trigger tied to the event that should resume it. It then removes itself from its current worker and frees up space.
+* The new Trigger instance is registered inside Airflow, and picked up by one or more *triggerer* processes
+* The trigger is run until it fires, at which point its source task is re-scheduled
+* The task instance resumes
+
+Using deferrable operators as a DAG author is almost transparent; writing them, however, takes a bit more work.
+
+.. note::
+
+    Deferrable Operators & Triggers rely on more recent ``asyncio`` features, and as a result only work
+    on Python 3.7 or higher.
+
+
+Using Deferrable Operators
+--------------------------
+
+If all you wish to do is use pre-written Deferrable Operators (such as ``TimeSensorAsync``, which comes with Airflow), then there are only two steps you need:
+
+* Ensure your Airflow installation is running at least one *triggerer* process, as well as the normal *scheduler*
+* Use deferrable operators/sensors in your DAGs
+
+That's it; everything else will be automatically handled for you. If you're upgrading existing DAGs, we even provide some API-compatible sensor variants (e.g. ``TimeSensorAsync`` for ``TimeSensor``) that you can swap into your DAG with no other changes required.
+
+Note that you cannot yet use the deferral ability from inside custom PythonOperator/TaskFlow code; it is only available to pre-built Operators at the moment.
+
+
+Writing Deferrable Operators
+----------------------------
+
+Writing a deferrable operator takes a bit more work. There are some main points to consider:
+
+* Your Operator must defer itself based on a Trigger. If there is a Trigger in core Airflow you can use, great; otherwise, you will have to write one.
+* Your Operator will be deleted and removed from its worker while deferred, and no state will persist automatically. You can persist state by asking Airflow to resume you at a certain method or pass certain kwargs, but that's it.
+* You can defer multiple times, and you can defer before/after your Operator does significant work, or only defer if certain conditions are met (e.g. a system does not have an immediate answer). Deferral is entirely under your control.
+* Any Operator can defer; no special marking on its class is needed, and it's not limited to Sensors.
+
+
+Triggering Deferral
+~~~~~~~~~~~~~~~~~~~
+
+If you want to trigger deferral, at any place in your Operator you can call ``self.defer(trigger, method_name, kwargs, timeout)``, which will raise a special exception that Airflow will catch. The arguments are:
+
+* ``trigger``: An instance of a Trigger that you wish to defer on. It will be serialized into the database.
+* ``method_name``: The method name on your Operator you want Airflow to call when it resumes, other than ``execute``.
+* ``kwargs``: Additional keyword arguments to pass to the method when it is called. Optional, defaults to ``{}``.
+* ``timeout``: A timedelta that specifies a timeout after which this deferral will fail, and fail the task instance. Optional, defaults to ``None``, meaning no timeout.

Review comment:
       We need to document how this timeout and the existing execution_timeout interact.
   
   (I _think_ that `execution_timeout` should be a total ceiling of runtime, i.e. time since task first started, right?)




-- 
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:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r639632711



##########
File path: airflow/models/trigger.py
##########
@@ -0,0 +1,161 @@
+# 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 datetime
+from typing import Any, Dict, List, Optional
+
+import jump
+from sqlalchemy import BigInteger, Column, String, func
+
+from airflow.models.base import Base
+from airflow.models.taskinstance import TaskInstance
+from airflow.triggers.base import BaseTrigger
+from airflow.utils import timezone
+from airflow.utils.session import provide_session
+from airflow.utils.sqlalchemy import ExtendedJSON, UtcDateTime
+from airflow.utils.state import State
+
+
+class Trigger(Base):
+    """
+    Triggers are a workload that run in an asynchronous event loop shared with
+    other Triggers, and fire off events that will unpause deferred Tasks,
+    start linked DAGs, etc.
+
+    They are persisted into the database and then re-hydrated into a single
+    "triggerer" process, where they're all run at once. We model it so that
+    there is a many-to-one relationship between Task and Trigger, for future
+    deduplication logic to use.
+
+    Rows will be evicted from the database when the triggerer detects no
+    active Tasks/DAGs using them. Events are not stored in the database;
+    when an Event is fired, the triggerer will directly push its data to the
+    appropriate Task/DAG.
+    """
+
+    __tablename__ = "trigger"
+
+    id = Column(BigInteger, primary_key=True)
+    classpath = Column(String(1000), nullable=False)
+    kwargs = Column(ExtendedJSON, nullable=False)
+    created_date = Column(UtcDateTime, nullable=False)
+
+    def __init__(
+        self, classpath: str, kwargs: Dict[str, Any], created_date: Optional[datetime.datetime] = None
+    ):
+        super().__init__()
+        self.classpath = classpath
+        self.kwargs = kwargs
+        self.created_date = created_date or timezone.utcnow()
+
+    @classmethod
+    @provide_session
+    def runnable_ids(
+        cls, session=None, partition_ids: Optional[List[int]] = None, partition_total: Optional[int] = None
+    ):  # pylint: disable=unused-argument
+        """
+        Returns all "runnable" triggers IDs, optionally filtering down by partition.
+
+        This is a pretty basic partition algorithm for now, but it does the job.
+        """
+        # NOTE: It's possible in future that we could try and pre-calculate a
+        # partition entry in a large virtual ring (e.g. 4096 buckets) and store
+        # that in the DB for more direct querying, but for now Jump is fast
+        # enough of a hash to do this all locally - about 0.1s per million hashes
+
+        # Retrieve all IDs first
+        trigger_ids = [row[0] for row in session.query(cls.id).all()]
+
+        # Short-circuit for "no partitioning"
+        if partition_ids is None or partition_total is None:
+            return trigger_ids
+
+        # Go through and map each trigger ID to a partition number,
+        # using a quick, consistent hash (Jump), keeping only the ones that
+        # match one of our partition IDs
+        return [x for x in trigger_ids if jump.hash(x, partition_total) + 1 in partition_ids]
+
+    @classmethod
+    def from_object(cls, trigger: BaseTrigger):
+        """
+        Alternative constructor that creates a trigger row based directly
+        off of a Trigger object.
+        """
+        classpath, kwargs = trigger.serialize()
+        return cls(classpath=classpath, kwargs=kwargs)
+
+    @classmethod
+    @provide_session
+    def bulk_fetch(cls, ids: List[int], session=None) -> Dict[int, "Trigger"]:
+        """
+        Fetches all of the Triggers by ID and returns a dict mapping
+        ID -> Trigger instance
+        """
+        return {obj.id: obj for obj in session.query(cls).filter(cls.id.in_(ids)).all()}
+
+    @classmethod
+    @provide_session
+    def clean_unused(cls, session=None):
+        """
+        Deletes all triggers that have no tasks/DAGs dependent on them
+        (triggers have a one-to-many relationship to both)
+        """
+        # Update all task instances with trigger IDs that are not DEFERRED to remove them
+        session.query(TaskInstance).filter(
+            TaskInstance.state != State.DEFERRED, TaskInstance.trigger_id.isnot(None)
+        ).update({TaskInstance.trigger_id: None})
+        # Get all triggers that have no task instances depending on them...
+        ids = [
+            x[0]
+            for x in (
+                session.query(cls.id)
+                .join(TaskInstance, cls.id == TaskInstance.trigger_id, isouter=True)
+                .group_by(cls.id)
+                .having(func.count(TaskInstance.trigger_id) == 0)
+            )
+        ]
+        # ...and delete them (we can't do this in one query due to MySQL)
+        session.query(Trigger).filter(Trigger.id.in_(ids)).delete(synchronize_session=False)
+
+    @classmethod
+    @provide_session
+    def submit_event(cls, trigger_id, event, session=None):
+        """
+        Takes an event from an instance of itself, and triggers all dependent
+        tasks to resume.
+        """
+        for task_instance in session.query(TaskInstance).filter(
+            TaskInstance.trigger_id == trigger_id, TaskInstance.state == State.DEFERRED
+        ):
+            # Add the event's payload into the kwargs for the task
+            next_kwargs = task_instance.next_kwargs or {}
+            next_kwargs["event"] = event.payload
+            task_instance.next_kwargs = next_kwargs
+            # Remove ourselves as its trigger
+            task_instance.trigger_id = None
+            # Finally, mark it as scheduled so it gets re-queued
+            task_instance.state = State.SCHEDULED

Review comment:
       That'll do, so leave it for now and we can think about a wider rebuild/re-work of the audit/tracing framework wholesale at a future time.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638979628



##########
File path: airflow/models/trigger.py
##########
@@ -0,0 +1,161 @@
+# 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 datetime
+from typing import Any, Dict, List, Optional
+
+import jump
+from sqlalchemy import BigInteger, Column, String, func
+
+from airflow.models.base import Base
+from airflow.models.taskinstance import TaskInstance
+from airflow.triggers.base import BaseTrigger
+from airflow.utils import timezone
+from airflow.utils.session import provide_session
+from airflow.utils.sqlalchemy import ExtendedJSON, UtcDateTime
+from airflow.utils.state import State
+
+
+class Trigger(Base):
+    """
+    Triggers are a workload that run in an asynchronous event loop shared with
+    other Triggers, and fire off events that will unpause deferred Tasks,
+    start linked DAGs, etc.
+
+    They are persisted into the database and then re-hydrated into a single
+    "triggerer" process, where they're all run at once. We model it so that
+    there is a many-to-one relationship between Task and Trigger, for future
+    deduplication logic to use.
+
+    Rows will be evicted from the database when the triggerer detects no
+    active Tasks/DAGs using them. Events are not stored in the database;
+    when an Event is fired, the triggerer will directly push its data to the
+    appropriate Task/DAG.
+    """
+
+    __tablename__ = "trigger"
+
+    id = Column(BigInteger, primary_key=True)
+    classpath = Column(String(1000), nullable=False)
+    kwargs = Column(ExtendedJSON, nullable=False)
+    created_date = Column(UtcDateTime, nullable=False)
+
+    def __init__(
+        self, classpath: str, kwargs: Dict[str, Any], created_date: Optional[datetime.datetime] = None
+    ):
+        super().__init__()
+        self.classpath = classpath
+        self.kwargs = kwargs
+        self.created_date = created_date or timezone.utcnow()
+
+    @classmethod
+    @provide_session
+    def runnable_ids(
+        cls, session=None, partition_ids: Optional[List[int]] = None, partition_total: Optional[int] = None
+    ):  # pylint: disable=unused-argument
+        """
+        Returns all "runnable" triggers IDs, optionally filtering down by partition.
+
+        This is a pretty basic partition algorithm for now, but it does the job.
+        """
+        # NOTE: It's possible in future that we could try and pre-calculate a
+        # partition entry in a large virtual ring (e.g. 4096 buckets) and store
+        # that in the DB for more direct querying, but for now Jump is fast
+        # enough of a hash to do this all locally - about 0.1s per million hashes
+
+        # Retrieve all IDs first
+        trigger_ids = [row[0] for row in session.query(cls.id).all()]
+
+        # Short-circuit for "no partitioning"
+        if partition_ids is None or partition_total is None:
+            return trigger_ids
+
+        # Go through and map each trigger ID to a partition number,
+        # using a quick, consistent hash (Jump), keeping only the ones that
+        # match one of our partition IDs
+        return [x for x in trigger_ids if jump.hash(x, partition_total) + 1 in partition_ids]
+
+    @classmethod
+    def from_object(cls, trigger: BaseTrigger):
+        """
+        Alternative constructor that creates a trigger row based directly
+        off of a Trigger object.
+        """
+        classpath, kwargs = trigger.serialize()
+        return cls(classpath=classpath, kwargs=kwargs)
+
+    @classmethod
+    @provide_session
+    def bulk_fetch(cls, ids: List[int], session=None) -> Dict[int, "Trigger"]:
+        """
+        Fetches all of the Triggers by ID and returns a dict mapping
+        ID -> Trigger instance
+        """
+        return {obj.id: obj for obj in session.query(cls).filter(cls.id.in_(ids)).all()}
+
+    @classmethod
+    @provide_session
+    def clean_unused(cls, session=None):
+        """
+        Deletes all triggers that have no tasks/DAGs dependent on them
+        (triggers have a one-to-many relationship to both)
+        """
+        # Update all task instances with trigger IDs that are not DEFERRED to remove them
+        session.query(TaskInstance).filter(
+            TaskInstance.state != State.DEFERRED, TaskInstance.trigger_id.isnot(None)
+        ).update({TaskInstance.trigger_id: None})
+        # Get all triggers that have no task instances depending on them...
+        ids = [
+            x[0]
+            for x in (
+                session.query(cls.id)
+                .join(TaskInstance, cls.id == TaskInstance.trigger_id, isouter=True)
+                .group_by(cls.id)
+                .having(func.count(TaskInstance.trigger_id) == 0)
+            )
+        ]
+        # ...and delete them (we can't do this in one query due to MySQL)

Review comment:
       I don't think so - I very much doubt any query planner is going to be doing better than "run the subquery and then copy the IDs across" anyway, we're just adding a little network latency.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638979016



##########
File path: airflow/models/trigger.py
##########
@@ -0,0 +1,161 @@
+# 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 datetime
+from typing import Any, Dict, List, Optional
+
+import jump
+from sqlalchemy import BigInteger, Column, String, func
+
+from airflow.models.base import Base
+from airflow.models.taskinstance import TaskInstance
+from airflow.triggers.base import BaseTrigger
+from airflow.utils import timezone
+from airflow.utils.session import provide_session
+from airflow.utils.sqlalchemy import ExtendedJSON, UtcDateTime
+from airflow.utils.state import State
+
+
+class Trigger(Base):
+    """
+    Triggers are a workload that run in an asynchronous event loop shared with
+    other Triggers, and fire off events that will unpause deferred Tasks,
+    start linked DAGs, etc.
+
+    They are persisted into the database and then re-hydrated into a single
+    "triggerer" process, where they're all run at once. We model it so that
+    there is a many-to-one relationship between Task and Trigger, for future
+    deduplication logic to use.
+
+    Rows will be evicted from the database when the triggerer detects no
+    active Tasks/DAGs using them. Events are not stored in the database;
+    when an Event is fired, the triggerer will directly push its data to the
+    appropriate Task/DAG.
+    """
+
+    __tablename__ = "trigger"
+
+    id = Column(BigInteger, primary_key=True)
+    classpath = Column(String(1000), nullable=False)
+    kwargs = Column(ExtendedJSON, nullable=False)
+    created_date = Column(UtcDateTime, nullable=False)
+
+    def __init__(
+        self, classpath: str, kwargs: Dict[str, Any], created_date: Optional[datetime.datetime] = None
+    ):
+        super().__init__()
+        self.classpath = classpath
+        self.kwargs = kwargs
+        self.created_date = created_date or timezone.utcnow()
+
+    @classmethod
+    @provide_session
+    def runnable_ids(
+        cls, session=None, partition_ids: Optional[List[int]] = None, partition_total: Optional[int] = None
+    ):  # pylint: disable=unused-argument
+        """
+        Returns all "runnable" triggers IDs, optionally filtering down by partition.
+
+        This is a pretty basic partition algorithm for now, but it does the job.
+        """
+        # NOTE: It's possible in future that we could try and pre-calculate a
+        # partition entry in a large virtual ring (e.g. 4096 buckets) and store
+        # that in the DB for more direct querying, but for now Jump is fast
+        # enough of a hash to do this all locally - about 0.1s per million hashes
+
+        # Retrieve all IDs first
+        trigger_ids = [row[0] for row in session.query(cls.id).all()]
+
+        # Short-circuit for "no partitioning"
+        if partition_ids is None or partition_total is None:
+            return trigger_ids
+
+        # Go through and map each trigger ID to a partition number,
+        # using a quick, consistent hash (Jump), keeping only the ones that
+        # match one of our partition IDs
+        return [x for x in trigger_ids if jump.hash(x, partition_total) + 1 in partition_ids]
+
+    @classmethod
+    def from_object(cls, trigger: BaseTrigger):
+        """
+        Alternative constructor that creates a trigger row based directly
+        off of a Trigger object.
+        """
+        classpath, kwargs = trigger.serialize()
+        return cls(classpath=classpath, kwargs=kwargs)
+
+    @classmethod
+    @provide_session
+    def bulk_fetch(cls, ids: List[int], session=None) -> Dict[int, "Trigger"]:
+        """
+        Fetches all of the Triggers by ID and returns a dict mapping
+        ID -> Trigger instance
+        """
+        return {obj.id: obj for obj in session.query(cls).filter(cls.id.in_(ids)).all()}
+
+    @classmethod
+    @provide_session
+    def clean_unused(cls, session=None):
+        """
+        Deletes all triggers that have no tasks/DAGs dependent on them
+        (triggers have a one-to-many relationship to both)
+        """
+        # Update all task instances with trigger IDs that are not DEFERRED to remove them
+        session.query(TaskInstance).filter(
+            TaskInstance.state != State.DEFERRED, TaskInstance.trigger_id.isnot(None)
+        ).update({TaskInstance.trigger_id: None})
+        # Get all triggers that have no task instances depending on them...
+        ids = [
+            x[0]
+            for x in (

Review comment:
       You're not wrong.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r685315580



##########
File path: docs/apache-airflow/concepts/deferring.rst
##########
@@ -0,0 +1,163 @@
+ .. 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.
+
+Deferrable Operators & Triggers
+===============================
+
+Standard :doc:`Operators <operators>` and :doc:`Sensors <sensors>` take up a full *worker slot* for the entire time they are running, even if they are idle; for example, if you only have 100 worker slots available to run Tasks, and you have 100 DAGs waiting on a Sensor that's currently running but idle, then you *cannot run anything else* - even though your entire Airflow cluster is essentially idle. ``reschedule`` mode for Sensors solves some of this, allowing Sensors to only run at fixed intervals, but it is inflexible and only allows using time as the reason to resume, not anything else.
+
+This is where *Deferrable Operators* come in. A deferrable operator is one that is written with the ability to suspend itself and free up the worker when it knows it has to wait, and hand off the job of resuming it to something called a *Trigger*. As a result, while it is suspended (deferred), it is not taking up a worker slot and your cluster will have a lot less resources wasted on idle Operators or Sensors.
+
+*Triggers* are small, asynchronous pieces of Python code designed to be run all together in a single Python process; because they are asynchronous, they are able to all co-exist efficiently. As an overview of how this process works:
+
+* A task instance (running operator) gets to a point where it has to wait, and defers itself with a trigger tied to the event that should resume it. This frees up the worker to run something else.
+* The new Trigger instance is registered inside Airflow, and picked up by a *triggerer* process
+* The trigger is run until it fires, at which point its source task is re-scheduled
+* The scheduler queues the task to resume on a worker node
+
+Using deferrable operators as a DAG author is almost transparent; writing them, however, takes a bit more work.
+
+.. note::
+
+    Deferrable Operators & Triggers rely on more recent ``asyncio`` features, and as a result only work
+    on Python 3.7 or higher.
+
+
+Using Deferrable Operators
+--------------------------
+
+If all you wish to do is use pre-written Deferrable Operators (such as ``TimeSensorAsync``, which comes with Airflow), then there are only two steps you need:
+
+* Ensure your Airflow installation is running at least one ``triggerer`` process, as well as the normal ``scheduler``
+* Use deferrable operators/sensors in your DAGs
+
+That's it; everything else will be automatically handled for you. If you're upgrading existing DAGs, we even provide some API-compatible sensor variants (e.g. ``TimeSensorAsync`` for ``TimeSensor``) that you can swap into your DAG with no other changes required.
+
+Note that you cannot yet use the deferral ability from inside custom PythonOperator/TaskFlow Python functions; it is only available to traditional, class-based Operators at the moment.
+
+
+Writing Deferrable Operators
+----------------------------
+
+Writing a deferrable operator takes a bit more work. There are some main points to consider:
+
+* Your Operator must defer itself based on a Trigger. If there is a Trigger in core Airflow you can use, great; otherwise, you will have to write one.
+* Your Operator will be stopped and removed from its worker while deferred, and no state will persist automatically. You can persist state by asking Airflow to resume you at a certain method or pass certain kwargs, but that's it.
+* You can defer multiple times, and you can defer before/after your Operator does significant work, or only defer if certain conditions are met (e.g. a system does not have an immediate answer). Deferral is entirely under your control.
+* Any Operator can defer; no special marking on its class is needed, and it's not limited to Sensors.
+
+
+Triggering Deferral
+~~~~~~~~~~~~~~~~~~~
+
+If you want to trigger deferral, at any place in your Operator you can call ``self.defer(trigger, method_name, kwargs, timeout)``, which will raise a special exception that Airflow will catch. The arguments are:
+
+* ``trigger``: An instance of a Trigger that you wish to defer on. It will be serialized into the database.
+* ``method_name``: The method name on your Operator you want Airflow to call when it resumes, other than ``execute``.
+* ``kwargs``: Additional keyword arguments to pass to the method when it is called. Optional, defaults to ``{}``.
+* ``timeout``: A timedelta that specifies a timeout after which this deferral will fail, and fail the task instance. Optional, defaults to ``None``, meaning no timeout.
+
+When you opt to defer, your Operator will *stop executing at that point and be removed from its current worker*. No state - such as local variables, or attributes set on ``self`` - will persist, and when your Operator is resumed it will be a *brand new instance* of it. The only way you can pass state from the old instance of the Operator to the new one is via ``method_name`` and ``kwargs``.
+
+When your Operator is resumed, you will find an ``event`` item added to the kwargs passed to it, which contains the payload from the trigger event that resumed your Operator. Depending on the trigger, this may be useful to your operator (e.g. it's a status code or URL to fetch results), or it may not be important (it's just a datetime). Your ``method_name`` method, however, *must* accept ``event`` as a keyword argument.
+
+If your Operator returns from either its first ``execute()`` method when it's new, or a subsequent method specified by ``method_name``, it will be considered complete and will finish executing.
+
+You are free to set ``method_name`` to ``execute`` if you want your Operator to have one entrypoint, but it, too, will have to accept ``event`` as an optional keyword argument.
+
+Here's a basic example of how a sensor might trigger deferral::
+
+    class WaitOneHourSensor(BaseSensorOperator):
+        def execute(self, context):
+            self.defer(trigger=TimeDeltaTrigger(timedelta(hours=1), method_name="execute_complete")
+
+        def execute_complete(self, context, event=None):
+            # We have no more work to do here. Mark as complete.
+            return
+
+This Sensor is literally just a thin wrapper around the Trigger, so all it does is defer to the trigger, and specify a different method to come back to when the trigger fires - which, as it returns immediately, marks the Sensor as successful.
+
+Under the hood, ``self.defer`` raises the ``TaskDeferred`` exception, so it will work anywhere inside your Operator's code, even buried many nested calls deep inside ``execute()``. You are free to raise ``TaskDeferred`` manually if you wish; it takes the same arguments as ``self.defer``.
+
+Note that ``execution_timeout`` on Operators is considered over the *total runtime*, not individual executions in-between deferrals - this means that if ``execution_timeout`` is set, an Operator may fail while it's deferred or while it's running after a deferral, even if it's only been resumed for a few seconds.
+
+
+Writing Triggers
+~~~~~~~~~~~~~~~~
+
+A Trigger is written as a class that inherits from ``BaseTrigger``, and implements three methods:
+
+* ``__init__``, to receive arguments from Operators instantiating it
+* ``run``, an asynchronous method that runs its logic and yields one or more ``TriggerEvent`` instances as an asynchronous generator
+* ``serialize``, which returns the information needed to re-construct this trigger, as a tuple of the classpath, and keyword arguments to pass to ``__init__``
+
+There's also some design constraints to be aware of:
+
+* The ``run`` method *must be asynchronous* (using Python's asyncio), and correctly ``await`` whenever it does a blocking operation.
+* ``run`` must ``yield`` its TriggerEvents, not return them. If it returns before yielding at least once event, Airflow will consider this an error and fail any Task Instances waiting on it. If it throws an exception, Airflow will also fail any dependent task instances.
+* A Trigger *must be able to run in parallel* with other copies of itself. This can happen both when two tasks defer based on the same trigger, and also if a network partition happens and Airflow re-launches a trigger on a separated machine.
+* When events are emitted, and if your trigger is designed to emit more than one event, they *must* contain a payload that can be used to deduplicate events if the trigger is being run in multiple places. If you only fire one event, and don't want to pass information in the payload back to the Operator that deferred, you can just set the payload to ``None``.
+* A trigger may be suddenly removed from one process and started on a new one (if partitions are being changed, or a deployment is happening). You may provide an optional ``cleanup`` method that gets called when this happens.
+
+Here's the structure of a basic Trigger::
+
+
+    class DateTimeTrigger(BaseTrigger):
+
+        def __init__(self, moment):
+            super().__init__()
+            self.moment = moment
+
+        def serialize(self):
+            return ("airflow.triggers.temporal.DateTimeTrigger", {"moment": self.moment})
+
+        async def run(self):
+            while self.moment > timezone.utcnow():
+                await asyncio.sleep(1)
+            yield TriggerEvent(self.moment)
+
+This is a very simplified version of Airflow's ``DateTimeTrigger``, and you can see several things here:
+
+* ``__init__`` and ``serialize`` are written as a pair; the Trigger is instantiated once when it is submitted by the Operator as part of its deferral request, then serialized and re-instantiated on any *triggerer* process that runs the trigger.
+* The ``run`` method is declared as an ``async def``, as it *must* be asynchronous, and uses ``asyncio.sleep`` rather than the regular ``time.sleep`` (as that would block the process).
+* When it emits its event it packs ``self.moment`` in there, so if this trigger is being run redundantly on multiple hosts, the event can be de-duplicated.
+
+Triggers can be as complex or as simple as you like provided you keep inside this contract; they are designed to be run in a highly-available fashion, auto-distributed among hosts running the *triggerer*. We encourage you to avoid any kind of persistent state in a trigger; they should get everything they need from their ``__init__``, so they can be serialized and moved around freely.
+
+If you are new to writing asynchronous Python, you should be very careful writing your ``run()`` method; Python's async model means that any code that does not correctly ``await`` when it does a blocking operation will block the *entire process*. Airflow will attempt to detect this and warn you in the triggerer logs when it happens, but we strongly suggest you set the variable ``PYTHONASYNCIODEBUG=1`` when you are writing your Trigger to enable extra checks from Python to make sure you're writing non-blocking code. Be especially careful when doing filesystem calls, as if the underlying filesystem is network-backed it may be blocking.
+
+Right now, Triggers are only used up to their first event, as they are only used for resuming deferred tasks (which happens on the first event fired). However, we plan to allow DAGs to be launched from triggers in future, which is where multi-event triggers will be more useful.
+
+
+High Availability
+-----------------
+
+Triggers are designed from the ground-up to be highly-available; if you want to run a highly-available setup, simply run multiple copies of ``triggerer`` on multiple hosts. Much like ``scheduler``, they will automatically co-exist with correct locking and HA.
+
+Depending on how much work the triggers are doing, you can fit from hundreds to tens of thousands of triggers on a single ``triggerer`` host. By default, every ``triggerer`` will have a capacity of 1000 triggers it will try to run at once; you can change this with the ``--capacity`` argument. If you have more triggers trying to run than you have capacity across all of your ``triggerer`` processes, some triggers will be delayed from running until others have completed.
+
+Airflow tries to only run triggers in one place at once, and maintains a heartbeat to all ``triggerers`` that are currently running. If a ``triggerer`` dies, or becomes partitioned from the network where Airflow's database is running, Airflow will automatically re-schedule triggers that were on that host to run elsewhere (after waiting 30 seconds for the machine to re-appear).
+
+This means it's possible, but unlikely, for triggers to run in multiple places at once; this is designed into the Trigger contract, however, and entirely expected. Airflow will de-duplicate events fired when a trigger is running in multiple places simultaneously, so this process should be transparent to your Operators.
+
+Note that every extra ``triggerer`` you run will result in an extra persistent connection to your database.
+
+
+Smart Sensors
+-------------
+
+Deferrable Operators essentially supersede :doc:`Smart Sensors <smart-sensors>`, and should be preferred for almost all situations. They do solve fundamentally the same problem; Smart Sensors, however, only work for certain Sensor workload styles, have no redundancy, and require a custom DAG to run at all times.

Review comment:
       Added one.




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#issuecomment-897095313


   The PR most likely needs to run full matrix of tests because it modifies parts of the core of Airflow. However, committers might decide to merge it quickly and take the risk. If they don't merge it quickly - please rebase it to the latest main at your convenience, or amend the last commit of the PR, and push it with --force-with-lease.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r685524604



##########
File path: airflow/cli/commands/triggerer_command.py
##########
@@ -0,0 +1,56 @@
+# 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.
+
+"""Triggerer command"""
+import signal
+
+import daemon
+from daemon.pidfile import TimeoutPIDLockFile
+
+from airflow import settings
+from airflow.jobs.triggerer_job import TriggererJob
+from airflow.utils import cli as cli_utils
+from airflow.utils.cli import setup_locations, setup_logging, sigint_handler, sigquit_handler
+
+
+@cli_utils.action_logging
+def triggerer(args):
+    """Starts Airflow Triggerer"""
+    print(settings.HEADER)
+    job = TriggererJob(capacity=args.capacity)
+
+    if args.daemon:
+        pid, stdout, stderr, log_file = setup_locations(
+            "triggerer", args.pid, args.stdout, args.stderr, args.log_file
+        )
+        handle = setup_logging(log_file)
+        with open(stdout, 'w+') as stdout:
+            with open(stderr, 'w+') as stderr:
+                ctx = daemon.DaemonContext(
+                    pidfile=TimeoutPIDLockFile(pid, -1),
+                    files_preserve=[handle],
+                    stdout=stdout,
+                    stderr=stderr,
+                )

Review comment:
       ```suggestion
           with open(stdout, 'w+') as stdout_handle, open(stderr, 'w+') as stderr_handle:
               ctx = daemon.DaemonContext(
                   pidfile=TimeoutPIDLockFile(pid, -1),
                   files_preserve=[handle],
                   stdout=stdout_handle,
                   stderr=stderr_handle,
               )
   ```
   
   nit: Probably for consistency, same as https://github.com/apache/airflow/blob/76ddf0fd09aa59ed974715e3e3f461b7b5fd7032/airflow/cli/commands/scheduler_command.py#L53-L59




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r685279806



##########
File path: airflow/jobs/scheduler_job.py
##########
@@ -1207,3 +1218,26 @@ def adopt_or_reset_orphaned_tasks(self, session: Session = None):
                     raise
 
         return len(to_reset)
+
+    @provide_session
+    def check_trigger_timeouts(self, session: Session = None):
+        """
+        Looks at all tasks that are in the "deferred" state and whose trigger
+        or execution timeout has passed, so they can be marked as failed.
+        """
+        num_timed_out_tasks = (
+            session.query(TaskInstance)
+            .filter(TaskInstance.state == State.DEFERRED, TaskInstance.trigger_timeout < timezone.utcnow())
+            .update(
+                # We have to schedule these to fail themselves so it doesn't
+                # happen inside the scheduler.
+                {
+                    "state": State.SCHEDULED,
+                    "next_method": "__fail__",
+                    "next_kwargs": {"error": "Trigger/execution timeout"},
+                    "trigger_id": None,
+                }
+            )
+        )
+        if num_timed_out_tasks:
+            self.log.info("Timed out %i deferred tasks without fired triggers", num_timed_out_tasks)

Review comment:
       Agreed. This isn't the last PR for this, just the first and biggest!




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638347552



##########
File path: airflow/jobs/scheduler_job.py
##########
@@ -1899,3 +1910,17 @@ def adopt_or_reset_orphaned_tasks(self, session: Session = None):
                     raise
 
         return len(to_reset)
+
+    @provide_session
+    def check_trigger_timeouts(self, session: Session = None):
+        """
+        Looks at all tasks that are in the "deferred" state and whose trigger
+        timeout has passed, so they can be marked as failed.
+        """
+        timed_out_tasks = session.query(TaskInstance).filter(
+            TaskInstance.state == State.DEFERRED, TaskInstance.trigger_timeout < timezone.utcnow()
+        )
+        num_tasks = timed_out_tasks.count()

Review comment:
       What do you mean by `func.count`? Do you mean limiting the count query to only be over a single column or something?




-- 
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:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
kaxil commented on pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#issuecomment-897096018


   Looks good to me -- I think we can follow up if there are more changes
   
   


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638981784



##########
File path: airflow/triggers/base.py
##########
@@ -0,0 +1,96 @@
+# 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 typing import Any, AsyncIterator, Dict, Tuple
+
+
+class BaseTrigger:

Review comment:
       I have an unreasonable dislike of abstract base classes, but I will do it for consistency.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638981784



##########
File path: airflow/triggers/base.py
##########
@@ -0,0 +1,96 @@
+# 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 typing import Any, AsyncIterator, Dict, Tuple
+
+
+class BaseTrigger:

Review comment:
       I have an unreasonable dislike of abstract base classes, but I will do it if you want.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#issuecomment-834683973


   [The Workflow run](https://github.com/apache/airflow/actions/runs/821220951) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638980167



##########
File path: airflow/models/trigger.py
##########
@@ -0,0 +1,161 @@
+# 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 datetime
+from typing import Any, Dict, List, Optional
+
+import jump
+from sqlalchemy import BigInteger, Column, String, func
+
+from airflow.models.base import Base
+from airflow.models.taskinstance import TaskInstance
+from airflow.triggers.base import BaseTrigger
+from airflow.utils import timezone
+from airflow.utils.session import provide_session
+from airflow.utils.sqlalchemy import ExtendedJSON, UtcDateTime
+from airflow.utils.state import State
+
+
+class Trigger(Base):
+    """
+    Triggers are a workload that run in an asynchronous event loop shared with
+    other Triggers, and fire off events that will unpause deferred Tasks,
+    start linked DAGs, etc.
+
+    They are persisted into the database and then re-hydrated into a single
+    "triggerer" process, where they're all run at once. We model it so that
+    there is a many-to-one relationship between Task and Trigger, for future
+    deduplication logic to use.
+
+    Rows will be evicted from the database when the triggerer detects no
+    active Tasks/DAGs using them. Events are not stored in the database;
+    when an Event is fired, the triggerer will directly push its data to the
+    appropriate Task/DAG.
+    """
+
+    __tablename__ = "trigger"
+
+    id = Column(BigInteger, primary_key=True)
+    classpath = Column(String(1000), nullable=False)
+    kwargs = Column(ExtendedJSON, nullable=False)
+    created_date = Column(UtcDateTime, nullable=False)
+
+    def __init__(
+        self, classpath: str, kwargs: Dict[str, Any], created_date: Optional[datetime.datetime] = None
+    ):
+        super().__init__()
+        self.classpath = classpath
+        self.kwargs = kwargs
+        self.created_date = created_date or timezone.utcnow()
+
+    @classmethod
+    @provide_session
+    def runnable_ids(
+        cls, session=None, partition_ids: Optional[List[int]] = None, partition_total: Optional[int] = None
+    ):  # pylint: disable=unused-argument
+        """
+        Returns all "runnable" triggers IDs, optionally filtering down by partition.
+
+        This is a pretty basic partition algorithm for now, but it does the job.
+        """
+        # NOTE: It's possible in future that we could try and pre-calculate a
+        # partition entry in a large virtual ring (e.g. 4096 buckets) and store
+        # that in the DB for more direct querying, but for now Jump is fast
+        # enough of a hash to do this all locally - about 0.1s per million hashes
+
+        # Retrieve all IDs first
+        trigger_ids = [row[0] for row in session.query(cls.id).all()]
+
+        # Short-circuit for "no partitioning"
+        if partition_ids is None or partition_total is None:
+            return trigger_ids
+
+        # Go through and map each trigger ID to a partition number,
+        # using a quick, consistent hash (Jump), keeping only the ones that
+        # match one of our partition IDs
+        return [x for x in trigger_ids if jump.hash(x, partition_total) + 1 in partition_ids]
+
+    @classmethod
+    def from_object(cls, trigger: BaseTrigger):
+        """
+        Alternative constructor that creates a trigger row based directly
+        off of a Trigger object.
+        """
+        classpath, kwargs = trigger.serialize()
+        return cls(classpath=classpath, kwargs=kwargs)
+
+    @classmethod
+    @provide_session
+    def bulk_fetch(cls, ids: List[int], session=None) -> Dict[int, "Trigger"]:
+        """
+        Fetches all of the Triggers by ID and returns a dict mapping
+        ID -> Trigger instance
+        """
+        return {obj.id: obj for obj in session.query(cls).filter(cls.id.in_(ids)).all()}
+
+    @classmethod
+    @provide_session
+    def clean_unused(cls, session=None):
+        """
+        Deletes all triggers that have no tasks/DAGs dependent on them
+        (triggers have a one-to-many relationship to both)
+        """
+        # Update all task instances with trigger IDs that are not DEFERRED to remove them
+        session.query(TaskInstance).filter(
+            TaskInstance.state != State.DEFERRED, TaskInstance.trigger_id.isnot(None)
+        ).update({TaskInstance.trigger_id: None})
+        # Get all triggers that have no task instances depending on them...
+        ids = [
+            x[0]
+            for x in (
+                session.query(cls.id)
+                .join(TaskInstance, cls.id == TaskInstance.trigger_id, isouter=True)
+                .group_by(cls.id)
+                .having(func.count(TaskInstance.trigger_id) == 0)
+            )
+        ]
+        # ...and delete them (we can't do this in one query due to MySQL)
+        session.query(Trigger).filter(Trigger.id.in_(ids)).delete(synchronize_session=False)
+
+    @classmethod
+    @provide_session
+    def submit_event(cls, trigger_id, event, session=None):
+        """
+        Takes an event from an instance of itself, and triggers all dependent
+        tasks to resume.
+        """
+        for task_instance in session.query(TaskInstance).filter(
+            TaskInstance.trigger_id == trigger_id, TaskInstance.state == State.DEFERRED
+        ):
+            # Add the event's payload into the kwargs for the task
+            next_kwargs = task_instance.next_kwargs or {}
+            next_kwargs["event"] = event.payload
+            task_instance.next_kwargs = next_kwargs
+            # Remove ourselves as its trigger
+            task_instance.trigger_id = None
+            # Finally, mark it as scheduled so it gets re-queued
+            task_instance.state = State.SCHEDULED

Review comment:
       Hm, I was leaving it to use the existing "this task is starting" logging, but we could add logging saying it was re-scheduled?




-- 
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:
users@infra.apache.org



[GitHub] [airflow] xinbinhuang commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638420317



##########
File path: airflow/api_connexion/endpoints/task_instance_endpoint.py
##########
@@ -248,7 +248,7 @@ def post_clear_task_instances(dag_id: str, session=None):
         error_message = f"Dag id {dag_id} not found"
         raise NotFound(error_message)
     reset_dag_runs = data.pop('reset_dag_runs')
-    task_instances = dag.clear(get_tis=True, **data)
+    task_instances = dag.clear(get_ti_instances=True, **data)

Review comment:
       Ah I see. I think these are some good reasons to move it to `get_ti_instances`, but I would still prefer to keep `get_tis` and add the `get_ti_keys`. 
   
   > because they are actually TaskInstance (model) instances.
   
   - Though it makes sense, I think it's a bit unconventional as normally people just reference the instances as lowercase to the model class name (i.e. `class User(Base)` -> `user = User(...)` ) as similar to an instance(object) of a class. 
   - Another concern from me is that people in the future may feel confused about the naming when they come across this.
   
   >  different to the old name (so I could make sure I had caught all references)
   
   The `dag.clear()` method is a public API. would this cause breaking changes as ? Though I doubt people actually use `dag.clear()` in writing the DAG, the method may still be used in testing. In this case, we still need to keep the old keyword with deprecation warnings.
   
   WDYT? I am also curious about other people's opinions.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] xinbinhuang commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638220428



##########
File path: airflow/api_connexion/endpoints/task_instance_endpoint.py
##########
@@ -248,7 +248,7 @@ def post_clear_task_instances(dag_id: str, session=None):
         error_message = f"Dag id {dag_id} not found"
         raise NotFound(error_message)
     reset_dag_runs = data.pop('reset_dag_runs')
-    task_instances = dag.clear(get_tis=True, **data)

Review comment:
       I believe `tis` stands for `task instances`, so maybe `get_task_instances` to make it more explicit? 
   
   Another problem, `ti` is also a "public" variable as it's used in xcom, task context, macros etc. Though in general, I prefer explicit over abbreviation, I don't think it's a good change to include in this PR.
   
   The same applies to the rest of `ti_instances`

##########
File path: airflow/api_connexion/endpoints/task_instance_endpoint.py
##########
@@ -248,7 +248,7 @@ def post_clear_task_instances(dag_id: str, session=None):
         error_message = f"Dag id {dag_id} not found"
         raise NotFound(error_message)
     reset_dag_runs = data.pop('reset_dag_runs')
-    task_instances = dag.clear(get_tis=True, **data)

Review comment:
       I believe `tis` stands for `task instances`, so maybe `get_task_instances` to make it more explicit? 
   
   Another problem, `ti` is also a "public" variable as it's used in xcom, task context, macros etc. Though in general, I prefer explicit over abbreviation, I don't think it's a good change to include in this PR.
   




-- 
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:
users@infra.apache.org



[GitHub] [airflow] xinbinhuang commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638420317



##########
File path: airflow/api_connexion/endpoints/task_instance_endpoint.py
##########
@@ -248,7 +248,7 @@ def post_clear_task_instances(dag_id: str, session=None):
         error_message = f"Dag id {dag_id} not found"
         raise NotFound(error_message)
     reset_dag_runs = data.pop('reset_dag_runs')
-    task_instances = dag.clear(get_tis=True, **data)
+    task_instances = dag.clear(get_ti_instances=True, **data)

Review comment:
       Ah I see. I think these are some good reasons to move it to `get_ti_instances`, but I still prefer to keep `get_tis` and add the `get_ti_keys`. 
   
   > because they are actually TaskInstance (model) instances.
   
   - Though it makes sense, I think it's a bit unconventional as normally people just reference the instances as lowercase to the model class name (i.e. `class User(Base)` -> `user = User(...)` ) as similar to an instance(object) of a class. 
   - Another concern from me is that people in the future may feel confused about the naming when they come across this.
   
   >  different to the old name (so I could make sure I had caught all references)
   
   The `dag.clear()` method is a public API. Would this cause breaking changes? Though I doubt people actually use `dag.clear()` in writing the DAG, the method may still be used in testing. In this case, we still need to keep the old keyword with deprecation warnings.
   
   WDYT? I am also curious about other people's opinions.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#issuecomment-834683383






-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638982713



##########
File path: airflow/triggers/temporal.py
##########
@@ -0,0 +1,76 @@
+# 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 asyncio
+import datetime
+from typing import Any, Dict, Tuple
+
+import pytz
+
+from airflow.triggers.base import BaseTrigger, TriggerEvent
+from airflow.utils import timezone
+
+
+class DateTimeTrigger(BaseTrigger):
+    """
+    A trigger that fires exactly once, at the given datetime, give or take
+    a few seconds.
+
+    The provided datetime MUST be in UTC.
+    """
+
+    def __init__(self, moment: datetime.datetime):
+        super().__init__()
+        # Make sure it's in UTC
+        if moment.tzinfo is None:
+            self.moment = pytz.utc.localize(moment)
+        elif moment.tzinfo == pytz.utc or getattr(moment.tzinfo, "name", None) == "UTC":
+            self.moment = moment
+        else:
+            raise ValueError(f"The passed datetime must be in UTC, not {moment.tzinfo!r}")
+
+    def serialize(self) -> Tuple[str, Dict[str, Any]]:
+        return ("airflow.triggers.temporal.DateTimeTrigger", {"moment": self.moment})
+
+    async def run(self):
+        """
+        Simple time delay loop until the relevant time is met.
+
+        We do have a two-phase delay to save some cycles, but sleeping is so
+        cheap anyway that it's pretty loose.
+        """
+        # Sleep an hour at a time while it's more than 2 hours away
+        while timezone.utcnow() - self.moment > datetime.timedelta(hours=2):
+            await (asyncio.sleep(3600))
+        # Sleep a second at a time otherwise
+        while self.moment > timezone.utcnow():
+            await asyncio.sleep(1)

Review comment:
       That won't work if the system clock on the machine changes.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#issuecomment-833100890


   [The Workflow run](https://github.com/apache/airflow/actions/runs/815011790) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638980822



##########
File path: airflow/triggers/base.py
##########
@@ -0,0 +1,96 @@
+# 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 typing import Any, AsyncIterator, Dict, Tuple
+
+
+class BaseTrigger:
+    """
+    Base class for all triggers.
+
+    A trigger has two contexts it can exist in:
+
+     - As part of a DAG declaration, where it's declared.

Review comment:
       It's not, this is an older comment, will fix




-- 
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:
users@infra.apache.org



[GitHub] [airflow] xinbinhuang commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638221307



##########
File path: airflow/api_connexion/endpoints/task_instance_endpoint.py
##########
@@ -248,7 +248,7 @@ def post_clear_task_instances(dag_id: str, session=None):
         error_message = f"Dag id {dag_id} not found"
         raise NotFound(error_message)
     reset_dag_runs = data.pop('reset_dag_runs')
-    task_instances = dag.clear(get_tis=True, **data)
+    task_instances = dag.clear(get_ti_instances=True, **data)

Review comment:
       I believe `tis` stands for `task instances`, so maybe `get_task_instances` to make it more explicit? 
   
   Another problem, `ti` is also a "public" variable as it's used in xcom, task context, macros etc. Though in general, I prefer explicit over abbreviation, I don't think it's good change to include within this PR.
   
   The same applies to the rest of `ti_instances`




-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638975780



##########
File path: airflow/jobs/triggerer_job.py
##########
@@ -0,0 +1,418 @@
+# 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 asyncio
+import importlib
+import os
+import signal
+import sys
+import threading
+import time
+from collections import deque
+from typing import Deque, Dict, List, Optional, Set, Tuple, Type
+
+from airflow.jobs.base_job import BaseJob
+from airflow.models.trigger import Trigger
+from airflow.triggers.base import BaseTrigger, TriggerEvent
+from airflow.typing_compat import TypedDict
+from airflow.utils.asyncio import create_task
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class TriggererJob(BaseJob, LoggingMixin):
+    """
+    TriggererJob continuously runs active triggers in asyncio, watching
+    for them to fire off their events and then dispatching that information
+    to their dependent tasks/DAGs.
+
+    It runs as two threads:
+     - The main thread does DB calls/checkins
+     - A subthread runs all the async code
+    """
+
+    __mapper_args__ = {'polymorphic_identity': 'TriggererJob'}
+
+    partition_ids: Optional[List[int]] = None
+    partition_total: Optional[int] = None
+
+    def __init__(self, partition=None, *args, **kwargs):
+        # Make sure we can actually run
+        if not hasattr(asyncio, "create_task"):
+            raise RuntimeError("The triggerer/deferred operators only work on Python 3.7 and above.")
+        # Call superclass
+        super().__init__(*args, **kwargs)
+        # Decode partition information
+        self.partition_ids, self.partition_total = None, None
+        if partition:
+            self.partition_ids, self.partition_total = self.decode_partition(partition)
+        # Set up runner async thread
+        self.runner = TriggerRunner()
+
+    def decode_partition(self, partition: str) -> Tuple[List[int], int]:
+        """
+        Given a string-format partition specification, returns the list of
+        partition IDs it represents and the partition total.
+        """
+        try:
+            # The partition format is "1,2,3/10" where the numbers before
+            # the slash are the partitions we represent, and the number
+            # after is the total number. Most users will just have a single
+            # partition number, e.g. "2/10".
+            ids_str, total_str = partition.split("/", 1)
+            partition_total = int(total_str)
+            partition_ids = []
+            for id_str in ids_str.split(","):
+                id_number = int(id_str)
+                # Bounds checking (they're 1-indexed, which might catch people out)
+                if id_number <= 0 or id_number > self.partition_total:
+                    raise ValueError(f"Partition number {id_number} is impossible")
+                self.partition_ids.append(id_number)
+        except (ValueError, TypeError):
+            raise ValueError(f"Invalid partition specification: {partition}")
+        return partition_ids, partition_total
+
+    def register_signals(self) -> None:
+        """Register signals that stop child processes"""
+        signal.signal(signal.SIGINT, self._exit_gracefully)
+        signal.signal(signal.SIGTERM, self._exit_gracefully)
+
+    def _exit_gracefully(self, signum, frame) -> None:  # pylint: disable=unused-argument
+        """Helper method to clean up processor_agent to avoid leaving orphan processes."""
+        # The first time, try to exit nicely
+        if not self.runner.stop:
+            self.log.info("Exiting gracefully upon receiving signal %s", signum)
+            self.runner.stop = True
+        else:
+            self.log.warning("Forcing exit due to second exit signal %s", signum)
+            sys.exit(os.EX_SOFTWARE)
+
+    def _execute(self) -> None:
+        # Display custom startup ack depending on plurality of partitions
+        if self.partition_ids is None:
+            self.log.info("Starting the triggerer")
+        elif len(self.partition_ids) == 1:
+            self.log.info(
+                "Starting the triggerer (partition %s of %s)", self.partition_ids[0], self.partition_total
+            )
+        else:
+            self.log.info(
+                "Starting the triggerer (partitions %s of %s)", self.partition_ids, self.partition_total
+            )
+
+        try:
+            # Kick off runner thread
+            self.runner.start()
+            # Start our own DB loop in the main thread
+            self._run_trigger_loop()
+        except Exception:  # pylint: disable=broad-except
+            self.log.exception("Exception when executing TriggererJob._run_trigger_loop")
+            raise
+        finally:
+            self.log.info("Waiting for triggers to clean up")
+            # Tell the subthread to stop and then wait for it.
+            # If the user interrupts/terms again, _graceful_exit will allow them
+            # to force-kill here.
+            self.runner.stop = True
+            self.runner.join()
+            self.log.info("Exited trigger loop")
+
+    def _run_trigger_loop(self) -> None:
+        """
+        The main-thread trigger loop.
+
+        This runs synchronously and handles all database reads/writes.
+        """
+        while not self.runner.stop:
+            # Clean out unused triggers
+            Trigger.clean_unused()
+            # Load/delete triggers
+            self.load_triggers()
+            # Handle events
+            self.handle_events()
+            # Handle failed triggers
+            self.handle_failed_triggers()
+            # Idle sleep
+            time.sleep(1)
+
+    def load_triggers(self):
+        """
+        Queries the database to get the triggers we're supposed to be running,
+        adds them to our runner, and then removes ones from it we no longer
+        need.
+        """
+        requested_trigger_ids = Trigger.runnable_ids(
+            partition_ids=self.partition_ids, partition_total=self.partition_total
+        )
+        self.runner.update_triggers(set(requested_trigger_ids))
+
+    def handle_events(self):
+        """
+        Handles outbound events from triggers - dispatching them into the Trigger
+        model where they are then pushed into the relevant task instances.
+        """
+        while self.runner.events:
+            # Get the event and its trigger ID
+            trigger_id, event = self.runner.events.popleft()
+            # Tell the model to wake up its tasks
+            Trigger.submit_event(trigger_id=trigger_id, event=event)
+
+    def handle_failed_triggers(self):
+        """
+        Handles "failed" triggers - ones that errored or exited before they
+        sent an event. Task Instances that depend on them need failing.
+        """
+        while self.runner.failed_triggers:
+            # Tell the model to fail this trigger's deps
+            trigger_id = self.runner.failed_triggers.popleft()
+            Trigger.submit_failure(trigger_id=trigger_id)
+
+
+class TriggerDetails(TypedDict):
+    """Type class for the trigger details dictionary"""
+
+    task: asyncio.Task
+    name: str
+    events: int
+
+
+class TriggerRunner(threading.Thread, LoggingMixin):
+    """
+    Runtime environment for all triggers.
+
+    Mainly runs inside its own thread, where it hands control off to an asyncio
+    event loop, but is also sometimes interacted with from the main thread
+    (where all the DB queries are done). All communication between threads is
+    done via Deques.
+    """
+
+    # Maps trigger IDs to their running tasks and other info
+    triggers: Dict[int, TriggerDetails]
+
+    # Cache for looking up triggers by classpath
+    trigger_cache: Dict[str, Type[BaseTrigger]]
+
+    # Inbound queue of new triggers
+    to_create: Deque[Tuple[int, BaseTrigger]]
+
+    # Inbound queue of deleted triggers
+    to_delete: Deque[int]
+
+    # Outbound queue of events
+    events: Deque[Tuple[int, TriggerEvent]]
+
+    # Outbound queue of failed triggers
+    failed_triggers: Deque[int]
+
+    # Should-we-stop flag
+    stop: bool = False
+
+    def __init__(self):
+        super().__init__()
+        self.triggers = {}
+        self.trigger_cache = {}
+        self.to_create = deque()
+        self.to_delete = deque()
+        self.events = deque()
+        self.failed_triggers = deque()
+
+    def run(self):
+        """Sync entrypoint - just runs arun in an async loop."""
+        # Pylint complains about this with a 3.6 base, can remove with 3.7+
+        asyncio.run(self.arun())  # pylint: disable=no-member
+
+    async def arun(self):
+        """
+        Main (asynchronous) logic loop.
+
+        The loop in here runs trigger addition/deletion/cleanup. Actual
+        triggers run in their own separate coroutines.
+        """
+        watchdog = create_task(self.block_watchdog())
+        last_status = time.time()
+        while not self.stop:
+            # Run core logic
+            await self.create_triggers()
+            await self.delete_triggers()
+            await self.cleanup_finished_triggers()
+            # Sleep for a bit
+            await asyncio.sleep(1)
+            # Every minute, log status
+            if time.time() - last_status >= 60:
+                self.log.info("%i triggers currently running", len(self.triggers))
+                last_status = time.time()
+        # Wait for watchdog to complete
+        await watchdog
+
+    async def create_triggers(self):
+        """
+        Drain the to_create queue and create all triggers that have been
+        requested in the DB that we don't yet have.
+        """
+        while self.to_create:
+            trigger_id, trigger_instance = self.to_create.popleft()
+            if trigger_id not in self.triggers:
+                self.triggers[trigger_id] = {
+                    "task": create_task(self.run_trigger(trigger_id, trigger_instance)),
+                    "name": f"{trigger_instance!r} (ID {trigger_id})",
+                    "events": 0,
+                }
+            else:
+                self.log.warning("Trigger %s had insertion attempted twice", trigger_id)
+
+    async def delete_triggers(self):
+        """
+        Drain the to_delete queue and ensure all triggers that are not in the
+        DB are cancelled, so the cleanup job deletes them.
+        """
+        while self.to_delete:
+            trigger_id = self.to_delete.popleft()
+            if trigger_id in self.triggers:
+                # We only delete if it did not exit already
+                self.triggers[trigger_id]["task"].cancel()
+
+    async def cleanup_finished_triggers(self):
+        """
+        Go through all trigger tasks (coroutines) and clean up entries for
+        ones that have exited, optionally warning users if the exit was
+        not normal.
+        """
+        for trigger_id, details in list(self.triggers.items()):  # pylint: disable=too-many-nested-blocks
+            if details["task"].done():
+                # Check to see if it exited for good reasons
+                try:
+                    result = details["task"].result()
+                except (asyncio.CancelledError, SystemExit, KeyboardInterrupt):
+                    # These are "expected" exceptions and we stop processing here
+                    # If we don't, then the system requesting a trigger be removed -
+                    # which turns into CancelledError - results in a failure.
+                    del self.triggers[trigger_id]
+                    continue
+                except BaseException as e:
+                    # This is potentially bad, so log it.
+                    self.log.error("Trigger %s exited with error %s", details["name"], e)
+                else:
+                    # See if they foolishly returned a TriggerEvent
+                    if isinstance(result, TriggerEvent):
+                        self.log.error(
+                            "Trigger %s returned a TriggerEvent rather than yielding it", details["name"]
+                        )
+                # See if this exited without sending an event, in which case
+                # any task instances depending on it need to be failed
+                if details["events"] == 0:
+                    self.log.error(
+                        "Trigger %s exited without sending an event. Dependent tasks will be failed.",
+                        details["name"],
+                    )
+                    self.failed_triggers.append(trigger_id)
+                del self.triggers[trigger_id]
+
+    async def block_watchdog(self):
+        """
+        Watchdog loop that detects blocking (badly-written) triggers.
+
+        Triggers should be well-behaved async coroutines and await whenever
+        they need to wait; this loop tries to run every 100ms to see if
+        there are badly-written triggers taking longer than that and blocking
+        the event loop.
+
+        Unfortunately, we can't tell what trigger is blocking things, but
+        we can at least detect the top-level problem.
+        """
+        while not self.stop:
+            last_run = time.monotonic()
+            await asyncio.sleep(0.1)
+            # We allow a generous amount of buffer room for now, since it might
+            # be a busy event loop.
+            time_elapsed = time.monotonic() - last_run
+            if time_elapsed > 0.2:
+                self.log.error(
+                    "Triggerer's async thread was blocked for %.2f seconds, "
+                    "likely by a badly-written trigger. Set PYTHONASYNCIODEBUG=1 "
+                    "to get more information on overrunning coroutines.",
+                    time_elapsed,
+                )
+
+    # Async trigger logic
+
+    async def run_trigger(self, trigger_id, trigger):
+        """
+        Wrapper which runs an actual trigger (they are async generators)
+        and pushes their events into our outbound event deque.
+        """
+        self.log.info("Trigger %s starting", self.triggers[trigger_id]['name'])
+        try:
+            async for event in trigger.run():
+                self.log.info("Trigger %s fired: %s", self.triggers[trigger_id]['name'], event)
+                self.triggers[trigger_id]["events"] += 1
+                self.events.append((trigger_id, event))
+        finally:
+            # CancelledError will get injected when we're stopped - which is
+            # fine, the cleanup process will understand that, but we want to
+            # allow triggers a chance to cleanup, either in that case or if
+            # they exit cleanly.
+            trigger.cleanup()
+
+    # Main-thread sync API
+
+    def update_triggers(self, requested_trigger_ids: Set[int]):
+        """
+        Called from the main thread to request that we update what
+        triggers we're running.
+
+        Works out the differences - ones to add, and ones to remove - then
+        adds them to the deques so the subthread can actually mutate the running
+        trigger set.
+        """
+        current_trigger_ids = set(self.triggers.keys())
+        # Work out the two difference sets
+        new_trigger_ids = requested_trigger_ids.difference(current_trigger_ids)
+        old_trigger_ids = current_trigger_ids.difference(requested_trigger_ids)
+        # Bulk-fetch new trigger records
+        new_triggers = Trigger.bulk_fetch(new_trigger_ids)
+        # Add in new triggers
+        for new_id in new_trigger_ids:
+            # Check it didn't vanish in the meantime
+            if new_id not in new_triggers:
+                self.log.warning("Trigger ID %s disappeared before we could start it", new_id)
+                continue
+            # Resolve trigger record into an actual class instance
+            trigger_class = self.get_trigger_by_classpath(new_triggers[new_id].classpath)
+            self.to_create.append((new_id, trigger_class(**new_triggers[new_id].kwargs)))
+        # Remove old triggers
+        for old_id in old_trigger_ids:
+            self.to_delete.append(old_id)
+
+    def get_trigger_by_classpath(self, classpath: str) -> Type[BaseTrigger]:
+        """
+        Gets a trigger class by its classpath ("path.to.module.classname")
+
+        Uses a cache dictionary to speed up lookups after the first time.
+        """
+        if classpath not in self.trigger_cache:
+            module_name, class_name = classpath.rsplit(".", 1)
+            try:
+                module = importlib.import_module(module_name)
+            except ImportError:
+                raise ImportError(
+                    f"Cannot import trigger module {module_name} (from trigger classpath {classpath})"
+                )
+            try:
+                trigger_class = getattr(module, class_name)
+            except AttributeError:
+                raise ImportError(f"Cannot import trigger {class_name} from module {module_name}")
+            self.trigger_cache[classpath] = trigger_class

Review comment:
       Gosh, the `utils` package keeps being a treasure-trove of handy things.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r677399435



##########
File path: airflow/jobs/triggerer_job.py
##########
@@ -0,0 +1,410 @@
+# 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 asyncio
+import os
+import signal
+import sys
+import threading
+import time
+from collections import deque
+from typing import Deque, Dict, List, Optional, Set, Tuple, Type
+
+from airflow.compat.asyncio import create_task
+from airflow.jobs.base_job import BaseJob
+from airflow.models.trigger import Trigger
+from airflow.triggers.base import BaseTrigger, TriggerEvent
+from airflow.typing_compat import TypedDict
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.module_loading import import_string
+
+
+class TriggererJob(BaseJob):
+    """
+    TriggererJob continuously runs active triggers in asyncio, watching
+    for them to fire off their events and then dispatching that information
+    to their dependent tasks/DAGs.
+
+    It runs as two threads:
+     - The main thread does DB calls/checkins
+     - A subthread runs all the async code
+    """
+
+    __mapper_args__ = {'polymorphic_identity': 'TriggererJob'}
+
+    partition_ids: Optional[List[int]] = None
+    partition_total: Optional[int] = None
+
+    def __init__(self, partition=None, *args, **kwargs):
+        # Make sure we can actually run
+        if not hasattr(asyncio, "create_task"):
+            raise RuntimeError("The triggerer/deferred operators only work on Python 3.7 and above.")

Review comment:
       This will be handled already by the `from airflow.compat.asyncio import create_task` on L 27

##########
File path: airflow/models/taskinstance.py
##########
@@ -1182,6 +1228,22 @@ def _run_raw_task(
                 self._prepare_and_execute_task_with_callbacks(context, task)
             self.refresh_from_db(lock_for_update=True)
             self.state = State.SUCCESS
+        except TaskDeferred as defer:
+            # The task has signalled it wants to defer execution based on
+            # a trigger.
+            self._defer_task(defer=defer)
+            self.log.info(
+                'Pausing task as DEFERRED. ' 'dag_id=%s, task_id=%s, execution_date=%s, start_date=%s',

Review comment:
       ```suggestion
                   'Pausing task as DEFERRED. dag_id=%s, task_id=%s, execution_date=%s, start_date=%s',
   ```

##########
File path: airflow/models/trigger.py
##########
@@ -0,0 +1,195 @@
+# 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 datetime
+from typing import Any, Dict, List, Optional
+
+from sqlalchemy import BigInteger, Column, String, func
+
+from airflow.models.base import Base
+from airflow.models.taskinstance import TaskInstance
+from airflow.triggers.base import BaseTrigger
+from airflow.utils import timezone
+from airflow.utils.session import provide_session
+from airflow.utils.sqlalchemy import ExtendedJSON, UtcDateTime
+from airflow.utils.state import State
+
+
+class Trigger(Base):
+    """
+    Triggers are a workload that run in an asynchronous event loop shared with
+    other Triggers, and fire off events that will unpause deferred Tasks,
+    start linked DAGs, etc.
+
+    They are persisted into the database and then re-hydrated into a single
+    "triggerer" process, where they're all run at once. We model it so that
+    there is a many-to-one relationship between Task and Trigger, for future
+    deduplication logic to use.
+
+    Rows will be evicted from the database when the triggerer detects no
+    active Tasks/DAGs using them. Events are not stored in the database;
+    when an Event is fired, the triggerer will directly push its data to the
+    appropriate Task/DAG.
+    """
+
+    __tablename__ = "trigger"
+
+    id = Column(BigInteger, primary_key=True)
+    classpath = Column(String(1000), nullable=False)
+    kwargs = Column(ExtendedJSON, nullable=False)
+    created_date = Column(UtcDateTime, nullable=False)
+    triggerer_id = Column(BigInteger, nullable=True)
+
+    def __init__(
+        self, classpath: str, kwargs: Dict[str, Any], created_date: Optional[datetime.datetime] = None
+    ):
+        super().__init__()
+        self.classpath = classpath
+        self.kwargs = kwargs
+        self.created_date = created_date or timezone.utcnow()
+
+    @classmethod
+    def from_object(cls, trigger: BaseTrigger):
+        """
+        Alternative constructor that creates a trigger row based directly
+        off of a Trigger object.
+        """
+        classpath, kwargs = trigger.serialize()
+        return cls(classpath=classpath, kwargs=kwargs)
+
+    @classmethod
+    @provide_session
+    def bulk_fetch(cls, ids: List[int], session=None) -> Dict[int, "Trigger"]:
+        """
+        Fetches all of the Triggers by ID and returns a dict mapping
+        ID -> Trigger instance
+        """
+        return {obj.id: obj for obj in session.query(cls).filter(cls.id.in_(ids)).all()}
+
+    @classmethod
+    @provide_session
+    def clean_unused(cls, session=None):
+        """
+        Deletes all triggers that have no tasks/DAGs dependent on them
+        (triggers have a one-to-many relationship to both)
+        """
+        # Update all task instances with trigger IDs that are not DEFERRED to remove them
+        session.query(TaskInstance).filter(
+            TaskInstance.state != State.DEFERRED, TaskInstance.trigger_id.isnot(None)
+        ).update({TaskInstance.trigger_id: None})
+        # Get all triggers that have no task instances depending on them...
+        ids = [
+            trigger_id
+            for (trigger_id,) in (
+                session.query(cls.id)
+                .join(TaskInstance, cls.id == TaskInstance.trigger_id, isouter=True)
+                .group_by(cls.id)
+                .having(func.count(TaskInstance.trigger_id) == 0)
+            )
+        ]
+        # ...and delete them (we can't do this in one query due to MySQL)
+        session.query(Trigger).filter(Trigger.id.in_(ids)).delete(synchronize_session=False)
+
+    @classmethod
+    @provide_session
+    def submit_event(cls, trigger_id, event, session=None):
+        """
+        Takes an event from an instance of itself, and triggers all dependent
+        tasks to resume.
+        """
+        for task_instance in session.query(TaskInstance).filter(
+            TaskInstance.trigger_id == trigger_id, TaskInstance.state == State.DEFERRED
+        ):
+            # Add the event's payload into the kwargs for the task
+            next_kwargs = task_instance.next_kwargs or {}
+            next_kwargs["event"] = event.payload
+            task_instance.next_kwargs = next_kwargs
+            # Remove ourselves as its trigger
+            task_instance.trigger_id = None
+            # Finally, mark it as scheduled so it gets re-queued
+            task_instance.state = State.SCHEDULED
+
+    @classmethod
+    @provide_session
+    def submit_failure(cls, trigger_id, session=None):
+        """
+        Called when a trigger has failed unexpectedly, and we need to mark
+        everything that depended on it as failed. Notably, we have to actually
+        run the failure code from a worker as it may have linked callbacks, so
+        hilariously we have to re-schedule the task instances to a worker just
+        so they can then fail.
+
+        We use a special __fail__ value for next_method to achieve this that
+        the runtime code understands as immediate-fail, and pack the error into
+        next_kwargs.
+
+        TODO: Once we have shifted callback (and email) handling to run on
+        workers as first-class concepts, we can run the failure code here
+        in-process, but we can't do that right now.
+        """
+        for task_instance in session.query(TaskInstance).filter(
+            TaskInstance.trigger_id == trigger_id, TaskInstance.state == State.DEFERRED
+        ):
+            # Add the error and set the next_method to the fail state
+            task_instance.next_method = "__fail__"
+            task_instance.next_kwargs = {"error": "Trigger failure"}
+            # Remove ourselves as its trigger
+            task_instance.trigger_id = None
+            # Finally, mark it as scheduled so it gets re-queued
+            task_instance.state = State.SCHEDULED
+
+    @classmethod
+    @provide_session
+    def ids_for_triggerer(cls, triggerer_id, session=None):
+        """Retrieves a list of triggerer_ids."""
+        return [row[0] for row in session.query(cls.id).filter(cls.triggerer_id == triggerer_id)]
+
+    @classmethod
+    @provide_session
+    def assign_unassigned(cls, triggerer_id, capacity, session=None):
+        """
+        Takes a triggerer_id and the capacity for that triggerer and assigns unassigned
+        triggers until that capacity is reached, or there are no more unassigned triggers.
+        """
+        from airflow.jobs.base_job import BaseJob  # To avoid circular import
+
+        count = session.query(cls.id).filter(cls.triggerer_id == triggerer_id).count()
+        capacity -= count
+
+        if capacity <= 0:
+            return
+
+        alive_triggerer_ids = [
+            row[0]
+            for row in session.query(BaseJob.id).filter(
+                BaseJob.end_date is None,
+                BaseJob.latest_heartbeat > timezone.utcnow() - datetime.timedelta(seconds=30),
+                BaseJob.job_type == "TriggererJob",
+            )
+        ]
+
+        # find triggers which have a triggerer_id not in list
+        trigger_ids = [
+            row[0]
+            for row in session.query(cls.id)
+            .filter(cls.triggerer_id.notin_(alive_triggerer_ids))
+            .limit(capacity)
+        ]
+
+        session.query(cls).filter(cls.id.in_(trigger_ids)).update(
+            {cls.triggerer_id: triggerer_id},
+            synchronize_session=False,
+        )

Review comment:
       By splitting this in to two queries this feels a bit racy to me.
   
   Either we should 
   - Combine in to a single SQL query
   - Use `FOR UPDATE` locking, or
   - Extend the conditional on the update so that it only updates rows where the trigger_id is still the "old" value (None, or not in alive_triggerer_ids).

##########
File path: docs/apache-airflow/concepts/deferring.rst
##########
@@ -0,0 +1,174 @@
+ .. 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.
+
+Deferrable Operators & Triggers
+===============================
+
+Standard :doc:`Operators <operators>` and :doc:`Sensors <sensors>` take up a full *worker slot* for the entire time they are running, even if they are idle; for example, if you only have 100 worker slots available to run Tasks, and you have 100 DAGs waiting on a Sensor that's currently running but idle, then you *cannot run anything else* - even though your entire Airflow cluster is essentially idle. ``reschedule`` mode for Sensors solves some of this, allowing Sensors to only run at fixed intervals, but it is inflexible and only allows using time as the reason to resume, not anything else.
+
+This is where *Deferrable Operators* come in. A deferrable operator is one that is written with the ability to suspend itself and free up the worker when it knows it has to wait, and hand off the job of resuming it to something called a *Trigger*. As a result, while it is suspended (deferred), it is not taking up a worker slot and your cluster will have a lot less resources wasted on idle Operators or Sensors.
+
+*Triggers* are small, asynchronous pieces of Python code designed to be run all together in a single Python process; because they are asynchronous, they are able to all co-exist efficiently. As an overview of how this process works:
+
+* A task instance (running operator) gets to a point where it has to wait, and defers itself with a trigger tied to the event that should resume it. This frees up the worker to run something else.
+* The new Trigger instance is registered inside Airflow, and picked up by one or more *triggerer* processes
+* The trigger is run until it fires, at which point its source task is re-scheduled
+* The scheduler queues the task to resume on a worker node
+
+Using deferrable operators as a DAG author is almost transparent; writing them, however, takes a bit more work.
+
+.. note::
+
+    Deferrable Operators & Triggers rely on more recent ``asyncio`` features, and as a result only work
+    on Python 3.7 or higher.
+
+
+Using Deferrable Operators
+--------------------------
+
+If all you wish to do is use pre-written Deferrable Operators (such as ``TimeSensorAsync``, which comes with Airflow), then there are only two steps you need:
+
+* Ensure your Airflow installation is running at least one ``triggerer`` process, as well as the normal ``scheduler``
+* Use deferrable operators/sensors in your DAGs
+
+That's it; everything else will be automatically handled for you. If you're upgrading existing DAGs, we even provide some API-compatible sensor variants (e.g. ``TimeSensorAsync`` for ``TimeSensor``) that you can swap into your DAG with no other changes required.
+
+Note that you cannot yet use the deferral ability from inside custom PythonOperator/TaskFlow Python functions; it is only available to pre-built Operators at the moment.
+
+
+Writing Deferrable Operators
+----------------------------
+
+Writing a deferrable operator takes a bit more work. There are some main points to consider:
+
+* Your Operator must defer itself based on a Trigger. If there is a Trigger in core Airflow you can use, great; otherwise, you will have to write one.
+* Your Operator will be stopped and removed from its worker while deferred, and no state will persist automatically. You can persist state by asking Airflow to resume you at a certain method or pass certain kwargs, but that's it.
+* You can defer multiple times, and you can defer before/after your Operator does significant work, or only defer if certain conditions are met (e.g. a system does not have an immediate answer). Deferral is entirely under your control.
+* Any Operator can defer; no special marking on its class is needed, and it's not limited to Sensors.
+
+
+Triggering Deferral
+~~~~~~~~~~~~~~~~~~~
+
+If you want to trigger deferral, at any place in your Operator you can call ``self.defer(trigger, method_name, kwargs, timeout)``, which will raise a special exception that Airflow will catch. The arguments are:
+
+* ``trigger``: An instance of a Trigger that you wish to defer on. It will be serialized into the database.
+* ``method_name``: The method name on your Operator you want Airflow to call when it resumes, other than ``execute``.
+* ``kwargs``: Additional keyword arguments to pass to the method when it is called. Optional, defaults to ``{}``.
+* ``timeout``: A timedelta that specifies a timeout after which this deferral will fail, and fail the task instance. Optional, defaults to ``None``, meaning no timeout.
+
+When you opt to defer, your Operator will *stop executing at that point and be removed from its current worker*. No state will persist, and when your Operator is resumed it will be a *brand new instance* of it. The only way you can pass state from the old instance of the Operator to the new one is via ``method_name`` and ``kwargs``.

Review comment:
       > No state will persist
   
   could be a little more explicit. How about
   
   
   > No state such as local variables or attributes set on ``self`` will persist

##########
File path: airflow/models/trigger.py
##########
@@ -0,0 +1,195 @@
+# 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 datetime
+from typing import Any, Dict, List, Optional
+
+from sqlalchemy import BigInteger, Column, String, func
+
+from airflow.models.base import Base
+from airflow.models.taskinstance import TaskInstance
+from airflow.triggers.base import BaseTrigger
+from airflow.utils import timezone
+from airflow.utils.session import provide_session
+from airflow.utils.sqlalchemy import ExtendedJSON, UtcDateTime
+from airflow.utils.state import State
+
+
+class Trigger(Base):
+    """
+    Triggers are a workload that run in an asynchronous event loop shared with
+    other Triggers, and fire off events that will unpause deferred Tasks,
+    start linked DAGs, etc.
+
+    They are persisted into the database and then re-hydrated into a single
+    "triggerer" process, where they're all run at once. We model it so that

Review comment:
       This isn't quite true anymore is it?




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638995366



##########
File path: docs/apache-airflow/concepts/deferring.rst
##########
@@ -0,0 +1,172 @@
+ .. 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.
+
+Deferrable Operators & Triggers
+===============================
+
+Standard :doc:`Operators <operators>` and :doc:`Sensors <sensors>` take up a full *worker slot* for the entire time they are running, even if they are idle; for example, if you only have 100 worker slots available to run Tasks, and you have 100 DAGs waiting on a Sensor that's currently running but idle, then you *cannot run anything else* - even though your entire Airflow cluster is essentially idle.
+
+This is where *Deferrable Operators* come in. A deferrable operator is one that is written with the ability to suspend itself and remove itself from the worker when it knows that it will have to wait, and hand off the job of resuming it to something called a *Trigger*. As a result, while it is suspended (deferred), it is not taking up a worker slot and your cluster will have a lot less resources wasted on idle Operators or Sensors.
+
+*Triggers* are small, asynchronous pieces of Python code designed to be run all together in a single Python process; because they are asynchronous, they are able to all co-exist efficiently. As an overview of how this process works:
+
+* A task instance (running operator) gets to a point where it has to wait, and defers itself with a trigger tied to the event that should resume it. It then removes itself from its current worker and frees up space.
+* The new Trigger instance is registered inside Airflow, and picked up by one or more *triggerer* processes
+* The trigger is run until it fires, at which point its source task is re-scheduled
+* The task instance resumes
+
+Using deferrable operators as a DAG author is almost transparent; writing them, however, takes a bit more work.
+
+.. note::
+
+    Deferrable Operators & Triggers rely on more recent ``asyncio`` features, and as a result only work
+    on Python 3.7 or higher.
+
+
+Using Deferrable Operators
+--------------------------
+
+If all you wish to do is use pre-written Deferrable Operators (such as ``TimeSensorAsync``, which comes with Airflow), then there are only two steps you need:
+
+* Ensure your Airflow installation is running at least one *triggerer* process, as well as the normal *scheduler*
+* Use deferrable operators/sensors in your DAGs
+
+That's it; everything else will be automatically handled for you. If you're upgrading existing DAGs, we even provide some API-compatible sensor variants (e.g. ``TimeSensorAsync`` for ``TimeSensor``) that you can swap into your DAG with no other changes required.
+
+Note that you cannot yet use the deferral ability from inside custom PythonOperator/TaskFlow code; it is only available to pre-built Operators at the moment.
+
+
+Writing Deferrable Operators
+----------------------------
+
+Writing a deferrable operator takes a bit more work. There are some main points to consider:
+
+* Your Operator must defer itself based on a Trigger. If there is a Trigger in core Airflow you can use, great; otherwise, you will have to write one.
+* Your Operator will be deleted and removed from its worker while deferred, and no state will persist automatically. You can persist state by asking Airflow to resume you at a certain method or pass certain kwargs, but that's it.
+* You can defer multiple times, and you can defer before/after your Operator does significant work, or only defer if certain conditions are met (e.g. a system does not have an immediate answer). Deferral is entirely under your control.
+* Any Operator can defer; no special marking on its class is needed, and it's not limited to Sensors.
+
+
+Triggering Deferral
+~~~~~~~~~~~~~~~~~~~
+
+If you want to trigger deferral, at any place in your Operator you can call ``self.defer(trigger, method_name, kwargs, timeout)``, which will raise a special exception that Airflow will catch. The arguments are:
+
+* ``trigger``: An instance of a Trigger that you wish to defer on. It will be serialized into the database.
+* ``method_name``: The method name on your Operator you want Airflow to call when it resumes, other than ``execute``.
+* ``kwargs``: Additional keyword arguments to pass to the method when it is called. Optional, defaults to ``{}``.
+* ``timeout``: A timedelta that specifies a timeout after which this deferral will fail, and fail the task instance. Optional, defaults to ``None``, meaning no timeout.

Review comment:
       I think that is how I would interpret it too, and I suspect a bit of work is needed on that front.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#issuecomment-834871478


   [The Workflow run](https://github.com/apache/airflow/actions/runs/821921441) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r681933392



##########
File path: airflow/jobs/local_task_job.py
##########
@@ -241,7 +241,6 @@ def _run_mini_scheduler_on_child_tasks(self, session=None) -> None:
                 include_upstream=False,
                 include_direct_upstream=True,
             )
-

Review comment:
       Done. Not sure how that snuck in there.




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638272734



##########
File path: airflow/api_connexion/endpoints/task_instance_endpoint.py
##########
@@ -248,7 +248,7 @@ def post_clear_task_instances(dag_id: str, session=None):
         error_message = f"Dag id {dag_id} not found"
         raise NotFound(error_message)
     reset_dag_runs = data.pop('reset_dag_runs')
-    task_instances = dag.clear(get_tis=True, **data)
+    task_instances = dag.clear(get_ti_instances=True, **data)

Review comment:
       Ah, but what I had to do here was separate the two concepts of:
   
   - TaskInstance model instances
   - TaskInstance PK tuples
   
   Since I had to re-write `dag.clear()` to remove a UNION, and wanted to make it slightly more efficient in the process, the internals of that function now work on PK tuples to deduplicate (rather than de-duplicating on all fields as previously), so it needed a new pair of keyword arguments to say what output you wanted. I felt `ti_instances` was a reasonable solution since it's both different to the old name (so I could make sure I had caught all references), and because they are actually TaskInstance (model) instances.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] xinbinhuang commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
xinbinhuang commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638420317



##########
File path: airflow/api_connexion/endpoints/task_instance_endpoint.py
##########
@@ -248,7 +248,7 @@ def post_clear_task_instances(dag_id: str, session=None):
         error_message = f"Dag id {dag_id} not found"
         raise NotFound(error_message)
     reset_dag_runs = data.pop('reset_dag_runs')
-    task_instances = dag.clear(get_tis=True, **data)
+    task_instances = dag.clear(get_ti_instances=True, **data)

Review comment:
       Ah I see. I think these are some good reasons to move it to `get_ti_instances`, but I would still prefer to keep `get_tis` and add the `get_ti_keys`. 
   
   > because they are actually TaskInstance (model) instances.
   
   - Though it makes sense, I think it's a bit unconventional as normally people just reference the instances as lowercase to the model class name (i.e. `class User(Base)` -> `user = User(...)` ) as similar to an instance(object) of a class. 
   - Another concern from me is that people in the future may feel confused about the naming when they come across this.
   
   >  different to the old name (so I could make sure I had caught all references)
   
   The `dag.clear()` method is a public API. Would this cause breaking changes? Though I doubt people actually use `dag.clear()` in writing the DAG, the method may still be used in testing. In this case, we still need to keep the old keyword with deprecation warnings.
   
   WDYT? I am also curious about other people's opinions.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] jedcunningham commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r687106610



##########
File path: airflow/jobs/triggerer_job.py
##########
@@ -0,0 +1,380 @@
+# 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 asyncio
+import os
+import signal
+import sys
+import threading
+import time
+from collections import deque
+from typing import Deque, Dict, Set, Tuple, Type
+
+from airflow.compat.asyncio import create_task
+from airflow.configuration import conf
+from airflow.jobs.base_job import BaseJob
+from airflow.models.trigger import Trigger
+from airflow.triggers.base import BaseTrigger, TriggerEvent
+from airflow.typing_compat import TypedDict
+from airflow.utils.log.logging_mixin import LoggingMixin
+from airflow.utils.module_loading import import_string
+
+
+class TriggererJob(BaseJob):
+    """
+    TriggererJob continuously runs active triggers in asyncio, watching
+    for them to fire off their events and then dispatching that information
+    to their dependent tasks/DAGs.
+
+    It runs as two threads:
+     - The main thread does DB calls/checkins
+     - A subthread runs all the async code
+    """
+
+    __mapper_args__ = {'polymorphic_identity': 'TriggererJob'}
+
+    def __init__(self, capacity=None, *args, **kwargs):
+        # Call superclass
+        super().__init__(*args, **kwargs)
+
+        if capacity is None:
+            self.capacity = conf.getint('triggerer', 'default_capacity', fallback=1000)

Review comment:
       ```suggestion
               self.capacity = conf.getint('triggerer', 'default_capacity')
   ```
   
   (Eventually I'll start my crusade against `fallback`, but this will mean 1 less when that time comes)




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #15389: AIP-40 prototype: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#issuecomment-823536699


   [The Workflow run](https://github.com/apache/airflow/actions/runs/768184459) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r685545772



##########
File path: airflow/cli/commands/triggerer_command.py
##########
@@ -0,0 +1,56 @@
+# 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.
+
+"""Triggerer command"""
+import signal
+
+import daemon
+from daemon.pidfile import TimeoutPIDLockFile
+
+from airflow import settings
+from airflow.jobs.triggerer_job import TriggererJob
+from airflow.utils import cli as cli_utils
+from airflow.utils.cli import setup_locations, setup_logging, sigint_handler, sigquit_handler
+
+
+@cli_utils.action_logging
+def triggerer(args):
+    """Starts Airflow Triggerer"""
+    print(settings.HEADER)
+    job = TriggererJob(capacity=args.capacity)
+
+    if args.daemon:
+        pid, stdout, stderr, log_file = setup_locations(
+            "triggerer", args.pid, args.stdout, args.stderr, args.log_file
+        )
+        handle = setup_logging(log_file)
+        with open(stdout, 'w+') as stdout:
+            with open(stderr, 'w+') as stderr:
+                ctx = daemon.DaemonContext(
+                    pidfile=TimeoutPIDLockFile(pid, -1),
+                    files_preserve=[handle],
+                    stdout=stdout,
+                    stderr=stderr,
+                )

Review comment:
       Nice catch, added




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#issuecomment-833039481


   [The Workflow run](https://github.com/apache/airflow/actions/runs/814862906) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638961397



##########
File path: airflow/models/dag.py
##########
@@ -1177,8 +1179,10 @@ def clear(
         :type dry_run: bool
         :param session: The sqlalchemy session to use
         :type session: sqlalchemy.orm.session.Session
-        :param get_tis: Return the sqlalchemy query for finding the TaskInstance without clearing the tasks
-        :type get_tis: bool
+        :param get_ti_keys: Return the sqlalchemy query for TaskInstance PKs without clearing the tasks
+        :type get_ti_keys: bool
+        :param get_ti_instances: Return the sqlalchemy query for TaskInstances without clearing the tasks
+        :type get_ti_instances: bool

Review comment:
       Unfortunately I can't do that without duplicating all the logic at the end of `clear()` - it's a bit of a mess of a function, and some parts of its logic depend on having instances at the end.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#issuecomment-834682577






-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638977833



##########
File path: airflow/models/taskinstance.py
##########
@@ -282,6 +296,18 @@ class TaskInstance(Base, LoggingMixin):  # pylint: disable=R0902,R0904
     executor_config = Column(PickleType(pickler=dill))
 
     external_executor_id = Column(String(ID_LEN, **COLLATION_ARGS))
+
+    # The trigger to resume on if we are in state DEFERRED
+    trigger_id = Column(BigInteger)
+
+    # Optional timeout datetime for the trigger (past this, we'll fail)
+    trigger_timeout = Column(UtcDateTime)
+
+    # The method to call next, and any extra arguments to pass to it.
+    # Usually used when resuming from DEFERRED.
+    next_method = Column(String(1000))

Review comment:
       Why are there some `Strings()` without `COLLATION_ARGS` then? Looking at it, I presumed they were only for the PKs.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #15389: AIP-40 prototype: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#issuecomment-832266592


   [The Workflow run](https://github.com/apache/airflow/actions/runs/811462101) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638346793



##########
File path: airflow/exceptions.py
##########
@@ -225,3 +226,30 @@ def __str__(self):
 
 class ConnectionNotUnique(AirflowException):
     """Raise when multiple values are found for the same conn_id"""
+
+
+class TaskDeferred(BaseException):
+    """
+    Special exception raised to signal that the operator it was raised from
+    wishes to defer until a trigger fires.
+    """
+
+    def __init__(
+        self,
+        *,
+        trigger,
+        method_name: str,
+        kwargs: Optional[Dict[str, Any]] = None,
+        timeout: Optional[datetime.timedelta] = None,
+    ):
+        super().__init__()
+        self.trigger = trigger
+        self.method_name = method_name
+        self.kwargs = kwargs
+        self.timeout = timeout
+        # Check timeout type at runtime
+        if self.timeout is not None and not hasattr(self.timeout, "total_seconds"):
+            raise ValueError("Timeout value must be a timedelta")

Review comment:
       I still prefer duck-typing to instance checking because I've been doing Python for Too Long. Happy to change it if you'd prefer!




-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638965384



##########
File path: airflow/jobs/triggerer_job.py
##########
@@ -0,0 +1,418 @@
+# 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 asyncio
+import importlib
+import os
+import signal
+import sys
+import threading
+import time
+from collections import deque
+from typing import Deque, Dict, List, Optional, Set, Tuple, Type
+
+from airflow.jobs.base_job import BaseJob
+from airflow.models.trigger import Trigger
+from airflow.triggers.base import BaseTrigger, TriggerEvent
+from airflow.typing_compat import TypedDict
+from airflow.utils.asyncio import create_task
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class TriggererJob(BaseJob, LoggingMixin):
+    """
+    TriggererJob continuously runs active triggers in asyncio, watching
+    for them to fire off their events and then dispatching that information
+    to their dependent tasks/DAGs.
+
+    It runs as two threads:
+     - The main thread does DB calls/checkins
+     - A subthread runs all the async code
+    """
+
+    __mapper_args__ = {'polymorphic_identity': 'TriggererJob'}
+
+    partition_ids: Optional[List[int]] = None
+    partition_total: Optional[int] = None
+
+    def __init__(self, partition=None, *args, **kwargs):
+        # Make sure we can actually run
+        if not hasattr(asyncio, "create_task"):
+            raise RuntimeError("The triggerer/deferred operators only work on Python 3.7 and above.")
+        # Call superclass
+        super().__init__(*args, **kwargs)
+        # Decode partition information
+        self.partition_ids, self.partition_total = None, None
+        if partition:
+            self.partition_ids, self.partition_total = self.decode_partition(partition)
+        # Set up runner async thread
+        self.runner = TriggerRunner()
+
+    def decode_partition(self, partition: str) -> Tuple[List[int], int]:
+        """
+        Given a string-format partition specification, returns the list of
+        partition IDs it represents and the partition total.
+        """
+        try:
+            # The partition format is "1,2,3/10" where the numbers before
+            # the slash are the partitions we represent, and the number
+            # after is the total number. Most users will just have a single
+            # partition number, e.g. "2/10".
+            ids_str, total_str = partition.split("/", 1)
+            partition_total = int(total_str)
+            partition_ids = []
+            for id_str in ids_str.split(","):
+                id_number = int(id_str)
+                # Bounds checking (they're 1-indexed, which might catch people out)
+                if id_number <= 0 or id_number > self.partition_total:
+                    raise ValueError(f"Partition number {id_number} is impossible")
+                self.partition_ids.append(id_number)
+        except (ValueError, TypeError):
+            raise ValueError(f"Invalid partition specification: {partition}")
+        return partition_ids, partition_total
+
+    def register_signals(self) -> None:
+        """Register signals that stop child processes"""
+        signal.signal(signal.SIGINT, self._exit_gracefully)
+        signal.signal(signal.SIGTERM, self._exit_gracefully)
+
+    def _exit_gracefully(self, signum, frame) -> None:  # pylint: disable=unused-argument
+        """Helper method to clean up processor_agent to avoid leaving orphan processes."""
+        # The first time, try to exit nicely
+        if not self.runner.stop:
+            self.log.info("Exiting gracefully upon receiving signal %s", signum)
+            self.runner.stop = True
+        else:
+            self.log.warning("Forcing exit due to second exit signal %s", signum)
+            sys.exit(os.EX_SOFTWARE)
+
+    def _execute(self) -> None:
+        # Display custom startup ack depending on plurality of partitions
+        if self.partition_ids is None:
+            self.log.info("Starting the triggerer")
+        elif len(self.partition_ids) == 1:
+            self.log.info(
+                "Starting the triggerer (partition %s of %s)", self.partition_ids[0], self.partition_total
+            )
+        else:
+            self.log.info(
+                "Starting the triggerer (partitions %s of %s)", self.partition_ids, self.partition_total
+            )
+
+        try:
+            # Kick off runner thread
+            self.runner.start()
+            # Start our own DB loop in the main thread
+            self._run_trigger_loop()
+        except Exception:  # pylint: disable=broad-except
+            self.log.exception("Exception when executing TriggererJob._run_trigger_loop")
+            raise
+        finally:
+            self.log.info("Waiting for triggers to clean up")
+            # Tell the subthread to stop and then wait for it.
+            # If the user interrupts/terms again, _graceful_exit will allow them
+            # to force-kill here.
+            self.runner.stop = True
+            self.runner.join()

Review comment:
       I'll put a long one on as a fallback in case SIGINT is sent non-interactively.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638660154



##########
File path: airflow/exceptions.py
##########
@@ -225,3 +226,30 @@ def __str__(self):
 
 class ConnectionNotUnique(AirflowException):
     """Raise when multiple values are found for the same conn_id"""
+
+
+class TaskDeferred(BaseException):
+    """
+    Special exception raised to signal that the operator it was raised from
+    wishes to defer until a trigger fires.
+    """
+
+    def __init__(
+        self,
+        *,
+        trigger,
+        method_name: str,
+        kwargs: Optional[Dict[str, Any]] = None,
+        timeout: Optional[datetime.timedelta] = None,
+    ):
+        super().__init__()
+        self.trigger = trigger
+        self.method_name = method_name
+        self.kwargs = kwargs
+        self.timeout = timeout
+        # Check timeout type at runtime
+        if self.timeout is not None and not hasattr(self.timeout, "total_seconds"):
+            raise ValueError("Timeout value must be a timedelta")
+
+    def __repr__(self) -> str:
+        return f"<OperatorDeferred trigger={self.trigger} method={self.method_name}>"

Review comment:
       This doesn't match the class name.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638970311



##########
File path: airflow/jobs/triggerer_job.py
##########
@@ -0,0 +1,418 @@
+# 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 asyncio
+import importlib
+import os
+import signal
+import sys
+import threading
+import time
+from collections import deque
+from typing import Deque, Dict, List, Optional, Set, Tuple, Type
+
+from airflow.jobs.base_job import BaseJob
+from airflow.models.trigger import Trigger
+from airflow.triggers.base import BaseTrigger, TriggerEvent
+from airflow.typing_compat import TypedDict
+from airflow.utils.asyncio import create_task
+from airflow.utils.log.logging_mixin import LoggingMixin
+
+
+class TriggererJob(BaseJob, LoggingMixin):
+    """
+    TriggererJob continuously runs active triggers in asyncio, watching
+    for them to fire off their events and then dispatching that information
+    to their dependent tasks/DAGs.
+
+    It runs as two threads:
+     - The main thread does DB calls/checkins
+     - A subthread runs all the async code
+    """
+
+    __mapper_args__ = {'polymorphic_identity': 'TriggererJob'}
+
+    partition_ids: Optional[List[int]] = None
+    partition_total: Optional[int] = None
+
+    def __init__(self, partition=None, *args, **kwargs):
+        # Make sure we can actually run
+        if not hasattr(asyncio, "create_task"):
+            raise RuntimeError("The triggerer/deferred operators only work on Python 3.7 and above.")
+        # Call superclass
+        super().__init__(*args, **kwargs)
+        # Decode partition information
+        self.partition_ids, self.partition_total = None, None
+        if partition:
+            self.partition_ids, self.partition_total = self.decode_partition(partition)
+        # Set up runner async thread
+        self.runner = TriggerRunner()
+
+    def decode_partition(self, partition: str) -> Tuple[List[int], int]:
+        """
+        Given a string-format partition specification, returns the list of
+        partition IDs it represents and the partition total.
+        """
+        try:
+            # The partition format is "1,2,3/10" where the numbers before
+            # the slash are the partitions we represent, and the number
+            # after is the total number. Most users will just have a single
+            # partition number, e.g. "2/10".
+            ids_str, total_str = partition.split("/", 1)
+            partition_total = int(total_str)
+            partition_ids = []
+            for id_str in ids_str.split(","):
+                id_number = int(id_str)
+                # Bounds checking (they're 1-indexed, which might catch people out)
+                if id_number <= 0 or id_number > self.partition_total:
+                    raise ValueError(f"Partition number {id_number} is impossible")
+                self.partition_ids.append(id_number)
+        except (ValueError, TypeError):
+            raise ValueError(f"Invalid partition specification: {partition}")
+        return partition_ids, partition_total
+
+    def register_signals(self) -> None:
+        """Register signals that stop child processes"""
+        signal.signal(signal.SIGINT, self._exit_gracefully)
+        signal.signal(signal.SIGTERM, self._exit_gracefully)
+
+    def _exit_gracefully(self, signum, frame) -> None:  # pylint: disable=unused-argument
+        """Helper method to clean up processor_agent to avoid leaving orphan processes."""
+        # The first time, try to exit nicely
+        if not self.runner.stop:
+            self.log.info("Exiting gracefully upon receiving signal %s", signum)
+            self.runner.stop = True
+        else:
+            self.log.warning("Forcing exit due to second exit signal %s", signum)
+            sys.exit(os.EX_SOFTWARE)
+
+    def _execute(self) -> None:
+        # Display custom startup ack depending on plurality of partitions
+        if self.partition_ids is None:
+            self.log.info("Starting the triggerer")
+        elif len(self.partition_ids) == 1:
+            self.log.info(
+                "Starting the triggerer (partition %s of %s)", self.partition_ids[0], self.partition_total
+            )
+        else:
+            self.log.info(
+                "Starting the triggerer (partitions %s of %s)", self.partition_ids, self.partition_total
+            )
+
+        try:
+            # Kick off runner thread
+            self.runner.start()
+            # Start our own DB loop in the main thread
+            self._run_trigger_loop()
+        except Exception:  # pylint: disable=broad-except
+            self.log.exception("Exception when executing TriggererJob._run_trigger_loop")
+            raise
+        finally:
+            self.log.info("Waiting for triggers to clean up")
+            # Tell the subthread to stop and then wait for it.
+            # If the user interrupts/terms again, _graceful_exit will allow them
+            # to force-kill here.
+            self.runner.stop = True
+            self.runner.join()
+            self.log.info("Exited trigger loop")
+
+    def _run_trigger_loop(self) -> None:
+        """
+        The main-thread trigger loop.
+
+        This runs synchronously and handles all database reads/writes.
+        """
+        while not self.runner.stop:
+            # Clean out unused triggers
+            Trigger.clean_unused()
+            # Load/delete triggers
+            self.load_triggers()
+            # Handle events
+            self.handle_events()
+            # Handle failed triggers
+            self.handle_failed_triggers()
+            # Idle sleep
+            time.sleep(1)
+
+    def load_triggers(self):
+        """
+        Queries the database to get the triggers we're supposed to be running,
+        adds them to our runner, and then removes ones from it we no longer
+        need.
+        """
+        requested_trigger_ids = Trigger.runnable_ids(
+            partition_ids=self.partition_ids, partition_total=self.partition_total
+        )
+        self.runner.update_triggers(set(requested_trigger_ids))
+
+    def handle_events(self):
+        """
+        Handles outbound events from triggers - dispatching them into the Trigger
+        model where they are then pushed into the relevant task instances.
+        """
+        while self.runner.events:
+            # Get the event and its trigger ID
+            trigger_id, event = self.runner.events.popleft()
+            # Tell the model to wake up its tasks
+            Trigger.submit_event(trigger_id=trigger_id, event=event)
+
+    def handle_failed_triggers(self):
+        """
+        Handles "failed" triggers - ones that errored or exited before they
+        sent an event. Task Instances that depend on them need failing.
+        """
+        while self.runner.failed_triggers:
+            # Tell the model to fail this trigger's deps
+            trigger_id = self.runner.failed_triggers.popleft()
+            Trigger.submit_failure(trigger_id=trigger_id)
+
+
+class TriggerDetails(TypedDict):
+    """Type class for the trigger details dictionary"""
+
+    task: asyncio.Task
+    name: str
+    events: int
+
+
+class TriggerRunner(threading.Thread, LoggingMixin):
+    """
+    Runtime environment for all triggers.
+
+    Mainly runs inside its own thread, where it hands control off to an asyncio
+    event loop, but is also sometimes interacted with from the main thread
+    (where all the DB queries are done). All communication between threads is
+    done via Deques.
+    """
+
+    # Maps trigger IDs to their running tasks and other info
+    triggers: Dict[int, TriggerDetails]
+
+    # Cache for looking up triggers by classpath
+    trigger_cache: Dict[str, Type[BaseTrigger]]
+
+    # Inbound queue of new triggers
+    to_create: Deque[Tuple[int, BaseTrigger]]
+
+    # Inbound queue of deleted triggers
+    to_delete: Deque[int]
+
+    # Outbound queue of events
+    events: Deque[Tuple[int, TriggerEvent]]
+
+    # Outbound queue of failed triggers
+    failed_triggers: Deque[int]
+
+    # Should-we-stop flag
+    stop: bool = False
+
+    def __init__(self):
+        super().__init__()
+        self.triggers = {}
+        self.trigger_cache = {}
+        self.to_create = deque()
+        self.to_delete = deque()
+        self.events = deque()
+        self.failed_triggers = deque()
+
+    def run(self):
+        """Sync entrypoint - just runs arun in an async loop."""
+        # Pylint complains about this with a 3.6 base, can remove with 3.7+
+        asyncio.run(self.arun())  # pylint: disable=no-member
+
+    async def arun(self):
+        """
+        Main (asynchronous) logic loop.
+
+        The loop in here runs trigger addition/deletion/cleanup. Actual
+        triggers run in their own separate coroutines.
+        """
+        watchdog = create_task(self.block_watchdog())
+        last_status = time.time()
+        while not self.stop:
+            # Run core logic
+            await self.create_triggers()
+            await self.delete_triggers()
+            await self.cleanup_finished_triggers()
+            # Sleep for a bit
+            await asyncio.sleep(1)
+            # Every minute, log status
+            if time.time() - last_status >= 60:
+                self.log.info("%i triggers currently running", len(self.triggers))
+                last_status = time.time()
+        # Wait for watchdog to complete
+        await watchdog
+
+    async def create_triggers(self):
+        """
+        Drain the to_create queue and create all triggers that have been
+        requested in the DB that we don't yet have.
+        """
+        while self.to_create:
+            trigger_id, trigger_instance = self.to_create.popleft()
+            if trigger_id not in self.triggers:
+                self.triggers[trigger_id] = {
+                    "task": create_task(self.run_trigger(trigger_id, trigger_instance)),
+                    "name": f"{trigger_instance!r} (ID {trigger_id})",
+                    "events": 0,
+                }
+            else:
+                self.log.warning("Trigger %s had insertion attempted twice", trigger_id)
+
+    async def delete_triggers(self):
+        """
+        Drain the to_delete queue and ensure all triggers that are not in the
+        DB are cancelled, so the cleanup job deletes them.
+        """
+        while self.to_delete:
+            trigger_id = self.to_delete.popleft()
+            if trigger_id in self.triggers:
+                # We only delete if it did not exit already
+                self.triggers[trigger_id]["task"].cancel()
+
+    async def cleanup_finished_triggers(self):
+        """
+        Go through all trigger tasks (coroutines) and clean up entries for
+        ones that have exited, optionally warning users if the exit was
+        not normal.
+        """
+        for trigger_id, details in list(self.triggers.items()):  # pylint: disable=too-many-nested-blocks
+            if details["task"].done():
+                # Check to see if it exited for good reasons
+                try:
+                    result = details["task"].result()
+                except (asyncio.CancelledError, SystemExit, KeyboardInterrupt):
+                    # These are "expected" exceptions and we stop processing here
+                    # If we don't, then the system requesting a trigger be removed -
+                    # which turns into CancelledError - results in a failure.
+                    del self.triggers[trigger_id]
+                    continue
+                except BaseException as e:
+                    # This is potentially bad, so log it.
+                    self.log.error("Trigger %s exited with error %s", details["name"], e)
+                else:
+                    # See if they foolishly returned a TriggerEvent
+                    if isinstance(result, TriggerEvent):
+                        self.log.error(
+                            "Trigger %s returned a TriggerEvent rather than yielding it", details["name"]
+                        )
+                # See if this exited without sending an event, in which case
+                # any task instances depending on it need to be failed
+                if details["events"] == 0:
+                    self.log.error(
+                        "Trigger %s exited without sending an event. Dependent tasks will be failed.",
+                        details["name"],
+                    )
+                    self.failed_triggers.append(trigger_id)
+                del self.triggers[trigger_id]
+
+    async def block_watchdog(self):
+        """
+        Watchdog loop that detects blocking (badly-written) triggers.
+
+        Triggers should be well-behaved async coroutines and await whenever
+        they need to wait; this loop tries to run every 100ms to see if
+        there are badly-written triggers taking longer than that and blocking
+        the event loop.
+
+        Unfortunately, we can't tell what trigger is blocking things, but
+        we can at least detect the top-level problem.
+        """
+        while not self.stop:
+            last_run = time.monotonic()
+            await asyncio.sleep(0.1)
+            # We allow a generous amount of buffer room for now, since it might
+            # be a busy event loop.
+            time_elapsed = time.monotonic() - last_run
+            if time_elapsed > 0.2:
+                self.log.error(
+                    "Triggerer's async thread was blocked for %.2f seconds, "
+                    "likely by a badly-written trigger. Set PYTHONASYNCIODEBUG=1 "
+                    "to get more information on overrunning coroutines.",
+                    time_elapsed,
+                )
+
+    # Async trigger logic
+
+    async def run_trigger(self, trigger_id, trigger):
+        """
+        Wrapper which runs an actual trigger (they are async generators)
+        and pushes their events into our outbound event deque.
+        """
+        self.log.info("Trigger %s starting", self.triggers[trigger_id]['name'])
+        try:
+            async for event in trigger.run():
+                self.log.info("Trigger %s fired: %s", self.triggers[trigger_id]['name'], event)
+                self.triggers[trigger_id]["events"] += 1
+                self.events.append((trigger_id, event))
+        finally:
+            # CancelledError will get injected when we're stopped - which is
+            # fine, the cleanup process will understand that, but we want to
+            # allow triggers a chance to cleanup, either in that case or if
+            # they exit cleanly.
+            trigger.cleanup()
+
+    # Main-thread sync API
+
+    def update_triggers(self, requested_trigger_ids: Set[int]):
+        """
+        Called from the main thread to request that we update what
+        triggers we're running.
+
+        Works out the differences - ones to add, and ones to remove - then
+        adds them to the deques so the subthread can actually mutate the running
+        trigger set.
+        """
+        current_trigger_ids = set(self.triggers.keys())

Review comment:
       It _can_ change while it's iterating through it inside `set()`, but most of `set()` and `.keys()` are down at the C level so the interrupt point here is very small.
   
   In addition, it doesn't matter if it mutates during this - the two threads are designed to be run out-of-sync anyway, and for any given key, it has a strict lifecycle where it gets added and then removed and then does not appear again, so for any mutation the subthread can do, it shouldn't cause a problem.
   
   Still, if a lock would make everyone feel better it's trivial to add!




-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638962554



##########
File path: airflow/jobs/scheduler_job.py
##########
@@ -1899,3 +1910,17 @@ def adopt_or_reset_orphaned_tasks(self, session: Session = None):
                     raise
 
         return len(to_reset)
+
+    @provide_session
+    def check_trigger_timeouts(self, session: Session = None):
+        """
+        Looks at all tasks that are in the "deferred" state and whose trigger
+        timeout has passed, so they can be marked as failed.
+        """
+        timed_out_tasks = session.query(TaskInstance).filter(
+            TaskInstance.state == State.DEFERRED, TaskInstance.trigger_timeout < timezone.utcnow()
+        )
+        num_tasks = timed_out_tasks.count()

Review comment:
       Oh, I see! SQLAlchemy is less clever than I thought, then. I'll add it in.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638976812



##########
File path: airflow/models/dag.py
##########
@@ -1232,23 +1253,13 @@ def clear(
                 )
             )
 
-        if start_date:
-            tis = tis.filter(TI.execution_date >= start_date)
-        if end_date:
-            tis = tis.filter(TI.execution_date <= end_date)
-        if only_failed:
-            tis = tis.filter(or_(TI.state == State.FAILED, TI.state == State.UPSTREAM_FAILED))
-        if only_running:
-            tis = tis.filter(TI.state == State.RUNNING)
-        if task_ids:
-            tis = tis.filter(TI.task_id.in_(task_ids))
-
         if include_subdags:
             from airflow.sensors.external_task import ExternalTaskMarker
 
             # Recursively find external tasks indicated by ExternalTaskMarker
-            instances = tis.all()
-            for ti in instances:
+            for ti in (
+                session.query(TI).filter(tuple_(TI.dag_id, TI.task_id, TI.execution_date).in_(tis)).all()

Review comment:
       But it's OK for now? The alternative is an entire refactor of `DAG.clear` and I'm not sure we want to do that here if we can help it.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638964177



##########
File path: airflow/exceptions.py
##########
@@ -225,3 +226,30 @@ def __str__(self):
 
 class ConnectionNotUnique(AirflowException):
     """Raise when multiple values are found for the same conn_id"""
+
+
+class TaskDeferred(BaseException):
+    """
+    Special exception raised to signal that the operator it was raised from
+    wishes to defer until a trigger fires.
+    """
+
+    def __init__(
+        self,
+        *,
+        trigger,
+        method_name: str,
+        kwargs: Optional[Dict[str, Any]] = None,
+        timeout: Optional[datetime.timedelta] = None,
+    ):
+        super().__init__()
+        self.trigger = trigger
+        self.method_name = method_name
+        self.kwargs = kwargs
+        self.timeout = timeout
+        # Check timeout type at runtime
+        if self.timeout is not None and not hasattr(self.timeout, "total_seconds"):
+            raise ValueError("Timeout value must be a timedelta")
+
+    def __repr__(self) -> str:
+        return f"<OperatorDeferred trigger={self.trigger} method={self.method_name}>"

Review comment:
       An exciting view into the past! :D




-- 
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:
users@infra.apache.org



[GitHub] [airflow] MatrixManAtYrService commented on pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
MatrixManAtYrService commented on pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#issuecomment-890406307


   Are deferrable tasks expected to work with the `airflow dags test ...` CLI invocation?
   
   I had some trouble: https://gist.github.com/MatrixManAtYrService/3901210eb9698c72492b0393a21500eb


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r639695257



##########
File path: airflow/models/dag.py
##########
@@ -1223,7 +1244,7 @@ def clear(
                     include_subdags=include_subdags,
                     include_parentdag=False,
                     dag_run_state=dag_run_state,
-                    get_tis=True,
+                    get_ti_keys=True,

Review comment:
       So I'm a little bit confused as to why we had to change this method in this PR at all.
   
   The only place this is called with `get_ti_keys` is in this recursive use here.
   
   So I guess what I'm asking is what in this deferrable PR caused a change in behaviour to the clear function necessitating this change?




-- 
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:
users@infra.apache.org



[GitHub] [airflow] ashb commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r677560283



##########
File path: airflow/models/trigger.py
##########
@@ -0,0 +1,195 @@
+# 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 datetime
+from typing import Any, Dict, List, Optional
+
+from sqlalchemy import BigInteger, Column, String, func
+
+from airflow.models.base import Base
+from airflow.models.taskinstance import TaskInstance
+from airflow.triggers.base import BaseTrigger
+from airflow.utils import timezone
+from airflow.utils.session import provide_session
+from airflow.utils.sqlalchemy import ExtendedJSON, UtcDateTime
+from airflow.utils.state import State
+
+
+class Trigger(Base):
+    """
+    Triggers are a workload that run in an asynchronous event loop shared with
+    other Triggers, and fire off events that will unpause deferred Tasks,
+    start linked DAGs, etc.
+
+    They are persisted into the database and then re-hydrated into a single
+    "triggerer" process, where they're all run at once. We model it so that
+    there is a many-to-one relationship between Task and Trigger, for future
+    deduplication logic to use.
+
+    Rows will be evicted from the database when the triggerer detects no
+    active Tasks/DAGs using them. Events are not stored in the database;
+    when an Event is fired, the triggerer will directly push its data to the
+    appropriate Task/DAG.
+    """
+
+    __tablename__ = "trigger"
+
+    id = Column(BigInteger, primary_key=True)
+    classpath = Column(String(1000), nullable=False)
+    kwargs = Column(ExtendedJSON, nullable=False)
+    created_date = Column(UtcDateTime, nullable=False)
+    triggerer_id = Column(BigInteger, nullable=True)
+
+    def __init__(
+        self, classpath: str, kwargs: Dict[str, Any], created_date: Optional[datetime.datetime] = None
+    ):
+        super().__init__()
+        self.classpath = classpath
+        self.kwargs = kwargs
+        self.created_date = created_date or timezone.utcnow()
+
+    @classmethod
+    def from_object(cls, trigger: BaseTrigger):
+        """
+        Alternative constructor that creates a trigger row based directly
+        off of a Trigger object.
+        """
+        classpath, kwargs = trigger.serialize()
+        return cls(classpath=classpath, kwargs=kwargs)
+
+    @classmethod
+    @provide_session
+    def bulk_fetch(cls, ids: List[int], session=None) -> Dict[int, "Trigger"]:
+        """
+        Fetches all of the Triggers by ID and returns a dict mapping
+        ID -> Trigger instance
+        """
+        return {obj.id: obj for obj in session.query(cls).filter(cls.id.in_(ids)).all()}
+
+    @classmethod
+    @provide_session
+    def clean_unused(cls, session=None):
+        """
+        Deletes all triggers that have no tasks/DAGs dependent on them
+        (triggers have a one-to-many relationship to both)
+        """
+        # Update all task instances with trigger IDs that are not DEFERRED to remove them
+        session.query(TaskInstance).filter(
+            TaskInstance.state != State.DEFERRED, TaskInstance.trigger_id.isnot(None)
+        ).update({TaskInstance.trigger_id: None})
+        # Get all triggers that have no task instances depending on them...
+        ids = [
+            trigger_id
+            for (trigger_id,) in (
+                session.query(cls.id)
+                .join(TaskInstance, cls.id == TaskInstance.trigger_id, isouter=True)
+                .group_by(cls.id)
+                .having(func.count(TaskInstance.trigger_id) == 0)
+            )
+        ]
+        # ...and delete them (we can't do this in one query due to MySQL)
+        session.query(Trigger).filter(Trigger.id.in_(ids)).delete(synchronize_session=False)
+
+    @classmethod
+    @provide_session
+    def submit_event(cls, trigger_id, event, session=None):
+        """
+        Takes an event from an instance of itself, and triggers all dependent
+        tasks to resume.
+        """
+        for task_instance in session.query(TaskInstance).filter(
+            TaskInstance.trigger_id == trigger_id, TaskInstance.state == State.DEFERRED
+        ):
+            # Add the event's payload into the kwargs for the task
+            next_kwargs = task_instance.next_kwargs or {}
+            next_kwargs["event"] = event.payload
+            task_instance.next_kwargs = next_kwargs
+            # Remove ourselves as its trigger
+            task_instance.trigger_id = None
+            # Finally, mark it as scheduled so it gets re-queued
+            task_instance.state = State.SCHEDULED
+
+    @classmethod
+    @provide_session
+    def submit_failure(cls, trigger_id, session=None):
+        """
+        Called when a trigger has failed unexpectedly, and we need to mark
+        everything that depended on it as failed. Notably, we have to actually
+        run the failure code from a worker as it may have linked callbacks, so
+        hilariously we have to re-schedule the task instances to a worker just
+        so they can then fail.
+
+        We use a special __fail__ value for next_method to achieve this that
+        the runtime code understands as immediate-fail, and pack the error into
+        next_kwargs.
+
+        TODO: Once we have shifted callback (and email) handling to run on
+        workers as first-class concepts, we can run the failure code here
+        in-process, but we can't do that right now.
+        """
+        for task_instance in session.query(TaskInstance).filter(
+            TaskInstance.trigger_id == trigger_id, TaskInstance.state == State.DEFERRED
+        ):
+            # Add the error and set the next_method to the fail state
+            task_instance.next_method = "__fail__"
+            task_instance.next_kwargs = {"error": "Trigger failure"}
+            # Remove ourselves as its trigger
+            task_instance.trigger_id = None
+            # Finally, mark it as scheduled so it gets re-queued
+            task_instance.state = State.SCHEDULED
+
+    @classmethod
+    @provide_session
+    def ids_for_triggerer(cls, triggerer_id, session=None):
+        """Retrieves a list of triggerer_ids."""
+        return [row[0] for row in session.query(cls.id).filter(cls.triggerer_id == triggerer_id)]
+
+    @classmethod
+    @provide_session
+    def assign_unassigned(cls, triggerer_id, capacity, session=None):
+        """
+        Takes a triggerer_id and the capacity for that triggerer and assigns unassigned
+        triggers until that capacity is reached, or there are no more unassigned triggers.
+        """
+        from airflow.jobs.base_job import BaseJob  # To avoid circular import
+
+        count = session.query(cls.id).filter(cls.triggerer_id == triggerer_id).count()
+        capacity -= count
+
+        if capacity <= 0:
+            return
+
+        alive_triggerer_ids = [
+            row[0]
+            for row in session.query(BaseJob.id).filter(
+                BaseJob.end_date is None,
+                BaseJob.latest_heartbeat > timezone.utcnow() - datetime.timedelta(seconds=30),
+                BaseJob.job_type == "TriggererJob",
+            )
+        ]
+
+        # find triggers which have a triggerer_id not in list
+        trigger_ids = [
+            row[0]
+            for row in session.query(cls.id)
+            .filter(cls.triggerer_id.notin_(alive_triggerer_ids))
+            .limit(capacity)
+        ]
+
+        session.query(cls).filter(cls.id.in_(trigger_ids)).update(
+            {cls.triggerer_id: triggerer_id},
+            synchronize_session=False,
+        )

Review comment:
       By the last point I meant something _like_ `UPDATE trigger set trigger_id = ? WHERE trigger_id is NULL LIMIT x`




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #15389: AIP-40 prototype: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#issuecomment-832266318


   [The Workflow run](https://github.com/apache/airflow/actions/runs/811425926) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


-- 
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:
users@infra.apache.org



[GitHub] [airflow] andrewgodwin commented on a change in pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
andrewgodwin commented on a change in pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#discussion_r638962002



##########
File path: airflow/api_connexion/endpoints/task_instance_endpoint.py
##########
@@ -248,7 +248,7 @@ def post_clear_task_instances(dag_id: str, session=None):
         error_message = f"Dag id {dag_id} not found"
         raise NotFound(error_message)
     reset_dag_runs = data.pop('reset_dag_runs')
-    task_instances = dag.clear(get_tis=True, **data)
+    task_instances = dag.clear(get_ti_instances=True, **data)

Review comment:
       I'm going to rename the `instances` one back to `get_tis` since the tests have already run with the rename and not found any dangling references, and it should behave the same in 99% of cases - we can keep backwards compat here without too much issue.




-- 
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:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #15389: AIP-40 prototype: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#issuecomment-832161520


   [The Workflow run](https://github.com/apache/airflow/actions/runs/811048277) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


-- 
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:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #15389:
URL: https://github.com/apache/airflow/pull/15389#issuecomment-833042457


   [The Workflow run](https://github.com/apache/airflow/actions/runs/814740884) is cancelling this PR. Building image for the PR has been cancelled


-- 
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:
users@infra.apache.org



[GitHub] [airflow] kaxil merged pull request #15389: AIP-40: Deferrable Tasks

Posted by GitBox <gi...@apache.org>.
kaxil merged pull request #15389:
URL: https://github.com/apache/airflow/pull/15389


   


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org