Skip to content

Commit

Permalink
Change AirflowTaskTimeout to inherit BaseException
Browse files Browse the repository at this point in the history
Code that normally catches Exception should not implicitly ignore
interrupts from AirflowTaskTimout.

Fixes apache#35644 apache#35474
  • Loading branch information
hterik committed Feb 20, 2024
1 parent 011cd3d commit beb0898
Show file tree
Hide file tree
Showing 2 changed files with 12 additions and 2 deletions.
5 changes: 4 additions & 1 deletion airflow/exceptions.py
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,10 @@ class InvalidStatsNameException(AirflowException):
"""Raise when name of the stats is invalid."""


class AirflowTaskTimeout(AirflowException):
# Important to inherit BaseException instead of AirflowException->Exception, since this Exception is used
# to explicitly interrupt ongoing task. Code that does normal error-handling should not treat
# such interrupt as an error that can be handled normally. (Compare with KeyboardInterrupt.)
class AirflowTaskTimeout(BaseException):
"""Raise when the task execution times-out."""


Expand Down
9 changes: 8 additions & 1 deletion tests/core/test_core.py
Original file line number Diff line number Diff line change
Expand Up @@ -71,11 +71,18 @@ class InvalidTemplateFieldOperator(BaseOperator):
op.dry_run()

def test_timeout(self, dag_maker):
def sleep_and_catch_other_exceptions():
try:
sleep(5)
# Catching Exception should NOT catch AirflowTaskTimeout
except Exception:
pass

with dag_maker():
op = PythonOperator(
task_id="test_timeout",
execution_timeout=timedelta(seconds=1),
python_callable=lambda: sleep(5),
python_callable=sleep_and_catch_other_exceptions,
)
dag_maker.create_dagrun()
with pytest.raises(AirflowTaskTimeout):
Expand Down

0 comments on commit beb0898

Please sign in to comment.