tirkarthi opened a new issue, #55134: URL: https://github.com/apache/airflow/issues/55134
### Apache Airflow version main (development) ### If "Other Airflow 2 version" selected, which one? _No response_ ### What happened? Trying to parse below dag gives import error due to `start_date` being a string. `start_date` and `end_date` is accepted as a string in Airflow 2. This is blocking automated migration of dags using ruff. It seems `end_date` is str is handled but `start_date` is not which should also be handled. There are tests for this but I am not sure why it happens during dag processing but not in the tests as it seems only triggered when `start_date` to dag constructor is datetime but `start_date` in default_args is str https://github.com/apache/airflow/blob/2b91be8e490714b4fef68a6d89a695caa1003745/task-sdk/src/airflow/sdk/definitions/dag.py#L461-L463 https://github.com/apache/airflow/blob/2b91be8e490714b4fef68a6d89a695caa1003745/task-sdk/tests/task_sdk/definitions/test_dag.py#L72-L76 ``` python Python 3.11.11 (main, Dec 4 2024, 08:55:08) [GCC 9.4.0] on linux Type "help", "copyright", "credits" or "license" for more information. >>> from airflow import DAG >>> import datetime >>> DAG("test", default_args={"start_date": "2025-01-01"}) <DAG: test> >>> DAG("test", start_date=datetime.datetime(2025, 1, 1), default_args={"start_date": "2025-01-01"}) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "<attrs generated methods airflow.models.dag.DAG>", line 86, in __init__ File "/home/karthikeyan/stuff/python/airflow/task-sdk/src/airflow/sdk/definitions/dag.py", line 468, in __attrs_post_init__ self.default_args["start_date"] = timezone.convert_to_utc(start_date) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ File "/home/karthikeyan/stuff/python/airflow/task-sdk/src/airflow/sdk/_shared/timezones/timezone.py", line 89, in convert_to_utc if not is_localized(value): ^^^^^^^^^^^^^^^^^^^ File "/home/karthikeyan/stuff/python/airflow/task-sdk/src/airflow/sdk/_shared/timezones/timezone.py", line 50, in is_localized return value.utcoffset() is not None ^^^^^^^^^^^^^^^ AttributeError: 'str' object has no attribute 'utcoffset' ``` ```python from __future__ import annotations from datetime import datetime from airflow import DAG with DAG( dag_id="start_date_str", start_date=datetime(2025, 1, 1), catchup=False, schedule="@daily", default_args={ "start_date": "2025-01-01", # "start_date": datetime(2025, 1, 1), "end_date": "2026-01-01", }, ) as dag: pass ``` import error ``` Traceback (most recent call last): File "/home/karthikeyan/stuff/python/airflow/task-sdk/src/airflow/sdk/_shared/timezones/timezone.py", line 89, in convert_to_utc if not is_localized(value): ^^^^^^^^^^^^^^^^^^^ File "/home/karthikeyan/stuff/python/airflow/task-sdk/src/airflow/sdk/_shared/timezones/timezone.py", line 50, in is_localized return value.utcoffset() is not None ^^^^^^^^^^^^^^^ AttributeError: 'str' object has no attribute 'utcoffset' ``` ### What you think should happen instead? _No response_ ### How to reproduce 1. Try parsing the below dag and see import error. 2. Commenting out `start_date` as datetime parses the dag. ```python from __future__ import annotations from datetime import datetime from airflow import DAG with DAG( dag_id="start_date_str", start_date=datetime(2025, 1, 1), catchup=False, schedule="@daily", default_args={ "start_date": "2025-01-01", # "start_date": datetime(2025, 1, 1), "end_date": "2026-01-01", }, ) as dag: pass ``` ### Operating System Ubuntu 20.04 ### Versions of Apache Airflow Providers _No response_ ### Deployment Virtualenv installation ### Deployment details _No response_ ### Anything else? _No response_ ### Are you willing to submit PR? - [x] 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]
