You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2021/11/01 09:51:27 UTC

[GitHub] [airflow] nathadfield opened a new issue #19346: Sensors no longer be respecting `retries` with timeouts.

nathadfield opened a new issue #19346:
URL: https://github.com/apache/airflow/issues/19346


   ### Apache Airflow version
   
   2.2.0
   
   ### Operating System
   
   PRETTY_NAME="Debian GNU/Linux 10 (buster)"
   
   ### Versions of Apache Airflow Providers
   
   n/a
   
   ### Deployment
   
   Astronomer
   
   ### Deployment details
   
   _No response_
   
   ### What happened
   
   Existing DAGs that use sensors with retry and timeout functionality are no longer retrying but instantly going to a failed state rather than up for retry.
   
   ### What you expected to happen
   
   Prior to v2.2 if you defined a sensor with a timeout duration of, for example, 60 seconds and 2 retries this would result in the task running a maximum of 3 possible attempts and a total duration of three minutes (180 seconds).
   
   In earlier versions, once the timeout was reached the task would enter into `up_for_retry` state.
   
   ### How to reproduce
   
   Using this simple DAG you can replicate the issue.
   
   ```
   from datetime import datetime, timedelta
   from airflow import models
   from airflow.sensors.external_task import ExternalTaskSensor
   
   default_args = {
       'owner': 'airflow',
       'retries': 2,
       'retry_delay': timedelta(minutes=5),
       'use_legacy_sql': False,
   }
   
   with models.DAG(
       dag_id='child_dag',
       default_args=default_args,
       start_date=datetime(2021, 10, 1),
       schedule_interval='0 0 * * *',
       catchup=False
   ) as dag:
   
       sensor = ExternalTaskSensor(
           task_id='sensor',
           external_dag_id='parent_dag',
           external_task_id='dummy',
           mode='reschedule',
           poke_interval=10,
           timeout=60
       )
   ```
   
   ### 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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] nathadfield commented on issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
nathadfield commented on issue #19346:
URL: https://github.com/apache/airflow/issues/19346#issuecomment-957224545


   @yuqian90 Ah, my misunderstanding.  Thanks for the clarification.  This does make more sense.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] nathadfield edited a comment on issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
nathadfield edited a comment on issue #19346:
URL: https://github.com/apache/airflow/issues/19346#issuecomment-956284325


   [Here's](https://github.com/apache/airflow/commit/a0e6a847aa72ddb15bdc147695273fb3aec8839d#diff-62f7d8a52fefdb8e05d4f040c6d3459b4a56fe46976c24f68843dbaeb5a98487R1164) the commit that changed this behaviour.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] nathadfield commented on issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
nathadfield commented on issue #19346:
URL: https://github.com/apache/airflow/issues/19346#issuecomment-956290397


   Looks like it was documented in the 2.2 update notes.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish edited a comment on issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
dstandish edited a comment on issue #19346:
URL: https://github.com/apache/airflow/issues/19346#issuecomment-1046065769


   @hojatbay
   
   > What if I don't want to or I cannot increase the timeout period? In my case I need to retry for a period of 24 hours so if I increase the timeout to 24 hours then the task will not release the slot in my environment for 24 hours (its basically a running task that takes 24 hours to complete). As other similar tasks come in, they also take some slots and my environment finally ends up with too many running tasks and causing other issues.
   
   This is a very obvious case for deferrable operators: https://airflow.apache.org/docs/apache-airflow/stable/concepts/deferring.html#:~:text=A%20deferrable%20operator%20is%20one,to%20something%20called%20a%20Trigger.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] nathadfield edited a comment on issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
nathadfield edited a comment on issue #19346:
URL: https://github.com/apache/airflow/issues/19346#issuecomment-956298775


   What I will say is that the change does create some scenarios that you might not want, e.g. if you have a long timeout setting but the task fails for some other reason, then it will restart with the timeout reset.
   
   With the previous behaviour you could be sure that the task would always fail at a certain time after it initially started.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] yuqian90 commented on issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
