Skip to content

Commit a92c47b

Browse files
authored
Removing deprecated code in hive provider (#38859)
* Removing deprecated code in hive provider * fixing tests and removing property * removing some more tests
1 parent 78f84b9 commit a92c47b

File tree

5 files changed

+10
-83
lines changed

5 files changed

+10
-83
lines changed

airflow/providers/apache/hive/CHANGELOG.rst

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -37,6 +37,14 @@ Breaking changes
3737
Changed the default value of ``use_beeline`` in hive cli connection to True.
3838
Beeline will be always enabled by default in this connection type.
3939

40+
Removed deprecated parameter ``authMechanism`` from HiveHook and dependent operators.
41+
Use the already existing ``auth_mechanism`` instead in your ``extra``.
42+
43+
Removed the method ``get_hook`` from hive operator. Use the ``hook`` property instead.
44+
45+
Removed the deprecated ``col_blacklist`` property from HiveStatsCollectionOperator.
46+
Please rename it to ``excluded_columns`` instead.
47+
4048
7.0.1
4149
.....
4250

airflow/providers/apache/hive/hooks/hive.py

Lines changed: 0 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -23,12 +23,9 @@
2323
import socket
2424
import subprocess
2525
import time
26-
import warnings
2726
from tempfile import NamedTemporaryFile, TemporaryDirectory
2827
from typing import TYPE_CHECKING, Any, Iterable, Mapping
2928

30-
from airflow.exceptions import AirflowProviderDeprecationWarning
31-
3229
if TYPE_CHECKING:
3330
import pandas as pd
3431

@@ -561,15 +558,6 @@ def get_metastore_client(self) -> Any:
561558
if not host:
562559
raise AirflowException("Failed to locate the valid server.")
563560

564-
if "authMechanism" in conn.extra_dejson:
565-
warnings.warn(
566-
"The 'authMechanism' option is deprecated. Please use 'auth_mechanism'.",
567-
AirflowProviderDeprecationWarning,
568-
stacklevel=2,
569-
)
570-
conn.extra_dejson["auth_mechanism"] = conn.extra_dejson["authMechanism"]
571-
del conn.extra_dejson["authMechanism"]
572-
573561
auth_mechanism = conn.extra_dejson.get("auth_mechanism", "NOSASL")
574562

575563
if conf.get("core", "security") == "kerberos":
@@ -872,15 +860,6 @@ def get_conn(self, schema: str | None = None) -> Any:
872860

873861
db = self.get_connection(self.hiveserver2_conn_id) # type: ignore
874862

875-
if "authMechanism" in db.extra_dejson:
876-
warnings.warn(
877-
"The 'authMechanism' option is deprecated. Please use 'auth_mechanism'.",
878-
AirflowProviderDeprecationWarning,
879-
stacklevel=2,
880-
)
881-
db.extra_dejson["auth_mechanism"] = db.extra_dejson["authMechanism"]
882-
del db.extra_dejson["authMechanism"]
883-
884863
auth_mechanism = db.extra_dejson.get("auth_mechanism", "NONE")
885864
if auth_mechanism == "NONE" and db.login is None:
886865
# we need to give a username

airflow/providers/apache/hive/operators/hive.py

Lines changed: 0 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -22,10 +22,7 @@
2222
from functools import cached_property
2323
from typing import TYPE_CHECKING, Any, Sequence
2424

25-
from deprecated.classic import deprecated
26-
2725
from airflow.configuration import conf
28-
from airflow.exceptions import AirflowProviderDeprecationWarning
2926
from airflow.models import BaseOperator
3027
from airflow.providers.apache.hive.hooks.hive import HiveCliHook
3128
from airflow.utils import operator_helpers
@@ -131,11 +128,6 @@ def hook(self) -> HiveCliHook:
131128
proxy_user=self.proxy_user,
132129
)
133130

134-
@deprecated(reason="use `hook` property instead.", category=AirflowProviderDeprecationWarning)
135-
def get_hook(self) -> HiveCliHook:
136-
"""Get Hive cli hook."""
137-
return self.hook
138-
139131
def prepare_template(self) -> None:
140132
if self.hiveconf_jinja_translate:
141133
self.hql = re.sub(r"(\$\{(hiveconf:)?([ a-zA-Z0-9_]*)\})", r"{{ \g<3> }}", self.hql)

airflow/providers/apache/hive/operators/hive_stats.py

