You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2023/01/11 00:23:07 UTC

[GitHub] [airflow] shyft-mike opened a new pull request, #28846: Updated app to support configuring the caching hash method for FIPS

shyft-mike opened a new pull request, #28846:
URL: https://github.com/apache/airflow/pull/28846

   This was created to address issues that occurred in a FIPS enabled environment. This allows you to set the CACHING_HASH_METHOD config value to override the default of md5. Also updates the serialized_dag hash code to work even in a FIPS environment.
   
   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of an existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #14966
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code changes, an Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvement+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in a newsfragment file, named `{pr_number}.significant.rst` or `{issue_number}.significant.rst`, in [newsfragments](https://github.com/apache/airflow/tree/main/newsfragments).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] uranusjr commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1066606000


##########
airflow/models/serialized_dag.py:
##########
@@ -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 = hashlib.new("md5", data=dag_data_json, usedforsecurity=False).hexdigest()

Review Comment:
   ```suggestion
           self.dag_hash = hashlib.md5(dag_data_json, usedforsecurity=False).hexdigest()
   ```
   
   The argument is available in all algorithms in hashlib, and according to the documentation:
   
   > The named constructors are much faster than `new()` and should be preferred.
   
   https://docs.python.org/3/library/hashlib.html#hashlib.new



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] jasonwashburn commented on pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "jasonwashburn (via GitHub)" <gi...@apache.org>.
jasonwashburn commented on PR #28846:
URL: https://github.com/apache/airflow/pull/28846#issuecomment-1478034756

   Looking forward to this update, is there still more to do here? Help needed?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #28846:
URL: https://github.com/apache/airflow/pull/28846#issuecomment-1492874081

   @shyft-mike ?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1089897466


##########
airflow/config_templates/config.yml:
##########
@@ -271,8 +270,8 @@ core:
       default: "0"
     default_task_retry_delay:
       description: |
-          The number of seconds each task is going to wait by default between retries. Can be overridden at

Review Comment:
   Can you please remove the unrelated changes to a separate PR ? I understand this is done by your editor automatically and it's cool to habve things fixed, but there are certain properties of such changes (for example being cherry-pickable) that calls for separating unrelated changes to separate PRs.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] uranusjr commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1090136984


##########
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,
+}

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



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] uranusjr commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1090135941


##########
airflow/config_templates/config.yml:
##########
@@ -1536,6 +1536,13 @@ webserver:
       type: boolean
       example: ~
       default: "False"
+    caching_hash_method:
+      description: |
+        The caching algorithm used by the webserver.

Review Comment:
   Should add a note that this value must be something available in `hashlib`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] vchiapaikeo commented on pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "vchiapaikeo (via GitHub)" <gi...@apache.org>.
vchiapaikeo commented on PR #28846:
URL: https://github.com/apache/airflow/pull/28846#issuecomment-1510532606

   @eladkal asked me to pick up because it seems like some folks are waiting on it and the PR went stale. I  cherry picked and fix merge conflicts [in this PR](https://github.com/apache/airflow/pull/30675). Not sure where we landed with the `getattr` vs `hashlib.new()` vs the dictionary indexing debate. Happy to do what the maintainers agree to but for now, I left @shyft-mike's approach as is.
   
   Moving this conversation to this PR: https://github.com/apache/airflow/pull/30675


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #28846:
URL: https://github.com/apache/airflow/pull/28846#issuecomment-1478055833

   > Looking forward to this update, is there still more to do here? Help needed?
   
   Conflicts resolution by @shyft-mike and confirmation from @uranusjr and @jedcunningham 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] uranusjr commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1090136514


##########
airflow/compat/hashlib_wrapper.py:
##########
@@ -0,0 +1,45 @@
+#
+# 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
+import sys
+from typing import Any
+
+# Check if "usedforsecurity" is available for hashlib
+if sys.version_info >= (3, 9):
+    HAS_USEDFORSECURITY = True
+else:
+    HAS_USEDFORSECURITY = False
+
+
+def md5(data: Any, used_for_security: bool | None = None):

