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 2022/07/25 10:09:22 UTC

[GitHub] [airflow] alexkruc opened a new pull request, #25280: Adding support for owner links in the Dags view UI

alexkruc opened a new pull request, #25280:
URL: https://github.com/apache/airflow/pull/25280

   Based on the discussion on #24728, this is the PR to allow the users to add their own hyperlinks to the `owner` property.
   
   The idea behind this PR is as follows - 
   Similar to what we do with DAG tags, I've created a separate table in the DB to store the links with the following schema:`dag_id :: owner_name :: owner_link`. 
    Before I reached this implementation, I tried saving the dict in the current `owners` column, but I faced challenges and was convinced that it was not the right way of implementation. The tradeoffs, as I see them are:
   1. The current `owners` field is a property of a DAG that contains all of them. This is a string column. To add the new implementation there, I had to save as strings the following object: `{owner1: link1}, {owner2:link2}`. This is not native to work with, as it's a list of dicts inside a string column. Also, delimiter problems are also occurring here, and if we change the delimiter of the `owners` field (`,` -> `;` for example), it breaks a lot of the current flow.
   2.  Link length - The links (especially `mailto` links that can contain a message template) can be long. The current `owners` column length is `2000`, and in DAGs with multiple owners, it can be a problem. By moving this logic to its own table, we can manage the link length independently. 
   3. Separation of concerns - Only the `views.py` file and the `DagModel` (for the orm bulk write) needs to be aware of owner links, all the other components can keep getting their `owners` field in the same way they are used to :) 
   
   This PR was tested on the following flows manually using local Airflow started by `breeze`:
   1. Using multiple DBs to verify the schema migration (SQLite, Postgres), started by `--backend` param to `breeze`.
   2. Verified that the new table is getting records added, edited, and deleted based on live DAG edits.
   3. Verified that in the DAGs view page, owners with valid links are transferred into a hyperlink and opened in a new tab for quality of life :) 
   4. Bad formatting of URLs, and potential exploits (calling hyperlink with `javascript:alert(1)` for example), are not getting loaded into the system and thrown out with `AirflowException`.
   5. Added `Owner Links` property to `Dag Details` page that shows the owner links,  and `None` if no links existing in the DAG.
   
   The DAGs I've used to test it (set in the local `files/dags/` folder):
   1. dag_with_owner_links.py
   ```python
   from datetime import datetime, timedelta
   from airflow import DAG
   from airflow.operators.bash import BashOperator
   
   owner_obj = {'name': "test_owner_link", "link": "https://www.google.com/search?q=blabla"}
   
   default_args = {'owner': owner_obj, "start_date": datetime(2021, 9, 9), "retries": 1,
                   'execution_timeout': timedelta(minutes=30)}
   dag = DAG("test_dag_with_links", default_args=default_args, schedule_interval=None,
             catchup=False)
   
   with dag:
       task = BashOperator(task_id='task_with_link', bash_command='echo Hello')
       task2 = BashOperator(task_id='task_with_no_link', bash_command='echo "Hello Again"', owner="bla")
       task3 = BashOperator(task_id='task_with_another_link',
                            bash_command='echo "Hello Again"',
                            owner={'name': "test_owner_email",
                                   'link': "mailto:someone@yoursite.com?subject=Mail from Our Site"})
   ```
   
   2. dag_with_no_links.py
   ```python
   from datetime import datetime, timedelta
   from airflow import DAG
   from airflow.operators.bash import BashOperator
   
   
   default_args = {'owner': "test_owner_link", "start_date": datetime(2021, 9, 9), "retries": 1,
                   'execution_timeout': timedelta(minutes=30)}
   dag = DAG("test_dag_with_no_links", default_args=default_args, schedule_interval=None,
             catchup=False)
   
   with dag:
       task = BashOperator(task_id='hello', bash_command='echo Hello')
       task2 = BashOperator(task_id='hello_again', bash_command='echo "Hello Again"', owner="bla")
       task3 = BashOperator(task_id='hello_again_2',
                            bash_command='echo "Hello Again"',
                            owner="blabla")
   ```
   
   3. bad_link_format.py
   ```python
   from datetime import datetime, timedelta
   from airflow import DAG
   from airflow.operators.bash import BashOperator
   
   owner_obj = {'name': "test_owner_link", "link": "wrong-website-format@bla"}
   
   default_args = {'owner': owner_obj, "start_date": datetime(2021, 9, 9), "retries": 1,
                   'execution_timeout': timedelta(minutes=30)}
   dag = DAG("test_dag_with_bad_formatted_links", default_args=default_args, schedule_interval=None,
             catchup=False)
   
   with dag:
       task = BashOperator(task_id='hello', bash_command='echo Hello')
   ```
   
   4. dag_with_link_explot.py
   ```python
   from datetime import datetime, timedelta
   from airflow import DAG
   from airflow.operators.bash import BashOperator
   
   owner_obj = {'name': "test_owner_link", "link": "javascript:alert(1)"}
   
   default_args = {'owner': owner_obj, "start_date": datetime(2021, 9, 9), "retries": 1,
                   'execution_timeout': timedelta(minutes=30)}
   dag = DAG("test_dag_with_exploited_link", default_args=default_args, schedule_interval=None,
             catchup=False)
   
   with dag:
       task = BashOperator(task_id='hello', bash_command='echo Hello')
   ```
   
   
   closes: #24728
   
   


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] alexkruc commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
alexkruc commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r931907662


##########
docs/apache-airflow/migrations-ref.rst:
##########
@@ -27,7 +27,9 @@ Here's the list of all the Database Migrations that are executed via when you ru
 +---------------------------------+-------------------+-------------------+--------------------------------------------------------------+
 | Revision ID                     | Revises ID        | Airflow Version   | Description                                                  |
 +=================================+===================+===================+==============================================================+
