[jira] [Created] (AIRFLOW-1503) AssertionError: INTERNAL: No default project is specified

2017-08-10 Thread chaitanya (JIRA)
chaitanya created AIRFLOW-1503:
--

 Summary: AssertionError: INTERNAL: No default project is specified
 Key: AIRFLOW-1503
 URL: https://issues.apache.org/jira/browse/AIRFLOW-1503
 Project: Apache Airflow
  Issue Type: Bug
  Components: gcp
Affects Versions: Airflow 1.8
 Environment: Unix platform
Reporter: chaitanya
Priority: Minor


Hi ,

New to airflow. Tried to run BigQuery query and store the result in another 
table. Getting the following error. 

Please let me know where to default project. 

Code: 

sql_bigquery = BigQueryOperator(
task_id='sql_bigquery',
use_legacy_sql=False,
write_disposition='WRITE_TRUNCATE',
allow_large_results=True,
bql='''
#standardSQL
SELECT ID, Name, Group, Mark, RATIO_TO_REPORT(Mark) 
OVER(PARTITION BY Group) AS percent FROM `tensile-site-168620.temp.marks`
''',
destination_dataset_table='temp.percentage',
dag=dag
)

Error Message: 
Traceback (most recent call last):
  File "/usr/local/bin/airflow", line 28, in 
args.func(args)
  File "/usr/local/lib/python2.7/dist-packages/airflow/bin/cli.py", line 585, 
in test
ti.run(ignore_task_deps=True, ignore_ti_state=True, test_mode=True)
  File "/usr/local/lib/python2.7/dist-packages/airflow/utils/db.py", line 53, 
in wrapper
result = func(*args, **kwargs)
  File "/usr/local/lib/python2.7/dist-packages/airflow/models.py", line 1374, 
in run
result = task_copy.execute(context=context)
  File 
"/usr/local/lib/python2.7/dist-packages/airflow/contrib/operators/bigquery_operator.py",
 line 82, in execute
self.allow_large_results, self.udf_config, self.use_legacy_sql)
  File 
"/usr/local/lib/python2.7/dist-packages/airflow/contrib/hooks/bigquery_hook.py",
 line 228, in run_query
default_project_id=self.project_id)
  File 
"/usr/local/lib/python2.7/dist-packages/airflow/contrib/hooks/bigquery_hook.py",
 line 917, in _split_tablename
assert default_project_id is not None, "INTERNAL: No default project is 
specified"
AssertionError: INTERNAL: No default project is specified



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (AIRFLOW-1355) Unable to launch jobs : DAGs not being executed.