Review Comment:
   Also I think `data` should not be `Any`? The stdlib documentation seems to indicate it must be `bytes`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #28846:
URL: https://github.com/apache/airflow/pull/28846#issuecomment-1511729576

   Implemented in https://github.com/apache/airflow/pull/30675


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1066645340


##########
airflow/models/serialized_dag.py:
##########
@@ -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 = hashlib.new("md5", data=dag_data_json, usedforsecurity=False).hexdigest()

Review Comment:
   Also we cannot use it unconditionally. usedforsecurity is available as of airflow 3.9 and we are supporting 3.7+.
   So if we want to add it, we need to add a compat library that will use it properly depending on Python version.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] uranusjr commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1091586216


##########
airflow/compat/hashlib_wrapper.py:
##########
@@ -0,0 +1,45 @@
+#
+# 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
+import sys
+from typing import Any
+
+# Check if "usedforsecurity" is available for hashlib
+if sys.version_info >= (3, 9):
+    HAS_USEDFORSECURITY = True
+else:
+    HAS_USEDFORSECURITY = False
+
+
+def md5(data: Any, used_for_security: bool | None = None):

Review Comment:
   Naming a submodule with the same name as a top-level module is not a problem (not in Python 3 anyway) since `airflow.compat.hashlib` and `hashlib` can never conflict. The `_wrapper` or `_compat` suffix is simply redundant. If you really feel strongly about having a suffix (this particular case is slightly different from `functools` since the `md5` function is a wrapper, not a shim), you can choose to put the module in `airflow.utils` or even directly under `airflow` (a la `airflow.typing_compat`) instead.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] uranusjr commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1091588434


##########
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,
+}

