amoghrajesh opened a new pull request, #58767:
URL: https://github.com/apache/airflow/pull/58767
<!--
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.
-->
<!--
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: #ISSUE
How to write a good git commit message:
http://chris.beams.io/posts/git-commit/
-->
### Problem
Secrets registered via `mask_secret()` on the worker / dag processor (the
ones those use task sdk secrets masker) are correctly masked in task logs but
exposed in as clear text in the Rendered Templates UI.
The root cause for this is that due to the recent move to move secrets
masker to a shared library and using vendoring per distribution:
https://github.com/apache/airflow/pull/54449, the task sdk uses a task sdk
secrets masker is recognised differently from the core secrets masker by python
even though they are logically the same. Python sees different import paths
(`airflow.sdk.` vs `airflow.`and assumes that the maskers are different)
The flow leading for logs vs rendered templates:
1. Task process calls `mask_secret()` -> sends `MaskSecret` message to
supervisor
2. Supervisor registers the pattern in its `SecretsMasker` instance
3. **Logs:** Flow through supervisor's `SecretsMasker.filter()` an are masked
4. **Rendered templates:** Sent via `SetRenderedFields` message and
supervisor forwards to API server which is stored in the database.
5. When UI tries to read it to display on the UI from the database, the
masker that is used to redact it is the one in airflow core, hence the patterns
registered with sdk masker are not recognised and masking fails.
### Solution
Since the supervisor already has all secret patterns (from `MaskSecret`
messages) but wasn't applying them to rendered template fields before sending
to the API server, the solution is to redact at the worker end itself and send
it across to the API server. This is the only way to do it and should not cause
any ripple effects too because rendered templates are ONLY used to display to
the user, and nothing else.
### Test
#### Test 1: Running a Dag with custom operator with templated fields
Dag:
```python
from __future__ import annotations
from datetime import datetime
from airflow.decorators import dag
from airflow.models import BaseOperator
from airflow.sdk.log import mask_secret
mask_secret("password")
class CustomOperator(BaseOperator):
template_fields = ("tup", "se")
def __init__(self, tup: tuple, se: set, *args, **kwargs):
super().__init__(*args, **kwargs)
self.tup = tup
self.se = se
def execute(self, context):
print(self.tup, self.se)
@dag(
# every minute on the 30-second mark
catchup=False,
tags=[],
schedule=None,
max_active_runs=1,
start_date=datetime(2021, 1, 1),
)
def hello_dag():
CustomOperator(
task_id="custom",
tup=(1, 2, "{{ds}}"),
se={1, 2, 3, "password"},
)
hello_dag()
```
**Earlier:**
Logs:
<img width="2469" height="944" alt="image"
src="https://github.com/user-attachments/assets/f853efb1-5085-4109-b7f1-393ce8faa0c5"
/>
Rendered fields:
<img width="2469" height="944" alt="image"
src="https://github.com/user-attachments/assets/de0de926-802b-4558-a8af-ebd79237871a"
/>
**After the fix:**
Logs:
<img width="2469" height="944" alt="image"
src="https://github.com/user-attachments/assets/7c50cda7-1173-4493-8fe7-269bee323fff"
/>
Rendered Fields:
<img width="2469" height="944" alt="image"
src="https://github.com/user-attachments/assets/c892e2c4-ee37-4a79-95d1-648dd60f3e2b"
/>
#### Test 2: Using rendered templates with macros
Plugin:
```python
from airflow.plugins_manager import AirflowPlugin
from airflow.models import Variable
from airflow.sdk.log import mask_secret
def get_masked_var(name: str):
value = Variable.get(name)
mask_secret(value)
return value
class TestPlugin(AirflowPlugin):
name = "test_plugin"
macros = [get_masked_var]
```
Set airflow variable using:
```shell script
airflow variables set my_var "secret123"
```
DAG:
```python
from airflow.sdk import DAG
from airflow.providers.standard.operators.bash import BashOperator
from datetime import datetime
with DAG('macros_and_rtif', start_date=datetime(2024, 1, 1), schedule=None)
as dag:
BashOperator(
task_id='test',
bash_command='echo {{ macros.test_plugin.get_masked_var("my_var") }}'
)
```
**Before:**
Logs:
<img width="2469" height="944" alt="image"
src="https://github.com/user-attachments/assets/f475a2a5-42a4-43bd-b8ee-3a08280b75eb"
/>
Rendered Templates:
<img width="2469" height="944" alt="image"
src="https://github.com/user-attachments/assets/8d5fce4c-6ca5-45fb-b304-d54dbd709c8c"
/>
**After:**
Logs:
<img width="2469" height="944" alt="image"
src="https://github.com/user-attachments/assets/80e8e9d1-0333-4a77-8ff1-1e1cc7fec45a"
/>
Rendered Templates:
<img width="2469" height="944" alt="image"
src="https://github.com/user-attachments/assets/7fedcbb5-53a1-4e12-b557-23bd062c262b"
/>
<!-- Please keep an empty line above the dashes. -->
---
**^ Add meaningful description above**
Read the **[Pull Request
Guidelines](https://github.com/apache/airflow/blob/main/contributing-docs/05_pull_requests.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
[airflow-core/newsfragments](https://github.com/apache/airflow/tree/main/airflow-core/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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]