Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[AIRFLOW-2855] Check Cron Expression Validity in DagBag.process_file() #3698

Merged
merged 1 commit into from
Aug 8, 2018
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
14 changes: 13 additions & 1 deletion airflow/models.py
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I may be missing something but do we need to perform the check twice?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

One is for .py DAG file, and the another is for package DAG file (.zip file).

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @yrqls21 , you're right. Thanks for pointing this out.

I'll modify accordingly.

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)
Expand Down
35 changes: 35 additions & 0 deletions tests/dags/test_invalid_cron.py
Original file line number Diff line number Diff line change
@@ -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')
Binary file added tests/dags/test_zip_invalid_cron.zip
Binary file not shown.
15 changes: 14 additions & 1 deletion tests/models.py
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand Down Expand Up @@ -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):
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I guess we do still want to keep the file in 644 instead of 755 mode.

"""
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):
"""
Expand Down