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 2020/12/03 09:07:27 UTC

[GitHub] [airflow] htgeis opened a new issue #12777: it won't take effect when updating the start_date in the default_args

htgeis opened a new issue #12777:
URL: https://github.com/apache/airflow/issues/12777


   **Apache Airflow version**: v1.10.14
   
   
   **Environment**:
   
   - **Cloud provider or hardware configuration**: 
   - **OS** (e.g. from /etc/os-release):  Ubuntu 16.04.3 LTS
   - **Kernel** (e.g. `uname -a`): Linux aor-k8s-ci-test-01 4.4.0-1111-aws #123-Ubuntu SMP Sat Jul 4 02:03:15 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux
   - **Install tools**: pip
   - **Others**: 
   
   **What happened**:
   it won't take effect when updating the start_date in the default_args. 
   e.g. a daily job like this
   `
              default_args = {
                   'owner': 'airflow',
                   'depends_on_past': False,
                   'start_date': timezone.datetime(2020, 5, 1)
               }
               dag = DAG(dag_id,
                         schedule_interval=timedelta(days=1),
                         max_active_runs=5,
                         catchup=True,
                         default_args=default_args)
   `
   It would schedule the first dag run of "2020/05/01" and if we updated start_date above to "2020/10/01", the next dag it would schedule is "2020/05/02" instead of "2020/10/01", which means it would not take effect. 
   
   **What you expected to happen**:
   
   when start_date has changed, the execution_time of the next dag run should not be early than it. 
   
   **How to reproduce it**:
   ```
       def test_dag_when_start_date_changes(self):
           """
           Test to check that a DAG could schedule reasonable when start_date defined in
           default_args has changed
           """
   
           def setup_dag(dag_id, schedule_interval, start_date, catchup):
               default_args = {
                   'owner': 'airflow',
                   'depends_on_past': False,
                   'start_date': start_date
               }
               dag = DAG(dag_id,
                         schedule_interval=schedule_interval,
                         max_active_runs=5,
                         catchup=catchup,
                         default_args=default_args)
   
               t1 = DummyOperator(task_id='t1', dag=dag)
               t2 = DummyOperator(task_id='t2', dag=dag)
               t2.set_upstream(t1)
               t3 = DummyOperator(task_id='t3', dag=dag)
               t3.set_upstream(t2)
   
               session = settings.Session()
               orm_dag = DagModel(dag_id=dag.dag_id)
               session.merge(orm_dag)
               session.commit()
               session.close()
   
               return dag
   
           now = timezone.utcnow()
           half_an_hour_ago = now - datetime.timedelta(minutes=30)
           next_month = now + datetime.timedelta(days=30)
   
           scheduler = SchedulerJob()
   
           dag1 = setup_dag(dag_id='dag_with_catchup',
                            schedule_interval=timedelta(days=1),
                            start_date=timezone.datetime(2020, 5, 1),
                            catchup=True)
           dr = scheduler.create_dag_run(dag1)
           self.assertIsNotNone(dr)
           self.assertEqual(dr.execution_date, timezone.datetime(2020, 5, 1))
           dr = scheduler.create_dag_run(dag1)
           self.assertIsNotNone(dr)
           self.assertEqual(dr.execution_date, timezone.datetime(2020, 5, 2))
           # update start_date
           dag1 = setup_dag(dag_id='dag_with_catchup',
                            schedule_interval=timedelta(days=1),
                            start_date=timezone.datetime(2020, 10, 1),
                            catchup=True)
           dr = scheduler.create_dag_run(dag1)
           self.assertIsNotNone(dr)
           self.assertEqual(dr.execution_date, timezone.datetime(2020, 10, 1))
           dr = scheduler.create_dag_run(dag1)
           self.assertIsNotNone(dr)
           self.assertEqual(dr.execution_date, timezone.datetime(2020, 10, 2))
   
           dag2 = setup_dag(dag_id='dag_with_catchup_cron',
                            schedule_interval='0 0 * * *',
                            start_date=timezone.datetime(2020, 5, 1),
                            catchup=True)
           dr = scheduler.create_dag_run(dag2)
           self.assertIsNotNone(dr)
           self.assertEqual(dr.execution_date, timezone.datetime(2020, 5, 1))
           dr = scheduler.create_dag_run(dag2)
           self.assertIsNotNone(dr)
           self.assertEqual(dr.execution_date, timezone.datetime(2020, 5, 2))
           # update start_date
           dag2 = setup_dag(dag_id='dag_with_catchup_cron',
                            schedule_interval='0 0 * * *',
                            start_date=timezone.datetime(2020, 10, 1),
                            catchup=True)
           dr = scheduler.create_dag_run(dag2)
           self.assertIsNotNone(dr)
           self.assertEqual(dr.execution_date, timezone.datetime(2020, 10, 1))
           dr = scheduler.create_dag_run(dag2)
           self.assertIsNotNone(dr)
           self.assertEqual(dr.execution_date, timezone.datetime(2020, 10, 2))
   
           dag3 = setup_dag(dag_id='dag_without_catchup_with_future_day',
                            schedule_interval='*/10 * * * *',
                            start_date=timezone.datetime(2020, 5, 1),
                            catchup=False)
           dr = scheduler.create_dag_run(dag3)
           self.assertIsNotNone(dr)
           self.assertGreater(dr.execution_date, half_an_hour_ago)
           # update start_date
           dag3 = setup_dag(dag_id='dag_without_catchup_with_future_day',
                            schedule_interval='*/10 * * * *',
                            start_date=next_month,
                            catchup=False)
           dr = scheduler.create_dag_run(dag3)
           self.assertIsNone(dr)
   
           dag4 = setup_dag(dag_id='dag_with_catchup_once',
                            schedule_interval='@once',
                            start_date=timezone.datetime(2020, 5, 1),
                            catchup=True)
           dr = scheduler.create_dag_run(dag4)
           self.assertIsNotNone(dr)
           self.assertEqual(dr.execution_date, timezone.datetime(2020, 5, 1))
           # update start_date
           dag4 = setup_dag(dag_id='dag_with_catchup_once',
                            schedule_interval='@once',
                            start_date=timezone.datetime(2020, 10, 1),
                            catchup=True)
           dr = scheduler.create_dag_run(dag4)
           self.assertIsNone(dr)
   ```
   
   **Anything else we need to know**:
   


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

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



