kaxil commented on a change in pull request #18310:
URL: https://github.com/apache/airflow/pull/18310#discussion_r710960302
##########
File path: tests/jobs/test_local_task_job.py
##########
@@ -707,6 +707,48 @@ def test_fast_follow(
if scheduler_job.processor_agent:
scheduler_job.processor_agent.end()
+ @conf_vars({('scheduler', 'schedule_after_task_execution'): 'True'})
+ def test_mini_scheduler_works_with_wait_for_upstream(self, caplog,
dag_maker):
+ session = settings.Session()
+ dep = {'A': 'B', 'B': 'C'}
+ with dag_maker(default_args={'wait_for_downstream': True},
catchup=False) as dag:
+ task_a = PythonOperator(task_id='A', python_callable=lambda: True)
+ task_b = PythonOperator(task_id='B', python_callable=lambda: True)
+ task_c = PythonOperator(task_id='C', python_callable=lambda: True)
+ for upstream, downstream in dep.items():
+ dag.set_dependency(upstream, downstream)
+
+ scheduler_job = SchedulerJob(subdir=os.devnull)
+ scheduler_job.dagbag.bag_dag(dag, root_dag=dag)
+
+ dr = dag.create_dagrun(run_id='test_1', state=State.RUNNING,
execution_date=DEFAULT_DATE)
+ dr2 = dag.create_dagrun(
+ run_id='test_2', state=State.RUNNING, execution_date=DEFAULT_DATE
+ datetime.timedelta(hours=1)
+ )
+ ti_a = TaskInstance(task_a, run_id=dr.run_id, state=State.SUCCESS)
+ ti_b = TaskInstance(task_b, run_id=dr.run_id, state=State.SUCCESS)
+ ti_c = TaskInstance(task_c, run_id=dr.run_id, state=State.RUNNING)
+ ti2_a = TaskInstance(task_a, run_id=dr2.run_id, state=State.NONE)
+ ti2_b = TaskInstance(task_b, run_id=dr2.run_id, state=State.NONE)
+ ti2_c = TaskInstance(task_c, run_id=dr2.run_id, state=State.NONE)
+ session.merge(ti_a)
+ session.merge(ti_b)
+ session.merge(ti_c)
+ session.merge(ti2_a)
+ session.merge(ti2_b)
+ session.merge(ti2_c)
+ session.flush()
+
+ job1 = LocalTaskJob(task_instance=ti2_a, ignore_ti_state=True,
executor=SequentialExecutor())
+ job1.task_runner = StandardTaskRunner(job1)
+ settings.engine.dispose()
+ job1.run()
+ ti2_a.refresh_from_db()
+ assert (
Review comment:
```suggestion
assert ti2_a.state=State.NONE
assert (
```
##########
File path: tests/jobs/test_local_task_job.py
##########
@@ -707,6 +707,48 @@ def test_fast_follow(
if scheduler_job.processor_agent:
scheduler_job.processor_agent.end()
+ @conf_vars({('scheduler', 'schedule_after_task_execution'): 'True'})
+ def test_mini_scheduler_works_with_wait_for_upstream(self, caplog,
dag_maker):
+ session = settings.Session()
+ dep = {'A': 'B', 'B': 'C'}
+ with dag_maker(default_args={'wait_for_downstream': True},
catchup=False) as dag:
+ task_a = PythonOperator(task_id='A', python_callable=lambda: True)
+ task_b = PythonOperator(task_id='B', python_callable=lambda: True)
+ task_c = PythonOperator(task_id='C', python_callable=lambda: True)
+ for upstream, downstream in dep.items():
+ dag.set_dependency(upstream, downstream)
+
+ scheduler_job = SchedulerJob(subdir=os.devnull)
+ scheduler_job.dagbag.bag_dag(dag, root_dag=dag)
+
+ dr = dag.create_dagrun(run_id='test_1', state=State.RUNNING,
execution_date=DEFAULT_DATE)
+ dr2 = dag.create_dagrun(
+ run_id='test_2', state=State.RUNNING, execution_date=DEFAULT_DATE
+ datetime.timedelta(hours=1)
+ )
+ ti_a = TaskInstance(task_a, run_id=dr.run_id, state=State.SUCCESS)
+ ti_b = TaskInstance(task_b, run_id=dr.run_id, state=State.SUCCESS)
+ ti_c = TaskInstance(task_c, run_id=dr.run_id, state=State.RUNNING)
+ ti2_a = TaskInstance(task_a, run_id=dr2.run_id, state=State.NONE)
+ ti2_b = TaskInstance(task_b, run_id=dr2.run_id, state=State.NONE)
+ ti2_c = TaskInstance(task_c, run_id=dr2.run_id, state=State.NONE)
+ session.merge(ti_a)
+ session.merge(ti_b)
+ session.merge(ti_c)
+ session.merge(ti2_a)
+ session.merge(ti2_b)
+ session.merge(ti2_c)
+ session.flush()
+
+ job1 = LocalTaskJob(task_instance=ti2_a, ignore_ti_state=True,
executor=SequentialExecutor())
+ job1.task_runner = StandardTaskRunner(job1)
+ settings.engine.dispose()
Review comment:
why do we need this?
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]