yuqian90 commented on issue #19346:
URL: https://github.com/apache/airflow/issues/19346#issuecomment-957136738


   I think the new behavior makes more sense, but I'm happy to hear scenarios that makes the old behavior more preferable. If timeout=3600s and retries=5, it means the task has a max of 3600s to finish. During that 3600s, it can retry at most 5 times. If either 3600s or 5 retries is reached, the task fails. So to address the problem initially reported in this issue, the user just need to increase the timeout.
   
   @nathadfield  This statement is not true: "if you have a long timeout setting but the task retries for some other reason, then it will restart with the timeout reset."
   
   The timeout does not reset after the task fails for some other reason. Let's say the timeout is 3600s and retries=1. After 600s, the sensor fails due to a transient connection issues. Then it retries, the timeout will not reset. It still has exactly 3000s left to finish. If it's not done within 3000s seconds, it'll fail.
   
   This is so because this piece of code here looks for the time the task made the first try.
   
   ```python
           if self.reschedule:
   
               # If reschedule, use the start date of the first try (first try can be either the very
               # first execution of the task, or the first execution after the task was cleared.)
               first_try_number = context['ti'].max_tries - self.retries + 1
               task_reschedules = TaskReschedule.find_for_task_instance(
                   context['ti'], try_number=first_try_number
               )
               if task_reschedules:
                   started_at = task_reschedules[0].start_date
               else:
                   started_at = timezone.utcnow()
   ```


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] dstandish commented on issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
dstandish commented on issue #19346:
URL: https://github.com/apache/airflow/issues/19346#issuecomment-1046065769


   > @yuqian90 What if I don't want to or I cannot increase the timeout period? In my case I need to retry for a period of 24 hours so if I increase the timeout to 24 hours then the task will not release the slot in my environment for 24 hours (its basically a running task that takes 24 hours to complete). As other similar tasks come in, they also take some slots and my environment finally ends up with too many running tasks and causing other issues.
   
   This is a very obvious case for deferrable operators: https://airflow.apache.org/docs/apache-airflow/stable/concepts/deferring.html#:~:text=A%20deferrable%20operator%20is%20one,to%20something%20called%20a%20Trigger.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] yuqian90 commented on issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
