Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
22 changes: 18 additions & 4 deletions providers/fab/docs/auth-manager/security.rst
Original file line number Diff line number Diff line change
Expand Up @@ -70,10 +70,24 @@ uses per-webserver default limit of 5 requests per 40 second fixed window. By de
rate limits is used between the gunicorn processes you run so rate-limit is applied separately for each process,
so assuming random distribution of the requests by gunicorn with single webserver instance and default 4
gunicorn workers, the effective rate limit is 5 x 4 = 20 requests per 40 second window (more or less).
However you can configure the rate limit to be shared between the processes by using rate limit storage via
setting the ``RATELIMIT_*`` configuration settings in ``webserver_config.py``.
For example, to use Redis as a rate limit storage you can use the following configuration (you need
to set ``redis_host`` to your Redis instance)

However, you can configure the rate limit to be shared between processes by using a common storage backend like Redis.

Example (using airflow.cfg):
.. code-block:: ini

[fab]
auth_rate_limit_storage_uri = redis://redis_host:6379/0
auth_rate_limit_storage_options = {"socket_timeout": 5}

Example (using environment variables):
.. code-block:: bash

export AIRFLOW__FAB__AUTH_RATE_LIMIT_STORAGE_URI="redis://redis_host:6379/0"
export AIRFLOW__FAB__AUTH_RATE_LIMIT_STORAGE_OPTIONS='{"socket_timeout": 5}'

Legacy Method:
Alternatively, these settings can be configured by adding them directly to your webserver_config.py file

.. code-block:: python

Expand Down
20 changes: 20 additions & 0 deletions providers/fab/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -245,6 +245,26 @@ config:
type: integer
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should update the documentation of the provider (providers/fab/docs/auth-manager/security.rst, Rate limiting section) to reflect this change.

example: ~
default: "1"
auth_rate_limit_storage_uri:
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I’d clarify in the description that only the Redis and Redis Cluster backends are supported.
Additionally, we should ensure that the redis extra is included in the fab pyproject.toml:

flask_limiter[redis]>3,<4,!=3.13

It might also be helpful to include examples of both supported backends, e.g.: redis://localhost:6379 or redis+cluster://redis0:6379,redis1:6379

description: |
Storage backend for rate limit data. The default backend is in-memory storage.
When specifying a storage URI, only Redis and Redis Cluster backends are supported.
Examples: ``redis://localhost:6379`` or ``redis+cluster://redis0:6379,redis1:6379``.
See `Flask-Limiter Storage Options
<https://flask-limiter.readthedocs.io/en/stable/#storage-backends>`__ for more details.
version_added: 3.0.1
type: string
example: ~
default: "memory://"
auth_rate_limit_storage_options:
description: |
Storage options for rate limit data storage backend.
See `Flask-Limiter Storage Options
<https://flask-limiter.readthedocs.io/en/stable/#storage-backends>`__ for more details.
version_added: 3.0.1
type: string
example: ~
default: null

auth-managers:
- airflow.providers.fab.auth_manager.fab_auth_manager.FabAuthManager
2 changes: 1 addition & 1 deletion providers/fab/pyproject.toml
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ dependencies = [

# https://github.com/dpgaspar/Flask-AppBuilder/blob/release/4.6.3/setup.py#L54C8-L54C26
# with an exclusion to account for https://github.com/alisaifee/flask-limiter/issues/479
"flask_limiter>3,<4,!=3.13",
"flask_limiter[redis]>3,<4,!=3.13",
]

# The optional dependencies should be modified in place in the generated file
Expand Down
14 changes: 14 additions & 0 deletions providers/fab/src/airflow/providers/fab/get_provider_info.py
Original file line number Diff line number Diff line change
Expand Up @@ -177,6 +177,20 @@ def get_provider_info():
"example": None,
"default": "1",
},
"auth_rate_limit_storage_uri": {
"description": "Storage backend for rate limit data. The default backend is in-memory storage.\nWhen specifying a storage URI, only Redis and Redis Cluster backends are supported.\nExamples: ``redis://localhost:6379`` or ``redis+cluster://redis0:6379,redis1:6379``.\nSee `Flask-Limiter Storage Options\n<https://flask-limiter.readthedocs.io/en/stable/#storage-backends>`__ for more details.\n",
"version_added": "3.0.1",
"type": "string",
"example": None,
"default": "memory://",
},
"auth_rate_limit_storage_options": {
"description": "Storage options for rate limit data storage backend.\nSee `Flask-Limiter Storage Options\n<https://flask-limiter.readthedocs.io/en/stable/#storage-backends>`__ for more details.\n",
"version_added": "3.0.1",
"type": "string",
"example": None,
"default": None,
},
},
}
},
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
# mypy: disable-error-code=var-annotated
from __future__ import annotations