Review Comment:
   `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.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1082853296


##########
airflow/models/serialized_dag.py:
##########
@@ -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 = hashlib.new("md5", data=dag_data_json, usedforsecurity=False).hexdigest()

Review Comment:
   Look for PY38 for example



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "Taragolis (via GitHub)" <gi...@apache.org>.
Taragolis commented on PR #28846:
URL: https://github.com/apache/airflow/pull/28846#issuecomment-1436011429

   @uranusjr @jedcunningham are you satisfied with the changes?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] uranusjr commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1091593375


##########
airflow/compat/hashlib_wrapper.py:
##########
@@ -0,0 +1,43 @@
+#
+# 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
+import sys
+
+# Check if "usedforsecurity" is available for hashlib
+if sys.version_info >= (3, 9):
+    HAS_USEDFORSECURITY = True
+else:
+    HAS_USEDFORSECURITY = False
+
+
+def md5(data: bytes, usedforsecurity: bool | None = None):

Review Comment:
   ```suggestion
   def md5(data: bytes, *, usedforsecurity: bool | None = None):
   ```
   
   This requires the callee to name the bool



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] shyft-mike commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "shyft-mike (via GitHub)" <gi...@apache.org>.
shyft-mike commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1091116159


##########
airflow/compat/hashlib_wrapper.py:
##########
@@ -0,0 +1,45 @@
+#
+# 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
+import sys
+from typing import Any
+
+# Check if "usedforsecurity" is available for hashlib
+if sys.version_info >= (3, 9):
+    HAS_USEDFORSECURITY = True
+else:
+    HAS_USEDFORSECURITY = False
+
+
+def md5(data: Any, used_for_security: bool | None = None):

Review Comment:
   I have made these suggested changes, leaving `_wrapper` at the end.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #28846:
URL: https://github.com/apache/airflow/pull/28846#issuecomment-1407598117

   Ah. One more small thing please @shyft-mike if I may - can you please add a newsfragment (misc?) explaining that on Python 3.9+ airflow is now FIPS-compatible when it comes to using hashing algorithms? I think it's worth mentioning and if you add a newsftagment (see https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#step-4-prepare-pr for details) it will make its way to release notes in a more prominentn way.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] uranusjr commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1090135796


##########
airflow/compat/hashlib_wrapper.py:
##########
@@ -0,0 +1,45 @@
+#
+# 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
+import sys
+from typing import Any
+
+# Check if "usedforsecurity" is available for hashlib
+if sys.version_info >= (3, 9):
+    HAS_USEDFORSECURITY = True
+else:
+    HAS_USEDFORSECURITY = False
+
+
+def md5(data: Any, used_for_security: bool | None = None):

Review Comment:
   I feel we should match the original signature as much as possible, namely
   
   1. Match the `usedforsecurity` argument name
   2. Name this module `hashlib` (removing the `_wrapper` part)
   
   Also would be a good idea to make `usedforsecurity` a keyword-only argument.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] boring-cyborg[bot] commented on pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on PR #28846:
URL: https://github.com/apache/airflow/pull/28846#issuecomment-1378071135

   Congratulations on your first Pull Request and welcome to the Apache Airflow community! If you have any issues or are unsure about any anything please check our Contribution Guide (https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst)
   Here are some useful points:
   - Pay attention to the quality of your code (flake8, mypy and type annotations). Our [pre-commits]( https://github.com/apache/airflow/blob/main/STATIC_CODE_CHECKS.rst#prerequisites-for-pre-commit-hooks) will help you with that.
   - In case of a new feature add useful documentation (in docstrings or in `docs/` directory). Adding a new operator? Check this short [guide](https://github.com/apache/airflow/blob/main/docs/apache-airflow/howto/custom-operator.rst) Consider adding an example DAG that shows how users should use it.
   - Consider using [Breeze environment](https://github.com/apache/airflow/blob/main/BREEZE.rst) for testing locally, it's a heavy docker but it ships with a working Airflow and a lot of integrations.
   - Be patient and persistent. It might take some time to get a review or get the final approval from Committers.
   - Please follow [ASF Code of Conduct](https://www.apache.org/foundation/policies/conduct) for all communication including (but not limited to) comments on Pull Requests, Mailing list and Slack.
   - Be sure to read the [Airflow Coding style]( https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#coding-style-and-best-practices).
   Apache Airflow is a community-driven project and together we are making it better 🚀.
   In case of doubts contact the developers at:
   Mailing List: dev@airflow.apache.org
   Slack: https://s.apache.org/airflow-slack
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] shyft-mike commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by GitBox <gi...@apache.org>.
shyft-mike commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1082843374


##########
airflow/models/serialized_dag.py:
##########
@@ -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 = hashlib.new("md5", data=dag_data_json, usedforsecurity=False).hexdigest()

Review Comment:
   Ah okay. I went with "new" since it used kwargs, so passing usedforsecurity would either work or just get ignored. But I see the precommit linting fails on that being unexpected.
   Are there any examples currently of where it makes decisions based on the python version?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] jedcunningham commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "jedcunningham (via GitHub)" <gi...@apache.org>.
jedcunningham commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1091142073


##########
airflow/models/serialized_dag.py:
##########
@@ -28,6 +27,7 @@
 from sqlalchemy.orm import Session, backref, foreign, relationship
 from sqlalchemy.sql.expression import func, literal
 
+import airflow.compat.hashlib_wrapper as hashlib_wrapper

Review Comment:
   ```suggestion
   from airflow.compat import hashlib_wrapper
   ```



##########
airflow/compat/hashlib_wrapper.py:
##########
@@ -0,0 +1,43 @@
+#
+# 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
+import sys
+
+# Check if "usedforsecurity" is available for hashlib
+if sys.version_info >= (3, 9):
+    HAS_USEDFORSECURITY = True
+else:
+    HAS_USEDFORSECURITY = False
+
+
+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 used_for_security: The value to pass to the md5 function's "usedforsecurity" param.
+        Defaults to None.
+    :return: The hashed value.
+    :rtype: _Hash
+    """
+    if HAS_USEDFORSECURITY and usedforsecurity is not None:
+        return hashlib.md5(data, usedforsecurity=usedforsecurity)  # type: ignore

Review Comment:
   ```suggestion
       from airflow import PY39
   
       if PY39 and usedforsecurity is not None:
           return hashlib.md5(data, usedforsecurity=usedforsecurity)  # type: ignore
   ```
   
   It'd be better to use our existing constant instead I think, and move your "Check if usedforsecurity is available" comment here.



##########
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,
+}

Review Comment:
   `getattr` is the right answer here then 👍



##########
airflow/models/serialized_dag.py:
##########
@@ -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 = hashlib_wrapper.md5(dag_data_json, False).hexdigest()

Review Comment:
   ```suggestion
           self.dag_hash = hashlib_wrapper.md5(dag_data_json, usedforsecurity=False).hexdigest()
   ```
   
   Let's name the bool, easier to grok later when looking just at the serialization stuff.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] shyft-mike commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "shyft-mike (via GitHub)" <gi...@apache.org>.
shyft-mike commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1090945214


##########
airflow/compat/hashlib_wrapper.py:
##########
@@ -0,0 +1,45 @@
+#
+# 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
+import sys
+from typing import Any
+
+# Check if "usedforsecurity" is available for hashlib
+if sys.version_info >= (3, 9):
+    HAS_USEDFORSECURITY = True
+else:
+    HAS_USEDFORSECURITY = False
+
+
+def md5(data: Any, used_for_security: bool | None = None):

Review Comment:
   Ah, I can definitely update the typing.
   To your 1st point though, it feels wrong to give it the same name as an existing stdlib module. I'd rather be explicit with what we are trying to do. Also, used_for_security felt more pythonic and easier to read. But I am by no means a python expert, so I could be swayed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] shyft-mike commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by GitBox <gi...@apache.org>.
shyft-mike commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1071642334


##########
airflow/www/app.py:
##########
@@ -131,7 +132,29 @@ def create_app(config=None, testing=False):
 
     init_robots(flask_app)
 
+    # Configure caching
+    webserver_caching_hash_method = conf.get(section="webserver", key="CACHING_HASH_METHOD", fallback=None)
     cache_config = {"CACHE_TYPE": "flask_caching.backends.filesystem", "CACHE_DIR": gettempdir()}

Review Comment:
   Ah, great, thanks! Addressing the other comments now 👍 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] uranusjr commented on pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on PR #28846:
URL: https://github.com/apache/airflow/pull/28846#issuecomment-1478985714

   I don’t have further comments but also lack proper knowledge to actually approve, so it’s better to leave this to others.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1083233955


##########
airflow/compat/hashlib_wrapper.py:
##########
@@ -0,0 +1,41 @@
+#
+# 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.
+
+import hashlib

Review Comment:
   NICE!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] shyft-mike commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "shyft-mike (via GitHub)" <gi...@apache.org>.
shyft-mike commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1090024726


##########
airflow/config_templates/config.yml:
##########
@@ -271,8 +270,8 @@ core:
       default: "0"
     default_task_retry_delay:
       description: |
-          The number of seconds each task is going to wait by default between retries. Can be overridden at

Review Comment:
   Can do 👍 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1066655259


##########
airflow/www/app.py:
##########
@@ -131,7 +132,29 @@ def create_app(config=None, testing=False):
 
     init_robots(flask_app)
 
+    # Configure caching
+    webserver_caching_hash_method = conf.get(section="webserver", key="CACHING_HASH_METHOD", fallback=None)
     cache_config = {"CACHE_TYPE": "flask_caching.backends.filesystem", "CACHE_DIR": gettempdir()}

Review Comment:
   You need to add this new parameter in `airflow/config_templates/config.yml` with description and defaults and let pre-commit do the job of updating the documentation.



##########
airflow/www/app.py:
##########
@@ -131,7 +132,29 @@ def create_app(config=None, testing=False):
 
     init_robots(flask_app)
 
+    # Configure caching
+    webserver_caching_hash_method = conf.get(section="webserver", key="CACHING_HASH_METHOD", fallback=None)
     cache_config = {"CACHE_TYPE": "flask_caching.backends.filesystem", "CACHE_DIR": gettempdir()}
+
+    if (
+        webserver_caching_hash_method is not None
+        and webserver_caching_hash_method.casefold() != "md5".casefold()
+    ):
+        if webserver_caching_hash_method.casefold() == "sha512".casefold():
+            cache_config["CACHE_OPTIONS"] = {"hash_method": hashlib.sha512}
+        elif webserver_caching_hash_method.casefold() == "sha384".casefold():
+            cache_config["CACHE_OPTIONS"] = {"hash_method": hashlib.sha384}
+        elif webserver_caching_hash_method.casefold() == "sha256".casefold():
+            cache_config["CACHE_OPTIONS"] = {"hash_method": hashlib.sha256}
+        elif webserver_caching_hash_method.casefold() == "sha224".casefold():

Review Comment:
   Calling casefold (or even lower) should be done where `webserver_caching_hash_method` is set. There is no reason to casefold already lower-case constants. Additionally probably what you want to do here is to have a dict/mapping of the hash name to hashlib algorithm and use it rather than sequence of if/elif.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] ephraimbuddy commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1066619689


##########
airflow/www/app.py:
##########
@@ -131,7 +132,29 @@ def create_app(config=None, testing=False):
 
     init_robots(flask_app)
 
+    # Configure caching
+    webserver_caching_hash_method = conf.get(section="webserver", key="CACHING_HASH_METHOD", fallback=None)
     cache_config = {"CACHE_TYPE": "flask_caching.backends.filesystem", "CACHE_DIR": gettempdir()}
+
+    if (
+        webserver_caching_hash_method is not None
+        and webserver_caching_hash_method.casefold() != "md5".casefold()
+    ):
+        if webserver_caching_hash_method.casefold() == "sha512".casefold():
+            cache_config["CACHE_OPTIONS"] = {"hash_method": hashlib.sha512}
+        elif webserver_caching_hash_method.casefold() == "sha384".casefold():
+            cache_config["CACHE_OPTIONS"] = {"hash_method": hashlib.sha384}
+        elif webserver_caching_hash_method.casefold() == "sha256".casefold():
+            cache_config["CACHE_OPTIONS"] = {"hash_method": hashlib.sha256}
+        elif webserver_caching_hash_method.casefold() == "sha224".casefold():

Review Comment:
   Are the RHS `casefolds` needed?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] uranusjr commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1091588434


##########
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,
+}

Review Comment:
   `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:
   
   ```python
   hashlib.md5(...)  # Better.
   hashlib.new("md5")(...)
   ```
   
   ```python
   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.
   
   ```python
   HASH_METHOD_MAPPING = {
       "md5": hashlib.md5,
       ...
   }
   HASH_METHOD_MAPPING[key]
   
   hashlib.new(key)
   
   getattr(hashlib, key)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "Taragolis (via GitHub)" <gi...@apache.org>.
