Skip to content

Commit

Permalink
Force to use Airflow Deprecation warnings categories on @deprecated
Browse files Browse the repository at this point in the history
… decorator (#39205)

* Force to use Airflow Deprecation warnings categories on @deprecated decorator

* Catch warning in Experimental API test
  • Loading branch information
Taragolis committed Apr 25, 2024
1 parent eca077b commit c2ef1da
Show file tree
Hide file tree
Showing 15 changed files with 314 additions and 91 deletions.
7 changes: 7 additions & 0 deletions .pre-commit-config.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -1001,6 +1001,13 @@ repos:
pass_filenames: true
files: ^airflow/.*\.py$
exclude: ^.*/.*_vendor/
- id: check-code-deprecations
name: Check deprecations categories in decorators
entry: ./scripts/ci/pre_commit/check_deprecations.py
language: python
pass_filenames: true
files: ^airflow/.*\.py$
exclude: ^.*/.*_vendor/
- id: lint-chart-schema
name: Lint chart/values.schema.json file
entry: ./scripts/ci/pre_commit/chart_schema.py
Expand Down
6 changes: 4 additions & 2 deletions airflow/api/common/experimental/get_code.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,13 @@
from deprecated import deprecated

from airflow.api.common.experimental import check_and_get_dag
from airflow.exceptions import AirflowException, DagCodeNotFound
from airflow.exceptions import AirflowException, DagCodeNotFound, RemovedInAirflow3Warning
from airflow.models.dagcode import DagCode


@deprecated(reason="Use DagCode().get_code_by_fileloc() instead", version="2.2.4")
@deprecated(
reason="Use DagCode().get_code_by_fileloc() instead", version="2.2.4", category=RemovedInAirflow3Warning
)
def get_code(dag_id: str) -> str:
"""Return python code of a given dag_id.
Expand Down
3 changes: 2 additions & 1 deletion airflow/api/common/experimental/get_dag_run_state.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,12 +24,13 @@
from deprecated import deprecated

from airflow.api.common.experimental import check_and_get_dag, check_and_get_dagrun
from airflow.exceptions import RemovedInAirflow3Warning

if TYPE_CHECKING:
from datetime import datetime


@deprecated(reason="Use DagRun().get_state() instead", version="2.2.4")
@deprecated(reason="Use DagRun().get_state() instead", version="2.2.4", category=RemovedInAirflow3Warning)
def get_dag_run_state(dag_id: str, execution_date: datetime) -> dict[str, str]:
"""Return the Dag Run state identified by the given dag_id and execution_date.
Expand Down
3 changes: 2 additions & 1 deletion airflow/api/common/experimental/get_task.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,12 +24,13 @@
from deprecated import deprecated

from airflow.api.common.experimental import check_and_get_dag
from airflow.exceptions import RemovedInAirflow3Warning

if TYPE_CHECKING:
from airflow.models import TaskInstance


@deprecated(reason="Use DAG().get_task", version="2.2.4")
@deprecated(reason="Use DAG().get_task", version="2.2.4", category=RemovedInAirflow3Warning)
def get_task(dag_id: str, task_id: str) -> TaskInstance:
"""Return the task object identified by the given dag_id and task_id."""
dag = check_and_get_dag(dag_id, task_id)
Expand Down
4 changes: 2 additions & 2 deletions airflow/api/common/experimental/get_task_instance.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,14 +24,14 @@
from deprecated import deprecated

from airflow.api.common.experimental import check_and_get_dag, check_and_get_dagrun
from airflow.exceptions import TaskInstanceNotFound
from airflow.exceptions import RemovedInAirflow3Warning, TaskInstanceNotFound
from airflow.models import TaskInstance

if TYPE_CHECKING:
from datetime import datetime


@deprecated(version="2.2.4", reason="Use DagRun.get_task_instance instead")
@deprecated(version="2.2.4", reason="Use DagRun.get_task_instance instead", category=RemovedInAirflow3Warning)
def get_task_instance(dag_id: str, task_id: str, execution_date: datetime) -> TaskInstance:
"""Return the task instance identified by the given dag_id, task_id and execution_date."""
dag = check_and_get_dag(dag_id, task_id)
Expand Down
10 changes: 5 additions & 5 deletions airflow/api/common/experimental/pool.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,15 +24,15 @@
from deprecated import deprecated
from sqlalchemy import select

from airflow.exceptions import AirflowBadRequest, PoolNotFound
from airflow.exceptions import AirflowBadRequest, PoolNotFound, RemovedInAirflow3Warning
from airflow.models import Pool
from airflow.utils.session import NEW_SESSION, provide_session

if TYPE_CHECKING:
from sqlalchemy.orm import Session


@deprecated(reason="Use Pool.get_pool() instead", version="2.2.4")
@deprecated(reason="Use Pool.get_pool() instead", version="2.2.4", category=RemovedInAirflow3Warning)
@provide_session
def get_pool(name, session: Session = NEW_SESSION):
"""Get pool by a given name."""
Expand All @@ -46,14 +46,14 @@ def get_pool(name, session: Session = NEW_SESSION):
return pool


@deprecated(reason="Use Pool.get_pools() instead", version="2.2.4")
@deprecated(reason="Use Pool.get_pools() instead", version="2.2.4", category=RemovedInAirflow3Warning)
@provide_session
def get_pools(session: Session = NEW_SESSION):
"""Get all pools."""
return session.scalars(select(Pool)).all()


@deprecated(reason="Use Pool.create_pool() instead", version="2.2.4")
@deprecated(reason="Use Pool.create_pool() instead", version="2.2.4", category=RemovedInAirflow3Warning)
@provide_session
def create_pool(name, slots, description, session: Session = NEW_SESSION):
"""Create a pool with given parameters."""
Expand Down Expand Up @@ -84,7 +84,7 @@ def create_pool(name, slots, description, session: Session = NEW_SESSION):
return pool


@deprecated(reason="Use Pool.delete_pool() instead", version="2.2.4")
@deprecated(reason="Use Pool.delete_pool() instead", version="2.2.4", category=RemovedInAirflow3Warning)
@provide_session
def delete_pool(name, session: Session = NEW_SESSION):
"""Delete pool by a given name."""
Expand Down
5 changes: 4 additions & 1 deletion airflow/auth/managers/fab/fab_auth_manager.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,14 @@

from deprecated import deprecated

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.fab.auth_manager.fab_auth_manager import FabAuthManager as FabAuthManagerProvider


@deprecated(
reason="Use airflow.providers.fab.auth_manager.fab_auth_manager.FabAuthManager instead", version="2.9.0"
reason="Use airflow.providers.fab.auth_manager.fab_auth_manager.FabAuthManager instead",
version="2.9.0",
category=RemovedInAirflow3Warning,
)
class FabAuthManager(FabAuthManagerProvider):
"""
Expand Down
4 changes: 3 additions & 1 deletion airflow/auth/managers/fab/security_manager/override.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

from deprecated import deprecated

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.fab.auth_manager.security_manager.override import (
FabAirflowSecurityManagerOverride as FabProviderAirflowSecurityManagerOverride,
)
Expand All @@ -27,7 +28,8 @@
@deprecated(
reason="If you want to override the security manager, you should inherit from "
"`airflow.providers.fab.auth_manager.security_manager.override.FabAirflowSecurityManagerOverride` "
"instead"
"instead",
category=RemovedInAirflow3Warning,
)
class FabAirflowSecurityManagerOverride(FabProviderAirflowSecurityManagerOverride):
"""
Expand Down
4 changes: 3 additions & 1 deletion airflow/www/security.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

from deprecated import deprecated

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.fab.auth_manager.security_manager.override import FabAirflowSecurityManagerOverride

EXISTING_ROLES = {
Expand All @@ -32,7 +33,8 @@
@deprecated(
reason="If you want to override the security manager, you should inherit from "
"`airflow.providers.fab.auth_manager.security_manager.override.FabAirflowSecurityManagerOverride` "
"instead"
"instead",
category=RemovedInAirflow3Warning,
)
class AirflowSecurityManager(FabAirflowSecurityManagerOverride):
"""Placeholder, just here to avoid breaking the code of users who inherit from this.
Expand Down
2 changes: 2 additions & 0 deletions contributing-docs/08_static_code_checks.rst
Original file line number Diff line number Diff line change
Expand Up @@ -146,6 +146,8 @@ require Breeze Docker image to be built locally.
+-----------------------------------------------------------+--------------------------------------------------------------+---------+
| check-cncf-k8s-only-for-executors | Check cncf.kubernetes imports used for executors only | |
+-----------------------------------------------------------+--------------------------------------------------------------+---------+
| check-code-deprecations | Check deprecations categories in decorators | |
+-----------------------------------------------------------+--------------------------------------------------------------+---------+
| check-compat-cache-on-methods | Check that compat cache do not use on class methods | |
+-----------------------------------------------------------+--------------------------------------------------------------+---------+
| check-core-deprecation-classes | Verify usage of Airflow deprecation classes in core | |
Expand Down
Loading

0 comments on commit c2ef1da

Please sign in to comment.