alealandreev opened a new issue, #47971: URL: https://github.com/apache/airflow/issues/47971
### Apache Airflow version Other Airflow 2 version (please specify below) ### If "Other Airflow 2 version" selected, which one? 2.10.3 ### What happened? Hello, I encountered with a bug. My DAG configs were: retries=1000, retry_delay=5 min (300 seconds), max_retry_delay=1h (3600 seconds). My DAG failed ~1000 times and after that Scheduler broke down. After that retries exceeded 1000 and stopped on 1017 retry attempt. I did my research on this problem and found that this happened due to formula **min_backoff = math.ceil(delay.total_seconds() * (2 ** (self.try_number - 1)))** in **taskinstance.py** file. So retry_exponential_backoff has no limit of try_number and during calculations it can overflow max Float value. So even if max_retry_delay is set formula is still calculating. And during calculations on very large retry number it crashes. Please fix bug. I also did pull request with my possible solution: https://github.com/apache/airflow/pull/47967 From Airflow logs: 2024-12-09 02:16:39.825 OverflowError: cannot convert float infinity to integer 2024-12-09 02:16:39.825 min_backoff = int(math.ceil(delay.total_seconds() * (2 ** (self.try_number - 2)))) 2024-12-08 09:29:14.583 [2024-12-08T06:29:14.583+0000] {scheduler_job_runner.py:705} INFO - Executor reports execution of mydag.spark_submit run_id=manual__2024-11-02T10:19:30.618008+00:00 exited with status up_for_retry for try_number 470 Configs: _with DAG( dag_id=DAG_ID, start_date=MYDAG_START_DATE, schedule_interval="@daily", catchup=AIRFLOW_CATCHUP, default_args={ 'depends_on_past': True, "retries": 1000, "retry_delay": duration(minutes=5), "retry_exponential_backoff": True, "max_retry_delay": duration(hours=1), }, ) as dag:_ <img width="947" alt="Image" src="https://github.com/user-attachments/assets/f3307b23-0307-4b4d-b968-3e1984fbe93c" /> <img width="1050" alt="Image" src="https://github.com/user-attachments/assets/f161329c-155c-4d92-b3b4-cf442d6ed036" /> ### What you think should happen instead? My pull request: https://github.com/apache/airflow/pull/47967 ### How to reproduce Use configs from above. Example: with DAG( dag_id=DAG_ID, start_date=MY_AIRFLOW_START_DATE, schedule_interval="@daily", catchup=AIRFLOW_CATCHUP, default_args={ 'depends_on_past': True, "retries": 1000, "retry_delay": duration(minutes=5), "retry_exponential_backoff": True, "max_retry_delay": duration(hours=1), }, ) as dag ### Operating System Ubuntu 22.04 ### Versions of Apache Airflow Providers _No response_ ### Deployment Official Apache Airflow Helm Chart ### 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]