-| ``0038cd0c28b4`` (head)         | ``44b7034f6bdc``  | ``2.4.0``         | Add Dataset model                                            |
+| ``38d6c28f9cf9`` (head)         | ``0038cd0c28b4``  | ``2.4.0``         | Add dag_owner_attributes table.                              |

Review Comment:
   I probably made mistakes with the revisions and the file name of the migration script :)
   Will like someone to double-check me :D



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
potiuk commented on PR #25280:
URL: https://github.com/apache/airflow/pull/25280#issuecomment-1203179734

   Hard to say - did you try to run them locally and debug them ? This is what I do when some tests are failing. This is also a great way to learn how things work. Debugging throught the failing tests related to my changes often uncovers problems that are not obvious and rather than being annoying and delegating analysis of failed tests to others, they are fantastic tool to drag attentions to parts I do not understand when I contribute a change, so I encourage you to dive deeper. 
   
   I do not think development is "done" when you have failing unit tests you do not understand why they are failing after my change.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r939893401


##########
airflow/migrations/versions/0116_2_4_0_add_dag_owner_attributes_table.py:
##########
@@ -0,0 +1,52 @@
+#
+# 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.
+
+"""add dag_owner_attributes table
+
+Revision ID: 1486deb605b4
+Revises: f4ff391becb5
+Create Date: 2022-08-04 16:59:45.406589
+
+"""
+
+import sqlalchemy as sa
+from alembic import op
+
+# revision identifiers, used by Alembic.
+revision = '1486deb605b4'
+down_revision = 'f4ff391becb5'
+branch_labels = None
+depends_on = None
+airflow_version = '2.4.0'
+
+
+def upgrade():
+    """Apply Add ``DagOwnerAttributes`` table"""
+    op.create_table(
+        'dag_owner_attributes',

Review Comment:
   I am attempting to see if we can detect the error here https://github.com/apache/airflow/pull/25590



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
potiuk commented on PR #25280:
URL: https://github.com/apache/airflow/pull/25280#issuecomment-1203810374

   > All unit tests are passing now :) I keep rebasing because new changes are incoming
   
   This means we are alive and kicking :). But I am afraid the last rebase is not pushed yet. 


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] alexkruc commented on pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
alexkruc commented on PR #25280:
URL: https://github.com/apache/airflow/pull/25280#issuecomment-1203808051

   All unit tests are passing now :) I keep rebasing because new changes are incoming 


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] alexkruc commented on pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
alexkruc commented on PR #25280:
URL: https://github.com/apache/airflow/pull/25280#issuecomment-1206306662

   > We should also add some documentation on this feature.
   
   I have added a documentation to the `howto` section :) I hope it's ok :)


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] uranusjr commented on pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
uranusjr commented on PR #25280:
URL: https://github.com/apache/airflow/pull/25280#issuecomment-1204976965

   Personally I’m fine with either approaches, but the one in this PR needs more work to handle edge cases where an owner name is associated to multiple links. From what I can tell, `owner_links` would behave weird if that happens.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r929807063


##########
airflow/models/baseoperator.py:
##########
@@ -16,6 +16,8 @@
 # specific language governing permissions and limitations
 # under the License.
 """Base operator for all operators."""
+from __future__ import annotations

Review Comment:
   We don't use yet __future__ annotations. We discussed that we want to do it, but in order to make cherry-picking easier, we will globally add them just before we release 2.4 to not make cherry-picking more difficult than it should be.
   
   https://lists.apache.org/thread/81fr042s5d3v17v83bpo24tnrr2pp0fp
   
   Can you please remove those for now?



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] uranusjr commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r938431179


##########
airflow/models/dag.py:
##########
@@ -2797,6 +2831,29 @@ def __repr__(self):
         return self.name
 
 
+class DagOwnerAttributes(Base):
+    """
+    Table defining different owner attributes. For example, a link for an owner that will be passed as
+    a hyperlink to the DAGs view
+    """
+
+    __tablename__ = "dag_owner_attributes"
+    dag_id = Column(
+        StringID(),
+        ForeignKey('dag.dag_id', name='dag.dag_id', ondelete='CASCADE'),
+        nullable=False,
+        primary_key=True,
+    )
+    owner = Column(String(100), primary_key=True, nullable=False)

Review Comment:
   We need validation and documentation somewhere to communicate this new 100-character restriction. (Likely not going to affect anyone, but still)



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] alexkruc commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
alexkruc commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r938690062


##########
airflow/models/dag.py:
##########
@@ -2557,6 +2569,14 @@ def bulk_write_to_db(cls, dags: Collection["DAG"], session=NEW_SESSION):
                     orm_dag.tags.append(dag_tag_orm)
                     session.add(dag_tag_orm)
 
+            orm_dag_links = orm_dag.dag_owner_links or []
+            for orm_dag_link in orm_dag_links:
+                if orm_dag_link not in dag.owner_links:
+                    session.delete(orm_dag_link)
+            for owner_name, owner_link in dag.owner_links.items():
+                dag_owner_orm = DagOwnerAttributes(dag_id=dag.dag_id, owner=owner_name, link=owner_link)
+                session.add(dag_owner_orm)

Review Comment:
   Can you elaborate a bit on your comment? I think I didn't fully get your intention :(
   I tried to change it to use `session. bulk_save_objects ` or `session. bulk_insert_mappings`, but it's not going to work because we can't use it because it breaches the foreign key constraint, as when we try to do this, the `dag` table is not yet populated.
   It gets populated in this function, but only when we call `session.flush()` here:
   https://github.com/apache/airflow/blob/main/airflow/models/dag.py#L2623
   
   AFAIK, running `session.add()` multiple times is adding a transaction, but it's "lazy" because nothing happens until we run `flush()`. BTW, this implementation is using the same flow as we use to update the tags, here:
   https://github.com/apache/airflow/blob/main/airflow/models/dag.py#L2547
   So this is why I used it 😅 
   
   Can you please explain a bit more or provide an example of the intention of this comment, so maybe I'll understand it more clearly and be able to add this enhancement?



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] uranusjr commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r938427403