Taragolis commented on PR #28846:
URL: https://github.com/apache/airflow/pull/28846#issuecomment-1435761064

   This feature required for python 3.9+ and our regular pipeline run in 3.7, so just in case I ran full test


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #28846:
URL: https://github.com/apache/airflow/pull/28846#issuecomment-1407597127

   Looks good one NIT only to separate out the formatting changes applied to Yaml code to a separate PR.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] Taragolis commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "Taragolis (via GitHub)" <gi...@apache.org>.
Taragolis commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1090777644


##########
airflow/compat/hashlib_wrapper.py:
##########
@@ -0,0 +1,45 @@
+#
+# 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
+import sys
+from typing import Any
+
+# Check if "usedforsecurity" is available for hashlib
+if sys.version_info >= (3, 9):
+    HAS_USEDFORSECURITY = True
+else:
+    HAS_USEDFORSECURITY = False
+
+
+def md5(data: Any, used_for_security: bool | None = None):
+    """Safely allows calling the hashlib.md5 function with the "usedforsecurity" param.
+
+    Args:
+        data (Any): The data to hash.
+        used_for_security (bool, optional): The value to pass to the md5 function's "usedforsecurity" param.
+            Defaults to None.
+
+    Returns:
+        _Hash: The hashed value.

Review Comment:
   We use reStructuredText / reST docstrings format ([PEP-287](https://peps.python.org/pep-0287/)) which is used by [Sphinx](http://sphinx-doc.org/) to generate documentation.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] shyft-mike commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "shyft-mike (via GitHub)" <gi...@apache.org>.
shyft-mike commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1091102239


##########
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,
+}

Review Comment:
   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.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1090998262


##########
airflow/compat/hashlib_wrapper.py:
##########
@@ -0,0 +1,45 @@
+#
+# 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
+import sys
+from typing import Any
+
+# Check if "usedforsecurity" is available for hashlib
+if sys.version_info >= (3, 9):
+    HAS_USEDFORSECURITY = True
+else:
+    HAS_USEDFORSECURITY = False
+
+
+def md5(data: Any, used_for_security: bool | None = None):

Review Comment:
   I'd rename the `usedforsecurity` but wouldn't remove the `_wrapper` (or rather rename `_wrapper` to `_compat` if the wrapper name is not nice.  I think this is usually the approach we take and I think there is a value in making sure that we know it's not "the" hashlib library. Might avoid some surprises (you woll only see the difference if you scroll up to the imports) and we might not forget to remove it when we bump min-airflow version to 3.9 (which is still long time from now)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] shyft-mike commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "shyft-mike (via GitHub)" <gi...@apache.org>.
shyft-mike commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1091100275


##########
airflow/compat/hashlib_wrapper.py:
##########
@@ -0,0 +1,45 @@
+#
+# 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
+import sys
+from typing import Any
+
+# Check if "usedforsecurity" is available for hashlib
+if sys.version_info >= (3, 9):
+    HAS_USEDFORSECURITY = True
+else:
+    HAS_USEDFORSECURITY = False
+
+
+def md5(data: Any, used_for_security: bool | None = None):
+    """Safely allows calling the hashlib.md5 function with the "usedforsecurity" param.
+
+    Args:
+        data (Any): The data to hash.
+        used_for_security (bool, optional): The value to pass to the md5 function's "usedforsecurity" param.
+            Defaults to None.
+
+    Returns:
+        _Hash: The hashed value.