[GitHub] [airflow] ashb commented on issue #12777: It won't take effect when updating the start_date in the default_args

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


   > the next dag it would schedule is "2020/05/02" instead of "2020/10/01", which means it would not take effect.
   
   Am I right in thinking that it would schedule those dags, bug not actually run any of the tasks _in_ those dag runs currently?


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

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



[GitHub] [airflow] htgeis commented on issue #12777: It won't take effect when updating the start_date in the default_args

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


   yeah, but it doesn't make much sense to create these dag runs with no tasks runnable. We have many users defined the start_date in the default_args and they have some requirements to change the start_date. Sometimes, it's a larger time interval between these two dates, thus it would waste time to waiting for these empty dag runs created and it's also not friendly to show dag runs in web ui as they would see a larger number of empty dags runs. 


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

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



[GitHub] [airflow] boring-cyborg[bot] commented on issue #12777: it won't take effect when updating the start_date in the default_args

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


   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.

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



[GitHub] [airflow] htgeis removed a comment on issue #12777: It won't take effect when updating the start_date in the default_args

Posted by GitBox <gi...@apache.org>.
htgeis removed a comment on issue #12777:
URL: https://github.com/apache/airflow/issues/12777#issuecomment-739113065


   yeah, but it doesn't make much sense to create these dag runs with no tasks runnable. We have many users defined the start_date in the default_args and they have some requirements to change the start_date. Sometimes, it's a larger time interval between these two dates, thus it would waste time to waiting for these empty dag runs created and it's also not friendly to show dag runs in web ui as they would see a larger number of empty dags runs. 


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

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



[GitHub] [airflow] htgeis commented on issue #12777: It won't take effect when updating the start_date in the default_args

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


   > > the next dag it would schedule is "2020/05/02" instead of "2020/10/01", which means it would not take effect.
   > 
   > Am I right in thinking that it would schedule those dags, bug not actually run any of the tasks _in_ those dag runs currently?
   
   yeah, but it doesn't make much sense to create these dag runs with no tasks runnable. We have many users defined the start_date in the default_args and they have some requirements to change the start_date. Sometimes, it's a larger time interval between these two dates, thus it would waste time to waiting for these empty dag runs created and it's also not friendly to show dag runs in web ui as they would see a larger number of empty dags runs.


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

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