##########
airflow/models/dag.py:
##########
@@ -2557,6 +2569,14 @@ def bulk_write_to_db(cls, dags: Collection["DAG"], session=NEW_SESSION):
                     orm_dag.tags.append(dag_tag_orm)
                     session.add(dag_tag_orm)
 
+            orm_dag_links = orm_dag.dag_owner_links or []
+            for orm_dag_link in orm_dag_links:
+                if orm_dag_link not in dag.owner_links:
+                    session.delete(orm_dag_link)
+            for owner_name, owner_link in dag.owner_links.items():
+                dag_owner_orm = DagOwnerAttributes(dag_id=dag.dag_id, owner=owner_name, link=owner_link)
+                session.add(dag_owner_orm)

Review Comment:
   Hmm this could be a bit inefficient. Can we use `IN` and bulk insert to make this at most two calls? (one for delete and one for insert)



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] eladkal commented on pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
eladkal commented on PR #25280:
URL: https://github.com/apache/airflow/pull/25280#issuecomment-1207164143

   I'm not sure why Up-to-date Checker is failing? The PR is rebased. seems like an issue with the checker code.
   Merging


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r939639980


##########
airflow/migrations/versions/0116_2_4_0_add_dag_owner_attributes_table.py:
##########
@@ -0,0 +1,52 @@
+#
+# 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.
+
+"""add dag_owner_attributes table
+
+Revision ID: 1486deb605b4
+Revises: f4ff391becb5
+Create Date: 2022-08-04 16:59:45.406589
+
+"""
+
+import sqlalchemy as sa
+from alembic import op
+
+# revision identifiers, used by Alembic.
+revision = '1486deb605b4'
+down_revision = 'f4ff391becb5'
+branch_labels = None
+depends_on = None
+airflow_version = '2.4.0'
+
+
+def upgrade():
+    """Apply Add ``DagOwnerAttributes`` table"""
+    op.create_table(
+        'dag_owner_attributes',

Review Comment:
   CC: @alexkruc 



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] uranusjr commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r939467538


##########
docs/apache-airflow/howto/add-owner-links.rst:
##########
@@ -0,0 +1,49 @@
+ .. 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.
+
+
+
+
+Add Owner Links to DAG
+=======================
+
+.. versionadded:: 2.4.0
+
+You can pass the 'owner_link' parameter for your DAG object, which will make the owner to become a clickable link

Review Comment:
   ```suggestion
   You can pass the ``owner_link`` parameter for your DAG object, which will make the owner to become a clickable link
   ```



##########
docs/apache-airflow/howto/add-owner-links.rst:
##########
@@ -0,0 +1,49 @@
+ .. 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.
+
+
+
+
+Add Owner Links to DAG
+=======================
+
+.. versionadded:: 2.4.0
+
+You can pass the 'owner_link' parameter for your DAG object, which will make the owner to become a clickable link
+in the main DAGs view page.
+You can use it to set a custom HTTP link (for example, the owner's Slack channel), or use a
+`maitlo <https://en.wikipedia.org/wiki/Mailto>`_ link to have an automated email message (up to 500 characters).
+
+Example:
+In your Dag file, add a 'owners_link' parameter to the DAG object that will hold a dictionary of an owner and it's link.

Review Comment:
   ```suggestion
   In your DAG file, add a ``owners_link`` parameter to the DAG object that will hold a dictionary of an owner and it's link.
   ```



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] alexkruc commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
alexkruc commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r938683101


##########
airflow/www/views.py:
##########
@@ -882,6 +892,18 @@ def index(self):
                 for name, in dagtags
             ]
 
+            owner_links_dict = {}
+            owner_links = session.query(DagOwnerAttributes).all()
+            # The structure we are going for is:
+            # {dag1: {owner1: link1, owner2: link2}, dag2: {owner1: link1}}
+            for owner_link_pair in owner_links:
+                owner_link_pair_dict = owner_link_pair.as_dict()
+                for dag in owner_link_pair_dict:
+                    if dag in owner_links_dict:
+                        owner_links_dict[dag].update(owner_link_pair_dict[dag])
+                    else:
+                        owner_links_dict.update(owner_link_pair_dict)

Review Comment:
   I changed it and deleted the `as_dict`. Your suggestion is indeed more elegant and better :) Thanks!



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] alexkruc commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
alexkruc commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r939868694


##########
airflow/migrations/versions/0116_2_4_0_add_dag_owner_attributes_table.py:
##########
@@ -0,0 +1,52 @@
+#
+# 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.
+
+"""add dag_owner_attributes table
+
+Revision ID: 1486deb605b4
+Revises: f4ff391becb5
+Create Date: 2022-08-04 16:59:45.406589
+
+"""
+
+import sqlalchemy as sa
+from alembic import op
+
+# revision identifiers, used by Alembic.
+revision = '1486deb605b4'
+down_revision = 'f4ff391becb5'
+branch_labels = None
+depends_on = None
+airflow_version = '2.4.0'
+
+
+def upgrade():
+    """Apply Add ``DagOwnerAttributes`` table"""
+    op.create_table(
+        'dag_owner_attributes',

Review Comment:
   Hey, sorry for causing trouble, and many thanks for opening a PR to fix this!
   I wasn't aware that this can cause issues, but now I know and it's a lesson for next time :) Thanks again! 



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] alexott commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
alexott commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r939641580


