def schedule(self):
     self.log.info("Starting the scheduler.")
     self._restore_unfinished_dag_run()
     while True:
         identified_message = self.mailbox.get_identified_message()
         origin_event = identified_message.deserialize()
         self.log.debug("Event: {}".format(origin_event))
         if SchedulerInnerEventUtil.is_inner_event(origin_event):
             event = SchedulerInnerEventUtil.to_inner_event(origin_event)
         else:
             event = origin_event
         with create_session() as session:
             if isinstance(event, BaseEvent):
                 dagruns = self._find_dagruns_by_event(event, session)
                 for dagrun in dagruns:
                     dag_run_id = DagRunId(dagrun.dag_id, dagrun.run_id)
                     self.task_event_manager.handle_event(dag_run_id, event)
             elif isinstance(event, RequestEvent):
                 self._process_request_event(event)
             elif isinstance(event, TaskSchedulingEvent):
                 self._schedule_task(event)
             elif isinstance(event, TaskStatusChangedEvent):
                 dagrun = self._find_dagrun(event.dag_id,
                                            event.execution_date, session)
                 tasks = self._find_scheduled_tasks(dagrun, session)
                 self._send_scheduling_task_events(tasks,
                                                   SchedulingAction.START)
                 if dagrun.state in State.finished:
                     self.mailbox.send_message(
                         DagRunFinishedEvent(dagrun.run_id).to_event())
             elif isinstance(event, DagExecutableEvent):
                 dagrun = self._create_dag_run(event.dag_id,
                                               session=session)
                 tasks = self._find_scheduled_tasks(dagrun, session)
                 self._send_scheduling_task_events(tasks,
                                                   SchedulingAction.START)
             elif isinstance(event, EventHandleEvent):
                 dag_runs = DagRun.find(dag_id=event.dag_id,
                                        run_id=event.dag_run_id)
                 assert len(dag_runs) == 1
                 ti = dag_runs[0].get_task_instance(event.task_id)
                 self._send_scheduling_task_event(ti, event.action)
             elif isinstance(event, StopDagEvent):
                 self._stop_dag(event.dag_id, session)
             elif isinstance(event, DagRunFinishedEvent):
                 self._remove_periodic_events(event.run_id)
             elif isinstance(event, PeriodicEvent):
                 dag_runs = DagRun.find(run_id=event.run_id)
                 assert len(dag_runs) == 1
                 ti = dag_runs[0].get_task_instance(event.task_id)
                 self._send_scheduling_task_event(ti,
                                                  SchedulingAction.RESTART)
             elif isinstance(event, StopSchedulerEvent):
                 self.log.info("{} {}".format(self.id, event.job_id))
                 if self.id == event.job_id or 0 == event.job_id:
                     self.log.info("break the scheduler event loop.")
                     identified_message.remove_handled_message()
                     session.expunge_all()
                     break
             elif isinstance(event, ParseDagRequestEvent) or isinstance(
                     event, ParseDagResponseEvent):
                 pass
             elif isinstance(event, ResponseEvent):
                 pass
             else:
                 self.log.error(
                     "can not handler the event {}".format(event))
             identified_message.remove_handled_message()
             session.expunge_all()
     self._stop_timer()
    def _find_scheduled_tasks(
            self,
            dag_run: DagRun,
            session: Session,
            check_execution_date=False) -> Optional[List[TI]]:
        """
        Make scheduling decisions about an individual dag run

        ``currently_active_runs`` is passed in so that a batch query can be
        used to ask this for all dag runs in the batch, to avoid an n+1 query.

        :param dag_run: The DagRun to schedule
        :return: scheduled tasks
        """
        if not dag_run or dag_run.get_state() in State.finished:
            return
        try:
            dag = dag_run.dag = self.dagbag.get_dag(dag_run.dag_id,
                                                    session=session)
        except SerializedDagNotFound:
            self.log.exception("DAG '%s' not found in serialized_dag table",
                               dag_run.dag_id)
            return None

        if not dag:
            self.log.error("Couldn't find dag %s in DagBag/DB!",
                           dag_run.dag_id)
            return None

        currently_active_runs = session.query(TI.execution_date, ).filter(
            TI.dag_id == dag_run.dag_id,
            TI.state.notin_(list(State.finished)),
        ).all()

        if check_execution_date and dag_run.execution_date > timezone.utcnow(
        ) and not dag.allow_future_exec_dates:
            self.log.warning("Execution date is in future: %s",
                             dag_run.execution_date)
            return None

        if dag.max_active_runs:
            if (len(currently_active_runs) >= dag.max_active_runs
                    and dag_run.execution_date not in currently_active_runs):
                self.log.info(
                    "DAG %s already has %d active runs, not queuing any tasks for run %s",
                    dag.dag_id,
                    len(currently_active_runs),
                    dag_run.execution_date,
                )
                return None

        self._verify_integrity_if_dag_changed(dag_run=dag_run, session=session)

        schedulable_tis, callback_to_run = dag_run.update_state(
            session=session, execute_callbacks=False)
        dag_run.schedule_tis(schedulable_tis, session)
        session.commit()

        query = (session.query(TI).outerjoin(TI.dag_run).filter(
            or_(DR.run_id.is_(None),
                DR.run_type != DagRunType.BACKFILL_JOB)).join(
                    TI.dag_model).filter(not_(DM.is_paused)).filter(
                        TI.state == State.SCHEDULED).options(
                            selectinload('dag_model')))
        scheduled_tis: List[TI] = with_row_locks(
            query,
            of=TI,
            **skip_locked(session=session),
        ).all()
        # filter need event tasks
        serialized_dag = session.query(SerializedDagModel).filter(
            SerializedDagModel.dag_id == dag_run.dag_id).first()
        final_scheduled_tis = []
        event_task_set = []
        if serialized_dag:
            dep: DagEventDependencies = DagEventDependencies.from_json(
                serialized_dag.event_relationships)
            event_task_set = dep.find_event_dependencies_tasks()
        else:
            self.log.error(
                "Failed to get serialized_dag from db, unexpected dag id: %s",
                dag_run.dag_id)
        for ti in scheduled_tis:
            if ti.task_id not in event_task_set:
                final_scheduled_tis.append(ti)

        return final_scheduled_tis
