I don't think there is an issue with TIDeps, there is already a
RunnableExecDateDep
<https://github.com/apache/incubator-airflow/blob/master/airflow/ti_deps/deps/runnable_exec_date_dep.py>
that covers that end dates of tasks.

Here is my toy example that I used to test this behavior -
https://pastebin.com/G9naqXhq

Here is a screenshot of the DagRuns:

​
The first two DagRuns are for 2018-02-01 and 2018-02-02, and were created
by the scheduler, and both tasks ran as expected.

The third run is for 2018-02-03 and was created by me running 'airflow
trigger_dag'. As you can see only the first task got run, which is what I
would have expected. 'airflow task_failed_deps' for the second task gives
the expected answer:

$ airflow task_failed_deps testing_end_dates demo_task_2 2018-02-03 -sd
dags/examples
[2018-02-22 15:32:52,966] {__init__.py:57} INFO - Using executor
CeleryExecutor
...
...
Task instance dependencies not met:
Execution Date: The execution date is 2018-02-03T00:00:00 but this is after
the task's end date 2018-02-02T00:00:00.

Although an additional problem is that the DagRun is still listed as
'running', presumably because the second task hasn't completed (even though
it never will). It's not totally clear to me what the state of that second
task instance should be maybe 'no status' is fine, or maybe they could be
marked as 'skipped' automatically.

I think the over all solution would be to:

   - updated the scheduler so that it will create DagRuns if the end date
   of any task is None or in the future
   - figure out what state task instances who's end date is before the
   execution date should be in
   - update DagRun model to ignore task instances with end dates before the
   execution date when updating the DagRun state

The first of those seems pretty easy to me, the second is mostly just
semantics, but the third seems a bit more complicated (after first glance
at that part of the code). I'll file a ticket and maybe find some time to
work on a PR; only started using Airflow recently and would like to start
contributing back to it.

Chris



On Thu, Feb 22, 2018 at 4:56 AM, Ash Berlin-Taylor <
ash_airflowl...@firemirror.com> wrote:

> That does sound like a bug, and I would have expected, as you did, that
> not specifying an end_date on some tasks means those tasks should run for
> ever.
>
> Changes that probably need making is that a task end_date of None on a
> task should me "greater" than other task dates in/around the lines you
> linked to.
>
> Do we need to add a TIDep https://github.com/apache/
> incubator-airflow/tree/master/airflow/ti_deps/deps <
> https://github.com/apache/incubator-airflow/tree/master/
> airflow/ti_deps/deps> to ensure the exec date is less than the task end
> date?
>
> -ash
>
> > On 21 Feb 2018, at 20:58, Chris Palmer <ch...@crpalmer.com> wrote:
> >
> > I was very surprised to find that if you set an end_date on any of the
> > tasks in a DAG, that the scheduler won't create DagRuns after the minimum
> > end_date of tasks. The code that does this is the 6 or so lines starting
> > here -
> > https://github.com/apache/incubator-airflow/blob/master/
> airflow/jobs.py#L867
> > .
> >
> > So if for example I have:
> >
> >   - a DAG with a start_date of 2018-02-01, no specific end_date and a
> >   daily schedule
> >   - One task in that DAG with no specified end_date
> >   - A second task in that DAG with an end_date of 2018-02-02
> >
> > The scheduler will create a DagRuns for 2018-02-01 and 2018-02-02 but
> will
> > not create a DagRun for 2018-02-03 or later.
> >
> > That seems completely counter intuitive to me. I would expect the
> scheduler
> > to keep creating DagRuns so that the first task can keep running.
> >
> >
> > Interestingly, if I manually created a DagRun for 2018-02-03 then the
> > scheduler would then only scheduled the first task for that
> execution_date
> > and actually respects the end_date of the second task.
> >
> > The only alternative to adding an end_date to a task is to edit the DAG
> and
> > remove those tasks from the DAG entirely. However, that means the
> webserver
> > is no longer aware of those tasks and I can't look at the historical
> > behavior in the UI.
> >
> >
> > Does anyone have explanation for why this logic is there? Is there some
> > necessary use case for that restriction that I'm not thinking about?
> >
> >
> > I could see a similar piece of code that checks to see if all tasks in
> the
> > DAG have specified end_dates and prevents the scheduler from creating
> > DagRuns passed the MAX of those dates. There is no point in creating
> > DagRuns if none of the tasks are going to be run, but as long as at least
> > one task can run for that execution_date I think the scheduler should
> > create it.
> >
> > Thanks
> > Chris
>
>

Reply via email to