wjddn279 opened a new pull request, #59430: URL: https://github.com/apache/airflow/pull/59430
## Motivation We observed that when runtime-varying values are used as arguments in DAG or Task constructors in Airflow, the DAG version increases infinitely. [slack](https://apache-airflow.slack.com/archives/CCQ7EGB1P/p1763408409990439?thread_ts=1763078254.725439&cid=CCQ7EGB1P) https://github.com/apache/airflow/issues/55768#issuecomment-3360876034 Checking for DAG version increments at runtime is difficult. The most accurate detection method would be to parse the DAG object twice and compare if values differ. However, this would nearly double the DAG parsing execution time. Therefore, I add a feature that exposes DAG warnings for these cases through AST-based static analysis before parsing in the dag-processor. While it cannot cover 100% of DAG usage patterns, it can cover most cases and has minimal performance impact (since `ast.parse` [already runs](https://github.com/apache/airflow/blob/main/airflow-core/src/airflow/dag_processing/processor.py#L514) on every DAG parse). ## Logic of static check The logic for detecting problematic situations through static check is as follows (I named this issue "runtime-varying"): 1. Statically analyze a single DAG file through `ast.parse`. 2. Traverse each node and check the following: - Has a variable been assigned a runtime-varying value? → This is to check if that variable is passed as an argument to a DAG or Task instance. ```python from datetime import datetime import random as rd start_date = datetime.now() # checked as tainted value random_value = f"random_{rd.randint(1,1000)" # checked as tainted value default_args = {'start_date': start_date} # checked as tainted value ``` 3. Check if the object is a DAG or Task declaration statement, and verify if runtime-varying variables or function calls are passed as arguments. - Check if it's a DAG declaration statement → We categorized DAG object definitions into 3 cases: ```python from airflow import DAG from airflow.decorators import dag dag = DAG(dag_id='dag_id, default_args=default_args) # DAG object definition imported from airflow module with DAG(dag_id='dag_id, default_args=default_args) as dag: # Defined as context manager in with statement @dag(dag_id='dag_id, default_args=default_args) # Defined via dag decorator ``` - Check if it's a Task declaration statement → This case can be categorized into 2 types: ```python task1 = PythonOperator(task_id='task_id', dag=dag) # When the DAG object checked above is passed as an argument with DAG(dag_id='dag_id, default_args=default_args) as dag: task2 = PythonOperator(task_id='task_id') # Function calls inside the with block where DAG context manager is declared ``` The cases covered by static checks are described in detail in the unit test code. ## User Notification for Static Check Errors I considered that static check failures are not severe enough to cause DAG parsing to fail, so I added them to DAG warnings. Warnings are added to DAGs generated from the DAG file and displayed in the UI as shown below. There seems to be an issue where `\n` characters in messages are ignored when displayed in the UI, which we plan to fix in a future PR. <img width="1700" height="750" alt="image" src="https://github.com/user-attachments/assets/f151199a-fbd2-4f2e-b407-f7f03214724f" /> ## future work If this PR is merged, the following items are planned for future work: - Merge the existing `ast.parse` with the `ast.parse` executed in this subprocess. - Fix the UI that displays DAG warnings. - Make DAG warnings more visible by displaying them in the DAG list as well. - Document the cases where DAG version increases infinitely and the coverage scope of this static check. <!-- 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/ --> <!-- 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]