##########
airflow/migrations/versions/0116_2_4_0_add_dag_owner_attributes_table.py:
##########
@@ -0,0 +1,52 @@
+#
+# 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.
+
+"""add dag_owner_attributes table
+
+Revision ID: 1486deb605b4
+Revises: f4ff391becb5
+Create Date: 2022-08-04 16:59:45.406589
+
+"""
+
+import sqlalchemy as sa
+from alembic import op
+
+# revision identifiers, used by Alembic.
+revision = '1486deb605b4'
+down_revision = 'f4ff391becb5'
+branch_labels = None
+depends_on = None
+airflow_version = '2.4.0'
+
+
+def upgrade():
+    """Apply Add ``DagOwnerAttributes`` table"""
+    op.create_table(
+        'dag_owner_attributes',

Review Comment:
   I have a fix for it... Will file PR in a minute



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] alexkruc commented on pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
alexkruc commented on PR #25280:
URL: https://github.com/apache/airflow/pull/25280#issuecomment-1199130722

   > Still... some bugs to fix ;)
   
   Yeah, this is why I'm keeping it as a draft for now :( 
   I'm quite struggling with failing tests, I can't understand why some of them fail and how to fix them.. I'm trying to debug it, but if I struggle for much more, I'll probably make a summary of my findings/things I don't understand and ask for further help :(


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] alexkruc commented on pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
alexkruc commented on PR #25280:
URL: https://github.com/apache/airflow/pull/25280#issuecomment-1202237307

   With the help of @eladkal's tips, I was able to fix the bug that caused `test_views_acl` tests to fail.. But I still would really appreciate help understanding what the issues with the failing are `tests/serialization/test_dag_serialization.py` tests :(   
   In any case, besides this, it looks like I'm done with the development, so I've marked it as "ready for review" to get comments :) 


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] alexott commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
alexott commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r939639061


##########
airflow/migrations/versions/0116_2_4_0_add_dag_owner_attributes_table.py:
##########
@@ -0,0 +1,52 @@
+#
+# 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.
+
+"""add dag_owner_attributes table
+
+Revision ID: 1486deb605b4
+Revises: f4ff391becb5
+Create Date: 2022-08-04 16:59:45.406589
+
+"""
+
+import sqlalchemy as sa
+from alembic import op
+
+# revision identifiers, used by Alembic.
+revision = '1486deb605b4'
+down_revision = 'f4ff391becb5'
+branch_labels = None
+depends_on = None
+airflow_version = '2.4.0'
+
+
+def upgrade():
+    """Apply Add ``DagOwnerAttributes`` table"""
+    op.create_table(
+        'dag_owner_attributes',

Review Comment:
   Addition of this table causes the issue when updating because the generated code:
   
   ```sql
   CREATE TABLE dag_owner_attributes (
   	dag_id VARCHAR(250) NOT NULL, 
   	owner VARCHAR(500) NOT NULL, 
   	link VARCHAR(500) NOT NULL, 
   	CONSTRAINT dag_owner_attributes_pkey PRIMARY KEY (dag_id, owner), 
   	CONSTRAINT dag_owner_attributes_dag_id_fkey FOREIGN KEY(dag_id) REFERENCES dag (dag_id) ON DELETE CASCADE
   )
   ```
   
   while to make it working, the `dag_id` needs to be declared as `dag_id VARCHAR(250) CHARACTER SET utf8 COLLATE utf8_bin NOT NULL`
   
   



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] alexkruc commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
alexkruc commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r932249890


##########
docs/apache-airflow/migrations-ref.rst:
##########
@@ -27,7 +27,9 @@ Here's the list of all the Database Migrations that are executed via when you ru
 +---------------------------------+-------------------+-------------------+--------------------------------------------------------------+
 | Revision ID                     | Revises ID        | Airflow Version   | Description                                                  |
 +=================================+===================+===================+==============================================================+
-| ``0038cd0c28b4`` (head)         | ``44b7034f6bdc``  | ``2.4.0``         | Add Dataset model                                            |
+| ``38d6c28f9cf9`` (head)         | ``0038cd0c28b4``  | ``2.4.0``         | Add dag_owner_attributes table.                              |

Review Comment:
   Yep, pre-commit generated this file, and it looks good :)
   What I'm not sure at is the value (that I put) as the revision ID.. I put there the last git revision of the `main` branch when I started working on this feature, but it's no longer relevant, probably :)
   What is the best practice here? Setting the revision of the first commit of this branch? meaning `c449b861c7c1ff8bb02fa7ffe0bee84e90af510e`? And if so - what do you guys do - 12 last chars, or first?
   
   Another question is about the file name of the migration script - I set the version there as 2.4.0, like the one before my added file, but I don't really know what to set as the affected version..



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
potiuk commented on PR #25280:
URL: https://github.com/apache/airflow/pull/25280#issuecomment-1199116141

   Still... some bugs to fix ;)


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] alexkruc commented on pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
alexkruc commented on PR #25280:
URL: https://github.com/apache/airflow/pull/25280#issuecomment-1193850381

   Now that I'm thinking of this, maybe I should rename the `dag_owner_links` table to something like `dag_owner_attributes` -> it sounds more generic if we would like to add more owner attributes in the future.. WDYT?


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] alexkruc commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
alexkruc commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r938684056


##########
airflow/models/dag.py:
##########
@@ -2797,6 +2831,29 @@ def __repr__(self):
         return self.name
 
 
+class DagOwnerAttributes(Base):
+    """
+    Table defining different owner attributes. For example, a link for an owner that will be passed as
+    a hyperlink to the DAGs view
+    """
+
+    __tablename__ = "dag_owner_attributes"
+    dag_id = Column(
+        StringID(),
+        ForeignKey('dag.dag_id', name='dag.dag_id', ondelete='CASCADE'),
+        nullable=False,
+        primary_key=True,
+    )
+    owner = Column(String(100), primary_key=True, nullable=False)

Review Comment:
   I changed it to 500 characters, just as the current Log table defines the owner :) Do you think we still need to add documentation and validation? And if so, does it need to be in this PR, or is it a separate enhancement? 
   https://github.com/apache/airflow/blob/main/airflow/models/log.py#L38



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] uranusjr commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r938433052


##########
airflow/www/views.py:
##########
@@ -882,6 +892,18 @@ def index(self):
                 for name, in dagtags
             ]
 
