예제 #1
0
    def _run_processor_manager(
        dag_directory: str,
        max_runs: int,
        processor_timeout: timedelta,
        signal_conn: MultiprocessingConnection,
        dag_ids: Optional[List[str]],
        pickle_dags: bool,
        async_mode: bool,
    ) -> None:

        # Make this process start as a new process group - that makes it easy
        # to kill all sub-process of this at the OS-level, rather than having
        # to iterate the child processes
        set_new_process_group()

        setproctitle("airflow scheduler -- DagFileProcessorManager")
        # Reload configurations and settings to avoid collision with parent process.
        # Because this process may need custom configurations that cannot be shared,
        # e.g. RotatingFileHandler. And it can cause connection corruption if we
        # do not recreate the SQLA connection pool.
        os.environ['CONFIG_PROCESSOR_MANAGER_LOGGER'] = 'True'
        os.environ['AIRFLOW__LOGGING__COLORED_CONSOLE_LOG'] = 'False'
        # Replicating the behavior of how logging module was loaded
        # in logging_config.py

        # TODO: This reloading should be removed when we fix our logging behaviour
        # In case of "spawn" method of starting processes for multiprocessing, reinitializing of the
        # SQLAlchemy engine causes extremely unexpected behaviour of messing with objects already loaded
        # in a parent process (likely via resources shared in memory by the ORM libraries).
        # This caused flaky tests in our CI for many months and has been discovered while
        # iterating on https://github.com/apache/airflow/pull/19860
        # The issue that describes the problem and possible remediation is
        # at https://github.com/apache/airflow/issues/19934

        importlib.reload(
            import_module(
                airflow.settings.LOGGING_CLASS_PATH.rsplit(
                    '.', 1)[0]))  # type: ignore
        importlib.reload(airflow.settings)
        airflow.settings.initialize()
        del os.environ['CONFIG_PROCESSOR_MANAGER_LOGGER']
        processor_manager = DagFileProcessorManager(
            dag_directory=dag_directory,
            max_runs=max_runs,
            processor_timeout=processor_timeout,
            dag_ids=dag_ids,
            pickle_dags=pickle_dags,
            signal_conn=signal_conn,
            async_mode=async_mode,
        )

        processor_manager.start()
예제 #2
0
파일: manager.py 프로젝트: dskoda1/airflow
    def start(self):
        """
        Use multiple processes to parse and generate tasks for the
        DAGs in parallel. By processing them in separate processes,
        we can get parallelism and isolation from potentially harmful
        user code.
        """
        self.register_exit_signals()

        set_new_process_group()

        self.log.info("Processing files using up to %s processes at a time ",
                      self._parallelism)
        self.log.info("Process each file at most once every %s seconds",
                      self._file_process_interval)
        self.log.info("Checking for new files in %s every %s seconds",
                      self._dag_directory, self.dag_dir_list_interval)

        return self._run_parsing_loop()
예제 #3
0
    def _start_by_fork(self):
        pid = os.fork()
        if pid:
            self.log.info("Started process %d to run task", pid)
            return psutil.Process(pid)
        else:
            # Start a new process group
            set_new_process_group()
            import signal

            signal.signal(signal.SIGINT, signal.SIG_DFL)
            signal.signal(signal.SIGTERM, signal.SIG_DFL)

            from airflow import settings
            from airflow.cli.cli_parser import get_parser
            from airflow.sentry import Sentry
            from airflow.utils.cli import get_dag

            # Force a new SQLAlchemy session. We can't share open DB handles
            # between process. The cli code will re-create this as part of its
            # normal startup
            settings.engine.pool.dispose()
            settings.engine.dispose()

            parser = get_parser()
            # [1:] - remove "airflow" from the start of the command
            args = parser.parse_args(self._command[1:])

            # We prefer the job_id passed on the command-line because at this time, the
            # task instance may not have been updated.
            job_id = getattr(args, "job_id", self._task_instance.job_id)
            self.log.info('Running: %s', self._command)
            self.log.info('Job %s: Subtask %s', job_id, self._task_instance.task_id)

            proc_title = "airflow task runner: {0.dag_id} {0.task_id} {0.execution_date_or_run_id}"
            if job_id is not None:
                proc_title += " {0.job_id}"
            setproctitle(proc_title.format(args))

            return_code = 0
            try:
                # parse dag file since `airflow tasks run --local` does not parse dag file
                dag = get_dag(args.subdir, args.dag_id)
                args.func(args, dag=dag)
                return_code = 0
            except Exception as exc:
                return_code = 1

                self.log.error(
                    "Failed to execute job %s for task %s (%s; %r)",
                    job_id,
                    self._task_instance.task_id,
                    exc,
                    os.getpid(),
                )
            except SystemExit as sys_ex:
                # Someone called sys.exit() in the fork - mistakenly. You should not run sys.exit() in
                # the fork because you can mistakenly execute atexit that were set by the parent process
                # before fork happened
                return_code = sys_ex.code
            except BaseException:
                # while we want to handle Also Base exceptions here - we do not want to log them (this
                # is the default behaviour anyway. Setting the return code here to 2 to indicate that
                # this had happened.
                return_code = 2
            finally:
                try:
                    # Explicitly flush any pending exception to Sentry and logging if enabled
                    Sentry.flush()
                    logging.shutdown()
                except BaseException:
                    # also make sure to silently ignore ALL POSSIBLE exceptions thrown in the flush/shutdown,
                    # otherwise os._exit() might never be called. We could have used `except:` but
                    # except BaseException is more explicit (and linters do not comply).
                    pass
            # We run os._exit() making sure it is not run within the `finally` clause.
            # We cannot run os._exit() in finally clause, because during finally clause processing, the
            # Exception handled is held in memory as well as stack trace and possibly some objects that
            # might need to be finalized. Running os._exit() inside the `finally` clause might cause effects
            # similar to https://github.com/apache/airflow/issues/22404. There Temporary file has not been
            # deleted at os._exit()
            os._exit(return_code)