2017-08-10 Thread Pavan KN (JIRA)

 [ 
https://issues.apache.org/jira/browse/AIRFLOW-1355?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Pavan KN updated AIRFLOW-1355:
--
Priority: Critical  (was: Major)

> Unable to launch jobs : DAGs not being executed.
> 
>
> Key: AIRFLOW-1355
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1355
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: DAG
>Affects Versions: Airflow 1.8
> Environment: Mac OS and Ubuntu
>Reporter: Pavan KN
>Priority: Critical
>
> Steps to re-produce:
> 1. Create new installation
> 2. Launch Airflow
> 3. Enable a DAG and trigger it manually
> DAG/Job won't get executed. Will stay in Running status, but no execution 
> starts and continues to stay at same status.
> Same issues are there with Sequential, Local and Celeri executors.
> Happening in 1.8 version. Tried on multiple Mac machines and on Ubuntu.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (AIRFLOW-940) Failing to unencrypt one variable shouldn't break the admin/variable page

2017-08-10 Thread ASF subversion and git services (JIRA)

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

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

Commit 0bc248fc7aa51f5890e550a63fcb28eb427f55a3 in incubator-airflow's branch 
refs/heads/master from [~erod]
[ https://git-wip-us.apache.org/repos/asf?p=incubator-airflow.git;h=0bc248f ]

[AIRFLOW-940] Handle error on variable decrypt

Invalid variables could break the variable view by
unhandled
InvalidToken exception from Fernet.

This commit converts the fernet error into an
AirflowException, given
that fernet is loaded dynamically. Also, the
exception is handled in
the VariableView by showing the token "INVALID" in
the UI render.

Closes #2510 from edgarRd/erod-error-handling-var-
decrypt


> Failing to unencrypt one variable shouldn't break the admin/variable page
> -
>
> Key: AIRFLOW-940
> URL: https://issues.apache.org/jira/browse/AIRFLOW-940
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: webserver
>Reporter: Dan Davydov
>Assignee: Edgar Rodriguez
>  Labels: beginner, starter
>
> At the moment failing to decrypt one of the airflow variables in the database 
> using the fernet key will cause the admin/variable page to crash. Instead 
> there should be an error message for only the broken variables at the top of 
> the page, and the other ones should still be loaded and displayed.
> This is what the trace on admin/variable currently looks like:
> {code}
> Traceback (most recent call last):
>   File "/usr/local/lib/python2.7/dist-packages/flask/app.py", line 1817, in 
> wsgi_app
> response = self.full_dispatch_request()
>   File "/usr/local/lib/python2.7/dist-packages/flask/app.py", line 1477, in 
> full_dispatch_request
> rv = self.handle_user_exception(e)
>   File "/usr/local/lib/python2.7/dist-packages/flask/app.py", line 1381, in 
> handle_user_exception
> reraise(exc_type, exc_value, tb)
>   File "/usr/local/lib/python2.7/dist-packages/flask/app.py", line 1475, in 
> full_dispatch_request
> rv = self.dispatch_request()
>   File "/usr/local/lib/python2.7/dist-packages/flask/app.py", line 1461, in 
> dispatch_request
> return self.view_functions[rule.endpoint](**req.view_args)
>   File "/usr/local/lib/python2.7/dist-packages/flask_admin/base.py", line 68, 
> in inner
> return self._run_view(f, *args, **kwargs)
>   File "/usr/local/lib/python2.7/dist-packages/flask_admin/base.py", line 
> 367, in _run_view
> return fn(self, *args, **kwargs)
>   File "/usr/local/lib/python2.7/dist-packages/flask_admin/model/base.py", 
> line 1814, in index_view
> return_url=self._get_list_url(view_args),
>   File "/usr/local/lib/python2.7/dist-packages/flask_admin/base.py", line 
> 307, in render
> return render_template(template, **kwargs)
>   File "/usr/local/lib/python2.7/dist-packages/flask/templating.py", line 
> 128, in render_template
> context, ctx.app)
>   File "/usr/local/lib/python2.7/dist-packages/flask/templating.py", line 
> 110, in _render
> rv = template.render(context)
>   File "/usr/local/lib/python2.7/dist-packages/jinja2/environment.py", line 
> 989, in render
> return self.environment.handle_exception(exc_info, True)
>   File "/usr/local/lib/python2.7/dist-packages/jinja2/environment.py", line 
> 754, in handle_exception
> reraise(exc_type, exc_value, tb)
>   File 
> "/usr/local/lib/python2.7/dist-packages/airflow/www/templates/airflow/variable_list.html",
>  line 18, in top-level template code
> {% extends 'admin/model/list.html' %}
>   File 
> "/usr/local/lib/python2.7/dist-packages/flask_admin/templates/bootstrap3/admin/model/list.html",
>  line 5, in top-level template code
> {% import 'admin/actions.html' as actionlib with context %}
>   File 
> "/usr/local/lib/python2.7/dist-packages/airflow/www/templates/admin/master.html",
>  line 18, in top-level template code
> {% extends 'admin/base.html' %}
>   File 
> "/usr/local/lib/python2.7/dist-packages/flask_admin/templates/bootstrap3/admin/base.html",
>  line 30, in top-level template code
> {% block page_body %}
>   File 
> "/usr/local/lib/python2.7/dist-packages/airflow/www/templates/admin/master.html",
>  line 96, in block "page_body"
> {% block body %}
>   File 
> "/usr/local/lib/python2.7/dist-packages/flask_admin/templates/bootstrap3/admin/model/list.html",
>  line 62, in block "body"
> {% block model_list_table %}
>   File 
> "/usr/local/lib/python2.7/dist-packages/flask_admin/templates/bootstrap3/admin/model/list.html",
>  line 110, in block "model_list_table"
> {% block list_row scoped %}
>   File 
> "/usr/local/lib/python2.7/dist-packages/flask_admin/templates/bootstrap3/admin/model/list.html",
>  line 165, 

incubator-airflow git commit: [AIRFLOW-940] Handle error on variable decrypt

2017-08-10 Thread saguziel
Repository: incubator-airflow
Updated Branches:
  refs/heads/master fa84d4999 -> 0bc248fc7


[AIRFLOW-940] Handle error on variable decrypt

Invalid variables could break the variable view by
unhandled
InvalidToken exception from Fernet.

This commit converts the fernet error into an
AirflowException, given
that fernet is loaded dynamically. Also, the
exception is handled in
the VariableView by showing the token "INVALID" in
the UI render.

Closes #2510 from edgarRd/erod-error-handling-var-
decrypt


Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/0bc248fc
Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/0bc248fc
Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/0bc248fc

Branch: refs/heads/master
Commit: 0bc248fc7aa51f5890e550a63fcb28eb427f55a3
Parents: fa84d49
Author: Edgar Rodriguez 
Authored: Thu Aug 10 17:26:05 2017 -0700
Committer: Alex Guziel 
Committed: Thu Aug 10 17:26:05 2017 -0700

--
 airflow/models.py   |  7 +-
 airflow/www/views.py|  5 +++-
 tests/www/test_views.py | 58 
 3 files changed, 68 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/0bc248fc/airflow/models.py
--
diff --git a/airflow/models.py b/airflow/models.py
index 35a09ef..c15491c 100755
--- a/airflow/models.py
+++ b/airflow/models.py
@@ -3831,7 +3831,12 @@ class Variable(Base):
 raise AirflowException(
 "Can't decrypt _val for key={}, FERNET_KEY configuration \
 missing".format(self.key))
-return fernet.decrypt(bytes(self._val, 'utf-8')).decode()
+try:
+return fernet.decrypt(bytes(self._val, 'utf-8')).decode()
+except:
+raise AirflowException(
+"Can't decrypt _val for key={}, invalid token or value"
+.format(self.key))
 else:
 return self._val
 

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/0bc248fc/airflow/www/views.py
--
diff --git a/airflow/www/views.py b/airflow/www/views.py
index 07e0e18..f8dffb9 100644
--- a/airflow/www/views.py
+++ b/airflow/www/views.py
@@ -2283,7 +2283,10 @@ class VariableView(wwwutils.DataProfilingMixin, 
AirflowModelView):
 def hidden_field_formatter(view, context, model, name):
 if wwwutils.should_hide_value_for_key(model.key):
 return Markup('*' * 8)
-return getattr(model, name)
+try:
+return getattr(model, name)
+except AirflowException:
+return Markup('Invalid')
 
 form_columns = (
 'key',

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/0bc248fc/tests/www/test_views.py
--
diff --git a/tests/www/test_views.py b/tests/www/test_views.py
index 5c0eefa..f20dca1 100644
--- a/tests/www/test_views.py
+++ b/tests/www/test_views.py
@@ -82,6 +82,64 @@ class TestChartModelView(unittest.TestCase):
 self.assertIn('Sort by Owner', response.data.decode('utf-8'))
 
 
+class TestVariableView(unittest.TestCase):
+
+CREATE_ENDPOINT = '/admin/variable/new/?url=/admin/variable/'
+
+@classmethod
+def setUpClass(cls):
+super(TestVariableView, cls).setUpClass()
+session = Session()
+session.query(models.Variable).delete()
+session.commit()
+session.close()
+
+def setUp(self):
+super(TestVariableView, self).setUp()
+configuration.load_test_config()
+app = application.create_app(testing=True)
+app.config['WTF_CSRF_METHODS'] = []
+self.app = app.test_client()
+self.session = Session()
+self.variable = {
+'key': 'test_key',
+'val': 'text_val',
+'is_encrypted': True
+}
+
+def tearDown(self):
+self.session.query(models.Variable).delete()
+self.session.commit()
+self.session.close()
+super(TestVariableView, self).tearDown()
+
+def test_can_handle_error_on_decrypt(self):
+# create valid variable
+response = self.app.post(
+self.CREATE_ENDPOINT,
+data=self.variable,
+follow_redirects=True,
+)
+self.assertEqual(response.status_code, 200)
+
+# update the variable with a wrong value, given that is encrypted
+Var = models.Variable
+(self.session.query(Var)
+.filter(Var.key == self.variable['key'])
+.update({
+   

[jira] [Commented] (AIRFLOW-1482) Error when try to backfill the example_trigger_controller_dag

2017-08-10 Thread Lily Chang (JIRA)

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

Lily Chang commented on AIRFLOW-1482:
-

We ran into the same "Executor reports task instance finished (XXX) although 
the task says its XXX. Was the task killed externally?" using airflow 1.8.0, 
celery 4.0.2, redis==2.10.5. Jobs scheduled, backfilled, or via airflow run can 
fail but if I keep rerun the jobs, that might get through at some point. Tried 
resetdb and still not working. Any idea what could cause this problem?

> Error when try to backfill the example_trigger_controller_dag
> -
>
> Key: AIRFLOW-1482
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1482
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: backfill
>Affects Versions: 1.8.1, 1.8.2
> Environment: Ubuntu: 16.04
> Python: 2.7
> CeleryExecutor
> Broker: Redis
>Reporter: Timothee N
>Priority: Blocker
> Attachments: airflow_1.png, airflow_2.png, airflow_3.png
>
>
> Hello,
> Running a backfill command for the 
> {noformat}example_trigger_controller_dag{noformat} example dag, result in the 
> failed task {noformat}test_trigger_dagrun{noformat}
> It seems to me that the problem comes from the TriggerDagRunOperator in the 
> example_trigger_controller_dag ?
> Backfill command: {noformat}airflow backfill -s 2017-07-10 -e 2017-07-13 
> --pool backfill example_trigger_controller_dag{noformat}
> Tested in 1.8.1 and 1.8.2rc1
> Here is the output log from the backfill command :
> {noformat}
> [2017-08-02 13:53:00,844] {__init__.py:57} INFO - Using executor 
> CeleryExecutor
> [2017-08-02 13:53:00,888] {driver.py:120} INFO - Generating grammar tables 
> from /usr/lib/python2.7/lib2to3/Grammar.txt
> [2017-08-02 13:53:00,902] {driver.py:120} INFO - Generating grammar tables 
> from /usr/lib/python2.7/lib2to3/PatternGrammar.txt
> /var/lib/airflow/local/lib/python2.7/site-packages/airflow/www/app.py:23: 
> FlaskWTFDeprecationWarning: "flask_wtf.CsrfProtect" has been renamed to 
> "CSRFProtect" and will be removed in 1.0.
>   csrf = CsrfProtect()
> [2017-08-02 13:53:01,033] {models.py:168} INFO - Filling up the DagBag from 
> /var/lib/airflow/dags
> [2017-08-02 13:53:01,332] {models.py:1128} INFO - Dependencies all met for 
>  00:00:00 [scheduled]>
> [2017-08-02 13:53:01,337] {base_executor.py:50} INFO - Adding to queue: 
> airflow run example_trigger_controller_dag test_trigger_dagrun 
> 2017-07-10T00:00:00 --pickle 1 --local --pool backfill
> [2017-08-02 13:53:06,267] {celery_executor.py:81} INFO - [celery] queuing 
> (u'example_trigger_controller_dag', u'test_trigger_dagrun', 
> datetime.datetime(2017, 7, 10, 0, 0)) through celery, queue=default
> [2017-08-02 13:53:06,330] {models.py:4164} INFO - Updating state for  example_trigger_controller_dag @ 2017-07-10 00:00:00: 
> backfill_2017-07-10T00:00:00, externally triggered: False> considering 1 
> task(s)
> [2017-08-02 13:53:06,334] {jobs.py:2020} INFO - [backfill progress] | 
> finished run 0 of 1 | tasks waiting: 0 | succeeded: 0 | kicked_off: 1 | 
> failed: 0 | skipped: 0 | deadlocked: 0 | not ready: 0
> [2017-08-02 13:53:11,273] {jobs.py:1743} ERROR - Executor reports task 
> instance  2017-07-10 00:00:00 [queued]> finished (failed) although the task says its 
> queued. Was the task killed externally?
> [2017-08-02 13:53:11,273] {models.py:1433} ERROR - Executor reports task 
> instance  2017-07-10 00:00:00 [queued]> finished (failed) although the task says its 
> queued. Was the task killed externally?
> None
> [2017-08-02 13:53:11,273] {models.py:1457} INFO - Marking task as FAILED.
> [2017-08-02 13:53:11,279] {models.py:1478} ERROR - Executor reports task 
> instance  2017-07-10 00:00:00 [queued]> finished (failed) although the task says its 
> queued. Was the task killed externally?
> [2017-08-02 13:53:11,281] {jobs.py:1694} ERROR - Task instance  example_trigger_controller_dag.test_trigger_dagrun 2017-07-10 00:00:00 
> [failed]> failed
> [2017-08-02 13:53:11,283] {models.py:4164} INFO - Updating state for  example_trigger_controller_dag @ 2017-07-10 00:00:00: 
> backfill_2017-07-10T00:00:00, externally triggered: False> considering 1 
> task(s)
> [2017-08-02 13:53:11,285] {models.py:4204} INFO - Marking run  example_trigger_controller_dag @ 2017-07-10 00:00:00: 
> backfill_2017-07-10T00:00:00, externally triggered: False> failed
> [2017-08-02 13:53:11,298] {jobs.py:2020} INFO - [backfill progress] | 
> finished run 1 of 1 | tasks waiting: 0 | succeeded: 0 | kicked_off: 0 | 
> failed: 1 | skipped: 0 | deadlocked: 0 | not ready: 0
> Traceback (most recent call last):
>   File "/var/lib/airflow/bin/airflow", line 28, in 
> args.func(args)
>   File 
> "/var/lib/airflow/local/lib/python2.7/site-packages/airflow/bin/cli.py", 

[jira] [Assigned] (AIRFLOW-1397) Airflow 1.8.1 - No data displays in Last Run Column in Airflow UI

2017-08-10 Thread Ace Haidrey (JIRA)

 [ 
https://issues.apache.org/jira/browse/AIRFLOW-1397?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ace Haidrey reassigned AIRFLOW-1397:


Assignee: user_airflow  (was: Ace Haidrey)

> Airflow 1.8.1 - No data displays in Last Run Column in Airflow UI
> -
>
> Key: AIRFLOW-1397
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1397
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: DAG, ui
>Affects Versions: 1.8.1
>Reporter: user_airflow
>Assignee: user_airflow
>Priority: Critical
>
> Recently upgraded Airflow version from 1.8.0 to 1.8.1. After upgrading, the 
> Last Run column in Airflow UI started showing as Blank for all the existing 
> dags.
> Created a pr for this bug: 
> https://github.com/apache/incubator-airflow/pull/2430



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Assigned] (AIRFLOW-1397) Airflow 1.8.1 - No data displays in Last Run Column in Airflow UI

2017-08-10 Thread Ace Haidrey (JIRA)

 [ 
https://issues.apache.org/jira/browse/AIRFLOW-1397?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ace Haidrey reassigned AIRFLOW-1397:


Assignee: Ace Haidrey  (was: user_airflow)

> Airflow 1.8.1 - No data displays in Last Run Column in Airflow UI
> -
>
> Key: AIRFLOW-1397
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1397
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: DAG, ui
>Affects Versions: 1.8.1
>Reporter: user_airflow
>Assignee: Ace Haidrey
>Priority: Critical
>
> Recently upgraded Airflow version from 1.8.0 to 1.8.1. After upgrading, the 
> Last Run column in Airflow UI started showing as Blank for all the existing 
> dags.
> Created a pr for this bug: 
> https://github.com/apache/incubator-airflow/pull/2430



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (AIRFLOW-1397) Airflow 1.8.1 - No data displays in Last Run Column in Airflow UI

2017-08-10 Thread Ace Haidrey (JIRA)

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

Ace Haidrey commented on AIRFLOW-1397:
--

this ticket can be closed!

> Airflow 1.8.1 - No data displays in Last Run Column in Airflow UI
> -
>
> Key: AIRFLOW-1397
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1397
> Project: Apache Airflow
>  Issue Type: Bug
>  Components: DAG, ui
>Affects Versions: 1.8.1
>Reporter: user_airflow
>Assignee: user_airflow
>Priority: Critical
>
> Recently upgraded Airflow version from 1.8.0 to 1.8.1. After upgrading, the 
> Last Run column in Airflow UI started showing as Blank for all the existing 
> dags.
> Created a pr for this bug: 
> https://github.com/apache/incubator-airflow/pull/2430



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (AIRFLOW-1502) Implement possibility to search dag runs via experimental api

2017-08-10 Thread Stanislav Kudriashev (JIRA)

 [ 
https://issues.apache.org/jira/browse/AIRFLOW-1502?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Stanislav Kudriashev updated AIRFLOW-1502:
--
Description: 
Implement possibility to search dag runs via experimental api.

*Format:*
{code}
http://localhost:8080/api/experimental/dag_runs/search/
{code}

*Request:*
{code}
http://localhost:8080/api/experimental/dag_runs/search/scheduled
{code}

*Response:*
{code}
{
  "items": [
{
  "dag_id": "trigger_1", 
  "dag_run_url": 
"/admin/airflow/graph?execution_date=2017-08-10+09%3A03%3A00_id=trigger_1", 
  "execution_date": "2017-08-10 09:03", 
  "run_id": "scheduled__2017-08-10T09:03:00", 
  "start_date": "2017-08-10 12:02", 
  "state": "running"
}, 
{
  "dag_id": "trigger_1", 
  "dag_run_url": 
"/admin/airflow/graph?execution_date=2017-08-10+09%3A04%3A00_id=trigger_1", 
  "execution_date": "2017-08-10 09:04", 
  "run_id": "scheduled__2017-08-10T09:04:00", 
  "start_date": "2017-08-10 12:02", 
  "state": "running"
}
  ]
}
{code}

  was:
Implement possibility to search dag runs via experimental api.

*Request:*
{code}
http://localhost:8080/api/experimental/dag_runs/search/scheduled
{code}

*Response:*
{code}
{
  "items": [
{
  "dag_id": "trigger_1", 
  "dag_run_url": 
"/admin/airflow/graph?execution_date=2017-08-10+09%3A03%3A00_id=trigger_1", 
  "execution_date": "2017-08-10 09:03", 
  "run_id": "scheduled__2017-08-10T09:03:00", 
  "start_date": "2017-08-10 12:02", 
  "state": "running"
}, 
{
  "dag_id": "trigger_1", 
  "dag_run_url": 
"/admin/airflow/graph?execution_date=2017-08-10+09%3A04%3A00_id=trigger_1", 
  "execution_date": "2017-08-10 09:04", 
  "run_id": "scheduled__2017-08-10T09:04:00", 
  "start_date": "2017-08-10 12:02", 
  "state": "running"
}
  ]
}
{code}


> Implement possibility to search dag runs via experimental api
> -
>
> Key: AIRFLOW-1502
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1502
> Project: Apache Airflow
>  Issue Type: Improvement
>  Components: api
>Affects Versions: 1.8.0
>Reporter: Stanislav Kudriashev
>Assignee: Stanislav Kudriashev
> Fix For: 1.8.1
>
>
> Implement possibility to search dag runs via experimental api.
> *Format:*
> {code}
> http://localhost:8080/api/experimental/dag_runs/search/
> {code}
> *Request:*
> {code}
> http://localhost:8080/api/experimental/dag_runs/search/scheduled
> {code}
> *Response:*
> {code}
> {
>   "items": [
> {
>   "dag_id": "trigger_1", 
>   "dag_run_url": 
> "/admin/airflow/graph?execution_date=2017-08-10+09%3A03%3A00_id=trigger_1",
>  
>   "execution_date": "2017-08-10 09:03", 
>   "run_id": "scheduled__2017-08-10T09:03:00", 
>   "start_date": "2017-08-10 12:02", 
>   "state": "running"
> }, 
> {
>   "dag_id": "trigger_1", 
>   "dag_run_url": 
> "/admin/airflow/graph?execution_date=2017-08-10+09%3A04%3A00_id=trigger_1",
>  
>   "execution_date": "2017-08-10 09:04", 
>   "run_id": "scheduled__2017-08-10T09:04:00", 
>   "start_date": "2017-08-10 12:02", 
>   "state": "running"
> }
>   ]
> }
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Work started] (AIRFLOW-1502) Implement possibility to search dag runs via experimental api

2017-08-10 Thread Stanislav Kudriashev (JIRA)

 [ 
https://issues.apache.org/jira/browse/AIRFLOW-1502?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Work on AIRFLOW-1502 started by Stanislav Kudriashev.
-
> Implement possibility to search dag runs via experimental api
> -
>
> Key: AIRFLOW-1502
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1502
> Project: Apache Airflow
>  Issue Type: Improvement
>  Components: api
>Affects Versions: 1.8.0
>Reporter: Stanislav Kudriashev
>Assignee: Stanislav Kudriashev
> Fix For: 1.8.1
>
>
> Implement possibility to search dag runs via experimental api.
> *Request:*
> {code}
> http://localhost:8080/api/experimental/dag_runs/search/scheduled
> {code}
> *Response:*
> {code}
> {
>   "items": [
> {
>   "dag_id": "trigger_1", 
>   "dag_run_url": 
> "/admin/airflow/graph?execution_date=2017-08-10+09%3A03%3A00_id=trigger_1",
>  
>   "execution_date": "2017-08-10 09:03", 
>   "run_id": "scheduled__2017-08-10T09:03:00", 
>   "start_date": "2017-08-10 12:02", 
>   "state": "running"
> }, 
> {
>   "dag_id": "trigger_1", 
>   "dag_run_url": 
> "/admin/airflow/graph?execution_date=2017-08-10+09%3A04%3A00_id=trigger_1",
>  
>   "execution_date": "2017-08-10 09:04", 
>   "run_id": "scheduled__2017-08-10T09:04:00", 
>   "start_date": "2017-08-10 12:02", 
>   "state": "running"
> }
>   ]
> }
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (AIRFLOW-1502) Implement possibility to search dag runs via experimental api

2017-08-10 Thread Stanislav Kudriashev (JIRA)

 [ 
https://issues.apache.org/jira/browse/AIRFLOW-1502?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Stanislav Kudriashev updated AIRFLOW-1502:
--
Description: 
Implement possibility to search dag runs via experimental api.

*Format:*
{code}
http://localhost:8080/api/experimental/dag_runs/search/
{code}

*Example:*
{code}
http://localhost:8080/api/experimental/dag_runs/search/scheduled
{code}

*Response:*
{code}
{
  "items": [
{
  "dag_id": "trigger_1", 
  "dag_run_url": 
"/admin/airflow/graph?execution_date=2017-08-10+09%3A03%3A00_id=trigger_1", 
  "execution_date": "2017-08-10 09:03", 
  "run_id": "scheduled__2017-08-10T09:03:00", 
  "start_date": "2017-08-10 12:02", 
  "state": "running"
}, 
{
  "dag_id": "trigger_1", 
  "dag_run_url": 
"/admin/airflow/graph?execution_date=2017-08-10+09%3A04%3A00_id=trigger_1", 
  "execution_date": "2017-08-10 09:04", 
  "run_id": "scheduled__2017-08-10T09:04:00", 
  "start_date": "2017-08-10 12:02", 
  "state": "running"
}
  ]
}
{code}

  was:
Implement possibility to search dag runs via experimental api.

*Format:*
{code}
http://localhost:8080/api/experimental/dag_runs/search/
{code}

*Request:*
{code}
http://localhost:8080/api/experimental/dag_runs/search/scheduled
{code}

*Response:*
{code}
{
  "items": [
{
  "dag_id": "trigger_1", 
  "dag_run_url": 
"/admin/airflow/graph?execution_date=2017-08-10+09%3A03%3A00_id=trigger_1", 
  "execution_date": "2017-08-10 09:03", 
  "run_id": "scheduled__2017-08-10T09:03:00", 
  "start_date": "2017-08-10 12:02", 
  "state": "running"
}, 
{
  "dag_id": "trigger_1", 
  "dag_run_url": 
"/admin/airflow/graph?execution_date=2017-08-10+09%3A04%3A00_id=trigger_1", 
  "execution_date": "2017-08-10 09:04", 
  "run_id": "scheduled__2017-08-10T09:04:00", 
  "start_date": "2017-08-10 12:02", 
  "state": "running"
}
  ]
}
{code}


> Implement possibility to search dag runs via experimental api
> -
>
> Key: AIRFLOW-1502
> URL: https://issues.apache.org/jira/browse/AIRFLOW-1502
> Project: Apache Airflow
>  Issue Type: Improvement
>  Components: api
>Affects Versions: 1.8.0
>Reporter: Stanislav Kudriashev
>Assignee: Stanislav Kudriashev
> Fix For: 1.8.1
>
>
> Implement possibility to search dag runs via experimental api.
> *Format:*
> {code}
> http://localhost:8080/api/experimental/dag_runs/search/
> {code}
> *Example:*
> {code}
> http://localhost:8080/api/experimental/dag_runs/search/scheduled
> {code}
> *Response:*
> {code}
> {
>   "items": [
> {
>   "dag_id": "trigger_1", 
>   "dag_run_url": 
> "/admin/airflow/graph?execution_date=2017-08-10+09%3A03%3A00_id=trigger_1",
>  
>   "execution_date": "2017-08-10 09:03", 
>   "run_id": "scheduled__2017-08-10T09:03:00", 
>   "start_date": "2017-08-10 12:02", 
>   "state": "running"
> }, 
> {
>   "dag_id": "trigger_1", 
>   "dag_run_url": 
> "/admin/airflow/graph?execution_date=2017-08-10+09%3A04%3A00_id=trigger_1",
>  
>   "execution_date": "2017-08-10 09:04", 
>   "run_id": "scheduled__2017-08-10T09:04:00", 
>   "start_date": "2017-08-10 12:02", 
>   "state": "running"
> }
>   ]
> }
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (AIRFLOW-1502) Implement possibility to search dag runs via experimental api

2017-08-10 Thread Stanislav Kudriashev (JIRA)
Stanislav Kudriashev created AIRFLOW-1502:
-

 Summary: Implement possibility to search dag runs via experimental 
api
 Key: AIRFLOW-1502
 URL: https://issues.apache.org/jira/browse/AIRFLOW-1502
 Project: Apache Airflow
  Issue Type: Improvement
  Components: api
Affects Versions: 1.8.0
Reporter: Stanislav Kudriashev
Assignee: Stanislav Kudriashev
 Fix For: 1.8.1


Implement possibility to search dag runs via experimental api.

*Request:*
{code}
http://localhost:8080/api/experimental/dag_runs/search/scheduled
{code}

*Response:*
{code}
{
  "items": [
{
  "dag_id": "trigger_1", 
  "dag_run_url": 
"/admin/airflow/graph?execution_date=2017-08-10+09%3A03%3A00_id=trigger_1", 
  "execution_date": "2017-08-10 09:03", 
  "run_id": "scheduled__2017-08-10T09:03:00", 
  "start_date": "2017-08-10 12:02", 
  "state": "running"
}, 
{
  "dag_id": "trigger_1", 
  "dag_run_url": 
"/admin/airflow/graph?execution_date=2017-08-10+09%3A04%3A00_id=trigger_1", 
  "execution_date": "2017-08-10 09:04", 
  "run_id": "scheduled__2017-08-10T09:04:00", 
  "start_date": "2017-08-10 12:02", 
  "state": "running"
}
  ]
}
{code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)