+            owner_links_dict = {}
+            owner_links = session.query(DagOwnerAttributes).all()
+            # The structure we are going for is:
+            # {dag1: {owner1: link1, owner2: link2}, dag2: {owner1: link1}}
+            for owner_link_pair in owner_links:
+                owner_link_pair_dict = owner_link_pair.as_dict()
+                for dag in owner_link_pair_dict:
+                    if dag in owner_links_dict:
+                        owner_links_dict[dag].update(owner_link_pair_dict[dag])
+                    else:
+                        owner_links_dict.update(owner_link_pair_dict)

Review Comment:
   I wonder if this logic belongs to a method on `DagOwnerAttributes` instead, say
   
   ```python
   @classmethod
   def get_all(cls, session) -> Dict[str, Dict[str, str]]:
       dag_links = collections.defaultdict(dict)
       for obj in session.query(cls):
           dag_links[obj.dag_id].update({obj.owner: obj.link})
       return dag_links
   ```
   
   (does this also mean `as_dict` isn’t needed anymore?)



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r939647467


##########
airflow/migrations/versions/0116_2_4_0_add_dag_owner_attributes_table.py:
##########
@@ -0,0 +1,52 @@
+#
+# 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.
+
+"""add dag_owner_attributes table
+
+Revision ID: 1486deb605b4
+Revises: f4ff391becb5
+Create Date: 2022-08-04 16:59:45.406589
+
+"""
+
+import sqlalchemy as sa
+from alembic import op
+
+# revision identifiers, used by Alembic.
+revision = '1486deb605b4'
+down_revision = 'f4ff391becb5'
+branch_labels = None
+depends_on = None
+airflow_version = '2.4.0'
+
+
+def upgrade():
+    """Apply Add ``DagOwnerAttributes`` table"""
+    op.create_table(
+        'dag_owner_attributes',

Review Comment:
   Merged.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r932239101


##########
docs/apache-airflow/migrations-ref.rst:
##########
@@ -27,7 +27,9 @@ Here's the list of all the Database Migrations that are executed via when you ru
 +---------------------------------+-------------------+-------------------+--------------------------------------------------------------+
 | Revision ID                     | Revises ID        | Airflow Version   | Description                                                  |
 +=================================+===================+===================+==============================================================+
-| ``0038cd0c28b4`` (head)         | ``44b7034f6bdc``  | ``2.4.0``         | Add Dataset model                                            |
+| ``38d6c28f9cf9`` (head)         | ``0038cd0c28b4``  | ``2.4.0``         | Add dag_owner_attributes table.                              |

Review Comment:
   >  This table is automatically updated by pre-commit by ``scripts/ci/pre_commit/pre_commit_migration_reference.py``
   



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r932618023


##########
docs/apache-airflow/migrations-ref.rst:
##########
@@ -27,7 +27,9 @@ Here's the list of all the Database Migrations that are executed via when you ru
 +---------------------------------+-------------------+-------------------+--------------------------------------------------------------+
 | Revision ID                     | Revises ID        | Airflow Version   | Description                                                  |
 +=================================+===================+===================+==============================================================+
-| ``0038cd0c28b4`` (head)         | ``44b7034f6bdc``  | ``2.4.0``         | Add Dataset model                                            |
+| ``38d6c28f9cf9`` (head)         | ``0038cd0c28b4``  | ``2.4.0``         | Add dag_owner_attributes table.                              |

Review Comment:
   No idea - I have never written a single migration - but I believe `alembic` does all for you. There is a section about it CONTRIBUTING - I am not sure how good it is - re version - yes 2.4.0 is good - we can change it afterwards.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
potiuk commented on PR #25280:
URL: https://github.com/apache/airflow/pull/25280#issuecomment-1199167009

   > > Still... some bugs to fix ;)
   > 
   > Yeah, this is why I'm keeping it as a draft for now :( I'm quite struggling with failing tests, I can't understand why some of them fail and how to fix them.. I'm trying to debug it, but if I struggle for much more, I'll probably make a summary of my findings/things I don't understand and ask for further help :(
   
   Yep. First thing first : ABR (Always Be Rebased):
   
   ![image](https://user-images.githubusercontent.com/595491/181749043-bd980350-b98e-4715-b81e-e923cc9911e7.png)
   


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] eladkal merged pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
eladkal merged PR #25280:
URL: https://github.com/apache/airflow/pull/25280


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] uranusjr commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r938427984


##########
airflow/models/dag.py:
##########
@@ -2781,6 +2801,20 @@ def validate_schedule_and_params(self):
                     "DAG Schedule must be None, if there are any required params without default values"
                 )
 
+    def validate_owner_links(self) -> Dict[str, str]:
+        """Parses a given link, and verifies if it's a valid URL, or a 'mailto' link"""
+        wrong_links = {}
+        for owner, link in self.owner_links.items():
+            result = urlparse(link)
+            if link.startswith('mailto:'):

Review Comment:
   ```suggestion
               if result.scheme == 'mailto':
   ```
   
   Since we already parsed the link, probably safer to rely on the result



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] uranusjr commented on pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
uranusjr commented on PR #25280:
URL: https://github.com/apache/airflow/pull/25280#issuecomment-1204865354

   I wonder if it’s better to only allow links on the DAG, or alternatively design the DAG interface like this instead:
   
   ```python
   with DAG(..., owner_links={"owner1": link1, "owner2": link2, ...}):
       task1 = Operator(..., owner="owner2")
   ```
   
   This would work a bit better with duplicates, and avoid weird cases like this:
   
   ```python
   with DAG(..., owner={"name": "uranusjr", "link": "https://uranusjr.com"}):
       task = Operator(..., owner={"name": "uranusjr", "link": "https://uranusjr.io"})  # Oops!
   ```


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] alexkruc commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
alexkruc commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r931907503