Exemplo n.º 3
0
    def _schedule_dag_run(
        self,
        dag_run: DagRun,
        session: Session,
    ) -> Optional[DagCallbackRequest]:
        """
        Make scheduling decisions about an individual dag run

        :param dag_run: The DagRun to schedule
        :return: Callback that needs to be executed
        """
        dag = dag_run.dag = self.dagbag.get_dag(dag_run.dag_id, session=session)

        if not dag:
            self.log.error("Couldn't find dag %s in DagBag/DB!", dag_run.dag_id)
            return 0

        if (
            dag_run.start_date
            and dag.dagrun_timeout
            and dag_run.start_date < timezone.utcnow() - dag.dagrun_timeout
        ):
            dag_run.set_state(State.FAILED)
            unfinished_task_instances = (
                session.query(TI)
                .filter(TI.dag_id == dag_run.dag_id)
                .filter(TI.run_id == dag_run.run_id)
                .filter(TI.state.in_(State.unfinished))
            )
            for task_instance in unfinished_task_instances:
                task_instance.state = State.SKIPPED
                session.merge(task_instance)
            session.flush()
            self.log.info("Run %s of %s has timed-out", dag_run.run_id, dag_run.dag_id)

            callback_to_execute = DagCallbackRequest(
                full_filepath=dag.fileloc,
                dag_id=dag.dag_id,
                run_id=dag_run.run_id,
                is_failure_callback=True,
                msg='timed_out',
            )

            # Send SLA & DAG Success/Failure Callbacks to be executed
            self._send_dag_callbacks_to_processor(dag_run, callback_to_execute)

            return 0

        if dag_run.execution_date > timezone.utcnow() and not dag.allow_future_exec_dates:
            self.log.error("Execution date is in future: %s", dag_run.execution_date)
            return 0

        self._verify_integrity_if_dag_changed(dag_run=dag_run, session=session)
        # TODO[HA]: Rename update_state -> schedule_dag_run, ?? something else?
        schedulable_tis, callback_to_run = dag_run.update_state(session=session, execute_callbacks=False)

        # This will do one query per dag run. We "could" build up a complex
        # query to update all the TIs across all the execution dates and dag
        # IDs in a single query, but it turns out that can be _very very slow_
        # see #11147/commit ee90807ac for more details
        dag_run.schedule_tis(schedulable_tis, session)

        return callback_to_run
