[jira] [Commented] (AIRFLOW-1027) Task details cannot be shown when PythonOperator calls a partial function

2018-11-01 Thread ASF GitHub Bot (JIRA)


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

ASF GitHub Bot commented on AIRFLOW-1027:
-

ashb closed pull request #2190: [AIRFLOW-1027] Fix error in task details when 
using functools.partial in PythonOperator
URL: https://github.com/apache/incubator-airflow/pull/2190
 
 
   

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/www/views.py b/airflow/www/views.py
index 15735b4b79..f39c388685 100644
--- a/airflow/www/views.py
+++ b/airflow/www/views.py
@@ -19,7 +19,7 @@
 import os
 import pkg_resources
 import socket
-from functools import wraps
+from functools import wraps, partial
 from datetime import datetime, timedelta
 import dateutil.parser
 import copy
@@ -220,7 +220,7 @@ def wrapped_markdown(s):
 'doc_yaml': lambda x: render(x, lexers.YamlLexer),
 'doc_md': wrapped_markdown,
 'python_callable': lambda x: render(
-inspect.getsource(x), lexers.PythonLexer),
+inspect.getsource(x if not isinstance(x, partial) else x.func), 
lexers.PythonLexer),
 }
 
 
diff --git a/tests/core.py b/tests/core.py
index 848553af3e..c1fcfa58e0 100644
--- a/tests/core.py
+++ b/tests/core.py
@@ -1652,6 +1652,13 @@ def test_fetch_task_instance(self):
 response = self.app.get(url)
 self.assertIn("run_this_last", response.data.decode('utf-8'))
 
+def test_dag_view_task_with_python_operator_using_partial(self):
+response = self.app.get(
+'/admin/airflow/task?'
+
'task_id=test_dagrun_functool_partial_id=test_issue_AIRFLOW_1027_dag&'
+'execution_date={}'.format(DEFAULT_DATE_DS))
+self.assertEquals(response.status_code, 200)
+
 def tearDown(self):
 configuration.conf.set("webserver", "expose_config", "False")
 self.dag_bash.clear(start_date=DEFAULT_DATE, end_date=datetime.now())
diff --git a/tests/dags/test_issue_AIRFLOW_1027.py 
b/tests/dags/test_issue_AIRFLOW_1027.py
new file mode 100644
index 00..0e14cfb759
--- /dev/null
+++ b/tests/dags/test_issue_AIRFLOW_1027.py
@@ -0,0 +1,44 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed 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.
+
+
+"""
+DAG designed to test a PythonOperator that calls a functool.partial
+"""
+import functools
+
+from datetime import datetime
+
+from airflow.models import DAG
+from airflow.operators.python_operator import PythonOperator
+
+DEFAULT_DATE = datetime(2016, 1, 1)
+default_args = dict(
+start_date=DEFAULT_DATE,
+owner='airflow')
+
+
+def func_with_two_args(arg_1, arg_2):
+pass
+
+
+partial_func = functools.partial(func_with_two_args, arg_1=1)
+
+
+dag = DAG(dag_id='test_issue_AIRFLOW_1027_dag', default_args=default_args)
+
+dag_task1 = PythonOperator(
+task_id='test_dagrun_functool_partial',
+dag=dag,
+python_callable=partial_func)


 


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


> Task details cannot be shown when PythonOperator calls a partial function
> -
>
> Key: AIRFLOW-1027
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1027
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: ui
>Affects Versions: 1.7.1
>Reporter: Adrian Partl
>Assignee: Adrian Partl
>Priority: Minor
>
> Showing task details of a PythonOperator that uses a `functools.partial` as a 
> callable results in the following error:
> {noformat}
>   File "/usr/lib/python2.7/site-packages/airflow/www/views.py", line 909, in 
> task
> special_attrs_rendered[attr_name] = attr_renderer[attr_name](source)
>   File "/usr/lib/python2.7/site-packages/airflow/www/views.py", line 224, in 
> 
> inspect.getsource(x), lexers.PythonLexer),
>   File "/usr/lib64/python2.7/inspect.py", line 701, in getsource
> lines, lnum = 

[jira] [Commented] (AIRFLOW-1027) Task details cannot be shown when PythonOperator calls a partial function

