def _emit_duration_stats_for_finished_state(self): if self.state == State.RUNNING: return duration = (self.end_date - self.start_date) if self.state is State.SUCCESS: Stats.timing('dagrun.duration.success.{}'.format(self.dag_id), duration) elif self.state == State.FAILED: Stats.timing('dagrun.duration.failed.{}'.format(self.dag_id), duration)
def heartbeat(self): # Triggering new jobs if not self.parallelism: open_slots = len(self.queued_tasks) else: open_slots = self.parallelism - len(self.running) num_running_tasks = len(self.running) num_queued_tasks = len(self.queued_tasks) self.log.debug("%s running task instances", num_running_tasks) self.log.debug("%s in queue", num_queued_tasks) self.log.debug("%s open slots", open_slots) Stats.gauge('executor.open_slots', open_slots) Stats.gauge('executor.queued_tasks', num_queued_tasks) Stats.gauge('executor.running_tasks', num_running_tasks) sorted_queue = sorted( [(k, v) for k, v in self.queued_tasks.items()], key=lambda x: x[1][1], reverse=True) for i in range(min((open_slots, len(self.queued_tasks)))): key, (command, _, queue, simple_ti) = sorted_queue.pop(0) self.queued_tasks.pop(key) self.running[key] = command self.execute_async(key=key, command=command, queue=queue, executor_config=simple_ti.executor_config) # Calling child class sync method self.log.debug("Calling the %s sync method", self.__class__) self.sync()
def verify_integrity(self, session=None): """ Verifies the DagRun by checking for removed tasks or tasks that are not in the database yet. It will set state to removed or add the task if required. """ from airflow.models.taskinstance import TaskInstance # Avoid circular import dag = self.get_dag() tis = self.get_task_instances(session=session) # check for removed or restored tasks task_ids = [] for ti in tis: task_ids.append(ti.task_id) task = None try: task = dag.get_task(ti.task_id) except AirflowException: if ti.state == State.REMOVED: pass # ti has already been removed, just ignore it elif self.state is not State.RUNNING and not dag.partial: self.log.warning("Failed to get task '{}' for dag '{}'. " "Marking it as removed.".format(ti, dag)) Stats.incr( "task_removed_from_dag.{}".format(dag.dag_id), 1, 1) ti.state = State.REMOVED is_task_in_dag = task is not None should_restore_task = is_task_in_dag and ti.state == State.REMOVED if should_restore_task: self.log.info("Restoring task '{}' which was previously " "removed from DAG '{}'".format(ti, dag)) Stats.incr("task_restored_to_dag.{}".format(dag.dag_id), 1, 1) ti.state = State.NONE # check for missing tasks for task in six.itervalues(dag.task_dict): if task.start_date > self.execution_date and not self.is_backfill: continue if task.task_id not in task_ids: Stats.incr( "task_instance_created-{}".format(task.__class__.__name__), 1, 1) ti = TaskInstance(task, self.execution_date) session.add(ti) session.commit()
def heartbeat(self) -> None: """Heartbeat sent to trigger new jobs.""" if not self.parallelism: open_slots = len(self.queued_tasks) else: open_slots = self.parallelism - len(self.running) num_running_tasks = len(self.running) num_queued_tasks = len(self.queued_tasks) self.log.debug("%s running task instances", num_running_tasks) self.log.debug("%s in queue", num_queued_tasks) self.log.debug("%s open slots", open_slots) Stats.gauge('executor.open_slots', open_slots) Stats.gauge('executor.queued_tasks', num_queued_tasks) Stats.gauge('executor.running_tasks', num_running_tasks) self.trigger_tasks(open_slots) # Calling child class sync method self.log.debug("Calling the %s sync method", self.__class__) self.sync()
def adopt_or_reset_orphaned_tasks(self, session: Session = None): """ Reset any TaskInstance still in QUEUED or SCHEDULED states that were enqueued by a SchedulerJob that is no longer running. :return: the number of TIs reset :rtype: int """ self.log.info("Resetting orphaned tasks for active dag runs") timeout = conf.getint('scheduler', 'scheduler_health_check_threshold') for attempt in run_with_db_retries(logger=self.log): with attempt: self.log.debug( "Running SchedulerJob.adopt_or_reset_orphaned_tasks with retries. Try %d of %d", attempt.retry_state.attempt_number, MAX_DB_RETRIES, ) self.log.debug("Calling SchedulerJob.adopt_or_reset_orphaned_tasks method") try: num_failed = ( session.query(SchedulerJob) .filter( SchedulerJob.state == State.RUNNING, SchedulerJob.latest_heartbeat < (timezone.utcnow() - timedelta(seconds=timeout)), ) .update({"state": State.FAILED}) ) if num_failed: self.log.info("Marked %d SchedulerJob instances as failed", num_failed) Stats.incr(self.__class__.__name__.lower() + '_end', num_failed) resettable_states = [State.QUEUED, State.RUNNING] query = ( session.query(TI) .filter(TI.state.in_(resettable_states)) # outerjoin is because we didn't use to have queued_by_job # set, so we need to pick up anything pre upgrade. This (and the # "or queued_by_job_id IS NONE") can go as soon as scheduler HA is # released. .outerjoin(TI.queued_by_job) .filter(or_(TI.queued_by_job_id.is_(None), SchedulerJob.state != State.RUNNING)) .join(TI.dag_run) .filter( DagRun.run_type != DagRunType.BACKFILL_JOB, DagRun.state == State.RUNNING, ) .options(load_only(TI.dag_id, TI.task_id, TI.run_id)) ) # Lock these rows, so that another scheduler can't try and adopt these too tis_to_reset_or_adopt = with_row_locks( query, of=TI, session=session, **skip_locked(session=session) ).all() to_reset = self.executor.try_adopt_task_instances(tis_to_reset_or_adopt) reset_tis_message = [] for ti in to_reset: reset_tis_message.append(repr(ti)) ti.state = State.NONE ti.queued_by_job_id = None for ti in set(tis_to_reset_or_adopt) - set(to_reset): ti.queued_by_job_id = self.id Stats.incr('scheduler.orphaned_tasks.cleared', len(to_reset)) Stats.incr('scheduler.orphaned_tasks.adopted', len(tis_to_reset_or_adopt) - len(to_reset)) if to_reset: task_instance_str = '\n\t'.join(reset_tis_message) self.log.info( "Reset the following %s orphaned TaskInstances:\n\t%s", len(to_reset), task_instance_str, ) # Issue SQL/finish "Unit of Work", but let @provide_session # commit (or if passed a session, let caller decide when to commit session.flush() except OperationalError: session.rollback() raise return len(to_reset)
def _execute(self): self.task_runner = get_task_runner(self) def signal_handler(signum, frame): """Setting kill signal handler""" self.log.error("Received SIGTERM. Terminating subprocesses") self.on_kill() raise AirflowException("LocalTaskJob received SIGTERM signal") signal.signal(signal.SIGTERM, signal_handler) if not self.task_instance._check_and_change_state_before_execution( mark_success=self.mark_success, ignore_all_deps=self.ignore_all_deps, ignore_depends_on_past=self.ignore_depends_on_past, ignore_task_deps=self.ignore_task_deps, ignore_ti_state=self.ignore_ti_state, job_id=self.id, pool=self.pool): self.log.info("Task is not able to be run") return try: self.task_runner.start() heartbeat_time_limit = conf.getint( 'scheduler', 'scheduler_zombie_task_threshold') while True: # Monitor the task to see if it's done return_code = self.task_runner.return_code() if return_code is not None: self.log.info("Task exited with return code %s", return_code) return # Periodically heartbeat so that the scheduler doesn't think this # is a zombie last_heartbeat_time = time.time() self.heartbeat() # If it's been too long since we've heartbeat, then it's possible that # the scheduler rescheduled this task, so kill launched processes. time_since_last_heartbeat = time.time() - last_heartbeat_time if time_since_last_heartbeat > heartbeat_time_limit: Stats.incr('local_task_job_prolonged_heartbeat_failure', 1, 1) self.log.error("Heartbeat time limited exceeded!") raise AirflowException( "Time since last heartbeat({:.2f}s) " "exceeded limit ({}s).".format( time_since_last_heartbeat, heartbeat_time_limit)) if time_since_last_heartbeat < self.heartrate: sleep_for = self.heartrate - time_since_last_heartbeat self.log.warning( "Time since last heartbeat(%.2f s) < heartrate(%s s)" ", sleeping for %s s", time_since_last_heartbeat, self.heartrate, sleep_for) time.sleep(sleep_for) finally: self.on_kill()
def update_state(self, session=None): """ Determines the overall state of the DagRun based on the state of its TaskInstances. :return: State """ dag = self.get_dag() tis = self.get_task_instances(session=session) self.log.debug("Updating state for %s considering %s task(s)", self, len(tis)) for ti in list(tis): # skip in db? if ti.state == State.REMOVED: tis.remove(ti) else: ti.task = dag.get_task(ti.task_id) # pre-calculate # db is faster start_dttm = timezone.utcnow() unfinished_tasks = self.get_task_instances(state=State.unfinished(), session=session) none_depends_on_past = all(not t.task.depends_on_past for t in unfinished_tasks) none_task_concurrency = all(t.task.task_concurrency is None for t in unfinished_tasks) # small speed up if unfinished_tasks and none_depends_on_past and none_task_concurrency: # todo: this can actually get pretty slow: one task costs between 0.01-015s no_dependencies_met = True for ut in unfinished_tasks: # We need to flag upstream and check for changes because upstream # failures/re-schedules can result in deadlock false positives old_state = ut.state deps_met = ut.are_dependencies_met(dep_context=DepContext( flag_upstream_failed=True, ignore_in_retry_period=True, ignore_in_reschedule_period=True), session=session) if deps_met or old_state != ut.current_state(session=session): no_dependencies_met = False break duration = (timezone.utcnow() - start_dttm).total_seconds() * 1000 Stats.timing("dagrun.dependency-check.{}".format(self.dag_id), duration) root_ids = [t.task_id for t in dag.roots] roots = [t for t in tis if t.task_id in root_ids] # if all roots finished and at least one failed, the run failed if (not unfinished_tasks and any(r.state in (State.FAILED, State.UPSTREAM_FAILED) for r in roots)): self.log.info('Marking run %s failed', self) self.set_state(State.FAILED) dag.handle_callback(self, success=False, reason='task_failure', session=session) # if all roots succeeded and no unfinished tasks, the run succeeded elif not unfinished_tasks and all( r.state in (State.SUCCESS, State.SKIPPED) for r in roots): self.log.info('Marking run %s successful', self) self.set_state(State.SUCCESS) dag.handle_callback(self, success=True, reason='success', session=session) # if *all tasks* are deadlocked, the run failed elif (unfinished_tasks and none_depends_on_past and none_task_concurrency and no_dependencies_met): self.log.info('Deadlock; marking run %s failed', self) self.set_state(State.FAILED) dag.handle_callback(self, success=False, reason='all_tasks_deadlocked', session=session) # finally, if the roots aren't done, the dag is still running else: self.set_state(State.RUNNING) self._emit_duration_stats_for_finished_state() # todo: determine we want to use with_for_update to make sure to lock the run session.merge(self) session.commit() return self.state
def heartbeat(self): """ This should be periodically called by the manager loop. This method will kick off new processes to process DAG definition files and read the results from the finished processors. :return: a list of SimpleDags that were produced by processors that have finished since the last time this was called :rtype: list[airflow.utils.dag_processing.SimpleDag] """ simple_dags = self.collect_results() # Generate more file paths to process if we processed all the files # already. if len(self._file_path_queue) == 0: self.emit_metrics() self._parsing_start_time = timezone.utcnow() # If the file path is already being processed, or if a file was # processed recently, wait until the next batch file_paths_in_progress = self._processors.keys() now = timezone.utcnow() file_paths_recently_processed = [] for file_path in self._file_paths: last_finish_time = self.get_last_finish_time(file_path) if (last_finish_time is not None and (now - last_finish_time).total_seconds() < self._file_process_interval): file_paths_recently_processed.append(file_path) files_paths_at_run_limit = [ file_path for file_path, stat in self._file_stats.items() if stat.run_count == self._max_runs ] files_paths_to_queue = list( set(self._file_paths) - set(file_paths_in_progress) - set(file_paths_recently_processed) - set(files_paths_at_run_limit)) for file_path, processor in self._processors.items(): self.log.debug( "File path %s is still being processed (started: %s)", processor.file_path, processor.start_time.isoformat()) self.log.debug("Queuing the following files for processing:\n\t%s", "\n\t".join(files_paths_to_queue)) for file_path in files_paths_to_queue: if file_path not in self._file_stats: self._file_stats[file_path] = DagFileStat( 0, 0, None, None, 0) self._file_path_queue.extend(files_paths_to_queue) # Start more processors if we have enough slots and files to process while (self._parallelism - len(self._processors) > 0 and len(self._file_path_queue) > 0): file_path = self._file_path_queue.pop(0) processor = self._processor_factory(file_path) Stats.incr('dag_processing.processes') processor.start() self.log.debug( "Started a process (PID: %s) to generate tasks for %s", processor.pid, file_path) self._processors[file_path] = processor # Update heartbeat count. self._heartbeat_count += 1 return simple_dags
def _run_file_processor( result_channel: MultiprocessingConnection, parent_channel: MultiprocessingConnection, file_path: str, pickle_dags: bool, dag_ids: Optional[List[str]], thread_name: str, callback_requests: List[CallbackRequest], ) -> None: """ Process the given file. :param result_channel: the connection to use for passing back the result :type result_channel: multiprocessing.Connection :param parent_channel: the parent end of the channel to close in the child :type parent_channel: multiprocessing.Connection :param file_path: the file to process :type file_path: str :param pickle_dags: whether to pickle the DAGs found in the file and save them to the DB :type pickle_dags: bool :param dag_ids: if specified, only examine DAG ID's that are in this list :type dag_ids: list[str] :param thread_name: the name to use for the process that is launched :type thread_name: str :param callback_requests: failure callback to execute :type callback_requests: List[airflow.utils.callback_requests.CallbackRequest] :return: the process that was launched :rtype: multiprocessing.Process """ # This helper runs in the newly created process log: logging.Logger = logging.getLogger("airflow.processor") # Since we share all open FDs from the parent, we need to close the parent side of the pipe here in # the child, else it won't get closed properly until we exit. log.info("Closing parent pipe") parent_channel.close() del parent_channel set_context(log, file_path) setproctitle(f"airflow scheduler - DagFileProcessor {file_path}") try: # redirect stdout/stderr to log with redirect_stdout(StreamLogWriter( log, logging.INFO)), redirect_stderr( StreamLogWriter(log, logging.WARN)), Stats.timer() as timer: # Re-configure the ORM engine as there are issues with multiple processes settings.configure_orm() # Change the thread name to differentiate log lines. This is # really a separate process, but changing the name of the # process doesn't work, so changing the thread name instead. threading.current_thread().name = thread_name log.info("Started process (PID=%s) to work on %s", os.getpid(), file_path) dag_file_processor = DagFileProcessor(dag_ids=dag_ids, log=log) result: Tuple[int, int] = dag_file_processor.process_file( file_path=file_path, pickle_dags=pickle_dags, callback_requests=callback_requests, ) result_channel.send(result) log.info("Processing %s took %.3f seconds", file_path, timer.duration) except Exception: # pylint: disable=broad-except # Log exceptions through the logging framework. log.exception("Got an exception! Propagating...") raise finally: # We re-initialized the ORM within this Process above so we need to # tear it down manually here settings.dispose_orm() result_channel.close()
def process_file( self, file_path: str, callback_requests: List[CallbackRequest], pickle_dags: bool = False, session: Session = None, ) -> Tuple[int, int]: """ Process a Python file containing Airflow DAGs. This includes: 1. Execute the file and look for DAG objects in the namespace. 2. Execute any Callbacks if passed to this method. 3. Serialize the DAGs and save it to DB (or update existing record in the DB). 4. Pickle the DAG and save it to the DB (if necessary). 5. Record any errors importing the file into ORM :param file_path: the path to the Python file that should be executed :type file_path: str :param callback_requests: failure callback to execute :type callback_requests: List[airflow.utils.dag_processing.CallbackRequest] :param pickle_dags: whether serialize the DAGs found in the file and save them to the db :type pickle_dags: bool :param session: Sqlalchemy ORM Session :type session: Session :return: number of dags found, count of import errors :rtype: Tuple[int, int] """ self.log.info("Processing file %s for tasks to queue", file_path) try: dagbag = DagBag(file_path, include_examples=False, include_smart_sensor=False) except Exception: # pylint: disable=broad-except self.log.exception("Failed at reloading the DAG file %s", file_path) Stats.incr('dag_file_refresh_error', 1, 1) return 0, 0 if len(dagbag.dags) > 0: self.log.info("DAG(s) %s retrieved from %s", dagbag.dags.keys(), file_path) else: self.log.warning("No viable dags retrieved from %s", file_path) self.update_import_errors(session, dagbag) return 0, len(dagbag.import_errors) self.execute_callbacks(dagbag, callback_requests) # Save individual DAGs in the ORM dagbag.sync_to_db() if pickle_dags: paused_dag_ids = DagModel.get_paused_dag_ids( dag_ids=dagbag.dag_ids) unpaused_dags: List[DAG] = [ dag for dag_id, dag in dagbag.dags.items() if dag_id not in paused_dag_ids ] for dag in unpaused_dags: dag.pickle(session) # Record import errors into the ORM try: self.update_import_errors(session, dagbag) except Exception: # pylint: disable=broad-except self.log.exception("Error logging import errors!") return len(dagbag.dags), len(dagbag.import_errors)
def _execute(self): self.task_runner = get_task_runner(self) def signal_handler(signum, frame): """Setting kill signal handler""" self.log.error("Received SIGTERM. Terminating subprocesses") self.on_kill() self.task_instance.refresh_from_db() if self.task_instance.state not in State.finished: self.task_instance.set_state(State.FAILED) self.task_instance._run_finished_callback( error="task received sigterm") raise AirflowException("LocalTaskJob received SIGTERM signal") signal.signal(signal.SIGTERM, signal_handler) if not self.task_instance.check_and_change_state_before_execution( mark_success=self.mark_success, ignore_all_deps=self.ignore_all_deps, ignore_depends_on_past=self.ignore_depends_on_past, ignore_task_deps=self.ignore_task_deps, ignore_ti_state=self.ignore_ti_state, job_id=self.id, pool=self.pool, ): self.log.info("Task is not able to be run") return try: self.task_runner.start() heartbeat_time_limit = conf.getint( 'scheduler', 'scheduler_zombie_task_threshold') # task callback invocation happens either here or in # self.heartbeat() instead of taskinstance._run_raw_task to # avoid race conditions # # When self.terminating is set to True by heartbeat_callback, this # loop should not be restarted. Otherwise self.handle_task_exit # will be invoked and we will end up with duplicated callbacks while not self.terminating: # Monitor the task to see if it's done. Wait in a syscall # (`os.wait`) for as long as possible so we notice the # subprocess finishing as quick as we can max_wait_time = max( 0, # Make sure this value is never negative, min( (heartbeat_time_limit - (timezone.utcnow() - self.latest_heartbeat).total_seconds() * 0.75), self.heartrate, ), ) return_code = self.task_runner.return_code( timeout=max_wait_time) if return_code is not None: self.handle_task_exit(return_code) return self.heartbeat() # If it's been too long since we've heartbeat, then it's possible that # the scheduler rescheduled this task, so kill launched processes. # This can only really happen if the worker can't read the DB for a long time time_since_last_heartbeat = ( timezone.utcnow() - self.latest_heartbeat).total_seconds() if time_since_last_heartbeat > heartbeat_time_limit: Stats.incr('local_task_job_prolonged_heartbeat_failure', 1, 1) self.log.error("Heartbeat time limit exceeded!") raise AirflowException( "Time since last heartbeat({:.2f}s) " "exceeded limit ({}s).".format( time_since_last_heartbeat, heartbeat_time_limit)) finally: self.on_kill()
def update_state( self, session: Session = None, execute_callbacks: bool = True ) -> Tuple[List[TI], Optional[callback_requests.DagCallbackRequest]]: """ Determines the overall state of the DagRun based on the state of its TaskInstances. :param session: Sqlalchemy ORM Session :type session: Session :param execute_callbacks: Should dag callbacks (success/failure, SLA etc) be invoked directly (default: true) or recorded as a pending request in the ``callback`` property :type execute_callbacks: bool :return: Tuple containing tis that can be scheduled in the current loop & `callback` that needs to be executed """ # Callback to execute in case of Task Failures callback: Optional[callback_requests.DagCallbackRequest] = None start_dttm = timezone.utcnow() self.last_scheduling_decision = start_dttm with Stats.timer(f"dagrun.dependency-check.{self.dag_id}"): dag = self.get_dag() info = self.task_instance_scheduling_decisions(session) tis = info.tis schedulable_tis = info.schedulable_tis changed_tis = info.changed_tis finished_tasks = info.finished_tasks unfinished_tasks = info.unfinished_tasks none_depends_on_past = all(not t.task.depends_on_past for t in unfinished_tasks) none_task_concurrency = all(t.task.max_active_tis_per_dag is None for t in unfinished_tasks) none_deferred = all(t.state != State.DEFERRED for t in unfinished_tasks) if unfinished_tasks and none_depends_on_past and none_task_concurrency and none_deferred: # small speed up are_runnable_tasks = (schedulable_tis or self._are_premature_tis( unfinished_tasks, finished_tasks, session) or changed_tis) leaf_task_ids = {t.task_id for t in dag.leaves} leaf_tis = [ti for ti in tis if ti.task_id in leaf_task_ids] # if all roots finished and at least one failed, the run failed if not unfinished_tasks and any(leaf_ti.state in State.failed_states for leaf_ti in leaf_tis): self.log.error('Marking run %s failed', self) self.set_state(State.FAILED) if execute_callbacks: dag.handle_callback(self, success=False, reason='task_failure', session=session) elif dag.has_on_failure_callback: callback = callback_requests.DagCallbackRequest( full_filepath=dag.fileloc, dag_id=self.dag_id, execution_date=self.execution_date, is_failure_callback=True, msg='task_failure', ) # if all leaves succeeded and no unfinished tasks, the run succeeded elif not unfinished_tasks and all(leaf_ti.state in State.success_states for leaf_ti in leaf_tis): self.log.info('Marking run %s successful', self) self.set_state(State.SUCCESS) if execute_callbacks: dag.handle_callback(self, success=True, reason='success', session=session) elif dag.has_on_success_callback: callback = callback_requests.DagCallbackRequest( full_filepath=dag.fileloc, dag_id=self.dag_id, execution_date=self.execution_date, is_failure_callback=False, msg='success', ) # if *all tasks* are deadlocked, the run failed elif (unfinished_tasks and none_depends_on_past and none_task_concurrency and none_deferred and not are_runnable_tasks): self.log.error('Deadlock; marking run %s failed', self) self.set_state(State.FAILED) if execute_callbacks: dag.handle_callback(self, success=False, reason='all_tasks_deadlocked', session=session) elif dag.has_on_failure_callback: callback = callback_requests.DagCallbackRequest( full_filepath=dag.fileloc, dag_id=self.dag_id, execution_date=self.execution_date, is_failure_callback=True, msg='all_tasks_deadlocked', ) # finally, if the roots aren't done, the dag is still running else: self.set_state(State.RUNNING) self._emit_true_scheduling_delay_stats_for_finished_state( finished_tasks) self._emit_duration_stats_for_finished_state() session.merge(self) return schedulable_tis, callback
def _log_file_processing_stats(self, known_file_paths): """ Print out stats about how files are getting processed. :param known_file_paths: a list of file paths that may contain Airflow DAG definitions :type known_file_paths: list[unicode] :return: None """ # File Path: Path to the file containing the DAG definition # PID: PID associated with the process that's processing the file. May # be empty. # Runtime: If the process is currently running, how long it's been # running for in seconds. # Last Runtime: If the process ran before, how long did it take to # finish in seconds # Last Run: When the file finished processing in the previous run. headers = ["File Path", "PID", "Runtime", "Last Runtime", "Last Run"] rows = [] for file_path in known_file_paths: last_runtime = self.get_last_runtime(file_path) file_name = os.path.basename(file_path) file_name = os.path.splitext(file_name)[0].replace(os.sep, '.') if last_runtime: Stats.gauge( 'dag_processing.last_runtime.{}'.format(file_name), last_runtime ) processor_pid = self.get_pid(file_path) processor_start_time = self.get_start_time(file_path) runtime = ((timezone.utcnow() - processor_start_time).total_seconds() if processor_start_time else None) last_run = self.get_last_finish_time(file_path) if last_run: seconds_ago = (timezone.utcnow() - last_run).total_seconds() Stats.gauge( 'dag_processing.last_run.seconds_ago.{}'.format(file_name), seconds_ago ) rows.append((file_path, processor_pid, runtime, last_runtime, last_run)) # Sort by longest last runtime. (Can't sort None values in python3) rows = sorted(rows, key=lambda x: x[3] or 0.0) formatted_rows = [] for file_path, pid, runtime, last_runtime, last_run in rows: formatted_rows.append((file_path, pid, "{:.2f}s".format(runtime) if runtime else None, "{:.2f}s".format(last_runtime) if last_runtime else None, last_run.strftime("%Y-%m-%dT%H:%M:%S") if last_run else None)) log_str = ("\n" + "=" * 80 + "\n" + "DAG File Processing Stats\n\n" + tabulate(formatted_rows, headers=headers) + "\n" + "=" * 80) self.log.info(log_str)
def heartbeat_callback(self, session: Session = None) -> None: Stats.incr('scheduler_heartbeat', 1, 1)
def _executable_task_instances_to_queued(self, max_tis: int, session: Session = None) -> List[TI]: """ Finds TIs that are ready for execution with respect to pool limits, dag max_active_tasks, executor state, and priority. :param max_tis: Maximum number of TIs to queue in this loop. :type max_tis: int :return: list[airflow.models.TaskInstance] """ executable_tis: List[TI] = [] # Get the pool settings. We get a lock on the pool rows, treating this as a "critical section" # Throws an exception if lock cannot be obtained, rather than blocking pools = models.Pool.slots_stats(lock_rows=True, session=session) # If the pools are full, there is no point doing anything! # If _somehow_ the pool is overfull, don't let the limit go negative - it breaks SQL pool_slots_free = max(0, sum(pool['open'] for pool in pools.values())) if pool_slots_free == 0: self.log.debug("All pools are full!") return executable_tis max_tis = min(max_tis, pool_slots_free) # Get all task instances associated with scheduled # DagRuns which are not backfilled, in the given states, # and the dag is not paused query = ( session.query(TI) .join(TI.dag_run) .options(eagerload(TI.dag_run)) .filter(DR.run_type != DagRunType.BACKFILL_JOB, DR.state != DagRunState.QUEUED) .join(TI.dag_model) .filter(not_(DM.is_paused)) .filter(TI.state == State.SCHEDULED) .options(selectinload('dag_model')) .order_by(-TI.priority_weight, DR.execution_date) ) starved_pools = [pool_name for pool_name, stats in pools.items() if stats['open'] <= 0] if starved_pools: query = query.filter(not_(TI.pool.in_(starved_pools))) query = query.limit(max_tis) task_instances_to_examine: List[TI] = with_row_locks( query, of=TI, session=session, **skip_locked(session=session), ).all() # TODO[HA]: This was wrong before anyway, as it only looked at a sub-set of dags, not everything. # Stats.gauge('scheduler.tasks.pending', len(task_instances_to_examine)) if len(task_instances_to_examine) == 0: self.log.debug("No tasks to consider for execution.") return executable_tis # Put one task instance on each line task_instance_str = "\n\t".join(repr(x) for x in task_instances_to_examine) self.log.info("%s tasks up for execution:\n\t%s", len(task_instances_to_examine), task_instance_str) pool_to_task_instances: DefaultDict[str, List[models.Pool]] = defaultdict(list) for task_instance in task_instances_to_examine: pool_to_task_instances[task_instance.pool].append(task_instance) # dag_id to # of running tasks and (dag_id, task_id) to # of running tasks. dag_max_active_tasks_map: DefaultDict[str, int] task_concurrency_map: DefaultDict[Tuple[str, str], int] dag_max_active_tasks_map, task_concurrency_map = self.__get_concurrency_maps( states=list(EXECUTION_STATES), session=session ) num_tasks_in_executor = 0 # Number of tasks that cannot be scheduled because of no open slot in pool num_starving_tasks_total = 0 # Go through each pool, and queue up a task for execution if there are # any open slots in the pool. for pool, task_instances in pool_to_task_instances.items(): pool_name = pool if pool not in pools: self.log.warning("Tasks using non-existent pool '%s' will not be scheduled", pool) continue open_slots = pools[pool]["open"] num_ready = len(task_instances) self.log.info( "Figuring out tasks to run in Pool(name=%s) with %s open slots " "and %s task instances ready to be queued", pool, open_slots, num_ready, ) priority_sorted_task_instances = sorted( task_instances, key=lambda ti: (-ti.priority_weight, ti.execution_date) ) num_starving_tasks = 0 for current_index, task_instance in enumerate(priority_sorted_task_instances): if open_slots <= 0: self.log.info("Not scheduling since there are %s open slots in pool %s", open_slots, pool) # Can't schedule any more since there are no more open slots. num_unhandled = len(priority_sorted_task_instances) - current_index num_starving_tasks += num_unhandled num_starving_tasks_total += num_unhandled break # Check to make sure that the task max_active_tasks of the DAG hasn't been # reached. dag_id = task_instance.dag_id current_max_active_tasks_per_dag = dag_max_active_tasks_map[dag_id] max_active_tasks_per_dag_limit = task_instance.dag_model.max_active_tasks self.log.info( "DAG %s has %s/%s running and queued tasks", dag_id, current_max_active_tasks_per_dag, max_active_tasks_per_dag_limit, ) if current_max_active_tasks_per_dag >= max_active_tasks_per_dag_limit: self.log.info( "Not executing %s since the number of tasks running or queued " "from DAG %s is >= to the DAG's max_active_tasks limit of %s", task_instance, dag_id, max_active_tasks_per_dag_limit, ) continue task_concurrency_limit: Optional[int] = None if task_instance.dag_model.has_task_concurrency_limits: # Many dags don't have a task_concurrency, so where we can avoid loading the full # serialized DAG the better. serialized_dag = self.dagbag.get_dag(dag_id, session=session) if serialized_dag.has_task(task_instance.task_id): task_concurrency_limit = serialized_dag.get_task( task_instance.task_id ).max_active_tis_per_dag if task_concurrency_limit is not None: current_task_concurrency = task_concurrency_map[ (task_instance.dag_id, task_instance.task_id) ] if current_task_concurrency >= task_concurrency_limit: self.log.info( "Not executing %s since the task concurrency for" " this task has been reached.", task_instance, ) continue if task_instance.pool_slots > open_slots: self.log.info( "Not executing %s since it requires %s slots " "but there are %s open slots in the pool %s.", task_instance, task_instance.pool_slots, open_slots, pool, ) num_starving_tasks += 1 num_starving_tasks_total += 1 # Though we can execute tasks with lower priority if there's enough room continue executable_tis.append(task_instance) open_slots -= task_instance.pool_slots dag_max_active_tasks_map[dag_id] += 1 task_concurrency_map[(task_instance.dag_id, task_instance.task_id)] += 1 Stats.gauge(f'pool.starving_tasks.{pool_name}', num_starving_tasks) Stats.gauge('scheduler.tasks.starving', num_starving_tasks_total) Stats.gauge('scheduler.tasks.running', num_tasks_in_executor) Stats.gauge('scheduler.tasks.executable', len(executable_tis)) task_instance_str = "\n\t".join(repr(x) for x in executable_tis) self.log.info("Setting the following tasks to queued state:\n\t%s", task_instance_str) if len(executable_tis) > 0: # set TIs to queued state filter_for_tis = TI.filter_for_tis(executable_tis) session.query(TI).filter(filter_for_tis).update( # TODO[ha]: should we use func.now()? How does that work with DB timezone # on mysql when it's not UTC? {TI.state: State.QUEUED, TI.queued_dttm: timezone.utcnow(), TI.queued_by_job_id: self.id}, synchronize_session=False, ) for ti in executable_tis: make_transient(ti) return executable_tis
def _process_executor_events(self, session: Session = None) -> int: """Respond to executor events.""" if not self.processor_agent: raise ValueError("Processor agent is not started.") ti_primary_key_to_try_number_map: Dict[Tuple[str, str, datetime.datetime], int] = {} event_buffer = self.executor.get_event_buffer() tis_with_right_state: List[TaskInstanceKey] = [] # Report execution for ti_key, value in event_buffer.items(): state: str state, _ = value # We create map (dag_id, task_id, execution_date) -> in-memory try_number ti_primary_key_to_try_number_map[ti_key.primary] = ti_key.try_number self.log.info( "Executor reports execution of %s.%s run_id=%s exited with status %s for try_number %s", ti_key.dag_id, ti_key.task_id, ti_key.run_id, state, ti_key.try_number, ) if state in (State.FAILED, State.SUCCESS, State.QUEUED): tis_with_right_state.append(ti_key) # Return if no finished tasks if not tis_with_right_state: return len(event_buffer) # Check state of finished tasks filter_for_tis = TI.filter_for_tis(tis_with_right_state) tis: List[TI] = session.query(TI).filter(filter_for_tis).options(selectinload('dag_model')).all() for ti in tis: try_number = ti_primary_key_to_try_number_map[ti.key.primary] buffer_key = ti.key.with_try_number(try_number) state, info = event_buffer.pop(buffer_key) # TODO: should we fail RUNNING as well, as we do in Backfills? if state == State.QUEUED: ti.external_executor_id = info self.log.info("Setting external_id for %s to %s", ti, info) continue if ti.try_number == buffer_key.try_number and ti.state == State.QUEUED: Stats.incr('scheduler.tasks.killed_externally') msg = ( "Executor reports task instance %s finished (%s) although the " "task says its %s. (Info: %s) Was the task killed externally?" ) self.log.error(msg, ti, state, ti.state, info) # Get task from the Serialized DAG try: dag = self.dagbag.get_dag(ti.dag_id) task = dag.get_task(ti.task_id) except Exception: self.log.exception("Marking task instance %s as %s", ti, state) ti.set_state(state) continue ti.task = task if task.on_retry_callback or task.on_failure_callback: request = TaskCallbackRequest( full_filepath=ti.dag_model.fileloc, simple_task_instance=SimpleTaskInstance(ti), msg=msg % (ti, state, ti.state, info), ) self.processor_agent.send_callback_to_execute(request) else: ti.handle_failure(error=msg % (ti, state, ti.state, info), session=session) return len(event_buffer)
def update_state(self, session=None): """ Determines the overall state of the DagRun based on the state of its TaskInstances. :return: State """ dag = self.get_dag() tis = self.get_task_instances(session=session) self.log.debug("Updating state for %s considering %s task(s)", self, len(tis)) for ti in list(tis): # skip in db? if ti.state == State.REMOVED: tis.remove(ti) else: ti.task = dag.get_task(ti.task_id) # pre-calculate # db is faster start_dttm = timezone.utcnow() unfinished_tasks = self.get_task_instances( state=State.unfinished(), session=session ) none_depends_on_past = all(not t.task.depends_on_past for t in unfinished_tasks) none_task_concurrency = all(t.task.task_concurrency is None for t in unfinished_tasks) # small speed up if unfinished_tasks and none_depends_on_past and none_task_concurrency: # todo: this can actually get pretty slow: one task costs between 0.01-015s no_dependencies_met = True for ut in unfinished_tasks: # We need to flag upstream and check for changes because upstream # failures/re-schedules can result in deadlock false positives old_state = ut.state deps_met = ut.are_dependencies_met( dep_context=DepContext( flag_upstream_failed=True, ignore_in_retry_period=True, ignore_in_reschedule_period=True), session=session) if deps_met or old_state != ut.current_state(session=session): no_dependencies_met = False break duration = (timezone.utcnow() - start_dttm).total_seconds() * 1000 Stats.timing("dagrun.dependency-check.{}".format(self.dag_id), duration) root_ids = [t.task_id for t in dag.roots] roots = [t for t in tis if t.task_id in root_ids] # if all roots finished and at least one failed, the run failed if (not unfinished_tasks and any(r.state in (State.FAILED, State.UPSTREAM_FAILED) for r in roots)): self.log.info('Marking run %s failed', self) self.set_state(State.FAILED) dag.handle_callback(self, success=False, reason='task_failure', session=session) # if all roots succeeded and no unfinished tasks, the run succeeded elif not unfinished_tasks and all(r.state in (State.SUCCESS, State.SKIPPED) for r in roots): self.log.info('Marking run %s successful', self) self.set_state(State.SUCCESS) dag.handle_callback(self, success=True, reason='success', session=session) # if *all tasks* are deadlocked, the run failed elif (unfinished_tasks and none_depends_on_past and none_task_concurrency and no_dependencies_met): self.log.info('Deadlock; marking run %s failed', self) self.set_state(State.FAILED) dag.handle_callback(self, success=False, reason='all_tasks_deadlocked', session=session) # finally, if the roots aren't done, the dag is still running else: self.set_state(State.RUNNING) self._emit_duration_stats_for_finished_state() # todo: determine we want to use with_for_update to make sure to lock the run session.merge(self) session.commit() return self.state
def _do_scheduling(self, session) -> int: """ This function is where the main scheduling decisions take places. It: - Creates any necessary DAG runs by examining the next_dagrun_create_after column of DagModel Since creating Dag Runs is a relatively time consuming process, we select only 10 dags by default (configurable via ``scheduler.max_dagruns_to_create_per_loop`` setting) - putting this higher will mean one scheduler could spend a chunk of time creating dag runs, and not ever get around to scheduling tasks. - Finds the "next n oldest" running DAG Runs to examine for scheduling (n=20 by default, configurable via ``scheduler.max_dagruns_per_loop_to_schedule`` config setting) and tries to progress state (TIs to SCHEDULED, or DagRuns to SUCCESS/FAILURE etc) By "next oldest", we mean hasn't been examined/scheduled in the most time. The reason we don't select all dagruns at once because the rows are selected with row locks, meaning that only one scheduler can "process them", even it is waiting behind other dags. Increasing this limit will allow more throughput for smaller DAGs but will likely slow down throughput for larger (>500 tasks.) DAGs - Then, via a Critical Section (locking the rows of the Pool model) we queue tasks, and then send them to the executor. See docs of _critical_section_execute_task_instances for more. :return: Number of TIs enqueued in this iteration :rtype: int """ # Put a check in place to make sure we don't commit unexpectedly with prohibit_commit(session) as guard: if settings.USE_JOB_SCHEDULE: self._create_dagruns_for_dags(guard, session) self._start_queued_dagruns(session) guard.commit() dag_runs = self._get_next_dagruns_to_examine(State.RUNNING, session) # Bulk fetch the currently active dag runs for the dags we are # examining, rather than making one query per DagRun callback_tuples = [] for dag_run in dag_runs: callback_to_run = self._schedule_dag_run(dag_run, session) callback_tuples.append((dag_run, callback_to_run)) guard.commit() # Send the callbacks after we commit to ensure the context is up to date when it gets run for dag_run, callback_to_run in callback_tuples: self._send_dag_callbacks_to_processor(dag_run, callback_to_run) # Without this, the session has an invalid view of the DB session.expunge_all() # END: schedule TIs try: if self.executor.slots_available <= 0: # We know we can't do anything here, so don't even try! self.log.debug("Executor full, skipping critical section") return 0 timer = Stats.timer('scheduler.critical_section_duration') timer.start() # Find anything TIs in state SCHEDULED, try to QUEUE it (send it to the executor) num_queued_tis = self._critical_section_execute_task_instances(session=session) # Make sure we only sent this metric if we obtained the lock, otherwise we'll skew the # metric, way down timer.stop(send=True) except OperationalError as e: timer.stop(send=False) if is_lock_not_available_error(error=e): self.log.debug("Critical section lock held by another Scheduler") Stats.incr('scheduler.critical_section_busy') session.rollback() return 0 raise guard.commit() return num_queued_tis
def _run_scheduler_loop(self) -> None: """ The actual scheduler loop. The main steps in the loop are: #. Harvest DAG parsing results through DagFileProcessorAgent #. Find and queue executable tasks #. Change task instance state in DB #. Queue tasks in executor #. Heartbeat executor #. Execute queued tasks in executor asynchronously #. Sync on the states of running tasks Following is a graphic representation of these steps. .. image:: ../docs/apache-airflow/img/scheduler_loop.jpg :rtype: None """ if not self.processor_agent: raise ValueError("Processor agent is not started.") is_unit_test: bool = conf.getboolean('core', 'unit_test_mode') timers = EventScheduler() # Check on start up, then every configured interval self.adopt_or_reset_orphaned_tasks() timers.call_regular_interval( conf.getfloat('scheduler', 'orphaned_tasks_check_interval', fallback=300.0), self.adopt_or_reset_orphaned_tasks, ) timers.call_regular_interval( conf.getfloat('scheduler', 'trigger_timeout_check_interval', fallback=15.0), self.check_trigger_timeouts, ) timers.call_regular_interval( conf.getfloat('scheduler', 'pool_metrics_interval', fallback=5.0), self._emit_pool_metrics, ) for loop_count in itertools.count(start=1): with Stats.timer() as timer: if self.using_sqlite: self.processor_agent.run_single_parsing_loop() # For the sqlite case w/ 1 thread, wait until the processor # is finished to avoid concurrent access to the DB. self.log.debug("Waiting for processors to finish since we're using sqlite") self.processor_agent.wait_until_finished() with create_session() as session: num_queued_tis = self._do_scheduling(session) self.executor.heartbeat() session.expunge_all() num_finished_events = self._process_executor_events(session=session) self.processor_agent.heartbeat() # Heartbeat the scheduler periodically self.heartbeat(only_if_necessary=True) # Run any pending timed events next_event = timers.run(blocking=False) self.log.debug("Next timed event is in %f", next_event) self.log.debug("Ran scheduling loop in %.2f seconds", timer.duration) if not is_unit_test and not num_queued_tis and not num_finished_events: # If the scheduler is doing things, don't sleep. This means when there is work to do, the # scheduler will run "as quick as possible", but when it's stopped, it can sleep, dropping CPU # usage when "idle" time.sleep(min(self._processor_poll_interval, next_event)) if loop_count >= self.num_runs > 0: self.log.info( "Exiting scheduler loop as requested number of runs (%d - got to %d) has been reached", self.num_runs, loop_count, ) break if self.processor_agent.done: self.log.info( "Exiting scheduler loop as requested DAG parse count (%d) has been reached after %d" " scheduler loops", self.num_times_parse_dags, loop_count, ) break
def collect_dags( self, dag_folder=None, only_if_updated=True, include_examples=conf.getboolean('core', 'LOAD_EXAMPLES'), safe_mode=conf.getboolean('core', 'DAG_DISCOVERY_SAFE_MODE')): """ Given a file path or a folder, this method looks for python modules, imports them and adds them to the dagbag collection. Note that if a ``.airflowignore`` file is found while processing the directory, it will behave much like a ``.gitignore``, ignoring files that match any of the regex patterns specified in the file. **Note**: The patterns in .airflowignore are treated as un-anchored regexes, not shell-like glob patterns. """ start_dttm = timezone.utcnow() dag_folder = dag_folder or self.dag_folder # Used to store stats around DagBag processing stats = [] FileLoadStat = namedtuple( 'FileLoadStat', "file duration dag_num task_num dags") dag_folder = correct_maybe_zipped(dag_folder) for filepath in list_py_file_paths(dag_folder, safe_mode=safe_mode, include_examples=include_examples): try: ts = timezone.utcnow() found_dags = self.process_file( filepath, only_if_updated=only_if_updated, safe_mode=safe_mode) dag_ids = [dag.dag_id for dag in found_dags] dag_id_names = str(dag_ids) td = timezone.utcnow() - ts td = td.total_seconds() + ( float(td.microseconds) / 1000000) stats.append(FileLoadStat( filepath.replace(dag_folder, ''), td, len(found_dags), sum([len(dag.tasks) for dag in found_dags]), dag_id_names, )) except Exception as e: self.log.exception(e) Stats.gauge( 'collect_dags', (timezone.utcnow() - start_dttm).total_seconds(), 1) Stats.gauge( 'dagbag_size', len(self.dags), 1) Stats.gauge( 'dagbag_import_errors', len(self.import_errors), 1) self.dagbag_stats = sorted( stats, key=lambda x: x.duration, reverse=True) for file_stat in self.dagbag_stats: # file_stat.file similar format: /subdir/dag_name.py filename = file_stat.file.split('/')[-1].replace('.py', '') Stats.timing('dag.loading-duration.{}'. format(filename), file_stat.duration)
def _check_for_removed_or_restored_tasks(self, dag: "DAG", ti_mutation_hook, *, session: Session) -> Set[str]: """ Check for removed tasks/restored tasks. :param dag: DAG object corresponding to the dagrun :param ti_mutation_hook: task_instance_mutation_hook function :param session: Sqlalchemy ORM Session :return: List of task_ids in the dagrun """ tis = self.get_task_instances(session=session) # check for removed or restored tasks task_ids = set() for ti in tis: ti_mutation_hook(ti) task_ids.add(ti.task_id) task = None try: task = dag.get_task(ti.task_id) should_restore_task = ( task is not None) and ti.state == State.REMOVED if should_restore_task: self.log.info( "Restoring task '%s' which was previously removed from DAG '%s'", ti, dag) Stats.incr(f"task_restored_to_dag.{dag.dag_id}", 1, 1) ti.state = State.NONE except AirflowException: if ti.state == State.REMOVED: pass # ti has already been removed, just ignore it elif self.state != State.RUNNING and not dag.partial: self.log.warning( "Failed to get task '%s' for dag '%s'. Marking it as removed.", ti, dag) Stats.incr(f"task_removed_from_dag.{dag.dag_id}", 1, 1) ti.state = State.REMOVED continue if not task.is_mapped: continue task = cast("MappedOperator", task) num_mapped_tis = task.parse_time_mapped_ti_count # Check if the number of mapped literals has changed and we need to mark this TI as removed if num_mapped_tis is not None: if ti.map_index >= num_mapped_tis: self.log.debug( "Removing task '%s' as the map_index is longer than the literal mapping list (%s)", ti, num_mapped_tis, ) ti.state = State.REMOVED elif ti.map_index < 0: self.log.debug( "Removing the unmapped TI '%s' as the mapping can now be performed", ti) ti.state = State.REMOVED else: self.log.info("Restoring mapped task '%s'", ti) Stats.incr(f"task_restored_to_dag.{dag.dag_id}", 1, 1) ti.state = State.NONE else: # What if it is _now_ dynamically mapped, but wasn't before? total_length = task.run_time_mapped_ti_count(self.run_id, session=session) if total_length is None: # Not all upstreams finished, so we can't tell what should be here. Remove everything. if ti.map_index >= 0: self.log.debug( "Removing the unmapped TI '%s' as the mapping can't be resolved yet", ti) ti.state = State.REMOVED continue # Upstreams finished, check there aren't any extras if ti.map_index >= total_length: self.log.debug( "Removing task '%s' as the map_index is longer than the resolved mapping list (%d)", ti, total_length, ) ti.state = State.REMOVED ... return task_ids
def process_file( self, file_path: str, callback_requests: List[CallbackRequest], pickle_dags: bool = False, session: Session = NEW_SESSION, ) -> Tuple[int, int]: """ Process a Python file containing Airflow DAGs. This includes: 1. Execute the file and look for DAG objects in the namespace. 2. Execute any Callbacks if passed to this method. 3. Serialize the DAGs and save it to DB (or update existing record in the DB). 4. Pickle the DAG and save it to the DB (if necessary). 5. Mark any DAGs which are no longer present as inactive 6. Record any errors importing the file into ORM :param file_path: the path to the Python file that should be executed :param callback_requests: failure callback to execute :param pickle_dags: whether serialize the DAGs found in the file and save them to the db :param session: Sqlalchemy ORM Session :return: number of dags found, count of import errors :rtype: Tuple[int, int] """ self.log.info("Processing file %s for tasks to queue", file_path) try: dagbag = DagBag(file_path, include_examples=False) except Exception: self.log.exception("Failed at reloading the DAG file %s", file_path) Stats.incr('dag_file_refresh_error', 1, 1) return 0, 0 if len(dagbag.dags) > 0: self.log.info("DAG(s) %s retrieved from %s", dagbag.dags.keys(), file_path) else: self.log.warning("No viable dags retrieved from %s", file_path) self.update_import_errors(session, dagbag) if callback_requests: # If there were callback requests for this file but there was a # parse error we still need to progress the state of TIs, # otherwise they might be stuck in queued/running for ever! self.execute_callbacks_without_dag(callback_requests, session) return 0, len(dagbag.import_errors) self.execute_callbacks(dagbag, callback_requests, session) session.commit() # Save individual DAGs in the ORM dagbag.sync_to_db(session) session.commit() if pickle_dags: paused_dag_ids = DagModel.get_paused_dag_ids( dag_ids=dagbag.dag_ids) unpaused_dags: List[DAG] = [ dag for dag_id, dag in dagbag.dags.items() if dag_id not in paused_dag_ids ] for dag in unpaused_dags: dag.pickle(session) # Record import errors into the ORM try: self.update_import_errors(session, dagbag) except Exception: self.log.exception("Error logging import errors!") # Record DAG warnings in the metadatabase. try: self.update_dag_warnings(session=session, dagbag=dagbag) except Exception: self.log.exception("Error logging DAG warnings.") return len(dagbag.dags), len(dagbag.import_errors)
def _execute(self): self._enable_task_listeners() self.task_runner = get_task_runner(self) def signal_handler(signum, frame): """Setting kill signal handler""" self.log.error("Received SIGTERM. Terminating subprocesses") self.task_runner.terminate() self.handle_task_exit(128 + signum) signal.signal(signal.SIGTERM, signal_handler) if not self.task_instance.check_and_change_state_before_execution( mark_success=self.mark_success, ignore_all_deps=self.ignore_all_deps, ignore_depends_on_past=self.ignore_depends_on_past, ignore_task_deps=self.ignore_task_deps, ignore_ti_state=self.ignore_ti_state, job_id=self.id, pool=self.pool, external_executor_id=self.external_executor_id, ): self.log.info("Task is not able to be run") return try: self.task_runner.start() heartbeat_time_limit = conf.getint( 'scheduler', 'scheduler_zombie_task_threshold') # LocalTaskJob should not run callbacks, which are handled by TaskInstance._run_raw_task # 1, LocalTaskJob does not parse DAG, thus cannot run callbacks # 2, The run_as_user of LocalTaskJob is likely not same as the TaskInstance._run_raw_task. # When run_as_user is specified, the process owner of the LocalTaskJob must be sudoable. # It is not secure to run callbacks with sudoable users. # If _run_raw_task receives SIGKILL, scheduler will mark it as zombie and invoke callbacks # If LocalTaskJob receives SIGTERM, LocalTaskJob passes SIGTERM to _run_raw_task # If the state of task_instance is changed, LocalTaskJob sends SIGTERM to _run_raw_task while not self.terminating: # Monitor the task to see if it's done. Wait in a syscall # (`os.wait`) for as long as possible so we notice the # subprocess finishing as quick as we can max_wait_time = max( 0, # Make sure this value is never negative, min( (heartbeat_time_limit - (timezone.utcnow() - self.latest_heartbeat).total_seconds() * 0.75), self.heartrate, ), ) return_code = self.task_runner.return_code( timeout=max_wait_time) if return_code is not None: self.handle_task_exit(return_code) return self.heartbeat() # If it's been too long since we've heartbeat, then it's possible that # the scheduler rescheduled this task, so kill launched processes. # This can only really happen if the worker can't read the DB for a long time time_since_last_heartbeat = ( timezone.utcnow() - self.latest_heartbeat).total_seconds() if time_since_last_heartbeat > heartbeat_time_limit: Stats.incr('local_task_job_prolonged_heartbeat_failure', 1, 1) self.log.error("Heartbeat time limit exceeded!") raise AirflowException( f"Time since last heartbeat({time_since_last_heartbeat:.2f}s) exceeded limit " f"({heartbeat_time_limit}s).") finally: self.on_kill()
def heartbeat(self, only_if_necessary: bool = False): """ Heartbeats update the job's entry in the database with a timestamp for the latest_heartbeat and allows for the job to be killed externally. This allows at the system level to monitor what is actually active. For instance, an old heartbeat for SchedulerJob would mean something is wrong. This also allows for any job to be killed externally, regardless of who is running it or on which machine it is running. Note that if your heart rate is set to 60 seconds and you call this method after 10 seconds of processing since the last heartbeat, it will sleep 50 seconds to complete the 60 seconds and keep a steady heart rate. If you go over 60 seconds before calling it, it won't sleep at all. :param only_if_necessary: If the heartbeat is not yet due then do nothing (don't update column, don't call ``heartbeat_callback``) :type only_if_necessary: boolean """ seconds_remaining = 0 if self.latest_heartbeat: seconds_remaining = self.heartrate - (timezone.utcnow() - self.latest_heartbeat).total_seconds() if seconds_remaining > 0 and only_if_necessary: return previous_heartbeat = self.latest_heartbeat try: with create_session() as session: # This will cause it to load from the db session.merge(self) previous_heartbeat = self.latest_heartbeat if self.state in State.terminating_states: self.kill() # Figure out how long to sleep for sleep_for = 0 if self.latest_heartbeat: seconds_remaining = ( self.heartrate - (timezone.utcnow() - self.latest_heartbeat).total_seconds() ) sleep_for = max(0, seconds_remaining) sleep(sleep_for) # Update last heartbeat time with create_session() as session: # Make the session aware of this object session.merge(self) self.latest_heartbeat = timezone.utcnow() session.commit() # At this point, the DB has updated. previous_heartbeat = self.latest_heartbeat self.heartbeat_callback(session=session) self.log.debug('[heartbeat]') except OperationalError: Stats.incr(convert_camel_to_snake(self.__class__.__name__) + '_heartbeat_failure', 1, 1) self.log.exception("%s heartbeat got an exception", self.__class__.__name__) # We didn't manage to heartbeat, so make sure that the timestamp isn't updated self.latest_heartbeat = previous_heartbeat
def _execute(self): self.task_runner = get_task_runner(self) # pylint: disable=unused-argument def signal_handler(signum, frame): """Setting kill signal handler""" self.log.error("Received SIGTERM. Terminating subprocesses") self.on_kill() raise AirflowException("LocalTaskJob received SIGTERM signal") # pylint: enable=unused-argument signal.signal(signal.SIGTERM, signal_handler) if not self.task_instance.check_and_change_state_before_execution( mark_success=self.mark_success, ignore_all_deps=self.ignore_all_deps, ignore_depends_on_past=self.ignore_depends_on_past, ignore_task_deps=self.ignore_task_deps, ignore_ti_state=self.ignore_ti_state, job_id=self.id, pool=self.pool, ): self.log.info("Task is not able to be run") return try: self.task_runner.start() heartbeat_time_limit = conf.getint( 'scheduler', 'scheduler_zombie_task_threshold') while True: # Monitor the task to see if it's done. Wait in a syscall # (`os.wait`) for as long as possible so we notice the # subprocess finishing as quick as we can max_wait_time = max( 0, # Make sure this value is never negative, min( (heartbeat_time_limit - (timezone.utcnow() - self.latest_heartbeat).total_seconds() * 0.75), self.heartrate, ), ) return_code = self.task_runner.return_code( timeout=max_wait_time) if return_code is not None: self.log.info("Task exited with return code %s", return_code) return self.heartbeat() # If it's been too long since we've heartbeat, then it's possible that # the scheduler rescheduled this task, so kill launched processes. # This can only really happen if the worker can't read the DB for a long time time_since_last_heartbeat = ( timezone.utcnow() - self.latest_heartbeat).total_seconds() if time_since_last_heartbeat > heartbeat_time_limit: Stats.incr('local_task_job_prolonged_heartbeat_failure', 1, 1) self.log.error("Heartbeat time limit exceeded!") raise AirflowException( "Time since last heartbeat({:.2f}s) " "exceeded limit ({}s).".format( time_since_last_heartbeat, heartbeat_time_limit)) finally: self.on_kill()
def verify_integrity(self, session: Session = None): """ Verifies the DagRun by checking for removed tasks or tasks that are not in the database yet. It will set state to removed or add the task if required. :param session: Sqlalchemy ORM Session :type session: Session """ from airflow.settings import task_instance_mutation_hook dag = self.get_dag() tis = self.get_task_instances(session=session) # check for removed or restored tasks task_ids = set() for ti in tis: task_instance_mutation_hook(ti) task_ids.add(ti.task_id) task = None try: task = dag.get_task(ti.task_id) except AirflowException: if ti.state == State.REMOVED: pass # ti has already been removed, just ignore it elif self.state != State.RUNNING and not dag.partial: self.log.warning( "Failed to get task '%s' for dag '%s'. Marking it as removed.", ti, dag) Stats.incr(f"task_removed_from_dag.{dag.dag_id}", 1, 1) ti.state = State.REMOVED should_restore_task = (task is not None) and ti.state == State.REMOVED if should_restore_task: self.log.info( "Restoring task '%s' which was previously removed from DAG '%s'", ti, dag) Stats.incr(f"task_restored_to_dag.{dag.dag_id}", 1, 1) ti.state = State.NONE session.merge(ti) # check for missing tasks for task in dag.task_dict.values(): if task.start_date > self.execution_date and not self.is_backfill: continue if task.task_id not in task_ids: Stats.incr(f"task_instance_created-{task.task_type}", 1, 1) ti = TI(task, self.execution_date) task_instance_mutation_hook(ti) session.add(ti) try: session.flush() except IntegrityError as err: self.log.info(str(err)) self.log.info('Hit IntegrityError while creating the TIs for ' f'{dag.dag_id} - {self.execution_date}.') self.log.info('Doing session rollback.') # TODO[HA]: We probably need to savepoint this so we can keep the transaction alive. session.rollback()
def verify_integrity(self, session: Session = NEW_SESSION): """ Verifies the DagRun by checking for removed tasks or tasks that are not in the database yet. It will set state to removed or add the task if required. :param session: Sqlalchemy ORM Session """ from airflow.settings import task_instance_mutation_hook dag = self.get_dag() tis = self.get_task_instances(session=session) # check for removed or restored tasks task_ids = set() for ti in tis: task_instance_mutation_hook(ti) task_ids.add(ti.task_id) task = None try: task = dag.get_task(ti.task_id) should_restore_task = ( task is not None) and ti.state == State.REMOVED if should_restore_task: self.log.info( "Restoring task '%s' which was previously removed from DAG '%s'", ti, dag) Stats.incr(f"task_restored_to_dag.{dag.dag_id}", 1, 1) ti.state = State.NONE except AirflowException: if ti.state == State.REMOVED: pass # ti has already been removed, just ignore it elif self.state != State.RUNNING and not dag.partial: self.log.warning( "Failed to get task '%s' for dag '%s'. Marking it as removed.", ti, dag) Stats.incr(f"task_removed_from_dag.{dag.dag_id}", 1, 1) ti.state = State.REMOVED continue if not task.is_mapped: continue task = cast("MappedOperator", task) num_mapped_tis = task.parse_time_mapped_ti_count # Check if the number of mapped literals has changed and we need to mark this TI as removed if num_mapped_tis is not None: if ti.map_index >= num_mapped_tis: self.log.debug( "Removing task '%s' as the map_index is longer than the literal mapping list (%s)", ti, num_mapped_tis, ) ti.state = State.REMOVED elif ti.map_index < 0: self.log.debug( "Removing the unmapped TI '%s' as the mapping can now be performed", ti) ti.state = State.REMOVED else: self.log.info("Restoring mapped task '%s'", ti) Stats.incr(f"task_restored_to_dag.{dag.dag_id}", 1, 1) ti.state = State.NONE else: # What if it is _now_ dynamically mapped, but wasn't before? total_length = task.run_time_mapped_ti_count(self.run_id, session=session) if total_length is None: # Not all upstreams finished, so we can't tell what should be here. Remove everything. if ti.map_index >= 0: self.log.debug( "Removing the unmapped TI '%s' as the mapping can't be resolved yet", ti) ti.state = State.REMOVED continue # Upstreams finished, check there aren't any extras if ti.map_index >= total_length: self.log.debug( "Removing task '%s' as the map_index is longer than the resolved mapping list (%d)", ti, total_length, ) ti.state = State.REMOVED ... def task_filter(task: "Operator") -> bool: return task.task_id not in task_ids and ( self.is_backfill or task.start_date <= self.execution_date and (task.end_date is None or self.execution_date <= task.end_date)) created_counts: Dict[str, int] = defaultdict(int) # Set for the empty default in airflow.settings -- if it's not set this means it has been changed hook_is_noop = getattr(task_instance_mutation_hook, 'is_noop', False) if hook_is_noop: def create_ti_mapping(task: "Operator", indexes: Tuple[int, ...]) -> Generator: created_counts[task.task_type] += 1 for map_index in indexes: yield TI.insert_mapping(self.run_id, task, map_index=map_index) creator = create_ti_mapping else: def create_ti(task: "Operator", indexes: Tuple[int, ...]) -> Generator: for map_index in indexes: ti = TI(task, run_id=self.run_id, map_index=map_index) task_instance_mutation_hook(ti) created_counts[ti.operator] += 1 yield ti creator = create_ti # Create missing tasks -- and expand any MappedOperator that _only_ have literals as input def expand_mapped_literals( task: "Operator") -> Tuple["Operator", Sequence[int]]: if not task.is_mapped: return (task, (-1, )) task = cast("MappedOperator", task) count = task.parse_time_mapped_ti_count or task.run_time_mapped_ti_count( self.run_id, session=session) if not count: return (task, (-1, )) return (task, range(count)) tasks_and_map_idxs = map(expand_mapped_literals, filter(task_filter, dag.task_dict.values())) tasks = itertools.chain.from_iterable( itertools.starmap(creator, tasks_and_map_idxs)) try: if hook_is_noop: session.bulk_insert_mappings(TI, tasks) else: session.bulk_save_objects(tasks) for task_type, count in created_counts.items(): Stats.incr(f"task_instance_created-{task_type}", count) session.flush() except IntegrityError: self.log.info( 'Hit IntegrityError while creating the TIs for %s- %s', dag.dag_id, self.run_id, exc_info=True, ) self.log.info('Doing session rollback.') # TODO[HA]: We probably need to savepoint this so we can keep the transaction alive. session.rollback()
def manage_slas(self, dag: DAG, session: Session = None) -> None: """ Finding all tasks that have SLAs defined, and sending alert emails where needed. New SLA misses are also recorded in the database. We are assuming that the scheduler runs often, so we only check for tasks that should have succeeded in the past hour. """ self.log.info("Running SLA Checks for %s", dag.dag_id) if not any(isinstance(ti.sla, timedelta) for ti in dag.tasks): self.log.info( "Skipping SLA check for %s because no tasks in DAG have SLAs", dag) return qry = (session.query( TI.task_id, func.max(TI.execution_date).label('max_ti')).with_hint( TI, 'USE INDEX (PRIMARY)', dialect_name='mysql').filter(TI.dag_id == dag.dag_id).filter( or_(TI.state == State.SUCCESS, TI.state == State.SKIPPED)).filter( TI.task_id.in_(dag.task_ids)).group_by( TI.task_id).subquery('sq')) max_tis: List[TI] = (session.query(TI).filter( TI.dag_id == dag.dag_id, TI.task_id == qry.c.task_id, TI.execution_date == qry.c.max_ti, ).all()) ts = timezone.utcnow() for ti in max_tis: task = dag.get_task(ti.task_id) if task.sla and not isinstance(task.sla, timedelta): raise TypeError( f"SLA is expected to be timedelta object, got " f"{type(task.sla)} in {task.dag_id}:{task.task_id}") dttm = dag.following_schedule(ti.execution_date) while dttm < timezone.utcnow(): following_schedule = dag.following_schedule(dttm) if following_schedule + task.sla < timezone.utcnow(): session.merge( SlaMiss(task_id=ti.task_id, dag_id=ti.dag_id, execution_date=dttm, timestamp=ts)) dttm = dag.following_schedule(dttm) session.commit() # pylint: disable=singleton-comparison slas: List[SlaMiss] = ( session.query(SlaMiss).filter(SlaMiss.notification_sent == False, SlaMiss.dag_id == dag.dag_id) # noqa .all()) # pylint: enable=singleton-comparison if slas: # pylint: disable=too-many-nested-blocks sla_dates: List[datetime.datetime] = [ sla.execution_date for sla in slas ] fetched_tis: List[TI] = (session.query(TI).filter( TI.state != State.SUCCESS, TI.execution_date.in_(sla_dates), TI.dag_id == dag.dag_id).all()) blocking_tis: List[TI] = [] for ti in fetched_tis: if ti.task_id in dag.task_ids: ti.task = dag.get_task(ti.task_id) blocking_tis.append(ti) else: session.delete(ti) session.commit() task_list = "\n".join(sla.task_id + ' on ' + sla.execution_date.isoformat() for sla in slas) blocking_task_list = "\n".join(ti.task_id + ' on ' + ti.execution_date.isoformat() for ti in blocking_tis) # Track whether email or any alert notification sent # We consider email or the alert callback as notifications email_sent = False notification_sent = False if dag.sla_miss_callback: # Execute the alert callback self.log.info('Calling SLA miss callback') try: dag.sla_miss_callback(dag, task_list, blocking_task_list, slas, blocking_tis) notification_sent = True except Exception: # pylint: disable=broad-except self.log.exception( "Could not call sla_miss_callback for DAG %s", dag.dag_id) email_content = f"""\ Here's a list of tasks that missed their SLAs: <pre><code>{task_list}\n<code></pre> Blocking tasks: <pre><code>{blocking_task_list}<code></pre> Airflow Webserver URL: {conf.get(section='webserver', key='base_url')} """ tasks_missed_sla = [] for sla in slas: try: task = dag.get_task(sla.task_id) except TaskNotFound: # task already deleted from DAG, skip it self.log.warning( "Task %s doesn't exist in DAG anymore, skipping SLA miss notification.", sla.task_id) continue tasks_missed_sla.append(task) emails: Set[str] = set() for task in tasks_missed_sla: if task.email: if isinstance(task.email, str): emails |= set(get_email_address_list(task.email)) elif isinstance(task.email, (list, tuple)): emails |= set(task.email) if emails: try: send_email(emails, f"[airflow] SLA miss on DAG={dag.dag_id}", email_content) email_sent = True notification_sent = True except Exception: # pylint: disable=broad-except Stats.incr('sla_email_notification_failure') self.log.exception( "Could not send SLA Miss email notification for DAG %s", dag.dag_id) # If we sent any notification, update the sla_miss table if notification_sent: for sla in slas: sla.email_sent = email_sent sla.notification_sent = True session.merge(sla) session.commit()
def verify_integrity(self, session: Session = NEW_SESSION): """ Verifies the DagRun by checking for removed tasks or tasks that are not in the database yet. It will set state to removed or add the task if required. :param session: Sqlalchemy ORM Session :type session: Session """ from airflow.settings import task_instance_mutation_hook dag = self.get_dag() tis = self.get_task_instances(session=session) # check for removed or restored tasks task_ids = set() for ti in tis: task_instance_mutation_hook(ti) task_ids.add(ti.task_id) task = None try: task = dag.get_task(ti.task_id) except AirflowException: if ti.state == State.REMOVED: pass # ti has already been removed, just ignore it elif self.state != State.RUNNING and not dag.partial: self.log.warning( "Failed to get task '%s' for dag '%s'. Marking it as removed.", ti, dag) Stats.incr(f"task_removed_from_dag.{dag.dag_id}", 1, 1) ti.state = State.REMOVED should_restore_task = (task is not None) and ti.state == State.REMOVED if should_restore_task: self.log.info( "Restoring task '%s' which was previously removed from DAG '%s'", ti, dag) Stats.incr(f"task_restored_to_dag.{dag.dag_id}", 1, 1) ti.state = State.NONE session.merge(ti) def task_filter(task: "BaseOperator"): return task.task_id not in task_ids and ( self.is_backfill or task.start_date <= self.execution_date) created_counts: Dict[str, int] = defaultdict(int) # Set for the empty default in airflow.settings -- if it's not set this means it has been changed hook_is_noop = getattr(task_instance_mutation_hook, 'is_noop', False) if hook_is_noop: def create_ti_mapping(task: "BaseOperator"): created_counts[task.task_type] += 1 return TI.insert_mapping(self.run_id, task) else: def create_ti(task: "BaseOperator") -> TI: ti = TI(task, run_id=self.run_id) task_instance_mutation_hook(ti) created_counts[ti.operator] += 1 return ti # Create missing tasks tasks = list(filter(task_filter, dag.task_dict.values())) try: if hook_is_noop: session.bulk_insert_mappings(TI, map(create_ti_mapping, tasks)) else: session.bulk_save_objects(map(create_ti, tasks)) for task_type, count in created_counts.items(): Stats.incr(f"task_instance_created-{task_type}", count) session.flush() except IntegrityError as err: self.log.info(str(err)) self.log.info( 'Hit IntegrityError while creating the TIs for %s- %s', dag.dag_id, self.run_id) self.log.info('Doing session rollback.') # TODO[HA]: We probably need to savepoint this so we can keep the transaction alive. session.rollback()
def update_state(self, session=None): """ Determines the overall state of the DagRun based on the state of its TaskInstances. :return: ready_tis: the tis that can be scheduled in the current loop :rtype ready_tis: list[airflow.models.TaskInstance] """ dag = self.get_dag() ready_tis = [] tis = [ ti for ti in self.get_task_instances( session=session, state=State.task_states + (State.SHUTDOWN, )) ] self.log.debug("number of tis tasks for %s: %s task(s)", self, len(tis)) for ti in tis: ti.task = dag.get_task(ti.task_id) start_dttm = timezone.utcnow() unfinished_tasks = [t for t in tis if t.state in State.unfinished()] finished_tasks = [ t for t in tis if t.state in State.finished() + [State.UPSTREAM_FAILED] ] none_depends_on_past = all(not t.task.depends_on_past for t in unfinished_tasks) none_task_concurrency = all(t.task.task_concurrency is None for t in unfinished_tasks) if unfinished_tasks: scheduleable_tasks = [ ut for ut in unfinished_tasks if ut.state in SCHEDULEABLE_STATES ] self.log.debug("number of scheduleable tasks for %s: %s task(s)", self, len(scheduleable_tasks)) ready_tis, changed_tis = self._get_ready_tis( scheduleable_tasks, finished_tasks, session) self.log.debug("ready tis length for %s: %s task(s)", self, len(ready_tis)) if none_depends_on_past and none_task_concurrency: # small speed up are_runnable_tasks = ready_tis or self._are_premature_tis( unfinished_tasks, finished_tasks, session) or changed_tis duration = (timezone.utcnow() - start_dttm) Stats.timing("dagrun.dependency-check.{}".format(self.dag_id), duration) leaf_task_ids = {t.task_id for t in dag.leaves} leaf_tis = [ti for ti in tis if ti.task_id in leaf_task_ids] # if all roots finished and at least one failed, the run failed if not unfinished_tasks and any( leaf_ti.state in {State.FAILED, State.UPSTREAM_FAILED} for leaf_ti in leaf_tis): self.log.error('Marking run %s failed', self) self.set_state(State.FAILED) dag.handle_callback(self, success=False, reason='task_failure', session=session) # if all leafs succeeded and no unfinished tasks, the run succeeded elif not unfinished_tasks and all( leaf_ti.state in {State.SUCCESS, State.SKIPPED} for leaf_ti in leaf_tis): self.log.info('Marking run %s successful', self) self.set_state(State.SUCCESS) dag.handle_callback(self, success=True, reason='success', session=session) # if *all tasks* are deadlocked, the run failed elif (unfinished_tasks and none_depends_on_past and none_task_concurrency and not are_runnable_tasks): self.log.error('Deadlock; marking run %s failed', self) self.set_state(State.FAILED) dag.handle_callback(self, success=False, reason='all_tasks_deadlocked', session=session) # finally, if the roots aren't done, the dag is still running else: self.set_state(State.RUNNING) self._emit_duration_stats_for_finished_state() # todo: determine we want to use with_for_update to make sure to lock the run session.merge(self) session.commit() return ready_tis
def emit_metrics(self): Stats.gauge('triggers.running', len(self.runner.triggers))
def collect_dags(self, dag_folder=None, only_if_updated=True, include_examples=conf.getboolean('core', 'LOAD_EXAMPLES'), safe_mode=conf.getboolean('core', 'DAG_DISCOVERY_SAFE_MODE')): """ Given a file path or a folder, this method looks for python modules, imports them and adds them to the dagbag collection. Note that if a ``.airflowignore`` file is found while processing the directory, it will behave much like a ``.gitignore``, ignoring files that match any of the regex patterns specified in the file. **Note**: The patterns in .airflowignore are treated as un-anchored regexes, not shell-like glob patterns. """ if self.store_serialized_dags: return self.log.info("Filling up the DagBag from %s", dag_folder) start_dttm = timezone.utcnow() dag_folder = dag_folder or self.dag_folder # Used to store stats around DagBag processing stats = [] dag_folder = correct_maybe_zipped(dag_folder) for filepath in list_py_file_paths(dag_folder, safe_mode=safe_mode, include_examples=include_examples): try: file_parse_start_dttm = timezone.utcnow() found_dags = self.process_file(filepath, only_if_updated=only_if_updated, safe_mode=safe_mode) file_parse_end_dttm = timezone.utcnow() stats.append( FileLoadStat( file=filepath.replace(settings.DAGS_FOLDER, ''), duration=file_parse_end_dttm - file_parse_start_dttm, dag_num=len(found_dags), task_num=sum([len(dag.tasks) for dag in found_dags]), dags=str([dag.dag_id for dag in found_dags]), )) except Exception as e: # pylint: disable=broad-except self.log.exception(e) end_dttm = timezone.utcnow() durations = (end_dttm - start_dttm).total_seconds() Stats.gauge('collect_dags', durations, 1) Stats.gauge('dagbag_size', len(self.dags), 1) Stats.gauge('dagbag_import_errors', len(self.import_errors), 1) self.dagbag_stats = sorted(stats, key=lambda x: x.duration, reverse=True) for file_stat in self.dagbag_stats: # file_stat.file similar format: /subdir/dag_name.py # TODO: Remove for Airflow 2.0 filename = file_stat.file.split('/')[-1].replace('.py', '') Stats.timing('dag.loading-duration.{}'.format(filename), file_stat.duration)
def update_state( self, session: Session = None, execute_callbacks: bool = True ) -> Tuple[List[TI], Optional[callback_requests.DagCallbackRequest]]: """ Determines the overall state of the DagRun based on the state of its TaskInstances. :param session: Sqlalchemy ORM Session :type session: Session :param execute_callbacks: Should dag callbacks (success/failure, SLA etc) be invoked directly (default: true) or recorded as a pending request in the ``callback`` property :type execute_callbacks: bool :return: Tuple containing tis that can be scheduled in the current loop & `callback` that needs to be executed """ # Callback to execute in case of Task Failures callback: Optional[callback_requests.DagCallbackRequest] = None start_dttm = timezone.utcnow() self.last_scheduling_decision = start_dttm dag = self.get_dag() ready_tis: List[TI] = [] tis = list( self.get_task_instances(session=session, state=State.task_states + (State.SHUTDOWN, ))) self.log.debug("number of tis tasks for %s: %s task(s)", self, len(tis)) for ti in tis: ti.task = dag.get_task(ti.task_id) unfinished_tasks = [t for t in tis if t.state in State.unfinished()] finished_tasks = [ t for t in tis if t.state in State.finished() + [State.UPSTREAM_FAILED] ] none_depends_on_past = all(not t.task.depends_on_past for t in unfinished_tasks) none_task_concurrency = all(t.task.task_concurrency is None for t in unfinished_tasks) if unfinished_tasks: scheduleable_tasks = [ ut for ut in unfinished_tasks if ut.state in SCHEDULEABLE_STATES ] self.log.debug("number of scheduleable tasks for %s: %s task(s)", self, len(scheduleable_tasks)) ready_tis, changed_tis = self._get_ready_tis( scheduleable_tasks, finished_tasks, session) self.log.debug("ready tis length for %s: %s task(s)", self, len(ready_tis)) if none_depends_on_past and none_task_concurrency: # small speed up are_runnable_tasks = ready_tis or self._are_premature_tis( unfinished_tasks, finished_tasks, session) or changed_tis duration = (timezone.utcnow() - start_dttm) Stats.timing("dagrun.dependency-check.{}".format(self.dag_id), duration) leaf_task_ids = {t.task_id for t in dag.leaves} leaf_tis = [ti for ti in tis if ti.task_id in leaf_task_ids] # if all roots finished and at least one failed, the run failed if not unfinished_tasks and any( leaf_ti.state in {State.FAILED, State.UPSTREAM_FAILED} for leaf_ti in leaf_tis): self.log.error('Marking run %s failed', self) self.set_state(State.FAILED) if execute_callbacks: dag.handle_callback(self, success=False, reason='task_failure', session=session) else: callback = callback_requests.DagCallbackRequest( full_filepath=dag.fileloc, dag_id=self.dag_id, execution_date=self.execution_date, is_failure_callback=True, msg='task_failure') # if all leafs succeeded and no unfinished tasks, the run succeeded elif not unfinished_tasks and all( leaf_ti.state in {State.SUCCESS, State.SKIPPED} for leaf_ti in leaf_tis): self.log.info('Marking run %s successful', self) self.set_state(State.SUCCESS) if execute_callbacks: dag.handle_callback(self, success=True, reason='success', session=session) else: callback = callback_requests.DagCallbackRequest( full_filepath=dag.fileloc, dag_id=self.dag_id, execution_date=self.execution_date, is_failure_callback=False, msg='success') # if *all tasks* are deadlocked, the run failed elif (unfinished_tasks and none_depends_on_past and none_task_concurrency and not are_runnable_tasks): self.log.error('Deadlock; marking run %s failed', self) self.set_state(State.FAILED) if execute_callbacks: dag.handle_callback(self, success=False, reason='all_tasks_deadlocked', session=session) else: callback = callback_requests.DagCallbackRequest( full_filepath=dag.fileloc, dag_id=self.dag_id, execution_date=self.execution_date, is_failure_callback=True, msg='all_tasks_deadlocked') # finally, if the roots aren't done, the dag is still running else: self.set_state(State.RUNNING) self._emit_duration_stats_for_finished_state() session.merge(self) return ready_tis, callback
def _log_file_processing_stats(self, known_file_paths): """ Print out stats about how files are getting processed. :param known_file_paths: a list of file paths that may contain Airflow DAG definitions :type known_file_paths: list[unicode] :return: None """ # File Path: Path to the file containing the DAG definition # PID: PID associated with the process that's processing the file. May # be empty. # Runtime: If the process is currently running, how long it's been # running for in seconds. # Last Runtime: If the process ran before, how long did it take to # finish in seconds # Last Run: When the file finished processing in the previous run. headers = [ "File Path", "PID", "Runtime", "# DAGs", "# Errors", "Last Runtime", "Last Run" ] rows = [] now = timezone.utcnow() for file_path in known_file_paths: last_runtime = self.get_last_runtime(file_path) num_dags = self.get_last_dag_count(file_path) num_errors = self.get_last_error_count(file_path) file_name = os.path.basename(file_path) file_name = os.path.splitext(file_name)[0].replace(os.sep, '.') processor_pid = self.get_pid(file_path) processor_start_time = self.get_start_time(file_path) runtime = ((now - processor_start_time).total_seconds() if processor_start_time else None) last_run = self.get_last_finish_time(file_path) if last_run: seconds_ago = (now - last_run).total_seconds() Stats.gauge( 'dag_processing.last_run.seconds_ago.{}'.format(file_name), seconds_ago) if runtime: Stats.timing( 'dag_processing.last_duration.{}'.format(file_name), runtime) # TODO: Remove before Airflow 2.0 Stats.timing( 'dag_processing.last_runtime.{}'.format(file_name), runtime) rows.append((file_path, processor_pid, runtime, num_dags, num_errors, last_runtime, last_run)) # Sort by longest last runtime. (Can't sort None values in python3) rows = sorted(rows, key=lambda x: x[3] or 0.0) formatted_rows = [] for file_path, pid, runtime, num_dags, num_errors, last_runtime, last_run in rows: formatted_rows.append( (file_path, pid, "{:.2f}s".format(runtime) if runtime else None, num_dags, num_errors, "{:.2f}s".format(last_runtime) if last_runtime else None, last_run.strftime("%Y-%m-%dT%H:%M:%S") if last_run else None)) log_str = ("\n" + "=" * 80 + "\n" + "DAG File Processing Stats\n\n" + tabulate(formatted_rows, headers=headers) + "\n" + "=" * 80) self.log.info(log_str)
def _emit_pool_metrics(self, session: Session = None) -> None: pools = models.Pool.slots_stats(session=session) for pool_name, slot_stats in pools.items(): Stats.gauge(f'pool.open_slots.{pool_name}', slot_stats["open"]) Stats.gauge(f'pool.queued_slots.{pool_name}', slot_stats[State.QUEUED]) # type: ignore Stats.gauge(f'pool.running_slots.{pool_name}', slot_stats[State.RUNNING]) # type: ignore