Skip to content

Commit

Permalink
Updated app to support configuring the caching hash method for FIPS v2 (
Browse files Browse the repository at this point in the history
  • Loading branch information
vchiapaikeo authored Apr 17, 2023
1 parent 6b5db07 commit 522083c
Show file tree
Hide file tree
Showing 8 changed files with 130 additions and 6 deletions.
7 changes: 7 additions & 0 deletions airflow/config_templates/config.yml
Original file line number Diff line number Diff line change
Expand Up @@ -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: |
Expand Down
4 changes: 4 additions & 0 deletions airflow/config_templates/default_airflow.cfg
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
4 changes: 2 additions & 2 deletions airflow/models/serialized_dag.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
"""Serialized DAG table in database."""
from __future__ import annotations

import hashlib
import logging
import zlib
from datetime import datetime, timedelta
Expand All @@ -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

Expand Down Expand Up @@ -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
Expand Down
37 changes: 37 additions & 0 deletions airflow/utils/hashlib_wrapper.py
Original file line number Diff line number Diff line change
@@ -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)
6 changes: 2 additions & 4 deletions airflow/www/app.py
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -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)

Expand Down
52 changes: 52 additions & 0 deletions airflow/www/extensions/init_cache.py
Original file line number Diff line number Diff line change
@@ -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)
1 change: 1 addition & 0 deletions newsfragments/28846.misc.rst
Original file line number Diff line number Diff line change
@@ -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.
25 changes: 25 additions & 0 deletions tests/www/test_app.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
# under the License.
from __future__ import annotations

import hashlib
import runpy
import sys
from datetime import timedelta
Expand All @@ -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
Expand Down Expand Up @@ -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"])
Expand Down

0 comments on commit 522083c

Please sign in to comment.