You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2020/10/23 12:12:15 UTC

[GitHub] [airflow] turbaszek opened a new pull request #11778: Use default view in dr op link

turbaszek opened a new pull request #11778:
URL: https://github.com/apache/airflow/pull/11778


   
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in [UPDATING.md](https://github.com/apache/airflow/blob/master/UPDATING.md).
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on pull request #11778: Use default view in dr op link

Posted by GitBox <gi...@apache.org>.
kaxil commented on pull request #11778:
URL: https://github.com/apache/airflow/pull/11778#issuecomment-721444944


   Can you please rebased your PR on latest Master since we have applied [Black](https://github.com/apache/airflow/commit/4e8f9cc8d02b29c325b8a5a76b4837671bdf5f68) and [PyUpgrade](https://github.com/apache/airflow/commit/8c42cf1b00c90f0d7f11b8a3a455381de8e003c5) on Master.
   
   It will help if your squash your commits into single commit first so that there are less conflicts.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on a change in pull request #11778: Use default view in dr op link

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11778:
URL: https://github.com/apache/airflow/pull/11778#discussion_r511183733



##########
File path: airflow/operators/dagrun_operator.py
##########
@@ -38,7 +39,8 @@ class TriggerDagRunLink(BaseOperatorLink):
     name = 'Triggered DAG'
 
     def get_link(self, operator, dttm):
-        return f"/graph?dag_id={operator.trigger_dag_id}&root=&execution_date={quote(dttm.isoformat())}"
+        view = conf.get('webserver', 'dag_default_view').lower()
+        return f"/{view}?dag_id={operator.trigger_dag_id}&root=&execution_date={quote(dttm.isoformat())}"

Review comment:
       When Airflow is installed at `http://localhost:8080/my-company/airflow,` users will redirect to `http://localhost:8080/graph` instead of `http://localhost:8080/my-company/airflow/graph`. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on pull request #11778: Use default view in dr op link

Posted by GitBox <gi...@apache.org>.
turbaszek commented on pull request #11778:
URL: https://github.com/apache/airflow/pull/11778#issuecomment-717299031


   I have no idea why but after those changes I'm unable to run the following DAGs:
   ```
   from airflow import DAG
   from airflow.operators.dagrun_operator import TriggerDagRunOperator
   from airflow.operators.dummy_operator import DummyOperator
   from airflow.utils.dates import days_ago
   
   with DAG("test_dr", start_date=days_ago(1), schedule_interval=None) as dag:
       tr = TriggerDagRunOperator(task_id="trigger", trigger_dag_id="triggered_dag")
   
   with DAG("triggered_dag", start_date=days_ago(1), schedule_interval=None) as dag2:
       DummyOperator(task_id="test")
   ```
   
   What I do?
   ```
   ./breeze --python=3.8 --backend=postgres
   export AIRFLOW__CORE__EXECUTOR=LocalExecutor
   airflow webserver -w 1 -D
   airflow scheduler
   ```
   Then I unpause tboth DAGs in webui and trigger the "test_dr".
   
   What I get:
   ```
   root@d07e8a7697db:/opt/airflow# airflow scheduler
     ____________       _____________
    ____    |__( )_________  __/__  /________      __
   ____  /| |_  /__  ___/_  /_ __  /_  __ \_ | /| / /
   ___  ___ |  / _  /   _  __/ _  / / /_/ /_ |/ |/ /
    _/_/  |_/_/  /_/    /_/    /_/  \____/____/|__/
   [2020-10-27 14:44:50,759] {scheduler_job.py:1269} INFO - Starting the scheduler
   [2020-10-27 14:44:50,759] {scheduler_job.py:1274} INFO - Processing each file at most -1 times
   [2020-10-27 14:44:50,761] {scheduler_job.py:1296} INFO - Resetting orphaned tasks for active dag runs
   [2020-10-27 14:44:50,806] {dag_processing.py:250} INFO - Launched DagFileProcessorManager with pid: 3458
   [2020-10-27 14:44:50,861] {settings.py:49} INFO - Configured default timezone Timezone('UTC')
   [2020-10-27 14:46:29,369] {scheduler_job.py:973} INFO - 1 tasks up for execution:
   	<TaskInstance: test_dr.trigger 2020-10-27 14:46:28.689412+00:00 [scheduled]>
   [2020-10-27 14:46:29,378] {scheduler_job.py:1007} INFO - Figuring out tasks to run in Pool(name=default_pool) with 128 open slots and 1 task instances ready to be queued
   [2020-10-27 14:46:29,379] {scheduler_job.py:1035} INFO - DAG test_dr has 0/16 running and queued tasks
   [2020-10-27 14:46:29,381] {scheduler_job.py:1088} INFO - Setting the following tasks to queued state:
   	<TaskInstance: test_dr.trigger 2020-10-27 14:46:28.689412+00:00 [scheduled]>
   [2020-10-27 14:46:29,390] {scheduler_job.py:1134} INFO - Sending TaskInstanceKey(dag_id='test_dr', task_id='trigger', execution_date=datetime.datetime(2020, 10, 27, 14, 46, 28, 689412, tzinfo=Timezone('UTC')), try_number=1) to executor with priority 1 and queue default
   [2020-10-27 14:46:29,391] {base_executor.py:78} INFO - Adding to queue: ['airflow', 'tasks', 'run', 'test_dr', 'trigger', '2020-10-27T14:46:28.689412+00:00', '--local', '--pool', 'default_pool', '--subdir', '/files/dags/td_test.py']
   [2020-10-27 14:46:29,392] {sequential_executor.py:57} INFO - Executing command: ['airflow', 'tasks', 'run', 'test_dr', 'trigger', '2020-10-27T14:46:28.689412+00:00', '--local', '--pool', 'default_pool', '--subdir', '/files/dags/td_test.py']
   [2020-10-27 14:46:41,266] {dagbag.py:436} INFO - Filling up the DagBag from /files/dags/td_test.py
   Running <TaskInstance: test_dr.trigger 2020-10-27T14:46:28.689412+00:00 [None]> on host d07e8a7697db
   ```
   But the task is not executed


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #11778: Use default view in dr op link

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #11778:
URL: https://github.com/apache/airflow/pull/11778#discussion_r512725557



##########
File path: airflow/operators/dagrun_operator.py
##########
@@ -38,7 +38,8 @@ class TriggerDagRunLink(BaseOperatorLink):
     name = 'Triggered DAG'
 
     def get_link(self, operator, dttm):
-        return f"/graph?dag_id={operator.trigger_dag_id}&root=&execution_date={quote(dttm.isoformat())}"
+        query = {"dag_id": operator.trigger_dag_id, "root": "", "execution_date": dttm.isoformat()}
+        return build_airflow_url_with_query(query)

Review comment:
       This will work only if inside of application context and I'm not sure if this is necessary
   ```
   RuntimeError: Attempted to generate a URL without the application context being pushed. This has to be executed when application context is available.
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #11778: Use default view in dr op link

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11778:
URL: https://github.com/apache/airflow/pull/11778#issuecomment-715440709


   [The Workflow run](https://github.com/apache/airflow/actions/runs/324432602) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks$,^Build docs$,^Spell check docs$,^Backport packages$,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek merged pull request #11778: Use default view in dr op link

Posted by GitBox <gi...@apache.org>.
turbaszek merged pull request #11778:
URL: https://github.com/apache/airflow/pull/11778


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on a change in pull request #11778: Use default view in dr op link

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11778:
URL: https://github.com/apache/airflow/pull/11778#discussion_r511158522



##########
File path: airflow/operators/dagrun_operator.py
##########
@@ -38,7 +39,8 @@ class TriggerDagRunLink(BaseOperatorLink):
     name = 'Triggered DAG'
 
     def get_link(self, operator, dttm):
-        return f"/graph?dag_id={operator.trigger_dag_id}&root=&execution_date={quote(dttm.isoformat())}"
+        view = conf.get('webserver', 'dag_default_view').lower()
+        return f"/{view}?dag_id={operator.trigger_dag_id}&root=&execution_date={quote(dttm.isoformat())}"

Review comment:
       I'm afraid now it won't work properly if you run Airflow with [``base_url``](https://airflow.readthedocs.io/en/latest/configurations-ref.html#base-url) or [``proxy-fix-prefix``]https://airflow.readthedocs.io/en/latest/configurations-ref.html#proxy-fix-x-prefix] configured.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] astleychen commented on a change in pull request #11778: Use default view in dr op link

Posted by GitBox <gi...@apache.org>.
astleychen commented on a change in pull request #11778:
URL: https://github.com/apache/airflow/pull/11778#discussion_r553369075



##########
File path: airflow/operators/dagrun_operator.py
##########
@@ -38,7 +39,8 @@ class TriggerDagRunLink(BaseOperatorLink):
     name = 'Triggered DAG'
 
     def get_link(self, operator, dttm):
-        return f"/graph?dag_id={operator.trigger_dag_id}&root=&execution_date={quote(dttm.isoformat())}"
+        view = conf.get('webserver', 'dag_default_view').lower()
+        return f"/{view}?dag_id={operator.trigger_dag_id}&root=&execution_date={quote(dttm.isoformat())}"

Review comment:
       Looks like it remains an open issue for v2 when prefixing `/base/` in `base_url`. Is there any issue open here for tracking?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on a change in pull request #11778: Use default view in dr op link

Posted by GitBox <gi...@apache.org>.
turbaszek commented on a change in pull request #11778:
URL: https://github.com/apache/airflow/pull/11778#discussion_r515906811



##########
File path: airflow/operators/dagrun_operator.py
##########
@@ -38,7 +39,8 @@ class TriggerDagRunLink(BaseOperatorLink):
     name = 'Triggered DAG'
 
     def get_link(self, operator, dttm):
-        return f"/graph?dag_id={operator.trigger_dag_id}&root=&execution_date={quote(dttm.isoformat())}"
+        view = conf.get('webserver', 'dag_default_view').lower()
+        return f"/{view}?dag_id={operator.trigger_dag_id}&root=&execution_date={quote(dttm.isoformat())}"

Review comment:
       When using `base_url` in the url and using breeze I'm redirected to `localhost:8080` instead of `0.0.0.0:28080`. So I think we don't have to use it it in url.
   
   When using `url_for` then the link doesn't work at all

##########
File path: airflow/operators/dagrun_operator.py
##########
@@ -38,7 +38,8 @@ class TriggerDagRunLink(BaseOperatorLink):
     name = 'Triggered DAG'
 
     def get_link(self, operator, dttm):
-        return f"/graph?dag_id={operator.trigger_dag_id}&root=&execution_date={quote(dttm.isoformat())}"
+        query = {"dag_id": operator.trigger_dag_id, "root": "", "execution_date": dttm.isoformat()}
+        return build_airflow_url_with_query(query)

Review comment:
       Please see https://github.com/apache/airflow/pull/11778#discussion_r515906811




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on a change in pull request #11778: Use default view in dr op link

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11778:
URL: https://github.com/apache/airflow/pull/11778#discussion_r513030856



##########
File path: airflow/operators/dagrun_operator.py
##########
@@ -38,7 +38,8 @@ class TriggerDagRunLink(BaseOperatorLink):
     name = 'Triggered DAG'
 
     def get_link(self, operator, dttm):
-        return f"/graph?dag_id={operator.trigger_dag_id}&root=&execution_date={quote(dttm.isoformat())}"
+        query = {"dag_id": operator.trigger_dag_id, "root": "", "execution_date": dttm.isoformat()}
+        return build_airflow_url_with_query(query)

Review comment:
       It seems to me that this link will be generated by the webserver, so this is expected behavior.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #11778: Use default view in dr op link

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11778:
URL: https://github.com/apache/airflow/pull/11778#issuecomment-715441003


   [The Workflow run](https://github.com/apache/airflow/actions/runs/324432617) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks$,^Build docs$,^Spell check docs$,^Backport packages$,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on a change in pull request #11778: Use default view in dr op link

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11778:
URL: https://github.com/apache/airflow/pull/11778#discussion_r511157629



##########
File path: airflow/operators/dagrun_operator.py
##########
@@ -38,7 +39,8 @@ class TriggerDagRunLink(BaseOperatorLink):
     name = 'Triggered DAG'
 
     def get_link(self, operator, dttm):
-        return f"/graph?dag_id={operator.trigger_dag_id}&root=&execution_date={quote(dttm.isoformat())}"
+        view = conf.get('webserver', 'dag_default_view').lower()
+        return f"/{view}?dag_id={operator.trigger_dag_id}&root=&execution_date={quote(dttm.isoformat())}"

Review comment:
       Should we add base_url here?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on a change in pull request #11778: Use default view in dr op link

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11778:
URL: https://github.com/apache/airflow/pull/11778#discussion_r512721829



##########
File path: airflow/operators/dagrun_operator.py
##########
@@ -38,7 +38,8 @@ class TriggerDagRunLink(BaseOperatorLink):
     name = 'Triggered DAG'
 
     def get_link(self, operator, dttm):
-        return f"/graph?dag_id={operator.trigger_dag_id}&root=&execution_date={quote(dttm.isoformat())}"
+        query = {"dag_id": operator.trigger_dag_id, "root": "", "execution_date": dttm.isoformat()}
+        return build_airflow_url_with_query(query)

Review comment:
       Can you use url_for here? I guess it should work.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #11778: Use default view in dr op link

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11778:
URL: https://github.com/apache/airflow/pull/11778#issuecomment-720412484


   The PR needs to run all tests because it modifies core of Airflow! Please rebase it to latest master or ask committer to re-run it!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #11778: Use default view in dr op link

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #11778:
URL: https://github.com/apache/airflow/pull/11778#discussion_r512693203



##########
File path: airflow/utils/helpers.py
##########
@@ -203,3 +205,14 @@ def cross_downstream(*args, **kwargs):
         DeprecationWarning, stacklevel=2
     )
     return import_string('airflow.models.baseoperator.cross_downstream')(*args, **kwargs)
+
+
+def build_airflow_url_with_query(query: Dict[str, Any]) -> str:
+    """
+    Build airflow url using base_url and default_view and provided query
+    For example:
+    'http://0.0.0.0:8000/base/graph?dag_id=my-task&root=&execution_date=2020-10-27T10%3A59%3A25.615587
+    """
+    view = conf.get('webserver', 'dag_default_view').lower()
+    base_url = conf.get('webserver', 'base_url')
+    return parse.urljoin(base_url, view) + "?" + parse.urlencode(query)

Review comment:
       nit: (not required) -- no strong opinion on which is better
   
   ```suggestion
       return f"{parse.urljoin(base_url, view)}?{parse.urlencode(query)}"
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] turbaszek commented on pull request #11778: Use default view in dr op link

Posted by GitBox <gi...@apache.org>.
turbaszek commented on pull request #11778:
URL: https://github.com/apache/airflow/pull/11778#issuecomment-723935632


   @kaxil @mik-laj happy to get another round of reviews 😄 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #11778: Use default view in dr op link

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #11778:
URL: https://github.com/apache/airflow/pull/11778#discussion_r512692453



##########
File path: airflow/operators/dagrun_operator.py
##########
@@ -39,8 +38,8 @@ class TriggerDagRunLink(BaseOperatorLink):
     name = 'Triggered DAG'
 
     def get_link(self, operator, dttm):
-        view = conf.get('webserver', 'dag_default_view').lower()
-        return f"/{view}?dag_id={operator.trigger_dag_id}&root=&execution_date={quote(dttm.isoformat())}"
+        query = {"dag_id": operator.trigger_dag_id, "root": "", "execution_date": dttm.isoformat()}

Review comment:
       Do we need to specify `"root": ""` everytime if it is blank.
   
   If so maybe just add it to `build_airflow_url_with_query` to add `"root": ""` if `root` key isn't present in the dict (query).
   
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #11778: Use default view in dr op link

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11778:
URL: https://github.com/apache/airflow/pull/11778#issuecomment-721628691


   [The Workflow run](https://github.com/apache/airflow/actions/runs/345278599) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org