From ee823c67ac4693b29d538ab8d8b98021bc4cd34f Mon Sep 17 00:00:00 2001 From: Victor Chiapaikeo Date: Mon, 17 Apr 2023 12:46:32 -0400 Subject: [PATCH] Updated app to support configuring the caching hash method for FIPS v2 (#30675) (cherry picked from commit 522083cab4705789c79466933238e3874253c8a6) --- airflow/config_templates/config.yml | 7 +++ airflow/config_templates/default_airflow.cfg | 4 ++ airflow/models/serialized_dag.py | 4 +- airflow/utils/hashlib_wrapper.py | 37 ++++++++++++++ airflow/www/app.py | 6 +-- airflow/www/extensions/init_cache.py | 52 ++++++++++++++++++++ newsfragments/28846.misc.rst | 1 + tests/www/test_app.py | 25 ++++++++++ 8 files changed, 130 insertions(+), 6 deletions(-) create mode 100644 airflow/utils/hashlib_wrapper.py create mode 100644 airflow/www/extensions/init_cache.py create mode 100644 newsfragments/28846.misc.rst diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index 43bad45e41d12..bf88cdaa7998e 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -1692,6 +1692,13 @@ webserver: type: string example: ~ default: "5 per 40 second" + caching_hash_method: + description: | + The caching algorithm used by the webserver. Must be a valid hashlib function name. + version_added: + type: string + example: "sha256" + default: "md5" email: description: | diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index e5d92429c0dc0..684a78992d6c2 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -856,6 +856,10 @@ auth_rate_limited = True # Rate limit for authentication endpoints. auth_rate_limit = 5 per 40 second +# The caching algorithm used by the webserver. Must be a valid hashlib function name. +# Example: caching_hash_method = sha256 +caching_hash_method = md5 + [email] # Configuration email backend and whether to diff --git a/airflow/models/serialized_dag.py b/airflow/models/serialized_dag.py index a6932793673b6..d3f49c64d3950 100644 --- a/airflow/models/serialized_dag.py +++ b/airflow/models/serialized_dag.py @@ -18,7 +18,6 @@ """Serialized DAG table in database.""" from __future__ import annotations -import hashlib import logging import zlib from datetime import datetime, timedelta @@ -35,6 +34,7 @@ from airflow.serialization.serialized_objects import DagDependency, SerializedDAG from airflow.settings import COMPRESS_SERIALIZED_DAGS, MIN_SERIALIZED_DAG_UPDATE_INTERVAL, json from airflow.utils import timezone +from airflow.utils.hashlib_wrapper import md5 from airflow.utils.session import NEW_SESSION, provide_session from airflow.utils.sqlalchemy import UtcDateTime @@ -102,7 +102,7 @@ def __init__(self, dag: DAG, processor_subdir: str | None = None) -> None: dag_data = SerializedDAG.to_dict(dag) dag_data_json = json.dumps(dag_data, sort_keys=True).encode("utf-8") - self.dag_hash = hashlib.md5(dag_data_json).hexdigest() + self.dag_hash = md5(dag_data_json, usedforsecurity=False).hexdigest() if COMPRESS_SERIALIZED_DAGS: self._data = None diff --git a/airflow/utils/hashlib_wrapper.py b/airflow/utils/hashlib_wrapper.py new file mode 100644 index 0000000000000..2415f3d00e94e --- /dev/null +++ b/airflow/utils/hashlib_wrapper.py @@ -0,0 +1,37 @@ +# +# 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 + +import hashlib + +from airflow import PY39 + + +def md5(data: bytes, *, usedforsecurity: bool | None = None): + """ + Safely allows calling the hashlib.md5 function with the "usedforsecurity" param. + :param data: The data to hash. + :param usedforsecurity: The value to pass to the md5 function's "usedforsecurity" param. + Defaults to None. + :return: The hashed value. + :rtype: _Hash + """ + if PY39 and usedforsecurity is not None: + return hashlib.md5(data, usedforsecurity=usedforsecurity) # type: ignore + else: + return hashlib.md5(data) diff --git a/airflow/www/app.py b/airflow/www/app.py index 5283718089100..246381f003e4e 100644 --- a/airflow/www/app.py +++ b/airflow/www/app.py @@ -19,11 +19,9 @@ import warnings from datetime import timedelta -from tempfile import gettempdir from flask import Flask from flask_appbuilder import SQLA -from flask_caching import Cache from flask_wtf.csrf import CSRFProtect from markupsafe import Markup from sqlalchemy.engine.url import make_url @@ -38,6 +36,7 @@ from airflow.utils.json import AirflowJsonProvider from airflow.www.extensions.init_appbuilder import init_appbuilder from airflow.www.extensions.init_appbuilder_links import init_appbuilder_links +from airflow.www.extensions.init_cache import init_cache from airflow.www.extensions.init_dagbag import init_dagbag from airflow.www.extensions.init_jinja_globals import init_jinja_globals from airflow.www.extensions.init_manifest_files import configure_manifest_files @@ -143,8 +142,7 @@ def create_app(config=None, testing=False): init_robots(flask_app) - cache_config = {"CACHE_TYPE": "flask_caching.backends.filesystem", "CACHE_DIR": gettempdir()} - Cache(app=flask_app, config=cache_config) + init_cache(flask_app) init_flash_views(flask_app) diff --git a/airflow/www/extensions/init_cache.py b/airflow/www/extensions/init_cache.py new file mode 100644 index 0000000000000..84d952dd7120e --- /dev/null +++ b/airflow/www/extensions/init_cache.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. +from __future__ import annotations + +import hashlib +from tempfile import gettempdir + +from flask_caching import Cache + +from airflow.configuration import conf +from airflow.exceptions import AirflowConfigException + +HASH_METHOD_MAPPING = { + "md5": hashlib.md5, + "sha1": hashlib.sha1, + "sha224": hashlib.sha224, + "sha256": hashlib.sha256, + "sha384": hashlib.sha384, + "sha512": hashlib.sha512, +} + + +def init_cache(app): + webserver_caching_hash_method = conf.get( + section="webserver", key="CACHING_HASH_METHOD", fallback="md5" + ).casefold() + cache_config = {"CACHE_TYPE": "flask_caching.backends.filesystem", "CACHE_DIR": gettempdir()} + + mapped_hash_method = HASH_METHOD_MAPPING.get(webserver_caching_hash_method) + + if mapped_hash_method is None: + raise AirflowConfigException( + f"Unsupported webserver caching hash method: `{webserver_caching_hash_method}`." + ) + + cache_config["CACHE_OPTIONS"] = {"hash_method": mapped_hash_method} + + Cache(app=app, config=cache_config) diff --git a/newsfragments/28846.misc.rst b/newsfragments/28846.misc.rst new file mode 100644 index 0000000000000..14d072c8779d4 --- /dev/null +++ b/newsfragments/28846.misc.rst @@ -0,0 +1 @@ +Various updates for FIPS-compliance when running Airflow in Python 3.9+. This includes a new webserver option, ``caching_hash_method``, for changing the default flask caching method. diff --git a/tests/www/test_app.py b/tests/www/test_app.py index 29b8653397c5d..61bfae3e12a87 100644 --- a/tests/www/test_app.py +++ b/tests/www/test_app.py @@ -17,6 +17,7 @@ # under the License. from __future__ import annotations +import hashlib import runpy import sys from datetime import timedelta @@ -27,6 +28,7 @@ from werkzeug.test import create_environ from werkzeug.wrappers import Response +from airflow.exceptions import AirflowConfigException from airflow.www import app as application from tests.test_utils.config import conf_vars from tests.test_utils.decorators import dont_initialize_flask_app_submodules @@ -228,6 +230,29 @@ def test_correct_default_is_set_for_cookie_samesite(self): app = application.cached_app(testing=True) assert app.config["SESSION_COOKIE_SAMESITE"] == "Lax" + @pytest.mark.parametrize( + "hash_method, result, exception", + [ + ("sha512", hashlib.sha512, None), + ("sha384", hashlib.sha384, None), + ("sha256", hashlib.sha256, None), + ("sha224", hashlib.sha224, None), + ("sha1", hashlib.sha1, None), + ("md5", hashlib.md5, None), + (None, hashlib.md5, None), + ("invalid", None, AirflowConfigException), + ], + ) + @dont_initialize_flask_app_submodules + def test_should_respect_caching_hash_method(self, hash_method, result, exception): + with conf_vars({("webserver", "caching_hash_method"): hash_method}): + if exception: + with pytest.raises(expected_exception=exception): + app = application.cached_app(testing=True) + else: + app = application.cached_app(testing=True) + assert next(iter(app.extensions["cache"])).cache._hash_method == result + class TestFlaskCli: @dont_initialize_flask_app_submodules(skip_all_except=["init_appbuilder"])