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/08/19 15:07:51 UTC

[GitHub] [airflow] SDubrulle opened a new issue #17734: Backfilling ignores task-level start_date and end_date

SDubrulle opened a new issue #17734:
URL: https://github.com/apache/airflow/issues/17734


   **Apache Airflow version**:
   
   2.1.0
   
   **OS**:
   
   Debian GNU/Linux 10
   
   **Apache Airflow Provider versions**:
   
   Irrelevant for this issue.
   
   **Deployment**:
   
   K8S using a custom helm chart
   
   **What happened**:
   
   When backfilling a DAG with a --mark-success flag, all tasks (within specified timerange) are set to success.
   
   **What you expected to happen**:
   
   I expected the task-level start_date and end_date to be taken into account. As such, a task-instance would only be created if the dag execution date is within the task date range.
   
   **How to reproduce it**:
   
   
   # BACKFILL COMMAND
   # airflow dags backfill -s 2020-08-01 -e 2021-8-18 --mark-success start_date_test_dag
   ##
   
   ## DAG
   from datetime import datetime
   from airflow.decorators import dag, task
   
   @dag(
       default_args={},
       description="Start date test DAG",
       schedule_interval="0 0 * * *",
       start_date=datetime(2021, 8, 1),
       catchup=True,
   )
   def start_date_test_dag():
   
       @task()
       def since_dag_start_date():
           print("Done")
   
       @task(start_date=datetime(2021, 8, 10))
       def task_defined_start_date():
           print("Done")
   
       [since_dag_start_date(), task_defined_start_date()]
   
   dag = start_date_test_dag()
   
   ## DAG END
   
   **Are you willing to submit a PR?**
   
   Sure, if this is unwanted behaviour and there is no workaround (other than backfilling each task individually).


-- 
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] uranusjr commented on issue #17734: Backfilling ignores task-level start_date and end_date

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


   I believe this difference originates in the dependency definitions for (manually-triggered) backfills and scheduler-triggered runs. In [`dependencies_deps.py`](https://github.com/apache/airflow/blob/efdfd15477f92da059fa86b4fa18b6f29cb97feb/airflow/ti_deps/dependencies_deps.py), there's a `ExecDateAfterStartDateDep` in `SCHEDULER_QUEUED_DEPS` that would skips a task if its `start_date` is later than the run's logical date (`execution_date`), but that dependency is not declared in `BACKFILL_QUEUED_DEPS`.
   
   This difference dates _way_ back to #5079 when `BACKFILL_QUEUED_DEPS` was introduced, and unfortunately no rationales were provided, from what I can tell. Maybe some of the people involved at the time could remember something?
   
   /cc @kaxil @ashb 


-- 
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] eladkal edited a comment on issue #17734: Backfilling ignores task-level start_date and end_date

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


   It make sense that backfill should respect the task definition for tasks `start_date` and `end_date`.
   
   Giving another example if we have a dag as:
   ```
   with DAG(
       'my_dag',
       default_args=default_args,
       schedule_interval='@daily',
       start_date=datetime(2019, 1, 1),
   ) as dag:
       t1 = BashOperator(
           task_id='print_date',
           bash_command='date',
       )
         t2 = BashOperator(
           task_id='print_date2',
           bash_command='date',
           start_date=datetime(2021, 1, 1)
       )
   ```
   
   and you are running `airflow dags backfill my_dag --start-date 2019-01-01 --end-date 2019-06-01`
   I wouldn't expect `print_date2` to be a part of this backfill at all. To my perspective the backfill `start-date` & `stop-date` specify the range of runs that needs to be created - in these runs it doesn't make sense to consider tasks that are not part of this window.
   
   If someone has a different take on that I would love to hear the reasons
   


-- 
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] boring-cyborg[bot] commented on issue #17734: Backfilling ignores task-level start_date and end_date

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on issue #17734:
URL: https://github.com/apache/airflow/issues/17734#issuecomment-901994730


   Thanks for opening your first issue here! Be sure to follow the issue template!
   


-- 
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] eladkal commented on issue #17734: Backfilling ignores task-level start_date and end_date

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


   It make sense that backfill should respect the task definition for tasks `start_date` and `end_date`.
   If we have a dag as:
   ```
   with DAG(
       'my_dag',
       default_args=default_args,
       schedule_interval='@daily',
       start_date=datetime(2019, 1, 1),
   ) as dag:
       t1 = BashOperator(
           task_id='print_date',
           bash_command='date',
       )
         t2 = BashOperator(
           task_id='print_date2',
           bash_command='date',
           start_date=datetime(2021, 1, 1)
       )
   ```
   
   and you are running `airflow dags backfill my_dag --start-date 2019-01-01 --end-date 2019-06-01`
   I wouldn't expect `print_date2` to be a part of this backfill at all. To my perspective the backfill `start-date` & `stop-date` specify the range of runs that needs to be created - in these runs it doesn't make sense to consider tasks that are not part of this window.
   
   If someone has a different take on that I would love to hear the reasons
   


-- 
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