Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 6 additions & 0 deletions .pre-commit-config.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -937,6 +937,12 @@ repos:
pass_filenames: true
files: \.py$
exclude: ^airflow/providers|^dev/.*\.py$|^scripts/.*\.py$|^tests/|^docker_tests/|^docs/.*\.py$|^airflow/utils/helpers.py$
- id: check-deferrable-default-value
name: Check default value of deferrable attribute
language: python
entry: ./scripts/ci/pre_commit/pre_commit_check_deferrable_default.py
pass_filenames: false
files: ^airflow/.*/sensors/.*\.py$|^airflow/.*/operators/.*\.py$
## ADD MOST PRE-COMMITS ABOVE THAT LINE
# The below pre-commits are those requiring CI image to be built
- id: mypy-dev
Expand Down
2 changes: 2 additions & 0 deletions STATIC_CODE_CHECKS.rst
Original file line number Diff line number Diff line change
Expand Up @@ -171,6 +171,8 @@ require Breeze Docker image to be built locally.
+-----------------------------------------------------------+--------------------------------------------------------------+---------+
| check-decorated-operator-implements-custom-name | Check @task decorator implements custom_operator_name | |
+-----------------------------------------------------------+--------------------------------------------------------------+---------+
| check-deferrable-default-value | Check default value of deferrable attribute | |
+-----------------------------------------------------------+--------------------------------------------------------------+---------+
| check-docstring-param-types | Check that docstrings do not specify param types | |
+-----------------------------------------------------------+--------------------------------------------------------------+---------+
| check-example-dags-urls | Check that example dags url include provider versions | |
Expand Down
3 changes: 2 additions & 1 deletion airflow/providers/google/cloud/operators/datafusion.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
from googleapiclient.errors import HttpError

from airflow import AirflowException
from airflow.configuration import conf
from airflow.providers.google.cloud.hooks.datafusion import SUCCESS_STATES, DataFusionHook, PipelineStates
from airflow.providers.google.cloud.links.datafusion import (
DataFusionInstanceLink,
Expand Down Expand Up @@ -759,7 +760,7 @@ def __init__(
gcp_conn_id: str = "google_cloud_default",
impersonation_chain: str | Sequence[str] | None = None,
asynchronous=False,
deferrable=False,
deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False),
poll_interval=3.0,
**kwargs,
) -> None:
Expand Down
7 changes: 3 additions & 4 deletions airflow/providers/snowflake/operators/snowflake.py
Original file line number Diff line number Diff line change
Expand Up @@ -23,16 +23,15 @@
from typing import TYPE_CHECKING, Any, Iterable, List, Mapping, Sequence, SupportsAbs, cast

from airflow import AirflowException
from airflow.configuration import conf
from airflow.exceptions import AirflowProviderDeprecationWarning
from airflow.providers.common.sql.operators.sql import (
SQLCheckOperator,
SQLExecuteQueryOperator,
SQLIntervalCheckOperator,
SQLValueCheckOperator,
)
from airflow.providers.snowflake.hooks.snowflake_sql_api import (
SnowflakeSqlApiHook,
)
from airflow.providers.snowflake.hooks.snowflake_sql_api import SnowflakeSqlApiHook
from airflow.providers.snowflake.triggers.snowflake_trigger import SnowflakeSqlApiTrigger

