[jira] [Commented] (AIRFLOW-2855) Need to Check Validity of Cron Expression When Process DAG File/Zip File

2018-08-07 Thread ASF subversion and git services (JIRA)


[ 
https://issues.apache.org/jira/browse/AIRFLOW-2855?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16572528#comment-16572528
 ] 

ASF subversion and git services commented on AIRFLOW-2855:
--

Commit d47580feaf80eeebb416d0179dfa8db3f4e1d2c9 in incubator-airflow's branch 
refs/heads/master from Xiaodong
[ https://gitbox.apache.org/repos/asf?p=incubator-airflow.git;h=d47580f ]

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

A DAG can be imported as a .py script properly,
but the Cron expression inside as "schedule_interval" may be
invalid, like "0 100 * * *".

This commit helps check the validity of Cron expression in DAG
files (.py) and packaged DAG files (.zip), and help show
exception messages in web UI by add these exceptions into
metadata "import_error".

> 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)


[jira] [Commented] (AIRFLOW-2855) Need to Check Validity of Cron Expression When Process DAG File/Zip File

2018-08-07 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/AIRFLOW-2855?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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 00..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 00..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 = 

[jira] [Commented] (AIRFLOW-2855) Need to Check Validity of Cron Expression When Process DAG File/Zip File

2018-08-05 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/AIRFLOW-2855?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16569486#comment-16569486
 ] 

ASF GitHub Bot commented on AIRFLOW-2855:
-

XD-DENG opened a new pull request #3698: [AIRFLOW-2855] Check Cron Expression 
Validity in DagBag.process_file()
URL: https://github.com/apache/incubator-airflow/pull/3698
 
 
   ### Jira
   
   - [x] My PR addresses the following [Airflow 
Jira](https://issues.apache.org/jira/browse/AIRFLOW/) issues and references 
them in the PR title. For example, "\[AIRFLOW-XXX\] My Airflow PR"
 - https://issues.apache.org/jira/browse/AIRFLOW-2855
 - In case you are fixing a typo in the documentation you can prepend your 
commit with \[AIRFLOW-XXX\], code changes always need a Jira issue.
   
   ### Description
   
   - [x] Here are some details about my PR, including screenshots of any UI 
changes:
   
   `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 is implemented by adding these exceptions into metadata `import_error`, 
whose entries will be shown as error messages in web UI).
   
   Related tests are added as well.
   
   **Screenshot**
   https://user-images.githubusercontent.com/11539188/43687000-0ced5a5e-9901-11e8-85ba-b1d8ce0fce9f.png;>
   
   
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines, and I 
have squashed multiple commits if they address the same issue. In addition, my 
commits follow the guidelines from "[How to write a good git commit 
message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - When adding new operators/hooks/sensors, the autoclass documentation 
generation needs to be added.
   
   ### Code Quality
   
   - [x] Passes `git diff upstream/master -u -- "*.py" | flake8 --diff`
   


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)