You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by "Benjamin Debeerst (JIRA)" <ji...@apache.org> on 2018/03/16 14:44:00 UTC

[jira] [Commented] (AIRFLOW-1837) Differing start_dates on tasks not respected by scheduler.

    [ https://issues.apache.org/jira/browse/AIRFLOW-1837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16401986#comment-16401986 ] 

Benjamin Debeerst commented on AIRFLOW-1837:
--------------------------------------------

As is, the provided dag code is incomplete, because some of the code was stripped. Here is some demo code to show the start date problem:
{code:java}
from datetime import datetime, timedelta

import airflow
from airflow.operators.bash_operator import BashOperator

with airflow.DAG(
        'start_dates_demo',
        dagrun_timeout=timedelta(hours=1),
        schedule_interval="@daily",
        start_date=datetime(2017, 8, 1),
        end_date=datetime(2017, 8, 2),
        default_args={
            'owner': 'demo',
            'start_date': datetime(2017,8,1),
        }) as dag:

    task_1 = BashOperator(task_id='task_1', bash_command="echo 1")
    task_2 = BashOperator(start_date=datetime(2017, 8, 2), task_id='task_2', bash_command="echo 2")

    task_3 = BashOperator(task_id='task_3', bash_command="echo 3")

    task_1 >> task_3
    task_2 >> task_3{code}
This always runs task_2, although it should only run it from 2017-08-02 onwards. I would expect it not have state skipped or no_status.

This is on Airflow 1.9
!everything_ran.png!

 

> Differing start_dates on tasks not respected by scheduler.
> ----------------------------------------------------------
>
>                 Key: AIRFLOW-1837
>                 URL: https://issues.apache.org/jira/browse/AIRFLOW-1837
>             Project: Apache Airflow
>          Issue Type: Bug
>    Affects Versions: 1.9.0
>            Reporter: Ash Berlin-Taylor
>            Priority: Major
>         Attachments: everything_ran.png
>
>
> It it possible to specify start_date directly on tasks in dag, as well as on the DAG. This is correctly handled when creating dag runs, but it is seemingly ignored when scheduling tasks.
> Given this example:
> {code}
> dag_args = {
>     "start_date": datetime(2017, 9, 4),
> }
> dag = DAG(
>     "my-dag",
>     default_args=dag_args,
>     schedule_interval="0 0 * * Mon",
> )
> # ...
> with dag:
>         op = PythonOperator(
>             python_callable=fetcher.run,
>             task_id="fetch_all_respondents",
>             provide_context=True,
>             # The "unfiltered" API calls are a lot quicker, so lets put them
>             # ahead of any other filtered job in the queue.
>             priority_weight=10,
>             start_date=datetime(2014, 9, 1),
>         )
>         op = PythonOperator(
>             python_callable=fetcher.run,
>             task_id="fetch_by_demographics",
>             op_kwargs={
>                 'demo_names': demo_names,
>             },
>             provide_context=True,
>             priority_weight=5,
>         )
> {code}
> I only want the fetch_all_respondents tasks to run for 2014..2017, and then from September 2017 I also want the fetch_by_demographics task to run. However right now both tasks are being scheduled from 2014-09-01.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)