if TYPE_CHECKING:
Expand Down Expand Up @@ -451,7 +450,7 @@ def __init__(
token_life_time: timedelta = LIFETIME,
token_renewal_delta: timedelta = RENEWAL_DELTA,
bindings: dict[str, Any] | None = None,
deferrable: bool = False,
deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False),
**kwargs: Any,
) -> None:
self.snowflake_conn_id = snowflake_conn_id
Expand Down
1 change: 1 addition & 0 deletions dev/breeze/src/airflow_breeze/pre_commit_ids.py
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
"check-core-deprecation-classes",
"check-daysago-import-from-utils",
"check-decorated-operator-implements-custom-name",
"check-deferrable-default-value",
"check-docstring-param-types",
"check-example-dags-urls",
"check-executables-have-shebangs",
Expand Down
2 changes: 1 addition & 1 deletion images/breeze/output-commands-hash.txt
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ setup:version:123b462a421884dc2320ffc5e54b2478
setup:201c30ea237ea013a9a209a77092a2e8
shell:13f90c5749811e2f00e24d95e44e946d
start-airflow:22c118d58b13a9d190e966bed5bb8ed8
static-checks:a0f6ae35129b99b88ce65aede2c3a150
static-checks:521edd8e7669b666b288ad0243b45be7
testing:docker-compose-tests:70167e67853cacd9ca784695d65a7846
testing:helm-tests:936cf28fd84ce4ff5113795fdae9624b
testing:integration-tests:35f0ac57157bf8fe227fd080cf216622
Expand Down
104 changes: 52 additions & 52 deletions images/breeze/output-commands.svg
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
32 changes: 16 additions & 16 deletions images/breeze/output_setup.svg
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
54 changes: 27 additions & 27 deletions images/breeze/output_setup_check-all-params-in-groups.svg
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
62 changes: 31 additions & 31 deletions images/breeze/output_setup_regenerate-command-images.svg
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
140 changes: 72 additions & 68 deletions images/breeze/output_static-checks.svg
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
108 changes: 108 additions & 0 deletions scripts/ci/pre_commit/pre_commit_check_deferrable_default.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,108 @@
#!/usr/bin/env python
#
# 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.
from __future__ import annotations

import ast
import glob
import itertools
import os
import sys
from typing import Iterator

ROOT_DIR = os.path.abspath(os.path.join(os.path.dirname(__file__), os.pardir, os.pardir, os.pardir))

DEFERRABLE_DOC = (
"https://github.com/apache/airflow/blob/main/docs/apache-airflow/"
"authoring-and-scheduling/deferring.rst#writing-deferrable-operators"
)


def _is_valid_deferrable_default(default: ast.AST) -> bool:
"""Check whether default is 'conf.getboolean("operators", "default_deferrable", fallback=False)'"""
if not isinstance(default, ast.Call):
return False # Not a function call.

# Check the function callee is exactly 'conf.getboolean'.
call_to_conf_getboolean = (
isinstance(default.func, ast.Attribute)
and isinstance(default.func.value, ast.Name)
and default.func.value.id == "conf"
and default.func.attr == "getboolean"
)
if not call_to_conf_getboolean:
return False

# Check arguments.
return (
len(default.args) == 2
and isinstance(default.args[0], ast.Constant)
and default.args[0].value == "operators"
and isinstance(default.args[1], ast.Constant)
and default.args[1].value == "default_deferrable"
and len(default.keywords) == 1
and default.keywords[0].arg == "fallback"
and isinstance(default.keywords[0].value, ast.Constant)
and default.keywords[0].value.value is False
)


def iter_check_deferrable_default_errors(module_filename: str) -> Iterator[str]:
ast_obj = ast.parse(open(module_filename).read())
cls_nodes = (node for node in ast.iter_child_nodes(ast_obj) if isinstance(node, ast.ClassDef))
init_method_nodes = (
node
for cls_node in cls_nodes
for node in ast.iter_child_nodes(cls_node)
if isinstance(node, ast.FunctionDef) and node.name == "__init__"
)

for node in init_method_nodes:
args = node.args
arguments = reversed([*args.args, *args.kwonlyargs])
defaults = reversed([*args.defaults, *args.kw_defaults])
for argument, default in itertools.zip_longest(arguments, defaults, fillvalue=None):
if argument is None or default is None:
continue
if argument.arg != "deferrable" or _is_valid_deferrable_default(default):
continue
yield f"{module_filename}:{default.lineno}"


def main() -> int:
modules = itertools.chain(
glob.glob(f"{ROOT_DIR}/airflow/**/sensors/**.py", recursive=True),
glob.glob(f"{ROOT_DIR}/airflow/**/operators/**.py", recursive=True),
)

errors = [error for module in modules for error in iter_check_deferrable_default_errors(module)]
if errors:
print("Incorrect deferrable default values detected at:")
for error in errors:
print(f" {error}")
print(
"""Please set the default value of deferrbale to """
""""conf.getboolean("operators", "default_deferrable", fallback=False)"\n"""
f"See: {DEFERRABLE_DOC}\n"
)

return len(errors)


if __name__ == "__main__":
sys.exit(main())