Review Comment:
   Thanks for that. Done 👍 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #28846:
URL: https://github.com/apache/airflow/pull/28846#issuecomment-1479544474

   Happy to approve and merge then as soon as conflicts get resolved @shyft-mike 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] jedcunningham commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "jedcunningham (via GitHub)" <gi...@apache.org>.
jedcunningham commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1091156961


##########
airflow/compat/hashlib_wrapper.py:
##########
@@ -0,0 +1,45 @@
+#
+# 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
+import sys
+from typing import Any
+
+# Check if "usedforsecurity" is available for hashlib
+if sys.version_info >= (3, 9):
+    HAS_USEDFORSECURITY = True
+else:
+    HAS_USEDFORSECURITY = False
+
+
+def md5(data: Any, used_for_security: bool | None = None):

Review Comment:
   I'd personally like `_compat` over `_wrapper`.
   
   Fwiw, this isn't our convention (at least looking at existing compat stuff). We have `functools`, not `functools_compat`.
   
   I'd rather see
   ```
   from airflow.compat.hashlib import md5
   md5(...)
   ```
   than
   ```
   from airflow.compat import hashlib_compat
   hashlib_compat.md5(...)
   ```
   
   My 2c.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] uranusjr commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1090135796


##########
airflow/compat/hashlib_wrapper.py:
##########
@@ -0,0 +1,45 @@
+#
+# 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
+import sys
+from typing import Any
+
+# Check if "usedforsecurity" is available for hashlib
+if sys.version_info >= (3, 9):
+    HAS_USEDFORSECURITY = True
+else:
+    HAS_USEDFORSECURITY = False
+
+
+def md5(data: Any, used_for_security: bool | None = None):