Exemplo n.º 4
0
 def _get_next_dagruns_to_examine(self, state: DagRunState, session: Session):
     """Get Next DagRuns to Examine with retries"""
     return DagRun.next_dagruns_to_examine(state, session)
Exemplo n.º 5
0
    def _execute(self, session=None):
        """
        Initializes all components required to run a dag for a specified date range and
        calls helper method to execute the tasks.
        """
        ti_status = BackfillJob._DagRunTaskStatus()

        start_date = self.bf_start_date

        # Get DagRun schedule between the start/end dates, which will turn into dag runs.
        dagrun_start_date = timezone.coerce_datetime(start_date)
        if self.bf_end_date is None:
            dagrun_end_date = pendulum.now(timezone.utc)
        else:
            dagrun_end_date = pendulum.instance(self.bf_end_date)
        dagrun_infos = list(self.dag.iter_dagrun_infos_between(dagrun_start_date, dagrun_end_date))
        if self.run_backwards:
            tasks_that_depend_on_past = [t.task_id for t in self.dag.task_dict.values() if t.depends_on_past]
            if tasks_that_depend_on_past:
                raise AirflowException(
                    f'You cannot backfill backwards because one or more '
                    f'tasks depend_on_past: {",".join(tasks_that_depend_on_past)}'
                )
            dagrun_infos = dagrun_infos[::-1]

        if not dagrun_infos:
            if not self.run_at_least_once:
                self.log.info("No run dates were found for the given dates and dag interval.")
                return
            dagrun_infos = [DagRunInfo.interval(dagrun_start_date, dagrun_end_date)]

        dag_with_subdags_ids = [d.dag_id for d in self._get_dag_with_subdags()]
        running_dagruns = DagRun.find(
            dag_id=dag_with_subdags_ids,
            execution_start_date=self.bf_start_date,
            execution_end_date=self.bf_end_date,
            no_backfills=True,
            state=DagRunState.RUNNING,
        )

        if running_dagruns:
            for run in running_dagruns:
                self.log.error(
                    "Backfill cannot be created for DagRun %s in %s, as there's already %s in a RUNNING "
                    "state.",
                    run.run_id,
                    run.execution_date.strftime("%Y-%m-%dT%H:%M:%S"),
                    run.run_type,
                )
            self.log.error(
                "Changing DagRun into BACKFILL would cause scheduler to lose track of executing "
                "tasks. Not changing DagRun type into BACKFILL, and trying insert another DagRun into "
                "database would cause database constraint violation for dag_id + execution_date "
                "combination. Please adjust backfill dates or wait for this DagRun to finish.",
            )
            return
        # picklin'
        pickle_id = None

        if not self.donot_pickle and self.executor_class not in (
            executor_constants.LOCAL_EXECUTOR,
            executor_constants.SEQUENTIAL_EXECUTOR,
            executor_constants.DASK_EXECUTOR,
        ):
            pickle = DagPickle(self.dag)
            session.add(pickle)
            session.commit()
            pickle_id = pickle.id

        executor = self.executor
        executor.job_id = "backfill"
        executor.start()

        ti_status.total_runs = len(dagrun_infos)  # total dag runs in backfill

        try:
            remaining_dates = ti_status.total_runs
            while remaining_dates > 0:
                dagrun_infos_to_process = [
                    dagrun_info
                    for dagrun_info in dagrun_infos
                    if dagrun_info.logical_date not in ti_status.executed_dag_run_dates
                ]
                self._execute_dagruns(
                    dagrun_infos=dagrun_infos_to_process,
                    ti_status=ti_status,
                    executor=executor,
                    pickle_id=pickle_id,
                    start_date=start_date,
                    session=session,
                )

                remaining_dates = ti_status.total_runs - len(ti_status.executed_dag_run_dates)
                err = self._collect_errors(ti_status=ti_status, session=session)
                if err:
                    if not self.continue_on_failures or ti_status.deadlocked:
                        raise BackfillUnfinished(err, ti_status)

                if remaining_dates > 0:
                    self.log.info(
                        "max_active_runs limit for dag %s has been reached "
                        " - waiting for other dag runs to finish",
                        self.dag_id,
                    )
                    time.sleep(self.delay_on_limit_secs)
        except (KeyboardInterrupt, SystemExit):
            self.log.warning("Backfill terminated by user.")

            # TODO: we will need to terminate running task instances and set the
            # state to failed.
            self._set_unfinished_dag_runs_to_failed(ti_status.active_runs)
        finally:
            session.commit()
            executor.end()

        self.log.info("Backfill done. Exiting.")
