[jira] [Commented] (AIRFLOW-2231) DAG with a relativedelta schedule_interval fails

2018-08-07 Thread Kaxil Naik (JIRA)


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

Kaxil Naik commented on AIRFLOW-2231:
-

Resolved by https://github.com/apache/incubator-airflow/pull/3174

> DAG with a relativedelta schedule_interval fails
> 
>
> Key: AIRFLOW-2231
> URL: https://issues.apache.org/jira/browse/AIRFLOW-2231
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: DAG
>Reporter: Kyle Brooks
>Priority: Major
> Attachments: test_reldel.py
>
>
> The documentation for the DAG class says using 
> dateutil.relativedelta.relativedelta as a schedule_interval is supported but 
> it fails:
>  
> Traceback (most recent call last):
>   File "/usr/local/lib/python3.6/site-packages/airflow/models.py", line 285, 
> in process_file
>     m = imp.load_source(mod_name, filepath)
>   File 
> "/usr/local/Cellar/python3/3.6.1/Frameworks/Python.framework/Versions/3.6/lib/python3.6/imp.py",
>  line 172, in load_source
>     module = _load(spec)
>   File "", line 675, in _load
>   File "", line 655, in _load_unlocked
>   File "", line 678, in exec_module
>   File "", line 205, in _call_with_frames_removed
>   File "/Users/k398995/airflow/dags/test_reldel.py", line 33, in 
>     dagrun_timeout=timedelta(minutes=60))
>   File "/usr/local/lib/python3.6/site-packages/airflow/models.py", line 2914, 
> in __init__
>     if schedule_interval in cron_presets:
> TypeError: unhashable type: 'relativedelta'
>  
> It looks like the __init__ function for class DAG assumes the 
> schedule_interval is hashable.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (AIRFLOW-2231) DAG with a relativedelta schedule_interval fails

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


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

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

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

[AIRFLOW-2231] Fix relativedelta DAG schedule_interval (#3174)

Fixes issues when specifying a DAG with a schedule_interval of type 
relativedelta.

> DAG with a relativedelta schedule_interval fails
> 
>
> Key: AIRFLOW-2231
> URL: https://issues.apache.org/jira/browse/AIRFLOW-2231
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: DAG
>Reporter: Kyle Brooks
>Priority: Major
> Attachments: test_reldel.py
>
>
> The documentation for the DAG class says using 
> dateutil.relativedelta.relativedelta as a schedule_interval is supported but 
> it fails:
>  
> Traceback (most recent call last):
>   File "/usr/local/lib/python3.6/site-packages/airflow/models.py", line 285, 
> in process_file
>     m = imp.load_source(mod_name, filepath)
>   File 
> "/usr/local/Cellar/python3/3.6.1/Frameworks/Python.framework/Versions/3.6/lib/python3.6/imp.py",
>  line 172, in load_source
>     module = _load(spec)
>   File "", line 675, in _load
>   File "", line 655, in _load_unlocked
>   File "", line 678, in exec_module
>   File "", line 205, in _call_with_frames_removed
>   File "/Users/k398995/airflow/dags/test_reldel.py", line 33, in 
>     dagrun_timeout=timedelta(minutes=60))
>   File "/usr/local/lib/python3.6/site-packages/airflow/models.py", line 2914, 
> in __init__
>     if schedule_interval in cron_presets:
> TypeError: unhashable type: 'relativedelta'
>  
> It looks like the __init__ function for class DAG assumes the 
> schedule_interval is hashable.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (AIRFLOW-2231) DAG with a relativedelta schedule_interval fails

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


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

ASF GitHub Bot commented on AIRFLOW-2231:
-

kaxil closed pull request #3174: [AIRFLOW-2231] Fix relativedelta DAG 
schedule_interval
URL: https://github.com/apache/incubator-airflow/pull/3174
 
 
   

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 1d832ab297..fbd651fdd0 100755
--- a/airflow/models.py
+++ b/airflow/models.py
@@ -26,7 +26,7 @@
 
 from builtins import str, object, bytes, ImportError as BuiltinImportError
 import copy
-from collections import namedtuple, defaultdict
+from collections import namedtuple, defaultdict, Hashable
 from datetime import timedelta
 
 import dill
@@ -3233,7 +3233,7 @@ def __init__(
 )
 
 self.schedule_interval = schedule_interval
-if schedule_interval in cron_presets:
+if isinstance(schedule_interval, Hashable) and schedule_interval in 
cron_presets:
 self._schedule_interval = cron_presets.get(schedule_interval)
 elif schedule_interval == '@once':
 self._schedule_interval = None
@@ -,7 +,7 @@ def following_schedule(self, dttm):
 cron = croniter(self._schedule_interval, dttm)
 following = timezone.make_aware(cron.get_next(datetime), 
self.timezone)
 return timezone.convert_to_utc(following)
