airflow-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Ash Berlin-Taylor <...@apache.org>
Subject Re: dag_run timeouts
Date Tue, 26 Feb 2019 10:50:05 GMT
Hmmm yeah, dag_run timeout is more closely a "cache eviction" sort of time out, and the only
time the dag run timeout comes in to play iis when a DAG has reached it's maximum active runs,
at which point one of the older ones that has exceeded it's timeout will be "evicted" (i.e.
failed).

Having an actuall timeout on the DAG run, after which time the run would be marked as failed
does sound like a useful thing.

I think we should:
- Rename dag.dagrun_timeout as it is not an actual timeout. (I don't have an idea what else
to call it. Perhaps "dagrun_evictable_after"? That with a bit of docs might be clear enough?)
- Add an execution_timeout to the DAG as a whole.

For the DAG execution_timeout we need to be clear about when the period starts. And also think
about how this is implemented so that it is possible to manually run/re-run individual tasks
after the run has timed-out, without them just immediately getting cancelled and timed-out.
Perhaps a new TaskDep, as those can already be ignored?

-ash

> On 15 Feb 2019, at 07:19, vardanguptacse@gmail.com wrote:
> 
> Hi team,
> 
> We wanted to enable dag_run timeouts on our dags but when we have gone through the behavior
of how dagrun_timeout works, we got to know, it only works on when below conditions are met.
> 
> 1) dagrun should be scheduled one i.e. not manually created
> 2) max_active_runs must be configured
> 
> When it works:
> During dagrun creation, if count of existing dagruns equate to max_active_runs configured
and previous runs are running longer than configured timeout, it will be marked as failed(https://github.com/apache/airflow/blob/master/airflow/jobs.py#L784)
> 
> How can we achieve below:
> 
> 1) With manually created dag_runs
> 2) Enabling timeouts on existing dag_runs without requiring triggering of new dag_runs
> 3) Though dag_run is marked as failed but running task will keep on running until it
reaches terminal state.
> 
> Workaround:
> We also explored execution_timeouts at an individual task level in combination with corresponding
trigger rule, this works perfectly for us.
> 
> Regards,
> Vardan Gupta


Mime
View raw message