import json
import logging
from functools import reduce
from typing import TYPE_CHECKING
Expand Down Expand Up @@ -151,6 +152,18 @@ def __init__(
self.update_perms = conf.getboolean("fab", "UPDATE_FAB_PERMS")
self.auth_rate_limited = conf.getboolean("fab", "AUTH_RATE_LIMITED")
self.auth_rate_limit = conf.get("fab", "AUTH_RATE_LIMIT")
self.auth_rate_limit_storage_uri = conf.get(
"fab", "AUTH_RATE_LIMIT_STORAGE_URI", fallback="memory://"
)
self.auth_rate_limit_storage_options = conf.get("fab", "AUTH_RATE_LIMIT_STORAGE_OPTIONS", fallback={})
if isinstance(self.auth_rate_limit_storage_options, str):
try:
self.auth_rate_limit_storage_options = json.loads(self.auth_rate_limit_storage_options)
except (json.JSONDecodeError, TypeError):
log.error(
"Could not parse AUTH_RATE_LIMIT_STORAGE_OPTIONS json string, defaulting to empty dict"
)
self.auth_rate_limit_storage_options = {}
if app is not None:
self.init_app(app, session)

Expand All @@ -173,6 +186,8 @@ def init_app(self, app, session):
app.config.setdefault("FAB_STATIC_URL_PATH", self.static_url_path)
app.config.setdefault("AUTH_RATE_LIMITED", self.auth_rate_limited)
app.config.setdefault("AUTH_RATE_LIMIT", self.auth_rate_limit)
app.config.setdefault("AUTH_RATE_LIMIT_STORAGE_URI", self.auth_rate_limit_storage_uri)
app.config.setdefault("AUTH_RATE_LIMIT_STORAGE_OPTIONS", self.auth_rate_limit_storage_options)

self.base_template = app.config.get("FAB_BASE_TEMPLATE", self.base_template)
self.static_folder = app.config.get("FAB_STATIC_FOLDER", self.static_folder)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,12 @@ def before_request():
g.user = get_auth_manager().get_user()

def create_limiter(self) -> Limiter:
limiter = Limiter(key_func=current_app.config.get("RATELIMIT_KEY_FUNC", get_remote_address))
storage_uri, storage_options = self._get_auth_storage_uri(), self._get_auth_storage_options()
limiter = Limiter(
key_func=current_app.config.get("RATELIMIT_KEY_FUNC", get_remote_address),
storage_uri=storage_uri,
storage_options=storage_options,
)
limiter.init_app(current_app)
return limiter

Expand Down Expand Up @@ -110,3 +115,9 @@ def _get_auth_manager_is_authorized_method(self, fab_resource_name: str) -> Call
resource_name=fab_resource_name,
user=user,
)

def _get_auth_storage_uri(self) -> str:
return current_app.config.get("AUTH_RATE_LIMIT_STORAGE_URI", "memory://")

def _get_auth_storage_options(self) -> dict:
return current_app.config.get("AUTH_RATE_LIMIT_STORAGE_OPTIONS", {})
53 changes: 53 additions & 0 deletions providers/fab/tests/unit/fab/www/test_app.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
# 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.

from __future__ import annotations

from airflow.providers.fab.www.app import create_app

from tests_common.test_utils.config import conf_vars


class TestFabAppRateLimitConfig:
@conf_vars({})
def test_rate_limit_config_defaults(self):
flask_app = create_app(enable_plugins=False)
assert flask_app.config["AUTH_RATE_LIMIT_STORAGE_URI"] == "memory://"
assert flask_app.config["AUTH_RATE_LIMIT_STORAGE_OPTIONS"] == {}

@conf_vars(
{
("fab", "auth_rate_limit_storage_uri"): "redis://my-redis-host:6379/1",
("fab", "auth_rate_limit_storage_options"): '{"socket_timeout": 10}',
}
)
def test_rate_limit_config_custom_redis(self):
flask_app = create_app(enable_plugins=False)

assert flask_app.config["AUTH_RATE_LIMIT_STORAGE_URI"] == "redis://my-redis-host:6379/1"
expected_options = {"socket_timeout": 10}
assert flask_app.config["AUTH_RATE_LIMIT_STORAGE_OPTIONS"] == expected_options

@conf_vars(
{
("fab", "auth_rate_limit_storage_options"): '{"invalid_json": "missing_quote',
}
)
def test_rate_limit_config_bad_json(self):
flask_app = create_app(enable_plugins=False)

assert flask_app.config["AUTH_RATE_LIMIT_STORAGE_OPTIONS"] == {}
2 changes: 1 addition & 1 deletion providers/fab/www-hash.txt
Original file line number Diff line number Diff line change
@@ -1 +1 @@
cc109e95a25c1bb018a85a6a40859234398d5a9ac5bf317197eed3e3ece64b0d
a2b04310801a9738df361eb5396108b83680d48a6b611bef80121d43b615bbb0
Loading