Lines changed: 1 addition & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -18,15 +18,13 @@
1818
from __future__ import annotations
1919

2020
import json
21-
import warnings
2221
from typing import TYPE_CHECKING, Any, Callable, Sequence
2322

24-
from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning
23+
from airflow.exceptions import AirflowException
2524
from airflow.models import BaseOperator
2625
from airflow.providers.apache.hive.hooks.hive import HiveMetastoreHook
2726
from airflow.providers.mysql.hooks.mysql import MySqlHook
2827
from airflow.providers.presto.hooks.presto import PrestoHook
29-
from airflow.utils.types import NOTSET, ArgNotSet
3028

3129
if TYPE_CHECKING:
3230
from airflow.utils.context import Context
@@ -79,18 +77,8 @@ def __init__(
7977
mysql_conn_id: str = "airflow_db",
8078
ds: str = "{{ ds }}",
8179
dttm: str = "{{ logical_date.isoformat() }}",
82-
col_blacklist: list[str] | None | ArgNotSet = NOTSET,
8380
**kwargs: Any,
8481
) -> None:
85-
if col_blacklist is not NOTSET:
86-
warnings.warn(
87-
f"col_blacklist kwarg passed to {self.__class__.__name__} "
88-
f"(task_id: {kwargs.get('task_id')}) is deprecated, "
89-
f"please rename it to excluded_columns instead",
90-
category=AirflowProviderDeprecationWarning,
91-
stacklevel=2,
92-
)
93-
excluded_columns = col_blacklist # type: ignore[assignment]
9482
super().__init__(**kwargs)
9583
self.table = table
9684
self.partition = partition

tests/providers/apache/hive/operators/test_hive_stats.py

Lines changed: 1 addition & 41 deletions
Original file line numberDiff line numberDiff line change
@@ -23,11 +23,9 @@
2323

2424
import pytest
2525

26-
from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning
26+
from airflow.exceptions import AirflowException
2727
from airflow.providers.apache.hive.operators.hive_stats import HiveStatsCollectionOperator
2828
from airflow.providers.presto.hooks.presto import PrestoHook
29-
from airflow.utils import timezone
30-
from airflow.utils.task_instance_session import set_current_task_instance_session
3129
from tests.providers.apache.hive import (
3230
DEFAULT_DATE,
3331
DEFAULT_DATE_DS,
@@ -372,41 +370,3 @@ def test_runs_for_hive_stats(self, mock_hive_metastore_hook):
372370
"value",
373371
],
374372
)
375-
376-
def test_col_blacklist_deprecation(self):
377-
warn_message = "col_blacklist kwarg passed to.*task_id: fake-task-id.*is deprecated"
378-
with pytest.warns(AirflowProviderDeprecationWarning, match=warn_message):
379-
HiveStatsCollectionOperator(
380-
task_id="fake-task-id",
381-
table="airflow.static_babynames_partitioned",
382-
partition={"ds": DEFAULT_DATE_DS},
383-
col_blacklist=["foo", "bar"],
384-
)
385-
386-
@pytest.mark.db_test
387-
@pytest.mark.parametrize(
388-
"col_blacklist",
389-
[pytest.param(None, id="none"), pytest.param(["foo", "bar"], id="list")],
390-
)
391-
def test_partial_col_blacklist_deprecation(self, col_blacklist, dag_maker, session):
392-
with dag_maker(
393-
dag_id="test_partial_col_blacklist_deprecation",
394-
start_date=timezone.datetime(2024, 1, 1),
395-
session=session,
396-
):
397-
HiveStatsCollectionOperator.partial(
398-
task_id="fake-task-id",
399-
partition={"ds": DEFAULT_DATE_DS},
400-
col_blacklist=col_blacklist,
401-
excluded_columns=["spam", "egg"],
402-
).expand(table=["airflow.table1", "airflow.table2"])
403-
404-
dr = dag_maker.create_dagrun(execution_date=None)
405-
tis = dr.get_task_instances(session=session)
406-
with set_current_task_instance_session(session=session):
407-
warn_message = "col_blacklist kwarg passed to.*task_id: fake-task-id.*is deprecated"
408-
for ti in tis:
409-
with pytest.warns(AirflowProviderDeprecationWarning, match=warn_message):
410-
ti.render_templates()
411-
expected = col_blacklist or []
412-
assert ti.task.excluded_columns == expected

0 commit comments

Comments
 (0)