-elif isinstance(self._schedule_interval, timedelta):
+elif self._schedule_interval is not None:
 return dttm + self._schedule_interval
 
 def previous_schedule(self, dttm):
@@ -3348,7 +3348,7 @@ def previous_schedule(self, dttm):
 cron = croniter(self._schedule_interval, dttm)
 prev = timezone.make_aware(cron.get_prev(datetime), self.timezone)
 return timezone.convert_to_utc(prev)
-elif isinstance(self._schedule_interval, timedelta):
+elif self._schedule_interval is not None:
 return dttm - self._schedule_interval
 
 def get_run_dates(self, start_date, end_date=None):
diff --git a/tests/core.py b/tests/core.py
index 6cfd10b02a..ca9de39a79 100644
--- a/tests/core.py
+++ b/tests/core.py
@@ -153,6 +153,46 @@ def test_schedule_dag_no_previous_runs(self):
 self.assertFalse(dag_run.external_trigger)
 dag.clear()
 
+def test_schedule_dag_relativedelta(self):
+"""
+Tests scheduling a dag with a relativedelta schedule_interval
+"""
+delta = relativedelta(hours=+1)
+dag = DAG(TEST_DAG_ID + 'test_schedule_dag_relativedelta',
+  schedule_interval=delta)
+dag.add_task(models.BaseOperator(
+task_id="faketastic",
+owner='Also fake',
+start_date=datetime(2015, 1, 2, 0, 0)))
+
+dag_run = 
jobs.SchedulerJob(**self.default_scheduler_args).create_dag_run(dag)
+self.assertIsNotNone(dag_run)
+self.assertEqual(dag.dag_id, dag_run.dag_id)
+self.assertIsNotNone(dag_run.run_id)
+self.assertNotEqual('', dag_run.run_id)
+self.assertEqual(
+datetime(2015, 1, 2, 0, 0),
+dag_run.execution_date,
+msg='dag_run.execution_date did not match expectation: {0}'
+.format(dag_run.execution_date)
+)
+self.assertEqual(State.RUNNING, dag_run.state)
+self.assertFalse(dag_run.external_trigger)
+dag_run2 = 
jobs.SchedulerJob(**self.default_scheduler_args).create_dag_run(dag)
+self.assertIsNotNone(dag_run2)
+self.assertEqual(dag.dag_id, dag_run2.dag_id)
+self.assertIsNotNone(dag_run2.run_id)
+self.assertNotEqual('', dag_run2.run_id)
+self.assertEqual(
+datetime(2015, 1, 2, 0, 0) + delta,
+dag_run2.execution_date,
+msg='dag_run2.execution_date did not match expectation: {0}'
+.format(dag_run2.execution_date)
+)
+self.assertEqual(State.RUNNING, dag_run2.state)
+self.assertFalse(dag_run2.external_trigger)
+dag.clear()
+
 def test_schedule_dag_fake_scheduled_previous(self):
 """
 Test scheduling a dag where there is a prior DagRun


 


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


> DAG with a relativedelta schedule_interval fails
> 
>
> Key: 

[jira] [Commented] (AIRFLOW-2231) DAG with a relativedelta schedule_interval fails

2018-03-29 Thread Kyle Brooks (JIRA)

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

Kyle Brooks commented on AIRFLOW-2231:
--

I've added a PR that fixes this issue to the issue links above.

I think I've followed the proper PR procedure, please let me know if it needs 
to be modified.

> DAG with a relativedelta schedule_interval fails
> 
>
> Key: AIRFLOW-2231
> URL: https://issues.apache.org/jira/browse/AIRFLOW-2231
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: DAG
>Reporter: Kyle Brooks
>Priority: Major
> Attachments: test_reldel.py
>
>
> The documentation for the DAG class says using 
> dateutil.relativedelta.relativedelta as a schedule_interval is supported but 
> it fails:
>  
> Traceback (most recent call last):
>   File "/usr/local/lib/python3.6/site-packages/airflow/models.py", line 285, 
> in process_file
>     m = imp.load_source(mod_name, filepath)
>   File 
> "/usr/local/Cellar/python3/3.6.1/Frameworks/Python.framework/Versions/3.6/lib/python3.6/imp.py",
>  line 172, in load_source
>     module = _load(spec)
>   File "", line 675, in _load
>   File "", line 655, in _load_unlocked
>   File "", line 678, in exec_module
>   File "", line 205, in _call_with_frames_removed
>   File "/Users/k398995/airflow/dags/test_reldel.py", line 33, in 
>     dagrun_timeout=timedelta(minutes=60))
>   File "/usr/local/lib/python3.6/site-packages/airflow/models.py", line 2914, 
> in __init__
>     if schedule_interval in cron_presets:
> TypeError: unhashable type: 'relativedelta'
>  
> It looks like the __init__ function for class DAG assumes the 
> schedule_interval is hashable.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (AIRFLOW-2231) DAG with a relativedelta schedule_interval fails

2018-03-22 Thread Kyle Brooks (JIRA)

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

Kyle Brooks commented on AIRFLOW-2231:
--

Thanks Joy, I was able to get relativedelta to work by making some slight 
changes to the model.py code.  I do think there's a use case for using 
relativedelta instead of cron syntax because it is more powerful.

I will submit a PR if and when I get approval to share the modifications.

Regarding not being able to reproduce the bug.  I would guess that the version 
of dateutil.relativedelta.relativedelta that you have implements __hash__ and 
so the class DAG __init__ does not fail like mine.  I have seen the same issue 
where the DAG will never be scheduled for classes that are hashable due to 
issues in the section of code you identified in your link above.

> DAG with a relativedelta schedule_interval fails
> 
>
> Key: AIRFLOW-2231
> URL: https://issues.apache.org/jira/browse/AIRFLOW-2231
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: DAG
>Reporter: Kyle Brooks
>Priority: Major
> Attachments: test_reldel.py
>
>
> The documentation for the DAG class says using 
> dateutil.relativedelta.relativedelta as a schedule_interval is supported but 
> it fails:
>  
> Traceback (most recent call last):
>   File "/usr/local/lib/python3.6/site-packages/airflow/models.py", line 285, 
> in process_file
>     m = imp.load_source(mod_name, filepath)
>   File 
> "/usr/local/Cellar/python3/3.6.1/Frameworks/Python.framework/Versions/3.6/lib/python3.6/imp.py",
>  line 172, in load_source
>     module = _load(spec)
>   File "", line 675, in _load
>   File "", line 655, in _load_unlocked
>   File "", line 678, in exec_module
>   File "", line 205, in _call_with_frames_removed
>   File "/Users/k398995/airflow/dags/test_reldel.py", line 33, in 
>     dagrun_timeout=timedelta(minutes=60))
>   File "/usr/local/lib/python3.6/site-packages/airflow/models.py", line 2914, 
> in __init__
>     if schedule_interval in cron_presets:
> TypeError: unhashable type: 'relativedelta'
>  
> It looks like the __init__ function for class DAG assumes the 
> schedule_interval is hashable.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (AIRFLOW-2231) DAG with a relativedelta schedule_interval fails

2018-03-21 Thread Joy Gao (JIRA)

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

Joy Gao commented on AIRFLOW-2231:
--

(That said, I was still unable to produce the bug you had, perhaps we are using 
different version of dateutil package, the issue I had was that the dag with 
relativedelta would simply never get scheduled)

> DAG with a relativedelta schedule_interval fails
> 
>
> Key: AIRFLOW-2231
> URL: https://issues.apache.org/jira/browse/AIRFLOW-2231
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: DAG
>Reporter: Kyle Brooks
>Priority: Major
> Attachments: test_reldel.py
>
>
> The documentation for the DAG class says using 
> dateutil.relativedelta.relativedelta as a schedule_interval is supported but 
> it fails:
>  
> Traceback (most recent call last):
>   File "/usr/local/lib/python3.6/site-packages/airflow/models.py", line 285, 
> in process_file
>     m = imp.load_source(mod_name, filepath)
>   File 
> "/usr/local/Cellar/python3/3.6.1/Frameworks/Python.framework/Versions/3.6/lib/python3.6/imp.py",
>  line 172, in load_source
>     module = _load(spec)
>   File "", line 675, in _load
>   File "", line 655, in _load_unlocked
>   File "", line 678, in exec_module
>   File "", line 205, in _call_with_frames_removed
>   File "/Users/k398995/airflow/dags/test_reldel.py", line 33, in 
>     dagrun_timeout=timedelta(minutes=60))
>   File "/usr/local/lib/python3.6/site-packages/airflow/models.py", line 2914, 
> in __init__
>     if schedule_interval in cron_presets:
> TypeError: unhashable type: 'relativedelta'
>  
> It looks like the __init__ function for class DAG assumes the 
> schedule_interval is hashable.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (AIRFLOW-2231) DAG with a relativedelta schedule_interval fails

2018-03-21 Thread Joy Gao (JIRA)

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

Joy Gao commented on AIRFLOW-2231:
--

Got a chance to look at code and turns out the doc is not accurate, 
dateutil.relativedelta.relativedelta is not supported, but datetime.timedelta 
is. In the case where you need monthly cadence, you can use either @monthly or 
cron syntax instead.

Alternatively, if you'd like to add relativedelta, you can  submit a PR and 
modify [this 
section|https://github.com/apache/incubator-airflow/blob/master/airflow/models.py#L3168-L3196]
 to support relativedelta.

> DAG with a relativedelta schedule_interval fails
> 
>
> Key: AIRFLOW-2231
> URL: https://issues.apache.org/jira/browse/AIRFLOW-2231
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: DAG
>Reporter: Kyle Brooks
>Priority: Major
> Attachments: test_reldel.py
>
>
> The documentation for the DAG class says using 
> dateutil.relativedelta.relativedelta as a schedule_interval is supported but 
> it fails:
>  
> Traceback (most recent call last):
>   File "/usr/local/lib/python3.6/site-packages/airflow/models.py", line 285, 
> in process_file
>     m = imp.load_source(mod_name, filepath)
>   File 
> "/usr/local/Cellar/python3/3.6.1/Frameworks/Python.framework/Versions/3.6/lib/python3.6/imp.py",
>  line 172, in load_source
>     module = _load(spec)
>   File "", line 675, in _load
>   File "", line 655, in _load_unlocked
>   File "", line 678, in exec_module
>   File "", line 205, in _call_with_frames_removed
>   File "/Users/k398995/airflow/dags/test_reldel.py", line 33, in 
>     dagrun_timeout=timedelta(minutes=60))
>   File "/usr/local/lib/python3.6/site-packages/airflow/models.py", line 2914, 
> in __init__
>     if schedule_interval in cron_presets:
> TypeError: unhashable type: 'relativedelta'
>  
> It looks like the __init__ function for class DAG assumes the 
> schedule_interval is hashable.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (AIRFLOW-2231) DAG with a relativedelta schedule_interval fails

2018-03-21 Thread Kyle Brooks (JIRA)

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

Kyle Brooks commented on AIRFLOW-2231:
--

I just attached is a dag file that should reproduce this.  Thank you.

> DAG with a relativedelta schedule_interval fails
> 
>
> Key: AIRFLOW-2231
> URL: https://issues.apache.org/jira/browse/AIRFLOW-2231
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: DAG
>Reporter: Kyle Brooks
>Priority: Major
> Attachments: test_reldel.py
>
>
> The documentation for the DAG class says using 
> dateutil.relativedelta.relativedelta as a schedule_interval is supported but 
> it fails:
>  
> Traceback (most recent call last):
>   File "/usr/local/lib/python3.6/site-packages/airflow/models.py", line 285, 
> in process_file
>     m = imp.load_source(mod_name, filepath)
>   File 
> "/usr/local/Cellar/python3/3.6.1/Frameworks/Python.framework/Versions/3.6/lib/python3.6/imp.py",
>  line 172, in load_source
>     module = _load(spec)
>   File "", line 675, in _load
>   File "", line 655, in _load_unlocked
>   File "", line 678, in exec_module
>   File "", line 205, in _call_with_frames_removed
>   File "/Users/k398995/airflow/dags/test_reldel.py", line 33, in 
>     dagrun_timeout=timedelta(minutes=60))
>   File "/usr/local/lib/python3.6/site-packages/airflow/models.py", line 2914, 
> in __init__
>     if schedule_interval in cron_presets:
> TypeError: unhashable type: 'relativedelta'
>  
> It looks like the __init__ function for class DAG assumes the 
> schedule_interval is hashable.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (AIRFLOW-2231) DAG with a relativedelta schedule_interval fails

2018-03-20 Thread Joy Gao (JIRA)

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

Joy Gao commented on AIRFLOW-2231:
--

Can't replicate this. Can you provide the dag file? thanks!

> DAG with a relativedelta schedule_interval fails
> 
>
> Key: AIRFLOW-2231
> URL: https://issues.apache.org/jira/browse/AIRFLOW-2231
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: DAG
>Reporter: Kyle Brooks
>Priority: Major
>
> The documentation for the DAG class says using 
> dateutil.relativedelta.relativedelta as a schedule_interval is supported but 
> it fails:
>  
> Traceback (most recent call last):
>   File "/usr/local/lib/python3.6/site-packages/airflow/models.py", line 285, 
> in process_file
>     m = imp.load_source(mod_name, filepath)
>   File 
> "/usr/local/Cellar/python3/3.6.1/Frameworks/Python.framework/Versions/3.6/lib/python3.6/imp.py",
>  line 172, in load_source
>     module = _load(spec)
>   File "", line 675, in _load
>   File "", line 655, in _load_unlocked
>   File "", line 678, in exec_module
>   File "", line 205, in _call_with_frames_removed
>   File "/Users/k398995/airflow/dags/test_reldel.py", line 33, in 
>     dagrun_timeout=timedelta(minutes=60))
>   File "/usr/local/lib/python3.6/site-packages/airflow/models.py", line 2914, 
> in __init__
>     if schedule_interval in cron_presets:
> TypeError: unhashable type: 'relativedelta'
>  
> It looks like the __init__ function for class DAG assumes the 
> schedule_interval is hashable.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)