def test_backfill_execute_subdag(self): dag = self.dagbag.get_dag('example_subdag_operator') subdag_op_task = dag.get_task('section-1') subdag = subdag_op_task.subdag subdag.schedule_interval = '@daily' start_date = timezone.utcnow() executor = MockExecutor() job = BackfillJob(dag=subdag, start_date=start_date, end_date=start_date, executor=executor, donot_pickle=True) job.run() subdag_op_task.pre_execute(context={'execution_date': start_date}) subdag_op_task.execute(context={'execution_date': start_date}) subdag_op_task.post_execute(context={'execution_date': start_date}) history = executor.history subdag_history = history[0] # check that all 5 task instances of the subdag 'section-1' were executed self.assertEqual(5, len(subdag_history)) for sdh in subdag_history: ti = sdh[3] self.assertIn('section-1-task-', ti.task_id) with create_session() as session: successful_subdag_runs = ( session.query(DagRun).filter( DagRun.dag_id == subdag.dag_id).filter( DagRun.execution_date == start_date) # pylint: disable=comparison-with-callable .filter(DagRun.state == State.SUCCESS).count()) self.assertEqual(1, successful_subdag_runs) subdag.clear() dag.clear()
def test_backfill_execute_subdag_with_removed_task(self): """ Ensure that subdag operators execute properly in the case where an associated task of the subdag has been removed from the dag definition, but has instances in the database from previous runs. """ dag = self.dagbag.get_dag('example_subdag_operator') subdag = dag.get_task('section-1').subdag executor = MockExecutor() job = BackfillJob(dag=subdag, start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, executor=executor, donot_pickle=True) removed_task_ti = TI(task=DummyOperator(task_id='removed_task'), execution_date=DEFAULT_DATE, state=State.REMOVED) removed_task_ti.dag_id = subdag.dag_id session = settings.Session() session.merge(removed_task_ti) session.commit() with timeout(seconds=30): job.run() for task in subdag.tasks: instance = session.query(TI).filter( TI.dag_id == subdag.dag_id, TI.task_id == task.task_id, TI.execution_date == DEFAULT_DATE).first() self.assertIsNotNone(instance) self.assertEqual(instance.state, State.SUCCESS) removed_task_ti.refresh_from_db() self.assertEqual(removed_task_ti.state, State.REMOVED) subdag.clear() dag.clear()
def test_heartbeat_failed_fast(self, mock_getpid): """ Test that task heartbeat will sleep when it fails fast """ mock_getpid.return_value = 1 self.mock_base_job_sleep.side_effect = time.sleep with create_session() as session: dagbag = DagBag( dag_folder=TEST_DAG_FOLDER, include_examples=False, ) dag_id = 'test_heartbeat_failed_fast' task_id = 'test_heartbeat_failed_fast_op' dag = dagbag.get_dag(dag_id) task = dag.get_task(task_id) dag.create_dagrun(run_id="test_heartbeat_failed_fast_run", state=State.RUNNING, execution_date=DEFAULT_DATE, start_date=DEFAULT_DATE, session=session) ti = TaskInstance(task=task, execution_date=DEFAULT_DATE) ti.refresh_from_db() ti.state = State.RUNNING ti.hostname = get_hostname() ti.pid = 1 session.commit() job = LocalTaskJob(task_instance=ti, executor=MockExecutor(do_update=False)) job.heartrate = 2 heartbeat_records = [] job.heartbeat_callback = lambda session: heartbeat_records.append(job.latest_heartbeat) job._execute() self.assertGreater(len(heartbeat_records), 2) for i in range(1, len(heartbeat_records)): time1 = heartbeat_records[i - 1] time2 = heartbeat_records[i] # Assert that difference small enough delta = (time2 - time1).total_seconds() self.assertAlmostEqual(delta, job.heartrate, delta=0.05)
def test_backfill_pool_not_found(self): dag = self._get_dummy_dag( dag_id='test_backfill_pool_not_found', pool='king_pool', ) executor = MockExecutor() job = BackfillJob( dag=dag, executor=executor, start_date=DEFAULT_DATE, end_date=DEFAULT_DATE + datetime.timedelta(days=7), ) try: job.run() except AirflowException: return self.fail()
def test_backfill_max_limit_check_complete_loop(self): dag = self._get_dag_test_max_active_limits( 'test_backfill_max_limit_check_complete_loop') start_date = DEFAULT_DATE - datetime.timedelta(hours=1) end_date = DEFAULT_DATE # Given the max limit to be 1 in active dag runs, we need to run the # backfill job 3 times success_expected = 2 executor = MockExecutor() job = BackfillJob(dag=dag, start_date=start_date, end_date=end_date, executor=executor, donot_pickle=True) job.run() success_dagruns = len(DagRun.find(dag_id=dag.dag_id, state=State.SUCCESS)) running_dagruns = len(DagRun.find(dag_id=dag.dag_id, state=State.RUNNING)) self.assertEqual(success_expected, success_dagruns) self.assertEqual(0, running_dagruns) # no dag_runs in running state are left
def test_backfill_rerun_upstream_failed_tasks(self): dag = DAG(dag_id='test_backfill_rerun_upstream_failed', start_date=DEFAULT_DATE, schedule_interval='@daily') with dag: t1 = DummyOperator( task_id='test_backfill_rerun_upstream_failed_task-1', dag=dag) t2 = DummyOperator( task_id='test_backfill_rerun_upstream_failed_task-2', dag=dag) t1.set_upstream(t2) dag.clear() executor = MockExecutor() job = BackfillJob( dag=dag, executor=executor, start_date=DEFAULT_DATE, end_date=DEFAULT_DATE + datetime.timedelta(days=2), ) job.run() ti = TI( task=dag.get_task('test_backfill_rerun_upstream_failed_task-1'), execution_date=DEFAULT_DATE) ti.refresh_from_db() ti.set_state(State.UPSTREAM_FAILED) job = BackfillJob(dag=dag, executor=executor, start_date=DEFAULT_DATE, end_date=DEFAULT_DATE + datetime.timedelta(days=2), rerun_failed_tasks=True) job.run() ti = TI( task=dag.get_task('test_backfill_rerun_upstream_failed_task-1'), execution_date=DEFAULT_DATE) ti.refresh_from_db() self.assertEqual(ti.state, State.SUCCESS)
def test_backfill_examples(self, dag_id, expected_execution_order): """ Test backfilling example dags Try to backfill some of the example dags. Be careful, not all dags are suitable for doing this. For example, a dag that sleeps forever, or does not have a schedule won't work here since you simply can't backfill them. """ dag = self.dagbag.get_dag(dag_id) logger.info('*** Running example DAG: %s', dag.dag_id) executor = MockExecutor() job = BackfillJob(dag=dag, start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, executor=executor, ignore_first_depends_on_past=True) job.run() self.assertListEqual( [((dag_id, task_id, DEFAULT_DATE, 1), State.SUCCESS) for task_id in expected_execution_order], executor.sorted_tasks)
def test_backfill_depends_on_past(self): """ Test that backfill respects ignore_depends_on_past """ dag = self.dagbag.get_dag('test_depends_on_past') dag.clear() run_date = DEFAULT_DATE + datetime.timedelta(days=5) # backfill should deadlock self.assertRaisesRegex( AirflowException, 'BackfillJob is deadlocked', BackfillJob(dag=dag, start_date=run_date, end_date=run_date).run) BackfillJob(dag=dag, start_date=run_date, end_date=run_date, executor=MockExecutor(), ignore_first_depends_on_past=True).run() # ti should have succeeded ti = TI(dag.tasks[0], run_date) ti.refresh_from_db() self.assertEqual(ti.state, State.SUCCESS)
def test_backfill_run_backwards(self): dag = self.dagbag.get_dag("test_start_date_scheduling") dag.clear() executor = MockExecutor(parallelism=16) job = BackfillJob(executor=executor, dag=dag, start_date=DEFAULT_DATE, end_date=DEFAULT_DATE + datetime.timedelta(days=1), run_backwards=True) job.run() session = settings.Session() tis = session.query(TI).filter( TI.dag_id == 'test_start_date_scheduling' and TI.task_id == 'dummy').order_by(TI.execution_date).all() queued_times = [ti.queued_dttm for ti in tis] self.assertTrue(queued_times == sorted(queued_times, reverse=True)) self.assertTrue(all(ti.state == State.SUCCESS for ti in tis)) dag.clear() session.close()
def test_backfill_rerun_failed_tasks_without_flag(self): dag = DAG( dag_id='test_backfill_rerun_failed', start_date=DEFAULT_DATE, schedule_interval='@daily') with dag: DummyOperator( task_id='test_backfill_rerun_failed_task-1', dag=dag) dag.clear() executor = MockExecutor() job = BackfillJob(dag=dag, executor=executor, start_date=DEFAULT_DATE, end_date=DEFAULT_DATE + datetime.timedelta(days=2), ) job.run() ti = TI(task=dag.get_task('test_backfill_rerun_failed_task-1'), execution_date=DEFAULT_DATE) ti.refresh_from_db() ti.set_state(State.FAILED) job = BackfillJob(dag=dag, executor=executor, start_date=DEFAULT_DATE, end_date=DEFAULT_DATE + datetime.timedelta(days=2), rerun_failed_tasks=False ) with self.assertRaises(AirflowException): job.run()
def test_backfill_fill_blanks(self): dag = DAG( 'test_backfill_fill_blanks', start_date=DEFAULT_DATE, default_args={'owner': 'owner1'}, ) with dag: op1 = DummyOperator(task_id='op1') op2 = DummyOperator(task_id='op2') op3 = DummyOperator(task_id='op3') op4 = DummyOperator(task_id='op4') op5 = DummyOperator(task_id='op5') op6 = DummyOperator(task_id='op6') dag.clear() dr = dag.create_dagrun(run_id='test', state=State.RUNNING, execution_date=DEFAULT_DATE, start_date=DEFAULT_DATE) executor = MockExecutor() session = settings.Session() tis = dr.get_task_instances() for ti in tis: if ti.task_id == op1.task_id: ti.state = State.UP_FOR_RETRY ti.end_date = DEFAULT_DATE elif ti.task_id == op2.task_id: ti.state = State.FAILED elif ti.task_id == op3.task_id: ti.state = State.SKIPPED elif ti.task_id == op4.task_id: ti.state = State.SCHEDULED elif ti.task_id == op5.task_id: ti.state = State.UPSTREAM_FAILED # op6 = None session.merge(ti) session.commit() session.close() job = BackfillJob(dag=dag, start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, executor=executor) self.assertRaisesRegex(AirflowException, 'Some task instances failed', job.run) self.assertRaises(sqlalchemy.orm.exc.NoResultFound, dr.refresh_from_db) # the run_id should have changed, so a refresh won't work drs = DagRun.find(dag_id=dag.dag_id, execution_date=DEFAULT_DATE) dr = drs[0] self.assertEqual(dr.state, State.FAILED) tis = dr.get_task_instances() for ti in tis: if ti.task_id in (op1.task_id, op4.task_id, op6.task_id): self.assertEqual(ti.state, State.SUCCESS) elif ti.task_id == op2.task_id: self.assertEqual(ti.state, State.FAILED) elif ti.task_id == op3.task_id: self.assertEqual(ti.state, State.SKIPPED) elif ti.task_id == op5.task_id: self.assertEqual(ti.state, State.UPSTREAM_FAILED)
def main(num_runs, repeat, pre_create_dag_runs, executor_class, dag_ids): """ This script can be used to measure the total "scheduler overhead" of Airflow. By overhead we mean if the tasks executed instantly as soon as they are executed (i.e. they do nothing) how quickly could we schedule them. It will monitor the task completion of the Mock/stub executor (no actual tasks are run) and after the required number of dag runs for all the specified dags have completed all their tasks, it will cleanly shut down the scheduler. The dags you run with need to have an early enough start_date to create the desired number of runs. Care should be taken that other limits (DAG concurrency, pool size etc) are not the bottleneck. This script doesn't help you in that regard. It is recommended to repeat the test at least 3 times (`--repeat=3`, the default) so that you can get somewhat-accurate variance on the reported timing numbers, but this can be disabled for longer runs if needed. """ # Turn on unit test mode so that we don't do any sleep() in the scheduler # loop - not needed on master, but this script can run against older # releases too! os.environ['AIRFLOW__CORE__UNIT_TEST_MODE'] = 'True' os.environ['AIRFLOW__CORE__DAG_CONCURRENCY'] = '500' # Set this so that dags can dynamically configure their end_date os.environ['AIRFLOW_BENCHMARK_MAX_DAG_RUNS'] = str(num_runs) os.environ['PERF_MAX_RUNS'] = str(num_runs) if pre_create_dag_runs: os.environ['AIRFLOW__SCHEDULER__USE_JOB_SCHEDULE'] = 'False' from airflow.jobs.scheduler_job import SchedulerJob from airflow.models.dagbag import DagBag from airflow.utils import db dagbag = DagBag() dags = [] with db.create_session() as session: pause_all_dags(session) for dag_id in dag_ids: dag = dagbag.get_dag(dag_id) dag.sync_to_db(session=session) dags.append(dag) reset_dag(dag, session) next_run_date = dag.normalize_schedule(dag.start_date or min(t.start_date for t in dag.tasks)) for _ in range(num_runs - 1): next_run_date = dag.following_schedule(next_run_date) end_date = dag.end_date or dag.default_args.get('end_date') if end_date != next_run_date: message = ( f"DAG {dag_id} has incorrect end_date ({end_date}) for number of runs! " f"It should be " f" {next_run_date}") sys.exit(message) if pre_create_dag_runs: create_dag_runs(dag, num_runs, session) ShortCircuitExecutor = get_executor_under_test(executor_class) executor = ShortCircuitExecutor(dag_ids_to_watch=dag_ids, num_runs=num_runs) scheduler_job = SchedulerJob(dag_ids=dag_ids, do_pickle=False, executor=executor) executor.scheduler_job = scheduler_job total_tasks = sum(len(dag.tasks) for dag in dags) if 'PYSPY' in os.environ: pid = str(os.getpid()) filename = os.environ.get('PYSPY_O', 'flame-' + pid + '.html') os.spawnlp(os.P_NOWAIT, 'sudo', 'sudo', 'py-spy', 'record', '-o', filename, '-p', pid, '--idle') times = [] # Need a lambda to refer to the _latest_ value for scheduler_job, not just # the initial one code_to_test = lambda: scheduler_job.run() # pylint: disable=unnecessary-lambda for count in range(repeat): gc.disable() start = time.perf_counter() code_to_test() times.append(time.perf_counter() - start) gc.enable() print("Run %d time: %.5f" % (count + 1, times[-1])) if count + 1 != repeat: with db.create_session() as session: for dag in dags: reset_dag(dag, session) executor.reset(dag_ids) scheduler_job = SchedulerJob(dag_ids=dag_ids, do_pickle=False, executor=executor) executor.scheduler_job = scheduler_job print() print() msg = "Time for %d dag runs of %d dags with %d total tasks: %.4fs" if len(times) > 1: print((msg + " (±%.3fs)") % (num_runs, len(dags), total_tasks, statistics.mean(times), statistics.stdev(times))) else: print(msg % (num_runs, len(dags), total_tasks, times[0])) print() print()