## Licensed to the Apache Software Foundation (ASF) under one# or more contributor license agreements. See the NOTICE file# distributed with this work for additional information# regarding copyright ownership. The ASF licenses this file# to you under the Apache License, Version 2.0 (the# "License"); you may not use this file except in compliance# with the License. You may obtain a copy of the License at## http://www.apache.org/licenses/LICENSE-2.0## Unless required by applicable law or agreed to in writing,# software distributed under the License is distributed on an# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY# KIND, either express or implied. See the License for the# specific language governing permissions and limitations# under the License.importhashlibimportimportlibimportimportlib.machineryimportimportlib.utilimportosimportsysimporttextwrapimporttracebackimportwarningsimportzipfilefromdatetimeimportdatetime,timedeltafromtypingimportTYPE_CHECKING,Dict,List,NamedTuple,Optional,UnionfromcroniterimportCroniterBadCronError,CroniterBadDateError,CroniterNotAlphaError,croniterfromsqlalchemy.excimportOperationalErrorfromsqlalchemy.ormimportSessionfromtabulateimporttabulatefromairflowimportsettingsfromairflow.configurationimportconffromairflow.exceptionsimport(AirflowClusterPolicyViolation,AirflowDagCycleException,AirflowDagDuplicatedIdException,SerializedDagNotFound,)fromairflow.statsimportStatsfromairflow.utilsimporttimezonefromairflow.utils.dag_cycle_testerimporttest_cyclefromairflow.utils.fileimportcorrect_maybe_zipped,list_py_file_paths,might_contain_dagfromairflow.utils.log.logging_mixinimportLoggingMixinfromairflow.utils.retriesimportMAX_DB_RETRIES,run_with_db_retriesfromairflow.utils.sessionimportprovide_sessionfromairflow.utils.timeoutimporttimeoutifTYPE_CHECKING:importpathlib
[docs]classFileLoadStat(NamedTuple):"""Information about single file"""
[docs]classDagBag(LoggingMixin):""" A dagbag is a collection of dags, parsed out of a folder tree and has high level configuration settings, like what database to use as a backend and what executor to use to fire off tasks. This makes it easier to run distinct environments for say production and development, tests, or for different teams or security profiles. What would have been system level settings are now dagbag level so that one system can run multiple, independent settings sets. :param dag_folder: the folder to scan to find DAGs :type dag_folder: unicode :param include_examples: whether to include the examples that ship with airflow or not :type include_examples: bool :param include_smart_sensor: whether to include the smart sensor native DAGs that create the smart sensor operators for whole cluster :type include_smart_sensor: bool :param read_dags_from_db: Read DAGs from DB if ``True`` is passed. If ``False`` DAGs are read from python files. :type read_dags_from_db: bool :param load_op_links: Should the extra operator link be loaded via plugins when de-serializing the DAG? This flag is set to False in Scheduler so that Extra Operator links are not loaded to not run User code in Scheduler. :type load_op_links: bool """
def__init__(self,dag_folder:Union[str,"pathlib.Path",None]=None,include_examples:bool=conf.getboolean('core','LOAD_EXAMPLES'),include_smart_sensor:bool=conf.getboolean('smart_sensor','USE_SMART_SENSOR'),safe_mode:bool=conf.getboolean('core','DAG_DISCOVERY_SAFE_MODE'),read_dags_from_db:bool=False,store_serialized_dags:Optional[bool]=None,load_op_links:bool=True,):# Avoid circular importfromairflow.models.dagimportDAGsuper().__init__()ifstore_serialized_dags:warnings.warn("The store_serialized_dags parameter has been deprecated. ""You should pass the read_dags_from_db parameter.",DeprecationWarning,stacklevel=2,)read_dags_from_db=store_serialized_dagsdag_folder=dag_folderorsettings.DAGS_FOLDERself.dag_folder=dag_folderself.dags:Dict[str,DAG]={}# the file's last modified timestamp when we last read itself.file_last_changed:Dict[str,datetime]={}self.import_errors:Dict[str,str]={}self.has_logged=Falseself.read_dags_from_db=read_dags_from_db# Only used by read_dags_from_db=Trueself.dags_last_fetched:Dict[str,datetime]={}# Only used by SchedulerJob to compare the dag_hash to identify change in DAGsself.dags_hash:Dict[str,str]={}self.dagbag_import_error_tracebacks=conf.getboolean('core','dagbag_import_error_tracebacks')self.dagbag_import_error_traceback_depth=conf.getint('core','dagbag_import_error_traceback_depth')self.collect_dags(dag_folder=dag_folder,include_examples=include_examples,include_smart_sensor=include_smart_sensor,safe_mode=safe_mode,)# Should the extra operator link be loaded via plugins?# This flag is set to False in Scheduler so that Extra Operator links are not loadedself.load_op_links=load_op_links
[docs]defsize(self)->int:""":return: the amount of dags contained in this dagbag"""returnlen(self.dags)
@property
[docs]defstore_serialized_dags(self)->bool:"""Whether or not to read dags from DB"""warnings.warn("The store_serialized_dags property has been deprecated. Use read_dags_from_db instead.",DeprecationWarning,stacklevel=2,)returnself.read_dags_from_db
@property
[docs]defdag_ids(self)->List[str]:""" :return: a list of DAG IDs in this bag :rtype: List[unicode] """returnlist(self.dags.keys())
@provide_session
[docs]defget_dag(self,dag_id,session:Session=None):""" Gets the DAG out of the dictionary, and refreshes it if expired :param dag_id: DAG Id :type dag_id: str """# Avoid circular importfromairflow.models.dagimportDagModelifself.read_dags_from_db:# Import here so that serialized dag is only imported when serialization is enabledfromairflow.models.serialized_dagimportSerializedDagModelifdag_idnotinself.dags:# Load from DB if not (yet) in the bagself._add_dag_from_db(dag_id=dag_id,session=session)returnself.dags.get(dag_id)# If DAG is in the DagBag, check the following# 1. if time has come to check if DAG is updated (controlled by min_serialized_dag_fetch_secs)# 2. check the last_updated column in SerializedDag table to see if Serialized DAG is updated# 3. if (2) is yes, fetch the Serialized DAG.# 4. if (2) returns None (i.e. Serialized DAG is deleted), remove dag from dagbag# if it exists and return None.min_serialized_dag_fetch_secs=timedelta(seconds=settings.MIN_SERIALIZED_DAG_FETCH_INTERVAL)if(dag_idinself.dags_last_fetchedandtimezone.utcnow()>self.dags_last_fetched[dag_id]+min_serialized_dag_fetch_secs):sd_last_updated_datetime=SerializedDagModel.get_last_updated_datetime(dag_id=dag_id,session=session,)ifnotsd_last_updated_datetime:self.log.warning("Serialized DAG %s no longer exists",dag_id)delself.dags[dag_id]delself.dags_last_fetched[dag_id]delself.dags_hash[dag_id]returnNoneifsd_last_updated_datetime>self.dags_last_fetched[dag_id]:self._add_dag_from_db(dag_id=dag_id,session=session)returnself.dags.get(dag_id)# If asking for a known subdag, we want to refresh the parentdag=Noneroot_dag_id=dag_idifdag_idinself.dags:dag=self.dags[dag_id]ifdag.is_subdag:root_dag_id=dag.parent_dag.dag_id# type: ignore# If DAG Model is absent, we can't check last_expired property. Is the DAG not yet synchronized?orm_dag=DagModel.get_current(root_dag_id,session=session)ifnotorm_dag:returnself.dags.get(dag_id)# If the dag corresponding to root_dag_id is absent or expiredis_missing=root_dag_idnotinself.dagsis_expired=orm_dag.last_expiredanddaganddag.last_loaded<orm_dag.last_expiredifis_expired:# Remove associated dags so we can re-add them.self.dags={key:dagforkey,daginself.dags.items()ifroot_dag_id!=keyandnot(dag.is_subdagandroot_dag_id==dag.parent_dag.dag_id)}ifis_missingoris_expired:# Reprocess source file.found_dags=self.process_file(filepath=correct_maybe_zipped(orm_dag.fileloc),only_if_updated=False)# If the source file no longer exports `dag_id`, delete it from self.dagsiffound_dagsanddag_idin[found_dag.dag_idforfound_daginfound_dags]:returnself.dags[dag_id]elifdag_idinself.dags:delself.dags[dag_id]returnself.dags.get(dag_id)
[docs]def_add_dag_from_db(self,dag_id:str,session:Session):"""Add DAG to DagBag from DB"""fromairflow.models.serialized_dagimportSerializedDagModelrow=SerializedDagModel.get(dag_id,session)ifnotrow:raiseSerializedDagNotFound(f"DAG '{dag_id}' not found in serialized_dag table")row.load_op_links=self.load_op_linksdag=row.dagforsubdagindag.subdags:self.dags[subdag.dag_id]=subdagself.dags[dag.dag_id]=dagself.dags_last_fetched[dag.dag_id]=timezone.utcnow()self.dags_hash[dag.dag_id]=row.dag_hash
[docs]defprocess_file(self,filepath,only_if_updated=True,safe_mode=True):""" Given a path to a python module or zip file, this method imports the module and look for dag objects within it. """# if the source file no longer exists in the DB or in the filesystem,# return an empty list# todo: raise exception?iffilepathisNoneornotos.path.isfile(filepath):return[]try:# This failed before in what may have been a git sync# race conditionfile_last_changed_on_disk=datetime.fromtimestamp(os.path.getmtime(filepath))if(only_if_updatedandfilepathinself.file_last_changedandfile_last_changed_on_disk==self.file_last_changed[filepath]):return[]exceptExceptionase:self.log.exception(e)return[]ifnotzipfile.is_zipfile(filepath):mods=self._load_modules_from_file(filepath,safe_mode)else:mods=self._load_modules_from_zip(filepath,safe_mode)found_dags=self._process_modules(filepath,mods,file_last_changed_on_disk)self.file_last_changed[filepath]=file_last_changed_on_diskreturnfound_dags
[docs]def_load_modules_from_file(self,filepath,safe_mode):ifnotmight_contain_dag(filepath,safe_mode):# Don't want to spam user with skip messagesifnotself.has_logged:self.has_logged=Trueself.log.info("File %s assumed to contain no DAGs. Skipping.",filepath)return[]self.log.debug("Importing %s",filepath)org_mod_name,_=os.path.splitext(os.path.split(filepath)[-1])path_hash=hashlib.sha1(filepath.encode('utf-8')).hexdigest()mod_name=f'unusual_prefix_{path_hash}_{org_mod_name}'ifmod_nameinsys.modules:delsys.modules[mod_name]timeout_msg=f"DagBag import timeout for {filepath} after {self.DAGBAG_IMPORT_TIMEOUT}s"withtimeout(self.DAGBAG_IMPORT_TIMEOUT,error_message=timeout_msg):try:loader=importlib.machinery.SourceFileLoader(mod_name,filepath)spec=importlib.util.spec_from_loader(mod_name,loader)new_module=importlib.util.module_from_spec(spec)sys.modules[spec.name]=new_moduleloader.exec_module(new_module)return[new_module]exceptExceptionase:self.log.exception("Failed to import: %s",filepath)ifself.dagbag_import_error_tracebacks:self.import_errors[filepath]=traceback.format_exc(limit=-self.dagbag_import_error_traceback_depth)else:self.import_errors[filepath]=str(e)return[]
[docs]def_load_modules_from_zip(self,filepath,safe_mode):mods=[]withzipfile.ZipFile(filepath)ascurrent_zip_file:forzip_infoincurrent_zip_file.infolist():head,_=os.path.split(zip_info.filename)mod_name,ext=os.path.splitext(zip_info.filename)ifextnotin[".py",".pyc"]:continueifhead:continueifmod_name=='__init__':self.log.warning("Found __init__.%s at root of %s",ext,filepath)self.log.debug("Reading %s from %s",zip_info.filename,filepath)ifnotmight_contain_dag(zip_info.filename,safe_mode,current_zip_file):# todo: create ignore list# Don't want to spam user with skip messagesifnotself.has_logged:self.has_logged=Trueself.log.info("File %s:%s assumed to contain no DAGs. Skipping.",filepath,zip_info.filename)continueifmod_nameinsys.modules:delsys.modules[mod_name]try:sys.path.insert(0,filepath)current_module=importlib.import_module(mod_name)mods.append(current_module)exceptExceptionase:self.log.exception("Failed to import: %s",filepath)ifself.dagbag_import_error_tracebacks:self.import_errors[filepath]=traceback.format_exc(limit=-self.dagbag_import_error_traceback_depth)else:self.import_errors[filepath]=str(e)returnmods
[docs]def_process_modules(self,filepath,mods,file_last_changed_on_disk):fromairflow.models.dagimportDAG# Avoid circular importis_zipfile=zipfile.is_zipfile(filepath)top_level_dags=[oforminmodsforoinlist(m.__dict__.values())ifisinstance(o,DAG)]found_dags=[]fordagintop_level_dags:ifnotdag.full_filepath:dag.full_filepath=filepathifdag.fileloc!=filepathandnotis_zipfile:dag.fileloc=filepathtry:dag.is_subdag=Falseifisinstance(dag.normalized_schedule_interval,str):croniter(dag.normalized_schedule_interval)self.bag_dag(dag=dag,root_dag=dag)found_dags.append(dag)found_dags+=dag.subdagsexcept(CroniterBadCronError,CroniterBadDateError,CroniterNotAlphaError)ascron_e:self.log.exception("Failed to bag_dag: %s",dag.full_filepath)self.import_errors[dag.full_filepath]=f"Invalid Cron expression: {cron_e}"self.file_last_changed[dag.full_filepath]=file_last_changed_on_diskexcept(AirflowDagCycleException,AirflowDagDuplicatedIdException,AirflowClusterPolicyViolation,)asexception:self.log.exception("Failed to bag_dag: %s",dag.full_filepath)self.import_errors[dag.full_filepath]=str(exception)self.file_last_changed[dag.full_filepath]=file_last_changed_on_diskreturnfound_dags
[docs]defbag_dag(self,dag,root_dag):""" Adds the DAG into the bag, recurses into sub dags. :raises: AirflowDagCycleException if a cycle is detected in this dag or its subdags. :raises: AirflowDagDuplicatedIdException if this dag or its subdags already exists in the bag. """self._bag_dag(dag=dag,root_dag=root_dag,recursive=True)
[docs]def_bag_dag(self,*,dag,root_dag,recursive):"""Actual implementation of bagging a dag. The only purpose of this is to avoid exposing ``recursive`` in ``bag_dag()``, intended to only be used by the ``_bag_dag()`` implementation. """test_cycle(dag)# throws if a task cycle is founddag.resolve_template_files()dag.last_loaded=timezone.utcnow()# Check policiessettings.dag_policy(dag)fortaskindag.tasks:settings.task_policy(task)subdags=dag.subdagstry:# DAG.subdags automatically performs DFS search, so we don't recurse# into further _bag_dag() calls.ifrecursive:forsubdaginsubdags:subdag.full_filepath=dag.full_filepathsubdag.parent_dag=dagsubdag.is_subdag=Trueself._bag_dag(dag=subdag,root_dag=root_dag,recursive=False)prev_dag=self.dags.get(dag.dag_id)ifprev_dagandprev_dag.full_filepath!=dag.full_filepath:raiseAirflowDagDuplicatedIdException(dag_id=dag.dag_id,incoming=dag.full_filepath,existing=self.dags[dag.dag_id].full_filepath,)self.dags[dag.dag_id]=dagself.log.debug('Loaded DAG %s',dag)except(AirflowDagCycleException,AirflowDagDuplicatedIdException):# There was an error in bagging the dag. Remove it from the list of dagsself.log.exception('Exception bagging dag: %s',dag.dag_id)# Only necessary at the root level since DAG.subdags automatically# performs DFS to search through all subdagsifrecursive:forsubdaginsubdags:ifsubdag.dag_idinself.dags:delself.dags[subdag.dag_id]raise
[docs]defcollect_dags(self,dag_folder:Union[str,"pathlib.Path",None]=None,only_if_updated:bool=True,include_examples:bool=conf.getboolean('core','LOAD_EXAMPLES'),include_smart_sensor:bool=conf.getboolean('smart_sensor','USE_SMART_SENSOR'),safe_mode:bool=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. """ifself.read_dags_from_db:returnself.log.info("Filling up the DagBag from %s",dag_folder)dag_folder=dag_folderorself.dag_folder# Used to store stats around DagBag processingstats=[]# Ensure dag_folder is a str -- it may have been a pathlib.Pathdag_folder=correct_maybe_zipped(str(dag_folder))forfilepathinlist_py_file_paths(dag_folder,safe_mode=safe_mode,include_examples=include_examples,include_smart_sensor=include_smart_sensor,):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)fordaginfound_dags),dags=str([dag.dag_idfordaginfound_dags]),))exceptExceptionase:self.log.exception(e)self.dagbag_stats=sorted(stats,key=lambdax:x.duration,reverse=True)
[docs]defcollect_dags_from_db(self):"""Collects DAGs from database."""fromairflow.models.serialized_dagimportSerializedDagModelwithStats.timer('collect_db_dags'):self.log.info("Filling up the DagBag from database")# The dagbag contains all rows in serialized_dag table. Deleted DAGs are deleted# from the table by the scheduler job.self.dags=SerializedDagModel.read_all_dags()# Adds subdags.# DAG post-processing steps such as self.bag_dag and croniter are not needed as# they are done by scheduler before serialization.subdags={}fordaginself.dags.values():forsubdagindag.subdags:subdags[subdag.dag_id]=subdagself.dags.update(subdags)
[docs]defdagbag_report(self):"""Prints a report around DagBag loading stats"""stats=self.dagbag_statsdag_folder=self.dag_folderduration=sum((o.durationforoinstats),timedelta()).total_seconds()dag_num=sum(o.dag_numforoinstats)task_num=sum(o.task_numforoinstats)table=tabulate(stats,headers="keys")report=textwrap.dedent(f"""\n ------------------------------------------------------------------- DagBag loading stats for {dag_folder} ------------------------------------------------------------------- Number of DAGs: {dag_num} Total task number: {task_num} DagBag parsing time: {duration}{table} """)returnreport
@provide_session
[docs]defsync_to_db(self,session:Optional[Session]=None):"""Save attributes about list of DAG to the DB."""# To avoid circular import - airflow.models.dagbag -> airflow.models.dag -> airflow.models.dagbagfromairflow.models.dagimportDAGfromairflow.models.serialized_dagimportSerializedDagModeldef_serialize_dag_capturing_errors(dag,session):""" Try to serialize the dag to the DB, but make a note of any errors. We can't place them directly in import_errors, as this may be retried, and work the next time """ifdag.is_subdag:return[]try:# We can't use bulk_write_to_db as we want to capture each error individuallydag_was_updated=SerializedDagModel.write_dag(dag,min_update_interval=settings.MIN_SERIALIZED_DAG_UPDATE_INTERVAL,session=session,)ifdag_was_updated:self._sync_perm_for_dag(dag,session=session)return[]exceptOperationalError:raiseexceptException:self.log.exception("Failed to write serialized DAG: %s",dag.full_filepath)return[(dag.fileloc,traceback.format_exc(limit=-self.dagbag_import_error_traceback_depth))]# Retry 'DAG.bulk_write_to_db' & 'SerializedDagModel.bulk_sync_to_db' in case# of any Operational Errors# In case of failures, provide_session handles rollbackforattemptinrun_with_db_retries(logger=self.log):withattempt:serialize_errors=[]self.log.debug("Running dagbag.sync_to_db with retries. Try %d of %d",attempt.retry_state.attempt_number,MAX_DB_RETRIES,)self.log.debug("Calling the DAG.bulk_sync_to_db method")try:# Write Serialized DAGs to DB, capturing errorsfordaginself.dags.values():serialize_errors.extend(_serialize_dag_capturing_errors(dag,session))DAG.bulk_write_to_db(self.dags.values(),session=session)exceptOperationalError:session.rollback()raise# Only now we are "complete" do we update import_errors - don't want to record errors from# previous failed attemptsself.import_errors.update(dict(serialize_errors))
@provide_session
[docs]def_sync_perm_for_dag(self,dag,session:Optional[Session]=None):"""Sync DAG specific permissions, if necessary"""fromflask_appbuilder.security.sqlaimportmodelsassqla_modelsfromairflow.security.permissionsimportDAG_PERMS,resource_name_for_dagdefneeds_perm_views(dag_id:str)->bool:dag_resource_name=resource_name_for_dag(dag_id)forpermission_nameinDAG_PERMS:ifnot(session.query(sqla_models.PermissionView).join(sqla_models.Permission).join(sqla_models.ViewMenu).filter(sqla_models.Permission.name==permission_name).filter(sqla_models.ViewMenu.name==dag_resource_name).one_or_none()):returnTruereturnFalseifdag.access_controlorneeds_perm_views(dag.dag_id):self.log.debug("Syncing DAG permissions: %s to the DB",dag.dag_id)fromairflow.www.securityimportApplessAirflowSecurityManagersecurity_manager=ApplessAirflowSecurityManager(session=session)security_manager.sync_perm_for_dag(dag.dag_id,dag.access_control)