[ https://issues.apache.org/jira/browse/AIRFLOW-2855?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16572527#comment-16572527 ]
ASF GitHub Bot commented on AIRFLOW-2855: ----------------------------------------- feng-tao closed pull request #3698: [AIRFLOW-2855] Check Cron Expression Validity in DagBag.process_file() URL: https://github.com/apache/incubator-airflow/pull/3698 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/airflow/models.py b/airflow/models.py index cf7eb0a64f..206106a4e9 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -62,7 +62,9 @@ from sqlalchemy.orm import reconstructor, relationship, synonym from sqlalchemy_utc import UtcDateTime -from croniter import croniter +from croniter import ( + croniter, CroniterBadCronError, CroniterBadDateError, CroniterNotAlphaError +) import six from airflow import settings, utils @@ -412,8 +414,18 @@ def process_file(self, filepath, only_if_updated=True, safe_mode=True): try: dag.is_subdag = False self.bag_dag(dag, parent_dag=dag, root_dag=dag) + if isinstance(dag._schedule_interval, six.string_types): + croniter(dag._schedule_interval) found_dags.append(dag) found_dags += dag.subdags + except (CroniterBadCronError, + CroniterBadDateError, + CroniterNotAlphaError) as cron_e: + self.log.exception("Failed to bag_dag: %s", dag.full_filepath) + self.import_errors[dag.full_filepath] = \ + "Invalid Cron expression: " + str(cron_e) + self.file_last_changed[dag.full_filepath] = \ + file_last_changed_on_disk except AirflowDagCycleException as cycle_exception: self.log.exception("Failed to bag_dag: %s", dag.full_filepath) self.import_errors[dag.full_filepath] = str(cycle_exception) diff --git a/tests/dags/test_invalid_cron.py b/tests/dags/test_invalid_cron.py new file mode 100755 index 0000000000..51a0e43cb5 --- /dev/null +++ b/tests/dags/test_invalid_cron.py @@ -0,0 +1,35 @@ +# -*- coding: utf-8 -*- +# +# 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. + +from airflow.models import DAG +from airflow.operators.dummy_operator import DummyOperator +from airflow.utils.timezone import datetime + +# The schedule_interval specified here is an INVALID +# Cron expression. This invalid DAG will be used to +# test whether dagbag.process_file() can identify +# invalid Cron expression. +dag1 = DAG( + dag_id='test_invalid_cron', + start_date=datetime(2015, 1, 1), + schedule_interval="0 100 * * *") +dag1_task1 = DummyOperator( + task_id='task1', + dag=dag1, + owner='airflow') diff --git a/tests/dags/test_zip_invalid_cron.zip b/tests/dags/test_zip_invalid_cron.zip new file mode 100644 index 0000000000..fe45153abe Binary files /dev/null and b/tests/dags/test_zip_invalid_cron.zip differ diff --git a/tests/models.py b/tests/models.py index 1c88ea47f7..5a0397dc08 100644 --- a/tests/models.py +++ b/tests/models.py @@ -56,7 +56,7 @@ from airflow.utils.trigger_rule import TriggerRule from mock import patch, ANY from parameterized import parameterized -from tempfile import NamedTemporaryFile +from tempfile import mkdtemp, NamedTemporaryFile DEFAULT_DATE = timezone.datetime(2016, 1, 1) TEST_DAGS_FOLDER = os.path.join( @@ -1038,6 +1038,19 @@ def test_zip(self): dagbag.process_file(os.path.join(TEST_DAGS_FOLDER, "test_zip.zip")) self.assertTrue(dagbag.get_dag("test_zip_dag")) + def test_process_file_cron_validity_check(self): + """ + test if an invalid cron expression + as schedule interval can be identified + """ + invalid_dag_files = ["test_invalid_cron.py", "test_zip_invalid_cron.zip"] + dagbag = models.DagBag(dag_folder=mkdtemp()) + + self.assertEqual(len(dagbag.import_errors), 0) + for d in invalid_dag_files: + dagbag.process_file(os.path.join(TEST_DAGS_FOLDER, d)) + self.assertEqual(len(dagbag.import_errors), len(invalid_dag_files)) + @patch.object(DagModel,'get_current') def test_get_dag_without_refresh(self, mock_dagmodel): """ ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > Need to Check Validity of Cron Expression When Process DAG File/Zip File > ------------------------------------------------------------------------ > > Key: AIRFLOW-2855 > URL: https://issues.apache.org/jira/browse/AIRFLOW-2855 > Project: Apache Airflow > Issue Type: Improvement > Components: DAG > Reporter: Xiaodong DENG > Assignee: Xiaodong DENG > Priority: Critical > > *schedule_interval* of DAGs can either be *timedelta* or a *Cron expression*. > When it's a Cron expression, there is no mechanism to check its validity at > this moment. If there is anything wrong with the Cron expression itself, it > will cause issues when methods _*following_schedule(**)*_ and > _*previous_schedule()*_ are invoked (will affect scheduling). However, > exceptions will only be written into logs. From Web UI, it’s hard for users > to identify this issue & the source while no new task can be initiated > (especially for users who’re not very familiar with Cron). > It may be good to show error messages in web UI when a DAG's Cron expression > (as schedule_interval) can not be parsed by *croniter* properly. > -- This message was sent by Atlassian JIRA (v7.6.3#76005)