yuqian90 commented on issue #19346:
URL: https://github.com/apache/airflow/issues/19346#issuecomment-957136738


   I think the new behavior makes more sense, but I'm happy to hear scenarios that makes the old behavior more preferable. If timeout=3600s and retries=5, it means the task has a max of 3600s to finish. During that 3600s, it can retry at most 5 times. If either 3600s or 5 retries is reached, the task fails. So to address the problem initially reported in this issue, the user just need to increase the timeout.
   
   @nathadfield  This statement is not true: "if you have a long timeout setting but the task retries for some other reason, then it will restart with the timeout reset."
   
   The timeout does not reset after the task fails for some other reason. Let's say the timeout is 3600s and retries=1. After 600s, the sensor fails due to a transient connection issues. Then it retries, the timeout will not reset. It still has exactly 3000s left to finish. If it's not done within 3000s seconds, it'll fail.
   
   This is so because this piece of code here looks for the time the task made the first try.
   
   ```python
           if self.reschedule:
   
               # If reschedule, use the start date of the first try (first try can be either the very
               # first execution of the task, or the first execution after the task was cleared.)
               first_try_number = context['ti'].max_tries - self.retries + 1
               task_reschedules = TaskReschedule.find_for_task_instance(
                   context['ti'], try_number=first_try_number
               )
               if task_reschedules:
                   started_at = task_reschedules[0].start_date
               else:
                   started_at = timezone.utcnow()
   ```


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] nathadfield commented on issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
nathadfield commented on issue #19346:
URL: https://github.com/apache/airflow/issues/19346#issuecomment-956193222


   Task log:
   
   ```
   --------------------------------------------------------------------------------
   [2021-11-01, 12:28:55 UTC] {taskinstance.py:1240} INFO - Starting attempt 1 of 3
   [2021-11-01, 12:28:55 UTC] {taskinstance.py:1241} INFO - 
   --------------------------------------------------------------------------------
   [2021-11-01, 12:28:55 UTC] {taskinstance.py:1260} INFO - Executing <Task(ExternalTaskSensor): sensor> on 2021-10-31 00:00:00+00:00
   [2021-11-01, 12:28:55 UTC] {standard_task_runner.py:52} INFO - Started process 611 to run task
   [2021-11-01, 12:28:55 UTC] {standard_task_runner.py:76} INFO - Running: ['***', 'tasks', 'run', 'child_dag', 'sensor', 'scheduled__2021-10-31T00:00:00+00:00', '--job-id', '33', '--raw', '--subdir', 'DAGS_FOLDER/child_dag.py', '--cfg-path', '/tmp/tmpbc8xmbmm', '--error-file', '/tmp/tmp36p794w3']
   [2021-11-01, 12:28:55 UTC] {standard_task_runner.py:77} INFO - Job 33: Subtask sensor
   [2021-11-01, 12:28:55 UTC] {logging_mixin.py:109} INFO - Running <TaskInstance: child_dag.sensor scheduled__2021-10-31T00:00:00+00:00 [running]> on host 986a9620cf97
   [2021-11-01, 12:28:55 UTC] {taskinstance.py:1427} INFO - Exporting the following env vars:
   AIRFLOW_CTX_DAG_OWNER=***
   AIRFLOW_CTX_DAG_ID=child_dag
   AIRFLOW_CTX_TASK_ID=sensor
   AIRFLOW_CTX_EXECUTION_DATE=2021-10-31T00:00:00+00:00
   AIRFLOW_CTX_DAG_RUN_ID=scheduled__2021-10-31T00:00:00+00:00
   [2021-11-01, 12:28:55 UTC] {logging_mixin.py:109} WARNING - /opt/***/***/models/taskinstance.py:1939 DeprecationWarning: Accessing 'execution_date' from the template is deprecated and will be removed in a future version. Please use 'logical_date' or 'data_interval_start' instead.
   [2021-11-01, 12:28:55 UTC] {external_task.py:173} INFO - Poking for tasks ['dummy'] in dag parent_dag on 2021-10-31T00:00:00+00:00 ... 
   [2021-11-01, 12:28:55 UTC] {taskinstance.py:1701} ERROR - Task failed with exception
   Traceback (most recent call last):
     File "/opt/airflow/airflow/models/taskinstance.py", line 1330, in _run_raw_task
       self._execute_task_with_callbacks(context)
     File "/opt/airflow/airflow/models/taskinstance.py", line 1456, in _execute_task_with_callbacks
       result = self._execute_task(context, self.task)
     File "/opt/airflow/airflow/models/taskinstance.py", line 1512, in _execute_task
       result = execute_callable(context=context)
     File "/opt/airflow/airflow/sensors/base.py", line 234, in execute
       raise AirflowSensorTimeout(f"Snap. Time is OUT. DAG id: {log_dag_id}")
   airflow.exceptions.AirflowSensorTimeout: Snap. Time is OUT. DAG id: child_dag
   [2021-11-01, 12:28:55 UTC] {taskinstance.py:1278} INFO - Immediate failure requested. Marking task as FAILED. dag_id=child_dag, task_id=sensor, execution_date=20211031T000000, start_date=20211101T122855, end_date=20211101T122855
   [2021-11-01, 12:28:55 UTC] {standard_task_runner.py:91} ERROR - Failed to execute job 33 for task sensor
   Traceback (most recent call last):
     File "/opt/airflow/airflow/task/task_runner/standard_task_runner.py", line 85, in _start_by_fork
       args.func(args, dag=self.dag)
     File "/opt/airflow/airflow/cli/cli_parser.py", line 48, in command
       return func(*args, **kwargs)
     File "/opt/airflow/airflow/utils/cli.py", line 92, in wrapper
       return f(*args, **kwargs)
     File "/opt/airflow/airflow/cli/commands/task_command.py", line 293, in task_run
       _run_task_by_selected_method(args, dag, ti)
     File "/opt/airflow/airflow/cli/commands/task_command.py", line 107, in _run_task_by_selected_method
       _run_raw_task(args, ti)
     File "/opt/airflow/airflow/cli/commands/task_command.py", line 184, in _run_raw_task
       error_file=args.error_file,
     File "/opt/airflow/airflow/utils/session.py", line 70, in wrapper
       return func(*args, session=session, **kwargs)
     File "/opt/airflow/airflow/models/taskinstance.py", line 1330, in _run_raw_task
       self._execute_task_with_callbacks(context)
     File "/opt/airflow/airflow/models/taskinstance.py", line 1456, in _execute_task_with_callbacks
       result = self._execute_task(context, self.task)
     File "/opt/airflow/airflow/models/taskinstance.py", line 1512, in _execute_task
       result = execute_callable(context=context)
     File "/opt/airflow/airflow/sensors/base.py", line 234, in execute
       raise AirflowSensorTimeout(f"Snap. Time is OUT. DAG id: {log_dag_id}")
   airflow.exceptions.AirflowSensorTimeout: Snap. Time is OUT. DAG id: child_dag
   [2021-11-01, 12:28:55 UTC] {local_task_job.py:154} INFO - Task exited with return code 1
   [2021-11-01, 12:28:55 UTC] {local_task_job.py:264} INFO - 0 downstream tasks scheduled from follow-on schedule check
   ```


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #19346:
URL: https://github.com/apache/airflow/issues/19346#issuecomment-956342382


   Thoughts @yuqian90 ?? ^^


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] nathadfield commented on issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
nathadfield commented on issue #19346:
URL: https://github.com/apache/airflow/issues/19346#issuecomment-956284325


   Here's the commit that changed this behaviour.
   
   https://github.com/apache/airflow/commit/a0e6a847aa72ddb15bdc147695273fb3aec8839d


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] nathadfield commented on issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
nathadfield commented on issue #19346:
URL: https://github.com/apache/airflow/issues/19346#issuecomment-957224545


   @yuqian90 Ah, my misunderstanding.  Thanks for the clarification.  This does make more sense.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] hojatbay commented on issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
