Skip to content

Commit

Permalink
Add Airflow specific warning classes
Browse files Browse the repository at this point in the history
Adding `RemoveInAirflow3DeprecationWarning` and `AirflowProviderDeprecationWarning`

Closes: #22356
  • Loading branch information
eladkal committed Aug 23, 2022
1 parent 5001656 commit 9c3cc36
Show file tree
Hide file tree
Showing 136 changed files with 539 additions and 365 deletions.
7 changes: 7 additions & 0 deletions .pre-commit-config.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -512,6 +512,13 @@ repos:
entry: ./scripts/ci/pre_commit/pre_commit_decorator_operator_implements_custom_name.py
pass_filenames: true
files: ^airflow/.*\.py$
- id: check-core-deprecation-classes
language: pygrep
name: Verify using of dedicated Airflow deprecation classes in core
entry: category=DeprecationWarning|category=PendingDeprecationWarning
files: \.py$
exclude: ^airflow/configuration.py|^airflow/providers|^scripts/in_container/verify_providers.py
pass_filenames: true
- id: check-provide-create-sessions-imports
language: pygrep
name: Check provide_session and create_session imports
Expand Down
2 changes: 2 additions & 0 deletions STATIC_CODE_CHECKS.rst
Original file line number Diff line number Diff line change
Expand Up @@ -155,6 +155,8 @@ require Breeze Docker image to be build locally.
+--------------------------------------------------------+------------------------------------------------------------------+---------+
| check-changelog-has-no-duplicates | Check changelogs for duplicate entries | |
+--------------------------------------------------------+------------------------------------------------------------------+---------+
| check-core-deprecation-classes | Verify using of dedicated Airflow deprecation classes in core | |
+--------------------------------------------------------+------------------------------------------------------------------+---------+
| check-daysago-import-from-utils | Make sure days_ago is imported from airflow.utils.dates | |
+--------------------------------------------------------+------------------------------------------------------------------+---------+
| check-decorated-operator-implements-custom-name | Check @task decorator implements custom_operator_name | |
Expand Down
4 changes: 2 additions & 2 deletions airflow/cli/commands/dag_command.py
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@
from airflow.api.client import get_current_api_client
from airflow.cli.simple_table import AirflowConsole
from airflow.configuration import conf
from airflow.exceptions import AirflowException, BackfillUnfinished
from airflow.exceptions import AirflowException, BackfillUnfinished, RemovedInAirflow3Warning
from airflow.executors.debug_executor import DebugExecutor
from airflow.jobs.base_job import BaseJob
from airflow.models import DagBag, DagModel, DagRun, TaskInstance
Expand All @@ -59,7 +59,7 @@ def dag_backfill(args, dag=None):

warnings.warn(
'--ignore-first-depends-on-past is deprecated as the value is always set to True',
category=PendingDeprecationWarning,
category=RemovedInAirflow3Warning,
)

if args.ignore_first_depends_on_past is False:
Expand Down
14 changes: 14 additions & 0 deletions airflow/exceptions.py
Original file line number Diff line number Diff line change
Expand Up @@ -338,3 +338,17 @@ class TaskDeferralError(AirflowException):

class PodReconciliationError(AirflowException):
"""Raised when an error is encountered while trying to merge pod configs."""


class RemovedInAirflow3Warning(DeprecationWarning):
"""Issued for usage of deprecated features that will be removed in Airflow3."""

deprecated_since: Optional[str] = None
"Indicates the airflow version that started raising this deprecation warning"


class AirflowProviderDeprecationWarning(DeprecationWarning):
"""Issued for usage of deprecated features of Airflow provider."""

deprecated_provider_since: Optional[str] = None
"Indicates the provider version that started raising this deprecation warning"
3 changes: 2 additions & 1 deletion airflow/executors/base_executor.py
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
from airflow.callbacks.base_callback_sink import BaseCallbackSink
from airflow.callbacks.callback_requests import CallbackRequest
from airflow.configuration import conf
from airflow.exceptions import RemovedInAirflow3Warning
from airflow.models.taskinstance import TaskInstance, TaskInstanceKey
from airflow.stats import Stats
from airflow.utils.log.logging_mixin import LoggingMixin
Expand Down Expand Up @@ -344,7 +345,7 @@ def validate_command(command: List[str]) -> None:
"""
The `validate_command` method is deprecated. Please use ``validate_airflow_tasks_run_command``
""",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
BaseExecutor.validate_airflow_tasks_run_command(command)
Expand Down
3 changes: 2 additions & 1 deletion airflow/hooks/S3_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,11 @@

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.amazon.aws.hooks.s3 import S3Hook, provide_bucket_name # noqa