Review Comment:
   I feel we should match the original signature as much as possible, namely
   
   1. Match the `usedforsecurity` argument name
   2. Name this module `hashlib` (removing the `_wrapper` part)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1089897466


##########
airflow/config_templates/config.yml:
##########
@@ -271,8 +270,8 @@ core:
       default: "0"
     default_task_retry_delay:
       description: |
-          The number of seconds each task is going to wait by default between retries. Can be overridden at

Review Comment:
   Can you please remove the unrelated changes to a separate PR ? I understand this is done by your editor automatically and it's cool to have things fixed, but there are certain properties of such changes (for example being cherry-pickable) that calls for separating unrelated changes to separate PRs.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] shyft-mike commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "shyft-mike (via GitHub)" <gi...@apache.org>.
shyft-mike commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1096599339


##########
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,
+}

Review Comment:
   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.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] shyft-mike commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "shyft-mike (via GitHub)" <gi...@apache.org>.
shyft-mike commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1096602359


##########
airflow/compat/hashlib_wrapper.py:
##########
@@ -0,0 +1,43 @@
+#
+# 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
+import sys
+
+# Check if "usedforsecurity" is available for hashlib
+if sys.version_info >= (3, 9):
+    HAS_USEDFORSECURITY = True
+else:
+    HAS_USEDFORSECURITY = False
+
+
+def md5(data: bytes, usedforsecurity: bool | None = None):