hojatbay commented on issue #19346:
URL: https://github.com/apache/airflow/issues/19346#issuecomment-1045304751


   @yuqian90 What if I don't want to or I cannot increase the timeout period? In my case I need to retry for a period of 24 hours so if I increase the timeout to 24 hours then the task will not release the slot in my environment for 24 hours (its basically a running task that takes 24 hours to complet). As other similar tasks come in, they also take some slots and my environment finally ends up with too many running tasks running and causing other issues. 


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #19346:
URL: https://github.com/apache/airflow/issues/19346#issuecomment-956194206


   Tentatively going to add it to 2.2.2 but if it is an actual issue. 
   
   @dstandish Can you try and reproduce this please


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] nathadfield commented on issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
nathadfield commented on issue #19346:
URL: https://github.com/apache/airflow/issues/19346#issuecomment-956298775


   What I will say is that the change does create some scenarios that you might not want, e.g. if you have a long timeout setting but the task fails for some other reason, then it will restart with the timeout reset.
   
   With the previous behaviour you could be sure that the task would always fail at a certain time after it started.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
kaxil commented on issue #19346:
URL: https://github.com/apache/airflow/issues/19346#issuecomment-956342382


   Thoughts @yuqian90 ?? ^^


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] yuqian90 commented on issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
yuqian90 commented on issue #19346:
URL: https://github.com/apache/airflow/issues/19346#issuecomment-957136738


   I think the new behavior makes more sense, but I'm happy to hear scenarios that makes the old behavior more preferable. If timeout=3600s and retries=5, it means the task has a max of 3600s to finish. During that 3600s, it can retry at most 5 times. If either 3600s or 5 retries is reached, the task fails. So to address the problem initially reported in this issue, the user just need to increase the timeout.
   
   @nathadfield  This statement is not true: "if you have a long timeout setting but the task retries for some other reason, then it will restart with the timeout reset."
   
   The timeout does not reset after the task fails for some other reason. Let's say the timeout is 3600s and retries=1. After 600s, the sensor fails due to a transient connection issues. Then it retries, the timeout will not reset. It still has exactly 3000s left to finish. If it's not done within 3000s seconds, it'll fail.
   
   This is so because this piece of code here looks for the time the task made the first try.
   
   ```python
           if self.reschedule:
   
               # If reschedule, use the start date of the first try (first try can be either the very
               # first execution of the task, or the first execution after the task was cleared.)
               first_try_number = context['ti'].max_tries - self.retries + 1
               task_reschedules = TaskReschedule.find_for_task_instance(
                   context['ti'], try_number=first_try_number
               )
               if task_reschedules:
                   started_at = task_reschedules[0].start_date
               else:
                   started_at = timezone.utcnow()
   ```


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] nathadfield commented on issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
nathadfield commented on issue #19346:
URL: https://github.com/apache/airflow/issues/19346#issuecomment-957224545


   @yuqian90 Ah, my misunderstanding.  Thanks for the clarification.  This does make more sense.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] nathadfield closed issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
nathadfield closed issue #19346:
URL: https://github.com/apache/airflow/issues/19346


   


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] nathadfield edited a comment on issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
nathadfield edited a comment on issue #19346:
URL: https://github.com/apache/airflow/issues/19346#issuecomment-956298775


   What I will say is that the change does create some scenarios that you might not want, e.g. if you have a long timeout setting but the task retries for some other reason, then it will restart with the timeout reset.
   
   With the previous behaviour you could be sure that the task would always fail at a certain time after it initially started.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] hojatbay edited a comment on issue #19346: Sensors no longer respecting `retries` with timeouts.

Posted by GitBox <gi...@apache.org>.
hojatbay edited a comment on issue #19346:
URL: https://github.com/apache/airflow/issues/19346#issuecomment-1045304751


   @yuqian90 What if I don't want to or I cannot increase the timeout period? In my case I need to retry for a period of 24 hours so if I increase the timeout to 24 hours then the task will not release the slot in my environment for 24 hours (its basically a running task that takes 24 hours to complete). As other similar tasks come in, they also take some slots and my environment finally ends up with too many running tasks and causing other issues. 


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org