Exemplo n.º 6
0
    def test_backfill_max_limit_check(self):
        dag_id = 'test_backfill_max_limit_check'
        run_id = 'test_dagrun'
        start_date = DEFAULT_DATE - datetime.timedelta(hours=1)
        end_date = DEFAULT_DATE

        dag_run_created_cond = threading.Condition()

        def run_backfill(cond):
            cond.acquire()
            # this session object is different than the one in the main thread
            with create_session() as thread_session:
                try:
                    dag = self._get_dag_test_max_active_limits(dag_id)

                    # Existing dagrun that is not within the backfill range
                    dag.create_dagrun(
                        run_id=run_id,
                        state=State.RUNNING,
                        execution_date=DEFAULT_DATE + datetime.timedelta(hours=1),
                        start_date=DEFAULT_DATE,
                    )

                    thread_session.commit()
                    cond.notify()
                finally:
                    cond.release()
                    thread_session.close()

                executor = MockExecutor()
                job = BackfillJob(dag=dag,
                                  start_date=start_date,
                                  end_date=end_date,
                                  executor=executor,
                                  donot_pickle=True)
                job.run()

        backfill_job_thread = threading.Thread(target=run_backfill,
                                               name="run_backfill",
                                               args=(dag_run_created_cond,))

        dag_run_created_cond.acquire()
        with create_session() as session:
            backfill_job_thread.start()
            try:
                # at this point backfill can't run since the max_active_runs has been
                # reached, so it is waiting
                dag_run_created_cond.wait(timeout=1.5)
                dagruns = DagRun.find(dag_id=dag_id)
                dr = dagruns[0]
                self.assertEqual(1, len(dagruns))
                self.assertEqual(dr.run_id, run_id)

                # allow the backfill to execute
                # by setting the existing dag run to SUCCESS,
                # backfill will execute dag runs 1 by 1
                dr.set_state(State.SUCCESS)
                session.merge(dr)
                session.commit()

                backfill_job_thread.join()

                dagruns = DagRun.find(dag_id=dag_id)
                self.assertEqual(3, len(dagruns))  # 2 from backfill + 1 existing
                self.assertEqual(dagruns[-1].run_id, dr.run_id)
            finally:
                dag_run_created_cond.release()
Exemplo n.º 7
0
    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)