cosinequanon opened a new issue, #29819:
URL: https://github.com/apache/airflow/issues/29819
### Apache Airflow version
2.5.1
### What happened
If an Operator specifies a template_field with `execution_timeout` then the
DAG will serialize correctly but throw an error during deserialization. This
causes the entire scheduler to crash and breaks the application.
### What you think should happen instead
The scheduler should never go down because of some code someone wrote, this
should probably throw an error during serialization.
### How to reproduce
Define an operator like this
```
class ExecutionTimeoutOperator(BaseOperator):
template_fields = ("execution_timeout", )
def __init__(self, execution_timeout: timedelta, **kwargs):
super().__init__(**kwargs)
self.execution_timeout = execution_timeout
```
then make a dag like this
```
dag = DAG(
"serialize_with_default",
schedule_interval="0 12 * * *",
start_date=datetime(2023, 2, 28),
catchup=False,
default_args={
"execution_timeout": timedelta(days=4),
},
)
with dag:
execution = ExecutionTimeoutOperator(task_id="execution",
execution_timeout=timedelta(hours=1))
```
that will break the scheduler, you can force the stack trace by doing this
```
from airflow.models import DagBag
db = DagBag('dags/siri/staging/exp_airflow_dags', read_dags_from_db=True)
db.get_dag('serialize_with_default')
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/usr/local/lib/python3.9/site-packages/airflow/utils/session.py",
line 75, in wrapper
return func(*args, session=session, **kwargs)
File "/usr/local/lib/python3.9/site-packages/airflow/models/dagbag.py",
line 190, in get_dag
self._add_dag_from_db(dag_id=dag_id, session=session)
File "/usr/local/lib/python3.9/site-packages/airflow/models/dagbag.py",
line 265, in _add_dag_from_db
dag = row.dag
File
"/usr/local/lib/python3.9/site-packages/airflow/models/serialized_dag.py", line
218, in dag
dag = SerializedDAG.from_dict(self.data)
File
"/usr/local/lib/python3.9/site-packages/airflow/serialization/serialized_objects.py",
line 1287, in from_dict
return cls.deserialize_dag(serialized_obj["dag"])
File
"/usr/local/lib/python3.9/site-packages/airflow/serialization/serialized_objects.py",
line 1194, in deserialize_dag
v = {task["task_id"]: SerializedBaseOperator.deserialize_operator(task)
for task in v}
File
"/usr/local/lib/python3.9/site-packages/airflow/serialization/serialized_objects.py",
line 1194, in <dictcomp>
v = {task["task_id"]: SerializedBaseOperator.deserialize_operator(task)
for task in v}
File
"/usr/local/lib/python3.9/site-packages/airflow/serialization/serialized_objects.py",
line 955, in deserialize_operator
cls.populate_operator(op, encoded_op)
File
"/usr/local/lib/python3.9/site-packages/airflow/serialization/serialized_objects.py",
line 864, in populate_operator
v = cls._deserialize_timedelta(v)
File
"/usr/local/lib/python3.9/site-packages/airflow/serialization/serialized_objects.py",
line 513, in _deserialize_timedelta
return datetime.timedelta(seconds=seconds)
TypeError: unsupported type for timedelta seconds component: str
```
### Operating System
Mac 13.1 (22C65)
### Versions of Apache Airflow Providers
apache-airflow-providers-amazon==5.1.0
apache-airflow-providers-apache-hdfs==3.2.0
apache-airflow-providers-apache-hive==5.1.1
apache-airflow-providers-apache-spark==4.0.0
apache-airflow-providers-celery==3.1.0
apache-airflow-providers-cncf-kubernetes==5.1.1
apache-airflow-providers-common-sql==1.3.3
apache-airflow-providers-datadog==3.1.0
apache-airflow-providers-ftp==3.3.0
apache-airflow-providers-http==4.1.1
apache-airflow-providers-imap==3.1.1
apache-airflow-providers-jdbc==3.3.0
apache-airflow-providers-jenkins==3.2.0
apache-airflow-providers-mysql==4.0.0
apache-airflow-providers-pagerduty==3.1.0
apache-airflow-providers-postgres==5.4.0
apache-airflow-providers-presto==4.2.1
apache-airflow-providers-slack==7.2.0
apache-airflow-providers-sqlite==3.3.1
apache-airflow-providers-ssh==3.4.0
### Deployment
Docker-Compose
### Deployment details
I could repro this with docker-compose and in a helm backed deployment so I
don't think it's really related to the deployment details
### Anything else
_No response_
### Are you willing to submit PR?
- [ ] Yes I am willing to submit a PR!
### Code of Conduct
- [X] I agree to follow this project's [Code of
Conduct](https://github.com/apache/airflow/blob/main/CODE_OF_CONDUCT.md)
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]