You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by ep...@apache.org on 2023/08/14 09:27:02 UTC

[airflow] branch v2-7-test updated (84a5e464dc -> f5d8201ea7)

This is an automated email from the ASF dual-hosted git repository.

ephraimanierobi pushed a change to branch v2-7-test
in repository https://gitbox.apache.org/repos/asf/airflow.git


    from 84a5e464dc Update RELEASE_NOTES.rst
     new eda4bc4b87 Enable test connection in UI if explicitly set to `Enabled` (#33342)
     new 9405b081d7 Modify pathspec version restriction (#33349)
     new 1cb3e52bb2 Make skip of trigger form in UI if no params are defined configurable (#33351)
     new 1b0b952bb3 Fix clearing behavior for downstream work task with non-collinear setup task (#33358)
     new e690f5c561 Rename `ExternalBranchPythonOperator` to `BranchExternalPythonOperator` (#33360)
     new cbcf8bc207 Limit aiobotocore for eager upgrade to prevent pip backtracking (#33362)
     new 5ae23ebe1c Clarify UI user types in security model (#33021)
     new a817f21fb5 Fix links in security docs (#33329)
     new f5d8201ea7 Remove user sessions when resetting password (#33347)

The 9 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 Dockerfile.ci                                      |   4 +-
 .../auth/managers/fab/security_manager/override.py |  53 +++++++-
 airflow/config_templates/config.yml                |  25 +++-
 airflow/models/dag.py                              |  20 ++-
 airflow/operators/python.py                        |   2 +-
 airflow/utils/db_cleanup.py                        |   4 +
 airflow/www/static/js/connection_form.js           |   2 +-
 airflow/www/views.py                               |   3 +-
 docs/apache-airflow/security/access-control.rst    |  19 +--
 docs/apache-airflow/security/index.rst             |   4 +-
 docs/apache-airflow/security/security_model.rst    |  34 +++--
 newsfragments/33351.significant.rst                |   6 +
 setup.cfg                                          |   2 +-
 tests/models/test_dag.py                           | 137 ++++++++++++++++++++-
 tests/operators/test_python.py                     |   6 +-
 tests/www/views/test_views_custom_user_views.py    | 112 +++++++++++++++++
 16 files changed, 381 insertions(+), 52 deletions(-)
 create mode 100644 newsfragments/33351.significant.rst


[airflow] 08/09: Fix links in security docs (#33329)

Posted by ep...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ephraimanierobi pushed a commit to branch v2-7-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit a817f21fb52d01cbf8dce924801d76f9b23c55b0
Author: Jed Cunningham <66...@users.noreply.github.com>
AuthorDate: Sun Aug 13 12:32:59 2023 -0600

    Fix links in security docs (#33329)
    
    (cherry picked from commit 0cb256411a02516dc9eca88b570abfb8c8a3c35b)
---
 airflow/config_templates/config.yml    | 2 +-
 docs/apache-airflow/security/index.rst | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml
index 8901383bb0..dc89dcc814 100644
--- a/airflow/config_templates/config.yml
+++ b/airflow/config_templates/config.yml
@@ -467,7 +467,7 @@ core:
         Before setting this to Enabled, make sure that you review the users who are able to add/edit
         connections and ensure they are trusted. Connection testing can be done maliciously leading to
         undesired and insecure outcomes. For more information on capabilities of users, see the documentation:
-        https://airflow.apache.org/docs/apache-airflow/stable/security/index.html#capabilities-of-authenticated-ui-users
+        https://airflow.apache.org/docs/apache-airflow/stable/security/security_model.html#capabilities-of-authenticated-ui-users
       version_added: 2.7.0
       type: string
       example: ~
diff --git a/docs/apache-airflow/security/index.rst b/docs/apache-airflow/security/index.rst
index b86fe72f4d..7d186a09fb 100644
--- a/docs/apache-airflow/security/index.rst
+++ b/docs/apache-airflow/security/index.rst
@@ -20,11 +20,11 @@ Security
 
 This section of the documentation covers security-related topics.
 
-Make sure to get familiar with the `Airflow Security Model <security_model>`_ if you want to understand
+Make sure to get familiar with the :doc:`Airflow Security Model </security/security_model>` if you want to understand
 the different user types of Apache Airflowâ„¢, what they have access to, and the role Deployment Managers have in deploying Airflow in a secure way.
 
 Also, if you want to understand how Airflow releases security patches and what to expect from them,
-head over to `Releasing security patches <releasing_security_patches>`_.
+head over to :doc:`Releasing security patches </security/releasing_security_patches>`.
 
 Follow the below topics as well to understand other aspects of Airflow security:
 


[airflow] 01/09: Enable test connection in UI if explicitly set to `Enabled` (#33342)

Posted by ep...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ephraimanierobi pushed a commit to branch v2-7-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit eda4bc4b87deb6095cb009825ace9bf87eead369
Author: Pankaj Koti <pa...@gmail.com>
AuthorDate: Sat Aug 12 18:57:36 2023 +0530

    Enable test connection in UI if explicitly set to `Enabled` (#33342)
    
    Following up PR https://github.com/apache/airflow/pull/32052/ the test connection is
    disabled in UI, API and CLI. The API and CLI strictly check for the
    config value to be set as `Enabled` for the functionality to be enabled,
    whereas the UI just checks that is it not set to `Disabled`. As a result
    setting values to the config param other than `Disabled`, enables the
    button in the UI. Even though the button gets enabled, the API forbids
    it as there is a strict check in the API that the value is set to
    `Enabled` and only then allows, however, it makes sense to also strictly
    check in the UI that value is set to `Enabled`.
    
    (cherry picked from commit 50765eb0883652c16b40d69d8a1ac78096646610)
---
 airflow/www/static/js/connection_form.js | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/airflow/www/static/js/connection_form.js b/airflow/www/static/js/connection_form.js
index 453be58411..2451da05a3 100644
--- a/airflow/www/static/js/connection_form.js
+++ b/airflow/www/static/js/connection_form.js
@@ -135,7 +135,7 @@ function handleTestConnection(connectionType, testableConnections) {
     $(testButton).hide();
     return;
   }
-  if (configTestConnection === "disabled") {
+  if (configTestConnection !== "enabled") {
     // If test connection is not enabled in config, disable button and display toolip
     // alerting the user.
     $(testButton)


[airflow] 09/09: Remove user sessions when resetting password (#33347)

Posted by ep...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ephraimanierobi pushed a commit to branch v2-7-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit f5d8201ea7935d17cecaf25fc90d4ef0ccdd627b
Author: Jarek Potiuk <ja...@potiuk.com>
AuthorDate: Sun Aug 13 21:21:43 2023 +0200

    Remove user sessions when resetting password (#33347)
    
    * Remove user sessions when resetting password
    
    When user's password is reset, we also remove all DB sessions
    for that user - for database session backend.
    
    In case we are using securecookie mechanism, resetting password does
    not invalidate old sessions, so instead we are displaying warning to
    the user performing the reset that in order to clear existing
    sessions of the user, the secure_key needs to be changed and it
    will invalidate all sessions for all users.
    
    Protection has been added in case the number of sessions in the DB
    is too big to effectively scan and remove sessions for the user. In
    such case we print warning for the user that sessions have not
    been reset, and we suggest to improve the way their deployment
    mechanisms create too many sessions - by either changing the way
    how automation of the API calls is done and/or by purging the
    sessions regularly by "airflow db clean".
    
    * Update airflow/auth/managers/fab/security_manager/override.py
    
    Co-authored-by: Hussein Awala <hu...@awala.fr>
    
    ---------
    
    Co-authored-by: Hussein Awala <hu...@awala.fr>
    (cherry picked from commit 2caa186935151683076b74357daad83d2538a3f6)
---
 .../auth/managers/fab/security_manager/override.py |  53 +++++++++-
 airflow/config_templates/config.yml                |  14 ++-
 airflow/utils/db_cleanup.py                        |   4 +
 tests/www/views/test_views_custom_user_views.py    | 112 +++++++++++++++++++++
 4 files changed, 181 insertions(+), 2 deletions(-)

diff --git a/airflow/auth/managers/fab/security_manager/override.py b/airflow/auth/managers/fab/security_manager/override.py
index f452fe912c..089b449422 100644
--- a/airflow/auth/managers/fab/security_manager/override.py
+++ b/airflow/auth/managers/fab/security_manager/override.py
@@ -19,14 +19,26 @@ from __future__ import annotations
 
 from functools import cached_property
 
-from flask import g
+from flask import flash, g
 from flask_appbuilder.const import AUTH_DB, AUTH_LDAP, AUTH_OAUTH, AUTH_OID, AUTH_REMOTE_USER
 from flask_babel import lazy_gettext
 from flask_jwt_extended import JWTManager
 from flask_login import LoginManager
+from itsdangerous import want_bytes
+from markupsafe import Markup
 from werkzeug.security import generate_password_hash
 
+from airflow.auth.managers.fab.models import User
 from airflow.auth.managers.fab.models.anonymous_user import AnonymousUser
+from airflow.www.session import AirflowDatabaseSessionInterface
+
+# This is the limit of DB user sessions that we consider as "healthy". If you have more sessions that this
+# number then we will refuse to delete sessions that have expired and old user sessions when resetting
+# user's password, and raise a warning in the UI instead. Usually when you have that many sessions, it means
+# that there is something wrong with your deployment - for example you have an automated API call that
+# continuously creates new sessions. Such setup should be fixed by reusing sessions or by periodically
+# purging the old sessions by using `airflow db clean` command.
+MAX_NUM_DATABASE_USER_SESSIONS = 50000
 
 
 class FabAirflowSecurityManagerOverride:
@@ -230,8 +242,47 @@ class FabAirflowSecurityManagerOverride:
         """
         user = self.get_user_by_id(userid)
         user.password = generate_password_hash(password)
+        self.reset_user_sessions(user)
         self.update_user(user)
 
+    def reset_user_sessions(self, user: User) -> None:
+        if isinstance(self.appbuilder.get_app.session_interface, AirflowDatabaseSessionInterface):
+            interface = self.appbuilder.get_app.session_interface
+            session = interface.db.session
+            user_session_model = interface.sql_session_model
+            num_sessions = session.query(user_session_model).count()
+            if num_sessions > MAX_NUM_DATABASE_USER_SESSIONS:
+                flash(
+                    Markup(
+                        f"The old sessions for user {user.username} have <b>NOT</b> been deleted!<br>"
+                        f"You have a lot ({num_sessions}) of user sessions in the 'SESSIONS' table in "
+                        f"your database.<br> "
+                        "This indicates that this deployment might have an automated API calls that create "
+                        "and not reuse sessions.<br>You should consider reusing sessions or cleaning them "
+                        "periodically using db clean.<br>"
+                        "Make sure to reset password for the user again after cleaning the session table "
+                        "to remove old sessions of the user."
+                    ),
+                    "warning",
+                )
+            else:
+                for s in session.query(user_session_model):
+                    session_details = interface.serializer.loads(want_bytes(s.data))
+                    if session_details.get("_user_id") == user.id:
+                        session.delete(s)
+        else:
+            flash(
+                Markup(
+                    "Since you are using `securecookie` session backend mechanism, we cannot prevent "
+                    f"some old sessions for user {user.username} to be reused.<br> If you want to make sure "
+                    "that the user is logged out from all sessions, you should consider using "
+                    "`database` session backend mechanism.<br> You can also change the 'secret_key` "
+                    "webserver configuration for all your webserver instances and restart the webserver. "
+                    "This however will logout all users from all sessions."
+                ),
+                "warning",
+            )
+
     def load_user(self, user_id):
         """Load user by ID."""
         return self.get_user_by_id(int(user_id))
diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml
index dc89dcc814..b9ea0c0e3f 100644
--- a/airflow/config_templates/config.yml
+++ b/airflow/config_templates/config.yml
@@ -1387,7 +1387,19 @@ webserver:
       default: ""
     session_backend:
       description: |
-        The type of backend used to store web session data, can be 'database' or 'securecookie'
+        The type of backend used to store web session data, can be `database` or `securecookie`. For the
+        `database` backend, sessions are store in the database (in `session` table) and they can be
+        managed there (for example when you reset password of the user, all sessions for that user are
+        deleted). For the `securecookie` backend, sessions are stored in encrypted cookies on the client
+        side. The `securecookie` mechanism is 'lighter' than database backend, but sessions are not deleted
+        when you reset password of the user, which means that other than waiting for expiry time, the only
+        way to invalidate all sessions for a user is to change secret_key and restart webserver (which
+        also invalidates and logs out all other user's sessions).
+
+        When you are using `database` backend, make sure to keep your database session table small
+        by periodically running `airflow db clean --table session` command, especially if you have
+        automated API calls that will create a new session for each call rather than reuse the sessions
+        stored in browser cookies.
       version_added: 2.2.4
       type: string
       example: securecookie
diff --git a/airflow/utils/db_cleanup.py b/airflow/utils/db_cleanup.py
index f9dae37242..79a804a9e5 100644
--- a/airflow/utils/db_cleanup.py
+++ b/airflow/utils/db_cleanup.py
@@ -38,6 +38,7 @@ from sqlalchemy.sql.expression import ClauseElement, Executable, tuple_
 
 from airflow import AirflowException
 from airflow.cli.simple_table import AirflowConsole
+from airflow.configuration import conf
 from airflow.models import Base
 from airflow.utils import timezone
 from airflow.utils.db import reflect_tables
@@ -115,6 +116,9 @@ config_list: list[_TableConfig] = [
     _TableConfig(table_name="celery_tasksetmeta", recency_column_name="date_done"),
 ]
 
+if conf.get("webserver", "session_backend") == "database":
+    config_list.append(_TableConfig(table_name="session", recency_column_name="expiry"))
+
 config_dict: dict[str, _TableConfig] = {x.orm_model.name: x for x in sorted(config_list)}
 
 
diff --git a/tests/www/views/test_views_custom_user_views.py b/tests/www/views/test_views_custom_user_views.py
index 71ab551ab7..771a103d69 100644
--- a/tests/www/views/test_views_custom_user_views.py
+++ b/tests/www/views/test_views_custom_user_views.py
@@ -17,7 +17,11 @@
 # under the License.
 from __future__ import annotations
 
+from datetime import datetime, timedelta
+from unittest import mock
+
 import pytest
+from flask.sessions import SecureCookieSessionInterface
 from flask_appbuilder import SQLA
 
 from airflow import settings
@@ -166,3 +170,111 @@ class TestSecurity:
         check_content_in_response("Deleted Row", response)
         check_content_not_in_response(user_to_delete.username, response)
         assert bool(self.security_manager.get_user_by_id(user_to_delete.id)) is False
+
+
+# type: ignore[attr-defined]
+
+
+class TestResetUserSessions:
+    @classmethod
+    def setup_class(cls):
+        settings.configure_orm()
+
+    def setup_method(self):
+        # We cannot reuse the app in tests (on class level) as in Flask 2.2 this causes
+        # an exception because app context teardown is removed and if even single request is run via app
+        # it cannot be re-intialized again by passing it as constructor to SQLA
+        # This makes the tests slightly slower (but they work with Flask 2.1 and 2.2
+        self.app = application.create_app(testing=True)
+        self.appbuilder = self.app.appbuilder
+        self.app.config["WTF_CSRF_ENABLED"] = False
+        self.security_manager = self.appbuilder.sm
+        self.interface = self.app.session_interface
+        self.model = self.interface.sql_session_model
+        self.serializer = self.interface.serializer
+        self.db = self.interface.db
+        self.db.session.query(self.model).delete()
+        self.db.session.commit()
+        self.db.session.flush()
+        self.user_1 = create_user(
+            self.app,
+            username="user_to_delete_1",
+            role_name="user_to_delete",
+        )
+        self.user_2 = create_user(
+            self.app,
+            username="user_to_delete_2",
+            role_name="user_to_delete",
+        )
+        self.db.session.commit()
+        self.db.session.flush()
+
+    def create_user_db_session(self, session_id: str, time_delta: timedelta, user_id: int):
+        self.db.session.add(
+            self.model(
+                session_id=session_id,
+                data=self.serializer.dumps({"_user_id": user_id}),
+                expiry=datetime.now() + time_delta,
+            )
+        )
+
+    @pytest.mark.parametrize(
+        "time_delta, user_sessions_deleted",
+        [
+            pytest.param(timedelta(days=-1), True, id="Both expired"),
+            pytest.param(timedelta(hours=1), True, id="Both fresh"),
+            pytest.param(timedelta(days=1), True, id="Both future"),
+        ],
+    )
+    def test_reset_user_sessions_delete(self, time_delta: timedelta, user_sessions_deleted: bool):
+
+        self.create_user_db_session("session_id_1", time_delta, self.user_1.id)
+        self.create_user_db_session("session_id_2", time_delta, self.user_2.id)
+        self.db.session.commit()
+        self.db.session.flush()
+        assert self.db.session.query(self.model).count() == 2
+        assert self.get_session_by_id("session_id_1") is not None
+        assert self.get_session_by_id("session_id_2") is not None
+
+        self.security_manager.reset_password(self.user_1.id, "new_password")
+        self.db.session.commit()
+        self.db.session.flush()
+        if user_sessions_deleted:
+            assert self.db.session.query(self.model).count() == 1
+            assert self.get_session_by_id("session_id_1") is None
+        else:
+            assert self.db.session.query(self.model).count() == 2
+            assert self.get_session_by_id("session_id_1") is not None
+
+    def get_session_by_id(self, session_id: str):
+        return self.db.session.query(self.model).filter(self.model.session_id == session_id).scalar()
+
+    @mock.patch("airflow.auth.managers.fab.security_manager.override.flash")
+    @mock.patch("airflow.auth.managers.fab.security_manager.override.MAX_NUM_DATABASE_USER_SESSIONS", 1)
+    def test_refuse_delete(self, flash_mock):
+        self.create_user_db_session("session_id_1", timedelta(days=1), self.user_1.id)
+        self.create_user_db_session("session_id_2", timedelta(days=1), self.user_2.id)
+        self.db.session.commit()
+        self.db.session.flush()
+        assert self.db.session.query(self.model).count() == 2
+        assert self.get_session_by_id("session_id_1") is not None
+        assert self.get_session_by_id("session_id_2") is not None
+        self.security_manager.reset_password(self.user_1.id, "new_password")
+        assert flash_mock.called
+        assert (
+            "The old sessions for user user_to_delete_1 have <b>NOT</b> been deleted!"
+            in flash_mock.call_args[0][0]
+        )
+        assert self.db.session.query(self.model).count() == 2
+        assert self.get_session_by_id("session_id_1") is not None
+        assert self.get_session_by_id("session_id_2") is not None
+
+    @mock.patch("airflow.auth.managers.fab.security_manager.override.flash")
+    def test_warn_securecookie(self, flash_mock):
+        self.app.session_interface = SecureCookieSessionInterface()
+        self.security_manager.reset_password(self.user_1.id, "new_password")
+        assert flash_mock.called
+        assert (
+            "Since you are using `securecookie` session backend mechanism, we cannot"
+            in flash_mock.call_args[0][0]
+        )


[airflow] 07/09: Clarify UI user types in security model (#33021)

Posted by ep...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ephraimanierobi pushed a commit to branch v2-7-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit 5ae23ebe1c440103a7f65563933dd258c7f207c8
Author: Jed Cunningham <66...@users.noreply.github.com>
AuthorDate: Sun Aug 13 12:06:28 2023 -0600

    Clarify UI user types in security model (#33021)
    
    * Clarify op users in security model
    
    * Switch the order around
    
    (cherry picked from commit f1fad44650e247b03f79e7db8d70d334e74468a7)
---
 docs/apache-airflow/security/access-control.rst | 19 +++++---------
 docs/apache-airflow/security/security_model.rst | 34 ++++++++++++-------------
 2 files changed, 22 insertions(+), 31 deletions(-)

diff --git a/docs/apache-airflow/security/access-control.rst b/docs/apache-airflow/security/access-control.rst
index e7543d1966..0ac191f6f8 100644
--- a/docs/apache-airflow/security/access-control.rst
+++ b/docs/apache-airflow/security/access-control.rst
@@ -34,9 +34,9 @@ regarding its security model.
 Default Roles
 '''''''''''''
 Airflow ships with a set of roles by default: Admin, User, Op, Viewer, and Public.
-Only ``Admin`` users could configure/alter the permissions for other roles. But it is not recommended
-that ``Admin`` users alter these default roles in any way by removing
-or adding permissions to these roles.
+By default, only ``Admin`` users can configure/alter permissions for roles. However,
+it is recommended that these default roles remain unaltered, and instead ``Admin`` users
+create new roles with the desired permissions if changes are necessary.
 
 Admin
 ^^^^^
@@ -49,38 +49,31 @@ Public
 
 Viewer
 ^^^^^^
-``Viewer`` users have limited viewer permissions
+``Viewer`` users have limited read permissions:
 
 .. exampleinclude:: /../../airflow/www/security.py
     :language: python
     :start-after: [START security_viewer_perms]
     :end-before: [END security_viewer_perms]
 
-on limited web views.
-
 User
 ^^^^
-``User`` users have ``Viewer`` permissions plus additional user permissions
+``User`` users have ``Viewer`` permissions plus additional permissions:
 
 .. exampleinclude:: /../../airflow/www/security.py
     :language: python
     :start-after: [START security_user_perms]
     :end-before: [END security_user_perms]
 
-on User web views which is the same as Viewer web views.
-
 Op
 ^^
-``Op`` users have ``User`` permissions plus additional op permissions
+``Op`` users have ``User`` permissions plus additional permissions:
 
 .. exampleinclude:: /../../airflow/www/security.py
     :language: python
     :start-after: [START security_op_perms]
     :end-before: [END security_op_perms]
 
-on ``User`` web views.
-
-
 Custom Roles
 '''''''''''''
 
diff --git a/docs/apache-airflow/security/security_model.rst b/docs/apache-airflow/security/security_model.rst
index 05c87222cc..addb69e882 100644
--- a/docs/apache-airflow/security/security_model.rst
+++ b/docs/apache-airflow/security/security_model.rst
@@ -46,11 +46,8 @@ varying access and capabilities:
    that DAG code uses to access external systems. DAG Authors have full access
    to the metadata database and internal audit logs.
 
-3. **Authenticated UI users**: They have access to the UI and API. Admin
-   users can manage permissions and execute code on workers. Connection
-   configuration users can configure connections and execute code on
-   workers. Operations users have access to DAG execution status. Trust
-   is crucial to prevent abuse and Denial of Service attacks.
+3. **Authenticated UI users**: They have access to the UI and API. See below
+   for more details on the capabilities authenticated UI users may have.
 
 4. **Non-authenticated UI users**: Airflow doesn't support
    unauthenticated users by default. If allowed, potential vulnerabilities
@@ -60,7 +57,11 @@ Capabilities of authenticated UI users
 --------------------------------------
 
 The capabilities of **Authenticated UI users** can vary depending on
-what roles have been configured by the Deployment Manager or Admin users as well as what permissions those roles have. Permissions on roles can be scoped as tightly as a single DAG, for example, or as broad as Admin. Below are three general categories to help conceptualize some of the capabilities authenticated users may have:
+what roles have been configured by the Deployment Manager or Admin users
+as well as what permissions those roles have. Permissions on roles can be
+scoped as tightly as a single DAG, for example, or as broad as Admin.
+Below are four general categories to help conceptualize some of the
+capabilities authenticated users may have:
 
 1. **Admin users**: They manage and grant permissions to other users,
    with full access to all UI capabilities. They can potentially execute
@@ -72,7 +73,11 @@ what roles have been configured by the Deployment Manager or Admin users as well
    They also have the ability to create a Webserver Denial of Service
    situation and should be trusted not to misuse this capability.
 
-2. **Connection configuration users**: They configure connections and
+2. **Operations users**: The primary difference between an operator and admin
+   if the ability to manage and grant permissions to other users - only admins
+   are able to do this. Otherwise assume they have the same access as an admin.
+
+3. **Connection configuration users**: They configure connections and
    potentially execute code on workers during DAG execution. Trust is
    required to prevent misuse of these privileges. They have full access
    to sensitive credentials stored in connections and can modify them.
@@ -81,17 +86,10 @@ what roles have been configured by the Deployment Manager or Admin users as well
    create a Webserver Denial of Service situation and should be trusted
    not to misuse this capability.
 
-3. **Operations users**: They have access to DAG execution status via
-   the UI. Currently, Airflow lacks full protection for accessing groups
-   of DAGs' history and execution. They can perform actions such as
-   clearing, re-running, triggering DAGs, and changing parameters.
-   Depending on access restrictions, they may also have access to
-   editing variables and viewing Airflow configuration. They should not
-   have access to sensitive system-level information or connections, and
-   they should not be able to access sensitive task information unless
-   deliberately exposed in logs by DAG authors. They should be trusted
-   not to abuse their privileges, as they can potentially overload the
-   server and cause Denial of Service situations.
+4. **Normal Users**: They can view and interact with the UI and API.
+   They are able to view and edit DAGs, task instances, and DAG runs, and view task logs.
+
+For more information on the capabilities of authenticated UI users, see :doc:`/security/access-control`.
 
 Responsibilities of Deployment Managers
 ---------------------------------------


[airflow] 04/09: Fix clearing behavior for downstream work task with non-collinear setup task (#33358)

Posted by ep...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ephraimanierobi pushed a commit to branch v2-7-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit 1b0b952bb37384e3318f8bf4d97c47c5ed7cd79c
Author: Daniel Standish <15...@users.noreply.github.com>
AuthorDate: Sun Aug 13 05:56:01 2023 -0700

    Fix clearing behavior for downstream work task with non-collinear setup task (#33358)
    
    * Fix clearing behavior for downstream work task with non-collinear setup
    
    With this kind of dag if you clear w1 downstream then you also clear w2:
    
    ```python
        s1 >> w1 >> [w2, t1]
        s1 >> t1
        s2 >> t2
        s2 >> w2 >> t2
    ```
    
    We need to make sure that the setup for w2 also gets cleared.  But, to avoid the need to recurse to arbitrary depth for setups of setups, let's just say that a setup cannot have a setup.  A setup can *come after* another setup, but it won't *be* a setup for the setup (and what's at stake is just the clearing behavior).
    
    * fixup
    
    * teardowns can't have setup / teardown either
    
    (cherry picked from commit 45713446f37ee4b1ee972ab8b5aa1ac0b2482197)
---
 airflow/models/dag.py    |  20 +++++--
 tests/models/test_dag.py | 137 ++++++++++++++++++++++++++++++++++++++++++++++-
 2 files changed, 150 insertions(+), 7 deletions(-)

diff --git a/airflow/models/dag.py b/airflow/models/dag.py
index 1cb9220e58..d4ff29db2f 100644
--- a/airflow/models/dag.py
+++ b/airflow/models/dag.py
@@ -2356,17 +2356,27 @@ class DAG(LoggingMixin):
         else:
             matched_tasks = [t for t in self.tasks if t.task_id in task_ids_or_regex]
 
-        also_include: list[Operator] = []
+        also_include_ids: set[str] = set()
         for t in matched_tasks:
             if include_downstream:
-                also_include.extend(t.get_flat_relatives(upstream=False))
+                for rel in t.get_flat_relatives(upstream=False):
+                    also_include_ids.add(rel.task_id)
+                    if rel not in matched_tasks:  # if it's in there, we're already processing it
+                        # need to include setups and teardowns for tasks that are in multiple
+                        # non-collinear setup/teardown paths
+                        if not rel.is_setup and not rel.is_teardown:
+                            also_include_ids.update(
+                                x.task_id for x in rel.get_upstreams_only_setups_and_teardowns()
+                            )
             if include_upstream:
-                also_include.extend(t.get_upstreams_follow_setups())
+                also_include_ids.update(x.task_id for x in t.get_upstreams_follow_setups())
             else:
-                also_include.extend(t.get_upstreams_only_setups_and_teardowns())
+                if not t.is_setup and not t.is_teardown:
+                    also_include_ids.update(x.task_id for x in t.get_upstreams_only_setups_and_teardowns())
             if t.is_setup and not include_downstream:
-                also_include.extend(x for x in t.downstream_list if x.is_teardown)
+                also_include_ids.update(x.task_id for x in t.downstream_list if x.is_teardown)
 
+        also_include: list[Operator] = [self.task_dict[x] for x in also_include_ids]
         direct_upstreams: list[Operator] = []
         if include_direct_upstream:
             for t in itertools.chain(matched_tasks, also_include):
diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py
index cd296c0ddc..b6827fa541 100644
--- a/tests/models/test_dag.py
+++ b/tests/models/test_dag.py
@@ -3730,7 +3730,9 @@ class TestTaskClearingSetupTeardownBehavior:
         s2 >> w2 >> t2
 
         assert set(w1.get_upstreams_only_setups_and_teardowns()) == {s1, t1}
-        assert self.cleared_downstream(w1) == {s1, w1, t1, w2, t2}
+        # s2 is included because w2 is included
+        assert self.cleared_downstream(w1) == {s1, w1, t1, s2, w2, t2}
+        assert self.cleared_neither(w1) == {s1, w1, t1}
         assert set(w2.get_upstreams_only_setups_and_teardowns()) == {s2, t2}
         assert self.cleared_downstream(w2) == {s2, w2, t2}
 
@@ -3750,7 +3752,9 @@ class TestTaskClearingSetupTeardownBehavior:
         s3 >> w2 >> [t3a, t3b]
         s3 >> [t3a, t3b]
         assert set(w1.get_upstreams_only_setups_and_teardowns()) == {s1a, s1b, t1}
-        assert self.cleared_downstream(w1) == {s1a, s1b, w1, t1, t3a, t3b, w2, t2}
+        # since w2 is downstream of w1, w2 gets cleared.
+        # and since w2 gets cleared, we should also see s2 and s3 in here
+        assert self.cleared_downstream(w1) == {s1a, s1b, w1, t1, s3, t3a, t3b, w2, s2, t2}
         assert set(w2.get_upstreams_only_setups_and_teardowns()) == {s2, t2, s3, t3a, t3b}
         assert self.cleared_downstream(w2) == {s2, s3, w2, t2, t3a, t3b}
 
@@ -3897,3 +3901,132 @@ class TestTaskClearingSetupTeardownBehavior:
                 "my_setup", include_upstream=upstream, include_downstream=downstream
             ).tasks
         } == expected
+
+    def test_get_flat_relative_ids_two_tasks_diff_setup_teardowns_deeper(self):
+        with DAG(dag_id="test_dag", start_date=pendulum.now()) as dag:
+            s1, t1, s2, t2, w1, w2, s3, w3, t3 = self.make_tasks(dag, "s1, t1, s2, t2, w1, w2, s3, w3, t3")
+        s1 >> w1 >> t1
+        s1 >> t1
+        w1 >> w2
+
+        # with the below, s2 is not downstream of w1, but it's the setup for w2
+        # so it should be cleared when w1 is cleared
+        s2 >> w2 >> t2
+        s2 >> t2
+
+        assert set(w1.get_upstreams_only_setups_and_teardowns()) == {s1, t1}
+        assert set(w2.get_upstreams_only_setups_and_teardowns()) == {s2, t2}
+        assert self.cleared_downstream(w1) == {s1, w1, t1, s2, w2, t2}
+        assert self.cleared_downstream(w2) == {s2, w2, t2}
+
+        # now, what if s2 itself has a setup and teardown?
+        s3 >> s2 >> t3
+        s3 >> t3
+        # note that s3 is excluded because it's assumed that a setup won't have a setup
+        # so, we don't continue to recurse for setups after reaching the setups for
+        # the downstream work tasks
+        # but, t3 is included since it's a teardown for s2
+        assert self.cleared_downstream(w1) == {s1, w1, t1, s2, w2, t2, t3}
+
+    def test_clearing_behavior_multiple_setups_for_work_task(self):
+        with DAG(dag_id="test_dag", start_date=pendulum.now()) as dag:
+            s1, t1, s2, t2, w1, w2, s3, w3, t3 = self.make_tasks(dag, "s1, t1, s2, t2, w1, w2, s3, w3, t3")
+        s1 >> t1
+        s2 >> t2
+        s3 >> t3
+        s1 >> s2 >> s3 >> w1 >> w2 >> [t1, t2, t3]
+
+        assert self.cleared_downstream(w1) == {s1, s2, s3, w1, w2, t1, t2, t3}
+        assert self.cleared_downstream(w2) == {s1, s2, s3, w2, t1, t2, t3}
+        assert self.cleared_downstream(s3) == {s1, s2, s3, w1, w2, t1, t2, t3}
+        # even if we don't include upstream / downstream, setups and teardowns are cleared
+        assert self.cleared_neither(w2) == {s3, t3, s2, t2, s1, t1, w2}
+        assert self.cleared_neither(w1) == {s3, t3, s2, t2, s1, t1, w1}
+        # but, a setup doesn't formally have a setup, so if we only clear s3, say then its upstream setups
+        # are not also cleared
+        assert self.cleared_neither(s3) == {s3, t3}
+        assert self.cleared_neither(s2) == {s2, t2}
+
+    def test_clearing_behavior_multiple_setups_for_work_task2(self):
+        with DAG(dag_id="test_dag", start_date=pendulum.now()) as dag:
+            s1, t1, s2, t2, w1, w2, s3, w3, t3 = self.make_tasks(dag, "s1, t1, s2, t2, w1, w2, s3, w3, t3")
+        s1 >> t1
+        s2 >> t2
+        s3 >> t3
+        [s1, s2, s3] >> w1 >> w2 >> [t1, t2, t3]
+
+        assert self.cleared_downstream(w1) == {s1, s2, s3, w1, w2, t1, t2, t3}
+        assert self.cleared_downstream(w2) == {s1, s2, s3, w2, t1, t2, t3}
+
+    def test_clearing_behavior_more_tertiary_weirdness(self):
+        with DAG(dag_id="test_dag", start_date=pendulum.now()) as dag:
+            s1, t1, s2, t2, w1, w2, s3, t3 = self.make_tasks(dag, "s1, t1, s2, t2, w1, w2, s3, t3")
+        s1 >> t1
+        s2 >> t2
+        s1 >> w1 >> s2 >> w2 >> [t1, t2]
+        s2 >> w2 >> t2
+        s3 >> s2 >> t3
+        s3 >> t3
+
+        def sort(task_list):
+            return sorted(x.task_id for x in task_list)
+
+        assert set(w1.get_upstreams_only_setups_and_teardowns()) == {s1, t1}
+        # s2 is included because w2 is included
+        assert self.cleared_downstream(w1) == {s1, w1, t1, s2, w2, t2, t3}
+        assert self.cleared_downstream(w2) == {s1, t1, s2, w2, t2, t3}
+        # t3 is included since s2 is included and s2 >> t3
+        # but s3 not included because it's assumed that a setup doesn't have a setup
+        assert self.cleared_neither(w2) == {s1, w2, t1, s2, t2, t3}
+
+        # since we're clearing upstream, s3 is upstream of w2, so s3 and t3 are included
+        # even though w2 doesn't require them
+        # s2 and t2 are included for obvious reasons, namely that w2 requires s2
+        # and s1 and t1 are included for the same reason
+        # w1 included since it is upstream of w2
+        assert sort(self.cleared_upstream(w2)) == sort({s1, t1, s2, t2, s3, t3, w1, w2})
+
+        # t3 is included here since it's a teardown for s2
+        assert set(w2.get_upstreams_only_setups_and_teardowns()) == {s2, t2, s1, t1, t3}
+
+    def test_clearing_behavior_more_tertiary_weirdness2(self):
+        with DAG(dag_id="test_dag", start_date=pendulum.now()) as dag:
+            s1, t1, s2, t2, w1, w2, s3, t3 = self.make_tasks(dag, "s1, t1, s2, t2, w1, w2, s3, t3")
+        s1 >> t1
+        s2 >> t2
+        s1 >> w1 >> t1
+        s2 >> t1 >> t2
+
+        def sort(task_list):
+            return sorted(x.task_id for x in task_list)
+
+        # t2 included since downstream, but s2 not included since it's not required by t2
+        # and clearing teardown does not clear the setup
+        assert self.cleared_downstream(w1) == {s1, w1, t1, t2}
+
+        # even though t1 is cleared here, s2 and t2 are not "setup and teardown" for t1
+        # so they are not included
+        assert self.cleared_neither(w1) == {s1, w1, t1}
+        assert self.cleared_upstream(w1) == {s1, w1, t1}
+
+        # t1 does not have a setup or teardown
+        # but t2 is downstream so it's included
+        # and s2 is not included since clearing teardown does not clear the setup
+        assert self.cleared_downstream(t1) == {t1, t2}
+        # t1 does not have a setup or teardown
+        assert self.cleared_neither(t1) == {t1}
+        # s2 included since upstream, and t2 included since s2 included
+        assert self.cleared_upstream(t1) == {s1, t1, s2, t2, w1}
+
+    def test_clearing_behavior_just_teardown(self):
+        with DAG(dag_id="test_dag", start_date=pendulum.now()) as dag:
+            s1, t1 = self.make_tasks(dag, "s1, t1")
+        s1 >> t1
+        assert set(t1.get_upstreams_only_setups_and_teardowns()) == set()
+        assert self.cleared_upstream(t1) == {s1, t1}
+        assert self.cleared_downstream(t1) == {t1}
+        assert self.cleared_neither(t1) == {t1}
+        assert set(s1.get_upstreams_only_setups_and_teardowns()) == set()
+        assert self.cleared_upstream(s1) == {s1, t1}
+        assert self.cleared_downstream(s1) == {s1, t1}
+        assert self.cleared_neither(s1) == {s1, t1}


[airflow] 05/09: Rename `ExternalBranchPythonOperator` to `BranchExternalPythonOperator` (#33360)

Posted by ep...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ephraimanierobi pushed a commit to branch v2-7-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit e690f5c56158ce61f9b9183713529bbb8296d4de
Author: Elad Kalif <45...@users.noreply.github.com>
AuthorDate: Sun Aug 13 10:54:24 2023 +0300

    Rename `ExternalBranchPythonOperator` to `BranchExternalPythonOperator` (#33360)
    
    (cherry picked from commit 9a1624b93d97b3a62cee1efbe1c520699e1323ee)
---
 airflow/operators/python.py    | 2 +-
 tests/operators/test_python.py | 6 +++---
 2 files changed, 4 insertions(+), 4 deletions(-)

diff --git a/airflow/operators/python.py b/airflow/operators/python.py
index 479fb600c4..36ae816509 100644
--- a/airflow/operators/python.py
+++ b/airflow/operators/python.py
@@ -778,7 +778,7 @@ class ExternalPythonOperator(_BasePythonVirtualenvOperator):
             return None
 
 
-class ExternalBranchPythonOperator(ExternalPythonOperator, SkipMixin):
+class BranchExternalPythonOperator(ExternalPythonOperator, SkipMixin):
     """
     A workflow can "branch" or follow a path after the execution of this task,
     Extends ExternalPythonOperator, so expects to get Python:
diff --git a/tests/operators/test_python.py b/tests/operators/test_python.py
index 986f3ae510..69e68838e7 100644
--- a/tests/operators/test_python.py
+++ b/tests/operators/test_python.py
@@ -40,8 +40,8 @@ from airflow.models.baseoperator import BaseOperator
 from airflow.models.taskinstance import TaskInstance, clear_task_instances, set_current_context
 from airflow.operators.empty import EmptyOperator
 from airflow.operators.python import (
+    BranchExternalPythonOperator,
     BranchPythonOperator,
-    ExternalBranchPythonOperator,
     ExternalPythonOperator,
     PythonOperator,
     PythonVirtualenvOperator,
@@ -1108,8 +1108,8 @@ class TestExternalPythonOperator(BaseTestPythonVirtualenvOperator):
             task._read_result(path=mock.Mock())
 
 
-class TestExternalBranchPythonOperator(BaseTestPythonVirtualenvOperator):
-    opcls = ExternalBranchPythonOperator
+class TestBranchExternalPythonOperator(BaseTestPythonVirtualenvOperator):
+    opcls = BranchExternalPythonOperator
 
     @pytest.fixture(autouse=True)
     def setup_tests(self):


[airflow] 02/09: Modify pathspec version restriction (#33349)

Posted by ep...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ephraimanierobi pushed a commit to branch v2-7-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit 9405b081d7e85ef9fe971f8322f56246fe6418a4
Author: Kevin Mills <35...@users.noreply.github.com>
AuthorDate: Sun Aug 13 10:20:15 2023 -0500

    Modify pathspec version restriction (#33349)
    
    Allow pathspec >=0.9.0 to meet requirements of other packages that depend on newer versions of pathspec.  For example, DVC requires pathspec >=0.10.3.
    
    (cherry picked from commit 699b3f4d58f52d9dde1bc063d6f84b1982ccd1db)
---
 setup.cfg | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/setup.cfg b/setup.cfg
index cdaad2dc2a..16e9930a38 100644
--- a/setup.cfg
+++ b/setup.cfg
@@ -122,7 +122,7 @@ install_requires =
     opentelemetry-api==1.15.0
     opentelemetry-exporter-otlp
     packaging>=14.0
-    pathspec~=0.9.0
+    pathspec>=0.9.0
     pendulum>=2.0
     pluggy>=1.0
     psutil>=4.2.0


[airflow] 03/09: Make skip of trigger form in UI if no params are defined configurable (#33351)

Posted by ep...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ephraimanierobi pushed a commit to branch v2-7-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit 1cb3e52bb2a128f95f826952344926fbd76eceb1
Author: Jens Scheffler <95...@users.noreply.github.com>
AuthorDate: Sun Aug 13 14:57:07 2023 +0200

    Make skip of trigger form in UI if no params are defined configurable (#33351)
    
    * Make skip of trigger form in UI if no params are defined configurable
    
    * Review feedback, remove negating bool
    
    * Review feedback, remove negating bool
    
    * Add newsfragment
    
    (cherry picked from commit c0362923fd8250328eab6e60f0cf7e855bfd352e)
---
 airflow/config_templates/config.yml | 9 +++++++++
 airflow/www/views.py                | 3 ++-
 newsfragments/33351.significant.rst | 6 ++++++
 3 files changed, 17 insertions(+), 1 deletion(-)

diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml
index 362789b01b..8901383bb0 100644
--- a/airflow/config_templates/config.yml
+++ b/airflow/config_templates/config.yml
@@ -1779,6 +1779,15 @@ webserver:
       type: string
       example: "sha256"
       default: "md5"
+    show_trigger_form_if_no_params:
+      description: |
+        Behavior of the trigger DAG run button for DAGs without params. False to skip and trigger
+        without displaying a form to add a dag_run.conf, True to always display the form.
+        The form is displayed always if parameters are defined.
+      version_added: 2.7.0
+      type: boolean
+      example: ~
+      default: "False"
 email:
   description: |
     Configuration email backend and whether to
diff --git a/airflow/www/views.py b/airflow/www/views.py
index 5f12a94969..ad9a9da98a 100644
--- a/airflow/www/views.py
+++ b/airflow/www/views.py
@@ -2031,6 +2031,7 @@ class Airflow(AirflowBaseView):
                     form_fields[k]["schema"]["custom_html_form"]
                 )
         ui_fields_defined = any("const" not in f["schema"] for f in form_fields.values())
+        show_trigger_form_if_no_params = conf.getboolean("webserver", "show_trigger_form_if_no_params")
 
         if not dag_orm:
             flash(f"Cannot find dag {dag_id}")
@@ -2057,7 +2058,7 @@ class Airflow(AirflowBaseView):
             if isinstance(run_conf, dict) and any(run_conf)
         }
 
-        if request.method == "GET" and ui_fields_defined:
+        if request.method == "GET" and (ui_fields_defined or show_trigger_form_if_no_params):
             # Populate conf textarea with conf requests parameter, or dag.params
             default_conf = ""
 
diff --git a/newsfragments/33351.significant.rst b/newsfragments/33351.significant.rst
new file mode 100644
index 0000000000..d8f91c4c20
--- /dev/null
+++ b/newsfragments/33351.significant.rst
@@ -0,0 +1,6 @@
+The trigger UI form is skipped in web UI with 2.7.0 if no parameters are defined in a DAG.
+
+If you are using ``dag_run.conf`` dictionary and web UI JSON entry to run your DAG you should either:
+
+* `Add params to your DAG <https://airflow.apache.org/docs/apache-airflow/stable/core-concepts/params.html#use-params-to-provide-a-trigger-ui-form>`_
+* Enable the new configuration ``show_trigger_form_if_no_params`` to bring back old behaviour


[airflow] 06/09: Limit aiobotocore for eager upgrade to prevent pip backtracking (#33362)

Posted by ep...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

ephraimanierobi pushed a commit to branch v2-7-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit cbcf8bc207f161ea1755cc2b1fa9c8a47d2e3477
Author: Jarek Potiuk <ja...@potiuk.com>
AuthorDate: Sun Aug 13 11:50:53 2023 +0200

    Limit aiobotocore for eager upgrade to prevent pip backtracking (#33362)
    
    The new aiobotocore release 2.6.0 caused `pip` to backtract when
    finding `good` set of dependencies. This limitation should stop
    the backtracking until other dependncies causing the conflicts
    will be updated.
    
    (cherry picked from commit c6c8c1afa5e0a04e8921ce9cb4979c3120b2d5be)
---
 Dockerfile.ci | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/Dockerfile.ci b/Dockerfile.ci
index 10d786a2fa..def7c6345b 100644
--- a/Dockerfile.ci
+++ b/Dockerfile.ci
@@ -1375,8 +1375,8 @@ RUN echo "Airflow version: ${AIRFLOW_VERSION}"
 # Those are additional constraints that are needed for some extras but we do not want to
 # force them on the main Airflow package. Currently we need no extra limits as PIP 23.1+ has much better
 # dependency resolution and we do not need to limit the versions of the dependencies
-# !!! MAKE SURE YOU SYNCHRONIZE THE LIST BETWEEN: Dockerfile, Dockerfile.ci
-ARG EAGER_UPGRADE_ADDITIONAL_REQUIREMENTS=""
+# aiobotocore is limited temporarily until it stops backtracking pip
+ARG EAGER_UPGRADE_ADDITIONAL_REQUIREMENTS="aiobotocore<2.6.0"
 ARG UPGRADE_TO_NEWER_DEPENDENCIES="false"
 ARG VERSION_SUFFIX_FOR_PYPI=""