diff --git a/airflow/dag_processing/manager.py b/airflow/dag_processing/manager.py index 570e8f306011a..e3358a3e1dd2b 100644 --- a/airflow/dag_processing/manager.py +++ b/airflow/dag_processing/manager.py @@ -735,14 +735,17 @@ def _refresh_dag_dir(self): dag_filelocs = [] for fileloc in self._file_paths: if not fileloc.endswith(".py") and zipfile.is_zipfile(fileloc): - with zipfile.ZipFile(fileloc) as z: - dag_filelocs.extend( - [ - os.path.join(fileloc, info.filename) - for info in z.infolist() - if might_contain_dag(info.filename, True, z) - ] - ) + try: + with zipfile.ZipFile(fileloc) as z: + dag_filelocs.extend( + [ + os.path.join(fileloc, info.filename) + for info in z.infolist() + if might_contain_dag(info.filename, True, z) + ] + ) + except zipfile.BadZipFile as err: + self.log.error("There was an err accessing %s, %s", fileloc, err) else: dag_filelocs.append(fileloc)