##########
airflow/migrations/versions/0115_2_4_0_add_owner_attributes_table.py:
##########
@@ -0,0 +1,51 @@
+#
+# 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.
+
+"""Add dag_owner_attributes table
+
+Revision ID: 38d6c28f9cf9

Review Comment:
   I probably made mistakes with the revisions and the file name of the migration script :) 
   Will like someone to double-check me :D 



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r932238089


##########
docs/apache-airflow/migrations-ref.rst:
##########
@@ -27,7 +27,9 @@ Here's the list of all the Database Migrations that are executed via when you ru
 +---------------------------------+-------------------+-------------------+--------------------------------------------------------------+
 | Revision ID                     | Revises ID        | Airflow Version   | Description                                                  |
 +=================================+===================+===================+==============================================================+
-| ``0038cd0c28b4`` (head)         | ``44b7034f6bdc``  | ``2.4.0``         | Add Dataset model                                            |
+| ``38d6c28f9cf9`` (head)         | ``0038cd0c28b4``  | ``2.4.0``         | Add dag_owner_attributes table.                              |

Review Comment:
   Isn't pre-commit double-checking yor :) ? 



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] uranusjr commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r938430630


##########
airflow/models/dag.py:
##########
@@ -2781,6 +2801,20 @@ def validate_schedule_and_params(self):
                     "DAG Schedule must be None, if there are any required params without default values"
                 )
 
+    def validate_owner_links(self) -> Dict[str, str]:
+        """Parses a given link, and verifies if it's a valid URL, or a 'mailto' link"""
+        wrong_links = {}
+        for owner, link in self.owner_links.items():
+            result = urlparse(link)
+            if link.startswith('mailto:'):
+                # netloc is not existing for 'mailto' link, so we are checking that the path is parsed
+                if not all([result.scheme, result.path]):
+                    wrong_links.update({owner: link})
+            elif not all([result.scheme, result.netloc]):
+                wrong_links.update({owner: link})
+
+        return wrong_links

Review Comment:
   This can be rewritten like this:
   
   ```suggestion
       def iter_invalid_owner_links(self) -> Iterator[Tuple[str, str]]:
           """Parses a given link, and verifies if it's a valid URL, or a 'mailto' link.
   
   		Returns an iterator of invalid (owner, link) pairs.
           """
           for owner, link in self.owner_links.items():
               result = urlsplit(link)
               if result.scheme == "mailto":
                   # netloc is not existing for 'mailto' link, so we are checking that the path is parsed
                   if not result.scheme or not result.path:
                       yield owner, link
               elif not result.scheme or result.netloc:
                   yield owner, link
   ```
   
   And called like this `wrong_links = dict(self.iter_invalid_owner_links())`. A few things to note:
   
   1. Using an iterator makes memory management slightly more efficient.
   2. `urlsplit` is preferred over `urlparse` these days unless there’s concrete reason, [according to the documentation](https://docs.python.org/3/library/urllib.parse.html#urllib.parse.urlsplit)
   3. Since we already parsed the link, it’s probably better to rely on the result instead of checking for the link prefix (which could cause bugs…? not sure, but using `result` removes the worry altogether)



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] alexkruc commented on pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
alexkruc commented on PR #25280:
URL: https://github.com/apache/airflow/pull/25280#issuecomment-1204924763

   > I wonder if it’s better to only allow links on the DAG, or alternatively design the DAG interface like this instead:
   > 
   > ```python
   > with DAG(..., owner_links={"owner1": link1, "owner2": link2, ...}):
   >     task1 = Operator(..., owner="owner2")
   > ```
   > 
   > This would work a bit better with duplicates, and avoid weird cases like this:
   > 
   > ```python
   > with DAG(..., owner={"name": "uranusjr", "link": "https://uranusjr.com"}):
   >     task = Operator(..., owner={"name": "uranusjr", "link": "https://uranusjr.io"})  # Oops!
   > ```
   @uranusjr We discussed it a bit on the issue description (https://github.com/apache/airflow/issues/24728), and also, one of my suggestions was to add this parameter on the DAG level. BUT -
   You lose the ability to set multiple owners for a dag, as some teams use DAGs in a multi-team manner to define one flow with several team responsibilities.
   
   Regarding your suggestion, having a DAG level parameter that has a "map" of all the owners and links sounds ok, but IMO it's a bit counterintuitive because you will have to specify the owner twice - once in the map with the link, and the second time on the task itself. 
   The way I currently implemented it keeps the link (if you need it) only once in the same place you define the owner. But if you think we should redesign it, I can, it actually might be even a bit easier to develop haha
   
   About your example, currently `owner` in a DAG is not a parameter, it's an attribute.. you can't set an owner like 
   ```python
   with DAG(..., owner=<MY_COOL_OWNER>)
   ```
   You have to set it as `default_args` that are propagating to each task anyway.. The `owner` attribute inside the DAG object returns a string with all the owners defined to a DAG. In case you set `default_args` as a value, and then set the `task.owner` to something else, it will not make any error as the `task.owner` attribute is overwriting the default args value.. I also tried it with the following DAG:
   ```python
   from datetime import datetime, timedelta
   from airflow import DAG
   from airflow.operators.bash import BashOperator
   
   owner_obj = {'name': "test_owner_link", "link": "https://www.google.com/search?q=blabla"}
   
   default_args = {'owner': owner_obj, "start_date": datetime(2021, 9, 9), "retries": 1,
                   'execution_timeout': timedelta(minutes=30)}
   
   dag = DAG("test_dag_with_links", default_args=default_args, schedule_interval=None,
             catchup=False)
   
   with dag:
       task = BashOperator(task_id='task_with_link', bash_command='echo Hello')
       task2 = BashOperator(task_id='task_with_no_link', bash_command='echo "Hello Again"', owner="bla")
       task3 = BashOperator(task_id='task_with_another_link',
                            bash_command='echo "Hello Again"',
                            owner={'name': "test_owner_email",
                                   'link': "mailto:someone@yoursite.com?subject=Mail from Our Site"})
       task4 = BashOperator(task_id='task_with_slack_link',
                            bash_command='echo "Hello Again"',
                            owner={'name': "test_owner_link", "link": "https://www.google.com/search?q=blublu"}
   
   ```
   In here we set the `owner_obj` as the default args, set few other owners and in `task4` I re-use the owner name in `owner_obj` again, but with a different link. This flow is not failing, but you're right that it's a bit misleading, because only the link in `task4` is kept (the reason is that `task.owner` is superior to `default_args`)..
   I think it's on the user side to understand that `default_args` are propagating to the task, and how it works..
   
   So in conclusion, if you guys think that adding a map inside `DAG(..., owner_links={...})` is the right thing, we can, but I think that you should set a link in the place that you use it, and not write the owner twice.
   It can also lead to some "issues" if several teams share the same DAG, and each team is in charge of some tasks.. In that case, no one would want to touch the `DAG` object, as they will renounce ownership of the DAG structure itself, but only be in charge of sporadic tasks inside 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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r939893924


##########
airflow/migrations/versions/0116_2_4_0_add_dag_owner_attributes_table.py:
##########
@@ -0,0 +1,52 @@
+#
+# 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.
+
+"""add dag_owner_attributes table
+
+Revision ID: 1486deb605b4
+Revises: f4ff391becb5
+Create Date: 2022-08-04 16:59:45.406589
+
+"""
+
+import sqlalchemy as sa
+from alembic import op
+
+# revision identifiers, used by Alembic.
+revision = '1486deb605b4'
+down_revision = 'f4ff391becb5'
+branch_labels = None
+depends_on = None
+airflow_version = '2.4.0'
+
+
+def upgrade():
+    """Apply Add ``DagOwnerAttributes`` table"""
+    op.create_table(
+        'dag_owner_attributes',

Review Comment:
   If I succeed, I will turn it into a regular test 



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r939883784


##########
airflow/migrations/versions/0116_2_4_0_add_dag_owner_attributes_table.py:
##########
@@ -0,0 +1,52 @@
+#
+# 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.
+
+"""add dag_owner_attributes table
+
+Revision ID: 1486deb605b4
+Revises: f4ff391becb5
+Create Date: 2022-08-04 16:59:45.406589
+
+"""
+
+import sqlalchemy as sa
+from alembic import op
+
+# revision identifiers, used by Alembic.
+revision = '1486deb605b4'
+down_revision = 'f4ff391becb5'
+branch_labels = None
+depends_on = None
+airflow_version = '2.4.0'
+
+
+def upgrade():
+    """Apply Add ``DagOwnerAttributes`` table"""
+    op.create_table(
+        'dag_owner_attributes',

Review Comment:
   Notting to apologise for @alexkruc ! 
   
   We just need to have tests covering it :). You were not supposed to know it, you were supposed to get a test failing if this was wrong :)



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] uranusjr commented on pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
uranusjr commented on PR #25280:
URL: https://github.com/apache/airflow/pull/25280#issuecomment-1206032669

   We should also add some documentation on this feature.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] uranusjr commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r938428284