2018-09-02 Thread Apache Spark (JIRA)


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

Apache Spark commented on AIRFLOW-1027:
---

User 'adrpar' has created a pull request for this issue:
https://github.com/apache/incubator-airflow/pull/2190

> Task details cannot be shown when PythonOperator calls a partial function
> -
>
> Key: AIRFLOW-1027
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1027
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: ui
>Affects Versions: Airflow 1.7.1
>Reporter: Adrian Partl
>Assignee: Adrian Partl
>Priority: Minor
>
> Showing task details of a PythonOperator that uses a `functools.partial` as a 
> callable results in the following error:
> {noformat}
>   File "/usr/lib/python2.7/site-packages/airflow/www/views.py", line 909, in 
> task
> special_attrs_rendered[attr_name] = attr_renderer[attr_name](source)
>   File "/usr/lib/python2.7/site-packages/airflow/www/views.py", line 224, in 
> 
> inspect.getsource(x), lexers.PythonLexer),
>   File "/usr/lib64/python2.7/inspect.py", line 701, in getsource
> lines, lnum = getsourcelines(object)
>   File "/usr/lib64/python2.7/inspect.py", line 690, in getsourcelines
> lines, lnum = findsource(object)
>   File "/usr/lib64/python2.7/inspect.py", line 526, in findsource
> file = getfile(object)
>   File "/usr/lib64/python2.7/inspect.py", line 420, in getfile
> 'function, traceback, frame, or code object'.format(object))
> TypeError:  is not a module, class, 
> method, function, traceback, frame, or code object
> {noformat}
> A sample dag definition for this is:
> {noformat}
> def func_with_two_args(arg_1, arg_2):
> pass
> partial_func = functools.partial(func_with_two_args, arg_1=1)
> dag = DAG(dag_id='test_issue_1333_dag', default_args=default_args)
> dag_task1 = PythonOperator(
> task_id='test_dagrun_functool_partial',
> dag=dag,
> python_callable=partial_func)
> {noformat}
> I will provide a PR with a fix for this.



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


[jira] [Commented] (AIRFLOW-1027) Task details cannot be shown when PythonOperator calls a partial function

2018-02-14 Thread Matthew Revell (JIRA)

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

Matthew Revell commented on AIRFLOW-1027:
-

This ticket only describes one cause for this error. It can also occur for 
other reasons. A more complete solution is proposed with 
[AIRFLOW-2099|https://issues.apache.org/jira/browse/AIRFLOW-2099]

> Task details cannot be shown when PythonOperator calls a partial function
> -
>
> Key: AIRFLOW-1027
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1027
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: ui
>Affects Versions: Airflow 1.7.1
>Reporter: Adrian Partl
>Assignee: Adrian Partl
>Priority: Minor
>
> Showing task details of a PythonOperator that uses a `functools.partial` as a 
> callable results in the following error:
> {noformat}
>   File "/usr/lib/python2.7/site-packages/airflow/www/views.py", line 909, in 
> task
> special_attrs_rendered[attr_name] = attr_renderer[attr_name](source)
>   File "/usr/lib/python2.7/site-packages/airflow/www/views.py", line 224, in 
> 
> inspect.getsource(x), lexers.PythonLexer),
>   File "/usr/lib64/python2.7/inspect.py", line 701, in getsource
> lines, lnum = getsourcelines(object)
>   File "/usr/lib64/python2.7/inspect.py", line 690, in getsourcelines
> lines, lnum = findsource(object)
>   File "/usr/lib64/python2.7/inspect.py", line 526, in findsource
> file = getfile(object)
>   File "/usr/lib64/python2.7/inspect.py", line 420, in getfile
> 'function, traceback, frame, or code object'.format(object))
> TypeError:  is not a module, class, 
> method, function, traceback, frame, or code object
> {noformat}
> A sample dag definition for this is:
> {noformat}
> def func_with_two_args(arg_1, arg_2):
> pass
> partial_func = functools.partial(func_with_two_args, arg_1=1)
> dag = DAG(dag_id='test_issue_1333_dag', default_args=default_args)
> dag_task1 = PythonOperator(
> task_id='test_dagrun_functool_partial',
> dag=dag,
> python_callable=partial_func)
> {noformat}
> I will provide a PR with a fix for this.



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