warnings.warn(
"This module is deprecated. Please use `airflow.providers.amazon.aws.hooks.s3`.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/hooks/base.py
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import warnings
from typing import TYPE_CHECKING, Any, Dict, List

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.typing_compat import Protocol
from airflow.utils.log.logging_mixin import LoggingMixin

Expand Down Expand Up @@ -49,7 +50,7 @@ def get_connections(cls, conn_id: str) -> List["Connection"]:
warnings.warn(
"`BaseHook.get_connections` method will be deprecated in the future."
"Please use `BaseHook.get_connection` instead.",
PendingDeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
return [cls.get_connection(conn_id)]
Expand Down
7 changes: 6 additions & 1 deletion airflow/hooks/base_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,11 @@

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.hooks.base import BaseHook # noqa

warnings.warn("This module is deprecated. Please use `airflow.hooks.base`.", DeprecationWarning, stacklevel=2)
warnings.warn(
"This module is deprecated. Please use `airflow.hooks.base`.",
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/hooks/dbapi.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,11 +17,12 @@
# under the License.
import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.common.sql.hooks.sql import ConnectorProtocol # noqa
from airflow.providers.common.sql.hooks.sql import DbApiHook # noqa

warnings.warn(
"This module is deprecated. Please use `airflow.providers.common.sql.hooks.sql`.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/hooks/dbapi_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,11 @@

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.common.sql.hooks.sql import DbApiHook # noqa

warnings.warn(
"This module is deprecated. Please use `airflow.providers.common.sql.hooks.sql`.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/hooks/docker_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,11 @@

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.docker.hooks.docker import DockerHook # noqa

warnings.warn(
"This module is deprecated. Please use `airflow.providers.docker.hooks.docker`.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/hooks/druid_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,11 @@

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.apache.druid.hooks.druid import DruidDbApiHook, DruidHook # noqa

warnings.warn(
"This module is deprecated. Please use `airflow.providers.apache.druid.hooks.druid`.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/hooks/hdfs_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,11 @@

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.apache.hdfs.hooks.hdfs import HDFSHook, HDFSHookException # noqa

warnings.warn(
"This module is deprecated. Please use `airflow.providers.apache.hdfs.hooks.hdfs`.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/hooks/hive_hooks.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.apache.hive.hooks.hive import ( # noqa
HIVE_QUEUE_PRIORITIES,
HiveCliHook,
Expand All @@ -28,6 +29,6 @@

warnings.warn(
"This module is deprecated. Please use `airflow.providers.apache.hive.hooks.hive`.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/hooks/http_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,11 @@

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.http.hooks.http import HttpHook # noqa

warnings.warn(
"This module is deprecated. Please use `airflow.providers.http.hooks.http`.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/hooks/jdbc_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,11 @@

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.jdbc.hooks.jdbc import JdbcHook, jaydebeapi # noqa

warnings.warn(
"This module is deprecated. Please use `airflow.providers.jdbc.hooks.jdbc`.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/hooks/mssql_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,11 @@

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.microsoft.mssql.hooks.mssql import MsSqlHook # noqa

warnings.warn(
"This module is deprecated. Please use `airflow.providers.microsoft.mssql.hooks.mssql`.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/hooks/mysql_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,11 @@

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.mysql.hooks.mysql import MySqlHook # noqa

warnings.warn(
"This module is deprecated. Please use `airflow.providers.mysql.hooks.mysql`.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/hooks/oracle_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,11 @@

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.oracle.hooks.oracle import OracleHook # noqa

warnings.warn(
"This module is deprecated. Please use `airflow.providers.oracle.hooks.oracle`.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/hooks/pig_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,11 @@

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.apache.pig.hooks.pig import PigCliHook # noqa

warnings.warn(
"This module is deprecated. Please use `airflow.providers.apache.pig.hooks.pig`.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/hooks/postgres_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,11 @@

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.postgres.hooks.postgres import PostgresHook # noqa

warnings.warn(
"This module is deprecated. Please use `airflow.providers.postgres.hooks.postgres`.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/hooks/presto_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,11 @@

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.presto.hooks.presto import PrestoHook # noqa

warnings.warn(
"This module is deprecated. Please use `airflow.providers.presto.hooks.presto`.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/hooks/samba_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,11 @@

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.samba.hooks.samba import SambaHook # noqa

warnings.warn(
"This module is deprecated. Please use `airflow.providers.samba.hooks.samba`.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/hooks/slack_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,11 @@

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.slack.hooks.slack import SlackHook # noqa

warnings.warn(
"This module is deprecated. Please use `airflow.providers.slack.hooks.slack`.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/hooks/sqlite_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,11 @@

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.sqlite.hooks.sqlite import SqliteHook # noqa

warnings.warn(
"This module is deprecated. Please use `airflow.providers.sqlite.hooks.sqlite`.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/hooks/webhdfs_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,11 @@

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.apache.hdfs.hooks.webhdfs import WebHDFSHook # noqa

warnings.warn(
"This module is deprecated. Please use `airflow.providers.apache.hdfs.hooks.webhdfs`.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/hooks/zendesk_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,11 @@

import warnings

from airflow.exceptions import RemovedInAirflow3Warning
from airflow.providers.zendesk.hooks.zendesk import ZendeskHook # noqa

warnings.warn(
"This module is deprecated. Please use `airflow.providers.zendesk.hooks.zendesk`.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
3 changes: 2 additions & 1 deletion airflow/jobs/scheduler_job.py
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
from airflow.callbacks.pipe_callback_sink import PipeCallbackSink
from airflow.configuration import conf
from airflow.dag_processing.manager import DagFileProcessorAgent
from airflow.exceptions import RemovedInAirflow3Warning
from airflow.executors.executor_loader import UNPICKLEABLE_EXECUTORS
from airflow.jobs.base_job import BaseJob
from airflow.jobs.local_task_job import LocalTaskJob
Expand Down Expand Up @@ -132,7 +133,7 @@ def __init__(
warnings.warn(
"The 'processor_poll_interval' parameter is deprecated. "
"Please use 'scheduler_idle_sleep_time'.",
DeprecationWarning,
RemovedInAirflow3Warning,
stacklevel=2,
)
scheduler_idle_sleep_time = processor_poll_interval
Expand Down
Loading

0 comments on commit 9c3cc36

Please sign in to comment.