Skip to content

Commit

Permalink
Rename SparkSqlOperator's field's name to comply with templated fie…
Browse files Browse the repository at this point in the history
…lds validation (#38045)
  • Loading branch information
shahar1 authored Mar 12, 2024
1 parent 37fd2a5 commit 58bffa6
Show file tree
Hide file tree
Showing 2 changed files with 15 additions and 4 deletions.
1 change: 0 additions & 1 deletion .pre-commit-config.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -340,7 +340,6 @@ repos:
^airflow\/providers\/apache\/spark\/operators\/spark_submit.py\.py$|
^airflow\/providers\/google\/cloud\/operators\/vertex_ai\/auto_ml\.py$|
^airflow\/providers\/apache\/spark\/operators\/spark_submit\.py$|
^airflow\/providers\/apache\/spark\/operators\/spark_sql\.py$|
^airflow\/providers\/databricks\/operators\/databricks_sql\.py$|
)$
- id: ruff
Expand Down
18 changes: 15 additions & 3 deletions airflow/providers/apache/spark/operators/spark_sql.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,9 @@

from typing import TYPE_CHECKING, Any, Sequence

from deprecated import deprecated

from airflow.exceptions import AirflowProviderDeprecationWarning
from airflow.models import BaseOperator
from airflow.providers.apache.spark.hooks.spark_sql import SparkSqlHook

Expand Down Expand Up @@ -52,9 +55,9 @@ class SparkSqlOperator(BaseOperator):
(Default: The ``queue`` value set in the Connection, or ``"default"``)
"""

template_fields: Sequence[str] = ("_sql",)
template_fields: Sequence[str] = ("sql",)
template_ext: Sequence[str] = (".sql", ".hql")
template_fields_renderers = {"_sql": "sql"}
template_fields_renderers = {"sql": "sql"}

def __init__(
self,
Expand All @@ -75,7 +78,7 @@ def __init__(
**kwargs: Any,
) -> None:
super().__init__(**kwargs)
self._sql = sql
self.sql = sql
self._conf = conf
self._conn_id = conn_id
self._total_executor_cores = total_executor_cores
Expand All @@ -90,6 +93,15 @@ def __init__(
self._yarn_queue = yarn_queue
self._hook: SparkSqlHook | None = None

@property
@deprecated(
reason="`_sql` is deprecated and will be removed in the future. Please use `sql` instead.",
category=AirflowProviderDeprecationWarning,
)
def _sql(self):
"""Alias for ``sql``, used for compatibility (deprecated)."""
return self.sql

def execute(self, context: Context) -> None:
"""Call the SparkSqlHook to run the provided sql query."""
if self._hook is None:
Expand Down

0 comments on commit 58bffa6

Please sign in to comment.