This is an automated email from the ASF dual-hosted git repository. jedcunningham pushed a commit to branch v2-2-test in repository https://gitbox.apache.org/repos/asf/airflow.git
commit e35588c6cbf0cb75ab914d8f31252c4ff87fde89 Author: Robin Edwards <[email protected]> AuthorDate: Thu Oct 14 20:46:51 2021 +0100 BugFix: Null execution date on insert to ``task_fail`` violating NOT NULL (#18979) Fixes #18943 null exec date on insert to task_fail The dag_run property isn't populated by refresh_from_db or when this is called from the failure handler when reaping zombies. This resulted in an IntegrityError violating the NOT NULL constraint on task_fail (cherry picked from commit 2966e3c5065d133c3db23e77db7424f752cc5e9c) --- airflow/models/taskinstance.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index b178ff0..2563109 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -1702,7 +1702,8 @@ class TaskInstance(Base, LoggingMixin): session.add(Log(State.FAILED, self)) # Log failure duration - session.add(TaskFail(task, self.execution_date, self.start_date, self.end_date)) + dag_run = self.get_dagrun(session=session) # self.dag_run not populated by refresh_from_db + session.add(TaskFail(task, dag_run.execution_date, self.start_date, self.end_date)) # Ensure we unset next_method and next_kwargs to ensure that any # retries don't re-use them.