##########
airflow/models/dag.py:
##########
@@ -2781,6 +2801,20 @@ def validate_schedule_and_params(self):
                     "DAG Schedule must be None, if there are any required params without default values"
                 )
 
+    def validate_owner_links(self) -> Dict[str, str]:
+        """Parses a given link, and verifies if it's a valid URL, or a 'mailto' link"""
+        wrong_links = {}
+        for owner, link in self.owner_links.items():
+            result = urlparse(link)

Review Comment:
   `urlsplit` is recommended over `urlparse` these days, according to the documentation.
   
   https://docs.python.org/3/library/urllib.parse.html#urllib.parse.urlsplit



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] alexkruc commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
alexkruc commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r938684665


##########
airflow/models/dag.py:
##########
@@ -2781,6 +2801,20 @@ def validate_schedule_and_params(self):
                     "DAG Schedule must be None, if there are any required params without default values"
                 )
 
+    def validate_owner_links(self) -> Dict[str, str]:
+        """Parses a given link, and verifies if it's a valid URL, or a 'mailto' link"""
+        wrong_links = {}
+        for owner, link in self.owner_links.items():
+            result = urlparse(link)
+            if link.startswith('mailto:'):
+                # netloc is not existing for 'mailto' link, so we are checking that the path is parsed
+                if not all([result.scheme, result.path]):
+                    wrong_links.update({owner: link})
+            elif not all([result.scheme, result.netloc]):
+                wrong_links.update({owner: link})
+
+        return wrong_links

Review Comment:
   Thanks for the tip! I changed the code to use an iterator :)
   I didn't commit your suggestion as I changed it a bit (just minor changes), but the flow is as you suggested, thanks!



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] alexkruc commented on pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
alexkruc commented on PR #25280:
URL: https://github.com/apache/airflow/pull/25280#issuecomment-1205582529

   @uranusjr You're right, in the described implementation, the last task with an `owner_link` overwrote the link, and the behavior was weird.
   I've tried to fix it but have not proceeded because I couldn't show it as a Broken DAG.
   
   Finally, I've decided to take your advice and make the `owner_link` to be a DAG parameter and not a task one, as you described:
   ```python
   with DAG(..., owner_links={"owner1": link1, "owner2": link2, ...}):
       task1 = Operator(..., owner="owner2")
   ```
   
   I've also changed the description of this feature on my first message with the new examples and added extra pictures to show the broken DAG behavior and such. 
   
   Can you please take a look again? 🙏 


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] alexkruc commented on pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
alexkruc commented on PR #25280:
URL: https://github.com/apache/airflow/pull/25280#issuecomment-1201032219

   I have still 2 failing tests that I need help with, I wrapped my head around them for a long time and I'll appreciate some help and another pair of eyes :( 
   
   * `tests/serialization/test_dag_serialization.py - test_dag_serialization_to_timetable`
   I've added the new attributes that I've added to the tasks dict (owner_link), but the diff in the assert is mentioning much more differences, each `__type` is prefixed with` <Encoding.TYPE:` and `__var` with `<Encoding.VAR:` which I can't understand where it's coming from.
   
   * `tests/www/views/test_views_acl.py - test_dag_autocomplete_success`
   For some reason, the endpoint dagmodel/autocomplete for query `?query=flow` returns multiple owner fields. It's a bit strange for me, as the endpoint is calling `DagModel.owners` which is unchanged. The purpose of my PR is not to change the model of the current owners in the DB, and introduce all attributes in a new table. I can't figure out why now the endpoint in view.py is multiplying it.. One theory I have is that another failed test is not cleaning up it's DAG's in the DB after failure, and we are resulting with a stale DB state, that fails this one as we don't start from empty.. But I can't verify 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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r932618023


