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
7 changes: 7 additions & 0 deletions airflow/config_templates/config.yml
Original file line number Diff line number Diff line change
Expand Up @@ -1591,6 +1591,13 @@ webserver:
type: boolean
example: ~
default: "False"
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 @@ -818,6 +818,10 @@ enable_swagger_ui = True
# Boolean for running Internal API in the webserver.
run_internal_api = False

# 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 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):
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
38 changes: 38 additions & 0 deletions airflow/utils/hashlib_wrapper.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
#
# 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 sqlalchemy.engine.url import make_url

Expand All @@ -36,6 +34,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 @@ -134,8 +133,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,
}
Comment on lines +27 to +34
Copy link
Member

Choose a reason for hiding this comment

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

This doesn’t seem to be necessary; we could use hashlib.new() or even getattr.

Copy link
Author

Choose a reason for hiding this comment

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

Hmm unless I'm misunderstanding what you mean, this is needed because the flask_caching config takes a reference to the hashlib function itself. I was also recommended against hashlib.new() since it is much less performant.

Copy link
Member

Choose a reason for hiding this comment

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

getattr is the right answer here then 👍

Copy link
Member

@uranusjr uranusjr Jan 31, 2023

Choose a reason for hiding this comment

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

hashlib.new() should not be preferred when you know what argument you’re using. hashlib.md5(...) is more performant than hashlib.new("md5", ...), but putting the functions behind a dict and getattr(hashlib, "md5")(...) both negate most of the performance gain.

To put it in another way:

hashlib.md5(...)  # Better.
hashlib.new("md5")(...)
HASH_METHOD_MAPPING = {
    "md5": hashlib.md5,
    ...
}
HASH_METHOD_MAPPING[key]  # Better.

HASH_METHOD_MAPPING = {
    "md5": hashlib.new("md5"),
    ...
}
HASH_METHOD_MAPPING[key]

# Assuming HASH_METHOD_MAPPING[key]

But these three are more or less the same.

HASH_METHOD_MAPPING = {
    "md5": hashlib.md5,
    ...
}
HASH_METHOD_MAPPING[key]

hashlib.new(key)

getattr(hashlib, key)

Copy link
Author

Choose a reason for hiding this comment

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

I'm not planning on changing anything related to this. The mapping gives us a reference to the actual function needed, which hashlib.new() doesn't do, and if I used getattr, I could give a config value of "name", or "new", or "file" and it would just pass that into flask cache. I'd rather just have a whitelist and give a descriptive error message rather than depending on it blowing up flask cache.



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