Review Comment:
   Made this change



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] shyft-mike commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "shyft-mike (via GitHub)" <gi...@apache.org>.
shyft-mike commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1091099843


##########
airflow/config_templates/config.yml:
##########
@@ -1536,6 +1536,13 @@ webserver:
       type: boolean
       example: ~
       default: "False"
+    caching_hash_method:
+      description: |
+        The caching algorithm used by the webserver.

Review Comment:
   Done 👍 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] shyft-mike commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "shyft-mike (via GitHub)" <gi...@apache.org>.
shyft-mike commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1123510383


##########
airflow/compat/hashlib_wrapper.py:
##########
@@ -0,0 +1,45 @@
+#
+# 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
+import sys
+from typing import Any
+
+# Check if "usedforsecurity" is available for hashlib
+if sys.version_info >= (3, 9):
+    HAS_USEDFORSECURITY = True
+else:
+    HAS_USEDFORSECURITY = False
+
+
+def md5(data: Any, used_for_security: bool | None = None):

Review Comment:
   Sorry, I thought I responded. I moved things under `utils`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk closed pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk closed pull request #28846: Updated app to support configuring the caching hash method for FIPS
URL: https://github.com/apache/airflow/pull/28846


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] shyft-mike commented on a diff in pull request #28846: Updated app to support configuring the caching hash method for FIPS

Posted by GitBox <gi...@apache.org>.
shyft-mike commented on code in PR #28846:
URL: https://github.com/apache/airflow/pull/28846#discussion_r1082832047


##########
airflow/www/app.py:
##########
@@ -131,7 +132,29 @@ def create_app(config=None, testing=False):
 
     init_robots(flask_app)
 
+    # Configure caching
+    webserver_caching_hash_method = conf.get(section="webserver", key="CACHING_HASH_METHOD", fallback=None)
     cache_config = {"CACHE_TYPE": "flask_caching.backends.filesystem", "CACHE_DIR": gettempdir()}
+
+    if (
+        webserver_caching_hash_method is not None
+        and webserver_caching_hash_method.casefold() != "md5".casefold()
+    ):
+        if webserver_caching_hash_method.casefold() == "sha512".casefold():
+            cache_config["CACHE_OPTIONS"] = {"hash_method": hashlib.sha512}
+        elif webserver_caching_hash_method.casefold() == "sha384".casefold():
+            cache_config["CACHE_OPTIONS"] = {"hash_method": hashlib.sha384}
+        elif webserver_caching_hash_method.casefold() == "sha256".casefold():
+            cache_config["CACHE_OPTIONS"] = {"hash_method": hashlib.sha256}
+        elif webserver_caching_hash_method.casefold() == "sha224".casefold():

Review Comment:
   This should be cleaned up quite a bit now :+1:



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org