##########
docs/apache-airflow/migrations-ref.rst:
##########
@@ -27,7 +27,9 @@ Here's the list of all the Database Migrations that are executed via when you ru
 +---------------------------------+-------------------+-------------------+--------------------------------------------------------------+
 | Revision ID                     | Revises ID        | Airflow Version   | Description                                                  |
 +=================================+===================+===================+==============================================================+
-| ``0038cd0c28b4`` (head)         | ``44b7034f6bdc``  | ``2.4.0``         | Add Dataset model                                            |
+| ``38d6c28f9cf9`` (head)         | ``0038cd0c28b4``  | ``2.4.0``         | Add dag_owner_attributes table.                              |

Review Comment:
   No idea - I have never written a single migration - but I believe `alembic` does all for you. There is a section about it CONTRIBUTING - I am not sure how good it is but it should be enough. 
   
   Re version - yes 2.4.0 is good - we can change it afterwards.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
potiuk commented on PR #25280:
URL: https://github.com/apache/airflow/pull/25280#issuecomment-1195315404

   > Now that I'm thinking of this, maybe I should rename the `dag_owner_links` table to something like `dag_owner_attributes` -> it sounds more generic if we would like to add more owner attributes in the future.. WDYT?
   
   Yep


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] eladkal commented on pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
eladkal commented on PR #25280:
URL: https://github.com/apache/airflow/pull/25280#issuecomment-1201041472

   From what I see `test_dag_autocomplete_success` is tested with `example_bash_operator`, `example_subdag_operator`
   i suggest to double check how this behavior works with SubDAGs


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] potiuk commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r932238089


##########
docs/apache-airflow/migrations-ref.rst:
##########
@@ -27,7 +27,9 @@ Here's the list of all the Database Migrations that are executed via when you ru
 +---------------------------------+-------------------+-------------------+--------------------------------------------------------------+
 | Revision ID                     | Revises ID        | Airflow Version   | Description                                                  |
 +=================================+===================+===================+==============================================================+
-| ``0038cd0c28b4`` (head)         | ``44b7034f6bdc``  | ``2.4.0``         | Add Dataset model                                            |
+| ``38d6c28f9cf9`` (head)         | ``0038cd0c28b4``  | ``2.4.0``         | Add dag_owner_attributes table.                              |

Review Comment:
   Isn't pre-commit double-checking your :) ? 



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] alexott commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
alexott commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r939642811


##########
airflow/migrations/versions/0116_2_4_0_add_dag_owner_attributes_table.py:
##########
@@ -0,0 +1,52 @@
+#
+# 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.
+
+"""add dag_owner_attributes table
+
+Revision ID: 1486deb605b4
+Revises: f4ff391becb5
+Create Date: 2022-08-04 16:59:45.406589
+
+"""
+
+import sqlalchemy as sa
+from alembic import op
+
+# revision identifiers, used by Alembic.
+revision = '1486deb605b4'
+down_revision = 'f4ff391becb5'
+branch_labels = None
+depends_on = None
+airflow_version = '2.4.0'
+
+
+def upgrade():
+    """Apply Add ``DagOwnerAttributes`` table"""
+    op.create_table(
+        'dag_owner_attributes',

Review Comment:
   opened #25579 for 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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] alexkruc commented on a diff in pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
alexkruc commented on code in PR #25280:
URL: https://github.com/apache/airflow/pull/25280#discussion_r932928662


##########
docs/apache-airflow/migrations-ref.rst:
##########
@@ -27,7 +27,9 @@ Here's the list of all the Database Migrations that are executed via when you ru
 +---------------------------------+-------------------+-------------------+--------------------------------------------------------------+
 | Revision ID                     | Revises ID        | Airflow Version   | Description                                                  |
 +=================================+===================+===================+==============================================================+
-| ``0038cd0c28b4`` (head)         | ``44b7034f6bdc``  | ``2.4.0``         | Add Dataset model                                            |
+| ``38d6c28f9cf9`` (head)         | ``0038cd0c28b4``  | ``2.4.0``         | Add dag_owner_attributes table.                              |

Review Comment:
   My bad, I completely missed this part in the CONTRIBUTING doc.. I regenerated the file using `alembic` and it created the revisions for me, you're totally right! :)
   
   thanks! 😄 



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] eladkal commented on pull request #25280: Adding support for owner links in the Dags view UI

Posted by GitBox <gi...@apache.org>.
eladkal commented on PR #25280:
URL: https://github.com/apache/airflow/pull/25280#issuecomment-1204918789

   > I wonder if it’s better to only allow links on the DAG, or alternatively design the DAG interface like this instead:
   
   Do we even allow to set owner on DAG level? I think this works only with default_args which is on Operator level.
   
   Anyway I think this one goes on hand with what we allow for owner.
   This PR is an enhancement of owner capabilities it does not take into account the issue of how/where owner should be defined.  I _think_ this is a bigger question for another discussion (Should owner be on task level or dag level or both)
   


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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